From 49ffa9667b0efa671cf0ec56dcda3ed1abefbe19 Mon Sep 17 00:00:00 2001 From: Heng Ge Date: Sun, 11 Jan 2026 23:56:59 -0800 Subject: [PATCH 1/9] feat: use independent region manifest for MemWAL --- docs/src/format/table/.pages | 1 + docs/src/format/table/index/system/.pages | 2 +- docs/src/format/table/index/system/mem_wal.md | 12 + docs/src/format/table/index/system/memwal.md | 27 - docs/src/format/table/mem_wal.md | 1349 +++++++++ docs/src/format/table/mem_wal_regional.png | Bin 0 -> 551389 bytes docs/src/images/mem_wal_overview.png | Bin 0 -> 109386 bytes protos/table.proto | 216 +- protos/transaction.proto | 17 +- rust/lance-index/src/mem_wal.rs | 389 ++- rust/lance/src/dataset/fragment.rs | 4 +- rust/lance/src/dataset/transaction.rs | 117 +- rust/lance/src/dataset/write/commit.rs | 2 +- rust/lance/src/dataset/write/merge_insert.rs | 58 +- .../dataset/write/merge_insert/exec/delete.rs | 4 +- .../dataset/write/merge_insert/exec/write.rs | 4 +- rust/lance/src/dataset/write/update.rs | 2 +- rust/lance/src/index/mem_wal.rs | 2585 +---------------- rust/lance/src/io/commit/conflict_resolver.rs | 168 +- 19 files changed, 1968 insertions(+), 2989 deletions(-) create mode 100644 docs/src/format/table/index/system/mem_wal.md delete mode 100644 docs/src/format/table/index/system/memwal.md create mode 100644 docs/src/format/table/mem_wal.md create mode 100644 docs/src/format/table/mem_wal_regional.png create mode 100644 docs/src/images/mem_wal_overview.png diff --git a/docs/src/format/table/.pages b/docs/src/format/table/.pages index ec66d452eb6..69209d956cc 100644 --- a/docs/src/format/table/.pages +++ b/docs/src/format/table/.pages @@ -5,4 +5,5 @@ nav: - Layout: layout.md - Branch & Tag: branch_tag.md - Row ID & Lineage: row_id_lineage.md + - MemTable & WAL: mem_wal.md - index diff --git a/docs/src/format/table/index/system/.pages b/docs/src/format/table/index/system/.pages index cedf138336b..03435c92bf2 100644 --- a/docs/src/format/table/index/system/.pages +++ b/docs/src/format/table/index/system/.pages @@ -1,4 +1,4 @@ title: System Indices nav: - Fragment Reuse: frag_reuse.md - - MemWAL: memwal.md + - MemWAL: mem_wal.md diff --git a/docs/src/format/table/index/system/mem_wal.md b/docs/src/format/table/index/system/mem_wal.md new file mode 100644 index 00000000000..73693cd6704 --- /dev/null +++ b/docs/src/format/table/index/system/mem_wal.md @@ -0,0 +1,12 @@ +# MemWAL Index + +The MemWAL Index is a system index that serves as the centralized structure for all MemWAL metadata. +It stores configuration (region specs, indexes to maintain), merge progress, and region state snapshots. + +A table has at most one MemWAL index. + +For the complete specification, see: + +- [MemWAL Index Overview](../../mem_wal.md#memwal-index) - Purpose and high-level description +- [MemWAL Index Details](../../mem_wal.md#memwal-index-details) - Storage format, schemas, and staleness handling +- [MemWAL Index Builder](../../mem_wal.md#memwal-index-builder) - Background process and configuration updates diff --git a/docs/src/format/table/index/system/memwal.md b/docs/src/format/table/index/system/memwal.md deleted file mode 100644 index 41e2948409c..00000000000 --- a/docs/src/format/table/index/system/memwal.md +++ /dev/null @@ -1,27 +0,0 @@ -# MemWAL Index - -The MemTable and Write-Ahead Log (MemWAL) Index is used for fast upserts into the Lance table. - -The index is used as the centralized synchronization system for a log-structured merge tree (LSM-tree), -leaving the actual implementation of the MemTable and WAL up to the specific implementer of the spec. - -Each region represents a single writer that writes to both a MemTable and a WAL, -and a region can have increasing generations of MemWALs. -Every time data is written into a WAL, the index is updated with the latest watermark. -If a specific writer of a region dies, a new writer is able to read the information in the specific region and replay the WAL. - -## Index Details - -```protobuf -%%% proto.message.MemWalIndexDetails %%% -``` - -## Expected Use Pattern - -It is expected that: - -1. there is exactly one writer for each region, guaranteed by optimistic update of the owner_id -2. each writer updates the MemWAL index after a successful write to WAL and MemTable -3. a new writer always finds unsealed MemWALs and performs replay before accepting new writes -4. background processes are responsible for merging flushed MemWALs to the main Lance table, and making index up to date. -5. a MemWAL-aware reader is able to merge results of MemTables in the MemWALs with results in the base Lance table. \ No newline at end of file diff --git a/docs/src/format/table/mem_wal.md b/docs/src/format/table/mem_wal.md new file mode 100644 index 00000000000..060f174e53c --- /dev/null +++ b/docs/src/format/table/mem_wal.md @@ -0,0 +1,1349 @@ +# MemTable & WAL Specification + +Lance MemTable & WAL (MemWAL) specification describes an Log-Structured-Merge (LSM) tree architecture for Lance tables, enabling high-performance streaming write workloads. + +## Prerequisites + +### Unenforced Primary Key + +MemWAL only works for Lance tables with unenforced primary key defined, +also the unenforced primary key: + +- Must have a [btree index](./index/scalar/btree.md) +- Must be included in the region spec's `source_ids` if a region spec is specified (see [Region Spec](#region-sepc) for more details) + +The last constraint is critical for correctness. +If two regions contain rows with the same primary key, the following scenario can cause data corruption: + +1. Region A receives a write with primary key `pk=1` at time T1 +2. Region B receives a write with primary key `pk=1` at time T2 (T2 > T1) +3. The row in region B is merged into the base table first +4. The row in region A is merged into the base table second +5. The row from Region A (older) now overwrites the row from Region B (newer) + +This violates the expected "last write wins" semantics. +By ensuring each primary key is assigned to exactly one region via the region spec, merge order between regions becomes irrelevant for correctness. + +### IVF Vector Index + +Although later migration is possible, if the main use case is IVF family vector index, +it is recommended to have these indexes on the Lance table before enabling MemWAL. +This is because IVF index needs to remain the same quantization codebook (e.g. PQ codebook) +across all the layers of the LSM tree for vector distance to be comparable. +Migrating one codebook to another is a complicated proceses requiring gradual migration +and coordination between readers and writers. + +## Overall Architecture + +![MemWAL Overview](../../images/mem_wal_overview.png) + +### Base Table + +Under the MemWAL setup, the Lance table is called the **base table**. + +### MemWAL Region + +**MemWAL Region** is the main unit to horizontally scale out writes. +Each region has exactly one active writer at any time, using **epoch-based fencing** to guarantee single-writer semantics without distributed coordination. +Writers claim a region by incrementing the writer epoch, then write data to that region. +Data in each region is merged into the base table gradually in the background. + +#### Region Identifier + +Each region has a unique identifier across all regions following UUID v4 standard. +When a new region is created, it is assigned a new identifier. + +#### Region Spec + +A **Region Spec** defines how the all rows in a table is logically divided into different regions, +enabling automatic region assignment and query-time region pruning. + +Each region spec has: + +- **Spec ID**: A positive integer that uniquely identifies this spec within the MemWAL index. IDs are never reused. +- **Region fields**: An array of field definitions that determine how to compute region identifiers. + +Each region is bound to a specific region spec ID, recorded in its [manifest](#region-manifest). +Regions without a spec ID (`spec_id = 0`) are manually-created regions not governed by any spec. + +A region spec's field array consists of **region field** definitions. +Each region field has the following properties: + +| Property | Description | +|----------|-------------| +| `field_id` | Unique string identifier for this region field | +| `source_ids` | Array of field IDs referencing source columns in the schema | +| `transform` | A well-known region expression, specify this or `expression` | +| `expression` | A DataFusion SQL expression for custom logic, specify this or `transform` | +| `result_type` | The output type of the region value | + +#### Unenforced Primary key Constraint + +The `source_ids` across all region fields must include all primary key columns. +This ensures rows with the same primary key always map to the same region, which is required for correctness (see [Unenforced Primary Key](#unenforced-primary-key)). + +#### Region Expression + +A **Region Expression** is a [DataFusion SQL expression](https://datafusion.apache.org/user-guide/sql/index.html) that derives a region value from source column(s). +Source columns are referenced as `col0`, `col1`, etc., corresponding to the order of field IDs in `source_ids`. + +Region expressions must satisfy the following requirements: + +1. **Deterministic**: The same input value must always produce the same output value. +2. **Stateless**: The expression must not depend on external state (e.g., current time, random values, session variables). +3. **Type-promotion resistant**: The expression must produce the same result for equivalent values regardless of their numeric type (e.g., `int32(5)` and `int64(5)` must yield the same region value). +4. **Column removal resistant**: If a source field ID is not found in the schema, the column should be interpreted as NULL. +5. **NULL-safe**: The expression should properly handle NULL inputs and have defined behavior (e.g., return NULL if input is NULL for single-column expressions). +6. **Consistent with result type**: The expression's return type must be consistent with `result_type` in non-NULL cases. + +#### Region Transform + +A **Region Transform** is a well-known region expression with a predefined name. +When a transform is specified, the expression is derived automatically. + +| Transform | Parameters | Region Expression | Result Type | +|-----------|------------|-------------------|-------------| +| `identity` | (none) | `col0` | same as source | +| `year` | (none) | `date_part('year', col0)` | `int32` | +| `month` | (none) | `date_part('month', col0)` | `int32` | +| `day` | (none) | `date_part('day', col0)` | `int32` | +| `hour` | (none) | `date_part('hour', col0)` | `int32` | +| `bucket` | `num_buckets` | `abs(murmur3(col0)) % N` | `int32` | +| `multi_bucket` | `num_buckets` | `abs(murmur3_multi(col0, col1, ...)) % N` | `int32` | +| `truncate` | `width` | `left(col0, W)` (string) or `col0 - (col0 % W)` (numeric) | same as source | + +The `bucket` and `multi_bucket` transforms use Murmur3 hash functions: + +- **`murmur3(col)`**: Computes the 32-bit Murmur3 hash (x86 variant, seed 0) of a single column. Returns a signed 32-bit integer. Returns NULL if input is NULL. +- **`murmur3_multi(col0, col1, ...)`**: Computes the Murmur3 hash across multiple columns. Returns a signed 32-bit integer. NULL fields are ignored during hashing; returns NULL only if all inputs are NULL. + +The hash result is wrapped with `abs()` and modulo `N` to produce a non-negative bucket number in the range `[0, N)`. + +### MemWAL Index + +A **MemWAL Index** is the centralized structure for all MemWAL metadata for a base table. +A table has at most one MemWAL index. +It stores: + +- **Configuration**: Region specs defining how rows map to regions, and which indexes to maintain +- **Merge progress**: Last generation merged to base table for each region +- **Region snapshots**: Point-in-time snapshot of all region states for read optimization + +The index is the source of truth for **configuration** and **merge progress**, but region state snapshots are read-only (each region's manifest is authoritative for its own state). + +Writers read the MemWAL index to get configuration (region specs, maintained indexes) before writing. +Readers use the index to get a snapshot of all region states, then query each region's data alongside the base table and merge results at runtime. + +A background process periodically updates region snapshots by listing regions and loading their manifests. +See [MemWAL Index Details](#memwal-index-details) for the complete structure. + +## Region Architecture + +![Region Architecture](mem_wal_regional.png) + +Within a region, writes enter its MemTable and are flushed to the regional WAL for durability. +The MemTable is flushed to storage as a Flushed MemTable based on memory pressure and other conditions. +Flushed MemTables are then asynchronously merged into the base table. + +Here are the details of the related components and concepts: + +### MemTable + +An in-memory Lance table that buffers incoming writes. +Each write inserts a fragment in the MemTable, making data immediately queryable without waiting for persistence. + +### WAL + +Write-Ahead Log (WAL) seves as the durable sotrage of MemTable. +A write to MemTable must be persisted also to the WAL to become fully durable. +Every time we write to the WAL, we call it a **WAL Flush**. + +The whole LSM tree's durability is determined by the durability of the WAL. +For example, if WAL is stored in Amazon S3, it has the 99.999999999% durability. +If it is stored in local disk, the data will be lost if the local disk is damanaged. + +#### WAL Entry + +A WAL consists of an ordered sequence of WAL entries starting from 1. Each entry is a Lance format file. +The writer epoch is stored in the Lance file's schema metadata with key `writer_epoch` for fencing validation during replay. + +#### File Location + +Each WAL entry is stored within the WAL directory of the region located at `_memwal/{region_id}/wal`. + +#### File Naming + +WAL files use bit-reversed 64-bit binary naming to distribute files evenly across the directory keyspace. +This optimizes S3 throughput by spreading sequential writes across S3's internal partitions, minimizing throttling. + +The filename is the bit-reversed binary representation of the entry ID with suffix `.lance`. +For example, entry ID 5 (binary `000...101`) becomes `1010000000000000000000000000000000000000000000000000000000000000.lance`. + +This bit-reversal permutation ensures that sequential entry IDs are spread across the entire keyspace, similar to how [data files use UUID-based naming](layout.md#data-files) for S3 throughput optimization. + +### Flushed MemTable + +A flushed MemTable is a complete Lance table created by flushing the MemTable to storage. + +!!!note + This is called Sorted String Table (SSTable) or Sorted Run in many LSM-tree literatures and implementations. + However, since our MemTable is not sorted, we just use the term flushed memtable avoid confusion. + +#### Generation + +Each flushed MemTable has a **generation** number starting from 1 that identifies its relative position among all flushed MemTables in the region. +When MemTable with generation `i` is flushed, the next MemTable gets generation number `i+1`. + +#### Flush Location + +The MemTable of generation `i` is flushed to `_memwal/{region_uuid}/{random_hash}_gen_{i}/` directory, where `{random_hash}` is an 8-character hex value generated at flush time. +The directory content follows [Lance table layout](layout.md). + +The actual directory path for each generation is recorded in the region manifest's `flushed_generations` list (see [Region Manifest](#region-manifest)). + +#### Merging Flushed MemTable + +Generation numbers determine merge order: lower numbers represent older data and must be merged to the base table first to preserve correct upsert semantics. + +### Region Manifest + +Each region has a manifest file containing epoch-based fencing tokens, WAL pointers, and flushed MemTable generation trackers. This is the source of truth for region state. + +The manifest is serialized as a protobuf binary file using the `RegionManifest` message. + +#### Contents + +The manifest contains: + +- **Fencing state**: `writer_epoch` (writer fencing token) +- **WAL pointers**: `replay_after_wal_id` (last entry flushed to MemTable), `wal_id_last_seen` (last entry seen at manifest update) +- **Generation trackers**: `current_generation` (next generation to flush), `merged_generation` (last generation merged to base) +- **Flushed generations**: `flushed_generations` list of generation number and directory path pairs (e.g., generation 1 at `a1b2c3d4_gen_1`) + +Note: `wal_id_last_seen` is a hint that may be stale since it's not updated on WAL write. +The manifest itself is atomically written, but recovery must try get newer WAL files to find the actual state beyond this hint. + +
+RegionManifest protobuf message + +```protobuf +%%% mem_wal.message.RegionManifest %%% +``` + +
+ +#### Versioning and Atomicity + +Manifests are versioned starting from 1 and immutable. Each update creates a new manifest file at the next version number. +Updates use put-if-not-exists or file rename to ensure atomicity depending on the storage system. +If two processes compete, one wins and the other retries. + +To commit a manifest version: + +1. Compute the next version number +2. Write the manifest to `{bit_reversed_version}.binpb` using put-if-not-exists +3. In parallel best-effort write to `version_hint.json` with `{"version": }` (failure is acceptable) + +To read the latest manifest version: + +1. Read `version_hint.json` to get the latest version hint. If not found, start from version 1 +2. Check existence for subsequent versions from the starting version +3. Continue until a version is not found +4. The latest version is the last found version + +This approach uses HEAD requests instead of LIST operations in cloud storage, +which is in general faster on cloud storage systems and +is friendly to systems like S3 Express that do not support lexicographically sorted listing. + +#### File Location + +All region manifest versions are stored in `_memwal/{region_id}/manifest` directory. + +#### File Naming + +Each region manifest version file uses bit-reversed 64-bit binary naming, the same scheme as [WAL files](#wal-file-naming). +For example, version 5 becomes `1010000000000000000000000000000000000000000000000000000000000000.binpb`. + +#### Region Manifest Transactions + +The region manifest is updated atomically in the following cases: + +| Trigger | Fields Updated | Details | +|---------|----------------|---------| +| [Initialization & Recovery](#initialization--recovery) | `writer_epoch` | Incremented when writer claims the region | +| [MemTable Flush](#memtable-flush) | `replay_after_wal_id`, `wal_id_last_seen`, `current_generation`, `flushed_generations` | After flushing MemTable to storage | +| [Merge to Base Table](#merge-workflow) | `merged_generation`, `flushed_generations` | After merging a flushed MemTable; removes merged entry | +| [MemWAL Index Builder](#memwal-index-builder) | `wal_id_last_seen` | Periodically scans WAL entries and updates hint | + +!!!note + WAL flush does **not** update the manifest to keep the hot write path fast. + +#### Fencing + +Writers use epoch-based fencing (`writer_epoch`) to ensure single-writer semantics. +See [Writer Fencing](#writer-fencing) for details. + +### Storage Layout + +Here is a recap of the storage layout with all the files and concepts defined so far: + +``` +{table_path}/ +├── _indices/ +│ └── {index_uuid}/ # MemWAL Index (uses standard index storage) +│ └── regions.binpb # Serialized region snapshots (protobuf binary) +│ +└── _memwal/ + └── {region_uuid}/ # Region directory (UUID v4) + ├── manifest/ + │ ├── {bit_reversed_version}.binpb # Serialized region manifest (bit-reversed naming) + │ └── version_hint.json # Version hint file + ├── wal/ + │ ├── {bit_reversed_entry_id}.lance # WAL data files (bit-reversed naming) + │ └── ... + └── {random_hash}_gen_{i}/ # Flushed MemTable (generation i, random prefix) + ├── _versions/ + │ └── {version}.manifest # Table manifest (V2 naming scheme) + └── _indices/ # indexes + ├── {vector_index}/ + └── {scalar_index}/ +``` + +### MemWAL Index Details + +The MemWAL Index uses the [standard index storage](index/index.md#index-storage) at `_indices/{UUID}/`. + +The index stores its data in two parts: + +1. **Index details** (`index_details` in `IndexMetadata`): Contains configuration, merge progress, and snapshot metadata +2. **Region snapshots**: Stored as a Lance file or inline, depending on region count + +#### Index Details Schema + +The `index_details` field in `IndexMetadata` contains a `MemWalIndexDetails` protobuf message with: + +| Field | Type | Description | +|-------|------|-------------| +| `snapshot_timestamp` | int64 | When the index was built (Unix timestamp in seconds) | +| `num_regions` | uint32 | Number of regions in the snapshot | +| `inline_snapshots` | bytes | Inline snapshot data for small region counts (optional) | +| `region_specs` | repeated RegionSpec | Region specs defining how rows map to regions | +| `maintained_indexes` | repeated string | Index names to maintain in MemTables | +| `merged_generations` | repeated MergedGeneration | Last generation merged to base table per region | + +**Configuration fields** (`region_specs`, `maintained_indexes`) are the source of truth for MemWAL configuration. +Writers read these fields to determine how to partition data and which indexes to maintain. + +- **Region specs** define how rows are partitioned into regions. Multiple specs can coexist during migration. Each spec has a unique `spec_id` that is never reused. See [Region Spec](#region-spec) for field definitions. +- **Maintained indexes** lists indexes (by name) to maintain in MemTables. The primary key btree index is always maintained implicitly and should not be listed here. For vector indexes, MemTables inherit quantization parameters (PQ codebook, SQ params) from the base table index to ensure distance comparability. See [Vector Indexes](#vector-indexes). + +**Merge progress** (`merged_generations`) tracks the last generation merged to the base table for each region. +This field is updated atomically with merge-insert data commits, enabling conflict resolution when multiple mergers operate concurrently. +Each entry contains the region UUID and generation number. + +**Region snapshot fields** (`snapshot_timestamp`, `num_regions`, `inline_snapshots`) provide a point-in-time snapshot of region states. +The actual region manifests remain authoritative for region state. + +
+MemWalIndexDetails protobuf message + +```protobuf +%%% mem_wal.message.MemWalIndexDetails %%% +``` + +
+ +#### Region Snapshot Storage + +Region snapshots are stored using one of two strategies based on the number of regions: + +| Region Count | Storage Strategy | Location | +|--------------|------------------|----------| +| <= 100 (threshold) | Inline | `inline_snapshots` field in index details | +| > 100 | External Lance file | `_indices/{UUID}/index.lance` | + +The threshold (100 regions) is implementation-defined and may vary. + +**Inline storage**: For small region counts, snapshots are serialized as a Lance file and stored in the `inline_snapshots` field. +This keeps the index metadata compact while avoiding an additional file read for common cases. + +**External Lance file**: For large region counts, snapshots are stored as a Lance file at `_indices/{UUID}/index.lance`. +This file uses standard Lance format with the region snapshot schema, enabling efficient columnar access and compression. + +#### Region Snapshot Schema + +Region snapshots are stored as a Lance file with one row per region. +The schema has one column per `RegionManifest` field, with region fields as columns: + +| Column | Type | Description | +|--------|------|-------------| +| `region_id` | `fixed_size_binary(16)` | Region UUID bytes | +| `version` | `uint64` | Region manifest version | +| `region_spec_id` | `uint32` | Region spec ID (0 if manual) | +| `writer_epoch` | `uint64` | Writer fencing token | +| `replay_after_wal_id` | `uint64` | Last WAL entry flushed to MemTable | +| `wal_id_last_seen` | `uint64` | Last WAL entry seen (hint) | +| `current_generation` | `uint64` | Next generation to flush | +| `merged_generation` | `uint64` | Last generation merged to base | +| `flushed_generations` | `list>` | Flushed MemTable paths | + +This schema directly corresponds to the fields in the `RegionManifest` protobuf message. + +#### Staleness Handling + +Since the index is eventually consistent, readers should handle stale data: + +- A flushed MemTable listed in `flushed_generations` may have been merged and garbage collected +- New flushed MemTables may exist that are not yet in `flushed_generations` +- WAL entries may have advanced beyond what the index shows + +The `snapshot_timestamp` field indicates when the index was built; readers can use this to estimate staleness and decide whether to refresh. + +For authoritative state, readers may load individual region manifests directly from `_memwal/{region_uuid}/manifest/`. + +## Writer Expectations + +### Writer Configuration + +Writers can be configured with the following options that affect write behavior: + +| Option | Description | +|--------|-------------| +| **Durable write** | Each write is persisted to WAL before reporting success. Ensures no data loss on crash, but adds latency for object storage writes. | +| **Indexed write** | Each write refreshes MemTable indexes before reporting success. Ensures new data is immediately searchable via indexes, but adds indexing latency. | + +Both options can be enabled independently. When disabled: + +- **Non-durable writes** buffer data in memory until a flush threshold is reached, accepting potential data loss on crash +- **Non-indexed writes** defer index updates, meaning newly written data may not appear in index-accelerated queries until the next index refresh + +### Initialization & Recovery + +A writer must claim a region before performing any write operations: + +1. Load the latest region manifest +2. Increment `writer_epoch` by one +3. Atomically write a new manifest +4. If the write fails (another writer claimed the epoch), reload the manifest and retry with a higher epoch +5. Read WAL entries sequentially from `replay_after_wal_id + 1` until not found +6. Replay valid WAL entries (those with `writer_epoch` ≤ current epoch) to reconstruct the MemTable with 1:1 fragment mapping (each WAL entry becomes one MemTable fragment) + +After recovery, the writer tracks subsequent fragment mappings as new WAL flushes occur (see [WAL Flush](#wal-flush)). + +### Writer Fencing + +Before any manifest update (MemTable flush), a writer must verify its `writer_epoch` remains valid: + +- If `local_writer_epoch == stored_writer_epoch`: The writer is still active and may proceed +- If `local_writer_epoch < stored_writer_epoch`: The writer has been fenced and must abort + +Fenced writers must stop all operations immediately and notify pending writes of the failure. + +For a concrete example of fencing between two writers, see [Appendix 1: Writer Fencing Example](#appendix-1-writer-fencing-example). + +### Write Operations + +Each write operation follows this sequence: + +1. Validate incoming records +2. Insert records into the MemTable, creating an in-memory fragment (immediately queryable via full scan) +3. Track the Lance data file in the new fragment for pending WAL flush +4. Optionally trigger WAL flush based on size, count, or time thresholds +5. For [durable writes](#writer-configuration), wait for WAL flush to complete before returning +6. For [indexed writes](#writer-configuration), update MemTable indexes before returning: + - Insert primary keys into the btree index + - For each vector column with a base table index: encode and insert into HNSW graph + - For each index in `maintained_indexes`: update the corresponding index structure + +### WAL Flush + +WAL flush batches pending MemTable fragments into a single Lance data file: + +1. Identify pending (unflushed) fragments in the MemTable +2. Start writing the WAL entry to object storage +3. Stream binary pages from each pending fragment's Lance data file directly to the WAL entry +4. Write the footer containing batched data file metadata and `writer_epoch` in schema metadata +5. Complete the WAL entry write atomically +6. Mark fragments as flushed in the MemTable +7. Record fragment mappings (MemTable fragment IDs in this batch → WAL entry ID relative to last replay) for index remapping during [MemTable Flush](#memtable-flush) + +!!!note + The region manifest is **not** updated on every WAL flush. The `wal_id_last_seen` field is a hint that can be updated: + + 1. **During MemTable flush** - when the region manifest is updated anyway + 2. **By a background index builder** - which scans WAL entries and updates each region's `wal_id_last_seen` + + This keeps the hot write path fast. On recovery, the writer reads WAL entries sequentially starting from `wal_id_last_seen + 1` to discover any WAL entries beyond what the manifest indicates. + +The [durable write](#writer-configuration) option also impacts flush behavior: + +| Mode | Behavior | Result | +|------|----------|--------| +| Durable write | Flush immediately, wait for completion | One or more Lance files per write | +| Non-durable write | Buffer until threshold, return immediately | Batched Lance files (fewer S3 operations) | + +### MemTable Indexing + +MemTable indexing differs from base table indexing to balance write performance with query capability. +Rather than maintaining all base table indexes, MemTables maintain a subset specified in the [MemWAL Index](#memwal-index). + +#### Primary Key Index + +The **primary key btree index** is always maintained for every MemTable, regardless of other index configuration. +This index is essential for: + +- **Staleness detection**: During vector/FTS search, checking if a candidate from an older generation has a newer version +- **Point lookups**: Fast O(log n) access by primary key +- **Deduplication**: Efficiently finding duplicate primary keys during merge + +The primary key index is implemented as an in-memory `BTreeMap` where `OrderableScalarValue` wraps Arrow's `ScalarValue` with `Ord` implementation (see `lance-index::scalar::btree::OrderableScalarValue`). +For multi-column primary keys, the map key is a tuple of `OrderableScalarValue` for each column. + +#### Vector Indexes + +MemTables **automatically inherit** vector indexing from base table indexes. +This inheritance is critical for **distance comparability** across generations. + +**Why inheritance is required:** + +When ranking search results across generations, distances must be comparable: + +| Component | Must Match Base Table? | Reason | +|-----------|------------------------|--------| +| Quantization (PQ codebook, SQ params) | **Yes** | Determines distance calculation | +| IVF centroids | No | Only affects partition assignment | +| Search structure (HNSW vs flat) | No | Only affects search efficiency | + +If MemTable used independent quantization, distances from MemTable and base table would not be comparable, leading to incorrect ranking. + +**Inheritance behavior:** + +For each vector index on the base table, MemTable automatically: + +1. **Inherits quantization parameters**: PQ codebook, SQ min/max, or no quantization (flat) +2. **Encodes vectors** using inherited quantization during writes +3. **Stores both** raw vectors (for potential refinement) and quantized codes +4. **Uses HNSW** as the search structure (optimal for small MemTable scale) + +| Base Table Index | MemTable Inherits | MemTable Search | +|------------------|-------------------|-----------------| +| IVF-PQ | PQ codebook | HNSW on PQ codes | +| IVF-SQ | SQ parameters | HNSW on SQ codes | +| IVF-FLAT | Nothing (no quantization) | HNSW on raw vectors | +| IVF-HNSW-PQ | PQ codebook | HNSW on PQ codes | + +**Write path with inheritance:** + +``` +Write batch with vectors: + 1. Load PQ/SQ codebook from base table index (cached) + 2. Encode vectors using inherited quantization + 3. Store raw vectors + quantized codes in MemTable + 4. Insert into HNSW graph for search + 5. On MemTable flush, serialize both raw vectors and codes +``` + +**Query path with comparable distances:** + +``` +Search across generations: + 1. Search MemTable HNSW → candidates with quantized distances + 2. Search base table IVF-PQ → candidates with quantized distances + 3. Distances are COMPARABLE (same quantization) + 4. Sort by distance directly + 5. Apply staleness filtering +``` + +!!!warning "PQ Codebook Migration" + When the base table's PQ codebook is retrained, MemTable must switch to the new codebook. + During migration, maintain compatibility by: + 1. Flushing current MemTable before codebook change + 2. New MemTable uses new codebook + 3. Query both old flushed MemTables (old codebook) and new MemTable (new codebook) separately + 4. Merge flushed MemTables to base table before they become incompatible + +#### Scalar Indexes + +The `maintained_indexes` field in `MemWalIndexDetails` lists additional base table indexes to maintain in MemTables. +These include both scalar indexes (typically full-text search indexes needed for real-time text search) and vector indexes. + +Most scalar indexes other than FTS are not needed in MemTables since the primary key btree handles point lookups and staleness detection. + +#### Full-Text Search Indexes + +FTS indexes in MemTables **inherit tokenizer configuration** from base table indexes to ensure consistent tokenization across generations. + +**Inheritance behavior:** + +| Inherited | Not Inherited | +|-----------|---------------| +| Tokenizer type (simple, ngram, jieba, etc.) | Corpus statistics (IDF, avgdl) | +| Language settings | Document frequencies | +| Token filters (lowercase, stemming, etc.) | Posting lists | +| Position storage setting | | + +**Why corpus statistics are NOT inherited:** + +BM25 scoring depends on corpus-level statistics: +- `N`: Total document count +- `avgdl`: Average document length +- `df(t)`: Documents containing term t + +These statistics are specific to each corpus (generation). If MemTable used base table's statistics, scores would be incorrect because: +- IDF would be wrong (term rarity differs between 10K MemTable vs 10M base table) +- avgdl would be wrong (document length distribution may differ) + +**Global BM25 scoring (Lucene-style):** + +At query time, statistics are **aggregated across all generations** for globally-comparable BM25 scores: + +``` +Query: "machine learning" + +Step 1: Aggregate corpus statistics + N_global = Σ gen.doc_count + avgdl_global = Σ gen.sum_total_term_freq / N_global + +Step 2: Aggregate term statistics (for query terms only) + df_global("machine") = Σ gen.fts_index.df("machine") + df_global("learning") = Σ gen.fts_index.df("learning") + +Step 3: Compute global IDF + IDF("machine") = log(1 + (N_global - df_global) / (df_global + 0.5)) + +Step 4: Search each generation with global stats + Each FTS index returns candidates scored with global IDF and avgdl + Scores are now COMPARABLE across generations + +Step 5: Merge and rank globally +``` + +This follows the same pattern as [Apache Lucene's multi-segment BM25 scoring](https://github.com/apache/lucene), where: +- Each segment (generation) stores its own corpus statistics +- At query time, statistics are summed across segments +- A single scorer with global parameters is used for all segments + +**Required FTS index statistics:** + +Each MemTable FTS index must expose: + +| Statistic | Description | Used For | +|-----------|-------------|----------| +| `doc_count` | Documents in this index | Global N | +| `sum_total_term_freq` | Sum of all document lengths | Global avgdl | +| `df(term)` | Documents containing term | Global IDF | + +These are summed at query time to compute global BM25 parameters. + +#### In-Memory Index Structure + +Each MemTable maintains indexes as in-memory data structures: + +| Index Type | In-Memory Structure | Description | +|------------|---------------------|-------------| +| Primary key btree | `BTreeMap` | Maps primary key value(s) to row ID | +| Vector (HNSW + quantization) | `HnswBuilder` + `Quantizer` | HNSW graph + inherited PQ/SQ codebook | +| Additional btree | `BTreeMap` | Maps indexed column value(s) to row ID | + +**Memory overhead** for ~20K vectors (64MB MemTable): + +| Component | Size | Notes | +|-----------|------|-------| +| HNSW graph structure | ~5-10MB | Neighbors + distances | +| Thread-safe overhead | ~1MB | `RwLock` per node | +| PQ codebook (cached) | ~1MB | Shared across MemTables | +| PQ codes storage | ~1-2MB | 64 bytes/vector typical | +| **Total** | ~10-15% of MemTable size | + +#### Index Update Timing + +Index update timing depends on the [indexed write](#writer-configuration) setting: + +| Mode | Index Update Timing | Query Behavior | +|------|---------------------|----------------| +| Indexed write | Synchronous: indexes updated before write returns | New data immediately searchable via indexes | +| Non-indexed write | Deferred: indexes updated in background or at next flush | New data may require full scan until index refresh | + +When indexes are updated (either synchronously or deferred): + +1. **Primary key btree**: Insert `(pk_value, row_id)` into `BTreeMap` +2. **Vector indexes**: For each vector column with a base table index: + - Encode vector using inherited quantization (PQ/SQ codebook) + - Insert into HNSW graph with `O(log n)` complexity +3. **Other indexes**: Update according to index-specific logic + +Entries reference MemTable fragment IDs and row offsets. + +#### Flushed MemTable Index Caching + +When a MemTable is flushed to storage: + +1. In-memory indexes are serialized to disk in the flushed MemTable's `_indices/` directory: + - **Primary key btree**: Written as Lance btree index format + - **Vector indexes**: HNSW graph + quantized codes written in Lance format + - **Raw vectors**: Stored in data files for potential exact distance refinement + - **Other indexes**: Written in their respective formats +2. The in-memory index structures are retained as a **cache** for readers in the same process +3. Remote readers load indexes from disk; local readers use the cached in-memory structures + +This caching strategy provides: + +- **Zero-latency index access** for readers in the writer's process +- **No index rebuild overhead** for local readers after flush +- **Standard disk-based access** for remote readers + +Fragment mappings enable index remapping during [MemTable Flush](#memtable-flush). These mappings are recorded: + +- During [Initialization & Recovery](#initialization--recovery): 1:1 mapping from replayed WAL entries +- During [WAL Flush](#wal-flush): mapping from batched MemTable fragments to WAL entry + +### MemTable Flush + +Flushing the MemTable creates a new flushed MemTable (generation) with data and indexes: + +1. Generate a random 8-character hex prefix (e.g., `a1b2c3d4`) +2. Create directory `_memwal/{region_uuid}/{random_hash}_gen_{current_generation}/` +3. Identify WAL entries to include (from `replay_after_wal_id + 1` to the last flushed entry) +4. Create table manifest with `base_paths` pointing to the WAL directory +5. Add fragment entries referencing WAL files via `base_id` +6. Remap indexes using in-memory fragment mappings: + - Read index entries referencing MemTable fragment IDs + - Translate to flushed MemTable fragment IDs using mappings (MemTable fragment ID → WAL entry ID relative to last replay) + - Write remapped indexes to `_memwal/{region_uuid}/{random_hash}_gen_{current_generation}/_indices/` +7. Write the manifest to `_memwal/{region_uuid}/{random_hash}_gen_{current_generation}/_versions/{version}.manifest` (using [V2 naming scheme](transaction.md#manifest-naming-schemes)) +8. Update the region manifest: + - Advance `replay_after_wal_id` to the last flushed WAL entry + - Update `wal_id_last_seen` + - Increment `current_generation` + - Append `(current_generation, {random_hash}_gen_{current_generation})` to `flushed_generations` + +The random prefix ensures that flush retries write to a new directory, avoiding conflicts with partially written files from failed attempts. Only the directory recorded in `flushed_generations` is considered valid. + +If the writer crashes before completing MemTable flush, the new writer replays WAL entries into memory with 1:1 fragment mapping, rebuilds the in-memory indexes, and can then perform a fresh MemTable flush with a new random prefix. + +## Background Job Expectations + +Background jobs run independently from writers and handle asynchronous maintenance tasks. + +### Merge to Base Table + +Flushed MemTables are merged to the base table in generation order using Lance's merge-insert operation. + +#### Merge Workflow + +1. Load the MemWAL Index and read `merged_generations[region_id]` +2. Load the region manifest and identify unmerged flushed MemTables from `flushed_generations`: those with generation numbers in range `(merged_generation, current_generation)` +3. For each flushed MemTable in ascending generation order: + - Look up the directory path from `flushed_generations` + - Open it as a Lance table + - Execute merge-insert into the base table, atomically updating the MemWAL Index: + - Set `merged_generations[region_id]` to this generation + - On commit conflict, apply [conflict resolution rules](#merge-commit-conflict-resolution) + - On successful commit, update the region manifest: set `merged_generation` to this generation and remove the entry from `flushed_generations` + - If the region manifest update fails, continue to the next generation (MemWAL Index is authoritative) +4. After merge, the flushed MemTable and its referenced WAL files may be garbage collected (see [Garbage Collection](#garbage-collection)) + +Ordered merge ensures correct upsert semantics: flushed MemTables with higher generation numbers overwrite those with lower numbers. + +#### Merge Commit Conflict Resolution + +When a merge-insert commit to the base table encounters a version conflict, the merger reads the conflicting commit's MemWAL Index: + +- **Incompatible conflict**: If the conflicting commit's `merged_generations[region_id] >= my_generation`, abort without retry. The data is either already merged (same generation) or superseded (higher generation). +- **Compatible conflict**: Otherwise, retry the commit as normal. + +After aborting due to an incompatible conflict, reload the MemWAL Index and region manifest, then continue to the next unmerged generation. + +This conflict resolution prevents redundant work and ensures mergers don't regress the merge progress. + +#### Concurrent Mergers and Idempotency + +Multiple mergers may operate on the same region concurrently. This is safe due to: + +1. **Atomic MemWAL Index update**: The `merged_generations` in MemWAL Index is updated atomically with the data commit +2. **Conflict resolution**: Incompatible commits (same region, higher/equal generation) cause abort, not retry +3. **Merge-insert idempotency**: If two mergers merge the same generation before either commits, both write identical data (primary key upsert semantics) + +If a merger crashes after committing to the base table but before updating the region manifest: + +- The MemWAL Index has `merged_generations[region_id] = N` +- The region manifest still has `merged_generation = N-1` +- Next merger reads MemWAL Index, sees generation N already merged, skips it +- Region manifest is eventually updated to catch up + +The MemWAL Index `merged_generations` and region manifest `merged_generation` may temporarily differ. +The MemWAL Index is authoritative for conflict resolution; the region manifest is eventually consistent and used for `flushed_generations` cleanup. + +For a concrete example, see [Appendix 2: Concurrent Merger Example](#appendix-2-concurrent-merger-example). + +#### Implementation Consideration: Atomic Index Maintenance + +The merge commit should atomically update both data and indexes in the base table. +If data is merged but indexes are updated separately (e.g., via a background rebuild), there is a window where: + +1. Merged data exists in the base table but is not covered by base table indexes +2. The flushed MemTable (with its indexes) has been garbage collected +3. Queries must fall back to brute-force scans for the unindexed data, degrading performance + +To avoid this performance degradation: + +| Index Type | Recommended Approach | +|------------|---------------------| +| **Btree** | Incremental insert during merge transaction | +| **FTS** | Incremental update to posting lists and statistics | +| **Vector (IVF)** | Add vectors to existing partitions without retraining centroids | + +For vector indexes, adding to existing IVF partitions may cause partition imbalance over time. +Periodic rebalancing (e.g., SPFresh-style centroid updates) can address this, but the rebalancing operation itself should also be atomic with any data changes it affects. + +If atomic index maintenance is not feasible for a particular index type, implementations should either: + +- **Delay garbage collection**: Keep flushed MemTable indexes until base table indexes are updated +- **Track index coverage**: Maintain separate `index_merged_generation` to know which generations are covered by base table indexes + +### MemWAL Index Builder + +A background process periodically builds a new region snapshot: + +1. Load the existing MemWAL Index to preserve configuration (`region_specs`, `maintained_indexes`) and merge progress (`merged_generations`) +2. List all region directories under `_memwal/` +3. For each region: + - Load the region manifest + - Scan WAL entries sequentially to find the actual last entry ID + - If the observed WAL ID is greater than `wal_id_last_seen`, update the region manifest + - Copy manifest fields (including `flushed_generations`) into a region snapshot row +4. Determine storage strategy based on region count: + - If `num_regions <= threshold`: Serialize as Lance file bytes to `inline_snapshots` + - If `num_regions > threshold`: Write as Lance file to `_indices/{UUID}/index.lance` +5. Create new `MemWalIndexDetails` with preserved configuration, merge progress, and new region snapshots +6. Update the table manifest with the new index metadata + +This process serves two purposes: + +- Keeps `wal_id_last_seen` up-to-date in region manifests (since writers don't update it on every WAL flush) +- Provides readers with an efficient snapshot of all region states + +The build frequency is implementation-defined. More frequent builds reduce staleness but increase I/O overhead. + +#### Configuration Updates + +To update MemWAL configuration (add/remove region specs or maintained indexes): + +1. Load the existing MemWAL Index +2. Modify the configuration fields (`region_specs`, `maintained_indexes`) +3. Keep the existing `merged_generations` and region snapshots (or rebuild snapshots) +4. Write the new index with updated configuration +5. Update the table manifest with the new index metadata + +Configuration changes are versioned with the table manifest, ensuring writers and readers see consistent configuration for each table version. + +### Garbage Collection + +Garbage collection removes obsolete data from the region directory. This is a file-only operation that does not update the region manifest. + +Eligible for deletion: + +1. **Flushed MemTable directories**: Generation directories where `generation <= merged_generation` +2. **WAL data files**: Files referenced only by deleted generations +3. **Old region manifest versions**: Versions older than the current version minus a retention threshold +4. **Orphaned directories**: Directories matching `*_gen_*` pattern but not in `flushed_generations` (from failed flush attempts) + +**Time travel consideration**: Garbage collection must not remove generations that are reachable by any retained base table version. When a reader opens an older table version, the MemWAL Index snapshot from that version references specific `merged_generation` values. Generations that satisfy `generation > merged_generation` for any retained table version must be preserved. + +Garbage collection must verify that no flushed MemTable still references a WAL file before deletion. + +## Reader Expectations + +### Consistency Guarantees + +Reader consistency depends on two dimensions: + +| Dimension | Options | +|-----------|---------| +| **MemTable access** | Has access to in-memory MemTable, or only persisted data | +| **Manifest source** | Reads region manifests directly, or uses MemWAL Index | + +**Strong consistency** requires both: + +1. Access to in-memory MemTables for **all** regions involved in the query +2. Reading region manifests directly (not via MemWAL Index) + +Otherwise, the query is **eventually consistent**. + +#### Consistency Matrix + +| MemTable Access | Manifest Source | Consistency | +|-----------------|-----------------|-------------| +| All regions | Region manifest | **Strong** | +| All regions | MemWAL Index | Eventually consistent (index may be stale) | +| Partial/None | Region manifest | Eventually consistent (missing unflushed data) | +| Partial/None | MemWAL Index | Eventually consistent (both sources of staleness) | + +#### Sources of Staleness + +- **Missing MemTable access**: Unflushed data in a writer's in-memory MemTable is not visible +- **Stale MemWAL Index**: Newly flushed MemTables are not visible until the index is rebuilt +- **Stale region manifest cache**: If readers cache region manifests, newly flushed MemTables may not be visible + +### Query Planning + +From the query planner's perspective, MemWAL data is abstracted as a mapping: + +``` +region -> generation -> Dataset +``` + +Where: + +- **Region**: UUID identifying the region +- **Generation**: Integer generation number (`-1` for base table, `1+` for MemTables) +- **Dataset**: Either in-memory MemTable or persisted flushed MemTable (Lance table) + +The planner collects datasets from: + +1. **Base table**: generation = -1 +2. **Flushed MemTables**: generations in range `(merged_generation, current_generation)` from region manifest or MemWAL Index +3. **In-memory MemTable**: generation = `current_generation` (if accessible) + +### Query Execution + +Query execution unions all datasets and deduplicates by primary key. + +**Deduplication ranking** uses two virtual columns: + +- `_gen`: Generation number (-1 for base, 1+ for MemTables) +- `_rowaddr`: Row address within the dataset + +The ordering for "newest" is: highest `_gen` first, then highest `_rowaddr` (within the same generation, later rows win). + +A single write batch may contain duplicate primary keys. Query execution must deduplicate, keeping only the newest row for each key. + +For detailed query plans by query type, see [Appendix 3: Query Execution Examples](#appendix-3-query-execution-examples). + +## Durability Guarantees + +| Mode | Guarantee | Latency | +|------|-----------|---------| +| Durable write | Data persisted to object storage before return | Higher (S3 PUT latency) | +| Non-durable write | Data in memory only until next flush | Lower (memory write only) | + +Writers using non-durable writes accept potential data loss between the last flush and a crash. +The flush interval and buffer size thresholds control the maximum data at risk. + +For configuration details, see [Writer Configuration](#writer-configuration). + +## MemWAL Optimizations + +One key reason we call the whole system MemWAL is that we could perform the following 2 optimziations +to minimize flush latency: + +### WAL Flush + +the list of fragments in MemTable can be viewed as an in-memory buffer of the WAL. This means instead of writing the same data twice to MemTable and WAL, we write data once to MemTable, and then WAL can be flushed from the data file in the fragments. + +### MemTable Flush + +because the list of WAL entries are Lance data files, +we can directly treat them as the data files of the flushed MemTable. + +## Appendices + +### Appendix 1: Writer Fencing Example + +This example demonstrates how epoch-based fencing prevents data corruption when two writers compete for the same region. + +#### Initial State + +``` +Region manifest (version 1): + writer_epoch: 5 + replay_after_wal_id: 10 + wal_id_last_seen: 12 +``` + +#### Scenario + +| Step | Writer A | Writer B | Manifest State | +|------|----------|----------|----------------| +| 1 | Loads manifest, sees epoch=5 | | epoch=5, version=1 | +| 2 | Increments to epoch=6, writes manifest v2 | | epoch=6, version=2 | +| 3 | Starts writing WAL entries 13, 14, 15 | | | +| 4 | | Loads manifest v2, sees epoch=6 | epoch=6, version=2 | +| 5 | | Increments to epoch=7, writes manifest v3 | epoch=7, version=3 | +| 6 | | Starts writing WAL entries 16, 17 | | +| 7 | Tries to flush MemTable, loads manifest | | | +| 8 | Sees epoch=7, but local epoch=6 | | | +| 9 | **Writer A is fenced!** Aborts all operations | | | +| 10 | | Continues writing normally | epoch=7, version=3 | + +#### What Happens to Writer A's WAL Entries? + +Writer A wrote WAL entries 13, 14, 15 with `writer_epoch=6` in their schema metadata. + +When Writer B performs crash recovery or MemTable flush: + +1. Reads WAL entries sequentially starting from `replay_after_wal_id + 1` (entry 13) +2. For each entry, checks existence using HEAD request on the bit-reversed filename +3. Continues until an entry is not found (e.g., entry 18 doesn't exist) +4. Finds entries 13, 14, 15, 16, 17 +5. Reads each file's `writer_epoch` from schema metadata +6. Entries 13, 14, 15 have `writer_epoch=6` which is ≤ current epoch (7) → **valid, will be replayed** +7. Entries 16, 17 have `writer_epoch=7` → **valid, will be replayed** + +#### Key Points + +1. **No data loss**: Writer A's entries are not discarded. They were written with a valid epoch at the time and will be included in recovery. + +2. **Consistency preserved**: Writer A is prevented from making further writes that could conflict with Writer B. + +3. **Orphaned files are safe**: WAL files from fenced writers remain on storage and are replayed by the new writer. They are only garbage collected after being included in a flushed MemTable that has been merged. + +4. **Epoch validation timing**: Writers check their epoch before manifest updates (MemTable flush), not on every WAL write. This keeps the hot path fast while ensuring consistency at commit boundaries. + +### Appendix 2: Concurrent Merger Example + +This example demonstrates how MemWAL Index and conflict resolution handle concurrent mergers safely. + +#### Initial State + +``` +MemWAL Index: + merged_generations: {region: 5} + +Region manifest (version 1): + merged_generation: 5 + current_generation: 8 + flushed_generations: [(6, "abc123_gen_6"), (7, "def456_gen_7")] +``` + +#### Scenario 1: Racing on the Same Generation + +Two mergers both try to merge generation 6 concurrently. + +| Step | Merger A | Merger B | MemWAL Index | Region Manifest | +|------|----------|----------|--------------|-----------------| +| 1 | Reads index: merged_gen=5 | | merged_gen=5 | merged_gen=5 | +| 2 | Reads region manifest | | | | +| 3 | Starts merging gen 6 | | | | +| 4 | | Reads index: merged_gen=5 | merged_gen=5 | merged_gen=5 | +| 5 | | Reads region manifest | | | +| 6 | | Starts merging gen 6 | | | +| 7 | Commits (merged_gen=6) | | **merged_gen=6** | merged_gen=5 | +| 8 | | Tries to commit | | | +| 9 | | **Conflict**: reads new index | | | +| 10 | | Sees merged_gen=6 >= 6, aborts | | | +| 11 | Updates region manifest | | merged_gen=6 | **merged_gen=6** | +| 12 | | Reloads, continues to gen 7 | | | + +Merger B's conflict resolution detected that generation 6 was already merged by checking the MemWAL Index in the conflicting commit. + +#### Scenario 2: Stale Merger with Out-of-Order Attempt + +Merger B has a stale view and tries to merge an older generation. + +| Step | Merger A | Merger B | MemWAL Index | Region Manifest | +|------|----------|----------|--------------|-----------------| +| 1 | Reads index, region manifest | | merged_gen=5 | merged_gen=5 | +| 2 | Merges gen 6, commits | | **merged_gen=6** | merged_gen=5 | +| 3 | Updates region manifest | | merged_gen=6 | **merged_gen=6** | +| 4 | Merges gen 7, commits | | **merged_gen=7** | merged_gen=6 | +| 5 | Updates region manifest | | merged_gen=7 | **merged_gen=7** | +| 6 | | Reads stale index | merged_gen=7 | merged_gen=7 | +| 7 | | Thinks gen 6 needs merging | | | +| 8 | | Tries to commit gen 6 | | | +| 9 | | **Conflict**: reads new index | | | +| 10 | | Sees merged_gen=7 >= 6, aborts | | | +| 11 | | Reloads index, skips gen 6, 7 | | | + +Even with a stale MemWAL Index, Merger B correctly detected that generation 6 was already merged by checking the authoritative MemWAL Index in the conflicting commit. + +#### Scenario 3: Crash After Table Commit + +Merger A crashes after committing to the table but before updating the region manifest. + +| Step | Merger A | Merger B | MemWAL Index | Region Manifest | +|------|----------|----------|--------------|-----------------| +| 1 | Reads index: merged_gen=5 | | merged_gen=5 | merged_gen=5 | +| 2 | Merges gen 6, commits | | **merged_gen=6** | merged_gen=5 | +| 3 | **CRASH** before region update | | merged_gen=6 | merged_gen=5 | +| 4 | | Reads index: merged_gen=6 | merged_gen=6 | merged_gen=5 | +| 5 | | Reads region manifest | | | +| 6 | | Region says gen 6 unmerged... | | | +| 7 | | But index says merged_gen=6 | | | +| 8 | | **Skips gen 6** (index authoritative) | | | +| 9 | | Merges gen 7, commits | | **merged_gen=7** | +| 10 | | Updates region manifest | | **merged_gen=7** | + +The MemWAL Index is authoritative. Even though the region manifest was stale, Merger B correctly used the MemWAL Index to determine that generation 6 was already merged. + +#### Key Points + +1. **MemWAL Index is authoritative**: The `merged_generations` in MemWAL Index is the source of truth for merge progress, updated atomically with data. + +2. **Region manifest is eventually consistent**: It may lag behind MemWAL Index after crashes, but is eventually updated by subsequent mergers. + +3. **Conflict resolution uses MemWAL Index**: When a commit conflicts, the merger checks the conflicting commit's MemWAL Index, not the region manifest. + +4. **No progress regression**: Because MemWAL Index is updated atomically with data, concurrent mergers cannot regress the merge progress. + +5. **Crash recovery is safe**: If a merger crashes after table commit but before region manifest update, subsequent mergers use MemWAL Index to skip already-merged generations. + +### Appendix 3: Query Execution Examples + +This appendix provides query plan examples. All examples assume the planner has collected datasets as: + +``` +datasets = { + -1: base_table, # generation -1 = base table + 1: flushed_gen_1, # flushed MemTable generation 1 + 2: flushed_gen_2, # flushed MemTable generation 2 + 3: in_memory_memtable, # current generation (if accessible) +} +``` + +The core pattern for all queries: + +1. **Union** all datasets with their generation number +2. **Deduplicate** by primary key, ranking by `(_gen DESC, _rowaddr DESC)` +3. **Apply** query-specific operators (filter, sort, limit) + +#### Scan Queries + +``` +GlobalLimitExec: limit=n + DeduplicateExec: partition_by=[primary_key], order_by=[_gen DESC, _rowaddr DESC] + UnionExec + ScanExec: dataset[gen=-1], projection=[columns], filter=[pushed_down] + ScanExec: dataset[gen=1], projection=[columns], filter=[pushed_down] + ScanExec: dataset[gen=2], projection=[columns], filter=[pushed_down] + ScanExec: dataset[gen=3], projection=[columns] +``` + +Early termination is possible with a streaming deduplicate operator. + +#### Vector Search Queries + +Vector search requires special handling for staleness detection. Consider this scenario: + +1. Base table has `pk=123` with vector `v1` that matches the query (distance = 0.1) +2. MemTable has `pk=123` with updated vector `v2` that doesn't match (distance = 0.9) +3. KNN search on base table returns `pk=123` (good score) +4. KNN search on MemTable does NOT return `pk=123` (v2 is far from query) +5. Without staleness detection, the old version from base table would be incorrectly returned + +The solution uses the **primary key btree index** to filter out stale results: + +``` +GlobalLimitExec: limit=k + SortExec: order_by=[_dist ASC] + FilterStaleExec: pk_indexes=[btree[gen=3], btree[gen=2], btree[gen=1]] + UnionExec + KNNExec: dataset[gen=3], k=k*overfetch -- highest gen first + KNNExec: dataset[gen=2], k=k*overfetch + KNNExec: dataset[gen=1], k=k*overfetch + KNNExec: dataset[gen=-1], k=k*overfetch +``` + +For each candidate from generation G, `FilterStaleExec` checks if the primary key exists in btree indexes of generations > G. If found, the candidate is filtered out. The newer version doesn't participate in ranking since it didn't match the query. + +#### Full-Text Search Queries + +Full-text search has two challenges across generations: + +1. **Staleness**: A document may match a query in an older generation but not in a newer generation after the text was updated +2. **Score comparability**: BM25 scores depend on corpus statistics which differ per generation + +**Solution:** Use global BM25 scoring (Lucene-style) with staleness filtering. + +``` +-- Physical plan +GlobalLimitExec: limit=k + SortExec: order_by=[_bm25 DESC] + FilterStaleExec: pk_indexes=[btree[gen=3], btree[gen=2], btree[gen=1]] + GlobalBM25Exec: -- aggregates stats, creates single scorer + UnionExec + FTSExec: dataset[gen=3], query="search terms" + FTSExec: dataset[gen=2], query="search terms" + FTSExec: dataset[gen=1], query="search terms" + FTSExec: dataset[gen=-1], query="search terms" +``` + +**GlobalBM25Exec** performs: + +1. Collects `doc_count` and `sum_total_term_freq` from all FTS indexes +2. Computes global `N` and `avgdl` +3. For query terms, sums `df(term)` from all indexes to compute global IDF +4. Passes global BM25 parameters to each `FTSExec` +5. All candidates receive globally-comparable BM25 scores + +This ensures fair ranking between base table (large corpus) and MemTable (small corpus) results. +See [Full-Text Search Indexes](#full-text-search-indexes) for details on the global scoring approach. + +#### Point Lookups + +Point lookups can short-circuit by checking newest generations first: + +``` +-- Physical plan (short-circuit evaluation) +CoalesceExec: return_first_non_null + -- Check newest generation first, take last row (scan is ordered by _rowaddr) + TakeLastExec: + ScanExec: dataset[gen=3], filter=[primary_key = target] + TakeLastExec: + ScanExec: dataset[gen=2], filter=[primary_key = target] + TakeLastExec: + ScanExec: dataset[gen=1], filter=[primary_key = target] + TakeLastExec: + ScanExec: dataset[gen=-1], filter=[primary_key = target] +``` + +Point lookups terminate early once the key is found. Since scans are naturally ordered by `_rowaddr`, we take the last matching row without explicit sorting. + +### Appendix 4: Execution Nodes + +This appendix describes custom execution nodes for MemWAL query execution. These nodes are optimized for MemWAL's data model where each dataset has a fixed `_gen` and rows are naturally ordered by `_rowaddr`. + +#### DeduplicateExec + +Deduplicates rows by primary key, keeping the row with highest `(_gen, _rowaddr)`. + +**Semantics:** +``` +For each primary key across all input datasets: + Keep the row with max(_gen), breaking ties by max(_rowaddr) +``` + +**Optimized implementation:** + +Since each dataset has a fixed `_gen` and rows are naturally ordered by `_rowaddr`: + +1. Process datasets from highest `_gen` to lowest +2. Maintain a set of seen primary keys +3. For each dataset: + - Scan rows (naturally ordered by `_rowaddr`) + - For each primary key, buffer rows until key changes, emit last row + - Skip primary keys already in seen set + - Add emitted primary keys to seen set +4. Stream results without full materialization + +**Complexity:** O(n) where n = total rows, with O(k) memory where k = unique primary keys. + +#### TakeLastExec + +Takes the last row from an ordered input stream. + +**Semantics:** +``` +Buffer rows until input exhausted, emit final row +``` + +**Optimized implementation:** + +Since we only need the last row: + +1. Iterate through input, keeping only the most recent row in memory +2. On input exhaustion, emit the buffered row (or nothing if empty) + +**Complexity:** O(n) time, O(1) memory (single row buffer). + +#### CoalesceFirstExec + +Returns the first non-null/non-empty result from multiple inputs, with short-circuit evaluation. + +**Semantics:** +``` +For each input in order: + Execute input + If result is non-empty, return it immediately + Otherwise, continue to next input +Return empty if all inputs are empty +``` + +**Implementation:** + +1. Inputs are evaluated lazily in order +2. On first non-empty result, return immediately without evaluating remaining inputs +3. Useful for point lookups: check newest generation first, return on match + +**Complexity:** Best case O(1) inputs evaluated, worst case O(k) where k = number of inputs. + +#### FilterStaleExec + +Filters out rows that have a newer version in a higher generation. Used for search workloads where the newer version may not appear in search results (e.g., updated vector no longer matches query). + +**Parameters:** + +- `pk_indexes`: List of primary key btree indexes for each generation, ordered by generation descending + +**Why btree index lookup is necessary:** + +A naive approach would only check if the same primary key appears in search results from newer generations. However, this fails when: + +1. Vector `v1` (generation 1) matches query → returned by KNN +2. Vector `v2` (generation 2, same pk) doesn't match query → NOT returned by KNN +3. Naive approach: pk only appears once in results → not filtered (WRONG) +4. Btree approach: check btree[gen=2] for pk → found → filtered out (CORRECT) + +**Behavior:** + +Stale results are **filtered out** (not included in output). They do not participate in final ranking. This is the correct semantic for search: if a row was updated and the new version doesn't match the query, the old matching version should not be returned. + +**Algorithm (iterative roll-up):** + +Process generations from highest to lowest, accumulating known primary keys: + +``` +known_pks = {} # pks confirmed to exist in processed generations + +For gen in [highest_gen, ..., lowest_gen]: + gen_candidates = candidates.filter(_gen == gen) + + For each candidate (pk, gen) in gen_candidates: + # Fast path: pk already seen in a higher generation's results + If pk in known_pks: + Filter out candidate (stale) + Continue + + # Slow path: check btree indexes of higher generations + For check_gen in [highest_gen, ..., gen+1]: + If pk_indexes[check_gen].contains(pk): + Filter out candidate (stale) + Break + + If not filtered: + Emit candidate + + # Roll up: add all pks from this generation's btree to known set + # This enables fast-path checks for lower generations + known_pks.add_all(pk_indexes[gen].keys()) +``` + +**Why roll-up matters:** + +Without roll-up, each candidate requires btree lookups in all higher generations. With roll-up: + +1. Process gen 3: emit candidates, add gen 3 pks to `known_pks` +2. Process gen 2: check `known_pks` first (O(1)), then btree if needed +3. Process gen 1: many pks already in `known_pks`, fewer btree lookups + +**Complexity:** + +- Best case: O(n) when most pks are found via `known_pks` fast path +- Worst case: O(n × g × log m) where n = candidates, g = generations, m = rows per generation + +**Optimization:** For in-memory btree indexes (cached from flushed MemTables), lookups are O(log m) with no I/O. The base table may use a different staleness check mechanism (e.g., deletion vectors) since it doesn't maintain an in-memory btree. + +#### Usage in Query Plans + +| Query Type | Execution Pattern | +|------------|-------------------| +| Scan | `DeduplicateExec` → streams deduplicated rows | +| Point Lookup | `CoalesceFirstExec` → `TakeLastExec` per dataset | +| Vector Search | `FilterStaleExec(pk_indexes)` → `SortExec(_dist)` → `LimitExec` | +| Full-Text Search | `GlobalBM25Exec` → `FilterStaleExec(pk_indexes)` → `SortExec(_bm25)` → `LimitExec` | + +**Note on Vector/FTS Search:** + +- **Staleness filtering:** `FilterStaleExec` uses primary key btree indexes to filter out stale versions even when the newer version doesn't appear in search results (e.g., updated vector/text no longer matches query). Stale results are removed before ranking, ensuring top-k contains only current versions. +- **Vector scoring:** Distances are directly comparable across generations because MemTable inherits quantization (PQ codebook) from base table. +- **FTS scoring:** BM25 scores are made comparable via `GlobalBM25Exec`, which aggregates corpus statistics across generations (Lucene-style multi-segment scoring). diff --git a/docs/src/format/table/mem_wal_regional.png b/docs/src/format/table/mem_wal_regional.png new file mode 100644 index 0000000000000000000000000000000000000000..cea8eebdd652b1ccfef2d1acdb8cf33c58e26ddf GIT binary patch literal 551389 zcmb4LcRbbm8!wTxWMyT960&8_GPBCc2qD`cdmp(f*&*2@S(({;l$lvZ$5v)YIyer8 z!|(G|O80l~?|1J%_jRtEb3WsF-p~8}ex9#@8!B?Aj#C{!bm-721$k+;Lx)a=fS1(A zj)I@C=I(AEI&|icg0z%|+u<)`1T`>|F~M(h41DtMrDc%lXO~zCIL-$=<}rKB!$!Ga z@b<>dH2KNv;pZzYDk#-7h#UQVGf*NuUaFX|Lx*v2kDZ~Hy#M`0R@MED1OE&C+;`!`<0pUqwYvMq2%Olt z3|HuWd;iJU`}4#oy7g-P?nca+ z7oI-v(G6VQwY;&j!b}s%Pcd6rZ`a`S{2zFBYmA>b=3o%|>6=o|0}kBejVp(dAee4~ z3nZ(3^@d?Th$(1&9k4i#U#{Ku+n#~FFRaF1aE~1(Ts?B$)ZxqA?Lcua&DkB(`kr9) z_My+oc=(e~X z&FW{9jOxe=;GnzPiKXy-QeVR^?4lz+?hz|zXe)g5{Qg#;*f=)0pOM!%-gWe7cYMhF z+*8e*?IQsX<4Q77T3dFi654gEa`Ma%*uffAuSPFTiOlj&m+SNu1l||if3p}!m3bl+ zfxS$5mm|3?`4W*A4PSzP`ua9jvlqJKqXyn3?7Z;*Wbj>Zzc#C9-*TP2^ zZX;4Uf;uY^b$LmR)CENbq+*?KMPrTPWl-|;>CP@3V!`a6gZ)Jl3NIvxC;jQer3K zuh?=f7aMG8d~3WdKX%+AnT6hx$uFaiT16IJ)|?nR0jti9w+&x0kSHu!j7GgJkaw?k zJo|DPLC(=N<>VPs4Vzf+%KCBbU@gAK364EH|U)Z_L_V6E4c}BgJN2 z*ZGeo*5O}lY<)m6wiQ%|@1=YzzhHg=VZvqDe9KegJ<)F%>&r^uv~}Aee7&pZ)oDyc zn`8CZI{V7Dy?}Rh;w{dlAW`%fY=U7qQ)NqQe#%+fB=)HSBl2jTk0c+Qo*&*I@N8~L zV13?9!mg9S*{{OgA3j46f8fvJ)Dsz{a_}^o6PK z7q(r%7CyK9wXHUi8*(W;IDT#!RqmZqqJ&9T%);kBh%AkC@viH@R=YKOPGpH_&31ig zm1IrkQ>))V30$uUH?H4lDL}MZ)9Ig*+0R!lSxKp9l0!beLT!fQXK6fR=#E*S?l-=A zdaBrSd^G3)78ZSNZ70~zBFy2zWK5{h!?pFF!BXLs@4Uu&u&J5x zhBrdYM|dbp<2vYl-1VHk6nWL4c*33u5Iy`=q1rFr5&!jnEmA4b$%W7IRNp7Ns%q?N z)stUx9CF*)$!M!=x1n&f-tjgV3hh`&(yUVY>wRiUdNZ`Ye=^d?qrQ8KRgsz|m(G{6 zxoPHq7%zY-B;jr^WD<>cO#jSfMfRarhX*oNrRNJ<=mtC6Smv46LV2Z z6`SZJQ$dauk!h(lc0UF0(3^Zcs6l$?ZkZp5a2z}g~0 zcJjnHM&P9V`SxZD{`~`ZkAp1jkhVi!Q6DP9ARJ9SLO8#5R7JgIvExA5J$W?aoGQV; z<-GIWl}0X|l#Av2P5ioB0QQ%e-W4QAMR_L}Z4Oxqtafh2BDp?9q0qRAoz@h!i;f&4 z%Y`em>g!udtJ}xN5}YsYgER$O%3ZbN{q=H>m%AoCXN_bj0@K1V0zt@A-CQQ|BHzy& zLQz}X$;Any!G#I>QytdWtsq;&jt}B>=ktcO_@|_LnBggR;=|24W4O(zA9+r2br;6> zNDcoiQcn%V77r|RBUWy^CTFD1)WUEfEqvpX)W?>^l)?;^ z#JGi#wfVljT%BU8kpv;f-jp(o5C%oHO53F6IzP9VQjm$<&pjBf1_}L*tG*lKc04fT z5xf!Zmr+vXga#J%&I?WS@D^h|VP!Kvm2u`_r*&jgKqUG;x(naT4x3;&r5z#=C-nurD)vrLD|tNQ z5ukVtPmaZ0{CSm!ah7l=&27tu0uCQp_BDbN#&}kH)_VLNRb1OR450QkG#%#?_ZK4C z@^e!ujpU|Ij;C41!X;EkyWFN|tg&8xZsQ_$odIr~twKR?zE+CDD=k_AxrTEsXxI0v z&V4?E5pvg~`G|D{!nug+D;AH67r<(CjDrLeYNRHwgf8kC7&% zhxrXo){1Ar&aA@D6(f&|XJO&>s&TEu0mb;PBA^M+ZZtKQ&V<^p;V${(uyorp+Y$cp2K>?q zH#>?ZIF_V`-VmRAIhT7eS+sSkbOfy!z8;A+(b@iqe&EID%=?9ZvyDck8`H57Qfo9} zbGE`_avylFlH3$;o6D)9^kgu$bMBTB<3w~-__^)6^3HONnRiAV@^v`gB$f%E2?Zae6{MQ*!6~_w?k_q9r?vi+{nNp;p?e=t5I(+&GOLpj5&FP@_2VD zw}rGRK-qXlji70+cH@c3XAK9H*Z6N13M7^9G{a&ITK8}rgDiwW{Nkx$!^!=; z--q2jzrpwIrFUqBfXnnGPdde`W^$kb^NVR2L>rhxm<7j z9X3grKX%ICEf~5sBl1woc!8lh%XC2&q+KCdVFNt~0%ax{|yR=M_^Hurv6jV>44_2@UIT zI~X~TMjOKI^lxA~^BK|8-Y(wFh21Xct2D#bxM_EEU_Ad&nZ1%b%e~BlJ)JXlyB(fv znKZ$)LCPS#m)kdJ4u^xhc#Z?sJh@e{kyK$A8PF6?$!oS|UGZj0NxN5`DG*n-fT&48 z#NK}LDoXef{MXmF(?7Z5zY*t3QMR|p4H*rxvJH5al^$8Xq5yXjA+=4PI-8$S1^U#~ zxnl8&OU(QENI~ZdPc{YcpVyn9=gs*d7ln2U;TZFp5RDzyS~3?MYvR?(U;Vxlt@$L{ z!nLi{66E&`@qNoi?*J{H!oYVUNf4fYbJtz#y?m(9Y>dXUrf>oUDwa4dVsZw!o5tGJ z&#FJdk>dXS^Uz(V$oX!}e7QTEp&b7#V@Q{(3B6sW^K#-SvXOlxfiLzH2AjlVuq&Yw z%Y&k&$VN*}NZfLDz%sASaTkVRg)<`2!Tf&J1siZ%-HzqCjM7bUS03+)3FTa|ol7#` z)h$#QU!M>>@*P{b2=EpprUH`{<;JfP{ZYge8tmCC|Na4}PA<0}Gi+e?%As#JUdeZ9 z_Wf)nui#d>TPTmb40wH)_e=qL`yMbXkdz9fzmERcnV3t5=YHv`cZrhI?9!9+He)ES zDrZV!xfU_YyNh|{aNU%k+Kq>rcNd$i$;bG>8yYMQ%N_h2D1!dN()qLVLxJxre=YzP zLn7C5#qw@TFr}FOvqPa|vK|Y$>gjm62PHbR^FP1+ufL`J@|gg}!eg@2{A)Zt0k`e% zjz-8b7y*<%Y^6h%b$GN$?PsOmhZq~2S;fDG81NN8VIS=1qUnS1kuz@@)j`$>@3dvc zhnJ-Hqd4&g0BemLC;mJ^-%ETgnW^OgDYCb@%YJoeh8hv{cvzg4R{O_=T~T}gl^{Y z^(w+;1m-MkMZTp$shM^xceBa8wNOLx&}`|=pC|v$K=Q=F?Lg7~Q}h42@n6~-2!L?( znVe83-s-l3$D%>D7*b1-ci?o=MrPC*jMDT<7!Lgi(DTGxn*ZvvK@uD%Sc^D9 zl!x}@5oM|X$*NPSBQrWxG=|WzKb31x`kR{fYq7`9{3zY~CmHhvr}koKu>VLygsgE? zxb2b055gN3%97V{ub;m|eDIBb9dUmVm?`+aZ4YCvSZlS4nWTpstZ)d6zpi<7M3HLV zB`RE=B9$p z)IYhskJK8{{<0h$U}@mH_YUM_sU&9rqu)t4dIiQYl|fdw=h)*tHGJTF z|8$N}bnlz%k*f5>atx}ek!@iAgTj_0it%;KAx(Ep0-oAl$$hbZessSs z=(Aq~(6S?;y**9g_J)>ek%vZ(TX6Z}s3`w-Q2(?7A5(fDnafA4H(nVA2vp4i$Aa2| zVz0aebN|4XD?c3pC>g;C44<%&`iI{FK(&r3#L02ovMk zG+B(ew}sLQz;$^ZF9bq8t6GUzegGC&_g1F8*ICrI`4>RL^u7 zEwgBRqrfP`oFeffXi{`>QV;e)pxyr}oIlUtV0}HQm7+f>ZwaFDk_rW z0-%-q%cOuyeT8e4^xKNK4jsPsk2w?9d7Q*tel;iKsb_pVplGE!4kZ7=I^V-5DE>`z zHv2r6=lYj7zNFWw=38-(%iTG&AHlwt56}Fz%KyioC1vom^kAiN*TsUW!yiwAKCVQ;tfE^(+0O`-C&Te)nB4B-=H`lkcz#3?^izT z!Nz`>Ke*^WhszCP->V9YAjLTBEiz>9c;O|4?4R8ilfVo6UciqS`18BL!vEq+p30=q z2~j@;z2%1#VHebe6BYfJ`2O`KzaL6ACszqks9gTTwD9NNo-u+y()9a!g3sRjwJ!nLrKfy6&v{+< z)aYKuJtyYU*^iTbB>A5d{7wb#(Ih_OD-oYOuEntLz2)-_N#1|;8Q|~(Pn;CBOq+zi z0mmyTgZtp0qV&(42A}aB=ZGqm4y<%QHu*i#VH`j5dq3>`pTBev?e+`Oe+TJ}8+Se` zFyhT+dR`~M`R$(mPf7m~_Is7zw@BuaWtz>cCa+%<0eof3ah$zTiGv^e-z)eod$>M9 z1)ZJG9|U${LQN$9Lxy*;`#}xu=?M8foPU1MO5#w4M20Cg{mi4NGiPuQuH^r57W#Ku z(9vj7S$dx01Dj#O2@_k_#tikqpTWQR(e)ebVwZiQ@0w*u-Io+OjQaUK-(UP+k&Y#& zaO;)d&9s56oi)RcTNAr*UFQMKzUm3=*R|J#@K@_qEmYQN@wQbjcbR#OJ=WgK^$E=- zM6QpyPON<>ue(w^d!;}jGa2ro-@^87TdR$%@yW2ihx66FFP z>|$Sz0V;d#A*a{#`16m-mOxC5^^Ka8M)UTF^3pi>-(=BxG-O-7)VpYat}1CfEs|#E zvo$hLx4R*yI<@c}e;;Wyl>DRP`tHVafJujzhwfe~D>;T()}_k1r1=+gSc|~=ce_m_ zs=+-0i4P?Y`)a#SZm;&J7N$xw9XaRurEtD%#FmZA*j2xv4zo~3lZ39BD~0u0`hnZK zl2^ZjnSVf{BJVopjF2zqgd>CIF^dIsLir0SOr2}IWgQ&Vxh_3=rcs)`&-83#RV1)$ zYti-F>B*T~XThPiNHwq=JlMZ3Y`;7SNg+I@kf*CvPGUJDcJ;OE4QqJhw{L9aY%Tlj zZuQN^+a*mNM&;vMf4;X=ipT(o$I`rTNL76E?zIqooMt>)5bq_;XZ6wmFo1epLe9ZAKE=*?esxTa*SQrHxV2pG zbLaVOu;2L{psKpnwk^s#xjvA5yvr6d@HRIJKXy4 z&6(I$L$g>@p2ci6~V--q+3j2%y5&WP6 z-}_m6f273kHHu3I+>gd|yP?C|3$fu%tf#@XOzM=!_^5!h+s(?Bzb>@(rMT}A zwQbB-u-%)^r;jOS7M4Vr2>VWX}S-t1eUhR=Ct2t;=_rb-9MH~{>TD#!pF^jV1u^BgOHqjJjTYOw1g zBW3fYuSrxz!@EUSJIltRwT%PHV8XCh(-Deh{QB-QQM`-U8uq(bwClO<$so=oV^oye zQiZxoBM<4Qjekr!F6ar9`U9)Dt6w>e9Y ztmlnWzk@1Rtd1?<^x5p-T@2J}vcj+58e1Mdy_!m1n+1zkp_VSXsx^YB{pzfv8O0J7 zb>Rbm7;dDrTHD2Vn+S32ylSR)$PCEp>)~8ZQy~Hw$=lP+ECy3aYaNC=JY8{D*t{o6 zlI0f*>UU!7h1n|>^Q(K$`=5(&88VGt^VvoFB)atF%lXRDx-->+%C8z;(kI&FTDO|$ z1oB0IT;zxP&gP&O{|AQzXwaMIL@{<2W>%@OIQ8HIc^9eJCSQ+gd#4s&w%#T5*|?@P zAioK9p^EG>MR5ovw(kwwx6-JKx=je?Xxp2Oz{-^qbHBYGw7apDM@w%fdW(q|%tw_L zW)*B@COBM|n-T-jvz(>R9P+hp>uaZ3AmT*jc87 z47ZJk;af|6)`nZdC46>%O^@TdH-O3(s+d#B*?pcRk1?F^lQQ`GLlfGf=0{xuj z^<;q0mtn77A7zQqsaneMp2IQRY`@j(jL?rB`UEAGFnZJx)Nu&SBscLo|8g|=cc}X= zOnetW*1T*eUAkNP;27^t6K{-+P2oWt zRTc^6eCtjSMGM4VZ~Y*uPfUOda=oDs&wDheDVD9ErsxF`Imq$QVdvgU*6GpmPbjCe zSra>9PhiF~@k?bR9L-w=o1#dA5Hjhw4iO>vdU0|h|Cm!BpW|%jT5&Qqw+!d4nj(8t z)h6fHAZoiBd*QhR$Ex`Z7&%Vjs3Q>SNnZgK;E)}7ddT~)WJ?A6ilO65B? z{*qzA$KTJ^d|+wjW^^je2ahqh}AB8FDve15cR@V1wcd{*x8w^-1`lGE?~b|&kceg zMk(dKNF`*)Yf4RSLxvZj_{NQ4s)5UXuQs{@gYGVj;8W7L1XOiVZ420z*@f9`Toan#S<+U_h7bzsK-V7jW z#p#_CetI(Yi+SnY2N?^`{E_3^VCCaQDn@oVz_8G4rLhC07z);gJT_ZFt6aRSsNUvkkYCg738YoJ3RuO&1U%&V*79c%hbL6M{KZbwx% zDM!h>tFUXEBNICx%9IOE8LXv}rvs9DS>Nr`L(68eTQ(w{=HjcdYv8-8VIrr5Nwj^i zDD_<0ex1dq`G7U5#9RGrq)Fakz)jfeARM@_|Mom2(*e6KsYz&57X&3JpV@Oj^Oos) zfd5Dj5LKO+*+4+_Ybx8geU*IOd=l@yA&5mOUM`d#Hj9VxECy+8Ptonl16JgQxF)*n z-m_Sx>gcb?S<45>rLN}EsSy6*taaTPpR>2`y)OF%)lS$5DeTquN=N2099?aR8o1R% z!x|Bn4`}cxKWjC~pyzUJA0beZB|tWmEp31e7sIH33I{9+7xC{A`F9SxQ4$bQqKW6T zssTHwPMU!!1Fqo=;Xt@hwc0JhOZ}yOcjsj!u3zV22Uc;^ zOVyeDdl3Vrbin$4qwrzF&U0j>D4SKCHs@u@i3~cnsuB^oe$H17IHj6$#u9ebQ=uX& z*#jtgKqA@pbcN+?XRPH+)UEDwMyp){J-O=NMcMFii-TwfZRc+hL(&ig-CC66HhwjL z;wo`V(GohnoKR%+N4+TXC^2LciH>GFK`@aZs$aeJ>A;C3kO-Nl1V&%M2N4JW|CjXd zy~Yy@5!&5eMfVqs&ws66Rw?Y2$5tSnkgE)k(S|HZIK6FtlV*@}%k}VLGH&HUD^D8C;oqEL3 z)2ETW(RF;~2B!AD1GfytmVl_Y1hWG3AfuHb{m*X1q~*sVUPW}B^^WduoBS)Pd+>G} zo)7?ZEIRQ)a7_C(y>;`G7MHwr&id^Y?q-P{WLV!dLNUIKn#*{|Z*V5s+g`|K7Fq@g zaLIjphyn>r?3~0dO9vmb7p)?DY5#+s=L5Ktaiy#U(Xiq!e$-f99j(itQ6&KmAY^wv zn%NA(-MdqB=BF=+)J)MVw~5*fu%0z=`!xFGtj^u=aE$y~fTp<%sGe)@!{B`0Urgu} z!40^>E(8KCZOFLfF;oc%bnn(Ux~@zgVljKF+iJYNis;?<#=tM0LZNcF_|LtTgSM7$ z0BPISi*0Niq9IbYh4C@m=;Vl?1R$WinKc`eL79U^RJQ@EWT7>-&9y_{@G%*|z~AF9 zEKW_=?f|~5h>ZShlKuw@VOYP8_&jxf1u7Yqphv+%%i%MSV-WR0-*t>MhnTN5O2CC| zQP`axUNaTSbiYqkLTt4JbRb-G_XL_}UWLuqEzxlj9qbgg~|W%zWztVovQ~J7jinAT9-lQB-z$u*SUG)gdEKR(d~c=O3dzD$*@OwSa4HL)GY&d8yUh2Uk$}lPNgW0`T?w)NC&!BxVd#2h$>yhi_~E~ zM%dHl6mz3?N2U%8Y}}BPy!CtL{Fj`NjFc?Qd>(()QLqjNe39ua0Ycj(q~yIUwdse} zK!*D58BV!{-IeJ2v>K!=75~!7l}`fzqXlVz>Yi7&^H3L_jWHR{|AUwiw>2A=+UAwi z(({iQFID0iyZA>gqeKRC@~c>z18KItR@((mhq=9~k9TYeJcS&1Qabp(d|e;JvAOhSJ-_t78l``IWPAon?Ud zssb3P?aj`~j|&C9niH@%2Fm1b1fwmPaGSCffZRe`p+Mrx?YUZ@sQClm(PQ!Ep*qYm~n{>M{cBgCJ>gRhkJ_yAoB9TPraR zKJvp?Sm7Utjjp0q-sV;rFLAF$y@@#E?gIBN>s}r6#DUF2x<6ex4K2w#|39e%0nK4y z72Mci@A)(2G zdcx879CH`m*_MW>^r+jBE3kuF=?c5x>!Gu1%|z1FIDOMVRr@h^Hm~hEBe|iM{pvl& zRzS#+Z98`d13a~~5}=fk%l=xmm>-UyYO}ijY`< zv{xnodtSS&o6h*>?n$zGz#h%)kOgQ;VBUvWp@-z321IGU0c9SoR2ZS2+fGEM_~QO` z__rRP9bOu*Z?9P*SQ#L*=;c!I6qYENaBBSQTY01b1~OedcP&ILf{q$dE|1iNJ)E@_ z7`=BbaFLjR6S?`S4o7v=hcRGzV=Bz<>*SBo2MXK6#XslnewaN6vE@gt0lGUq=ti&n zJcoigNnq1sw++4=)OEJJr%>xEcR?4@jk=y8_&Ta$sQzJi`L5_n8{?RZ<%sx>u$O2F zHQ`DbETQjXB+$S&@&t+5NIHT3ZNB!#UfJQQwH_IvQaP+&$}gtR7oS!zW5zUE|5@F) zhn&GIcADP+EDMb}bg<`TTbUxGcY~PB`j1*-=CWRdUWa6b`h1^163GoZdf|W6tV&E# z_E1MK%YYHw=CXJ=#6jX@^?aDFJ48tVT^?sSX$Twn5z6Xqkng!bp$#(TSg z)ecXO*5(@kk(h4WACKc5SrKqAi)Sn!&6 zUv_Gd#|#0{q@F9i(##3x-YX$|22SGhX^~brQ%y0DiM%IslYyGZY9DieIyjS)HapEf?BMul zHEx{vf{dPp%3jiLDhA>hSG*82$al-YWs09HXmlMDdT}lHA~ztU!NWV`t&M}g-7r+M zDV&(6+yNCk!xeZOmH>+tkx%Y1kv182ETfEOTB4|h6R+jCNNj&1RR3^o+Yq~OOPyKa z8frRCYf5ei*d4^2-XoKT7f4hD&pXz<3aB5fe|PmnNXc70%QQq<_@0*x+z}cfxv6dt zFavIaEbFV{v=DVA^3!H9pT3jea@Nob32_nx?esG0X`{n#;J`J=Q6bH-y6bHtKqc*9 z%=GLWD8V7k>BC{3+fd~!=@H+$J6_u5M2@;$*`NvF^oHPt(q z#Z2Ismdw1ePkd8rYN~FO8@dY6BJX8CAV^1uxGw#3Hg)kiAXDdSKhGt)8q9VgZaa2q z_#E3xT=p_ItIm3iuO24ywPLVq^aH@X&ME-$oW=Pgft=p+iu0degncd3v)ytN6d9#z_2IxSH+dk$- z0RUKSeTq_hf))=#3dfE>(P}a55QJWcP~;dY%8!0CVq49=REr+3)Hk^jFzYybA%yQ@ z^z`Xbjm@!Tc=^le`9{~;MSWE|rHId)4U@2r=H^+SFmiRA3)@&g>1(I>c5TO{?K7BddH{#rd;V#Rqu7S^aAfCYZh7$h!DlAh$5Rv z9Xl7@x~?FmHN##8;XgNV4vy&oPIlCami*&x;MLg%;s?G^J%=a<2A=p5aE2?fQ$`bR z7GgVbKDCBZi_m3}<`nQfb1vE~imuI#G^>P7aw1v{8+)wRASf>o#zHPLkn!C%BnsTI zQTkfj;HFjRHX869Jey00jf2=*N@DHmzUUd5Ty2aArNl!f#R3zhV;GPiq1s8Z0t=m6 zGjCM1(gQXipQ*p=$m;vdX!Zzo4&YUgvSBV4wK3G5LKlQ9T=~+}`cZFu<(boL ztU3QvmwHbiox&EJHa__yv@R30YM-A6@O(Fr?3P;m?&Q=IaJ=Qs6YOO%-L7?!w1D97 zi?wLU6#+*V7Co7;YK6Z;Oz@bShAzL?lceLOJ{XI&yGzJ-T*G1USUK_SnS zlUsCM81Nc(CgGa((gJp`(4w*WxFb~{J=JnOo}%D3w#lxeF}F$3d9zzz0n2$-2gum; zsc_@wSoyDk(?P0UlOzy%FJ*eJeTAvqVlyu>&`Hyr($=53f8?}PqyT@A z{|nI5K)0lZV}a{ub63KenKsQ<6Ae{meW3R8vRj@e`s{3)w67e02KH04Q082y+aLg> zZ`o2&zMNAqdSN~@IZ5qY$>8McMaTX0F%1b^sYsXL}t z6_{$9te8V)bhg~tPF%UDp6L8Rr(tgNj=lt01n+WX+ z9K_XxOHIvyw88cydHIr|-M2QzuzpuLgDKA8)@%kwJV)2bjfNwH3DwcNKBC&=qYOI? z+hD3RE{Hd5HG)&u*`wPrDe_IerhaF&rn|`yXdK0?P zWtJQZ^MbS+Nzi2A1fW5Jwa5^=PFSSo5UjTidc+Sjh{hc%{<>u5DKy|qH#_)A^Mto(F@-Y`Zb%m|Qn!9Z+ zI~KIlUzp0Md;%UMzuHKR$6NX4(L|NqVa#a8i=BEvY1GG$d-^`p76zgk9PquNlo+GKQ907n9tpjo6^o{LOK-NslUfybIzqIr< zzA#Ruw)VR{@s8=^KW$t|szQGa!Z+>_9q1gssRbi6ii*|N^8r|aMI!sTSSXnAVX0<@;rPdM{q8&;f+jwF} zN^^F1!C{4zgyDv|5Qs8{P`gYPOfQ30YOEv7LfA}TCvZFFO~N1KH6QWVtxfR4Yw$Ow zjOI5ww%J7lNM+S}c#}6fA4O!ks)Fh|N-=@1LyL`7G#KR4N6ZoDoIK|~9d6nFl%>)< z0DVKt&lWu0v&lqTV|PW%Ee{x#dD$0bh9D6`XE;d!++5L9lRA9cw^^ ze)8y|iA%N4(93ZlV&;V{_kzMeHLa_+P3q<20-EYJDy%OrJ4qOC^k<4k4PSWJH0SHW zmb1N@yxS=XV+#3FiZEayzQIG?mG{`X1cX|e_G^}9M|Y;(PS{r;Fkt!cKK;L#aS?}R zTv{tR%>(d}t49Ny0-!diX{1}lF3`7DI*9CVfViWxCf)A4Xh=0mjbGb3)g`;s9m~rM zA^D5;IKs(-QA1E4SU6vQ0xCdZkC_hO#cd(NCe|_p23$|n@?RvBX1L1f*StV2Fy^Gr zsjps*9~qAy;-3@WvCs;QJS)^Fiu^Uk7MF??%@F!^_-R5qSm; z%|GZ&Sjvbt(XzdtF5LyA7;!7-7QA=CnfJT4)+D*l#&X{aeJ%I6Z&ZA!6fcO)jacDblS+^PJ6F`!4ga)`d zU6~Zl+mJr#w_lYqP}60SGXMizUHY;#a&Ovnt@8k7*cxJ%*nB$-&+&=7)nqjwtD}IY z6m*c+051C9t(*hnt4}pjHb|l_oXWN8!vLgyI-Ixj3Jki;JgV&$UQBblCszKxrhV2Q zvl<}_*=uV+7j-TU4JnG^j^vw)=F;WhH63{a9>-Q>PPlh9un3y0DouNEihqzE(hw}m zoc)k`w*)>F%r!f7fevJdZgs`!!Ud_Nlpvo}!);9rM^&&D2I_YA+gJO^oYi3}R?zBg z10WnDtc(*_|72nTH?vg7^e#f*HSa=b$-;=GjwnSfkjm5igp1miOy%!K+ado63lv~- zH3GA{=l7;ZX=Ptv_MT>>Q;^trP1rjXCLx|-?{%_Cs8yqr?em(6oxy5*Enj|r-?>p} zR&6%*S!<8?S4V5Qt;CD4=0FN+Sjo*mPmPYlo|i8#Vi!SUby0Ahjaas9)S)8@chcly zfywxFdZ=-3hNv)zjXFiv6~*r`>0F{3nt_4V@l+B%WhE8=u5wH8Vb|7$#R8RzK1=Om$fJ>X?YYJ!H`$+uC# z_;RZmRkLx{8PT<0Ox=~OoRPO?Ow#>_rCKB)`fUaOj85nDwv=6bI{6&oLP#Xw5(-#6 zZN8li^%nQe-Uvn$kz5+o_bT;IoqCR}8IVaev_al+D)*6*z?2T}Y|lw#LIWf$wDPu^ z$+uiuX)KEm6|Uq6e3MLyEkNlL z^>{G5klDnt_#CYrH3Q9&$hpScm2+=#cg~A}%t%tUfXn5h4td#fQO$#(^DQF`T3B*X zuJ&BnmLL})Pi+Ykvj&aqDos}F41WG z=3?|l&O+A}PWQKpaY3>)5J*uuC%E1lcb}WlMCaK;Oq?yh!g!)32tOT+TP=suw%dx% z1FjmyaFc=dvh8KNQK;9oH-Qx@j`>Ww72=G-%F91SU&U68S&v_5L?S*@Z|#D6fW`Os z4vMdn16$FJvJ@KUK`;|@B<*g`g)Q`Ty^d5Bk3B8=t)X)|7^59dvGP_G&I7f(O8skH z2aWtm*;HzvY1%?eC$MtkAm*`W*jGEDFkte-MrrE=N5vf_o!UFVHCA5jk?_IZ>#c3s z%F%xp9{fIE*~FWPdC$;Z_L>u+4&p^K-kgN&Z!_;qGQe;EYte)?9xKr1&V)4vAKe>L z!F?IL&|h<1)3LhdZkY3npGEW821 zYLqrU-DL-N^mgsE=A5JEeS9G2u5r@;eCpwsP5k4JNd3_CzAC_`aU$kGi|u;>tXF)O zjAtIzJn++gZVX0)7Tq2#yS0k+B3tPuim>JqTSLzXu`NfI-E;12gGP&ukU`EH>-@?L zO?$MdkeE2IATt>VWy`5B!Mxq_2tyz5oa{2w@NeM*m$aOPmTOk6a;lrMFNOC%8CVAp z)R9f4zLPK~Y7UMvW%K!@n*?}z4~0=NAP3&#%`OJ+kC5igDw_36bRDm(%*vU9{9c(< z$!Zl~@G4~{y66TnJ+truyg%KIUTk?PYb+)N1@)_`2>V`yiN_gVNx1p&6uzBDUp!Ul z#fh(=ojZJVY&*$j#K_hUxMZGRP3X1;M_;V((g6#lbRclDcDT-Y@vLdBkIdBp;F=G6 z^sZiK4B&_$0`s7C?ia@DrT@4y-uhl|0`p;1YHGA_MI}e})@^&4oJnp|_cTdHScQa`8LamzVey;tb zZh3b6=Pd+~iVhMx#@&!A_n75PrmPl&6}HQd*D`E_xiv%?&6X*rr%neyax*^?HwX-j z)|>81ot@2NBApBIbgwBJUeP*lXu%h3xtH~9S#vI}Hf z?7t!5e^1E#sAO+G5rG-{Noz8~&$27ms##>;$W`;_EdcJnv<7KGEU@rZp#C(*XLotG z)QVrN49t3jei2x1VkL7YID%RSCW-5;Ul$lC@l?w#vWEtjpfOFzXesr4S0ip)G6M8u zUo*L30yJ~(fvl+E0L5bd8Xe{aVlL3XvmTXam+l7pl7Rk(gvN~q`CNCwhZ&e*K|?>$ z*z^y#FbL*imxS}&CIgt&ZkN}r13jT<;A}evD<69XwZupH{c?lRGl;tN(qSuxaKT=$ zd^baZ=qt4Ts#D0jqjvCQ0cvRc#e!pD!ELD$UF`Nsg0{*I!L4;Na7$W%jj1`tKC$?# zBZ>E}M+9bN_Qy-2emDjBrGF_2ML2IkL5k+Ofti#FAR!RHLf&A6NCK)0B8Y>}A@r%GV27 zWrN8S*uae{cDUG|nmqh97e5B#nBj2bD|jw3m)g4$k^p6B<~9Dhh)6KmrLqb5RyP%4 zil(5n7c}4($76cEgSSL+HUKyA84dGIBZE{)E~jFuYbAE2&DYkTE^{yFf{tW+Xy`Ec zG;Q%{4O41{JKmVxx|M4di7M13v;?EVs?fw5_SF=k!cdxc0$3U|qvDV#a9CW7f~N8p zt4$0FfStl^+Jy&=%YkYhO1-Se_z=&9VO{$IiybO-qvok|AgHzyrCTs9Z<9^XCfr;D z1;48BlB)#U`{Kx+=E@^EOi#}Io3?^)CSDT!`tCfB=Y=zZ!227?A3f7|ZrOkS7HD${ z!a9}!z}V=FYsa7wgHWx^oUJFO^HCPlIx`4*?`onqJVE0z4RnUNAQO>tExJYz z$$02y?L^V^>m~+lc_nIyYptWjv>VA7}ueCH3YO zG#aADbo}9RR-ik6_&bsShQf9wrj1TE+sz`NK1(j!-P!H4$AieWCL3xYue|B`Q&i1M~_& z*HYy=-Oe}v5to1m_vOVG*G79h$5(k;695!~!e*u597~d*M}&liq)W*%W@*c?yPyYU z7s!+VxF&c?ryWK|rwY|Vhmn4^sXDa@JNVM{Y8&Wwj?UULwlXKV8P8g2F@^|_1~jDt z+I3|nZOMm6FJY3Z0g7+TRQ3-ELpEth3o`ZxCo=*V#} z;L37eAS=HRrx|)8`ZS%7CLEAeyOI3lQ&5*nz_9|-q@{W{KkYHv-91eNhCaJ;IrU^j zFYmPw7A>0>jv`Y$v|51{8O{^!6Eqw58+y+{OCN^N8p#AhHfta}>$J#ns`_1Yw^yOZ zq?(V+uxHB#G8E2}6}UCG*=LF%;b4e`T^QaVW;Y7;ZknwQLz8`h{u7J}^buhyP+u*! z$Lk`E&o(l4tnsD3S{pRLIV8dA0!-~SfKg(l*5t69Dp2t_+fU{7`7b)+$KHZw&oZHT z5gDAl&vN}@F>lb`N56uve+%x{4C66l!GDg&^8y#tYFF{iptjP~eSA`C*=e(z$*l6i|K4~-tT&)Ym z;;bt>SH1d^?OQ03JfwQJGasJS&C02i#c6r&Pa8MFOzRfrAEkFCu%4cdu(CYcnjZNs{t1rU9jP=V%|AF|{P%=UW?m6grD`@Hi z(@3}TWN%LIEAZ}Mdk@b6UugU07}ThKYK{W!JGjQyEcp9x_<;1as(Ecg4i}47dnF)9 zYU7PasOj1V-Z3}KiXW}-q7xW(8T!LSYfuA7ruRKHc%p@rK(^2uJR7&9=%DO)8!KvC z4j#Af(WGM|j^n6Z>m?;HKo#X8+C)vFycmy zzi|+H?mwJe2RPR2`_Fq++F2zPh0J7UBs7fd$QB}Hgk)s%HW3MtO_`b5k#))_WRD^g z*;IC3;eS7`-T9qP{m*rs>w4q$9nW(=_r5Vq5f7U;A3EC~sPSjh2x(1rIM0)rR~;YvKL6bYd~9b`jS*TB-qs1P;-KfTy`d4l z_U}-uv=*-JQO~yev?IwX74A8=!3Mqh;5@qmEdnTxgwjiRUrUMTpPfv3BLLPt^g<9C zVVsXS$xj^zcjS5XoL`EA{+Cx1sVx%iQ8ChiKFaEb(UtV59AvzMO+Goo~dJY-`G zgi|FXUHx_loEd*3N$Hjjs786)bv2BhM%VQUP33Sy6zcuyg{MVS*N0G>Z@F!oNiBjs zal*=q??KhII_)c-F@WyTeyB}Ljkwqd5;}te+o0^>TI``Ztpg3sg9Z~fVN8p?cyvT~U7<3ZZ02!6ND% zk*J|kmRP(&To-|2)S=o$@{2FIXNA1w)1Uwlqnh*)f=?6k9yd0lM~v)od(iYv|BWq2 zB!-ngxf-W~Ll^yg^v|Lkrg4yP!d{uM$ zSj}vuzD#%O(Fpm=LiY`5u-T{jt!Rv}q@2Dt+}TM0PP^iVL8MDs%2X=0Z@WD*Lg^;9 zO_cRIEpfDdN#4st{;D(}qNZc_bM4(`tmxik4)DNyyK+hSa*H#LMrY?Qk1T$-yBOH1 zs(*9~2l6|m`VUHz@&$k2GB{Fm@@OC>}3LG4XxFq&E)Am&L*Qgwp0hx?go2&K^2TSZVeSQJ& z9(zjK7zQbP`2=3<12t;5!-0MlELz~ySIuT;&v!kZP~Gzhnrr0D!(^4|UklU+UJeWF z%qQ*A&kTf(5SDJp=?b$ON-c}AcH7|nzzS;Vy|U>;Qz@#BosjgFHr+a#>`@%MewS)Q z*M%{XH>l)wox2LO$A_%*AO70G|M@{n+3Cbe(r{kd_ngWX-T2EkbB|EIIqB`v^rb*i z_!D$O1$2%zJYSKP1sNyr@l*A0)Lz6ed7yA(>8LHK*?(z! z4qyRb_hjKxfzpe zI^e0FG^sb?VpBB3Uu`R!Fc4PWd11uH`Jxp%eSuK>4F~t#T}j4hBk2#hViFeL(+Y11 z<@6i+$4J(T@$9QPLC53JFQqN%&G_KD`?rb&FRyNilW&ZhF-taQ^Zo zZXQjrWI%D#i#dV1_~u)00|ntPpLt3v?5H&zCmKVydLxr=)o_Gz7;IoBxe)Y1WI-ou zlxx1y`GmB$yd8O2nJoQt{Uopjo)G1x@oK2r>J#_mtX9FrQB!0-$h z^5fIBio}f^&{hG1kbK9BFf8$+yR5HRC)G5h{SNY~r6iomG~5?%Q8<%(T&EI>UqK?7&O%Fwj1b=&j05IzYWU-{btv9g3*}Uea zA~aFjt?w;CoI;w(3QZ;br~|C0zgl20vEj>w35j;s&yOlk^e91wX+6{2lOLb%Qip+4 zEqw9*-rjo~=TR4x`e-S%oDYyCC_4-rYfFt3&F6g*%fEPBe(xEMNXn!8PLq^_HFzgK zI{pP>dq0||pZa|scAn@$4R$4`>O8wchtTW|tIB{uHpxlHA;=@;76i{wT3mVb?Hej4 zhk`q_|9HfHmgJ1RDds1=gm4A}QM1R|NA6pqu{xW1p2@Feb7(|6>lM^^oZpzFBOM9- zGIW0Zg{XHtI&m*mJx05Snv{l?Z}z+0@)- zmc*Kg2WsJ`kb__?*aW30H4UEl)+oRGa_rd`^0ZNxN)+-%pEi#bGb?fd($TS&`uVci z#O9GOszZ%m?uzfXI;U6r+DQxZSt8QB!~EobTbotceVuoZ+M=cE&hIaKm6s%t*Uf5n2BrVhl^V-wVowS zUUh|avMMHh9ec*a-Mv03vp#Xi1IXwf_?dk#Kv`%ox_w7hS;CaU+?WRbl7@4E*=}KmQz~G$B2>%68=@ao`TD+}S(p&Nm3TcbpxRB^m zS1-aUP%^{sLl>OTX`ycssJC`=YcgH7|Edd2-WGX``Nbm#hEUU^_uU}c3!m&eX0%$M zk5H1ME3c7H_eLs>_=}ArT`9%lH%=sc4xpelegSBU#?I%3aOn6+BXzD^A6C8_iXN7L zByoRWuZ730hi7lIZr3IEvFWpgBuQM704GFM*#?cawa~;cN#-mnR2AMcyE|cDWvj^y zR2=VKoU{6&>i7}0I1JFI9#AH5ZPbSPOE`xcG4T|duNrq($$K)KN4fV0xMC)P`_i3Z zh$ad&T3YPJ`m(M5XLs&6fBBpz8MtF@v@^EHQYC9p?u=%HMS8Sldz$IYa5lAt=V%o# zl@;}DpSbavEuOvZ7riAi0y(4KotpO4vra=J2!f(%CldP<%?itz6QY`-nlSO|Lo;l# zx{QEw2vILG7X8rbEr;e9g19crnoPu?$fF;MIx@WNHhMVlzO`Ax0rnCUA~>g8+CGYzs87( zz!HP`)pQFgEpTsJY5Os#B`4{ges0FP5p`z@W5tXq>cI9<$cc0R3qE+5z3j+ldCFf00A=7=MPuIaQ zn`eTPv?n?dNJ(g7U5!Ip;zbXebpj;OLQqGIVN%=>EdLCqVY&;#^{OuHP6@q&T0`qM z!t8rXJeT<}4Sz3k7~NGnyDmY$Y2u-=L>QN1B9TJKNl8ijC>B@U)3PNRan7L%`z)Z@paVPCWx zbW#5l*TJzZ*%IaM7u~cWv&N*nA&4`2Y)W12D`am=-I-9rM}e-Wc;*<(iB42N*>?@* z7T+css}OIx3dODqI1uVknp)Z)_wsh5D6Rc_DQ}Z<1P)Sf>Qan9pw6XW?^Plxnkpcj zh#_wl%u@Y(0Kr6nLasJ)iCw3(7bF@u|N1(yR^{Ln12DgvLke#7Q zH7bXuA72ETK(>h^HAr4CLkQ9=E#=!8LM;_o-!u0h6f&|mw9u}m#`^f8Cr^`v_qkCR zl#lh#AU(sF%6yW}U-adv#3BpWfM}bmf>Z@wpToAt@E(-uoP}tjkk3#5{z$tgp^p}O zx!wH0A^$`n1R+ro&kp0~{(k#+m6j zcL}q+x9jd!8Ng$jczb+{ zSILhz)7u*tAd(n&minO2(N4-?uzE}+TL!wjMsi_dxq*bYGyk1Kf$FA%Jx!WgFyUHn5sD78!FU5XStAh1$Q94Ah=Sv#4qY~i&NshXGQ8>U(6eDyl?9(bMy>45cS}a zav6O_qn1e!+C-KM9JhI2q2=GnHZ>0dBoP~CXM{=!b*Q_x2|P{In;ZSTV$S;ka#x~w znl-sh;2>mfoJHLXv3QN)b6Pjk-GG{f-O^5Lm|jc=3UYENvLDWst=t+WGzfD9!IO~Q zAY+MefRfo6?}ky!q~n<3_3@S{l|SdV=PF&VzHQ73cY%acB#MxK(ME@eUV%ah;{iTpWqw5ZEEK_8EiTJov}!ev18r6+_J zS+%B%NBk4wi3@2qd2(qu51n;(H$o>E4a)MER7$Z^s<^hoW8+7>lW4Kr|La+$IE2ww z`f;)R9C7v|fDTC`TMJ(P91L{vrsdluJzUBKm@O)QiVXx7zbh3r!EjF7Ka4N%W<>7} z05?YfdH_%}L3ow${Eu-!*t#y*3%NIz^kL45!@g>Yy$jzxm`(K89ZM7e9oQ)3Ld+isG3>_S;U}ARU!N_Q&40L!;crk=(Oik<0CL! z;Z&=6?wA&whk6(dQ*WJ;lOkBzwFUkCzt#3Khaa{aiY8Ahqsj0aJg%+tT`Ug@0rhA$ zoPmnPRT`o6z~n%nNtnav8%v0&vJ$--rPP%ff7-EAEIOUPv=};k>%qNIx9J2>!8i6- zGrtXOLv;aD!i+<}24o87ChM#gf=7CAYWtfjQ_yM0-DcYx$~rhRQX*b~C0~6iBeXH4 z{|Uz0r;4!k_E(=ig*m_30MA09x1P%8BR2Imt0cntvV^XjXjGs&H;gJ}x`%x5eH{*6 zd-vWSTDcuXo(tD+m=Sh9Y#KD19XUyhZ>h(gq5JlsS17y707d^O;}Q##!14Zq*C1~g z3w80MkgPe+T4sYFVEiEu`pNz&n7MDrss`KgF3WQRA<@`s{Tp@y%v@k%2nHwe;;DzoZ5_|t?&)#4@hES}mWomZnx5V>l&9Dj>7 zwgLG4K&2*skcHjD=DMf=WTq;>nIW1Xd~dS9y3x}xIgr4OF(_Mce5!d3CcCeS$md@k zA-L)_2#c7eCOBSPAZC|?FzKPH4%qqlB!RR>Weuz270|1E%p|pB*ZHY>?20vnG=Ew4_l;& znfq_g9WPLoyT6ec6nLrQQvvc!SU@ykMCROZpy8=Wmph@a#6H7T`M(x^y`3Qax3@Q< z8lD}u)rfE5Ppz?s9%*tq!eF{2a=j5)fx7Gzw2$>Mrbg*sMmn3D)cu4XCO{><448>&CXvARt0B}X~wU;LukX3gP{cf^v2fxB^Qcge#w%ljv+yCUS^RGs_8?7nhX)?xpQb=Ow` zobaf;p&LsXjX!7I4TaV0KYeB|UbI9a{vWKn>^s4C`0D}>ezbXi_T_;rjp3wG;~Q5U1=u(O+t*0w%LHkMK`*^DhDA$ESdsD%kyTK}RpE zad{bnv%l263wOiIZXbk25?)txSASpUw*Z`$RY2FF1BMs#qC%I`6D~!n1c%nh-@iq~ zi>_4!>Rpz!HbLq_QZNN_?4e|7FOoqsnPX53C8jsXv0tz^T;HMe!R|jZU;5R9qCCU8v((M|LWYyQ;c%Pwtb^UGS z0?3MVhRfI=IKFt{m&RbR44%jV_H{XnP+uu75m>AY>n=R?f2a&}*osuqbv%wolVAup zFzZ}hda1>iU+oVsoy~tThm=~@N3g& z`aF3l7qDt)e)ct%j$_;{2>m5B#ZD~U0e->GN3alX{l?!zQTG}E4F!fYpKn+cuHQVga!NNFl-B_ zwqq)3fF|b2Z$?#XGBMaApinKb(T0J9mABU#)nAV}u1GOOln6gK40|4@{wx{ShyvFC z-aCDgN8o(%75ZtbHpKSg*bwT;Z4bEuDv; zGS2SXz9bb?iy&8wc_KlUUd0Is0xWQSK+*b(BZyZ%_6cA-5D`8)JJ?8ZTzKlb6^ z72iYb`EYh~UVl?O3E7vm=Ng?%aT28J zlufPl0`mu;Wv2hZ11uQo4R8tHPt5mF_Qn=7uq0he&$I@^ci{-GF#z!Wnv>wOY#|dR zyrWk1;pbEK8-EW%Aq?ky{FM1sJ&H9PCh!nH-)uoyyRKSx{O-gRh%nJU2%_$j9&VJ@(ie1jq3xB_SH3){pDE=hyvY7-+z1e(M8G8 zN!#oiys&u7H~saC^-05;UY1o~4Ewst*jd(`f&U$rthP*80;IHdCSE{060w`W@y~bG zfB~^YS_KT?aqz`{Ok&g$VMs&AnqNYo?&P@X{+U@f)ctY#mC=&mQ9Zdvr^{ zPp0?3y}ku)j3Bmqj0gUlpL9`~yAZWPbYmzLwZ_Jmvm$+T?&9EeOVJ*MudTx&qb<#4 zhev~WowBV&d(Og7_{Unx0-!rnf@|{?+pg;xDlyZ=nz}5yq>e9mUK>$cxUrTVPSf}7 zFtM#Xltp9VR?rs4o8SnaDbip2^Ek0}>xalyK7HME@04;l%~;&jTgn%YMsWe6TK4>A z=qIiPYF(4+RM5jVAyBZy6^@l9T%vhhUg-r@jAMhu@O+He(!qOpGQxLb?r6*SG1oi` zo~G{OpWmnOmrRdV)f=7{?oCHMJ2V=VIn-lMTgd{Y#yzKto4P{8a!VX^^ANKId0RPO zxT_i6J1XlZ&fm$}ULJ*CP{nZ69m4GnmCAP(?jhA{C}Y7%VIzDfv{aeywnpY-?{OOP7PLtgVHq%I>47na0g$94n*yRMGFM znv-QJ8M3U0M+Z{!DHw!ImU01;EiIc5+nVG)Wf2H}G8^2hZkhLF_HC@%f%)fg*K6nA zrn3lW)z0<(zUMvp{uG1x(&^&f(07y-cJIHIQJA;8CiSV#QqcC0Xb6`M zlu>XzdWYjs7A|!~J6O_OnA$P8OOGtSXToAiFEK#MLy5s>2cKB>%*d>Y zV+ch;Gi#|UEAINSyQPyH;m>nctjEV zXpDEt4fg+u#@O?GswvL!zWTfgyE*0Apt3KM&l>|)MslETmF==bdw&5naCoS^4cWAl zf*EV-{IoXSu@IB&bv{O0l4qVIxwXsgu&C!t`l|DvA03s9_OEp_|u zUdhx$Iu>`FUEZhgG}V~fbN4F~Ho6`(La;Y9dJQwBAunGzeW);o$JN3eopPHd+s{&J zK$ZR4{C$ricL8h+>pc&ZhAi77=9(9UX8Wl1xJ~tg<%q}nG`PdWg5P=3-Lo0C8k}-K zc)?n+3WK}FGDF?x6dPhY4)^%7mEV;s5OFej-!43zzQ3Tw$$5^MhTXOLyr@uNnS(A1 ziK0aZO_ylLyP&ee*EBEY_3hR0hE-z4IUR0sZgA+KB12oAqewB~Tsz`ENzD_Ovr)8+ zrN#LRnb@p05*TrrOJ7dZdU-ijF_gNSe95qzE1XM@3)^VWM`6JL87Ju5_ z?oiVUq#7C2dm`?JexFuvV=Nu8x|h>X)_;ML_WZ&=lTMGdKQm~@@rU=MzBqnA=hg&~ zgBQ$rx>=Ojh-Qy~{^K{hms8wCTM=h!6`41dnK0O_sjJHoPLa~LOK(CComm`*S1E`3 z+t}20A*8XTbqO9X2OiR!QAE+nxy5H4)@`RNbwSn))+TmhM9PgKMPlE`%~9Mmr884w zg_CeEsX6)kfiW)*_wTo4YB>gaz4*ytU*f!)y_%nNkq>w5JB8MRZN|Y}uz@^vqBZvQ zOpZi{t3E|m+VNtwi$Xx|Hhj$1d%E}{?s#n4+snA|e&S1V8t;mm6ni~n^~c@>Tq-MZ z7=4Nx_I1vh%oG)(YVm4!ycb^`+c&`$ELUrj>wNd~R9voC_3r8BLJ5<+M|v!rVip~^ z38N!lKl8zHV=yU;lmV%63QKrhhQ!CID)#k+yKR6R<3%&LL71T#k7$Q*eyA0D{)UmU zTQov91LV9p?v20A)fIIPJD1@+mgjz7CBylBuj2S|d9m;(9E~Yz#mbh6X|n|fI^B|} z^qPz_-n71?&oc1k5o73{=-@5k6U_>c5_aw&>mqN1ZL*;zf3JJwLfadjH^!vRjK);v z_i@T*q`iq|wY+9mte_C0Sd_SIc;G{gB)EqxhcwY(!^xt7BcQybIor#Zm9xlP=MLn-12#_d`Ljl)3d3 zc@L?|A^X8>xyitmW-zD59cg2`00e3<&-|XhDAi_T4$V(bKr|C8KG&yAk5|M|k0GE} zuot>VIAm)FmBQsbR4nhM#tVc$JK`(1rF^k_MK`sS z3|Jg}Aw^^AQ%U<~`scxB8@RabVTWpLNmUags|0hNnFwY*u%U9zXu;DOxut|wTO!ka5K8>`iN zHK)CeRk@q!B`CGP^Y=JdX=r`#va!c@FRUdBqy_v93tBO&(U z=Qc%Wy3qE>g?81ZVW)Agr1OOaq#Bil1tnj__xeh9UXMu?ePP}8^lPvScJ+2btf`IS z)j2yl{6Y~;F|PpB^Y~1&>9N5l6MuuiKj5mgAD=r1I^BZ%`{&_`}U3$ z^CeZ)%BVRCacFt84(^)OgAc-4TD@n#Hm6KC=*4@r4jSjd5Vt$Wy(^zu2L&l-Cg3Ze z{LMdTP3eUoMbVUB^*B{~Pj9^!|3rPP$Cw*G`FmV9X%|_b__!yBuCasmWG*$xx9*YD z?&`-KBL=P4c9t`(fR7+1*DMXIgw-h`p!+q&}PTqs$BXH2S z&klgwnWWuCzMlaUO07BN%4yAdl+HaUUzM8!Ke^b+QExU{-9hMx7mjx+F8 z(M}nh+EI0Ht|m{rP{iXqO%_ezLw62v9i1wfuHZ(bI>3`4MQfk=ML)P5cex+ z`?SFkz@?Vhr@h`KM66h>eDR*DZU~LV(@8J&X^=7NX7{Avlc{-VLUtTWo_Kw194$`a{FKD~$YkY- z%iMbI7LoHD$+eRV1FozC`-XQp8i_iKwR(+Cw>5?LH9vFcyyebXN~$4R7k6D;qr;#& zh1cbKC3lvc{^zB5apBNo!*(;iUPtIj_W2~`1_I36O{Ya9t`HwkXzzTUldjFwc;%az z13R%_Zy^;(eC{4T&!D2 zG?cbY_wZSj2L2XS9O;CowK2TCAiVdLq#>zshJn4qyC&Y@9{_r2Y}Z7c?ANfL@|<}` zmD8&C$+p_+ryEnL4H&d*cv?hy(HiGWCdQdP*J!eDOd3!eKa?={DaJEK*iW)H+}lZ3 zUWg4xHF2)#n9nSQVuNBb?^lkNgzOSqZ($#yb76{Kq#7FClhVF;58ajXFyRh&EXm>U zkGo9g7$?Xu;D*z^SKxW+UT)`)a@^e))K;G-^q*Vl`*~HX=~a~bu5K#Cnv&HpU}l*Q z7({8O9n@&n^uv(o$4_pF0W(zSBp7;xM{tOIe) zrB}zE6x3dMWg=&bV)Z_Ig4KyXbofL@)_Xh<>Da`LON?$j;MrnKU+CvZ`S!B5x#@Ve z5(tg%pnvtoF&xPWPQU|cCSqqYdmF^9k*riOuoBM5AJjpl_Q%=S0bS?5$B7yV@c^Ea zP~Y5q+U2#xU!t8dRqx8p9+q~g-WP2}%P-IR@#pZFvA~6RF?BUe=30G9@7_A!(Ou#9 z{BKH@W9ddzzlcBd;_TM*THHNs?UuO3Rv?2%E)m|=Z{opvBon(vcwZ7AHM>%}%ghx1 zzfr1p#hXflYl88VOA#Ys$j{v_WZFfjxoRPkU?&-|PlCj2WRKC6?rM2^qXPm`DB%`L zUb9>u5@{wPqU;ZndQ3}5lpz(N^-TP832$d{a8NgEovyKpGw!LD-p~0icQ&0{Yh-S5 z)mKu0*mm-r+;uOSWve$?sz+RIy|?ZG(N?A%2kuu8(2P)YxRM%nJ1o68K8XN{F|YMo zLszbVerkDI2T^Zbz4|P}D(Zyixy|p^)TFUOBybb)-S&O*`-%J5u<#!@WE|>tVQGlz zNHj**bRunL)%izGcx1hu*{z#5ZyJt%euMe|qfK7k+Qd|6XGUiSt*ML8h5sOMh<((W z(9`2XRp8nek4VR;|6EtWFAq-Szzxy(yL${V;_*uRmfdZT6jg%1XlqUooS@xb+(fqe zCd})HR(%h@gC)D!@%gVf@&G<_U6alVCSpF_M(;-le=MV5)L38yP$RABE?A7WD&if> z&WPO^5$$DBSkzXXTPPNjL^T*JlVVIm(xDsTi2@3bBxhy@h;`=Yxu@f{wwN!S;!H?SBZ^_`{;2B9XQtZz2j}zBB5*fc>D~qXSg;oD6>E&g)BbXDxoF|DW&1 z-bpO>?KPTruJ)>S?;7j+fZ<&8M6jFE*rSA=7qe(lo#)jq*N-r7mx3#nNRhQzC7B|z z>oZpJp9}ZIpMOi`0~qlbnP7i<*^q$uR~?)VU#0UEqggF%Vh2AqJb`Us!jT^p!W=)m z)9m}i)hVl`6Ii2xvBT?%Rg1i+T5A5n5VO-E)73r)xadWo|(*Yc~^7?n#Om+qBalM}*L$IM|_)yo|S;h~lX-w3Lg46HzS6@^U&= z-hqI2DXF(Jj>NV4?id_PcY%eU<XvcD+lcVM*Zf5ADR#rivCx5Z>pmx7T0#@zQ!cbf@C( zq$E91LyG}L$z<#YXV8sz&h8>b?yA16qIX0Ferpuws!d%bb)e2(9tC@2_*XnLWR40% zl#3)9Xmsevt}NZpiIiPy?L(vUDfV*}YC#nHHh>mwyGlNe6$#%`%uK6B2yltYxr=Wd zQATKqjQHZE@gJ5YZPkyLbZp-lxY}{s_m)8-sf20Dd)!=z6FkTU1@>+uz?2ZPLrc?| z#rgNg!dD*Vc7i&lZhp_W$udc#otYLRmOJUwuR22L;ra*9v|z^rVShQ*=tvIKFk$nQ z!~`0hMp`&%{^)rY-nWJld!Nt~CJ57c@|Ct=ok$tK+l3@13|HenU*Gt>VTx9SWGeK-Q|W=GG%G&_R=pjW3B`G&zsmKEW#>br%BLUJvWpBxtyUgq&FHI2nOS`nNZHsOpsE+3XoFNx~H(+uwl5VXCvJe=-% zxyZ$EMg33&)vuOMUIJ}G%)9uFmovSMh|54u`bY_0E!o5I9JriNA;DJER6I+g6S7v# zEyPo+ms=nCjSsK)3aT4k+WT}}C0DPq%Y+H?mvmJIa=nODv|H{fXBRKp{=fHGq;M0wc<0!F(~B)0ONOe( z86$?gc_XkzjWqWcTpah}?9Ad|vk?l7ioItS-FJB=ouw}DB{yf)+HNCEyCK&3@cWPe zmC54XV5xT2h~n3f9bG(<9(81S5&|?1N0dY!!S2=vr#GUn%8QFE$k~d@TP2u8I}PrN zXnmA8)}p|Ge=UQ<5rm8H7cmA>E?|}{ig9FkKz^XpGpFFiwyn<Vz1LI4*rup6aNi*GR$t*1wFN?1N+diPfo6 zw`W%t2)_gC$rx>FBnVOh4iYoMaN-%?&Fq^`t@;k=Mb635n<^$zlE|NyCvx!p`B6@Gws{+If(EU;W!}XH3?vs zo~Z-!Y40#VE$xy0@=Ad%e_TNTKwG4g0YTHM1x9@90X;Yv4UonF(9c&#M$VRl$9{3F zmet&9LyXlTc^!1QEqQJWrQxGQJJ`S)kpq}hfCOk3#hU$r6M;bY%Q@o|-5a?OQ)mX_ zvWD3h-nfIbFLwA8%(Q)tK{&WPw1}^%4gpmMS#Das75+oSRm_H!1F@jfmW;GtsimPfs)S}wgE;iW z?0`RovtmlB6H!_sXwMIyw9HG$`=7>!mA`$otj&;a#Aok*dc1%7O{FLj?ERbxPmMi@ zQPFmsc+i!2^DdPg<9XX|Kgj8@1D!{>6v7hI+tc(uUBS!B# zU3S1ANa=re=`}&<_pU#;?EWn(zn>QaJ!t0^vP-uhGO}RWc55A0i}ZVi_8jNX5n(h> z57_Xs2xq+o@zev@tgkZgNM-dS;e5E5S**}HhFFG5*(GlgNPg*M_|s5Ah=S9 z;IJM6Q6gL(nqg=sF;d&xCSMEiVanNi5k&)~u#-a@XXKka5T`nT^i)tE?goZA89M+h z5?2hhK#ByCAsMT}G`#Zp-z&9{72;+Eb8E=F9?7r(7BQJx`6)G80>XXt-+%vIH$V## zCApE>;VKT0Xf>7Mr?%^9?l?X!-*&$cpvs;G3kOviAl^$UV2C_xnE7bG^}B<6TJ0VnmQz86ugwu2-YnXuGz?;|wMd)=}KL29+w4JOqaB_#PH@PKgu+RDzEZBnM5H9OY@h@^FKuU*Fw zlg|eW#QQuBsAb1_g;}})7exj!2i&uBk@ZEZ?IfrPs*_B!?gY2U4)48wW_Lif|(B!+fu-D>&deX>~>060qm&$VH^(_fr7 zPBj7FZU#ce{gw+*h4lct7l?4kEdj6dQF%2aIoq6OhWlYIT^4^FhAs;Si;*&#zud$S zg&+gkV86}}?_!DMuKych!oDTjYaIwM&Pkv6Q%NMbu&mwgo$CHm=~P`p{P3pfX;CYO zz(8H(#Bce!I^48m+?q%@z-!Rle0AL|*34*|#FDa^Z zD8;^QunzH=A=6KmaBc{&O>ITF>JdlXI3f%^4&3E|)1k76q`Cg!-F01sz6RFp~ ztN1K%o!2!-pnjII<(=Yi2;O!JKyv1MG`zM#)FLf%ef5D|FnCMJ{pHiW;<~RVg>dEB z79&2WtE3beYmEivhTcU$~(?Gn-H4Ka?(S9l49AoMmY;PZwnS4GDo&$D=L?ah@A7doD+aQy{mtHSG=7P?)Qh^~S=UsDv-2xXT2^+z0Ez*G93en-LwL`xjE>~? zJ;4KGU&yg0=j)5OdFQ;fP&fb+r_HWG zSHx>xBYuXV0!_RA=$u|nuwqQ50U8m^UEuh{OPVk3Pv3Te{FNtL5c1`aWR-&mCE#b@ z;bgiydwM=thdn}KK*pU0M3`2I+294s88w6>-$rkii5C%4$0Bv$z?WDCCemKDJDAA< zz~p!puSf)fzF{aiN;lGY6S3{4eBqh;1}k|~MV<*S-GCo)`PV7B1?PRiJH}Fif6hZk zf|f@J3yxftX?NoZT4Hv9boy~{7EFi~;spYq!pToPbF>%%13AsKb~FvkMfyn$gbCiL zL}1tm1$L?ufu-@XafSiQu#G|gBoKI*pPqON?Xo?2eI4)~8oozR!Xjz6{6D6)R@Xun z7jt-%~Fq9IBgVYlTp<`T;{qW+Xs`zH)R_%x4aO zt$LCv>f#CjF(@tOHSBj#7@$v~2#9hs|e;RV6{RmMeq-(Sa=`>{sNGySmecQ2h7bb_^_X9w(kvd3YJ;Xm}lR2kGz~i$t z)k@$(MB2y!?pA&QP;Ifi`Ta6E)7~7WAD-J7yq`pn{5e*vaQ2w8@-Qg4O-+-~p95HV zC4qfaUaBUS_`3amHhsCLMA&|*y0KcQc(6s~3^lyRC9MDCa3&DOhu}7B+}JyGd3c}o zy9@mgC1@M2_W?c(`dMI5kU)85;N$wLKzLXafPeZ`3UNw^+PfY_z)kQ#89>eyV^Mva zHz7ADgU#X77DFg@^Yjk`%)7BN4lWU(F`*#E>UXrmVN78TKr}V%hKjw2YTg)MM!Z4@ z*;vb(DWVtIGGAv<#L;#WZ<$mxX$_(_7cVBI7Ivvr4BYm$FNn{+ug-nkCIf5qy241F zcK6jw3F~QGFF+0c>DhIs6IzFxaFpxURdQ7eMkfn{af~jz#D}g+;&4WbA z?~p@`Aow!sWsgMzYw3%A;A=mFPT86jxL8^dUu_rg9 zQG-jGvs8Q}EXEBH=jc54@+Mo7{Y{G)91RP{0|%>@asi~msI8m~yA6?G3Djyn^|Yws zX9xuejh@Y;MpQfTTF0YESt3q5BYkJ{`h(D9s#o-LG4Gr?++f+MFO!Hp4ZM}Jph!}| z1ec!b!80CkoktaVI|&h$l_$l&(@JHfZxdEkjV=pnutnZseHpS#@183zB8N^~#BEGy zh$5>&)a~7}O0Z!Dbl>10HL!A;eqcEPT?)W`d;e~hDas!(MSKEYOF4wf-rAz9-wix! z_ATHcG5Z=#PPJX{7*D>QpFp24;2NcQpj4rqdbkUv?C^lMMhLx=-xkrA#y4AQmZ_Rw ztlFe>jMya&@cUGv=X$S#N}cC_CLX~;X!+k2faR1WdI4>e>#guRDhn>|$&dZ9;mNID z=3R57+-ZPn91VPBFHEPsFC9U#B|6Nhl$Ag~z+Pu0X4#HdFs%VDHMcjp05<@P3^;{0 z&c?v8s$*tWb*^635X9v6ip`gUw$TS9JT~g>GaRMUh}C&24@I4~)l9Lqm-zlP|3`q~ z2RXW?LkF%v#D2^xHAA`FAGkzC4hQI!TJl!01qwM z!_<#AwkU_!O?D9}=fJ7B){0oo>!6HO558fj20aUHX^@}LhYY|9qzMHy{hZj_mkHa> z?RsYr>B+n!xZO)AC3hemsd5x*R~-Kep<3lz-_;2_sOm826t6kGkV8XVa;p_`66&R5 zJ~Oe?Am4>@3u6tV0iE${#O=s--cL2cUB5RtX!LZny^zKcKD{tQb5HI8U`4R4bx zQ&@|LY3+R_GY=%+VO2siL%Fe{MWqNCI$K(~2Su8Gfd7;S=SaE-LgxYQ`!w-hI0#D?q=bx}dF5aDAODZqd#N;}Ps;juP`>dYUWIOD z%lmfn(i~zeLJs-4{FkDaKB?xl%Yn+f zW*S)T0TCB1K8pr`obl3BzOxk92~!)41#!6myw}L0P;L+S@{(SS!aYFKF%Wfux*Bv@ zax-8R`ul4;O;_3>;v^K$UF^|6glL@%fnYS$H?x>Ts`7nGJ2AHcYv)4qqNTiu;_|vL zeE5f;z>o2TranDzXB{0zwuP?4R&5akt!%Y7X*V|sgP8nFL|=pWeEF_CZv&UZ5YhI6 zb7ZEP6}r9v=!#;9bl({31?u~s0myfGt;vvagoRn|DZ$=JzXynH)olYHm65%bKUpyDvWi+2zRbkEOa4zZj)4}4 z&6u(zC{mpU%)||c}V4# z_%|29K84>Gp8HOAVz;e+d4DR2gl!=Xvy-L4-}$ z#%T2*r*Z%82NQr{mrm`JFvYriAo_z&#>Q=wIj9nFkF9_&Di0tDB{8d0UW*RLLSx{n zXlT`;$T$mcwbxt;vgBAJEJL_MntHEi26FljS06%g=uhmANq7txFl1UE0faZ@SScjqo|!sX^#Gqq zz4)o?h@<@S4?!AY5QlmY;_3G#pG# z68D9dw=!OCX2I8R_L~6w(L6;mf^xKFCLAxKX>D$N<6^Z^&?SX?%O$q1oUq=m& zlN?|d!+0wD+JO2J`;d<6QnNrcA&uC1UkRWDkI?iRxfq)(sy2k{w;vmNW(mUM@v*Wg z6iN!gQG2*9MR_@VrnO@6B$q}~nX8Xt7j8&oG_fj%=D>0FyK#Z$_zhXzXrBtULO}#h z>isaMgy&7A4Zk@|pRvu43ojcQ?1fE?t9mPcQ7iEJAd%}*LaN*NtUFC!Oj(|*$w!anw~_q$*MgaW9$w#^BFvU3 zRF^XZc9pkea>U%`z&OIIly&y8YEIO9>%EOUK0gqiKO&Gc1SM`bFb>tBI|)gMN7 z-KY7ZF_azh5`)H{3nT3jG=A+Rq(v)UP@!`WL4+w?4Cj|jg;iiuoqC(6|XfqA5T zpbZ{X6~u*dy6vmeBP~c2o!w!VoYNx-;Ysv5_e;#Jpx^4D&RT!Ty~iK9+Vw#Ap<1Ktog}0ECOZhbaKwKoVFXvP?B+Dzutq;itaWiEP0NTC)uN(|> z=&?cT!IB%YxX3Gh6Ert{_sK$S&YMW-qrTtBX{YjRUrYxLlgKY1I57r6>hQ_=vt0)v zC8owAD;>g@de83NbTap5N?Qc74XKf~wEK>uzTwjFQik!_cA`0{5bYcHfA*C-LcM(u z32Jo2{l58UvWd)sKo=7WR-e@oGQmgk&)6el3B_!l*~#oe85ppXLB}?JcdLpfJ=$*5l>s@XXmC7JH1o#4 z19fT4c)&y#)Gf}uPw?Ez>If;k9Wt-^TOj#V4|$7V-`23th>#|STo{V|{lEZGxweE2 zXC9|f)VEqGKVW0$<5TrzGKji!8ne%9>ra}ux-8^YGgfu7%cO{?i|RBiMVWEh61I4nIpleB=?!P`$I zbP{~+0)eN)Wr);XA@QnDvwY%FpGOQ!UEKJ`U)6ZO7h*BPDzyy`5eyZf$5 z;EEZ9ymg&H?&RPA*{NSiEujnz3Y>>%{c_AB(s@XT`(5XvewD2yFE$+o*a<0QuN8#8 zG;}>T@2W$-$cG*jE`JbQRA=Z}bJavv#u*kQZY368aeZnWF-hAg`wNH%h|wQOBbt@u z*2w1kFIHA#;=y}ZZeD@YZ>ct6NzzlOch;uYo;#?^Y2o@!&gp)s(_K(8lvuwFy2VJR1o9IRoVh_L`o{JE!M z>47XIM7aK4Z7h8Y??q2yQS;sN*LNgbM6Pd=hN*?@Q}+(1Nd!U0++s`vpJyD;|LVw= z1gbTs5qx*499zV>E-+F|YB5%^$9^bPyag0m#a&i=#A2?Ros4`0t1()OR+DT`a4(@O zCdrcc?4+sF3oe?t=yoGUq-S2dh`_^1_Z=hn>u0RG{~fU4!^KSSN{nQpkQPtENF7c+ zZ92R6%Ci%cyvbGx*FN)bR|}3sb$JO3LEp=@KvAfZga^(+a`=!;r~nUF3a2@$-=sb{ zqpzMYX$D~bxk^ykF+mU8S?tD?s3&aP3bAFH&oTd9*i#7M+SI-`#6s2%{27ZeeYop^ zgUide_~9213j<9eU^}+#RDy6A`-vzqZ?rtHM9>*8O?c>COihG0gSMj>+q<#P&!pC!zp+KiT27T3N< z}QUEE>>)s*W?){P5`^g1T;q|%^v-q1BXV7#PR_6w(QC0?^y*fiX9vRW!*nJn5ID(iG|YakX$j<9t6dx z8K$eQ5A!bS0J?u9<5)S=w}dhnp!tX0sSO_PFwRvo0}T_CTo|t>47bYbJ`y#P*Hmy2 z-SfF>WfS5MIbN2TUp57XkM!g3$jaZ{3@9fDzdS!xUQ~;ym#Pm}421yUt@Ax%gwQ|p zdC{V0NF72U5L*DNtW0JtxTrc|EXjAb?bz`oOkMJs$o&Vz_KH5{ey*=~QJoySEtdai zJO9Z`T&iI2O!2_15x_e`dinuC(tlih-r;J%u*l92MN%C-V{{tl$iHn@(K#N zpyGxGdoEp{XGWOSb{X7Gl5HEt0)Q6VxXa6j4kE07O`K#-z80Gg06*$tgh$)T)z!Z-#L9299Xj;(yvNim@*ux(?d%D=8Up%)Nz zuLH?@k)dc6dmOrEPLg0DYbKcjG0%kE^0W{@)ZD2=`d?l^s{v$4gF}5lC>J|29SXRh zcVih~qw*&$3YMtD_J;lGUh)p{zy2%1${f#xJKnQ!woH9pdJ-)gHp5%t*Ow4}>z846 zi4+2pb;a+Zy+e%mt(rtwQ>t1tdV`YwHmXQ4=UMCityLqQ{of)ewd$bghCb?uEqlPV z`QD+w9{j)CbEj~4iqt$N$>Z=c9c1+6%OAC-sQ%Ua{3ToC+3*pqg`Tb&IBnuD|Mihy z|3$MX@*8I|qIY|6xg`CUNd6ZOfJ6{BQ(g?hv5$XVUN29u9DV-Yy1yQlAg=#pfD*o^ z_B1Rg^B+xXJs!w{JyyKF^Bq0*?-vddgq6;jJQH}69J9T@LdbojL>{Y+8moupKUzFA z>hKZY4n8BI>SH1F-Z-5r^WP(CL#R6NAa?y9jS_Ef9Mu%TYxob3y?%jyTXFMGaBy#{ z^}t~&GsMjOuL4HcoH-~6%WBi#Izig?OIt1P@%l3C6_NOLf_^=g(<98?8}bChikdr;! zp}(GQJj1*2-+$kHn$R?cR4Fy5sv@Y=fF<5VK{JUYfDAqdK@0z!I-Onqc)S6 zibEKh_(dRRTi2Mmxnl68)B2?biTI}-{#`mxo`f|fEqH!g6q5BXG}v_a*q9C3{^saZ zZRK8PgZ-yh{|iHuc}E&Oc-516m^t*(mCzg?5rO7Mo!<~r-!P>6`xsV@FYZxG>O zHXeAVyPzpAMG+;Ni#irU48(n#a1;n{l4~JR`T+9|BKzX zYJqs6G;Pg`!&-#R`Oox=wT)hXCa~8jAQtBMk)2vZ*!I|Nw68`a<2N?E`2S@GEkAP> zQh9?18cyI|5ZkW3(#j?9AC=|m0+abag0Nf{}YJgHai5Z#0@Lgbf!~~ckJ35 zul%rnNB*urvvgo(ja+nAh|D5Ii>X7!5S{_YIJb$z;?n3_vnHC-nbS*(@Ef4kanM@Z#A zL&e2o5+|53H9@*hKR)Bc@7CuSY1cpR#}__9P=6w4EAMG7SEy&@99@QTtn3lnam+5P zIzkm>y__T)5AoKKo9(k;TqztCG$!i)hesyVXu(wob??QB6G`kQpL9&t8mIfV6BU4d z!!)4}omGjyLE@^%HmG(O%TeGaJwfQ#9w7gvF#em!d@YNy4tzAdYAmG zPFwySKt%w(gYjw#(o2{k3i=iQWA)en)%5!Hn64*aDT0i-TR5P_m~`Cl|KrY}L7Gi7 z4euCmqFCbBClhw^-|p}=MY?im(Ni`8lqFY~0p}qYiwESp2 zeP<)M;Q?wTiA4X~U=^+i{HgJ&2{FRh)jn!0K$G|!I`Y0cb7n8pE|$;F`gDFbtK*o(EO z$%@ih6I#H=ohrH`O9(FI*RT#xmDA3RN?^`w77hn*Tj9GwMt5{=|D2-W@Ev@`mWLuB z7G(6LiliNHaMKsLv&Xi`rL(fK$(f#KPic19uv?f7hN|@~O#T#W+~yM6IP#L-&FK)- zKc8ZK93g~YIlG9f*C~8l6WY7k-5AuN7(Q7*?FUqmAKTBsl2gLo(tk!=ePZR z_po_W-dWv2RVN2`#4?8ojtwj+5BrQ_u5mftn~D6=NmKSCn{IEZCuqccYZgLSZI8v zrulNNnaO5fi<4~~4&s;De9f7!ptsFBl1gfqk$WrUYoL!vD*SH8)QS>I9>aAKGm|se zoYOjAu8?|k`I+`&ud(>Eg?zFo(##zFraANG&ujktG8n({PYf{vghEqSpilh%94H)w z?IGznV>tFmn z&*j%?OCu}kC9nD;7?@w@M4Xgo6p*>;YaTL< z;Y|KEZLWY*yzl=T3PO;Vh-=kvnmxuQV`a_h%^{QMQ^VJB`EH@NkRm}f(Ghj7J6S;H z0a^0yp&=%@%|*$_uL{@WiypQ9Z!c0ruk=nedE!x*H-N13t6_oQaHxTlOb(?~s1nfJu&I=_52i z1&d=I`8>>Xw^*}?CSx$&9;VXM)5V-Jcwhd>bi ziqgc@m-g;4aH%89QpIiI^=pf$ywI~yed>Pt8a!}W0lJ#t5`fnb_-F~jNM>x;6#(O_ zV#rpz2RrS(U&gw^mcR83?Sat!>?<=+`kaXt);FyoQOMH($dAR{8mC6@EmG>#9;)`u&w=Sc4j?gR^c!7(Nvv^bI+YVa-Je`yqi{Gu6H-p^EXG@uZbckSC&epriR04=J!Mxa9_Eh*YZM}Z{pRPx%a zIjH(h-mLGY5dXKQJ)IK}a_nF7%^>*(x?nswRoBgzkFj)4&HwHYNqA~9M z;l&{rUh&Qn&UwK>97x9MrRORKtCnWW1Mx5%BkB0FAQFqSA=70k1AEK297z95(K*j@ zgMf4Udt+CB7gq14cX7xuWBD5p5AYz~5R${3hTDy1vX0@?2mZ*o~JK7xC^Jk8a4rhP%!;PfL4Z3x_T}&tKu9sFjYgPnT$6!=(x*25qUOKrkJ2oCNxN43DpF#qyc-V}ooF z8N_S=8<mEa~M~=rYHzEcm*9qE*J%VO_Kg)`mFOTOs0(0Hzr|`LK#GU3lGVuXm_%SPV&E z37wS=?cEwB$%TN3@S)}vopYN9Cl2ekya8BAP}~FUZ((&Y0_1&mgCrS+(#f+%*@jII z9<1)s^8dyBBt~`KD_;m4BX>t#-Cro9-z&#ckg=|;=1gTB;|-rg&yFhL9BPuz%Mi$X~HoY2~R`9^YIQB;M(!8L)Rj z9V8Qv68UF+5@q($T*Yp(q$2BN;WV$QfWPDPFzNE#*75pH z_15UOfm1SSZy+;IPm0qfNCD;piT|(p(gfdv5R2SF^>%h&%Jlph#l623KH{24#v_2T z2+^URjKw49Q`%*T8;ICP?F|iB1Lu3Uxe?w$g?RR#O$+)0jJu4Mfr`7Q@a3NOl^k;< zu7+&Hj}YCXBRz4;pQ{KltPHInA>vOT3!?wcg-6Z1SdcWS?NnerShw_0A7HXSc;d+j z+^%{PnpUtc{~o(8riVRXZ;0!aYky1cBo{_vc#txVyljC9dvcWiH9Nn;1pJ-PnN|C5!7G z=L_n*7C3^rPA){@<|H+gArMF+&)#mlCb<`o*09$*;YCJslC3bn8k(kfvOfv5Q*M7| zh$|25WW1I1>bq$QEkI3|w`fJal1P}=#F$2xt&sj;b|sOnMt0i!Ov1%ZN-MbDB63A7 z4mD=SHnY`w67f@V=V2!9Fg3Kr>Wq&vZa^e&%vEBQ;XPC=Fl-;3O}W&|R7sj4}VZ^7hz8Nk8tw2`WlqSl?Y_2Dt|}OZwWs~E|71bkTPW7nK0jouQ9yV?HZ#on%a4a?;EaU767#NgCUf3VbJTIpXEvjZ z7%f`)i|$a2m-onMc16-?m zC8p8NSoc{-@*0%w*4?5rjXeCt`f*mBGG92cJ!+Xmz1vR!&2YirHNjwIk1h3iW4tEcYinC$w!6w|bs9sY2q1*cJ(`U!gA2CdRo@32_NNTg9m@ zGXA>8mVuo+_B6pUyu)+^=^Y?Wa+<&VwYskINT(a~GVcda)9CHdu8J^~FfxmMM~4H! z$`5qYVf#^0S*YjrRx`>f4hk}N`RMQh55z1U^^=!!yigwpfPrIFRf!*a)E*c%)+V*! ztKq)S1bX0eDL83*>fSDYD@mSWyMWh>O226DOmAEq^sS@^jvwIDpZRu&^@n0}mB=Ni z=simt4HP*;c(M8$J7prH1lC+(h!ix$Jv-&qqlN##02t+k{pMM}UILtfR8UYIrbg_H zsorDu)KG(P?Tt7mRY_y^?NfwH?-s3(`3(>P&LIHx^#>o{LEsShZ3?}DdOtGEgts#6 z;bm>R&fF%&3^ zWF%t`V3uPB&AJ89y`Ymx(u?k)L(($jL zGo9nn_AONSP@NQV$0f}f)H#b_J4+m0-}#|MqyIST>`C?1Ga&Q+s>KHIjn)apz(okV z0u5bzby}f;7EurOgf<4=Kuvy6z96g#r#*gGD|8_>*-PpYL(7)*`dZW4Ibx`XWDxcl zs4=;HCVK>nID68h|H;!+-vQ@*7d>2ciQ-xWx9 zs?Do`^@3A8HN#$v`3YhO`>}j*c`V%z{o2*|KcfpLBrE6=0mCb$t*>cB&m@N!@Zc6B7|%!;Rr5xDc{UrT}wFdL`Zkdnurf+9hwt zORNYa*%>m$fz#Ll@=2(Q?fdqj&sCP9M4D;Xrrc@=A;AANXh+#Eg9*}i4`4@-4s!ZX z3cSmeZ3EvjG1fJS^D^{=B`euEAJcvHLB27*P;ofXz7cO=1GG5&p{{!!3^71h>4zSA zLRVT###&*tB|qTd5gVm&$OQG@+J1R!=n?% zo?aeGfMqTlA6QS?r;8b>p4o7duE|p=t)L|OQ{P!I1`6CK#tGdVxA{*l1n3n?TYtK# z-e-jsVsUG%WZBA}b+XwlT+H$L(Dth?n-P)d3l!cCOkT?EzR%}TKTg4iKbiAT6+Vte zsr~g9R#j-My)yfHr)Mw_@J!<$?b(Vgqx?*_j zEG%AU@ zzFPnvDTIpW#Bg0ho*`)JBv6qM@DsE^vY}3aD=e)yCg1a|ZI7CRtG4c?cXg zq9XeUmc5pS0t;9*u0UpprtIt8$?y3yx-Gs2_-F)h0cwFqH;6KIQnY!rHo%b%@^~0c z0tP7@L473puaG$@w*&O*s+zWg$Dom2oncHcoro|9Q%>Hq*k{8{MT^dP-!tA_nmbbG z9RhXC=_0waa2Hmj%%Pfk0K=U70?j#fh#H6xYU4u=UeGb{&PwKI)OhPup_s?`V^)5( zFP?34tr4w1W2fSw!+wJM8oqC>)Po>}iUd@8U}E{XA;_*$$sBZwO?Y zdW1fTSKrdYKLW}v0TD%802ta2Mh|z22xVA#4S?2ApLQbj5ja*`x*wX|35~f3_LBf8 z2mv|VNiqK)(ChKEzYL-99%Siqb&&=*4U!b<1{LKMMk?mk$jqJMs*x}~7j?EyQJ0>) z7s09Q;Ybm#?9rSaFo+JS$@`Q0rmIB>??){M32wPx%}KHlq9I{!3N9-?uAMIPm3oDB zBhAn6JUrRHxn>J#;mg~Hdx&Kh`ZW&CJ%_D3_e6zb#1Av-i2!u{i*v&o&@5);XI|4^ zpO`p3r69GX^ztMSSPflU(f_DNSO`#-vHKRFwL=k}@_Xcs62Ef>G1gaW74vB;ZF3m) z7ZaX4H<7xQ;egOV@ChYC%)Ui86kjycJKOM~ zItyWqavREv7(2I;UD~>8lEH>SsAw%jw0bW>a@d^xVX}Fr&`L=NPPx6&$RR~}rsf+d zoiS4Qi*Yyo3&<4TDjwWlnzYFvEJ-=aGs~Dw_ZmV#`86EfA=!t^g6`4;GBThm>eRt& zgm*N5wb<8&kXaqk4}cKamWY6hrDsw;T;b##R4}iw9^BJE<6_x&qE^wg8L%pCig)TW zd*v=FkX$;m-5jo06?Eq?RODP_g)S;WH)c|Hf{hDxp3xUYI=!A5^J%*BJtVi;pM!a)hbIVxNMnGL=+5qQFcQP8B3<>ekmhU>a6thfYgG zWvL@cLeLdssz}sIEOL`SplQc{(RqRg=GxTJ7J!=qt#jW-&YBQpktr&2N zmn$BHUTxM_t5rCdCW^ilPkPPmCzM`5H0Zkg?7|K~)HOFAf>E_iV!GfyXW#9pIRy>p z)p^H{j1+Gk>}i9CcmBBqNVktw5fbiLq?c*Fj{+`Ns`Sk4bYr*v{;q{#FRy;17)$5i z$YMIaw_T(n+lm^>tPtHYVC#!#B}KqNegA+U+8LZ9r8#O=SDv=}j;eidnfjVu&T`~W z?+~_E;9Pb-YB6%{WNiC1c{neZsAfxxyYqMo8FU$I%hf7Um7UvGy8BvJhysDypf1@T zuSGcqtejwr?&r|v@*?Y!2??*l$Or(>K4{*A>*C4dA)UjMa6x5`JWv4KWT;u@<6Ls-G3g;y_+)d15AUVcjX6G!i(>*P#{lEQyyS(H_ zu7_B>3{b@(3q)L%w$_Ie0p$U97wmoW&9RNyp(aN5wQu>l`K%5v{Y20p9_ZLA z%H>SyBiE3gIE5(JJ%Ur6w~=t1D0h3m*uNl4J*d7K04C&pKe5|#^^l-_?Almos=~3l^qgY!N_4w0Dy=X zf#-n?XUjbgHG|v_=r#F}2O4+`mT=_h&m`mxc^ap8AUa%6K`z7}5>Th(18e>A(UcxI z3QL4`Y_{ap4s`WVwLT%LN$81JAKl(f(m_zR~yy0vE~Km0t}X6m|m*t<9m z$c~89+L$%IwAi+k3>|B@dSa^S8MqVn*}rJe<9^0=iQBfk}SG=C5`L14QkR8Yv!!(<{sDIsJ_=mJV@2c(PyX z1zDc;<)&_v_bA(A^>N43<2;CRX=mu?SS6}AXI5RNEZ7i$@G1>Tw`2q74%-DgI`~gY zq9kPuFx?PnfemW>&)888HPUIdin)7%<#-^a_9uimq@b^zRT&X73_+{a zBQu>A%x3(-DfOXpv7ez)farLQTsUsKZzF?l~p`A01KrvK9FzM!K zGuvkg%0wTiPUQhe<6i9J&fHmm<(o==fBr@}O{~-TTz=~T2S+BOE(MIUR9UY5@Okyk zJEcZ2YU*6NIqxV=1qSAS+wS0VC$)w52yI&T7%`STI41+Yv9${J;VQ}ay=BV--BQ?A zI_sqauk-@WRX>~=h=o9jD?n_a-SspUz?Yn7)m)QUv|#F)n!gwLNp;WyYh>cSg01;< z)B`HHJ|kK?f>O7XN5K{W^&ggqNE%oG|?@=H9eZV2pj@zEmIYSz7!o2vcEn};JLEG$C2fl!>VBSP| zZADD@aZrTz*KiUEmZiba`(Jmn4!RgqXxeGQjFFM@L`Tbssz2XQy`y4s06Q&*51(fl^^s9(qM0N2}of22L z?pdD-7X%9PtC%#fx&Zd94Pdk1RSx84;VayLXmm}XY!r|HO<9@1V{M4vEwT4{U`82T z*pbY-BjL2kLHTg`4WGI!-4fV)Beu|8vaOpq!uXO>PyGC~R+i>eyMj1Ugs`I8DXTj~ zprW_8g9X{s;f(o6mt1pJ2gN)iuHP*OetLL|w#`o2`HSbgj!=n=0l@fdc8dyMK*=T) z9lZARGa}3Y%E#+tyvnK(1(PeV`VDy*y@i+zt;h9qUp9n8z@76uFj6@u_?Q(c{68=* z$7Wqmm_v)zan~IwHZ>J1}{)Y+g`w<^Rim+ohZsFdv_I=Mw__l60VHU=1|&&r>WOwx+BumFFj;$ zpJChbBqDU;g&irvvpp!3gV+H$e+acI2|2YdJZVGv%=>ij9|0;#9hykt1PH7I)Y_X6 z-luEg+(5t~A@8^!obqeuc0(koQgyhr1mEN#6sS%TaDvS%OQ!&+3u(#f0|TA=@9XdD zUO2*``||rh_26pj0FIGld}B;|rKEy>GsHVAy;GHlA!{ZUk{h?pKOFj+zXIt~P0UOh z+Z&?5qErNU+y^pdugi#$mFTDBGNFvP>;za!%KRmOh$Nu%DWd5Bfu`KVf+2f{tY=RGg{+xWZiE+9NF74PYTKA8-Z*}3)n|YhJ<`hCvSRB*a^vQY&e zfExErcE0r|+PfY&JXB)WZ;Y}qWr43d0Y&H_^K zK%z1@q5%h`B&aJ;h1#M%d8xGP6WnEbK~nH}>ext(3t+e#MMhDOwzMSifqw*b$M&%Z zR^=syO~-8#Uk61#|BO?t|130mXi@$|u%hbhD~Mibsb2Eq=dAOZdex?eu)H?=+JOr7 z(W&4K->23(bW7LsBt8^Q42jI5vwQ`yx4NX{R7oxm@gqW-8F}+Sgw&4tSYK{{-HOl_ z&E*KNaV}e^4)-t;!@&0z(K>iIt;=5owbdg6;sVU-&_FO2<(lXa)}qZ+#-X3FJ2)31 zto82vkjb#C-Z|GJyE2L8|BL}1Q_1o2oR@*4N9F+Zk|-iEgO0GlYa`}ief1l$J1?2L za!4%7%(rimjXWf@fT9pQ`s~+5Ak=V&5GOgv~y{3c&QFe z3?eWL5gAtTd0Bv1t3HM2{FkP}`yILbo*KM^uYl>PT=VS?^A-Jm2NTt-B{V{T#bf+K zrG?s1dD}ZOvTLBm&0K%@w}+GEBXa%&$)fOE7U8!CX7^EwIAnM_fA*9JoEsc@vr~Zd z(uvxWjdl?s`XwH&h_?xVl*1mAnQ{E2%nE`hzP@s-VTO+;z>!&JYwy%)c#ehLpHB(& zU4Yz@4G_~*M_=tBItg5?IIa2AvYXS9=XbqkN6|E!-9hQDo<2?2k%Nxl*V^-RQ)lw; zFCvZ%g;w?46y{!4GZ?_s$EkjM;{AHqVj_i^pz>q{=0Pt5`d_m+}@e5zy7JZYj z`^%oIaC>g(Va|l|o3&s~iRL?Jh>6NfB*({(205iE6 zND&&KyWH`-M{J)fE@IW=ivdD`5G>q!kF*)go}0!Wf@;smhpbZ` z&WqrxgjrZzaCQ}Aa+B%%szYD#^QiN%#b1}^TgwN?kkR3MrKOX538B!&rw$W8l$Vq! z{Ho}pgN^iI=6i6jKH3F;{s{5b3{QtYV=FH?ezm?^l<0DH5%QotXJ71JfILRcHezG1 zg^A`m)5ppVMGKR4ej@1%RS@gf{~|lxenVwN7DpfO5j$9h`=bdr0bQae@btTS%PDAw zU25FwV6e}FJv0nQDUjc!MIx>d+Nk$+Lgua$j+%YxjV;9OKxiob7!INvU-h zM)l@9ig_OgcQirFXtwe5r?d+)g`;x-;#C{v?{08$DK`j<9;?(%l!cH-KMK|RDWx{J zRIw$?bguWdxNanXVWXPV+<}e+I~uJsM=|; zo73y+6$kx_Ya_q1*7&}8#!k$Rppg=ZT`BrBUMiLoDH5d7pL}=Ajrj#|Y}%KWdakZ> z__A~EbTiFstA_p|slY*+v^y4}$z&g`vs3!K@MMkWXSnHV?xrUl{~nQp+#?>{25#1J z%dc03h@>aYT5MTRc7*0Aljt7jUHh&P0AZuE$?EBO2I*faE;a|WN6Xg`O)vzsmE+BC zy7oF*ni?;q_vzsUY4~{L1W(MX*=Fx_Oq1VqA+!6jvF^xc2rDxh*tcaYnutCOpU4|( z(|va!Q}u}f{txP+cTrKxuEDZ2tSc{C^YZgf;cgveH<3KnvRrxW!A(icYx$S;L~WD4 zWdKdvVWzMG%Y8P^o=nI{A3d3gN)C7mU*t%#fALL`EcoE4deMJ3_ddn_Y3Efr0(`L$ z5V-0$nORvH--@p=X*dhK&>v0^@QT;(-e(u8rHT)%uueBDL(E`{jti>X{^XHDnkjjO z;(n@SxWXkH^YmWm9M{m6NUe z4JVSRk{k_~f+wpkbX0I4$e}rCQ#hocL`yloOzU=@Yd)YeKNghlcEt0JY4wF^FA_Hh zUhT;32Iglv4bgst!W^yo>_TdJ$;8z3RedI>pnSAT)BZ8T0yNxDR43b3>k5A0q=}2_ z_Y|h9^D(xQo~%_TdImhRI>bHOGJaS>Jbl8QvPdI+;5doZ8I!b-)~!OhlDe+5pq*Nu z`;oOjJxJpL5oM30Fm2^D4xk;gs-zRw2$eI1bzJ3$ZL-{!&Y;+>j4{BPX&#pHeR7TKfO|vy;BPS%D1=|w2Y!lKYL+7^ zBEXoDmn6{{Yz}k~5J)WNs2lS<6a*uwJP7WP!8io(s#R-L{C&@k4%G`qNhH5Vb4xrI z2F((+)g|^L%x{9ygwPBxXT-U12NB-`oL2zdoJZhnQXR+yb0HDZGLDKm>6OZ+(n93t z6~9I7{8Luw=wx@x`=v}>*NOw5u9x63J_DmKa8X*jg(~=d+#;*y-tRwBPR{i(*y7gx zbDZbP&J5tsS!j*HFbGJJeP@5kRp=pk-2hd8&`gy3NeG3wpaBJrMJPSoznOI!HIOXX)dEB04@pR9nT9ji|Ck z=5p0gSY1hR@hNv`h9})VG6Bx>diU=xT#F6Fy_`P6=yj(FDkwB!2gZ45 zp0B~{QK zK5R&GtE>%wZLztpyY1C9<2*2j#Yz{bnxwK}mKERfxxX~UTgX-dRLV)ir%a(*t)!J-CSG8}}mJhct*S*&3P1{-vE z$3X=TGKP(#n=e8|WXbF6meK9CLQw+5qP#_Wq?7M+$A!1{8O2%FeMi=+vn>xlXMy|( zBJ-(~ce5gFt#a7*cjevXh}@pvJd&Nb zDs|6;dW12ISIfsC`NZ$_H7C5Nsb$2Sa!TdUls<`cSzA8YqE9=NBmC(#&wSVMOv)d+ zJe04Q950+T?DllVDV#O!_QWSqCOZ;n+N!CqSgBJpDGsEU>@ui`vPtjI@E1Guv%3U^ z#N@rA%?>tZEM=OC;lU5(8Fo;9$Kd(mPklK~(`dPoH_1w(JE;VzrqoCrPZZ1H<7t!q zN=haN{u+H4Gv#h%a5y$6L)YtWQIzJv*s0Fgz=d-;WZ8D74x2X13p$$S>VJcL=^mcz zavd|$oP*^_mWZBM8U3=Ady^At2GHGEVS8v>#|t&%8w2w%cUm;$4vd58Fd$su%5-#V z$`7}E(_}h##-Y+8I*Wvu>A5Q-ueU_#F8X*Wz<4J!pYMB(NN9H|y_IlP{X8DeSh3dJ7*{6}aBUJa)%)8Y_gTM5}9}zq&xu%_;_B zhYwts$Y_-nJWA?qj?n%Z3#|$CC%ks1K*_}plAaC zMWyhtz)U}dMHG|9KrDveZVX3cTA{)pEIV6qXBQ*vFPLbKuF?8e3-hjkdOeU zYFSR92DruDYMlP$Sc3_jC!&XZUu3vv&V6VSZPaJ~8Oz&5HD%{gG$zDZrOTo>Tz;k_ z@~ue>OF8(kx`r@453;q%JFMY5c!XKXE#5_+a)T*o&hbYahtC*f^ICHV6!U%wAb%>P zDQ$;ytg%W^FjVCTEKhHhVimbcS~fe=4mtJ9Zvx9Wme_-&gbN_alg7hXLqzp)Pun>! zsCj-;b+{S-L{(xhr*0b)(*V_Vl8PU*Fjq02S*-b$uEmGjO=#-sA-f2rg*(ua-?9Q`EE_5lT; zI|_Z-O1YO{8ntA~^S>x^kET$)pRwKT%$o5fpeSlIN5FJzR}9mXhOQ|-BWx;1ys~^? zYr3KOcPo3qGk6UnM*V~D2z^@WVHQK>QQ_vv7lFv*Cauh4^hcXMn6?gf5K?&tXsR`f zmWBXJNfB$tEOV1ApGUk+x}gzCS$7O;ngF{vkM6t#R8rSNiQ0XH?H{ku@D%2UxFV0I zLQ+jZ_kAO+`5(H8dMxE_5AbuZGTtjrS?Yvd^DrKs$9sxVF!(klwTytV;aIv5x_gB6 zENQ^a>j9VmZy?QW5ZHCObV~X|D8nG1b{hSkK3VK!ka*#4SJb zQ|8vV?p`0=dM`X?U*En%?ChhE6Rvppb~p9NJ^=@dQ>vF*&PCe`+p6|U54&ibb1f*T z^aWANH~m}*ZhttPxTe9u$vdxzRURG+<>%a{7pLbicGmgxxxmimibRdLSOby5mLtSx zI*bZ_a+;T&D8Ppy9y?9g(af<*8<6a_%2)O-Ur*m*zI4OeVxOt*V-z)3KA2Ei5H9v% z92G&M$xF|S8-p$eI%ua=-z6_@u!)hZ-Wup*_dql^yEFMZmnB0&knXe_tj*Ch> zF;Wf5yvQiJqf(<)(=JCf=!`EO^}{5;4?A+NZwBYmHyy(t!E(3vhU?TN)$8V!=N$+5jI zhsjz`ld|NzEadLF_LkViP9HIod^Dgi&c%n>?W&y@Oa*WP>UXKXDjXWzY3_(C>A_U?5AK0lquw6cv1KpJh8u-&fnH*^)4J zr%cln85yK>Ei^whKD@YauVGkysX6Iu+(3Dy0RPkX$?iW=R0SFfKdePISAN(?7AXKI zSNzq~@K_kxYfn6T9JiIUgE2OfbGw{gr%_~Q`lY+RZd4?iQTJaIp?dXDl*Y{9v@ShS z6p7Q}J=Lo~vSl7}0`Ct{g$+VPHZ5WL{&=OsvR#{n<0?Bz?P; z9E5lTvnAzv&y?OQkK`8Tdh}ZTr3iOj=y$wthLBDA{_rB>3Q*Kx$Y_n2vRyKRG7SQs zme%;<>o4#9?&bSNXvT+lnYPP6G%6@9sfE<01T#zo;VE>P1xt@F#WmEq?S4&wLoo^| zyt{LprL+03I~@7$lwoRSo^KaqrN)Gz0Ouo3Slr{P%C@NK9v~FAN|xUIj=zjz`RDdu zN+V6d$yK2=My@QxF=r zdUBN)41k|?mJ9BbSQeLpfN)=nvp)Ir`QzOYW-_H+UHEyjZEsy=>|qjtml-l#@w|*r z&MwHDA(c9(Gmh#85_H~XjP2tmrn8jG`ZQcZYdM3`+V!XKBVz!QPUS3UAWy%fNMd%T zf$=p5iWVkWF&^bINH#y=kxr~f(qHD;dY|qsT}_TbZ^`9zU++3Jm{9iW0ykq(Xh z30c$9+(V}pd}b|f<;jN)u#0o(Zo1oW+QOeoq`oiKxAXdl;7KfoE*8}OIcAla)NI2ulDgRL7Cip z<=8OYDbe1CT4~=)C>W@`S(iM0ac$RH_M~-oIK1ZDpW#(jDP}Mv1t{bwuC6GBZx?2B zbjG>tlBXD?lJnO$41)d?d_v5yVzLsnDF%yLv!^>4zKoFEgML3~avtL>ERTP(G_|y7 z)`LHu_FyH5%*iq4efCl*v$9N-E`DVCLwN($u zKvhjWf7NfHpDv(!oI`2Uz7weqttH*MlYE7a`FVxYe=NYW zu`EFWtnKW_OT}+dVW~uK3CTB!ExV6|hxzh_9?2e@!Rhyf-&luf~A-gi2T(-lpZ;*+#hAEZ_LkWV+*gqI5m<&9kvpEJPzF+t2&=`0J>`DY)T zeRbqvRD_;P!cY*EQ&@xIW8d!vz1zh2P3`2`PdB;gt)q<4{SD*0(cG9s5NZ}ioHiv9)EYlL67 z9S9F94kZ(;;Je%%u6N%wy6m#1)SdFnGcZ7{nkG5FWUv(flQ82El_(e(NQct~8ZZa8 z-@=c7dm>j{1w}p4E`}tFc&LtuatZfEr6S$mf)~glA+X(iJ6?z7)F0!zPxO8$Jnw@z zjUL60S*TTyus9oqCc7jw#f%O=(I4V5tuGy{ul&h7mAM#rreH_FWf*f33zHV}`rCA) znB66W7I);&pZtmTVD^Cs`FfMU`i1xG1`7d_PfKJpqSjrq>{$wgwAJhhHeKGg(e0B^ zwY*nvFQ*strMG)?XCTShADmBy*%LZ4aI}>pS1(iB{qjY3MyHvVA1qa^{gv`3DoZ(1 zDt&z>Ds_A1yx4tknvC5)QHXx*LY1Zat4Z*!)m%TeBq=$GhU;;hQR93UZEoObsQoIv z@+}%Xt9_W$yEG?8Z1&2qxDc^KhmgaozCZ73kr;J#m*-sV?Jn*592q&4AmH)rhbRUuW~G;IC$7yI6^g| zJv`eLEAHqpAOdf%5?n%w%brUy;m zdY5cEkyOV=FMu60NPFN)%IomCj)_LhuEAl~Z z6|>B6tZ0g+w96v)djx&S%eF>_09BOT)Bdmf)a1QmU z7amR||7_`ZA9QRL33iD7(W4{lw*5LjHS=@7*?g(l`2!X-H)dp~U++A8KL!)^&bUW7 z;9XLb#E%)y5nI!Gry|Rcqp51mT@RO%i)NFtZSkiXqOtdSx)ou?xmgpTM zVt=L|sNL#wuius-`|H5r$B|cL^@}W0hZj>|64 zQP|iYL9RWJht#Z*l=6yh{@OD#F3&koL&x)&hg)E#WjuB&KT&GPMq{x0uqULkcbbZm5`i!@C;uNfNHcp9<1#<5vcCJTn)1wF>eb=yAn+Hs8092**s-@<_>9ub@8S<>JZn1Ci;Ae! z#Mp_%UEZk_;9xdFD#u7pY@0|N?&-byEPQlCn~STo(WGQsT}>>Ml15AzM`T_|6BB{S z24;i0y(3oj&JtU_(6~8EwaP=2&;K7^?;Y1<*1Qi(z$l8MpeUliN)r`9svuQYl&&JZ zx>7_+lpcCuExUsB8k&mqwp8gsS2_vOLPwVplL%-+zyw0xbEBxczwh(@;|D2s?sLw} zoSAE`nPJ}RZQH|-W=DD&9@G=fGmS+PaAa2 z#-|-rihb)UGkl_%R!1EF0MMOT&-Ksk+{O38O$ArVAMUZ1KDEr4UpYNq@q#Xp1bG?L z7LBC|O3&BnSuEs?ViUYru_V1lVNR*3L=?e5zI|+aqcoIBR&wu8Nk)_dep_WX7*RCz zGAJ(9tYK^@06vx@$RCu7yZ3_Y&*EgY9)*%-gEIQfCxO@*Xjoa(pC{-^EXFG+NlGg= zz;f5tt+n1Yd|j+iu%=a(9V=y(Ru=qvu&KeYY-9AmhruQFiBtAf4Ov=Q2JGwTMd?XT zp3}ay4U;A3$G_Bh5wI0L9Dv9kf5ranO-v90iQmu6^RKqEsQm+iGGp+ql^QZfykME(3-9HcW zig*pQGKS^^GO*~ew6PTX-zC&P|M7w-CE2v%15q%csn~PZlk0DknarPQ0m23b{xJ<| zU*MHL4d$s=b5VOhQv^uH*~zh(7Qp!qeHSovkVkWz9K}CGBMpq#BFdb7=Djj~#2%A0<)_dxE|~WT1s*LmyjBJOf4-cLK~hZ0`ow8diEpH=zzh#2sUH92j8$Q(9IzO6RyBpZL}_#&n>*iIN8Yv`6l$+_Z2OTT?{%AN6a!0Zq-LLE|;< zlS8J#Atd80O9hROJ(txfv27-3m{vfrod3YK=AcR%WY85HS*=3rS`r=ez%jU~qfrMY zOqrA-dtFN{q(O_q)s|2lVH4O(@v}RT24Y8Kf4d7}`R1FQt;K+)Nt=VXmaVC0*WDPV ztX!oKH*dTFBBdZHu@&c6*KJd1;hO7ptjKxgh_$U9B7wTN)a4{Gk!qPas~kk5u?z19fwH`+gjGOEl(e>#DrX zq>Bd68L^g)x}SsbffTSGEwd{9r#hbX5JQ{B1`gb=oD!c`bsx(mHs+Tl6FQE@2ZWT- zA!bq3<5K@Or@(?5Kdc5C&S>wDUEAJ%%)F5_ed$d9ddqDu1MBL4o~9zdOrA7ZI4B&y z5jV#Js2O}0G(|S?@}D^7kX_qkZKSZH=3A~{wceO~D@Q*u2fV?6PAjee3x{_DK4pQE zwvUz4Sr0?MJd7H53bURGXHQLj`M22Kb!(<{#H0QPvrhXgr?!C$jsIK<>MB&e)eWgy z9NnhDHxY8`=i-tZ9ZZFgx@g9p$g)M5)_n!8O43<}8b;ozXvw6O<_@-~Xgnc~C?uj0 z2N7L%MpIBIt5DQJGATQ0K%YD)S5AZI<<9nwN{AF3#BGTxq6Lh%8D~-w;RJXD;bls4O|ZB?@@ zxSmc6Sy~N|7Am@4YKK0y+){{PldDUr7vj=9;`upeX-8A(d!(y=i>rIt{5lW;X=wKI zdBp4;OM;VxCzKeyiE#s!~MgXNWV5ms3bOSK#{7w2s{hFy6Sb>vi;cp7J&m!%Q zAP^rE{LiBfKoC^*pCVa{K5F#5P)1(!nhL<3^kCX);YYlpT*uC>=D6XjP9sn*T}$&~ zU;O(jEn)F!c%;&va#o@pJ<;Q>t2&D!&8nE)kkehu4RCBn^7T4jpTZ^wzJ^$w$ zCDX|jTD{Y(N}o-;1PaM})$YG=Bok%{`qgeq=u zF54iL33l->U52yv$n8^e4}HfjG-96Gc!==f3*Lv1>E+%*)<^J{vOiYLic((&gBjzgdd?Os|kCb>s@K7Sq4_PRSWUwhf)?a z7v6!n7{sKi;Oh+4ik1T&>kXh4WJk(ewk+&Px z4X0KHTvtKS{hiU4(9p6s0$2#@oOvUlmA2-TPk-zUxW&KFT%GHov1;N*O1iH z)61ykV=hc@xOY-7!!?x-CLpAZK3NV_%)YRJ##xE4<>rRO?NNtnIBy;AWZHZ8#5W7M zO*c3FMX7tDK7Tga#d`RZ9ARR5y>~hRGV4)Aq8Lfkk*M9=4&vJ=nD(?#Xq+#8wnRci zOVEH!7+>>RY$m8JNpc?T8E-2@`5KIiN(ZZD#Ty#1MHE~tz07%_V@ZYo@65)V*N1xr zj_@2!mWAl_nF#i}kB-vg<_?SQ2CF#8k8ngH>n1=Q+LDD+>C@29heF?)FQUP$)wXr% zV4_|Wy|d>pea%a`YcHg%hELv!+`$DL zECOtBo*AX?*8{fiTM6)bSIqJla-VKtj<@(om<6xY=ayKy$jYt!=sNvXs=qk_Th){2 zK%=DDZQ-GygN2!J#6#If=OhRCU!I*(Gu2Zv$ZmpdR`j)m#a0FNjk8r?wfg~SXrZka z8n{-*Df*#w#Pupo4zzVaEqq8iTZ(G|bWUlr`u;>XUzV0L&c97gw~Wn;Gb#U2oU5(c z%V*74Cqm#J%CSp*9bh2zw0RR(dbj3a5ScQNxl-i`#*2ExNx&L3++#X8G1KFl;Q90D zIk~%vWK(1KT_+h2g|hP*>UWE*CJU+BX}PV>_}`%sH*o@od43(5EZsl?zIh^x*0N!y z)-an=)~Oov8G2m|u9{WqQ{r0Es~QuJdB85@#ax1%foaAx@MVajtThjFs}xr&M(+#= z&9GsFv45~`@SEb>POx9%eY9MW+*IuF&U@;3{I*z*a0eO2`KixO;t0_kakJy^&I_1@ zI9zph^c`}5%M;RV5=U=|ff@yZV(Q(j0fpHc}fo2Epa@zVHmU%Q?* z!A;Q)hro}hx!IEKb@4bMF#2+z4?k-hZv+b!8j|k8f{Lrkgb6S_n~q78MZh`j=Kc_ zmyH6djT&d^zaTk1I|Ay?%#Dh$J?1RzRFG;{p>js9fe;Y=8+RVpiI&zq9(VnlRqlZs zs+RX$a$qkg)|^s4^xHDvPuJ9s6%ws)d0EST2pI~$C|3|zD)rEbI}{igi^b%o@H^ZH z>r`m2-zl#h5*o`-xsRpiuTMahMquisg{%00hG%q_+BvNV5dW%m83PQ&&$hmAc82+7X$Yr13~s zb({rKyeOGTEi|jJ%r*0eOTz}6+1Ig@GUQ2R@x~P*bDb?TC;tUt*^7YMWW{Gr zXd>qbb%N93uEF){fA||MGL%5?yf%)2V6t0n6_EFsjlwR&bLXt+ax!PyEASh#=I0{pD^;f>(-HAFZ#7!|Wrw>q@SI^h zE(qQ>IV-Lt$?x_OL`R;0q>&i3J9y<@ud8k41g+FhioXZ#gT6lH5SJa7>quYul)D~; z4mA!bLtMm#4LPfkEey~-2DXg5@PAec@4`LigFaL@nuWl{2o}FYWKes@p+HX;9{yyk z2+Z*{%ZSTTQYa=E7$~<_+9-(KexagK>arT~w!@GbayRe%WP!7YXK*j=bOfkcr`0)& z%~`IF1M@Z)fN!wX=^KV z#|Ly(YRpuNr8m~u_dW5H?olIv0(9R_E1N~&FebC8p&Ld7cg{||unVO~CVVu@XV_vc zdgDX1Zn$94TKBy&tXUpV1k582z2)fX3;a50(!xeS57_K7y){O%U#`-zfh;; zt~Dh)vZOqtP=++nDfbK~^0=9f2MxJ}r!?Rknqbd8qag-S^PPsC*->fSlT;5~<=RqU zFOsZ7d1n);a*~MAPAlCCRc4;c`0NW015_7hM`yJP`G#@$Gm2By9ot;0brLS1t2vL} zU{bA)R4m5&<#`2@4cC6^$Q-7b3e$r9T8|PLB1+zPbLQWUKIU(Z+#~X`f0>R!BYy$= z^(kiVj9*=wR>)-Tia^+yjk8=9f7NJJFkDJ;&JJ9RXCTjG;29RJE&GxPabW=cDwQt2 z$n~zfcDuwo;fT&Oco^;E45$nDF9`ASCyiElY=p(2>I}N;k(@a$N8Z(kvNje^-lDIA z^Bp!{JOQUWeX?v$Y`C}MqLY)o`4gk&g}MfvGDI+TTXU|~L$jB!4(c}l($Wyzg5!vt zjpF5Zz$2PYcaMw8K3aZusOh#7KF5~*cXaa}H30&eaw~}#&{bRSY}fR}ESR~t4mFCQ z)Zv0l8$k)%DNc1AWq$v4z}3lV==xiI2|x|BrbUPB*#L&!$xOB;PM%JCf^De0~h$Q{zxCStlf4P~3~2X9^?kcUWB* z9wxm?=kf>zY41T~k>O%>-LTEes?{5d2BW1eF(JW4BxZU2M^HUhhV{6u(#q*VC>J1% z^?7IL#k?zbj0|XMT(BQ>TsbMt-XeEb4blCfl=t<7aRXI=-UvS1mgny}$L`iv7Jq&y zK#@xS%0JZjZo3%5DE~|&DCo1zSkNcfrV?Iadtx*KGNPB%?@ksUilxf#ZGyA#`00+I zReta8Bq=e(??jU}DCV#5Tbio-Fnhd{ksg#1E^zX&tnXQ;@LUv0k}HGJm;>j zf6X3j+2)J8&F{&>=&w58CdgY$f~g2z6<<47rzoSOtZEXtZr>?1Z=O?}4jHdwFDGPRZ=Ls)Ul)`OeX#f=Sj8V`(yxkCjlWU@aoYe9|?$(;pjX{m^V-7tI@OOi-SH~E>kLX_;jbaiP?7t6jB)9h1>f%?PWf4ZobdxbDNHiH&RFTQf}^8DILAr z5oycDeRe_VC8Fb>Zcw9jT~^q8gjPg;vk17Y8)=#m8q5I=Ddg+RK9h^fR@02(#o0of zTYRf;k*w0$_!$DG!Q3E%;mpAs@;eWUPh4b3`$+ zh!&3F|P4uqIH1<`r`G4Eu@CLqnV(Kr4)om|)B zs0-hgG!|;W-XSY2zosXw3;%Nx^m>uv?-)nKSXYm@nQ@mNUd{f@l~X+Y^n8Y*v#_UM z$nz{Z9|iQY7!r&4=b@Ac;U=7^4&j5N&CNC>8Wj6vPNr&rG2$v&1{$&>XAl!t#?z9x z4QFJ=rddM_K7UDTJY=Gl5V1KTj;)Ll0 z%N||<{RO-_YxN1}UfC4+2%*4WFB;Tes*PEu;T3F^v&e+k5RUYbM}}7u5dAm?ZN0aL zF!AUXJP*JTEHCAyMyhU0?cA)CVi^Dqb;I9yPR+Z^q}n2md1gAx_yjjthmw#pDSjGX zu8NQ$NYsR`EU0|h)KM@c_5NBbs3ZLxw`K(YB$VI(fHha2q#hIK0MB<08F@vi$;tJ( zOe^`+vf|oe%40o7c6THO)(Otr9(f#K;~5-ftQHzatpz2~f$)VV&jmTv^OMldkrO)t zo0Ol{d-$m!47+Nb=H0JNHj)U4jd{dMkh)%sgU_U@m&2#P%-FarXKbz(!$R@I+~^tn zBq;!r)S}9w+*-HM>paD=s144pLPW!zxnG>GYcEzO*AKohb&p-sZfTBp4&%szYN9_j z>MVgf%>#n#2RZ$(T=jZl9B#uq>$x)OTz0ZSr7}~vo8&u?hW)%$y=04sCRT@(ty@j} z!0wf4HxQJ$)W686;E3EK&|yA3tP06`U3;tM8xpY8Ju!oMrp|K4vKHG`)I|BzYFm9h^5I& zt&IS`Byn73GN)O=ooLS~;JYWLvnG9-~iEg3?sFMV-*ByLz3V=!oQMGglj~?nVh>z#IM^ zdL5~{Hpgw=3{99r+VDbhGNzHCc4}?u;OyJ7V%zTiHjZ2KDzVJDkW^fwmL$ryPMxr4 zFLG~w^8V<6DfxNc!DLJn$Bw_9(|tPV7Sb(&LPwYRs)|>u^3+Ni0Z-*4$Lw%-0hW-K zikV8+8(Pl|w1D|hBCCLZ=n0ly$t7TJC8acgtTf0pp?r*?bL{5wii8^KBy$A3$jWiQ z*MU6$!Q25kmh4K?k;oXVGb5yT2d&r{K!{eS)df$rvUz+SPHlC~Ag9w}F~SYlmOwPJEl8c{lxJ$$KIW7053$bj)9=rv`14M-&J&c51dzj7XN7OrVtXfYgJP%Jj$SsvtQee+~^q zr1uQ=)92qq{o>|c3vJw`kF=|s%D(gJ)fMUL5HOd0bQYNEIlB!<8^ni^D+hVb#iEji z+^OP8Tzn<-QNO9M2U2q*gQ))Dw`8|TYh3^^KUtR|_eq`oiuAod^th@UxV*cCS-6mu zN-_4kpugg3^7%=k;YD{es+iw99g&0`SCKs-&5ky93g_gxn^3F0KA^!g291T)jb$-3 zzzSQu>S$)x74`~hq*#Q-k9YvI5kfA4ZXuhcG^T%VzSH{1iDx86{@FjnHu|t4*+&)A z51eO-_O7rfwe=JK#B_$xkn^Gq=NDZ8N~+d6r}h;(l6p~{TrllZ z+r0Ra;1#R_Sk`)sfG0+z^soVI0Wa>Z@zDOzC>ts2*C3aD6O+P?c^48rFKdm%PP7^^m7I_jUxU5T?p~45Vtno$^GL((&XX=o3JL z7%yH+I@$NSB8@WjsOjl*E_RDT$dJw^5v_faMUpQ*s$QJsuW+F|Q3WY64->3wm8`Hi zwPia(2|)&!xB28)rFXMOQf887&>M-Gk=Cq!f~PfgO*{P2gHgY!ok{kM3CZ?PmZPD@ z*J8o$eJjdFlq57qviP~Bv(ZR>ox7J|aa8f<&AbXef`gqv*T63?sZ6nnhE}N&};QlJ2tZF*NBn;5=2462{7P45cTBjwVPg#e{MH|D*$ehFJn2Csz#7 z!m;!8voJ;8+Zmd0FGtN=#;mi>~h{HSIW*=4d-!JnrE#;KXE7NIq>4vOiYIc#&fo(nbqOm?*) zTdL`zqdp!*A7zc^_d48pAJR9jjPL24;D)5cGvhpHqN6wUDpok0!+O>>8L=+3BJeQ~ zf0q^qz3z2q`(XG8!Tm@P{_xTScj%Yb*x=&kWRA!vlBK0|pd6J*R#xXc-62rJgGnJ< zge&o;#H?w;>z%J6!*zJ_V{Rp{&4Z#oV;=cf`>46Oxj5lGu5g@!OvHekM!pA-^XhYm zT{@a>;S#*RCTGWKjjvc{q85>c0<^jz*HHeRmeDou+KY)J3XW|Ohz`;jyCrAX zE{b(x;8Z~ELUjn^ZuoAf`A>yo_q*Dgw3E)3bk3u_)xV&)kaGR5w$!vZFDGl7PK!~f z4y)^42TeNIL#|z+RfVXee0kV-Dcy<_+f7|@$;%k#yn;o4;W?1xIw!Q=DwJToRO?1O zil-e2_MaS-)WUHF=!WTK2J>9;^uI62ORS9F0KvdX&clGP;$jgJ+7VpjlKM5jzlGlm z%jCfdx2BOmiGEZa)c#-Tq3$JTBh4r%))j{FA5KN3gLLOj_>qmVGNxsx6pGVviav(Re0vq=UP7^Z#K4*TC5aGGnNdhyOAzWCaW_fYrW z&g$p3rcmHCYw7-!Dyk6~9q6zPTC2|qksInANEJ~_<<2KFQcp3*3FqIEl2^49Qh zFI6AL@4Gl*K~dFkrMX@V=24CFlEP`du90Xzj*FyE-agV|CH789Yn|>^=rynuX+ZA$ zx#|(K%<(bxXM`7Y#We$s{#HjA-w-fG(HPWBURFpzC;AM)wYra;xuu4v?@(gqMmD;? z6WVw$)CF}oh-16V?~v@s|1wP}&f!x~^Ux}&wZBCNNg;FD4`&eZ9V&Du;_LHDTJ~)m zvY(cqWHTt*_v-4lU8#seWZ~-2h;A$$jj*w`v?Qgk@|&kA{Q>o%6q{De@=_;+PQ3y$ z+164Qa8$G)aB8n?{zu^2WzIoqm>nff^z_{?g?i|0>9Hh2$zEFFtUGs|_ApgR6>$+J zu&-mBL=jJ&$nwgGk;vXIy3+Rb9&%>DmLu zOxW9}OwEb})+9=W9JVR>0O7sGb+uID<);Rl-n6ORN{FZ3m)Z4Gbh#AnCT2|dkY;%eg>OOBr7`D+} zv=gntxA@8RQ~0i&G7WmC(HY_@b7v;tp(ka5hD}V+0TJj>wNS!@ir1Yfjfd}jSZ2!34(rT3zI zlAJbYIv>u%Kt<0Lx7wdjz$w=*d3f##1>_>KRBUkKY0*ADwuLQ@(*NtYVpKtJmr8h^ zx46j?wvU6YfKi7ib(mQb$)p>KmnZ@)p9R$Ll-$@%dA@z-{wT;Lj)K?L^|~LSIzk1@ zK%Z?vnu1To)b=%jT#FB( zuLDZ@L~RGKafMJLJ!uG3VU#+5WyJj00~r-IvG1+Jz?NiZ0i_91N!p(x#HO${!l&nV z%)v7MH*tu>(jAYiscHE1KNtlZY_Lj+yP(aQLdY#9Xzw4){=|9dZo%8-?So;M!R=3! z=TkC$M*g~Wxf6YbBXU`^;wKSggT3Bq{-3c^kvHBQ7m+f!DZ+D=>zfdO&8Hvy(lou+ zgtc$wh{-va+%PfMbrCpr`nr%niF^X)&ELRQ*2AaBYVP@*4iyNNOehRrKV03_>)~5z za!}*_@4Q`5b8iZWC;qbj_}d|?>Yg9Z|NcSWzW{W|9^c*Dqk2|8UpxQ9NAr@5QF;uB zj>=+)ef}V2X$G-cu#V?%q+#kgiW@ypP-pfzkS6I(TcFfDgEFD66h}zL0GVwn<8UELI-4y4b(xKqsNprh#v_rt20UzWZeMD0DX7K$0Jaz;A1yICmBo7a!ew zRA*Z!g+k$2xpU~NdL@SOZ+_ZY6bmppB_XKbh^r68>N{&E{^LB@V0>GH_J+k`FtIs4 z?hcM$55|5Bw(@O&WwEhlUx!L-v0)g7v7P4xWQ;376ds48CEG(If(N(ve0vJ@%qOTJ za1h9F+73iQKGht^IIpAyeFJ?k$9LV9`|ah@0*L&lMZw*^}OiG zU+p)xSN%&}1P;FZ=U~~(w+8N)!!W;Z|M=u*2>{!9yCsb8+t7v0%?-n>L$-`{>)ayW ziOuP<-(PNLr1iU9W-l+?N)6XOVj&*uBJ}kyzCZB?*tPYZ(DN~Ae7lZVTs;20f5X@J z>uJDz#Ip*`n(o{Z+a4uf(ed4ueO==oSjCS|-hAFyzHu*jz0t;}JU{-? zQl5l0{Y%V&&@LS6yYQdS`If492Bj(zL`s0muqDZ99J5=8vGu+G`S*}Q1b{~n*+kIa zltU&OM*1II=WjavL&U$H9mtRXN7JD3X1KPvp~piGK4alSFM&Zxp4`#@+pgORs6Ct` zkG`wk*DwAbWd`5=_gfl6zMDe8dGYddFwO&`;F2A!b~Td_3v;+_ABKHoU)wxm}_8mTLzvt(_>4Aog(v?l`sJOyNa#fn)mkLke2}RxdTbB6i=Alw~ zu5Q}!Eop#MO##NaPv_kg^>~mA^(uD{`S*FD_<-Uod*2NP7qv1Q;z(+Gwc1Z7Qf`Ua z>JGbQ|2EaH&&VE7=tRv$B~c>G;p)hr*TUgsrrG=85ZIB%M1`$uX!FSJYd8kKfso(d z@PBV^8yI2j5c%@YpL|q%HQ4Z$6SO5l|4}&T+ut?INyq*-EfMI#e&JiRdrNyUFSjRr zyQa1t^1sCXn|mgMhw<({lIkj!_^tGE7wqTbfwuW4_UP8y{-sKMdf-C*?HHm2yuYRy zMq=w_zU|ysqxLQg(;U2}+rK#3(7nj+(*ESw3rY9M`syo>Z(h~dP?j!~By^*U+gtc{ z&YFw+L?f6Zegqox&4)E*?Bp$5=EqDSdJnxn+}Ms65(xXx9991s3>gecMVIIPiorDQ z%>Tb%09Xqx82?AKR?B27W{B-ogYUvg9Y9Ikb8?)rvU4yx6#Mv27d_rfLUKSkFRsES z@W2>8?r2ud(aM~~&)wduU!0J}cd{Pv(RgmFY)n!`vXt#S7eyUaf;A+^X(x@L>FnCl z*}wjK^JFZ6oqbL_YT%;#zfo*V?snNnN)Z8aFO>jKaT&dyW3Z&jwp%)edV+A{I<{;P z>9mi&Y!n5ssQ8@4d!62t_*}b-3Mqc5^vlFG9p&13f(0YAzp>VcDRIKy?1u^3R2?8O zPL20L$g{q!0GbW7WsEp>_B;%;{H`+k+QJe3MG<~-8mVgIyHOD^WkO*@hn&AEaz`>l z*${OOzZObqt21)s&EGW}q{lMS-KWC{z3y{t;HM`-G6&p&+iYMaIPen3_YVpF^nu#H zWOM!WU#Ba=5#D6>H+bVF_sdd=8t3o5h&NOZHgeI84qV$fF7-f*`Nar)(PBvI2lG>3>=f_f;0(cZwr|ul)5C3OgjvBEN-`cD*snyCaoDL3WC1FA0j9-6^ zNV2Z+ksCF0RACwGW90UhKo4&XuV!y7dSKNX7A2daN*I!P;rz)Ti%+j@;8vB}QyJwm zyep^0?-*EFwQC8s^Q>;y=R#X{Pt|zOAaZ;2`IdT7FL%BVf>e?-Hp-)`E^Z?$!K(o6 z>9NOChh@~y+M#3dqma+O5?1p~r-x4gd(-sW@VEcwN@v5rof^tumqVWg{IlaW%jEoC3oy4>~mleydCE+ry!MVN4{1vMK{gOfAP6!Bc( zuPU88wCnFv_^_Ld(A6U`Y#aO^vl4D4-AeYpS&+N4*iM*mU1@Hj3pu>!IRA~b4UH=u zp5IPK^AhH}ONJQ%N*|dqvNXb*s2rF$puVx8euY3<0EgwEIq%Z+|m(8}pvBe$fQ! z-lCxy?*5|ps&ywlK7oOCHcDBtY*%m3$qMz0@s}{4)d|#vKalA4>mCZpHs;SdZx}^6M<3BWN&O8m?=20v_UK?hc+|T#p~aCB=J^SYc79EJT7NeOzQCc zc|%QMf(cmKoOL%cneC^h|XDe zArljb%jBH8JAFq@6l?x^5^P+}&CR0`Y((74qja@VM|Ub``vb5juk(_GfKr9{Qo`va zX~8;dTV6pXSQhM3n)j@MvHi$KmdgA(7Mba)IWGoItS2_C@MoP1)7%Z#RP|2yNiFAA zcPgMweYPkreVs0$kJN{29$og;0i-k3y607s*&!e0?A*Pu$vNrg7Z#3D@fAm-kLE3Y z?pk-Uibc85mYpR^6MS>uc$hKIAyy_j!^(5gqw8!%-D~v6g$`VqHq;GW%I%RjX8qa- zHnYD&IVmI~2N6Adu7-&F%l)J>{==f?XO2Ha6-p$cF{gJ9?ubzu0?Wf9aXVg6eT&RR z=Pa+T7Ot>7%FTn1Q8}D0*{5t|QC0EVn4zXm#-0sdZ1|T80Bp(1?qYY)q`z9+^M{oM zEJ4i;nXH7?6FU4KlW{rgt?J$N1^0XLIR?ADaZy3@K{{P^@zfKN?CSNBWH(Pg^mLGx z?jxVobF*^P*NSTFeQ_!gSB<8P1y@&?uGitYL&ko>F}S&Y@d(B}4^^SP=@OG+Pt(u8 z&U6J!Hz_;#+QYRe0!AFyArdAN=n;Ak*HUkp8coiOGC$gK2Q5mv%Y6`>AvrUH4lnB0 z-z`}tiw3Vy~rqf0XD3ycVdeXwsN79?Sp6rZ`GPu-Q?zl8<3?4Grd#2u(xkslJn z$E33iy7_)7vtt>p%_KR?Gc}I%H*#_fu&7*by;)X%R_@>`c)(G=`ZP$~J%69TPo9^i z{dL&NMSBi_!)b@*gCLi&`?Qo)iRDd4N5`SYEiVsqj)U@WHHYg}r_evt)P%;Idmp{5 z9Z&kLY?pmkg!aswOYas+k9TTWp_;v3et%BFvEFXC>^u1axep(gBiNtH`AdVHliev* zV8@@d|B)s(o@+z*+eW8(g@`#eg%kv4e6ST^l(>$P5JEz^Eq=ZIgm_Q4IF)vy&JISX z)SS7g@sq!*2Km#c6(2%IAWz857#p9HE=R=aC`Z>5Oj>XO)4r^wJfHU!=iV&7O5dIL zPu5Jl*XLK(#vJp;!uX8E>K9Yts@?N-!dWy&_e`NdmY3QSEA*a#YJ-pl4o^Ws5H`4^ zz!BMI5PI8E9Dr<}Idi7xl;u~agT$i{2;CnJV!7$&JmlP8fF(adTG0V0RiV~#PGKEO z8CdUE$I*ms;MRc-Ax0eK@~TJI`U&4ut%&~X790`R(xoLuLhl-O374;*Rgt9kw{7pw zdPp3fq{>H7!JbJ*_{e^ecbQQr7eiIUl=D2n_Gp@88*C$96U1AA9y*u|hLKtvxmHEbc5t zaZAw{s>*7uH9M(SH`F{~!w%7agjI^hI1gGBM%P202JkPv#Ih^C8p|_~v3zs$x(kyG7^p+DW;3!d_9Wh7t$rewv()#mu(3Xxy=Nx^$h)iuo z?$G-bkD2B0obuUme=ewp_H23z(1jF!DVPp;TwXkmztvyXG}$~=r>@~*^oQTXn)@w& zRPU}?RlP1Xae^(w7yBk;Zr<8(+K^N`bzF+PLJb-o&v12$B-1P9%EUPorDMEQ@v+ty zNiTfL_-X4kJ$ciFRP`*H;v11Oiv%BjsylCof-oVl$;N$&MO#Ueka0&f1+E%4?D~pU z$w_59-<&mpv?6AfPFGn|aALGej*v zx?Pp2+%6*7eml0eD!7KjQ++<*T9c{#kPQC!oJ+^dn=_lI{tD{PHmN>g_{is6+TWodihmLiI3CdSVTw$(KzL=& z&0NDN8D*Eil&1q}tO*G;L&r+0>hWgGq5mt!@FxTO0g9|;zG#JMT5bRK8~bkOX>L5q zl>fj@8658u{1K{RR<#Umv*CAf6LS2+2$!9v(C<{!KBz3|}i} z1kiorxbkJM()o1vvDIg^rFyMJ^4WYs`zc>q5S4>Z3)I^#qXMi3%sdgr1RObN@w34~ zCpD-Pjnq!V*AytBqITu52rRRq+NB{o5;+yPahT(~Z=e5{H}9qoqOi6JH-%$tf!_m3 z^SdZ6#qMou>-(DyU4m!Maxd`*#j)LnXD`RaX&9o?{rEZhmAN186z;^7r%&t56yqBc zjy+6~AF3U1r!9$R49`|K|1wFwIH37f9%SYiTDjwC{mop$3qD5C2BlB`x>JPmtu_`W zxM3&LUmHm2vjmq_$Ezk)2`{qpCTWU)pRnCrp^k8{a7)WPf9=@fGJ4`v5t6Esu@>C4 zNMs>1%Lr(R@+&Yp3*MB;pT_7tK6;ycf%Eq1e=QD#@V}rW;Zgee_kA}9yL}aE%*1Yo za(y^nVUMETj4Vc+-JR0D!ll|NG!-wsKV>^AJ({0uSXpC@BW8Bpu#29l^;_lD)whE; zgI_q@)jvwI@>#1>w@p5)frHg4I`lbQZ#qWYLFA^bEA|jnl2;U}UjPjrsInGBA%XXh z-kdRbgRs1`d_ULGy<5+U?A}!w7)hefbDn;t8DjTAbw;qds*ql1hF$Y{R=^_E=d;RG z2A%k8!%~}-wvu1fGgWWjx|PX`ni2PsCBmDOv;Y4E`Fm; zfwl755ey_g?#vLV@;v7lqXX}jS13JSUL}W_?-*itOB+sBg0Wq4(-mfaB`Yk-CQ8%# zWis5Wnb0^L{O-s|uhs83k5HU*Qxm^=VHVz`r}mOC(8XNW3yh25;?&Wkm3c-I*}6$> zkqG*IxbK4;Cl*%O7Gcb|Fut@(q|$u-R8<@B7CpQtG6<6oFtNzsy&bd?+tp>3^`T>n z%KNYO{=u1JH<)#h@4fQP-!Q&EH5UO2*|hyM7}$Ih80gbIPlvB=u29ATwSMiKB>S)R zlI&BYf`(#7s>UOohFuY#kKDl3>Em&|JrWv{?8o9f4UK9fqCydsjSv0XHL_>{^V~#u zELa2%tqH%+wM($G+IecMI+l@}r9@lxurn&})$Xd^gF^Cru;ulf!jm&L&Mx62Y?-R= zKxV@Solwh5TyUH{G2lEkaJq4IDF9thbunXv_^D@|eRMe&xUIvx+%)bv8aQRLlY(7( zbuT*TdnSUFP8Y5XvWkx6auy%#sJy`RLoGi?l3A`+;y-1VP&&It4sA;A2QcA1Cpj|c zYusXg0=543q+}=;R_L$z!ia^+V)TQPeY)uVvD+&@jgHQi}1~#Ve+>-RW@?{rqh*O#&xID`TB-`#(K_Zah>J#{&4Rjy;eP@9!;z~-*Ex> zD+#;fn)cP2jL=M8Q$kYzU>S*=@-2Fv*U~)p!|~TQ0%mvl;{wQWUE=Mtg?XKh(a@Ke ziyi>J_ceJJVoJ%2OjBP$J5l(I`VJCd@&uN!&ZYzsRBiHrL#H7AfW{+h@6=!)tMZu6 z2Rs9#yEovb!|3GZSgjP=Ix@p3%Lq-P;it*R=MR|(<0F ziWKhR4O+@_j@2#hnHbr{vD2HQ->x5_BKI&PXGw`xcJVQQO~Ex&#qlxu8;xD!WUq<_tGgG9Gn)!;2AFc>T0~E<`lU%eS6D9f&ck^0N)XAS>|&|Ai<#QA zD(cdKD|T11@@v$B8w!?g;8W*_s(G`>7nbc+XW8wGkRVZc_;ifxuWBicQ!Vgh*--@& z4FFo_#jSP+ipa2FL?xoeP}1OvXL@y`-kWBOHgK@!Mw&^1Fqe*yn&G+;b#;D9ISC+y;tw@-?B0axwvmqv>5w?c>r} z5rtVABymn*qEsT;AlL=o?vP!~<^){3$H)8RuZX(iXrk=0a%yD-1snYP%-r;=cj3<0A(SpxeoAtHIhLkH;g<(1)(o9LrhQ0*a%B{R&(X^OceL#p8Rv_28Dc9YS0U-`vssv5H(P3hD?ff>oH#E4Xu zWp+uQ_vKaLU=g-~L-Pr7fsTyp3W(u6Gznb;x~>d?IDhvne>RaJmaaYz(6G`!Sn?z60?S9O?7Gd5db2;DCFR2-1*V~MI<9?S!E_+BBSOWWRrVlAoYiy_V=HcGwll<*6L5V)I{%JkSFh zuko1|Q|S>eTOA+D1noN~9JBH1W$lu*LBtyplYk+~F{Ojm@y&O{X!ODdF5m+7l)4gH zP#U|D5Mt7cUhyWy2bhp!1M)UD_oJBa1Bbp)b=66^V3*8AS^=V(jkq?11Uu!XZ@+f5 z$vtQM)iP^=IVIa~^A>>P!Q1 zc3|Sy&3HH1wy*${q~w4>Am54poZ|>{)FnFvvSV8uSMgZAk89_)?n`odV3Btk_M9%b zlOkSjX&>lbZk{x+o{2;1(A2Y9rN{S(qmy6fWy*n-$AuV|=rT;o2y{|Fe2PbXZC-Q1 zs^|(rh-2QyJ}^3vS-ku?q?yQ~&5cjakVQ)}@a3O(^)SY@`&m>~S=;R|c+IU*3)O@O z)kC6ErH_)nCyw;>Fd<(JHHyv$ayHZLMhl!hQJNcf6WHA8>1f@e6Cg*FqajXX%=Evv zkKH|eqW`x&1|l*csDN!c1A=mx-(tfrQ}kmL&7`7Z{+1;3Ety zk!gk8AqCP(MqwCBRAW?jrd3NGIQl!y1%d{&j2popA7UCgXKh0%#K7Sj4F!l23m0#G zfs+Nc?YYiLdk(sZz&KQ3v^AHFEZYI9kMkTSYXmgqC{)X3Ar8scbp6eR>fJd@2h{Om zOKoPem5o!{H)|}D6E?JU5p(cCzJK_!yS-vEdr<42PqNt0hjw%k8oBPE9}LZ|xEMV? z855GCMWnmQ=I?kcUm^mKhF(t5Fb7A%XR(5UoQYFww1Hgrc`0dpc2%YA5<@tJ;YsD# znGn}%#{auX^_8qHIX|FBxyCI4Ek@^Mzc>z)3e`F0wX)UoRq?1Cmti$#V>2s2jojU9 z1V|=5^S4LhUfQQ%0)@f#XZA~&<# z4~;Cki@0n2J*IRMm6({Q5NK^_X&EmOfeGTwm-;yrV1T3^ap{&MioP%9v*j?$--IR7 zA8BSs+gYV89Qyad!74HH9EFD#7w3Pq2ewzt1uRj`HGLy3WZdh%vhCGS#Vf|72)j6m z{la~x%b9=A^)E|3r!J>wsAu`g*Iqrt229^t z)M+lpRt~qX#VXQRV9KI4RO#;No{vy=ZDF6?uq%6xgh4SofyAh6o!2vgq23(45_Sc4 zIZn3888=qTG_bk6{rjPPb(J1L81`ZB18*e1(j6i#h1qfBh;tR(t;es$?9>O@NN|II z%d~&BM1#{bY z0Yt|w>nTFOP@}BYaqW+8+G%j4#TV_@xM-h4hDtTYohKI#%YsU;CTu_w{gO5U@&H zJ1vpQklzQRH$~L#f^dWqr6xRAL%8IlYV${B>R#v~M2|!mfcy78#xv_(?DBLjD&_ge zl6=>dz{0hIGS`P>)Ce%%vXEfRmBqD6|0YNoP6{wX4nE|E3Gm%!)VyZ|`@X(k=JlEp zGJwNd{656FFQ2pLx=@*k`;c3vMj<~V)a121>g{fMFgBgh#AWu|z>;8{@1EM~>YY8+ zm>VbUmh9?wNEp13_5IHkx0!6UlmK44&%+xLUk^tMwcdJyO@ywR;`JV<<~;e-A%<~Z zIT!GjwxgE26EoV&3UuMWd4M#gJdL1m3U^m~AS`Y{ZcXZMIj({`9D6w(Fjs=@|3A*& zJD%$I{~yokkWs0uA}b=9AtJNvGP1W*_D-_rsaG#8WJ^||WM^-uX@=}gamWsFtmFKy z=P~Q`?e%_re*d^}JkRyKuE)GzkL&SxToze00n|+&}xLv*1(7P5c-iATpa4F!lk6FHnlBoE}Lm=;FDAG?<+7=7}d zBpgje)?W<1*riJTv(vMnHM_NHa%F+vgi*~xsFKx2|&GwPX z?SUvUl4QyD*;k9c!ZS;47@h2`aco za(su&^GcJqU&-9(t>-@w7IwmvCb2iaQRQCV#Z9z*1$QaZDWP5wCTc#SY-=2ciJ(f6@Oo*mX0RzelumUykQD>#P9B{>r9g#6iy7a#N0m2#}~ z1(z(L5B|B3uYX3-<@@c{Q?w)VY@tfktkNU$S0a;^!U==vWN}1F6 zxG6G(?9673;pU$wD1mppdq6gVxS`0uhk}WZ8@cXOY>oO9pEsixN%;8cWKI_^=-Zf# z%lb)tvitT|inB}v_KQa5PrSsNKp`U?P1nE=`}ZC~UG5K_fvckoZ=sVSg-lXG%9Rym zDkfX;Cl2JHWT#yr%C?tu^3aM0F@8A~wONDlVZ|g4AJY`U=_=!Sy{E3SL0ci3vBWvMVIj!wrYT}QzxbX_;kL<{w}{5QUEpY$o1T$tnMZJ%l+km!Vsn~?OR3Y}^qZC`Oxb-Xy=I8gn{BcTbI zYGNujEH8VCPXxINL?+>+rE<(t!L$Oww_!mYA!M3VMOBI_n^xyJ<-~0N?aH-np9YTS zY4y)oB@@VQ)#(c(4>n}ruAkZ80Fbk=qw+P78|6uyESW6`4@$;iy3E*epG_3?7d>hE z@aEcsB@(-A?_Mi?9pKjn@Mu$^ zJ)stlL??CMRR@qF)d0o+CiS=)i2_O+oJ zw=OkAyy`^vA~I_c4HOnKIC@xGK5(~@y*xwN3eg$lJ|q&hC?OHJ=3NxJI9XrFLU`ZR z-uqaw8X{L?(N`p^f{d0fD&rQ(LEfB-C8Auc^2HQqW|19u|DF?LkAeBRy!WQzieE3`^ zD#tP|j$A$^m57F2k$2=)#`;k2mro10@ywAq`&Kj?A2rhc(BQlgW83*idL7bCa<;1Z zYn2}5?@U8cGK8PL;3lgAh)LtVGZ*9?C2cUhwz%fK>fVyvt3LQ@5|te6k3AY453H7I zHrvuw#2dnxcoJ#@8GMeVzmy}vl|AZz0i;unB;E!4@q7)bbmDw%ru_vI`}uW1*oFT| zc|l+Y9}KQdl3ev_P)9((odxU|Kaa1v*pxXkCyaj0nI$}yO5T@&`RJrBXmoetuHL5- zKXVG*+sL0qtLGD6n*_C3_u9el1?4K%o#JbSh0P4VM-(4Em}Eu|OQPx0COgiE5trmn zolWvpt`h|vBEycRVyZlGu~)gL?g1Z6L6ft`*MTCtSQKc_bTL$jjP?Xc(j_()1TW;l zo3AkKuLc_5Y5NAm4D|tCue#mPp_Tb;Wax+1#jMY)#hEc$xXZ4aF$qq9TqCpyRc;L_ z_?Y-Ir~L(I~EiXA*Qx zr@@`2uZi0dIJ3OlS~MrFzruqTL90-NNN+wbf{ zDiU+iR}?gS>y?}6Oq;H}=Y8zedzq!{>V6UbvK*n*0>Fv?k1%NN%s)yadSw9|J+kw!xGG zz})P1Bwzv*m`b8bjTq%-Ie#P%Y-o-_@6zNf`dsuxN8~C+^nFxekMM@dH}b(npSA;; zs>xqnFZrW%2`3!6)BMU-bltLU0>7qf*<08ipFL^9_one1$mQQe%bRf>AZGZ=h@HQ# z0fD2On~vFs{CKQcANGtYl^iw~8S?pIPrOta++y0EWxA!E7Pi2lM&6Tpz~ynN*5dr+ zetr20Ni^7I8YB+jCa`&!NNiMFeYZB5+GN$@H?DSd*f4dp1*)TG67^=-Aq$|i=%dK< zSX^q#iPukUwoUO%7DVPKYAgIm2PkR?>n5rFD<`(BEq z+B=ZHe251^8N?0d;uDyGRk}+;Evc(gUjoCg)4|}MSQ7n$sd~Rb?dVqp@P^OxY8s^5G6j!LS+C;N~gBv)_j8!yyH^wgZqW&%2O>_@CPWmp8{M-sVV zJzvhK`7xy8j0cKC@TDo+%m;Fq4x5mLBna%(BXqvvX@gjRnx`6T)Bs4|F;WX2mV(b0 zhIqSbZ0N?e#W7+meM}}#sEN*$awMQ$khQIiz3{S<+_SLvWNm&4T@ABq?`PX%exAl(OD!(qIIz1=bvO zFS;+R*({(2CyxEB%jsrMB*@mxv`HdJNx#r=#Hs+5y5@&v@I&`3rnE=X$|Ih`SA{zY z603q!;4hi^CcHKTdk|ZZa=eaR)kkgY_-<-tpjj{Iou+}&k%A|e9PI+x3o)fUokVxD z^`waiU}U^Gw?P|{K(i;=+RG{YUE1k&B`LchRmlIi5K#TAfKfoe zoxrS&Dd#(=jZj2^NFTrM`PN4cFR&mNs{-C)nN}YeRC2?mLlCTih_u+@AZsoR*+FKw zlA5ArPZ?hN@IAgj@Tw-NQH~>Lx~3hoyo$Sg99L;D zAIlz)As`(HLf+dt1tr2uiJ!5Tu&MZ7(B_L5@J&&Kp7@vQ2R?e>id&tEbnstk)6|CkwQ-Hjg#%4?^GH{o3wm+o}aUl&=M9wG4tYTp=!^IM_|7vVHxju{;g(ZXR$(HLur&Mgfx7@kw90}V)9PDp}_9i!qu66CC~50Nt;q}1Ev8gSaiw2=3<*7s5@kB zcbg{OT9Y*pY<%q_$O^SuX>i*AsLkWT*u4)Wc0yoM^V84@Mwne@F!gpto6JwTj zII6BO$UkPAv!n7uT}ee*X;Yv)Nc;1%%@xBvNouyzEq2)U&o?AO;o@7~P^S6zXwWZz zRM~?^0K9PP24|G)=ANjDonl}7w^3ZAsja2Ju4O!`2Wc&KG6;~4{MllwY>nxr8l>I_ z*bie3MMob7yaeKp=M6CQzrKCP1tn1eu9<*pxz4Yr(pv)rl=sY$&xzn7@S5dd{+}k( zfm1z9%qo>kcOi5fB(0vTS=>BAIl`2qgu*+BJzw3`YYi%eW`zovvd5y!cEvG(7*|mE zq#Qm|Ft){%f9hdmAt@IxZxoaf3!U90gWVgDS$&?wO{2OiNqbcSq(R(~8^s;BA%Nuj zv4IrOB4`q*a;VrB!;-pLkAOG^TiS9YJ5&6nHf)nIzt|_#JCG}+SA03^Eg165J^TxH zzP#xaescr=gmq9B%9AH(xP^(-pB>uWiQ`u#{wV` zVFgNb^-?!zEdX?}$p7^2jPVQHD`tO~AYT|Hyz@D!l%Okvn)5DY@I(DkzxOf-@e>A$ zBk6pe?^mJ}=(8h{9Wi0p2w0Q^YNIs}W`XF%Jp z{u7^3R)8pvKV9S4JA>oF{_6i-dQssJT}cKUcD_OlqMbW`Y_9CzrGEhmkQ)budmf8+ zhKhwF=r6j4gQJ-yk1Lu`>3%xAVS6Flgxcdow#fF2eYMI2zOI8o{}2n`#cd4iI^EMw z!tgY%=QYV@*8TQa|1a*LUXVh?AJ(g&VMC1A34m3krRes2;XBA$9|xykQivNc5(Y)u zwZqf97He}x|CDYdGvP5oP<5=1KX{L4F3Rs|Bh>u$dqP}*7VmEh#pe5CKWC%BnZsDo z{4ZV~#BQl_1b&qnXz{If*9W*l!~H|xL09v5elIfz#~PW5A&U8*@$$dtoyc+0u3uBO z|E$)|(By*1&cCAmEVLPGdwWBUlQks(2ag}dlJm2~IKa+}S6l2<%se}b-C zEeEPkZIJ*R|E6;O-*gCh4Y2P-%FA3lM^WG5;oFhMP4D?{3)s>W{mrspk)qas+`e(`^z!1hDL`Gx>NbEKfFb)m*Dd#uAgq&g}5 zpE9y@-_G{4_yjsAi0Nb8@r%&_L2EUrZOGnH|7_uo=K*~?x1V#*RSn4N@Qls)(C!%j zMwNybFofR4Qxn@?vbyUpwD;{0VgFt28!EsMC6a&9ZW7`XFM|Bn1?+&Jy&wOgr(E%1 zh>sUX|8b!kG?V|}BBT(BU;v~Hh=onUCVp*#{vV_YFUH5AWqf(&kR`Ko)U{2*Kdk>k z801wrFS<^h0pY)y*~ZAdq)HQ)ORm%vE4)x7cofwD~Y3f_pB`F@d@uKp}rt1S^u|qaXfR zV1s|l?f;e3bf8&8?m1xtJV}t_#9dqFBIhXB&d^=U+d2~A6DRQcMF+sl|J4wIX(CPm zNWk{ajvbmv^A)N8AI`ktC17{A7>|uC!kDmayV1rrPkxckExPFA>d5ptZFN+m?@i=MD%V0c#+ld3MV-H^lqX{-PBB z)1Dl$f_B|Y3i*V9=zkzs{6Y^y?LMwv2jRX#606-lJDXbZZ}@qe+Dti4 zkeC+bq0I=V@I9Dz*8jgC^cJOonBIYDkkURykWD}QjSJDC{{6c|-c-(i$k(Q-ULb|Q zS>TIkmndc{vwkz7J*w=Fx zud_Mo{L6QMFQg;PfAE_u{bSzX|3H@Bxqx-+u}8Q6G8Y6jybYg?_MfEr6F0ypeede_ zL|=YsIJP;+@2r64ye)*tr%=lv=Kz^|+b>+;O(gt%RcqUA?sv(yXH^TlU=|y}&#CeM z1?|s%?=1E|4FZx7B(>%GA2137@v7Sd(|gu~*P8$3Z=i?n0$+PjD3=J?O|7=4K_7l8 zJ1%w4!v2Z6P{*NdAAyU2Y!B3DanG~=a{*gVgZC^nUjj$IH^bYu?iAgUGmGD^PapOLNc} zV6*qv)>M8kGXfv)k%k?z?X{QSI0-2E5t&|#|p{$I52&&&|x%5fa5g!+c&0xYH?T8{dy4IF6f z!>~vr*s@;^*!d(1T|GTLjjooK7X0>jM_ufUrAYQH4NnEUQXvD$@R$wX;E0a5HgfM% z$Fu!gNFGDg9jUAlm;I$wk?{MsLDp8QxbSXFIgxV+f9kNUzhHoEZlL~xyMj2Vg(?Y? zUf5R3I`nGPcS4XH3zB<4Z0{*;1wDl>7ZMy-L5&gd-D=0bN#gmF^8`kV&j#5usa>H1 zRhz(csLEe4!qXDPMUuLCVhCD<0u^x1XxqXz(dP*k|4$eNn5P?H2yThu82lD`k|O87 zJJB7Q8$yX@m&kzo%)cUKyN5iGfd89K+))&yGLYJCc)T(@2f}0z4R+bL^C(qc7;+UQP(@5g)Pg=~UjO-E0OY~|3#(gg zu+xweR1p$-|3{3o*(!MzIIQt{)6F;{Ku^Z-wm(VUZwuIN#D}s1M*4AwhBcm;2tnAt zG}_u3;#aJW3I{`6UD?+q4sx`Z6C?hAumv#y>{&zaQH{YrUT3uPa{Mp5GQZ7lsYx)4 z@D4`PL%&DA|6@lsgp+=8e!gJnC(_@JH-T&+Ucg-cd+5j%ijBM8xVUAuAvyoGy%JvX zei7zid+X(0`Xq$fKlYVvEF&=t|E>vs~8D z3jx`$q^xSJ1S}=Z(>(?06k@qwa_vCNQbG9Czl#uPFbWw{IUZi#?AkDJdVw$S+hzmk z$qF#khor2}@RwXqhC-4#bWEMxveA1c_5aq@4q{d-GjsElumtD~=k`l16JHVR>@TN? z%s@yEBF(5^d0c$tF1;XvAf~CYrW;T#`O$ViD~$#nnguSodAOBL zmBq9)>Pf~=2-6azY`ycBI0gZ%^=QuK8UOAg_Q1bv3~VD(po5Q)Jwuo(-qMi&?`e?Qe@)V;iN0JXHFn z8zk1MtgWtjkFFTbXko~*$&<(i3>=>hS28S}^^5KFFn2Ue^x7DS{akSRt)vHAxBG*_ zZHv_arQEDY{;e;CaD})1wZq!YU$%|I_QwbXg25l+tl{m%AcaK~>?IRV!$?*YnG|6P z@ZkKBL>rL*1<75G16bN5n?dZM|4V2Cure{+=Ewbg$1Jz-RkIw@*#4Mc%N4obf=AJk z1n2)5kKz;NF7Nsx-=u7$wS_uB`WQTz)nl#Z+ev3`Z~%^qB~29qiG_PwA8f6S&xQbG ze(s73B>Z^rI!#qP(in%nQGK#?fXHSGK=Wb1(Z*Z{h^RiMl|rb%@i`x@&@M{cYd27) zD@ZW?r&ISFr6GQBLs!rdv0P9yx=A)Z0~pE4l}{Y68njc|ES3n{wi3od0FOjObWTEmfn=ju8~KRd=k|&4V>kKFoMtjt zOCuMZK0iqRSQOkWrWv5zPEdIFWv^#O0n{0Yf;#&943SZ<7dPXQM2nGQbH!_GkL)$0 z>~(2O>%_b?W(@kVHZkveAiK=CVCw2CacjTMG?No|Bhi^GlA^Qa(&>8J7Co91NTst& zv1a5xa)rmcrE)dj&*r6Ug@Dsq0dQkzzMM_#c~4UhR#Z3m0ybY z^g_ZJlhx(Q`Fp|T+1`tJjGU6#1jbCrQN>6OFSdHDt^+@rC+*p$+$kC zBKUSE-EBu3eTn+9#?;BkCAZAdYWA^exuu1`Fx7z?i^6Rn-jD$#>#4Zl*}{LzT7ZLF zMwkM#eP@Qz;@R%n9^*-~*Wx{c>7wNO)+f-jj<-{8^>7Dqsg~3ygj@EljiS5B%k9~x zCD4D(chOgecSc{+zmZ?#2wWe%W*;Tu?PAq#jpdy zoUaB(-0H~~mx|DmRZA)7I+VVEd~kK9{brrdO846c&v)AnE%2@M6Oq+CBRU@#la|M+bN{h;yKw=)a!UoaEqphI4OktIuK z%7oBl1w%(?!pLH8B^%3=1(q?p)iqDda-fEa-Yna8t4`vE43s8N6Ya|%LHW`nP&QB( zb1Nz`ax>?6+ma@ND=fB8bC~!*m`Qr5JkR8s@kW=I8|-qtZ}zjuB8fF8G{vlk^U5lX zgb3g9kpqKR*KW)wnuvq^I3RnuGY=~lyMwtCa5Y|ZMQis=)>H89 zlo-grF+-~jDZIo?bqkQUYs+tk6Oiq&(LLxRIur%2tIeDLj7{KW^r^P3vvrd!7Elud zcQv$!F{(+pHqsFh-T#^HI&kpTMwxM^W5oG-{tH-awnt;kx|!j4jGr!#7dqWS9h8&6 zVS{TOo%L1TLsRpGV`7yH)X(MnCwkBygT+#&yoSl@F&hhm!sP+w=HFwZ#4D`VRE+DI z=pI(C*OI%yJ`AjU^E=bzS3{3C7Qq#t!3`*pBY|urhPAzcxl>fCx;)tuH;t6|3e>97Wh(pqkUaIoF7;&k|!4p9X!rH$ zU~XQ|4;<)zulR0{N6lw?7qGbX;7{Mw1IlZYovw2WAUQqCt_@t8|Ke6z80)pfDf-sV zK2f$ap?8(!|00NZ3fBjEl|zHuH?i%=M<0kG3C)g@B~ z;R##XY}CHpnqbR;Atc~!q1zg|B^Iq>zXWuyrPl^1IYHVC9Hgdb4FKW zi>Ajq4OVfGmyK)H3(PL)pp~1v%~n2m4fJ_%R`+nrMLCB&GR}XK|?vu|Np`Ta;BqfY54W2EGAeQ)HjqfgPTcX2Z&4xYjCk#w`(Rx5d7GX^|e>QF7y;ntMWN6Q2T zS5`M>M!F>ZRzj=JE%|h9XI4R1fJm0X-V+6_fA4dC1PbzJoHV!6LqLc;G32dW2cQqi zg3u21o7_b)4@kY&CQW!4ee3J%(p9SbJUZ&Q>rEYe;^v1a3L^ss*C*nT)~aP=%5_gM zgV*905__Xw4X#jCJ)R^7*Z56exL0u`)E|ph+8zO zOXyoNM!S;x$dyb|n7Hz#j!10;&a9(8#rfcRabKb(gBmyur&FCM(>E)1_*4i$(RVXH z^2$t~UeJXnMCa>!<%meOp;|*OV2+-I=!8%bENwHdzJvWoYr;l(`L{(zPRq)hCM!S3 zs`Pk@5k9>J+vmc33jfLR#Sao2b7NZ92Cl`s2GcoSxH&Q=xn^ytwz_7r5If(I$6&A| zwlQ0`z<8_r7RbMU8N29DkCqL-P*Og3BKS#!e-j3fWDgP_;HOCSg3V z@X#_MN@A4({rbS_4;BaC#d(ZysrpzZ;U93($zpA>+Zy}Z9^n8?cz~jjT-}(j#ZFZ-* zL~Faowyq%>V5(k?YxVmXR$k6W_s??qRSsv|S_q5R0attBEE~MEpi7s9*q%6EY}`jZ zc(wAJq`~JLsrs~~3P+!c_4)&iJdR$C!x!&ve9u&P7;qkr|Gy~i4 z?7WiDL!z|yPJW>Cwdx%mPjHPK)X%14?eFaJ&%==zFLjR}^%9yh8|2xX)%_DAx4P}C zbUx8%gcMPw;4n5d^bEch;6jaTbZyh(`03zMgR#$+9&6nY#2FC{Gd~7e+$|3jbzsVU z$1|&1JxrVD^>CUDKki^6>H5*Qa89EI@_OD4KN>o+{$Sr9*!ct#Z0Dz3iV>$CI!O)f zcx>aXSJmZ-eN)E4XFbAd&u=Wtq4lc=i<7eg>9qNIT~_Sl5)0NO#*EO}8#CujtNzw< zwZC-@p1&Q>%aKOHgAVqz?NM$vR)p{*b#YB;QO>rV_D40tNDgBH4Qf-yj zZ7RwQpguQ7IE?pBO* z{6g%Q*dXSqrG-uW>`8biM~-EuN_URKJ@5HXX2U8la+g%z;J!YEmx14z87(AjuO16F z#}#g@*y?^BH=BF4Rv0;VVp=9lOYkde#(r|0786dt7YivbC|byA=KS};*pMKJyLQ%N zupXbhF zMP87kN*u~V?t?|bQi6DF^=6E83on^}DI-4UId^${{fDM?SNYwO8kM1L5ye@!#a>l@ zd-o#$v1XFtb{pY?#U^fc1L2meC0R?Q>2+fZ#uCdh^DkyBq){%g(?Mq`_(zYWC|%J{ zH%QSLoeVuUXH=4WnfJA3rbW&0jcd}lOeVF9I(8OTLoeYzM@aMfvXXzs8561anq)Bw%AJkB zt)JMc7OzR560{FS%mipjT&$C5c9%$<|8ZGhvCjRPqf+|_;IS-HtPYfCramP;I6(kE z0)2EuAspo;x58q{PHYX*iDMuZWkq z9uMnu$z4`8$v&tr+@_)|_=RMA@)q^Iv^rn7A#-9ab&NuR`>L+T*UF$bRQM;ifsr$5&!3x%$o-`{s3r{j1?8vM)NUD ztjCO4Xjy*z)~)?9oO#F{IebNwn-DDe+zf?YK5S!CI-`vOR7m8gY*PTf^ zf>a<(4fc=lSNWkd!W(9=_%`k8EY}a+>PP6jqL=kCs?>Ggto8U8<_NU4I4+|iV3RsL zRM_^7j2WGp@&`#H)$Iw@AD-8ZTVAQKm>&5yS$ZWeX#ZoA{gp23Y+~&8@#X7uZ;&{~*hj%oUlhoe5Qz9x<(x!i8L~ zj_e^lJIVKcxl14OIE95Zsg_vOmPzcLd{kuJdosT#l}+W`^fPsWzF(H-4OVMEPM6lo zcDjbV6dqs}KI9=+D!x?rJUR3(`t!v>bB~QGzcsxLx3%-R<7MrUYh?P%qpj~rd*Ib9 zPxxU64xfCI;;3?drsmvr)Z+Zxbk#>}UetpA!00@xB>LV@!ZY4{C2LAhL4VN$b$l&` z?`4(Yj8D+8%7xl4L+qT`%EGI~Po5dLW%mwexjSbbu)K*(5kEkDoH?AwKt(0n?@ct0 zQ@{K#%Cq5JWn}Ic>9AUA3EX|RYa69(C4KzfFDSXS#j3cv>ar!h%}dH;#`z=Bl&6M> zkeBX})_2pG*q^X@bL=!%J;k-vvWk%L}}FtwIks;RnWi?Zt%33aZ+yZa$bJ z`zXrHO*vsbX6%0Eb#kW~hjCiI`C^pZVx&E@?@>ecjD*2&Y$YxqWX5rQH%6VM)7B2r-pP1l#hF?47qzQ_+2*d4Ejz{>r&pXWNX3t z*U!3`)0Zu>o*Z@Vpv@+j=4S{=FYD)2cAVg(DV%2YjAB+kJ}JI*sKmGVxJcPtd&}z5 zft00>pAU`sZ7hFuf85DvQYkD?4UhI_$aZF{G-|x>p9dd^o0ma#ruNEJGR0gtrt-nd zb^3`E$(uvj2RFKzVwYau-3NO^rHezi_#e{FGUgnTaK*a==*P@aCiN|=51D>-4gchj zwsMD`v;^?b(#pU>T7tntC>&Iio*}5T)E6E;FWwNt8Y8f($lN6LWnqeZotDfe~ z+7pH?>NCF9uMMA_({C)zDLDz+MyjV>)#K295gvPU;_0n^RCS=tG38q%ZOv)qqeMNj zBCrQk_lLVHP39hC81qY}3B($v$LD`PC$k!8!t{P%S#o@NXhH6hLO~ww{ees-&7^x| zG$BMfg;9`Ekh6CPyL9g`cHw3Tg-W4VzW(&zw`R=klNCK3OMu~n9IX?35?NM>Y~zJl z^)H>L>i2V#>iGPKl;w~mW;&vPz*YJnT|ZVB&U*h`DUafAN}65JYD%>&X-d1f&?Ko( zMQ45}TSb^*Wj_JjoA5B}+~C0~qooi>%fpUw9&gRnixs}SHxS#T{ty$vC22r&F1V7A z72=v_vX!O7Ovs^*O&<)XOnLOe&rHO_+mQ~Zj@T0PT!vKPekN+bXAONVcVN>RVKIC+icHvwNcF;G zBO9Nd3hws|Y4EXdN12={6piGE_nlSymdek3xwDKkxu8_b)4Y=B1jS&)715ueO6@j( zi}6rXElu?u%|8*R>D&6mQYK91j#isYXm51|2UVJOQAT7Dn}kWBUi{-r@KDwtV@;AW zw?|_RJv*+bgQ=V^`N?q6fYxNtNF~+y3MmcGv1=FKp6x9Cp)_|DmTfHjxp<-Zez)!8 z&SyWp;zQF*2Uuu~dS!VWI+cH_$l1?&D~O5<%C4&FvnGxD@b)wKh>Z0AwXw25)4kCY ziRec#!ie2TrUr{TZ8_hhGnTLBgR3aBv{Ziv-H5!npK@b$xHHmg!o;cKuhl+;2h%;` zVkyMnK)>Pxb=>Fc)33<5n~bora*(M4`rt~DD7k5iK3qzadH*-_v+fc*y&6$XJTQWO zOtB6nCm+37(n7PTHz(P1Y^HhVR~`nwsAbfCi+$tqEPql^)?sojEq{9XMSkTXp~uaJ zGX8IKZpqG7b|Paf?8(<2IkkMBa8fUuJ1(;DuG*|CiR@#oTkrB}nPf)ld_BNOu~m(v zz|tKF@~8KE+E+GM=C;Ak*yO3gFwrbE;O!Z0x8{|Hap*XDKf<4-S0qba%evcH0o8Sz z-wiiLt@x}mNycxbO6~avwgd*9?o}Rm&-Dstg6CEB&RS{zvdV#>gq?h zv@`7{kJPn>Py``!8j(ur25eW>vNQrmg4Y=1dG#nFM|fjSwj@vI<=*=Ia-GiwyAmZC zMPs~jX~1AU&Teo9N9LiC>uf%hBfOvO#fWg^3(f@Y!6W%&_b9}=1*|N)9_)j0Feg$J zpBlG1<@?ldEr~+<)~HHYKr3Bc@36d_yX8@PB(R8@LoZy7WUDi?p5e_9z$Vd;C<-&( zL`ZJCJpm5n{!DPPfXlPO!i-9E)S=J@6YZiY%6r5gbcbTuGLyCS^ZYa8JHN@DSbn#5 zYbEo9dSx94!Q@9z`H&YMWy0i1Te8@m!9|UGvkWV=6Z^wi{QVUXO<8d-LqFJMxno@F zDaMayb^kb4?}H>=6!)2C?{MS3u#!Q9Je+irpp$H>_gQRm2=CxC`ex(V%Ii@~klmeZWLxX?D=9 z*oa}?Prp1b7c+%GMc1W&5&?vxqigcE#&q4(wbPO%L>#-H!de4 zU;wm7=J8hc`iFu_`Pv5)GP58?h)T>!*Lz*g(AF}J^J9RKt; z+q3m}Os03yx~bk$k|eXQBh`jk7vy0qA8`q{MYkvq?)(ZIa)=a7%j10CEXZK`(Ztv%K z*!}U$vz6t29)*Ji^Mf5#unyA$I}i*H#{2^bztiHOmMJ{?_4nv2icG{^I!v6LR|K!i zP{{KO2n4#UEk(vN`}-H6%4H`<(0WLA3inR#8qb+1Nve^Xy$oxgF@ErAgEu#1@5~gv zS6Kztb?I16FK^gna`=?|>In+jcOhwh9~ZxBCLIP+p_#Zr)u~v0AU+K2qZ+5X4iu&Z4VZ?Ci<~LOC+4tEz*5_J1^$KtP-Qew=DE1}g^+8*g zzs+17*_sBO^U8%Ro`i7C&rOB_t-8JT^kQ$F7jN)_tv1=xR1Y(<(c?y-sQ&2}=x2Sw zT$7zjWqtU~@dehL#ik)inWTb^{`sn4ov%{*{1&a}m+b@1zo8C)>X#7I z8+kuvDH|sLQ=Z}H`L7?D<9@2uo3~=pxx>oEBF=|=&SMxK20P+#j>{JxZG4wWu2yG+ ze13qGzJ82J-iZ?@e9bAhz=I6IjY|+904@nbO6ltOT?TCPYmYUL95o7c@ojwM|A_F^ zVV~ibRV)&0G5hQ2-82OQx;LVgtLzW;RP(2-XnjtUg3z@jgwDGsqG~(QT#*F{mBXE? z&TnfACKRhk$k1t7m}s{*f%{sJdRtH(dgU%IF6_^#ZX58!?nvdV9e_0dZS4pRqPu5<}+PQ-_zg9&Hk>3!@_*s1#^ z`wof9Fu6$=dM|e0s5L0NDkw6NZ6aK^8X*!Y;FfldV;xyofUf0HMi;OX)z7<`gh~48 zGir}qv8+PqISvkpN22wYJMzeDg3~2C+jFQZlHR6~kLh=0cIP}W7r95wbW;HV+Q2ABt=>|e|#jIoD#oA%|6YEX0RqL8R9ayys z9#V2xXr0W*gGr)s9L2C}XkA25h;h;R&?1wWvqf$td=6uZ z+V4S0TiA53+W~RxIr*rAr3rChNU*(+175|iyDcp^oZI9*1Ol1o)101 z8_vL&rVnkDoO;L>=2_FDO06GXrAs2gsP|6oJ1xOv*^e$x?>M_OiVl?ASID=d0q zK6cF}Y>usN;bkAe%Y)OWh6Q>~)G`OVlhqk*GeI~ONMdd(SY0kdc- zl4|MnFm=Vf26Z5O#~{{%!V@*WfMX1Y16^Omid}_*7Z>m@9S?jI0vJtB*q-X&7kv5p8Wk`9s*q6VKDxZ8% z%6wRc=_O3??|rb>tZFZ=zt^1L@sZuq4ftQlN8{ndU9$eZkm3k|#%p6TB5Ms4FDCR> zZl$C+7Ck~gmG7->7`s?^XKH5t%QGLd+P+@%m2VrN0#(_4y|cNgy}@Z3Bw|+egE&I* zXRDOl>KCl&UGZR%CKWhb6iYThUL+C@Z})hfWdW}rw8NWL$s#7ftP*|Zd)4`;vYBkX=b(H&4A zdRfj`M@Me(HWG@1UaiDv5A(pt*d^H#n!H31@9qnoAA7xaS`${e4y}dsL{2>+RDPFM%9*m_S&$xY4#6TQvZSj}R zx^xPM5I2jTPxN^E*rJSY$C#(aIa-~$yDAtGsnZs^Fd4^MU~`hUJ&l2#J;6OATyo}2 zy=kmwF?yC>`u%I~^)J$xJPOE2YC?On5C-+3RS<~{uXh}e8Vpi7UQ%o%T!Qs;Mt@pM zh%~L!sT-^&g{e>}Xd139iLzUdyvo*5N;Hv|n0z<-;1pVnxJw;@37RgP?Bq;Zt;s*y zZ$2K&K~SCL_Nrbxm8+ASmIeEX_X&dJFSOO^N9`oRGLiq^gGp z*7~DHOjB!njBOtac_(d!r2#x7Xp2 ztl)Apj|}RGWfOkqJ6X9*Pj10(AK&x|d-n9Pyh9_zinLDi?Wct{Kfb|q)B7HOOOk1f zm{3MapYxb`OX-SU%lC4dTc5_MQl$ISIhwA7WMD-)ocs?aEc$_vD*0m)7H+RY5oza}-aK~1N%X3E7EKA?Bkqz$R2TZ4 zn)$nu$8N>^?)neEga-rNrnRSXdd$U6 z6$QrHt;gY>uMA$PK9C@6tf7)|jeW=qJHE&p2_j4f-5Y^U-Woz}_^(mq-)-G=-M|cO zdkg;s!uud5Sq{RzbMFvTTdX@pbKxdxH9F>%R{p@(&=zs8JAQL|5y?lZ#V86l$~2ay zuD^IB&YN218PQ~M` zVa2G!yIg937MJ*VvQ>SiYE{ryi6XJl!;@3*K+?s&yriX074JU+U;;Y6iJe(BR zodulwH*RdC&tsz!^y4r46)Z=SvO}?Ff+FJY*^ajFbdu+6a-q{1+FC^Yz*7!r^Ia}} zJsb)Ke_#5Ptu51&e9dn$bmfz2s^o2ToA;yiBO`gV3SZyJUedCZ1W;c=N>YAOW^R&5 zg6FRBy26Zyh&%{RN~xr~g?u-<{6a*zU0^~fnBC%jA^qpMuMBz9v!8Dva2q$%HOP@i zU^Bf&dDkhdzhHe!hJVmYw#<4cK>;PDNQqLZ&^Vikd9F8j?pV$>S0u#{A0U z$Ne`0wm&B4hd)(?_GzfhY7@k9*&cEX*>qwDSf7 z!t8IVk9`wLIcq@0S=Dlz%dn@AJ8X7A~`?YY|=@n zV9(>9EwsDF{<5%Qfw|rXKrtbLIE52jqx%mWu3?nSj50q^jRUBrj>0IWE=Su^!2)9; z$KM5x#7@+nVyu#=RbcosZ8>AdIa?vdqnk(AgW2H3{ePTYcRbbK|K~;-C94!d_LfmZ zxRouGy{X9FdtXhm2_-8dWk#}B5+Qp>$lf=5^E+=`(WlS&{{Hc}xcA(1Ugx!6&)4f6 z0`m8P1ZMTYBr}tSJ#~fwbu$t2v1pgipp#*biCD=TALqUO>Ud1CI&=vFcEOx->`Ax^zP_$D%K)@0M0f=?Y8!t18IKw0~BGDxLWuJ;m0|gsGe|t1o>El=F|}vf(tDln=-)%#NWo(I!7@HrI-k*F)BQ^ z$Ln;Q6cEF4IE($|6J9v>5c`)fD7)cg5)M2skIjNx?HON#BJ&me3Xcjl=Hy)IN_@Lh z?A)3$nH!ax{6whEzy9dmgp&&YYgamC-MjYE>dl)YzGT#Lwk3+~Y;L@%ci#+|qR07Y z!x_9)-l%KUC<&>hPdEo0f01AQoYDPuw=Cp6Qf%agueg0y80nm|w##+V2lYhA?ua0D z@!e>*#M{ljRql9d7cgpo;dAbwJom&XjlTv_jg;;)=nbp2s0&o67m|B=n#H{J{W>M6 zY=zoHWM@si?xkz8J_ej`l$K*+fmfjlDIj^qhF@wyJavw@crTlYmX@#D-_f_pfEHur zEK1@-jw7hpT$XPqdqt76M))K-zS=@lZsj=+^YQsCoBeydA|z>qw#lcVCHO|bDd&X53hDWt~7A<8qz1IUlIY5C_)<>(#YQvMVDAp z!D}@1K36Ar{Nw^wC+}wB?3#)uiKVarzU9lZLWSr+rt5`YR($P&aNa$~5hnSdMSrKB z{D4=KNHm_4$z$Lx0i}I?V#o6UY?t*IVGod%D6IBb?xPpTH9N7+$4reVBk-I&<`?7{ z(ii?wAxt74x0{XFJmxV3yW929$rvkXSH+r{e5d#LpU znW_4fJ?TMN14_q2Svs>d|7TB?Qj%Cd15=bUwGMg`&33fQ&b@SsA^i6*2wWB76EHwl zotSjw79Pmf1RcL;6FZqlGPB z*g87GgXx}%eUE7fvj{X1C^n?MVR-*ba3tkW_o7G5(`}F;IZ_Gf!q?N4wSi5zpj?wH z>PBIPOh48amPr);CFW(W_;>$hPmk^Q*R8|M0z3NFgkSJXb9b3qC6OZ6Miz@=wt{`n z+YBX>MTbbX2y+t}w7abrj>UPOJE_kWHd%P%luUx!d|SrIr|ag(@3}g!^+szRv28Qh zjAUS|3EzCQ8PP^$$S{mmajOiJEX+@h%tX=pspj8&45!9`^3fAo?rEA`PVb-!7U!xW z!z&}TB7ipU(|#d5e*1;sQVAmIjkB0mQ$;Aq@_?yh5e`0xlB3xrn;*SCv1w#J&7PP? zJY!ViAGj-c%yskgvAKc9`mD*)v2F7KBOh^$9{QUJ61OxSX}-C6yGh^}h6YHL8FY{*A?(@`U?#L4XuH`S+ zVdoX18z?`kPuz$zyCUWlf~fcD`}ux5qddyG*@{zOkn&}2H4v}!0uIwBzKt|<%q0M< zBv~~^#852J7Q>C|poN(h8rd6I_S7=hngrB}a)$(HB(EcxGd;G^ORw7_Goxu=oyst~ zB~D4L?cVuLlz+K44Db8vv!i%zW!W!khnh`dLJep5q4alUlysJX2UEnumXdD8)#(6_ z>rq_&?iX7`{0n4at~koPu14IpsCn~Mbgt~V1B_y@`1*J8D_j1W&1+|!?)DlZwZbMN zBw7ag=YLE)bg0ebGWR(C%HbpAbT+FM@A|)$s_lTAQm(JVAP|#Jrs3&??L!rY;_q6{zy`9 z>!kHF%)0i8rLrdU9DU7MKZ?+*t>qF`Izz)dWhVraYK?saqz%=!&)cvD;4aILf4}fp zc*&oqcuDQ;cSm0<23(n5Lzix!VEaYB%O8GxvepV`;u@?A&2n2ekMD{qTsjjlBFwD9 zFGkv$X3UP4zquLkGI6-B!G}KOVd|99)pZ+lsU=8wGNSi%uFtBfT7> zRip2@zf{hAU)LL>!9C6`Gju`a`_GoX5$aMix#Ju~bC3E;ZeqI4MAN0TZxJR(nD%CA znNKVRr)s`u($lheG&d|&Tg^bh0)>Q&6OoLA6y_|VK69MiQx*0TIFUDt!$jt5hm;AB zy|>+xf`auWNHgc+jsvKK@g-Agwt_wmhbx<5$6tPsT2LvP-1QXsn_uH7ZbOMD?#*vC z1(BlY-AxIhAhe=QtPR6eeR2+@sG|||7KV4kbH#PSZq1j3Xurs){ctvZ807R5#U4oG z`M;n0agjJ7OjcVkUs=$ovFL$j+ad+XjM3(g4wL7bOjI?y{|u4OP%ESor~GEyS{j4O z1tfaYWcZgT;DPMg8-y^p@dS7f^;=#mR})Z&HU2>M*f$&pN@T(N98RXuEj1JuI#d@V zB5O$IwS#m#R+7sY8bIPq$Z^rA)_!5$XBaR1G~uQF@B*bR_;e6h1QI!D@4JHCF!_Mn z4Jm&n)}OV=ZS|$C1fqZstN{s@-Ah_=4##JOM#igSp03u@vnX$~Mx5!_YAD+J(Y!eK ztw(8O;9U+$$@T{x^Uds{l?jqKZz4~;n$0xYiR&@ zOzTs-{NCFMZn;JT45p6i3h%8N+@`vYPu>b){CcWXYdh(<`%g9-t1>gQ!Ak8G{Rv-v z^ht7s*CJ(Z1%svMToPKMGM=tY*z?qCoqRYmAN#otr&eg`tN!hAmH_ zWP3nct%UJ<*w%&yF3rm?eT=4nSw8CL4L)x5d@R=f{E$YwBb1Om)}yt63_+wal3H?swgq4+BtgwF3JTk2Vlp+^ zi(kq0`&0J3Uutm}ns>?|*BtZ!D~Rh>Ev5ZXe6JJ*G=q6pOnmJ*3&>965i?p!2;|VY zPf_w&!nMzRS9y(xq<^0^L$G_yU6LRawjNWfKe2{M^ zi3JZjM*x&gAWyJ$xGx^fj$5Wn>A5IuzBGAby!rF+^LXjCP;vElo2AzsrY=y9y2;8& zjYV*_XnwYKsd(<7*D+HZww{W!Jk!l_q)Pj{?Bva&UmvfJ()HEwsp_X$7FPl5Un-tV!X?%;#f;RF2Eub{O)a@G281!~_LSfXnme(w$pxUVf1a0mOzt zj2II4`6+|ovteWjI{-j26xyo>*iFY)HVT-8kt_Vb_^t<)Ix!byuSJ|`wkW@LR>|#{!l7Tsfa_LYXgCqVl`bc}I{xyDyYJ`| zrORAG@lmGh-P?Wsix_M9uN~hu7u4KGiTtjgCxS5F_4E2&(eLG7-XTAM7sL?Wt*#pp7Gd0q^@*GH$9%nrIN@GrI)VgEx^?B?}enVu|*(WRbO)G0RDn2kuugiWrp%i` z$HQWiTKQJ4FsktdB@w4wO78o^8XQ~W=k~yw=KonM< z792^P;4s9bvZ|M#JShmg0-QnFYY=pYA=3FZF|!`hqq%Ks+0DLWY}6m#VZB&2e`Gq2 z+;asdL3L8&(r}_`0A0Tpw!Qf_@9>>hpRj19`y2fF7&clb=sZ64@-PSg6^p7@0)d9|RlrN9Hg;|j?|ZDY9d)uK2qy|*5?`*?Ht9=!`QnmZ)3>DEL`#gE)}53SYGm#- zAMdevtix4%k}OAJQiI``Opc_9s!VT8oAk^~w_*(i(TYEhO`jTz=gd~qR4^$QGSpl? zrE*rtja5D`FzAYWyfbEu_4gH;FHvH$f^Q?0vGY>5CRD~bbzhHZ>A;ll9D`jt-Kg_3 zyOtva*=yvs>^VROE2Zds9+rMS@TMoV=qI@=GI~|ET%FPngvuzP z|8X`D%h^-0b{P2YGRqoDHwHTjCz75mv-srbuKUyI2swXi6u8Oix;+^ydCr*uSI!Mt zmD{Vuu)a+BlYLG@XM6D8xRCB^oqUI@%NiVk-g2583YFif8m@ek75nZIYDg%a`;z0r z%*qd~j-rygNgg*w_zQ{waD@xh)(a+b$MVy9LCd5)5tN}Ok|h`E2NQ4 zgg?0VEg43E3QWHKiU?GkPx2DSNX1|jU!!$kS+ek!G1~b&o*VI4d^cnD#<{QqCbDffj<1wVkAHoQRIc@{TC067E2Agj z(*ofj{PYcLNxAe54In(KLfhOd2n2?k$;@|)%v6Dfo{^E{l5%q4si~}YB^Oq;+=a@; z_dWI<43Wfr;Dt!S9hRBAd;MUbn7E89z{XaW;S1zeLJ&PAAxJl5p2quThyV}gN7t{F zG23Cl=5mvebw{vrIXIK3aU}&j<52m=IKOqhTVuJ-qF2;?aydY|UuA^pjYi>9R>u!7 zkB|4=h~O2x&XXB8eCoqc`_c)K5I#sHzo5~mU0sx&BOh2^S&FlB*B^n&_?{(N!yujNY_3=pqWnxPL51<4x=@o#lFrLU*4v-v z69J&9t7NlLOV_`ZFZ*?eXo%?Aa*@c>)7YD{^Djt*77Km6a}@KNImLoyX9q-WTP(U3 zCY8rs4RUnyPe&3A;PFNL}??Y3|C`IPTOrzy6e4xPGiB82-R7b{kHzx zcY*EqdQ5j(1Q=EA!ytUw&5EOvG%gOvV4e1i5K)DgXF(lpp`*xrHFv>0Ga2hQiVTkv z`?!^-8n>?t4#;6laokK$Zw#$p*s~|n9>&aEM+AG;Y zWZd;MR4|2UUN!ICVqX_;x<(KeXX7n^J-s=ShZB~>zqFDZHayG9s+CieP4?4JB&l<0 z`Cb!UW652a;65 zB`9{gjb(V>i@{2y9 z)7F%q{^=yu&k52*Vx+Y{yQf6BCTtq^tet(aI>H8R| zQ#IDoH{LNlQadx8R{a4&mh<8&f|9QON>x-KSZ?~u zWf}GAi^Y$}jcngABHY-n)8W~(!KOghaI^^3;`E-*@JMAit#L7Z zv@Y7wr}eGf5DDMQi*Ii$DQ$I59&4ZOb*tBv#>eX7j|Z)-F)P1 z5V-AEq8y=hXZq~a3T}gd=mH*1C^N;m)QEe5_3#@&1s*yJAD$R&gi%HC>X2e8l?%w)fb@PNS9 zmGNh}>>PPt@G)vtE)_^oNLf$?^tE?Nh^&?d^oFJNSWz)+;8~}a`A>Vx5DHl8mKO0P zyob_uJ1N#NzBbxmCCd6x!NOFcwFg@0X7hfT%hfT$X}PKP0Q|E9Xjk@Znb4xuXCYtZ zJz>W;f4_F9wcxEo?#Q5A&ermS;KZ~LNgn0X08wjcn?INca!zjL95$v-k|u3UNvTw1 zK@={x3e&m9KA#k&j6Kr4p9`Q7Y}YJRR_BJ01Wf|ImZMkS<8@GGlni%r>!j_n}d zcN_;eEvyv+IFu7m8PSr{P3-#k2t-U7(qlz#BTegl@Yr{z4QBCM?}jtB%Y`Nq4b0~x z&omuj3FDx-n5@#M6eEhHA5iH3!09ooqOBpWEdJdll$b?%Na8BuqTtr2#P*Q5qLZur zA;@B0w{{PZ_k-Y&a%%5R$UJMLQ;lMvG^Ep)3dd5W8xxWVa~jFCkCUXj%HllTN*h8F zEfwA@erlQdL`i|gHoDW2=%hfk>bbX8lUl;5OHMUwwk3_o{2yH;W0_KkcZWVFnd8e` zV$$Nb-Cmrbe;{5&N1`bx`?^l^^qef0XT1i(8c&o;Csu;c{l%5-q1&lb--~Orlz+s1 zW`(e$o-{BI^Wz($mf=X9b8gP)3p>`U6B`^em9hKV!vjF6z2_xsov8DC4 zMfOGk->)x!1`e@fxh8dSv*-1!+clvi+i!%KFle<3u z05yOfX{s6#@4jIacVlk7)cvZm(0Th(()`O`6M}v`NDp+jlUI=9$a=ut7*=Ee1lKoDYDJwlyG$l z42)O971zvjKxiiFT9RriSpD#ZJ91`w3-YP<2=#q2Ap~H`BJHH7+{OLiV z)^rl`3sgY`rB&_dv1Ub~=ta%4GYjT8qgtU9JN2Fbghc|sWFe?k>h}*pjbv+*-tjdn z$`V0o<&S(MI+|LBfdly!Y1Kl846K}X8M1f}f^u<;r{TiN^TX8&6oP~7NtE(T+WwHn{{X17yU8#DI@3@(wss`xag%o zvLa{GP!MLC~K#e1LUe#^oGuxM#r&`mYaW_#n?_xC1A z`T~!90(tIWUq27ihS5n^&DeygyuYgAV7@Fzi02IVwKoS~dBbukU?}v<5leG}6AgD; zx$39_Nnzc&=8}fu7nx{-iHtWP^q)`!r*?1j1u1PZZm4CI9aLu?Rc`9QtJVt80&j zDoE{U)0D&q>v!Nop!KI_lRHd0<7-FaWsq%EbzNo$!N!5vC8un3NHa!c&(VlWQD)Sh!+ zlm2EGK@G~P7xgqE_*zEVN%TOL?n$ai`cD7?e(ocZG2dknO||KwP0i9rMnzPq$>4nK zEZcH=YWK8ydyoNznZX%5Lxm0(ha2r=t!d0^@!!52d++ij)HlAG=Ir&6FVhYU!D-tw zzLJ!N7@jv_8iX6QL$B8E_GX0KsM9YSU!M|&Ngy$j`B}Ph2C4L;!fFIFT#~iQ?41q8 z+VE_)u2Sa{G#DNsqk3vyAniyjt`s9GW=9;s#KJzLgDrjFw{tP1BC%AlZa#!;^i4F+ zo=~z)e|F_mFCDdPOySKrxyI`;?nCNKE;p)U?POCm+c`3?om5@-+fMj&1U*Zv<{9=D zZXQQ*i6+H*-}<-I;kkNWim>5~qS?f$uECfh`Pa5qrE4*4wM25uJR|<+NNXU>;y1bn z`itTZhT?hGzwlmmAakHO%m40$rt8PI@7Spr3D>sq3Xv$6dnFB$xo>HtOU2N> zrzxs^+}rl4}21q?{ZtTnab(o*eO;{u-2)2x7=< zLL3Q*?I>sK%QUygKV5b!FIcbDK&FR;@d|R3ZU#l_m~iOkkw@Qwd2i{@NjTgtcw%~ZzX4!+)aoCp?%X3p$*xq91qn{qGYLen^V`z!hq-r^k(Vkvnc1V2B_T5 zBuw<3J&UX$kU3*yslV>{=xQPAwhrc64*m@v9&3r1BW8V#0`CSgq8HDL0QmZ9l=)2g zhpe%93qL(i1ML;X$ASenosYZ9q)&2*Mrg06CTI!DlIA+An@_C<`?}r%P$l6F(#x7Z zulJ68GTAx_;Ekt>Cm(E6^br>0N`A80U{H0>Uo+@?dMtMdnOoBKrI1p+k3fV-D4Kjb zsAsux}v_hyyksnoR3-`*S?)6%=VWHQsFTg?#Wb{><#D+VIRPpC@fp)WS@@LQ0z4##XX+ zvx(R}$Mu0Jm<#a^udT3g?7I$+)$nWk#rqWT-aLZGirEY8ktd)WC$S<`6KL&|^vK>lkl|@>|sH7MP?f#HzWfzJOvj#*I0?=^kAD zvY}LEw{W>G?G$D?B{u=ee?g!=8KAm0 zp@!?gW>{N&vPIYST^8j`)&PUsQPS!)Ze97!20i5Y?)`&e_f=BsRRC_KS$~1&{B2t0 zxa|FOTAnC^?|@a@Ri9<6Fp=ksz=r1r}8SbIp3O2OG>$bIq%IRGY zh{TH?_X_`v+l$~R7TmsVb)JtRM3f8Z1pH20EgbI8Mzz;&G`rjg2SHA@>0|?2H(RZA zI6J$;clWo$LMi1o#_MuiER?Zj$n%J2)B&v`w#4@(yEwb1`Vi}!# zu{BJ3SgPtv$jO{bGgG(KeptQ|I#yq3oZ7Sf(-l-=p2j+K^KG`yJqeg|bF{CKZyNQp zFq+C;aB6zh&GXj7>2*-&LS<3gs)Hl8{UsUAvw{ha4Cf@)+ea}{3qZ_&i5#Y^dkjW^ z)j1qqO*587%zkH+pXC@Cwry_GgTfU`bQK&>&1#XT;44u-aT#>dH?vHh0NvV)>v~0= z#yYN{J@a!0)-oBln|~(TzU@ej;+f=wbJ3U3Kn4=PRw=AV0L+1vK`}$0HmHaHPA~)= z4R&u(h5B5>%-Txc-*DKJ=$5k{xtB*2P6+jLIb*%_V*kdcAZubYr4qbB3RjL6?|ATO?V9WmH84 z7qm7^Nmoj2-cyt3p1ze|Qmd%8JdG@dQ-o2GzaAv+JSLAg?Mp@-VTEU@8sx0*kt-!8ad%+ z*#&>*5&qup?p$Qpcm`Xzi+EiKgu7|?aBxP}>j-CBcWu}qPYG^~Xmeo7@Xe-EP8SGu z2v!;2QK$^O+?=+3ys5Cshjc=cXZp7Dy7F9}hPl);P`jrT3tS9- zg0whK_mt$~()bmmMmKMu6NkeonTQKG4>n#k#M50CR&D1ixpo%vC5DBHTfWp4>x9Kz z+Ps@Dtoj3r3n8$V%DuD;!FMQ}CIrt(9_S$!SVp7F}gkCa#&axO?v%vh%EOEJPx_cM1U3r z+L_*?ft1Nkk^9#VQ2eAu%@H8MCAPPkdbnFfL)2nW<1TNvugWC{q(18Z0IBj0&0Esu zB_$>LpBX4j*_^BJkWmibn-rg(@|jX`xES=~w2zG;;eChhf^>(*#wx>b^kVj)s>fP) zyToO38lTshO-3<%)&F!%$B4bAFPhuK&yeu?(iDd%@@Pour|h|g_Ncj817R53oO@Y+3``y*upf|!Y#S{v50MN`eKX%4ew|vBy~KeUrd(bH-{)@ zyO%;t=L%#(T*T^#WS%-LeNMFQIAUR>6f>N6S9;adm61jG!q1S1}3|)PQcOHyC_af1`Z&uMEudXqH$}ZiQ}biVkjp z@AbYcJcal3!tu70P;NpTiUOqZ3xQ&vRA-OaE1Mvv=`eLGY?MhY?#&J7EI0Q7FM3}d z3+KoB-)}ww+Vcj;T}~^c84qU(br;;qsQ0!uSwC7pIy*P?AWbxcIhBw(JM(UBSa6Vo zPGX9Xj}o;veZld9P5Bt-UNYyZ5Ks)!sJGa9wg7-kL0GfRlePNg&&bDJkYvqyZ7tTo zB6#(_xK2b1qk8~)@VgD|w+1q<^%f7+&ye`ge?D^zW99l?s4VVF=9V_*j4+l@o3Qe< zmu!lAB#P7{d{_SQT*N}~Ysm|4I-yN>`?!O77Vwyt{UaDu66J!qNd=kHEZjbJE;}I# zoi~c@Mn8tRZfEE^`>)KLf=mdP{QzP9;*v^&QQ_=ktGaCQM6qu))+3Js&Q>#Z*u9{> zPN_x({Fra9j$8J?mmJM_e;p8p5-j25Ad5u3G*~I&Z0t zll$!W%zNDlf3%bPR|U$@Xt$8@h8|j{T4Z>4p(f)aT#cRqYDJo3%i&B~ z*C@_?y9S_UBmH3#g<*EIH(@nL9C*V)p30yVcQjxoSCguiME{FjM=Q3P)+B@=kSmtv zxKd}(4Tgy4h&3<1o)>%>77%eI4x91?kx=kDMFhvm04-Jvc1;##hg+GZRA0+MEcKUl?GRImF{FApGuolCY&SPYbfw+C64X*N3Dc1Fd$_GM$e2I zH!|;fA50uSzchxeN}e9kT}VG4t%uf7lKr@CXf+jY?qj6T+{#ZA-$L_j3pfAP6q~qb zLdkha>dh1xhVlwRku4e}YYXhvHx1zz;LhJ?NjNf(!-IrwFq zd&B6ZM;AlSABkA~{<)&C#A$P;L`J{u*b)7yHB$v|Mrg@jFPFsfDp0_@@=jznF3y1? z=)zWtLP{1?KvCmbwqM=yaT#83Kwb={=OR0|51vp4hmqp6=V*Lf!WyGpZ?9_zhUBy(hff27+cZ_+%DE2L zhK5O<3h*D3B@4d4ZPfg*ca=s>D?{mha1nR>6T_^UN25`Mw|EywKY*aeu)KVr!=7^ts;sL<;#seyVDDrnjnnkNh(=H#h_z(^>7*WkG&LL|mxMa#|IB0+&Y%C}o^ zAtzwe%nLMxRj?45Um~P}SXyGl$Vrrq=0y_7YbXB5R~n9Z&@E^o%^fTv^!&R9kJ^v7 zPri|2<0i$ZUj^-u)(N*VK5Pkc1TEiN3{)?i zzn~S+U-DLERHVrbpdUGBdaOZ0-RbAd2!se2oVgHT2qorx`Hjy=5OD z$K+jf5+RAXb_r133|66mhAtB*|^d4FgRZ7lr+!D z(cSJLl*CNY!#xgg%)3woy_eyDm(WCTRFqlDR?}I)BOoC=_N#D5Q{f__*lb4y1$k zc8ZEe4l(i$-1qQ%HGxCrg&Pff`1dS-@N4mZ(-^zf9{2-y&YhQ}$vmAXcwmn^gu-Em zgw4^8)#O- zSrAJ=n@;dWJGb8ozJH_tLN!ZIP<&_4`?620kC>gi9??NEqg2H|&}{`bkY~W+8a5;s zaAVv?mtREKd-Zp8hPp!cw=0f4G&Dp@M-eO*TwV{77*ww4#=lD_;y=@_8Xr*p?1Z@x zL*_0Tc=vL5j`p7y!J-NzgkegNN_u0B)9#7=ubU*6Ks@+k8t%{B=YRvqR6i-(dXRzo zwb7~fRn9&s{^{mC-$7FoeY~X(EYqokaNp~W_;q*xcFGEIz*8%E&3jH5Fqlzj;{Wq1 zpsh*cC;m7m!S?&!%_HvExTXz^qevh{zstChPrs(cDJ>IkUeiJ^}uL-**3w@(94J2mznYVW~!_s9shA zE3~$m?!N2{oGYwOO~eeCc;g1?dgO;!z+c)Io`eXr&Hq)?kkbOK{D7VQ)&4S>r(!Su zPPISXC}6JL*72QF^ss_;USFs<<19)MC;k8%e|Gztdu7^*jtcxI+XJBel z+V;0~e&F0ZWNvr%x0?vwzt}@3rvwLc%&1s-|E2@h0d870f?>>u(f1KyO-ApsFw)*zoVY1QiLxxc?w{e8i*Q!s&WHXG4M3@|nh9fJ|lqVhMv4H6d_+Me}PB>GZQ z6TGlsTmAJb&uJMd$q#47_Ox^_EcG9abtJ=%PaX@N^7qRpM_ShF@$6xom`|7V!--NBFrzZiPXKe^|H-mO9#QxMVl(DY3 z2B@RT*nAX>ijYHu{k=pT;0ciLB9H1jrmF%u$0yKBL>rhl(I}gv--!2{ai=A5IyF?+Hdv(+ypT#6Xuu2Mwo?ee?I4D3@aR=+F7wYi2ZqB(Z$zv6aO(GsprKb`;bZMf z2Z}(4XoLDsXCMDB}IotyLUwd08siTF7Cya4o40odI73@=|J~2eBTq^iMe(85s$ogWmi; zgO?|M|AwCP@MnM7d*BUp8RDH0WRSiN7QfR+ss4MkTc!iwW}ncBcuoku>c@?jAaJFD z2Ya17wi^=ct=UhEKJZsA0uuHr9@DnmpWPoXxF2;FdrW@~m#+x)K;V^@J$!;Vhp`|_ z7NUoqz2LaGr)PaMovzQaK-Zza};OIoK%+@G3Ko zX@K}g99@RO`|dmsm;|`HWk=kDjc9v+{Y_+dcf2oA#E_wb!L0fAF9FP7;QL=aC%`q{ zfURho1|m_Oo5+Pf((WJ!?QZ(81e!uAk!cQsb2~*Q4*1Fcr*W~22KutT@U_T8_*G0k zio;3?u;_25bk{rh6W_yuJ=h3Ap#4Ahu|ovQ68IDitr6|g>ui)1qWBj65rYp*_5O{I z{(g65M1TJ)Ek?|4zfBaAffmNo zv2F($W{)lZ!Ij5?X3k%D^#%mkO}K_x63)&AdiU;0eK2klA(Du*@T1w=WSIs6a#;7Z z{-Kp+2|DvlUsTwv8kA#jazxL+sEsPx)Ciy`AFzn+D4M?|?(OvUgLPuxyx1hY@$41x zyPJkHr8&rF9fW(Dx9lyf;?WLy;m|lX?{L;>Y6QFlILkv8FSQqR=7d41NJU>PzuG>R0q2Hc09n*wRKG8vdG7_bml!xOonX; z!*)2A*@XEDntz0%2A?=Wf{+n_5n!2@PaUQMA{TA)z&m$81FdHJ9$tO)L=QPg$ADHY z8S>RM@smGhuKMREVC=W&S6oHV6Tr*rAr-qi(a3vXtMZbFlE3xXyx_5>4bk#hw?AR| z$AbK1r;@ILdHP0saiL`gL812fwLKR0#Wcj*p~fxn4SW;UgvU{v#}`0X|0lO{_VgPb zt2pxR_eq(CFmJMON3xcZLX6bPo79*K=`^(dwDOTQkO9afh}{hZL4qL{D8QMLEs=m7 zf>NMU|8pWtR~o%|(2G3+K2VwXZEl!wGO;;9k}%+NNdKq7S0YI_@iOy{1_|+|Rt6@W zm<~v4h|na=Aio~xfyKmgFYHHa9C-n*_k2LYflhM`NwNH?3$&5$BYPz3VM|J~ z{Xz8A`cKq6uoIAmdQ9NV5aLKEx!_NWSxi=KA?VzBLA_uHFTT%lRJxGY%u4vcEaEp+ z_q$y0^C*;RvP?$w236Sb5`ugxU9&1sG&RCR^din@77wfVZfw5d>wqOTafp7CYd9Ya z^sfHyAfPk};t_&A5X{`7DNt3!R$W+rh@EBAm4#UEI{54sLJU4S7}j$j;GCZC4k<}l ze5#`dczjn@_gzC^Ojt2_tVBp;$eZYTa`L++^5juk&2tQc55nWYV`Bj;6DtM|{yNYA zN7>nbQ6ajxKmx&{6+uz?`8i{}Q9DGHAV4ob?GGEOMEw|w{5?l>fD$MopeUY@1E=+- z=`o}Ika4kCJeCT!2bh8Wuj>J#iYX89v@_@z2lO@2L~wu$AT{yJSfF$k0ZdIzO+HQi zVpA2csp1vH4OBc<2|y~&%gq8(pP#&Wo(^MdKiLE^3mn*NP?mI$JP9 z*^6Jlp%~#1C_(o$V^^%_!-DQFK$M@Kyu0_{Ffv?Jgb%{fW-ozs?pu5I0#1cq^Dk?n zZuBq@N`rf4Rb$*ykrGPiYx{BDz9l^T{ypLx5a1mqhnqs4lOIhQ5NKBnL)1LmUn2U; zncrV{$48|jP0BH%yZj6^Bl~fPE{FkHoA-JNCGX-BZ==|kN*;aU{}GWO+#px8 z;X^FLj_Au0QBj~)+&2-Zy&qC6zmdw57z)-OaZqT9qj^@~Snds=K5)!Gt^(Kf)XL8X z2&Enc;Oac^__pogd^n!XlLYFISIg|q;%Nw~bYMY9!T*uZ{uFbgc~ZgfBuTLe&lmo- z1n?x!!yX>6ZhIHHD+m($o_@5xSF;I`-5GB!a5YrPuZStRs1fGhg4O`y?$t|xhBRQaC zm;Qmuu9xnPV(4cD5*KMF=zSEu8(U9bzbjqNb_c9W#fr}S&thS@gOQ2zI+`|FFvgF& zOBFm4sw;aZvCIdY$?gHe7lKve_sh~$-yDrR#5srP`^)vgqK}oN22Z{S4i9JO>gwvz z7g+&O40ukN_5rrWe*qQ`a=Ha+$ahcBihq3^_3C#MzN3GBk7Lk0`2b>r&_FjroQfQM z7v*(8QT~S%^0KpBL$WgXJ(h^7vdsy?$+Z zy0j=te5ksB(@49HW9bh50_J95?Q5_7-5%Hhiaz^MA!-@uK?k5BS&<@j0}KO4+&!r6 zcUJt5#Ru?SJg|^8{>SQVBC>;;@WJ?RsmPR4$@&2bR-cM_NZN^j*vgFv)w)Z(R>_cHSoCv0uvIcon?Mg9Dm1}Ug}hW zf?CN+DfGZG4-N?-vJ)T&2FQw}ALHOT@M(a)yn`VL;L5sefni}IximC1pEAA{{KuUJ zo71bzK_qlu2wV1$+W%t5&al%|yiYDBNPaJHyoN(x z6~At6?-jUoCel}zB_9g-C5$~v=svi0aB>1{ktiO*Cbc)fSxG7p-D1>T2^ct~I9`$2_yJ+Af;r9oBPaNkFH7X-xYQo)(s4#YAw zSd-Y@Em-@NE`e|Mmv?eDzl0XNy2AnYIkeax zp0Vnc!p{4VfUg(>x+E_kLW*3^iC!BB`EJwGn3(ke^ ziO6#SzSp*XV`=a!3OsaR1p6MHs4l^>_w$8&3+{+ox#H;+&k8@q64!-u0H>a+GyF17 zUv`T5<{ zSur{UJW|A6%nkaGd7mo(;&u#B67W9NZ;I6^Q??PEv3a*SkNF){KOTBY2bzP3>@%!R zxiWU_gr$&I;^2DV#yE{82R8tx>IJ5#LgV=o!Cu0!Os?$8+wLpl-AJ^Z#}}dA4e(Er z)081sLTVbS2`rzXWia>`L}G*t00v}R^rq?^Y}v2~14k5!y1l1HmT~a^QZ~u~;3wLV z@P|itRTE<#VimVHv33rpmk~gr5Gssw6QPq{E$f9%<_`uGbKnnfBQ;q6?*cRqM!&k)=9G7N%noPY5J#qtCB!2Q(1l)jifqlKzuQ7zJP_ zulWp40_s59Ut|9Ifjgk}O3+Q9p)-kp z()y0wy?5_*&=tbkB_mw)B%J3AS}z&g1TLBnTmz4s!Wqh2VHeE|O}GfnKrY)@eI#l6 zoq3+*&WkX#_mRHk0G5zu~P zE5|;r^%M-_Y$9BC=4cE}T?UJg0Azi{ah`juzU)93*S>fE8pJx@zL`jI5p)<6N8s0G z$9a>dtKszu6ya#WOlA2B@G|5@T_stZg$QFwkeZ4jDJdbb*+}bmu4W<4D_ z0jFSS`jo$wT_tqB^CUrwg4U+}In=9m86~%Qx&kZ6jghow>EQHNc%>7`y@$XhRmF ze(J;Er~c{D%*9^8hT8OLq%hcV!_xu4xmVr?zyQAEG-cDyq9@3$=@gAF-~Wrl73K$g z%`zqTsJGFwoSG&G@7v*0Mdiv|Ue1QsW|T5>I46!?5W$+Lb062ukpes4)W^`Va<}FV zakSW1Cip6SQU?2!xr`#cZCrD*)B+n}R18X=EwGC+gP+)_XmAWizg>r| zW>58bYYtP&9Sz7l|0{Xbq}CqDW1Bf&4>D6j=t&(7mdOHy4%-_lj|4xPD3wi9MGyCdy)mmNj@ZtU%C5jjR7=KhLZ{VUY@sf}I z5oZZnr2x$0m6Zl6RHygNs+4&yP8WB)eh59IYnjahN_^0;;BY$Pz4Ceg1@}wna~z`Z+EwiC3<9Q7 zu2GNHCKC0w|9kImYHKB$zY#!Lc?Hu>2QqbKuK73nX4f+g2r{w}3}L$M;0;p0X0*&_pd#% zyKZW1)KSR9BIwXeV1?iypJdf=#&5&-*;HDT-u_T+Bfo zv(zio;I@N%xy;~PX3UQ{ zZh;2RN>#j!o{Zj&RvELdt{mqNnxJz|1)n{nCv)Oss>NX5aKTB9!-E7iwzc2gXEi#+ zy)oe!m)~P%yzse=T0pkkYE)dT|MpWYQmJUyPYkq%5~&kcBFRUiXlbD;Fl{_M(s3*? zDdCi-wK({k9%vfW;!u69Fz|>LhX5RD`*FF_>z7*KQ+wy?O3>HJeHkxP)sS=2YhthI zd3+4~Ze)m+hkDcT>#3_|o~4YJ>1=#ZVnOe-er6i|kvV!1Aa0uBTkr2aQ`L{_k`Ks- zJFsqslmJtI1uF!(sD|#E)+Pll-}qssyKk)a@!7wlH{GoQEC}sMY(3SH51CoFS;W>7 z7W>R1n#JEk2JQydL~Zo?UoJkvoDD~5ALBV&m#s^yWf>~pzv0*MdVskXCElyf-*ZDI zwDC_1RT|2N^dg?^`!B_%KOPjs?w&9Of|lcXvPUm3N7EX?f!! zb}yvC4jefGP+viaszzhp_i0(Ks>S!meOk4Bo^lbf2e^O2JGnI==ju%GoYjncH)8c9sKko&pK`%cytW%?+VWmfP^>m`X*^SoLxVh@J~uy#jd*O z+)CI*Iyg{*^Bnf?6sK=XBuF#_E6k#K{nFqxZ~*~Y+th|QeI-_JT)-c+AT6h3P;Qta zBCA~D{Fj1kIY1JtyYlXhi<-h`w9m1qWt5nn587-uEEv;ffy$o}L$8j9Z3vFO>5_JM*BMZ{enV-Z#~s3B z0enMspPko@CVurd@_3nuN*Uh%k@%sO56i4-{0r@9Hta7-ZadC27oTcA&a3jfz*%L3 zwO^l)R$EjQ9pMiCpc;9%8-n5;_ItHI^On@ma}nH!-20;`Pb=ePvK_{G#(IB852);X z5&$(yLkk0@h_)Qq%kTOZq1Rmk=a@t_9F8h>cH(HgZue%v0}5y;;3>2x)twG04vs{#I75kLp~T<-2kk4&Dw}b2sM8<5ET?9A zOg3{{$R9>?qFBkoue=EGc)G6z3U*0Rp@=8!W4FTfGzdHV!4+p_Ds@_K1`8FH-9CzUi)RBwE29c$!i5+SkAES5Vaze$u^KgKd=8}b%5}Aztm4Y_%v;9B#v#$%VbcL-T%#P2J8Vk zB^61>cek}PSidDLMWQEz7f&i<&I^DixM<klhTgg{nnM1Eg&zC(23I9D@W}C4BXJ&z#+w6}-B4ttv#=|Sx^-O9t zWyg7t=?OX=&Z~BO&P2(rkGFO@H0_KtE|B>l1AOFfgu`E6MUEwItGu@LZZ|s=I6UB9 z&if_e-b>w(@gO`3qvL;yae4Q`u&1}|MqN$K&mQsiEZHb#>0clLc*iK>?@7F8)3NtM zUP3wVNS`~k>|tk%TtAcp{BnQq=!!&fyA(BeHTCV5r$Wa9A|k+1%`YAvX1c!f$+0e| z;!u(;+3bpNR>NUDu*u%B7b?W-8XMa$fhSp>YN4tFg?gTt+L3pLZIV$bd#A(_nBM(% z{UGd{PE7;y!=G-9n&W!=V;bD7jsDnI^wab{g@Fw+Ccr9IE$nHoOPuCIu_4sA0E{2; zyoKVbxc^(OdqE|)Jk|&VzDbFIE{k_xwDs}kzfDa{I>h*izlMkvCxjVdRE}bGSk?q8 zRj$Mc(w?)7TOieXWBh2hLgu6xt?heOpb`QC=u@k6WdxzOxZdT(LA@ajpWD?g{1%rK}66I4R=$!(StfaHiQn6{@D5aKSUQ4 zJJYgWo%NuImkO_|DZ@Pu=8>b47tb?=ni@hPV1qn%Us0tIQwtaKddNcb1C&e2zIkL{D5iztWU$(4Fb4}Zd^`80{uw$Mu3OExQT)I)fr%}v?L z$^PChJ3V<=Imxrm4hC-U{a_Q()8HG3Bx>c>NZP z?*}qqKPmcI|A8YENStb<=`nZHZz50-hiH8tRQy>k{lxl%TF^Np_;KOXHPAEm2M2L~SMNW!#vbpa zDI)Rfy)1mT@-7_9>l0NSGJ*7Xa~`?c5Vzmqw|-d)o9si-PJSA2Lhh{Qksfh&2bqyK zViJ%Y*f68uRp+NSs+R^(XZ2Vdd>08+N$D7mizci2r-6G9^WsVauTTayzyhYu+&n#5 zMrbtV7#j<9GIlhqKoArmo%h9PyqmFBHfAV&9b9HT>0eY)J9%VCk>emmeSeKiBmKeP z)tE{*Yx`dMHm1f0m|%o>(L0&2`Wp?$jHE;L=B`Y|#^-%Wi<_l@NmC8?hQ^3oGgxCo zo#k@w9JOHS@6~_kh;Rc2cI@=DP}_nQn`Y0=t!Jx_;kT9CP)We5${kaH`dK*^;j_E2 zZm)L7t{(u=PWO7?XFp{l+84inQH+#){Gmu0z&_}j)Dh#dB?kws`3qIx?U`VGK4 zikVg0I$ln(JSbXQVtGCt`SFVaILjhf&g!H5kx9scsC$NRaBnRqsG(uuWN!>f3J8o^SW(--rX?QUhwitD^((ruBUM&vFs@H4qoN?5Ut~1f??B$?>~Ft9j@FM`Ms=nU*jy);6E%EW`6hLM!O=5sr5PIv;IpVYIhDLh z@VMusJ0(H?MEQD8D(K{>rXJ>2o%-hB_GQKK+}R9Wbqp7_epw~o5AzD6aqQ$&vu#d) zok47-)r9wa?egzZ!aXSFu)*pu55480EWdI#6=%#fOSz)@H7OnvesUDNE`7<3Fr5<@+z;GWTxOJch1io)}uy-T_{S-P8 zLKKy>-q1GA8E(BPC@H~Hi}(KGK8U0sjDl|mJ4{+ij7*p z+?{)au4%rr5;DFx!4s9J0-vWFMxzZk@- zbRw1I`hQLqXyBTSug(fV>2D^5Ly(t7=1fn<)EbiNV@ZAE*w-r)1x`m!@%lED_&9<$ zo;KS)p9=0_M0z-|x`1fFaJlMUSxh#idtM3Zm>)4$?GK$60_BP{*5$PaV-rDoP6S#D zALX$9a?kM$FWFBG)5G^l+TMr|ROhT#+yX$%wRGE0-te;$6HyjaI~`#wlEUhKpNER1 z7(e}b&B8QTY`K{^7aoY$-qKA0&`+nNV!(@pHg!6#U}C>$qACoiXMbe$ILh%QbmkCd zK{re(g+uhH^PEttBU`UVgj|C0Qwy~o@70zB53fnXDo`!?MLmo-M&xE-(?l+|yiQqp z{qoaN@nE;MuR33gb9kN~CJM57Z+=QN*(UZ^m(=@>Eqyha3@;~)jp2%-CqC20hfmWr zi3XD|4O)QUXk&z_x4 zZyKydei7uWD~cl)3Vc!NK;F956(0NA2bApB99iF3{f^oT!u)2ql$4Z5>gP`>9*+)i zfa>O7J5EwCg<2iz$3P86?l|5SXaNTnWn4Zg=MWPx8GnI_0;Winw15*rYBZc139mtB zH~au#Gp^M8P~d#^R{CSP3c9Bt0*!8=(&^!A$qT7tP@8o`-T3LJ%-1q3cQMAG->?x0 zx2A6zs?ZQ>oksiVSNj$TKAU)RB;iJ9dd?CuCCAiuY8M}hg970YJ%$r;YBL|ljh()! zf3|wK*D zLzcC#``s%Dw((<7N)1k`yZw(xpa9V69=uKSq=l+|81P2jSyjFaVPnU?iZ#^uTq-9% z(e}L#Mm6TkG}}f5M-359uvH^tSMDY`V>JSrJ4)X-)#j;DPcvR)15bt^zsvDyuzy9) zq-uys<%dREuskSd$KCtHEn1+IYc(&WjH)h?U7I`8EG@*whcqhvI9{IZE>S_Sh?|@9 zS-2O9w#~qZHvS2mN_2)|y086&{W+$U62J1x!`Iuw%5OObDSmED-*=O;Lm3s98hPN> zoX6xys%)=g>DayG)OufkE2V~{=0MK>ettvvwOe> z3wTaOZ`2_ycD-kG-N_cxKh-n^KGy$u7TEO+M_s-gmXw<6E$e!)0IG=7*VNR|u5f;J zL&o5;Yq=s%CwRM@1^on|fk6{Iv9mCe(BHD0x%4AXgR}PZX}_|MF)y?8B8(Fqw4Q3@ z+G<5t-bcy7Q|gdXDtyrq&*Y7d)ZYZ4$-^1bh%>sMY5L_DLjjYd942;~m)?_Q7`~A% zrCUa@*n=#eTQTBmACaJeThlcSmHOG+zkBL;$NlVQ#n-yd)oD2gi*?=Ee||n`B)L18 zhKie=j(vg~?KAQANo?HZ8(pne-2KK(Y~9Q6)j1!yxZi!R&XME$+184kt13Kh<)s-?)$Opj;}GB1KDO%I`c-35W87}{b*Ly&_bHYR&2eO zMTeKmG%|0xrM9#s<$zI!ie-lytcLFu`_Z|JPuX`CmKzOM7!O+*=XT7^X`MLZY1%|+ zM$dhDf|5bZx};JCEXjNxN-WQzK>GXLVH;Xv-LF$U@xwex7P;z3HSoOOn8}eqxMT=# zQ+JRWviC=c_Zu7kG3zqP&ci8>kA)-mIZjU+CMx)rO6qMh9UL!qNrU8U+YvP!^5E>W z%90Flu<9JCP6P#RY7|Q?ml(j|pq@%f?8-mozdNMyUih~9ScS2f0la@_I(^JqPuz$Y z?#NQ2fw%^Ls*?NWB8o45gju9IJBJP#QD-pu(8^b}|73N-zT%&*=&)b(W_6JVqeOU#yT0-A?5Xqaf9bu* zDED*K zdUfs;{tf$sR{833A=j+?agWNtSts$I_zWG!$F9rt0yOrYbvtj)2@+a&|LIcUvBdJJ zjx7J#Ki8utdS1L28EiJbAhyUZ`iyvnIfIz>pXhiW*YP}+jhb5UnY!RQ zu6YXPu1Tm(##R;kQSkW@b&dGurhgxP?E1)D|Lk#RMu3fjWxKsoZypM+or@f%PiLDi z(MH7`j{V>q#~v-T#M&Ysr&&B~>R@=E%4O9$XUe5`_-^jfcmd;6kajszffy`ubncbx z;bd#3rlH5C%|p-1ordWfei1~WnuVcL$%$z$v?7cx3aN0$n{8Q%T&znt_-*4rM*uPu z2sjZz&q;d<9CL_aTRHgBhYXE3O&>;4>g>(66R{=!!w;P@TtmRI6lfTmq(Xucu9kC& zC)DHG!8h_lQ`4mq5nOFvN0x4udD!0Rsg{6i0H5?mL*Zm~QqEp$SKlV*GYoPKjKx_w zR7e-aGKVM%tPZ#$OV)GF#j@>=`>M2i>7T9@SN1zzJmT15#fT83urzuRtT9)RQJy-+ zeJf*f^fVz4{mNg=6ZWd*191^NFmdzO5rHbjnjHNBUeo6OM+R zua!uf{8p4Qs8}(CkAFZ|`7q{i?E9gbl-L?2o|$_IT`a6*yla4Ds!>VQhaba>I*ySwAaDOa!U35 zW+0%segbt5RF^$rEAK9aUkE;-#NoB}>f4@1x?Gh5RF0-}D}O}@zwnd!6J91X5eC-Nd*5SXTH zr}X?M4>=F;15N-tFFQIP$dXBFUDl<#@VBkd02$~31JkL(lMK)~E1+!x>Uo0`Hs+AC6(#@+aUQGIMB(osAAVyt9rk4yiW8 zFt9TI^tt11G|Eh%GJoQ<2$EqK+YlJ`lg5)f!yWBBpMPCG&qgRd9sTg9)&;;Wq1Xj` z_p|xb=c`})*&8}^gLoj2X4vAX5E5%(H8&^6_&%_V;*?(qyl%+eRm4nJrL>~Qg((8aQLQ5Tg&(18~ zYhbC1urN4Kbt|qQ|2^CxWQ$~NGnQJ337=G42 zD>C3pr=u&3x<)6T?CuzNa9S=o^8AI1baZbImRvY+ zfJqKvhSE9kmg1mE-r<<NeWLfGnt6KCfae zrs?HwEv{LcH*9GUf#-)yd1~1S{V1(l3~iaJnm*>SdLn=*Ib_>7WL`qh>Z1$)HA}3S za#X3e-=k$Gvov>6G2RQ+m=h9hu^97d%2lPicV_%}#n|G4r%G6X`WNu-<)IZGo@y?h zsZ-rNozkj3-XVs`dw0iFzp^jtl?ov83~AI2Mi|s7?~+zEHEaxzmKI2iwg1Y%fx_&9!4Z_S-~-AYMhAUByCc+C^p0G) zNJaYA0$~>@$ivi}?(LhnPe_t5q_8|0FleUer%UxuNGYFpORTi|ItVk zAF{4{G2|w6Z0NeZ$DNtTxr;Y$S6t>U?8QM?^7&U1QRKjEbIAYHVJ?0PDvhiD4(%Axn=>+WB2< zy-Tyy1I~_NrAz`e#~GFw85!IA9p7KS%T8$gF>>bc&BA7Q%#eH-)2-jp;cM^G7s6Rv zC-a+!a4U41?8V)Ho#!L-IzfeJIb|F=dJgAoKiF@u&R0@5jryD&R;b>8IHCdR;^X{Xj4yI`I^Bc zqKxgj9T{|39k5fS;;mL32h{R%0gudM7$Ac`iSh`)Hf)XXtmc@k{8! zfa#=Qo5kGJgGNHB)A3~${k=$k$FL0h=S0UJ21Ta#l^PR6dQ68#v!o>LvoG_EW$4$x zj})aRvx3N-d#AVj()+UCGas*H=^rJtbbPDeN!ya;=oNAkk}Z+3{_kV3C6h-%dJ(b#d#4o^x+wVCKHit*hT{Q*0OO*k$=O$Ft|;%xYlJ4-c!h zy?nIQqbo=CwiKut-3cYMpE(C zHRfNp`P7>)al(mxOwcKIf7=Xyg_+c!_8MU20*RGPVVmkj`e2!kiP@%fB9d@k-};8< z5*OeW6qbM+_#_Iu2Wr-XDz&~Ri=KnUTFxIc>(%3p@pne-rgDh&Y_nS^inN#F8L~6d zPA`1WmtpLQE$!8*=B>7FVz)n!wx-WivHq!9yLYWkE~xY}11z0z&bIBfW;}8}CPzwA z%dR{rGg^9~ODD?y_{{*hp0hKuTA`WyM-x3{upNVEINM#~lxmmdE@_=LGAW}B2_o|G z3~4v!g+$+QQCIF3mwI&~ys>wJ^}&HbE9tsc$=K){0*h=pD(%`S!up4{@XjIP53f?& zvMtc!M}b*M3yvt85aT6@UHf=IRIzL5s6y1}Dao$_%QO#sPrhg@Ya~qCpOVBTt4#>w zDla4KeC)<&mV^5vq7<(jd;B{vY9+&q+ht<)@`T#HVW9}|_;EoiQC)ktsfax-Ekbj$18RJM#wjNS} z_Poi+JNePXH1iS-Gb(8mM>Pjebs|zCrrHYS)P`uM)TwBd^0zZe))~4Sma%k-&1zG* zIYjhe%_I}LC2Z_Gjb#U=I?ERkb7@X8%1O4B3t`7)Y|Wl!J^693=NE|)cheS3e$LAe z=$0YH#-c9WtD%{xnce;;qD}<#2QBzaYO$4!o|+Pr{R>mPzW0mc(-O ztF=|QltN5(7BU>^ZxKeEdN3JOUgsBD>u2RKEZZZszgV>~++4pgvF`brUg`!AMSL(4 zK-X}>*!NcqM2-KDxVmos0qw`(9<(?kSg=CMLPw1cv>0B&fT%Ob^OK#gbL`v1AW!k~ z0>{)!(RX3VwM&w)%aVIitozP0+Dszs*pmlJ>s-RLVl2MIiaGJ`2GYk09H?2gHLa}x zr1ZQGJ(HZZySuuC==LHk_2=hMuw{#&RLj7WQq~vYd&=$&8Q_1CTn?FwI2zakZs3)? z0c^~g--;iA=wucQID8JNA91=R?OMbxtTk+Gf}y4tJNo!-7i~n;vUYxS)M`f1-I!!| zdIfQ%c;c@sAKMlKi9E?jCm`E&b4mU}*RAy;QdCh8GdM&dQ46m^-FU^WBC@tfuq?%# zA=xdg3!{jcNvwZ)9!Ynxi}WH8*FIAZyt`|?l%v86a|nab7&8}Ux~5%T>HI^j za~nF48v&nJC;wrYi}|E1!Q3@T#tfE{AEJQ-5-v)dxEMSvg+!D(y#I&Y;*ap0(T)`> z-Hc;dq>He!@FM2yaegb&Vfz=C)>SU*7U#z=ph zpXmJ9*Vh+6I7&s^Ap+Zh$}<$e<7*n`QPmfrWVtm)Ooq#^t?9&2zz>6=G$y-CoF2?u zN^VX7eYiloPBMwqR(R3(a{FD`D~{&6u@qQvL-?r7q(SVXF4JR*wd=;8d_P`?3as(c zlf*N%%v4wvG$GrNhi9xv&-2=~qu$gICN>9tiNe!v{Y75=Jk86S4JKhjd=->Hk2NL47@g#G!PGTbEIItiggI@vE&?<IU z&g*;n&-En^A8k5go21APwQV#U53Fb>M2o3rmu9jaJ$mHW8HY!hQ|PQuv%V}CoQtNu zD=9Eg@x>n`F%2KhwIF0)(s!3`Keq4q#^TkPK(4b>a3W{m-N6dQIR!pZX&#%Skv{;? z`PdMFCVu&SgX0hAz77O-lFxVGx?&w~()DoCRRCu`fqxtIb_Is6P!}J5=hd%oEG*k+ zRA6X|+=V@K09dcx`y)dj>7d2n-3*Gn43{8#TK@pE5t(fgGr=VC7x<~`MAMk+YY^*B zX3s*fT>l+=h$%o8`U~8u%tf>b@g-5?N-sub;6M)u@!5pBXy&Z_*JN9ZA%|bR%h*%$RvetIadUkccBV*yh62G{C#mx7`CSOoiLe-PE^t_1PQsxA8*p z(fity0;SIUe5{U6MA#8mAm0G|g36Z{D_{3xFCvQh{NC&mdlp+r8WKH^F!`H_CE2kk z(7Rt+PGGxv?b`Eu!H)2gCm<{H$ze5UJN~#@#cNbPxu=L2dF*iPvv{ z_jib({~;Ag_!uuqZ}RM8Q!>BaCK=h3ODLlqiy>KKpMYIhb=sO&WsTi;jWtIy3)Ikz zh#7f1fr5Yx)(xX$e+*5sBe@!2#!QCHTm)rE%UI`=H?%kftlR16mI(cYM*)qogkL1tug#zHx)LI#PN zHxN;)9rwUFv<5<^?3%{D***JC9-($hiLrl0pa^V~2zx@?^l)eRNMWxW1oq`1}SUMGBylVf=Y8IP(P|VH7`cZ@m0l zv-`QEiGE4hoiLz2;D+tR90BavSf+yKyNc;&i_G1MqYdmj)*oK3-PH=5OkeMDh!kPxTZTukRfPMjT zA)1C;+G%pU%Orv%*x3W%;n*Eape2d=b|Q+`$HCL<^1@sYIXYRlHeiAad}wKi2yS$X zQj}jx9*Igh#7t7)B+b!?(?K-63}3Juz!*Hk$ZVGe#M5~i=A+==K_d>r*(yE!thsS2% z+3~8;)E(xYQq2hc6gZ*Yw5KFqqr(t(SZ*d2qpr7I$X@d$F{~ zjvk6hfVQ#A}n=5i^v5hz3 zKAsjX6@Jf9Elu|J{81w9)5z@$L!YD=-*Vw7Zwk8@PRS2L#qtXxC$Di6q>5kb^NH+- zM~G;x7@7Qa>6Og>8*!bYERb*5V$gwLg=j<;lL)?np(ZKNZ8&-lGQ1$0_Vxo*;~04K zF1+-vKi?6tq3dcwbq)l*jdem&oq>a63Abq~N}vcP3+6}43;Vkr4Wr^JGqH?1h~;&e zFSZWvjo=lX+wL8J2n#qwEP@qoz18tUP&!Z!oFD4anfhaR-vRF|hMGk~^?*@!1ED&c zOiR}F4Ty-!dqF-9!=%InYVsA-D zcPlUv{xhZ0zdL`CSMZ|pfFLhuU7?oQR|bmZvyErdU;C1ja~%Z~FcSD50d=;w#qAA( zev{DRU^2F&<7K)rIv-WbRTNO*zC51~Ox>u^@aMVLF?eo@%562L%_XiQxyUad$btqG zHFGXbbmS!t)N<0$(3GWHacv0tb{^*WMrP^?lJyqV4-zCiiNLT1K8M6zxJx2_B7z?3 zY(b1fE%q)AGaDIsB7edo4g_hb)7M(Gk3BY0XRLh~C`fo!ur6iXA(+<|uy9Yo>KLva zfY>y~!(n|G@i;~%DDSQ&$Mk`q7hA;zI$%=89nl(hUj41Z~fNhKAS=%0;Kqpg(- z3=v$g$nYaENgH3jVMn2b=SqRN$sNW`@2ocaPX8JGMB$OP4N5}+;H6nr< zVLD8qNMn0I&H!gfet`&@JnP^R>rW?iYX8)kf1v}S_NwPT%U_;rPJo_87x!0Pi$$xs z9|0z)I05qNa#vn$!i}uyCm2Pr&^rS3&1Ts#j8!#A|*Vr^D!ZzRcNRGTTG}eH-?n6R|!Rb2rq!2-; za0AOxp^=Kf59N0c?A}8X^*=-$KQV-vma2|{Z~*<7oA?iD-@NjBBZCzA0dLvlG0qS& zRtp`qP&xtps;9Q#1w)-tw}uZXlR@4W>SEtvu>Xdr7ia$bE!oO0;c<2wNz5y(!>cmo zMB<_~cHrjR9=%=_Kwm(l=mBjulSSHbfjv6lH>kKkmT_-kGPIWK zq%1c{`VdB!s9tOefxId}6}i|Zk_a#3^%WEptRKFew3+?nHp1Tl=)_+Fi+?$4!?Fj& zryV6uQ|S-IT)1zxDUhp6Xmbkgts(`J5PlJup% z2=jftf(;e4%l#{w*Z-sw(b^s{fKhf<7~}m>v4TOQMYg1-9fPjL-GUWu>W2@=U22pJ zSfJtXFDYxF8q0Q$ZcBGJzg}0SbzvjguK}3B25m2F@K6)n{Ys{|~4V8G}FI^Q~W^{PDLU5#=7nru+Zb zMH&KGI$(9T$402b2p{bK^?n4U_(2h~q_p_eP39rmjq&W83nzd0-)eDK2#Dx$a3_>5 z{_RHcH~+3d>xE$`PhtX?S@I9u1Mv<0m12NYUfwp=W&wI@T{}s_GJ=9Y%ex2WN?=6{ zOmflq1NxmvvFZM1EBnkb)@?#`&uaTzV&^KArx!j z-Jsg~k?Jq}8zecx3rXs~jpS`QiSMG(VNjSU1~x57Y=M6l#ISfPc-2NFYFGedG@6Wu zNu%L2scZg!#3_;v>RJv9Bd@Ybd>z3OEn_ifwx-AVfEX~?_5diJvcskCj_ba11RFzO zzhckPc&cp#LfrTVP_d3okcljwdcTiy_TYu>l_8MJg0I=1k42^cpMNwDi(jIu*B?-$ zSFU|Sta_E38z8?`BqPD_a3P4Nc2$gk^K4KJ(KySpJHKI zm$nuK5O$1$dwYojx_3)Wuu`OG!CnacizH>V5MuCd04VQGJeLHOjx8$c6EULOhlLFO zx~MV_cD%mR8|k~qS$XR4wmoSSS^<%yjKSvu+q36*@_JlOQjk_Y4zBH^LxTMu`$_0- zs2@o3Ds}@m2!(@9{~Ca;JGc$Hx4?vU1w%x{-Z;uYnzu;;*8g{n+wh8H6;3gNv`RZJ z5dkLM8OdCOBAH*`l%Z7O@Nx8=ZEX5iH6U?Z#ab;hnU{Kfxb~9h};mv&xwuap8-`^fFC&x{f z5x=(5Sdn^xkHG?v3jfsX9nKSBNIIZ77OS!8dj zXJ9ZM9776qwvGKSijs2v+ssGdI6t|o5s%JoV>%QGL0jEcib;DHng77TdYRt7?Q zgwW_ijlA5*-WGo@ntF{^CYZ7JG$S zXqAUT!|aRT0HBQ_@FCPnf5Xsvj&ucUSaLTsM2+K1fvKjRt}gBhb^o?{CVC%YPax_w zFvi`S?V!kkgE}(^l>n7{z4mSpJ4Z{qZA*|e=Z43{2f!n^$$_Gni6l9XBT=RH0K*C` zN*9D^RS)4%)jQyh`AGKIslX(%Lx9YXK7O-Yx4xF`#{Q-1x1XU`3N26yWGBSS$w5h) zy~7zOp+rv?J6Q#>E=(dGJDK@EEgYmfCpm!H>|x~=pvPUp8u`z41FM?3PqF9Uc7L}F z8#(g+hXIZEg>+{61q%ZfjQ7@yOFM~gWB*}8M(7yC!~fFB4bXDfXaRNwQ?B;jg5CJ9 zJYw_We~(d^714#fzCI`aat9mAIwaH6u<8W%R&IfJ#BIadX&&@Wl&xpg+rYnWW8S4f zHMV2>9&l5>yaE-=C)lFN>J15F4^KSZ1`Pj2hU=zk-Tc(20HhVTbeX85Zy{VkyVUDD z#>x>9=afRow+8p!I%1Ia{2TaENA;#!J7UnoZB!n zuAqmarSrpO&;!K}ZBm|Uo(Fv}B%Z0EP?Z0V=mK^{2}qLa9QAn&{x+#xERZ-CS<$ZW z|FXOd-btu}u^NJ2_qrUwb&5`b^d_nP1O~e!LcsseP54b_oa6$q?|7o32uCQ;L7Uc* zgCLSJf{x4Gyao^-jB{KH=`^N0Y!)ed2xYd2?^-27v|jETv*g`^c3_*t_M)p%($!?P^6|JH8O-v z&Rq&!Z|tG)`CqX?yF40@tZo3drvf_2stB48JjmPv+7C44*}lg*lrIfAG+7{zExs?82(x|Gnb`eTHHtB+zJDxZ?MWQPftA z8$cM3q@q+s7eiUsWIvbmHoV`GN5yV}g#2OCT%@SP6SNgT;rNFDm$qXFc*dN(0^B*6 z08qgAn~>!n$FS_%5CPr;qGdK0#s!jb zF;3hd^o7V?xZU3j{bqo;AyeJs0aTC{8R^^Fd8hhB%l-6(pXB4XwvBP^`oJ*FF*1Te z=v(;JZO9>w@lUtebb&*YDJ@C_%A6iIJH|alDF$#|F=)=awR01F-QeAs_Sn0%JL0VQ zyKJKtK$gKy)giBEf(chhHwm#=rR46!F!Y&2qrzl< zZ1kq#WMSJornj@GJ{%964Pnn|O(xN6CVRcN9Ca4G2s2=~I0&PYik?$~UDgLE-e=^t zn)el83pw0U|A`?LIR#EJMm^pXH~GURj)Bnx^5pE^`?n4KVhR&-T}R>^kfl26wJID@ z#ZWUq$DmkM(CQ6smj`TS0_&XG3B61_Fb5hG^$0s9wS_jXnW8XYUN%Exr?aA9Edzd=SqkAo1fFx6{3;}94EIIrGVGYOipo@0(~ znSFu#wt01CVe$f_bcc+alItjmGaL(d!sjPomp7DgD*<_WHR~# z7hI{IZkA?A5&~s>Zfcc#6h4+I6f+Yd08{B~@5wz>*R9H^wR^~1@CwLa0C*%IMpD&7A!C7ZB?WNd?knC#cj z0Sebkl+chej!BxqLA6}5GA)`)o3dy{jN>+MB!BpKErOZ?Z|ML?96Dpu!Yguz1V`MzCaArrn6 zX$8CiV}+ovM^FHSFv39B3nUPW;1=|Iit%mnOcF$MNYZIg&3c1>_wFx{-i7S#U91{! zG9tl#i?ja-m#zw2P(M)14LNOBmE9#E1=l)ZYIqcEoIc*(*o#$X01mlzc*1!V;p2P; z0s2_#-qmVJ+L{0lIhLm{x-%3;`C=Nj;6k!SvJ_%e+&m0&NWN#E&62DPyJzo&lWV~I zKR`BheZ%bU;mQ57w%4+y=E9RSMJ~`Ei#vu@WBmf&kCw*gZ<1dFdZ>AWZuGp+=A~Do z-6q&uswi@+;8t8f0-y=j;H-KrA~3`}NGYtqeE@^p=?}1TjJW<^Ab^>EE9n5YI#4qZ z7(z;rd2iG|iCDZ)aBMNYpg^Z{%t*TPuCTC>P}qmyWMz+$C{~74p(oc~^C|SHC+JA= z+zJ-+GN9*L6hGmWxIV)Kt;kMAT!$XI1-_Df%Gd6UCA|wz__Z8Zis6D|oq1W^z0c9< zIPidrtUzMb7qD{z@?z#!QQO)#C}~OQ5%xQ9!-slQfK41!<2YiS@_Dbch{BvanX?>lg{l_^=~k zdR=jBs(8t$vp-^GKK<}5{qk91=k72Ek*HJ5C3|Alp*khp4rv}+atqyIy z4m@a?V?P(X|5B^u%^)cWHv87qm>abnTFBSF#WJ;S^ciNdZBKqB`Xydjo~z@#!TO3j zJY;q(bS$^i?)h^ul}rzw<*_SvNBLKtbUWqTt>sDX^?017Ds)Ic$K5CYged)SsZ#=8 zfx3y~C09C0JkknMt?OQ9UNa5Vv@?hP_!ZIBX^)G}&pG#_=V?P&(5HC!z1qEk-fd;U z+QIqA$WzJfj@0C6SJ&02RM!Zg&PHJYApnVBcm z$clk0FLF@(1N-2~sYVIg3BfegOyz;#H|qUKo_2L(Z+Oas>v*KNFRO%A3)GrMd#o&p z*QOeGrDxkGDDnE3l%*5b1KpQ;_ztoOz?wlJb)?^81CmPl{=(THv zwl?aQ8R+FfK~sou|{;M8SH zU;s=m`biL1n9&3&jkQ*~Tab2ekbxF`qOxp}uC$uQ7eai{X>~j0-so1ZledlEe>S!B z`jW&QSE;O_u5z9Ga$YMkNbtld&P^QXxm^eI@I zlvO9bSmKLINq%-JjbQfCRI8u<2^k^$$taE{E$qeBz0zj)FV1&oOmGvGe^0L6J!I;x zCB+lRyJi(xerZ+RKHbi2N$;;EzSNXOZiVYNeIOdzxo4EwNup4eomlAGOg^S&;SQ(A- zNxjl#1~kGVQq!>&m6PXJY!x{JG(XYrd5QZEh9O>_q5)tNhI^{{$Gn94 zFPKO5E><2gUF&uiD>PbWzNv2)A*U>KsFQzrvQ0u1=$i&mm$G@$y247OOy!VDmYcn0 zUk4YCN~jaT=%gQv7P%`fs@f?Zf_}QZ=u^BlIUih;Tz7|F%F03ZOK-fEZ_bE?+wKVq z+)|2*&dLvMHYcIy6a+arVzWF+;>q{D@K2>_2 zf2^%SJ4mz6hI%fmFL>pOZMD0)4^SIsueRgsN%1w{6cAp(X-SON)1ZM7&v2}5SIGGs_A8MKBSn_iPHWw@}sJ=doyI)3FFABEg*Sx>PyDv=a|Ro zNn*!!O^H1zouN&sG?ch(GK|z_*=U@6(7`X$mdj{;15@xM*ZPHNc>ORYpJ=b89GkND z)Iz;aW!o8qzLHV@Jw0(Hf?Tjuj|LWg*L1hWB6_`eCD64%5g!Nl&I)|B;yZ?Owx01w ztZXuF1oDJNidP!b>oFO6;lBDyzT%bS&&H4AVXB$zQfdKtb>(TnhS*7G0R0iBzm(24 zmyvWcaVcxPj3;KgXCxE^J(F~zU5$MySxH8F93#WA5)X}cWn(3jRCOVRU6}a{Xw(P> z?%u;xMp9DL!;D}e0RxPc@sZdb$;d3G@o4Tn`*FsL0l2DK*{7NccEwcjAQ**ezimh{~vzeMa25Kf3e4ZAS#cVqx8GQIgA!>Z_mO160+>$ls(DFrC-UK8 zINoYdj7Bs@KzouXqdL2K?AfX&lg6m5K3=@?i$e|J0@0E-*~Kj(#ai-U?MKIDUct)I z`rc<7POj8lOvw@_*cp!;RjAX@Id7X2dpAoQbuWq7WWu|sXFGhsty07;HRLpO zf(S-8v_obuXd;2d*{NafQFhZ3hXJNZH+n;!B9;oNx!hIXWgu2Dl5Uh*Yg{Khyh<4l z8;>>eF@E=oC$Aoxru(~@kC`9GGm^TMx-Jm&PyzBX`SVAS;ws+vh)siLgK+-6d`yz5 z$c zkwbRT*j&r@J`H?6zTDP=oHsq09(xx9Sh-wzLz-6IFB2bf0T?w%6Iz+_LegLzhw8we z5q2oYBqxkSrLVC?Y+rG1z&$n0kyh`_tF6gze)at$pY}e{ak-?Wec^fT-S`!v)`sCY z+Ow(E*?B5Jefrao4wkEI@Ee7BcnS@2I}D~V5iWQ3g$&7~i`o&&d*+iY(ULvCmSJf@ zm-;cEyZmGx72BhMZx!58`P{dKllUt0)hZcPi7<6aL|op6 z3RoX*bAShNLJ?uQuLPIJZ1z-=S7F>W!Hh~5Sk<8W#`NP{NN>0#VrD!v2S(ZSE^}bA zQ8WxJ{Mi!S4Oojf*Kuhdr}kcX{G2*h-Xb+Kx$WuXws534T(u}m+a|rjb40bvVdIp{_ga8Wi2X{dn5os!D4$*#8%1;MtbA?PIod zV@C2`$!LPqr-lQm;ErnLtm%j9V<5p?RweNC5xQl+5_{#jCIyGd4?_GfntM2F#xC)F38pR^@0vi&1OtOe#-s>^8t2k1}s}EJ?8SIHm|ByWQm1hc+87=W%}MPcr_h+~D?N zCz)WJPYu@5@I7 zS|9&DvHITL4O`P@EU6TmU=gbd`L}Qj6Qil}ZzMeL33mG7v=u!SUcOW$v4FaazCI%% zmnX-zzio>EZ%4lO-Srw}HJ>X0?+N+?@SnyfaLtO-c>9gdsfrVPjS&(&j(s6jk-s<3 z>|6X<1-#YjQ1N^&x^M~j&=lb40!@-Df8&qlmks3+q~H#k&Y&FN0+g@i+kEoQQ+`*% zcpo5xf?QVSV*7w42>}__0=K!qnd6>`4SJnXkt>oHMb`t~ngMU6V?-0Wd3gFpe4~Q@ zIVH1j&o|#hG%rohS6oT7%LM+QU)Fdtti2Wtg%OC)E>t|!2%k?F@@g`cnI7_@H}PN- z+Ie<|hyF}O>`=P0Mp3$nEQ0bshF`sSUu6ULdk&;44R_NvD#)5{CqDmc65+pA%fMGW zUVw@yy5;*pHu=nn|5ky3-pl_B?7dK|$n>+s>4|VSv$U;h-_}8C|V9WnW zCH`X-2WDh}_MjVa8CltA+jLyP*z$ z%ELcXB!8^`JRq`jvpaeof#C~wCk*~SZ{Xc#_yOpf-Jcu$Wz5UFFZd`O{l|L!yZqz$ zwrAbBcNYyG?fGCZ-fgS{NB`E5_&2!%*fqps+u&}W-gxR0)Ga%FOd!ubY4gFIA3sc$ zO$&I$KW+m0;s509z~A4OE6+pQyGIU=`tuT*2i@_1ytV)SbS=XWj;D2V@Wh9t&L4eG ze_rLE1oZvGgT}$`Gud(T8z1ig;9|NI8t>oQ2J zCcq^s!dF-IP@CF66l6cV$Nz2D-a7~+yztk$;-{!m590w5~U)X1Slo>v1z6t$5Cfql&!o4;?(pO&X{sV<{ zzxt{2e_VifclXgv-SAEfspRD~NrL_sZUEr#3r z#s244@ZkzyJWGc_iqMvwY40hpKi3<`-v72NJcyv6`)K-ke+*8>Dos_fBrX_g1*Q7OGxYeeqC)yk^CoW_U^I%oznhyVzP6S z03Kg;;ivZZ!PXxV$KG@He^nlU5QfFGPn3VeEr3qL!w16taiiqBynege&MflQA5vh} zbZrAI-3DGP?7?@RzZWc^^j9Rn$;p5Gy79w5Kir`h2{=WWw4~%MN{bG!v0Ut{T;iA0sGs2Ixp|1TLoZxD=DRb1@)H^t)Z#6 zUC*`i=)QFh>cW3q4x-xu%e{4PI-*LX zf#ld;*MIC|SvN}%`t=ZZx;Z!&jpV`OUv)n^aFhsiy(ew@gGRg;{^{nwJPMGgzbyvz zs2`-jf6Fcs3(M>m3jB#ar)HMw9lW_O9t^rfcArs>|8kQjkR&zDY|#G;KIA^Qh4BHH z`S2D0(HLAh@;LbQ3FMv9$t)YEny+`8Vqx)DK~uQ&r~G?bVCzoI{~`q>zEbXh&ByBj zzUtrm34dD2%ma@x>`U{mH929f$PvSHhh-7Y4Vh6ACVskw;L`-~PO#zU#P4NY1I;Rb zS=tW|{_#J*+r&qkPc-YSzo3h-I%}O_sSjlZmlRD+$G6hn)3Q@akAvU3BL%(7U4+u1 zUv4oGmirc2*WFX0o@Ey+=E_qqF6F=jbKkE-A@sTsHdHTLyvpSPI z?q#PeoY2$V-wvR1ZZ&veHkLi|}!4YVp$-^I8`l01AFZhEF`cFOA?;qqDzU5`K za5Cn)D+2B1o*5{0cqA&)p=)h?B2P9(^Gj)N%T~(E#Hgy^g!wk*nq(=wkc}QPbsMY; zpJG0g<=fY-Hjo;eMGSaau`Iv8LZn2haOP|_IGcc7!ky}Pap#0(43n8;eA~2U$HvD{b8 zw6r1D)e5WcFGUirDQr5mT{?o!_Zu1&QtVP&u``h&&WNbS;q|P^l0ph2*I8&LPA z&{DY%@n&0?2zhnt6jdl<<0%Eryke>?{q0WXjFG;dx3#tFa*dosvYNwmJ&WI*gnk}| z=`b1cCrw6DTu@kluxZslZ3><(tqu>Eq3M0ThScZJ4K_mZlmqm_c-=I}8OP{I9CiM9 zr~H%Bv2vd-YoNqvH9i<#`f*QkCZ5xX?(lbWS}L89I@WtCniGZ%t47mih)C>EoOODE zxbyN_N1ASwh-2EU+&mM>S8^>!2L)Cyq%GwZ^A9R>!v58%7)k1 zC%f9(?$#HCX9)-J`kim#y*|?~9yFH|fhMI#fJWLlsMB!jV#HVXu%F8PxsBg#gMYFl zEuX~b1&(KQAc(_Dzwb%ijEbU#bUx}mFfh` zS6CL{xarhFZRb=K^@x<(?>r4B1f~}C+Snkbgw+Ymgaj}~j#uvgtx@A25qc;rFtc_C zh7#Rc=AsWrGNgJ@{CRo~VaA(5`(n+DF;)bw%tM`dE-Wn$??@!!<^EfI9~%fAop_|yJs+P;2jgy`lt^^1wd z4N{p8jrxvtv_^|Ko5xX^uXFD<2>V1uBlL8!@H2`-#Gc+_V1CnE%E5nluvr?@6?wxi zavg`4E*gQf=Yv7tD()tDKz*U~dM3#se5U6ZT{zFAiA7)C*!eYJ$a9GqFOwL1P|%jT z&{*lVlHZqIJ5L;xCXR@N)ge+=FIjY1T-2rk!?=QE53MbK83ps+d3NW0=JmlH`XIm` z`zghrEh!HeA=cldpw1XTmZPGvvHmw0L80cPBIInHG;c362yETS1YY&A$0B*WMmH;y z*0nySB6IIH$gRey9Hr#mZDnCuXLXOS##-*^TY#Q0+fYj74*An`c{Fvq>)EoXXP%S- z*~MSbNNTJT3i(w? zg-fE4X0UTdZ4P|`3ry+WT9HP7=$oIz)6dj(*$lV#F?sZhe5-qkp5j>Sh+mk*d< zMA3v{Okf0G?Fz1h$0v_>_y-L|2)WfEhd@l^1|Dy5p8ejVzj*pY2Jmh!HM9T-(FCVv zjE9Av!!l{2s}_Skfq~_RHNVpdz)=*-GZ;w_yc5Wf_?agTa25n_>X>9h0E6-%BMbDVeqk^@E}Z>PR#|KPX<{<)5BeXSo8eSe zo!cd9HF(vBFVUH_N<8G$rORS;Gejid$}?MX^iO@3!+8Dx9?)1GiuZ|nxu>VM5c&=a zLdznK1rE3j1DbngXuZu`Yo(M`l68wS()rUwFt_cwR@!Dlq)Xj|E-A?STV{*_?_m;Y zYbH6ceG+4#Gy6bH*(X}imlh-=m_iFB@b(M*s)7_#*R|i#B08^h>6%@0h2Cp1;eEZg zVxq@-Z@m@@D6?*EGWIEce6pr*CL*nQY&UZZFPwfZJ=lVsy!z_cV8r5+5L>CrrY_n5 zOea0q^K(y?Ef-ILS3aO^Q31X4n8?o#e|-Q9FQ5hEJkSo8uUrrA3A4WWEf?b0H5AFd z;#`0c#mRxPHNdLm7KtdjYeq5yr>4yHIt)tN6=Z*$8YrPLtX5JiMV#x#d@#DRK|>L& z(s8^*F~H<>kqBfLyraZ8y~+}hN;S{9UY8|30a}OSVJ%Bwb-5@HGc|mhKY+o6^J>bX z(K86e*>uZ<*2ms)&Jp=C;}I#LMZQDD z>>YX&?XHVPBbpOs{te!u>J%VaAGZ2e_e^4(T;KjU*tk@>mab<#)x zj1u&QP$$YrYqw+gDtRSy(ZKI1YqS{axSNQ{^|^<6$n#Be>8j8U$Qp#DPT&#kJ~Auer#EK`H(pX#xE+9R{fd%a76hw$>9d7y6n=OrPshP zRR}0b)XkhZMNTx?lp=6lA48K>1lDyd8@ywu2qh}bTK>f-x$+39F&7EjJyCej2QxaE zV#5qFsL8SdOS}hj(s+nn*A>mP%)2}k%Cac9;#=57>sks6{L;qN*mw4`Jtd92lh<0j zpuA2=!6l%W4&UDl!YTPXnR_#mjT4TA?}(%sgZbG{73owr4QqdV z>l;6*54iyTTwj1VH<-sBvLn=ka*^F#2eO5!EgUD{z2$s499xoly6&bO>DIo|3bB2(h8%k}{L%rR(gPtQS zf7|FjUAlwvt|;lWQ!O?sYEgGK{;Wq8{6C;VuKoN5{kF z9M5l{%r=eeU~ihVY+GM9>7ZMZf_D66;6($v46SgTrhPPM@;Xc60=gt0ap7GzA2 z5AvV+Lz|ZZe=kk3l^0v&0?x!qSLQ2(!+L*daHHv|CQ&nPscOm>sMcrCBBEDnuYb zC$tvRiRETPEIV2zJCduZ@TImMSLlE z&3v~hH9RpIQ%47Rqq;dHN(sLRG8=rABdNEe7xz$qfU*t0%L-nif5YXRJF*^&pK4HH zal@9gYMH4Gd+MVC^e3|_n9tN3hyimm-DLV=y?(^InpMU&cD6?M;i^$6H*psGog3%) z+>=D(Gq9U;1oCJ1o$s5(CFkoX7^01zfoZWe-C7WsJ0JPthgbEywftN}Iqa_G*vg@Y zZ5DvqW={u5(HSqBu){09RJf<9WSHKfWDUxNq{+0=YKIMFu9@-j$*}P=^zf$zTj6&=#UolkCh@?XOb#?pf|y|b^T`lEUU?!UZ8+K;F8I?Wn2W=0wG{oj+;g+z z&F8s*hEo85Of7&qmvo}X0PvfnZ95$329OBdw98U?Ud~*>UaiC;x7Bp}@cq@fZa(R! z5js9eMn1!VsW?-UW1WF+#SmQ#82chs`2v8f7hN@X!&%b0HMR->G z(wPxl<}QJNp7<|RczAg3>OMK;_3EBqB?^p-qEW*O!d)KBKl<|-lbER zk2+M~65)$%q{*=F57w)b=BSiZ_0C77_<~y!TBH3zwSx7CQ2U$l>3h%b>BSk&c-^3v zDa1gNzu4C0@t?%h?S6zfT8cQ=d9!ZZBNw&1V4<4-NyzxEeb1+TLg^i&k}l)FmEArw77o0>|t z2c~B6IE4~Pdo+uC3RfLdpJ{Fwg13Vi_PT9`Zf4H_*mhV;QPK{_$vMOaN=wETq3UNn zhO_U{?g-BgM&PtvCG8~Aicn)3&waWPI6YT#sd4&GmYsERiW3kJ&cG!BSm=-iI4inc zqs*N51Re+wx+=4=tG0FWBtWkkmpo6fnm6E@Q3`GiFhT~G$o8Vk983V@~%8%`#}B9rrZxA?uAb z2B*#EbC=0=S=N;eF_{IbfFYcgd0XW|x|Csg8Qr8v81dX60>n0MkkEjAPx`;pp%RzG z0dtnxCK)0E`qfihVSMJW?X&e=b_ziLYb>Dp^YG%$=>D+D>&*|EP?Mjg)HE{`HanJ- z=Ct7qQxj!X!kQQ3{jcMP(Y)1pEDC`bXcJ93zka&g4$N^|ej_WGGV1seeTg-iBUk{v zqB%!sjf+r}f2aw$#|7xLa&X}0RS+@z2wM%3`GQ1|RVQ|MoZ>4M_4cfQc4_Fzp2<9) z0g#Z;ujzW*UT#Ck)TpRVr1$Gem3z|R@mXP`zr56dpK#v`QUo`wx^H|EbN)-lb!swb z&`c83Tt4aJYEz-FAUkmpd+u5h>ycn)5bkQ?BDg;rpFFM!zioP98t1i~x4u z)1+kdp-hzjSgyv3u?Ub}_B1x9$@j=KUe%leUyMkZ^GpUIf#Cp(K$NuXY<<%;T&=90 z(>A?3`USC5Bx5|(=9^)uW>(VKKYj#mLBkxvTcOY3T_KT#cajVG5P%2Vs~D>jKo*M2 z5fsPUjY(`v@=~pW00)3!UB1Hk1cynYRhoon5?3QW1$ZX=ja>MQ%GI!vvX#Mk+_St} zqxqO3Rr96l*l=8Pi035;yLKQK9052h;bs?*)4lBv%FJ=)B}UJT-9(f%>lyU^D(t>x zU$va5ar5D1XVX%Xvcic7f0I=B{7V(t5U6EM8<5M)G7_)2M@qN16NHUaNj$%9zZ0zH zUM5K_A@CtbzS(O1g-A$;f?c-svjLFC=$h9QzjVmGg531uP0^w(%SH0S^os_%3#Gfe zNT%*3Hj9&Fd$KpkJG_q3q{#{oYi24l6Aad_qbD=zjY~fjfS7deSU4|B+bT2h%S}ko51FfM)U6EybCUzI zt`nyyok?P9l}$%Z%vjINX!%9rqVZ<~B{W9#e1z+O~!rb%R z8wr2wJ<~WFfR2*$$mNvU3egPcCWM@7JR@0Ep`cC5-a=?(MA8Im6!eaG$&w2Jy?Ip& zP@7I0@S`y^DJ#j624qOwUtGa6^PF!+SMJhR@B36h-6`TTL$20%u8}IGIkGz23(+(t zHrUIh%lYb(0Xh!YJ93sG`N0Bd4M1Oo5b}qbY7rR zkle)cuESAa37`RdZ~j1`=0l=EPx-}zi;&9l+lsOffw4F{1GG=yHhyA)Q0zG( z5taqCoywrX&6!^>j>TIS5y=k@nJ3tyA#V8GlR3=u)eEM)C&xf5AF!!c0TPNXY=;!A z@dU0>oM3FccWK~4Pnful zttY|bJfStJ%(2>(WX!aAa(U_aGzMIfZ0|9`&a#MOSYby&F9d*U0Wn+##uYp7%>%t% zeo2_4u6QJfsJ4Ux9MG)i)cy)IjXg2}YOcAOlCJqyuWFuorF-x#xi19%ft@zcD3N&IiFb zs6gGb{z5EeYvYV9GX~N^{MqKR$8WMR-0NpmjaM9w_#-L5hmjBCaj zS~R1~zWr^y&&4j_>WmGwT8SU`_(Lv$zO2ve1jUV*gUfBXv(az2Qg?3O&ICmz#N(#V zw_G&8KR~MWK~{lt3*5`LsHx^=e3#T$BKFCYL$;V#LPb=_{x z6-oXjC(Hik@w+LVAPx08_r&C6=lEg#4%Kpzk7zs!;kIHLXRH9p-Tsy1?YT>Z=)EB# zU;M0!Etv;e84HtJim#=tY`Rx zlGBo$lI6dqwX7c4HCB!u*X$DVF>y33}d;IZ*I zB=W^JbF;#D1H&gvZ(2ca%)nvS{D{+1RfgH2`tcruYx>hGZ(Uf2lqLnXc0CA5uQ->~ zyrF>N`BbuVb&Axv@-k@sWu>puGATWY5$6gs&kdsP-4rgcD97k4E=;H(K}bIi=2w7H z)wBI-+r(7vGxfHE$*`$`@LO}*Tv;XVTMgOv1u&l(G|$JGr#Hvi*jXDqlE;@zOubZH zY**I*ePH8knvK#rNQ;_(^EmP^B$ROmCAh9C}OnUbxXdHJ8&p1E!DEqL7@st{!XUaQp3mE9oz&x#YA%t zZx9%vfr`c;9}VA&SS0frdo>*cLMy{a#&SNP7tY1-3QPeVEg)`d=^h=#N0=ipS)SM^PRt!63Qa8?tCLNa5uRlKei$6So)I{t^ zRqrY(9eQ*tZ}V2auxe&WrQhu5=YfwAAY{57scC%u+2+h$epkkS|6T{NVDswp&+H7+ zps{r$uL8R|!I%o~iXbqaz}y}D%c;C9u|vxzFRuJH0m3HJ)a5tcGdI5c_2ZdTf1P== zu?r!9qqFKn*~5HBhM!3pA^%*TW2JLSy38ZSwpU+VZeVw#d;2iv=JONCPV?s-t1N=~ z>vg!^gJk58-)u+e)0#Za5bI}yMG5cMr!$qE+D8N?q|SnWm0Yf!G=fpGxC(CQu8cjK zw)CDlfFttE4{I;qZocGQGl`h7QX+&HBs7WykPn#LFv{z#hhb|t4WBllIx;AY1n0;l zY$d(luYPqH1Hx%n^69vol-cq?-@>fg>HL%eVw~E8i{9V9hRiZ%LY=zP5<5fLa-&g( zv{0{RE}D&OloFcz>U9D(Aolw{J>eoVA0+&{Stu_8mYLPCq7~)I+{-SK4IS!6*7Rs* z-!hg98-A%yN-@s&lxbz2x=Gn-897LyIY)dh8|XQT3s@)P4^ul56Z>tJZ8x3IhsG=v z>ikO_cp^!gxOXa!xXvsc^j>-3jd=Qum|TzebH->K{QlqZ?e~!4357fUWwwRHaD^9! zF+sANTJkP`-@;*TYE217DYRLV|AH+q2l}bJFYp1)UA%EL+U}m*8M_W_;-kp};!kQ~lx#|efvfWG(-zqmb!ntzS7_p zu*xptwOsdCJ2~0zW6&_X#xCd+BKJu%7rrr4yIb^|%@K>?(4mo-i1o`$T6IRX<(fU_ z>jUV8I?pH6wgf5sVq6NG5x4t(zD-m?f^gZO@7180J8{c;@l|3%8W@wk*ZP2^*!1NL zF_^kl?jZ+O$%pgt{^L|On}d(R=>ee&!wyifrb2JGvGI>B`kQ=QqPY5|xbV6Sm2JSO zW42_DR#^ww+kaQnw9d@i5k5ZKJztXsVFpOG^xE>3-?#hEb`(POXJ)%g1243R1uB=s ztWZa~rc%mISg16H(_r^Kn-|zACp+L@iG;?Y9l-5aSg$lFjt^Z0qF5MUK?jvS=G;w# zM&kT_sQ4wYB@~r*(Y7w|aRv@f_yYA;BR{zg6&n2jdoZ$ARj~5Np-ItFt$xZUDjzcu zNp~W;1^y7;LLTlC5K&GekNoFfj5aA;bjjqGfijs9b)K}W%LEznCa*ZbKKfEI$+1kI ziz--Gp1zz`pzw~UAz&TP_*5p5ez<9w;N8<6dA@>`X^{}tPcr2MJJqD+WfYt%8QOL8 zi!6A9GY4eUcQudiKUF0&EI-jFFefoQv+Q`(i!zdR(+JaN`$#VkCc|n&{(I;4)b0xitnqZ zxp$xwKEMbL*GgsY6LvkoT7l8*r~c9x1{B~&R7T9ezxu&2ivckI&}7L$Gu-Rv-xC$T z?nmSK$?Ay7AXt)lb5lM4bNZHML0EJR>-3$fLdpY9a}6kOMJ#yA(1Q52QbDxk_%6&v za{SW})lShUOW6neyl>bE%bVsL2~&=byngBT;gvnnchinRPNtF%=C?f{?Yuc=^~=la zPnSk>acW4m6vs9OH$zbvblL~jz@%Cyoy-FawdQDS`KIZ0@^rJLTmVJnIMM+ zP~A>evRQvEFOyWqmKwh+7Ik#Q&lL9aE8EeDLO~L6qg|Z!m6^!F$wA$4k?;33g?;aBTHBy9& zYnJ3Qo!r*P8t@)LV{K2lOirm?f#0tVg!+19`m^p?x^?#a@@U{R)%E-W=hr>p=mFO9 z`69IktBrt=(wj0z9oi#&npkpqc4hp9!TKq}bK=+cbU-HR;=&!{d!K;w-a_a4t*i~F zA1cSB@4oZbgJV9lyZiy-ZQa!4b$8*x#icVj2rzZJ-^Dw-<;M2QI-KFyx#bqA(X&#M zEmCxQr#^?fQo6c*x@qd+W6xB$NuB2LX(#{}DG35B5aS?f7!JE|VNDRn&Aq!pi%oSP zdV>rmqlRi$h7syg3MTHKzYA(B8@m$r7td7Yz;BccYBN9~E8>+2wNhsFaY~=70aTM+ z9n@nJo!ofc=rP_bpGaB*jFL0p$dXHsMQ^qcm2@5212k8)s#@KBVny+<{B73h_k#pt zLig!olCRfn?N+kSCcS_h!-xddTmGmW2yM-9G0aXaQn-6&{H4dkD33Fw`j9gVPj4d5 zmGN1krUK;10ukh%b(Rl`IfU~pV9L&Go(4qpC}Uc$i8s9Vs$f^vvsWW3mWb)m^QH9l z+a&vd(TrgxhGG#!^N#2e?3)Q7wXh0FuPcY zyi)Nk?zBuTV0Hms(L=755z&dI&Tmf{oSVs!4`a8=F1#eeHdR6n?A(5JNpPeIeAA8p z+G*ildZKynQE#L%&S^ULz z7EDjPSJF~$iSb}6ZV9mpI<-!`t>S8%o%vfw`im5}ZnjWXG;UHt6X8WZ%<8^Gi9XjM z|K+noOF#W`QsUbq;lgEV9YA|Fk1GSJYY2w>%ulHSB3)(&t|BgzTg35U+H@Gqo7T1d(7;Hd?=t{LRG?=p}o|3nihQc~8h*h1;WEZ>4N@ z70fK9^H)>4t_}@>-#`>n;Ofkt^GVyDv(7*-)I|Ajd^*S1QmH9mrYut< zNy?Z1c}XEeT)r}|)mgqx>ff1jEI?D@UUJKQ_`z&}o6xk&rPR!8*1?8U9`tLwuUN!c zWb1gJvyD?o#ibOA}tbeaFnNyI>MqV8|T_=bZb`8RM7-@r_pE?(8V8w*8 zmLO`=J>At~jvg19cSWtXY!fSvOhHdIvF1ms-po1?FBj4$uaPoW_8S%JK)#z)6x(QP zC`BiF051x!d#h-v$mJ)KZ9=l4?9?mU9^v!i**n`#hJ?&k>q`&<_8*OvT6%uJ!JV}+BOZDekF9;)XkBY6CHJ)|DGP*ig;1^Jl#!Qj)eV?3t;~M4PJEb z1z^AA>@U+F2X#~O<4)2IqoF`TM-Vp)?7- zx^7+AIZs_XWw4kE`|-RQUHDz-~S!y3Cx1;d+TvkF>d{n-7LUgEQMaV96sZt2LaR)xlF|k%GsT z`(pc)K{mWP&^nL|8XkzQS5AJzPhA(%fg9ID)DZx1g=q1lWZSc_Wjg&Zdm(oWFGd*K1K7iv-dtYzek z5rX{y4dAWP{T|IDjYM?Yt>Js&mqo%sSBRu@UNR+}qx8NEFd9PlI#jFjZHKw-TD|A- zf}QlUP6a=H5;L(WOu%$o-HFB%)^2s9JVt(ivX5JCHJ@B4e8#9jra7a#92K*v1)8Ik z8$^2g>AwwC)K{m|sA)ah_rMIS+2y}dOh$42BMtv+A zBF>+qLhhmoi8Eodv*DYi;3Qq3sme-QOJ%Zu1f;el!BO(Ja!)0dS?)N5xuV&gva=1h zw>B=NFtbp?i4E)(^Y(gnv3{CZGk*lYq^M`7yB3hGWq5EzozCn9F{R097AN&~&PoZI zyxq+Fz-X$zX~0gDe_z5a;PsKU;Qr+M#p3TV=j(#Bft3?C2n=+Xop8CDZah z19@w^Z}#l*iP-Vsq}X1|6JCEwoP6B8@hzA+3Ql`x70SU9PMZvzi9vD*BH~(}E=&3z z<$h+F!~qFa-iUja)P7o-9eL+~k3RcBQ<33lsxu1&6|+OEsN1(sVQ~>*cLH-X(Cak( zk(;Xv^Si1mqE`E&1@_@3!K>GyVWp;~d31>c@z=vBKC$stYGd?hSYO#7uk%)r%*vzA zo{E)UZ=Wv>BS`Z((X7!|VfvQJ#}TjFm-T*rJu!)>!SKFL{7y@6rI5J_0@Zh&2Ew&O zYD^!gdHAh_fmIo|lO?_ODuFL#Sgt=xv!Y=r-{#sx3 zNcnk^_6+U(5-WZ?NbgX68proDPew1nnL@ZIpU#=KRGFL~cIeazAB8GIY3rS?s|C@H z3uy&mSwR-S5iTU3E{^=BuaRM_><{Gk!XtS6VU@HCFny-BiwJlf8|fQm70iXY@oGX7 zpR>wfEtIN+8*|xmvJL_II=zvhBg+b(`(QdJ*(H@9S-33|Cl26#fFjG#oOL53L_c*_ zddJXLpoQ_>0l|N5w)!N4M6 z>*)GKPI<**mXX1670^vOD}QI3Q+n_VmGuG8Cm<-Yn^7xgJX7IUq%fllmfEg&^7MXo zAIeJ$4REZ+uz_*v;oVOe61^7t{Fs9+UW$*3%MQ`Y&v(k1jPH0$^x zDm9F>Yc0KC(+I0Cs?>RPwFY=H?A(g^vD~5ApiM?W!kl-kc|3%m#B7Vr zF;^SS*$vkU&=Ih427Hl$rI;i;H?8xF@Gi<}0u+2yEJa2xg=yg|EQZmhK9$F)lAbVH zN#Wy|mN+Qde7MzOaLeG~7y8EXxHHy+dKW$AULZb|`_H{2nnnk3Ml;|v=s|RI?omP} zF1so8OX=n%N@ve`Wt6^XeQ;4)kQQB7YWhR-bi|AJTfSkAc&UddK`Gq?H@M-PAx9qX zrrz9@GVDA|z`}ePQcwBS#xC<$YmlZZFTA<0e?w17(fca}6q+p{QzCi?`k|SmoG%&V zeVnpVm*XrDQtnvEy;kZ(RUp0Hu0X2se>mKvfTM#}hS3qF{RUm@>s>5+rzUMcYYzuU zF*eKE?|0wdyVMq2Yj!9&hZxMKx@x{9!-FAe?^1FtVhTYh*z##18pnB=Ax%H-FBd6(_z3Rqu=gleFrLk zp>`|Iq}h2CL2zTeQkMG^BE}+DEAK)kmm4ZNwK=CKu+vjUot?0RSh)rQ{_*+WWSzs3gy@YwHEPNl@(JDTLo5+$W2cTb|)&yqtmDEy^)LWFHc>YF{w$wg76 z>aq7DOxh(yERoX?8&KALy~s09d<%LOF6syA@}fnk`KaY{x6IV`uGmj1?rjw|9MC1r zs)*E71k`KrcrTA>$W)5}V#wT{RPFc7YZJ80xdKYyk?*M(f|GL*}4+hJE6K9)VVNX4)N;u{E-o1QZ7INRW zh4H?1>;suRc;p3Y>`a+{LDfjhnx@?WtzT9t?#!XzBe6ECOXxZ-w~N)2H%Y8VS(E0y zRb~g82emtEvh1+M8$nx=b!ixBV)_#ff5lC9CU%n%PcINAVY1PQS^y^M_vtUjGE+gF zfrsh>#6HwcH&U48XfT;c;=Zc*4DifKcQPAMT&CNCl!gSkbNyG9nW6lKC3cr^Zs{6b zB&e9rfh#d#^QM-~9WG0alPT8A^1oI#ulhH%W;u4*Xru;^1g|1UKSxP?C-W9GenGk^ zYGiA#%UjTE04g9Z5&FvB?vtugj6#?X=FOgj9)35So4w&~* zmh}m+X1kQ4uU9_a1r_dGrBk1U^AzpR3LzLcA#4#@7=-^GTb2EEf#wl*!s zGv^2*UJUXWILa2#W6uV^cFH6uZ*c_Y#ma@y*#I1TaZ7dImiLRckt+G~aQF`kw-H5j zeD!pN0S@G2&((qX_7KN=R57M>kE^}ZOj~J9Sz*Jnr7)MFc+mgQjAm98FC#-vmUkBF zd@CDpMLylx%Hb9fiq3aL=&zzg=L4b^g7~XVjL%fQt1GXtdk8FrQeEK+5oibJ<$1a* z$kI;rTz*cJlNDakdCTidGn3I-Z|0O18h{GOcXB07 zk5lJO-D`Oae2L&ZaD=0nMQ;V*U_@E%$3^po1U3j*LC2pW0e2*vV(Ritz2ja<5a@}s z+SNSk5^~VLMi#>#@}IRXXU7hd*K%>xBRKa(&?ieAO(X;jK|ruHM^ff^vcArs#40=;xbe4?KQz z5FAT#1+6O&PN%s~|LJbM7^mLT{?9QZN1ZlR{QGq3<>VG{rV<7CFH~qE`g}zY_m$!H zEwICnSJs#PL$%P#e8tUAXRz6w7TnNyTppvd@~J!}?5FEGF|4cYvx^%|p`gd}p1kHPvIFm?3^ zmP@7jG=o;~BsU=wta9I6P_~vcJI*JWhFDfN8l@_9MMvq(E>Zi~vs%6kknTewL#;f-Ap=>Ci-&SmYl`C2IzjhL>%!8Eg? zX--xAb7=&uT*3gs%2U}c{h*a!+^Cvc0Ksvk7SOG3J6jkgOi#h&Qv(B zCxSz#*bU`ZdH|xBdLCvMm(fWvf3-h50lPkqCEeAdCyh)Qf<_UFxdFejTFW{t*LvaV zy#EG>1?A@yaQ~06uMVhc+rE|~C5RFtf+!%}T>=8qjUX){-5t`UbVzrXbaxBV-3=<; z-SMr%Exqr(@AnV)+#~F>&ssCa9COUI+4o^1T;`kRChek^oQ7TXYMqQORm~h#Rx;(f z6B$bUT9CluK8u}vtXCsaC;@icKc6d9Bsy+4*@hXKFGsl;n0)cvaj`R*&GySz5hM8+ zue_tk1_lqIgQK>_)sA}sKn7%aed1!)8MqpCeQXxh&vup=PMSWm18fkB^Q@B?L&>uA zdN!Z+c%=(v7t}q3JBKsvoSE?lUxbb}qZ%!$s~Z%SilnU8fU0G>=H``cJEuiA^=zD= zN}<85S!0W2vLYCwoH-rluHR=Z8@ZxY*|&aBaXVOZE1k+9W&8A`kMPPIHf3x4)G zMKR`FV#r0p=Q^YhNnv7I6Wuy>H@j?EtSq!sM! zZ4}aubFGQkj8aZBDy3~-lHU!Q2ihG^uanz~#IxwFkb4>#rI4>|kniRW>dF+mM#nFJYiJ6|GB?&RTq;B0t{UNjBeciG)r8Dc zAaLSKID)C<4}6^%K@ReglWiBphZW<~HcGfjD+xbtJ;>l3}%u^$WMxsMnP7qD8^L6?eTjEL`c z2l4<-1N-*HuRj)?VB~%7K>mE|b23B(lR~lJaurM6$3@omyqi{~Ld0ZDJf}bUy%Me3 ze3|oZ`FP%RwNcl+UDDFGm~F%98|JvCMEAQU)BL3+)*E9oI<1K0YK4hQ1`rpk4>Y`l zkssdTqh>bw;+NR04(OQHN|+%zQ(Aog6uHv#uB@Y^Pxj5_h%8sFOQpm6o9De>LvB)? z8f^tslX+ouOsk0(4c=nEdbe4k`vIQHeo!D$wQ*YODa&aDro4NuBK@?i&dr{;iE~v% z22JBkMJ{~M=Jv2q1&gV%_K%6FbYEk;o1p1~#jw<7LiB}3^pynCr5c-yQaPE>aMR)C ze&?_xhk}4l$P$Lv6qYBn)71y+2xF|ZcrJ=e8^UegcwK{q4WSruEWRNEg06+SdSa3e zHTriZ8&ddm9CPJM`mZJityF@^+I9|xwx*O=iZ4#WY+n}Jm(E3-;?U_T?%+rt1DT%{ZgcUono5avb4l_qnu%7fw#VR1_B?Uc%1&i z0#4DM$2~sYP2y1HhFMm+$?J`Na?GPVuThOEo)}b77GwR?mS$qlVjN-OBT`tIRlUql4v=v#-cH2j8%|<<&8A^dCQs zYkF>?HLBh%b$Ol(w#5N2obASG_@&hMg6CX|r&8SZ52^+3xHrNg;+!FSJcUIidk0fW zMTkT9Ix^&IJs$eC@g1n^{OU1oZU=CR&`ta8{Ws{s1@9(pwQKzaa)X`_IDAjb{a`V; zfK=*!=&?c__3niu0t!QJA7$?D0N)Q$kxq^WwksVgN!@s=wSKdlp4eivWHh5+Lc*2716)n%?r1K983FfEl?pBJvtVd3Rkcd3JFRd}wmr!v~mP0jKRU8f1H=(0R=CT|k^ZmPbzSkkG zlw{<{W9rN9S+J$o%0wK~^$MpqEAVWE^p#{1x$mNABHfw3C(7XYkE%J#g*r1t8?Zd7 zO%66b(OT))UmDA@nRkuw1`oF6#TmQ%Q9f-Yo5j9FpDx!#DoTt=xGkr5(5#_2hoP%` z-m%ca=!ow|wX{@)b#NtCE>61JYG_!%bC4m5CZfe^xg#Wk(tmRgZ+*rf*i8t(v$Ta8 z?1;1?L|#b{35QNb@nD7XUpoZ;*%}g${i}oB`dd6LDlMfr`&xQYrt)5Kd-^H1-r^C8 z^`NC1U)ZzyRfeN3N(sYFe}>1@=vO4B7xK&HIEykBVp!QP9ROI`U^+9W-r*)r$3jUwSu7Ef%q0>Bh}k7eJ>5;~a<9wM zCz%Dy$wa}G$fI_=3X`w-BX0eE!Q9VH?HXNTv$NJgI)P6OM@GdD`F_o{$@)dXT~{FY z<$Di(FUGUSW0SQ?)Y*%a_eNf*mARILKl#{xAP zdgs^q+XPxfQdA+f`p3qT7_D%H8PgRJIfOzu^P=i0Qh3=fbpu9}i<&y{Q?ZmcDrd6v z7sTEC?QFKc-JD5*SSxvX7^agTPZU9s!3x7#FMQavQP{oR;JkElyrm=7;F)7lY`k|ZwlQB5J)7$TW-#RaaC&N4?W)W-(yZ*Jc%<6 zb?)b;JvaH8XlaUOUcw`Dwy#;&rbtyXlVFE~9Y&RdoWZs=WklU?W!;O@In^**qDCni z?^qDNF{f&0Qu&knn#{FoycXaO!fCgLF-4n>sT#RMWS3$5vj7(8cf6?KCF{BEA7J3$B}QjuHR zTk{r)mkJm>jU15ZN4I-%C5b^&OKmN?poDk9q)1yszn_TRd6c3&( zj8`n$ubf4|$tGh9zOKJ>*kn}L*+9Std7yq=F`YitG#!7!K21DEkOWl*Le+BM07Sg2h}Rvhl+79?J(>t4{R61;$>^ zvFl|^$IRlibZMi0Q;=Ep ziNz$1YI<9qlFa0lL!nCG!&eV4&2%=20 zz~xw1rw@Vi=&Ma&g!WW4TAe9_qn(@u7?;F)^kRb==?oFs1%3Cz)-?xi+A zw$)OaKx3l$iwj6-$$O{%R5JFZ90k`^nZ6)KDT}<$xg%GGu0*cky>N1}YEjH=PU|l^ zt~*H1&kb1^P@YdZeg7nQCmR0arUiF_UG>gg6r@?pwTzO3IY$9c@jOUfUAuU^v-~;{ zjE7*nwOPqy*dIK}DIy~lX8v#OEsAh6|3<`YDsH3y3KRIh0f3LxpUbs`gq?E(a1GA$ zC&cDcY)i9pfYW zDa^%+9~e?Ty7%^;r|mB^hrZ1WunSppDVCMO>^04ZyY8bid8GYb^k`kiW>ax605g&1 zn-n+Kd~sGC9XPlpf)Rq<+m~UKKT(*ZB&Xdy(@5COz3Vm-&BgI${`*VX-hm9>$8u^P zctNwI2xF*WRLxPM3)C$0KMA&KHquWGvfx<5FN)P_B%WsP%{j^}Wkm6NzVg^OwWCZB zi%~J9XNX!={)*l5gqmFBu-JGr?ov2MaWI|9^-Xp+4I?CNKMMZU&`J)x&Dusw2qq;_ zbaGr%+7QHj0t*y8$2)|&K5l8SPM(5H8tDp!YC+0`{=BzX8ebkR8tdO+4UtXV4HN3^ ze|Yz!L>`%J8Q!G4E`M6lVp+*zWA*EX3#yGC3Mj5Ur=Hfe(bYi!h~|N0lBs+|VczA{ ziN6|bTp=9xoG(H*f*PcCGJ>OB$swe_a^@sqK96|m`FuR9$C zj(j*#B}65NjLg95o$hV#D0s9`9s9N-?=6^0WI$~;z4RDc+i(>f&B<&DXWO94F4qq! z9*t*}l2j}H+Hof%5ZRH^V!K-AYO6A1`SMK0==l+N^z7y9deiuw7(M>b*L9C(W?z*nF%BjNuQTfU=;93qXc9kVc_bxBkAg~RaeXG( z6kQ;|n7eu=sn+n$S`~*W{eYq)GnhQr)heCE1ZQLb=TQa>9WQf`24U}5sxBICXbP%N z4UbRZqc=$pNqO$L!=g^q*_Y}Ub%>vEhax@^apQ<5e&NPb2Kxl-zhJcADRF zjZ6iQbVN)BzgA?!hp&E1uL!)(88I6hGef%Bpj(g_i=+;HWue(*ixd84yPy_Qft=O6 zhv!(FSe=)cBOSOC9ec2hsXP#+>pkoiCtdRGLw}Srte+@; z_7}+>do&*WVmKznR@Mz+P5&Z?e^;JRV7!7{^!nHLeK6f0JbNje@X{v0WaP`6Vg~AE zG(CtzN`}jD*=*U-*RMzqU+Iq%6&Aex*_MhWwu~9Q8D6*gV$eT;uUgo8UW_&-AiPny zk+;q`OY#hv@I$!O>gO%}$%|u=?GVBWlQJ3FrhHCq;X!q_{4fhMkqC-v*YoRsdI;j9 zOt)NZk?hMEtBNen8i#TdXEhyAz&Ct{&!*Q43f~$cE@YG8IOpJVrCr_l&R?=UnY!QuiV0pGIAazCUv!yoSEqyi_j+U3DgzblukAxn8zNL_ zBlFI#fsEjMi$gaCK)D=H{^q_VpQo7qNtonfs8nb*Zl*hKhFWR-$lVWpCGn|w_$HSl zq+$@>VEi57dEgIBhvRU70`dg}N+PPymZR^fk)z*{^y9gIZ`DZe$ZoEXys|qT&??Npy!eMxoq$q2mxZ% zyHq71Nrx+&L2-PXWX#t)P7-3whD1w_hjeZ1_GgyaEC!+a@G$_J?U(<&SbkZNc@xk)1_~tb<-I{o^qrKE%#*?S`h}BZ5O$W&E*GjXt-g#dVE$UpPvq`w9Y2= z_>i8^!<*j-XRE4Epx_XdN77){4^$bi^tG9MBTYnA*!!{lxcMe2F2-Q8>TMWF8(hz) zz242G?K;_!HcN(Wq^nGe6A9@X(g;e@^;E1+iFL*?Z{s+XMuZse>K93NdZA>T#_0LE zAlJi3uCjjIcqtpl8dYXg7{LHs;LLG5#P13`Dy7*|;I1B1w^#%kVfv68jLh|%V<2)p z^MzP(jm?z@pEdQ;P9yi1>LElJSR|ojkOO_;gH{iwF9L9Iu}n!a(uXoKjN5}`tUx4Z zu}IxJPcCFUSN>sdB3z*OCNx)$(dFUlh7>?f>0P_uWsLSTv(J)lo)k*pY(|RfzSyb# ze_*c(BcH}`{hVBpZ{_0!$7t%ZK0o}gyVhN|KB-_4vy5rXoR<$f++0%{TZ z05XP@!xnS)K`*Nysuw)LD(hR7@`n}gXmv-B1DVsj=HRFMYgPL~25g7ccGCybS zmKHH7(~nn@o?^K@)sl>PA)?bgB^96yXdY`AXNh5rB4z)_zVJH+Jc>ta!^G9EKS6rG zMTEW}rfpC>JmOW?J1lQKm`a9mDSu3bdc~Do$Q>Jq526Rs){)%lLl{^pIHu`fUED#9 zLQzmRb4net=QfTXRh;+4q~1Iq8V_W6#^DDazkV=l$wT2M<7R~=?Uoz@febzS-5ZkW zVA&A(F-CzgDzp;l{bP=pK8 z#}|6vOE`0&zu>Rtlrt&DBuzYHvFd#XGKeKyAS1x^LF(>TCP&%B{tj$Z9(*PkWkNTe z_sFPBG7~@I!8f{PL~eiZ)BVak{_!vWqy$aHCMt97#f}hqPOVHYo>R#A)?~RterT-K ziYUMsMAgdWy9ksT{biBV8B?vkG9XWr+l{NH&R;qYZ4`>_G4_!ybo^m+WDGEp0Z=H0 zdH{h9Y`s)%h5d{41n5q~>%W`eU)YUJB-w6f&ZV1R*az<(+yiuTOj;W9azkGtQmnHa zQ8lOHx2uz~UGuJuZ$GROkXy{z3%nvFf!2A*MPN~Guib}%uit8aLQeVm)&-K2A&39r z4FArC=*mr0!#*CUY`5h40zOQn$6t2@>H}1!ub$9rnb+Mx=Kr@}{rgf-SCfY8fd~l( z;bO1Tl?f%I!m(lgW)7$|cX$a?;s)=!r(!N_{{I)`Z(8yV*6M6+BI0 z6MrA9Hvb$`I-0@l>i>;~p%=JiJ^sGZpC1U#a8Fzzx-o**n6N=#AWwh0M>n?^9?nQK zCiLJF94o=!zciUBOkdzX>+{#TL#Y+~&-f5|cfxxmPLaO<{02`da*+!x`fK1WAFJY& zh<^Q~|4q}}-f%QBli^55p(e0R7oA7{e1}%gJ65-QK?mJ#@lOn}s6JorWIOV$^z`(E zCvez#%?+jt$;(Gxp8gEpU&~7B%9Mx`SC;+=?ut#^ZRwvY{rT!Q8W@!Th9Qt2G4%Z( z4&>u?N9)h}f`=b)%f5(&p1>|*n!dbAOP5S|Ih4YO@H27&&6OBs^>nkeUwd^RWvJRp z&tNe1Q#_!}Wg1SMR;WgZvKl1xaqKUfyr z6-HGx@Zjk|q(l3~d;ho%^M*T=zt~^$7uX&T!BgYfFWV@6{&ygfSQ^g%7?S`Y;K>H7 ztmMJEH?isz?pVCWSERE1*%r7D?$8h38cal*FueZ?OUSmklH?j05D-wOW;B*Z?;*mp zz;{=CpTc8;-F7n?jqC?kqU~^Th4Ex+tw6DbfIQPJeL$pkDv67^E4C*qZ5fly|y}>G~99E}F}+qRe1eP#K(^7e*@8hlT%R{^w38 zCZ&8w?M}mm9WTtuV=6`ICbZR{e=H38>N6W8@6oR@?{?Mxqw4|#8&APA;3K;;X+9=^ z2h7H=$qNN&qp}i18T;7yz-dO5^DY-AiYt>fHb#n9JulIiG&5gg|6;TEUNkZM71RFy zJb-ob=!o3s>K!l2@z^cX)v^BrRJMlmd=4YfClV%$g^~~{h{khuN+jaZ_`Lv)$mc6} zJ~gXD6SW(z5DLPjr+%1azBN_;-8${z-yh9$ABm3m*X6v80)Lo=`zjKX@x;ImY#^e89m~m`sa?kw`#Ix5QwW zgxlL3*$)eMlsLAQ}#>yp6#=BmhF=%(7D%CrbgNI4eG8j+LWve$f zj1;J<)Y$FNlaWjtj{b4JUqbBP+!Oxi0Z6P4Q3{MA zC)8NUAbA*n8ILeN*ctTIKHY4PHnH8MxUb$SSDI4W%{y1-DU}K$?W=pjqiCNWuI*o6 z?1+I!)MqG{8-|94zF?hqDy4c?B`u$)NN^Ya{^za;N~}euOy_$zX!M^_Q2)CA-^l`( zxJmPmSn~HyU8XoYjvj^wK-cs+PVq3(PX!Fhf9({jI)^hW7TD>s<&RcmEjIQ-}H zf-p3A|M|Z^U;Xz_hptT!sI^+(*pd44;@iUFFdHti#(+M@S;RNZW{}0w4Tdur^!nl< z5<7mB1)tYIE2u=LdvQ#~46~O>CK-ls<}5AvRcJuKeXY0ks*Nu7PP@$r44SQo)i>7{ zftwW6Diz@y1+|iV4CwcdCy^B#bMh5u*0y#-^-Y&q`MQG%-SXC^R$JN%R;RNE2>*(I_nEX z6sO1^ov6j#qYWIWN$n=qf$LN2DrC-eb*O>7dTGSHJ-~iP80x>-+%e$@z#DLD+ z!Ul?Y=aU4^;|VRx_3SJ!PC+7$#pUI$huoK_BcFj6fAJy-1R3!Kr9uHyb_ys6Q)!z3 z0{>Td`VY^#{S9bFN~8IlKz-qLKcpAy9saaRW#l2f77;5W<9gN->>q32>IcGPv85E< zLl7ph-N7ln@LW+UX7zHfu1~dMUfSfmb3PgcF#z3qUYO}!xnV@zUfWX!FcP*lnj2-k zI>3v=YD$$AZ|MiTh6Ozfk7}XA`9C-?#beiu~~%e2`(Ta9LaK;5558sJmdZ~TO?sKP|*eg>Lu*T?cV#vAC6+Jf=SV0}4{Mr5Iw zR;D-e_^G zIUKZJ0)h1gf>X6n>({f?y|gfGcJbH>m%x1{O-KjyZaeY7dC@yt)7^a++hH?=LNAzJ zw%p>vj{qtNf6*+H;)fZF*MMXtIv5U6K4&ve4!~w4Sv?LhJY5Wv>O!WiL(TsF6^E3X zx>wP$RJZr*I$mx92ni1ft$KhjQuM#N$jei$Ws%S7s{7SS@Jk>8UVnX^zaeQKR*~xW zju66wbsP1@GTK77XMY;DkGeb2ftb$2`hy^0Q`5@=N-w>{gv3NM+xD@7h*|JC+IF4o zUtz$PmDYzWH{L_uIux*Y(Fc(DCxQVGMZdC1yzLtV!Qyk?pQbx1_!@)wfbT#fdk2cod6q90-`X8JQ818zFA*@4Bbf#WDq#v zj8@CyF`V`#P&jFMy8C|GUQbW2+x6x`-M;#z+K@Q4loWo)t4ah9-jOdsU%fy?(?-7q)vSrgSF4 z?>iu|2CbFDA^qG~y}LhD`T#SDkE-bQ7!F9T(vv-)HKQy{X>y2Ky+cezVEI=tVdrm} z(qpteQ$6(M4Iwt?2lEk6p@!@84e89smeUSL>mnL;&K0mHwyF=3K91 z1uEZtzbOpP1vBNH&+KLh9Mqwa;JL#p0byK^tP~e9pnDl0UVUso??W@%CDVfLYFeqh1?Gj*wJdlU1Hspw&S+5V*V5%$| zO}fNv{#EVr2!R_94D2ct_Wr)Gzo*b*@?>}92F77C7w*&J>4tB(Lc%<>iF3B2a{EFn zefy3V$0Y|17erCC2?3vJTJTXn@`N5`l(1uC^WNvd;D9NudFO95!(`r<$E9=oIfIx* z*Ah==4;WrT5Q&m`>R`jO8j_!@G+?|@K}noybZs=lY`=zt|GxBK zg?_8`p$5+;(DdWS8sAPGVQAb&xLn)F%aY;YgWi>isq&JGL7`8ZG-rxzJ}5AHeJ@Dc zEmgH~iqpxZ1x#%#3~qkjTooFy1LgxUF(Wh?rD{|^2 zev>Lom+prYErOO2SLl*z!)Pj9)+RXCO8*gzn zN?jBx?QAv}CUX7?JIQJsz3IG+{quNXB-`?`9q%r@#-z?M!LV@nbHc~tQ`$Szr0m`g z1lH2;-`<@DF`xn>?k|DuYf^{w#xjQmb6-z%C`xC`WHgHM)ea)x96?-rw+AcD)Il$2 zhwo`qu1&S&(qaPWzTC$hWIx`XRUwgxB{DEDc%kN8Z`U13z2U5=f1&QQ_XGWBTV z6W@BmGF|wb4xw#ol1ofgT}xtGK$Ki+WFR+^_*Pog(i32-Zwj>C{)l~^maMT^BRQn zp$Zd~!DMPKn$=9D&{Xz$kG|mRTig8WhZ{i?K$QQ2faVQ6>GJv^2K?SDPfz52i`8B{ zuaWheroLBO5j*!hnViiEHa448u3$1I19P3E9Hxz`wsUq#o7i+b$HBPdw{7+Ms|s9~ z8|R<4H>-msX<9;)AR^+%)AQw(5HI1fzmUJ#tI;EtE!v;Sqw0n!sW2|iOS@Ie;=hD) znC%g7(0pR=+{Fsy0h_Sc$FUO*s=+P*U(#*2rrt|wgA)n%3tg|eJAzWMj=dSB%MC~Q z+pD*0HuydHnH>2VU9J{e9&$64a2&3WiWO8Z!zl}@UVA;nqrq836UUQ!iqf&S={WzK%VC6VB##lUGawb)s4eK3QGm{g z*2Y+VsuEwK^YLUdgnqM48HBtK9%`FYpOo!Z2ME@{$%Y|ZMDgv#@TENi+++2U@d*jk zvBit-2+uzkHeOHVlAO*0c2bn%Bo027fGAq6`PD<(B9Dek5nZt6Uo89jvW~IF0%ioX z5=5a}agU|ry#c_d79`TC{P#cyGEV1ge@T!e*s7n4brW;-+a+MGuu$$#fKdDKTz#o4 zBGCBRJkUm<@uVsd_i-MGi|jjCty~`TSK_iTGmn`h zO|Qi-A*q=Dl4(ORLlm=AlhB;c6D`qx8GG143NQdD0@F^QvQC|pgk}b%D)SV5x7J4| zcoo3aFzSOjw|Y%M0ycuZyab=4J4z6huU0>W_+jYGduB4#Q+G}uSYO@In95^n4@$L4 zQ`H795ZLG$B75Rk`w=fcfbx!rl~uvxN~*M2-#ki}@!p6K-c!Q^U_wv&SHMy7Q@YE8 zdxzH}DlC8P4*q?Zn*W(oP`M#2So`~d7lYMLtx8KJ5MEu+-^51B`nV0lmQU9_IG@jrt&n{1C;2#vQFK!P4>Fl-rnp*I;8RwdW@dZ1 z?PtB)dwv%vkQ39U8|JFT6z}Ju;F{AF^{c7={GJX>jKR8_gKak_t~aF6?8R_z?LfJ^ zhXuujfimms z5kG!)5MbeGzW$c|`e!iuDn!55KkI1vTuX21L9FjLEVcDd!ysv}>!(n`!JLRm3wl9C znm|27Q+sg~W8vSF;-u2`H@P`oy zcuoTyJSj1v4L8@0gH$gZ_FU=i8VN=@ZdVI=SRBEkMF$#Tj)X-2jR<&lNtWWk!GYvpIL1-gC$qyPv@$qWg zH-gWXbL(gupFp{$)z-27%30fSW2_|OJ>OZqD#xV7 z#a7AiRdtElhkDmVVfNkiT+7w1%B9_lou`!-vq}9ss}pZ3SWGAo^m8O_$R2&2H)C!Sc+Q} znp6N^h}kQef_3Qq(hvRtl*9xged&wXaw*Xgmzeo&m$c42DWYzd)9>D&y@msD@thAs zW-QvD2}V9B?E{ru1dGX32Vgq&wDq7_t|xYh5}fc96C(0LJ*ATu1YHTyEq^Em(Mz?H~jHB_D&WrV3guXk|()9)%ODD%PwF?yChg= z9}S_i*391@Y6>x#uJ|<4pJ!38qA!pNxZG(BS@Eap1EpIgV*?gfDRiwqokKW z4C*ab24^Otv7U-*T_mLvZC{f(X&`Du?gBJ{DTi)y*C%8w92h+JDT5-PzD5A1p=_h5 zp8!*Xu3a(m{lNfS_r4{Sn~R-Be|VSENR?+GbZ4YmjcP=ngkr!3^Z)ZF+hbCs4~ zv^f{wGu?gat})vo$ttpAKTlUqr(kLO!6d4Ei5=8kmie zbko#lOC*ebNQpE$j<|I031*V@f=35`c>oV1$~@c-WvBrMAj zvZPpPO6H)0CT7Ee$M>J$dvl-SuVcN_bGad* z`RupNlz7hyM1C=RR!XrUV8jqXCUYX36>|B?f^dU*HHufq(Qo7Ecn$mFNNvXQLe>E& zDEGz4=<-36SWRpk)+zG3L%-%!2AJw{% zNFm_F*nRqgU8K+b?mw;wrG-3}nGo4z+yvmLlP4LOWLt#roQXjCzn3!)c$>u_j%~!8 ze7UsEP2=30*W}!5>0w0FJWSUh8!nxL5!ziRCjVBzsX+;(Vq7s#i^Eq%hRHBE@-_`i zIPXx!QN7K$`-|sTAJ*RZ9O1^z8a8>t2aFV|&*x2*83T-)xs>=Cru4(fFQ&4g~k zV9pn@nBVf$x(_XIjR&H@gWUhwsirp&D$rs3$HQ)4L6aXL$>*i(OUC~r6(JXQGie^@ zUo{7a#NC0AlSiVQwmFO8P-ybwJ0I$^)7e|{>&agBFlHD>(a4N)9d~v|iUiutM}YYq z8XeOHB~dj?-}I3$1fh>k_s7y1bfAhi%l&pdu|TEL^!1LARGqo7UUth?VAAuI_3C$# zzM`8)Q0Pq3k}?jL1l_W-?yHEf*ZDXJ2MmUPG^TwT+~8Rb!GUqQJ<$Zsuw+m~4rpVq zBGSy2(skn~igSI%kv(VzxKogT{T^G>StMhaRElT*fe=Xb*mP|ajg0uK+0E8gZSkCp@~eY-q6==D2@n1t*5z&Zn} z5h!Yz6NtRPp&J5cYIHl)BB5v(=Z-I6m!5+sQgrsjF!tu`cQG(9EKcerMSy;f@?kLx zXr4eDE8|F0AE) z=S&{F$MpWrVIBFf`PRRAn;G+eF2sIAwA?G%D`cb1j1KjPA8daOBwOPC5!6I zwk}Yu7WUNQXImTW@EsEpX+%|b(R`qPou&5#3P;|OLO@xsmT3R*|2yI#qeiC01ibd;-O#IUK#d6((*ks5uVez?`A zJoSTbIoz37*X>U;Y#7X^8^qRTl@tp;+ov*}ZXMYa4}8CENWo-E+2a;il8_GKDpwJ&;$Sxe6!4-E)=w zU?OTkRP12W@+mJ&Z891V2cgEMyrB6E7;F+}35wZ*gb<1JV=Q&yi1#f-`Et2Fu)ZHl z20(dE6}N%w*{QF|eg`uWBEk|QxVeqd4FljNJ**-zwwB@o9p7n%F4>b^MNG$(xVHQuX*6ngzh^1 zph4)T8X2)pVis-x&=#P2;bfM%t=?niCt9Fsqbnz<#C`JA&(??z=j>8?BP+>eDr@A@ z{ObIu#Qs3{K(ufO^r(6^I-E|_Z_cAQFLzs(BY4Agb#ywT)LjfFhbR>JE+pdF220e_ zDTZdud<-)~g#+8!0IS5K0iz5pOaCI6ZeRV)u<}sd%2Ldf|CFV;zdO;3U;Ekq6owFo z4IL#!(`G|7=6EaPhgPr%P#UH$*V)VH_2S$PI#{_%DueO!Y%m}rlVYZxy>MMu@oP$P zaIesl%!p;8hlq2wx0y%=S!f**k~5Fy`0ay^kB-yc{j3;V&Wo8Bs78?USgiGU(Rk;I zw>WIlhBWWUBH-`SmH1YPM&7rH_9^c#KhvCD?1~_?eGb2mCS20M$xcexl2t7GPzgG2 z0xHO{!meUQAzi30kWHS)^Q9UnqM@pgC7epo2VPrSi#)Q$bZD-XtJnS9ac@`mv)TOF z!HO6lV+H1MN0Z@uM;4~rF4&|sL?bBV>f(fq)q6MLL0>ys$7VD~j(Vf)2dI>X;vgna zIf2d%=<|!R#fZPMZah2rhNZ3;w4su+<=kBt%#5frS)wy1;BujtU=w*CpkpRJe|4SD za8Fr`i%_Tx=jy80gk#?b;I(8%pizLPV?|E_hXQbz$0=yoBm{7ZOuz3G^!xlXKI5hZ z3#5yD>=OGHqJyM0+ZQKmnZ^22-*>r%$D9T_sh37&qg`&tS2bIJ3x5&<7-E zHuSmhR5CHFC5|B1dYA0{dnQS+48q-ebvS)uvNP`HQ3U&rWxgsYECzcl+WIw+%R(o` zLpsT#ts)F(6b%8_O)8N~7Y(Hq1zz3`NGB$x^2xou z8s-i6FSk(uQ4nATjc|Q1@fn2GVSNL04XD;`;4-vO5sf~6JNS8>?qopx$(iP44%f&pz(~=+_CP2V4@DrhM4*g;p>~bFMee-E%mr z8cJ&gU-S>!{Tu{;1pdGSlkd(XH#aty3TpNOp)8gcq*^=~Kpw-Upbk`6Rup^d%bHJ3uQdUPz$h&UvDK_4(J z95?n!Ioe?0X2(9?yz#X0Ig<>Vy5CV0o)2a*5B*K;3nA|KwvuSW&j z-0uVfo)E-}K$#~7heLgX*_!hEwvD`2UrDAoeZG4drov#`qO_ni3@qHO-<>nuZ1H}m zB6k)cmGE-3P#hU>i6#I;LwY$z#7Q$r_sF4*O(<3QD#H~I$TWtmG#tSB-^(!upK5nO z(*px#-(u(S>J0#Pb7eQsK6W;X*`pX6nV;S>X3yb~c6YaOo4EngdqCaCQPJJ@!{Bh* ztkK@KrDoI(t@a{EB90|;PTuQf2xtmJw4dns5l^*5jpVeQ-r;HznO!OJ#+KH>Mp`mo^T3Z>LL;dWl zZ&)%q6LerCUGxPS5qej;rAb4X%*qpS!7%AQ1z{7dYbFsE^$vZzq}I!?(HgH2^)&<{ zQus{9m9Lu-1jHeuM31k#d&}bmEYJ3DcZUa$Lq2VB3id$pDCG~%*9q*;*UB_fkc$wb z>)kWW@+PgxHRMibkG}4M?qJLVK(G|z#0WtFf-;`(S16K7=Jho8rsz1Ml*`d(KW_F! zpx$tzlrMO_Q!A^*|A?v7h^<&Kewr$K;a^$gR|V zl>G7ge6imJN|6nYaZ%U?8zM=Qe><_mPufiuT2cNxt7+PO`u=V~`Nw8y26W^XI_trt z;T9s%3JwVfp=KCgKKBM`#NSvgh}%LYbP3&xfq`n{3lF|LEOadUiMx(Gh95AZ+_P)B zf|!`=sU;F52#7*7<4NNu{NKy&->RlHUT9nuz6oo4L;!Rg@f~Q8I99Vrpe*eMxK-dp z%jOM^z6NV0w8_sr{!S~8uk__aIZC-?DHl{=tcl5={B!No#V92G!-%c?vawJX!Z9?SVZ?%S`c8^_2$AZ+^*wa}Cb5Gv+NQCAz&h-N!h%Z)$B#y3YY@@fp>c zTC(gEz*5zoRG_qO05C3$r=P$iHx9lPnu?gFyMb__yu)(X?Llvy=4djwoV;Aghds^4 zm(KBXP*rl%6-gd^2$Zwh?`55T5+lrXx-?}}NL=cZg!8wm_WvUeY2t-|VOLcE;wZjg z-sY4VZagl7^|$0(*T&<=nj6W20pe*hps+^}&5>>{3G(wmCYpBYqh@D^yDiMepmx$1M|M zM;8W#@FIX5LmJ5c$z#hkfUG8RE1!aijlcBu{bw)VPF0vt2F{wH8!Y(9IZ}@Y+z0&K z@N7{9gc48;zBra4I%9Rf|Jmmn$76&fzX-4En*p_);i`hm!bV zIYuup=b&rL<0Vt6kXj3!9aESH;AyEHgK?Neq3w62LK_e#Z=1%476zlzo-`u@cMbBU z^as4>5af?;&c_?`KovB42`XkRXYq}LA3$zvZ*z_dZT|B5Cb4=?{vgg!d6GZFu|50UQGNjxzBJO=uC5{vcE1HL)!0VlZ1=m! z8jkvL6@ca+yTx7^u>?OwXjgbKZaYnMdkF?609)ZalDe5X91MXzs~i1L)*k5pIzpsg zg=o#YP%2GViV=BChKfd%lwo-Y=9zrq#TSH)_9_Lse2cNGPD8Lk_uY9WAJut}_-`Ji zUW4}ZH$;5)p89JKNE)SzTz~+|=9EovjCYsg z%tUQ%E95YS_W=>_du#;I7>G`NWr23zPxpQblvaF*irJcm3)KG{Xea$q=|-b#vn0W3 z#l4x1#E5%_DV?JmlrVg?OOJqh(Q73E?(-xJRS= zd^cvV27yecyX}!y8EKf+Zn8FwDh2dZi&-FB+F_mff0VrkJk|aC2b^Ohl9?R}2^kq7 z>xhg74KR zjO%lKuIoLRbh*o9JZ*cG9hYX<%UxF@Bw;zsf=F-)K${N7PCvLCot0(MY!uU-elW+0 z@MkUH7LG6>pjci48uHNg0i!j1CMNske)EGzm5j{Ns9L?CHOOc6;k3&J6#fw$`*ZaS zu$!tp=^C;LVdesVpoO;oB zt68MJV>HYqz~DRn~v9~1rNrgx&(wvIqg1Jo@zpi&g+wK>N4w$N13<+C4^45NdiN3pVp z0C~V%BA}G+h0n#>&Bslxo2F{#^O!kEFp+sBFJYDf!JL;nVM=VISy2IO zH4px0z3(poH{{dEQNxm!rjy8_TrD1N`>C`_ky+K4N&paOC+r9s&`C`=0=gGM;>BtL zZ1^AbnH(VCwI;oC&})6hW8({;I}5Yqi2FZgMt1(EbP^j}_8AlDf4l2JSK* zb%m7WyJ;cUwNG^!4T^hC|O5K!1;Xe zWzksyr1kM|4Z*79879R8=$XzK*K+bXF@->~&S^25H z^5VoF%%%MV;BUAVa_RjC7f^1c^xfRgY9VMr-C0dNXMjxLx=k3q;1O-((+3}YVUVz9 zkNeVgKBc(D9|^xiGMOie9&+GVh47jsYIe-}nm z6>TaHbs9+@&b$gW0LL$Ag5uDtG4?`vw!o&;ZcMgpp+nTSrQ^D2mtnYwh{zA5e_=MYHUizz z!L!zV->+~9{gqe2DC1Bj;r{y!xxi(#@n~bb<>=DlgUwCX&l~$^0rhaM7f7HXU{2DN z$@*I5j!b>B;g_}-q|3;03Jb409uPd zA!sUb$n|Eke_yvha*V+1aCuV5ceyl3eqna00sZigk1Q2%{?fh83Zew9A^`1QnnZZG zyxQzt7y7;tcMS0|3tOUlp^bWFzCn?z-pKYFY5rJnbX`KwR)J>63#Od`L*9&}UqGW* z#ug^vKa~-`LCn}53Vg7w3K5)g@>4zXW@cvj()}gntD}JzithHEC3sp0$?Xqdtp&D@ zGcKxq+&Xx3xXQyu4(a*XRN@(0pdZ+z;A=wJtxW3CJNTB}jS>3B9`umCX#@63yt$*J z#LDvO%5+}!1GBBNu^`beJKLC@RMiyOWFIXAV*fb@O9X&_e6xyw>7EH2c;oJqeD;?; z3y^%g!Yn7- zwG^u>{X?&ctwDaFIZB9!H~$jhmFCd1#Tv&+hEcS)j%IP@6VWOta=yJWNiWRg#^9k* zxH8K>VFxl;h#(UE);G>E;U|nbqE0Qj29+kIz0OY;OD%OD4-@6GaI2Qsry)0%Q`T4p zA%wbd3-wPAs4KHz5q;2x7Ml{>3122j<~-i!eFpI}qrCF=MZHKk^Bt`GlbI=fN?3ErD{$}MkI%! zt!L_1g$`e*e!qm*vYsp@;V30Qn@usLM|-;9_E)t6I<_wA+INXA6VbH-VfN!K&8|tm%eB=7R6bm@>xFS38i3XfNvkTf_7iM*xE}u=vYV=xHfM7?w z?jAxw+q^l&%uL=C6m-n=^3JwBU?!SN%J5jD4l%f{=r=bveYbM`-Y&G&GDC0#aLt8V zAFallG7>)u5@vB1D}?2*)5P05JfiDiJVS4E5iS> zL8(%5?dUz7Bq?Hk^9}(9+s}ueu1o`y3gl35zU2kykj_aa<$UBatooG6VOS;sEr=29 zXn`A{NRhN*tOOc%Yadq`H1Z4vwNoU5p$MQkU@ZX%dXo4o47FAxWCdk71X1&g??-}= zR)!yy!6~HgI<=KeBkj5}$5lGynq?tN*8b7~8o0Fm7r;91^VgQ2SnRr#1OMjVLcedH z_6DczbrB1emG9RSi&J@Xv7tG6k3%5E-@v(QVXoXXc7y*b0ssS$JF(Th`x9-4vJxHb z(^97%txecRN2)(S$KhKf$vsnVZ*K`5qdVJFOT%x{S>v+{e{i3#FU&DjT!R2&vhs$w z6=X6m6k#2UlhB8bAXPiTi?ImZFTvdUENtT9&(tUA&zbahoQ-PDx-wB{TDTD+C&TYU zzcVk_tep?YKn5VY5G`09U7stsGp!{0)k@<7qgT|2HA+kG<{|UMC9a?*J4B^najEyd zxK{2ZBl(+-j(uZqNul$s;>}8>4dp!)IUmF&VI3wB5gh^b`vw}hG?a^|EvF%u#wG$j zXQ*q<-~zPnEO(qORK&Zrc#^6S1W_>TIu2LTO(IA^S3WreAExH5dD*uec3Fb7(%WP8 z`{#!GMFmym`V0qunR|zcx}rXBOJCx5{l~33^0bcH+^wB!84*a{2&fvBcbre=1w`+g zbvOfU!W7`5$Xi3K0X+XK>b^kvBZ&`fADtt=dL&^T z`L;5Z5=`5FwqkmAS&&JV|H0Illy5}Ez!4fg=(0Ih%LI@iRA7HRy@yUaQgF*A=m$j| z1T(se9qyQgHh;*@7RF0<4a&I>1v97LUoqqYY-9=jPGKu|QFc+LbKRC4tk9fq?JfL# zd@_?cSiBCOY`@*$?TO&hZHrOc<(iN%qm;1!u1P^#@;B_})ij7^rqKn=`f4K zntgX0hoyN$TK)Pz@9IJiI7l7vc@5fAso&d0nBi7mfQEhra%9RrmS2DIZPgy~@d8ahAf$S=z~tw_FDy1t>VEOH_F*s0O>>{_17 z;!97QRGT(-*_w7csw5p3w#?XiYJ3sc>Ed0lrMCxsUFIak5Gt^7!$VTRo+R2qh9H`2o(Kq{ScoT2WxwYF|? z?=cWkz+-eI;yR_*>g;f}q@g58t(E+cIAUhVi0wKIbRp)b2~wLBDPAaeo60nr@Zylk zc$@xL&hT4N^>IF`AnpbWEG8MAB<|O4T$43e{R>kmp_R6YX9%}sy(*J7r>+>%G(O#I z`dFs!2$86eVfhQ`K8KJ{4QM(E79e)Z)ev@9Ejo;pXA z<2eA+fEH*C#`Bv&;9h#3Wfl#IK&N}%u{>7!54n`5$TQ9SNX{m7S@tnemVvF9QIDIL zt?V4#N@CxkPH}sgnY#?x6Y)-WP)TPHU<(J=HEe;JO zqLl}IweTmjf-Gfftlddu%p=S%lW1G{*`PD7lM!upnzNGg)AYN>5VNP;7h1 zYhxXvrdDGO4aRNdA{)C$`vjzyQRONKEksxu?Mrr>@Xj?_`Xac6(ww!9vS30#+w!%^ z!@@(DDKG4Ij=xR6MAtoY8ZMGlgCC2n*j%qwnYEl!m;ZW8ka}bcj%?s+(MGEZNXzj? zqwnhHX^OUs*~3lq!7D4HHmFT=mVUAiw9oLea zJ*3^_5uK0Z7PV)KZ6o8k)@Qenve&iSdL7b6uR45yC={A->zvq6FG?93v(BnX zF(U$yCa7-0+Lc9&tsYZCqwuiEzHXn2QgO{NVH)#+Dnvl=*7UmrDDPBgaab$v7eHj0gm+kR3{v`x|Tx7&*k{pdN{ z2=q1)P==nv`-F6gExELr!V}!pF2*?-_!Qp95sSWpeZ?}fS9M#TD5lwP)GM|6iU9SE z;jX9K;a=eugH==ptJD5_Xm3s*mMnsn-5f8DEr~`JfJRFEE5oY>N>2qf(7K`A*0hJm&BBrl7QsOXHHji_3Hw=}MRvFyk9R}L1R z%j%T_DK4p{-jT!Q_k8Y3X0MwuidkryJ~`NOH4W>i?3)9UYH*v*JWO+<`F$p?1444yo zEMV0H)S5^h2YDcTJ|vt4I~*$nJ`(a(y2!wJ#hlUSE&;u$W1Brb5p6;C;it(@HXgN! zezJ}6gz^;vYyf{>2kEITC-ruwWF1?$&xv$r^CRearD(nj2>39hZT-gzP5&RDAvmXB zQzh-*QGZ|2(6zXrKhApR+q)1@bfYfC38wfYpel!d(gGO8>Or|W%m+UuN+B|oB?1;q zqUY{}ZoRp$*s!@~zd4z>wVF*JaBHkzBw z7F)je9F7bNxh5_lh#b2B z24q1LCP7qG&E_pj_hhPB8~(Cj>$kGc90p>i!sDR&3ltV8s2vYBnXZfDzE^O6XJ zu*csa5VLGbAMh3?H+qFnpj$}F9_z;Q+cR(e}v})s$rbn1zkuTB}k!1?N^Rrkc8C?9-TQ+PoHRnmL z;+rWTFT+1;;g|r;M16tjhwoiZ{h>_6M1b1s{`|cA?e5MqOuY7Ank+i*792bMza4%@ z+G9*{ew}_Z6&p8!{d)45ttxTdo$IX;=t~b>vy_-~-=;lUM@tqu&ODxp7v+jIwHdV3 zF~1Vx_Qij;zE*3YgN~oAv8gK5iM}q#j6?V<)RMl8x9(#4Kt7YDeT&AS8ILHn4*;qc zAJK;DJrA`|2_XIc*&g>y<{k2Zz>K28Y*E+Vm%Yz}ncL4VKyFxAewh){EU!;48P+zR z`kWuS2*Iv!CP(zpTHr*|I3{?}a(hzp7vKYwE!n{)3DFrG6QP-THB)#8hoI$q9J3q7 zpfh*aX1?b#6_Mq97QP0s=MN6Mww7onB}I&8#G5(y9rHd7-b% z3iwM}1d_R9+BxEAEJysmOx)`Umw;jQ9s`97_P6CxVay=HQ02n9j2RrYt@BM^eEzU{R2+9A`Sss5YkFuTizYivaq9Y<~ ze9yw{v3zItyYa|_f~lI{{Ww`!pKSx%yVxcYk>p;|3fH1h5u0@&W?TmSc!nK}*Ek4w zr{AgadwxZq7123s*H9U)SV(2+mv(O~5p_&Uzp7=`_0V+O@Myo=GIg!|y6}2d6GnD= zx$^s6LlIp3x<6qFW>S@kYK{03P7$qggGNDHV3Dj367j%V|}zb2CHw2nMyk z+{x`hitBV?v%q!7QG}|r9cjRqMUbiFRn4GN#`mm508x#3P37O2pfE3DV+JMWc_v^3 zAMDNv+F(p~4RI7}ixvTK(c8EHjW=;cwJ)olYqHHfbIf}m9!sF^?Xb@w{~afPVUmDC z!;(b9|2DVk+%uZV)Qs2?hwxL^uFmN`tEH(j9cqN5qhi1nW6mfxQBg+_~_%6IP zbV*Cf01g-%mGfadd0woDK=koL6ibn0pbIDOB#1vd&4a?< zWd9C0qs_9h1*EgJ#QqtPggGUK)qyBx{k@3)tcHVB^@H9lwi~_rY=~N=W!Om?Dm|-) z!}DdUA1o_@qMWXoJ6eCK)q~LC2>o}i7AL*T8$IrgwG8z7l^;u<&-yjaCDh{in0PLR z&_ihf94tOn60Gd7#M*x9EPiQt``)=p=o1~y+E5-?z3B!ymhvXX3mDSODlDk=&C?F8 zT!{AYmoJb6gRreUU|K@eA`Lizq@fxgW=2qOPD+I)0#%_!-LI0ZY^3VxJXA5l$BY|o z%35Y`iue^gvHa9G`f#HD;IOlFNwLN8uVW1g;0xlCf{HlVkHhjRG;Bdua`l65sDzo) zke!rcP>wED_(KESXmSsvc%@AO2tLX=+zD^=U4+ZSf19_wFSwYkaN&5S&HUQ`%%<+vA-ie?aKu!zuUFN z`XxbVDH?|FGnH#xjvtF;GT9q+vCt9|!K5I&!U3|k1C08^L;7k%;k7Tu&a7@TJX`(5jgf<$jJ z4#DAf%TKKx^V|}Q8s9ifw!j7_5YCoJ=OAV{t%+!7FN6H~L2GGjq@CUkom+($ZgER{ z7fBTvRN0(R$_L=|xh_NDFFGPn<^LpU$KXRYnaDHF=#0+19DA~ulIJY<3-0%+0o220 zjIVL)66Pyx9r*bxi5F(e9~^CUvUW_O5*^U@k9C|aHXB4mAW3PzsME(1#pjx>B?g?F z#K$2m$&(Fr8Q`0HHBWM zOwGyI#F$nYB#EE;#w||7CQA{%dasv{7GPEb0rnL6$auGzsPQ#l*TL3~l9@!c%2%fG zntpf|Kuk8*jL#E$rgUmZT#uW5FdYP@F_n*tf~I(pCTb3VRKZ~4Iu+A18{Z-57sZ}r zYej&qP~X!jUCT;uyCkI0zP7-cqshq*&LdOPTZWqH5RW5cfqMvBm<7xOPVgiY@19N+ zg$FVM-F({TggOvx8;zuA#U`xP5FnJ{RHh#jb<{4&p=&7R(LN&kudWQbo8Xq?Uw9@2 z(vE)>UE~1Ly%#crj&UO87q-gfm?uDec#!ClE|29a#fo!w_$s4uSWjZ>w0t;4Qdc~O`cVvpxzD_3_`X0bN2ql0L{X5|Z z0j$tzmZ0n0<~Q={Z4OY{FF1xS3e<=dP~ec5|A!@{Pji@hjjXp z8`UUy1cj_t?L>*AE^yQ|gO-jR4y5QT0E12yxbarTb$N!Jr{eovvDFdZx0#xp1%!C> zlpkN(rL0`5^0+Je!|v7}xgc8U5HHRD7JE05DRG~pV_`3zJ*m3LR9|^hjRS>WIK8L`Zd)pC%ea30?px^ip9-g z(%TVRzM#f301_~Zc4JW8-NG!$-r{*KbEvgjuesUf4kTVc3VGNp%H7lf)m$0=APSSi z>nBJadUFa2Mw$5yodkk@s{>LI)``3l5*AiET^{DJnr|DjRNxj@*(qtOZ>y%BWX;GO znX2y;|L*FGrd1F1!fVoD{C!kNeYvFAntkIwF|nj`-NZ^m)nr0g+{44I3cx}inwt1o z3usjU2xD^~_Yd1KgT!s_rM%VhFRP%>RBS@jd~_eHvY~ufEjGJOm8H9qr&T*_L891IODcu;XsSPK zFi{HRtS^GxF@-%-kQr9re(?N4FP!w|f;BAjt>_!O-T0lfNK}G4qk8y3;)u+#23%{m zFAJU6B~PWCfl{@7t|I8j%u&k^k6xc>YA-mn4)?ygiDEXkf09C^F;$>UezfHkFuM<> zmAMRjF&SIAR6IYH)(|J62DTOFJ@v(ejT;hcnZfuv>7ja6SpG)k=YbC!+J<9@G&DO% zHV@UWEerPX8TxeA&L?{Q*h>sr?GH6w zm_1NdR>s%3)tJ%G6jNE39S1UNp7pADmIHdw*AUL(?z_lm!)fT(oG0BZYcmX=zk zDl_6kaXE1FK19(;o)LT>3;Zt=vc~UF!C&L+8=dk$G2N9wFtW1|PI1L|vMj#q8u-+A zAWdoytb`0R{UWDXKF%ZMO4z&{Vw?Hz^j>zY#;5wwJU#4q>!U!w<9K-J{O9{pPJ|q> zfz1}0vO!Otd@KgjB@d1E<+rd>gVIz%fy06v_Vj`=FVw@ z3he~X^&72@a7j(G?k}u=QFV=)o%hngWF&dJb@#we1;jsck3R(yE70>^+v*qN?nIEj z00kOHs$*0f_b#QAmU}<05cvptUF5ezg@W8B_d@095Ye|Cy;z-n_6 zX^>|yRDkK2C!#w+wK^&)+=%atW6kI497EBZb1D7tjPF5gWd&^VY}#=X6ZwYGdad`e zWmS%|c9Q0gu)`P~^Ax*X+q-+o+C7in{N9v+h5hhA>8CcOu%qKV1Z>9Y;Hvi|luG}J z7gJzLD3yVQDziO62DqjPa`x1v&LB2IhX-kt_VO|xZ3Qb)7l(rwxP{N@cYL%|{U zey9~CQzhx@O!=KerS6_6hed;u1xAS7#u!!;)H065m)+=(wO#E-eQ!?n6D%&QPxZIUnEG{^o zD3RP)n=pf5fW}G&SU~x3fV=@O2vw1a3hy8PRPmhaowu7ZC~IEZuTbkt{` z0A6s>Xje;U7%Ny_Ae!UvqWI-nh8%U}sT8K0k90)Mn}2U=3JImDr6r8~H6@@`+H*-v^?`(=%{q|NGB+Loikf5`kY8`c@t8TzO)rD>ZdmwH^ zk^h0m23~ylSGcSfLZ|kEFMqajVpzgW&G9R_d3VAG&8{z$HxV_8Vd^Av0p1Q}|O>Xo`DW-ybWmDt-(KDY8wp zsnm+F{HS4jx>SFz_i_BnCuRi9(4AKu;oFl=qazHaI>!AO;87X+v7*ePgtS$5j-be! z!l__*$2Hfc*MM_F?&0H%7(C! zeDZNj3#9oejgeY! z^`07>ELoAc&ink7JSCZDTO5iCO*+XhQ*W3_&TuCL-R|u3n6CakDtqzu0MmwT-pWWevcROF)F&fh2U zw4WbNMIHXFlt^NTYNPz)P}_?+yDe;a8J{?&LPrX}Ku*^BA1_=B3XobC z)SXkED_y;gtqs$=P9EDipBIV^Hqc-nv&gaLb_H9OgP7)rlTYVR_nm3?iznQs4ly8x zJsMLzY^iO`_a{393kggxLpGw%XGnZy5h3~~({Jk?l`t)cOP5EKt)$zq1o*Fs8`2!a zmr-MK>r9Y8LRC<(SX_TIA1)1+SGLUc!R4fEwnTD;aJ2~BC=g~eiHG>oVa?G`4#-U& z5phJ9L?^nRpnW+xs1qzGZwt*9vN;TkhyLl_oY;3)E41I{R(^1EuJ$0(n&)a0w>y9L zPzyrJ)|h(VTI$x-=)+e_Km78{RDlI@r?a`U34W z&!u%P3>b+mCU<@YARcz6KdCGsDFj#br@N?l8v{ZfGLo|DeA4O)DyrSZrMhUOI;mSrminC)j8NL?} z@z!Nm^u^s;I>54|jaBi>RIgzxD4#t%O@Du~t$QwA_v?oo)?$|hZ!B6fEOs;OZ)GL= zZaK&W_95YG--=uPZl@y$GL;aMk|8l>XBl7NifN#T-ZH_cUk>;(5u>TtvAvZ(ZszCe z<7gkwVWt^j+6;??J+>xejn~!&{NM;AaA9aveN9R^^I6^ zIm#WW;r@k`eXzvFK=qU;Ew!r6YaE4KmD)VrW)Ok$*b*(d=4;4^x%=tiVT?2^G~$)FV$N*w^yird#jj}>C4SC@YL6I06ctk zKCo)xW`WnM4B{Bp&4`5vO2R_-;l+u0xs&uEn;Fm+4o#J@f>tL%-rKloYYO5rE=7{) z{#^ae-~OOKqJ3~6-o)osW9+If9wEsm6^6dx+2RXV!cFW83RkQJrfN6(tgGvy%|9A^ z%JvBITne9=>%Hvgh~eIl3!*3|bW{WoxduHix;y}GAfuZL_1~%?Y9cfX!Pxr=>O7$m z!XB??DyF&nzimG}XZ-nf#o4DbUvPt8ZbW*G_;~BXtKN(G#`~J_=a)AP-Xyz`b5%!+ z5$Z|;(}AYg93=QmMv>y~pRcUHU@dU0z5D4912J}u`_ zLhUN#8>Yh8gtxGTJ~fYjoY8eEk6~K~EY4PV7C8=`tx+;eM6}zi@3_rOy;DFS6;S+a z1`ODju5!!+yidns1Qn08$Fc7!ea;o%M0DQ(OXVh(*JCibGzjVdEMBVig;VPbR1=b! z-$#7c2w#g`%+ev7LXnSvQ!W70;g<%M@Qg< zifccEg6>A$m#fI_B>svD)rXU;3ef{Mi;l7NUEI32#=%p;G&C%02$7eooDsyFQ2l0L zs5@NxdAjr)#%1(SKmV;H#Ftd0O5e|nmi>?DoBP}%|ADP9kcYsuE)JnD`x#Wamml9luM7) zSph*clce5wFV?yU;M$C z0O5KQ*Ff&iaLqwvBeWl`BdDYT{Xr@S3bzxC!lI%(1@)?a5+12c!HI|+Q})Jo0kuI8 zNO@}>9mmI^khkhr6Uc8aeTpalq;i~cc(hMM`C=$Q$aFi{*wN^CoyHedxgonQLie7O zt(h^8q(C|qfOq~K824oc(Xl;;d=~l{?XNZ>%M24aiD_A&&o+J~5^6UHk&(4SLbv4( zW=>yYYGeK7DXsF7VVA&o`v)4H+b-2Vz#NrleC}0iUIh~_5T@nflCX8Jsl8?2t^qQa zR2e-1E8xdY5d#1616U|bB?_PHs9_VzSLy_{hE4K%TJ#EK?yk!^Mk;bh{P0JV z56b;2ZWzhzn*N-}t*&D2lpJ0j*a)`i8|iHVEyLm|+NBR-Az8gBvQBtL~|udj=kt z@}X6Y&*9wT_u^1RKsCWfS-+_-IqX=C>6 zO0+(w4)I#|{Y}Aq!{8qTdxZ+c9l5J8sWa-yYzG!Kh|vqPcWTB)aMgHgq$2ofj9x#c zMD{CFWs<_xKKmb21NN09Qdq-*YL)l7m%Vl-J*d?;o-vS_-Zcy0K@&LDw0F z)&NNR&BG@}bZ&305iT%lI)AU-#J$GAdL`>5SVy0k^%(6nywu7r!=&x=ButqMcyy=f zia`$_g%#QoK~&-vQo;#wGY+w0qmkGJ($Q8p{&;XS^kedckdD@sFEOMS`BN;w@+$<$ zN^57&h<~V|_IQ-jl@)ozLxAAUO-;GrWx{vg#jk*iX)=~9SOWTv>9Mm{T{__~vptxK zB_?as0*KmZLvY37^W`UifdzKKAu0i~VJE60Ii_tKfpYw4+)<3DaRWYu3{(BqeV{DQ zntXok_hTNEZW1LtzU*9Iz0`d-JG^Q?q>+@P+IB?+k+#OKJZ&Ew>yu1z3W@3&Ja_s{ zBko+#SFP~$PV^15c7nkEYy-7syQ4j!C4i&@JQPb}ZbXC9_I`QE(a2@=`a zODed4+OBOXqrlg!JSwgN!5v_TA0@#cd{~PwAL8L}iMXG_FUn)Rn*W-#@l^Yr@H_e~dQk!C4N+ViMrlNNW-P`9eG_FtywT+-< zbZ3fDU|AgMp8m|Cc~O8j4O$G;(T1U%SAvF`5q)ohOWx#1^`JO;myYjWm_TUGQ~%2t z4TkhL%GjUN2XPaFB8jKNhXVnHrJqKxF!19lC?pRX*$O7I#EuYWNgzImF?Z0|(u*gq z6Rn;CW8DU!ifO-A=plx^;5DXjzv4wY)6}F{-dJf6wg7F$a6d`uiBtx@J^(mixh#H<-5lZwl+41LSn zU-&ZG5SwFhj;mN=T|Tp7HCg6Fl&UXwPj00ahvdRTi+d4BIJ!C^Ca$G{kBS4%!UHr6 z!U^=`lRr*ImXZTmf8cxe7t|m1S9DE3D)*hn^K*{b6p~wu6@C%3pL7JK@3F8=hS!fO z2RL+w0}R9HT=Bc~OP5-)DPQ$w`y6*AwR;b)Qy)_m)N))B)2C^~Dfjk{THvxzB;~1hi(Lvc|RQZmQEA?B{88x@}93o%ah0>5d$5)UJ zN0V(Gi_pu7A>Dkc^K4PuFyn=RSH{FGPfeb*89 zK7llBipAOBci;o_ZlXQy3u~nC zTSnRGV@;GA?t19fxh3tIT^y=(XR0j ziNNT~@+Y_mf@Tdz_*WGCHC=zM;a_nQ=|mn@XG)VLdfruD`fh!S`T390K(%j$t*`07 zVs3(D%f*kyy%)#L6VC9x&~TqQ(_L7t`^&Wt$?9hSKtU$Dg3B+(gpU>T`-1kLcYxn{ z4dVR_riU*dP(}W)>l8^mxuN1nQ_ILMX!U@>YJwz(2cTve?xTmEFSl^2Ic&Q>|6{o( z$JD0gKN4pAbyj=xBapF{jc|wmQib>pcmGxV`*o|V3-`u*jP#`05wHq4+cQ5-U4KZU zW>t%NUVQucyUk87nc%D-2Ag+R5N|^;*?wZ(g;vYFLS&(5zg2ObQQa->q1eB6+o z65ZcRBFO?rnDf`8{yj23|Mb@&2An`Zt)F^+TTmgB{QqUJ^?M;8bG=&;$_MyM9qQio zU#rK*nZl^!k07}BvH!c((f&(G*g<>b1Zy)QD1|&Ri3^tO?g#w-O!(+ZUe5is=5 zoMlc&Dy+h2Z`S=U34fx9hJM5a+0Fn2q*F(31QGzhrsjWprn(|DXcWto0-3%E>?lTE z*xQczJR{&0{&Q&Sg5izl9SqN?<6OVwz4q^S{h!_f$SV#pCrOam4AIp z5^_}S<-18CBRR-FK2Cu6f!xkg&j^|lD*UG}e&Ww3A(8j~pVra8ruu)*MDi(EMHF^4 zv_3MFT}-3^*QWvFJ7-3=LL#eQ2VXYMpCH%&I0XL+Ajq6wSAf8%{Yof`5OoBKJ9VF( zcw2Wd(BF@aP-97AZLN?Da4<(vAEb9a{GEE_OpWm-gMHr5T7Y-sfnY2KY%x6Km$#aX zy{qlC=CvmZ+KibZ7_dEUxgJ28F-8TLK}d4&CsEtsbs@r3{F|BcV8=i${V#yB&3BoR@pvMocnTTO|5Io%d&k7+-?% z^jl1RoX{&Gyx zg>7f;@(k3cok@)nemz$z?&2G}tr0yT_`mI)9{a%)9uf`vICP(XIY@yDqys*!%e0U(`kBx&s`LJHD+#hw#)=fRx z1I(0bZ}!L2B^b+qe68}=`2IcMKKPP&Czd24f>JPnH<$t@xA;)<8h%{=CC`o~C8Rjt zB~P07xaaQxeipUR5u^^ZI z)%Mp4+nZ^$swAHKaPV8>VUZ;;gt*x`@15!kWlfa4b<(rnR)~9p!Ze4g z4WjTMTk=u7#r4mTVoLJHy9c@zo;9G8ME@+z(?2)XGih!uX>dc@DQ95_^@wU=N$D@o zzTJP0WY^^9Yc|9=2Qg4rU9`7dXNa8K;_~QJ6v`pSs~7TSiIRBn7uN$Yo+q%B!3*OM z4{YYwk~jx`&zL+8)`!&$Z@l{!kNt1H+A)8=c6rNNPY6TYdB;#vrPqG2%}wp!);8#F z{99Tu?M(Do9*K&>3vVwC@4Q8m4GQ?nl%kC-4+MCM1e!4(dKd9mq&fBS0I2Tl2L4P4 zuTLLzgMHlIh@5i8GK1e7CqQ&O9!#ZgXbmo5+YiF}>`ZUenq~hLXm($oe5$5vfA>E3 z#tWIEv_5dCmidCKlig3xN`FHvvtP!cGKd4wGja#i-iM(Msem;)oIYGwL?qp4(!txq zGh(z9$+2wmGPX;7*<7YScL==dA0G7OpX-~+6Bm%-rA8=ue1?GGjp*pzH70Z3`^+EH z>MstP?K(!{nk;+*2GzD#u!eYd-U!HpWT&*jaYu~K=foMi(ldwl-g#8cJ9-aavpYDC zlTfr`@HF0a4}3;QIcb14#FUE@bsF05|I~B~T9I9kKmD`VKz0SbT3sbZ$K^3fV1Mhv znELZzwVzUhAl6Ns)^3(tdmQIMRI>`wS^JOc3cfy5-w5f}-N-JvjDx5UDkGVnU-;*) zCR1hV2J6as3A6USp^<@3PTlg*qBH~MQg!#RQx9Qj|2KTR3sxf=FBd&OxnkTTtC*MT zt|gs6+O?sMu`LQCJH2bKie%J(jPBlQQ1M6k05E~jV9~?ggZS*-p6jRn{RAY)0rk)u zRt+iI1b^?hAm7dSacZJ$^(s%z2Vw`Xsp%nhsSuhSD;H_Mw^WzmjTw8pVK-fg*;-u| zn^uqUyxFb6ebhX5(Flb{|Px+k>(5kd-M>2ZhDEXCeGl z4ViyVH4WBDP>Q^89QKszGlzZ8-0Si+4reMoRZ?On&$|rv2Ih$`oe?lizl(l)Xll$b z@nFKGM8;QN7o;Wo-d!4?`1ntpeKLNYGzPo!@}lRDV;}=FgtV5rYeyuy41_UGW)O?~sNf z@0%OL_WypM^HfPlZx@#BXZ)!RhxK~yDvzZ~O^?g(OXG|Yd0T9mv$jL0*I&M0iMqr2 zO~lS~Z|eR)?(e4Db`x7InIxy|u{`*5n%HXLMNEm_1}ZsJT5f0IYZ)?=4oG|4bhBN* zSGJOF8@F+Dz2V$|S#6R9kvn7LP_Qn>qjPLvIlxHuIivlzg}?)3gh{6;rth~{&WXMl z3SN)v(f!w8){yKrrFZumsINKtFq>g8M?cmHlcM0LcsVX^M#ltG&18TR^(PPO9xOmS zY)SaVTa=k?(pNgG5DDTjT(TsIZd`#=GLpCEt>JXzl;1t{TqSQ-u80TQcvMWMHzp*J9b@iuAw!d8Ypl)+7|FzY$aQTfVub!2 zZgax?b9+O#JH6L+(Zlxz-G-GVi@JD5y9fFVu!@k~e_hI3QRT((>@qwDpKIU-nG2r3)gx6?cH}-$!NAV>l++SuBcrRPI62H02j-X1jUv z^+z3M*kdDfr>_galYH~TtYbYIa!ts)db2ke28E=1GC;!ocF*KvJkBY~CV=t3J3vO{ ziVug61C~d;y4>*9&gm#ykw$nX^l5zFgMW%t{ zc)?TYom^Y_>OxsV>mU$&aO;4EP?qD9Si-64I9sAT^Qsf{ays6h|1${jwxHHMGcg;_ z5>_syxLe-)`w&DF-~sr0h*@O{M6;M%&+om*DHg0s@2f(UB!e?gXsMH_G^XokT(cU@ zX2Dk3rwM=Re*CMi4$6{?@7k?RkcUCZwcT!OZOeXFYLsU+F`4`y_$-#1tFYoJaTh%_ zhF6vs#nrPqMX%ctwH^9-dLzjMk< zVCs7kimV{>YD$isb88u!j}T!V0cDw+_!MHvoi6Xz2p~00LWZPVqM*GSMNZ-3z#s7A)1TV8DGmakv^ zd|w}GDkIn3$JzVOKP*X@m9;9vObp>Sx3uF#GRDZ*_`IeV(fi|7OeAu^f~@O~uh$^; z=fX0ByR;uy5>-g72RfNC-69E6Kf}I-QFmCiue|ZZrY1I|o|&@{g?3F=ObUnCF1@)n zyrf&URxz|gn9pk;mtsC{ z)V;UVeUeGKW=@zfKA7_Bq2*Pha2Sn)LD0e4;?+p(sY?0gU}%W#-by~M-HWNz5>(^F z@GXaZ*Ja0)9;m-U%5mU{QD)|Ko4TuF>~ zqx4yF`!Bq_wQkF{OyaNa(`BCCISZ0Vfu6kI26PR{-CN__ts_k9nx{=|evah+n9g(5 z>_x*9TuGR)^`INeto&6rp#S?TIxxAS9`5ZBv^yajN5d+)Q-$d2 zE6u6g>$^%>B$WiJgu-nr1B;7`0c66-c|NZUFzqwG*Dy`}9iQsV;ej&)Fv%T>YrW}Ud-pQ;zZkZA68u0P>W9i*Jn)F*3 z6YWkg1s~4e+j-f&RalmZ7PXG6{OYUQdsn(g4!kE(_%d4F-9J+_sO!V z(F~a?_xBdNc8V6vb43j zT5Yb4KrjE;N+2uIO-y^p@#G6gY8nsyS2n(22cK|iq25UzwJS8ubOyH##kMH; z<_4oi^p@LGASDF2WE5HAjH-Hz{5}+Z?|%Xpzbee5bF<>z=O@10{_B(BIdRy~oWG0X zBy=d(?u?b#lJy+Pp~iwUie^4g!Jeugt}5y#KJQ*F)mlmfnZ;vj)cRE;N}2R(!}!xl zdA`a9hS_y(6TP-UAWJ{<#{F#XW=j3@P2<(|$e!1LAkGhVG@G zYs9JO$hW5C+FrC~@X*65m5t=oRL<3)#gV4(Cc6Scv?vC*5x zq5t|?1#mJU^DIX&<(4u;xZuIqy+2v|pH2pa)xAq7Ng*tR<6+%cyT5_k!m8)Xr&N9p zPj8z%iWj2gyr_ZYVbj%%k6E%B@xM7_I{z{c>-NyX@)lFy-){*?!><3q-lIJ8?srSR z+VKnCDJ<=+`D@a<5q#HrL)_kHVku&cow$*Bwer-b41a0ZBuvn%V|K^paQE%}LEAX5 z463Q4&dW-Fb{@&G9{k=`ugSi9NNOCkdrkFq>RjLM-_xI}3H|*75`QTq3Vv6l z`(G1}&ytG1i~ES>i$`1X?h{>CLGzeO6Nw!b-ot-j z@r9_E?3ZiLQ+%(?*Oi=^dY^U0Kcx%w0LgUFhC;U&&VDD7UC0d__ZW6raHVct(IRc< z^0^!v8_Vf1*%6MI9{b`EJq+vuBml?Yt2<%fRm+lpSW%8?83&ZtLTuG&XlVFMnjX8G z2Dm3l0HZ8l&-t_8+f)T|ofj6WCmiw}e@A$JtQJ7zjXYVA#y%Zpi9^q(@^;I>dyKOF z?(YS)C)hy|08fA6b%(<4NR#OxC7)3@AA#Y_#HL{U4?*Zv)O?Ihom>5S%#_E>Ms1$c zu?-+q)PU;uIAVbcW~5CZ{j+$`JK-Y^B2uPhDRPIn_pjSDIp5vBKk0mtFf5QfD0-%krW>NeJ{ z5FdWseR$+~JIOPOyI=Sq;naP4_pB;kjL{^|qob5hz45fCCd6@vlLRoYocq}Pxf$kI zC$yr-$_V*M zUsaN*<6smGIDimZj6^#<@>I!;b)o^`B>;`bT=)W7*FseR!#5`cE#Kv5rGpy|fesUU z^QO(&yRvm>2Og~7p8cqaC~KXs0_h=;f2mpjxm|r$DVT1tNMM8zMFM;#6%AAz)$c{@ zBZ=E4*}h#Al(@TS=ecV(jg^sUUYkgzatkwMVLWsY9^(I@>np>mT(@Xt2?hv?5=tvb zNEno$AP9nVHwa3%#G*mK0D*;spv0m(7Sg>C6Y1^}q)SODx$|>x_u1#%d(V&aY~w-r z*7weL%rVCtBgNO}E__4fZp4aum5%9K-5*`Kw=W3=%K0@n%bydpi!ZV4l0xL~ARdYU z6X}mry2Qqv2fw!J(&7CjP|^UO^-9QwK3tYc0N-60yw^v`=)xe9;JMlNZ7>LzEJyw_ z#8I44VQ9&?FOYQP&g&>PRUKX3Sj3~y@r#u+tm6cZ>I$U1yXUL>pO^XX|N6fNKJO4% zDu2D|yfOtU{tT?O8ou3{nEh!;&I!dz`&9cOz7_WmY8m8Xm0<4L?#Z$Ex$-TLIYA^tF-B;j8{ z75{oB#B$K7y^Gkqb@H#N&2jc|PY35K)X!L%&d7mM4-jelZNU{ge1YI7y0D1xq*DlG zt{e6ipY1VMaPpZ(qeY~t)Uo~bkfD}2W`k*)sIv*qh`a4c6LYD5Oo!EJ_A s0 zS!GdR@`_|`%e-I^33(gV3+~uq=Os*j?QIu8HjT&0bIren>A;Ppfv7BTy&lVfjYX!+hPJRn$EKf4HhL0(3>4nf|$ zKgSrD0y`8|_;Buni`?V6)5!wijMHG<`D)0ny2Zt)w%r4wJsf{+&kU6n=c+rkcnG*G zYR7<6`7F_z%Tl%kQ#4^waq-kl;-hHD5NLqAjK?E9%xuK{Te!5z@^B@=Vadhb6rC_W zrY4-i6*Slw_|L^1vxP`5R$jB_Fte7}0iG?e)_jwn`G*0Aq6Q|Z<6J*3jPEiS*EGku zf5Q3VXHHcY8=L)kCdfI4j!sGfHnU;+7QY)C^&9F?6#Cvv)A^t_z7FNxh+xc@M3jqk zwN>%E8BJc&SZ9F_kE4?lGwuiC+8U~Z_i}Hl5~GBzipV3RG=cZHkn|ZjBqkR!KuNo~ z%@*K@T6or19|@U#+6#{VB%Y*0WwyM{c9m09Ds_m@ewE7-&8C~PB-0=Y{_d58zEtTK z(KnVWB<6ebIT16_eecY+0NBjG%W^POYOUoL(A3<_N1y6E*O@W(Tsy147zqfUU5M@3 zsrxxWqEqWps=*IT(_a^X&x3gqWw55?<@mm6Oj1d4ugUP~|nxpd9Rf&Z{az*mfoRuWib)N+c~W1Tw7J@6Hd29x!mdp4f4?a2*1(H$^C@{<(J_X$jo zEB9Mj2z52!T0A4K<-FyOUtYM=4^E2LP?wS51BF<=aJZXz9z^cI2W$f9AbRpCE#E3U zrRQdi-|4k4U)kag4w!@$mBvocNoVFzYc3G2dL);crZQbxkK6U;4PFFe!oUPR@uY`@ z>tmTnSW*pGCrdqS5Soc+#)7#O82HMF79JOiJhq>ybA?#=rAGT*U0q%A02lQG@P2QC zylI=hG@UdXTIJv4bTENB^!qWfl!?{Uo{a6^()7alfGhkbYOUENKrWL`Yf(FR|`ZOz>Kj z=^!Zr(JnKspdHr-9YXXa8h}^?M4$@Lc}t!$G$NRl4rzHEBIi?Qjzd^P_sWGu2*GRk z_9b#{kiwZwSs96?e<%F$2FUscSHSn5kNd^cyT?aQHl9(sd0*%wIMmCrKvv`9cVI}h z-|7SwfICqth&aX~Iu-UI)cEfH&SQ1}nLq5U>5p4lBt0`@se*LuDX@-?0vo|b_$;^& zA2|^w>b=uYIg=2#LI*C_j`+!kWO+#H=0rhujPW}Dd%m%1?lEqBt*_DfRZt-+Z~^Jzi@F zzUklastR5tp_5ioxLSBSuK*b+~h4>Ah<4 zqg#@`7tI3NNzyk^lZbcM4&5!#A9a=5jq(2B0-V4{da&eX{5v8NDs7isSUE5?ND8vA z+wU!rf!9X9$++@6MsW1h*dk-X_B>G!0$u&h&UXBO%k*FlGnPR;w@az&D|ViEc46Fm zYlQ9aQL>jIAt7(t@(3=#0lk8{4etyymDs=L$C(8~Faai9BUMFt#ObJjeTb78l=vJa6sOM*oHQPbncZ zUsvr%ee%GIeoWK9W2ppfk!g1oqa<}K)3`Rl7)g2Xr9j!s&3eUVnDpX%6*<(~cmNxn z;4q3LXh=SmsX{NXBDIN>Rv<9+KE&Dq5K2SAuP8=p*uL)WOd4pt+{Lf0VqH~Q^RKk- zLzGC6*{62-(=%At1c&=h_v5wNk_$Fq|F_+sj(d2K5r5cHGxts~)i%`(Jv8Vd_P4s} z_xHaINi*L1^5qNu<5h33pWpAhPep0T&t!>$*IooN1!)gH@Q)t9o}n==mv&F@up*N#X$qg6tZdNi7`^6uByHk0Q*{*C zKb7?lD3#Rq{rYbd4fiD-H`LJ7mCOh7Z;bL^I6h|{4P*^Cv9b)n0x3^ z{uV9{Ryh5wyPDqK`QQ#(fSAQD56&JOfQg-Uf@IysNYS+?K^$rs{be}iWPjUQgY*Hx zZR)8N&Fad*o>~Ys0fEJs3$kgc{WdVvHtJ&iGVQNY#yDQ02bLPiM+o_B2h|~+lQoud z4j!hUza-yJ#lvD33UQeU9*umvZAGR{CKDf?6DTDQqU*>UT@TfcrX=lP6c*N5gd z;yU2=V*_{Z!kge;Ds|-aiU2gkVjagVqA#x$7l00GH*Eg`)pdu7C(wkpJ5zxNKi1Mc zjYKO?`@XOVfM6seDFdBMu{>BRH|jP?I>ZAPa&dlnZg(>weq(31p`0uF1%IG1$oVtW zye6KrN%FG4*4&{!qJd;A)VX+&)wbrtw`M58vEQN%;te$^KGm=@P&zd9uC>%*lAGq?&I_kKDYD8BsH;|SA6 z<@HLx-6E0vIYGzBME0lo_#3VLX0A?Y07a3~*fjoE%?h?V3@1jIl#c4w<$ zPOu=cBgn3&rlzK3TfGtB7ew{Hz_FDfX(Qign=GwY1l~({%F> z+G4m-d`u~rl>1Q-lMqeCt*f1|3E(ygB0#nIvc_MkDHhsLX9tOib3v^=-SW;HE$WTg z$mhIkc5$pQ-a|h_{+v7XTneu;2P)RD8tKZMM>@gZ=&T$ikC^@c)w=%fU*Z4$2=e<+ zLu*pDd=i;jz)vU!6)e@ejv)8St32xbI}`p!#X|6~bs@T$0m;RdkL8o;?CNTXRoQ&? z`}82~eF+4_RcNn5c+5`jGuqE!_IR5HYdDlk>%tKN4T+>>=^&M0i?S!>j%ZIj1N^Q zEEzv1icH3IS=Xq22q8|%^37tOCX|vtd-eWxWiT^0SMOd^Z$5r!{jG0M3MwTl>+JRS z_YZA+h`)0czenq_l%1V0H-PP5Ta&PN)q?+d-)PR$Q|fyV%uN-SNcxC)@y&ZOFx#rW zb>@N`7~!~@VD9|y)6P-y1*Gz))cy6jC@-NtE@4!c^0^iE+5e)={rm}s$wr8lhC`Y` z&K^k**j>k7z#j0=T>h8&mzhQ{AhY80a>Zn5Qe=X^*!re9dQi^80<93*HonG zP35>tU%Kjbet=5xerIbXk-d8KQX;!COQh&$F9Mf1+cF&%t&PiyG1op)Sj_dRi_;Ju)5qAeQq(dPbuPB?w%zP)Zx4&Q*GTBiwhSyxt3=Ie zBPV1B)VZ(y?4&GNYLS2NOiV)JG1j?L=A7l%M&QN}w`~%wzr;WP9O-nbB(MK{pr`KE ziH($oC52brnBx0AH%p2~*-6`$GrLyKr<5~r8hwL&+~(_P!V#iS=3JoT?FWEn3+--c zK0u(1P*2@k@CQrJXzku0r5K6c1+t(tjev!i0d@!DMm!es&wb9@qQnNjKY44oDak1A z_!(BW@^EC`k56kHo?S5*nzQDf#fYJ2lLH`2DK<&$y5W^41@RfqDL(5R(#0r7jWT}< z{>xrD%?uDEw#DGKHR8IJA;_lUcN5?Cb*X%86nwKsrzu+S$I?`jl9TIss%9@#!^WUZ zerfM&nF>LlQ}fjT*CFTw6Ql4A7lKXFd#wEZAK;-__9w+mm&q|>xQd&vjXvO$O>{l{ z4}^qo@f#)6e}^O@uV0#>FBNFLMWOnw9xwchyc6)=oXGm7ABD~{i1Gez9$i)jb zBabE^7~_QIK~f-T`*OX->2coTbU9nmdUn}5Dw%L)b%58}e%`h6L95(6zSF0&1PDeh zN99C{N>ke8&t&-LP_iRN-$Rhh>?pc`2jl;|OM+Y5u^L7(rxB`^7R0stYl#OX0L}Xa zhHQ)-Er7qgVn2qj#X5XCCsfx*rQ7rD*^wa#o3E+f_IdTgj0N%&epk-U%VH!5C>%iT z)S7HQxH5U*cjUz0ziKKIJ*wn)0AkWCg3PfmQGZ=LS)zVO&Z%BD^zr%QMxpTat)X@O zPGweGdQbd(@MU~O`Q8;3dhZ^Nn78D`MAZ6MOn$UxfYZWTw}tZ!p=UMPaY5f-q8A+wi7b_*(CJvefv@l0MIxjAHgKhw)z0}6rb_+pHnjG~ zq-q_vRF2vS=EwGJglrKV!4)-La=*nG+li}aJ&hb~wIE|g!p2L=vrZ}fQPvg~Zvfj3 zK`x~{^V$eJaF+qoS45T$pI1l5O8Gh=jKqf~q^M}jXE(;*gparfOfPL{un6>TuqaC~ z7aVbpgoMP)%(a!A4yn>2kny28XPr`MYy9ZF0=9Crzf-&FH<_Mf}o?-#-W;&;U-dAYb#pNkv<4NrE8rZRb1;ppXRjuL5k3f zM15X|clSHkcyqmZixSTqqtjWn^MLpdTc|JyiM0#Bcx}2bZUE96JnZohDbya9z^7v0 zqc!7d=#zfA@>>L6fhZG0Cv?r45%VBoMJKjY?dciOUq6lQNi`uz!QR8^G}6}Ee_o+? z9WE65m|c&ri0h=Guw^<$lFk3)>N~{! z@6`@J57;3|V`#l*E6Mcg&=BNF@(@xPICu2B{F=J_aaTA=(P2KMztp;~f3g;8V&Txd z>HBvimu=t`N;H7UOc~HWo^koD?W3eD7s8a2;2Jz9G{RICg2{WSktBrO?D^pdq;s>{ z75pDAKmbj=(vcbfhd^`lYXIYPUR`Ug|BYdJkJcW`Tc8&v>kc-$dKk>vod%n*<(w)V9f(ZC!1>$NZfL5B1uS1A z(i8zV$oeb`I6XBVF*M=1H4o!LbD3TOJ{^>L!9dUYxs~!*h6)3Mqxx@d3wXg7cKLM9 zyq`7B<}_i{R8ztD|GBBkJ1-hTB9NO)#5;zsadyg`>B`pn!gJl1ledTDoQdX?#cNH1a6 zWt%})7u85JTH6wp$jzqG$X$BY=3H`>cc%YZN%#ZanZ^G0tozRyW$>HWIK4rksJcyb z{Q^=n#Ws#*McS1JIA9Y~++|B^fV}=pC33L0_Sw#Q{s`JJcxVeUEP{vBwdR~!g)*0b zl@jpTbE^hYC=cTOdLdS^3!M_$TX|xEIOx-bu-ksnJo8s-z?iSGUyD8!(EEA_pX)^) zHzSB_DXT~#D?c$-B_)FdsCvaMLUp~0aipZCOCisxx#4lz(Cf#YHKJOJCr6Yva};;r z@YQTs%Vs*-i-${Mq|WG&Qc-s61Q`=Oo!L$vKaS<~9?fRmN0n89Xr2s)E8dark6 zSgZOe(}+vWAs{0`l~<7SZ5D9%Wpf#EN4jJ=^G)z7Rwb|eg%u&$aHW(VbPVG(n)usn zL|ou9-bC?4brz3=%W;HVob$OaN~E<2;{um11;M8>&u4NLatrXW*B@2#?kw+H&{G7a zi5(}3hSK3sh8$0;+7WxY&E`Bh`1Lg-rKL~XnABbeqG<{< z2v56{F#kanzoCWIy(Gw6JZJmxfHktP@R?SY#}uKn0x8|~H;$C!4gdMvfBzSnSvmCC zX+Bjl(@?}U4nF*+MBSFxaKwQA4JR}D?s;;%O_q;-@`6a6SYc5CJ1(QIP`RJC>_0l# zdIV?R;r-3EW@FE#(pMw!V{F20HV6|E`P9-Z-g_b*?Pi4Y;zr3&sjO6aktg3Zdt$j2 zXq2EO$=r_9d>;PQFM27(lHAo;iC8|MgS5=|;A-Y&7l3dYf{WtM2yc09XqivxC)yql z%D)?3(K@`=%&Vlbum&{$>&r>f*r{NTB*LfO-|LT-0m_Z_r-6Xvmqssiba?9a96f=e zGRtS#+Xnyam|w3MTSf~E>t?OqctQ1yUdUgOyZb!_Z_An5~hBYSYV{gCPMne>2;e6Kn zKs|wF$wJmw0q=qT%Cx}j7KZFWPQLHtK5z1Pc6 zfpv_NB8k6d8dM8*EA_6uJpRxG z?q700zr#mk?n~$}sV{DeP5yP5y=xh>vKFewvLGT8{61iCyuB9_Pe@Wt&COm5qpeOErCYm9TlL-=c~8u)Vjh(s;u+gpe62-ZIjp5wXR+=x zgC|!(T%*>IeZ|*jZ>yo-ErVn!pI8quMQWFNk3eY9`7r6XwZBHlhTUB* z?@U*On%-{@9Bp4@P`BsQJbJg5UyojSV|o>cp?-u}AAmbho<86+hSnU3$I3R&6lTpa z#_B0n;95Ue=M_p-#Jb@Wo#&JC#K?(CX7hLEuVl#BkMuRr0ie}uSJ}RGu&p%V*cGjU zm0qb{5$&RVzXd&dl9Oq!X1+Z334g2%q-QFKi^ENUp^r-HW)(Kid4KgY-5KWx7*9^u z>;)C&Z4;>^LX_O)TIc$7GKQPiT@vBb{aCHQeCek^)@0#j5yRL7UYJKz2*!z;uYc~g zNV@ARRHfD#F10=|xe!1j^=G)?R#KZwVIREC5NV*O|&<)$UAxQk z(`avHlw8n!yNkji)w;ht(TDLzdgNbXE-@8CcEUg8-D{5VU*7g+5pz?-vAloNNn*r= z7JnN@pf3S-$0oSNV#I_9yXWpiK3kkx;+#4)bN1Lbs=$SYj3jdO73@uSM;n59lPoe5 zTAMe1w(vgp78*-;>CIO=&pntAbORDGZgi7Q?`<<_6IXGFfzS37@2iLdHL?10=JEIA zY>J*wdA#@0o={T~F$H9Uog&shzQ&4OX!$B!COns&nvJqppZ)<5=xsO0WlN#-WRIV4gek?mZTY?Cb5EDfw4*bWqjWP>WC>|2DK zSz6Cd5bA1h&k>Oo??fdS9UB*tg_p+zC3WL0{RJ&4^l;w_F|ox>09FdA);HJNCBzS9 zRYTV7{xHjm$1wUT#P>33t6~pG_M1CgNmlAFJm0~ksAyb}ITuX)4PH3vrEWE)QeX;W z{mGr{*t3|KCl{minLo6&KGwjm$F*$oADtA0wLsREsEFIUv+D+}d z11~@b7K2LKz|NF`5f^kXU%uhtWK7#+OXWXLP%{vuK{NL$?zBz~C9vF8Bq(*Lr>l~7 ze#BMQ|KQxm%N67PE-)Q4PkYLlV_&h@I~~pqsHkeu!d3T-989E{6n)n(9NXI{ zc*IYHp-9f^zW_}A8A5N0RWomNjhgSX(J*?whs{UO#Bf<@8JDrj{eltA67b#pm{X}L ziny9pSx=b6C=l>z8G4t}LoZQ(mtkHkzvhc9w+~L5(VOqsxC+)Jdi_{qZfie{Ef`OY zK?|BzEikQiR2u!Md#FH5&z8R$igZn1gumN*#)+{hZX+I4drrL> z739hh&MCbOD)^3*E+go^yvc6a+xl-p26giUW(Ss|OQFpg5?;ugWPN2~kZST%>N3g6 zGKh4JLtX85FifPS$(G0NKx(PAnzWChxhqCB>I3`oZV0M~;`M22_Dbl-0(zQacToi? zF%)S|05+u}uSK^3Ymj7IhqpJid0m;(y1R2Cq4M<)CGo9u$6JGh*Ep3w%8+UD&&T1< z{0z-JL36Q{`e$jUET^SYi;_nf!^5`(VDi@83s0%fc^QddWjZF*SJ~Rt=Iwxl*-&K47B(1&v3m$eO$*sF^$MID<=%jw-!%0#(8c0Cb5cAiFRbASV%2malQ0rkcQvC5`Wl!l4 zlBBoN$z08M46f1F7Fy_#?!faM%~}vvU}^H$Vi5Ac6*fXLDw~9V!TmGIT0AQ62V+n3 zmq{J>uASD-&U&jV#~p?CLW{P^#>XU$Um`MW(O>`7wWh&*Oi~aJA2k*Ac(MvvihCqi z?pi5I!pnVQubo{QZ)^Byzx1-u&*2|-fh?cUDZT8(;WqrGXZ!9Oh?BQ?N0Uqd*~~|D z#!5(dE?$uD=MAj%8o#R!wsIrptX3%M3Z*`diOP@xt=N|(jJKr`djE<+VHYOEDY=*? z;vGZ-M%$N4LOiVe*%K3bTH)j?{%Z(dM1?bJ>B&HmaTJ@F<+#ZQQR|?n^xEU-PC5^iBiIXtW*;TxI{KP8Vmi(or+fo(nuFQr5q@+K#5^(MqvB@E&)J}Gz59JD$lbcLQhdK_6hwNDQIPT5iBM-e}0&^lM^E^3%?yUf~|y!MVhb^!X+gb&r1EX3^}hkDXh z5O?a8Ai4E7=F%eNv7AxUiu1m41x|_8FNB|7diVfv5%1E9-3vV}^RDkpT}7iX0namU zODOyLj8<2lt@}+LTs55xkM=9R2~}B=LR<0E6-BNClYzT-uxXy zmTYteLoptm-4PBwx2RyHS+DoZrMw#9>VPH%>}QNyAtVtCls0_Wo#Ry}IOVaq>(be9 z9}Z8Xy_)eLWaEM^PQe<^y~T&evsLy&%l$hxTT@zZ1DmPzYzoJD^{|uP+s-08os?;3 zs29`Q=HSbK8<$z%t*E^GsH?#XX^W@px6EM`vp zPBqMzb~yXmx3Gml^@JL=bG!|je&=^Y(~_zn$Mqe>SxRb?_Ln)^36>uBI2xF7H^;VK zSD1TiKf@)0it(a;%WF=fYyb%vhxEOvp3qY6rJB#ekn0SqY4e7N5OaNT%G(Q=yQ$;G zZPOtvJxcXTG*dDgE&bZ>qM$42)vH2{*3#bnH*p5IioN!b^gnOd-*xJcUo7QaIzH86 z?zD-&0lg7k(S|2jcne@jubpH40yF^pwP?_x{3_2r^WZ#D7IT7Bdf-ABLZzrN1qmnQ zqz;oy)7=N12bsUL*Jlosw9{x84QV+d#h;Chuc%xeXjZ2ZTt*lw5g=_xM>6GO9>8F? z=rNga1DVLW@{OArYA((*Yi+k~HP9}dpc!E>(3HMIbEEou5sHGDe@q@3ebScV^dNs zmsjCliwjRvkz(-rULR8O@Qyv^dw}|{%E^C8Ior! ztL*zEVGE`OF+e1?rnD*8_7$r;xi(Tek53nrZG10&r*)Z#QabUw#!`)~_c+AYNR2Ax^A0ThRPAs5O5af?*pr8k>Za3q!z(!`Qi_T}NPg)m zfgf%VsAtp6ct_dASxcI9W3vv8Z6$Qy7B$gvPHk-h3M41oQnCs{pG68Ck}+!{ed^B= zY-4v>M;_zX^(`Hk$Di!Zjv~Xq{E|-D`t5F^rS{~#{;E= zI@UDtvQ7MRo~5W%%9>0SMaL=bPh}Mysu9C%AFqp={;N(YpG1}X7v1ywE&DxAItGqN z$1lGAYwL46Vu?knkI2j?LC}onG@XOwOkZEUGV|%I zQ$%AFg9N(tp{1A4pf;G2cDwspEi$)|`94p@ILfdrgm2_U2Vo|!!z3%_P`X&@rzeCY z^GOU_HtMwN!<~1TJr=x?-jhD4N5%jft0X*^CYdT=>3ocyO_$zPWEdKk&kqY(XJ|Ao z;3i&(?qqEhU*M^W!RA#uz23%4@T_ zjin;P8RU+`zT~z;MvqDPnqns!duH4;r(=#=IpLf{c;zC0&Fu!^b1#nQ^hxNCyuU_K zQwrp2K(2-$;}s*{x!qu#z4usG)|IKOG?e>#h|`NG&WAFoQ+#o zss3lbs+2{{;Qx!JERdBV1@PtnYffM#Jr0ymP<|eY{2hk)t+H6XjyaZ+^&^~Z1^Q2X z5uZR&NgeXKBc3b**)#cA8-CB8O^Z#2sO;yZw9p=W`S?P9Gk(u57IkSt_)dU3rAcyY zkOytCunec01by1ZruCG{5P#z{VSXzSVbIKTSH(K%*tdr^CAEHYd@+P`CSt=dhzx=n z*9ug5uOhCo)(YwQi4$9?R(ty~4t|9(;T1Hh&vE!$Zm=*}D9m)u+Opc+GGvpV>S;LE zv7llpk2|zrNv{2{o~Tyg_~Q`@iwyG+!N|z7OWM1D=Hi6k-_Y^bAPOTQIsOO{u4b5f z>AQ#==GzGu=HOJm)24!PK&eEhL$t@;Y`i5vKuzngdurKHMOn#e0TP9>F=_R?Z94)g+dF-qQe=T z8L?Hf;-m3alJ42%yLww_{a+v?_+V?LTD5#$ge|V97;ntkptICv{$ly5*VJsnt?zFS z+Xm3P9P>AH9Esa*T(y01V~Y^;v*hk??fu=%L-~I(ga5+Ue(NOdf1B7gvH_~z&J+6E z6_4`F_uB32c_9UlUkQ_sLb57VU*)aga=mi9j6+W!)(Sb*8VvUpD%v0JD}%Yfklr42 zS0?jsriUtE%Dx@J+ewj@+?smsd27(<&0qb-FJeYx3(BvSTXeh=7%+)UV$SQNluT*O z1bBlhp2^cd*Q~jC#gz(8W}+mnoxZ>+EQNC_0z7<_8P5fSyztt+V2{TxuuY?24`~49 zBQ-JW83({xw^S?8n|jYIz{vEC#7U9v>KF644igG(<}-qkjcaHM*R{d~c`*XXj(Cs` z*f>W`5tBSgJyY4!e)pDaH4Fvh+t|3NuZ_U;Y;=(ZV_DoHux7V*aeiPhRr=&uMoNYy z1&4?Pw1_IY*Muh z`3^2?Rq5Tl4>8B0N{CGeu$~YV$9dhz|5TKJc*-3j zi+_Y-${#_vp#I}aveoZDF1@JSQi>Jz_ORLT5@&AA3$3BKngYN_7lWlRx@OUy)W0Mn zVDTgrNbkltja27&w~6N&iXS@!!yW^u^dRj`w}&=5^ELx1cY=+iiR1}MT2G=6IY;z^jS8z-jU9+-em^jdxHn#M>(3kyQ5(YO7yXrjKn5>E};o4WkPXTQt*6CzOQFvk@% zPp8do=P9-lE9;|sGm9)#xeB0HPFBf?pNg0t&QxMlJLNCDvv9*Nfx4qvcMJfmyk7-!H;YeqI5yVm ze@P-IQm$Ci@Rb~{?Ac$M?yHjO>KeuZb8%21Q`R9x8{@ynM-$)LH2@~A#P@o)vMkJ| zIvN~c#udQ9&NsHTF^iXp-q?Q#h^`Rf1OPqZlE}6<)!qi88aJ@N;F4~7C9<*!v%l`R z&!ykZE|B|OJNEEzf1p1|84&c{y8oI+Gby0Hc#x)Ne=I?Aq3*JZ@erzB46{xnNAqE< z7XdK4Rs%{{Y+!y7(sVj45%g@3kd~=E9QX3EypWHDWH4Qv0@kAEDh{-s8J57U@T0@J zuTBbonI|2rZ0r_5Yxn9kl`h70&mkDLIE|jlyPorsATF|eWoc!eDa(5zqhbyCBI%1B zan%L-=o9KQl4AmBqpkGMwfh6Fw6MDrSBl`s2--#4PRg2`|Drd+JfI_)WJWaMFU6{q zKsO|$1&+~m`&fOPu!}bcwA&3Qj)PlhJD^V<=bugeHA}kTp}8BH zGjm&?ZKfiLPlRy-MeqGNtGC3!b%fu%XFcW>?_={+l=saq;;QG{7K{Q|_W~D~e!-9b zAv(h;yX_HC;?A`>njh&4o?qZme;YD!sON0V6T9dwXklT|Znc<}q~@9`_3FF#=jYWg zHbx%6z3`@@qiT3H64wS~hmH=VO-{I9iTIdScZXrUHZe=PlQ>#Sq{IhiOfd=P9X2%2 z^$2_vIoYVvR?&f>p!<3Qk5}66q?}!pGc@ueDvIEr4_zj;ZFe68n!pAGx0~>?A7%Ck zp}*1Ve2<^z3uvLtkj|u{DHqG`H#)2mk!zo4e(IxPbVTgfG=j zERS(Z4OWvAW4|`bKTTcWfvvYc1?P|aC_070Ryj@yfeSC4n4M@Vj{>Nb#;AnF66dZ0c`x3ZmLep(`)mpHFt~i^a-i# zj(C?Oq}4W33v$C?Z?r?I+hLg6d41}Y;U~(dC|tQw^@f_XeC@@dmOYTkSIguvhshF% z=y|vKR(#4igA*>^U8Y?so{H8R1leQi$2=ew6lK@Bx{5VYIX^~4YJ6WL#26)o&hF$X zo1;badMxkRXVt;}T8rRdg#RMEGZVtvoYym?Uoh0`ZM;|PDb5XZ%BZo5OX#>Hk!oQn zGNX8LxMH!WDOSoW@WvBi{~2PR?@NNlpH3J$hGm2+tPAOZ3wR^UC1uC+RWk3v06v&R zUp>t7;gy*HsiIX(M$(~YAT%79kix1aMXvtDyUsd21TjNvcr#zlb5V(6fo8?9g$u!4 z*F3p>%0O}9DTs%i7M(fJOQJP(Ung1X`!N#-sd!JF14)5?A{OtBF4*pz0bXVNm!3uP zQ?EGd3kso6wMwpgesKTX!@=XmyzGg!XE`HsV^XS22f&=|Q%cP%*uDoP%Jo`RZC+z* zUQ^}vW24J(ijCI80T%YZO24Wg&JEK*pGgljDrC0h?i%bl>c$)SM;rF{TNL(yAoibK z_kUWgr^oiv!Jz!YgtsmL=j_H$a z63jNX=@kCPDL2$3v<7;Rt_Q6$^_P%%QNb6&**TNN)1xO6B+*B_fsGc?<8tZty%??i zF7DPcX>a2maMsq*SFAWIH1bqo!7GdjEWx_Tm_ji)eh5D!nn40)4kw@*9G!48yT>*quYFAZcTj)6Q?51 zYd07n$8+vy#asv4o#UZ&!fIb7;P(ZE)VyI3?AG(=k}HFL+1{~Js2{HB!)(}Ki*^Nl zv0$4?)VUON_D551r9Ac8Nq?z{WkC@}Z#0I)r=9<6Qvn1EIR7l8WOthLd(!--@sS>gs`WlaTg!K4ZX8%2faD+fj?F7a^YX(vx;a#_fu%}#4&38a zkR#@Sb)As3phJ^pW>ayiFlPtt2BKc>S5YCm`CY85f!1!jvbTr4x7v{G2x+%#0i4KyM0_T?^upV7)njw4yARa_(2-lNg7WUtVUM02^El8d=yeu~MiIu56}=s@3_>dpqYvRCNBnyMeiPQ1n& zf2m$ore{(|-9RnEwS2Ci&5dZ34wUj#q@C?Ttvv~k07zxLjf%6Y`l2pXqPV>J*6*-_ zpNwoDI0IEj&Tj)-UiGDL0kqp0`+)oowgOg)l|HZ2kAHuEQfbfN&My$-kosf!8iSUV z42p>-Is)w+J018spTs-!P>8KUXLmH97k!57`Yq>M>{srL*~TmI*XcGin0>U%vu8O) zZw#+YFPC*E`)!@)yl1>pyPM@hNk3?Qf3{o*@^NI;hhkbaV)Ct=!vso*NtpJ zU@B+E^ft@&|7qtH=Dz>=;S}Qr5dThwiPoMnYy zMQsE%dm`lobSqG{R*;a9!dixHPbA};z9qgLd(&pCm1a~J`!{|LCgH*VNkqiqzRh-hg51h%p z=3X6Gf~W?WW#MaueoXNzf*=IMWAA(Yy5DfBu;5o%irIXr8%f*i*>m z-yJ$FSm3XpV!7j!vcMmkVG5ouuEgAISm=ace0_DyJW-U5a+X}HE$wqiQ>HZ^9l11f z9DDnRS6a*%_J`kKh_i0n8k4aK09Pn-4&XZ#;ZK&@O-CyICz$26f@61Bc`g9YzBQuo zbY8c87)wF;EyL0qG+Za~V^#xLXN9@yvevq4?y)NTd#fk3L>DZ(PZK?;9NW^sv6kn- zIG}SqCh0A|tKM*x|HloTsUh6kS_G<>bC?e{U9MXk!9x$=-X-%l-R&1a^6=2VOVsc3 z^bd5uTL~Nz$E|w4zb!?MfAcs~GfGo%^QpXtUiqjpt`%Fksdls%h7Z@ZsMgR&)Mrf5 zZe5`W!($7rc1HPG&LlB$R^puYhO@d_GoO&nhKbRgylY;OjT>lUjeF=6HMTBz>SDR0 zb09c6K40?V6YUWEwdIynN%XPqn|7<{hZ=0lv$VsLo1|To6)CMxzj~a9Z2(XYcrU@B z1MMFaGxx62MA!F=AE4>Hv?A0&+tu^Mb(05EoS`3yxl!yZhB@{DBSw*r7yi2A&%f@~ z*mwLMHkLm^yyT{<4*f6mi-njOyRIr>v6Hj{&A;H~w}!y(gdse3Qi`h6JX$LK`;uA? z9~#G_G2k#Y?<6N4;4l=0Ya3*Iw5Jm6&W2e`bBY!;4!@SgTOk_oeuB@V7lyKDa8x(7 z-ug?;IJ(~4=vEs^Gcu@`?_iSfPHfLQTDAH3MDe5bxi0ff)zuihmc>g(*B9TbpU2_I zS@~oGH=WA)U2CCz-z=}&&o6v^nCv=}%M)UZNs8hIF$GAmF*k%rkX;}Sb}jL@G+!jg z0w~-(J7w5nP$&3MACI%ek1uKrY7l++89Kh3@U2NJh_AA`FEVsePR#=}6}5h9JEJtN zOYz{xG1zlsj;3#``MkulZ*=<-W9JabkcXWeXdZi$C621KCrOcaZe8`(>Yb5UXTq-- zes#e(rY2p9O5W#NeU3k~P<(sr5qfN{GEM9y5U;VgSCMA-ZhZZ^3@V_o?hfx2Pn~P<@YZt?i2dl{8gs_{OsG$m0JjI=NK{9vx`1Oy7 zPM)f-ehFS^XGb68CcI zh%6P^iPZ(n2sAF&(5;$m+Gt67QLl>ZNdssPY~GU!mLRw2%w2-jFIsJ|Ejp<8w4(JX zB8n6|090iRB+d_@k8-*ag;6<9*}_M6R%dOyIP>24+egAb=Q-+pv5{+R_DEx2d%My6 ziCo|Tw_N5d7^2Cu&;&$d2su&MGCNXTjwqx@mxw*o-oy%fc=9&Ot0#^4lki-3;`o&) zymM)``IUq?2fX3s{N?XALH}+w->`f`O8?69)p?y)xo3)2DV%oTlmzMMPZVtqJ1HTn zBQ`!nW{9Is;ZQ0q#S=#;jh~KCNUl*?Z^N2U8@p^`F0o`o&v_B!fEZ4i9->y4y3i4lH+U zq9o2A85|RwrkQ7?cAdxIi{%f2|TH9sBnf_8qTgCcBlMERMd4(Hh>k(d55!+iM*12r(@vASwabM=9wX6)&8`YKq$ z(Op;ckmB@}u0Oq_n4gpY?CE zIqiLP6dZdLwvwd+^?_+zwCG5}D9tryUYg5ljXtrTOF+ZYVPq<~PJTm~#skQry zp8ccEsgAfT%CKa`lzAIms4w12R&6{$_@&i=pf@aC;J=NN4$TkGj?BuaOG^t1;ST4< zy+9LM8HjBq6?EZ*Q8Tpk_hG!okERoy2xPu?wp;vOTb91xv9Dwf%;J_$SZW9G?VA$vP4JOZuTtZW=)LKearTDj;Z|x2N03wx zfXFtXAj_!QpOtWA;oBcB!1*3!ch-zon05-|CBsyjijTQndte-Ny%%oXp#=_~?GRLa%bcA`?HF+&Qklm*1O zp`@a1@r{ka3h}0@Z?>qtef5$+m&H9q@F+F%ey^i4G_i90V=R{Z5V>^bC$fy$+92e9Y(T)Fln-CU1t41vSAW?FPC`&w>i2aYR z`Clv>dpT6eDXJN_W3TTcOFQb4N$TRvRw1Ki1&WD4-IZ$=v!eBDKTRN zxIOhMu54p*fcF@gMy74d7e#C@<$9;vc!NJG)LGsj?vmV$e4c2s(HE9xe1E z_(>PRaNq<=&{?z;7havAh_xF01=9T3y$zil-{<=E$7%M(#jEX-?wm)5H07?`aTpt# z*tE2(l4`639X(f#;0uvW?JjRRR}BA_o6MG>6vgb~g!Ht8&m?dI^FJ_)%VR(MwxCoMXx1}uKIva z)az;RrYq4GztG{|Eu?L)rgij_MOAek6NR9HZSS^Yms9?(ztl^^jo$d=SMsYa(UFVo0&t+c=~4-xJ2i6pLnHb6fE z&?r1>%}1PHjpp2{Lfv!b{wBN(dt){p2{Vn8c3tMrMQ8Zt15no8R}I8Z#Qx??_%m;L zS;t8^cd(zM`FPuoW?LLvQmT;Hg$P^a{n)QXVjIyfW^kae(GfPcO2w&RKTi{)W~-t! z@V?CDJPIIXjKbCF>kC|)GEvZ6P32H*5pFG=sSG?e zhzKq#aHIGmE|lM<{|!@?;D$UT?DW%|D%z!fH2h}>;F*p)^Sy4U(~qV;W~I36%<545T!vd zNRhaV(jko?rGk`nmxMVgA<}szFRgS4f`LkRmrAD+A|UwN2kiO2^S-}-=Hr0iea>^v zK6|gd*4ppV`y(@y(k3WQ+6>}b&c!W`SPj%8m_d=g;2lQaV|7Sg&1BfaHP17~vAyZ= z#pZLp_Kw6hQzlw&=8GwpvR=tsztb8WWR}!}V>#mXTFbiVOpaH$wDl9smmh3Iqqz*( zbBFx*@Yy17?Onb9^cc z?mG8Yjfx^k#R2hg)B<%q>ZiC@)`^tE3kqJJKfF{NxWhl8TaKsk9935a((8h!c0!p^ zSgg3D1RHJfILPIXVndh9-&KXgkM@AX{xZ&YZdj#pM@Q0`8+_kh6%3We82z7m)UUVz z=+{4V9D;}%7DYNY`wvKgq^4GLUL4+%KOVa2kFoqoS05A-y8f96cY=0elrXkI33myp ziH|KyQTDOBNn>^n5Lyd!uOPuP zck=KUqQKbn2RFLBP!qP%nDOo76N$82{ec{7ozqa7z;?C%fv~WY;0x>t$@zzO@#@~g zPq7QpYJXnz+%a}S*Ja!5CFtakNzpaQP~)0g#?H{%9(3I{-jG`V7BQ9jZ!iFMB>>d% z$IGvj4KO8Fs16fU+CdIA$EYlyaARyWynw?{o+)U|Jy`0n`=HOCQvl>|E0S}>G zrQX(_W|{^0!aH#zO{T865tYHb8BnGT^e>d18I>F80&wHclR!1`t#|^Cf>BZeDvX5K zaR3lzkO^kqv+K|HxFq!wCju{6O$7Fl0VNpp@>d zZ@gD`82kwKVHesB%eq8G68pDi)Eg$bSj?jSJht+JsM`YBPLl~@XVF{jM!IW9zyhQp zRQ&$Jo(qn*TiEH`xSYPMuUWnyto#Uar_T|o`{XD1JYgIE>F5Py_p9~6ZV9ErLIVyh ziV`Klv!bwnDh2#%9}I3EIM^1^5H=%!awPns!YEU3a=p_VyR2g2_eeaN+=OX;qej2Y zet2i0nYd%8rj1`WLFpD~R^N;~w|68%qBBe2dL-B1ofryt8I`HR?m1h*^Z5PKDtwHd zQ`d_~TJK#Pp~wb=FhaU;7|D+9+gG9IPbR!FTKl}Jgc@C1icHO8rPrmlH#zV1bXKVd z^%bQ)8eee_ObesRms+{i{rGfA4$+pKsw~FM(L<#~L3l80jUyt4EC`b!O^)a1qgvPF z@jnlOy&R<&;fO3IKb{o4vEHdjs@ixytexq#ARfIEQVD6r;#dZ^v4ap->lB#CtJ)+7 z)C7BNo)y(4Zg2ux%eQdKXOKr@V+jbH+-@kshcN6k_!h$YtD^1VU5S8UtXU|skg?ZHJvMI z`$tQ{X5ujjIcoTw@;(D`HRpEg*g^Tjk4|g4={}XBvou0?IC$pugEOGfgf~rChoUuj z6etRGJ*PUI;C~HAQ2A2@+rl0`1V$0tglR0EvtfI$SyO9mX#$eAaogMgeBBIbFQH%B z()v0-m4v;mnPCFSC;24t4PuT}^-CuZnv7bY?ZKxWBEVJImPGW)g8OS=ao)@2Sy{g_ zk(x46`wWJSf$PM?H}ljVZOxx?jV+jbB%|k!<)7G)(W`o2?0D?$dtwhLW2*=^RRv9< zp~E?L)*`Do8Tli)O0&x9SUU!hH=n6lay=l;7!>@izd{+lC$4tnc#C`cS_ zBh8pGA265EFsw8cL#QqDArq~KXhtc2{O^Bywqj*Xhqh0PLrP#cbX(O?Xceqgd!O~d z4H6zA^;NPYi`9fct!83|b*-9C5L|p|$%ITv-fvSs6)E+i6k3q;nvw}a9N{9?QgC5> zPcAqAguk(c>&F||{L?3Jq5P2AYDLSxY0Ai`fq|Ov#`e8OD?kQ&q7(=@)6nPR^`niy z$CsB047SQ!%)nE50#~>8$BI98c7iwH?#Bihd z{*)ja~BUs7vBiD)iG z0FK2YM-9VIK)N!Ge0k}@&xSGl-1y=D9m;(skU&+qfjs}~74Jh!5iVqVL6aNO<7s5f zf+y6InodzvRD8n%crcPwjYMq(ba8l}e!&BlHHh|MdJoMxA)fF-%rkInn7Zp`u}~p# zb!!*UW-bGRp{}zE4W-_A-O6nkbub(LiF$&b@tZG8FnE9m1?`sQ{gy3$#vnV1b_vJ? z^5HX?pY!fUTRUV=>A zEOmPz8$NLvS(E_6ncHQIY9vNwGZN7TzB<@`Az8SWU{tq&Ydrq@?sTlHX@5 zi<2f{dQRGx{raH;iH?3w0b*f%KS5tl@M&*!N&fOdetX1?ZxU(I^)sTLn0+V@8E*sb zrb8(N8qCKUWp8hDA>K;ACHPi0$QfLUFun|O&}uoh?+R?0+%1;B+7_c#6O$-DGu6YVfR~K?OZN4c)P|o& z7J@P@)sTW@?AyKhj`I)W5j^%@Pg2J;{X1AT&>2_e?7{lcIYJJAlrdkYxHj$DHJKZAm#7!_@|S@9A-{mRW1{dp&8uv82@PM>g4 z-uLxGG2!Jm*O)y4t`_6xBg$m0QQkME+7b`C^IrqYt~hu(_-Qg^?1nDUOB$a7Pwn6+ zx_gN}Fy>#YOkgD?GHF{z7kWu};@0{q#H0qw%xgjsNu#ev`WSsKelwd=VBEE*1^*S<_Hs2s@H@pv65p`x2y~UyP)GwM18qO%laOOss zzFjL+JQWPTt=|6#F&lparBfl`OrMyu{(TpxoizlYtN?mX-GJszXAXm7$9dF~ai-fe zKGVj0y`DFN%SNQe|IBdx{`Iddb3b@3su$VrPJCTx=&w^z0kxH}6u4QI;5tzNDN{d$ zHfOqH7p^%%#{W3vjVYOW$d-`bW`J(QlWz>{Z!ZHD>qm>V6n3_m__MrkQyXp{Y6Pz` zI{en2_kyxs}w6^&rK^kfQ77=8HLLj9Uwe*Naphl)%H?j9k=jIS?!D@phFr~Mw& z{`E`n{N%Nz{cdf#qHq%G$dpySy-o;Rk*H3$$95q!`hUx$esIqJ*Asu=q}4p|i@IK) zVEp}=;3@v=_58X9%4-NuT-yIg{M)Zl{sV*DtIH7ZWZue7chV_CG5lyH*vDz03w#Q) zx@@04%3m)S&chY|9tHQ?IsLl3n~x~pJ@|6}6p~@0CEO;UA>{txrQy`Rd#k0GeA773ky`;p!B+ zBckH|B2$mp()Ry%pR)VFK4rlt$$Z-h1XTKdwV01fkrB}d7vjU?T^Y&xer8B~-;QJheT>x7PX)sG))O@yc6ckL$f!4fK;An}KKtc~la zRj*Jfqz$)KD*^jVVINlhmUC|`b8GUJ{9pSs6B^!(_uNG17rU%qyVV0MX0P70N@Q2{=@iU|?X~>l|Sm`Uu!@WwtTbwJ~#j=nO#Blm`Y=J#a&kl>)EB z#B%5>Fl$Ub2)?<$eIgXK&YF>|6soWMg%a=Yfdcjh18H?jRnDJ&!71ANfk4j5-I#JF z?k?w;7tyR*#t6QWCNyc};Yc%1;-bD|4fkE6E`+uRVqhFwIJB?>W6jDU(|PaHR=23m1SjQKiEd zeK6Jd2v%k+R&&s@w`P*M`w)@>Fzog;0t{CAt>^1b4xK|E%RL5TJKiUk{)4=` z@-2JXL3gveC8soI(ZvTkrvg`)&P}L#Oa0jNU)TG;L#r7jY*=oSASXuU>zD~}h#8C2 zI=@H+(NOIkC}3~&fl?^9<5C?m-9;D~_%eNRg^uDmM4*7yN*)lwJ9nI%G6c6*#fNO# zw-3IDrmV}I@&2+>fC(@3TQalmc@^~m=sR^$H~;$ACiu^44&DF}4OXA{*xF*Vxj`;% zGY!72EgvB+?Z4j)<=YAi8D*efM0JPphkBZbpHu+M z#7+Q3^=(*yVm;kw0vlmI`Tkj5A3%Hxim zl6FOdzoP)?X{tn)_mo4SrZ(swWsQn<@cSr!oe_q|ceri4{Uaiy<4=bDDjWUjZ2jwn z2P=TPP*9WVQ5d4diHG^X#Jl+NCVW=;%0x7E6p+QIoUn<=m*i2;r0X-pK0)g8ut8`` z=5Ki27XO12ux-ghRPY{y^`}~*+QDwf&rQygtor-R3sn~U#h=gcMs)h^tmw|sAN_^} zY}^5Eq+_yYMMn==@;?yWSV8_n6f_PLnqDwu=1siwI&{HYTAn=z#V(hw5d$S)vO5oV zkWs@4$z$18w7k%AhAC;ixCk!5FU3z)WwG!`=|Q1q-fZqO_2MmU|Eu8p9|;=qyw5eb zB6|N$`!>`3(Z(GMlQuYbj|L9*2-2G)@thLQBbWgzi{=f1WBBB1Bt z0row}hP5ynAf%U3N*6w0!JG2kTN*i(mf~NK3c(BbEeQTnjePs&<_q+KjWw5omEZho zYfJ_B*aZ>mfZdlTIB<6#?-U}#S&{F)e>aqJBr|1sm;B-LK;HGNXSJXThlHEeBoOFKQX{ILpTIvUdgWa$H^jQp+PH`>0CB}c zROqOX!h!~L+96pfq;|20ySegl+5!hom3E29>hL3EX5C{=35t`5(+aE^tK9Xq)junl z9$~94Gwws1Zu8pw`lo+YPU!1DcCC*cVrnZ{`fpC$z=v#8h&axo;>Q5qVXM$n%AJEQ z^)qcVtEaHmGpvuqvngO`aYqiH-6shzJUM11BXHLfpys10YeKMj0|N!~pBiyL1M&R{t9LYl6Lj%$1+ZTW>r+nexBK zM}IG$zUG#CkPs(MdNk9*6&<=|m{FMxLo0jGObvJ_<99RAED)5>>@&;Y(=!A@;3Mef z)qui8$sKMkOPBIMcyP!{ASItLtw&HBw}EibMTagZpg9qUv2YFedzx3t(eaJ2+6H<35cf zcyHa#fO31(lnKYP4}8Dh&IK9iLQ1j`3czw=0Od-k)u#NX>*r}}FL_l`*RRKiu!SV& zYq9*JJcGDZ-Sqbw{UtC=7O)vn_kWkPxOYC!BCI{RiRYJtM@S|$KW3Vk?45deGbFZg zzK3+!ExiC(-7YO`ijhXhJ@g^EA<3m&$u#Fae}V%uH$+^k!wDe2i>rfVSiZXqq9x3J zD{u>JnBIe1pAIOBGeMl{1ie2K`bqeidO0q~r?AUS08q`{V3(O5fbAEo7%S4nsuaC9 zBM#Xrf3xE?mwvFgj6A5ISM&(9ZQUTn&>EZZTLxrnsmj<{LW0Fg=fNb>5y6a~Zp2ZYTFQLh&dl#XA8Xe`QY~If{vpQuC9}x9-yN@9PRO_R@LKSNb)fKo0vbE!7W7c*X zD}*Cf^)bJ*$G}e+32WcqNP!;jaWVxL5H4M9Qp@uMgZpjrwe`vPFw_R)OOyWCF0RI> zQv>z@#t15gY+rzQ6+?^ss*wZzdhOXn7VHc=p2ffxVq7`On{Qjro}7W_HTmIid+W#< zJ``<6dU z_9S_lb3iMbo33&qolx)|`Ct2ML!}%jMuPQE%+~k{0dGV@pAoW$N1!Ja358z1Qxy^Z zj4mU|T`q7|a^zG~R^V#k8Y9Hz+g^H#=!bCELpPSUBWP`PJS8s3xc2@elm|O}j|)IQ z)bROjDCW;1xr(5O88cW?*~0quz#Kk?%d_AGf+SC8QoEwiaHJm71i#C0841u^Kv3Kv zRcm9<>2eX$(R@0qqRKw1j5rWPOKg#UsG4{H_{6fh%_zWdW#$FJ?z|#l=!Y?*Kp2xOC9|Vs_2BG5^F&~8M(~&)T%^F2YKR``3*kcX7%4AA0 zzM$*YrAMZb3Y&!{I9WwST@ZVN{TAcv`s$8SXKOA#Q)9sA1uVPss!kCzJ3yKCu4E6D z5L8t zOTLmqDMdwS3HEekEatp*s-`#@&Sd1=NOgiG{3FNo4$!_ChK>@s)S3%)(54W)Ce7gb z9HJV--Sr2LZPj4tgdQlnQPAPLq{7&SI9HH)l$Aji*{|Y472RC)2l?+ysQze+#(yW3 z6=;+u^?9fqZXY^oGgv`j%E77KGy=);ap(#JOES(%AVk?Z7FzY#ap+$7n_atM$TD}z zXD{u+LqV-Y#4X1hA6hGfm%Rs5?RtIl?kU)L0)Bjy+lSZ_6CuoX!&Z%Tp~-KmL!I(V z%pA9jz~ykCSvfWKWG^x(P}DX9dVG>7Tp`JJP&&uv>um!`Pr;?SSSmgTRC7R~yN~oE zy1LyKC#di4jrR`s_!B%((~$CQB)7S|WCxXUyvdW#`B@Ims_uPxVOER14b&!kN`>y!bv9C`Jf|+vHgPj6AIPv))w}3!>ea3 z12!5k?*a@@j0Yh5cre3X%a}q4^nIxwNF2o#B>n`iB@fMr>j3L*!ybK(6)H03&Tmz< zQkAl~7HbTS0_Qbji%7A2&n?MYkQo!M^8`R?3O+lbByuU@xY%D{c@EhL9&f&#fBeF8 zvX&FQ;1liJWc0Q=Bmr}cZ)vz`9SqKL5m>@gI&}fWkdbCXI|s)2ku!!TphV>teDhE} z6aI4TtVL$NLPC;h@Vz1tMMZ1qpkx-0L~2*Efsd_`<~`dz+WWpXa5syolr41s55PS8 z6NrJsf}^(nM3k|vx2BWKoT&{p8P>v)brB13CNzQdyh_yQy|?=%5pJAQrv?1KHN0!5*S80yvT9*xfpEo+3gFtRBpVk<0SF zL#?HItac>k3>0T%IZdzF`u}2)X*FfCdL-;z-2G<6?`bK@8YIQ){I&1+b8t0QXnD1tRuN1b>ru089&3i! z=8$WJRULo3=TdF>o4&*a2t$XNikKo*Bm|ol$8s=^@sqNlkhbrQvP2AkgaEN_DL-0c z;MFKyE+4u42@vI6r?6$XMq)#qtPWl?Bcfg;<6F%_dIChz~Dwe+<>TI~EAfSC^&EgP*L zpAzZ~3J}Vy3fDkYtgX3%C^I1=@(2?59fCTNC%jY~w~f{nT==G-Y)+L9>-vTT*xVs& zrw;!B0s~T#-Q|=Ag2#c1@P4b4Dc>#T1jc2lyJtaL?mLJDw#LPx`xwnwuXR_o!E2?8 zlaLn|UBW6turrSTOiO(_;c-y}e|I&Q{ulxkI{dZ6r}uA?n>78w!uxb39W8oivzsOlIMxur$-z-8 zevt7D*8 zHsN$F2qhr{3d^`;P72-*od>x@BOijF&4$dG8HFgYBK-jUu437KM?Z>qMv{Rl)S6s} z%s%P}{!k>EU(wWdcNl5&P*DxVsR;|D_jP<}vh6n_w&C`oVaJM0%IvSCe7Je<^R6QS z)4s`>>izb$Z&)MzBSQAxKMG)}iKRB^XDsh<7d(Uk2W?cfEfDh5Y#NZ?7=pciJU;?i_pJr#TPsCryG& zHCSAo{P$`8kH#J>eemC~)~ZBR*3C;sgZ(`a?2&=Q+srNo_o?$pChgn+fQU%ClwzB$EXb5@;uXInCyctCQnpi zU#;+m6q{%7o8bdrKHW464l>o}hC6F9@`X}Kfdh92*v~<@NH?5-oHJE-Kna(10x(#Q zs9KTK90q>#V8@x>u3UazSrS3oMahKGURZHSde`SPgoFg%=37+ZzudKOaM|(R8NXk) zzmPjL4nmJkBYG?6*P#OASO5$5xKlX33h4QSCI?iSQk75RMmp_kxakBrOrLHSYODh6 zdH@PLJ&`$OufFKE;D@(9#B_JS8CUsfqOj$qS5^$Aiy!M?&7a-&x?``>Y~@hCDSi9qYb zHY&HdOn^Xvy7LVg!IP}sr-;}-7Np! znJ^1600&$IM}?m>(Ze@)s!^yk+tc-h8UI?IMq-Kox2y!ZLr`hB_7nMGXDM@#_@7I_ z;I0)Rr{R2qi+=o<>-aks0G@**V{#HK#GkbpX5s(zTYo^w<$JA5TwuUszDqtL7fOJW zyp`G@IqZlI)=mduV+Qn>&D!Ps<8VPpmtsE@o<9z5e&P8P^J?@f7X|oPj{t2-07_2# zKcs>mfDwPBv7+#w+W+M4ek)}Pu-@CL{lzBb_vFy+(tDEnp(7|Vu|xH78cwoOe|-pP z1;QrnzESAXlle+JtgXyRwisnwoFWm0q{JfER~iz^a?>Winq^<3;15vX9`!4aPX#1F z0J>!?ygz6HjLLqPR>a%-Z$OQpgxY-fcPsKzz9$a^v-`zEHZV9$TQ}t+lsiW#ug>X# zD}SO$t%Tn)Un)fiSMe&E5)$fP$vy!U;QOF(-$X4i5183o=b6RPgzLKx zAAtkDw(jUM)Ke~ZXaW5c6jeaRFGZa~&;>2w=i}qcddbZm&SLhbhvN)fc`wT(&GL3x zE?Gs#bb?nIC`hNL_`h5RYo)sG_s#N0jFYZ!qkVLmg!+w}Py>zW?h#15UOatKRqU%i z)Dhlz_d2rs&~~@ZYm!PH-0NpJ5BkcjOih;`R&E6+ka?3KjaXEfATkEd_q9pbBm1Wz zitunOlsghqRVKnf2>WUr`IYxRHf9W+sljO*vTkg{$Sn z_6BPq=WJ}I{cJ}=S@t}=K6z7d7B$%MQW0MNcne?u6nMqwIGP`LT*7|u=4<0T_#OZy z)Ze-+#A~uapUy}kXsqOQm>%;*&3YBWZ!af!`0OPcsVPk zNW})m4(eOGG6Sp1b6ca%w|xurAqwT228t6D8L}}I{2dMe+G;zfiaDq4Oj@x_axYKl zI8~HkA=%-_%6*f*I^TM&TEnZx@e1F#$M|D2T*HIVXY+LRAssPkPeNLc{^NZlz&$IU zVdFi7yf10WPn31_G!FU=*@pBeWH-#YQBQ}!+U9nNNV9om*MagXa5Fq(j>|z9V@~t4YoIk1hW>;X%x>H_Ssntp zYkk;d>&G8%2$nTIPYTuHw4lKzBL9EwaXDqip!!jmT9VKfuP`?s!)x#c&9L_Kj=!ff z*Y`)s8R(53RofDjeORzgFzwFVX8>Yb4m-I)5}Z@O4eAbUrd%WALl5>~iM+qm!Hq(` z-one~&9ko~EmYYI6ptQk+)tu+>ikVRJ*_wV$^Ju;k!k{~th;X2CE6$0$8(4r(P>|9 z7EpY2GMB_h1oR^g+-A*=NfrwCLmg)=++z=QA`bdb;U-YmI~x8UZw~QUwk|dC;N~}t za712@OL8MRAX2NP=?#oXH?u&k7U!3s0&{H%0XJCbx*S$N?M~nsC&YySq46y`kckx|RHb8%d7Bi z+XYz|==}+BNI8egX^GReu8w{jLa(2Q@@kMAi{Qv@=X+Y6ku()Y%RLI=s=oUncz0h) zy7#Tr`??;=rI-m~x-7l(Gug>V^Fob9)p>_JHJF%`zWpto_Dbhku z&9Ni0J4qJb5jEqf=k;bSJC85_UcpZ*+vq_ew|OZNt{{ki!2V3lM%}K;Gw5IcMqr6> zb%)*k+XY$PR0ijv>3@s9=>cHY?Z5*Yu&N|7(yE*$*+P5X2!dg|-LS)#M(Ryn?2|!g!%Und-PZ@Mb3CxDNl6 zC7`9IIzzb2l6oqO)vZGm3%Fu~rQHTG19@{u@u&omwNn z{^AM^I-+~|-pm=&z$J@6MM0Dz@%%{SwGuKDH4tqZ*Sar^NmqNw>PRX+^!8wKs(e8I z9wqK`g@Vr2T4FsG=PE=8-sDrnObTzyiXb_q&~cAdPfb=QYsXOJ(;Y*y9}fl?;J@tT za+{^G`!ZSHK@}~HZsloxa-F4)Ypr}_W1mazTYr8j^$iQix;jSHzgp?o@_f~x|N;HCSa9ttv2LozI1!avZ!89-)bXzWDl zyQjb_40DW@q|xK!Qrw19Z_0n%=hV{$cM>h><;O^J_YTHZJVO+8WQ459Pj-21rrt>(q7^?nZd^bWi%ZE)xiHHilb46X_CE9Fb3QJ%8i;g*NzOP11YHG@#=I| zO2>G_-*1j6IVZaJbc1`*e#^ov8pRSPU-DEpFJ;~U@A@d~<`c%WrqG_wxbwPft4J75 zR)di+PbKD+ADu2YQJY==Ceqix{N9eY=l^`8yPCV*M~FmE9BwNElewa`rw(i4dp+a! z3e7LmCLe1!LU7k#dh$6h+9IH6)nHUfEz7g(2A&6gpnTOp!YI;iI^DA5O7Rz#%4>Kl z5p4fkrRxVouSUQf)PoKikh2~g9fD@woDy>o1P5w=cVkeyax2X+j+e}n(K8fytwm| zHI#E!y7h&PscLMqrtG-rSB2I=)3n!6%$xsB9<-DGT*E%^zu9Zhjs)bhi-0!!6~G(oD&o?48B$ z$hy-v_)>sWV7iCsxy9(FTo$czw}E8X1m*A(uGf~l2E9H6RyFy)oxAUkKZbRfwP;>U z8KhJ!uI^aNwuoF658Ksq96Mp33GBv^_330W7E@Gq|8$MR6ewnxT(qdE3gd;@OTGI` z-ZBJ#c`$POp+9~c3kIZ5x&Xjy+pEx+u9a=*38^;Ma%Ef76PRY!{|>q6fR&FqX)tP$ z^T|B}tKtXDvwV=@^?;Pgc;!^4@4h{{HWJoiEcM{*nY`r&jP#OA_iKSoHMyNkrJ@Eb zUMvyRIC!;+9;Yib==}$Zx!XUgHeSbE@C^w+U|!fwZKWytblAf_rsckpFb7kVZqU}h zji&bI+C8U~?gS0>{#-}R9A9Y$NQww*VNxwZ0LA-y#{#_KrFnz(GcuLp?#OSvcBriM zGMMD`I|!%@Fm9~JMd>^Q5}O`0PCk%b?n)iPyva|kKQTc_ zDNa6GHbgzR#gU0%AxB^NRJJ3E)e+aCE=WObPW;S2zTVF44o(gyngkT7(->}MN|5Ls zDRJT(7j9lml&vXA;cqj{HBYX#YfqdXOn2y1(Pg?#!Txslq0C*F>!g8WUGXNkhM-#B zq|G;LVil9Wr~k_Y@j(BV+=BPJVg4bz%jx@B@<55KPOGaW)$1tv0?BD)#B&gQ{|X~e z$lTvzDZzLM;3(3oK;34K^~Rj05{fTQhORMHwv8NUCVWYS9mWiSuU5iA_7>4HfL&Qg zSc|&949t~xDH6=gST22j0l=W zR9#T3MMw?5>S8x}hip{f^3UctdrIOs?VD$>n=>@>rAccq|@CAxyMZS9y&BR`jP(y@bqr}18oI}*iv)9>FtnW}ci z!LBg3zpvSdLbT&ePLG%#SENH9b8*!xH3pKHor+EKNsAZu>&3K&cSA6zdgrFHZ-OdW zwt-Ug2p>mG>N2Jzo4)_kqVRm)%vtl`c_Ak6Fl|v5%O-k;|3%*yJN28N&9_mdj@nE1*pqL(RV}^3^4^jq ze%skqhfRn2?{73s+B*^*);dady(6zjT5o%T@|zk1Xp4a3Tw7wrJ$1g@Qc#LWU;uk6 z=Afu3gtOB(BwsJ5R)J4Zit*iY8?lP0NOQrQ>e8@$Bw@$uM8%V8&U6>)eX=@A%^t$~ zSt`oCc6^+-r5<+zSQJIo*qo(z5-HvlE?ITG(3wS-AVK=Z3wE`HbkpDv@StdPu<|fQ zV(rQ?7KGAWW*E34+q@OfXk^S1l;P)!hDsw0k*?$29+)t00?g7LfVmlRK}>y?_shkl zU~(eUZB|FQro?))NQ>qJGx>Efs!D&)?;$uM9xQ=7f6O_&7g?BdYT4{eM zRp*jeUq|HARbBf`lacBt$@3#oPv@&22ee)Fao;}l#4g8vY=wN*LNB1CFxUQG&}saY zoQ2`G;^uj+xXOvUvo)t^F?AH|<5|nX0h5CzA14==AB+;q>CAY@otxLPXV2R?6h89D z=aY|>v@NF3YLu5u^fx%Jt$sYX{bztL@^FR!X4DPV0Gukv&i?4;xt);5lkA3mK!8~* zMd9!ro)Th|y2)gIUoU_A*=L9=hj@f&#xR!u67Wn5H)h}QqtQud*6S1!;$m+Maml=N z1LnIzM;jpX_l~$veS|2>DO4_%(*?kGCdU^v0L}@n*2u&0t6a)+h(HhMd|hkhh5L#E z$NI>dL=H1Ck~rd3r1=zAu09@#49=2u4W}^8^3Gy$Tp?rAvxr>qXm0JD>k>bQA1Mig za<8j+T|p&gpygF9gkux)F)6Ac+ zs(nA}fk#JV!DLBmS!Tl(y_DV_)z{*$74;)a!7$AJ!F+@l~j zPv*o?=`vBlR{g#&VgX}ovpq_*n0)5I+4T-j!s>(2?$dhT`oTWcgYXxYDOQDjM_R&QoUnfq39YO)1QBfFPBpm=P zbRkgM`yit^qIOgUeCm+6x}b6^9UsFg^6Ns>)K{xjbk?1a=+f*rVCtfBuwk1Q=()W6 zZb|&{wk+T!@$NO2C*g+bHSm6v25D{U2L&&eootyFoS}ZgkB;US=Fw733agx$*pe(8 zBhyBC8dBFt9)kM`H|tSOSr~~HopwJ~IL2J{rJ7>#$BT5B>tthJU!6l)sv0rbWB7F3 z&gHrG9w*TwwW64v&I}yTpS6z{ZW%hS)%VCZqdx2I4OHS>yb^h2EV70 zt@Y@x$b3bml9&8(A;%|8K6P=s$K26Q{d}5{9^;mEtzmbeq?^@PtW66ve;Zu#J1psz zo^U;NizIdU^3pS)agDHgNO3xed-DuYG2_3>=AbxLWre%0T8X_*VvEmPGG$cdXc;2X z6e6_b4_4=lv>B&}q`9V$lZHlXDAy%a;ZRv(B+@s5p;SFrDcZD@KM^7N@T)P22abc*Zhjcg&HPlX?jbLfqjI^tTxaO6MaT z=qI!vaA-Yp6Bt`*59Tu9wwBj%-ZyrFZXi?mz3{S+X_7 zJ2L%+CIS<7luq1Dmi0>1dT9rT&X^7(OX@&B9@(5YDc=zZ;6c-3So&X%f2CfQZr06D zpArDSyqCs9>Hl1Iw64(|tq8lxN17&!Zy=-Ix9{OMEI{$ie5ID4-t6Ln0@Q*8F@Ll6hE z*~I(Z&&nD+C+Smh!eqdhj=zO^Cy8XtQY)o2z1`cnWX_l>^$>g`C4x#Om18O$^r|$J z>7PxcCb&eO3eWOSvRx|P-&T3U{bcI}{W};pv1<)m4b~PF8a)SlLPPoTkIaZ^(Jl54 ziG`7>ac>(cEo=*QD|*saKbE+vH|?KnrX4qWyajkLdK-uld_GgVM4|`8NmNKB!IO~ zxr_Vh8I@g0%mXVJ5f8>=}!%;VuWlokb%vi_y&A7q? z(R)}g*#_RgE0KsE={31NbI?<~zW8E@RqO1BnYHYXsWBQl#%b3#?Lxq>+OzNlYQKZOV?+13W{!@G>_`paiYj9gUP$%Fo`JFLZ5Sc zcShXG6l^0&o0sY`Ab5NO zfnXZ4#itpk{M90Z3~18A5#u|DtX{NsosOyV6(<(ff4*806$YlJ68M-DJ8Zz1zVgd- z9aDL=38KVgtQ963lbOc3UiJ6Wr3*&7~c1GxK+Dp4FDd z-O=COp5p|Iq!ckm-%#-35nVB2t<{}h%M#&@@Y;ATS<+X(dIG5}DD;%_+8m?Uj|hnztLo@AUo^huJ}2-)nCxY{M|)P!i^(56QYp?`|M z!#0;lHvHk3<xE zT9@9y@4@XQ)>HvJOSLEY?ukd5%Rq%H}|c*(e=V-@5V)Hzr;)^r}~HR_z2$~W3X+p zaMiQC-70=oy{Aou2X%EF4+l+k74&a~N;x>GeOvd1mR>`mHP_CY|hI*`>!lJMy(Hvxl8joRNvf zs)WRStB48Zcm3j4v2xjy3>~~RnK!!=oQlUYn3SzCB|9C&i>BK>)uxvU&y9|i%&d)- zgtlqcMS$9<*!#*C$2dwi0aH1BIam9MWopqw1Jen#W(LWgFQ^(vlOeiA%&bxORUOCV zA12KDH#G=kE$WCxwSMC{ewQo-A^V%t|A_qZp2{#RSrO~_vDE~^x=j78KUk_#KBZAB zOH(>hsSO67lO0%lJCZu$I3Sw6s%M!s;-)4@%6V{bk}u6oecm1)bH!uv#YQlE*T zV+W6(W1Lyfx?b_HOYv3s3(s;BqYEIE-9pw%M3BuS7fA3>!Vi6%Se=GU8_T<+T$9#82^49X+^bAMOp}I>G2VI+O zV*#(+S{|~l-+Xj>lh71I7Wk)+`L0Iiy#_C7gafr&wBva~b}36A!zztmxz#S=h3}Zf zH09}Nf9#u|_-k~MN{!1hK9($42jAeU=~RdSN`r{CdmIvo_NCGhgI9m{4)*Q zVgp=W?L-Mna(Lrx2_!3`cr~dD)EPw$hx@*0D08QI;}yw^mhQR7wK=>aihKtk@l9Cv zZv)ZOPQiJf5Zc*_@oC*g%+*-fdHVCtcdaoOXs_W_N#+m;%g;Tyx+B5gUe9eIYuri2 zbz}QfP9KdQz%HDWI2NEbk)v)hD+-9H>Ve2PN}{Wej_Zz37^#KNsxLo9if`jXuIOXy zjh?%>_2VL9S2-e+(n1$wo-ge)&@ov zh*v`^@vK!u8q^QY;nfM6UHa&}#}++APd!f^IB>C*%O?AzEsCUo_;b2rbw z%Df}ip)qStG+1o&p*{FM#d~IV-w_mAuQYp7E2g&_d$Rr0u4t(e6>oMJdN(bkd~=2l z4d!kFM4s|haUBE5S_tD9J#(YH2g!wjqHt_|+$ae#KRz`Cqa$#CwAMr<$AHW1Pw>-w z9Py86;qcRf&Bce)Dvx8jFcU{d%FWbc@_OAl+#9M>#$uf<9=OLc9@DzM7)i~p_TpXa zAI8ESQ|BE5CS9rO2=0%`nQltTo|R&Rj~%A-Il5zQRzShhcH<6!1?A;0DO`=9>(i_^ zURt6MeOC0P#&xTZW;b?WHD@N>^kn+*dO-U>#-tNU-dleUs}xMI*ncT-v#Oc=oOqj- zdkcpadktx_?G+B=8wlOie5CP(YYR~)gHoh-$2qy|Kit`Pc~p;qDkih{x732Iqi* z7vsHqGPxji3p`g*=GA&kU2Ud}Gq+7Eo5v$NyY}@UF_otA&YpO`{Xk=P$e*f#d}+n@ zOV}EpO~qxO)+F^F=P#`!EsQ@eAM6etDndP(dL|G_G~CUJd%vGlL8nx^{SF$H{*EosJG(+V@~WjYhGq7+0HA08#;8#7*5YxVf;nWy zQ}65rDDJOy1toz6b}!xyaW5K*6Vd;CWoNwDLwH=WOW8!TQ0LqIzE+U;9R*(~;Bpq={DiJnNFn`BA@KGG^+rw+=hnoJ|#FmyT$a%ZWHw zw+Z;GVMuOYF2Lt|d|46=D5?Ftme{pD5vG8=$BmVGIb5BQP;BD+xh?>ct*^#2czSXO z!!Oz5k~eoQ++%r&j`fOV(1}ycL~Cj1K=DTtU`iU{96mU(FFFp^_ohF#C|@mq%s*&K zmZ(LzlVX3VIF6+E-FR8KskSk8fxcKr_~uL5Pr~UdvGTU7_kkXFo!r5;{0oi#-?krZ zX1&&|45EWL;7)lVRblSaO+&!`Sf|$jamiXqDxo*;GqH;xu=y=2#fpx9 z^6#YNS~TsDksxdJ@fHp)q0j5x(HmkkSe~6I_A-lxWPrC~`js{(`OF8^*P4mCJ9MsN zZt99KCuo)wg2r1a`FIvltm#>CL&BH<3Oz!YmGuMMvP%z=O?6I$rjI-O+vSB_WK`ac z(Mw}C&7eI@+;BF+=aP2*f+olzSGH}o9JC~)I7-H^iZlpSCrAp_w#8I)I`atGm5`RW z_-3aio#I&XEXm^6QH#O2Z8du0_$enxJ=Fd2;oAL)_1NCo+8IwR*;w;5c@}yh0nG3o zV(aLYP9=lD+vNe*dLkfZnvtflm}c;0Ca8mhUC9>r-+0pM_YtYeL7-Wkru&Cb70pTc zB>b=?HBm~5p)F&ZkNK4HME_D6(~{@zBWzBm7z6NOxW|s2WiK^e@We6-Bmw;s--7j6 z3dw%v)DqB`C+g|=zAfJ@DHpg}L%YUQeHzwGJ$qLT2Idobz+BL$d7HYhX@&!L!jKfn zuXz|k5eUiHU0AD9iFEsh1w7(d#3Y#-hB1`YO^Y-` z??81wr)>n7y`6~;y9@Q4h>zUHM_s;b-G9Q!n$&Jv(NfrIpKwQcQnyqJylGV)%rchq zL-B!34StcU@r&p8`Izrvcj_xq7c$N0oPHqkYY+iXK}I|s#Vu$l?`kvr<;70OF%DZF z%1*bJy_kNkVp`;_)5u}x>ArW)<4vW#8iOumts&lYLYfmhVtU^>I=VJ>;>a^PlRc%j z_pTnfcrij#ZtX6I8n4h_7=Et7D>Gq}#e*l6=sg>ceu)P76;h= z2W(03hs-vp3z}Y= zT}}#0)+K$5{WL>w{cO)q%aenZ*}NqPj}h^rBP!v0e9I+8RUb8t&B4T6mtQ4nMdSOT zTC%}*LbqbNNH4A8*vtpvCymvO9y{(CRFv(UpH+Fhf96qNefo=&yU?{6jjw5+<`pPj zGOnsRlvU)DAbB`49un~f6}K&q&w9<&T(dDsmqRw?Cavk9EeBcs2!HH7A8K#|i`laM zCtGYrOIXCj>ATGgfVbV)@@;B7o}Mf?E!UjBvFS8vBr~D@qE*TVAfu6UiGZfi=2-ad z!+Vat|HCJT%)wTQj`M{?pL?(8h>O20c(+hV5NO5T-D&*mHf)(gxT3hq;ZOau3<>0w zZzM(jKg!-ZE~@SOAC|cwWl(}5CEXzi(qoWHqev^Llyo;^(=9D1gM@^%G$M%73?R}9 zNQZR8v(F59Klk3>_xXnx40F!jd&PUjUVH6CGN_qBdvYb%vsUWKBi2M1Nh4U_$#0{^OLLJMyq>4 zdkvL(J&RgqE)+ zkillQ?)Vt4*4YKt8CG!DKH2x*$cYGV2GRG1I%d!C=!mvcD>%=@p87m%u4f2%A6)Wnbx< zO#a^&=n&AJQ<6x~N9tR>E-4;2ftk=kyi&wP8Fy!1+JmA@>U>e!*)G^Li{r z23J)Jf0wiSwIHR0cZDT!a_wjP;hcb_bqpXQ5%2fcncOxf=}%2QuCs+mhA4(=c32Wn z?I7}nm%T$Z?MO~+ABLa(5jAX=%E?xouwQ=B&c;E&uzf;0$|QYNS)T#5yG8J^!YpFa zaXYO#LG@r#c>B$)fK~%3`^ngb4*ehjq$@n84NCX03Ulko6>*o>Q3?5Y>Ux!BEFAg_ zreHxyB*tAL3mHSbJ7|DnXZU&i`74+EScTy;{7Fn$Z@Gh`f}et06}3d-*ODaCHcHtZ zB$%f5TwTL!FFv-SpAmm$V(&((aiWAXdy|3}0KDj8Mq+$1#*nW3_KtcW3sn7 zi~?NVDO66wwbGZ9e$nu}!U26R*sxY|JIZ)&=<<}0MrcLOV=@|Fd477(eyzY2hAhT9 zX+q7R0Rcytpc5HSMQYQHu(}V9Cn(>;DqNuSY{>kB|Lc!ETT71p3qOYvfNs*aiBc4> zOyiUj&aQa!)ydw`%>nR7gC)f4x50nkP`Kh>@^a2U9ae%_-0=_ZyB@K@*k9_4w9rE6gAQ zZ_k0V&FmQX-BzuC*G>(x8mueE1|97V6TnMLqWB3SbLI0ei$CNQ3!Pt_6JH%-q`L1h z3>35a1F=oAP6D1EG`=cRg5<=<&@6-njiWns_M(N~ezw0aL=kF{dFg??mdB+LGkMq0 zDK)=>^(Lb?CQbGR&oVbMYmB3{_3}Kr$s6k-LH2^LVb^teS5GU?I2d#3zZV8(MK8CU00G)a+BUC>!`8_jLWbL&(npPsQF z%PiTDnM^jn=fO2u+!^+K;l}w*n8;2mmmRym=?6@OJ6r^NjP-TEf-Hxr!rj~MeH%&@ zN=@BWO29i1!dh7|7YZur7c<^>$02|`1Ogy83#C9ZUJFk^ojcU8dKtdcgU9{uu+|F| zx!qR8u#_BoTpHW142+L-y`pxX-|LmX96Z4W zMul-65tQI2l4kGQ9b4iI#n440XlY^1suY{<4qx%2%CN#(#t6`^Au9&G*^m~BmrhF2 zX?u}@@6(Yfk2?7mz>OM}pE{Qjj_I`^H?FS8l({04wlt~by7QE7v>D^VF0uVAcm5dW z|Fw8W+jb`c6_fK`$&2!Th4&Y1W=D;^^y&P}r`BDjVMYe?en|>MTJha7DwPishGQ@s z!W8~ek(&3mZl6SJr#@`1XK;14r)r$OAu{dWR`V`(HbWyyR9A47*;f&Cz01~$SzjDC z>=4KYcRQ!uBgl%6ypcZjtyt17{!i_joHkoHdifQ{QXzLgf$7Q{VY?suI~+jx#&LRy zJy&vbGc^q?Q(Mzf1F+r}94Kv*bGRZc$ zH_?Ug8m;PS2{Fc2N*~=e%{8Lljpd`(*1;_@k>q-6jZu!#)X4VsNwMPi?aRmw=4gA!KnG3u7+#M*MaFzL#Z{p_{w6EvSQu9;}bgB3$CLPihQ0}{!EwZ%oZPOavt zn83X+qa%td3Z4UR+d2lyOlHXXXmmY(jW%;x_|OOy z>tq&COkQp+mWfE54v2nQFL&azEiqT&ouzl^tQ5eI+xB!YB7*lm=cjFfjtX@=;_~ zxRTI~y(sOVaH31w-01mwdx@7w>H8s9Ra~5Btnlk<$$A@t??1bv3~#HI8!KP46F%Yc zr}6Voe8-vNYGgSxuWo&Hxlrpp;x8(l*QgR|RTSN%pT}MN%Kqe})@>rWWIgelY65{J zQ`((diOoU-f~Yj@%B{rLJ@@j9&6K8zWaH6(JLT{G#B=A-twMK2KjA5i>|c8Ipc&Sl z3t)E1ZP`Q=#zdx^yl9-vWB-Bc*DqFJVn_(a`fs{OradVW=`=&xR9#cIEr`z4J{!~; z=Byx8`ZRpD-AuoY?068FP242E;))vndzn@=kEKXj_(iqU>jk4evA3Fo>?hj70J~!h zc?6ih&>w*xN84hM(E)N|-MKGy3^5FYdY;!!Ief3HgTr7r_(4U=SoT&*B)lk~p=vBH zkEICCFne+l+D+tVjYoI3n~aDDfy$-$)kinzr9jkyfgRvl&#YvFAS`U=ZkWymtW&^V z{cR2ySMp$5z-4(XEuVU$3#c*rJdv^u7`WM<$Uleo{pZc{qMYoL2z8Kl9&0xnptDL- zd;BvPG?`28e)44>kEAFfHGldvy_77cP-QBz3wLVla-W&M=>2 zngU-0?%KDX7;6IB)#LloN=g){5Ri`plKz-}1k9->tXRpeqCu;EJBovsLsW<@oWv^r z1&0U&m3o?9k|_88;W7_2x%LFC5r99^*<%W$4)9xqP=^v?B9Wzud-YiZ#c7jDzllT9n^n4>XR@PhKL<>fQX8IysE8gpFWvs&{_)oPct; zeQZhKn`rb0rC`8JEeaw@u^99yNQhal!&g+G`)nO9STXuOg?jR%xtrjFlk&HIqyMAh zXs!?mR9)I@C()h??`37Mue*{<%vuazF+5ez-sU*)-~s)f+{!b+cMS~sSiR=EZ2R)(er6N7?2iZ29L)g&j@L<7TP<7eI3bNF6NiYFIwCi^G$3JQY z@en&k zvl2gt)9PMeRmkDV9b{W3XAwDb}t-`jfKPZm}vZx*Bd5TZxlZ3<49J|pq3Ks zV0Jo!m#QR4hBDls{;NC})@+UoSrp}pmWc9;+K$eRHukzU6^+KO6!FId#X6qCf??wk z(ZRir?fycy1EXgIe7jPK;1%?eN z1h8QKoOH{Zg!j4b&{{+UZ%R55FZ;l|O{yhnmI%tO;P0(lgjiYO$c+0)Wj$x(1710T zt8PTU6w8x79wr4 ze)pMFLLrXoV_Dm`vLZ?{md)Q*u_>?6ahM>92gw-_S|op5^|-|SXBumCkIW5_5_-$O z4E8703mL3L4JHG}!A3chg)%Gn*xi9^W8KvyVZCw}O!|g#5nG5=}e$f|wDQ2V9h#U?XN z$vvB1+qp~Dzke7SPMEK>|G2rp7nkYrWu7>n_3u&;fZa~~w`H$U5M4m?F@*(7j$vsr zuOS@646vOV9aH~ewv>WmdU>G%J2-;ILQQQ#{hc378A~V*E|q}Mx(oLOc(e{d(DUL3}X#Jab{Doz$)xq^M!F zG5?QFDDx15!`4hF6`qBRBMC-ZN zFV2KW(Ufh!nsFE^C%`{(UNHTt_~a`vZ+o`8sKf67_eQGUZRcnJI zMSx;(+#UI+(V&IP-36_P$CIp(wxR)KtegAhBIrC8f{3og7?6;1+S@k*L+lh$MD32X zc5Lx4?ztZ)Lnkd}m~}qG-bwG0aF448SPhTB$#EbL7C=3a7TQ(x4l-VHv`CCy2b^bt zr{NQgmBYI;9l=$S!P}~h`fu$ATH~*sZ@6znDLK4#Um4=3og*bjn=x5VdKaAPv*Hf7{72(!=somm~j&)P#{7`KWWlpxQ zaf|TEjDHh`gU>193BN+6DSw_L48{z{npjy&>von$V)ahxT*JbHseC7m0a3fzNY6Pb zBcVG6zkeG(N%Wv})$#P;T-dtAu)c`D1AztLvVCc-m3obh_J=tM3)GI#TY&{-nJ=aM}9Z11S_PJG8E z_T%#Rxs9a)-!?%S;mIGD^xXc;b@eTc_pqQW7;N1gw!0qGDE7xMJ3PF#bLB7o`ZFY8 zH#K)(xORWpw=v4JFa-sLY37N6pWVT%dtW0XRz{oWd{$u7$m4y4cP+#v7OR1z&WBCo zBc#pWIA`VWb!QJxN9e`5*T*!M5qU~(w|8~w3a|Z%Te%26_z8>peTJ=_?cdNk&G$UR zUM}F!clNChT@WE;Bb@vmW9C(Z^SWrOd@!7;-FQB=(o9KwV_YsqU*BcbTaZb&#O>F% z%uws3rHNx(d(Vuw0Y2R{Wg+Y8pEyTtovm5B*UuRJHfBqw$sx%BhpT$GPW2sVW`Y}R*JU^moqg`? z+Kwxy*S>pB$?*^=euas zR%2O0ob<(e`)WFIwUY5a?D6hU>e3059K;;_r^}eJa51z}&V?z6nu9B*EzADcv=#@j z&yzPbZ)NH%<)P*}?&%57K)V;9mFbC#&lv#-7 zL1!`C2ho`I!)FQ7OSc~KbsCK}$5wL$O2ldQtNhxqEI+fdIXBV3tRQMhyx@K>xS{wI zp;41xV845XZm7Y6MW-?r^kTakl?n%h{BIqcXQwk^%u>3o13l0Z1Ot7YO}>77Pi(h? z_{(7WFNa2|i8W8jSZMr=h~&T*@qsW(4PfXJT)nXeqYf*6Y#4 z6t$ARkQ-m^_7zS@|41x2^j4Dw@;RJ+Hm;eNetK-B7GbAi4eXbHhVh2Y!5} zT>fXdIK|PN0vpmSM`qBH?WJ|#@n2%*1((B9#srT;?m8~)>OO%|qJdRQQ#p^ev)u$A znL)A36AjG^w%t79o7W#Y_f);eEaH2;I$F_9PLbIuY~1`(!u@T@&c;Vslq_IRaZl0) zdP%#&)nbT6z3=sdUlLwpM@L7~Q*(`qfu?1zBAW1tz7i>4KV%Q5fh(QbXJq&gdqSbo z4Jbwn5zN*Y`j$38qZwsR=a#-1^M|kDbbFk?u_n{P zfCV$hd38tFRl~1fS|@^=x7JJ8@LPw5h2!TwiY;M@H7A#n`H{y;66=?_scqHW7H2jZ zrE>L0!R4oS+ykc{>N~Cs53NWxl;#gm*#gnztf&lQQ>zu*zHxhc3lXi1QQ7T|97BzHJ0R3wh{X&JE=x<-zMLjuEYR=k zPWpgO_+%-S*ma%KQoX%4?r?`CW{`Zx2GsEbU`@ohZ`ytpyeGoZ*2#Y=z> zY7E)-J(^un+SAzGKg&3Ht;SmV^0KdeK?DJV|^>R(bexrPx%&6Xwl_nmOP8Uxx={dz1()Oo>Sv+@&xgml!*7x zfRLR0J>ZiEz7)qPkdYXcf>UARxv{sj-%k!!VyfGj1i0zb;V@8aTx%(@a<#P>B>%j5 zPHATwyE}47u;sMLR%|;^j%TJGz{%fobP#484qP#Y(uEmFp^?W-W`LIp23r*f((`^x z&qYa*PoR3q_2(wDCoHt>$6u?SuK?a{l)}bxZksH#-!5(>e8+D1e`Ywq!0d#JS5dOz z3<6ZUBuKal+rmBd(*AUJ?r{I;`qXCjw>ey?0grIGL8q;#o%>Bmvs4b0-=3Q!a+tRn6l+^qAXFP#eU<#hQ4#RdMj(y_y^1G&4VB<6Or?DF>F|P)LI?B|JeT zlGlBWG7ybq0;R`}VH6_@4A}IveNU1_NXOb(${l}5%>BM}+oaP50B%5VL_7(hr};Mr z`v;t?c9&MQImq>Tw&$Onspv2G3Y@?rt2@YM{qsa}0A<-kQ%_+)SP7@taRD(FS-_oGq3PmFO8!!g&+07S~j3`#?HA& z;5s=|5YZy^$egxAS6r+Yk5d&Z#mkEN0^o2QJRg&OUSHR>W2~)E>DGnv?I-z$^=+PK zTz70$r>wWu<}#=~)H~)*9)#=(_i(WP$$mQL}5%ICA z3uEKESM@X?I%PK!+@rYtg_be!zPX^AA&2c27;!AvD+W#ZgU`|G4DZ5L&G`xvBL*N{ z!RPenrz}3>gGn8k!;_?IA;ns8xCYQgsAV_rJt6f|vTfjfL~1k(S`5Y>d%ZjI$UaIV z`blNK(R0Z}2%uCy13S6RJFoWKlhQ~R(X9~*3q zNVaG39hYmHS$DY*t%~)3#<_IZ@5&-L8+*VB zG*ymdb}7{XwVy~h-6KuOgMSae)$Ccl?LxC|$?>x|7#oWJqzN?Aj}V+!yApq&x|m=7 zW*^`Q0zlZ3giWNtiKYfK?Y_P+ZM25C1zJ`Cm9Eod_;F(YG7380LXbV~E-?j{FoIT8GR{REvW zKX8Qq{Z-J^HNo3*pJGrX=o-SvqJrmz3cfd-0$YFyV+i*r_V=I_l9AnwI56#EE?_Tv z>@0SLG>z-7*q%b57sc}URMB28;K+mj_vcmF-8x%CgiWbY zX(8diF(tJRD#|%t&yPsVzKZVA6WkjP-uLT;p(*ynjmk&(L@79kckkY5MLriMGCqq) zM+UR0y%V^(3EN#Xr3$>^f8XdMvpQ1kUBhA20kQ~o-?O%lay=i4;$OX%=4!0bf#yv| z0n;Sxwk3|F+h1cHLjuJSvh0#VSqf4@b!&lZM^g3R@YE?q5Yc%hVs;LPufPXoXQ&!GlSh=E!B4T|^PPDH0H1f%V@mU&@Iec8_V z#S;4-44>qr1FatM7So{A=5L|PXf~vjf^AJk>Tud`c-WLm31;ybUlxe@_Be!52xBoU z;ytrXsd4vb&;|eVrC-rt_X+#y4YX8)E=FKqP@E?h!z*e6vYua+Rm-QkiZ#YFpHqBX zLoQrA34+6mAT_ohk3tKlCWySaKOCsWsM$Vgar@S_t(_;}LZK8VoEu(;v$a7`+FaPP zvI>ohDC~Ju+7E*zpddm#7kr+;8wtrasJ?GjD;(^w!T-X;+f999#*M>AR*Wji7)?`V zZ$A;88RMP8y7t21PF4s!AKUNRfr`SRt&$9J9b&qm(^x7X0F$Ph76=IttlT_p?n-m7l_xv^T!ZAcsIc%E8x;FwachAaIYfAMMZIY5%a-n#bi;duR)4E zc8LfyR_#BO@B(rT@Wz%+@y;^vz{>rZc$8GB@;O^tN2eZBuVnpBjX94fCP7nkw5QdG zcj5Rmj7Go-v6BB+HzruB!dOF2d9LVp^Pe}42u3C39p3kX^WdpVnjk423orsirjQ*d zM%lC%baG9`lF}*bAp*7si+@~QnQsEydGkDfK!W7qabb{JXt03>HI~J?vL_XLs)Pwk zPBQ#?$Rg_h;{ozsKf#{zf(Nc+@pYB~;Vbrh3Zqkwftj?r$dk#{f|#A&{jVG5P3`G; z0=vmkp;22fiENr+3IdPlWmtL#@ilp>@f1C3EEqKqofG!U zU6UU>(>f^+mp1{-C>?Cy}m49X5T&pg;ACN!iSYr=sq2Y(Ki z$1{myo&N;xur|9@mw9^HLKQ-vaq+H3?LKKQIEmGP(5GO5HDCHebQ5t5;YRo}0B(sF zmg6PP<)P&J)78-93nMrK89&ovYcZ17)+FI@?i&&lE`iaFS+PDH#)SkH7#lGolP{85 zDu+Z$YdLO0ZR6DgP-7tf^Rr)dQh{M(CtTD&&%f_TR4@T$_r=ASY}1*rYyl@K+k7}QU=jnm+J#P;K<>(EXA3?h+wmzk$p zDr{|lydUIq+SKb6tVxtucD8k~e=4PGlSEo%Q036J(lJCdk?|j7aJ18LW{fzn(+A}T ziE{*BAC#3^wT~u6gz<(*Fw|5*{rc~_XZqx8pv`3&(_NkaH9$+G_q1Ndhe*kH-9mc2 zz1Sv35C)`;W{8}0A1|>$PlLM_eW1_-MSW=n%?X4Ekt|_X4|4!?*{bb`iOMEleTK#l zl?%CutFV*IkQPgI>oU3v`aI*lhm7>If!!kF#Xj^#NEXnZ*ViaGsKMyIg{1mMztGip z!Du$9o6xAc+WQ%`l}Jp~ihonx-b5paO)Cj>!*M9h@E+amRV>tmmhwsQ8TZ<+^|0VQP5^`>jZ3A{eV* zCCmS;lK>yNG803};?P~PVvu~D60`KZoJBiSjnVDK{SdS3TF2btP#D~9-EGat6o9Om z9WVYJOd>NSf>Nz^rofrVah4~N^QMn4Uuou|{QO6yhnz%OP}e-;t-1PvY(vP7I8H=F zkD-+EbvmvBPm@mFL>*C|JgSGFDuiz}v!tr&6xXd-jjMFi&(-8B`hAmFO@Ku~6qEBh zWB5zQ0s!7-p6(e&K78(=GDNu6*20Yj{OhFQV zv1u|=kC)!DV*6zJo<9((r1E}a%dMUsWph|++Ecwu_5peEdV;VDq(C*xw7u3cO2uaO zmhxnp!|iH>-{^f~ZRFpJP+-?_wQTBfC+U~?g}p_dL}-!n((b(wA${cFWA6+ z=aq8)c7m`qW0}L53rvAX2Cg)vZw%vOrnKFpz-J313kg`HQJfvkdoSyv@StX-GnO-i zL>jeL_ZOodS+44VOLz_TZhvDMsAmmMG(~6OO|sPMIkNI(=Puuutr_AwhNyCN8UD(; zW(jI=>XM|cxUiYrPJhr*xnP5&;AFbcrkiMZ*iiovVdSf)& zm=!rolW)X5(Uq1DNA2%uL54p60&qwLUtf!U_5_VN>QZ;X^;19~QLUj^ZErX7Rwbyp zD25|8#Fl0+3%S%M^fu=Tt>2jKunU{_P{OBYQvt#t)|Y#yqKF2qdnqo&W75E6g=Dl~-?=0V2D{zb2<-%_aK)9H$lmU#35SzY; zV&eAS*J%*kWWoEcGn*hSXu`eG@Qj$p2N%bRNH>eTQ;gG+rDM}?cA;0+hq3W7QbT*S zeq}q|vRUKB!am`+oQ(I7=RSj~5h-l4eYxD#DBvjEW57o;2HTakKa>f!IWNHCbE14Q zGl)~wqVM&JkjLig?YO)$i(8sT6kG!Xx(eiSZ@M2avR&Rc|MP5g?65T;q#*-F92e1I zs~iHa9}dpjkOd*3+=|Kd1R5OFhA7iBpkuFO*l)}s`>scmN_osjB^i#z5+)hQFyO!Z zmok7B=>%tDcX={rli{zjwYZMGt?zC*bH{{AG5Unn=8}{FW+##KEt|%h4-ECWdnP|6 zF7aEONEE<0)pRXqu*@Mc-Yoz?)J3Lip4J0B0vh1ahbi73qu*$Gr6TNba@t^U^1CJP8Vi<dyG5a7h zT-sW%*xtg%Yfph|S;dpcY0;R>xqlBPA9l36-`A$|{URIFq*!BOYARcfo>^GtujKHh zbgfqp#7AE}Xq@#o$kKJrx)j*+=QPd{i#u_xl5&xZMG(u%c`ffYyuV0Ytv+)K+4 zrMv|TU^h7L182a$`C6>#)<=`%V7okayVjquDoX*x$g>U2EUr-1oF-qUQ^@&%5L{JV zQ?-D^2Uo8#HO1(%w>nPbQXS{okS(C5=IVJEXM|n|O^x39=>-P8J-GaPZH|jR9oIW0 z^eXJs&7@6&J?n9{#!JH(`ZOj~GVV~C#4gfkW_#`!yOOyw$BDO9ZCG~d5|((_*}W!_ z@Sz^x(w;I_JSDe~S#2wiugtcVO5#4ruLLvLbMFA76J9}yrujBLA=nb*-Ausm6cvPl z9Ih^xWmJo@_rgfEkNjfX`jlyDTz*z-t-02|oP?9N97(lI+@#|g%W6-yH{Qzyv>U7k zu%aT(1M@ysgeBgwf!}t{W_+SrFDh@4(pju($n$;XhEXhWMVw>*?b=tTCC+MYr>5r` zbuYUsPZcMwsC7(ki{8C(Z#YcN07Q#BTyuv|AFv%mO1-VuZ-2!FdUi&3vy&v}MeFn0VW5on_|IICSrB_c{1VDGDbop3mFoJ6ewQ&Wssb z$uUrm|2=WJ#BlfD@`paqIBO0)8G}P-_X9Iv#NWc$@xpEa2UAnn{vhcJ{$YbZ_OE*i z@M%1d4i_L7UTP2GRTXWnzKH=aL@C}&2>#1BfH82~^4~X+Ev&G?BWnPJ*~{^yup#pP zh;u#8N|lQ`0m`gp<3qzZbH1`AS65!RW@`2P9VX*vdb83FJqkPy%Gc!LJ}kzt9-QKj z#QAqZFYyGC$V^BQ5EHRx!R1NhWPnL_>(srVi7YL5VSp!WV>d+Z>THlB@bsNMs|L|w zxGHwuA-P1;Y;VeFE*vM5En!AV%`MCpX#XY6GhB*~ydF41HNF*3avc>cahgpIShR&; zA%;1kmw0&lVhRdHAjkH2H515_6A}IO?=wd~l3<6(8fYNOxIJ6rKP z_{(@*@cLMtN_RtE+ADZ!Gxwx;>%4p0yjYoj4-H#uqNt3^PBM0|oq9deZD`E<2r9XN zS_NeCaa{LdW%5iEyrMaLeZ83J961Z!pMwd6BE97H@LTgEE;z zMuQYf^+cbz7BjQmirAX{n>qcY<69y2$dwSZ zk6@}zx3!r$eJ&b<>vZs+YKSxze>ahI{XNXwr5Zwv@Fk`OdDqZ_&%ai{#bqMNXOkJd z!YY+BFZah@&JGfTY=S{|`hv}p_KXJ#*7=&|Tb5qHOts5OEv-y_VsA6t^AIP9KuxM` zh9}_q(#{E~BUui4!ZAz@%X*gOEA+tL)x;J5=(#q z2cF$4?FRa_&_>1KuoYU{p{-fS79)p)?I6zvqJ)nau=m$sa;TIt!yg~+H` zr1IOooFO{c0RiiufJ$_AB?Lof#IYDwt|z3Xrm_^49AqKUvrw0-9o5VfbHo=#%G7BiLvRm2~Gq1iG`&` zr>q2HbG{#}rsyrt6a+D^G;cCcl!!p<*%9Pi7bnVw~2KtWD4Tf}Yd-8>jW{ zo}p~f9ZlH@m)U6{!X=LjpvhkWKI3n{aggDB86h1Ic3D+X-F-ie9i|3HCz+KKfi#;g zeu~5t@`A}p;4~ZvBStC#evA{pi<3~j`ZeE(>ij5Gu`N|ao8zF- zx2cPuJQ>07lG^@VkKu<~z`06uw6gbTC zBXmX4yCQCBvT8;`#=KlLR&s`gZ*4wzD7(aheTo~^6=c%+Cihgg#P(M6dyz-$Qw%HB z)$T%p{9i0cJL48&WWAYrBo~}Yx7IoMwh~W^mKGM-j5I{etrIqrg=nc>>}l0bCu5n( zSJ83(k~O&H`0kR>f_%aw*k3PBc#mL*go%X&mh$-ortH}GJ3wI=Yz zmkW}sruSq*^K!r8CknE*>n^@;ol72C2_*E)*0@)jsJ^|hRDQPByu#%|Dic#$()HZM z+@UXoN5Cp`v7q0}QyNi&>=ae0xUPtf@!eq!_JZr8Y`u+93LE6IrFFap- zx5e!eNam_NX*L|i><7Y}HZx-g2S{F*mzDkyufzBv;5G#$W6%rc9WL&;Ke?A(e8UGd zdQSK*)NV0Ca8Nbd5&rs0cVj{rA1VOXb9;+8N&)vow5YV-mhR%Tkj%p}P^;5TSi01G z;~8W6LhZEPk1NJiU+o760fT$8wyy#5bPuTXhn((rFVTI$Ber(8BnKe+Y>@mnLDdS~ zK>Kn`G{7WsgHBE~6j`84`-%GRg6)hmkt_t9nT7O*qlXYpL0KZOBsrC4&62uzj2-5( zR(wNiAKkv=ww`N`TYK+3&v{6&HV0@VU#cyiCr%FR6N#=U+l;;D7@)DUpdL|c102YK8tIlUZ?<1v`4g;p>siipfSCA*u2pbVkv>?-ouB>`@&0Y6eq8h@`pq7^-QcmK z(0N^+W^y&N$&5nF?c3@X%JbD}F;i#FEOV98CFc`PoEnAiBV zvvzorZ>N#__cpUj0Y#%`Te+-hd?|c*c|@{u7r01eK{;UeGhj89{M9?(rh@aBht!2N z!pj41_zpnd0NTu!L;o0y_WBNeQP~5r((l4R^YM50$IYkLAp9Tf6m=8jj^Zu&lTo8a z`!Nw&lYCmChO!$J`AGfR(%R9N!-?Qze7x?fMxkpxBQ=mbvd!T=r z()JB1>zZqP%a7Tzw?6V44wZeWaS^2^8lagAt$I9nNpOJcd!O~r>i9XKFCwB-6#Dw+ zWk1#;7Yskul>hieg)&&SJ=Ie*_IBCcXeob~;IqiowFg7^FE4A~op|C1Os6&vDn2e( zJM4DxpPxwt-!5p=&Tsr!@vEp+(U9bu%-ve~G4V6h;Iqn$T1Rkz>-K42X%S)bO zX+f8l+xRUxb+SG#pEg{K!mmz{`sW(sNJ+I=xdc?P=e1@W1w4uOd>=lH?zqlXE62^{$o` zQJ;;UT?byqyRvyVF{jD=xrnwvHcmn6l|;)g&N5S!YMElmf7Yfg)J?98`Fmc@zEPMQ zf0|A6_g94o9yjgzcqNI8+iSJn*3Bm)Y8eU#E>Dhrtr+~~n%odQt3UL;zR555bXWF= zp>Od41Csq7Tfe#ellVYD5y>|c>a;zFJ2)a4M`^U}R&v?ya}pu`1tC2r17vlUM!ai| z+FOi853BA_8On>?>UuF7u(56Z0GNF_>yB?jgY%@PxHo%?mU40jw_`HBM`_A`6Y09!=B1&hReD|%CTl-gO>JMg%CZtu1!>=?N1%4 zS+mTumzztozu$JP5quFxqOWP>1Vm)r2a4tjfgtp z);g`&>B|{A8W-=j^ge4|bC8K8f1%^1)K}DJ}6ThBE&#DT; zlcj$84>R6|m2v%xf4YylP{&1D$ErOAZ5<*rUepx5n5Eo_T-*RW+Nc+)Db#b% zMsCwoOpaW(&hjUaq^I>&*Z&a*{TAN@WJvNmgFq4h?n5%y@s(h*ewN zq+R=LS&g6gJ@7p<&6&Xdxj-84o8_BRPSb12+Dmia&3I~(NIl43Se8Db%3qVWM)k*3 z(+2eOXNnJLST8Uo*MxC2a{(UDBR>yrT9?#%yIP!;d9?0d)tm(RffiET*NP-mSDb|{YetVZL6~soB9`2I4Aq_ z>c5TPZZfYgPeRVmSA}ammd=j z96FoXa`orX=>Zs3dS3sPyV4YsbxS?L zvLb-O{arkhiOSlzVPU{1O;MqYt(<5KvN|vC+SNc28-7GAUg_HWg7uDqhDMR|p19+6 zCHoJFzIWc|mJj{~z9$_d8BqaJ+SG2U_eMkYx9v@-Ji7kM-suvmkzm9^IiZbaqYNIKWXzpb~bxm>if^84-3-sx%)BIQGeP(e5L#fuh>VuJe!6NMJiY+<%oOEwyfL{!R|r$;7ZncZ@mPxFwTY{GJC{7WqoHY_JH=67<1^4F z@Z3`HBDi0?{8^2zlFa4 z>A8L?j{d%^=lLt@AK9BFsIYzYQB~-c-PvG1*sM*gZ6bOS)<~hwg2JN%lf$vTh z`WH#?duBg8!{IsVuQ61mKXof_hjwSmF*roaM{w1$1_VOuvzn6;;DFRSdjtpUBt5Ue z)U1!R2mt2njaA7rBz)x(o34B^JHoyN8P{t7=7o1nqXx9zw_#_GoX2grq~zw%$~3re zgPU~6a{P?pToyMiA-CX1b~&C#+drU7!4^KW#k9?=`~2MG(Lpg#bkesiu)>(R9H z15(Jy$%>NpIs=gW_B7jiQ*pqXK}c6ZQPk>vRiQ_OC>uJn04b)%_R25U+3ItW1Kj*q zhM%#}jLB+o;I6<-N^=7 zp_x8hr_R!zpK`EDFKd+Gb8FKz$funl8|boWmK`7{t1zZ0OiRq z=4`o4-vY7`{o`AdL|+ptN$S(jPEOn1eP}F6wN-z1bLkDZCY#l5SYPP=sfFFdRsyKytnZ>SfR|%(;>5v%7rP-drVRuh59*H&OZ!7H& zujSG!pF^w+M3HOQ@te@Q`_4v7(wr&T8VR%>4i6;Lt0CJ zQ0ietwTnL*nl|)rbmW-gTx{dF@UwUL!Bbw|?UKE~H(LAH*PTyQ z)u?+8DHA$AJd#&`r#m$z^kEuEBv(>IXbYoekS8jE zjX}?4g=fviOP6MA-*D3d4P7CRyn(DCFI!9RC)~vxf&)9N zXV=S!v&TZ}ud@+0;5Mz7xn^W6`{dqhd`(&g}=fi7>_a^t8+Ma`&8AVh8 z6VMiLDD%OyOG4~!gzI7pJa@(tbM;a2YOl+8mYbqeS>OrM*b-3^;+dD{J~VV8sr?Hs zTU}k8?%~NTH~;m%S-hjbczKDy%;NG!FFwMD@GM9Vvdo&4S(W@gIo?dkl@$jH2|&+o z^PRYoY5Srtt(YosNWq@q;X(D=Wt?EgCW02!;ty-mf$jEyJoBuI>-7tvrv&d&}CB+xMR4w_K?eIFpTbV45t8Gd;s^xrz0vHxQElDWNDIWbk24CPY zoNXX4uKbx8SAW0zX1u{7o2T!kf*<1K=Qcai{K&v7rcsthe| ztQ{|*)iMt8XgV?Q=>C*Fs^i(yylKLcS5#*pJiZDWkCyCbfN*neC_bKO+(<=U=bY>I z@DJl=>tON_3K6#XdXTfzKuUvs4ow*UL${DyI^j7b+YZ*P!v#=6L1bgA{5PY%Gxvs3 zKZJ{r)zTK6MiqU*o@eK^{nDUE$TiIycF9See!9mpAf5eiO(9WAiWfKw3bCmSpG;p$ z%`!L=?9*Rj;svKlKr;t}lKyBHG3IMt@MfGv^ajy6n1&KnXNIm6ioRu0VYr9dT8qkN zPMmL~eG){!^%>GIR*K*Gu0HwzAr-+YqTKC& zH`LujGq2pd26)>@< zG+nWEbkHuG)Y#q*BovK6&Y_y$)|AgLg=|F6+WOW&uAKPxtdFeZX`jAG2Tu0e$)&$= zj@`#|ySO#vhpMAUvd)rpygC9mNq(ET=_VT|wp0CNJgG0c?zna#hZ-PW_2M|~62a$& zDE^^O+PORDJ-+>Zu~hf}QFi5lP_EzqnoE&3gchPC$yQRxHZ7K9ON-r*B7|gT7*k0S zLS+qw>{NDxk+NjZzK>n@?8Y$jd)~2kzjyiFf39l0^StLd`}v%6LN{7fUW!prtR?v$ z3IIQ0-bdY)(or_T%ulYI(yuef_Zp4$v4~YX(h2thke$+eC_G(()zjyC)ZsE+A~NGg zAg46$K22hX=Do$|V9zR-zLpDkNjj&13w9ar=IH$16U9#t3>b=BHjWoNhwcABQR+sY*vc89Wiw zOZ0W|;b*EoPFOk$eAy{EG1|)6AiU>U9w&$3>sZl6I}ENGY1<8%GCJRmqFO~$BefF% zVt-w4QI^f{jatsHutWTw#wqzPMJwW^+E1tEs_Sb$A*%T5Y!5g@T)v&j%~ewa3r8+l zrhFsyT;Edya%^uz=K|EU`BKFhT^5KKU)zIuzezyOpPZ+!#U(^&y_*m@wPOBd)wq!MGDA4y1JZeuxrdbc=A9*GJFFM^cy z;#^2%9U-TUoo@%evdXmRu{6^Yn@q$!-8~_+r1PT8C#qM=+jsu7@t(M!Z8X4gUh`O( z!`H+@;-7=Tv!-(-zEkD4L+Pr22I#PJKDsY4GC!2)t)_uWSZLg)B^xf;(4lz3lwXI> z2_IC_n;{h*;GiYXOBm1$xzh-S1F&m#`Rlftbs^N=hKs%f4z6eR;K&kITC;OW)mQrG z)uc4Xoq?udZxn%k{e?g)OOC&NE2CQ2jEbolzuV23PtTS|G8ZS5N?4YpGZ)?D^6@-_ z%GY7ZE0VWdC@yERJt%={AGholn0pS=FWi87EU67AEt_z}eL$7eCQDI~(|nPz51S$o zpsXdBezx@Eqd?4+*_dd7tg}%3G;hl}o(W zHMlx$xA*5Rv>gtar}Z_TRFzhkNt8mi?V3zU5O_9HO)%3(&+i%}@sATqUQf18D>IJ? zH%Ls^bOg=vfUwrkLDtQu!{NH&Y?p@ApJ#Z$WQe^J`Li;a`#K%RqUFMI+s%nrp&Z3R zn+BF2IR}7V^@ZjU6H4Ws(#Lka@yoElOto7FJAiz}bIa!9a$t#FdKc;@2_=;Qp-fPU z<2^`nD|I?dr!Vwc5{f70@%p(EnN+?Y{jGf&mt0*9S3 zsz~&>7jPg$33Bxo`Vle3-2qbj+YY;!WDd+0`tLF944}WZI1`6uWfFXRO^$sKc1AKt zSN(}2ROnl>l=9evLkXYh;Z(c#97>V`oj?IAqUyBAOBL~uwEUSIrGR(iM%6xeSn~6) zB$BLo4zgQ|R4sX@B+I(6x+llV<$5fst)-*{O5?cXea-9zIh$pRJxK^C-fN`dOCHd< z_k&3-`mLl0-XdrhCiSX@q3jNtK9w$z0v#2KpK%)Rwd-JMdMhTW)PcIpl0W%*vYb!N z12>U+CDg9Z+_K2Sy%^M~R2D`N&SlputKNx~Jn;py3R19sE%VdL?TDHAN$X1o1@deM z<)k8Ot0O^PKSeAPGl+t%=SH*l?g!vj{+_)nKo>Xxh z912-Z)tqNLbt)n2{oOiUm;A3p?D?6tPdE+IGYg-ED>EUHD0j1)^z16yUH(ICj46p< zgST4%SGX~6(u&vW=nk40zk(MB!e?qrqj!lx`7ux%KA;w6ZLe=>qWNU8Sx1;L&tWnp zX|HiWTK@9DiB=ha3TlC+NY3AZ@-&k_$Ao1xt(RM;uhU|lM2)05l|pHN?J1zt3uWUl zG$3({5L;>_y@_l;lYo2ER5=LR$g}Wi9V2YshZ!yM>m80>o%Z6U=pQ1G8jzxdvz*WP zi47JN`L-9qOqb`f60>MAS`19Wa-Ppk=W*@ecz>LabZf&%ynhK4RGc{G5h`grl0|H% zne5BEeqLeW-eK;srAd2VTqGMs$&AExypkM%4B|3nO6Y9Cn zyxK(pY^MS3voPQ!g8Ze%yAMTrQbat>r>{SZ$YDo<2LO=fu%BA)?eP@B)F@U8<;@rK z2o9=_8p#_PxPj)%njeqGI(N($#T@2GUdL-}H)C^O>PoOIuFmk%5)ezRD;hGj5Ib+n zhPqOL={lNs)~1&})(P$Da`t>x=xobXk0aPleXm0PBSYuM4y)eJe5v{<%%ZzUFr+e6 z^bD{;AKZqpt+owvG(0Kdgv(Jne}YPp{jD`ec3&!4R52R}d+hFT&PCc|S`vu=X)+_GFaW|9M|)!Z;La8=3k2Ck#~ z`Er;zDCA(;?tY)|K-HbKoWKbPWan+^(Tf$OU|6Y~a{?F&H^mjOfPkFehx5?}LJsus z1-*e6J)XHI&yp19@X}Bc!_W@fw?lC0nmbz8s&$^iE(eq%8f;A$7iP_Ii zIC#cvD1}*rHwUD~Z8Ejy=}O(;t=}Wxz#QhfF};}>(4bHR74>L21`07doN`WFbKvB}3G zgqN(nev{joY$dY@Iakjs@M0FSoDBvl4)9@Ylw9F%ROGZ%ZjVV&WJPMhMc@2|BdMR| z^i8lmcEVmWcj=%QpxL>Q)3^9UZLSJ(=t+1HC|qcW%x{bvL3c8pl3Y`$^e|_hA90c^ zaXs3cHfyEbtWQ;MamFY9aMMM^f~vm$PofcS-*z2qwHM;%syj98ruU1?8wn0heV-W$ z>fZrc%GA3+-QK31J<_(AtB!-y2*AJFj*g95))mDRBb20$CrkFn20rZ1o?OrombKtj zQMM`#G=F!3?+#xmKp8yVfvly)2H0I*&P?^5h;_8>D1037eAZ4v#AVJM1yvvrS1~W# zNObpZpwl7<374>}+!yZD{V3VTWX%RDS8i|r@Lbkdo<=Nb-pmSf_&ILBh$F?9sc$<1 zI*_LM((WOh(`Ro(At2xG(7lAvr2vt|V6NN1pXQhUfhvfA>QB!bh1GcLW;zzWjJp?y zaS=s#=k?_`_qa~F33x8F_gpRRy6e8LxEYkaLP9c~6vcLR%#?VHl`*bqzcTg++6`-U z7Q)FNKm7O#Xh%U%_^!FN)W?8(LKMlnu?)D(LO;{jVF2;5?h{U6D(StkuY8QKDLevA z)#*4}q!fuF%m(zLLrC-}Nqnh@e*HGfnt3b>RKE?Ks^jZ{@_KSi_>w4`to-5Zr6-Sj zXFh3_Xl$RTF~;`ae?U0PT+A@v9^7(Q^U1l1i{aIvTn~VEOQgl|kQ6(f(pTzp5K820 zX2)Jq&f=piVlIx!c6)aB60XRZlqIMO!6D`e<7rJ0%42vYABs7UM&%L)Lm$!&I;oh4 z((<+e0Fjd3rxaf1k^7{&WW+`3KDzS%| zP>Gk;CcLv{GKCx6JIfq4lyxo0r0(|I)nc{uw8bUdory@TS0bD>fiR2QrEsM` zl`*ywz-u5Ebh6!jUPNZ9bCVGj`t^&Jb4SFAp&AebtGT$0tCEx?K9~uc9AV7|7}QSV z!H#<`2Ly)#oqL4EoVmPP;<=N>hd%nDw%HhnfRrAngQR-$(1Lk>+l zn}51Bmg9K3dWp6e#MSC@=MLnzlwK@s0m)*q+TD@S$HIhs=7E>WeJn``k%@FRYBVB& zeC>UWzR*9=GJE%e=|!6&6KLpW{w{hW!$6=?K)k&M5r zpcoLIgF(hjMQ;BU4-SbdOZ#Vn@F&Ia8hKI=%G2G7MpKo}kizpD+W9$xgmxTK=mLPL z7nl^R(BZu0V!7}8QzWF)KVFr~QNp-~+7`UA{I!ViW|(h{{pB?KIFop`wx?V*a6G8N zf_p5u4GK#J>iiHfd5agJbd_7Ua)8o;h6ZWpwW(u5$6gwsU){lj1PcEs>_L}7+c|whO$*mjZ9H1#s`*;KrLJ05Z12%8Gf* zWmsPz`SIS+XlR4Cmn{# zs}~A8d6H5!emF8x9#rgtN-<~lwC2CuKQ+SIcw&^^FS%~~h(rtDoqRXU+*q&GO!%3n zX1UIpc#_R5m1VnaLJ5}Sa5IzefCE&_)gQWjkn=2JCX_uqeS6fRw@a#1VuTvL%X%#r zFl*Tt(IVSxGMg{cLxG-Z0j5gl%AyuQ#&(Y(_xxEP&>5p|)$I4MoUn}x)plrX=V!VlgL9-a8 zwSp`BHRa+hW0tR*d7J$b{8}Wnf^%0@)&!ecF@qhBzFLY1&&AiGdVphDv@EQnvv6)I zjpu?qy{PkOqvdio0|XhkEPr&Ug3I|a9{B(D8mWH6QuZ-&wWToH*Vb4W;JYYyf%43C zb-f&rUM>MudesHPDUcjZP-)CNEQU1&R+DI-w!KHhfahB2<8a9-OV>KCs_T<_~NlJ#!*GNjSm8)8)dgCOc`E0I5I^d^n31{alRD%LZ zQzp_<@3;4MylD%5pT1r*a;rvNJIY9z!~0T_fM8u5yZET|L!^&K2bVUvumIJ$Pd^y1 zw$8@R5J-o)wCRk0EJ9ca92&!Q+*+2GBPP- zR23SOo%$W*#G8BbZ9qVBS@T3bvGnonY6cPuD7YYA>?dF#6=iMu&R$s@2dbPR#^H)R z@3@;7$BuI*jUp@Xr{|I3eB@{?nud4ob^qdi^<@SW`Kn3MV~CEbNIQ%($CzP#%l&EzH?uctdiQ;i`s^SK*44a1uW z^G%mqEAN1qxu$yoC1T=pVlJtdwKLWabT5E#T$0&a+3Bp9aa4lXzO!8%%!24ga4d0| zHl=}h7o;eQL5Z3v6S3@WKN;ulB#xBdmp56+aE5RNHUCoM-OE4FPxpbLPYQ3(pH-bI zg(et+e$ZV*>_3BiXmY*9%r9!cDDNFKZlrx5B~rVj{?IeuLt-+ChSrRP%=uhMEd40B*yaq;|ozfJ9k4 z*pE`1wVG?>X}QAj_}KHN>Ptd(x{US~ts^(tq{enQb_>iu2LL%v2od;_(q-8J>viCa zR#^6R10-RosHlt4IC}wH+k;qh;y8$FnD^6PS6mEeu=`L-1Fvv7a=B1_Eb6)fx!n#mMA|0u>3<<7H`qqXIE(3@CAP-0s0N5E_31rSNCj13D33ZBNn& z<-ivcqb`3+3#obb3G4NkWN!xv{>6(%NfR~xeu}~XprdXu-SJ$q_TVbQer{tAGP@ILxVNL6-E`3)L?tzXP|4Vu%X z<(WV~$szS&W8%QZxDy9-+E42SMcZd%C2#2<{iXlIq4i2%IZrYZNMpw@QEI*baIY3I zuNk)HNpVf{*?blkDz! zAUq8h#J=pH4{Y;@{^EdKL=%;6*rIxaKMw*q{H`Jkft-3L0Obb)O zYF;23Jyi=EOZz!f0EF;qq<5fAp6$+Hqx>B{otE=#&cAdGAA>YD*(Re&G=Lx^62D2; zC`ja$KzT?=%AoQgYt=ox1&KXFbo5m`SGEL~*;>>p-XF4$*xv5T>yFv+PSUc^DNUZdr z#xq}l*Rm(0~pa%|pKrLMZkfvP_bQY>#H))6Bb3h{aTMt47ifU`ihZ7ALbaI)Grp%O z2Tm~@sks=z;q-h^f0N+;SS!0?g_Qm)aDS zC_vNWPbodUT~+Z;*ER3_QkAw)!0g_abOUIS`)G>C1HsOn1TKk?FCzl=xMXe#&*l!0 z0!WgMRw!AsP)=mPy)$P6PeeBA5FM1Aq2HgO(Ku z8J>)<>cjTMGd^{mk-}6jbgsi_d2=ogAP01QP6zd;Yd2>i=EBFqoo4bi!!okzVsi?B zHxejncrwMdaJGG#Z-2-*A)x}lj|Vwe*>se(XjtnF$|}jr;Bm(uV<_BQB|?NIY-3za zEHq6j{^22L0xV@PRqiS~cyZRGpk|qaZKbo+YfD+qN=_tyi{(Sh(?AfC*TXqqT_&(Y z!%)}I>Hab+vHNO?Q_o3a$M_c;eu@l!)aiM(Ec4C6j-<;Xaa)chb9vb9@+%Z+IvxOq z1D89AecAQGm{CzOnRLK&%EcI zgqEYUeV&~WkKk2Q`7>TO1vx^211nf=n*Cthfw?qPw>6qQ_2LoqHpW2HMk7v6^~_zU zF9OIovuih#14wgECjlY&2)LfRX%_(uDhmF>@nHO`tq||ku^O#X>=!KBS z09zS#)F7W&cm}*W)N_MvfB1U8)D*_8Xc&raAOz}BgT`wyA6mO$(JrQT*@0nMgg3T2 z(V|eF(rK4~^xkiI=bncMi1K{^&Jc2;Q*5j?gU{Z0IP$b*h2gdPvhUO2wjOugGx93W zqMtIU8^2{s_as3WE8?Hq(s@#5A4%t@Sr60$KhhJG+HU=xi-dIapZz8 zKpe_OHHGcYR;aTx0G@+xvwl>oSj>#|_&BKN!5OJ`ZY}tUG5|fAx?MBu!OZ54aDcJN z6&{PtZBr=$cz z`U1z99x84C&@6|K1;OLEbl;6?OeaEdC9uy0Hv)ogx4h48Qz2b`n)!S65NM?gQ$&QPbY zvOTqiPxgSG8O19${B^sxbL-zNOdctQy38O{rp>lTAVKK zXwsoaNsKg+o4VYv%{)+-Md1i*lWu+Y0Rv8>wOl}Z-J|A|V04=6E$}05#}>Jp-V=aVBPwnm6T6He z2V*M! zi+`X`hZk;ajan$=-4c>;PMe-1vuiYETK-DWT&zJ0nC6;{R)gTRVHa|Z8SMnkxfg+-H*^c@mo z?Y{XfnK$(r`L|V_vAA#b^V|C?hY`eq_)ilDP(F-JM1A0lp)d!cpM1P9UBOJt``Mm1 zyYS=wj32h2`Y&)5>j!kO4IO$e+-jxGiy6)t8!tZzdcDZ@)YOwYRqS!8ydK=LvVKjJn#xzk>`_#5F1i{(9fC>q+7JF-2K6 z1_*nt#$9V<4Z)Zy=OoV6_DrNjAbkZ~lI~j_1g$!gAjj;Hv`19b1>Y3scFZpvVeiD^ z(ErtK1VAFAo0xtC@Rgn)FRka{`g>!+Bdo{l6b7YahAfg;6D;yg@8>pM`&4&&ut>Rj z!HF#6^$k8H7nnO3A?PjC)WPzgs?zcs;~qxI)Hp>zqW0mD67BFHtY@UjrgnNb#|{WC zPWLrAE>Q95nxjU}{yXxkG(`D=v8rL!>U+G7vu_$%LpiLV*WhJ$Pq=qxtZ!_GX>1pd zw4Y4=!YtTuclG*_m!{|(YJN^%rBmnwX{d||RXmh2 ziDk>iMyXKnLss){0SofoHdSb=n|JR&-h5$k@D3peE4@B|>X=V`W#pM_TduiuZ>&kJ z+~wk(`?jjJWXj-kt^AE`L$2xo5KGP^>-b<#)on4PqY)`_*ree9-uv@#3~= zh~L5y4}*fk?8cO;DmE*6$PI(WcYuyxFwzLx35)yke9Mq7X{sAwtZOZKtSfc>M|e3_ zE}`2v_~05p!ewDPi6h|EAz=FLinyjCD9{mb?$y#6eP3 zyn}ocuuk&O7TT?juHC5okeOh891Dn4qkL6gfq5+vDtVXQIX|uv&b+MdAgdd zyaUY+*|hy|g@hXuH$mqn9dL3P^&0>P!_5Eks8El@g@_vzV3Y#ns;trKA7>q}e60Q{ zI999uZo><%a9yfzTb;(guFQgBuiutEqR)fGqH4^Zsx&9rf>^TBh`_++JDn`rZQINy zdprefTczX7q=*Z>a$=$#mL;r{Rbms?ttn&^LOyLY&AosBVlce$pr~=wwXjbjEBPfb zyc3h*6aZvuY2&amsSQHwg}dyBOqCkv3gcPtYZ zW-~yd9F{A%@gLs}`dEY^G|_4_^7QG|@9L!bpcryX=*c_|g}tex)|$}~9d%ch4f*Rr zd3k*Jp{-Soe`D5CAHZWv;#Ymi%!z(T`!mHI_ITT_ldWINcv9NqlUd9Hw9K;?+6N_@ z;t{{^*<_$-*kXx(-XjJv%3*Bypi_Igsk;s@+t&BI4i z|Hici`du*oxZ-pPi)FVPb1|cuHr;kM!j5gFfWzSF3XEWJ9rZJ za>#nWm~GS&Se&0g2-PH~v5Gv-?UI- zO^ZN-+PR@Y?HhYBSCx#(l>+i)@BgB4`ZR%3Pi9N$+N7dR=)ggNe$!w4vktunNn!NU zwcWfxyHP*ex@KrACo=B6XJQf$Psv%uTjua}t?qXL4Y%aImWHeBzB#Y5`flrrn=F>; z?5DuK-&=L}oI2ub$>g)vDA=`r^ae-;>R3cVRudFxT>foVe0?;S{e5S+KF-%q8rtO( zDeIG5E}{?&imW>~-3Z(Z8q+?x{f;B$@TG`l-a5qlbb*_j<9Fze>Qh{U);+Kl5i6z# zn;5+2EpT45FQiw-gs)H;;1)n+Lam&gaN0yUVkhkD(_N+wR~NUvxkbglJaXL1vNgaJkRI`^#n@Vi}wuJ$)+OoN7ObzTN!nB~)(Bmdg1^B2CZ z)5dF?J7RBAtk>q(p0ZUVke?o*3j6cv?xY({I-_c9Y6+}a2cwvaZ>~%S{r@dI zUVT&0Uk`>u^<4-A!x{1W_A;(%;=dyV987gk2=c2gxZJ=;A-vgjkI;n-`-uOUi8Y6J z?QH;Y{yORrx{#lz3TYUQf9v}4*x=hUI=3e4j_zG2*9l9eDOevN^q^L5mlWgm-@Tc# zV-f$;EmqNhwymq#{*kvuRl#Xs!Pa0upqyQQJDBBg1=>l-9rqT1K-SJFPhayl%V}%M zA=kGS>l70i5OddlL)GBv_wJ$qGv|R$K5OzP9T-58cGF_ITcGCP1m1IHkRsHL_HR^O zVNY_3xjq^4lAX^P%0whn9ajk4Tj_Ch$Oi1uv0?T34!saK`N7R}hnW2D6%zEXqVymk z8|luy50-wc=G+x9AG^MGslx3K^uNq>Ts`%sVR~SO`yIf&UQ07Y0Ay=G^9)*#yg|UN zHgWau|DphOAiF@tt4LGt=bF7n`?X8>=QmkRci~2R5*>j$q>%>#A=49{Am8Z00lgF7j^Lr-lY6p>KW3LGfgNIgy;`|#&yT_=9r<4p z>5~TYtYfy%|GRj{Cc+nmC!i~}1O!6- z_pAI(7-AX0d!$btt9bkeSjGEn>(Jf5rQ{ZZ+dz|(&Dq|1s38iT{R^8ywEnB6h_IwQ zC~O>M8TLtTjsI7k1Ml7OY`xmO$}@j+xgXmAEpikGw{Gl=1%CP83-gA|la+657*Hp} z8izI!0o@2o{?ES;1obJT1b%%zeRl0qooNDp8_N3I1+1K5L({$F0vF=o7`!@QynkQK zRw(m7YBTat;g>*^X#BH80JuO0ufVr1i~FnQpa5&?0tN5Ne@?$ZWpvWo5FFY1{$@I~5G=yj%jlbuZYi$Nq0SEH(}PE0;L zbSUPev$c(qj+d4b%!{AhW$TM>lJt#3ubJd~&^owucOd&^$7*svirNHJnKgwR1G`(T zW!aZ^a@*eFG#27YxSO{dj53iI{X9OF$h#bxF!Q5@rL{4D-n9Ij*@Q)Y)kAZQTcddE zL5G0J!#e*ze^UY^jA}2zGr^cRzL!i&nTc0upKIxgfjqY*KmlU=e+afjh;qN{H!VN* z)S77AL9*pHad_*}m?oj=-7uGHS&v~h)=C)|$tVVzUef=~;1K>U7|Smdo{2=HiCctS znl_z3`WpA8mX(5T6@b?e-AC_I?4ok%E~=g|X)ZhAnh|;il5fz*>nM>rTP$ zh-~be6dQMgLLH`i_}q|ct$YE^u$Xy;^2jj)k4|Qy^5D>NkgnKy)e!5bNeqkYp-nP; zLJ_{Ea@YLg4V_HBsfYncikCP9FTm0T^qvBZrOeJ3rA#{vc7h~I<10NEuVXS~i9(le zjKIurkB639G7`sZO#HnE=ItHBW9E_PO(b-u*Bs25H?Edjwc7tk!%u;V>M>MxS`1w_ zd4%`D8|-}X7P|0o-MUAMHyR^Y3eDl@nJ{HAA^qxM;{svVm*9~#_OPS>WO4FJZ?dlC z0#=P2c+~>zMi*h%NUwIVi}e6d?cQF|9|!P!L6Q! zyr*r4A90ff8%)g=;_?K$1gtxMG#h_$o_@bPKGs;c$f}?PcKJ%x)^3-S1Ix|%D>(FTGbrY%9Eiw(3 zXa$1xa2whLe9dMD>flA7wl0YqLW{Xw;g;y)IAf*#+)S2W^$1ffST?fHG$s_TOZn8t zq?0dZ2JB}Uk<;yA-oE}sbTR$je=$}sp^;Ztr4Oua*|@XA8d~~;07*Gx3g>}J(9W&R zo;NQgkpU1JGaA9b1jLL9t=rkykFMkYCJUl;AWJ*kAO})(C$V3|7*xnxB>U-3WQY}l zl~6MBP^(k_fhz)(!)kiqnvRZToIy4YzK=XnJIKxHV_h$^I-9@dEJf1aep!8d0ko>Y z#$;fv(saTTe%4>6Oi&*x%s{?|4uc%#=pDWX%Cx=iKCK^2-x@1MPypR{+n&aQV;B6$ zWpESLY6Z?)?&E*i*Ap=n>N*n)(lxdIqx&gA z0rm)m<@ru?pUN#Y=YUq#^qn!ew*F18jz?_u*86U-e!0b5$;s6Bwq)5Cp?Uze zx~d|9PcN6BCT-a>b!Hc@&M@2nL}JHIIZXB0&hG(lgb@S?idV;9+8#Whd(85iIY;yX zlNJ6PmMOUWx6Himz6ia9e?9IcRC9*uBpS>>fdZz>zF^{XRh{t1nX=0}o`&c z46}SZMI8dq1j?ZGxayQ#TS%KrXKNz{GAHOx##86lmBPvoo7K!3+X0_de;%-#HLmI{ zkc6XPNuO$=9N6L*6$fqrOm7Dxg>3Vpi^j)xfX3pQyXbCd+e5x0wRh!(_x?dMH6pO1W-1h(ZHH#Q&|==h$cvO%qQ-VhkvE2t)fYGt zAl$ouqp}XjJCb*e)jxN20E7c)18>~IC{MHZL<0QQ#W!GtAN|OR%r7wKTa5JkB~3zh%qFhHsSy46X;**_n@+buzjfXNEK#rV1enoLbPzxJZ%-VD zxa#)JGH|+7NT=bFWx@7Ij%#rCc^GWV92M~Szq|jJTbAZLWdz4-d+MdrMHzqO&un?I6~o-DgMM|ab_$0p_F=nD>)8FI>AqS^%7+=SfkqvpUy3Of7CR^49XZq-^nsvJ~?k-P!ppy_i3D z3^p9-I01c=%I_>wKr)(ei7@*wF4J8a@pBwNUy9j9n_>FclX57{D^J3 z_uf*Fc;JOU;rL5N*;K#RfoCi3a86VPKl37-1Z;t{>t(u4PQAAxt>Mw2e_^Jy*AS zYbu+Q<}%#i9Zm>%%F_kr!$q%v6%c~UrI#PPaUG+iexcSihDBD}^9X+l?r<9HU`R|q zxgp>CW#HVuGs`mU=q7a%v75qQW;o0HJ^M#Kihg?bQ*`JAwG^eO5AoT`==uEKOebYh zvyThXB41&RD(Dt!>P>k{GFQhUek**Ox(=ReXwwarYlBy^qO&g>s}8vu+avB?Fp-~% zsx9>oTYh2n>h{V8dbr*a%w)|kTQh3)0$af-yFyb9>xS`Ct@QxYle910m1iyMUMYY@LO~r*K9XWd=_Kv;ckc zxBUb6T&Fu_eCMQ^yu^wFgx1)B{Ixt`LBtCqO{c>$^CG{KT;?-78z|TVIgO(3;?K;`7pAkY9lorP0RNnkjXg zpKhFFbe>3U{jkE{y^~>p zeqYsfV0NuutsGA`dCdDT!(KY{4$5)D+xZ!p$z+6+>GU|x-f1EuzXIn}?G%DrdM(U% ze{o?@-0Fz(mUVr{IU_!Ar79R-yu!!n9<3U`j5zrH-GqJbY;NMK&WH0Oj`M_PToUhu5oGWsc#FRu~j|H4!dAs{!luqSdy`#ACk**m>fBu*=YF zr3H@g=JgzIKq+Q9xO5`7-z?$n;w3hd@x19|H|Ow&gc#nw@8Hc0VTGG5E2P3UvjFsG z@P(@fc<>yk%b2d_RRBVTa#sLpIU$%dow_FkwaAEy!^Hwd6a zYr5sZSZn&1y$S~pyimQc!eIsVR4(HS?jdZsGnvW?AG4;t+3qJ}9VgY!R~;GwL=X;{ zSjtT?&mWOS)GhTgqn4M44h7>DqJ=|mDOlk!oz4c2zJGq(IcsWR7QDy#UduoT9!m^v z>eL?R7w?}Nv{B>R(9bKPiD7=Qp4>;%?O1Hnv9L!EQs>W zn-U{0U8|7QbMoCS8e;8~9xC(Qj@0EdcWdS*4xg8YAN*b+@FsWAY$D4+?l^Mk8;%eb zGU7}NPJ$RNm-MxR4thJ!qQA0ZKm%azs!2X4D-x`htuz@D*xR=6+V;sn|7|?-AEM@#Y`@s;Gbm8R8ym7BQe1c*A zEVD~A{fFodN@n*#%s3j9j*34|)#6nVZ+vn~}s9)~}c1dL~oUu`M z`VwtGMs_gzU$p~!cozJC{la^N2)w@awtt3`P89LIO=G#7A6<9i%^zqw>OO(wJMZOc0cHJX z>26$Q1dRD0zMzOiE4Q14wTWa64^sBZqeDN+K%eqZ~L{Q`FXzciu4^u3`AnsO)WBq3bl$RBXL|@rbHI7R z(QY~zH3|Ngy}*opkB^m_`POO1h9qqdMi9XcCW%E`my$yhW99k2^N?YhZbU4+K2+X8 z631r@+N$yGIRR-4imM^^*W$^!yUmiZrcl;syWut#^ie4s3|ivcxB8Z+UKO}^O#K{~ zCJqbFFwMUSN9kA)je|&&7?K90&N4qHgxB{k@VTa-HIB7-xA4*3P*c;kki;pp<3b@cW$$*HidYEfTT6+K8&)tMn^Blk)Jx)wAa3 z7d+i7&qxkN*tH2P)$oKYeU`K9ttQr?T6FTVmny1JUn=j?uHUTFz6az);C_WJU?jN8 zat4!4xf+u;oQviVSB9-N1_*GV(C2>7h!ybIM>Di()6OURTNir62^F13C-SDpJD1D{ zoLr_^D}9zKc{&n9$}>@w0q5A}zbofucV_;gB~|kear5`$ zPVhh&CM8Vo1eaY{v&a|5zx?|!bL{Dh{xT-H@wXxEiEg27cN)}56>g^N^7V1KF?~fK z%lX~2Z~~gmoH#OpdT<0=xsbxPuspC`z*&~;A{cfcE1!5VBd43%wI{T)CUhpub>}0& zkg%a30c2Ox1G1TA5(lsg7ASE6#o=EDs6jCc-T)H5K4qbBAPpgEyhL7F zu1T?VCHhBnLCAB&lBmc}5>>}8CClELho>Ax)t8Hvh?n%TUHR7UZsms+-?hsHxL)J3 zzfh|J0q6epJ)AIfx6$i6yT!uye(dNJn=VnO62u>4g_hqJ>QJP%U|Ht{gU?w)vx@t_ zS1B*Q(3*b8;XF0S*T{2-I6Y5NnVxIH#^I|;qwc|W%dJ}e=`SI^#=PWuTp&()`FFV5 zhALRk7OX`p1?PcoNQ@lVzjC+;M|3{o;FTKE5W(e?0PWCO(v)m4pf?KhGW5MSm{IRm@rpU&XpWXl}y4vwLJ>rGo`?{t2HzAmUU*(dPd^#P+qbi{r6^2Js;7Q}ZlM?Q|pYvQN;pwlSX`mRw`3C3`H12rQoAc^9;fndCRjF zCd8tv`OC!*@!`--2MZ9z1k4W4{c?XD!qkYH_r{q+j-^g1>6O$p6|F`197f^Y1V@Uq1(2;hQBPu==h(2H%(`7U4^&Gnz5hTk z<-BU^zm73FrA2{WQ&}IlAADMR{qFY1MWIMla#?o2e?hwG+2uzafp;3V<^pY@)Chn2 zyfgpcYSyue^ViQ3i46P;i^bvSl9^>eIn1ZO3k;K#xWm0j<`KHqr%S8x{X3}Z4vLmGGw^i&F4P6oH2$ zKmcK$GtwKMw7IX~6eGp#v5x|POXoNzaw!duc6p`-Y2VYQk5y;K4o~87a{;%!e|=(` zUKmg&@(v~N zyDFkpufgi*OA)%?m1mFl4>~Fy4r-je4l?#iJH&gf_5?#VA+%yt!>Qg~>VxQ#EGovK zy~O!`bz{GV#ZsKwym@Xv#~bB?-zmt{Uh0v1Lr^{b0$I=IcpG=z;W5n#GHUxA-HGjf z2NXVgd@!Igsf;6yAZB;M|yr zG_x8}!`lDuo$YP@P;l(Hn>hn${H7Md5Tnr#RUAd{b#QJ%Me2Qk6|S4_EJ*wmcL3jH zC%1|c{Sa8HJtbat^g*zz8Ra%c`{u8~0t<&w*$%Bn&aD?eh0i={3rQgf<2sP-XQYhv z+fE*U7=C7a?liOHJ7H`H{+X<^`g9R#1Yt%j+_Us}z)E* z5qF?aQJtc586)wqpoqzS96mERWg&Uu9q|V@5%bkqg83#$s!BazL(rsX)Lx2|chEHvQY?b?Q+SkQ!jUEp}6_ zK#=^|z}SMXha39>q{C+2fNwjmjB=U&9zH)*`SmdIqt_#c!a9D)Y}YBjnj94%<_<)L zKb#Zr3Kc0`Y5WD@gEj2rWxZ?^1z#Os2A%E#FEft+@h^-!&c zN;wvZyd6mn*FSop?`(@SoAu)kEy$)U^9aKHA7$4a*W|LaC5Da&q5>isr56D~kWQ>r z0YRDwL?t3sq<4tDBE7ex0i*~>l^#1Cr9)^cMY@6zF#_Md34!1__uTvabACCJyzlPJ z?Ck8!GtW9Dm#c!r|D+BGT2+|jWib{E5y_kDY7@$zUUyx0d_dGqunS-Z!`XrrpOAKu zEHcP3V#XMrQV8f#KVaot8pcT%e+Y|eq+G!GZ{2wya4O}3-CvE`sJ4$z7q5L4rHO9l zsXkPAFV261K2UF9nI}oR58V!MveMVRN!jghZPRbs4u^7jB4-SF_ih(n8PB7!-ulXG zU!D+H|DFmp_BD34E640a?*p2T-*F#)=G@)I-W82pOiK2N*xVIdvh$fHr;HIe0PGs2shi z)AOEtg8HdJ3-ZhLXFZI0kaElRX}93T0x3&Zw2e#Qx|}aYr?Z4Sga_jF914$u?O7&w zj*ITTPfo*c!R^cav~{Dn9MLnyOTYhONM7_zol%CzrR7#ePK9xh7Q|C6q||zxCMXEB z{zf8Z98^76X+YznAPukN#j;)Ft9^&rqInaTFRC)ne9)8XsB#!Lu3P0bn`43a6J>RR z->5|~K;y?VMohUDv=}Rufvek0b_UM-6&XyP2m;Zw|Mho|f`dCm?4Dg*_}*Za^_X?P z{$I#Nv17x;t2g&C-`#|%gWsTPf!u`Y0V2s*V!D522Muw}G#ZYl{FaT$KjtgQ@`_)I z2`Ge6Oz&m8^5n%NG3UmdiMYP%v0rr0jCWUkV0dse#laI@-fJXKKQ|(yZdPK^{??nJ z;Z(}+!wWg|52HtAY#p3DQ@jWwdAU~HJvI$ZDt0uHl*}UKl}s2Wp@-Yq&yrKRk<1a- z_la&b@q(WefU;4^PaiFKUAq_|HY_U|IEIRdG!}vFU}BMRp7WFHSy`-@JcZ(c0~rZ z7nJ?JCg5y{s-YQ1n+>~;jA$MJ@hdY;wVc%bdhwc2#s}yRF9OXUOuqY?6LXqx-Lr_my1_hjXkNBK)O zEv5swV#fsYSU7_+BCpZv=UFIZ^dH17YX{+);KkkVnt9&5lKGTl|B$dWC)$&Yvx7}p zUgSPx$!Z)p%o~jt^$|d#v|+^M$iEKUOf_9yEKnjs5w;U@PN-i%5;`lp?vVq@L*-nT z?|OH@t_kT!;$CgaFV8jFk&I%{IfAmfu2sLFMl0sduOE&Sw0Hlapr|;W!&-JO$^OHu z9Vdg3xybHmFd$zseQTcqNu;rP5htdI_MJ4O7jL=PL&XbdcLYX;_E%45mD=V6Gg~D# z)6>WS{5^eAE*c}k^mMgvQR*0o8!vf%ya}8fsA(`pB>t7Iewo6nf~tmbzi5BE&o4hKCe?~?{`st4z%V*F3VP_F?i^~Qtn4i4dD z8WD8z^>_8n@ZsDPY```e*lUJ}sVu)0-Vt%FeR+E^vrGI0&)AaWI0fj1S~L^u=F_xO z%675G2?%vd>_@Cwsa#eUDL{R_K? zWl|amTBuH#tCH1XK>5R$a>H|2<4`>^A=4$2i%nB^pbE}wA9x@<;5DNxQ3U_Sxr6{C}v zTQnl0jr8FQlFoj9y+ie%SUPHExfJg%=k zLhGsy+67?YU==+^up{zzVEhyk82Qle8;o_+K>1>qW8*0#Y6aGrV}TD8XhWS&5lY`J zA9du5N_p*mBnkz*o@Vz>dey#`&wfs4wJvVPJY$HMDOjvRi3ycef0!*bNFpv<{7&A4 z!O+&xe8D?>nK)CKt2%5G=3eIxkX)w>i&rYP5{gO_&2|WB`Inh3o74*?DjNctd+SBBMaYs6vcpH;nEwA zLJw+~yq3mHF+^0$cyO1MK*csLUBpbm zMf;W4C(`*TE)(O(EKI`7fb#!C2%vG*V<+ivdF~juh6tfp9Dsk-buj9j95l*(aZ%dc z=wmyaWw%*lttD86HC78#d#ZDf2n^$%x7DZ2PDcWplgGHA7C;Wj`DU(Z7~$uRza~$j zwqRTfzsqXvoeuFE5gu5n9!1}Noru(gUlImGt(p;hk2C0~w^=!OZijnJK2q>OIVl6?6|Q4o!2NItLwaTzKOW>=|)art&^{DQvkc7E7v=9m!KiZNBK zNgnz9M?UgGm{JC7l;+`%{4E|OQx{5=R|pktxQ`Yul-^gs>pYUb$~Y_K;&sqOIe9mp z9?*$#h-mj^qKBZu^Xr0XiQjwnm~f;OzWc53Z^d_uO@}gkvzwPKw$#I$g!FZ}YKar3 z_?5-|1th5L59S)G;%>@LpmMv?u+NCREi`BlI4NNAH@D#yYAufLNVg)i6<5Ze~Wa zPp4Y0r9Dpv#f9-N`tEL@JbKiC@bk^!z$OcJFt(jFHY0_J(@Et%k7knQl$G(j8z#s9%lKXOBB19f1X|5(+zhX**!V9Zx?`4&(BlvxtxJ^p0@rzdayyM}-;7 zJ5}B{I&gBhAT7H!DK;Sqho!AsI8$8moUW7LpkYB=cJY}7h~YikYI0{I9M#P015Jdk zJP=7;*W!VlLU#;@-5#Z^_F$l$go3`h@en890%DS+t zQj^bj*wck1a;XX&lzvMVK2&`u%x!-wIHK7zSB&o%==61<+dJgu{5^Z?&Ws?Kg0OFz*39rLm|` zkBEh~#sX5nvg+9|-b{DmfSI&Hm>k7UEppWd7Yj@NcIUlZ@?w};Cse*q?CQ+jpD>tr z6y=EdT~~AW`m;0pK6KV5vVIr6#sP=Urq{LWj>HjjF3^?pyH9+tQ$nE+Dm)P26De+ zr=q7WGwEb_nl1`R9e^_j>&H9NW=LB5ht z|2gn{JFL$A1)cS9%Cvi8p^VJ9No=?(RI$bZ|90z<-@4{1|L(!6it^EB2J%%J> zPkhr(Vfu&IW6Hc=Z1&oMLcT7?S01m}BXQ}x1&fd3r849|N`Ag$qU9@(;;7~%7QgnYl8cq; z28kZ(Fx$i4fs;u&4px1RhNjb5ffsbL4WCvkZUafPhm3rZDh{(%-70!LrAo$GlS<+o7060kcSs6t1Rbc~U$D>cX! zgaMgIFsCsYE0@{MlP(o^`A&y0B%C_uV)I35jkI{1 z*iD#rTIm{WuWv(hT~&2d+!5)d&BI28x+zwOz?&7lVT6vkt)gPu-F_TKf*zaOkF@ti1gP-wn|PNu1vpOZYKN-rKBZdH!#`JG>zVt9ox zRiwS!JGf|K%09{cZnXu7Kb>HU=aN12-z6@Dc^HnE3rW#~0%MNsa!Q+AP9YV(3it{T zmI^+cG{Bec*_KBFkC^Z9`hGFFWhnY&SRDuYa}6P|ufEO!SoIGEYnXAsdH#6aiS2dB zCzE`q%n_XjGk8u6AY!LI>cJP7twW3yYWO$n{37hpa~eoFU9DL7Y5wX@ad^v!Om{wy zL(lJzA5R~)&#ku|7$*898PeUDna`JBJPvOGo~&*>IOAcoyHc}H(iZQNKX}6fw+?Z4 zmYmvyIZNx(Oek8cD3P=mGc2lc6~7ueulJi<-U>`jzW~`k<2n$XJi+1GquHY!B>>Z= z7438W^y=oww{Lm1dn4-XQ_nN4-rKRdxnDv^)nG%pN_^Ykkz=BN(@9(e1joy<=?>ap!1_@0MQ)F{8Vv zdp<~W+)R~T8uiW<6)AQlxs>Tg(PH6~gT;$&Srxk6Y?dQtLdZcZaF7{}gI=ne&s7I3 zy~|Bccu-|4?WNpYCOx2t>4Epqd?_8R@0#7C_>QoUpEQvzlv8DMc9}QCYInyaO7tGN(iwbNNv`c)TcN2k6PW8ZjcVlKevk_{INVImHLgxHO+00KZX58RmYwE> z(1MlX!?1X*L^Use9x1%bCmGmuKhC7`Gw~93{~1^oWAE};p+i$QLU&*7xe@9RJ|_M%oEH6Ex7UR_C^ihsp)nros3U^c6HxRwt)(7};#gA>A2VSp)S?_!XHsp| zFS1Cvi{U-TZ7-ODSP8iQDZVM;sf?97wxc79kw5E%XR~FzfaA0bNA|bM91&kxKJcZ^ zX0h<0bamRia;x-J@m%yd+m-LN)$VnU9A{08V3n}a+&RnS1*_?hg}iKw2;x~G{VZI2 zmWlnAQ|H$FkUP^IsGyh56m!T$^wC2&L$+8*b$(d#*G9dCk=#6M|Gxa)>Or_)uvqz7 zPpeRb`mZBJUdxAW!6U|5oL z%7yhzt8Pc(36)6;799sSTXQs5O^p18zY`20R2DvRs%!~p2`=+4Iv`Z_rkfL7?I8H* zM{N!M8P3_Jo@g~a){94HEGj27&w3vyX)jCiBK8FONk3J!EP5xAS?~)pk=bnMwMBdS+W`Dx1f2pxVdi8gFlZRQ#j|B(^%Q-`OA-0m#4;l zU?FakZ7q`N_lT$-G*kg;k%+z^ICJ#uehZ+1ZNwFKwr) z6P35MmcF>-S_UiCn^RCFI4O3KRo}WW~o4MeK2(lJ*>k=)_#xW{;r&@xDfU9VUgcbc7E527?LePH zJv@`MhIm$WA_^lxJa=x|WWp!uN|w%^T4XPmV7uME99yPdudRXI=fiyuIr?=6eXo!d zOl{PYX*qAtd%JMO&sbwzOeLB7!Y+K_k||+I=n+m8?xHp_FmbpgtoeI}$@F~T5>{MD z4^$~|EVMT^#E|1q3U_%M^EU??QrgK9=4dl2dfH}|w5vJ&o~_BU{O+H%`@&vWI6G(- z(AjAlXn{=>&mX@L%EG1L8>fR4EsH`b*1JBdzb+%izximL-voSw{{t&>jK|FwKX>qZ z!h-mnw+T5*ZWoQ_+-YZ%qG>{hzk|C=k>eFn__X{0V`7rl?r}|&A!OI@4%F8Yt>{9d z7G2LZ&+Tunh!MZ9c1v?=)9ecgyooBtk!fhomhRrDWrMHRfX%|f@6vSFD|(Ad6%Wxo zxwb^jtBTKl11Y>$c2w&eEL?Kr%iDtK^|U1(Zn@5JBC2SmaIxaO5Zaq({=zF{tmZrN5Fx=$Zgk1*-Q9a)YWXxtyMf0x&2q=DoglO*6X7s#ir>~O z(c(QJduyq)lC?gV=a;2fNrgwghTsG1#ftC~F8zgxg-^t_=b4Ajj>biYUP5ZnU5i-k zU~D)t_zoFCPoLuUr4bpBj6I{D?0WL@tufwoi6oLwUE=VIG{#wuG-t6(l_F5A;FgJ_11CGa;t+TzFD@WNTXb@ zPSyvNwX%)tB!Y0pU^=&WX|zfkuXolXYGGh%Q9OAVO?)V7g zQ}s`sSm54ZW$6BLiJzZi?DAEF9j3!p{BZv36}N){F(n*$oi9hf5p0IMKd>^D2ez*; z>YPQ)*mHYD5z8b)DumKhemKV4S$&?7d>*3E0x71g`z)XAz$`N=wQv@X)RB=ZmV zYnENztJ1YFx2(8Lm4=cF2rEu@J6qegkUZSFQ1ozZ2P1uO5FG>nxpzXqd69}-q&Pj1 zIiaMW!2BheL!yib(wbW^XoBV8>IA>=+WioiLRyo5tYW?n6;v2Kx99Ndr7RYOPSwr! zb%QM;$uIYEPo8!v&;?~vk032NY{b>!#jZ}(x+Oc)=D|3R5K}AV@3ouuG~ZC=^}fh? z8SmZSIlFA!aTsheKK&f^ju`fkb#v-vEp3-SR5ne`zFKl_qWL^k8ACiou!+x4 zYI7R+=3aa|eM%^G3H;~Ty{pE=D>ImBP?)HU+2fVup6gm(k-aq9uX%RfQrlb)-|QdP znVsvCy+m!PAK3Fqtr0uG*~Q@HiR~~_Jiw)e;7!zY6f#g$a_8TYsx<7E=TVeG0ykB%-#pI8&J7!<2E0Tl{PA3GMI;|no#emUW$01)E!%tZ!6MhxT}(p z+b}J&58Q84!zV?DVL|AX9Yjv0-acieq^IJV+l)>z`>a!Ow_L{-ui4`+a?yVhHflS1 z@ew+EzP_4$b`h~+DLC19^Mgn?9Bzns`OD+0`mNg+EEX8zYUol9!xR$lE+s;NV zEh41U!Ezn(YJW@G5axE89{lAdm|7sitL;c{sHhxnEdI&~E+`q|5h52-EVYP&OFczS z-$ysUW)ZGIwqTFwWlF((*=qa~YV}KViMq+fQlmE37xd09DBm%g4ExO0j2+U3FNlP^ z{GvH^JQN-N%HVzPz#;)TJ)hkyAkhCN@h9O<-ZhY75JPlBCEklW^-I1(?F$J{&n=p) z=s6_dv=rl7KE*~frDNTWyBjJZqf>VeL<_Q&i8`63%v1@_MfI7~&h%gJ8+OpPt3NC7 zU=knovHwf0`LHRmK@ZXVy7!`gs13q4>_#+TUH$_+$fPo?e|qBRa?`JKhM(1dYsz!% zeBO7dJZVCfjj#xBX$p5yOG^|QR-F<{bsV*wFq_7!_m9AyHv(C8c&G8B6_R^AaQE$Na;6 zMMMY&mf`saYy)N8NJP9`21v8D01|xYx&?%Y*EbRbz?0noq>0gF#PVgvx6qV{r_N+|E8WO%O_)nBRbx`s^9A#&We-`O>2te!QVMQSR`Bq`L} zSP)%$C0SuKabD+Z&(LB>1S3CpSG>(}gPdcYYW3|*4zhW@>YAk`VNSg^lKv@;nSj?2APtyOaY$%PI3i_zsx?xw}nez0Re4BA?tzst*D`aOC$-7)g6U%2n7xi^!;SKa%0A6(zAYz_b(%oE5BgHdd=O(fv zg7b|GN?NCs>(TUm5gpAOHl;J7Y2BmQ6X8?wy_@m!a_(9^C%Z3-7W{U9GTZs+-I>l^ z<6#RSqsQo!f4mK+!D@0B#vl2l=})ZX%d^MCZ`RqHARI=Mrs+R_c|dIB8_U%Fl^+%1 z=H<;d(9Jie*$+0$>=)VWKbNghH#ndy*%L9*eyUGey!lv)?-G7^VYy~0%w_EgBrz4x`r>@KG5@l7H6ocRPq_k^c8ANoUy z|09t=4x0CWZyRjm{!5e3-O^;85dy%F=rK>8Jc$pS_vju2rX#ZXxAdT-ZFkiyfnR^o zR;KIF=2KU&AWgvCl~5=?$?~8HIm=>cR{ZF90OkoXnNjom{fh=GNlT@r65f`DA{<&} zBF&-FeJ^ri;rz0OCBnIm9p#~y!U#r1+E+20{pj=)K4;t2bX2a?mJiK$J;;YJ%;E| z*yIgSol4ehwgz+|Q8Tn?3ow5W2jZf6<))A?SssWJigZ!Dm|ktosgzb8LXUQHl5m*o z@Sdhkt-Z}=(Q+JjE*j`uj4w31yQFTz_S|i1#QLd-`46j4jV^Ol5l6n1Zk>ypZ`5*g z>v1>GY0U|5deYMWWKsD;wDb|AqQl4q{r=5UOQD@06tczJekqj%-=5oj#R`l}#|T+T zJ)t~pjJZ>}V|2PJ;H2KO{Tm6)G66EC`AP9l);8}2zQJs`AYTX#mKz&gShuN#h0cMV z=@QDCjv$223-pxM6x>v3E6pWZCW~3Hylmx~JCp>%sg;T!7))_#by!oT23_PxQX`_Z z_qEN`6`~L9`Kf!Z1`Trw8g;U!xo6bby)+Pn(+1NSy;h~oVfhKaZ!Z)c!Xm-+vM(mT z@zpqwKEP4q9~A-ABbjSv1vv^1cKuV3s6o_)v!tE9QkD->wINr)!N0tNF|Fp9JfIfc z;FQEwK9#Q=Apn5}BIcIRE>=Y2rmCu@cdgr{KU6El@^fhQL}=2W>#S+U@-jZARCJhK znYp6ZLwy=}0`GFj|+NsC!K%yfHtZ!mK zqJ*G6orqoNuXUNum^^%|toa%1bGrm^so*7REz3CEeY*;h=l zuS*ai6Ja(d<6RodL^^YI`lg~D#f$JnbjnTme=h{nh{hi1y?1Xn649AfY3$uNuVa^M ztIF{sb@$;qx9=C8Sia@~E6F!-sK~c^Xph|z{R@>HRNd;hZ5Th+=FuO1g3Y7f1j=g# z^>c%2){F|EaZZ8>*h!=!8CWExtnV&LJJ!rbP%J!s?O~S$tKbIg(b#9f-y8q<&soey-iBZ z0PlwZo^xV(+Q4Q#7y%%4Ki%A6nZ=4+?1mdJq^>~0nd{OCEmS(JN**s#68`W^iUU0; zsq@&LGP(~)a{xwHcTp$v;7G>%2>S@u8Z3)M18x~BZi5RnSQtyoc%Qu-&+ImHTYOZt zRwcZxU(Kw^KXOVlKG}$ok5gY|)o>~22}&CU<^M^@7-p88z_ngKEqZ4YCI?BEhQ^&W zh?3DDfnQ1)A~ws^AL(_y$_91rYGDA@fWs{5k&wJ@_E05%a%$LYX=c2%ND8D<`=9kg zTi@ziPGm=S$`dq!F>@Ra0|-PaO@Kr@K>T@FVrGfQ2W}>qd|2M_*F0Yu8~!{sAe9B1 znFc6h5>&)*)nXi&K}(A0`8YnDSIlQ_dDg@lHP@|O*41`7?>hdy$i_D?(4e8!x^a>u zUIY#(2Yi%M+X!+^C?D9cRZF$s;_Q!W`hXZHis^y-F>u3)-_D0c{o@q#Sbfkt;S}8s zL#1y2kYxr3jk)$6q_XLx1tM_RZzfV7V?;Yi0-;jrzn z15$6Wp;y(PdXWANWeuZqdo+3NDJwuG*4aF`YaeNW`oz|i8$<$WZdCFE)mO7hA3mCx z(o*j|z-#@4w_UyjDm)F7*O*BeVGl{?v6fU0p-P|m@w@JjBtg9lpeW?-Ta{0`NZ$|c zFA_N{|AErjOWLG?DUe#EKsDU7dh;_YOKMW>2cRPl$ile{#LVf$E^?C%9jBC(kFx>o zHUt}nzBNKH4W!%$Pt2_%IVnbJyd!V#?-PXulw1JiWvg>rYmvv{zzkSeGUZ82N}tJE z-_79gz?)D2*T&p&@gI#z|* zd|_l=x5icuXgQJBB!-s(yzxCifTt*t@9m%k1x?DxViJI~!pp5i0W^1+wk-&^2rwCV z^zCClRr>m7oeI`>XG9Q!!?3c+qj}zm$k9>2M?RyVh zSZ&H8jR0s2G%Ex?JmEzmCIPG)NV=0k?s~)8BT!Wyev3^wSqw<+*!J=Q>AGJtlW%lM z6@)o9HtM@?K}2Jj@|*Sf?RsM0+F6bl=ZGl?Dn2-xrS?~sk^2u=Y&5z$eU2B+B__rtX@jSjGRQGA zd6EU}Y{H~x$D{y(aa3H+T+NVAn&7%u7l#?Bm`T^yOI(JPa{{<|^R)fo1E^JlHCPv~ z(Sx2!02}fzxLsV(NUZ6=k_ce+UbXQh26+muG(`?{B8!+3{p`ltgJo9HqZ001d!Y0$&^XqMl%4>o|V3PMwP?RmO~vv-;p(@ z%V0{?@{+;^dX|J;vVlFJW??XjGD*f}4EB+3rmUlk*6(KT%=^_X?&2xzO|j&d$zKQ=8g065)I79|R5j$_QZEn^^%M z8j^&EXb4S(P6gto8M^qJSDYMQ@z7kf<}o0p5=S-YFWAkZ^Ip8)Y+&N`Y{L`NtCan+xLW>=Va*QUEf{Nz>vHHo}K6 zfK4#Sr|l3{(yX`9DjzswS^vv-kM3|?#s=<`;(aM(>Z5NV$VGa$A4;m$7b!&AQjRo5 zs}9lw>LJO&Cf+7#csp#YO=J90^mSZ0KJ(np?Ev(|_`^L+cS&+0viUVGMt&PDb+&4t>62pm3(*n&qEDW^xW~?w9JGtF)JWsQJ%CVojc++guH%8x z;Lo2gM3fYyg?o`>&n*wm@6S`iyng-qW_Wb-qno^f5uv7GWkyYNdF4)|BU010ygw); z+W7PX)J*GhC9VYAjQG13mO6I>Z16T2WVmQvF>Q(IAjKCLMiXy3y?G4 z^F0v)m%$kdVcb6~SRtB!oGjRyIMXr!Wg{ap_4OKn|8;zrLK5SXF^Pvw<6+SQL! zGi^e@CC9Y4*oCV(8$2Jsj>z5)EmVr-I7pRUU0Ctl{x znqLA@O|P)5%hr=e$oARJ#6V_HJaYgrtF%X(I71_nlEwISp9^42eEHcwkGUAi^q!Ik+QT*)vuPjlzLbW$u8@R*Z zO(G$3$;DQCqHK=Gntbu;riljK?3o9R53ZS^17dKEDh4x7J8N-%Do`W+*M0eqn{olG z%+PW@VRriEUCZO`IqFE@$ks;e0Hq^N`mqWkQ}jS5NPkJQ_uQl&1c`!`Jy5v*Ugl0k zLkYOA#vf)oVLOMm9*Ys3v$JR(y^D47^A`lCxz$Z^jQP%*GX@j%W)>$px%G=a9!N(D z-j5hCnki5%N-*C@Y6wD~MBQEAw~qB$_Od6RYQKcgFD8JKBlqzG$6>D-v0LW1ur>#c zL7m9UJorD%;`tULn*~E8Afe^~qLu7CrU_+KN2rP;B zrTq5}ch4V;A~i&;*$VK9Y_PhRco1p0mVf>%karAl`otf}8Adhp8o$f{p(J8z^w<2v zDZ%rRFS+C`?wJZn0lJuLUnkfaWl+;Iss7(b7o#=hS}Q;EWt7^j>2&o2#LtC04xi<< zT}-sH8mKw`>NF)oMOw_{=1{Im*R$E9hsWcL((Xl-fsQM2>z7IBN*lvC*$bhlt35 zM;<3A2MP{Oeag*^xPIWdyNLq&(BcCWdZ-r+t(q-|ofT~~((lzmz_thGh{P1zEZYKq z*;71v<}-M?(F(WiHqxg$5gbmRyQxvQs6bfL8lJ*=55}*$lb~W-#iydDus|MB`3`5Q zd#qcb<}9es27o3@g62573XpWVR@DKOBJ_GC?OOEObXXpZzI$E*Qmh!apfty{hYRhX zR+0FZ_Y__t*H$_<&2Qmt>s(XcYKqI@1d{q3>R^Nzn2Hpt@x-E&fDd7CFhY_TOep3T zg8_CY^P=XcvIWi<{9VT=g;%-KA;??NTpHPj8qcsiKc8~~5q^S12T&E$a;ud+E@HUK ze@KNt#KvkYVl>gF@7T#G$d{y8A<0Cr6i4{r7-AqcZDoIM->%`U?;-}4+)K;_^X>&f z5iMl^&YsI5Px2Vy#&wAr*EE?(o&V=EN7y`sg8(x6A1{+KNO57tku$bEXgrSxM zo2ME96!3XMF&%42BAHs$!5qO0Qjb_bugS}pZhsl)O!ikamW85qw}hi8EdtFX?yCiD#>{Z z+l&hvmJNwm1O5)$Zn<*rDUv)CMwYfEBMkF(45jew4$D*B_d0YYHX*)>c43}92aBbc zmZT`T0Ct440m4yG#PO-4wwYRKXKj&=hX&6nbCuH6{36~Pt!GDylr3^JlkCw|kS$P& z0Et4a~5zEv#X*-B)XG)t+tQqW&n z;yNHyLGG;;QcZ<(3e%GRfXBrZ}hvaBkHk=}y9V8yzM)A&MWjOTd0tD?p16K8brGkW!o9gGi7BM1dE z(eJ&`cF)3JPzDG6DvuXr>59K(o`Fp;%0mEArEN8X<8F0@92(Le6nSNp6f0YyzE#NB zGESN4W$wo@(tT9L|H@2a9d|o3(L2!^N-};WPUoZ4rgla+6R@Z0*F%cRFh(PCKgb+yz_35E{I@*dWt ztJeF(l=-EYwuLpUMX*IRtfx2(6yke4R3fV5kf5Timt!)(KGrMajnz8|%4hKm@_!K9 zHJGI+neo^A?_89S^=J3Tr-h{_&20m~w&|sWnW*xKxt5LhE{yQ$+hakN$OrwFoc8V5 zUK2Fo*2@nPgx7P!^AnDA^+ovC7mnESo&=ftVgLB~>B>Z+N=7B2AVDs4Da_-nnUJ|Q zqBE;9&7T%~JvUML@snJ=%N^d=sPa^#FbX7fK5c?6S{|r=6#{(j{yAQYDx3LAyX4@C z3`X(t=JcJn0XmW5Nn=8T^cmT_AO;2ceH9>r%|=#9)pNd~yU03`c?d?_u5nwJ?!2cV zT}!k`qFT%Rt(Pyh)Dh(-`h1lLzAdWw1lKDP-*B=jb}a`hCw(r>-f0$(Hy0yadA@EM ziS7IO>9Cqg&qbTk!rUG}Mq+fB73l8PY?MU0<1f<5CgfE~B zX@uA;g_pcrUZ1|lnw-Ot*T#9k>>02Z6*;cAoc8bewAc_9>=bUPXPCbM-lW9l%mkSM zS0wV+@1B`PP+~^!XxMwD*7NG3u2Z+_?7H*lW&5_5YUej9p4!!%26UD(~aJG7LHNp^7Ka4kHG@ zNR5O-+=N20;mY9}VNK)W$?!W`6oobkq1J9^!f3%~`-}~$M(aE+HK+JPArJc3B^M0? z!F&AZMF`XgIYLdwr%>lDMOiyQ!PMa6+wDxCc`bs>n*gF4b40j$SdZ0SGc-)|c#Bz* zCcoyGGPt_NAgcAHW!j`wS_B_?*SX5@boJ_Kl7SU5L<}53SN_fs9XKygG+={P7;r@o zU<<^h5AutIAMc3(#r^_tt+zk3{Kgyp8q9z}mY(y!N|_iKU)~vL6irGPtT{@t9eS}_ z1u)9lmYcU|HWP3=r!mZRZdRrAy_=_L2tmF$mBBsETVFu`iz)5jJnX4&CRv?T=V_|h zH&fJfwh0@8^$63jEC}ehT72a+^5dEa$R9xPy&3e<)&#{+G7^x8KpT3ja#&j;@-c_s zuTJ+&_NSy?L5lwbUjeE6uKf+3=QX1xEbOF@U{BTZm|L`yW0bY8>6}2SFiskRIr@() zQ9t3y&YGc6tlI;dZJI!~=abx;f@e>^&vf}?a5w~vX`d7~#;+tYVy1L9(ZqnbWD+%3 zXxyXB_qkblHme<&T=x-b~r(&{jDo(f1j$Df%2}pwmO*Vxxu&-ti ze^GAfG&KTTqaYQlk4PLJLrEGc_SZ+~dXM?M4#H5YiiUt|n!!vo#Fmrwh^!<~y|v|* z)lCHoZ<>@7P0?DTwmd|4zFH#C|A)&w|BVWl>nW(AZFAK`CB!joO*K zNmF=zDV`J<$9u?UaP^MuU(2ULB7uJY4}{16K=+30qLoWNB)!1UP9oc*{1_73IV9sB zHUJEin$Mgv%1@IJ`BR?&=Y}j)+$~apY|Q}ISc%t14}1YxNZ^R2vr9p{#$(+pkhAkt z;{kE|Iyb!1wijOCKL?j#HO^C!j1MeQW^!{tvFSl&k635(#`@zw))5!oL#vjrNYDRq zMZGIT7Qkv^_8^_Ht5|@>D#4M_`1uCi_+#t&}(Nj5Ms159loGn)9s2NDwpMVmB z5dUy^EuUGF6{=hNL;|Jz0umDw?TtTDaO+IZ>netHHVr|;EQFQlF|ka5*cQOcKiB9m z-dH1~`=^uEe_TB41pVs)MNUxMzp0pHr_h0!%)qOE8yqFw lLC=zhqoLkoqNp4i8 zwkF5*1r?J49Y!m^y@HC1m{O)Y|0CJ&dNC|74m=@Cp&M#l`?6jC&%5OQ{q^LDn?vqZ zJMW&r|EODXMw8kIq@V`#Vo!mHwLHM8-v0hoYD1S{dof&$zr>bGx^x~0z!09za_e{vmbtA72lR6Kq@{zmixavl>ka(PmS{RV3N`>+gXs`2Y`!^&3w0b}(P512jd{ z`y%-B+F&VS8R3Iyj-guX9HS`~?!;ueCKllD1c83+%D^q2QrMt;iUe0hvg$cEE?*`n z5v8E4>>y!P<@cq?Gd;sPb3=iJq2Vl8i?Qbd&QnD>|28u`r*NPJf38L-{NtL`gsl>GMO(G z$Ij{^__zHENL`c@WEWC#;DIK?WJzfAY!e;)V=A2TCj!y_l2TvUy#9Bt!dO zJQ4UJS0(NpxJqH~UrZYw3`W4iEWGM}*DRajh}Q&7Uf8{rxVQH?Jp)^sxKB~=LKI=* zbl^AsthE1UFEywG7ETj?{<^(lo-;q)OL-VcF8@U{w}5)By$l5?)1lB!uUr4$gRzZ! zDi@ToSA^U@F#*D$P@YUrdP{CCTOet}FzT7;kcwVzBqNac^)1Q5IaH)C-Yf}*G!-I^4e zCOrie8|3%D{XS$9<|>3@XqtFzygq<5(Uku6|06wmJw7#J29&{9_f=I@+)P}%L}h#% z8wqQ32Jjkd2yhy3t^OSFt zqumcTVH)9p^FvMy_puVTvD1H}>5*SWWUsMiua?T+OH@cQYM>TXR}^?6KFml0K&Q^Q z9*K_P^P$aWo6KMaJtjgXW;|EJ3`tiIyJaVqBijhx8j zHSmu&mmn(vDT~2O(GG45xHfDxT8_Vm2ZAbrHBs^uWKsV&H>(82{I|PwS92|H)NrwXvJ29So?4{@Gh8X1 z?@)xXQ!*SWP)>W(bJF40o=JZ7Mm@?P)%fQ*!DjVx6V=5FfZ_7Z|GrhAE&$)8GkGmzts(@GAX43lx2lSNyu5?FMvEYzEOE!q z)&1rFb(&d~8FWB`y&=;iArN#_rVf4K&pjCMV?z97(~J>-q=*UTL6tZFr`_=Lejr|VSBBwAt=R z@T!seU!aOqp*CbabA3(e(J7 zAYzsAZB1Jnc1-d+|L{GeIM9bSGwx6e&tGeGu~kI@$Mo`lbjsxr>ick0(?-Pc63pd) zc16?#aI4X2Oj^*{|A$0C!cUX0=$z(N%9z)nYZO`iecf)b4YrtP;0XnlKVVQb1*5=p z=g)gv|EoarGQktFw7Q{F))x+N!{?f&Ngw{zG*^dh4rL(i3z-C+Eoq4VkLj;Z5~U0r zsKdlrVKOHPMI7ttmi4dxxfhWn53U=L2iJ{=R=qg7mRW^t3Ikf=|L3&I;6C8?_T-aE zj#h2Y0`{27F|NLA^|Cy!bI^4z%;<&T1C#t~_)nBMtPVP`14evs99q8s3-Ew$cLp6^ z8x|b3bQdI7uj(=!nW~PmBhMJ3=OOAT z{C-uFkn_US99*n2BKqws-YMSx>*rOR{Q9xhp2@+@6oiI7Pub}Iv<&=CeSk06AdtTW zU?@0@xZB(jfUBK?rc7RPGfY~pWfbYpMa^-VdRQ^r@OMPUo~7gh$but@8J7Tmv!3E7 z$>=|X5$6Gt*4k7rl^yYcz2i7n;WXHb6)fEkQrR+SRRi~6Q!DY`4g#joZoVUAlDzGVAfU`$nhv;i#5PBR zrFZj_YWl)fv7AtkaY_wvV@a@mV6(}&i=AC6`i(030i0^LRT&OavI}5ZAR|o1spI4z z?{h1Ye89tLz)ue~moVzsL_x*Pt65+8CIr!1+e4a^7?4G>EPmTdNG8Z7WE zGys~a6tNJwsNxO?hK5U^eY37YK*9>5@VT-KfQTV(bGC%>ZL5iH8{^lqQ@RWKS+VR! zMn=>x+?ZOGx`6pXV+~#d`cl@g9@9-|&pe21WsccjHBoaNer1Rr9c zOLt^SKfIdc3<20*Oa@>95Vq1Su`;kxvArWNVOh}?0eY#lAHJa~>O#^*jT3@?mEZ3B zd&T!)AE@|#Kj!y7Sz0Z+mJBuM#28Ge4A+->S1karw0^)x{^-@3DZvUqL`)M0E}p}LZE5xy)J016vJsYivm+P699 z_Qa=uHoNdD`m%x7Hl@F-R3~cn?1ajP%9s0fg9Kv1=nj2iErXHH4XDcado*B)JbFrYz-9h1`1qBl?s3oGN* z2KG@Y(5PvUhRVo5^dkhfMB@rUTL?aKuNBpVH`~1DKekddu zrGX2V*S!;PL#L6ffc649?`4~utYpwRhpe<6PKB?_Qd zxr0ex-U5Tx%G0%`+Xw0?S5k<8rtZDr5!roG1{k*wL%DwQMSnR(wsN;k!RvfH%h9l7 zOu`%43D@HyDSyn59oE}wGXYe|o8N!iSbR*tkNs=j$n*r@!Xilk=Lf;XW{NmT_mDm~ zj^LFZi_xTS%oj~9unD@mzy4k{Ri1_H4EYe-$V(eF0vj_lb%$yG5^PZqt-2Rmz(@|< zbxtAu*WGG1oHFS>`cWrS=O*ojy#ajEZA0K9g5Z2EwWjU zURUrwKe`m~?vPV(kSH*W%uhEkcrWmW05+?7!C>s!SmpfemHl`D7~FU}sGn0@8R-L8 zC|KU+WnA~j1sH!=EPq&aTO=Sc8GCxrj0~|NO$U_#vs**JkgXz35I}bUsM6|J7T?(G znm3^UWutST@L9XbOKJ&#;eqpwtU4e+Z!|F>HGfEnV%&1D;ivEpH@ac*rt9HUpU zMw7>J8`Ga)ZlK2bGX?pJzvXbK>OJ}b-HIU-^_G%QO@1Lc>Xs-d_#s!rp?l)oq{K

3QUZ7=!E;a1?*5xWkgm*>+o7@dZGJ7^7`|rj8)!2L zj~M>p&F|=AxuHaka;Ncy-T^3lL1_PY)RZLJylY5B{gw=a>A33NNWdW|Zve=af{>4b z#5YKOJ+{I1!y&`wOgBPQD~72)!cOyt{U2Xn9TsKxMJpqyfPv(vprEt}Sad7WDJ6|T z#{d#CG^j`m(nv@*NDeWS2m;dG-7w@ZzyNd4V0?bRd!OqcJ_Ag>=j^lhT5GTKo}QpD z{~D!gy*z%ki-7+O~McAbGFvf8j9)IfpePr^VjPP%{2W_3D zN-5gLtP%y<_k;}DW+!|Ih)!&D_kMpfPaB+b8@ZB<+kPA6=K-UHEHNy_-U-#GXwLUf z=}G1NDZc!9dy;=DJr4fp7C%JkT>n{Wl}c!Is+XStAF^<{h!ez_1Y6>BaDRTBYqFi& z;rF3x_BcHVjiR^)46!s49<^zZC34nQsf8B)xZ8$&|M8993GjIG_v+~6?A$fIjp++9 z+_y9xBV@pViy{Qr-AskY{GV2$XF2YId;~-Ta3NWs;%VoWazDkn+b;fPz0}|L3H1~s zApak&@(kto{GgQJ2O^Jm%$>V{e*M6>t-(L_# ze4YmEsXy&~>!g!0&;H?Ju$`M7MYpD{on3ER;qNXouS)HSM)B9H!9r1af5(+(X=SmW z%UzJo_2y7SaSMLvPC}Gmw_5ShcfjtY@@YdNT4{_J06bLr%bWe3%SOlz1 zo?rGV_QXe4PW=A=DMDmXNa;%+2v*>wDKfayTk7pf^P5AYgMhVCZJS@+bIXm^T&V9) zS~lQ>C;+D6*^I;O<+5q@xKF$z;d%Ht*UmP~UO-rwqbLXLfqUyg_j@F8&%iCT-zEiJ z&mE_*(@Q-(p~9g^H_dk7r`n>DDa@7U>z@r#|GF}W-{LHUwnZ*|=~MaTDfhgAANI3V zw~T8f9-H4jv7Zil)=o`L)oXS;6@FlS@qg#0zc`S=3(xbPIEkfcrNM6L)kv}3HO}Cd z2u{N-DKwSr!ET780WD+=5i49X+KK@OAe}7etB{L-IPp`QX~a_;oEbZU+jBW6@dezw zo~>ZzM|DtuT}+s}r^x`y7Pm9v-$g;E^tbE;ZZcXOd_Ix0;Ov2aS|Y#gck{OHLtb-y@qeWUVG8)M z0`vdly){t7!~AC z__Ghgf`7oz+MTJqYJ}^fRgzd={GS2%#di(DAWPKEobT+|9ACd4)YUcRn*+B=Ck>DL zc@2XE`su&50NgjCB)N7n|AV-5?6se%o$Ofb<@iuz)ff0BpFSCPlFeVl<@Xi+sSQfO zqC=9lfTl4mRb4ik24|IT)<0YNxP*D3M=a3t`{@A+qJK^tn!zm!ngOwsDKK~lalJ;i z^nLMy*zeoR{1hVo%V6(Qe&-BiuqV9Nm0Vf9Y}-=qb_Dx4fV|5Jjq_yzH=jCTn4r@~ z{%=AwNCNRMw^N@9WF5?9YjwzFsU~r~5!GXYT<(c~_m1K&UQR_c_g|nz!kZvEq!CJZ zU3!6S?lYowt8i{(!#Y?0PV*O1@azNp*}sF;fCGHYk2x9tB1KW$Et%TcCp9G6<>!B& z3M!^7FY7A5bW@yau`YZ@wWbDFB9GM|2!L1YhTgF zA0;BhS$>Lj=UJ2-BzBHcv+)$uS$;qDk2&x6@BwTAb~y`wly@^tF^ypYH|TEp!5I{A z*X_TadfZoLyOV!(2^@4lSpfVX$&Sx)?->;X_Ea`@b#?XD|8^;J;FbKzY`-QJ1oYHx z{Cmyxnba2g6Ecn%&DJHKrC46vo$8A&mi&b>!RI)zPr87;-|B$kcj;ZFIH@mR2*a*X z8!~?zT+YTp@j)MtlPvmoPWmOJWD@~DiBq3$Br?*Me(q&hnD*|-ZFX>g#J_2vpVazf zb8#;{5O;(Io}6rJEHL!66SdeU_hY`pjfeVwT)Ib5xJAHM-()SJ}x%_bMH-{}Aq6 zU-%-avh7!2{CCX%BPS~M?_e--Sc98yh~4}i<(*>n>{0j_d(S;j^-0je0XOk8nwH9* zwl|0TxpfCF-PckQIB=W3^uNmz^E0P|W2W}+y&mOK1qZ?Y$$3vr`vPaH6VP~+iF;S25YY*$TEMs| z9<%(}0Q?iy{T&7P=g1-ZTpv=b9Q5^N9Whz0tUQ^O?q|03fg7ut4r_Jn(du{8`jJ5f zW>4bT@!tp0fzKn$s1RlfRCRSCkOVw@|H&>Gg-wFB<2=v%mzb5@@N|wTvZvZper}%@ z1xI}S-RwO179ii%yPkXbyh&+6w_ow&tHoFv8BM-_M+GdH3XcOax=uizQ^5I8r#$QM zw^A(xI;$%M2MNi=4SaBS;TfA6ns?4h*pE&yt==`SDczrk9?#D=xgRzy5Z?gji@YOK zBK5=&RHL11%J#Q7v0L*aHAf?{0|=x&dSUNG;*%4rt+|wD9p3x1Ob3XIasg{lswm%= zq|*bf2q!AD1fC9n}EQ06eZuniX7jrPw;qf+1DeK z1do_e6k_4=h7=z^^-;<#M&AcxEOd`WRbj+*>>+w~VhlSMKOa4m&bUdJh_5Uv{^+up zANEM$UTpb7eDuitxWIk%SN|b_S7~wE@eVu~!B_9nD}=b;or0XB^8o)n(Dq0PF(NK< z#phS5MF&?OG!0!$No*##Y~8UGAAh~btC*dw?KSoYZMi!5h1-5vwQ}kmwMDSdZo!CE zB|BSRtGJNx!6JR^JwrcTr@8RmogPKwBn4Fn9zMx;a4xi>kf9`v%^RV^jjs+%sgc8_ zBUa}3joW{~O75%F4CuR$8U2X5%wM(t(TZDeR%B1k=X{(VYCGIJSDg-7xzpe{+aRgL! zrYVzWqvmgTh}uJLzZLx?8slF3e&p<&(K-She}tL9a(wQjbrA;NF0@b~{`G8#v2@!ox4TCE6{e9(*2UChlQ>8H`yk{%Gx?+3~56;dTL& zM+pbL=`C8>wd2eQE`>Q&D#WvwEiv)VNYnLdj8*ab&8zskWLz4=&ZOg!gV8fy=s-N6@(vY9-0V0dgdp53H| z^d==jsxf8NmyDOtr%^ho;vq{t>1QRU&^1_uW$BPnTG>BX@R;lP(a)LvIlZ=cXeh^= zY_EEEPmfCJLq|tP&FEbLlTNxGG^#L~d{)eZ|0XVCPr~-sf1=(HKB=pw1#feLe*Gny zYH@jatA8`MY?p1*2M+P)%r%CnK6 zSN0ofKp$$H;96R;eB9SJ=U%;defYJ%UkL$%9UKpgtr)W((EW13+>OL;H80xp6vUEp zZ#w}?Gg^}_7yW#_YFnYlyr*xj#vNR+Hwc$}y|ajKPlZlbA0MoY@00)awL7IDNPl>m ze5{et6473$c(J~Id3F9r-q1V|?Y~C@Hzc*3qNe@$QdgUYFMk$3x$J0ClSbM*_xfrB zRE^@D4)vuMrC%WNU|@AFDEzqMy!nl4Az)M4-vhIIXg%hKC828tkz{Jt6ZTrMAU089LwF-CE z_&e83y)ST|Lr#e7)m^gix#n<(=QE$>dpO210xZe%x+vsf?`kr6?6{|njb;d)F$0VY3Xvp!owoxAcGe0Nx zLr|^vCE=%qHzr~;Whf2m$pult?uTDNsPd^;A0MG7pqGSpJ|MPQ*uL-+G3~bfc%l@Q z@0NQTGy74o#X+)L>WZ`)0Y8r`vG8UCS@rXl`EwhdqV$kR9b4Zq#|c-)SS5(Gv=|xF zbUY{iX8X75`55z6fr^gEPAIS4Wy;C>QaApG#4+o|ajCH+Zmrkgz8{Ol<|=cb0x zeM)$PIsSFhTFHQ3DM-P?iDKTXvib*4NK`ADnPLa~wV{@ktA+k+@R^`ekbus#%Eb!~ zjpCov=e!DO0mHN0(_Tl#i^i?%@muygFrvwAysE>kRxmnX{0Tgx?;EETgb(x7Z8Hw7 zMrd2zB~WDGG~~9Om8zWdBDNTJTF5PC*}g2AXaFfP0(Y)w7@nIt&V5;dOLCjr!I4y9OZ{e|Jt)GIUED$+bQlwx@GIZPY*B?}LKj;ZHobjp;i+WOMa8~hf-5twj?po

PG%LXOGG1kRG)CxK9SJ*5k#?-uM;wXlGb64YR(DOi!A z^KJ~T%sC7hmev!oF>thkOJzwD9YycFrP70EUI-e2j=CZ;!=(rJGSlPD5{`G$$s+Cu z>~ua_i*p>md;Ey*8eXZ_pLw4o+hBYv9;q@d1%>99yDxN2cuLXu%G^CkWgjl!nf)Ci z2M~t96le=2E~z)-KNYwSId~yfe2-ot;JpH+n+{${u*fl7*S49l6y);ZTiH3!W1-dM ztV=Vv^qz4!xm?UU|iw?!!Be{Dq8f zl-jog<(aMMITm?sQZGjBm(?7xOC8FYEw`}c(^Oq^2USl|D`&b2B+UMH5v+^AOu&7) z843*pFniy_YC+;z$nCi4e|wr;G5Zc@4Xq29iU*7XnFAs4(i%`!((L}+6JEy~w zh|ZlP8}K`)%Sxw~_qHYg?W`kTDRUyI}eS`qHX48>x>*{E3%! zO+LRph+NtUaaxEQkR_o4=rQVHt9kc#?9l{v=rNTp`OfawXX13hkm{5W!wAi^C3#Fvc+4@;lu9Z@{9uFXF z)WzwNA&+y^M-kJn5->U{v9akBmv)>#!d}Wi_%N$QYinf_3R-WRey0R*edN7DIe&Zs z>Y~k8yBg(Vx!L)nhvAh8JqiMCm6j;}Y3+!H%bZkiujBdsT0K7(6mBMx!7?hj#uXuf z`_&>H9{Lg$lJ@eJ!sk;3BhZ?$i$Xg}+AjjRgFq0Nrv(38=WQq?EPEZ6&9VlQAV2qeR;gUu#V9@#^Orb)|?#~6<`a)-q&tXin*5n5)fBPPj# z7U(g@vZLLRj*EP=A12$TL1B!-Ro7iG_oXjsTRCf7u*F^(2h2C=i8-sRUuG3aN?(-* zEMcSe^cU(DGm`zeNPWF03oCh*9E$`j9Pf)>PF4HrV-E(n9af)W+8wK3P+P%2=TR^eM65GcWwWDQ?n!=6kfcx#|V0OIQKklzB zXXjreeOuVsuA%Q*l5_R+*^9T?WGh8Gs)j~x9}Q(n`&K{~%XPBuIxc7HADL|+5<{*_ z)aUA2*AYyaVAHFKi*IUXxcda0lyK>`7_Qdit;m`vQVm+z;%7S3l^FB9r5fnUv1`|3 zsv%DIeOUoiuZnlWX(5d9kEmx@fUJi{Aw`-+Mouf8t0^IZ$8H5}+#~okmz?Sjh&uQd zX#APl(G;5^|JJ}|h{YX$h?T8G5$S!iv0%>Tmp`KJ8gkZsPhz!y_$R1w)5EXr(NFIo zDg-iOZ*n95&m}XfDk|2bZSmzTEzeK?GZV z&1Iu5O;46QMcD1|wZLMcx0l|Smx9{Y9``=r z8{#cr;ko3?o|3e#5AKB@DPNm#MwX}NBoWnOn}MM~9Z9)3`~;#=tRohME5o|(N2so8 zmq4nFQX+=_*bAt$OJG5?JB-M=x^2hSRoTom zMk6x;WfHfZxFDj)H%G2A{k@qv!9wJ4>XNtkjUzlAU*@%!J=;xV?rfoFK~=uySXO73 z&J?I)*Y$W_fb!v$doSPX!j@7az%oMWc8FlBxV7({r{>Oi8AFJ_b$g@3{rJArDdp$$ z05Mz}jkU+BLr|dHU)h;WUol(k{9HN?TZu+42_WIgIIPj%)AM4=RXbr>el_KzLSc`h z$k>jNHlrghwbcpL^pAh*=0>5&Qzr=KL5(YRR4Wv6^=UAH~PTj_q{3 z!?+#M@frWul6y`Ng{y77ES+Oxx`pqKt(2dTM0XY*IF?H(-^rS2^}6i#ACzy7KS+Fn zk7V)ZO`G)w34ewSAGtL)Dp7=Pw%`h*ntH(^uHc3inlW<^y(Cj1DTJ9RO(Gy4DQ0uZ zLYo*LJe{q1OsUy$k$3V`>;uy$B)nA7hBji)O!4z1&>t5zd1ufQ?jjv+M9bq@v_xU* z1$v{$XU_ja!jo+|*CC9tSgp=S>}tjG+ncSrv?)FG5_a{5o!TPsI*3TkvB5)aYz)2# zj)tA(17yfsFS>W;Bg8k{B;B}uXQ^imz?34u3fUIr#R;9H7c5Fj4#w8qs2CztD_w?L zzm%)M)ujHn~h${ieWzUU2JzLq%zY?}4D|9WK;)#c7 zBZ2VV%u=Aye0@OYm3$}l$8r9Ay@%?Wh7>)qi+uv$L9I6?x)mgAlHr;kKClP)ytdP4 zCic%|n;7mfzm&@p+mE5&(S7NQhOkQ4Q9j3ZzJ-*CJEZ&vq1|@uZ?^Zv>HH!2Ra<^z zu$5eg`Do)|H_RHV zh1V+G&6%C{jzoQh+48Fjv$VMvN6XRU&aEx1qIBsHRq9`Bgyj1tkJwZ9EQgEz>{UrU zD&t8dAE@d?6A-aaG6ljHq(4(D&Vc&mSq=%nQH`HjkX#(7IbJkUF}UURK++?ZQhxG& z!oJ)$rVH2jd&%GD@i1f?hX91sJ55S?%POaZ095%!)0Z3uqj%pI#a1%C`0T;y!S^Y> zqjXqhcK1O!Ih7;A@3r^&;E} z_Whb-`8TS|iZw>9TZkJNiBqbm*XO50CA4hX_F*ry!1EEP+qNgHoNjPzpz-TPfgimu zyr6mrLx4IWSe;sQ-GLVJ3Pe(3E(9n=4;c5afs%vcA=-{(_TKU0JJu@l_1|U0@ya?iBQWuwY`Eq}R-#hLvgwR$nC`j_wgj8f5XP`6bH!@qsYAcJU9bc%d8d*82!jiQV* z0)stFpD6xeyIr88avPMBAyCVyR)C(<4mJAB$s*^O3X_H2N5`gHHW3h)ZnyO~ZK>$a zKnA!Jzt(!5MulWffvlPRG$B(Zy^&)ZP~E!@@kXlmy6&BSMKJEXF4QwpS2k+fN;kN- zTGUZ$M15=MQQhu=Rcd^K!-T7&3M3KshJPvL<7%&pVhPBpXTAcOX?2yUJ-UB{q~r$> z`HUlhGBWzE__Z9~Fp{G$&uQNDI6rxruyG7M?azs%q;S(($juMo=ZUeQA^)(Bgnqri zW9J{0C+6{(nrv?Zdu;4DY9ra{|GnVZ*VT;VpnHs1uF}5Q5|pkwLUHLB(Ir7W6m|E7 z9G;X20Lk*wKFe(kRr)LA*DgMbo~Ly`+;o_FOEvuA#-mWj{)Y1W>I1W(e(f$5)yr-o zotynmN@oT&tbtaAI+}I=8|Iq4^KiT5Z&fzja49$aMn&GSD?9(#s!eI|@4WG++J2)l z(IAecP3_5x_@a+bVv;Qi)&mJiG96)HS;@Us8c%-wQIVT)cb1G7AEPx5+qecE=x;pH zdGe6$7Ls^bP1J^MH!jlkc0nz~DbvhoWxwt$EknO$W@?1Yr-|k0tb9(To?M?$373IR zf%U2vmqZQ=DqlB0kd|bmpF3+XOK)$P}yckf@;tx)>2NWO%QMgZo?Mx9(1Pk zN2X}6Kv1T!iIN%D97nr=nk>*$cojev^N|W@C;Xjd*;s#ZKqy2*%QRIYTNQnP<86_; zrO36u!s#Yjcb(GLXJA)ZvsLaBgNbBZM;OG}_kRJ*jtQl^a_|~t^On*|0D!{7l|4kW z@j)FH9x=~Htm4#_0}hnyG=O5oI4CA|NwvbAZF2??rt>f^<*f<8ku67_`CMy3PINiL z=hs12Nqh4#O@rU)Y=zH3D7|*bgK3P_$Fhk_suQm0v4L!75E`#r|6;)`VW$F{KYJYK zUCAvyR!_O1)yVrRtN)pu%Xku(zXyVPTF$4cO73|`xU;i^FLw7aQEA77a$@7T*+-Kg zZWJ{iQ@m7DkD7rno~ptiE7N#%2YLeFJavxtCDq1Ju;qle_QwFBGpWx25z}S*P`$%t zE?gsPc&-L}1jSK{mIwrRU2DrIguuJPP_Bs>$8+PO!~MfD(i8O=KYemP-gf{m83vW) zrQ^6$`V#w2HTx<8J+2=1@~(dEX16Tonz_TWVa0j{wXVgNZ=*>M6~R5yzKsYIq&OQt zc$%?2GWI6V^YbI0if?lCo%cM=^UBkZt`}&9M|Ve{A_C~qSKi}ky^r1?Cjf5Bc8r~A z3@ieCyjOq|`WDDfTk7QiGCkfuGRV#?-MT6aK-Vq+La!H;-~S3}O^QUlb95i5JVBNY zQ!R-|xpXA2^2qM=4mW9^!CzV3Gl{rU7_5~H~r` zgl_XcqSGcDJ}L?ik@NE6iV+wQdFTPH;OhchR&@Y8sPB{LlVb6hLhtceudGms7#$xi zZY;nks)#+HSn8MNgp#%pv1VwCqNVh=9*v%%cXN zektRkHMN3AF9Lp3cS&NGy(u3{b1cXaDy8e+sPwCHa{9g0Ij}%z9?>+8GmNz?jb?pc zke=SuF1RgMiR#y019X}r(=S43IV1VU(Tqy-!+1hO<`{d>if4mvkm-HYZs=@cLww49 zZaf+!e|3Z`&*BDX5)|TOvSpNJL1AsWFp4!h3|<1t&jwEwupXtuaE@X%ciq^8v0xbbwq4A1~|gM+%$=6cR0qv zxB+;5FY1r)f*xMX3!lv9X}U!sC1igz%PjKc0{398%LJkxHC-BA65m>X(7(M@>24yB z3lFB8LZ8+&RjLmW9-9Jr4u^ zOjvrZK;=*V2Oce{tC}P5f{G45iXA7W(*3 zj6+sg5>O#(xDi*5fE*F&mvvouf0hhKLo7kcIuETtdZ}6ZsbO%_ta@S)b~-zl?ie8Y;B)ot`~kF-8Da z6A-c503ED>Oj816qjrhl>z4kv3*S5P>3`0nE~~mycZd|$;#-rwYWht`4`H!4tl7G` znTtKftdD?DvEt8n9DL?8saFP+w4pCe*Uf7%W9t_EIhxgd!%ub(R`Q+Em^n`OA?G;7f7 zfCr4@lf)&QaQRCUfRf?LU{*|>BeuUq6C?bYzfWKfb~Pk=UBDE9d9R&+rG|Dfs$`~e z8K=8~<`C@By8DPaqkf?Tn^zjegUs1?k1p~tNLeTqplPvt zO<6Qyr#{n7;qdGOy*`=theSHB{79J>65Njq_jPrLq^>$Ge)h2d3y{!DC7dql%|ED& zEkJz<2>E0flx+jYN7snyIL)7u5rvW_mrl50m{z|W%j3#TsSm9i+~PEhtip?{)o6!8 z9BYz^|E@a854&wi9R&|kI+vg&Gn7>rVR1PO{xqR;2!D_jA4Ue8?zU>9P4j3yBoII@)hUAE$JreX~s zSL2rb7?*v}tdbM@o>SD$?*QhWfbeF8f_*-w28$5^iDuhNoV0C#D{+<-N}%<0Z`eHF z30(^MBhIbV>{XzT*ZMPo0ufL{eLRzZKiIBgZ@~l`#=P3HC!y%EG|`*q1U`uDA^3J9Ktt+$Ii9~v5VK?t7WWZ+Ve zd`I9mGZxbgzPo^-(^p7++XUKJQ}3fk@JWzeO=}M4IKMUzDiWNA!l&L*Ug!?i<$d~$s}6q(y}~_c;9Wsx=`x~0>Ia)sB0=Kh zi*14{W(Wp0f?%`gobxt9w%@HlLmI8FawcXIfc9vwK`uR0se75lR=oPsmvF5h?-XBv zEtqZx9jQJH8MdAY= z`{D7N)%JF-u172SWwb!rNHcy5jV_Q|Q-caraF${&Y}tSlxz1j?m>oNu9)GstY$f%m zS$6JdQiS|UWj)BY8Y%^F(yASV7y`N|SOeO4tDa}C7c9g(w*xL4_ih;|G@L`SuHNnF zlnW>1+ZbzAxn@*z(3ii8BgM%<{4HS~!&0OPyV>h{Aha^&CnLV8G15v3y9qM@I?N1%zXOwk1c*e)a zhw?A3<6V^@@)r~v`8Ps;JxWx9jHiS=Ecwv*yZEa?h_Z5u6TL`L+&TpOLDvNO`=7qJ zpl#o$CXY8s)W{FIY2Ia8=Etn*pPpmZYie?9jt|m!3YaZNs5~ma-}Gn5eNMD1us{3J zTl}py6wyaMn{x`;5dJBe=+MD$6)bBmf`=7zQ^mun29wqm5gn-!TDM+*Eg9#|;c?B5 z^wR2&tf_RlDuzXJXUDk#RAlMTUV4Y|yjo>Ygo3!)U?9vlk zKxcBPMpw))SOv;X%d~Pf-$d%^nRkAHF?R~3_3NeXR0)Ef#EML*w3!C`_B66c*;}hL zvN(tHvfO|lvP*h}%RC;5h4p7GP!agAtR)_dCyw*j#nW2X;-xKYFvVItCz@Ax%0=X{ zhRAe~xDIq}q;k48@5D6>FfrQG*fd-oPoYOaY?0*)2kO~R)2>_EiS#+<`w_hVBU6F*Vs4)3|j9?VBc*LNB5OwGx=^QwsspKW*vX^(V7y)iE6eZw?tIZBY~Q1 ziIvYbN9`$!j({q^({4Cvo&SRU@|O*gQG|Amjss;k&HvTKe#PS%JO^LVP{5z0`G5R$ zTO@kplb_GlgOGcqi4K4kc+7ZAt|=niIX%wd9>F6k&_qnDZNjM@)O(BcJr1sOw|b~U zDO@r(7ZVe!7mEdu!~s~GBWO6j(qgoDoC6t~9?#}7XRCbaR<%>ty*^#B6r8PO%(j(% z*GJ^AZZ@;*!x-RyX=SlEu}pnsB-hpVLLdGSuDc|rR0Zo%3VloJW_OrmpJ%TFwd~h! zac#w|COuHZ~iuIS1M_NCk=&6Mes zP|{l*s$U^8OgIrlu5MD~peuM4r1Tfl!!iND{o!V?>?sO8(71H!*_QmYA8WfsvCrxB zTWX4Wplu{Jod6@xbYR?%x2`Ab?<%o0eX`l!Y09^(YN7L+Shrnn|8)@7g za9wT}SkHm$6j8pHy&4S?kjlgyK+w9S@{H!i6$>;dzoIdRk!U2gI$p8m7GJvPg#FH^ zg!`I(KCnAoMM4z?q$bHdR|in)teH3xo>LLYVyWWYZd0PNkeRk}CG7jw3G*_eTt8CN z5S}9#vBz4?@gY~fL2EtmSY;!i3KI(`-S#+@)+6nSqv1{6zT8*5M}7HjTr}+as=<$^1~6e zhaSfnD8#xe=Q+Q~=+&@0Nev-pqz1*;9F-se5Ze-V$M`fgMS`WJOD3R2UuLQ`hg34B zD+wCFwa8_Ntjoe5SG`3O&?i02$4>t%>Dq3%O;?O!5 z%65u4M>Q$1U7GO}Yo?U1rp!&1`VU^_V{4{bVx!e>2;+(>)QjcV-oaD+jMlW(iOdEN zS@EumhpJFnoZPFH-Bbbv+9+|qM#n`Nq_WpEg@(>EqS1j}ASYR?c(nkX;N^1NL+JB- zhlIL!7`KMFk&1T*MjD6u=nT+b`^J$zo*br>BS7);wqno$j0j0HR`c`Ja zWr$O@m-JRRFu73IR(7-HV&pccq^4ud7BajzbtEfq=7e$^qAUU`*rM}?SLA&y45KFizaQ?_26772_x~~SH&{b3#xW*^fS4_O|Bh_Cfy07xwOu! zcPjz5lwSy`#$I=t4M`Jgt~)>q0-L2XkXOvi9E56U#T`5I*0Z|!;WKP`@FZ<=xf@M* z9ly1J6-~8l#e$n6Cou->!mmyq6yBPY6_%cH+{`JBg#DWZAEsP{->p#4~fHbKc8wZtgnU;tlv_8D8#ZO55tPgg{kE9mwjaWM;(!|*i)v= z$TaGXJ#yG;W*)(rKw;kH;|+Ng@Yd`g8GM^o*B*W*9GUFf>VNHjnC?*p=hvVXy*{L) zA~_Q@1S;4-%>-cU74B6rml+9jLDe09+(#DVC?WUR>zV!SFu_Ufm)}3!NW1R9QMK`K z1ZaM8Vkn}2)VAJ3_ksBk#|%$y(S5ONE#NM?Z1xlrelXVRS?rp9F>stxPkP)t-6 z$ThnIkZjDWxtqDWqL;p{Wv{ZXoE14d!{O2)goZz6j|~!#SYBgoli6!$)yygGmN7*# z*=WmbDR-NX+9>KcwabmkhMh{G+v9ueOAFfjJMEV~67B>5qgwTXZllI)fTW+o^Pp}| z(>xA9CT$B7C5b3cIN(TH(IrfhDcuqB`D$-h?K5MOkXwyz>Z6yPg9(=u3a{}tg_BR0 zDhs`4sWYAZ9+I|Bd7w}TX!FS3CYl!&RC+F(-_pb;2<$uhk6eM&%%z)9QKNAnoO{%_ zjq&X7NnCec+sn&xWxjCWM$R)4`H8ITHfSS$Aj8Y@{Z*sil3J5V67}(3_5Sq>6mbF5 zZ>dwsM^peC=a~}1fAOE}-qKUfC^Iiz@W^2R#|D4!-;s1XEJ zyoF_#d+B8p5~(XPxC6vKCf}gImMGA^R(tx=`SE)Q&}NLx$)CXjk+dr1_9rC+ect#T zr%{_G#$B_JVypOhR+A0D)i>3hfRu*h)cBAF<6Gp+xRYcs_3Z0Ar^qZWQ*kI+4RCl= zjSK_#k$m&1`bF)j+ z8Qy%qX_$=j+^}=p$hMd|L+pa~H)06<0o)i4ppEcKD5-D2Ri{Zw3@Ni0!)ueY6hAM^ zp|VaZJwjAQ?dU@B)I~9{nVkR9PCtx*j!pTIukn2Ow^sEH@8vkWK1JVPgw|M30 zh3wU~h3+pWGBBeUQt?T3-K!V$;VuX?GO;)$pH_-{GFW#RR<5<3azR9-s0wI#(c% zPm-Y`Dou_9L2_d6f44iuI!Fc9ia)Z7y4Dd;<-q6Qi3KSC1siZS`}x;dihanL=G zv$u&v$(!>LgdWV78=8IOyz{h`^&Hyc^u>a_JP!GY3ovJx<}>{|VE#OCW5T+F1vrFL zhzH-!CXF-Tym%_>l>DW@q*9F9*@^OMMH#IOhGUG1Z}BBUH7Ky_YilTuZD3LAPl9H| z8^=5Spjuqj)GE7i>$ppinz+DUf^kJGuC>tX20RmEP+*a+9ph zMI5?k<=5op72&pmSI(lt>GKIs?nxk8Kv=tmkASIe{4{hA*!#C%tAS0O1G7f;p z!|dEX0kJ@gq#}==;77I(%UG*Kz(3Dy@H&Z}=c$&#Q$l1`)MD}{XD;xJb~-D(U0HUy z(3G5DRVZl}-x&UeJo1ArFieePC?^+3Ia-s&6n_Y%*cl4Cfz-@0cF;ok%G#-HJ=elW z4F|!Rm7r&AS{DV?Z@#O}{)96C8TSFKX@O(Inat$+Pkci4vhUg^)IGaq*T!=9vGk{B znmZ~3I(Gd0NZD?`b`4HN84Lf2$4wMr`Qq{uh^Grlq%ujZ=2hKAPY0e*B9F4nBi`k4 zU>&bt!#-FU%6p|V5xgTPSHwP5+RNq}&oxzyxl%R(ch-Wtg-@t`{U$3+y`QI5_&JKu z)p47#!^|dBBBl0E#~$EB2ZB)^kL9jEelI?g*?lJd|PTyp?Q^l)`~>ukGPmj8I;nl{st{Dw_k z)x=&Sk4x+=HkFMyFOF8DAFS_J>cpVu)Rk&z=5H?_Dc#vFt46VZ41~-fi2IZmW$Txg zV^-V%Nf^HFysEq%5q^$zz*!1@mq2tmaBj+O=N0I^5P4ZA8+L61B0k{)UDKHS#F@cW zk>_=WVv?HOKZs3Ps;<^~tt2wj2cMAxCj{>T-^;+x8F<3cs&U$>;YEJr>(o=aj{8RP zE4e#FgJ{bMDwo{Z+Y^OWLqIX5B6mf;N+1>ir-%8IwHH7mu&bzxw!4f<=YhAP(uIv^CX~!h&UL^x*S@=axe}TS zgvL^!*bG#Z$3)^RaaO=Z9IUI@taRY*Hu_4tqZFt1RaV1pY_`8AKL+~@{N&oMsck9g z6ed^SXXO>*SX##E6^yX9F|DPjVO6brjxs{=z(;2FidgW4%qRbNszT+Chw+aw{&w!@ zI8IQ=(Ha>qQ(Vbx2u=9&yR*~;fPQmVF&|-x*emE?CD4m$)*1H5t9utAN_s#2KX^slOH}OqvuQ9scIXbO;w^T zO=49pa4kL?v^U1%ZOH}mL}eX?9JQM9jN|Mhv}xqnmZT7?qrd{hh<6A?-8p(B} zW*^!)CrZ{TjZOFc9$g@2`SOsE#0D*o5=?!Wks^P~Q~}L7ac_C9MdLcV|45nFxun*& zGZhP&ksm-mZnR&|*|Ntm$69aa73i6CH6L#Rh3eamAd>Taz#>e29U5|TeG!-~oq8Pk z89SFp`y4Clyw5sEsy&MU2~;uS%*2s4YJ1FCRd1ye`Kl@cQ%33;{v&H0B9qu7q=Obm zxkdI`p;Q+}6h~j(0lSEFsX@rb^a?xbb#MRBjWGR7>n39V@U?M?;Bl8((-D$Z_ub|p z@gnp^m-^yF&jaz+f9v~C8vg!rBJV3e%wHluDBbw^F2l@;Wrj<+UBHfIOY%=PgU9`L z-mk#W6q|;)E67n`1>DcUZ}&F7UTU3Z2lf#|!)mXy*I&~~xox%)x7LdXSPvi3@LDJ0 zngD4Ayo}lK?XjhaBcvm4n??*|1|a6gjt+f2)p|7Kg4oel9nlcfGcQa-%} z9~HR&BtujjaYN{dDtDHLCL>Q+n~9B>&SWLzu7jv`^)Yqmlj;*cD-N`po%7 z>ewSJRrSW(A^MPn<+eG1?}EejBxlC`XgP9cKZ8Ds0PimUS)AJ1gO~-E%PGx37&(xh zcf+-xpG8UgE1*nYv((i-)Qs2a_tn3m>^_3Fo2ZDm;1!r@UlmZ}B*E$Jh}iBUy@8Va zv&oA@pfBz2rI(rn3Pg&tq+L1OX6$1S|1yg+Ac2`ioi|As z<5BJ~nhju-DYI}G-A=Wr<0CLL9~F}IX9Trp)7?z~19tNAA2NPj+|SyC zPl29;@n}xr)}3dwIFlS!FmC3EEF=W3ceb}x`{kEoWmhWY57pIVJ5)>oivsH2q1lL&8kgEF6A@b#n=*ZQOk*uuG*3XY}YDiAPQy zT(kw(d=@P^wlY)tdA%7~1;>=N>pBiuYjqS`kD@a%5ARoJAM8?Sq?z~eK!M3;mqO)F z&t_J_!|!INFNEQuU8C}VKM#L$YBA~0b`M=;tNL#(02j@kEs)#5`?n8vJ~CjNK2Xwq zkj6teJsv}ub-rd|lTHOj2f4(uvVonZAZln;7rAOUd6ZhIZ5#n`!b-=g&=m`~hdidV z@o!0^pwVyBhP#e9^7Qxh84|ry`Jlt)nW=>ZoveH8**wW?dru8Z&R0(gA#UhG;lU$8 zUVA{dwi+?;z5om#ltu7(R&qL8s*Gch1%^N;BM+_{7T&7)XedC+6OzwHkbUc%4gxB! zPAm~Ft}I{wo>KkE!_*2BK17^W!rIlHQZ%6L&W2zS*$V8!)zk@@%@0&v!^M?MX{RL6ppIG> zuc{z4fnz2t+=xKIzONg}C~fAnRpjty`0Nct)+g zS%+?FQC4djYYA?{2+i^dF{ytap$j~LJIdS*mKx1!hQw%!j5g_}3TQ~h zv4$d*W(9hn?>$pqit|V4wkczh(G(53M9~mzi3(yhmF);knN?%YZm>dir@wh7U+EpH zFjlDLE@Cp^s8%)}p61%&lKU}iGJXH2hvQuFQ;)F|(dae)4urDosk^LA>mOq;2r@cm zh$`4t2tw&Ja271pdMkXqZ1PsLSlcLY`%-$Yk}>VM2TX~R{f(tc6T>p+H*)uZ_io?M z`<}icy|ia#os6r<4kgme>kEa{aM^1$Ea)wEr+%?wr^w9?_WTQqpI7&!LjKE~*F&EZyBnO)_; z2F~Dz^WRL6zQXDQv+06@cw@Zl?$CGD{ly>A_t`n#m%A*rFv{3vF?q6C$;w>KWDJpR z#gwL&1+Bdz~XHuL;eB8O%mt(ona%9bC5S?~Q5Cv_E54 zSrd9V@G16A63RY4Ly{>_(g(h_>~vX|gGQr+XX4*B zh;_!RqOpb7qx)&=)CX>Atw-{$%g&h{$NBC@=9*vDk8|HnyPZT&UuMJ& z&BNq0?thL~w(Hno#{gg4o*%-{EEf8Xv~ARu@iY$YmIBqevgkRGcU@7T59}5@2Mx8j zX4v%i{k|~{XIzx=>ogYfM&7SR38Tw9_We}Yok2~i8W8|ni*jR}{j14zfca+JQk*#4 zdKXkWNq?EfY`%57Yq_PvW{fM3nO*Rhah3tQkmF4gQWN)Q6Yn{rOY`AV*lpW&DEza% zl_;8FBFl8NrpPu3Go1di7=Hb}ORJAzvSjc@Nm=3S{fjhtBxoJ00mr@qy*{@_#sbIFab8=DS`i#%F}_c zfuNeJQh58v43Xz(6t#LeL8A z7%p34pO?vF(f-Z~2YZF(Jnf{RD+9NHXF&v|Y%>UY_;Q|8HpB-I)sh_8ZfCl{o=)G3 zZ~uq0H;;$14c~{Ej5JaRWs8#ROZMF;*<~lopeMU*8Dy+Or4`8%qRB3dvMsG4gc|X7J@BP02_)N{*GxvQh=XIXPd7Q@yalz&q_ze=Rf$e@{c$gps zxq;&K{3b1hse;8flR-E?`0Q0e*O60K6AfYxG_LC@ zNX#yEYx9shSNNlzlmGcZuQZ@K3DNOJRscN8er(7NDifG@0%dIS85B}f9e(+~$)&O*fIQtC-R&2##SCY2FLDT0b2y=scu{Z3?~>?YpJKYrL1*1}eSysKqOO zt#F9p8hy_>-fgx5NC(MUoHYGm5ks_~Q)j zD|CN=Ek1iI=ok!1Y=Fs?7ZqMFK2B#sz1Ggzpn zHXTkBTXH$($0IJ!wbgFfQx4e|_S|w7h?t&C*vuXMWpvQ)6@--C5kQGVE7AFK%@)&gT;d ze44Ge62)pJqgq^-V@T{C(XY|mMZ2$op;af!o^Ia+3t5`~s1UohYHVQ)XNHK4+Jc6c z9^zIU$>I-9xKYTOh;SYGr4ZDsgEG{A|hU>sE%M{M?Md z;aBK|tIqwQ_JA%p{^A2>L;;|zo_e2q+E_oA((vD|kH&jd8&J+w8}v5P6xLKvYlrZ# zt6cK+jWGG=rM^UileEhlNpjUgn#!Wm#$izR70LVXnD{&Eb?^4aAS1$0vb@fBo&Nco zTlyT`iY(=N9xv(!*X06%P5%y@6;-o75m1G>l;JqM!E~^q#B&;uc!Lh^&zpj+%bGQ4 z7P{u*@!wsOlg*5r*KX~ri&$fR_<{6LX?K6G`7oupcw#-15^#}m`gkE7w>>Ti)0fH$KV)vy#g>1H|a!zfZtNn_G1#44DR`y@TIa5B* zgPP=jtv4!cApPTi@%y30cu@$z*M>2-ul!2sRW$?d)zCNEHux2Oex2v!L_k=!x)&AP zitGSonL~hWd>(V^>4+rU{V=G9^!Aes1ME(k2LJ@w`yE(xTRWr!S{2UX4Y2CGk9R;w z%qRC<+)jIsI5^Wh6|BEm8yfNHDVfAOyt>hO8D>qk49$O+Sn~}SnYtHpzLU*mQt*Pd zhCQft9vQdz7w~f_UZQ!EfYl{G_LA$_(=D~Vw!wMik1DlwbPx`q2*>0W{y=D z^MZc0q(-}`EMR{;^f!bwT#;m5Q5 zu|y@4Fo3B=6F=9rm4`2u7MTd)bJnl=_(R97Z)}m5-^_jH%6`w3`Tq+8 zB5`4`&(;8B(ZN#xx(r4Q=c45k`|+Nh&_|jYe_j;qus%HahGO~8+YxZQ8fv_r(?$B! zKR<@3sB7uGO(QnEpBhgSh~ZXDPUq$oDnKB^|ox zjfK%yc7hr827&>%N}2{Z<-PA0=F{GeiH5mal===+f}zS1a7=@*w}cprDM^KnfStZ3 z$ZiR08xm%HPwqMN$Vwq&B|kRKCmlMb0f5sqzaL}u7^3~UW^3IpgJ?k2dd7K|? z2qg00MYZbemvNl!(k1#nkR zH8Pb;;pa90Q1-A!`Jpj=~o@aM;B;K{!F5>%bX}^?`Ae)B!-vbt0el^)DHF;2R^|nvwsk_e6B%?X!ml+Q=Z#_?Ai`c;epo}ko zxdTWSnl@H6;$-S#qo2LpPo~QX$-{g!VYYCxd_@FNkn~c6vwi#1h?bW+qpHgIgLyU8 zQ`&N=tKGZyt4KbHg(P3r77MS_Yq_kwX&ut58uD#r@Pe|!_6>*C=>h|e8Isb=^rHvG4Z3Ew zqu+=X30{3m37KO6#&|OOZVME^OQ}vc6;=s%pV_3;yOI-JWf{0dpg8j%_1wJ1#3w3##HNdm zOf^m7C2ew3*0G&IL7ZvcD?t_u*~1!?y%V)hfDp*@aQVW(5z&}m*=*n;FgdbdkJA$; z#Ix!NO*}h^$3HHE82dej?CXF9u?6(Ank_>*E^?$gGv(7&$B1jx2xQt zQG_bc(@=iKz6Ilf+rGm1$-~uA7{lwGkSiaRi3G5m9~^YQP_{{y!A}}b{MasNKihJ_ z`AQfZbG~AnYk_uY<*nYQh=$d%>LYNt2*HB74Iyah_zB27#1peuLB`mBYx+Z!w}4@q z(_4efdqj+9ZOr1Qvu=uWh-c2_!(_tR&GAO6xxqw(kXE!z4W_OL4Pe3)K; zJ%u@WO}uET$Vm+zy9J7rW%%>CfFeZTYoYvi00F)1ba?_cYzyT`F zax{J%ZW@w!wjf7PHT!Pkez)^_$%wdy%r0U77~pufW`=&#tyTyG#N#-8VZI>hJgKe1 z2>36071!%CWz>~keum&RGGmUtrBmVf*+kLf5sL2WbBYFMpepV!8*@Oth(RuU>G&bP zaAHlx()gmefd5^;t{xlVv?PB!ipf~a)Exe@Bzcl8TmTAp4O&5^p+Q5UTVrQR4LM+A zm6_nNvC&Lv&|2@?$W5KFsEqU2J?0wLfd|Q|Z7HB_zt%C93Zk(6UeS?AOE~TFhO(RG zyjBC_%XBJ`BnOt=4XQ;e@pF;*`Pq*r9YIq3La=Jnmn#yND202i$!(JvLskXNy@ql_ zIX=Z}TdblJBO{*OlW%}g9jwQOkc5mf2~iQN9S+MTQi6xPp%?-FcoR@6MN$vSKeU%& zxf}%Q4KN4&C|khE$E48HgVJ=j^}c*H)-#qr%qOc$s~H*|LcXlHbwu`UIVrer3QK3I z@#8`m)>$&YU~a!PM1iK`8c4^nDjYIYBXITyyh)HNc%G57nwBqrZr*U7WB+crv@oyg z{e9~F`^Ys+XXNM2R6iYqZ-fJKq}KpGdZDd>dJE7S5YzSQXi=66pOZapG~Oc)$A+v| zCtv3T@rpB~MFtfLF9z|(;=4QAhuxu2U&!4N zi1NJu(BdlAkWV9`puU_O%rB$x0s!)?BfU2d$qeiB1aQ}9#$&!h!MbdE z9HvIQf!LYv);Esp^M1}C7b&6d(lhdQjMqCj9OSPD^Mv;S{7KAKOLwpH6(P3g{2{L@ z258_8m1qW`m!tAP^Z&CrAib1@HVUE$vQ?C4@Xtj(uxHgA%{3Rj8AHvLk~ z{0>bj8r;oKbIHp4pNDvU^0zw`^!!e9sYw+E3PzMKH8d5zsTJy_fEl=HC=M#`*;XQZ z9nQ91OF*Jy>x+JXYCFH1QSPeqP$?4a)Y?N$Tk?nZ$2n$|Bc}(Yh~uB?o7OtbysR43 z3sM+M@@HjsXOwqGZUm04_g*k7uVMvNTbn(^+(%K;W$=!l7V!Q8Pn8^X9hSd_w?v z^ADN;M6i2wOk$5MK+8PsNV6Qh2;;%e${?Pqst5+5*b+&egKN#%K0`B8#jo8=vx z5zka0@tsej&1c?PH9vt}k|pia`{vsL$`%8v3oh7T@9*YD$A>MYPsmtDd=hbZ0bfSTrA)Cz^{#+jdI)tuH;E4j}Ii?-g=MzUg%s=3e`sdai5MA;A&=HV0Fky zbW7|zj~y_1+%#NNB}6?50iol?5zW-QX>|D!la{a;=3R*_W>r&-Q4nahI5WBhaP&dm zCyv|OfXkRU4%R<4K;ZeFSTosa<(p$WaBCJS^heuej|?aZx4T(Ownz~MG;N;JOf%Z|nSXZRzS`=8c~A0Ex&A4>`Kc~5}{ z0&&L=8eOt_nkK$Qn3nDUxh|Fdjp1zaD9z0p+=5OyC&z7lI2ppi#uWzO%E*V5b4JOj$1F}EPl^O zT`#y@$$B~CC$r|=>{NYgRG$E0}x8QoxMz@kvfC_1|WKA3#4r^|g2v59b6c5WGyR7(`=4v^T z`_~Hn&9A$I$orm9YJJbz+W4gHang2FNbc_z>us=D2<)(M<;T{3@=AAU-&xwfeFfuC zYMY%Z?Cm$XMT}9)=I!4IzBHg2i6@k$Gg2pFh>sajm&AuLt4$CZ4XbN; zND4I+A(;9^G=WN7!v-2bW2ih`j-QiQG701pVnC69AfVydjLW-if~1a>ls@(q!dnCw zZ{x^U+f2;ud-FltwyYUqG}8Id54j>g}>+SanAC;WKL@1o^DMML_fk zJO6b{1ZktnAQ-v!xOJa2Kf4`Rv6enQ449>+s8-g?+)abE{(Wrx)MEgR%Ug0Jac$ij zH!bx)u4FZH9$jZWrIhDkd)5dpB@&O?qxE*!6o9qcn{CX3tBqFOJOz7#VkI>gu9wZ(=SQt&FD0}E?E605uIf-% zS7&er@LETcF;S~fISZ)!Za8!OA{dF3*&q8;UVU&ieP(plflSD8^-#!IE_8j|DaNn= z9kZZgQ=J)%>8zg@I-x`Pp|@`JEU)pPV67Jcr$KVj%Oi7jFNtFs1Zzf8=p-+FaSQ>q z6cJhAIBKb`i$mv400t}5?M5SK2guuep zy4V1R!607lAXLh5Q+;N#dwat~iS|HV(%$)^UWl5krae*CpV%J~E8tb1?5{E!o4Kbh zUpROUqz?V8CCuMnB;|L?@+-_Q)axmAOTR+3$(JmiS4tTR_M`ZS4aNc*sZXxPD^#O% zBe}$9^@h-Y)mU*J)NFrA7H+%AI2&y6BUJn+DFNh_pEev+_iK1#)|+72tT~r}3ziOJ zod@2Ah_K?xJxq^`YJ}ht>liHEO#K5yg;m!s1dA1rs9FI97_vc6-nql|AbGevM`?H_ z6j49v5OzehIJnc&Xa(gh2%nX;$@5DR5wvdCr~dS)H1@%CzcNv58i5&DL(KKM+DVo~ z!9QID>{HJteY)EFLw2x$-SCgGobvb!^`MgN-UcW-J3%(iYW7pS9MGQ+Lj=6p2g)Ww zfb#`O+EG?wjjP@_Mp5sgz1PVOf7PL(N^6e5EZT}yZY6!mUJ8J85RfhzltQnk&#s30 z^b87PII9w@Wk{TfP654!^48LGkEVh_Now+fr7G+x&|GWz8V3FuHD7VzVIrb$R|h1u zJGbW(M8G#xMfaJ(`OgpyzjX%$h|fKZP0l@m&s6I(YlA|z<;EP`IsQF{Ksdr2-y8&w zE7~3Hv67Fx$RDW!6e98tYod+T08C^1_K}7U7sAvkZoi(SDen{5<;u6)ly~18&C8I~ zA(X_OR6>E&MbU5|`pB3SdhuFjX&JXF2917_@RdLL>@&2)7k>A$p zC!Wo1Ud&-w0Fp9!Q)v|j=*;2l%Oa?JzzI0z8!q(FALK+dn4>Y4ru#M1EEml%v4?>` zi-wVyUbwkb3^NHwWLLxOEe?85|3bmLqq>zI$iW^GrP_`7_(bw-ySmsvdK zQtTBzYHa`!->C!V5(;~nurZYcT=GLJcRh?K*<)O=AIb5DsOa7JVp}9-}^U0sAL&=^{-6aFX^AJ z@Zo`zpzuViEA}U5=vMl(@Wj%pximoy0$&>7(D*4y@C&GZpI~t-Zg!lI0+hbIW+nbhf;uKM*Q4&Tsub1{?+Lxv z9Qjt_PzVae1q-ZI4-wxkBqd8c3x(s__+tgJ{DrqgN-7cxwb@a-bvrDSjoJ;bADCm; z(dcR<14V2#TafB9)YWWUbx)q;Qk&uE=HqN@rg+TF6P>b_WMsPSE| z!Za!=VOL-JYmgMMqvR!p=7w0RQjYN#&Wf$+{i6U1ucN2}$y7NZMa)cn*#_ASj**MWG@y}CcrdrF#q6vJ^{A4IZn|kt8&Dfe zuFx4$LFHq@=YE(n)?&_?AZwW)ChpCeg-|}7AH(kX83yh4y65z!2}?%pdh>+`cag`l zF4OWW3e{@NcnO`nrZP;6!z5^WS>T&opbzWq6NLVgzJ*f8i3y zad<%l6UfdT><*lrxc?sxD{x29X;cfRdDQ_9{c+~3$uB_+rISdXBbJOdhdGLx&onS> zeQoEFZ=h~4WxQyY&LWd1FDF7Y_Bm?l7&=<^Ryd`09A7%w=&|yU%BCRzr8*!uS)CT} z%(>6(8neYR+Z+JEdi(sJ$AvFvr;Q^^nyughF(V(#)u`b!fDXUnECRWr&d%N~u4+r{ zW^u}hW-H$U<))qr=0Ws^TNwGG9~k@8sB)2#l-gAxbDf#;sL!l+JSj4FpLr{VAE;~Z zPvgoSw@$o}Wq$%)iiex%9t8y#0a^BE`^*xfVmNgj3!JxfJLVWR>!uTuSApP&7via< z5vOj)iPCbK8cGV!BB(@3O*o!-t#Cbfl{ryH*tlzPH|VVJ=x}TzW08@=de}r$XotxT z)r_4PC%iMh|1XD*S#}@R9$?qk`@&RTtbal@ zBY)!ns}}lxd3gq?Mf9$g@akqPrOBg-#%6KH5SdoS3((mE<|Iq#y-tI!8+(R6 zYhddIn&IQPOpZR0t0afI2_aqM89e3*VYMja>9|zkl6}MR!ohohi4%G=1Z$X5L=7cq;6&GXPx3|<_k;!cDSg>9}KEZ}1b6 z2L!)ta!!1|$_@0g@}6G<<>zm_f&3ll!Jx-@ECR|dN8L!0d;$u;hd@2lh-p_v(4$F{ zhI8wnB)kelAj)p#%({3W9;OV5W1c;qok#O1$<65ViIzu8Y2nD!haI5`sgvoOujsY9kW6OSWFpymR%}4NlTO1m!FxG zc&xzwBB;iD#(VAzW364brJ?B+~ffkS2Vx?hXI_)CZB#0)a4jOR{jZ@8Ff%di5u6w&3ier zoPYAzE6gJ^{wjOq1u^xmOU2tl7X&@(mF*4-1OsaRi-NPd(c^E%-h@5z0y@A~Bk^Rj zGm0p|p!Hq>2@kt7f%XwpFmACg0H{e#@D6A)h%%N4oof(wXP?%M+(~f#ydSrw zNM=o!&in2MjvGie$Y`Bf_dLzOD40OyrLpZW{``Zrgk)LtyjHFtche^ZOpLU!$#DNU zJ+Z}fqQGvjhA?+?EjP_S_tg#84Vk%_*kao`@Myz61LJli5(GYP}k>6`!a zyMzwh?a)EvqZ$>-bG3>}Z!dkN!C)|NR7o#9oe^UH{4Q$#wZBu@09CX%MnF9&JC9Bi zz8vSmF*cXxllkQS6vmqK3 zYUcy(<54!?Qn>EB`}dbbBIV!(tiR3oj(K6x2TDq&#cVws08hRN{vpT`HhS}Y#%TN=D3K^lFETHka)UuKs(Xk8>l9$i?G((9{y5{ao=GTP-a!f$}tly-#+lysz&m%Qp-W&i4_`?kVG@%WHAl% zLO9S6Dg&kbcG*A4nuoD~4)UT3h#Dp+re7$#J_a<51A)6e97(-PZ~6qpMFFFhN(KM( z%)eg!3Dc43fX3;*MD0lV^%j90_KRPP1)9^YcK`%347`rdAQ4>+t%Kk~B{CEW*@#Pm zl80e-*Zb$HP3xAYb627$cWj$yn! zfN#GAJ};2izFRh$f?Ydr8M|SPK;d~6sPDTM^^ktdm;*|)!Cs|*-Hiv;rA%5Kj zk;y(n)BpDt`#Dt~=m~#QClC_A5WL@ifwmbP?OP{mVw3rAqmPRJxl&39nY(izzwdNs7yO3yw?A=10!hEKVX(rW4@x2I`stt7 zzAU2(_-}mvf4iBVUyfr3$EnPzBrnLS{9SkadkR?`BH8{nIbgm&A}2wmETpnLjmtAX zkqtAa|9$V!gn^1vUC<47uX$DttYYV&{_~W7KgjRvaBi?@T81)EP4y1A-)y02tR+_} zcK?2RdmZ4CAhEBKQPitO9S))Q1(Gcneh>Z5C!xvzKdcjGn$hvoeN)#);X#r+SisfG z7We*YFa#~;_|w#LEOhZQ%#vUJr6C3Qyalj|e?KWSh`$~OD+32@Y?7y^-~CFu{9GqF z$C_Qhm158ZQAkL#7+ z-2x>Tow-Q!#sAm`VBr4?Zf<|#-&_6)6X+!%8Iy1ovTn2Vk5B(EA9H7FiFRNRl_>9e zzjYqKz4-riD?cwX7Xx;$-`v_Ze(irY#Q)b(8yiu~(~{^wiy`B51tfy0L>z6otDf)e%0W$4B4frVcEd$s&}b9-SSufl2f zawfkYHj&*~Gk@(QH{;0u9fP>DMv(t^+xhFGWzZt1iZinJ&{qB5e+w?E`A2Jv)!C?_ zorghI(WrnU=Mit^|NGqiSw8=7bI$lbSgHizfatEA6aUvp#qX!~*pZCb{?S5&3ShP6 zodZI1awnvKL((>S4&cmf#wxog3Z6yB!G&mldQ_ z*vn|icu64~MDf;WLK%sWmR6QxNng`$-P`Sdgln)i%U>(HEkN9z@GWWX&5soX@sK;= zYtNHj#>3jTW1#y3*Nn40rnn$#+LcuHXFX6dLU+0mYC8EDY^qZO&X0gK6p96m?!<+h z`uh6fTd?2jf`dBw=fLB}z>!w5PWPJ8ab>OA5Xg% zls_uIiXQP{+6Y5WC{bZvKOCTBkGvMHi z0KV^I9T?&x!>&u@LU)GK9(-P}@R6Oe@H?Kokx$9(x|?;RR*hZ`8;1ic?+RgcE3(A3 z!C)PjZhq#T3gx(;>hL$0S5|C*u+rglWCmzXB}~osSF;fC6F}5=>rn4W8mS|a|2)5D z(AR3a2l0Pe{z_+%fYs5-5RH~a(LalSIF@&AYV7z~^r4W8rM0bo)o*34eqjGSxxuC# zwYigi3`+8>D)@)VWb3N3NPG}#>pjk$A!+b_!{yk*t6vVqJW$#_iuG)|G#Gx#VC+11 z=I(Bz3<02L;ix?gM)20S11q|kE9YM&U z^J$9Q>^3-6aeS@4;y-~}B=QcNh3@$moNOqu*-Bd`XK7zdG0y zvY((R8zS;mJplSwr*%ftSAFT-*iKd)(1ZDxK?IAu203t7i^J!LEx6k%IxwfhK-Sja z0cZelL7)4b(BFmuaO)4US|H8lsI zXZRanO5r5j)N9FZ&?WL6Q*_9W1_8vCR7SeRCIF7In(~&?!msVtXuR_=UQ$89J#6|2 zqX2G6fWrX4q#q5EMf@*H4km-=ZhhXk?s>M&H2@~nOOaf)@ZV23(fyt< z!l!@4SR7pVbT{!nqO(--l@o@PSbCybbEc%zA)|w1x>IQM5W`>V@~4*%`Qgv@LcwNB z^vNVf!5cszEUs2HmL=D#;*TfBs~Z_i_t6&FqQt>caDeuKFqIgW6>r>h%&A6O{e|=+ zjG3I!6OVQ)=4ebQRm-^5x_o}%H6V^V|4^g-bh}QwFqh22J|+C#eCzx}HSOvu^2>$6 zNzO)8Kf6d`rqhEwl-wXE8?ie&MCTT*=Rb~KF`Y=Jqipya!N@&ZK7`YasE?!VrX}8 zw#H0+rvL-TU~ryB$H#^Y>yoC>Pht@c+^mU3SWTt}`2|#Y``6!$`lDY!cWsHk z-3(?_sxW!R?R;JJoPCkTRO6$V!3amgtZQ-C%H-$-vBhaL%=j%iF0R18PA6nO9pMc0 zDA-sQ;F0j{SHAtr)5V@)!!1B7JQ8LqBj~G5Elk4U4hQ+BoWtB=ZGpC0;7IP0W^`w2 zTbEi@d|qU%?(!vfn^PR?#*rMSsso}lIA(5k)=LT+%;b>R-7lngy&9M8DmBvR;(u$75C=y7xZSak;vSJ&mw{B~Wz1Z?oZeux+Fg zzodaM%|YbokF1oU1>M1>&&w!-*wfXxuonji!A1UOqe}xa!Z+3p1b>E}I2=soujmLg zwgJHji=`X{GxmuzV}2Pqf;Tn%fe_v4Q-_=$`6YR}1#P^GI6;FJ7n(d3-6@gb$-zRC z8~nn}xyt5wb>b#qZ_fMTQ>&?~wo{tulA=dnKIB-)lzyjQ6P;FaV zJ>eVI)Ophve9e+pbmx&Rc1!z5p0+auoffKF9Vxiy0=yzatJAZGz_o`zr-OV;FcV=( zW2UUr-611@fw=iT-Rfgm&b$M~A@vphC2d8?cbHO)>@n2V8p(JeX!OI)ZU}PW-4E+y z`t77b_U`3X2%%8e*O&FLsFL>}Xk=I)<-Hse6y{-bDlN&1k^WRQ#cLrrPSrq0vwE#XN~;D1dPUd2Qbu)%MuR}&$i%I)<&GPm0LRPyZdMbuzjqw^=5muIJyXh|x0OfSnLJKZxzl-kDviD`teQx8Z_ zkDh5qZEbWd-NoE|cVW1%1qPPSS+wDo5bUu-jBe3X_5TggL8l+kz07y=EX&`6bb}8! z$+2+Xgj5iLDENDEpT1fcAEKr3tY7q6&cm+rJX{$8FAb`z$8 zJZP_W)Kp!bvt1(_KOKuUa;By;K(%Aka(0-oi0}?J$-9e>2y^I>Mo~txDpCRgTt-~m z-=tl$wA}FUMy(h(=~+RgZm&zd3j-X_}{T4GKNl&sM%75z;|EDbrF%& zKE)V+@nY$09GQdVewGHXr^fE-Z+r@o9YNEX`w`y=xTVuGv9$28+55m{Ei&b2-J9ia zIU}xdXo^jv5Bc}6aJ=F_2?iIpZu3O$38dO#duvM82BpAl+AAeYSBMYn}% zy|2X2qbQSKHb2`YOVOl>;pS~l<$BtZiOci0wC{0?s&$>MppJjAS~LI^;~aX_*oiu0 z9+SMjG*rYaAH9Le4d1Tv(Zl`BqL(YT@N%D&l3Jt^$B< z0PqM*1-Od-3PB=~$~1Em`7lEk->io{mMoX^cv;|CLXz6io5)gRhTgms57qKdG}szpeQcoG~CB4mIxUAEX42DF;`jbQ=?>q3zJf8AL^G`1d}* zo8x{>the6~&j&tL-j@`FpLS1v{c6NemnbBzjEetkTO?!#a=~5+>OFL zL=Hmbq_^?lt;{6v;KTW06qQQriJ7H$hv#k~ZNMo>_OT+ZERTwUbF=F9A7L8o{XNDz zi@adF(VbtaiztY}rOk(arE0i^;AM>u zXOo&or}JNHGjDm+VnIZ@#mRl@R0{Luo=j2Y+QQ-yB|kJ zWy=un_t(yk;7F(5hqcQ$6*$+mP2mIJ-F~+Zl3(@$F^nkEZp=P5UERUj;ZIe%*o$`a zcs*2aapDnd&b)1ZAKAFV({AC$K9BQgo2j*ZIlR`lrFkQpFHoZnVD@!prmgn|F}%%4 zUpO%~vCOS*qmSCElh@i76e^#jj6KZt$?Xerp>R9{jqNxNPS$yh#)~-trIx>A8<5`v zE4yX)gS;S!i~!C-pv)g(KFCudD_CF=-PWq^ld(KygS^&Sd7o5YjwnD+cL$M$31r0>@TiBA>$Ie{J5V;~Cjzt4G7&I#FFjy1*K=7pU zm|Ov>ZYBFuUDqqV?YVrzMhLP;@s;OKZ}u(ulg!oH&>QocY$YYbZDbzSb6i+cY-6YwejSqiIRap|)N%81rU=Xa{5(&0SIQTSymQaKo3L?X z1QtCS&rN5?aPFclQP@4YB!LOnY7f{PxY+4wz41hM@Ft#Lccq~iv?HDkSnXzmEr+KwuYwd3E zMxa|1`{;}Xh#2=#Z}8Q^EYNQOh|zyRtA)(*%rO+cNjWpzFSC4$$G!Vu$UEL8hW^hn z&Wt*znpL;5W?a#BGd{lJM&p(0Uv+1KD+7#im+^ckBg|FH5LJu%jicfM0=2bXX3e*v z4AKsrog*c<(+nTQQ88uAd&gfB&h|Vtc&+WGBr&rn>`Vo zP5y$i?OwG$G*~`S60;aSBi{SLJun8D2VYig3EG$sIhiMN=F?A~=ojvYy|Bla6Iw}e&rTY8osQJf>((kb9(~txYa#V+Nly6$M&}QxS)pQ zU`g;E)XABEuKOSJCGR{pJYHC#t$HdTY}Tfz;zE9BO5FX?l>=wzZb**pfgzZ?YZt35 z26aqGe09@fD?&H}ub_@pEu0w~ty)8Yg;`Nh>eeznsvRZJo~iu=`ggg*^scIrs#2$E zugr0DEtQX31*uLbFMRJAw?wc6Ib;W=8xJT<$XZqzrz6beDMO_*jiey2N6hu7L65~? zL#_@n1=c@l%9mIwi(eZEgqphBEli+n_{PKGvH0~Eo*=bekhPMa-q5gmbuLtu{SKEmT=QT}x+ARMR&{`Mto(KzxI zaSWq$J9_0PBOP&cP?pg^Gn}Xf*0gD2y@^9}{jeU=nzWCf)f*eRXpeJs8CV21r*^m( zCKrfBdeeM}I$0imyPbMIL=SEMwu?f1eT$Fq9=7IP>?!yf=V>spE6drM+iEPBxyM1 z$l^R(;9${twu^POR?L+w9_g-QHf942(6AS&$LN#PW1INY54tO!QcyOWSaW>mw z35x|%BNw62jq`I*7P!UZ+;BHgmV7*`rFyS3@LNHP?fZvyjxVFm`+15qXZ0p%qa{r1 zQ3v2JKqEcA$jDSq(W7^BX6mM0-M_#SM3`J;TAaL;3_i=n!Ll=OCn_~4)L|~?CY8FL znc(&Xvl=H9fa6Mk@KQnvJ|bw-E%U4Ro?f~>e%yL=qrA4&%7!?1#>8ts{FbFhdFh!0 zFW1Nu?(KX{3%S$3;0I$}$L7c9lGx6v?JT-@A?&xDI|deUuYWx$vYv+JUJ3F**xDQp zxsn=p*jsf9=n9zunx7ExenAg_hbO;XlbAf3E9~SUT_;h?`mN#I5bCNsjgkej%Cxp6l`&5(3i2en$D5Hy2n#o2*Fp+47s`&zgpL}-PIYJpYYP| zsRIo-fLL92#&eS~P8oCi@>PA`4f%9R4=?t+Q25ZTpx5)!mo zd>?vNU$O95#fL7KIi^OPv$ap%ONtkUU|&zoK|OS1)Qb<9#Gp=L<-q1U}hQeZ-nh0VZ@h<2zV5(ffT5*OF3lwG<$2>^%p z2yP~VZIQul>oyOnDiX<0joKnDcCBqXxT{Y8agw*^z)3qY4SL}Yyv2kn!I7T;6YmUc zt+ixvzs|Wboy8nfNyt0g{py4{zW8A5JH&DRIO4DcxjS{JI$+|tzaB>h7l;}_O<{63<#?F-nCbCF@2-AX|xc16svVXYf)(#d>&{(g@ ziJP~h_6EIM!C|XJ?XX|fI=piozcpv`+LVsv46cxYBco+pD|BKh{YG;e6VVuWf#k$W zyRCi3mO9iss|%2gn%AXW5}^i~(Xe7Lc3>eQM<9mL`VrjDTxug*roAVxddQ#qA;@Q~ z$xPl?9D`hgfpwRL+3>3_R^Obn>AVPl~cEKi@fq|zyDc3tNf+pQIvs@e`oV6?u%)?&FbiWUi^B^jkA)YOZ{ z&5_*K;Y?rX1=)KyxSN}t=Jm$bnY*?IhRX9h5BQcwgvk?C;5N6m?;8Bck&tx!{$*MI zHQNq~4eI)VNAKh)x7|C{7>R|+=|&WD1C5i}Ua!ok9@hJilSKtvIA~m|F3qDl+IJw{ zmUDyY-N9R*AV11MnNIjs>z(XaYZij-%P|}if0}a%zAonU{72drt?k9uPyZbTgd$7U zN@S~Y>K*V)Z7SCFx-i9&;zqvuu3(By$3J&#Hp?%W7nt3l(NfmUUU(`|(=<^}DLz`; z%SKbiJ88ew7_rrLQnm0GkIL&H7|8YP)bSnvYPQa)`fxQA8>PEMmA-ii;gFL2t>^oy6@#Y&3hhZUDGCV zr)FX?g>x9?&I&dI@&*h;^3?IWYvJPoR5|!N_E`sLzLEh{#dv&kyhZ(6!%mMu@&oUV zcN|~^#fvuE?cDsYOv*(lJ7y^Os+y{D+m9&FKt}LhjFzpy#7j!KTk)OKQK!>TMM+G5~^|$6eZy$=Yka=9}ne}?IDKS=4EiVl6?Ec%#o?4*lMYz3XuRCDLUSKf_2$kt^^zu9sK znCx4f1XbkqbJ?|>W=drjXt6`EY-#lEF1*PPqp@8dW~2bisE65ik)E_T?7kr2u1pHr zC_YwptUomvgoe!xnkVT`KlM$Co_IJsBT!K-Lq>^owkod z0)b^QWG2X}EOobbRN+0<79&_Jkwb1#dL6M|$LR~qr{PzNhrPVxEtO&Ib_Gy4z%pAW zA>;_7rZHu5n^EPZr=*!P4$;Bb+1zGi?XSOR*exnwQQsr{Mw<$IoQ^oLTSn$w?~`;i z&}Dg_%lO2yHci7>T0-*q(t1Qu>DC)Q6OEl^!F%+}<}LO+LR&<@T3B~AhA?>+i^#wu zk;o%}Nhb9CTn^LE$Ll+elV7imIFu$Co=CpMV?A>9ENJL0B|=aR9xSV?rM7XVf6h{o zeU?(F(qGN3_4xo+3$Eq*MeHMqa<}`~4Oz@RYaZ-bY3@8 z{$0LW^RUZNf&`SV-u&vDQ=gtQLx?hhy#E4d2ire!-*uItsAexcZ@e@0+%^}VkP5ai zdfG3)fBT`hMRB|gl(>@zbPmntNJmu#UPDqIYvg$<58J`r3fk{L@!FVtJhORPKi}zw zZ=#d*T>edb6=h*-vCLx*_0hvw0U*jhiXa|l^h#+IaJ!bAt4dOv=p9!BCler~*R_qg zx|f%Jtqm#lHkkNYWSfvX$KohUqX#zpE2oklq^_Wigi|dvFtLNarCY^e0&a?Q%9=Pm zYI)Gv;_@B~{HuSq?=zSU+7Z|LdEy)FwV3BS0OR&&z{3l2XvY#Kw?#&&uy%+(OZN@z zz(%w$h>{0R?dPl1D3gIHG7&0O&;2M!7@muJR(CLw#l`m|`Ie=pj1TYai;}TO58zxh zpFo9hn2ui{p>%H7RTi-K<(mp>fc4aBQ2a z{1QiLULdJFy!PSK+*%I#WDT4(jO1Tyz5dlJWpPc7^);hgCa_E$4{8^l?v^Rdy`X4_ zrMSM4WGIc}bmbUVT2$$cpH?et6yv`QwY5wX?E$hEs;a8irBY-th9~I=*8sFSWaYlK zEM=;DZS~h)i?;xVtPJB+a1aJR@Fifv_VMw&UaqPrEgzvKkZ!TfZUTPA ziXo{D78H)f?=$Nc*uswz%&93HL8YyEq|tJ}pslsTAFOX31?NBlQn^$yp)Czx3TdbewLkn`}-$-VuK00bFO zy%4X(4pw|K4DzlWK3rRDjEB?8-#bPhkhPDF#k!e6(BdEKqLKOIH<{k_%<(mEam=i{!W{O&j`7#Jjynpt|l?U`3s>W&g#!4~QSYAA=t9w@CU5CXRk z%M|N^0|9OikoN?o{BO&y9XS2LBl|g%7>|4t@l(bAZTRLLtY2knnn&&FJGzQ}zcMxm z5keWLhoS4C0VGEaNL2Z_!97R4!`+t{ipoGNA)|rB-t&%6X4b;A#a(zd{_YqECQOSrn$ZK#*XVu+ai0n;;tb}Ac4Kpc1wnD?q-lMW*ud+inCwud| z?qk2d@Av2P{`~r*$8pYmU-xyr=JPeK>n4syQQC($|G>rO83K~&IOAGYynHJY53~Rw z7chIwxi>|iHp6P)GSd&C;>*i_lG`#X7RSbVS(w#Cd~XFYw=^s9P@$J%3~!4Faq3KL z8^HMHsjBsKU-Z~9?r}pITLF9*fY{bx6S@-EbaLlq-6t(TC6Aoo4P+r-UTVR*FiHDT z2xF184VcJ4CpLbGYqEwBOs|JYQ@&O13Vyz5t2xM>2|x7xx>JFXHIhY42^_u zl`e2)qO-k^O;Viw=Uo)1|D<1lv!2T9%ySk|>p{rj_=%!fAk>6@K5*CR5@jE1V1ICt;>k`(JR4&a44ScKDW4K(MD zAtSMvk9E5SnJ)jiW2@{8y@bf@6thjd^Ga->e8I9;V3_5bk7qx(cnYRphYO0A;Zd3Y zE*9@seOEcHVBpj7%m#dEP@R~7j5syb_VRAGwxgnKW}gKPk``->+#x2#On*r7{yX3+ zn}jizqq*a|;?qY|ec0)mUcs&j_pb{PzB^7K|4!y4m0dU*RVsy+CC@h`nTx1h1+R0) z)Ijh1h`j^sch>W8%AJ}|6BPy@!W`(q`?A@kHqT;OCDyWAlbDSB#keewu3einAMB6Q z4sBkzGT`+NebAIvThnmwBR;XGG+b#o&#-t@fPPqylw?^(qm&Jx)7YjTRTkF#s%vgV ztSfk2nGWl{d(8k#eCQ`Xej5yV1v{^Qy?Zt)R$d;J*L#Kgri8Ty^(709s<0LV$RZ<+ zaEes#e64=Bz@t1XVX!Oy_|>UUL09BT+fEq^=?PwA;$Z$nM4>FNR+;<THK=-P^l zLbbC9JDW?f6_W8ICwZ-3l7N~rFUIoP4a;w~SBCp+Wym9re>(}G!3+d9k>uCJ(~-hI zu|n{8S0HpHcr})e{F8|(Z}D?xn+!UHDbPK1q(!U-gySof7CGcMd05NVB=P?&aw$K3 z*bEv`!0RiLiv9DD^#0j_;0+eBY8ZU6ZJSg#Czld+`|yfN3bshfV1X zT^o~UE$@PVlX_v6#7Q*f&Ld}_CJoXwwg`hgJ44+9ejs!z*Mih&d zU5uSsJeCAjU$oVW;NeqBA(=J?$50uJh#L2r+~@Z1amBNNbwuP_=YRm~CxEb$KNBg@ z?WbLA__+=YKBlIU0knSp)YJ>CSKrC>G{=Gyr2z$GU~*@}0Mbz9m2Ip}B)xrM4`;+a z_P85x_VQI~c4`ox3JB~X^Vn6`0#}QR@e4rj%Q!c0A)?zmickgC4+x+Z|GJ2wKYb-~K>39~}QubN_JgN+?L z@E*O=9=$IuKcr`K#GxZ+kR4de9@#%dd_OW6cH$kovzH5aR~;)&&Fe(+3JV=RBABq@ zjosKuf0GM-PgPW^T9JUUE$q@O$`UWsftwYxjcO8Aq^Sf;;sD7k0vC(S~U3Z)E5*c+?^V63?jZSnOhz| zspUYgoQv&GhvO(_W49Ko3@2T-=^YEr#JzeH10dw<`ILI_a_5+yY&81Htvhd@UyIsVzp&*NX6SkWLsz92GH|MR9$A66wa z(-!yT9LfE(`sZpv_t4~rQkOikWy_lds*R3(*!A+t=E^|;yzL=N0pulvMIeRKAd42E zT#E6{D*~ny2ejG~&%Cxo&@gVTA^D4|p*B-i)D_yM+Za~@5ZB=gQ+Uo~pB8^6jj|L! z&lGe2TnUAF#u{7&N8q>*aW!G?NF6NfB?Cx-ft(uy6IkRVRh8r z&31B6v7lJLN@ES1#$~RLAX$_I8(?M@V=Ctzp@9De+&2_322qFUi92nzD*AJOZ!jAf zML*5f(=cfpU7Wqd@y=NGt2gmkw0Hn(U<#ywc-lKBPGkxlV9;>GW+GnnApU&6^%@j! z5%1xQ{55yhj%%>7jh#p!{N5qw3N~uyBa%|ZQ#T*1e-Bj>KTxtB93HxxJj={HkhYoz zdcd?yQqw(IG_pH#BRZ8CEVZ8mDrc4EUBHS~J!YBkgKJ1jjK6IbG?n&0!%> z!G5!tTqn58S`S1BW-SOCbToIOL7i`Itij#j^NgM5fEsC7#KG@6t#&0C1`& z{rGTwx`gSLwqnTt-9_~PY3j574$EycU?N^D)uUq}S8JP;L|7Yze`4Uu97}+d*F9M1 zq$VTon-^{z4qK}aD{1{+v*mw_ThUIjS$?q2UP90Q9K0-BFjjv zPO}BKk<|X`P1VT`XZL_q^=+c}{N}8Nw+G^7E6niO(ar1t@{;?3R4I7VJ@#d4VCx8* zBGgadk>4ig(d(4b06e}&df)lvn2pn!kB(20!AdDFG=X3tUxaA(ED*LYIWfo8^vYskz-ZuZO3)!1%&DAF%FRB11 z{vH*Ss9Z0KG;OluFX?ib4-Dk=^+eHiXx@`bvPGOyB##=#bCTR zFE1%`pJM}ZV~hC%>>8qZSx_eY$@&!@6!y5nn#=6{3*FDpXg}QsN^u87z;$@X6dfHb zKok+q+O+1cct7y_!iJSA`;Yyq5V#SG=emJ+YaI2qUgt%#96tm*>>_$Y3G@>h$Ivt> zAZavj&)o)C;UU#Cg&%yDJ8ma-6M~Z4XO)h_PcaRwKjB7xcqa6nhxsIo&J(XzoLwm1 zmuDxg#(>wtd-^GQ5FHsL%)lzW$HBGf+nUzRw5I@U{tUIK$@4S2 zvR|DDLGP0_ko`zX@x2H<^3KeLl>ovzPsNjnhPzFA$mAJUpyHY-0w1wXb>V8fC57~Q^X$BOqTln>17C?u)G&D+QG5E>~O*X2(X8D6HSqO7@ zfmi5~(r}-;mI}@idgmCL;*uWz`=Kg&D0UJW{Op9DDO{wI2Ri?910hRG{+gNVeha-5 z84J-nNarC!35tUyEAZD+wo8%!g-u}O3b3hElk9$wqE}OpM#q_Ri<8GNoQNhpp4=eX zr7n#MY{K$(Q(QooQBg$^a_DO!`uUE8N=8z}PohKHaepnja{MZ~FC-pVJzOIm=p87z zV6l`Uy?+=+6E^Ip0{Qq;It0bem@*)esea&$BV$Nr0rG2t2!ry^Ezwd6;Lyj@ro5mH z3wZ-e347m;?JflD(Ls3SD2cpyT0?ir$nwOK$G6vy8eP*enoOVYQ)MU$|2h9k!ugX zbCR`iy`_34dv;2Q_~}L5>=VXy1s%TzNYp;xv@X%)Dso!-8Il!s?@1eluEHqZw?|18 zu{Y<{Qe*61@EYr-bgHn|b=tZzrGU4aCv@FO%Fj&);TVn5C5&)17HWD6smK`e!_IE5 z3c~xDvfb_4i1b^2UF2QQ5JgbF@D4XNDJR|AJt5z5pNDJn!dT4wTH*tSf1I`K)f(yK zsKW9q$;IcYLHQ>>+bk!DM#=J}Q(WD6k>OeE?6W-g`f3-66fWO|7*B=ayX%gRAt#`% zy$wDIyAOC@7O9ml-dj?J67ZXYpc?%tyEB}M=Vf(}VY?a*sVmqQg|USGfc_8c6Th^g zTYQcgNnc;d>IlJY_8T;^w%rzJI|3~4m4ow{JrF_QnP!9+BcP6~2?7Jk9Yx4s9AHei zlj7F4zYn)M?wI}lj#cyoU0@US2yFWLF;|Io6_`(;v>r7d#~4!L4%m+mQUE#or6aVN zzAXtXL-Kj-iJKA_UC>UzEQvC_vG<`KA)hme;HTIw8E#Yu=Ez4}Nx2=qr;W@+u}5hR zY;zmtYvFenpaq>oDaKeC!fLcfqoiT7_IVfovYXcfVFe)Mr?sYZnjNM4Nqj#@N(z?( zoL;ZYwx{r*>-yI28^rd$0?+j2e3?cn6(mhcAtu^h%jPKY#EIqtl3rEo3OdVtK<%c7 z7_9g}@(Jbs9NGf`aYacO0}}hgi~_eoCJ@Wc+apk#=V@^Oj}UTr^BlLJM9AI41{c8r z{|LhLHfWz>uU+zy9c(G22JU|u6Fh5gFD__=s-YwQ3J(OJR$cFtCqvtMHKhC?#s&vx zd@^jmdQlK$(=#60IXHxZ!sPz^luD18qo)8IO8l!1%}l0!Y9N&{-tUR%o^?F3*)sfM z?C>jO5ToWBMBvCwUyNw)V|bv=P)m=+NbYY&?;GF{i+TI^k9TIBrRmYT2k|{2i2vuy z+j~1d4TDXu0DKStuY|{HR{7+Ns0LR({4fRW3XV-5>W<0qAe4D6%yob6wzkE_N?T&$ z@GFH08wpuDuvsdrudmNwYiFAXvDmy6G(a;OF! zyJiIk8@bh#J7<2?h1}rYiH`P3X+rJeflp{Lox ze;nBp42_ct1m}c8_-(Ug(ZLz7@eb_I2Yf>F2I+DtZ?-+<9wult2~=xA?phyJ%b71e|S-7g#&pfcJsI0>)t?S?u=@chWyqf-DmxtO0Hpc30`fH z(O&&20ODUI6fJpIFp0W_uGWruLCga_o?-pDvd_muTR{#99wfR5QFzH#*RV!@V-uS?vhI$ z9!QRjnZ+MhI1#?DS9XcVrXkyQiUTOzmot~@(Xjw?iN{lF@ke{PfbQeK6|_*n(3}2c zq&-p&^A$bA!<(Y%ahQS8Ae7-@)`Va^{d4-v%VfJGixhkVkTN=;H2TxNr8x3E z+A1BowCNs*JG9svw(t>UxX1rGpZ~Of`N-6P=mAWydhPY*DdlVb-=hF}Fh>Jyq+CSW z0tzw^%slkpeayozpn~(zX3Z?R>bck9fNO3^+Ms z6L|HRjCLr8?mPWbH#{LtbLg3-3>e=C_hvav1msFl@*MPp*Qp+1M0-E%J)I2y!r^6w z-=tyixutg}-nze}o;G|r5V2z4z&&x# zDH{t02g%&K#riSSL+W4!;DRPW7*L?9VjXD`hjZBROlfxGEetIoOLfe4Z}(RLjnji! zuJ~V{=7I3Rx*^^o<0D}j4#&#!Mf8`rG6h5|iP+!a@}fN82m_-*!I5bIE-qn8$o$O% zezO7xsFPNdhs_g*C<&GXc7rYt&}D~^cmz!$2h>4Rvbw|rYwzdXXCS9=ykCOL7Ky<& z9YyB)11Tbo^AI5TFFXNPzjVbQKvM%bLI~)uld;CID~3EeW#|6oO7!Y5VFDO1c6I~d|9J7v;C~(S^(%xQ zIOk`wBjvfqGp$pwk-ww;zgQSCscAZ>&Zw9Ml1J_GWdE+8e-ZA(gghkl;4(uTpu%S2 z7T1pO0&xdpeSff>5C0@3a4h0X;Gm331V^Oyg(&tl{Cn`fmku}=1F)1Tp(>YlABucW z)I@Y>qR`j-l5ptzgLNwP0mjs>ulYqokFAwP#Q*F0?-7rj9>}}64mq$Cu^RC2PyUDb zL-GVn!~=q{juh>rr>ti2sz31LV#7VI{eK7uK8@0Cu$b^)kH-T4b#^$yhW|X1_0uQ-BX3BetWQV3K1ot@(as026;Cq9H!Ea?5MO(=H9_=&|IB8%z zqsLZYg5(~5J_hz}eVWY#%`R5y@GY*CxpY?`Alit9Bc%FgGq`_Q1t*Y?m9$sq2$^M( zSrXP-iF^zJ@tc9TU*jj)Vs~8GfkFGv;)*8NIbak6zxc7z!&DfX>`XqiODcbrkvnfz zptmTouSz=~ZsdDV?J;V1LVKole|~T*2>=t;9FRwO4dimy2p_r<|L<)z&$XvO@Gw+E z4_``yFpz^s{&$DB<49u90tq4hrh4h*9*+U{GA`0wIrJyAlB0HYM}T%s8T0oAz$@X@Oy(VbguDu?{RhufIS3?Rmk4{2@csua*b|Kdd$><5C03Zf zA~{PiR(;T4jEQNH2lu~RRGk`#dVcCTArGM_r57-xf6o(yD^SKZbvdB}F{ofNIEsjl z&?M_|8NiBh0N&-Nka+UZ@2pb?80n2z-`QifcFe8vp1p;={WJF{MQu?Se6_+9NfB&I9f3J|K-_PuG=zr zPZv=AzzDuh_V77GbVO_5cKZI1E)IA?15)4=hEf|#b%+x2gVBV=ik8a|}F!}FuN4x19NP$Dz zu8<!yIa8u5sjQCXi9Vh4I8A=vM`e{W&2L&c2eT?bQ@j-NEO zV+bw(_rZWM%LZ;GE5t?M1>4=Doc>Qh?GVEdvA5ymQ9R0A1mVnk+}TlDnkG{WroWe! zefmHjAeM)u0m+>!UZOqZ#bP($5{K*pdeDEiO-LskP|)EE_BKHnQP~EfWftt z%QOy_go~TTJe67apJ(h|5@Sjvt=duJRnwF~lU+Ek(lg7MV#MuLtK;BM-}ssTB~2 zdiG>@m_5$z3`wUsIgm)DwKNk85K7g|hyFYOct_V!U4mGmd(9KLX@2o4jGg19^&JAxIOlc_cORTI;bT}H z@lpRNy!{A8fhT-+^8sjn%ITJE{OkLHE3F0UcO(DpLGT@}jzAqICL})x5MtjV9w;_~ z4GU!0Fzv3P`GcGz#TPlc=zV8*nDmE{27(2Q!165V_6zV}=DQfSN0bH(c33ud$NmeF zr2+5nlj3sJ#FUJkg*AZG{o!Z*i~FJLWq|}LSI;nV{da-*hY{-u-+~Q%hV3jz31FJ+ zG5@cG1EiOUy&$d6JAGIDUz)f?y?C<2uzRiJFEOc=o?-wU#55p=+zTk9yx~4dlK(0bU^sb@G>!j6 zzj)09(TX$n>hP1mV)pIc!T0|+qR^5DJWXm2?JwJnst>5}_cEm!@L4<8O7qN3mxC(- z>y)Hm{_^B3bhJk*7eF;LNbvdpc4bQPu@0y{Rk^>W83C5Vzf{KsMSYcI`eTQl@;buf z(LTp^#ASa$?{Cqh@NhjCu0G3u-@t0JJ*o5c!|_AGo&3dE0LA50X*R<$6Gl1A;u9+T zrDJj7T0`HGaW$UpSeJXJnE%=6$F{>;N?kvsv@;qQ?8twt3tU1DEp_2H0TsCiC&Cf= zr{YU_u515Z5phh&@C12AtYI>xor?|x{GUY=O(|wZAQ=^Qyx*T8JmD=|#l_CcFR~Ig zai1{~!xl-JUL5cfeffXoGeQ<}M&orRb8ug6niP(}oQb3t#LLMz^$MVfafrYNsQd6E z4ht>xeeHFySVvv!W;Luwd!?8?tM~UZ162wSj|jg0cZ=M$DCwf8$uSZkRMsr#)Sd4LtjWT zU$Pl2Us)Wb9-LH^DzDmhX6SSOz1p+%+iTkoytLrP>~nW&G0_wHu4?Bqf5Dm^Q1I9Egp}#cK**rTvN?q{*RYFR=;b2;{+>$w4v#sYpJ5;G4TiPJ5 za{udIF5vI!O_d`6O}TQH67v?*EvbZbYl&d z-#NE^@ZE7fpStx0@1z%9EF7YbGWo57o(&ZYE&E!FZqnfV@`QD@YmV34@m zmfJ1d;^nm-`H|9q?UXN~XY?GMiq>1W9UINX+S?rKNC;^z{f3optmqca_6(f0`(3v&9|y46PqYWs>fbdd?I(EwFq z?bbn=ogtjP&3+8M`dV?s^CQZ&ixXlq%AR7ER;knV`-f5&bI(jT*v&}I+RZOck5Un& zi8?AqVbcb$CziyhopAg;TbM-F=@emzxK$$d7H zrOuLT&mV1uodmR22-xznjJU2WVBdXb=nWp3H_^5`Z_xEKy|lSbHI^pS(b}`aBdw>O zV^mEOjLzh%rRpZS&#!vXF%%{Uly29~+bm;N4XYu}@r{J~8W!(?&{}0g)4}q|S(|d+ zGMS>&F)19o2K2xv?0M!B*c)r}XJ%*qd>BfnXb3Rh$~Hfr`DLToA;zArw|sN(IG=p8 z|KKX)Tx+iWZ?*>Q5H(y#XdNfthoxSdy8m6WIY)QBXz>&_{zu(`+;n5w;;QJv$%(x2 zw&R14P;)y>P<6)SSqt<2Y3X6Tn!Vigxr@4Vl@^8F*k^NxLzdn8PQusMj{*tBTOwO0 z7&t{9I9pW&ZHwCL7j1qWnyqJ|W6e&@k5H46t)b^Ne~mVK$Uu@yHhV@3l5UBt9V0Z| zV$CoaH$+8&Op0AznaF%lbAicvaXD{oOn0z(vcb>6Q=~oX$?x$L-NH|EuRAWT*|de) zY+^gNC$DS=Mk(eA-yIWP-==m1-3p>G{TD7w(fOOcJ*-q!)&(;0ssb~G3-OPp$Jd8O zQPS-n{G@Wtx>I%gY{gucvx|P$-{jxQwW)7T5e5FXNdi{z7q?ge!TnEZK3hAh0_KqU zPw$!1%7o6-KH4)*45I6ERV-8W<2(FEWMbV;ZnbM2XF_f8MVBmQ^tAeL3L6aZ+ss!C z=}$INjGt(mU(#XY5O_47H&;#hK)ckn)34Rp&R?92ieg;$%P%qDy7dg+(WeKNdcv5g zkqd5=4;tH_Sqoucx?hPc^pct5j3Dr_pJ5%#T zRUX$u1|vYPpHHepKv>!6dfW*cLJXKHyv12IP=kcJujGp=^Y*+XB_=ST}jt!VX4Mz0xf z&!c=T*CL6or{)-1JRGP=FHfB8ydsgx!SKdyp`$K;b3&hVrl2zZ^65mqsW;%oO0a7u z@Vwbx{19zH&0>bPCGj(OrDR@D`!WA~a_A$4SNZr8G4Cgw`C>Uj8P+r0hEcMpW!2m+ zA+?_m8a2$lo)}dbbNUoF7OV!u2g?4QMA_W#vnh#K81*Y6saN73-SFcpQ2&Rd6fcPQ z+IViFtI|M%O?i=qX)c6jNy12s<6iB|Njlas$B^W(Q(EE%J!x^ae5o#LGv>2j+`THD zIYlU9Hc;&{ilkQ0#?J;GL*9cCF>XXV!&d78`p!8l3JEs!q-)P$SLDVxOBAgQi>Ls# zAfxN^Ri`Q~?DBqg20|X8%nw>WohcS+$KvK-hMx9sLhb0hRoWP~XJ+l7Yw4vGhoujF`ocP5?ZzWusfY4~(tJ;*@r&fr zHgAST5kjs`&(%n2#%D~-bzSvKej*UAhsb4pHx51bkuO~oBp)}sjp{0;)~P7G^p=#) zyLXzWPu8gaCcIj3vBbr0jK>z4xpmR1J4g3oXS|pYRn*-mwyvyCo5hhQbPd9Sw`Ev=qVnE26`cO`v36f)4MgKA(<*~doF=i9>fZF}4lV)BxgJ8Zfh)>Sl5+WQb)(HC|& zz9xCniZFC}oIss2TSdp^&t01yMw}}9Dile-%bNJ*XNDYbF3y)$tj-QAt+PMyMy0j# zfnOa7C|OUY$c=6n(sLPjzw#@hF`_M2l+WWsc`(^b;f01Cn+lzG)#i%*{jmbiH|_0& zUFv_@%wCA7`q(ISWlYB(`*je)Db1C4uGjPveR_Z8R=74hGecZL|H+1a>nC|z<*tJz zbqWm(zVReJ(Z%z%I z6ItO^9bPe%()}AoQ|?>6>0MXW%g@hqlGaBhzX}vHFZ|*T#Yz1+#FUH6_7Bq-W1|H5hc6P;`uRyU4eMoO*)bKj zwte$Sb+~dTYHV^#3E3>NzHNs>c{#-VsfEc-&fUUiz2$hGqjZ}P8d)-vVc6_KkQ6BS z!`tS+gQsgX5mVWxHc-WBa~-8?l_qG^2s-&N-BxD{dxEx3>ORtmj}UMC>FS@C*?;r$ z#;@)NY#19Gn;!)~{OM?t9(s5DO7y$(R|~^)(rKuO>j+szw(UQQWRvIBkHdb`U;4E? z3&NCbpD*e$!Pid0Jed`MUM5lBGmJ&w4^eaB5%49l2&MG_BlQAwq;*TBdRM)u zf_koeiQLO^?yM2!p{oeY9~WV*^8KG0+=S}60N8`C>8#(HvRSilw%?pK@@&ZLbktMf zl@VJ`6Ej3T?zMc_#L|4nClzy2r2c`sd^;!mFoCM{0B)$jUvqF3ZUBYkJ`ewE}CQ z800`#>TtP#Q%XZEnc>$As$X7EElAm`AVIv_$C3;ensVsxngxP7a6CdQHJ6^rss)d* z3HXfnB#$;zIOiU&G&A-8W><&mLhngeeoZzG){g>DxpjljEGna(%jTQxdL?j=5$}Sr zTm0WD_PN-^yh*)G-k7~8 zRTE9l*#2V9S-!fSN=DazRxQCr?zHk?4IUGUtTcdHjG>o56d`*mOD#|XQ%y4PM>U_< zs2ZLus>-*3#ZkVLEnXs*+d+pVG*XbkRUi~>VcjaJe4nn7Ymejk#_b`sTRl;F14r?@ zd7XJui%uO9v+TI;Mgid#XTkonY*tyucKsHeE~=`vdaiTPtW_i@ImI42RHUlI?l4?x zothtZBjY5tTiYyJ4i2asIT;gvz4jhB1n=$pdgW+B*?Ie-h46?Gw$z&)@8!F7u2x^P zZ4AB6u?k$oh@Q$&j=okV8Zv4wm@aCV0PW*zIc;> zrb!0LD3k))I+{TX?^sL1_4rN3NeyYh-|Ek0EB#F6+NcqI@J_{QYwq*;O~C)MBl{~~ z_`H5O+oP*q1qHw#EOT`t^Hr81n@1%`d(3yY0#(~dp%ogX8H-XGk)I58+fb!_V`NX?7Z^V%0 z#ae9<%;!Apinc0ncPN0y1O=(O7pbYWPcQ2~*mQ zkM=hn;0hO#Q@VtxvBynvre**NDs-I2NNxHPm@0q^qZZ?v$J4Ov z&Q5YTKaO4T?PNcB=wesmpZY&UOdW+{c-XgZ4s3?T_;toOUZN$Z(sg=(`VU|uiZaKt znvjjzoEex@{(wBV(hPv2LXvLErcDfciLd|Id%5QK6S zS`r>w(*OX%TtuulOVnoYsaE$#`j?nUpCGpp679er4*G5_$ypBi_itD|OTSAgF$$Nv zMYEZ#cRyd%{Cmk}TdPC_fHbdjZ+&U(cO`m~CdiGt&r)*-@`Xz(@=>`ZPKI(+X?h_q zK8!bAF8+b_Isp+OTWZUhKjGEZI{u zdi;9)%8Twsk$3mM%9I3UDl?u2&D<*^t!E~F&IQ~Z|5QF+GyH#$0}IA*0IB@?qC2Ok zo{87wbRW~{(lwtR6RCn#UWS)9x((NQh+4?B+!lYVCr22E5^-|=Xgt&JLIqjI&w7~P zozBF@fj=%fx}|;y_9NA@${s7&dI(v#2W$kEd5xt;_0wm*GlT=bxFDK_vUhYIGhVbz z1TC17G=Hi+Y;m@>O1fD(CFIV3YUFXJST~1aCdY_}y4Hn7zd?4uR0QeZB^K=PCo9~r z`{DF_2g`;#2Iszu+4+D@O2o>5Oh0kWvrBkrroCiMu@d#IKhcH(j$P zp5nwI$#%SRLG&^{vpUkq+qpUx!={3uo2+OyvBWkKM6$Mg9`8SFQ`--{BWQ$6g^n4<%=J4Z9KHo{j z9;TABvua;*fYz>DGbk7r+Duoe>r}}6er|H6MQ9H#G|TaFjvmFt-{RmV_%YI5BW#l$&_lx&*NB9m(0do zRRx`yAnN|L+&}+1=SDM>7$c+3V?h~=n`{aDU5v`*(Es&OQS7^VB->?)75i_>E@#bO)7LN@L4 zoR#*urvpGh$dz9-2i^x%AY_%dnNTYk*5y7^@l_EA)nqtvqeAQJmp6jQKq7ROIodD4 zx~-@+sI0SR%VW&3&(dbDc)PCnJq4n*bz2C$fGx{RWtp0#m8im2mI0`viel&6N@3-U zYKlft;y}rJP0Tb(AIkbWfAF-lI^*!_7{*Z!{xOgCUdY;Rux&4HQGDx0t{rVn^4J$x z!V$>ZR~Sa6*|O(J#M{pCZY=o)%#pCsfpd(FZV;|dzN@Qj<|{qR)VC?Mz(4U=I-k~0$Edx%wA($RrEprYuWvDfX{uGS9vAZ9e2YxC3N zLUZnRYf!Eb%H8sT>otcL0-IkTQ%LTlg@l za-W$i>C?@&S}wPAXtX|@?ygccw^07GkkHwa4B3A?BIbaOA-B!>U+3A+*%X*<{H~g* z6&eO$BQUop=PStS1?M`gF6Pu|r-BTx_iX8A2P-b>v^ygz&x|)CpG=F@T9@_w=C_Y$ zS4aLlTC8y%ELH5KKou`oxNNtZTn3emTb2jqgeDn`5Bn{f#9;r7vWyhE=JPt*<6Le* znI@UYxwO(v$;`%~c0Tfl$L%r|KVqLayV%Ql*~#t~6HGy<{+9eDUl1@eOC#5Ms>7^C z#&X6v()Ig=-gW*CD4*z_Cd+it%+t)LI(Oen$8~`tP%>OqbM!Y&)7#eX=>rdvq!zck z6h-8-z!4%&Hhxr%O^!<%ZpD`7&%;D*M>HRelmE6S8P-l=j^ehON*k|`qG^R$b`d)S zY8cPDM=Wf)ZdBr&iY}Q8aUMxi0j^OG!h$v0eLc@O#U4oaT@Y;eApr75PCUbqgx#C`LBT$C$yQ8}E9-8#s#Xb6eaKhvgn@i7 zV;p#a7fhZ#7RpsB4txtJv3w=l$YVSya=9=cwsH5?wBqq7fhU*wLKECTcA6D}-Q^%k zNYT<5H;tAIdC{jXN*+P&a?eilow;An?X4+FbqPy8Z?$l}i|Ms%!Dpt4Xq1d#+KZyx zSAL>a-T*gl%dLn_XO40)jgJq}5kFpXgZVz4&O|0guc_8k*!x>zNFxQpBz?H5wyWr1 zO5Si2`%uy=lv>KlcHFZNZT7949~s;3!_ST(*RY^;Lwvm7T2@LABSS>|vXN8E_8%{6 zKPKyZ{uz|}J=~JDWSJ`J(#;3IM^u+u6OVC<-d73E@O?6)J@u;4tY|P#G%coNT`QhH zF*geMt+>rwD(ajMZ+Mh#<3I`h?Nasa&;>_yxeUOem)f&KVcluoUa5lIRCyb1XPr=s zuSCJC2Pm{Zf8%7_ytqA8+&SM7;9{&1nbZ3qf7+KWC%~Cg;6jUuBx~a`RoWQ+V(0c} z%fSXW=?YLwQ73i=I2MtK^ZcBsaH4gc00o*sM#}{>(aK?`ZI;r#FJMZPdgMT>nn8 zEoc{Yc5dKH`d6-cK7~w7CF!bu*X`>m?02Vo(mR}jflPdqHwaEy@_kmA`YN{uyyc7( z%4J#W+7fSV^z!9XG)+thi3C3iB}(%~r|82>whF?Nm=LkyWl#IHr%xoYa)Ug)YkXrY zl-g`~^yqdZ|5&|-Yj?D2SKIPHD~vC`-9$2B?ZsOFb?Q0)K;=fE1|cqvAJ$JEv%N^T z{n_#bf9@)iF&uV|cx83{X0}vtpl}JOHQ-XanD;zf`eic{ZC)`O!9nP{tjV_=zunz9 z9(0{l)~64&BU0)=s3x6VktZ9O=1Vzy%k(>BZcqZ*BqiKa<3;U$_S!%ofQQ|}Uzv?4 zPllp&zvs- zwOQ%L!8>A`@iwNrehx1|!xueHw5(Gqz0i}Q{hfCmm-l1?qUIyY?GvxBz}<+F zJsALzTu%CMBE^U$S36JZFuQf?*O=1n_YJA7cq6WUrA_nLZOYn-Y`?9u?v z_D^Jy%SnnoLAh2FX4`?eIv2oRvNuZfvPRE}m=q%M&3=1SSr^fuGkq%q^+rFg9mnAX z8M%V0@y-g|C=<>fQlQuR8D`7+Ot$&R{SVz(C3>*bEl zsU+y+^FsABRL?r)p!Vr*b#Lb_!8P0EWf&VWfpl5dNf|16nC0qeYc&r5{}PdF`IdgA z$e)Ce`5KZh2GnV>WRpBy1m>bvwvrlYQ@So&fO14t{Iav$X#v-Dl0o21D{US0f|HWY zw63QKQ=9Pyx>!ayFUE{Axz6C`j~2LGV>2hkh`JJrC@|yijn|2xh%zs{Ss*YOKFK&c zQnXr}pBf76WOWY0SS*x$Uf^d_fRF@+wmR+Q0^qf323H9feKz?g>dtQcv4je<6|&kF&yi8)$qI7~^gL$syQNda zpr$G2q&q7E`3A{XVDB6`^UFT-<1f0oLFQ1ELtr>rSKXiO#Tda84q*-`#dEg1x}QL> zQAEEl>_#O2Wot*)8`vyRCOEf1so>QnsEiJV#*_qP(SC)B4tcoKQ28m;qa zpmMgo+M)FmZ%(7wqe7`L&_B0D{OI?rc>dFuS@&e1y4ZGKzH@xL1bV+3%j=a{lQYkj z-$#_F^wpXPHN4C%V!=UyRLj}LpI-7m6H2hMJ|YUj_!#PeR*8fq^!J zYQ2LMC#_iDTna28xBGH=7Sy^EdNX?9^0B;d2=C;qo0I~$y%qmTb2O~Dn8<2Ykg4)Kjt-b%h@^XTlBBu zV?3EkV&_@Ah{@#orEEc#2xqbGni%hj5x(-rQ8I@BJR8hPSdZ z9jYx{yC4A)?~@S@Jy6}~`&uXFTn7ui5(jMDRr^Nz>2r3L#e|YA{vVU9ySYynPrZc# zt;^djy}m5CwilWrVUoQLTH_`a{tv&WY~_xg>#R!8Eq?7B(~vzt=s4P&x_OH|Lr`h0 zcSP(=U!cQjkRy2W0QvfS&Xk~brIYjdSJwxHjmo%fzNc_Xva+?UgSeCu$~nEA&~aRI zEbS4?@qwzFz__!%C@?d%DC)OK|8vT`5=xai_x_G&GyYZ_(2(7@{JS?a+6O}UX>$AF z5CQ{PTSI5;d4{t|&W0OjY_GXQ=@!h=@7@*0owE?Q=xAPEg_UlLzLje4CyU9H(#$7$ zzMiraECpH-`y9%b#Wu*F#H{G+*tOd>uje@%J)u%B-_TVIC{1&;n#u0=B&+l>5j!8} zZnjFXbX~Fad4!6=hIJ{g$uj1;6=Gs7Z_|*l7wagVRb89LgDrecq`<;{d!=IKJy47i2a@R{(-}}!cJ1MM#hNo!Sgt5Z$x@pu zh*NfZS4<_5E$0Op0$lPNMGb$yYAJHNySHc6R1PFl%!mXlb6Ha6<0hr%PaiKJte5Wx>l!nL#pmS;<|7hd9#TMw24GLxeCgJP)hzP9B} z!97KIF7>(nN{3=anVJjzuCq4JAK9CNUaEq;-wRKM;@LyIq0S2{v1RW^P!I~XiL&n;c*)7vVGTtH#S~V*~{0T!s6L{5-El$XO~7CR1PK5^PtXy;gl_^WFowyB1ucs#cgVQ% zjclw%2Z>HHD!ftZ9|0R9mE;Y6rJ9#Zt)C}4sq3PD-hI^FHpz`6d#&;^$WA0>CvO{q zl&o>(&(Rl{A({eeKTN$e?30RZv1!M-?|5pCd~}x%(t2P13IsToEqe9at%+Vl^#1Kn zZz{LJTJ5H_+UB_hV0So=f!FR}12KJ7$4;h3$&!h~sk4g?!C{)RbitRVLIONwpr(vi zdwAux=mjZPZEmyMNJpXEhNr%!ZTeNj3gnt`>l>GQvsC1pEvlYVQuFoAD#w#>=JZY$ zXgEY{rC)2L7+RR8u4CYdSH1k+)Pq*kaj@K5I>wC;N?H6^^@w+P3oA3lV+gM8*J zR8C4#>_aEvh3(qIX3KtmfUsTwWJEzU)ji-gAN8mEhyFatXzdo)w;TED`XI*PgfgVY zWj4;0cZzdr8N4F1Clf*5QN*a}S=H~pLNoiv8zvokU5+!W_pF=X*(`=rphqmtUNva? z#S{gw-f8wc!SGti*f)XYcf&1bPNaWme5X!DO$-{V6m=Yjmu4%xbII*WD&gR8vX@rG ziar;-mZ;A&^=vwUKDX3KYqF@(3FYOQMT^V*%50J`7>&bfI3Gt@@T@}A#7r%Wu0J_x z3q#^!ea{{UOrPN>jvIB%tzua&rErdP(@oBqF1fsX^*YIKRI_Dwj$g`pq^^?N!g4=p zvWj@@SBxqz(M4X z(}KWkU*1fI;bpgtrGVA*52r8h0q@o~#bm>ax7<38((nv^ybc16dEeO(~ zv^3JK64Ko%-AFg{-G>s~b$8$IzhRhp=H7eqp7T5BSW1r+Nbn-Bg*kkGe$jfBHaE&2 zSHka<#g&2|O#*2nv+d<7g+?B}BnveSn5G~T*%g~R-UBuFX8_7 zkc5Wmm=Opy3nnuYf>MxA(4Rxx?NN&T1FSU^hK$v7#7$3IwRg zjXy!~4Hm<$(In{r`4b2{LsNnX=;;b2 z->f3nsuZV*&1$lBGIY!P-#5>VLm;9M>($F?Luvl^6h22#GnO^!8}|A3PPl=d!yF*h zUGwQ%lXstUFH0};`TPwRwl$wkjZkIVvI3E0teM!Lyz7HvCN=rPt~O&qJ4U3&FTc>C zYf{9BbK8w#3taco9Q8qe0bnD6Sb{EOyi1H*Rw;pJq_Fm`grMLeKm!9K=WEQ%&*%2a zi>l-Y5{w<|F7K$0g#m|J1D$tkEniSkxe~R#mxD`I3HhvJq$GH#>ayzVO=25baki=N zY=cpDL#t>wmv5Q(nMu{u7yY;$HSF8oze8HIc@2rPSm8+@!jwKci7_&hpZtYsmN#qS zsxy(3L?50CO*pUZQVv_@Tx42rUwcK0hGT+~Ov-k3%4NbrPa@px-o<{l*_zKL4UJaJ z944Ayt|NV)rnkNGAN5hT>_Wz{Eevm(2{C4$v&~dlTpx%u*6_M}dUQ|-jnpF4eB1D= zMy=2gvu3OJaQ^5!e?E@vo_&}TU+!^1562k7QSf^F2C9arFiKMN6jx9}G ztWmv(doH*NR&8Ey_a&f_7s3%lbf8KN3U=icEJ-o7vuxtY-Agl|z8&M6rJHYgPEqE$ zx|4+8*R87WcIlbClnR5l%!GLso40)KZ>Z=b&#i+%1*QvGZFGVv&_=!qPjf(HDNA5w z${b(Tc)`)oq1tp5ZNnr>Z^wuiz24oFxrp{kaO4P z*jz;lrIz~cx%w_W>CC*l5GL3ECA)8%Cfl^qbZW&7uZ@#es_CwGvt!n)v4*pZA^i( za!zGDFWyC!P(xZ)L6=+kMD`+EKZXLT%4}AfGrM(>!3&O97x`LB?WE2j|Fe<3zCq*CgO{zFKNs<_2b6!{7{*0U-j%rZ@-jQC2Lm(p~}Fe z%&1Ke!rdy_H^g}R)S;hST4iQ&@mS?7lFT#(H$LiyXtcMbU0H?%-9(+N;yF{x9aA-z zH9kETcUkI8Z!k?LV`{dDTRj_IycuLYUUh4-R|T~9*}>9jPu{-D$+y6qWYA>U6r$Hn zA-cvrX8^%*AZ(KoOw=oq;Wf~t!-lb`FHz6R`NC$p2-@wOggJ!;M-E~EZ_v59)F*Xg zgMExbML-z5nW5MoxB4Bnrm$}C_Cobn*;X}SBXcXG=IhRWUspBtoy0RrwMTuo280$x z_$TbH6{bkRgt}s)>g~sbuT;ib;0%RH-JxE}Hr%xfGSX zm~sp(++qH%$G2vWMX`C~9mcars#oVP%SR(4CG1kGOgM5jVhGG=-J-Vdbc1Zcg*nf) z7)guI3-d#irF?SS1ecd`NCD@?887yaa{-m5bVo#3Qjit_oSt&e%i-Dm=h_sm8`~JZ zq?Uh-B%4c25=yerZS|`Nv&n8_@uPLxV0ZH26E4XGHqkG*Ef3kx%IkH`Du?D(;eya- zOQM}WEk5Tx@TSY|-P{l?pU8RN4Lqo;1S;zWNwLu=;U_&mh8ru8k0uVWdgiBebF0`4 zrmj6%yE)$!SvcsDzv8uN+(m1Yuh*_qg$t;EUMvLOP?~j zB$AkS`=QniMz1>nm#eU`x=Qn$_4{pM(#?TPg2j~%T+Enn-DwA@Pq;Tm6pBxVh0R2? zU?rJYF3hFWo3waMjE!6QAQjX_3KgB1ojH4j?S}1sDLQM##f|Imh&~u!^N&JI{?>qM z*rEl1`~*BRE=ER|>r(3uRfO*pbgq^8^LoCmvx(x*w@1}h`C(l&_)JUHYlEGoe>sS4 zbC41vt5VKewzMw)J%orUXrY2v*s|aOh>$Bc;Q;-lsyEUe~Vs4PP*qWK#lQx*{+oT3mU*eYyh8s+f2pzE`=upMj&kHM*7VRiB*$bOpYdRZ?|FS6%m| zH<4F96O{Gh$N+FBI6_?rTSQ3JOnjsFkstl4I{?IC0W=+3(`J6wliA{F-!)K?lgWI4 z6{^l|Oje#C_t<&}0$;#zD&L2pOj0)}2$4lPUBrVzO$A%yq4&oNmFlrfYs2zJeAb4Y zQbt!2oB|zVPj4c=hBO&($>5&5Jyr#nv9QXyOFl^9G zLl2;jpu#Y0eqiU~E2NYbSuew)X@7hd=x+i7=Q}LWpRS z$V9-Xm`9b^VWW2~P3nt~;aC(N|9Q5BI_pBu;n#N@fCi9yCQ`Ja8)Fon!e-p*q_WPo z!nM>^U=jbL!9mVVdye~tCx_-thu%>1;GmN+ooJ4>4#daEZ)c1G$SFeImkV=IgEKK`0B;;O7ZdI>2Nojj=4VpV+4 z%fW~$;6{315XP7~?^vmrS=v>rwY>Q+&z(!lZLqbaNF_ELuKjeq|pg^P@r&Wa6z3z26o8-$x!{zy{+?sV#Y z8m9q18Gp^fe8%Z9z5U~<)-9&Vgo5W1yo(0Acvea2f%b649RM%z>MtXAHn@nL|sphkDZdN4`bTP8Ans76`>F^+?ZLY76AMc zslM#}YE=>pdfZq|Q5blAP~`;-N|K!we3T7XlnXF%2$6dU&3>zMmH9(xii;2bY^9dMA0)c6{q?0CLN9m z0-Gx`PTRqA#Dqt*L&*hhY|Extop)g8v8&dwxSIn%ocpj%aAo6O(=9N@_CCAw;W^D18IX1Bn z)0R_y`GE!DK9UqyA9TwbeD5+=nBY~@331%fm|q;{#eVlOboLFvV@3swvv!RIu#FFH z?&6howVpu;<(Qc_ypK|-pKlpXr`Y71eG*i`e~I$(OjK|xCIoXrhaWhj!VdbxPzLno zKn}MC!1rW|Yb?*KxYc6vFatkBPCwr#j!6hsOC>oiq;B!DXv?;{Jeg8iKD84{+r-re zJ!UR__B@W{%>{%pFEVbJRH&F>I5^`)qdiR(s4Ug-2!H$aR6n5~uUS(SB0I!HP|2>w z2nraFh3g=iwC4)`G-BT#fMs3)hAMj(es%^o^eGy!0}(G#IhLm{-4gGL(d@YS_0`K# zZ|V!?2aWb3)=)q8F+bc|q~Ev{M6wqKexs@e0vH+3?k;z&@32TOf8ZyQhr2ThgpvZ#|V5 zp-8^vR3kHsqSO4du^3m?imi7llRc+;IOtN0Ip)hntlV4k>%%P5&@t_ zc~k5I(qVBi=lhC3O*oGYFS+}bg4K_WF}1@VXgIximc{*sT%R*6;_ehFX+^J-CMiiZ zvh?cM_E6Xd*9v$un%MmpHZuX=J4JtTn5bbyP(sUI!#@ z-NYSw0=9(6q{VJ2yVZSFL-~l|`^6+5-?6x@eK=Tx_ZzqOTIeH*R@*9G%Q+ly#a)UD zA3}9}({7t33>zVrtPxUQF2r`TKnl?_rCQJV1;V2$rS@;!1Z7M%Q?vvzt*0Sudk@I3`ne@c zTZN>VnRgqs(`cPPlM(eaZ4ZM(eI^PpfU?E#=FYZ8%?~gyHko}t>w*%b6mf;aK=J%- z*ZmT69R~!k0&lIbb^Tlm@Xpa~&2FkLgn46=qLW)hff$!@KdVupsei%PdD|Zzgb5lC zjMz5(O-z0?TFEecOfPON7b&~-{QT=eOQoA_$`0=z+GV0@xOu#D@6A@d2Bqu7cg8kM zCL1ryvd3XvNFFl!qz%KL+% za80Q-xXqTU^});_$n?2rD#U$X&OYaLa>qt{+VKayNnM!)z8<~^1BU>Y^lg9A1sK)q>7T84EIVpZ{-d6DAXs~%Sc94`6cls1K*gGy4^tm-q> z#|zu>rYU7WEoIrwT63Z%BWZ&Ts96W312h##$h~SD%kVrr=c^r7beD3)_Yg6B*VRCW zn_H@rFB&< zXA?LMRzrcgy5g+>WbK$7RqU8@l=?jK6p*0G+TvGBo}Ks zLrGoPlpy;MAM3D>=p^7Rk9WB05xP>WYbrky>f0l|&erd!usz)LDa+HrQl47X!Y`ev7{U@} z^_jt=)@9?(8$@AwhOhrN&d~p+Ik<2p=jJ+Ze|MH~|Fp*Cr|@`;41HFa1m`zKr#62q zBTGT?Eit8G<^#xrUsNQyl&GHGebwQ1V){rJp0f&o5jn%t$X!}Bs%(PI!xBXL79OyT z640K%e)=*fJ&g%lVYFbT4%`6c-4FjXV{UB2zG3vTB!h@Ub~>wLN-DuvY3yMNw}X?C z)s~7$`FYy%o0n^@YR`7iLwKm3@N-Zu7{bkFu0R!DyEB)F>H& zn3=9jJw9%)M3%h<&pNkH>^2hy=~mG91f9=eA75jEZoe#68|6KM1dQw9sF!#p#y^5{mDcQPIt^*W}l>1kK+bE@D$ z&O+Z}o2tI|L+#9$3vI13Z2DuKxqgaLcO;b!6WXt)R#fHs$;AZbb$JZ5UY6xx$Z!?q zdkaeb%JB1*FJmNH^E1ri+O2vpxYH)r8^80hKi_HB>e{UDAm&0=v7*8XVMP>hv}iak ztyED$;eUYSC3P}5=zGLb$JC9S zlW~|}nhKG@5FUs#?6!2ON$+a+>bK2RZ~<`*@Cgx>7nihyDUEH+@au}rS0U)%Il@6_ zW1f^-E(IUH(w}Fi2gMMPu6KVnZ90b=0D={fH=&s7Tz`o-6J-PZMD2ckt1Q$(oo4#f zUe&<}<&TkeC(!oK)BZc^<0N}FzRk#@TIBVO$1m17}{_3K54W zQS6mWN4`b=LW-g8l&;J=8Ce=D$5OymyqGFbm7$Wt=FaJ>4ws?0)r<#CE5%fX49&+h zyFQ^{7VG6;2oXYDWY|BRtC&r=Jwj_eKjLMU5Mq#3&78X~r*eWq)lLf~NHIcrvrW|{ zBI{y>N2SwAy?jy$%XoSU3ls(6h~(H6mI^K~JP`|1#l%+03^-_@oa{1JDdghyjv;A{ z*oWR+e9}W~0$MLb&hbo@3n?;OBDh*055exNK@H(@QDfp~oF_2QxJGBLmN1&9w67sWW*aQVgIB5y*CdAq44@tJ#T3%vMX~y_frkGDnKrjQLD+; z(l%(#k1U+OF_dUFLeDt*1ww0B*;3mu8Na*()1tdkPbxCz;TXfNz@vLzOcTT@+>2Av zgt`pV4M8zjX`6E>9&~AWyBtrpuLz))C#Bw(>{bjU*EC+8b9BX@SwX_UJMdV5KLDLHUr2ypL%99UU5~6!^&Ac!>E-w>S)_s^JdB)nQ*mK8 zd?q>Tm;4`^HnKMv`OO5wDydQmmu>on-FQwXUYO}_-KK3zmE&>IH(r>X z`SC;63!i6{cyZ%gX3jb5>7gH}ldHMREL<&Q6;n$DW!$8Fs&3mQNoA%I3m|RQhio%G zI=wX$?_p|F(n4&6OU91?E5D2Lvo`s0&AaHdk5=?#L4Hcny!^Ducu9w5Y{GS~VLYjh zU2)b0fMksYsSJHyi$2qrC}*~)f6Qo;A-Qzk7i!Pt(1aBw*@WF&g)n*$kTRLa)*mXu zjD(trys=Amn=9IidimkjS;QdT+RCC=ke#W_22@c96_B3=EhusdB)ryE#c3`e-p&m|DCnDJy#%d9Cwzn=%-@&_T*MB1?4 zW$}5*1jA1fTZ`q_8ZDj^OZWJn-v?Xqqp?vxhlSd%82oT0lf7Iy!20aX{=d=*;x;|Y z(n;Fb^UsToeWzHIyGS*MllqyUa+T0?0C!hWUhc%zRKYD|D z{<{G4SSXdqTjx)^=ZnI$+R=l&N@piS-m&o%fD~qw!M7eJP9t0kt zyRlKRRa_xz)Nqy)loV!PvaRjc%G2ErbMtijWGPQx*K3vp731$-){GM~f}Va&M9VPu z>0k}Fm!$m;)xt>h*xzTUoLZWZd)=HByrk`mv+5#{!pZpREFsozCL^C7ob;+r#%@=gNZx}P7fLTqq~EgG5P4hsqdG-jf>7SL_N7KWk!+$x0XacJ zZqqk6#xW=Lxl)(){2;@3U2+oZGuiz^Po4TMk?}hQ7eD8U2KK=1{a1Y+YI{&1H5e&T zoMKXrUcJREJI4dOo%!?gEfFV4lLozw@%(%n-pH2aAY%##vuLe6`_o57TTjA(2THidLDl&0nr?3g_YyXV=?) zXwq5C?|fcH?z*7i3W!PO%r_BfjGsJRkuFjf%2^y=lwsV@!t&6rF=uG9cktZ~=h&#E zTtD?AO}E};54RLY9>;LWfy;@>;w?BjFQ77XgHm~Vt13`TF+!AxYy7JE^6NV=v1^hM zUh%PpdHtE_XYWT^Pp3rc`)1zz%o^^#3#k_GFVgSXm{XYt6?R#rspTY+OfdS;>ntnn zUC1pNqEV`x22K&b*Q6T#j_CUAZ-BE zbwPDRz88J4g_@@BR43s>#s+E^VbT#*p8~Rc5LP){m!4BG@&eWE6ViKD#sfFNa(zUj zJdFFY`AWy5uY@fs8i#Hw1-4F%Q5TDJao$0#qbR;$8jM+WQ!qb*N=q*?iTS&WOZTSjIt*=!=O(%2Pr^K~)<%a`FPq3#hyH0K zx)YUdITNdwE0pm=@D*8uB*ODn>lP8kGN>bEZx(Sg0;N1%5%UlZ5z4{8<+;S3r1qq> zk&{)2m=K4`0`X29dn(V-3;&u(_ywSM^hLYhuMBVD2fE(|MWeFdX|>pP;TC2grpSDg zDu=mJ0Is_^G52r_NA}Z)sSx-r^o(%ZpoeOmGV05QcKlG{%=aRMA2ds`3$wXvVDe43 z(P6Pyz0GFSxLh9Bsr~Umk+F^t0_66~GnA*0tJwfdXIwdP$!+i6DhpRbapf!N&mpMx zyMejO8gCvUoWt!Yjm=}jR&}CYAV5Ze8M;ieO}3m?2g&mC$N5S)`i3^T=G{NO>;ih^ zQWPUqk=p8|&XEf%w~Tt`=GXmsN0UTeRuE+w)pY3QWhhQ%4HSGDPD1wQtF}^wztJXt z&E!0We2aA1?sq37+FZ`%p0yoVCUlU&jcUCqtFU8?=PaW#=QNzOmZ;!9)1B$zizOZX zJ?i6;nH>qCTc6BlvCQf_sk@l(`v{i(L49qsM9!&0O9-x8PQo38*c8$Ey7u=mZ7r)rpBOo8eAYAv|l(*+$#^2jhZYjjGgx zB8I5Te9h7>n4tE*W5!oseVg0Med+oj2WL_nNfOt>wp_Rjt%YF35~?e?kO9;~fH|8t zRsOd09r=yKfEi;9mMJz4P$Q9p(j~!c5fxnPrZ4;WZiEVq5V9k1e$-dziGmuxz2JKa zn&SXpbfcWRT$pS*y2)&Iy zB;HFrw^V){)?xpgyvZn3adX8`qceImxzT9Ni8tv!iv^?Bw zj`ok>v<-a!d+&z>Th=dC7~zW0aR@NHf0SSAFV4rD1*%Fj`8%cDYVQLj65=#+Ya*I2 z%U^YqiKTmeGH1Y58^bdj%7uyS6w1&sIqj5&xZ+3;HUtH$VnfNev$o?Ng8c$`7OkoT zVNPw6e7QpzQUb!`--wt1w{jK?z9RTgQN#*>xZ!x<)*}}+H^!@vF&8~od6}W}rctqN zyCB0$1r-puHK}k&!TTsH9#M%BGt-b(xuG}d30-0)%IEC2vkcpLRMY;1X;Hrk8|c=3 z_c6hhF0ZNUW`9$!LXXDh5m|8W6_D-GsQ{py(}Tut^uenW-zWGR&xbztlphUalsY>R zvjkXRsdOGhQt^=;g%T}R%x_(A9F6DK&Iy999In(tbmSJ%tD|AsN>?GwQBD7gaCT7C zsCnhd=U#80nxO6nNDycG#u(wdmzJQif?DYEK_kHgyohtG?B9o<|14k zh$3tE%6ci=`ps+*lmf{_E;YvLCHwyAL0~ z)OfDevF`JIcC8&I30jnL%N_QCm79K648@Oh9!R`;Xz;$1hBdID->IZQM{lz-6p5?@ z0vIlGrb}J0a(eS17Lct!w47bMhHMIAMDiF7<;;9k));c4!(Z)7)t)X8+lhTI7ZSKp zxYTvcQ%s%8vw%iRp4Ar7x99c21)$3K2?Ve{+mq?Ei0^!12DSCC1T&7_%aLPj8H zUY6jlFA0mLxS?Djo>tGEUHV12{k8mPAa7gma%uq5DGzJ)=w%e7435Gc5cvb#?-0iM)j5WD*gyvl%>NaL2xy6)Qy!H&X?S5 zp@5yrt~JDK*AE5=n$_G1--CpKcTquODBP&~x1W~yqFZ-W`J!_azlU{YCzn8IJZhoq zNf#FetFN+lYjS{#E1`qcS~Zi6ao${7*`H~{AI{}B>P!HnLL*Wd6pyz@Iwni;bkh+< zF4hoz^p)X-ii{`um^~#xI&<8tgixl+U)4738|$2}wJDO!SVq^bK_vk13$P8sgRd+O{JeBD+b$L;X=o6e1q6{9noUueTU#4Ph@nkH$Ni5SbCl-YC- z8-~gDtWO4i?6@4we?NBDL0?G0^?(p6ctimPi@*7Rn1d`nxAx77Z0gJb$5Mo zTtRlKVJi)$+Hj*V=@KJAz+>Sqa;Nqd+F|d*GZ{Icju4-cl?1VXC0M;IOV=8_a`{50 zG^nG!iSXaq6#7(`0|-x=^vxTa<|F2k|3r*ycXCfO|H8ewuml-ng)tL_b7cMK_|)DJ z8!l>fx%n_u|+B))~LPV>UDp$VWcq8AgVrM6!H0ETLz&cZyJ0IsJ) z!*ke#JQOBA8XvMzT)zScZv23iEoKkmutCrhe%>+=W# zZJbK;20ofKK9f77sNOqMqrJkO?~zO_bTmDoH}Fg(vOv&>=S5=eD#|rTeE^yiYq8 z2tkvmAr?c5?GtfDp*lZ%%tN`5Xd1OXkcszx+RE=bt{#BCm+}BfRcMF;R>q?t9nMxNIHBQ;3uw*g=Oe^#O#I1LbK(QRfI5GB|^+rE)DJkuefGN z@h|7O4jC!umu3ZT7Y;sIn0hdyzq655$&2shCl_F48axTg_P6;A`WIh8zO2a_+Ulqe z+^AqETItVHV(0Vj5JoW1(E8D|({Lzg(}bS%W?`{j!%VsV$)-qddsh)iiZRQ{M`Ten zYJ~+3dL~%alAmmCdos)ZnCjtnworQ^h|}XIN~B$}S=de++#o1*6zI92+xiCAPT}^* z0BErQsu>>WyR4qg4p+bK? zTc1_X(Z>})ml-Apn(QGgMo&nyQge^p6^q*$;U8C4)$X6xGDZ25o`mh-j<`>~gO2G4 zDg3xOJy4Hxj&|fmEo2UMEkJ}1yf*(ltqgvT3}`)$P(WEoq9fIf!oMMdl1|bV_usVG0L(E!5@!kH$TtL@4e4pAsxgnl;YQmRly zn~PZo3B^ViGU61>L9VnM^5CU%)?bOpvWuUt9R1Rq9cM-i(0UX`#jC|oQ;UjXal0E* zFzDXuuXuuNx%q)&o~H%5`vZ4okT5l^gj->DaI+|BH&(h_tfV1tk184s`y|b`Sdg+Q zktb_6lcV03=laE$YXKlZ$FQ}erenjA4|{H{)vk>4v`pZ|M-W2aH6||>sWw*oz+~yYens>16)z-@jrDZVHv0YX;!B)j z1lddd>R07Ph${&+;%?_9p9#-m=(ypghJh!!$GW zO2pC~LN?X0Sw=$99}J4GHBGs9qprVluv7s(+AcSfkbh6iFTd)-ro6);{h7g&+n~3{ zg03S1UY0IWOXMF~Xi>0UbK&Mfw1#O4^$98m30yWUQdxg3PMp1`FT#t!n6(s8MErSC|(&=Lc&Zob1x%JT`;Lv zs_T0xR(dlPqlSZ?Jc)R&OG`PiE07kbM{7 zwN+q9W)p>z-KQD9B&8wZ!dvT?A*GSZg{mtP5smVon7X7I5{b3n<*eR4M6*bS!7%$~4WgdT$wC@~lqedCVtH^}w zQSF`Dp*&42UzeKs-yua@D5S+&lM&ZglovBNDt-vqmIZ=qpst_$tECX1dk`Wfna#M{ zdba5K;5_H5S7??BL(_7?xVDEM!E2z+vj=aib|sAInY3KfA6rpetX9XUC&uj0Gd>F} zHlbOfzKX72KrO2hDO4qEJ?$AbKRM)F_I)$uLg?eT?M>#+>yiFfr-*dR0-5vbRof+P zrlgyMAV7G+h9QAOPo^r}r(q?VZt7#b*?~^H0S3?uu(IzT3xso$hl{U|dUtJbDIG0z$l~x%w6qavM zP42>Fv%2GrZOFi8=my37E1oUZVxJS+3Vb}d;w>AJJ39z`vluf_BR=XQp4GWD=RNvq z3EPaXOX`al4X5>-j2W%3lEaRrx2&KLZA(;ca{k60?_|{}G{V3Ga{+|T7DMMCICTi2 zI5x&w7-|6Tq6%*Y(G4vWcW@hlIu*5Ni5Zo7C}&R*RuB~#DFLTX&;@t3KSSTAqLlF(cogGC&TRCn2v3{!!YVSQEBdr*A7-(=ay zWIk%hi6~E;K}fTxxHg3Kc1)VISi(eNUXy+9ix4x3E~e~-O78Z#3%dEaNOA9~4FH%k z!t)1dGpZ(ZYU)d|X+!K@2U|OSbK1v->~3sp_uj;7X@3xF1@jSsCtRbdeq!{3p=gSB-+lgSJ9X-M~f z6kI3S!VeC2?>LW4@A%NEEPD_O_*;dJ<->1}vx~6+eGbp;jw?MPr~ft{f&>E^?PT0| z19+}f*`L8T8&F_*2iw?(Zv1AP%dx=@a103UaYFMhXo3R&FT-%7$)K5YG;g6cpXp5q z<>MrfbO)wAXyxO$$8S&O*lDD?qXH{nVrSnkq3)uO8u>e;q&r=FJ22xHpDY5Q^}9fF74}DD#$b{~Dr?BYFI?1b}1zAPoDaR~E!p zG?%Mf-QXH1A$tYyKNs`+?;|L%F5a@Y###D+aj<$s8<-u_)Fl^jsiOV=IWqT+17g%w zvPsaaa?<}e_x&$3=Uw?BO`(L2vsYxgyV3t`<~)1?j3B1&)z=<9q|E<1a~=%l0Uq+W zI>!Dq_BfiKFz&yonnMma<}XG-O_9E^=t1;|$Uv#tpLTJmBK}Ro0INm;#xpj(2*@La zM^K|K9XJaIPxvi);WyEtnuUw8TH=QXPz&`0xg=>3C;;C_8Xk)J!OX~`B>Q!G4cLUF z6wE`IkUIX1*snX$@qPOCC)e$%)MK9W-jksXS1SVI+Ne4;9!f5FFb|CM(D3bV?qDL0 z&K<=2yUr|+jKLY~pF2r|CX5681%D2C-mgUtbCw|fNUBDBxZ}MSM=jecc(LR52N8b; zsD*A%!mxK+4tn{XJsP=AA`3|Z6)VE1H!oA?8<-yL+pV_$Fwj=t^|u-cBJXvSS7$Ft zX@!A#!bzsndkbx$i8z>Ew>y{Kb^2Mrd5Bds@4|6^ze?!%fI|G3LMOG&7JT2$^^fC2 zxP9{gTS*l^8Qko=Sp z1GLw>UhWZ&`Hd)~9uA`4<*CC*{PN=W{QGnA{NWNo4e8?k#p7@Rf3uInpvGBfn)v`9 z9+>U0dmd_4C?S7wsQy_`zkb6sn7_vpn#%Cw1Hg{CUWdCpF4=@oL-q?gyY(Xa=of>J z$|+dWZw~L6LqiWeg51~RHc(202O58WM+aLyda8dt5lLUjuUJYScmIF<{O&5@Hqh9P zpR?une=p+AQ&2>DNDJu2?^kqmw*1~#faib1_&9d}6WQb>O)**$IhS3cM!j=`f{E&1d1 zt6xN^(EtT-iHGdZk>z`51oAyc6#nkB_Y_eX!;L2cC4vcVQS>vbldTCIu?CL2U)kIB zZq*GHT|*NlEoj2IrVt0l@_57Ow{N$7)8#nua2vK@{D<W-`llF}nq z<*!0}IBK32fsYG3#+YbZ=>@?qsBw{_l>Sq#Gc%!jlIVNN#AsbS=Dn zZ}@;({zOV`oH(ziEdR^e_KuY)8aywirG)^r-lUA6JkAaOAbv;KOh*RtimOz4p1bWQ zk`GDn3#UTw{~be9(Id*i-9q-oW08FpKX?KYe+m14+!UBauNzfUeB#BCiRJLzGp~O; z&Gx7EI#|;=aII{*!I<>Zz=K(t@`q&IfAOo`1V?eGB@mAx*SV%7?_m0Jivaxc|8}VG zG&JC{X3?tiKpTI9dgA8)vVdL1L1hCdLhgiy3fOZIVLkFbWC9Fq5coob6KzR~ z)dG+29Xn4d*%OPu(N(?$B@OTIitwB|aq##vSUpDZCI_(qoyRA@wGniCq4!~uf}MvZ z&L1pse=7RHpS$lLnqn}va$|XlKsnFcg_zrRScTlnw?32L*hJQ!Bk1=8<==*=pTWfi z!vc)`@+K=}Xoz-~i%`eE!MeZIlIwt=Z?U!{06ot{#U@2jrZP#)0Ww7t!9dfeLx1>) zIz5ycF|s#1q4~PDP0+~qAQiaR;eS9UeIPP7_3D5s6-R)J^t(wh2_>OTi?E2>4yXq~ z*+WVH4Q+l#ap0ggO`gS6n|h-Nq3%t+lNY{GzNt-L8us(0le(bXC832^0ZML;t`=^C z_WIW=#9ws=+yr=Dx>Zvj7-1-;uyyQv3w|Snn?I2UK&eRK+<=;c9DVrTyAF~R7r@Qs zEe5^lv5)@*!5l8{uj|;?U5G7UvKR8H4uu5xx=g4w&;+HUEBNjAAu|>ffFuW95}=Q{N~?`GyqD&>2?{nv<4(c?FJPTHehSBba-Tkb)t~`%ppnF%d~KGc z|2=YX?1wZ}Bi2cMnVRggY!3@3_DFdSr{TYn28^2rs`rE2qrGYKd+I<_1kQBq5dYEU zQ)vN7jkaH|`I+@3M|Jj=;8?(RRxv=E3ZF;U0X0f(99#P(4MgP!nLQc|%%9${;M z)aq=nzAtrL@zbM0Kj1Ux0r632FIM6w*q{A;sJ;%(=n>L(5>UCFStR%LF=aa9(ioIuSt9 zPCe5Ve+uv2b1QSTHmWnKDpvL$hoHuJ!g6HK5WD^TSzztP7=2^m;BXb`o`A{j{Dvf; z?)lBTak1*Rh33JNTr^`e**g-9tr3gF%Avt=q0~-4c2)9^21+v`?C3j#@)LE z+Lw0yJK=v>GL-`)2~OwufV`MtjQ>77f4Is6H|$W_w5Wi6I?HIA>rCf(Ot=2i#qjX( zp%)QG2q_SR;JibB>}|+m5ZrM3Z_oT9iv@f-^zl~*d;lE<+y>+IL6^}3{{5}SM@WGu zNTJJBUiryBxSyWp{)^>4$_Aji8H9Hi($QTZ6#+KWV0}XSpuYTIZ+{iWV>IyrTo|Rc zr%O8VunxPY?mzLLIBu8v2!5a*b#a5C2w#H&to(mhCQsqjP>SP{cH|M*>7lIt_WTYx z1bTrR54J&3aN|fAaE}20uFLjSo+Jb01NhU@iyvcvZvFxhRiNtbqmw z$}l6Ig4Muo9E}N5@trXH#RmOU*bN1+?TuwO2IahaUc}s{DLw*0?TT~z91lDTVTeN_ z;-Pq^UzFnkVSn-4cHcgF@ud0_ir2{of)On=SN7`6;6o@O`(_7}Mn!qcMzY@6Jx40u z6Bqu`RREdlZ~i2IYJPhob-uC(tfxT-0s33uamj%a`Ms6GC!owlsJ>PVm~9%Np4*q9 zY23vaUm0xh|MD9_krimKWpDD5KjINgtUJ%RzvwNe=;nG@`JcH6qGLn?hw~EU?7un* zlGCu>JPZ9aYGL4sPT~`PdxD~HpQOKv6qug*XBUq+1c+($*S{hAfn4v}hG`5YjTa!I zaY3$qpIZBe5pfOH^%iURRif>5_oOWGB!ZEtwXp74Re*MA&;>@R_Zi0i23XRoUG zEVAf=^T_ad`UnsPbV;ubW0Gp(p!j*`RZD;R>xuoNBPXc>&c15OcGqZtCB$L<*aLJQ z^4{Mhp9YW}`8d|q$$8>S2_Jo`xZYsFQ0Vy5($d1IKFE;r=)5P|K)C9}dl)uXkqC~x zJEz{P6X|j5;ma9ZUEK@!FfN$1@jC>JCrII-8u;OJT3_o0R@O(~%4lQirxdnXr8vyyY9}FkkDL)*?$Zcw^0_~PB-G?lGJd_~j~6|~|CK7e z1Pw=>*E3ibCEyL;tW`|2@529LA7(Lhe$D4KIGmtAY6BqB#n3f8i}0q-mYhGbv|R&% zf%4$ji}!C9^@R@yiz!`4PSoM2c>q#(>V(CiYTSR3#|XIpCt-GfXB9OA%(h&v$5edx8@~ zA7THi>~6voj@+_$dEXrT&d+_$n8AdOeg zM{V)kw`~Wae2fAQPh(*p!~*s=+DkCr{)O9aWvN?hZyM_13F;IAc3bxUMtFD{doTWu zDR#-kLY`{3I_vX*JEuscSNjO61$=LmW?-1?%o)N^$!p4;HQB<`B%{hcmp^F?Y4O2MXi6;M8K^5 zEo(<4(c>~Twb4eZ96OcN*RV;^JwhHz=$6|;hc@=0< zvL|a*%b&HE2Qnr?Wa59W72KcjQ}>hahvIllkVPRqzGUFHdn7|>-~NtX{-P1YFC3Ro zpJxnwI(n2IYSgGqNBseOv@?Xdjb)N}uwLwjUo&KXPo<*@4A4W(8)#Cxh1OuL-q5}J z$wMMx9a}e6z?+gVZo4+WrSGeDO${7av?!w_Pb_A55_vBeoR=}JvO!~GJkI$f4Gsul z3{;N?H|TaZf#Ac9!z>~$S|D@lIBT)*cE*rI2p%E!nFpxJJ0%+>=WxmsINqVrB|i;1 zQ1S;1v8!VgF90)y$6fLBbjAZpz9-1GXMP*@wz#sWV)9#kOn-sRe(i~no{+_Z=LbR& zp$Jm|jv68m{S09sb@tnzdTkud`aox7hzv|(UQv*{_BnEjEfHvMA;bob6z|r~vIKjF zX4-kOR57HMqf`lVXjPL|pG0}nAx;uF^ zz`cajPwJ{Yw|VO%sf?;J7>5~H@|~8qDV;EWH?O}-Z^Q%|opj|grF~2IAL*j_UBYUC zs09Rci>>V|S_C;8C#S+!WUeYcw7fb)DT$|pjun5}`@y0j7(TiU)B9;8Ba&dBHaz(o z1QqR`+Fp401OfD=*MwNrLP+q?(w>l%P@sxl@4g=Lb@vELd@x#ydkQwTrzqpZTm;gN z3K9^XzpB%6Tr8$PDFUVqu_T@Ou;O;2{peV?ncwm0ICEe>L_ut5CVrK(ZTd#4#A>l8 z=G6?6$g^fZl;}BKR*bPycio&kk^yZ;g5(b@2fQ9b9bs^k2_eDgf-Pba23lhHNjxp7 zK@AF{Bh}Fy_5X|pgp=pQ&<`1yz`H~Y9R`F8+Le!mPA&ba6qcl@paZA^B7rw`NB0jL zyQ?~eZZe=TZy0=Amt+20K-*Tk1zWDiUalt$rf$B8a>hVq;A>_qQfvn8YjDa3 zvY-CN9%cA0NY)3UU*X7D=KVsgqZ#90v&TNWtec@2v6Jy(1I0_XrO4592JHoc46I>lxP|6>yEH8B+4k&je3_4%d5}X|1_U& z0b|9v5G-4CJ3lrtz2tOrL6?qzlEmlb6tX6OO1sXA838~H(~hY)k0d{BJJ2#&3?w?5 zQ?G>gKGKXQ$+5phwO6oZA*E%?AhC}$%axJKm9%d;yiE7l_{$l(`+QU3tncC`f`rwQ zVas+YJAw9uehJ9iuC{-4Gu6c(3E?NU>Ky5m~08fDWVCNmW61Ri|sj*7HRzdK&+s(P$p%kywb}kg@qd1ZjTvn zkC}~~V0u%6ea8A@JGPRh*!6}xhz%=@J~stx?+3cnsP=uTRn^Ijk!at32`+O`gTwCC z$~=-EflEa2q(x@;Lf{4p@*BmgfkK&ZkXJp69Vc-w;dIPdjy4`(fEjO8hkso*3uNNi zo|-y~5`?nzBWbhUFu6Z?%2NDEv?KB7L7Y+jambopr%t>fg26K|ly|ZN@EPt%p7$EMkC^OlqO_PkLY)j(6{S+`5Q!fo!U9^+ znWE2iW6VW2L}SoTmIUF6-a@I9REzXE4!qw}}(`BZC=jhP;>4r^foMU(kbxiA3zkA*WWyGU^h*z>~*hl9)9i zBV*g084dE_I!WY&F@jMdPCS$(6ve?ud<&>oYNY*vO zD16K}7WA$lyiqCUC@5Q1o@~I)U-MP}SNgDVN zC(q5{Eu#@N8(3{CQTU2WD|VuEkq_(`w=YWukKjSHYiBNUG*0ej9~CZ4+{i7b6Sst+ zuuw}n1<7sq^*o$j%3kZ?bKQ z7Kq#fNUeD7veMESkDEo2yztT9&)>8EHZBs5>1J_5MEDm6u*>S-u7@xLr^jj;ws8Md zXn>#Y9f+_jzHkt8Vajx{RZ9ROCA10+W;Vvq83{ppWgx%z77%aKgM}V1KLd##NDkZb zOypJcJ^DaO%Zw07fHWzt7*Ki^@_PFK_2Z@35uY5^N%Ff}&T0f8G#nEhi24*k#V_1Y zL0kf8O^6NrY)3osfAnaJ3jmDrkN5WLrs3Eb~b~QPt`0+-* zxETV#&3>2DT%C1kBGsR1Pax-Tijo=l}z5bYM*6vsJF4cX%ShgKuv)@fSS*0J(}^SfazEcw(`AYu*z7 zpG}f{W8@+0`2wGZXK|=v=VyFOqg5*RAFlwnN#;U#s@2I?wlv2S{?r28_=&axA#HBX zJ{3=``&Tq?3Eb|bfS~Xbs-C)mJ&>|r+a*2Aik?6F#o-fy{sITTEmDyRiJk z&PsKVU}SYGqk~7M)!+Pc-olA0FE}c95z8PkW{>$e8JUpWm%ks&OEJrJ%r#7?jkMO0 zsjDlaER``&X0*J8&~tn5#3Ao-Et*SGz5DKOn5XCf_1!#j>817nE*r-cTg;{<_KSbd zZP7O*hs0k>N_CM%*oo{<|48jLw<-f5$OoJ)h>`6l_&orr4!|zmit&~#}x+P`R*vej2x{*Q7 z?ihe$aLeOSJ+{0BswYoYjScv3uHn_@NU9%Zrd!6bB8m;CrN+u@#u{@brRE$$H}POr z@Zi4y43Hg=I?emM5k;trcmYGXEBQtY5<1#GuJLroFJ37v+*AXPJP{>vlZ^VY^AB3? z36dD&rCU2lNu5HmmUE|6Qd(pYmO|0Hj#O6T4BXyTngWERaha19MWNQF#S}&jw)kp# z%a9Q1-Am;=L&&AOJz1F`B5Ok}1nPbo!q4BfQRAZD z%iM&Ykk@+@!P-(<7;|p{;=3DRxnm(<$>uzX>dYPeuBcJnx?NvZa~W$7pXzOF?vEI4 zX%x4$BO#6`#@gOi-Sgt1F-$9zb0}*?IkQvuWJF(Qr(C5Q_)TG)NoEBR_Ir_&IFeWL zTdD}Sa_(J2g*xx|_bHsbfs(Xwo`Fb(ppL@r<=lD#Pf5k+|H{$(?TsTL&U-+HpkOGx z!PSB6w{fanKTmaDaFmst5Gnab(kJ$!GiK4r!aWQyaX&ZlP7>1D;$Y_FaeFO9V~gs; zkK@xpL1e)KmEKe&HOk3R>^JLdq2yhAYT{bFAePn#_Aj=N5KrCzG1ZtvzlRrD;H_`~#J-vKk^s^m+s$1ynO;YgL1!ReG)>7UX#|&xDPW}!Q z;ukh-M#Gx2-0vMX`tnp6iDuXN&s-pP5&6OvD_nSYX!bH0>8y3Vo% zPJg<96>h_!b-X2ZSi6r1Nz8MhtG@X_b>&lmFM_cyL4&Vs9bqD$-C8#F1R$&ee_DqW z@!!J=7-g}sB6L95UE6#8hi!?o+UU|gfoPC0_~aL_8e>W}ivz{Bjb7muP~{_Zy!!WDPCK{9p_UC# zxHb3>5Ia~2l>Id2dzQw%rugzVt9+YokrzB=f4+aq0=KLDyNllOzqcT`5Rv^Ogyq8L zD zz!^T>g76oeaSPb7-<#ZWNAbCWZN)RCU005)ii0#Bre(*O%p6ZYub~`dT72lMLGA`x zfuPnw!YbV=Tls1-$F(Y+%^EjwQybJm5cjs@RznGMnF&Vsf4CQ6b@QYlCc3ZZUwG_o zM<<3COs>{*C!uAYq~UwKIZkN?4n!*skUV;|$v7s**OQ;+7yO?4uN)Q_3Gm3?_P5uT zaZuxp0d#j8$bkoCVtYOIgeoQl6dMcwDqHyr(K3`t=fyDupQoKXLKU^dmfVNGz|n6Hef+f@O?V_SZHY7&j8(HT zy3)_wKEtixE#T0}cOEbYz9VW_|7R#|LzcC&lAV@IQYnL*0Ukd#yjf#DrQCZ=K#m{c z?kxxU4kU^$z`lHXkg)S0j+7Qtwp>^~sHG+H33{OX+2^2<4oR0>?`vjb83QYa;=>xs za!6;~F9~K)kfBzJzKw01ENJp%UXI^Jsx)M)#u(A3(t=VB+)==mdK` z-PlCvSo9X4E&)^*lynr|i1fy;x!zlXK{7W`gg#U$HW{Q=kuW+Mb0Mj$@GFK}^J4tU8ybIoQD;nfIxaOxo5kP^ zu1wTNq9%Nc;2bV4e2foXYX-R!c&o*mD!uo3h1)C%Y|o~%4_amg_o!BfP$#S5JZ`lw zzPx+iw~d!k^5MuuFl)2Z{1%WdtG|5MtfBRWA_@eCEz|?Tj+~x5+Bj_N=M;tze9mX# zLA%rMRsAh)a+&ILM$7TT*@WW1Js{JfHv-Kl`J@Ct82k-ZIIkxl?Bis!DIkxB;27HvBQ#DdO@*Q z4Z$)%4%HFCrMOj?D?YmUhYLyhR|g$Lcdf3#UP}h^RV26Kw_~tv?crx3rfBOc`KlU8 z#uO%Rr6M?GT$CQHJE2Y2kFL&PR7P06!Rt~(D7?mJbdNpl)b}Vcl3$M}sUW-{2M1u{ zwdk>!JDv;SD|=>uglr5K2;WRe_>u85ziO#Jkevjyu9@iC$>}g!qoC!Xx~I#(dZid+ z1%}ZY5zZY+JDhACQ+_vg&ozFC-lLf~RT1`S=RtgfT3;BNn|?6q^zu2MsKz~B(Z>CP zB@u_b%sCmPgr^#o)UB3&-lryYAYH1xD>N;J4W*?1?To=2H>Aa1(M|*g;wvBr9=AyN`HlCcl%?iXgZl!(d0hI_KSU08 z{FN^B26JSOSBxQLA7b0EZ;@95r+8f`yPUvquq5TH>ZaYVJcK8{W{I?*Cu9<|O zk#vA390Sq0!@y=DGyDj{F334E%?3Y!a%!tp_ABw1@^(~&NK3ZE642tXg5a!7jQ-80} z!z}Mw%e&`c(7t32g(uk7$ZHC&t1CRB*~x+Oq)}nd_oRmZ)B>7`!a~jeOY>D{$dg)q$XeK011b#)F+15x{=GR4ti(oOnRz);B1AI z15{n0Oj+=!qSBCXfTo@|#;7zOm~Kth>T$GnUg`Z8`iDe{R3OZeG~~RmKr+>+ z`K4pmZJfPSHX9xkR5wHWGblWDf_KRwr*@}W5Kp7kcIZ5xn{GZ8_+Y=hyNm#&AJhT0 zr0w%J3yhEUwr{zm=zeTh%gt+zEeVgBOOH&PMvO;`%$|xMk{%BQxqoYatYfP?V^G|f zL$pxrE;;+axW-r!XpYAw%<1}mE;!HRf=16*4vM|?RXwD$?sgD#hX=D{jDA;VLKl34 z1I{1^r#0A^XAmlLH_qNxRRjEZ!rYwOge(?^ZwR_$Ko#dG#ulQhe%0S2;(bXd%m_!T z(Js5oG-_32n>y1}oilq(t7}A$u0eWVhF$Fyyg_6wdbbXANZZ;Vt zAhi+{ljbrkZpl?HfS7Qe@^aif+9#>8a2K8BndmT@`BaAv8xpzaD$Bum2YOS|A`Bh! zRz3H)V8dOe+W&Ah3(Br-8HI5>(F#Zy*+XL#=2pSGIn+N}zvZ5Nu6Fo%szV9lCxJ2QNGbp@sZ!<_gtA?B0 zcY~Hm*aCEY<#v+3N$hPTybeBq{m4qM%y~*qZ{bcUY5-OH(iBKZJgCu=c7fF6*@|1+ zqPU~}{YRsl%jP-CNpltw(3z5AN|MBP%TpcbOKN)J-GO4Z4Qk&4N8?4b>5YZxbT7|` zUap_Q(uDd5{Zz8w=eekHrO+~}FFOo?7<7=FZkFpE0BZ_@xJo*(QQT)S0&*y9VK7o- zi_fIWa`?r5_PJst|G8)GP-QM9jLKIA2VesO%w)Q8%Mq!5#ZU@;g}%Zp;B>uJlQ=pb z!fX4$kr65?gP_?)1dm-!JX(#RGF8aWFD~q#yB1Z=CH=>=&Ej!Ea zA<|aOC%dIdNK|XM)|Sk7_`HIfHz7?^35o~Cwp*Ofl~k=}JPJ^GQsFXj(QC3q%PXt) zs%M+-3`cpfLPuNtmR7I7mV@obWvv~hmWJLh95Rq|sF22MiHrvpd%Jwmmg%yy=-l#a zXkUmggQdbqU15Zc1!6T=b5YIEd}(@pK04jSp5PCwe1MKtiAQ=epF0Pfj^q*zalSA@ zb=e%}52Q%Oq;{A@DAlxcX8naCeKk};(;P90qpo}$A7dKS6PS15bB5NWG|!QSCSDp^ zSk;al&VUz!JAW80vx}bQ*0>r*XfrZ= zf4cgo_k(_~-u7?MGitKE;`|msMA3Gj1$g()MEm^syD-jzsCD?>Q$ee3&zisckNRT_Rr%O)*d>w6UJ zvla2oxX9{Qi9aqdQYV-gX{{F+SLiyJ+z~{f7XM9D-?5^YI$Pt1hM{nol$bD#hchnx zQ{_~`ix0)iGUNE9g@ZTPTux;ZYqGZ>@_+5U$j81H4_hoN;4lfioK1SL19r*ZO4SOnX7s9oY@s2iBCq*ZYc^ z@~fpc8*)Y!G&`q!ADSeFSqJo%}tyiJMvr0O?2yGd)B zTVirYb7)))d@>*@(BN^(RU^dqyox+r9#6iEo^QXPHgC^A&JMf=an9v3%b$Ntb!eMKD&v*$s>=APv_MxjvilGyiRK~YiIdhBj4mKAT;N@Rnup1T(6I9r zC`MBCcvjcYN6B8uW`J>@gfIx9o+!@LyIq5CnM#qbTo{RI$$<;Qk72Dkd7sIc#j&AS z-Y_2K%50Z}U1AR7Bf6a~c2-_r{AZN1@o?{?j>vD`)VfRB8-Hi-1q5k|oV!EuzP#C% z4p{|us-t_Bal#3YmJ?gsP{K9p2J*ioS!@sW>e_04;04YdrWmeY`4KC4Q&#hd&5B}S z9u9Af43}e=!onr-{XMk}(6Mhx9eR`EERm7Pn~0UM6~fANLD;IH(oCH!HY=)Dc zF;ot8CapF;?@~vfTS-81@hXCQ3X)E zwmaQH3;8a17lIFoj+pedXQP7+e&h@_^$g;ZIA5WQM;xt;%z%h`%qPwgyFl-6z6l$+ zE_svy7G(_CHjx)y@eV4RAZY9>NC_pdSwjkJeGIIN8ymd@vl_YE1W(D!dgYywv;m7(`I(ohkk;C$)P3pTNx?XIg$FnIF=JUU zG0!4HCb*O%*eDD5V=@<68?4d%!%x9l-0Zi3PdjsOPMuKhidt5uVVadbzVVNL$Ff*$ zzwv~Hxim!lz4s=@s*M<);@K#}5Zs zKOb8|sTnux&On4RQN@zk#|z^et#AzJ^pyGs2Xg)RXkv3cv$kyNm26Z(O1(U~Dqp@%OGeJphR%L1dT#wi5mQsw| zo@0HVN-8$iv}lGPSYqE=pNQemR(@s1+uXFjXPhqI*}S{NQ?Xs?fxI_WlwhY7YN#we zFK!APo8H=4g=|q2EQmA(p3Az!y4UB#GyH<@C-n1@9^1-OPooOM7uvK zBOglt5&!gldb~ZsGrQ%LNLH39_ZeCt$&N(F8T3-!-pnnM!l*PeEKczb8LhEhcHT$B z;d5|yGZuN2a6t;YzyOr49gXj*vb2O=6K9eH}bWWL5y zt%H-z)>rPb480{~4Ymz465h_%{CT-P&OUKmP14Jf-q+CA)<-B-9&@(oP$)K~@LZm( zW0J{z=H+dBf*u~wInFtv3{zL;s8@{WwECRb_s@AhFg`bxgXN5MI?=va8nIsIAJiLa zuBoSEtRr64drXtMg?A@k*CjOHd7_r|%f)S*Kc)E?q}E3(Z~W7I#uB|zsj{^K{jOqX z<0=k)$6B#ZDPCSViLCtvlcOcTd%4mpZmxA2+W2io+{M9S^3^D{xJ!!lJ?h=>^KJ9} zW>VqdwAAr42To8X&Nv42c#|~|CsxD7I{#|6_$;okW$}euzax*js+g!dcob&DFz&~A zj1RGI?MaJV#Qn8C8Gc5Q`sppyZA`@COs?)7eNIIPyb!voXV1Dzp?UC1K{}Ehu2+CY za;hkEK6(hp?RjFeZq`T(!a_*J9JdDs;+^9_*~HuntHo=XjKn;Lxsa+n)j70|a%jQb z;QU}qS(t({M@bo`tlTw!eI#w5E?6TlJkT{v8DA8F4x}of+@d3bJe?UUDS0cIEBSeD z@Xo+hh3OhqH)zJaZDh_T;6~N!B~DR6RJ4@nqKgF}obCqj9F+!CK>hQ^wAJ zh!&z7bjgimWp1<*x%WXj$;rms-f4Xwl{(nG)?5}=tNZR4tf43v z_%qv#71_`AAV{P&<5$D2zcX(v`#+~~uZQR+d8*Djg9kuU86J%JS+WSEp>E z`gq$?GP#+NSzP_YgzCj;6PIt=Zx&Gv+Ga+xsBI{TsO2%;a|near#PxbkywV@caA;| zx1!4E@?_r=x`t$==D9>Sw_UNRAGDc_%7XzQr<1ekKFIdKdWC$Ryk$e$5ttJM6%2x3 zput@~vhr9-Q3=Pb>Ck=a5;yW>Al5J;GRZLcSeapCJMoZV#C=!#$+GE6{?0=G9bWqo zsUioPzCI4*WrxIGDbXX)auSv&Lb{L35u4@$xHt16w0>qW1^f;lem6(u2S^5JDFMxJ z+u&O6LsAx9Q%(WBN7-S_<~`r5jagxrS& z=bQ`9Gs!}`14X#8-27w)Ew9`~D-RjNAsK%)^JZbcdpp3Ua~MzaT%K6ySa$|>8^E8tA*eY1*~3}l(GE?j0=US?fl*|6;TG-Jl`C~SW~Kl}+1*~YEU(l$g@Bt0Hf z#>?XWz{@M+593vo@d3k2tKx4&^9b_!V);f2W%M#W{qSw2FviLAO(;T*iv}UANiZ;m zq)a{CcfrxoasEazlv>(QVqe^?Dip{m@y>q@&ed*c5?73GY|!uG&bj~{35byE>phb_ z+9iCDkZ_%v92`u@yv-n{rA?c}?e2zCO^i+*v(_39rw@iE{LFgq?gPoLJ@?T`)U(Ze z_o>KC^lFN*u#NWbisDo7qMO4%wSdIxEHf>$4(%%OBY(H-6zU5}>Wk|y1UjHz*F1X! zd92k(jZ^e3ha6AV+Wz_!;+QyBYboGjBzwDslPpX<$pYV}vYKk4Y3XyydeUSWlbB2_ z(66UUwzL1dOhsQEW}iXlxui9F%6n+tKJJlt+n~&~-=3l-`lC2j&aH44e(i&S^P;e$ z70(oER?K*Lnv5#E8&C+JJx&Ie%ZN9okn@I#7^^U^0A9Y!YnO^Vm!?U2w?QUO&{CN? zGB|*}7cOC$8|GTlMURZLdMZ9Nx8 zED)PRd##0AP`6h^2`*2@4}`=2rsl3^j-++8_I*u4Ty(7*wlLa%&w?^Hj-P)L+Re&c zU)=>%rR=@^<9M^yI47hW>^R81E*mOcv6*bkcg+{wHb{1IEoVfn#O2Y^H-9)$7W2mE z(8=_7CL=%BmF!n^Hv+oJFYTVI>+0#rMDMIK>3mGo)I+e#Q8!DIq?ys!{TaMT9u59@ z*6i_h-HBw$&B29Rms-9G)+=h#a}&yJabG~C`_+SYlyl8KBn$VWg3H{?mg$0uWyvYR z$nkGM9zGKMGje;bXQD=is3=j=?-@FFPBGH0?b8-8I^RyO=*yRM3&- zs2r_$ad9h}=q6)EFeA~Cs*Gh-C%vYNgmIi1YBx&J4DFkOcga+*wUN515syhq#-xD% z@Odk!g-lpk7MM-a+5U##JecY__2X+Rd3%JqQ`^;l8gO4Ew0seRd6jBQbEbs5s-yxI z!auvhmz7|Qk*X|+)8=f#C}?xZTeLUjY*&3_yzPVb$REmGqN6(pflAZz2C@vwd{dD0 z_O#><7;`=GScf8b)>AHxKAIhv#HZrS^(@Q_e5rMHgomH2EMA@}iB2Y8u|x&auS^6m z-RB>^JB)veH|g_Cte<2jgC?KJx=vq~XL*2I(sFvPGBLxhzm`-mJ|cRh!Z!#tDb{Bc zRy67v*7g}Q+jBxrpzvq!=H55LG0#Ka@+m4`Aw;N&9>g*x?;v+n2lvR?yP;M`S8&-_ z9@{e0@8^PoT2Qbz_0@=Z_Nzk#!lK@^)1CRP;k;<7`4nP^GEy^#cqGi5z`}V^=5GAI zG`9-^o-errZkE1tI8*-nUCXqJ^;=`9z^gGShN%POfDyy7(Gz2}LV=d~6Zz)b`YGC- zNUU)Mc2@heO(#3}IMKkxNiEv5va07FFLjgYAg_m){`M1%BusJ*SF2APx3}RZjSemX z$DzRpwjt|_<*x^e433uBw^vE|A7-r6p$t2da5RtmWx3;LSImnFcPJJazNtJa-dACd#@xMvb%&@h1rtT8jsO{O!D zWw)y2w|r$c|AY`Z5B_Oa-|&`=6WAR4K^zr|jr@M*`k^J`)uxKKuac+RWE_gt!|~w+ zXGiDBkJs@~sK3$u%JDT00qP&wD%EKKr|EV@za(y4j=KO*qg7kZUH(Xi4DoSC1;Lu> zZiR85eJCYO>+`>-_Yh+JdHW4sp2d1afuP`kO%RqCmG8U3VZx;P?fU`}NBffo{{j;f z|7U7W0-DR+8$yUtk@nni|O% z4K0Vyn|EU zcaK!LlhcfehS`lsKL%gKclrf{b~%m3MgE8)t{E^?Be&LVqLXqZtfww;V`sZ;CD{98 zB0&l)QU!htaO>S|?~wW8)4r#inp9o||6}r-X~V|Q#N&ERg4nL^+K)SdZs=tHUbP2B z`3np`ZFq*A$T+7EKH6ZYhg|!BVOLMLL%AMqRt4dcJZ>PBMCza?NX7<4T~oZEw)Uk%7twUwKZI&0|QyF;MI+DdqPs1CTTf|El>3# zfsK?I@o2=PiqLK+P6eFV)wLvNGPHZK&q!n(MX&DaJdV5nDdP;}raucIb66L<6yxj1 zThoQvFxztsh}Uu9ZcoilejUnR^#vVa*?W3l2=Pm<4j5DNFg(=m}x!MqSIL9432wiv5tY0 z23Z~YW0hMR0nT$^wsl0{*aWEIY8}+M9J|t2r}S*+DIJ7#2)BotEEq7p{#ZAe9f0z< zxC+LGv7FUUun-Q8hq}tR* zE>BMHYGjZ~@Hv>}Ds;HH>Na9sT+0o?fqlUcyPE?1!lCyYIjuXCGnt?inimrq z*bzhEafquS&9BEpCW2sb*`H!p-o?Qt@F`=H%c)h-);~;1k?e}~&iieh8v#mGNi*80 z_IIxvUmG9o;753%{O}vgDfj3m85LvRkBrR)|2ygR51Re&Y#ZM(1RzmD+YIu**5cUB zm+J#GJPpZ?TSYsP^b^1KFq)M#GB5BY)`2@G-zZB199e!UiGFkI_zPKU?%S^^6{>Ai zbi@3y>j6!?_JpLigIIETT_JMjpptHEmF9x?LHes9h^jjJo|-A~6P1mrlZ|Q8z&0P` zjA32Fo!S7TgrMf1z~8#C_dzciGBfaeLBK`h+BdR@8;bHsQyZWDi3H%0{xwE!JSR9yfAW@OeIorMl)Nzhv0s)SWnQE{ z2=qLBd%Zm&5dXk9uHSI(?vT&jq4?Z{CYgu}pX?yVC|EY7?yluGBtH@@KMi1}*17*} zmK-wEsnXTWX2jWE$?5C6Z>+I41EizY5~yn|0JE5p^CS03<*u)9sEDYe=G!7mBNjq& zDk0E2A5Ggf;Lvb2YcAO#_Fl`|^jH~<2JSd6MX=&|xEFRE88KMW3B$Ye2Q(~h*sl8< z5Zhj!<+FD8L`bL8F$Bx_Pgw5NFQcz9|6|;mq5@##;=cjQumCiSK=@=wTD%vG!;Pxm zX1qMJwIn1g=UCt$ZOU<>79-3LN;wWbT%a+BUhBx+^0#&lYhX4?W;~4N*I@9rwk~bM zc$XeZCSO8Me$jpV+Ac=$!*g0~!s1+xv&vV*;51{rrvCOAXv4N`~|NT7`62-;+=YH&Yxa6FCXPAa95MHz0eAQYFVQ4!$Q5INZZb3RE- zG#~OSDL=^mpj2Fv>0}s*cO|mSHPqVt8@<0T&7r4+h)3Z+go#!5E|=6p8BJZG8ex~r zGKHn@uTZ)Ey_6s}co)vsM*8Fhh}53pKpGiJ7#mm%W6i-MQB$@RN7B|m1teR86ytCl zj2hncdiNn!NzPhano09g3^6eTU9u4IUi`PX?oWb+<7TC=?<%m&7bJ|s zyVR88EL+;!uoQA*}Y#3@;#9rw}Ze+2NhQxSnO4<9|k0^H~e`s7%+%3+bC4#&@O9 zRxRlpkpP&{Kh}nr-*NE7ZTATgKF;{VBvg99OM>`cwXm9Mx3#w{uRM@^Zy^b(8o~kY zO3=<`e)deQzTVcs@g*zaKRKJ;ZuLgeP7+s?j8IgB36D!47xmZpi1?Xh8ATUx!X z!}^?LKQT4np(A&`obx@0TvkS2?tyohcPfB~CVuwRzwYPUZ)CzHRb=Ss`j0LBc%ti% z`;_Tsbi#e+Yeo2o%7L3dwE$Pt+FWFLR!vC2Nn%x;+L+bJ{Fx2~JsI9w;A}tAt>^w| ze`mdRkLjftWdS+e2+$FwH%Tx5#F_Rl-PS+O(l)B0bjXSHqPTjP*9V)HJ|#Ru&vB?4 zNc}t6k3BF`O_&mWl3IYHukQ#3eM@h#HgFcZo<{tJR&7TCwx!5oU25h#g zmOXeJr&tNAy%o425g~~ChV7-=Uz{D7UH0+WnA!g2#oGsvjb5;S9XJ?S|N2nm;Hi6F z?|e!d!N|-F3r+9oL(cq*$z6nm0IS+r`t1O8S%JJty+7PK1d$l>J9vkip!^V){_&e23A696!B~at}Rz z;uuR{mDLeRMj{7M{MbMot|HCu>f(op@wHLB|FGLc{FW3G?l+CpyO^(17apD~W`}i& zWQ7jH*Tm3+yDCCYbv};K`Q>}+gCSs(3ZPHa=AkX0{B6Hy;vUjEEdx6}o=q-M0NYh8 zjbZ=SI5)@pT_xYdy)G>+{hSBpdb5tjEPc-V!>alRWZoRCdQdZSbX(iz%|GB&)2THW zBV`QEOcSnutK2eOI;ZbBb(9mlnsYf(DQuxcdlM(j_p?&VS5gzSbZ?AWw3w1YSMMU7d)N#f zCW~3>nWEpF=XQ@;q{uHrV8}Bpv<&N_*~RvbUAsgZu+!2+poTp*5W97urTt?-RfLgv~lxlprSb%s4%b*0-e+@%3K-7!r>+4l0d+dbN~3C z;PCHx;wyqA@dd`>nEi;}*hkQQ`g!%Qo*@y$%2$$udny@mG283SdA?=6hJ;`lXa(yh z-|^rDxU7M?Lbm}1l381rB6wh+LCgtf^ZRmg;>=0nrMOP>>wJ1va6zwRQ|Dj5`I zvDFBdJL~-vc+yOo5zhSjg()lQQ+dk+u@kq?pqOXVE!ow6YVI?Z0X{pOI=DWkf@$5v z`UXs0`P#M}=8i<|c901bN%5X=dwX!F@r(E;IOsdey%b*+Dz z#Qz&-H3bCDfy3HB!1_OKgDs0?2PqDavb~x*!<}8PE(2}lQf|o;!s=>+*sW1V=&DCI zJ7nST9D40zH=A|KfINaTM0q;2bW}O}pG*%Kcofc?M|S#Qe~7|rl>VN2Xpe17UTiHy zOfHp&q&P>eda_gNX#GZLDuxw%)vs)p077^wjXQxY@&7HHnF9!D;MmxtPT!rPvDa2B zh`SgH`GQSuTfH_;)zfhL05JjBROoPiT8=6$M=WA$CJ=UYQCW5CbMfBGMg@(L?@HWk z&xdw)avH`jmjlGT@bF4;8zVJ@OM_23uWV)wpCtBky{|6(Aq}1yad{bUzx-y>wJEqv zP1`&F!G)OLFT(pwchiZt_sRpG{dqQ^dWL#H`l*Zq>Q!V6TfgDq;c;`WLb&u;4E1xV z7QeL;F|y+XsMRS6@`)5Nhk>SkxXj~$+((;5(Vs)%)I5=_gjmLh}K(nQ)!BvuY2s9SI)w9<=nhc>CQK9tVpjO4kQ&gzjec#MT<()4jS_;A7XJ zJUUUFu&hR$MXo&!WWS3du0q*$fowg3qkqJrZ3LvV!1||_Kz02uvwnp+r!=jD z=2q|a6qT>K8qP^!tFl<$$e|G4VZA*kv=8ULw*OQV7w!`J;pr;%?T`191qsl&iF^Gm z%~3h;wMudMQ!dd`ZcEPsyK8c6`58f}fq~*!VD?5+OdCLOJT%Pvz?Hb{ICz8^&oXp# z!eID0AT!->y1(t8ldWul&^4m^lUb6rBZ|K|0Mgv@I1=q;fAK%7o7DDRRQMnJlwr5GWKILY%|Co>Y{*ZlHc=L-2WN!fABG3QzzLnaL_KXSIiC{pl>vi}1a~Tep|uu z)_1W(3@LY{>ghV;G=$iS#1qUREEN+t=1e$sj9{ldQ=}pp!3YT`ejy!q|3k|(pI)ef zMB=`C&Q(2X(`-iILL>RHhjaM*cICK;ncfJF)O$Yp*&7w4zH&h}d3#2K zSM5zZ+mOw8C7D8lvihE;qCPzG=FfYiI)N^+%h1Reepe2EG!C9}WUH*%bgCREfY5%q z{l6;}LA2l0H~C*XaeK-*ogp)N>fZNW{R~Rh<~cLi-wO(pjP)PSQ7Y9Z*+=GQ(LJC;Z{Y(ml4vfS zxSrOWm%LcOjP+3jEwh3Ww(htb1DlFQq@s;d@W%arPiz5i(lWnXXGuFGSTy8OlzROC z?z;Y&O#<@~RgO1$4`wm6`Wx}Qc6*q3Thm(VI=g_ydLWPB5@^**% zbY72SH+0Ey$KWcShMf7WUBwiAki{iFNHRHi;$l5A+&>6E+%<+fSx7xv2%O7fOu2An z!e}}6caQOgMyJlI>vEFzxcdYp!%O~eSNd49L1d9i$A+)(Qurx-ECAamUoy0ZnNgvv zXGGKPYK4D)0{%@ZV<=~D_4Gs0Ax~0GYQ$Vbt8TIhmX=NY3}{e3zt@GzHX6kL8}<3$ab92}RoUQ3jY{!H6GM53vgd;X9zy@@ zu>4K_ zqaHA}+AFHn-XW!E#Yt#q2b2viqhjdk|Kl zH{xF&I-x~<7jG5idk|RcPe}6HcM(fSVoqr>a@R=F0K!)`)aH0VoPEdNZ z(i1uaD$(eVyDG+TdI7EVlaK3^2m|Dr>n10sqcLx?7-a+}^>qzizW#D|fLqh~xKsBl zT|n-bdKWvNxuMD?yIs7v#CK;vBuhcm?+6R7J=VQ3;n8T^?{30Pv@<{k@xT0t_i^t} zEnt(TN6Y0F{OcHjsRFkxs7fd}n!I#D+9<@}w#WLO)cNgxo83pz8n+h;z!W#gYKh;U z7#^!VIu|jmo2qrV<*Jv+v`2fA^0MI5NE2yh7j~h6JhRFVVI4%)`$g(~6Ig%5knoE` zO^fwU2fXJzVRju!rylPM1_a%%oE=L&90F*T0^%e{k?YRH${0BD3-#s@zm1+&KE*+; zmS7vlx!`~G@L?l94!~-5HhwB<1r-}oc&YTpW4P-;$T+Z{3H{{whIwdK8*?hI9Ejha zZuQU5fV%|br$+N1$Giu1?@=p_iy3IMx_JrB>Ng7P2^B1#Y2Kt#vYJwe*D~uX*W%a|G63qYkN8)}ugIl^^Oj_EZy}kFPoI{}6Mvw;^H5Z#4 zl9;~dc5g*9U#A|2w@Gt?EBuT=i1J&kBN*YSWA?EV_pf6oYDNrYz2x`3r6Re?8qQXp z>jYlN$?LeBsxB0-N3YGWAgm=F%E})cjOiMyPsz59`1Wa0%pXNIpei?T`ZNsQ&OZ7F z95sCri#3KXX`8x`y2wHO`?p>1YH5BsW?HSD1X>WcC%@lR+5U5bKUdV{)<6!3dbiq~ zhqzRj)SGRZwsD&fU>`*Yk>3s|PyeNzy(9$Hto+afW-GT*jDVyNKE(x{Vf8M&zO+E-+EpB}fx@+91r$-dZXbdDxAJXml-ML3^;kQ{q&&VaoIfbI56Z~Qh zKkt!ZjBJiF+*AVrVdz`WM$IvM&oDYX#H?87NR+t9AutX+9we~%v+(@$*Z=R%He*4p zZHZO!s`mYzofm?F6uXMdIcNKeO*k&}rI%p_KlhyWi|?1XlAt{|zqc^*ljn^)pKIw0 zh5D;i9&^(&gofNrv+RmpoXfW7ONG_Ly3(B9C!KT^6^|0@ffJz**;EaNYl!Q~ zWt~IN2x%s}62A)@-4A3uesuM;oV{z^*Z6M#;ibwaDbX5DrCE)>_?yxD$!|9bLYx(A znXu0%32QM5p~4=`OLY}$A)K!=yF0apF#Ot3EVtDtxrsMp&99M$bckCj4herMe)^ez zAC+D}s*zZHR)+RRn7nf$0rZ#?bLs`h&)4t!7 z(-I{tHl*8q18peLf9$4q(z6pHd<3kuH|{>ATnPp%O0u|2q*0~c9JdLsdz)7ql0#2{ zjxhG7e}}Jw_x4h7v3$j?Ggef+{TksOC=1&gu7a8*UmsT|f)F@MZkWczMdXchV?&b3 z(ADbdu=ljlinV8f6VXr?Lng^SO$r#r@=^e+AdqUrv7vn(6Sg+6ynTj-$Q)~&elObf z(TH)ESc~I~ckk2ubg9sQ*^$3f&4BZc-XH%H{GZd^Sf?te|n5= zi#wo%(bVOI~)uQl#O`Y#J~X7R~i2keTo|}SLx>d!ur+Xu=Q$4-y1@Zj|0& zc@pk#yM7dmB8)V)G_zp&l`WfY)-Qhc zH1M55Q9!H|rdS)l=ZmJ>W;=R7zt9y!CYs>9QSjq82=F!ITmBXu;dmL&CA0W<$DvF6 zh@(Jtts@F596NOT+B84$wq9>q;_)@M7%3I!f~70*AQ`xFhj@Kr*y%XSC?=xrxBsyz-4P`aQX(b%9K>on@5LpnmGVWUvOT^(z_&$X8)E5Oy~|zlEZ&vFuzw@NG)!&8vJr1?xr55gsVjUkqr4BVPfG`715DuJ{2y)QP33MM z9LRSJ=R$Gh@XzT=q%R?JcEJn7mwMPk0cvrPb9!BmDncz!dQv9?N0V}{)FXrJ2$yR# z@ZUCM@OPkmr+)vV3Tw7Iw%>?KTh?+qbqV=|Hzl@PA6}QsufQ%>%^5b78WNR>(@LZq zrAITr5gTVZ)P2Wz-B@K^2)LdNayB+y2Ke-sD$V3Jf|_i z*kik#uFxF*v%}$kHY;x+B;uSzyMqa*=+FW2j^Xw*iy4cTZD0kCQP2M$XI}yj_4fXs zX^1FW)>J~42%)kY6(L1PmMn$rOCnL2$!!s`W?!bV3uO&qx=FS)S<4=YNXov1|M|>j z#;xzYzkm1jx(&_ebIy6T^E~h8InP-%XzVYv)$we1e*-r>ckblU!pndyC!-ZQ6yJg< z74rc$f8Kle!<(i-Bn~lMl<=D-mn|m~w z`!euHI=uW{Pb6oDWA64X3Fkv6K;+yP+{&is{qlPj6{_{`{~ETK^VNDSJUrWYr7y^$ z=eZ-hj*r~zf2c$dh%m;0gnI(Jm z^{w1-%Ql7}-_2T=a89q-X7Ollxl0@(hI)$Q&e%kwrB>@Eb-Xv+bPNL;Hc@Of#D2(! z`NN)t1Yd3oeap%@)_tkWJik?J>iamj4bB&S~v z;EDTmW@INUeg#pTX!7s;sWAD?G_bXLfPU%w&XS=vos~BC`ur6Wx%tI+d!jx00DQFi zbND^*sO5(2bxhACLkD8Uiq^rp^i0mE_ZLAH=n3_n&sl7^9Pgrp z-(`{R6A9+=mVt_N`)0LxUzKZ>y5XQ8)121inbfLf^^f5~NuT3Y#wI}Jh406h5cfJG z@2ko`1Bbr>bE7!r|0}e3QnAX4$Y!9@*f=LLmRhFlWI-zq=DL`|DzV;sXe$n5 zUf*gFH-XJtE3QuL_uiH3i_bHX8F2ENi0g2v+u_LY7afido`nWeb(FqyuiRZ1!5eL# zQeA(sYs5eqdeGwxA|d+BK-3APY{fBWS5dkJJBPLVClnvqU+H&-aE;G_yF(X&2f?v2 z8X65}o_VWi`DU)L*@WKV`9OQVxVK&%q&AKGeG9ej%|~t=`v^RN&G*m$GMn%ByKcpQ zv0xe`SugiL-|40NiEj9l%FMCh2-eQ8J+7-?E;FjcsdUc2;CO*n9IRd`of7e%vmHL; z>g!nc<50dxhHvvoVv`lVsYWo=dvM1-Zy*d-4-l{ukbyKG-}cV(WUIyB*&~Hy zZ5<&bM;P6(zO%6#@V;c`dR6CGOvg2@fzdF}>9(!xktvZ1;*AyzH7@O41e@hUU3&{*YUGxMUD2 z$O%u$YcWE+s@31UGCK9tcUddBF9MyYSPvpcTgBNdM;-K6vkZN&iTxSC{b>evN*n?q z99N(+g=bfN-vI#Na4Lo*t!?hGTb4m;-GPh8|W1ZS7)x{)oQgU zYq0p+fxCw@O&cO5!FytgO+w<@c@dofOD?$7B_KRLrqW+hp51@ZG5^*_$x5{S%Hfb| zgLxfCZve8j?l&pBvct^*r>l~uP#{GdePwN;`atMn%39(18y{^Pb!M-D1DhA+R_b;7 zrtGH;g5W`k^Mf2Q@X*ipJLqKVaO8R)`b)$6gO{JJ+_Ky|Z(a@S?vev3UT5fKuMUNT zob6xaN_qd>mu2VlOtf_JTv4%5nSh&s&i1m;rG7axd99EAS2}@Py3#uQ_uuqVB}6P> za-|O#YClK075ZwoW61XcV-Fdii5i)sKuNq9Q*Ji?ltC69NFzt3qkIaTWvGD69(+pqMUmoxTihVT>PW+Rg5u&$A%~u_7EzM0Y?Ne4hHqsw_6WzbexQ4wpr@v?ZdF{kuXfB8x zY6F+!zgc>2()6U#f106Yr zm*k|^V)oRyuWZaocx_&`(_62B%XuARrF~}$Np~MW*LPtS-MX!%#_&N0Z zUUkyLT!5yC>6xNPBrx8dWK`NP|6L zb`2^4&Zh|#GoBiwwK{zxA<@jv_Rz`G$&pjp$h zxmWPwsYx#!iaENKN5!Xz2dXP7tzMbH)erKl-0|nk*|jn?`RmC$?i`*tyjS!H9Q-ma z5>1Cn$6{lVH@o;G#jXi#mZjtUSnPPNe&Ft=h?4Vivn0I-A258hnfyc{#}v?6pWiED zwH2JJ5WEAI!s7qP%1_y=x zoXfuNlMp!q!Sd;QRzHD2a^W_}0`*C`4~4G2?U(tYG%KzA7$;?bG-y!}HJ7{ZE5;Q# zu$%L~jWB(1ZR7s|Kddvww903ZKkM zl|-R}TTo-Q(kr%I`^{uk1rNX$xTnAUn$dCXalun(^d11EqRlniaWLfYjyjAs%}%Kw z;W8R~1NK#Jihze(4}i(IVxP_7^b{g#(LXM<0gXhtHfuD*A|w#0^YIe_4UMwLVQLWt zio%mG=F$#utz{;wSP62eWVs}~aUXjYH!V-Po6^z|R$68AqxCqhhVpvfe3jH_b%l8K zi^=avz^s6C4S2CrXS2Jatjh0PC68}_eo{0H6`G3EoV%$PIJppC(-<-Z!UXcUWVE;s z>&iU2v5a>bQ4e9H!$ZWqp_e1po_@O5Pp@iHXUum#WjL75+M)%mwNF++l@ADs>f%VT z#Dqr0lu7Qfv4n$>VR-ddeT@Fk z<@!NM*nNNw*7|n8lg|NY%()@uopek_4pbpRSP`ITN#%*$xY)`J^e@wBk);w)x^p8g zi7F(Op~zY?E4iRDp2d=HmoVtP6~;Nb`DFi=7mPeu@W0W`OGst(hQBACzBds7D7auS!$mA-XY!~|Y{?3V}9 zqf3J0yVx}RwHuT^wsFXJIpF|lpa(l-uR5jr^!kq^4VQCNJ%x%Ap6>-4A&AcF9-MRe zki5dGzEE!*ZCBmHtWtc9>Zuq1E}VcrC+_9?;_z0`ZdwXEN3@v+l8cmUjJMz14MUG8eK zx71#h$mwxUjo#L1uqotD0%*!S<7yuleX7ovNN|O&Rvlue{{#H{uWjwD77KBgko^uft4^TcRI?bwrss*hTKSCuCuw8!Aw7S?y zW%7lp@i#n>ngsgBwFEY1GQ+zBb+u;c`@}|q`sdp9LTEkC@oqlOCDY(oZFmX0&s*xL z!Kshp6QT@20Vsw3)e>@L>Fx$Kc=|wDcg3ZuG;wkwC#zJce_m*Ix!M0riEWeQORz`g zbt+6aA9Jh4^({Ei>D@~s3Xj%8SmEpg$Pi!sCd>ga=4g;ANH(&8@D(V8OEL1^ioe>& z8Jr<&`I6HbU7ocCkSKn2>U3*Nl@+@vxWp-esct~48Wjxp81{R+5{F7RZ?Ft5(gT+s z3DB(1{ONgL%O_l){VSorqV=wgoYL_E6-zb9Z*X8x7aTAM$A z*0<1nG9=01U4{E#UuK@fOhE(DM`Aay1ekyHiC;B3QFqLIz8*JmTP^jk$OzUxuD2KG zW@w>I(!iBlh$r|xH;>Mm*GhLlL*iPh<3z`99Lqt6p{{>k-DJsZ63ZLx_mVHmAEwp? ze6J^(tgk>87FZ_%r|&RlDn4NylGXM;vBFIiOa65Z_{;dK&1hNDf2H* z?U{tG3u`jLEYdnbeS!?Bci{(PNARNf!KF-T{TOu`#MOzevr=xYGaOay=)i1%%%k$? z?eJN$^$i^t_hPwF!H0m~W5vi7VCP^YVI)Ase`8$|htc-u=3daY( z)y3fpYd=+@gZ#hPLhvzXg)-wBIK$%9U3TT$5D&ugC=9BR09S>;Lz)tM=IVIH~(@NZ-SSLp`Gyy2eOX;K)mauabMWMTdBd)0wKVG#)KahX&>*I z482^O)jWoC@xZr@E}>dMaa`l${>5gff>&AU)uP08W4Q@U9d5a`AHnXpX`|*qKR0_& zjB*S&g5+fsJk%P^pJnk0n#cW35q?J>Ot2G|NSF-~3ISW&Q&nD7K20v2(X=P7b=NK) zgB_nT0Pbqahqmms={H4VYz^16d=qd9M&PFa&!f~0<~>%Y=6dZI6q$0F%tK(Vdmp{I zq)Z9}?)~eA1zL8A*i$iTFe4EZ|0^0>c$nD9d@MythFB;v8YgZLjhs&Ns_xaST30|OZr84~%m%)Gil=(nin3Ir~EmmMN!W*Wk7d_os8gYBu z)+Di-^*dF-d;~?&G6c>+YY`L+oBS0C%v~)k!e2K{2X zFXnK^ICf6HbKEfr6G3Yy2)b6Y$bx&puE-a*vT*8HzER_NKyN;q?OyM?Vko!#@trty zkRb8rJ_|i@aLmSxo)fB68)6fN>?CMPe-{JNc(;Aev@xG}8ykmYg*+;7eF*Tei{c7E zDg-5<$tBQ$JflZrlr6HDZ7)6I&VYLo`d$I89p#P2;JSz29rcP*Su>%|7_n#)fD=z3N}}!zh}o1g_>Xj~>|+L6zjUOmkLwvsil+D$St*sjqZ) z-n7q@Q<7lYMz!~R7&j@8UN?Z`ZVN_n39)j0;mMKq$Me3cnJ@RY|3n4z3m~kaCUS?y z+gF^;PhW4HbJ(Ttufvsfj|4aqR2wulriWS->;>n0aH~Pp(_1qM1jwD*XX|D?TVtsM z+9O4jiu4cG-H=S+vr>O45aJL^m`E|6wm&r(hBAuQ$K>APhjBv@kt^hELYW02ljH$#~-B4rpj za7K>j!DOU1MWN83bw+F|Pj&QraAmVt1hSl@CWK`X;?Cd$8^!V&z4Lpj6_`4ErP+Z6 z&%4`*SOA|P9#4eal8^M@WGy6Ob~_-91Z;HNI!=V}2YVU?zh%8C)1*Q%zLG;$uQz#^ zA^CG`Dg(r|mGA%>-4WDhOCv0HC;Us% z-T=PYl%I4wS&ZnJ9P|69xX6V`*@z@ej8bE% z+oV}p1?4xnw6k1O+I(mprS{&U+Pq=gwtTjX93NTwO5vLnr7>!cEidOqMIOBGgVD%q z2Z`Zx@tM0=nKV*j5eM%X!KhA=lNj>|y3oe@3~yT`Xs*KpwhB;2^O^qVui|Gu{L--z zst=%^nZ$?%fcj|T^`9UKq(TKdTL(Yxi%NeM=CgbJCl4*gUdYUamk$8x|S0KJ>B2Q%6@??b_t-xALGVsYp6ScL>z%j)xX$z;Qdw@ zoG+|8F-qcf;rUZs=j(Y-dM4oS?vT7fB}7gj$V%g6@GvM|b1MT~dLuf0nCWOF^Ni0u zGiR&gj8oQQdxk&B50S~T0d^1cA9jFAgFaqMuuBdH^Z?{_g#%ooq1k@#Gkr1cQc3oc zyz_4Q0~&s{^E;i4pRi{ipn^Fw>oSsU4baO7m4%=!4mdW1mRb3k6Ku5=+Ikg zx#$3O^{nIE5i*$vaZmv+mxW=6voJ-!eBi^(^_QmBz}=J;&s}6f4d9#rr2e`oG=iam zH3f`7r#c+liCJAvT+4=AMeZ)Nqh&+!W;4PZmO6{(FiK-7ju)MnSd2Hqtj0b)N3Z) zlSC%tR%)~xb?naeZBF07b&53K_t=bpwNCA86c%du{Qk^TeW4)!bQ;t>oy*$T%n1)c4oZSSJHRXI&rnH~U%#utyW~mv?C}KS zWDRVcFdJ8zYJxoyTx12!4Rw|F^E86Sl0`P6%^(LfZlmM1Z+RB73DrTD37uzNUoEg% zzR(@V=vi2tV;FCE@By&hpL)J#=oLnUd{@Nf{OJD z9ojNy!y&8z;`s}ZEs0p7zCY6(I}o1AF!lK>zhf*XK5+;jQfSKSO_h?N^mVPg7U^jtg@s}|!@!6)rtO;_& zu8xmlY>a+ra7Tgh`v!DCludKvYTX4wJsN3ztTRcgHi-V@cY zj3ewFBp#{7Q$#Ww@kR4~yqD;rAd8HY1NCj+m(mV$pp6LDovIxy3lV^zajD(Lo5dda zj(HIo74`BlgTSm_M~(<;-zgCrjXU(4$zoy16=eOo`Hq}>8$~3H-4b2K*j9Zf-%qFC z^t*3pN@rh5vau(FCDy%`!$+0R`>jt4+6o{EmeAdTB+^3k$fEtM5OIN8XG|e15FU~$ zrTJ=q(#m5Q@x|0R+Du%z5mRgtLivc(K4hwS%8jZ_kkyq*>qAC$R#I&4ne$ZROMh(u z7jUjCMi%F0%wAvWb6;8hb*5ALe%PjP&{44^^G#!?1qL5auTKixySBJ$ZFFkx+ec}! zlS8XZwP0dx^t+0ZCGOKqloABkNF2p5H3QRtGJZ5ja5D8=M1gCWOcB12TU#3e71$EZ zGxPySvHbhLgQ*2DAvQh82H9uCrkT(I>6HaZl!yoEg9~d8f zJELw?r$kLIO#_HX0wK@{TE00`=bG+7_33*-)(+ei@Km_nP=~1Us9Q|BknM|-8k>xd zT*|Y)2#%;3XbjxVBS28W{9*67KpBM1@bL!B8Jcj=rY~D$^XXF}!x|Oz0qmUL47aT4 zZoo^35yp)@KbET}9|9-^J+#i6R^sv0 zTGI2O<8ef?)PlIAr=}wY>LIy{FD~AaUj)Z%$`6D|G*8_!DK*XV1ZcM^Z;C2zURg&p z4t+g0yi4q#uoN!R@u+09OHs}vqMA=8H_uLf%IR;NkM5SdjouEgWs?QY!Sg$Hki*$b z&r>$l&|Lbg6M;MG0?Dy4Z7_?@GuwB`foE+Vsq#NI6l1Ly>VQFalxm^iL7&4bz4aUU zubUccKH$TOS(30Ayg4LTSDf!o5*v~ekNdTWP?%iDV}t$vQC3km9)FIr!`8{NvC2v{ z42R#r3~m)LN;O$t?452m*%)-e%SF&<|0#^Pt`wwSH}XrKcB++pPzEaGg`ya9GFLz` z+OI+;*ePZ=P)Ru0k|jVevAD@}?M1Vbp2dG4S!vSzX~iC5gfH)iWZ#W+wbQFPLX>+H+>=%R= ziMLilFS9g(H4yqllHN_pD%`J0^ym_kboh3B_uxSX#oEk?q-$7>(~0-Kct#b0J1ou= z)Z=Tlz$23Pkhp$4>z3#@SMoy6Jd&N@btN%FCug{3Mfh9oW(;nsvwVz@(YnQYFj7X# z9G^=E*)zL;-V+UN>36JY|{GYLuXr(I@J->(amLKf*G za!D)4fA}+1*T=zVz`Vgtlw3O2U8S!5f&M~kYi&2xYXjZtBh@1W`xW=G#HJ$))+4=t zVb}VL0jnc9fCDDW_<>W1#wxMv^Z`482T)0{xw_VnWw61(^SIB1Ucsb!Q>^%p5)7z~ zXf%zz;}Fcse=s*A_%Hix$Pg+DYi31l@WIuY!~WC~f}rC#n&Ys7j03;U6R`j@R;Xsk zFgU)gBUiEt2yC*VrU|$Gt?}Mmlu9~I#ZZGyi@4Q#R=rW zwKyNFMzW05Y5u9A@XO(~PKDp_GXdv7c}Q|i7-2o*Od5eWUmQ~~25^8^U%PvmKIMzP zx?T(@iuf#GfBfc&0!6KcAtU(xV7`6O)kudW20(yD;v@9opT=T{4r;aYifX9I`=i^_ zZJ@LBzz(u!nk*~}srZK!(N=}IiJDlA{YPA^4zpf}oYsI7eAjidfTiP)_>(;kl+_01 zgRFZ#Z6>IzL^_U*U&l>4r+tu#jM!KZFC2_?0$(nyt*$USb6VLOR$plb zyP>nrb9Z7E?^dF0~((NU_tz#olUAk z$T>s9>>N^foXerB`mbywUWm;rX^(>6b2?;c9epLo7NB&2pS^F4$o_zxsC^$1gHV8j zmXsRc`p1dP2PV*pFu%ETYQ5Z!MAh|s+rh2T-n{BC^%A{6P;il&EX~MWuX$vrY{A){ zL{oqbUWIudJK3P5Qc)-m?(t~ci+-^F5*(LF)}2fc$r;Ih3_b4=uE=C1dMNqLo}m%GKmNu4SpDy(52OI%iCZ+R(lWe8*xH+AS3bY_;!=0fc^@i%@1zd7!}>0c0qQNWxit4vgHz#Q z0E-B`ecZAhF7vx9+17C#^Fk3Gpj|mlsGB(w*stf9U z_>11&6RXh`j|m6%m8Is1>!IOCLbLL!e0L&Rm?KI|s)jN|(H{TZT#>C6l+(mb_MZ%- z($KQhkT}fSSh+G3UHS2sceVy7hMRx-Q7YZHM=?OX*1?oXh)uRd;j%oqLo1?1HKOGL zXsdc!rZH+s5so~3$2OiK}`M_`3m zDZ({a{tAZ_O=SH|D>o9%rxnyp^%Bnecyb-q zsqWe{;<;x|!S&{Gkd$4x?dRQ9PLC1L+1Qvr2w_Tg0R zai1;L4CSTY2w?bVGQ#h>#Ki{N=xtvyD0j3?mi&>7+(zew{J?;Eom z%8hTSG}cn88^8ijE zfOeBzy4R{YyPv+N0o(vJ=zZ^ezWUDhnbvAn4t33y+gTaD8Z+G`alU3a{n@7ySJUGo zZQk+I#{`Y9cK1ogj};zM-{SdDyUy4{+<7*iY9(7$&K z!nY>XWqt}}_?x%N_%d5{4`h7Qo(d|n5fimY)Ee3%diHJC>aA72&ipYaZ^spnu>Pkg z*=5G4&C(0wR(^eqtKKTEDb0QhPX4djpmvA^8HSo8#LiD923=bvMQ(~n?el|T^w@l4A-SMhK1O&-zMh&9Z1Zv+a68l8n*Q?h z*LEA9mpZL^S9S`DXTBN`nsaWq?ef)F47$GNI@eX~c{9NWly+TWVi@DuB0Fxogo&6z zy;q)Xt9gE{G_JM`uaq%HA8eoWUB}NG zg~G1=Rl^5LqAJkXEOhu1HE3rpA}XZ!=sK{n2JftrFzM7h8$oA+6Ac=N*d+dV{I-?8 zF4#FnsNUYRR7kbFDlZ@4U%{J`-WEJu!#uB(kC7U4f&Hp5o`+ZI{td|{9}f<-0F z=%%^0>P3bE-<~OUk+l|pqDg3pJBhnV%!=Qb9L$i`79H+!KJ6*GP8BeeT^7=TEcC&d z){WMk;PZbtiZXudwdb&{f1=M!gO(FQVxKZ^R%!AAD(d6WsjRd1_5tcNf$6NNV5C|3 z^QIP8^o!(*+jGz5_a&T4uxNdg5f}eGKQ}QyOR*boy-?y??eG`35jPSd@J*s1?gP$w zTB3KdQT}=L^#6@J5v6$4_Z?6$a%-9P8O)A<@>R_DV= z6k{Ppj7?Z%$l7y3iGRDLZFaUQs`$H8G)rZ(E# zt(MCMw4IZ0YiwF}j?r5g2_)1CAu%sN2k54gepv5~N1}IejJBZPrFXadoiW?CW%NmY z6euZ8J9R3cwIuF&VUsT}t6}PU!8y0VR7Q_dMzQI4RL=8qnVnk4-dSq; zyX=tfff!`?11_^hao^6OC(ia`$4bZ)7<(BNeMASV(I^o%M)qqEv|Xglgbuxui#FMw zO!_rW*UANfWH_EPxK8>tu<3sxrO1AS82(4S0K>I_3N7O$@l8Zc23_U5{$OA65(B*( zfPE+O${kG9L}M_qumZvsG6YG(Ic*@ZxbLjO9M+$}zy4p*jy9lw+DRX9*4Z)^xk5n( zd@Mt;PX_IO3LJB_+;Sh**fJz5ng5Gb29?PZ6@uU@u=mt$_|jdR-;bm1KO+p(>8;RN zaI=cB@$suy{nhBn;vj&fYzF11@@~u--S9N>4HH_{AIz}qJw)>Ug7R_`7BMJr08L<} z8ZNAN^u)&?(X>uaZll9P1WSO|sD`=`Sn`^Whs=y%4B)C&j zEA(%GLOH0LN=@iIn}d_MLfHL3_c!>{J5h&#pg1zD0z4=O)#fgAVDSgVM4mSaf$%TIAn=tG^5J!Qo$ch@zY4%I|fUC}enB04)hz{>uYyI^r7^ z{);pZ=tDXm8ha4zSiz`$lo4=|Cc^9ww7Del6y#DOia?o>J7^a0@f?P0oeE+}NE%jN z$y0zxt!-xylWjPU>FWrmZ-~rV$FdNO)GPvez70@4o{6~?lq$zfvq%^50Gb8*EMUEf zrHpA)=QiD0Akx znl2Oe_;KDE^&>Ocd}^t&qI$%1^wPz7JwiC(4^l$4TObNXJ>s3Cg%HQjE zYlafH^HD|YArkygYXL%{dxMH!KvxzB-mQvM8nSc|<_C4#Mu*{_E7(M$4=wjkp8t>8 zs}sB%8PF>y)m}=7Am#iA(7u!uMS!Iezf4jEiV1ar{%ff#wIqW||SonYqVqZU;B8xxlIBrN~>BiUzeLz?~kv=70cdE_AdD4cCD+kYXEa3*-g zO>?7HP<;;SA)Rax&fo!g;7voJsCI6CNA-@%^$vXN0ZTXXdDO9Rs`%Y7j0SM2k2b=K zjED=N;Eijt3z|s8LF!;a+G`YvkkA%SIqYl_-|?zDRWnjg#l@;NShFPj;OLu?goE=V z%faKx{=L^nl|GE9v8Kc&gY!JgmwbH|y_!zXRC26ZJT&wDhFQC3f0H;r^eLErx;B@= zmD!(Y?JY?=upp)-n;`?##DnQm;$aq87a{LmZiY6CbS@a1s1e8*I~<>MyyN~jxjrz&6ARMOl_?wgqOUIGQM74Wu`NQ46Qz18Jo8E-lptF5j+n#^6Ttcn>M zFIjVS^z!%l+9lK7IaW=-WB7UNWXyy^eo{)@7_N81ReBHCm1iw=O>xrt@SvKZ&-IZN zrG-Akixdfp&CSGFh7PBZ&!7dIVvd!(aP%^bl>pdkixCx7$ooFsheXgH>$e^g>Hl88 zcX;VeOX;V?)@YRRikxSHK}wdzh$;rF~{xA%Kt)U0qaiok{(pH zb5P^E{7nDmli$m3jh)91wB~72 zJvcqlH5qD~c!q@+OGo8Aa|iKD*>z_Mr{w7T2*Urx5En55wERrnHGAacr?&2V-ma_+v|>KBW$v&y#Wt7p@;kg*p4Glzt7 zPrdlPdk{9Ipo#+#Kk zR;zsobS7+gzQfNu_+8Y8!RgnBMcC?j{k)!rXfkkK+kS0(@aHRNjTF&d$>@cov|XlW z{3b=mCP(;H#Fg*@SGkv|x;4uG-2S!gJk_sWdohG}?tF|T*3M!y=V}XBma}rV(Pd_jK`TH>#Ky8iM_A} zyCVt1k=8ect078b`%BI2x!xrhz8Dk@&={-;G;GU)D#eRT z5fowhDAd)r(xZ5$H~~HbHE6lD<1QwQmMQQ+RVr5l@6gFd9~vxpUt0}xXhjO?oQ|~U zdvre6S$N`0l73EzxrV61?3C5{H9nW0iQnJk-tBaMZk7EjqG#1V0CvamJ^keSC0XTd zuX!h@O?IQi*{}z|41(`sbd_zBq>K7>%HKLauRf_zy|^a9^?i+Fu6uE1VLr^n>U-(9 z`EFN z)Rbe_kiXF5vwoT`{!=f|n0kpVTei<0R6gNUd~RsWGiGX%o`3#BS9?=%K*k6X_aCAn zd2{7zT>jdzH8YpY>SGG7Q(%=fB2&J`vUBE%C+F3o9{Wd;~u$*RQ`=)`*$*j@Ro?0dAi|Aws<;$z=80E-TYOpcXLNn z8SpN#M<8bO@=$rI5W&6G(_#XEbq+8*B~#hB-jK8&83_2@8V<3aYA3&Sj@!U*)o{rE zymi_;sK{re9_;A|X;{N=#)h2gntA$ml)%qj$Cct1PV*l|@z17yCK+=kPH!rl>Y~20 z0I%Du^O!46;}G4{gN;$Noc1zCsXPKK76UaLa?X=zntObfJY<#!M;MZqR&<=VDj@Ux zErI+NS{^RxOWOA)6{Z^Mi9rsNT^#ZrUY}9BIlyQ`9J=#(+CGLc3$VQ8`)Q+9d4*3l zl}C(aS)^Uz7_M;-$;<(hybMz(dbpg%sD;xPr#{zbg>Q;js5O)Sqf3hmp=Y<9HZ`%v-19=mpe5^CKRFlubia zQ#kGJ`qXe$B%+#@lL8MI1Fl-uW`P8R1nnqPC^!Y@GZLthtQTeVzNvQf<@$~c_O}^Q zrVV2K%6_cf4~?GVITj(o6|%66A(#o|VYfeem9Mi6uRk_0X33ViF}_0Ecc(AE$%{Vp zgby$nMLcDf~+x0ka~$zCDE*+HnyVZ-qZB9uUKiG*AtjqqNDmfZ{?a?PoBS7eC~5 z{rqrXr9HHdIZqx|+XcKXi2<@6j=3iZEFqKUdBTHb&BN24ceF z*)#-n5md)0>dCkKg1;#AP(to{;+gE!obn=wLuqJ=lKBjjG4KH3y_U>=|Nbs=b z`O$>acxZ26;NK?5fjggQ;De$0m*~HmPvxwQq zMzVMoeF63SOb2ir^c4+LcJwFIcty8=y^ntYmB)moBZ6$sMKJVyNbS*-Z| z`LvePuvI81t&nwifJBBHYGzXw7Me())rpUT#tj^I*-i}7n6w{w?eze5BKs`l0D>C! zF|ry@^iE*iJ%-M{7lKuaNtRCjJkeRk7_fSlCY+(c4d~lR_@HOZynK*^#uLdF7KaC9 zn#=?vKiNU+k%hRihowKrz)Pa)WZ+J>vl$A42;LJ2N|$@bQJ~Cw#jTcU9s*Pfl(}r9 zqT;W-oHP?b*l@79u%!(EZc@Pr!oK;Bf-%0*1)PkS&m!`Ps2k|oYdJRkFHdTMKP<|0p zw-p@QItlQnqQB1IJvYnG@WDKzVv{`}%DE?TJYo%c0{w>CV6BS z&sgFUWcWf0f7P$ite2y>yOY*sXfzQSO`7MDt|i0*>iacb2bdLG?(p0OrvOoYOp;_G z4L1i}8>XcG+5(hW*isvQPc+PHB*(}m^*R3$ib8P%dRcP36=4M{t~v%xLv6F+y-lca zHsHFFlac{yWuQBp*y&Am0NOh+iYVj4OK-dLdjcgCE~bSy$SmV%hTH6Yv=FwGbV4M+ z2Nj|g#M26vjO+y_FPS~CnJCpo{G+!14P0H!F4c;^4G-!H$0BZAX11_2U=*zyjaBq^ z_bec$OEGu1+Tq=Xb!rJ$AtXX1fpSe6>|b^&4%ga7HYuF!l_1;tz}@666`iNw;bw^8 zMbWVeGRR-WZ@FE(Xf5((L-Gnq5c&tKJtw3FIh1bp(~h*)13Dm5XgR?Fd0(izVW@b? zLx?trgK&r5iTzbQcNMJIto~95+vy0r==ZiAyx)Y^kUKC}N~y}9i<2``yMR>hgFxSb z6BN@LB;e{x!tsT{<)<-bdF9_DJkH&jA!4Zb=~F;Q2bk~fbU%A;2M~LEg82k0!iPRX zSRjHPg87*A@k{_RT0GXURPgtH{^FukN&p1)odb(+&Hzb#IP={H>KAjIlyK92>$gT{ zpdzuPgSkt`WOoNirnSMTz`F}?>jc2@ih$)4`0WE>#SI#{3KzCj9AsPP8eR_)$%>v4 zjC5^F_xFv{m@CPMne9qP%uYXAx)$56P{|1c)FgS{;O<|@q0^5!Xal|sunzj?!R?^fyNF+(#7Y_per_pl zumW-{BNXO&LNq`33(o@_Hgb`Ub7QS44mFizff@HV1|PR5B1h;6W}-_J$Z2JjgAle9 z3iNv(2!Q~h+I25OA|eRx;48tVq^mSnp?92tw3_jEW&V@oVRM_r)LB(Q(f6N)JPsAq9|$e&dz^1z`{t^qh)hGzg&!-lL-Y+R)5oK%@9Buvg`0fR!M7L{|qeS!wSgyxamv zI@v!!+O@u)5T;pl2Hml(BK>qyS1tS5Q|NT$ampou(}=442ZVOH+-k;qWdYQe-o_u2 zz3}}#fXhZ!*8zC67xm0WlQ}AgntLMCa*Dr=gYvad;d`$|4m$jMwYm=u4fhq71b;Np z9Wm?yTV;XPLUJoFfmYl2ATXFl*H9+?FAe;4^sm7;lX+7hu%PEuLBT)~fXIN_A>5$~ z;robO2|hvXSlvV_Zs^!zxdEb-oL*#P;bx8n62bpEitJo8`Xsdn4sg|Dkr_0`PpjMr zZHU##tvK)blgT3lafEhsGZ}{<+kn#*+?QXm$AZ}B>9{VwFb*={rql+6;DJFELO`*J z_{5RA9{5RP%rRlz1mRvg8jLg~&bW84?l_G_?|bP=Jc-npBx#5g0{4OevHGjvF!W_! zQ^e^!%oF26nqhiQu9{C|b`8ALM-8E}E*mpQM8~`&?1az2;8#8n>F-5zkPIBq)ihve z^InL_tKnIU>ey=FPj~Rb-+y%+MpO8p45}CrvIm)jvk}1GI`hWyJc#H%Qz#?Y#o!@3 zhU7~ecZ~hhpZq7FejhP=s>=P|)hp3eheQU-kIj>)k&gHe2oQKbHG@Fvxco85UqQ0nx$ zRU0(}YVoFKKDknXPs!B~z2tLl&k@sqM>5f@}g_ZiOK<`fqcS z&HR!DFlI5mkDI+Lvf7^ZSHgPR3jvWm|LIulIEvw=q*2lz+0}0~q@^o`g0WEh|U86u7K&7u;2*tc9JTua(@e;SB2?SNe%o!ll zU;{{ol4hfERb${nG$k=KMS&+|n~3v>XNQtUts}DQd9`()_%pCG^)JKmabmd|CWf`w zeP4OS9-oa6AVsPYPf+qL2zZ4+WwNqB#|emPd?@gTy*(9j5GWWIMD0OZFR;fw*qoCR z!7jj_ppQ_`#FHoiCJ3s(&iI7@|3(F-u4Q=*VmPNcbu)P^vI(`F){c~-BFZe3Q6E)7 zy&tNb?MH{i65grH3#5*^Joa{rD%B|p`vty0RA@X?aZ~YHjEqmC+A`{UwZv zoOICg{y0xW0_37|8O@T7B*TX$b;lwS0m_!+H$D{QTMkWeJ1Un60;j_xZjUUdg8p%!?2LFLvWS|9!gn*Er=@m#f(Ckuxgf^dklysN} zu?G28)s8MT(0s6)VVgH^R>Gb%qgVk-oE>|JLdrOAV*a9Iz3mpQPa7PIyuH3Rg1!74 zP#`ltYFdv9V*Q4IGqE8fvCqSnw0WUiew&$zcvVs(E z#j_nVe{++gHOE-p$bW>E1GyF7j8U{Z0-b*?{8Un)*+r3z1|5TE5Ym8}1AJfkQ0A<6 zRT5ttew-W|YEUE#Rej6=jVOg=BZY}$3Ftyyrqy0#*B(dH!!I=_#(|wO0&E#`L(k~T z)97(52i#PLleRy9D4mQ7F8V&S)q!WcW z3f~37ar1oL9K$gjVb+(kZlZrr5EA)bb7U%kBAH^dR7y_-)wcudNHG}DS_p+8n8Hys zX7rhr!iV^D`~WC#*j3sh4Y0Bx-m_$3G}NOeGm>BYx>cbOJNuO^Y%58R#jG6Flu&RsaPdo(?_sG@j+xb5r0I+Km`0~_=?u;`^R_nqsIBx-Hz2XgQW5{Y#yf*69`yv1FrM_BQwI8`M+ zO7jLUn(_eAg`_oV!OFzDtk0;*tBy@B9IQ~aSE#uLtU2bgrBJ6zT$K^~#4|9XA2 zO{i#KqwGa88h~p>4*&-E{hG^Sou+-ERy(O+A+*8-MR>6H+%6&(KqA^dI83qM0agJ3 z6j-f-RTCjy4HT}LPOt9eVR_+U@(DuU`$xF`&v_$4q5bX|eR{=)unn5dzv%&fJk%qA z|7m88c77lsO$3^tqBu#bB7E3j@}M_BQvLx#;{;#spxlpKn6LmUrdZXVRl=?myy7Z- z4nF9R5dm-j_A}Q#C~2|@cAIEn{-{P|kSC&|qrO!9#0QBBB4ajDG|0#W zo&W%l|F#gkWEX9yIQS!NF=6V@e>}f@O#U6=`aK}Uz0lzqg}eZBh*klxcp_N;-)4fJ z_bhPL+K%YmM0nAU?jp|jzZgw|W`dEo0Y<$|vqmB2{4!i2APriyshpr{<7-F@LVL zMBFX#v+ET@yHc_lCwfmou8f9e$*$=3dMP(FA2!%kx*Y)Q(AX=Hqe|vcLA6Yjo7MmD zUcq|+T?KXk+>}7b*KCp$NeGDCWE&e%0M=KF+6#1tizOkFpa1H3 zRpg#rYNbJ$bb3Qrt(_*W0RdSu4%G~yFu{d5$Oc`4H24<<03liCcAhNd?{TP6VbHP;LSc=rj)XQrekxr1`(NA7B-_=TULew&^1J>8aO(bNvIs^WABz|~gAk9fX5J}7-`FA>8P$xMNIDsYUrwxda z5vono9?p)SxhRa|L*>TsEKEQ;_REGtZ&h^7%WV%=&%GY*Du zA?B)0@j?h7YQY=az>Ou|II?ZrxM$+QE6gh{E_H8%EfIJ_S~TToxL*TiTiT=`vAjCf4dM&td55LQ6)2FkuQJ?)@XEvPdVd{FYKGn0g3qNAQ!foH%a@(_5$I0%46*K$IDewl z5~U9kdvY$L{6O7iOX#`YSh^x(2sDM?FDf%`Kk|Q_82i=dx1`F8F@b;}S{_U7LT06?-&!mZvT)KZSd^;2iFcn?G z31~Euyfr`WSAXz^uAj2}ba}{!+%#6EsXn4Wef8M*u4ogo63Z2*@|)U!x+e$dN{Ob~?5u%sza z$}3`iYw&J`lIsc?wtyf1kFxiU=em90$4lI4Cq&AKA|yN6RQBpNUN%JtAuF3`*&}_*vKlTWm?O$1ScVD?2k`7~eUbfd8 z}5p&uE}p9VG9M|djE_4`j3woQ}!Gqr7 zP2#40w@*;rz~Ae_$5^h%f&cq8Kb*Mt@7(&o4q*v$isog+_7IM${ro;bZ@&2PW?GpT zSpS01aRhty{t-!juh&68O@^Obz@LBe{T<^P5qZ^t$EAT2O$3PZOBzqJgtf_cBp`|x z^nZ$|gUGP667Un#fQzr>zr&w@?5sa`c#{sCZ>GPLzSoj>9zXa;o7BH|;g3Ut3P8$l z??EG3Kfd@=8*EyE^R>QBzXzVg|58?df|ws4=iet}5y=`cMYnnQ{&#l&ne@<0{o{mu z|7?&8;{y7%b^ikf{<#vnukr6UhtGtv!IvC(frOfxoP5+mcJdiHp0+vr3|CY@D zuPu2(B)~e2z4v+{_4AbfKU?zF#cqNOu*I%I^HV}Ud;(mfyHA7&tOeo(uU@e3K6BUm z2b2D7EBOC~52Xk`gobvOelfeJd0Jl;nQ(i=0FI;cW`#Z!Uhx)a+o6{>7Wz(SYwZfFZw-{L8KTBar+r z0Pdhfk~uN}X4aQDk4oxd@$F*Q#0Ol?HsSFjkB-CKel3+V6o zWNUs8V>4VroD#$SOk{$INEEq%FRpd+S=n4 z(KbEd0xJcLSZF*5^4$VKc5myCb^P_eexIJ3?r`p(;Ew>vn8!<*c_Z%>SoayAFIy5*m?M(!%{nPZt!7$zPzYtTwDEdI!JYF(VrOL`yOPz`QX>D zqmd7f-f-&pD2=O{i}=LLZ>Np^SGUF~oRL@;XHnnV|6E<}z2x6T)7=+^E^7eqQ`k?e zI3;vog8210@N?w)GOH-0%*Cb$e0eWLJI>dP)v-59w` zaQLOM0@K9@4=$ly3oSO=+bd;g12MvK+Kqh@qnz^m#FW!?m2+dx&t>!FAD_@Qs2O5&+SH7(CVAbL z$sK3k8N&V>b31fN92h06y_W5_$6bYxpm)!B3H`kRAFdGR{i|Ig0-;9*H)nGicqZak z$)Br>5$?Uz-O07}{g`(%#o7F^E@-3d1`MGI$fG$8@-lz^5|xYihp$o5-s-vk)nx4b zA!cdREc*0}g*cKm)XYkjnmrj`gdzIan!{ zW}IVuvqJ0TbONl}TsoXtg4=U`3s@Ve_g>7UUzjvMp%d&4uIb<9i&gS&=TVM9ArL06 z6huu(fAo%nOh8cH&haBGJrHs^Kni!F;1){sz1vSIEx7*4ui1kxChUH3<=C(`p|+4bwhyB%%j-)rLT z`@RU*h-^3Ks@NMf?q*h?cV{(dh^o`K2ii4z=07Ia2J>J(r;mbH+M`-p4xvuC2I1V@Uw#H#alpbSsXJKmXMc5=%)R>S^s4@1}@vY{Fl z683WyB)mPxj_ht8nC?lGmoO6JaLMvshVxP?J9->0J1Z{X^?VC;?BRC*`b1DrMg2iy z2_YNXu4kJ_?5+6kV@V`_^ZDbGl8G_)Mrccm-c?-bb#aLRJ9wwXN2b8IS%RX&WTu%y z(mKxbHFWA4spD{K5X@y5KESv1Sz%#0)n8*VUN>x_tK?<74Tz>2(?#PtA}e(RMU1oPB>t>g!7B zN{LlDftb~!(dnWYSum$bI!tDY+0VY)&)b|Jx-J$+NV7Gey%V8d7Z&j53-r0xGNi`a zluT$ELiCcEw(33LWCD%|S=VB!{PUq2SiM<7X8!#%3eDkqyZ-=u zztVErTbyW_#YDmA`(5T$i_CV_~|!5(Imcru!gA#g=Y#&x9P~hwTtYhc}NQvzs#3=GD=SU^kF8{hI6S zQo}F&<9jrj;H>}oXwkPrq?QAM4u0A~-G8PiAp_i;<|!PPjjz4(((W14Bh6CG(QVo; zTSSCID{v?mhnQWsQ8+aO`fF=BuS_OX^S&z2w?~A{5N<+4)QpnHTP7$dt(t_#jI}q^ z?8mU7;~`29-ijZJf&e}C?J4D}4xjTODnIhWqNh+=ykcs`+!_UANNerHb+z^Rh4 zf?JZ!(!zU^%MD`eX!DZsNHkcu&?fspbZ@u!Bj-Bux-%t<{n3WqNgXG@y9;kX!6;5n z)$!xhxN04565IWSf3E#|T|fRdqJ~<=j3}f-IIp+FCO{nF08>!u^9A_4{fm#89nWrU z^zO)R(=_?uA?}Wz9RUD(Ashfjow$Wufdzf2wjPsE4`Bw$Q5r)v?Lys|Vzxga> za_wlNz{cd_5?lvaYIOj&P?*ZB*@~}>MD=!n>1YD@^Jk%#VbQ$TX$32Fso?jY*d z=Zu$pLL+OO@|Qb}ThNJV5Zu}PoUo&YnHumDj8SbD&)_(*S5P%t-6$EGl@&irbQnUTYzu{WfIppV5_yMIOhTFc_WAFFG#q>(9GQ ziJO>r_xq1O0lo{Sh!pAD^EKmiQa%$}>Q;H8=*)>l2jNb+tqJRjV}wDcIB_Nz~zBW7OrBHP8^y}ORd;G ze~8I!=@8GcdjeGG&k5)z$i-^a`UyP?;S#7jm;pQJR$Eg~%YwiRgussF%UX%c_(y9Q zsPCMU68+WFB~*pWM&`L^`HN#?^xkvf|41hGl5sN+MeWCp#xA;2oJiUp=m6iHebP#y zB=qy8X!Fvv6aJ*vN@!;X1LfK#ExN+Du%TKh;aoxG$DBs5jm>{#ZscD=Sr8pn({Nb> zBR`>@bBk~~4?$}e)5%ph&C$CN>Kz_Q{I#Rp=PGt)Uco|PgRfIx8GKbVZDj5>l|DU| z7;9>_??c)a<<6>@{*jLtv!(U5=-6VMmOJY7asNQUPcHQRH!}50F|5mQeH8Ry?fWHA z%HX?iLfCP8&CGY5S>B1Cxb&ra;H?x2Nh!16^F-MRzFa(L3$X?5lVYw*`{jHdUx4r#zDK@ z$^%pnS^z7|@uyAm&rW4G3K-UNwo60JU|Tp}%c!8mqN~ekqnr%qu%L)Xu{E-a!Xb@e zsO%|(mqC|fqBm#OMq(T9S_A59tP6yPN`U9>@sHi}n*$f(LR3j*>I=DFN#bOa4LURs zGq^2>Z@-BQ4YiZscM9UKwSIEg#VDKhg*rmL<98mSu88Ij#{@O9hXGWS?N&tdc++J& z+l#Fk&YbD6PYg$*ZDdiEvwgs%K@-rgeA=Vj8rsNs2xuHK5ghbWkX6P;U;4?mB+AJF zCN(Ww?e}WY3)RFNH1gQqo_#7Pow>DIB`BL&;e0p1rh-^G9wjMFD2A}}>L$5Lkzy>x z<2!&Q^e`;VuBUdOYGdEf#|B78^i=PT<{=w&?_qZN*-Iy+7HYuK~91BvAm7v+u zDr?tr!1iu0=V7PdXvv{EATie2o@aY?paqhC1l+fqnA4#jiCrlk;WwQ8l4*$W#f*J> zQ8O=GzML}-*Garl5jpo}-h43qO{WnySfNeJVM>1Ou;(j@ZNPnM*~}lA@-)lC{ zm82>hNhrn>hpqph!2c6CCf=7VR&DC}-<9!h$$lDl9&`4Ttfk=2TBD_$3lu1^ zk=mc@@No;)h9ji{V~=qU`sWOVs3+tV4>1`6A{^7D+)(MXY~+`+X?E$|=n3s_Sg#_t z(Xjvan8dS|wSRyIA{_h1>rbMm^-vC_Qtv~6^S-`lgS-?|Fs$EGt6KWRc;4xJ8chJQ z^gT4~faHVnI?E$_H;28+6v%k*-__R=)GId-$J`X{@w^$$w!RC%6nk;-r+B z?T6EB3tk~}Kq$q-o=o7_Ymd$mr#D<1ii(58j8=8d(9deIE1p?%V=`ZYoJz&@n#m!`5 ze2($sp)VbPoyD?@JEtGKMJG}|?!BM$;!7)lZHa*7&vyfePbPP@@;RI+o>2rf^y_JV z!TA8c_(ROaECA+Z>D{D7eaMtnqSWw2eQE_}lrrp<(iPS(O`+UB9HzYs%B&e^%&-Aa zh2}7IyEpcu+NNS+$D6%De2&{;sZ}`?=fFVjmk?1UN$5H_F#7Ksync#HuAz7*ZPz~i z?h1sSL{em0Sr?;%*T=~CdP~HDnMnd#`WqhmD2eJkyJz~GqS3q06F&+HPN!x_ksZ%I z@c}kqN~X(rR<=3BjA%8(+*XWKq-mjHCy;H;27((M2$W*2GzDi}kHy;*50LL-Q$&Fp zwOZ0;$W7HnavEdwmr$J3kMV9hSPXe8J)I@-4Z?g*1E+g0@IM9K)&C8|_g!jiYo%Av zX;PO`D$?YQD*c07P~%ZG1|8UqwwXSV)KJxP~ zcrL{We@D(#3O6BSG}N7*)?A{5Bh-9u5?Ba2P54ORQ$g&dK&T}Esw{L=Nvc9=1gd0n zdN;=D(v9g5UlX8&j0~NEgGI{x_?N%rOczchK+9f2{yqmi=cR3UMzKfv7n_=KHvs88 z=P9D|)rO+Kd7M6%rwEWvDD9E<+Q7?>uCM(bMB}rahE^)ITaSt5r9W@in>n*@*0LM9 zk1Xz%AWq|m7x5gqC;Ca=BJIlY#yR{LBjqyB?ZOQJ5gKX|w_%6N0X?UMxL}21?I?>{ z`Z3KJdu5|m7ju*B@!CUSbv%jgV|I)_R@a)U73BDp0-eJ3LXyL^B=63PrCr<=76f8} zx>H@5L`TQ4IbH(WR|6;@!_*G%Hxu?9QC!AP`ei4AEPBW)?fjA5r|ZhF_jmtfI$?F> zm9KZ3#M3Sj*o+GR|E7=@&k7|4QQ+$Z+UXE6i^Wf3i3qI^eR#uHd3|y}r6WtX?SRi% zmt~_2>)(#+i(~Y(9@~R-f)c1ew;z4}bo?(-j;{si%$a1>&>{}pWchR9kZ$X<`*=R( z(Vt)TCPv>^>vmQoaran(KeJp1(4-yRx>U8B?w}Z3iP+k#788(XFZp_6Aa@lg5&Eru zlCtj={n}t)OJ3 z7N^`A6_XOyN4a@y)3)Au_93M-SFfYxSbnhJiw1l(_h6=o^t`SNO2B*dF%jye+FCZK zKrRFvFWFdwKt;(AlUec@TU0q+tRvgPBHqc-QbDsS z+8^l!4vfAhlKSoQ;8dw0JfGrmrHtCe2}RL1Nt&Gn!5#L=OXUvQ#dMXmS0Sl6wJd2m zOy!P27|IdE1-(uiyDqcBg)thJ3ZHxsWZtzg`PstE|cM}(pX$26nn|*nw zRSH{n)Qr=E`x|ypGU4fau&+7Cu%+4i6uUew9Hc4=6k{%>zdg=qSgtvst7HH2!!A?= ztnvBITtaOLgcww#G5g$tpwGbX+)o%Q9=ld&ATLd{Q>$G1>?rT#Ya-n*`jIB`(rVf$ z5ZomgWbLXcAh@25O(m0BRlx_(@k3lR*5n?h?O;1cNv*o@(U>x2k4^w`>jtlhT@NU8W#a-z;d>! z(3Gszn_c4fqi|aMRJhd6cpL_2*?YD__DT=wccRfU4v3LYv9nLRbRu=Oe4N~_F%P7Z z0eT!xBMjT4a$9Y8o@|^)Wa;y$AS#4-MaC4P{wh}6rhw&I-I?iJa-D3M=##?11Wm-T z+;jb$N_&2C0TCT{v+Lz?Kb^7u{5Q5SY1P}O{T}W&k(+wBSBkf$%$S`*I2X7hTa%09 z0*ZT&WsMtK+jHBd7520j4|BFk$dHTO)GX$zaL`epYr9bV)wBqUKb#zJ{#WLXlf&!R z-!o^EEkN#k`0MYnS%|R30})L=&a0#I`L3?`qiTYjWG{hoQvSmIt^)$JMpc7f!;0-z zx@{6aUMjjM6V1bkSb6$sH;PoN^4b&EPT4qmP!N83YGK;LZN7glHKUZr1ope(&en>V zeU_c{ppVBSq+%N__5wTBy#hFj7Kik1i_S{oW?iRjxH8~P|^UFC*o+PGz(>;90y zVTa9OuCk6<20k=QhXDVw_IVlFawGA%(xySf@QM}KHn;G^mNq6X5#A3w1tIlt_V}$> z3RlIswNs3*t_vwU4s$3NI$o-BFvc~zY^i14Ve5lQ4E7WAqtNUj52z|`UrzX1u#o-sfgUwid(zMN$fB3*d@!b`U9pCj2o3>0nlehWuSelwbJ+w|l{eN-xHhw-FE zT}CgC)Y$%v&3GrPDLQdFR4u;$z*Y88xn>z}cfyY*Wmg^iAlk>iZ*L&7(F!Cb>@g_P znaucU2P)4^^RINVAK?tcb|q_X_LI=N>D5-%^Hr8FhDmphauWpI11d5RR*b8nAa_1>MW zUZT_|4pSs6%c6zmHI19_(+y|U zZgEFbF(Cu%!tWz~pv|2}u4og^X3{=u0ae5R-W(BZ{5Ty^eCSd>RW`G^2^+f@)c`OjatK$ggMQeR z)duf15@c{A18|I5l74_Kxzhd{09dYJi?-VN&eF|&(e;yIU5=vW%+}%bQ~C5fV8pim z;P|=9RhaIfl=ykGL(66%GKXPku8OiTjUnb(c(_TOd*d@(=2NCZx&>Z&J@&?)N|#Xl zw2)WuvlFHXCKlOK1zRofRmpOv{EY$S()nO3(56DQKA4jXMc?nFC%33RJNAfB_QFur zxJo9Oq#k+wbk&Av8~3%X?x924t5?@M8l6LwrivGjL8iqLm+Azk-)Eyl<+GkbQ*Jx$wiG7G=0N$7d zFHdZ#IIj&Uq%9I2C7>eC@UbVXeA$4c?761dVe595R{w*;B_oK1xRzphg-(P@#4)7Y z()P-dVyr`p3>VhVagfIFf-4Qd9aT;3wN+Fg@eBniG(((@q|Tq))i3623}3 zBS~z{NR*|l=ZdGwH^y2Z0H0x3zCu|{Tj6;%0ApMPq=3|J^Mt$*0$fOU`vI*>y3%(Bmz9ZY|6Qj4A#HE9r8XbBR zBJS~Y@%k$>Rt`>td7s`((Q2n!g*YlQE>n$89WN$H!mGMjj%io@i9ct7VI_g|8pbJj zEK^M!w?;9cwbQTlK=fhFnAe=`! z*avA7#Hpd>#wi`o)ARd1!xOK^iF32l2KPd!9{P|#sXKq4oepvKWATBqDGriXqRlH$ zVT19P;baprUcLT6F)eAfC3c%b^`FdPx+wLKWH?%VqB-(XpNiB>%QBV!hy41ov!B+av;a$1nx;&scWs*4=eTl;Qo^sfD)1WfTC9Ibw5SeWOTcU)tM8 zp-o@N+FzXe!R>OM_**zjxW6W{+G|kJ3g5TKhoVk@?C79*)%-W`U z%6z3)h4U7&Pq^Y4?ToRziTCqUr)I%|^D!EiT4fZiiL1r17=2Ev<2miLeK_T&`~JCb z62|t=y!>w0UvycwYlc$T_PG+dN&@+cE&sniIw1bgD*P)VqAEKx?y zsz$1;S^FG-w`l5nOnhX9FM}C<$_*T~A_)|2ASWjvQoyc4(k^)KJ?x`U7vT2H?CctU z^C5t8@E3C)G;m#HF(BoNxRXzS_lr0b)rec>N~GywA_s-+?JRE8??(E#UR<7nX7Su9 zL~UM%#k2ycK-qxPakENro3&za#h$Kw{2(gPJb5#bSY?PP0KOq`ZPnDF$<+E<)|Ifq zBX!)&tnJeW!tyu9%htf9 z+icv~l&dc`6)%PgPri+TGc!dDcdw6?pze!#>-9u$8!M4Dx~h!@oSfmz)Df2U^czTH!u-8jXCM&RUl6K9+galfjfo*)WLKmJ zU16mBaFI3(%p}c7!&|_)@ggRIPU*aTw~}kl7z^WSqD*~j+A@4^b3^O&h;U+$xk7hh zE1k!YsK%vdjAQTFKaX>bIxgi!XIHl^`OwKb3i`NjHOwoBt&JNUb_|F4*N_2fZbKCF zaBl5n#*V04DJ1`3(~_mO2qn%FDyj?DLD1O^ooKmol!S)<_$Qd9_>hZb!SX)1Cw?nk zsMN5=ou}1r=g!sZKpYoBoh|S2qsx56Ki-7wtWox9e$zJ*Ps4Im!ZZLYr(ZDpk z+MAe#+Lq2uyFcF^bd}bWKs7tQ$wK55(QnIa#i;8ITeA%yxRaS8s}%`Mq=V<%1jLlt zRSXYRESU5N`$VgZ#fe(zH}B%eVsHYbOnXuiuvoz6NdO_Bh{Lz8c32;{sUwyy!#fTj zpDL6n$xz49V{X=s11`#M1V;b5O=i(ms6@!KRau*#+`OECe){0{+kMC4(W4lvJZ&ZoFjbES5_L0h z$v#9j;wb;Lz|jlrS$>muL}JRJcSCE}5P)dvCyjnL{ZTEE=O`^?rr*4S>b$OXYWC^B zDE)lyvL3kNO`VSom>*(8+FGYoi%iS*#!M`d%Yp*c*RMD%PZko6&*TsI5)noWOfReY zXv4wWr8Mo@b)c}R=W!};nL$b&grhWlLF%};)6*1`se@6F{Wv#NWW9@4s!lc*Sg7^r zN>{%kv7LKKxpg7Kb8UX!O_cgLgPGn#7!!P82Gy8SOQjQEn;B~zA%b^rLxAD(FHncWO(D zaj>xXYXRq2AhxjM02{R#3QjFG?u$KKBg}KLL14z(HmH*OUb4g;*emiYy#6R5VYI-l z3}_=5TVhPw`SVNON z?eIoN+KLo1kPl4?fw)n?wYrR6cL;3af!#2K8E9_recu2WPRWfaz@sUBEonI@ElbzuNjekZzizOGM zIQ7IM73=RjaH}l!m-Cp>E0eE3G>d^vR9;AA^U^H%&k8uESL;5eBt}s%+U(k`C>vI; z0H?dwTzkN-{f!1KCNNw8E@NR?V0G3G2h(fk&4)+iaeG9lR2?fTPBQW#S)0BxSoK|( zH(1%YD-623G||Z9rd}M;e~%0K-u^kt>sncMV2YP)-*4;m4UN4t(2Zz|v*ack1OAeP z{Uvp5ALGL&o%%iK5nK_ zna!D4=9dg-Xqh{`5@6S|HPousMroaSzl@9PfwNQIqe5SsXs{&dp9Z| z+fBYllE%j|p?v(T&#S@{`#e`VR{5^1kG@nu%hyiN+{e`5wI>cg#>45f>x%=VKL>98 z^kV*$p7sJ1hEWlOzed1}I`8HGu2m*$HHr7s?)8~v0-@0m)GWD;nKG9`sM1Ts@?tDs1MEC@bZtsa zAj)0p-f}7RE;z8%iR|wzd5X(rKrJLE6JO0THsY(Acx9f72?AMCJ6u(USD9=LF$&as z*iz^4jl5eKv*wMlRVk;$$gLIA)7F}i)WZQOXRSGvkWAGOO(4fAmCxhkr%W(Wnt>oj zj~Tx}*=}FFKAviPKD3$IKRZWZK+N{DI7Ltpi9cIXAIq_R{wp|oPgG^-WZQoV$N#w^ z0G^QT^rM8*_pupI82cQ5kV!%dI4ForOZW+_-!XM&4`+3?0r-6h&oAxu`-9<$l_@9f zgwNW&J251&c<5qNSL~(j6_*{Z3QyAT=#*O0>A{(>2d4v*S{&*jb} zednAd&~;ktyvpreIe<96OEfVGOvB&PL10bdeJJV-2TlK|6%tiIu~61`##ADTl4Er( zP-$Lp?C)(shYrCBII$C!(9zUE%;BF6Y)Ybi?276Pvt7mvU@C^ZB~o?Vl;M*`!o!g3 zdKOdkneJeS|4dy}OGOk~Fc^7gU?7}BZ-krf>i*QhZE{n`o111WX*Qg!(=$F}i-L7n&wQ~oWFrwZ7 zeO)`4u#yNu_fasNE1LB?w-D#eKEmh0FH2*{#*|ev^!hHs=@iWFY;<1EHoQPssH^8! zY;VkI3y!MKk`zI0ljk-~-gMA%o?yuZqxWH>#Q8orD8@tI;tv9oOvGN@p_(D%+Hg!ydNVz327c!Vy_2Bv5HarO+$bfTQ9?G6dYVoNP{MIL zSMdeimkBI~0_{XxuyUk;0#KNlTtM zC<&DzudAu+b7@jzo7m4klGEaox9UCfA$7i$2hW6uf2g*F0zK4MV12ZX%rtTXtI+nS zvAEA2;n?hkH19xJ9{MeS1U0n{(q)_TAK4co>fKwZmtmJ$E!1-*2FeL=uQW{Nj{9zx z?QG4+ffMbnJ#2Ji;_%Ruu0pDTRyN=y$pI~Qw^sxizY%|VWdFBvQpoC|M6RTOIy#D% z3#~`sL8_}4w$(~ff+xCe{K@>r&JnIIe_x()D7CZME)~zdPfN2KtGBJJ-3JX&b;Xto z_jN4pW1M>!E_z+e5^poaF6v`UE=04`QEJ(Iz&-PRjNrAubtm%7q@~?TG)RAviv(z7 zOGsYVCHCES^a_<;iGhLANnt|N7?PDGHI`vlZ!GDjDmd;Gaywc%nj-bMQlb)tL2e+4 zh62btAm1luc?fT0uGfGQ7N|jD#;i$jMLe8eINHC!ibI_NwXULBaD1`-??Jsp*WO$Qih=TZw4lqtsLs$2o{ucoPKukUU+|9)j6XA56VVT?O6Cmij zDccrIk;d5n#QBc<({yw=9;Tb|0r zFYZ@%eMbHaOP4p>y$GwH>YGEoV5T>;_53JYHs_FiJOpQsq2+XU8PNO|UX>2ETCV6k z*>xNlsxUf>nmn%Qr5;jprcfH;2c5cA;-RlWV+{Xj{(M|kkCL)xF40>uVOjzqvb{W+ z&27kZ*F2)(*i*4FjPy8FI&Tp?q;{>=XZr zbauNMBRCSTCDqcS$l0WeO8z-g4(jR4DV$*#|JPFM{0dEzY$oRl9%q+vo;j zF-P?IS*TJ47*m6KUh!rr%XVGlR9qU^*(%#n4xW$wfSce9_g5M! zYPYjSw?m&%-+^N1&4sD46Cjv^1zv{Y1(via+USe@+`SY0&F~ZjWIn(e(m-2K+@kY<)!;x zDcx&JQ_X!QWtd^#nkwmMCrCYtx&WhQ0N6F$SSX%;!oan-02O8bmcuBgL2UR+gimX> zDafFcfs7XY3p0RC?$sXhMy*W%>-Y%L2x6^?TaR|`I!SvBJ_4Visb+4}Bq2kgfhb%k zf*bjbA*`f_Q^xXhvJcaW3e|T#t#bim>}`Fm^pz~zvM(S`H$(H7hMM6-O6n`gIc}0z z8}0SxP|3Z4c6;B1`|KLZ;jXqO=sKG^fm+O|Zw!9oUlwOIv~4iS3y$#G`hx2lbY&B0 zHKAL0{g!b(eWHHk>KX3e!3c~oR)QUI09c*q8*P80nGPHf9`A$;wLPl0*oPSJk3~#Q z-X8IGg!RfmC`C-A16JW|p<$(cumgSuukMJi;JWgsX6fgjv6J(Zc+PpKBx)73F~%u1 z18U^uUbh%q=DUbsCrmG8j zL@Z~`*sKq)fbHIS?C2sGb5zdwR$m51rSAR=U^zwOdCQ>#<_4Q#_~fZZiV4ayF2n4x z5=a%Zo{t+ql+r`r)MN9Q<_n?VpQPOQQdRr(h@j{V`Ska47z&H)cyjzpihMa%UYBY5SW=tOFq`5BIn`b_i^0j( zO>c(HKm-pt9DZcBQI)~knjrcpd@Udo;C9uu;pG}R1HFLtUt9oN9%VRPLw!D|9?d{9 zbJ{R(^zH|OdDmHW@p#6!h*Oq?6L{Ict7b&2r!1G@KQXMz&FGGR@nUMUal6AUNX=ct zs5FiedE5-Jh!aou73mWXLbo{q=)twu?&}DBh?gzO*2zGl%FztBHqlY-w(u9CL2Sxe zCx5k9qkG#>2L?>L_*~?0ziY0Tyu@n{fRAi9QE+=l*R|+8n*;_(;b4~g)&){Vrt2)% zyI38;mM#s2{+)GL9tzd)ufqGugDQQ=)jQ{D_f%b9z`tITcE_|5=(I?Ts$Q_w+r9gP zFe4XMj7>2gcKWB6kg09`+cB3gomTfS$KL9i%!A_6S6^M+01mZq{$A-8l&{96In=Zl zA)Q)3T>|jQ>l3+g-T+q-OswbWG3GUqcQ+xuW0A3~g0l?#r7eiyr(PX(r|}hs!b}16 zfl4+$#{bJ>;IvsWYu>TBGj)u|Q-{hWIgm4TgxIU*GgyicfvrNCJLH z$l#0Mg6GT0Zql!3pPqRJjk4+Tv(m*}U~d~Nl`G$nEDrgw4fYaU&pjbmM!?O$ShAy7 z#thWqMjm5gk7at#$lDtmxsXo{V#sFBg6YF#%zABep)s_!!t#6%yDaxm5C;t7Z6<-SSyfvlsZub+3pvzTIdAkJi0b*eGwUA$Z+o*&eX1WC5lsysMWgVY&VJi zM<|EQ`4`xex4YQihSQ6*Q#`$kJI`N%3%_}4vvGCjqX(|t?e80 z{j%e%th^Il6(`j%Rz!a`9x8(t!qi|sW+vK#%5$supWSN;G9`tj+x&$5@hQTUS#;^# zaF;=TkT=CO9+Zjqdoz!5aQ7HuwYyx4Y|PlAS7K#|C{~PN(J)GSNBJohX>q$PE*=$d z$bdK8=Q~2Ak4s zB8}t1r;ziEzKPl`pubR*MGlt}V4(gO5^PWgmP*}Q9zEr&y{CLO2Dr8;rn~9c%znEY^(5mgHJJjj{D7wFMcE6B8*?1D&ow8 z7Oj=&h;U0nnmZggOe53!+g^UmOS+Ln|{Kw+Wb+1HH!!n?6y z^)TWRNmL8ei{&atEHTg1diTC?^rkno^VC-Hp9BmE&8+fiz&KIqD(%ptp4av&FIAxBT%iuM#X98;Yo*YVq&dBVCV`eWF`(Ci_exfB$Kjs>;N1#yrbj7$Bd*J;Ui0x6Q(yp9oDJbmgoPv2g^99cZ{1Eho?U~ zKHy-#U9)(BML51R&uZVpk592x_KfyX_T$M|3spe@K<$b483DE~8_l7mLDVYmF#D?GU}VOg8+fbN0kU_QQ=W(jl3Lye@@ zIK%e!Pm3@<(RIjp!k6N2Ehay>^}E-Q>c$;c{5p5u=&FM_tThFEU&2=T;2Mw|5a_X} z`5Ma`RTaOY3jC>|wmAA- ziZ(A9xMJ+J=V@&*fY#jUbKk=DYy!G`J)d)HN^onjg`Rd)%d(ymTSEwIhBD6U@iPrI zbH%;DR6bDhB9N1DRJF08U;`D!+P&DQG<0ac85K z(n@xX3e$UGMp>}(+zK>AEVb6ikXM~sQll%kWO$=y7jQC_`}!Wm)L!Nn)AnCuw-*zI zMT)2YGSAtH!yL!xy+pu$=+?(&F(0yEoyW-`N~;zkpat1@wu<*#uUp8EeM82PG1dt& z+@nyozf|p@$EhMyJ#{YIjh3A{jHX8nwUT$S_bw)I4Gr%=-mk6qYps|+c%4cbw?zPg zt6fDzt=EHYd={_9hI?fIq}U%yMy9CT`WYS~afk7|peH`U@~a6pomhtH(@5ZBpnan@ zY~nGbjml4P;V-R(;sPE7IPV-DeXG~BJE;!}+c=mmEy^Ps(;asPN906OjJuNYd+C5m z}V>hLj& zX&1k7J-=L%kkv!bOvfz!FVvJsM5ven_A0fGg=Plj5RdX%UNGnCaM^hEMX8MGT(GHf z91aJUZ5T2=hf$B-0tcil5=T0rd9dZyQ*un9u2ULLz*pDH2Gby+TR$}D?0PKRstuuH zjH_&|J}&2qJMU(dF5{bg2bZ+VMDK`l$2XvhE#~y~`@p`oMd6Y+2CD6)BS^~x9MWo( zlAKdKGJ*Qb)!U&aiRfR-g-kjUVr)GO(3vAiGNVZNjENvAr;(!R>A6#3>NiOdHFn=q zN%+a%6>$o>oc@iD!|aBilh&$;V$}%-xkewQJeXo2oVRoJ&m3(k_bDC&|X>Oay)e?1n zp-yb6)aMmTXsSoRCFl>oQ~T2ydq*-Z`te6sYE%{H8Yu|!;6E+Io`F_&aL z#ZPM_DpZSH6rbz5y9Adr{jgzgRZ}Tj4Ov+&L0HR>HC?}gvws)+*)NXPS?XRj@uo`^ zN`pS?Gp%bG4&4(efkqU95Rb9M*ZPl6-XbTwQ8pi(5ri?|o=D4<@5HB#NC@F7{&pn%u@3RGmwo~~?1C}I=FzE#yvaBjoW32>7N66)5pDyjAQy*l@qCop zhMC<*b$N{crx#wA)Fy(24d+UU0`ml6R8elc;6sKC^Vb0O6On6eVaT?-Uaa}eBYQ^G zob+#@i~{}cG`mfTp?*)6j|(ej8hwGD_ds=F+*OgATA1gk`ASuFp?P3|A5*PjS}-(> z&9xYTh%AE?a3s_;-^-oj4pq8>Lp#7u&nVa|2e9&q)(iqw3XTm`Bkxd4#UeN~d1YhG zylz&bu>ycsC9uv4!?MY3{CVSlA;n2)DcOsf9cuiX0P|%+ys6`QZm=~(D+I}iREO$p29xp1=5x~DV=(4N4z`%KHRf6s>o<1HwOy6%q}rY5g(KzBJ-YMG zn0KgeVnGjPi_93^d2cl2fwmX0*;;DR=V=SX{Kfg4fhAzZg7+-F&#l&;VFd+sC5n(` z-MZe@N-nbLUcsw*q2U57Dpne!kh(%VQ%eB{~hLRGtHv$}QHsT>5lFx7~zKj~s>U2;$i19Gf>$f=A z_IrHHFML%*=y2SPPEuU$N4<~8xlnI2j1r)>TKXf~hN_J^=;w_oh5wd#Kb`LcBOz*? zHw4>8RL1E~`@uP-()sK-qtc)f-{(f(Vjji?9lh|?jW%FX83RxE__aD9;|%Jcf+w`V z;Kq@Cu+l&iqorq`R%;15TcYhzvA2?XJvUa+b4w%@v*jwXP))-LX`kk7*9Z)a4)Ho( z<^GckP}L775`JFrSgXN1&TrL;3Uk43A+k5>4GfTwXE7*_o!WK08_k%-3%Xpkg7^>N{+ zKF-egXC?!@QX-*m`$8Nkid#W2N+IYx-;Zfcun*s)Sl zgCCsMitdlP{daILe&*$(eHWsv?(PwvX)j!2S-WOF{g^~cMNQ(leZQ0RxKB;|P}We@ z6*}6`5I?c;xm^=rf-cra#p&$2YPOK54sIH<+0i#_ee!cwJV;u5p_8zNpo(b8pT^Ks&FK~AZo91sJ7^4PvkD(P4~!CAP$G{fc*jY(-t?4h z!t-rF{;BK5r8?N~I=5@p?ki7ggZI5>#9^VnnH)Z@gMAow9H4$^;_N0=DuL9i^q4hO zg#DF>D_b*`Ec6E}JP#B26EZSZJX%#bZ~r)EKt9}zmeLm#n+2EQRV$BAnS%u&&Ky~4{hSFx)Y1^^IwdK9a^;Y4t~*} z8Nk%(Ju-`cfc$Y4zQ)ts8ClX%gj+im%+3J`GLa$u+}h#2qt3+?x)mX5^N0+@v=uMn zcY3V6vpaBPx?ECaj)EZ8lHkl~*X$!PDJs8O67OUSb=khGmpPnLdgcO~Lwc?}t``T- zCz~7RkDLj5C=$ZGwa5WI27fi&*u8HRX2{f|E^+KPjeA+TW zpgs-$d0KO1=^0CU=_`pWzsM@mg48EuZ)fIYv&+M{=vTTU>6M)HkVLi6m~EY)@!0(a$B>djNLHURxs%E?7=B*)7+-}9`Ahh zt>H*BEwyhi?B#71C%{%K3BD1}?mDKJhiHTW{S7*dl}D?Xj?x3v*+qWu?OyDCnk}%g zinD&R1MjdjL3M!YEm6@l{Nt%L{`AC0mNV@-}#{5WsE=k4+!p>;o z76m5y+o`}wUCvmL4R|Y3B;Z#1OC_P#c{8tG^%jgFoZCLZ$SvX=Uon`0AHxGZc2e(* zq1cy}x|n(W+SXj5vAdzl<9^PQ@1$CT%Q^kp2XAPRv|4l8pK0{5-2y6%AA7S%nW=2qif_kD!)t!D?D?@*V*SCr#SA_v}yj5m(o-0eJYD|k|nqvP5g82T5yAE^0f>Y z?V_4~#2)4|nFNeC8DD?Layj{+RLft`)w^H(QHm}52kwC?dwXSa!65X*zJAW%x0K8O z-cfz0WC@T^Tab4fMDJm}?kNu*Nx)qiYOLu!s2U*Vcz|SbLrXQ(tKDi)Nkin%3RDT@ z$q79*jIdF?m~4BaOsK4IXK80Gv~(p{&-(lK%lL0SuXmI&jA`%#I6(SZ5@t3xtc~k+ z4vaQ66mENU>w7}M{)AzXsZz6gD;Hm_?!IGAs%TE6ZT{`2X35l2qzLwCt)s(pZtIJU zx<13<`WV$K4^P={P3L$8`XZv;o$Dgv+lze=74}xcsGeG7>a;t+5*KxOEraHe>6IQ+ z9tWs^BDCc7x+8QM$icuvsDLCWyza~Ke>3g5ymLscZGS%Z(O2QBe{pcOAhtvjlWG7y z0vX$ajL-wZQao?DI^~ghUX%hK=PRoLmAEUMq%x;h?q;y6+gkz&YR z?XOrRH-^3vDrx4hP0JzHUttRQ>#`vomm=Zf&|NG}iQJw6W%dJM~!x4_!-K%eM@e0$p6CD9r ze8WSE$x}ze9JF2pGv_1ymbQC*v{cs_-c>y7f@V{8Qq@Ye@A=`5%o#W;>xkyuvX2YL z@RrcfDc1kG<%@ez#Fh8*?GEEIM|Mt)T|Jkd@YaCUMCv?SyUq-|a@|+|`cIecsb!(_ z`%gZ|+}4LUxMF(Z=hq(P@I@cY%4WgnMy=_aY>q@BEdC4&^ZdfjVmd~=C4Ne6oi=@CNuXa5QS$(_8h_KshXeBDj5Wk~R0aH(u=&;Sum7@P$`b#ZS9 zR%#>tlC}o}&o0?`V7UV75I?rvZ}D=`lo}^Uz}|&-wzyyfGrGlxmHR=azyICz)hb#W z3&0I9NZ!Y8d)@kCh7u4Q*Y5Q&5_;neHOvdWuZKaK*C9S7*STZCKK027xMC&_|My#w z9>CL3RTLp?k91|<-T76{=_x;YY9IbvYMNBe^Q-ZABny|k!v*Hj-DTnrSh@RR)2(IO zK483I(wEfx5onj*fxM*sD!_ofP4?Z6Tck%PM+>nteF^IlM0L9*{|czQF?}xLY$hkC zU*ruL0hM_5QwgP4?bSAaMbg$dDfatwUbgp4P^vfI>PHYGhM1Zzl>8^oEn$BPEO^!m z-j+q%B!l~73(ogGWGttxYzH#gs@>rmQan%xok7Md5_u47uY)$MxT7w<&AdBb`}AsB zY}EE@CXL+CXE}9w(Rq08;4YGWRFTFbxjYvA?a=vNOOOpLxkv-i-W7@m9*#qXS0_Lg z0*^3GuE_z@`*0WH8>N8n-wVx0S_IR5rTS4{Xo%jZeZmYPK$?$UNo2j#7c;c=$Ao}_ z-2}DJhPEJ!G?AhXLYwi90nk;VW<|NGNJW*2?3-KxPxS3#Tzl+$e3M5g)73bJhoi{NPg#Cb293q~bsCVJ3o_?6Cb$H5K^s9es-c#FC zQ^ex&40Pgn1lFB@;5&H0U6!F|1LMJT>At#i-*f&1#(>Pf58v9$b-CfW`YBzlxY$!K z}eb?`V&T$F4z&_B;kWTceNO>GM@P4$r;V048p(G>FC5Ft&i7b=_xv66wKe zfASrf!a6i*I2UuJx|QjlAipL=r`>Hm*EE^8b2M~&_#)gcE@>E*%Oa{5Tec?LoiSi( zAL`PP97ln5fqEW3eR;=EzPHj+aBSdC`#balU)(Om#*2x!FpULt55U zrH35!_0WV!u%p?g0`fF|cov&@I%*7u<_)&^-MCu8R>-KQL{)xN4}Ys8o~_!Xe-V)% z7e#8yrq9U7-+lUh(3sEbEPAafz6{>P`Rn-heg5R9zvJ7Gv8X<{WrQk=)tq`X4^I#_ zq#wzB;u5x*cbD*Fc7nqhL>WM>;F~B5OMehT6)`>@aFwDUE)oQh)X3`IOOH+kqNl)R z`$e_)2FjJ-2USzcH+?xtxjSbrpHu#eiR%!|X=UTNheZcU6gHT@On%i%K}-SiGEroj zdF$vW?7uAwup`%@;&gz$E`^=osn@KVm+I`W_7~l~eSuD-y0#y|&0$@CGIFz$gzrY^0|-;IrCd&tjxG6s8Y< z`n0=%GAsLpI(EY^=?&m2!+Q?EdU8(rIFUBFS&Mu2BX(-ytH%#v1uO&OQhy*>PB2M* z8=5aOpH@buY!fDF*uy|bjX`y_4_k?XZ1qaL1uYho#}lxZ(A9^IeS14$;9! z(6#GaI%jAjp*Bq-FlTJB4BkLphIO)uRQ&kgn^IT&;~ zUrRfzP~9;idY<=NY%-9b3a2z5;d+$!YT0-mVy)xL-aRn5WE?a?0CuY10tnDNcLNW* z;G*uE#5Onk`$T;FdcAZwvHR?N25=#77%aY6SB7}SQld2(){I}=szGDnu z#B>bpR~hc8v<@{a!vDs7aq`m(Z46AOIn^1;*Ut|}R_S4D?c02pU26}`yzSR%k)%ku zIgza67@N~s7*qrv8QEx1LOk$KZJB=J6ox1Mfc0kJ<-ka`DAliQAsnVqvIHya0_)6YUEijq zjDli(P&KF@pjLa)as6a#(R@MAU6&Bm}GL#n|(K)+@MSTvE#I7UIsb8eq(nFZUVv1?wrt%vryWW3f%s|vZ(7T48v@jGvSVyUf!4UbA67{Tp^tw)wn`N zuWhXlV=Sh7kM$v3;9+WV{ynBI3nw4HTNOel{^5q#@arK80TdGkP>N+<%Z)5NH$LDy zAU~5(a&yZ-X=!PW&&R&8kMbe}=#$O~bwOchmQ_I#w!Fo}R~>y;4VH`ya8sUL^5J$u z*pT*1$mRaV0)B^=A-G7~Q zZP}@aJs9cgv8XxCVP$GYV1x09<}2v~F7}0MPO< z>JH5SAcB(p%34PE8o(izL^nan)?EY=bk$(c=+}r7PcokAd-kbcbNQxCo1**9PItvD zMxA4qW~RO74i?eXy_ao8QHqR#G@dzufPSXL+#~`eY6tArhbgn|A(=0E5PqFMTm3n2 ztK$63Omm`9=j^QaW-0y~U*i|R(=8FMH2o#|0{$!7tLgRE2-6od;6Al&DFsQyA8=Es zyRupg8pbT@7spDAX6_!iY7nN8epzuq%w>v}u;K%;50trrt{Ll;vqc4Cce!^--y9nH zytSGt7Ag&}JxMp+wQW`o#Zn5C- zgbyN;sjVNxa=(tl!hT#j+W7hClp|JJfVi4zQ2&RrpX(~N1zbQ6f8qxz=fRza1F%Rv zyhh@HZ43oe8YaO1-(YF^WdD}-gM=HsXY3?}TPT?OnWGmTM~e*zfO zy;8sjncQ-*7#03Yt+K;YQy#lF+pi$>7@QRQCF5fvK?YnkG z!&tZ5=_>>Um#d6dq>$DE(xG64q-2REfLjPio4=k%M_|w3xcn2Bxlc8geqpGRyWm`SG=tf~ zr5A_+>|u}@){%xs8(6>4i(fR;FDZrI=z`PYF?u;Cj+NC2&|Drl2Y6LB3(iAfF`AD8 z%e!rcGsq=um8SVKn#okW&4-ps0QbKoeal_IvfWR(`Kjcqr;Xk9-?k@EGAlO#wCxoq zc_e#VA6xl;`a$=7wHc*5|3L4qD!TT zzve#hdrsxB7{IN|l2ReKyo3fe`G2?s>7%$d_TGQ*~0(2n#AAV0`hEgCXx;8E9B}bmX0>9#? zK!wt81o7T`MIXtr0IGsZ1r6(REAkoT>)jhLEdQ0b2#Uu+BK|~Y3H~B!qP7&f(@*`x zbpA+$$m35Ak*hM1z|x=|LpbFcmD(V&nhli|0DAW^n7{tz6na{1U0q$8svRAhR;^nI z0;7CD)2mIl(Mr$`fYVEb0cp4olBq-yb!9NDo=>K@+T3Xw;%gf{nf$i zoJ3x#;gA1m+A$r85crBtB+l#w+E8MfQ9@QYz#X#Swa*1+!W8qVoR2<#^Ad(x?qK1e ziH+jKKmy#m-@Dfj-+``N7#sXAa@`3D9h^I|tz3+uhr%^PN$_vh56MuAO2Ss(33ay- zr5gi?Nbw}UbQ+mLAiD)x384-EwUZEB!!Vx4fMPOtDH@$=aqfDHE zJ_rHmBv|&bs3M=)Cd5P>o){ZDTm7Jl)>XnCWL2L#3mA7eZb$zZVXhAT6BTs?+9C0c zWj3mVKeVBtp|ND6G!>tlSF!Pd0}0Up*@RbYEVuwvVRlaY!`H;QucjN@c2Pe|SeODM z>^fXTRjLUT2B+VCEq|Lr0=Y zij7XCh+i0Jcq@Qe^UKz?G>amXPIs9#cL zzK_p>B(w8nETg)l_#NnD8YD4AKK<{72gd0xs9Evn&6MDw^j{3bS7J#VhAj0*zIC#L zGvsh@BTll7!hmfcGHUF!>h}N0{rK-7Jt`eE&@j>w*Tq`4;0_B=a*)w0_t5+6*CCT@ zfF7k$WtNjbdJ#nzWifEDj{ort1n?&70g-*>Hhb~^MeB)by|?PmZnG1ZnSwFAJ!2pR z%D<9+M1T`wf_$L4I~l&*qY=ZFpHLZ8Mj(Elps{hrC~$8L_hirjA^wWrZrJ@X8c?HY zBqT!#n^qpiBBfSj!uVHe-3I|6vApCkk|A}E^~)b=3s-^8{@7t}jc<_-B zA0CE*35Zz62xH@eu>mUj*xKV;64;E={NfURq!v^%|1XXz!mtHMcp+>H6755%EWXH( z{r47@jcD8A0-IdXJw)oN;ASJ|;M$G=6Z5G$9z~Whit`Kn{UQ`%rZZphE^rWu<0;Df z{P4}_;{WQIQ5J)0L~x1{2iy1c?O-_f|B)i#8a>0lzZ2TJd6e>AJV~V8--w}m&2JN=UCBQ^|cbJ7KU=dCp;hDOVm#Lu_ ztPn#U!vC~O|3T187=u*26RA5$;t1R{a};INJo#dO3Z&xBAnsR#M+LFOYAK5Ee?}@2 z&pGaH3Q`fTk1nRYB?7fSXgTsNzlZ};(~{mYjj~Ha2#k)osmT}$MDP-fW7FP6xq*}^ z!^DO75Qlj=t0mFRc1pq5)zw8xwM5Om_)G?;v?jxD3QLl}4A3<)Y3u8=ANkghRy8s* z;ut*k}r zhb3^_2E=f=r`dh+!Ko=@Ir$sZqZkGIX)!08BPz}UwTP+N5HFI}T;K482n+mORR3>6 zL$m1bu47M3Yt=jW+0*_ON{KZh;t%(Osq7j8vQ)6ATvDQ@nb~rUFcyu`5orTe4HM?g zZ-dZGr@V|oRpZv9O{vr4h_IxvZKXQCr7DnnuPE^(@H}}V32t(TXY6{Elg%gz8=$;g z>5TP^zr?o#p<8jis7p$^b!s;pbwjB)B!c4)q>LqsruooQTc#NR8HWhT5a zQ!fgpV?$JS46}qDwv4(6fjh`HHU1`gYbgc=O74S+xsGY@!}vi|&Nx*@#LJF7sHAp><0} z%>~YF1oFHug;Anjl7OV&=q}SBxa50@EM?@Zmj0-Y7ooF0Ma5f+2_YMbAftkXD1c3c z2gk<7`bvh(t8&)FQ2!NJRtxan0c8^s+0yVJZ!MyFTTOE0AA6%1Ac{2&LH(6=mV z^}RCP+uI9e7AIK)m1^w0Om})VvST2p8%;&IggS_Z^W2*Efh;t!ThRR*oW9ouUbQ!F z7T|WJEQdfn2<1W}Z!OoM#w_a7-zZt%D9Ed>uGS7vX=!bhgA9I>RO%}B5W2dHjgXvv zkpI->nJT{E7UNa#uR1N3-xPvAGbQh7`%rx~=*mHf0zt}|rdWAG>3MAJ%I#y?11w9f z>Pvn;<{Ef-+RF2E)#5c|{YYY61#J9@w`Ge3wKe?kt8hbWPD~0$Y?9j!%%%aH%o+EF z2rw)!gFo%Xh~mg0?sK|#;t}|EG@+$H$p@hektSCl|dj~tEBZe&?*|s z4X77_SW82oNSmy+=Qrr-VqWo!VID-@2xMy31vu5DS6!J_IVKF~H5G0nauiu=QLH|_ z-EBX9{4iNvI(EG9Qzy9#9Vq=elRBZGEJ6}&|Hf}39th_d`Qy`rm@f#3AD)>^6yret zL~~{eGeQl7PG8R2kI{$MUHylMC)`8SFv*cK6owV8S)~X9%jh1yqM{-PAME}0eNMDa ztP_qNm2|E*gtP$`qCArtRcDaBqayuIP)`tpsOpx|gYYjBd$HHL7_Zvvto>Llx?;m_ z+Jhz^#zOH8<=m%NZ6DiU26ZZ*dSoyFb17~ox{TWnpvB&$C%2;%(Se8rutKq@U45{f zntZm*L-Er&$GB3K{DOjY%Pey~u2UQkgqSg1K?kg%25aWcvP2Q@dYawM-F0*)ZxI#c|1*qmPA?$;H0ldgk|H6O+ ziMRaBc4SG>eQQZ zSarbnA?eefNEM`T3R=gzqAcAz%H)-$Dft>WUzQ0A;xBDua9Rh)Q%eZH)@WNk`{Ynt zvN(2XOY)HSW<_r91@NS?yK#+79}eB9(q|&%=4cDCz_ch;5lkirU%GV_n0-*t4nw4+HRe>lYh zn6cyqL`S6S+!1`Nj`Om1G%n0;*p$A>g_%*&73Wn~mi{92l!x15&+aQC`Nk;Pm=^)^ zWN(67(#auh_=9=Ho$!?kx0lOclUuBh{F`7>@M9zGLukEKz=}I0dl@lyu%GVCBLzUi4k_VFOH#@Y`_nqt0$BGo)^N# zKLP(jt%rS?nq?S%W*hL?;QDvJtOIIdLqTmnCo!&=%-c@92h2 zslEcv=dp4KqCru=9m)`E?->vc3McM^|7FRCX0^1ZK#$AzA4g1krg9fV79TQ~9ajX3 zs((^1ixe;5+EHR_<+I9%z%5qr>_v1rUYX$&EpH&7LQ-RoH{!SgW2U83zTcgQ2t8OJ zC`e8JT1ujZktl=)(2*pptz=;PiH% z$9}!cY@a&MEc!VXK)wh3KYaTs6X-+xj8!fUkH^QyT_erNE@HC|p6qMk{sN+P#}qyw zP@5TQymDJKUk{5~hOfS03(R}2w>(}2v;O?`^PyZ~?WLy%Oj-Z7P6m+sBYYCjj95#6Sw} z8B?OV;4A*d(%fS&fP%EgRHX76Xp6;WMB4}|-zXREa$!^aAORlut*A!K2@brWZK2a2=}6nvcGCG?Xi_b5i%Mea=U17S-X2x`I^BT;;Yf?<~y{XJTCn)#Suw zAu+{|0Tl!yGu#yNA{6|ZDj`RgBsak|GLDXpkIpS7G603exkWIRB-+C7VbsTj&H865 z6bS}>_Y~#HQ+}=3gC5?o1DZ17GnM*SJu0$%pmK3|aYzOH#pt{r53?3KG!Y0ndL7>k zW$TmOp{L#?`U&>jt9(E_C0SH!t*W7Rg{}M@edNFHZEpMSpTd);t?7qDV*@~NF?d%!3*&ezf z0rtJ?C%9&Z7>2j$ZuE|qMwAdQO>Cbaju2Y*$BwPYNRs|8*}1(htT0bxI%HBhnd?fY z`umzd`bYf%RkV$O^bE?Q<#E+WBU%`W?R8kRB>Bqy8f&1|xn{YX*Tp?y z3f+Y-BMOP3(x_X0e)KmX2>ARP_{S?kym8-GCEisRl&p50Z7Mgwgn5@{em`#dv2~Uf z_L#@|47Qw@$58sJ0%BjAarmNr3%;ONaES;u{u$Qitiku|x?lI%-N39-+Pgu6?+Nee z4;j0+I9zH_^lfSWh>3P<`qy|>QeR))Pj+SIRxB#D!(LyvPtk}fy}ej& z}TW5@Q5o>;o>l_;0biTP(INEKquYSvRxD z=a0+#o%4#MkMt$qYVf1`FFwU$G2?v7_trx3>72BN!Ti>5xn0wtB|eqY_#Bs_)9c%s zy~UPlR1_9JRtoKK^#*A|!n9E`q?rg(3%3)D$h^fFVgz(x8F8n|px>u45!wliJsoHx zWQ4DhjXC5xrkLp!wqhLer{vG&UwT5s4c?1UnWFEdkr3NwYe(B6kvp|mEPoA98&epZdJXCv_lXKt<(2`{4Lh3a0TL=8NY(X7~Gk zlsaNR`(3I3_D0dDfUtO-cw znT&h*0UXA!!JZQpg0U8cdawP%se5HuEdDXis!Ql}0(#}6fikiJA>`{82y|SU8%-0A zgG%hOpEWG8Lx*N#l2&=l26VxCSo8?xy#GVEn?QyS0mcuWW{^WC|EZ&jzMO3ywC?aG zCYZ|vF6Vx%&aW;?b8=)?`t$|<09?YLYZNr}9Lk2yZ2%Ib;S&fMTCFqTVE} zO0;<}k69S#+6u|HvLr;OEnL>G_s#o405F+eKuOF7fQguVBK__@$TUpPwj`w~pju)` z@WNd|H!sMK>n*FX+-HUT)xr~$zCV|$9$#%W~1Nd;C?L0@K(uVBU>X2eA zZ6XW}MA<|E616UliY5w#Pjf-d5PKb11v641{Gl&(kCr!#@$t?3BoP{zo%Z3~>7hV1 zZb)E3v@fOd4uZz|fDVBlRoa%;;HDU6x7Ds=-A)^LK)y!Vo(gxG3Pzl#I> z{1JAxg*9t55f<_-$>|jGq2jZF&6>W6KhtDvriRss*!d56RfNq~;lfk|EM!m9us&h+ zYg{2*(DH1Q$If+H3pr(Fp&eI`e;n$HyN4SJZ8gA%+;SDM} zAPg!E#v0?QgZrpF1SqKyj+dj`nVO>c2^WtWMOP1Pq>+m8PyJ@7XeDe$%Shzi$-E*A zrp67wd*qH@KfC~ACH3vlpD4R+2nF%y;1zq&M2uwmbrLtDv>TCrtwW`fw<4d04kJ)L z`LXY8_*tIriHX6RJ;-WK%6KudRDf~}I9FE&)MPNADps<|{|h|si$XI+QR3EJb1rWg za4VVAY0@kZPz+E#nVcZO)V$AfICqz`wd}3S`A5CUv4E>|z{j?=pt>nW84<9LR&6)L z3=a+MFISPG`!LwM&=_v)R?d+dhYe7zU4Fc|@9d>ZG~qgg<+agEiCw>BNDzuJjyU41 zR0IH;1xYecsY<(U?w~RwfQrxm{4NL{Zkf#v@C&f z6p(b--)-8O5S5uc8#*L+HCgDKJDF{+GSlP<=+Q2ri=?mJcV9u~h7*q2b{i!Oc?75~ z0V!a8ms(;1)+J(C((y7C?a2r5e+j0Z{(P4}V0J4*h*U0$RGWOS-AAhb$u-BWpo0ck zYLVCVTJ>RIyj5r8>Pb~Xc`mmg1|D=E(H%sp%P^k_*ryGQlTP%nLInzDA>$)^3uvO* z1n)(Q6xg0D{>X3u9p%Zob~+-|v-ENHWG6N7g={yDG@K~eZT4R2^gC@#!y$SWmP5RJ z;;S1Sa;{XFe>?G;-=bn8e^3t&;Vqi;_l z-i?ytrQ+=8TaxW8+-BgUH*LAp`&eqAjuN|_sGuT?7VyP4L0!$1eGT+H_r-9MQRK-S zMXVzpMXUsaIwSHw_@Gqjf=zHY>RRBg+=Jb1o=U%f4?)I4nJlIVol2L6qX|)=dI5|9 znHWg0p;sGhMmM1{f1W$ze5j+K4wRp%;Hgo3oHY_CsrT@YScJ=Cx=gJ5pknDK@)6zS zlzgJIxA%cp3x@*RAYGp86{tuq3(HvEo)f14no<|&_$SESm-cZE@A6F_t;X4W<~5)hDqv~iTFpDuniHpV;|Q- zB75O{f;Jn}lEQRQW@{F&iCu(CLy7TCdX!KrZE#E{sQ4vt8i`0OeNY{DLP0o9GK%q2 z;ZDq@xFSu>IV;&$KsdS3K5+u2Ni3rHQ%#!+n%bNM@nrrh8K`9tNdE)a z$V+cki@T6uIN z{xwn%FKlmZt4Ss&>EAovm3o-z`p9W&Ih|3wUkF^`F%MW}R}OE)aTpaM zB^$%xAw&^B!QVzza|@}}vKO`YxjLVPg@v3L-(e4(c2cE;2P+RzY`ufRP5pEB)~c3g zQIq#VU%m)csLpF-+=zj>IpP>sI=&#&JPBEbBGOVE8bj($UonCpsc~DG%P5f=$h8Nk z&S`%JcXXT$i9bTxpw&3G)k?=T`4G07R!+cbbfg9(&=!OL`0?Z0id}7kqF2JvHsTnX z9w_uZ@NjN8B#AJfI2nh#O`5KM`b6(F{;C}Ki*U$!q@T*MqX4=KcJ>sTF>2A1*f~(# zE^4O?*0H$#`*L8DSQdh5!=sV;?2{C|RRSr33)r|jp$GLe2(^t=13B3gJN2upi}Hs! z^2$d9!2jH9r_Izp;tUylh=}r&c|x>?7_OC3VSzbMwkqtnz>ti#YLyK;&)zDVQF7>? z7%~G8jaHm1pyo|4O}5lh9>xEA!JCV1GkUK6RE&qz!U5GiSnOrOHR2xusi($JN=`>P zq{-F)WRj$F_X`+XT3WUNnGNMmMuZN@ml9TirZpRKhFAd!16a*g5inrK9s)?F0~pQ}gtTBjqd!ceTz{U2JR0TM|ps$n=sUjOrF z7sPW8fd10=zR-2f-_MhZQi8zxl^_maS<4mZRUyqN0QYrVMK6{ZP~z;O&>s?ZJsgHW zQI89xljYY*#uw@Dgw+S1AW{iLIA4JbqX1YtU$zno1ITd3LStLNgtzq`8Zk!5F8fGhPB{TQUW-3FPNa#oWcB*79~e%?4yG_BEbCxsxn`7v5_ zN2d`F__gS(-W8Y6hsq}h@(_UqN!DRRa6o$(Sfg{RU;AM-a9jb2mVy@q+`|VN(kueh z_N8Vf0thbqN&gB2BnLr4AF7Ic{CAcD5Gv&VA|4xG#Hc``Aww`%j$ODSWsTBgn}Gjt z+XSfTYYq3E2V31#3?lc`iy|kZH>8vIw!ws^IsK7>@-#dsTl_tVCT*VKqLLKGNrbu%ZwJ!@1cuRmM9sEwJqa;yN za|W0=RQOnRLfU-vi%jVsN`u>iOxRI)*%o*5$yl&Zmm z=$@DV5hO4@>qBTp`1sM}<{s zN$_yDi5k~c2Zx+!B-adTxbZEFW`myJF}_vIF|CoB-9=q0Z%p`6lIL zw%pDx4l8VLbZmD@{jp_-T=m_CVppH4iI}lo+wgQL@2c3Ro2*7pC3WYGdQ}?*l8Qy% zah3TV>NHW-7YvK&$h2yV-QoM)yeE|nx~OScFd32f0TJd8(HmJ}#Q7jp5SbyLuyn6B2C{x*1-SN>tks{HW+KWb>#awX_ zMz!N_hXgM=YigzJ`nw9L(MU;AQz#d4&n^j@Q*6HxU2WP?elsicNI{r|`C7xmRviY`69Q{MgY2MjHFWI?iv56pe%3Jc>hO zJdzI$_wrU>i+k{LS3yU)+ocYRX@%Ji&Vl-fA5YKD4b2|x@ONGj=Vl!f*ib90YFuJx z|MF&)SZwjN8Drx|6-VYb#4706U+T!(700pnTg(*aeSYb2h2fD~O4Gx~UFOWItqN5H zdwc^v@eSziq4xvv#%PbO2QZE`(HR8MeHcv#CjOk6V3-_->r73&mJSl2%eYa~x1q8d^R5lfA}v^FtFU9o}AEGh@rdJDii&ibU{e_wF3+<<&i^q4n~E z0iVhA`ZtZ@v4O7J```7Hr<_S1)s0^L@<>r}-Bgcj$roPTZ$^{TX=ehTi{@ob2}Wcd zsf;KsuG?Wam6TN+TF9HArZBXwq&&*Nq(ohFzF00xDE9N#Yw7oZDB^{zH04@h?Fn zDFn6@!vJ`0_0Q@p$}=z{uT$oNK#FUpcI8lXtNc0JHRA={xm(9>&$E5o>Kga-9M6+3 znW0YY_K0o84IHI4Ilve~GRpvc%ZECvMkDG4V$IV9gNxxzr)uTu3ykd)${!q(kon54 z3*c`2$JUlNNliiVs0LroRgpaNCJtTWdZo1T2a>ZLuD$t5{hAWCdAqhojM@c>SS^(t z63qGR_S&l5w#qh2ZYcBWd%4(#R+dExH}A=^Qx7zpILoE z-eH}i&MxOK>vT(EnvCs}-gu_AHXr@Et{U3&@!u)qq@(s zThph*O`6i}B4gpAkuAPo3j;%qhTXo#YQE{2y`Nti;cRStl(!`B7;x3np=0^=^&_(_ zooBTxnmZzzb-KKJCzJQL^90}Oh$+e>kg?)9i=(fNQu>_X*%K^n> zt6So=^BXF?zqSC)&t+9#atk}8xu(5>r=i5j%{R@BL+$9-URR!;Yd!uVlL)V9nA^T| zh8KJ^?05wxM*~DUGE2hec$0*ZUKa8W3{JeQHtDeKQ+iW$CO^W0gSRxS!?!g2Z|0P& zdAHXH@Pa#RXBz?b-`rAWd;061niu#M8h7w|?ws3L<5^emBXZxXku%;UYVW;`i`Ioz zPplmI5mkLC_+`QeD2=!C-j&nfYPYMv+jy=}UBWg+e%+X(zevaERS~P~%&$X6X%UA? ztGzvoe2nYG(-UXc{^&Y)`by8RSanMyhwhTt@L)H7r^3ncVfVPN138)94Nbnutq5jK zitD~9CU({%j2tvl>HNl`i9^%4U$^yv!V$HP_$99XUC4EJlMML2&sY(+wQQELcw%9?**m|#$O_%G3 zJOJYs)hUysW4-HiADx4L%`{K%9nxG`5Xhn1G_mzemGe=JXZNbpJvr4%Y=4w5u_=+? zR=TyvqZGuKiHqG2jroa0Ny*q^uZORMCFRYYyy@_)n;#i*bv+rYSuGNg=@#6R+nsw{ zpow1&8)$5Dbak+ApL08J{i)?c zf^z{w$5M{onlU!CU-lH2_^wBf*P_WzZ=BBB0wl1%?rwk?fnzt&pmv1is%i?0QvDNGC=??y4->y4L z&7Nz`n-0^pQ{e4zx(cetI@x?k%m1^JS*?=$kANPh?+>au4M0fSex2#?@fWf8n!DLi zKfX8Z#ykE!A%T=+7y!M zw*|tFw)nd1kOX3BK9`!m-~D@4)wJr{T_yu-5$WqI8n*RodT*HX^3q#V3LS~seQ`cR z8Wkh;5k|(|!ytm(WOd*;VjZ37SwcDWZn7mY?fx3uOgRkfG#ye9TS5Y$oi$^c=vyLh z6E2?$3MI07>qzXgoL9o$9pN!yAM@OG%ruK`cf|%qIwxh1u5afpxiuVRB>U#w_T|&o zF=9ZYKo3Cn>ANajzm8B*Z2z!4xAlR$lwzDwZApiYVj*JO{0J112SFU+?2%P(R9kdN zf>)#?D?2cO-RPp9DACQt-DS?4%Q-nUUMUa%6gw0%b?*Jvo^ajZsO3GUCgp&a?iD*% zjGHVkjh#}E&q_TMxY@Vl>swiKO)d96!C0$-G;>d>$z%BuZnCP~+U5Bb!z z61E*fV2<458`8QleIQP=U8C7OsCaYVpnK}wTdP|qe=26`N7~k6o4TivLRnU}vNiGU zf1hg}zSk{e-#l=$Bmc(D2p;469LHO(71;D!jSEKv)`3~mkdXBu(b3DJSSB!ZNVfl7 z#GK64Gj^I<1|8eKH;~O9UXi@4Y$M&E&yQQdFgxY{AnV6)FS~BFketC~yu`*<~7Uc048c8tQKl8L@)MwHom zQBV9nEOcVPt^6($qL^rBTy#S&+RD?XAx5)Vu|tNw6A5Xh{D#!gr2dE?y;x4X@e+n$ z`f$Ndu}Z2NhI@hp)`oMbNexReRQtadY*DL1YLE$57P<*!Z-0u3hdQ$X3FZ821<48| zI?=``O&F$;ycZjRoZj@M+rveR7Vpzg-hF@ukqe`))SHkTG)r5!P(=38cAW< zHjPR08yBWYt|2iqAAg~VCw>qxp`{|aLMZaT{9;8O)_3Bod5?dgr0K~!iC_e1-AGvC zH&KQBJP3+Kb4HWCNIJy-G7dQ^4px{<@kic2B*9O@3_J6m34;N}2$njRk+K;1`wrrk z%NmT^aPw^w424gAfn` z1(LH=%n)w>tAH$F6S}5;GAaG%kt_%}aJH!V^@|qG&!?}gQGnK)B?wn7UX)P3=Z7Pv Sh!g&@NJB+OIdzZa`Tql`NVjMJ literal 0 HcmV?d00001 diff --git a/docs/src/images/mem_wal_overview.png b/docs/src/images/mem_wal_overview.png new file mode 100644 index 0000000000000000000000000000000000000000..008c84d0724fbf47c128c5cced39803cf320b24f GIT binary patch literal 109386 zcmZr&2Q-{p*H$LFs6q7JYls?*7SToTy^P*_j}}owh;F2)L3E-M1i|Qx5WNOL^!C3a z_uen}`~J0NWzCp(&e>=0XZL*)qpm8Ci$#uwgoK2v@I+Pv2?>;mgoLVqfd>3V)=DP^ z2?-xbK~_r37x_odoo|%gXT5n!ifn^499(knCF*7{A*%`0w-V%&i4+F%!Pv#z?O*~| zE;%!Wk@)!VbYs2J*a&IM^5OCDsJ7jkyO`g-I_PT_YvupmLglxXF(Yv^CP{#g{6DX- zW+L?~S*Se5FTN|H|M_}D4&_rV#`92&mFQ5A%9&qUNetrCUoUVqChI?ohJC_#0@EVp zmLvIl6$mK`d-UJW0zU#@5F|m1uX1A)8ByqIQS@1$2eJRGb^FGX6&W*A^jAkIJVz0H z^v^>7AvHvlsk_{u&nKRK@>n$qZ0pM~FT1*BZ2@AT5f2ad*QUg;3y# z*L6-6@grJV2E?KNeEBibN@-H(m28bup(t|v>#M&WgVBcJU}lnk$WI2S{tw^(esA?W z^hKsMu#5HCX7P_dmqxsYrGPwn5EKwIv%DJ91tQCK&X*dZ$}TcQKKn10SS5xk-kvZ= z1vu_d@O4ScAN1cvp3wSRcF^CI-%-ru19 zo7PNB0PXzja>QZY4A6uBmCbW-00OG_hGqQz;_}~8`~B=nUZ}m^V}Q~Jj0Z)L?F@;J zEDCe|OIi+x#A;=ADHGdcd0HU>KPscCT-%Y4PnQz|^4KO05i4C$rv$#!ZzTRZ-Kb1gJ>w7J>&-L(t1uAjYhT3grHn$$F~k}B|Lhqy6l zPVH~5fIPo%y-g zr9>Zew)&dMS>!Kq{T9(bAcBbl%XlT4u4%Si>^2?$s2=uK;=nZ5T$8H#h=P$0vpB!n z3vofG_BF%ad-nm*XLl8F>c6x$3tNE@-G(OeabyJ8 zk00p9n6UD$pYCgqhM}VOz^mk`{oWb7cHHq^ju(Q@`FwY({~p|A9DKGKMH^h%O;Vkr zCedg(TOW}B;?sM;RNuvf#$3N{yX+nkS*Mgal#JGPjAgIHSfpmE{2qKUbMigpJBhU> zoq+<_&xagg8+KxRCVX=J);6nBP40^SQV#$qQW7t1xV}DVn+@1&u^pDYX|9y+04;y) zy35Z@%988*CdaMk{yynsE5LAe5o<{@w6~Arv2@;Ri_wN~y>f9y_=?6aSMQ4EgBOkP50oAIo!C6TT-iLu2=K#nhyY!kh?o_)!4>+( zOFbR^BKjlaN+J!%-BvUr(;UlH|Jn(iYAT;^e7R=X0lSU+-Z3evqJ0Sx=a5;;L~+Nj z>3mi%9mWdPPE!p6c4~L#1NLg^WTNy8vW>!@uF67RW5T>!&bLaB4+eOIj0o$+z= z2lt?hW3Iy#z_WtuFe7ZcXUbC)O;N+7^YQaux($n;kVzLZ_QmlJK``P8g-qXD26TPe zE_ZNrOAk7p*{Pd#R(iJm$$e7681n&{Uz)y6r5veojo!ict6d-O{ch4(w!&u#VRF5It5-I=ybd{~#ygzi$En2l=hGi@HN)q!XFQbznJUTB_I zCrd_K&eijBzqfZCvl2dxyy)?jPo6SWVZy+n>|UPI0bny<7d7?}*kAI`d5+DnmwvF2 ziN0IL*ZmGMp5m{3Q0yE|FnfGDIt%DInKGC~ema^am<-2d{E$WREXEDO8*oYEw?eQC zJF0u{I2Mi z+~D&~-vCOdgET{DJB;^$VspQ}*&h@5l(e|cP!CAXmTWL?fOVgrNijo`YFZ7VVqk z2t=pOaYE0f|}wbFOwZue3La7yUy>$aV1SH`7sDz6B$ zrd2$p{q=_Wa9Fa<2{0M^bXC0NuY%vk3%;n|acdBf77*fIkA_;;oG!)oc2+RGHhM5Q zE{!qC8D9!ulZN(k!1Q{w#cL#i_QS?=PEA!2>t90aH`?L^K#&O$III8e^F3%=A~E4n z=7egJt_cqI{d~;(w%I?;LFc4xt@E$j~Jz4)`O^QtL1s$+dnZGF8uFoB0wlVTC zN^1g#8k_XrI)%G-53)_a3_%Rf9D`&lpJrzA1j(02}U{ zpLw6(B>@|S2)G*8-$sP$cwsPk3_ag>PUS2#OZMi-SgE*&cZJZfyUWF_LN?#OR%{Nr znEjsCunI&r5U#M?eq#VxFV`F_F!b=BCovh;3t!$3c)RwTA%S&_)#4|0f8vhb2X@{~ z%t)5tRFHr|_R6Pm4W$_)kHJ*oSlwd{QmU7c3Y6$!g?2w>pW2_Roe?meJ)@Q6Ln&0W z{qQk}OQGw~tTqU+Q*mw2U!R_=X+JBNe^(LULm&3^?c*81KAP%y!J}oLn39nBMGRq+ z^I6CSCd@HsIa*xo)SF&vt4IWcFA~w1;Mq0N*YZSfez;%d`8VA&vW$QDL7lb1MCt-? z6^Dd3coL`3&s?|p$G8kM!&KWwDooC_#`AQGwFy&C&+Nk>cQ_!l*-v7V6@||F&j*~J zE3R5ib)w=uQjhD*e$#XqZDBK0UxY@V!~vM3hnsFu%#1O|PZ{bLl;0uriIGj3rqv*- zux&7&r!6vq2XgOf5ijK9_0>i1#l*>sElu)kG6DaMCrXbPdI7ucpljl1ed+aS&Uz(8JJqZ#L3^IBEcfe)?FzC1|nkDh)#km##+Elm|`QWM{NH}0`eWYibLh>*fE zzR4BhT%Tutgpg`Z#nz|_r)@%s}m&a=lx61R&680Q{U___x7UOA9_7% z`;osB_HFX}izq>&>lpK)vvi)&R{UKH`1UcwCbQ40pBp6=lT{1!2qjXQj=KJVv+TwzP_?Uw z>q7fyySSGlZ{#nMBO`$rb&A~SM9eFes+m`=$e5({bjq5m>Cw@+CQ~q>Kiv5!q|KKmv4>g?+U9#0yH#FK z2T5rz4c|+^*uNCTpVW*JZ8{k>U*N`_sg@1JqtOz}Zp%G~In#~XP>^?$*Ks9L$IA?b z$?$$Y7Puq~hoY4A`rkd3Z}{9Otw3!wGQDDun6I)XgYpVDUpP8}L0F!nL_K}lTYDh! zmt0Bvn#W2UTL9#%}0LVi}4~=XxtV`BvfGC(Ehplu~8Re8d zEAP2BfzE_+h>p4XjMo*rQ@!Zbud%Y1JLn($n@)-*RBiPgj3lWIKp$PNW=MJkai$VG zFj7ROTy`7Ry>sLOGVv1>jNY13v)RmpVkJLrsO0MsmAvJwl7q6nx2|zl9HeOeXFcdf0O zeqwQaFvEa-i@6&FGToPo{Z-zy`$iE+%U)3=;hc}STl^e)6G@SfqcHL1C4(lZ=<_k6 zsJ=x^mTYzJ(tEl_m0#7Ufxy&Za^iYcP}40Zx?uFj|1hK_-7a-PO!)Ab;(|r}n z?)X(|dewiT>Q#M@8`Ek@b6^td#=*1Cinp(DavO+sNod|}_u86u*pqOmlGU~Hk-07N zby&MDo--W_2>y6~f=;23ncG}NzdG--w76QH_=6rc4-%YpP2X0sC#ob%K?44I0U zu*WCgsqCe;9wExFyhwma(7%Hk|C}AK!rA(1cJwWg+Fl6~OQLT@u&ru{!q5CT5UHcB zaBtow%Zbye2}<+2j?A3MK-Kl2%h)GhJ*y`*t02RBYgbJJ4VPT2?qO5LiIxcaaTET!KWVzm~d#?Co9M#f0S=jBuNiL@Ah$u;Y^yxh#9`*H2{fU(x*jvQ$} z8c}&GX3$xpuuJoPRHOM%ryh}t6)vEp1Pcb3UvjK$k5kxnHs}(ZLY~D%7YuE3is!8( z;63fhC9Crv!DrcBA} zf(H(I^2NU1(`Su)iL(_%!IWFEYj=+41##HFJ!t5Zm@@Syfaszq6JAzTYHVLjY^}9x z`}|Z8)kzqU{yhH`_4<3jp16$1t3cx}L9C%jOA1hD5ZqFL?aF(O^t{P$++w|p0Amei zaIpL~QdVsDqOdkK*id;M;tQ1IP@QPWHF)3W+U*qP)!lQh+a2k-@bl2g-D5$d0DFXA zmUUD0d4J9xsL#eu5{r?1;=nO?Or-6*M;w{Uj=~~fEMtCp^!W|;NpUwVk`;kCvCz(T zJ_cy+6!Naoc}5QEgi%DRLI)HPHF;IPM%+t?sf!6o(Z(HF z@<+9lKZn72U1of(X0d_7qzO*=A(zWE@uDPHcVu{F^ekYlia&q|L|B^Q>&*edIy0Y{ z?Sp3|g(1#VhKtfC2eYDe*$qVZg{$smlwmol|ERVF)fVXgYW(o|`ur&&l`=K`cdAod5|*<%K0Z35`rA>ChL4U#0!nTs7*KOsi#e z@P0&9!ZP*&Ha4fLderCB&Z5t4xeg|6C0{3d1ny?*l3^W=2~;_8zpA(`AEW>kK$Sr5 zYaMQ#p}t%hj)1y0=H;NgD_%_}9_HNSK|=PaF3tc_8$v7O?Yo@$YdJ zE=SAzbWnllM3O6aTXI56vr+Y}2@|!KFn7ZO%CZrmP~S)Mt~igQ(@vJ_?27shGzv4W zO!X!yXq$Sdsl|d+Kp&|bQQLg;=o$@BW88l?#DgB4?{WNmkLMPiMCmq}cJ^$tDsI!q z$^kp_HqBX_ug+0CLkQCLDNW%qVr+fi;R?hU#KKC{ajVYhA`TtZoYbY|_-+q-*ZlWvYSWf#x= z>6O(eG`#xGSaMb5EZzl>4D9SS?Q!p$04b5uH+P_Nx)OUUm)z@>*i>tsH#5vh-V~yW zqI#zN!-983$7($3Zlv?C`GqcHurA6H_LUYE$=TH=%P16NsR5I!_Wu zuz@)$hK7i)R)bKuB?9MPJ3ATQ&wQ#&PyHUi<~BLG8p3(a}sd*Zhfy%Q_k!o6QouvGF3uk-E# zsc9Fq{$;S$LXX~?d<^Glkui{Mz!Dhw&>9+$Q`j9(nVc`H#T?{W@L zVWMwUPWF_thNB+@hBRi$fzO_UPcXhdI{d25Z`K@bv(?X0L5b}Ho5x{VLcLRS;Y=2L z2g1S>Y_}u6h|Sad(T!M+%i8_nFEx=>;$+>2l2@B*N=v*6Jo`mm9r~KB-s}bih@fqx z_Z}8e?+?y$?xr7AOd?}=uY5g40;M!x`vitD1jR{8B%)?rM&BVNi5{zZ|9$-tvrsnb zxUtaen)I@lPy!;soM8s+4^bXXNt?5`XAw|f@oM=>bIRZ6{xImdGRtB`=z{5BAW)>4 zc=IZ9Y_l;JBan=@A(*Zyk9%a^eVLxqeNGvRNGJ#UNco$Q{@J1dE3qR`JbfrS?=`8< zLc)FK7W3LhTAE4sOKMs<%cV>>&rNn}E^G8I9m*WtLhC?&pzzncumXyn_HnQ3qfXMI zpSf=*HhQ{t>xE9ux5O8|&m9k(5w1KPB*r3QK~-o@YUji(^^8B(Y*x+}AfiMcTBcDGjKCje3#n;4BbK-xK#Ac~-CsRF$kDw5Q$RtXLV*T~z~Pboily~AAGNTD$Hr2Fo# z^1wuGl&kIXAg0#WTFhd}44Hzf{PSR~@i}WhJ;G;_Bl6RirA_oQbr?>((|pQrV9q6z z4R3b0OYf%43$3i5pRETsnA;t17Us2be#wbTz2oh7x|!?0Ra~AFL#h*Y*?~$_6q8SM z^JetXh~PdCQ8{eE$^xjcLMVi#Y5;0sz=@P z#^7n=oyXsBK4iQyQv+fnOHM{DER{#X-#(4%J1OMurYfi`J+95`a9L7qI)Ap1^ii10 zX(=&^bjN?my8;sA=1n}iI*leZyBWnGAg#U(L`Ks2X3jZ;A@I@9^I22qG=VT*)!-af zqIEoa2q$s>tV5^uZGE>LPzKFP+U&-N$r1{nkYVDUQ<=jv)o5w)$nNl}(y<7SmU~A| ze7&L3k^>~c>71m-G!}Q9dpWZ7i}dCGdY~u?)wE!A{@}WqdnaLdR7wc3JYiZ-!MxmS z#~=PEK%V!$ces@n{gK3R6do-M5O*)oAFmg(HNl#G{dX&N*L_1tyrIK0DN! zXHgEbYPk%_EgBW*yp}oxI!&H=NS4Gsu681ose^bP^<@M-x{nA-EY)xfTcnqbd-8!= z&Z16?}nJRM~3#$^q^sQ1$uTUa&3D0!rv2 z(KAC{zKok*yj*mUopr2^U)BVALX+o2Z~BW_qDG59p+F6EV~d7AQ!-y$KL`trnIH$L zHn2n~Uk=cH4DJ-SS9^2O}co|o9u z4iAH8H?iBK&rWF;7PH^0Ke~iz7N z8vU5CWq~48^ETCN#x}Gg(v|&qt$JrYCb3z;0Doff; z=Wt0!NfSTPwr*=tBN7;CeVd^m zPEhA|tf3Js;QI}AxESvW)zW+oL*da$Ke{0P0xym0RZu`t9;cz_KGDM&ar4fIWJ>+Z z?(j+eWR3Ol+Z+y2dTc|LCq(-3s4WCASa9`GXqnIH6wu|7JN;x_X4gcK zD9RhN@|G0h#BPhxdvoteNnOdkd_}b3ko2 zMm?J^cY6cp@-;UCEmDSfrf*w$j(Y^)OgjdTU5mY_SQzBpdb4Je27KKmHM zfwm|%^%=(P9x7sK4(|ghIK%0r*?|XrKw)y~z8^xN_M=^r1?+S~%)@;1uL*8+$Vj|` z=*T^uub8AJ;3+Ke@>plYg;*Wv-_X5gHyq?0!LcA#%wv(OLJ2FeR41ZoLQ2 zkGY&(E1iG9_(q)-PoAM?u#}{!&5OGRX<>c2!d3F}XIBq<3q|7bfcOS6a+Vn1VSP?J zR=#1aLSjIoK-2f#3mTgzN>ex?0vURhyQ(G-QCPy52~Weqvr5`6pltfEPT7a$yXYu; z34q9ZC=k=8r!Z0Z_8;esPSn?svr}}7w{y!bZyB6{Q4$3MHma*wiiCESnW6RbChr<| zBcYuX9%1@ye(6T$9;amjw5P+2aSjKmbwJ1fL9Oo2c*{VBckULGXLEU+>l@GcDXE;` z5Ewg1W0d*0f)y{ES8EV7X&Q2s{AdN}cHDCO;KI?+gkO}juDX(3rZ51}KQ(-+w^+bX zR*!@+I4#|Yg}2`PajWv6pMxT*EDGbJ-jR8pc#dEvc_*uU%ZWxH(6iYk0JHwUTbZMr zxf_k2aom37vx7l)x#O79$uc7yzWw;_>B$|H)`F3DVyabV9N(06PNAse!Vp)tf^5l* zr4Oyr=E<%&VPfn3D*Y2UWd$;KMLw6P1nNl@lvBP77=J21W9&1xYGWDM`R)S0odj1P z{$o8nqNeFnfSS(b$*fD8?|^{ma7vn{PYint!bzhL2$902l$WAik$0E{_=(zJsUGox z=BljJXN}k8wBL^^tniMwCWf|u%?2EV08QUUVq7O;OQ4KwCB-lFiTH-6ZrValj09fw zm_+``cBXWGO$*D1`)A2&Ts@KwG2PZeRKsPX&gkA|r`TQ2{)mR;0iv!PC_Zdbj7x=G zD)M286l*odtR>O{+?Brp3Fjl|h6WEtqz3({#nISmDO{S80zZ|br4J27l+5EDS8rwn zvjflK5-B6tsHlg6^|4MaqRo z9Rbn3I;%0kPo1Rjx$+Auj#fhA;)1zts^^>YRz8Oy1o6k}la zXdM$af3RCHcOO>;ZWr;>&P@;Os1=)am^*r)a(L4FqGmZ?4qHh4$a8L5f2Db|AS;ev z8qu-V+a|pE?27Y@$8`p%O{OfIzEDpvMhXrgK=%)UpOyA=er)K*#ax>v-*l*FveTng z{qYHy@&Sf}Kq?~sJtZ#?^GClJk(sIcNNtlZ=(XJI#~*yOb>K0G!FVYmka^F1l-aJt zID?n=gX{7`_jbv3D_*-N2X4b@PcAPxi10#UpVP3oF@U9b6Y#nBmC7;ZrD*~e9PtEv zi|}Y9%;=-QXmlioNh!Z}C+7OY+OhLh35RAv@-J8x2b(oaZvr13$OmATX)@<2HwYt9 zFEYfUV~}OkRALQJ+T^0QVUI>C!|$`@H{3cnDrA9&GL0Tc0-YqC2S2r~YqpD(EMNt1 zsNRYn4rkHdjfEN+her_^E*nY0{RPZo@+K;{GWu!2)*&e<3-+;Ln9Au)|Cag$}!N*HOetfk`H(k>X9%^yDE;(kWWjF+vTp`e7d!yUo#S#6*+Qxc~D zXs=iZpq&Q}COUm~G*rmKAt4f&d0;=zy3EO)i-{&iIjU2WGFx4iY>O$BduKS}*Lc`r z3*hTTmSH~;Nwor@F@S#mmq_rG8cAmYe9E8Q$|S^uAtd);3yg7*=rx2g(#MR!8!yy* zkM{a1*cr%t_Ht|rsn#t8`3D6wd9l2}WP9lDt=c+mB|qx7A($6OUGb7mq9~%lcyUB} zV7d?CaV6|1REUHl;RX}qc8E+s6^V|pGYhZ^6T;Q;6JL8gl++YjzoARv8upJ`>fo7M zs4K+w3?t`b%+97m8qFadmF@%H7y=%rr~ix`V%k4+tfSyU1CAfSaCWHtM2!kRyHaH}f=HD&#A58pydom!q(rHabX=5^ zA?QFhRg;8zcx|^ot@xZ4%iPc)+y1&{r*#riO=4U!Cq^tWz*$Y)vY9mvH~#gLvu4vV$L1ww zyfKfSDqw2w(gNiM^hZ|OPjBbepaCGVpVh00YW~$+af!6q9z!BMg4lH{6DeZR$qd3v z*lvzYGNsW@B-v8E5iepuTJA5ay(^0Uv zU(UgH-sj}-01`as=KaTqc*D+XZTyxE)Ly=<9XpjabvS+dKQXbW4?FYex!>_p#(T2KHg(6B=?)+w=yVAMec0&3cNp zJV4AKhN;rGA6vnffjNz@_@|NwUl_c*aD~Wr9@wHy4xr@U-y|4vWfg^+Ir{W`tc{Zh zy`cNN6wE1$AJya9a$;{CSkp31K1h#EgKWRr@(NM!I|jnTS68-0`pjJ%cdg&-NZ5?NhZ{=^G3Z3gKNHm{WYfFC^va{v>-<5(sA)m@lH z=fE-l`vuDYi7LUyGlunQ|G(MZA$lnM7%@!dkK2M6u>1D{Ir89#z^}dt0S_Ep1!@Ch zTYt=C7Nv$R{WTB@u15sAP7(e8jCX=T1}b1g0*jy$L8u@==&ShkcN)JZDr-29Q#gW_qH4QK+)xh(`C+y9{K9Or$cX_{z8YW27A>-UZVJ4 zC;9!68d*dR`Vi4-!qr`6|L@br6R5Kz4)}!^IPQbXpIV-O95*bN3EK3}_^TG80%t9O z^$!tW(2!F#{x}X47a?*Pa8^CcPy_?w)AciL(M7z9aqfAi@dhJ^xC zPl&o}#LU+ZuU`K@Uq1(5W3H3|v%9N1F?HraJ#qIff$_XpH>a+p;wS%oe0+LZU}`)J z4M7!>08?>d`^L!IWsnhzaytI(k^H|XBWHoW`u+PIML;6nK@UXOURQKSd}L$M)_nNS z8Kgmo{P#CtU?Tuiw>Ek!`hQE@lMs0V=(hfGVgL=#3Ur8c5UNP8o)c=Zk{w!yyMBpW z>i;LLa)5;IJ^ezDLWhEIaELh1410f}BOI3@O!GvK{dO=Ozz*C89-!Qm_H-2(eFi4x z=#z}4G}Q=3gZ|nQDutr_D7kv`qb2!Q#lGKvgr_6LV%D6H)gC!rLbYms5UTg_BO)ke zaqfFMx3tm@fO-;X@>Z8h{~GK@J|J#i{<{`{(rO0G(b5_?w`lPpfbkQ6{0h;_ruU%B zgogJ*nrr_#2dojKbx2JQ-FLzZn{)!Ue;@;F^z{+@y?+LxX=y-O*#E)T-*K@v|Lxzx z{4rTu|1EQHJwkR|frx=?m?p@R=089FtsVGb$1DIUFC^c3^W^vcBbq-oIw=I@;e`6L zf=phv2nOL*igi* zKFk}0hC`;VQGiLr{OWdg>aT+%Akd7M{6^0$at7uZ|8-01 zNUVfN9S5;31P!o!nNm-0UCTeb0dHfl+Wtk=7sRE62Zu#k>bHLgdxW3PYc5S)k}pb( z325zq*s(yE1nqlI*K`Xra4<#@^MAg(Wumw$Kn=BgImQ1&F0eIN0^u8n*Wr+|SGSir zWTAZs@xxXqQ*-`>UC%7=zym<|){s;_kU+c5VN9e{rr54uT{NO;I_(STl&~Suq zNZ}c3Bj6mSf#Uv;&-e>+U=M_=G|KH)0cFa78~+!G&%tQG6I$ysnt%JBjk{q0b7$GbrAsc zj}6`Z_YE9BM2KdRYsE8;ii}?{Q~x-tc*c)ik^qtv`u|+U8J7Y8Bx>g6KaPxwM%J^s zU{Bq6Jy&_*mq01QK3_`n0sh4b8{V?<0pw=_py9h2Z zwl}(EYffz)zc@|}s6Cf(dafsT>tYgUq28?w)E?Iy#pEx*p#Th!Sq~Ih790c!QdE;2 zO20lQU@%Cny}h&oT+5>Ur2v0a!$TAxo)JeQ*;=rHk?|$(!mNVrCH-$ecxE9vIjeE9 zy5{NB9nPWU;~=O6+!oMP!YJ2Fp@Vcb9N7}Cd}^{hIXu{#$Z1Z!^--BB?Jm`ssgfvV zHmmOgf#1LrLb1ksK#&lEYjs)p>F0!|jHn9%e#zBaC;&kR;K2RK@3X~**bq|j7BqF^ zc{N3I1d!`_FMW{-N+(mS8p>Vu1_2u!$`G84H ziSFNqJzcil#YJDaUHQdGkRilvqRr)G?+xYcsZS6(6YzsS$hM9Dhci0mmTS;pu`=p_ z+oJYEf(-$b;Mo=I$WH?dr zw!GXhg!T)7p=HScRXT|bVv+eW-3?5~Vk$ykYokTC)eW4oJc|2(W(kZN1i9dK_LFu> zr!YyJ8y;xW758H+zqKA<4~&SOiue`EQ3EGNn)iK#$f?>O(L?|@IVWGUbSy)T-jCcB zel;}QWEZLP7;zvN2ql=}o_#NQqY4CtUP_aJF%_7M0%ophlL#ZdNte+sVv(Hn#E;b_ z42;U6p-X@*unGg#)Av<$GI-pET9AHiHQ+;vjUpv#@SK6+oN;5vQ z6OnOef%)N*ZFjF_d)lID+DN}4Zmj@`3n)zeV!M8X3Ryap7-97O4l*-j+f5_$QG1kM zJe?tVvAwP@f*uJFt;`5*69-^iD!rpGZjDvu8OVT@ClJYwetU znVM$335Q9o7Fqr>V;-PO?{q=xu_u_)v6#Wv@hhbQw;WFZi5kGh6Yv=EtV;?}C;b@x ze3weX0p}y#PD;XbJyD+TttowCKTZq?n--;vsoeslpw{QOTh4wI}`$C_f_=<&h zBIykf`Hnl{R$8_=Y`*9&5q(^Mi>YQGuF?-W^H-wZA`;4h?euBEv(at?Ar2bxB=!t?>itGyuVhn#JM@*NSIQaOv) z5CgUsYK(E}q}$?T3g9e1py|<^Ab0c&pyUH%0O1Rmf6F?HMRg`SHaJJFJxodlJi`|b zME-Yf70_UEEgL(lyB$#)nG${LW^2TdLr|psI8+|6N4^w%MRWZC!C0V0FVjx5uH(>q zrEzX)QhO*t*9ZUz$m)O_R^lTjC3sZKJuN}+RrNSdx!Gm{o`*U!pl~rP)Q4QZdovn; z)c_aNjY;dgH4f{Tr%sQUTzebyU6;(H5sM|gM9{>Kpy|?c7V7NG@`I%C7Y+-5$ontx z9Jdh%Kvo;>+np{g4j%~0n;qhbA~vI2pOYKPqkKS$xyAMpe&!EB=qi{5Q;S5vy7Do* zM+L!876W3&aZCoQ0~WL3brpl(Rs#ze&a3P*FdO&ilOfvs-{Bi8%R-{Z{Q--9%e3=T zOLNJ*(P`jT7!MI_M-Hlhu#$vNtSx&$n9EfdUhQ;}k~vw?w@F2k9Ln_yy%xG;LY-y= zYu~L1{ed2@cAOz$8?H~+$LJcJeV5Oy@d(wjib3eNzrlT02Zj{JvL99p9VO2J?rH-( zxi9UdY|se<$PTl5%2g<-Yk0j?zO0pzJO0?qqmvs8&%?IhAMu4z0)k5)T$CFFzG{>bC5Z9Mv zp+1Niu2q4ez8XtOjCKKa85UqC)FY90XnNvx20)+l z8X7hc_fHT;jRr8RYJI>&OWTgF7#%WWSzZDTKKZHV^{_#Spssw0_0M4@a}<*s;H*& zdkY5(?U)1v;;WRQ1{dp-WiK&fvrK?FNgovUJdBliB&SCOlm5jy6{)WfMVA#s^}x?B z*-0-^kz8rZb@pf2ZPKs8psq$S6)SyGFrj-N5ov%635p7HD&$!w>?pN?;4CZ<2+Y~? zhHJU(wTm^&h$l%Rb?IcGx|kbl`Yh@W6uQH*Oxwy@0NuCPf%_?Qd6hgg;qqg<#mwIx z=ROxh4to!eL@?QRNh0Tax1|qzMyPlBV|9cjLF9Mp3IUSE$v|P25K&(DBnF zZ(UrTKUABhDx&12rH|k@tt{<0(6g&?8e*XbiedojqeDrEs}4VCjb0L{$HTLLlkQvv zJ}?y(HH$MOQ2Ag)UWFKpU=K3@J$ZaL~JmXfpcI z#{3876gE-wuMKX?{?)9XZnyYFh75nC499(V_Jn}}@O)vD`0C|lwC%z|T;CPV?d(=~ z5NGg16h5hQEwZMD$9%^{T~xMrW}7btaHCpmS*MZD*?f$d(U}s_`9Z)$Ce)jHIZe;0MM(xo8N*Z=09p&h z_tU0yg@8?fmn-IJ3T~NelcyCz4Z;HPVY;b@`;^*1)d46kL@X=8h}Y=<4%Z=Az!r#0 zAIsUIwrAuprnWdCibV717a8aupzq8_F@nF z(oBH%=iqe$aKqhCWmQ(w~CJ_EmBH)i7`N_D804nBLY zU$!>qGid*ir`c?Bw?M;qG~IW&Vsx)k;6v+mdd+;<3cz@vGB|%^lgXz-*(=0Q;lGYN z*kH@xH-3oYe#)bxsb{c)a&{)Hs%EJJk`PCI;rxke4 zMbv0P{E`1a+x+^B;VMq0@?q((?6J{zSJzvsDW6@=2MTcj?DS(;slsGYUkXSIXvO42 zrGEVUYC?;e9%c?AFXrCE1CEmH6L+`u<>jja5s)vpg5kP}pK1hCfryU-=j(SkgRL69 zY3YYU)gu#MQ+44R%;}iGg@GM2@_?!twQiTuD_o{M{2hv@W89Rh%o4O5aOijbHr==z zL)CACpFRQfimlp@dvskJyT`Hf?xhq)3z?&td{aO2<~w7kbzv2=bk`*`d$s zZ5j55PA_6)cu??b=aBZjdopANZ$pc!=wY%b&63SZF$(ki^CF#@z8uyQGa%8bV<#?% zqKS8T^N;j-e??Rhy>{}43_|cP{sRIAHls%*4s-QEm_P51-e)0t@NLHs{v#D9x5g)? zW3Jy|L)?YQcy*2)<1`gAK7Vee?Y)o5qUolJP=W$rK|VEd#A+qr)|AUH!5i72S%U z&ruWxE*bs+Jh&Ml{@JBmkv3_GRK_2ekm7+f&QG7Y7^<7kjAsqH2WrYe?=^qe;6It3 z^7qO!<~m-P5pq`vmMLqV)F@xM{_M4lQ|+G{Hak>YRvMkXBq$yFooxBL&96IlC+k!g z88xRZYkAca(do7w!%mWh<}QNaI+u>g!J$4WJ~j7F)PcIJJc<(&#NwIUdOx-?o@0a} zgM31b+j7rLgU3*%P@Z2;PdvFb4liF!Qm@O`t;Xo3NwxNM<5Dp{ld^_`07$Bxjh};c z>St_ms?OvGPi;e%M0YL@A@BTcOk=Gvv7`Q(H~o=;fTPZfuKKw!>M_ zkrn0E(&YWen6oP)lN!`;5u81af;0iY__zKKUn4n51|eOswpBj%Sg%h|2%fr}G)}qP zHI|&pjo3E0p!XF2U^J;;)wE;F@ng_tQ6hzLZzbnRd3>woyJf0XV{J(!-1`JY%1|zl ziUNz0d~A!xO9FOti+l}6J^#Z&Axq@3=T7yt$YaVx6am+M)xiIR@G8Dj7HWyfi*NlA zjIJF0EiQpHeJe0uy~#YO)9j9OZNbuvd9rP9s%;ES!DOI6J@sm_6{`4hONHv{)Q1nNCXK&LF zZt8O(3pi+E{`N5c5s=9EF(+MKr})q}9<-YIn)*ibelCla8^s$>dF&8v{AD|y-k0v7M2xE7BEe=2dQ#%Q2&8Qt_|aq()EXmJ;EqR#G@=?@~iH1idV z=`g0&lmI+jJmEsl8?F^9OB4;KrH+f(TMrPU{Pslq6ELzg{W=n@yq0i5sDF>B%jFeE zoBpN4pv*pCubBxJ<`$_se@g6|j2v4scPrM9p#P*U#d~7=E@_AvR{D(DM48#hE)hwJ zQ2M>B1EvpOIY?QKxJuv&xUV0BF~@myW{EpiD;zzCXU_-ssTenwlL=P}_B8)b1)SZ4 z@b_4;U><@e%+$)lymsBtvaX^QB?iWjJfyaJ=g8kB2UTYGj!_%NmimFL4(NzNF8Ge( z!Z%Uzg(n`=ZJTVqG5`S=zjQ$=&Cflnn$ziwvg?V7xe4@9m|4F(2M++F$qyQ=lOu~y zqMEaEqF0+ME`@D>fDNFaEtNkiJ9jS|F9@fND}v9ns)BPf2?gcaf0=^l zQN-CHZ{DgUVOIr%YujkO17##!o*d#XXSWGER7I9pl@%2C!ir7cB(EHQ) z?CJvi{=4>(qm&je(5q?;fh;5xl=)@#*5-@25NY#+Z5)(KH1-K%DJCMF^3)|ZWk94? z7`(ws4mWIEdy<_tKOWG-Vn7Dmkfl4za@ZZ#O-30>F>#_FWLU2@+7$h)zunVlJhN;>o;{X|5|7~ZV717?lCJdg@qHz{sg77B+xpsCg7 zl7;ePYT4h=B^N-X(&K<=4V;LPDUDC?Al2!$O!(&6y3EMx6bHytlb|RbT$?aC7#Z_= zd#E%=;jw{nhFLpjg_mxNEMN$JN)%$^LeSYUDMjP8S;p%%F4V3@%$9`0jS;u|Ww<$;t&I{)-9>ywN%-z7gooKrokfxB;&0 zX7e*cylb_i#q=@~)EgC%;4K2W&=E#5nEORRz6Bxe`+Ovm*UX;Zi$p7e_xX}{(7ySo zE41yB5s`U_)=EbLLJC z0Tb#iHj20d$?4eN>J`1k`iMM|X z*s4<{sc);+J*9sxO@1%`ph33W87ruZey;cvw`-D11%e7cAlJPK#DsEmqWE2OK5^Jd z;$=f7A;@8)=tyR%Oi|iMHMEBX5`~x6o{5gNyF#<0_bDMNrcc2W%rHmCW68lzPI4wR z9^*yFqgLG-z0Xl5Hv3%#1AsAR6n8tCGU4fhsupxu@%TPM_-1t(*s_ord2YRC@8BKB z&9k?UFImxe2)Zm_a;fZ2Sq;s4gQ3_}(1-kbQSq+}D8VaVg)cVs#O(}UZY<8vz&Jqc zCqk*`9aXapjU5WtjkH1gwFkc9B`4!zl^Wu$9<5ItRuZF$_9ZbS_w&&AQokm7R%TEq z7klOd8{iH8Y;K#@*4?Ax7gQv~9IK4(7A_`}Y}8YJE39mq3^~TEC8z#lK}gnpne$z0nA0bf$Z|b^C&DwJfFxi;V~UYyl5?zs*%cX zolUg`kbAKo2=F)Gqh-RWp2Ys?^Y8sCM)}YF194!aQKt0b1jr?~LdJ+*{Rnl#G8tU9 z+STClVYn(FYTWh>3Ohd)CXe~d5$$9ELDY^w?q!6Qqb%|9~ z4c);=6e2WB%3y{_N)r|f-FnRYv}Tbr!8#`_&RKHZv8{q|xYnT%2Qg~yGyw z9P9@?H;C`L-doH-)c<0){k(i+<}pf46sdxyNXY%;Ze$ZGS*<}ACuHXnDm)!3a5mG6 z8o$g($&Kcj#FD-LdcEt(q`)%EntH7C#@PR>OU8e&0MxkUt3h`~?}!9w=7S z_>2W+qTJ^q@${ktWaqkhw(p%5#f!Q=MvV*>=PkONfoO)Emh?0{MQ0J_3xjID(QgF8fjoA!Dx@&$W-9tso&*9F6S=vBHd zQ9&XYf8q=}^1Gr(w`F%9{yaPhX(sBfn*#!HuXQp;`2i@UTsU{a@5^jgS^QS~+!RWcgr;4f`}SHj~Bb0uD@G`u7e|BmRm!rbANr z626R{b|qnyks-8b5UJ)x5d|_|^GbsPeif`7VXoA{PZ%k_k{S1~{W}88Au^aNA7UM3 zGMIgHVe$0G;_H$6QUy5%A;6XjV6@*klnPs2333jsjKJDXk`_Zq9DRmszH@^kPTzi_ z#5co5Glxjb2-OM86@N5J7Tb$!tn$spAljUt`HNcc|8ormqF>7fF_J;fBUzp>BV5hh z6)?;m5coW1dT(_mnBrHk_zXRX){OLpp4crn+mx}E4KhPeRzy` zNw)rmJ(GQ}?!D=hGWzo&KjYs)vC|K|dJ}X1J-dfRj|QlyQRh%3jusSX&_)uX6$FBZ zEZ?jvX4i_sbT+Nf2;RD!9__(qsuhOD%&AfM-d%hdELi(nI_7Vv6#|hJBMgUou(5nc zEpOB%Mu;l5p?~cWN8h5!_@AN+?GoY8gnqir>70g9*8-Pv@V^+7hD3(3Z|=jAv8uo_ z`~RkAbP3<3Ad9+^E_x7xqNR|`vwIZF6bWgfo)=?k;YbS;u$BX*@Q395WquZg1z^oP zzPhW(|N7!@A!`2rG1hoXy^WgPp4aHHs2JJ>Yt8sXt>tv#0=bDAI_;1qx;Vm1+U)3z z$aY>0-CkTQ^r3)jWPy|l8apBPUxtDn9sE~POH}DlYtCOMamf9qTjQcN7HcKKuFyNh zd9{DN5QIzNyg!7{PoihbQJ+P{=H`*{xa~ zAmEiME{pSVWFm7S!T#?kzP?d;N0ZU5|A53*g8B$Jb2U}5M+m*LzVfO*8h|6qJ53tj zAwK#57-W`1@`FHFO9r04O)YoSZGm0h%O(oj{-0|V;+rA3fRL&Q))a6Np(CK zZ`Ul(g07w)NLI6{Gi2BUOmj?pDOzy1%$*AQi{#|A3E7|Ki9E^D1g2mppzRueh}P0F zEg!a^*{V_trvfmc#<{``GW~W6+IUqG;L>G2p;$6eJ@0WJE9siHyk+m|M{T+nf@gLW zHfpsVMTZzXn~x__yrGR01U7mnD!W_Xs~#ZFuJ#^{rrL{Mu7nzQZ~vXOPJHnjF8qf? z)8!V4IEdF_>6NFX1!ap7qy80TZK{P9RgHWU0%OASsRNtM!+m8_oPdu)lE(NzMjW_^M2g5=TTTHaNpyByY^8C`%x$f4L4q2 zZU-K^E#>F+j3BLgn0v3qDRrS&#g9bR@Uq#i8Jz`BK3$~`e0h)=tvi54?t0J;A5ztv zp1In!eE6AF1@}#tZo1cuj>k6pc8Olii05nOhq(t_Q*M@;&tG+Fd$~%CGo@dpdsMI? z#(?pUu@_AQla6b9V4AJ({KZ*Qr+BV%X#?*QMK7@C1ZbMSR8zijd{;2|Zf0FKfE47tqf4FfuK}j+b)}`%L6OQz>l&I8GCvi6#aOsTsvYtK-VLxsmij&G zdqIyXnk zF;ArXOK(=&Vlp^P5n}w=1OrDpHjlmn1%>B-U@Ha@!;748QbI-@)85-IhcNZ z-pQbY1FA7=nSRZo&+SeA&TW}RRk@@$5UIPzH|>Jz?Rz;8QRj|mI;P=NEl(~$n~GMj zBD}DylHHE{aC=sC?5_~{F%=bE&rWL2;j=qRom4hzQodE$oDE}AJRfR^2)K*kV;`Mj zULRd<`$fE z*qa_BBTRXc09P~TV6VuHiUKj0*a8-}=(JNcTDqL z;riYo^k|RpWh_U(P<^SMQy87;trofL3X2oGV$$^(u6P{)H@B&sv2dE->&F%4hI0oI zjMq7xM}bE%P1HED5*kzy17cNRPz09IrNr3(&NUU9B0l#zrnG}a^u%=Zm8X)>?;mC& z*T>4M`)yf4h8Me6dld1Ou?+-1mGzHl-mM&`dpZ}o%+6g@&ly;q$R6&OBE1d?7YqA5 z)6ah7iUKwxku=y``OvIzz+iuF6LL27{V=0+`;(Y8Uv;ioGWTM76gxx3_R?uG_ASY; zl@Q)l$I;kx6$kQH&Q=!R%`LCD7)|$U6WN+Jvr3dt^K%LOQt&s|9I8i%9(`%kFbeQ( z*Htmbx6dl9g@T>zAJ*ASgxIMZ{TpSRH1!_{TwI}(?{;|A>;HlczL-SU=LBCYxjVhn)Ji(Yp|G5T{2L)pyp`<)r&nklwD<&CQEr(lV zaMfQmM!Va~Kgqcpv)>-#1c-B0F!%a#rq!g4DhgMz*YCHKs7YSk(k?IDY@{d?1PRFr zS=fS!1RF1rA#KXYn9={LPIvC=~$J8Ie&12CrE1bWnOaU>K6Kv%q%WidFmOfE`m~`H znDR^c=fRzB?TvJ1M(c6@1)BoVBS z;%8gtP5D}}ws!2m?SJ4UyWog?KI`yp@mg>d2(zH}C)?N1%>>^1nlk@#7 z3Y@cLjqdYVA6^apfV!R!%r*hRVuFNU&ZkXTiT1i>?)bd$lWEi=X=?LWT$)%OygA_% z{4*jCDF<;;E#Gw8pLr`TvKNj&5%wdpr`KdQ^wVdTvFr^eL} zjIJn?;H=45(_-*n%lk(1fuQuGV9MKH;{xfAo2ueiHR`=p~Jc~keNgeiq-Y@ zuRapVRDr*`J_muWm?f&rXw-N^AKKkU1>8W_I9!|{@%8W!UgauTN!i?}dv8V_)bCnc zr8uc}Ux;H#;iXwYcgo@GQPLgmZP}X*H_$FQ)^V--=wdiFm0|3}*q#J$Qq#}>1W}zf zSY^;R(&Mon9981P{xi+~a_<*|m%=YOA++!&eq9aPFU4>CYP!CP?PAxE%71cR zlo@Ir=UJuGf6|gmM&;IZ!ch(~uiNi6g#vgL=}xVWc!33U7*~iEgjD3Qci^&x`>r`t zIGGxIAtm<4wk7NBX*JJ$XdQ*~azOc#)jmzolpymgVJC3Yll3|l4IL163D#xkbix(V zYm7tu)`S>~9&y_gKqO6~{CQ^^ChbGnR$l*OsEL1fA6tc#gH#G#5M*|l>zaRKk4(c%1kaQqV`4O-$v8k67 z_BR*pFD6&ggJA=&dcP}1&YhPak4hUWiX&e4oP^*G3L9+~7u%#?>JSbr*)v%dv5j+0 zvoiaW)86sT8uN|9`5(^i*TDQJZ_q=({uy`R3OT; z6I|X{uT!eKw75F*Jge1lny#G`6}g?O2_zVS+Xz{FN04D(r+- zM|B_v;}MwLxlXp>2o3MZQwXF|^#ka&# zGc4VbUst=)a_(^ri~qp_9_=hHinG?55oMaMNl8&J*I6R3Po565z@a607yGf9i;}(^ zJ0XSX~SBCCx@HtaNASr{r5qS#V3`1nsQFbi$)6s_m@&7J*Cu__Yi>+L#r zDnbvQ?TMsi{ACiov3se}Wj|;b5&H#K-A=MF_wM^>vG6|pq3Mr=REf}YF0_gK%M;d7 zwqG};(5lta8Uur@k|wXKAR+?9J2l&_0RF2K*W3W^x5VM-0!RI?MST*3c{T#;y+a@K=w%yiN75e>dR4HptN z8u+XC`Wk`{1%Aa9{AR%yo^r*ybvekn-6=!)!?zAco0$@;@LrpHOr<0DFOw`REoQZ_ zH3i;M+khS8vWiH(3CT-&%hUw9bnLhjtcjvH56}$}0FJZXPAu)*#b&ngnYKro>q_ig zShk8M-w?6I!>|4hvu>v5zuvDGF&#>Tn(moJ2ME7Zz#CX_P$GkNJag{$!JH9hN;SGM z+TI%uaUfW1poM9ew_zpQRQ6l7I1E1ZOE^w_VXb8sn}prJo0qEkQ2KMFnxp7|F0-3d z+_0LCxjAO^D7NH>f}5t})7jpYb8&v|x;dFO;h#|GB4SRLlhFUFtU~zp3gYIM2 zCvsAO({AT1mJ_Knl}8qLVr@!6Pz*pukw_d`pMd1~=%aqq;P(c| z^*9H=o*h3mtzn`x!*GJzjck<@0^C8LJEl#|#3R<>s<~>~_Oh%`7$3-(q?_kfaF&)^ z>=_da4x5Cq52q87hxtX`_jcdhspXqD;=4X6{ZMdtd$Z+C(3*~ZF>|-g5GSzdDduhZ zYh6wQgH@$Ja4iT=A-uuQ1MgedA$;Xv?v&(}|58SxM@6^q&6ig`lF4@k@84bg)pYSl z^;yInO>ZGi0WO6*VWhX(iZD&+NvJL=R9ChPI$f^_WizPy0!kyOg z;1>Kdf>W9b((SGWyp;lg)w}LHF2$FR<6yjz5!`qK26ud2cC6$jvI*-gqie{+!@NVd zt@)k3HcA_>I3`}T3MiJC{(STha+JK4C+ZY4y~^2OgzM;OM2oH0YhY07DEZtm*~Rv3 z|75ayomiWjN1Y*z5jH;O|2;bnKBY*9f|Z>?=z8+dknOKX`LKpj)rh2O{6F4Nsp2S9 z3MD6aP&(#T5!QF>I~Ov%#wgqDGeX{%RD)zS)$b!$?9hU$_Kh-1vRkUJjFkU-dFsooE_`o zV&E@UNGEZ5!IDhf_Fdn{k zedN;qu8I1m;Fq1LXe_I0%JStv2Cv3DeGsbouR<~9X-l~MEdSbcLEdIyOCHR65p7>O znR{7ARgsqwsrM#!0p=5R`ajSYQ_`hHdqG)x3zZ&KBLeRAeM<4yZL+}woYcvvQ>RQ+ z?1f|gEKKO+Q~lAbSLE|)>F48hmH?@e;){)!V-POFQ+ne93tzb)NT#w@7GG6Bzy zBc@4vEk*c@8;X{sz#DUnbw^-7rF+9Y-6Q%-rzSd5!fld-zl?+<6l*Q^j^><4aYDbU zdi&f_pQ?t3o&0FDY-11q=pa@{TYtP--SbjgdFKF}EpR)g6{K~Y)is_-L_*IciGKtVHYX}+c}mppUD|cTY>P0~ zagrvNP&vC~wa^A}t{IT=`U5Stj!S)GY}GZf-xhATZ6wAUdBU zR`Q(E{F7{4^Yxui&~Z?D5%WHk4lP<{`XM>-g7^XjNIv!vNmU-77=NPQ%YSPXElKne zI?@KmW4z-fJAp_#E<qEDEr-1Z`IY6RLi_JF0KBv7 z_+z3}8R1ZEay~t{@&XB}Y=rwf7e6g!3irDZ)l5IzGX4s5_HlK3S(S&fYeQKeRDURT zLsadT8~$vYlTmn-LEQUZ{;AeAW8banW@IoQAx{Zti{qD0_(*Vqz!ildA_Pm?a4|zM zwFr;qKwLi&U09EiaA)R~|J77tPL6qW9h#7?X6&gQ2%YIeKS>A@6TobmCNu&wuzo-F zb@Z2o*#+s?zLC}0=ilIjmMv+(Sz!4rk8T*(Rl zw-+bo>=khmQ4wIrl16Emup_RH8?sSlbNvco9fq`KCdfJ z{Z?p`pm5<;RfGS^PRy0vF4(Uuz%r?JFPKTrH=#M~naHo5qSy%0t0H=xm3<6FD$i4O z^0qhtWVv!@9= zuHEt}O{u17S!9`N!}#iP*d7O*hZDDPc<*x?B$wtRL^(!F-o~`g^TvM;*7D!P z3esxo1sx2|HIlc3I^wtA?Eq;y-a`iKk~QgBX)=F7Bd_ahs`dpNDc_Q9DvuX%CTO6v zud}XqtUtq~a+V0|SP=Dl(m93*4#|v@mmc6^pkencRi}H7LBmqcr-9@v2d#kRx=}Ot*N+_no&4E$ObT>$^HV_Tf ziu^|kt9bjsYdUkDUe6tSvTY9G$wCE1XH)PlGpt#R*+{nug49F06Q%tb0*|sv1Xr+T zZ7$bvo^d=w5CB$TI9Q>3B^~`8ohxl;MU3 zlaK+bscC=v-s5@-t-`Nb8~xVpk#5y%KDR*P-D=&J`JCSioL*`6i4oyYzf4}laj;wc_=LQf~7DT_ynA=@R zxNml}hf`^Yc=(hgQ74_72MUd}>^E@}eBx|a@(s)ga$gD359KH-yXp|lx&Aw;!_~Mi zODbBj_s8n_WQ4o*<<(-njUk{WV|dT{&lWj*4GO%FJ=be5 zgS`i;Q`*GgI{>`j&M^h6UX~5k^%|us9V$n!W8rwCO#D@f!^UIZ=aKNTe?^XFNnHWW zdlsC{!zF5!`+s-Q3t78}5*-HGNOkd_m$y&|*4@}z%f$0EmR$;8fM-NhSZv~w?ifAq z(!=b0a2K-;jon{SVPKgh4l7?&onfY{i9)}ic$O{-sgc=reX~V_7fqZ z`M=h}1?Audk;_w_r!V853N5v42`&d;AbuT@#!R&#bQiBWMXf?*rtWi!vh^!A3=e=E zd(q4N%9DGT1;k_6_%|1J*Y_NewtyC~jUHZqeOOVN`XLXM;L8`hc`pdw2b@$o*jx)v zd`|ra$V}(>uT=~l@7$gokCOQznLtyW*(~RaIT<{UOY8b)Es=S&Z?5fH17lO0q;#rC zXNVg!;)re&G9~&c^f+H^7cqPog(Y3z`;8miv&Eq^bYB}EdwmYTm`Epy?A}Fq1;9L} zKOsi~1hHC!B(3It6-=9F&!6%IL;nD$Z{2Z)Ib9E=lL5QVwz*|!un~&7;Eu+2rv~u^ zW0$Y}33xis#6ECq7_nzX{!qmFNrGY+)v=pxtv#(x%~Emor@XmGR&m(VsMc?d&h0p1 z?F(uEBTgOSY@!3jxT55p%ugN!@(x!F8$DXa-4M4sRVJSb6UCGg&rMenc6xz{{!$uI zCLx<3+U>?-TzdI!;hB_zf>v(pg!bgn;UVsK?c3&i{kjLHng+5oenHNs-;S@sv{;9G(e!p#q$*S^lE4XqeB=@=LKr9A7CDtAMT#9E^8&Hv* z7w)eJ+=QKXYsa+jZz6MiYe;>Is{NO zohRa6_m_|uKfk%!>mA9==kDc%s{hd~l+Ju#woa^`Lw}hXA)>L;Q9h;RV8@(<%G85}0WUGFrUvIv?Qz1-NL9T7eqk=N?!gF??ULGHUmSnK}q8G&GYg3H2r z(rFN9{}SBQ`af8}k9K|52jYynQ%6B&kzY>J+-IH*_XugaQVMURe8#Z~0yehjPAJnR zLY_DeMXpBf^*9CrzMhlTft+B8VB%S8!DLZ_35H1&=oWW5nr?OSb$$Y*d6-gz5q^`2 zX6(-t4-=Y$U^!jN0A%zLo7r;+RZ&Bz9$Y&l;XQ9UpU`A*y>Ywz3l>IT>X)o4xlr3q znWeD(kgN0iZ?5o%1-GxRW@0^l{*Ley45QQcnDT}UdR`sB6jAloO`5h`4TBVT=|gf0 zcN-?1vB)(VXP16EZ*@hLcWk$srW(Wiq$LpqvwK5Ja&TS;WY}b>@LzUHg^SVjxTlYn z31*Kcrrh@ouB6HMdjrX4uA{`DiK)&NPG_@&Ll6JV{TeN;=O7y3q`06XVeNen=;y2G zjuPnra$zFKL`>2afa;|%ezaXKGDr{`b?jKu=}p1WG2{%R%l@1K8W&?mQJzSqK0Qp5Tzt#`n% z<5Ngh8871ywl3uZY$CjO=QBYkBDt!$C4fo3jI=_MxgYfhrV|%$=CHNas}@NlX_!j3SPU4kFm6^n)Y9|em~HEoPa|4; zt6NzudczOOFPJBLHr5n;0NjnXo%HEgHz_PyaeIoevRTG)0ND%<+7In)w_+Z%NSeNa z;{b(5{|!rxiTi`L&Vu+47Hz;GUIPpOs7B%MLlNB(6Nc zS~8b5mUkK}R#$IeJ!j5rx}tjNr;I#y;Nc_L31SaYMe%tU8|0S#{_9KE&== zJid{Kb$xU&k_5t+O?)&BBcAL>#y%sM;OV7*ZO3UiZU;x4DIFDlm!OX|PSWd(Z;bF) z4>eup&-r{!9>#=E4fl(B6!%S9yuOT2mY#>vZ`@E{`I|lXu#{x~ zfVyc)IX#=5)gD&KzTbLmw$#&E{p%O|Cibn)ujfcsZq_%mA3x!b5O75x&kz-Ydu+kiq2{^e zg}B0Fyjort1h;PFW~`8tAkil=FYH{`iYmVak5ObxYi?58F@DKpr*Bsv$BC&h<*e({ zY20}w;r$1_UG;EzNS#qBND?jMN=cY!p@#_b{!K=^evPwUu)9cG*f9}e?t0B;qE zINjk2>zDJ3z}H$1=gS>g?ZLyyUD5R?$qs)Kk!Iz|r^bU}e7rNCX1*my2u~0t8HV$- z9d|J-jwT6((2CmD)4o^ys>c!nSJH(zfjupZ;Y%A*J-!b%_ki)kz)&nlIiz>A$hg6fwb zYkS`-pMmG~=!%fblz4Ts%!;Qmn7>K~KREh}Je2B^hqUAb|DgX(ia%W>I~n{lKjM8YOxGDDG^niAz~KhIYb79z=OC1ZSi37&f!xbV(lXNt0RqCmYKo9$rb!sRkMZkOOehu>ks{R$$}MuF7Pjo*&8&4L7s zQrhowr=2gMiN9LOh>}c}x#~KVhs*BOPn-Nkc$=T~Cf)j*ruS<^sVtCUecOxNkaGv< z&oPxhkveWwA4f&5=tNuqDtt6{Wi`a#R7Cqrx46I-y9-gU{X8#)6WC37&;4=PJBe2G zr9iV0|G0;Q9IC#FdU_;l4nNs68K({d8Sw~lTkvLb))Sqe{`_XRG%#!mEiD^lU}<$) zgIoL4^5tH~lU%jy0h%G^m@rk5Vabi8-XTZ*dJY7Y?=%cyN-k5^zg0NvmzQc{csAWR z#6k{g{0B6XMI6~Qm5YCpJ4pkrkPX*Meh{DnOSJJc~hXb;1{AR@Xv{j1{= zu^)!Z(laz>?#Ll6Ce!Wiz9lo21>Q=coX* z^kX5P-7q2d#$OapohU2U-(@`OwV@oZ{AI`Qym)5V;UE?_+x$HoQQk zS7P}uI$odth~e`bRkYN7W4MO;bDf0dBCb9?)7{4U;TxK_(s!uB1%~5Y2$y@pK?_Xs z^9WmHO2g?ZZlfV0ExAUe z{)f~EttAU71}gA)aW{;}outP4KaN(#Oc{dU=dfsRcs(r8IumS_6Oj-hH6oH1VH^KA zm!-;N%=+enHe@ceeB2(DAz%0J>v;@mYl#zw4Ec$9HW^-;Gs) z0;2yj7bu}IK;J}kQYkd6{#DKYpNRlfW>FvOF<>2L-K-46@Bf*kpYhH98)=1n!7Kmz zzI_#o@@AtQ+C>>4oH6Ab{`Z~2gzrKUt*@lV2rvI?5Q9{)t~ zFEREb_H*hmv1)WhLPiCd-fFt*+biUm^WnJK>;u4lthB1h)b`p)rTAc@@)14hjUM30 z@X?X24@;n1lZq}(fNqsF#j|Hcm8cwjJ5z9*7km#wZV9+%mmPeV-3M((KCv4VuBbMY z(}H_6ifg{TITLk1J((mUb1=%8xOe|ymyE260{+hk)EG|YS)k>EP3uI0`g8p5;QU|h zK@ci7{=-h~m1UZ4uU@o9!AqV;h-$lpo3CQU=qR!ziWIuz4OEz&x?Z&tMnKEXPPcEw z9XtnmDTlH0)v^&3a01i-jsRVOnV~n8!c>a9P3K#zz5qR7L`w%hHRRsK?-~u@T`$q>kX9Lo?xcdp< zr2lKK7#6g@l57+A!)v=07jcP6^(_Fs_Ysx#ku_X>?j@Dug6rvIUOF!T%Hkqx{KoK?s{I^)xW$%q>!*L z71b|#{qV#2{U0o#{Qf*n)ROj?+c&X(zSp4r@WWNcngf7l{5HE@z0_x-yzg$1gwNM4 zKRH1#g&b2x^Z2?4cX9dI0vrk>aX#|LKD}W|H?x|Zksd0qt<;0Qf&Ik2 z(?Kq+ZI7*r^0oMe>MBo$yA^x3P{_=adK6H-V3ADj_LKd|qZ0LJb+_G2h@e|!Y3^~z zcPJVbHF(0jWFjUCvc^q2FWB+ndf)FvX0v1cTKG|=F0YUGWJkyicCu(MKl?iAn-1Vz zjqDU*XtncC<6f_Ya#{tghKF9g@t3O_+f1C)sZ^o+eTCW$80uDzseLr-7@;@pR3YdA zjk<*^muFG+Jm_g2C*u|0T=_U_ko`lyz!xu;fQHx|e--#%CjKw$OOEpNx8lM%E-zKQPe3?aVM@x=Z<8Ie0Zh7EG>n zV2AVXr64I^Aq)ydFsBztnsSOSVGXAhvy`s%!XE=7A>ksgCvtw`Nxl(QSSo&^SP44C z<`&TxhiE#}xpJpAPlA@eNA+ib{!B9S)yjdC-ztH^^Y0JFu(n*$=_bxpcQ~s zIeb`#cD#3c-FctN{_CR|yUs64C$Fy%rne6s3Kh5C&Ir4sD_E7kZT2T3YzlDZtn8Ug zL_aSM99T>6?05zV^SK9UU5~F`91tu$0S$CNiD8Xi*tRsAt-F_U4ZL9F%#Y%FVynO% zrQw(iS%F)NahJP^l%>9w7&n*waDjX%d~^QAU2*gNwR2!eWjIC6dK`SCgKYNOsG=HC z*dnf{k-E2#TUfKB&&FGQ{>tD&OO2tGwNJlMXpTntVl@?0$BLF6@alDFIECSw+>ovu)=Y*FgEq|!ro4$eB)2oqw>xf)_7g(Gmt~3 zHqKf*U%wNaPRAAOORL|cz0!PgRt99*a^k0Upo6>RUS&js>gvSmDZSz8SYheR-{H3` z%v1<|&DI)5uev1dtLA$|78^uq4)5yszALXR4m2i1OhB%mi}5szG zPFO%s@NYizwu=L&Od{mv3S(-vR{m6Jl6(sHEk9J1NOwMir}VmXU2M#5@KE9${1&}c zw~tr^)HBdu7I=_iAl{>SzJ@K#7w~GPZhFKhkh~>+fU!;Z?C9p{1B~QtwWbivfru2R zIBk#SyYN35Z*+=+2Q(QKUOX1_+@rcp*o^w!RAg|`9ysaBH^1iYlHsI+lHXR3!?J^B2`g}}r-2mvE8`Vto?V*dd1;dwLHB#oW%!hG`;L&&EpeTmAc#f+h$&my3MZ6GgXX`E(Md|3*OB zWINP@0JlRHUBK;oGo|~E{@J`sI)z;uzVq?0^sKUyu(9B7jbYit;t;Z5i9*Rz-gdi} zpr{wTPmUPx91!Ny`BbwtB!QUHZ`}c5X$UcePQ%KBdi$}TpU=Om-suP;2YWitq>+wU zO1k7YEN|T7)(_MuEccog2O>Itr!=UAmi+~cd{*Wz`%n&zm4UF;P)>X0rz&sJwZINl z%WVN8eh^TRfIDgXnfcx(XbwE}j!a-=mz1mHre>$Y_~OOgS?;aoCgg5?MH$xfL>8yTPcQC~$Mw8-^=oM?=DTrF$(#q9 zr@PBC8#-)s;ESlkq}*?KwVWnMP>@c_PD)LGigVq!U3#TMv904+QfpPIqRO0q`wEBs z)5>FkTx8QN>(OI#=~zjN@}LT0pLaCg`(4GXq{%kW>8T%>lqBiP z8`8hbufGT`&Fy&Ikqj-kAuQf8A#0>sEsxhZY>=B9bTy^q;O``7SVmP~VIa5$o6o`nahqN9QEIS6suTfu<3KUKY4&*Saq2uV_UI&*h?&buhIvH4d_I& z0ytf2Sv{IU88xv;1qMUerBl1!I$7JGWsbKpGp7H_6d$|>jHRbRa^BR;)lPXHGFHf+90bn~2pMJVQ zlBeVPV~x?_EzU6m zU#YXZRlP?Jy<@`lA59bQ5I%Ygao&D0t^_3&6wqDH(zFSit<$9P;y*1s4SH%PI-o>~ zr{#%6R8Mg1^`NTJyauh0RvqL42v%`k2k5Qm<2z<9iz1#@2d;n0PvSKPrgW5XzPVFE zf3eLxxeHmTkU0bihS>w3X_J(4w)Dt2OZ6P@&Jr>!J+jhY_Uc`WpgDLSCk|zRFOkZ& z>pL6hwVnF0MLjvC`^7rwQKMnR5QS=sV)jqF^L>8C^db*I?AgCJ2NxUg-)}Id%ZH6t z0T}}}emXZAUbjYo1L3(~OnP1}Y+c#7dSQ)!5Hl zDFEOgRZ4-8^N>{NR~^|)O8aBPXs5@CW#J(%uaXq&v+7%fO-FoS&5E_LX9BlFgi_OW zh1}mdVlBW}%M|!i7MY7B06jRF)Cw`_VcAG{Lf13mj6PS&3D;iUdX5K4CZqWMU^f%! zWH8fQ(uOCH%oPq@o@Ps3zo$T;4o5&iSAaKVIZpkra5y$!DWYS=6t?{|WnxPnRq4*y?wz7X-%|0|=_y?<);r!H$9KD21#P_c->Ad7 znOLcm;w9~chQ3`)@-0rCiC! z{Z~B3SX8@k?)DH}5v5-#u8Tc<>XU?Z4Frv`Z4&NBy_+su(3jzU!fLaIxL5N7f@Eql zuXh|CJ~L=qgz&Ah3Nt*De2k7c`@#~=dTtxBs>E3Lf(tK#a?KdJ`^9}ye zEg%4?0$ip!5-6=qB@uEJ&zeZZJxP6Ly5U$ov%sDxfwg&UD%dWr+N?VV~}S&!GpJOGa?f$sk{Ab1T&G20Vl zh=tvU4oji8^=sCs&hA~`4aMFK&`Z#DV=PuHrH2%DTSctSXBD^Z<~kuj39PyhgpdS} znEBYs3yXVgG~0jl@R1q9E|#rWeFpIN_iO%mTli^KV4&zg5g`M9X~W73_NbBmNsFDJP#t zO~9ej$`5VzxBB?8>`T(RZu#17ofHQolVobS3^r5glI9)Lp;A^85&ouvXcUQQ-WAFaL2mw0DGk|EQ;7FQE8O2_xVhWfL%mLwjVU#!@Dd^~d;85w0ulu&zV z0*KJDUy5rsvp`S@-m(gm5A*%BQRux5>XTq}iF<$2n|$UN887-dq2}?@E!p>~=PFhT zunHxTrNpWHRSA{fU4{~U+8nK1-EDOLnDd0}-UHPfyskNtVv&HXmkFQw2qx%#D?rF4 zs`J4BNAI}^g^t;52H9DIl1Om%Xh!YZ8|%+Jg7^GTD&2D{3k@x`Y#f?#%-d^I@{>v^ zvKQ}pz^*LvUENjzzmC%F0Z)>J0C#m>?^My%x+LBx8zmws0!lOz;dPw-#eRR5wE8S? z%jx`dw$t=`z)T^%SKQ2N#(NJB6Km+#&q^(}J!sdhVJ3<1NAA~WYL+(Eu?sAEl*I3V z?l5;W6scifF2QL%h2m%ZSRnn4x2inTI&{t=%WraSJ4?^8XzBb8zwGOS*Xv7fJ|%Pz}z`X@Q;F>XKIn2$VKJ^5hcjoYsu zFCcio>A!PMKGJm2*0d{c57BknlB3ieO8P9Sf$w%Q?_yYeYg(6|%W`0_-&{E(tRZY* z`;lOipQC=8n)(7?7H`18AlmJkQAl65$O(T1SF(Ek&>a51|Z;gf~l^{`^lNa>{*-U(v%-Xz8J&Cz8y&&l9obhhlB)XJ;v)MBK3 z)?LkMGKM_OLH_Zc9&SEggix0Y0@T}5nMbr)-`K@pHMihF4zJsc+4}k6n<^{#?|R@t zl`*uM#zddLA@d*qwqcF;eAzO2wzgm1{1Oz7=iF}C?v^z*L`IB*dI!2w%nni4ivWWa zjvU1=v$W$nwfn^P&YB%$_Bl|3e7XgViBAy?JWKqE)f*DE3qQlH^Tdun@$b>HdCYiz zD)FEGM zG7TP1vj%6e{Iko8EJbgMz9%unvpHqy+bhjHva4g)Es@jP6ol3(Z;=u&-O6LjIy2)& z4!|=qG{LoyjLRWA zyW5V*ZHFQ^JY>11#}O(y<|7jSnegFa>E7GoCeoV)g^xY^RCo96I|3S-n``ykoVvjC zkDXsHT)AOQFl|+CDM`P7fR6UF7QK_@THOZ@9S7sCGksDv+T(H*JATcgNh4E6kM zJE>F(Q0~2A1DWQPj|uO+O~S#NFWVdIy%Pc#E(1B>9yS3-plY=I6Y!`9-mh_(ycQ8~ zvJxhDqpmqLwc&BP&$(h}v)eu2ZN#=9==dVz@oR;tp}1$I+iAWJeK-+GT{TT&92Km& zb`wmPsmK>}zs8o`%6=H><=5oeI>^Lk-Zj=Nem#8}f$Xb(ZwtatQzD#5bCbg#m)*ts)bnVj^GzaUwGJ_+4d% zb!okO8+DoP*J`%WqUxu^Hzm|7{qxJ;jobIg&s&CcmykPuB{V%Kk?z>}89vPLhGgO8 zSA5e-w!_Y#Dmq-fS;p@?x5sD`H{<{xoA`5_Clv#A>wA-)1`bGK*Ibpr71qUiABgfZxf6Qwy#P;6@{flT zuC~or@4YSJ(Ey{`jP2`f7f!I&lS9m5`x>>69Y(E9 zRrl{-F-W9oXsFxpwoT?WNcplVb9#cOPz8z~+KL)@v5)V(bkJEG>#EYV?4q1q0f3yno|^K_wrYp94-0JB-c-Ea5ottQ)f?Zb&-!e^pq z8}#m{XiE`Cxso@6Kd`NVJPEpkP3aoEP;whLToL>|Uc{jOhZX9sUidMZ7)!>p+qc5F z0-oSa9U7ZiE>omxSe-pl{@kNDFe2EIZ@0W#jk&8)>Dw?ZcPU;BD%6D+tnN9tdetMXfXB6{$K zT=b(AYj7p@tij&IcDrESGHXd_{nmXQJ=PT}6GMOCSBKlW!P9Xkf9XuAw>QtMAk z2=b)tQDvzdPVeZeGZV^UeVi9sCA{Ov4bzsNdTf9w15q9Sgm;NsH%+O^+9IfL0lYH~ zaZdp@a}HW;ihEB=Bt-|hAjl+NNiL{lt_1}lu^>L!*{~_26TSZFM5S}*#FmDCSagY& zkAFjaR@_=se?$2PC=xY7yuth6TDKR#@J420=ZnXu`Xk*8?6?pwmqH!4wdzM%-KsU@ zMfH1$%WqcWzu@WBoc`b+chKx?{aR_6uj=%yH+(`zHKo*PyrYUjBGZH=l%3$1@~)v` zPvz2*@{(jMPC?BFKys1f#jZ{*%zYrfU)vH!<#+X(x5W&d-z2xe_v=_8;r!ouf~?6_ z-PAk^+3QbuFNUc9Cdo{)9=&7+)lJLj37bCEO?dx}a6Pb{vw+x0Jam>6UpY+C>HKVx zit*Er3@rbkYE3<)+e)+js&Qfch-0(U2!>BjJYQ?-(sxx-?C-l3j_)7eb$i0CXWv;f z?Z?l)*ZJAL!+b?l`$^@n{(9AFjHBP}m2uDGv?hv{H)2P(qf%pj;8JrW?h*+FyjBUY zG+Lx-0aZ;6@lGR6EH0*dmkj2&JJYZ2WQbzUUwxK|u*6A+P#@+ky-PH@gGX6~gsu&3 zMg5emQywvmDz&Yx<@W#uaP-}?EU~r-7Gv4OGna+$^VPYzdS=jWs(k0@V7z=O8^{S3 z|Jj=TFy*b%adi(oxR*F97K#n|HlFUvXjX$vi*ZQ==(aE=24Nf?5x z6WEd03S#pL;(kYnK~M&H4^oQmbf13SwxJGn41)U=~Na_DaS z?o^~c-MJx(j3aZ3E{Vi+rjtAh-JC7<3M>xTj?`A4+rR9o_=5w_+)T{3gUTr7?W#+p zXw_Ep(fW~e$$?0UIva?Ir7T%ulOjDDRGrf9gx0$s4m~D$J~kKYQq8VQ>y&iaoq0PT zAZ5-ISDnh_zK;8DVSwsj%rI^B;nw0aL^Y|V6kA2}EvnZ>iz+lNxJOLHHoQ-E%Ph>$Cjg8C z-S=C|0I0--&I5QUhpT5rSxNP@p%x+Q^=I|8_Zyfp851?6FX8tiE8UhwwRfcjj zP;X7@Px6Mx@DA}Y?L2ul=HSPVxVG6n3h>FzGi&w-bAyRF$MpQIuyy&v^Mxybmxb)04s-s69J3}o967~S#2bS^8kVr zvCfs6wg=EZ>U>=3{n!{*YL&;#wKPQw>&`qzkA83|q!#bA7jP|&6f*}92&hoaV^X}W zSUK~y|DihL-|843i%B+lRg^Bjz@80Wm~TnlDBd6Z&2_0p(y2aG0KmSida+e9V#+9a zAZZ=;D%@U2f2~|IOKcUg4r$P+Dlg!AWynzOxbAUWLgqSbS}DQZJ$)>GQyph&JKw#m zpW^gMXWgb>_k3w2P5Sh&thdmt=m^hDdKz2$qqlr7{IyTx^|sVgL}3n`1t0avREqc={%VzoBY+TkX%!RrvS(mgSRpN8fPOfNB-& zjor>`MsI$}=9{bT#67 zax-~cX{EiUboE{3So`zu2lGJ*Ah;1)w3c+zAv*eOlRG(@MQPN~TR$pbg*puVJf{Xt zqRufo*T6hY=}?y*tH~JFOg`5Ie8Uy^}T}EWuKv%(uSZ2l)~k ze^3Z_>8x=+edF6#k;Xd1Zhcq&kvl+GZ(_=t(%>Fj2OB+s&sf*5><-R~S;Sf27c@V{ zo3)I$jW)#_SjzJZ!Q5LW+2^DivlYXfZx|f6{kqdhlzmVMPp5_x(&VmsJ_k zTfrF5g-%_zTOKwU9+9kyIZe8qtb0bel!&a{=$4qLQ>EDYAcuo$o`p)ZeltO~iztwp0tw#|)zL&p~)#U~hVz z#D~Y6K-X+B<0Z_f(JiK zDBPcpguLOG7)%P*(#>hh3<5bfniwD_iJNi>UaEVr77@2<9-PTeK>whT|9V&F<${zf z1J{Bk*ENHa+@|2Wi;dagt80yM_Zo{G2id+0*%srW#Dgwhku(dT-PC+!6p%dM_q$gy zT#D)O3469}<7Dz_h6Hnmr>+wSssQMKl`*+R8T2+O-=f(90<63I`|+>(f?{OeLp!Al z>t)KriOvQ>)PWYRRv$~+e{UxlV-cUR$VM>SD!G!F!gZbA9y`|R6I)~kLY=O0*i?f? z<8>q2nKOuq9gOPY5*w_zD{%q$MI*_)M$K?6;-MnSx!b`=uf<5*nSq^@R%q#XJGksR zx(p`9@6gD@H0q7(dV-FMk<+s8M^pWqZBuOTBO1Y+IZ-IzJtb=|>TolBa!`NGsxH=J z#C&#JvfD02+0d55VKQtIq(5rSyN}X@4yBGhvjltNrspxq8Ys{!3Dnl2?8WQZYE`{Q?HW~>P|UkPs>4qe5?z5B?fjji(c1EH_8Pmb}*#;Aja2J(uYtu=sR zfFa;8HSg&aR_a>aIJw8ty0a&iKI_M0bmm&#HI=!2gL3lsUW4=R+#3)+Jd+o2L-2-=mKkN+gLQ=U!Y;{odZH!RV&@TnHn<9p894s{k>+)}(9E zISR@=o*ui3A$CDv@9lAdf|vf9*8B>XRO)t@)qOwc83#Ar{#e>GLvHdT-LJps$xnLM zdVGFmW+k$Z-#=(Nu~FW=%kV1wzW1S}6%0O96Br0`tR17AV zQ(%nia|L&4oK;fjjV8zvB1f`>lA5P{e^Rz~PM`L}0~f=&<_j9uP3|xW;^R?rSF2$i z_w5c&h3~7)TLb(Lbd@Td!__a-)PR&fyjhM!Cv~#?RQej}lvz;kmd|Rswt>4V=IHG)id*>VJD(4qT_PJ!_$8A-Fbdxy*RF z6?CUH19@yOgjh_kECeMq8KSl-XQ!J2-bc?WV^%?U<-~lo!g@ARi?=efm=Ue${ygv} z0zUL>=pg~3H~`4M4VR!0G?%Gg4fAhcd&BA5V5lfpV~EMKUPW@5Dp0I)2Ge+LhH|s} zjTGX`FhJ4x5oO?MJxi0)J%!BFERM;xf ztO4J|nG%PuFSfkdCWLgt)4?zTGUwhDm2O3f!k;{Qxl$hNRu_9keXKC1V*l%WK>hD- zh0F+Nqw8e{PZM}bng=Z@Tqx^#XK!GR@rfPV$ZI0V6ojeKS3Mn=jykmsw>7lbwxceP zslg0Npa$~SOg@s8l(Gx|!nv*DuWoP2#)tlJJEotu0hA3bm!@wY&+|D+mA6i#cj@%X z7t{|vKF34!N|D5>Fu^8j0Hu4!izPyRhDBhDSH!tn+R2Qzs+vFcy4^%~04TE2XBXR_ z+7H_-0SF0M&S%ARO<)ail_5=+gKfrF(u3S3bV)`pLdoPWOE zoHzz$oq|=?ok}g&aLlfZ;TPcHPGTvP**|LrMMik&z15v$F66dP!p|s$?aSgpHVLQa zrD?ymI{*sR%lg{)Ny*EC{@9G<>yHSEU;@Y!M zivK}ORq5D~5H&%K(`T|;@Y!Lz9;#=fKY7M$HEuT!IMEk>w>3R?rFK^4cr( zmficF9=vlAm2vn*F~r$LQkZD<#jNFFt<)U$*Bj!E{caSQs}3nM0DdCzBizNxW=ZnUY6QFo{-WPuNf2jaT*wi*k4kFN?D8`tQbs z-#FlWo8C(d^2q%*-Yf$ol3>Z^lUMk---^|wM#83x@6`v&#^gB`J4hfkPY#N|oJYTe z03`MP%r}Enc_zKt=A;*yZwE1iR!xFBc(fLsay%LmJznO~;$YHAt?p-JlPbD_#PgUz z`1?UJTPa~2Zy^IUCW8CuMk%GOMIso*Je48}ng%(y!Y1suzT4_K1ALlv3!H&imi3te zuwPvC*wz;mjfPcJh7=tN+YRVAwNLw#1YM2PcDS$C5_nM8y?)hN=a8)aO)IM2?NOmh z_E=%}q>VP~>d8Y;XHI%ryGm?%%RjeFt+NRm70LcBw@HLxVASIU-u>Ua3gZntaF_LW z@q4#8o3s&fV>|jsyT6PflnB)-)D!Yg?~+<%x`)Dj^(T|;CH-YiZkRDrsn66Hb3YJ1 zsac+3*Erl1kQ>`u9?c3R!@)==7h3=5;wG=x9s+}txTX8=9=$E=tV}rFZ+Vxl(9UPD zgl_U$dpUIpBy|mq8#mulylolIYl0kP{34l^^sGEiprpuR*exzGn^q>CP-lN> z?^Ba*|80fuS+|=9w~U_MOOSZMHDEf{<}Gw0I_o~Kl~M&!UwWC*q-zHiX3edtz9vdoOQ5q#V8N%P&sLUv!&7dvm z;|E99#$+~y21&7qN@{}uLjCUCS_Cp+cJ)qY%(%P+cO$daDDrp9F1sJ;ie0&4OFFqRiHky z4nQRfPL{z6=G`3tvq1N3QnfR6iIl6;Dk6CNGV7hPMQxr_>FY`^m0Me&4OoVU?aMewtXvzRZUf;xrzuyQOkR4J&N|d zB&a{_g~#$)e&hDDB1{ay`P2aM5GO^wSx$r8tB<5>=D=NvON}`tQPmYx=Dm-s^#L7o3vt#9dhdv@B4I_IC1RU@&0MKY(jv2R-RCH*=*=eUIgy zmhn0rEUu0U>UK|$0(x`FLbTs71|ifblfT%oG^(m7AhHWAF*Hrjiv?C6BwHMap004Y zo`ow1ZgR^orG`mEwH{0KT?F?Yjyr33S5n)6s@V~vQERhTH$;c#Cyg*k&VF#2&bj{n zN~CbSbh4!7U1R9g(4KiBQ?8jcM^1&EAYRE1B%3#evJ$O(Av{;`pOa5{6K%c z-zu~ovCj^ivWwkza5D>t=6yBa?sj-&+~rQ8BB^lmQRC1GqRE$e>Gd~6PaFbjN3{6w zQ@2sU1Q;j_I(9u9EtZL0WbOk7Ulk%Wj2p|J`;Ff=0L%ag>8~j$cbUQ^xuAHtaCbT@ z={<;-mUX`0XM>W{!2Uk)uH2LaJTaeft)&P{C6k@QwaRM>3AZ-4ZtTiZMADfYjZxg- zPiC~LIVx#cd9z%|67b=hzJ26*-u9vZtmfefu%iA>MJ7O1>K!n^H9#gcHIIS65_`1H z?(G7y_pRg7LMz$p*nf-@20I76rEc>sYrlEY?|ELkfs%o4+4n6tGE)TcgY(PvPu`SP zx~6;M*0ZWYU%gK96uFti|4d&#zW=>$ZW9~i?F}HZKh`Nl-uuSzCl&x@1wUfV%%|f2#x_5$7+?~ z7l6!@RY>f*uEg7oi>txJBvFRQCNsY&KiB&Vl3eV@nEk4zR%F&S6cpLTwk)#dBeQ=4 zfj*j+D3b~FJXI3t&0#GCHgye6sWkVJnoS$x_bS-SGqs7dWkE1@zQMWW-mh_+$$WvZ z46IRPSPGGQ32vnYA680pF3rc8-W+wwh*bMR2nZVJk~g0_V=%jGBFYuJA_y-rn+c#* zc8znqEGHu=IHt5o?G*u=+ExJ2vz4Yz7EVkX=4@Dg6%Xt5?#sJ6d!sevtohF(g%i~lGHwCFkw@#)RqT0eCu{cfk@tZ^S2`Y+#! zKLilysJFXGyFZPsPU`?{^Yf2MCyB_pg@M`74?Nt_7NRC|LxQkY9XO!KtSr2s$OL5b zH*e)qTf`xjSmvn;HfTIAV4>G`(%0_;f_Zhs0&|slbP;#L?{aWPNi8aVf%oD@Uz(END zp#DebtkUhmD5+Oaax;XFQBPH^_5f4#uBgH8_8Up&c3n85T_BPokN=^+DkHfhoU}q8+kzQitGjdOfmc%$m zomzz*mH)Lfu2{tKI8Ns=AbPExRIF)LfNx1RKBqX|r7@_`uog^!R3(1Bjxo(XvqBtH z6pO=al%M>L)N&HTp9{MJV@?AjlMF6loFn*-#Uc)8gCX2njg=6%0f6+e$|bcC};%MZb?Ky^Pp;&S`%{6l!v(JMT^KZLk; zDXEgTWAN|}s&ZF5TkO@@0Fp*>j0cs{fXY$#4_9V(sNfZ%KB;3gAHl5CF&~Fw&U*+S zA9i!^r3n_(?KRTe=i7E#4?sJ>t4@iM`>N*XWKUM=4{6GWfSTc6_V_pT40UbXY>8xZ zK+|#LQ%=-#&WA=h3GdO|l6`aQ?LlA`)-4B3*?EcrW=fb69gc4Q{8e*ngyAp;v$=%0 zwWovxgxnc6Bb`X3v(p5amY^nKg@sKSRmDK1d}jCvrD6&6oTrg$*<|S_UrZ_<9AHrqkrEfdU60$s4la%Eu@K6(g*!8si z6|d*IYJWnRIK((JJ_i8sv zLX2Mi@XXj&Ry=NvJRJs%@w}36b}n*5Qlp{2e!ct!6}s0ccySnvNLa4FS9B&qB~n3z zN+YyEOB|4OQ+YUxmFvNp>>_-;mq{MHf2sP{Vm{#hPf)q}u?fWi)tON^IZwcRx;Zdoj3q ztWEiWq}rxbawPm|;xHhuu-=&Zq_HmHs><_@+QhFBa1MK$_V4q(mQI8)^%0T1&5ai8 z_Ip(tDc2x|gx z_0VL9Q>)-+X!B(5CRi>Kp}12@M~>V?79JERiR?TJNDjcR z>}iYGg$+Ce@nZ0hP=+C5Bv=H>F%zgwX1=Q!T;B~-6}pv54aCKdl51?5owG^ z1MWATlB;5?_00_jZu7q?)a0^orpB>@Q8{AUMulL&%0tcs%qTnjE5(tTA{kkUgZnXpizh6!X7G_`h}UdG8Jf2;m@Yw(#V*kdke5V z4a$q$b-R7mv-@FxxAielHaf=LthsUQ@prqTKiPp8gi!CV#YR6d?4jI}*fPrWZG(tu zPB891IrZG_fISPeIGOFlbTaxtIVe#V71JCitwb!j0Frv78X~lDbi?IwK03)RTyrlcSCn*PGL~nW}NYbIhvT=T1oAmQ=DR z#s+H}m-X0U^^JiAchcK)JeN_q*OV^(gc-*XRNZ~ib)Q+wLkHfc4bQ#rWyu%yEuDSv z`~J2&wi;&0SU!#xa2&EP!?Zc5xGS(DKeWwlfA7;}+c)^Oa8E_mb1CtX7-tjN^LnA3 z7Y2L&TKG6aJ*b?j3Fv+b@OprFU&omN5C!Qwk92 zQI`gpj1UH;wrQk}n_ruGD6|HCyzmWFvN2^Mi;s(mn!f|fq4gr{jmWTjt$-J!_98-! zkFg$qmR!(rJ?@=khT&^*TzHjOn|pKH z7hL-U`#OJfYiXzzib&X9A~GV1O??#mqrMq^M|F_qq(TClCSsw+->j+&Ar>C;&M*{y z^HoFTv5$C4k}vnnw#p5vFcmpm)co!HRf>f>W$lE~uZgaqI%NtEB$o3Y&*Du9%hlW% zFmsI0hd6T|H^p9 zCt*}4#VGUSv-execxwzAmGG-{Ve4fENjq3Cr+5l#Xd3I-_Zs0d2lS2GY=h4AFPVu@ zPpMSwQ!tA4ee~ZnZC}~L)Y5!#fXorDfG<6H;ue6nFF}W+Ow0?zuaA}i^$I%6>?iU& z5lL~EZY2BO$-s2ICWbE{<3KTLg_kseW|& z_LX`Y3~lWVU%f@&q#}dJAaO4IDZ!bbKCGW~92n0+*z1RIqJIW41*mNvxRKvv;kS+7 zSNu8Tn-p|trTFwlkc-=H__C9co3%#^}Sh!ywvn`<9wcsYI2E^2Zu|f9zME#Z=QIFpK7nVI+-#)Db z2b?ZO3(4emANtIeC{`kRzNhJ!SZ>o>_Zp+_jBeaZ?z-VCv6uF;f}xRvmCk2Dyjrg# z%Sck-=uMm@PmceQame%z{`}ks`yex46C6XaS}szndBKp8|I^3liG6)qpOyg;c7f)v zzcrpc+KnO@v-P!Y)>oVwW8)?0_0%RxG{>606*PRajWyq3QS|fd?GH+V6s;nI_s!Qs z=%V%Vs`~b(UUAK+F}d5-*{#}PMfE(kjtv)gnJ8xS5#N8&^rCI&_G8QVUf=TPt%3y- zrF2(VOYN^raOxnZOwp{#uHCjQJZQGfU6p@WR%hFe^Vu?h&0Heo6+d!s{PJ0P4f~yl z=p`N!6fa?{#L!r@?JLdvDJfS`oMZOKqO70U4V)Do@paV?d>Z#Gn7Q*xZDxa&$A(5- zkU+(!9`K_2yjc&ynl!%(d}t9@Q7V zh>aM8J*>scSk-E9?ZtQ7g?;?j_K$r~_9&dp6Skdj*vVJl^uwREv)5)<8lqKLG5gs= z#Zy}7g%fS{LfrLV3weJ2SataGU?DrssfBpH*AHj#xtYCF(w|s>isx9>yGyo@?1KGL z(jWB6!F4fgus3xkiP@GsT?%{`_(nHAE{xsDEsKw|)V{^wv_B*FE*X#EH)~zFj@ri2 zPH$O<;P9CV)hIFb{V@%y+20F`nX~*~Id%?e-Eo{l4QHNMYBg4fPmN3w+g?8S8bWLQ z)nUvwyff5DNW~o&i3iKUiIo@-*56doYmKrVLe~@JhUB;GQH*orxu9jWX+s6}9|;N* zFiSRyT(xcCSo@gzc&IQ;b^pc#{czv%hY3qKQ!X~wT@V;zBrGQfw-Wi{S!}6ZhY9ChevTb&U2L`dUZyiogy(*Yp%2UPyV~Ew(G{%zF zVFgJ>2)nxfK2NvNGl4`mRQUmI}Jk5mYEa|*ybCyYk^&ILJh zH3}G_oAq2J9hxW!#&pTd_S@vy{`azU9+R!s*J1j4a8Z&k;;~(~xcXRoWV||ELI1LMHs7xA+VZ_$f_(3>wVAJZ5Jn}*%Ax-<2MgniDSLGs6T7D&cpmz!YT|QwcorulQ4K?;0xz zJj_e!iuNa#N~OPCk2P*ypg|dbTcAbpw?v62_{9{{Q^Pc98!IHHYNm;)eFpJuTM-(h z-ZeK-*U>;*<3oKEQPkjSJT#4FqDg1)8J^J+C%~~JLD?@6-WJWS@9jF<62*=RN}3wb z`Z$3|fBrH5Q;5&ncpH2;A&ZPg(5Al{ts?=-<3X|LJVO*)^J1Y>2&cC)9q zvPp=RESrg~tAR&xmZ`)rJ)THUXhA4kqptDwOgnYicT_$Y8Zz71x!x1I1wZw&ZV;q1vEwq_=-P0t@Z zf4$LbdvsqiQLLk?Tu7M@J1TvuvwcCird|0O7I32!8qePB^iE5v7F3SW;#pKQ=pSwB zs52)_l-;8Nv7>BYuMfvMDMgQ5v!8r@qZ_Do2Fy?rEiSg7U-99 z6&TDvFBJ}0&Jv^2jrKPgH_<*NlS{G<`5ejh%a2@rg_0y$(X1svWztcm-F_awL_qkop|3|%~y!&jd>gBD>X2w6A z35)V#hOCW-GKGnPDl$sR3q2ofHS4y7kOjL8ljUAMZ!HZ>(jS}0{pvtBTCWQ5Lzbe% zFNf&wbzr%u@X+BRVKBVd*eS{nI|N{4+y&%Le#NR#Q&5?djb=$l)eFsWFfV#0*h2&o z@W`qNqj{twBsUBCC7^KNNQjqDW~K2Yx2qFHzRRLTY=C7dB(yLO7;(ylFs@-d!6KR9 zE2B%di6(vr{niA;Ll7hlG6|B|TP=DtCW9r#xxixl9|3^r*x^h#=G7?x$L~W2qTaEM z=WgZ3?a~G}{C{6Y7~BzE1VWOg;P7+lli+r-cUi+9*FsF8C_#q1PAfy-uGcuPlgqOJ zp1LRWSAPKwGu5?ZQpsfc=lk)$iS+?W^F=Ijm4Im^_~I(D7)UG76Qu`0QQ=9KV(ZU; z{^u5DI9(= zsF9yB7a?6&o$H>Qr*TV2aC+pq>2c$OuFj{Wl>~^#!29$;oW~tb+nBW; zjgZx?#ECJGfOY4O!F4qOG4O1D8tgkaqG3e3Vf~@BAeQVsFFhz)^uz z$u<98;Lnw?#DQ`ylQu0qxL5ao1W?X{Dlu-U+`%aa+jA&C29+3$`^enuvO8dN@z3dh zKlD>2KbB4ofebP2RYuA`GsPcSrU!}h$n>VkhFUn}iJrHZ9<;w7Ugq@QYpEE+MiTyZ zm;?t*@mIUz+@r>8ou50wW6ycJxdqUaK&a-g6(*}ful^KY;hW`XicisrCV|KPAT(L)=K9oJyKxk^iGTFq_aq z)0vda0U;GM`~!bKRAI;kjMV>W643r$7nThU&lOyx5sU{nHa5B_^Up*1?}4WHVf*U5 z5{@b~u#sROqaJ}jm*Am?o|wezs$6aJmkAnibdOxy!jifrwa zgHR58pqW81s4!cBjcs2=_nfjpNM#%C_4YuJK|$@%6-28lX^5mYc9s^tGQyYmL9zXN@~-=l7D1^*ih zH{n<~^Z)cp+!%M`kmr13R!0yCFzZ+?i(-9G}lxyTK8okiw9CHUC!Kz3C#aJL}W1I;W zaNNFyQMJF3vIpJ)xWiV?+tOG0(~!``$?#s+h8BJ?M}7k*DMzp3PN7#A#a4o?dGfBuT3n>s^~IG^bci{$b_4|z%z zgS3b)b%U0?LfIf`{7*?fXTD2OEPvrZ&(8{e;ALumN^08v6JOp?R}vz^1nw0RW&mfd zJms3!ZPwVCdhI*`?4AFf)-?oQGItuoJQAyK9_JhEjpx<`H=Ku79xp=O<9?6^(R(-8 z*K6a(AD@SzFC;)5;zbXp$SIuH_%V#{dGHAc!0$_Mf$h8$wr~`kPd6|-++DTAi(5LL zve*Zma(p2E<-#$`rbE%bHSfz-tswQ=rGq+52;D4|zh8$6UW~kt7gub>oPMJp7nMnq zjPZB2<|Y8A1ss{IMUhyP#zGsR55ctEr|vH#68@Ede+`o$h@LgBhpXq9MiuLS%#Sfb z+@*nJ?=pxG?|t{(%{ZTjzyuo1%AN7{L-S7@IZh1h2DgOV!2dP%as0->01A!j$sGoo zx8E63%Mk?9;*k#_0|~^I;ljRB!t}i~2l$H3QMO!m-SL0T1Bimy|I%Lh*l4p32ZzF1 z0Q&V?;^khsD39Z-n1F7bJMx^Rz9BAH{hE zm(lkU->ZH^eAG2b>&31VO~y&J5Cf8m?q^UJlczc>CB!Z(6tpZ@@kr#%GL=6{OOW^x z=h}PMrt|*e9%0<~IMJZq9z6UH&RaYP&D}fGmLETD`wDp+yUY?L>X*&7#j)GM8gBX; ztl0AD^=aIS+Ro$;(WBe?JgfGdtwl)Gq1UJdL_SSLX;$zhj4uyvcf;TS1x5m-&4)e`o-y}_S1n_U? zJ9ruBN@gs?`km&s*AUw)){qCaFT@wBXD=-487Ck;a&ybcLax?m)!v-FH8>%ej15pFsPHq4kAz)eTc;!LH$P*=OJqM@sT z4V*gq;WtdAH}sli$y4YrcRjc;)h#@6Icl3EV%FRMnBOdt} zjE&g6A-<*`2NJR_)Em+iM4#EuDo2`wneG%7G%u_Xg;?YAhXmgWto}x9e%deUKW1MU z?cy@t?dK1NjbRErKslmMhu zN|-5*oa;k!Y!^AojmE~ya6NJ)V;heLoD$BEAIwb0-_0-slv8-Z5XteZxlouI40hkONCakz|>C3B|Hw<6%`uHr}EeD2gjNQ`qeluJaaWz}) zqLKf+cU=X8q5|#r$|Gh#(Fg`Vr4k&+8j-RX!Oh#0spd;9#XLg4LlzbK+9$iIYUo`^ zUe?93^ui@WmE$3gZRL>`$Hw`$FJ`BRdz4921iExoq=YpjBa~`{x)jUI0Sdl7Y9jwk z+Kg|#Vu1&z4#g@0pQtljG$r@}IM2uCXOPmIE@?M;8{O{sq8pDGbj~+v@$(T35W6M8 z+UrnZV-XYrB6A;M)>ZUGEbPT$aBx1Q-4JhNehPMJMR7{=x727aYtgmOGfyqVPYw2 zY;ia-i@od0=Ko$AG%*|CdB-lz$(hIh@w&uI;Myag2>yel74`msy!m0+w+JcO^9l3Z zO2EB#B$6M4DSA4`_Cj>rRA_K&A8OB~kB>S!gA{t-i_Wx0{DFW6$1BN*_ZHYgH#)v_ z{>%lx{i0L}QBi*|>hR-n+J%#a#D>M5#{qu=AD>B#rvKt+9F#ZWNsPLRbe0;Eb^QGK zqp;BgcF6LNN5TX-rgZ2MUoH;M_Jn0<(?|AXp33tKUg!RgAd&RYApC{2EW8lc9aL`9 z?1V&AIgJh%-CNcF30~Df5t|~}2eUU(e)P%TK@vFL6q|#O#8XPkObNyu;Fq5(19}%1 zf#xV>Sr#ng{#zIG4zq;#fFj|i?0=@Aiyw2Plbbx~xFZn{+An^Pq73koN~h&){WmNG zN86!-nbU&~wadyHU#!To6=1%s3B9G&HNxe@<5#^6c;^Lk`-@T2;~2n`(IS`uDj6l|3^xn6m(AV z7QD~+7u=*mT~MgvzfNPe0i?&hA@iT@O5$ap=^!luI8#%U=C3a0s@LJay_ej7NZ)_#wLKUyI4aoPjYsHT180ozT7uSG7ccQ)aL34C6|z6yg;A|K++X*K z^#45rTwjg^T*zFAniwLwbjR&N{?~oFkS8a&kf{n|4-(Ez@jvJO({$7b9uhMpr;@90 z;^Kt=^FHG3wL9zs$$R?VU<9~6v+Pt>l1J&J=UH7gHU*$49km2-A|JOqf~;jd*X#BY z>vjh8c1NE#d>zPfoKKezU-75#$liHI;^Zmv@;7p*+T7?u?mH6Vvyi$_OJLeiCo5b6 z*@ej{3_yNfJJ8_isrNL-G+!a=PUZeg(CFcE(eo^T`sIEw7bRdO3i7xBm<8EtqU&18 zy;oO<+t?f8bIx`K4Mx4tE_T}OB)2K(h(c2F|HPYUMYtacfw}M>l;rjEjDqCKM#jeb!$=`#-dvUs?{t;<%2(*Zr zeU+609_fAWFG+N9w2*gs6X%md&Cp)qJr+#NQ*66t+-HogKl;W~dF;j$)=eT;hi zl1a{Ju6#b$a-^Mak~Do>?6oR7nS;uMww%GouI+`jYl1@CKe$%K&JIFFc8834ZD*uD z#HG$4qU&097GO~9*j+Aa@RP=>mLB^5sJaTMDzh#shzckz zpoB;_NOwp#NSAbXhm=TzfOHE;cXxM(beDuQ64LPB2RbwVT5mZl9lm$J8z=VJ``pc` zk~9?L4}nQ~AGwx=L&NNF{9O8K9v4cS~7-(_nM-E(dHCT-3Akjbwy( zWX3-seU8m*)}lShj;0Nhi4}rYNAZx(lr)_6iFjyAW-aQbu`TOSlp>?L=-Jvy@)MD* zODR}iQd3h)nb;ghNOPB5gIn2|=dAcSc7W*X(^Hya&@6fU;$+*9YUslrXymH|)bq9i zA{R?uNH&#xR(wuYobH(? zd)zoHaVBY!_<1^JM>d3t^RRKTd=1#o}XM@oFob@;g5YlRAw0ekUaAjq_|(0KZPT# z)TDKg-ibVDdJEa0%WbfJGwE%{pK;hmUO6p!zA^UaQl|BJ3H$DHr%XEbS*0??J6e(U ztCkfbKU{}5(U~*6RhiW2)IvRLjLB~dPu1%{qoC7y=q} z@gPTcJ3efT5Yc5T?|S8eJOEE9^!bfau??k+FYnqqz9IrPbN_mx?ci&di_NU6g31{R ze?O0KSIhyf(>d*^eClt50Aev8S5O+Duw}4fci+s2sa1(%$zDp8^V`asJD&;6k7?DHdxXXbcE>%2iMLgyu!I+81%)hsV z?7|s^5)2zbtm6#IxT`=)53K9ivR}(*X`bt2E$q{cA)ecEaC8~Vy!(s1tbAnFHavih zXh~R|yDF$!n6dh@yN+Le=KQ|S!9GuKrN%oG5G?wX>0@$m&|1Eq5yz;b%IP@VoNpLL z!w@Jp9JD@dncXr-vR6DR(7{l*Y7xOJ_wh;y&{xQkN}8i`S&uW^`4&VIxJq5K8Xjgx zNbDL!vc4W%qcUIZ}-{{q`7Fu*E^eas{QHhB}jnVx>3*kSU$BQYYp zcISIOOeMNUlR!p5aG|Ef~1lQFrLgazeZ;rPQhS4YXW&(gxEzeyA6is|fv zt0$v>NQ!n^G~V7^HZNCc`V&9zTVS5IlQW&P*y{S=hlp#o7Nvxkd#R{Wn8Kvp3R4GO z$?L-&UcR%|M-jp1=`##9vt>m(szmVL5)q9wVpnZasjHlc#2*6G9-QMdRTt=Y<&S@y z)QFhY7MiH4aJKawV2KEQh6VacdgvgyGXKJ0uHsA7-DX*I9WBrS3lj0(VOl?|{*7l5 z!9s7R4O=_rWEZMEaxk)+|7B`MM#ofBa;VeV#`z|lZf86D!Yeu*f#^k|s@)h(_e~Nn zCoH=4G z6Eiz|jhZdYFtaI~En3XizC>(PQzf}%3^ZliuRjthH7*;I6&E~QK~=UmKc}kd+ZcuG zS}A|}5JP5fgnJQ-GC#&LsCyM&@qBC2lx+pE1&?P!sG{6L=N33(t+c-(^1V0CffP|I z{DdBw5%=IYJ6MUSJzef0!(>*b>%o}pf$^xj@O`ro3_9w zC|eUiX1~Dccbf$QlBr@WzR$5>zB8G{{Lk%GRl^IC=V+qeJE*X{YoZt%0Y@-1>#>M1;BKC<#-{MDr*zbig|;I{}Boa^2b5#ley z3uvQX@s}i;lz|T2X(8;r2Pfrg;I_!aK3YpmO;>AW>aq?U&jn?moS8AuFnd z+LGvKj2{KI%IYVZqg}``SG8lNz902b@hu87{3S)*e6G7x7~XZ3807ukD3XG_5uga4 z4VsF_Z*5r_7lTC+g^}X9wo9R zIOkQ5T_-U4NBX6#5q97r*rx>b?=AXT>1hVy2w0EjtX7^yV zHb=`0hqUHw2bh$K5pJnU+EE=wL`i*mx*dRD4ozQl-bj+bBAslEF!xSGlVWXg2Ub?1 zp}Kp&>DGd_;W!-s0<>>4cDTJh70#_FFBcf`7f7v;Pk&7U?t~D2VObkQPC-UFNMNw& zb{br+f3Du-e*MhvXV|A$827ZB)27>Lwv9wx$OT1i;1P5cScH_Kt8ZTr$eTphT!>-u zXZ$X32jC9Lf2#WCkV;;H)}$2JGGfGU`l!lcn_Ra+XgOq8^<4=2k$8P~^IMi1Z-aPa zl5iF6ig87TeDBsz;65_H?^@GSrW^N=bvx72gfGM`jQ* za{w=nuqYHn+gNR0+C$>0KYRc#Td10MDBEwmq}gc=Pc$>6SoJS2FP{PdEZz>Ka78=8 zskyB-ar8(DBaK9vKKb%`f~5s3s|UCg;~|esKYCsfh{Ohq_f)|=BhXoZ02X$k+_B{m zCX9QY(PB2f=+7QvME$bvqzFCyD7|-4bu51Y)<1cUIb3<(?p`^-=RtxL`oL10W9j2V z`lKZnTTQn78vJP80S+dXJ z{ND)Uxs{*n&2t?1c%OZgv*`SEv?H=c4mY_8?nvHyiPGmV2UN@&a=94g({`hh2wig^ zTGU+1$2R@e;`QW-qG0_w;*+%)U;z|0MF}ulzuWXt$0*=MgP5;D8_s8uG{37j8;i{* zy(Dqi@*`7B@}$TnQAw7prB`0M*0{*r|5ejGFF>lF;~t_Q7b+it2^G@}u3^FTr-G%n zZj4b?RW$>FaiuZPewt%ocYT8k!o4EGe{ded^{$4~#>I?D zmggz7ki`>?mbZ2z1J`i<&AxL5c;^Fo-vW<`?kySx{yIMCLEjN_NU<}QGnZOMHwx0DqSnXl0rOu0YO2^b zMamfdC+JX5=>?EnH6fvGr|UA`%TMq5Vi2MRuJY*c2lGsh@2LaeOFciX-!Zf{A>a*H z7at+uhX5c!nHyYHc1z#xYdgd>iL8zsbzt32&q3unQ)QVP7*{M8Il!Mm&mf7K5FXu#UOJgx=^yV?u9UO_m+kZ4TyG7Mta8eo z+>Km;^Z+EtFZf16d2U|R6@j#9dI6+664NGm1qL9Ki?iz`%)!&h=tx%YTh|+iycm7^ zf5o?Zssb56V7`QY>>cIRTI~+=)wNuBQ`Jx3Hw9|KWxgl<3_}8k%8xO7WWG^Bi(*Ln zdw7?!J7OuR4%^f!{)asU-Yyqn=M;)7-)7AK42@R~`3xjd2p%$p#G`wl6^bR;=z2Gy z+!sbf!EeyP^UR<{b>v1+tJOKEG9(LcAYz5T9*JPM8Ag_y2~LunhZpM(ovDW5zB=Fs zJ{%>rnwdH5wQL-Z|!lDNhetyix{*#Z0KtUk@^ru;Z zFJkbXQ$Prf%i-`Dms)4^F}QRCM0-iQwcCYZ?52e^nFI8FRAHRNy`&#O5ph2t!xn~z zha9)+IIK1bwe+X>J&e@seZP-Wgun(Khb?!wv-DA~fl%ydI_0ewYTbvcUtMLrh-|B0 z@;UY8eR;!QL|T$3QVP^qJIdQY?~TaRm%tf0{PV8CkeV4GuH*n96Qq#CngN&v;>u=$ znKC@TIRUP76_yh(W2CRdh#+!?f26dA2r6&vI&w}5;olep0+)#Z_Z329r1pcTca5hN`5J*!9;a(@J5&sp1c*lXah4>qO?q zz5c%dRHg_f9#W*{Pq&6tD|s!TsZuW&Z*L5N3pIGnYQH(s;fsK6kYG`3yyr_%01M?! z@@Y{e(f|2Gsyleq;t&WY`JYb)s!Uxa1Yf4!G+l)JtqeZ&s02gs$?ojrE7ya!bpT^B z2e%Rx$_#B~B14NRiqy1|P$y7&=WpHT<%~E`k|JQPmccnPIpfcVyGM`Z6CPlQ!Fcf0 zL=SZY$cTQ5S}eMYPh}9Gx8(jMNlccDbho!WGf9z2QSnJoTv=;i$eIdA2(mXi~UpTh5QQd3>d!HC5lP&FJm#t_YG&m^r8gyqU2yjWv%}Q zj(0ws&Hsx95XgAIsHf)K2o<6Hc^GdX>}w2^KwMOF*p5m$w|{f^yD9_kI51SXSgQ8- z)%W4gFk&SpTVV4rd>i>w&eeeL5?FP)AYIfhI}a3-6HY4krU4Sv`Jlq+)FuD5sCyr&THaO4q% z5N9}(oZMG}_&XH(&ze#JuoS_?h=~rRTbOqZ{N^|TIKlXuw;iyu=NGd7(BJMRJKYnE zl@aHW$Fu*R)4wah$nE(!F%>XDIkV&MXt0nTcin~s0yWVC71!1yuz&10K+YkRVlfD} zk{!mQJgFU%#NSO0 zCQ=7Y8xRsQSq7M{$?+fjGXy~tnY9m)w_gZ}?Yysq$T-3vgR+m5J&O~Vi-N#4>I=<}RGs4ISS!XvVCN{{w zlO;mbviXYIpa##(D41%w*f>#D#MfXe(#NI{yq$eOslxQHeM~Wj9C)>x8Srhq_%NWm za=?>D8$1ol>w^VIUN4k=%xZhW0`40Xm&E)WDo;`()yEdKJx5!SA3o`tqnq-c zE!j`Bi<-RYbErGhzoUc@X=ae!33pU&jh;b^vYekE2DvRQ;b%n33_esOzdV|aygY~x z)h$^Iww^G?022u0dQ(Fo-Au_OTMQ?^TN4eoRBv zXOH$XX00yuS>LsT=q8_~Q%;O!p}RAV_*wuj4K!d;R;l`UA+mRmJQo7Ju;EXJUQ=(# zGZm((PZst<-PtA3~0H_@fl$O}j9=JRwlzm8=Ks=|!xAvC_Ioj^ian-{V|2rXV?+EQDuYPdm1JhkZY1-4-lv?_ zv*|9Bdo5SI@At-Gck-Z6OzsRP;MYKiozTeRg2ph_Z)+8gNb-}}sG z_dOXlJEc{%a+hdD>AB5eK|B6}fA%EjzbwzF zHPa&0tLh`!Bga<-GyXl<-AiQ8$H)y0LxQO_%z_E1SKy;WLUh zoZ6b)K{|svFr<8B?C7J4ca(6D*c&xNf#VJJ_&SFX%DpDnySaTp&ncn}LE%N$GNrz^ zSZG7+&6G!t=b!KL@u=V@?ou<-ycw{KA6fgV7Cg}yYg&e~#XR#X@3-rS-7ZOR#_&*v z$wc1o4JX;#@kF^P$BZWN>ziC_#H{Nwio~)Z0?xr=q@h#3byRq%IYOHvWf5#ZtP(?! z;Q?E;0%F=6@)LBVm1(Sxnu7(%(bj!^6EL7z=*)TuBS(^F#xS8mYzw>qI+n?}(bP)z z^yNX^umpZ$Ri)gJr#jXB5GIl>tuOP6EFT3k@4~QlwH_r685X|t&T{>+?jGjSK@PO4 zxp{!Q0a>*{$b^?Yj9y?0CyVrUlPxx{pbaI9E}IZY4v*m*~p zCMRufcu&Tbn2Y6HY`Q`c4R}8$IQgRdO%?n1#0h>?`f7g|B|o-4Cx+&?YLaroz3U{0GG~dBu0x9OcnP)^JWf)XwpCW z^&1o*&#fW*>u$k94F~`>g3LC_9!o+%@Hz$bz+2ujv&XdxsYG_c0~PDd{7)%yWJCQgZiIGOb{&DOc6=L&G!!~!Ih#5;U|HF>|j z8n8x*V8B3%G4)r9;rSd?`6FAs!YADS5(_|(I@BPR#}>J6kee=Wt94{xdtT1e}* zz7!Uys)MwV0W9!<>v2#lhvtq?poE}ZUhV)OhTj*b@0=%)kLpBBN{HV1k-y(EqR8J{ zvFJ*Gw!K}V*E{Y9Q854ec^$yZT@rxDT~aJ-Y9AB0h`AI)i`Xs zOFTKiTj)S>9|YaId1R<$KoH^4yExZoUu-Z|E;>{L`RzH}Yj0nCkKGlMVOpUFYu+or zW9yiISo81AP?-N=QN+Aa;%>KL;@9i7C;XCE@SnFyA%GbZZ9`dl#b7ME>}4)nb10aC z3d1@CsiRu_1jW_0r|xcFvwr3!(tUq&nPKTYVFP_FuLG3vNXJRE)#kC7sW{9g2~uDe zjX0~zGEi{0hd*vZc+kX397Xvbl37Zzs5#0&x;W{Q6>Cb&^Ol|QHSY|;#TKG0rFP^j_md>| zN`3$0ryp-^WMeCH`G%9v#sg~>E6!`)gg4&TMrv$eJQQ7}%O6&B`kBZBBx-&FqMxiZ zFC?o6s;}kXf~GZ>s^O&tUQ8_7@_oLob6wJ;F`CJ{A85uu&`|~5^|Nb_NU@R+;dj%?f&U!O1OuGzu456v+rlsG^Wp~T^mX5 z7CBO=F#{nuuco#(taid7^sNoBt%L+sWB*)4Du;Wxk%ZmqTA3(rk+SG*Og0OUl^)Mj zqh%Nyw_lhGK_;DOGKKeNTNT!dUv!Bi0$U22CC%(9!k8n!EXOSgt5Q52I=0P;l9|dJ zPWVMP(_lHyhH6OL21?h!+V)qD_W9CkR{J?#+(Av~%fdBv%90}W)5PMucJ47@Z5UP$ z*hD#)L55105J4bwpZGDRTAsR}!?xDZTk)V*Us9>JyBEb_3Xf8BN|7Q`_ss)rzKl*j zp0$IVRp&Nk*khp9tVgU$%uBq~em$^3=~)RM0S<=&w6w2eE8Z*F*{?|xmL zQL{nR9-Q~86OU!@L$8-L5VI415f+ITE-bBP?wAPOZX2Q$u1=x->NpjR%Bng4QCT&l zxb4MH%d4|roz=Z?pMBhamL>A?eD<=gNsZ(-)B6xD2K3MCo; zEzmnNleIrS{cgS?IU^199NFOm!PipV3Yz)^K;hs-w&J?c0{fkO;Aq*C+1RyKF`_&2 zE7It}oR)a)@xo!10MYB!BDM;l!WV8)gzCdAZm+vvXIFdHyS5ZaCpP{pL-llyLZjfq zSl{qqQ$C{j()%so zg|f69;79hVd^+5sm|W47JxS@OMTT!6k`S)q=Lg@=Cf`1K^@W0&q>AuW2%Lq+3&S04 z_1QyME6@j{_nLFnIdv-RHLLe7>x6qsKf>7QLxRViS30N3@{4%FvRrGJQYlyxn-PZc zAv~V8ub=HXqjl!&`?d-f#rr^bRb)9Nd07d&BpgfecRP~tCDg0zUbk52ILg}mrXO!B zH&WhpRz@!UR*E_3Z%ULpUu7~?XA~=-}EmQFsnrOOZImMYXqMpl87gVIF>2fd4`p^c!Jf!vJWB}Ank<%F zH|j}|9UnSJ#i|gxW{Mi+w#N+ffMUcGFeed>`RB5tJ;zMX z{-jo9fe%YZ<)h?qRdP+3tx&M_J@p-6 zrF>{geq1cGi?x0J0TY^jIm!6BN~80mKW3oVGINN1L%;Bkd~~SYtZzFBrUSmC@Q|{sw$=w&pUJRG-lFA zQL3i1>!jU2Do;u+YO`yklA!1sU~uw*4^BzX+ex^$N3K+GN&($8e3&`x0jaA_D*T@a z<+Tq?D>j{nk!u!c1%%L7tW&$s&!9r*BT#Ox2(8SvhL$*U0p({JGo0Tgas(nFBQxkL zSl(}@0>dj z2NG(l!c;FGUamh5PGCAjiCYbVLxm^!lKfyo(i+g05%E2cqKZ%)EwK!vG|#!R>V!jt zXI?RnN=l@m0ohFR^;1umxO5cT{fNwMO+LAiFq2l-#>6Y*4feR}+Q0A^uNs<2ZO_W% z;0zm^v-t^Ok;@`sfZU^rGtcO7Tl2}OeI}&7T(u2VO19-3`CS}`A!N>qxT^|7A@tGZ z<>FBKEZZbc@wD$BM#`9GFV1Z zs6F%Wc9Qn&%1naPoPto^-qU2?2*x%4dl|727PN6P8B+btiyAoHnz;?0b7IKcMDjg& zvYb&am(YvGs~(DWem0kPwJ-+h2ss1W9w2F4b3@FSv1|6-2{N7(CukFVXi4eUd)iOQ zSg)hIVc;I2WaB_BvN11xdRPQvCBZlDL@*U}Ni-DK>S z<&(n-;S20~9h7N<6f$#<;EsLb4IgfoO;+8}yhQdiAXw5=QupvyrH4TL)Q&N^ZY}Jh zdMj@k6uic@z$i@k9(a;V4($z4GLElWnf@V~PBQ7A z3fo{qa*37cF(=sw&@d3J^^J!{&%h9{LIGyk$&rbk5ay1O^Fqx2UP>w>2rY_Ky}iqt z66m5ZNici>VWCnl8Z3r4H}L2ck=#{bi_K|b{3 z2MkX;5NIo-mz0Ny`j>cYoAEDT74n40t6L((V$sDT_w*w45e z%%)9;Ly8q#?8=I z2qIY8fxuXoFTd#CShMW0<>raN$V&2h{WbF@$Y&Q3zZ}*>OsCjx_w=*O8p643+B(Lk zB;OzBxO8pdW)fAZ(kpSTZvio4iXDjV#SF8eqkszfeh&+zP(fQOJ`XDRvdfd1pOjq! zG$+-%XQIr%@}wCuPBrP;Z?29)C2GS6YX?ni?Me&+FB2n}kX)BkXm>Vl4)mOQlo@_j-5I4-fV7PX!sR=Kyl{%Um9uDK6wcSrQ3Dd%s^FwkS1b9mpnXqsUuC-S7}L2izL6B$2X=&9Yafxe$Q9ytYC%Ixn+86*VdvF zB12sUlXku63a-XB`07T(634Ko`fi}6`5Ut>%7hFXhecW|>$~(|-w~NQTpss&?GAw# zfI7iHAQ=HE(Bne2XIQb(G%g}1?4?E`_1f!dofA!^9I#3#Z+vnzC&O47+yXByyReC%jajMD2 zSlRomtREv`-_ZSH&vgCOB5_^8t`cA$(Zc}Up4LNEulD+9sDj{t1`>U&US(c(PF`A9 zprA?3ktNET3a$MIgCR@g-H2?mrWyPeFx_xC#ii1>(RgmhDX7wdPF4z!{rU%3NCk6@ zp*a_~^!*ylJu`6!HOlZ9+FvWNv#3ecL#(#ie4}{F^0dy`Z z+nm@@e7ohy3@?b9&X@yu;z?>W5v4Fm5riK6T?=0hau#%w*t&c`Nm!yxvX1~`=#0?xQb$RO|1U* zP>%U`^zC}CtW4Z~sA&z83+9*gYPu2_p5it9`gDCfU-dSLbibNns`FYKPD9u9nrh}93fASz9~Y8b|27?DmR>}2Q=2?y7&N0SY!Vz~wpiY8x-pwf zwl7Z52U}!zVt0}!{B%+?OwVk^hptNSFy>mgNTgN+4TxOKvkFzY6w z%Uf@w-de9rGGEN;njKhdPd(MO2v!zbX4rx_f!o>2g!3P1=4##x+dt}Bru&WvM zGm{)%ISWS3qRqS)!F1*Xupeca@^YkTa zLe&q{iIGA#qNZ!cQNa=Sl{LS8f4zcYi)0a1ILOGem7oD5#uxX)&s2wK~j?HhBON4Ur57Z5K8C%9VA(pA-Mlt9JPlo3|n4>LUw|Z4g-3mPm zTpbwEE)@B&qLi$P-?yRc>)3-b+ts*2N5Yr`*O&b9DuD5nDe_OFvqV+W*A@ z0*i;w_nY7ER!hcI%2jHR7gHTlHe7Wld9HmwQ~Fv-H>shuu{%3kEOfZ-V~k_(_bZ)x zLi~pX?n`p_{fsOEW7HyU2c!gvkJp?*SJV-M=;sF~8#~cQA;I!3&Zk?qGfds4zo_Do zdp>>g<{A3n{w-R?&C5H!_b$`9z_;24w_iZ0IFp(l4etSQ+5xa>#E;=2ybkp;iTpjFUn$a)E;5D)2f2Ds}5N3gGF?+#s>)~CIClwfyKCSD=1roF%#Q!f>Tf0lQ15Q^i0!zX%#7n*|!n@(3lC(KtL9$V9-^nHPDNfVP={ zB$U)&Egbqvg)mA`4g*#HKE1ySu4>_y9DC4TyfEKbBVGJCK)2ZH>+{Un8mjLra9;Y# z%hhX|?l0c^ZbYNtx%9#EFX6xPt+?FNy$-=OU>M>y{K7O~H##eJJ9nqExZ=4!97T#- zGc4kC2yM7M9DN&6#N*7P_@?y~4ZS!Q%X#jVKpC{I-6P5>=p%>G#xo(6#J1y2gV?xl z5>AJMd1dX{Ym*B^a$ldn#i{?b(`E*?q)@Jz$DCDy+t){g&K3kQf)SQ(3ZEHl9Px#%%|(|IEE`LJNU5U{FnKC*gtkt(aul>$12>azud)zuSl^ud7=Mt3 z>R)DrrM4Wy*8R&!8EI-(FfE0pa9<3fnjh~c*(fa0uSe$J zq`)@ERvzlF7}g??oO?S=Z$fy_SxeAG;EMC|dMJuCCbmVU4g)SmRgze_r4x)*AT&)L%F;2X-p)j3UubZ_9ug{Qz^H&V zhlh1031|j4KvcbeMpKm7Z%nup4&|KBVI_v4aN{9{Xi^pbXi`O9LLmslPzZb^wrJEU znIYkrDjyyCGTkl_e;57 z{St;S)IOV&l=Too2*9u3;9509k$C2~OFtGEGYQq;ldFKs^8jIPAexLH#G1%R!+fp& zk(^WM^DlE zPx}NK6j&A(lqr7qra5AB=U8BUNKdNVrC1O0rwVJ2yIkwq!BLU*9~S1(KD&%>Xv?ky zFGl|QVA^onvh@lm^LD=QBhuwN$-Zf4#%%@CNCTh;tAoj8=7Nu!bxxd;7tPCbh{DeO zfC_crbLlRY^K14V;;96A&(GD^fd(r>+n50#AKZ1d?%(W}wQpXYna^ydi_>ok#;A~^ zk$$=*rgHJGTW*{D+E=<_-50Tf8?$SmhQ~Yw_|W0G0*757v3^2ZBwmjE0 z*7y>8n6=?>DIL&Z2}y_wx$JMJGTiGG_QTt%$|$8Ts^&l3@IoB0&0sQDKilm$vmFrb z^`o9(sC?*1k?;_BP1MYeIFv0jEmaL@CQ85J8R5WuxjnQuq(!X>i>FhmVU`U^e!k3V9EijJOVT=maKRNOBU z{}x*|wWw6)MbkL+oAn55_WE<)v!^M=Ugzx8`ek;ploF3;Pc;Jr+4S76e)pNtr%K;O zs6G1n-8VE-A`|WH169i|s1|O+I9-dZ^lBrTu8Dr#p)#SDQso@|RaWV`i9W*gnM4K$ z(SzFL-d1+kDg2pZ{)>RxUk0&Fud27^M0^G*?Lc4Xe4L&mG{>6_y`ZDZ)ycjX4c#EF zo!ECnQ!A7ODDwK`8pv^nPi`exb&|tD|0QT>&Q%nVXD{j7VcXG7__cb7W6FWtXVJg{ zUFwRf@yh1dcAQpPv$mAbWRgaMdCIJ8z^p%(u5!0>R#i5sHw0|dz~;A4aOHCfbuI&V zbanD5%3i{GYdQ?DIu6`-!FoLpv(lKhoSTWdv0 z{oCwJSU$^F*EiuLzb=cKihH~@Juc{+bmtZQ5KDhpx=;V+*q>M$u;oZ{3czJw(dO7` z3Z1&(cqJd*lwyB*QuXMzQoTfsVzN_G^WDkomwY!3mpaJ)Bfi19_M?3|%`$T~&t15Zho`7uDExBh|8 z`8;vTqWrrM`p*XUy5?R)(#_>(6Y92-b$~ez5muK0tqMy&jme!IrER*BmB&^n!~-bM z)`qfi`ac`Y88&^bWhXeRb1K%9wj0a5N@G?z@ng6G9e&!VO`d?5#)q>Ik$CUh%1YAx zdUmHe9DRm<&OGY+G5Fu770i`1TsOwH)Kg(QrO*>%TvfBQC7}g~Y<=tdbHs8X#GwhD zVw--JO}&wRC^-6;80v-67u*(e?559ZRPoce3|DepT#1Z_F1q&o938mY9^#qTL*{Wz zUhqW9qUCm)Js?HbtGr3w9paHFhh#^}?q^pI)6|ACv^&orQ}|a0ZAia?D1G*Fzsb_0GjYEpII%z3 z{i4VH_S%s&Yxf&2yInCv%$8TwXkyuLqFB5;nk7ApK=#PxtG4gU9UyzD1X}2Uudaup zb%(u_<^>fKfuHkZf1Cnkx*A}8ndM|emjdH3?hZKFBAR0I$q$lgVH~9sPc3Q_tGzRA zenR>>|H)D-;GWOlx-y{8r>`K~T2c=Z9*K0-FfVqr|SZQ$~Sd{UO<-Tv%n3SG~S(`U{^*BeGrF#Ts!8pERi&x*1N@28Zdz(oPn(mDQ{pJf=Sd z(I&(lK_|t8FkYW*;O%ZpFNCfRu=bgSF~U(5THv@=BTVaZaa5kJIBYRfPQ$rHm!!Ga zmqt`7FFf9_40&&NFu}Za{BZYc-yp23+qL781@q3wfn9)*W<3QEHk>CA*~|jvBtjUR zIG)mTC!hm00|7}na5fV_nZc#z0&e3i>A+xcm0WY7e7Sagd0M3(tH3ig|DiJknvpqn>6H_FXezlS8QHiPH)Uwxbs!<2@grBH4O$H&ly%z2!AUNq( zrbsd}HZB2jzy+|3jZ=k9AZ8S~$uAqk60L*7$@c*AsJ9ajkvJi_g&Xo@@(d~jNz!WQ zvX`t6cd=CJ863N7QUdM(y64)2!)mdu#%QCI4WoLTmIa9s=4!oNW&UuK<2RR1UEAL~ zVO*m?=&Z~%@4gHdZCQg6&4kx<68_7fwvL8vpg1MKZk4$Bv#Lqd*SCB9Wy06O1GCm} zvu3y8ww7Dzn7wV`(#DH}B$0R+UBPnvJ--u2_NnN^(#7AS61wv&k{Aa!B^`SOt;06% zn4VFS6(!nm%e|kRpJOi?Zd(a4oLaB5Y=ljV+r?Z+I4*wG_nbt) zJ@&x9IGrYyHR2lN=y!0l8ABX-fL7|4@YJ#SH!D=}Bdutr8OT3@Si}g@-r-9<(C8L8 z?l1fXGL0w3r$|R^*T!?G!qz#VVu)J@nmz<`xA9T=(=?vxY)=#<%zNBk zYdgGC!%-qUOavnF-yk#xGVUEZ{?FvO5d@SFA+*C`59w?#Ws%t#H6*d!Ep9_(s7#MT znuVy?z`AN!?Ze6^gEO~-^a$4e362%->ziQ~K0I66>LYk1yCJmNuK>Q9)_b(Re_0=I zUTH?hotH|V%lCkijZ9>Tgd_>e7P7oAorVEf0DSO2)A;E$8Ou@F@@%+u3ujhyr!ZoP}0BOVmesW z7rH=RoX_UbBx!bB(>2vLiNHc-#xUe?3iB^?k)U9~RcNlQk* zuCDcNc5SuK9AF#-ap)Lo?iJlOwM9I*$qG*>$vr&e`KDB}7H5z5OF*C?xK>R>4sBb6mQxhRXG|Eqc=`T z>h(iW%B2Dw0JX&eff;h5e6mwTrsvPs@#(d91uoyv8rR+Tbr7Itj@SwheO(4bsMEgK z7BBocG1aX+lRj+aOcZH0X~j)|MG>wDd>ma%U=oW)fcX~-IN6yVr2!gqf1~w%*uKmk zp*#zU@9h=%(=S1vg)53M#=bG+Y-Ajj+KmZenHkVdVJBM6qNm)Gx^EaF4VTcMCiH=q zq$)b@@)1l-&a!iZh#G$U$eRDt0R=RDJY&KrTL)l+Q^p#2NPtFVB zl$Fi9oU8m^AAQ$HwuN?cWm@V#UUmcpN9y%k4mJY&rxSCqU=a?I17j#g0S>y*ta zl;fjhLIH(k35Eg2FdfLqUz)ZCSY~m}8`?SCRus zIBhk@>b8Of1mw#uq2`4jr)+`5_leF3Sc6w;=y^Gy-N;<)n(7%g!F4;Dbo)e?;n)*A z*Ds@s&s!&eLg$6Q>M`EzJ3bPE_eB~FI}N9$K3l~Brr#eO4RK#mk;T8!=3AO1vqm5rfPC7;B&o{ zU^VT&3!X3@=LSb*yj0k(hsGT$vohp3+_HA)w;!5;=VHm@4UMR4KCiH(w(mNK=sz_q zhh&`^UP8q1)DbRy*OfPbzbnx!oyVn&5bjl!pQ;Kfk_~1g64GH2TacPo z-f`?@Vm>6DwLVe)?JF{AJA3^sD1~dNwlg!?IJ!EQ8LZMv_k@tzbiTe>cw>+ zF6))W?D=Y++0nz>p^Ial>B<*Ywab_$em`j7ulR6>RYSD=lso585K%5_&8H@Bn3fpT zoO;$6t|?@}1qbc|KS63vLBiaH^PPqF6;Kn0S!>(3CCmtgzA^`yMHRITj_(s{6eI%VjUd;c!tv@;S=HO`s?*ZJU;Y@DBWykN? z7jf}|GqC+JOILB$1TS8jQ33TS5n7u+3L(hLy6pJDe`7(JHL!p)Lz~x8?tB|9|gB_w619NVSoQ5vW?e?X~+iJ3?LD^ zBS93wEeX?)5wP@>?cckful)wUF!RzdQ9FpH*2uI7n1?xpxA1tnx<%AcSb6=+o@^ zpFoI? zFDU5%Fyk`}=C1iGSuOPPbww9dzjKVhZzjdhCsh0;{3X|<3fk(Q&fph(TIg}CM>jK! zBXsW(sksz&aq%7~52Y*k5mYuT5hNHv>s0*=MF-h`j*}`O=$s7zbNLv-HgySdT1WjhDT?F)0 z31HH8fpdh#`FPofU>K|I+e~oY@&xbMLM~>dcF>f!0+KaLvB03@C}cGp%Al9wx9J3! zsy^0Q)M8j`8hsKjytY{Y9R&8xZN`;U+#n4gG`qyp2KbkN))K+wz;mF8#B~bv9k}uA z*#b?CaO=6z(G)b6oq$Mu2BZ||WSh!-j}T7ong{k0eJmRU$uy1AJVP=Ip>POQS=ZV6 z9>RN9gf(V%C>#k(9;@n-&j27n`Ni+Q&ivAth@<%qOwRhRFNdz$cY@qyx@|e zP3`3qynmVf{iWgu_fZLs_ZfBa9t6YdFs>+sILP0ITc6V6kf2prFM2`3;)C@Xm;Hvx zuOGI^+ftb=Qwet+j)y#MMkNhaSnJmdEaE0BCF7LaZMT{=`i>5>wpI~AlsVhL%MT>3rt zy6*da-kIl_ar{Te!TtUAJil=qpMx2HuD6>$Vi7U1HdE&|Az!&n>Qz`&p~1&W|Izte z_C)g{S?QSEiFrIB1&*YzAiaSK<-RN<13)1Wxrquv;qc$PLtb;*2$*WEW_y)Li6fYM zzd4kgn~_BE@dVty-Q=_zh2KO7uCPDVM^pCGz<6zs9blo&VjhhK+ej?0Q#>oxespb) zEZAlJL~Ew)%}Mp;-gU*Hy(*pmy8hm4ay+r&it}BC`=Oqnfqx;YIMk(LHPaWrZLG(HUJ{>+6a9y8yjiwCap1N>sjnr>JkHRc5+YdTzljvO+|#tQn`e zlgU1~r@?rFGqR2rhS|zPbuu9#9!yZ>)x!NFwFbc+ESN(J>Nw2p^}W@ zo(aUDvDbw$NQ7%pHc!4X^HZ}&w{*A7e+*-@6Re*G(14rjfxnD2{xM)6aNwD>RU#u^ zBe|V^Bz(x#4^v4~|NMz0IFa5;E9zUk8u;^N=Ozo=jKEXrmk1I?gg3g0-Zo3vePe6@ zpPbD>NC>T22+H#EqPq}8HGoD~E!Q(P?mM-!`TJxFOQ5-{rU_uU64hov4sXFA8z6;*t@_a+n$d6Ht;5bVKsE9WWZp6a$~!;WFc=$Z&=BsN1H}&rUG|rr@VdLMuY91o0?i!Pfa`3t+8rq~^UYeAwv4^DA?{S>eIR13 zx!{m^8%Z5rpY#eGx5QI%OE>-h8J0e9& zG=dhZ1%1%;*Iwz)mJY5f^{!CdN`}{;8B$`wLqGQRSq%Nehl2lB(64an?tJ^!Zp?Jz zc+UQq!oZt^#rd^L@Zy$*3{6+QwD}D>b&T$SwNS`3i9b|sr=ijh^kRr=&f58C7XPQ4 zbHxEam+~Uclqi0M(v#@Wg(_X0ivyGka@-F9IUdN5vro;!_Xj)y3`Mm79aJD^9n5^h zoc}v@!Ze8VrhUbj&Da%4LEnj&_rLurr>vEzbHgj;oBMVIf2Igh@%ted%dklsqGbdp zoH_AvM+=P6k8>N^Ja4mSh4SNAQr>^>;!oNPGX-3$Xs8*Mw$>KGxsuw@Nn?L&d`V(P zw25&3XOi$ck*CvB6!V&07YWNr1xRG8lY^bO@>Z@8p^Nt7(jMqbs>zkkMZ8~%Nej-@ zF1Q?Er)gJbO;;o(&RP{G9S&eXQ2FC%sxRD4&v*bPu2|#WKRN534?8x>jx)4M35ij@ zd0l8OBJfkuL$2()c8c9O9y&eVDnnZY%7p5h$z{-t6kzworF{c6`c#<>XJW3oKvCIS zP7ohMTX`NWb+CZHed2rY*DTGkI%UD(EZHKK=v9B<;ZO%XbW@*n@!T|sdy8WMln$MY zF=G4zn$1{}%Isn@ltzH_oIR~ogHTIGc9pF*n-Z_%9 z#{5jbBg53+JQ&5o30r^#ntUBn%Nqz&exct+B__khLO&ScSUt(-jn4vrRt`@`3z+LG zX)O0n+Q`pL{K5NIJkkAc;(-SD^*Nk*z&nXy<%8aWW6pQ{uPSvuKTL66ZVy5=T^^Rs zf0U}eVW}SiZbCA--SMA^UjX^s`vytgb_1s?V8geRdSi4@0}?YVi{6Q8+K7&&&KTax zbB#B@)adc7j+0+ow0MK|g}Id%eR*+4P|8N;>@J}F^GeMP&|V6=?_yY7Xlq{;$41#=?}}C%F7J6vX2s$KcU8s zqCm6@0r}UFMjB7x)STQ3!G8Yu#rpZ)lezgo=~V0(rgM$NoOZ%|t;K^XT0`0YcJ6&m&Ld2DU-D@rV|gWtPvS$m? zE*MBEH69#@Ow(^@B;v@FKUqo)ox?3k5OG|k7jo*w91=B|jHYvr>5l%*O4F)=&gFkK z*0(taGp+7|^*;etw0Hh7ui9y4?-S2D{z`W;^|w!~4=kl&1B6z>GZ^?CDKAJARZXiH7DLvW7DTM}U!<-8kpZxmNPBo=<;cveG)IAByvcbF18h>y_4hh!f*<5ZyG*RV?B2 z1+NQmlR6SUF8`3BRn(FG8o3;6tl#6EP7(Iu*^D9_s!OO1JM0}m5%lLM!s9EAJQ9!Q zZk}wVnkA%KJjP}E6yNk`{OVbc^y79ugrmzwk7%vt=Ql9HZi=H;1=CL^M1AkdlpLpH zfexm{&K67S`y(4P7n}o4CuZesIxu@U6mtr=duD|CSAbx3BkT&yNlQbnYpUVCt{%l4 zMx##~31r>>-uEO{v}8HW1$0{K>z4kw=LmIx^T_{kx?b2hz2JQ&p#&jilb@J)1x~=w zL^OaF^*dY57+YCB>6rjmjd z8i+!LdqG_)w_6_CvqQw6ZaUMzMQ1U^hCd7jqebP#NO4E6SA@!2Qy;EgYzT8N^9@*k z{1;$x-w%k#m0v3hKAP4lyK@C9Zwtt&13-5L-qRpP$SUeP%n<>|KwvF&I8qD0a2)WWi9*e5)-C?DB<-H)-RG-ct9^Y8yQ?;)D=E>o2=!sX; zp@eZv)fyp&FYPi(OOd09bag=A>4&m*$gI?8n*Wjg7iWqAWkTVvEp7Ptpp65kr0fO^Ub)fFcl%!9~Z>ip@bso>-^(@-5P zu2h6>73Fvrf_ANLe6-#4rY^maxqNF?Npi6W5Zxcq2Ohs-NGiEr)nSwBkR>y839dZ* z8GGa&H}Tv8WR>F~&08{^fZRhuEpT*r&ui7c^MrZ8j~PQS7fY8ofkFWCH>23yra>gY zRg_mg+w@I*-1&Ss_pVu6C?3v;GKtXhzjw_yPUeQf@0Zhln0$uo zO}FGj1f%{av8L^<_1(y0t)4L`IL}IcU0cGM&2Dzh{Nqwy;$m{GQ|&@%XivVebr7kP z$a!d6~x+)d#l zQQ_~MjKdhGx4qG}ZOgdwW}T-8kPn;XQX{nK0$;&q70ZP)nhmj1q09_x9w$TqGma2IFbQGoRyy8&aM zR_QL7drq+nctkdfF|;Lz_$%Lr6pqu&aUDy4994jCvlPT)>&aUJ^FMDER+ghIJX-0Q zTZuNRof}WAR0QV@SOgRIs>u7wpBP!hNZyRf6%n4J)0=~P%(w>QHBQuk)qv@}Ok;fJ zBJJHU)M*xMFqSaeB7#+8lhy+;rVKfWmm;qQ$>DF7_vFmZpH2aw42ls4952{Uu(t9Q?iGa;JV3WfH^w*)IPu}fb zx|o3a%uQH2wLP8o`p+#TBkt5ic;~r%s+?p;g*ft$nW40#muvLlMsohsC)yWLL#6ro z)p34chZ6sQgGx;-QtSCCd^c$@#OlhV4+=+zEClZ@gt1F0>Xkk81yIiz=6QJAQPSYD zk=O)b*FE+)_%alPDk`ex3tyck9^VLDORRxP;eJUAV@o*|wGRMuQ_#(md!4)V#}%I^ z_y0&#B8Dq{_xW&nePqiR-N~0dNsA>qEfhVkBqMrnuN@H5jV%%&qcmcE$V1ce_nPv1 zXAj;piJAvzvF4PU-GEyWce zAkxVd1yl`@Gv<|*3R`x z4EC#htRsT&PPtKDD^*040?mvt(98g#W@_3sB{~bqOVG+&-QQK&0WkXb$J7h;$&L&Z zd9Ah8TtEA9%Ww}?=YIHlDHY~?3(iX@%h&lq(S0bQ-_3%(Fq(D55Dp1!*V+)?*E($_ zFOypvT%s`UK)5sk7G+N15)jT1ONVkNbO+m>AzRQ_)fv+8`A;S-Z>p4m7%20bz9Kq#-Ru2Q6co(Eh6ThGI&HP40HXh4;hOl~oaL^v%aY zPdZh~i8S5jWb45ab0Ab>pEfWRyM=XII;?8bxSi@KxselAZr<5OPv^YSO3b-^U1XC4 zD&M#pbU(f9{lPI0B`>@iKK<~1QCfrZ!F+MYHHizzlk^$Hn=cu>8fAo+s!j6@_$o$-F3smDP zSpt|VVbj%V6yP#`W4(SQi;^S#73>=!vU~K+JcMC= z_G(S2gp|Lwg*}4`bS6{%m{b{5G+=qSmpS?O|*R zZy5A-I^>sJ6Gx`x_p;J>_t2dCzxAHtX?ahx=Jhyr_@ATLohrCsY&nJntDgAI(l6~A zQoFjmE~6*H3IE0(e*GLPR`kLJKh6b(^aHG4C$Vs#agv?*?jK6d?$mgT1_@@^C!J|G z0Y-T()7{~}vjEn%H+TEVYtnZhEg`nm;g7+QoLyeZp9x~5_B*tN$8_I^1w!%KyqT&m zT_G$1baEwjndIqRk5Lb5)d`jRE%Ld>Yy^W2Hk81^YsH@pvmXsc$7?cYS_#7<+IRF2 ze$*IDagsUTu7Or&#qg8tgfZEfFVF27JB~UobcP#w>3HMwofMC|YpEVhksNO$CON}c z7Ok-e_GfrttX^>K#ml^~Yvj3?K?=_G%37^rIF#WM%Gz0qes6Oh$rqusy%r<*txl|z zUG9Mi3ODs@!(Sj2gS7`Wo}l;SUWc-JGuCBu<%BVC-3PB^taJ(G@eNcv`WPtml9m(s zi=wjl|9omOI+W2{EMqf1zoH1Qzuq-AFLRLT0w@rKhE zr^s(`D3eYidX*-TaslKaodGltG)NO_%+4X~<|O0(19B$W3U=eL59s z@w{K=d3#}o`ym>ale+u z;n6{wDeM>7aPxyWf+dMyN*HI8{XQ5^{{WFDnxgOAb-2Orh5%2I&0>+|R}=~35-k-) zHZfY^`6DQTwf4JDe~We!Q|S>J9m3_6&g}RtehakcPDoDW?~IqmzZa3ioE+aBBO_2z zU*Dr*x`RU)ar8yc3O!%?-*aFoHL>`5bTUt(4O!-#z=EhF+7>(M?Zs-|ItOACTE2G} zIv3b~L$Rb3Q5oU_@g*7aw>~2&?_&>A^xuNSW2x3<^_|ykWD_%dh(lYB-!sy#E`{Ch z@WKiAI={W6zM~FIjl;RX)9D)qS>vhx%KA;VJ(&3_Ym2bql9y)_aLc^3Ykh%HjAkXL z=OU2&M%OKVC-yqF=E$wWiBlL#=g;$h(5_~;H|yUd8(FwUTka=XV56v>IpU1crWPo? zITXghqkz1QSL3Q8rc75u{6Bn>L-PBVGDjGTIkL00C8~TYg!ulp`@IrqOIAQ)3F+$0 z(!}qk$E{1hnR-O$(DJPW&QNS6XCTR zCr_~Y)uVCfnn~nQ2Z3`3Sp#rmb*>1Av*J>iQ#_jq;N|b@jAG=xz|sY05aH71(Ef<# zbiao%9|UXe7b=^%#vfpw_R149gbavVUyxd{3s%{BU!EjVpBgLRtk;zg(Fm-prWOU~ znYce6qrg31HaJ0kI7xg6Sp!o+o-;uCAIRa?N`4nXZMMd0th75nNX4Wyr;Qym64SO$ zRoPoZE==#HyC*8oVQ89-TK!xOlLF9m_lle$B|!`ivjoSLyWfEMKoB(IoELiVGJl@L z?7(<4;nq}ph#h-Y4l2Zjo*yYxJO7q4uuIxl?Js&YkfF71VU<4N61z+rrMN`_HKYm~ z?FW~O1?J}BKqLmQb$rCoI>WM&*EAH)1*6=K{Qy7LC><(YL*{(D zeZ6O3Ux@C)=M2aAGplt{$f6F8v%DlrD^E*bv>^2DwSt}`- zzlxH=Z_YA_9{&yI6Zy?bj~mXPl(kMleKotOl}=97TOQ7pc{`b^<({Op?Lt}+9KDrl zUy`B=f;W6tC5v))4MW~_$cTr$h@7wK-AQG}J%6z9bqbo*17CzX{W!VucymGivQ%Uq;014qm+D(MvBo>W@F4t8wsO&txxWSU!oD8H&2c zOrvw_V?H6IAk_&coV-DZw2%cZ5YmJ>i!JnF5dLix8g_hxOr{wQ>)oQhy4OfctoYtL zz`BQ3Ukj%H1ju+lV$-?AUp3hcD(z9$vlU0U$y#ByFq#*weSlzCOXxdDtYe5nlIJ-3){t2nrb!*#?2RYc@L zE#lZlDlLID3Xzv#zn7N-mQh>T>HF(IOe|WI)OxhwndNs>u3Q>erfpQ6+*xn&0OB@s zvN)>9PGRRq@<>sswr4pi-HnDs47tPw-plZJo3YHWq#ByCPMSy6fj zR@V`8z2457iC6En^Xd)1sL*uB(rr!L z_1J6rq(!$o3?*Z)fj-+BVB#CFfe@Xt{{G61k^#`zrz}MYfNXE%T*H!R!bZ_14_`O+ zq0S?j*5tS6iqc zNVj5Jz~{$M8p6KT;+?Z_%MWIZ#)G7>;%F02dh)>(up%F#KvT%fAYLI(EU~MXn}s64 zao~2>MTkSWB7Ga^IK=nP-&AQ?;JqFsqnt~B{l}(ihyXv}?;5*BY!_De1DVZ$@AXJ| z@0tb)9nJ=HwF>p;6!T*o{GAHhS{Q>D?ZVl={4S}@L z)M;$6TD=4w!H7)x(XnqgXBe*Y4b=5>tO&YD)rF(mc(yDZoWCA73whX<2F}8UxV2no z9fE}fAI&t>%8-5X5x`e|1{rKDQzja509T`@)PwmJnQ$3@@B6dWF9a7cBAzSZyDBmu zD=0cwra^X=-Vz*X8QJt~R^>RvV03Ug1@j8ABC zMBH^~S+_(lb<0~mhchrtpO!hE;e-6*u;KJJ)#+d$6np`;(7|SEUF9q1uVZC?$%E`P z(!vKdSw>yPMxUOPGrHOQ(Ekf5@KF@O5AFmpU7u{M*VF04b<))jQ257i&MbRuN2x0Q zF+SGKO4o-Aj=z#WEYZYG7I z0NZd|nExITpqzwOg`YJ`Y!vBTJo{!pKIk2gQ^H=BV7eI&B~%IqS%i$rw3Z0J<6>6` zMY%&)>5UsmCr*)In|Bz*%!9+q<)IDLq(q;~p@E3`2 zU^Ie{dehEmo6KIbHCmW7>pAUsG*};DGk1{w- zm**-uEjStVs>6rxF=}@sp+=JPVdwjIy74CPYL?PGcR%q+hYGuG?I?@pdVZRO* z4Y6e)l%xleNgrvmdq4TXJC*wjfp9>h#J9#(Msjf_H`@Z^r}Tj zpB>aUfvJI};ViPc0(RE?X`rgptlM%=xqW=}luT z^ZuWy?6t7YpuaKn4#Un($O)BgtEQ`jAMPcZasDe#OYbj5Z4(qcRwX@;0E$7B`C{P7 z1l&;W8;r1MwnUI@a{8D zIMm~2u};cb453WMym)St>ie;S*#7V6DHi3^6#@0FOH6cDluoRh1L{?ueP!i| zHuKwBW#@Zx7bIUQ|Fqs|v_e7sI{Dz2F#SF#;{0xc^hLDzG?%7V%n~mzMFRd#W9+QN z!6Pi2LLB(VPSkz(*TOa$Bfg3U&$H zP4rroEC^_BmiGx_SpZ3cVt*cX4f7YtVRwBEIyXOt_*6b8=fh0*9Nw?rtmwnnM1A$2 zvi-uSxmt#tUy$3}2TlS}q>GuP3actRvVPQNeM&ooa9)B*eV#?L$gYf|jVjI}rZ!T= zuY|g&r>wf=PMm!nm`~C&Jvf=g=28J+i-nMoj=%Qu@r;auL@UfG{j*M53jBD~I-V5^ z2a0#+jF=8(b++DW$jhiW)bIqJ0&Z;zqr1+uHh=t2dbBPHN)_k{@1#V_Q{pD~#m=nI4t<8yJFW+%9MPGV6}tfQ8EIme^wC zj<(hFo~8!eyo8G?OK7-8$;49Uw5!Qf4Izz+X=^Z1C;7Cz@xrkOSl>w?ey*fMt%d{r z=^(g)nW1b?uL+PNeqDil6es(l@=&bgsVs8gI@3b9urmvU`{(>M z@BrmsTX0rRm`53c)J5H^Yf&NXx|%|k%ucmt^O!VV7r~T8M}vmzak+Z^^}y(ien;h+ zA6GT3qs~f8=uYTjmIb-VYe>ZF{$-Pfx;cTMkO^*XR^CPRZYwM?x)slhEimdF6tt|V(RFCNsykY&0DRx6jE^d663mZd>%9?;IY5L$`YU1kN zy-6M1WMCU-AQZt;5cKhoi!t?%IWh78$h94Pqp~+?yAh$EnT7uSa{OWN$FlUAQK9nT zR5P(kkC1F&em)UjG5edUcNA0A{ArSW4q-WdId|zf&if@k8u!nobX$Mr!#yU+mN=al z*&4f{D0Yv|ICf{;#nC@mr}O}~YkElVi3(wJ&#OFOl|U0gv#Jh9oUbm6%WIm>4v?pO z03nl)g#nCuO+kw3AH?|kNB0L*vwaBNK?C8#mu8U%g=3!NxR2dijmaT-+%IvKo?-`C zI6qlsKSH%TZx$pT^?*$Zh2!F0TSJ#qAcmaY@^1Ct3@_}Qa{2A8O1!u8%4R+`%9mi(m4 zx&H>lA>QD=7BL+1vtUOSR6=JM+k?Z0_fh}mQ+$;Tv~t%y-}=J-y4L&wxD(HZhspK@ zlh=Z}pHeqf>3m@|g|NgwA;z*kc<(hJb~<2&eY>p)hE?DjwXPicaXLEcM=wO@-h$)r&q#1J{erP)raSNZ)d_Z zf*omOQNn74<)=}6uuR&GJZf0xl)Vz?Isy0*0>(GDKEq7wG;ry8r16-UprJ3gZ z=nv_~ZT1DN6|sPvg~6btz4yc9PAn7!8n%T}haiS$Cj4`=sGeQiX^3&OLucILe2gHb&8V;DizS$gJm;HPjh+sc&oH z8?(eI!TsE4PGpK}|1>#Vu#yu`)%#=NK(pRQW}eFtzLN?`84GlH=*;PDYhO~P7svoQ z_Z;jal;n5mWVFT=PG19RU^CIPDf`k<_)Xin1(2zY<6nZ6)@?qk_t>C&PtGFqlT2t05;^wHmaK_}a} zdg16ks+hm4AJn(a6KN5rh_?%8J;Jry{Xq_!#oSu})q;Or#|9j!T%#+mYAXkR?3%|n z)V6<@W3f><-AsSc?)uTgI?&Q|z)iA*W`gT*LeeBmx4 zf&hlrVUcRH%xXHBa|9^dDxb}NUIS|Q3eZ0N+CBabq*MRI6*g!QuFfNZuz7@lcKB4u zcnu$$!S{eHC-1*a@W7+&_&@S5*3oGg%jX=a{^fVoyg-5J-x(=yrmk69UBW4bHI#9X zl-~xT(fuF%p79s!6TXwLTYH%u95)U&vL4q{BFkspPEwro`C?y;3M(P99*0p1Mzz+~ zUx2!7b8Mersw2#iIrz&?-JlTJqA{!NR}RsyKny4~dyI=U6+Ngo@xN5Syun(HTwmnx zi;+TcnbMlAv;2^fE#!)rW)KKQ(H6;HSoT;PIDC?dQY@<60nh~kq|g<;-lC7ME-!-U z8R|j=GtL&H{*v=L)h&X&1!X-ycBPnC|HbrY_Yi4e7Ey$OlPj4@33qwW`{X^1k9I!> zH+_2c`cu=OK+zx{^=N%C07SBZNLI#F-s+`U3Fi>= zFtJ)@k^H14B(xV7!IJWAGRJ z)G!61;`SHJ7yiIzHU*q9Yy$_n7pI5&uKkLaGN%_GUOZs*&;&pCESoIpzrhw9TeuR9 z)to;ao&BI|;_79XVY3rGe`M@n|ptcHC%eP6glT?!UC z0(>YK)9ljOznVeVsf!)TG;%c;% zVByR;99EQHqG&nSZs#Ug_7zaY9RIT>I-gOzw&lP1^XT3Slkc18-6 z*kIL!uaCgp`;dn<2^7EgDk@M#pDXU%{=AM`P`(LK)*B^H^8{ZtK=)<(Cml-SX5cXt zAtsgt07#*>Ra4OI8PfqUHzj!l$({E}muZAsc>oo!NGyqU8IH$M12{=rD9wcqg z#rh;Y6;E7^zx73?C=y4&0f@6@PtM@?-n3bS zF&GsvtD6FGU=;`-o*|R{n;ktLybC0a%pEVBry3HU$o2qrB_FfAciDoyEpR6d_Pxz@ zMdYRcI?Yg|Ha35fPv;JuwqXiTl~={G${7KQO3*WX>sJ;;+H4fO)2RVR!vEoEhERw5 zmO13{DmW|byC9oI0YPpnc7dP3SxwG)E@tZhF}oG4*oPY^;jDqu_PDzYUdo@^_~;Wk zuW}jni3esISs_S_@5WsI6*@*jk~W4@6(Fee;`x65+hHnA)=OIM%=XIH!XeQ5t6n!y^3T1?<-j3Txpt)r%vTaD7IVNNUA}(`gdv+iPEZYU z%bR!zhOlYV7hqvu?l&%52|f*Mc)Bl8mTUnT0CBb7=|TRXi0+T3BLGap+TXztGZ*BS z+u_9=0%uu&fi$2vSuLM1g6-P(g*e#%k1b))6gkXG#US?jQ68@n4TTt^zz ze^uM&X!%vtx3h~Hht(3lBc-0Hsw_U;A#K50Ooh|%A77XWkHXNWDD`1_)`kxmmB*pj zUEocfmEOcl*66r;2A@ZxX#Cy#*ruqdx(G0w0?icGxmphBmj?~cbO zOFVCY{oB>2ba3O|_UueMj;(bq+Jdb}N@<4G&3xiO%yqYT_KpcmP1YAvmywGB&)a(X zHGP2_1Y{Dbe1d*61Q(zg;iX{41~nT4<-P;hx>R@284H%hdP}biyuM4^3#0l0Vksd> zcV9YSmD%8>oO^O_ndS^T@Go$r%L1=(#!p0Se*0gjac8O5CZzi3-#C)^OOUukD#taDsIbsN)&z%(y zpCx~*s+pArT%~uOYm^r^HF<{Xa5NWR+0#y!-B^bdo!bU1rBk{Co%%aWc_RK0M zCIACMW)f!)bn{rG*!0L#C007?YWem|#OYefH)b=UQPD0I3t%gb4T;731*|O& zv)?^rx!#@pcNS0yuq#Sff(@u)@VJIRvbF8i$&f$b8M2ADCx4Q?T_NWDR1S#vZwO?8 z2WV_Pli51|VtY*>A$l;8FsC08d)_rEq-&ns`HOT#D_TxSf^=f*> zG#m#qL&tpHet;-)lIu767fo74A^daA5KJmkpq!u7N|g!|4$jMH;Jf>88AO2#&k`jS z&A*+90n@B6?HbelU~F4GtW-mMPA62gF1vDRkur%4OQ(HZ&f|EIK~A(-cRBA42XuzP z=MM;bd5pNhh<@(0rtAtk7(Qh7{-`Z?5uJkXCj2e=q%l&V)dKFc4FJUk(K zZbDhD@~V92Ek6~39%&m);g5jMn}^nVnY()v$U3V*LNGqp5FWjW>6gqm1&15s!jLzI zKz3?bbzAzzbwYME1>d7@;0|mz`PIL}{Qsal9DsQ{9ifvTv&wte2KOzjHp&xt#$Qi0 zBXcc*_rZ@(n3S^9?eE7hE7A_HTbVCDN}?`~=bS-N5O-)Ip`I^+4K?cgoRA=j{)kNY zXv6TI0`(bwV5iOLw6GGrJf)yxS@R@ju4`)h3)Hgf)m0Iz&!NopX;~#{6R1dp4E;Pd zs$ve=J-@J?&Z#?OhU^PlXWxL|4nb-&Z&LZaMo+~Nd5&p*hcvd=5Uln&frCDM`HR&f; zvT6L%kS(a-B+{*fYi3-lf!|>|UY(6Gzje3-Tp5bHzBIROw2SIiQl-XCCi`^MW0*#b zQqPpkhrm{tHT*GnR=y6FxY_^U9C#tpbMVS|1kkCNZ=l52gUA)T*PMyX?7`+)LNMvC zQuXIxE!FR@e!Q6kCtK39HZHoCB3GdNdi$N4Pm2&v+p(2+k^$U?4}M6Tw>wIA7qpdHIXYWzTKfvZ%l~xxkM(ibj2Ml@OxsYn#PoxeMBf^Z;#x%UQu8KKh z)SOa&9WD{hf%_*flq0CjeMVb&s^m&~SW0RBxv@o`f+Kd3m0~@F2p~Cw)sWn6bST>4 ztchjry6tsE;#8C9>_0NLHa;gryeM!m)4yx`4VaYiEUeD~&o-pE{^EKyis-8R#@&Vy zF|-^ba@;g>Qm=g1vBID97DR! z|C_?lwxbK76u2b_DQhR%p8z4;{}LF$#H0!+%u_eDyn$Lr99>JuMXfm&E4%)+cCn~I z=@Qchz%)M0BBuAt=768A^A(uFKT_yyo1knOl(nzOwa!tt{~v_GNN@6rasVt#nB3p^R;pE$JHfnAs#`Rf84ge#|lsk$_i9N)E5C2l8-(-8&B ztb{#U*=BFj0+Qg`v=0nW^9Wvt5L*52_iplV^liPbkN%h8sLG*$D?#NB&s!TKSFblf?>>F&n%EZ}iq|f> zu~~T_9|92M>(}zq9024=Qt(UOa}p_%yS1ie+NVLh@ldI)ZzHFMfuqBu(lZn?*N&5z z2SwU$Vqt$%;9z2CcroM{JrSH6o2Cv2^Gye_z-sV8mP7I}obaT2g z{MN6w8_U;~-)TlX5Ph0TMjC_;!SNr8WXQL4wmM@<2diN%TeuIGPwThorFlO0acD315P z6hGVgP&^qLWDDn}^txf*IqNm&h|Q$lkd!~vsR4z43-iL)4P$-Iv3KnFP@csBYmicE zd6Hz9oZ9d&^q_Z1wH12WS(@1Nn>=i>q|!OnAM^#_qgf6~QMC_-wNd&*fbqe_e(?8| zU|(8lfp)Z|AGV(YuaJ^`Sh@-%LFmW9u%TtIjTVkxX=7)4mev2z&;Ob4XyL9qlr)Br zwmnUp2!7O@JahjKcwe`{{2Er^H))yFf#giz1YEop*L$j;!OVd}*BR8&WVoLnt-Nnf z7PaWFN&aX9sxHd8XG?XQgR-{3_2^mn`W|nb?>?*Imzjqof@J~4<(*7U`hzl?6df}f zg3@jyFbmSYNp`Lg;CEnuy&!xCj8Ld$$+8lPPFJvqHemQ=rt`TRq-cj+((wgg{YF-3) zb~&j^S1fCrO^-41ur9{X^>5y+%$VNCZdFAzFB0gmT2_U%&~OHJ8)7F6HaBv(fhs???T=m{G%JC6fhsUxI;6N2T5gU=|q?o`ik zi;%`k?YEQ3)W0zt_wV2L2VXK*%6Ctbfm)3ZXOw$)JIaDPW5XzV-Ug$pso9VluY`so zPIwbS?zb7wdvh~$(P$R9+=|t6#>Ewk#ZC?y#&m&!(;){+^6>_g4OA4?TpGxI zghq8Sf6_7eVdC>t9vvLRD9)$5oawtAkt2{1zVJ=x zcWgK?>BlhVHjSZhNh->({y5(|*RL4pIW=bsxKqGy%+ZZi8Ts#`GId-Y-6<0;M4MN;KFAB|A}z(f9jjM6E4~@2;m(JRjV>d zAFnDS(hp2b-h`su{x#_D2@)(d@-oz0P9`he-D{g~`W1r?w)`-`^{&Ivkak46kAkvK zYQ{xHg$9UE$3;&8^0V^$98lj^?)47bKUklxDV+tnyZDk=VCPH&gfpH}BM)vl+W^&i z{N$j#fB%#J`~4X=4lK6Whp_n9c2yb;Mf3OE05b+>_kC7(?te`6p^2*|e!EsRSh)d3 z^b@%)iF!b{;~q6cgoaZCTYccq_~(A{YIG4a9rskNui`a|iIDLNnuO5jCe)BS3*EAV zIE4Psy=GuxW}~B15_{|d{Hk9?5^MUmKx1qs<{8C127cJ2+bYc`z)UK>=E@Z=5FWgI zuux;^07*6T=p25adaEf`_&L{hhfvzVG;Nxdpx|<}{V_O#)hh~LNLM7{KLSy+|0m@Q zAS|hOi&K{<8d8Sv2Y!g)6!7A7Q0}s0tW$0C-pT?vfoPHbc`vB5%CH^*&(RkKlLDm; z<`|vD>jvEW9PdImbcl(fywuiQb7TpirPjLanwA>BMhe0Cm)*0h@}; zRnJ2Td;lB?@hbx!`YcYJt?eoe`0p*gp#O5<#X%0dI>9k z^kVOP8I-9;&iYms3%FluVhW9AEN>00l0uRt#wYmjo_2Ip(vcL1i!E|vVxiAmrdMkX21b6t?yO=N## z*7S{Ty!3d>56MzbYoH3YB2OX}UXG6SesEwsB5G_nfv0fJl?^xxgG6g7Y(0GxWpi-h z8Kws(Rde&(oBjLbl*cFN&V07M#~J2)|3VG+D0A%3DF9Ri&g<=z>OfdoYW9xYw&J&p zbVRyh79t^mRln#&*E*k+dW2gBm9sslcc%MLA&cg~goe^KJ3hS$ew zayiBGv^e{zoQn$?Ygy{Lb_V|J(UuCH#ka6}#~yZtQo)ftAyjY;CM3DhPG7I{^nQVs z3M%=}U6!MlIY`Q*c~w7DJ5{*D^=T6x%Vfh)H+_73V$};deIEypfcy+M;7|STATx-v z1TLdPr1R03-UzPx4qe_Wn8xy8xYLmODu|GumzaJ8KSnRR6g7mW}$AFxk!Q=j|R^Qt!%{tWU6OAba0nfXN1 zxcwZFTPveD(S(xlOHxN*)C^cE70=rXEgnR=FR`@gYTGnM^R?Ix#2sVRp9E}H zsA(9K#(!9POM>~nZivYA>k7h;rcOb=@Dm*uF0rSl>);aVDnWC2BQ@P?PB9u1SrcuA z^{(U@5{c24e3vXiR`+`CC34Pr3K3&XGil5^Pry0gB}*_ zp5sIIoG|JyEIe(@`2r=!qx@m(_rQ%H8uCoH#{=+udTBL?9_7$}UVPQar_p2eR!yuC z%VP(pzwizFQW5`IIIH3&O28TsdW4myEjE#ric@}fhX^HzP2L3aa5g!fzH!rn%LFdi zj*a(Y`)uVHo3X>jW$wx7sHGN}1!}_zzd2L05@!SJk2-)+h1_0XX(w(y$3VI_M>xghJ1@gYTmT1dOP1#(j>x$aZ^ISr;j z3ezN1>54AF#;?E7PH`@8C1&K|p*-7LVo07V)Ct!ap&JjbJ=UFi>1ZTebNdjZ>yJkT z;aFdIz|m%Xi%1S5>FL)agcO@_7afU~V*6(H&Lqv@C+ffhTqp}7a*CYBRo6Y+OW}Ho zY_4FOuW;&ws{-kl_9n>SND8@_DmlH0lGt)ImNP54@y?u6exkvpoJv5w0?V#UO%ra8 zJ8$E-$jq}ke}FxHW~g-?9}0Ame5>y_4BBupk1OAh5OhcSik&(y@O`2=bj-N8nFfx0 zFN!6gZ+4n@H;j2#kdu6905N9j@>G6!7p5`$gda(dKcKRqg5=RYMOa`#I8|4mZvVW+ zJG_-+BudayjCpm_&)O{Bs`lme2?PF@G6Nwy02hdTM6fbTjyC-|G)-uE#^LC%|?cwre#Gm^5!e2NHD%HdGu(Co!WVJ*1wI_MsRax`dqGI3d`;ye!-OsfV&r9X~boxta0k zc`qW64j-XzXA|28xi5Zxj;u;Lyuo;1nGgrp_M5ZS={;;mhRB&G$_6%cgKKB! zpUW%SlAY8^Cm!_mUxFvRQ}-5|*?2qRhKOI1ooMw~^5arICq8gSYl;Ko7}fkjb`YR< zxchTG$No{AcW{`ep4MgpJ0s307oM>~-`g$fm5wQmlQ_OhP1+CY?u-r0eHE5&I?^3p z!f~ym*{^?}U+C^!{*n(n!}@M|7tzA-?`1`xs>p6}aS-cq z1Nw~c`8Tm>y`12kZN4>IKwDy9v!XrI4S&&=%9A>OdXB3`qVvUbD@S!p4Ny`u_GW3) zK4TUuR4AS(YZ$^{+0-a%d9^lyd4Nenmy=x7D0B45$T(e8YA1z}ytX(s2%=5NA)Vm< zxdLx%2`G3?%#6bzLO1Cc9GiB;^Rn;B2QyxMLCQv=AMb9TZsGqKiIMz3C?j1tBl4MDwQ`z}QAHxf88_g2k+v0qR@`xP%d_(Umaf3Rz zdIVoLM_gA#<`!;&&e>bvGBz|g#l>hw_f3p*H2ie4W_PiXop6cv`Pz)iTuNH+9ZQ)w zevQo`AZ<*LV(E7CnW}E+UA~LM@=^^k_Xf#@k}wwk#uoEUA5_eVY)vPoVD>?h2k=!4 z88?%?*t)XwWgBn!ZbrR7~UXg-q*QEIg$E7;=Z; znSdX*-obg!Y`sCjIs|k$XNE>NYa7Rb9h2o$y&qFW;d=~`i<dIjN-J^qy$n^?t<$?!TJ%!U&E1Id2o!f@>-@_Jr8zrFgqJo<+Xt&{Tl4J5X$ypur z^ifHzTJMj1)r<$!Oul{8;*eZA`4xYC@uP*&oWIrniKjaGqH~K}`S|L_N-fIp59fuu zmg#{a)18xrUE4mC4r#EdJQrbrXP8W8^KW$+{_uZTdvZ+?NOiE^DtE9=94|+*tZ}d$ zFWj5Z0wO`ag4xg2d~KMZ)IsqqFgG@z@ zjM!HxvW{)svPA1Z1S0ma%W3C>8xy&;f)lB?eBcmG>wdM*D_U=|9c^JyD#z3Q(`f|~z zXqM8OpHfm%5q;9z8H4d;HdSP(G5N_2>s3;Ox12uSG?>c`-@{sYTCISH#G3)2)>8m$S$6` z6n8x+@}$4@WnMELsY3g#BzxfDm5O^?aS2eO3wAw4USXW9C=5`1?#lu+Kv{Y&*nq9uMEU zJI7mV0#9NE;|T&KPv7~HCR=Exj)rvXl^ABADeO@KmI)!|rJTV-ceSj94 z01qb+JsqB$V%?~4$Jy!W`>sV{8u)j0RT@v!zg~vU&ru#m1`>jUT8LI`rdl8^UgG)d zyI`U3`3Ncbp4}6<_pmWWx0W9wmcmYf!~u@|h$N(h!6#LD@Ms>D!`==MLWX%~>0>4| zP3C1CD#rRFA?jo}McBaaypV!#P$z6=yBeJ$u-Z8|WH_<;MulkrDA&2_`1?Zj7pdPO z`gT$fVaxxNKtLl1zOvu zGguxCXmfqU#fFbmvt7G46i0bDY+DW!*s^bstHRUF1&qMSCq zY^h5*xw{Tm6p;3lzG^|QmT6vgv}?>h4%1M@E2K*Dt2PPj5^XVo$^uw7>L|fFVqJ9f zaIZlpZS!gtPt4QqW6CDGgDlZ9II50_HyS(wf@7LLT~&qzo~KS0Vo5s&4`&)x-=nc| zIO-G{T`m)m z9j1xjm-a+cwWR8SC*{*jv^zAj2ei`;r)OaA2x7P1Ti)0I>;qd3#4a>eH8$s$hNAMo zbXOmAb0wDjK$TVmMXQ-=0Eg>&GqO)su6ad=oBOm_R^dwl+x@`MQ3}E8eR&qfbRDsz z)_G7WAowNz5+IiZBeCT+U)71S z3iSa39e&C{!Hrf>phijtB<#3>9HAB9Adka{XBF}ZoY)ttk7^SHn;Q+!`WKFpLeH>f z!@(|KE_wJcU)SwOYed;ebNy!PrR)g@w)$N?>dSU~ z^!wH}0tm8ADsEj|>X})2M`5aG(LJLm&&-!wuj4R0tMcmO9ac+6Rm+9*-jjV_K0K;6 zX(2bE&vOh}eGweAFqor@xJ(s#7tF=^sJH-%NxKe*lNYG|4#Y4z=Z58CwZ~)!`Gry7 z`kh~5r4cE3Ep^r3y`ywT-8I8%6r{_eT*ssN)Kq`O2$^}E5Cg2G1W>r(!%qyxO1NM?NzIB4X zn!YnoEVyHPvLxzUy+{FYwow~qolo49fgAz!Ko~?>0$QA@mMA`T3F~1Q%N(NXHS$>y z+5Ks}l-EVA8utm`K{?Li)WGm6os&`2-*5M}oD|gl*!op9;=$Oe^RWei=<8PFZqJ7A z1;`$*($^ol915ZYHRE4FPUTYwKE$_%ixWA|$Oc|!3SO2~jlrC94^`@#3QCJh_k~{Z zqh_Z03;wVq+$rnaqh8(z67`;)I8K`bx2_+Ez8+q0;=mhx^FzP8hlS^+W%yY2W!~YP zYZ04tP`{f}$MzGIqhWNeAo5vyW6etj#7f~ZFZBECVxBypf@d+SC=k$bXE3N;HqR9* zVgNzfDQ`>g*@=9^PDEWx^}!; zO)WXdww5CdJZ)WY*7(zVD&L+!p@O%=zXIuR$o%|5Kp}wlwBk%7-+^QL%i{(yXI*&L zqg|rGUV;N#P^c?s87kXSU_n8Jl1wM04ngw+g2vR6ifZo@Ae~omdEVzhR4+SsxbN~H zA|jLf*AI7poxoFfpTtk_R;4yI1G44!l=e8$xB-}kWxlJ()|aEdhuqbmpg#?@*f80X zZLvJ34gRmkc(lW*>=+x!RH39CEBS$48TEVMa0Zp7V+pZkxw*~7>PwgPBfyeQo&su& z^_A>BhY>WkifPC?V?$|3m>O8ASkOSs(}5PhNJm*XvbYe8YFM+r1Y9u?vHo5UAqV_; z_>$VQ)64FKyMvx!SSqpywT^vL9%1!{ZtFucu+Vrba@;C}t49b}WrpV?AlqSgOZWabR#7F?i;Zy)RQgAE`ZzCLNN`HV_IgHzwf-Wr*4|S*dffs{iM9Vf4=w3 zeQb9>sIpT7!IbWd{S(dsQs_3^trO$^K|N+WN1DF9J(rqn;L6~b4S$sSzpq^ME9ia*@T8(mHoO~v*%k&LoPmK2`t9P|UHZw>*Fbgbejpv4?j7^G z@8nYtO6JD>JYRGpti+{v0N{N*KvWq5PL!tRRV{U>DNaC9mumd7P?JrXCw#rDqO?>Z z5!~^ZFVg2iN#!TA_J=-YCUpL_DqE{7Yd2ZrO^=9~pbwXWQ*`_1 zF`I-#%Q2a0!Yx@aeA0Ci3`m@}>nNcd(RpX{DDbZ@euP1L#uO&rF$ zZCTaTvIcW5cmDZWJ{BQi*Tk&%?cK--1bk2V;P@@i)p7h`~xDfMo+faMEQm+ZsK!C-EnqSbJVv1YK58nE?4N?P@ zSe*K5E(WbXhX-q2Tq7>xoJ}Do_(7RqLrtiZ5<+5`d*`R}l^U^~pY?LiIx3)=rHA!U zWoqv`s0uX@S25=re`XuJs;V?;4FTNofceP;h>yfF?YV8B%NljN4ipl0hAK}2{rJg6 zsBko0MgTx3s>5L6>gOS?3aYUWB6U3j0|Pnc*RUH}AZj!S+Wxr$WLqf!Kn~LiK~=C; zKvCnOYZ3N&n$kV(bH99m0-&I{a2Y`0jDYZ% zTXv_&OTt^giqums`+;Q5@_qx66wnw&py3=SRtL3*oCDB*NrX*yHnHg~DsS_vgIp0f zp20(Mo7Xnx{cBz1e!ihk0#ziqML>;K(?{&z$K-P1`Vbzy8U&EHEi`!+9-nI|0O+Ej z^BM4<*+ATz+hR9C!sq^;DiBhsyAkfUINxb*3sTYpu9zTZtDp?ru_0U8eQ=Ruj!kvl zFCYAM`+*#KNWFA_5J2HQtV2(hWeq>ka(@w~&OS#R8>%XpfKVg=#-`pkAAJL13-zIb zDz7q)@Am2d@%aLvH4S6>*HgBaGNNr}o3*J_c%Rj5PKXYQ2CscgCB)adhfA+FYTdjF z%0-GHX0{hoc78r$b{he4G|f5*7St$w~`{d!Ms)7ohM?hd|~Lb2K*pId0ZYkrP0&}J@c z=K5N(*Yx|2D!-CZ!4&B4b1e+#J^2ZpM+j6)BSl>Jq2jIFwYgGOEA9r>5g+1v$eO^b zl0rBlJ`xKz8~OQ`rq43 z>b#!Cb0iR*?!t&E@g_lWOdv(Q{m*{{SG?8aZjeqCE;46pT3HSytQ)%npkV`0wd91a zW>XM+s(Ao|nQ;XEBf zx+A9qrBAwwBOc`Z_65X7!xP!>Odjd6xer>PrBybH@dA-4=JnE%|F=%jCQ=7^+Z>-` z0gG2?Edi;2&*2;(55(7zZLS99$WVK>K#`#K>FiBnca42rPh9yMXY z+sYISsy;>X+9-CxT9JY;R$n{;E`^66PnwoU=LHRbmPlkeS^=_J<`lTsqnyU|{m(kQ z2H@HA9}7g;apNOp-~Zv*C`!^%gRocUVC6~^*Xf}6uHR?R`u?|f=$buV)~UBOpk+@? zD)i02 zd)oG^{{fLt5nYhKVu1X7;(@?-otho@f?#a*5* zHhQo7yawpzzO~qLhz}RfiEPH@-}q2zV%(c%yslq%NolnS1kmkKw;vxp`xb!T-sQT3 zx{gU&xAo5MGzcMFVi)*>uklXneE~N|rtG?P)|I9yAYk75wLP5esnXn5-om^r_b>F| zgffL(bX4$F1qJOTseT`I7OjH->s@H-|7I0A;nw%iUt`fMz@>{)$1T-ydD5Lt)^a-7 zR_rU8Twmb&I7DtbyXX*`*yTs$&btdWxEe3K~q+Xp(l;oX`S-?!imVFYCTf;$6@+y-jltMl2Pi87iHv`LV4b0B> zt})xLzZbapudP#!phNn`BN@-@tM;ysAOfFeYyd3!P5h&eU|koC)@`N3bpd#9@nK6o zA$FtOw*8QV7as$QW0QgM4CMd%ZF+%IG=wdf%bU`#U(U2I9+G1#ANbbnakDdZvQLMN z4{ew4A|5qk#>xV)Ur(RO_%zWsaJ)$c{j z97dA=)hloc5>DJOcylzE4)n(Q+KT$TB1a-=-PbF)CK!O62RYx-qjPv*&`aqgg`lg#R8?-ElL z)E5J>A8OvNK@_tQFaCsDP+a4b$Ozq?hyq>15JnBV2ZFyB0?q{+LS=$HDL7i2rS(nU zz-TB<`I`!Fup(&6E8L_TqLc#9(6ZHIa0b{c672`^tW^Nsa$xtw*nfInn3nDb(hpMc zB`@pV(}`9Qx>JcEkN8J>#w?_3{WIVi!*017+?0;9>r3b6ygNaL$yA=SnylsO?5lif zijI{}(xepBV)Eg`nRdwqB(Csu_=}#_5e!h0g zsnBx)c`W!}pvv5fNE!eQ6&^km7MyHuvvJ?%iBqx(no+YV1pRmulpLMLJ6Md&trMor zVFxsB%=^bs#Cy$cP!a)@-b}8A(jG{cTr5*ta|fkgl0O?bWUWXEpAGunT0*34tTuQh zx=tHg>_4bu5KXBisC9_bk&u9fZgGw?7FZ|Qo zm1Y_#yYigrOvk}(kkc_X@AVYFsV;xaA?>cH{II)t%6eA@H0|^ElUQ0=lShhXa-3~D zLHd;=oa+S{$&A1-zQGrUVZ`}C~h9}}D^)SPabB$~^)6sf*G zx?j#%M-S>dmcZR;^%Iy&C89UKKUx7g@W7JF(@7m&?QtMA#5JgM1MY+BoPkiXrFvxD z%y$x$hc(m`0^g{x2H4n2t1CucDGtQ4z{R&acEUKIKMCB z|8j+8VtGWoF>CvdT5SyU3OR1ApGevI>M#c^7USA~*RtE?Pz1-}6U}MFL<@~0xeo}e zBBtZ$3Ttu4`6Q^nc#PIXr*)rltF2ihlFsFIl^Y_Uj)+p{$uT3Vpyj)&3RhGw7X@Ank>FMLo223oqoR3C&vgP@EJKAz9u5h zt`zf_qW8IKf75`>lM?@wo@L`cU^>_W)b%S1-)@5@lmMX4rigeCKXf*n%k=ftJ5KW%z=1}?l7-fam=FEY zbN`(GU~QMVF1}b>UIB4BnK<&+HFlW zev}9$Rhi?~f1^Z^M9mNy5ULl4qeArcYM$7M2dn^qw*yFTEC`p#-j>h}>fA*+;wo7_ zX=Bx3{LS%tGoJFkwLtMz><^qPF>3c~y9?Jr6(D3KSoJLf*wcwRmn-MVuNIyB+2e== zH4)3~{HMgx4*o<}-H$Jy)H_CdH*`X}j9*C$Omv>U| zz`@p{l0(492yyu2T^KjmVLCxyO`_l9O~quYtzElLB{}3GqG=?vSQ9z#;lMWtT04p( zB779&05-(M!=sEhLh3Ch?6N$c#`WC#uMkWp0eNG#D~m+c34n?gxaF62p!#S@Y7kyO zU+`9VA)Gy2+@Z#v{IDVv|zxxmO#IufC}r`05*S zsW|_;N9HhD>5OsrDeEkb^|T1-CT8PG2&otufQFEYKArgiH!)%Yc{P51en$+_8I|{t zb~(Tllfh_*;|iKb|?I&)wWwNB9}NA%WgW{Xk$qty514 z6#@6(0H}gw8C(A!B2mv7LP-r%dU|$O8ZYVg=k|n({RB%P#&ssm;uc|B}S zmg$A6%tq^}$BTUckus}jEgE<(&nWi?;u8&3+)6T5FPhx7hp+m}n2u*{coq`@n8tr8^i=R%LBjdxVM^zKlU1u&ec(1$ujB3ZdjBp!>c0yJ?gR+3VzTbZ z%kzt;Gs^QP1L(&egbB28FsKMGR++KkRR~vn=Y9U*9ZW)F%Cj!OV~Ms|5Jcs_(Xv?x z7oaDw;c%1~BHX@Xd?cK`nMh&+;qX81f5Ky`%Ds3ogvhoAxXQ98Y4JW6;TpO;zx>yB zumLD=wzr_RLO5RkwByMbEVFILeg0fYAx1~%Sj-fNasC~%z;?jd8{2?|tk*$ymyWq8 zu*@5Rm8M?W8t&;i(svXQ&%V}>G>C+2SUtjDdZXbIy7r?8WXs! zXu`rO0p5%f%Mw8!z+{>1VlV!zHPj2v-prt{1QQ6;kQWKo%~PI8I2+wK zM;FR(6vp>^1-s`Gbrv?HErbxS%)6qeePn4U) zA=@B%PjG^hE4q?SSs!e>aN0Vu|28KCHX@o~M;29`)5Fw(y_gseV6l2B-n#)kj7mu7 z-j0LtUvuMO-?W(s_Y?Z`f2}hkq-;vd1-C@gO4BbS5%m89Yv?N+9%e-}Q>8DUA z$;(K_S?+(t92Efgw`ml+=Pb|os+#ulUQ!+C{c^!aP81H?YNMFt`>*=u`%a({%=FE= z1xE5CZE&_h`Twz$f89c-!)?5B!yGQT!df`VVKS{z2RTg3UH5!Ce&g+TBt`+7Z&OWj z2|6fTPOLW^>UwKY(-chPsF;-Zbg4OHE zJ39BDIWv2N00%_Rb1}5sJ(lHtra1a~`e_mme6X(~-%iQJ0@y8kaiTvJQBYNo`p1 zE?j}f3cds(_yOm9OglYkl4AMnuD!6awwBDS>~QeBGo00vi;^!KF^6A2?6-}d&6(;e zSkd?1VmbArX1*ksxUip3vViD2aWDB?jpDSyRtp~4}HFYCo~vK_NbCy_Z%$O zP4g(|TxqmeUaEazb1Pprs3N!Rg|n61JjF6UK(Y~+b#cy>TZ4ce?2YWnXB`#(YsZnq z%UH>NGR&x>xhzae)zk=VyHEzRQIWy3789)?XEWLQz{v2lJgM1hgG93JxO;EmCzO9?2Mk3={|gGO2!xyh`v8~So6PP2H7Pwz^GZ3Z};>YigV>$n9xHtgaX2Zf2 zb{SLxx~}4vfZB`s<$`BB@OF-@a7dy2^VIGq5fKn(Qamj;v2|Fa&3S}O719xM_Bd!6-a z|D|7a@M6{e;;!ES&x2B9z>u~881lbIrU9bSUCe%(17!Hpu|LvgkES;PywEFOMC~uJ zzBg_|EtRagKA3Q7)h<-#uQ9c=s3ySG`x~$DTNT04Y5!U}3vl6Q-+yrF_x52;5nw}q w{VR$qbcddbYG=pXl~;?3ii!qO>Gn}s73Im;B&tV&|DnPuYF^H}bS3Qn01GK=ga7~l literal 0 HcmV?d00001 diff --git a/protos/table.proto b/protos/table.proto index 02e85f25844..8c032803817 100644 --- a/protos/table.proto +++ b/protos/table.proto @@ -504,80 +504,154 @@ message FragmentReuseIndexDetails { } } +// ============================================================================ +// MemWAL Index Types +// ============================================================================ + +// Region manifest containing epoch-based fencing and WAL state. +// Each region has exactly one active writer at any time. +message RegionManifest { + // Region identifier (UUID v4). + UUID region_id = 11; + + // Manifest version number. + // Matches the version encoded in the filename. + uint64 version = 1; + + // Region spec ID this region was created with. + // Set at region creation and immutable thereafter. + // A value of 0 indicates a manually-created region not governed by any spec. + uint32 region_spec_id = 10; + + // Writer fencing token - monotonically increasing. + // A writer must increment this when claiming the region. + uint64 writer_epoch = 2; + + // The most recent WAL entry ID that has been flushed to a MemTable. + // During recovery, replay starts from replay_after_wal_id + 1. + uint64 replay_after_wal_id = 3; + + // The most recent WAL entry ID at the time manifest was updated. + // This is a hint, not authoritative - recovery must list files to find actual state. + uint64 wal_id_last_seen = 4; + + // Next generation ID to create (incremented after each MemTable flush). + uint64 current_generation = 6; + + // Last generation merged to base table. + // Generations <= merged_generation can be garbage collected. + uint64 merged_generation = 7; + + // List of flushed MemTable generations and their directory paths. + repeated FlushedGeneration flushed_generations = 8; +} + +// A flushed MemTable generation and its storage location. +message FlushedGeneration { + // Generation number. + uint64 generation = 1; + + // Directory name relative to the region directory. + string path = 2; +} + +// A region's merged generation, used in MemWalIndexDetails. +message MergedGeneration { + // Region identifier (UUID v4). + UUID region_id = 1; + + // Last generation merged to base table for this region. + uint64 generation = 2; +} + +// Index details for MemWAL Index, stored in IndexMetadata.index_details. +// This is the centralized structure for all MemWAL metadata: +// - Configuration (region specs, indexes to maintain) +// - Merge progress (merged generations per region) +// - Region state snapshots +// +// Writers read this index to get configuration before writing. +// Readers read this index to discover regions and their state. +// A background process updates the index periodically to keep region snapshots current. +// +// Region snapshots are stored as a Lance file with one row per region. +// The schema has one column per RegionManifest field, with region fields as columns: +// region_id: fixed_size_binary(16) -- UUID bytes +// version: uint64 +// region_spec_id: uint32 +// writer_epoch: uint64 +// replay_after_wal_id: uint64 +// wal_id_last_seen: uint64 +// current_generation: uint64 +// merged_generation: uint64 +// flushed_generations: list> message MemWalIndexDetails { + // Snapshot timestamp (Unix timestamp in seconds). + int64 snapshot_timestamp = 1; + + // Number of regions in the snapshot. + // Used to determine storage format without reading the snapshot data. + uint32 num_regions = 2; + + // Inline region snapshots for small region counts. + // When num_regions <= threshold (implementation-defined, e.g., 100), + // snapshots are stored inline as serialized bytes. + // Format: Lance file bytes with the region snapshot schema. + optional bytes inline_snapshots = 3; + + // Region specs defining how to derive region identifiers. + // This configuration determines how rows are partitioned into regions. + repeated RegionSpec region_specs = 7; + + // Indexes from the base table to maintain in MemTables. + // These are index names referencing indexes defined on the base table. + // The primary key btree index is always maintained implicitly and + // should not be listed here. + // + // For vector indexes, MemTables inherit quantization parameters (PQ codebook, + // SQ params) from the base table index to ensure distance comparability. + repeated string maintained_indexes = 8; + + // Last generation merged to base table for each region. + // This is updated atomically with merge-insert data commits, enabling + // conflict resolution when multiple mergers operate concurrently. + // + // Note: This is separate from region snapshots because: + // 1. merged_generations is updated by mergers (atomic with data commit) + // 2. region snapshots are updated by background index builder + repeated MergedGeneration merged_generations = 9; +} - repeated MemWal mem_wal_list = 1; +// Region spec definition. +message RegionSpec { + // Unique identifier for this spec within the index. + // IDs are never reused. + uint32 spec_id = 1; - message MemWalId { - // The name of the region that this specific MemWAL is responsible for. - string region = 1; + // Region field definitions that determine how to compute region identifiers. + repeated RegionField fields = 2; +} - // The generation of the MemWAL. - // Every time a new MemWAL is created and an old one is sealed, - // the generation number of the next MemWAL is incremented. - // At any given point of time for all MemWALs of the same name, - // there must be only 1 generation that is not sealed. - uint64 generation = 2; - } +// Region field definition. +message RegionField { + // Unique string identifier for this region field. + string field_id = 1; - // A combination of MemTable and WAL for fast upsert. - message MemWal { - - enum State { - // MemWAL is open and accepting new entries - OPEN = 0; - // When a MemTable is considered full, the writer should update this MemWAL as sealed - // and create a new MemWAL to write to atomically. - SEALED = 1; - // When a MemTable is sealed, it can be flushed asynchronously to disk. - // This state indicates the data has been persisted to disk but not yet merged - // into the source table. - FLUSHED = 2; - // When the flushed data has been merged into the source table. - // After a MemWAL is merged, the cleanup process can delete the WAL. - MERGED = 3; - } - - MemWalId id = 1; - - // The MemTable location, which is likely an in-memory address starting with memory://. - // The actual details of how the MemTable is stored is outside the concern of Lance. - string mem_table_location = 2; - - // the root location of the WAL. - // THe WAL storage durability determines the data durability. - // This location is immutable once set at MemWAL creation time. - string wal_location = 3; - - // All entries in the WAL, serialized as U64Segment. - // Each entry in the WAL has a uint64 sequence ID starting from 0. - // The actual details of how the WAL entry is stored is outside the concern of Lance. - // In most cases this U64Segment should be a simple range. - // Every time the writer starts writing, it must always try to atomically write to the last entry ID + 1. - // If fails due to concurrent writer, it then tries to write to the +2, +3, +4, etc. entry ID until succeed. - // but if there are 2 writers accidentally writing to the same WAL concurrently, - // although one writer will fail to update this index at commit time, - // the WAL entry is already written, - // causing some holes within the U64Segment range. - bytes wal_entries = 4; - - // The current state of the MemWAL, indicating its lifecycle phase. - // States progress: OPEN -> SEALED -> FLUSHED - // OPEN: MemWAL is accepting new WAL entries - // SEALED: MemWAL has been sealed and no longer accepts new WAL entries - // FLUSHED: MemWAL has been flushed to the source Lance table and can be cleaned up - State state = 5; - - // The owner identifier for this MemWAL, used for compare-and-swap operations. - // When a writer wants to perform any operation on this MemWAL, it must provide - // the expected owner_id. This serves as an optimistic lock to prevent concurrent - // writers from interfering with each other. When a new writer starts replay, - // it must first atomically update this owner_id to claim ownership. - // All subsequent operations will fail if the owner_id has changed. - string owner_id = 6; - - // The dataset version that last updated this MemWAL. - // This is set to the new dataset version whenever the MemWAL is created or modified. - uint64 last_updated_dataset_version = 7; - } + // Field IDs referencing source columns in the schema. + repeated int32 source_ids = 2; + + // Well-known region transform name (e.g., "identity", "year", "bucket"). + // Mutually exclusive with expression. + optional string transform = 3; + + // DataFusion SQL expression for custom logic. + // Mutually exclusive with transform. + optional string expression = 4; + + // Output type of the region value (Arrow type name). + string result_type = 5; + + // Transform parameters (e.g., num_buckets for bucket transform). + map parameters = 6; } + diff --git a/protos/transaction.proto b/protos/transaction.proto index 638898f4197..17d96486736 100644 --- a/protos/transaction.proto +++ b/protos/transaction.proto @@ -235,8 +235,8 @@ message Transaction { repeated DataFragment new_fragments = 3; // The ids of the fields that have been modified. repeated uint32 fields_modified = 4; - /// The MemWAL (pre-image) that should be marked as merged after this transaction - MemWalIndexDetails.MemWal mem_wal_to_merge = 5; + /// List of MemWAL region generations to mark as merged after this transaction + repeated MergedGeneration merged_generations = 5; /// The fields that used to judge whether to preserve the new frag's id into /// the frag bitmap of the specified indices. repeated uint32 fields_for_preserving_frag_bitmap = 6; @@ -305,15 +305,12 @@ message Transaction { repeated DataReplacementGroup replacements = 1; } - // Update the state of the MemWal index + // Update the merged generations in MemWAL index. + // This operation is used during merge-insert to atomically record which + // generations have been merged to the base table. message UpdateMemWalState { - - repeated MemWalIndexDetails.MemWal added = 1; - - repeated MemWalIndexDetails.MemWal updated = 2; - - // If a MemWAL is updated, its pre-image should be in the removed list. - repeated MemWalIndexDetails.MemWal removed = 3; + // Regions and generations being marked as merged. + repeated MergedGeneration merged_generations = 1; } // An operation that updates base paths in the dataset. diff --git a/rust/lance-index/src/mem_wal.rs b/rust/lance-index/src/mem_wal.rs index 7ba1cab80c4..f80eb678e3a 100644 --- a/rust/lance-index/src/mem_wal.rs +++ b/rust/lance-index/src/mem_wal.rs @@ -1,208 +1,291 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright The Lance Authors -use crate::{Index, IndexType}; +use std::any::Any; +use std::collections::HashMap; +use std::sync::Arc; + use async_trait::async_trait; -use lance_core::cache::DeepSizeOf; +use deepsize::DeepSizeOf; use lance_core::Error; use lance_table::format::pb; -use lance_table::rowids::segment::U64Segment; -use prost::Message; use roaring::RoaringBitmap; use serde::{Deserialize, Serialize}; use snafu::location; -use std::any::Any; -use std::collections::{BTreeMap, HashMap}; -use std::sync::Arc; +use uuid::Uuid; + +use crate::{Index, IndexType}; pub const MEM_WAL_INDEX_NAME: &str = "__lance_mem_wal"; -#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Serialize, Deserialize, DeepSizeOf)] -pub enum State { - Open, - Sealed, - Flushed, - Merged, +/// Region identifier (UUID v4). +#[derive(Debug, Clone, PartialEq, Eq, Hash, Serialize, Deserialize)] +pub struct RegionId(pub Uuid); + +impl PartialOrd for RegionId { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } } -impl From for pb::mem_wal_index_details::mem_wal::State { - fn from(state: State) -> Self { - match state { - State::Open => Self::Open, - State::Sealed => Self::Sealed, - State::Flushed => Self::Flushed, - State::Merged => Self::Merged, - } +impl Ord for RegionId { + fn cmp(&self, other: &Self) -> std::cmp::Ordering { + self.0.as_bytes().cmp(other.0.as_bytes()) } } -impl TryFrom for State { - type Error = Error; +impl DeepSizeOf for RegionId { + fn deep_size_of_children(&self, _context: &mut deepsize::Context) -> usize { + 0 // UUID is 16 bytes fixed size, no heap allocations + } +} + +impl RegionId { + pub fn new() -> Self { + Self(Uuid::new_v4()) + } + + pub fn from_uuid(uuid: Uuid) -> Self { + Self(uuid) + } +} - fn try_from(state: pb::mem_wal_index_details::mem_wal::State) -> lance_core::Result { - match state { - pb::mem_wal_index_details::mem_wal::State::Open => Ok(Self::Open), - pb::mem_wal_index_details::mem_wal::State::Sealed => Ok(Self::Sealed), - pb::mem_wal_index_details::mem_wal::State::Flushed => Ok(Self::Flushed), - pb::mem_wal_index_details::mem_wal::State::Merged => Ok(Self::Merged), +impl Default for RegionId { + fn default() -> Self { + Self::new() + } +} + +impl From<&RegionId> for pb::Uuid { + fn from(region_id: &RegionId) -> Self { + Self { + uuid: region_id.0.as_bytes().to_vec(), } } } -impl TryFrom for State { +impl TryFrom<&pb::Uuid> for RegionId { type Error = Error; - fn try_from(value: i32) -> lance_core::Result { - match value { - 0 => Ok(Self::Open), - 1 => Ok(Self::Sealed), - 2 => Ok(Self::Flushed), - 3 => Ok(Self::Merged), - _ => Err(Error::invalid_input( - format!("Unknown MemWAL state value: {}", value), + fn try_from(uuid: &pb::Uuid) -> lance_core::Result { + if uuid.uuid.len() != 16 { + return Err(Error::invalid_input( + format!("Invalid UUID length: {}", uuid.uuid.len()), location!(), - )), + )); } + let mut bytes = [0u8; 16]; + bytes.copy_from_slice(&uuid.uuid); + Ok(Self(Uuid::from_bytes(bytes))) } } -#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Serialize, Deserialize, DeepSizeOf)] -pub struct MemWalId { - pub region: String, +/// A flushed MemTable generation and its storage location. +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] +pub struct FlushedGeneration { pub generation: u64, + pub path: String, } -impl From<&MemWalId> for pb::mem_wal_index_details::MemWalId { - fn from(mem_wal: &MemWalId) -> Self { +impl From<&FlushedGeneration> for pb::FlushedGeneration { + fn from(fg: &FlushedGeneration) -> Self { Self { - region: mem_wal.region.clone(), - generation: mem_wal.generation, + generation: fg.generation, + path: fg.path.clone(), } } } -impl TryFrom for MemWalId { - type Error = Error; - - fn try_from(mem_wal: pb::mem_wal_index_details::MemWalId) -> lance_core::Result { - Ok(Self { - region: mem_wal.region.clone(), - generation: mem_wal.generation, - }) +impl From for FlushedGeneration { + fn from(fg: pb::FlushedGeneration) -> Self { + Self { + generation: fg.generation, + path: fg.path, + } } } -impl MemWalId { - pub fn new(region: &str, generation: u64) -> Self { +/// A region's merged generation, used in MemWalIndexDetails. +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash, Serialize, Deserialize, DeepSizeOf)] +pub struct MergedGeneration { + pub region_id: RegionId, + pub generation: u64, +} + +impl MergedGeneration { + pub fn new(region_id: RegionId, generation: u64) -> Self { Self { - region: region.to_owned(), + region_id, generation, } } } -#[derive(Debug, Clone, PartialEq, Eq, Hash, PartialOrd, Serialize, Deserialize, DeepSizeOf)] -pub struct MemWal { - pub id: MemWalId, - pub mem_table_location: String, - pub wal_location: String, - pub wal_entries: Vec, - pub state: State, - pub owner_id: String, - pub last_updated_dataset_version: u64, +impl From<&MergedGeneration> for pb::MergedGeneration { + fn from(mg: &MergedGeneration) -> Self { + Self { + region_id: Some(pb::Uuid::from(&mg.region_id)), + generation: mg.generation, + } + } } -impl From<&MemWal> for pb::mem_wal_index_details::MemWal { - fn from(mem_wal: &MemWal) -> Self { +impl TryFrom for MergedGeneration { + type Error = Error; + + fn try_from(mg: pb::MergedGeneration) -> lance_core::Result { + let region_id = mg.region_id.as_ref().ok_or_else(|| { + Error::invalid_input("Missing region_id in MergedGeneration", location!()) + })?; + Ok(Self { + region_id: RegionId::try_from(region_id)?, + generation: mg.generation, + }) + } +} + +/// Region manifest containing epoch-based fencing and WAL state. +/// Each region has exactly one active writer at any time. +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] +pub struct RegionManifest { + pub region_id: RegionId, + pub version: u64, + pub region_spec_id: u32, + pub writer_epoch: u64, + pub replay_after_wal_id: u64, + pub wal_id_last_seen: u64, + pub current_generation: u64, + pub merged_generation: u64, + pub flushed_generations: Vec, +} + +impl From<&RegionManifest> for pb::RegionManifest { + fn from(rm: &RegionManifest) -> Self { Self { - id: Some(pb::mem_wal_index_details::MemWalId::from(&mem_wal.id)), - mem_table_location: mem_wal.mem_table_location.clone(), - wal_location: mem_wal.wal_location.clone(), - wal_entries: mem_wal.wal_entries.clone(), - state: pb::mem_wal_index_details::mem_wal::State::from(mem_wal.state.clone()) as i32, - owner_id: mem_wal.owner_id.clone(), - last_updated_dataset_version: mem_wal.last_updated_dataset_version, + region_id: Some(pb::Uuid::from(&rm.region_id)), + version: rm.version, + region_spec_id: rm.region_spec_id, + writer_epoch: rm.writer_epoch, + replay_after_wal_id: rm.replay_after_wal_id, + wal_id_last_seen: rm.wal_id_last_seen, + current_generation: rm.current_generation, + merged_generation: rm.merged_generation, + flushed_generations: rm.flushed_generations.iter().map(|fg| fg.into()).collect(), } } } -impl TryFrom for MemWal { +impl TryFrom for RegionManifest { type Error = Error; - fn try_from(mem_wal: pb::mem_wal_index_details::MemWal) -> lance_core::Result { - let state = State::try_from(mem_wal.state)?; - + fn try_from(rm: pb::RegionManifest) -> lance_core::Result { + let region_id = rm.region_id.as_ref().ok_or_else(|| { + Error::invalid_input("Missing region_id in RegionManifest", location!()) + })?; Ok(Self { - id: MemWalId::try_from(mem_wal.id.unwrap())?, - mem_table_location: mem_wal.mem_table_location.clone(), - wal_location: mem_wal.wal_location.clone(), - wal_entries: mem_wal.wal_entries, - state, - owner_id: mem_wal.owner_id, - last_updated_dataset_version: mem_wal.last_updated_dataset_version, + region_id: RegionId::try_from(region_id)?, + version: rm.version, + region_spec_id: rm.region_spec_id, + writer_epoch: rm.writer_epoch, + replay_after_wal_id: rm.replay_after_wal_id, + wal_id_last_seen: rm.wal_id_last_seen, + current_generation: rm.current_generation, + merged_generation: rm.merged_generation, + flushed_generations: rm + .flushed_generations + .into_iter() + .map(FlushedGeneration::from) + .collect(), }) } } -impl MemWal { - pub fn new_empty( - id: MemWalId, - mem_table_location: &str, - wal_location: &str, - owner_id: &str, - ) -> Self { +/// Region field definition. +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] +pub struct RegionField { + pub field_id: String, + pub source_ids: Vec, + pub transform: Option, + pub expression: Option, + pub result_type: String, + pub parameters: HashMap, +} + +impl From<&RegionField> for pb::RegionField { + fn from(rf: &RegionField) -> Self { Self { - id, - mem_table_location: mem_table_location.to_owned(), - wal_location: wal_location.to_owned(), - wal_entries: pb::U64Segment::from(U64Segment::Range(0..0)).encode_to_vec(), - state: State::Open, - owner_id: owner_id.to_owned(), - last_updated_dataset_version: 0, // placeholder, this will be filled during build_manifest + field_id: rf.field_id.clone(), + source_ids: rf.source_ids.clone(), + transform: rf.transform.clone(), + expression: rf.expression.clone(), + result_type: rf.result_type.clone(), + parameters: rf.parameters.clone(), } } +} - pub fn wal_entries(&self) -> U64Segment { - U64Segment::try_from(pb::U64Segment::decode(self.wal_entries.as_slice()).unwrap()).unwrap() +impl From for RegionField { + fn from(rf: pb::RegionField) -> Self { + Self { + field_id: rf.field_id, + source_ids: rf.source_ids, + transform: rf.transform, + expression: rf.expression, + result_type: rf.result_type, + parameters: rf.parameters, + } } +} - /// Check if the MemWAL is in the expected state - pub fn check_state(&self, expected: State) -> lance_core::Result<()> { - if self.state != expected { - return Err(Error::invalid_input( - format!( - "MemWAL {:?} is in state {:?}, but expected {:?}", - self.id, self.state, expected - ), - location!(), - )); +/// Region spec definition. +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] +pub struct RegionSpec { + pub spec_id: u32, + pub fields: Vec, +} + +impl From<&RegionSpec> for pb::RegionSpec { + fn from(rs: &RegionSpec) -> Self { + Self { + spec_id: rs.spec_id, + fields: rs.fields.iter().map(|f| f.into()).collect(), } - Ok(()) } +} - pub fn check_expected_owner_id(&self, expected: &str) -> lance_core::Result<()> { - if self.owner_id != expected { - return Err(Error::invalid_input( - format!( - "MemWAL {:?} has owner_id: {}, but expected {}", - self.id, self.owner_id, expected - ), - location!(), - )); +impl From for RegionSpec { + fn from(rs: pb::RegionSpec) -> Self { + Self { + spec_id: rs.spec_id, + fields: rs.fields.into_iter().map(RegionField::from).collect(), } - Ok(()) } } -#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] +/// Index details for MemWAL Index, stored in IndexMetadata.index_details. +#[derive(Debug, Clone, Default, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] pub struct MemWalIndexDetails { - pub mem_wal_list: Vec, + pub snapshot_timestamp: i64, + pub num_regions: u32, + pub inline_snapshots: Option>, + pub region_specs: Vec, + pub maintained_indexes: Vec, + pub merged_generations: Vec, } impl From<&MemWalIndexDetails> for pb::MemWalIndexDetails { fn from(details: &MemWalIndexDetails) -> Self { Self { - mem_wal_list: details.mem_wal_list.iter().map(|m| m.into()).collect(), + snapshot_timestamp: details.snapshot_timestamp, + num_regions: details.num_regions, + inline_snapshots: details.inline_snapshots.clone(), + region_specs: details.region_specs.iter().map(|rs| rs.into()).collect(), + maintained_indexes: details.maintained_indexes.clone(), + merged_generations: details + .merged_generations + .iter() + .map(|mg| mg.into()) + .collect(), } } } @@ -212,42 +295,50 @@ impl TryFrom for MemWalIndexDetails { fn try_from(details: pb::MemWalIndexDetails) -> lance_core::Result { Ok(Self { - mem_wal_list: details - .mem_wal_list + snapshot_timestamp: details.snapshot_timestamp, + num_regions: details.num_regions, + inline_snapshots: details.inline_snapshots, + region_specs: details + .region_specs + .into_iter() + .map(RegionSpec::from) + .collect(), + maintained_indexes: details.maintained_indexes, + merged_generations: details + .merged_generations .into_iter() - .map(MemWal::try_from) + .map(MergedGeneration::try_from) .collect::>()?, }) } } +/// MemWAL Index provides access to MemWAL configuration and state. #[derive(Debug, Clone, PartialEq, Eq, DeepSizeOf)] pub struct MemWalIndex { - pub mem_wal_map: HashMap>, + pub details: MemWalIndexDetails, } impl MemWalIndex { pub fn new(details: MemWalIndexDetails) -> Self { - let mut mem_wal_map: HashMap> = HashMap::new(); - for mem_wal in details.mem_wal_list.into_iter() { - if let Some(generations) = mem_wal_map.get_mut(&mem_wal.id.region) { - generations.insert(mem_wal.id.generation, mem_wal); - } else { - mem_wal_map.insert( - mem_wal.id.region.clone(), - std::iter::once((mem_wal.id.generation, mem_wal)).collect(), - ); - } - } + Self { details } + } - Self { mem_wal_map } + pub fn merged_generation_for_region(&self, region_id: &RegionId) -> Option { + self.details + .merged_generations + .iter() + .find(|mg| &mg.region_id == region_id) + .map(|mg| mg.generation) } } #[derive(Serialize)] struct MemWalStatistics { - num_mem_wal: u64, - num_regions: u64, + num_regions: u32, + num_merged_generations: usize, + num_region_specs: usize, + num_maintained_indexes: usize, } #[async_trait] @@ -262,15 +353,17 @@ impl Index for MemWalIndex { fn as_vector_index(self: Arc) -> lance_core::Result> { Err(Error::NotSupported { - source: "FragReuseIndex is not a vector index".into(), + source: "MemWalIndex is not a vector index".into(), location: location!(), }) } fn statistics(&self) -> lance_core::Result { let stats = MemWalStatistics { - num_mem_wal: self.mem_wal_map.values().map(|m| m.len()).sum::() as u64, - num_regions: self.mem_wal_map.len() as u64, + num_regions: self.details.num_regions, + num_merged_generations: self.details.merged_generations.len(), + num_region_specs: self.details.region_specs.len(), + num_maintained_indexes: self.details.maintained_indexes.len(), }; serde_json::to_value(stats).map_err(|e| Error::Internal { message: format!("failed to serialize MemWAL index statistics: {}", e), @@ -287,6 +380,6 @@ impl Index for MemWalIndex { } async fn calculate_included_frags(&self) -> lance_core::Result { - unimplemented!() + Ok(RoaringBitmap::new()) } } diff --git a/rust/lance/src/dataset/fragment.rs b/rust/lance/src/dataset/fragment.rs index cfe003a7233..8428cf619b4 100644 --- a/rust/lance/src/dataset/fragment.rs +++ b/rust/lance/src/dataset/fragment.rs @@ -2852,7 +2852,7 @@ mod tests { updated_fragments: vec![updated_fragment1], new_fragments: vec![], fields_modified: fields_modified1, - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: Some(UpdateMode::RewriteColumns), inserted_rows_filter: None, @@ -2925,7 +2925,7 @@ mod tests { updated_fragments: vec![updated_fragment2], new_fragments: vec![], fields_modified: fields_modified2, - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: Some(UpdateMode::RewriteColumns), inserted_rows_filter: None, diff --git a/rust/lance/src/dataset/transaction.rs b/rust/lance/src/dataset/transaction.rs index e5916b7bae1..090ef465727 100644 --- a/rust/lance/src/dataset/transaction.rs +++ b/rust/lance/src/dataset/transaction.rs @@ -48,12 +48,12 @@ use super::write::merge_insert::inserted_rows::KeyExistenceFilter; use super::{blob::BLOB_VERSION_CONFIG_KEY, ManifestWriteConfig}; use crate::dataset::transaction::UpdateMode::RewriteRows; -use crate::index::mem_wal::update_mem_wal_index_in_indices_list; +use crate::index::mem_wal::update_mem_wal_index_merged_generations; use crate::utils::temporal::timestamp_to_nanos; use deepsize::DeepSizeOf; use lance_core::{datatypes::BlobVersion, datatypes::Schema, Error, Result}; use lance_file::{datatypes::Fields, version::LanceFileVersion}; -use lance_index::mem_wal::MemWal; +use lance_index::mem_wal::MergedGeneration; use lance_index::{frag_reuse::FRAG_REUSE_INDEX_NAME, is_system_index}; use lance_io::object_store::ObjectStore; use lance_table::feature_flags::{apply_feature_flags, FLAG_STABLE_ROW_IDS}; @@ -245,8 +245,8 @@ pub enum Operation { new_fragments: Vec, /// The fields that have been modified fields_modified: Vec, - /// The MemWAL (pre-image) that should be marked as merged after this transaction - mem_wal_to_merge: Option, + /// List of MemWAL region generations to mark as merged after this transaction + merged_generations: Vec, /// The fields that used to judge whether to preserve the new frag's id into /// the frag bitmap of the specified indices. fields_for_preserving_frag_bitmap: Vec, @@ -267,11 +267,11 @@ pub enum Operation { schema_metadata_updates: Option, field_metadata_updates: HashMap, }, - /// Update the state of MemWALs. + /// Update merged generations in MemWAL index. + /// This is used during merge-insert to atomically record which + /// generations have been merged to the base table. UpdateMemWalState { - added: Vec, - updated: Vec, - removed: Vec, + merged_generations: Vec, }, /// Clone a dataset. @@ -450,7 +450,7 @@ impl PartialEq for Operation { updated_fragments: a_updated, new_fragments: a_new, fields_modified: a_fields, - mem_wal_to_merge: a_mem_wal_to_merge, + merged_generations: a_merged_generations, fields_for_preserving_frag_bitmap: a_fields_for_preserving_frag_bitmap, update_mode: a_update_mode, inserted_rows_filter: a_inserted_rows_filter, @@ -460,7 +460,7 @@ impl PartialEq for Operation { updated_fragments: b_updated, new_fragments: b_new, fields_modified: b_fields, - mem_wal_to_merge: b_mem_wal_to_merge, + merged_generations: b_merged_generations, fields_for_preserving_frag_bitmap: b_fields_for_preserving_frag_bitmap, update_mode: b_update_mode, inserted_rows_filter: b_inserted_rows_filter, @@ -470,7 +470,7 @@ impl PartialEq for Operation { && compare_vec(a_updated, b_updated) && compare_vec(a_new, b_new) && compare_vec(a_fields, b_fields) - && a_mem_wal_to_merge == b_mem_wal_to_merge + && compare_vec(a_merged_generations, b_merged_generations) && compare_vec( a_fields_for_preserving_frag_bitmap, b_fields_for_preserving_frag_bitmap, @@ -1026,20 +1026,12 @@ impl PartialEq for Operation { } ( Self::UpdateMemWalState { - added: a_added, - updated: a_updated, - removed: a_removed, + merged_generations: a_merged, }, Self::UpdateMemWalState { - added: b_added, - updated: b_updated, - removed: b_removed, + merged_generations: b_merged, }, - ) => { - compare_vec(a_added, b_added) - && compare_vec(a_updated, b_updated) - && compare_vec(a_removed, b_removed) - } + ) => compare_vec(a_merged, b_merged), (Self::Clone { .. }, Self::Append { .. }) => { std::mem::discriminant(self) == std::mem::discriminant(other) } @@ -1711,7 +1703,7 @@ impl Transaction { updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge, + merged_generations, fields_for_preserving_frag_bitmap, update_mode, .. @@ -1923,17 +1915,11 @@ impl Transaction { final_fragments.extend(new_fragments); Self::retain_relevant_indices(&mut final_indices, &schema, &final_fragments); - if let Some(mem_wal_to_merge) = mem_wal_to_merge { - update_mem_wal_index_in_indices_list( - self.read_version, - current_manifest.map_or(1, |m| m.version + 1), + if !merged_generations.is_empty() { + update_mem_wal_index_merged_generations( &mut final_indices, - vec![], - vec![MemWal { - state: lance_index::mem_wal::State::Merged, - ..mem_wal_to_merge.clone() - }], - vec![mem_wal_to_merge.clone()], + current_manifest.map_or(1, |m| m.version + 1), + merged_generations.clone(), )?; } } @@ -2143,18 +2129,11 @@ impl Transaction { final_fragments.extend(unmodified_fragments); } - Operation::UpdateMemWalState { - added, - updated, - removed, - } => { - update_mem_wal_index_in_indices_list( - self.read_version, - current_manifest.map_or(1, |m| m.version + 1), + Operation::UpdateMemWalState { merged_generations } => { + update_mem_wal_index_merged_generations( &mut final_indices, - added.clone(), - updated.clone(), - removed.clone(), + current_manifest.map_or(1, |m| m.version + 1), + merged_generations.clone(), )?; } Operation::UpdateBases { .. } => { @@ -2893,7 +2872,7 @@ impl TryFrom for Transaction { updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge, + merged_generations, fields_for_preserving_frag_bitmap, update_mode, inserted_rows, @@ -2908,7 +2887,10 @@ impl TryFrom for Transaction { .map(Fragment::try_from) .collect::>>()?, fields_modified, - mem_wal_to_merge: mem_wal_to_merge.map(|m| MemWal::try_from(m).unwrap()), + merged_generations: merged_generations + .into_iter() + .map(|m| MergedGeneration::try_from(m).unwrap()) + .collect(), fields_for_preserving_frag_bitmap, update_mode: match update_mode { 0 => Some(UpdateMode::RewriteRows), @@ -3014,23 +2996,11 @@ impl TryFrom for Transaction { .collect::>>()?, }, Some(pb::transaction::Operation::UpdateMemWalState( - pb::transaction::UpdateMemWalState { - added, - updated, - removed, - }, + pb::transaction::UpdateMemWalState { merged_generations }, )) => Operation::UpdateMemWalState { - added: added + merged_generations: merged_generations .into_iter() - .map(|m| MemWal::try_from(m).unwrap()) - .collect(), - updated: updated - .into_iter() - .map(|m| MemWal::try_from(m).unwrap()) - .collect(), - removed: removed - .into_iter() - .map(|m| MemWal::try_from(m).unwrap()) + .map(|m| MergedGeneration::try_from(m).unwrap()) .collect(), }, Some(pb::transaction::Operation::UpdateBases(pb::transaction::UpdateBases { @@ -3225,7 +3195,7 @@ impl From<&Transaction> for pb::Transaction { updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge, + merged_generations, fields_for_preserving_frag_bitmap, update_mode, inserted_rows_filter, @@ -3237,7 +3207,10 @@ impl From<&Transaction> for pb::Transaction { .collect(), new_fragments: new_fragments.iter().map(pb::DataFragment::from).collect(), fields_modified: fields_modified.clone(), - mem_wal_to_merge: mem_wal_to_merge.as_ref().map(|m| m.into()), + merged_generations: merged_generations + .iter() + .map(pb::MergedGeneration::from) + .collect(), fields_for_preserving_frag_bitmap: fields_for_preserving_frag_bitmap.clone(), update_mode: update_mode .as_ref() @@ -3288,23 +3261,11 @@ impl From<&Transaction> for pb::Transaction { .collect(), }) } - Operation::UpdateMemWalState { - added, - updated, - removed, - } => { + Operation::UpdateMemWalState { merged_generations } => { pb::transaction::Operation::UpdateMemWalState(pb::transaction::UpdateMemWalState { - added: added - .iter() - .map(pb::mem_wal_index_details::MemWal::from) - .collect::>(), - updated: updated - .iter() - .map(pb::mem_wal_index_details::MemWal::from) - .collect::>(), - removed: removed + merged_generations: merged_generations .iter() - .map(pb::mem_wal_index_details::MemWal::from) + .map(pb::MergedGeneration::from) .collect::>(), }) } diff --git a/rust/lance/src/dataset/write/commit.rs b/rust/lance/src/dataset/write/commit.rs index 5070ee5e65f..9b5855efb77 100644 --- a/rust/lance/src/dataset/write/commit.rs +++ b/rust/lance/src/dataset/write/commit.rs @@ -758,7 +758,7 @@ mod tests { new_fragments: vec![], removed_fragment_ids: vec![], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, diff --git a/rust/lance/src/dataset/write/merge_insert.rs b/rust/lance/src/dataset/write/merge_insert.rs index 02af35fb519..de1d9c987cc 100644 --- a/rust/lance/src/dataset/write/merge_insert.rs +++ b/rust/lance/src/dataset/write/merge_insert.rs @@ -94,8 +94,7 @@ use lance_datafusion::{ utils::StreamingWriteSource, }; use lance_file::version::LanceFileVersion; -use lance_index::mem_wal::{MemWal, MemWalId}; -use lance_index::metrics::NoOpMetricsCollector; +use lance_index::mem_wal::MergedGeneration; use lance_index::{DatasetIndexExt, IndexCriteria}; use lance_table::format::{Fragment, IndexMetadata, RowIdMeta}; use log::info; @@ -313,9 +312,8 @@ struct MergeInsertParams { delete_not_matched_by_source: WhenNotMatchedBySource, conflict_retries: u32, retry_timeout: Duration, - // If set, this MemWAL should be marked as merged, and will be committed to replace the - // MemWAL that is currently in the index with the same ID. - mem_wal_to_merge: Option, + // List of MemWAL region generations to mark as merged when this commit succeeds. + merged_generations: Vec, // If true, skip auto cleanup during commits. This should be set to true // for high frequency writes to improve performance. This is also useful // if the writer does not have delete permissions and the clean up would @@ -426,7 +424,7 @@ impl MergeInsertBuilder { delete_not_matched_by_source: WhenNotMatchedBySource::Keep, conflict_retries: 10, retry_timeout: Duration::from_secs(30), - mem_wal_to_merge: None, + merged_generations: Vec::new(), skip_auto_cleanup: false, use_index: true, source_dedupe_behavior: SourceDedupeBehavior::Fail, @@ -513,45 +511,11 @@ impl MergeInsertBuilder { self } - /// Indicate that this merge-insert uses data in a flushed MemTable. - /// Once write is completed, the corresponding MemTable should also be marked as merged. - pub async fn mark_mem_wal_as_merged( - &mut self, - mem_wal_id: MemWalId, - expected_owner_id: &str, - ) -> Result<&mut Self> { - if let Some(mem_wal_index) = self - .dataset - .open_mem_wal_index(&NoOpMetricsCollector) - .await? - { - if let Some(generations) = mem_wal_index.mem_wal_map.get(mem_wal_id.region.as_str()) { - if let Some(mem_wal) = generations.get(&mem_wal_id.generation) { - mem_wal.check_state(lance_index::mem_wal::State::Flushed)?; - mem_wal.check_expected_owner_id(expected_owner_id)?; - self.params.mem_wal_to_merge = Some(mem_wal.clone()); - Ok(self) - } else { - Err(Error::invalid_input( - format!( - "Cannot find MemWAL generation {} for region {}", - mem_wal_id.generation, mem_wal_id.region - ), - location!(), - )) - } - } else { - Err(Error::invalid_input( - format!("Cannot find MemWAL for region {}", mem_wal_id.region), - location!(), - )) - } - } else { - Err(Error::NotSupported { - source: "MemWAL is not enabled".into(), - location: location!(), - }) - } + /// Mark MemWAL region generations as merged when this commit succeeds. + /// This updates the merged_generations in the MemWAL Index atomically with the data commit. + pub fn mark_generations_as_merged(&mut self, generations: Vec) -> &mut Self { + self.params.merged_generations.extend(generations); + self } /// Crate a merge insert job @@ -1590,7 +1554,7 @@ impl MergeInsertJob { updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge: self.params.mem_wal_to_merge, + merged_generations: self.params.merged_generations.clone(), fields_for_preserving_frag_bitmap: vec![], // in-place update do not affect preserving frag bitmap update_mode: Some(RewriteColumns), inserted_rows_filter: None, // not implemented for v1 @@ -1661,7 +1625,7 @@ impl MergeInsertJob { // On this path we only make deletions against updated_fragments and will not // modify any field values. fields_modified: vec![], - mem_wal_to_merge: self.params.mem_wal_to_merge, + merged_generations: self.params.merged_generations.clone(), fields_for_preserving_frag_bitmap: full_schema .fields .iter() diff --git a/rust/lance/src/dataset/write/merge_insert/exec/delete.rs b/rust/lance/src/dataset/write/merge_insert/exec/delete.rs index 69fc71246ea..3bc75100b9e 100644 --- a/rust/lance/src/dataset/write/merge_insert/exec/delete.rs +++ b/rust/lance/src/dataset/write/merge_insert/exec/delete.rs @@ -260,7 +260,7 @@ impl ExecutionPlan for DeleteOnlyMergeInsertExec { let merge_stats_holder = self.merge_stats.clone(); let transaction_holder = self.transaction.clone(); let affected_rows_holder = self.affected_rows.clone(); - let mem_wal_to_merge = self.params.mem_wal_to_merge.clone(); + let merged_generations = self.params.merged_generations.clone(); let result_stream = futures::stream::once(async move { let delete_row_addrs = Self::collect_deletions(input_stream, metrics).await?; @@ -275,7 +275,7 @@ impl ExecutionPlan for DeleteOnlyMergeInsertExec { updated_fragments, new_fragments: vec![], fields_modified: vec![], - mem_wal_to_merge, + merged_generations, fields_for_preserving_frag_bitmap: dataset .schema() .fields diff --git a/rust/lance/src/dataset/write/merge_insert/exec/write.rs b/rust/lance/src/dataset/write/merge_insert/exec/write.rs index 3d5527f47bd..87714e6d46d 100644 --- a/rust/lance/src/dataset/write/merge_insert/exec/write.rs +++ b/rust/lance/src/dataset/write/merge_insert/exec/write.rs @@ -856,7 +856,7 @@ impl ExecutionPlan for FullSchemaMergeInsertExec { let transaction_holder = self.transaction.clone(); let affected_rows_holder = self.affected_rows.clone(); let inserted_rows_filter_holder = self.inserted_rows_filter.clone(); - let mem_wal_to_merge = self.params.mem_wal_to_merge.clone(); + let merged_generations = self.params.merged_generations.clone(); let is_primary_key = self.is_primary_key; let updating_row_ids = { let state = merge_state.lock().unwrap(); @@ -927,7 +927,7 @@ impl ExecutionPlan for FullSchemaMergeInsertExec { updated_fragments, new_fragments, fields_modified: vec![], // No fields are modified in schema for upsert - mem_wal_to_merge, + merged_generations, fields_for_preserving_frag_bitmap: dataset .schema() .fields diff --git a/rust/lance/src/dataset/write/update.rs b/rust/lance/src/dataset/write/update.rs index 99c5fc8d6f8..47e86457f7d 100644 --- a/rust/lance/src/dataset/write/update.rs +++ b/rust/lance/src/dataset/write/update.rs @@ -388,7 +388,7 @@ impl UpdateJob { // are moved(deleted and appended). // so we do not need to handle the frag bitmap of the index about it. fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap, update_mode: Some(RewriteRows), inserted_rows_filter: None, diff --git a/rust/lance/src/index/mem_wal.rs b/rust/lance/src/index/mem_wal.rs index bb1d93b3834..55a00b41d8a 100644 --- a/rust/lance/src/index/mem_wal.rs +++ b/rust/lance/src/index/mem_wal.rs @@ -1,21 +1,28 @@ // SPDX-License-Identifier: Apache-2.0 // SPDX-FileCopyrightText: Copyright The Lance Authors -use crate::dataset::transaction::{Operation, Transaction}; -use crate::index::DatasetIndexInternalExt; -use crate::Dataset; +//! MemWAL Index operations. +//! +//! The MemWAL Index stores: +//! - Configuration (region_specs, maintained_indexes) +//! - Merge progress (merged_generations per region) +//! - Region state snapshots (eventually consistent) +//! +//! Writers no longer update the index on every write. Instead, they update +//! region manifests directly. This module provides functions to: +//! - Load the MemWAL index +//! - Update merged generations (called during merge-insert commits) + +use std::sync::Arc; + use lance_core::{Error, Result}; -use lance_index::mem_wal::{MemWal, MemWalId, MemWalIndex, MemWalIndexDetails, MEM_WAL_INDEX_NAME}; -use lance_index::metrics::NoOpMetricsCollector; -use lance_index::{is_system_index, DatasetIndexExt}; +use lance_index::mem_wal::{MemWalIndex, MemWalIndexDetails, MergedGeneration, MEM_WAL_INDEX_NAME}; use lance_table::format::{pb, IndexMetadata}; -use prost::Message; use snafu::location; -use std::collections::{HashMap, HashSet}; -use std::sync::Arc; use uuid::Uuid; -fn load_mem_wal_index_details(index: IndexMetadata) -> Result { +/// Load MemWalIndexDetails from an IndexMetadata. +pub(crate) fn load_mem_wal_index_details(index: IndexMetadata) -> Result { if let Some(details_any) = index.index_details.as_ref() { if !details_any.type_url.ends_with("MemWalIndexDetails") { return Err(Error::Index { @@ -38,511 +45,66 @@ fn load_mem_wal_index_details(index: IndexMetadata) -> Result Result> { Ok(Arc::new(MemWalIndex::new(load_mem_wal_index_details( index, )?))) } -/// Find the latest generation -pub async fn find_latest_mem_wal_generation( - dataset: &Dataset, - region: &str, -) -> Result> { - let Some(mem_wal_index) = dataset.open_mem_wal_index(&NoOpMetricsCollector).await? else { - return Ok(None); - }; - - let Some(generations) = mem_wal_index.mem_wal_map.get(region) else { - return Ok(None); - }; - - // MemWALs of the same region is ordered increasingly by its generation - if let Some(latest_mem_wal) = generations.values().last() { - Ok(Some(latest_mem_wal.clone())) - } else { - Err(Error::Internal { - message: format!("Encountered MemWAL index mapping that has a region with an empty list of generations: {}", region), - location: location!(), - }) +/// Update merged_generations in the MemWAL index. +/// This is called during merge-insert commits to atomically record which +/// generations have been merged to the base table. +pub(crate) fn update_mem_wal_index_merged_generations( + indices: &mut Vec, + dataset_version: u64, + new_merged_generations: Vec, +) -> Result<()> { + if new_merged_generations.is_empty() { + return Ok(()); } -} - -pub async fn create_mem_wal_generation( - dataset: &mut Dataset, - region: &str, - generation: u64, - new_mem_table_location: &str, - new_wal_location: &str, - owner_id: &str, -) -> Result { - let mem_wal = MemWal::new_empty( - MemWalId::new(region, generation), - new_mem_table_location, - new_wal_location, - owner_id, - ); - let txn = Transaction::new( - dataset.manifest.version, - Operation::UpdateMemWalState { - added: vec![mem_wal.clone()], - updated: vec![], - removed: vec![], - }, - None, - ); - dataset - .apply_commit(txn, &Default::default(), &Default::default()) - .await?; - - Ok(mem_wal) -} - -/// Advance the generation of the MemWAL for the given region. -/// If the MemWAL does not exist, create one with generation 0, and -/// `expected_owner_id` should be None in this case. -/// If the MemWAL exists, seal the one with the latest generation, -/// and open one with the same name and the next generation. -/// If the MemWALIndex structure does not exist, create it along the way. -pub async fn advance_mem_wal_generation( - dataset: &mut Dataset, - region: &str, - new_mem_table_location: &str, - new_wal_location: &str, - expected_owner_id: Option<&str>, - new_owner_id: &str, -) -> Result<()> { - let transaction = if let Some(mem_wal_index) = - dataset.open_mem_wal_index(&NoOpMetricsCollector).await? - { - let (added_mem_wal, updated_mem_wal, removed_mem_wal) = if let Some(generations) = - mem_wal_index.mem_wal_map.get(region) - { - if let Some(latest_mem_wal) = generations.values().last() { - // TODO: technically should check against all WAL locations - if latest_mem_wal.wal_location == new_wal_location { - return Err(Error::invalid_input( - format!( - "Must use a different WAL location from current: {}", - latest_mem_wal.wal_location - ), - location!(), - )); - } + let pos = indices + .iter() + .position(|idx| idx.name == MEM_WAL_INDEX_NAME); - if let Some(expected_owner_id) = expected_owner_id { - latest_mem_wal.check_expected_owner_id(expected_owner_id)?; - } else { - return Err(Error::invalid_input( - format!( - "Expected creating generation 0 for MemWAL region {}, but found current latest MemWAL: {:?}", - region, latest_mem_wal - ), - location!())); - } + let new_meta = if let Some(pos) = pos { + let current_meta = indices.remove(pos); + let mut details = load_mem_wal_index_details(current_meta)?; - if latest_mem_wal.mem_table_location == new_mem_table_location { - return Err(Error::invalid_input( - format!( - "Must use a different MemTable location from current: {}", - latest_mem_wal.mem_table_location - ), - location!(), - )); + // Update merged_generations - for each region, keep the higher generation + for new_mg in new_merged_generations { + if let Some(existing) = details + .merged_generations + .iter_mut() + .find(|mg| mg.region_id == new_mg.region_id) + { + if new_mg.generation > existing.generation { + existing.generation = new_mg.generation; } - - let (updated_mem_wal, removed_mem_wal) = - if latest_mem_wal.state == lance_index::mem_wal::State::Open { - let mut updated_mem_wal = latest_mem_wal.clone(); - updated_mem_wal.state = lance_index::mem_wal::State::Sealed; - (Some(updated_mem_wal), Some(latest_mem_wal.clone())) - } else { - (None, None) - }; - - let added_mem_wal = MemWal::new_empty( - MemWalId::new(region, latest_mem_wal.id.generation + 1), - new_mem_table_location, - new_wal_location, - new_owner_id, - ); - - Ok((added_mem_wal, updated_mem_wal, removed_mem_wal)) } else { - Err(Error::Internal { - message: format!("Encountered MemWAL index mapping that has a region with an empty list of generations: {}", region), - location: location!(), - }) + details.merged_generations.push(new_mg); } - } else { - if let Some(expected_owner_id) = expected_owner_id { - return Err(Error::invalid_input( - format!( - "Expected advancing MemWAL region {} from owner ID {}, but found no generation yet", - region, expected_owner_id - ), - location!())); - } - - Ok(( - MemWal::new_empty( - MemWalId::new(region, 0), - new_mem_table_location, - new_wal_location, - new_owner_id, - ), - None, - None, - )) - }?; - - Transaction::new( - dataset.manifest.version, - Operation::UpdateMemWalState { - added: vec![added_mem_wal], - updated: updated_mem_wal.into_iter().collect(), - removed: removed_mem_wal.into_iter().collect(), - }, - None, - ) - } else { - // this is the first time the MemWAL index is created - if let Some(expected_owner_id) = expected_owner_id { - return Err(Error::invalid_input( - format!( - "Expected advancing MemWAL region {} from owner ID {}, but found no MemWAL index", - region, expected_owner_id - ), - location!())); - } - - Transaction::new( - dataset.manifest.version, - Operation::UpdateMemWalState { - added: vec![MemWal::new_empty( - MemWalId::new(region, 0), - new_mem_table_location, - new_wal_location, - new_owner_id, - )], - updated: vec![], - removed: vec![], - }, - None, - ) - }; - - dataset - .apply_commit(transaction, &Default::default(), &Default::default()) - .await -} - -/// Add a new entry to the MemWAL -pub async fn append_mem_wal_entry( - dataset: &mut Dataset, - mem_wal_region: &str, - mem_wal_generation: u64, - entry_id: u64, - expected_owner_id: &str, -) -> Result { - let mutate = |mem_wal: &MemWal| -> Result { - // Can only append to open MemWALs - mem_wal.check_state(lance_index::mem_wal::State::Open)?; - mem_wal.check_expected_owner_id(expected_owner_id)?; - - let mut updated_mem_wal = mem_wal.clone(); - let wal_entries = updated_mem_wal.wal_entries(); - updated_mem_wal.wal_entries = - pb::U64Segment::from(wal_entries.with_new_high(entry_id)?).encode_to_vec(); - Ok(updated_mem_wal) - }; - - mutate_mem_wal(dataset, mem_wal_region, mem_wal_generation, mutate).await -} - -/// Mark the specific MemWAL as sealed. -/// Typically, it is recommended to call [`advance_mem_wal_generation`] instead. -/// But this will always keep the table in a state with an unsealed MemTable. -/// Calling this function will only seal the current latest MemWAL without opening the next one. -pub async fn mark_mem_wal_as_sealed( - dataset: &mut Dataset, - mem_wal_region: &str, - mem_wal_generation: u64, - expected_owner_id: &str, -) -> Result { - let mutate = |mem_wal: &MemWal| -> Result { - // Can only seal open MemWALs - mem_wal.check_state(lance_index::mem_wal::State::Open)?; - mem_wal.check_expected_owner_id(expected_owner_id)?; - - let mut updated_mem_wal = mem_wal.clone(); - updated_mem_wal.state = lance_index::mem_wal::State::Sealed; - Ok(updated_mem_wal) - }; - - mutate_mem_wal(dataset, mem_wal_region, mem_wal_generation, mutate).await -} - -/// Mark the specific MemWAL as flushed (data on disk but not merged) -pub async fn mark_mem_wal_as_flushed( - dataset: &mut Dataset, - mem_wal_region: &str, - mem_wal_generation: u64, - expected_owner_id: &str, -) -> Result { - let mutate = |mem_wal: &MemWal| -> Result { - // Can only flush sealed MemWALs - mem_wal.check_state(lance_index::mem_wal::State::Sealed)?; - mem_wal.check_expected_owner_id(expected_owner_id)?; - - let mut updated_mem_wal = mem_wal.clone(); - updated_mem_wal.state = lance_index::mem_wal::State::Flushed; - Ok(updated_mem_wal) - }; - - mutate_mem_wal(dataset, mem_wal_region, mem_wal_generation, mutate).await -} - -/// Mark the specific MemWAL as merged (data merged into source table) -pub async fn mark_mem_wal_as_merged( - dataset: &mut Dataset, - mem_wal_region: &str, - mem_wal_generation: u64, - expected_owner_id: &str, -) -> Result { - let mutate = |mem_wal: &MemWal| -> Result { - // Can only merge flushed MemWALs - mem_wal.check_state(lance_index::mem_wal::State::Flushed)?; - mem_wal.check_expected_owner_id(expected_owner_id)?; - - let mut updated_mem_wal = mem_wal.clone(); - updated_mem_wal.state = lance_index::mem_wal::State::Merged; - Ok(updated_mem_wal) - }; - - mutate_mem_wal(dataset, mem_wal_region, mem_wal_generation, mutate).await -} - -/// Mark the specific MemWAL as flushed, in the list of indices in the dataset. -/// This is intended to be used as a part of the Update transaction after resolving all conflicts. -pub(crate) fn update_mem_wal_index_in_indices_list( - dataset_read_version: u64, - dataset_new_version: u64, - indices: &mut Vec, - added: Vec, - updated: Vec, - removed: Vec, -) -> Result<()> { - let new_meta = if let Some(pos) = indices - .iter() - .position(|idx| idx.name == MEM_WAL_INDEX_NAME) - { - let current_meta = indices.remove(pos); - let mut details = load_mem_wal_index_details(current_meta)?; - let removed_set = removed - .iter() - .map(|rm| rm.id.clone()) - .collect::>(); - details - .mem_wal_list - .retain(|m| !removed_set.contains(&m.id)); - - for mut mem_wal in added.into_iter() { - mem_wal.last_updated_dataset_version = dataset_new_version; - details.mem_wal_list.push(mem_wal); - } - - for mut mem_wal in updated.into_iter() { - mem_wal.last_updated_dataset_version = dataset_new_version; - details.mem_wal_list.push(mem_wal); } - new_mem_wal_index_meta(dataset_read_version, details.mem_wal_list)? + new_mem_wal_index_meta(dataset_version, details)? } else { - // This should only happen with new index creation when opening the first MemWAL - if !updated.is_empty() || !removed.is_empty() { - return Err(Error::invalid_input( - "Cannot update MemWAL state without a MemWAL index", - location!(), - )); - } - - let mut added_with_version = Vec::with_capacity(added.len()); - for mut mem_wal in added.into_iter() { - mem_wal.last_updated_dataset_version = dataset_new_version; - added_with_version.push(mem_wal); - } - - new_mem_wal_index_meta(dataset_read_version, added_with_version)? + // Create new MemWAL index with just the merged generations + let details = MemWalIndexDetails { + merged_generations: new_merged_generations, + ..Default::default() + }; + new_mem_wal_index_meta(dataset_version, details)? }; indices.push(new_meta); Ok(()) } -/// Owner ID serves as a pre-check that the MemWAL has not changed owner before commit. -/// Each writer is required to keep an invariant of its owner ID for a MemWAL. -/// At any point in time, there should be only 1 writer that owns the right to mutate the MemWAL, -/// and the owner ID serves as the optimistic lock for it. -/// Specifically, before a writer starts to replay a WAL, it should call this method to claim -/// ownership and stop any additional writes to the MemWAL from other writers. -/// -/// Consider a distributed cluster which currently has node A writing to the table's MemWAL. -/// A network partition happens, node A is not dead but fails the health check. -/// Node B is newly assigned and starts the WAL replay process which modifies the owner ID. -/// In this case, if node A is doing a modification to the same MemWAL including adding an entry, -/// sealing or flushing, advancing the MemWAL generation, it will receive a commit conflict failure. -/// In theory, all the writes from node A should abort after seeing this failure without retrying. -/// However, if the writer decides to retry the operation for any reason (e.g. a bug), without the check, -/// the retry would succeed. The `expected_owner_id` in all write functions serves as the guard to -/// make sure it continues to fail until the write traffic is fully redirected to node B. -pub async fn update_mem_wal_owner( - dataset: &mut Dataset, - region: &str, - generation: u64, - new_owner_id: &str, - new_mem_table_location: Option<&str>, -) -> Result { - let mutate = |mem_wal: &MemWal| -> Result { - if new_owner_id == mem_wal.owner_id { - return Err(Error::invalid_input( - format!( - "Must use a different owner ID from current: {}", - mem_wal.owner_id - ), - location!(), - )); - } - - if let Some(new_mem_table_location) = new_mem_table_location { - if new_mem_table_location == mem_wal.mem_table_location { - return Err(Error::invalid_input( - format!( - "Must use a different MemTable location from current: {}", - mem_wal.mem_table_location - ), - location!(), - )); - } - } - - let mut updated_mem_wal = mem_wal.clone(); - updated_mem_wal.owner_id = new_owner_id.to_owned(); - if let Some(new_mem_table_location) = new_mem_table_location { - updated_mem_wal.mem_table_location = new_mem_table_location.to_owned(); - } - Ok(updated_mem_wal) - }; - - mutate_mem_wal(dataset, region, generation, mutate).await -} - -/// Trim all the MemWALs that are already merged. -pub async fn trim_mem_wal_index(dataset: &mut Dataset) -> Result<()> { - if let Some(mem_wal_index) = dataset.open_mem_wal_index(&NoOpMetricsCollector).await? { - let indices = dataset.load_indices().await?; - - // group by name to get the latest version of each index - // For delta indices, we take the highest dataset version - let mut index_versions = HashMap::new(); - for index in indices.iter() { - if !is_system_index(index) { - let current_version = index_versions.entry(index.name.clone()).or_insert(0); - *current_version = (*current_version).max(index.dataset_version); - } - } - - let min_index_dataset_version = index_versions.values().min().copied().unwrap_or(u64::MAX); - - let mut removed = Vec::new(); - for (_, generations) in mem_wal_index.mem_wal_map.iter() { - for (_, mem_wal) in generations.iter() { - if mem_wal.state == lance_index::mem_wal::State::Merged { - // all indices are caught up, can trim it - if mem_wal.last_updated_dataset_version <= min_index_dataset_version { - removed.push(mem_wal.clone()); - } - } - } - } - - let transaction = Transaction::new( - dataset.manifest.version, - Operation::UpdateMemWalState { - added: vec![], - updated: vec![], - removed, - }, - None, - ); - - dataset - .apply_commit(transaction, &Default::default(), &Default::default()) - .await - } else { - Err(Error::NotSupported { - source: "MemWAL is not enabled".into(), - location: location!(), - }) - } -} - -async fn mutate_mem_wal( - dataset: &mut Dataset, - region: &str, - generation: u64, - mutate: F, -) -> Result -where - F: Fn(&MemWal) -> Result, -{ - if let Some(mem_wal_index) = dataset.open_mem_wal_index(&NoOpMetricsCollector).await? { - if let Some(generations) = mem_wal_index.mem_wal_map.get(region) { - if let Some(mem_wal) = generations.get(&generation) { - let updated_mem_wal = mutate(mem_wal)?; - - let transaction = Transaction::new( - dataset.manifest.version, - Operation::UpdateMemWalState { - added: vec![], - updated: vec![updated_mem_wal.clone()], - removed: vec![mem_wal.clone()], - }, - None, - ); - - dataset - .apply_commit(transaction, &Default::default(), &Default::default()) - .await?; - - Ok(updated_mem_wal) - } else { - Err(Error::invalid_input( - format!( - "Cannot find MemWAL generation {} for region {}", - generation, region - ), - location!(), - )) - } - } else { - Err(Error::invalid_input( - format!("Cannot find MemWAL for region {}", region), - location!(), - )) - } - } else { - Err(Error::NotSupported { - source: "MemWAL is not enabled".into(), - location: location!(), - }) - } -} - +/// Create a new MemWAL index metadata entry. pub(crate) fn new_mem_wal_index_meta( dataset_version: u64, - new_mem_wal_list: Vec, + details: MemWalIndexDetails, ) -> Result { Ok(IndexMetadata { uuid: Uuid::new_v4(), @@ -551,9 +113,7 @@ pub(crate) fn new_mem_wal_index_meta( dataset_version, fragment_bitmap: None, index_details: Some(Arc::new(prost_types::Any::from_msg( - &pb::MemWalIndexDetails::from(&MemWalIndexDetails { - mem_wal_list: new_mem_wal_list, - }), + &pb::MemWalIndexDetails::from(&details), )?)), index_version: 0, created_at: Some(chrono::Utc::now()), @@ -564,2016 +124,77 @@ pub(crate) fn new_mem_wal_index_meta( #[cfg(test)] mod tests { use super::*; - use crate::dataset::{WriteDestination, WriteMode, WriteParams}; - use crate::index::vector::VectorIndexParams; - use crate::utils::test::{DatagenExt, FragmentCount, FragmentRowCount}; - use arrow_array::types::{Float32Type, Int32Type}; - use lance_datafusion::datagen::DatafusionDatagenExt; - use lance_datagen::{BatchCount, Dimension, RowCount}; - use lance_index::mem_wal::{MemWalId, MEM_WAL_INDEX_NAME}; - use lance_index::optimize::OptimizeOptions; - use lance_index::{DatasetIndexExt, Index}; - use lance_linalg::distance::MetricType; - - #[tokio::test] - async fn test_advance_mem_wal_generation() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Initially, there should be no MemWAL index - let indices = dataset.load_indices().await.unwrap(); - assert!(!indices.iter().any(|idx| idx.name == MEM_WAL_INDEX_NAME)); - - // First call to advance_mem_wal_generation should create the MemWAL index and generation 0 - let initial_version = dataset.manifest.version; - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Verify the MemWAL index was created - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should be created"); - - // Load and verify the MemWAL index details - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!(mem_wal_details.mem_wal_list.len(), 1); - let mem_wal_index = open_mem_wal_index(mem_wal_index_meta.clone()).unwrap(); - let stats = mem_wal_index.statistics().unwrap(); - assert_eq!( - serde_json::to_string(&stats).unwrap(), - dataset.index_statistics(MEM_WAL_INDEX_NAME).await.unwrap() - ); - - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!(mem_wal.id.region, "GLOBAL"); - assert_eq!(mem_wal.id.generation, 0); - assert_eq!(mem_wal.mem_table_location, "mem_table_location_0"); - assert_eq!(mem_wal.wal_location, "wal_location_0"); - assert_eq!(mem_wal.state, lance_index::mem_wal::State::Open); - assert_eq!(mem_wal.last_updated_dataset_version, initial_version + 1); - - // Second call to advance_mem_wal_generation should seal generation 0 and create generation 1 - let version_before_second_advance = dataset.manifest.version; - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", - ) - .await - .unwrap(); + use lance_index::mem_wal::RegionId; - // Verify the MemWAL index now has two generations - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!(mem_wal_details.mem_wal_list.len(), 2); - - // Find generation 0 (should be sealed) and generation 1 (should be unsealed) - let gen_0 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 0) - .expect("Generation 0 should exist"); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - - // Verify generation 0 is sealed - assert_eq!(gen_0.id.region, "GLOBAL"); - assert_eq!(gen_0.id.generation, 0); - assert_eq!(gen_0.mem_table_location, "mem_table_location_0"); - assert_eq!(gen_0.wal_location, "wal_location_0"); - assert_eq!(gen_0.state, lance_index::mem_wal::State::Sealed); - // Verify the sealed MemWAL has updated version - assert_eq!( - gen_0.last_updated_dataset_version, - version_before_second_advance + 1 - ); - - // Verify generation 1 is unsealed - assert_eq!(gen_1.id.region, "GLOBAL"); - assert_eq!(gen_1.id.generation, 1); - assert_eq!(gen_1.mem_table_location, "mem_table_location_1"); - assert_eq!(gen_1.wal_location, "wal_location_1"); - assert_eq!(gen_1.state, lance_index::mem_wal::State::Open); - // Verify the new MemWAL has correct version - assert_eq!( - gen_1.last_updated_dataset_version, - version_before_second_advance + 1 - ); - - // Test that using the same MemTable location should fail - let result = advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", // Same as current generation - "wal_location_2", // Different WAL location - Some("owner_1"), - "owner_2", - ) - .await; - assert!( - result.is_err(), - "Should fail when using same MemTable location as current generation" - ); - - // Test that using the same WAL location should fail - let result = advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_2", // Different MemTable location - "wal_location_1", // Same as current generation - Some("owner_1"), - "owner_2", - ) - .await; - assert!( - result.is_err(), - "Should fail when using same WAL location as current generation" - ); - } + #[test] + fn test_update_merged_generations() { + let mut indices = Vec::new(); + let region1 = RegionId::new(); + let region2 = RegionId::new(); - #[tokio::test] - async fn test_append_new_entry_to_mem_wal() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Test failure case: MemWAL is not enabled - let result = append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0").await; - assert!(result.is_err(), "Should fail when MemWAL is not enabled"); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", + // First update - creates new index + update_mem_wal_index_merged_generations( + &mut indices, + 1, + vec![MergedGeneration::new(region1.clone(), 5)], ) - .await .unwrap(); - // Test failure case: region doesn't exist - let result = append_mem_wal_entry(&mut dataset, "NONEXISTENT", 0, 123, "owner_0").await; - assert!(result.is_err(), "Should fail when region doesn't exist"); - - // Test failure case: generation doesn't exist - let result = append_mem_wal_entry(&mut dataset, "GLOBAL", 999, 123, "owner_0").await; - assert!(result.is_err(), "Should fail when generation doesn't exist"); - - // Test success case: append entry to generation 0 - let version_before_append = dataset.manifest.version; - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0") - .await - .unwrap(); - - // Verify the entry was added - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - - // Check that the WAL entries contain the entry_id - let wal_entries = mem_wal.wal_entries(); - assert!( - wal_entries.contains(123), - "WAL entries should contain entry_id 123" - ); - // Verify the MemWAL version was updated after append - assert_eq!( - mem_wal.last_updated_dataset_version, - version_before_append + 1 - ); - - // Test appending multiple entries - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 456, "owner_0") - .await - .unwrap(); - let version_after_second_append = dataset.manifest.version; - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 789, "owner_0") - .await - .unwrap(); - - // Verify all entries were added - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - - let wal_entries = mem_wal.wal_entries(); - assert!( - wal_entries.contains(123), - "WAL entries should contain entry_id 123" - ); - assert!( - wal_entries.contains(456), - "WAL entries should contain entry_id 456" - ); - assert!( - wal_entries.contains(789), - "WAL entries should contain entry_id 789" - ); - // Verify the MemWAL version was updated after the last append - assert_eq!( - mem_wal.last_updated_dataset_version, - version_after_second_append + 1 - ); - - // Test failure case: cannot append to sealed MemWAL - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - let result = append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 999, "owner_0").await; - assert!( - result.is_err(), - "Should fail when trying to append to sealed MemWAL" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Sealed, but expected Open"), - "Error message should indicate the MemWAL is sealed, got: {}", error); - - // Test failure case: cannot append to flushed MemWAL - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - let result = append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 999, "owner_0").await; - assert!( - result.is_err(), - "Should fail when trying to append to flushed MemWAL" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Flushed, but expected Open"), - "Error message should indicate the MemWAL is flushed, got: {}", error); - } - - #[tokio::test] - async fn test_seal_mem_wal() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Test failure case: MemWAL is not enabled - let result = mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!(result.is_err(), "Should fail when MemWAL is not enabled"); + assert_eq!(indices.len(), 1); + let details = load_mem_wal_index_details(indices[0].clone()).unwrap(); + assert_eq!(details.merged_generations.len(), 1); + assert_eq!(details.merged_generations[0].region_id, region1); + assert_eq!(details.merged_generations[0].generation, 5); - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", + // Second update - updates existing region + update_mem_wal_index_merged_generations( + &mut indices, + 2, + vec![MergedGeneration::new(region1.clone(), 10)], ) - .await .unwrap(); - // Test failure case: region doesn't exist - let result = mark_mem_wal_as_sealed(&mut dataset, "NONEXISTENT", 0, "owner_0").await; - assert!(result.is_err(), "Should fail when region doesn't exist"); - - // Test failure case: generation doesn't exist - let result = mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 999, "owner_0").await; - assert!(result.is_err(), "Should fail when generation doesn't exist"); - - // Verify generation 0 is initially unsealed - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Open, - "Generation 0 should initially be open" - ); - - // Test success case: seal generation 0 - let version_before_seal = dataset.manifest.version; - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Verify generation 0 is now sealed - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Sealed, - "Generation 0 should now be sealed" - ); - // Verify the MemWAL version was updated after sealing - assert_eq!( - mem_wal.last_updated_dataset_version, - version_before_seal + 1 - ); + assert_eq!(indices.len(), 1); + let details = load_mem_wal_index_details(indices[0].clone()).unwrap(); + assert_eq!(details.merged_generations.len(), 1); + assert_eq!(details.merged_generations[0].generation, 10); - // Create a new generation and test sealing it - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", + // Third update - adds new region + update_mem_wal_index_merged_generations( + &mut indices, + 3, + vec![MergedGeneration::new(region2, 3)], ) - .await .unwrap(); - // Verify generation 1 is unsealed - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Open, - "Generation 1 should be open" - ); - - // Seal generation 1 - let version_before_seal_gen1 = dataset.manifest.version; - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - - // Verify it's sealed - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Sealed, - "Generation 1 should be sealed" - ); - // Verify the MemWAL version was updated after sealing generation 1 - assert_eq!( - gen_1.last_updated_dataset_version, - version_before_seal_gen1 + 1 - ); - - // Test that sealing an already sealed MemWAL should fail - let result = mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 1, "owner_1").await; - assert!( - result.is_err(), - "Should fail when trying to seal an already sealed MemWAL" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 1 } is in state Sealed, but expected Open"), - "Error message should indicate the MemWAL is not open, got: {}", error); + assert_eq!(indices.len(), 1); + let details = load_mem_wal_index_details(indices[0].clone()).unwrap(); + assert_eq!(details.merged_generations.len(), 2); - // Test that sealing an already flushed MemWAL should fail - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - let result = mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!( - result.is_err(), - "Should fail when trying to seal an already flushed MemWAL" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Flushed, but expected Open"), - "Error message should indicate the MemWAL is already flushed, got: {}", error); - } - - #[tokio::test] - async fn test_flush_and_merge_mem_wal() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Test failure case: MemWAL is not enabled - let result = mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!(result.is_err(), "Should fail when MemWAL is not enabled"); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", + // Fourth update - lower generation should not update + update_mem_wal_index_merged_generations( + &mut indices, + 4, + vec![MergedGeneration::new(region1.clone(), 8)], // lower than 10 ) - .await .unwrap(); - // Test failure case: region doesn't exist - let result = mark_mem_wal_as_flushed(&mut dataset, "NONEXISTENT", 0, "owner_0").await; - assert!(result.is_err(), "Should fail when region doesn't exist"); - - // Test failure case: generation doesn't exist - let result = mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 999, "owner_0").await; - assert!(result.is_err(), "Should fail when generation doesn't exist"); - - // Verify generation 0 is initially unflushed - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Open, - "Generation 0 should initially be open" - ); - - // Test failure case: cannot flush unsealed MemWAL - let result = mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!( - result.is_err(), - "Should fail when trying to flush unsealed MemWAL" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Open, but expected Sealed"), - "Error message should indicate the MemWAL is not sealed, got: {}", error); - - // Seal generation 0 first - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Test success case: mark sealed generation 0 as flushed - let version_before_flush = dataset.manifest.version; - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Verify generation 0 is now flushed - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Flushed, - "Generation 0 should now be flushed" - ); - // Verify the MemWAL version was updated after flushing - assert_eq!( - mem_wal.last_updated_dataset_version, - version_before_flush + 1 - ); - - // Test failure case: cannot flush already flushed MemWAL - let result = mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!( - result.is_err(), - "Should fail when trying to flush already flushed MemWAL" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Flushed, but expected Sealed"), - "Error message should indicate the MemWAL is already flushed, got: {}", error); - - // Test success case: mark flushed generation 0 as merged - let version_before_merge = dataset.manifest.version; - mark_mem_wal_as_merged(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Verify generation 0 is now merged - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices + let details = load_mem_wal_index_details(indices[0].clone()).unwrap(); + let r1_mg = details + .merged_generations .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Merged, - "Generation 0 should now be merged" - ); - // Verify the MemWAL version was updated after merging - assert_eq!( - mem_wal.last_updated_dataset_version, - version_before_merge + 1 - ); - - // Test failure case: cannot merge already merged MemWAL - let result = mark_mem_wal_as_merged(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!( - result.is_err(), - "Should fail when trying to merge already merged MemWAL" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Merged, but expected Flushed"), - "Error message should indicate the MemWAL is already merged, got: {}", error); - } - - #[tokio::test] - async fn test_update_mem_wal_owner() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await + .find(|mg| mg.region_id == region1) .unwrap(); - - // Test failure case: MemWAL is not enabled - let result = update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 0, - "new_owner_id", - Some("new_mem_table_location"), - ) - .await; - assert!(result.is_err(), "Should fail when MemWAL is not enabled"); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Test failure case: region doesn't exist - let result = update_mem_wal_owner( - &mut dataset, - "NONEXISTENT", - 0, - "new_owner_id", - Some("new_mem_table_location"), - ) - .await; - assert!(result.is_err(), "Should fail when region doesn't exist"); - - // Test failure case: generation doesn't exist - let result = update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 999, - "new_owner_id", - Some("new_mem_table_location"), - ) - .await; - assert!(result.is_err(), "Should fail when generation doesn't exist"); - - // Test failure case: cannot replay with same MemTable location - let result = update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 0, - "new_owner_id", - Some("mem_table_location_0"), - ) - .await; - assert!( - result.is_err(), - "Should fail when using same MemTable location" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!( - error.to_string().contains( - "Must use a different MemTable location from current: mem_table_location_0" - ), - "Error message should indicate the MemTable location must be different, got: {}", - error - ); - - // Test success case: start replay with different MemTable location - let version_before_owner_update = dataset.manifest.version; - update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 0, - "new_owner_id", - Some("new_mem_table_location"), - ) - .await - .unwrap(); - - // Verify the MemTable location was updated - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.mem_table_location, "new_mem_table_location", - "MemTable location should be updated" - ); - // Verify the MemWAL version was updated after owner change - assert_eq!( - mem_wal.last_updated_dataset_version, - version_before_owner_update + 1 - ); - - // Test success case: can replay generation 1 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "new_mem_table_location_1", - "wal_location_1", - Some("new_owner_id"), - "owner_1", - ) - .await - .unwrap(); - - let version_before_gen1_owner_update = dataset.manifest.version; - update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 1, - "owner_1_new", - Some("mem_table_location_1"), - ) - .await - .unwrap(); - - // Verify the MemTable location was updated for generation 1 - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - - assert_eq!( - gen_1.mem_table_location, "mem_table_location_1", - "Generation 1 MemTable location should be updated" - ); - // Verify the MemWAL version was updated after generation 1 owner change - assert_eq!( - gen_1.last_updated_dataset_version, - version_before_gen1_owner_update + 1 - ); + assert_eq!(r1_mg.generation, 10); // Should still be 10 } - #[tokio::test] - async fn test_trim_mem_wal_index_with_reindex() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Test failure case: MemWAL is not enabled - let result = trim_mem_wal_index(&mut dataset).await; - assert!(result.is_err(), "Should fail when MemWAL is not enabled"); - - // Create MemWAL index and multiple generations - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", - ) - .await - .unwrap(); - - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_2", - "wal_location_2", - Some("owner_1"), - "owner_2", - ) - .await - .unwrap(); - - // Verify we have 3 generations initially - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!( - mem_wal_details.mem_wal_list.len(), - 3, - "Should have 3 generations initially" - ); - - // flush and merge generation 0 - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - mark_mem_wal_as_merged(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Test case 1: No indices exist (besides MemWAL index itself) - // Should trim merged MemWAL since no other indices exist - trim_mem_wal_index(&mut dataset).await.unwrap(); - - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!( - mem_wal_details.mem_wal_list.len(), - 2, - "Should have 2 generations after trimming (no other indices)" - ); - - // Verify generation 0 was removed - let gen_0_exists = mem_wal_details - .mem_wal_list - .iter() - .any(|m| m.id.generation == 0); - assert!(!gen_0_exists, "Generation 0 should be removed"); - - // Test case 2: Create index after MemWAL flush, then flush another generation - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_3", - "wal_location_3", - Some("owner_2"), - "owner_3", - ) - .await - .unwrap(); - - // Seal, flush and merge generation 1 - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - mark_mem_wal_as_merged(&mut dataset, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - - // Create an index after the MemWAL was merged - dataset - .create_index( - &["i"], - lance_index::IndexType::Scalar, - Some("scalar_after".into()), - &lance_index::scalar::ScalarIndexParams::default(), - false, - ) - .await - .unwrap(); - - // Should trim the merged MemWAL since the index was created after it - trim_mem_wal_index(&mut dataset).await.unwrap(); - - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!( - mem_wal_details.mem_wal_list.len(), - 2, - "Should have 2 generations after trimming (index created after MemWAL)" - ); - - // Verify generation 1 was removed - let gen_1_exists = mem_wal_details - .mem_wal_list - .iter() - .any(|m| m.id.generation == 1); - assert!(!gen_1_exists, "Generation 1 should be removed"); - - // Test case 3: Create index before MemWAL flush - // Create another index before flushing the next generation - dataset - .create_index( - &["i"], - lance_index::IndexType::Scalar, - Some("scalar_before".into()), - &lance_index::scalar::ScalarIndexParams::default(), - false, - ) - .await - .unwrap(); - - // Now flush and merge generation 2 (created before the vector index) - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 2, "owner_2") - .await - .unwrap(); - mark_mem_wal_as_merged(&mut dataset, "GLOBAL", 2, "owner_2") - .await - .unwrap(); - - // Should NOT trim generation 2 since the index was created before it - trim_mem_wal_index(&mut dataset).await.unwrap(); - - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!( - mem_wal_details.mem_wal_list.len(), - 2, - "Should still have 2 generations (index created before MemWAL, so cannot trim)" - ); - - // Verify generation 2 still exists - let gen_2_exists = mem_wal_details - .mem_wal_list - .iter() - .any(|m| m.id.generation == 2); - assert!(gen_2_exists, "Generation 2 should still exist"); - } - - #[tokio::test] - async fn test_trim_mem_wal_index_with_delta_index() { - // Create a dataset with enough data for vector index clustering - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(5), FragmentRowCount::from(100)) - .await - .unwrap(); - - // Create initial vector index - dataset - .create_index( - &["vec"], - lance_index::IndexType::Vector, - Some("vector_index".into()), - &VectorIndexParams::ivf_pq(8, 8, 8, MetricType::Cosine, 50), - false, - ) - .await - .unwrap(); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Seal the MemWAL - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Append new data files to the dataset (without rewriting existing files) - let new_data = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col( - "i", - lance_datagen::array::step_custom::(500, 1000), - ) - .into_reader_rows(RowCount::from(100), BatchCount::from(5)); - - // Append some new data - let write_params = WriteParams { - mode: WriteMode::Append, - ..WriteParams::default() - }; - dataset = Dataset::write( - new_data, - WriteDestination::Dataset(Arc::new(dataset)), - Some(write_params), - ) - .await - .unwrap(); - - // Flush and merge the MemWAL separately - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - mark_mem_wal_as_merged(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Verify the MemWAL is now merged - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!(mem_wal_details.mem_wal_list.len(), 1); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!(mem_wal.state, lance_index::mem_wal::State::Merged); - - // Now use optimize_indices to create delta index (this is how delta indices are actually created) - dataset - .optimize_indices(&OptimizeOptions::append()) - .await - .unwrap(); - - // Verify we now have multiple indices with the same name (delta indices) - let indices = dataset.load_indices().await.unwrap(); - let vector_indices: Vec<_> = indices - .iter() - .filter(|idx| idx.name == "vector_index") - .collect(); - assert_eq!(vector_indices.len(), 2); - // If we have delta indices, verify they work correctly - // Verify the delta index has a higher dataset version than the original - let mut versions: Vec<_> = vector_indices - .iter() - .map(|idx| idx.dataset_version) - .collect(); - versions.sort(); - assert!( - versions[versions.len() - 1] > versions[0], - "Latest delta index should have higher dataset version than original" - ); - - // Now the MemWAL should be trimmed because the delta index was created after the merge - // Our logic should take the maximum dataset version for each index name - trim_mem_wal_index(&mut dataset).await.unwrap(); - - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - assert_eq!( - mem_wal_details.mem_wal_list.len(), - 0, - "MemWAL should be trimmed because delta index was created after flush" - ); - } - - #[tokio::test] - async fn test_flush_mem_wal_through_merge_insert() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Add some entries to the MemWAL - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 456, "owner_0") - .await - .unwrap(); - - // Seal and flush the MemWAL (required before merging) - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Verify the MemWAL is flushed but not merged - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Flushed, - "MemWAL should be flushed but not merged yet" - ); - - // Create new data for merge insert - let new_data = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step_custom::(1000, 1)) - .into_df_stream(RowCount::from(100), BatchCount::from(10)); - - // Create merge insert job that will merge the MemWAL - let merge_insert_job = crate::dataset::MergeInsertBuilder::try_new( - Arc::new(dataset.clone()), - vec!["i".to_string()], - ) - .unwrap() - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll) - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 0), "owner_0") - .await - .unwrap() - .try_build() - .unwrap(); - - // Execute the merge insert - let (updated_dataset, _stats) = merge_insert_job.execute_reader(new_data).await.unwrap(); - - // Verify that the MemWAL is now marked as merged - let indices = updated_dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.state, - lance_index::mem_wal::State::Merged, - "MemWAL should now be merged" - ); - - // Test that trying to mark a non-existent MemWAL as merged fails - let mut merge_insert_job = crate::dataset::MergeInsertBuilder::try_new( - updated_dataset.clone(), - vec!["i".to_string()], - ) - .unwrap(); - merge_insert_job - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll); - - let result = merge_insert_job - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 999), "owner_0") - .await; - assert!( - result.is_err(), - "Should fail when trying to mark non-existent MemWAL as merged" - ); - - // Test that trying to mark a MemWAL from non-existent region fails - let result = merge_insert_job - .mark_mem_wal_as_merged(MemWalId::new("NONEXISTENT", 0), "owner_0") - .await; - assert!( - result.is_err(), - "Should fail when trying to mark MemWAL from non-existent region as merged" - ); - - // Test that trying to mark an unflushed MemWAL as merged fails - // First, create a new generation that is unsealed - let mut dataset_for_advance = updated_dataset.as_ref().clone(); - advance_mem_wal_generation( - &mut dataset_for_advance, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", - ) - .await - .unwrap(); - - // Update our reference to use the new dataset - let updated_dataset = Arc::new(dataset_for_advance); - - // Verify that generation 1 exists and is unsealed - let indices = updated_dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Open, - "Generation 1 should be open" - ); - - let mut merge_insert_job_unsealed = crate::dataset::MergeInsertBuilder::try_new( - updated_dataset.clone(), - vec!["i".to_string()], - ) - .unwrap(); - merge_insert_job_unsealed - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll); - - let result = merge_insert_job_unsealed - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 1), "owner_1") - .await; - assert!( - result.is_err(), - "Should fail when trying to mark unsealed MemWAL as merged" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 1 } is in state Open, but expected Flushed"), - "Error message should indicate the MemWAL is not flushed, got: {}", error); - - // Test that trying to mark an already merged MemWAL as merged fails - let mut merge_insert_job_merged = crate::dataset::MergeInsertBuilder::try_new( - updated_dataset.clone(), - vec!["i".to_string()], - ) - .unwrap(); - merge_insert_job_merged - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll); - - let result = merge_insert_job_merged - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 0), "owner_1") - .await; - assert!( - result.is_err(), - "Should fail when trying to mark already merged MemWAL as merged" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } is in state Merged, but expected Flushed"), - "Error message should indicate the MemWAL is already merged, got: {}", error); - - // Test that merge insert with mark_mem_wal_as_merged works correctly when MemWAL is in proper state - // Seal and flush generation 1 and then test the merge insert - let mut dataset_for_seal = updated_dataset.as_ref().clone(); - mark_mem_wal_as_sealed(&mut dataset_for_seal, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - mark_mem_wal_as_flushed(&mut dataset_for_seal, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - let updated_dataset = Arc::new(dataset_for_seal); - - // Verify generation 1 is now flushed but not merged - let indices = updated_dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Flushed, - "Generation 1 should be flushed" - ); - - // Create merge insert that merges generation 1 - let new_data_valid = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step_custom::(4000, 1)) - .into_df_stream(RowCount::from(75), BatchCount::from(5)); - - let merge_insert_job_valid = crate::dataset::MergeInsertBuilder::try_new( - updated_dataset.clone(), - vec!["i".to_string()], - ) - .unwrap() - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll) - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 1), "owner_1") - .await - .unwrap() - .try_build() - .unwrap(); - - // Execute the merge insert - this should succeed - let (final_dataset, _stats) = merge_insert_job_valid - .execute_reader(new_data_valid) - .await - .unwrap(); - - // Verify that the MemWAL is now marked as merged - let indices = final_dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should still exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should still exist"); - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Merged, - "Generation 1 should now be merged" - ); - } - - #[tokio::test] - async fn test_replay_mem_wal_with_split_brain_writer() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Add some entries to the MemWAL - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 456, "owner_0") - .await - .unwrap(); - - // Simulate a network partition scenario where another node starts replay - // This changes the MemTable location from "mem_table_location_0" to "new_mem_table_location" - update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 0, - "new_owner_id", - Some("new_mem_table_location"), - ) - .await - .unwrap(); - - // Verify the MemTable location was updated - let indices = dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - let mem_wal = &mem_wal_details.mem_wal_list[0]; - assert_eq!( - mem_wal.mem_table_location, "new_mem_table_location", - "MemTable location should be updated after replay" - ); - - // Now simulate a split-brain scenario where the original writer (node A) - // tries to perform operations using the old MemTable location - - // Test 1: append_mem_wal_entry with old owner_id should fail - let result = append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 789, "owner_0").await; - assert!( - result.is_err(), - "Should fail when using old owner_id for append" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } has owner_id: new_owner_id, but expected owner_0"), - "Error message should indicate owner_id mismatch, got: {}", error); - - // Test 2: mark_mem_wal_as_sealed with old owner_id should fail - let result = mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!( - result.is_err(), - "Should fail when using old owner_id for seal" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } has owner_id: new_owner_id, but expected owner_0"), - "Error message should indicate owner_id mismatch, got: {}", error); - - // Test 3: mark_mem_wal_as_flushed with old owner_id should fail - // First seal the MemWAL using the correct owner_id - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "new_owner_id") - .await - .unwrap(); - - let result = mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0").await; - assert!( - result.is_err(), - "Should fail when using old owner_id for flush" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } has owner_id: new_owner_id, but expected owner_0"), - "Error message should indicate owner_id mismatch, got: {}", error); - - // Test 4: advance_mem_wal_generation with old owner_id should fail - let result = advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), // Using old owner_id - "owner_1", - ) - .await; - assert!( - result.is_err(), - "Should fail when using old owner_id for advance generation" - ); - - // Check the specific error message - let error = result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } has owner_id: new_owner_id, but expected owner_0"), - "Error message should indicate owner_id mismatch, got: {}", error); - - // Test 5: merge_insert with mark_mem_wal_as_merged using old owner_id should fail - // First flush the MemWAL using the correct owner_id so it's ready for merging - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "new_owner_id") - .await - .unwrap(); - - // Try to create merge insert job that merges using the old owner_id - let mut merge_insert_job_builder = crate::dataset::MergeInsertBuilder::try_new( - Arc::new(dataset.clone()), - vec!["i".to_string()], - ) - .unwrap(); - - let build_result = merge_insert_job_builder - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll) - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 0), "owner_0") // Using old owner_id - .await; - - assert!( - build_result.is_err(), - "Should fail when using old owner_id for merge insert merge" - ); - - // Check the specific error message - let error = build_result.unwrap_err(); - assert!(error.to_string().contains("MemWAL MemWalId { region: \"GLOBAL\", generation: 0 } has owner_id: new_owner_id, but expected owner_0"), - "Error message should indicate owner_id mismatch for merge insert, got: {}", error); - } - - #[tokio::test] - async fn test_concurrent_mem_wal_replay_and_modifications() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Add some entries to the MemWAL - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 456, "owner_0") - .await - .unwrap(); - - // Clone the dataset multiple times to simulate concurrent operations - let mut dataset_clone_append = dataset.clone(); - let mut dataset_clone_seal = dataset.clone(); - let mut dataset_clone_flush = dataset.clone(); - let mut dataset_clone_advance = dataset.clone(); - - // Start replay operation on the original dataset - let replay_result = update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 0, - "new_owner_id", - Some("new_mem_table_location"), - ) - .await; - - // Test all concurrent operations against the replay - let append_result = - append_mem_wal_entry(&mut dataset_clone_append, "GLOBAL", 0, 789, "owner_0").await; - let seal_result = - mark_mem_wal_as_sealed(&mut dataset_clone_seal, "GLOBAL", 0, "owner_0").await; - let flush_result = - mark_mem_wal_as_flushed(&mut dataset_clone_flush, "GLOBAL", 0, "owner_0").await; - let advance_result = advance_mem_wal_generation( - &mut dataset_clone_advance, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", - ) - .await; - - // Test merge_insert merge operation separately (requires flushed MemWAL) - // Advance to a new generation and seal it for merge insert test - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("new_owner_id"), - "owner_1", - ) - .await - .unwrap(); - - // Seal and flush the new generation - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 1, "owner_1") - .await - .unwrap(); - - let dataset_clone_merge_insert = dataset.clone(); - - // Start replay operation on the new generation - let replay_result_merge_insert = update_mem_wal_owner( - &mut dataset, - "GLOBAL", - 1, - "new_owner_id", - Some("new_mem_table_location_merge"), - ) - .await; - - // Test merge_insert merge operation - let mut merge_insert_job_builder = crate::dataset::MergeInsertBuilder::try_new( - Arc::new(dataset_clone_merge_insert), - vec!["i".to_string()], - ) - .unwrap(); - - let merge_insert_job = merge_insert_job_builder - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll) - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 1), "owner_1") - .await - .unwrap() - .try_build() - .unwrap(); - - // Create some data for the merge insert - let new_data = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step_custom::(2000, 1)) - .into_df_stream(RowCount::from(50), BatchCount::from(5)); - - // Execute the merge insert (this should fail due to version conflict) - let merge_insert_result = merge_insert_job.execute_reader(new_data).await; - - // Replay should succeed and all other operations should fail due to version conflict - assert!(replay_result.is_ok(), "Replay operation should succeed"); - assert!( - append_result.is_err(), - "Append operation should fail due to version conflict" - ); - assert!( - seal_result.is_err(), - "Seal operation should fail due to version conflict" - ); - assert!( - flush_result.is_err(), - "Flush operation should fail due to version conflict" - ); - assert!( - advance_result.is_err(), - "Advance generation operation should fail due to version conflict" - ); - - // For merge insert test, replay should succeed and merge insert should fail - assert!( - replay_result_merge_insert.is_ok(), - "Replay operation for merge insert test should succeed" - ); - assert!( - merge_insert_result.is_err(), - "Merge insert flush operation should fail due to version conflict" - ); - } - - #[tokio::test] - async fn test_concurrent_mem_wal_append_and_merge_insert_flush() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Add some entries to generation 0 - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 456, "owner_0") - .await - .unwrap(); - - // Seal and flush generation 0 (required for merge insert merge) - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Advance to generation 1 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", - ) - .await - .unwrap(); - - // Add some entries to generation 1 - append_mem_wal_entry(&mut dataset, "GLOBAL", 1, 789, "owner_1") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 1, 790, "owner_1") - .await - .unwrap(); - - // Clone the dataset to simulate concurrent operations - let mut dataset_clone_append = dataset.clone(); - let dataset_clone_merge_insert = dataset.clone(); - - // Test concurrent operations: append to generation 1 and merge_insert merge generation 0 - let append_result = - append_mem_wal_entry(&mut dataset_clone_append, "GLOBAL", 1, 791, "owner_1").await; - - // Create merge insert job that merges generation 0 - let mut merge_insert_job_builder = crate::dataset::MergeInsertBuilder::try_new( - Arc::new(dataset_clone_merge_insert), - vec!["i".to_string()], - ) - .unwrap(); - - let merge_insert_job = merge_insert_job_builder - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll) - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 0), "owner_0") - .await - .unwrap() - .try_build() - .unwrap(); - - // Create some data for the merge insert - let new_data = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step_custom::(2000, 1)) - .into_df_stream(RowCount::from(50), BatchCount::from(5)); - - // Execute the merge insert - let merge_insert_result = merge_insert_job.execute_reader(new_data).await; - - // Both operations should succeed since they operate on different generations - assert!( - append_result.is_ok(), - "Append to generation 1 should succeed" - ); - assert!( - merge_insert_result.is_ok(), - "Merge insert flush of generation 0 should succeed" - ); - - // Get the updated dataset from the merge insert result - let (updated_dataset, _stats) = merge_insert_result.unwrap(); - - // Verify the final state using the updated dataset - let indices = updated_dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - - // Find generation 0 and generation 1 - let gen_0 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 0) - .expect("Generation 0 should exist"); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - - // Verify generation 0 is merged (after merge_insert) - assert_eq!( - gen_0.state, - lance_index::mem_wal::State::Merged, - "Generation 0 should be merged" - ); - - // Verify generation 1 is unsealed and unflushed - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Open, - "Generation 1 should be open" - ); - - // Verify that generation 1 has the new entry - let wal_entries = gen_1.wal_entries(); - assert!( - wal_entries.contains(791), - "Generation 1 should contain the new entry 791" - ); - } - - #[tokio::test] - async fn test_concurrent_mem_wal_advance_and_merge_insert_flush() { - // Create a dataset with some data - let mut dataset = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step::()) - .into_ram_dataset(FragmentCount::from(2), FragmentRowCount::from(1000)) - .await - .unwrap(); - - // Create MemWAL index and generation 0 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_0", - "wal_location_0", - None, - "owner_0", - ) - .await - .unwrap(); - - // Add some entries to generation 0 - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 123, "owner_0") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 0, 456, "owner_0") - .await - .unwrap(); - - // Seal and flush generation 0 (required for merge insert merge) - mark_mem_wal_as_sealed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - mark_mem_wal_as_flushed(&mut dataset, "GLOBAL", 0, "owner_0") - .await - .unwrap(); - - // Advance to generation 1 - advance_mem_wal_generation( - &mut dataset, - "GLOBAL", - "mem_table_location_1", - "wal_location_1", - Some("owner_0"), - "owner_1", - ) - .await - .unwrap(); - - // Add some entries to generation 1 - append_mem_wal_entry(&mut dataset, "GLOBAL", 1, 789, "owner_1") - .await - .unwrap(); - append_mem_wal_entry(&mut dataset, "GLOBAL", 1, 790, "owner_1") - .await - .unwrap(); - - // Clone the dataset to simulate concurrent operations - let mut dataset_clone_advance = dataset.clone(); - let dataset_clone_merge_insert = dataset.clone(); - - // Test concurrent operations: advance to generation 2 and merge_insert flush generation 0 - let advance_result = advance_mem_wal_generation( - &mut dataset_clone_advance, - "GLOBAL", - "mem_table_location_2", - "wal_location_2", - Some("owner_1"), - "owner_2", - ) - .await; - - // Create merge insert job that merges generation 0 - let mut merge_insert_job_builder = crate::dataset::MergeInsertBuilder::try_new( - Arc::new(dataset_clone_merge_insert), - vec!["i".to_string()], - ) - .unwrap(); - - let merge_insert_job = merge_insert_job_builder - .when_matched(crate::dataset::WhenMatched::UpdateAll) - .when_not_matched(crate::dataset::WhenNotMatched::InsertAll) - .mark_mem_wal_as_merged(MemWalId::new("GLOBAL", 0), "owner_0") - .await - .unwrap() - .try_build() - .unwrap(); - - // Create some data for the merge insert - let new_data = lance_datagen::gen_batch() - .col( - "vec", - lance_datagen::array::rand_vec::(Dimension::from(128)), - ) - .col("i", lance_datagen::array::step_custom::(2000, 1)) - .into_df_stream(RowCount::from(50), BatchCount::from(5)); - - // Execute the merge insert - let merge_insert_result = merge_insert_job.execute_reader(new_data).await; - - // Both operations should succeed since they operate on different generations - assert!( - advance_result.is_ok(), - "Advance to generation 2 should succeed" - ); - assert!( - merge_insert_result.is_ok(), - "Merge insert flush of generation 0 should succeed" - ); - - // Get the updated dataset from the merge insert result - let (updated_dataset, _stats) = merge_insert_result.unwrap(); - - // Verify the final state using the updated dataset - let indices = updated_dataset.load_indices().await.unwrap(); - let mem_wal_index_meta = indices - .iter() - .find(|idx| idx.name == MEM_WAL_INDEX_NAME) - .expect("MemWAL index should exist"); - - let mem_wal_details = load_mem_wal_index_details(mem_wal_index_meta.clone()).unwrap(); - - // Find all generations - let gen_0 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 0) - .expect("Generation 0 should exist"); - let gen_1 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 1) - .expect("Generation 1 should exist"); - let gen_2 = mem_wal_details - .mem_wal_list - .iter() - .find(|m| m.id.generation == 2) - .expect("Generation 2 should exist"); - - // Verify generation 0 is merged (after merge_insert) - assert_eq!( - gen_0.state, - lance_index::mem_wal::State::Merged, - "Generation 0 should be merged" - ); - - // Verify generation 1 is sealed (due to advance) but unflushed - assert_eq!( - gen_1.state, - lance_index::mem_wal::State::Sealed, - "Generation 1 should be sealed due to advance" - ); + #[test] + fn test_empty_merged_generations_noop() { + let mut indices = Vec::new(); - // Verify generation 2 is unsealed and unflushed - assert_eq!( - gen_2.state, - lance_index::mem_wal::State::Open, - "Generation 2 should be open" - ); + // Empty update should be a no-op + update_mem_wal_index_merged_generations(&mut indices, 1, vec![]).unwrap(); - // Verify that generation 1 has the expected entries - let wal_entries = gen_1.wal_entries(); - assert!( - wal_entries.contains(789), - "Generation 1 should contain entry 789" - ); - assert!( - wal_entries.contains(790), - "Generation 1 should contain entry 790" - ); + assert!(indices.is_empty()); } } diff --git a/rust/lance/src/io/commit/conflict_resolver.rs b/rust/lance/src/io/commit/conflict_resolver.rs index 0fd52ab2a8b..c09c5dbcd90 100644 --- a/rust/lance/src/io/commit/conflict_resolver.rs +++ b/rust/lance/src/io/commit/conflict_resolver.rs @@ -13,7 +13,7 @@ use lance_core::{ Error, Result, }; use lance_index::frag_reuse::FRAG_REUSE_INDEX_NAME; -use lance_index::mem_wal::MemWal; +use lance_index::mem_wal::MergedGeneration; use lance_table::format::IndexMetadata; use lance_table::{format::Fragment, io::deletion::write_deletion_file}; use snafu::{location, Location}; @@ -343,8 +343,8 @@ impl<'a> TransactionRebase<'a> { other_version: u64, ) -> Result<()> { if let Operation::Update { - mem_wal_to_merge, inserted_rows_filter: self_inserted_rows_filter, + merged_generations: self_merged_generations, .. } = &self.transaction.operation { @@ -515,21 +515,14 @@ impl<'a> TransactionRebase<'a> { Operation::Overwrite { .. } | Operation::Restore { .. } => Err( self.incompatible_conflict_err(other_transaction, other_version, location!()) ), - Operation::UpdateMemWalState { added, updated, .. } => { - self.check_update_mem_wal_state_not_modify_same_mem_wal( - added, - mem_wal_to_merge.as_slice(), - other_transaction, - other_version, - )?; - self.check_update_mem_wal_state_not_modify_same_mem_wal( - updated, - mem_wal_to_merge.as_slice(), - other_transaction, - other_version, - )?; - Ok(()) - } + Operation::UpdateMemWalState { + merged_generations: other_merged_generations, + } => self.check_merged_generations_conflict( + other_merged_generations, + self_merged_generations, + other_transaction, + other_version, + ), } } else { Err(wrong_operation_err(&self.transaction.operation)) @@ -1205,68 +1198,33 @@ impl<'a> TransactionRebase<'a> { other_version: u64, ) -> Result<()> { if let Operation::UpdateMemWalState { - added, - updated, - removed: _, - .. + merged_generations: self_merged_generations, } = &self.transaction.operation { match &other_transaction.operation { Operation::UpdateMemWalState { - added: committed_added, - updated: committed_updated, - removed: _, + merged_generations: other_merged_generations, } => { - // 1. if the current or last committed job is trimming flushed MemWALs, - // it is compatible with any other UpdateMemWalState commits - if (committed_added.is_empty() && committed_updated.is_empty()) - || (added.is_empty() && updated.is_empty()) - { - return Ok(()); - } - - // 2. MemWALs of different regions can be changed at the same time - self.check_update_mem_wal_state_not_modify_same_mem_wal( - committed_added, - added, - other_transaction, - other_version, - )?; - self.check_update_mem_wal_state_not_modify_same_mem_wal( - committed_added, - updated, + // Two UpdateMemWalState transactions conflict if they're updating + // the same region's merged_generation + self.check_merged_generations_conflict( + other_merged_generations, + self_merged_generations, other_transaction, other_version, - )?; - self.check_update_mem_wal_state_not_modify_same_mem_wal( - committed_updated, - added, - other_transaction, - other_version, - )?; - self.check_update_mem_wal_state_not_modify_same_mem_wal( - committed_updated, - updated, - other_transaction, - other_version, - )?; - Ok(()) + ) } Operation::Update { - mem_wal_to_merge, .. + merged_generations: other_merged_generations, + .. } => { - if mem_wal_to_merge.is_some() { - // TODO: This check could be more detailed, there is an assumption that - // once a MemWAL is sealed, there is no other operation that could change - // the state back to open, and at that point it can always be flushed. - Ok(()) - } else { - Err(self.incompatible_conflict_err( - other_transaction, - other_version, - location!(), - )) - } + // Update transactions with merged_generations can conflict + self.check_merged_generations_conflict( + other_merged_generations, + self_merged_generations, + other_transaction, + other_version, + ) } Operation::UpdateConfig { .. } | Operation::Rewrite { .. } @@ -1343,50 +1301,26 @@ impl<'a> TransactionRebase<'a> { } } - fn check_update_mem_wal_state_not_modify_same_mem_wal( + fn check_merged_generations_conflict( &self, - committed: &[MemWal], - to_commit: &[MemWal], + committed: &[MergedGeneration], + to_commit: &[MergedGeneration], other_transaction: &Transaction, other_version: u64, ) -> Result<()> { - if !committed.is_empty() { - if to_commit.is_empty() { - return Ok(()); - } - - if committed.len() > 1 { - return Err(Error::Internal { - message: format!( - "Committing multiple MemWALs is not supported, but found committed: {:?}", - committed - ), - location: location!(), - }); - } - - if to_commit.len() > 1 { - return Err(Error::NotSupported { - source: format!( - "Committing multiple MemWALs is not supported, but found attempt to commit: {:?}", - to_commit - ) - .into(), - location: location!(), - }); - } - - let committed_mem_wal = committed.first().unwrap(); - let to_commit_mem_wal = to_commit.first().unwrap(); - if committed_mem_wal.id == to_commit_mem_wal.id { - return Err(self.incompatible_conflict_err( - other_transaction, - other_version, - location!(), - )); + // Check if any region has conflicting updates + for committed_mg in committed { + for to_commit_mg in to_commit { + if committed_mg.region_id == to_commit_mg.region_id { + // Same region being updated - this is a conflict + return Err(self.retryable_conflict_err( + other_transaction, + other_version, + location!(), + )); + } } } - Ok(()) } @@ -1861,7 +1795,7 @@ mod tests { removed_fragment_ids: vec![], new_fragments: vec![], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -1873,7 +1807,7 @@ mod tests { removed_fragment_ids: vec![2], new_fragments: vec![], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -1888,7 +1822,7 @@ mod tests { updated_fragments: vec![Fragment::new(4)], new_fragments: vec![], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -1990,7 +1924,7 @@ mod tests { removed_fragment_ids: vec![], new_fragments: vec![sample_file.clone()], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -2005,7 +1939,7 @@ mod tests { removed_fragment_ids: vec![], new_fragments: vec![sample_file], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -2127,7 +2061,7 @@ mod tests { removed_fragment_ids: vec![0], new_fragments: vec![sample_file.clone()], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -2140,7 +2074,7 @@ mod tests { removed_fragment_ids: vec![], new_fragments: vec![sample_file.clone()], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -2299,7 +2233,7 @@ mod tests { updated_fragments: vec![fragment0.clone()], new_fragments: vec![fragment2.clone()], fields_modified: vec![0], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -2495,7 +2429,7 @@ mod tests { removed_fragment_ids: vec![], new_fragments: vec![fragment2], fields_modified: vec![0], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, @@ -2918,7 +2852,7 @@ mod tests { removed_fragment_ids: vec![], new_fragments: vec![], fields_modified: vec![], - mem_wal_to_merge: None, + merged_generations: Vec::new(), fields_for_preserving_frag_bitmap: vec![], update_mode: None, inserted_rows_filter: None, From 27f3efcf755d1f4986f1aef028a2c32261c8f81b Mon Sep 17 00:00:00 2001 From: Heng Ge Date: Mon, 12 Jan 2026 22:07:21 -0800 Subject: [PATCH 2/9] docs: address PR review comments on MemWAL spec - Mark as experimental - Restructure sections (architecture first, then components) - Use bloom filter instead of btree for PK existence - Fix vector index: use same IVF type/centroids/codebook as base - Add region pruning, sync/async ops summary - Update query plans for multi-region (no cross-region dedup) - Rename _memwal to _mem_wal - Remove redundant sections Co-Authored-By: Claude Opus 4.5 --- docs/src/format/table/mem_wal.md | 1010 +++++++++--------------------- 1 file changed, 295 insertions(+), 715 deletions(-) diff --git a/docs/src/format/table/mem_wal.md b/docs/src/format/table/mem_wal.md index 060f174e53c..6de8ac2d921 100644 --- a/docs/src/format/table/mem_wal.md +++ b/docs/src/format/table/mem_wal.md @@ -1,37 +1,6 @@ -# MemTable & WAL Specification +# MemTable & WAL Specification (Experimental) -Lance MemTable & WAL (MemWAL) specification describes an Log-Structured-Merge (LSM) tree architecture for Lance tables, enabling high-performance streaming write workloads. - -## Prerequisites - -### Unenforced Primary Key - -MemWAL only works for Lance tables with unenforced primary key defined, -also the unenforced primary key: - -- Must have a [btree index](./index/scalar/btree.md) -- Must be included in the region spec's `source_ids` if a region spec is specified (see [Region Spec](#region-sepc) for more details) - -The last constraint is critical for correctness. -If two regions contain rows with the same primary key, the following scenario can cause data corruption: - -1. Region A receives a write with primary key `pk=1` at time T1 -2. Region B receives a write with primary key `pk=1` at time T2 (T2 > T1) -3. The row in region B is merged into the base table first -4. The row in region A is merged into the base table second -5. The row from Region A (older) now overwrites the row from Region B (newer) - -This violates the expected "last write wins" semantics. -By ensuring each primary key is assigned to exactly one region via the region spec, merge order between regions becomes irrelevant for correctness. - -### IVF Vector Index - -Although later migration is possible, if the main use case is IVF family vector index, -it is recommended to have these indexes on the Lance table before enabling MemWAL. -This is because IVF index needs to remain the same quantization codebook (e.g. PQ codebook) -across all the layers of the LSM tree for vector distance to be comparable. -Migrating one codebook to another is a complicated proceses requiring gradual migration -and coordination between readers and writers. +Lance MemTable & WAL (MemWAL) specification describes a Log-Structured-Merge (LSM) tree architecture for Lance tables, enabling high-performance streaming write workloads. ## Overall Architecture @@ -41,84 +10,13 @@ and coordination between readers and writers. Under the MemWAL setup, the Lance table is called the **base table**. -### MemWAL Region +### Region -**MemWAL Region** is the main unit to horizontally scale out writes. +A **Region** is the main unit to horizontally scale out writes. Each region has exactly one active writer at any time, using **epoch-based fencing** to guarantee single-writer semantics without distributed coordination. Writers claim a region by incrementing the writer epoch, then write data to that region. Data in each region is merged into the base table gradually in the background. -#### Region Identifier - -Each region has a unique identifier across all regions following UUID v4 standard. -When a new region is created, it is assigned a new identifier. - -#### Region Spec - -A **Region Spec** defines how the all rows in a table is logically divided into different regions, -enabling automatic region assignment and query-time region pruning. - -Each region spec has: - -- **Spec ID**: A positive integer that uniquely identifies this spec within the MemWAL index. IDs are never reused. -- **Region fields**: An array of field definitions that determine how to compute region identifiers. - -Each region is bound to a specific region spec ID, recorded in its [manifest](#region-manifest). -Regions without a spec ID (`spec_id = 0`) are manually-created regions not governed by any spec. - -A region spec's field array consists of **region field** definitions. -Each region field has the following properties: - -| Property | Description | -|----------|-------------| -| `field_id` | Unique string identifier for this region field | -| `source_ids` | Array of field IDs referencing source columns in the schema | -| `transform` | A well-known region expression, specify this or `expression` | -| `expression` | A DataFusion SQL expression for custom logic, specify this or `transform` | -| `result_type` | The output type of the region value | - -#### Unenforced Primary key Constraint - -The `source_ids` across all region fields must include all primary key columns. -This ensures rows with the same primary key always map to the same region, which is required for correctness (see [Unenforced Primary Key](#unenforced-primary-key)). - -#### Region Expression - -A **Region Expression** is a [DataFusion SQL expression](https://datafusion.apache.org/user-guide/sql/index.html) that derives a region value from source column(s). -Source columns are referenced as `col0`, `col1`, etc., corresponding to the order of field IDs in `source_ids`. - -Region expressions must satisfy the following requirements: - -1. **Deterministic**: The same input value must always produce the same output value. -2. **Stateless**: The expression must not depend on external state (e.g., current time, random values, session variables). -3. **Type-promotion resistant**: The expression must produce the same result for equivalent values regardless of their numeric type (e.g., `int32(5)` and `int64(5)` must yield the same region value). -4. **Column removal resistant**: If a source field ID is not found in the schema, the column should be interpreted as NULL. -5. **NULL-safe**: The expression should properly handle NULL inputs and have defined behavior (e.g., return NULL if input is NULL for single-column expressions). -6. **Consistent with result type**: The expression's return type must be consistent with `result_type` in non-NULL cases. - -#### Region Transform - -A **Region Transform** is a well-known region expression with a predefined name. -When a transform is specified, the expression is derived automatically. - -| Transform | Parameters | Region Expression | Result Type | -|-----------|------------|-------------------|-------------| -| `identity` | (none) | `col0` | same as source | -| `year` | (none) | `date_part('year', col0)` | `int32` | -| `month` | (none) | `date_part('month', col0)` | `int32` | -| `day` | (none) | `date_part('day', col0)` | `int32` | -| `hour` | (none) | `date_part('hour', col0)` | `int32` | -| `bucket` | `num_buckets` | `abs(murmur3(col0)) % N` | `int32` | -| `multi_bucket` | `num_buckets` | `abs(murmur3_multi(col0, col1, ...)) % N` | `int32` | -| `truncate` | `width` | `left(col0, W)` (string) or `col0 - (col0 % W)` (numeric) | same as source | - -The `bucket` and `multi_bucket` transforms use Murmur3 hash functions: - -- **`murmur3(col)`**: Computes the 32-bit Murmur3 hash (x86 variant, seed 0) of a single column. Returns a signed 32-bit integer. Returns NULL if input is NULL. -- **`murmur3_multi(col0, col1, ...)`**: Computes the Murmur3 hash across multiple columns. Returns a signed 32-bit integer. NULL fields are ignored during hashing; returns NULL only if all inputs are NULL. - -The hash result is wrapped with `abs()` and modulo `N` to produce a non-negative bucket number in the range `[0, N)`. - ### MemWAL Index A **MemWAL Index** is the centralized structure for all MemWAL metadata for a base table. @@ -129,12 +27,12 @@ It stores: - **Merge progress**: Last generation merged to base table for each region - **Region snapshots**: Point-in-time snapshot of all region states for read optimization -The index is the source of truth for **configuration** and **merge progress**, but region state snapshots are read-only (each region's manifest is authoritative for its own state). +The index is the source of truth for **configuration** and **merge progress**, but region state snapshots are for read-only optimization (each region's manifest is authoritative for its own state). Writers read the MemWAL index to get configuration (region specs, maintained indexes) before writing. -Readers use the index to get a snapshot of all region states, then query each region's data alongside the base table and merge results at runtime. +Readers use the index to get a [snapshot of all region states](#reader-consistency), then query each region's data alongside the base table and merge results at runtime. -A background process periodically updates region snapshots by listing regions and loading their manifests. +A [background process](#memwal-index-builder) periodically updates region snapshots by listing regions and loading their manifests. See [MemWAL Index Details](#memwal-index-details) for the complete structure. ## Region Architecture @@ -145,64 +43,47 @@ Within a region, writes enter its MemTable and are flushed to the regional WAL f The MemTable is flushed to storage as a Flushed MemTable based on memory pressure and other conditions. Flushed MemTables are then asynchronously merged into the base table. -Here are the details of the related components and concepts: - ### MemTable -An in-memory Lance table that buffers incoming writes. +An in-memory Lance table that buffers incoming writes. Each write inserts a fragment in the MemTable, making data immediately queryable without waiting for persistence. ### WAL -Write-Ahead Log (WAL) seves as the durable sotrage of MemTable. +Write-Ahead Log (WAL) serves as the durable storage of MemTable. A write to MemTable must be persisted also to the WAL to become fully durable. Every time we write to the WAL, we call it a **WAL Flush**. The whole LSM tree's durability is determined by the durability of the WAL. For example, if WAL is stored in Amazon S3, it has the 99.999999999% durability. -If it is stored in local disk, the data will be lost if the local disk is damanaged. - -#### WAL Entry +If it is stored in local disk, the data will be lost if the local disk is damaged. A WAL consists of an ordered sequence of WAL entries starting from 1. Each entry is a Lance format file. The writer epoch is stored in the Lance file's schema metadata with key `writer_epoch` for fencing validation during replay. -#### File Location - -Each WAL entry is stored within the WAL directory of the region located at `_memwal/{region_id}/wal`. - -#### File Naming +Each WAL entry is stored within the WAL directory of the region located at `_mem_wal/{region_id}/wal`. WAL files use bit-reversed 64-bit binary naming to distribute files evenly across the directory keyspace. This optimizes S3 throughput by spreading sequential writes across S3's internal partitions, minimizing throttling. - The filename is the bit-reversed binary representation of the entry ID with suffix `.lance`. For example, entry ID 5 (binary `000...101`) becomes `1010000000000000000000000000000000000000000000000000000000000000.lance`. -This bit-reversal permutation ensures that sequential entry IDs are spread across the entire keyspace, similar to how [data files use UUID-based naming](layout.md#data-files) for S3 throughput optimization. - ### Flushed MemTable A flushed MemTable is a complete Lance table created by flushing the MemTable to storage. !!!note This is called Sorted String Table (SSTable) or Sorted Run in many LSM-tree literatures and implementations. - However, since our MemTable is not sorted, we just use the term flushed memtable avoid confusion. - -#### Generation + However, since our MemTable is not sorted, we just use the term flushed MemTable to avoid confusion. Each flushed MemTable has a **generation** number starting from 1 that identifies its relative position among all flushed MemTables in the region. When MemTable with generation `i` is flushed, the next MemTable gets generation number `i+1`. -#### Flush Location - -The MemTable of generation `i` is flushed to `_memwal/{region_uuid}/{random_hash}_gen_{i}/` directory, where `{random_hash}` is an 8-character hex value generated at flush time. +The MemTable of generation `i` is flushed to `_mem_wal/{region_uuid}/{random_hash}_gen_{i}/` directory, where `{random_hash}` is an 8-character hex value generated at flush time. The directory content follows [Lance table layout](layout.md). The actual directory path for each generation is recorded in the region manifest's `flushed_generations` list (see [Region Manifest](#region-manifest)). -#### Merging Flushed MemTable - Generation numbers determine merge order: lower numbers represent older data and must be merged to the base table first to preserve correct upsert semantics. ### Region Manifest @@ -211,8 +92,6 @@ Each region has a manifest file containing epoch-based fencing tokens, WAL point The manifest is serialized as a protobuf binary file using the `RegionManifest` message. -#### Contents - The manifest contains: - **Fencing state**: `writer_epoch` (writer fencing token) @@ -221,7 +100,7 @@ The manifest contains: - **Flushed generations**: `flushed_generations` list of generation number and directory path pairs (e.g., generation 1 at `a1b2c3d4_gen_1`) Note: `wal_id_last_seen` is a hint that may be stale since it's not updated on WAL write. -The manifest itself is atomically written, but recovery must try get newer WAL files to find the actual state beyond this hint. +The manifest itself is atomically written, but recovery must try to get newer WAL files to find the actual state beyond this hint.

RegionManifest protobuf message @@ -232,10 +111,8 @@ The manifest itself is atomically written, but recovery must try get newer WAL f
-#### Versioning and Atomicity - Manifests are versioned starting from 1 and immutable. Each update creates a new manifest file at the next version number. -Updates use put-if-not-exists or file rename to ensure atomicity depending on the storage system. +Updates use put-if-not-exists or file rename to ensure atomicity depending on the storage system. If two processes compete, one wins and the other retries. To commit a manifest version: @@ -251,21 +128,16 @@ To read the latest manifest version: 3. Continue until a version is not found 4. The latest version is the last found version -This approach uses HEAD requests instead of LIST operations in cloud storage, -which is in general faster on cloud storage systems and -is friendly to systems like S3 Express that do not support lexicographically sorted listing. - -#### File Location +This approach uses HEAD requests instead of LIST operations in cloud storage, which is generally faster and is friendly to systems like S3 Express that do not support lexicographically sorted listing. -All region manifest versions are stored in `_memwal/{region_id}/manifest` directory. +!!!note + This works because the write rate to region manifests is significantly lower than typical read rates. Region manifests are only updated when region metadata changes (MemTable flush, merge completion), not on every write. This ensures HEAD requests will eventually terminate and find the latest version. -#### File Naming +All region manifest versions are stored in `_mem_wal/{region_id}/manifest` directory. -Each region manifest version file uses bit-reversed 64-bit binary naming, the same scheme as [WAL files](#wal-file-naming). +Each region manifest version file uses bit-reversed 64-bit binary naming, the same scheme as WAL files. For example, version 5 becomes `1010000000000000000000000000000000000000000000000000000000000000.binpb`. -#### Region Manifest Transactions - The region manifest is updated atomically in the following cases: | Trigger | Fields Updated | Details | @@ -278,12 +150,10 @@ The region manifest is updated atomically in the following cases: !!!note WAL flush does **not** update the manifest to keep the hot write path fast. -#### Fencing - Writers use epoch-based fencing (`writer_epoch`) to ensure single-writer semantics. See [Writer Fencing](#writer-fencing) for details. -### Storage Layout +## Storage Layout Here is a recap of the storage layout with all the files and concepts defined so far: @@ -293,7 +163,7 @@ Here is a recap of the storage layout with all the files and concepts defined so │ └── {index_uuid}/ # MemWAL Index (uses standard index storage) │ └── regions.binpb # Serialized region snapshots (protobuf binary) │ -└── _memwal/ +└── _mem_wal/ └── {region_uuid}/ # Region directory (UUID v4) ├── manifest/ │ ├── {bit_reversed_version}.binpb # Serialized region manifest (bit-reversed naming) @@ -304,12 +174,13 @@ Here is a recap of the storage layout with all the files and concepts defined so └── {random_hash}_gen_{i}/ # Flushed MemTable (generation i, random prefix) ├── _versions/ │ └── {version}.manifest # Table manifest (V2 naming scheme) - └── _indices/ # indexes - ├── {vector_index}/ - └── {scalar_index}/ + ├── _indices/ # Indexes + │ ├── {vector_index}/ + │ └── {scalar_index}/ + └── bloom_filter.bin # Primary key bloom filter ``` -### MemWAL Index Details +## MemWAL Index Details The MemWAL Index uses the [standard index storage](index/index.md#index-storage) at `_indices/{UUID}/`. @@ -318,7 +189,7 @@ The index stores its data in two parts: 1. **Index details** (`index_details` in `IndexMetadata`): Contains configuration, merge progress, and snapshot metadata 2. **Region snapshots**: Stored as a Lance file or inline, depending on region count -#### Index Details Schema +### Index Details Schema The `index_details` field in `IndexMetadata` contains a `MemWalIndexDetails` protobuf message with: @@ -334,15 +205,13 @@ The `index_details` field in `IndexMetadata` contains a `MemWalIndexDetails` pro **Configuration fields** (`region_specs`, `maintained_indexes`) are the source of truth for MemWAL configuration. Writers read these fields to determine how to partition data and which indexes to maintain. -- **Region specs** define how rows are partitioned into regions. Multiple specs can coexist during migration. Each spec has a unique `spec_id` that is never reused. See [Region Spec](#region-spec) for field definitions. -- **Maintained indexes** lists indexes (by name) to maintain in MemTables. The primary key btree index is always maintained implicitly and should not be listed here. For vector indexes, MemTables inherit quantization parameters (PQ codebook, SQ params) from the base table index to ensure distance comparability. See [Vector Indexes](#vector-indexes). - **Merge progress** (`merged_generations`) tracks the last generation merged to the base table for each region. This field is updated atomically with merge-insert data commits, enabling conflict resolution when multiple mergers operate concurrently. Each entry contains the region UUID and generation number. **Region snapshot fields** (`snapshot_timestamp`, `num_regions`, `inline_snapshots`) provide a point-in-time snapshot of region states. The actual region manifests remain authoritative for region state. +When `num_regions` is 0, the `inline_snapshots` field may be `None` or an empty Lance file with 0 rows but proper schema.
MemWalIndexDetails protobuf message @@ -353,7 +222,89 @@ The actual region manifests remain authoritative for region state.
-#### Region Snapshot Storage +### Region Identifier + +Each region has a unique identifier across all regions following UUID v4 standard. +When a new region is created, it is assigned a new identifier. + +### Region Spec + +A **Region Spec** defines how all rows in a table are logically divided into different regions, +enabling automatic region assignment and query-time region pruning. + +Each region spec has: + +- **Spec ID**: A positive integer that uniquely identifies this spec within the MemWAL index. IDs are never reused. +- **Region fields**: An array of field definitions that determine how to compute region values. + +Each region is bound to a specific region spec ID, recorded in its [manifest](#region-manifest). +Regions without a spec ID (`spec_id = 0`) are manually-created regions not governed by any spec. + +A region spec's field array consists of **region field** definitions. +Each region field has the following properties: + +| Property | Description | +|----------|-------------| +| `field_id` | Unique string identifier for this region field | +| `source_ids` | Array of field IDs referencing source columns in the schema | +| `transform` | A well-known region expression, specify this or `expression` | +| `expression` | A DataFusion SQL expression for custom logic, specify this or `transform` | +| `result_type` | The output type of the region value | + +#### Primary Key Constraint + +The `source_ids` across all region fields must include all primary key columns. +This ensures rows with the same primary key always map to the same region, which is required for correctness. + +If two regions contain rows with the same primary key, the following scenario can cause data corruption: + +1. Region A receives a write with primary key `pk=1` at time T1 +2. Region B receives a write with primary key `pk=1` at time T2 (T2 > T1) +3. The row in region B is merged into the base table first +4. The row in region A is merged into the base table second +5. The row from Region A (older) now overwrites the row from Region B (newer) + +This violates the expected "last write wins" semantics. +By ensuring each primary key is assigned to exactly one region via the region spec, merge order between regions becomes irrelevant for correctness. + +#### Region Expression + +A **Region Expression** is a [DataFusion SQL expression](https://datafusion.apache.org/user-guide/sql/index.html) that derives a region value from source column(s). +Source columns are referenced as `col0`, `col1`, etc., corresponding to the order of field IDs in `source_ids`. + +Region expressions must satisfy the following requirements: + +1. **Deterministic**: The same input value must always produce the same output value. +2. **Stateless**: The expression must not depend on external state (e.g., current time, random values, session variables). +3. **Type-promotion resistant**: The expression must produce the same result for equivalent values regardless of their numeric type (e.g., `int32(5)` and `int64(5)` must yield the same region value). +4. **Column removal resistant**: If a source field ID is not found in the schema, the column should be interpreted as NULL. +5. **NULL-safe**: The expression should properly handle NULL inputs and have defined behavior (e.g., return NULL if input is NULL for single-column expressions). +6. **Consistent with result type**: The expression's return type must be consistent with `result_type` in non-NULL cases. + +#### Region Transform + +A **Region Transform** is a well-known region expression with a predefined name. +When a transform is specified, the expression is derived automatically. + +| Transform | Parameters | Region Expression | Result Type | +|-----------|------------|-------------------|-------------| +| `identity` | (none) | `col0` | same as source | +| `year` | (none) | `date_part('year', col0)` | `int32` | +| `month` | (none) | `date_part('month', col0)` | `int32` | +| `day` | (none) | `date_part('day', col0)` | `int32` | +| `hour` | (none) | `date_part('hour', col0)` | `int32` | +| `bucket` | `num_buckets` | `abs(murmur3(col0)) % N` | `int32` | +| `multi_bucket` | `num_buckets` | `abs(murmur3_multi(col0, col1, ...)) % N` | `int32` | +| `truncate` | `width` | `left(col0, W)` (string) or `col0 - (col0 % W)` (numeric) | same as source | + +The `bucket` and `multi_bucket` transforms use Murmur3 hash functions: + +- **`murmur3(col)`**: Computes the 32-bit Murmur3 hash (x86 variant, seed 0) of a single column. Returns a signed 32-bit integer. Returns NULL if input is NULL. +- **`murmur3_multi(col0, col1, ...)`**: Computes the Murmur3 hash across multiple columns. Returns a signed 32-bit integer. NULL fields are ignored during hashing; returns NULL only if all inputs are NULL. + +The hash result is wrapped with `abs()` and modulo `N` to produce a non-negative bucket number in the range `[0, N)`. + +### Region Snapshot Storage Region snapshots are stored using one of two strategies based on the number of regions: @@ -370,10 +321,10 @@ This keeps the index metadata compact while avoiding an additional file read for **External Lance file**: For large region counts, snapshots are stored as a Lance file at `_indices/{UUID}/index.lance`. This file uses standard Lance format with the region snapshot schema, enabling efficient columnar access and compression. -#### Region Snapshot Schema +### Region Snapshot Schema Region snapshots are stored as a Lance file with one row per region. -The schema has one column per `RegionManifest` field, with region fields as columns: +The schema has one column per `RegionManifest` field plus region spec columns: | Column | Type | Description | |--------|------|-------------| @@ -386,10 +337,11 @@ The schema has one column per `RegionManifest` field, with region fields as colu | `current_generation` | `uint64` | Next generation to flush | | `merged_generation` | `uint64` | Last generation merged to base | | `flushed_generations` | `list>` | Flushed MemTable paths | +| `region_values` | `struct<{field_id}: {result_type}, ...>` | Region field values from spec | -This schema directly corresponds to the fields in the `RegionManifest` protobuf message. +This schema directly corresponds to the fields in the `RegionManifest` protobuf message plus the computed region values. -#### Staleness Handling +### Staleness Handling Since the index is eventually consistent, readers should handle stale data: @@ -399,7 +351,19 @@ Since the index is eventually consistent, readers should handle stale data: The `snapshot_timestamp` field indicates when the index was built; readers can use this to estimate staleness and decide whether to refresh. -For authoritative state, readers may load individual region manifests directly from `_memwal/{region_uuid}/manifest/`. +For authoritative state, readers may load individual region manifests directly from `_mem_wal/{region_uuid}/manifest/`. + +### Vector Index Configuration + +If the main use case is IVF family vector index, it is recommended to have these indexes on the Lance table before enabling MemWAL. +This is because IVF index needs to remain the same quantization codebook (e.g. PQ codebook) across all the layers of the LSM tree for vector distance to be comparable. + +MemTables automatically inherit vector indexing from base table indexes. +For each vector index on the base table, MemTable uses the same index type (IVF-PQ, IVF-SQ, etc.) with the same centroids and quantization parameters. +This ensures distances are precise and comparable across generations. + +The base table vector index should not change the codebook once MemWAL is enabled. +To switch codebooks, a migration is required: create another vector index with the new codebook, configure MemTable to maintain both indexes, and eventually drop the old index after all data has been merged with the new codebook. ## Writer Expectations @@ -417,6 +381,18 @@ Both options can be enabled independently. When disabled: - **Non-durable writes** buffer data in memory until a flush threshold is reached, accepting potential data loss on crash - **Non-indexed writes** defer index updates, meaning newly written data may not appear in index-accelerated queries until the next index refresh +### Synchronous vs Asynchronous Operations + +Writer operations can be categorized by their synchronous or asynchronous nature: + +| Operation | Mode | Description | +|-----------|------|-------------| +| Write to MemTable | Synchronous | Data inserted into in-memory fragments | +| WAL Flush | Configurable | Synchronous with durable writes, asynchronous otherwise | +| Index Update | Configurable | Synchronous with indexed writes, asynchronous otherwise | +| MemTable Flush | Asynchronous | Triggered by thresholds, runs in background | +| Merge to Base Table | Asynchronous | Background job merges flushed MemTables | + ### Initialization & Recovery A writer must claim a region before performing any write operations: @@ -425,22 +401,11 @@ A writer must claim a region before performing any write operations: 2. Increment `writer_epoch` by one 3. Atomically write a new manifest 4. If the write fails (another writer claimed the epoch), reload the manifest and retry with a higher epoch -5. Read WAL entries sequentially from `replay_after_wal_id + 1` until not found -6. Replay valid WAL entries (those with `writer_epoch` ≤ current epoch) to reconstruct the MemTable with 1:1 fragment mapping (each WAL entry becomes one MemTable fragment) +5. After initialization, read WAL entries sequentially from `replay_after_wal_id + 1` until not found +6. Replay valid WAL entries (those with `writer_epoch` <= current epoch) to reconstruct the MemTable with 1:1 fragment mapping (each WAL entry becomes one MemTable fragment) After recovery, the writer tracks subsequent fragment mappings as new WAL flushes occur (see [WAL Flush](#wal-flush)). -### Writer Fencing - -Before any manifest update (MemTable flush), a writer must verify its `writer_epoch` remains valid: - -- If `local_writer_epoch == stored_writer_epoch`: The writer is still active and may proceed -- If `local_writer_epoch < stored_writer_epoch`: The writer has been fenced and must abort - -Fenced writers must stop all operations immediately and notify pending writes of the failure. - -For a concrete example of fencing between two writers, see [Appendix 1: Writer Fencing Example](#appendix-1-writer-fencing-example). - ### Write Operations Each write operation follows this sequence: @@ -451,8 +416,8 @@ Each write operation follows this sequence: 4. Optionally trigger WAL flush based on size, count, or time thresholds 5. For [durable writes](#writer-configuration), wait for WAL flush to complete before returning 6. For [indexed writes](#writer-configuration), update MemTable indexes before returning: - - Insert primary keys into the btree index - - For each vector column with a base table index: encode and insert into HNSW graph + - Insert primary keys into the bloom filter + - For each vector column with a base table index: encode and insert using the same index type as base table - For each index in `maintained_indexes`: update the corresponding index structure ### WAL Flush @@ -465,17 +430,17 @@ WAL flush batches pending MemTable fragments into a single Lance data file: 4. Write the footer containing batched data file metadata and `writer_epoch` in schema metadata 5. Complete the WAL entry write atomically 6. Mark fragments as flushed in the MemTable -7. Record fragment mappings (MemTable fragment IDs in this batch → WAL entry ID relative to last replay) for index remapping during [MemTable Flush](#memtable-flush) +7. Record fragment mappings (MemTable fragment IDs in this batch -> WAL entry ID relative to last replay) for index remapping during [MemTable Flush](#memtable-flush) !!!note The region manifest is **not** updated on every WAL flush. The `wal_id_last_seen` field is a hint that can be updated: - + 1. **During MemTable flush** - when the region manifest is updated anyway 2. **By a background index builder** - which scans WAL entries and updates each region's `wal_id_last_seen` This keeps the hot write path fast. On recovery, the writer reads WAL entries sequentially starting from `wal_id_last_seen + 1` to discover any WAL entries beyond what the manifest indicates. -The [durable write](#writer-configuration) option also impacts flush behavior: +The WAL flush behavior depends on the [durable write](#writer-configuration) option: | Mode | Behavior | Result | |------|----------|--------| @@ -487,230 +452,36 @@ The [durable write](#writer-configuration) option also impacts flush behavior: MemTable indexing differs from base table indexing to balance write performance with query capability. Rather than maintaining all base table indexes, MemTables maintain a subset specified in the [MemWAL Index](#memwal-index). -#### Primary Key Index - -The **primary key btree index** is always maintained for every MemTable, regardless of other index configuration. -This index is essential for: - -- **Staleness detection**: During vector/FTS search, checking if a candidate from an older generation has a newer version -- **Point lookups**: Fast O(log n) access by primary key -- **Deduplication**: Efficiently finding duplicate primary keys during merge - -The primary key index is implemented as an in-memory `BTreeMap` where `OrderableScalarValue` wraps Arrow's `ScalarValue` with `Ord` implementation (see `lance-index::scalar::btree::OrderableScalarValue`). -For multi-column primary keys, the map key is a tuple of `OrderableScalarValue` for each column. - -#### Vector Indexes - -MemTables **automatically inherit** vector indexing from base table indexes. -This inheritance is critical for **distance comparability** across generations. - -**Why inheritance is required:** - -When ranking search results across generations, distances must be comparable: - -| Component | Must Match Base Table? | Reason | -|-----------|------------------------|--------| -| Quantization (PQ codebook, SQ params) | **Yes** | Determines distance calculation | -| IVF centroids | No | Only affects partition assignment | -| Search structure (HNSW vs flat) | No | Only affects search efficiency | - -If MemTable used independent quantization, distances from MemTable and base table would not be comparable, leading to incorrect ranking. - -**Inheritance behavior:** - -For each vector index on the base table, MemTable automatically: - -1. **Inherits quantization parameters**: PQ codebook, SQ min/max, or no quantization (flat) -2. **Encodes vectors** using inherited quantization during writes -3. **Stores both** raw vectors (for potential refinement) and quantized codes -4. **Uses HNSW** as the search structure (optimal for small MemTable scale) - -| Base Table Index | MemTable Inherits | MemTable Search | -|------------------|-------------------|-----------------| -| IVF-PQ | PQ codebook | HNSW on PQ codes | -| IVF-SQ | SQ parameters | HNSW on SQ codes | -| IVF-FLAT | Nothing (no quantization) | HNSW on raw vectors | -| IVF-HNSW-PQ | PQ codebook | HNSW on PQ codes | - -**Write path with inheritance:** - -``` -Write batch with vectors: - 1. Load PQ/SQ codebook from base table index (cached) - 2. Encode vectors using inherited quantization - 3. Store raw vectors + quantized codes in MemTable - 4. Insert into HNSW graph for search - 5. On MemTable flush, serialize both raw vectors and codes -``` - -**Query path with comparable distances:** - -``` -Search across generations: - 1. Search MemTable HNSW → candidates with quantized distances - 2. Search base table IVF-PQ → candidates with quantized distances - 3. Distances are COMPARABLE (same quantization) - 4. Sort by distance directly - 5. Apply staleness filtering -``` - -!!!warning "PQ Codebook Migration" - When the base table's PQ codebook is retrained, MemTable must switch to the new codebook. - During migration, maintain compatibility by: - 1. Flushing current MemTable before codebook change - 2. New MemTable uses new codebook - 3. Query both old flushed MemTables (old codebook) and new MemTable (new codebook) separately - 4. Merge flushed MemTables to base table before they become incompatible - -#### Scalar Indexes - -The `maintained_indexes` field in `MemWalIndexDetails` lists additional base table indexes to maintain in MemTables. -These include both scalar indexes (typically full-text search indexes needed for real-time text search) and vector indexes. - -Most scalar indexes other than FTS are not needed in MemTables since the primary key btree handles point lookups and staleness detection. - -#### Full-Text Search Indexes - -FTS indexes in MemTables **inherit tokenizer configuration** from base table indexes to ensure consistent tokenization across generations. - -**Inheritance behavior:** - -| Inherited | Not Inherited | -|-----------|---------------| -| Tokenizer type (simple, ngram, jieba, etc.) | Corpus statistics (IDF, avgdl) | -| Language settings | Document frequencies | -| Token filters (lowercase, stemming, etc.) | Posting lists | -| Position storage setting | | - -**Why corpus statistics are NOT inherited:** - -BM25 scoring depends on corpus-level statistics: -- `N`: Total document count -- `avgdl`: Average document length -- `df(t)`: Documents containing term t - -These statistics are specific to each corpus (generation). If MemTable used base table's statistics, scores would be incorrect because: -- IDF would be wrong (term rarity differs between 10K MemTable vs 10M base table) -- avgdl would be wrong (document length distribution may differ) - -**Global BM25 scoring (Lucene-style):** - -At query time, statistics are **aggregated across all generations** for globally-comparable BM25 scores: - -``` -Query: "machine learning" +MemTables maintain a **primary key bloom filter** for efficiently checking whether a primary key exists in a generation. +This enables staleness detection during search queries without requiring expensive point lookups. -Step 1: Aggregate corpus statistics - N_global = Σ gen.doc_count - avgdl_global = Σ gen.sum_total_term_freq / N_global +For vector indexes, MemTables use the same index type as the base table (e.g., IVF-PQ with the same centroids and PQ codebook). +This ensures distances are precise and directly comparable across generations. +The centroid assignment also impacts recall, so using the same centroids ensures consistent search quality. -Step 2: Aggregate term statistics (for query terms only) - df_global("machine") = Σ gen.fts_index.df("machine") - df_global("learning") = Σ gen.fts_index.df("learning") - -Step 3: Compute global IDF - IDF("machine") = log(1 + (N_global - df_global) / (df_global + 0.5)) - -Step 4: Search each generation with global stats - Each FTS index returns candidates scored with global IDF and avgdl - Scores are now COMPARABLE across generations - -Step 5: Merge and rank globally -``` - -This follows the same pattern as [Apache Lucene's multi-segment BM25 scoring](https://github.com/apache/lucene), where: -- Each segment (generation) stores its own corpus statistics -- At query time, statistics are summed across segments -- A single scorer with global parameters is used for all segments - -**Required FTS index statistics:** - -Each MemTable FTS index must expose: - -| Statistic | Description | Used For | -|-----------|-------------|----------| -| `doc_count` | Documents in this index | Global N | -| `sum_total_term_freq` | Sum of all document lengths | Global avgdl | -| `df(term)` | Documents containing term | Global IDF | - -These are summed at query time to compute global BM25 parameters. - -#### In-Memory Index Structure - -Each MemTable maintains indexes as in-memory data structures: - -| Index Type | In-Memory Structure | Description | -|------------|---------------------|-------------| -| Primary key btree | `BTreeMap` | Maps primary key value(s) to row ID | -| Vector (HNSW + quantization) | `HnswBuilder` + `Quantizer` | HNSW graph + inherited PQ/SQ codebook | -| Additional btree | `BTreeMap` | Maps indexed column value(s) to row ID | - -**Memory overhead** for ~20K vectors (64MB MemTable): - -| Component | Size | Notes | -|-----------|------|-------| -| HNSW graph structure | ~5-10MB | Neighbors + distances | -| Thread-safe overhead | ~1MB | `RwLock` per node | -| PQ codebook (cached) | ~1MB | Shared across MemTables | -| PQ codes storage | ~1-2MB | 64 bytes/vector typical | -| **Total** | ~10-15% of MemTable size | - -#### Index Update Timing - -Index update timing depends on the [indexed write](#writer-configuration) setting: - -| Mode | Index Update Timing | Query Behavior | -|------|---------------------|----------------| -| Indexed write | Synchronous: indexes updated before write returns | New data immediately searchable via indexes | -| Non-indexed write | Deferred: indexes updated in background or at next flush | New data may require full scan until index refresh | - -When indexes are updated (either synchronously or deferred): - -1. **Primary key btree**: Insert `(pk_value, row_id)` into `BTreeMap` -2. **Vector indexes**: For each vector column with a base table index: - - Encode vector using inherited quantization (PQ/SQ codebook) - - Insert into HNSW graph with `O(log n)` complexity -3. **Other indexes**: Update according to index-specific logic - -Entries reference MemTable fragment IDs and row offsets. - -#### Flushed MemTable Index Caching +For full-text search indexes, MemTables inherit tokenizer configuration from base table indexes to ensure consistent tokenization across generations. +Each generation maintains its own corpus statistics (document count, term frequencies) which are aggregated at query time for globally-comparable BM25 scores. When a MemTable is flushed to storage: -1. In-memory indexes are serialized to disk in the flushed MemTable's `_indices/` directory: - - **Primary key btree**: Written as Lance btree index format - - **Vector indexes**: HNSW graph + quantized codes written in Lance format - - **Raw vectors**: Stored in data files for potential exact distance refinement - - **Other indexes**: Written in their respective formats -2. The in-memory index structures are retained as a **cache** for readers in the same process -3. Remote readers load indexes from disk; local readers use the cached in-memory structures - -This caching strategy provides: - -- **Zero-latency index access** for readers in the writer's process -- **No index rebuild overhead** for local readers after flush -- **Standard disk-based access** for remote readers - -Fragment mappings enable index remapping during [MemTable Flush](#memtable-flush). These mappings are recorded: - -- During [Initialization & Recovery](#initialization--recovery): 1:1 mapping from replayed WAL entries -- During [WAL Flush](#wal-flush): mapping from batched MemTable fragments to WAL entry +1. Indexes are serialized to disk in the flushed MemTable's `_indices/` directory following the Lance table index format +2. The primary key bloom filter is serialized to `bloom_filter.bin` in the generation directory +3. The in-memory index structures may be retained as a cache for readers in the same process ### MemTable Flush Flushing the MemTable creates a new flushed MemTable (generation) with data and indexes: 1. Generate a random 8-character hex prefix (e.g., `a1b2c3d4`) -2. Create directory `_memwal/{region_uuid}/{random_hash}_gen_{current_generation}/` +2. Create directory `_mem_wal/{region_uuid}/{random_hash}_gen_{current_generation}/` 3. Identify WAL entries to include (from `replay_after_wal_id + 1` to the last flushed entry) 4. Create table manifest with `base_paths` pointing to the WAL directory 5. Add fragment entries referencing WAL files via `base_id` 6. Remap indexes using in-memory fragment mappings: - Read index entries referencing MemTable fragment IDs - - Translate to flushed MemTable fragment IDs using mappings (MemTable fragment ID → WAL entry ID relative to last replay) - - Write remapped indexes to `_memwal/{region_uuid}/{random_hash}_gen_{current_generation}/_indices/` -7. Write the manifest to `_memwal/{region_uuid}/{random_hash}_gen_{current_generation}/_versions/{version}.manifest` (using [V2 naming scheme](transaction.md#manifest-naming-schemes)) + - Translate to flushed MemTable fragment IDs using mappings (MemTable fragment ID -> WAL entry ID relative to last replay) + - Write remapped indexes to `_mem_wal/{region_uuid}/{random_hash}_gen_{current_generation}/_indices/` +7. Write the manifest to `_mem_wal/{region_uuid}/{random_hash}_gen_{current_generation}/_versions/{version}.manifest` (using [V2 naming scheme](transaction.md#manifest-naming-schemes)) 8. Update the region manifest: - Advance `replay_after_wal_id` to the last flushed WAL entry - Update `wal_id_last_seen` @@ -721,6 +492,17 @@ The random prefix ensures that flush retries write to a new directory, avoiding If the writer crashes before completing MemTable flush, the new writer replays WAL entries into memory with 1:1 fragment mapping, rebuilds the in-memory indexes, and can then perform a fresh MemTable flush with a new random prefix. +### Writer Fencing + +Before any manifest update (MemTable flush), a writer must verify its `writer_epoch` remains valid: + +- If `local_writer_epoch == stored_writer_epoch`: The writer is still active and may proceed +- If `local_writer_epoch < stored_writer_epoch`: The writer has been fenced and must abort + +Fenced writers must stop all operations immediately and notify pending writes of the failure. + +For a concrete example of fencing between two writers, see [Appendix 1: Writer Fencing Example](#appendix-1-writer-fencing-example). + ## Background Job Expectations Background jobs run independently from writers and handle asynchronous maintenance tasks. @@ -776,37 +558,12 @@ The MemWAL Index is authoritative for conflict resolution; the region manifest i For a concrete example, see [Appendix 2: Concurrent Merger Example](#appendix-2-concurrent-merger-example). -#### Implementation Consideration: Atomic Index Maintenance - -The merge commit should atomically update both data and indexes in the base table. -If data is merged but indexes are updated separately (e.g., via a background rebuild), there is a window where: - -1. Merged data exists in the base table but is not covered by base table indexes -2. The flushed MemTable (with its indexes) has been garbage collected -3. Queries must fall back to brute-force scans for the unindexed data, degrading performance - -To avoid this performance degradation: - -| Index Type | Recommended Approach | -|------------|---------------------| -| **Btree** | Incremental insert during merge transaction | -| **FTS** | Incremental update to posting lists and statistics | -| **Vector (IVF)** | Add vectors to existing partitions without retraining centroids | - -For vector indexes, adding to existing IVF partitions may cause partition imbalance over time. -Periodic rebalancing (e.g., SPFresh-style centroid updates) can address this, but the rebalancing operation itself should also be atomic with any data changes it affects. - -If atomic index maintenance is not feasible for a particular index type, implementations should either: - -- **Delay garbage collection**: Keep flushed MemTable indexes until base table indexes are updated -- **Track index coverage**: Maintain separate `index_merged_generation` to know which generations are covered by base table indexes - ### MemWAL Index Builder A background process periodically builds a new region snapshot: 1. Load the existing MemWAL Index to preserve configuration (`region_specs`, `maintained_indexes`) and merge progress (`merged_generations`) -2. List all region directories under `_memwal/` +2. List all region directories under `_mem_wal/` 3. For each region: - Load the region manifest - Scan WAL entries sequentially to find the actual last entry ID @@ -854,97 +611,66 @@ Garbage collection must verify that no flushed MemTable still references a WAL f ## Reader Expectations -### Consistency Guarantees +### Reader Consistency -Reader consistency depends on two dimensions: +Reader consistency depends on two factors: access to in-memory MemTables and the source of region metadata. -| Dimension | Options | -|-----------|---------| -| **MemTable access** | Has access to in-memory MemTable, or only persisted data | -| **Manifest source** | Reads region manifests directly, or uses MemWAL Index | +Strong consistency requires access to in-memory MemTables for all regions involved in the query and reading region manifests directly. +Otherwise, the query is eventually consistent due to missing unflushed data or stale MemWAL Index snapshots. -**Strong consistency** requires both: +### Region Pruning -1. Access to in-memory MemTables for **all** regions involved in the query -2. Reading region manifests directly (not via MemWAL Index) +Before executing queries, the planner evaluates filter predicates against region specs to determine which regions may contain matching data. +This pruning step reduces the number of regions to scan. -Otherwise, the query is **eventually consistent**. +For each filter predicate: -#### Consistency Matrix +1. Extract predicates on columns used in region specs +2. Evaluate which region values can satisfy the predicate +3. Prune regions whose values cannot match -| MemTable Access | Manifest Source | Consistency | -|-----------------|-----------------|-------------| -| All regions | Region manifest | **Strong** | -| All regions | MemWAL Index | Eventually consistent (index may be stale) | -| Partial/None | Region manifest | Eventually consistent (missing unflushed data) | -| Partial/None | MemWAL Index | Eventually consistent (both sources of staleness) | +For example, with a region spec using `bucket(user_id, 10)` and a filter `user_id = 123`: -#### Sources of Staleness +1. Compute `bucket(123, 10) = 3` +2. Only scan regions with bucket value 3 +3. Skip all other regions -- **Missing MemTable access**: Unflushed data in a writer's in-memory MemTable is not visible -- **Stale MemWAL Index**: Newly flushed MemTables are not visible until the index is rebuilt -- **Stale region manifest cache**: If readers cache region manifests, newly flushed MemTables may not be visible +Region pruning applies to both scan queries and prefilters in search queries. ### Query Planning -From the query planner's perspective, MemWAL data is abstracted as a mapping: - -``` -region -> generation -> Dataset -``` - -Where: +The query planner collects datasets from multiple sources and assembles them for unified query execution. +Datasets come from the base table (representing already-merged data), flushed MemTables (persisted but not yet merged), and optionally in-memory MemTables (if accessible). -- **Region**: UUID identifying the region -- **Generation**: Integer generation number (`-1` for base table, `1+` for MemTables) -- **Dataset**: Either in-memory MemTable or persisted flushed MemTable (Lance table) +Each dataset is tagged with a generation number: -1 for the base table, and positive integers for MemTable generations. +Within a region, the generation number determines data freshness, with higher numbers representing newer data. +Rows from different regions do not need deduplication since each primary key maps to exactly one region. -The planner collects datasets from: - -1. **Base table**: generation = -1 -2. **Flushed MemTables**: generations in range `(merged_generation, current_generation)` from region manifest or MemWAL Index -3. **In-memory MemTable**: generation = `current_generation` (if accessible) +The planner also collects bloom filters from each generation for staleness detection during search queries. ### Query Execution -Query execution unions all datasets and deduplicates by primary key. - -**Deduplication ranking** uses two virtual columns: - -- `_gen`: Generation number (-1 for base, 1+ for MemTables) -- `_rowaddr`: Row address within the dataset - +Query execution unions datasets within each region and deduplicates by primary key. +Deduplication uses two virtual columns: `_gen` (generation number, same value for all rows in a generation) and `_rowaddr` (row address within the dataset). The ordering for "newest" is: highest `_gen` first, then highest `_rowaddr` (within the same generation, later rows win). A single write batch may contain duplicate primary keys. Query execution must deduplicate, keeping only the newest row for each key. -For detailed query plans by query type, see [Appendix 3: Query Execution Examples](#appendix-3-query-execution-examples). - -## Durability Guarantees - -| Mode | Guarantee | Latency | -|------|-----------|---------| -| Durable write | Data persisted to object storage before return | Higher (S3 PUT latency) | -| Non-durable write | Data in memory only until next flush | Lower (memory write only) | - -Writers using non-durable writes accept potential data loss between the last flush and a crash. -The flush interval and buffer size thresholds control the maximum data at risk. - -For configuration details, see [Writer Configuration](#writer-configuration). - -## MemWAL Optimizations - -One key reason we call the whole system MemWAL is that we could perform the following 2 optimziations -to minimize flush latency: +For scan queries with filters, the plan unions datasets from relevant regions (after pruning), deduplicates within each region, then applies any remaining filters. +Early termination is possible with a streaming deduplicate operator. -### WAL Flush +For vector search queries, each generation's index is searched independently. +Results from base table and MemTable generations use the same distance metric since they share identical index configuration. +The bloom filter is used to detect stale results: for each candidate from generation G, check if the primary key exists in any bloom filter from generations > G. +If found, the candidate is filtered out because a newer version exists that did not match the query as well. -the list of fragments in MemTable can be viewed as an in-memory buffer of the WAL. This means instead of writing the same data twice to MemTable and WAL, we write data once to MemTable, and then WAL can be flushed from the data file in the fragments. +For full-text search queries, corpus statistics (document count, term frequencies) are aggregated across all generations to compute global BM25 parameters. +Each generation's FTS index is then searched with the global parameters, producing comparable scores. +Staleness filtering uses bloom filters similar to vector search. -### MemTable Flush +For point lookups, the planner can short-circuit by checking newest generations first and returning immediately when the key is found. -because the list of WAL entries are Lance data files, -we can directly treat them as the data files of the flushed MemTable. +For detailed query plan examples, see [Appendix 3: Query Execution Examples](#appendix-3-query-execution-examples). ## Appendices @@ -987,8 +713,8 @@ When Writer B performs crash recovery or MemTable flush: 3. Continues until an entry is not found (e.g., entry 18 doesn't exist) 4. Finds entries 13, 14, 15, 16, 17 5. Reads each file's `writer_epoch` from schema metadata -6. Entries 13, 14, 15 have `writer_epoch=6` which is ≤ current epoch (7) → **valid, will be replayed** -7. Entries 16, 17 have `writer_epoch=7` → **valid, will be replayed** +6. Entries 13, 14, 15 have `writer_epoch=6` which is <= current epoch (7) -> **valid, will be replayed** +7. Entries 16, 17 have `writer_epoch=7` -> **valid, will be replayed** #### Key Points @@ -1037,27 +763,7 @@ Two mergers both try to merge generation 6 concurrently. Merger B's conflict resolution detected that generation 6 was already merged by checking the MemWAL Index in the conflicting commit. -#### Scenario 2: Stale Merger with Out-of-Order Attempt - -Merger B has a stale view and tries to merge an older generation. - -| Step | Merger A | Merger B | MemWAL Index | Region Manifest | -|------|----------|----------|--------------|-----------------| -| 1 | Reads index, region manifest | | merged_gen=5 | merged_gen=5 | -| 2 | Merges gen 6, commits | | **merged_gen=6** | merged_gen=5 | -| 3 | Updates region manifest | | merged_gen=6 | **merged_gen=6** | -| 4 | Merges gen 7, commits | | **merged_gen=7** | merged_gen=6 | -| 5 | Updates region manifest | | merged_gen=7 | **merged_gen=7** | -| 6 | | Reads stale index | merged_gen=7 | merged_gen=7 | -| 7 | | Thinks gen 6 needs merging | | | -| 8 | | Tries to commit gen 6 | | | -| 9 | | **Conflict**: reads new index | | | -| 10 | | Sees merged_gen=7 >= 6, aborts | | | -| 11 | | Reloads index, skips gen 6, 7 | | | - -Even with a stale MemWAL Index, Merger B correctly detected that generation 6 was already merged by checking the authoritative MemWAL Index in the conflicting commit. - -#### Scenario 3: Crash After Table Commit +#### Scenario 2: Crash After Table Commit Merger A crashes after committing to the table but before updating the region manifest. @@ -1086,264 +792,138 @@ The MemWAL Index is authoritative. Even though the region manifest was stale, Me 4. **No progress regression**: Because MemWAL Index is updated atomically with data, concurrent mergers cannot regress the merge progress. -5. **Crash recovery is safe**: If a merger crashes after table commit but before region manifest update, subsequent mergers use MemWAL Index to skip already-merged generations. - ### Appendix 3: Query Execution Examples -This appendix provides query plan examples. All examples assume the planner has collected datasets as: +This appendix provides query plan examples using custom execution nodes optimized for MemWAL's data model. +See [Appendix 4: Execution Nodes](#appendix-4-execution-nodes) for detailed node descriptions. -``` -datasets = { - -1: base_table, # generation -1 = base table - 1: flushed_gen_1, # flushed MemTable generation 1 - 2: flushed_gen_2, # flushed MemTable generation 2 - 3: in_memory_memtable, # current generation (if accessible) -} -``` +All examples assume multiple regions with datasets organized as: -The core pattern for all queries: +``` +region_A: + gen -1: base_table (rows with region_A's pk range) + gen 1: flushed_gen_1 + gen 2: in_memory_memtable -1. **Union** all datasets with their generation number -2. **Deduplicate** by primary key, ranking by `(_gen DESC, _rowaddr DESC)` -3. **Apply** query-specific operators (filter, sort, limit) +region_B: + gen -1: base_table (rows with region_B's pk range) + gen 1: flushed_gen_1 + gen 2: flushed_gen_2 + gen 3: in_memory_memtable +``` #### Scan Queries +For scan queries, each region is processed independently since primary keys don't overlap between regions. + ``` -GlobalLimitExec: limit=n - DeduplicateExec: partition_by=[primary_key], order_by=[_gen DESC, _rowaddr DESC] +UnionExec # Union results from all regions + # Region A + DeduplicateExec: partition_by=[pk], order_by=[_gen DESC, _rowaddr DESC] UnionExec - ScanExec: dataset[gen=-1], projection=[columns], filter=[pushed_down] - ScanExec: dataset[gen=1], projection=[columns], filter=[pushed_down] - ScanExec: dataset[gen=2], projection=[columns], filter=[pushed_down] - ScanExec: dataset[gen=3], projection=[columns] + ScanExec: region_A[gen=2], filter=[pushed_down] + ScanExec: region_A[gen=1], filter=[pushed_down] + ScanExec: region_A[gen=-1], filter=[pushed_down] + # Region B + DeduplicateExec: partition_by=[pk], order_by=[_gen DESC, _rowaddr DESC] + UnionExec + ScanExec: region_B[gen=3], filter=[pushed_down] + ScanExec: region_B[gen=2], filter=[pushed_down] + ScanExec: region_B[gen=1], filter=[pushed_down] + ScanExec: region_B[gen=-1], filter=[pushed_down] ``` -Early termination is possible with a streaming deduplicate operator. - #### Vector Search Queries -Vector search requires special handling for staleness detection. Consider this scenario: - -1. Base table has `pk=123` with vector `v1` that matches the query (distance = 0.1) -2. MemTable has `pk=123` with updated vector `v2` that doesn't match (distance = 0.9) -3. KNN search on base table returns `pk=123` (good score) -4. KNN search on MemTable does NOT return `pk=123` (v2 is far from query) -5. Without staleness detection, the old version from base table would be incorrectly returned - -The solution uses the **primary key btree index** to filter out stale results: +Vector search uses bloom filters to detect stale results across generations within each region. ``` GlobalLimitExec: limit=k SortExec: order_by=[_dist ASC] - FilterStaleExec: pk_indexes=[btree[gen=3], btree[gen=2], btree[gen=1]] - UnionExec - KNNExec: dataset[gen=3], k=k*overfetch -- highest gen first - KNNExec: dataset[gen=2], k=k*overfetch - KNNExec: dataset[gen=1], k=k*overfetch - KNNExec: dataset[gen=-1], k=k*overfetch + UnionExec # Union results from all regions + # Region A + FilterStaleExec: bloom_filters=[bf[gen=2], bf[gen=1]] + UnionExec + KNNExec: region_A[gen=2], k=k*overfetch + KNNExec: region_A[gen=1], k=k*overfetch + KNNExec: region_A[gen=-1], k=k*overfetch + # Region B + FilterStaleExec: bloom_filters=[bf[gen=3], bf[gen=2], bf[gen=1]] + UnionExec + KNNExec: region_B[gen=3], k=k*overfetch + KNNExec: region_B[gen=2], k=k*overfetch + KNNExec: region_B[gen=1], k=k*overfetch + KNNExec: region_B[gen=-1], k=k*overfetch ``` -For each candidate from generation G, `FilterStaleExec` checks if the primary key exists in btree indexes of generations > G. If found, the candidate is filtered out. The newer version doesn't participate in ranking since it didn't match the query. +For each candidate from generation G, `FilterStaleExec` checks if the primary key exists in bloom filters of generations > G. +If found, the candidate is filtered out because a newer version exists that was not as relevant to the query. #### Full-Text Search Queries -Full-text search has two challenges across generations: - -1. **Staleness**: A document may match a query in an older generation but not in a newer generation after the text was updated -2. **Score comparability**: BM25 scores depend on corpus statistics which differ per generation - -**Solution:** Use global BM25 scoring (Lucene-style) with staleness filtering. +Full-text search aggregates corpus statistics across generations for globally-comparable BM25 scores. ``` --- Physical plan GlobalLimitExec: limit=k SortExec: order_by=[_bm25 DESC] - FilterStaleExec: pk_indexes=[btree[gen=3], btree[gen=2], btree[gen=1]] - GlobalBM25Exec: -- aggregates stats, creates single scorer - UnionExec - FTSExec: dataset[gen=3], query="search terms" - FTSExec: dataset[gen=2], query="search terms" - FTSExec: dataset[gen=1], query="search terms" - FTSExec: dataset[gen=-1], query="search terms" -``` - -**GlobalBM25Exec** performs: - -1. Collects `doc_count` and `sum_total_term_freq` from all FTS indexes -2. Computes global `N` and `avgdl` -3. For query terms, sums `df(term)` from all indexes to compute global IDF -4. Passes global BM25 parameters to each `FTSExec` -5. All candidates receive globally-comparable BM25 scores - -This ensures fair ranking between base table (large corpus) and MemTable (small corpus) results. -See [Full-Text Search Indexes](#full-text-search-indexes) for details on the global scoring approach. + UnionExec # Union results from all regions + # Region A + FilterStaleExec: bloom_filters=[bf[gen=2], bf[gen=1]] + GlobalBM25Exec # Aggregates stats across region A generations + UnionExec + FTSExec: region_A[gen=2], query="search terms" + FTSExec: region_A[gen=1], query="search terms" + FTSExec: region_A[gen=-1], query="search terms" + # Region B + FilterStaleExec: bloom_filters=[bf[gen=3], bf[gen=2], bf[gen=1]] + GlobalBM25Exec # Aggregates stats across region B generations + UnionExec + FTSExec: region_B[gen=3], query="search terms" + FTSExec: region_B[gen=2], query="search terms" + FTSExec: region_B[gen=1], query="search terms" + FTSExec: region_B[gen=-1], query="search terms" +``` + +`GlobalBM25Exec` collects document counts and term frequencies from all FTS indexes within the region, computes global BM25 parameters, and passes them to each `FTSExec` for comparable scoring. #### Point Lookups -Point lookups can short-circuit by checking newest generations first: +Point lookups first determine the target region using the region spec, then short-circuit by checking newest generations first. ``` --- Physical plan (short-circuit evaluation) -CoalesceExec: return_first_non_null - -- Check newest generation first, take last row (scan is ordered by _rowaddr) - TakeLastExec: - ScanExec: dataset[gen=3], filter=[primary_key = target] +# After region pruning: only region_A needs to be checked +CoalesceFirstExec: return_first_non_null TakeLastExec: - ScanExec: dataset[gen=2], filter=[primary_key = target] + ScanExec: region_A[gen=2], filter=[pk = target] TakeLastExec: - ScanExec: dataset[gen=1], filter=[primary_key = target] + ScanExec: region_A[gen=1], filter=[pk = target] TakeLastExec: - ScanExec: dataset[gen=-1], filter=[primary_key = target] + ScanExec: region_A[gen=-1], filter=[pk = target] ``` -Point lookups terminate early once the key is found. Since scans are naturally ordered by `_rowaddr`, we take the last matching row without explicit sorting. +Point lookups terminate early once the key is found. ### Appendix 4: Execution Nodes -This appendix describes custom execution nodes for MemWAL query execution. These nodes are optimized for MemWAL's data model where each dataset has a fixed `_gen` and rows are naturally ordered by `_rowaddr`. +This appendix describes custom execution nodes for MemWAL query execution. #### DeduplicateExec Deduplicates rows by primary key, keeping the row with highest `(_gen, _rowaddr)`. - -**Semantics:** -``` -For each primary key across all input datasets: - Keep the row with max(_gen), breaking ties by max(_rowaddr) -``` - -**Optimized implementation:** - -Since each dataset has a fixed `_gen` and rows are naturally ordered by `_rowaddr`: - -1. Process datasets from highest `_gen` to lowest -2. Maintain a set of seen primary keys -3. For each dataset: - - Scan rows (naturally ordered by `_rowaddr`) - - For each primary key, buffer rows until key changes, emit last row - - Skip primary keys already in seen set - - Add emitted primary keys to seen set -4. Stream results without full materialization - -**Complexity:** O(n) where n = total rows, with O(k) memory where k = unique primary keys. +Since each dataset has a fixed `_gen` and rows are naturally ordered by `_rowaddr`, this can be implemented as a streaming operator without full materialization. #### TakeLastExec -Takes the last row from an ordered input stream. - -**Semantics:** -``` -Buffer rows until input exhausted, emit final row -``` - -**Optimized implementation:** - -Since we only need the last row: - -1. Iterate through input, keeping only the most recent row in memory -2. On input exhaustion, emit the buffered row (or nothing if empty) - -**Complexity:** O(n) time, O(1) memory (single row buffer). +Takes the last row from an ordered input stream using O(1) memory. #### CoalesceFirstExec -Returns the first non-null/non-empty result from multiple inputs, with short-circuit evaluation. - -**Semantics:** -``` -For each input in order: - Execute input - If result is non-empty, return it immediately - Otherwise, continue to next input -Return empty if all inputs are empty -``` - -**Implementation:** - -1. Inputs are evaluated lazily in order -2. On first non-empty result, return immediately without evaluating remaining inputs -3. Useful for point lookups: check newest generation first, return on match - -**Complexity:** Best case O(1) inputs evaluated, worst case O(k) where k = number of inputs. +Returns the first non-empty result from multiple inputs with short-circuit evaluation. +Inputs are evaluated lazily in order; on first non-empty result, remaining inputs are not evaluated. #### FilterStaleExec -Filters out rows that have a newer version in a higher generation. Used for search workloads where the newer version may not appear in search results (e.g., updated vector no longer matches query). - -**Parameters:** - -- `pk_indexes`: List of primary key btree indexes for each generation, ordered by generation descending - -**Why btree index lookup is necessary:** - -A naive approach would only check if the same primary key appears in search results from newer generations. However, this fails when: - -1. Vector `v1` (generation 1) matches query → returned by KNN -2. Vector `v2` (generation 2, same pk) doesn't match query → NOT returned by KNN -3. Naive approach: pk only appears once in results → not filtered (WRONG) -4. Btree approach: check btree[gen=2] for pk → found → filtered out (CORRECT) - -**Behavior:** - -Stale results are **filtered out** (not included in output). They do not participate in final ranking. This is the correct semantic for search: if a row was updated and the new version doesn't match the query, the old matching version should not be returned. - -**Algorithm (iterative roll-up):** - -Process generations from highest to lowest, accumulating known primary keys: - -``` -known_pks = {} # pks confirmed to exist in processed generations - -For gen in [highest_gen, ..., lowest_gen]: - gen_candidates = candidates.filter(_gen == gen) - - For each candidate (pk, gen) in gen_candidates: - # Fast path: pk already seen in a higher generation's results - If pk in known_pks: - Filter out candidate (stale) - Continue - - # Slow path: check btree indexes of higher generations - For check_gen in [highest_gen, ..., gen+1]: - If pk_indexes[check_gen].contains(pk): - Filter out candidate (stale) - Break - - If not filtered: - Emit candidate - - # Roll up: add all pks from this generation's btree to known set - # This enables fast-path checks for lower generations - known_pks.add_all(pk_indexes[gen].keys()) -``` - -**Why roll-up matters:** - -Without roll-up, each candidate requires btree lookups in all higher generations. With roll-up: - -1. Process gen 3: emit candidates, add gen 3 pks to `known_pks` -2. Process gen 2: check `known_pks` first (O(1)), then btree if needed -3. Process gen 1: many pks already in `known_pks`, fewer btree lookups - -**Complexity:** - -- Best case: O(n) when most pks are found via `known_pks` fast path -- Worst case: O(n × g × log m) where n = candidates, g = generations, m = rows per generation - -**Optimization:** For in-memory btree indexes (cached from flushed MemTables), lookups are O(log m) with no I/O. The base table may use a different staleness check mechanism (e.g., deletion vectors) since it doesn't maintain an in-memory btree. - -#### Usage in Query Plans - -| Query Type | Execution Pattern | -|------------|-------------------| -| Scan | `DeduplicateExec` → streams deduplicated rows | -| Point Lookup | `CoalesceFirstExec` → `TakeLastExec` per dataset | -| Vector Search | `FilterStaleExec(pk_indexes)` → `SortExec(_dist)` → `LimitExec` | -| Full-Text Search | `GlobalBM25Exec` → `FilterStaleExec(pk_indexes)` → `SortExec(_bm25)` → `LimitExec` | - -**Note on Vector/FTS Search:** - -- **Staleness filtering:** `FilterStaleExec` uses primary key btree indexes to filter out stale versions even when the newer version doesn't appear in search results (e.g., updated vector/text no longer matches query). Stale results are removed before ranking, ensuring top-k contains only current versions. -- **Vector scoring:** Distances are directly comparable across generations because MemTable inherits quantization (PQ codebook) from base table. -- **FTS scoring:** BM25 scores are made comparable via `GlobalBM25Exec`, which aggregates corpus statistics across generations (Lucene-style multi-segment scoring). +Filters out rows that have a newer version in a higher generation. +For each candidate with primary key `pk` from generation G, checks bloom filters of generations > G. +If the bloom filter indicates the key may exist in a newer generation, the candidate is filtered out. +False positives from bloom filters may cause some valid results to be filtered, but this is acceptable for search workloads where approximate results are expected. From 5a7d3790440de52093f7648cc3899cb033c4d291 Mon Sep 17 00:00:00 2001 From: Heng Ge Date: Tue, 13 Jan 2026 00:07:48 -0800 Subject: [PATCH 3/9] Update --- docs/src/format/table/mem_wal.md | 454 +++++++++++---------- docs/src/format/table/mem_wal_regional.png | Bin 551389 -> 0 bytes docs/src/images/mem_wal_regional.png | Bin 0 -> 566176 bytes 3 files changed, 237 insertions(+), 217 deletions(-) delete mode 100644 docs/src/format/table/mem_wal_regional.png create mode 100644 docs/src/images/mem_wal_regional.png diff --git a/docs/src/format/table/mem_wal.md b/docs/src/format/table/mem_wal.md index 6de8ac2d921..c8d514c171f 100644 --- a/docs/src/format/table/mem_wal.md +++ b/docs/src/format/table/mem_wal.md @@ -2,6 +2,9 @@ Lance MemTable & WAL (MemWAL) specification describes a Log-Structured-Merge (LSM) tree architecture for Lance tables, enabling high-performance streaming write workloads. +!!!note + MemWAL requires the table to have an [unenforced primary key](index.md#unenforced-primary-key) defined. + ## Overall Architecture ![MemWAL Overview](../../images/mem_wal_overview.png) @@ -13,10 +16,23 @@ Under the MemWAL setup, the Lance table is called the **base table**. ### Region A **Region** is the main unit to horizontally scale out writes. + Each region has exactly one active writer at any time, using **epoch-based fencing** to guarantee single-writer semantics without distributed coordination. Writers claim a region by incrementing the writer epoch, then write data to that region. Data in each region is merged into the base table gradually in the background. +Regions must contain rows that are **mutually exclusive**. +Two regions contain rows with the same primary key, the following scenario can cause data corruption: + +1. Region A receives a write with primary key `pk=1` at time T1 +2. Region B receives a write with primary key `pk=1` at time T2 (T2 > T1) +3. The row in region B is merged into the base table first +4. The row in region A is merged into the base table second +5. The row from Region A (older) now overwrites the row from Region B (newer) + +This violates the expected "last write wins" semantics. +By ensuring each primary key is assigned to exactly one region via the region spec, merge order between regions becomes irrelevant for correctness. + ### MemWAL Index A **MemWAL Index** is the centralized structure for all MemWAL metadata for a base table. @@ -37,7 +53,7 @@ See [MemWAL Index Details](#memwal-index-details) for the complete structure. ## Region Architecture -![Region Architecture](mem_wal_regional.png) +![Region Architecture](../../images/mem_wal_regional.png) Within a region, writes enter its MemTable and are flushed to the regional WAL for durability. The MemTable is flushed to storage as a Flushed MemTable based on memory pressure and other conditions. @@ -48,6 +64,12 @@ Flushed MemTables are then asynchronously merged into the base table. An in-memory Lance table that buffers incoming writes. Each write inserts a fragment in the MemTable, making data immediately queryable without waiting for persistence. +In addition to the data fragments, a MemTable maintains: + +- **Primary key bloom filter**: For efficient existence checks during staleness detection +- **In-memory index builders**: Incremental index structures that mirror base table indexes, enabling indexed queries on unflushed data +- **WAL fragment mapping**: Tracks correspondence between MemTable fragment IDs and WAL entry IDs for index remapping during flush + ### WAL Write-Ahead Log (WAL) serves as the durable storage of MemTable. @@ -131,7 +153,7 @@ To read the latest manifest version: This approach uses HEAD requests instead of LIST operations in cloud storage, which is generally faster and is friendly to systems like S3 Express that do not support lexicographically sorted listing. !!!note - This works because the write rate to region manifests is significantly lower than typical read rates. Region manifests are only updated when region metadata changes (MemTable flush, merge completion), not on every write. This ensures HEAD requests will eventually terminate and find the latest version. + This works because the write rate to region manifests is significantly lower than read rates. Region manifests are only updated when region metadata changes (MemTable flush, merge completion), not on every write. This ensures HEAD requests will eventually terminate and find the latest version. All region manifest versions are stored in `_mem_wal/{region_id}/manifest` directory. @@ -144,7 +166,7 @@ The region manifest is updated atomically in the following cases: |---------|----------------|---------| | [Initialization & Recovery](#initialization--recovery) | `writer_epoch` | Incremented when writer claims the region | | [MemTable Flush](#memtable-flush) | `replay_after_wal_id`, `wal_id_last_seen`, `current_generation`, `flushed_generations` | After flushing MemTable to storage | -| [Merge to Base Table](#merge-workflow) | `merged_generation`, `flushed_generations` | After merging a flushed MemTable; removes merged entry | +| [MemTable Merger](#merge-workflow) | `merged_generation`, `flushed_generations` | After merging a flushed MemTable; removes merged entry | | [MemWAL Index Builder](#memwal-index-builder) | `wal_id_last_seen` | Periodically scans WAL entries and updates hint | !!!note @@ -161,7 +183,7 @@ Here is a recap of the storage layout with all the files and concepts defined so {table_path}/ ├── _indices/ │ └── {index_uuid}/ # MemWAL Index (uses standard index storage) -│ └── regions.binpb # Serialized region snapshots (protobuf binary) +│ └── index.lance # Serialized region snapshots (Lance file) │ └── _mem_wal/ └── {region_uuid}/ # Region directory (UUID v4) @@ -189,18 +211,9 @@ The index stores its data in two parts: 1. **Index details** (`index_details` in `IndexMetadata`): Contains configuration, merge progress, and snapshot metadata 2. **Region snapshots**: Stored as a Lance file or inline, depending on region count -### Index Details Schema +### Index Details -The `index_details` field in `IndexMetadata` contains a `MemWalIndexDetails` protobuf message with: - -| Field | Type | Description | -|-------|------|-------------| -| `snapshot_timestamp` | int64 | When the index was built (Unix timestamp in seconds) | -| `num_regions` | uint32 | Number of regions in the snapshot | -| `inline_snapshots` | bytes | Inline snapshot data for small region counts (optional) | -| `region_specs` | repeated RegionSpec | Region specs defining how rows map to regions | -| `maintained_indexes` | repeated string | Index names to maintain in MemTables | -| `merged_generations` | repeated MergedGeneration | Last generation merged to base table per region | +The `index_details` field in `IndexMetadata` contains a `MemWalIndexDetails` protobuf message with the following key fields: **Configuration fields** (`region_specs`, `maintained_indexes`) are the source of truth for MemWAL configuration. Writers read these fields to determine how to partition data and which indexes to maintain. @@ -209,7 +222,7 @@ Writers read these fields to determine how to partition data and which indexes t This field is updated atomically with merge-insert data commits, enabling conflict resolution when multiple mergers operate concurrently. Each entry contains the region UUID and generation number. -**Region snapshot fields** (`snapshot_timestamp`, `num_regions`, `inline_snapshots`) provide a point-in-time snapshot of region states. +**Region snapshot fields** (`snapshot_ts_millis`, `num_regions`, `inline_snapshots`) provide a point-in-time snapshot of region states. The actual region manifests remain authoritative for region state. When `num_regions` is 0, the `inline_snapshots` field may be `None` or an empty Lance file with 0 rows but proper schema. @@ -251,22 +264,6 @@ Each region field has the following properties: | `expression` | A DataFusion SQL expression for custom logic, specify this or `transform` | | `result_type` | The output type of the region value | -#### Primary Key Constraint - -The `source_ids` across all region fields must include all primary key columns. -This ensures rows with the same primary key always map to the same region, which is required for correctness. - -If two regions contain rows with the same primary key, the following scenario can cause data corruption: - -1. Region A receives a write with primary key `pk=1` at time T1 -2. Region B receives a write with primary key `pk=1` at time T2 (T2 > T1) -3. The row in region B is merged into the base table first -4. The row in region A is merged into the base table second -5. The row from Region A (older) now overwrites the row from Region B (newer) - -This violates the expected "last write wins" semantics. -By ensuring each primary key is assigned to exactly one region via the region spec, merge order between regions becomes irrelevant for correctness. - #### Region Expression A **Region Expression** is a [DataFusion SQL expression](https://datafusion.apache.org/user-guide/sql/index.html) that derives a region value from source column(s). @@ -321,7 +318,7 @@ This keeps the index metadata compact while avoiding an additional file read for **External Lance file**: For large region counts, snapshots are stored as a Lance file at `_indices/{UUID}/index.lance`. This file uses standard Lance format with the region snapshot schema, enabling efficient columnar access and compression. -### Region Snapshot Schema +### Region Snapshot Arrow Schema Region snapshots are stored as a Lance file with one row per region. The schema has one column per `RegionManifest` field plus region spec columns: @@ -337,21 +334,16 @@ The schema has one column per `RegionManifest` field plus region spec columns: | `current_generation` | `uint64` | Next generation to flush | | `merged_generation` | `uint64` | Last generation merged to base | | `flushed_generations` | `list>` | Flushed MemTable paths | -| `region_values` | `struct<{field_id}: {result_type}, ...>` | Region field values from spec | - -This schema directly corresponds to the fields in the `RegionManifest` protobuf message plus the computed region values. +| `region_field_{field_id}` | varies | Region field value (one column per field in region spec) | -### Staleness Handling +For example, with a region spec containing a field `user_bucket` of type `int32`: -Since the index is eventually consistent, readers should handle stale data: - -- A flushed MemTable listed in `flushed_generations` may have been merged and garbage collected -- New flushed MemTables may exist that are not yet in `flushed_generations` -- WAL entries may have advanced beyond what the index shows - -The `snapshot_timestamp` field indicates when the index was built; readers can use this to estimate staleness and decide whether to refresh. +| Column | Type | Description | +|--------|------|-------------| +| ... | ... | (base columns above) | +| `region_field_user_bucket` | `int32` | Bucket value for this region | -For authoritative state, readers may load individual region manifests directly from `_mem_wal/{region_uuid}/manifest/`. +This schema directly corresponds to the fields in the `RegionManifest` protobuf message plus the computed region field values. ### Vector Index Configuration @@ -363,10 +355,12 @@ For each vector index on the base table, MemTable uses the same index type (IVF- This ensures distances are precise and comparable across generations. The base table vector index should not change the codebook once MemWAL is enabled. -To switch codebooks, a migration is required: create another vector index with the new codebook, configure MemTable to maintain both indexes, and eventually drop the old index after all data has been merged with the new codebook. +To switch codebooks, a migration is required: create another vector index with the new codebook, configure MemTable to maintain both indexes, and eventually drop the old index after all readers are using the new codebook and all MemTables have indexes using the new codebook. ## Writer Expectations +A writer operates on a single region within a single process and may spawn asynchronous tasks for background operations like WAL flush and MemTable flush. + ### Writer Configuration Writers can be configured with the following options that affect write behavior: @@ -387,11 +381,11 @@ Writer operations can be categorized by their synchronous or asynchronous nature | Operation | Mode | Description | |-----------|------|-------------| -| Write to MemTable | Synchronous | Data inserted into in-memory fragments | -| WAL Flush | Configurable | Synchronous with durable writes, asynchronous otherwise | -| Index Update | Configurable | Synchronous with indexed writes, asynchronous otherwise | -| MemTable Flush | Asynchronous | Triggered by thresholds, runs in background | -| Merge to Base Table | Asynchronous | Background job merges flushed MemTables | +| [Initialization & Recovery](#initialization--recovery) | Synchronous | Claims region and replays WAL entries | +| [Write to MemTable](#write-operations) | Synchronous | Data inserted into in-memory fragments | +| [WAL Flush](#wal-flush) | Configurable | Synchronous with durable writes, asynchronous otherwise | +| [Index Update](#memtable-indexing) | Configurable | Synchronous with indexed writes, asynchronous otherwise | +| [MemTable Flush](#memtable-flush) | Asynchronous | Triggered by thresholds, runs in background | ### Initialization & Recovery @@ -402,9 +396,9 @@ A writer must claim a region before performing any write operations: 3. Atomically write a new manifest 4. If the write fails (another writer claimed the epoch), reload the manifest and retry with a higher epoch 5. After initialization, read WAL entries sequentially from `replay_after_wal_id + 1` until not found -6. Replay valid WAL entries (those with `writer_epoch` <= current epoch) to reconstruct the MemTable with 1:1 fragment mapping (each WAL entry becomes one MemTable fragment) +6. Replay valid WAL entries (those with `writer_epoch` <= current epoch) to reconstruct the MemTable with 1:1 [WAL fragment mapping](#wal-fragment-mapping-construction) (each WAL entry becomes one MemTable fragment) -After recovery, the writer tracks subsequent fragment mappings as new WAL flushes occur (see [WAL Flush](#wal-flush)). +After initialization, the writer updates the [WAL fragment mapping](#wal-fragment-mapping-construction) as new [WAL flushes](#wal-flush) occur. ### Write Operations @@ -430,7 +424,7 @@ WAL flush batches pending MemTable fragments into a single Lance data file: 4. Write the footer containing batched data file metadata and `writer_epoch` in schema metadata 5. Complete the WAL entry write atomically 6. Mark fragments as flushed in the MemTable -7. Record fragment mappings (MemTable fragment IDs in this batch -> WAL entry ID relative to last replay) for index remapping during [MemTable Flush](#memtable-flush) +7. Update the [WAL fragment mapping](#wal-fragment-mapping-construction) (MemTable fragment IDs in this batch -> WAL entry ID and positions) for index remapping during [MemTable Flush](#memtable-flush) !!!note The region manifest is **not** updated on every WAL flush. The `wal_id_last_seen` field is a hint that can be updated: @@ -468,6 +462,26 @@ When a MemTable is flushed to storage: 2. The primary key bloom filter is serialized to `bloom_filter.bin` in the generation directory 3. The in-memory index structures may be retained as a cache for readers in the same process +### WAL Fragment Mapping Construction + +The WAL fragment mapping tracks the correspondence between MemTable fragment IDs and WAL entry IDs. +This mapping is essential for remapping indexes during [MemTable flush](#memtable-flush), since indexes reference MemTable fragment IDs but the flushed MemTable references WAL entry IDs. + +The mapping is structured as: `MemTable fragment ID -> (WAL entry ID, position within entry)` + +Where: + +- **MemTable fragment ID**: The fragment's position in the MemTable (0-indexed within the current generation) +- **WAL entry ID**: The WAL entry containing this fragment's data (relative to `replay_after_wal_id`) +- **Position within entry**: The fragment's position within the WAL entry (since multiple fragments may be batched) + +The mapping is updated in two scenarios: + +1. **[Initialization & Recovery](#initialization--recovery)**: During WAL replay, each replayed WAL entry creates MemTable fragments with 1:1 mapping (one fragment per WAL entry, position 0) +2. **[WAL Flush](#wal-flush)**: After flushing pending fragments to a new WAL entry, the mapping records which MemTable fragments were written to which WAL entry and their positions + +During [MemTable flush](#memtable-flush), indexes are remapped by translating MemTable fragment IDs to the corresponding WAL entry references using this mapping. + ### MemTable Flush Flushing the MemTable creates a new flushed MemTable (generation) with data and indexes: @@ -477,9 +491,9 @@ Flushing the MemTable creates a new flushed MemTable (generation) with data and 3. Identify WAL entries to include (from `replay_after_wal_id + 1` to the last flushed entry) 4. Create table manifest with `base_paths` pointing to the WAL directory 5. Add fragment entries referencing WAL files via `base_id` -6. Remap indexes using in-memory fragment mappings: +6. Remap indexes using the [WAL fragment mapping](#wal-fragment-mapping-construction): - Read index entries referencing MemTable fragment IDs - - Translate to flushed MemTable fragment IDs using mappings (MemTable fragment ID -> WAL entry ID relative to last replay) + - Translate to flushed MemTable fragment IDs using the mapping - Write remapped indexes to `_mem_wal/{region_uuid}/{random_hash}_gen_{current_generation}/_indices/` 7. Write the manifest to `_mem_wal/{region_uuid}/{random_hash}_gen_{current_generation}/_versions/{version}.manifest` (using [V2 naming scheme](transaction.md#manifest-naming-schemes)) 8. Update the region manifest: @@ -490,7 +504,7 @@ Flushing the MemTable creates a new flushed MemTable (generation) with data and The random prefix ensures that flush retries write to a new directory, avoiding conflicts with partially written files from failed attempts. Only the directory recorded in `flushed_generations` is considered valid. -If the writer crashes before completing MemTable flush, the new writer replays WAL entries into memory with 1:1 fragment mapping, rebuilds the in-memory indexes, and can then perform a fresh MemTable flush with a new random prefix. +If the writer crashes before completing MemTable flush, the new writer replays WAL entries into memory with 1:1 [WAL fragment mapping](#wal-fragment-mapping-construction), rebuilds the in-memory indexes, and can then perform a fresh MemTable flush with a new random prefix. ### Writer Fencing @@ -507,7 +521,7 @@ For a concrete example of fencing between two writers, see [Appendix 1: Writer F Background jobs run independently from writers and handle asynchronous maintenance tasks. -### Merge to Base Table +### MemTable Merger Flushed MemTables are merged to the base table in generation order using Lance's merge-insert operation. @@ -520,32 +534,23 @@ Flushed MemTables are merged to the base table in generation order using Lance's - Open it as a Lance table - Execute merge-insert into the base table, atomically updating the MemWAL Index: - Set `merged_generations[region_id]` to this generation - - On commit conflict, apply [conflict resolution rules](#merge-commit-conflict-resolution) + - On commit conflict, apply [conflict resolution rules](#conflict-resolution-and-concurrency) - On successful commit, update the region manifest: set `merged_generation` to this generation and remove the entry from `flushed_generations` - If the region manifest update fails, continue to the next generation (MemWAL Index is authoritative) -4. After merge, the flushed MemTable and its referenced WAL files may be garbage collected (see [Garbage Collection](#garbage-collection)) +4. After merge, the flushed MemTable and its referenced WAL files may be garbage collected (see [Garbage Collector](#garbage-collector)) Ordered merge ensures correct upsert semantics: flushed MemTables with higher generation numbers overwrite those with lower numbers. -#### Merge Commit Conflict Resolution - -When a merge-insert commit to the base table encounters a version conflict, the merger reads the conflicting commit's MemWAL Index: - -- **Incompatible conflict**: If the conflicting commit's `merged_generations[region_id] >= my_generation`, abort without retry. The data is either already merged (same generation) or superseded (higher generation). -- **Compatible conflict**: Otherwise, retry the commit as normal. - -After aborting due to an incompatible conflict, reload the MemWAL Index and region manifest, then continue to the next unmerged generation. - -This conflict resolution prevents redundant work and ensures mergers don't regress the merge progress. - -#### Concurrent Mergers and Idempotency +#### Conflict Resolution and Concurrency Multiple mergers may operate on the same region concurrently. This is safe due to: 1. **Atomic MemWAL Index update**: The `merged_generations` in MemWAL Index is updated atomically with the data commit -2. **Conflict resolution**: Incompatible commits (same region, higher/equal generation) cause abort, not retry +2. **Conflict resolution**: When a merge-insert commit encounters a version conflict, the merger reads the conflicting commit's MemWAL Index. If `merged_generations[region_id] >= my_generation`, abort without retry (data already merged or superseded). Otherwise, retry the commit as normal. 3. **Merge-insert idempotency**: If two mergers merge the same generation before either commits, both write identical data (primary key upsert semantics) +After aborting due to a conflict, reload the MemWAL Index and region manifest, then continue to the next unmerged generation. + If a merger crashes after committing to the base table but before updating the region manifest: - The MemWAL Index has `merged_generations[region_id] = N` @@ -567,7 +572,7 @@ A background process periodically builds a new region snapshot: 3. For each region: - Load the region manifest - Scan WAL entries sequentially to find the actual last entry ID - - If the observed WAL ID is greater than `wal_id_last_seen`, update the region manifest + - If the observed WAL ID is greater than `wal_id_last_seen`, update the region manifest (ignore errors since this is best-effort) - Copy manifest fields (including `flushed_generations`) into a region snapshot row 4. Determine storage strategy based on region count: - If `num_regions <= threshold`: Serialize as Lance file bytes to `inline_snapshots` @@ -582,21 +587,9 @@ This process serves two purposes: The build frequency is implementation-defined. More frequent builds reduce staleness but increase I/O overhead. -#### Configuration Updates - -To update MemWAL configuration (add/remove region specs or maintained indexes): - -1. Load the existing MemWAL Index -2. Modify the configuration fields (`region_specs`, `maintained_indexes`) -3. Keep the existing `merged_generations` and region snapshots (or rebuild snapshots) -4. Write the new index with updated configuration -5. Update the table manifest with the new index metadata - -Configuration changes are versioned with the table manifest, ensuring writers and readers see consistent configuration for each table version. - -### Garbage Collection +### Garbage Collector -Garbage collection removes obsolete data from the region directory. This is a file-only operation that does not update the region manifest. +The garbage collector removes obsolete data from the region directory. This is a file-only operation that does not update the region manifest. Eligible for deletion: @@ -613,14 +606,43 @@ Garbage collection must verify that no flushed MemTable still references a WAL f ### Reader Consistency -Reader consistency depends on two factors: access to in-memory MemTables and the source of region metadata. +Reader consistency depends on two factors: + +1. access to in-memory MemTables +2. the source of region metadata (either through MemWAL index or region manifests) Strong consistency requires access to in-memory MemTables for all regions involved in the query and reading region manifests directly. Otherwise, the query is eventually consistent due to missing unflushed data or stale MemWAL Index snapshots. -### Region Pruning +Reading a stale MemWAL Index does not impact correctness, only freshness: + +- **Merged MemTable**: If a flushed MemTable has been merged to the base table but not yet garbage collected, readers query both. Deduplication by primary key ensures correct results since both contain the same data. If the MemTable has been garbage collected, readers fail to open it, which is also safe because the data already exists in the base table. +- **Stale snapshot**: The snapshot may not reflect the latest region manifest state. Flushed MemTables added after the snapshot was built are not queried. The result is eventually consistent but correct for the snapshot's point in time. + +The `snapshot_ts_millis` field indicates when the index was built; readers can use this to estimate staleness and decide whether to refresh. +For stronger consistency, readers may load individual region manifests directly from `_mem_wal/{region_uuid}/manifest/`. + +### Query Planning + +#### MemTable Collection -Before executing queries, the planner evaluates filter predicates against region specs to determine which regions may contain matching data. +The query planner collects datasets from multiple sources and assembles them for unified query execution. +Datasets come from: + +1. base table (representing already-merged data) +2. flushed MemTables (persisted but not yet merged) +3. optionally in-memory MemTables (if accessible). + +Each dataset is tagged with a generation number: -1 for the base table, and positive integers for MemTable generations. +Within a region, the generation number determines data freshness, with higher numbers representing newer data. +Rows from different regions do not need deduplication since each primary key maps to exactly one region. + +The planner also collects bloom filters from each generation for staleness detection during search queries. + +#### Region Pruning + +Before executing queries, if region spec is available, +the planner evaluates filter predicates against region specs to determine which regions may contain matching data. This pruning step reduces the number of regions to scan. For each filter predicate: @@ -637,17 +659,6 @@ For example, with a region spec using `bucket(user_id, 10)` and a filter `user_i Region pruning applies to both scan queries and prefilters in search queries. -### Query Planning - -The query planner collects datasets from multiple sources and assembles them for unified query execution. -Datasets come from the base table (representing already-merged data), flushed MemTables (persisted but not yet merged), and optionally in-memory MemTables (if accessible). - -Each dataset is tagged with a generation number: -1 for the base table, and positive integers for MemTable generations. -Within a region, the generation number determines data freshness, with higher numbers representing newer data. -Rows from different regions do not need deduplication since each primary key maps to exactly one region. - -The planner also collects bloom filters from each generation for staleness detection during search queries. - ### Query Execution Query execution unions datasets within each region and deduplicates by primary key. @@ -670,7 +681,116 @@ Staleness filtering uses bloom filters similar to vector search. For point lookups, the planner can short-circuit by checking newest generations first and returning immediately when the key is found. -For detailed query plan examples, see [Appendix 3: Query Execution Examples](#appendix-3-query-execution-examples). +### Expected Query Plans + +This section provides query plan expectations using custom execution nodes optimized for MemWAL's data model. +See [Appendix 3: Execution Nodes](#appendix-3-execution-nodes) for detailed node descriptions. + +All query plans assume the following MemWAL setup: + +``` +base_table: shared across all regions (gen -1) + +region_A: + gen 1: flushed_gen_1 + gen 2: in_memory_memtable + +region_B: + gen 1: flushed_gen_1 + gen 2: flushed_gen_2 + gen 3: in_memory_memtable +``` + +#### Scan Queries + +For scan queries, the base table is scanned once and each region's MemTables are scanned separately. +Deduplication happens per primary key across all generations. + +``` +DeduplicateExec: partition_by=[pk], order_by=[_gen DESC, _rowaddr DESC] + UnionExec + # Base table (shared) + ScanExec: base_table[gen=-1], filter=[pushed_down] + # Region A MemTables + ScanExec: region_A[gen=2], filter=[pushed_down] + ScanExec: region_A[gen=1], filter=[pushed_down] + # Region B MemTables + ScanExec: region_B[gen=3], filter=[pushed_down] + ScanExec: region_B[gen=2], filter=[pushed_down] + ScanExec: region_B[gen=1], filter=[pushed_down] +``` + +Existing Lance index optimizations (scalar indexes, fragment pruning, etc.) continue to apply within each scan. + +#### Point Lookups + +Primary key-based point lookups first determine the target region using the region spec, then short-circuit by checking newest generations first within that region, falling back to the base table. + +Bloom filters optimize point lookups by skipping generations that definitely don't contain the key: + +1. Check the bloom filter for each MemTable generation (newest first) +2. If the bloom filter returns negative, skip that generation (key definitely not present) +3. If the bloom filter returns positive, try to take last matching row of that generation +4. If the key is found, return immediately without checking older generations + +``` +# After region pruning: only region_A needs to be checked +# Bloom filters checked before each scan to skip unnecessary I/O +CoalesceFirstExec: return_first_non_null + BloomFilterGuard: bf[region_A][gen=2] + TakeLast: region_A[gen=2], filter=[pk = target] + BloomFilterGuard: bf[region_A][gen=1] + TakeLast: region_A[gen=1], filter=[pk = target] + TakeLast: base_table[gen=-1], filter=[pk = target] +``` + +Existing Lance index optimizations (scalar indexes, fragment pruning, etc.) continue to apply within each lookup. + +#### Vector Search Queries + +Vector search uses bloom filters to detect stale results across all generations. + +``` +GlobalLimitExec: limit=k + SortExec: order_by=[_dist ASC] + FilterStaleExec: bloom_filters=[bf[region_A][gen=2], bf[region_A][gen=1], bf[region_B][gen=3], bf[region_B][gen=2], bf[region_B][gen=1]] + UnionExec + # Base table (shared) + KNNExec: base_table[gen=-1], k=k + # Region A MemTables + KNNExec: region_A[gen=2], k=k + KNNExec: region_A[gen=1], k=k + # Region B MemTables + KNNExec: region_B[gen=3], k=k + KNNExec: region_B[gen=2], k=k + KNNExec: region_B[gen=1], k=k +``` + +For each candidate from generation G, `FilterStaleExec` checks if the primary key exists in bloom filters of generations > G. +If found, the candidate is filtered out because a newer version exists that was not as relevant to the query. + +#### Full-Text Search Queries + +Full-text search aggregates corpus statistics across all generations for globally-comparable BM25 scores. + +``` +GlobalLimitExec: limit=k + SortExec: order_by=[_bm25 DESC] + FilterStaleExec: bloom_filters=[bf[region_A][gen=2], bf[region_A][gen=1], bf[region_B][gen=3], bf[region_B][gen=2], bf[region_B][gen=1]] + GlobalBM25Exec # Aggregates stats across all generations + UnionExec + # Base table (shared) + FTSExec: base_table[gen=-1], query="search terms" + # Region A MemTables + FTSExec: region_A[gen=2], query="search terms" + FTSExec: region_A[gen=1], query="search terms" + # Region B MemTables + FTSExec: region_B[gen=3], query="search terms" + FTSExec: region_B[gen=2], query="search terms" + FTSExec: region_B[gen=1], query="search terms" +``` + +`GlobalBM25Exec` collects document counts and term frequencies from all FTS indexes, computes global BM25 parameters, and passes them to each `FTSExec` for comparable scoring. ## Appendices @@ -792,118 +912,7 @@ The MemWAL Index is authoritative. Even though the region manifest was stale, Me 4. **No progress regression**: Because MemWAL Index is updated atomically with data, concurrent mergers cannot regress the merge progress. -### Appendix 3: Query Execution Examples - -This appendix provides query plan examples using custom execution nodes optimized for MemWAL's data model. -See [Appendix 4: Execution Nodes](#appendix-4-execution-nodes) for detailed node descriptions. - -All examples assume multiple regions with datasets organized as: - -``` -region_A: - gen -1: base_table (rows with region_A's pk range) - gen 1: flushed_gen_1 - gen 2: in_memory_memtable - -region_B: - gen -1: base_table (rows with region_B's pk range) - gen 1: flushed_gen_1 - gen 2: flushed_gen_2 - gen 3: in_memory_memtable -``` - -#### Scan Queries - -For scan queries, each region is processed independently since primary keys don't overlap between regions. - -``` -UnionExec # Union results from all regions - # Region A - DeduplicateExec: partition_by=[pk], order_by=[_gen DESC, _rowaddr DESC] - UnionExec - ScanExec: region_A[gen=2], filter=[pushed_down] - ScanExec: region_A[gen=1], filter=[pushed_down] - ScanExec: region_A[gen=-1], filter=[pushed_down] - # Region B - DeduplicateExec: partition_by=[pk], order_by=[_gen DESC, _rowaddr DESC] - UnionExec - ScanExec: region_B[gen=3], filter=[pushed_down] - ScanExec: region_B[gen=2], filter=[pushed_down] - ScanExec: region_B[gen=1], filter=[pushed_down] - ScanExec: region_B[gen=-1], filter=[pushed_down] -``` - -#### Vector Search Queries - -Vector search uses bloom filters to detect stale results across generations within each region. - -``` -GlobalLimitExec: limit=k - SortExec: order_by=[_dist ASC] - UnionExec # Union results from all regions - # Region A - FilterStaleExec: bloom_filters=[bf[gen=2], bf[gen=1]] - UnionExec - KNNExec: region_A[gen=2], k=k*overfetch - KNNExec: region_A[gen=1], k=k*overfetch - KNNExec: region_A[gen=-1], k=k*overfetch - # Region B - FilterStaleExec: bloom_filters=[bf[gen=3], bf[gen=2], bf[gen=1]] - UnionExec - KNNExec: region_B[gen=3], k=k*overfetch - KNNExec: region_B[gen=2], k=k*overfetch - KNNExec: region_B[gen=1], k=k*overfetch - KNNExec: region_B[gen=-1], k=k*overfetch -``` - -For each candidate from generation G, `FilterStaleExec` checks if the primary key exists in bloom filters of generations > G. -If found, the candidate is filtered out because a newer version exists that was not as relevant to the query. - -#### Full-Text Search Queries - -Full-text search aggregates corpus statistics across generations for globally-comparable BM25 scores. - -``` -GlobalLimitExec: limit=k - SortExec: order_by=[_bm25 DESC] - UnionExec # Union results from all regions - # Region A - FilterStaleExec: bloom_filters=[bf[gen=2], bf[gen=1]] - GlobalBM25Exec # Aggregates stats across region A generations - UnionExec - FTSExec: region_A[gen=2], query="search terms" - FTSExec: region_A[gen=1], query="search terms" - FTSExec: region_A[gen=-1], query="search terms" - # Region B - FilterStaleExec: bloom_filters=[bf[gen=3], bf[gen=2], bf[gen=1]] - GlobalBM25Exec # Aggregates stats across region B generations - UnionExec - FTSExec: region_B[gen=3], query="search terms" - FTSExec: region_B[gen=2], query="search terms" - FTSExec: region_B[gen=1], query="search terms" - FTSExec: region_B[gen=-1], query="search terms" -``` - -`GlobalBM25Exec` collects document counts and term frequencies from all FTS indexes within the region, computes global BM25 parameters, and passes them to each `FTSExec` for comparable scoring. - -#### Point Lookups - -Point lookups first determine the target region using the region spec, then short-circuit by checking newest generations first. - -``` -# After region pruning: only region_A needs to be checked -CoalesceFirstExec: return_first_non_null - TakeLastExec: - ScanExec: region_A[gen=2], filter=[pk = target] - TakeLastExec: - ScanExec: region_A[gen=1], filter=[pk = target] - TakeLastExec: - ScanExec: region_A[gen=-1], filter=[pk = target] -``` - -Point lookups terminate early once the key is found. - -### Appendix 4: Execution Nodes +### Appendix 3: Execution Nodes This appendix describes custom execution nodes for MemWAL query execution. @@ -912,9 +921,12 @@ This appendix describes custom execution nodes for MemWAL query execution. Deduplicates rows by primary key, keeping the row with highest `(_gen, _rowaddr)`. Since each dataset has a fixed `_gen` and rows are naturally ordered by `_rowaddr`, this can be implemented as a streaming operator without full materialization. -#### TakeLastExec +#### TakeLast -Takes the last row from an ordered input stream using O(1) memory. +Efficiently finds the last matching row for a filter predicate without full scan. +If the primary key has a btree index, directly queries the btree to get the result. +Otherwise, scans fragments in reverse order and within each fragment takes the last matching row. +Returns immediately upon finding a match, avoiding unnecessary I/O on earlier fragments. #### CoalesceFirstExec @@ -927,3 +939,11 @@ Filters out rows that have a newer version in a higher generation. For each candidate with primary key `pk` from generation G, checks bloom filters of generations > G. If the bloom filter indicates the key may exist in a newer generation, the candidate is filtered out. False positives from bloom filters may cause some valid results to be filtered, but this is acceptable for search workloads where approximate results are expected. + +#### BloomFilterGuard + +Guards a child execution node with a bloom filter check. +Given a primary key, checks the bloom filter before executing the child node. +If the bloom filter returns negative (key definitely not present), returns empty without executing the child. +If the bloom filter returns positive (key may be present), executes the child node normally. +Used in point lookups to skip unnecessary scans of generations that don't contain the target key. diff --git a/docs/src/format/table/mem_wal_regional.png b/docs/src/format/table/mem_wal_regional.png deleted file mode 100644 index cea8eebdd652b1ccfef2d1acdb8cf33c58e26ddf..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 551389 zcmb4LcRbbm8!wTxWMyT960&8_GPBCc2qD`cdmp(f*&*2@S(({;l$lvZ$5v)YIyer8 z!|(G|O80l~?|1J%_jRtEb3WsF-p~8}ex9#@8!B?Aj#C{!bm-721$k+;Lx)a=fS1(A zj)I@C=I(AEI&|icg0z%|+u<)`1T`>|F~M(h41DtMrDc%lXO~zCIL-$=<}rKB!$!Ga z@b<>dH2KNv;pZzYDk#-7h#UQVGf*NuUaFX|Lx*v2kDZ~Hy#M`0R@MED1OE&C+;`!`<0pUqwYvMq2%Olt z3|HuWd;iJU`}4#oy7g-P?nca+ z7oI-v(G6VQwY;&j!b}s%Pcd6rZ`a`S{2zFBYmA>b=3o%|>6=o|0}kBejVp(dAee4~ z3nZ(3^@d?Th$(1&9k4i#U#{Ku+n#~FFRaF1aE~1(Ts?B$)ZxqA?Lcua&DkB(`kr9) z_My+oc=(e~X z&FW{9jOxe=;GnzPiKXy-QeVR^?4lz+?hz|zXe)g5{Qg#;*f=)0pOM!%-gWe7cYMhF z+*8e*?IQsX<4Q77T3dFi654gEa`Ma%*uffAuSPFTiOlj&m+SNu1l||if3p}!m3bl+ zfxS$5mm|3?`4W*A4PSzP`ua9jvlqJKqXyn3?7Z;*Wbj>Zzc#C9-*TP2^ zZX;4Uf;uY^b$LmR)CENbq+*?KMPrTPWl-|;>CP@3V!`a6gZ)Jl3NIvxC;jQer3K zuh?=f7aMG8d~3WdKX%+AnT6hx$uFaiT16IJ)|?nR0jti9w+&x0kSHu!j7GgJkaw?k zJo|DPLC(=N<>VPs4Vzf+%KCBbU@gAK364EH|U)Z_L_V6E4c}BgJN2 z*ZGeo*5O}lY<)m6wiQ%|@1=YzzhHg=VZvqDe9KegJ<)F%>&r^uv~}Aee7&pZ)oDyc zn`8CZI{V7Dy?}Rh;w{dlAW`%fY=U7qQ)NqQe#%+fB=)HSBl2jTk0c+Qo*&*I@N8~L zV13?9!mg9S*{{OgA3j46f8fvJ)Dsz{a_}^o6PK z7q(r%7CyK9wXHUi8*(W;IDT#!RqmZqqJ&9T%);kBh%AkC@viH@R=YKOPGpH_&31ig zm1IrkQ>))V30$uUH?H4lDL}MZ)9Ig*+0R!lSxKp9l0!beLT!fQXK6fR=#E*S?l-=A zdaBrSd^G3)78ZSNZ70~zBFy2zWK5{h!?pFF!BXLs@4Uu&u&J5x zhBrdYM|dbp<2vYl-1VHk6nWL4c*33u5Iy`=q1rFr5&!jnEmA4b$%W7IRNp7Ns%q?N z)stUx9CF*)$!M!=x1n&f-tjgV3hh`&(yUVY>wRiUdNZ`Ye=^d?qrQ8KRgsz|m(G{6 zxoPHq7%zY-B;jr^WD<>cO#jSfMfRarhX*oNrRNJ<=mtC6Smv46LV2Z z6`SZJQ$dauk!h(lc0UF0(3^Zcs6l$?ZkZp5a2z}g~0 zcJjnHM&P9V`SxZD{`~`ZkAp1jkhVi!Q6DP9ARJ9SLO8#5R7JgIvExA5J$W?aoGQV; z<-GIWl}0X|l#Av2P5ioB0QQ%e-W4QAMR_L}Z4Oxqtafh2BDp?9q0qRAoz@h!i;f&4 z%Y`em>g!udtJ}xN5}YsYgER$O%3ZbN{q=H>m%AoCXN_bj0@K1V0zt@A-CQQ|BHzy& zLQz}X$;Any!G#I>QytdWtsq;&jt}B>=ktcO_@|_LnBggR;=|24W4O(zA9+r2br;6> zNDcoiQcn%V77r|RBUWy^CTFD1)WUEfEqvpX)W?>^l)?;^ z#JGi#wfVljT%BU8kpv;f-jp(o5C%oHO53F6IzP9VQjm$<&pjBf1_}L*tG*lKc04fT z5xf!Zmr+vXga#J%&I?WS@D^h|VP!Kvm2u`_r*&jgKqUG;x(naT4x3;&r5z#=C-nurD)vrLD|tNQ z5ukVtPmaZ0{CSm!ah7l=&27tu0uCQp_BDbN#&}kH)_VLNRb1OR450QkG#%#?_ZK4C z@^e!ujpU|Ij;C41!X;EkyWFN|tg&8xZsQ_$odIr~twKR?zE+CDD=k_AxrTEsXxI0v z&V4?E5pvg~`G|D{!nug+D;AH67r<(CjDrLeYNRHwgf8kC7&% zhxrXo){1Ar&aA@D6(f&|XJO&>s&TEu0mb;PBA^M+ZZtKQ&V<^p;V${(uyorp+Y$cp2K>?q zH#>?ZIF_V`-VmRAIhT7eS+sSkbOfy!z8;A+(b@iqe&EID%=?9ZvyDck8`H57Qfo9} zbGE`_avylFlH3$;o6D)9^kgu$bMBTB<3w~-__^)6^3HONnRiAV@^v`gB$f%E2?Zae6{MQ*!6~_w?k_q9r?vi+{nNp;p?e=t5I(+&GOLpj5&FP@_2VD zw}rGRK-qXlji70+cH@c3XAK9H*Z6N13M7^9G{a&ITK8}rgDiwW{Nkx$!^!=; z--q2jzrpwIrFUqBfXnnGPdde`W^$kb^NVR2L>rhxm<7j z9X3grKX%ICEf~5sBl1woc!8lh%XC2&q+KCdVFNt~0%ax{|yR=M_^Hurv6jV>44_2@UIT zI~X~TMjOKI^lxA~^BK|8-Y(wFh21Xct2D#bxM_EEU_Ad&nZ1%b%e~BlJ)JXlyB(fv znKZ$)LCPS#m)kdJ4u^xhc#Z?sJh@e{kyK$A8PF6?$!oS|UGZj0NxN5`DG*n-fT&48 z#NK}LDoXef{MXmF(?7Z5zY*t3QMR|p4H*rxvJH5al^$8Xq5yXjA+=4PI-8$S1^U#~ zxnl8&OU(QENI~ZdPc{YcpVyn9=gs*d7ln2U;TZFp5RDzyS~3?MYvR?(U;Vxlt@$L{ z!nLi{66E&`@qNoi?*J{H!oYVUNf4fYbJtz#y?m(9Y>dXUrf>oUDwa4dVsZw!o5tGJ z&#FJdk>dXS^Uz(V$oX!}e7QTEp&b7#V@Q{(3B6sW^K#-SvXOlxfiLzH2AjlVuq&Yw z%Y&k&$VN*}NZfLDz%sASaTkVRg)<`2!Tf&J1siZ%-HzqCjM7bUS03+)3FTa|ol7#` z)h$#QU!M>>@*P{b2=EpprUH`{<;JfP{ZYge8tmCC|Na4}PA<0}Gi+e?%As#JUdeZ9 z_Wf)nui#d>TPTmb40wH)_e=qL`yMbXkdz9fzmERcnV3t5=YHv`cZrhI?9!9+He)ES zDrZV!xfU_YyNh|{aNU%k+Kq>rcNd$i$;bG>8yYMQ%N_h2D1!dN()qLVLxJxre=YzP zLn7C5#qw@TFr}FOvqPa|vK|Y$>gjm62PHbR^FP1+ufL`J@|gg}!eg@2{A)Zt0k`e% zjz-8b7y*<%Y^6h%b$GN$?PsOmhZq~2S;fDG81NN8VIS=1qUnS1kuz@@)j`$>@3dvc zhnJ-Hqd4&g0BemLC;mJ^-%ETgnW^OgDYCb@%YJoeh8hv{cvzg4R{O_=T~T}gl^{Y z^(w+;1m-MkMZTp$shM^xceBa8wNOLx&}`|=pC|v$K=Q=F?Lg7~Q}h42@n6~-2!L?( znVe83-s-l3$D%>D7*b1-ci?o=MrPC*jMDT<7!Lgi(DTGxn*ZvvK@uD%Sc^D9 zl!x}@5oM|X$*NPSBQrWxG=|WzKb31x`kR{fYq7`9{3zY~CmHhvr}koKu>VLygsgE? zxb2b055gN3%97V{ub;m|eDIBb9dUmVm?`+aZ4YCvSZlS4nWTpstZ)d6zpi<7M3HLV zB`RE=B9$p z)IYhskJK8{{<0h$U}@mH_YUM_sU&9rqu)t4dIiQYl|fdw=h)*tHGJTF z|8$N}bnlz%k*f5>atx}ek!@iAgTj_0it%;KAx(Ep0-oAl$$hbZessSs z=(Aq~(6S?;y**9g_J)>ek%vZ(TX6Z}s3`w-Q2(?7A5(fDnafA4H(nVA2vp4i$Aa2| zVz0aebN|4XD?c3pC>g;C44<%&`iI{FK(&r3#L02ovMk zG+B(ew}sLQz;$^ZF9bq8t6GUzegGC&_g1F8*ICrI`4>RL^u7 zEwgBRqrfP`oFeffXi{`>QV;e)pxyr}oIlUtV0}HQm7+f>ZwaFDk_rW z0-%-q%cOuyeT8e4^xKNK4jsPsk2w?9d7Q*tel;iKsb_pVplGE!4kZ7=I^V-5DE>`z zHv2r6=lYj7zNFWw=38-(%iTG&AHlwt56}Fz%KyioC1vom^kAiN*TsUW!yiwAKCVQ;tfE^(+0O`-C&Te)nB4B-=H`lkcz#3?^izT z!Nz`>Ke*^WhszCP->V9YAjLTBEiz>9c;O|4?4R8ilfVo6UciqS`18BL!vEq+p30=q z2~j@;z2%1#VHebe6BYfJ`2O`KzaL6ACszqks9gTTwD9NNo-u+y()9a!g3sRjwJ!nLrKfy6&v{+< z)aYKuJtyYU*^iTbB>A5d{7wb#(Ih_OD-oYOuEntLz2)-_N#1|;8Q|~(Pn;CBOq+zi z0mmyTgZtp0qV&(42A}aB=ZGqm4y<%QHu*i#VH`j5dq3>`pTBev?e+`Oe+TJ}8+Se` zFyhT+dR`~M`R$(mPf7m~_Is7zw@BuaWtz>cCa+%<0eof3ah$zTiGv^e-z)eod$>M9 z1)ZJG9|U${LQN$9Lxy*;`#}xu=?M8foPU1MO5#w4M20Cg{mi4NGiPuQuH^r57W#Ku z(9vj7S$dx01Dj#O2@_k_#tikqpTWQR(e)ebVwZiQ@0w*u-Io+OjQaUK-(UP+k&Y#& zaO;)d&9s56oi)RcTNAr*UFQMKzUm3=*R|J#@K@_qEmYQN@wQbjcbR#OJ=WgK^$E=- zM6QpyPON<>ue(w^d!;}jGa2ro-@^87TdR$%@yW2ihx66FFP z>|$Sz0V;d#A*a{#`16m-mOxC5^^Ka8M)UTF^3pi>-(=BxG-O-7)VpYat}1CfEs|#E zvo$hLx4R*yI<@c}e;;Wyl>DRP`tHVafJujzhwfe~D>;T()}_k1r1=+gSc|~=ce_m_ zs=+-0i4P?Y`)a#SZm;&J7N$xw9XaRurEtD%#FmZA*j2xv4zo~3lZ39BD~0u0`hnZK zl2^ZjnSVf{BJVopjF2zqgd>CIF^dIsLir0SOr2}IWgQ&Vxh_3=rcs)`&-83#RV1)$ zYti-F>B*T~XThPiNHwq=JlMZ3Y`;7SNg+I@kf*CvPGUJDcJ;OE4QqJhw{L9aY%Tlj zZuQN^+a*mNM&;vMf4;X=ipT(o$I`rTNL76E?zIqooMt>)5bq_;XZ6wmFo1epLe9ZAKE=*?esxTa*SQrHxV2pG zbLaVOu;2L{psKpnwk^s#xjvA5yvr6d@HRIJKXy4 z&6(I$L$g>@p2ci6~V--q+3j2%y5&WP6 z-}_m6f273kHHu3I+>gd|yP?C|3$fu%tf#@XOzM=!_^5!h+s(?Bzb>@(rMT}A zwQbB-u-%)^r;jOS7M4Vr2>VWX}S-t1eUhR=Ct2t;=_rb-9MH~{>TD#!pF^jV1u^BgOHqjJjTYOw1g zBW3fYuSrxz!@EUSJIltRwT%PHV8XCh(-Deh{QB-QQM`-U8uq(bwClO<$so=oV^oye zQiZxoBM<4Qjekr!F6ar9`U9)Dt6w>e9Y ztmlnWzk@1Rtd1?<^x5p-T@2J}vcj+58e1Mdy_!m1n+1zkp_VSXsx^YB{pzfv8O0J7 zb>Rbm7;dDrTHD2Vn+S32ylSR)$PCEp>)~8ZQy~Hw$=lP+ECy3aYaNC=JY8{D*t{o6 zlI0f*>UU!7h1n|>^Q(K$`=5(&88VGt^VvoFB)atF%lXRDx-->+%C8z;(kI&FTDO|$ z1oB0IT;zxP&gP&O{|AQzXwaMIL@{<2W>%@OIQ8HIc^9eJCSQ+gd#4s&w%#T5*|?@P zAioK9p^EG>MR5ovw(kwwx6-JKx=je?Xxp2Oz{-^qbHBYGw7apDM@w%fdW(q|%tw_L zW)*B@COBM|n-T-jvz(>R9P+hp>uaZ3AmT*jc87 z47ZJk;af|6)`nZdC46>%O^@TdH-O3(s+d#B*?pcRk1?F^lQQ`GLlfGf=0{xuj z^<;q0mtn77A7zQqsaneMp2IQRY`@j(jL?rB`UEAGFnZJx)Nu&SBscLo|8g|=cc}X= zOnetW*1T*eUAkNP;27^t6K{-+P2oWt zRTc^6eCtjSMGM4VZ~Y*uPfUOda=oDs&wDheDVD9ErsxF`Imq$QVdvgU*6GpmPbjCe zSra>9PhiF~@k?bR9L-w=o1#dA5Hjhw4iO>vdU0|h|Cm!BpW|%jT5&Qqw+!d4nj(8t z)h6fHAZoiBd*QhR$Ex`Z7&%Vjs3Q>SNnZgK;E)}7ddT~)WJ?A6ilO65B? z{*qzA$KTJ^d|+wjW^^je2ahqh}AB8FDve15cR@V1wcd{*x8w^-1`lGE?~b|&kceg zMk(dKNF`*)Yf4RSLxvZj_{NQ4s)5UXuQs{@gYGVj;8W7L1XOiVZ420z*@f9`Toan#S<+U_h7bzsK-V7jW z#p#_CetI(Yi+SnY2N?^`{E_3^VCCaQDn@oVz_8G4rLhC07z);gJT_ZFt6aRSsNUvkkYCg738YoJ3RuO&1U%&V*79c%hbL6M{KZbwx% zDM!h>tFUXEBNICx%9IOE8LXv}rvs9DS>Nr`L(68eTQ(w{=HjcdYv8-8VIrr5Nwj^i zDD_<0ex1dq`G7U5#9RGrq)Fakz)jfeARM@_|Mom2(*e6KsYz&57X&3JpV@Oj^Oos) zfd5Dj5LKO+*+4+_Ybx8geU*IOd=l@yA&5mOUM`d#Hj9VxECy+8Ptonl16JgQxF)*n z-m_Sx>gcb?S<45>rLN}EsSy6*taaTPpR>2`y)OF%)lS$5DeTquN=N2099?aR8o1R% z!x|Bn4`}cxKWjC~pyzUJA0beZB|tWmEp31e7sIH33I{9+7xC{A`F9SxQ4$bQqKW6T zssTHwPMU!!1Fqo=;Xt@hwc0JhOZ}yOcjsj!u3zV22Uc;^ zOVyeDdl3Vrbin$4qwrzF&U0j>D4SKCHs@u@i3~cnsuB^oe$H17IHj6$#u9ebQ=uX& z*#jtgKqA@pbcN+?XRPH+)UEDwMyp){J-O=NMcMFii-TwfZRc+hL(&ig-CC66HhwjL z;wo`V(GohnoKR%+N4+TXC^2LciH>GFK`@aZs$aeJ>A;C3kO-Nl1V&%M2N4JW|CjXd zy~Yy@5!&5eMfVqs&ws66Rw?Y2$5tSnkgE)k(S|HZIK6FtlV*@}%k}VLGH&HUD^D8C;oqEL3 z)2ETW(RF;~2B!AD1GfytmVl_Y1hWG3AfuHb{m*X1q~*sVUPW}B^^WduoBS)Pd+>G} zo)7?ZEIRQ)a7_C(y>;`G7MHwr&id^Y?q-P{WLV!dLNUIKn#*{|Z*V5s+g`|K7Fq@g zaLIjphyn>r?3~0dO9vmb7p)?DY5#+s=L5Ktaiy#U(Xiq!e$-f99j(itQ6&KmAY^wv zn%NA(-MdqB=BF=+)J)MVw~5*fu%0z=`!xFGtj^u=aE$y~fTp<%sGe)@!{B`0Urgu} z!40^>E(8KCZOFLfF;oc%bnn(Ux~@zgVljKF+iJYNis;?<#=tM0LZNcF_|LtTgSM7$ z0BPISi*0Niq9IbYh4C@m=;Vl?1R$WinKc`eL79U^RJQ@EWT7>-&9y_{@G%*|z~AF9 zEKW_=?f|~5h>ZShlKuw@VOYP8_&jxf1u7Yqphv+%%i%MSV-WR0-*t>MhnTN5O2CC| zQP`axUNaTSbiYqkLTt4JbRb-G_XL_}UWLuqEzxlj9qbgg~|W%zWztVovQ~J7jinAT9-lQB-z$u*SUG)gdEKR(d~c=O3dzD$*@OwSa4HL)GY&d8yUh2Uk$}lPNgW0`T?w)NC&!BxVd#2h$>yhi_~E~ zM%dHl6mz3?N2U%8Y}}BPy!CtL{Fj`NjFc?Qd>(()QLqjNe39ua0Ycj(q~yIUwdse} zK!*D58BV!{-IeJ2v>K!=75~!7l}`fzqXlVz>Yi7&^H3L_jWHR{|AUwiw>2A=+UAwi z(({iQFID0iyZA>gqeKRC@~c>z18KItR@((mhq=9~k9TYeJcS&1Qabp(d|e;JvAOhSJ-_t78l``IWPAon?Ud zssb3P?aj`~j|&C9niH@%2Fm1b1fwmPaGSCffZRe`p+Mrx?YUZ@sQClm(PQ!Ep*qYm~n{>M{cBgCJ>gRhkJ_yAoB9TPraR zKJvp?Sm7Utjjp0q-sV;rFLAF$y@@#E?gIBN>s}r6#DUF2x<6ex4K2w#|39e%0nK4y z72Mci@A)(2G zdcx879CH`m*_MW>^r+jBE3kuF=?c5x>!Gu1%|z1FIDOMVRr@h^Hm~hEBe|iM{pvl& zRzS#+Z98`d13a~~5}=fk%l=xmm>-UyYO}ijY`< zv{xnodtSS&o6h*>?n$zGz#h%)kOgQ;VBUvWp@-z321IGU0c9SoR2ZS2+fGEM_~QO` z__rRP9bOu*Z?9P*SQ#L*=;c!I6qYENaBBSQTY01b1~OedcP&ILf{q$dE|1iNJ)E@_ z7`=BbaFLjR6S?`S4o7v=hcRGzV=Bz<>*SBo2MXK6#XslnewaN6vE@gt0lGUq=ti&n zJcoigNnq1sw++4=)OEJJr%>xEcR?4@jk=y8_&Ta$sQzJi`L5_n8{?RZ<%sx>u$O2F zHQ`DbETQjXB+$S&@&t+5NIHT3ZNB!#UfJQQwH_IvQaP+&$}gtR7oS!zW5zUE|5@F) zhn&GIcADP+EDMb}bg<`TTbUxGcY~PB`j1*-=CWRdUWa6b`h1^163GoZdf|W6tV&E# z_E1MK%YYHw=CXJ=#6jX@^?aDFJ48tVT^?sSX$Twn5z6Xqkng!bp$#(TSg z)ecXO*5(@kk(h4WACKc5SrKqAi)Sn!&6 zUv_Gd#|#0{q@F9i(##3x-YX$|22SGhX^~brQ%y0DiM%IslYyGZY9DieIyjS)HapEf?BMul zHEx{vf{dPp%3jiLDhA>hSG*82$al-YWs09HXmlMDdT}lHA~ztU!NWV`t&M}g-7r+M zDV&(6+yNCk!xeZOmH>+tkx%Y1kv182ETfEOTB4|h6R+jCNNj&1RR3^o+Yq~OOPyKa z8frRCYf5ei*d4^2-XoKT7f4hD&pXz<3aB5fe|PmnNXc70%QQq<_@0*x+z}cfxv6dt zFavIaEbFV{v=DVA^3!H9pT3jea@Nob32_nx?esG0X`{n#;J`J=Q6bH-y6bHtKqc*9 z%=GLWD8V7k>BC{3+fd~!=@H+$J6_u5M2@;$*`NvF^oHPt(q z#Z2Ismdw1ePkd8rYN~FO8@dY6BJX8CAV^1uxGw#3Hg)kiAXDdSKhGt)8q9VgZaa2q z_#E3xT=p_ItIm3iuO24ywPLVq^aH@X&ME-$oW=Pgft=p+iu0degncd3v)ytN6d9#z_2IxSH+dk$- z0RUKSeTq_hf))=#3dfE>(P}a55QJWcP~;dY%8!0CVq49=REr+3)Hk^jFzYybA%yQ@ z^z`Xbjm@!Tc=^le`9{~;MSWE|rHId)4U@2r=H^+SFmiRA3)@&g>1(I>c5TO{?K7BddH{#rd;V#Rqu7S^aAfCYZh7$h!DlAh$5Rv z9Xl7@x~?FmHN##8;XgNV4vy&oPIlCami*&x;MLg%;s?G^J%=a<2A=p5aE2?fQ$`bR z7GgVbKDCBZi_m3}<`nQfb1vE~imuI#G^>P7aw1v{8+)wRASf>o#zHPLkn!C%BnsTI zQTkfj;HFjRHX869Jey00jf2=*N@DHmzUUd5Ty2aArNl!f#R3zhV;GPiq1s8Z0t=m6 zGjCM1(gQXipQ*p=$m;vdX!Zzo4&YUgvSBV4wK3G5LKlQ9T=~+}`cZFu<(boL ztU3QvmwHbiox&EJHa__yv@R30YM-A6@O(Fr?3P;m?&Q=IaJ=Qs6YOO%-L7?!w1D97 zi?wLU6#+*V7Co7;YK6Z;Oz@bShAzL?lceLOJ{XI&yGzJ-T*G1USUK_SnS zlUsCM81Nc(CgGa((gJp`(4w*WxFb~{J=JnOo}%D3w#lxeF}F$3d9zzz0n2$-2gum; zsc_@wSoyDk(?P0UlOzy%FJ*eJeTAvqVlyu>&`Hyr($=53f8?}PqyT@A z{|nI5K)0lZV}a{ub63KenKsQ<6Ae{meW3R8vRj@e`s{3)w67e02KH04Q082y+aLg> zZ`o2&zMNAqdSN~@IZ5qY$>8McMaTX0F%1b^sYsXL}t z6_{$9te8V)bhg~tPF%UDp6L8Rr(tgNj=lt01n+WX+ z9K_XxOHIvyw88cydHIr|-M2QzuzpuLgDKA8)@%kwJV)2bjfNwH3DwcNKBC&=qYOI? z+hD3RE{Hd5HG)&u*`wPrDe_IerhaF&rn|`yXdK0?P zWtJQZ^MbS+Nzi2A1fW5Jwa5^=PFSSo5UjTidc+Sjh{hc%{<>u5DKy|qH#_)A^Mto(F@-Y`Zb%m|Qn!9Z+ zI~KIlUzp0Md;%UMzuHKR$6NX4(L|NqVa#a8i=BEvY1GG$d-^`p76zgk9PquNlo+GKQ907n9tpjo6^o{LOK-NslUfybIzqIr< zzA#Ruw)VR{@s8=^KW$t|szQGa!Z+>_9q1gssRbi6ii*|N^8r|aMI!sTSSXnAVX0<@;rPdM{q8&;f+jwF} zN^^F1!C{4zgyDv|5Qs8{P`gYPOfQ30YOEv7LfA}TCvZFFO~N1KH6QWVtxfR4Yw$Ow zjOI5ww%J7lNM+S}c#}6fA4O!ks)Fh|N-=@1LyL`7G#KR4N6ZoDoIK|~9d6nFl%>)< z0DVKt&lWu0v&lqTV|PW%Ee{x#dD$0bh9D6`XE;d!++5L9lRA9cw^^ ze)8y|iA%N4(93ZlV&;V{_kzMeHLa_+P3q<20-EYJDy%OrJ4qOC^k<4k4PSWJH0SHW zmb1N@yxS=XV+#3FiZEayzQIG?mG{`X1cX|e_G^}9M|Y;(PS{r;Fkt!cKK;L#aS?}R zTv{tR%>(d}t49Ny0-!diX{1}lF3`7DI*9CVfViWxCf)A4Xh=0mjbGb3)g`;s9m~rM zA^D5;IKs(-QA1E4SU6vQ0xCdZkC_hO#cd(NCe|_p23$|n@?RvBX1L1f*StV2Fy^Gr zsjps*9~qAy;-3@WvCs;QJS)^Fiu^Uk7MF??%@F!^_-R5qSm; z%|GZ&Sjvbt(XzdtF5LyA7;!7-7QA=CnfJT4)+D*l#&X{aeJ%I6Z&ZA!6fcO)jacDblS+^PJ6F`!4ga)`d zU6~Zl+mJr#w_lYqP}60SGXMizUHY;#a&Ovnt@8k7*cxJ%*nB$-&+&=7)nqjwtD}IY z6m*c+051C9t(*hnt4}pjHb|l_oXWN8!vLgyI-Ixj3Jki;JgV&$UQBblCszKxrhV2Q zvl<}_*=uV+7j-TU4JnG^j^vw)=F;WhH63{a9>-Q>PPlh9un3y0DouNEihqzE(hw}m zoc)k`w*)>F%r!f7fevJdZgs`!!Ud_Nlpvo}!);9rM^&&D2I_YA+gJO^oYi3}R?zBg z10WnDtc(*_|72nTH?vg7^e#f*HSa=b$-;=GjwnSfkjm5igp1miOy%!K+ado63lv~- zH3GA{=l7;ZX=Ptv_MT>>Q;^trP1rjXCLx|-?{%_Cs8yqr?em(6oxy5*Enj|r-?>p} zR&6%*S!<8?S4V5Qt;CD4=0FN+Sjo*mPmPYlo|i8#Vi!SUby0Ahjaas9)S)8@chcly zfywxFdZ=-3hNv)zjXFiv6~*r`>0F{3nt_4V@l+B%WhE8=u5wH8Vb|7$#R8RzK1=Om$fJ>X?YYJ!H`$+uC# z_;RZmRkLx{8PT<0Ox=~OoRPO?Ow#>_rCKB)`fUaOj85nDwv=6bI{6&oLP#Xw5(-#6 zZN8li^%nQe-Uvn$kz5+o_bT;IoqCR}8IVaev_al+D)*6*z?2T}Y|lw#LIWf$wDPu^ z$+uiuX)KEm6|Uq6e3MLyEkNlL z^>{G5klDnt_#CYrH3Q9&$hpScm2+=#cg~A}%t%tUfXn5h4td#fQO$#(^DQF`T3B*X zuJ&BnmLL})Pi+Ykvj&aqDos}F41WG z=3?|l&O+A}PWQKpaY3>)5J*uuC%E1lcb}WlMCaK;Oq?yh!g!)32tOT+TP=suw%dx% z1FjmyaFc=dvh8KNQK;9oH-Qx@j`>Ww72=G-%F91SU&U68S&v_5L?S*@Z|#D6fW`Os z4vMdn16$FJvJ@KUK`;|@B<*g`g)Q`Ty^d5Bk3B8=t)X)|7^59dvGP_G&I7f(O8skH z2aWtm*;HzvY1%?eC$MtkAm*`W*jGEDFkte-MrrE=N5vf_o!UFVHCA5jk?_IZ>#c3s z%F%xp9{fIE*~FWPdC$;Z_L>u+4&p^K-kgN&Z!_;qGQe;EYte)?9xKr1&V)4vAKe>L z!F?IL&|h<1)3LhdZkY3npGEW821 zYLqrU-DL-N^mgsE=A5JEeS9G2u5r@;eCpwsP5k4JNd3_CzAC_`aU$kGi|u;>tXF)O zjAtIzJn++gZVX0)7Tq2#yS0k+B3tPuim>JqTSLzXu`NfI-E;12gGP&ukU`EH>-@?L zO?$MdkeE2IATt>VWy`5B!Mxq_2tyz5oa{2w@NeM*m$aOPmTOk6a;lrMFNOC%8CVAp z)R9f4zLPK~Y7UMvW%K!@n*?}z4~0=NAP3&#%`OJ+kC5igDw_36bRDm(%*vU9{9c(< z$!Zl~@G4~{y66TnJ+truyg%KIUTk?PYb+)N1@)_`2>V`yiN_gVNx1p&6uzBDUp!Ul z#fh(=ojZJVY&*$j#K_hUxMZGRP3X1;M_;V((g6#lbRclDcDT-Y@vLdBkIdBp;F=G6 z^sZiK4B&_$0`s7C?ia@DrT@4y-uhl|0`p;1YHGA_MI}e})@^&4oJnp|_cTdHScQa`8LamzVey;tb zZh3b6=Pd+~iVhMx#@&!A_n75PrmPl&6}HQd*D`E_xiv%?&6X*rr%neyax*^?HwX-j z)|>81ot@2NBApBIbgwBJUeP*lXu%h3xtH~9S#vI}Hf z?7t!5e^1E#sAO+G5rG-{Noz8~&$27ms##>;$W`;_EdcJnv<7KGEU@rZp#C(*XLotG z)QVrN49t3jei2x1VkL7YID%RSCW-5;Ul$lC@l?w#vWEtjpfOFzXesr4S0ip)G6M8u zUo*L30yJ~(fvl+E0L5bd8Xe{aVlL3XvmTXam+l7pl7Rk(gvN~q`CNCwhZ&e*K|?>$ z*z^y#FbL*imxS}&CIgt&ZkN}r13jT<;A}evD<69XwZupH{c?lRGl;tN(qSuxaKT=$ zd^baZ=qt4Ts#D0jqjvCQ0cvRc#e!pD!ELD$UF`Nsg0{*I!L4;Na7$W%jj1`tKC$?# zBZ>E}M+9bN_Qy-2emDjBrGF_2ML2IkL5k+Ofti#FAR!RHLf&A6NCK)0B8Y>}A@r%GV27 zWrN8S*uae{cDUG|nmqh97e5B#nBj2bD|jw3m)g4$k^p6B<~9Dhh)6KmrLqb5RyP%4 zil(5n7c}4($76cEgSSL+HUKyA84dGIBZE{)E~jFuYbAE2&DYkTE^{yFf{tW+Xy`Ec zG;Q%{4O41{JKmVxx|M4di7M13v;?EVs?fw5_SF=k!cdxc0$3U|qvDV#a9CW7f~N8p zt4$0FfStl^+Jy&=%YkYhO1-Se_z=&9VO{$IiybO-qvok|AgHzyrCTs9Z<9^XCfr;D z1;48BlB)#U`{Kx+=E@^EOi#}Io3?^)CSDT!`tCfB=Y=zZ!227?A3f7|ZrOkS7HD${ z!a9}!z}V=FYsa7wgHWx^oUJFO^HCPlIx`4*?`onqJVE0z4RnUNAQO>tExJYz z$$02y?L^V^>m~+lc_nIyYptWjv>VA7}ueCH3YO zG#aADbo}9RR-ik6_&bsShQf9wrj1TE+sz`NK1(j!-P!H4$AieWCL3xYue|B`Q&i1M~_& z*HYy=-Oe}v5to1m_vOVG*G79h$5(k;695!~!e*u597~d*M}&liq)W*%W@*c?yPyYU z7s!+VxF&c?ryWK|rwY|Vhmn4^sXDa@JNVM{Y8&Wwj?UULwlXKV8P8g2F@^|_1~jDt z+I3|nZOMm6FJY3Z0g7+TRQ3-ELpEth3o`ZxCo=*V#} z;L37eAS=HRrx|)8`ZS%7CLEAeyOI3lQ&5*nz_9|-q@{W{KkYHv-91eNhCaJ;IrU^j zFYmPw7A>0>jv`Y$v|51{8O{^!6Eqw58+y+{OCN^N8p#AhHfta}>$J#ns`_1Yw^yOZ zq?(V+uxHB#G8E2}6}UCG*=LF%;b4e`T^QaVW;Y7;ZknwQLz8`h{u7J}^buhyP+u*! z$Lk`E&o(l4tnsD3S{pRLIV8dA0!-~SfKg(l*5t69Dp2t_+fU{7`7b)+$KHZw&oZHT z5gDAl&vN}@F>lb`N56uve+%x{4C66l!GDg&^8y#tYFF{iptjP~eSA`C*=e(z$*l6i|K4~-tT&)Ym z;;bt>SH1d^?OQ03JfwQJGasJS&C02i#c6r&Pa8MFOzRfrAEkFCu%4cdu(CYcnjZNs{t1rU9jP=V%|AF|{P%=UW?m6grD`@Hi z(@3}TWN%LIEAZ}Mdk@b6UugU07}ThKYK{W!JGjQyEcp9x_<;1as(Ecg4i}47dnF)9 zYU7PasOj1V-Z3}KiXW}-q7xW(8T!LSYfuA7ruRKHc%p@rK(^2uJR7&9=%DO)8!KvC z4j#Af(WGM|j^n6Z>m?;HKo#X8+C)vFycmy zzi|+H?mwJe2RPR2`_Fq++F2zPh0J7UBs7fd$QB}Hgk)s%HW3MtO_`b5k#))_WRD^g z*;IC3;eS7`-T9qP{m*rs>w4q$9nW(=_r5Vq5f7U;A3EC~sPSjh2x(1rIM0)rR~;YvKL6bYd~9b`jS*TB-qs1P;-KfTy`d4l z_U}-uv=*-JQO~yev?IwX74A8=!3Mqh;5@qmEdnTxgwjiRUrUMTpPfv3BLLPt^g<9C zVVsXS$xj^zcjS5XoL`EA{+Cx1sVx%iQ8ChiKFaEb(UtV59AvzMO+Goo~dJY-`G zgi|FXUHx_loEd*3N$Hjjs786)bv2BhM%VQUP33Sy6zcuyg{MVS*N0G>Z@F!oNiBjs zal*=q??KhII_)c-F@WyTeyB}Ljkwqd5;}te+o0^>TI``Ztpg3sg9Z~fVN8p?cyvT~U7<3ZZ02!6ND% zk*J|kmRP(&To-|2)S=o$@{2FIXNA1w)1Uwlqnh*)f=?6k9yd0lM~v)od(iYv|BWq2 zB!-ngxf-W~Ll^yg^v|Lkrg4yP!d{uM$ zSj}vuzD#%O(Fpm=LiY`5u-T{jt!Rv}q@2Dt+}TM0PP^iVL8MDs%2X=0Z@WD*Lg^;9 zO_cRIEpfDdN#4st{;D(}qNZc_bM4(`tmxik4)DNyyK+hSa*H#LMrY?Qk1T$-yBOH1 zs(*9~2l6|m`VUHz@&$k2GB{Fm@@OC>}3LG4XxFq&E)Am&L*Qgwp0hx?go2&K^2TSZVeSQJ& z9(zjK7zQbP`2=3<12t;5!-0MlELz~ySIuT;&v!kZP~Gzhnrr0D!(^4|UklU+UJeWF z%qQ*A&kTf(5SDJp=?b$ON-c}AcH7|nzzS;Vy|U>;Qz@#BosjgFHr+a#>`@%MewS)Q z*M%{XH>l)wox2LO$A_%*AO70G|M@{n+3Cbe(r{kd_ngWX-T2EkbB|EIIqB`v^rb*i z_!D$O1$2%zJYSKP1sNyr@l*A0)Lz6ed7yA(>8LHK*?(z! z4qyRb_hjKxfzpe zI^e0FG^sb?VpBB3Uu`R!Fc4PWd11uH`Jxp%eSuK>4F~t#T}j4hBk2#hViFeL(+Y11 z<@6i+$4J(T@$9QPLC53JFQqN%&G_KD`?rb&FRyNilW&ZhF-taQ^Zo zZXQjrWI%D#i#dV1_~u)00|ntPpLt3v?5H&zCmKVydLxr=)o_Gz7;IoBxe)Y1WI-ou zlxx1y`GmB$yd8O2nJoQt{Uopjo)G1x@oK2r>J#_mtX9FrQB!0-$h z^5fIBio}f^&{hG1kbK9BFf8$+yR5HRC)G5h{SNY~r6iomG~5?%Q8<%(T&EI>UqK?7&O%Fwj1b=&j05IzYWU-{btv9g3*}Uea zA~aFjt?w;CoI;w(3QZ;br~|C0zgl20vEj>w35j;s&yOlk^e91wX+6{2lOLb%Qip+4 zEqw9*-rjo~=TR4x`e-S%oDYyCC_4-rYfFt3&F6g*%fEPBe(xEMNXn!8PLq^_HFzgK zI{pP>dq0||pZa|scAn@$4R$4`>O8wchtTW|tIB{uHpxlHA;=@;76i{wT3mVb?Hej4 zhk`q_|9HfHmgJ1RDds1=gm4A}QM1R|NA6pqu{xW1p2@Feb7(|6>lM^^oZpzFBOM9- zGIW0Zg{XHtI&m*mJx05Snv{l?Z}z+0@)- zmc*Kg2WsJ`kb__?*aW30H4UEl)+oRGa_rd`^0ZNxN)+-%pEi#bGb?fd($TS&`uVci z#O9GOszZ%m?uzfXI;U6r+DQxZSt8QB!~EobTbotceVuoZ+M=cE&hIaKm6s%t*Uf5n2BrVhl^V-wVowS zUUh|avMMHh9ec*a-Mv03vp#Xi1IXwf_?dk#Kv`%ox_w7hS;CaU+?WRbl7@4E*=}KmQz~G$B2>%68=@ao`TD+}S(p&Nm3TcbpxRB^m zS1-aUP%^{sLl>OTX`ycssJC`=YcgH7|Edd2-WGX``Nbm#hEUU^_uU}c3!m&eX0%$M zk5H1ME3c7H_eLs>_=}ArT`9%lH%=sc4xpelegSBU#?I%3aOn6+BXzD^A6C8_iXN7L zByoRWuZ730hi7lIZr3IEvFWpgBuQM704GFM*#?cawa~;cN#-mnR2AMcyE|cDWvj^y zR2=VKoU{6&>i7}0I1JFI9#AH5ZPbSPOE`xcG4T|duNrq($$K)KN4fV0xMC)P`_i3Z zh$ad&T3YPJ`m(M5XLs&6fBBpz8MtF@v@^EHQYC9p?u=%HMS8Sldz$IYa5lAt=V%o# zl@;}DpSbavEuOvZ7riAi0y(4KotpO4vra=J2!f(%CldP<%?itz6QY`-nlSO|Lo;l# zx{QEw2vILG7X8rbEr;e9g19crnoPu?$fF;MIx@WNHhMVlzO`Ax0rnCUA~>g8+CGYzs87( zz!HP`)pQFgEpTsJY5Os#B`4{ges0FP5p`z@W5tXq>cI9<$cc0R3qE+5z3j+ldCFf00A=7=MPuIaQ zn`eTPv?n?dNJ(g7U5!Ip;zbXebpj;OLQqGIVN%=>EdLCqVY&;#^{OuHP6@q&T0`qM z!t8rXJeT<}4Sz3k7~NGnyDmY$Y2u-=L>QN1B9TJKNl8ijC>B@U)3PNRan7L%`z)Z@paVPCWx zbW#5l*TJzZ*%IaM7u~cWv&N*nA&4`2Y)W12D`am=-I-9rM}e-Wc;*<(iB42N*>?@* z7T+css}OIx3dODqI1uVknp)Z)_wsh5D6Rc_DQ}Z<1P)Sf>Qan9pw6XW?^Plxnkpcj zh#_wl%u@Y(0Kr6nLasJ)iCw3(7bF@u|N1(yR^{Ln12DgvLke#7Q zH7bXuA72ETK(>h^HAr4CLkQ9=E#=!8LM;_o-!u0h6f&|mw9u}m#`^f8Cr^`v_qkCR zl#lh#AU(sF%6yW}U-adv#3BpWfM}bmf>Z@wpToAt@E(-uoP}tjkk3#5{z$tgp^p}O zx!wH0A^$`n1R+ro&kp0~{(k#+m6j zcL}q+x9jd!8Ng$jczb+{ zSILhz)7u*tAd(n&minO2(N4-?uzE}+TL!wjMsi_dxq*bYGyk1Kf$FA%Jx!WgFyUHn5sD78!FU5XStAh1$Q94Ah=Sv#4qY~i&NshXGQ8>U(6eDyl?9(bMy>45cS}a zav6O_qn1e!+C-KM9JhI2q2=GnHZ>0dBoP~CXM{=!b*Q_x2|P{In;ZSTV$S;ka#x~w znl-sh;2>mfoJHLXv3QN)b6Pjk-GG{f-O^5Lm|jc=3UYENvLDWst=t+WGzfD9!IO~Q zAY+MefRfo6?}ky!q~n<3_3@S{l|SdV=PF&VzHQ73cY%acB#MxK(ME@eUV%ah;{iTpWqw5ZEEK_8EiTJov}!ev18r6+_J zS+%B%NBk4wi3@2qd2(qu51n;(H$o>E4a)MER7$Z^s<^hoW8+7>lW4Kr|La+$IE2ww z`f;)R9C7v|fDTC`TMJ(P91L{vrsdluJzUBKm@O)QiVXx7zbh3r!EjF7Ka4N%W<>7} z05?YfdH_%}L3ow${Eu-!*t#y*3%NIz^kL45!@g>Yy$jzxm`(K89ZM7e9oQ)3Ld+isG3>_S;U}ARU!N_Q&40L!;crk=(Oik<0CL! z;Z&=6?wA&whk6(dQ*WJ;lOkBzwFUkCzt#3Khaa{aiY8Ahqsj0aJg%+tT`Ug@0rhA$ zoPmnPRT`o6z~n%nNtnav8%v0&vJ$--rPP%ff7-EAEIOUPv=};k>%qNIx9J2>!8i6- zGrtXOLv;aD!i+<}24o87ChM#gf=7CAYWtfjQ_yM0-DcYx$~rhRQX*b~C0~6iBeXH4 z{|Uz0r;4!k_E(=ig*m_30MA09x1P%8BR2Imt0cntvV^XjXjGs&H;gJ}x`%x5eH{*6 zd-vWSTDcuXo(tD+m=Sh9Y#KD19XUyhZ>h(gq5JlsS17y707d^O;}Q##!14Zq*C1~g z3w80MkgPe+T4sYFVEiEu`pNz&n7MDrss`KgF3WQRA<@`s{Tp@y%v@k%2nHwe;;DzoZ5_|t?&)#4@hES}mWomZnx5V>l&9Dj>7 zwgLG4K&2*skcHjD=DMf=WTq;>nIW1Xd~dS9y3x}xIgr4OF(_Mce5!d3CcCeS$md@k zA-L)_2#c7eCOBSPAZC|?FzKPH4%qqlB!RR>Weuz270|1E%p|pB*ZHY>?20vnG=Ew4_l;& znfq_g9WPLoyT6ec6nLrQQvvc!SU@ykMCROZpy8=Wmph@a#6H7T`M(x^y`3Qax3@Q< z8lD}u)rfE5Ppz?s9%*tq!eF{2a=j5)fx7Gzw2$>Mrbg*sMmn3D)cu4XCO{><448>&CXvARt0B}X~wU;LukX3gP{cf^v2fxB^Qcge#w%ljv+yCUS^RGs_8?7nhX)?xpQb=Ow` zobaf;p&LsXjX!7I4TaV0KYeB|UbI9a{vWKn>^s4C`0D}>ezbXi_T_;rjp3wG;~Q5U1=u(O+t*0w%LHkMK`*^DhDA$ESdsD%kyTK}RpE zad{bnv%l263wOiIZXbk25?)txSASpUw*Z`$RY2FF1BMs#qC%I`6D~!n1c%nh-@iq~ zi>_4!>Rpz!HbLq_QZNN_?4e|7FOoqsnPX53C8jsXv0tz^T;HMe!R|jZU;5R9qCCU8v((M|LWYyQ;c%Pwtb^UGS z0?3MVhRfI=IKFt{m&RbR44%jV_H{XnP+uu75m>AY>n=R?f2a&}*osuqbv%wolVAup zFzZ}hda1>iU+oVsoy~tThm=~@N3g& z`aF3l7qDt)e)ct%j$_;{2>m5B#ZD~U0e->GN3alX{l?!zQTG}E4F!fYpKn+cuHQVga!NNFl-B_ zwqq)3fF|b2Z$?#XGBMaApinKb(T0J9mABU#)nAV}u1GOOln6gK40|4@{wx{ShyvFC z-aCDgN8o(%75ZtbHpKSg*bwT;Z4bEuDv; zGS2SXz9bb?iy&8wc_KlUUd0Is0xWQSK+*b(BZyZ%_6cA-5D`8)JJ?8ZTzKlb6^ z72iYb`EYh~UVl?O3E7vm=Ng?%aT28J zlufPl0`mu;Wv2hZ11uQo4R8tHPt5mF_Qn=7uq0he&$I@^ci{-GF#z!Wnv>wOY#|dR zyrWk1;pbEK8-EW%Aq?ky{FM1sJ&H9PCh!nH-)uoyyRKSx{O-gRh%nJU2%_$j9&VJ@(ie1jq3xB_SH3){pDE=hyvY7-+z1e(M8G8 zN!#oiys&u7H~saC^-05;UY1o~4Ewst*jd(`f&U$rthP*80;IHdCSE{060w`W@y~bG zfB~^YS_KT?aqz`{Ok&g$VMs&AnqNYo?&P@X{+U@f)ctY#mC=&mQ9Zdvr^{ zPp0?3y}ku)j3Bmqj0gUlpL9`~yAZWPbYmzLwZ_Jmvm$+T?&9EeOVJ*MudTx&qb<#4 zhev~WowBV&d(Og7_{Unx0-!rnf@|{?+pg;xDlyZ=nz}5yq>e9mUK>$cxUrTVPSf}7 zFtM#Xltp9VR?rs4o8SnaDbip2^Ek0}>xalyK7HME@04;l%~;&jTgn%YMsWe6TK4>A z=qIiPYF(4+RM5jVAyBZy6^@l9T%vhhUg-r@jAMhu@O+He(!qOpGQxLb?r6*SG1oi` zo~G{OpWmnOmrRdV)f=7{?oCHMJ2V=VIn-lMTgd{Y#yzKto4P{8a!VX^^ANKId0RPO zxT_i6J1XlZ&fm$}ULJ*CP{nZ69m4GnmCAP(?jhA{C}Y7%VIzDfv{aeywnpY-?{OOP7PLtgVHq%I>47na0g$94n*yRMGFM znv-QJ8M3U0M+Z{!DHw!ImU01;EiIc5+nVG)Wf2H}G8^2hZkhLF_HC@%f%)fg*K6nA zrn3lW)z0<(zUMvp{uG1x(&^&f(07y-cJIHIQJA;8CiSV#QqcC0Xb6`M zlu>XzdWYjs7A|!~J6O_OnA$P8OOGtSXToAiFEK#MLy5s>2cKB>%*d>Y zV+ch;Gi#|UEAINSyQPyH;m>nctjEV zXpDEt4fg+u#@O?GswvL!zWTfgyE*0Apt3KM&l>|)MslETmF==bdw&5naCoS^4cWAl zf*EV-{IoXSu@IB&bv{O0l4qVIxwXsgu&C!t`l|DvA03s9_OEp_|u zUdhx$Iu>`FUEZhgG}V~fbN4F~Ho6`(La;Y9dJQwBAunGzeW);o$JN3eopPHd+s{&J zK$ZR4{C$ricL8h+>pc&ZhAi77=9(9UX8Wl1xJ~tg<%q}nG`PdWg5P=3-Lo0C8k}-K zc)?n+3WK}FGDF?x6dPhY4)^%7mEV;s5OFej-!43zzQ3Tw$$5^MhTXOLyr@uNnS(A1 ziK0aZO_ylLyP&ee*EBEY_3hR0hE-z4IUR0sZgA+KB12oAqewB~Tsz`ENzD_Ovr)8+ zrN#LRnb@p05*TrrOJ7dZdU-ijF_gNSe95qzE1XM@3)^VWM`6JL87Ju5_ z?oiVUq#7C2dm`?JexFuvV=Nu8x|h>X)_;ML_WZ&=lTMGdKQm~@@rU=MzBqnA=hg&~ zgBQ$rx>=Ojh-Qy~{^K{hms8wCTM=h!6`41dnK0O_sjJHoPLa~LOK(CComm`*S1E`3 z+t}20A*8XTbqO9X2OiR!QAE+nxy5H4)@`RNbwSn))+TmhM9PgKMPlE`%~9Mmr884w zg_CeEsX6)kfiW)*_wTo4YB>gaz4*ytU*f!)y_%nNkq>w5JB8MRZN|Y}uz@^vqBZvQ zOpZi{t3E|m+VNtwi$Xx|Hhj$1d%E}{?s#n4+snA|e&S1V8t;mm6ni~n^~c@>Tq-MZ z7=4Nx_I1vh%oG)(YVm4!ycb^`+c&`$ELUrj>wNd~R9voC_3r8BLJ5<+M|v!rVip~^ z38N!lKl8zHV=yU;lmV%63QKrhhQ!CID)#k+yKR6R<3%&LL71T#k7$Q*eyA0D{)UmU zTQov91LV9p?v20A)fIIPJD1@+mgjz7CBylBuj2S|d9m;(9E~Yz#mbh6X|n|fI^B|} z^qPz_-n71?&oc1k5o73{=-@5k6U_>c5_aw&>mqN1ZL*;zf3JJwLfadjH^!vRjK);v z_i@T*q`iq|wY+9mte_C0Sd_SIc;G{gB)EqxhcwY(!^xt7BcQybIor#Zm9xlP=MLn-12#_d`Ljl)3d3 zc@L?|A^X8>xyitmW-zD59cg2`00e3<&-|XhDAi_T4$V(bKr|C8KG&yAk5|M|k0GE} zuot>VIAm)FmBQsbR4nhM#tVc$JK`(1rF^k_MK`sS z3|Jg}Aw^^AQ%U<~`scxB8@RabVTWpLNmUags|0hNnFwY*u%U9zXu;DOxut|wTO!ka5K8>`iN zHK)CeRk@q!B`CGP^Y=JdX=r`#va!c@FRUdBqy_v93tBO&(U z=Qc%Wy3qE>g?81ZVW)Agr1OOaq#Bil1tnj__xeh9UXMu?ePP}8^lPvScJ+2btf`IS z)j2yl{6Y~;F|PpB^Y~1&>9N5l6MuuiKj5mgAD=r1I^BZ%`{&_`}U3$ z^CeZ)%BVRCacFt84(^)OgAc-4TD@n#Hm6KC=*4@r4jSjd5Vt$Wy(^zu2L&l-Cg3Ze z{LMdTP3eUoMbVUB^*B{~Pj9^!|3rPP$Cw*G`FmV9X%|_b__!yBuCasmWG*$xx9*YD z?&`-KBL=P4c9t`(fR7+1*DMXIgw-h`p!+q&}PTqs$BXH2S z&klgwnWWuCzMlaUO07BN%4yAdl+HaUUzM8!Ke^b+QExU{-9hMx7mjx+F8 z(M}nh+EI0Ht|m{rP{iXqO%_ezLw62v9i1wfuHZ(bI>3`4MQfk=ML)P5cex+ z`?SFkz@?Vhr@h`KM66h>eDR*DZU~LV(@8J&X^=7NX7{Avlc{-VLUtTWo_Kw194$`a{FKD~$YkY- z%iMbI7LoHD$+eRV1FozC`-XQp8i_iKwR(+Cw>5?LH9vFcyyebXN~$4R7k6D;qr;#& zh1cbKC3lvc{^zB5apBNo!*(;iUPtIj_W2~`1_I36O{Ya9t`HwkXzzTUldjFwc;%az z13R%_Zy^;(eC{4T&!D2 zG?cbY_wZSj2L2XS9O;CowK2TCAiVdLq#>zshJn4qyC&Y@9{_r2Y}Z7c?ANfL@|<}` zmD8&C$+p_+ryEnL4H&d*cv?hy(HiGWCdQdP*J!eDOd3!eKa?={DaJEK*iW)H+}lZ3 zUWg4xHF2)#n9nSQVuNBb?^lkNgzOSqZ($#yb76{Kq#7FClhVF;58ajXFyRh&EXm>U zkGo9g7$?Xu;D*z^SKxW+UT)`)a@^e))K;G-^q*Vl`*~HX=~a~bu5K#Cnv&HpU}l*Q z7({8O9n@&n^uv(o$4_pF0W(zSBp7;xM{tOIe) zrB}zE6x3dMWg=&bV)Z_Ig4KyXbofL@)_Xh<>Da`LON?$j;MrnKU+CvZ`S!B5x#@Ve z5(tg%pnvtoF&xPWPQU|cCSqqYdmF^9k*riOuoBM5AJjpl_Q%=S0bS?5$B7yV@c^Ea zP~Y5q+U2#xU!t8dRqx8p9+q~g-WP2}%P-IR@#pZFvA~6RF?BUe=30G9@7_A!(Ou#9 z{BKH@W9ddzzlcBd;_TM*THHNs?UuO3Rv?2%E)m|=Z{opvBon(vcwZ7AHM>%}%ghx1 zzfr1p#hXflYl88VOA#Ys$j{v_WZFfjxoRPkU?&-|PlCj2WRKC6?rM2^qXPm`DB%`L zUb9>u5@{wPqU;ZndQ3}5lpz(N^-TP832$d{a8NgEovyKpGw!LD-p~0icQ&0{Yh-S5 z)mKu0*mm-r+;uOSWve$?sz+RIy|?ZG(N?A%2kuu8(2P)YxRM%nJ1o68K8XN{F|YMo zLszbVerkDI2T^Zbz4|P}D(Zyixy|p^)TFUOBybb)-S&O*`-%J5u<#!@WE|>tVQGlz zNHj**bRunL)%izGcx1hu*{z#5ZyJt%euMe|qfK7k+Qd|6XGUiSt*ML8h5sOMh<((W z(9`2XRp8nek4VR;|6EtWFAq-Szzxy(yL${V;_*uRmfdZT6jg%1XlqUooS@xb+(fqe zCd})HR(%h@gC)D!@%gVf@&G<_U6alVCSpF_M(;-le=MV5)L38yP$RABE?A7WD&if> z&WPO^5$$DBSkzXXTPPNjL^T*JlVVIm(xDsTi2@3bBxhy@h;`=Yxu@f{wwN!S;!H?SBZ^_`{;2B9XQtZz2j}zBB5*fc>D~qXSg;oD6>E&g)BbXDxoF|DW&1 z-bpO>?KPTruJ)>S?;7j+fZ<&8M6jFE*rSA=7qe(lo#)jq*N-r7mx3#nNRhQzC7B|z z>oZpJp9}ZIpMOi`0~qlbnP7i<*^q$uR~?)VU#0UEqggF%Vh2AqJb`Us!jT^p!W=)m z)9m}i)hVl`6Ii2xvBT?%Rg1i+T5A5n5VO-E)73r)xadWo|(*Yc~^7?n#Om+qBalM}*L$IM|_)yo|S;h~lX-w3Lg46HzS6@^U&= z-hqI2DXF(Jj>NV4?id_PcY%eU<XvcD+lcVM*Zf5ADR#rivCx5Z>pmx7T0#@zQ!cbf@C( zq$E91LyG}L$z<#YXV8sz&h8>b?yA16qIX0Ferpuws!d%bb)e2(9tC@2_*XnLWR40% zl#3)9Xmsevt}NZpiIiPy?L(vUDfV*}YC#nHHh>mwyGlNe6$#%`%uK6B2yltYxr=Wd zQATKqjQHZE@gJ5YZPkyLbZp-lxY}{s_m)8-sf20Dd)!=z6FkTU1@>+uz?2ZPLrc?| z#rgNg!dD*Vc7i&lZhp_W$udc#otYLRmOJUwuR22L;ra*9v|z^rVShQ*=tvIKFk$nQ z!~`0hMp`&%{^)rY-nWJld!Nt~CJ57c@|Ct=ok$tK+l3@13|HenU*Gt>VTx9SWGeK-Q|W=GG%G&_R=pjW3B`G&zsmKEW#>br%BLUJvWpBxtyUgq&FHI2nOS`nNZHsOpsE+3XoFNx~H(+uwl5VXCvJe=-% zxyZ$EMg33&)vuOMUIJ}G%)9uFmovSMh|54u`bY_0E!o5I9JriNA;DJER6I+g6S7v# zEyPo+ms=nCjSsK)3aT4k+WT}}C0DPq%Y+H?mvmJIa=nODv|H{fXBRKp{=fHGq;M0wc<0!F(~B)0ONOe( z86$?gc_XkzjWqWcTpah}?9Ad|vk?l7ioItS-FJB=ouw}DB{yf)+HNCEyCK&3@cWPe zmC54XV5xT2h~n3f9bG(<9(81S5&|?1N0dY!!S2=vr#GUn%8QFE$k~d@TP2u8I}PrN zXnmA8)}p|Ge=UQ<5rm8H7cmA>E?|}{ig9FkKz^XpGpFFiwyn<Vz1LI4*rup6aNi*GR$t*1wFN?1N+diPfo6 zw`W%t2)_gC$rx>FBnVOh4iYoMaN-%?&Fq^`t@;k=Mb635n<^$zlE|NyCvx!p`B6@Gws{+If(EU;W!}XH3?vs zo~Z-!Y40#VE$xy0@=Ad%e_TNTKwG4g0YTHM1x9@90X;Yv4UonF(9c&#M$VRl$9{3F zmet&9LyXlTc^!1QEqQJWrQxGQJJ`S)kpq}hfCOk3#hU$r6M;bY%Q@o|-5a?OQ)mX_ zvWD3h-nfIbFLwA8%(Q)tK{&WPw1}^%4gpmMS#Das75+oSRm_H!1F@jfmW;GtsimPfs)S}wgE;iW z?0`RovtmlB6H!_sXwMIyw9HG$`=7>!mA`$otj&;a#Aok*dc1%7O{FLj?ERbxPmMi@ zQPFmsc+i!2^DdPg<9XX|Kgj8@1D!{>6v7hI+tc(uUBS!B# zU3S1ANa=re=`}&<_pU#;?EWn(zn>QaJ!t0^vP-uhGO}RWc55A0i}ZVi_8jNX5n(h> z57_Xs2xq+o@zev@tgkZgNM-dS;e5E5S**}HhFFG5*(GlgNPg*M_|s5Ah=S9 z;IJM6Q6gL(nqg=sF;d&xCSMEiVanNi5k&)~u#-a@XXKka5T`nT^i)tE?goZA89M+h z5?2hhK#ByCAsMT}G`#Zp-z&9{72;+Eb8E=F9?7r(7BQJx`6)G80>XXt-+%vIH$V## zCApE>;VKT0Xf>7Mr?%^9?l?X!-*&$cpvs;G3kOviAl^$UV2C_xnE7bG^}B<6TJ0VnmQz86ugwu2-YnXuGz?;|wMd)=}KL29+w4JOqaB_#PH@PKgu+RDzEZBnM5H9OY@h@^FKuU*Fw zlg|eW#QQuBsAb1_g;}})7exj!2i&uBk@ZEZ?IfrPs*_B!?gY2U4)48wW_Lif|(B!+fu-D>&deX>~>060qm&$VH^(_fr7 zPBj7FZU#ce{gw+*h4lct7l?4kEdj6dQF%2aIoq6OhWlYIT^4^FhAs;Si;*&#zud$S zg&+gkV86}}?_!DMuKych!oDTjYaIwM&Pkv6Q%NMbu&mwgo$CHm=~P`p{P3pfX;CYO zz(8H(#Bce!I^48m+?q%@z-!Rle0AL|*34*|#FDa^Z zD8;^QunzH=A=6KmaBc{&O>ITF>JdlXI3f%^4&3E|)1k76q`Cg!-F01sz6RFp~ ztN1K%o!2!-pnjII<(=Yi2;O!JKyv1MG`zM#)FLf%ef5D|FnCMJ{pHiW;<~RVg>dEB z79&2WtE3beYmEivhTcU$~(?Gn-H4Ka?(S9l49AoMmY;PZwnS4GDo&$D=L?ah@A7doD+aQy{mtHSG=7P?)Qh^~S=UsDv-2xXT2^+z0Ez*G93en-LwL`xjE>~? zJ;4KGU&yg0=j)5OdFQ;fP&fb+r_HWG zSHx>xBYuXV0!_RA=$u|nuwqQ50U8m^UEuh{OPVk3Pv3Te{FNtL5c1`aWR-&mCE#b@ z;bgiydwM=thdn}KK*pU0M3`2I+294s88w6>-$rkii5C%4$0Bv$z?WDCCemKDJDAA< zz~p!puSf)fzF{aiN;lGY6S3{4eBqh;1}k|~MV<*S-GCo)`PV7B1?PRiJH}Fif6hZk zf|f@J3yxftX?NoZT4Hv9boy~{7EFi~;spYq!pToPbF>%%13AsKb~FvkMfyn$gbCiL zL}1tm1$L?ufu-@XafSiQu#G|gBoKI*pPqON?Xo?2eI4)~8oozR!Xjz6{6D6)R@Xun z7jt-%~Fq9IBgVYlTp<`T;{qW+Xs`zH)R_%x4aO zt$LCv>f#CjF(@tOHSBj#7@$v~2#9hs|e;RV6{RmMeq-(Sa=`>{sNGySmecQ2h7bb_^_X9w(kvd3YJ;Xm}lR2kGz~i$t z)k@$(MB2y!?pA&QP;Ifi`Ta6E)7~7WAD-J7yq`pn{5e*vaQ2w8@-Qg4O-+-~p95HV zC4qfaUaBUS_`3amHhsCLMA&|*y0KcQc(6s~3^lyRC9MDCa3&DOhu}7B+}JyGd3c}o zy9@mgC1@M2_W?c(`dMI5kU)85;N$wLKzLXafPeZ`3UNw^+PfY_z)kQ#89>eyV^Mva zHz7ADgU#X77DFg@^Yjk`%)7BN4lWU(F`*#E>UXrmVN78TKr}V%hKjw2YTg)MM!Z4@ z*;vb(DWVtIGGAv<#L;#WZ<$mxX$_(_7cVBI7Ivvr4BYm$FNn{+ug-nkCIf5qy241F zcK6jw3F~QGFF+0c>DhIs6IzFxaFpxURdQ7eMkfn{af~jz#D}g+;&4WbA z?~p@`Aow!sWsgMzYw3%A;A=mFPT86jxL8^dUu_rg9 zQG-jGvs8Q}EXEBH=jc54@+Mo7{Y{G)91RP{0|%>@asi~msI8m~yA6?G3Djyn^|Yws zX9xuejh@Y;MpQfTTF0YESt3q5BYkJ{`h(D9s#o-LG4Gr?++f+MFO!Hp4ZM}Jph!}| z1ec!b!80CkoktaVI|&h$l_$l&(@JHfZxdEkjV=pnutnZseHpS#@183zB8N^~#BEGy zh$5>&)a~7}O0Z!Dbl>10HL!A;eqcEPT?)W`d;e~hDas!(MSKEYOF4wf-rAz9-wix! z_ATHcG5Z=#PPJX{7*D>QpFp24;2NcQpj4rqdbkUv?C^lMMhLx=-xkrA#y4AQmZ_Rw ztlFe>jMya&@cUGv=X$S#N}cC_CLX~;X!+k2faR1WdI4>e>#guRDhn>|$&dZ9;mNID z=3R57+-ZPn91VPBFHEPsFC9U#B|6Nhl$Ag~z+Pu0X4#HdFs%VDHMcjp05<@P3^;{0 z&c?v8s$*tWb*^635X9v6ip`gUw$TS9JT~g>GaRMUh}C&24@I4~)l9Lqm-zlP|3`q~ z2RXW?LkF%v#D2^xHAA`FAGkzC4hQI!TJl!01qwM z!_<#AwkU_!O?D9}=fJ7B){0oo>!6HO558fj20aUHX^@}LhYY|9qzMHy{hZj_mkHa> z?RsYr>B+n!xZO)AC3hemsd5x*R~-Kep<3lz-_;2_sOm826t6kGkV8XVa;p_`66&R5 zJ~Oe?Am4>@3u6tV0iE${#O=s--cL2cUB5RtX!LZny^zKcKD{tQb5HI8U`4R4bx zQ&@|LY3+R_GY=%+VO2siL%Fe{MWqNCI$K(~2Su8Gfd7;S=SaE-LgxYQ`!w-hI0#D?q=bx}dF5aDAODZqd#N;}Ps;juP`>dYUWIOD z%lmfn(i~zeLJs-4{FkDaKB?xl%Yn+f zW*S)T0TCB1K8pr`obl3BzOxk92~!)41#!6myw}L0P;L+S@{(SS!aYFKF%Wfux*Bv@ zax-8R`ul4;O;_3>;v^K$UF^|6glL@%fnYS$H?x>Ts`7nGJ2AHcYv)4qqNTiu;_|vL zeE5f;z>o2TranDzXB{0zwuP?4R&5akt!%Y7X*V|sgP8nFL|=pWeEF_CZv&UZ5YhI6 zb7ZEP6}r9v=!#;9bl({31?u~s0myfGt;vvagoRn|DZ$=JzXynH)olYHm65%bKUpyDvWi+2zRbkEOa4zZj)4}4 z&6u(zC{mpU%)||c}V4# z_%|29K84>Gp8HOAVz;e+d4DR2gl!=Xvy-L4-}$ z#%T2*r*Z%82NQr{mrm`JFvYriAo_z&#>Q=wIj9nFkF9_&Di0tDB{8d0UW*RLLSx{n zXlT`;$T$mcwbxt;vgBAJEJL_MntHEi26FljS06%g=uhmANq7txFl1UE0faZ@SScjqo|!sX^#Gqq zz4)o?h@<@S4?!AY5QlmY;_3G#pG# z68D9dw=!OCX2I8R_L~6w(L6;mf^xKFCLAxKX>D$N<6^Z^&?SX?%O$q1oUq=m& zlN?|d!+0wD+JO2J`;d<6QnNrcA&uC1UkRWDkI?iRxfq)(sy2k{w;vmNW(mUM@v*Wg z6iN!gQG2*9MR_@VrnO@6B$q}~nX8Xt7j8&oG_fj%=D>0FyK#Z$_zhXzXrBtULO}#h z>isaMgy&7A4Zk@|pRvu43ojcQ?1fE?t9mPcQ7iEJAd%}*LaN*NtUFC!Oj(|*$w!anw~_q$*MgaW9$w#^BFvU3 zRF^XZc9pkea>U%`z&OIIly&y8YEIO9>%EOUK0gqiKO&Gc1SM`bFb>tBI|)gMN7 z-KY7ZF_azh5`)H{3nT3jG=A+Rq(v)UP@!`WL4+w?4Cj|jg;iiuoqC(6|XfqA5T zpbZ{X6~u*dy6vmeBP~c2o!w!VoYNx-;Ysv5_e;#Jpx^4D&RT!Ty~iK9+Vw#Ap<1Ktog}0ECOZhbaKwKoVFXvP?B+Dzutq;itaWiEP0NTC)uN(|> z=&?cT!IB%YxX3Gh6Ert{_sK$S&YMW-qrTtBX{YjRUrYxLlgKY1I57r6>hQ_=vt0)v zC8owAD;>g@de83NbTap5N?Qc74XKf~wEK>uzTwjFQik!_cA`0{5bYcHfA*C-LcM(u z32Jo2{l58UvWd)sKo=7WR-e@oGQmgk&)6el3B_!l*~#oe85ppXLB}?JcdLpfJ=$*5l>s@XXmC7JH1o#4 z19fT4c)&y#)Gf}uPw?Ez>If;k9Wt-^TOj#V4|$7V-`23th>#|STo{V|{lEZGxweE2 zXC9|f)VEqGKVW0$<5TrzGKji!8ne%9>ra}ux-8^YGgfu7%cO{?i|RBiMVWEh61I4nIpleB=?!P`$I zbP{~+0)eN)Wr);XA@QnDvwY%FpGOQ!UEKJ`U)6ZO7h*BPDzyy`5eyZf$5 z;EEZ9ymg&H?&RPA*{NSiEujnz3Y>>%{c_AB(s@XT`(5XvewD2yFE$+o*a<0QuN8#8 zG;}>T@2W$-$cG*jE`JbQRA=Z}bJavv#u*kQZY368aeZnWF-hAg`wNH%h|wQOBbt@u z*2w1kFIHA#;=y}ZZeD@YZ>ct6NzzlOch;uYo;#?^Y2o@!&gp)s(_K(8lvuwFy2VJR1o9IRoVh_L`o{JE!M z>47XIM7aK4Z7h8Y??q2yQS;sN*LNgbM6Pd=hN*?@Q}+(1Nd!U0++s`vpJyD;|LVw= z1gbTs5qx*499zV>E-+F|YB5%^$9^bPyag0m#a&i=#A2?Ros4`0t1()OR+DT`a4(@O zCdrcc?4+sF3oe?t=yoGUq-S2dh`_^1_Z=hn>u0RG{~fU4!^KSSN{nQpkQPtENF7c+ zZ92R6%Ci%cyvbGx*FN)bR|}3sb$JO3LEp=@KvAfZga^(+a`=!;r~nUF3a2@$-=sb{ zqpzMYX$D~bxk^ykF+mU8S?tD?s3&aP3bAFH&oTd9*i#7M+SI-`#6s2%{27ZeeYop^ zgUide_~9213j<9eU^}+#RDy6A`-vzqZ?rtHM9>*8O?c>COihG0gSMj>+q<#P&!pC!zp+KiT27T3N< z}QUEE>>)s*W?){P5`^g1T;q|%^v-q1BXV7#PR_6w(QC0?^y*fiX9vRW!*nJn5ID(iG|YakX$j<9t6dx z8K$eQ5A!bS0J?u9<5)S=w}dhnp!tX0sSO_PFwRvo0}T_CTo|t>47bYbJ`y#P*Hmy2 z-SfF>WfS5MIbN2TUp57XkM!g3$jaZ{3@9fDzdS!xUQ~;ym#Pm}421yUt@Ax%gwQ|p zdC{V0NF72U5L*DNtW0JtxTrc|EXjAb?bz`oOkMJs$o&Vz_KH5{ey*=~QJoySEtdai zJO9Z`T&iI2O!2_15x_e`dinuC(tlih-r;J%u*l92MN%C-V{{tl$iHn@(K#N zpyGxGdoEp{XGWOSb{X7Gl5HEt0)Q6VxXa6j4kE07O`K#-z80Gg06*$tgh$)T)z!Z-#L9299Xj;(yvNim@*ux(?d%D=8Up%)Nz zuLH?@k)dc6dmOrEPLg0DYbKcjG0%kE^0W{@)ZD2=`d?l^s{v$4gF}5lC>J|29SXRh zcVih~qw*&$3YMtD_J;lGUh)p{zy2%1${f#xJKnQ!woH9pdJ-)gHp5%t*Ow4}>z846 zi4+2pb;a+Zy+e%mt(rtwQ>t1tdV`YwHmXQ4=UMCityLqQ{of)ewd$bghCb?uEqlPV z`QD+w9{j)CbEj~4iqt$N$>Z=c9c1+6%OAC-sQ%Ua{3ToC+3*pqg`Tb&IBnuD|Mihy z|3$MX@*8I|qIY|6xg`CUNd6ZOfJ6{BQ(g?hv5$XVUN29u9DV-Yy1yQlAg=#pfD*o^ z_B1Rg^B+xXJs!w{JyyKF^Bq0*?-vddgq6;jJQH}69J9T@LdbojL>{Y+8moupKUzFA z>hKZY4n8BI>SH1F-Z-5r^WP(CL#R6NAa?y9jS_Ef9Mu%TYxob3y?%jyTXFMGaBy#{ z^}t~&GsMjOuL4HcoH-~6%WBi#Izig?OIt1P@%l3C6_NOLf_^=g(<98?8}bChikdr;! zp}(GQJj1*2-+$kHn$R?cR4Fy5sv@Y=fF<5VK{JUYfDAqdK@0z!I-Onqc)S6 zibEKh_(dRRTi2Mmxnl68)B2?biTI}-{#`mxo`f|fEqH!g6q5BXG}v_a*q9C3{^saZ zZRK8PgZ-yh{|iHuc}E&Oc-516m^t*(mCzg?5rO7Mo!<~r-!P>6`xsV@FYZxG>O zHXeAVyPzpAMG+;Ni#irU48(n#a1;n{l4~JR`T+9|BKzX zYJqs6G;Pg`!&-#R`Oox=wT)hXCa~8jAQtBMk)2vZ*!I|Nw68`a<2N?E`2S@GEkAP> zQh9?18cyI|5ZkW3(#j?9AC=|m0+abag0Nf{}YJgHai5Z#0@Lgbf!~~ckJ35 zul%rnNB*urvvgo(ja+nAh|D5Ii>X7!5S{_YIJb$z;?n3_vnHC-nbS*(@Ef4kanM@Z#A zL&e2o5+|53H9@*hKR)Bc@7CuSY1cpR#}__9P=6w4EAMG7SEy&@99@QTtn3lnam+5P zIzkm>y__T)5AoKKo9(k;TqztCG$!i)hesyVXu(wob??QB6G`kQpL9&t8mIfV6BU4d z!!)4}omGjyLE@^%HmG(O%TeGaJwfQ#9w7gvF#em!d@YNy4tzAdYAmG zPFwySKt%w(gYjw#(o2{k3i=iQWA)en)%5!Hn64*aDT0i-TR5P_m~`Cl|KrY}L7Gi7 z4euCmqFCbBClhw^-|p}=MY?im(Ni`8lqFY~0p}qYiwESp2 zeP<)M;Q?wTiA4X~U=^+i{HgJ&2{FRh)jn!0K$G|!I`Y0cb7n8pE|$;F`gDFbtK*o(EO z$%@ih6I#H=ohrH`O9(FI*RT#xmDA3RN?^`w77hn*Tj9GwMt5{=|D2-W@Ev@`mWLuB z7G(6LiliNHaMKsLv&Xi`rL(fK$(f#KPic19uv?f7hN|@~O#T#W+~yM6IP#L-&FK)- zKc8ZK93g~YIlG9f*C~8l6WY7k-5AuN7(Q7*?FUqmAKTBsl2gLo(tk!=ePZR z_po_W-dWv2RVN2`#4?8ojtwj+5BrQ_u5mftn~D6=NmKSCn{IEZCuqccYZgLSZI8v zrulNNnaO5fi<4~~4&s;De9f7!ptsFBl1gfqk$WrUYoL!vD*SH8)QS>I9>aAKGm|se zoYOjAu8?|k`I+`&ud(>Eg?zFo(##zFraANG&ujktG8n({PYf{vghEqSpilh%94H)w z?IGznV>tFmn z&*j%?OCu}kC9nD;7?@w@M4Xgo6p*>;YaTL< z;Y|KEZLWY*yzl=T3PO;Vh-=kvnmxuQV`a_h%^{QMQ^VJB`EH@NkRm}f(Ghj7J6S;H z0a^0yp&=%@%|*$_uL{@WiypQ9Z!c0ruk=nedE!x*H-N13t6_oQaHxTlOb(?~s1nfJu&I=_52i z1&d=I`8>>Xw^*}?CSx$&9;VXM)5V-Jcwhd>bi ziqgc@m-g;4aH%89QpIiI^=pf$ywI~yed>Pt8a!}W0lJ#t5`fnb_-F~jNM>x;6#(O_ zV#rpz2RrS(U&gw^mcR83?Sat!>?<=+`kaXt);FyoQOMH($dAR{8mC6@EmG>#9;)`u&w=Sc4j?gR^c!7(Nvv^bI+YVa-Je`yqi{Gu6H-p^EXG@uZbckSC&epriR04=J!Mxa9_Eh*YZM}Z{pRPx%a zIjH(h-mLGY5dXKQJ)IK}a_nF7%^>*(x?nswRoBgzkFj)4&HwHYNqA~9M z;l&{rUh&Qn&UwK>97x9MrRORKtCnWW1Mx5%BkB0FAQFqSA=70k1AEK297z95(K*j@ zgMf4Udt+CB7gq14cX7xuWBD5p5AYz~5R${3hTDy1vX0@?2mZ*o~JK7xC^Jk8a4rhP%!;PfL4Z3x_T}&tKu9sFjYgPnT$6!=(x*25qUOKrkJ2oCNxN43DpF#qyc-V}ooF z8N_S=8<mEa~M~=rYHzEcm*9qE*J%VO_Kg)`mFOTOs0(0Hzr|`LK#GU3lGVuXm_%SPV&E z37wS=?cEwB$%TN3@S)}vopYN9Cl2ekya8BAP}~FUZ((&Y0_1&mgCrS+(#f+%*@jII z9<1)s^8dyBBt~`KD_;m4BX>t#-Cro9-z&#ckg=|;=1gTB;|-rg&yFhL9BPuz%Mi$X~HoY2~R`9^YIQB;M(!8L)Rj z9V8Qv68UF+5@q($T*Yp(q$2BN;WV$QfWPDPFzNE#*75pH z_15UOfm1SSZy+;IPm0qfNCD;piT|(p(gfdv5R2SF^>%h&%Jlph#l623KH{24#v_2T z2+^URjKw49Q`%*T8;ICP?F|iB1Lu3Uxe?w$g?RR#O$+)0jJu4Mfr`7Q@a3NOl^k;< zu7+&Hj}YCXBRz4;pQ{KltPHInA>vOT3!?wcg-6Z1SdcWS?NnerShw_0A7HXSc;d+j z+^%{PnpUtc{~o(8riVRXZ;0!aYky1cBo{_vc#txVyljC9dvcWiH9Nn;1pJ-PnN|C5!7G z=L_n*7C3^rPA){@<|H+gArMF+&)#mlCb<`o*09$*;YCJslC3bn8k(kfvOfv5Q*M7| zh$|25WW1I1>bq$QEkI3|w`fJal1P}=#F$2xt&sj;b|sOnMt0i!Ov1%ZN-MbDB63A7 z4mD=SHnY`w67f@V=V2!9Fg3Kr>Wq&vZa^e&%vEBQ;XPC=Fl-;3O}W&|R7sj4}VZ^7hz8Nk8tw2`WlqSl?Y_2Dt|}OZwWs~E|71bkTPW7nK0jouQ9yV?HZ#on%a4a?;EaU767#NgCUf3VbJTIpXEvjZ z7%f`)i|$a2m-onMc16-?m zC8p8NSoc{-@*0%w*4?5rjXeCt`f*mBGG92cJ!+Xmz1vR!&2YirHNjwIk1h3iW4tEcYinC$w!6w|bs9sY2q1*cJ(`U!gA2CdRo@32_NNTg9m@ zGXA>8mVuo+_B6pUyu)+^=^Y?Wa+<&VwYskINT(a~GVcda)9CHdu8J^~FfxmMM~4H! z$`5qYVf#^0S*YjrRx`>f4hk}N`RMQh55z1U^^=!!yigwpfPrIFRf!*a)E*c%)+V*! ztKq)S1bX0eDL83*>fSDYD@mSWyMWh>O226DOmAEq^sS@^jvwIDpZRu&^@n0}mB=Ni z=simt4HP*;c(M8$J7prH1lC+(h!ix$Jv-&qqlN##02t+k{pMM}UILtfR8UYIrbg_H zsorDu)KG(P?Tt7mRY_y^?NfwH?-s3(`3(>P&LIHx^#>o{LEsShZ3?}DdOtGEgts#6 z;bm>R&fF%&3^ zWF%t`V3uPB&AJ89y`Ymx(u?k)L(($jL zGo9nn_AONSP@NQV$0f}f)H#b_J4+m0-}#|MqyIST>`C?1Ga&Q+s>KHIjn)apz(okV z0u5bzby}f;7EurOgf<4=Kuvy6z96g#r#*gGD|8_>*-PpYL(7)*`dZW4Ibx`XWDxcl zs4=;HCVK>nID68h|H;!+-vQ@*7d>2ciQ-xWx9 zs?Do`^@3A8HN#$v`3YhO`>}j*c`V%z{o2*|KcfpLBrE6=0mCb$t*>cB&m@N!@Zc6B7|%!;Rr5xDc{UrT}wFdL`Zkdnurf+9hwt zORNYa*%>m$fz#Ll@=2(Q?fdqj&sCP9M4D;Xrrc@=A;AANXh+#Eg9*}i4`4@-4s!ZX z3cSmeZ3EvjG1fJS^D^{=B`euEAJcvHLB27*P;ofXz7cO=1GG5&p{{!!3^71h>4zSA zLRVT###&*tB|qTd5gVm&$OQG@+J1R!=n?% zo?aeGfMqTlA6QS?r;8b>p4o7duE|p=t)L|OQ{P!I1`6CK#tGdVxA{*l1n3n?TYtK# z-e-jsVsUG%WZBA}b+XwlT+H$L(Dth?n-P)d3l!cCOkT?EzR%}TKTg4iKbiAT6+Vte zsr~g9R#j-My)yfHr)Mw_@J!<$?b(Vgqx?*_j zEG%AU@ zzFPnvDTIpW#Bg0ho*`)JBv6qM@DsE^vY}3aD=e)yCg1a|ZI7CRtG4c?cXg zq9XeUmc5pS0t;9*u0UpprtIt8$?y3yx-Gs2_-F)h0cwFqH;6KIQnY!rHo%b%@^~0c z0tP7@L473puaG$@w*&O*s+zWg$Dom2oncHcoro|9Q%>Hq*k{8{MT^dP-!tA_nmbbG z9RhXC=_0waa2Hmj%%Pfk0K=U70?j#fh#H6xYU4u=UeGb{&PwKI)OhPup_s?`V^)5( zFP?34tr4w1W2fSw!+wJM8oqC>)Po>}iUd@8U}E{XA;_*$$sBZwO?Y zdW1fTSKrdYKLW}v0TD%802ta2Mh|z22xVA#4S?2ApLQbj5ja*`x*wX|35~f3_LBf8 z2mv|VNiqK)(ChKEzYL-99%Siqb&&=*4U!b<1{LKMMk?mk$jqJMs*x}~7j?EyQJ0>) z7s09Q;Ybm#?9rSaFo+JS$@`Q0rmIB>??){M32wPx%}KHlq9I{!3N9-?uAMIPm3oDB zBhAn6JUrRHxn>J#;mg~Hdx&Kh`ZW&CJ%_D3_e6zb#1Av-i2!u{i*v&o&@5);XI|4^ zpO`p3r69GX^ztMSSPflU(f_DNSO`#-vHKRFwL=k}@_Xcs62Ef>G1gaW74vB;ZF3m) z7ZaX4H<7xQ;egOV@ChYC%)Ui86kjycJKOM~ zItyWqavREv7(2I;UD~>8lEH>SsAw%jw0bW>a@d^xVX}Fr&`L=NPPx6&$RR~}rsf+d zoiS4Qi*Yyo3&<4TDjwWlnzYFvEJ-=aGs~Dw_ZmV#`86EfA=!t^g6`4;GBThm>eRt& zgm*N5wb<8&kXaqk4}cKamWY6hrDsw;T;b##R4}iw9^BJE<6_x&qE^wg8L%pCig)TW zd*v=FkX$;m-5jo06?Eq?RODP_g)S;WH)c|Hf{hDxp3xUYI=!A5^J%*BJtVi;pM!a)hbIVxNMnGL=+5qQFcQP8B3<>ekmhU>a6thfYgG zWvL@cLeLdssz}sIEOL`SplQc{(RqRg=GxTJ7J!=qt#jW-&YBQpktr&2N zmn$BHUTxM_t5rCdCW^ilPkPPmCzM`5H0Zkg?7|K~)HOFAf>E_iV!GfyXW#9pIRy>p z)p^H{j1+Gk>}i9CcmBBqNVktw5fbiLq?c*Fj{+`Ns`Sk4bYr*v{;q{#FRy;17)$5i z$YMIaw_T(n+lm^>tPtHYVC#!#B}KqNegA+U+8LZ9r8#O=SDv=}j;eidnfjVu&T`~W z?+~_E;9Pb-YB6%{WNiC1c{neZsAfxxyYqMo8FU$I%hf7Um7UvGy8BvJhysDypf1@T zuSGcqtejwr?&r|v@*?Y!2??*l$Or(>K4{*A>*C4dA)UjMa6x5`JWv4KWT;u@<6Ls-G3g;y_+)d15AUVcjX6G!i(>*P#{lEQyyS(H_ zu7_B>3{b@(3q)L%w$_Ie0p$U97wmoW&9RNyp(aN5wQu>l`K%5v{Y20p9_ZLA z%H>SyBiE3gIE5(JJ%Ur6w~=t1D0h3m*uNl4J*d7K04C&pKe5|#^^l-_?Almos=~3l^qgY!N_4w0Dy=X zf#-n?XUjbgHG|v_=r#F}2O4+`mT=_h&m`mxc^ap8AUa%6K`z7}5>Th(18e>A(UcxI z3QL4`Y_{ap4s`WVwLT%LN$81JAKl(f(m_zR~yy0vE~Km0t}X6m|m*t<9m z$c~89+L$%IwAi+k3>|B@dSa^S8MqVn*}rJe<9^0=iQBfk}SG=C5`L14QkR8Yv!!(<{sDIsJ_=mJV@2c(PyX z1zDc;<)&_v_bA(A^>N43<2;CRX=mu?SS6}AXI5RNEZ7i$@G1>Tw`2q74%-DgI`~gY zq9kPuFx?PnfemW>&)888HPUIdin)7%<#-^a_9uimq@b^zRT&X73_+{a zBQu>A%x3(-DfOXpv7ez)farLQTsUsKZzF?l~p`A01KrvK9FzM!K zGuvkg%0wTiPUQhe<6i9J&fHmm<(o==fBr@}O{~-TTz=~T2S+BOE(MIUR9UY5@Okyk zJEcZ2YU*6NIqxV=1qSAS+wS0VC$)w52yI&T7%`STI41+Yv9${J;VQ}ay=BV--BQ?A zI_sqauk-@WRX>~=h=o9jD?n_a-SspUz?Yn7)m)QUv|#F)n!gwLNp;WyYh>cSg01;< z)B`HHJ|kK?f>O7XN5K{W^&ggqNE%oG|?@=H9eZV2pj@zEmIYSz7!o2vcEn};JLEG$C2fl!>VBSP| zZADD@aZrTz*KiUEmZiba`(Jmn4!RgqXxeGQjFFM@L`Tbssz2XQy`y4s06Q&*51(fl^^s9(qM0N2}of22L z?pdD-7X%9PtC%#fx&Zd94Pdk1RSx84;VayLXmm}XY!r|HO<9@1V{M4vEwT4{U`82T z*pbY-BjL2kLHTg`4WGI!-4fV)Beu|8vaOpq!uXO>PyGC~R+i>eyMj1Ugs`I8DXTj~ zprW_8g9X{s;f(o6mt1pJ2gN)iuHP*OetLL|w#`o2`HSbgj!=n=0l@fdc8dyMK*=T) z9lZARGa}3Y%E#+tyvnK(1(PeV`VDy*y@i+zt;h9qUp9n8z@76uFj6@u_?Q(c{68=* z$7Wqmm_v)zan~IwHZ>J1}{)Y+g`w<^Rim+ohZsFdv_I=Mw__l60VHU=1|&&r>WOwx+BumFFj;$ zpJChbBqDU;g&irvvpp!3gV+H$e+acI2|2YdJZVGv%=>ij9|0;#9hykt1PH7I)Y_X6 z-luEg+(5t~A@8^!obqeuc0(koQgyhr1mEN#6sS%TaDvS%OQ!&+3u(#f0|TA=@9XdD zUO2*``||rh_26pj0FIGld}B;|rKEy>GsHVAy;GHlA!{ZUk{h?pKOFj+zXIt~P0UOh z+Z&?5qErNU+y^pdugi#$mFTDBGNFvP>;za!%KRmOh$Nu%DWd5Bfu`KVf+2f{tY=RGg{+xWZiE+9NF74PYTKA8-Z*}3)n|YhJ<`hCvSRB*a^vQY&e zfExErcE0r|+PfY&JXB)WZ;Y}qWr43d0Y&H_^K zK%z1@q5%h`B&aJ;h1#M%d8xGP6WnEbK~nH}>ext(3t+e#MMhDOwzMSifqw*b$M&%Z zR^=syO~-8#Uk61#|BO?t|130mXi@$|u%hbhD~Mibsb2Eq=dAOZdex?eu)H?=+JOr7 z(W&4K->23(bW7LsBt8^Q42jI5vwQ`yx4NX{R7oxm@gqW-8F}+Sgw&4tSYK{{-HOl_ z&E*KNaV}e^4)-t;!@&0z(K>iIt;=5owbdg6;sVU-&_FO2<(lXa)}qZ+#-X3FJ2)31 zto82vkjb#C-Z|GJyE2L8|BL}1Q_1o2oR@*4N9F+Zk|-iEgO0GlYa`}ief1l$J1?2L za!4%7%(rimjXWf@fT9pQ`s~+5Ak=V&5GOgv~y{3c&QFe z3?eWL5gAtTd0Bv1t3HM2{FkP}`yILbo*KM^uYl>PT=VS?^A-Jm2NTt-B{V{T#bf+K zrG?s1dD}ZOvTLBm&0K%@w}+GEBXa%&$)fOE7U8!CX7^EwIAnM_fA*9JoEsc@vr~Zd z(uvxWjdl?s`XwH&h_?xVl*1mAnQ{E2%nE`hzP@s-VTO+;z>!&JYwy%)c#ehLpHB(& zU4Yz@4G_~*M_=tBItg5?IIa2AvYXS9=XbqkN6|E!-9hQDo<2?2k%Nxl*V^-RQ)lw; zFCvZ%g;w?46y{!4GZ?_s$EkjM;{AHqVj_i^pz>q{=0Pt5`d_m+}@e5zy7JZYj z`^%oIaC>g(Va|l|o3&s~iRL?Jh>6NfB*({(205iE6 zND&&KyWH`-M{J)fE@IW=ivdD`5G>q!kF*)go}0!Wf@;smhpbZ` z&WqrxgjrZzaCQ}Aa+B%%szYD#^QiN%#b1}^TgwN?kkR3MrKOX538B!&rw$W8l$Vq! z{Ho}pgN^iI=6i6jKH3F;{s{5b3{QtYV=FH?ezm?^l<0DH5%QotXJ71JfILRcHezG1 zg^A`m)5ppVMGKR4ej@1%RS@gf{~|lxenVwN7DpfO5j$9h`=bdr0bQae@btTS%PDAw zU25FwV6e}FJv0nQDUjc!MIx>d+Nk$+Lgua$j+%YxjV;9OKxiob7!INvU-h zM)l@9ig_OgcQirFXtwe5r?d+)g`;x-;#C{v?{08$DK`j<9;?(%l!cH-KMK|RDWx{J zRIw$?bguWdxNanXVWXPV+<}e+I~uJsM=|; zo73y+6$kx_Ya_q1*7&}8#!k$Rppg=ZT`BrBUMiLoDH5d7pL}=Ajrj#|Y}%KWdakZ> z__A~EbTiFstA_p|slY*+v^y4}$z&g`vs3!K@MMkWXSnHV?xrUl{~nQp+#?>{25#1J z%dc03h@>aYT5MTRc7*0Aljt7jUHh&P0AZuE$?EBO2I*faE;a|WN6Xg`O)vzsmE+BC zy7oF*ni?;q_vzsUY4~{L1W(MX*=Fx_Oq1VqA+!6jvF^xc2rDxh*tcaYnutCOpU4|( z(|va!Q}u}f{txP+cTrKxuEDZ2tSc{C^YZgf;cgveH<3KnvRrxW!A(icYx$S;L~WD4 zWdKdvVWzMG%Y8P^o=nI{A3d3gN)C7mU*t%#fALL`EcoE4deMJ3_ddn_Y3Efr0(`L$ z5V-0$nORvH--@p=X*dhK&>v0^@QT;(-e(u8rHT)%uueBDL(E`{jti>X{^XHDnkjjO z;(n@SxWXkH^YmWm9M{m6NUe z4JVSRk{k_~f+wpkbX0I4$e}rCQ#hocL`yloOzU=@Yd)YeKNghlcEt0JY4wF^FA_Hh zUhT;32Iglv4bgst!W^yo>_TdJ$;8z3RedI>pnSAT)BZ8T0yNxDR43b3>k5A0q=}2_ z_Y|h9^D(xQo~%_TdImhRI>bHOGJaS>Jbl8QvPdI+;5doZ8I!b-)~!OhlDe+5pq*Nu z`;oOjJxJpL5oM30Fm2^D4xk;gs-zRw2$eI1bzJ3$ZL-{!&Y;+>j4{BPX&#pHeR7TKfO|vy;BPS%D1=|w2Y!lKYL+7^ zBEXoDmn6{{Yz}k~5J)WNs2lS<6a*uwJP7WP!8io(s#R-L{C&@k4%G`qNhH5Vb4xrI z2F((+)g|^L%x{9ygwPBxXT-U12NB-`oL2zdoJZhnQXR+yb0HDZGLDKm>6OZ+(n93t z6~9I7{8Luw=wx@x`=v}>*NOw5u9x63J_DmKa8X*jg(~=d+#;*y-tRwBPR{i(*y7gx zbDZbP&J5tsS!j*HFbGJJeP@5kRp=pk-2hd8&`gy3NeG3wpaBJrMJPSoznOI!HIOXX)dEB04@pR9nT9ji|Ck z=5p0gSY1hR@hNv`h9})VG6Bx>diU=xT#F6Fy_`P6=yj(FDkwB!2gZ45 zp0B~{QK zK5R&GtE>%wZLztpyY1C9<2*2j#Yz{bnxwK}mKERfxxX~UTgX-dRLV)ir%a(*t)!J-CSG8}}mJhct*S*&3P1{-vE z$3X=TGKP(#n=e8|WXbF6meK9CLQw+5qP#_Wq?7M+$A!1{8O2%FeMi=+vn>xlXMy|( zBJ-(~ce5gFt#a7*cjevXh}@pvJd&Nb zDs|6;dW12ISIfsC`NZ$_H7C5Nsb$2Sa!TdUls<`cSzA8YqE9=NBmC(#&wSVMOv)d+ zJe04Q950+T?DllVDV#O!_QWSqCOZ;n+N!CqSgBJpDGsEU>@ui`vPtjI@E1Guv%3U^ z#N@rA%?>tZEM=OC;lU5(8Fo;9$Kd(mPklK~(`dPoH_1w(JE;VzrqoCrPZZ1H<7t!q zN=haN{u+H4Gv#h%a5y$6L)YtWQIzJv*s0Fgz=d-;WZ8D74x2X13p$$S>VJcL=^mcz zavd|$oP*^_mWZBM8U3=Ady^At2GHGEVS8v>#|t&%8w2w%cUm;$4vd58Fd$su%5-#V z$`7}E(_}h##-Y+8I*Wvu>A5Q-ueU_#F8X*Wz<4J!pYMB(NN9H|y_IlP{X8DeSh3dJ7*{6}aBUJa)%)8Y_gTM5}9}zq&xu%_;_B zhYwts$Y_-nJWA?qj?n%Z3#|$CC%ks1K*_}plAaC zMWyhtz)U}dMHG|9KrDveZVX3cTA{)pEIV6qXBQ*vFPLbKuF?8e3-hjkdOeU zYFSR92DruDYMlP$Sc3_jC!&XZUu3vv&V6VSZPaJ~8Oz&5HD%{gG$zDZrOTo>Tz;k_ z@~ue>OF8(kx`r@453;q%JFMY5c!XKXE#5_+a)T*o&hbYahtC*f^ICHV6!U%wAb%>P zDQ$;ytg%W^FjVCTEKhHhVimbcS~fe=4mtJ9Zvx9Wme_-&gbN_alg7hXLqzp)Pun>! zsCj-;b+{S-L{(xhr*0b)(*V_Vl8PU*Fjq02S*-b$uEmGjO=#-sA-f2rg*(ua-?9Q`EE_5lT; zI|_Z-O1YO{8ntA~^S>x^kET$)pRwKT%$o5fpeSlIN5FJzR}9mXhOQ|-BWx;1ys~^? zYr3KOcPo3qGk6UnM*V~D2z^@WVHQK>QQ_vv7lFv*Cauh4^hcXMn6?gf5K?&tXsR`f zmWBXJNfB$tEOV1ApGUk+x}gzCS$7O;ngF{vkM6t#R8rSNiQ0XH?H{ku@D%2UxFV0I zLQ+jZ_kAO+`5(H8dMxE_5AbuZGTtjrS?Yvd^DrKs$9sxVF!(klwTytV;aIv5x_gB6 zENQ^a>j9VmZy?QW5ZHCObV~X|D8nG1b{hSkK3VK!ka*#4SJb zQ|8vV?p`0=dM`X?U*En%?ChhE6Rvppb~p9NJ^=@dQ>vF*&PCe`+p6|U54&ibb1f*T z^aWANH~m}*ZhttPxTe9u$vdxzRURG+<>%a{7pLbicGmgxxxmimibRdLSOby5mLtSx zI*bZ_a+;T&D8Ppy9y?9g(af<*8<6a_%2)O-Ur*m*zI4OeVxOt*V-z)3KA2Ei5H9v% z92G&M$xF|S8-p$eI%ua=-z6_@u!)hZ-Wup*_dql^yEFMZmnB0&knXe_tj*Ch> zF;Wf5yvQiJqf(<)(=JCf=!`EO^}{5;4?A+NZwBYmHyy(t!E(3vhU?TN)$8V!=N$+5jI zhsjz`ld|NzEadLF_LkViP9HIod^Dgi&c%n>?W&y@Oa*WP>UXKXDjXWzY3_(C>A_U?5AK0lquw6cv1KpJh8u-&fnH*^)4J zr%cln85yK>Ei^whKD@YauVGkysX6Iu+(3Dy0RPkX$?iW=R0SFfKdePISAN(?7AXKI zSNzq~@K_kxYfn6T9JiIUgE2OfbGw{gr%_~Q`lY+RZd4?iQTJaIp?dXDl*Y{9v@ShS z6p7Q}J=Lo~vSl7}0`Ct{g$+VPHZ5WL{&=OsvR#{n<0?Bz?P; z9E5lTvnAzv&y?OQkK`8Tdh}ZTr3iOj=y$wthLBDA{_rB>3Q*Kx$Y_n2vRyKRG7SQs zme%;<>o4#9?&bSNXvT+lnYPP6G%6@9sfE<01T#zo;VE>P1xt@F#WmEq?S4&wLoo^| zyt{LprL+03I~@7$lwoRSo^KaqrN)Gz0Ouo3Slr{P%C@NK9v~FAN|xUIj=zjz`RDdu zN+V6d$yK2=My@QxF=r zdUBN)41k|?mJ9BbSQeLpfN)=nvp)Ir`QzOYW-_H+UHEyjZEsy=>|qjtml-l#@w|*r z&MwHDA(c9(Gmh#85_H~XjP2tmrn8jG`ZQcZYdM3`+V!XKBVz!QPUS3UAWy%fNMd%T zf$=p5iWVkWF&^bINH#y=kxr~f(qHD;dY|qsT}_TbZ^`9zU++3Jm{9iW0ykq(Xh z30c$9+(V}pd}b|f<;jN)u#0o(Zo1oW+QOeoq`oiKxAXdl;7KfoE*8}OIcAla)NI2ulDgRL7Cip z<=8OYDbe1CT4~=)C>W@`S(iM0ac$RH_M~-oIK1ZDpW#(jDP}Mv1t{bwuC6GBZx?2B zbjG>tlBXD?lJnO$41)d?d_v5yVzLsnDF%yLv!^>4zKoFEgML3~avtL>ERTP(G_|y7 z)`LHu_FyH5%*iq4efCl*v$9N-E`DVCLwN($u zKvhjWf7NfHpDv(!oI`2Uz7weqttH*MlYE7a`FVxYe=NYW zu`EFWtnKW_OT}+dVW~uK3CTB!ExV6|hxzh_9?2e@!Rhyf-&luf~A-gi2T(-lpZ;*+#hAEZ_LkWV+*gqI5m<&9kvpEJPzF+t2&=`0J>`DY)T zeRbqvRD_;P!cY*EQ&@xIW8d!vz1zh2P3`2`PdB;gt)q<4{SD*0(cG9s5NZ}ioHiv9)EYlL67 z9S9F94kZ(;;Je%%u6N%wy6m#1)SdFnGcZ7{nkG5FWUv(flQ82El_(e(NQct~8ZZa8 z-@=c7dm>j{1w}p4E`}tFc&LtuatZfEr6S$mf)~glA+X(iJ6?z7)F0!zPxO8$Jnw@z zjUL60S*TTyus9oqCc7jw#f%O=(I4V5tuGy{ul&h7mAM#rreH_FWf*f33zHV}`rCA) znB66W7I);&pZtmTVD^Cs`FfMU`i1xG1`7d_PfKJpqSjrq>{$wgwAJhhHeKGg(e0B^ zwY*nvFQ*strMG)?XCTShADmBy*%LZ4aI}>pS1(iB{qjY3MyHvVA1qa^{gv`3DoZ(1 zDt&z>Ds_A1yx4tknvC5)QHXx*LY1Zat4Z*!)m%TeBq=$GhU;;hQR93UZEoObsQoIv z@+}%Xt9_W$yEG?8Z1&2qxDc^KhmgaozCZ73kr;J#m*-sV?Jn*592q&4AmH)rhbRUuW~G;IC$7yI6^g| zJv`eLEAHqpAOdf%5?n%w%brUy;m zdY5cEkyOV=FMu60NPFN)%IomCj)_LhuEAl~Z z6|>B6tZ0g+w96v)djx&S%eF>_09BOT)Bdmf)a1QmU z7amR||7_`ZA9QRL33iD7(W4{lw*5LjHS=@7*?g(l`2!X-H)dp~U++A8KL!)^&bUW7 z;9XLb#E%)y5nI!Gry|Rcqp51mT@RO%i)NFtZSkiXqOtdSx)ou?xmgpTM zVt=L|sNL#wuius-`|H5r$B|cL^@}W0hZj>|64 zQP|iYL9RWJht#Z*l=6yh{@OD#F3&koL&x)&hg)E#WjuB&KT&GPMq{x0uqULkcbbZm5`i!@C;uNfNHcp9<1#<5vcCJTn)1wF>eb=yAn+Hs8092**s-@<_>9ub@8S<>JZn1Ci;Ae! z#Mp_%UEZk_;9xdFD#u7pY@0|N?&-byEPQlCn~STo(WGQsT}>>Ml15AzM`T_|6BB{S z24;i0y(3oj&JtU_(6~8EwaP=2&;K7^?;Y1<*1Qi(z$l8MpeUliN)r`9svuQYl&&JZ zx>7_+lpcCuExUsB8k&mqwp8gsS2_vOLPwVplL%-+zyw0xbEBxczwh(@;|D2s?sLw} zoSAE`nPJ}RZQH|-W=DD&9@G=fGmS+PaAa2 z#-|-rihb)UGkl_%R!1EF0MMOT&-Ksk+{O38O$ArVAMUZ1KDEr4UpYNq@q#Xp1bG?L z7LBC|O3&BnSuEs?ViUYru_V1lVNR*3L=?e5zI|+aqcoIBR&wu8Nk)_dep_WX7*RCz zGAJ(9tYK^@06vx@$RCu7yZ3_Y&*EgY9)*%-gEIQfCxO@*Xjoa(pC{-^EXFG+NlGg= zz;f5tt+n1Yd|j+iu%=a(9V=y(Ru=qvu&KeYY-9AmhruQFiBtAf4Ov=Q2JGwTMd?XT zp3}ay4U;A3$G_Bh5wI0L9Dv9kf5ranO-v90iQmu6^RKqEsQm+iGGp+ql^QZfykME(3-9HcW zig*pQGKS^^GO*~ew6PTX-zC&P|M7w-CE2v%15q%csn~PZlk0DknarPQ0m23b{xJ<| zU*MHL4d$s=b5VOhQv^uH*~zh(7Qp!qeHSovkVkWz9K}CGBMpq#BFdb7=Djj~#2%A0<)_dxE|~WT1s*LmyjBJOf4-cLK~hZ0`ow8diEpH=zzh#2sUH92j8$Q(9IzO6RyBpZL}_#&n>*iIN8Yv`6l$+_Z2OTT?{%AN6a!0Zq-LLE|;< zlS8J#Atd80O9hROJ(txfv27-3m{vfrod3YK=AcR%WY85HS*=3rS`r=ez%jU~qfrMY zOqrA-dtFN{q(O_q)s|2lVH4O(@v}RT24Y8Kf4d7}`R1FQt;K+)Nt=VXmaVC0*WDPV ztX!oKH*dTFBBdZHu@&c6*KJd1;hO7ptjKxgh_$U9B7wTN)a4{Gk!qPas~kk5u?z19fwH`+gjGOEl(e>#DrX zq>Bd68L^g)x}SsbffTSGEwd{9r#hbX5JQ{B1`gb=oD!c`bsx(mHs+Tl6FQE@2ZWT- zA!bq3<5K@Or@(?5Kdc5C&S>wDUEAJ%%)F5_ed$d9ddqDu1MBL4o~9zdOrA7ZI4B&y z5jV#Js2O}0G(|S?@}D^7kX_qkZKSZH=3A~{wceO~D@Q*u2fV?6PAjee3x{_DK4pQE zwvUz4Sr0?MJd7H53bURGXHQLj`M22Kb!(<{#H0QPvrhXgr?!C$jsIK<>MB&e)eWgy z9NnhDHxY8`=i-tZ9ZZFgx@g9p$g)M5)_n!8O43<}8b;ozXvw6O<_@-~Xgnc~C?uj0 z2N7L%MpIBIt5DQJGATQ0K%YD)S5AZI<<9nwN{AF3#BGTxq6Lh%8D~-w;RJXD;bls4O|ZB?@@ zxSmc6Sy~N|7Am@4YKK0y+){{PldDUr7vj=9;`upeX-8A(d!(y=i>rIt{5lW;X=wKI zdBp4;OM;VxCzKeyiE#s!~MgXNWV5ms3bOSK#{7w2s{hFy6Sb>vi;cp7J&m!%Q zAP^rE{LiBfKoC^*pCVa{K5F#5P)1(!nhL<3^kCX);YYlpT*uC>=D6XjP9sn*T}$&~ zU;O(jEn)F!c%;&va#o@pJ<;Q>t2&D!&8nE)kkehu4RCBn^7T4jpTZ^wzJ^$w$ zCDX|jTD{Y(N}o-;1PaM})$YG=Bok%{`qgeq=u zF54iL33l->U52yv$n8^e4}HfjG-96Gc!==f3*Lv1>E+%*)<^J{vOiYLic((&gBjzgdd?Os|kCb>s@K7Sq4_PRSWUwhf)?a z7v6!n7{sKi;Oh+4ik1T&>kXh4WJk(ewk+&Px z4X0KHTvtKS{hiU4(9p6s0$2#@oOvUlmA2-TPk-zUxW&KFT%GHov1;N*O1iH z)61ykV=hc@xOY-7!!?x-CLpAZK3NV_%)YRJ##xE4<>rRO?NNtnIBy;AWZHZ8#5W7M zO*c3FMX7tDK7Tga#d`RZ9ARR5y>~hRGV4)Aq8Lfkk*M9=4&vJ=nD(?#Xq+#8wnRci zOVEH!7+>>RY$m8JNpc?T8E-2@`5KIiN(ZZD#Ty#1MHE~tz07%_V@ZYo@65)V*N1xr zj_@2!mWAl_nF#i}kB-vg<_?SQ2CF#8k8ngH>n1=Q+LDD+>C@29heF?)FQUP$)wXr% zV4_|Wy|d>pea%a`YcHg%hELv!+`$DL zECOtBo*AX?*8{fiTM6)bSIqJla-VKtj<@(om<6xY=ayKy$jYt!=sNvXs=qk_Th){2 zK%=DDZQ-GygN2!J#6#If=OhRCU!I*(Gu2Zv$ZmpdR`j)m#a0FNjk8r?wfg~SXrZka z8n{-*Df*#w#Pupo4zzVaEqq8iTZ(G|bWUlr`u;>XUzV0L&c97gw~Wn;Gb#U2oU5(c z%V*74Cqm#J%CSp*9bh2zw0RR(dbj3a5ScQNxl-i`#*2ExNx&L3++#X8G1KFl;Q90D zIk~%vWK(1KT_+h2g|hP*>UWE*CJU+BX}PV>_}`%sH*o@od43(5EZsl?zIh^x*0N!y z)-an=)~Oov8G2m|u9{WqQ{r0Es~QuJdB85@#ax1%foaAx@MVajtThjFs}xr&M(+#= z&9GsFv45~`@SEb>POx9%eY9MW+*IuF&U@;3{I*z*a0eO2`KixO;t0_kakJy^&I_1@ zI9zph^c`}5%M;RV5=U=|ff@yZV(Q(j0fpHc}fo2Epa@zVHmU%Q?* z!A;Q)hro}hx!IEKb@4bMF#2+z4?k-hZv+b!8j|k8f{Lrkgb6S_n~q78MZh`j=Kc_ zmyH6djT&d^zaTk1I|Ay?%#Dh$J?1RzRFG;{p>js9fe;Y=8+RVpiI&zq9(VnlRqlZs zs+RX$a$qkg)|^s4^xHDvPuJ9s6%ws)d0EST2pI~$C|3|zD)rEbI}{igi^b%o@H^ZH z>r`m2-zl#h5*o`-xsRpiuTMahMquisg{%00hG%q_+BvNV5dW%m83PQ&&$hmAc82+7X$Yr13~s zb({rKyeOGTEi|jJ%r*0eOTz}6+1Ig@GUQ2R@x~P*bDb?TC;tUt*^7YMWW{Gr zXd>qbb%N93uEF){fA||MGL%5?yf%)2V6t0n6_EFsjlwR&bLXt+ax!PyEASh#=I0{pD^;f>(-HAFZ#7!|Wrw>q@SI^h zE(qQ>IV-Lt$?x_OL`R;0q>&i3J9y<@ud8k41g+FhioXZ#gT6lH5SJa7>quYul)D~; z4mA!bLtMm#4LPfkEey~-2DXg5@PAec@4`LigFaL@nuWl{2o}FYWKes@p+HX;9{yyk z2+Z*{%ZSTTQYa=E7$~<_+9-(KexagK>arT~w!@GbayRe%WP!7YXK*j=bOfkcr`0)& z%~`IF1M@Z)fN!wX=^KV z#|Ly(YRpuNr8m~u_dW5H?olIv0(9R_E1N~&FebC8p&Ld7cg{||unVO~CVVu@XV_vc zdgDX1Zn$94TKBy&tXUpV1k582z2)fX3;a50(!xeS57_K7y){O%U#`-zfh;; zt~Dh)vZOqtP=++nDfbK~^0=9f2MxJ}r!?Rknqbd8qag-S^PPsC*->fSlT;5~<=RqU zFOsZ7d1n);a*~MAPAlCCRc4;c`0NW015_7hM`yJP`G#@$Gm2By9ot;0brLS1t2vL} zU{bA)R4m5&<#`2@4cC6^$Q-7b3e$r9T8|PLB1+zPbLQWUKIU(Z+#~X`f0>R!BYy$= z^(kiVj9*=wR>)-Tia^+yjk8=9f7NJJFkDJ;&JJ9RXCTjG;29RJE&GxPabW=cDwQt2 z$n~zfcDuwo;fT&Oco^;E45$nDF9`ASCyiElY=p(2>I}N;k(@a$N8Z(kvNje^-lDIA z^Bp!{JOQUWeX?v$Y`C}MqLY)o`4gk&g}MfvGDI+TTXU|~L$jB!4(c}l($Wyzg5!vt zjpF5Zz$2PYcaMw8K3aZusOh#7KF5~*cXaa}H30&eaw~}#&{bRSY}fR}ESR~t4mFCQ z)Zv0l8$k)%DNc1AWq$v4z}3lV==xiI2|x|BrbUPB*#L&!$xOB;PM%JCf^De0~h$Q{zxCStlf4P~3~2X9^?kcUWB* z9wxm?=kf>zY41T~k>O%>-LTEes?{5d2BW1eF(JW4BxZU2M^HUhhV{6u(#q*VC>J1% z^?7IL#k?zbj0|XMT(BQ>TsbMt-XeEb4blCfl=t<7aRXI=-UvS1mgny}$L`iv7Jq&y zK#@xS%0JZjZo3%5DE~|&DCo1zSkNcfrV?Iadtx*KGNPB%?@ksUilxf#ZGyA#`00+I zReta8Bq=e(??jU}DCV#5Tbio-Fnhd{ksg#1E^zX&tnXQ;@LUv0k}HGJm;>j zf6X3j+2)J8&F{&>=&w58CdgY$f~g2z6<<47rzoSOtZEXtZr>?1Z=O?}4jHdwFDGPRZ=Ls)Ul)`OeX#f=Sj8V`(yxkCjlWU@aoYe9|?$(;pjX{m^V-7tI@OOi-SH~E>kLX_;jbaiP?7t6jB)9h1>f%?PWf4ZobdxbDNHiH&RFTQf}^8DILAr z5oycDeRe_VC8Fb>Zcw9jT~^q8gjPg;vk17Y8)=#m8q5I=Ddg+RK9h^fR@02(#o0of zTYRf;k*w0$_!$DG!Q3E%;mpAs@;eWUPh4b3`$+ zh!&3F|P4uqIH1<`r`G4Eu@CLqnV(Kr4)om|)B zs0-hgG!|;W-XSY2zosXw3;%Nx^m>uv?-)nKSXYm@nQ@mNUd{f@l~X+Y^n8Y*v#_UM z$nz{Z9|iQY7!r&4=b@Ac;U=7^4&j5N&CNC>8Wj6vPNr&rG2$v&1{$&>XAl!t#?z9x z4QFJ=rddM_K7UDTJY=Gl5V1KTj;)Ll0 z%N||<{RO-_YxN1}UfC4+2%*4WFB;Tes*PEu;T3F^v&e+k5RUYbM}}7u5dAm?ZN0aL zF!AUXJP*JTEHCAyMyhU0?cA)CVi^Dqb;I9yPR+Z^q}n2md1gAx_yjjthmw#pDSjGX zu8NQ$NYsR`EU0|h)KM@c_5NBbs3ZLxw`K(YB$VI(fHha2q#hIK0MB<08F@vi$;tJ( zOe^`+vf|oe%40o7c6THO)(Otr9(f#K;~5-ftQHzatpz2~f$)VV&jmTv^OMldkrO)t zo0Ol{d-$m!47+Nb=H0JNHj)U4jd{dMkh)%sgU_U@m&2#P%-FarXKbz(!$R@I+~^tn zBq;!r)S}9w+*-HM>paD=s144pLPW!zxnG>GYcEzO*AKohb&p-sZfTBp4&%szYN9_j z>MVgf%>#n#2RZ$(T=jZl9B#uq>$x)OTz0ZSr7}~vo8&u?hW)%$y=04sCRT@(ty@j} z!0wf4HxQJ$)W686;E3EK&|yA3tP06`U3;tM8xpY8Ju!oMrp|K4vKHG`)I|BzYFm9h^5I& zt&IS`Byn73GN)O=ooLS~;JYWLvnG9-~iEg3?sFMV-*ByLz3V=!oQMGglj~?nVh>z#IM^ zdL5~{Hpgw=3{99r+VDbhGNzHCc4}?u;OyJ7V%zTiHjZ2KDzVJDkW^fwmL$ryPMxr4 zFLG~w^8V<6DfxNc!DLJn$Bw_9(|tPV7Sb(&LPwYRs)|>u^3+Ni0Z-*4$Lw%-0hW-K zikV8+8(Pl|w1D|hBCCLZ=n0ly$t7TJC8acgtTf0pp?r*?bL{5wii8^KBy$A3$jWiQ z*MU6$!Q25kmh4K?k;oXVGb5yT2d&r{K!{eS)df$rvUz+SPHlC~Ag9w}F~SYlmOwPJEl8c{lxJ$$KIW7053$bj)9=rv`14M-&J&c51dzj7XN7OrVtXfYgJP%Jj$SsvtQee+~^q zr1uQ=)92qq{o>|c3vJw`kF=|s%D(gJ)fMUL5HOd0bQYNEIlB!<8^ni^D+hVb#iEji z+^OP8Tzn<-QNO9M2U2q*gQ))Dw`8|TYh3^^KUtR|_eq`oiuAod^th@UxV*cCS-6mu zN-_4kpugg3^7%=k;YD{es+iw99g&0`SCKs-&5ky93g_gxn^3F0KA^!g291T)jb$-3 zzzSQu>S$)x74`~hq*#Q-k9YvI5kfA4ZXuhcG^T%VzSH{1iDx86{@FjnHu|t4*+&)A z51eO-_O7rfwe=JK#B_$xkn^Gq=NDZ8N~+d6r}h;(l6p~{TrllZ z+r0Ra;1#R_Sk`)sfG0+z^soVI0Wa>Z@zDOzC>ts2*C3aD6O+P?c^48rFKdm%PP7^^m7I_jUxU5T?p~45Vtno$^GL((&XX=o3JL z7%yH+I@$NSB8@WjsOjl*E_RDT$dJw^5v_faMUpQ*s$QJsuW+F|Q3WY64->3wm8`Hi zwPia(2|)&!xB28)rFXMOQf887&>M-Gk=Cq!f~PfgO*{P2gHgY!ok{kM3CZ?PmZPD@ z*J8o$eJjdFlq57qviP~Bv(ZR>ox7J|aa8f<&AbXef`gqv*T63?sZ6nnhE}N&};QlJ2tZF*NBn;5=2462{7P45cTBjwVPg#e{MH|D*$ehFJn2Csz#7 z!m;!8voJ;8+Zmd0FGtN=#;mi>~h{HSIW*=4d-!JnrE#;KXE7NIq>4vOiYIc#&fo(nbqOm?*) zTdL`zqdp!*A7zc^_d48pAJR9jjPL24;D)5cGvhpHqN6wUDpok0!+O>>8L=+3BJeQ~ zf0q^qz3z2q`(XG8!Tm@P{_xTScj%Yb*x=&kWRA!vlBK0|pd6J*R#xXc-62rJgGnJ< zge&o;#H?w;>z%J6!*zJ_V{Rp{&4Z#oV;=cf`>46Oxj5lGu5g@!OvHekM!pA-^XhYm zT{@a>;S#*RCTGWKjjvc{q85>c0<^jz*HHeRmeDou+KY)J3XW|Ohz`;jyCrAX zE{b(x;8Z~ELUjn^ZuoAf`A>yo_q*Dgw3E)3bk3u_)xV&)kaGR5w$!vZFDGl7PK!~f z4y)^42TeNIL#|z+RfVXee0kV-Dcy<_+f7|@$;%k#yn;o4;W?1xIw!Q=DwJToRO?1O zil-e2_MaS-)WUHF=!WTK2J>9;^uI62ORS9F0KvdX&clGP;$jgJ+7VpjlKM5jzlGlm z%jCfdx2BOmiGEZa)c#-Tq3$JTBh4r%))j{FA5KN3gLLOj_>qmVGNxsx6pGVviav(Re0vq=UP7^Z#K4*TC5aGGnNdhyOAzWCaW_fYrW z&g$p3rcmHCYw7-!Dyk6~9q6zPTC2|qksInANEJ~_<<2KFQcp3*3FqIEl2^49Qh zFI6AL@4Gl*K~dFkrMX@V=24CFlEP`du90Xzj*FyE-agV|CH789Yn|>^=rynuX+ZA$ zx#|(K%<(bxXM`7Y#We$s{#HjA-w-fG(HPWBURFpzC;AM)wYra;xuu4v?@(gqMmD;? z6WVw$)CF}oh-16V?~v@s|1wP}&f!x~^Ux}&wZBCNNg;FD4`&eZ9V&Du;_LHDTJ~)m zvY(cqWHTt*_v-4lU8#seWZ~-2h;A$$jj*w`v?Qgk@|&kA{Q>o%6q{De@=_;+PQ3y$ z+164Qa8$G)aB8n?{zu^2WzIoqm>nff^z_{?g?i|0>9Hh2$zEFFtUGs|_ApgR6>$+J zu&-mBL=jJ&$nwgGk;vXIy3+Rb9&%>DmLu zOxW9}OwEb})+9=W9JVR>0O7sGb+uID<);Rl-n6ORN{FZ3m)Z4Gbh#AnCT2|dkY;%eg>OOBr7`D+} zv=gntxA@8RQ~0i&G7WmC(HY_@b7v;tp(ka5hD}V+0TJj>wNS!@ir1Yfjfd}jSZ2!34(rT3zI zlAJbYIv>u%Kt<0Lx7wdjz$w=*d3f##1>_>KRBUkKY0*ADwuLQ@(*NtYVpKtJmr8h^ zx46j?wvU6YfKi7ib(mQb$)p>KmnZ@)p9R$Ll-$@%dA@z-{wT;Lj)K?L^|~LSIzk1@ zK%Z?vnu1To)b=%jT#FB( zuLDZ@L~RGKafMJLJ!uG3VU#+5WyJj00~r-IvG1+Jz?NiZ0i_91N!p(x#HO${!l&nV z%)v7MH*tu>(jAYiscHE1KNtlZY_Lj+yP(aQLdY#9Xzw4){=|9dZo%8-?So;M!R=3! z=TkC$M*g~Wxf6YbBXU`^;wKSggT3Bq{-3c^kvHBQ7m+f!DZ+D=>zfdO&8Hvy(lou+ zgtc$wh{-va+%PfMbrCpr`nr%niF^X)&ELRQ*2AaBYVP@*4iyNNOehRrKV03_>)~5z za!}*_@4Q`5b8iZWC;qbj_}d|?>Yg9Z|NcSWzW{W|9^c*Dqk2|8UpxQ9NAr@5QF;uB zj>=+)ef}V2X$G-cu#V?%q+#kgiW@ypP-pfzkS6I(TcFfDgEFD66h}zL0GVwn<8UELI-4y4b(xKqsNprh#v_rt20UzWZeMD0DX7K$0Jaz;A1yICmBo7a!ew zRA*Z!g+k$2xpU~NdL@SOZ+_ZY6bmppB_XKbh^r68>N{&E{^LB@V0>GH_J+k`FtIs4 z?hcM$55|5Bw(@O&WwEhlUx!L-v0)g7v7P4xWQ;376ds48CEG(If(N(ve0vJ@%qOTJ za1h9F+73iQKGht^IIpAyeFJ?k$9LV9`|ah@0*L&lMZw*^}OiG zU+p)xSN%&}1P;FZ=U~~(w+8N)!!W;Z|M=u*2>{!9yCsb8+t7v0%?-n>L$-`{>)ayW ziOuP<-(PNLr1iU9W-l+?N)6XOVj&*uBJ}kyzCZB?*tPYZ(DN~Ae7lZVTs;20f5X@J z>uJDz#Ip*`n(o{Z+a4uf(ed4ueO==oSjCS|-hAFyzHu*jz0t;}JU{-? zQl5l0{Y%V&&@LS6yYQdS`If492Bj(zL`s0muqDZ99J5=8vGu+G`S*}Q1b{~n*+kIa zltU&OM*1II=WjavL&U$H9mtRXN7JD3X1KPvp~piGK4alSFM&Zxp4`#@+pgORs6Ct` zkG`wk*DwAbWd`5=_gfl6zMDe8dGYddFwO&`;F2A!b~Td_3v;+_ABKHoU)wxm}_8mTLzvt(_>4Aog(v?l`sJOyNa#fn)mkLke2}RxdTbB6i=Alw~ zu5Q}!Eop#MO##NaPv_kg^>~mA^(uD{`S*FD_<-Uod*2NP7qv1Q;z(+Gwc1Z7Qf`Ua z>JGbQ|2EaH&&VE7=tRv$B~c>G;p)hr*TUgsrrG=85ZIB%M1`$uX!FSJYd8kKfso(d z@PBV^8yI2j5c%@YpL|q%HQ4Z$6SO5l|4}&T+ut?INyq*-EfMI#e&JiRdrNyUFSjRr zyQa1t^1sCXn|mgMhw<({lIkj!_^tGE7wqTbfwuW4_UP8y{-sKMdf-C*?HHm2yuYRy zMq=w_zU|ysqxLQg(;U2}+rK#3(7nj+(*ESw3rY9M`syo>Z(h~dP?j!~By^*U+gtc{ z&YFw+L?f6Zegqox&4)E*?Bp$5=EqDSdJnxn+}Ms65(xXx9991s3>gecMVIIPiorDQ z%>Tb%09Xqx82?AKR?B27W{B-ogYUvg9Y9Ikb8?)rvU4yx6#Mv27d_rfLUKSkFRsES z@W2>8?r2ud(aM~~&)wduU!0J}cd{Pv(RgmFY)n!`vXt#S7eyUaf;A+^X(x@L>FnCl z*}wjK^JFZ6oqbL_YT%;#zfo*V?snNnN)Z8aFO>jKaT&dyW3Z&jwp%)edV+A{I<{;P z>9mi&Y!n5ssQ8@4d!62t_*}b-3Mqc5^vlFG9p&13f(0YAzp>VcDRIKy?1u^3R2?8O zPL20L$g{q!0GbW7WsEp>_B;%;{H`+k+QJe3MG<~-8mVgIyHOD^WkO*@hn&AEaz`>l z*${OOzZObqt21)s&EGW}q{lMS-KWC{z3y{t;HM`-G6&p&+iYMaIPen3_YVpF^nu#H zWOM!WU#Ba=5#D6>H+bVF_sdd=8t3o5h&NOZHgeI84qV$fF7-f*`Nar)(PBvI2lG>3>=f_f;0(cZwr|ul)5C3OgjvBEN-`cD*snyCaoDL3WC1FA0j9-6^ zNV2Z+ksCF0RACwGW90UhKo4&XuV!y7dSKNX7A2daN*I!P;rz)Ti%+j@;8vB}QyJwm zyep^0?-*EFwQC8s^Q>;y=R#X{Pt|zOAaZ;2`IdT7FL%BVf>e?-Hp-)`E^Z?$!K(o6 z>9NOChh@~y+M#3dqma+O5?1p~r-x4gd(-sW@VEcwN@v5rof^tumqVWg{IlaW%jEoC3oy4>~mleydCE+ry!MVN4{1vMK{gOfAP6!Bc( zuPU88wCnFv_^_Ld(A6U`Y#aO^vl4D4-AeYpS&+N4*iM*mU1@Hj3pu>!IRA~b4UH=u zp5IPK^AhH}ONJQ%N*|dqvNXb*s2rF$puVx8euY3<0EgwEIq%Z+|m(8}pvBe$fQ! z-lCxy?*5|ps&ywlK7oOCHcDBtY*%m3$qMz0@s}{4)d|#vKalA4>mCZpHs;SdZx}^6M<3BWN&O8m?=20v_UK?hc+|T#p~aCB=J^SYc79EJT7NeOzQCc zc|%QMf(cmKoOL%cneC^h|XDe zArljb%jBH8JAFq@6l?x^5^P+}&CR0`Y((74qja@VM|Ub``vb5juk(_GfKr9{Qo`va zX~8;dTV6pXSQhM3n)j@MvHi$KmdgA(7Mba)IWGoItS2_C@MoP1)7%Z#RP|2yNiFAA zcPgMweYPkreVs0$kJN{29$og;0i-k3y607s*&!e0?A*Pu$vNrg7Z#3D@fAm-kLE3Y z?pk-Uibc85mYpR^6MS>uc$hKIAyy_j!^(5gqw8!%-D~v6g$`VqHq;GW%I%RjX8qa- zHnYD&IVmI~2N6Adu7-&F%l)J>{==f?XO2Ha6-p$cF{gJ9?ubzu0?Wf9aXVg6eT&RR z=Pa+T7Ot>7%FTn1Q8}D0*{5t|QC0EVn4zXm#-0sdZ1|T80Bp(1?qYY)q`z9+^M{oM zEJ4i;nXH7?6FU4KlW{rgt?J$N1^0XLIR?ADaZy3@K{{P^@zfKN?CSNBWH(Pg^mLGx z?jxVobF*^P*NSTFeQ_!gSB<8P1y@&?uGitYL&ko>F}S&Y@d(B}4^^SP=@OG+Pt(u8 z&U6J!Hz_;#+QYRe0!AFyArdAN=n;Ak*HUkp8coiOGC$gK2Q5mv%Y6`>AvrUH4lnB0 z-z`}tiw3Vy~rqf0XD3ycVdeXwsN79?Sp6rZ`GPu-Q?zl8<3?4Grd#2u(xkslJn z$E33iy7_)7vtt>p%_KR?Gc}I%H*#_fu&7*by;)X%R_@>`c)(G=`ZP$~J%69TPo9^i z{dL&NMSBi_!)b@*gCLi&`?Qo)iRDd4N5`SYEiVsqj)U@WHHYg}r_evt)P%;Idmp{5 z9Z&kLY?pmkg!aswOYas+k9TTWp_;v3et%BFvEFXC>^u1axep(gBiNtH`AdVHliev* zV8@@d|B)s(o@+z*+eW8(g@`#eg%kv4e6ST^l(>$P5JEz^Eq=ZIgm_Q4IF)vy&JISX z)SS7g@sq!*2Km#c6(2%IAWz857#p9HE=R=aC`Z>5Oj>XO)4r^wJfHU!=iV&7O5dIL zPu5Jl*XLK(#vJp;!uX8E>K9Yts@?N-!dWy&_e`NdmY3QSEA*a#YJ-pl4o^Ws5H`4^ zz!BMI5PI8E9Dr<}Idi7xl;u~agT$i{2;CnJV!7$&JmlP8fF(adTG0V0RiV~#PGKEO z8CdUE$I*ms;MRc-Ax0eK@~TJI`U&4ut%&~X790`R(xoLuLhl-O374;*Rgt9kw{7pw zdPp3fq{>H7!JbJ*_{e^ecbQQr7eiIUl=D2n_Gp@88*C$96U1AA9y*u|hLKtvxmHEbc5t zaZAw{s>*7uH9M(SH`F{~!w%7agjI^hI1gGBM%P202JkPv#Ih^C8p|_~v3zs$x(kyG7^p+DW;3!d_9Wh7t$rewv()#mu(3Xxy=Nx^$h)iuo z?$G-bkD2B0obuUme=ewp_H23z(1jF!DVPp;TwXkmztvyXG}$~=r>@~*^oQTXn)@w& zRPU}?RlP1Xae^(w7yBk;Zr<8(+K^N`bzF+PLJb-o&v12$B-1P9%EUPorDMEQ@v+ty zNiTfL_-X4kJ$ciFRP`*H;v11Oiv%BjsylCof-oVl$;N$&MO#Ueka0&f1+E%4?D~pU z$w_59-<&mpv?6AfPFGn|aALGej*v zx?Pp2+%6*7eml0eD!7KjQ++<*T9c{#kPQC!oJ+^dn=_lI{tD{PHmN>g_{is6+TWodihmLiI3CdSVTw$(KzL=& z&0NDN8D*Eil&1q}tO*G;L&r+0>hWgGq5mt!@FxTO0g9|;zG#JMT5bRK8~bkOX>L5q zl>fj@8658u{1K{RR<#Umv*CAf6LS2+2$!9v(C<{!KBz3|}i} z1kiorxbkJM()o1vvDIg^rFyMJ^4WYs`zc>q5S4>Z3)I^#qXMi3%sdgr1RObN@w34~ zCpD-Pjnq!V*AytBqITu52rRRq+NB{o5;+yPahT(~Z=e5{H}9qoqOi6JH-%$tf!_m3 z^SdZ6#qMou>-(DyU4m!Maxd`*#j)LnXD`RaX&9o?{rEZhmAN186z;^7r%&t56yqBc zjy+6~AF3U1r!9$R49`|K|1wFwIH37f9%SYiTDjwC{mop$3qD5C2BlB`x>JPmtu_`W zxM3&LUmHm2vjmq_$Ezk)2`{qpCTWU)pRnCrp^k8{a7)WPf9=@fGJ4`v5t6Esu@>C4 zNMs>1%Lr(R@+&Yp3*MB;pT_7tK6;ycf%Eq1e=QD#@V}rW;Zgee_kA}9yL}aE%*1Yo za(y^nVUMETj4Vc+-JR0D!ll|NG!-wsKV>^AJ({0uSXpC@BW8Bpu#29l^;_lD)whE; zgI_q@)jvwI@>#1>w@p5)frHg4I`lbQZ#qWYLFA^bEA|jnl2;U}UjPjrsInGBA%XXh z-kdRbgRs1`d_ULGy<5+U?A}!w7)hefbDn;t8DjTAbw;qds*ql1hF$Y{R=^_E=d;RG z2A%k8!%~}-wvu1fGgWWjx|PX`ni2PsCBmDOv;Y4E`Fm; zfwl755ey_g?#vLV@;v7lqXX}jS13JSUL}W_?-*itOB+sBg0Wq4(-mfaB`Yk-CQ8%# zWis5Wnb0^L{O-s|uhs83k5HU*Qxm^=VHVz`r}mOC(8XNW3yh25;?&Wkm3c-I*}6$> zkqG*IxbK4;Cl*%O7Gcb|Fut@(q|$u-R8<@B7CpQtG6<6oFtNzsy&bd?+tp>3^`T>n z%KNYO{=u1JH<)#h@4fQP-!Q&EH5UO2*|hyM7}$Ih80gbIPlvB=u29ATwSMiKB>S)R zlI&BYf`(#7s>UOohFuY#kKDl3>Em&|JrWv{?8o9f4UK9fqCydsjSv0XHL_>{^V~#u zELa2%tqH%+wM($G+IecMI+l@}r9@lxurn&})$Xd^gF^Cru;ulf!jm&L&Mx62Y?-R= zKxV@Solwh5TyUH{G2lEkaJq4IDF9thbunXv_^D@|eRMe&xUIvx+%)bv8aQRLlY(7( zbuT*TdnSUFP8Y5XvWkx6auy%#sJy`RLoGi?l3A`+;y-1VP&&It4sA;A2QcA1Cpj|c zYusXg0=543q+}=;R_L$z!ia^+V)TQPeY)uVvD+&@jgHQi}1~#Ve+>-RW@?{rqh*O#&xID`TB-`#(K_Zah>J#{&4Rjy;eP@9!;z~-*Ex> zD+#;fn)cP2jL=M8Q$kYzU>S*=@-2Fv*U~)p!|~TQ0%mvl;{wQWUE=Mtg?XKh(a@Ke ziyi>J_ceJJVoJ%2OjBP$J5l(I`VJCd@&uN!&ZYzsRBiHrL#H7AfW{+h@6=!)tMZu6 z2Rs9#yEovb!|3GZSgjP=Ix@p3%Lq-P;it*R=MR|(<0F ziWKhR4O+@_j@2#hnHbr{vD2HQ->x5_BKI&PXGw`xcJVQQO~Ex&#qlxu8;xD!WUq<_tGgG9Gn)!;2AFc>T0~E<`lU%eS6D9f&ck^0N)XAS>|&|Ai<#QA zD(cdKD|T11@@v$B8w!?g;8W*_s(G`>7nbc+XW8wGkRVZc_;ifxuWBicQ!Vgh*--@& z4FFo_#jSP+ipa2FL?xoeP}1OvXL@y`-kWBOHgK@!Mw&^1Fqe*yn&G+;b#;D9ISC+y;tw@-?B0axwvmqv>5w?c>r} z5rtVABymn*qEsT;AlL=o?vP!~<^){3$H)8RuZX(iXrk=0a%yD-1snYP%-r;=cj3<0A(SpxeoAtHIhLkH;g<(1)(o9LrhQ0*a%B{R&(X^OceL#p8Rv_28Dc9YS0U-`vssv5H(P3hD?ff>oH#E4Xu zWp+uQ_vKaLU=g-~L-Pr7fsTyp3W(u6Gznb;x~>d?IDhvne>RaJmaaYz(6G`!Sn?z60?S9O?7Gd5db2;DCFR2-1*V~MI<9?S!E_+BBSOWWRrVlAoYiy_V=HcGwll<*6L5V)I{%JkSFh zuko1|Q|S>eTOA+D1noN~9JBH1W$lu*LBtyplYk+~F{Ojm@y&O{X!ODdF5m+7l)4gH zP#U|D5Mt7cUhyWy2bhp!1M)UD_oJBa1Bbp)b=66^V3*8AS^=V(jkq?11Uu!XZ@+f5 z$vtQM)iP^=IVIa~^A>>P!Q1 zc3|Sy&3HH1wy*${q~w4>Am54poZ|>{)FnFvvSV8uSMgZAk89_)?n`odV3Btk_M9%b zlOkSjX&>lbZk{x+o{2;1(A2Y9rN{S(qmy6fWy*n-$AuV|=rT;o2y{|Fe2PbXZC-Q1 zs^|(rh-2QyJ}^3vS-ku?q?yQ~&5cjakVQ)}@a3O(^)SY@`&m>~S=;R|c+IU*3)O@O z)kC6ErH_)nCyw;>Fd<(JHHyv$ayHZLMhl!hQJNcf6WHA8>1f@e6Cg*FqajXX%=Evv zkKH|eqW`x&1|l*csDN!c1A=mx-(tfrQ}kmL&7`7Z{+1;3Ety zk!gk8AqCP(MqwCBRAW?jrd3NGIQl!y1%d{&j2popA7UCgXKh0%#K7Sj4F!l23m0#G zfs+Nc?YYiLdk(sZz&KQ3v^AHFEZYI9kMkTSYXmgqC{)X3Ar8scbp6eR>fJd@2h{Om zOKoPem5o!{H)|}D6E?JU5p(cCzJK_!yS-vEdr<42PqNt0hjw%k8oBPE9}LZ|xEMV? z855GCMWnmQ=I?kcUm^mKhF(t5Fb7A%XR(5UoQYFww1Hgrc`0dpc2%YA5<@tJ;YsD# znGn}%#{auX^_8qHIX|FBxyCI4Ek@^Mzc>z)3e`F0wX)UoRq?1Cmti$#V>2s2jojU9 z1V|=5^S4LhUfQQ%0)@f#XZA~&<# z4~;Cki@0n2J*IRMm6({Q5NK^_X&EmOfeGTwm-;yrV1T3^ap{&MioP%9v*j?$--IR7 zA8BSs+gYV89Qyad!74HH9EFD#7w3Pq2ewzt1uRj`HGLy3WZdh%vhCGS#Vf|72)j6m z{la~x%b9=A^)E|3r!J>wsAu`g*Iqrt229^t z)M+lpRt~qX#VXQRV9KI4RO#;No{vy=ZDF6?uq%6xgh4SofyAh6o!2vgq23(45_Sc4 zIZn3888=qTG_bk6{rjPPb(J1L81`ZB18*e1(j6i#h1qfBh;tR(t;es$?9>O@NN|II z%d~&BM1#{bY z0Yt|w>nTFOP@}BYaqW+8+G%j4#TV_@xM-h4hDtTYohKI#%YsU;CTu_w{gO5U@&H zJ1vpQklzQRH$~L#f^dWqr6xRAL%8IlYV${B>R#v~M2|!mfcy78#xv_(?DBLjD&_ge zl6=>dz{0hIGS`P>)Ce%%vXEfRmBqD6|0YNoP6{wX4nE|E3Gm%!)VyZ|`@X(k=JlEp zGJwNd{656FFQ2pLx=@*k`;c3vMj<~V)a121>g{fMFgBgh#AWu|z>;8{@1EM~>YY8+ zm>VbUmh9?wNEp13_5IHkx0!6UlmK44&%+xLUk^tMwcdJyO@ywR;`JV<<~;e-A%<~Z zIT!GjwxgE26EoV&3UuMWd4M#gJdL1m3U^m~AS`Y{ZcXZMIj({`9D6w(Fjs=@|3A*& zJD%$I{~yokkWs0uA}b=9AtJNvGP1W*_D-_rsaG#8WJ^||WM^-uX@=}gamWsFtmFKy z=P~Q`?e%_re*d^}JkRyKuE)GzkL&SxToze00n|+&}xLv*1(7P5c-iATpa4F!lk6FHnlBoE}Lm=;FDAG?<+7=7}d zBpgje)?W<1*riJTv(vMnHM_NHa%F+vgi*~xsFKx2|&GwPX z?SUvUl4QyD*;k9c!ZS;47@h2`aco za(su&^GcJqU&-9(t>-@w7IwmvCb2iaQRQCV#Z9z*1$QaZDWP5wCTc#SY-=2ciJ(f6@Oo*mX0RzelumUykQD>#P9B{>r9g#6iy7a#N0m2#}~ z1(z(L5B|B3uYX3-<@@c{Q?w)VY@tfktkNU$S0a;^!U==vWN}1F6 zxG6G(?9673;pU$wD1mppdq6gVxS`0uhk}WZ8@cXOY>oO9pEsixN%;8cWKI_^=-Zf# z%lb)tvitT|inB}v_KQa5PrSsNKp`U?P1nE=`}ZC~UG5K_fvckoZ=sVSg-lXG%9Rym zDkfX;Cl2JHWT#yr%C?tu^3aM0F@8A~wONDlVZ|g4AJY`U=_=!Sy{E3SL0ci3vBWvMVIj!wrYT}QzxbX_;kL<{w}{5QUEpY$o1T$tnMZJ%l+km!Vsn~?OR3Y}^qZC`Oxb-Xy=I8gn{BcTbI zYGNujEH8VCPXxINL?+>+rE<(t!L$Oww_!mYA!M3VMOBI_n^xyJ<-~0N?aH-np9YTS zY4y)oB@@VQ)#(c(4>n}ruAkZ80Fbk=qw+P78|6uyESW6`4@$;iy3E*epG_3?7d>hE z@aEcsB@(-A?_Mi?9pKjn@Mu$^ zJ)stlL??CMRR@qF)d0o+CiS=)i2_O+oJ zw=OkAyy`^vA~I_c4HOnKIC@xGK5(~@y*xwN3eg$lJ|q&hC?OHJ=3NxJI9XrFLU`ZR z-uqaw8X{L?(N`p^f{d0fD&rQ(LEfB-C8Auc^2HQqW|19u|DF?LkAeBRy!WQzieE3`^ zD#tP|j$A$^m57F2k$2=)#`;k2mro10@ywAq`&Kj?A2rhc(BQlgW83*idL7bCa<;1Z zYn2}5?@U8cGK8PL;3lgAh)LtVGZ*9?C2cUhwz%fK>fVyvt3LQ@5|te6k3AY453H7I zHrvuw#2dnxcoJ#@8GMeVzmy}vl|AZz0i;unB;E!4@q7)bbmDw%ru_vI`}uW1*oFT| zc|l+Y9}KQdl3ev_P)9((odxU|Kaa1v*pxXkCyaj0nI$}yO5T@&`RJrBXmoetuHL5- zKXVG*+sL0qtLGD6n*_C3_u9el1?4K%o#JbSh0P4VM-(4Em}Eu|OQPx0COgiE5trmn zolWvpt`h|vBEycRVyZlGu~)gL?g1Z6L6ft`*MTCtSQKc_bTL$jjP?Xc(j_()1TW;l zo3AkKuLc_5Y5NAm4D|tCue#mPp_Tb;Wax+1#jMY)#hEc$xXZ4aF$qq9TqCpyRc;L_ z_?Y-Ir~L(I~EiXA*Qx zr@@`2uZi0dIJ3OlS~MrFzruqTL90-NNN+wbf{ zDiU+iR}?gS>y?}6Oq;H}=Y8zedzq!{>V6UbvK*n*0>Fv?k1%NN%s)yadSw9|J+kw!xGG zz})P1Bwzv*m`b8bjTq%-Ie#P%Y-o-_@6zNf`dsuxN8~C+^nFxekMM@dH}b(npSA;; zs>xqnFZrW%2`3!6)BMU-bltLU0>7qf*<08ipFL^9_one1$mQQe%bRf>AZGZ=h@HQ# z0fD2On~vFs{CKQcANGtYl^iw~8S?pIPrOta++y0EWxA!E7Pi2lM&6Tpz~ynN*5dr+ zetr20Ni^7I8YB+jCa`&!NNiMFeYZB5+GN$@H?DSd*f4dp1*)TG67^=-Aq$|i=%dK< zSX^q#iPukUwoUO%7DVPKYAgIm2PkR?>n5rFD<`(BEq z+B=ZHe251^8N?0d;uDyGRk}+;Evc(gUjoCg)4|}MSQ7n$sd~Rb?dVqp@P^OxY8s^5G6j!LS+C;N~gBv)_j8!yyH^wgZqW&%2O>_@CPWmp8{M-sVV zJzvhK`7xy8j0cKC@TDo+%m;Fq4x5mLBna%(BXqvvX@gjRnx`6T)Bs4|F;WX2mV(b0 zhIqSbZ0N?e#W7+meM}}#sEN*$awMQ$khQIiz3{S<+_SLvWNm&4T@ABq?`PX%exAl(OD!(qIIz1=bvO zFS;+R*({(2CyxEB%jsrMB*@mxv`HdJNx#r=#Hs+5y5@&v@I&`3rnE=X$|Ih`SA{zY z603q!;4hi^CcHKTdk|ZZa=eaR)kkgY_-<-tpjj{Iou+}&k%A|e9PI+x3o)fUokVxD z^`waiU}U^Gw?P|{K(i;=+RG{YUE1k&B`LchRmlIi5K#TAfKfoe zoxrS&Dd#(=jZj2^NFTrM`PN4cFR&mNs{-C)nN}YeRC2?mLlCTih_u+@AZsoR*+FKw zlA5ArPZ?hN@IAgj@Tw-NQH~>Lx~3hoyo$Sg99L;D zAIlz)As`(HLf+dt1tr2uiJ!5Tu&MZ7(B_L5@J&&Kp7@vQ2R?e>id&tEbnstk)6|CkwQ-Hjg#%4?^GH{o3wm+o}aUl&=M9wG4tYTp=!^IM_|7vVHxju{;g(ZXR$(HLur&Mgfx7@kw90}V)9PDp}_9i!qu66CC~50Nt;q}1Ev8gSaiw2=3<*7s5@kB zcbg{OT9Y*pY<%q_$O^SuX>i*AsLkWT*u4)Wc0yoM^V84@Mwne@F!gpto6JwTj zII6BO$UkPAv!n7uT}ee*X;Yv)Nc;1%%@xBvNouyzEq2)U&o?AO;o@7~P^S6zXwWZz zRM~?^0K9PP24|G)=ANjDonl}7w^3ZAsja2Ju4O!`2Wc&KG6;~4{MllwY>nxr8l>I_ z*bie3MMob7yaeKp=M6CQzrKCP1tn1eu9<*pxz4Yr(pv)rl=sY$&xzn7@S5dd{+}k( zfm1z9%qo>kcOi5fB(0vTS=>BAIl`2qgu*+BJzw3`YYi%eW`zovvd5y!cEvG(7*|mE zq#Qm|Ft){%f9hdmAt@IxZxoaf3!U90gWVgDS$&?wO{2OiNqbcSq(R(~8^s;BA%Nuj zv4IrOB4`q*a;VrB!;-pLkAOG^TiS9YJ5&6nHf)nIzt|_#JCG}+SA03^Eg165J^TxH zzP#xaescr=gmq9B%9AH(xP^(-pB>uWiQ`u#{wV` zVFgNb^-?!zEdX?}$p7^2jPVQHD`tO~AYT|Hyz@D!l%Okvn)5DY@I(DkzxOf-@e>A$ zBk6pe?^mJ}=(8h{9Wi0p2w0Q^YNIs}W`XF%Jp z{u7^3R)8pvKV9S4JA>oF{_6i-dQssJT}cKUcD_OlqMbW`Y_9CzrGEhmkQ)budmf8+ zhKhwF=r6j4gQJ-yk1Lu`>3%xAVS6Flgxcdow#fF2eYMI2zOI8o{}2n`#cd4iI^EMw z!tgY%=QYV@*8TQa|1a*LUXVh?AJ(g&VMC1A34m3krRes2;XBA$9|xykQivNc5(Y)u zwZqf97He}x|CDYdGvP5oP<5=1KX{L4F3Rs|Bh>u$dqP}*7VmEh#pe5CKWC%BnZsDo z{4ZV~#BQl_1b&qnXz{If*9W*l!~H|xL09v5elIfz#~PW5A&U8*@$$dtoyc+0u3uBO z|E$)|(By*1&cCAmEVLPGdwWBUlQks(2ag}dlJm2~IKa+}S6l2<%se}b-C zEeEPkZIJ*R|E6;O-*gCh4Y2P-%FA3lM^WG5;oFhMP4D?{3)s>W{mrspk)qas+`e(`^z!1hDL`Gx>NbEKfFb)m*Dd#uAgq&g}5 zpE9y@-_G{4_yjsAi0Nb8@r%&_L2EUrZOGnH|7_uo=K*~?x1V#*RSn4N@Qls)(C!%j zMwNybFofR4Qxn@?vbyUpwD;{0VgFt28!EsMC6a&9ZW7`XFM|Bn1?+&Jy&wOgr(E%1 zh>sUX|8b!kG?V|}BBT(BU;v~Hh=onUCVp*#{vV_YFUH5AWqf(&kR`Ko)U{2*Kdk>k z801wrFS<^h0pY)y*~ZAdq)HQ)ORm%vE4)x7cofwD~Y3f_pB`F@d@uKp}rt1S^u|qaXfR zV1s|l?f;e3bf8&8?m1xtJV}t_#9dqFBIhXB&d^=U+d2~A6DRQcMF+sl|J4wIX(CPm zNWk{ajvbmv^A)N8AI`ktC17{A7>|uC!kDmayV1rrPkxckExPFA>d5ptZFN+m?@i=MD%V0c#+ld3MV-H^lqX{-PBB z)1Dl$f_B|Y3i*V9=zkzs{6Y^y?LMwv2jRX#606-lJDXbZZ}@qe+Dti4 zkeC+bq0I=V@I9Dz*8jgC^cJOonBIYDkkURykWD}QjSJDC{{6c|-c-(i$k(Q-ULb|Q zS>TIkmndc{vwkz7J*w=Fx zud_Mo{L6QMFQg;PfAE_u{bSzX|3H@Bxqx-+u}8Q6G8Y6jybYg?_MfEr6F0ypeede_ zL|=YsIJP;+@2r64ye)*tr%=lv=Kz^|+b>+;O(gt%RcqUA?sv(yXH^TlU=|y}&#CeM z1?|s%?=1E|4FZx7B(>%GA2137@v7Sd(|gu~*P8$3Z=i?n0$+PjD3=J?O|7=4K_7l8 zJ1%w4!v2Z6P{*NdAAyU2Y!B3DanG~=a{*gVgZC^nUjj$IH^bYu?iAgUGmGD^PapOLNc} zV6*qv)>M8kGXfv)k%k?z?X{QSI0-2E5t&|#|p{$I52&&&|x%5fa5g!+c&0xYH?T8{dy4IF6f z!>~vr*s@;^*!d(1T|GTLjjooK7X0>jM_ufUrAYQH4NnEUQXvD$@R$wX;E0a5HgfM% z$Fu!gNFGDg9jUAlm;I$wk?{MsLDp8QxbSXFIgxV+f9kNUzhHoEZlL~xyMj2Vg(?Y? zUf5R3I`nGPcS4XH3zB<4Z0{*;1wDl>7ZMy-L5&gd-D=0bN#gmF^8`kV&j#5usa>H1 zRhz(csLEe4!qXDPMUuLCVhCD<0u^x1XxqXz(dP*k|4$eNn5P?H2yThu82lD`k|O87 zJJB7Q8$yX@m&kzo%)cUKyN5iGfd89K+))&yGLYJCc)T(@2f}0z4R+bL^C(qc7;+UQP(@5g)Pg=~UjO-E0OY~|3#(gg zu+xweR1p$-|3{3o*(!MzIIQt{)6F;{Ku^Z-wm(VUZwuIN#D}s1M*4AwhBcm;2tnAt zG}_u3;#aJW3I{`6UD?+q4sx`Z6C?hAumv#y>{&zaQH{YrUT3uPa{Mp5GQZ7lsYx)4 z@D4`PL%&DA|6@lsgp+=8e!gJnC(_@JH-T&+Ucg-cd+5j%ijBM8xVUAuAvyoGy%JvX zei7zid+X(0`Xq$fKlYVvEF&=t|E>vs~8D z3jx`$q^xSJ1S}=Z(>(?06k@qwa_vCNQbG9Czl#uPFbWw{IUZi#?AkDJdVw$S+hzmk z$qF#khor2}@RwXqhC-4#bWEMxveA1c_5aq@4q{d-GjsElumtD~=k`l16JHVR>@TN? z%s@yEBF(5^d0c$tF1;XvAf~CYrW;T#`O$ViD~$#nnguSodAOBL zmBq9)>Pf~=2-6azY`ycBI0gZ%^=QuK8UOAg_Q1bv3~VD(po5Q)Jwuo(-qMi&?`e?Qe@)V;iN0JXHFn z8zk1MtgWtjkFFTbXko~*$&<(i3>=>hS28S}^^5KFFn2Ue^x7DS{akSRt)vHAxBG*_ zZHv_arQEDY{;e;CaD})1wZq!YU$%|I_QwbXg25l+tl{m%AcaK~>?IRV!$?*YnG|6P z@ZkKBL>rL*1<75G16bN5n?dZM|4V2Cure{+=Ewbg$1Jz-RkIw@*#4Mc%N4obf=AJk z1n2)5kKz;NF7Nsx-=u7$wS_uB`WQTz)nl#Z+ev3`Z~%^qB~29qiG_PwA8f6S&xQbG ze(s73B>Z^rI!#qP(in%nQGK#?fXHSGK=Wb1(Z*Z{h^RiMl|rb%@i`x@&@M{cYd27) zD@ZW?r&ISFr6GQBLs!rdv0P9yx=A)Z0~pE4l}{Y68njc|ES3n{wi3od0FOjObWTEmfn=ju8~KRd=k|&4V>kKFoMtjt zOCuMZK0iqRSQOkWrWv5zPEdIFWv^#O0n{0Yf;#&943SZ<7dPXQM2nGQbH!_GkL)$0 z>~(2O>%_b?W(@kVHZkveAiK=CVCw2CacjTMG?No|Bhi^GlA^Qa(&>8J7Co91NTst& zv1a5xa)rmcrE)dj&*r6Ug@Dsq0dQkzzMM_#c~4UhR#Z3m0ybY z^g_ZJlhx(Q`Fp|T+1`tJjGU6#1jbCrQN>6OFSdHDt^+@rC+*p$+$kC zBKUSE-EBu3eTn+9#?;BkCAZAdYWA^exuu1`Fx7z?i^6Rn-jD$#>#4Zl*}{LzT7ZLF zMwkM#eP@Qz;@R%n9^*-~*Wx{c>7wNO)+f-jj<-{8^>7Dqsg~3ygj@EljiS5B%k9~x zCD4D(chOgecSc{+zmZ?#2wWe%W*;Tu?PAq#jpdy zoUaB(-0H~~mx|DmRZA)7I+VVEd~kK9{brrdO846c&v)AnE%2@M6Oq+CBRU@#la|M+bN{h;yKw=)a!UoaEqphI4OktIuK z%7oBl1w%(?!pLH8B^%3=1(q?p)iqDda-fEa-Yna8t4`vE43s8N6Ya|%LHW`nP&QB( zb1Nz`ax>?6+ma@ND=fB8bC~!*m`Qr5JkR8s@kW=I8|-qtZ}zjuB8fF8G{vlk^U5lX zgb3g9kpqKR*KW)wnuvq^I3RnuGY=~lyMwtCa5Y|ZMQis=)>H89 zlo-grF+-~jDZIo?bqkQUYs+tk6Oiq&(LLxRIur%2tIeDLj7{KW^r^P3vvrd!7Elud zcQv$!F{(+pHqsFh-T#^HI&kpTMwxM^W5oG-{tH-awnt;kx|!j4jGr!#7dqWS9h8&6 zVS{TOo%L1TLsRpGV`7yH)X(MnCwkBygT+#&yoSl@F&hhm!sP+w=HFwZ#4D`VRE+DI z=pI(C*OI%yJ`AjU^E=bzS3{3C7Qq#t!3`*pBY|urhPAzcxl>fCx;)tuH;t6|3e>97Wh(pqkUaIoF7;&k|!4p9X!rH$ zU~XQ|4;<)zulR0{N6lw?7qGbX;7{Mw1IlZYovw2WAUQqCt_@t8|Ke6z80)pfDf-sV zK2f$ap?8(!|00NZ3fBjEl|zHuH?i%=M<0kG3C)g@B~ z;R##XY}CHpnqbR;Atc~!q1zg|B^Iq>zXWuyrPl^1IYHVC9Hgdb4FKW zi>Ajq4OVfGmyK)H3(PL)pp~1v%~n2m4fJ_%R`+nrMLCB&GR}XK
|?vu|Np`Ta;BqfY54W2EGAeQ)HjqfgPTcX2Z&4xYjCk#w`(Rx5d7GX^|e>QF7y;ntMWN6Q2T zS5`M>M!F>ZRzj=JE%|h9XI4R1fJm0X-V+6_fA4dC1PbzJoHV!6LqLc;G32dW2cQqi zg3u21o7_b)4@kY&CQW!4ee3J%(p9SbJUZ&Q>rEYe;^v1a3L^ss*C*nT)~aP=%5_gM zgV*905__Xw4X#jCJ)R^7*Z56exL0u`)E|ph+8zO zOXyoNM!S;x$dyb|n7Hz#j!10;&a9(8#rfcRabKb(gBmyur&FCM(>E)1_*4i$(RVXH z^2$t~UeJXnMCa>!<%meOp;|*OV2+-I=!8%bENwHdzJvWoYr;l(`L{(zPRq)hCM!S3 zs`Pk@5k9>J+vmc33jfLR#Sao2b7NZ92Cl`s2GcoSxH&Q=xn^ytwz_7r5If(I$6&A| zwlQ0`z<8_r7RbMU8N29DkCqL-P*Og3BKS#!e-j3fWDgP_;HOCSg3V z@X#_MN@A4({rbS_4;BaC#d(ZysrpzZ;U93($zpA>+Zy}Z9^n8?cz~jjT-}(j#ZFZ-* zL~Faowyq%>V5(k?YxVmXR$k6W_s??qRSsv|S_q5R0attBEE~MEpi7s9*q%6EY}`jZ zc(wAJq`~JLsrs~~3P+!c_4)&iJdR$C!x!&ve9u&P7;qkr|Gy~i4 z?7WiDL!z|yPJW>Cwdx%mPjHPK)X%14?eFaJ&%==zFLjR}^%9yh8|2xX)%_DAx4P}C zbUx8%gcMPw;4n5d^bEch;6jaTbZyh(`03zMgR#$+9&6nY#2FC{Gd~7e+$|3jbzsVU z$1|&1JxrVD^>CUDKki^6>H5*Qa89EI@_OD4KN>o+{$Sr9*!ct#Z0Dz3iV>$CI!O)f zcx>aXSJmZ-eN)E4XFbAd&u=Wtq4lc=i<7eg>9qNIT~_Sl5)0NO#*EO}8#CujtNzw< zwZC-@p1&Q>%aKOHgAVqz?NM$vR)p{*b#YB;QO>rV_D40tNDgBH4Qf-yj zZ7RwQpguQ7IE?pBO* z{6g%Q*dXSqrG-uW>`8biM~-EuN_URKJ@5HXX2U8la+g%z;J!YEmx14z87(AjuO16F z#}#g@*y?^BH=BF4Rv0;VVp=9lOYkde#(r|0786dt7YivbC|byA=KS};*pMKJyLQ%N zupXbhF zMP87kN*u~V?t?|bQi6DF^=6E83on^}DI-4UId^${{fDM?SNYwO8kM1L5ye@!#a>l@ zd-o#$v1XFtb{pY?#U^fc1L2meC0R?Q>2+fZ#uCdh^DkyBq){%g(?Mq`_(zYWC|%J{ zH%QSLoeVuUXH=4WnfJA3rbW&0jcd}lOeVF9I(8OTLoeYzM@aMfvXXzs8561anq)Bw%AJkB zt)JMc7OzR560{FS%mipjT&$C5c9%$<|8ZGhvCjRPqf+|_;IS-HtPYfCramP;I6(kE z0)2EuAspo;x58q{PHYX*iDMuZWkq z9uMnu$z4`8$v&tr+@_)|_=RMA@)q^Iv^rn7A#-9ab&NuR`>L+T*UF$bRQM;ifsr$5&!3x%$o-`{s3r{j1?8vM)NUD ztjCO4Xjy*z)~)?9oO#F{IebNwn-DDe+zf?YK5S!CI-`vOR7m8gY*PTf^ zf>a<(4fc=lSNWkd!W(9=_%`k8EY}a+>PP6jqL=kCs?>Ggto8U8<_NU4I4+|iV3RsL zRM_^7j2WGp@&`#H)$Iw@AD-8ZTVAQKm>&5yS$ZWeX#ZoA{gp23Y+~&8@#X7uZ;&{~*hj%oUlhoe5Qz9x<(x!i8L~ zj_e^lJIVKcxl14OIE95Zsg_vOmPzcLd{kuJdosT#l}+W`^fPsWzF(H-4OVMEPM6lo zcDjbV6dqs}KI9=+D!x?rJUR3(`t!v>bB~QGzcsxLx3%-R<7MrUYh?P%qpj~rd*Ib9 zPxxU64xfCI;;3?drsmvr)Z+Zxbk#>}UetpA!00@xB>LV@!ZY4{C2LAhL4VN$b$l&` z?`4(Yj8D+8%7xl4L+qT`%EGI~Po5dLW%mwexjSbbu)K*(5kEkDoH?AwKt(0n?@ct0 zQ@{K#%Cq5JWn}Ic>9AUA3EX|RYa69(C4KzfFDSXS#j3cv>ar!h%}dH;#`z=Bl&6M> zkeBX})_2pG*q^X@bL=!%J;k-vvWk%L}}FtwIks;RnWi?Zt%33aZ+yZa$bJ z`zXrHO*vsbX6%0Eb#kW~hjCiI`C^pZVx&E@?@>ecjD*2&Y$YxqWX5rQH%6VM)7B2r-pP1l#hF?47qzQ_+2*d4Ejz{>r&pXWNX3t z*U!3`)0Zu>o*Z@Vpv@+j=4S{=FYD)2cAVg(DV%2YjAB+kJ}JI*sKmGVxJcPtd&}z5 zft00>pAU`sZ7hFuf85DvQYkD?4UhI_$aZF{G-|x>p9dd^o0ma#ruNEJGR0gtrt-nd zb^3`E$(uvj2RFKzVwYau-3NO^rHezi_#e{FGUgnTaK*a==*P@aCiN|=51D>-4gchj zwsMD`v;^?b(#pU>T7tntC>&Iio*}5T)E6E;FWwNt8Y8f($lN6LWnqeZotDfe~ z+7pH?>NCF9uMMA_({C)zDLDz+MyjV>)#K295gvPU;_0n^RCS=tG38q%ZOv)qqeMNj zBCrQk_lLVHP39hC81qY}3B($v$LD`PC$k!8!t{P%S#o@NXhH6hLO~ww{ees-&7^x| zG$BMfg;9`Ekh6CPyL9g`cHw3Tg-W4VzW(&zw`R=klNCK3OMu~n9IX?35?NM>Y~zJl z^)H>L>i2V#>iGPKl;w~mW;&vPz*YJnT|ZVB&U*h`DUafAN}65JYD%>&X-d1f&?Ko( zMQ45}TSb^*Wj_JjoA5B}+~C0~qooi>%fpUw9&gRnixs}SHxS#T{ty$vC22r&F1V7A z72=v_vX!O7Ovs^*O&<)XOnLOe&rHO_+mQ~Zj@T0PT!vKPekN+bXAONVcVN>RVKIC+icHvwNcF;G zBO9Nd3hws|Y4EXdN12={6piGE_nlSymdek3xwDKkxu8_b)4Y=B1jS&)715ueO6@j( zi}6rXElu?u%|8*R>D&6mQYK91j#isYXm51|2UVJOQAT7Dn}kWBUi{-r@KDwtV@;AW zw?|_RJv*+bgQ=V^`N?q6fYxNtNF~+y3MmcGv1=FKp6x9Cp)_|DmTfHjxp<-Zez)!8 z&SyWp;zQF*2Uuu~dS!VWI+cH_$l1?&D~O5<%C4&FvnGxD@b)wKh>Z0AwXw25)4kCY ziRec#!ie2TrUr{TZ8_hhGnTLBgR3aBv{Ziv-H5!npK@b$xHHmg!o;cKuhl+;2h%;` zVkyMnK)>Pxb=>Fc)33<5n~bora*(M4`rt~DD7k5iK3qzadH*-_v+fc*y&6$XJTQWO zOtB6nCm+37(n7PTHz(P1Y^HhVR~`nwsAbfCi+$tqEPql^)?sojEq{9XMSkTXp~uaJ zGX8IKZpqG7b|Paf?8(<2IkkMBa8fUuJ1(;DuG*|CiR@#oTkrB}nPf)ld_BNOu~m(v zz|tKF@~8KE+E+GM=C;Ak*yO3gFwrbE;O!Z0x8{|Hap*XDKf<4-S0qba%evcH0o8Sz z-wiiLt@x}mNycxbO6~avwgd*9?o}Rm&-Dstg6CEB&RS{zvdV#>gq?h zv@`7{kJPn>Py``!8j(ur25eW>vNQrmg4Y=1dG#nFM|fjSwj@vI<=*=Ia-GiwyAmZC zMPs~jX~1AU&Teo9N9LiC>uf%hBfOvO#fWg^3(f@Y!6W%&_b9}=1*|N)9_)j0Feg$J zpBlG1<@?ldEr~+<)~HHYKr3Bc@36d_yX8@PB(R8@LoZy7WUDi?p5e_9z$Vd;C<-&( zL`ZJCJpm5n{!DPPfXlPO!i-9E)S=J@6YZiY%6r5gbcbTuGLyCS^ZYa8JHN@DSbn#5 zYbEo9dSx94!Q@9z`H&YMWy0i1Te8@m!9|UGvkWV=6Z^wi{QVUXO<8d-LqFJMxno@F zDaMayb^kb4?}H>=6!)2C?{MS3u#!Q9Je+irpp$H>_gQRm2=CxC`ex(V%Ii@~klmeZWLxX?D=9 z*oa}?Prp1b7c+%GMc1W&5&?vxqigcE#&q4(wbPO%L>#-H!de4 zU;wm7=J8hc`iFu_`Pv5)GP58?h)T>!*Lz*g(AF}J^J9RKt; z+q3m}Os03yx~bk$k|eXQBh`jk7vy0qA8`q{MYkvq?)(ZIa)=a7%j10CEXZK`(Ztv%K z*!}U$vz6t29)*Ji^Mf5#unyA$I}i*H#{2^bztiHOmMJ{?_4nv2icG{^I!v6LR|K!i zP{{KO2n4#UEk(vN`}-H6%4H`<(0WLA3inR#8qb+1Nve^Xy$oxgF@ErAgEu#1@5~gv zS6Kztb?I16FK^gna`=?|>In+jcOhwh9~ZxBCLIP+p_#Zr)u~v0AU+K2qZ+5X4iu&Z4VZ?Ci<~LOC+4tEz*5_J1^$KtP-Qew=DE1}g^+8*g zzs+17*_sBO^U8%Ro`i7C&rOB_t-8JT^kQ$F7jN)_tv1=xR1Y(<(c?y-sQ&2}=x2Sw zT$7zjWqtU~@dehL#ik)inWTb^{`sn4ov%{*{1&a}m+b@1zo8C)>X#7I z8+kuvDH|sLQ=Z}H`L7?D<9@2uo3~=pxx>oEBF=|=&SMxK20P+#j>{JxZG4wWu2yG+ ze13qGzJ82J-iZ?@e9bAhz=I6IjY|+904@nbO6ltOT?TCPYmYUL95o7c@ojwM|A_F^ zVV~ibRV)&0G5hQ2-82OQx;LVgtLzW;RP(2-XnjtUg3z@jgwDGsqG~(QT#*F{mBXE? z&TnfACKRhk$k1t7m}s{*f%{sJdRtH(dgU%IF6_^#ZX58!?nvdV9e_0dZS4pRqPu5<}+PQ-_zg9&Hk>3!@_*s1#^ z`wof9Fu6$=dM|e0s5L0NDkw6NZ6aK^8X*!Y;FfldV;xyofUf0HMi;OX)z7<`gh~48 zGir}qv8+PqISvkpN22wYJMzeDg3~2C+jFQZlHR6~kLh=0cIP}W7r95wbW;HV+Q2ABt=>|e|#jIoD#oA%|6YEX0RqL8R9ayys z9#V2xXr0W*gGr)s9L2C}XkA25h;h;R&?1wWvqf$td=6uZ z+V4S0TiA53+W~RxIr*rAr3rChNU*(+175|iyDcp^oZI9*1Ol1o)101 z8_vL&rVnkDoO;L>=2_FDO06GXrAs2gsP|6oJ1xOv*^e$x?>M_OiVl?ASID=d0q zK6cF}Y>usN;bkAe%Y)OWh6Q>~)G`OVlhqk*GeI~ONMdd(SY0kdc- zl4|MnFm=Vf26Z5O#~{{%!V@*WfMX1Y16^Omid}_*7Z>m@9S?jI0vJtB*q-X&7kv5p8Wk`9s*q6VKDxZ8% z%6wRc=_O3??|rb>tZFZ=zt^1L@sZuq4ftQlN8{ndU9$eZkm3k|#%p6TB5Ms4FDCR> zZl$C+7Ck~gmG7->7`s?^XKH5t%QGLd+P+@%m2VrN0#(_4y|cNgy}@Z3Bw|+egE&I* zXRDOl>KCl&UGZR%CKWhb6iYThUL+C@Z})hfWdW}rw8NWL$s#7ftP*|Zd)4`;vYBkX=b(H&4A zdRfj`M@Me(HWG@1UaiDv5A(pt*d^H#n!H31@9qnoAA7xaS`${e4y}dsL{2>+RDPFM%9*m_S&$xY4#6TQvZSj}R zx^xPM5I2jTPxN^E*rJSY$C#(aIa-~$yDAtGsnZs^Fd4^MU~`hUJ&l2#J;6OATyo}2 zy=kmwF?yC>`u%I~^)J$xJPOE2YC?On5C-+3RS<~{uXh}e8Vpi7UQ%o%T!Qs;Mt@pM zh%~L!sT-^&g{e>}Xd139iLzUdyvo*5N;Hv|n0z<-;1pVnxJw;@37RgP?Bq;Zt;s*y zZ$2K&K~SCL_Nrbxm8+ASmIeEX_X&dJFSOO^N9`oRGLiq^gGp z*7~DHOjB!njBOtac_(d!r2#x7Xp2 ztl)Apj|}RGWfOkqJ6X9*Pj10(AK&x|d-n9Pyh9_zinLDi?Wct{Kfb|q)B7HOOOk1f zm{3MapYxb`OX-SU%lC4dTc5_MQl$ISIhwA7WMD-)ocs?aEc$_vD*0m)7H+RY5oza}-aK~1N%X3E7EKA?Bkqz$R2TZ4 zn)$nu$8N>^?)neEga-rNrnRSXdd$U6 z6$QrHt;gY>uMA$PK9C@6tf7)|jeW=qJHE&p2_j4f-5Y^U-Woz}_^(mq-)-G=-M|cO zdkg;s!uud5Sq{RzbMFvTTdX@pbKxdxH9F>%R{p@(&=zs8JAQL|5y?lZ#V86l$~2ay zuD^IB&YN218PQ~M` zVa2G!yIg937MJ*VvQ>SiYE{ryi6XJl!;@3*K+?s&yriX074JU+U;;Y6iJe(BR zodulwH*RdC&tsz!^y4r46)Z=SvO}?Ff+FJY*^ajFbdu+6a-q{1+FC^Yz*7!r^Ia}} zJsb)Ke_#5Ptu51&e9dn$bmfz2s^o2ToA;yiBO`gV3SZyJUedCZ1W;c=N>YAOW^R&5 zg6FRBy26Zyh&%{RN~xr~g?u-<{6a*zU0^~fnBC%jA^qpMuMBz9v!8Dva2q$%HOP@i zU^Bf&dDkhdzhHe!hJVmYw#<4cK>;PDNQqLZ&^Vikd9F8j?pV$>S0u#{A0U z$Ne`0wm&B4hd)(?_GzfhY7@k9*&cEX*>qwDSf7 z!t8IVk9`wLIcq@0S=Dlz%dn@AJ8X7A~`?YY|=@n zV9(>9EwsDF{<5%Qfw|rXKrtbLIE52jqx%mWu3?nSj50q^jRUBrj>0IWE=Su^!2)9; z$KM5x#7@+nVyu#=RbcosZ8>AdIa?vdqnk(AgW2H3{ePTYcRbbK|K~;-C94!d_LfmZ zxRouGy{X9FdtXhm2_-8dWk#}B5+Qp>$lf=5^E+=`(WlS&{{Hc}xcA(1Ugx!6&)4f6 z0`m8P1ZMTYBr}tSJ#~fwbu$t2v1pgipp#*biCD=TALqUO>Ud1CI&=vFcEOx->`Ax^zP_$D%K)@0M0f=?Y8!t18IKw0~BGDxLWuJ;m0|gsGe|t1o>El=F|}vf(tDln=-)%#NWo(I!7@HrI-k*F)BQ^ z$Ln;Q6cEF4IE($|6J9v>5c`)fD7)cg5)M2skIjNx?HON#BJ&me3Xcjl=Hy)IN_@Lh z?A)3$nH!ax{6whEzy9dmgp&&YYgamC-MjYE>dl)YzGT#Lwk3+~Y;L@%ci#+|qR07Y z!x_9)-l%KUC<&>hPdEo0f01AQoYDPuw=Cp6Qf%agueg0y80nm|w##+V2lYhA?ua0D z@!e>*#M{ljRql9d7cgpo;dAbwJom&XjlTv_jg;;)=nbp2s0&o67m|B=n#H{J{W>M6 zY=zoHWM@si?xkz8J_ej`l$K*+fmfjlDIj^qhF@wyJavw@crTlYmX@#D-_f_pfEHur zEK1@-jw7hpT$XPqdqt76M))K-zS=@lZsj=+^YQsCoBeydA|z>qw#lcVCHO|bDd&X53hDWt~7A<8qz1IUlIY5C_)<>(#YQvMVDAp z!D}@1K36Ar{Nw^wC+}wB?3#)uiKVarzU9lZLWSr+rt5`YR($P&aNa$~5hnSdMSrKB z{D4=KNHm_4$z$Lx0i}I?V#o6UY?t*IVGod%D6IBb?xPpTH9N7+$4reVBk-I&<`?7{ z(ii?wAxt74x0{XFJmxV3yW929$rvkXSH+r{e5d#LpU znW_4fJ?TMN14_q2Svs>d|7TB?Qj%Cd15=bUwGMg`&33fQ&b@SsA^i6*2wWB76EHwl zotSjw79Pmf1RcL;6FZqlGPB z*g87GgXx}%eUE7fvj{X1C^n?MVR-*ba3tkW_o7G5(`}F;IZ_Gf!q?N4wSi5zpj?wH z>PBIPOh48amPr);CFW(W_;>$hPmk^Q*R8|M0z3NFgkSJXb9b3qC6OZ6Miz@=wt{`n z+YBX>MTbbX2y+t}w7abrj>UPOJE_kWHd%P%luUx!d|SrIr|ag(@3}g!^+szRv28Qh zjAUS|3EzCQ8PP^$$S{mmajOiJEX+@h%tX=pspj8&45!9`^3fAo?rEA`PVb-!7U!xW z!z&}TB7ipU(|#d5e*1;sQVAmIjkB0mQ$;Aq@_?yh5e`0xlB3xrn;*SCv1w#J&7PP? zJY!ViAGj-c%yskgvAKc9`mD*)v2F7KBOh^$9{QUJ61OxSX}-C6yGh^}h6YHL8FY{*A?(@`U?#L4XuH`S+ zVdoX18z?`kPuz$zyCUWlf~fcD`}ux5qddyG*@{zOkn&}2H4v}!0uIwBzKt|<%q0M< zBv~~^#852J7Q>C|poN(h8rd6I_S7=hngrB}a)$(HB(EcxGd;G^ORw7_Goxu=oyst~ zB~D4L?cVuLlz+K44Db8vv!i%zW!W!khnh`dLJep5q4alUlysJX2UEnumXdD8)#(6_ z>rq_&?iX7`{0n4at~koPu14IpsCn~Mbgt~V1B_y@`1*J8D_j1W&1+|!?)DlZwZbMN zBw7ag=YLE)bg0ebGWR(C%HbpAbT+FM@A|)$s_lTAQm(JVAP|#Jrs3&??L!rY;_q6{zy`9 z>!kHF%)0i8rLrdU9DU7MKZ?+*t>qF`Izz)dWhVraYK?saqz%=!&)cvD;4aILf4}fp zc*&oqcuDQ;cSm0<23(n5Lzix!VEaYB%O8GxvepV`;u@?A&2n2ekMD{qTsjjlBFwD9 zFGkv$X3UP4zquLkGI6-B!G}KOVd|99)pZ+lsU=8wGNSi%uFtBfT7> zRip2@zf{hAU)LL>!9C6`Gju`a`_GoX5$aMix#Ju~bC3E;ZeqI4MAN0TZxJR(nD%CA znNKVRr)s`u($lheG&d|&Tg^bh0)>Q&6OoLA6y_|VK69MiQx*0TIFUDt!$jt5hm;AB zy|>+xf`auWNHgc+jsvKK@g-Agwt_wmhbx<5$6tPsT2LvP-1QXsn_uH7ZbOMD?#*vC z1(BlY-AxIhAhe=QtPR6eeR2+@sG|||7KV4kbH#PSZq1j3Xurs){ctvZ807R5#U4oG z`M;n0agjJ7OjcVkUs=$ovFL$j+ad+XjM3(g4wL7bOjI?y{|u4OP%ESor~GEyS{j4O z1tfaYWcZgT;DPMg8-y^p@dS7f^;=#mR})Z&HU2>M*f$&pN@T(N98RXuEj1JuI#d@V zB5O$IwS#m#R+7sY8bIPq$Z^rA)_!5$XBaR1G~uQF@B*bR_;e6h1QI!D@4JHCF!_Mn z4Jm&n)}OV=ZS|$C1fqZstN{s@-Ah_=4##JOM#igSp03u@vnX$~Mx5!_YAD+J(Y!eK ztw(8O;9U+$$@T{x^Uds{l?jqKZz4~;n$0xYiR&@ zOzTs-{NCFMZn;JT45p6i3h%8N+@`vYPu>b){CcWXYdh(<`%g9-t1>gQ!Ak8G{Rv-v z^ht7s*CJ(Z1%svMToPKMGM=tY*z?qCoqRYmAN#otr&eg`tN!hAmH_ zWP3nct%UJ<*w%&yF3rm?eT=4nSw8CL4L)x5d@R=f{E$YwBb1Om)}yt63_+wal3H?swgq4+BtgwF3JTk2Vlp+^ zi(kq0`&0J3Uutm}ns>?|*BtZ!D~Rh>Ev5ZXe6JJ*G=q6pOnmJ*3&>965i?p!2;|VY zPf_w&!nMzRS9y(xq<^0^L$G_yU6LRawjNWfKe2{M^ zi3JZjM*x&gAWyJ$xGx^fj$5Wn>A5IuzBGAby!rF+^LXjCP;vElo2AzsrY=y9y2;8& zjYV*_XnwYKsd(<7*D+HZww{W!Jk!l_q)Pj{?Bva&UmvfJ()HEwsp_X$7FPl5Un-tV!X?%;#f;RF2Eub{O)a@G281!~_LSfXnme(w$pxUVf1a0mOzt zj2II4`6+|ovteWjI{-j26xyo>*iFY)HVT-8kt_Vb_^t<)Ix!byuSJ|`wkW@LR>|#{!l7Tsfa_LYXgCqVl`bc}I{xyDyYJ`| zrORAG@lmGh-P?Wsix_M9uN~hu7u4KGiTtjgCxS5F_4E2&(eLG7-XTAM7sL?Wt*#pp7Gd0q^@*GH$9%nrIN@GrI)VgEx^?B?}enVu|*(WRbO)G0RDn2kuugiWrp%i` z$HQWiTKQJ4FsktdB@w4wO78o^8XQ~W=k~yw=KonM< z792^P;4s9bvZ|M#JShmg0-QnFYY=pYA=3FZF|!`hqq%Ks+0DLWY}6m#VZB&2e`Gq2 z+;asdL3L8&(r}_`0A0Tpw!Qf_@9>>hpRj19`y2fF7&clb=sZ64@-PSg6^p7@0)d9|RlrN9Hg;|j?|ZDY9d)uK2qy|*5?`*?Ht9=!`QnmZ)3>DEL`#gE)}53SYGm#- zAMdevtix4%k}OAJQiI``Opc_9s!VT8oAk^~w_*(i(TYEhO`jTz=gd~qR4^$QGSpl? zrE*rtja5D`FzAYWyfbEu_4gH;FHvH$f^Q?0vGY>5CRD~bbzhHZ>A;ll9D`jt-Kg_3 zyOtva*=yvs>^VROE2Zds9+rMS@TMoV=qI@=GI~|ET%FPngvuzP z|8X`D%h^-0b{P2YGRqoDHwHTjCz75mv-srbuKUyI2swXi6u8Oix;+^ydCr*uSI!Mt zmD{Vuu)a+BlYLG@XM6D8xRCB^oqUI@%NiVk-g2583YFif8m@ek75nZIYDg%a`;z0r z%*qd~j-rygNgg*w_zQ{waD@xh)(a+b$MVy9LCd5)5tN}Ok|h`E2NQ4 zgg?0VEg43E3QWHKiU?GkPx2DSNX1|jU!!$kS+ek!G1~b&o*VI4d^cnD#<{QqCbDffj<1wVkAHoQRIc@{TC067E2Agj z(*ofj{PYcLNxAe54In(KLfhOd2n2?k$;@|)%v6Dfo{^E{l5%q4si~}YB^Oq;+=a@; z_dWI<43Wfr;Dt!S9hRBAd;MUbn7E89z{XaW;S1zeLJ&PAAxJl5p2quThyV}gN7t{F zG23Cl=5mvebw{vrIXIK3aU}&j<52m=IKOqhTVuJ-qF2;?aydY|UuA^pjYi>9R>u!7 zkB|4=h~O2x&XXB8eCoqc`_c)K5I#sHzo5~mU0sx&BOh2^S&FlB*B^n&_?{(N!yujNY_3=pqWnxPL51<4x=@o#lFrLU*4v-v z69J&9t7NlLOV_`ZFZ*?eXo%?Aa*@c>)7YD{^Djt*77Km6a}@KNImLoyX9q-WTP(U3 zCY8rs4RUnyPe&3A;PFNL}??Y3|C`IPTOrzy6e4xPGiB82-R7b{kHzx zcY*EqdQ5j(1Q=EA!ytUw&5EOvG%gOvV4e1i5K)DgXF(lpp`*xrHFv>0Ga2hQiVTkv z`?!^-8n>?t4#;6laokK$Zw#$p*s~|n9>&aEM+AG;Y zWZd;MR4|2UUN!ICVqX_;x<(KeXX7n^J-s=ShZB~>zqFDZHayG9s+CieP4?4JB&l<0 z`Cb!UW652a;65 zB`9{gjb(V>i@{2y9 z)7F%q{^=yu&k52*Vx+Y{yQf6BCTtq^tet(aI>H8R| zQ#IDoH{LNlQadx8R{a4&mh<8&f|9QON>x-KSZ?~u zWf}GAi^Y$}jcngABHY-n)8W~(!KOghaI^^3;`E-*@JMAit#L7Z zv@Y7wr}eGf5DDMQi*Ii$DQ$I59&4ZOb*tBv#>eX7j|Z)-F)P1 z5V-AEq8y=hXZq~a3T}gd=mH*1C^N;m)QEe5_3#@&1s*yJAD$R&gi%HC>X2e8l?%w)fb@PNS9 zmGNh}>>PPt@G)vtE)_^oNLf$?^tE?Nh^&?d^oFJNSWz)+;8~}a`A>Vx5DHl8mKO0P zyob_uJ1N#NzBbxmCCd6x!NOFcwFg@0X7hfT%hfT$X}PKP0Q|E9Xjk@Znb4xuXCYtZ zJz>W;f4_F9wcxEo?#Q5A&ermS;KZ~LNgn0X08wjcn?INca!zjL95$v-k|u3UNvTw1 zK@={x3e&m9KA#k&j6Kr4p9`Q7Y}YJRR_BJ01Wf|ImZMkS<8@GGlni%r>!j_n}d zcN_;eEvyv+IFu7m8PSr{P3-#k2t-U7(qlz#BTegl@Yr{z4QBCM?}jtB%Y`Nq4b0~x z&omuj3FDx-n5@#M6eEhHA5iH3!09ooqOBpWEdJdll$b?%Na8BuqTtr2#P*Q5qLZur zA;@B0w{{PZ_k-Y&a%%5R$UJMLQ;lMvG^Ep)3dd5W8xxWVa~jFCkCUXj%HllTN*h8F zEfwA@erlQdL`i|gHoDW2=%hfk>bbX8lUl;5OHMUwwk3_o{2yH;W0_KkcZWVFnd8e` zV$$Nb-Cmrbe;{5&N1`bx`?^l^^qef0XT1i(8c&o;Csu;c{l%5-q1&lb--~Orlz+s1 zW`(e$o-{BI^Wz($mf=X9b8gP)3p>`U6B`^em9hKV!vjF6z2_xsov8DC4 zMfOGk->)x!1`e@fxh8dSv*-1!+clvi+i!%KFle<3u z05yOfX{s6#@4jIacVlk7)cvZm(0Th(()`O`6M}v`NDp+jlUI=9$a=ut7*=Ee1lKoDYDJwlyG$l z42)O971zvjKxiiFT9RriSpD#ZJ91`w3-YP<2=#q2Ap~H`BJHH7+{OLiV z)^rl`3sgY`rB&_dv1Ub~=ta%4GYjT8qgtU9JN2Fbghc|sWFe?k>h}*pjbv+*-tjdn z$`V0o<&S(MI+|LBfdly!Y1Kl846K}X8M1f}f^u<;r{TiN^TX8&6oP~7NtE(T+WwHn{{X17yU8#DI@3@(wss`xag%o zvLa{GP!MLC~K#e1LUe#^oGuxM#r&`mYaW_#n?_xC1A z`T~!90(tIWUq27ihS5n^&DeygyuYgAV7@Fzi02IVwKoS~dBbukU?}v<5leG}6AgD; zx$39_Nnzc&=8}fu7nx{-iHtWP^q)`!r*?1j1u1PZZm4CI9aLu?Rc`9QtJVt80&j zDoE{U)0D&q>v!Nop!KI_lRHd0<7-FaWsq%EbzNo$!N!5vC8un3NHa!c&(VlWQD)Sh!+ zlm2EGK@G~P7xgqE_*zEVN%TOL?n$ai`cD7?e(ocZG2dknO||KwP0i9rMnzPq$>4nK zEZcH=YWK8ydyoNznZX%5Lxm0(ha2r=t!d0^@!!52d++ij)HlAG=Ir&6FVhYU!D-tw zzLJ!N7@jv_8iX6QL$B8E_GX0KsM9YSU!M|&Ngy$j`B}Ph2C4L;!fFIFT#~iQ?41q8 z+VE_)u2Sa{G#DNsqk3vyAniyjt`s9GW=9;s#KJzLgDrjFw{tP1BC%AlZa#!;^i4F+ zo=~z)e|F_mFCDdPOySKrxyI`;?nCNKE;p)U?POCm+c`3?om5@-+fMj&1U*Zv<{9=D zZXQQ*i6+H*-}<-I;kkNWim>5~qS?f$uECfh`Pa5qrE4*4wM25uJR|<+NNXU>;y1bn z`itTZhT?hGzwlmmAakHO%m40$rt8PI@7Spr3D>sq3Xv$6dnFB$xo>HtOU2N> zrzxs^+}rl4}21q?{ZtTnab(o*eO;{u-2)2x7=< zLL3Q*?I>sK%QUygKV5b!FIcbDK&FR;@d|R3ZU#l_m~iOkkw@Qwd2i{@NjTgtcw%~ZzX4!+)aoCp?%X3p$*xq91qn{qGYLen^V`z!hq-r^k(Vkvnc1V2B_T5 zBuw<3J&UX$kU3*yslV>{=xQPAwhrc64*m@v9&3r1BW8V#0`CSgq8HDL0QmZ9l=)2g zhpe%93qL(i1ML;X$ASenosYZ9q)&2*Mrg06CTI!DlIA+An@_C<`?}r%P$l6F(#x7Z zulJ68GTAx_;Ekt>Cm(E6^br>0N`A80U{H0>Uo+@?dMtMdnOoBKrI1p+k3fV-D4Kjb zsAsux}v_hyyksnoR3-`*S?)6%=VWHQsFTg?#Wb{><#D+VIRPpC@fp)WS@@LQ0z4##XX+ zvx(R}$Mu0Jm<#a^udT3g?7I$+)$nWk#rqWT-aLZGirEY8ktd)WC$S<`6KL&|^vK>lkl|@>|sH7MP?f#HzWfzJOvj#*I0?=^kAD zvY}LEw{W>G?G$D?B{u=ee?g!=8KAm0 zp@!?gW>{N&vPIYST^8j`)&PUsQPS!)Ze97!20i5Y?)`&e_f=BsRRC_KS$~1&{B2t0 zxa|FOTAnC^?|@a@Ri9<6Fp=ksz=r1r}8SbIp3O2OG>$bIq%IRGY zh{TH?_X_`v+l$~R7TmsVb)JtRM3f8Z1pH20EgbI8Mzz;&G`rjg2SHA@>0|?2H(RZA zI6J$;clWo$LMi1o#_MuiER?Zj$n%J2)B&v`w#4@(yEwb1`Vi}!# zu{BJ3SgPtv$jO{bGgG(KeptQ|I#yq3oZ7Sf(-l-=p2j+K^KG`yJqeg|bF{CKZyNQp zFq+C;aB6zh&GXj7>2*-&LS<3gs)Hl8{UsUAvw{ha4Cf@)+ea}{3qZ_&i5#Y^dkjW^ z)j1qqO*587%zkH+pXC@Cwry_GgTfU`bQK&>&1#XT;44u-aT#>dH?vHh0NvV)>v~0= z#yYN{J@a!0)-oBln|~(TzU@ej;+f=wbJ3U3Kn4=PRw=AV0L+1vK`}$0HmHaHPA~)= z4R&u(h5B5>%-Txc-*DKJ=$5k{xtB*2P6+jLIb*%_V*kdcAZubYr4qbB3RjL6?|ATO?V9WmH84 z7qm7^Nmoj2-cyt3p1ze|Qmd%8JdG@dQ-o2GzaAv+JSLAg?Mp@-VTEU@8sx0*kt-!8ad%+ z*#&>*5&qup?p$Qpcm`Xzi+EiKgu7|?aBxP}>j-CBcWu}qPYG^~Xmeo7@Xe-EP8SGu z2v!;2QK$^O+?=+3ys5Cshjc=cXZp7Dy7F9}hPl);P`jrT3tS9- zg0whK_mt$~()bmmMmKMu6NkeonTQKG4>n#k#M50CR&D1ixpo%vC5DBHTfWp4>x9Kz z+Ps@Dtoj3r3n8$V%DuD;!FMQ}CIrt(9_S$!SVp7F}gkCa#&axO?v%vh%EOEJPx_cM1U3r z+L_*?ft1Nkk^9#VQ2eAu%@H8MCAPPkdbnFfL)2nW<1TNvugWC{q(18Z0IBj0&0Esu zB_$>LpBX4j*_^BJkWmibn-rg(@|jX`xES=~w2zG;;eChhf^>(*#wx>b^kVj)s>fP) zyToO38lTshO-3<%)&F!%$B4bAFPhuK&yeu?(iDd%@@Pour|h|g_Ncj817R53oO@Y+3``y*upf|!Y#S{v50MN`eKX%4ew|vBy~KeUrd(bH-{)@ zyO%;t=L%#(T*T^#WS%-LeNMFQIAUR>6f>N6S9;adm61jG!q1S1}3|)PQcOHyC_af1`Z&uMEudXqH$}ZiQ}biVkjp z@AbYcJcal3!tu70P;NpTiUOqZ3xQ&vRA-OaE1Mvv=`eLGY?MhY?#&J7EI0Q7FM3}d z3+KoB-)}ww+Vcj;T}~^c84qU(br;;qsQ0!uSwC7pIy*P?AWbxcIhBw(JM(UBSa6Vo zPGX9Xj}o;veZld9P5Bt-UNYyZ5Ks)!sJGa9wg7-kL0GfRlePNg&&bDJkYvqyZ7tTo zB6#(_xK2b1qk8~)@VgD|w+1q<^%f7+&ye`ge?D^zW99l?s4VVF=9V_*j4+l@o3Qe< zmu!lAB#P7{d{_SQT*N}~Ysm|4I-yN>`?!O77Vwyt{UaDu66J!qNd=kHEZjbJE;}I# zoi~c@Mn8tRZfEE^`>)KLf=mdP{QzP9;*v^&QQ_=ktGaCQM6qu))+3Js&Q>#Z*u9{> zPN_x({Fra9j$8J?mmJM_e;p8p5-j25Ad5u3G*~I&Z0t zll$!W%zNDlf3%bPR|U$@Xt$8@h8|j{T4Z>4p(f)aT#cRqYDJo3%i&B~ z*C@_?y9S_UBmH3#g<*EIH(@nL9C*V)p30yVcQjxoSCguiME{FjM=Q3P)+B@=kSmtv zxKd}(4Tgy4h&3<1o)>%>77%eI4x91?kx=kDMFhvm04-Jvc1;##hg+GZRA0+MEcKUl?GRImF{FApGuolCY&SPYbfw+C64X*N3Dc1Fd$_GM$e2I zH!|;fA50uSzchxeN}e9kT}VG4t%uf7lKr@CXf+jY?qj6T+{#ZA-$L_j3pfAP6q~qb zLdkha>dh1xhVlwRku4e}YYXhvHx1zz;LhJ?NjNf(!-IrwFq zd&B6ZM;AlSABkA~{<)&C#A$P;L`J{u*b)7yHB$v|Mrg@jFPFsfDp0_@@=jznF3y1? z=)zWtLP{1?KvCmbwqM=yaT#83Kwb={=OR0|51vp4hmqp6=V*Lf!WyGpZ?9_zhUBy(hff27+cZ_+%DE2L zhK5O<3h*D3B@4d4ZPfg*ca=s>D?{mha1nR>6T_^UN25`Mw|EywKY*aeu)KVr!=7^ts;sL<;#seyVDDrnjnnkNh(=H#h_z(^>7*WkG&LL|mxMa#|IB0+&Y%C}o^ zAtzwe%nLMxRj?45Um~P}SXyGl$Vrrq=0y_7YbXB5R~n9Z&@E^o%^fTv^!&R9kJ^v7 zPri|2<0i$ZUj^-u)(N*VK5Pkc1TEiN3{)?i zzn~S+U-DLERHVrbpdUGBdaOZ0-RbAd2!se2oVgHT2qorx`Hjy=5OD z$K+jf5+RAXb_r133|66mhAtB*|^d4FgRZ7lr+!D z(cSJLl*CNY!#xgg%)3woy_eyDm(WCTRFqlDR?}I)BOoC=_N#D5Q{f__*lb4y1$k zc8ZEe4l(i$-1qQ%HGxCrg&Pff`1dS-@N4mZ(-^zf9{2-y&YhQ}$vmAXcwmn^gu-Em zgw4^8)#O- zSrAJ=n@;dWJGb8ozJH_tLN!ZIP<&_4`?620kC>gi9??NEqg2H|&}{`bkY~W+8a5;s zaAVv?mtREKd-Zp8hPp!cw=0f4G&Dp@M-eO*TwV{77*ww4#=lD_;y=@_8Xr*p?1Z@x zL*_0Tc=vL5j`p7y!J-NzgkegNN_u0B)9#7=ubU*6Ks@+k8t%{B=YRvqR6i-(dXRzo zwb7~fRn9&s{^{mC-$7FoeY~X(EYqokaNp~W_;q*xcFGEIz*8%E&3jH5Fqlzj;{Wq1 zpsh*cC;m7m!S?&!%_HvExTXz^qevh{zstChPrs(cDJ>IkUeiJ^}uL-**3w@(94J2mznYVW~!_s9shA zE3~$m?!N2{oGYwOO~eeCc;g1?dgO;!z+c)Io`eXr&Hq)?kkbOK{D7VQ)&4S>r(!Su zPPISXC}6JL*72QF^ss_;USFs<<19)MC;k8%e|Gztdu7^*jtcxI+XJBel z+V;0~e&F0ZWNvr%x0?vwzt}@3rvwLc%&1s-|E2@h0d870f?>>u(f1KyO-ApsFw)*zoVY1QiLxxc?w{e8i*Q!s&WHXG4M3@|nh9fJ|lqVhMv4H6d_+Me}PB>GZQ z6TGlsTmAJb&uJMd$q#47_Ox^_EcG9abtJ=%PaX@N^7qRpM_ShF@$6xom`|7V!--NBFrzZiPXKe^|H-mO9#QxMVl(DY3 z2B@RT*nAX>ijYHu{k=pT;0ciLB9H1jrmF%u$0yKBL>rhl(I}gv--!2{ai=A5IyF?+Hdv(+ypT#6Xuu2Mwo?ee?I4D3@aR=+F7wYi2ZqB(Z$zv6aO(GsprKb`;bZMf z2Z}(4XoLDsXCMDB}IotyLUwd08siTF7Cya4o40odI73@=|J~2eBTq^iMe(85s$ogWmi; zgO?|M|AwCP@MnM7d*BUp8RDH0WRSiN7QfR+ss4MkTc!iwW}ncBcuoku>c@?jAaJFD z2Ya17wi^=ct=UhEKJZsA0uuHr9@DnmpWPoXxF2;FdrW@~m#+x)K;V^@J$!;Vhp`|_ z7NUoqz2LaGr)PaMovzQaK-Zza};OIoK%+@G3Ko zX@K}g99@RO`|dmsm;|`HWk=kDjc9v+{Y_+dcf2oA#E_wb!L0fAF9FP7;QL=aC%`q{ zfURho1|m_Oo5+Pf((WJ!?QZ(81e!uAk!cQsb2~*Q4*1Fcr*W~22KutT@U_T8_*G0k zio;3?u;_25bk{rh6W_yuJ=h3Ap#4Ahu|ovQ68IDitr6|g>ui)1qWBj65rYp*_5O{I z{(g65M1TJ)Ek?|4zfBaAffmNo zv2F($W{)lZ!Ij5?X3k%D^#%mkO}K_x63)&AdiU;0eK2klA(Du*@T1w=WSIs6a#;7Z z{-Kp+2|DvlUsTwv8kA#jazxL+sEsPx)Ciy`AFzn+D4M?|?(OvUgLPuxyx1hY@$41x zyPJkHr8&rF9fW(Dx9lyf;?WLy;m|lX?{L;>Y6QFlILkv8FSQqR=7d41NJU>PzuG>R0q2Hc09n*wRKG8vdG7_bml!xOonX; z!*)2A*@XEDntz0%2A?=Wf{+n_5n!2@PaUQMA{TA)z&m$81FdHJ9$tO)L=QPg$ADHY z8S>RM@smGhuKMREVC=W&S6oHV6Tr*rAr-qi(a3vXtMZbFlE3xXyx_5>4bk#hw?AR| z$AbK1r;@ILdHP0saiL`gL812fwLKR0#Wcj*p~fxn4SW;UgvU{v#}`0X|0lO{_VgPb zt2pxR_eq(CFmJMON3xcZLX6bPo79*K=`^(dwDOTQkO9afh}{hZL4qL{D8QMLEs=m7 zf>NMU|8pWtR~o%|(2G3+K2VwXZEl!wGO;;9k}%+NNdKq7S0YI_@iOy{1_|+|Rt6@W zm<~v4h|na=Aio~xfyKmgFYHHa9C-n*_k2LYflhM`NwNH?3$&5$BYPz3VM|J~ z{Xz8A`cKq6uoIAmdQ9NV5aLKEx!_NWSxi=KA?VzBLA_uHFTT%lRJxGY%u4vcEaEp+ z_q$y0^C*;RvP?$w236Sb5`ugxU9&1sG&RCR^din@77wfVZfw5d>wqOTafp7CYd9Ya z^sfHyAfPk};t_&A5X{`7DNt3!R$W+rh@EBAm4#UEI{54sLJU4S7}j$j;GCZC4k<}l ze5#`dczjn@_gzC^Ojt2_tVBp;$eZYTa`L++^5juk&2tQc55nWYV`Bj;6DtM|{yNYA zN7>nbQ6ajxKmx&{6+uz?`8i{}Q9DGHAV4ob?GGEOMEw|w{5?l>fD$MopeUY@1E=+- z=`o}Ika4kCJeCT!2bh8Wuj>J#iYX89v@_@z2lO@2L~wu$AT{yJSfF$k0ZdIzO+HQi zVpA2csp1vH4OBc<2|y~&%gq8(pP#&Wo(^MdKiLE^3mn*NP?mI$JP9 z*^6Jlp%~#1C_(o$V^^%_!-DQFK$M@Kyu0_{Ffv?Jgb%{fW-ozs?pu5I0#1cq^Dk?n zZuBq@N`rf4Rb$*ykrGPiYx{BDz9l^T{ypLx5a1mqhnqs4lOIhQ5NKBnL)1LmUn2U; zncrV{$48|jP0BH%yZj6^Bl~fPE{FkHoA-JNCGX-BZ==|kN*;aU{}GWO+#px8 z;X^FLj_Au0QBj~)+&2-Zy&qC6zmdw57z)-OaZqT9qj^@~Snds=K5)!Gt^(Kf)XL8X z2&Enc;Oac^__pogd^n!XlLYFISIg|q;%Nw~bYMY9!T*uZ{uFbgc~ZgfBuTLe&lmo- z1n?x!!yX>6ZhIHHD+m($o_@5xSF;I`-5GB!a5YrPuZStRs1fGhg4O`y?$t|xhBRQaC zm;Qmuu9xnPV(4cD5*KMF=zSEu8(U9bzbjqNb_c9W#fr}S&thS@gOQ2zI+`|FFvgF& zOBFm4sw;aZvCIdY$?gHe7lKve_sh~$-yDrR#5srP`^)vgqK}oN22Z{S4i9JO>gwvz z7g+&O40ukN_5rrWe*qQ`a=Ha+$ahcBihq3^_3C#MzN3GBk7Lk0`2b>r&_FjroQfQM z7v*(8QT~S%^0KpBL$WgXJ(h^7vdsy?$+Z zy0j=te5ksB(@49HW9bh50_J95?Q5_7-5%Hhiaz^MA!-@uK?k5BS&<@j0}KO4+&!r6 zcUJt5#Ru?SJg|^8{>SQVBC>;;@WJ?RsmPR4$@&2bR-cM_NZN^j*vgFv)w)Z(R>_cHSoCv0uvIcon?Mg9Dm1}Ug}hW zf?CN+DfGZG4-N?-vJ)T&2FQw}ALHOT@M(a)yn`VL;L5sefni}IximC1pEAA{{KuUJ zo71bzK_qlu2wV1$+W%t5&al%|yiYDBNPaJHyoN(x z6~At6?-jUoCel}zB_9g-C5$~v=svi0aB>1{ktiO*Cbc)fSxG7p-D1>T2^ct~I9`$2_yJ+Af;r9oBPaNkFH7X-xYQo)(s4#YAw zSd-Y@Em-@NE`e|Mmv?eDzl0XNy2AnYIkeax zp0Vnc!p{4VfUg(>x+E_kLW*3^iC!BB`EJwGn3(ke^ ziO6#SzSp*XV`=a!3OsaR1p6MHs4l^>_w$8&3+{+ox#H;+&k8@q64!-u0H>a+GyF17 zUv`T5<{ zSur{UJW|A6%nkaGd7mo(;&u#B67W9NZ;I6^Q??PEv3a*SkNF){KOTBY2bzP3>@%!R zxiWU_gr$&I;^2DV#yE{82R8tx>IJ5#LgV=o!Cu0!Os?$8+wLpl-AJ^Z#}}dA4e(Er z)081sLTVbS2`rzXWia>`L}G*t00v}R^rq?^Y}v2~14k5!y1l1HmT~a^QZ~u~;3wLV z@P|itRTE<#VimVHv33rpmk~gr5Gssw6QPq{E$f9%<_`uGbKnnfBQ;q6?*cRqM!&k)=9G7N%noPY5J#qtCB!2Q(1l)jifqlKzuQ7zJP_ zulWp40_s59Ut|9Ifjgk}O3+Q9p)-kp z()y0wy?5_*&=tbkB_mw)B%J3AS}z&g1TLBnTmz4s!Wqh2VHeE|O}GfnKrY)@eI#l6 zoq3+*&WkX#_mRHk0G5zu~P zE5|;r^%M-_Y$9BC=4cE}T?UJg0Azi{ah`juzU)93*S>fE8pJx@zL`jI5p)<6N8s0G z$9a>dtKszu6ya#WOlA2B@G|5@T_stZg$QFwkeZ4jDJdbb*+}bmu4W<4D_ z0jFSS`jo$wT_tqB^CUrwg4U+}In=9m86~%Qx&kZ6jghow>EQHNc%>7`y@$XhRmF ze(J;Er~c{D%*9^8hT8OLq%hcV!_xu4xmVr?zyQAEG-cDyq9@3$=@gAF-~Wrl73K$g z%`zqTsJGFwoSG&G@7v*0Mdiv|Ue1QsW|T5>I46!?5W$+Lb062ukpes4)W^`Va<}FV zakSW1Cip6SQU?2!xr`#cZCrD*)B+n}R18X=EwGC+gP+)_XmAWizg>r| zW>58bYYtP&9Sz7l|0{Xbq}CqDW1Bf&4>D6j=t&(7mdOHy4%-_lj|4xPD3wi9MGyCdy)mmNj@ZtU%C5jjR7=KhLZ{VUY@sf}I z5oZZnr2x$0m6Zl6RHygNs+4&yP8WB)eh59IYnjahN_^0;;BY$Pz4Ceg1@}wna~z`Z+EwiC3<9Q7 zu2GNHCKC0w|9kImYHKB$zY#!Lc?Hu>2QqbKuK73nX4f+g2r{w}3}L$M;0;p0X0*&_pd#% zyKZW1)KSR9BIwXeV1?iypJdf=#&5&-*;HDT-u_T+Bfo zv(zio;I@N%xy;~PX3UQ{ zZh;2RN>#j!o{Zj&RvELdt{mqNnxJz|1)n{nCv)Oss>NX5aKTB9!-E7iwzc2gXEi#+ zy)oe!m)~P%yzse=T0pkkYE)dT|MpWYQmJUyPYkq%5~&kcBFRUiXlbD;Fl{_M(s3*? zDdCi-wK({k9%vfW;!u69Fz|>LhX5RD`*FF_>z7*KQ+wy?O3>HJeHkxP)sS=2YhthI zd3+4~Ze)m+hkDcT>#3_|o~4YJ>1=#ZVnOe-er6i|kvV!1Aa0uBTkr2aQ`L{_k`Ks- zJFsqslmJtI1uF!(sD|#E)+Pll-}qssyKk)a@!7wlH{GoQEC}sMY(3SH51CoFS;W>7 z7W>R1n#JEk2JQydL~Zo?UoJkvoDD~5ALBV&m#s^yWf>~pzv0*MdVskXCElyf-*ZDI zwDC_1RT|2N^dg?^`!B_%KOPjs?w&9Of|lcXvPUm3N7EX?f!! zb}yvC4jefGP+viaszzhp_i0(Ks>S!meOk4Bo^lbf2e^O2JGnI==ju%GoYjncH)8c9sKko&pK`%cytW%?+VWmfP^>m`X*^SoLxVh@J~uy#jd*O z+)CI*Iyg{*^Bnf?6sK=XBuF#_E6k#K{nFqxZ~*~Y+th|QeI-_JT)-c+AT6h3P;Qta zBCA~D{Fj1kIY1JtyYlXhi<-h`w9m1qWt5nn587-uEEv;ffy$o}L$8j9Z3vFO>5_JM*BMZ{enV-Z#~s3B z0enMspPko@CVurd@_3nuN*Uh%k@%sO56i4-{0r@9Hta7-ZadC27oTcA&a3jfz*%L3 zwO^l)R$EjQ9pMiCpc;9%8-n5;_ItHI^On@ma}nH!-20;`Pb=ePvK_{G#(IB852);X z5&$(yLkk0@h_)Qq%kTOZq1Rmk=a@t_9F8h>cH(HgZue%v0}5y;;3>2x)twG04vs{#I75kLp~T<-2kk4&Dw}b2sM8<5ET?9A zOg3{{$R9>?qFBkoue=EGc)G6z3U*0Rp@=8!W4FTfGzdHV!4+p_Ds@_K1`8FH-9CzUi)RBwE29c$!i5+SkAES5Vaze$u^KgKd=8}b%5}Aztm4Y_%v;9B#v#$%VbcL-T%#P2J8Vk zB^61>cek}PSidDLMWQEz7f&i<&I^DixM<klhTgg{nnM1Eg&zC(23I9D@W}C4BXJ&z#+w6}-B4ttv#=|Sx^-O9t zWyg7t=?OX=&Z~BO&P2(rkGFO@H0_KtE|B>l1AOFfgu`E6MUEwItGu@LZZ|s=I6UB9 z&if_e-b>w(@gO`3qvL;yae4Q`u&1}|MqN$K&mQsiEZHb#>0clLc*iK>?@7F8)3NtM zUP3wVNS`~k>|tk%TtAcp{BnQq=!!&fyA(BeHTCV5r$Wa9A|k+1%`YAvX1c!f$+0e| z;!u(;+3bpNR>NUDu*u%B7b?W-8XMa$fhSp>YN4tFg?gTt+L3pLZIV$bd#A(_nBM(% z{UGd{PE7;y!=G-9n&W!=V;bD7jsDnI^wab{g@Fw+Ccr9IE$nHoOPuCIu_4sA0E{2; zyoKVbxc^(OdqE|)Jk|&VzDbFIE{k_xwDs}kzfDa{I>h*izlMkvCxjVdRE}bGSk?q8 zRj$Mc(w?)7TOieXWBh2hLgu6xt?heOpb`QC=u@k6WdxzOxZdT(LA@ajpWD?g{1%rK}66I4R=$!(StfaHiQn6{@D5aKSUQ4 zJJYgWo%NuImkO_|DZ@Pu=8>b47tb?=ni@hPV1qn%Us0tIQwtaKddNcb1C&e2zIkL{D5iztWU$(4Fb4}Zd^`80{uw$Mu3OExQT)I)fr%}v?L z$^PChJ3V<=Imxrm4hC-U{a_Q()8HG3Bx>c>NZP z?*}qqKPmcI|A8YENStb<=`nZHZz50-hiH8tRQy>k{lxl%TF^Np_;KOXHPAEm2M2L~SMNW!#vbpa zDI)Rfy)1mT@-7_9>l0NSGJ*7Xa~`?c5Vzmqw|-d)o9si-PJSA2Lhh{Qksfh&2bqyK zViJ%Y*f68uRp+NSs+R^(XZ2Vdd>08+N$D7mizci2r-6G9^WsVauTTayzyhYu+&n#5 zMrbtV7#j<9GIlhqKoArmo%h9PyqmFBHfAV&9b9HT>0eY)J9%VCk>emmeSeKiBmKeP z)tE{*Yx`dMHm1f0m|%o>(L0&2`Wp?$jHE;L=B`Y|#^-%Wi<_l@NmC8?hQ^3oGgxCo zo#k@w9JOHS@6~_kh;Rc2cI@=DP}_nQn`Y0=t!Jx_;kT9CP)We5${kaH`dK*^;j_E2 zZm)L7t{(u=PWO7?XFp{l+84inQH+#){Gmu0z&_}j)Dh#dB?kws`3qIx?U`VGK4 zikVg0I$ln(JSbXQVtGCt`SFVaILjhf&g!H5kx9scsC$NRaBnRqsG(uuWN!>f3J8o^SW(--rX?QUhwitD^((ruBUM&vFs@H4qoN?5Ut~1f??B$?>~Ft9j@FM`Ms=nU*jy);6E%EW`6hLM!O=5sr5PIv;IpVYIhDLh z@VMusJ0(H?MEQD8D(K{>rXJ>2o%-hB_GQKK+}R9Wbqp7_epw~o5AzD6aqQ$&vu#d) zok47-)r9wa?egzZ!aXSFu)*pu55480EWdI#6=%#fOSz)@H7OnvesUDNE`7<3Fr5<@+z;GWTxOJch1io)}uy-T_{S-P8 zLKKy>-q1GA8E(BPC@H~Hi}(KGK8U0sjDl|mJ4{+ij7*p z+?{)au4%rr5;DFx!4s9J0-vWFMxzZk@- zbRw1I`hQLqXyBTSug(fV>2D^5Ly(t7=1fn<)EbiNV@ZAE*w-r)1x`m!@%lED_&9<$ zo;KS)p9=0_M0z-|x`1fFaJlMUSxh#idtM3Zm>)4$?GK$60_BP{*5$PaV-rDoP6S#D zALX$9a?kM$FWFBG)5G^l+TMr|ROhT#+yX$%wRGE0-te;$6HyjaI~`#wlEUhKpNER1 z7(e}b&B8QTY`K{^7aoY$-qKA0&`+nNV!(@pHg!6#U}C>$qACoiXMbe$ILh%QbmkCd zK{re(g+uhH^PEttBU`UVgj|C0Qwy~o@70zB53fnXDo`!?MLmo-M&xE-(?l+|yiQqp z{qoaN@nE;MuR33gb9kN~CJM57Z+=QN*(UZ^m(=@>Eqyha3@;~)jp2%-CqC20hfmWr zi3XD|4O)QUXk&z_x4 zZyKydei7uWD~cl)3Vc!NK;F956(0NA2bApB99iF3{f^oT!u)2ql$4Z5>gP`>9*+)i zfa>O7J5EwCg<2iz$3P86?l|5SXaNTnWn4Zg=MWPx8GnI_0;Winw15*rYBZc139mtB zH~au#Gp^M8P~d#^R{CSP3c9Bt0*!8=(&^!A$qT7tP@8o`-T3LJ%-1q3cQMAG->?x0 zx2A6zs?ZQ>oksiVSNj$TKAU)RB;iJ9dd?CuCCAiuY8M}hg970YJ%$r;YBL|ljh()! zf3|wK*D zLzcC#``s%Dw((<7N)1k`yZw(xpa9V69=uKSq=l+|81P2jSyjFaVPnU?iZ#^uTq-9% z(e}L#Mm6TkG}}f5M-359uvH^tSMDY`V>JSrJ4)X-)#j;DPcvR)15bt^zsvDyuzy9) zq-uys<%dREuskSd$KCtHEn1+IYc(&WjH)h?U7I`8EG@*whcqhvI9{IZE>S_Sh?|@9 zS-2O9w#~qZHvS2mN_2)|y086&{W+$U62J1x!`Iuw%5OObDSmED-*=O;Lm3s98hPN> zoX6xys%)=g>DayG)OufkE2V~{=0MK>ettvvwOe> z3wTaOZ`2_ycD-kG-N_cxKh-n^KGy$u7TEO+M_s-gmXw<6E$e!)0IG=7*VNR|u5f;J zL&o5;Yq=s%CwRM@1^on|fk6{Iv9mCe(BHD0x%4AXgR}PZX}_|MF)y?8B8(Fqw4Q3@ z+G<5t-bcy7Q|gdXDtyrq&*Y7d)ZYZ4$-^1bh%>sMY5L_DLjjYd942;~m)?_Q7`~A% zrCUa@*n=#eTQTBmACaJeThlcSmHOG+zkBL;$NlVQ#n-yd)oD2gi*?=Ee||n`B)L18 zhKie=j(vg~?KAQANo?HZ8(pne-2KK(Y~9Q6)j1!yxZi!R&XME$+184kt13Kh<)s-?)$Opj;}GB1KDO%I`c-35W87}{b*Ly&_bHYR&2eO zMTeKmG%|0xrM9#s<$zI!ie-lytcLFu`_Z|JPuX`CmKzOM7!O+*=XT7^X`MLZY1%|+ zM$dhDf|5bZx};JCEXjNxN-WQzK>GXLVH;Xv-LF$U@xwex7P;z3HSoOOn8}eqxMT=# zQ+JRWviC=c_Zu7kG3zqP&ci8>kA)-mIZjU+CMx)rO6qMh9UL!qNrU8U+YvP!^5E>W z%90Flu<9JCP6P#RY7|Q?ml(j|pq@%f?8-mozdNMyUih~9ScS2f0la@_I(^JqPuz$Y z?#NQ2fw%^Ls*?NWB8o45gju9IJBJP#QD-pu(8^b}|73N-zT%&*=&)b(W_6JVqeOU#yT0-A?5Xqaf9bu* zDED*K zdUfs;{tf$sR{833A=j+?agWNtSts$I_zWG!$F9rt0yOrYbvtj)2@+a&|LIcUvBdJJ zjx7J#Ki8utdS1L28EiJbAhyUZ`iyvnIfIz>pXhiW*YP}+jhb5UnY!RQ zu6YXPu1Tm(##R;kQSkW@b&dGurhgxP?E1)D|Lk#RMu3fjWxKsoZypM+or@f%PiLDi z(MH7`j{V>q#~v-T#M&Ysr&&B~>R@=E%4O9$XUe5`_-^jfcmd;6kajszffy`ubncbx z;bd#3rlH5C%|p-1ordWfei1~WnuVcL$%$z$v?7cx3aN0$n{8Q%T&znt_-*4rM*uPu z2sjZz&q;d<9CL_aTRHgBhYXE3O&>;4>g>(66R{=!!w;P@TtmRI6lfTmq(Xucu9kC& zC)DHG!8h_lQ`4mq5nOFvN0x4udD!0Rsg{6i0H5?mL*Zm~QqEp$SKlV*GYoPKjKx_w zR7e-aGKVM%tPZ#$OV)GF#j@>=`>M2i>7T9@SN1zzJmT15#fT83urzuRtT9)RQJy-+ zeJf*f^fVz4{mNg=6ZWd*191^NFmdzO5rHbjnjHNBUeo6OM+R zua!uf{8p4Qs8}(CkAFZ|`7q{i?E9gbl-L?2o|$_IT`a6*yla4Ds!>VQhaba>I*ySwAaDOa!U35 zW+0%segbt5RF^$rEAK9aUkE;-#NoB}>f4@1x?Gh5RF0-}D}O}@zwnd!6J91X5eC-Nd*5SXTH zr}X?M4>=F;15N-tFFQIP$dXBFUDl<#@VBkd02$~31JkL(lMK)~E1+!x>Uo0`Hs+AC6(#@+aUQGIMB(osAAVyt9rk4yiW8 zFt9TI^tt11G|Eh%GJoQ<2$EqK+YlJ`lg5)f!yWBBpMPCG&qgRd9sTg9)&;;Wq1Xj` z_p|xb=c`})*&8}^gLoj2X4vAX5E5%(H8&^6_&%_V;*?(qyl%+eRm4nJrL>~Qg((8aQLQ5Tg&(18~ zYhbC1urN4Kbt|qQ|2^CxWQ$~NGnQJ337=G42 zD>C3pr=u&3x<)6T?CuzNa9S=o^8AI1baZbImRvY+ zfJqKvhSE9kmg1mE-r<<NeWLfGnt6KCfae zrs?HwEv{LcH*9GUf#-)yd1~1S{V1(l3~iaJnm*>SdLn=*Ib_>7WL`qh>Z1$)HA}3S za#X3e-=k$Gvov>6G2RQ+m=h9hu^97d%2lPicV_%}#n|G4r%G6X`WNu-<)IZGo@y?h zsZ-rNozkj3-XVs`dw0iFzp^jtl?ov83~AI2Mi|s7?~+zEHEaxzmKI2iwg1Y%fx_&9!4Z_S-~-AYMhAUByCc+C^p0G) zNJaYA0$~>@$ivi}?(LhnPe_t5q_8|0FleUer%UxuNGYFpORTi|ItVk zAF{4{G2|w6Z0NeZ$DNtTxr;Y$S6t>U?8QM?^7&U1QRKjEbIAYHVJ?0PDvhiD4(%Axn=>+WB2< zy-Tyy1I~_NrAz`e#~GFw85!IA9p7KS%T8$gF>>bc&BA7Q%#eH-)2-jp;cM^G7s6Rv zC-a+!a4U41?8V)Ho#!L-IzfeJIb|F=dJgAoKiF@u&R0@5jryD&R;b>8IHCdR;^X{Xj4yI`I^Bc zqKxgj9T{|39k5fS;;mL32h{R%0gudM7$Ac`iSh`)Hf)XXtmc@k{8! zfa#=Qo5kGJgGNHB)A3~${k=$k$FL0h=S0UJ21Ta#l^PR6dQ68#v!o>LvoG_EW$4$x zj})aRvx3N-d#AVj()+UCGas*H=^rJtbbPDeN!ya;=oNAkk}Z+3{_kV3C6h-%dJ(b#d#4o^x+wVCKHit*hT{Q*0OO*k$=O$Ft|;%xYlJ4-c!h zy?nIQqbo=CwiKut-3cYMpE(C zHRfNp`P7>)al(mxOwcKIf7=Xyg_+c!_8MU20*RGPVVmkj`e2!kiP@%fB9d@k-};8< z5*OeW6qbM+_#_Iu2Wr-XDz&~Ri=KnUTFxIc>(%3p@pne-rgDh&Y_nS^inN#F8L~6d zPA`1WmtpLQE$!8*=B>7FVz)n!wx-WivHq!9yLYWkE~xY}11z0z&bIBfW;}8}CPzwA z%dR{rGg^9~ODD?y_{{*hp0hKuTA`WyM-x3{upNVEINM#~lxmmdE@_=LGAW}B2_o|G z3~4v!g+$+QQCIF3mwI&~ys>wJ^}&HbE9tsc$=K){0*h=pD(%`S!up4{@XjIP53f?& zvMtc!M}b*M3yvt85aT6@UHf=IRIzL5s6y1}Dao$_%QO#sPrhg@Ya~qCpOVBTt4#>w zDla4KeC)<&mV^5vq7<(jd;B{vY9+&q+ht<)@`T#HVW9}|_;EoiQC)ktsfax-Ekbj$18RJM#wjNS} z_Poi+JNePXH1iS-Gb(8mM>Pjebs|zCrrHYS)P`uM)TwBd^0zZe))~4Sma%k-&1zG* zIYjhe%_I}LC2Z_Gjb#U=I?ERkb7@X8%1O4B3t`7)Y|Wl!J^693=NE|)cheS3e$LAe z=$0YH#-c9WtD%{xnce;;qD}<#2QBzaYO$4!o|+Pr{R>mPzW0mc(-O ztF=|QltN5(7BU>^ZxKeEdN3JOUgsBD>u2RKEZZZszgV>~++4pgvF`brUg`!AMSL(4 zK-X}>*!NcqM2-KDxVmos0qw`(9<(?kSg=CMLPw1cv>0B&fT%Ob^OK#gbL`v1AW!k~ z0>{)!(RX3VwM&w)%aVIitozP0+Dszs*pmlJ>s-RLVl2MIiaGJ`2GYk09H?2gHLa}x zr1ZQGJ(HZZySuuC==LHk_2=hMuw{#&RLj7WQq~vYd&=$&8Q_1CTn?FwI2zakZs3)? z0c^~g--;iA=wucQID8JNA91=R?OMbxtTk+Gf}y4tJNo!-7i~n;vUYxS)M`f1-I!!| zdIfQ%c;c@sAKMlKi9E?jCm`E&b4mU}*RAy;QdCh8GdM&dQ46m^-FU^WBC@tfuq?%# zA=xdg3!{jcNvwZ)9!Ynxi}WH8*FIAZyt`|?l%v86a|nab7&8}Ux~5%T>HI^j za~nF48v&nJC;wrYi}|E1!Q3@T#tfE{AEJQ-5-v)dxEMSvg+!D(y#I&Y;*ap0(T)`> z-Hc;dq>He!@FM2yaegb&Vfz=C)>SU*7U#z=ph zpXmJ9*Vh+6I7&s^Ap+Zh$}<$e<7*n`QPmfrWVtm)Ooq#^t?9&2zz>6=G$y-CoF2?u zN^VX7eYiloPBMwqR(R3(a{FD`D~{&6u@qQvL-?r7q(SVXF4JR*wd=;8d_P`?3as(c zlf*N%%v4wvG$GrNhi9xv&-2=~qu$gICN>9tiNe!v{Y75=Jk86S4JKhjd=->Hk2NL47@g#G!PGTbEIItiggI@vE&?<IU z&g*;n&-En^A8k5go21APwQV#U53Fb>M2o3rmu9jaJ$mHW8HY!hQ|PQuv%V}CoQtNu zD=9Eg@x>n`F%2KhwIF0)(s!3`Keq4q#^TkPK(4b>a3W{m-N6dQIR!pZX&#%Skv{;? z`PdMFCVu&SgX0hAz77O-lFxVGx?&w~()DoCRRCu`fqxtIb_Is6P!}J5=hd%oEG*k+ zRA6X|+=V@K09dcx`y)dj>7d2n-3*Gn43{8#TK@pE5t(fgGr=VC7x<~`MAMk+YY^*B zX3s*fT>l+=h$%o8`U~8u%tf>b@g-5?N-sub;6M)u@!5pBXy&Z_*JN9ZA%|bR%h*%$RvetIadUkccBV*yh62G{C#mx7`CSOoiLe-PE^t_1PQsxA8*p z(fity0;SIUe5{U6MA#8mAm0G|g36Z{D_{3xFCvQh{NC&mdlp+r8WKH^F!`H_CE2kk z(7Rt+PGGxv?b`Eu!H)2gCm<{H$ze5UJN~#@#cNbPxu=L2dF*iPvv{ z_jib({~;Ag_!uuqZ}RM8Q!>BaCK=h3ODLlqiy>KKpMYIhb=sO&WsTi;jWtIy3)Ikz zh#7f1fr5Yx)(xX$e+*5sBe@!2#!QCHTm)rE%UI`=H?%kftlR16mI(cYM*)qogkL1tug#zHx)LI#PN zHxN;)9rwUFv<5<^?3%{D***JC9-($hiLrl0pa^V~2zx@?^l)eRNMWxW1oq`1}SUMGBylVf=Y8IP(P|VH7`cZ@m0l zv-`QEiGE4hoiLz2;D+tR90BavSf+yKyNc;&i_G1MqYdmj)*oK3-PH=5OkeMDh!kPxTZTukRfPMjT zA)1C;+G%pU%Orv%*x3W%;n*Eape2d=b|Q+`$HCL<^1@sYIXYRlHeiAad}wKi2yS$X zQj}jx9*Igh#7t7)B+b!?(?K-63}3Juz!*Hk$ZVGe#M5~i=A+==K_d>r*(yE!thsS2% z+3~8;)E(xYQq2hc6gZ*Yw5KFqqr(t(SZ*d2qpr7I$X@d$F{~ zjvk6hfVQ#A}n=5i^v5hz3 zKAsjX6@Jf9Elu|J{81w9)5z@$L!YD=-*Vw7Zwk8@PRS2L#qtXxC$Di6q>5kb^NH+- zM~G;x7@7Qa>6Og>8*!bYERb*5V$gwLg=j<;lL)?np(ZKNZ8&-lGQ1$0_Vxo*;~04K zF1+-vKi?6tq3dcwbq)l*jdem&oq>a63Abq~N}vcP3+6}43;Vkr4Wr^JGqH?1h~;&e zFSZWvjo=lX+wL8J2n#qwEP@qoz18tUP&!Z!oFD4anfhaR-vRF|hMGk~^?*@!1ED&c zOiR}F4Ty-!dqF-9!=%InYVsA-D zcPlUv{xhZ0zdL`CSMZ|pfFLhuU7?oQR|bmZvyErdU;C1ja~%Z~FcSD50d=;w#qAA( zev{DRU^2F&<7K)rIv-WbRTNO*zC51~Ox>u^@aMVLF?eo@%562L%_XiQxyUad$btqG zHFGXbbmS!t)N<0$(3GWHacv0tb{^*WMrP^?lJyqV4-zCiiNLT1K8M6zxJx2_B7z?3 zY(b1fE%q)AGaDIsB7edo4g_hb)7M(Gk3BY0XRLh~C`fo!ur6iXA(+<|uy9Yo>KLva zfY>y~!(n|G@i;~%DDSQ&$Mk`q7hA;zI$%=89nl(hUj41Z~fNhKAS=%0;Kqpg(- z3=v$g$nYaENgH3jVMn2b=SqRN$sNW`@2ocaPX8JGMB$OP4N5}+;H6nr< zVLD8qNMn0I&H!gfet`&@JnP^R>rW?iYX8)kf1v}S_NwPT%U_;rPJo_87x!0Pi$$xs z9|0z)I05qNa#vn$!i}uyCm2Pr&^rS3&1Ts#j8!#A|*Vr^D!ZzRcNRGTTG}eH-?n6R|!Rb2rq!2-; za0AOxp^=Kf59N0c?A}8X^*=-$KQV-vma2|{Z~*<7oA?iD-@NjBBZCzA0dLvlG0qS& zRtp`qP&xtps;9Q#1w)-tw}uZXlR@4W>SEtvu>Xdr7ia$bE!oO0;c<2wNz5y(!>cmo zMB<_~cHrjR9=%=_Kwm(l=mBjulSSHbfjv6lH>kKkmT_-kGPIWK zq%1c{`VdB!s9tOefxId}6}i|Zk_a#3^%WEptRKFew3+?nHp1Tl=)_+Fi+?$4!?Fj& zryV6uQ|S-IT)1zxDUhp6Xmbkgts(`J5PlJup% z2=jftf(;e4%l#{w*Z-sw(b^s{fKhf<7~}m>v4TOQMYg1-9fPjL-GUWu>W2@=U22pJ zSfJtXFDYxF8q0Q$ZcBGJzg}0SbzvjguK}3B25m2F@K6)n{Ys{|~4V8G}FI^Q~W^{PDLU5#=7nru+Zb zMH&KGI$(9T$402b2p{bK^?n4U_(2h~q_p_eP39rmjq&W83nzd0-)eDK2#Dx$a3_>5 z{_RHcH~+3d>xE$`PhtX?S@I9u1Mv<0m12NYUfwp=W&wI@T{}s_GJ=9Y%ex2WN?=6{ zOmflq1NxmvvFZM1EBnkb)@?#`&uaTzV&^KArx!j z-Jsg~k?Jq}8zecx3rXs~jpS`QiSMG(VNjSU1~x57Y=M6l#ISfPc-2NFYFGedG@6Wu zNu%L2scZg!#3_;v>RJv9Bd@Ybd>z3OEn_ifwx-AVfEX~?_5diJvcskCj_ba11RFzO zzhckPc&cp#LfrTVP_d3okcljwdcTiy_TYu>l_8MJg0I=1k42^cpMNwDi(jIu*B?-$ zSFU|Sta_E38z8?`BqPD_a3P4Nc2$gk^K4KJ(KySpJHKI zm$nuK5O$1$dwYojx_3)Wuu`OG!CnacizH>V5MuCd04VQGJeLHOjx8$c6EULOhlLFO zx~MV_cD%mR8|k~qS$XR4wmoSSS^<%yjKSvu+q36*@_JlOQjk_Y4zBH^LxTMu`$_0- zs2@o3Ds}@m2!(@9{~Ca;JGc$Hx4?vU1w%x{-Z;uYnzu;;*8g{n+wh8H6;3gNv`RZJ z5dkLM8OdCOBAH*`l%Z7O@Nx8=ZEX5iH6U?Z#ab;hnU{Kfxb~9h};mv&xwuap8-`^fFC&x{f z5x=(5Sdn^xkHG?v3jfsX9nKSBNIIZ77OS!8dj zXJ9ZM9776qwvGKSijs2v+ssGdI6t|o5s%JoV>%QGL0jEcib;DHng77TdYRt7?Q zgwW_ijlA5*-WGo@ntF{^CYZ7JG$S zXqAUT!|aRT0HBQ_@FCPnf5Xsvj&ucUSaLTsM2+K1fvKjRt}gBhb^o?{CVC%YPax_w zFvi`S?V!kkgE}(^l>n7{z4mSpJ4Z{qZA*|e=Z43{2f!n^$$_Gni6l9XBT=RH0K*C` zN*9D^RS)4%)jQyh`AGKIslX(%Lx9YXK7O-Yx4xF`#{Q-1x1XU`3N26yWGBSS$w5h) zy~7zOp+rv?J6Q#>E=(dGJDK@EEgYmfCpm!H>|x~=pvPUp8u`z41FM?3PqF9Uc7L}F z8#(g+hXIZEg>+{61q%ZfjQ7@yOFM~gWB*}8M(7yC!~fFB4bXDfXaRNwQ?B;jg5CJ9 zJYw_We~(d^714#fzCI`aat9mAIwaH6u<8W%R&IfJ#BIadX&&@Wl&xpg+rYnWW8S4f zHMV2>9&l5>yaE-=C)lFN>J15F4^KSZ1`Pj2hU=zk-Tc(20HhVTbeX85Zy{VkyVUDD z#>x>9=afRow+8p!I%1Ia{2TaENA;#!J7UnoZB!n zuAqmarSrpO&;!K}ZBm|Uo(Fv}B%Z0EP?Z0V=mK^{2}qLa9QAn&{x+#xERZ-CS<$ZW z|FXOd-btu}u^NJ2_qrUwb&5`b^d_nP1O~e!LcsseP54b_oa6$q?|7o32uCQ;L7Uc* zgCLSJf{x4Gyao^-jB{KH=`^N0Y!)ed2xYd2?^-27v|jETv*g`^c3_*t_M)p%($!?P^6|JH8O-v z&Rq&!Z|tG)`CqX?yF40@tZo3drvf_2stB48JjmPv+7C44*}lg*lrIfAG+7{zExs?82(x|Gnb`eTHHtB+zJDxZ?MWQPftA z8$cM3q@q+s7eiUsWIvbmHoV`GN5yV}g#2OCT%@SP6SNgT;rNFDm$qXFc*dN(0^B*6 z08qgAn~>!n$FS_%5CPr;qGdK0#s!jb zF;3hd^o7V?xZU3j{bqo;AyeJs0aTC{8R^^Fd8hhB%l-6(pXB4XwvBP^`oJ*FF*1Te z=v(;JZO9>w@lUtebb&*YDJ@C_%A6iIJH|alDF$#|F=)=awR01F-QeAs_Sn0%JL0VQ zyKJKtK$gKy)giBEf(chhHwm#=rR46!F!Y&2qrzl< zZ1kq#WMSJornj@GJ{%964Pnn|O(xN6CVRcN9Ca4G2s2=~I0&PYik?$~UDgLE-e=^t zn)el83pw0U|A`?LIR#EJMm^pXH~GURj)Bnx^5pE^`?n4KVhR&-T}R>^kfl26wJID@ z#ZWUq$DmkM(CQ6smj`TS0_&XG3B61_Fb5hG^$0s9wS_jXnW8XYUN%Exr?aA9Edzd=SqkAo1fFx6{3;}94EIIrGVGYOipo@0(~ znSFu#wt01CVe$f_bcc+alItjmGaL(d!sjPomp7DgD*<_WHR~# z7hI{IZkA?A5&~s>Zfcc#6h4+I6f+Yd08{B~@5wz>*R9H^wR^~1@CwLa0C*%IMpD&7A!C7ZB?WNd?knC#cj z0Sebkl+chej!BxqLA6}5GA)`)o3dy{jN>+MB!BpKErOZ?Z|ML?96Dpu!Yguz1V`MzCaArrn6 zX$8CiV}+ovM^FHSFv39B3nUPW;1=|Iit%mnOcF$MNYZIg&3c1>_wFx{-i7S#U91{! zG9tl#i?ja-m#zw2P(M)14LNOBmE9#E1=l)ZYIqcEoIc*(*o#$X01mlzc*1!V;p2P; z0s2_#-qmVJ+L{0lIhLm{x-%3;`C=Nj;6k!SvJ_%e+&m0&NWN#E&62DPyJzo&lWV~I zKR`BheZ%bU;mQ57w%4+y=E9RSMJ~`Ei#vu@WBmf&kCw*gZ<1dFdZ>AWZuGp+=A~Do z-6q&uswi@+;8t8f0-y=j;H-KrA~3`}NGYtqeE@^p=?}1TjJW<^Ab^>EE9n5YI#4qZ z7(z;rd2iG|iCDZ)aBMNYpg^Z{%t*TPuCTC>P}qmyWMz+$C{~74p(oc~^C|SHC+JA= z+zJ-+GN9*L6hGmWxIV)Kt;kMAT!$XI1-_Df%Gd6UCA|wz__Z8Zis6D|oq1W^z0c9< zIPidrtUzMb7qD{z@?z#!QQO)#C}~OQ5%xQ9!-slQfK41!<2YiS@_Dbch{BvanX?>lg{l_^=~k zdR=jBs(8t$vp-^GKK<}5{qk91=k72Ek*HJ5C3|Alp*khp4rv}+atqyIy z4m@a?V?P(X|5B^u%^)cWHv87qm>abnTFBSF#WJ;S^ciNdZBKqB`Xydjo~z@#!TO3j zJY;q(bS$^i?)h^ul}rzw<*_SvNBLKtbUWqTt>sDX^?017Ds)Ic$K5CYged)SsZ#=8 zfx3y~C09C0JkknMt?OQ9UNa5Vv@?hP_!ZIBX^)G}&pG#_=V?P&(5HC!z1qEk-fd;U z+QIqA$WzJfj@0C6SJ&02RM!Zg&PHJYApnVBcm z$clk0FLF@(1N-2~sYVIg3BfegOyz;#H|qUKo_2L(Z+Oas>v*KNFRO%A3)GrMd#o&p z*QOeGrDxkGDDnE3l%*5b1KpQ;_ztoOz?wlJb)?^81CmPl{=(THv zwl?aQ8R+FfK~sou|{;M8SH zU;s=m`biL1n9&3&jkQ*~Tab2ekbxF`qOxp}uC$uQ7eai{X>~j0-so1ZledlEe>S!B z`jW&QSE;O_u5z9Ga$YMkNbtld&P^QXxm^eI@I zlvO9bSmKLINq%-JjbQfCRI8u<2^k^$$taE{E$qeBz0zj)FV1&oOmGvGe^0L6J!I;x zCB+lRyJi(xerZ+RKHbi2N$;;EzSNXOZiVYNeIOdzxo4EwNup4eomlAGOg^S&;SQ(A- zNxjl#1~kGVQq!>&m6PXJY!x{JG(XYrd5QZEh9O>_q5)tNhI^{{$Gn94 zFPKO5E><2gUF&uiD>PbWzNv2)A*U>KsFQzrvQ0u1=$i&mm$G@$y247OOy!VDmYcn0 zUk4YCN~jaT=%gQv7P%`fs@f?Zf_}QZ=u^BlIUih;Tz7|F%F03ZOK-fEZ_bE?+wKVq z+)|2*&dLvMHYcIy6a+arVzWF+;>q{D@K2>_2 zf2^%SJ4mz6hI%fmFL>pOZMD0)4^SIsueRgsN%1w{6cAp(X-SON)1ZM7&v2}5SIGGs_A8MKBSn_iPHWw@}sJ=doyI)3FFABEg*Sx>PyDv=a|Ro zNn*!!O^H1zouN&sG?ch(GK|z_*=U@6(7`X$mdj{;15@xM*ZPHNc>ORYpJ=b89GkND z)Iz;aW!o8qzLHV@Jw0(Hf?Tjuj|LWg*L1hWB6_`eCD64%5g!Nl&I)|B;yZ?Owx01w ztZXuF1oDJNidP!b>oFO6;lBDyzT%bS&&H4AVXB$zQfdKtb>(TnhS*7G0R0iBzm(24 zmyvWcaVcxPj3;KgXCxE^J(F~zU5$MySxH8F93#WA5)X}cWn(3jRCOVRU6}a{Xw(P> z?%u;xMp9DL!;D}e0RxPc@sZdb$;d3G@o4Tn`*FsL0l2DK*{7NccEwcjAQ**ezimh{~vzeMa25Kf3e4ZAS#cVqx8GQIgA!>Z_mO160+>$ls(DFrC-UK8 zINoYdj7Bs@KzouXqdL2K?AfX&lg6m5K3=@?i$e|J0@0E-*~Kj(#ai-U?MKIDUct)I z`rc<7POj8lOvw@_*cp!;RjAX@Id7X2dpAoQbuWq7WWu|sXFGhsty07;HRLpO zf(S-8v_obuXd;2d*{NafQFhZ3hXJNZH+n;!B9;oNx!hIXWgu2Dl5Uh*Yg{Khyh<4l z8;>>eF@E=oC$Aoxru(~@kC`9GGm^TMx-Jm&PyzBX`SVAS;ws+vh)siLgK+-6d`yz5 z$c zkwbRT*j&r@J`H?6zTDP=oHsq09(xx9Sh-wzLz-6IFB2bf0T?w%6Iz+_LegLzhw8we z5q2oYBqxkSrLVC?Y+rG1z&$n0kyh`_tF6gze)at$pY}e{ak-?Wec^fT-S`!v)`sCY z+Ow(E*?B5Jefrao4wkEI@Ee7BcnS@2I}D~V5iWQ3g$&7~i`o&&d*+iY(ULvCmSJf@ zm-;cEyZmGx72BhMZx!58`P{dKllUt0)hZcPi7<6aL|op6 z3RoX*bAShNLJ?uQuLPIJZ1z-=S7F>W!Hh~5Sk<8W#`NP{NN>0#VrD!v2S(ZSE^}bA zQ8WxJ{Mi!S4Oojf*Kuhdr}kcX{G2*h-Xb+Kx$WuXws534T(u}m+a|rjb40bvVdIp{_ga8Wi2X{dn5os!D4$*#8%1;MtbA?PIod zV@C2`$!LPqr-lQm;ErnLtm%j9V<5p?RweNC5xQl+5_{#jCIyGd4?_GfntM2F#xC)F38pR^@0vi&1OtOe#-s>^8t2k1}s}EJ?8SIHm|ByWQm1hc+87=W%}MPcr_h+~D?N zCz)WJPYu@5@I7 zS|9&DvHITL4O`P@EU6TmU=gbd`L}Qj6Qil}ZzMeL33mG7v=u!SUcOW$v4FaazCI%% zmnX-zzio>EZ%4lO-Srw}HJ>X0?+N+?@SnyfaLtO-c>9gdsfrVPjS&(&j(s6jk-s<3 z>|6X<1-#YjQ1N^&x^M~j&=lb40!@-Df8&qlmks3+q~H#k&Y&FN0+g@i+kEoQQ+`*% zcpo5xf?QVSV*7w42>}__0=K!qnd6>`4SJnXkt>oHMb`t~ngMU6V?-0Wd3gFpe4~Q@ zIVH1j&o|#hG%rohS6oT7%LM+QU)Fdtti2Wtg%OC)E>t|!2%k?F@@g`cnI7_@H}PN- z+Ie<|hyF}O>`=P0Mp3$nEQ0bshF`sSUu6ULdk&;44R_NvD#)5{CqDmc65+pA%fMGW zUVw@yy5;*pHu=nn|5ky3-pl_B?7dK|$n>+s>4|VSv$U;h-_}8C|V9WnW zCH`X-2WDh}_MjVa8CltA+jLyP*z$ z%ELcXB!8^`JRq`jvpaeof#C~wCk*~SZ{Xc#_yOpf-Jcu$Wz5UFFZd`O{l|L!yZqz$ zwrAbBcNYyG?fGCZ-fgS{NB`E5_&2!%*fqps+u&}W-gxR0)Ga%FOd!ubY4gFIA3sc$ zO$&I$KW+m0;s509z~A4OE6+pQyGIU=`tuT*2i@_1ytV)SbS=XWj;D2V@Wh9t&L4eG ze_rLE1oZvGgT}$`Gud(T8z1ig;9|NI8t>oQ2J zCcq^s!dF-IP@CF66l6cV$Nz2D-a7~+yztk$;-{!m590w5~U)X1Slo>v1z6t$5Cfql&!o4;?(pO&X{sV<{ zzxt{2e_VifclXgv-SAEfspRD~NrL_sZUEr#3r z#s244@ZkzyJWGc_iqMvwY40hpKi3<`-v72NJcyv6`)K-ke+*8>Dos_fBrX_g1*Q7OGxYeeqC)yk^CoW_U^I%oznhyVzP6S z03Kg;;ivZZ!PXxV$KG@He^nlU5QfFGPn3VeEr3qL!w16taiiqBynege&MflQA5vh} zbZrAI-3DGP?7?@RzZWc^^j9Rn$;p5Gy79w5Kir`h2{=WWw4~%MN{bG!v0Ut{T;iA0sGs2Ixp|1TLoZxD=DRb1@)H^t)Z#6 zUC*`i=)QFh>cW3q4x-xu%e{4PI-*LX zf#ld;*MIC|SvN}%`t=ZZx;Z!&jpV`OUv)n^aFhsiy(ew@gGRg;{^{nwJPMGgzbyvz zs2`-jf6Fcs3(M>m3jB#ar)HMw9lW_O9t^rfcArs>|8kQjkR&zDY|#G;KIA^Qh4BHH z`S2D0(HLAh@;LbQ3FMv9$t)YEny+`8Vqx)DK~uQ&r~G?bVCzoI{~`q>zEbXh&ByBj zzUtrm34dD2%ma@x>`U{mH929f$PvSHhh-7Y4Vh6ACVskw;L`-~PO#zU#P4NY1I;Rb zS=tW|{_#J*+r&qkPc-YSzo3h-I%}O_sSjlZmlRD+$G6hn)3Q@akAvU3BL%(7U4+u1 zUv4oGmirc2*WFX0o@Ey+=E_qqF6F=jbKkE-A@sTsHdHTLyvpSPI z?q#PeoY2$V-wvR1ZZ&veHkLi|}!4YVp$-^I8`l01AFZhEF`cFOA?;qqDzU5`K za5Cn)D+2B1o*5{0cqA&)p=)h?B2P9(^Gj)N%T~(E#Hgy^g!wk*nq(=wkc}QPbsMY; zpJG0g<=fY-Hjo;eMGSaau`Iv8LZn2haOP|_IGcc7!ky}Pap#0(43n8;eA~2U$HvD{b8 zw6r1D)e5WcFGUirDQr5mT{?o!_Zu1&QtVP&u``h&&WNbS;q|P^l0ph2*I8&LPA z&{DY%@n&0?2zhnt6jdl<<0%Eryke>?{q0WXjFG;dx3#tFa*dosvYNwmJ&WI*gnk}| z=`b1cCrw6DTu@kluxZslZ3><(tqu>Eq3M0ThScZJ4K_mZlmqm_c-=I}8OP{I9CiM9 zr~H%Bv2vd-YoNqvH9i<#`f*QkCZ5xX?(lbWS}L89I@WtCniGZ%t47mih)C>EoOODE zxbyN_N1ASwh-2EU+&mM>S8^>!2L)Cyq%GwZ^A9R>!v58%7)k1 zC%f9(?$#HCX9)-J`kim#y*|?~9yFH|fhMI#fJWLlsMB!jV#HVXu%F8PxsBg#gMYFl zEuX~b1&(KQAc(_Dzwb%ijEbU#bUx}mFfh` zS6CL{xarhFZRb=K^@x<(?>r4B1f~}C+Snkbgw+Ymgaj}~j#uvgtx@A25qc;rFtc_C zh7#Rc=AsWrGNgJ@{CRo~VaA(5`(n+DF;)bw%tM`dE-Wn$??@!!<^EfI9~%fAop_|yJs+P;2jgy`lt^^1wd z4N{p8jrxvtv_^|Ko5xX^uXFD<2>V1uBlL8!@H2`-#Gc+_V1CnE%E5nluvr?@6?wxi zavg`4E*gQf=Yv7tD()tDKz*U~dM3#se5U6ZT{zFAiA7)C*!eYJ$a9GqFOwL1P|%jT z&{*lVlHZqIJ5L;xCXR@N)ge+=FIjY1T-2rk!?=QE53MbK83ps+d3NW0=JmlH`XIm` z`zghrEh!HeA=cldpw1XTmZPGvvHmw0L80cPBIInHG;c362yETS1YY&A$0B*WMmH;y z*0nySB6IIH$gRey9Hr#mZDnCuXLXOS##-*^TY#Q0+fYj74*An`c{Fvq>)EoXXP%S- z*~MSbNNTJT3i(w? zg-fE4X0UTdZ4P|`3ry+WT9HP7=$oIz)6dj(*$lV#F?sZhe5-qkp5j>Sh+mk*d< zMA3v{Okf0G?Fz1h$0v_>_y-L|2)WfEhd@l^1|Dy5p8ejVzj*pY2Jmh!HM9T-(FCVv zjE9Av!!l{2s}_Skfq~_RHNVpdz)=*-GZ;w_yc5Wf_?agTa25n_>X>9h0E6-%BMbDVeqk^@E}Z>PR#|KPX<{<)5BeXSo8eSe zo!cd9HF(vBFVUH_N<8G$rORS;Gejid$}?MX^iO@3!+8Dx9?)1GiuZ|nxu>VM5c&=a zLdznK1rE3j1DbngXuZu`Yo(M`l68wS()rUwFt_cwR@!Dlq)Xj|E-A?STV{*_?_m;Y zYbH6ceG+4#Gy6bH*(X}imlh-=m_iFB@b(M*s)7_#*R|i#B08^h>6%@0h2Cp1;eEZg zVxq@-Z@m@@D6?*EGWIEce6pr*CL*nQY&UZZFPwfZJ=lVsy!z_cV8r5+5L>CrrY_n5 zOea0q^K(y?Ef-ILS3aO^Q31X4n8?o#e|-Q9FQ5hEJkSo8uUrrA3A4WWEf?b0H5AFd z;#`0c#mRxPHNdLm7KtdjYeq5yr>4yHIt)tN6=Z*$8YrPLtX5JiMV#x#d@#DRK|>L& z(s8^*F~H<>kqBfLyraZ8y~+}hN;S{9UY8|30a}OSVJ%Bwb-5@HGc|mhKY+o6^J>bX z(K86e*>uZ<*2ms)&Jp=C;}I#LMZQDD z>>YX&?XHVPBbpOs{te!u>J%VaAGZ2e_e^4(T;KjU*tk@>mab<#)x zj1u&QP$$YrYqw+gDtRSy(ZKI1YqS{axSNQ{^|^<6$n#Be>8j8U$Qp#DPT&#kJ~Auer#EK`H(pX#xE+9R{fd%a76hw$>9d7y6n=OrPshP zRR}0b)XkhZMNTx?lp=6lA48K>1lDyd8@ywu2qh}bTK>f-x$+39F&7EjJyCej2QxaE zV#5qFsL8SdOS}hj(s+nn*A>mP%)2}k%Cac9;#=57>sks6{L;qN*mw4`Jtd92lh<0j zpuA2=!6l%W4&UDl!YTPXnR_#mjT4TA?}(%sgZbG{73owr4QqdV z>l;6*54iyTTwj1VH<-sBvLn=ka*^F#2eO5!EgUD{z2$s499xoly6&bO>DIo|3bB2(h8%k}{L%rR(gPtQS zf7|FjUAlwvt|;lWQ!O?sYEgGK{;Wq8{6C;VuKoN5{kF z9M5l{%r=eeU~ihVY+GM9>7ZMZf_D66;6($v46SgTrhPPM@;Xc60=gt0ap7GzA2 z5AvV+Lz|ZZe=kk3l^0v&0?x!qSLQ2(!+L*daHHv|CQ&nPscOm>sMcrCBBEDnuYb zC$tvRiRETPEIV2zJCduZ@TImMSLlE z&3v~hH9RpIQ%47Rqq;dHN(sLRG8=rABdNEe7xz$qfU*t0%L-nif5YXRJF*^&pK4HH zal@9gYMH4Gd+MVC^e3|_n9tN3hyimm-DLV=y?(^InpMU&cD6?M;i^$6H*psGog3%) z+>=D(Gq9U;1oCJ1o$s5(CFkoX7^01zfoZWe-C7WsJ0JPthgbEywftN}Iqa_G*vg@Y zZ5DvqW={u5(HSqBu){09RJf<9WSHKfWDUxNq{+0=YKIMFu9@-j$*}P=^zf$zTj6&=#UolkCh@?XOb#?pf|y|b^T`lEUU?!UZ8+K;F8I?Wn2W=0wG{oj+;g+z z&F8s*hEo85Of7&qmvo}X0PvfnZ95$329OBdw98U?Ud~*>UaiC;x7Bp}@cq@fZa(R! z5js9eMn1!VsW?-UW1WF+#SmQ#82chs`2v8f7hN@X!&%b0HMR->G z(wPxl<}QJNp7<|RczAg3>OMK;_3EBqB?^p-qEW*O!d)KBKl<|-lbER zk2+M~65)$%q{*=F57w)b=BSiZ_0C77_<~y!TBH3zwSx7CQ2U$l>3h%b>BSk&c-^3v zDa1gNzu4C0@t?%h?S6zfT8cQ=d9!ZZBNw&1V4<4-NyzxEeb1+TLg^i&k}l)FmEArw77o0>|t z2c~B6IE4~Pdo+uC3RfLdpJ{Fwg13Vi_PT9`Zf4H_*mhV;QPK{_$vMOaN=wETq3UNn zhO_U{?g-BgM&PtvCG8~Aicn)3&waWPI6YT#sd4&GmYsERiW3kJ&cG!BSm=-iI4inc zqs*N51Re+wx+=4=tG0FWBtWkkmpo6fnm6E@Q3`GiFhT~G$o8Vk983V@~%8%`#}B9rrZxA?uAb z2B*#EbC=0=S=N;eF_{IbfFYcgd0XW|x|Csg8Qr8v81dX60>n0MkkEjAPx`;pp%RzG z0dtnxCK)0E`qfihVSMJW?X&e=b_ziLYb>Dp^YG%$=>D+D>&*|EP?Mjg)HE{`HanJ- z=Ct7qQxj!X!kQQ3{jcMP(Y)1pEDC`bXcJ93zka&g4$N^|ej_WGGV1seeTg-iBUk{v zqB%!sjf+r}f2aw$#|7xLa&X}0RS+@z2wM%3`GQ1|RVQ|MoZ>4M_4cfQc4_Fzp2<9) z0g#Z;ujzW*UT#Ck)TpRVr1$Gem3z|R@mXP`zr56dpK#v`QUo`wx^H|EbN)-lb!swb z&`c83Tt4aJYEz-FAUkmpd+u5h>ycn)5bkQ?BDg;rpFFM!zioP98t1i~x4u z)1+kdp-hzjSgyv3u?Ub}_B1x9$@j=KUe%leUyMkZ^GpUIf#Cp(K$NuXY<<%;T&=90 z(>A?3`USC5Bx5|(=9^)uW>(VKKYj#mLBkxvTcOY3T_KT#cajVG5P%2Vs~D>jKo*M2 z5fsPUjY(`v@=~pW00)3!UB1Hk1cynYRhoon5?3QW1$ZX=ja>MQ%GI!vvX#Mk+_St} zqxqO3Rr96l*l=8Pi035;yLKQK9052h;bs?*)4lBv%FJ=)B}UJT-9(f%>lyU^D(t>x zU$va5ar5D1XVX%Xvcic7f0I=B{7V(t5U6EM8<5M)G7_)2M@qN16NHUaNj$%9zZ0zH zUM5K_A@CtbzS(O1g-A$;f?c-svjLFC=$h9QzjVmGg531uP0^w(%SH0S^os_%3#Gfe zNT%*3Hj9&Fd$KpkJG_q3q{#{oYi24l6Aad_qbD=zjY~fjfS7deSU4|B+bT2h%S}ko51FfM)U6EybCUzI zt`nyyok?P9l}$%Z%vjINX!%9rqVZ<~B{W9#e1z+O~!rb%R z8wr2wJ<~WFfR2*$$mNvU3egPcCWM@7JR@0Ep`cC5-a=?(MA8Im6!eaG$&w2Jy?Ip& zP@7I0@S`y^DJ#j624qOwUtGa6^PF!+SMJhR@B36h-6`TTL$20%u8}IGIkGz23(+(t zHrUIh%lYb(0Xh!YJ93sG`N0Bd4M1Oo5b}qbY7rR zkle)cuESAa37`RdZ~j1`=0l=EPx-}zi;&9l+lsOffw4F{1GG=yHhyA)Q0zG( z5taqCoywrX&6!^>j>TIS5y=k@nJ3tyA#V8GlR3=u)eEM)C&xf5AF!!c0TPNXY=;!A z@dU0>oM3FccWK~4Pnful zttY|bJfStJ%(2>(WX!aAa(U_aGzMIfZ0|9`&a#MOSYby&F9d*U0Wn+##uYp7%>%t% zeo2_4u6QJfsJ4Ux9MG)i)cy)IjXg2}YOcAOlCJqyuWFuorF-x#xi19%ft@zcD3N&IiFb zs6gGb{z5EeYvYV9GX~N^{MqKR$8WMR-0NpmjaM9w_#-L5hmjBCaj zS~R1~zWr^y&&4j_>WmGwT8SU`_(Lv$zO2ve1jUV*gUfBXv(az2Qg?3O&ICmz#N(#V zw_G&8KR~MWK~{lt3*5`LsHx^=e3#T$BKFCYL$;V#LPb=_{x z6-oXjC(Hik@w+LVAPx08_r&C6=lEg#4%Kpzk7zs!;kIHLXRH9p-Tsy1?YT>Z=)EB# zU;M0!Etv;e84HtJim#=tY`Rx zlGBo$lI6dqwX7c4HCB!u*X$DVF>y33}d;IZ*I zB=W^JbF;#D1H&gvZ(2ca%)nvS{D{+1RfgH2`tcruYx>hGZ(Uf2lqLnXc0CA5uQ->~ zyrF>N`BbuVb&Axv@-k@sWu>puGATWY5$6gs&kdsP-4rgcD97k4E=;H(K}bIi=2w7H z)wBI-+r(7vGxfHE$*`$`@LO}*Tv;XVTMgOv1u&l(G|$JGr#Hvi*jXDqlE;@zOubZH zY**I*ePH8knvK#rNQ;_(^EmP^B$ROmCAh9C}OnUbxXdHJ8&p1E!DEqL7@st{!XUaQp3mE9oz&x#YA%t zZx9%vfr`c;9}VA&SS0frdo>*cLMy{a#&SNP7tY1-3QPeVEg)`d=^h=#N0=ipS)SM^PRt!63Qa8?tCLNa5uRlKei$6So)I{t^ zRqrY(9eQ*tZ}V2auxe&WrQhu5=YfwAAY{57scC%u+2+h$epkkS|6T{NVDswp&+H7+ zps{r$uL8R|!I%o~iXbqaz}y}D%c;C9u|vxzFRuJH0m3HJ)a5tcGdI5c_2ZdTf1P== zu?r!9qqFKn*~5HBhM!3pA^%*TW2JLSy38ZSwpU+VZeVw#d;2iv=JONCPV?s-t1N=~ z>vg!^gJk58-)u+e)0#Za5bI}yMG5cMr!$qE+D8N?q|SnWm0Yf!G=fpGxC(CQu8cjK zw)CDlfFttE4{I;qZocGQGl`h7QX+&HBs7WykPn#LFv{z#hhb|t4WBllIx;AY1n0;l zY$d(luYPqH1Hx%n^69vol-cq?-@>fg>HL%eVw~E8i{9V9hRiZ%LY=zP5<5fLa-&g( zv{0{RE}D&OloFcz>U9D(Aolw{J>eoVA0+&{Stu_8mYLPCq7~)I+{-SK4IS!6*7Rs* z-!hg98-A%yN-@s&lxbz2x=Gn-897LyIY)dh8|XQT3s@)P4^ul56Z>tJZ8x3IhsG=v z>ikO_cp^!gxOXa!xXvsc^j>-3jd=Qum|TzebH->K{QlqZ?e~!4357fUWwwRHaD^9! zF+sANTJkP`-@;*TYE217DYRLV|AH+q2l}bJFYp1)UA%EL+U}m*8M_W_;-kp};!kQ~lx#|efvfWG(-zqmb!ntzS7_p zu*xptwOsdCJ2~0zW6&_X#xCd+BKJu%7rrr4yIb^|%@K>?(4mo-i1o`$T6IRX<(fU_ z>jUV8I?pH6wgf5sVq6NG5x4t(zD-m?f^gZO@7180J8{c;@l|3%8W@wk*ZP2^*!1NL zF_^kl?jZ+O$%pgt{^L|On}d(R=>ee&!wyifrb2JGvGI>B`kQ=QqPY5|xbV6Sm2JSO zW42_DR#^ww+kaQnw9d@i5k5ZKJztXsVFpOG^xE>3-?#hEb`(POXJ)%g1243R1uB=s ztWZa~rc%mISg16H(_r^Kn-|zACp+L@iG;?Y9l-5aSg$lFjt^Z0qF5MUK?jvS=G;w# zM&kT_sQ4wYB@~r*(Y7w|aRv@f_yYA;BR{zg6&n2jdoZ$ARj~5Np-ItFt$xZUDjzcu zNp~W;1^y7;LLTlC5K&GekNoFfj5aA;bjjqGfijs9b)K}W%LEznCa*ZbKKfEI$+1kI ziz--Gp1zz`pzw~UAz&TP_*5p5ez<9w;N8<6dA@>`X^{}tPcr2MJJqD+WfYt%8QOL8 zi!6A9GY4eUcQudiKUF0&EI-jFFefoQv+Q`(i!zdR(+JaN`$#VkCc|n&{(I;4)b0xitnqZ zxp$xwKEMbL*GgsY6LvkoT7l8*r~c9x1{B~&R7T9ezxu&2ivckI&}7L$Gu-Rv-xC$T z?nmSK$?Ay7AXt)lb5lM4bNZHML0EJR>-3$fLdpY9a}6kOMJ#yA(1Q52QbDxk_%6&v za{SW})lShUOW6neyl>bE%bVsL2~&=byngBT;gvnnchinRPNtF%=C?f{?Yuc=^~=la zPnSk>acW4m6vs9OH$zbvblL~jz@%Cyoy-FawdQDS`KIZ0@^rJLTmVJnIMM+ zP~A>evRQvEFOyWqmKwh+7Ik#Q&lL9aE8EeDLO~L6qg|Z!m6^!F$wA$4k?;33g?;aBTHBy9& zYnJ3Qo!r*P8t@)LV{K2lOirm?f#0tVg!+19`m^p?x^?#a@@U{R)%E-W=hr>p=mFO9 z`69IktBrt=(wj0z9oi#&npkpqc4hp9!TKq}bK=+cbU-HR;=&!{d!K;w-a_a4t*i~F zA1cSB@4oZbgJV9lyZiy-ZQa!4b$8*x#icVj2rzZJ-^Dw-<;M2QI-KFyx#bqA(X&#M zEmCxQr#^?fQo6c*x@qd+W6xB$NuB2LX(#{}DG35B5aS?f7!JE|VNDRn&Aq!pi%oSP zdV>rmqlRi$h7syg3MTHKzYA(B8@m$r7td7Yz;BccYBN9~E8>+2wNhsFaY~=70aTM+ z9n@nJo!ofc=rP_bpGaB*jFL0p$dXHsMQ^qcm2@5212k8)s#@KBVny+<{B73h_k#pt zLig!olCRfn?N+kSCcS_h!-xddTmGmW2yM-9G0aXaQn-6&{H4dkD33Fw`j9gVPj4d5 zmGN1krUK;10ukh%b(Rl`IfU~pV9L&Go(4qpC}Uc$i8s9Vs$f^vvsWW3mWb)m^QH9l z+a&vd(TrgxhGG#!^N#2e?3)Q7wXh0FuPcY zyi)Nk?zBuTV0Hms(L=755z&dI&Tmf{oSVs!4`a8=F1#eeHdR6n?A(5JNpPeIeAA8p z+G*ildZKynQE#L%&S^ULz z7EDjPSJF~$iSb}6ZV9mpI<-!`t>S8%o%vfw`im5}ZnjWXG;UHt6X8WZ%<8^Gi9XjM z|K+noOF#W`QsUbq;lgEV9YA|Fk1GSJYY2w>%ulHSB3)(&t|BgzTg35U+H@Gqo7T1d(7;Hd?=t{LRG?=p}o|3nihQc~8h*h1;WEZ>4N@ z70fK9^H)>4t_}@>-#`>n;Ofkt^GVyDv(7*-)I|Ajd^*S1QmH9mrYut< zNy?Z1c}XEeT)r}|)mgqx>ff1jEI?D@UUJKQ_`z&}o6xk&rPR!8*1?8U9`tLwuUN!c zWb1gJvyD?o#ibOA}tbeaFnNyI>MqV8|T_=bZb`8RM7-@r_pE?(8V8w*8 zmLO`=J>At~jvg19cSWtXY!fSvOhHdIvF1ms-po1?FBj4$uaPoW_8S%JK)#z)6x(QP zC`BiF051x!d#h-v$mJ)KZ9=l4?9?mU9^v!i**n`#hJ?&k>q`&<_8*OvT6%uJ!JV}+BOZDekF9;)XkBY6CHJ)|DGP*ig;1^Jl#!Qj)eV?3t;~M4PJEb z1z^AA>@U+F2X#~O<4)2IqoF`TM-Vp)?7- zx^7+AIZs_XWw4kE`|-RQUHDz-~S!y3Cx1;d+TvkF>d{n-7LUgEQMaV96sZt2LaR)xlF|k%GsT z`(pc)K{mWP&^nL|8XkzQS5AJzPhA(%fg9ID)DZx1g=q1lWZSc_Wjg&Zdm(oWFGd*K1K7iv-dtYzek z5rX{y4dAWP{T|IDjYM?Yt>Js&mqo%sSBRu@UNR+}qx8NEFd9PlI#jFjZHKw-TD|A- zf}QlUP6a=H5;L(WOu%$o-HFB%)^2s9JVt(ivX5JCHJ@B4e8#9jra7a#92K*v1)8Ik z8$^2g>AwwC)K{m|sA)ah_rMIS+2y}dOh$42BMtv+A zBF>+qLhhmoi8Eodv*DYi;3Qq3sme-QOJ%Zu1f;el!BO(Ja!)0dS?)N5xuV&gva=1h zw>B=NFtbp?i4E)(^Y(gnv3{CZGk*lYq^M`7yB3hGWq5EzozCn9F{R097AN&~&PoZI zyxq+Fz-X$zX~0gDe_z5a;PsKU;Qr+M#p3TV=j(#Bft3?C2n=+Xop8CDZah z19@w^Z}#l*iP-Vsq}X1|6JCEwoP6B8@hzA+3Ql`x70SU9PMZvzi9vD*BH~(}E=&3z z<$h+F!~qFa-iUja)P7o-9eL+~k3RcBQ<33lsxu1&6|+OEsN1(sVQ~>*cLH-X(Cak( zk(;Xv^Si1mqE`E&1@_@3!K>GyVWp;~d31>c@z=vBKC$stYGd?hSYO#7uk%)r%*vzA zo{E)UZ=Wv>BS`Z((X7!|VfvQJ#}TjFm-T*rJu!)>!SKFL{7y@6rI5J_0@Zh&2Ew&O zYD^!gdHAh_fmIo|lO?_ODuFL#Sgt=xv!Y=r-{#sx3 zNcnk^_6+U(5-WZ?NbgX68proDPew1nnL@ZIpU#=KRGFL~cIeazAB8GIY3rS?s|C@H z3uy&mSwR-S5iTU3E{^=BuaRM_><{Gk!XtS6VU@HCFny-BiwJlf8|fQm70iXY@oGX7 zpR>wfEtIN+8*|xmvJL_II=zvhBg+b(`(QdJ*(H@9S-33|Cl26#fFjG#oOL53L_c*_ zddJXLpoQ_>0l|N5w)!N4M6 z>*)GKPI<**mXX1670^vOD}QI3Q+n_VmGuG8Cm<-Yn^7xgJX7IUq%fllmfEg&^7MXo zAIeJ$4REZ+uz_*v;oVOe61^7t{Fs9+UW$*3%MQ`Y&v(k1jPH0$^x zDm9F>Yc0KC(+I0Cs?>RPwFY=H?A(g^vD~5ApiM?W!kl-kc|3%m#B7Vr zF;^SS*$vkU&=Ih427Hl$rI;i;H?8xF@Gi<}0u+2yEJa2xg=yg|EQZmhK9$F)lAbVH zN#Wy|mN+Qde7MzOaLeG~7y8EXxHHy+dKW$AULZb|`_H{2nnnk3Ml;|v=s|RI?omP} zF1so8OX=n%N@ve`Wt6^XeQ;4)kQQB7YWhR-bi|AJTfSkAc&UddK`Gq?H@M-PAx9qX zrrz9@GVDA|z`}ePQcwBS#xC<$YmlZZFTA<0e?w17(fca}6q+p{QzCi?`k|SmoG%&V zeVnpVm*XrDQtnvEy;kZ(RUp0Hu0X2se>mKvfTM#}hS3qF{RUm@>s>5+rzUMcYYzuU zF*eKE?|0wdyVMq2Yj!9&hZxMKx@x{9!-FAe?^1FtVhTYh*z##18pnB=Ax%H-FBd6(_z3Rqu=gleFrLk zp>`|Iq}h2CL2zTeQkMG^BE}+DEAK)kmm4ZNwK=CKu+vjUot?0RSh)rQ{_*+WWSzs3gy@YwHEPNl@(JDTLo5+$W2cTb|)&yqtmDEy^)LWFHc>YF{w$wg76 z>aq7DOxh(yERoX?8&KALy~s09d<%LOF6syA@}fnk`KaY{x6IV`uGmj1?rjw|9MC1r zs)*E71k`KrcrTA>$W)5}V#wT{RPFc7YZJ80xdKYyk?*M(f|GL*}4+hJE6K9)VVNX4)N;u{E-o1QZ7INRW zh4H?1>;suRc;p3Y>`a+{LDfjhnx@?WtzT9t?#!XzBe6ECOXxZ-w~N)2H%Y8VS(E0y zRb~g82emtEvh1+M8$nx=b!ixBV)_#ff5lC9CU%n%PcINAVY1PQS^y^M_vtUjGE+gF zfrsh>#6HwcH&U48XfT;c;=Zc*4DifKcQPAMT&CNCl!gSkbNyG9nW6lKC3cr^Zs{6b zB&e9rfh#d#^QM-~9WG0alPT8A^1oI#ulhH%W;u4*Xru;^1g|1UKSxP?C-W9GenGk^ zYGiA#%UjTE04g9Z5&FvB?vtugj6#?X=FOgj9)35So4w&~* zmh}m+X1kQ4uU9_a1r_dGrBk1U^AzpR3LzLcA#4#@7=-^GTb2EEf#wl*!s zGv^2*UJUXWILa2#W6uV^cFH6uZ*c_Y#ma@y*#I1TaZ7dImiLRckt+G~aQF`kw-H5j zeD!pN0S@G2&((qX_7KN=R57M>kE^}ZOj~J9Sz*Jnr7)MFc+mgQjAm98FC#-vmUkBF zd@CDpMLylx%Hb9fiq3aL=&zzg=L4b^g7~XVjL%fQt1GXtdk8FrQeEK+5oibJ<$1a* z$kI;rTz*cJlNDakdCTidGn3I-Z|0O18h{GOcXB07 zk5lJO-D`Oae2L&ZaD=0nMQ;V*U_@E%$3^po1U3j*LC2pW0e2*vV(Ritz2ja<5a@}s z+SNSk5^~VLMi#>#@}IRXXU7hd*K%>xBRKa(&?ieAO(X;jK|ruHM^ff^vcArs#40=;xbe4?KQz z5FAT#1+6O&PN%s~|LJbM7^mLT{?9QZN1ZlR{QGq3<>VG{rV<7CFH~qE`g}zY_m$!H zEwICnSJs#PL$%P#e8tUAXRz6w7TnNyTppvd@~J!}?5FEGF|4cYvx^%|p`gd}p1kHPvIFm?3^ zmP@7jG=o;~BsU=wta9I6P_~vcJI*JWhFDfN8l@_9MMvq(E>Zi~vs%6kknTewL#;f-Ap=>Ci-&SmYl`C2IzjhL>%!8Eg? zX--xAb7=&uT*3gs%2U}c{h*a!+^Cvc0Ksvk7SOG3J6jkgOi#h&Qv(B zCxSz#*bU`ZdH|xBdLCvMm(fWvf3-h50lPkqCEeAdCyh)Qf<_UFxdFejTFW{t*LvaV zy#EG>1?A@yaQ~06uMVhc+rE|~C5RFtf+!%}T>=8qjUX){-5t`UbVzrXbaxBV-3=<; z-SMr%Exqr(@AnV)+#~F>&ssCa9COUI+4o^1T;`kRChek^oQ7TXYMqQORm~h#Rx;(f z6B$bUT9CluK8u}vtXCsaC;@icKc6d9Bsy+4*@hXKFGsl;n0)cvaj`R*&GySz5hM8+ zue_tk1_lqIgQK>_)sA}sKn7%aed1!)8MqpCeQXxh&vup=PMSWm18fkB^Q@B?L&>uA zdN!Z+c%=(v7t}q3JBKsvoSE?lUxbb}qZ%!$s~Z%SilnU8fU0G>=H``cJEuiA^=zD= zN}<85S!0W2vLYCwoH-rluHR=Z8@ZxY*|&aBaXVOZE1k+9W&8A`kMPPIHf3x4)G zMKR`FV#r0p=Q^YhNnv7I6Wuy>H@j?EtSq!sM! zZ4}aubFGQkj8aZBDy3~-lHU!Q2ihG^uanz~#IxwFkb4>#rI4>|kniRW>dF+mM#nFJYiJ6|GB?&RTq;B0t{UNjBeciG)r8Dc zAaLSKID)C<4}6^%K@ReglWiBphZW<~HcGfjD+xbtJ;>l3}%u^$WMxsMnP7qD8^L6?eTjEL`c z2l4<-1N-*HuRj)?VB~%7K>mE|b23B(lR~lJaurM6$3@omyqi{~Ld0ZDJf}bUy%Me3 ze3|oZ`FP%RwNcl+UDDFGm~F%98|JvCMEAQU)BL3+)*E9oI<1K0YK4hQ1`rpk4>Y`l zkssdTqh>bw;+NR04(OQHN|+%zQ(Aog6uHv#uB@Y^Pxj5_h%8sFOQpm6o9De>LvB)? z8f^tslX+ouOsk0(4c=nEdbe4k`vIQHeo!D$wQ*YODa&aDro4NuBK@?i&dr{;iE~v% z22JBkMJ{~M=Jv2q1&gV%_K%6FbYEk;o1p1~#jw<7LiB}3^pynCr5c-yQaPE>aMR)C ze&?_xhk}4l$P$Lv6qYBn)71y+2xF|ZcrJ=e8^UegcwK{q4WSruEWRNEg06+SdSa3e zHTriZ8&ddm9CPJM`mZJityF@^+I9|xwx*O=iZ4#WY+n}Jm(E3-;?U_T?%+rt1DT%{ZgcUono5avb4l_qnu%7fw#VR1_B?Uc%1&i z0#4DM$2~sYP2y1HhFMm+$?J`Na?GPVuThOEo)}b77GwR?mS$qlVjN-OBT`tIRlUql4v=v#-cH2j8%|<<&8A^dCQs zYkF>?HLBh%b$Ol(w#5N2obASG_@&hMg6CX|r&8SZ52^+3xHrNg;+!FSJcUIidk0fW zMTkT9Ix^&IJs$eC@g1n^{OU1oZU=CR&`ta8{Ws{s1@9(pwQKzaa)X`_IDAjb{a`V; zfK=*!=&?c__3niu0t!QJA7$?D0N)Q$kxq^WwksVgN!@s=wSKdlp4eivWHh5+Lc*2716)n%?r1K983FfEl?pBJvtVd3Rkcd3JFRd}wmr!v~mP0jKRU8f1H=(0R=CT|k^ZmPbzSkkG zlw{<{W9rN9S+J$o%0wK~^$MpqEAVWE^p#{1x$mNABHfw3C(7XYkE%J#g*r1t8?Zd7 zO%66b(OT))UmDA@nRkuw1`oF6#TmQ%Q9f-Yo5j9FpDx!#DoTt=xGkr5(5#_2hoP%` z-m%ca=!ow|wX{@)b#NtCE>61JYG_!%bC4m5CZfe^xg#Wk(tmRgZ+*rf*i8t(v$Ta8 z?1;1?L|#b{35QNb@nD7XUpoZ;*%}g${i}oB`dd6LDlMfr`&xQYrt)5Kd-^H1-r^C8 z^`NC1U)ZzyRfeN3N(sYFe}>1@=vO4B7xK&HIEykBVp!QP9ROI`U^+9W-r*)r$3jUwSu7Ef%q0>Bh}k7eJ>5;~a<9wM zCz%Dy$wa}G$fI_=3X`w-BX0eE!Q9VH?HXNTv$NJgI)P6OM@GdD`F_o{$@)dXT~{FY z<$Di(FUGUSW0SQ?)Y*%a_eNf*mARILKl#{xAP zdgs^q+XPxfQdA+f`p3qT7_D%H8PgRJIfOzu^P=i0Qh3=fbpu9}i<&y{Q?ZmcDrd6v z7sTEC?QFKc-JD5*SSxvX7^agTPZU9s!3x7#FMQavQP{oR;JkElyrm=7;F)7lY`k|ZwlQB5J)7$TW-#RaaC&N4?W)W-(yZ*Jc%<6 zb?)b;JvaH8XlaUOUcw`Dwy#;&rbtyXlVFE~9Y&RdoWZs=WklU?W!;O@In^**qDCni z?^qDNF{f&0Qu&knn#{FoycXaO!fCgLF-4n>sT#RMWS3$5vj7(8cf6?KCF{BEA7J3$B}QjuHR zTk{r)mkJm>jU15ZN4I-%C5b^&OKmN?poDk9q)1yszn_TRd6c3&( zj8`n$ubf4|$tGh9zOKJ>*kn}L*+9Std7yq=F`YitG#!7!K21DEkOWl*Le+BM07Sg2h}Rvhl+79?J(>t4{R61;$>^ zvFl|^$IRlibZMi0Q;=Ep ziNz$1YI<9qlFa0lL!nCG!&eV4&2%=20 zz~xw1rw@Vi=&Ma&g!WW4TAe9_qn(@u7?;F)^kRb==?oFs1%3Cz)-?xi+A zw$)OaKx3l$iwj6-$$O{%R5JFZ90k`^nZ6)KDT}<$xg%GGu0*cky>N1}YEjH=PU|l^ zt~*H1&kb1^P@YdZeg7nQCmR0arUiF_UG>gg6r@?pwTzO3IY$9c@jOUfUAuU^v-~;{ zjE7*nwOPqy*dIK}DIy~lX8v#OEsAh6|3<`YDsH3y3KRIh0f3LxpUbs`gq?E(a1GA$ zC&cDcY)i9pfYW zDa^%+9~e?Ty7%^;r|mB^hrZ1WunSppDVCMO>^04ZyY8bid8GYb^k`kiW>ax605g&1 zn-n+Kd~sGC9XPlpf)Rq<+m~UKKT(*ZB&Xdy(@5COz3Vm-&BgI${`*VX-hm9>$8u^P zctNwI2xF*WRLxPM3)C$0KMA&KHquWGvfx<5FN)P_B%WsP%{j^}Wkm6NzVg^OwWCZB zi%~J9XNX!={)*l5gqmFBu-JGr?ov2MaWI|9^-Xp+4I?CNKMMZU&`J)x&Dusw2qq;_ zbaGr%+7QHj0t*y8$2)|&K5l8SPM(5H8tDp!YC+0`{=BzX8ebkR8tdO+4UtXV4HN3^ ze|Yz!L>`%J8Q!G4E`M6lVp+*zWA*EX3#yGC3Mj5Ur=Hfe(bYi!h~|N0lBs+|VczA{ ziN6|bTp=9xoG(H*f*PcCGJ>OB$swe_a^@sqK96|m`FuR9$C zj(j*#B}65NjLg95o$hV#D0s9`9s9N-?=6^0WI$~;z4RDc+i(>f&B<&DXWO94F4qq! z9*t*}l2j}H+Hof%5ZRH^V!K-AYO6A1`SMK0==l+N^z7y9deiuw7(M>b*L9C(W?z*nF%BjNuQTfU=;93qXc9kVc_bxBkAg~RaeXG( z6kQ;|n7eu=sn+n$S`~*W{eYq)GnhQr)heCE1ZQLb=TQa>9WQf`24U}5sxBICXbP%N z4UbRZqc=$pNqO$L!=g^q*_Y}Ub%>vEhax@^apQ<5e&NPb2Kxl-zhJcADRF zjZ6iQbVN)BzgA?!hp&E1uL!)(88I6hGef%Bpj(g_i=+;HWue(*ixd84yPy_Qft=O6 zhv!(FSe=)cBOSOC9ec2hsXP#+>pkoiCtdRGLw}Srte+@; z_7}+>do&*WVmKznR@Mz+P5&Z?e^;JRV7!7{^!nHLeK6f0JbNje@X{v0WaP`6Vg~AE zG(CtzN`}jD*=*U-*RMzqU+Iq%6&Aex*_MhWwu~9Q8D6*gV$eT;uUgo8UW_&-AiPny zk+;q`OY#hv@I$!O>gO%}$%|u=?GVBWlQJ3FrhHCq;X!q_{4fhMkqC-v*YoRsdI;j9 zOt)NZk?hMEtBNen8i#TdXEhyAz&Ct{&!*Q43f~$cE@YG8IOpJVrCr_l&R?=UnY!QuiV0pGIAazCUv!yoSEqyi_j+U3DgzblukAxn8zNL_ zBlFI#fsEjMi$gaCK)D=H{^q_VpQo7qNtonfs8nb*Zl*hKhFWR-$lVWpCGn|w_$HSl zq+$@>VEi57dEgIBhvRU70`dg}N+PPymZR^fk)z*{^y9gIZ`DZe$ZoEXys|qT&??Npy!eMxoq$q2mxZ% zyHq71Nrx+&L2-PXWX#t)P7-3whD1w_hjeZ1_GgyaEC!+a@G$_J?U(<&SbkZNc@xk)1_~tb<-I{o^qrKE%#*?S`h}BZ5O$W&E*GjXt-g#dVE$UpPvq`w9Y2= z_>i8^!<*j-XRE4Epx_XdN77){4^$bi^tG9MBTYnA*!!{lxcMe2F2-Q8>TMWF8(hz) zz242G?K;_!HcN(Wq^nGe6A9@X(g;e@^;E1+iFL*?Z{s+XMuZse>K93NdZA>T#_0LE zAlJi3uCjjIcqtpl8dYXg7{LHs;LLG5#P13`Dy7*|;I1B1w^#%kVfv68jLh|%V<2)p z^MzP(jm?z@pEdQ;P9yi1>LElJSR|ojkOO_;gH{iwF9L9Iu}n!a(uXoKjN5}`tUx4Z zu}IxJPcCFUSN>sdB3z*OCNx)$(dFUlh7>?f>0P_uWsLSTv(J)lo)k*pY(|RfzSyb# ze_*c(BcH}`{hVBpZ{_0!$7t%ZK0o}gyVhN|KB-_4vy5rXoR<$f++0%{TZ z05XP@!xnS)K`*Nysuw)LD(hR7@`n}gXmv-B1DVsj=HRFMYgPL~25g7ccGCybS zmKHH7(~nn@o?^K@)sl>PA)?bgB^96yXdY`AXNh5rB4z)_zVJH+Jc>ta!^G9EKS6rG zMTEW}rfpC>JmOW?J1lQKm`a9mDSu3bdc~Do$Q>Jq526Rs){)%lLl{^pIHu`fUED#9 zLQzmRb4net=QfTXRh;+4q~1Iq8V_W6#^DDazkV=l$wT2M<7R~=?Uoz@febzS-5ZkW zVA&A(F-CzgDzp;l{bP=pK8 z#}|6vOE`0&zu>Rtlrt&DBuzYHvFd#XGKeKyAS1x^LF(>TCP&%B{tj$Z9(*PkWkNTe z_sFPBG7~@I!8f{PL~eiZ)BVak{_!vWqy$aHCMt97#f}hqPOVHYo>R#A)?~RterT-K ziYUMsMAgdWy9ksT{biBV8B?vkG9XWr+l{NH&R;qYZ4`>_G4_!ybo^m+WDGEp0Z=H0 zdH{h9Y`s)%h5d{41n5q~>%W`eU)YUJB-w6f&ZV1R*az<(+yiuTOj;W9azkGtQmnHa zQ8lOHx2uz~UGuJuZ$GROkXy{z3%nvFf!2A*MPN~Guib}%uit8aLQeVm)&-K2A&39r z4FArC=*mr0!#*CUY`5h40zOQn$6t2@>H}1!ub$9rnb+Mx=Kr@}{rgf-SCfY8fd~l( z;bO1Tl?f%I!m(lgW)7$|cX$a?;s)=!r(!N_{{I)`Z(8yV*6M6+BI0 z6MrA9Hvb$`I-0@l>i>;~p%=JiJ^sGZpC1U#a8Fzzx-o**n6N=#AWwh0M>n?^9?nQK zCiLJF94o=!zciUBOkdzX>+{#TL#Y+~&-f5|cfxxmPLaO<{02`da*+!x`fK1WAFJY& zh<^Q~|4q}}-f%QBli^55p(e0R7oA7{e1}%gJ65-QK?mJ#@lOn}s6JorWIOV$^z`(E zCvez#%?+jt$;(Gxp8gEpU&~7B%9Mx`SC;+=?ut#^ZRwvY{rT!Q8W@!Th9Qt2G4%Z( z4&>u?N9)h}f`=b)%f5(&p1>|*n!dbAOP5S|Ih4YO@H27&&6OBs^>nkeUwd^RWvJRp z&tNe1Q#_!}Wg1SMR;WgZvKl1xaqKUfyr z6-HGx@Zjk|q(l3~d;ho%^M*T=zt~^$7uX&T!BgYfFWV@6{&ygfSQ^g%7?S`Y;K>H7 ztmMJEH?isz?pVCWSERE1*%r7D?$8h38cal*FueZ?OUSmklH?j05D-wOW;B*Z?;*mp zz;{=CpTc8;-F7n?jqC?kqU~^Th4Ex+tw6DbfIQPJeL$pkDv67^E4C*qZ5fly|y}>G~99E}F}+qRe1eP#K(^7e*@8hlT%R{^w38 zCZ&8w?M}mm9WTtuV=6`ICbZR{e=H38>N6W8@6oR@?{?Mxqw4|#8&APA;3K;;X+9=^ z2h7H=$qNN&qp}i18T;7yz-dO5^DY-AiYt>fHb#n9JulIiG&5gg|6;TEUNkZM71RFy zJb-ob=!o3s>K!l2@z^cX)v^BrRJMlmd=4YfClV%$g^~~{h{khuN+jaZ_`Lv)$mc6} zJ~gXD6SW(z5DLPjr+%1azBN_;-8${z-yh9$ABm3m*X6v80)Lo=`zjKX@x;ImY#^e89m~m`sa?kw`#Ix5QwW zgxlL3*$)eMlsLAQ}#>yp6#=BmhF=%(7D%CrbgNI4eG8j+LWve$f zj1;J<)Y$FNlaWjtj{b4JUqbBP+!Oxi0Z6P4Q3{MA zC)8NUAbA*n8ILeN*ctTIKHY4PHnH8MxUb$SSDI4W%{y1-DU}K$?W=pjqiCNWuI*o6 z?1+I!)MqG{8-|94zF?hqDy4c?B`u$)NN^Ya{^za;N~}euOy_$zX!M^_Q2)CA-^l`( zxJmPmSn~HyU8XoYjvj^wK-cs+PVq3(PX!Fhf9({jI)^hW7TD>s<&RcmEjIQ-}H zf-p3A|M|Z^U;Xz_hptT!sI^+(*pd44;@iUFFdHti#(+M@S;RNZW{}0w4Tdur^!nl< z5<7mB1)tYIE2u=LdvQ#~46~O>CK-ls<}5AvRcJuKeXY0ks*Nu7PP@$r44SQo)i>7{ zftwW6Diz@y1+|iV4CwcdCy^B#bMh5u*0y#-^-Y&q`MQG%-SXC^R$JN%R;RNE2>*(I_nEX z6sO1^ov6j#qYWIWN$n=qf$LN2DrC-eb*O>7dTGSHJ-~iP80x>-+%e$@z#DLD+ z!Ul?Y=aU4^;|VRx_3SJ!PC+7$#pUI$huoK_BcFj6fAJy-1R3!Kr9uHyb_ys6Q)!z3 z0{>Td`VY^#{S9bFN~8IlKz-qLKcpAy9saaRW#l2f77;5W<9gN->>q32>IcGPv85E< zLl7ph-N7ln@LW+UX7zHfu1~dMUfSfmb3PgcF#z3qUYO}!xnV@zUfWX!FcP*lnj2-k zI>3v=YD$$AZ|MiTh6Ozfk7}XA`9C-?#beiu~~%e2`(Ta9LaK;5558sJmdZ~TO?sKP|*eg>Lu*T?cV#vAC6+Jf=SV0}4{Mr5Iw zR;D-e_^G zIUKZJ0)h1gf>X6n>({f?y|gfGcJbH>m%x1{O-KjyZaeY7dC@yt)7^a++hH?=LNAzJ zw%p>vj{qtNf6*+H;)fZF*MMXtIv5U6K4&ve4!~w4Sv?LhJY5Wv>O!WiL(TsF6^E3X zx>wP$RJZr*I$mx92ni1ft$KhjQuM#N$jei$Ws%S7s{7SS@Jk>8UVnX^zaeQKR*~xW zju66wbsP1@GTK77XMY;DkGeb2ftb$2`hy^0Q`5@=N-w>{gv3NM+xD@7h*|JC+IF4o zUtz$PmDYzWH{L_uIux*Y(Fc(DCxQVGMZdC1yzLtV!Qyk?pQbx1_!@)wfbT#fdk2cod6q90-`X8JQ818zFA*@4Bbf#WDq#v zj8@CyF`V`#P&jFMy8C|GUQbW2+x6x`-M;#z+K@Q4loWo)t4ah9-jOdsU%fy?(?-7q)vSrgSF4 z?>iu|2CbFDA^qG~y}LhD`T#SDkE-bQ7!F9T(vv-)HKQy{X>y2Ky+cezVEI=tVdrm} z(qpteQ$6(M4Iwt?2lEk6p@!@84e89smeUSL>mnL;&K0mHwyF=3K91 z1uEZtzbOpP1vBNH&+KLh9Mqwa;JL#p0byK^tP~e9pnDl0UVUso??W@%CDVfLYFeqh1?Gj*wJdlU1Hspw&S+5V*V5%$| zO}fNv{#EVr2!R_94D2ct_Wr)Gzo*b*@?>}92F77C7w*&J>4tB(Lc%<>iF3B2a{EFn zefy3V$0Y|17erCC2?3vJTJTXn@`N5`l(1uC^WNvd;D9NudFO95!(`r<$E9=oIfIx* z*Ah==4;WrT5Q&m`>R`jO8j_!@G+?|@K}noybZs=lY`=zt|GxBK zg?_8`p$5+;(DdWS8sAPGVQAb&xLn)F%aY;YgWi>isq&JGL7`8ZG-rxzJ}5AHeJ@Dc zEmgH~iqpxZ1x#%#3~qkjTooFy1LgxUF(Wh?rD{|^2 zev>Lom+prYErOO2SLl*z!)Pj9)+RXCO8*gzn zN?jBx?QAv}CUX7?JIQJsz3IG+{quNXB-`?`9q%r@#-z?M!LV@nbHc~tQ`$Szr0m`g z1lH2;-`<@DF`xn>?k|DuYf^{w#xjQmb6-z%C`xC`WHgHM)ea)x96?-rw+AcD)Il$2 zhwo`qu1&S&(qaPWzTC$hWIx`XRUwgxB{DEDc%kN8Z`U13z2U5=f1&QQ_XGWBTV z6W@BmGF|wb4xw#ol1ofgT}xtGK$Ki+WFR+^_*Pog(i32-Zwj>C{)l~^maMT^BRQn zp$Zd~!DMPKn$=9D&{Xz$kG|mRTig8WhZ{i?K$QQ2faVQ6>GJv^2K?SDPfz52i`8B{ zuaWheroLBO5j*!hnViiEHa448u3$1I19P3E9Hxz`wsUq#o7i+b$HBPdw{7+Ms|s9~ z8|R<4H>-msX<9;)AR^+%)AQw(5HI1fzmUJ#tI;EtE!v;Sqw0n!sW2|iOS@Ie;=hD) znC%g7(0pR=+{Fsy0h_Sc$FUO*s=+P*U(#*2rrt|wgA)n%3tg|eJAzWMj=dSB%MC~Q z+pD*0HuydHnH>2VU9J{e9&$64a2&3WiWO8Z!zl}@UVA;nqrq836UUQ!iqf&S={WzK%VC6VB##lUGawb)s4eK3QGm{g z*2Y+VsuEwK^YLUdgnqM48HBtK9%`FYpOo!Z2ME@{$%Y|ZMDgv#@TENi+++2U@d*jk zvBit-2+uzkHeOHVlAO*0c2bn%Bo027fGAq6`PD<(B9Dek5nZt6Uo89jvW~IF0%ioX z5=5a}agU|ry#c_d79`TC{P#cyGEV1ge@T!e*s7n4brW;-+a+MGuu$$#fKdDKTz#o4 zBGCBRJkUm<@uVsd_i-MGi|jjCty~`TSK_iTGmn`h zO|Qi-A*q=Dl4(ORLlm=AlhB;c6D`qx8GG143NQdD0@F^QvQC|pgk}b%D)SV5x7J4| zcoo3aFzSOjw|Y%M0ycuZyab=4J4z6huU0>W_+jYGduB4#Q+G}uSYO@In95^n4@$L4 zQ`H795ZLG$B75Rk`w=fcfbx!rl~uvxN~*M2-#ki}@!p6K-c!Q^U_wv&SHMy7Q@YE8 zdxzH}DlC8P4*q?Zn*W(oP`M#2So`~d7lYMLtx8KJ5MEu+-^51B`nV0lmQU9_IG@jrt&n{1C;2#vQFK!P4>Fl-rnp*I;8RwdW@dZ1 z?PtB)dwv%vkQ39U8|JFT6z}Ju;F{AF^{c7={GJX>jKR8_gKak_t~aF6?8R_z?LfJ^ zhXuujfimms z5kG!)5MbeGzW$c|`e!iuDn!55KkI1vTuX21L9FjLEVcDd!ysv}>!(n`!JLRm3wl9C znm|27Q+sg~W8vSF;-u2`H@P`oy zcuoTyJSj1v4L8@0gH$gZ_FU=i8VN=@ZdVI=SRBEkMF$#Tj)X-2jR<&lNtWWk!GYvpIL1-gC$qyPv@$qWg zH-gWXbL(gupFp{$)z-27%30fSW2_|OJ>OZqD#xV7 z#a7AiRdtElhkDmVVfNkiT+7w1%B9_lou`!-vq}9ss}pZ3SWGAo^m8O_$R2&2H)C!Sc+Q} znp6N^h}kQef_3Qq(hvRtl*9xged&wXaw*Xgmzeo&m$c42DWYzd)9>D&y@msD@thAs zW-QvD2}V9B?E{ru1dGX32Vgq&wDq7_t|xYh5}fc96C(0LJ*ATu1YHTyEq^Em(Mz?H~jHB_D&WrV3guXk|()9)%ODD%PwF?yChg= z9}S_i*391@Y6>x#uJ|<4pJ!38qA!pNxZG(BS@Eap1EpIgV*?gfDRiwqokKW z4C*ab24^Otv7U-*T_mLvZC{f(X&`Du?gBJ{DTi)y*C%8w92h+JDT5-PzD5A1p=_h5 zp8!*Xu3a(m{lNfS_r4{Sn~R-Be|VSENR?+GbZ4YmjcP=ngkr!3^Z)ZF+hbCs4~ zv^f{wGu?gat})vo$ttpAKTlUqr(kLO!6d4Ei5=8kmie zbko#lOC*ebNQpE$j<|I031*V@f=35`c>oV1$~@c-WvBrMAj zvZPpPO6H)0CT7Ee$M>J$dvl-SuVcN_bGad* z`RupNlz7hyM1C=RR!XrUV8jqXCUYX36>|B?f^dU*HHufq(Qo7Ecn$mFNNvXQLe>E& zDEGz4=<-36SWRpk)+zG3L%-%!2AJw{% zNFm_F*nRqgU8K+b?mw;wrG-3}nGo4z+yvmLlP4LOWLt#roQXjCzn3!)c$>u_j%~!8 ze7UsEP2=30*W}!5>0w0FJWSUh8!nxL5!ziRCjVBzsX+;(Vq7s#i^Eq%hRHBE@-_`i zIPXx!QN7K$`-|sTAJ*RZ9O1^z8a8>t2aFV|&*x2*83T-)xs>=Cru4(fFQ&4g~k zV9pn@nBVf$x(_XIjR&H@gWUhwsirp&D$rs3$HQ)4L6aXL$>*i(OUC~r6(JXQGie^@ zUo{7a#NC0AlSiVQwmFO8P-ybwJ0I$^)7e|{>&agBFlHD>(a4N)9d~v|iUiutM}YYq z8XeOHB~dj?-}I3$1fh>k_s7y1bfAhi%l&pdu|TEL^!1LARGqo7UUth?VAAuI_3C$# zzM`8)Q0Pq3k}?jL1l_W-?yHEf*ZDXJ2MmUPG^TwT+~8Rb!GUqQJ<$Zsuw+m~4rpVq zBGSy2(skn~igSI%kv(VzxKogT{T^G>StMhaRElT*fe=Xb*mP|ajg0uK+0E8gZSkCp@~eY-q6==D2@n1t*5z&Zn} z5h!Yz6NtRPp&J5cYIHl)BB5v(=Z-I6m!5+sQgrsjF!tu`cQG(9EKcerMSy;f@?kLx zXr4eDE8|F0AE) z=S&{F$MpWrVIBFf`PRRAn;G+eF2sIAwA?G%D`cb1j1KjPA8daOBwOPC5!6I zwk}Yu7WUNQXImTW@EsEpX+%|b(R`qPou&5#3P;|OLO@xsmT3R*|2yI#qeiC01ibd;-O#IUK#d6((*ks5uVez?`A zJoSTbIoz37*X>U;Y#7X^8^qRTl@tp;+ov*}ZXMYa4}8CENWo-E+2a;il8_GKDpwJ&;$Sxe6!4-E)=w zU?OTkRP12W@+mJ&Z891V2cgEMyrB6E7;F+}35wZ*gb<1JV=Q&yi1#f-`Et2Fu)ZHl z20(dE6}N%w*{QF|eg`uWBEk|QxVeqd4FljNJ**-zwwB@o9p7n%F4>b^MNG$(xVHQuX*6ngzh^1 zph4)T8X2)pVis-x&=#P2;bfM%t=?niCt9Fsqbnz<#C`JA&(??z=j>8?BP+>eDr@A@ z{ObIu#Qs3{K(ufO^r(6^I-E|_Z_cAQFLzs(BY4Agb#ywT)LjfFhbR>JE+pdF220e_ zDTZdud<-)~g#+8!0IS5K0iz5pOaCI6ZeRV)u<}sd%2Ldf|CFV;zdO;3U;Ekq6owFo z4IL#!(`G|7=6EaPhgPr%P#UH$*V)VH_2S$PI#{_%DueO!Y%m}rlVYZxy>MMu@oP$P zaIesl%!p;8hlq2wx0y%=S!f**k~5Fy`0ay^kB-yc{j3;V&Wo8Bs78?USgiGU(Rk;I zw>WIlhBWWUBH-`SmH1YPM&7rH_9^c#KhvCD?1~_?eGb2mCS20M$xcexl2t7GPzgG2 z0xHO{!meUQAzi30kWHS)^Q9UnqM@pgC7epo2VPrSi#)Q$bZD-XtJnS9ac@`mv)TOF z!HO6lV+H1MN0Z@uM;4~rF4&|sL?bBV>f(fq)q6MLL0>ys$7VD~j(Vf)2dI>X;vgna zIf2d%=<|!R#fZPMZah2rhNZ3;w4su+<=kBt%#5frS)wy1;BujtU=w*CpkpRJe|4SD za8Fr`i%_Tx=jy80gk#?b;I(8%pizLPV?|E_hXQbz$0=yoBm{7ZOuz3G^!xlXKI5hZ z3#5yD>=OGHqJyM0+ZQKmnZ^22-*>r%$D9T_sh37&qg`&tS2bIJ3x5&<7-E zHuSmhR5CHFC5|B1dYA0{dnQS+48q-ebvS)uvNP`HQ3U&rWxgsYECzcl+WIw+%R(o` zLpsT#ts)F(6b%8_O)8N~7Y(Hq1zz3`NGB$x^2xou z8s-i6FSk(uQ4nATjc|Q1@fn2GVSNL04XD;`;4-vO5sf~6JNS8>?qopx$(iP44%f&pz(~=+_CP2V4@DrhM4*g;p>~bFMee-E%mr z8cJ&gU-S>!{Tu{;1pdGSlkd(XH#aty3TpNOp)8gcq*^=~Kpw-Upbk`6Rup^d%bHJ3uQdUPz$h&UvDK_4(J z95?n!Ioe?0X2(9?yz#X0Ig<>Vy5CV0o)2a*5B*K;3nA|KwvuSW&j z-0uVfo)E-}K$#~7heLgX*_!hEwvD`2UrDAoeZG4drov#`qO_ni3@qHO-<>nuZ1H}m zB6k)cmGE-3P#hU>i6#I;LwY$z#7Q$r_sF4*O(<3QD#H~I$TWtmG#tSB-^(!upK5nO z(*px#-(u(S>J0#Pb7eQsK6W;X*`pX6nV;S>X3yb~c6YaOo4EngdqCaCQPJJ@!{Bh* ztkK@KrDoI(t@a{EB90|;PTuQf2xtmJw4dns5l^*5jpVeQ-r;HznO!OJ#+KH>Mp`mo^T3Z>LL;dWl zZ&)%q6LerCUGxPS5qej;rAb4X%*qpS!7%AQ1z{7dYbFsE^$vZzq}I!?(HgH2^)&<{ zQus{9m9Lu-1jHeuM31k#d&}bmEYJ3DcZUa$Lq2VB3id$pDCG~%*9q*;*UB_fkc$wb z>)kWW@+PgxHRMibkG}4M?qJLVK(G|z#0WtFf-;`(S16K7=Jho8rsz1Ml*`d(KW_F! zpx$tzlrMO_Q!A^*|A?v7h^<&Kewr$K;a^$gR|V zl>G7ge6imJN|6nYaZ%U?8zM=Qe><_mPufiuT2cNxt7+PO`u=V~`Nw8y26W^XI_trt z;T9s%3JwVfp=KCgKKBM`#NSvgh}%LYbP3&xfq`n{3lF|LEOadUiMx(Gh95AZ+_P)B zf|!`=sU;F52#7*7<4NNu{NKy&->RlHUT9nuz6oo4L;!Rg@f~Q8I99Vrpe*eMxK-dp z%jOM^z6NV0w8_sr{!S~8uk__aIZC-?DHl{=tcl5={B!No#V92G!-%c?vawJX!Z9?SVZ?%S`c8^_2$AZ+^*wa}Cb5Gv+NQCAz&h-N!h%Z)$B#y3YY@@fp>c zTC(gEz*5zoRG_qO05C3$r=P$iHx9lPnu?gFyMb__yu)(X?Llvy=4djwoV;Aghds^4 zm(KBXP*rl%6-gd^2$Zwh?`55T5+lrXx-?}}NL=cZg!8wm_WvUeY2t-|VOLcE;wZjg z-sY4VZagl7^|$0(*T&<=nj6W20pe*hps+^}&5>>{3G(wmCYpBYqh@D^yDiMepmx$1M|M zM;8W#@FIX5LmJ5c$z#hkfUG8RE1!aijlcBu{bw)VPF0vt2F{wH8!Y(9IZ}@Y+z0&K z@N7{9gc48;zBra4I%9Rf|Jmmn$76&fzX-4En*p_);i`hm!bV zIYuup=b&rL<0Vt6kXj3!9aESH;AyEHgK?Neq3w62LK_e#Z=1%476zlzo-`u@cMbBU z^as4>5af?;&c_?`KovB42`XkRXYq}LA3$zvZ*z_dZT|B5Cb4=?{vgg!d6GZFu|50UQGNjxzBJO=uC5{vcE1HL)!0VlZ1=m! z8jkvL6@ca+yTx7^u>?OwXjgbKZaYnMdkF?609)ZalDe5X91MXzs~i1L)*k5pIzpsg zg=o#YP%2GViV=BChKfd%lwo-Y=9zrq#TSH)_9_Lse2cNGPD8Lk_uY9WAJut}_-`Ji zUW4}ZH$;5)p89JKNE)SzTz~+|=9EovjCYsg z%tUQ%E95YS_W=>_du#;I7>G`NWr23zPxpQblvaF*irJcm3)KG{Xea$q=|-b#vn0W3 z#l4x1#E5%_DV?JmlrVg?OOJqh(Q73E?(-xJRS= zd^cvV27yecyX}!y8EKf+Zn8FwDh2dZi&-FB+F_mff0VrkJk|aC2b^Ohl9?R}2^kq7 z>xhg74KR zjO%lKuIoLRbh*o9JZ*cG9hYX<%UxF@Bw;zsf=F-)K${N7PCvLCot0(MY!uU-elW+0 z@MkUH7LG6>pjci48uHNg0i!j1CMNske)EGzm5j{Ns9L?CHOOc6;k3&J6#fw$`*ZaS zu$!tp=^C;LVdesVpoO;oB zt68MJV>HYqz~DRn~v9~1rNrgx&(wvIqg1Jo@zpi&g+wK>N4w$N13<+C4^45NdiN3pVp z0C~V%BA}G+h0n#>&Bslxo2F{#^O!kEFp+sBFJYDf!JL;nVM=VISy2IO zH4px0z3(poH{{dEQNxm!rjy8_TrD1N`>C`_ky+K4N&paOC+r9s&`C`=0=gGM;>BtL zZ1^AbnH(VCwI;oC&})6hW8({;I}5Yqi2FZgMt1(EbP^j}_8AlDf4l2JSK* zb%m7WyJ;cUwNG^!4T^hC|O5K!1;Xe zWzksyr1kM|4Z*79879R8=$XzK*K+bXF@->~&S^25H z^5VoF%%%MV;BUAVa_RjC7f^1c^xfRgY9VMr-C0dNXMjxLx=k3q;1O-((+3}YVUVz9 zkNeVgKBc(D9|^xiGMOie9&+GVh47jsYIe-}nm z6>TaHbs9+@&b$gW0LL$Ag5uDtG4?`vw!o&;ZcMgpp+nTSrQ^D2mtnYwh{zA5e_=MYHUizz z!L!zV->+~9{gqe2DC1Bj;r{y!xxi(#@n~bb<>=DlgUwCX&l~$^0rhaM7f7HXU{2DN z$@*I5j!b>B;g_}-q|3;03Jb409uPd zA!sUb$n|Eke_yvha*V+1aCuV5ceyl3eqna00sZigk1Q2%{?fh83Zew9A^`1QnnZZG zyxQzt7y7;tcMS0|3tOUlp^bWFzCn?z-pKYFY5rJnbX`KwR)J>63#Od`L*9&}UqGW* z#ug^vKa~-`LCn}53Vg7w3K5)g@>4zXW@cvj()}gntD}JzithHEC3sp0$?Xqdtp&D@ zGcKxq+&Xx3xXQyu4(a*XRN@(0pdZ+z;A=wJtxW3CJNTB}jS>3B9`umCX#@63yt$*J z#LDvO%5+}!1GBBNu^`beJKLC@RMiyOWFIXAV*fb@O9X&_e6xyw>7EH2c;oJqeD;?; z3y^%g!Yn7- zwG^u>{X?&ctwDaFIZB9!H~$jhmFCd1#Tv&+hEcS)j%IP@6VWOta=yJWNiWRg#^9k* zxH8K>VFxl;h#(UE);G>E;U|nbqE0Qj29+kIz0OY;OD%OD4-@6GaI2Qsry)0%Q`T4p zA%wbd3-wPAs4KHz5q;2x7Ml{>3122j<~-i!eFpI}qrCF=MZHKk^Bt`GlbI=fN?3ErD{$}MkI%! zt!L_1g$`e*e!qm*vYsp@;V30Qn@usLM|-;9_E)t6I<_wA+INXA6VbH-VfN!K&8|tm%eB=7R6bm@>xFS38i3XfNvkTf_7iM*xE}u=vYV=xHfM7?w z?jAxw+q^l&%uL=C6m-n=^3JwBU?!SN%J5jD4l%f{=r=bveYbM`-Y&G&GDC0#aLt8V zAFallG7>)u5@vB1D}?2*)5P05JfiDiJVS4E5iS> zL8(%5?dUz7Bq?Hk^9}(9+s}ueu1o`y3gl35zU2kykj_aa<$UBatooG6VOS;sEr=29 zXn`A{NRhN*tOOc%Yadq`H1Z4vwNoU5p$MQkU@ZX%dXo4o47FAxWCdk71X1&g??-}= zR)!yy!6~HgI<=KeBkj5}$5lGynq?tN*8b7~8o0Fm7r;91^VgQ2SnRr#1OMjVLcedH z_6DczbrB1emG9RSi&J@Xv7tG6k3%5E-@v(QVXoXXc7y*b0ssS$JF(Th`x9-4vJxHb z(^97%txecRN2)(S$KhKf$vsnVZ*K`5qdVJFOT%x{S>v+{e{i3#FU&DjT!R2&vhs$w z6=X6m6k#2UlhB8bAXPiTi?ImZFTvdUENtT9&(tUA&zbahoQ-PDx-wB{TDTD+C&TYU zzcVk_tep?YKn5VY5G`09U7stsGp!{0)k@<7qgT|2HA+kG<{|UMC9a?*J4B^najEyd zxK{2ZBl(+-j(uZqNul$s;>}8>4dp!)IUmF&VI3wB5gh^b`vw}hG?a^|EvF%u#wG$j zXQ*q<-~zPnEO(qORK&Zrc#^6S1W_>TIu2LTO(IA^S3WreAExH5dD*uec3Fb7(%WP8 z`{#!GMFmym`V0qunR|zcx}rXBOJCx5{l~33^0bcH+^wB!84*a{2&fvBcbre=1w`+g zbvOfU!W7`5$Xi3K0X+XK>b^kvBZ&`fADtt=dL&^T z`L;5Z5=`5FwqkmAS&&JV|H0Illy5}Ez!4fg=(0Ih%LI@iRA7HRy@yUaQgF*A=m$j| z1T(se9qyQgHh;*@7RF0<4a&I>1v97LUoqqYY-9=jPGKu|QFc+LbKRC4tk9fq?JfL# zd@_?cSiBCOY`@*$?TO&hZHrOc<(iN%qm;1!u1P^#@;B_})ij7^rqKn=`f4K zntgX0hoyN$TK)Pz@9IJiI7l7vc@5fAso&d0nBi7mfQEhra%9RrmS2DIZPgy~@d8ahAf$S=z~tw_FDy1t>VEOH_F*s0O>>{_17 z;!97QRGT(-*_w7csw5p3w#?XiYJ3sc>Ed0lrMCxsUFIak5Gt^7!$VTRo+R2qh9H`2o(Kq{ScoT2WxwYF|? z?=cWkz+-eI;yR_*>g;f}q@g58t(E+cIAUhVi0wKIbRp)b2~wLBDPAaeo60nr@Zylk zc$@xL&hT4N^>IF`AnpbWEG8MAB<|O4T$43e{R>kmp_R6YX9%}sy(*J7r>+>%G(O#I z`dFs!2$86eVfhQ`K8KJ{4QM(E79e)Z)ev@9Ejo;pXA z<2eA+fEH*C#`Bv&;9h#3Wfl#IK&N}%u{>7!54n`5$TQ9SNX{m7S@tnemVvF9QIDIL zt?V4#N@CxkPH}sgnY#?x6Y)-WP)TPHU<(J=HEe;JO zqLl}IweTmjf-Gfftlddu%p=S%lW1G{*`PD7lM!upnzNGg)AYN>5VNP;7h1 zYhxXvrdDGO4aRNdA{)C$`vjzyQRONKEksxu?Mrr>@Xj?_`Xac6(ww!9vS30#+w!%^ z!@@(DDKG4Ij=xR6MAtoY8ZMGlgCC2n*j%qwnYEl!m;ZW8ka}bcj%?s+(MGEZNXzj? zqwnhHX^OUs*~3lq!7D4HHmFT=mVUAiw9oLea zJ*3^_5uK0Z7PV)KZ6o8k)@Qenve&iSdL7b6uR45yC={A->zvq6FG?93v(BnX zF(U$yCa7-0+Lc9&tsYZCqwuiEzHXn2QgO{NVH)#+Dnvl=*7UmrDDPBgaab$v7eHj0gm+kR3{v`x|Tx7&*k{pdN{ z2=q1)P==nv`-F6gExELr!V}!pF2*?-_!Qp95sSWpeZ?}fS9M#TD5lwP)GM|6iU9SE z;jX9K;a=eugH==ptJD5_Xm3s*mMnsn-5f8DEr~`JfJRFEE5oY>N>2qf(7K`A*0hJm&BBrl7QsOXHHji_3Hw=}MRvFyk9R}L1R z%j%T_DK4p{-jT!Q_k8Y3X0MwuidkryJ~`NOH4W>i?3)9UYH*v*JWO+<`F$p?1444yo zEMV0H)S5^h2YDcTJ|vt4I~*$nJ`(a(y2!wJ#hlUSE&;u$W1Brb5p6;C;it(@HXgN! zezJ}6gz^;vYyf{>2kEITC-ruwWF1?$&xv$r^CRearD(nj2>39hZT-gzP5&RDAvmXB zQzh-*QGZ|2(6zXrKhApR+q)1@bfYfC38wfYpel!d(gGO8>Or|W%m+UuN+B|oB?1;q zqUY{}ZoRp$*s!@~zd4z>wVF*JaBHkzBw z7F)je9F7bNxh5_lh#b2B z24q1LCP7qG&E_pj_hhPB8~(Cj>$kGc90p>i!sDR&3ltV8s2vYBnXZfDzE^O6XJ zu*csa5VLGbAMh3?H+qFnpj$}F9_z;Q+cR(e}v})s$rbn1zkuTB}k!1?N^Rrkc8C?9-TQ+PoHRnmL z;+rWTFT+1;;g|r;M16tjhwoiZ{h>_6M1b1s{`|cA?e5MqOuY7Ank+i*792bMza4%@ z+G9*{ew}_Z6&p8!{d)45ttxTdo$IX;=t~b>vy_-~-=;lUM@tqu&ODxp7v+jIwHdV3 zF~1Vx_Qij;zE*3YgN~oAv8gK5iM}q#j6?V<)RMl8x9(#4Kt7YDeT&AS8ILHn4*;qc zAJK;DJrA`|2_XIc*&g>y<{k2Zz>K28Y*E+Vm%Yz}ncL4VKyFxAewh){EU!;48P+zR z`kWuS2*Iv!CP(zpTHr*|I3{?}a(hzp7vKYwE!n{)3DFrG6QP-THB)#8hoI$q9J3q7 zpfh*aX1?b#6_Mq97QP0s=MN6Mww7onB}I&8#G5(y9rHd7-b% z3iwM}1d_R9+BxEAEJysmOx)`Umw;jQ9s`97_P6CxVay=HQ02n9j2RrYt@BM^eEzU{R2+9A`Sss5YkFuTizYivaq9Y<~ ze9yw{v3zItyYa|_f~lI{{Ww`!pKSx%yVxcYk>p;|3fH1h5u0@&W?TmSc!nK}*Ek4w zr{AgadwxZq7123s*H9U)SV(2+mv(O~5p_&Uzp7=`_0V+O@Myo=GIg!|y6}2d6GnD= zx$^s6LlIp3x<6qFW>S@kYK{03P7$qggGNDHV3Dj367j%V|}zb2CHw2nMyk z+{x`hitBV?v%q!7QG}|r9cjRqMUbiFRn4GN#`mm508x#3P37O2pfE3DV+JMWc_v^3 zAMDNv+F(p~4RI7}ixvTK(c8EHjW=;cwJ)olYqHHfbIf}m9!sF^?Xb@w{~afPVUmDC z!;(b9|2DVk+%uZV)Qs2?hwxL^uFmN`tEH(j9cqN5qhi1nW6mfxQBg+_~_%6IP zbV*Cf01g-%mGfadd0woDK=koL6ibn0pbIDOB#1vd&4a?< zWd9C0qs_9h1*EgJ#QqtPggGUK)qyBx{k@3)tcHVB^@H9lwi~_rY=~N=W!Om?Dm|-) z!}DdUA1o_@qMWXoJ6eCK)q~LC2>o}i7AL*T8$IrgwG8z7l^;u<&-yjaCDh{in0PLR z&_ihf94tOn60Gd7#M*x9EPiQt``)=p=o1~y+E5-?z3B!ymhvXX3mDSODlDk=&C?F8 zT!{AYmoJb6gRreUU|K@eA`Lizq@fxgW=2qOPD+I)0#%_!-LI0ZY^3VxJXA5l$BY|o z%35Y`iue^gvHa9G`f#HD;IOlFNwLN8uVW1g;0xlCf{HlVkHhjRG;Bdua`l65sDzo) zke!rcP>wED_(KESXmSsvc%@AO2tLX=+zD^=U4+ZSf19_wFSwYkaN&5S&HUQ`%%<+vA-ie?aKu!zuUFN z`XxbVDH?|FGnH#xjvtF;GT9q+vCt9|!K5I&!U3|k1C08^L;7k%;k7Tu&a7@TJX`(5jgf<$jJ z4#DAf%TKKx^V|}Q8s9ifw!j7_5YCoJ=OAV{t%+!7FN6H~L2GGjq@CUkom+($ZgER{ z7fBTvRN0(R$_L=|xh_NDFFGPn<^LpU$KXRYnaDHF=#0+19DA~ulIJY<3-0%+0o220 zjIVL)66Pyx9r*bxi5F(e9~^CUvUW_O5*^U@k9C|aHXB4mAW3PzsME(1#pjx>B?g?F z#K$2m$&(Fr8Q`0HHBWM zOwGyI#F$nYB#EE;#w||7CQA{%dasv{7GPEb0rnL6$auGzsPQ#l*TL3~l9@!c%2%fG zntpf|Kuk8*jL#E$rgUmZT#uW5FdYP@F_n*tf~I(pCTb3VRKZ~4Iu+A18{Z-57sZ}r zYej&qP~X!jUCT;uyCkI0zP7-cqshq*&LdOPTZWqH5RW5cfqMvBm<7xOPVgiY@19N+ zg$FVM-F({TggOvx8;zuA#U`xP5FnJ{RHh#jb<{4&p=&7R(LN&kudWQbo8Xq?Uw9@2 z(vE)>UE~1Ly%#crj&UO87q-gfm?uDec#!ClE|29a#fo!w_$s4uSWjZ>w0t;4Qdc~O`cVvpxzD_3_`X0bN2ql0L{X5|Z z0j$tzmZ0n0<~Q={Z4OY{FF1xS3e<=dP~ec5|A!@{Pji@hjjXp z8`UUy1cj_t?L>*AE^yQ|gO-jR4y5QT0E12yxbarTb$N!Jr{eovvDFdZx0#xp1%!C> zlpkN(rL0`5^0+Je!|v7}xgc8U5HHRD7JE05DRG~pV_`3zJ*m3LR9|^hjRS>WIK8L`Zd)pC%ea30?px^ip9-g z(%TVRzM#f301_~Zc4JW8-NG!$-r{*KbEvgjuesUf4kTVc3VGNp%H7lf)m$0=APSSi z>nBJadUFa2Mw$5yodkk@s{>LI)``3l5*AiET^{DJnr|DjRNxj@*(qtOZ>y%BWX;GO znX2y;|L*FGrd1F1!fVoD{C!kNeYvFAntkIwF|nj`-NZ^m)nr0g+{44I3cx}inwt1o z3usjU2xD^~_Yd1KgT!s_rM%VhFRP%>RBS@jd~_eHvY~ufEjGJOm8H9qr&T*_L891IODcu;XsSPK zFi{HRtS^GxF@-%-kQr9re(?N4FP!w|f;BAjt>_!O-T0lfNK}G4qk8y3;)u+#23%{m zFAJU6B~PWCfl{@7t|I8j%u&k^k6xc>YA-mn4)?ygiDEXkf09C^F;$>UezfHkFuM<> zmAMRjF&SIAR6IYH)(|J62DTOFJ@v(ejT;hcnZfuv>7ja6SpG)k=YbC!+J<9@G&DO% zHV@UWEerPX8TxeA&L?{Q*h>sr?GH6w zm_1NdR>s%3)tJ%G6jNE39S1UNp7pADmIHdw*AUL(?z_lm!)fT(oG0BZYcmX=zk zDl_6kaXE1FK19(;o)LT>3;Zt=vc~UF!C&L+8=dk$G2N9wFtW1|PI1L|vMj#q8u-+A zAWdoytb`0R{UWDXKF%ZMO4z&{Vw?Hz^j>zY#;5wwJU#4q>!U!w<9K-J{O9{pPJ|q> zfz1}0vO!Otd@KgjB@d1E<+rd>gVIz%fy06v_Vj`=FVw@ z3he~X^&72@a7j(G?k}u=QFV=)o%hngWF&dJb@#we1;jsck3R(yE70>^+v*qN?nIEj z00kOHs$*0f_b#QAmU}<05cvptUF5ezg@W8B_d@095Ye|Cy;z-n_6 zX^>|yRDkK2C!#w+wK^&)+=%atW6kI497EBZb1D7tjPF5gWd&^VY}#=X6ZwYGdad`e zWmS%|c9Q0gu)`P~^Ax*X+q-+o+C7in{N9v+h5hhA>8CcOu%qKV1Z>9Y;Hvi|luG}J z7gJzLD3yVQDziO62DqjPa`x1v&LB2IhX-kt_VO|xZ3Qb)7l(rwxP{N@cYL%|{U zey9~CQzhx@O!=KerS6_6hed;u1xAS7#u!!;)H065m)+=(wO#E-eQ!?n6D%&QPxZIUnEG{^o zD3RP)n=pf5fW}G&SU~x3fV=@O2vw1a3hy8PRPmhaowu7ZC~IEZuTbkt{` z0A6s>Xje;U7%Ny_Ae!UvqWI-nh8%U}sT8K0k90)Mn}2U=3JImDr6r8~H6@@`+H*-v^?`(=%{q|NGB+Loikf5`kY8`c@t8TzO)rD>ZdmwH^ zk^h0m23~ylSGcSfLZ|kEFMqajVpzgW&G9R_d3VAG&8{z$HxV_8Vd^Av0p1Q}|O>Xo`DW-ybWmDt-(KDY8wp zsnm+F{HS4jx>SFz_i_BnCuRi9(4AKu;oFl=qazHaI>!AO;87X+v7*ePgtS$5j-be! z!l__*$2Hfc*MM_F?&0H%7(C! zeDZNj3#9oejgeY! z^`07>ELoAc&ink7JSCZDTO5iCO*+XhQ*W3_&TuCL-R|u3n6CakDtqzu0MmwT-pWWevcROF)F&fh2U zw4WbNMIHXFlt^NTYNPz)P}_?+yDe;a8J{?&LPrX}Ku*^BA1_=B3XobC z)SXkED_y;gtqs$=P9EDipBIV^Hqc-nv&gaLb_H9OgP7)rlTYVR_nm3?iznQs4ly8x zJsMLzY^iO`_a{393kggxLpGw%XGnZy5h3~~({Jk?l`t)cOP5EKt)$zq1o*Fs8`2!a zmr-MK>r9Y8LRC<(SX_TIA1)1+SGLUc!R4fEwnTD;aJ2~BC=g~eiHG>oVa?G`4#-U& z5phJ9L?^nRpnW+xs1qzGZwt*9vN;TkhyLl_oY;3)E41I{R(^1EuJ$0(n&)a0w>y9L zPzyrJ)|h(VTI$x-=)+e_Km78{RDlI@r?a`U34W z&!u%P3>b+mCU<@YARcz6KdCGsDFj#br@N?l8v{ZfGLo|DeA4O)DyrSZrMhUOI;mSrminC)j8NL?} z@z!Nm^u^s;I>54|jaBi>RIgzxD4#t%O@Du~t$QwA_v?oo)?$|hZ!B6fEOs;OZ)GL= zZaK&W_95YG--=uPZl@y$GL;aMk|8l>XBl7NifN#T-ZH_cUk>;(5u>TtvAvZ(ZszCe z<7gkwVWt^j+6;??J+>xejn~!&{NM;AaA9aveN9R^^I6^ zIm#WW;r@k`eXzvFK=qU;Ew!r6YaE4KmD)VrW)Ok$*b*(d=4;4^x%=tiVT?2^G~$)FV$N*w^yird#jj}>C4SC@YL6I06ctk zKCo)xW`WnM4B{Bp&4`5vO2R_-;l+u0xs&uEn;Fm+4o#J@f>tL%-rKloYYO5rE=7{) z{#^ae-~OOKqJ3~6-o)osW9+If9wEsm6^6dx+2RXV!cFW83RkQJrfN6(tgGvy%|9A^ z%JvBITne9=>%Hvgh~eIl3!*3|bW{WoxduHix;y}GAfuZL_1~%?Y9cfX!Pxr=>O7$m z!XB??DyF&nzimG}XZ-nf#o4DbUvPt8ZbW*G_;~BXtKN(G#`~J_=a)AP-Xyz`b5%!+ z5$Z|;(}AYg93=QmMv>y~pRcUHU@dU0z5D4912J}u`_ zLhUN#8>Yh8gtxGTJ~fYjoY8eEk6~K~EY4PV7C8=`tx+;eM6}zi@3_rOy;DFS6;S+a z1`ODju5!!+yidns1Qn08$Fc7!ea;o%M0DQ(OXVh(*JCibGzjVdEMBVig;VPbR1=b! z-$#7c2w#g`%+ev7LXnSvQ!W70;g<%M@Qg< zifccEg6>A$m#fI_B>svD)rXU;3ef{Mi;l7NUEI32#=%p;G&C%02$7eooDsyFQ2l0L zs5@NxdAjr)#%1(SKmV;H#Ftd0O5e|nmi>?DoBP}%|ADP9kcYsuE)JnD`x#Wamml9luM7) zSph*clce5wFV?yU;M$C z0O5KQ*Ff&iaLqwvBeWl`BdDYT{Xr@S3bzxC!lI%(1@)?a5+12c!HI|+Q})Jo0kuI8 zNO@}>9mmI^khkhr6Uc8aeTpalq;i~cc(hMM`C=$Q$aFi{*wN^CoyHedxgonQLie7O zt(h^8q(C|qfOq~K824oc(Xl;;d=~l{?XNZ>%M24aiD_A&&o+J~5^6UHk&(4SLbv4( zW=>yYYGeK7DXsF7VVA&o`v)4H+b-2Vz#NrleC}0iUIh~_5T@nflCX8Jsl8?2t^qQa zR2e-1E8xdY5d#1616U|bB?_PHs9_VzSLy_{hE4K%TJ#EK?yk!^Mk;bh{P0JV z56b;2ZWzhzn*N-}t*&D2lpJ0j*a)`i8|iHVEyLm|+NBR-Az8gBvQBtL~|udj=kt z@}X6Y&*9wT_u^1RKsCWfS-+_-IqX=C>6 zO0+(w4)I#|{Y}Aq!{8qTdxZ+c9l5J8sWa-yYzG!Kh|vqPcWTB)aMgHgq$2ofj9x#c zMD{CFWs<_xKKmb21NN09Qdq-*YL)l7m%Vl-J*d?;o-vS_-Zcy0K@&LDw0F z)&NNR&BG@}bZ&305iT%lI)AU-#J$GAdL`>5SVy0k^%(6nywu7r!=&x=ButqMcyy=f zia`$_g%#QoK~&-vQo;#wGY+w0qmkGJ($Q8p{&;XS^kedckdD@sFEOMS`BN;w@+$<$ zN^57&h<~V|_IQ-jl@)ozLxAAUO-;GrWx{vg#jk*iX)=~9SOWTv>9Mm{T{__~vptxK zB_?as0*KmZLvY37^W`UifdzKKAu0i~VJE60Ii_tKfpYw4+)<3DaRWYu3{(BqeV{DQ zntXok_hTNEZW1LtzU*9Iz0`d-JG^Q?q>+@P+IB?+k+#OKJZ&Ew>yu1z3W@3&Ja_s{ zBko+#SFP~$PV^15c7nkEYy-7syQ4j!C4i&@JQPb}ZbXC9_I`QE(a2@=`a zODed4+OBOXqrlg!JSwgN!5v_TA0@#cd{~PwAL8L}iMXG_FUn)Rn*W-#@l^Yr@H_e~dQk!C4N+ViMrlNNW-P`9eG_FtywT+-< zbZ3fDU|AgMp8m|Cc~O8j4O$G;(T1U%SAvF`5q)ohOWx#1^`JO;myYjWm_TUGQ~%2t z4TkhL%GjUN2XPaFB8jKNhXVnHrJqKxF!19lC?pRX*$O7I#EuYWNgzImF?Z0|(u*gq z6Rn;CW8DU!ifO-A=plx^;5DXjzv4wY)6}F{-dJf6wg7F$a6d`uiBtx@J^(mixh#H<-5lZwl+41LSn zU-&ZG5SwFhj;mN=T|Tp7HCg6Fl&UXwPj00ahvdRTi+d4BIJ!C^Ca$G{kBS4%!UHr6 z!U^=`lRr*ImXZTmf8cxe7t|m1S9DE3D)*hn^K*{b6p~wu6@C%3pL7JK@3F8=hS!fO z2RL+w0}R9HT=Bc~OP5-)DPQ$w`y6*AwR;b)Qy)_m)N))B)2C^~Dfjk{THvxzB;~1hi(Lvc|RQZmQEA?B{88x@}93o%ah0>5d$5)UJ zN0V(Gi_pu7A>Dkc^K4PuFyn=RSH{FGPfeb*89 zK7llBipAOBci;o_ZlXQy3u~nC zTSnRGV@;GA?t19fxh3tIT^y=(XR0j ziNNT~@+Y_mf@Tdz_*WGCHC=zM;a_nQ=|mn@XG)VLdfruD`fh!S`T390K(%j$t*`07 zVs3(D%f*kyy%)#L6VC9x&~TqQ(_L7t`^&Wt$?9hSKtU$Dg3B+(gpU>T`-1kLcYxn{ z4dVR_riU*dP(}W)>l8^mxuN1nQ_ILMX!U@>YJwz(2cTve?xTmEFSl^2Ic&Q>|6{o( z$JD0gKN4pAbyj=xBapF{jc|wmQib>pcmGxV`*o|V3-`u*jP#`05wHq4+cQ5-U4KZU zW>t%NUVQucyUk87nc%D-2Ag+R5N|^;*?wZ(g;vYFLS&(5zg2ObQQa->q1eB6+o z65ZcRBFO?rnDf`8{yj23|Mb@&2An`Zt)F^+TTmgB{QqUJ^?M;8bG=&;$_MyM9qQio zU#rK*nZl^!k07}BvH!c((f&(G*g<>b1Zy)QD1|&Ri3^tO?g#w-O!(+ZUe5is=5 zoMlc&Dy+h2Z`S=U34fx9hJM5a+0Fn2q*F(31QGzhrsjWprn(|DXcWto0-3%E>?lTE z*xQczJR{&0{&Q&Sg5izl9SqN?<6OVwz4q^S{h!_f$SV#pCrOam4AIp z5^_}S<-18CBRR-FK2Cu6f!xkg&j^|lD*UG}e&Ww3A(8j~pVra8ruu)*MDi(EMHF^4 zv_3MFT}-3^*QWvFJ7-3=LL#eQ2VXYMpCH%&I0XL+Ajq6wSAf8%{Yof`5OoBKJ9VF( zcw2Wd(BF@aP-97AZLN?Da4<(vAEb9a{GEE_OpWm-gMHr5T7Y-sfnY2KY%x6Km$#aX zy{qlC=CvmZ+KibZ7_dEUxgJ28F-8TLK}d4&CsEtsbs@r3{F|BcV8=i${V#yB&3BoR@pvMocnTTO|5Io%d&k7+-?% z^jl1RoX{&Gyx zg>7f;@(k3cok@)nemz$z?&2G}tr0yT_`mI)9{a%)9uf`vICP(XIY@yDqys*!%e0U(`kBx&s`LJHD+#hw#)=fRx z1I(0bZ}!L2B^b+qe68}=`2IcMKKPP&Czd24f>JPnH<$t@xA;)<8h%{=CC`o~C8Rjt zB~P07xaaQxeipUR5u^^ZI z)%Mp4+nZ^$swAHKaPV8>VUZ;;gt*x`@15!kWlfa4b<(rnR)~9p!Ze4g z4WjTMTk=u7#r4mTVoLJHy9c@zo;9G8ME@+z(?2)XGih!uX>dc@DQ95_^@wU=N$D@o zzTJP0WY^^9Yc|9=2Qg4rU9`7dXNa8K;_~QJ6v`pSs~7TSiIRBn7uN$Yo+q%B!3*OM z4{YYwk~jx`&zL+8)`!&$Z@l{!kNt1H+A)8=c6rNNPY6TYdB;#vrPqG2%}wp!);8#F z{99Tu?M(Do9*K&>3vVwC@4Q8m4GQ?nl%kC-4+MCM1e!4(dKd9mq&fBS0I2Tl2L4P4 zuTLLzgMHlIh@5i8GK1e7CqQ&O9!#ZgXbmo5+YiF}>`ZUenq~hLXm($oe5$5vfA>E3 z#tWIEv_5dCmidCKlig3xN`FHvvtP!cGKd4wGja#i-iM(Msem;)oIYGwL?qp4(!txq zGh(z9$+2wmGPX;7*<7YScL==dA0G7OpX-~+6Bm%-rA8=ue1?GGjp*pzH70Z3`^+EH z>MstP?K(!{nk;+*2GzD#u!eYd-U!HpWT&*jaYu~K=foMi(ldwl-g#8cJ9-aavpYDC zlTfr`@HF0a4}3;QIcb14#FUE@bsF05|I~B~T9I9kKmD`VKz0SbT3sbZ$K^3fV1Mhv znELZzwVzUhAl6Ns)^3(tdmQIMRI>`wS^JOc3cfy5-w5f}-N-JvjDx5UDkGVnU-;*) zCR1hV2J6as3A6USp^<@3PTlg*qBH~MQg!#RQx9Qj|2KTR3sxf=FBd&OxnkTTtC*MT zt|gs6+O?sMu`LQCJH2bKie%J(jPBlQQ1M6k05E~jV9~?ggZS*-p6jRn{RAY)0rk)u zRt+iI1b^?hAm7dSacZJ$^(s%z2Vw`Xsp%nhsSuhSD;H_Mw^WzmjTw8pVK-fg*;-u| zn^uqUyxFb6ebhX5(Flb{|Px+k>(5kd-M>2ZhDEXCeGl z4ViyVH4WBDP>Q^89QKszGlzZ8-0Si+4reMoRZ?On&$|rv2Ih$`oe?lizl(l)Xll$b z@nFKGM8;QN7o;Wo-d!4?`1ntpeKLNYGzPo!@}lRDV;}=FgtV5rYeyuy41_UGW)O?~sNf z@0%OL_WypM^HfPlZx@#BXZ)!RhxK~yDvzZ~O^?g(OXG|Yd0T9mv$jL0*I&M0iMqr2 zO~lS~Z|eR)?(e4Db`x7InIxy|u{`*5n%HXLMNEm_1}ZsJT5f0IYZ)?=4oG|4bhBN* zSGJOF8@F+Dz2V$|S#6R9kvn7LP_Qn>qjPLvIlxHuIivlzg}?)3gh{6;rth~{&WXMl z3SN)v(f!w8){yKrrFZumsINKtFq>g8M?cmHlcM0LcsVX^M#ltG&18TR^(PPO9xOmS zY)SaVTa=k?(pNgG5DDTjT(TsIZd`#=GLpCEt>JXzl;1t{TqSQ-u80TQcvMWMHzp*J9b@iuAw!d8Ypl)+7|FzY$aQTfVub!2 zZgax?b9+O#JH6L+(Zlxz-G-GVi@JD5y9fFVu!@k~e_hI3QRT((>@qwDpKIU-nG2r3)gx6?cH}-$!NAV>l++SuBcrRPI62H02j-X1jUv z^+z3M*kdDfr>_galYH~TtYbYIa!ts)db2ke28E=1GC;!ocF*KvJkBY~CV=t3J3vO{ ziVug61C~d;y4>*9&gm#ykw$nX^l5zFgMW%t{ zc)?TYom^Y_>OxsV>mU$&aO;4EP?qD9Si-64I9sAT^Qsf{ays6h|1${jwxHHMGcg;_ z5>_syxLe-)`w&DF-~sr0h*@O{M6;M%&+om*DHg0s@2f(UB!e?gXsMH_G^XokT(cU@ zX2Dk3rwM=Re*CMi4$6{?@7k?RkcUCZwcT!OZOeXFYLsU+F`4`y_$-#1tFYoJaTh%_ zhF6vs#nrPqMX%ctwH^9-dLzjMk< zVCs7kimV{>YD$isb88u!j}T!V0cDw+_!MHvoi6Xz2p~00LWZPVqM*GSMNZ-3z#s7A)1TV8DGmakv^ zd|w}GDkIn3$JzVOKP*X@m9;9vObp>Sx3uF#GRDZ*_`IeV(fi|7OeAu^f~@O~uh$^; z=fX0ByR;uy5>-g72RfNC-69E6Kf}I-QFmCiue|ZZrY1I|o|&@{g?3F=ObUnCF1@)n zyrf&URxz|gn9pk;mtsC{ z)V;UVeUeGKW=@zfKA7_Bq2*Pha2Sn)LD0e4;?+p(sY?0gU}%W#-by~M-HWNz5>(^F z@GXaZ*Ja0)9;m-U%5mU{QD)|Ko4TuF>~ zqx4yF`!Bq_wQkF{OyaNa(`BCCISZ0Vfu6kI26PR{-CN__ts_k9nx{=|evah+n9g(5 z>_x*9TuGR)^`INeto&6rp#S?TIxxAS9`5ZBv^yajN5d+)Q-$d2 zE6u6g>$^%>B$WiJgu-nr1B;7`0c66-c|NZUFzqwG*Dy`}9iQsV;ej&)Fv%T>YrW}Ud-pQ;zZkZA68u0P>W9i*Jn)F*3 z6YWkg1s~4e+j-f&RalmZ7PXG6{OYUQdsn(g4!kE(_%d4F-9J+_sO!V z(F~a?_xBdNc8V6vb43j zT5Yb4KrjE;N+2uIO-y^p@#G6gY8nsyS2n(22cK|iq25UzwJS8ubOyH##kMH; z<_4oi^p@LGASDF2WE5HAjH-Hz{5}+Z?|%Xpzbee5bF<>z=O@10{_B(BIdRy~oWG0X zBy=d(?u?b#lJy+Pp~iwUie^4g!Jeugt}5y#KJQ*F)mlmfnZ;vj)cRE;N}2R(!}!xl zdA`a9hS_y(6TP-UAWJ{<#{F#XW=j3@P2<(|$e!1LAkGhVG@G zYs9JO$hW5C+FrC~@X*65m5t=oRL<3)#gV4(Cc6Scv?vC*5x zq5t|?1#mJU^DIX&<(4u;xZuIqy+2v|pH2pa)xAq7Ng*tR<6+%cyT5_k!m8)Xr&N9p zPj8z%iWj2gyr_ZYVbj%%k6E%B@xM7_I{z{c>-NyX@)lFy-){*?!><3q-lIJ8?srSR z+VKnCDJ<=+`D@a<5q#HrL)_kHVku&cow$*Bwer-b41a0ZBuvn%V|K^paQE%}LEAX5 z463Q4&dW-Fb{@&G9{k=`ugSi9NNOCkdrkFq>RjLM-_xI}3H|*75`QTq3Vv6l z`(G1}&ytG1i~ES>i$`1X?h{>CLGzeO6Nw!b-ot-j z@r9_E?3ZiLQ+%(?*Oi=^dY^U0Kcx%w0LgUFhC;U&&VDD7UC0d__ZW6raHVct(IRc< z^0^!v8_Vf1*%6MI9{b`EJq+vuBml?Yt2<%fRm+lpSW%8?83&ZtLTuG&XlVFMnjX8G z2Dm3l0HZ8l&-t_8+f)T|ofj6WCmiw}e@A$JtQJ7zjXYVA#y%Zpi9^q(@^;I>dyKOF z?(YS)C)hy|08fA6b%(<4NR#OxC7)3@AA#Y_#HL{U4?*Zv)O?Ihom>5S%#_E>Ms1$c zu?-+q)PU;uIAVbcW~5CZ{j+$`JK-Y^B2uPhDRPIn_pjSDIp5vBKk0mtFf5QfD0-%krW>NeJ{ z5FdWseR$+~JIOPOyI=Sq;naP4_pB;kjL{^|qob5hz45fCCd6@vlLRoYocq}Pxf$kI zC$yr-$_V*M zUsaN*<6smGIDimZj6^#<@>I!;b)o^`B>;`bT=)W7*FseR!#5`cE#Kv5rGpy|fesUU z^QO(&yRvm>2Og~7p8cqaC~KXs0_h=;f2mpjxm|r$DVT1tNMM8zMFM;#6%AAz)$c{@ zBZ=E4*}h#Al(@TS=ecV(jg^sUUYkgzatkwMVLWsY9^(I@>np>mT(@Xt2?hv?5=tvb zNEno$AP9nVHwa3%#G*mK0D*;spv0m(7Sg>C6Y1^}q)SODx$|>x_u1#%d(V&aY~w-r z*7weL%rVCtBgNO}E__4fZp4aum5%9K-5*`Kw=W3=%K0@n%bydpi!ZV4l0xL~ARdYU z6X}mry2Qqv2fw!J(&7CjP|^UO^-9QwK3tYc0N-60yw^v`=)xe9;JMlNZ7>LzEJyw_ z#8I44VQ9&?FOYQP&g&>PRUKX3Sj3~y@r#u+tm6cZ>I$U1yXUL>pO^XX|N6fNKJO4% zDu2D|yfOtU{tT?O8ou3{nEh!;&I!dz`&9cOz7_WmY8m8Xm0<4L?#Z$Ex$-TLIYA^tF-B;j8{ z75{oB#B$K7y^Gkqb@H#N&2jc|PY35K)X!L%&d7mM4-jelZNU{ge1YI7y0D1xq*DlG zt{e6ipY1VMaPpZ(qeY~t)Uo~bkfD}2W`k*)sIv*qh`a4c6LYD5Oo!EJ_A s0 zS!GdR@`_|`%e-I^33(gV3+~uq=Os*j?QIu8HjT&0bIren>A;Ppfv7BTy&lVfjYX!+hPJRn$EKf4HhL0(3>4nf|$ zKgSrD0y`8|_;Buni`?V6)5!wijMHG<`D)0ny2Zt)w%r4wJsf{+&kU6n=c+rkcnG*G zYR7<6`7F_z%Tl%kQ#4^waq-kl;-hHD5NLqAjK?E9%xuK{Te!5z@^B@=Vadhb6rC_W zrY4-i6*Slw_|L^1vxP`5R$jB_Fte7}0iG?e)_jwn`G*0Aq6Q|Z<6J*3jPEiS*EGku zf5Q3VXHHcY8=L)kCdfI4j!sGfHnU;+7QY)C^&9F?6#Cvv)A^t_z7FNxh+xc@M3jqk zwN>%E8BJc&SZ9F_kE4?lGwuiC+8U~Z_i}Hl5~GBzipV3RG=cZHkn|ZjBqkR!KuNo~ z%@*K@T6or19|@U#+6#{VB%Y*0WwyM{c9m09Ds_m@ewE7-&8C~PB-0=Y{_d58zEtTK z(KnVWB<6ebIT16_eecY+0NBjG%W^POYOUoL(A3<_N1y6E*O@W(Tsy147zqfUU5M@3 zsrxxWqEqWps=*IT(_a^X&x3gqWw55?<@mm6Oj1d4ugUP~|nxpd9Rf&Z{az*mfoRuWib)N+c~W1Tw7J@6Hd29x!mdp4f4?a2*1(H$^C@{<(J_X$jo zEB9Mj2z52!T0A4K<-FyOUtYM=4^E2LP?wS51BF<=aJZXz9z^cI2W$f9AbRpCE#E3U zrRQdi-|4k4U)kag4w!@$mBvocNoVFzYc3G2dL);crZQbxkK6U;4PFFe!oUPR@uY`@ z>tmTnSW*pGCrdqS5Soc+#)7#O82HMF79JOiJhq>ybA?#=rAGT*U0q%A02lQG@P2QC zylI=hG@UdXTIJv4bTENB^!qWfl!?{Uo{a6^()7alfGhkbYOUENKrWL`Yf(FR|`ZOz>Kj z=^!Zr(JnKspdHr-9YXXa8h}^?M4$@Lc}t!$G$NRl4rzHEBIi?Qjzd^P_sWGu2*GRk z_9b#{kiwZwSs96?e<%F$2FUscSHSn5kNd^cyT?aQHl9(sd0*%wIMmCrKvv`9cVI}h z-|7SwfICqth&aX~Iu-UI)cEfH&SQ1}nLq5U>5p4lBt0`@se*LuDX@-?0vo|b_$;^& zA2|^w>b=uYIg=2#LI*C_j`+!kWO+#H=0rhujPW}Dd%m%1?lEqBt*_DfRZt-+Z~^Jzi@F zzUklastR5tp_5ioxLSBSuK*b+~h4>Ah<4 zqg#@`7tI3NNzyk^lZbcM4&5!#A9a=5jq(2B0-V4{da&eX{5v8NDs7isSUE5?ND8vA z+wU!rf!9X9$++@6MsW1h*dk-X_B>G!0$u&h&UXBO%k*FlGnPR;w@az&D|ViEc46Fm zYlQ9aQL>jIAt7(t@(3=#0lk8{4etyymDs=L$C(8~Faai9BUMFt#ObJjeTb78l=vJa6sOM*oHQPbncZ zUsvr%ee%GIeoWK9W2ppfk!g1oqa<}K)3`Rl7)g2Xr9j!s&3eUVnDpX%6*<(~cmNxn z;4q3LXh=SmsX{NXBDIN>Rv<9+KE&Dq5K2SAuP8=p*uL)WOd4pt+{Lf0VqH~Q^RKk- zLzGC6*{62-(=%At1c&=h_v5wNk_$Fq|F_+sj(d2K5r5cHGxts~)i%`(Jv8Vd_P4s} z_xHaINi*L1^5qNu<5h33pWpAhPep0T&t!>$*IooN1!)gH@Q)t9o}n==mv&F@up*N#X$qg6tZdNi7`^6uByHk0Q*{*C zKb7?lD3#Rq{rYbd4fiD-H`LJ7mCOh7Z;bL^I6h|{4P*^Cv9b)n0x3^ z{uV9{Ryh5wyPDqK`QQ#(fSAQD56&JOfQg-Uf@IysNYS+?K^$rs{be}iWPjUQgY*Hx zZR)8N&Fad*o>~Ys0fEJs3$kgc{WdVvHtJ&iGVQNY#yDQ02bLPiM+o_B2h|~+lQoud z4j!hUza-yJ#lvD33UQeU9*umvZAGR{CKDf?6DTDQqU*>UT@TfcrX=lP6c*N5gd z;yU2=V*_{Z!kge;Ds|-aiU2gkVjagVqA#x$7l00GH*Eg`)pdu7C(wkpJ5zxNKi1Mc zjYKO?`@XOVfM6seDFdBMu{>BRH|jP?I>ZAPa&dlnZg(>weq(31p`0uF1%IG1$oVtW zye6KrN%FG4*4&{!qJd;A)VX+&)wbrtw`M58vEQN%;te$^KGm=@P&zd9uC>%*lAGq?&I_kKDYD8BsH;|SA6 z<@HLx-6E0vIYGzBME0lo_#3VLX0A?Y07a3~*fjoE%?h?V3@1jIl#c4w<$ zPOu=cBgn3&rlzK3TfGtB7ew{Hz_FDfX(Qign=GwY1l~({%F> z+G4m-d`u~rl>1Q-lMqeCt*f1|3E(ygB0#nIvc_MkDHhsLX9tOib3v^=-SW;HE$WTg z$mhIkc5$pQ-a|h_{+v7XTneu;2P)RD8tKZMM>@gZ=&T$ikC^@c)w=%fU*Z4$2=e<+ zLu*pDd=i;jz)vU!6)e@ejv)8St32xbI}`p!#X|6~bs@T$0m;RdkL8o;?CNTXRoQ&? z`}82~eF+4_RcNn5c+5`jGuqE!_IR5HYdDlk>%tKN4T+>>=^&M0i?S!>j%ZIj1N^Q zEEzv1icH3IS=Xq22q8|%^37tOCX|vtd-eWxWiT^0SMOd^Z$5r!{jG0M3MwTl>+JRS z_YZA+h`)0czenq_l%1V0H-PP5Ta&PN)q?+d-)PR$Q|fyV%uN-SNcxC)@y&ZOFx#rW zb>@N`7~!~@VD9|y)6P-y1*Gz))cy6jC@-NtE@4!c^0^iE+5e)={rm}s$wr8lhC`Y` z&K^k**j>k7z#j0=T>h8&mzhQ{AhY80a>Zn5Qe=X^*!re9dQi^80<93*HonG zP35>tU%Kjbet=5xerIbXk-d8KQX;!COQh&$F9Mf1+cF&%t&PiyG1op)Sj_dRi_;Ju)5qAeQq(dPbuPB?w%zP)Zx4&Q*GTBiwhSyxt3=Ie zBPV1B)VZ(y?4&GNYLS2NOiV)JG1j?L=A7l%M&QN}w`~%wzr;WP9O-nbB(MK{pr`KE ziH($oC52brnBx0AH%p2~*-6`$GrLyKr<5~r8hwL&+~(_P!V#iS=3JoT?FWEn3+--c zK0u(1P*2@k@CQrJXzku0r5K6c1+t(tjev!i0d@!DMm!es&wb9@qQnNjKY44oDak1A z_!(BW@^EC`k56kHo?S5*nzQDf#fYJ2lLH`2DK<&$y5W^41@RfqDL(5R(#0r7jWT}< z{>xrD%?uDEw#DGKHR8IJA;_lUcN5?Cb*X%86nwKsrzu+S$I?`jl9TIss%9@#!^WUZ zerfM&nF>LlQ}fjT*CFTw6Ql4A7lKXFd#wEZAK;-__9w+mm&q|>xQd&vjXvO$O>{l{ z4}^qo@f#)6e}^O@uV0#>FBNFLMWOnw9xwchyc6)=oXGm7ABD~{i1Gez9$i)jb zBabE^7~_QIK~f-T`*OX->2coTbU9nmdUn}5Dw%L)b%58}e%`h6L95(6zSF0&1PDeh zN99C{N>ke8&t&-LP_iRN-$Rhh>?pc`2jl;|OM+Y5u^L7(rxB`^7R0stYl#OX0L}Xa zhHQ)-Er7qgVn2qj#X5XCCsfx*rQ7rD*^wa#o3E+f_IdTgj0N%&epk-U%VH!5C>%iT z)S7HQxH5U*cjUz0ziKKIJ*wn)0AkWCg3PfmQGZ=LS)zVO&Z%BD^zr%QMxpTat)X@O zPGweGdQbd(@MU~O`Q8;3dhZ^Nn78D`MAZ6MOn$UxfYZWTw}tZ!p=UMPaY5f-q8A+wi7b_*(CJvefv@l0MIxjAHgKhw)z0}6rb_+pHnjG~ zq-q_vRF2vS=EwGJglrKV!4)-La=*nG+li}aJ&hb~wIE|g!p2L=vrZ}fQPvg~Zvfj3 zK`x~{^V$eJaF+qoS45T$pI1l5O8Gh=jKqf~q^M}jXE(;*gparfOfPL{un6>TuqaC~ z7aVbpgoMP)%(a!A4yn>2kny28XPr`MYy9ZF0=9Crzf-&FH<_Mf}o?-#-W;&;U-dAYb#pNkv<4NrE8rZRb1;ppXRjuL5k3f zM15X|clSHkcyqmZixSTqqtjWn^MLpdTc|JyiM0#Bcx}2bZUE96JnZohDbya9z^7v0 zqc!7d=#zfA@>>L6fhZG0Cv?r45%VBoMJKjY?dciOUq6lQNi`uz!QR8^G}6}Ee_o+? z9WE65m|c&ri0h=Guw^<$lFk3)>N~{! z@6`@J57;3|V`#l*E6Mcg&=BNF@(@xPICu2B{F=J_aaTA=(P2KMztp;~f3g;8V&Txd z>HBvimu=t`N;H7UOc~HWo^koD?W3eD7s8a2;2Jz9G{RICg2{WSktBrO?D^pdq;s>{ z75pDAKmbj=(vcbfhd^`lYXIYPUR`Ug|BYdJkJcW`Tc8&v>kc-$dKk>vod%n*<(w)V9f(ZC!1>$NZfL5B1uS1A z(i8zV$oeb`I6XBVF*M=1H4o!LbD3TOJ{^>L!9dUYxs~!*h6)3Mqxx@d3wXg7cKLM9 zyq`7B<}_i{R8ztD|GBBkJ1-hTB9NO)#5;zsadyg`>B`pn!gJl1ledTDoQdX?#cNH1a6 zWt%})7u85JTH6wp$jzqG$X$BY=3H`>cc%YZN%#ZanZ^G0tozRyW$>HWIK4rksJcyb z{Q^=n#Ws#*McS1JIA9Y~++|B^fV}=pC33L0_Sw#Q{s`JJcxVeUEP{vBwdR~!g)*0b zl@jpTbE^hYC=cTOdLdS^3!M_$TX|xEIOx-bu-ksnJo8s-z?iSGUyD8!(EEA_pX)^) zHzSB_DXT~#D?c$-B_)FdsCvaMLUp~0aipZCOCisxx#4lz(Cf#YHKJOJCr6Yva};;r z@YQTs%Vs*-i-${Mq|WG&Qc-s61Q`=Oo!L$vKaS<~9?fRmN0n89Xr2s)E8dark6 zSgZOe(}+vWAs{0`l~<7SZ5D9%Wpf#EN4jJ=^G)z7Rwb|eg%u&$aHW(VbPVG(n)usn zL|ou9-bC?4brz3=%W;HVob$OaN~E<2;{um11;M8>&u4NLatrXW*B@2#?kw+H&{G7a zi5(}3hSK3sh8$0;+7WxY&E`Bh`1Lg-rKL~XnABbeqG<{< z2v56{F#kanzoCWIy(Gw6JZJmxfHktP@R?SY#}uKn0x8|~H;$C!4gdMvfBzSnSvmCC zX+Bjl(@?}U4nF*+MBSFxaKwQA4JR}D?s;;%O_q;-@`6a6SYc5CJ1(QIP`RJC>_0l# zdIV?R;r-3EW@FE#(pMw!V{F20HV6|E`P9-Z-g_b*?Pi4Y;zr3&sjO6aktg3Zdt$j2 zXq2EO$=r_9d>;PQFM27(lHAo;iC8|MgS5=|;A-Y&7l3dYf{WtM2yc09XqivxC)yql z%D)?3(K@`=%&Vlbum&{$>&r>f*r{NTB*LfO-|LT-0m_Z_r-6Xvmqssiba?9a96f=e zGRtS#+Xnyam|w3MTSf~E>t?OqctQ1yUdUgOyZb!_Z_An5~hBYSYV{gCPMne>2;e6Kn zKs|wF$wJmw0q=qT%Cx}j7KZFWPQLHtK5z1Pc6 zfpv_NB8k6d8dM8*EA_6uJpRxG z?q700zr#mk?n~$}sV{DeP5yP5y=xh>vKFewvLGT8{61iCyuB9_Pe@Wt&COm5qpeOErCYm9TlL-=c~8u)Vjh(s;u+gpe62-ZIjp5wXR+=x zgC|!(T%*>IeZ|*jZ>yo-ErVn!pI8quMQWFNk3eY9`7r6XwZBHlhTUB* z?@U*On%-{@9Bp4@P`BsQJbJg5UyojSV|o>cp?-u}AAmbho<86+hSnU3$I3R&6lTpa z#_B0n;95Ue=M_p-#Jb@Wo#&JC#K?(CX7hLEuVl#BkMuRr0ie}uSJ}RGu&p%V*cGjU zm0qb{5$&RVzXd&dl9Oq!X1+Z334g2%q-QFKi^ENUp^r-HW)(Kid4KgY-5KWx7*9^u z>;)C&Z4;>^LX_O)TIc$7GKQPiT@vBb{aCHQeCek^)@0#j5yRL7UYJKz2*!z;uYc~g zNV@ARRHfD#F10=|xe!1j^=G)?R#KZwVIREC5NV*O|&<)$UAxQk z(`avHlw8n!yNkji)w;ht(TDLzdgNbXE-@8CcEUg8-D{5VU*7g+5pz?-vAloNNn*r= z7JnN@pf3S-$0oSNV#I_9yXWpiK3kkx;+#4)bN1Lbs=$SYj3jdO73@uSM;n59lPoe5 zTAMe1w(vgp78*-;>CIO=&pntAbORDGZgi7Q?`<<_6IXGFfzS37@2iLdHL?10=JEIA zY>J*wdA#@0o={T~F$H9Uog&shzQ&4OX!$B!COns&nvJqppZ)<5=xsO0WlN#-WRIV4gek?mZTY?Cb5EDfw4*bWqjWP>WC>|2DK zSz6Cd5bA1h&k>Oo??fdS9UB*tg_p+zC3WL0{RJ&4^l;w_F|ox>09FdA);HJNCBzS9 zRYTV7{xHjm$1wUT#P>33t6~pG_M1CgNmlAFJm0~ksAyb}ITuX)4PH3vrEWE)QeX;W z{mGr{*t3|KCl{minLo6&KGwjm$F*$oADtA0wLsREsEFIUv+D+}d z11~@b7K2LKz|NF`5f^kXU%uhtWK7#+OXWXLP%{vuK{NL$?zBz~C9vF8Bq(*Lr>l~7 ze#BMQ|KQxm%N67PE-)Q4PkYLlV_&h@I~~pqsHkeu!d3T-989E{6n)n(9NXI{ zc*IYHp-9f^zW_}A8A5N0RWomNjhgSX(J*?whs{UO#Bf<@8JDrj{eltA67b#pm{X}L ziny9pSx=b6C=l>z8G4t}LoZQ(mtkHkzvhc9w+~L5(VOqsxC+)Jdi_{qZfie{Ef`OY zK?|BzEikQiR2u!Md#FH5&z8R$igZn1gumN*#)+{hZX+I4drrL> z739hh&MCbOD)^3*E+go^yvc6a+xl-p26giUW(Ss|OQFpg5?;ugWPN2~kZST%>N3g6 zGKh4JLtX85FifPS$(G0NKx(PAnzWChxhqCB>I3`oZV0M~;`M22_Dbl-0(zQacToi? zF%)S|05+u}uSK^3Ymj7IhqpJid0m;(y1R2Cq4M<)CGo9u$6JGh*Ep3w%8+UD&&T1< z{0z-JL36Q{`e$jUET^SYi;_nf!^5`(VDi@83s0%fc^QddWjZF*SJ~Rt=Iwxl*-&K47B(1&v3m$eO$*sF^$MID<=%jw-!%0#(8c0Cb5cAiFRbASV%2malQ0rkcQvC5`Wl!l4 zlBBoN$z08M46f1F7Fy_#?!faM%~}vvU}^H$Vi5Ac6*fXLDw~9V!TmGIT0AQ62V+n3 zmq{J>uASD-&U&jV#~p?CLW{P^#>XU$Um`MW(O>`7wWh&*Oi~aJA2k*Ac(MvvihCqi z?pi5I!pnVQubo{QZ)^Byzx1-u&*2|-fh?cUDZT8(;WqrGXZ!9Oh?BQ?N0Uqd*~~|D z#!5(dE?$uD=MAj%8o#R!wsIrptX3%M3Z*`diOP@xt=N|(jJKr`djE<+VHYOEDY=*? z;vGZ-M%$N4LOiVe*%K3bTH)j?{%Z(dM1?bJ>B&HmaTJ@F<+#ZQQR|?n^xEU-PC5^iBiIXtW*;TxI{KP8Vmi(or+fo(nuFQr5q@+K#5^(MqvB@E&)J}Gz59JD$lbcLQhdK_6hwNDQIPT5iBM-e}0&^lM^E^3%?yUf~|y!MVhb^!X+gb&r1EX3^}hkDXh z5O?a8Ai4E7=F%eNv7AxUiu1m41x|_8FNB|7diVfv5%1E9-3vV}^RDkpT}7iX0namU zODOyLj8<2lt@}+LTs55xkM=9R2~}B=LR<0E6-BNClYzT-uxXy zmTYteLoptm-4PBwx2RyHS+DoZrMw#9>VPH%>}QNyAtVtCls0_Wo#Ry}IOVaq>(be9 z9}Z8Xy_)eLWaEM^PQe<^y~T&evsLy&%l$hxTT@zZ1DmPzYzoJD^{|uP+s-08os?;3 zs29`Q=HSbK8<$z%t*E^GsH?#XX^W@px6EM`vp zPBqMzb~yXmx3Gml^@JL=bG!|je&=^Y(~_zn$Mqe>SxRb?_Ln)^36>uBI2xF7H^;VK zSD1TiKf@)0it(a;%WF=fYyb%vhxEOvp3qY6rJB#ekn0SqY4e7N5OaNT%G(Q=yQ$;G zZPOtvJxcXTG*dDgE&bZ>qM$42)vH2{*3#bnH*p5IioN!b^gnOd-*xJcUo7QaIzH86 z?zD-&0lg7k(S|2jcne@jubpH40yF^pwP?_x{3_2r^WZ#D7IT7Bdf-ABLZzrN1qmnQ zqz;oy)7=N12bsUL*Jlosw9{x84QV+d#h;Chuc%xeXjZ2ZTt*lw5g=_xM>6GO9>8F? z=rNga1DVLW@{OArYA((*Yi+k~HP9}dpc!E>(3HMIbEEou5sHGDe@q@3ebScV^dNs zmsjCliwjRvkz(-rULR8O@Qyv^dw}|{%E^C8Ior! ztL*zEVGE`OF+e1?rnD*8_7$r;xi(Tek53nrZG10&r*)Z#QabUw#!`)~_c+AYNR2Ax^A0ThRPAs5O5af?*pr8k>Za3q!z(!`Qi_T}NPg)m zfgf%VsAtp6ct_dASxcI9W3vv8Z6$Qy7B$gvPHk-h3M41oQnCs{pG68Ck}+!{ed^B= zY-4v>M;_zX^(`Hk$Di!Zjv~Xq{E|-D`t5F^rS{~#{;E= zI@UDtvQ7MRo~5W%%9>0SMaL=bPh}Mysu9C%AFqp={;N(YpG1}X7v1ywE&DxAItGqN z$1lGAYwL46Vu?knkI2j?LC}onG@XOwOkZEUGV|%I zQ$%AFg9N(tp{1A4pf;G2cDwspEi$)|`94p@ILfdrgm2_U2Vo|!!z3%_P`X&@rzeCY z^GOU_HtMwN!<~1TJr=x?-jhD4N5%jft0X*^CYdT=>3ocyO_$zPWEdKk&kqY(XJ|Ao z;3i&(?qqEhU*M^W!RA#uz23%4@T_ zjin;P8RU+`zT~z;MvqDPnqns!duH4;r(=#=IpLf{c;zC0&Fu!^b1#nQ^hxNCyuU_K zQwrp2K(2-$;}s*{x!qu#z4usG)|IKOG?e>#h|`NG&WAFoQ+#o zss3lbs+2{{;Qx!JERdBV1@PtnYffM#Jr0ymP<|eY{2hk)t+H6XjyaZ+^&^~Z1^Q2X z5uZR&NgeXKBc3b**)#cA8-CB8O^Z#2sO;yZw9p=W`S?P9Gk(u57IkSt_)dU3rAcyY zkOytCunec01by1ZruCG{5P#z{VSXzSVbIKTSH(K%*tdr^CAEHYd@+P`CSt=dhzx=n z*9ug5uOhCo)(YwQi4$9?R(ty~4t|9(;T1Hh&vE!$Zm=*}D9m)u+Opc+GGvpV>S;LE zv7llpk2|zrNv{2{o~Tyg_~Q`@iwyG+!N|z7OWM1D=Hi6k-_Y^bAPOTQIsOO{u4b5f z>AQ#==GzGu=HOJm)24!PK&eEhL$t@;Y`i5vKuzngdurKHMOn#e0TP9>F=_R?Z94)g+dF-qQe=T z8L?Hf;-m3alJ42%yLww_{a+v?_+V?LTD5#$ge|V97;ntkptICv{$ly5*VJsnt?zFS z+Xm3P9P>AH9Esa*T(y01V~Y^;v*hk??fu=%L-~I(ga5+Ue(NOdf1B7gvH_~z&J+6E z6_4`F_uB32c_9UlUkQ_sLb57VU*)aga=mi9j6+W!)(Sb*8VvUpD%v0JD}%Yfklr42 zS0?jsriUtE%Dx@J+ewj@+?smsd27(<&0qb-FJeYx3(BvSTXeh=7%+)UV$SQNluT*O z1bBlhp2^cd*Q~jC#gz(8W}+mnoxZ>+EQNC_0z7<_8P5fSyztt+V2{TxuuY?24`~49 zBQ-JW83({xw^S?8n|jYIz{vEC#7U9v>KF644igG(<}-qkjcaHM*R{d~c`*XXj(Cs` z*f>W`5tBSgJyY4!e)pDaH4Fvh+t|3NuZ_U;Y;=(ZV_DoHux7V*aeiPhRr=&uMoNYy z1&4?Pw1_IY*Muh z`3^2?Rq5Tl4>8B0N{CGeu$~YV$9dhz|5TKJc*-3j zi+_Y-${#_vp#I}aveoZDF1@JSQi>Jz_ORLT5@&AA3$3BKngYN_7lWlRx@OUy)W0Mn zVDTgrNbkltja27&w~6N&iXS@!!yW^u^dRj`w}&=5^ELx1cY=+iiR1}MT2G=6IY;z^jS8z-jU9+-em^jdxHn#M>(3kyQ5(YO7yXrjKn5>E};o4WkPXTQt*6CzOQFvk@% zPp8do=P9-lE9;|sGm9)#xeB0HPFBf?pNg0t&QxMlJLNCDvv9*Nfx4qvcMJfmyk7-!H;YeqI5yVm ze@P-IQm$Ci@Rb~{?Ac$M?yHjO>KeuZb8%21Q`R9x8{@ynM-$)LH2@~A#P@o)vMkJ| zIvN~c#udQ9&NsHTF^iXp-q?Q#h^`Rf1OPqZlE}6<)!qi88aJ@N;F4~7C9<*!v%l`R z&!ykZE|B|OJNEEzf1p1|84&c{y8oI+Gby0Hc#x)Ne=I?Aq3*JZ@erzB46{xnNAqE< z7XdK4Rs%{{Y+!y7(sVj45%g@3kd~=E9QX3EypWHDWH4Qv0@kAEDh{-s8J57U@T0@J zuTBbonI|2rZ0r_5Yxn9kl`h70&mkDLIE|jlyPorsATF|eWoc!eDa(5zqhbyCBI%1B zan%L-=o9KQl4AmBqpkGMwfh6Fw6MDrSBl`s2--#4PRg2`|Drd+JfI_)WJWaMFU6{q zKsO|$1&+~m`&fOPu!}bcwA&3Qj)PlhJD^V<=bugeHA}kTp}8BH zGjm&?ZKfiLPlRy-MeqGNtGC3!b%fu%XFcW>?_={+l=saq;;QG{7K{Q|_W~D~e!-9b zAv(h;yX_HC;?A`>njh&4o?qZme;YD!sON0V6T9dwXklT|Znc<}q~@9`_3FF#=jYWg zHbx%6z3`@@qiT3H64wS~hmH=VO-{I9iTIdScZXrUHZe=PlQ>#Sq{IhiOfd=P9X2%2 z^$2_vIoYVvR?&f>p!<3Qk5}66q?}!pGc@ueDvIEr4_zj;ZFe68n!pAGx0~>?A7%Ck zp}*1Ve2<^z3uvLtkj|u{DHqG`H#)2mk!zo4e(IxPbVTgfG=j zERS(Z4OWvAW4|`bKTTcWfvvYc1?P|aC_070Ryj@yfeSC4n4M@Vj{>Nb#;AnF66dZ0c`x3ZmLep(`)mpHFt~i^a-i# zj(C?Oq}4W33v$C?Z?r?I+hLg6d41}Y;U~(dC|tQw^@f_XeC@@dmOYTkSIguvhshF% z=y|vKR(#4igA*>^U8Y?so{H8R1leQi$2=ew6lK@Bx{5VYIX^~4YJ6WL#26)o&hF$X zo1;badMxkRXVt;}T8rRdg#RMEGZVtvoYym?Uoh0`ZM;|PDb5XZ%BZo5OX#>Hk!oQn zGNX8LxMH!WDOSoW@WvBi{~2PR?@NNlpH3J$hGm2+tPAOZ3wR^UC1uC+RWk3v06v&R zUp>t7;gy*HsiIX(M$(~YAT%79kix1aMXvtDyUsd21TjNvcr#zlb5V(6fo8?9g$u!4 z*F3p>%0O}9DTs%i7M(fJOQJP(Ung1X`!N#-sd!JF14)5?A{OtBF4*pz0bXVNm!3uP zQ?EGd3kso6wMwpgesKTX!@=XmyzGg!XE`HsV^XS22f&=|Q%cP%*uDoP%Jo`RZC+z* zUQ^}vW24J(ijCI80T%YZO24Wg&JEK*pGgljDrC0h?i%bl>c$)SM;rF{TNL(yAoibK z_kUWgr^oiv!Jz!YgtsmL=j_H$a z63jNX=@kCPDL2$3v<7;Rt_Q6$^_P%%QNb6&**TNN)1xO6B+*B_fsGc?<8tZty%??i zF7DPcX>a2maMsq*SFAWIH1bqo!7GdjEWx_Tm_ji)eh5D!nn40)4kw@*9G!48yT>*quYFAZcTj)6Q?51 zYd07n$8+vy#asv4o#UZ&!fIb7;P(ZE)VyI3?AG(=k}HFL+1{~Js2{HB!)(}Ki*^Nl zv0$4?)VUON_D551r9Ac8Nq?z{WkC@}Z#0I)r=9<6Qvn1EIR7l8WOthLd(!--@sS>gs`WlaTg!K4ZX8%2faD+fj?F7a^YX(vx;a#_fu%}#4&38a zkR#@Sb)As3phJ^pW>ayiFlPtt2BKc>S5YCm`CY85f!1!jvbTr4x7v{G2x+%#0i4KyM0_T?^upV7)njw4yARa_(2-lNg7WUtVUM02^El8d=yeu~MiIu56}=s@3_>dpqYvRCNBnyMeiPQ1n& zf2m$ore{(|-9RnEwS2Ci&5dZ34wUj#q@C?Ttvv~k07zxLjf%6Y`l2pXqPV>J*6*-_ zpNwoDI0IEj&Tj)-UiGDL0kqp0`+)oowgOg)l|HZ2kAHuEQfbfN&My$-kosf!8iSUV z42p>-Is)w+J018spTs-!P>8KUXLmH97k!57`Yq>M>{srL*~TmI*XcGin0>U%vu8O) zZw#+YFPC*E`)!@)yl1>pyPM@hNk3?Qf3{o*@^NI;hhkbaV)Ct=!vso*NtpJ zU@B+E^ft@&|7qtH=Dz>=;S}Qr5dThwiPoMnYy zMQsE%dm`lobSqG{R*;a9!dixHPbA};z9qgLd(&pCm1a~J`!{|LCgH*VNkqiqzRh-hg51h%p z=3X6Gf~W?WW#MaueoXNzf*=IMWAA(Yy5DfBu;5o%irIXr8%f*i*>m z-yJ$FSm3XpV!7j!vcMmkVG5ouuEgAISm=ace0_DyJW-U5a+X}HE$wqiQ>HZ^9l11f z9DDnRS6a*%_J`kKh_i0n8k4aK09Pn-4&XZ#;ZK&@O-CyICz$26f@61Bc`g9YzBQuo zbY8c87)wF;EyL0qG+Za~V^#xLXN9@yvevq4?y)NTd#fk3L>DZ(PZK?;9NW^sv6kn- zIG}SqCh0A|tKM*x|HloTsUh6kS_G<>bC?e{U9MXk!9x$=-X-%l-R&1a^6=2VOVsc3 z^bd5uTL~Nz$E|w4zb!?MfAcs~GfGo%^QpXtUiqjpt`%Fksdls%h7Z@ZsMgR&)Mrf5 zZe5`W!($7rc1HPG&LlB$R^puYhO@d_GoO&nhKbRgylY;OjT>lUjeF=6HMTBz>SDR0 zb09c6K40?V6YUWEwdIynN%XPqn|7<{hZ=0lv$VsLo1|To6)CMxzj~a9Z2(XYcrU@B z1MMFaGxx62MA!F=AE4>Hv?A0&+tu^Mb(05EoS`3yxl!yZhB@{DBSw*r7yi2A&%f@~ z*mwLMHkLm^yyT{<4*f6mi-njOyRIr>v6Hj{&A;H~w}!y(gdse3Qi`h6JX$LK`;uA? z9~#G_G2k#Y?<6N4;4l=0Ya3*Iw5Jm6&W2e`bBY!;4!@SgTOk_oeuB@V7lyKDa8x(7 z-ug?;IJ(~4=vEs^Gcu@`?_iSfPHfLQTDAH3MDe5bxi0ff)zuihmc>g(*B9TbpU2_I zS@~oGH=WA)U2CCz-z=}&&o6v^nCv=}%M)UZNs8hIF$GAmF*k%rkX;}Sb}jL@G+!jg z0w~-(J7w5nP$&3MACI%ek1uKrY7l++89Kh3@U2NJh_AA`FEVsePR#=}6}5h9JEJtN zOYz{xG1zlsj;3#``MkulZ*=<-W9JabkcXWeXdZi$C621KCrOcaZe8`(>Yb5UXTq-- zes#e(rY2p9O5W#NeU3k~P<(sr5qfN{GEM9y5U;VgSCMA-ZhZZ^3@V_o?hfx2Pn~P<@YZt?i2dl{8gs_{OsG$m0JjI=NK{9vx`1Oy7 zPM)f-ehFS^XGb68CcI zh%6P^iPZ(n2sAF&(5;$m+Gt67QLl>ZNdssPY~GU!mLRw2%w2-jFIsJ|Ejp<8w4(JX zB8n6|090iRB+d_@k8-*ag;6<9*}_M6R%dOyIP>24+egAb=Q-+pv5{+R_DEx2d%My6 ziCo|Tw_N5d7^2Cu&;&$d2su&MGCNXTjwqx@mxw*o-oy%fc=9&Ot0#^4lki-3;`o&) zymM)``IUq?2fX3s{N?XALH}+w->`f`O8?69)p?y)xo3)2DV%oTlmzMMPZVtqJ1HTn zBQ`!nW{9Is;ZQ0q#S=#;jh~KCNUl*?Z^N2U8@p^`F0o`o&v_B!fEZ4i9->y4y3i4lH+U zq9o2A85|RwrkQ7?cAdxIi{%f2|TH9sBnf_8qTgCcBlMERMd4(Hh>k(d55!+iM*12r(@vASwabM=9wX6)&8`YKq$ z(Op;ckmB@}u0Oq_n4gpY?CE zIqiLP6dZdLwvwd+^?_+zwCG5}D9tryUYg5ljXtrTOF+ZYVPq<~PJTm~#skQry zp8ccEsgAfT%CKa`lzAIms4w12R&6{$_@&i=pf@aC;J=NN4$TkGj?BuaOG^t1;ST4< zy+9LM8HjBq6?EZ*Q8Tpk_hG!okERoy2xPu?wp;vOTb91xv9Dwf%;J_$SZW9G?VA$vP4JOZuTtZW=)LKearTDj;Z|x2N03wx zfXFtXAj_!QpOtWA;oBcB!1*3!ch-zon05-|CBsyjijTQndte-Ny%%oXp#=_~?GRLa%bcA`?HF+&Qklm*1O zp`@a1@r{ka3h}0@Z?>qtef5$+m&H9q@F+F%ey^i4G_i90V=R{Z5V>^bC$fy$+92e9Y(T)Fln-CU1t41vSAW?FPC`&w>i2aYR z`Clv>dpT6eDXJN_W3TTcOFQb4N$TRvRw1Ki1&WD4-IZ$=v!eBDKTRN zxIOhMu54p*fcF@gMy74d7e#C@<$9;vc!NJG)LGsj?vmV$e4c2s(HE9xe1E z_(>PRaNq<=&{?z;7havAh_xF01=9T3y$zil-{<=E$7%M(#jEX-?wm)5H07?`aTpt# z*tE2(l4`639X(f#;0uvW?JjRRR}BA_o6MG>6vgb~g!Ht8&m?dI^FJ_)%VR(MwxCoMXx1}uKIva z)az;RrYq4GztG{|Eu?L)rgij_MOAek6NR9HZSS^Yms9?(ztl^^jo$d=SMsYa(UFVo0&t+c=~4-xJ2i6pLnHb6fE z&?r1>%}1PHjpp2{Lfv!b{wBN(dt){p2{Vn8c3tMrMQ8Zt15no8R}I8Z#Qx??_%m;L zS;t8^cd(zM`FPuoW?LLvQmT;Hg$P^a{n)QXVjIyfW^kae(GfPcO2w&RKTi{)W~-t! z@V?CDJPIIXjKbCF>kC|)GEvZ6P32H*5pFG=sSG?e zhzKq#aHIGmE|lM<{|!@?;D$UT?DW%|D%z!fH2h}>;F*p)^Sy4U(~qV;W~I36%<545T!vd zNRhaV(jko?rGk`nmxMVgA<}szFRgS4f`LkRmrAD+A|UwN2kiO2^S-}-=Hr0iea>^v zK6|gd*4ppV`y(@y(k3WQ+6>}b&c!W`SPj%8m_d=g;2lQaV|7Sg&1BfaHP17~vAyZ= z#pZLp_Kw6hQzlw&=8GwpvR=tsztb8WWR}!}V>#mXTFbiVOpaH$wDl9smmh3Iqqz*( zbBFx*@Yy17?Onb9^cc z?mG8Yjfx^k#R2hg)B<%q>ZiC@)`^tE3kqJJKfF{NxWhl8TaKsk9935a((8h!c0!p^ zSgg3D1RHJfILPIXVndh9-&KXgkM@AX{xZ&YZdj#pM@Q0`8+_kh6%3We82z7m)UUVz z=+{4V9D;}%7DYNY`wvKgq^4GLUL4+%KOVa2kFoqoS05A-y8f96cY=0elrXkI33myp ziH|KyQTDOBNn>^n5Lyd!uOPuP zck=KUqQKbn2RFLBP!qP%nDOo76N$82{ec{7ozqa7z;?C%fv~WY;0x>t$@zzO@#@~g zPq7QpYJXnz+%a}S*Ja!5CFtakNzpaQP~)0g#?H{%9(3I{-jG`V7BQ9jZ!iFMB>>d% z$IGvj4KO8Fs16fU+CdIA$EYlyaARyWynw?{o+)U|Jy`0n`=HOCQvl>|E0S}>G zrQX(_W|{^0!aH#zO{T865tYHb8BnGT^e>d18I>F80&wHclR!1`t#|^Cf>BZeDvX5K zaR3lzkO^kqv+K|HxFq!wCju{6O$7Fl0VNpp@>d zZ@gD`82kwKVHesB%eq8G68pDi)Eg$bSj?jSJht+JsM`YBPLl~@XVF{jM!IW9zyhQp zRQ&$Jo(qn*TiEH`xSYPMuUWnyto#Uar_T|o`{XD1JYgIE>F5Py_p9~6ZV9ErLIVyh ziV`Klv!bwnDh2#%9}I3EIM^1^5H=%!awPns!YEU3a=p_VyR2g2_eeaN+=OX;qej2Y zet2i0nYd%8rj1`WLFpD~R^N;~w|68%qBBe2dL-B1ofryt8I`HR?m1h*^Z5PKDtwHd zQ`d_~TJK#Pp~wb=FhaU;7|D+9+gG9IPbR!FTKl}Jgc@C1icHO8rPrmlH#zV1bXKVd z^%bQ)8eee_ObesRms+{i{rGfA4$+pKsw~FM(L<#~L3l80jUyt4EC`b!O^)a1qgvPF z@jnlOy&R<&;fO3IKb{o4vEHdjs@ixytexq#ARfIEQVD6r;#dZ^v4ap->lB#CtJ)+7 z)C7BNo)y(4Zg2ux%eQdKXOKr@V+jbH+-@kshcN6k_!h$YtD^1VU5S8UtXU|skg?ZHJvMI z`$tQ{X5ujjIcoTw@;(D`HRpEg*g^Tjk4|g4={}XBvou0?IC$pugEOGfgf~rChoUuj z6etRGJ*PUI;C~HAQ2A2@+rl0`1V$0tglR0EvtfI$SyO9mX#$eAaogMgeBBIbFQH%B z()v0-m4v;mnPCFSC;24t4PuT}^-CuZnv7bY?ZKxWBEVJImPGW)g8OS=ao)@2Sy{g_ zk(x46`wWJSf$PM?H}ljVZOxx?jV+jbB%|k!<)7G)(W`o2?0D?$dtwhLW2*=^RRv9< zp~E?L)*`Do8Tli)O0&x9SUU!hH=n6lay=l;7!>@izd{+lC$4tnc#C`cS_ zBh8pGA265EFsw8cL#QqDArq~KXhtc2{O^Bywqj*Xhqh0PLrP#cbX(O?Xceqgd!O~d z4H6zA^;NPYi`9fct!83|b*-9C5L|p|$%ITv-fvSs6)E+i6k3q;nvw}a9N{9?QgC5> zPcAqAguk(c>&F||{L?3Jq5P2AYDLSxY0Ai`fq|Ov#`e8OD?kQ&q7(=@)6nPR^`niy z$CsB047SQ!%)nE50#~>8$BI98c7iwH?#Bihd z{*)ja~BUs7vBiD)iG z0FK2YM-9VIK)N!Ge0k}@&xSGl-1y=D9m;(skU&+qfjs}~74Jh!5iVqVL6aNO<7s5f zf+y6InodzvRD8n%crcPwjYMq(ba8l}e!&BlHHh|MdJoMxA)fF-%rkInn7Zp`u}~p# zb!!*UW-bGRp{}zE4W-_A-O6nkbub(LiF$&b@tZG8FnE9m1?`sQ{gy3$#vnV1b_vJ? z^5HX?pY!fUTRUV=>A zEOmPz8$NLvS(E_6ncHQIY9vNwGZN7TzB<@`Az8SWU{tq&Ydrq@?sTlHX@5 zi<2f{dQRGx{raH;iH?3w0b*f%KS5tl@M&*!N&fOdetX1?ZxU(I^)sTLn0+V@8E*sb zrb8(N8qCKUWp8hDA>K;ACHPi0$QfLUFun|O&}uoh?+R?0+%1;B+7_c#6O$-DGu6YVfR~K?OZN4c)P|o& z7J@P@)sTW@?AyKhj`I)W5j^%@Pg2J;{X1AT&>2_e?7{lcIYJJAlrdkYxHj$DHJKZAm#7!_@|S@9A-{mRW1{dp&8uv82@PM>g4 z-uLxGG2!Jm*O)y4t`_6xBg$m0QQkME+7b`C^IrqYt~hu(_-Qg^?1nDUOB$a7Pwn6+ zx_gN}Fy>#YOkgD?GHF{z7kWu};@0{q#H0qw%xgjsNu#ev`WSsKelwd=VBEE*1^*S<_Hs2s@H@pv65p`x2y~UyP)GwM18qO%laOOss zzFjL+JQWPTt=|6#F&lparBfl`OrMyu{(TpxoizlYtN?mX-GJszXAXm7$9dF~ai-fe zKGVj0y`DFN%SNQe|IBdx{`Iddb3b@3su$VrPJCTx=&w^z0kxH}6u4QI;5tzNDN{d$ zHfOqH7p^%%#{W3vjVYOW$d-`bW`J(QlWz>{Z!ZHD>qm>V6n3_m__MrkQyXp{Y6Pz` zI{en2_kyxs}w6^&rK^kfQ77=8HLLj9Uwe*Naphl)%H?j9k=jIS?!D@phFr~Mw& z{`E`n{N%Nz{cdf#qHq%G$dpySy-o;Rk*H3$$95q!`hUx$esIqJ*Asu=q}4p|i@IK) zVEp}=;3@v=_58X9%4-NuT-yIg{M)Zl{sV*DtIH7ZWZue7chV_CG5lyH*vDz03w#Q) zx@@04%3m)S&chY|9tHQ?IsLl3n~x~pJ@|6}6p~@0CEO;UA>{txrQy`Rd#k0GeA773ky`;p!B+ zBckH|B2$mp()Ry%pR)VFK4rlt$$Z-h1XTKdwV01fkrB}d7vjU?T^Y&xer8B~-;QJheT>x7PX)sG))O@yc6ckL$f!4fK;An}KKtc~la zRj*Jfqz$)KD*^jVVINlhmUC|`b8GUJ{9pSs6B^!(_uNG17rU%qyVV0MX0P70N@Q2{=@iU|?X~>l|Sm`Uu!@WwtTbwJ~#j=nO#Blm`Y=J#a&kl>)EB z#B%5>Fl$Ub2)?<$eIgXK&YF>|6soWMg%a=Yfdcjh18H?jRnDJ&!71ANfk4j5-I#JF z?k?w;7tyR*#t6QWCNyc};Yc%1;-bD|4fkE6E`+uRVqhFwIJB?>W6jDU(|PaHR=23m1SjQKiEd zeK6Jd2v%k+R&&s@w`P*M`w)@>Fzog;0t{CAt>^1b4xK|E%RL5TJKiUk{)4=` z@-2JXL3gveC8soI(ZvTkrvg`)&P}L#Oa0jNU)TG;L#r7jY*=oSASXuU>zD~}h#8C2 zI=@H+(NOIkC}3~&fl?^9<5C?m-9;D~_%eNRg^uDmM4*7yN*)lwJ9nI%G6c6*#fNO# zw-3IDrmV}I@&2+>fC(@3TQalmc@^~m=sR^$H~;$ACiu^44&DF}4OXA{*xF*Vxj`;% zGY!72EgvB+?Z4j)<=YAi8D*efM0JPphkBZbpHu+M z#7+Q3^=(*yVm;kw0vlmI`Tkj5A3%Hxim zl6FOdzoP)?X{tn)_mo4SrZ(swWsQn<@cSr!oe_q|ceri4{Uaiy<4=bDDjWUjZ2jwn z2P=TPP*9WVQ5d4diHG^X#Jl+NCVW=;%0x7E6p+QIoUn<=m*i2;r0X-pK0)g8ut8`` z=5Ki27XO12ux-ghRPY{y^`}~*+QDwf&rQygtor-R3sn~U#h=gcMs)h^tmw|sAN_^} zY}^5Eq+_yYMMn==@;?yWSV8_n6f_PLnqDwu=1siwI&{HYTAn=z#V(hw5d$S)vO5oV zkWs@4$z$18w7k%AhAC;ixCk!5FU3z)WwG!`=|Q1q-fZqO_2MmU|Eu8p9|;=qyw5eb zB6|N$`!>`3(Z(GMlQuYbj|L9*2-2G)@thLQBbWgzi{=f1WBBB1Bt z0row}hP5ynAf%U3N*6w0!JG2kTN*i(mf~NK3c(BbEeQTnjePs&<_q+KjWw5omEZho zYfJ_B*aZ>mfZdlTIB<6#?-U}#S&{F)e>aqJBr|1sm;B-LK;HGNXSJXThlHEeBoOFKQX{ILpTIvUdgWa$H^jQp+PH`>0CB}c zROqOX!h!~L+96pfq;|20ySegl+5!hom3E29>hL3EX5C{=35t`5(+aE^tK9Xq)junl z9$~94Gwws1Zu8pw`lo+YPU!1DcCC*cVrnZ{`fpC$z=v#8h&axo;>Q5qVXM$n%AJEQ z^)qcVtEaHmGpvuqvngO`aYqiH-6shzJUM11BXHLfpys10YeKMj0|N!~pBiyL1M&R{t9LYl6Lj%$1+ZTW>r+nexBK zM}IG$zUG#CkPs(MdNk9*6&<=|m{FMxLo0jGObvJ_<99RAED)5>>@&;Y(=!A@;3Mef z)qui8$sKMkOPBIMcyP!{ASItLtw&HBw}EibMTagZpg9qUv2YFedzx3t(eaJ2+6H<35cf zcyHa#fO31(lnKYP4}8Dh&IK9iLQ1j`3czw=0Od-k)u#NX>*r}}FL_l`*RRKiu!SV& zYq9*JJcGDZ-Sqbw{UtC=7O)vn_kWkPxOYC!BCI{RiRYJtM@S|$KW3Vk?45deGbFZg zzK3+!ExiC(-7YO`ijhXhJ@g^EA<3m&$u#Fae}V%uH$+^k!wDe2i>rfVSiZXqq9x3J zD{u>JnBIe1pAIOBGeMl{1ie2K`bqeidO0q~r?AUS08q`{V3(O5fbAEo7%S4nsuaC9 zBM#Xrf3xE?mwvFgj6A5ISM&(9ZQUTn&>EZZTLxrnsmj<{LW0Fg=fNb>5y6a~Zp2ZYTFQLh&dl#XA8Xe`QY~If{vpQuC9}x9-yN@9PRO_R@LKSNb)fKo0vbE!7W7c*X zD}*Cf^)bJ*$G}e+32WcqNP!;jaWVxL5H4M9Qp@uMgZpjrwe`vPFw_R)OOyWCF0RI> zQv>z@#t15gY+rzQ6+?^ss*wZzdhOXn7VHc=p2ffxVq7`On{Qjro}7W_HTmIid+W#< zJ``<6dU z_9S_lb3iMbo33&qolx)|`Ct2ML!}%jMuPQE%+~k{0dGV@pAoW$N1!Ja358z1Qxy^Z zj4mU|T`q7|a^zG~R^V#k8Y9Hz+g^H#=!bCELpPSUBWP`PJS8s3xc2@elm|O}j|)IQ z)bROjDCW;1xr(5O88cW?*~0quz#Kk?%d_AGf+SC8QoEwiaHJm71i#C0841u^Kv3Kv zRcm9<>2eX$(R@0qqRKw1j5rWPOKg#UsG4{H_{6fh%_zWdW#$FJ?z|#l=!Y?*Kp2xOC9|Vs_2BG5^F&~8M(~&)T%^F2YKR``3*kcX7%4AA0 zzM$*YrAMZb3Y&!{I9WwST@ZVN{TAcv`s$8SXKOA#Q)9sA1uVPss!kCzJ3yKCu4E6D z5L8t zOTLmqDMdwS3HEekEatp*s-`#@&Sd1=NOgiG{3FNo4$!_ChK>@s)S3%)(54W)Ce7gb z9HJV--Sr2LZPj4tgdQlnQPAPLq{7&SI9HH)l$Aji*{|Y472RC)2l?+ysQze+#(yW3 z6=;+u^?9fqZXY^oGgv`j%E77KGy=);ap(#JOES(%AVk?Z7FzY#ap+$7n_atM$TD}z zXD{u+LqV-Y#4X1hA6hGfm%Rs5?RtIl?kU)L0)Bjy+lSZ_6CuoX!&Z%Tp~-KmL!I(V z%pA9jz~ykCSvfWKWG^x(P}DX9dVG>7Tp`JJP&&uv>um!`Pr;?SSSmgTRC7R~yN~oE zy1LyKC#di4jrR`s_!B%((~$CQB)7S|WCxXUyvdW#`B@Ims_uPxVOER14b&!kN`>y!bv9C`Jf|+vHgPj6AIPv))w}3!>ea3 z12!5k?*a@@j0Yh5cre3X%a}q4^nIxwNF2o#B>n`iB@fMr>j3L*!ybK(6)H03&Tmz< zQkAl~7HbTS0_Qbji%7A2&n?MYkQo!M^8`R?3O+lbByuU@xY%D{c@EhL9&f&#fBeF8 zvX&FQ;1liJWc0Q=Bmr}cZ)vz`9SqKL5m>@gI&}fWkdbCXI|s)2ku!!TphV>teDhE} z6aI4TtVL$NLPC;h@Vz1tMMZ1qpkx-0L~2*Efsd_`<~`dz+WWpXa5syolr41s55PS8 z6NrJsf}^(nM3k|vx2BWKoT&{p8P>v)brB13CNzQdyh_yQy|?=%5pJAQrv?1KHN0!5*S80yvT9*xfpEo+3gFtRBpVk<0SF zL#?HItac>k3>0T%IZdzF`u}2)X*FfCdL-;z-2G<6?`bK@8YIQ){I&1+b8t0QXnD1tRuN1b>ru089&3i! z=8$WJRULo3=TdF>o4&*a2t$XNikKo*Bm|ol$8s=^@sqNlkhbrQvP2AkgaEN_DL-0c z;MFKyE+4u42@vI6r?6$XMq)#qtPWl?Bcfg;<6F%_dIChz~Dwe+<>TI~EAfSC^&EgP*L zpAzZ~3J}Vy3fDkYtgX3%C^I1=@(2?59fCTNC%jY~w~f{nT==G-Y)+L9>-vTT*xVs& zrw;!B0s~T#-Q|=Ag2#c1@P4b4Dc>#T1jc2lyJtaL?mLJDw#LPx`xwnwuXR_o!E2?8 zlaLn|UBW6turrSTOiO(_;c-y}e|I&Q{ulxkI{dZ6r}uA?n>78w!uxb39W8oivzsOlIMxur$-z-8 zevt7D*8 zHsN$F2qhr{3d^`;P72-*od>x@BOijF&4$dG8HFgYBK-jUu437KM?Z>qMv{Rl)S6s} z%s%P}{!k>EU(wWdcNl5&P*DxVsR;|D_jP<}vh6n_w&C`oVaJM0%IvSCe7Je<^R6QS z)4s`>>izb$Z&)MzBSQAxKMG)}iKRB^XDsh<7d(Uk2W?cfEfDh5Y#NZ?7=pciJU;?i_pJr#TPsCryG& zHCSAo{P$`8kH#J>eemC~)~ZBR*3C;sgZ(`a?2&=Q+srNo_o?$pChgn+fQU%ClwzB$EXb5@;uXInCyctCQnpi zU#;+m6q{%7o8bdrKHW464l>o}hC6F9@`X}Kfdh92*v~<@NH?5-oHJE-Kna(10x(#Q zs9KTK90q>#V8@x>u3UazSrS3oMahKGURZHSde`SPgoFg%=37+ZzudKOaM|(R8NXk) zzmPjL4nmJkBYG?6*P#OASO5$5xKlX33h4QSCI?iSQk75RMmp_kxakBrOrLHSYODh6 zdH@PLJ&`$OufFKE;D@(9#B_JS8CUsfqOj$qS5^$Aiy!M?&7a-&x?``>Y~@hCDSi9qYb zHY&HdOn^Xvy7LVg!IP}sr-;}-7Np! znJ^1600&$IM}?m>(Ze@)s!^yk+tc-h8UI?IMq-Kox2y!ZLr`hB_7nMGXDM@#_@7I_ z;I0)Rr{R2qi+=o<>-aks0G@**V{#HK#GkbpX5s(zTYo^w<$JA5TwuUszDqtL7fOJW zyp`G@IqZlI)=mduV+Qn>&D!Ps<8VPpmtsE@o<9z5e&P8P^J?@f7X|oPj{t2-07_2# zKcs>mfDwPBv7+#w+W+M4ek)}Pu-@CL{lzBb_vFy+(tDEnp(7|Vu|xH78cwoOe|-pP z1;QrnzESAXlle+JtgXyRwisnwoFWm0q{JfER~iz^a?>Winq^<3;15vX9`!4aPX#1F z0J>!?ygz6HjLLqPR>a%-Z$OQpgxY-fcPsKzz9$a^v-`zEHZV9$TQ}t+lsiW#ug>X# zD}SO$t%Tn)Un)fiSMe&E5)$fP$vy!U;QOF(-$X4i5183o=b6RPgzLKx zAAtkDw(jUM)Ke~ZXaW5c6jeaRFGZa~&;>2w=i}qcddbZm&SLhbhvN)fc`wT(&GL3x zE?Gs#bb?nIC`hNL_`h5RYo)sG_s#N0jFYZ!qkVLmg!+w}Py>zW?h#15UOatKRqU%i z)Dhlz_d2rs&~~@ZYm!PH-0NpJ5BkcjOih;`R&E6+ka?3KjaXEfATkEd_q9pbBm1Wz zitunOlsghqRVKnf2>WUr`IYxRHf9W+sljO*vTkg{$Sn z_6BPq=WJ}I{cJ}=S@t}=K6z7d7B$%MQW0MNcne?u6nMqwIGP`LT*7|u=4<0T_#OZy z)Ze-+#A~uapUy}kXsqOQm>%;*&3YBWZ!af!`0OPcsVPk zNW})m4(eOGG6Sp1b6ca%w|xurAqwT228t6D8L}}I{2dMe+G;zfiaDq4Oj@x_axYKl zI8~HkA=%-_%6*f*I^TM&TEnZx@e1F#$M|D2T*HIVXY+LRAssPkPeNLc{^NZlz&$IU zVdFi7yf10WPn31_G!FU=*@pBeWH-#YQBQ}!+U9nNNV9om*MagXa5Fq(j>|z9V@~t4YoIk1hW>;X%x>H_Ssntp zYkk;d>&G8%2$nTIPYTuHw4lKzBL9EwaXDqip!!jmT9VKfuP`?s!)x#c&9L_Kj=!ff z*Y`)s8R(53RofDjeORzgFzwFVX8>Yb4m-I)5}Z@O4eAbUrd%WALl5>~iM+qm!Hq(` z-one~&9ko~EmYYI6ptQk+)tu+>ikVRJ*_wV$^Ju;k!k{~th;X2CE6$0$8(4r(P>|9 z7EpY2GMB_h1oR^g+-A*=NfrwCLmg)=++z=QA`bdb;U-YmI~x8UZw~QUwk|dC;N~}t za712@OL8MRAX2NP=?#oXH?u&k7U!3s0&{H%0XJCbx*S$N?M~nsC&YySq46y`kckx|RHb8%d7Bi z+XYz|==}+BNI8egX^GReu8w{jLa(2Q@@kMAi{Qv@=X+Y6ku()Y%RLI=s=oUncz0h) zy7#Tr`??;=rI-m~x-7l(Gug>V^Fob9)p>_JHJF%`zWpto_Dbhku z&9Ni0J4qJb5jEqf=k;bSJC85_UcpZ*+vq_ew|OZNt{{ki!2V3lM%}K;Gw5IcMqr6> zb%)*k+XY$PR0ijv>3@s9=>cHY?Z5*Yu&N|7(yE*$*+P5X2!dg|-LS)#M(Ryn?2|!g!%Und-PZ@Mb3CxDNl6 zC7`9IIzzb2l6oqO)vZGm3%Fu~rQHTG19@{u@u&omwNn z{^AM^I-+~|-pm=&z$J@6MM0Dz@%%{SwGuKDH4tqZ*Sar^NmqNw>PRX+^!8wKs(e8I z9wqK`g@Vr2T4FsG=PE=8-sDrnObTzyiXb_q&~cAdPfb=QYsXOJ(;Y*y9}fl?;J@tT za+{^G`!ZSHK@}~HZsloxa-F4)Ypr}_W1mazTYr8j^$iQix;jSHzgp?o@_f~x|N;HCSa9ttv2LozI1!avZ!89-)bXzWDl zyQjb_40DW@q|xK!Qrw19Z_0n%=hV{$cM>h><;O^J_YTHZJVO+8WQ459Pj-21rrt>(q7^?nZd^bWi%ZE)xiHHilb46X_CE9Fb3QJ%8i;g*NzOP11YHG@#=I| zO2>G_-*1j6IVZaJbc1`*e#^ov8pRSPU-DEpFJ;~U@A@d~<`c%WrqG_wxbwPft4J75 zR)di+PbKD+ADu2YQJY==Ceqix{N9eY=l^`8yPCV*M~FmE9BwNElewa`rw(i4dp+a! z3e7LmCLe1!LU7k#dh$6h+9IH6)nHUfEz7g(2A&6gpnTOp!YI;iI^DA5O7Rz#%4>Kl z5p4fkrRxVouSUQf)PoKikh2~g9fD@woDy>o1P5w=cVkeyax2X+j+e}n(K8fytwm| zHI#E!y7h&PscLMqrtG-rSB2I=)3n!6%$xsB9<-DGT*E%^zu9Zhjs)bhi-0!!6~G(oD&o?48B$ z$hy-v_)>sWV7iCsxy9(FTo$czw}E8X1m*A(uGf~l2E9H6RyFy)oxAUkKZbRfwP;>U z8KhJ!uI^aNwuoF658Ksq96Mp33GBv^_330W7E@Gq|8$MR6ewnxT(qdE3gd;@OTGI` z-ZBJ#c`$POp+9~c3kIZ5x&Xjy+pEx+u9a=*38^;Ma%Ef76PRY!{|>q6fR&FqX)tP$ z^T|B}tKtXDvwV=@^?;Pgc;!^4@4h{{HWJoiEcM{*nY`r&jP#OA_iKSoHMyNkrJ@Eb zUMvyRIC!;+9;Yib==}$Zx!XUgHeSbE@C^w+U|!fwZKWytblAf_rsckpFb7kVZqU}h zji&bI+C8U~?gS0>{#-}R9A9Y$NQww*VNxwZ0LA-y#{#_KrFnz(GcuLp?#OSvcBriM zGMMD`I|!%@Fm9~JMd>^Q5}O`0PCk%b?n)iPyva|kKQTc_ zDNa6GHbgzR#gU0%AxB^NRJJ3E)e+aCE=WObPW;S2zTVF44o(gyngkT7(->}MN|5Ls zDRJT(7j9lml&vXA;cqj{HBYX#YfqdXOn2y1(Pg?#!Txslq0C*F>!g8WUGXNkhM-#B zq|G;LVil9Wr~k_Y@j(BV+=BPJVg4bz%jx@B@<55KPOGaW)$1tv0?BD)#B&gQ{|X~e z$lTvzDZzLM;3(3oK;34K^~Rj05{fTQhORMHwv8NUCVWYS9mWiSuU5iA_7>4HfL&Qg zSc|&949t~xDH6=gST22j0l=W zR9#T3MMw?5>S8x}hip{f^3UctdrIOs?VD$>n=>@>rAccq|@CAxyMZS9y&BR`jP(y@bqr}18oI}*iv)9>FtnW}ci z!LBg3zpvSdLbT&ePLG%#SENH9b8*!xH3pKHor+EKNsAZu>&3K&cSA6zdgrFHZ-OdW zwt-Ug2p>mG>N2Jzo4)_kqVRm)%vtl`c_Ak6Fl|v5%O-k;|3%*yJN28N&9_mdj@nE1*pqL(RV}^3^4^jq ze%skqhfRn2?{73s+B*^*);dady(6zjT5o%T@|zk1Xp4a3Tw7wrJ$1g@Qc#LWU;uk6 z=Afu3gtOB(BwsJ5R)J4Zit*iY8?lP0NOQrQ>e8@$Bw@$uM8%V8&U6>)eX=@A%^t$~ zSt`oCc6^+-r5<+zSQJIo*qo(z5-HvlE?ITG(3wS-AVK=Z3wE`HbkpDv@StdPu<|fQ zV(rQ?7KGAWW*E34+q@OfXk^S1l;P)!hDsw0k*?$29+)t00?g7LfVmlRK}>y?_shkl zU~(eUZB|FQro?))NQ>qJGx>Efs!D&)?;$uM9xQ=7f6O_&7g?BdYT4{eM zRp*jeUq|HARbBf`lacBt$@3#oPv@&22ee)Fao;}l#4g8vY=wN*LNB1CFxUQG&}saY zoQ2`G;^uj+xXOvUvo)t^F?AH|<5|nX0h5CzA14==AB+;q>CAY@otxLPXV2R?6h89D z=aY|>v@NF3YLu5u^fx%Jt$sYX{bztL@^FR!X4DPV0Gukv&i?4;xt);5lkA3mK!8~* zMd9!ro)Th|y2)gIUoU_A*=L9=hj@f&#xR!u67Wn5H)h}QqtQud*6S1!;$m+Maml=N z1LnIzM;jpX_l~$veS|2>DO4_%(*?kGCdU^v0L}@n*2u&0t6a)+h(HhMd|hkhh5L#E z$NI>dL=H1Ck~rd3r1=zAu09@#49=2u4W}^8^3Gy$Tp?rAvxr>qXm0JD>k>bQA1Mig za<8j+T|p&gpygF9gkux)F)6Ac+ zs(nA}fk#JV!DLBmS!Tl(y_DV_)z{*$74;)a!7$AJ!F+@l~j zPv*o?=`vBlR{g#&VgX}ovpq_*n0)5I+4T-j!s>(2?$dhT`oTWcgYXxYDOQDjM_R&QoUnfq39YO)1QBfFPBpm=P zbRkgM`yit^qIOgUeCm+6x}b6^9UsFg^6Ns>)K{xjbk?1a=+f*rVCtfBuwk1Q=()W6 zZb|&{wk+T!@$NO2C*g+bHSm6v25D{U2L&&eootyFoS}ZgkB;US=Fw733agx$*pe(8 zBhyBC8dBFt9)kM`H|tSOSr~~HopwJ~IL2J{rJ7>#$BT5B>tthJU!6l)sv0rbWB7F3 z&gHrG9w*TwwW64v&I}yTpS6z{ZW%hS)%VCZqdx2I4OHS>yb^h2EV70 zt@Y@x$b3bml9&8(A;%|8K6P=s$K26Q{d}5{9^;mEtzmbeq?^@PtW66ve;Zu#J1psz zo^U;NizIdU^3pS)agDHgNO3xed-DuYG2_3>=AbxLWre%0T8X_*VvEmPGG$cdXc;2X z6e6_b4_4=lv>B&}q`9V$lZHlXDAy%a;ZRv(B+@s5p;SFrDcZD@KM^7N@T)P22abc*Zhjcg&HPlX?jbLfqjI^tTxaO6MaT z=qI!vaA-Yp6Bt`*59Tu9wwBj%-ZyrFZXi?mz3{S+X_7 zJ2L%+CIS<7luq1Dmi0>1dT9rT&X^7(OX@&B9@(5YDc=zZ;6c-3So&X%f2CfQZr06D zpArDSyqCs9>Hl1Iw64(|tq8lxN17&!Zy=-Ix9{OMEI{$ie5ID4-t6Ln0@Q*8F@Ll6hE z*~I(Z&&nD+C+Smh!eqdhj=zO^Cy8XtQY)o2z1`cnWX_l>^$>g`C4x#Om18O$^r|$J z>7PxcCb&eO3eWOSvRx|P-&T3U{bcI}{W};pv1<)m4b~PF8a)SlLPPoTkIaZ^(Jl54 ziG`7>ac>(cEo=*QD|*saKbE+vH|?KnrX4qWyajkLdK-uld_GgVM4|`8NmNKB!IO~ zxr_Vh8I@g0%mXVJ5f8>=}!%;VuWlokb%vi_y&A7q? z(R)}g*#_RgE0KsE={31NbI?<~zW8E@RqO1BnYHYXsWBQl#%b3#?Lxq>+OzNlYQKZOV?+13W{!@G>_`paiYj9gUP$%Fo`JFLZ5Sc zcShXG6l^0&o0sY`Ab5NO zfnXZ4#itpk{M90Z3~18A5#u|DtX{NsosOyV6(<(ff4*806$YlJ68M-DJ8Zz1zVgd- z9aDL=38KVgtQ963lbOc3UiJ6Wr3*&7~c1GxK+Dp4FDd z-O=COp5p|Iq!ckm-%#-35nVB2t<{}h%M#&@@Y;ATS<+X(dIG5}DD;%_+8m?Uj|hnztLo@AUo^huJ}2-)nCxY{M|)P!i^(56QYp?`|M z!#0;lHvHk3<xE zT9@9y@4@XQ)>HvJOSLEY?ukd5%Rq%H}|c*(e=V-@5V)Hzr;)^r}~HR_z2$~W3X+p zaMiQC-70=oy{Aou2X%EF4+l+k74&a~N;x>GeOvd1mR>`mHP_CY|hI*`>!lJMy(Hvxl8joRNvf zs)WRStB48Zcm3j4v2xjy3>~~RnK!!=oQlUYn3SzCB|9C&i>BK>)uxvU&y9|i%&d)- zgtlqcMS$9<*!#*C$2dwi0aH1BIam9MWopqw1Jen#W(LWgFQ^(vlOeiA%&bxORUOCV zA12KDH#G=kE$WCxwSMC{ewQo-A^V%t|A_qZp2{#RSrO~_vDE~^x=j78KUk_#KBZAB zOH(>hsSO67lO0%lJCZu$I3Sw6s%M!s;-)4@%6V{bk}u6oecm1)bH!uv#YQlE*T zV+W6(W1Lyfx?b_HOYv3s3(s;BqYEIE-9pw%M3BuS7fA3>!Vi6%Se=GU8_T<+T$9#82^49X+^bAMOp}I>G2VI+O zV*#(+S{|~l-+Xj>lh71I7Wk)+`L0Iiy#_C7gafr&wBva~b}36A!zztmxz#S=h3}Zf zH09}Nf9#u|_-k~MN{!1hK9($42jAeU=~RdSN`r{CdmIvo_NCGhgI9m{4)*Q zVgp=W?L-Mna(Lrx2_!3`cr~dD)EPw$hx@*0D08QI;}yw^mhQR7wK=>aihKtk@l9Cv zZv)ZOPQiJf5Zc*_@oC*g%+*-fdHVCtcdaoOXs_W_N#+m;%g;Tyx+B5gUe9eIYuri2 zbz}QfP9KdQz%HDWI2NEbk)v)hD+-9H>Ve2PN}{Wej_Zz37^#KNsxLo9if`jXuIOXy zjh?%>_2VL9S2-e+(n1$wo-ge)&@ov zh*v`^@vK!u8q^QY;nfM6UHa&}#}++APd!f^IB>C*%O?AzEsCUo_;b2rbw z%Df}ip)qStG+1o&p*{FM#d~IV-w_mAuQYp7E2g&_d$Rr0u4t(e6>oMJdN(bkd~=2l z4d!kFM4s|haUBE5S_tD9J#(YH2g!wjqHt_|+$ae#KRz`Cqa$#CwAMr<$AHW1Pw>-w z9Py86;qcRf&Bce)Dvx8jFcU{d%FWbc@_OAl+#9M>#$uf<9=OLc9@DzM7)i~p_TpXa zAI8ESQ|BE5CS9rO2=0%`nQltTo|R&Rj~%A-Il5zQRzShhcH<6!1?A;0DO`=9>(i_^ zURt6MeOC0P#&xTZW;b?WHD@N>^kn+*dO-U>#-tNU-dleUs}xMI*ncT-v#Oc=oOqj- zdkcpadktx_?G+B=8wlOie5CP(YYR~)gHoh-$2qy|Kit`Pc~p;qDkih{x732Iqi* z7vsHqGPxji3p`g*=GA&kU2Ud}Gq+7Eo5v$NyY}@UF_otA&YpO`{Xk=P$e*f#d}+n@ zOV}EpO~qxO)+F^F=P#`!EsQ@eAM6etDndP(dL|G_G~CUJd%vGlL8nx^{SF$H{*EosJG(+V@~WjYhGq7+0HA08#;8#7*5YxVf;nWy zQ}65rDDJOy1toz6b}!xyaW5K*6Vd;CWoNwDLwH=WOW8!TQ0LqIzE+U;9R*(~;Bpq={DiJnNFn`BA@KGG^+rw+=hnoJ|#FmyT$a%ZWHw zw+Z;GVMuOYF2Lt|d|46=D5?Ftme{pD5vG8=$BmVGIb5BQP;BD+xh?>ct*^#2czSXO z!!Oz5k~eoQ++%r&j`fOV(1}ycL~Cj1K=DTtU`iU{96mU(FFFp^_ohF#C|@mq%s*&K zmZ(LzlVX3VIF6+E-FR8KskSk8fxcKr_~uL5Pr~UdvGTU7_kkXFo!r5;{0oi#-?krZ zX1&&|45EWL;7)lVRblSaO+&!`Sf|$jamiXqDxo*;GqH;xu=y=2#fpx9 z^6#YNS~TsDksxdJ@fHp)q0j5x(HmkkSe~6I_A-lxWPrC~`js{(`OF8^*P4mCJ9MsN zZt99KCuo)wg2r1a`FIvltm#>CL&BH<3Oz!YmGuMMvP%z=O?6I$rjI-O+vSB_WK`ac z(Mw}C&7eI@+;BF+=aP2*f+olzSGH}o9JC~)I7-H^iZlpSCrAp_w#8I)I`atGm5`RW z_-3aio#I&XEXm^6QH#O2Z8du0_$enxJ=Fd2;oAL)_1NCo+8IwR*;w;5c@}yh0nG3o zV(aLYP9=lD+vNe*dLkfZnvtflm}c;0Ca8mhUC9>r-+0pM_YtYeL7-Wkru&Cb70pTc zB>b=?HBm~5p)F&ZkNK4HME_D6(~{@zBWzBm7z6NOxW|s2WiK^e@We6-Bmw;s--7j6 z3dw%v)DqB`C+g|=zAfJ@DHpg}L%YUQeHzwGJ$qLT2Idobz+BL$d7HYhX@&!L!jKfn zuXz|k5eUiHU0AD9iFEsh1w7(d#3Y#-hB1`YO^Y-` z??81wr)>n7y`6~;y9@Q4h>zUHM_s;b-G9Q!n$&Jv(NfrIpKwQcQnyqJylGV)%rchq zL-B!34StcU@r&p8`Izrvcj_xq7c$N0oPHqkYY+iXK}I|s#Vu$l?`kvr<;70OF%DZF z%1*bJy_kNkVp`;_)5u}x>ArW)<4vW#8iOumts&lYLYfmhVtU^>I=VJ>;>a^PlRc%j z_pTnfcrij#ZtX6I8n4h_7=Et7D>Gq}#e*l6=sg>ceu)P76;h= z2W(03hs-vp3z}Y= zT}}#0)+K$5{WL>w{cO)q%aenZ*}NqPj}h^rBP!v0e9I+8RUb8t&B4T6mtQ4nMdSOT zTC%}*LbqbNNH4A8*vtpvCymvO9y{(CRFv(UpH+Fhf96qNefo=&yU?{6jjw5+<`pPj zGOnsRlvU)DAbB`49un~f6}K&q&w9<&T(dDsmqRw?Cavk9EeBcs2!HH7A8K#|i`laM zCtGYrOIXCj>ATGgfVbV)@@;B7o}Mf?E!UjBvFS8vBr~D@qE*TVAfu6UiGZfi=2-ad z!+Vat|HCJT%)wTQj`M{?pL?(8h>O20c(+hV5NO5T-D&*mHf)(gxT3hq;ZOau3<>0w zZzM(jKg!-ZE~@SOAC|cwWl(}5CEXzi(qoWHqev^Llyo;^(=9D1gM@^%G$M%73?R}9 zNQZR8v(F59Klk3>_xXnx40F!jd&PUjUVH6CGN_qBdvYb%vsUWKBi2M1Nh4U_$#0{^OLLJMyq>4 zdkvL(J&RgqE)+ zkillQ?)Vt4*4YKt8CG!DKH2x*$cYGV2GRG1I%d!C=!mvcD>%=@p87m%u4f2%A6)Wnbx< zO#a^&=n&AJQ<6x~N9tR>E-4;2ftk=kyi&wP8Fy!1+JmA@>U>e!*)G^Li{r z23J)Jf0wiSwIHR0cZDT!a_wjP;hcb_bqpXQ5%2fcncOxf=}%2QuCs+mhA4(=c32Wn z?I7}nm%T$Z?MO~+ABLa(5jAX=%E?xouwQ=B&c;E&uzf;0$|QYNS)T#5yG8J^!YpFa zaXYO#LG@r#c>B$)fK~%3`^ngb4*ehjq$@n84NCX03Ulko6>*o>Q3?5Y>Ux!BEFAg_ zreHxyB*tAL3mHSbJ7|DnXZU&i`74+EScTy;{7Fn$Z@Gh`f}et06}3d-*ODaCHcHtZ zB$%f5TwTL!FFv-SpAmm$V(&((aiWAXdy|3}0KDj8Mq+$1#*nW3_KtcW3sn7 zi~?NVDO66wwbGZ9e$nu}!U26R*sxY|JIZ)&=<<}0MrcLOV=@|Fd477(eyzY2hAhT9 zX+q7R0Rcytpc5HSMQYQHu(}V9Cn(>;DqNuSY{>kB|Lc!ETT71p3qOYvfNs*aiBc4> zOyiUj&aQa!)ydw`%>nR7gC)f4x50nkP`Kh>@^a2U9ae%_-0=_ZyB@K@*k9_4w9rE6gAQ zZ_k0V&FmQX-BzuC*G>(x8mueE1|97V6TnMLqWB3SbLI0ei$CNQ3!Pt_6JH%-q`L1h z3>35a1F=oAP6D1EG`=cRg5<=<&@6-njiWns_M(N~ezw0aL=kF{dFg??mdB+LGkMq0 zDK)=>^(Lb?CQbGR&oVbMYmB3{_3}Kr$s6k-LH2^LVb^teS5GU?I2d#3zZV8(MK8CU00G)a+BUC>!`8_jLWbL&(npPsQF z%PiTDnM^jn=fO2u+!^+K;l}w*n8;2mmmRym=?6@OJ6r^NjP-TEf-Hxr!rj~MeH%&@ zN=@BWO29i1!dh7|7YZur7c<^>$02|`1Ogy83#C9ZUJFk^ojcU8dKtdcgU9{uu+|F| zx!qR8u#_BoTpHW142+L-y`pxX-|LmX96Z4W zMul-65tQI2l4kGQ9b4iI#n440XlY^1suY{<4qx%2%CN#(#t6`^Au9&G*^m~BmrhF2 zX?u}@@6(Yfk2?7mz>OM}pE{Qjj_I`^H?FS8l({04wlt~by7QE7v>D^VF0uVAcm5dW z|Fw8W+jb`c6_fK`$&2!Th4&Y1W=D;^^y&P}r`BDjVMYe?en|>MTJha7DwPishGQ@s z!W8~ek(&3mZl6SJr#@`1XK;14r)r$OAu{dWR`V`(HbWyyR9A47*;f&Cz01~$SzjDC z>=4KYcRQ!uBgl%6ypcZjtyt17{!i_joHkoHdifQ{QXzLgf$7Q{VY?suI~+jx#&LRy zJy&vbGc^q?Q(Mzf1F+r}94Kv*bGRZc$ zH_?Ug8m;PS2{Fc2N*~=e%{8Lljpd`(*1;_@k>q-6jZu!#)X4VsNwMPi?aRmw=4gA!KnG3u7+#M*MaFzL#Z{p_{w6EvSQu9;}bgB3$CLPihQ0}{!EwZ%oZPOavt zn83X+qa%td3Z4UR+d2lyOlHXXXmmY(jW%;x_|OOy z>tq&COkQp+mWfE54v2nQFL&azEiqT&ouzl^tQ5eI+xB!YB7*lm=cjFfjtX@=;_~ zxRTI~y(sOVaH31w-01mwdx@7w>H8s9Ra~5Btnlk<$$A@t??1bv3~#HI8!KP46F%Yc zr}6Voe8-vNYGgSxuWo&Hxlrpp;x8(l*QgR|RTSN%pT}MN%Kqe})@>rWWIgelY65{J zQ`((diOoU-f~Yj@%B{rLJ@@j9&6K8zWaH6(JLT{G#B=A-twMK2KjA5i>|c8Ipc&Sl z3t)E1ZP`Q=#zdx^yl9-vWB-Bc*DqFJVn_(a`fs{OradVW=`=&xR9#cIEr`z4J{!~; z=Byx8`ZRpD-AuoY?068FP242E;))vndzn@=kEKXj_(iqU>jk4evA3Fo>?hj70J~!h zc?6ih&>w*xN84hM(E)N|-MKGy3^5FYdY;!!Ief3HgTr7r_(4U=SoT&*B)lk~p=vBH zkEICCFne+l+D+tVjYoI3n~aDDfy$-$)kinzr9jkyfgRvl&#YvFAS`U=ZkWymtW&^V z{cR2ySMp$5z-4(XEuVU$3#c*rJdv^u7`WM<$Uleo{pZc{qMYoL2z8Kl9&0xnptDL- zd;BvPG?`28e)44>kEAFfHGldvy_77cP-QBz3wLVla-W&M=>2 zngU-0?%KDX7;6IB)#LloN=g){5Ri`plKz-}1k9->tXRpeqCu;EJBovsLsW<@oWv^r z1&0U&m3o?9k|_88;W7_2x%LFC5r99^*<%W$4)9xqP=^v?B9Wzud-YiZ#c7jDzllT9n^n4>XR@PhKL<>fQX8IysE8gpFWvs&{_)oPct; zeQZhKn`rb0rC`8JEeaw@u^99yNQhal!&g+G`)nO9STXuOg?jR%xtrjFlk&HIqyMAh zXs!?mR9)I@C()h??`37Mue*{<%vuazF+5ez-sU*)-~s)f+{!b+cMS~sSiR=EZ2R)(er6N7?2iZ29L)g&j@L<7TP<7eI3bNF6NiYFIwCi^G$3JQY z@en&k zvl2gt)9PMeRmkDV9b{W3XAwDb}t-`jfKPZm}vZx*Bd5TZxlZ3<49J|pq3Ks zV0Jo!m#QR4hBDls{;NC})@+UoSrp}pmWc9;+K$eRHukzU6^+KO6!FId#X6qCf??wk z(ZRir?fycy1EXgIe7jPK;1%?eN z1h8QKoOH{Zg!j4b&{{+UZ%R55FZ;l|O{yhnmI%tO;P0(lgjiYO$c+0)Wj$x(1710T zt8PTU6w8x79wr4 ze)pMFLLrXoV_Dm`vLZ?{md)Q*u_>?6ahM>92gw-_S|op5^|-|SXBumCkIW5_5_-$O z4E8703mL3L4JHG}!A3chg)%Gn*xi9^W8KvyVZCw}O!|g#5nG5=}e$f|wDQ2V9h#U?XN z$vvB1+qp~Dzke7SPMEK>|G2rp7nkYrWu7>n_3u&;fZa~~w`H$U5M4m?F@*(7j$vsr zuOS@646vOV9aH~ewv>WmdU>G%J2-;ILQQQ#{hc378A~V*E|q}Mx(oLOc(e{d(DUL3}X#Jab{Doz$)xq^M!F zG5?QFDDx15!`4hF6`qBRBMC-ZN zFV2KW(Ufh!nsFE^C%`{(UNHTt_~a`vZ+o`8sKf67_eQGUZRcnJI zMSx;(+#UI+(V&IP-36_P$CIp(wxR)KtegAhBIrC8f{3og7?6;1+S@k*L+lh$MD32X zc5Lx4?ztZ)Lnkd}m~}qG-bwG0aF448SPhTB$#EbL7C=3a7TQ(x4l-VHv`CCy2b^bt zr{NQgmBYI;9l=$S!P}~h`fu$ATH~*sZ@6znDLK4#Um4=3og*bjn=x5VdKaAPv*Hf7{72(!=somm~j&)P#{7`KWWlpxQ zaf|TEjDHh`gU>193BN+6DSw_L48{z{npjy&>von$V)ahxT*JbHseC7m0a3fzNY6Pb zBcVG6zkeG(N%Wv})$#P;T-dtAu)c`D1AztLvVCc-m3obh_J=tM3)GI#TY&{-nJ=aM}9Z11S_PJG8E z_T%#Rxs9a)-!?%S;mIGD^xXc;b@eTc_pqQW7;N1gw!0qGDE7xMJ3PF#bLB7o`ZFY8 zH#K)(xORWpw=v4JFa-sLY37N6pWVT%dtW0XRz{oWd{$u7$m4y4cP+#v7OR1z&WBCo zBc#pWIA`VWb!QJxN9e`5*T*!M5qU~(w|8~w3a|Z%Te%26_z8>peTJ=_?cdNk&G$UR zUM}F!clNChT@WE;Bb@vmW9C(Z^SWrOd@!7;-FQB=(o9KwV_YsqU*BcbTaZb&#O>F% z%uws3rHNx(d(Vuw0Y2R{Wg+Y8pEyTtovm5B*UuRJHfBqw$sx%BhpT$GPW2sVW`Y}R*JU^moqg`? z+Kwxy*S>pB$?*^=euas zR%2O0ob<(e`)WFIwUY5a?D6hU>e3059K;;_r^}eJa51z}&V?z6nu9B*EzADcv=#@j z&yzPbZ)NH%<)P*}?&%57K)V;9mFbC#&lv#-7 zL1!`C2ho`I!)FQ7OSc~KbsCK}$5wL$O2ldQtNhxqEI+fdIXBV3tRQMhyx@K>xS{wI zp;41xV845XZm7Y6MW-?r^kTakl?n%h{BIqcXQwk^%u>3o13l0Z1Ot7YO}>77Pi(h? z_{(7WFNa2|i8W8jSZMr=h~&T*@qsW(4PfXJT)nXeqYf*6Y#4 z6t$ARkQ-m^_7zS@|41x2^j4Dw@;RJ+Hm;eNetK-B7GbAi4eXbHhVh2Y!5} zT>fXdIK|PN0vpmSM`qBH?WJ|#@n2%*1((B9#srT;?m8~)>OO%|qJdRQQ#p^ev)u$A znL)A36AjG^w%t79o7W#Y_f);eEaH2;I$F_9PLbIuY~1`(!u@T@&c;Vslq_IRaZl0) zdP%#&)nbT6z3=sdUlLwpM@L7~Q*(`qfu?1zBAW1tz7i>4KV%Q5fh(QbXJq&gdqSbo z4Jbwn5zN*Y`j$38qZwsR=a#-1^M|kDbbFk?u_n{P zfCV$hd38tFRl~1fS|@^=x7JJ8@LPw5h2!TwiY;M@H7A#n`H{y;66=?_scqHW7H2jZ zrE>L0!R4oS+ykc{>N~Cs53NWxl;#gm*#gnztf&lQQ>zu*zHxhc3lXi1QQ7T|97BzHJ0R3wh{X&JE=x<-zMLjuEYR=k zPWpgO_+%-S*ma%KQoX%4?r?`CW{`Zx2GsEbU`@ohZ`ytpyeGoZ*2#Y=z> zY7E)-J(^un+SAzGKg&3Ht;SmV^0KdeK?DJV|^>R(bexrPx%&6Xwl_nmOP8Uxx={dz1()Oo>Sv+@&xgml!*7x zfRLR0J>ZiEz7)qPkdYXcf>UARxv{sj-%k!!VyfGj1i0zb;V@8aTx%(@a<#P>B>%j5 zPHATwyE}47u;sMLR%|;^j%TJGz{%fobP#484qP#Y(uEmFp^?W-W`LIp23r*f((`^x z&qYa*PoR3q_2(wDCoHt>$6u?SuK?a{l)}bxZksH#-!5(>e8+D1e`Ywq!0d#JS5dOz z3<6ZUBuKal+rmBd(*AUJ?r{I;`qXCjw>ey?0grIGL8q;#o%>Bmvs4b0-=3Q!a+tRn6l+^qAXFP#eU<#hQ4#RdMj(y_y^1G&4VB<6Or?DF>F|P)LI?B|JeT zlGlBWG7ybq0;R`}VH6_@4A}IveNU1_NXOb(${l}5%>BM}+oaP50B%5VL_7(hr};Mr z`v;t?c9&MQImq>Tw&$Onspv2G3Y@?rt2@YM{qsa}0A<-kQ%_+)SP7@taRD(FS-_oGq3PmFO8!!g&+07S~j3`#?HA& z;5s=|5YZy^$egxAS6r+Yk5d&Z#mkEN0^o2QJRg&OUSHR>W2~)E>DGnv?I-z$^=+PK zTz70$r>wWu<}#=~)H~)*9)#=(_i(WP$$mQL}5%ICA z3uEKESM@X?I%PK!+@rYtg_be!zPX^AA&2c27;!AvD+W#ZgU`|G4DZ5L&G`xvBL*N{ z!RPenrz}3>gGn8k!;_?IA;ns8xCYQgsAV_rJt6f|vTfjfL~1k(S`5Y>d%ZjI$UaIV z`blNK(R0Z}2%uCy13S6RJFoWKlhQ~R(X9~*3q zNVaG39hYmHS$DY*t%~)3#<_IZ@5&-L8+*VB zG*ymdb}7{XwVy~h-6KuOgMSae)$Ccl?LxC|$?>x|7#oWJqzN?Aj}V+!yApq&x|m=7 zW*^`Q0zlZ3giWNtiKYfK?Y_P+ZM25C1zJ`Cm9Eod_;F(YG7380LXbV~E-?j{FoIT8GR{REvW zKX8Qq{Z-J^HNo3*pJGrX=o-SvqJrmz3cfd-0$YFyV+i*r_V=I_l9AnwI56#EE?_Tv z>@0SLG>z-7*q%b57sc}URMB28;K+mj_vcmF-8x%CgiWbY zX(8diF(tJRD#|%t&yPsVzKZVA6WkjP-uLT;p(*ynjmk&(L@79kckkY5MLriMGCqq) zM+UR0y%V^(3EN#Xr3$>^f8XdMvpQ1kUBhA20kQ~o-?O%lay=i4;$OX%=4!0bf#yv| z0n;Sxwk3|F+h1cHLjuJSvh0#VSqf4@b!&lZM^g3R@YE?q5Yc%hVs;LPufPXoXQ&!GlSh=E!B4T|^PPDH0H1f%V@mU&@Iec8_V z#S;4-44>qr1FatM7So{A=5L|PXf~vjf^AJk>Tud`c-WLm31;ybUlxe@_Be!52xBoU z;ytrXsd4vb&;|eVrC-rt_X+#y4YX8)E=FKqP@E?h!z*e6vYua+Rm-QkiZ#YFpHqBX zLoQrA34+6mAT_ohk3tKlCWySaKOCsWsM$Vgar@S_t(_;}LZK8VoEu(;v$a7`+FaPP zvI>ohDC~Ju+7E*zpddm#7kr+;8wtrasJ?GjD;(^w!T-X;+f999#*M>AR*Wji7)?`V zZ$A;88RMP8y7t21PF4s!AKUNRfr`SRt&$9J9b&qm(^x7X0F$Ph76=IttlT_p?n-m7l_xv^T!ZAcsIc%E8x;FwachAaIYfAMMZIY5%a-n#bi;duR)4E zc8LfyR_#BO@B(rT@Wz%+@y;^vz{>rZc$8GB@;O^tN2eZBuVnpBjX94fCP7nkw5QdG zcj5Rmj7Go-v6BB+HzruB!dOF2d9LVp^Pe}42u3C39p3kX^WdpVnjk423orsirjQ*d zM%lC%baG9`lF}*bAp*7si+@~QnQsEydGkDfK!W7qabb{JXt03>HI~J?vL_XLs)Pwk zPBQ#?$Rg_h;{ozsKf#{zf(Nc+@pYB~;Vbrh3Zqkwftj?r$dk#{f|#A&{jVG5P3`G; z0=vmkp;22fiENr+3IdPlWmtL#@ilp>@f1C3EEqKqofG!U zU6UU>(>f^+mp1{-C>?Cy}m49X5T&pg;ACN!iSYr=sq2Y(Ki z$1{myo&N;xur|9@mw9^HLKQ-vaq+H3?LKKQIEmGP(5GO5HDCHebQ5t5;YRo}0B(sF zmg6PP<)P&J)78-93nMrK89&ovYcZ17)+FI@?i&&lE`iaFS+PDH#)SkH7#lGolP{85 zDu+Z$YdLO0ZR6DgP-7tf^Rr)dQh{M(CtTD&&%f_TR4@T$_r=ASY}1*rYyl@K+k7}QU=jnm+J#P;K<>(EXA3?h+wmzk$p zDr{|lydUIq+SKb6tVxtucD8k~e=4PGlSEo%Q036J(lJCdk?|j7aJ18LW{fzn(+A}T ziE{*BAC#3^wT~u6gz<(*Fw|5*{rc~_XZqx8pv`3&(_NkaH9$+G_q1Ndhe*kH-9mc2 zz1Sv35C)`;W{8}0A1|>$PlLM_eW1_-MSW=n%?X4Ekt|_X4|4!?*{bb`iOMEleTK#l zl?%CutFV*IkQPgI>oU3v`aI*lhm7>If!!kF#Xj^#NEXnZ*ViaGsKMyIg{1mMztGip z!Du$9o6xAc+WQ%`l}Jp~ihonx-b5paO)Cj>!*M9h@E+amRV>tmmhwsQ8TZ<+^|0VQP5^`>jZ3A{eV* zCCmS;lK>yNG803};?P~PVvu~D60`KZoJBiSjnVDK{SdS3TF2btP#D~9-EGat6o9Om z9WVYJOd>NSf>Nz^rofrVah4~N^QMn4Uuou|{QO6yhnz%OP}e-;t-1PvY(vP7I8H=F zkD-+EbvmvBPm@mFL>*C|JgSGFDuiz}v!tr&6xXd-jjMFi&(-8B`hAmFO@Ku~6qEBh zWB5zQ0s!7-p6(e&K78(=GDNu6*20Yj{OhFQV zv1u|=kC)!DV*6zJo<9((r1E}a%dMUsWph|++Ecwu_5peEdV;VDq(C*xw7u3cO2uaO zmhxnp!|iH>-{^f~ZRFpJP+-?_wQTBfC+U~?g}p_dL}-!n((b(wA${cFWA6+ z=aq8)c7m`qW0}L53rvAX2Cg)vZw%vOrnKFpz-J313kg`HQJfvkdoSyv@StX-GnO-i zL>jeL_ZOodS+44VOLz_TZhvDMsAmmMG(~6OO|sPMIkNI(=Puuutr_AwhNyCN8UD(; zW(jI=>XM|cxUiYrPJhr*xnP5&;AFbcrkiMZ*iiovVdSf)& zm=!rolW)X5(Uq1DNA2%uL54p60&qwLUtf!U_5_VN>QZ;X^;19~QLUj^ZErX7Rwbyp zD25|8#Fl0+3%S%M^fu=Tt>2jKunU{_P{OBYQvt#t)|Y#yqKF2qdnqo&W75E6g=Dl~-?=0V2D{zb2<-%_aK)9H$lmU#35SzY; zV&eAS*J%*kWWoEcGn*hSXu`eG@Qj$p2N%bRNH>eTQ;gG+rDM}?cA;0+hq3W7QbT*S zeq}q|vRUKB!am`+oQ(I7=RSj~5h-l4eYxD#DBvjEW57o;2HTakKa>f!IWNHCbE14Q zGl)~wqVM&JkjLig?YO)$i(8sT6kG!Xx(eiSZ@M2avR&Rc|MP5g?65T;q#*-F92e1I zs~iHa9}dpjkOd*3+=|Kd1R5OFhA7iBpkuFO*l)}s`>scmN_osjB^i#z5+)hQFyO!Z zmok7B=>%tDcX={rli{zjwYZMGt?zC*bH{{AG5Unn=8}{FW+##KEt|%h4-ECWdnP|6 zF7aEONEE<0)pRXqu*@Mc-Yoz?)J3Lip4J0B0vh1ahbi73qu*$Gr6TNba@t^U^1CJP8Vi<dyG5a7h zT-sW%*xtg%Yfph|S;dpcY0;R>xqlBPA9l36-`A$|{URIFq*!BOYARcfo>^GtujKHh zbgfqp#7AE}Xq@#o$kKJrx)j*+=QPd{i#u_xl5&xZMG(u%c`ffYyuV0Ytv+)K+4 zrMv|TU^h7L182a$`C6>#)<=`%V7okayVjquDoX*x$g>U2EUr-1oF-qUQ^@&%5L{JV zQ?-D^2Uo8#HO1(%w>nPbQXS{okS(C5=IVJEXM|n|O^x39=>-P8J-GaPZH|jR9oIW0 z^eXJs&7@6&J?n9{#!JH(`ZOj~GVV~C#4gfkW_#`!yOOyw$BDO9ZCG~d5|((_*}W!_ z@Sz^x(w;I_JSDe~S#2wiugtcVO5#4ruLLvLbMFA76J9}yrujBLA=nb*-Ausm6cvPl z9Ih^xWmJo@_rgfEkNjfX`jlyDTz*z-t-02|oP?9N97(lI+@#|g%W6-yH{Qzyv>U7k zu%aT(1M@ysgeBgwf!}t{W_+SrFDh@4(pju($n$;XhEXhWMVw>*?b=tTCC+MYr>5r` zbuYUsPZcMwsC7(ki{8C(Z#YcN07Q#BTyuv|AFv%mO1-VuZ-2!FdUi&3vy&v}MeFn0VW5on_|IICSrB_c{1VDGDbop3mFoJ6ewQ&Wssb z$uUrm|2=WJ#BlfD@`paqIBO0)8G}P-_X9Iv#NWc$@xpEa2UAnn{vhcJ{$YbZ_OE*i z@M%1d4i_L7UTP2GRTXWnzKH=aL@C}&2>#1BfH82~^4~X+Ev&G?BWnPJ*~{^yup#pP zh;u#8N|lQ`0m`gp<3qzZbH1`AS65!RW@`2P9VX*vdb83FJqkPy%Gc!LJ}kzt9-QKj z#QAqZFYyGC$V^BQ5EHRx!R1NhWPnL_>(srVi7YL5VSp!WV>d+Z>THlB@bsNMs|L|w zxGHwuA-P1;Y;VeFE*vM5En!AV%`MCpX#XY6GhB*~ydF41HNF*3avc>cahgpIShR&; zA%;1kmw0&lVhRdHAjkH2H515_6A}IO?=wd~l3<6(8fYNOxIJ6rKP z_{(@*@cLMtN_RtE+ADZ!Gxwx;>%4p0yjYoj4-H#uqNt3^PBM0|oq9deZD`E<2r9XN zS_NeCaa{LdW%5iEyrMaLeZ83J961Z!pMwd6BE97H@LTgEE;z zMuQYf^+cbz7BjQmirAX{n>qcY<69y2$dwSZ zk6@}zx3!r$eJ&b<>vZs+YKSxze>ahI{XNXwr5Zwv@Fk`OdDqZ_&%ai{#bqMNXOkJd z!YY+BFZah@&JGfTY=S{|`hv}p_KXJ#*7=&|Tb5qHOts5OEv-y_VsA6t^AIP9KuxM` zh9}_q(#{E~BUui4!ZAz@%X*gOEA+tL)x;J5=(#q z2cF$4?FRa_&_>1KuoYU{p{-fS79)p)?I6zvqJ)nau=m$sa;TIt!yg~+H` zr1IOooFO{c0RiiufJ$_AB?Lof#IYDwt|z3Xrm_^49AqKUvrw0-9o5VfbHo=#%G7BiLvRm2~Gq1iG`&` zr>q2HbG{#}rsyrt6a+D^G;cCcl!!p<*%9Pi7bnVw~2KtWD4Tf}Yd-8>jW{ zo}p~f9ZlH@m)U6{!X=LjpvhkWKI3n{aggDB86h1Ic3D+X-F-ie9i|3HCz+KKfi#;g zeu~5t@`A}p;4~ZvBStC#evA{pi<3~j`ZeE(>ij5Gu`N|ao8zF- zx2cPuJQ>07lG^@VkKu<~z`06uw6gbTC zBXmX4yCQCBvT8;`#=KlLR&s`gZ*4wzD7(aheTo~^6=c%+Cihgg#P(M6dyz-$Qw%HB z)$T%p{9i0cJL48&WWAYrBo~}Yx7IoMwh~W^mKGM-j5I{etrIqrg=nc>>}l0bCu5n( zSJ83(k~O&H`0kR>f_%aw*k3PBc#mL*go%X&mh$-ortH}GJ3wI=Yz zmkW}sruSq*^K!r8CknE*>n^@;ol72C2_*E)*0@)jsJ^|hRDQPByu#%|Dic#$()HZM z+@UXoN5Cp`v7q0}QyNi&>=ae0xUPtf@!eq!_JZr8Y`u+93LE6IrFFap- zx5e!eNam_NX*L|i><7Y}HZx-g2S{F*mzDkyufzBv;5G#$W6%rc9WL&;Ke?A(e8UGd zdQSK*)NV0Ca8Nbd5&rs0cVj{rA1VOXb9;+8N&)vow5YV-mhR%Tkj%p}P^;5TSi01G z;~8W6LhZEPk1NJiU+o760fT$8wyy#5bPuTXhn((rFVTI$Ber(8BnKe+Y>@mnLDdS~ zK>Kn`G{7WsgHBE~6j`84`-%GRg6)hmkt_t9nT7O*qlXYpL0KZOBsrC4&62uzj2-5( zR(wNiAKkv=ww`N`TYK+3&v{6&HV0@VU#cyiCr%FR6N#=U+l;;D7@)DUpdL|c102YK8tIlUZ?<1v`4g;p>siipfSCA*u2pbVkv>?-ouB>`@&0Y6eq8h@`pq7^-QcmK z(0N^+W^y&N$&5nF?c3@X%JbD}F;i#FEOV98CFc`PoEnAiBV zvvzorZ>N#__cpUj0Y#%`Te+-hd?|c*c|@{u7r01eK{;UeGhj89{M9?(rh@aBht!2N z!pj41_zpnd0NTu!L;o0y_WBNeQP~5r((l4R^YM50$IYkLAp9Tf6m=8jj^Zu&lTo8a z`!Nw&lYCmChO!$J`AGfR(%R9N!-?Qze7x?fMxkpxBQ=mbvd!T=r z()JB1>zZqP%a7Tzw?6V44wZeWaS^2^8lagAt$I9nNpOJcd!O~r>i9XKFCwB-6#Dw+ zWk1#;7Yskul>hieg)&&SJ=Ie*_IBCcXeob~;IqiowFg7^FE4A~op|C1Os6&vDn2e( zJM4DxpPxwt-!5p=&Tsr!@vEp+(U9bu%-ve~G4V6h;Iqn$T1Rkz>-K42X%S)bO zX+f8l+xRUxb+SG#pEg{K!mmz{`sW(sNJ+I=xdc?P=e1@W1w4uOd>=lH?zqlXE62^{$o` zQJ;;UT?byqyRvyVF{jD=xrnwvHcmn6l|;)g&N5S!YMElmf7Yfg)J?98`Fmc@zEPMQ zf0|A6_g94o9yjgzcqNI8+iSJn*3Bm)Y8eU#E>Dhrtr+~~n%odQt3UL;zR555bXWF= zp>Od41Csq7Tfe#ellVYD5y>|c>a;zFJ2)a4M`^U}R&v?ya}pu`1tC2r17vlUM!ai| z+FOi853BA_8On>?>UuF7u(56Z0GNF_>yB?jgY%@PxHo%?mU40jw_`HBM`_A`6Y09!=B1&hReD|%CTl-gO>JMg%CZtu1!>=?N1%4 zS+mTumzztozu$JP5quFxqOWP>1Vm)r2a4tjfgtp z);g`&>B|{A8W-=j^ge4|bC8K8f1%^1)K}DJ}6ThBE&#DT; zlcj$84>R6|m2v%xf4YylP{&1D$ErOAZ5<*rUepx5n5Eo_T-*RW+Nc+)Db#b% zMsCwoOpaW(&hjUaq^I>&*Z&a*{TAN@WJvNmgFq4h?n5%y@s(h*ewN zq+R=LS&g6gJ@7p<&6&Xdxj-84o8_BRPSb12+Dmia&3I~(NIl43Se8Db%3qVWM)k*3 z(+2eOXNnJLST8Uo*MxC2a{(UDBR>yrT9?#%yIP!;d9?0d)tm(RffiET*NP-mSDb|{YetVZL6~soB9`2I4Aq_ z>c5TPZZfYgPeRVmSA}ammd=j z96FoXa`orX=>Zs3dS3sPyV4YsbxS?L zvLb-O{arkhiOSlzVPU{1O;MqYt(<5KvN|vC+SNc28-7GAUg_HWg7uDqhDMR|p19+6 zCHoJFzIWc|mJj{~z9$_d8BqaJ+SG2U_eMkYx9v@-Ji7kM-suvmkzm9^IiZbaqYNIKWXzpb~bxm>if^84-3-sx%)BIQGeP(e5L#fuh>VuJe!6NMJiY+<%oOEwyfL{!R|r$;7ZncZ@mPxFwTY{GJC{7WqoHY_JH=67<1^4F z@Z3`HBDi0?{8^2zlFa4 z>A8L?j{d%^=lLt@AK9BFsIYzYQB~-c-PvG1*sM*gZ6bOS)<~hwg2JN%lf$vTh z`WH#?duBg8!{IsVuQ61mKXof_hjwSmF*roaM{w1$1_VOuvzn6;;DFRSdjtpUBt5Ue z)U1!R2mt2njaA7rBz)x(o34B^JHoyN8P{t7=7o1nqXx9zw_#_GoX2grq~zw%$~3re zgPU~6a{P?pToyMiA-CX1b~&C#+drU7!4^KW#k9?=`~2MG(Lpg#bkesiu)>(R9H z15(Jy$%>NpIs=gW_B7jiQ*pqXK}c6ZQPk>vRiQ_OC>uJn04b)%_R25U+3ItW1Kj*q zhM%#}jLB+o;I6<-N^=7 zp_x8hr_R!zpK`EDFKd+Gb8FKz$funl8|boWmK`7{t1zZ0OiRq z=4`o4-vY7`{o`AdL|+ptN$S(jPEOn1eP}F6wN-z1bLkDZCY#l5SYPP=sfFFdRsyKytnZ>SfR|%(;>5v%7rP-drVRuh59*H&OZ!7H& zujSG!pF^w+M3HOQ@te@Q`_4v7(wr&T8VR%>4i6;Lt0CJ zQ0ietwTnL*nl|)rbmW-gTx{dF@UwUL!Bbw|?UKE~H(LAH*PTyQ z)u?+8DHA$AJd#&`r#m$z^kEuEBv(>IXbYoekS8jE zjX}?4g=fviOP6MA-*D3d4P7CRyn(DCFI!9RC)~vxf&)9N zXV=S!v&TZ}ud@+0;5Mz7xn^W6`{dqhd`(&g}=fi7>_a^t8+Ma`&8AVh8 z6VMiLDD%OyOG4~!gzI7pJa@(tbM;a2YOl+8mYbqeS>OrM*b-3^;+dD{J~VV8sr?Hs zTU}k8?%~NTH~;m%S-hjbczKDy%;NG!FFwMD@GM9Vvdo&4S(W@gIo?dkl@$jH2|&+o z^PRYoY5Srtt(YosNWq@q;X(D=Wt?EgCW02!;ty-mf$jEyJoBuI>-7tvrv&d&}CB+xMR4w_K?eIFpTbV45t8Gd;s^xrz0vHxQElDWNDIWbk24CPY zoNXX4uKbx8SAW0zX1u{7o2T!kf*<1K=Qcai{K&v7rcsthe| ztQ{|*)iMt8XgV?Q=>C*Fs^i(yylKLcS5#*pJiZDWkCyCbfN*neC_bKO+(<=U=bY>I z@DJl=>tON_3K6#XdXTfzKuUvs4ow*UL${DyI^j7b+YZ*P!v#=6L1bgA{5PY%Gxvs3 zKZJ{r)zTK6MiqU*o@eK^{nDUE$TiIycF9See!9mpAf5eiO(9WAiWfKw3bCmSpG;p$ z%`!L=?9*Rj;svKlKr;t}lKyBHG3IMt@MfGv^ajy6n1&KnXNIm6ioRu0VYr9dT8qkN zPMmL~eG){!^%>GIR*K*Gu0HwzAr-+YqTKC& zH`LujGq2pd26)>@< zG+nWEbkHuG)Y#q*BovK6&Y_y$)|AgLg=|F6+WOW&uAKPxtdFeZX`jAG2Tu0e$)&$= zj@`#|ySO#vhpMAUvd)rpygC9mNq(ET=_VT|wp0CNJgG0c?zna#hZ-PW_2M|~62a$& zDE^^O+PORDJ-+>Zu~hf}QFi5lP_EzqnoE&3gchPC$yQRxHZ7K9ON-r*B7|gT7*k0S zLS+qw>{NDxk+NjZzK>n@?8Y$jd)~2kzjyiFf39l0^StLd`}v%6LN{7fUW!prtR?v$ z3IIQ0-bdY)(or_T%ulYI(yuef_Zp4$v4~YX(h2thke$+eC_G(()zjyC)ZsE+A~NGg zAg46$K22hX=Do$|V9zR-zLpDkNjj&13w9ar=IH$16U9#t3>b=BHjWoNhwcABQR+sY*vc89Wiw zOZ0W|;b*EoPFOk$eAy{EG1|)6AiU>U9w&$3>sZl6I}ENGY1<8%GCJRmqFO~$BefF% zVt-w4QI^f{jatsHutWTw#wqzPMJwW^+E1tEs_Sb$A*%T5Y!5g@T)v&j%~ewa3r8+l zrhFsyT;Edya%^uz=K|EU`BKFhT^5KKU)zIuzezyOpPZ+!#U(^&y_*m@wPOBd)wq!MGDA4y1JZeuxrdbc=A9*GJFFM^cy z;#^2%9U-TUoo@%evdXmRu{6^Yn@q$!-8~_+r1PT8C#qM=+jsu7@t(M!Z8X4gUh`O( z!`H+@;-7=Tv!-(-zEkD4L+Pr22I#PJKDsY4GC!2)t)_uWSZLg)B^xf;(4lz3lwXI> z2_IC_n;{h*;GiYXOBm1$xzh-S1F&m#`Rlftbs^N=hKs%f4z6eR;K&kITC;OW)mQrG z)uc4Xoq?udZxn%k{e?g)OOC&NE2CQ2jEbolzuV23PtTS|G8ZS5N?4YpGZ)?D^6@-_ z%GY7ZE0VWdC@yERJt%={AGholn0pS=FWi87EU67AEt_z}eL$7eCQDI~(|nPz51S$o zpsXdBezx@Eqd?4+*_dd7tg}%3G;hl}o(W zHMlx$xA*5Rv>gtar}Z_TRFzhkNt8mi?V3zU5O_9HO)%3(&+i%}@sATqUQf18D>IJ? zH%Ls^bOg=vfUwrkLDtQu!{NH&Y?p@ApJ#Z$WQe^J`Li;a`#K%RqUFMI+s%nrp&Z3R zn+BF2IR}7V^@ZjU6H4Ws(#Lka@yoElOto7FJAiz}bIa!9a$t#FdKc;@2_=;Qp-fPU z<2^`nD|I?dr!Vwc5{f70@%p(EnN+?Y{jGf&mt0*9S3 zsz~&>7jPg$33Bxo`Vle3-2qbj+YY;!WDd+0`tLF944}WZI1`6uWfFXRO^$sKc1AKt zSN(}2ROnl>l=9evLkXYh;Z(c#97>V`oj?IAqUyBAOBL~uwEUSIrGR(iM%6xeSn~6) zB$BLo4zgQ|R4sX@B+I(6x+llV<$5fst)-*{O5?cXea-9zIh$pRJxK^C-fN`dOCHd< z_k&3-`mLl0-XdrhCiSX@q3jNtK9w$z0v#2KpK%)Rwd-JMdMhTW)PcIpl0W%*vYb!N z12>U+CDg9Z+_K2Sy%^M~R2D`N&SlputKNx~Jn;py3R19sE%VdL?TDHAN$X1o1@deM z<)k8Ot0O^PKSeAPGl+t%=SH*l?g!vj{+_)nKo>Xxh z912-Z)tqNLbt)n2{oOiUm;A3p?D?6tPdE+IGYg-ED>EUHD0j1)^z16yUH(ICj46p< zgST4%SGX~6(u&vW=nk40zk(MB!e?qrqj!lx`7ux%KA;w6ZLe=>qWNU8Sx1;L&tWnp zX|HiWTK@9DiB=ha3TlC+NY3AZ@-&k_$Ao1xt(RM;uhU|lM2)05l|pHN?J1zt3uWUl zG$3({5L;>_y@_l;lYo2ER5=LR$g}Wi9V2YshZ!yM>m80>o%Z6U=pQ1G8jzxdvz*WP zi47JN`L-9qOqb`f60>MAS`19Wa-Ppk=W*@ecz>LabZf&%ynhK4RGc{G5h`grl0|H% zne5BEeqLeW-eK;srAd2VTqGMs$&AExypkM%4B|3nO6Y9Cn zyxK(pY^MS3voPQ!g8Ze%yAMTrQbat>r>{SZ$YDo<2LO=fu%BA)?eP@B)F@U8<;@rK z2o9=_8p#_PxPj)%njeqGI(N($#T@2GUdL-}H)C^O>PoOIuFmk%5)ezRD;hGj5Ib+n zhPqOL={lNs)~1&})(P$Da`t>x=xobXk0aPleXm0PBSYuM4y)eJe5v{<%%ZzUFr+e6 z^bD{;AKZqpt+owvG(0Kdgv(Jne}YPp{jD`ec3&!4R52R}d+hFT&PCc|S`vu=X)+_GFaW|9M|)!Z;La8=3k2Ck#~ z`Er;zDCA(;?tY)|K-HbKoWKbPWan+^(Tf$OU|6Y~a{?F&H^mjOfPkFehx5?}LJsus z1-*e6J)XHI&yp19@X}Bc!_W@fw?lC0nmbz8s&$^iE(eq%8f;A$7iP_Ii zIC#cvD1}*rHwUD~Z8Ejy=}O(;t=}Wxz#QhfF};}>(4bHR74>L21`07doN`WFbKvB}3G zgqN(nev{joY$dY@Iakjs@M0FSoDBvl4)9@Ylw9F%ROGZ%ZjVV&WJPMhMc@2|BdMR| z^i8lmcEVmWcj=%QpxL>Q)3^9UZLSJ(=t+1HC|qcW%x{bvL3c8pl3Y`$^e|_hA90c^ zaXs3cHfyEbtWQ;MamFY9aMMM^f~vm$PofcS-*z2qwHM;%syj98ruU1?8wn0heV-W$ z>fZrc%GA3+-QK31J<_(AtB!-y2*AJFj*g95))mDRBb20$CrkFn20rZ1o?OrombKtj zQMM`#G=F!3?+#xmKp8yVfvly)2H0I*&P?^5h;_8>D1037eAZ4v#AVJM1yvvrS1~W# zNObpZpwl7<374>}+!yZD{V3VTWX%RDS8i|r@Lbkdo<=Nb-pmSf_&ILBh$F?9sc$<1 zI*_LM((WOh(`Ro(At2xG(7lAvr2vt|V6NN1pXQhUfhvfA>QB!bh1GcLW;zzWjJp?y zaS=s#=k?_`_qa~F33x8F_gpRRy6e8LxEYkaLP9c~6vcLR%#?VHl`*bqzcTg++6`-U z7Q)FNKm7O#Xh%U%_^!FN)W?8(LKMlnu?)D(LO;{jVF2;5?h{U6D(StkuY8QKDLevA z)#*4}q!fuF%m(zLLrC-}Nqnh@e*HGfnt3b>RKE?Ks^jZ{@_KSi_>w4`to-5Zr6-Sj zXFh3_Xl$RTF~;`ae?U0PT+A@v9^7(Q^U1l1i{aIvTn~VEOQgl|kQ6(f(pTzp5K820 zX2)Jq&f=piVlIx!c6)aB60XRZlqIMO!6D`e<7rJ0%42vYABs7UM&%L)Lm$!&I;oh4 z((<+e0Fjd3rxaf1k^7{&WW+`3KDzS%| zP>Gk;CcLv{GKCx6JIfq4lyxo0r0(|I)nc{uw8bUdory@TS0bD>fiR2QrEsM` zl`*ywz-u5Ebh6!jUPNZ9bCVGj`t^&Jb4SFAp&AebtGT$0tCEx?K9~uc9AV7|7}QSV z!H#<`2Ly)#oqL4EoVmPP;<=N>hd%nDw%HhnfRrAngQR-$(1Lk>+l zn}51Bmg9K3dWp6e#MSC@=MLnzlwK@s0m)*q+TD@S$HIhs=7E>WeJn``k%@FRYBVB& zeC>UWzR*9=GJE%e=|!6&6KLpW{w{hW!$6=?K)k&M5r zpcoLIgF(hjMQ;BU4-SbdOZ#Vn@F&Ia8hKI=%G2G7MpKo}kizpD+W9$xgmxTK=mLPL z7nl^R(BZu0V!7}8QzWF)KVFr~QNp-~+7`UA{I!ViW|(h{{pB?KIFop`wx?V*a6G8N zf_p5u4GK#J>iiHfd5agJbd_7Ua)8o;h6ZWpwW(u5$6gwsU){lj1PcEs>_L}7+c|whO$*mjZ9H1#s`*;KrLJ05Z12%8Gf* zWmsPz`SIS+XlR4Cmn{# zs}~A8d6H5!emF8x9#rgtN-<~lwC2CuKQ+SIcw&^^FS%~~h(rtDoqRXU+*q&GO!%3n zX1UIpc#_R5m1VnaLJ5}Sa5IzefCE&_)gQWjkn=2JCX_uqeS6fRw@a#1VuTvL%X%#r zFl*Tt(IVSxGMg{cLxG-Z0j5gl%AyuQ#&(Y(_xxEP&>5p|)$I4MoUn}x)plrX=V!VlgL9-a8 zwSp`BHRa+hW0tR*d7J$b{8}Wnf^%0@)&!ecF@qhBzFLY1&&AiGdVphDv@EQnvv6)I zjpu?qy{PkOqvdio0|XhkEPr&Ug3I|a9{B(D8mWH6QuZ-&wWToH*Vb4W;JYYyf%43C zb-f&rUM>MudesHPDUcjZP-)CNEQU1&R+DI-w!KHhfahB2<8a9-OV>KCs_T<_~NlJ#!*GNjSm8)8)dgCOc`E0I5I^d^n31{alRD%LZ zQzp_<@3;4MylD%5pT1r*a;rvNJIY9z!~0T_fM8u5yZET|L!^&K2bVUvumIJ$Pd^y1 zw$8@R5J-o)wCRk0EJ9ca92&!Q+*+2GBPP- zR23SOo%$W*#G8BbZ9qVBS@T3bvGnonY6cPuD7YYA>?dF#6=iMu&R$s@2dbPR#^H)R z@3@;7$BuI*jUp@Xr{|I3eB@{?nud4ob^qdi^<@SW`Kn3MV~CEbNIQ%($CzP#%l&EzH?uctdiQ;i`s^SK*44a1uW z^G%mqEAN1qxu$yoC1T=pVlJtdwKLWabT5E#T$0&a+3Bp9aa4lXzO!8%%!24ga4d0| zHl=}h7o;eQL5Z3v6S3@WKN;ulB#xBdmp56+aE5RNHUCoM-OE4FPxpbLPYQ3(pH-bI zg(et+e$ZV*>_3BiXmY*9%r9!cDDNFKZlrx5B~rVj{?IeuLt-+ChSrRP%=uhMEd40B*yaq;|ozfJ9k4 z*pE`1wVG?>X}QAj_}KHN>Ptd(x{US~ts^(tq{enQb_>iu2LL%v2od;_(q-8J>viCa zR#^6R10-RosHlt4IC}wH+k;qh;y8$FnD^6PS6mEeu=`L-1Fvv7a=B1_Eb6)fx!n#mMA|0u>3<<7H`qqXIE(3@CAP-0s0N5E_31rSNCj13D33ZBNn& z<-ivcqb`3+3#obb3G4NkWN!xv{>6(%NfR~xeu}~XprdXu-SJ$q_TVbQer{tAGP@ILxVNL6-E`3)L?tzXP|4Vu%X z<(WV~$szS&W8%QZxDy9-+E42SMcZd%C2#2<{iXlIq4i2%IZrYZNMpw@QEI*baIY3I zuNk)HNpVf{*?blkDz! zAUq8h#J=pH4{Y;@{^EdKL=%;6*rIxaKMw*q{H`Jkft-3L0Obb)O zYF;23Jyi=EOZz!f0EF;qq<5fAp6$+Hqx>B{otE=#&cAdGAA>YD*(Re&G=Lx^62D2; zC`ja$KzT?=%AoQgYt=ox1&KXFbo5m`SGEL~*;>>p-XF4$*xv5T>yFv+PSUc^DNUZdr z#xq}l*Rm(0~pa%|pKrLMZkfvP_bQY>#H))6Bb3h{aTMt47ifU`ihZ7ALbaI)Grp%O z2Tm~@sks=z;q-h^f0N+;SS!0?g_Qm)aDS zC_vNWPbodUT~+Z;*ER3_QkAw)!0g_abOUIS`)G>C1HsOn1TKk?FCzl=xMXe#&*l!0 z0!WgMRw!AsP)=mPy)$P6PeeBA5FM1Aq2HgO(Ku z8J>)<>cjTMGd^{mk-}6jbgsi_d2=ogAP01QP6zd;Yd2>i=EBFqoo4bi!!okzVsi?B zHxejncrwMdaJGG#Z-2-*A)x}lj|Vwe*>se(XjtnF$|}jr;Bm(uV<_BQB|?NIY-3za zEHq6j{^22L0xV@PRqiS~cyZRGpk|qaZKbo+YfD+qN=_tyi{(Sh(?AfC*TXqqT_&(Y z!%)}I>Hab+vHNO?Q_o3a$M_c;eu@l!)aiM(Ec4C6j-<;Xaa)chb9vb9@+%Z+IvxOq z1D89AecAQGm{CzOnRLK&%EcI zgqEYUeV&~WkKk2Q`7>TO1vx^211nf=n*Cthfw?qPw>6qQ_2LoqHpW2HMk7v6^~_zU zF9OIovuih#14wgECjlY&2)LfRX%_(uDhmF>@nHO`tq||ku^O#X>=!KBS z09zS#)F7W&cm}*W)N_MvfB1U8)D*_8Xc&raAOz}BgT`wyA6mO$(JrQT*@0nMgg3T2 z(V|eF(rK4~^xkiI=bncMi1K{^&Jc2;Q*5j?gU{Z0IP$b*h2gdPvhUO2wjOugGx93W zqMtIU8^2{s_as3WE8?Hq(s@#5A4%t@Sr60$KhhJG+HU=xi-dIapZz8 zKpe_OHHGcYR;aTx0G@+xvwl>oSj>#|_&BKN!5OJ`ZY}tUG5|fAx?MBu!OZ54aDcJN z6&{PtZBr=$cz z`U1z99x84C&@6|K1;OLEbl;6?OeaEdC9uy0Hv)ogx4h48Qz2b`n)!S65NM?gQ$&QPbY zvOTqiPxgSG8O19${B^sxbL-zNOdctQy38O{rp>lTAVKK zXwsoaNsKg+o4VYv%{)+-Md1i*lWu+Y0Rv8>wOl}Z-J|A|V04=6E$}05#}>Jp-V=aVBPwnm6T6He z2V*M! zi+`X`hZk;ajan$=-4c>;PMe-1vuiYETK-DWT&zJ0nC6;{R)gTRVHa|Z8SMnkxfg+-H*^c@mo z?Y{XfnK$(r`L|V_vAA#b^V|C?hY`eq_)ilDP(F-JM1A0lp)d!cpM1P9UBOJt``Mm1 zyYS=wj32h2`Y&)5>j!kO4IO$e+-jxGiy6)t8!tZzdcDZ@)YOwYRqS!8ydK=LvVKjJn#xzk>`_#5F1i{(9fC>q+7JF-2K6 z1_*nt#$9V<4Z)Zy=OoV6_DrNjAbkZ~lI~j_1g$!gAjj;Hv`19b1>Y3scFZpvVeiD^ z(ErtK1VAFAo0xtC@Rgn)FRka{`g>!+Bdo{l6b7YahAfg;6D;yg@8>pM`&4&&ut>Rj z!HF#6^$k8H7nnO3A?PjC)WPzgs?zcs;~qxI)Hp>zqW0mD67BFHtY@UjrgnNb#|{WC zPWLrAE>Q95nxjU}{yXxkG(`D=v8rL!>U+G7vu_$%LpiLV*WhJ$Pq=qxtZ!_GX>1pd zw4Y4=!YtTuclG*_m!{|(YJN^%rBmnwX{d||RXmh2 ziDk>iMyXKnLss){0SofoHdSb=n|JR&-h5$k@D3peE4@B|>X=V`W#pM_TduiuZ>&kJ z+~wk(`?jjJWXj-kt^AE`L$2xo5KGP^>-b<#)on4PqY)`_*ree9-uv@#3~= zh~L5y4}*fk?8cO;DmE*6$PI(WcYuyxFwzLx35)yke9Mq7X{sAwtZOZKtSfc>M|e3_ zE}`2v_~05p!ewDPi6h|EAz=FLinyjCD9{mb?$y#6eP3 zyn}ocuuk&O7TT?juHC5okeOh891Dn4qkL6gfq5+vDtVXQIX|uv&b+MdAgdd zyaUY+*|hy|g@hXuH$mqn9dL3P^&0>P!_5Eks8El@g@_vzV3Y#ns;trKA7>q}e60Q{ zI999uZo><%a9yfzTb;(guFQgBuiutEqR)fGqH4^Zsx&9rf>^TBh`_++JDn`rZQINy zdprefTczX7q=*Z>a$=$#mL;r{Rbms?ttn&^LOyLY&AosBVlce$pr~=wwXjbjEBPfb zyc3h*6aZvuY2&amsSQHwg}dyBOqCkv3gcPtYZ zW-~yd9F{A%@gLs}`dEY^G|_4_^7QG|@9L!bpcryX=*c_|g}tex)|$}~9d%ch4f*Rr zd3k*Jp{-Soe`D5CAHZWv;#Ymi%!z(T`!mHI_ITT_ldWINcv9NqlUd9Hw9K;?+6N_@ z;t{{^*<_$-*kXx(-XjJv%3*Bypi_Igsk;s@+t&BI4i z|Hici`du*oxZ-pPi)FVPb1|cuHr;kM!j5gFfWzSF3XEWJ9rZJ za>#nWm~GS&Se&0g2-PH~v5Gv-?UI- zO^ZN-+PR@Y?HhYBSCx#(l>+i)@BgB4`ZR%3Pi9N$+N7dR=)ggNe$!w4vktunNn!NU zwcWfxyHP*ex@KrACo=B6XJQf$Psv%uTjua}t?qXL4Y%aImWHeBzB#Y5`flrrn=F>; z?5DuK-&=L}oI2ub$>g)vDA=`r^ae-;>R3cVRudFxT>foVe0?;S{e5S+KF-%q8rtO( zDeIG5E}{?&imW>~-3Z(Z8q+?x{f;B$@TG`l-a5qlbb*_j<9Fze>Qh{U);+Kl5i6z# zn;5+2EpT45FQiw-gs)H;;1)n+Lam&gaN0yUVkhkD(_N+wR~NUvxkbglJaXL1vNgaJkRI`^#n@Vi}wuJ$)+OoN7ObzTN!nB~)(Bmdg1^B2CZ z)5dF?J7RBAtk>q(p0ZUVke?o*3j6cv?xY({I-_c9Y6+}a2cwvaZ>~%S{r@dI zUVT&0Uk`>u^<4-A!x{1W_A;(%;=dyV987gk2=c2gxZJ=;A-vgjkI;n-`-uOUi8Y6J z?QH;Y{yORrx{#lz3TYUQf9v}4*x=hUI=3e4j_zG2*9l9eDOevN^q^L5mlWgm-@Tc# zV-f$;EmqNhwymq#{*kvuRl#Xs!Pa0upqyQQJDBBg1=>l-9rqT1K-SJFPhayl%V}%M zA=kGS>l70i5OddlL)GBv_wJ$qGv|R$K5OzP9T-58cGF_ITcGCP1m1IHkRsHL_HR^O zVNY_3xjq^4lAX^P%0whn9ajk4Tj_Ch$Oi1uv0?T34!saK`N7R}hnW2D6%zEXqVymk z8|luy50-wc=G+x9AG^MGslx3K^uNq>Ts`%sVR~SO`yIf&UQ07Y0Ay=G^9)*#yg|UN zHgWau|DphOAiF@tt4LGt=bF7n`?X8>=QmkRci~2R5*>j$q>%>#A=49{Am8Z00lgF7j^Lr-lY6p>KW3LGfgNIgy;`|#&yT_=9r<4p z>5~TYtYfy%|GRj{Cc+nmC!i~}1O!6- z_pAI(7-AX0d!$btt9bkeSjGEn>(Jf5rQ{ZZ+dz|(&Dq|1s38iT{R^8ywEnB6h_IwQ zC~O>M8TLtTjsI7k1Ml7OY`xmO$}@j+xgXmAEpikGw{Gl=1%CP83-gA|la+657*Hp} z8izI!0o@2o{?ES;1obJT1b%%zeRl0qooNDp8_N3I1+1K5L({$F0vF=o7`!@QynkQK zRw(m7YBTat;g>*^X#BH80JuO0ufVr1i~FnQpa5&?0tN5Ne@?$ZWpvWo5FFY1{$@I~5G=yj%jlbuZYi$Nq0SEH(}PE0;L zbSUPev$c(qj+d4b%!{AhW$TM>lJt#3ubJd~&^owucOd&^$7*svirNHJnKgwR1G`(T zW!aZ^a@*eFG#27YxSO{dj53iI{X9OF$h#bxF!Q5@rL{4D-n9Ij*@Q)Y)kAZQTcddE zL5G0J!#e*ze^UY^jA}2zGr^cRzL!i&nTc0upKIxgfjqY*KmlU=e+afjh;qN{H!VN* z)S77AL9*pHad_*}m?oj=-7uGHS&v~h)=C)|$tVVzUef=~;1K>U7|Smdo{2=HiCctS znl_z3`WpA8mX(5T6@b?e-AC_I?4ok%E~=g|X)ZhAnh|;il5fz*>nM>rTP$ zh-~be6dQMgLLH`i_}q|ct$YE^u$Xy;^2jj)k4|Qy^5D>NkgnKy)e!5bNeqkYp-nP; zLJ_{Ea@YLg4V_HBsfYncikCP9FTm0T^qvBZrOeJ3rA#{vc7h~I<10NEuVXS~i9(le zjKIurkB639G7`sZO#HnE=ItHBW9E_PO(b-u*Bs25H?Edjwc7tk!%u;V>M>MxS`1w_ zd4%`D8|-}X7P|0o-MUAMHyR^Y3eDl@nJ{HAA^qxM;{svVm*9~#_OPS>WO4FJZ?dlC z0#=P2c+~>zMi*h%NUwIVi}e6d?cQF|9|!P!L6Q! zyr*r4A90ff8%)g=;_?K$1gtxMG#h_$o_@bPKGs;c$f}?PcKJ%x)^3-S1Ix|%D>(FTGbrY%9Eiw(3 zXa$1xa2whLe9dMD>flA7wl0YqLW{Xw;g;y)IAf*#+)S2W^$1ffST?fHG$s_TOZn8t zq?0dZ2JB}Uk<;yA-oE}sbTR$je=$}sp^;Ztr4Oua*|@XA8d~~;07*Gx3g>}J(9W&R zo;NQgkpU1JGaA9b1jLL9t=rkykFMkYCJUl;AWJ*kAO})(C$V3|7*xnxB>U-3WQY}l zl~6MBP^(k_fhz)(!)kiqnvRZToIy4YzK=XnJIKxHV_h$^I-9@dEJf1aep!8d0ko>Y z#$;fv(saTTe%4>6Oi&*x%s{?|4uc%#=pDWX%Cx=iKCK^2-x@1MPypR{+n&aQV;B6$ zWpESLY6Z?)?&E*i*Ap=n>N*n)(lxdIqx&gA z0rm)m<@ru?pUN#Y=YUq#^qn!ew*F18jz?_u*86U-e!0b5$;s6Bwq)5Cp?Uze zx~d|9PcN6BCT-a>b!Hc@&M@2nL}JHIIZXB0&hG(lgb@S?idV;9+8#Whd(85iIY;yX zlNJ6PmMOUWx6Himz6ia9e?9IcRC9*uBpS>>fdZz>zF^{XRh{t1nX=0}o`&c z46}SZMI8dq1j?ZGxayQ#TS%KrXKNz{GAHOx##86lmBPvoo7K!3+X0_de;%-#HLmI{ zkc6XPNuO$=9N6L*6$fqrOm7Dxg>3Vpi^j)xfX3pQyXbCd+e5x0wRh!(_x?dMH6pO1W-1h(ZHH#Q&|==h$cvO%qQ-VhkvE2t)fYGt zAl$ouqp}XjJCb*e)jxN20E7c)18>~IC{MHZL<0QQ#W!GtAN|OR%r7wKTa5JkB~3zh%qFhHsSy46X;**_n@+buzjfXNEK#rV1enoLbPzxJZ%-VD zxa#)JGH|+7NT=bFWx@7Ij%#rCc^GWV92M~Szq|jJTbAZLWdz4-d+MdrMHzqO&un?I6~o-DgMM|ab_$0p_F=nD>)8FI>AqS^%7+=SfkqvpUy3Of7CR^49XZq-^nsvJ~?k-P!ppy_i3D z3^p9-I01c=%I_>wKr)(ei7@*wF4J8a@pBwNUy9j9n_>FclX57{D^J3 z_uf*Fc;JOU;rL5N*;K#RfoCi3a86VPKl37-1Z;t{>t(u4PQAAxt>Mw2e_^Jy*AS zYbu+Q<}%#i9Zm>%%F_kr!$q%v6%c~UrI#PPaUG+iexcSihDBD}^9X+l?r<9HU`R|q zxgp>CW#HVuGs`mU=q7a%v75qQW;o0HJ^M#Kihg?bQ*`JAwG^eO5AoT`==uEKOebYh zvyThXB41&RD(Dt!>P>k{GFQhUek**Ox(=ReXwwarYlBy^qO&g>s}8vu+avB?Fp-~% zsx9>oTYh2n>h{V8dbr*a%w)|kTQh3)0$af-yFyb9>xS`Ct@QxYle910m1iyMUMYY@LO~r*K9XWd=_Kv;ckc zxBUb6T&Fu_eCMQ^yu^wFgx1)B{Ixt`LBtCqO{c>$^CG{KT;?-78z|TVIgO(3;?K;`7pAkY9lorP0RNnkjXg zpKhFFbe>3U{jkE{y^~>p zeqYsfV0NuutsGA`dCdDT!(KY{4$5)D+xZ!p$z+6+>GU|x-f1EuzXIn}?G%DrdM(U% ze{o?@-0Fz(mUVr{IU_!Ar79R-yu!!n9<3U`j5zrH-GqJbY;NMK&WH0Oj`M_PToUhu5oGWsc#FRu~j|H4!dAs{!luqSdy`#ACk**m>fBu*=YF zr3H@g=JgzIKq+Q9xO5`7-z?$n;w3hd@x19|H|Ow&gc#nw@8Hc0VTGG5E2P3UvjFsG z@P(@fc<>yk%b2d_RRBVTa#sLpIU$%dow_FkwaAEy!^Hwd6a zYr5sZSZn&1y$S~pyimQc!eIsVR4(HS?jdZsGnvW?AG4;t+3qJ}9VgY!R~;GwL=X;{ zSjtT?&mWOS)GhTgqn4M44h7>DqJ=|mDOlk!oz4c2zJGq(IcsWR7QDy#UduoT9!m^v z>eL?R7w?}Nv{B>R(9bKPiD7=Qp4>;%?O1Hnv9L!EQs>W zn-U{0U8|7QbMoCS8e;8~9xC(Qj@0EdcWdS*4xg8YAN*b+@FsWAY$D4+?l^Mk8;%eb zGU7}NPJ$RNm-MxR4thJ!qQA0ZKm%azs!2X4D-x`htuz@D*xR=6+V;sn|7|?-AEM@#Y`@s;Gbm8R8ym7BQe1c*A zEVD~A{fFodN@n*#%s3j9j*34|)#6nVZ+vn~}s9)~}c1dL~oUu`M z`VwtGMs_gzU$p~!cozJC{la^N2)w@awtt3`P89LIO=G#7A6<9i%^zqw>OO(wJMZOc0cHJX z>26$Q1dRD0zMzOiE4Q14wTWa64^sBZqeDN+K%eqZ~L{Q`FXzciu4^u3`AnsO)WBq3bl$RBXL|@rbHI7R z(QY~zH3|Ngy}*opkB^m_`POO1h9qqdMi9XcCW%E`my$yhW99k2^N?YhZbU4+K2+X8 z631r@+N$yGIRR-4imM^^*W$^!yUmiZrcl;syWut#^ie4s3|ivcxB8Z+UKO}^O#K{~ zCJqbFFwMUSN9kA)je|&&7?K90&N4qHgxB{k@VTa-HIB7-xA4*3P*c;kki;pp<3b@cW$$*HidYEfTT6+K8&)tMn^Blk)Jx)wAa3 z7d+i7&qxkN*tH2P)$oKYeU`K9ttQr?T6FTVmny1JUn=j?uHUTFz6az);C_WJU?jN8 zat4!4xf+u;oQviVSB9-N1_*GV(C2>7h!ybIM>Di()6OURTNir62^F13C-SDpJD1D{ zoLr_^D}9zKc{&n9$}>@w0q5A}zbofucV_;gB~|kear5`$ zPVhh&CM8Vo1eaY{v&a|5zx?|!bL{Dh{xT-H@wXxEiEg27cN)}56>g^N^7V1KF?~fK z%lX~2Z~~gmoH#OpdT<0=xsbxPuspC`z*&~;A{cfcE1!5VBd43%wI{T)CUhpub>}0& zkg%a30c2Ox1G1TA5(lsg7ASE6#o=EDs6jCc-T)H5K4qbBAPpgEyhL7F zu1T?VCHhBnLCAB&lBmc}5>>}8CClELho>Ax)t8Hvh?n%TUHR7UZsms+-?hsHxL)J3 zzfh|J0q6epJ)AIfx6$i6yT!uye(dNJn=VnO62u>4g_hqJ>QJP%U|Ht{gU?w)vx@t_ zS1B*Q(3*b8;XF0S*T{2-I6Y5NnVxIH#^I|;qwc|W%dJ}e=`SI^#=PWuTp&()`FFV5 zhALRk7OX`p1?PcoNQ@lVzjC+;M|3{o;FTKE5W(e?0PWCO(v)m4pf?KhGW5MSm{IRm@rpU&XpWXl}y4vwLJ>rGo`?{t2HzAmUU*(dPd^#P+qbi{r6^2Js;7Q}ZlM?Q|pYvQN;pwlSX`mRw`3C3`H12rQoAc^9;fndCRjF zCd8tv`OC!*@!`--2MZ9z1k4W4{c?XD!qkYH_r{q+j-^g1>6O$p6|F`197f^Y1V@Uq1(2;hQBPu==h(2H%(`7U4^&Gnz5hTk z<-BU^zm73FrA2{WQ&}IlAADMR{qFY1MWIMla#?o2e?hwG+2uzafp;3V<^pY@)Chn2 zyfgpcYSyue^ViQ3i46P;i^bvSl9^>eIn1ZO3k;K#xWm0j<`KHqr%S8x{X3}Z4vLmGGw^i&F4P6oH2$ zKmcK$GtwKMw7IX~6eGp#v5x|POXoNzaw!duc6p`-Y2VYQk5y;K4o~87a{;%!e|=(` zUKmg&@(v~N zyDFkpufgi*OA)%?m1mFl4>~Fy4r-je4l?#iJH&gf_5?#VA+%yt!>Qg~>VxQ#EGovK zy~O!`bz{GV#ZsKwym@Xv#~bB?-zmt{Uh0v1Lr^{b0$I=IcpG=z;W5n#GHUxA-HGjf z2NXVgd@!Igsf;6yAZB;M|yr zG_x8}!`lDuo$YP@P;l(Hn>hn${H7Md5Tnr#RUAd{b#QJ%Me2Qk6|S4_EJ*wmcL3jH zC%1|c{Sa8HJtbat^g*zz8Ra%c`{u8~0t<&w*$%Bn&aD?eh0i={3rQgf<2sP-XQYhv z+fE*U7=C7a?liOHJ7H`H{+X<^`g9R#1Yt%j+_Us}z)E* z5qF?aQJtc586)wqpoqzS96mERWg&Uu9q|V@5%bkqg83#$s!BazL(rsX)Lx2|chEHvQY?b?Q+SkQ!jUEp}6_ zK#=^|z}SMXha39>q{C+2fNwjmjB=U&9zH)*`SmdIqt_#c!a9D)Y}YBjnj94%<_<)L zKb#Zr3Kc0`Y5WD@gEj2rWxZ?^1z#Os2A%E#FEft+@h^-!&c zN;wvZyd6mn*FSop?`(@SoAu)kEy$)U^9aKHA7$4a*W|LaC5Da&q5>isr56D~kWQ>r z0YRDwL?t3sq<4tDBE7ex0i*~>l^#1Cr9)^cMY@6zF#_Md34!1__uTvabACCJyzlPJ z?Ck8!GtW9Dm#c!r|D+BGT2+|jWib{E5y_kDY7@$zUUyx0d_dGqunS-Z!`XrrpOAKu zEHcP3V#XMrQV8f#KVaot8pcT%e+Y|eq+G!GZ{2wya4O}3-CvE`sJ4$z7q5L4rHO9l zsXkPAFV261K2UF9nI}oR58V!MveMVRN!jghZPRbs4u^7jB4-SF_ih(n8PB7!-ulXG zU!D+H|DFmp_BD34E640a?*p2T-*F#)=G@)I-W82pOiK2N*xVIdvh$fHr;HIe0PGs2shi z)AOEtg8HdJ3-ZhLXFZI0kaElRX}93T0x3&Zw2e#Qx|}aYr?Z4Sga_jF914$u?O7&w zj*ITTPfo*c!R^cav~{Dn9MLnyOTYhONM7_zol%CzrR7#ePK9xh7Q|C6q||zxCMXEB z{zf8Z98^76X+YznAPukN#j;)Ft9^&rqInaTFRC)ne9)8XsB#!Lu3P0bn`43a6J>RR z->5|~K;y?VMohUDv=}Rufvek0b_UM-6&XyP2m;Zw|Mho|f`dCm?4Dg*_}*Za^_X?P z{$I#Nv17x;t2g&C-`#|%gWsTPf!u`Y0V2s*V!D522Muw}G#ZYl{FaT$KjtgQ@`_)I z2`Ge6Oz&m8^5n%NG3UmdiMYP%v0rr0jCWUkV0dse#laI@-fJXKKQ|(yZdPK^{??nJ z;Z(}+!wWg|52HtAY#p3DQ@jWwdAU~HJvI$ZDt0uHl*}UKl}s2Wp@-Yq&yrKRk<1a- z_la&b@q(WefU;4^PaiFKUAq_|HY_U|IEIRdG!}vFU}BMRp7WFHSy`-@JcZ(c0~rZ z7nJ?JCg5y{s-YQ1n+>~;jA$MJ@hdY;wVc%bdhwc2#s}yRF9OXUOuqY?6LXqx-Lr_my1_hjXkNBK)O zEv5swV#fsYSU7_+BCpZv=UFIZ^dH17YX{+);KkkVnt9&5lKGTl|B$dWC)$&Yvx7}p zUgSPx$!Z)p%o~jt^$|d#v|+^M$iEKUOf_9yEKnjs5w;U@PN-i%5;`lp?vVq@L*-nT z?|OH@t_kT!;$CgaFV8jFk&I%{IfAmfu2sLFMl0sduOE&Sw0Hlapr|;W!&-JO$^OHu z9Vdg3xybHmFd$zseQTcqNu;rP5htdI_MJ4O7jL=PL&XbdcLYX;_E%45mD=V6Gg~D# z)6>WS{5^eAE*c}k^mMgvQR*0o8!vf%ya}8fsA(`pB>t7Iewo6nf~tmbzi5BE&o4hKCe?~?{`st4z%V*F3VP_F?i^~Qtn4i4dD z8WD8z^>_8n@ZsDPY```e*lUJ}sVu)0-Vt%FeR+E^vrGI0&)AaWI0fj1S~L^u=F_xO z%675G2?%vd>_@Cwsa#eUDL{R_K? zWl|amTBuH#tCH1XK>5R$a>H|2<4`>^A=4$2i%nB^pbE}wA9x@<;5DNxQ3U_Sxr6{C}v zTQnl0jr8FQlFoj9y+ie%SUPHExfJg%=k zLhGsy+67?YU==+^up{zzVEhyk82Qle8;o_+K>1>qW8*0#Y6aGrV}TD8XhWS&5lY`J zA9du5N_p*mBnkz*o@Vz>dey#`&wfs4wJvVPJY$HMDOjvRi3ycef0!*bNFpv<{7&A4 z!O+&xe8D?>nK)CKt2%5G=3eIxkX)w>i&rYP5{gO_&2|WB`Inh3o74*?DjNctd+SBBMaYs6vcpH;nEwA zLJw+~yq3mHF+^0$cyO1MK*csLUBpbm zMf;W4C(`*TE)(O(EKI`7fb#!C2%vG*V<+ivdF~juh6tfp9Dsk-buj9j95l*(aZ%dc z=wmyaWw%*lttD86HC78#d#ZDf2n^$%x7DZ2PDcWplgGHA7C;Wj`DU(Z7~$uRza~$j zwqRTfzsqXvoeuFE5gu5n9!1}Noru(gUlImGt(p;hk2C0~w^=!OZijnJK2q>OIVl6?6|Q4o!2NItLwaTzKOW>=|)art&^{DQvkc7E7v=9m!KiZNBK zNgnz9M?UgGm{JC7l;+`%{4E|OQx{5=R|pktxQ`Yul-^gs>pYUb$~Y_K;&sqOIe9mp z9?*$#h-mj^qKBZu^Xr0XiQjwnm~f;OzWc53Z^d_uO@}gkvzwPKw$#I$g!FZ}YKar3 z_?5-|1th5L59S)G;%>@LpmMv?u+NCREi`BlI4NNAH@D#yYAufLNVg)i6<5Ze~Wa zPp4Y0r9Dpv#f9-N`tEL@JbKiC@bk^!z$OcJFt(jFHY0_J(@Et%k7knQl$G(j8z#s9%lKXOBB19f1X|5(+zhX**!V9Zx?`4&(BlvxtxJ^p0@rzdayyM}-;7 zJ5}B{I&gBhAT7H!DK;Sqho!AsI8$8moUW7LpkYB=cJY}7h~YikYI0{I9M#P015Jdk zJP=7;*W!VlLU#;@-5#Z^_F$l$go3`h@en890%DS+t zQj^bj*wck1a;XX&lzvMVK2&`u%x!-wIHK7zSB&o%==61<+dJgu{5^Z?&Ws?Kg0OFz*39rLm|` zkBEh~#sX5nvg+9|-b{DmfSI&Hm>k7UEppWd7Yj@NcIUlZ@?w};Cse*q?CQ+jpD>tr z6y=EdT~~AW`m;0pK6KV5vVIr6#sP=Urq{LWj>HjjF3^?pyH9+tQ$nE+Dm)P26De+ zr=q7WGwEb_nl1`R9e^_j>&H9NW=LB5ht z|2gn{JFL$A1)cS9%Cvi8p^VJ9No=?(RI$bZ|90z<-@4{1|L(!6it^EB2J%%J> zPkhr(Vfu&IW6Hc=Z1&oMLcT7?S01m}BXQ}x1&fd3r849|N`Ag$qU9@(;;7~%7QgnYl8cq; z28kZ(Fx$i4fs;u&4px1RhNjb5ffsbL4WCvkZUafPhm3rZDh{(%-70!LrAo$GlS<+o7060kcSs6t1Rbc~U$D>cX! zgaMgIFsCsYE0@{MlP(o^`A&y0B%C_uV)I35jkI{1 z*iD#rTIm{WuWv(hT~&2d+!5)d&BI28x+zwOz?&7lVT6vkt)gPu-F_TKf*zaOkF@ti1gP-wn|PNu1vpOZYKN-rKBZdH!#`JG>zVt9ox zRiwS!JGf|K%09{cZnXu7Kb>HU=aN12-z6@Dc^HnE3rW#~0%MNsa!Q+AP9YV(3it{T zmI^+cG{Bec*_KBFkC^Z9`hGFFWhnY&SRDuYa}6P|ufEO!SoIGEYnXAsdH#6aiS2dB zCzE`q%n_XjGk8u6AY!LI>cJP7twW3yYWO$n{37hpa~eoFU9DL7Y5wX@ad^v!Om{wy zL(lJzA5R~)&#ku|7$*898PeUDna`JBJPvOGo~&*>IOAcoyHc}H(iZQNKX}6fw+?Z4 zmYmvyIZNx(Oek8cD3P=mGc2lc6~7ueulJi<-U>`jzW~`k<2n$XJi+1GquHY!B>>Z= z7438W^y=oww{Lm1dn4-XQ_nN4-rKRdxnDv^)nG%pN_^Ykkz=BN(@9(e1joy<=?>ap!1_@0MQ)F{8Vv zdp<~W+)R~T8uiW<6)AQlxs>Tg(PH6~gT;$&Srxk6Y?dQtLdZcZaF7{}gI=ne&s7I3 zy~|Bccu-|4?WNpYCOx2t>4Epqd?_8R@0#7C_>QoUpEQvzlv8DMc9}QCYInyaO7tGN(iwbNNv`c)TcN2k6PW8ZjcVlKevk_{INVImHLgxHO+00KZX58RmYwE> z(1MlX!?1X*L^Use9x1%bCmGmuKhC7`Gw~93{~1^oWAE};p+i$QLU&*7xe@9RJ|_M%oEH6Ex7UR_C^ihsp)nros3U^c6HxRwt)(7};#gA>A2VSp)S?_!XHsp| zFS1Cvi{U-TZ7-ODSP8iQDZVM;sf?97wxc79kw5E%XR~FzfaA0bNA|bM91&kxKJcZ^ zX0h<0bamRia;x-J@m%yd+m-LN)$VnU9A{08V3n}a+&RnS1*_?hg}iKw2;x~G{VZI2 zmWlnAQ|H$FkUP^IsGyh56m!T$^wC2&L$+8*b$(d#*G9dCk=#6M|Gxa)>Or_)uvqz7 zPpeRb`mZBJUdxAW!6U|5oL z%7yhzt8Pc(36)6;799sSTXQs5O^p18zY`20R2DvRs%!~p2`=+4Iv`Z_rkfL7?I8H* zM{N!M8P3_Jo@g~a){94HEGj27&w3vyX)jCiBK8FONk3J!EP5xAS?~)pk=bnMwMBdS+W`Dx1f2pxVdi8gFlZRQ#j|B(^%Q-`OA-0m#4;l zU?FakZ7q`N_lT$-G*kg;k%+z^ICJ#uehZ+1ZNwFKwr) z6P35MmcF>-S_UiCn^RCFI4O3KRo}WW~o4MeK2(lJ*>k=)_#xW{;r&@xDfU9VUgcbc7E527?LePH zJv@`MhIm$WA_^lxJa=x|WWp!uN|w%^T4XPmV7uME99yPdudRXI=fiyuIr?=6eXo!d zOl{PYX*qAtd%JMO&sbwzOeLB7!Y+K_k||+I=n+m8?xHp_FmbpgtoeI}$@F~T5>{MD z4^$~|EVMT^#E|1q3U_%M^EU??QrgK9=4dl2dfH}|w5vJ&o~_BU{O+H%`@&vWI6G(- z(AjAlXn{=>&mX@L%EG1L8>fR4EsH`b*1JBdzb+%izximL-voSw{{t&>jK|FwKX>qZ z!h-mnw+T5*ZWoQ_+-YZ%qG>{hzk|C=k>eFn__X{0V`7rl?r}|&A!OI@4%F8Yt>{9d z7G2LZ&+Tunh!MZ9c1v?=)9ecgyooBtk!fhomhRrDWrMHRfX%|f@6vSFD|(Ad6%Wxo zxwb^jtBTKl11Y>$c2w&eEL?Kr%iDtK^|U1(Zn@5JBC2SmaIxaO5Zaq({=zF{tmZrN5Fx=$Zgk1*-Q9a)YWXxtyMf0x&2q=DoglO*6X7s#ir>~O z(c(QJduyq)lC?gV=a;2fNrgwghTsG1#ftC~F8zgxg-^t_=b4Ajj>biYUP5ZnU5i-k zU~D)t_zoFCPoLuUr4bpBj6I{D?0WL@tufwoi6oLwUE=VIG{#wuG-t6(l_F5A;FgJ_11CGa;t+TzFD@WNTXb@ zPSyvNwX%)tB!Y0pU^=&WX|zfkuXolXYGGh%Q9OAVO?)V7g zQ}s`sSm54ZW$6BLiJzZi?DAEF9j3!p{BZv36}N){F(n*$oi9hf5p0IMKd>^D2ez*; z>YPQ)*mHYD5z8b)DumKhemKV4S$&?7d>*3E0x71g`z)XAz$`N=wQv@X)RB=ZmV zYnENztJ1YFx2(8Lm4=cF2rEu@J6qegkUZSFQ1ozZ2P1uO5FG>nxpzXqd69}-q&Pj1 zIiaMW!2BheL!yib(wbW^XoBV8>IA>=+WioiLRyo5tYW?n6;v2Kx99Ndr7RYOPSwr! zb%QM;$uIYEPo8!v&;?~vk032NY{b>!#jZ}(x+Oc)=D|3R5K}AV@3ouuG~ZC=^}fh? z8SmZSIlFA!aTsheKK&f^ju`fkb#v-vEp3-SR5ne`zFKl_qWL^k8ACiou!+x4 zYI7R+=3aa|eM%^G3H;~Ty{pE=D>ImBP?)HU+2fVup6gm(k-aq9uX%RfQrlb)-|QdP znVsvCy+m!PAK3Fqtr0uG*~Q@HiR~~_Jiw)e;7!zY6f#g$a_8TYsx<7E=TVeG0ykB%-#pI8&J7!<2E0Tl{PA3GMI;|no#emUW$01)E!%tZ!6MhxT}(p z+b}J&58Q84!zV?DVL|AX9Yjv0-acieq^IJV+l)>z`>a!Ow_L{-ui4`+a?yVhHflS1 z@ew+EzP_4$b`h~+DLC19^Mgn?9Bzns`OD+0`mNg+EEX8zYUol9!xR$lE+s;NV zEh41U!Ezn(YJW@G5axE89{lAdm|7sitL;c{sHhxnEdI&~E+`q|5h52-EVYP&OFczS z-$ysUW)ZGIwqTFwWlF((*=qa~YV}KViMq+fQlmE37xd09DBm%g4ExO0j2+U3FNlP^ z{GvH^JQN-N%HVzPz#;)TJ)hkyAkhCN@h9O<-ZhY75JPlBCEklW^-I1(?F$J{&n=p) z=s6_dv=rl7KE*~frDNTWyBjJZqf>VeL<_Q&i8`63%v1@_MfI7~&h%gJ8+OpPt3NC7 zU=knovHwf0`LHRmK@ZXVy7!`gs13q4>_#+TUH$_+$fPo?e|qBRa?`JKhM(1dYsz!% zeBO7dJZVCfjj#xBX$p5yOG^|QR-F<{bsV*wFq_7!_m9AyHv(C8c&G8B6_R^AaQE$Na;6 zMMMY&mf`saYy)N8NJP9`21v8D01|xYx&?%Y*EbRbz?0noq>0gF#PVgvx6qV{r_N+|E8WO%O_)nBRbx`s^9A#&We-`O>2te!QVMQSR`Bq`L} zSP)%$C0SuKabD+Z&(LB>1S3CpSG>(}gPdcYYW3|*4zhW@>YAk`VNSg^lKv@;nSj?2APtyOaY$%PI3i_zsx?xw}nez0Re4BA?tzst*D`aOC$-7)g6U%2n7xi^!;SKa%0A6(zAYz_b(%oE5BgHdd=O(fv zg7b|GN?NCs>(TUm5gpAOHl;J7Y2BmQ6X8?wy_@m!a_(9^C%Z3-7W{U9GTZs+-I>l^ z<6#RSqsQo!f4mK+!D@0B#vl2l=})ZX%d^MCZ`RqHARI=Mrs+R_c|dIB8_U%Fl^+%1 z=H<;d(9Jie*$+0$>=)VWKbNghH#ndy*%L9*eyUGey!lv)?-G7^VYy~0%w_EgBrz4x`r>@KG5@l7H6ocRPq_k^c8ANoUy z|09t=4x0CWZyRjm{!5e3-O^;85dy%F=rK>8Jc$pS_vju2rX#ZXxAdT-ZFkiyfnR^o zR;KIF=2KU&AWgvCl~5=?$?~8HIm=>cR{ZF90OkoXnNjom{fh=GNlT@r65f`DA{<&} zBF&-FeJ^ri;rz0OCBnIm9p#~y!U#r1+E+20{pj=)K4;t2bX2a?mJiK$J;;YJ%;E| z*yIgSol4ehwgz+|Q8Tn?3ow5W2jZf6<))A?SssWJigZ!Dm|ktosgzb8LXUQHl5m*o z@Sdhkt-Z}=(Q+JjE*j`uj4w31yQFTz_S|i1#QLd-`46j4jV^Ol5l6n1Zk>ypZ`5*g z>v1>GY0U|5deYMWWKsD;wDb|AqQl4q{r=5UOQD@06tczJekqj%-=5oj#R`l}#|T+T zJ)t~pjJZ>}V|2PJ;H2KO{Tm6)G66EC`AP9l);8}2zQJs`AYTX#mKz&gShuN#h0cMV z=@QDCjv$223-pxM6x>v3E6pWZCW~3Hylmx~JCp>%sg;T!7))_#by!oT23_PxQX`_Z z_qEN`6`~L9`Kf!Z1`Trw8g;U!xo6bby)+Pn(+1NSy;h~oVfhKaZ!Z)c!Xm-+vM(mT z@zpqwKEP4q9~A-ABbjSv1vv^1cKuV3s6o_)v!tE9QkD->wINr)!N0tNF|Fp9JfIfc z;FQEwK9#Q=Apn5}BIcIRE>=Y2rmCu@cdgr{KU6El@^fhQL}=2W>#S+U@-jZARCJhK znYp6ZLwy=}0`GFj|+NsC!K%yfHtZ!mK zqJ*G6orqoNuXUNum^^%|toa%1bGrm^so*7REz3CEeY*;h=l zuS*ai6Ja(d<6RodL^^YI`lg~D#f$JnbjnTme=h{nh{hi1y?1Xn649AfY3$uNuVa^M ztIF{sb@$;qx9=C8Sia@~E6F!-sK~c^Xph|z{R@>HRNd;hZ5Th+=FuO1g3Y7f1j=g# z^>c%2){F|EaZZ8>*h!=!8CWExtnV&LJJ!rbP%J!s?O~S$tKbIg(b#9f-y8q<&soey-iBZ z0PlwZo^xV(+Q4Q#7y%%4Ki%A6nZ=4+?1mdJq^>~0nd{OCEmS(JN**s#68`W^iUU0; zsq@&LGP(~)a{xwHcTp$v;7G>%2>S@u8Z3)M18x~BZi5RnSQtyoc%Qu-&+ImHTYOZt zRwcZxU(Kw^KXOVlKG}$ok5gY|)o>~22}&CU<^M^@7-p88z_ngKEqZ4YCI?BEhQ^&W zh?3DDfnQ1)A~ws^AL(_y$_91rYGDA@fWs{5k&wJ@_E05%a%$LYX=c2%ND8D<`=9kg zTi@ziPGm=S$`dq!F>@Ra0|-PaO@Kr@K>T@FVrGfQ2W}>qd|2M_*F0Yu8~!{sAe9B1 znFc6h5>&)*)nXi&K}(A0`8YnDSIlQ_dDg@lHP@|O*41`7?>hdy$i_D?(4e8!x^a>u zUIY#(2Yi%M+X!+^C?D9cRZF$s;_Q!W`hXZHis^y-F>u3)-_D0c{o@q#Sbfkt;S}8s zL#1y2kYxr3jk)$6q_XLx1tM_RZzfV7V?;Yi0-;jrzn z15$6Wp;y(PdXWANWeuZqdo+3NDJwuG*4aF`YaeNW`oz|i8$<$WZdCFE)mO7hA3mCx z(o*j|z-#@4w_UyjDm)F7*O*BeVGl{?v6fU0p-P|m@w@JjBtg9lpeW?-Ta{0`NZ$|c zFA_N{|AErjOWLG?DUe#EKsDU7dh;_YOKMW>2cRPl$ile{#LVf$E^?C%9jBC(kFx>o zHUt}nzBNKH4W!%$Pt2_%IVnbJyd!V#?-PXulw1JiWvg>rYmvv{zzkSeGUZ82N}tJE z-_79gz?)D2*T&p&@gI#z|* zd|_l=x5icuXgQJBB!-s(yzxCifTt*t@9m%k1x?DxViJI~!pp5i0W^1+wk-&^2rwCV z^zCClRr>m7oeI`>XG9Q!!?3c+qj}zm$k9>2M?RyVh zSZ&H8jR0s2G%Ex?JmEzmCIPG)NV=0k?s~)8BT!Wyev3^wSqw<+*!J=Q>AGJtlW%lM z6@)o9HtM@?K}2Jj@|*Sf?RsM0+F6bl=ZGl?Dn2-xrS?~sk^2u=Y&5z$eU2B+B__rtX@jSjGRQGA zd6EU}Y{H~x$D{y(aa3H+T+NVAn&7%u7l#?Bm`T^yOI(JPa{{<|^R)fo1E^JlHCPv~ z(Sx2!02}fzxLsV(NUZ6=k_ce+UbXQh26+muG(`?{B8!+3{p`ltgJo9HqZ001d!Y0$&^XqMl%4>o|V3PMwP?RmO~vv-;p(@ z%V0{?@{+;^dX|J;vVlFJW??XjGD*f}4EB+3rmUlk*6(KT%=^_X?&2xzO|j&d$zKQ=8g065)I79|R5j$_QZEn^^%M z8j^&EXb4S(P6gto8M^qJSDYMQ@z7kf<}o0p5=S-YFWAkZ^Ip8)Y+&N`Y{L`NtCan+xLW>=Va*QUEf{Nz>vHHo}K6 zfK4#Sr|l3{(yX`9DjzswS^vv-kM3|?#s=<`;(aM(>Z5NV$VGa$A4;m$7b!&AQjRo5 zs}9lw>LJO&Cf+7#csp#YO=J90^mSZ0KJ(np?Ev(|_`^L+cS&+0viUVGMt&PDb+&4t>62pm3(*n&qEDW^xW~?w9JGtF)JWsQJ%CVojc++guH%8x z;Lo2gM3fYyg?o`>&n*wm@6S`iyng-qW_Wb-qno^f5uv7GWkyYNdF4)|BU010ygw); z+W7PX)J*GhC9VYAjQG13mO6I>Z16T2WVmQvF>Q(IAjKCLMiXy3y?G4 z^F0v)m%$kdVcb6~SRtB!oGjRyIMXr!Wg{ap_4OKn|8;zrLK5SXF^Pvw<6+SQL! zGi^e@CC9Y4*oCV(8$2Jsj>z5)EmVr-I7pRUU0Ctl{x znqLA@O|P)5%hr=e$oARJ#6V_HJaYgrtF%X(I71_nlEwISp9^42eEHcwkGUAi^q!Ik+QT*)vuPjlzLbW$u8@R*Z zO(G$3$;DQCqHK=Gntbu;riljK?3o9R53ZS^17dKEDh4x7J8N-%Do`W+*M0eqn{olG z%+PW@VRriEUCZO`IqFE@$ks;e0Hq^N`mqWkQ}jS5NPkJQ_uQl&1c`!`Jy5v*Ugl0k zLkYOA#vf)oVLOMm9*Ys3v$JR(y^D47^A`lCxz$Z^jQP%*GX@j%W)>$px%G=a9!N(D z-j5hCnki5%N-*C@Y6wD~MBQEAw~qB$_Od6RYQKcgFD8JKBlqzG$6>D-v0LW1ur>#c zL7m9UJorD%;`tULn*~E8Afe^~qLu7CrU_+KN2rP;B zrTq5}ch4V;A~i&;*$VK9Y_PhRco1p0mVf>%karAl`otf}8Adhp8o$f{p(J8z^w<2v zDZ%rRFS+C`?wJZn0lJuLUnkfaWl+;Iss7(b7o#=hS}Q;EWt7^j>2&o2#LtC04xi<< zT}-sH8mKw`>NF)oMOw_{=1{Im*R$E9hsWcL((Xl-fsQM2>z7IBN*lvC*$bhlt35 zM;<3A2MP{Oeag*^xPIWdyNLq&(BcCWdZ-r+t(q-|ofT~~((lzmz_thGh{P1zEZYKq z*;71v<}-M?(F(WiHqxg$5gbmRyQxvQs6bfL8lJ*=55}*$lb~W-#iydDus|MB`3`5Q zd#qcb<}9es27o3@g62573XpWVR@DKOBJ_GC?OOEObXXpZzI$E*Qmh!apfty{hYRhX zR+0FZ_Y__t*H$_<&2Qmt>s(XcYKqI@1d{q3>R^Nzn2Hpt@x-E&fDd7CFhY_TOep3T zg8_CY^P=XcvIWi<{9VT=g;%-KA;??NTpHPj8qcsiKc8~~5q^S12T&E$a;ud+E@HUK ze@KNt#KvkYVl>gF@7T#G$d{y8A<0Cr6i4{r7-AqcZDoIM->%`U?;-}4+)K;_^X>&f z5iMl^&YsI5Px2Vy#&wAr*EE?(o&V=EN7y`sg8(x6A1{+KNO57tku$bEXgrSxM zo2ME96!3XMF&%42BAHs$!5qO0Qjb_bugS}pZhsl)O!ikamW85qw}hi8EdtFX?yCiD#>{Z z+l&hvmJNwm1O5)$Zn<*rDUv)CMwYfEBMkF(45jew4$D*B_d0YYHX*)>c43}92aBbc zmZT`T0Ct440m4yG#PO-4wwYRKXKj&=hX&6nbCuH6{36~Pt!GDylr3^JlkCw|kS$P& z0Et4a~5zEv#X*-B)XG)t+tQqW&n z;yNHyLGG;;QcZ<(3e%GRfXBrZ}hvaBkHk=}y9V8yzM)A&MWjOTd0tD?p16K8brGkW!o9gGi7BM1dE z(eJ&`cF)3JPzDG6DvuXr>59K(o`Fp;%0mEArEN8X<8F0@92(Le6nSNp6f0YyzE#NB zGESN4W$wo@(tT9L|H@2a9d|o3(L2!^N-};WPUoZ4rgla+6R@Z0*F%cRFh(PCKgb+yz_35E{I@*dWt ztJeF(l=-EYwuLpUMX*IRtfx2(6yke4R3fV5kf5Timt!)(KGrMajnz8|%4hKm@_!K9 zHJGI+neo^A?_89S^=J3Tr-h{_&20m~w&|sWnW*xKxt5LhE{yQ$+hakN$OrwFoc8V5 zUK2Fo*2@nPgx7P!^AnDA^+ovC7mnESo&=ftVgLB~>B>Z+N=7B2AVDs4Da_-nnUJ|Q zqBE;9&7T%~JvUML@snJ=%N^d=sPa^#FbX7fK5c?6S{|r=6#{(j{yAQYDx3LAyX4@C z3`X(t=JcJn0XmW5Nn=8T^cmT_AO;2ceH9>r%|=#9)pNd~yU03`c?d?_u5nwJ?!2cV zT}!k`qFT%Rt(Pyh)Dh(-`h1lLzAdWw1lKDP-*B=jb}a`hCw(r>-f0$(Hy0yadA@EM ziS7IO>9Cqg&qbTk!rUG}Mq+fB73l8PY?MU0<1f<5CgfE~B zX@uA;g_pcrUZ1|lnw-Ot*T#9k>>02Z6*;cAoc8bewAc_9>=bUPXPCbM-lW9l%mkSM zS0wV+@1B`PP+~^!XxMwD*7NG3u2Z+_?7H*lW&5_5YUej9p4!!%26UD(~aJG7LHNp^7Ka4kHG@ zNR5O-+=N20;mY9}VNK)W$?!W`6oobkq1J9^!f3%~`-}~$M(aE+HK+JPArJc3B^M0? z!F&AZMF`XgIYLdwr%>lDMOiyQ!PMa6+wDxCc`bs>n*gF4b40j$SdZ0SGc-)|c#Bz* zCcoyGGPt_NAgcAHW!j`wS_B_?*SX5@boJ_Kl7SU5L<}53SN_fs9XKygG+={P7;r@o zU<<^h5AutIAMc3(#r^_tt+zk3{Kgyp8q9z}mY(y!N|_iKU)~vL6irGPtT{@t9eS}_ z1u)9lmYcU|HWP3=r!mZRZdRrAy_=_L2tmF$mBBsETVFu`iz)5jJnX4&CRv?T=V_|h zH&fJfwh0@8^$63jEC}ehT72a+^5dEa$R9xPy&3e<)&#{+G7^x8KpT3ja#&j;@-c_s zuTJ+&_NSy?L5lwbUjeE6uKf+3=QX1xEbOF@U{BTZm|L`yW0bY8>6}2SFiskRIr@() zQ9t3y&YGc6tlI;dZJI!~=abx;f@e>^&vf}?a5w~vX`d7~#;+tYVy1L9(ZqnbWD+%3 zXxyXB_qkblHme<&T=x-b~r(&{jDo(f1j$Df%2}pwmO*Vxxu&-ti ze^GAfG&KTTqaYQlk4PLJLrEGc_SZ+~dXM?M4#H5YiiUt|n!!vo#Fmrwh^!<~y|v|* z)lCHoZ<>@7P0?DTwmd|4zFH#C|A)&w|BVWl>nW(AZFAK`CB!joO*K zNmF=zDV`J<$9u?UaP^MuU(2ULB7uJY4}{16K=+30qLoWNB)!1UP9oc*{1_73IV9sB zHUJEin$Mgv%1@IJ`BR?&=Y}j)+$~apY|Q}ISc%t14}1YxNZ^R2vr9p{#$(+pkhAkt z;{kE|Iyb!1wijOCKL?j#HO^C!j1MeQW^!{tvFSl&k635(#`@zw))5!oL#vjrNYDRq zMZGIT7Qkv^_8^_Ht5|@>D#4M_`1uCi_+#t&}(Nj5Ms159loGn)9s2NDwpMVmB z5dUy^EuUGF6{=hNL;|Jz0umDw?TtTDaO+IZ>netHHVr|;EQFQlF|ka5*cQOcKiB9m z-dH1~`=^uEe_TB41pVs)MNUxMzp0pHr_h0!%)qOE8yqFw lLC=zhqoLkoqNp4i8 zwkF5*1r?J49Y!m^y@HC1m{O)Y|0CJ&dNC|74m=@Cp&M#l`?6jC&%5OQ{q^LDn?vqZ zJMW&r|EODXMw8kIq@V`#Vo!mHwLHM8-v0hoYD1S{dof&$zr>bGx^x~0z!09za_e{vmbtA72lR6Kq@{zmixavl>ka(PmS{RV3N`>+gXs`2Y`!^&3w0b}(P512jd{ z`y%-B+F&VS8R3Iyj-guX9HS`~?!;ueCKllD1c83+%D^q2QrMt;iUe0hvg$cEE?*`n z5v8E4>>y!P<@cq?Gd;sPb3=iJq2Vl8i?Qbd&QnD>|28u`r*NPJf38L-{NtL`gsl>GMO(G z$Ij{^__zHENL`c@WEWC#;DIK?WJzfAY!e;)V=A2TCj!y_l2TvUy#9Bt!dO zJQ4UJS0(NpxJqH~UrZYw3`W4iEWGM}*DRajh}Q&7Uf8{rxVQH?Jp)^sxKB~=LKI=* zbl^AsthE1UFEywG7ETj?{<^(lo-;q)OL-VcF8@U{w}5)By$l5?)1lB!uUr4$gRzZ! zDi@ToSA^U@F#*D$P@YUrdP{CCTOet}FzT7;kcwVzBqNac^)1Q5IaH)C-Yf}*G!-I^4e zCOrie8|3%D{XS$9<|>3@XqtFzygq<5(Uku6|06wmJw7#J29&{9_f=I@+)P}%L}h#% z8wqQ32Jjkd2yhy3t^OSFt zqumcTVH)9p^FvMy_puVTvD1H}>5*SWWUsMiua?T+OH@cQYM>TXR}^?6KFml0K&Q^Q z9*K_P^P$aWo6KMaJtjgXW;|EJ3`tiIyJaVqBijhx8j zHSmu&mmn(vDT~2O(GG45xHfDxT8_Vm2ZAbrHBs^uWKsV&H>(82{I|PwS92|H)NrwXvJ29So?4{@Gh8X1 z?@)xXQ!*SWP)>W(bJF40o=JZ7Mm@?P)%fQ*!DjVx6V=5FfZ_7Z|GrhAE&$)8GkGmzts(@GAX43lx2lSNyu5?FMvEYzEOE!q z)&1rFb(&d~8FWB`y&=;iArN#_rVf4K&pjCMV?z97(~J>-q=*UTL6tZFr`_=Lejr|VSBBwAt=R z@T!seU!aOqp*CbabA3(e(J7 zAYzsAZB1Jnc1-d+|L{GeIM9bSGwx6e&tGeGu~kI@$Mo`lbjsxr>ick0(?-Pc63pd) zc16?#aI4X2Oj^*{|A$0C!cUX0=$z(N%9z)nYZO`iecf)b4YrtP;0XnlKVVQb1*5=p z=g)gv|EoarGQktFw7Q{F))x+N!{?f&Ngw{zG*^dh4rL(i3z-C+Eoq4VkLj;Z5~U0r zsKdlrVKOHPMI7ttmi4dxxfhWn53U=L2iJ{=R=qg7mRW^t3Ikf=|L3&I;6C8?_T-aE zj#h2Y0`{27F|NLA^|Cy!bI^4z%;<&T1C#t~_)nBMtPVP`14evs99q8s3-Ew$cLp6^ z8x|b3bQdI7uj(=!nW~PmBhMJ3=OOAT z{C-uFkn_US99*n2BKqws-YMSx>*rOR{Q9xhp2@+@6oiI7Pub}Iv<&=CeSk06AdtTW zU?@0@xZB(jfUBK?rc7RPGfY~pWfbYpMa^-VdRQ^r@OMPUo~7gh$but@8J7Tmv!3E7 z$>=|X5$6Gt*4k7rl^yYcz2i7n;WXHb6)fEkQrR+SRRi~6Q!DY`4g#joZoVUAlDzGVAfU`$nhv;i#5PBR zrFZj_YWl)fv7AtkaY_wvV@a@mV6(}&i=AC6`i(030i0^LRT&OavI}5ZAR|o1spI4z z?{h1Ye89tLz)ue~moVzsL_x*Pt65+8CIr!1+e4a^7?4G>EPmTdNG8Z7WE zGys~a6tNJwsNxO?hK5U^eY37YK*9>5@VT-KfQTV(bGC%>ZL5iH8{^lqQ@RWKS+VR! zMn=>x+?ZOGx`6pXV+~#d`cl@g9@9-|&pe21WsccjHBoaNer1Rr9c zOLt^SKfIdc3<20*Oa@>95Vq1Su`;kxvArWNVOh}?0eY#lAHJa~>O#^*jT3@?mEZ3B zd&T!)AE@|#Kj!y7Sz0Z+mJBuM#28Ge4A+->S1karw0^)x{^-@3DZvUqL`)M0E}p}LZE5xy)J016vJsYivm+P699 z_Qa=uHoNdD`m%x7Hl@F-R3~cn?1ajP%9s0fg9Kv1=nj2iErXHH4XDcado*B)JbFrYz-9h1`1qBl?s3oGN* z2KG@Y(5PvUhRVo5^dkhfMB@rUTL?aKuNBpVH`~1DKekddu zrGX2V*S!;PL#L6ffc649?`4~utYpwRhpe<6PKB?_Qd zxr0ex-U5Tx%G0%`+Xw0?S5k<8rtZDr5!roG1{k*wL%DwQMSnR(wsN;k!RvfH%h9l7 zOu`%43D@HyDSyn59oE}wGXYe|o8N!iSbR*tkNs=j$n*r@!Xilk=Lf;XW{NmT_mDm~ zj^LFZi_xTS%oj~9unD@mzy4k{Ri1_H4EYe-$V(eF0vj_lb%$yG5^PZqt-2Rmz(@|< zbxtAu*WGG1oHFS>`cWrS=O*ojy#ajEZA0K9g5Z2EwWjU zURUrwKe`m~?vPV(kSH*W%uhEkcrWmW05+?7!C>s!SmpfemHl`D7~FU}sGn0@8R-L8 zC|KU+WnA~j1sH!=EPq&aTO=Sc8GCxrj0~|NO$U_#vs**JkgXz35I}bUsM6|J7T?(G znm3^UWutST@L9XbOKJ&#;eqpwtU4e+Z!|F>HGfEnV%&1D;ivEpH@ac*rt9HUpU zMw7>J8`Ga)ZlK2bGX?pJzvXbK>OJ}b-HIU-^_G%QO@1Lc>Xs-d_#s!rp?l)oq{K

3QUZ7=!E;a1?*5xWkgm*>+o7@dZGJ7^7`|rj8)!2L zj~M>p&F|=AxuHaka;Ncy-T^3lL1_PY)RZLJylY5B{gw=a>A33NNWdW|Zve=af{>4b z#5YKOJ+{I1!y&`wOgBPQD~72)!cOyt{U2Xn9TsKxMJpqyfPv(vprEt}Sad7WDJ6|T z#{d#CG^j`m(nv@*NDeWS2m;dG-7w@ZzyNd4V0?bRd!OqcJ_Ag>=j^lhT5GTKo}QpD z{~D!gy*z%ki-7+O~McAbGFvf8j9)IfpePr^VjPP%{2W_3D zN-5gLtP%y<_k;}DW+!|Ih)!&D_kMpfPaB+b8@ZB<+kPA6=K-UHEHNy_-U-#GXwLUf z=}G1NDZc!9dy;=DJr4fp7C%JkT>n{Wl}c!Is+XStAF^<{h!ez_1Y6>BaDRTBYqFi& z;rF3x_BcHVjiR^)46!s49<^zZC34nQsf8B)xZ8$&|M8993GjIG_v+~6?A$fIjp++9 z+_y9xBV@pViy{Qr-AskY{GV2$XF2YId;~-Ta3NWs;%VoWazDkn+b;fPz0}|L3H1~s zApak&@(kto{GgQJ2O^Jm%$>V{e*M6>t-(L_# ze4YmEsXy&~>!g!0&;H?Ju$`M7MYpD{on3ER;qNXouS)HSM)B9H!9r1af5(+(X=SmW z%UzJo_2y7SaSMLvPC}Gmw_5ShcfjtY@@YdNT4{_J06bLr%bWe3%SOlz1 zo?rGV_QXe4PW=A=DMDmXNa;%+2v*>wDKfayTk7pf^P5AYgMhVCZJS@+bIXm^T&V9) zS~lQ>C;+D6*^I;O<+5q@xKF$z;d%Ht*UmP~UO-rwqbLXLfqUyg_j@F8&%iCT-zEiJ z&mE_*(@Q-(p~9g^H_dk7r`n>DDa@7U>z@r#|GF}W-{LHUwnZ*|=~MaTDfhgAANI3V zw~T8f9-H4jv7Zil)=o`L)oXS;6@FlS@qg#0zc`S=3(xbPIEkfcrNM6L)kv}3HO}Cd z2u{N-DKwSr!ET780WD+=5i49X+KK@OAe}7etB{L-IPp`QX~a_;oEbZU+jBW6@dezw zo~>ZzM|DtuT}+s}r^x`y7Pm9v-$g;E^tbE;ZZcXOd_Ix0;Ov2aS|Y#gck{OHLtb-y@qeWUVG8)M z0`vdly){t7!~AC z__Ghgf`7oz+MTJqYJ}^fRgzd={GS2%#di(DAWPKEobT+|9ACd4)YUcRn*+B=Ck>DL zc@2XE`su&50NgjCB)N7n|AV-5?6se%o$Ofb<@iuz)ff0BpFSCPlFeVl<@Xi+sSQfO zqC=9lfTl4mRb4ik24|IT)<0YNxP*D3M=a3t`{@A+qJK^tn!zm!ngOwsDKK~lalJ;i z^nLMy*zeoR{1hVo%V6(Qe&-BiuqV9Nm0Vf9Y}-=qb_Dx4fV|5Jjq_yzH=jCTn4r@~ z{%=AwNCNRMw^N@9WF5?9YjwzFsU~r~5!GXYT<(c~_m1K&UQR_c_g|nz!kZvEq!CJZ zU3!6S?lYowt8i{(!#Y?0PV*O1@azNp*}sF;fCGHYk2x9tB1KW$Et%TcCp9G6<>!B& z3M!^7FY7A5bW@yau`YZ@wWbDFB9GM|2!L1YhTgF zA0;BhS$>Lj=UJ2-BzBHcv+)$uS$;qDk2&x6@BwTAb~y`wly@^tF^ypYH|TEp!5I{A z*X_TadfZoLyOV!(2^@4lSpfVX$&Sx)?->;X_Ea`@b#?XD|8^;J;FbKzY`-QJ1oYHx z{Cmyxnba2g6Ecn%&DJHKrC46vo$8A&mi&b>!RI)zPr87;-|B$kcj;ZFIH@mR2*a*X z8!~?zT+YTp@j)MtlPvmoPWmOJWD@~DiBq3$Br?*Me(q&hnD*|-ZFX>g#J_2vpVazf zb8#;{5O;(Io}6rJEHL!66SdeU_hY`pjfeVwT)Ib5xJAHM-()SJ}x%_bMH-{}Aq6 zU-%-avh7!2{CCX%BPS~M?_e--Sc98yh~4}i<(*>n>{0j_d(S;j^-0je0XOk8nwH9* zwl|0TxpfCF-PckQIB=W3^uNmz^E0P|W2W}+y&mOK1qZ?Y$$3vr`vPaH6VP~+iF;S25YY*$TEMs| z9<%(}0Q?iy{T&7P=g1-ZTpv=b9Q5^N9Whz0tUQ^O?q|03fg7ut4r_Jn(du{8`jJ5f zW>4bT@!tp0fzKn$s1RlfRCRSCkOVw@|H&>Gg-wFB<2=v%mzb5@@N|wTvZvZper}%@ z1xI}S-RwO179ii%yPkXbyh&+6w_ow&tHoFv8BM-_M+GdH3XcOax=uizQ^5I8r#$QM zw^A(xI;$%M2MNi=4SaBS;TfA6ns?4h*pE&yt==`SDczrk9?#D=xgRzy5Z?gji@YOK zBK5=&RHL11%J#Q7v0L*aHAf?{0|=x&dSUNG;*%4rt+|wD9p3x1Ob3XIasg{lswm%= zq|*bf2q!AD1fC9n}EQ06eZuniX7jrPw;qf+1DeK z1do_e6k_4=h7=z^^-;<#M&AcxEOd`WRbj+*>>+w~VhlSMKOa4m&bUdJh_5Uv{^+up zANEM$UTpb7eDuitxWIk%SN|b_S7~wE@eVu~!B_9nD}=b;or0XB^8o)n(Dq0PF(NK< z#phS5MF&?OG!0!$No*##Y~8UGAAh~btC*dw?KSoYZMi!5h1-5vwQ}kmwMDSdZo!CE zB|BSRtGJNx!6JR^JwrcTr@8RmogPKwBn4Fn9zMx;a4xi>kf9`v%^RV^jjs+%sgc8_ zBUa}3joW{~O75%F4CuR$8U2X5%wM(t(TZDeR%B1k=X{(VYCGIJSDg-7xzpe{+aRgL! zrYVzWqvmgTh}uJLzZLx?8slF3e&p<&(K-She}tL9a(wQjbrA;NF0@b~{`G8#v2@!ox4TCE6{e9(*2UChlQ>8H`yk{%Gx?+3~56;dTL& zM+pbL=`C8>wd2eQE`>Q&D#WvwEiv)VNYnLdj8*ab&8zskWLz4=&ZOg!gV8fy=s-N6@(vY9-0V0dgdp53H| z^d==jsxf8NmyDOtr%^ho;vq{t>1QRU&^1_uW$BPnTG>BX@R;lP(a)LvIlZ=cXeh^= zY_EEEPmfCJLq|tP&FEbLlTNxGG^#L~d{)eZ|0XVCPr~-sf1=(HKB=pw1#feLe*Gny zYH@jatA8`MY?p1*2M+P)%r%CnK6 zSN0ofKp$$H;96R;eB9SJ=U%;defYJ%UkL$%9UKpgtr)W((EW13+>OL;H80xp6vUEp zZ#w}?Gg^}_7yW#_YFnYlyr*xj#vNR+Hwc$}y|ajKPlZlbA0MoY@00)awL7IDNPl>m ze5{et6473$c(J~Id3F9r-q1V|?Y~C@Hzc*3qNe@$QdgUYFMk$3x$J0ClSbM*_xfrB zRE^@D4)vuMrC%WNU|@AFDEzqMy!nl4Az)M4-vhIIXg%hKC828tkz{Jt6ZTrMAU089LwF-CE z_&e83y)ST|Lr#e7)m^gix#n<(=QE$>dpO210xZe%x+vsf?`kr6?6{|njb;d)F$0VY3Xvp!owoxAcGe0Nx zLr|^vCE=%qHzr~;Whf2m$pult?uTDNsPd^;A0MG7pqGSpJ|MPQ*uL-+G3~bfc%l@Q z@0NQTGy74o#X+)L>WZ`)0Y8r`vG8UCS@rXl`EwhdqV$kR9b4Zq#|c-)SS5(Gv=|xF zbUY{iX8X75`55z6fr^gEPAIS4Wy;C>QaApG#4+o|ajCH+Zmrkgz8{Ol<|=cb0x zeM)$PIsSFhTFHQ3DM-P?iDKTXvib*4NK`ADnPLa~wV{@ktA+k+@R^`ekbus#%Eb!~ zjpCov=e!DO0mHN0(_Tl#i^i?%@muygFrvwAysE>kRxmnX{0Tgx?;EETgb(x7Z8Hw7 zMrd2zB~WDGG~~9Om8zWdBDNTJTF5PC*}g2AXaFfP0(Y)w7@nIt&V5;dOLCjr!I4y9OZ{e|Jt)GIUED$+bQlwx@GIZPY*B?}LKj;ZHobjp;i+WOMa8~hf-5twj?po

PG%LXOGG1kRG)CxK9SJ*5k#?-uM;wXlGb64YR(DOi!A z^KJ~T%sC7hmev!oF>thkOJzwD9YycFrP70EUI-e2j=CZ;!=(rJGSlPD5{`G$$s+Cu z>~ua_i*p>md;Ey*8eXZ_pLw4o+hBYv9;q@d1%>99yDxN2cuLXu%G^CkWgjl!nf)Ci z2M~t96le=2E~z)-KNYwSId~yfe2-ot;JpH+n+{${u*fl7*S49l6y);ZTiH3!W1-dM ztV=Vv^qz4!xm?UU|iw?!!Be{Dq8f zl-jog<(aMMITm?sQZGjBm(?7xOC8FYEw`}c(^Oq^2USl|D`&b2B+UMH5v+^AOu&7) z843*pFniy_YC+;z$nCi4e|wr;G5Zc@4Xq29iU*7XnFAs4(i%`!((L}+6JEy~w zh|ZlP8}K`)%Sxw~_qHYg?W`kTDRUyI}eS`qHX48>x>*{E3%! zO+LRph+NtUaaxEQkR_o4=rQVHt9kc#?9l{v=rNTp`OfawXX13hkm{5W!wAi^C3#Fvc+4@;lu9Z@{9uFXF z)WzwNA&+y^M-kJn5->U{v9akBmv)>#!d}Wi_%N$QYinf_3R-WRey0R*edN7DIe&Zs z>Y~k8yBg(Vx!L)nhvAh8JqiMCm6j;}Y3+!H%bZkiujBdsT0K7(6mBMx!7?hj#uXuf z`_&>H9{Lg$lJ@eJ!sk;3BhZ?$i$Xg}+AjjRgFq0Nrv(38=WQq?EPEZ6&9VlQAV2qeR;gUu#V9@#^Orb)|?#~6<`a)-q&tXin*5n5)fBPPj# z7U(g@vZLLRj*EP=A12$TL1B!-Ro7iG_oXjsTRCf7u*F^(2h2C=i8-sRUuG3aN?(-* zEMcSe^cU(DGm`zeNPWF03oCh*9E$`j9Pf)>PF4HrV-E(n9af)W+8wK3P+P%2=TR^eM65GcWwWDQ?n!=6kfcx#|V0OIQKklzB zXXjreeOuVsuA%Q*l5_R+*^9T?WGh8Gs)j~x9}Q(n`&K{~%XPBuIxc7HADL|+5<{*_ z)aUA2*AYyaVAHFKi*IUXxcda0lyK>`7_Qdit;m`vQVm+z;%7S3l^FB9r5fnUv1`|3 zsv%DIeOUoiuZnlWX(5d9kEmx@fUJi{Aw`-+Mouf8t0^IZ$8H5}+#~okmz?Sjh&uQd zX#APl(G;5^|JJ}|h{YX$h?T8G5$S!iv0%>Tmp`KJ8gkZsPhz!y_$R1w)5EXr(NFIo zDg-iOZ*n95&m}XfDk|2bZSmzTEzeK?GZV z&1Iu5O;46QMcD1|wZLMcx0l|Smx9{Y9``=r z8{#cr;ko3?o|3e#5AKB@DPNm#MwX}NBoWnOn}MM~9Z9)3`~;#=tRohME5o|(N2so8 zmq4nFQX+=_*bAt$OJG5?JB-M=x^2hSRoTom zMk6x;WfHfZxFDj)H%G2A{k@qv!9wJ4>XNtkjUzlAU*@%!J=;xV?rfoFK~=uySXO73 z&J?I)*Y$W_fb!v$doSPX!j@7az%oMWc8FlBxV7({r{>Oi8AFJ_b$g@3{rJArDdp$$ z05Mz}jkU+BLr|dHU)h;WUol(k{9HN?TZu+42_WIgIIPj%)AM4=RXbr>el_KzLSc`h z$k>jNHlrghwbcpL^pAh*=0>5&Qzr=KL5(YRR4Wv6^=UAH~PTj_q{3 z!?+#M@frWul6y`Ng{y77ES+Oxx`pqKt(2dTM0XY*IF?H(-^rS2^}6i#ACzy7KS+Fn zk7V)ZO`G)w34ewSAGtL)Dp7=Pw%`h*ntH(^uHc3inlW<^y(Cj1DTJ9RO(Gy4DQ0uZ zLYo*LJe{q1OsUy$k$3V`>;uy$B)nA7hBji)O!4z1&>t5zd1ufQ?jjv+M9bq@v_xU* z1$v{$XU_ja!jo+|*CC9tSgp=S>}tjG+ncSrv?)FG5_a{5o!TPsI*3TkvB5)aYz)2# zj)tA(17yfsFS>W;Bg8k{B;B}uXQ^imz?34u3fUIr#R;9H7c5Fj4#w8qs2CztD_w?L zzm%)M)ujHn~h${ieWzUU2JzLq%zY?}4D|9WK;)#c7 zBZ2VV%u=Aye0@OYm3$}l$8r9Ay@%?Wh7>)qi+uv$L9I6?x)mgAlHr;kKClP)ytdP4 zCic%|n;7mfzm&@p+mE5&(S7NQhOkQ4Q9j3ZzJ-*CJEZ&vq1|@uZ?^Zv>HH!2Ra<^z zu$5eg`Do)|H_RHV zh1V+G&6%C{jzoQh+48Fjv$VMvN6XRU&aEx1qIBsHRq9`Bgyj1tkJwZ9EQgEz>{UrU zD&t8dAE@d?6A-aaG6ljHq(4(D&Vc&mSq=%nQH`HjkX#(7IbJkUF}UURK++?ZQhxG& z!oJ)$rVH2jd&%GD@i1f?hX91sJ55S?%POaZ095%!)0Z3uqj%pI#a1%C`0T;y!S^Y> zqjXqhcK1O!Ih7;A@3r^&;E} z_Whb-`8TS|iZw>9TZkJNiBqbm*XO50CA4hX_F*ry!1EEP+qNgHoNjPzpz-TPfgimu zyr6mrLx4IWSe;sQ-GLVJ3Pe(3E(9n=4;c5afs%vcA=-{(_TKU0JJu@l_1|U0@ya?iBQWuwY`Eq}R-#hLvgwR$nC`j_wgj8f5XP`6bH!@qsYAcJU9bc%d8d*82!jiQV* z0)stFpD6xeyIr88avPMBAyCVyR)C(<4mJAB$s*^O3X_H2N5`gHHW3h)ZnyO~ZK>$a zKnA!Jzt(!5MulWffvlPRG$B(Zy^&)ZP~E!@@kXlmy6&BSMKJEXF4QwpS2k+fN;kN- zTGUZ$M15=MQQhu=Rcd^K!-T7&3M3KshJPvL<7%&pVhPBpXTAcOX?2yUJ-UB{q~r$> z`HUlhGBWzE__Z9~Fp{G$&uQNDI6rxruyG7M?azs%q;S(($juMo=ZUeQA^)(Bgnqri zW9J{0C+6{(nrv?Zdu;4DY9ra{|GnVZ*VT;VpnHs1uF}5Q5|pkwLUHLB(Ir7W6m|E7 z9G;X20Lk*wKFe(kRr)LA*DgMbo~Ly`+;o_FOEvuA#-mWj{)Y1W>I1W(e(f$5)yr-o zotynmN@oT&tbtaAI+}I=8|Iq4^KiT5Z&fzja49$aMn&GSD?9(#s!eI|@4WG++J2)l z(IAecP3_5x_@a+bVv;Qi)&mJiG96)HS;@Us8c%-wQIVT)cb1G7AEPx5+qecE=x;pH zdGe6$7Ls^bP1J^MH!jlkc0nz~DbvhoWxwt$EknO$W@?1Yr-|k0tb9(To?M?$373IR zf%U2vmqZQ=DqlB0kd|bmpF3+XOK)$P}yckf@;tx)>2NWO%QMgZo?Mx9(1Pk zN2X}6Kv1T!iIN%D97nr=nk>*$cojev^N|W@C;Xjd*;s#ZKqy2*%QRIYTNQnP<86_; zrO36u!s#Yjcb(GLXJA)ZvsLaBgNbBZM;OG}_kRJ*jtQl^a_|~t^On*|0D!{7l|4kW z@j)FH9x=~Htm4#_0}hnyG=O5oI4CA|NwvbAZF2??rt>f^<*f<8ku67_`CMy3PINiL z=hs12Nqh4#O@rU)Y=zH3D7|*bgK3P_$Fhk_suQm0v4L!75E`#r|6;)`VW$F{KYJYK zUCAvyR!_O1)yVrRtN)pu%Xku(zXyVPTF$4cO73|`xU;i^FLw7aQEA77a$@7T*+-Kg zZWJ{iQ@m7DkD7rno~ptiE7N#%2YLeFJavxtCDq1Ju;qle_QwFBGpWx25z}S*P`$%t zE?gsPc&-L}1jSK{mIwrRU2DrIguuJPP_Bs>$8+PO!~MfD(i8O=KYemP-gf{m83vW) zrQ^6$`V#w2HTx<8J+2=1@~(dEX16Tonz_TWVa0j{wXVgNZ=*>M6~R5yzKsYIq&OQt zc$%?2GWI6V^YbI0if?lCo%cM=^UBkZt`}&9M|Ve{A_C~qSKi}ky^r1?Cjf5Bc8r~A z3@ieCyjOq|`WDDfTk7QiGCkfuGRV#?-MT6aK-Vq+La!H;-~S3}O^QUlb95i5JVBNY zQ!R-|xpXA2^2qM=4mW9^!CzV3Gl{rU7_5~H~r` zgl_XcqSGcDJ}L?ik@NE6iV+wQdFTPH;OhchR&@Y8sPB{LlVb6hLhtceudGms7#$xi zZY;nks)#+HSn8MNgp#%pv1VwCqNVh=9*v%%cXN zektRkHMN3AF9Lp3cS&NGy(u3{b1cXaDy8e+sPwCHa{9g0Ij}%z9?>+8GmNz?jb?pc zke=SuF1RgMiR#y019X}r(=S43IV1VU(Tqy-!+1hO<`{d>if4mvkm-HYZs=@cLww49 zZaf+!e|3Z`&*BDX5)|TOvSpNJL1AsWFp4!h3|<1t&jwEwupXtuaE@X%ciq^8v0xbbwq4A1~|gM+%$=6cR0qv zxB+;5FY1r)f*xMX3!lv9X}U!sC1igz%PjKc0{398%LJkxHC-BA65m>X(7(M@>24yB z3lFB8LZ8+&RjLmW9-9Jr4u^ zOjvrZK;=*V2Oce{tC}P5f{G45iXA7W(*3 zj6+sg5>O#(xDi*5fE*F&mvvouf0hhKLo7kcIuETtdZ}6ZsbO%_ta@S)b~-zl?ie8Y;B)ot`~kF-8Da z6A-c503ED>Oj816qjrhl>z4kv3*S5P>3`0nE~~mycZd|$;#-rwYWht`4`H!4tl7G` znTtKftdD?DvEt8n9DL?8saFP+w4pCe*Uf7%W9t_EIhxgd!%ub(R`Q+Em^n`OA?G;7f7 zfCr4@lf)&QaQRCUfRf?LU{*|>BeuUq6C?bYzfWKfb~Pk=UBDE9d9R&+rG|Dfs$`~e z8K=8~<`C@By8DPaqkf?Tn^zjegUs1?k1p~tNLeTqplPvt zO<6Qyr#{n7;qdGOy*`=theSHB{79J>65Njq_jPrLq^>$Ge)h2d3y{!DC7dql%|ED& zEkJz<2>E0flx+jYN7snyIL)7u5rvW_mrl50m{z|W%j3#TsSm9i+~PEhtip?{)o6!8 z9BYz^|E@a854&wi9R&|kI+vg&Gn7>rVR1PO{xqR;2!D_jA4Ue8?zU>9P4j3yBoII@)hUAE$JreX~s zSL2rb7?*v}tdbM@o>SD$?*QhWfbeF8f_*-w28$5^iDuhNoV0C#D{+<-N}%<0Z`eHF z30(^MBhIbV>{XzT*ZMPo0ufL{eLRzZKiIBgZ@~l`#=P3HC!y%EG|`*q1U`uDA^3J9Ktt+$Ii9~v5VK?t7WWZ+Ve zd`I9mGZxbgzPo^-(^p7++XUKJQ}3fk@JWzeO=}M4IKMUzDiWNA!l&L*Ug!?i<$d~$s}6q(y}~_c;9Wsx=`x~0>Ia)sB0=Kh zi*14{W(Wp0f?%`gobxt9w%@HlLmI8FawcXIfc9vwK`uR0se75lR=oPsmvF5h?-XBv zEtqZx9jQJH8MdAY= z`{D7N)%JF-u172SWwb!rNHcy5jV_Q|Q-caraF${&Y}tSlxz1j?m>oNu9)GstY$f%m zS$6JdQiS|UWj)BY8Y%^F(yASV7y`N|SOeO4tDa}C7c9g(w*xL4_ih;|G@L`SuHNnF zlnW>1+ZbzAxn@*z(3ii8BgM%<{4HS~!&0OPyV>h{Aha^&CnLV8G15v3y9qM@I?N1%zXOwk1c*e)a zhw?A3<6V^@@)r~v`8Ps;JxWx9jHiS=Ecwv*yZEa?h_Z5u6TL`L+&TpOLDvNO`=7qJ zpl#o$CXY8s)W{FIY2Ia8=Etn*pPpmZYie?9jt|m!3YaZNs5~ma-}Gn5eNMD1us{3J zTl}py6wyaMn{x`;5dJBe=+MD$6)bBmf`=7zQ^mun29wqm5gn-!TDM+*Eg9#|;c?B5 z^wR2&tf_RlDuzXJXUDk#RAlMTUV4Y|yjo>Ygo3!)U?9vlk zKxcBPMpw))SOv;X%d~Pf-$d%^nRkAHF?R~3_3NeXR0)Ef#EML*w3!C`_B66c*;}hL zvN(tHvfO|lvP*h}%RC;5h4p7GP!agAtR)_dCyw*j#nW2X;-xKYFvVItCz@Ax%0=X{ zhRAe~xDIq}q;k48@5D6>FfrQG*fd-oPoYOaY?0*)2kO~R)2>_EiS#+<`w_hVBU6F*Vs4)3|j9?VBc*LNB5OwGx=^QwsspKW*vX^(V7y)iE6eZw?tIZBY~Q1 ziIvYbN9`$!j({q^({4Cvo&SRU@|O*gQG|Amjss;k&HvTKe#PS%JO^LVP{5z0`G5R$ zTO@kplb_GlgOGcqi4K4kc+7ZAt|=niIX%wd9>F6k&_qnDZNjM@)O(BcJr1sOw|b~U zDO@r(7ZVe!7mEdu!~s~GBWO6j(qgoDoC6t~9?#}7XRCbaR<%>ty*^#B6r8PO%(j(% z*GJ^AZZ@;*!x-RyX=SlEu}pnsB-hpVLLdGSuDc|rR0Zo%3VloJW_OrmpJ%TFwd~h! zac#w|COuHZ~iuIS1M_NCk=&6Mes zP|{l*s$U^8OgIrlu5MD~peuM4r1Tfl!!iND{o!V?>?sO8(71H!*_QmYA8WfsvCrxB zTWX4Wplu{Jod6@xbYR?%x2`Ab?<%o0eX`l!Y09^(YN7L+Shrnn|8)@7g za9wT}SkHm$6j8pHy&4S?kjlgyK+w9S@{H!i6$>;dzoIdRk!U2gI$p8m7GJvPg#FH^ zg!`I(KCnAoMM4z?q$bHdR|in)teH3xo>LLYVyWWYZd0PNkeRk}CG7jw3G*_eTt8CN z5S}9#vBz4?@gY~fL2EtmSY;!i3KI(`-S#+@)+6nSqv1{6zT8*5M}7HjTr}+as=<$^1~6e zhaSfnD8#xe=Q+Q~=+&@0Nev-pqz1*;9F-se5Ze-V$M`fgMS`WJOD3R2UuLQ`hg34B zD+wCFwa8_Ntjoe5SG`3O&?i02$4>t%>Dq3%O;?O!5 z%65u4M>Q$1U7GO}Yo?U1rp!&1`VU^_V{4{bVx!e>2;+(>)QjcV-oaD+jMlW(iOdEN zS@EumhpJFnoZPFH-Bbbv+9+|qM#n`Nq_WpEg@(>EqS1j}ASYR?c(nkX;N^1NL+JB- zhlIL!7`KMFk&1T*MjD6u=nT+b`^J$zo*br>BS7);wqno$j0j0HR`c`Ja zWr$O@m-JRRFu73IR(7-HV&pccq^4ud7BajzbtEfq=7e$^qAUU`*rM}?SLA&y45KFizaQ?_26772_x~~SH&{b3#xW*^fS4_O|Bh_Cfy07xwOu! zcPjz5lwSy`#$I=t4M`Jgt~)>q0-L2XkXOvi9E56U#T`5I*0Z|!;WKP`@FZ<=xf@M* z9ly1J6-~8l#e$n6Cou->!mmyq6yBPY6_%cH+{`JBg#DWZAEsP{->p#4~fHbKc8wZtgnU;tlv_8D8#ZO55tPgg{kE9mwjaWM;(!|*i)v= z$TaGXJ#yG;W*)(rKw;kH;|+Ng@Yd`g8GM^o*B*W*9GUFf>VNHjnC?*p=hvVXy*{L) zA~_Q@1S;4-%>-cU74B6rml+9jLDe09+(#DVC?WUR>zV!SFu_Ufm)}3!NW1R9QMK`K z1ZaM8Vkn}2)VAJ3_ksBk#|%$y(S5ONE#NM?Z1xlrelXVRS?rp9F>stxPkP)t-6 z$ThnIkZjDWxtqDWqL;p{Wv{ZXoE14d!{O2)goZz6j|~!#SYBgoli6!$)yygGmN7*# z*=WmbDR-NX+9>KcwabmkhMh{G+v9ueOAFfjJMEV~67B>5qgwTXZllI)fTW+o^Pp}| z(>xA9CT$B7C5b3cIN(TH(IrfhDcuqB`D$-h?K5MOkXwyz>Z6yPg9(=u3a{}tg_BR0 zDhs`4sWYAZ9+I|Bd7w}TX!FS3CYl!&RC+F(-_pb;2<$uhk6eM&%%z)9QKNAnoO{%_ zjq&X7NnCec+sn&xWxjCWM$R)4`H8ITHfSS$Aj8Y@{Z*sil3J5V67}(3_5Sq>6mbF5 zZ>dwsM^peC=a~}1fAOE}-qKUfC^Iiz@W^2R#|D4!-;s1XEJ zyoF_#d+B8p5~(XPxC6vKCf}gImMGA^R(tx=`SE)Q&}NLx$)CXjk+dr1_9rC+ect#T zr%{_G#$B_JVypOhR+A0D)i>3hfRu*h)cBAF<6Gp+xRYcs_3Z0Ar^qZWQ*kI+4RCl= zjSK_#k$m&1`bF)j+ z8Qy%qX_$=j+^}=p$hMd|L+pa~H)06<0o)i4ppEcKD5-D2Ri{Zw3@Ni0!)ueY6hAM^ zp|VaZJwjAQ?dU@B)I~9{nVkR9PCtx*j!pTIukn2Ow^sEH@8vkWK1JVPgw|M30 zh3wU~h3+pWGBBeUQt?T3-K!V$;VuX?GO;)$pH_-{GFW#RR<5<3azR9-s0wI#(c% zPm-Y`Dou_9L2_d6f44iuI!Fc9ia)Z7y4Dd;<-q6Qi3KSC1siZS`}x;dihanL=G zv$u&v$(!>LgdWV78=8IOyz{h`^&Hyc^u>a_JP!GY3ovJx<}>{|VE#OCW5T+F1vrFL zhzH-!CXF-Tym%_>l>DW@q*9F9*@^OMMH#IOhGUG1Z}BBUH7Ky_YilTuZD3LAPl9H| z8^=5Spjuqj)GE7i>$ppinz+DUf^kJGuC>tX20RmEP+*a+9ph zMI5?k<=5op72&pmSI(lt>GKIs?nxk8Kv=tmkASIe{4{hA*!#C%tAS0O1G7f;p z!|dEX0kJ@gq#}==;77I(%UG*Kz(3Dy@H&Z}=c$&#Q$l1`)MD}{XD;xJb~-D(U0HUy z(3G5DRVZl}-x&UeJo1ArFieePC?^+3Ia-s&6n_Y%*cl4Cfz-@0cF;ok%G#-HJ=elW z4F|!Rm7r&AS{DV?Z@#O}{)96C8TSFKX@O(Inat$+Pkci4vhUg^)IGaq*T!=9vGk{B znmZ~3I(Gd0NZD?`b`4HN84Lf2$4wMr`Qq{uh^Grlq%ujZ=2hKAPY0e*B9F4nBi`k4 zU>&bt!#-FU%6p|V5xgTPSHwP5+RNq}&oxzyxl%R(ch-Wtg-@t`{U$3+y`QI5_&JKu z)p47#!^|dBBBl0E#~$EB2ZB)^kL9jEelI?g*?lJd|PTyp?Q^l)`~>ukGPmj8I;nl{st{Dw_k z)x=&Sk4x+=HkFMyFOF8DAFS_J>cpVu)Rk&z=5H?_Dc#vFt46VZ41~-fi2IZmW$Txg zV^-V%Nf^HFysEq%5q^$zz*!1@mq2tmaBj+O=N0I^5P4ZA8+L61B0k{)UDKHS#F@cW zk>_=WVv?HOKZs3Ps;<^~tt2wj2cMAxCj{>T-^;+x8F<3cs&U$>;YEJr>(o=aj{8RP zE4e#FgJ{bMDwo{Z+Y^OWLqIX5B6mf;N+1>ir-%8IwHH7mu&bzxw!4f<=YhAP(uIv^CX~!h&UL^x*S@=axe}TS zgvL^!*bG#Z$3)^RaaO=Z9IUI@taRY*Hu_4tqZFt1RaV1pY_`8AKL+~@{N&oMsck9g z6ed^SXXO>*SX##E6^yX9F|DPjVO6brjxs{=z(;2FidgW4%qRbNszT+Chw+aw{&w!@ zI8IQ=(Ha>qQ(Vbx2u=9&yR*~;fPQmVF&|-x*emE?CD4m$)*1H5t9utAN_s#2KX^slOH}OqvuQ9scIXbO;w^T zO=49pa4kL?v^U1%ZOH}mL}eX?9JQM9jN|Mhv}xqnmZT7?qrd{hh<6A?-8p(B} zW*^!)CrZ{TjZOFc9$g@2`SOsE#0D*o5=?!Wks^P~Q~}L7ac_C9MdLcV|45nFxun*& zGZhP&ksm-mZnR&|*|Ntm$69aa73i6CH6L#Rh3eamAd>Taz#>e29U5|TeG!-~oq8Pk z89SFp`y4Clyw5sEsy&MU2~;uS%*2s4YJ1FCRd1ye`Kl@cQ%33;{v&H0B9qu7q=Obm zxkdI`p;Q+}6h~j(0lSEFsX@rb^a?xbb#MRBjWGR7>n39V@U?M?;Bl8((-D$Z_ub|p z@gnp^m-^yF&jaz+f9v~C8vg!rBJV3e%wHluDBbw^F2l@;Wrj<+UBHfIOY%=PgU9`L z-mk#W6q|;)E67n`1>DcUZ}&F7UTU3Z2lf#|!)mXy*I&~~xox%)x7LdXSPvi3@LDJ0 zngD4Ayo}lK?XjhaBcvm4n??*|1|a6gjt+f2)p|7Kg4oel9nlcfGcQa-%} z9~HR&BtujjaYN{dDtDHLCL>Q+n~9B>&SWLzu7jv`^)Yqmlj;*cD-N`po%7 z>ewSJRrSW(A^MPn<+eG1?}EejBxlC`XgP9cKZ8Ds0PimUS)AJ1gO~-E%PGx37&(xh zcf+-xpG8UgE1*nYv((i-)Qs2a_tn3m>^_3Fo2ZDm;1!r@UlmZ}B*E$Jh}iBUy@8Va zv&oA@pfBz2rI(rn3Pg&tq+L1OX6$1S|1yg+Ac2`ioi|As z<5BJ~nhju-DYI}G-A=Wr<0CLL9~F}IX9Trp)7?z~19tNAA2NPj+|SyC zPl29;@n}xr)}3dwIFlS!FmC3EEF=W3ceb}x`{kEoWmhWY57pIVJ5)>oivsH2q1lL&8kgEF6A@b#n=*ZQOk*uuG*3XY}YDiAPQy zT(kw(d=@P^wlY)tdA%7~1;>=N>pBiuYjqS`kD@a%5ARoJAM8?Sq?z~eK!M3;mqO)F z&t_J_!|!INFNEQuU8C}VKM#L$YBA~0b`M=;tNL#(02j@kEs)#5`?n8vJ~CjNK2Xwq zkj6teJsv}ub-rd|lTHOj2f4(uvVonZAZln;7rAOUd6ZhIZ5#n`!b-=g&=m`~hdidV z@o!0^pwVyBhP#e9^7Qxh84|ry`Jlt)nW=>ZoveH8**wW?dru8Z&R0(gA#UhG;lU$8 zUVA{dwi+?;z5om#ltu7(R&qL8s*Gch1%^N;BM+_{7T&7)XedC+6OzwHkbUc%4gxB! zPAm~Ft}I{wo>KkE!_*2BK17^W!rIlHQZ%6L&W2zS*$V8!)zk@@%@0&v!^M?MX{RL6ppIG> zuc{z4fnz2t+=xKIzONg}C~fAnRpjty`0Nct)+g zS%+?FQC4djYYA?{2+i^dF{ytap$j~LJIdS*mKx1!hQw%!j5g_}3TQ~h zv4$d*W(9hn?>$pqit|V4wkczh(G(53M9~mzi3(yhmF);knN?%YZm>dir@wh7U+EpH zFjlDLE@Cp^s8%)}p61%&lKU}iGJXH2hvQuFQ;)F|(dae)4urDosk^LA>mOq;2r@cm zh$`4t2tw&Ja271pdMkXqZ1PsLSlcLY`%-$Yk}>VM2TX~R{f(tc6T>p+H*)uZ_io?M z`<}icy|ia#os6r<4kgme>kEa{aM^1$Ea)wEr+%?wr^w9?_WTQqpI7&!LjKE~*F&EZyBnO)_; z2F~Dz^WRL6zQXDQv+06@cw@Zl?$CGD{ly>A_t`n#m%A*rFv{3vF?q6C$;w>KWDJpR z#gwL&1+Bdz~XHuL;eB8O%mt(ona%9bC5S?~Q5Cv_E54 zSrd9V@G16A63RY4Ly{>_(g(h_>~vX|gGQr+XX4*B zh;_!RqOpb7qx)&=)CX>Atw-{$%g&h{$NBC@=9*vDk8|HnyPZT&UuMJ& z&BNq0?thL~w(Hno#{gg4o*%-{EEf8Xv~ARu@iY$YmIBqevgkRGcU@7T59}5@2Mx8j zX4v%i{k|~{XIzx=>ogYfM&7SR38Tw9_We}Yok2~i8W8|ni*jR}{j14zfca+JQk*#4 zdKXkWNq?EfY`%57Yq_PvW{fM3nO*Rhah3tQkmF4gQWN)Q6Yn{rOY`AV*lpW&DEza% zl_;8FBFl8NrpPu3Go1di7=Hb}ORJAzvSjc@Nm=3S{fjhtBxoJ00mr@qy*{@_#sbIFab8=DS`i#%F}_c zfuNeJQh58v43Xz(6t#LeL8A z7%p34pO?vF(f-Z~2YZF(Jnf{RD+9NHXF&v|Y%>UY_;Q|8HpB-I)sh_8ZfCl{o=)G3 zZ~uq0H;;$14c~{Ej5JaRWs8#ROZMF;*<~lopeMU*8Dy+Or4`8%qRB3dvMsG4gc|X7J@BP02_)N{*GxvQh=XIXPd7Q@yalz&q_ze=Rf$e@{c$gps zxq;&K{3b1hse;8flR-E?`0Q0e*O60K6AfYxG_LC@ zNX#yEYx9shSNNlzlmGcZuQZ@K3DNOJRscN8er(7NDifG@0%dIS85B}f9e(+~$)&O*fIQtC-R&2##SCY2FLDT0b2y=scu{Z3?~>?YpJKYrL1*1}eSysKqOO zt#F9p8hy_>-fgx5NC(MUoHYGm5ks_~Q)j zD|CN=Ek1iI=ok!1Y=Fs?7ZqMFK2B#sz1Ggzpn zHXTkBTXH$($0IJ!wbgFfQx4e|_S|w7h?t&C*vuXMWpvQ)6@--C5kQGVE7AFK%@)&gT;d ze44Ge62)pJqgq^-V@T{C(XY|mMZ2$op;af!o^Ia+3t5`~s1UohYHVQ)XNHK4+Jc6c z9^zIU$>I-9xKYTOh;SYGr4ZDsgEG{A|hU>sE%M{M?Md z;aBK|tIqwQ_JA%p{^A2>L;;|zo_e2q+E_oA((vD|kH&jd8&J+w8}v5P6xLKvYlrZ# zt6cK+jWGG=rM^UileEhlNpjUgn#!Wm#$izR70LVXnD{&Eb?^4aAS1$0vb@fBo&Nco zTlyT`iY(=N9xv(!*X06%P5%y@6;-o75m1G>l;JqM!E~^q#B&;uc!Lh^&zpj+%bGQ4 z7P{u*@!wsOlg*5r*KX~ri&$fR_<{6LX?K6G`7oupcw#-15^#}m`gkE7w>>Ti)0fH$KV)vy#g>1H|a!zfZtNn_G1#44DR`y@TIa5B* zgPP=jtv4!cApPTi@%y30cu@$z*M>2-ul!2sRW$?d)zCNEHux2Oex2v!L_k=!x)&AP zitGSonL~hWd>(V^>4+rU{V=G9^!Aes1ME(k2LJ@w`yE(xTRWr!S{2UX4Y2CGk9R;w z%qRC<+)jIsI5^Wh6|BEm8yfNHDVfAOyt>hO8D>qk49$O+Sn~}SnYtHpzLU*mQt*Pd zhCQft9vQdz7w~f_UZQ!EfYl{G_LA$_(=D~Vw!wMik1DlwbPx`q2*>0W{y=D z^MZc0q(-}`EMR{;^f!bwT#;m5Q5 zu|y@4Fo3B=6F=9rm4`2u7MTd)bJnl=_(R97Z)}m5-^_jH%6`w3`Tq+8 zB5`4`&(;8B(ZN#xx(r4Q=c45k`|+Nh&_|jYe_j;qus%HahGO~8+YxZQ8fv_r(?$B! zKR<@3sB7uGO(QnEpBhgSh~ZXDPUq$oDnKB^|ox zjfK%yc7hr827&>%N}2{Z<-PA0=F{GeiH5mal===+f}zS1a7=@*w}cprDM^KnfStZ3 z$ZiR08xm%HPwqMN$Vwq&B|kRKCmlMb0f5sqzaL}u7^3~UW^3IpgJ?k2dd7K|? z2qg00MYZbemvNl!(k1#nkR zH8Pb;;pa90Q1-A!`Jpj=~o@aM;B;K{!F5>%bX}^?`Ae)B!-vbt0el^)DHF;2R^|nvwsk_e6B%?X!ml+Q=Z#_?Ai`c;epo}ko zxdTWSnl@H6;$-S#qo2LpPo~QX$-{g!VYYCxd_@FNkn~c6vwi#1h?bW+qpHgIgLyU8 zQ`&N=tKGZyt4KbHg(P3r77MS_Yq_kwX&ut58uD#r@Pe|!_6>*C=>h|e8Isb=^rHvG4Z3Ew zqu+=X30{3m37KO6#&|OOZVME^OQ}vc6;=s%pV_3;yOI-JWf{0dpg8j%_1wJ1#3w3##HNdm zOf^m7C2ew3*0G&IL7ZvcD?t_u*~1!?y%V)hfDp*@aQVW(5z&}m*=*n;FgdbdkJA$; z#Ix!NO*}h^$3HHE82dej?CXF9u?6(Ank_>*E^?$gGv(7&$B1jx2xQt zQG_bc(@=iKz6Ilf+rGm1$-~uA7{lwGkSiaRi3G5m9~^YQP_{{y!A}}b{MasNKihJ_ z`AQfZbG~AnYk_uY<*nYQh=$d%>LYNt2*HB74Iyah_zB27#1peuLB`mBYx+Z!w}4@q z(_4efdqj+9ZOr1Qvu=uWh-c2_!(_tR&GAO6xxqw(kXE!z4W_OL4Pe3)K; zJ%u@WO}uET$Vm+zy9J7rW%%>CfFeZTYoYvi00F)1ba?_cYzyT`F zax{J%ZW@w!wjf7PHT!Pkez)^_$%wdy%r0U77~pufW`=&#tyTyG#N#-8VZI>hJgKe1 z2>36071!%CWz>~keum&RGGmUtrBmVf*+kLf5sL2WbBYFMpepV!8*@Oth(RuU>G&bP zaAHlx()gmefd5^;t{xlVv?PB!ipf~a)Exe@Bzcl8TmTAp4O&5^p+Q5UTVrQR4LM+A zm6_nNvC&Lv&|2@?$W5KFsEqU2J?0wLfd|Q|Z7HB_zt%C93Zk(6UeS?AOE~TFhO(RG zyjBC_%XBJ`BnOt=4XQ;e@pF;*`Pq*r9YIq3La=Jnmn#yND202i$!(JvLskXNy@ql_ zIX=Z}TdblJBO{*OlW%}g9jwQOkc5mf2~iQN9S+MTQi6xPp%?-FcoR@6MN$vSKeU%& zxf}%Q4KN4&C|khE$E48HgVJ=j^}c*H)-#qr%qOc$s~H*|LcXlHbwu`UIVrer3QK3I z@#8`m)>$&YU~a!PM1iK`8c4^nDjYIYBXITyyh)HNc%G57nwBqrZr*U7WB+crv@oyg z{e9~F`^Ys+XXNM2R6iYqZ-fJKq}KpGdZDd>dJE7S5YzSQXi=66pOZapG~Oc)$A+v| zCtv3T@rpB~MFtfLF9z|(;=4QAhuxu2U&!4N zi1NJu(BdlAkWV9`puU_O%rB$x0s!)?BfU2d$qeiB1aQ}9#$&!h!MbdE z9HvIQf!LYv);Esp^M1}C7b&6d(lhdQjMqCj9OSPD^Mv;S{7KAKOLwpH6(P3g{2{L@ z258_8m1qW`m!tAP^Z&CrAib1@HVUE$vQ?C4@Xtj(uxHgA%{3Rj8AHvLk~ z{0>bj8r;oKbIHp4pNDvU^0zw`^!!e9sYw+E3PzMKH8d5zsTJy_fEl=HC=M#`*;XQZ z9nQ91OF*Jy>x+JXYCFH1QSPeqP$?4a)Y?N$Tk?nZ$2n$|Bc}(Yh~uB?o7OtbysR43 z3sM+M@@HjsXOwqGZUm04_g*k7uVMvNTbn(^+(%K;W$=!l7V!Q8Pn8^X9hSd_w?v z^ADN;M6i2wOk$5MK+8PsNV6Qh2;;%e${?Pqst5+5*b+&egKN#%K0`B8#jo8=vx z5zka0@tsej&1c?PH9vt}k|pia`{vsL$`%8v3oh7T@9*YD$A>MYPsmtDd=hbZ0bfSTrA)Cz^{#+jdI)tuH;E4j}Ii?-g=MzUg%s=3e`sdai5MA;A&=HV0Fky zbW7|zj~y_1+%#NNB}6?50iol?5zW-QX>|D!la{a;=3R*_W>r&-Q4nahI5WBhaP&dm zCyv|OfXkRU4%R<4K;ZeFSTosa<(p$WaBCJS^heuej|?aZx4T(Ownz~MG;N;JOf%Z|nSXZRzS`=8c~A0Ex&A4>`Kc~5}{ z0&&L=8eOt_nkK$Qn3nDUxh|Fdjp1zaD9z0p+=5OyC&z7lI2ppi#uWzO%E*V5b4JOj$1F}EPl^O zT`#y@$$B~CC$r|=>{NYgRG$E0}x8QoxMz@kvfC_1|WKA3#4r^|g2v59b6c5WGyR7(`=4v^T z`_~Hn&9A$I$orm9YJJbz+W4gHang2FNbc_z>us=D2<)(M<;T{3@=AAU-&xwfeFfuC zYMY%Z?Cm$XMT}9)=I!4IzBHg2i6@k$Gg2pFh>sajm&AuLt4$CZ4XbN; zND4I+A(;9^G=WN7!v-2bW2ih`j-QiQG701pVnC69AfVydjLW-if~1a>ls@(q!dnCw zZ{x^U+f2;ud-FltwyYUqG}8Id54j>g}>+SanAC;WKL@1o^DMML_fk zJO6b{1ZktnAQ-v!xOJa2Kf4`Rv6enQ449>+s8-g?+)abE{(Wrx)MEgR%Ug0Jac$ij zH!bx)u4FZH9$jZWrIhDkd)5dpB@&O?qxE*!6o9qcn{CX3tBqFOJOz7#VkI>gu9wZ(=SQt&FD0}E?E605uIf-% zS7&er@LETcF;S~fISZ)!Za8!OA{dF3*&q8;UVU&ieP(plflSD8^-#!IE_8j|DaNn= z9kZZgQ=J)%>8zg@I-x`Pp|@`JEU)pPV67Jcr$KVj%Oi7jFNtFs1Zzf8=p-+FaSQ>q z6cJhAIBKb`i$mv400t}5?M5SK2guuep zy4V1R!607lAXLh5Q+;N#dwat~iS|HV(%$)^UWl5krae*CpV%J~E8tb1?5{E!o4Kbh zUpROUqz?V8CCuMnB;|L?@+-_Q)axmAOTR+3$(JmiS4tTR_M`ZS4aNc*sZXxPD^#O% zBe}$9^@h-Y)mU*J)NFrA7H+%AI2&y6BUJn+DFNh_pEev+_iK1#)|+72tT~r}3ziOJ zod@2Ah_K?xJxq^`YJ}ht>liHEO#K5yg;m!s1dA1rs9FI97_vc6-nql|AbGevM`?H_ z6j49v5OzehIJnc&Xa(gh2%nX;$@5DR5wvdCr~dS)H1@%CzcNv58i5&DL(KKM+DVo~ z!9QID>{HJteY)EFLw2x$-SCgGobvb!^`MgN-UcW-J3%(iYW7pS9MGQ+Lj=6p2g)Ww zfb#`O+EG?wjjP@_Mp5sgz1PVOf7PL(N^6e5EZT}yZY6!mUJ8J85RfhzltQnk&#s30 z^b87PII9w@Wk{TfP654!^48LGkEVh_Now+fr7G+x&|GWz8V3FuHD7VzVIrb$R|h1u zJGbW(M8G#xMfaJ(`OgpyzjX%$h|fKZP0l@m&s6I(YlA|z<;EP`IsQF{Ksdr2-y8&w zE7~3Hv67Fx$RDW!6e98tYod+T08C^1_K}7U7sAvkZoi(SDen{5<;u6)ly~18&C8I~ zA(X_OR6>E&MbU5|`pB3SdhuFjX&JXF2917_@RdLL>@&2)7k>A$p zC!Wo1Ud&-w0Fp9!Q)v|j=*;2l%Oa?JzzI0z8!q(FALK+dn4>Y4ru#M1EEml%v4?>` zi-wVyUbwkb3^NHwWLLxOEe?85|3bmLqq>zI$iW^GrP_`7_(bw-ySmsvdK zQtTBzYHa`!->C!V5(;~nurZYcT=GLJcRh?K*<)O=AIb5DsOa7JVp}9-}^U0sAL&=^{-6aFX^AJ z@Zo`zpzuViEA}U5=vMl(@Wj%pximoy0$&>7(D*4y@C&GZpI~t-Zg!lI0+hbIW+nbhf;uKM*Q4&Tsub1{?+Lxv z9Qjt_PzVae1q-ZI4-wxkBqd8c3x(s__+tgJ{DrqgN-7cxwb@a-bvrDSjoJ;bADCm; z(dcR<14V2#TafB9)YWWUbx)q;Qk&uE=HqN@rg+TF6P>b_WMsPSE| z!Za!=VOL-JYmgMMqvR!p=7w0RQjYN#&Wf$+{i6U1ucN2}$y7NZMa)cn*#_ASj**MWG@y}CcrdrF#q6vJ^{A4IZn|kt8&Dfe zuFx4$LFHq@=YE(n)?&_?AZwW)ChpCeg-|}7AH(kX83yh4y65z!2}?%pdh>+`cag`l zF4OWW3e{@NcnO`nrZP;6!z5^WS>T&opbzWq6NLVgzJ*f8i3y zad<%l6UfdT><*lrxc?sxD{x29X;cfRdDQ_9{c+~3$uB_+rISdXBbJOdhdGLx&onS> zeQoEFZ=h~4WxQyY&LWd1FDF7Y_Bm?l7&=<^Ryd`09A7%w=&|yU%BCRzr8*!uS)CT} z%(>6(8neYR+Z+JEdi(sJ$AvFvr;Q^^nyughF(V(#)u`b!fDXUnECRWr&d%N~u4+r{ zW^u}hW-H$U<))qr=0Ws^TNwGG9~k@8sB)2#l-gAxbDf#;sL!l+JSj4FpLr{VAE;~Z zPvgoSw@$o}Wq$%)iiex%9t8y#0a^BE`^*xfVmNgj3!JxfJLVWR>!uTuSApP&7via< z5vOj)iPCbK8cGV!BB(@3O*o!-t#Cbfl{ryH*tlzPH|VVJ=x}TzW08@=de}r$XotxT z)r_4PC%iMh|1XD*S#}@R9$?qk`@&RTtbal@ zBY)!ns}}lxd3gq?Mf9$g@akqPrOBg-#%6KH5SdoS3((mE<|Iq#y-tI!8+(R6 zYhddIn&IQPOpZR0t0afI2_aqM89e3*VYMja>9|zkl6}MR!ohohi4%G=1Z$X5L=7cq;6&GXPx3|<_k;!cDSg>9}KEZ}1b6 z2L!)ta!!1|$_@0g@}6G<<>zm_f&3ll!Jx-@ECR|dN8L!0d;$u;hd@2lh-p_v(4$F{ zhI8wnB)kelAj)p#%({3W9;OV5W1c;qok#O1$<65ViIzu8Y2nD!haI5`sgvoOujsY9kW6OSWFpymR%}4NlTO1m!FxG zc&xzwBB;iD#(VAzW364brJ?B+~ffkS2Vx?hXI_)CZB#0)a4jOR{jZ@8Ff%di5u6w&3ier zoPYAzE6gJ^{wjOq1u^xmOU2tl7X&@(mF*4-1OsaRi-NPd(c^E%-h@5z0y@A~Bk^Rj zGm0p|p!Hq>2@kt7f%XwpFmACg0H{e#@D6A)h%%N4oof(wXP?%M+(~f#ydSrw zNM=o!&in2MjvGie$Y`Bf_dLzOD40OyrLpZW{``Zrgk)LtyjHFtche^ZOpLU!$#DNU zJ+Z}fqQGvjhA?+?EjP_S_tg#84Vk%_*kao`@Myz61LJli5(GYP}k>6`!a zyMzwh?a)EvqZ$>-bG3>}Z!dkN!C)|NR7o#9oe^UH{4Q$#wZBu@09CX%MnF9&JC9Bi zz8vSmF*cXxllkQS6vmqK3 zYUcy(<54!?Qn>EB`}dbbBIV!(tiR3oj(K6x2TDq&#cVws08hRN{vpT`HhS}Y#%TN=D3K^lFETHka)UuKs(Xk8>l9$i?G((9{y5{ao=GTP-a!f$}tly-#+lysz&m%Qp-W&i4_`?kVG@%WHAl% zLO9S6Dg&kbcG*A4nuoD~4)UT3h#Dp+re7$#J_a<51A)6e97(-PZ~6qpMFFFhN(KM( z%)eg!3Dc43fX3;*MD0lV^%j90_KRPP1)9^YcK`%347`rdAQ4>+t%Kk~B{CEW*@#Pm zl80e-*Zb$HP3xAYb627$cWj$yn! zfN#GAJ};2izFRh$f?Ydr8M|SPK;d~6sPDTM^^ktdm;*|)!Cs|*-Hiv;rA%5Kj zk;y(n)BpDt`#Dt~=m~#QClC_A5WL@ifwmbP?OP{mVw3rAqmPRJxl&39nY(izzwdNs7yO3yw?A=10!hEKVX(rW4@x2I`stt7 zzAU2(_-}mvf4iBVUyfr3$EnPzBrnLS{9SkadkR?`BH8{nIbgm&A}2wmETpnLjmtAX zkqtAa|9$V!gn^1vUC<47uX$DttYYV&{_~W7KgjRvaBi?@T81)EP4y1A-)y02tR+_} zcK?2RdmZ4CAhEBKQPitO9S))Q1(Gcneh>Z5C!xvzKdcjGn$hvoeN)#);X#r+SisfG z7We*YFa#~;_|w#LEOhZQ%#vUJr6C3Qyalj|e?KWSh`$~OD+32@Y?7y^-~CFu{9GqF z$C_Qhm158ZQAkL#7+ z-2x>Tow-Q!#sAm`VBr4?Zf<|#-&_6)6X+!%8Iy1ovTn2Vk5B(EA9H7FiFRNRl_>9e zzjYqKz4-riD?cwX7Xx;$-`v_Ze(irY#Q)b(8yiu~(~{^wiy`B51tfy0L>z6otDf)e%0W$4B4frVcEd$s&}b9-SSufl2f zawfkYHj&*~Gk@(QH{;0u9fP>DMv(t^+xhFGWzZt1iZinJ&{qB5e+w?E`A2Jv)!C?_ zorghI(WrnU=Mit^|NGqiSw8=7bI$lbSgHizfatEA6aUvp#qX!~*pZCb{?S5&3ShP6 zodZI1awnvKL((>S4&cmf#wxog3Z6yB!G&mldQ_ z*vn|icu64~MDf;WLK%sWmR6QxNng`$-P`Sdgln)i%U>(HEkN9z@GWWX&5soX@sK;= zYtNHj#>3jTW1#y3*Nn40rnn$#+LcuHXFX6dLU+0mYC8EDY^qZO&X0gK6p96m?!<+h z`uh6fTd?2jf`dBw=fLB}z>!w5PWPJ8ab>OA5Xg% zls_uIiXQP{+6Y5WC{bZvKOCTBkGvMHi z0KV^I9T?&x!>&u@LU)GK9(-P}@R6Oe@H?Kokx$9(x|?;RR*hZ`8;1ic?+RgcE3(A3 z!C)PjZhq#T3gx(;>hL$0S5|C*u+rglWCmzXB}~osSF;fC6F}5=>rn4W8mS|a|2)5D z(AR3a2l0Pe{z_+%fYs5-5RH~a(LalSIF@&AYV7z~^r4W8rM0bo)o*34eqjGSxxuC# zwYigi3`+8>D)@)VWb3N3NPG}#>pjk$A!+b_!{yk*t6vVqJW$#_iuG)|G#Gx#VC+11 z=I(Bz3<02L;ix?gM)20S11q|kE9YM&U z^J$9Q>^3-6aeS@4;y-~}B=QcNh3@$moNOqu*-Bd`XK7zdG0y zvY((R8zS;mJplSwr*%ftSAFT-*iKd)(1ZDxK?IAu203t7i^J!LEx6k%IxwfhK-Sja z0cZelL7)4b(BFmuaO)4US|H8lsI zXZRanO5r5j)N9FZ&?WL6Q*_9W1_8vCR7SeRCIF7In(~&?!msVtXuR_=UQ$89J#6|2 zqX2G6fWrX4q#q5EMf@*H4km-=ZhhXk?s>M&H2@~nOOaf)@ZV23(fyt< z!l!@4SR7pVbT{!nqO(--l@o@PSbCybbEc%zA)|w1x>IQM5W`>V@~4*%`Qgv@LcwNB z^vNVf!5cszEUs2HmL=D#;*TfBs~Z_i_t6&FqQt>caDeuKFqIgW6>r>h%&A6O{e|=+ zjG3I!6OVQ)=4ebQRm-^5x_o}%H6V^V|4^g-bh}QwFqh22J|+C#eCzx}HSOvu^2>$6 zNzO)8Kf6d`rqhEwl-wXE8?ie&MCTT*=Rb~KF`Y=Jqipya!N@&ZK7`YasE?!VrX}8 zw#H0+rvL-TU~ryB$H#^Y>yoC>Pht@c+^mU3SWTt}`2|#Y``6!$`lDY!cWsHk z-3(?_sxW!R?R;JJoPCkTRO6$V!3amgtZQ-C%H-$-vBhaL%=j%iF0R18PA6nO9pMc0 zDA-sQ;F0j{SHAtr)5V@)!!1B7JQ8LqBj~G5Elk4U4hQ+BoWtB=ZGpC0;7IP0W^`w2 zTbEi@d|qU%?(!vfn^PR?#*rMSsso}lIA(5k)=LT+%;b>R-7lngy&9M8DmBvR;(u$75C=y7xZSak;vSJ&mw{B~Wz1Z?oZeux+Fg zzodaM%|YbokF1oU1>M1>&&w!-*wfXxuonji!A1UOqe}xa!Z+3p1b>E}I2=soujmLg zwgJHji=`X{GxmuzV}2Pqf;Tn%fe_v4Q-_=$`6YR}1#P^GI6;FJ7n(d3-6@gb$-zRC z8~nn}xyt5wb>b#qZ_fMTQ>&?~wo{tulA=dnKIB-)lzyjQ6P;FaV zJ>eVI)Ophve9e+pbmx&Rc1!z5p0+auoffKF9Vxiy0=yzatJAZGz_o`zr-OV;FcV=( zW2UUr-611@fw=iT-Rfgm&b$M~A@vphC2d8?cbHO)>@n2V8p(JeX!OI)ZU}PW-4E+y z`t77b_U`3X2%%8e*O&FLsFL>}Xk=I)<-Hse6y{-bDlN&1k^WRQ#cLrrPSrq0vwE#XN~;D1dPUd2Qbu)%MuR}&$i%I)<&GPm0LRPyZdMbuzjqw^=5muIJyXh|x0OfSnLJKZxzl-kDviD`teQx8Z_ zkDh5qZEbWd-NoE|cVW1%1qPPSS+wDo5bUu-jBe3X_5TggL8l+kz07y=EX&`6bb}8! z$+2+Xgj5iLDENDEpT1fcAEKr3tY7q6&cm+rJX{$8FAb`z$8 zJZP_W)Kp!bvt1(_KOKuUa;By;K(%Aka(0-oi0}?J$-9e>2y^I>Mo~txDpCRgTt-~m z-=tl$wA}FUMy(h(=~+RgZm&zd3j-X_}{T4GKNl&sM%75z;|EDbrF%& zKE)V+@nY$09GQdVewGHXr^fE-Z+r@o9YNEX`w`y=xTVuGv9$28+55m{Ei&b2-J9ia zIU}xdXo^jv5Bc}6aJ=F_2?iIpZu3O$38dO#duvM82BpAl+AAeYSBMYn}% zy|2X2qbQSKHb2`YOVOl>;pS~l<$BtZiOci0wC{0?s&$>MppJjAS~LI^;~aX_*oiu0 z9+SMjG*rYaAH9Le4d1Tv(Zl`BqL(YT@N%D&l3Jt^$B< z0PqM*1-Od-3PB=~$~1Em`7lEk->io{mMoX^cv;|CLXz6io5)gRhTgms57qKdG}szpeQcoG~CB4mIxUAEX42DF;`jbQ=?>q3zJf8AL^G`1d}* zo8x{>the6~&j&tL-j@`FpLS1v{c6NemnbBzjEetkTO?!#a=~5+>OFL zL=Hmbq_^?lt;{6v;KTW06qQQriJ7H$hv#k~ZNMo>_OT+ZERTwUbF=F9A7L8o{XNDz zi@adF(VbtaiztY}rOk(arE0i^;AM>u zXOo&or}JNHGjDm+VnIZ@#mRl@R0{Luo=j2Y+QQ-yB|kJ zWy=un_t(yk;7F(5hqcQ$6*$+mP2mIJ-F~+Zl3(@$F^nkEZp=P5UERUj;ZIe%*o$`a zcs*2aapDnd&b)1ZAKAFV({AC$K9BQgo2j*ZIlR`lrFkQpFHoZnVD@!prmgn|F}%%4 zUpO%~vCOS*qmSCElh@i76e^#jj6KZt$?Xerp>R9{jqNxNPS$yh#)~-trIx>A8<5`v zE4yX)gS;S!i~!C-pv)g(KFCudD_CF=-PWq^ld(KygS^&Sd7o5YjwnD+cL$M$31r0>@TiBA>$Ie{J5V;~Cjzt4G7&I#FFjy1*K=7pU zm|Ov>ZYBFuUDqqV?YVrzMhLP;@s;OKZ}u(ulg!oH&>QocY$YYbZDbzSb6i+cY-6YwejSqiIRap|)N%81rU=Xa{5(&0SIQTSymQaKo3L?X z1QtCS&rN5?aPFclQP@4YB!LOnY7f{PxY+4wz41hM@Ft#Lccq~iv?HDkSnXzmEr+KwuYwd3E zMxa|1`{;}Xh#2=#Z}8Q^EYNQOh|zyRtA)(*%rO+cNjWpzFSC4$$G!Vu$UEL8hW^hn z&Wt*znpL;5W?a#BGd{lJM&p(0Uv+1KD+7#im+^ckBg|FH5LJu%jicfM0=2bXX3e*v z4AKsrog*c<(+nTQQ88uAd&gfB&h|Vtc&+WGBr&rn>`Vo zP5y$i?OwG$G*~`S60;aSBi{SLJun8D2VYig3EG$sIhiMN=F?A~=ojvYy|Bla6Iw}e&rTY8osQJf>((kb9(~txYa#V+Nly6$M&}QxS)pQ zU`g;E)XABEuKOSJCGR{pJYHC#t$HdTY}Tfz;zE9BO5FX?l>=wzZb**pfgzZ?YZt35 z26aqGe09@fD?&H}ub_@pEu0w~ty)8Yg;`Nh>eeznsvRZJo~iu=`ggg*^scIrs#2$E zugr0DEtQX31*uLbFMRJAw?wc6Ib;W=8xJT<$XZqzrz6beDMO_*jiey2N6hu7L65~? zL#_@n1=c@l%9mIwi(eZEgqphBEli+n_{PKGvH0~Eo*=bekhPMa-q5gmbuLtu{SKEmT=QT}x+ARMR&{`Mto(KzxI zaSWq$J9_0PBOP&cP?pg^Gn}Xf*0gD2y@^9}{jeU=nzWCf)f*eRXpeJs8CV21r*^m( zCKrfBdeeM}I$0imyPbMIL=SEMwu?f1eT$Fq9=7IP>?!yf=V>spE6drM+iEPBxyM1 z$l^R(;9${twu^POR?L+w9_g-QHf942(6AS&$LN#PW1INY54tO!QcyOWSaW>mw z35x|%BNw62jq`I*7P!UZ+;BHgmV7*`rFyS3@LNHP?fZvyjxVFm`+15qXZ0p%qa{r1 zQ3v2JKqEcA$jDSq(W7^BX6mM0-M_#SM3`J;TAaL;3_i=n!Ll=OCn_~4)L|~?CY8FL znc(&Xvl=H9fa6Mk@KQnvJ|bw-E%U4Ro?f~>e%yL=qrA4&%7!?1#>8ts{FbFhdFh!0 zFW1Nu?(KX{3%S$3;0I$}$L7c9lGx6v?JT-@A?&xDI|deUuYWx$vYv+JUJ3F**xDQp zxsn=p*jsf9=n9zunx7ExenAg_hbO;XlbAf3E9~SUT_;h?`mN#I5bCNsjgkej%Cxp6l`&5(3i2en$D5Hy2n#o2*Fp+47s`&zgpL}-PIYJpYYP| zsRIo-fLL92#&eS~P8oCi@>PA`4f%9R4=?t+Q25ZTpx5)!mo zd>?vNU$O95#fL7KIi^OPv$ap%ONtkUU|&zoK|OS1)Qb<9#Gp=L<-q1U}hQeZ-nh0VZ@h<2zV5(ffT5*OF3lwG<$2>^%p z2yP~VZIQul>oyOnDiX<0joKnDcCBqXxT{Y8agw*^z)3qY4SL}Yyv2kn!I7T;6YmUc zt+ixvzs|Wboy8nfNyt0g{py4{zW8A5JH&DRIO4DcxjS{JI$+|tzaB>h7l;}_O<{63<#?F-nCbCF@2-AX|xc16svVXYf)(#d>&{(g@ ziJP~h_6EIM!C|XJ?XX|fI=piozcpv`+LVsv46cxYBco+pD|BKh{YG;e6VVuWf#k$W zyRCi3mO9iss|%2gn%AXW5}^i~(Xe7Lc3>eQM<9mL`VrjDTxug*roAVxddQ#qA;@Q~ z$xPl?9D`hgfpwRL+3>3_R^Obn>AVPl~cEKi@fq|zyDc3tNf+pQIvs@e`oV6?u%)?&FbiWUi^B^jkA)YOZ{ z&5_*K;Y?rX1=)KyxSN}t=Jm$bnY*?IhRX9h5BQcwgvk?C;5N6m?;8Bck&tx!{$*MI zHQNq~4eI)VNAKh)x7|C{7>R|+=|&WD1C5i}Ua!ok9@hJilSKtvIA~m|F3qDl+IJw{ zmUDyY-N9R*AV11MnNIjs>z(XaYZij-%P|}if0}a%zAonU{72drt?k9uPyZbTgd$7U zN@S~Y>K*V)Z7SCFx-i9&;zqvuu3(By$3J&#Hp?%W7nt3l(NfmUUU(`|(=<^}DLz`; z%SKbiJ88ew7_rrLQnm0GkIL&H7|8YP)bSnvYPQa)`fxQA8>PEMmA-ii;gFL2t>^oy6@#Y&3hhZUDGCV zr)FX?g>x9?&I&dI@&*h;^3?IWYvJPoR5|!N_E`sLzLEh{#dv&kyhZ(6!%mMu@&oUV zcN|~^#fvuE?cDsYOv*(lJ7y^Os+y{D+m9&FKt}LhjFzpy#7j!KTk)OKQK!>TMM+G5~^|$6eZy$=Yka=9}ne}?IDKS=4EiVl6?Ec%#o?4*lMYz3XuRCDLUSKf_2$kt^^zu9sK znCx4f1XbkqbJ?|>W=drjXt6`EY-#lEF1*PPqp@8dW~2bisE65ik)E_T?7kr2u1pHr zC_YwptUomvgoe!xnkVT`KlM$Co_IJsBT!K-Lq>^owkod z0)b^QWG2X}EOobbRN+0<79&_Jkwb1#dL6M|$LR~qr{PzNhrPVxEtO&Ib_Gy4z%pAW zA>;_7rZHu5n^EPZr=*!P4$;Bb+1zGi?XSOR*exnwQQsr{Mw<$IoQ^oLTSn$w?~`;i z&}Dg_%lO2yHci7>T0-*q(t1Qu>DC)Q6OEl^!F%+}<}LO+LR&<@T3B~AhA?>+i^#wu zk;o%}Nhb9CTn^LE$Ll+elV7imIFu$Co=CpMV?A>9ENJL0B|=aR9xSV?rM7XVf6h{o zeU?(F(qGN3_4xo+3$Eq*MeHMqa<}`~4Oz@RYaZ-bY3@8 z{$0LW^RUZNf&`SV-u&vDQ=gtQLx?hhy#E4d2ire!-*uItsAexcZ@e@0+%^}VkP5ai zdfG3)fBT`hMRB|gl(>@zbPmntNJmu#UPDqIYvg$<58J`r3fk{L@!FVtJhORPKi}zw zZ=#d*T>edb6=h*-vCLx*_0hvw0U*jhiXa|l^h#+IaJ!bAt4dOv=p9!BCler~*R_qg zx|f%Jtqm#lHkkNYWSfvX$KohUqX#zpE2oklq^_Wigi|dvFtLNarCY^e0&a?Q%9=Pm zYI)Gv;_@B~{HuSq?=zSU+7Z|LdEy)FwV3BS0OR&&z{3l2XvY#Kw?#&&uy%+(OZN@z zz(%w$h>{0R?dPl1D3gIHG7&0O&;2M!7@muJR(CLw#l`m|`Ie=pj1TYai;}TO58zxh zpFo9hn2ui{p>%H7RTi-K<(mp>fc4aBQ2a z{1QiLULdJFy!PSK+*%I#WDT4(jO1Tyz5dlJWpPc7^);hgCa_E$4{8^l?v^Rdy`X4_ zrMSM4WGIc}bmbUVT2$$cpH?et6yv`QwY5wX?E$hEs;a8irBY-th9~I=*8sFSWaYlK zEM=;DZS~h)i?;xVtPJB+a1aJR@Fifv_VMw&UaqPrEgzvKkZ!TfZUTPA ziXo{D78H)f?=$Nc*uswz%&93HL8YyEq|tJ}pslsTAFOX31?NBlQn^$yp)Czx3TdbewLkn`}-$-VuK00bFO zy%4X(4pw|K4DzlWK3rRDjEB?8-#bPhkhPDF#k!e6(BdEKqLKOIH<{k_%<(mEam=i{!W{O&j`7#Jjynpt|l?U`3s>W&g#!4~QSYAA=t9w@CU5CXRk z%M|N^0|9OikoN?o{BO&y9XS2LBl|g%7>|4t@l(bAZTRLLtY2knnn&&FJGzQ}zcMxm z5keWLhoS4C0VGEaNL2Z_!97R4!`+t{ipoGNA)|rB-t&%6X4b;A#a(zd{_YqECQOSrn$ZK#*XVu+ai0n;;tb}Ac4Kpc1wnD?q-lMW*ud+inCwud| z?qk2d@Av2P{`~r*$8pYmU-xyr=JPeK>n4syQQC($|G>rO83K~&IOAGYynHJY53~Rw z7chIwxi>|iHp6P)GSd&C;>*i_lG`#X7RSbVS(w#Cd~XFYw=^s9P@$J%3~!4Faq3KL z8^HMHsjBsKU-Z~9?r}pITLF9*fY{bx6S@-EbaLlq-6t(TC6Aoo4P+r-UTVR*FiHDT z2xF184VcJ4CpLbGYqEwBOs|JYQ@&O13Vyz5t2xM>2|x7xx>JFXHIhY42^_u zl`e2)qO-k^O;Viw=Uo)1|D<1lv!2T9%ySk|>p{rj_=%!fAk>6@K5*CR5@jE1V1ICt;>k`(JR4&a44ScKDW4K(MD zAtSMvk9E5SnJ)jiW2@{8y@bf@6thjd^Ga->e8I9;V3_5bk7qx(cnYRphYO0A;Zd3Y zE*9@seOEcHVBpj7%m#dEP@R~7j5syb_VRAGwxgnKW}gKPk``->+#x2#On*r7{yX3+ zn}jizqq*a|;?qY|ec0)mUcs&j_pb{PzB^7K|4!y4m0dU*RVsy+CC@h`nTx1h1+R0) z)Ijh1h`j^sch>W8%AJ}|6BPy@!W`(q`?A@kHqT;OCDyWAlbDSB#keewu3einAMB6Q z4sBkzGT`+NebAIvThnmwBR;XGG+b#o&#-t@fPPqylw?^(qm&Jx)7YjTRTkF#s%vgV ztSfk2nGWl{d(8k#eCQ`Xej5yV1v{^Qy?Zt)R$d;J*L#Kgri8Ty^(709s<0LV$RZ<+ zaEes#e64=Bz@t1XVX!Oy_|>UUL09BT+fEq^=?PwA;$Z$nM4>FNR+;<THK=-P^l zLbbC9JDW?f6_W8ICwZ-3l7N~rFUIoP4a;w~SBCp+Wym9re>(}G!3+d9k>uCJ(~-hI zu|n{8S0HpHcr})e{F8|(Z}D?xn+!UHDbPK1q(!U-gySof7CGcMd05NVB=P?&aw$K3 z*bEv`!0RiLiv9DD^#0j_;0+eBY8ZU6ZJSg#Czld+`|yfN3bshfV1X zT^o~UE$@PVlX_v6#7Q*f&Ld}_CJoXwwg`hgJ44+9ejs!z*Mih&d zU5uSsJeCAjU$oVW;NeqBA(=J?$50uJh#L2r+~@Z1amBNNbwuP_=YRm~CxEb$KNBg@ z?WbLA__+=YKBlIU0knSp)YJ>CSKrC>G{=Gyr2z$GU~*@}0Mbz9m2Ip}B)xrM4`;+a z_P85x_VQI~c4`ox3JB~X^Vn6`0#}QR@e4rj%Q!c0A)?zmickgC4+x+Z|GJ2wKYb-~K>39~}QubN_JgN+?L z@E*O=9=$IuKcr`K#GxZ+kR4de9@#%dd_OW6cH$kovzH5aR~;)&&Fe(+3JV=RBABq@ zjosKuf0GM-PgPW^T9JUUE$q@O$`UWsftwYxjcO8Aq^Sf;;sD7k0vC(S~U3Z)E5*c+?^V63?jZSnOhz| zspUYgoQv&GhvO(_W49Ko3@2T-=^YEr#JzeH10dw<`ILI_a_5+yY&81Htvhd@UyIsVzp&*NX6SkWLsz92GH|MR9$A66wa z(-!yT9LfE(`sZpv_t4~rQkOikWy_lds*R3(*!A+t=E^|;yzL=N0pulvMIeRKAd42E zT#E6{D*~ny2ejG~&%Cxo&@gVTA^D4|p*B-i)D_yM+Za~@5ZB=gQ+Uo~pB8^6jj|L! z&lGe2TnUAF#u{7&N8q>*aW!G?NF6NfB?Cx-ft(uy6IkRVRh8r z&31B6v7lJLN@ES1#$~RLAX$_I8(?M@V=Ctzp@9De+&2_322qFUi92nzD*AJOZ!jAf zML*5f(=cfpU7Wqd@y=NGt2gmkw0Hn(U<#ywc-lKBPGkxlV9;>GW+GnnApU&6^%@j! z5%1xQ{55yhj%%>7jh#p!{N5qw3N~uyBa%|ZQ#T*1e-Bj>KTxtB93HxxJj={HkhYoz zdcd?yQqw(IG_pH#BRZ8CEVZ8mDrc4EUBHS~J!YBkgKJ1jjK6IbG?n&0!%> z!G5!tTqn58S`S1BW-SOCbToIOL7i`Itij#j^NgM5fEsC7#KG@6t#&0C1`& z{rGTwx`gSLwqnTt-9_~PY3j574$EycU?N^D)uUq}S8JP;L|7Yze`4Uu97}+d*F9M1 zq$VTon-^{z4qK}aD{1{+v*mw_ThUIjS$?q2UP90Q9K0-BFjjv zPO}BKk<|X`P1VT`XZL_q^=+c}{N}8Nw+G^7E6niO(ar1t@{;?3R4I7VJ@#d4VCx8* zBGgadk>4ig(d(4b06e}&df)lvn2pn!kB(20!AdDFG=X3tUxaA(ED*LYIWfo8^vYskz-ZuZO3)!1%&DAF%FRB11 z{vH*Ss9Z0KG;OluFX?ib4-Dk=^+eHiXx@`bvPGOyB##=#bCTR zFE1%`pJM}ZV~hC%>>8qZSx_eY$@&!@6!y5nn#=6{3*FDpXg}QsN^u87z;$@X6dfHb zKok+q+O+1cct7y_!iJSA`;Yyq5V#SG=emJ+YaI2qUgt%#96tm*>>_$Y3G@>h$Ivt> zAZavj&)o)C;UU#Cg&%yDJ8ma-6M~Z4XO)h_PcaRwKjB7xcqa6nhxsIo&J(XzoLwm1 zmuDxg#(>wtd-^GQ5FHsL%)lzW$HBGf+nUzRw5I@U{tUIK$@4S2 zvR|DDLGP0_ko`zX@x2H<^3KeLl>ovzPsNjnhPzFA$mAJUpyHY-0w1wXb>V8fC57~Q^X$BOqTln>17C?u)G&D+QG5E>~O*X2(X8D6HSqO7@ zfmi5~(r}-;mI}@idgmCL;*uWz`=Kg&D0UJW{Op9DDO{wI2Ri?910hRG{+gNVeha-5 z84J-nNarC!35tUyEAZD+wo8%!g-u}O3b3hElk9$wqE}OpM#q_Ri<8GNoQNhpp4=eX zr7n#MY{K$(Q(QooQBg$^a_DO!`uUE8N=8z}PohKHaepnja{MZ~FC-pVJzOIm=p87z zV6l`Uy?+=+6E^Ip0{Qq;It0bem@*)esea&$BV$Nr0rG2t2!ry^Ezwd6;Lyj@ro5mH z3wZ-e347m;?JflD(Ls3SD2cpyT0?ir$nwOK$G6vy8eP*enoOVYQ)MU$|2h9k!ugX zbCR`iy`_34dv;2Q_~}L5>=VXy1s%TzNYp;xv@X%)Dso!-8Il!s?@1eluEHqZw?|18 zu{Y<{Qe*61@EYr-bgHn|b=tZzrGU4aCv@FO%Fj&);TVn5C5&)17HWD6smK`e!_IE5 z3c~xDvfb_4i1b^2UF2QQ5JgbF@D4XNDJR|AJt5z5pNDJn!dT4wTH*tSf1I`K)f(yK zsKW9q$;IcYLHQ>>+bk!DM#=J}Q(WD6k>OeE?6W-g`f3-66fWO|7*B=ayX%gRAt#`% zy$wDIyAOC@7O9ml-dj?J67ZXYpc?%tyEB}M=Vf(}VY?a*sVmqQg|USGfc_8c6Th^g zTYQcgNnc;d>IlJY_8T;^w%rzJI|3~4m4ow{JrF_QnP!9+BcP6~2?7Jk9Yx4s9AHei zlj7F4zYn)M?wI}lj#cyoU0@US2yFWLF;|Io6_`(;v>r7d#~4!L4%m+mQUE#or6aVN zzAXtXL-Kj-iJKA_UC>UzEQvC_vG<`KA)hme;HTIw8E#Yu=Ez4}Nx2=qr;W@+u}5hR zY;zmtYvFenpaq>oDaKeC!fLcfqoiT7_IVfovYXcfVFe)Mr?sYZnjNM4Nqj#@N(z?( zoL;ZYwx{r*>-yI28^rd$0?+j2e3?cn6(mhcAtu^h%jPKY#EIqtl3rEo3OdVtK<%c7 z7_9g}@(Jbs9NGf`aYacO0}}hgi~_eoCJ@Wc+apk#=V@^Oj}UTr^BlLJM9AI41{c8r z{|LhLHfWz>uU+zy9c(G22JU|u6Fh5gFD__=s-YwQ3J(OJR$cFtCqvtMHKhC?#s&vx zd@^jmdQlK$(=#60IXHxZ!sPz^luD18qo)8IO8l!1%}l0!Y9N&{-tUR%o^?F3*)sfM z?C>jO5ToWBMBvCwUyNw)V|bv=P)m=+NbYY&?;GF{i+TI^k9TIBrRmYT2k|{2i2vuy z+j~1d4TDXu0DKStuY|{HR{7+Ns0LR({4fRW3XV-5>W<0qAe4D6%yob6wzkE_N?T&$ z@GFH08wpuDuvsdrudmNwYiFAXvDmy6G(a;OF! zyJiIk8@bh#J7<2?h1}rYiH`P3X+rJeflp{Lox ze;nBp42_ct1m}c8_-(Ug(ZLz7@eb_I2Yf>F2I+DtZ?-+<9wult2~=xA?phyJ%b71e|S-7g#&pfcJsI0>)t?S?u=@chWyqf-DmxtO0Hpc30`fH z(O&&20ODUI6fJpIFp0W_uGWruLCga_o?-pDvd_muTR{#99wfR5QFzH#*RV!@V-uS?vhI$ z9!QRjnZ+MhI1#?DS9XcVrXkyQiUTOzmot~@(Xjw?iN{lF@ke{PfbQeK6|_*n(3}2c zq&-p&^A$bA!<(Y%ahQS8Ae7-@)`Va^{d4-v%VfJGixhkVkTN=;H2TxNr8x3E z+A1BowCNs*JG9svw(t>UxX1rGpZ~Of`N-6P=mAWydhPY*DdlVb-=hF}Fh>Jyq+CSW z0tzw^%slkpeayozpn~(zX3Z?R>bck9fNO3^+Ms z6L|HRjCLr8?mPWbH#{LtbLg3-3>e=C_hvav1msFl@*MPp*Qp+1M0-E%J)I2y!r^6w z-=tyixutg}-nze}o;G|r5V2z4z&&x# zDH{t02g%&K#riSSL+W4!;DRPW7*L?9VjXD`hjZBROlfxGEetIoOLfe4Z}(RLjnji! zuJ~V{=7I3Rx*^^o<0D}j4#&#!Mf8`rG6h5|iP+!a@}fN82m_-*!I5bIE-qn8$o$O% zezO7xsFPNdhs_g*C<&GXc7rYt&}D~^cmz!$2h>4Rvbw|rYwzdXXCS9=ykCOL7Ky<& z9YyB)11Tbo^AI5TFFXNPzjVbQKvM%bLI~)uld;CID~3EeW#|6oO7!Y5VFDO1c6I~d|9J7v;C~(S^(%xQ zIOk`wBjvfqGp$pwk-ww;zgQSCscAZ>&Zw9Ml1J_GWdE+8e-ZA(gghkl;4(uTpu%S2 z7T1pO0&xdpeSff>5C0@3a4h0X;Gm331V^Oyg(&tl{Cn`fmku}=1F)1Tp(>YlABucW z)I@Y>qR`j-l5ptzgLNwP0mjs>ulYqokFAwP#Q*F0?-7rj9>}}64mq$Cu^RC2PyUDb zL-GVn!~=q{juh>rr>ti2sz31LV#7VI{eK7uK8@0Cu$b^)kH-T4b#^$yhW|X1_0uQ-BX3BetWQV3K1ot@(as026;Cq9H!Ea?5MO(=H9_=&|IB8%z zqsLZYg5(~5J_hz}eVWY#%`R5y@GY*CxpY?`Alit9Bc%FgGq`_Q1t*Y?m9$sq2$^M( zSrXP-iF^zJ@tc9TU*jj)Vs~8GfkFGv;)*8NIbak6zxc7z!&DfX>`XqiODcbrkvnfz zptmTouSz=~ZsdDV?J;V1LVKole|~T*2>=t;9FRwO4dimy2p_r<|L<)z&$XvO@Gw+E z4_``yFpz^s{&$DB<49u90tq4hrh4h*9*+U{GA`0wIrJyAlB0HYM}T%s8T0oAz$@X@Oy(VbguDu?{RhufIS3?Rmk4{2@csua*b|Kdd$><5C03Zf zA~{PiR(;T4jEQNH2lu~RRGk`#dVcCTArGM_r57-xf6o(yD^SKZbvdB}F{ofNIEsjl z&?M_|8NiBh0N&-Nka+UZ@2pb?80n2z-`QifcFe8vp1p;={WJF{MQu?Se6_+9NfB&I9f3J|K-_PuG=zr zPZv=AzzDuh_V77GbVO_5cKZI1E)IA?15)4=hEf|#b%+x2gVBV=ik8a|}F!}FuN4x19NP$Dz zu8<!yIa8u5sjQCXi9Vh4I8A=vM`e{W&2L&c2eT?bQ@j-NEO zV+bw(_rZWM%LZ;GE5t?M1>4=Doc>Qh?GVEdvA5ymQ9R0A1mVnk+}TlDnkG{WroWe! zefmHjAeM)u0m+>!UZOqZ#bP($5{K*pdeDEiO-LskP|)EE_BKHnQP~EfWftt z%QOy_go~TTJe67apJ(h|5@Sjvt=duJRnwF~lU+Ek(lg7MV#MuLtK;BM-}ssTB~2 zdiG>@m_5$z3`wUsIgm)DwKNk85K7g|hyFYOct_V!U4mGmd(9KLX@2o4jGg19^&JAxIOlc_cORTI;bT}H z@lpRNy!{A8fhT-+^8sjn%ITJE{OkLHE3F0UcO(DpLGT@}jzAqICL})x5MtjV9w;_~ z4GU!0Fzv3P`GcGz#TPlc=zV8*nDmE{27(2Q!165V_6zV}=DQfSN0bH(c33ud$NmeF zr2+5nlj3sJ#FUJkg*AZG{o!Z*i~FJLWq|}LSI;nV{da-*hY{-u-+~Q%hV3jz31FJ+ zG5@cG1EiOUy&$d6JAGIDUz)f?y?C<2uzRiJFEOc=o?-wU#55p=+zTk9yx~4dlK(0bU^sb@G>!j6 zzj)09(TX$n>hP1mV)pIc!T0|+qR^5DJWXm2?JwJnst>5}_cEm!@L4<8O7qN3mxC(- z>y)Hm{_^B3bhJk*7eF;LNbvdpc4bQPu@0y{Rk^>W83C5Vzf{KsMSYcI`eTQl@;buf z(LTp^#ASa$?{Cqh@NhjCu0G3u-@t0JJ*o5c!|_AGo&3dE0LA50X*R<$6Gl1A;u9+T zrDJj7T0`HGaW$UpSeJXJnE%=6$F{>;N?kvsv@;qQ?8twt3tU1DEp_2H0TsCiC&Cf= zr{YU_u515Z5phh&@C12AtYI>xor?|x{GUY=O(|wZAQ=^Qyx*T8JmD=|#l_CcFR~Ig zai1{~!xl-JUL5cfeffXoGeQ<}M&orRb8ug6niP(}oQb3t#LLMz^$MVfafrYNsQd6E z4ht>xeeHFySVvv!W;Luwd!?8?tM~UZ162wSj|jg0cZ=M$DCwf8$uSZkRMsr#)Sd4LtjWT zU$Pl2Us)Wb9-LH^DzDmhX6SSOz1p+%+iTkoytLrP>~nW&G0_wHu4?Bqf5Dm^Q1I9Egp}#cK**rTvN?q{*RYFR=;b2;{+>$w4v#sYpJ5;G4TiPJ5 za{udIF5vI!O_d`6O}TQH67v?*EvbZbYl&d z-#NE^@ZE7fpStx0@1z%9EF7YbGWo57o(&ZYE&E!FZqnfV@`QD@YmV34@m zmfJ1d;^nm-`H|9q?UXN~XY?GMiq>1W9UINX+S?rKNC;^z{f3optmqca_6(f0`(3v&9|y46PqYWs>fbdd?I(EwFq z?bbn=ogtjP&3+8M`dV?s^CQZ&ixXlq%AR7ER;knV`-f5&bI(jT*v&}I+RZOck5Un& zi8?AqVbcb$CziyhopAg;TbM-F=@emzxK$$d7H zrOuLT&mV1uodmR22-xznjJU2WVBdXb=nWp3H_^5`Z_xEKy|lSbHI^pS(b}`aBdw>O zV^mEOjLzh%rRpZS&#!vXF%%{Uly29~+bm;N4XYu}@r{J~8W!(?&{}0g)4}q|S(|d+ zGMS>&F)19o2K2xv?0M!B*c)r}XJ%*qd>BfnXb3Rh$~Hfr`DLToA;zArw|sN(IG=p8 z|KKX)Tx+iWZ?*>Q5H(y#XdNfthoxSdy8m6WIY)QBXz>&_{zu(`+;n5w;;QJv$%(x2 zw&R14P;)y>P<6)SSqt<2Y3X6Tn!Vigxr@4Vl@^8F*k^NxLzdn8PQusMj{*tBTOwO0 z7&t{9I9pW&ZHwCL7j1qWnyqJ|W6e&@k5H46t)b^Ne~mVK$Uu@yHhV@3l5UBt9V0Z| zV$CoaH$+8&Op0AznaF%lbAicvaXD{oOn0z(vcb>6Q=~oX$?x$L-NH|EuRAWT*|de) zY+^gNC$DS=Mk(eA-yIWP-==m1-3p>G{TD7w(fOOcJ*-q!)&(;0ssb~G3-OPp$Jd8O zQPS-n{G@Wtx>I%gY{gucvx|P$-{jxQwW)7T5e5FXNdi{z7q?ge!TnEZK3hAh0_KqU zPw$!1%7o6-KH4)*45I6ERV-8W<2(FEWMbV;ZnbM2XF_f8MVBmQ^tAeL3L6aZ+ss!C z=}$INjGt(mU(#XY5O_47H&;#hK)ckn)34Rp&R?92ieg;$%P%qDy7dg+(WeKNdcv5g zkqd5=4;tH_Sqoucx?hPc^pct5j3Dr_pJ5%#T zRUX$u1|vYPpHHepKv>!6dfW*cLJXKHyv12IP=kcJujGp=^Y*+XB_=ST}jt!VX4Mz0xf z&!c=T*CL6or{)-1JRGP=FHfB8ydsgx!SKdyp`$K;b3&hVrl2zZ^65mqsW;%oO0a7u z@Vwbx{19zH&0>bPCGj(OrDR@D`!WA~a_A$4SNZr8G4Cgw`C>Uj8P+r0hEcMpW!2m+ zA+?_m8a2$lo)}dbbNUoF7OV!u2g?4QMA_W#vnh#K81*Y6saN73-SFcpQ2&Rd6fcPQ z+IViFtI|M%O?i=qX)c6jNy12s<6iB|Njlas$B^W(Q(EE%J!x^ae5o#LGv>2j+`THD zIYlU9Hc;&{ilkQ0#?J;GL*9cCF>XXV!&d78`p!8l3JEs!q-)P$SLDVxOBAgQi>Ls# zAfxN^Ri`Q~?DBqg20|X8%nw>WohcS+$KvK-hMx9sLhb0hRoWP~XJ+l7Yw4vGhoujF`ocP5?ZzWusfY4~(tJ;*@r&fr zHgAST5kjs`&(%n2#%D~-bzSvKej*UAhsb4pHx51bkuO~oBp)}sjp{0;)~P7G^p=#) zyLXzWPu8gaCcIj3vBbr0jK>z4xpmR1J4g3oXS|pYRn*-mwyvyCo5hhQbPd9Sw`Ev=qVnE26`cO`v36f)4MgKA(<*~doF=i9>fZF}4lV)BxgJ8Zfh)>Sl5+WQb)(HC|& zz9xCniZFC}oIss2TSdp^&t01yMw}}9Dile-%bNJ*XNDYbF3y)$tj-QAt+PMyMy0j# zfnOa7C|OUY$c=6n(sLPjzw#@hF`_M2l+WWsc`(^b;f01Cn+lzG)#i%*{jmbiH|_0& zUFv_@%wCA7`q(ISWlYB(`*je)Db1C4uGjPveR_Z8R=74hGecZL|H+1a>nC|z<*tJz zbqWm(zVReJ(Z%z%I z6ItO^9bPe%()}AoQ|?>6>0MXW%g@hqlGaBhzX}vHFZ|*T#Yz1+#FUH6_7Bq-W1|H5hc6P;`uRyU4eMoO*)bKj zwte$Sb+~dTYHV^#3E3>NzHNs>c{#-VsfEc-&fUUiz2$hGqjZ}P8d)-vVc6_KkQ6BS z!`tS+gQsgX5mVWxHc-WBa~-8?l_qG^2s-&N-BxD{dxEx3>ORtmj}UMC>FS@C*?;r$ z#;@)NY#19Gn;!)~{OM?t9(s5DO7y$(R|~^)(rKuO>j+szw(UQQWRvIBkHdb`U;4E? z3&NCbpD*e$!Pid0Jed`MUM5lBGmJ&w4^eaB5%49l2&MG_BlQAwq;*TBdRM)u zf_koeiQLO^?yM2!p{oeY9~WV*^8KG0+=S}60N8`C>8#(HvRSilw%?pK@@&ZLbktMf zl@VJ`6Ej3T?zMc_#L|4nClzy2r2c`sd^;!mFoCM{0B)$jUvqF3ZUBYkJ`ewE}CQ z800`#>TtP#Q%XZEnc>$As$X7EElAm`AVIv_$C3;ensVsxngxP7a6CdQHJ6^rss)d* z3HXfnB#$;zIOiU&G&A-8W><&mLhngeeoZzG){g>DxpjljEGna(%jTQxdL?j=5$}Sr zTm0WD_PN-^yh*)G-k7~8 zRTE9l*#2V9S-!fSN=DazRxQCr?zHk?4IUGUtTcdHjG>o56d`*mOD#|XQ%y4PM>U_< zs2ZLus>-*3#ZkVLEnXs*+d+pVG*XbkRUi~>VcjaJe4nn7Ymejk#_b`sTRl;F14r?@ zd7XJui%uO9v+TI;Mgid#XTkonY*tyucKsHeE~=`vdaiTPtW_i@ImI42RHUlI?l4?x zothtZBjY5tTiYyJ4i2asIT;gvz4jhB1n=$pdgW+B*?Ie-h46?Gw$z&)@8!F7u2x^P zZ4AB6u?k$oh@Q$&j=okV8Zv4wm@aCV0PW*zIc;> zrb!0LD3k))I+{TX?^sL1_4rN3NeyYh-|Ek0EB#F6+NcqI@J_{QYwq*;O~C)MBl{~~ z_`H5O+oP*q1qHw#EOT`t^Hr81n@1%`d(3yY0#(~dp%ogX8H-XGk)I58+fb!_V`NX?7Z^V%0 z#ae9<%;!Apinc0ncPN0y1O=(O7pbYWPcQ2~*mQ zkM=hn;0hO#Q@VtxvBynvre**NDs-I2NNxHPm@0q^qZZ?v$J4Ov z&Q5YTKaO4T?PNcB=wesmpZY&UOdW+{c-XgZ4s3?T_;toOUZN$Z(sg=(`VU|uiZaKt znvjjzoEex@{(wBV(hPv2LXvLErcDfciLd|Id%5QK6S zS`r>w(*OX%TtuulOVnoYsaE$#`j?nUpCGpp679er4*G5_$ypBi_itD|OTSAgF$$Nv zMYEZ#cRyd%{Cmk}TdPC_fHbdjZ+&U(cO`m~CdiGt&r)*-@`Xz(@=>`ZPKI(+X?h_q zK8!bAF8+b_Isp+OTWZUhKjGEZI{u zdi;9)%8Twsk$3mM%9I3UDl?u2&D<*^t!E~F&IQ~Z|5QF+GyH#$0}IA*0IB@?qC2Ok zo{87wbRW~{(lwtR6RCn#UWS)9x((NQh+4?B+!lYVCr22E5^-|=Xgt&JLIqjI&w7~P zozBF@fj=%fx}|;y_9NA@${s7&dI(v#2W$kEd5xt;_0wm*GlT=bxFDK_vUhYIGhVbz z1TC17G=Hi+Y;m@>O1fD(CFIV3YUFXJST~1aCdY_}y4Hn7zd?4uR0QeZB^K=PCo9~r z`{DF_2g`;#2Iszu+4+D@O2o>5Oh0kWvrBkrroCiMu@d#IKhcH(j$P zp5nwI$#%SRLG&^{vpUkq+qpUx!={3uo2+OyvBWkKM6$Mg9`8SFQ`--{BWQ$6g^n4<%=J4Z9KHo{j z9;TABvua;*fYz>DGbk7r+Duoe>r}}6er|H6MQ9H#G|TaFjvmFt-{RmV_%YI5BW#l$&_lx&*NB9m(0do zRRx`yAnN|L+&}+1=SDM>7$c+3V?h~=n`{aDU5v`*(Es&OQS7^VB->?)75i_>E@#bO)7LN@L4 zoR#*urvpGh$dz9-2i^x%AY_%dnNTYk*5y7^@l_EA)nqtvqeAQJmp6jQKq7ROIodD4 zx~-@+sI0SR%VW&3&(dbDc)PCnJq4n*bz2C$fGx{RWtp0#m8im2mI0`viel&6N@3-U zYKlft;y}rJP0Tb(AIkbWfAF-lI^*!_7{*Z!{xOgCUdY;Rux&4HQGDx0t{rVn^4J$x z!V$>ZR~Sa6*|O(J#M{pCZY=o)%#pCsfpd(FZV;|dzN@Qj<|{qR)VC?Mz(4U=I-k~0$Edx%wA($RrEprYuWvDfX{uGS9vAZ9e2YxC3N zLUZnRYf!Eb%H8sT>otcL0-IkTQ%LTlg@l za-W$i>C?@&S}wPAXtX|@?ygccw^07GkkHwa4B3A?BIbaOA-B!>U+3A+*%X*<{H~g* z6&eO$BQUop=PStS1?M`gF6Pu|r-BTx_iX8A2P-b>v^ygz&x|)CpG=F@T9@_w=C_Y$ zS4aLlTC8y%ELH5KKou`oxNNtZTn3emTb2jqgeDn`5Bn{f#9;r7vWyhE=JPt*<6Le* znI@UYxwO(v$;`%~c0Tfl$L%r|KVqLayV%Ql*~#t~6HGy<{+9eDUl1@eOC#5Ms>7^C z#&X6v()Ig=-gW*CD4*z_Cd+it%+t)LI(Oen$8~`tP%>OqbM!Y&)7#eX=>rdvq!zck z6h-8-z!4%&Hhxr%O^!<%ZpD`7&%;D*M>HRelmE6S8P-l=j^ehON*k|`qG^R$b`d)S zY8cPDM=Wf)ZdBr&iY}Q8aUMxi0j^OG!h$v0eLc@O#U4oaT@Y;eApr75PCUbqgx#C`LBT$C$yQ8}E9-8#s#Xb6eaKhvgn@i7 zV;p#a7fhZ#7RpsB4txtJv3w=l$YVSya=9=cwsH5?wBqq7fhU*wLKECTcA6D}-Q^%k zNYT<5H;tAIdC{jXN*+P&a?eilow;An?X4+FbqPy8Z?$l}i|Ms%!Dpt4Xq1d#+KZyx zSAL>a-T*gl%dLn_XO40)jgJq}5kFpXgZVz4&O|0guc_8k*!x>zNFxQpBz?H5wyWr1 zO5Si2`%uy=lv>KlcHFZNZT7949~s;3!_ST(*RY^;Lwvm7T2@LABSS>|vXN8E_8%{6 zKPKyZ{uz|}J=~JDWSJ`J(#;3IM^u+u6OVC<-d73E@O?6)J@u;4tY|P#G%coNT`QhH zF*geMt+>rwD(ajMZ+Mh#<3I`h?Nasa&;>_yxeUOem)f&KVcluoUa5lIRCyb1XPr=s zuSCJC2Pm{Zf8%7_ytqA8+&SM7;9{&1nbZ3qf7+KWC%~Cg;6jUuBx~a`RoWQ+V(0c} z%fSXW=?YLwQ73i=I2MtK^ZcBsaH4gc00o*sM#}{>(aK?`ZI;r#FJMZPdgMT>nn8 zEoc{Yc5dKH`d6-cK7~w7CF!bu*X`>m?02Vo(mR}jflPdqHwaEy@_kmA`YN{uyyc7( z%4J#W+7fSV^z!9XG)+thi3C3iB}(%~r|82>whF?Nm=LkyWl#IHr%xoYa)Ug)YkXrY zl-g`~^yqdZ|5&|-Yj?D2SKIPHD~vC`-9$2B?ZsOFb?Q0)K;=fE1|cqvAJ$JEv%N^T z{n_#bf9@)iF&uV|cx83{X0}vtpl}JOHQ-XanD;zf`eic{ZC)`O!9nP{tjV_=zunz9 z9(0{l)~64&BU0)=s3x6VktZ9O=1Vzy%k(>BZcqZ*BqiKa<3;U$_S!%ofQQ|}Uzv?4 zPllp&zvs- zwOQ%L!8>A`@iwNrehx1|!xueHw5(Gqz0i}Q{hfCmm-l1?qUIyY?GvxBz}<+F zJsALzTu%CMBE^U$S36JZFuQf?*O=1n_YJA7cq6WUrA_nLZOYn-Y`?9u?v z_D^Jy%SnnoLAh2FX4`?eIv2oRvNuZfvPRE}m=q%M&3=1SSr^fuGkq%q^+rFg9mnAX z8M%V0@y-g|C=<>fQlQuR8D`7+Ot$&R{SVz(C3>*bEl zsU+y+^FsABRL?r)p!Vr*b#Lb_!8P0EWf&VWfpl5dNf|16nC0qeYc&r5{}PdF`IdgA z$e)Ce`5KZh2GnV>WRpBy1m>bvwvrlYQ@So&fO14t{Iav$X#v-Dl0o21D{US0f|HWY zw63QKQ=9Pyx>!ayFUE{Axz6C`j~2LGV>2hkh`JJrC@|yijn|2xh%zs{Ss*YOKFK&c zQnXr}pBf76WOWY0SS*x$Uf^d_fRF@+wmR+Q0^qf323H9feKz?g>dtQcv4je<6|&kF&yi8)$qI7~^gL$syQNda zpr$G2q&q7E`3A{XVDB6`^UFT-<1f0oLFQ1ELtr>rSKXiO#Tda84q*-`#dEg1x}QL> zQAEEl>_#O2Wot*)8`vyRCOEf1so>QnsEiJV#*_qP(SC)B4tcoKQ28m;qa zpmMgo+M)FmZ%(7wqe7`L&_B0D{OI?rc>dFuS@&e1y4ZGKzH@xL1bV+3%j=a{lQYkj z-$#_F^wpXPHN4C%V!=UyRLj}LpI-7m6H2hMJ|YUj_!#PeR*8fq^!J zYQ2LMC#_iDTna28xBGH=7Sy^EdNX?9^0B;d2=C;qo0I~$y%qmTb2O~Dn8<2Ykg4)Kjt-b%h@^XTlBBu zV?3EkV&_@Ah{@#orEEc#2xqbGni%hj5x(-rQ8I@BJR8hPSdZ z9jYx{yC4A)?~@S@Jy6}~`&uXFTn7ui5(jMDRr^Nz>2r3L#e|YA{vVU9ySYynPrZc# zt;^djy}m5CwilWrVUoQLTH_`a{tv&WY~_xg>#R!8Eq?7B(~vzt=s4P&x_OH|Lr`h0 zcSP(=U!cQjkRy2W0QvfS&Xk~brIYjdSJwxHjmo%fzNc_Xva+?UgSeCu$~nEA&~aRI zEbS4?@qwzFz__!%C@?d%DC)OK|8vT`5=xai_x_G&GyYZ_(2(7@{JS?a+6O}UX>$AF z5CQ{PTSI5;d4{t|&W0OjY_GXQ=@!h=@7@*0owE?Q=xAPEg_UlLzLje4CyU9H(#$7$ zzMiraECpH-`y9%b#Wu*F#H{G+*tOd>uje@%J)u%B-_TVIC{1&;n#u0=B&+l>5j!8} zZnjFXbX~Fad4!6=hIJ{g$uj1;6=Gs7Z_|*l7wagVRb89LgDrecq`<;{d!=IKJy47i2a@R{(-}}!cJ1MM#hNo!Sgt5Z$x@pu zh*NfZS4<_5E$0Op0$lPNMGb$yYAJHNySHc6R1PFl%!mXlb6Ha6<0hr%PaiKJte5Wx>l!nL#pmS;<|7hd9#TMw24GLxeCgJP)hzP9B} z!97KIF7>(nN{3=anVJjzuCq4JAK9CNUaEq;-wRKM;@LyIq0S2{v1RW^P!I~XiL&n;c*)7vVGTtH#S~V*~{0T!s6L{5-El$XO~7CR1PK5^PtXy;gl_^WFowyB1ucs#cgVQ% zjclw%2Z>HHD!ftZ9|0R9mE;Y6rJ9#Zt)C}4sq3PD-hI^FHpz`6d#&;^$WA0>CvO{q zl&o>(&(Rl{A({eeKTN$e?30RZv1!M-?|5pCd~}x%(t2P13IsToEqe9at%+Vl^#1Kn zZz{LJTJ5H_+UB_hV0So=f!FR}12KJ7$4;h3$&!h~sk4g?!C{)RbitRVLIONwpr(vi zdwAux=mjZPZEmyMNJpXEhNr%!ZTeNj3gnt`>l>GQvsC1pEvlYVQuFoAD#w#>=JZY$ zXgEY{rC)2L7+RR8u4CYdSH1k+)Pq*kaj@K5I>wC;N?H6^^@w+P3oA3lV+gM8*J zR8C4#>_aEvh3(qIX3KtmfUsTwWJEzU)ji-gAN8mEhyFatXzdo)w;TED`XI*PgfgVY zWj4;0cZzdr8N4F1Clf*5QN*a}S=H~pLNoiv8zvokU5+!W_pF=X*(`=rphqmtUNva? z#S{gw-f8wc!SGti*f)XYcf&1bPNaWme5X!DO$-{V6m=Yjmu4%xbII*WD&gR8vX@rG ziar;-mZ;A&^=vwUKDX3KYqF@(3FYOQMT^V*%50J`7>&bfI3Gt@@T@}A#7r%Wu0J_x z3q#^!ea{{UOrPN>jvIB%tzua&rErdP(@oBqF1fsX^*YIKRI_Dwj$g`pq^^?N!g4=p zvWj@@SBxqz(M4X z(}KWkU*1fI;bpgtrGVA*52r8h0q@o~#bm>ax7<38((nv^ybc16dEeO(~ zv^3JK64Ko%-AFg{-G>s~b$8$IzhRhp=H7eqp7T5BSW1r+Nbn-Bg*kkGe$jfBHaE&2 zSHka<#g&2|O#*2nv+d<7g+?B}BnveSn5G~T*%g~R-UBuFX8_7 zkc5Wmm=Opy3nnuYf>MxA(4Rxx?NN&T1FSU^hK$v7#7$3IwRg zjXy!~4Hm<$(In{r`4b2{LsNnX=;;b2 z->f3nsuZV*&1$lBGIY!P-#5>VLm;9M>($F?Luvl^6h22#GnO^!8}|A3PPl=d!yF*h zUGwQ%lXstUFH0};`TPwRwl$wkjZkIVvI3E0teM!Lyz7HvCN=rPt~O&qJ4U3&FTc>C zYf{9BbK8w#3taco9Q8qe0bnD6Sb{EOyi1H*Rw;pJq_Fm`grMLeKm!9K=WEQ%&*%2a zi>l-Y5{w<|F7K$0g#m|J1D$tkEniSkxe~R#mxD`I3HhvJq$GH#>ayzVO=25baki=N zY=cpDL#t>wmv5Q(nMu{u7yY;$HSF8oze8HIc@2rPSm8+@!jwKci7_&hpZtYsmN#qS zsxy(3L?50CO*pUZQVv_@Tx42rUwcK0hGT+~Ov-k3%4NbrPa@px-o<{l*_zKL4UJaJ z944Ayt|NV)rnkNGAN5hT>_Wz{Eevm(2{C4$v&~dlTpx%u*6_M}dUQ|-jnpF4eB1D= zMy=2gvu3OJaQ^5!e?E@vo_&}TU+!^1562k7QSf^F2C9arFiKMN6jx9}G ztWmv(doH*NR&8Ey_a&f_7s3%lbf8KN3U=icEJ-o7vuxtY-Agl|z8&M6rJHYgPEqE$ zx|4+8*R87WcIlbClnR5l%!GLso40)KZ>Z=b&#i+%1*QvGZFGVv&_=!qPjf(HDNA5w z${b(Tc)`)oq1tp5ZNnr>Z^wuiz24oFxrp{kaO4P z*jz;lrIz~cx%w_W>CC*l5GL3ECA)8%Cfl^qbZW&7uZ@#es_CwGvt!n)v4*pZA^i( za!zGDFWyC!P(xZ)L6=+kMD`+EKZXLT%4}AfGrM(>!3&O97x`LB?WE2j|Fe<3zCq*CgO{zFKNs<_2b6!{7{*0U-j%rZ@-jQC2Lm(p~}Fe z%&1Ke!rdy_H^g}R)S;hST4iQ&@mS?7lFT#(H$LiyXtcMbU0H?%-9(+N;yF{x9aA-z zH9kETcUkI8Z!k?LV`{dDTRj_IycuLYUUh4-R|T~9*}>9jPu{-D$+y6qWYA>U6r$Hn zA-cvrX8^%*AZ(KoOw=oq;Wf~t!-lb`FHz6R`NC$p2-@wOggJ!;M-E~EZ_v59)F*Xg zgMExbML-z5nW5MoxB4Bnrm$}C_Cobn*;X}SBXcXG=IhRWUspBtoy0RrwMTuo280$x z_$TbH6{bkRgt}s)>g~sbuT;ib;0%RH-JxE}Hr%xfGSX zm~sp(++qH%$G2vWMX`C~9mcars#oVP%SR(4CG1kGOgM5jVhGG=-J-Vdbc1Zcg*nf) z7)guI3-d#irF?SS1ecd`NCD@?887yaa{-m5bVo#3Qjit_oSt&e%i-Dm=h_sm8`~JZ zq?Uh-B%4c25=yerZS|`Nv&n8_@uPLxV0ZH26E4XGHqkG*Ef3kx%IkH`Du?D(;eya- zOQM}WEk5Tx@TSY|-P{l?pU8RN4Lqo;1S;zWNwLu=;U_&mh8ru8k0uVWdgiBebF0`4 zrmj6%yE)$!SvcsDzv8uN+(m1Yuh*_qg$t;EUMvLOP?~j zB$AkS`=QniMz1>nm#eU`x=Qn$_4{pM(#?TPg2j~%T+Enn-DwA@Pq;Tm6pBxVh0R2? zU?rJYF3hFWo3waMjE!6QAQjX_3KgB1ojH4j?S}1sDLQM##f|Imh&~u!^N&JI{?>qM z*rEl1`~*BRE=ER|>r(3uRfO*pbgq^8^LoCmvx(x*w@1}h`C(l&_)JUHYlEGoe>sS4 zbC41vt5VKewzMw)J%orUXrY2v*s|aOh>$Bc;Q;-lsyEUe~Vs4PP*qWK#lQx*{+oT3mU*eYyh8s+f2pzE`=upMj&kHM*7VRiB*$bOpYdRZ?|FS6%m| zH<4F96O{Gh$N+FBI6_?rTSQ3JOnjsFkstl4I{?IC0W=+3(`J6wliA{F-!)K?lgWI4 z6{^l|Oje#C_t<&}0$;#zD&L2pOj0)}2$4lPUBrVzO$A%yq4&oNmFlrfYs2zJeAb4Y zQbt!2oB|zVPj4c=hBO&($>5&5Jyr#nv9QXyOFl^9G zLl2;jpu#Y0eqiU~E2NYbSuew)X@7hd=x+i7=Q}LWpRS z$V9-Xm`9b^VWW2~P3nt~;aC(N|9Q5BI_pBu;n#N@fCi9yCQ`Ja8)Fon!e-p*q_WPo z!nM>^U=jbL!9mVVdye~tCx_-thu%>1;GmN+ooJ4>4#daEZ)c1G$SFeImkV=IgEKK`0B;;O7ZdI>2Nojj=4VpV+4 z%fW~$;6{315XP7~?^vmrS=v>rwY>Q+&z(!lZLqbaNF_ELuKjeq|pg^P@r&Wa6z3z26o8-$x!{zy{+?sV#Y z8m9q18Gp^fe8%Z9z5U~<)-9&Vgo5W1yo(0Acvea2f%b649RM%z>MtXAHn@nL|sphkDZdN4`bTP8Ans76`>F^+?ZLY76AMc zslM#}YE=>pdfZq|Q5blAP~`;-N|K!we3T7XlnXF%2$6dU&3>zMmH9(xii;2bY^9dMA0)c6{q?0CLN9m z0-Gx`PTRqA#Dqt*L&*hhY|Extop)g8v8&dwxSIn%ocpj%aAo6O(=9N@_CCAw;W^D18IX1Bn z)0R_y`GE!DK9UqyA9TwbeD5+=nBY~@331%fm|q;{#eVlOboLFvV@3swvv!RIu#FFH z?&6howVpu;<(Qc_ypK|-pKlpXr`Y71eG*i`e~I$(OjK|xCIoXrhaWhj!VdbxPzLno zKn}MC!1rW|Yb?*KxYc6vFatkBPCwr#j!6hsOC>oiq;B!DXv?;{Jeg8iKD84{+r-re zJ!UR__B@W{%>{%pFEVbJRH&F>I5^`)qdiR(s4Ug-2!H$aR6n5~uUS(SB0I!HP|2>w z2nraFh3g=iwC4)`G-BT#fMs3)hAMj(es%^o^eGy!0}(G#IhLm{-4gGL(d@YS_0`K# zZ|V!?2aWb3)=)q8F+bc|q~Ev{M6wqKexs@e0vH+3?k;z&@32TOf8ZyQhr2ThgpvZ#|V5 zp-8^vR3kHsqSO4du^3m?imi7llRc+;IOtN0Ip)hntlV4k>%%P5&@t_ zc~k5I(qVBi=lhC3O*oGYFS+}bg4K_WF}1@VXgIximc{*sT%R*6;_ehFX+^J-CMiiZ zvh?cM_E6Xd*9v$un%MmpHZuX=J4JtTn5bbyP(sUI!#@ z-NYSw0=9(6q{VJ2yVZSFL-~l|`^6+5-?6x@eK=Tx_ZzqOTIeH*R@*9G%Q+ly#a)UD zA3}9}({7t33>zVrtPxUQF2r`TKnl?_rCQJV1;V2$rS@;!1Z7M%Q?vvzt*0Sudk@I3`ne@c zTZN>VnRgqs(`cPPlM(eaZ4ZM(eI^PpfU?E#=FYZ8%?~gyHko}t>w*%b6mf;aK=J%- z*ZmT69R~!k0&lIbb^Tlm@Xpa~&2FkLgn46=qLW)hff$!@KdVupsei%PdD|Zzgb5lC zjMz5(O-z0?TFEecOfPON7b&~-{QT=eOQoA_$`0=z+GV0@xOu#D@6A@d2Bqu7cg8kM zCL1ryvd3XvNFFl!qz%KL+% za80Q-xXqTU^});_$n?2rD#U$X&OYaLa>qt{+VKayNnM!)z8<~^1BU>Y^lg9A1sK)q>7T84EIVpZ{-d6DAXs~%Sc94`6cls1K*gGy4^tm-q> z#|zu>rYU7WEoIrwT63Z%BWZ&Ts96W312h##$h~SD%kVrr=c^r7beD3)_Yg6B*VRCW zn_H@rFB&< zXA?LMRzrcgy5g+>WbK$7RqU8@l=?jK6p*0G+TvGBo}Ks zLrGoPlpy;MAM3D>=p^7Rk9WB05xP>WYbrky>f0l|&erd!usz)LDa+HrQl47X!Y`ev7{U@} z^_jt=)@9?(8$@AwhOhrN&d~p+Ik<2p=jJ+Ze|MH~|Fp*Cr|@`;41HFa1m`zKr#62q zBTGT?Eit8G<^#xrUsNQyl&GHGebwQ1V){rJp0f&o5jn%t$X!}Bs%(PI!xBXL79OyT z640K%e)=*fJ&g%lVYFbT4%`6c-4FjXV{UB2zG3vTB!h@Ub~>wLN-DuvY3yMNw}X?C z)s~7$`FYy%o0n^@YR`7iLwKm3@N-Zu7{bkFu0R!DyEB)F>H& zn3=9jJw9%)M3%h<&pNkH>^2hy=~mG91f9=eA75jEZoe#68|6KM1dQw9sF!#p#y^5{mDcQPIt^*W}l>1kK+bE@D$ z&O+Z}o2tI|L+#9$3vI13Z2DuKxqgaLcO;b!6WXt)R#fHs$;AZbb$JZ5UY6xx$Z!?q zdkaeb%JB1*FJmNH^E1ri+O2vpxYH)r8^80hKi_HB>e{UDAm&0=v7*8XVMP>hv}iak ztyED$;eUYSC3P}5=zGLb$JC9S zlW~|}nhKG@5FUs#?6!2ON$+a+>bK2RZ~<`*@Cgx>7nihyDUEH+@au}rS0U)%Il@6_ zW1f^-E(IUH(w}Fi2gMMPu6KVnZ90b=0D={fH=&s7Tz`o-6J-PZMD2ckt1Q$(oo4#f zUe&<}<&TkeC(!oK)BZc^<0N}FzRk#@TIBVO$1m17}{_3K54W zQS6mWN4`b=LW-g8l&;J=8Ce=D$5OymyqGFbm7$Wt=FaJ>4ws?0)r<#CE5%fX49&+h zyFQ^{7VG6;2oXYDWY|BRtC&r=Jwj_eKjLMU5Mq#3&78X~r*eWq)lLf~NHIcrvrW|{ zBI{y>N2SwAy?jy$%XoSU3ls(6h~(H6mI^K~JP`|1#l%+03^-_@oa{1JDdghyjv;A{ z*oWR+e9}W~0$MLb&hbo@3n?;OBDh*055exNK@H(@QDfp~oF_2QxJGBLmN1&9w67sWW*aQVgIB5y*CdAq44@tJ#T3%vMX~y_frkGDnKrjQLD+; z(l%(#k1U+OF_dUFLeDt*1ww0B*;3mu8Na*()1tdkPbxCz;TXfNz@vLzOcTT@+>2Av zgt`pV4M8zjX`6E>9&~AWyBtrpuLz))C#Bw(>{bjU*EC+8b9BX@SwX_UJMdV5KLDLHUr2ypL%99UU5~6!^&Ac!>E-w>S)_s^JdB)nQ*mK8 zd?q>Tm;4`^HnKMv`OO5wDydQmmu>on-FQwXUYO}_-KK3zmE&>IH(r>X z`SC;63!i6{cyZ%gX3jb5>7gH}ldHMREL<&Q6;n$DW!$8Fs&3mQNoA%I3m|RQhio%G zI=wX$?_p|F(n4&6OU91?E5D2Lvo`s0&AaHdk5=?#L4Hcny!^Ducu9w5Y{GS~VLYjh zU2)b0fMksYsSJHyi$2qrC}*~)f6Qo;A-Qzk7i!Pt(1aBw*@WF&g)n*$kTRLa)*mXu zjD(trys=Amn=9IidimkjS;QdT+RCC=ke#W_22@c96_B3=EhusdB)ryE#c3`e-p&m|DCnDJy#%d9Cwzn=%-@&_T*MB1?4 zW$}5*1jA1fTZ`q_8ZDj^OZWJn-v?Xqqp?vxhlSd%82oT0lf7Iy!20aX{=d=*;x;|Y z(n;Fb^UsToeWzHIyGS*MllqyUa+T0?0C!hWUhc%zRKYD|D z{<{G4SSXdqTjx)^=ZnI$+R=l&N@piS-m&o%fD~qw!M7eJP9t0kt zyRlKRRa_xz)Nqy)loV!PvaRjc%G2ErbMtijWGPQx*K3vp731$-){GM~f}Va&M9VPu z>0k}Fm!$m;)xt>h*xzTUoLZWZd)=HByrk`mv+5#{!pZpREFsozCL^C7ob;+r#%@=gNZx}P7fLTqq~EgG5P4hsqdG-jf>7SL_N7KWk!+$x0XacJ zZqqk6#xW=Lxl)(){2;@3U2+oZGuiz^Po4TMk?}hQ7eD8U2KK=1{a1Y+YI{&1H5e&T zoMKXrUcJREJI4dOo%!?gEfFV4lLozw@%(%n-pH2aAY%##vuLe6`_o57TTjA(2THidLDl&0nr?3g_YyXV=?) zXwq5C?|fcH?z*7i3W!PO%r_BfjGsJRkuFjf%2^y=lwsV@!t&6rF=uG9cktZ~=h&#E zTtD?AO}E};54RLY9>;LWfy;@>;w?BjFQ77XgHm~Vt13`TF+!AxYy7JE^6NV=v1^hM zUh%PpdHtE_XYWT^Pp3rc`)1zz%o^^#3#k_GFVgSXm{XYt6?R#rspTY+OfdS;>ntnn zUC1pNqEV`x22K&b*Q6T#j_CUAZ-BE zbwPDRz88J4g_@@BR43s>#s+E^VbT#*p8~Rc5LP){m!4BG@&eWE6ViKD#sfFNa(zUj zJdFFY`AWy5uY@fs8i#Hw1-4F%Q5TDJao$0#qbR;$8jM+WQ!qb*N=q*?iTS&WOZTSjIt*=!=O(%2Pr^K~)<%a`FPq3#hyH0K zx)YUdITNdwE0pm=@D*8uB*ODn>lP8kGN>bEZx(Sg0;N1%5%UlZ5z4{8<+;S3r1qq> zk&{)2m=K4`0`X29dn(V-3;&u(_ywSM^hLYhuMBVD2fE(|MWeFdX|>pP;TC2grpSDg zDu=mJ0Is_^G52r_NA}Z)sSx-r^o(%ZpoeOmGV05QcKlG{%=aRMA2ds`3$wXvVDe43 z(P6Pyz0GFSxLh9Bsr~Umk+F^t0_66~GnA*0tJwfdXIwdP$!+i6DhpRbapf!N&mpMx zyMejO8gCvUoWt!Yjm=}jR&}CYAV5Ze8M;ieO}3m?2g&mC$N5S)`i3^T=G{NO>;ih^ zQWPUqk=p8|&XEf%w~Tt`=GXmsN0UTeRuE+w)pY3QWhhQ%4HSGDPD1wQtF}^wztJXt z&E!0We2aA1?sq37+FZ`%p0yoVCUlU&jcUCqtFU8?=PaW#=QNzOmZ;!9)1B$zizOZX zJ?i6;nH>qCTc6BlvCQf_sk@l(`v{i(L49qsM9!&0O9-x8PQo38*c8$Ey7u=mZ7r)rpBOo8eAYAv|l(*+$#^2jhZYjjGgx zB8I5Te9h7>n4tE*W5!oseVg0Med+oj2WL_nNfOt>wp_Rjt%YF35~?e?kO9;~fH|8t zRsOd09r=yKfEi;9mMJz4P$Q9p(j~!c5fxnPrZ4;WZiEVq5V9k1e$-dziGmuxz2JKa zn&SXpbfcWRT$pS*y2)&Iy zB;HFrw^V){)?xpgyvZn3adX8`qceImxzT9Ni8tv!iv^?Bw zj`ok>v<-a!d+&z>Th=dC7~zW0aR@NHf0SSAFV4rD1*%Fj`8%cDYVQLj65=#+Ya*I2 z%U^YqiKTmeGH1Y58^bdj%7uyS6w1&sIqj5&xZ+3;HUtH$VnfNev$o?Ng8c$`7OkoT zVNPw6e7QpzQUb!`--wt1w{jK?z9RTgQN#*>xZ!x<)*}}+H^!@vF&8~od6}W}rctqN zyCB0$1r-puHK}k&!TTsH9#M%BGt-b(xuG}d30-0)%IEC2vkcpLRMY;1X;Hrk8|c=3 z_c6hhF0ZNUW`9$!LXXDh5m|8W6_D-GsQ{py(}Tut^uenW-zWGR&xbztlphUalsY>R zvjkXRsdOGhQt^=;g%T}R%x_(A9F6DK&Iy999In(tbmSJ%tD|AsN>?GwQBD7gaCT7C zsCnhd=U#80nxO6nNDycG#u(wdmzJQif?DYEK_kHgyohtG?B9o<|14k zh$3tE%6ci=`ps+*lmf{_E;YvLCHwyAL0~ z)OfDevF`JIcC8&I30jnL%N_QCm79K648@Oh9!R`;Xz;$1hBdID->IZQM{lz-6p5?@ z0vIlGrb}J0a(eS17Lct!w47bMhHMIAMDiF7<;;9k));c4!(Z)7)t)X8+lhTI7ZSKp zxYTvcQ%s%8vw%iRp4Ar7x99c21)$3K2?Ve{+mq?Ei0^!12DSCC1T&7_%aLPj8H zUY6jlFA0mLxS?Djo>tGEUHV12{k8mPAa7gma%uq5DGzJ)=w%e7435Gc5cvb#?-0iM)j5WD*gyvl%>NaL2xy6)Qy!H&X?S5 zp@5yrt~JDK*AE5=n$_G1--CpKcTquODBP&~x1W~yqFZ-W`J!_azlU{YCzn8IJZhoq zNf#FetFN+lYjS{#E1`qcS~Zi6ao${7*`H~{AI{}B>P!HnLL*Wd6pyz@Iwni;bkh+< zF4hoz^p)X-ii{`um^~#xI&<8tgixl+U)4738|$2}wJDO!SVq^bK_vk13$P8sgRd+O{JeBD+b$L;X=o6e1q6{9noUueTU#4Ph@nkH$Ni5SbCl-YC- z8-~gDtWO4i?6@4we?NBDL0?G0^?(p6ctimPi@*7Rn1d`nxAx77Z0gJb$5Mo zTtRlKVJi)$+Hj*V=@KJAz+>Sqa;Nqd+F|d*GZ{Icju4-cl?1VXC0M;IOV=8_a`{50 zG^nG!iSXaq6#7(`0|-x=^vxTa<|F2k|3r*ycXCfO|H8ewuml-ng)tL_b7cMK_|)DJ z8!l>fx%n_u|+B))~LPV>UDp$VWcq8AgVrM6!H0ETLz&cZyJ0IsJ) z!*ke#JQOBA8XvMzT)zScZv23iEoKkmutCrhe%>+=W# zZJbK;20ofKK9f77sNOqMqrJkO?~zO_bTmDoH}Fg(vOv&>=S5=eD#|rTeE^yiYq8 z2tkvmAr?c5?GtfDp*lZ%%tN`5Xd1OXkcszx+RE=bt{#BCm+}BfRcMF;R>q?t9nMxNIHBQ;3uw*g=Oe^#O#I1LbK(QRfI5GB|^+rE)DJkuefGN z@h|7O4jC!umu3ZT7Y;sIn0hdyzq655$&2shCl_F48axTg_P6;A`WIh8zO2a_+Ulqe z+^AqETItVHV(0Vj5JoW1(E8D|({Lzg(}bS%W?`{j!%VsV$)-qddsh)iiZRQ{M`Ten zYJ~+3dL~%alAmmCdos)ZnCjtnworQ^h|}XIN~B$}S=de++#o1*6zI92+xiCAPT}^* z0BErQsu>>WyR4qg4p+bK? zTc1_X(Z>})ml-Apn(QGgMo&nyQge^p6^q*$;U8C4)$X6xGDZ25o`mh-j<`>~gO2G4 zDg3xOJy4Hxj&|fmEo2UMEkJ}1yf*(ltqgvT3}`)$P(WEoq9fIf!oMMdl1|bV_usVG0L(E!5@!kH$TtL@4e4pAsxgnl;YQmRly zn~PZo3B^ViGU61>L9VnM^5CU%)?bOpvWuUt9R1Rq9cM-i(0UX`#jC|oQ;UjXal0E* zFzDXuuXuuNx%q)&o~H%5`vZ4okT5l^gj->DaI+|BH&(h_tfV1tk184s`y|b`Sdg+Q zktb_6lcV03=laE$YXKlZ$FQ}erenjA4|{H{)vk>4v`pZ|M-W2aH6||>sWw*oz+~yYens>16)z-@jrDZVHv0YX;!B)j z1lddd>R07Ph${&+;%?_9p9#-m=(ypghJh!!$GW zO2pC~LN?X0Sw=$99}J4GHBGs9qprVluv7s(+AcSfkbh6iFTd)-ro6);{h7g&+n~3{ zg03S1UY0IWOXMF~Xi>0UbK&Mfw1#O4^$98m30yWUQdxg3PMp1`FT#t!n6(s8MErSC|(&=Lc&Zob1x%JT`;Lv zs_T0xR(dlPqlSZ?Jc)R&OG`PiE07kbM{7 zwN+q9W)p>z-KQD9B&8wZ!dvT?A*GSZg{mtP5smVon7X7I5{b3n<*eR4M6*bS!7%$~4WgdT$wC@~lqedCVtH^}w zQSF`Dp*&42UzeKs-yua@D5S+&lM&ZglovBNDt-vqmIZ=qpst_$tECX1dk`Wfna#M{ zdba5K;5_H5S7??BL(_7?xVDEM!E2z+vj=aib|sAInY3KfA6rpetX9XUC&uj0Gd>F} zHlbOfzKX72KrO2hDO4qEJ?$AbKRM)F_I)$uLg?eT?M>#+>yiFfr-*dR0-5vbRof+P zrlgyMAV7G+h9QAOPo^r}r(q?VZt7#b*?~^H0S3?uu(IzT3xso$hl{U|dUtJbDIG0z$l~x%w6qavM zP42>Fv%2GrZOFi8=my37E1oUZVxJS+3Vb}d;w>AJJ39z`vluf_BR=XQp4GWD=RNvq z3EPaXOX`al4X5>-j2W%3lEaRrx2&KLZA(;ca{k60?_|{}G{V3Ga{+|T7DMMCICTi2 zI5x&w7-|6Tq6%*Y(G4vWcW@hlIu*5Ni5Zo7C}&R*RuB~#DFLTX&;@t3KSSTAqLlF(cogGC&TRCn2v3{!!YVSQEBdr*A7-(=ay zWIk%hi6~E;K}fTxxHg3Kc1)VISi(eNUXy+9ix4x3E~e~-O78Z#3%dEaNOA9~4FH%k z!t)1dGpZ(ZYU)d|X+!K@2U|OSbK1v->~3sp_uj;7X@3xF1@jSsCtRbdeq!{3p=gSB-+lgSJ9X-M~f z6kI3S!VeC2?>LW4@A%NEEPD_O_*;dJ<->1}vx~6+eGbp;jw?MPr~ft{f&>E^?PT0| z19+}f*`L8T8&F_*2iw?(Zv1AP%dx=@a103UaYFMhXo3R&FT-%7$)K5YG;g6cpXp5q z<>MrfbO)wAXyxO$$8S&O*lDD?qXH{nVrSnkq3)uO8u>e;q&r=FJ22xHpDY5Q^}9fF74}DD#$b{~Dr?BYFI?1b}1zAPoDaR~E!p zG?%Mf-QXH1A$tYyKNs`+?;|L%F5a@Y###D+aj<$s8<-u_)Fl^jsiOV=IWqT+17g%w zvPsaaa?<}e_x&$3=Uw?BO`(L2vsYxgyV3t`<~)1?j3B1&)z=<9q|E<1a~=%l0Uq+W zI>!Dq_BfiKFz&yonnMma<}XG-O_9E^=t1;|$Uv#tpLTJmBK}Ro0INm;#xpj(2*@La zM^K|K9XJaIPxvi);WyEtnuUw8TH=QXPz&`0xg=>3C;;C_8Xk)J!OX~`B>Q!G4cLUF z6wE`IkUIX1*snX$@qPOCC)e$%)MK9W-jksXS1SVI+Ne4;9!f5FFb|CM(D3bV?qDL0 z&K<=2yUr|+jKLY~pF2r|CX5681%D2C-mgUtbCw|fNUBDBxZ}MSM=jecc(LR52N8b; zsD*A%!mxK+4tn{XJsP=AA`3|Z6)VE1H!oA?8<-yL+pV_$Fwj=t^|u-cBJXvSS7$Ft zX@!A#!bzsndkbx$i8z>Ew>y{Kb^2Mrd5Bds@4|6^ze?!%fI|G3LMOG&7JT2$^^fC2 zxP9{gTS*l^8Qko=Sp z1GLw>UhWZ&`Hd)~9uA`4<*CC*{PN=W{QGnA{NWNo4e8?k#p7@Rf3uInpvGBfn)v`9 z9+>U0dmd_4C?S7wsQy_`zkb6sn7_vpn#%Cw1Hg{CUWdCpF4=@oL-q?gyY(Xa=of>J z$|+dWZw~L6LqiWeg51~RHc(202O58WM+aLyda8dt5lLUjuUJYScmIF<{O&5@Hqh9P zpR?une=p+AQ&2>DNDJu2?^kqmw*1~#faib1_&9d}6WQb>O)**$IhS3cM!j=`f{E&1d1 zt6xN^(EtT-iHGdZk>z`51oAyc6#nkB_Y_eX!;L2cC4vcVQS>vbldTCIu?CL2U)kIB zZq*GHT|*NlEoj2IrVt0l@_57Ow{N$7)8#nua2vK@{D<W-`llF}nq z<*!0}IBK32fsYG3#+YbZ=>@?qsBw{_l>Sq#Gc%!jlIVNN#AsbS=Dn zZ}@;({zOV`oH(ziEdR^e_KuY)8aywirG)^r-lUA6JkAaOAbv;KOh*RtimOz4p1bWQ zk`GDn3#UTw{~be9(Id*i-9q-oW08FpKX?KYe+m14+!UBauNzfUeB#BCiRJLzGp~O; z&Gx7EI#|;=aII{*!I<>Zz=K(t@`q&IfAOo`1V?eGB@mAx*SV%7?_m0Jivaxc|8}VG zG&JC{X3?tiKpTI9dgA8)vVdL1L1hCdLhgiy3fOZIVLkFbWC9Fq5coob6KzR~ z)dG+29Xn4d*%OPu(N(?$B@OTIitwB|aq##vSUpDZCI_(qoyRA@wGniCq4!~uf}MvZ z&L1pse=7RHpS$lLnqn}va$|XlKsnFcg_zrRScTlnw?32L*hJQ!Bk1=8<==*=pTWfi z!vc)`@+K=}Xoz-~i%`eE!MeZIlIwt=Z?U!{06ot{#U@2jrZP#)0Ww7t!9dfeLx1>) zIz5ycF|s#1q4~PDP0+~qAQiaR;eS9UeIPP7_3D5s6-R)J^t(wh2_>OTi?E2>4yXq~ z*+WVH4Q+l#ap0ggO`gS6n|h-Nq3%t+lNY{GzNt-L8us(0le(bXC832^0ZML;t`=^C z_WIW=#9ws=+yr=Dx>Zvj7-1-;uyyQv3w|Snn?I2UK&eRK+<=;c9DVrTyAF~R7r@Qs zEe5^lv5)@*!5l8{uj|;?U5G7UvKR8H4uu5xx=g4w&;+HUEBNjAAu|>ffFuW95}=Q{N~?`GyqD&>2?{nv<4(c?FJPTHehSBba-Tkb)t~`%ppnF%d~KGc z|2=YX?1wZ}Bi2cMnVRggY!3@3_DFdSr{TYn28^2rs`rE2qrGYKd+I<_1kQBq5dYEU zQ)vN7jkaH|`I+@3M|Jj=;8?(RRxv=E3ZF;U0X0f(99#P(4MgP!nLQc|%%9${;M z)aq=nzAtrL@zbM0Kj1Ux0r632FIM6w*q{A;sJ;%(=n>L(5>UCFStR%LF=aa9(ioIuSt9 zPCe5Ve+uv2b1QSTHmWnKDpvL$hoHuJ!g6HK5WD^TSzztP7=2^m;BXb`o`A{j{Dvf; z?)lBTak1*Rh33JNTr^`e**g-9tr3gF%Avt=q0~-4c2)9^21+v`?C3j#@)LE z+Lw0yJK=v>GL-`)2~OwufV`MtjQ>77f4Is6H|$W_w5Wi6I?HIA>rCf(Ot=2i#qjX( zp%)QG2q_SR;JibB>}|+m5ZrM3Z_oT9iv@f-^zl~*d;lE<+y>+IL6^}3{{5}SM@WGu zNTJJBUiryBxSyWp{)^>4$_Aji8H9Hi($QTZ6#+KWV0}XSpuYTIZ+{iWV>IyrTo|Rc zr%O8VunxPY?mzLLIBu8v2!5a*b#a5C2w#H&to(mhCQsqjP>SP{cH|M*>7lIt_WTYx z1bTrR54J&3aN|fAaE}20uFLjSo+Jb01NhU@iyvcvZvFxhRiNtbqmw z$}l6Ig4Muo9E}N5@trXH#RmOU*bN1+?TuwO2IahaUc}s{DLw*0?TT~z91lDTVTeN_ z;-Pq^UzFnkVSn-4cHcgF@ud0_ir2{of)On=SN7`6;6o@O`(_7}Mn!qcMzY@6Jx40u z6Bqu`RREdlZ~i2IYJPhob-uC(tfxT-0s33uamj%a`Ms6GC!owlsJ>PVm~9%Np4*q9 zY23vaUm0xh|MD9_krimKWpDD5KjINgtUJ%RzvwNe=;nG@`JcH6qGLn?hw~EU?7un* zlGCu>JPZ9aYGL4sPT~`PdxD~HpQOKv6qug*XBUq+1c+($*S{hAfn4v}hG`5YjTa!I zaY3$qpIZBe5pfOH^%iURRif>5_oOWGB!ZEtwXp74Re*MA&;>@R_Zi0i23XRoUG zEVAf=^T_ad`UnsPbV;ubW0Gp(p!j*`RZD;R>xuoNBPXc>&c15OcGqZtCB$L<*aLJQ z^4{Mhp9YW}`8d|q$$8>S2_Jo`xZYsFQ0Vy5($d1IKFE;r=)5P|K)C9}dl)uXkqC~x zJEz{P6X|j5;ma9ZUEK@!FfN$1@jC>JCrII-8u;OJT3_o0R@O(~%4lQirxdnXr8vyyY9}FkkDL)*?$Zcw^0_~PB-G?lGJd_~j~6|~|CK7e z1Pw=>*E3ibCEyL;tW`|2@529LA7(Lhe$D4KIGmtAY6BqB#n3f8i}0q-mYhGbv|R&% zf%4$ji}!C9^@R@yiz!`4PSoM2c>q#(>V(CiYTSR3#|XIpCt-GfXB9OA%(h&v$5edx8@~ zA7THi>~6voj@+_$dEXrT&d+_$n8AdOeg zM{V)kw`~Wae2fAQPh(*p!~*s=+DkCr{)O9aWvN?hZyM_13F;IAc3bxUMtFD{doTWu zDR#-kLY`{3I_vX*JEuscSNjO61$=LmW?-1?%o)N^$!p4;HQB<`B%{hcmp^F?Y4O2MXi6;M8K^5 zEo(<4(c>~Twb4eZ96OcN*RV;^JwhHz=$6|;hc@=0< zvL|a*%b&HE2Qnr?Wa59W72KcjQ}>hahvIllkVPRqzGUFHdn7|>-~NtX{-P1YFC3Ro zpJxnwI(n2IYSgGqNBseOv@?Xdjb)N}uwLwjUo&KXPo<*@4A4W(8)#Cxh1OuL-q5}J z$wMMx9a}e6z?+gVZo4+WrSGeDO${7av?!w_Pb_A55_vBeoR=}JvO!~GJkI$f4Gsul z3{;N?H|TaZf#Ac9!z>~$S|D@lIBT)*cE*rI2p%E!nFpxJJ0%+>=WxmsINqVrB|i;1 zQ1S;1v8!VgF90)y$6fLBbjAZpz9-1GXMP*@wz#sWV)9#kOn-sRe(i~no{+_Z=LbR& zp$Jm|jv68m{S09sb@tnzdTkud`aox7hzv|(UQv*{_BnEjEfHvMA;bob6z|r~vIKjF zX4-kOR57HMqf`lVXjPL|pG0}nAx;uF^ zz`cajPwJ{Yw|VO%sf?;J7>5~H@|~8qDV;EWH?O}-Z^Q%|opj|grF~2IAL*j_UBYUC zs09Rci>>V|S_C;8C#S+!WUeYcw7fb)DT$|pjun5}`@y0j7(TiU)B9;8Ba&dBHaz(o z1QqR`+Fp401OfD=*MwNrLP+q?(w>l%P@sxl@4g=Lb@vELd@x#ydkQwTrzqpZTm;gN z3K9^XzpB%6Tr8$PDFUVqu_T@Ou;O;2{peV?ncwm0ICEe>L_ut5CVrK(ZTd#4#A>l8 z=G6?6$g^fZl;}BKR*bPycio&kk^yZ;g5(b@2fQ9b9bs^k2_eDgf-Pba23lhHNjxp7 zK@AF{Bh}Fy_5X|pgp=pQ&<`1yz`H~Y9R`F8+Le!mPA&ba6qcl@paZA^B7rw`NB0jL zyQ?~eZZe=TZy0=Amt+20K-*Tk1zWDiUalt$rf$B8a>hVq;A>_qQfvn8YjDa3 zvY-CN9%cA0NY)3UU*X7D=KVsgqZ#90v&TNWtec@2v6Jy(1I0_XrO4592JHoc46I>lxP|6>yEH8B+4k&je3_4%d5}X|1_U& z0b|9v5G-4CJ3lrtz2tOrL6?qzlEmlb6tX6OO1sXA838~H(~hY)k0d{BJJ2#&3?w?5 zQ?G>gKGKXQ$+5phwO6oZA*E%?AhC}$%axJKm9%d;yiE7l_{$l(`+QU3tncC`f`rwQ zVas+YJAw9uehJ9iuC{-4Gu6c(3E?NU>Ky5m~08fDWVCNmW61Ri|sj*7HRzdK&+s(P$p%kywb}kg@qd1ZjTvn zkC}~~V0u%6ea8A@JGPRh*!6}xhz%=@J~stx?+3cnsP=uTRn^Ijk!at32`+O`gTwCC z$~=-EflEa2q(x@;Lf{4p@*BmgfkK&ZkXJp69Vc-w;dIPdjy4`(fEjO8hkso*3uNNi zo|-y~5`?nzBWbhUFu6Z?%2NDEv?KB7L7Y+jambopr%t>fg26K|ly|ZN@EPt%p7$EMkC^OlqO_PkLY)j(6{S+`5Q!fo!U9^+ znWE2iW6VW2L}SoTmIUF6-a@I9REzXE4!qw}}(`BZC=jhP;>4r^foMU(kbxiA3zkA*WWyGU^h*z>~*hl9)9i zBV*g084dE_I!WY&F@jMdPCS$(6ve?ud<&>oYNY*vO zD16K}7WA$lyiqCUC@5Q1o@~I)U-MP}SNgDVN zC(q5{Eu#@N8(3{CQTU2WD|VuEkq_(`w=YWukKjSHYiBNUG*0ej9~CZ4+{i7b6Sst+ zuuw}n1<7sq^*o$j%3kZ?bKQ z7Kq#fNUeD7veMESkDEo2yztT9&)>8EHZBs5>1J_5MEDm6u*>S-u7@xLr^jj;ws8Md zXn>#Y9f+_jzHkt8Vajx{RZ9ROCA10+W;Vvq83{ppWgx%z77%aKgM}V1KLd##NDkZb zOypJcJ^DaO%Zw07fHWzt7*Ki^@_PFK_2Z@35uY5^N%Ff}&T0f8G#nEhi24*k#V_1Y zL0kf8O^6NrY)3osfAnaJ3jmDrkN5WLrs3Eb~b~QPt`0+-* zxETV#&3>2DT%C1kBGsR1Pax-Tijo=l}z5bYM*6vsJF4cX%ShgKuv)@fSS*0J(}^SfazEcw(`AYu*z7 zpG}f{W8@+0`2wGZXK|=v=VyFOqg5*RAFlwnN#;U#s@2I?wlv2S{?r28_=&axA#HBX zJ{3=``&Tq?3Eb|bfS~Xbs-C)mJ&>|r+a*2Aik?6F#o-fy{sITTEmDyRiJk z&PsKVU}SYGqk~7M)!+Pc-olA0FE}c95z8PkW{>$e8JUpWm%ks&OEJrJ%r#7?jkMO0 zsjDlaER``&X0*J8&~tn5#3Ao-Et*SGz5DKOn5XCf_1!#j>817nE*r-cTg;{<_KSbd zZP7O*hs0k>N_CM%*oo{<|48jLw<-f5$OoJ)h>`6l_&orr4!|zmit&~#}x+P`R*vej2x{*Q7 z?ihe$aLeOSJ+{0BswYoYjScv3uHn_@NU9%Zrd!6bB8m;CrN+u@#u{@brRE$$H}POr z@Zi4y43Hg=I?emM5k;trcmYGXEBQtY5<1#GuJLroFJ37v+*AXPJP{>vlZ^VY^AB3? z36dD&rCU2lNu5HmmUE|6Qd(pYmO|0Hj#O6T4BXyTngWERaha19MWNQF#S}&jw)kp# z%a9Q1-Am;=L&&AOJz1F`B5Ok}1nPbo!q4BfQRAZD z%iM&Ykk@+@!P-(<7;|p{;=3DRxnm(<$>uzX>dYPeuBcJnx?NvZa~W$7pXzOF?vEI4 zX%x4$BO#6`#@gOi-Sgt1F-$9zb0}*?IkQvuWJF(Qr(C5Q_)TG)NoEBR_Ir_&IFeWL zTdD}Sa_(J2g*xx|_bHsbfs(Xwo`Fb(ppL@r<=lD#Pf5k+|H{$(?TsTL&U-+HpkOGx z!PSB6w{fanKTmaDaFmst5Gnab(kJ$!GiK4r!aWQyaX&ZlP7>1D;$Y_FaeFO9V~gs; zkK@xpL1e)KmEKe&HOk3R>^JLdq2yhAYT{bFAePn#_Aj=N5KrCzG1ZtvzlRrD;H_`~#J-vKk^s^m+s$1ynO;YgL1!ReG)>7UX#|&xDPW}!Q z;ukh-M#Gx2-0vMX`tnp6iDuXN&s-pP5&6OvD_nSYX!bH0>8y3Vo% zPJg<96>h_!b-X2ZSi6r1Nz8MhtG@X_b>&lmFM_cyL4&Vs9bqD$-C8#F1R$&ee_DqW z@!!J=7-g}sB6L95UE6#8hi!?o+UU|gfoPC0_~aL_8e>W}ivz{Bjb7muP~{_Zy!!WDPCK{9p_UC# zxHb3>5Ia~2l>Id2dzQw%rugzVt9+YokrzB=f4+aq0=KLDyNllOzqcT`5Rv^Ogyq8L zD zz!^T>g76oeaSPb7-<#ZWNAbCWZN)RCU005)ii0#Bre(*O%p6ZYub~`dT72lMLGA`x zfuPnw!YbV=Tls1-$F(Y+%^EjwQybJm5cjs@RznGMnF&Vsf4CQ6b@QYlCc3ZZUwG_o zM<<3COs>{*C!uAYq~UwKIZkN?4n!*skUV;|$v7s**OQ;+7yO?4uN)Q_3Gm3?_P5uT zaZuxp0d#j8$bkoCVtYOIgeoQl6dMcwDqHyr(K3`t=fyDupQoKXLKU^dmfVNGz|n6Hef+f@O?V_SZHY7&j8(HT zy3)_wKEtixE#T0}cOEbYz9VW_|7R#|LzcC&lAV@IQYnL*0Ukd#yjf#DrQCZ=K#m{c z?kxxU4kU^$z`lHXkg)S0j+7Qtwp>^~sHG+H33{OX+2^2<4oR0>?`vjb83QYa;=>xs za!6;~F9~K)kfBzJzKw01ENJp%UXI^Jsx)M)#u(A3(t=VB+)==mdK` z-PlCvSo9X4E&)^*lynr|i1fy;x!zlXK{7W`gg#U$HW{Q=kuW+Mb0Mj$@GFK}^J4tU8ybIoQD;nfIxaOxo5kP^ zu1wTNq9%Nc;2bV4e2foXYX-R!c&o*mD!uo3h1)C%Y|o~%4_amg_o!BfP$#S5JZ`lw zzPx+iw~d!k^5MuuFl)2Z{1%WdtG|5MtfBRWA_@eCEz|?Tj+~x5+Bj_N=M;tze9mX# zLA%rMRsAh)a+&ILM$7TT*@WW1Js{JfHv-Kl`J@Ct82k-ZIIkxl?Bis!DIkxB;27HvBQ#DdO@*Q z4Z$)%4%HFCrMOj?D?YmUhYLyhR|g$Lcdf3#UP}h^RV26Kw_~tv?crx3rfBOc`KlU8 z#uO%Rr6M?GT$CQHJE2Y2kFL&PR7P06!Rt~(D7?mJbdNpl)b}Vcl3$M}sUW-{2M1u{ zwdk>!JDv;SD|=>uglr5K2;WRe_>u85ziO#Jkevjyu9@iC$>}g!qoC!Xx~I#(dZid+ z1%}ZY5zZY+JDhACQ+_vg&ozFC-lLf~RT1`S=RtgfT3;BNn|?6q^zu2MsKz~B(Z>CP zB@u_b%sCmPgr^#o)UB3&-lryYAYH1xD>N;J4W*?1?To=2H>Aa1(M|*g;wvBr9=AyN`HlCcl%?iXgZl!(d0hI_KSU08 z{FN^B26JSOSBxQLA7b0EZ;@95r+8f`yPUvquq5TH>ZaYVJcK8{W{I?*Cu9<|O zk#vA390Sq0!@y=DGyDj{F334E%?3Y!a%!tp_ABw1@^(~&NK3ZE642tXg5a!7jQ-80} z!z}Mw%e&`c(7t32g(uk7$ZHC&t1CRB*~x+Oq)}nd_oRmZ)B>7`!a~jeOY>D{$dg)q$XeK011b#)F+15x{=GR4ti(oOnRz);B1AI z15{n0Oj+=!qSBCXfTo@|#;7zOm~Kth>T$GnUg`Z8`iDe{R3OZeG~~RmKr+>+ z`K4pmZJfPSHX9xkR5wHWGblWDf_KRwr*@}W5Kp7kcIZ5xn{GZ8_+Y=hyNm#&AJhT0 zr0w%J3yhEUwr{zm=zeTh%gt+zEeVgBOOH&PMvO;`%$|xMk{%BQxqoYatYfP?V^G|f zL$pxrE;;+axW-r!XpYAw%<1}mE;!HRf=16*4vM|?RXwD$?sgD#hX=D{jDA;VLKl34 z1I{1^r#0A^XAmlLH_qNxRRjEZ!rYwOge(?^ZwR_$Ko#dG#ulQhe%0S2;(bXd%m_!T z(Js5oG-_32n>y1}oilq(t7}A$u0eWVhF$Fyyg_6wdbbXANZZ;Vt zAhi+{ljbrkZpl?HfS7Qe@^aif+9#>8a2K8BndmT@`BaAv8xpzaD$Bum2YOS|A`Bh! zRz3H)V8dOe+W&Ah3(Br-8HI5>(F#Zy*+XL#=2pSGIn+N}zvZ5Nu6Fo%szV9lCxJ2QNGbp@sZ!<_gtA?B0 zcY~Hm*aCEY<#v+3N$hPTybeBq{m4qM%y~*qZ{bcUY5-OH(iBKZJgCu=c7fF6*@|1+ zqPU~}{YRsl%jP-CNpltw(3z5AN|MBP%TpcbOKN)J-GO4Z4Qk&4N8?4b>5YZxbT7|` zUap_Q(uDd5{Zz8w=eekHrO+~}FFOo?7<7=FZkFpE0BZ_@xJo*(QQT)S0&*y9VK7o- zi_fIWa`?r5_PJst|G8)GP-QM9jLKIA2VesO%w)Q8%Mq!5#ZU@;g}%Zp;B>uJlQ=pb z!fX4$kr65?gP_?)1dm-!JX(#RGF8aWFD~q#yB1Z=CH=>=&Ej!Ea zA<|aOC%dIdNK|XM)|Sk7_`HIfHz7?^35o~Cwp*Ofl~k=}JPJ^GQsFXj(QC3q%PXt) zs%M+-3`cpfLPuNtmR7I7mV@obWvv~hmWJLh95Rq|sF22MiHrvpd%Jwmmg%yy=-l#a zXkUmggQdbqU15Zc1!6T=b5YIEd}(@pK04jSp5PCwe1MKtiAQ=epF0Pfj^q*zalSA@ zb=e%}52Q%Oq;{A@DAlxcX8naCeKk};(;P90qpo}$A7dKS6PS15bB5NWG|!QSCSDp^ zSk;al&VUz!JAW80vx}bQ*0>r*XfrZ= zf4cgo_k(_~-u7?MGitKE;`|msMA3Gj1$g()MEm^syD-jzsCD?>Q$ee3&zisckNRT_Rr%O)*d>w6UJ zvla2oxX9{Qi9aqdQYV-gX{{F+SLiyJ+z~{f7XM9D-?5^YI$Pt1hM{nol$bD#hchnx zQ{_~`ix0)iGUNE9g@ZTPTux;ZYqGZ>@_+5U$j81H4_hoN;4lfioK1SL19r*ZO4SOnX7s9oY@s2iBCq*ZYc^ z@~fpc8*)Y!G&`q!ADSeFSqJo%}tyiJMvr0O?2yGd)B zTVirYb7)))d@>*@(BN^(RU^dqyox+r9#6iEo^QXPHgC^A&JMf=an9v3%b$Ntb!eMKD&v*$s>=APv_MxjvilGyiRK~YiIdhBj4mKAT;N@Rnup1T(6I9r zC`MBCcvjcYN6B8uW`J>@gfIx9o+!@LyIq5CnM#qbTo{RI$$<;Qk72Dkd7sIc#j&AS z-Y_2K%50Z}U1AR7Bf6a~c2-_r{AZN1@o?{?j>vD`)VfRB8-Hi-1q5k|oV!EuzP#C% z4p{|us-t_Bal#3YmJ?gsP{K9p2J*ioS!@sW>e_04;04YdrWmeY`4KC4Q&#hd&5B}S z9u9Af43}e=!onr-{XMk}(6Mhx9eR`EERm7Pn~0UM6~fANLD;IH(oCH!HY=)Dc zF;ot8CapF;?@~vfTS-81@hXCQ3X)E zwmaQH3;8a17lIFoj+pedXQP7+e&h@_^$g;ZIA5WQM;xt;%z%h`%qPwgyFl-6z6l$+ zE_svy7G(_CHjx)y@eV4RAZY9>NC_pdSwjkJeGIIN8ymd@vl_YE1W(D!dgYywv;m7(`I(ohkk;C$)P3pTNx?XIg$FnIF=JUU zG0!4HCb*O%*eDD5V=@<68?4d%!%x9l-0Zi3PdjsOPMuKhidt5uVVadbzVVNL$Ff*$ zzwv~Hxim!lz4s=@s*M<);@K#}5Zs zKOb8|sTnux&On4RQN@zk#|z^et#AzJ^pyGs2Xg)RXkv3cv$kyNm26Z(O1(U~Dqp@%OGeJphR%L1dT#wi5mQsw| zo@0HVN-8$iv}lGPSYqE=pNQemR(@s1+uXFjXPhqI*}S{NQ?Xs?fxI_WlwhY7YN#we zFK!APo8H=4g=|q2EQmA(p3Az!y4UB#GyH<@C-n1@9^1-OPooOM7uvK zBOglt5&!gldb~ZsGrQ%LNLH39_ZeCt$&N(F8T3-!-pnnM!l*PeEKczb8LhEhcHT$B z;d5|yGZuN2a6t;YzyOr49gXj*vb2O=6K9eH}bWWL5y zt%H-z)>rPb480{~4Ymz465h_%{CT-P&OUKmP14Jf-q+CA)<-B-9&@(oP$)K~@LZm( zW0J{z=H+dBf*u~wInFtv3{zL;s8@{WwECRb_s@AhFg`bxgXN5MI?=va8nIsIAJiLa zuBoSEtRr64drXtMg?A@k*CjOHd7_r|%f)S*Kc)E?q}E3(Z~W7I#uB|zsj{^K{jOqX z<0=k)$6B#ZDPCSViLCtvlcOcTd%4mpZmxA2+W2io+{M9S^3^D{xJ!!lJ?h=>^KJ9} zW>VqdwAAr42To8X&Nv42c#|~|CsxD7I{#|6_$;okW$}euzax*js+g!dcob&DFz&~A zj1RGI?MaJV#Qn8C8Gc5Q`sppyZA`@COs?)7eNIIPyb!voXV1Dzp?UC1K{}Ehu2+CY za;hkEK6(hp?RjFeZq`T(!a_*J9JdDs;+^9_*~HuntHo=XjKn;Lxsa+n)j70|a%jQb z;QU}qS(t({M@bo`tlTw!eI#w5E?6TlJkT{v8DA8F4x}of+@d3bJe?UUDS0cIEBSeD z@Xo+hh3OhqH)zJaZDh_T;6~N!B~DR6RJ4@nqKgF}obCqj9F+!CK>hQ^wAJ zh!&z7bjgimWp1<*x%WXj$;rms-f4Xwl{(nG)?5}=tNZR4tf43v z_%qv#71_`AAV{P&<5$D2zcX(v`#+~~uZQR+d8*Djg9kuU86J%JS+WSEp>E z`gq$?GP#+NSzP_YgzCj;6PIt=Zx&Gv+Ga+xsBI{TsO2%;a|near#PxbkywV@caA;| zx1!4E@?_r=x`t$==D9>Sw_UNRAGDc_%7XzQr<1ekKFIdKdWC$Ryk$e$5ttJM6%2x3 zput@~vhr9-Q3=Pb>Ck=a5;yW>Al5J;GRZLcSeapCJMoZV#C=!#$+GE6{?0=G9bWqo zsUioPzCI4*WrxIGDbXX)auSv&Lb{L35u4@$xHt16w0>qW1^f;lem6(u2S^5JDFMxJ z+u&O6LsAx9Q%(WBN7-S_<~`r5jagxrS& z=bQ`9Gs!}`14X#8-27w)Ew9`~D-RjNAsK%)^JZbcdpp3Ua~MzaT%K6ySa$|>8^E8tA*eY1*~3}l(GE?j0=US?fl*|6;TG-Jl`C~SW~Kl}+1*~YEU(l$g@Bt0Hf z#>?XWz{@M+593vo@d3k2tKx4&^9b_!V);f2W%M#W{qSw2FviLAO(;T*iv}UANiZ;m zq)a{CcfrxoasEazlv>(QVqe^?Dip{m@y>q@&ed*c5?73GY|!uG&bj~{35byE>phb_ z+9iCDkZ_%v92`u@yv-n{rA?c}?e2zCO^i+*v(_39rw@iE{LFgq?gPoLJ@?T`)U(Ze z_o>KC^lFN*u#NWbisDo7qMO4%wSdIxEHf>$4(%%OBY(H-6zU5}>Wk|y1UjHz*F1X! zd92k(jZ^e3ha6AV+Wz_!;+QyBYboGjBzwDslPpX<$pYV}vYKk4Y3XyydeUSWlbB2_ z(66UUwzL1dOhsQEW}iXlxui9F%6n+tKJJlt+n~&~-=3l-`lC2j&aH44e(i&S^P;e$ z70(oER?K*Lnv5#E8&C+JJx&Ie%ZN9okn@I#7^^U^0A9Y!YnO^Vm!?U2w?QUO&{CN? zGB|*}7cOC$8|GTlMURZLdMZ9Nx8 zED)PRd##0AP`6h^2`*2@4}`=2rsl3^j-++8_I*u4Ty(7*wlLa%&w?^Hj-P)L+Re&c zU)=>%rR=@^<9M^yI47hW>^R81E*mOcv6*bkcg+{wHb{1IEoVfn#O2Y^H-9)$7W2mE z(8=_7CL=%BmF!n^Hv+oJFYTVI>+0#rMDMIK>3mGo)I+e#Q8!DIq?ys!{TaMT9u59@ z*6i_h-HBw$&B29Rms-9G)+=h#a}&yJabG~C`_+SYlyl8KBn$VWg3H{?mg$0uWyvYR z$nkGM9zGKMGje;bXQD=is3=j=?-@FFPBGH0?b8-8I^RyO=*yRM3&- zs2r_$ad9h}=q6)EFeA~Cs*Gh-C%vYNgmIi1YBx&J4DFkOcga+*wUN515syhq#-xD% z@Odk!g-lpk7MM-a+5U##JecY__2X+Rd3%JqQ`^;l8gO4Ew0seRd6jBQbEbs5s-yxI z!auvhmz7|Qk*X|+)8=f#C}?xZTeLUjY*&3_yzPVb$REmGqN6(pflAZz2C@vwd{dD0 z_O#><7;`=GScf8b)>AHxKAIhv#HZrS^(@Q_e5rMHgomH2EMA@}iB2Y8u|x&auS^6m z-RB>^JB)veH|g_Cte<2jgC?KJx=vq~XL*2I(sFvPGBLxhzm`-mJ|cRh!Z!#tDb{Bc zRy67v*7g}Q+jBxrpzvq!=H55LG0#Ka@+m4`Aw;N&9>g*x?;v+n2lvR?yP;M`S8&-_ z9@{e0@8^PoT2Qbz_0@=Z_Nzk#!lK@^)1CRP;k;<7`4nP^GEy^#cqGi5z`}V^=5GAI zG`9-^o-errZkE1tI8*-nUCXqJ^;=`9z^gGShN%POfDyy7(Gz2}LV=d~6Zz)b`YGC- zNUU)Mc2@heO(#3}IMKkxNiEv5va07FFLjgYAg_m){`M1%BusJ*SF2APx3}RZjSemX z$DzRpwjt|_<*x^e433uBw^vE|A7-r6p$t2da5RtmWx3;LSImnFcPJJazNtJa-dACd#@xMvb%&@h1rtT8jsO{O!D zWw)y2w|r$c|AY`Z5B_Oa-|&`=6WAR4K^zr|jr@M*`k^J`)uxKKuac+RWE_gt!|~w+ zXGiDBkJs@~sK3$u%JDT00qP&wD%EKKr|EV@za(y4j=KO*qg7kZUH(Xi4DoSC1;Lu> zZiR85eJCYO>+`>-_Yh+JdHW4sp2d1afuP`kO%RqCmG8U3VZx;P?fU`}NBffo{{j;f z|7U7W0-DR+8$yUtk@nni|O% z4K0Vyn|EU zcaK!LlhcfehS`lsKL%gKclrf{b~%m3MgE8)t{E^?Be&LVqLXqZtfww;V`sZ;CD{98 zB0&l)QU!htaO>S|?~wW8)4r#inp9o||6}r-X~V|Q#N&ERg4nL^+K)SdZs=tHUbP2B z`3np`ZFq*A$T+7EKH6ZYhg|!BVOLMLL%AMqRt4dcJZ>PBMCza?NX7<4T~oZEw)Uk%7twUwKZI&0|QyF;MI+DdqPs1CTTf|El>3# zfsK?I@o2=PiqLK+P6eFV)wLvNGPHZK&q!n(MX&DaJdV5nDdP;}raucIb66L<6yxj1 zThoQvFxztsh}Uu9ZcoilejUnR^#vVa*?W3l2=Pm<4j5DNFg(=m}x!MqSIL9432wiv5tY0 z23Z~YW0hMR0nT$^wsl0{*aWEIY8}+M9J|t2r}S*+DIJ7#2)BotEEq7p{#ZAe9f0z< zxC+LGv7FUUun-Q8hq}tR* zE>BMHYGjZ~@Hv>}Ds;HH>Na9sT+0o?fqlUcyPE?1!lCyYIjuXCGnt?inimrq z*bzhEafquS&9BEpCW2sb*`H!p-o?Qt@F`=H%c)h-);~;1k?e}~&iieh8v#mGNi*80 z_IIxvUmG9o;753%{O}vgDfj3m85LvRkBrR)|2ygR51Re&Y#ZM(1RzmD+YIu**5cUB zm+J#GJPpZ?TSYsP^b^1KFq)M#GB5BY)`2@G-zZB199e!UiGFkI_zPKU?%S^^6{>Ai zbi@3y>j6!?_JpLigIIETT_JMjpptHEmF9x?LHes9h^jjJo|-A~6P1mrlZ|Q8z&0P` zjA32Fo!S7TgrMf1z~8#C_dzciGBfaeLBK`h+BdR@8;bHsQyZWDi3H%0{xwE!JSR9yfAW@OeIorMl)Nzhv0s)SWnQE{ z2=qLBd%Zm&5dXk9uHSI(?vT&jq4?Z{CYgu}pX?yVC|EY7?yluGBtH@@KMi1}*17*} zmK-wEsnXTWX2jWE$?5C6Z>+I41EizY5~yn|0JE5p^CS03<*u)9sEDYe=G!7mBNjq& zDk0E2A5Ggf;Lvb2YcAO#_Fl`|^jH~<2JSd6MX=&|xEFRE88KMW3B$Ye2Q(~h*sl8< z5Zhj!<+FD8L`bL8F$Bx_Pgw5NFQcz9|6|;mq5@##;=cjQumCiSK=@=wTD%vG!;Pxm zX1qMJwIn1g=UCt$ZOU<>79-3LN;wWbT%a+BUhBx+^0#&lYhX4?W;~4N*I@9rwk~bM zc$XeZCSO8Me$jpV+Ac=$!*g0~!s1+xv&vV*;51{rrvCOAXv4N`~|NT7`62-;+=YH&Yxa6FCXPAa95MHz0eAQYFVQ4!$Q5INZZb3RE- zG#~OSDL=^mpj2Fv>0}s*cO|mSHPqVt8@<0T&7r4+h)3Z+go#!5E|=6p8BJZG8ex~r zGKHn@uTZ)Ey_6s}co)vsM*8Fhh}53pKpGiJ7#mm%W6i-MQB$@RN7B|m1teR86ytCl zj2hncdiNn!NzPhano09g3^6eTU9u4IUi`PX?oWb+<7TC=?<%m&7bJ|s zyVR88EL+;!uoQA*}Y#3@;#9rw}Ze+2NhQxSnO4<9|k0^H~e`s7%+%3+bC4#&@O9 zRxRlpkpP&{Kh}nr-*NE7ZTATgKF;{VBvg99OM>`cwXm9Mx3#w{uRM@^Zy^b(8o~kY zO3=<`e)deQzTVcs@g*zaKRKJ;ZuLgeP7+s?j8IgB36D!47xmZpi1?Xh8ATUx!X z!}^?LKQT4np(A&`obx@0TvkS2?tyohcPfB~CVuwRzwYPUZ)CzHRb=Ss`j0LBc%ti% z`;_Tsbi#e+Yeo2o%7L3dwE$Pt+FWFLR!vC2Nn%x;+L+bJ{Fx2~JsI9w;A}tAt>^w| ze`mdRkLjftWdS+e2+$FwH%Tx5#F_Rl-PS+O(l)B0bjXSHqPTjP*9V)HJ|#Ru&vB?4 zNc}t6k3BF`O_&mWl3IYHukQ#3eM@h#HgFcZo<{tJR&7TCwx!5oU25h#g zmOXeJr&tNAy%o425g~~ChV7-=Uz{D7UH0+WnA!g2#oGsvjb5;S9XJ?S|N2nm;Hi6F z?|e!d!N|-F3r+9oL(cq*$z6nm0IS+r`t1O8S%JJty+7PK1d$l>J9vkip!^V){_&e23A696!B~at}Rz z;uuR{mDLeRMj{7M{MbMot|HCu>f(op@wHLB|FGLc{FW3G?l+CpyO^(17apD~W`}i& zWQ7jH*Tm3+yDCCYbv};K`Q>}+gCSs(3ZPHa=AkX0{B6Hy;vUjEEdx6}o=q-M0NYh8 zjbZ=SI5)@pT_xYdy)G>+{hSBpdb5tjEPc-V!>alRWZoRCdQdZSbX(iz%|GB&)2THW zBV`QEOcSnutK2eOI;ZbBb(9mlnsYf(DQuxcdlM(j_p?&VS5gzSbZ?AWw3w1YSMMU7d)N#f zCW~3>nWEpF=XQ@;q{uHrV8}Bpv<&N_*~RvbUAsgZu+!2+poTp*5W97urTt?-RfLgv~lxlprSb%s4%b*0-e+@%3K-7!r>+4l0d+dbN~3C z;PCHx;wyqA@dd`>nEi;}*hkQQ`g!%Qo*@y$%2$$udny@mG283SdA?=6hJ;`lXa(yh z-|^rDxU7M?Lbm}1l381rB6wh+LCgtf^ZRmg;>=0nrMOP>>wJ1va6zwRQ|Dj5`I zvDFBdJL~-vc+yOo5zhSjg()lQQ+dk+u@kq?pqOXVE!ow6YVI?Z0X{pOI=DWkf@$5v z`UXs0`P#M}=8i<|c901bN%5X=dwX!F@r(E;IOsdey%b*+Dz z#Qz&-H3bCDfy3HB!1_OKgDs0?2PqDavb~x*!<}8PE(2}lQf|o;!s=>+*sW1V=&DCI zJ7nST9D40zH=A|KfINaTM0q;2bW}O}pG*%Kcofc?M|S#Qe~7|rl>VN2Xpe17UTiHy zOfHp&q&P>eda_gNX#GZLDuxw%)vs)p077^wjXQxY@&7HHnF9!D;MmxtPT!rPvDa2B zh`SgH`GQSuTfH_;)zfhL05JjBROoPiT8=6$M=WA$CJ=UYQCW5CbMfBGMg@(L?@HWk z&xdw)avH`jmjlGT@bF4;8zVJ@OM_23uWV)wpCtBky{|6(Aq}1yad{bUzx-y>wJEqv zP1`&F!G)OLFT(pwchiZt_sRpG{dqQ^dWL#H`l*Zq>Q!V6TfgDq;c;`WLb&u;4E1xV z7QeL;F|y+XsMRS6@`)5Nhk>SkxXj~$+((;5(Vs)%)I5=_gjmLh}K(nQ)!BvuY2s9SI)w9<=nhc>CQK9tVpjO4kQ&gzjec#MT<()4jS_;A7XJ zJUUUFu&hR$MXo&!WWS3du0q*$fowg3qkqJrZ3LvV!1||_Kz02uvwnp+r!=jD z=2q|a6qT>K8qP^!tFl<$$e|G4VZA*kv=8ULw*OQV7w!`J;pr;%?T`191qsl&iF^Gm z%~3h;wMudMQ!dd`ZcEPsyK8c6`58f}fq~*!VD?5+OdCLOJT%Pvz?Hb{ICz8^&oXp# z!eID0AT!->y1(t8ldWul&^4m^lUb6rBZ|K|0Mgv@I1=q;fAK%7o7DDRRQMnJlwr5GWKILY%|Co>Y{*ZlHc=L-2WN!fABG3QzzLnaL_KXSIiC{pl>vi}1a~Tep|uu z)_1W(3@LY{>ghV;G=$iS#1qUREEN+t=1e$sj9{ldQ=}pp!3YT`ejy!q|3k|(pI)ef zMB=`C&Q(2X(`-iILL>RHhjaM*cICK;ncfJF)O$Yp*&7w4zH&h}d3#2K zSM5zZ+mOw8C7D8lvihE;qCPzG=FfYiI)N^+%h1Reepe2EG!C9}WUH*%bgCREfY5%q z{l6;}LA2l0H~C*XaeK-*ogp)N>fZNW{R~Rh<~cLi-wO(pjP)PSQ7Y9Z*+=GQ(LJC;Z{Y(ml4vfS zxSrOWm%LcOjP+3jEwh3Ww(htb1DlFQq@s;d@W%arPiz5i(lWnXXGuFGSTy8OlzROC z?z;Y&O#<@~RgO1$4`wm6`Wx}Qc6*q3Thm(VI=g_ydLWPB5@^**% zbY72SH+0Ey$KWcShMf7WUBwiAki{iFNHRHi;$l5A+&>6E+%<+fSx7xv2%O7fOu2An z!e}}6caQOgMyJlI>vEFzxcdYp!%O~eSNd49L1d9i$A+)(Qurx-ECAamUoy0ZnNgvv zXGGKPYK4D)0{%@ZV<=~D_4Gs0Ax~0GYQ$Vbt8TIhmX=NY3}{e3zt@GzHX6kL8}<3$ab92}RoUQ3jY{!H6GM53vgd;X9zy@@ zu>4K_ zqaHA}+AFHn-XW!E#Yt#q2b2viqhjdk|Kl zH{xF&I-x~<7jG5idk|RcPe}6HcM(fSVoqr>a@R=F0K!)`)aH0VoPEdNZ z(i1uaD$(eVyDG+TdI7EVlaK3^2m|Dr>n10sqcLx?7-a+}^>qzizW#D|fLqh~xKsBl zT|n-bdKWvNxuMD?yIs7v#CK;vBuhcm?+6R7J=VQ3;n8T^?{30Pv@<{k@xT0t_i^t} zEnt(TN6Y0F{OcHjsRFkxs7fd}n!I#D+9<@}w#WLO)cNgxo83pz8n+h;z!W#gYKh;U z7#^!VIu|jmo2qrV<*Jv+v`2fA^0MI5NE2yh7j~h6JhRFVVI4%)`$g(~6Ig%5knoE` zO^fwU2fXJzVRju!rylPM1_a%%oE=L&90F*T0^%e{k?YRH${0BD3-#s@zm1+&KE*+; zmS7vlx!`~G@L?l94!~-5HhwB<1r-}oc&YTpW4P-;$T+Z{3H{{whIwdK8*?hI9Ejha zZuQU5fV%|br$+N1$Giu1?@=p_iy3IMx_JrB>Ng7P2^B1#Y2Kt#vYJwe*D~uX*W%a|G63qYkN8)}ugIl^^Oj_EZy}kFPoI{}6Mvw;^H5Z#4 zl9;~dc5g*9U#A|2w@Gt?EBuT=i1J&kBN*YSWA?EV_pf6oYDNrYz2x`3r6Re?8qQXp z>jYlN$?LeBsxB0-N3YGWAgm=F%E})cjOiMyPsz59`1Wa0%pXNIpei?T`ZNsQ&OZ7F z95sCri#3KXX`8x`y2wHO`?p>1YH5BsW?HSD1X>WcC%@lR+5U5bKUdV{)<6!3dbiq~ zhqzRj)SGRZwsD&fU>`*Yk>3s|PyeNzy(9$Hto+afW-GT*jDVyNKE(x{Vf8M&zO+E-+EpB}fx@+91r$-dZXbdDxAJXml-ML3^;kQ{q&&VaoIfbI56Z~Qh zKkt!ZjBJiF+*AVrVdz`WM$IvM&oDYX#H?87NR+t9AutX+9we~%v+(@$*Z=R%He*4p zZHZO!s`mYzofm?F6uXMdIcNKeO*k&}rI%p_KlhyWi|?1XlAt{|zqc^*ljn^)pKIw0 zh5D;i9&^(&gofNrv+RmpoXfW7ONG_Ly3(B9C!KT^6^|0@ffJz**;EaNYl!Q~ zWt~IN2x%s}62A)@-4A3uesuM;oV{z^*Z6M#;ibwaDbX5DrCE)>_?yxD$!|9bLYx(A znXu0%32QM5p~4=`OLY}$A)K!=yF0apF#Ot3EVtDtxrsMp&99M$bckCj4herMe)^ez zAC+D}s*zZHR)+RRn7nf$0rZ#?bLs`h&)4t!7 z(-I{tHl*8q18peLf9$4q(z6pHd<3kuH|{>ATnPp%O0u|2q*0~c9JdLsdz)7ql0#2{ zjxhG7e}}Jw_x4h7v3$j?Ggef+{TksOC=1&gu7a8*UmsT|f)F@MZkWczMdXchV?&b3 z(ADbdu=ljlinV8f6VXr?Lng^SO$r#r@=^e+AdqUrv7vn(6Sg+6ynTj-$Q)~&elObf z(TH)ESc~I~ckk2ubg9sQ*^$3f&4BZc-XH%H{GZd^Sf?te|n5= zi#wo%(bVOI~)uQl#O`Y#J~X7R~i2keTo|}SLx>d!ur+Xu=Q$4-y1@Zj|0& zc@pk#yM7dmB8)V)G_zp&l`WfY)-Qhc zH1M55Q9!H|rdS)l=ZmJ>W;=R7zt9y!CYs>9QSjq82=F!ITmBXu;dmL&CA0W<$DvF6 zh@(Jtts@F596NOT+B84$wq9>q;_)@M7%3I!f~70*AQ`xFhj@Kr*y%XSC?=xrxBsyz-4P`aQX(b%9K>on@5LpnmGVWUvOT^(z_&$X8)E5Oy~|zlEZ&vFuzw@NG)!&8vJr1?xr55gsVjUkqr4BVPfG`715DuJ{2y)QP33MM z9LRSJ=R$Gh@XzT=q%R?JcEJn7mwMPk0cvrPb9!BmDncz!dQv9?N0V}{)FXrJ2$yR# z@ZUCM@OPkmr+)vV3Tw7Iw%>?KTh?+qbqV=|Hzl@PA6}QsufQ%>%^5b78WNR>(@LZq zrAITr5gTVZ)P2Wz-B@K^2)LdNayB+y2Ke-sD$V3Jf|_i z*kik#uFxF*v%}$kHY;x+B;uSzyMqa*=+FW2j^Xw*iy4cTZD0kCQP2M$XI}yj_4fXs zX^1FW)>J~42%)kY6(L1PmMn$rOCnL2$!!s`W?!bV3uO&qx=FS)S<4=YNXov1|M|>j z#;xzYzkm1jx(&_ebIy6T^E~h8InP-%XzVYv)$we1e*-r>ckblU!pndyC!-ZQ6yJg< z74rc$f8Kle!<(i-Bn~lMl<=D-mn|m~w z`!euHI=uW{Pb6oDWA64X3Fkv6K;+yP+{&is{qlPj6{_{`{~ETK^VNDSJUrWYr7y^$ z=eZ-hj*r~zf2c$dh%m;0gnI(Jm z^{w1-%Ql7}-_2T=a89q-X7Ollxl0@(hI)$Q&e%kwrB>@Eb-Xv+bPNL;Hc@Of#D2(! z`NN)t1Yd3oeap%@)_tkWJik?J>iamj4bB&S~v z;EDTmW@INUeg#pTX!7s;sWAD?G_bXLfPU%w&XS=vos~BC`ur6Wx%tI+d!jx00DQFi zbND^*sO5(2bxhACLkD8Uiq^rp^i0mE_ZLAH=n3_n&sl7^9Pgrp z-(`{R6A9+=mVt_N`)0LxUzKZ>y5XQ8)121inbfLf^^f5~NuT3Y#wI}Jh406h5cfJG z@2ko`1Bbr>bE7!r|0}e3QnAX4$Y!9@*f=LLmRhFlWI-zq=DL`|DzV;sXe$n5 zUf*gFH-XJtE3QuL_uiH3i_bHX8F2ENi0g2v+u_LY7afido`nWeb(FqyuiRZ1!5eL# zQeA(sYs5eqdeGwxA|d+BK-3APY{fBWS5dkJJBPLVClnvqU+H&-aE;G_yF(X&2f?v2 z8X65}o_VWi`DU)L*@WKV`9OQVxVK&%q&AKGeG9ej%|~t=`v^RN&G*m$GMn%ByKcpQ zv0xe`SugiL-|40NiEj9l%FMCh2-eQ8J+7-?E;FjcsdUc2;CO*n9IRd`of7e%vmHL; z>g!nc<50dxhHvvoVv`lVsYWo=dvM1-Zy*d-4-l{ukbyKG-}cV(WUIyB*&~Hy zZ5<&bM;P6(zO%6#@V;c`dR6CGOvg2@fzdF}>9(!xktvZ1;*AyzH7@O41e@hUU3&{*YUGxMUD2 z$O%u$YcWE+s@31UGCK9tcUddBF9MyYSPvpcTgBNdM;-K6vkZN&iTxSC{b>evN*n?q z99N(+g=bfN-vI#Na4Lo*t!?hGTb4m;-GPh8|W1ZS7)x{)oQgU zYq0p+fxCw@O&cO5!FytgO+w<@c@dofOD?$7B_KRLrqW+hp51@ZG5^*_$x5{S%Hfb| zgLxfCZve8j?l&pBvct^*r>l~uP#{GdePwN;`atMn%39(18y{^Pb!M-D1DhA+R_b;7 zrtGH;g5W`k^Mf2Q@X*ipJLqKVaO8R)`b)$6gO{JJ+_Ky|Z(a@S?vev3UT5fKuMUNT zob6xaN_qd>mu2VlOtf_JTv4%5nSh&s&i1m;rG7axd99EAS2}@Py3#uQ_uuqVB}6P> za-|O#YClK075ZwoW61XcV-Fdii5i)sKuNq9Q*Ji?ltC69NFzt3qkIaTWvGD69(+pqMUmoxTihVT>PW+Rg5u&$A%~u_7EzM0Y?Ne4hHqsw_6WzbexQ4wpr@v?ZdF{kuXfB8x zY6F+!zgc>2()6U#f106Yr zm*k|^V)oRyuWZaocx_&`(_62B%XuARrF~}$Np~MW*LPtS-MX!%#_&N0Z zUUkyLT!5yC>6xNPBrx8dWK`NP|6L zb`2^4&Zh|#GoBiwwK{zxA<@jv_Rz`G$&pjp$h zxmWPwsYx#!iaENKN5!Xz2dXP7tzMbH)erKl-0|nk*|jn?`RmC$?i`*tyjS!H9Q-ma z5>1Cn$6{lVH@o;G#jXi#mZjtUSnPPNe&Ft=h?4Vivn0I-A258hnfyc{#}v?6pWiED zwH2JJ5WEAI!s7qP%1_y=x zoXfuNlMp!q!Sd;QRzHD2a^W_}0`*C`4~4G2?U(tYG%KzA7$;?bG-y!}HJ7{ZE5;Q# zu$%L~jWB(1ZR7s|Kddvww903ZKkM zl|-R}TTo-Q(kr%I`^{uk1rNX$xTnAUn$dCXalun(^d11EqRlniaWLfYjyjAs%}%Kw z;W8R~1NK#Jihze(4}i(IVxP_7^b{g#(LXM<0gXhtHfuD*A|w#0^YIe_4UMwLVQLWt zio%mG=F$#utz{;wSP62eWVs}~aUXjYH!V-Po6^z|R$68AqxCqhhVpvfe3jH_b%l8K zi^=avz^s6C4S2CrXS2Jatjh0PC68}_eo{0H6`G3EoV%$PIJppC(-<-Z!UXcUWVE;s z>&iU2v5a>bQ4e9H!$ZWqp_e1po_@O5Pp@iHXUum#WjL75+M)%mwNF++l@ADs>f%VT z#Dqr0lu7Qfv4n$>VR-ddeT@Fk z<@!NM*nNNw*7|n8lg|NY%()@uopek_4pbpRSP`ITN#%*$xY)`J^e@wBk);w)x^p8g zi7F(Op~zY?E4iRDp2d=HmoVtP6~;Nb`DFi=7mPeu@W0W`OGst(hQBACzBds7D7auS!$mA-XY!~|Y{?3V}9 zqf3J0yVx}RwHuT^wsFXJIpF|lpa(l-uR5jr^!kq^4VQCNJ%x%Ap6>-4A&AcF9-MRe zki5dGzEE!*ZCBmHtWtc9>Zuq1E}VcrC+_9?;_z0`ZdwXEN3@v+l8cmUjJMz14MUG8eK zx71#h$mwxUjo#L1uqotD0%*!S<7yuleX7ovNN|O&Rvlue{{#H{uWjwD77KBgko^uft4^TcRI?bwrss*hTKSCuCuw8!Aw7S?y zW%7lp@i#n>ngsgBwFEY1GQ+zBb+u;c`@}|q`sdp9LTEkC@oqlOCDY(oZFmX0&s*xL z!Kshp6QT@20Vsw3)e>@L>Fx$Kc=|wDcg3ZuG;wkwC#zJce_m*Ix!M0riEWeQORz`g zbt+6aA9Jh4^({Ei>D@~s3Xj%8SmEpg$Pi!sCd>ga=4g;ANH(&8@D(V8OEL1^ioe>& z8Jr<&`I6HbU7ocCkSKn2>U3*Nl@+@vxWp-esct~48Wjxp81{R+5{F7RZ?Ft5(gT+s z3DB(1{ONgL%O_l){VSorqV=wgoYL_E6-zb9Z*X8x7aTAM$A z*0<1nG9=01U4{E#UuK@fOhE(DM`Aay1ekyHiC;B3QFqLIz8*JmTP^jk$OzUxuD2KG zW@w>I(!iBlh$r|xH;>Mm*GhLlL*iPh<3z`99Lqt6p{{>k-DJsZ63ZLx_mVHmAEwp? ze6J^(tgk>87FZ_%r|&RlDn4NylGXM;vBFIiOa65Z_{;dK&1hNDf2H* z?U{tG3u`jLEYdnbeS!?Bci{(PNARNf!KF-T{TOu`#MOzevr=xYGaOay=)i1%%%k$? z?eJN$^$i^t_hPwF!H0m~W5vi7VCP^YVI)Ase`8$|htc-u=3daY( z)y3fpYd=+@gZ#hPLhvzXg)-wBIK$%9U3TT$5D&ugC=9BR09S>;Lz)tM=IVIH~(@NZ-SSLp`Gyy2eOX;K)mauabMWMTdBd)0wKVG#)KahX&>*I z482^O)jWoC@xZr@E}>dMaa`l${>5gff>&AU)uP08W4Q@U9d5a`AHnXpX`|*qKR0_& zjB*S&g5+fsJk%P^pJnk0n#cW35q?J>Ot2G|NSF-~3ISW&Q&nD7K20v2(X=P7b=NK) zgB_nT0Pbqahqmms={H4VYz^16d=qd9M&PFa&!f~0<~>%Y=6dZI6q$0F%tK(Vdmp{I zq)Z9}?)~eA1zL8A*i$iTFe4EZ|0^0>c$nD9d@MythFB;v8YgZLjhs&Ns_xaST30|OZr84~%m%)Gil=(nin3Ir~EmmMN!W*Wk7d_os8gYBu z)+Di-^*dF-d;~?&G6c>+YY`L+oBS0C%v~)k!e2K{2X zFXnK^ICf6HbKEfr6G3Yy2)b6Y$bx&puE-a*vT*8HzER_NKyN;q?OyM?Vko!#@trty zkRb8rJ_|i@aLmSxo)fB68)6fN>?CMPe-{JNc(;Aev@xG}8ykmYg*+;7eF*Tei{c7E zDg-5<$tBQ$JflZrlr6HDZ7)6I&VYLo`d$I89p#P2;JSz29rcP*Su>%|7_n#)fD=z3N}}!zh}o1g_>Xj~>|+L6zjUOmkLwvsil+D$St*sjqZ) z-n7q@Q<7lYMz!~R7&j@8UN?Z`ZVN_n39)j0;mMKq$Me3cnJ@RY|3n4z3m~kaCUS?y z+gF^;PhW4HbJ(Ttufvsfj|4aqR2wulriWS->;>n0aH~Pp(_1qM1jwD*XX|D?TVtsM z+9O4jiu4cG-H=S+vr>O45aJL^m`E|6wm&r(hBAuQ$K>APhjBv@kt^hELYW02ljH$#~-B4rpj za7K>j!DOU1MWN83bw+F|Pj&QraAmVt1hSl@CWK`X;?Cd$8^!V&z4Lpj6_`4ErP+Z6 z&%4`*SOA|P9#4eal8^M@WGy6Ob~_-91Z;HNI!=V}2YVU?zh%8C)1*Q%zLG;$uQz#^ zA^CG`Dg(r|mGA%>-4WDhOCv0HC;Us% z-T=PYl%I4wS&ZnJ9P|69xX6V`*@z@ej8bE% z+oV}p1?4xnw6k1O+I(mprS{&U+Pq=gwtTjX93NTwO5vLnr7>!cEidOqMIOBGgVD%q z2Z`Zx@tM0=nKV*j5eM%X!KhA=lNj>|y3oe@3~yT`Xs*KpwhB;2^O^qVui|Gu{L--z zst=%^nZ$?%fcj|T^`9UKq(TKdTL(Yxi%NeM=CgbJCl4*gUdYUamk$8x|S0KJ>B2Q%6@??b_t-xALGVsYp6ScL>z%j)xX$z;Qdw@ zoG+|8F-qcf;rUZs=j(Y-dM4oS?vT7fB}7gj$V%g6@GvM|b1MT~dLuf0nCWOF^Ni0u zGiR&gj8oQQdxk&B50S~T0d^1cA9jFAgFaqMuuBdH^Z?{_g#%ooq1k@#Gkr1cQc3oc zyz_4Q0~&s{^E;i4pRi{ipn^Fw>oSsU4baO7m4%=!4mdW1mRb3k6Ku5=+Ikg zx#$3O^{nIE5i*$vaZmv+mxW=6voJ-!eBi^(^_QmBz}=J;&s}6f4d9#rr2e`oG=iam zH3f`7r#c+liCJAvT+4=AMeZ)Nqh&+!W;4PZmO6{(FiK-7ju)MnSd2Hqtj0b)N3Z) zlSC%tR%)~xb?naeZBF07b&53K_t=bpwNCA86c%du{Qk^TeW4)!bQ;t>oy*$T%n1)c4oZSSJHRXI&rnH~U%#utyW~mv?C}KS zWDRVcFdJ8zYJxoyTx12!4Rw|F^E86Sl0`P6%^(LfZlmM1Z+RB73DrTD37uzNUoEg% zzR(@V=vi2tV;FCE@By&hpL)J#=oLnUd{@Nf{OJD z9ojNy!y&8z;`s}ZEs0p7zCY6(I}o1AF!lK>zhf*XK5+;jQfSKSO_h?N^mVPg7U^jtg@s}|!@!6)rtO;_& zu8xmlY>a+ra7Tgh`v!DCludKvYTX4wJsN3ztTRcgHi-V@cY zj3ewFBp#{7Q$#Ww@kR4~yqD;rAd8HY1NCj+m(mV$pp6LDovIxy3lV^zajD(Lo5dda zj(HIo74`BlgTSm_M~(<;-zgCrjXU(4$zoy16=eOo`Hq}>8$~3H-4b2K*j9Zf-%qFC z^t*3pN@rh5vau(FCDy%`!$+0R`>jt4+6o{EmeAdTB+^3k$fEtM5OIN8XG|e15FU~$ zrTJ=q(#m5Q@x|0R+Du%z5mRgtLivc(K4hwS%8jZ_kkyq*>qAC$R#I&4ne$ZROMh(u z7jUjCMi%F0%wAvWb6;8hb*5ALe%PjP&{44^^G#!?1qL5auTKixySBJ$ZFFkx+ec}! zlS8XZwP0dx^t+0ZCGOKqloABkNF2p5H3QRtGJZ5ja5D8=M1gCWOcB12TU#3e71$EZ zGxPySvHbhLgQ*2DAvQh82H9uCrkT(I>6HaZl!yoEg9~d8f zJELw?r$kLIO#_HX0wK@{TE00`=bG+7_33*-)(+ei@Km_nP=~1Us9Q|BknM|-8k>xd zT*|Y)2#%;3XbjxVBS28W{9*67KpBM1@bL!B8Jcj=rY~D$^XXF}!x|Oz0qmUL47aT4 zZoo^35yp)@KbET}9|9-^J+#i6R^sv0 zTGI2O<8ef?)PlIAr=}wY>LIy{FD~AaUj)Z%$`6D|G*8_!DK*XV1ZcM^Z;C2zURg&p z4t+g0yi4q#uoN!R@u+09OHs}vqMA=8H_uLf%IR;NkM5SdjouEgWs?QY!Sg$Hki*$b z&r>$l&|Lbg6M;MG0?Dy4Z7_?@GuwB`foE+Vsq#NI6l1Ly>VQFalxm^iL7&4bz4aUU zubUccKH$TOS(30Ayg4LTSDf!o5*v~ekNdTWP?%iDV}t$vQC3km9)FIr!`8{NvC2v{ z42R#r3~m)LN;O$t?452m*%)-e%SF&<|0#^Pt`wwSH}XrKcB++pPzEaGg`ya9GFLz` z+OI+;*ePZ=P)Ru0k|jVevAD@}?M1Vbp2dG4S!vSzX~iC5gfH)iWZ#W+wbQFPLX>+H+>=%R= ziMLilFS9g(H4yqllHN_pD%`J0^ym_kboh3B_uxSX#oEk?q-$7>(~0-Kct#b0J1ou= z)Z=Tlz$23Pkhp$4>z3#@SMoy6Jd&N@btN%FCug{3Mfh9oW(;nsvwVz@(YnQYFj7X# z9G^=E*)zL;-V+UN>36JY|{GYLuXr(I@J->(amLKf*G za!D)4fA}+1*T=zVz`Vgtlw3O2U8S!5f&M~kYi&2xYXjZtBh@1W`xW=G#HJ$))+4=t zVb}VL0jnc9fCDDW_<>W1#wxMv^Z`482T)0{xw_VnWw61(^SIB1Ucsb!Q>^%p5)7z~ zXf%zz;}Fcse=s*A_%Hix$Pg+DYi31l@WIuY!~WC~f}rC#n&Ys7j03;U6R`j@R;Xsk zFgU)gBUiEt2yC*VrU|$Gt?}Mmlu9~I#ZZGyi@4Q#R=rW zwKyNFMzW05Y5u9A@XO(~PKDp_GXdv7c}Q|i7-2o*Od5eWUmQ~~25^8^U%PvmKIMzP zx?T(@iuf#GfBfc&0!6KcAtU(xV7`6O)kudW20(yD;v@9opT=T{4r;aYifX9I`=i^_ zZJ@LBzz(u!nk*~}srZK!(N=}IiJDlA{YPA^4zpf}oYsI7eAjidfTiP)_>(;kl+_01 zgRFZ#Z6>IzL^_U*U&l>4r+tu#jM!KZFC2_?0$(nyt*$USb6VLOR$plb zyP>nrb9Z7E?^dF0~((NU_tz#olUAk z$T>s9>>N^foXerB`mbywUWm;rX^(>6b2?;c9epLo7NB&2pS^F4$o_zxsC^$1gHV8j zmXsRc`p1dP2PV*pFu%ETYQ5Z!MAh|s+rh2T-n{BC^%A{6P;il&EX~MWuX$vrY{A){ zL{oqbUWIudJK3P5Qc)-m?(t~ci+-^F5*(LF)}2fc$r;Ih3_b4=uE=C1dMNqLo}m%GKmNu4SpDy(52OI%iCZ+R(lWe8*xH+AS3bY_;!=0fc^@i%@1zd7!}>0c0qQNWxit4vgHz#Q z0E-B`ecZAhF7vx9+17C#^Fk3Gpj|mlsGB(w*stf9U z_>11&6RXh`j|m6%m8Is1>!IOCLbLL!e0L&Rm?KI|s)jN|(H{TZT#>C6l+(mb_MZ%- z($KQhkT}fSSh+G3UHS2sceVy7hMRx-Q7YZHM=?OX*1?oXh)uRd;j%oqLo1?1HKOGL zXsdc!rZH+s5so~3$2OiK}`M_`3m zDZ({a{tAZ_O=SH|D>o9%rxnyp^%Bnecyb-q zsqWe{;<;x|!S&{Gkd$4x?dRQ9PLC1L+1Qvr2w_Tg0R zai1;L4CSTY2w?bVGQ#h>#Ki{N=xtvyD0j3?mi&>7+(zew{J?;Eom z%8hTSG}cn88^8ijE zfOeBzy4R{YyPv+N0o(vJ=zZ^ezWUDhnbvAn4t33y+gTaD8Z+G`alU3a{n@7ySJUGo zZQk+I#{`Y9cK1ogj};zM-{SdDyUy4{+<7*iY9(7$&K z!nY>XWqt}}_?x%N_%d5{4`h7Qo(d|n5fimY)Ee3%diHJC>aA72&ipYaZ^spnu>Pkg z*=5G4&C(0wR(^eqtKKTEDb0QhPX4djpmvA^8HSo8#LiD923=bvMQ(~n?el|T^w@l4A-SMhK1O&-zMh&9Z1Zv+a68l8n*Q?h z*LEA9mpZL^S9S`DXTBN`nsaWq?ef)F47$GNI@eX~c{9NWly+TWVi@DuB0Fxogo&6z zy;q)Xt9gE{G_JM`uaq%HA8eoWUB}NG zg~G1=Rl^5LqAJkXEOhu1HE3rpA}XZ!=sK{n2JftrFzM7h8$oA+6Ac=N*d+dV{I-?8 zF4#FnsNUYRR7kbFDlZ@4U%{J`-WEJu!#uB(kC7U4f&Hp5o`+ZI{td|{9}f<-0F z=%%^0>P3bE-<~OUk+l|pqDg3pJBhnV%!=Qb9L$i`79H+!KJ6*GP8BeeT^7=TEcC&d z){WMk;PZbtiZXudwdb&{f1=M!gO(FQVxKZ^R%!AAD(d6WsjRd1_5tcNf$6NNV5C|3 z^QIP8^o!(*+jGz5_a&T4uxNdg5f}eGKQ}QyOR*boy-?y??eG`35jPSd@J*s1?gP$w zTB3KdQT}=L^#6@J5v6$4_Z?6$a%-9P8O)A<@>R_DV= z6k{Ppj7?Z%$l7y3iGRDLZFaUQs`$H8G)rZ(E# zt(MCMw4IZ0YiwF}j?r5g2_)1CAu%sN2k54gepv5~N1}IejJBZPrFXadoiW?CW%NmY z6euZ8J9R3cwIuF&VUsT}t6}PU!8y0VR7Q_dMzQI4RL=8qnVnk4-dSq; zyX=tfff!`?11_^hao^6OC(ia`$4bZ)7<(BNeMASV(I^o%M)qqEv|Xglgbuxui#FMw zO!_rW*UANfWH_EPxK8>tu<3sxrO1AS82(4S0K>I_3N7O$@l8Zc23_U5{$OA65(B*( zfPE+O${kG9L}M_qumZvsG6YG(Ic*@ZxbLjO9M+$}zy4p*jy9lw+DRX9*4Z)^xk5n( zd@Mt;PX_IO3LJB_+;Sh**fJz5ng5Gb29?PZ6@uU@u=mt$_|jdR-;bm1KO+p(>8;RN zaI=cB@$suy{nhBn;vj&fYzF11@@~u--S9N>4HH_{AIz}qJw)>Ug7R_`7BMJr08L<} z8ZNAN^u)&?(X>uaZll9P1WSO|sD`=`Sn`^Whs=y%4B)C&j zEA(%GLOH0LN=@iIn}d_MLfHL3_c!>{J5h&#pg1zD0z4=O)#fgAVDSgVM4mSaf$%TIAn=tG^5J!Qo$ch@zY4%I|fUC}enB04)hz{>uYyI^r7^ z{);pZ=tDXm8ha4zSiz`$lo4=|Cc^9ww7Del6y#DOia?o>J7^a0@f?P0oeE+}NE%jN z$y0zxt!-xylWjPU>FWrmZ-~rV$FdNO)GPvez70@4o{6~?lq$zfvq%^50Gb8*EMUEf zrHpA)=QiD0Akx znl2Oe_;KDE^&>Ocd}^t&qI$%1^wPz7JwiC(4^l$4TObNXJ>s3Cg%HQjE zYlafH^HD|YArkygYXL%{dxMH!KvxzB-mQvM8nSc|<_C4#Mu*{_E7(M$4=wjkp8t>8 zs}sB%8PF>y)m}=7Am#iA(7u!uMS!Iezf4jEiV1ar{%ff#wIqW||SonYqVqZU;B8xxlIBrN~>BiUzeLz?~kv=70cdE_AdD4cCD+kYXEa3*-g zO>?7HP<;;SA)Rax&fo!g;7voJsCI6CNA-@%^$vXN0ZTXXdDO9Rs`%Y7j0SM2k2b=K zjED=N;Eijt3z|s8LF!;a+G`YvkkA%SIqYl_-|?zDRWnjg#l@;NShFPj;OLu?goE=V z%faKx{=L^nl|GE9v8Kc&gY!JgmwbH|y_!zXRC26ZJT&wDhFQC3f0H;r^eLErx;B@= zmD!(Y?JY?=upp)-n;`?##DnQm;$aq87a{LmZiY6CbS@a1s1e8*I~<>MyyN~jxjrz&6ARMOl_?wgqOUIGQM74Wu`NQ46Qz18Jo8E-lptF5j+n#^6Ttcn>M zFIjVS^z!%l+9lK7IaW=-WB7UNWXyy^eo{)@7_N81ReBHCm1iw=O>xrt@SvKZ&-IZN zrG-Akixdfp&CSGFh7PBZ&!7dIVvd!(aP%^bl>pdkixCx7$ooFsheXgH>$e^g>Hl88 zcX;VeOX;V?)@YRRikxSHK}wdzh$;rF~{xA%Kt)U0qaiok{(pH zb5P^E{7nDmli$m3jh)91wB~72 zJvcqlH5qD~c!q@+OGo8Aa|iKD*>z_Mr{w7T2*Urx5En55wERrnHGAacr?&2V-ma_+v|>KBW$v&y#Wt7p@;kg*p4Glzt7 zPrdlPdk{9Ipo#+#Kk zR;zsobS7+gzQfNu_+8Y8!RgnBMcC?j{k)!rXfkkK+kS0(@aHRNjTF&d$>@cov|XlW z{3b=mCP(;H#Fg*@SGkv|x;4uG-2S!gJk_sWdohG}?tF|T*3M!y=V}XBma}rV(Pd_jK`TH>#Ky8iM_A} zyCVt1k=8ect078b`%BI2x!xrhz8Dk@&={-;G;GU)D#eRT z5fowhDAd)r(xZ5$H~~HbHE6lD<1QwQmMQQ+RVr5l@6gFd9~vxpUt0}xXhjO?oQ|~U zdvre6S$N`0l73EzxrV61?3C5{H9nW0iQnJk-tBaMZk7EjqG#1V0CvamJ^keSC0XTd zuX!h@O?IQi*{}z|41(`sbd_zBq>K7>%HKLauRf_zy|^a9^?i+Fu6uE1VLr^n>U-(9 z`EFN z)Rbe_kiXF5vwoT`{!=f|n0kpVTei<0R6gNUd~RsWGiGX%o`3#BS9?=%K*k6X_aCAn zd2{7zT>jdzH8YpY>SGG7Q(%=fB2&J`vUBE%C+F3o9{Wd;~u$*RQ`=)`*$*j@Ro?0dAi|Aws<;$z=80E-TYOpcXLNn z8SpN#M<8bO@=$rI5W&6G(_#XEbq+8*B~#hB-jK8&83_2@8V<3aYA3&Sj@!U*)o{rE zymi_;sK{re9_;A|X;{N=#)h2gntA$ml)%qj$Cct1PV*l|@z17yCK+=kPH!rl>Y~20 z0I%Du^O!46;}G4{gN;$Noc1zCsXPKK76UaLa?X=zntObfJY<#!M;MZqR&<=VDj@Ux zErI+NS{^RxOWOA)6{Z^Mi9rsNT^#ZrUY}9BIlyQ`9J=#(+CGLc3$VQ8`)Q+9d4*3l zl}C(aS)^Uz7_M;-$;<(hybMz(dbpg%sD;xPr#{zbg>Q;js5O)Sqf3hmp=Y<9HZ`%v-19=mpe5^CKRFlubia zQ#kGJ`qXe$B%+#@lL8MI1Fl-uW`P8R1nnqPC^!Y@GZLthtQTeVzNvQf<@$~c_O}^Q zrVV2K%6_cf4~?GVITj(o6|%66A(#o|VYfeem9Mi6uRk_0X33ViF}_0Ecc(AE$%{Vp zgby$nMLcDf~+x0ka~$zCDE*+HnyVZ-qZB9uUKiG*AtjqqNDmfZ{?a?PoBS7eC~5 z{rqrXr9HHdIZqx|+XcKXi2<@6j=3iZEFqKUdBTHb&BN24ceF z*)#-n5md)0>dCkKg1;#AP(to{;+gE!obn=wLuqJ=lKBjjG4KH3y_U>=|Nbs=b z`O$>acxZ26;NK?5fjggQ;De$0m*~HmPvxwQq zMzVMoeF63SOb2ir^c4+LcJwFIcty8=y^ntYmB)moBZ6$sMKJVyNbS*-Z| z`LvePuvI81t&nwifJBBHYGzXw7Me())rpUT#tj^I*-i}7n6w{w?eze5BKs`l0D>C! zF|ry@^iE*iJ%-M{7lKuaNtRCjJkeRk7_fSlCY+(c4d~lR_@HOZynK*^#uLdF7KaC9 zn#=?vKiNU+k%hRihowKrz)Pa)WZ+J>vl$A42;LJ2N|$@bQJ~Cw#jTcU9s*Pfl(}r9 zqT;W-oHP?b*l@79u%!(EZc@Pr!oK;Bf-%0*1)PkS&m!`Ps2k|oYdJRkFHdTMKP<|0p zw-p@QItlQnqQB1IJvYnG@WDKzVv{`}%DE?TJYo%c0{w>CV6BS z&sgFUWcWf0f7P$ite2y>yOY*sXfzQSO`7MDt|i0*>iacb2bdLG?(p0OrvOoYOp;_G z4L1i}8>XcG+5(hW*isvQPc+PHB*(}m^*R3$ib8P%dRcP36=4M{t~v%xLv6F+y-lca zHsHFFlac{yWuQBp*y&Am0NOh+iYVj4OK-dLdjcgCE~bSy$SmV%hTH6Yv=FwGbV4M+ z2Nj|g#M26vjO+y_FPS~CnJCpo{G+!14P0H!F4c;^4G-!H$0BZAX11_2U=*zyjaBq^ z_bec$OEGu1+Tq=Xb!rJ$AtXX1fpSe6>|b^&4%ga7HYuF!l_1;tz}@666`iNw;bw^8 zMbWVeGRR-WZ@FE(Xf5((L-Gnq5c&tKJtw3FIh1bp(~h*)13Dm5XgR?Fd0(izVW@b? zLx?trgK&r5iTzbQcNMJIto~95+vy0r==ZiAyx)Y^kUKC}N~y}9i<2``yMR>hgFxSb z6BN@LB;e{x!tsT{<)<-bdF9_DJkH&jA!4Zb=~F;Q2bk~fbU%A;2M~LEg82k0!iPRX zSRjHPg87*A@k{_RT0GXURPgtH{^FukN&p1)odb(+&Hzb#IP={H>KAjIlyK92>$gT{ zpdzuPgSkt`WOoNirnSMTz`F}?>jc2@ih$)4`0WE>#SI#{3KzCj9AsPP8eR_)$%>v4 zjC5^F_xFv{m@CPMne9qP%uYXAx)$56P{|1c)FgS{;O<|@q0^5!Xal|sunzj?!R?^fyNF+(#7Y_per_pl zumW-{BNXO&LNq`33(o@_Hgb`Ub7QS44mFizff@HV1|PR5B1h;6W}-_J$Z2JjgAle9 z3iNv(2!Q~h+I25OA|eRx;48tVq^mSnp?92tw3_jEW&V@oVRM_r)LB(Q(f6N)JPsAq9|$e&dz^1z`{t^qh)hGzg&!-lL-Y+R)5oK%@9Buvg`0fR!M7L{|qeS!wSgyxamv zI@v!!+O@u)5T;pl2Hml(BK>qyS1tS5Q|NT$ampou(}=442ZVOH+-k;qWdYQe-o_u2 zz3}}#fXhZ!*8zC67xm0WlQ}AgntLMCa*Dr=gYvad;d`$|4m$jMwYm=u4fhq71b;Np z9Wm?yTV;XPLUJoFfmYl2ATXFl*H9+?FAe;4^sm7;lX+7hu%PEuLBT)~fXIN_A>5$~ z;robO2|hvXSlvV_Zs^!zxdEb-oL*#P;bx8n62bpEitJo8`Xsdn4sg|Dkr_0`PpjMr zZHU##tvK)blgT3lafEhsGZ}{<+kn#*+?QXm$AZ}B>9{VwFb*={rql+6;DJFELO`*J z_{5RA9{5RP%rRlz1mRvg8jLg~&bW84?l_G_?|bP=Jc-npBx#5g0{4OevHGjvF!W_! zQ^e^!%oF26nqhiQu9{C|b`8ALM-8E}E*mpQM8~`&?1az2;8#8n>F-5zkPIBq)ihve z^InL_tKnIU>ey=FPj~Rb-+y%+MpO8p45}CrvIm)jvk}1GI`hWyJc#H%Qz#?Y#o!@3 zhU7~ecZ~hhpZq7FejhP=s>=P|)hp3eheQU-kIj>)k&gHe2oQKbHG@Fvxco85UqQ0nx$ zRU0(}YVoFKKDknXPs!B~z2tLl&k@sqM>5f@}g_ZiOK<`fqcS z&HR!DFlI5mkDI+Lvf7^ZSHgPR3jvWm|LIulIEvw=q*2lz+0}0~q@^o`g0WEh|U86u7K&7u;2*tc9JTua(@e;SB2?SNe%o!ll zU;{{ol4hfERb${nG$k=KMS&+|n~3v>XNQtUts}DQd9`()_%pCG^)JKmabmd|CWf`w zeP4OS9-oa6AVsPYPf+qL2zZ4+WwNqB#|emPd?@gTy*(9j5GWWIMD0OZFR;fw*qoCR z!7jj_ppQ_`#FHoiCJ3s(&iI7@|3(F-u4Q=*VmPNcbu)P^vI(`F){c~-BFZe3Q6E)7 zy&tNb?MH{i65grH3#5*^Joa{rD%B|p`vty0RA@X?aZ~YHjEqmC+A`{UwZv zoOICg{y0xW0_37|8O@T7B*TX$b;lwS0m_!+H$D{QTMkWeJ1Un60;j_xZjUUdg8p%!?2LFLvWS|9!gn*Er=@m#f(Ckuxgf^dklysN} zu?G28)s8MT(0s6)VVgH^R>Gb%qgVk-oE>|JLdrOAV*a9Iz3mpQPa7PIyuH3Rg1!74 zP#`ltYFdv9V*Q4IGqE8fvCqSnw0WUiew&$zcvVs(E z#j_nVe{++gHOE-p$bW>E1GyF7j8U{Z0-b*?{8Un)*+r3z1|5TE5Ym8}1AJfkQ0A<6 zRT5ttew-W|YEUE#Rej6=jVOg=BZY}$3Ftyyrqy0#*B(dH!!I=_#(|wO0&E#`L(k~T z)97(52i#PLleRy9D4mQ7F8V&S)q!WcW z3f~37ar1oL9K$gjVb+(kZlZrr5EA)bb7U%kBAH^dR7y_-)wcudNHG}DS_p+8n8Hys zX7rhr!iV^D`~WC#*j3sh4Y0Bx-m_$3G}NOeGm>BYx>cbOJNuO^Y%58R#jG6Flu&RsaPdo(?_sG@j+xb5r0I+Km`0~_=?u;`^R_nqsIBx-Hz2XgQW5{Y#yf*69`yv1FrM_BQwI8`M+ zO7jLUn(_eAg`_oV!OFzDtk0;*tBy@B9IQ~aSE#uLtU2bgrBJ6zT$K^~#4|9XA2 zO{i#KqwGa88h~p>4*&-E{hG^Sou+-ERy(O+A+*8-MR>6H+%6&(KqA^dI83qM0agJ3 z6j-f-RTCjy4HT}LPOt9eVR_+U@(DuU`$xF`&v_$4q5bX|eR{=)unn5dzv%&fJk%qA z|7m88c77lsO$3^tqBu#bB7E3j@}M_BQvLx#;{;#spxlpKn6LmUrdZXVRl=?myy7Z- z4nF9R5dm-j_A}Q#C~2|@cAIEn{-{P|kSC&|qrO!9#0QBBB4ajDG|0#W zo&W%l|F#gkWEX9yIQS!NF=6V@e>}f@O#U6=`aK}Uz0lzqg}eZBh*klxcp_N;-)4fJ z_bhPL+K%YmM0nAU?jp|jzZgw|W`dEo0Y<$|vqmB2{4!i2APriyshpr{<7-F@LVL zMBFX#v+ET@yHc_lCwfmou8f9e$*$=3dMP(FA2!%kx*Y)Q(AX=Hqe|vcLA6Yjo7MmD zUcq|+T?KXk+>}7b*KCp$NeGDCWE&e%0M=KF+6#1tizOkFpa1H3 zRpg#rYNbJ$bb3Qrt(_*W0RdSu4%G~yFu{d5$Oc`4H24<<03liCcAhNd?{TP6VbHP;LSc=rj)XQrekxr1`(NA7B-_=TULew&^1J>8aO(bNvIs^WABz|~gAk9fX5J}7-`FA>8P$xMNIDsYUrwxda z5vono9?p)SxhRa|L*>TsEKEQ;_REGtZ&h^7%WV%=&%GY*Du zA?B)0@j?h7YQY=az>Ou|II?ZrxM$+QE6gh{E_H8%EfIJ_S~TToxL*TiTiT=`vAjCf4dM&td55LQ6)2FkuQJ?)@XEvPdVd{FYKGn0g3qNAQ!foH%a@(_5$I0%46*K$IDewl z5~U9kdvY$L{6O7iOX#`YSh^x(2sDM?FDf%`Kk|Q_82i=dx1`F8F@b;}S{_U7LT06?-&!mZvT)KZSd^;2iFcn?G z31~Euyfr`WSAXz^uAj2}ba}{!+%#6EsXn4Wef8M*u4ogo63Z2*@|)U!x+e$dN{Ob~?5u%sza z$}3`iYw&J`lIsc?wtyf1kFxiU=em90$4lI4Cq&AKA|yN6RQBpNUN%JtAuF3`*&}_*vKlTWm?O$1ScVD?2k`7~eUbfd8 z}5p&uE}p9VG9M|djE_4`j3woQ}!Gqr7 zP2#40w@*;rz~Ae_$5^h%f&cq8Kb*Mt@7(&o4q*v$isog+_7IM${ro;bZ@&2PW?GpT zSpS01aRhty{t-!juh&68O@^Obz@LBe{T<^P5qZ^t$EAT2O$3PZOBzqJgtf_cBp`|x z^nZ$|gUGP667Un#fQzr>zr&w@?5sa`c#{sCZ>GPLzSoj>9zXa;o7BH|;g3Ut3P8$l z??EG3Kfd@=8*EyE^R>QBzXzVg|58?df|ws4=iet}5y=`cMYnnQ{&#l&ne@<0{o{mu z|7?&8;{y7%b^ikf{<#vnukr6UhtGtv!IvC(frOfxoP5+mcJdiHp0+vr3|CY@D zuPu2(B)~e2z4v+{_4AbfKU?zF#cqNOu*I%I^HV}Ud;(mfyHA7&tOeo(uU@e3K6BUm z2b2D7EBOC~52Xk`gobvOelfeJd0Jl;nQ(i=0FI;cW`#Z!Uhx)a+o6{>7Wz(SYwZfFZw-{L8KTBar+r z0Pdhfk~uN}X4aQDk4oxd@$F*Q#0Ol?HsSFjkB-CKel3+V6o zWNUs8V>4VroD#$SOk{$INEEq%FRpd+S=n4 z(KbEd0xJcLSZF*5^4$VKc5myCb^P_eexIJ3?r`p(;Ew>vn8!<*c_Z%>SoayAFIy5*m?M(!%{nPZt!7$zPzYtTwDEdI!JYF(VrOL`yOPz`QX>D zqmd7f-f-&pD2=O{i}=LLZ>Np^SGUF~oRL@;XHnnV|6E<}z2x6T)7=+^E^7eqQ`k?e zI3;vog8210@N?w)GOH-0%*Cb$e0eWLJI>dP)v-59w` zaQLOM0@K9@4=$ly3oSO=+bd;g12MvK+Kqh@qnz^m#FW!?m2+dx&t>!FAD_@Qs2O5&+SH7(CVAbL z$sK3k8N&V>b31fN92h06y_W5_$6bYxpm)!B3H`kRAFdGR{i|Ig0-;9*H)nGicqZak z$)Br>5$?Uz-O07}{g`(%#o7F^E@-3d1`MGI$fG$8@-lz^5|xYihp$o5-s-vk)nx4b zA!cdREc*0}g*cKm)XYkjnmrj`gdzIan!{ zW}IVuvqJ0TbONl}TsoXtg4=U`3s@Ve_g>7UUzjvMp%d&4uIb<9i&gS&=TVM9ArL06 z6huu(fAo%nOh8cH&haBGJrHs^Kni!F;1){sz1vSIEx7*4ui1kxChUH3<=C(`p|+4bwhyB%%j-)rLT z`@RU*h-^3Ks@NMf?q*h?cV{(dh^o`K2ii4z=07Ia2J>J(r;mbH+M`-p4xvuC2I1V@Uw#H#alpbSsXJKmXMc5=%)R>S^s4@1}@vY{Fl z683WyB)mPxj_ht8nC?lGmoO6JaLMvshVxP?J9->0J1Z{X^?VC;?BRC*`b1DrMg2iy z2_YNXu4kJ_?5+6kV@V`_^ZDbGl8G_)Mrccm-c?-bb#aLRJ9wwXN2b8IS%RX&WTu%y z(mKxbHFWA4spD{K5X@y5KESv1Sz%#0)n8*VUN>x_tK?<74Tz>2(?#PtA}e(RMU1oPB>t>g!7B zN{LlDftb~!(dnWYSum$bI!tDY+0VY)&)b|Jx-J$+NV7Gey%V8d7Z&j53-r0xGNi`a zluT$ELiCcEw(33LWCD%|S=VB!{PUq2SiM<7X8!#%3eDkqyZ-=u zztVErTbyW_#YDmA`(5T$i_CV_~|!5(Imcru!gA#g=Y#&x9P~hwTtYhc}NQvzs#3=GD=SU^kF8{hI6S zQo}F&<9jrj;H>}oXwkPrq?QAM4u0A~-G8PiAp_i;<|!PPjjz4(((W14Bh6CG(QVo; zTSSCID{v?mhnQWsQ8+aO`fF=BuS_OX^S&z2w?~A{5N<+4)QpnHTP7$dt(t_#jI}q^ z?8mU7;~`29-ijZJf&e}C?J4D}4xjTODnIhWqNh+=ykcs`+!_UANNerHb+z^Rh4 zf?JZ!(!zU^%MD`eX!DZsNHkcu&?fspbZ@u!Bj-Bux-%t<{n3WqNgXG@y9;kX!6;5n z)$!xhxN04565IWSf3E#|T|fRdqJ~<=j3}f-IIp+FCO{nF08>!u^9A_4{fm#89nWrU z^zO)R(=_?uA?}Wz9RUD(Ashfjow$Wufdzf2wjPsE4`Bw$Q5r)v?Lys|Vzxga> za_wlNz{cd_5?lvaYIOj&P?*ZB*@~}>MD=!n>1YD@^Jk%#VbQ$TX$32Fso?jY*d z=Zu$pLL+OO@|Qb}ThNJV5Zu}PoUo&YnHumDj8SbD&)_(*S5P%t-6$EGl@&irbQnUTYzu{WfIppV5_yMIOhTFc_WAFFG#q>(9GQ ziJO>r_xq1O0lo{Sh!pAD^EKmiQa%$}>Q;H8=*)>l2jNb+tqJRjV}wDcIB_Nz~zBW7OrBHP8^y}ORd;G ze~8I!=@8GcdjeGG&k5)z$i-^a`UyP?;S#7jm;pQJR$Eg~%YwiRgussF%UX%c_(y9Q zsPCMU68+WFB~*pWM&`L^`HN#?^xkvf|41hGl5sN+MeWCp#xA;2oJiUp=m6iHebP#y zB=qy8X!Fvv6aJ*vN@!;X1LfK#ExN+Du%TKh;aoxG$DBs5jm>{#ZscD=Sr8pn({Nb> zBR`>@bBk~~4?$}e)5%ph&C$CN>Kz_Q{I#Rp=PGt)Uco|PgRfIx8GKbVZDj5>l|DU| z7;9>_??c)a<<6>@{*jLtv!(U5=-6VMmOJY7asNQUPcHQRH!}50F|5mQeH8Ry?fWHA z%HX?iLfCP8&CGY5S>B1Cxb&ra;H?x2Nh!16^F-MRzFa(L3$X?5lVYw*`{jHdUx4r#zDK@ z$^%pnS^z7|@uyAm&rW4G3K-UNwo60JU|Tp}%c!8mqN~ekqnr%qu%L)Xu{E-a!Xb@e zsO%|(mqC|fqBm#OMq(T9S_A59tP6yPN`U9>@sHi}n*$f(LR3j*>I=DFN#bOa4LURs zGq^2>Z@-BQ4YiZscM9UKwSIEg#VDKhg*rmL<98mSu88Ij#{@O9hXGWS?N&tdc++J& z+l#Fk&YbD6PYg$*ZDdiEvwgs%K@-rgeA=Vj8rsNs2xuHK5ghbWkX6P;U;4?mB+AJF zCN(Ww?e}WY3)RFNH1gQqo_#7Pow>DIB`BL&;e0p1rh-^G9wjMFD2A}}>L$5Lkzy>x z<2!&Q^e`;VuBUdOYGdEf#|B78^i=PT<{=w&?_qZN*-Iy+7HYuK~91BvAm7v+u zDr?tr!1iu0=V7PdXvv{EATie2o@aY?paqhC1l+fqnA4#jiCrlk;WwQ8l4*$W#f*J> zQ8O=GzML}-*Garl5jpo}-h43qO{WnySfNeJVM>1Ou;(j@ZNPnM*~}lA@-)lC{ zm82>hNhrn>hpqph!2c6CCf=7VR&DC}-<9!h$$lDl9&`4Ttfk=2TBD_$3lu1^ zk=mc@@No;)h9ji{V~=qU`sWOVs3+tV4>1`6A{^7D+)(MXY~+`+X?E$|=n3s_Sg#_t z(Xjvan8dS|wSRyIA{_h1>rbMm^-vC_Qtv~6^S-`lgS-?|Fs$EGt6KWRc;4xJ8chJQ z^gT4~faHVnI?E$_H;28+6v%k*-__R=)GId-$J`X{@w^$$w!RC%6nk;-r+B z?T6EB3tk~}Kq$q-o=o7_Ymd$mr#D<1ii(58j8=8d(9deIE1p?%V=`ZYoJz&@n#m!`5 ze2($sp)VbPoyD?@JEtGKMJG}|?!BM$;!7)lZHa*7&vyfePbPP@@;RI+o>2rf^y_JV z!TA8c_(ROaECA+Z>D{D7eaMtnqSWw2eQE_}lrrp<(iPS(O`+UB9HzYs%B&e^%&-Aa zh2}7IyEpcu+NNS+$D6%De2&{;sZ}`?=fFVjmk?1UN$5H_F#7Ksync#HuAz7*ZPz~i z?h1sSL{em0Sr?;%*T=~CdP~HDnMnd#`WqhmD2eJkyJz~GqS3q06F&+HPN!x_ksZ%I z@c}kqN~X(rR<=3BjA%8(+*XWKq-mjHCy;H;27((M2$W*2GzDi}kHy;*50LL-Q$&Fp zwOZ0;$W7HnavEdwmr$J3kMV9hSPXe8J)I@-4Z?g*1E+g0@IM9K)&C8|_g!jiYo%Av zX;PO`D$?YQD*c07P~%ZG1|8UqwwXSV)KJxP~ zcrL{We@D(#3O6BSG}N7*)?A{5Bh-9u5?Ba2P54ORQ$g&dK&T}Esw{L=Nvc9=1gd0n zdN;=D(v9g5UlX8&j0~NEgGI{x_?N%rOczchK+9f2{yqmi=cR3UMzKfv7n_=KHvs88 z=P9D|)rO+Kd7M6%rwEWvDD9E<+Q7?>uCM(bMB}rahE^)ITaSt5r9W@in>n*@*0LM9 zk1Xz%AWq|m7x5gqC;Ca=BJIlY#yR{LBjqyB?ZOQJ5gKX|w_%6N0X?UMxL}21?I?>{ z`Z3KJdu5|m7ju*B@!CUSbv%jgV|I)_R@a)U73BDp0-eJ3LXyL^B=63PrCr<=76f8} zx>H@5L`TQ4IbH(WR|6;@!_*G%Hxu?9QC!AP`ei4AEPBW)?fjA5r|ZhF_jmtfI$?F> zm9KZ3#M3Sj*o+GR|E7=@&k7|4QQ+$Z+UXE6i^Wf3i3qI^eR#uHd3|y}r6WtX?SRi% zmt~_2>)(#+i(~Y(9@~R-f)c1ew;z4}bo?(-j;{si%$a1>&>{}pWchR9kZ$X<`*=R( z(Vt)TCPv>^>vmQoaran(KeJp1(4-yRx>U8B?w}Z3iP+k#788(XFZp_6Aa@lg5&Eru zlCtj={n}t)OJ3 z7N^`A6_XOyN4a@y)3)Au_93M-SFfYxSbnhJiw1l(_h6=o^t`SNO2B*dF%jye+FCZK zKrRFvFWFdwKt;(AlUec@TU0q+tRvgPBHqc-QbDsS z+8^l!4vfAhlKSoQ;8dw0JfGrmrHtCe2}RL1Nt&Gn!5#L=OXUvQ#dMXmS0Sl6wJd2m zOy!P27|IdE1-(uiyDqcBg)thJ3ZHxsWZtzg`PstE|cM}(pX$26nn|*nw zRSH{n)Qr=E`x|ypGU4fau&+7Cu%+4i6uUew9Hc4=6k{%>zdg=qSgtvst7HH2!!A?= ztnvBITtaOLgcww#G5g$tpwGbX+)o%Q9=ld&ATLd{Q>$G1>?rT#Ya-n*`jIB`(rVf$ z5ZomgWbLXcAh@25O(m0BRlx_(@k3lR*5n?h?O;1cNv*o@(U>x2k4^w`>jtlhT@NU8W#a-z;d>! z(3Gszn_c4fqi|aMRJhd6cpL_2*?YD__DT=wccRfU4v3LYv9nLRbRu=Oe4N~_F%P7Z z0eT!xBMjT4a$9Y8o@|^)Wa;y$AS#4-MaC4P{wh}6rhw&I-I?iJa-D3M=##?11Wm-T z+;jb$N_&2C0TCT{v+Lz?Kb^7u{5Q5SY1P}O{T}W&k(+wBSBkf$%$S`*I2X7hTa%09 z0*ZT&WsMtK+jHBd7520j4|BFk$dHTO)GX$zaL`epYr9bV)wBqUKb#zJ{#WLXlf&!R z-!o^EEkN#k`0MYnS%|R30})L=&a0#I`L3?`qiTYjWG{hoQvSmIt^)$JMpc7f!;0-z zx@{6aUMjjM6V1bkSb6$sH;PoN^4b&EPT4qmP!N83YGK;LZN7glHKUZr1ope(&en>V zeU_c{ppVBSq+%N__5wTBy#hFj7Kik1i_S{oW?iRjxH8~P|^UFC*o+PGz(>;90y zVTa9OuCk6<20k=QhXDVw_IVlFawGA%(xySf@QM}KHn;G^mNq6X5#A3w1tIlt_V}$> z3RlIswNs3*t_vwU4s$3NI$o-BFvc~zY^i14Ve5lQ4E7WAqtNUj52z|`UrzX1u#o-sfgUwid(zMN$fB3*d@!b`U9pCj2o3>0nlehWuSelwbJ+w|l{eN-xHhw-FE zT}CgC)Y$%v&3GrPDLQdFR4u;$z*Y88xn>z}cfyY*Wmg^iAlk>iZ*L&7(F!Cb>@g_P znaucU2P)4^^RINVAK?tcb|q_X_LI=N>D5-%^Hr8FhDmphauWpI11d5RR*b8nAa_1>MW zUZT_|4pSs6%c6zmHI19_(+y|U zZgEFbF(Cu%!tWz~pv|2}u4og^X3{=u0ae5R-W(BZ{5Ty^eCSd>RW`G^2^+f@)c`OjatK$ggMQeR z)duf15@c{A18|I5l74_Kxzhd{09dYJi?-VN&eF|&(e;yIU5=vW%+}%bQ~C5fV8pim z;P|=9RhaIfl=ykGL(66%GKXPku8OiTjUnb(c(_TOd*d@(=2NCZx&>Z&J@&?)N|#Xl zw2)WuvlFHXCKlOK1zRofRmpOv{EY$S()nO3(56DQKA4jXMc?nFC%33RJNAfB_QFur zxJo9Oq#k+wbk&Av8~3%X?x924t5?@M8l6LwrivGjL8iqLm+Azk-)Eyl<+GkbQ*Jx$wiG7G=0N$7d zFHdZ#IIj&Uq%9I2C7>eC@UbVXeA$4c?761dVe595R{w*;B_oK1xRzphg-(P@#4)7Y z()P-dVyr`p3>VhVagfIFf-4Qd9aT;3wN+Fg@eBniG(((@q|Tq))i3623}3 zBS~z{NR*|l=ZdGwH^y2Z0H0x3zCu|{Tj6;%0ApMPq=3|J^Mt$*0$fOU`vI*>y3%(Bmz9ZY|6Qj4A#HE9r8XbBR zBJS~Y@%k$>Rt`>td7s`((Q2n!g*YlQE>n$89WN$H!mGMjj%io@i9ct7VI_g|8pbJj zEK^M!w?;9cwbQTlK=fhFnAe=`! z*avA7#Hpd>#wi`o)ARd1!xOK^iF32l2KPd!9{P|#sXKq4oepvKWATBqDGriXqRlH$ zVT19P;baprUcLT6F)eAfC3c%b^`FdPx+wLKWH?%VqB-(XpNiB>%QBV!hy41ov!B+av;a$1nx;&scWs*4=eTl;Qo^sfD)1WfTC9Ibw5SeWOTcU)tM8 zp-o@N+FzXe!R>OM_**zjxW6W{+G|kJ3g5TKhoVk@?C79*)%-W`U z%6z3)h4U7&Pq^Y4?ToRziTCqUr)I%|^D!EiT4fZiiL1r17=2Ev<2miLeK_T&`~JCb z62|t=y!>w0UvycwYlc$T_PG+dN&@+cE&sniIw1bgD*P)VqAEKx?y zsz$1;S^FG-w`l5nOnhX9FM}C<$_*T~A_)|2ASWjvQoyc4(k^)KJ?x`U7vT2H?CctU z^C5t8@E3C)G;m#HF(BoNxRXzS_lr0b)rec>N~GywA_s-+?JRE8??(E#UR<7nX7Su9 zL~UM%#k2ycK-qxPakENro3&za#h$Kw{2(gPJb5#bSY?PP0KOq`ZPnDF$<+E<)|Ifq zBX!)&tnJeW!tyu9%htf9 z+icv~l&dc`6)%PgPri+TGc!dDcdw6?pze!#>-9u$8!M4Dx~h!@oSfmz)Df2U^czTH!u-8jXCM&RUl6K9+galfjfo*)WLKmJ zU16mBaFI3(%p}c7!&|_)@ggRIPU*aTw~}kl7z^WSqD*~j+A@4^b3^O&h;U+$xk7hh zE1k!YsK%vdjAQTFKaX>bIxgi!XIHl^`OwKb3i`NjHOwoBt&JNUb_|F4*N_2fZbKCF zaBl5n#*V04DJ1`3(~_mO2qn%FDyj?DLD1O^ooKmol!S)<_$Qd9_>hZb!SX)1Cw?nk zsMN5=ou}1r=g!sZKpYoBoh|S2qsx56Ki-7wtWox9e$zJ*Ps4Im!ZZLYr(ZDpk z+MAe#+Lq2uyFcF^bd}bWKs7tQ$wK55(QnIa#i;8ITeA%yxRaS8s}%`Mq=V<%1jLlt zRSXYRESU5N`$VgZ#fe(zH}B%eVsHYbOnXuiuvoz6NdO_Bh{Lz8c32;{sUwyy!#fTj zpDL6n$xz49V{X=s11`#M1V;b5O=i(ms6@!KRau*#+`OECe){0{+kMC4(W4lvJZ&ZoFjbES5_L0h z$v#9j;wb;Lz|jlrS$>muL}JRJcSCE}5P)dvCyjnL{ZTEE=O`^?rr*4S>b$OXYWC^B zDE)lyvL3kNO`VSom>*(8+FGYoi%iS*#!M`d%Yp*c*RMD%PZko6&*TsI5)noWOfReY zXv4wWr8Mo@b)c}R=W!};nL$b&grhWlLF%};)6*1`se@6F{Wv#NWW9@4s!lc*Sg7^r zN>{%kv7LKKxpg7Kb8UX!O_cgLgPGn#7!!P82Gy8SOQjQEn;B~zA%b^rLxAD(FHncWO(D zaj>xXYXRq2AhxjM02{R#3QjFG?u$KKBg}KLL14z(HmH*OUb4g;*emiYy#6R5VYI-l z3}_=5TVhPw`SVNON z?eIoN+KLo1kPl4?fw)n?wYrR6cL;3af!#2K8E9_recu2WPRWfaz@sUBEonI@ElbzuNjekZzizOGM zIQ7IM73=RjaH}l!m-Cp>E0eE3G>d^vR9;AA^U^H%&k8uESL;5eBt}s%+U(k`C>vI; z0H?dwTzkN-{f!1KCNNw8E@NR?V0G3G2h(fk&4)+iaeG9lR2?fTPBQW#S)0BxSoK|( zH(1%YD-623G||Z9rd}M;e~%0K-u^kt>sncMV2YP)-*4;m4UN4t(2Zz|v*ack1OAeP z{Uvp5ALGL&o%%iK5nK_ zna!D4=9dg-Xqh{`5@6S|HPousMroaSzl@9PfwNQIqe5SsXs{&dp9Z| z+fBYllE%j|p?v(T&#S@{`#e`VR{5^1kG@nu%hyiN+{e`5wI>cg#>45f>x%=VKL>98 z^kV*$p7sJ1hEWlOzed1}I`8HGu2m*$HHr7s?)8~v0-@0m)GWD;nKG9`sM1Ts@?tDs1MEC@bZtsa zAj)0p-f}7RE;z8%iR|wzd5X(rKrJLE6JO0THsY(Acx9f72?AMCJ6u(USD9=LF$&as z*iz^4jl5eKv*wMlRVk;$$gLIA)7F}i)WZQOXRSGvkWAGOO(4fAmCxhkr%W(Wnt>oj zj~Tx}*=}FFKAviPKD3$IKRZWZK+N{DI7Ltpi9cIXAIq_R{wp|oPgG^-WZQoV$N#w^ z0G^QT^rM8*_pupI82cQ5kV!%dI4ForOZW+_-!XM&4`+3?0r-6h&oAxu`-9<$l_@9f zgwNW&J251&c<5qNSL~(j6_*{Z3QyAT=#*O0>A{(>2d4v*S{&*jb} zednAd&~;ktyvpreIe<96OEfVGOvB&PL10bdeJJV-2TlK|6%tiIu~61`##ADTl4Er( zP-$Lp?C)(shYrCBII$C!(9zUE%;BF6Y)Ybi?276Pvt7mvU@C^ZB~o?Vl;M*`!o!g3 zdKOdkneJeS|4dy}OGOk~Fc^7gU?7}BZ-krf>i*QhZE{n`o111WX*Qg!(=$F}i-L7n&wQ~oWFrwZ7 zeO)`4u#yNu_fasNE1LB?w-D#eKEmh0FH2*{#*|ev^!hHs=@iWFY;<1EHoQPssH^8! zY;VkI3y!MKk`zI0ljk-~-gMA%o?yuZqxWH>#Q8orD8@tI;tv9oOvGN@p_(D%+Hg!ydNVz327c!Vy_2Bv5HarO+$bfTQ9?G6dYVoNP{MIL zSMdeimkBI~0_{XxuyUk;0#KNlTtM zC<&DzudAu+b7@jzo7m4klGEaox9UCfA$7i$2hW6uf2g*F0zK4MV12ZX%rtTXtI+nS zvAEA2;n?hkH19xJ9{MeS1U0n{(q)_TAK4co>fKwZmtmJ$E!1-*2FeL=uQW{Nj{9zx z?QG4+ffMbnJ#2Ji;_%Ruu0pDTRyN=y$pI~Qw^sxizY%|VWdFBvQpoC|M6RTOIy#D% z3#~`sL8_}4w$(~ff+xCe{K@>r&JnIIe_x()D7CZME)~zdPfN2KtGBJJ-3JX&b;Xto z_jN4pW1M>!E_z+e5^poaF6v`UE=04`QEJ(Iz&-PRjNrAubtm%7q@~?TG)RAviv(z7 zOGsYVCHCES^a_<;iGhLANnt|N7?PDGHI`vlZ!GDjDmd;Gaywc%nj-bMQlb)tL2e+4 zh62btAm1luc?fT0uGfGQ7N|jD#;i$jMLe8eINHC!ibI_NwXULBaD1`-??Jsp*WO$Qih=TZw4lqtsLs$2o{ucoPKukUU+|9)j6XA56VVT?O6Cmij zDccrIk;d5n#QBc<({yw=9;Tb|0r zFYZ@%eMbHaOP4p>y$GwH>YGEoV5T>;_53JYHs_FiJOpQsq2+XU8PNO|UX>2ETCV6k z*>xNlsxUf>nmn%Qr5;jprcfH;2c5cA;-RlWV+{Xj{(M|kkCL)xF40>uVOjzqvb{W+ z&27kZ*F2)(*i*4FjPy8FI&Tp?q;{>=XZr zbauNMBRCSTCDqcS$l0WeO8z-g4(jR4DV$*#|JPFM{0dEzY$oRl9%q+vo;j zF-P?IS*TJ47*m6KUh!rr%XVGlR9qU^*(%#n4xW$wfSce9_g5M! zYPYjSw?m&%-+^N1&4sD46Cjv^1zv{Y1(via+USe@+`SY0&F~ZjWIn(e(m-2K+@kY<)!;x zDcx&JQ_X!QWtd^#nkwmMCrCYtx&WhQ0N6F$SSX%;!oan-02O8bmcuBgL2UR+gimX> zDafFcfs7XY3p0RC?$sXhMy*W%>-Y%L2x6^?TaR|`I!SvBJ_4Visb+4}Bq2kgfhb%k zf*bjbA*`f_Q^xXhvJcaW3e|T#t#bim>}`Fm^pz~zvM(S`H$(H7hMM6-O6n`gIc}0z z8}0SxP|3Z4c6;B1`|KLZ;jXqO=sKG^fm+O|Zw!9oUlwOIv~4iS3y$#G`hx2lbY&B0 zHKAL0{g!b(eWHHk>KX3e!3c~oR)QUI09c*q8*P80nGPHf9`A$;wLPl0*oPSJk3~#Q z-X8IGg!RfmC`C-A16JW|p<$(cumgSuukMJi;JWgsX6fgjv6J(Zc+PpKBx)73F~%u1 z18U^uUbh%q=DUbsCrmG8j zL@Z~`*sKq)fbHIS?C2sGb5zdwR$m51rSAR=U^zwOdCQ>#<_4Q#_~fZZiV4ayF2n4x z5=a%Zo{t+ql+r`r)MN9Q<_n?VpQPOQQdRr(h@j{V`Ska47z&H)cyjzpihMa%UYBY5SW=tOFq`5BIn`b_i^0j( zO>c(HKm-pt9DZcBQI)~knjrcpd@Udo;C9uu;pG}R1HFLtUt9oN9%VRPLw!D|9?d{9 zbJ{R(^zH|OdDmHW@p#6!h*Oq?6L{Ict7b&2r!1G@KQXMz&FGGR@nUMUal6AUNX=ct zs5FiedE5-Jh!aou73mWXLbo{q=)twu?&}DBh?gzO*2zGl%FztBHqlY-w(u9CL2Sxe zCx5k9qkG#>2L?>L_*~?0ziY0Tyu@n{fRAi9QE+=l*R|+8n*;_(;b4~g)&){Vrt2)% zyI38;mM#s2{+)GL9tzd)ufqGugDQQ=)jQ{D_f%b9z`tITcE_|5=(I?Ts$Q_w+r9gP zFe4XMj7>2gcKWB6kg09`+cB3gomTfS$KL9i%!A_6S6^M+01mZq{$A-8l&{96In=Zl zA)Q)3T>|jQ>l3+g-T+q-OswbWG3GUqcQ+xuW0A3~g0l?#r7eiyr(PX(r|}hs!b}16 zfl4+$#{bJ>;IvsWYu>TBGj)u|Q-{hWIgm4TgxIU*GgyicfvrNCJLH z$l#0Mg6GT0Zql!3pPqRJjk4+Tv(m*}U~d~Nl`G$nEDrgw4fYaU&pjbmM!?O$ShAy7 z#thWqMjm5gk7at#$lDtmxsXo{V#sFBg6YF#%zABep)s_!!t#6%yDaxm5C;t7Z6<-SSyfvlsZub+3pvzTIdAkJi0b*eGwUA$Z+o*&eX1WC5lsysMWgVY&VJi zM<|EQ`4`xex4YQihSQ6*Q#`$kJI`N%3%_}4vvGCjqX(|t?e80 z{j%e%th^Il6(`j%Rz!a`9x8(t!qi|sW+vK#%5$supWSN;G9`tj+x&$5@hQTUS#;^# zaF;=TkT=CO9+Zjqdoz!5aQ7HuwYyx4Y|PlAS7K#|C{~PN(J)GSNBJohX>q$PE*=$d z$bdK8=Q~2Ak4s zB8}t1r;ziEzKPl`pubR*MGlt}V4(gO5^PWgmP*}Q9zEr&y{CLO2Dr8;rn~9c%znEY^(5mgHJJjj{D7wFMcE6B8*?1D&ow8 z7Oj=&h;U0nnmZggOe53!+g^UmOS+Ln|{Kw+Wb+1HH!!n?6y z^)TWRNmL8ei{&atEHTg1diTC?^rkno^VC-Hp9BmE&8+fiz&KIqD(%ptp4av&FIAxBT%iuM#X98;Yo*YVq&dBVCV`eWF`(Ci_exfB$Kjs>;N1#yrbj7$Bd*J;Ui0x6Q(yp9oDJbmgoPv2g^99cZ{1Eho?U~ zKHy-#U9)(BML51R&uZVpk592x_KfyX_T$M|3spe@K<$b483DE~8_l7mLDVYmF#D?GU}VOg8+fbN0kU_QQ=W(jl3Lye@@ zIK%e!Pm3@<(RIjp!k6N2Ehay>^}E-Q>c$;c{5p5u=&FM_tThFEU&2=T;2Mw|5a_X} z`5Ma`RTaOY3jC>|wmAA- ziZ(A9xMJ+J=V@&*fY#jUbKk=DYy!G`J)d)HN^onjg`Rd)%d(ymTSEwIhBD6U@iPrI zbH%;DR6bDhB9N1DRJF08U;`D!+P&DQG<0ac85K z(n@xX3e$UGMp>}(+zK>AEVb6ikXM~sQll%kWO$=y7jQC_`}!Wm)L!Nn)AnCuw-*zI zMT)2YGSAtH!yL!xy+pu$=+?(&F(0yEoyW-`N~;zkpat1@wu<*#uUp8EeM82PG1dt& z+@nyozf|p@$EhMyJ#{YIjh3A{jHX8nwUT$S_bw)I4Gr%=-mk6qYps|+c%4cbw?zPg zt6fDzt=EHYd={_9hI?fIq}U%yMy9CT`WYS~afk7|peH`U@~a6pomhtH(@5ZBpnan@ zY~nGbjml4P;V-R(;sPE7IPV-DeXG~BJE;!}+c=mmEy^Ps(;asPN906OjJuNYd+C5m z}V>hLj& zX&1k7J-=L%kkv!bOvfz!FVvJsM5ven_A0fGg=Plj5RdX%UNGnCaM^hEMX8MGT(GHf z91aJUZ5T2=hf$B-0tcil5=T0rd9dZyQ*un9u2ULLz*pDH2Gby+TR$}D?0PKRstuuH zjH_&|J}&2qJMU(dF5{bg2bZ+VMDK`l$2XvhE#~y~`@p`oMd6Y+2CD6)BS^~x9MWo( zlAKdKGJ*Qb)!U&aiRfR-g-kjUVr)GO(3vAiGNVZNjENvAr;(!R>A6#3>NiOdHFn=q zN%+a%6>$o>oc@iD!|aBilh&$;V$}%-xkewQJeXo2oVRoJ&m3(k_bDC&|X>Oay)e?1n zp-yb6)aMmTXsSoRCFl>oQ~T2ydq*-Z`te6sYE%{H8Yu|!;6E+Io`F_&aL z#ZPM_DpZSH6rbz5y9Adr{jgzgRZ}Tj4Ov+&L0HR>HC?}gvws)+*)NXPS?XRj@uo`^ zN`pS?Gp%bG4&4(efkqU95Rb9M*ZPl6-XbTwQ8pi(5ri?|o=D4<@5HB#NC@F7{&pn%u@3RGmwo~~?1C}I=FzE#yvaBjoW32>7N66)5pDyjAQy*l@qCop zhMC<*b$N{crx#wA)Fy(24d+UU0`ml6R8elc;6sKC^Vb0O6On6eVaT?-Uaa}eBYQ^G zob+#@i~{}cG`mfTp?*)6j|(ej8hwGD_ds=F+*OgATA1gk`ASuFp?P3|A5*PjS}-(> z&9xYTh%AE?a3s_;-^-oj4pq8>Lp#7u&nVa|2e9&q)(iqw3XTm`Bkxd4#UeN~d1YhG zylz&bu>ycsC9uv4!?MY3{CVSlA;n2)DcOsf9cuiX0P|%+ys6`QZm=~(D+I}iREO$p29xp1=5x~DV=(4N4z`%KHRf6s>o<1HwOy6%q}rY5g(KzBJ-YMG zn0KgeVnGjPi_93^d2cl2fwmX0*;;DR=V=SX{Kfg4fhAzZg7+-F&#l&;VFd+sC5n(` z-MZe@N-nbLUcsw*q2U57Dpne!kh(%VQ%eB{~hLRGtHv$}QHsT>5lFx7~zKj~s>U2;$i19Gf>$f=A z_IrHHFML%*=y2SPPEuU$N4<~8xlnI2j1r)>TKXf~hN_J^=;w_oh5wd#Kb`LcBOz*? zHw4>8RL1E~`@uP-()sK-qtc)f-{(f(Vjji?9lh|?jW%FX83RxE__aD9;|%Jcf+w`V z;Kq@Cu+l&iqorq`R%;15TcYhzvA2?XJvUa+b4w%@v*jwXP))-LX`kk7*9Z)a4)Ho( z<^GckP}L775`JFrSgXN1&TrL;3Uk43A+k5>4GfTwXE7*_o!WK08_k%-3%Xpkg7^>N{+ zKF-egXC?!@QX-*m`$8Nkid#W2N+IYx-;Zfcun*s)Sl zgCCsMitdlP{daILe&*$(eHWsv?(PwvX)j!2S-WOF{g^~cMNQ(leZQ0RxKB;|P}We@ z6*}6`5I?c;xm^=rf-cra#p&$2YPOK54sIH<+0i#_ee!cwJV;u5p_8zNpo(b8pT^Ks&FK~AZo91sJ7^4PvkD(P4~!CAP$G{fc*jY(-t?4h z!t-rF{;BK5r8?N~I=5@p?ki7ggZI5>#9^VnnH)Z@gMAow9H4$^;_N0=DuL9i^q4hO zg#DF>D_b*`Ec6E}JP#B26EZSZJX%#bZ~r)EKt9}zmeLm#n+2EQRV$BAnS%u&&Ky~4{hSFx)Y1^^IwdK9a^;Y4t~*} z8Nk%(Ju-`cfc$Y4zQ)ts8ClX%gj+im%+3J`GLa$u+}h#2qt3+?x)mX5^N0+@v=uMn zcY3V6vpaBPx?ECaj)EZ8lHkl~*X$!PDJs8O67OUSb=khGmpPnLdgcO~Lwc?}t``T- zCz~7RkDLj5C=$ZGwa5WI27fi&*u8HRX2{f|E^+KPjeA+TW zpgs-$d0KO1=^0CU=_`pWzsM@mg48EuZ)fIYv&+M{=vTTU>6M)HkVLi6m~EY)@!0(a$B>djNLHURxs%E?7=B*)7+-}9`Ahh zt>H*BEwyhi?B#71C%{%K3BD1}?mDKJhiHTW{S7*dl}D?Xj?x3v*+qWu?OyDCnk}%g zinD&R1MjdjL3M!YEm6@l{Nt%L{`AC0mNV@-}#{5WsE=k4+!p>;o z76m5y+o`}wUCvmL4R|Y3B;Z#1OC_P#c{8tG^%jgFoZCLZ$SvX=Uon`0AHxGZc2e(* zq1cy}x|n(W+SXj5vAdzl<9^PQ@1$CT%Q^kp2XAPRv|4l8pK0{5-2y6%AA7S%nW=2qif_kD!)t!D?D?@*V*SCr#SA_v}yj5m(o-0eJYD|k|nqvP5g82T5yAE^0f>Y z?V_4~#2)4|nFNeC8DD?Layj{+RLft`)w^H(QHm}52kwC?dwXSa!65X*zJAW%x0K8O z-cfz0WC@T^Tab4fMDJm}?kNu*Nx)qiYOLu!s2U*Vcz|SbLrXQ(tKDi)Nkin%3RDT@ z$q79*jIdF?m~4BaOsK4IXK80Gv~(p{&-(lK%lL0SuXmI&jA`%#I6(SZ5@t3xtc~k+ z4vaQ66mENU>w7}M{)AzXsZz6gD;Hm_?!IGAs%TE6ZT{`2X35l2qzLwCt)s(pZtIJU zx<13<`WV$K4^P={P3L$8`XZv;o$Dgv+lze=74}xcsGeG7>a;t+5*KxOEraHe>6IQ+ z9tWs^BDCc7x+8QM$icuvsDLCWyza~Ke>3g5ymLscZGS%Z(O2QBe{pcOAhtvjlWG7y z0vX$ajL-wZQao?DI^~ghUX%hK=PRoLmAEUMq%x;h?q;y6+gkz&YR z?XOrRH-^3vDrx4hP0JzHUttRQ>#`vomm=Zf&|NG}iQJw6W%dJM~!x4_!-K%eM@e0$p6CD9r ze8WSE$x}ze9JF2pGv_1ymbQC*v{cs_-c>y7f@V{8Qq@Ye@A=`5%o#W;>xkyuvX2YL z@RrcfDc1kG<%@ez#Fh8*?GEEIM|Mt)T|Jkd@YaCUMCv?SyUq-|a@|+|`cIecsb!(_ z`%gZ|+}4LUxMF(Z=hq(P@I@cY%4WgnMy=_aY>q@BEdC4&^ZdfjVmd~=C4Ne6oi=@CNuXa5QS$(_8h_KshXeBDj5Wk~R0aH(u=&;Sum7@P$`b#ZS9 zR%#>tlC}o}&o0?`V7UV75I?rvZ}D=`lo}^Uz}|&-wzyyfGrGlxmHR=azyICz)hb#W z3&0I9NZ!Y8d)@kCh7u4Q*Y5Q&5_;neHOvdWuZKaK*C9S7*STZCKK027xMC&_|My#w z9>CL3RTLp?k91|<-T76{=_x;YY9IbvYMNBe^Q-ZABny|k!v*Hj-DTnrSh@RR)2(IO zK483I(wEfx5onj*fxM*sD!_ofP4?Z6Tck%PM+>nteF^IlM0L9*{|czQF?}xLY$hkC zU*ruL0hM_5QwgP4?bSAaMbg$dDfatwUbgp4P^vfI>PHYGhM1Zzl>8^oEn$BPEO^!m z-j+q%B!l~73(ogGWGttxYzH#gs@>rmQan%xok7Md5_u47uY)$MxT7w<&AdBb`}AsB zY}EE@CXL+CXE}9w(Rq08;4YGWRFTFbxjYvA?a=vNOOOpLxkv-i-W7@m9*#qXS0_Lg z0*^3GuE_z@`*0WH8>N8n-wVx0S_IR5rTS4{Xo%jZeZmYPK$?$UNo2j#7c;c=$Ao}_ z-2}DJhPEJ!G?AhXLYwi90nk;VW<|NGNJW*2?3-KxPxS3#Tzl+$e3M5g)73bJhoi{NPg#Cb293q~bsCVJ3o_?6Cb$H5K^s9es-c#FC zQ^ex&40Pgn1lFB@;5&H0U6!F|1LMJT>At#i-*f&1#(>Pf58v9$b-CfW`YBzlxY$!K z}eb?`V&T$F4z&_B;kWTceNO>GM@P4$r;V048p(G>FC5Ft&i7b=_xv66wKe zfASrf!a6i*I2UuJx|QjlAipL=r`>Hm*EE^8b2M~&_#)gcE@>E*%Oa{5Tec?LoiSi( zAL`PP97ln5fqEW3eR;=EzPHj+aBSdC`#balU)(Om#*2x!FpULt55U zrH35!_0WV!u%p?g0`fF|cov&@I%*7u<_)&^-MCu8R>-KQL{)xN4}Ys8o~_!Xe-V)% z7e#8yrq9U7-+lUh(3sEbEPAafz6{>P`Rn-heg5R9zvJ7Gv8X<{WrQk=)tq`X4^I#_ zq#wzB;u5x*cbD*Fc7nqhL>WM>;F~B5OMehT6)`>@aFwDUE)oQh)X3`IOOH+kqNl)R z`$e_)2FjJ-2USzcH+?xtxjSbrpHu#eiR%!|X=UTNheZcU6gHT@On%i%K}-SiGEroj zdF$vW?7uAwup`%@;&gz$E`^=osn@KVm+I`W_7~l~eSuD-y0#y|&0$@CGIFz$gzrY^0|-;IrCd&tjxG6s8Y< z`n0=%GAsLpI(EY^=?&m2!+Q?EdU8(rIFUBFS&Mu2BX(-ytH%#v1uO&OQhy*>PB2M* z8=5aOpH@buY!fDF*uy|bjX`y_4_k?XZ1qaL1uYho#}lxZ(A9^IeS14$;9! z(6#GaI%jAjp*Bq-FlTJB4BkLphIO)uRQ&kgn^IT&;~ zUrRfzP~9;idY<=NY%-9b3a2z5;d+$!YT0-mVy)xL-aRn5WE?a?0CuY10tnDNcLNW* z;G*uE#5Onk`$T;FdcAZwvHR?N25=#77%aY6SB7}SQld2(){I}=szGDnu z#B>bpR~hc8v<@{a!vDs7aq`m(Z46AOIn^1;*Ut|}R_S4D?c02pU26}`yzSR%k)%ku zIgza67@N~s7*qrv8QEx1LOk$KZJB=J6ox1Mfc0kJ<-ka`DAliQAsnVqvIHya0_)6YUEijq zjDli(P&KF@pjLa)as6a#(R@MAU6&Bm}GL#n|(K)+@MSTvE#I7UIsb8eq(nFZUVv1?wrt%vryWW3f%s|vZ(7T48v@jGvSVyUf!4UbA67{Tp^tw)wn`N zuWhXlV=Sh7kM$v3;9+WV{ynBI3nw4HTNOel{^5q#@arK80TdGkP>N+<%Z)5NH$LDy zAU~5(a&yZ-X=!PW&&R&8kMbe}=#$O~bwOchmQ_I#w!Fo}R~>y;4VH`ya8sUL^5J$u z*pT*1$mRaV0)B^=A-G7~Q zZP}@aJs9cgv8XxCVP$GYV1x09<}2v~F7}0MPO< z>JH5SAcB(p%34PE8o(izL^nan)?EY=bk$(c=+}r7PcokAd-kbcbNQxCo1**9PItvD zMxA4qW~RO74i?eXy_ao8QHqR#G@dzufPSXL+#~`eY6tArhbgn|A(=0E5PqFMTm3n2 ztK$63Omm`9=j^QaW-0y~U*i|R(=8FMH2o#|0{$!7tLgRE2-6od;6Al&DFsQyA8=Es zyRupg8pbT@7spDAX6_!iY7nN8epzuq%w>v}u;K%;50trrt{Ll;vqc4Cce!^--y9nH zytSGt7Ag&}JxMp+wQW`o#Zn5C- zgbyN;sjVNxa=(tl!hT#j+W7hClp|JJfVi4zQ2&RrpX(~N1zbQ6f8qxz=fRza1F%Rv zyhh@HZ43oe8YaO1-(YF^WdD}-gM=HsXY3?}TPT?OnWGmTM~e*zfO zy;8sjncQ-*7#03Yt+K;YQy#lF+pi$>7@QRQCF5fvK?YnkG z!&tZ5=_>>Um#d6dq>$DE(xG64q-2REfLjPio4=k%M_|w3xcn2Bxlc8geqpGRyWm`SG=tf~ zr5A_+>|u}@){%xs8(6>4i(fR;FDZrI=z`PYF?u;Cj+NC2&|Drl2Y6LB3(iAfF`AD8 z%e!rcGsq=um8SVKn#okW&4-ps0QbKoeal_IvfWR(`Kjcqr;Xk9-?k@EGAlO#wCxoq zc_e#VA6xl;`a$=7wHc*5|3L4qD!TT zzve#hdrsxB7{IN|l2ReKyo3fe`G2?s>7%$d_TGQ*~0(2n#AAV0`hEgCXx;8E9B}bmX0>9#? zK!wt81o7T`MIXtr0IGsZ1r6(REAkoT>)jhLEdQ0b2#Uu+BK|~Y3H~B!qP7&f(@*`x zbpA+$$m35Ak*hM1z|x=|LpbFcmD(V&nhli|0DAW^n7{tz6na{1U0q$8svRAhR;^nI z0;7CD)2mIl(Mr$`fYVEb0cp4olBq-yb!9NDo=>K@+T3Xw;%gf{nf$i zoJ3x#;gA1m+A$r85crBtB+l#w+E8MfQ9@QYz#X#Swa*1+!W8qVoR2<#^Ad(x?qK1e ziH+jKKmy#m-@Dfj-+``N7#sXAa@`3D9h^I|tz3+uhr%^PN$_vh56MuAO2Ss(33ay- zr5gi?Nbw}UbQ+mLAiD)x384-EwUZEB!!Vx4fMPOtDH@$=aqfDHE zJ_rHmBv|&bs3M=)Cd5P>o){ZDTm7Jl)>XnCWL2L#3mA7eZb$zZVXhAT6BTs?+9C0c zWj3mVKeVBtp|ND6G!>tlSF!Pd0}0Up*@RbYEVuwvVRlaY!`H;QucjN@c2Pe|SeODM z>^fXTRjLUT2B+VCEq|Lr0=Y zij7XCh+i0Jcq@Qe^UKz?G>amXPIs9#cL zzK_p>B(w8nETg)l_#NnD8YD4AKK<{72gd0xs9Evn&6MDw^j{3bS7J#VhAj0*zIC#L zGvsh@BTll7!hmfcGHUF!>h}N0{rK-7Jt`eE&@j>w*Tq`4;0_B=a*)w0_t5+6*CCT@ zfF7k$WtNjbdJ#nzWifEDj{ort1n?&70g-*>Hhb~^MeB)by|?PmZnG1ZnSwFAJ!2pR z%D<9+M1T`wf_$L4I~l&*qY=ZFpHLZ8Mj(Elps{hrC~$8L_hirjA^wWrZrJ@X8c?HY zBqT!#n^qpiBBfSj!uVHe-3I|6vApCkk|A}E^~)b=3s-^8{@7t}jc<_-B zA0CE*35Zz62xH@eu>mUj*xKV;64;E={NfURq!v^%|1XXz!mtHMcp+>H6755%EWXH( z{r47@jcD8A0-IdXJw)oN;ASJ|;M$G=6Z5G$9z~Whit`Kn{UQ`%rZZphE^rWu<0;Df z{P4}_;{WQIQ5J)0L~x1{2iy1c?O-_f|B)i#8a>0lzZ2TJd6e>AJV~V8--w}m&2JN=UCBQ^|cbJ7KU=dCp;hDOVm#Lu_ ztPn#U!vC~O|3T187=u*26RA5$;t1R{a};INJo#dO3Z&xBAnsR#M+LFOYAK5Ee?}@2 z&pGaH3Q`fTk1nRYB?7fSXgTsNzlZ};(~{mYjj~Ha2#k)osmT}$MDP-fW7FP6xq*}^ z!^DO75Qlj=t0mFRc1pq5)zw8xwM5Om_)G?;v?jxD3QLl}4A3<)Y3u8=ANkghRy8s* z;ut*k}r zhb3^_2E=f=r`dh+!Ko=@Ir$sZqZkGIX)!08BPz}UwTP+N5HFI}T;K482n+mORR3>6 zL$m1bu47M3Yt=jW+0*_ON{KZh;t%(Osq7j8vQ)6ATvDQ@nb~rUFcyu`5orTe4HM?g zZ-dZGr@V|oRpZv9O{vr4h_IxvZKXQCr7DnnuPE^(@H}}V32t(TXY6{Elg%gz8=$;g z>5TP^zr?o#p<8jis7p$^b!s;pbwjB)B!c4)q>LqsruooQTc#NR8HWhT5a zQ!fgpV?$JS46}qDwv4(6fjh`HHU1`gYbgc=O74S+xsGY@!}vi|&Nx*@#LJF7sHAp><0} z%>~YF1oFHug;Anjl7OV&=q}SBxa50@EM?@Zmj0-Y7ooF0Ma5f+2_YMbAftkXD1c3c z2gk<7`bvh(t8&)FQ2!NJRtxan0c8^s+0yVJZ!MyFTTOE0AA6%1Ac{2&LH(6=mV z^}RCP+uI9e7AIK)m1^w0Om})VvST2p8%;&IggS_Z^W2*Efh;t!ThRR*oW9ouUbQ!F z7T|WJEQdfn2<1W}Z!OoM#w_a7-zZt%D9Ed>uGS7vX=!bhgA9I>RO%}B5W2dHjgXvv zkpI->nJT{E7UNa#uR1N3-xPvAGbQh7`%rx~=*mHf0zt}|rdWAG>3MAJ%I#y?11w9f z>Pvn;<{Ef-+RF2E)#5c|{YYY61#J9@w`Ge3wKe?kt8hbWPD~0$Y?9j!%%%aH%o+EF z2rw)!gFo%Xh~mg0?sK|#;t}|EG@+$H$p@hektSCl|dj~tEBZe&?*|s z4X77_SW82oNSmy+=Qrr-VqWo!VID-@2xMy31vu5DS6!J_IVKF~H5G0nauiu=QLH|_ z-EBX9{4iNvI(EG9Qzy9#9Vq=elRBZGEJ6}&|Hf}39th_d`Qy`rm@f#3AD)>^6yret zL~~{eGeQl7PG8R2kI{$MUHylMC)`8SFv*cK6owV8S)~X9%jh1yqM{-PAME}0eNMDa ztP_qNm2|E*gtP$`qCArtRcDaBqayuIP)`tpsOpx|gYYjBd$HHL7_Zvvto>Llx?;m_ z+Jhz^#zOH8<=m%NZ6DiU26ZZ*dSoyFb17~ox{TWnpvB&$C%2;%(Se8rutKq@U45{f zntZm*L-Er&$GB3K{DOjY%Pey~u2UQkgqSg1K?kg%25aWcvP2Q@dYawM-F0*)ZxI#c|1*qmPA?$;H0ldgk|H6O+ ziMRaBc4SG>eQQZ zSarbnA?eefNEM`T3R=gzqAcAz%H)-$Dft>WUzQ0A;xBDua9Rh)Q%eZH)@WNk`{Ynt zvN(2XOY)HSW<_r91@NS?yK#+79}eB9(q|&%=4cDCz_ch;5lkirU%GV_n0-*t4nw4+HRe>lYh zn6cyqL`S6S+!1`Nj`Om1G%n0;*p$A>g_%*&73Wn~mi{92l!x15&+aQC`Nk;Pm=^)^ zWN(67(#auh_=9=Ho$!?kx0lOclUuBh{F`7>@M9zGLukEKz=}I0dl@lyu%GVCBLzUi4k_VFOH#@Y`_nqt0$BGo)^N# zKLP(jt%rS?nq?S%W*hL?;QDvJtOIIdLqTmnCo!&=%-c@92h2 zslEcv=dp4KqCru=9m)`E?->vc3McM^|7FRCX0^1ZK#$AzA4g1krg9fV79TQ~9ajX3 zs((^1ixe;5+EHR_<+I9%z%5qr>_v1rUYX$&EpH&7LQ-RoH{!SgW2U83zTcgQ2t8OJ zC`e8JT1ujZktl=)(2*pptz=;PiH% z$9}!cY@a&MEc!VXK)wh3KYaTs6X-+xj8!fUkH^QyT_erNE@HC|p6qMk{sN+P#}qyw zP@5TQymDJKUk{5~hOfS03(R}2w>(}2v;O?`^PyZ~?WLy%Oj-Z7P6m+sBYYCjj95#6Sw} z8B?OV;4A*d(%fS&fP%EgRHX76Xp6;WMB4}|-zXREa$!^aAORlut*A!K2@brWZK2a2=}6nvcGCG?Xi_b5i%Mea=U17S-X2x`I^BT;;Yf?<~y{XJTCn)#Suw zAu+{|0Tl!yGu#yNA{6|ZDj`RgBsak|GLDXpkIpS7G603exkWIRB-+C7VbsTj&H865 z6bS}>_Y~#HQ+}=3gC5?o1DZ17GnM*SJu0$%pmK3|aYzOH#pt{r53?3KG!Y0ndL7>k zW$TmOp{L#?`U&>jt9(E_C0SH!t*W7Rg{}M@edNFHZEpMSpTd);t?7qDV*@~NF?d%!3*&ezf z0rtJ?C%9&Z7>2j$ZuE|qMwAdQO>Cbaju2Y*$BwPYNRs|8*}1(htT0bxI%HBhnd?fY z`umzd`bYf%RkV$O^bE?Q<#E+WBU%`W?R8kRB>Bqy8f&1|xn{YX*Tp?y z3f+Y-BMOP3(x_X0e)KmX2>ARP_{S?kym8-GCEisRl&p50Z7Mgwgn5@{em`#dv2~Uf z_L#@|47Qw@$58sJ0%BjAarmNr3%;ONaES;u{u$Qitiku|x?lI%-N39-+Pgu6?+Nee z4;j0+I9zH_^lfSWh>3P<`qy|>QeR))Pj+SIRxB#D!(LyvPtk}fy}ej& z}TW5@Q5o>;o>l_;0biTP(INEKquYSvRxD z=a0+#o%4#MkMt$qYVf1`FFwU$G2?v7_trx3>72BN!Ti>5xn0wtB|eqY_#Bs_)9c%s zy~UPlR1_9JRtoKK^#*A|!n9E`q?rg(3%3)D$h^fFVgz(x8F8n|px>u45!wliJsoHx zWQ4DhjXC5xrkLp!wqhLer{vG&UwT5s4c?1UnWFEdkr3NwYe(B6kvp|mEPoA98&epZdJXCv_lXKt<(2`{4Lh3a0TL=8NY(X7~Gk zlsaNR`(3I3_D0dDfUtO-cw znT&h*0UXA!!JZQpg0U8cdawP%se5HuEdDXis!Ql}0(#}6fikiJA>`{82y|SU8%-0A zgG%hOpEWG8Lx*N#l2&=l26VxCSo8?xy#GVEn?QyS0mcuWW{^WC|EZ&jzMO3ywC?aG zCYZ|vF6Vx%&aW;?b8=)?`t$|<09?YLYZNr}9Lk2yZ2%Ib;S&fMTCFqTVE} zO0;<}k69S#+6u|HvLr;OEnL>G_s#o405F+eKuOF7fQguVBK__@$TUpPwj`w~pju)` z@WNd|H!sMK>n*FX+-HUT)xr~$zCV|$9$#%W~1Nd;C?L0@K(uVBU>X2eA zZ6XW}MA<|E616UliY5w#Pjf-d5PKb11v641{Gl&(kCr!#@$t?3BoP{zo%Z3~>7hV1 zZb)E3v@fOd4uZz|fDVBlRoa%;;HDU6x7Ds=-A)^LK)y!Vo(gxG3Pzl#I> z{1JAxg*9t55f<_-$>|jGq2jZF&6>W6KhtDvriRss*!d56RfNq~;lfk|EM!m9us&h+ zYg{2*(DH1Q$If+H3pr(Fp&eI`e;n$HyN4SJZ8gA%+;SDM} zAPg!E#v0?QgZrpF1SqKyj+dj`nVO>c2^WtWMOP1Pq>+m8PyJ@7XeDe$%Shzi$-E*A zrp67wd*qH@KfC~ACH3vlpD4R+2nF%y;1zq&M2uwmbrLtDv>TCrtwW`fw<4d04kJ)L z`LXY8_*tIriHX6RJ;-WK%6KudRDf~}I9FE&)MPNADps<|{|h|si$XI+QR3EJb1rWg za4VVAY0@kZPz+E#nVcZO)V$AfICqz`wd}3S`A5CUv4E>|z{j?=pt>nW84<9LR&6)L z3=a+MFISPG`!LwM&=_v)R?d+dhYe7zU4Fc|@9d>ZG~qgg<+agEiCw>BNDzuJjyU41 zR0IH;1xYecsY<(U?w~RwfQrxm{4NL{Zkf#v@C&f z6p(b--)-8O5S5uc8#*L+HCgDKJDF{+GSlP<=+Q2ri=?mJcV9u~h7*q2b{i!Oc?75~ z0V!a8ms(;1)+J(C((y7C?a2r5e+j0Z{(P4}V0J4*h*U0$RGWOS-AAhb$u-BWpo0ck zYLVCVTJ>RIyj5r8>Pb~Xc`mmg1|D=E(H%sp%P^k_*ryGQlTP%nLInzDA>$)^3uvO* z1n)(Q6xg0D{>X3u9p%Zob~+-|v-ENHWG6N7g={yDG@K~eZT4R2^gC@#!y$SWmP5RJ z;;S1Sa;{XFe>?G;-=bn8e^3t&;Vqi;_l z-i?ytrQ+=8TaxW8+-BgUH*LAp`&eqAjuN|_sGuT?7VyP4L0!$1eGT+H_r-9MQRK-S zMXVzpMXUsaIwSHw_@Gqjf=zHY>RRBg+=Jb1o=U%f4?)I4nJlIVol2L6qX|)=dI5|9 znHWg0p;sGhMmM1{f1W$ze5j+K4wRp%;Hgo3oHY_CsrT@YScJ=Cx=gJ5pknDK@)6zS zlzgJIxA%cp3x@*RAYGp86{tuq3(HvEo)f14no<|&_$SESm-cZE@A6F_t;X4W<~5)hDqv~iTFpDuniHpV;|Q- zB75O{f;Jn}lEQRQW@{F&iCu(CLy7TCdX!KrZE#E{sQ4vt8i`0OeNY{DLP0o9GK%q2 z;ZDq@xFSu>IV;&$KsdS3K5+u2Ni3rHQ%#!+n%bNM@nrrh8K`9tNdE)a z$V+cki@T6uIN z{xwn%FKlmZt4Ss&>EAovm3o-z`p9W&Ih|3wUkF^`F%MW}R}OE)aTpaM zB^$%xAw&^B!QVzza|@}}vKO`YxjLVPg@v3L-(e4(c2cE;2P+RzY`ufRP5pEB)~c3g zQIq#VU%m)csLpF-+=zj>IpP>sI=&#&JPBEbBGOVE8bj($UonCpsc~DG%P5f=$h8Nk z&S`%JcXXT$i9bTxpw&3G)k?=T`4G07R!+cbbfg9(&=!OL`0?Z0id}7kqF2JvHsTnX z9w_uZ@NjN8B#AJfI2nh#O`5KM`b6(F{;C}Ki*U$!q@T*MqX4=KcJ>sTF>2A1*f~(# zE^4O?*0H$#`*L8DSQdh5!=sV;?2{C|RRSr33)r|jp$GLe2(^t=13B3gJN2upi}Hs! z^2$d9!2jH9r_Izp;tUylh=}r&c|x>?7_OC3VSzbMwkqtnz>ti#YLyK;&)zDVQF7>? z7%~G8jaHm1pyo|4O}5lh9>xEA!JCV1GkUK6RE&qz!U5GiSnOrOHR2xusi($JN=`>P zq{-F)WRj$F_X`+XT3WUNnGNMmMuZN@ml9TirZpRKhFAd!16a*g5inrK9s)?F0~pQ}gtTBjqd!ceTz{U2JR0TM|ps$n=sUjOrF z7sPW8fd10=zR-2f-_MhZQi8zxl^_maS<4mZRUyqN0QYrVMK6{ZP~z;O&>s?ZJsgHW zQI89xljYY*#uw@Dgw+S1AW{iLIA4JbqX1YtU$zno1ITd3LStLNgtzq`8Zk!5F8fGhPB{TQUW-3FPNa#oWcB*79~e%?4yG_BEbCxsxn`7v5_ zN2d`F__gS(-W8Y6hsq}h@(_UqN!DRRa6o$(Sfg{RU;AM-a9jb2mVy@q+`|VN(kueh z_N8Vf0thbqN&gB2BnLr4AF7Ic{CAcD5Gv&VA|4xG#Hc``Aww`%j$ODSWsTBgn}Gjt z+XSfTYYq3E2V31#3?lc`iy|kZH>8vIw!ws^IsK7>@-#dsTl_tVCT*VKqLLKGNrbu%ZwJ!@1cuRmM9sEwJqa;yN za|W0=RQOnRLfU-vi%jVsN`u>iOxRI)*%o*5$yl&Zmm z=$@DV5hO4@>qBTp`1sM}<{s zN$_yDi5k~c2Zx+!B-adTxbZEFW`myJF}_vIF|CoB-9=q0Z%p`6lIL zw%pDx4l8VLbZmD@{jp_-T=m_CVppH4iI}lo+wgQL@2c3Ro2*7pC3WYGdQ}?*l8Qy% zah3TV>NHW-7YvK&$h2yV-QoM)yeE|nx~OScFd32f0TJd8(HmJ}#Q7jp5SbyLuyn6B2C{x*1-SN>tks{HW+KWb>#awX_ zMz!N_hXgM=YigzJ`nw9L(MU;AQz#d4&n^j@Q*6HxU2WP?elsicNI{r|`C7xmRviY`69Q{MgY2MjHFWI?iv56pe%3Jc>hO zJdzI$_wrU>i+k{LS3yU)+ocYRX@%Ji&Vl-fA5YKD4b2|x@ONGj=Vl!f*ib90YFuJx z|MF&)SZwjN8Drx|6-VYb#4706U+T!(700pnTg(*aeSYb2h2fD~O4Gx~UFOWItqN5H zdwc^v@eSziq4xvv#%PbO2QZE`(HR8MeHcv#CjOk6V3-_->r73&mJSl2%eYa~x1q8d^R5lfA}v^FtFU9o}AEGh@rdJDii&ibU{e_wF3+<<&i^q4n~E z0iVhA`ZtZ@v4O7J```7Hr<_S1)s0^L@<>r}-Bgcj$roPTZ$^{TX=ehTi{@ob2}Wcd zsf;KsuG?Wam6TN+TF9HArZBXwq&&*Nq(ohFzF00xDE9N#Yw7oZDB^{zH04@h?Fn zDFn6@!vJ`0_0Q@p$}=z{uT$oNK#FUpcI8lXtNc0JHRA={xm(9>&$E5o>Kga-9M6+3 znW0YY_K0o84IHI4Ilve~GRpvc%ZECvMkDG4V$IV9gNxxzr)uTu3ykd)${!q(kon54 z3*c`2$JUlNNliiVs0LroRgpaNCJtTWdZo1T2a>ZLuD$t5{hAWCdAqhojM@c>SS^(t z63qGR_S&l5w#qh2ZYcBWd%4(#R+dExH}A=^Qx7zpILoE z-eH}i&MxOK>vT(EnvCs}-gu_AHXr@Et{U3&@!u)qq@(s zThph*O`6i}B4gpAkuAPo3j;%qhTXo#YQE{2y`Nti;cRStl(!`B7;x3np=0^=^&_(_ zooBTxnmZzzb-KKJCzJQL^90}Oh$+e>kg?)9i=(fNQu>_X*%K^n> zt6So=^BXF?zqSC)&t+9#atk}8xu(5>r=i5j%{R@BL+$9-URR!;Yd!uVlL)V9nA^T| zh8KJ^?05wxM*~DUGE2hec$0*ZUKa8W3{JeQHtDeKQ+iW$CO^W0gSRxS!?!g2Z|0P& zdAHXH@Pa#RXBz?b-`rAWd;061niu#M8h7w|?ws3L<5^emBXZxXku%;UYVW;`i`Ioz zPplmI5mkLC_+`QeD2=!C-j&nfYPYMv+jy=}UBWg+e%+X(zevaERS~P~%&$X6X%UA? ztGzvoe2nYG(-UXc{^&Y)`by8RSanMyhwhTt@L)H7r^3ncVfVPN138)94Nbnutq5jK zitD~9CU({%j2tvl>HNl`i9^%4U$^yv!V$HP_$99XUC4EJlMML2&sY(+wQQELcw%9?**m|#$O_%G3 zJOJYs)hUysW4-HiADx4L%`{K%9nxG`5Xhn1G_mzemGe=JXZNbpJvr4%Y=4w5u_=+? zR=TyvqZGuKiHqG2jroa0Ny*q^uZORMCFRYYyy@_)n;#i*bv+rYSuGNg=@#6R+nsw{ zpow1&8)$5Dbak+ApL08J{i)?c zf^z{w$5M{onlU!CU-lH2_^wBf*P_WzZ=BBB0wl1%?rwk?fnzt&pmv1is%i?0QvDNGC=??y4->y4L z&7Nz`n-0^pQ{e4zx(cetI@x?k%m1^JS*?=$kANPh?+>au4M0fSex2#?@fWf8n!DLi zKfX8Z#ykE!A%T=+7y!M zw*|tFw)nd1kOX3BK9`!m-~D@4)wJr{T_yu-5$WqI8n*RodT*HX^3q#V3LS~seQ`cR z8Wkh;5k|(|!ytm(WOd*;VjZ37SwcDWZn7mY?fx3uOgRkfG#ye9TS5Y$oi$^c=vyLh z6E2?$3MI07>qzXgoL9o$9pN!yAM@OG%ruK`cf|%qIwxh1u5afpxiuVRB>U#w_T|&o zF=9ZYKo3Cn>ANajzm8B*Z2z!4xAlR$lwzDwZApiYVj*JO{0J112SFU+?2%P(R9kdN zf>)#?D?2cO-RPp9DACQt-DS?4%Q-nUUMUa%6gw0%b?*Jvo^ajZsO3GUCgp&a?iD*% zjGHVkjh#}E&q_TMxY@Vl>swiKO)d96!C0$-G;>d>$z%BuZnCP~+U5Bb!z z61E*fV2<458`8QleIQP=U8C7OsCaYVpnK}wTdP|qe=26`N7~k6o4TivLRnU}vNiGU zf1hg}zSk{e-#l=$Bmc(D2p;469LHO(71;D!jSEKv)`3~mkdXBu(b3DJSSB!ZNVfl7 z#GK64Gj^I<1|8eKH;~O9UXi@4Y$M&E&yQQdFgxY{AnV6)FS~BFketC~yu`*<~7Uc048c8tQKl8L@)MwHom zQBV9nEOcVPt^6($qL^rBTy#S&+RD?XAx5)Vu|tNw6A5Xh{D#!gr2dE?y;x4X@e+n$ z`f$Ndu}Z2NhI@hp)`oMbNexReRQtadY*DL1YLE$57P<*!Z-0u3hdQ$X3FZ821<48| zI?=``O&F$;ycZjRoZj@M+rveR7Vpzg-hF@ukqe`))SHkTG)r5!P(=38cAW< zHjPR08yBWYt|2iqAAg~VCw>qxp`{|aLMZaT{9;8O)_3Bod5?dgr0K~!iC_e1-AGvC zH&KQBJP3+Kb4HWCNIJy-G7dQ^4px{<@kic2B*9O@3_J6m34;N}2$njRk+K;1`wrrk z%NmT^aPw^w424gAfn` z1(LH=%n)w>tAH$F6S}5;GAaG%kt_%}aJH!V^@|qG&!?}gQGnK)B?wn7UX)P3=Z7Pv Sh!g&@NJB+OIdzZa`Tql`NVjMJ diff --git a/docs/src/images/mem_wal_regional.png b/docs/src/images/mem_wal_regional.png new file mode 100644 index 0000000000000000000000000000000000000000..451ee95414f43934ea661869ed14a3c2fd09d67e GIT binary patch literal 566176 zcmbSUcOcc@`;Qb+A$yC46_V`LFvcj7^uI|0(ea>^9@p@j*bIx_WB_nA-qF3u3x!rg*MiCEcCvvHN&bU9+Bv2lMZRx5^)SP*Lv-17$&ZGw{9QzkW@z_ z5kXw(IKuXfzghqKm2Zj9$S0j}n3(98+^*(3MKQSI+|A*>VvKS483Uce4B__{4UI3n z<}!+5HY=0f*@+{9lfMLz3Z0D_irx>Sg_mj2&@r&^Xhob3ZM=Au@3gx6Z}0|e^~imR zt@k%S*w03L4L$Y<*UfeDJ?|)-qgCHKpa`ua8ri9x2cQO6!l`-%9vDq>i<84fhkxh! z!)!kpexDKC>G-joPe3E>%!r&JQZEK}UQ}XZ5yfUWF5#Oi*_0;M;;hhdq0c% z(*Hz9u(T!7%LQMhTJ2l` z?l{J}%LYwnJU>0PI@v$J^3qOh&1f8c6ewsvTAto7h8LM)M7joIEr46(D?8tB%?JXR*F_1jWE zrz(4JWbtWTdep1hxY~9vwIyU&oh$Brs-^6sbHt+Wy{vdtk*d zkmu3TTOIt}rt^=J*ygGNpTh2H>KL|#q8{%+HA(Mb>UjemukZkAlJ}<5pkb2cWE)(+ zyBojm4Kk3(fffDNmc1QkjMTd?!PAIRp5iq0>1PdThJn`4HJ?!YnEt>f)g)L{gOg?M zmCWzWG%7vv^+m0{vvrr(quR9Y>_PmkQGh0DhoaBZ?N%)qf`s(PiiUYo`;voZBek-% zx7lseQ9Ie(LM6X;xdGmqyy)ecPidn~(}ME4o(Lr3?BR{o(TW3Q=M7in#iwPzfSy@) zX8`vHhpF?Lj$NR2X#tj;u8beay_Z31XPfR0%8VQ-;;2iN^GF_`sG-~0EG)cttlCF* zOAAa9Wmcq##SC3KG!|cP*q?jbTvTe85>?AvYUdSG%WFQ!QKMRFC-ZKCw^S$~v4&z4 z_fQCGjj+=ggr3J?53peq=wA}EtJyw_4mPb-ie%5dTq=zl3*|$;8_iT|4SY$}J?cF! zH%nse^_};Dwf#c1|BkbPJwks#uuEdl=56{p3@oX5k+6t(vIog>c3$tsf=eB^-d7iB zMvRi1JNgAWhqaHJTJ?=k4D#}hg(6mu(LMr8nRq8po*V!tz z?$$)${zEaU9auA}7i;U%yp~P;>c;)U)COiMbqD=D4FabXg%&g{jQoZa>T`MR;W7rS zUq0xa36C5%c6rh5f&=d%nP13RyW=y8n@%mu4!=*5^wu6m;<^OewCOuVMM}&x+!4<=49pr zzq(|SFtx!;^Mw`j?TH#B-z;hdKP&0bwUT&d$U3aJo|28p>2~PN_O3MaO!42gO%=Mx z=m3$%?qbS;kZi@~@IhP*vlXY(j9KZAw(ypX)r72@KIoX9TeeP$*wWv~lV}EznXjVv zt>-@to;Ub4N)+vp+-)GjOBv$djQ^!9sMI=P*=8YTP`0ry!^M0RS>56TeLD|lcq8WY z5xLYYFFYF0b9#sGLOh3F-|LVcdB#X*-74Cs)~J=pv+T2nFZuag`wx8%v*HFP3nezZ z(i zv@`LIurZ7Nyz3+R-1L=E%c6+pjLNV{Tdp1xzlB%g7qgF@8z((C7xa(j3aj0^_j-Qk z$&Im!IkB%NU&(mKucC6D!!wuA)B3%;SQPsi8KsOIEl&MDESAG)jHhV7-GW(}M7@2h zOGG@BBolvs8#~zG@Lb)}^4*T1^;p?ZL2y?u(mK!eg3x)A3qt)ZZ!GFXNn+#5PMNUS z8R;&3r9Zu~0)RE9`LAa%(7F=CF}?0bA7kY+7p0$;)ChIqIX#W@L?Aq}J1{Hj1hh5% zba+T3&-`+>OTK#3le00ZtKsYju7nPf2@3e7%gLTAOIr8g>JCL!&mmQMiuda9_J(e} zcm<)q@65OQ3tz=~>9?(@to7bmr0(GCh?upFeLf#?wDd+nrRM#*I0xb872mIqHboBm zj9oUVJI!ynV*F1}lzvlE0wcMrmQ}kWdzq^qakSo+K-`b?S_a(6UJOgnPhMM5_ciJx zS&hp&$2;d9yN212(IK;J`!QvW1o?gXoe%o6Z6n)s^S?-5-8~D-(_}VS=4<@brXR$! zQusi?vod&D=!#EHTzx*3X=7LA2#G8}9A*{D2QVNOMCJHQgXXNZJoC|pcdEADc}DaTav52YVcF3_5u5l31M4{6 zlfXl;qhK(HFA3#1*loezzebK>vUzCvxsjF`u(bK>^HLC6hm$Idt+*1LuS2VY-4Mj| zpMmRK8w{U=uld?gw=HBvz&h9c>zmh%3tq5(K}@c5CHn&$Q9OF|Ie%{?>eAmcu=&h_xx$Cs5a)G9xVL81g3%M)Fw>CB&WdSyg z_)6c{wKHz*R#z(w^S{>C8Zpo!@r=~VDxu+w&e0k& z)$UQfEGT{98VFE~_;}}bDe`lS-pkyELKn4v^0I%eLrce9zB9?Q>wTaUfevV7kpKOF zjw9^M>$(!X82ZTPGKqtcMaRPEx`0X0Tt%3HEqr4kzE-T2;fnAO9Zx+RQv-^ux|xz8 z%O7jiH4$08J{3fs*MCS8dX!OE>i%5u^gJ(_VfDI<;EYk9N&Z~OXv-{CM%}~CY#hgL z9=#3%G~)HWxkB%`+^k4aa;GpC-%T{H2_V#sZHi3E$rIAH>}1QV{*(#=gySkQGz334 zXLUOwfcfJ@N!>5$-t~7v3o2%vH~I^n)XCY&WL5?H+SFDSDg4s88hC<8WpUNWF`{ze z_R3J#yHc-SpF9H^F?`c#H!*aUpCb~S4lZTKv|6H4y)u)pXy@XTEyxEcS&Ws?{EW_n zPdzMUn8-%F~ENx=jY}fCmi7?(%Ukz9UK7`N3|V2j=W6;#c>^ z@O31Eu$JP3@f-bJhIMlGnkfLsTCE4%F%5R8b@4j{Fk)F<=zLw_ zVm7TqR|VXsUf9j0?;e>PCO%0WY5B5wQfpw8j}vy<`y1t-JqRc+Oo%7u#VxsG2+)c| zo<^UZF;q`p^B&?c>Kj_fQC;O=pWsUAAf6zAk4Z&&&OhlF*x~|4ms$-YUfP9VqZ}jH z2KKM$_qIY7P!!Yxf&V!T)aV@Dw{gL)xH;)w!Ff7c)>Re}-wE%Pk0hcC2CwDHIK1jQ z2|J^2bej?^UAeCS`EV5_^yx{>RSGgBb>$@fcz6DYn2YHQuxj05wboK^>-N;zvwwAx(kRDPzGVhnq4 zkTp~(%K5%YETj4Tj75|vJ?z7Pzgk=l6;}d}+_dxc)vE$y{nq%HWt59aX2FuUNk7&x~>oya=^Qz zPcJC1N~C9jE2H5zdD?sZME*7m zCAsRVjF9w&E|==J<=xkEGbw8&gD88;j~befJYYW|yb^b@LKE3FysZ+X-Xfj(Qhk## zyyYF^z}0smE7XSN2PX#yIJvT$t*ZM!s@KkJtRmDQPMp;o$!{pQ8PJ!ot3sjp8>dXW ze$K1;a$D*{#jp3j247BKd_hmZNS>g|e(F4(roej}x|*#AdWaOihGxwkBT~SdAKErH z_Bi-Q9(h^!YFpdw>?;L1vB{EOS|cZh-pq|4g>7CYfea6!nzEdDx||P}O1W0_<{tT$hb$gGGPb!nEZ1Q32I|v!X5TN| zKCO4Nr>ezW>^-qZnR~kl-TR**k|?uq!y&z~+W`n*RNLu&Z3!yV&LyjAHg(_&KWx3~ zeg;PbvkBG5u6JG~^t#T>JtG?(a*Ky!KFUWiQEBZ)WLcqADbsuN-mEgt zYam9ZsT9-i9tJ4S|FAnivfFv!rgG`vxs>?mHLCM)G)4L3=H|YYkvvpMB=$1+v3wKy z>)$nT)69Xew^bAci+24DI4$2JQBTgWKEuyEQCi0Pnc2%DS?>(C7+t-Phz9PFM~5s0 z-}yp&yla#|F0|=?HPgKxtJ2cG^MEw-gq#4!rArDR7mf#P5}RP^$Uo^I)I8IDuE;L0 zRAaS!Xb-hwhuu#C6W`6bZB2Y*VY(Pl1{l@#wkDL6UrIx?Cb6MHp7X<_AMYD39@W0x z^Cnymeb{ZTLm6ZS-)QnElyrlKMM`7|e)gj%)J6hofH9UiAB&H$e)zV-oISbUq&f zWL=;6e9%)oRI`yHDV(9by<1K|5L*-7{lwksLQ4<3`-nEGmd;JQ;uCvAl#fKr<5Rk9 zb@CSWeLw-fQKI>5iF0dfcYl4;4t`qzizVzXj(9~6*M}&doN9?yj416hY1Ort!bRvY zFs|&fo&S*T&ZGf_5=DG(ye;#^!wNsKmkId{1)s(biz{);&oj}>v2*bA+uEx5!tv2| z?Ig5O|EtKm*xjuKK|B2I&G9e9Ue`NC3B5fy)_n*)J)+xm-!L@oupYoTUC#q+To6iK`uhS&ym1$2hQWYpqD!@cV;H1K_Y`r$C*sQ zwq62*B0z$Xpa!_cpq?F6E*0$*xDBs2miTv;H}aLp_Z!!zC44^V8nC-1wNhVt9py9M zD{ADfR7ZH{3+aYNY1^S&{7ZrY0fZen{w8p-^S3fI=~%n)4GDd|YjVnGm7IP5P|PRr zGBm%)H%_1SeD!Zey#2uwr(Bpl-vGgi4rda3)_ncy=a#PbE0*{&!2NGdVeFmEe`-HU z72`B$Su^m<>7V~%utoE}GMee#tja0RRYm8TeORL@`}I~x+OmUv`AbO6^6uwYjJL5C zO6Cs^4Fy;zjU{C2#YnN3_yG*0TjsHcN{8v6!ejoMEO+f>BrELQTha`rRvhc&s%&1( zN|3uD8P(qw5_nN1C$ftlKz;eoE)wJ1Id*=2{wI@3DV*|WdSG-%x7G@37B`E}97L*8 z0U|z$p5HZiG*C+Krujb|Z%E!$lFhMEU5s+mZ-B>h}`^g$rVF1U``e z`k9~9``m^-e(W+FORjG^rELI8MKQrjy&Eb{WfFkUt-CQw?jks#RF=7 zeKdO^__Dz=6{`tLtg4&X$nkf#XOI5A*5^k&b^`atQ~so($4}pqtd|I~fB5j>m*d>K z$Yro%J`91an$_0k|7jRJl_K;+<3w8k1=C($sI>+Tc9RVr$b(0~!1XyE}WcpINR zVE69rXG9UF;tZVoma#s7+K^r0#dn;+xPJY&dF@MCV7lko-|3wZ0Y0WLPCn5t)`#y| z&V|ET2Z{mP4~8~E#$99}%18gk?a)1&#Ey<8xi35i6-1@3M^^n1dw-NGE6S&}we@{^ z@UEn}*yRYzu0K(t`^#EIsL^L;X8Z^eZga|uAM(4Dwk>NBHng=pbNsIu@+zU2HGiyu z`=dK+-n+?ZI!tCFPr_E(3Xi_mjOpGR6|rB$uc)^5cjLu>Xd11^PcE3rcU6yAz5BQ3 zz8E5wL3sqGdyZD)Af9tmvriDeyUocVEZqI``#)mZ1?U$4KfJDhG4iezj3iHZk1{}t zkzDeV$;)jFipT3M4Y~t1`U3XrGNJi_7@`o;!}I6@p{LY%uu7@GlS+glyd$bJ(T5$TtO?{h`HOk@OKymCSG$yOq26*tWfFy!cOVw-rV7RA0J@1zU)f0n%|> z3TWuOSXKKSlHCM~-ZwAaW+G?&VC{3!kAppE_fYgCY5Rxl00u)ly=3w^aBB=&9JCwT zTfF(=ufe}q1m;8HnbS~JY^uFE$St{IvjeCbv>lNS&?w(&J`fofEe+02dpinHS@-)O z>b)JzY^I*mgvto7nXyA*KvaZ!kB$77gF=b^1Rki(I{Dr@_m9?UFA2iw@JR`AN|Wn+ zGc}hQiImlSj0&bJ%)ZNsXSzuk<@CJd_(CSBeH=Y&zc668z_iIhkAqa}9orv?ntnwq zos$-xd~On1-3}`m?MC^rBn3Z@AnYe?wenVfeaZKt%4@uSKv8XK4#@KDfigAGOMP*4 zTr8z&2!O8+z4`4eRne>SIL6;Sy1c##_} zy*{Uf%nW~Fq&2WQQzy&jxb&qFJbWqxxxNzLC@UfRyZScMFr0QKeEZPk#PmSvaX!v{aPvp#IJFa9S6jN;!Aw{YDkSYz#Ih6$FD8XYs2Eg!SihfsTqK;F`LL8ic>Sn)g8OoGQ|ZmNG;^DA2-mF z911m+$(wDEG#;JJOmdn&h}gDFhLG?Ur##;KW2KOumE28I&*d=8{Gr5Xgd;n0>4}4b z5S883Cxnl;-F+M6N_Y3GULPT}hi1dd@MXnBy`i_b_Ifr=AH)4L7fVRu>=!z@K}{zv z#JGqzS}OK8^qY5Vcpm|soeAbCmiYu>EsesPz}#|XYa%eOzv zfAR=}D>zspz$7w2%fZ@axiKOk_czanKDpYa<1N_x1YxnhG!$Pr-^MI@Q4;a_ z93in)oo$zI(Ok0e93>KXUQYX7f{0=3j$PA^CQcLD^(Ukme}r-(Md)r~oXDY z4~1re5ET^0>ZkMxeanch`%^M}OGm_o#GMyj`i#hEl#%RSpU+lPrft3FenCD6S*|FK za$F!16x49)NMeQBPUIdIPt`2s)36@!J6uV&<+&JWwRbm~{dG!^qv670Aa50^ui-Zl zeO1fs!QwIdoM!cBiSxBiI56kVJoL{%34de&qJi)Qtu8TNuM2&HTGucCS!dzZw_qwb zSZ4W;q_XvatY#WAn>E|VUkO1QsREeb12P+>I1J=uT}CXICe7mc=Q@XUXjWA6KzU@@ zi_zvuv?te1khvks2xO(Skju@DS1_q=n?2%N>i2C7>6(oz={v!4?n$a`XX3YFNM!{_ z@6_FF)^fCeJAD+iYQ>_L?4jDg%Kw;tZW*nI9Ox7zO?Pg6)aS`f%`;xjoJz?PH|7Li z{Q{Q~r!v32Jf98kWD;ZGKtdsGxk_9}{{Fj*M~qcYQ00#4%B*RPHP3ZFcX0+KssTIPl9L0_j&QNhMCs#>N`_ekmfW{ zbT3xShIFq!_?dGY)8kC6;6leG2spfGUaz$*Qm{&b{qDT<(Fpz<|9vkwh;g&FrD`&Ta5C@8{&oL5}*(;uQ37?S_;g7d)dn*Ewort^VjRy+&X*K@q(G! z!{@sOSS`tTDh}A^c!BKB=%$Y+C0MJQmsUMn1!m%@HLt?(|L!L@#dMp!#k`0K=O)qTg)3T{%pC) zz;YiABsr$%JS~&qAOCLIV**MCcxuLl=Bl|*bvwpeA(#|iNlB`yUG7Gt?=3hl@i5F> zX&H(xCk1D}&Ay2jIE#w=;xzVV%f*;Nuc&7NE04orL7P*8y*!5Hyzjt&2sDT=kOw6> zzU`#>^~!mcDtl=#gpJRkQJOmz3jWE=3U>YUG&e}J?53mo6shf?Fa|~D?z7ZWPMEWT z6WZo55XdhV_7bH^n3|d*P8eHVEG1Mjoz z0$IiV{KxRa=X3Va1t5SF^(>keX~gRiXGYFa0#rrhX47sy?Rkk+@b$Wo-7iNq9S5S%daHqcOveXJ>Z zPh~=9eacxlMr-|7Kr(vROr5Z>eIG@)gzy?dR^bPF%#GYGo6%(7w!EbkH>ml1_gepq z5Vt=l;#>??$u~dq1PeyQtTY>D?#$ou)_fJjn;UskB{*se`Su`+8Z_K>rt-JwvK}%k zId(qA*nU(knfQeuNETL&VJj`S1$%+Npg#y2Sa%0`!9O>E6-+5$d#o)@-#S{L=!i z_hPOLCES!!9ITk350~|+J4~}!Jo^(B^ik;}*A|R3L1i@-isy`O=qLGxBWGU+ z6r3rau5+0GhS>D`Kl}PQ>k`8SqG5WWTgy(LZ+{m@cLqk zauUJ}d&VmK`bBv{oifuQ0L7%?YvacJ&Pq0v{?i5H3x(fuKb4bcb>v-@QFlETE<4UV zb=~BPk2rOZ3?Cwe9M1y$vK7v{X{cW-2qhI}Vcwy+*m>u$>nuCg8qQ)bM_ zx%u@ICQYbCr+G7>G7Y0bf4xB;z;v1B=ArPlcDQuAG29*n0@v%1OTvXOvzJTlL0A~I zeGyv4+iNI2^NSu#Ma%o3hvjgH5&BbF zc9{6hQw{XvuobYE`)Q06jfnMAE7Mq4TnhIAniO#&ti(@QGpGwh*Ekh6;$sV`4s(h6 zb%G00+}(DjW*78L!b|Kw!&s?I%tIm1qv18YRZTO)vijHaFMDlZlOfk8g96Jl-}RMl zWYNPnkd|{(f=qp!tAnO8>@cnXdpYJJa$<5lrNLs-ra8ZXziZm-rh%D^SMRer0ByrM zuL-Wtr8~-yPA4zpY%tv^0 zpUMyvxKc}(Ax@1}2QA@1`Czd&XbOMsc}}R;627AGmXxd8Zszvj;h7i2;~Wfb3bjHj zV`j7U;_><2YZDq`gpyf7i0oPmk}yEB9ssPFdXIY=sDS$ zr)G~_pG0=$89FYF8#{(*0#1h{4_nGsrUTL(>2BouLSKrQQ}Hae%Ql6%O`H7nbP*mY z3@AK)WK-e&pc1Em!ZBS04RV@h^yJ&jE{I<+Yq)VfmcgvWq>@U_Ua{!_K=E4cuz*iuoqI6@g0bA;fewQRR;)X&}*Y<73u=zN!_LR1I{QyK? zDQnr!l}ub+C`F!Sv%X=o^hGNi^2HD?B@S+;o&^KERpgN$3>El^hN~y5_0)E4u98&$<9^)jYQTJstmAUyc)q()_b>nKm=;U9p=X!@nl}0nj3S0ZpEavJ*Q2gmA7JdY=IOG2czRZ(Y{nH%EcPO zm%bX?h_xm=(POjp-XsKpH=H1!5|wdyJIqEax#QMH*o-6~h*GUYZ5n*T-(&%N)Z~#@ z6l&3?k8op78O~KF^tkR*-M#v?dzP|M=-Xs9(E>sX(*&Iv~ zzunL7w2+j0oc^R#7&#-{J;k2#{X@@{$}mXURD#cG)>Dyl@T0h@!gRIc%ONF5(JM0Q zu)fwh2a7hlnXmb^5Ss)A_;w}+w;?G)B6F&o$V=F-ym)$DjtCY`va)0$D1We$yO#Jd zJU|F&4o*W>ZZ_bj>>qTGWjC1nK>Uhw_un8a|9f6ScTJT3?YF6>^(S`O?8dnR{zje2 zMT=>GJwmQGtPsTu+m&_=<(po1EPmEXvRK;UY=9vMA3Boyn1TjHFPyM*t z&=tZH7g#J@*BvgT$e2R@qt1T96iS(fxj98)H*)5vGS+`W(QQsEA884YC2#qX@Z!(s z`M4_IR;rbYp8lL+bh|?=WwHb7L}bvPp57urHnaCXW%tL=f+J-NS<^CkJufM2jI2Ov zKQBOxNxjlinI#nI2v*NVee!VxaP#A!_-b@SK23jj^B?Kt?3ov#JYk-qAZ{`K6A)SbIJ|(chE4KTG;ys_c zDITX@5L~pp)ebmL%oF-lj$n~NQ7yV4bxf|*g`@IUfMG=-ZmY}k5#iI-^#%AO{~(~W zb?f@%IG?uMb&dr{Id3SC2XAQj)wPp1Xu$OS)zQv*Wvltn`Om9lZw7aeC$@xbL!FHm zf6f+)tsp+RGPi+M5r-%0P#?T?<9Y?M7NkNy8dta{Wwn=@#ncvF&Z+MFcWDrv(u=WFHAr zE?Yo6P$iY6I^;%eQ}D}KO{i^8syNbOKX2+bTVr!`XDyg2NvUiLlVkq?0CzTZ*SL|T zWUAxZD3szXK|Hci=Q1jj-PM@6?ePCr#)G{C*hqia^$uwhl(SCtM6Ezzn?!J8PW$6m zE=h>#Y;5nE&(gd`JXc9{aUO7}vk)CA=lBD2`tQ>|pa`j|Nh=rbA9TFB6t-q3*nFTNxlv7_cApzUN?8 zP8rL+ss18%=+hkLvW9+4Gtq-4S%Wo_x&Vt$5{Wu%0U+0z9)(*V*}`SByKD>=p4$Lm zb?a*Jtv&hXf42R-az{CF-!&o1?n#yG@6d7G-kWrgDP)(rFW(!7NFxMP5s6yAOLKkt zi*5HxS4pD|5(R!P7+Xu)0LG%7X~ZQez(JWrpN;wjm0ZR*2Z7=FM6I<)Zfi$9K`1S4 zNIOEGZUIlsF5}Hbu1(UE=4AmEGF)RNdIl`Sq~rMGH8+oH!q(#*!}Dlz!%wF^VRomq*@p z%3Qk@(v-ynFeyi2_7a8)QXaieP7E@O06_YMPyKVAWq9148~ZGj1I|+4nyR|ekUGZMKtzw$$#2O)>*> z#+#sGu2HA^^7>+>@cadjr9}90qG;c!P1;N?|AIPG%0&GnFLkA@%!COo_RBmXj}9(@ zSU7S8ZrrW$l)uJ7lDPi&mPsj<;`zPV!T)B+uqZ(ExZRZ2>u@7tJ~|;#gW1X5Y1TNK zVp*7IKIa9aNTQhnv&}Wn^md?`(h^RZbR^NVy8Hdv#Vs!2Zg*cSu0_z3#UAJQ4`6?# zrU%#@bH6-t9C4$2!ywk87u^Hbg)-jx3vE`j(KKuEC-OVo3zbvUTL&a=0$W}`>O${Oa1-Iph=D*5LH*1u@2e;7(P!dyp0tn2^u)3)fgUocWovoEz{ z%RINdw`>wkkjAl(P4+o<9C>Up_}T|`OPD8j7tb6Mp}x z@90MN`iieG!2a>^g5&xKbNIN=|)wRGNUa7*NT4qzEyUIq%@z z&yg#RNaMOv3d7m>_`8Hv1^s8)ZkQ;(&OxsRn)=JE7!5N9)Kq$sufAqrZFNMhL1nTq zAplup-PwwGs6a}kA7Yq=$X$;-yZ}e8VG>gm{PIhcQr5H{h>Z28gaamAO_Uu9LIIR& zEz@P^g-m)%!xtt^BUWYwEP7s$KsX2Flig*~brv&UxfcYuYG6=3juHFW^4Cs4aN1C> zLkVO_>X6eB%q~!Xiifu7iYzi8*^OL4np5G4N^0N^Z?Gj_#6+V0%Six}+QwmvzQqwj z$;f!21@d!Gx`mVjTy8-XrgG^4i-r7d!7vESPnF`*Ee9A@5>Wn*Z~SnHbDL94L*Q88m230A|TwAu{Pfx57C?I*~1Xcnc*hC z?CEW{0O%mX7kFzGnciyvO3BLoL!jKi1G*jBJ2i~^9x-}bmC@%>kB$|91oKC0X3z8y z67{qmhirb!!cmQUbK~KVkPx+})LfJ-)Q6tv3p`8Lheo{?5aW*T%7mrbp;T@O{@8Xujn~FIvm;jt@SF{oTQ>Yl1#jjC>YJY(0K>_q2 zAe0b=C;|{KE$l-OvV@9RVbD0 zP?cm7U=(@yGuK+_lDvQ5S z%YK_qcBMQb(SWb!T%PS9?ufh!YZv>0GzS!k48*3An-O_2#PC3PY0?q75`HBK1gddR zV;1yw1pJU8k=iUJd+%YB7k*cii-u`Ro`^?3=><;9s^{}sI8J(#7{wgYA5?D&F^LT* z&A-bb7-g+*@&P>G0O~r;m$P!BJ_8D+Dept|X=lpU)XU$T;{uiYM`%%{b-)ber{t$y znpauOnOO&F12+RG%+i9yhEkdD4k(^ArvvY=v0}crO$7FabXQP9t@D4QkcX@`WrW5A z^4QGuKwDB6IkB`IkAwVa(bHG$s*rNtHn+R+P+2e_A5$j-D+LBG+I!OgAA+d4R5!-| z;0&Yk=)g=p?&knKl^kN^F*L38xT*1xk7~PL0`h~$f`NN z%P$L5O^VvfiNu#rSOIv+J;g*KPHlaE87!gMI7q${3IH(-dP^V{8;=3}2rU^lrBQiSKQ7J>l#M$Pb2R`il?aQd&_HA0<-8T>`ro!kY8*8DaUEX{(-T zx`n>?)KKDQ18R+T?pqnufU0a`-uV1k>^o3bh>)dxUoI_-8opi~>0i{>+Dh1V;Q68YmWicZYJxpZ+JEKQ)upc75#q?*MXS2m7%lW1^hID z5SnO~R{A~B?nJxWg}qmONbf-@pU?V$)_Uy29tlFuvK)sCZro7O&f%JBCB#~1fXp|Z z8&GJ7hr%m&pdJd+9V(*y8%ijc?;%pB(4-gKF|yd z=f`TMXLzgk%Y3`B*1cK7YY`xG>9C0J8$ArF$X?-HxC@{lYF0t^Fb1427kuC51C^A^ ziTIn6?)U=Aa^-kt3uv$_gAVxXO$e{|3CDdX=W-{`S^IP@1%Dl_m4x3kByzy`b{F8}rLDzpF~Anqiu=Q6#ZY7n`Ufp&V3>L4Ztiq$I&_ zQrrv01|`IY2W!rM2*{jwF9~wMK|y#3kbpeikm$o>(9I?2EDUs1vZvlu#QQLEwGP=a zS9OhM`CH0bAs@}Oi+BuF+@S*B$t3c=j(?NzsQs7@jU)}|LxMukH-nXuHCgLwg$CF2 z;Vr*}A%BfrjX-{!>bP>aZ;4V=EUd3Lx2bgq^i5HrQy)N$Q0}u8bdm(UR#^QN056z( zyY!+*EM3Ab&;f8xIQilZNH*jzHU&b3F*tIaV%`QMUE$P#^EI~Wn5+dkEOORW)-LKv zkA!XV7dCi-`G8yC3a}9&aArYw{2G0}C$r2yGZ6KUDhS}P=FldqWv7_W`AV`Ca-whs zpbBq5rEIk|2<$@*PI4{3K*B222~yxOvUZ3iTo7oQ6iKoMCU}AT1Jn$4x5+)xu#8BR z&O3)sl2ErSS3P6V2~&>r7i8RF=1q4PQ+iLOuXT2>bvoS{I60X=S#$xpDCDC44ZZ9%rx?K#$Ewyy=l+JWzdTBM}$|y?YIe4c{uKd)66L7Xhys3N?krLcuhd zS-S)9OCb_v(>=?5-oizSEYl!UWw3|fDYFtGn`SE@`5K{Sy54HaKAE3^I$j9{y6UI< z>(8gds4Akq_ygu9k3Y>ip-VrO$@Br7Us(j3PE>WxVm)4XzL*WC$ z$}%F2CMLD?0%Zf>pNbZX|DhI5zfSn-_(yy7&K{r;@_c}~bq><+`nfSdTDI6AMoFEBPzONqIOl8?|alCM$ zi!tTbG}KB{?x!0yp55(`%xr>UOF>|=^%)=+9%Qx5mCOSu-Y+hFICeS*eOF;OGScbp zzFdo*Ce)Q~DdVqrT~1&cjFw^^>&zhYj(;i4qK#3okE~7tIJNWg>Op;1aBXgNs?BOl z1{Iya)R+%C*ROy@bG`5Rw z@Dq=c02SnuHy~yKYJhEw2e!1)UNLqTmm#p)&Mo}b*_Q%)HXk$tTgbq#}m zG~nz8tePl7XCP7`Cg)cblxEI?gh?1mJ`rG-ji6S^1}OUUR-XZEq7-PxQfvcpq~X0{ zVguAwAOwiC7|5k1)A=YD{qjqmhH|L$!3e;ZcAFv@j1bn}HWB{L#dAB8+|Cxadhm9a zKVazS6hx#tSg>@Q$DjiZk?w8(I8plyws>##X%h7(R}$I#AvdG7RIS_qWzw@**@YJh zQEB#o7Xt&CZKOR{0U&E^siEY|mf93m1g*vu^M z%?-*tj?t8?7ElW^3se-Tvm3kw)xnuU=YA+_b()STsTuy*)5CWZ`?r4mza6F5mV(km z48srv{vy<$;3c#?jDks}xWWR3z@JM)nB5?*v3Y zqTnVb7Vy^Z*c-eSkZNFiJ8T1IsG5}A>jQP*BFO5j#PR@*nxBH|-%if%${W3ix@>JL zVyc`&Z`c)wQ1dHYoDQ0vF3tP!HIsSEyPel|>7x5Oy;_!{Sa| zE+&=Ptu{US?x%t<%5A+2yoz>uAM4u)>-{W03C4IMG;;ljb4EcF1AWxoe&Lp&CVDb- z24swuKS>XxJPz1!0-`kDeVR11L!Ex=?Od51!MA3S)#G?epnpc-Nrlr<~C zfGoCXhYspHVd%F&+>ezq_&NLYGZ<4BfmG|#oZK1`|q^`L2c4c zubSfz6S^?JC;;V2S%`J&Nqh;NIVtR;nv|e0>~BwMY}DB;aL~ST{J#f>g3Mht6igL?Sfx!=U6E?WwL6rW>uMEjxRA2-5 zA|3c9Y*7$(3FSCmZVaGz@SCt+t%H)T8Smt^htvXCu!w;ApF*`BC@^vTF{%5PJaTW>JZGVfExi(dh# z_VS-gp1Viz#Y;QCZ(j>D0d9~iAj5uJHdrh~0GRI{@NZxdh`R!+;EyHXlWshe-5#Cp zKn?&{E7;J_L+aK4Ku;YZPCM|u9{`mBvV`7Ss=lhq)ev;4D`1rUUaJ3Fzsi=lL`;Bs zmnVBX_IKL8IdsPWceO2S$md^%)cyzk1S2`w-IxBq%YdAY_W1s`^+_;vva_-E)E>ZCb|=zvY*ppMpTiO?w$q&~9FWjf1Iy8BS}z z=MH-ckN;~eFkM=3&4UlP`$Lb6{U&t1bR!F`#s17=&zT-c=%R-YW|D#!SRS{Fd;eeC z`VQ(yKH(CrUefme`aVe5MF=i|V;*H6>CWw0L*N6nT{rx0?9FayvMFH{g!lli8hBl< z{{~~1%Ive*W<%Jp0q8dXtivtuY-Kh6sZKQq+L*{!03ckI0>?m6doE3ccTm*MP7V}2 z+9VUjX&gVVy>s>X#%X<>KtWj`dt59R0(Ab8QF{s6q-p8WeK9V}76Mm53Mm(|t~92Kp@y8cBJ;rIb7-*sT`d>9p~8|NS@bjfSoR!DFsR z6g%X`FyYkpwHv!N=(nkE|MtmOsQlSsO3$(JQ0+F1vV+~pg24ffwZ5+Z{01A^#P7`7 zm-j#Y4T~y%3c7$kgw_>x96R_;2OFAm3nKlFv0Mw;h=ZuGv1uw>o^3a1`4=-IjTN~B zn2Mmc|MSx?53pSJ*+~$}1oT_MJa5mfHn2e~7Zi5cwF7@l3uukO2-Xo=yMF6{fk}IR zt?Zi!gcwBAJ7qmvTsbWXn)PmRKsxa+%ilF4@DmnpV(*uNHtOdU^x7l`-q1i`V;*E3 z7GQs|(DvX@@ISjAwYf$AlM{(Ff&ng(at@!Krfjs=7#Bkt={{Yws z)K64w`2P*t{ZDN|33Qkaq*OR=tpDFY4*Uv~l$|WEPrj>rP;J~Wd_cSVkTH?#_v_>D z{W&A4RHv$48}wHt@g->iJvvO7Z}nn0-X7VdT2-ndPqziK@!~Jl1^)yfQC*wf*sT^7 zl@#<}dr%*c+F!Q?b`RL78!y==1I(!)a(n-eN6ov~1sI42R?`RQd*Ip~-UEIgNcA}e zV8t;{Bn;$4`KXd!xauULjJ885{ihUm#3z*uWY=Yl}(bSLrM zH8~RFhXc;oH{h^NEtMP{Vsodnf{rOcG@qW zmDT1wR`nbIupjSz2*!a<&qf{u?uRDuPRDTj{}n2Lg3%L!Th=6ehCTOtDrw85ZL9^_ zDIN`AC~>29u6wT0l*Y%y+|-b*nDIyKDKZE~;1K@Qu;X`cNfk33WLH=qeABYgngeM7 zHTs7QRoqdHzgjyQIuVG6w=p<3&LDYQE8fR#Ww+z}*|)fWz0L$?0^Jakodaq1t5b&$ zvXV^!dxEgku&8jRco4^BPsRlnw2$x(1r3N`7l_|;ZvMZa$=04hHbe>W4zxaKJ2C|3 zlT{2348Ly}sZF`bGM=leGs0a|_`t8-VKhKc>8#g$#|PVa3L1mjuU}(xv6H)(b-gb$ zhP#|qB?<{tYl>`M1TT~B#LlV0A3K(vO1{Z*F?+12utJ=r{@z!n`y(B2pM@Hjap@9PxPgk zv;Kh;)7L3D;!ZTug{v5UxsWZTs7BD|y2^Eizbq_}Mo3Yu<)v&A_jOW=xU$vv=@~=K z__9g$nyDj(q@oTKuc>-v$_Pepd!8Mv%kQ{4I6cxC( zN|2{_lRsS3_6?%$a(Go@$hDvuQXQQz(Zy>*F6Q?NnJzTBaNWmpkIRg>rr38UL+=G9 zzHGlzrXtqe`tL61RTX+&f6|4&ztEJBc!EMwW94C%m)W*&+A(YtD%`w*k9Oli>2H$S zE!=NOYDY6ITGv%Rj$j_W=g?|z>*;Ov-maK_P)wlSW?U`R*JMd;a@ltqI`c3zi zQJ05CMk#Z5kx}WhD|nB~2)jh}K6?NyFt01^I=WhKiJNFqbyWY6KeAdo{60n5yvU00$g*(Ch=#etLo>zZL_Wv60@IT+ z887ThJ5qH#TezPl<|$DbTxY2tQAzZ=Oy{p>o}^ewjxU*^!t8+bTq(^`i5}rKp46yR z#g|NK)HLleq-1Wq7e)$y;`w2rZ!&eBFTE)Zh@W`oOGVlfyE1|yfu@88I)A?P^+%Ni zql$*+6$E0X?R=Fl!A^@Xz83B4(#nMo|e&&^OPQ?NkRPLEr)87SpanG5K4 zfLVksot()ry!`-Kn+~3Psb)RQ^j?@s>Z4J0%a+R0G6GY+_Ountkmg*A*2>PN?>khZ z6X2`1esLoXx+RJ;{PINtT6RTr-R|$hTtzAWv>R|Eq!Vhnd#Z(uhU9-RJia%{WSo*U zBqpWU=X&{Gp-aLecXK*8(?wAMeDTr4hfgdy(s^6ar7|Ul$tKB_pC1%Y{2*p)1U7ZN zva~~trT%45JMa14ggm9B`dlNwN@@4_M-GZZma0m3Qr{asWSP#nDPUgXeVI5bu3Vv9 zU{LAn!?)5M-gN)R*>}Lzy#N2}IMG6j1}g2+AVn%V(W0q6G?XTVw39kqG$m;(2Mz6Y zqDg3>lom>HLmHAO5!L_oK6UEU<$mw)e;*I`ajJ7Z=RIC~JYS(D8*jI7Y=W%3)KTFa zpF?eBQ3J7=yFH46((eg#@1CNbxR*&2*84hFbmp#_!n4eGZ9DHg z_T9R}>_dVfm$9j2pK4ZNz=v0vO?8=H*=mZWmDVETr2hvKa=%l02b zdriiyNPlB(m|SUiPjN6zs3YX|?(FL`NA9hY8N<*Es)`xO;3z_WGR zpV|C6Pkp|GB`>cy+$QIcd3B4WL(rD(nogt(jyhGjLrl&*Y1BQ};@gR8=8pAmE>y5I z#lDd5$m*=+(5Zfvrvis`*i<^lrR~@~rMkfo0Zgd3O(pPS=1mm^L-UO%#17sxSD=q5 zCzbm=F~JSa3%4hzpr=r_@iIToni(_NJ49fs4&{p2Apl(OQ%(PVcxv5U5t)W5xwpX; znMBWD1*&VWTj`UM7Yzk;SmD#61&CwE|`f2xcWq-hQ)`sv8ZbVUl7je0k_jH;!f-lBV zrS_t&CTM`Cm6@mR0o4qKr9b`=B3sQuS7{E9SCe}cnb~yhZ_ka5*#|0;b!!}0M-5Di z9>215$Q~Tp2lP~H=uw&ZHIYlF{R580KeMlGw?m|PdvU_I(@O7Ha*}NGy4YLu&S??Z zTaSeEP7O~a+9ZjJ+hly5N&~(D$>hyt=)|O9!nRXrsMM7C4mGygGeE zUNsQ6jar$q`UCF@MoL*mKi`&03~$~69!r|=b&_Ecr>+zG*8$}{8$DZFL*IS2c1+&V zyjc+; zw%Y~(m%hJ|;LAECoc+Wtx*Vu6JMo>qv4RIjJ5y6<@e%!vqsn3h+U;N4FA4!Q+2kC2 zOFJa^Lz<*Av)=w-X}V{UzJJ5k9Uy>)41a9e_|+E3$|e|o2p8(e;7dlm(BG3C_bWsi z8(uTHnW@zf;c)~0)>r3iXkN`M-Z3O=mC9AiIc(DV%GjwO#@pWP_1^9cLt_KCPrtmo zuex>U5l6vu4xK#z$75=mhS!>gM{=Zoe&bBERWP)QFmW`DZF(S+_3D_JLZn7D>z#+U zy?3vXztdGAI$Hh8vAZKl*{Yh^h1t)mYnx4Q$MB1Mf5UrLiR+s3S6B8Kjt>-rR!;mj z`Bk_#qtHIOwDPG#`^4VNrz2rmkA45vw>yo5$Dps_2Q>zIbHueHVFC?o-=aZs-dvTy zW_v^B*YLWL2njLK;0eu}H6=8@TIB@OcRcD{)##|TlMNI=S~bgwwyEoQlcN1VJ^8MWFZ{(D zQ=j9kFbZudv$t`gj-|ls2$ROC`r^kYh5A%?7iizv^?0oKZSc5F_aXC!2;g$L&U~F_s2dg5OudkN#*}C157b>|yslf84x3bD}4z)4q6m+kM$lRQ4;_%t^teQb_ z@DKRo{qj{@OtbGpFP~jZ-8W;Ytl2wo`qN-OZI|3N#RAL7j-iR-ICBSu%5NgA+*>^q zm)H35rOR5`eH)hXY#Xhfv@<^u{E4q?yWPaF%-x?7HGXeToeRhtfAWp(b?MLg+dA?0 zRLaA{OZf8Y^L}dA`0^#2WOZ@bP1=Mzk`&2nlXU0MV%~c;%-w1=oQAzld7P!yX7DGI z8^zCa+}3Y>&#tpJ>KvFxrX}|6KMkK60~0oaaT@Y$S!J z8HX>t_x5@Go|l#7;Y~Y}r2;}d*RQwq_)%@v{Y|))?TB&~U)qa1gGbhHy+=DrwAuzY zUN3U0bI7PPJT5ai?=;+(iupSx+pOBw+rhcD?&8ID`|xa?aA$B`Jo)@ zG3ZPZZ&qBR$xg)6wi_1wHYkW_qgM~a2_lbfXJiXF$ZY>*{j!mI2`0`RG-}~~Wo1|B|=5@Wu zvkLV&QSs`NhT0ulbebAo!p~WDN18`Z+_iCi$2@eU%YUFBgh1%J0z;<)qXw%ojnQP2 zo5o@_#sj;gdK|(#{Qlxwpv43M9!^-Pm@zmfPsSUbBvwhW1hH-IQsF(ZKH|crAU6BA zHJ@*sv^`qEZeCn-S7qYk4v7x5Uz3}!x$Gipd=1TBPK62K{-w+_CaH=rVK%segJ>1` z)}z+oePtfFGaYMnve|M(M%H`_5ePZlnX~Z-YaG%1K-7)5gI|hQ&$)iq_`q_~TCE4E zCCd756ubj3P^@7)%oM(iIlv=z^tyUQJx9*e9;a8@Qfz)KJk26+79r%qT`Y3-RYe#_ zE~tiLqC7(Gb8S8ZjA{^tJzXjQj&IGYYI^Q-QQ#rE#SMQP-g4R05^<)Qmj#J4yeYSqn z8*(gw9Pq$dqn0quk<#^E@4oSEDaIUF#(ZwmFt+zZIS$4|&M*m0cGh)M)?Zt5(5wFw z1%<%G@wBNa9p;xA9Z|-eS+Xb}+N1TV&+1puc|l~r)Y{W?J8EfZzoE+?qI^SOW`Q>^ z6$ief*gbj-Dbfb%WEfdODvXEfGBX$C>OR}yA0MTtdj2gGRYP-67#w`tcO~n)xz5{84It7%+j~ipm%HZs#nlNsRUQpUbUHD;I@K$`?iA za3UZ}QcD_{VkOVKRwye6HJj2OGH+O1+0oXfJe~Eb@TR#3Hm+slAW4FLG@*d17m79;8G~lL;#`ecgP$^#QfrD0Pe-`)H_$(aZ)@P@5IJPZZRih z!og&ofe}EuEAOFWo>eEsUNUR-PUkS=bA3AotSe0)%wdA91+tAo$8!j$i;SA5=ap7H2>Q5$js z|8phj95$iZB8TTB)xiHDfzXTvM%R+1x8ccepzamr>!v~f zn|4-RlmqJ6+TkK2~w~% zA_a<%%}Z>`n0nP0V1+`3r$u74(nf^mhN`S#VZ&Em2F)o-GA;f4f=Ew>B$o=Gg{eDp zoDqH?v>YClCdtA?uBw^+awwd9@xSZ92weCA*6@a=rspifZssa?u~KaP<3;xkA=@xY zk!`^J=OmJF{Gq%iz1V~HZ$86=8Nwa_N_b2vw3jqou6UknQJ$i+koEY#OtOO-0u{c} ztNVNy%j2<4ut&)`P2-W#JQW)#~JYrl_dM$xVQGD3Oa?*Dy2;mkq2 zk>`xrO^fWqH(xYR+-tN*%;7~)jMEvqql*@t!t!fp*8Q3YEciMvN z25aRi=G&hNq6}Ej=)A1kAuktBMRl_^(*9xk!1uDn!wUhfH#(S4e-=Y=fmJXWCiY_) zq%j2i=PN#BXiaA8$fcoOLH(vq#G5(h9AIJ;`GcWMI)OApqz_aQJHFSD67+9b^U0NN zJd(f(K1n(+NixR*)A1V-PHK^uQ%|OJ2+zPV=}~E?dU{YDHfE7J6gvq|d-98x#~<@V z1(&>V5Tu1th_vZ|Z=aTHU%qja-CLY+c$9jr(tE}loxj+W8Ao*A;H1=FQt@T z`u_PNz+Iy*wC4R>fXfdYB=S1Y{DBX^kNp zXZfbS>h^<|p#7XNlM9w3p&N@0Cw8rU#)ZL#@DSEee28>Wqts)Ijf2jE*405wGU6NL zJ&$T7U@;!p!<6(zcuS29cmE~OhVP)4-MbB28@@hPHJmfrW|vb-ioHR`YO=hTJ0E2# zL&}dMDpFy&mE^=P@Y#&JO-;Wkp4{^5*r80+abIMtN4QH)o{HDUIH5&Ri*Ms-EcD-K$Qo%rl! zOaXE0-R|I5&)(jCs;)mg5KBWmR~5v>=iIr~^W%#IY1;Ys`7E*whu9o?TcI+IAHWz3i?9Ph#+-4g#?D~r~h7^wKbW9k1(mh4Biyyk(8`-_Y9aEM|5Cxh04XBF3 zq&?rG32VfWyFb%pI!yRTL#E`1iTb_(3cg<6n-sUn9fN_z@uLVh2jzUd$T!1!xSB2zfuj9aw^4 zC)Gwu2L)L=|FQ(&A;-(WHG)c+=a=El@YysaOnjo(jFL308Stdx`=Hx)kshqzDH z?zkI?1`9|7|4E&K@M9sIuN>zx?aob`l;vroD z*+!m|MT=H4#)L-|WAk_jj-$Y=Q4){xtb;}7MR70n|sTs`TY6URI z7zAiD2C#`KyfczVz9kG!Kb3kS*1qrCA{HcYV^;r9M%O(E-(ORCyK*@G3pPPw3mkQV z+=VsVmQmYTSuA0a-55+=lFk{L=^3vZ{OWnr z3OYOwcAsy8#!HW{KchH4SOGL-3zR|{B@EkT<=oy6^rr3m2ETGr-zDTt=A$T~bcbXg z!L{P`=~i1-&(Sp}t1dZ%Kqe)UHsfg)!ymY*ntJHr`RJJM+Qp{hi+gCM*mRfUJ7{S* zosx1AXc>I)`=SKf+2fqvDe|Kc4f+isqt_}srcO=5TqD5ziHm`0H^fJRi7B)!g6k-&F}Rz)hbb%{Xgk)v_&cDSM_PLSeJ*)5$Jj44s*NC2g`WNI-GJ^>E{h-^B7#_b?cH-4ZeQb#GXZ52IT)TX&*Egp* z0H(r+7MKu|m^rhIDNyC?)@`NE|Sb(lK1q?qJAQGibLRU5y85{R`w@04=< zU{>LnK{%V-F zA%oEMR$cQ1#&;??dW>#6^P2;?BLD*>ajY-*;hkr=xH|3tg*#m2Y%zX2OY?>$%8~FL z1F=RVoEFR}w03$yAQ)>l1IWENTHzxAC(UC$Tam#lKuZRtB8Z^I^bi!${%wgL;kcbH z?(B2y{|wm2A}%m@Y5<^_9dDi5Ophi>WjuQm;;5Eo&pFWO5x?co9_iFiUqHsRHRd&h zy;K#QaG7-j@GQp{H@U$X{b6yC@EQc$_x$8W!VK6yaX1|%zGs(uutLf|RAXpTUcki0 z79dN~SW#&JnVs-{@$OL+VEGKU*_Ilu1JJYSNLX;3WfJdXXV!8~;a7>KN;k^lv;^(d zgNG~L9}DLE5@_?UN2#hJVUdKXRX0LoD*z zk3-se}V|VW>=IY*PFLkavnn zp;TT;Z%FpLSHy|e0g5|x<{}~_9i+voyLJ)J{YJh~VfN};*Hecx+&_~21YF_Bsx$Nw zCU<1omH3#p90x}w9g0JR5eVNvM8Gdw=Z*#zon7MwW#FO=-A+%BS0Chhb)f+nRDe+| zFi{T}loC56rFsA_*+j;p6vXbN!RzW^mLnHKzae2MGr&AgI|-5mRZ5Y74)71m_)`)H zYv#N&qQQrlt>tQ!4~hUns_i4q>X(2$+ltg>Z5T%s!XqU%ZtqG#uS2PJM4pbgH5gAoyKvU{{FXH zbm}{OdZVbGkiQT?!O&BYM=y_)V*wI(2GR6O7Z|~r`isLp?mCex zh?|KJ<&_L0Q{|liI_Aj}3F#YQd0=h*rSLjXDj3s?yM?$6YXbGu*&fvjI~%1Ne-?iO(oLbM2=*kj%YVX>6Wx@DChRt;$Lfs;D)O;3a)LajdB|q8y+wr%V zo7pLa$=E(?97L|#q*gNVT-wz9avzgI%gbH;0YiQXPAMB0w*siq#LQv;Tbt|!cEYe@ zy`A2vYYfK*FHKFnR_q3mQd*~MUHhqgBstM4X)6${kWI$`$?3(yKG`7 zp&?#u5ORby9@Ls|wp%P>Qw0B5`XjOcFwtv7Px^q3E!8S8pD%I06u2mPf>sLR?cqhn zpm-IV;J`?l4Ff@{NB9?M9AX5H#a!x=Z6+9dw%F#}UM_J;14D!~KJk$C~TGsksKS_DNGKQeOpMZ( zm-EHLM{*eZ8#Vb(Ovk16LIfqeaku!^<+Tv@yT)682u(9WqiD(iP_yMPT9*Av9Plgi zYlFj0N=(^b;QjB7K#QDALGd53Kepe|G*TUt*<5)CP5LvF8*kG;xrV-prYu7&zr*Z^ z;iv5t8(mN!sWt!5!xnHbqG2q1gu>-Sp|i{I&QxRw7V zB&6sAu9P5)1zgr|H!JU3)g0xxsTPsjbqT!4eAYI-yk^jnSK_vF&Z8z`vb%F^Vz zGTiTlEohVXA$mkrhO$1bG~SSuU$H zw$BCiA6@PQk-Cq!txENqatp7zEHLdZdg1>jbfkOsD)EJzi$IDCf119H-Z$v6A020~ z6caHYD4b-G*EFz28XXHhi3Tt`{rWyS9#Skef};5a?~<&MJ&WJ~LkOW2Qt!v&Xj3XI zK(M$%Y+1zqv&oU1Vl>1sQBWuOpoN&%3?N+lmBHYhlVAC4VaFE@KNW%sNR2zee+Q>h z3p&rNAa-y?IKXdi8~}D3G3p^#yAt@{EL3dFlGhTdBIz@drDY}zCj7aQOkg8ian(@X zJmwM!`E(7dSvZcxzo#MDW&eQBzjf>gw#RFH7~1V@FWOESVsM zT^iIUiRU>BxpN<`SzX>@6(l_gpD}rNC!M=@?*_%@@il83a&t+;wjn%ODYyCL3}9qx zqd}p*0K+NuH8KACd25v~YAgAH7RBVZ0JO{U+cHEY~ zA6}a#M~Z;}N$le*K$sgWF=wj$!;dHA!WPODq~Ky$@!C!w(46uzEq{;$gMAZMb^>;1 z8VGLw90(QcTC2rGCk2DH5P)kc3}3vy!0iA&TwsjY^z?1Uo zF0)m=0-3-9*ZBIdzzCBGPmY~#hn+gE{2T$iaL*Tq24Jd{TeoP8iB0gUUn{x13GR(^ zMhjWpR53C(t1L)C((U}y&xe{&>6U{!#WHF*aV*{D&#;Ix*VoCQ?aF?HA4WN^0R#n% z;z)co`dF>2pwU-F4`=8$9Bk?L@ypDT`~h6x3e(P`TEIYWZvGoQ)1ec$qrq>gxc<`o zi}rVb^jDC~65RDE0?0I7u<|O(H-AZ6NI&3tY5JFwm;HM!NxF~~rq;`#)S&c&HdLM9 zw=?%Ur9~nG+8;tfQ?8@Zt(sA)5nL8@1jm#@5O6z_UNodnoUy6b5>`-UJl$fg(w+PNNsiOTT7QzCD+&O7*^f&|yM#h%!diEJ`)LP7CDJv)o(LjE;QxY!0FGYq)z<#W#FU z8)KFJ0tRy`X!|0kRpvnLEucb^1{*C8?0yN#Edh;{x#HzloD3mWkz0J7ud|kC(ShaL z;Dr1SFoi7!B8W&=P7YbkB;>+SER@uB|20~d?yIx`rrhfqhQBO_T^sG)R-o+nU`u&S zM+$mqL(X?#YUlmB2R_jrCJ{bcBDwO5k|qsn`nzii`c#h5#`uNNGz z*-C+`OKglVFX|W|1e*ylQt?LP`J%gWKxfnVR5byDpIBw2{Ybyw)?7fnj8s=Wg3I^t*6M_id9po3WfR7X1jTN@uqka<1RH4p6kKCtEZ7Bj zU(6Q@TZU4vz^^Y!ZgG88lAx)?C&ND%K|{qf$|SyOHisZ*4Fs@y4TJch; z#A;8@-kG0hmibK_pVFFLMtX*CU`~tjB*H~UfwIi2o!um?>CtMIVh#=wNQv5qeBm(d zzT6wWEB&h5egu==ibm%=m3iLk7@ zpLWu`rny`HI7_?B`s}LAD6_jWXr^!tgOHC<$V=5PUsBeHP$m6)w1RXFl**{t4R$Cl zBMQMk-6Tf~0NKt?n8E&6oJ1~OiuF$Sx5 zjWrV=&}~k6(c|_+ytO`(uNBPC_SWn1KaiHT_uV-PX0F6Bd&p>5>Z<-Av>Tb{u-5DK z92=gzSM25Ck!omU`u-lRp*fmGXBc9!RGe+Y5|>WYSnQuUD#8Yay|joBht_86zdi-w zpnz`XZH?0U-M^;oYa4h_in`kwrE6aOLb2S9-f!>&L3|hM#r2y;@8>d0t|a;ah=$fH42|`u6Gy5%-MOU z(ifFvf$Ufs2opy2PMCwr7LGZAowU;C)-Uo8l^Q;U6LmVzVPD^POcezhNfiRUp_0kw z-7C3IT}t#Cd!|MT0`ItYwE)r}sUZ1EE4ql|FfGXNPA((?h_0s(qu5}*@u8A<XR~lWKqduU1*%>DITnh)&j{j^eefxin_XIGa-wE!E1F3v zZhZ)HhdNAETVdGms#}dDCkO@#5GdI&0r%Au*V>jvU72QhV!QR@OD&!US*|pJ-*3 zPs5}llfr9IM8&Ci`V@+64#sfTH^kc~h#)7qG(BwS1hB`4z9&3eE@7T76b{eQ`kz8i zb9!A9$fc$t*H80JX$o|8q%$sS~vZx;aqiz zg?;xa@RWqKubRvzEUB5B*Dgpx0a&KM3Pm7(T?2gI_>lg!Ndr{ZmTtrzU;xpHBduUs z0LVLG`}9P>%#Qce7txr(lYzk;y3XSLjQGbATj(| zz<%7ZLyd`j1C@iomU_&}4aaf5Fm$*TBGz$eS{fNdo&w2Rx#9eWt4Y_6 zLUCqL5VA5yD@S$|-R^oGmJf-Li?OkzkeC0>A*@;k)j%9P{n@SRV9Vo#=tyHsAD&;O zfvPqLI|87#*9lF~Mb&_$xjG<6clWzEt1Rcm^`X(DsOaO)ASjzhMBoT74XBivlZwr! zF=h++UH?Wu(nS}b&f=)OhsO}Jv|g;*Y~P{d8vMC+X2KAaaT);T+ea$L10S^&NsOc(-8$C$+;j3QAhIlw;aZXNpF;jHB%$`%`_OK-#C%0a6S@} z;5TV3T2=z7Fcb#1k}~nLq)n3be~^--f&m}MaDO?RGXD|xNL8*%#Gv>%h$xNW1OVFye3mqN-=P^zUUA3^AxWc*RiLQ^=L!Hwh6Q&XeL< zD0n&i`D0Ri)g{It6aeRda_ta1b$2tNuh@k}|6!fPRFsm@7mE(h>d9*;(h9^`P$3aF z=k?J*z#Ynt%S;mX6DqO3u8pv0XF8Q-`?PM%!U@eQN*qCpS~_%&KIVSQs>Wl#jsu(} z5$|_k>WSxsww6ziRZ|q?AwESlR4RANDhCn^u`AReNx4i(eig;lU<gvuYt0)D+@S2%yT zrg8sw(9_LzMJ(cnZ$dU`m39+j{(+@24HlnQR!wYNjoElJZKnwr;qKMZ5*Zt2x@4^u& z@qHp+FRb$4MZx@c%pVyp56EmRp@9t2hQGh=`04*nVx+^(0spea*VbW{ZO5tMn&uAu zzn(8)74-^!$HUMH;qt;m3rsWI5p3Ji2L1Up&oUGGj(;3-2+tNCVCcY>3!63*-cqs- znXeW;OggoHQa^r^$d+m_r{BqRKnR`kR<~4R-Iw-9l0M&GIWJdH+6TmxuN58|Kq-#F zI*|HmE(b7E!u4Cwn<7OG5eLht-8R)h2^9Sn*g?9*ihn^surI^Yum@K~%C^5Ku+{gm zJjDBv91D>DCQh+VhQL65Xay05?^t6`` z%#zST=z3W2(pnxU(;M#49wuvl_sfB|(beJ{ATK)*Hk1;Ll{12J+BD7h%V^?ZOH0AV zXUEag|EEeJ9Sz0p>FNT5Q&;AfsX(KUO-!Wck*IIr@(b`R?VPfV0MKuLktEi&5iGQ1~K^O9bxrNVp34)J}cP&_#`VfIw;eRQ|}ua}Hh zf(w!g`mPP_2Keg&ge8!&zq^m6esl9aq}T?Woi^h=nIRee|8YlzW_Sdhzt8%StCtsl z{J$)KBq3|q`nA?jhmWahEGdrw-C(v5$NM)Mw|^5p1{LUvfq4bS z#`E!Jgex=4n>T3s|8rsseT48374?0mZaalZ;z40$X^7Ufw9pQ>qYAgfw8#%wBzym4 z_{_a&OB>1^7jMqJhh1S}{i@^y@}8YOz9>i)z^Jb%q19I7(ZF$1FfCg6g8Z7@MwI~~ z+H{jN!BZt$LDsGbn2cD4U(MEhio8#FfG_y>Ez2(*g%VOB#{0a{G6o~YedGRJk^lSr z#&q6*X{iN;EF#piJzhVJuirH1A?&AkK9u&}24P?ynwd9)aTX2>G*zKHDdMrn4r^r7 zT{=QU_Y{8n$3swvRHZ(i&FAvM0O>`_6VzgI^l4VtH_KqgVCD39h9Zt)_Q`$GbC#Yz z6g(F-E2W%5pZ>$K=X(P&hj?j?FY33ST6mL4(1Gu6+izQQgX8l5EJ?7~kZ zvZ4%Vk~cFiv+zBrEwVCxUU8n?{O05Tm8^yL3 z+f2hKOS>^IWIGd2^CQe`nyKml^l-u?TvpIaNGw6j2YW!=S%i8jG5Xj~upl=kZ4 zVrduJc_1Vj#1jb^k4!jbjict|cMbCpR?UB8_Qe9jU!l%CtNfUX|NW^WVm&Co13qy6a^OE^9g}ICg7lX>6Q>5&&cLlFsRh_hHJk?d zsOHxEU&wA5-Uj5N$hUD9qSs^O7vk-;luH zmso=CjT%dg-kCD#9r-mnF>S&dlv1Jma?^#Bg-8{gxNt(?G?EpK$k=&F{fyM4!`_J* zwbMOyfK>zj4Vm!nin*S1a{?=;fjjb>??SK%37Z&|_}>fMWf2Zg)LqT*RJ*f{qNoY? zntH=*fI`j!A`afkD+@(8p=f{h@ ziWR7C_r1XM)7Ob3)VzTyzbelxyWCSRLeff0IKmfIFh*ShbF0$|M^Eb>gtNk^pgZo> zNcqC15p5-dw@0#2=D)VSi3 z653A1AHLxx3UA4&;`T1OFTwz3IkPnL*~5uFZfn;W;p%|=bfxj%cd;Lv2@$wGG%k6U zF+?)NMkK(LwQE#B?_G`Fh_}yzERj>e$uD{!PwMw_(iflNW_-P>npugzVC*m}6+dE- ztv}Uu?B28d&Rtw#MSxC{v#-$^>~aNrrWgco}kfm!tB2TYS=0mwph5ha*XgNLM76`HAm zp1Gw2!vxMeZ$Y)E(GgzZlnwmMLoXr(va)A^v;a2ZDJj;xuzOOCm%IvL2NnMrHM#eh z&O&pmBUzv;_E-79V=rAMFnP_^U^rgJVS|(j>S6^+hA!Ci%9r5PMkF=hk&c;xkAKJe z&~ac`~q%5o~lJ#W zG*Z&^_H>i8C-%bJ7prpORjIH1(a#_V;oX)!irZ@I78-L_?aSSJ^}BTEn}wsRt5EqP3c#QAfv9U#<5wdcXSmN9`RrGmtBoUuHxp4p+ zvYGMN){xYaU2R3z1sSah4^xL#b6@4t<&Sy65-^vZDWMSL7j-g8XZ{@4k zGCHc^X-*@vBgDhwXKzGf0nNIMn1Y4i=RczE)XYIjpMADP&}AaR)`HIK09beia#god; z$h}R(z+Cx_ddI|u_V=CsRzDX%pA;E9gz9;v>kg2jV_TH5yw?18H8Vct2F_R35}Yui zOS~$QgCx!zR2Av{x1q;41Yq_f4j-*l^GIG2E@|eaYG>xzq7GELg0{yANejYPY;R~( zc3`BphH(d*MAGxU;*hId=-Hl?0f=N~4HZHXwztL@H98gKx*DGvVWQ%1A6Nuz&|DPA zjNacY0$(Hde!mlm6B#mbm4T6vf>`6+b@gf$flhYm$a1nFgx^C`O`qSZV$z<(0Y)UV ztX_@{+4kImrFKUZ^6-4&Y!sHZG~0;S2X9Y{XsxTH_et~(>M-dinlPVnK^h?{{YmFI z3t-5z#htp>FFd;{GVa9T{>=W_bqJ>gS}w$}g^>ZHVFGC@f;Yyu4U6|KGU{LxVA=u! zMlIQMqD!6~fZed*;Sh{5rg*+lfAt30maGs~l=;TvS&uI=<;GBK|JuL?b!!6kRdDd{ zGuBU@;m4@1xG4@<`*~SfrOVi?Miw(@*F;Mm$>WD1wveM(x((!+-kEvZefGr)ZLt5U zOuZDNNcdZhet5WAh^F#y3nNaYw&LaYnGhATgKi99mycT;!nxh5FXn^h()$56nMH9kCSEvaWX@y?)#pU!u-I< z{j0jr?S}9GbBw@bR_1y34(n-O?;aAdDdTBJ$m4LrHq#r<|*$9(>j zArsC;>Ae^7LLg4AJ>WEMUerr}`TLC&{m#5ufW>+GvED*`b>8tMoTP>V$E3`AEE&^1 z0MC$4f@f&fr1GG+Bxrg9o*nXa5}yr|o@1>~u;ahWBXZcm12qE8ZV_HAu8Xf?WC@CO zGW6UDxLm1HjoYa(=UAFIExa3DSP3lrjMbakvUJj{w7+oUU4@zV*n_1gLYYaM38Ymm z5H;a0u0fhv!!BcRL`U@N(K|N-w~zBPN?u+Mk|D$cnJ^0i3NhL@)_#jDaC)C=Kq|yn z*0#E*ya+7!%Uys!2z3gB=XjOVF5tmM9@cIdKz-9P9b0sek|7~rv z6#PI;gwyma=k44}2*hkPe+bT}o>+lJmBgUleb5@!m1g;&nezYU_S0=w7zDY2rX zPA}emo=O6%7@3;2NM8rE3$!rJJV?6tF*TkeF_F`4A70#`>nB6kktvNVx?z8hFJ z#okYu7&kRDI+5=d_kq@t44G}+kJ78qOcFujsf)OZolnI0JOUS?rNLEdGc{|^@B1`u zwU=P@+Gh!i(&^`j$@uu zRAN+B(UG2*+&&>1&3T)a(eYfM+#(yA@B@?Y0-ypOw?sf(wAF6{_fXc`kAS-^zSq#A?N^tV9o{lBw-UCI>5FYXm28|@sEXh zj=BW7_12)POd*>2m-F4I^;>>XvZD+Z}D=K-G_-wb54$T13Af; zoFooFYQ26jPwyt1k<7##X5Bf+w^?xl-eB@`l`H_hGoTv&M1zMgLp$;0SpPE_r-iO~ z(gEPo0%uEmNd&N0)4CyU>)!0TGE{uBtLXS<|4_)p;{LQmiZ)9*(Z>ow-BTclp?P>PJcBw3QJr( zRc{agcBG{stHG)_ z?j+0DastXs>hbxIaV$HHgglNLpx85+{mUZvgK#bE8T5F?G zho-`OleKM4m5}^n>`plsW7=wNm6mqv)9XIF$2~2Rn<0VGFzl}&o{p1z^eKrtnZ0Yo4puDV1s%f-E zL((p7&HHNhj}@ez^NaElv42BvS3_fzQXsB?8X-9$P^y+M6=o;Nu>dLEqSbcP)0N+Q zLtU0b5$UmXZ%?Bg!e>4q2rG9e^)z&NInV3~fzYK%;OK&K7|UGyH7y{7^^;%72N&VJ(30lUrwyP zcaO{OV?m79_RWqdNy)FFIMp>*slmAzaS`f~z7sZEH^sondH}<;>{f#IXG#M zah8T5A2;r=00>!BuJ~3Ezt&aQs|%?v$P<1%a9qF01&R6q43v&$WmIurI-+iNg@C4e zvZ}p-TFL!T>8o{THG(i0fJVcSn2wK0-t?!8l7^-GWwZR3$9qCIb~Nh6NJ;;Cdc^Pb zP(~0$zUYJ@tJuEHc?0wciH{7RYUPs`5vZG>OFpxGvQHl>%Y2`_8>5Eg5bMyZJyJ-3 zFn?YE1Fou~HCR(JQTe*+rM&)0EU!9$%q{UxA53hz_7JA1iHnd^{ZTcJm%JJ?y9pkX z)W{t{@4i(&e>2eO#}4NJnoqTdF5tT8Du5`?N;!T*G$hRg1rsq)4qhSsR%6`y^US;H zi4V}^9jm?b{S|3wotob942=7|s@D1TPl5s;I=}OTUd-Dr`S({qA$u}#_gI$Ilqdib zDp9*HslsD)jK`hYA&a7S5Os7%;l~#`XGay6mQ^eV&3JfO!DbyhlJT@cjUeavi2KaS z4|1}b!6bY~eXIdm`73iWjo*KL`V(q=$1;<7*ImR#s`dm1Amj}=*4;Vz59m}5Hw$5~ zoQ(AeB5zTx?{c+odFl%3nMdjq{8P#_izbHJL5OvwLJ8LPky3OiBR=Zs0*@y{Wp>#Kg%l#_bhL-kr8Rd6H ze8TQdoouTWdvfrQ|954(5IOf+n3N&)!8ys@+?K*Sp>;ToCoPnWW-qn*;6@)XaW0<;MYRI`lQIEFkw0h8qge0Lbkx6CIXRfs zz2PHcKNg>jzxV8uPQYYqf%D7DJ6h|-iW*u9PkQdDLFKb$fK=j^$VtUVDk27uo0CtZ zFQ0(Y<8*--6W(Fya~$Zu8KDhr2To-m2G;K+-j@o2{Hik^(C8$;vR?bq;2_nS82ue2 z@L{9&LF7mK`ef;+Fg^qkIQjM3%&Waxg=A}kq*^cS4}?-lNQksO9&BHT z8k4NRSuyMjvTJalo+Og@&?EKIJhjG!MB#JG)1 zr@LNH$B^(^5E2XQ>$=pF?RY9CZ>ZJZ*3>HQ$X}#9!~E5rljE@xA9S1_mU|Pp-0jd4 zDn5F#e4J~%suUN%hPeZBQU_}|Pf3C@n4rWQoMpaq)>&0K@%|BjLhzRBi?bcHth1UL zH1_yvN0waY!5c@fA*{oY%I3C8lo&-8&F9Mkr{`X4p>{z7?P!v}2@$^%Xlq5Nee%VJ z5Ef)7G)w3;ux^nlcv=g*k-Qz+?@72o0G+tmAD~rrP#E85n_^o=rm-pA4jrDJFVj$j zordrL2JkDn)PYUR%b-00!SftU!1rCz*A~gqf!6O9o+*T)m6S*}7yU_qwoq`cog zt#iyY#L_B&0lS>+JD4#B22M=!BS7sDrVgmxHVM{+7J|4N;5hw&R)~be;C)_g5&#ur zd!Q>tX~e>kU<`-a`&!a8=~!L7>f_<7(9xh*>2S27R3^sFXEx!e%Stshr{Lstkd;1T zHC%DzRt-tr%?6I5f}0U zAOXJGFb?vDid_c4j*%45^*eK*SI`LzMI8V?kZ8o61hCGSjuSc?0Qd;?I*YDS=mET= z77&9`s0C*Xqy~fbN=3ON9Sl1Fw?qx~KkOELmSEUJczc~wn%Qc)7B{9aOrv_lNLLM8 zVVC3pT}-BrT4>y_;u;Qin9dhshZozkH1)@g3nh1NO?ZV#P*4dG#@E0lsaPz;se1%SgWPLd=GA{{34}y@2O^=~&1_Izk~w z5+X=2FXxE^{ zfN%q);;(P_w2G9kG_*JgdIcU9eU%zRze5U6KnI9oH8&Dty3yELsc958dpc9PiXx1? zRu%}zS~Pk5m~)eDLlX@a&Ye$k z6U(9Z5T`Bjg3vJRBuJKGOSiDa<@dLCh=P( zXN+e?Ud}9R%qpS1UF;bG=#Jz<(P1&VBs>SlJzezHGx2ni5W7uWzpC8S z9aw9Y8p@ydi8Vh14kD*WP7MWs7r&LX|NW#84)1HU3|U;@4ej?N4#(||?n2*J!x{O~ zq|hk6L5PK1bkFr_k~%?Y;VFmV*;&%Ty^9d&>~PV>5lB#B+SdC_l?v+ zDBi8FwF(P>R_YBBx*)aG)Ikf&CWY5pV&W+UORlzy(poTx&J+y2;%b3~OkDr@kpkjShPFE7(Rv%ujRi2a@y~{M396 zZsGHTI^QjX=+FDg-QH{ZVhZg!cGV)e*yxFHe{a}N<8i(&!bN$<5+#OhjRqGIPl|nT z*1cmBpw%L?rvGO8+>6(v^a$jft&9(wu#T&Z&Ix&3ZH5DMA|I5F+gv6+1(LYh!W#iZ z4OBfsApEsaegO0CBcwBZhceL7of5X%sni_4kIK*FgVg$8`?n)$|F>$97PM4+L42+< z%&{p?ITEcG%UCOsc4#H8qZKhnjaEWGbk_A^XV@*Tf<~Dt#13yPGpzK!z6I(o7Nd&4 z@OzV{cM3g_KDu4_J!gi|&t#tMZ!PFK&_hR2%!#;xskhy!x0PiZt~rsrlVO*%-cV4_0Gcz?#pc%^r+`4c~=)SP}vWwn0rzHDUZb7rH2TN z_WH%3wP*!(!x+gHY2Xn&s`y+~CnH&yDhJVorn$J_l=Uyc zj$dl1S4PG>4tLL2kH!taRZRbI2=rF7#+-$=m+P{`O!YjM+FQ|LY*z`@rA}6Q2qn18%s-O*z8}`Mm54Z3EUd3z=rN<}(ejM& z1rxYH9XQ##_UIdg;y%uL0BR&gGODzUFG zHC$LaBySLbR^?42?&~BHkktO%8FB2R({M#=2TjO#Q(S6(Pnp|pj{YXhtsb{!p;hI_Goz*a8}Jrq=DwAP zxDs3jDEXTGERHCbSb2a9kQ#LErZ}Q?z^<%{&}!p;?{Dww&zYWjbb*0Bw#z?+8YG^o zLh_1o$XK^E^Txee_7ncR-25Iuq=;8oH$B+%ZiQXW#m5Q`Tg4zhb@Go9f&3o(28qzm z?*#)MTq2bWf)$Nq$ zjxSvuliJd2_4SRiIySY!w09yoTJA~d+?o`zj9IZ;A>5qIM_*6@EgkOjUn}gtaR`&% z6F0bt7A1wU_Oq|bnOKsKgLb7O>?Zh&b!p-ej+gO=t|Wp2kfQlz4%>6h3JEIL-8w3| zK5J|#MneU#ic*G6pnToFp>S)=dozvW`TJd%IvJfUw7i)$?3D(o#Z}m`(q{q*+#pR+ zim;0E&83__)eiWYQ)wYV@??;sm#7M)M0=oCa%nDD0kGqo;5A-RTAgt&kJTH4t-h13 zXN2d{(-JbQR;5XUh8xIRaVGgO`%YmV!|8hdvYH`UxVkhv`e>-M-Oj~d#NmBUQ9V17 zg)q)jR?%2UqdSNUe?Maf5i&=V!L!CT@af4et*ws*mK&|i#M69~GAzi};GHV8g=dPx zOW|r3Pj^q0SByy#3X8(|Ubkr=o;QH4iW7iEs-fNi?NP=++w2fhm$`sX+_0I!#7E2l z_uBl_3d;XjH$fV7^J6BlAR-o%%gQy>Y z?D}FJojQ8dHZ=taTyh7--o(jdyxk@M{79k*?jGEeYq5XGJql@&z~OyuLGt>ysUNIn z>LtB~Mfe+8liw>9-|C$+&|FmS@xC+<;bikGyn1)IY^}DS5VG}!;%gIVq@7{{kzOkh z4dgA=bBAtJprL@uyPzCykeJ)etBc&cJC3!W$bJjueEP%8cpjnJP1IN)M5052t!FS8 z>(tN_(2P%9?u*ZosoB`8<8OV?_I^L|$3WMZmhcVwfN;~TEPUg(#8TZ_`}i)?6Z5TV zQihu1>-tpUk3`OB)W6VdCEvwQpAb_ zy)3`uJ!&YrruT~UoB-iA^9gyWRdw7UyP~2Z4e!lPE5R!&)7I5;9Sdi0o`W=re%i{B zIy*P^R+kn*)opRP%7%w3IV_qx^m~B&any3pO2?D#hEbo0Y=r`8aAiP=Y4MKUmYv&a z`EN8+NKIPx9->3NuglN`7A6by?1I?(HnJ*Y%91wI4T;8)7y%5}5u^o)b)zs9u&Cst z!%)5~F}B@+N7-I?Q62}QIz@dn!e6) z+la`W2QMYDj|J*6fQ>Uqm}vBwY)rm7XTAjs{1)6vW@+(GN@%s*4HocYTb2s?;6xdt zpxFx7D-krtjKgjp9hBSvA*;k-Bw_GTjxhzkC?Djva^(h>ea^LbZzRhM;yp%}K>$G+ z+EQ>yDd&?Yhl*#QNbqdexv@7HO4j@)xi2fT?jt1?V_}X5EeRUv);jT+fhYOTlt?N-sUV z{%6YpdG9-NrpnGsa zsbB{3d?&BruI>z4Z`xW{Q!UgvRsu0kxE)D430q09i-StD6;h(aP)cQcs*RX+M1u- zahSYIb3p;bQEuQ&iVG6ZkVrGPJZB7P8EVF{%`O;Z6ObW=)o6-~k(@MANj1J!LntGZ z(#klE*n8^LSU9JrL@2?bxg2u;{s$zk%xcMweL6Ac6G%MQbNcDKgbmEQMwImil}E^J z0;6xQF7Va5m65g!wNiZ9QVlVAOl}k?DZ9mbf)yKGh?-`2BP~cU$kZvwQr|YRBq?;0 zv|sr1pIQB)sQ6@*@VHj!RH(~F?|tnl&nf7Nf^ibD+rSKpr+-)7zYZsUc5>s7Dk_5I zCNyc7{+>iH>adVGWB0NIizGvKHNN1AC$0Q)4D(Hj?5v1eBhq#H_(XWblm$!S(Ff5~ zgW_y9_eIX5WQ-xqQe_wXeLoZdCfU00%$JL9Q{JRF5=L@iD(EtRUsso5EU94 zlF$g9yz=%gExwZvv}99!{K%EIZ4W@{NN@L-U9k}4UMxCox(IrMw#0IG@0pM9j3_L# zkDnY06x88-eN}3z2ldEqj@eN=O+kl389AKf1P3%#f;3Bv+wNoI+9{=can3V+3_|vw zK?P6H+_h$-{rz0SahmJU2Apf@EL(n^3u!A7OM)|bO@EPj0`pG3o*s{6&9>E`+qHH1 z8^k_#I~hR!OtG+!GedJby9PX2vAjt?^2IP6O__A4ETxuF{5U#UBDc&&&LLT>uEsV} zXO2c-xq#`W_K^kOn#X%X*RidI7M)}~1rZRl@GUz#yM7VNDGAmaC>1Gh_hQiC>U|E@ z(_}rt<0Vq_I%o|uGaY4U2H|Gb@vB1L?6auCl&+2RYyn=bwEU)Qry|{j&1okvx>#Hl zyma0CZ(5qAF46SAk@EgN55B%IRy%p3vGMzU+^e0J!&bp*0D~WO3GlcI8A}XcMPxg5 z3woILiUaPsnQP%-pD_wFjA_(a|yU%KClb zu2grGL zho+amG+7S%97BjciG9N4BQ%cVzMY_R>B>D?e1O*2*mRAB5cboDx+bNj7;c_gD41@6 zD)GYuLM`2gE_|La+NjkndpNZOK1{=-A5i`U**D~p^BczUlVE`UGw?%!X+LrABOeg7 zHKj9mkV`se^>nfZp{7XKFr#hj3lhxU>UxF@eKzqtDKC3bvnJ~! zXEZ_22bOQ{$@I;KZBIAxI>U(?Ggnm&$uETFU?nB`2%1u-PlPjKPTT@pAg*|U!=^XC zAn%845d5n()yzjC)p-!()7M6VqV6T--?>&-vuqwO%ynwg|INV58&%XY%qY&INye&{ zXT797AZ0=xHlVK!hifjqW(&{@B|C}DulJ>rx78U-58yj1sfp-NQpEXwgIq3^dZhKe zTpyoU5SfZf^htJE6IF`QCNX2u#PiaMqzqMKyvz97DivD236jtVI-!RmlqEUWaa|UJ zf|otlAr%|Px3z2lKN_V+! z4Soum2xZXClWSx2bzVw%7d|p|o!Pm^j(ajwH#LqfvB_=BW)xrU(dcA2=2>=mWj%Q8 zBPf7q7<;QRA!r#iZLOkm(=24^&ktN7wW_8t#W1!*85arVM-FXMHTl(`9S!8Qlh2iP zE?yX!4NegIc!MyB#)I9%Uq3X2!Th<7R7>TT!WXrm{{VGa0DXrQuEK|_A9oWu!f+#r zZ6_M((%xCkWMn33sk2Sr1OPD-ZN(cNbCwE50_1`VTeDl60)nMRrWEwSGSzzKg9cOu z)V5OnASjJAlERLce0@uMM-gAE79+&XTd04zMBNirJKOeQ=Hc2f%PMW-mvS1szVb5L z$aLTu#kJ=u-Z1l1n$A$k?2&|U%;T(I#B^s`-|corEt}7kAmNVjh5AG4h7?|av9~Mx z9(mc{N-cA=-Q1KcB#P-MwrK@4+M^-NK3{~!t!ob<2 zoBOjDlm}^Y#0p5!Y_c_CneosqgCkK)Q!O&uQl>gFv9@KRvTLTJiS#Aj0Ir5(v(tHE z1_d_&E7alK|K6)FOY~zZ_7n@Y<9$p!yJSv-xLRq^;5rr8G5{fYMlX*n3~sJ=HF|`_YU$yIR zodf~mwlM#@QB<7krp0|Ho0~S>u3OzNDH(INy~D;Ztt>7wq!+zpdslAC&)&`(a-`%b zcZ2x^Upo)d7jh3SW)0z?iGmXYN~AhPB5dT)LlF3OT4tBb*II$d4>xwfDtAM#e2Q`J zs&$RK@aF`&)6{|~OD-7NaYbp#AGpxpRsAo_w;y5UoQ?@O@M^lOv$o|NK!ahPOuVL^ z64a_Rh20j&wopSMd!hzvDZ!<0^KvD6RKkbi~DDY<}Vr5TU(UoqpUss!htr>J`_mEa45 z4-Z_EH|a{H&oaQ@X^Zt4l3cg@Cfl|3ykP7TUYs&LUrP6emFkj~lFe6nk3KgZx4A01 zH2SDS-Q!s27;v@3nS_#GgLLt&2hlgUZJSqXB$r>7Oe#&>nfoFTUqIrWJGjDxcdeZ5 zaBHe3-+dWI2%4n#D)HkWlg{73^U!gBwM9~>NnT0+M>~()bP>>)PNzD5fG`9xO~X;z zf>5N9K~cstCOp4>W6WK6b|F&OOuewF;~y!cfY|L8NnAcFw;mb_M}?+)%rZ@juV>UWajU)`i+$DHzP2T-erGL-YdykH7Xr zuYfzA4$k*ed-O<`W=8AObU!2zO?*H6;j^@Vsf$rkf>&YQV*IEBcbEJyPG;Vf#?3&K+ z(C=BmEXed(@ZZj2u(B2JepOplu>?rrao)_ZgVQ|MwKn|pOpXAE06BN2`pX`eH*VUV zZPy|7uf)5+Bg(E-v;tsW(Qd!4#d&E}wZs4~i8UCkt%juwG;PKxk%?Z3v3j*N>(VFD z-Z@%5h6K+)>~%6gb`~g|4>}KhlxL1N#lGa)DxLKdKs6Lj8XEM;MrpVsv7%+Qg+e^IfkL(=NTJGH@_x_mPd^VCM`DN(8U3`2Wdfnl8 zAfaU595Yn;{SZlelOlJA9+}6;-6222?13$X(QHK2OF;lz)94qO^<0UVFbbmc?w1Fj zuK?a%1GwxfGH&^JaMYnl2+j|z@fx8P)YqC3d1a`l7ME{d-ZzJlK2_VvtJxS(;XCid z+le4{>{7$wTZiwcLq#`Lzi{4^;#r6uooQ;+ryQdv{La0Ox{mf%x06Wz3=@LWI5$#xid(fwB!^R#W6Im>N1q*&cjf7G?91wvpJ?Qj9&?qz3xG06jF9Ool`Vv&=A zJk-c$23I{|$ZXB-7GgWqQn zbk^Pr2v8YZ?8`RV0wF@KhQ_Z*(*2qag;OcOF595%JlWwI{pPeI&2>TBAQ~v|?PXd* zjJvPbo_)Q4547OpOv&!-7tvoe?vq|F>DuGg9X0+A3E_!qDkyOyy|@&jrKL$U$BLOJ zb~p|LO1Jx|;1Ks3$rh99+5jQS3OYGeDsoD}1T?ot(}hev9XyjD=hb-5+u_%Cdq-Wg zy}6ED(@f=X`vcn732XgZl2tp#3Pc9`j?=k7ZKBb$#dtkeT{OL0OtQR}n(StpQL_SR z;bJ~FMqjbxi-KJPHpVJ-Q>TAu?&Ez##K0SnUPZd&RzNYRs~6xLm561PB1@_%>VxW? zn&_JA8!kKJE#&WN$6UP;rcFTIOV+T)XBf#E!XU*jS{YfawzVd$o}NX>h^bRlK_^?1 zN!0<}tnBP=uVRU{V&NN<&hrWVu1yJLLc-X6ANH`~4mrWYDGw8QdNEEQLEBEM+G&Zp zQ!2r@eIgvf#k&2-jTxsNZ;Zox6#qRqhPs%p_k((_CMm}V6TOVL9+8c0`G(WzQP42*(M8k zPAGHt-A?GtK^{1sqk@SaGEa&~^c5U=Ly0btg9SES{4_Ss&o?vND#&IFC_I+Qm&$90 z_tIX!m2^XmGlXak^HfHskp6U^g7iD&_~huqJ&oi?(PYr-%#=LoIr9|V)GCT2XduA- zV*J6mecU?<9zYgUw(AG8lvN~ntAKHlvgHVS#yb_X;_52`rM~CkhlRsCZ-no;-vyPY zc2WM2WQaMGR}-M2f1GY2FW9$D3LLPCQ}zyDC_QumxpjGou-8fO1Vu}j_SUjvS^bC> z0PsH~GI+UK@FZ_}+%jR`QTtE+#;WDRAs znlAg7RJ{R>Qr5nqg+4gfF!ZW*ZmGzqeYrBU>yNHwFd7A zv?Jb;_E=Z0gv|D_racVlg!P%bP%T7@gFLDh&8P2EdI|FP-`5s?77st7$akD*lNYG= zss7;%WOi9>sPm&ld{5V_#ZlHMNzkS)^flRQV9%vVZ826ZD2fqd!6 zjoxYcBnDrmrQKB&Y(P!0D`j*n_Xg!hNPr56UoXS z4kx|>fhxK^OXD&rvk>(O9iF&_A(*Gs)%#n=Mt`iVj4~gF5m|ORCL!8RZAoZI?yyUX zB4-tkb0H8EG)N@8uf5IbG0&)3>u@9?Fu=mDUg!XkHzI~-h>*Mo z942OaEbkm5$Cx2y;o39{kddX%=)cKF`PcLhEMFP6jS2^@AB%u(G*M8$)-t*vZcl0gnG`elmso);`HT||NqsUwr`g+L zx?=^{XZ`&APOM3Tz}2J=FSIAt(loh!XYI}Dt@3`g6{SEKBbf1qWR}GnS<K{{1$+Z8Qj!OjpVp%!-C8ArPhzC#t5aS2G`z=8sQUANPlIp=qP!0%Og{VhXT##guZ)Uun(-Oj2Apr`fzbme|ZmUP(s%l0Ycy}PlTU#^O5K{D{G zrN&IBHFMs+z;&H^Qjh##O<8W(=pJuy)AulsZ|`;`l4&^Y+^50~4YCT(J++9xeGS8f zCSn;Evb(&2*V%$(y&o9~T!E7L0*cY4tkJHB6Cl_+bflsFy+S?Xt+=H_eqFA}ovEkG z<79?F!=Kb|=wTjdq_gak3fl20Zquj7;v6*8h|FB~2J4x=nny;47VIJt)0=CvIq42) zu!LFDP?gg@yXXp|&2*)PzJ^#;c2Ig>UwD~OTo2Bs+zH`^hFyZk_y7$Nu8{(k@dQNg zEX70^_gS~F9u<)0nP0_Mj%Cm!Y-~)JL6hk3_yY?F4wClE8Umu)NY=1X3IayCr=l03 za}pP(rad4-k48o3#sUwWW8k@@KJJjXW>95h7He0J@E^Sbbv)n%0kiG(rwHsA>};*r z@u_|my^;S?O#N4uk#F$KHeS_mMJmZ33_A;_B8UM_qBfR83Mi-JzrwICHc>2i$N1yMF6TZKHPDA1ZX9ml zSvl;Uq^yZZM%o$c9t>=nK-&9AdbuTb0~E+V)_#%c&8CB*j0n>zZOrf~tBZEke&#d> zFIPsWJ3@06fXe8Ia~GeSu=2Ne6~t6ltS1CZfgg2?4Za~l-v)eq)Tl0L7_v-x)$d_I z$*Pve*wDh3Fys*9SP%GyvairJtP~OY zT`S_p5N15c=he^DyA9TCD<4!eQ@=m5OC*j|Boav6y%aD$ux`7 zc1fOM7=f9E7Y^^m#(_C=+k60j-X*d1ukSQT6UD*W(+adeC!oAk!@S=D13cxAk!XKN zvrfc;?K1)T=943PPd*pdr5No8vK`MOrEcSuT3w~7TISMtiKp|Qn?zNF+69@ zHz-M$p^37~l6bC=w9k ztQ9%&g^9SUv|;F#NuhX~J@0Os7Yg<<9~esz#+Uj^&vxpI>Pi8#Mmv!mGd3kMpvq@D zDu_Ew-Xm|pdD0zcaluu}b|c8v<0;3~*5>TiwU)sJtw-O~--2mK zjOm~0*w160ecdBnTk4&ETn{I#H|{w(sd~=rR9x0PbSREG^KH2>GvtLadA*LH@Z8g& ztgaeXUSiR~f=}5-^TetIDCKF1F13$LO~YX*Z)^x2NUyCI?=Bg;fDS=Zpw7nHvzE4> zeL0vRHM{&T_}>wL|F)4(MIUKAX>|}|WVL63);?+*o!-f#pdV&RX=lv#I|$ZdtpA$?rnqU#D%jY z%wayc0A$}NejFR>fAX(Oh67NhD?_pC04=)|)|VT>BWr2)ly=%q`Mqc}Z+mkPB>%@F zjQT#rsIm!I9_w>5mdv)7fXJ;g6OlLMFbAxo!d$}*HnOk#8Bg)40vEDf@A~Wh_V#dC z`nlh8O19z?@==OSj0_bqtJ^jjG_}iF3-?&b$MV zNkDslL5S}taw6(q9~Ef@0sy!O--Vv<46HNy3@K~9dryI8boh~J zM|{>gc^1xFP-}E)d8L<|FbQiLdR-0a-LB<1X$N2YpnP*h`ON$Pb$$T6zigQmLi-;x zm3|8JaY^NZ7{X1H%t1!xjUe`{wwLfWJ6S4&T4 z1=6b|9^j&Ex&t8Ksi_A<%8W>R{ecY|r#LtwX)IN=&D&utUzZpbdcmR}uXAi!Jl)L) zgj4fZMqE!ct5LHz9PUBFSK9FfzAy~zlRtLepB7c>N2zYF1TWulN421Doe=Y;& zLT9qNn+NN3Thnvx|C{{UN7Lwd#c|Uj2ZTmpPI5QelqiIBSO0g zu2>X!1>;-ll@XH~H%0e`39+no=Ag;Zl}pklC5xcW@TwdWv>tt6d5%@|repwdHT>Pzz)gdkiwK$uAC*H1@vVqpyU}xYXoJ|0>bbA+DHsJwP zwS35$OOxQXks2w0tocURyG%QvRJG+ zS}eMN7Ho7ne3GXj!0(M81BR#GYNK=6BmGl-1BnN&J`lbrp8t>o3n+!eMm9gOhG~V1 z5#X!0)N#H`h@OFynR6Xsl_2Yhw0QP`@z!BNlQFPzxcaX9OUIMno{YHklID*8P{Mjk zWSQ1(YQHyAFOlmKHEv!~?sdRakZUnV$6-D)1t!1R`PMS%kbL; zU(hp^y9=%ry!nNHF$&fr?n$9}hTuHeL*0k~H#VPPSO`)?xyh>f5tPEhRW>4_2Z?si z%?7&N`GPz+bHUdo712^&P14F7>80jz8gyX9Gaz6r#pJ1_ysT4xgX}@QW3nbn$EpKm z(Fnt)gRgXjpjJ;EPS>WPId^~8NK?oCSk0|~Q~l64X%gxf=eqn(MG~f-n>BmYAEQZp zTu^4W|9n!1x;#_(1&O7$mcV0FfsRmNr=AFyqQuMhz$7!}TK7tq@wEY#t?|jp+UqiX zfz*Y6T)leUL_yA;n;U!Be;|>(C}2ugseXmZSD)$WmGx=cNcLsM$KNW0D zkgtoQa%K-ma2IrzueRr;l6&Ernh=o!?Eugkx7B#k5wu4z;oWxSvOb#Tmu=*mRY*yz zokpl-)Dh5Su3=S7Gt(Zm!6|8xLZ-X!XJMsd%!~U31oVm>drtq9Z-P_yS)XNdMr+^U z^qw2DI@V0SPBhI&e#y0B$kHLr5HrmV;>(|b&8kmc$@p!7f%?w8S(2xbmv`&JZO$9u z6TS%jScewmA#IR9fFvEdfU5u=>?5oZl#3Y@q;8Dc(OuyuRa?#%(x2#q7YWc$Z-dXj3AI-JGgm5a3+@i z#rj{^25;c0jiE;6HtnmvXXU7jr~i7`JR_3a&p&^yrebokBw;xbmP3SqbF=Mo8}<6w zZ3CF>MJMdDcH2Z%FG=A7Pw+r6EG~|jcP7`^2n+l3E0Gh(#c_IjYSa8=IxtV=T+0Qf zTuxO=Fba-`g*5H?6yTmDM#*C96a(7jcmu<8Q{+1Sn_6_iJ|UC2QGc-?=8<~TZoXP< zoDh_jnVHF-xUn&9j1SI9*p>RK&*e|;8G`v4NxWUi^2n!o42DKw{l@gE9Jxr9q`+dq>O%*&jt189n@Sb!+j5sMGff zoy>rxI%)5)B8*OxonjM|@hEO33GPJn;V@x^({40xC|}-aKM4%%l+)fi8>j%%vB!yY z+X68p+BKA(hEsbDHkCZ zDv_Lzfofj!k2EdREf=F|?J-G`WE6msI@MpIoSJ0E2ETXP<540knQdF*S+*KgHY4w8 zw>8Yf+$k+aD=|rYHU<&{N^_z(J%b3KnhJsN^EQDTRvFT6 z2~-Cj__)keN%y7(8z0WshN6t&Yfh1OPp5M3 z=yQqCIv8Z}9&jS{Xd%b75byP&w&_oP7IKc#mnrTdz93t%ZXE<0T}{c^LzH-)N>5sR zUfbcern_JL)=CT-OeELpu3L6Gk}{6Ie*UpZy>D)05rylK2BRD^r$kt_;CBQ+aFb*M z8KV<!`S*fFdn$q`B(NDvZO1lp+7wlD>(#;J-y-dC2{gCGto zod@&}m`LJtfC`Ysqh!^Nm%h4_ta8UXZ0$eSDx0s_9umskre72X(Eb)!=q1zl80m4? zp20Txnei+@Nm8ymbIS7!nfI9XDss>h$kd&{+)*;wF_ufAVfP0zD>0GJQmuds_NFIFSA=m_7S zJ2fVV|Fs6R*EVpd>w%>a4GJZJ*?c!9UyJZDKRp^mPYEN4XY%Ry1+(OW!K*mhgyaqu zNQ>;GdhZC$VuW+?b3hxKuS+?30nnaC9u+X{(zvv=89Zj-_&~EKMBTOn_Q#?= zMX@KR5j}{E%g64Ll_r?^WPt(4j zzN54sU|At((46XKtvJ_t|45R*DXmts$SFlk@hUhZzm6~*=V)N`8LeZWa7~Nn9S-+j zBwzKH8ONNd0~-%kZf8-bu{B9WwfAv^$jmI=HZ&cuBV|8tqDZ=5`$(+&ll_8E3@A<} zAnJt73YE|N75k8_sz5QlJI^!k?C0Udb=`Pk9B$Q)NX+;Pl{WhUg_g$=ecCSJMV53U zujg-&-|(Lb>&yncAb*jswl3$w{v7(YBSbWI@fi!Tg^34}s0n8&u1diY-!<&sba7{K zGqwRvtS{P~nQD}zC0R1_ajrMGs3+gYvRn`ek@cZzb#j(TUai>V^%@m{ku^uKw&mY> zuZ@@46@fg;Om^EGP)1SIJ@GnYDHeZDmCNpQdar;xvpd_g4I*jOhC(a36~0M<(Z2;IQOJi zCTcZRvo%mI@iW1U)d~(wj(G0OGAx)~T4kbbsz*yNj&%`&(1rK4@Trb$$aPG+<;a<| zBT8HAy>IIckdM(u%g;{H(t_wZ+TZ_Cf1`i+d))Jjtt9%>MJ1L#N+}?O1E`ctOA_tw zcvEV3jn7|Ep=_Brax%X|Y_H~7-~R<=QDeL>>r_-<@-)ayr@QSir&umS%kBUrY=uXF zMRUrG&um{@PSF=oI?#ymWw~u(Dhf=Ji~u1=slIPV-kc2&2MS}omU?tY;PmOvZH9mO z)+Vj5j9njjI@N!KypOBoPIZz46DfR(>q8&jfCZvfdm3*hVRcw7Ak_Dk}bH z_5{#RB3Tu%(DbQdH03Fjf-C-{_hA;SI$!Y-5vL&qBqpEEeq8OTHI9S>a58P-B*jBs zxE=RHaBW$vFU$cO=H83T{$O+VEJaHnklAA{YW8PD0R-F%By z207$pxI@=W^Kb>_tkRZXzkLB?B2G**=t|d#!IJnzN3dQzie)Q;`dy=*o;q=wyaE|% z;i7F={kVxw4<_I?+(_okQiI7|{G+Ks5_Rkn>6P=np+}lNMVXsJwOl)SR`oJzV3qU+ zER3G0D)mGvOw1eO^1>rA(UgR0U_3-TDg8LpM_P(Na2Ls%dQLkVkK<^|nReP6Hs4#& zX=XFX@#tzxVg)&ynN6a3v-B=dC*>}0LT(qPFUtER&X z>1j~iq9&XpJ$If(fi$EF4_r%JldE{E*}+toBM6U7v1o)IyW6a{eJ2_TX&uTsu;m1a z?7Bwo<02c^@?I$H&5yx6ZCN|}-Anua|KMa5EaazQbL_XU_><gNP$Q!W6$5= za$zU%46`nq9lS?R9urPyh4DE<=2kN>`^mQYr2X)F5c|VKl|eO{S_|b<9&I4ySnzZV zu_N{B!vlD_%9cnocXt)iJ%BXJrif(JeBr{77iMO5B;_xShKI=HV`yaJM6$2c_fX?; zi#5W;o~$=nh^glrW5Xo@P)SC_T<)1F@@D@-rluW!A!D#a+rOULaXmy6l_I1db zD4AT%j#cs_e!lH`)9ES!Z%Qzyi$!c(bHvNs&h(h2`jGGT^BrklAt_`Q+$&2dU+NGH zXi`=E*`->oG1&^$9%al;@tK^c101){u_~|=)0kpY$hB}uN#q><()W-yRMkcS?1`rx z(7AN9N_zRUv&TWx>L*bfthWP9%U3;M3Uf)4Sin>+e!L%5VH&*{5_QKeIl|D9Aon#G zTzEXaznE!2znoSY&bdFt(wnrl(n65zGhxi(NK?PB*Rt+%8dE+Hwb#74i3M069K|x> zEL=+mVmFtMqQf)ifs+m*Ke;}sTOPyuEIG$f-W^7=Z`!*1U4e4QO+1g#8d@U0xi7&! zFTcgr2udUf&WZcE7v)?JSX%}P7XMBO*`!toVUJiJB&V%@9x?KSYDkGFgsSaP`zR86 z;MM!vx8zs3q94)jCmiNeaS;v0Xmg4rVPnz6a?iLY&CarKRRR%xGt76oGBbl@lH0Aq z*D&NlWZtl@Kh2G$d7tt0mB*431$79~BBt&+!h>Y` zs7%yzHc_jOYWgMIdqz{@Igj}-vtA#E9B6$h(CA_{acd=UCVDxBmGTW6KmmQE5_^|R z83gmLgYE4EyfCv=nR+ZOQDk&9HJa%r@tORDUA_nSe^7@l!O}y4L;(kD*Nw!V<$S>F zpt5@dm{3qG-cY7+3K6-B@BpcW=%(Z-)d4_}6gcB=X?zCML>Vtvxnqs~CbR8PV{Ei} z3mK!$1JfkCl6&HP2fcte7)I7y!$-5>dd{sh?*m&ig>TgJqu96p?jn-Bn4v1f{Lz8Z zw?%&$*cMG6A!j6*HoK7et3;hR+6w`DThj*c!g?oaK~kZo5)Bz_WF{euBH1;QPs(AE zW*agBE*``Q8|S9nob7G@7&7vNv7yF#F%S0f2YkSx*Wd(A!A9FAzDbYgpysdL#X1&NoXpbjHl zykZkMY7W(esNFwoudoUo#Y5Jp6`B1ROTIXn{)+tYM0()*WA1@2DWM#9&SkfxnRY0!S^43| zaAVc$9B@hR$YUDnW}|NH{Y{YeXXgH~!-vv502sMKmdX5Xl!LQsUR2QY8~UFaC# z!~XFl?!tMWihYJ2pRsE2TWl`#{TnRAuRc@Dp|avpB1lVJ9MnJs(AQE2JO0 zmKsL!A={`nMp#ix@LHDM*$*tQ{#0?=Tqy3bgB{7zS*K2ia>)cj!b05iif5B+W9dTQ z7~xlZ=6jtxxAXijVha_Pq{o5+XQQ;hC_R8*N; z06)|^(rSBPNF+JIDYf$91sPhlcB|qw!hCVlVY$df^yX)6^!+hb(NX>fo!mY%LfXV48K2 zpLuDt1o~YanV@-|xl^c&=sTca$_HED69`Q7^z?M>#Q&yd20OBf6*qomf1u+(xX16s z0P#b)-Cnov{rq}K+Qh*3za<7cNv*LkFv&PqHY%Kxo}u~Cium(6AL%oseT%6g?c!fx zPbu09Xnbid&kG;!{PlOC5BCZGmk&c<>D|Te)%|h;_J9U*j4cogN?NokbXIs zH;{B^pAyh7zP)v&K|l9vRt4-=L@xu*X2E+<1uayk94mq0h-m5Hft*VhhrEAQ2EwFT zKV+VMCWF7)hI+XNGKU1`k<{&1JpHn=G|%abaQqIQjO|Vqr7+KHi0cbB#z5ghHIfi6uCiJ3AwvX#FYSIzL>i*GlT9 zE?p1*`}i(jsP|6sgGaLjRVE8O-V;W?ktM_I+_kWuA<<{pm4JRLvES`c#!vW03tFVw zKT#-YjKVy9clntpT%5-~96z7YxC?wVlQ#}DI|zFyC@A<;4*Qcn+}zxp*?&!e6#L^L zZhzzNhvxUAh2x;~XH&-=jC84HGtvL~B4B$(4KB}IW4d28ugTh->|sV&Cs4q{4D%)^`TO&rhT^W}*4(OBY^H82kJG4E^x#L?MlJO|G-Qe&9c^Gy2X@ zB~ty61pD!m;hJ9=f`cg3*LLK$A8g`5ocG^^IF^5Y2<&IE!tEo0P;{$8vH2Gn)vAM( zzah~7`wMKxnOjfcK;96y{`_O{$1=^w?B}cbA3^E2UrORAi&Zk`1w1+t5JIjK{fj$} z-|ug1TfUV1=Py3IOM&5oraIh7-fQ z`Y-SQ_b~tW6CKxvt5E7TTJ^J-N)p+5{^!5_r?mR7(ETGzvC%#dI(e2W^FPJNttUR< zf%fCu{9oQ^1N8%;7CybTzZ1TkfXSYJ9Pg+lF56{g1b*X-TsJV(Jjd7yLjBOW=|61~17ZM}1l3}hhcg&crb z7|QgUV`V8Hi_`z<_dKyo&K?xl+ZaJg9(a+3S~gWRHUcC`)p#>jbc)YE z%x96s8M?Yw#3Upn+G{gYQ<)i>Kj!M_<@l!VIlaU8;BEy5tD|}6sK&K@6%M8cN}Mq+ ztpDK%2u=W%|84*DJ(~mfK9q&ppKXr8{N~LU4h(5OXAXw^NptXI$FIT}i}!HK4U(4G zDQ=aV#Hn*>`=0%N!+yP^zQ^gtXJ^$y`KZ~c<%`4*H~g)W8N+6h^ZklmL|NQXZ0FxV zqR@qkgdef)@1OlIHx3a9|8uoU`5}ztH&%=+Ke=I=KiuJJ!kp@|kTzcK@4 zo5p7hSfql}M)Tqqsgv#I{ntweADcKO*jSB!;i@|M=VRkt=Y##I_Dw!8eLOp5*zwn+r~y52v=m5Y3BicSfx}SO54JopVIG_O9Y{>!c$2 z^lc&b3n20Abj({N_7uNtG>HDyMm%eZ03hvH3O!l)7{)yT-+8vaM$E3^y#G`hO}5!} z`a% z-0E^*Ug-6Zus%aX{BF;spQ^#%Yej|haYeSqzeV0I%CnSF20RpD;UcRJSMIZStU9bR zQgUjuBw|f{McCS4b5Ly6G=An96?)Nabk7QfW^&YP3H`NecE|QszTy&gHX-u0ARQF* zx8_hXPz&#TNLa9VAU43>s2O&e(=OpjWg)I}`!_H>lM?k|AbhM)JihdZX_?3K$|V~1 z?ECu)vVYpB zYCGrG?sN{a`&l$ujkpS^Mdk##1i{4X9HW5Q{YJWrKb>NJrZ`nfNFtT*Os6espwncco*vr&I`Go~uFPRq1`tu<;@F+Qbx7u|`F{03cXd zk`BiBPjM<4DBBsz`cI5^s>|V}rt;)j8qLB^8{5r132QP_iQN9pySb(}`1*El)jAj@ z95<$!W9U>0iwd^pQp(Mg(8WL9$4NQHp%j;P+Csf{Od!=vFYK>nGuL8LvDpH230x>!#8W{Ek`%`s#z85=wltJMXJkeR>l9PEsi@(%yf8)=~X(8^!$zjBi2SpjZGk z;da6Dpi#3K#oJ#qfV2_?y2eSoP&D5fv1+y4PSht#q&==^Z_e|SH8Xexe70Uxjd?&z zHey{dHJ*L%vxN0kzCys1GV{aE-WMRk?4+2Q>QW_N6@OmCt^i z1x8OQU(_$QtLzu zW{7oker3$;*~Rk`6|W<0+uwga5llTZLcL%nDz_}bGRoV}ra~v6STY}1z0w%*(m>vB zqpFl;hQv;_dX}{M6>~QP&?CO%_>nTk)@iGIPq&Lv-$u&j-QTO7BoSyM7OPzkvis)k zUR+w*E4o=xQSrc45SsQ2EM?Dd#`ZOI+H?2XZ@SOK3Ox>^IO*~!f?I#oaJ60Q_?`R5 zFJ&Lwmwo4@0m+D$mRG~_Yu|~fWUhBhotKVmh^^)eygY06jhax&HL_+}BIjSk^=|eJ z1F76`$_>21pcpSfd&6<~&!>8Vt2a-}`%k!aNBluj#qxN5$G8S@h2RGk8r@lC^~vdX zCy9?zb}EWq;d-btCG2 zsq{h*$&;|zU2_ZzPpxjnwboLP6~3xm(ZxlJ>$n>6Ho3YM%@@%(2p^cLmSD-QnO|G8 z%&Gfy|DgaJuDAIHrjp*%k6({CFpQmU^zIp|OYxY^@p$eLa>B6W*j%l{W!JvC))8wq zqaio{DTuuC=W?>v3k!}98Br2^3v@gWzIWD8PSPZO5+aw#Pw0)B3CpuW_T3CDCUifO zo;_TG;aAU~iVP(VS7Z&9rcMc6%g<3FYAxlXEFTUQNFI#rd40H6&Cf5kizMd<79cZ) z@hKJGQD!^6i(CKN^f?KL8#M2iUvExmYJ`nHQYTALWcA;?Jf$$luH&(bX``$|tn@N7 zp@Pd!iZ3$C_O3hy?Zs6M0eYc9H-FFb#z7i0Ijc6$0&~uLA5rW-Zk&eslqOEjMYb#a z51Ejh19==Lv{q_bo&vhmo>%C@s|}e33&q|b7DeJ>pJE9?F+-I`}~kP=e&&)3Aj2K5Cdc=>FSDDloN>tp`) zu_6L{_w9>Ns;4_9bcJ%9-uK(QP?=vwdlLQKBmW;|-yKNx|NUP#id1AoBtj%Y5t;Y) zE_;=kttga{J+IqsA$w(y%(8A~HkFKIhwQSinR#=$_x_$Q<6ZC1_-TfB6T!Hq!>jP{>$HT+`1#sZ=li1P@W>-6ugm@l=Jyp`O{uGTYk0>y z?MFit;pQDv#j?6bkF5RBu+OG5a48vzaaOga)X4EtGk3VkpU16w zbY%L>j@Zkj)QO*mZfgoF{Yw$FOtT;Qw6AcvigST+E$B4vIRw|lP$^ZddEC}3!}usx zxiurJ%)M$&4=+B%ybHWbR?kP%s~6!G?Ds$r?_%h`w$9S)0B&ckr;y(y87Qu{eHRmI zdb*131(cHZA3LRD?+v*Vjwf(F=hQJ+9&{*GoiknUjfm1jrQYtx5BTvYU=->B$))vwG_$JKey z!EdIa@DcY;4ad2f*#Hyi93-2}PXig(^NS1b{-XPnuFt^6+5Vu1L`dtb>W5p1}0tZ3E-U zNC11L*hRsq>ZY1Yx)w1KW4yL;GUejX!L*}B#1nC>!%)ntLz|S%2$08 zX|xg6W#Thu0UP5(YcksM?; zSnJoA!21FfisE?>aA+L2+9>NDh5zEXTzw<= zNOe3vtvc_tf#+Wn99;nI%STj=hn7;;5!e@~j82Wdjy=63=l5;ahRNhx1ESy;3|`#` z+KkRLAd=3ll@tfm3d24EY65+)c0_5tQttrz3GczcBk%IjoW;*Z6Lm02QJb#O4dSn> zc$6ueo+Au1?UQ^xujg)F!wk`lBBk)|5z(!ZnztG5;XM+A%GHhor={1@Fx$RmvN*dJ z@0$4jp|d@k@8(tTF$2_(P155w{oAK;J?GqKvpCxiqo+wXKND(tq}lty!5;1g{?6(q zxIUxQ!NrfIgNuc;(LC+!=!=Wo;HZ%G&Ju%2c498e-7kj<|A{^TeDw7G0orLjAbZlD zGvwPNo`(2opsC0$eN0M}oIbWOT>j>FztEivLWsL_;XOsON(Ji(XADfzHhcw%m&}F{ zVGi+Yhaw_+mz9#sU>Dsk9t^B|6slmDC1%N`udz^LAK2i9iF!y8F6HCE7QA2lNBp}W z2@oRPsD^(n7pq%iBP4f?F{y8IRO>&vW>#~X4%hznh5IwaJ0KP;y|#uQ-ED%1(t}fd zmGdu*8G#fP%S>5}kdOz@;Tbv-mLQo~JWUfCxp-j7m}#ZT@I$ikNU$gN8ytr#$oV$F zD+D&bu>a;L-p)dj_4l|m;JtpLp#eSL0YIXK=xkI-&)LD-C9bnqtGp15END3yU9I*a z)8Blyh3;Gtg8C36K^l*I$-Y2cd{F;x?|jBaDelwva$iBr(8N{qi&q*Rg-M-po~QS{ z8aQ#4bB$JE6dy#}>v?>0NullP5oRi%IsAloUa-a8FRq1%L6$p0<`LGXI;d0N=Cl4n zY`8LCLBW>}Ce(426EhS}{s_a59q&@di^cJ-tIsO$R%tONxsN(}S5AI1s^)PcwRK_h z%LS;!7WOYKcIo3DQSb~?d2WDL_Fjm>-PpXEwODkmMvg~;yr9opJtq&XxNmX9d zjlvLwxoZO0&+G=tvE0B6SQWYKAgr;RG$!14aZwR3OF1EKw}R61X8xDYZS zD{y&%PMgnJQTE8`Y_X6!_5uO%(AcsY$(PV+!+^59;A!(yp{%B1_OG`V+#G%NrX3bS zmGo-t2g7BXDtNA=UW-MSvs@e{zQpN$^7If(RJA#+#*TN1m>#hYwCD}4Zhx?dyZgi~ z@Y`4-O;;bS@i3|I{A9&VaES*4sihs)WJa!!P-7mllt+rghJc<2$`zpsnRrV!gy)r@I8s`PgDBhyaa2(UOUXP%C0!mXRD^!!g4t>ua46ZZ%94+AC=Y1NN>y{f( zJlMf4%$5SD+fL@@~V_1PEHPt?RQol0H=}+T%_eckBYz1-GNJV#@5Uoo=56<9# zdlGrdqI`R5o$86-aERq0_#8ZL$&@NPUQF2S<8h0fR zW?;g(0<>O4bGOIAw)M;1Rr?y>O}V>9Vyf~3D+j}_A6X*zJW?*~D^v zuZw6Rlgdft8N8E=-p$DeYq-wZ`~wHWtBoUsF9d}iuJbg^-MqRQ6}n(CH+$K;xzuN3 zs9A}yg&}Sf_SVlc8Ar?s*x;b7L$b?k zT9~~cvLRCqMrI3_##`jA-H|n`?zK}ysY}k}^uC@5u8>d8Nk;|o@~h$2RS=_8M7=ig zH$V@CUQtk#(Ak$4(7laAIz=M9^yp2PwM8Wk`r_1+H1zJq?n9qG0Tk;<+ADR|7n^cN zq_CndD?N?tXr45V`mg9}&Lb_)cFItK`xb1xeCW=); z>*Jod*#&!dsa}=!96SBe#_>pmKahkP=*7L@ry)ol43KAQtUrl7JB%w_lZ^@fcK;I- z_ifFHTc55bCn73wpK5GYWHw#LJdstmnwv*g5Oq3tdYwMk=6SZA5BNI`Ye|7( zc*WHyXs?{KZ)yK@+rV{$VFLEo&smwDaueMNLUU<-1^v$lGaUG6_Q>sS0_0vCIbDAD z#pbp2>+8guWjS3L%}*Ma`%jsqv0U%5vQY5^)8%{JkFYnhC@>>a6`shaw8^XJ$rGgc z?o{iNk9yM(&aj|XwgC&3uWp z1a1SnJ;-hk{6CinT3i-sKzWYh-8@IYRrW=y6ecE_=SF=`!Tr%NHA~ea!HK<~8pPJ9evKizl&?GvCQOrV3lSMhZD5J&o9hNhv1gtSO{A zPm7Z3uF2GdKG9FLrM`S%5eUJ<*G+t7Zl3g9ov*CcEK?IxYuNYAx06# zD^*+}u57-)QrqZ%IHKxk@~^8*Xv}N^kK4D(u=fDwFRl>`lS(;dwbWbbX zq0zH2G|=zxKzlrqzw=Xj*5GR?=VUSb;OFVZK;e?iQHtv|jmoF|(B=uJ?hbo;uCA26 zb9SgG4n$;=r>I{ac`apogto;KlVg|}m%;hc_f0{|aYwEuot_x(c4=6kjn&|x`&GRb zn`&mkgM(GF11mC^>m$K!VlC3;Rdmy*fvi9JCB;g3?a9%h0co2Ziy=!#v(p{x+)J>Z zI|=F?+aA_?&@oMC)gqDO#@i4{o>Ldjw10vPtrvyi>dKj9<;>P&eb3x^W9`{6M17vcA{G%xHd-YkXwS7c z(rj~o^fh%{?|LPZe3cms&ADVmt|ww$y6Dj>D!*Mz*e!6R;;6>h4fU!f4uPL4|yg$x3T=8|vDn<$?I*H?%mTtkb4?!0zFRwX1S z4>urpPmEVL)Zh>LdwF5@g1$tJzQk$u4$S%4ereWuKNsy1&VJT;4A^h79&D(5q|;>B zB-Kt~B~{RQz!Q0QpF8f!FR3x_8*j{G!FgP?lo}RJ%60cRFSs#^J)5v8jA=P;Q1`7Y zx91XUp23%RoRC()uq4httj;&9vYQ-Lr~35=h+jJmlw&|^b3YKm=Z|z~rca`o*fmaC zkEHc#wZ?*kE3n1vw%sEgl;-Y>^J8;UM)t-cVTe?+j2<^k@cz_53AyZbI-Az&#fE%N zac*M<7}+v;h5+~(icwb}kVwCnAL*PR#cC5Ms#|jL&Wms79~hq%-opZ*ecZS}L*T+I zPlUO_1HHBNETBoQ$|uV*etEd6_peRTuKn9qdF3ob>3 zx_KZt*KI)mXOJylU_^#- z8@sfFG}8!lU!jGoX|{;pKC~@uql^vLu#e;`!bBTcUkQ^|EW<_hWPNtvww{WQIO-S=-d=jyY_D5N+;Slw80=yPrSw#~|`D|6_=`7Z)u>W>h` zh)S_~PxFbe>5-e~XC>Sv7QJaM%33E3+S+ioE5eo*$81lDBoe-`U979QEzzW7K;iVH z_G>)+ZOFXIg}{&RDd<&9O-+-}r&L#0XGt*ptx6!5g?In^6UN{Yf2im0qbxFE-}Ile z#fMc8JoMX}VSUw|z%LVY^i9}@hEbZsS*}|Gb#ENGPI<)vN1>9t1aRXdRtfQx>6PR1 zJnwD}P&gGJj#LI+jgGYW;)%*N9G=6@Eu|*OOU5A$H+gPcN4o{j70JjjPgQPStrLkw z-iiWqUNca^@p}{!=x5ckLCdOGMD|b{Q%L&XnszpABrbaWp5vXrT5@+rC+V;6OZFw zo}1WksjO!Yvu3$-1N$`>KhSZifvde3cJWu1!NG93nzm!7=k?u=^0Z%tX$l`u93xy- zl8j{{NO=yv86GI)nxqU+GQEFjZK*><$`kI}#-}+%EqBe*2u4LNcg51{2Yx=W(R(r& zHT$kYHAp=Z+p%{TQXE`eY-KrOkZ2EHbEE>s9)VN(bCflAoc>*$wMSJ=1x z45oKLt=rS}@8AB;vJ&C)win<$%qSYqIgb6RXN6qQ)r!~X#bu54w45$nXdOR z8F*r)(Qm{-o74lBC9mFHpX<98`)Y*xhtM_CFR%m=U;ae`@(a2zWFI;V!Lzoc@kH(? zr;LyJ;vqWSB&o+2a+h-+@dEY}ciwPR`R#VMRnn^q!93}!7TMrLg_O{hD4a!&B-4aI z?#?V?uIhQQ z?JH2IeH~DofUGQFoxp-;zW)MWUWcynT;x8q{o(PW9RA*7twTZgS1QL;*R;?#PwJP| z!p4r++*f;SxsG&>_pzLrToS*VwRu%DaK2KEEW_n}75nC5$TX?S-sQR?2!xjwiM9%- z9^^f)+7Si;n)W}23>&e7fQw2kjGLCdy$_}@pL`P5C#b*4U6patCtn>5Ku08pFR=A_ z#$Dx-ez^Z|J{G3wc<|2T(eJh%*Y2NU$BZ;2M=@t-b0bV6U>!*XiA&G1+hwoIFr-=e46J{z?pI+YF`YLuPuR|ssg(DOE^rTAe3455kmg3`6AmqzPavT zsMGkwtDEG<8;`XI?T?$EnT<@tBh#N_kI|n~3D2113ND^Got(rtd5Kox5S2Bm_IpT= zEU9(Y^P0~6k1Afo*HYGAGEEh5?L0H!p@&kb%zwW+=1xG3hk{fj`UjbWw7qrdx`7&? z6N&5&p7}e!dyDY(0O(~w%I@ITqO7Y|MhwcTFQSj3@t}o45%st~Pxy>=~@(pcc|HBif3K+@2 z^X;`)XZqqj+)ncRp+s+1K?Woss5tvV&>tyOgdLUdw)0=E2{gZJqF63ODnrAl%xkO@ zD%YRF6^8 zvDjA}3^6|xjwzKhDY2Ra_-p|}coXFk^7`tis!WWFx+xm5j+DW>MUeZ;{X#v^iJN|N zEf?()k;o8Tif?c029bE@u@GRs-;KQu0tDk{i1m?S27i@ZMwT=m`@`RN2+<6djlP1P zB}Ew=qI*r9%OFUgH=RZv)MIldYzCNIGBVh{A-d@#5h@VfxNHnBpx-^ zRxC~1cvz)H%AJ3Jx^OlQBG;f{6cqn>i$@_&7P~^lnf4?5|CW2FLWIkAHy2Z7v4)I! zG70TuxfR7~`kaL5h8uFMas_=(IR=2xfhnl_V7`KUi;EbsD z<$_xbDQDU*!dj+DeOFWZu0CL;t9%Yc4?B0MM(Lc=wwd5?z9{uOj1CNd8|?* zM`qtW^w47SjxgtEmYkQYUwgic4TzzQ8GVT_s}7B?HOon;iYQcu#;?no-a_l(mOPGr z_OAcwUX%FpE*UdlNEM^`rkPo=PSv})DJnuuzk;gRVJG>x%zKGaFXw}Yjz81+C+83C zLU~01y+9WYCRko$zF?JhCcx!)H`PaobVLJ@VfnLMEccBYXqRK)taS9?!qMr;;~zNM zJ79g@;=}d#4yCI_(n}a?4A3z}f=B52pn}+6)3%u^VWY#{{t!J^mHgw->%x!Lq{VM- zTh4dh`Xa=rR1B*G@6t6e6Jl!OkVOf`2XC-Zrs ziz_iBQ-oJCliOZT?L$~7nYc}V5&hW=AA=Exo%=|>3=7)bI~eNIsjy-_AF~h8QSkWZ z$Ml<9;f1k`yB|crh=<+LY8f}mHIj``&R@T;=2aSs#Mb(f7aY(i#Kj(8Y#=J#mjg)cl?i88+ zIT|FlH;NT=^G^9QACMjbg@<8}ZvBmWNhpBT9ZEQ&dgg`}8vLO@JL>;5blK?(pX>=5 z7U=9&vk+b#J=yvGaR6Q@npHf|M}#|P6V=Xi9D_-73g!Eu+@%H@cJ{Cco`bMJ`bHx8*qq4~+wuH4|9$9wNFiVhq;Cv*de1wWQKEGY7mXork?bZ; zy70q_$+_*+w!YH7M}kM=GUfZ+1^67BpWq&Tk>U*a?_ zIC<=i@D9CBj$J5AyFihpZ@+PBe;jLV|J;W@9%WV*AGjoN^qxy*AJs-_+!X2i? z`5OH}wWVbJ7jIM+yzj<6Zi@6*r301tXzLMcIpKna=xJe;aP)jQg5rpQKEaN7o5lPe z)2)EkG%bSl@U&|{smci#q1_kIOGYHNRXARgAyJEsgOZCIWav2)30qupN$chAOWPl6 zKec^(lxNJu@4JLd-&()T0*W>`lTo4#-Nuwo<+XM3AKP z?FgiwPDU2v3Aad#$>A7YX+a_9Z1kCfEkrR=$dj1g5MMhL?9Pr@lGBBVXuPF6zsspi zz6cBKZ+0e?7Eb0ZY_x5~HuJN$=fVy~xXi|MEpQwzlCVn6B268!ypU2DP2rS+a6!12 zphQzz=!E^O0>5PhRi3W4i3(2i)AtkSy6ryOh#SgqF(rsS{v&u~npf2e3 zS)iDUl3dO@7OnX46kC7YiJOdRqNCkAbt`13Rv#BL0q5%3BeUOD9(de{LJ)I{7-{%-1!(3J;yNx4NfH&OAgaTwDS^dT*!Osa?XZ zB?kCuw0&*N^EL^o&|(qtl$J-fuV60_0Gqdm9aU$>mQq~y-H}yG&_inTi}vv!|9V}<CN*QT?4(aUI|T85CXh^3OOk+fMI*6tkW@p9I`n;w-;iSfAU(ZXjE z{nE3;(eWLx#zYlPT1zaR!FYY^?~4wkX*@Rm<=Coq-9?EzY`Z|3hl=Uw%(d{#qc~e> zId(%BS>o~NXvJX%xZA;cDv*5xLF@RLo9ov}Y_$0&VPAD!$83o5B7AHY zC)ugUzfXPVi`_Dc_G4%T=95fJ3Om&%tSt|BLiuf#BF)JTl5!H)z!ELu59PAou-uPo zDQ(ZxeMr`+sXoD(vsE57DL5V44+OQqHVqRdqS2cfcS7dph))-&SME?%wRJXN4gSx+ z8R)NGin?%F4m~at?fZ*I=`q>dF+n{OzlY$R0b=OPP|7`Mk6$Cj^A{Fx%~8aG|>LAI3BNQIYr> zs^2>pa33rUR0E3AS^Cu0ENC1`^=k&){39cJLv>V z1~+~c5#=q?Ro{shh_F!}6&rSHD_tMKvY2=hSg~;WR1io1sm< z33L?mSt~dhv!E#e&uFF(qut+WQ3shUT^6)87~qWgzUgCli}@#Mo^4AMa#Ko*(uc$4 zO17r;!Pw>#Y>pKmO*OgtL!DJmDcYAT)PHhIcR!@6g`^kQm1(ll|`C|scs5qJO`3_!Mt32XBO<^3hBIGLwDK$}HBBO6;)nzucy19d8)~{K_5M9GQCL*d@Wj7Oy6=l2txi z|3M+G@8SEkhxK-Ofqlm#4Ne#~$02847vYFg0hR=PK2WP1W|huhqVXW#1KR(&%18VD z4(@jZ>_3;KA?!H}i>|!#eY&GmEQ@Oi)KU?ijg*-CDxC^kd@3G`)Sa6pS466AFB7p9 zaJwB1le*D+K2DHPtqXJOh@&w_3pR~^#!M`55J!C=LoM%r>dt_ zRXe&g@zaVIWDk4IBKxwkHXM$~Z*2)`H$Tt*wxP_E&W9;OmUeea<66~|zAcqHed>O9 zw8GZq8Dz@0ZJdrC_O4Y+w{grd{CbAf(kJAXR*4uMkWpj)J5jXjLjG2GySf-?X{cH* z*@ryf-t3QXL8omY1ZCp`i(gR7b=f8xAC?v*pa)5>e&o$yM=)wVGXh!AbXS+aCwm7m z;n9a3X{)|SSYG8T?B;2=!@nj?A^vr72c!f&=T>G{dyaaCczbpn#fHi+ zCy5kb59z4-^J_d1+*n(B8rt$s^4?cUmWiroeOKXxYwj-BF`?H(jh3FXEUI~WjrI1; z#IEEA2QRu=OW>a>-_KilJas|aQIG%E$hF{fb>spgua}uO|AfB@TXl=ct+#4dj=m#Z z)A zx8JpWgnJcg0ia&4u`xGi*P@t{*;Zl`RaBM0

v{_bX9(S!_0+wn);1x}Z6i$cBKl zr){fA(MD%oy^{{KsRm2o-4+TeKQ|<~U79BrcR@PVL92-w7>n4Fq4}qfr#v|?D%*K= z;W2aMitW+4u(~~706`t!SRx;l_%!sLlAB%i(u`GoHB08a%*J{cYS<&RF-#ZUU+$LG ze37R|4aU2;__YwDrYS03YFi*L<$O1|Eze*$ZD#G251Vol7Ji1XQDBhuQd%=~Sa@O$ zJ77Wp^GT+LH9h*}%c_^xApPe$B3x5&ztM>+BOcW;;`8;cMGtZcbU0}6D%XPi7gg`s zr>o0R((nt_(ZOAMd@Fz3h3zkr-@{yfQ!am3j_rI3K}R0o@PH%FXeZiDp&E1;Kx&`K zD_>sLcIM1xD(`U+)W2u>N!Srb!u{yf;&PrVnR;_MoL7lYd&}26$X~FmEG2_D4&t)F z4J=pwl)m~|z57j{%$9sVOOrNCVoM15m(9FJ%K0+k@zu%_50j|VTSWCQ%olIsv7?1D zQKBA;=e0M3*9V>3o@lQXpMU9TUyaJCnew*Ca!@80xMkX#zArKpmC|_{x3Gd@pRBcGuKf6v&f|G%>4_CKoK25!e)UZg_OYg` zz}aA}+{)xT3k=0{>Uo?liGhJXyaMJNWw9~@Fez=pqK3z z0sc1Bu!hx$pxxVt)eanCH_lJZs-j4&vj~e7af@ z?xJGS?_5y};z8E!OOY@dvF*~RG7oQ{~$1fWBHe2;c2Kvh7l4kk_cKK zo`UF2?l8!i$(r%Q80BTz<}j?VdF0&ha)W&muc^qh)sUBJO1b}$oS}x;QjBflNzxb6 z-XyWu0Yy&QpnHIMYudDKP$kWZuL{mcJ%+y};YMod6ssw?`hqJ6dU2g}@Dn*2DGH6@eeAtUgtWMTe$Qd>*GT>BfZr) zCjQ{{9?gsX8iIaTtoF3twjua^+~Dx4&H}&lU8}3O{%#HV8JN2w=i0%k;K}6B!Q+0E zeKximy+|vcAyiDRK#P1qBeuadPC&wWs%X$Zinm*v=u0pHzGkGhc9+S};#^j_!zbqK zYK{bg|9ofSf!RzP2KSuakAbDiHL|FC@!HrX z24h5+(@R5#{OH{COXG3sDBHJLGUtbdzR*47_+b%j)Dx&Z`*M*mj>EFD5Y7=s$D5&8 z=%%X?)lL+1h{6)Kcci;U_CLc;OuA1$2KTe{F4*jL_Mq4MJ~;sVK0C-C(iVI9;t+9j zm5#q);IzZnIcsez8Ctr7)I*nYi5U!_G{JLjYe56K-jFM|M2gyo;PUFZ1(;b}6GD^bmMim2+jm)2_^HiW$$=ijPl`HEiutNugad2Ti7*%`%UkvJI_}?_x5a};qq7#f!DPP3Kfu6Bf_0_-Y8@&sm6j%i@5onGhH;N zW+TT51Xh?PZ5{RM3;T~)c|n?+`y>knnpsaH5Oy1&-f zgDMr6&zoBOuQCg9+(4fNeLf(%AVESTv&&JeXU1>dzg9A0A6V0Mr^`t>kyDyHe zfmO<>xv2j#-OaQZS0`l1ZZ9Q-266>Gt>_>t5w1J^dD5 zr#!Sg1pvK@t;_}^L)VR)PaL~422RBo<4OpfaT%Zsf4G>`IaU`HqRZ0P$Tk~XUioHD zu~6cX2IUb$PH66<@e` zE-MIxfA)aA{J%H!Nd5>gWT#)0ol@H-sQnGhb$HEGyGzH&eS=;-^R>Rv^A}JseB91? z@O=_fm=G;XwNu$tdL*|Fe2puSo3?r7vtA9sz}Oi-86~Em%%6X2q2Tl9Z$w<7T~5yD zgzE3Xq{iIf&!}N9I|)+C)_CNt#0#P4j-NF!5hoEasB_YGG3n>Cdu z=Nu9WQwjR@qp0YMW7H`uH`b3QNeM1~iKKYRUDB^~>f8vc@_kBc_!ey(m_aAw?GejBE10RHfX#AKcFb7Yu`yX1~0`>o~>I7-XilKpl9 z40NVYq2pFL7a6@k^u4}oi$(NncM=SU4q$WX=L25&APyY{ZIHBzvoIJ;mKqSwp|cM+ z6G*xDe;yV@UEPk^PTuZp?5+MeTr_zf^F>;POTe5Fax(g|I=9BurSxKQZf5zjB@ura zbn)pj5wX0uA<~V0WKx$|`0Zw13zLLIkMpSF8bcRl;^T`wr%T&>Iu#OQFAgq5={qbH zP005$+ZJ~&tIE8+AM5;GzN=IDf^jSxHrQQkB<{H~>?F4OPL)lDs#3+`>O%A>XTR~x z3!=uHEz<95BBTmFD+#i=uyw3Mv2vAJDqDK0dWh5|{%D3j9gaPeW?Pp{Uqp*LPNspD z3j6`(4j#ADDIrNY7yHO<@1IBuf(JJxxPKp6tRM{~$3$Id^yyQltch1y(X}-Dg>chDJR ztoJ-N_CI+a5Hx~70_?3RFH|?mr~y>vByc5P_(OdK=$NjiZpe9(%>v&SaAbD#!!MqO zDx6Xr+h_xHGpDDuBP4fMZAoJQSL<{K7DKA3cgX44gh|M;!KrJ!i4->aU&s(ZrW#+x&Y+PQJLzn0pF5#H|j$iA8Kl_}$g$c5!>M_QiYjdg}DB|4L^ z>I$43_A`}>`lj~=0c-Q1y8OH!M^E2ExSe_fVVusYHKrot^2pZbL9>7ohv8h3A{4tj zZL(4ysj%W|wJrde8W19!av7RC41xL9+AmhX^qgv`qbepmM)t(1uABq#uu;lOH%-bM zzSxy?+{b<=i1D4Y$!}tg^k=zLwa$!D60RvzEdL&nAQurp*ThNiOpJBe=v2g_Niu^W zqTeWYO=)r_8QCA~Daf$uLnK1-#wyOncVEtEp1U*0`>6?L>H|pO0NaUYTXE zQ};6y1;)e@$8zu1C?$2yKFF)77_Ppf@6c2q@86t<*TvU1pd?bBhB|z-BLWxdiIq>f zrrbtEpj(!_JKIFBf|g%qe4Uq}2}|WvAogdBL7KS5y#9-glrC3A6oIgizHxHXHXe#z zJ{mJx#glS>#WF_GzVt@Tv~qgjn=vRyoBnA!9{}9o7q0|cJMJjf?k0J=T(8n-Kn80aJum5g^p?{wfL#)T7ALT z(7IHy)6oE@%_sp}<>1Gyta?pDj>+E28mf33|SQa z9u^=)%B&fT@b(z|WYvGHA%jk>g2UCsbwm>W&Kt~6Q!yF5J2ZILJu(}Ee~~ot(cd*Go2|iXzLdqa%hM zEyQ1aZKU{IB|dd8D{UjKx-ON|-f5R957{TW5I#3qZMG#b`Ltf(115NtU}tUUQ6Rr? zxJ|46@=pl!*-o4;@fhjgtXME6_GK0&%=qQ-)O(qww$FP+y#?~XKsrh8+{?~#y!qI1 zVT}H-r%@jA{&kUr^+6%o<;F=CjIm=k2{xrNX9JbU>I!9PvY7zjiC0~Fe{GZnGWxP=G73dcqq{`Gy%{(FqXsNwoxk2 z>3EVt?8Vs+opiU=B&T9yD}pocZxO#Hbjr(=>f@JOMU|JXN5=9A-Vy>SX6!c;i_I#LpPAT9|HFuS?=_| zNcM&CB-v|kYaa{}H|VrL-kZr(IudZjq;#&ctOD8fxx49f=+~vD;G9T+nSa&>Fvy^6 z4XF#BHN#Ui(XH(q$-t?2`rDoG%bmyJ(JvDVS81ulj3&#JGg#lnR014g)YoJy-9fa< z;`3}5el|xWElMEJ*6B!Qt2f4UI3Z-frII%drF5qA`%4GxNdgGxwoP zx0y!JfmFL#3Ueni()cA5&o7^doFRs-q{Vf@;D3wn_93M8SMfRR8~IaX4}jF zK`JNJIaF1N#||<3{7Eq!SMNZ&UUm3qa#4!~Ds+9A6u%0>zBdt4_TTau zqe{w0X9j>q!RwXV4_Bpc=)+wGZYx2+V&PB9^O3W%4y}DP-(+k6cp2GU)+IJJTb|1DoE{r8N3zi9W^`8v@it3*6C3A7mafA$5V=h_lF0c(KtG zS_n`8;_NMF1@{iBBw_PvDLOxarLUn!;l!?0!ZiUE#@6bKLvCHtya8W>3$JM1aM#Wkc>^w=-T|u5(SrHWWgBStT!l2mj zZDsx0$2(rf1n0?{_~keyZE@IDpm;2;qAUpOb{}P`#vKQ0_+&Lyte!B#IbI&XZQPKs ztf9_~&db-2{h7tstsim1#H)i|{nDGV50A-1S34OMg(`>~pw@duS&?~(EE40j@@1Pv z$ilstBsx6f+gSkbb<1i%QcG2us$V0a6epQamW!l6uqzebPR;35Y)R$5>pOFH3{s(d zYe*i5-gDt>rkD9ZYNzptxMadg#cV;GV7v)2z0Kr`&Es8E1#p<@$$@C*pE$Dhc_LJ=P4uYn4w@d!~&fRBFE1 zaGPnEvD1GQaY|^=I|=9??6)TX%LMI|Ao7NSzd(88!?d z#zxn!55nFc^rcFhxnQnpHg7P5u;*zD9orxGM@+Nr!t5nI_MQMZ#~ZZgoH`YF|Jv1I z=!=wI7MQ>j+9z@6A=%w~$4AI$^{$U)ZqOi3uJMm36qr^at4<=IbO zsmgW5A79!D&aX#`s~hoNxb>4W>_K%&t=aqW*F;FF)Ip4M(5h9h;$@gOw%;AoG<+p4E=@RyqymK! z8d8_J!uF3+vY=$99MgGhpbB?=R6=B@0xhinZsFH3|4cUzMdtTlIDElh+hDb9=3*Fe zDGHN@+S=etANTPkynZ6{l4$qt4VFO=^`2r~tLig|fh26of5de`xM7uL-;@^S#!b4} zZIl-BRc=bIHz`E_8eCsgs*za`wi$o2RzO|im>|Y7B?*^e?LNWXL z!}FJ&;^7?Y3KZgY@13h!LGnt@SRjVkQkx-dRupcPFLhr{oF(il^ zy$b>9#Mg;Mq2)ho6k>gOP(|6;?3_w33EWdw4X*6Uv0MzRw(#gF8{qtEa`A7i>vnZ9 zOrd1vtGg;Rx#S~U!NLh$nGqJsn8lc7{AwYH$3D z+Q>^W+`|m@3fiE$7i(^KeE!1sl1-+MC%z4CmATrSWRpFeGR=zyLo2ytVRda5$byJ{OG8y+ z=g+QK17eg?lT>!jYJmv$0!E3=Pza8UtJVl>B+s0S;L!WwGtyra@%AU4!6XcF77X7( z?cr9``!CLywv1~ye1@E`m+lK;-LoZcxJUiR{kLb8CN`qNee1*_l*-g71CnIEsGw74XBd0Hh1ol;$4)Z@4GB#(Vx#rKP_TDYe_-x_=46#-p;nC80qBb##V*kMa`{=sRi8yJ70&K3;q138l>E4A z@O`4#2C7LCcp&Lqp$sM~qx62%oPo>E7$X7X-FLW%Fip8~)$Cr+jpoE8(V?F$Sjb?u zd~+S6Z#oIxznbJ^S2|gs0+U}|X!l6bU-!qmZbR9GqRBXum=Z$WK&R{a;9IAhD=w#S zbsyguPZQq+S6xP$&ah~5MM&Yqm5X8>+%~VS5&DcXR(Va$iB+k%qDy=oLp+ z4k~y5kBaBdkF^T{K{}dR#0d*Laju6r`6VWNyJ2!`yk1W2^?&JPTYuKnY#UAZEtUV$ zg0}PdKhFci(&uT=X`!$Ll~;g$jaE4A*Lh*Sdxy6L)`T8rpb`I%!T@>}hl*Sh z5(f1@l?Fa_ni7bL2rS{*aE};kpaqRuqJ74@Gz58!B zjlKuOmpvu5gA46^>@W|Iy@#1vh@JbY{Y5_akMct3p8jaYy?d+wI1@yt z?}P3Cde1`q8OrQ1$N$sf_ry>R^!GOZYAy;Q`XN}*{}7Ary{>=H7`HHjXYXYu|60)W zf1@o!3;k0fcURVqh8IQ`?>8^Bp-($lvkFZMaNeLCMKrW`nGTX zA7gJF7iG7$4a-Q1D4_x(A>Ab*Ju2M-(xtQrNOxI?bW1nLt$+g3FsL-r0@4Z$Fx1dJ z@U03=)u(ZLlb@IO-rxM*n?3%=sPv5lx0Y=+utlAyGi_zNCUxrH@;Sc^= zyZ^NBmo&kDzpt`5d$#xplkW6k&anceK*Qggki5TqvQZ$xLWt)HNT3s(hO8Kh0@NJ3XKp+l`2s>Kfc)IG`Oi=KHCSpzN^YY!#YHzo9 z1MkOorw)Kko?hVisC=mgt!`WNFONhDkJpZ*pUGot1Jl3D` z@GGP}UT%IY-B>=gVk|JY1}x-L%!M3T2+tc4c{4>vQJbvDF96+)j_|e6e8zQaGbG#5 zILgW8W{EM{|690xx|Mv&U@a!u7XzUtSPSUbgB+YF>^i=^2B0$lC5TZaMYNLO-N`T~ zpHUsre}DVaOP1UPmke5_Jl+ye*#m!?8E=>y+%z3Sk3_v7c|1<0+>9QsWL>)QfweIDjQy}i3SZ$zjZ*+Xz2>pS7B?OEW{k^RJ zQsE;ZLQ~z$TA}yEdl&1-#se%$R7u3#tslu$PdACsK-3HB(nVWQnhl z{)bra*Y1&p058~WsdGG)Q=(0?r{M?ck_;*oRx5Ik%R4T26(vAYc-5jBA%9(URsdu1 z--tiITv!(!Q1`D^3R#iTL!Xz92U?A%Ab`Q@j%Eta+t(@-iB4)&HYoAOW6VlJ8oREolEtva-EvHzUXz%#5$uWF#GXhb zGK};&S}jlLr2dp9w5z{qivg3N7EGu7%PN8a#KsC=Il4*A5)gKO!IvzOh3oQAXBTw< zHQm6PD5L^_%9{CnHaLNF_n296*Wc6y>yX})N^u>3qbmojy#ZZX`iWur+Z96{u_i>y zwoPpAJ{UrFC(hd-|M(gjB}Ap-DWUgKnN=jH8R#(a_<;%w=(a5uCzLX zWdw+UXn@-^-kx%!dhg7l4yXSxuXq|ZgZ>UMx{@LJ`Q><~>kqsh`V)CQ?G_P6!!Y3Yse4mL<4%5w%U=E7X1z8a_E?m*ei@llKx%IR~S=R7h(KAh0ghN0%iem z(EhK!2~JtsjQ0VsP8tB#`E{Sa8aYlr{-s^uTmJM`^-@RQeoo5#0OZ^KtIHpp*#UJE zIER{yAJiW+BUQ$?^Tj3f!7#3Pa(1M>vidp~vh;f6#LN(8}Kf2`y^~AacAgr(e z@wpc|IN)-pPZ8#|2{2JV&uaAkpF=!C=5QziL5mL%U)LUK+e?IVN8aN(xtY@kznoJ| zI90j-W8F%iU{mnJDKk!DJ8mj}Kv6bQAA#7PZ{dqET*LM)~2dgF?JrVR1)`K*+ z^JImW&COP2MYGfTJ?8KTXW9K{3s=6S$zBQcNgAkhn^P4vpa(FfZGC7jZUs5PoIQP~ zUlHf;?K?grd?!KIjwkr=k8%O8I0?@-&a@}S?R|@ISm2ozO;IB}+hs!IAzDEr?_Qw= z!5m@&O*nTm?4(&XdKbf#+*K~Klocg$6BojT3o#p6jYa;e%>CA}6CwFc{ROtYQp1?O zg^oz~ykr=CBs0%VG5PZvTye6DdF(u|4dUa-xXI@ERR>dBQIDGEH=A-~=U&y4GwfRB z6RTW#MgPI=vuud-f9Kx&{nEeI+y4c+?1h_|d}MR{JBZ1wR%OlJnxuJDNb<^=1%6s+ z?R~ndhx}14#a+0d0{79}(H$gR%bB#Iji4uPq3wpP_5T6+i{7_U_O!*{FL}Ioe=r7F zh5;wGSuVng7IFX{5i}Y43C`0jnxaU^fSpoSj(0)EuZ|20z_tgPMlkb;H5v^5L1Fa& zX2Jp?3g}UG$lwFWP92xJSRtddge`P;AN;7GkF*uC9vZZUL|h_nF@XDAExemiJHTFl z+V?Wchp~8i{y8Z`Z=^LoN=pMIg>cD03`B$^abXMt zc7awbSxLb<1NQkw4+4$N+}2vn6wa+LvvKypDHY^+$K!e>GIe)u+5lB&+e`_~Tdf)BLB0JC2NJ$-pHj9a;HKMuxz*p(~h@|MdaV z5;6bGgt3l(bFn#%QcBe8KA=}6%@f@@iLB^#0B($%n_IQ7{}kV^M1{sjD1`$+<4Czz zH*$5vSY>k&dW`sgdcqYU?MWa>{SSMLcfNPcSj6AU3su5E(}W%ylKtn9q8b}w@yZA6{l6cnHf1ANA+Nw82`N?a* zh-6C(-H)u+f8cGV{GiU5eE?cP9b+`v_QoObfRtgnEs&wcCyZF))c=B=2)XiJ;*Yg% zg{)Li*E?&dl?8n z<^<{J-3O{3?vFhbWelRPCgw3h)1E`RN!#QYx>YjkNwx0kem@`Z>u@xSjbZawB=7&? zTs?A@qVkRiT^BCK-Y>~RZ{t~TcRs@20$e_|x>~S!V18*E%auL=%|GFqTFL|k911aV z^pP^UGfT3^uB#6c`YTs+wATO0^rj^PtE-!dJUOJBK_ag44oODT%8{s*o(DZ{F>Rm< z#Q5Cb+W67&in8qu$=aQdfKDLj==zG#lz2m|KYsivvPV+)pZLkrREgxTvV22aI>Jy` zFRHVQlzPzRao#wL-p@VGho1n{dhQ`o#TjB0#)n8EKiBbdms+QdgtZpNpg4AoqGyB7 z_71Qx*!5L~flx16_WaseO1C%GLjIohP}mSsS1tHQonOM{XC3=;fzL zo|mG;GOM>=(MOO)Q)Tl(QC%Dgc-JE^!^>)gK@T1qq4x&>lCHL@nHNS?YKD-L_^3Y*xBs8#hZtab@7n7e!hC17mriX zd`f9(-#GZxG)k(dP2{m*6aeJ7j>bbrv_My~(m+8(BmZ(B9cT@fb>DjOPt3hPSM-$C z_0Ko`%ZyQ?@#ViLznn(DKN%#$vJ|9?X;Qc&#X?OZ-o12(G!QXMcmwfV&3NT`dxT;& zHt+X~bX~*nJT*I~cl~Ykk1Ty98s?-d_0}TO5nC-ZEfe#BsvEe)@EbSCryuEhIb_-? z2Ihdb1i&Rzr@A7v)Yo?&Y1rE>*GIg2g~k#G7dEbkX#S;eKzRRkA8?u%0s zi?$U`h0ycZVYhw`;yze*J6vZOd`?wUO~Ct3_b)-VG?#KXYJTL6rcM`Lg^KCuBvf=#eSiBCMl? zc>;d~)JKo?-&6?f@EY1D90>K?K3drD2g|OTEf_qq)xNuzgu$9UfhGIsX6c6TXzr-*8!oNtnfmQN5Lwe-Q|DCZWdkPesCk+i_-i=@H*N5gjuKt9~ z98Ua*x1ItOWl7_mL+jcRAamVq|JsjCS)aa?>ffX$8T6DzqxT(YfPV*R*YV<>SZ?s? zzwHXt4zJ#A&P9^lDvNS1bmyI4UJBt$$q0}kr7$?JAA(51)ei>FjNCOp#t zaMpk#&w&aKfX=*gWfG_H%J_ay`m?9bY1=K;x z$)5;c`?x#w=tYnG@lo`0=`KyAG&VN2e0l@Ac4am67c$KHKYrz&=nSk~lEqzO%eQE> zzhMtA93}v@l)~GSZ=qZTaD~$()+94z`-6K9Vy$ikoU;TdeSbdlubL5HcUduwKzs0` ze6qzqHGe$J6F)MrB;jDZhCWV;;phsMFz9B>nX}+V2T>VVn3cDGAfz{yR!bss8TS8W zQKT*rlKhvup7C!)hx);p$9DrHs{l#;TG#qnVWZ$6RP zoO9p*EK5!jx@|`)^iYZ{sIsp0>$3H+!krs;Jr3XLNAOQMsx;!vs=mCgQ zh=)Y$QZ5lP4(=Vi;Vn{Skl%?jxV!2CpNuo2YYY|fzj_+R&eEWxTqj<>?Bf~^fEsxX zK=#8M*Offlj~L?!$pY)@)>-aowvFli%JzClLyuEyCnkudi`Ad3&b?`KI5%LXgCoYH ziKFmF|G=JkyL3)Izo$|GY6syn7Q&Ra?e+@+L*ug(C}p_k8!7sTj2 z!w~NDXDFG{ZvCZV=pzR>%!ojEx=LL$E}zuMx&k39?(~tewEmK-vP>!0ZD^rs%@7dG z{6*UWd?Ua|noU_?p<^ILXV&F0LjIji`nSV&dJZ*PeTf-aaMSGSpbkX$D79~Zu5t&d zq#R{bpgdS;9I3B$lK0MFTnA1=V1+&Lbw5Crx$&2x@~PL=9;ZhpF=JEs^6W*5nNG|H z$k5Ze)63gQcxT;u?Q&8*aC#1Oqu{2})uRdd8ul&#J=%^pZU|B$cmT9k@`0{OlFO~% zUe!&r?$XH((?TF^%Bf?GnIa&A2$pa5I!1b^4oxJOUpM{gC2(R=D$#u6aY@YRd_MrO zjQ!KMp@sM_2MsXcwLKO;_TuNNaK+d@EGq>!J}zBHDdY0pT za>rzJ?8snIf9uGM)HQ>8Qv3~5IPi=|lo=-v0Y>(IBL_nLoMkn|PICcA-w2kbNs&}5 zlq_nIZCwqZN|=jj{gm@XDGn#I-$(4-`cyPons8(!rLoA!zUBVAql829b?%Xxn*P;@ z>M)H0b!1Imf6>pet)jO1npE?+kSVdsYUP@(y@etVi;vZ|?lAbkOST^aOuX~?m)+?%$R8F1iz{hNkMhl$58fEl7_;Tu`cmyI<9L{_ z?^^EZnGdIPr_R&Q9rIMrH^Zvtp2t6CNQ4;zMr8@Am_`Z6A1pV;A!eRkU5<76x*fxmF~A0Wl#&myL&h z4J3kpCVTGpyW!--3;oeTp{h<}SGV=pIZk3zd$&c3G`sBzVE;r^+dK%j&y9(=P+2yX z4q%6pT|(Ci8l(sM7F4gGbI|Ks-Hc(PqOP`;zWwi>v^godUA=wmG){>qz&RtD_Ey$O zeOViQl;XDNNV=}D%cw-gYcxV0@u&a(zq+~!u(*SQ`ltG&YL2dKa(Q-SQ94o7Vf>vA zv5fGR)5DX2FYa^b*RLxA8ftPxf4(-|u5g6v-{RYFh45s4=~s;SE2(&DXU>4r{kg}) z&%5+7JegFXpq0q|=Q@J!6+5vERqa$HQNA_9sfo)4-8YDDH!=+{0phIs@nZh6kEder zr>Q^AInh9WSowF;hmJ^C_xDoloS=me&Y~ei4Ll|qyC40BX4~eFK!B^=j}lf)z0|SQOe+c z66TkF&sJ>3B8EunKY}sTZAkWQXYeGDo+I}Nf+IP)d_mVghy^e`450hqg)QIL z9a3$pRPKh&WT{^qV)LFM@kZ3C)QBYWt+gQ{9aR+93Q`iOs*x%8!`v2=)_THR#w@=E z((!2XVdosKWPRz?3Gfyn@R@N$G#A=NnS1-JIen+6vvn<~j2w)XWTmsaCnQnb^xPlN9c=vS^?XcwORPA)rS` zVw4ND@%cuNjW6uschhiL+C;})p6pygv6XETemN5z(Y&b_3Sgci)n4C9b=yi}tSWln z`CtVnCbH+i2SF{r$aAxrF%t_@myr&8Bi=7pAFs z*<6;7zX77wM#qi=r0V^m5ih!By70MeohRw#H(peAgq0?CX)_6Ke`yq1$7n z^D!w;$W%l`XDs&OQi26ClC@9SlR!*|k4bB9TK>gtsQK~j25ruxdzx?>l)Rfa#agnU?iw9X~ zg!2IJA4|SrmaaV7qS&64Jun>>9R^}$>5`u}*xJQro~_rY6ed1^?%ud(2wIhJMR%r1 z5FPGTc_5G2ZemhUYmI`F_ADI$0>hA9FEU*$^*Dcj-`s|eVQel zT&fbS$_dxpt7n8I`xJ5ToxFP=5W28MHb1Pdfm$2RU z>OcVWB5jx6d%m0-8{<@aPcNwu2j`yi-ZP2nF{IWP@`ihp$fv4ISXq2nc!t_jrw@>E znD6%V&58a&1^Qb%@mGWA;BSOIJ&&Q(KOMuJ@-!r5~je{utVBg)CBOXxcXU9(XGV!vl;s#4i7=HZjlg zv`y49oC4G+7MM$~UA-%3;{vkLVzY=!7tBgsD7D(%P^mTah=htV{D|*MSS^};5Y#t{ zGzk;cxLQs1w87-P)@V$C4p|^04-INaXojFTrvcOB&X05)Bo~8PL)urm1H1Y4J_5!Z z^5Mc%ZbO}Cw?-l^APyf3+m0uz(MM0^8x?Xc;qa8NlkRyJ ztYb#YvRW;Ade!vnNT=;#=_X4GBze9p zq_Yx>yft68iO2?~&TU9X)NHwEJSB-%NHkJ!@`lehi5ks=&h4^xDVQ+3Z@TX3!WH<* zL)tHswe(7(-Q%>t=dsgxNC{2%fvHgi2(JSnF2Bq+H?6-i1z1w2^He)a{YZJz!`<&r z8`Pb0Ffns;;T6BmI8wm(w+0b3NW_qPXmSV)qdk1><#A{@9ZIVM?TzgQVRqN911y|gUnY@E7u6woGP)A}6vCUQ= zp(-$j`d$l#h!M++?#u%Z6WOsd=&I0ief;~QjZ1+VGn6Ql6KZ5!(BS@uhFE6G$xnwP z$rpwvO|o;Z#;{&pro)b=-p+IGArDwcr0u|7=;6d&kA{jwHNSWo`e`bHEVgV+ty725 zd;hJ3d=O`;+t>qVOx!74rw5<67X=NYdV8id(B8Uz@I%c}^1&6?5}hy+R-{Q0;MI?! ziWhM$zsG)hYIRBi2c}RFhMGQ_?%&5Enl3VbWgwNI+y@i0g|%fe0PbeK`Yp=A`}&K( zlq;{j=k8|T8r~Z7Su4LbQRt}xNbhZ`#(5BJP3_9#;xnmM15Z0;^}Ot*hb(tftma~? zvR!RT4LoK7$^hEkeldc?lPalnZ>cj?nN>&i@Ib|vXKQM~L$B!H^7fX*%4s8Sawg6I8oxfZ|vbm`~zEp!A}(+PqQ24Kul(wpA>y@$zWu&1Vslr0co8J|$WnQ|UV<7dUo@ zS-t1uz}#k%yV}rNeOr%suAzD*&ZGXr`U~3ok8-J#lEinP zpY2V|^`fzHd#{N}XzbKI(bX)z9}g$_T95$kn1Nvvw%t>Ok%}U7&T(u6F*~+NaeBpq5s;qyOq*E4sfWH(etI|hh~NY zB|3NTN_5j!p0=ih(qN~A!nvX(YMc)`o2cBGQHW@|ULR^-Jp|XAnZoSYGya5Ri++o7 zO4@mRG}VXi$FsyjE17G2hqm)*NaoY21**@`W(OKS3Q= z)kO<+%-TzSUhvEry&xpA=)G#x+dqD@I+~F8r;>H9S*-WU`#pW9?uMQrL*bpC zDWy`mAkip|&51Y|qw6r%=dBI`iruH#0kUb zs{6|D7}9;=T+Pe=V^>~wZYIY{Y9fP^Ji1a4)7D&V24-+Mr1HG?MDdX^=hjH`#+y_r zGoi;zFLtZ^A7ph^)bTXHzU zFfhcB0cHqV3oMWq>Br*(12hV=V;i8N6&r~QEv`32^>i2svxotB1|{LhQJ2g*^u+Tn zdeW3S(dSVDfFQL!pN_KWe{vh&sV%Jt^t-?kc1QEa;*}+~ttWTc=AMhEKcQ2-{{EX| zsuoY21bK^rH}w$f;>}5|yu+^XEa%eiJRB1D;rlTZ`g`BMT98l^A!{J1DT#aEuVv#lHQBTa_Zuh-X1&k z_tfb9f*;B_YZ}5SkqU(#UwVDvo)cHSvzaa~I=1#@@3oh39dg?&$8+U^cNtzUXfIU& z`=KPUmA{^6M;U{CL4?bD?W=Fr;--Yp_gp-5By4qn+UmD@E`{l#l138M{4n|XwwJ_K z5{Uz;1oC-Xc0255X8Kn4$l1JWti}b_Si)lW_kZG0%eszxm=xI2yL(Np!(t*oO%~TR zrjC}a?-=*F)=Eeb@3b+#i>@*-GuzjriPVM^Ka|aq8JIP z5a`^>0!VIl5n_lO_k;YR9y&VHjWRAvCPwlvRk$A9{%4JA*K>Extc*QL@GeJ}HNUHt zKK1y)C%aw%PB~EL@S8chY zl^v3U+<7Er&Wi1LaeuwKq@@7(>9|8u9n*-fSKE`lyU*>YTz{F_7PQFsZTiZ_x| z5k1h(@9P8X=5?}TCd2XEKJyz|{p-F+fmALrIc~5|c_vYUW0mXlMdcSIiQVa=mt2>s zFSX>ZNY)=&f&DM)J)9E4V%|TS@q5cUc4dZ87BB^B)MtN0cA<|!`=s{i85=mKWXKmW zD^pRMVvf`Fg8@iC3^5?f(fZaE1fe8Z>r|-(*b?iD>p3hqSsN0vOnbd?hl8$=rwF6{Cs!oHPNa$TmP2a769^j7#&d;Dd{i07jvIkH&_Bs^0 z*|cw5tug-45J%?Hwe`3u*fdIazTR*3GGl?7wSD(o!yRKuNb+mRSQ|HbPs+8Ai$#6- zGm|W5Bf2Wz(^^cV3{Pi2bs1weaPHQ@k2~1;I@!CR9;ScO?g^9&ulmgwl197Q&7P6o zmBum{6G!oaLq%IrOavK83w*_?L5+C?T*=B-l$czMVtaER@ z(o+aYl{oBn`V!)r>&R6uzF09nX!S+qS_*oSKHD^sAcr#6v z+wxIxB>u>=G)e5K@9vM~W)p~S_O9=i`7(KC_9buYOtgQh2(NI}5UNZ{@BFwkeN^#M1>h3_(>mQH4 z-GC$uykSql=H5;o?b&ejmB?^>-|s0QoU!pv5)c%ed$%zbmX!}X=)L57|E4aP3aLAY z5aJ9sUhqF~(SWxwJ!GQesx^%wF_rF9VbO0y;f*dB^uWdkW`jxYQ2OtS z1YNEFqL2mlMHUith2?dtHt-k#i6TOjINMS0jTva+=qlTM#)BiYwW`-!W_}=m##*pK zg>|qpTm+c{b-FR5O*y(l&)E+|cGeA>xFeq9hdlrsl@pEal!rWvlkIBf@OWs0jVbr6 zkJqhTggt34nneT+kb*!1Y{sEBj1aF$xa82-)ayY}T-44_slST4Pp1dW8&QWz92|#! zXsmTAA5+WEEpIN`o`o3DylE1w#G5H`)(sd=!M>f5FnGaaWp?VhcLF120rr{GF8DCS{P)X$#g#m6nVUB`pV zCl6*+xd_gm&)Vk7!NI{XQTtOL&Hw`3u4mKeVqr?Y7zxdCFsRCZgqgaTU*-bj7uv>a zAzDWhHCr8((LCFpy`A1{N~tSv*H+bKu7(D*C<$1-yO35cFQaw@EK{7TQfa2+2itfmt20hP5lp#YyWJe%S{sot` zANFJwJSvk$r#l-UpgR|egGZJ?;WD~)e zf{LHKr2Nl7*N3Frm=C0WET=yu61+!%+nwAX?zu=r-v2hnUeO(Xe}@BAb*)J^Ta&V4 zt-X-`=PL?_cTEl7iHvi0NN##Dd-=jfqd&;|9IW1H!Z3+s!$qClQ9|;)Iev$q zJGnv~>;0cd;KhhG%0l;Zt^g!MVbk6W@9N6;XS@3|FDM&qGrCpGO#G?G)s$nl~00fFF>K?1&zNeX!GTQ`y|aPiLz#;yr(mXOZ8 z9xvb2YMP%h>`{`B;Xt~DfW0-K(Z`siIMimTDz&qplE`1#35Dyv!N{TTF=AAhQ6?d{ zV6@XKxQg8EbC@-r!um4jCUmkhG)52&>61M=+O0U;`U+Dj#j_ZgPiUc;xHNVcYO{KKJmLhIe2%omsYoH7PEZ|uk>V3>hz@B zh)r5Q=+7-{iPIS{a@#*QScOcZ-}E-pb@8|-yy>M^P`yGq7*o+{m#SB4u3HczQt*r+ zRgdvTK&fL@+&r#i6}0?H%*Uz~PIspsE{83VVt9m{LYDer06eR5=B5Lc_*9A%GL$5X z(4XcUu-A0s`SUZM`(Xnu)UbV1_OXP$70aEuMGJV_T^0GEnaQawtns!_r~??+Ow||B zu{4uqt&fJq_$Y%6(Kir2Di#RNrjF|v{#H3}Y<64Wn)!?sS;(++%vfh|6mYb+5=xMq z_A63rPSi1k&^jF&{adDqhr4&K_^jnkrVVXY_kIw~z3p2?%x^W$kvm-e0;-7rjLq1q zyDSP-sg>tmIXly{YeXuyyN<`Z(atoLjAIv_>ygA3X5gMv?cgc1^CR$p_WxOv5mw+#2j*;Vd_8349U+Q2mhMs{OY23+Qgl{33A@w~X zo%+Y>*?h}*^u^-X+hLc5l^hhq!&J4F@AoG9EDBfi91f$5UWwhTSGU@tvhI?7?tmHM z_UN|zf!)lQ{#oft7hyd^_i0ug?i($*#1(ISt%V9#TGZZYdxhTN(B)IUnq;4GDI_dv zOCp8K3PC=$tDt2YjZ;)BUX7Jg|MQECVnt@lKKA9suan9wGI}$YF#zcC$j zaJU_!?r3nWQV;;N5;_MEH{oBR_4CvE0c)qIvGNT}))Q+Y9bmd-YKfi%AU8d4a?Ez>F$PMarElMe4O`OCMD0^BtR>H!!CqzR9Fv zg<)%Ccn+n9Z2OHwS$+KuU!&%HjsiNry1N+O1z|b2_bMMZGKEsc=Bvuh3D|J#?5Id| z#vXEyu3SM?Kdy#ehv3TIgfu#9g{O*B&uAUCEQP?k**c>Ka12>etKGu9BoG|kY23ad zQI>Z+Y$o1UO@-XjYFRogwITwR7gN`7^Ou!9wYPOB2u8wveZQe}@{^BVI`}L0KG-wJ zakA~L%s|CAhLTKsRO;1}8gU7NsCaLaay^GXVm$3kO+e-{q#=kjX6Pg9E}uq@qP@Gh zrHqc~JSX{qqy=K+r(&MQhhUYO>MQ#--LC%I7QJz8!_)^0z4hsL0=!#WqdAwQd;z9w6#Z$Q$9P;-3Xc$T_8wt=yz8u(LOk+wX=IF3bDO{D^s`q zqECS`RV%Exw|JopHLVV185revPSc@Qxa`s7w z(yi_GDdoxUWy6Buiu3p0D0aOReoPY{RhMCxONMd1(puIbRbi@PUXQFQ(6UYzm@&22 zbzy2vF`aqye)@WwUz|+RU&9)O$5$|`g15hasua8H^H?$0EMl4mRXFsW>OjREfWVaGH}KFryxmF*6^ zA(1Qka^8(qr;Ep?_uDN~oXtfdU+?}mj-_ifj?5?y@6gG#mJ)V*0+ z@t}`UVdvA+0QQ~LCipeg5(^~82b`NBN^)^)!qw~sc2sPxVyp2)%&Yx45-k+SoU0i0 zDh$4KHQ1qXLV50b+7{&DCNFMd(kd7axXk#8buw=V;smS-Mj^+Ghi_bJtDwPG;~)sX z1z64TqJa_e7>Q%gZp9X~oXX9kZr%&q+7M~Po^cI#Qny*L2@X zH!N};@O7_v4i?Xt1lO5xF#RdcBf91epOwbRA{RvGAXY`zg{5&Oh1$K551OMnHrLwd zs=~Xk6AJr-U5I1VL8B`d#DL76yAR#QjL@uI5 zmGHif8DE2X)5U2k(<(DHwuO$W#AtO|1w`n$`YTt$L13STJ8J=*!8mMl9fq@M#j)A? z>dQy7Mj<`L1owsQN~`yl+l%bBI8}%&Fh6qrbJ{?kI-+oSq-#3)~^-ADZs>1}Ygnhqi-n6LM&=<N{yR130PYa-MX;3^|d$)pZ)QYHL~NfOZsr zDLpF}I3Xn#cM}X=0u^}5`)VjxBa)P=>UMvY1oPV&F&e90>{crsv*`|ASEQgt_JijO zX%XMt$sKH97pLY2VJ$r|6!_QJ)v7e5Dt3m6^z_l7KVLQHy1u)mrl8iCw$a1 zUH+2VQgfI%chj7pW7qM^1)ZHe#=(jn5);*@XN0nIAyP2+pQGcMW~UlPisj(mIgkbaC9*9!N-IhOOC^P&i30tk)ni1 zWOi$Io<%)uQ&OpAYfv!v?cCCf$H%1!xA4~`NN9DZ^lXKrs3XuJVmb*aSt>@Kie31O zz}L%qE){ZA2c{oJ1RfMQRe8QHk*ikfcr<9(nyT8a&(Y-6SmfkBlWyo>Jf3oG{_;BQ z)^$JV7?#61+TK)Z2WuA8+e_rmt{2^>UU{t6_YuJ3Q`*fW4;9S5o)v7azwg?z&ePe} zY2-62Xe>FLy1II&$Z?)g$$`cXN{G2G)M;1e_JKG%7$!0$khF`p7@W$pTVt<$$+kNjJ zZRtvb?k=_ImPdmL2)S4NYvvAevky=WZ31Ms&XS1hJxby~L|&P}YOdg$aTNN#xaUNc zj)imK+PS&riOE;8-EQNBf{RXC)8b<8VXJU?kKW$e_Cvet`RaR^h{2B^h~DvQh0Vzqm0~Z$P*J`4MI7^E zr3T~Ca_YVj(JevrX05b#-p*7!4-5D##S}YW`}YuCCM8LIyI5Av4c4pcFYP3s%2mh6 z(GZ$r7(7x7A6H!BTR?dxxn5<_iA^J7;%X&qE`$Q(mt~R~P+#oFwsH^$U+7QHGL4^H z9o9&nvl}iqG^G9}rJ2>fJ{*djKl`AF7tXeu|K<7X%f=Hsm%4rs5G<5$R9u9&aQVg^ ze36KF8<`TZRy9WJoKO-Az6vjv8lPK{3X*Hz4at@ZpmIQPY_$=Dg~aYjD1sBpo?+$q zTI8%9`BqJvZiYYIWNO8Yd5h7J!vd?CsA~}1OByt=Tc*43=EblUxeB)s87uC+jWZvG zHwd9Lq!q z*g`?SdX4KWC=}yojVq9#LUo^ZhF<$Qg8_r;qw@j}Wc=c3k7ejIBzmS==iXq5iv!jA zz*<**bUdMzqv7nZ%8mF^2}LUhxgw|9pOkd6@RX)n?v4aIj~3j_{n3@RsZ?KA z;nE28`|pQ4^N_ODu@y~~!vVcy{AsdhU-;<0R~ApK)gMdbHe&Rx;2WShms|SPEIz)! zwv8HONq?^MKdc>yQnu7p5O>SWWlt25H1`=%Ng*G)Tj?q;Gnb;Nvyb)GZHYhCIh3v` zN3N~SC}v=M=kC`}%D#tdsfzKXikvmm9PWsAq3L0EtG$7}3^R)0`T2t(PAz?j#_|Di zi<%r1r|xFl-8fRr+Z<{5^^l#!mc_|&*nmoxETW6kvMek}6w=MAidrwu8YOs|J+46A zMxw6O(yjty%CFptNxJeF%d?=8*-H|3A!nR(X*hF7yMM2 ze(IRWV1ndLa@e#`>0bZxl7n1gW$1v*VktJ(R7vm4rIIv}SNd*!6zFIx(A&PI;wdma z&6%CP95c>tI%95v_xQ>iyhd3l7dDf3*P{^yjC!R)TuJ@HH)h^NzV!@?nK4Rcpb&&h z5K0d-&#rXBs=RFb`~vN!3nD>S-H+QPls|Ywx=uAruc-x{HVH^Y$zu~FKft{ERnB12 z(bo{)W%_-u?0~X`LOgE+hmw#tO5f`f!r|7MdiJAu;@HZa7R$Qka2W|fm;GkMyz=n3 z(c9uX6QoMh9!R;*po%3c&RbWjo?3UV%X%$cH!V5+;LXBA#U| zYu@Q&Sf54&5nkt1#U4R+t5Y-z(`RW5p0U|RS2cc(^e5jwQ+Z*M zhHvggYj>emcfBlNZ{BF;vEC{@i|gmWgMU;(Zs7u?l3U~K9qf=K&M;*|*2*Z=-V0(` zY8$#2K7LWpN>!os7`N~eurdyU-BqMoNn03R+}pa~yTKxJ2ICO2`wp=L@jp-7i|q3@ zN}?GmJxsy(l_CwH7V;`zCg_GhcxVX&F(xazjb6JYYUJH(`Y;5NHz0y1y9GIq^+&ex zSOP}4>du?UgDCyAM^!xganN#rwSd3c9t7vl7opjP=}lEPGPP;3c-I@%y&S8u9zgqy zqU<2g-s$=mz6Vhh*j~(AqMK5opJ=B&MXd-P&4%BUKhQ4gQ=f9Y;Iem>?8mo55Pra? zZ?QN?x6Qyuf&$vmLB@?zE{lS2y+}sc-3$r%M;;0JrW89;OPtH8FV?gl&)kX1R@Y1w zys+K;y*<0{a$!rwVr2GK8lE&X6nl2++qbZ_{@t2){=7nq*u`z7y|Rd&hH9H+M|1ui zQ$?-{M1uJ$Y)`mZFm*j@=SF0f#X#ejaKd|EN^2dT&^rNP%_tSc7!{cQoTJ_s?7nQD zWL9ywUG-k3O8k}jVVC7@kbnqp9}FrU@C-GzTL=@jc7hFsuVu(w^MAh|FqMC%x%~m{<=@Z{2t2uq*+<4s^UC| zqjk4tjegiqyO=y(6M5$4nl;i`?~F++MMH)$`SZ1&zSmu$-1f_%2FQ_wHZBeO>Y&@d zD1W%!YhW*`^hPxvM)Kn|6_U@59{{udYA7v&A?~Yv=JbK!U=bx02%P0=1@R zTYI2EX|nVy(qbQP%euzNt0Ml(<#kb z7OLRw_qE#g3b&$lQpXs}#z)j%UE$~oL+Eg<;$Rs0tFp7ME%sHI9*!wC;WsEmJF({y zVqg|wO$6z}xXg|4?Brc9@BHvZ!K?v*$i$q1v~0DYquu;^)vqc5E)X*;I2hK0q!SlC}$4OWdY%TU8jW^e!8B*<27ZuF$O3u4X7C zC(vvzS-hDJ_c&kkHhF))FM0K|iB>bp)sP+;EE|n=1E%+i4)A%Fdb&UAwp;vBU&Bbb=ABt7huDH{IgUE&Zf-&^kO044CA!t)A*# z`|-kyj^0XfoMxl>=A)@thd47*3WCibWB0DzBP_OEIihnM$zhS?;}k$dpkd{CfKc0b zRs`(HO}FLn6n2^J?VH^+ObY2NnJxn^Htf2c8j8gJxAC4i-qkV0uC*(3?2^^*e&Tzv zRRQ+xRviIkg=jPBny?$x$7s2xT4?*B&cWn>5OHenv_r)~_g%Gjl6lwW?TIijL(kPW z$(V8*~@Ab^375sTE+jG<#-khQYPdidl4!p3i(%f;aG=Z>@m2IRB;T1m4*HMpif zKk!+2NnTodp7|vTObuK7^Kjgwo$9#{A|Uv7Rd*|Q+?-l*ZMNrAlZtk6{+x(AG0P+c z({+H6ZrIoRw~lR&3JY+Cu`1upD0mZGALAS=xKW-OvC3iZ+zSG>eqSRrPp;}R(Z6~KJAZNsdfr>;B5{HlQ z_yc(I6n1Y8f+20g8rS0s#y65rt|AxZwa`k|26kN**J93bEtNzL`@E&7?OxMkq{v?6 z|J0cU;86QL44)?pV?2rg^Ob=wRD9VhKKWN)YiZd{Q5oRrDrC4BrhN2O8QkHJ3#O6Ci2|ociQ2QAy%bYndWM^b z**0#41chHU8Hg2pj1YH5eetHRUU(!zkK4~RfRI2eS`@_rxkbXdu491;%` zqGl8Vr2+3(H&j%TACgC03KcE)hm#ZOC#bbaGS;XKS%lv*U8@_^u&a3|vojKUsl;#? zTTM=-Of^|RA!^F0m3+PV<5)Krqpg2hwNqc-UJ~DXAHp*NGVQ{t$+}sTFLlQm8pz}C zGiMt=udJ%U?TxJlLB=fY7*3;))BTAh`WL!pw5~SQQu14gyRJLDm*2W;q}sR3y zF1vPP@S{po09{qq1^!Q_%32Ao<6O}~8<#;WMJB;5B~;tN^wa_gBDgK7{_|CG)H|+9 zf@?Fqv0I8f355yP&c(LDH6EU5*yPc%xst@OkHlJ5c{vT6#X^l1BjrZY(5A%uUw$i{ ztGiLtKW;Sl@Vk+*L))r&cx?xj%yJ=m^f)_P%qSAk3=htIT`ydAYHPL2*RPmekejd= zIP`iRLPc?&ayxza<=A`gc5b~Y=DJB;TAWebv!}8#xlh}AX8<9MSv{7i0?0Ixcm4Ed0vq$xwd8c&3CG8S7od)5CgRM8A$}Vgq?}qV=f!P z_qXq)HsvpB1$}fbg^;bG2jtSD_8yRAN1(z@O-+SMu7q#gOfLTpmp^mF?fW5U%Ob$? z=5Aimdb+5u&%!-YQs`=`tMPG3TgnN3iGVQK6_1PIwd$2OC)T-C%FJ&u>k8^#4CA2x zq?AraE^Rd-VO%}n`rbdDf!LW_9g2BXp@euPBru$QfW5vpn~m=Ot!itR$!Y-k$ldGA zYt8Rgkm=4Foi-GaFy_Nv%crJW?Ci-$F}+$sMJ30ov73kEY`OR`Zf>Hx;H~pxe4EDi zgzv0(AFPy>@|}KRIPjjNdOWhQ&dGN~^j_KemYC zyUu08?_d`+t6abRaOF2*A z@Kdw4KDsvC*;A65(&l3rhgz$)KMx#s)_YmP$jNTbD$dN=S$Y3h5L)_m&p!Q##(x#j zxT6xXl^dDURCVrkXOpgrI)h*&-JFo)TR-v;CfZHCi%RsDgKw8;l6f$d)kk04dum?g z0dMe##mD$mo>1S6?nA|Ir4C|n*$|>hzzKkd&A5B`H6e4tk%0C_W0Bq4Hs3t6HkA7H z7?S+I1HeAv3WPX#taG9Uk8q`Eb8Kyu>b@0i)1FHOadOIYP^=}GR?2Xj`;>oTHzR#N z2koXU=i`c<8EPAwlwCdE@!_4S&Gq-!vt93$b>>5s9*}nZ3iv!B7F_1Pvl0$00&gx@ ze|U`YWmx|&mB)wPB^p6<#oXxDWpk@t{|}>A!vi?j;sxTEL#z7e#OCf9m(oFQdZKGV zyUJa?<}!dC8)Lq|&+*oFaY0l2^-z8meev4qA{Ps(D#9U#Mv`8QA@z>2<)tqx5m~?} zs7|qSlIf=)WF2vPfYb2hBlnbQFcjk{OuLo4tk+eKnVXa%QMNtOmg2$~Bjog6R8pAF z@LbqMqaC*TBCa)~@T+@dOTU_|?!LVZL03O=X`S*sg`DuJfn_MrA10PA%suBf9()I- z8uOgoR!&e`Bb{(KGZK*q4=#Nz?`D?X7B=`?GtnsRYS@JV7JM!G*v_#%r(3t5c4TTZ zofmwlbvyQ9vD;Mtlpit-FR@rBpztS>FznVVhX@^-1b&F7dQ1tJ=JTKJ~(6RS5r z`vV%>wrtWx>COw;dI^3X(vA*uE@O=sP~dT&sSf@sY}DHl!lq=D zL~GG-KSB<|G>tE-^(GqGVks@IWEh+~5D99?vi{8o^gP3zT8XS_uZal7yQOO(P^fof zDUUv4v4^Yh>p-lGjuC(f0&Y!RegIq*4Y$vJcOE3g~lbrf}L;G#323hmRCR9293l!0jVo&^G!V^ z{Z>kGGt_kwUiFN6+cW?Kx_qzm-SB%dh^04VY-Vh76Io~qze?ARQ=~Wo{#W_5Xs~P1 zz1ll^zqz&IdBy3}nOb15tBh&cO*bq064mTXiA_c0^`ce9^&2_O33c*F4=<41CDo%K z|Hl2W)}V=AqoB4hDYcD$Wn`=AbhUdQTScjve_x7`P{w_xHsQjGhmln8hxjjsv5d%& zk1dfWPItX}7)*i+N8nE|G!qvdw^W4Xw6!y^G94ISx@h#5 zL9*C+Ufj*$*Q?8m_L14c@0~KkO)Z3I``x%SN8N;!Blgxmy#wLejoYo=`=QvKl)UXT z1yIt{y`7K4hHhUhqFJ?62W(CRx6ooAtBECrzq zhnWBtx|QrTI*y`%@g_h>NxRzvUHYoO==pGK97phuDgOhPE??8V1|a3%FuzIkc$4BE ze*Z*b;M>KHBmy1vGPk`K&i<*kzCR;2rGYuSf+#*)C4=J@)JK1WK(}_6j%k ze#CgWW0ok{T`rjONw0KChn?9h&g&5;O;X%9(lS5)kI5aGfmC}l4WBjuR;@XC zzh}6Zwr){x?_+y+7MlQuapnW5P@SUHni?P}O6+pNEBgEB+ZdOF^Pbs7b|y|;)Jt6s zrq7;>xE-j=zKUd#J6**!nkmHgJX`3L=4gkf!&50lD@iz2*VwjpRRswy9%%^;z66}U zE(~|WWbN)~>5?`ldTLp)`YBaNs0Z8GMt+iWJ0r=LMmTnYY&*Uzd~ydhFEGZsv}MlaVg_@hYl>WV1YzF-ECzZPBW zwMh$QhkVxzrk?xx0vzym%wiz%OE%58A0jainXba%!+ik-%@e@iL& zp&x^q3WOXNtUcBqizSQAIqj$LXurTX$;*fkG}3*O54nu3t~M&M=NxquEhM_0bkC^& zsW-)5L;tnU`%!G&c4E})UxL$6zS@6xNlO`Hj+*~Mqc$bevHv*DQO_KF>D|8H_#fG|3#ZD;O-ZDMbGBK&NH`w*?LQ>6)a&+vkvnGX6uX1 z<^IZZsWn^YIX-BccjUjK*DH;)_1JsE8#^-a;%fxJa?jOuoQ}>_?|?9Pa(R2L<&C?0WE8jl1k8oU6`S zDaO`W>-def|2i*crRzRwgJ*;xUWoiLleAL9tfbATbItrclE<@8D%N3 zofkKK_=BZ&^$nlltdiS$PLA!Cc_Np!$E@?Ejk?s<)Xt*Cn~Qv-a*~eM6S4iW$40_J z1J~$E#=>*;@k%TK#OattS=d%tmB+STY@N|urkL7m; zh)uha990|FlsD;Q-PG?(F z0UTJ@;Hi?+P{JOlDH@oRi*tu`CJJX}EaM;*a_EcsGpApiNTl+PPftamz} z_F$^@m@s2Yyn<5cc4%pL&>iABUZ?J-4wtT9^x@E~H$+ReK_nN6v3 zhRZmwgU#Z8z`K%Wg1fp2Gy!C&-|bVV_6PK-PGJR)6U$6IFIz5>N|-P0!}#j79loe zBSQRnI$1mgbQ&P~LgO3xV7_22bm3yMuD&fS<}EH_ zSh6(GUg8hI^~O{8qYit|Up#Xm<27X!jDy6I2KA*e3&Yr;kA?%jLoN z(+CB#i5e%vyR(r$L^4)Kcj)V9?osvHj;_ATZ1YNtXKyp!c_R@)o36(qr^~(LNUsi; z$(bH|I+ePS%26k0(HOkpyGMkp{WI%9-gb_De6fIQLeECCA^+P3>{ft9S8^{-c8gRG zanuL#r<*N58hULTVv!Vvo=!^QAPVZhTk>DCviI!`2GL3d5MA@sBCt$E20D3Bc#z(@ z_dWP7Rc82ZO(MB(#@9=2=p@<~*WA`aKia>WNcCyW*HJUI8`25knKUa{*$9N<0!u>g z0Ms-5?dOU=B7}mPf-O*SQ-W~r`~~Arh4mlbLzzd}S(U%+FFnU5y9!U^71paw)bG7D z>^DkMla71#%j1_#qctn!GNEUL#7`9`tPk>$%}M8 zpHrU~6g3y?+fiJeW@Z^LaJ?o#TU`wDhN(R3v466ITniL<3q!g6pA_~(XA#vSUz+rZ zZ&i8w;UAbh*7LVrru07D%?1!p>Nq>rSrILR>FI?EzjMjH6SlCDvYL{``dd~jBgO){ zr8BQ0A{KAC-On8sikAD?9xjnR8{&kn&7?pPKg_(!AIW?}yHMw0fV}G*!^ZudhSq|w z(JICHdrxZ;&>w$y1>kJKQU|tRZ}o>N_Q*HkflH%u;j9darB0#siNSRJijy3Sov#N< z?kW*6({OFqu#|kpVN_ew>vXlyU7})tW1RkT_3=xAWOb}lVH4J3))sOkJR%jbt8_MN z!L#vA3x@tnAa%yyT9G5Kw}&RJuFq`q)8axDMFlgbWPc=I!rbK9`Imfwq`Yc{eS#er zVOQ4%f3*&EJX{=K(yI$ItrI_`l@qW5ay+*9Ug$M%-!yHbak}(iM)TaryUmWL9-~e& zv_WbfCfYYucXGI?>{yzgzLI!u=rS;3yVx#<;6j66eKVL^D-BW$jONFqTX`om(r@;X ztCKZ1E;g&za8;4{R%Vn=IkdP(_-3A`yQTDT7bn>3?zU=6MXEphBu%H#_A9|9+h+vg ziv#Npi)Bq}X}mbPz&57kA62$=Kh36r(SsiyXFa5Jp({@mN%>Eq&ko7ir8%4ot) zY`CH-GpN$|`+Rpm8TRLUWgvGyYkSMdaHGcwGKhqGr@-al7sG=MDy;NyUJ3X+?^U2^ zaBZ1iI<2$Lqgf88>dzJvAb;Q%-U=l}6u7gExN2ofGS;>7i_sY^<8n0pFlhWinmW3Hcg{SK4eRjmlF7xUIiFrme zip&w!y)T>!XiUHLC4I4nHd4-+(szgrX^2IK@9$eXIOr=|vh;JazB9c(O{E%p9Kgt5 zn3F{5Hnk!#i7>~Cko_+Rn@t+ePaE09ksTLRU(>N%PjAjaWp}I@Y+fHwEJJfrXmuB2;`^Jrib(%>G4W&32ggCrUm~(^zLSm=$&f;FLS!>uR{sn+8x)(AEE%8Ee z=HI9GiZVUxxAPMTaBOO8sswH=F}IbI;9ld>YaR$pqb{!R`Y3tI_ufqwi_+<+3|9mG z==l!Z#ql7yVu9i-syyc=mw58^%%T_j8j_2qyT)F@u3b%A)0<-+;}!m4!&no9ng8LX zicy_hBF1jR@NTuha;XxOL;|73UA8Hzp08Td_h@(D+&EV6fUL(bLFs5vCKbQy<~cYx z2=dtaBHPG6E$iNMnxSfa>9&tDR`^@$Pw5K~@9z#dCteXCSBI3)2*cXS;S z%c_{=(?)%rp2Kg$gt=UYR~A7Q%Hm?1&Tuq{vX1M_3}@`V_wPl_Qgo@#mb`oq@|hUP zRt)%}*H`_Mu{Xz>q(idheIn>h6kz{j{^tNg;5xdoAe#vMBRb|J5} zscnu3SH7^cyPz_FS8;8fg+(Lz_42Gkg7e-63K@5@*En}jS8A}yb=fWKsahyx;pxeK zIpJxzNS_(19Q}$Jy?^7I*-V*Q0*vQK9`=iE`oW4j&uWG4jFDr@+dPyhkiF$OC+Yx%i0p6gS>LpmRLVU5>;32OWno8&f*#En^KKJj zL-HJl+ZBg&)+R`|x~6f*&WE_+UJd<&$=dM9%Rvb`6mp4Ev2{tM?+vSG9t^L>YSjWv zY$WC_xr|rtieEh8WV^7a&DxXXg{Qm9*TD85v49kmoa}sm@fOQ@weh>Y#QveaTHh}v zc#NH&FhouDb4@RTqIIpAYL6FKyX?*}+obXQSr_yv|AxCpMG_J~Q>h@Rb?J)rtNm!= z?;m}8hjR7+OoVJx@9QvCN6B%DuOEPA;4x05-~P9iA2RW=zl*0K@7;UTRDPC_(7Bcg z0F~zkPJowr82CJc5q2RC2z(sa&x0iBv@535kkgs;Q~@MTzcmQqN?N=s!L-;e(_9`$C|< z&yrxY<6x3whSAMHaU>WnQHwhPE+v_;b3TFKU>+CAp&PirfO1wO`ESSYXh~244*?P( z4);8bw>rUm;lk%MKlq9DAu!-PKFBUq4022gAu3vY0eE_PdPL2=d2i~7Yg|VYMapAv z%A9!a#J+gngyG_Y*pJXua7YMYx|osn(Ss;(0*-PNVD)*76Ch0J9Q#-#|1dhfDctXm zGkbE*2Wz?omywZ4{!HU>{A|p~~=Z!AS$~9-jl1?Tdy%{%|3-?$CThBHZBZC(7~m zXZ`;l>;5@b6NCA6KPG|zPWK|zDBMG&^AB(4>$#KtCBqW@bPFvC)F1u%711?73)co7 zvcNH(sJNFNKca&*?ZE4y;OlSV+5dEKP)A#^|EFurAoj}iX@W(|eScE>>plACf1Atx zti-{^Pmd*O9G~HUAl?`DpooZ3?q^S)peAx*9iPMjU~p_9`w#!)dv4#y1n9rk9A7q6 z%JF9)L3rF>ZG0psruye4dr*QCtO@q=J@#sGUzDML%bS*OL8^kL`YXw1Y4xrNqN6!g zOSsn^8~Naz{)yc{;n3khBBQ{pFqbHhYx{rsc%hv2-!1Ft{oa>>{z{&qx>F32qcGuq zP!-{SJ$ewz`P-IRmVlURhMRA+{w%!x8>4*ee$JX`WT)_Z#!bluQ(Hg&@Lr?fN!<7VRxgh{Z@Abj6i^0 z?xh=zvvG+ ze4$1D-CXzgSMoxBcSO{`7QtRue)7Yh+t(g9Q)oDR$u*qYLctAFP({uTtdXJ#9 zI9wjpqa3>#R5U%2g0jCN0&a{MA573=8)Kd<uH)QQgnBmbZEJ+l0+bhC6hlj%VW z%%dEGj*f1fC#=$(R+xbZylWQj{AeFQr_pmK`0Kv>UDxlE+5SG`3xhNJVyx73aAxc0 zEU_pT_Uo41-OqdEEZat~023^CxaNtdJNv&9jwl{Xgh0$b?_+(oZ#e-AjZHJ>gZ!iQ z>VgPCP@U|J1C!j}ir@SVO>4gj_P9`hHtP@RfkZE0H!T&__3TqpvZ{N$($Ko{mn;7_Q8}Ab&Col>Oz99B(vjGN=@dRI&+jg3c&r%wi zyk})DIRdefj?DUcMq;%oBIt~F<1sSEc#iTnO8ond6G1hjU&bGTy51*>?%>WEQP|et z?5{J}<#&RfGSj*vVQA=h)Y|0&1-OT@8z18Ziq3y0XK?t^kjiJ~i+4c1QkAfu_sGad zAvN8QKt_9=6Sf7nctpV*MQe8f-xpq3S3MtW{j{H@`UnePy`dy4n>Ac>uw(dGu+8Hf zdN7wiQqMp^e{Or@<7K1KBMqPo?kP3t_MY<|#vK{eT_x4{}zUZD4t?Zf1{q>>dikN#Ra; zZh*HEQobMg;wEVD&%khrFRht1#byQro_g!P zj}ykelK3qj+%=G=#0B?uUrmIC5iJ?D9x%lI@5R$mzCyfe1)18$=QC)a;<+wUI*ivm z6PmD{#P5z#j8XOGw0F;drv?bFJj!*G92aq=r>CbXs>M7y&)@r+tm*#wP}v|~pe^r3 z8ygyWqx*I3QDrpyx9!hrUqWEOejYr7D2IX_q+z26WQURC&c1#*FI@lwkS8Ze@S_(f zH?Aci;N+aii3Df`kxs{`6B^(e2Qu*3Y7VyG4@-e}J<$TxX<2q^dEd0c$A&+RdAa+H zR;b>eZq-2#gYuGZ5Lo|?{D;$i5+Z-iK{dvz{7+E|X$Z_Shh)eo_ zm&#CRk%(pZYOkaX>Q|}8?@%KdpxI0~qk}n1Qc_Z)tmyv8nngzBga5+5&?s( z{dew09hh+*O!&>c@M#ZP&~qY0en6V2*a?RKXq+o9#9boXk43N?4>$J<8Ywj-uzznn z^y#D$I{eZ-(0%H$r1M?r!$bpi0%=)U!YvvQqaMS*G8nQJX^Oah-2-G?`nnQF&2=_C zPkGmbV@(D~9$EeXUjP64j3^%0v8UrjMA~&)&mTW zL{YR?3>KOQK5%_j5XA##tBp)_82X`%a>UJ;H-I%Q!OAOTt9~UAL=g2UI+UgRT9IbrW5dhQ{Ol@J1ffCN)KGG#g(}lo*Pxcf7o^(T*e#ijy30 z=))v^*!ltr^J0Xe^#0GLOL;0coyoUU3)Z&%c!#$Ftk;%~mR5|%UOcde==a-SV0Sin zvELoLk~`&Tymq=#%IL?3&_! z{MC9m!logfLsdesu_+LqJxkBe&u<^!%yl@aVNv&shs=b35whp$6+|$;eQxdH%hX=Axll;GDLDC}Q&39!w2U#vw>%d%e7%3EN>!7 zzEUeJOg@5*lDNRhk;z^T9}Ce#qwtn^TbUoj(9t%?t0$scje`)u<6GN?-;Trsk`kzc@RDW90kpD4MJTL)h<-2bm zfPTERG|H_E`a2}~KKLqv1=`~sr<8mR_kzEk=-&;^eTkK?hp+_vL1!6}1Lk=P!%Z^?NmypTK)?xrwD zyx&6QfC#+lV3q&Xa5xy~&)Dv7ArD}syKeZ9k`Qz7_J0ey?Jt}Y;oF^88FzVd*8EAy>QM8YcJ2Y|B9o!?ukbHVw1etxY>C;`;C`yW`aH; zm7Jej*d-S*3I^=Jkv&1Af74V)bU|tcOxSNf1sa|A4LgHtNRp_jWb36DMwc!*F!;Fp zq6&Vd+hmRPl-A~vctxlP+dNKhhurx8#W<`Wn4Sc|ez+Y9aB|XQ-@29N9O@Px*Gk}l z@?+aSmKKE3js8seV_P0>{ohebBqAT~)Yj(N4SP=;bQ<>NfC}+VfJB6|$vPd-{u5E8 zrx?$5aXf`Fo+nRUefnmNGux@TEKTN8VksU5o0GzEQJCa55nkS?pCDV+ZbKn-84V2$ z6_WTX)SN8m-trfD@E3$0{}JS;?-FyO90(?CREuZih-BYNu)?wg8-hYUCd2lX5UXkF z$xhj2X%iK>E1Uyn2tmD5n>aCeF2dZzf^?2J!o*!8b3Z!9bUmdoC>I#T`H#tl7eq-M z|BZAJn~={{y5>g!Jl1@)@$()S1kDnI{Aj-Ra0#1V1?<<>P*KIJ=gH9F7{j8733dI* z19^iSjE_BTv#mWMdC)WwQO@>1ND}c6$ZOZGAtj}nrIDnjbTMe8 zP9~_!d@r(tczHUO2`S*4f)g|k(|$NUmJsxyQM6?e1Cg7v&E*`pG%YA0AhAwB5h+5l zy!3g2=IO!fzU8B|{e$EAXLbjl=FeGbHG9wDFoQ-)AF#&X&w0@sf{cyPWyy8s>7J zQ~UT}-ng*4WN=7hyUUYcd@FQk*X50rz*<4g=bd_-LHBj7DYzCwBS zQnkL9PMaL-xg#tSzy_Tju|cBy_ne#)DdT zC^$h5w0_(^ab(~lOun77pamUQA%Teo(?Ky$G_HK?^~9Z^2x|k0-K;x|JwE2{O5G*( zCXoWw^5?DU-|+DPd0oN0ez1W^PE;FFN6w$A3gG3%?!YRxqiGo+5Q6t(v+eUdaK3jB zB@*H?&}}{qYw;TUqlrhmx0%C0H3{t1XTT%_2C%dyG+fGu3t`dZ$2b?`3ikCs!AVFS z6R9Ill`hYt1`*|7FbC$pK6bNf6e>?MySz>t+tHK9GZc_17@Ps>1Rppyse3Yc1^m9% z#x@BORC~G4UNOLAg#u9o7y`6puLeFcX8H_O5vmB;>xj?^_HC6N;xjkJWkl>nCa zGysm8g$yi8)afEWI-d9RGCP|5?I{qdiBFp(FTz+)yjso;Y`DYc@zG(0Csn(Fc#>h; zt4mxrUir9ty8Axou^dh;buj{<6no6M8z(g&=q!^Ij418oIfx=4Z=><(e=F@Ok_gp& z`XFKpR2+`LJs)@&LFg2F;^j8TkNU!Z{0})dtk-SZnZ~e{)oX(Yylp*7CMFh@5{18H z(0<$vFW)|wTz{jy|wx^MxXBMJ}i&+_M3Hr=xb&T?6kxLT;Le#f|^qzK6HDB z;uv8J%ec#)*sR|Q^O}byHnH?m*Bw7GJ8GW_+9y3W=T5;Do31mZy)O#Dzd|Vc@F;r% zY#-ATlsp>u@=JjK14Jzkj95e~s*M_oQ2mf>)(o|NsU+n6a$Aqm*ad;vl~F8lp^(La`|*4VNm=xQASu z@Ldx+F69paH`fm_X*x#|sXP#+Vt-&X;31_XS@j&?cgP^LX&9~N)~7R8fjogoA`H%N zg4kDPE*tD=*I3`VdGf3Q-X8{E-rq(r%>t~j`{2Gi7ZZKFsuq;A;;X2GLe(n`jZ$b} z9@FI7gGBNnfumOMHo?j8>y80nA6i!d*7c*v)MzZeD9BQvU@g4>hbfz*M(Vg`Y;BG+tD#ko73$r&rtOFvtoDE2>f~4vbLYG3G4!r0rhha+5^wPBoERF4o{ICSfbw~EEsNxB0j)#fc*JI zB(12JYm?ZV6G5*8l40?y0XVwkSsq+mtBeQ!3-TM(3G@?=uW+9t23$Sjj9)`xya8`& z{?1(8USX>gfKK&mN}sH#MIw?Rli+n8t+(e32&r}O$e z2054yzWuZxvvQNr*Rw({{~EP1f8M;4wp4-A*Gg*MZJ7Fc(*53r8sad>#2 z)kk7HxdCCJlWlwl1mubeOmJ4U5cTB;3kXjXxC4pwhnbs(d(JgZa_uF*xc(X(Y=TEb2gcWMCF&)U?(?FLS@P6tE z!n7wx=f|)-f|Gj~g5u&>fyquNw6s@e?PkYonKC)vQlQ!+Y-N~X`}$x)$YCr_(B%)> zgB~`3j~4ao-St9tp%H>Kqsr!F3GDr33WQequ1MO9Z%MXU2-a&9|Nr7;f^NkoN!9be zzT%z|n!0UhU!$C47|NiMW!gqLHC_RXnJJ(B9t;r53+d1Oe?CBQ%U{qAe#R#iwEvK& zflDJJE4w%fx}>qfe%dCMCzPBX?k*^+OHQTG<}KLS!t-ftxD{#uB<8WrMS7!-Lv;LS z;Oamqb1JoPmy1t`^{F8x+M59K5p{F+p3nP$Jd7xb`S{P7;((w1bZN*O@Pem1x{slu zApmiogPT=}oz4Mh5q}82wR%3xyre%g_H=4>IA1C4c|}c9TN`oH*;sL?qWH?TR0{r=EtkHqyETJ)akN@*Fb-56!!avNA7I~vg#DH z2z&xL|I46fbFO(mTWFc~MSytkzCr!8ompmAonxU|3-xu)RV%shKEqz&nebA0@!kS_ z!{L`B)Ovz-CsX~Cj5(E{a_;@^2h5CT3*b@#-UO&N_uA4%N-&pb?UFQTv)ux^M2xfE z%pV}w9tezPx)bw-P6&d`D?^YQt~*xovwa)EI~EKDitZqz0ly9I{H$WRyJ`4T3u>5)0dG|n+^WsXvhEay+8_dS^_>1;7tv#cc7!(!Bd%x3oUM6~7feMf zHQEf1JVtp)5Y;6WuZM6AYYYCf2L>%d5T2;0bCIKjUy7Kc8(|Tk5I(1(Hs*lnuh%dF zYjKtVYuUIvJC|i-_DWeBbP$2E9BHZiB0&Y=$Y!@l*k!d+c03UQomY+Ozh=7o^irQ2 zrGWC^)F#}9k>CrF)l*Hj;ZO!>6sS=%%Dy3f#cX@v#0u5Mv-fB8(_41c)AJ2vU;fUQwGdzfwClR;N{JrVo|d^lqB7 zy?Jv<^m829pmz{NOm!neEMd9BGtpR}^P+E~P|>q*73UU4PsOi~w>{6^z=g#H1xMF3DUaLy)b1f$iGFkVKQu+rGGc%=3C813oUK$`yM*cTVMH+ z{}u}XhQk2-AGP_K%W~x^> z&NXpr3W;XbIFBWB;Isu|wNIYNR@IygJ!Y}phe@7{G4IO;s3-Ta({e|$0zF2EJ;!BTf6z({ z1ujb5T(J`RYIkrZwP~e+J}n7PsbV1%bnj0V5z?qDut_(FDCXz3ZoFub>k?b2!Xum` zGHdd|*AC^WH02db>5hzk29JB0_+r00?!fnEdRRbrIB7+JIWkk%i8|;)i>i#_nVY%t zjRM2N@6YX<4q!$gCXoLZI}z;he}-{I5jk&}sL=7dXfzKG5K4;9P=D3jkSechXgX9m ztHA1zR{x9!Pq66-3jjUvPA6=)I;8{AQ(~$xOe75zGHF&^VYhI5_3+GyFS-*^ZkgL| zFLR{MRzyz0qOHT!PGWJx%(_Kp@#Rse38%FxtED6&sIlteOizDF=bcM0Ja}m(8^X7; zOLY?{g>MM&Sfx+*StFQ;<^q%==Ttg}(;xJXywaJvYBfjoz^L7~7wXUqwNP2I;Q-yd z!2%a)kjI)1A5zyp;;Ub;Lx+WBbePhBkVRuJd99m4ySEeAkPO3F+qHNzu81>AafbRT z>mA1WHR6^0tQR+H>G37z3bulqO4dwvjHxHb8_a0H)fWlQ0k&=rBB5rqBZVeqx2}zo zOw9d=wJv{*?g|QrW$y0fYrb;HZY7<2@m0$2l9Miv$=&ZU4U&&4?aIi;ayF(?9(3}J zY^WL60HH(vnI^Mr>E-Rr8M1JB4ymk|?WsAcMbH>Tve&`4qYl(wVG&(_;X!u8_+8MN zh%N#sLPpT?Bia#5$!cn~cd^>K&A&V*vM-MIyn-gv%c>nZ8L_l#G__)e$q@^~P3nm) z{jyS<5lOQ3@-!6?XQMZuXLzNmo*#pZ%ijhI#Ew*8N5y=$aL>=Au*^ z6D=ih?}N-ba1E^R2mW0sOgsfB4T6naPNsunp%F{EwxI%=GgnZaUMd4EEkrRv&F!#h z9jh(L?qVd=;+L)Hl^m_bPrbn(1pT$3^`Rq&2-Lt;FC&F7(`Vg&?ELN0xPogO6jdXU zam@73NqAddIBSG{{Em^(l7x%vuCdT9viO-1+Yr=t%DrXL4Yg)>R%TbAWGk5^<$%gQ zulCOe6!U^cX=3d<+@O}DPlnkbm#5znbjz|QdYt(szXNHoZe27mPBh4PVN0< z2lkfmXO_uSfu}t?<710eh41-ulh3I23#wMWN)hO2o0|M|d&o_=lbd@@E>teWe@*n1 zrQkW9YVlJNptEcbQ!7eH_gp;PsLq2zTVW|4(wf&nE6{Z}P$RwqaAgB@MT!+DrI8~T z%YQXp%dT`u=(u3rw}t~Y5Y+IKh!H#*w$A7)zhTe13kn3qFuo|4@R=q z>{+(}sj#xl9K+YUg_~P_x0;k!|{`v)QwXDy=*sy(JQ-n0dNBTU(5s zF|MU(vJ2nq?GzOzcZiR(t@KHdTdXWfx9WXx{yDmkJ@x}1C+aP2)y!T4mQQ!xK-a7| zw~e=jo?a{0R24nBW?i;@--VIxm?EAxHZfgUU7P^ju1YW`vOr6ZJM=WCW@8Pf;UQgX zE(N#Cy#{~Wmgq04dgj7bc%vGs{>CmUzOY2HH&N0Q*9AKIjqi}h4KZ8%m?k1y9Un;+ z_+ek$ZZ;bCvM9&CcDZ<=KR&sWN62;2K&YmFZ<~?a&;WGeER^##P9tH<&`W#nyd~on zy{yQw!8Rid-Hc=Pg4r^~u%`Jm^F%&qUi3#{LZ2{B`int~9xgyvcCh@&F%9S}0A*dk zSXPByJ%pt_NC9?2C)scRY9GJ?7q)C9+5xp!!bg=mW3n*Ou2;`3e@0pvCK)pq;-r*wS7%0Dv>1PtP(;LSe3)Y)@GYM zB+O{G_Yx80_)DNFaCqnvAE>7p(|V@5u-@8NTv)HL^|jh;paFDB9S%3n2nm>(l*|0~ zDj9aRvf^E&R6;B}XlM68_2O2OgTK<{$zHw>?|WRW<%S;TI<%E9MCkDNVOQynrep*s z?1;>xt5LG##??l0&=;QdNfYUnk?`cm2|@Ry16A0N#rrHU?xn=y)ZAVz3AZY#sf&!> zavG~fsc0ydqjn5b$sD{|998y6(8$qsup&AO7@i8`Rggvbl`oSUF3Dq=0&HY|E0>KO zy#Dg2H{sc33B2n@bETeE-6f&hjkZ7eSUxe&Jv0a(N|E^*T3pn1^Ig!6?nGyC9u4LR zhB{$}{Z{{f32qH5l-pGp_07H{hOBEO9R?>l>NUjmktfpJ)hm$HxJgdgD=BzQ zCClxt@?bb_s)_UPcGSzI{Oyp<#h$P6!&T{a)%Af}HNJs*7j~RJ1>N;mB;Io78L!}0 zTiWoyt>!!X=r;ROimh7UM+VPIf<|6DBJ&7$?pYIc%@2 zodGN&)#sS-4eB?jcVjN}yNC!P(Qks@b1OH*0lU+QonnkiV2yl?w$Ro)m{22USzW(+ zt0VJViSv3h!3O_Z|Fn>@mUkn~@v>)r^x5GGMjK+?EHo~Enk-5q@tm^6b}2$Wn#Xpc zMZRN-nelw$%DtUzyE#$0&qH5JBc6ihR-m=|24&ss1wFU5SV}?rE~nlECY?7?c}&)8 zt`;$<#~c^fxulal002iN12{O8L7A%+gOWk)MCcy%F&#etTV_ul(HIFt3~JBSj+`7G zC^ZpeC=OXp2}YB^&Ck6y>{}UZdeCPoyf?`z7v!bOq?VhKU|5&i3n=bUbM-B%CPM3Z zALW{5sHwZ*Le|UAXqwxb);g)Prs2HIQ2GHBP{0M`q=l(w_yoHJ+v z1>2GiSM=V1t0f9Uz0u9L7klv?*vu3=wZBV);>X0P4?F`BrxfbXS(IcHKq3${cijxM z)$-R75=1N9sy=P89EP5|S?KAV7LZVuu<|y6)oWJOfW7%)eN>lPMsbb@=r`GpVQKnv zoK6NbAMe-+-&sLu?A>~+!antV(|FIr6T0sOvgA9kHTYfnfmbY}TD#GdNRg|>Uc6pu z7qJ}`I7%k)Rz51LtE(l&HnLN~4sYS+J*UqT2`OvEpudKN=2-xKyk;C42kr9(_o_!0 z$9p459b9VadpP#Yf27J((X!8~f0OOF>F+1#RWT?Edd8icp5ZolWH@p)_LTcJnm%e0Sc~?*g{`?w)@U}8Z{Iny*i5Btvz;L8W_W*vZY1yfUagH* zkZ<1>qy?LX^rRO-mNeqk3%gIDH_TL*`{rcs_m)Tm~H) z2nwu|2N}q$0BefiI*GY~1Ui<1pFYN!R@O5_(V#!>lN$(-G!)khVWg&#N2kyE?R)3|B%$Yree%CaU?z+lCYgxwg^JuC($=`zNf z7l%q_l2y}u9jZv?p5E~>?SCg2LFh2FkzwV$q0(2(z@z@FWvr|tWu!UDH<9GsQbe>T zGH1S$S|LT_Whn22^*h}n3E^Eqr?nu^ZEq!gh`HJQoI>oWYdjXhx^5Y7mu2r?^ly4I z%hs1mD^O1BvXIzk)%#GwPIrO0i|{LTiT0*;e~+po&s%HZm|EBOz+e*GvSg&y|90c& z#TU7Z;F9*;F@^54qsVB0UQ9Tdgi!dvV4hR4WwKku(YW$8-cOF1$#t}NybZc0UzH3)bOw7h5`sEqi9EL z0TtGs?(!Hkml-Y*24Vm-W&j=oq0%p^ytp|T%|?5e*z;HZ)4|I8;KP~7Qu(+kPCKt#;agVoZ2>3V7wf&@zK1F%H%z2FdYFq=a z$Rolb;Q6_%M7$+Zq0X_Cj0HNQjm$m!Ho9v}NVOB5J!E_xp?*M}Q={=n)BP?lda5J& zWba72fx-s;*Za?+frRjsa+{ug%MA)l-xR*!RD~V?(E+yenxs(XT4)R~$@J>gZ%^YJ z7-s20_geCg)!)-#o=OC*`D9m?wc2m>RyBiP_mF}>_&B8>i0zM}cva~|UC4IRX1llD_r#`P;CUyjfHn@9 zv#(f*aamg)YclfllO16Jj8q(;ElG;ZTnp&3)SA&PO=j6oSCHF(x^+G| zQhci!g+_@Um->xU%#t}*!qq~p+EkMGUp0V-gOI z(_7XfK6;C?Y6sHlkVusgGYj%4Sadnn9g{7oX-8X7m`A{AFuY;}%FPWyBXs|G+EUpG zT0ERW>8o~EZZuvgd2aux=e6?tVpZONWr->O>@oTmGHjrEVAJmHPdMaPC9bYGYsKho zW7mPF(w@qHIF8%(64Huk20sR>rljTqCaWJ1pagn&+{CQ^LXqK;?t7OOirVf#V~h1tQ^6Vop{i%1p z4frnm<@Fa}Tlet119T}2`%ig@eS3!|F5qPBYe`6>!g~6W5gIoil(f0@o;~f4hU!9O zMs8W=6Cr6<(cx+Ekbgp+5So+*+5gM-JII8p!9>ZfMAvkJnL^A`8WdQazUn&b)6ug% z()mE(m<;diGL=M`2_d&xB;)2-m&WZZtvjJ(*B!sjI%71ICaP;3i$ft|#_5(Ys!@?I zI)mYwhC_8SUbL+;RHNBf<(uDfSe3C_Yw6Bj={5F)iIXj(JrU3YNSUVfalN!B4shuDTZc#~;8BO2%xO}e|&@@&U)5I}>1ISr_ zxOF&{z-hpXvz#WM@`i7HQ*E*J+vx13$_p6-Ka<#*%De8xF${GP(RN(}?0$6(VlLzT zS;L!}xV(Vj^|mCVebTHObbtde26yqAxXP=ORu>>LeaDRw>F6Jfr0u!e=2&fgPAD3$ zuZZfq?s&GVs5_OEiWJs$1^Z(_VyObbFyk3?4v&Wl)BnA+sz@omjJ+8udqC|qpfkhH zEn3OqhThYJLQyk=*F{-~3&Vl<{xNo|Kd#Kh>OqyZlJ_pBDi$|8J=sa+zA~S3I~j8c zT0jHi_M=O64~l_OaorsJa{swQ)C$-MFO)q4{$HYL=2Qqh6kHr{e%RRoS5bh%Rg*kx zPG}la>8-nRy1Y3O!|DPG)gyWj9#G8nbIMS+l`5sWJNb@ypl&1@&xl5Cz?{u;ks#Yo z20VuaS5+E1%Z9Y(!}6C@?`#${uMU2Wo{m5-Q(L;U`Y^86V|0zzltzt5Ec%~L@!U$11mFZ4FqJ!=Xy$9$GAcSGT(ZbnF26lf#ezI6Z6LYoVvwmvpKWIYZv zNzfeO2jL`p7qy8!q>P!-M~hSb8$ao^YLL+Z>Z&fXR%<$rwv zEn?B>O>ki@;A-5JSeLRs2U#y2apT*lH>YnZZTBl`H86j)dyYOUu{k1!+>*%A%}5lF zST9_kAsLmGkXRQvM3j)nu~5vj+oNVdroFHVqFu&q-+2@;jtdmET}2tb>*orDe2FPI;k)23350-P&6d; z1%xNTpmgKAyxLiM!_wUK8RrGb>yV}rX*lsA@?-?XEr(@JT@GG<*C?w!k!cDxUGg6b=s{gd59I*D%E5kG29 zk6K__;w^R&M9Mqzb{T{NbqYS*4+59XA*D9K`+^}qKeLoOyB6(t241n zQf9R}97L}Hq4JYO7L5G1*D7XB%(%X`ad&4oNYt)M82|S8Img>c_Thx=U zD{SD|A<8APlcfVhw^l#O;^5*Qk<$KttpB)MCS5?hsHDD+9Ok4Ke4C@5j#%NM|4 znM)yXP4=emA9V0T8qJLbIW+Cl`=_kgPaGe`PkuZ-nGDJi)5-JGDNa9rcKSd671|uF zFvx2ELCbL9$PZXnpiI!F1f&!^9$t}0m>cL$)gXifXw@mutK5E;vN-N>5qnKzzVjRF zYWH&ot&d3_^gS8Q0@eR<_3I8Z^4~qre?I{762PMLdI201fXj%%-r)nt;0s3Vu7)G7 z0dVPE1!*hLYE&iagsvNUN`HU>$OAVZRGeP|-QSGqYKr8SlbN&@SJ_E;O{yujToagPo8tN~!LH}0Ow>zGQ;Mbxg|Ac#9i7M=h@f)3b zVdJ4IbI#qGCGZI+#mMqTmn=X;v zvH`O^0#IQ$j&u+zfS_`x&*k#mBfy^ieNz+={4jsVr)BUPC- z$})ivh!R>Es8csAn={F<8eRPLfxlPe(N*EY#X(f!DWYohbLCs2d?TTZ_rWiJI*)vE zbebcG0MM8XcMSFlQZNxPGXg{eV;oW4W7jCboleiWbAU`lFg+{+FjcXUn6NiN`c9AP zz9+fe{1?T@zj1ZaQzQdkvfzsH>Nvlgk96|!%Zx{}29{AGLEqw^1cmwD>43$6{4dD{{7~EIuxJd0J8MxyQN9U9{%V5R@?Y(r^9*plW+gMqi^Fb z6Vuy9oiqNoahWH0a7_M%lmeglue$u7kEW%BP*h)T9kELvefu8l;=h7ju$}(PISesq zAsVT`G+h~|kvjsTf+quB`!WEL{k;(n4(?x+TmN|d_h(0dhV=Ik;Slq~hXfB)O_8Af z|NAMRexwc=dp*{4>~KwSATuG@hMoC$GpOMn5?L~SXA1bw)s{C^#-N-X&<}2b{+_&p zpO_QzX%8Im-=_b!QwWj=-?J(dOQLvG3n01y*ua5e5I!v^&razZ)P5+*11i?=-B_mS zJ`fCp=C><7d~#F`^j7)omg{P!U(=X2KlB!r6fQ}g079G1_s6r}LY4mzIzEW-8FI*g z%|f$OE1#`5mOIZmIf6cLv00PuFR_MN#O`^1#1I|)tmD(kxJc;7)skN&%rXvjirFC>{@6x7n zh@eHW6Nc^A0z)c#UV*XP^Tam_-m1KO>G0EVxpPhU#K`c6-flp@W{Pw3f7++ZvFBu1 zm))!*k8Xqy!1}U>A9GS@eR>@FWT?DFXAd2|SdeDpU*i^MVI_J$Q+3ExbkvIB0(O&W zl_Txo+{rrjH~0s7X@w#eZD2sAx?Y07g|X;*{`haGQ74d<{J0x*WVvFJOsqxpB-Ooc z{_<%Ib@pbe#A`QDwhqabbOg1(6D1Y?#sNA-CU5RB`=?OS!2*u|GxR5`ihMeSb-Gx#1zJk0SE zGzH<%H2X_LHA21^5Jt-r<4+!ZG@$*9ubV0|2qR(;s`=B3^0|d`=rw`}5rRckF2nm) zao+47{&JAgSMg`h{MwtKD%q((^l6Ux*Z*5_->amxb&TDq9l;h@-v>IX^0JP-??h{a z&17>T7rZa+j<;9jE$1xdS$kAFigvm512mDC<5CMIm-GH5PCRQ7{kYHjcOm6ScxB+5 zKMpmR6N#JG=w?kz_ut*;VO0}a&_H4plwwxPY6hVFYDQ_0NhJFjnc7xn)El)JD+hFL z8^pf1w(f%7ZS)i-C&C4^0Cu-d40TR~LYi~J_V*R%%TH)U_HGn0xU?$Iyc_#$o(6Go z$BfS0fHafu3V#!U&(4|WJ(;dQi{{rK$=~!m!6ee~xfn?MwrFMPLz0@1u1yQ@FRkjB z*uEmio#K8}Ws271nSWtiT<3psb|R@s2jKgY_)x5x$H|YD;n^EXKKsk6i`I%(=1LmnpG3pKDPr9B6e7}(5aDU zO?K1%Xa%&)si`P;w30VtQp%m-h^gLUkH)!YlS8!~;R+Zf{czqVVsU1dZvZJ#rA?i& zP=FNmFWu7mxTA<0`e1(=5dVuH{aErjQsNA zls0`+*c(@&U9$;)f}jz1G@BtIjrDo&7X;lS`%@8#2=2#e)qE0Q=V#@(UJvKmA9iV2 zmLM!Qrwryc&`)@zkE9$-GJSfuifZGI@k7}xF)OgyS}f#3I4Ve=IDquyp%xjmP-;+q?>jbrF9?Ejl3tFT{5bm^KLen(&@2=s?Ox{ zA6DrAIs`bW>5~qx`5}C$RZ$65Uv%A0F zcLk`KRP#I8L^=*;qoc3mE`f zlu#>dl_124mr)5&rB8o)yDk9K_0|^m5#!i`XIrZ&;EEi4W7=BK0@h=O^bP(7Dg62z zM|@HFx;G4z7B1@jWHbX=0?Qp_Wj9TSJ6olaA9(lpibSQ2`^aJ4Oxo!9l!w4hYrF^l zvNV!4K?ZPf>g`u%dZ&s&XXWq3iI^bn6vh2^+K0q?a}7Xmt!NaKl^w8Dk${9n7J&ry zT}?Uo9z2b9>!jtv8xh)?i>=Oc&$dMDOh(hSJ}ydS^jo#Jzq_jxzR!)v^s`4N5$A&S zh0P#q$vNH*sjrFz>z3zf^leHvOW7r3i_}!YuRGF3?$$iEjwFbN`fU#!*Guexz~huR zFF_+9;&sZgSgD=0m4bs1#iw}2Ii&vfB}4rZxoq^a?6nF$8A-QIcKEkXoaOz3KY;T2 z`jivbgVO3WSM-=ycwR&Pvi{E&{joei;m)_cPUX5gYHhcY;1CgBwKsN7_{7h~dvz}E zTXQ_C7V~a-+bF^*NbS%je6;-KLPfveCqtB;ge^09EUq6tP6SCV38H1(AeBZPcSBct zvBOk`-B4fFP(N2MNT>wd10d$ftuGo5+@<4oYJo^iyV%J6&|4yH<9k|$`UP>_6?_i< zO)TWN2vw>5bMGj1p=RZ-d-&V8Y;<&VAt50wQ8aspaMX!+5Ib#P%2gD9id6Y)v;Sol)pYw7m-? zdaQq-EsC7F<7%s~dqYYaE_rgtntRBsf+6K5Q2DgpBEwuMH&?#3>s$g*6>bfIIyn|r zC3-;N1wOT`1oh*_R%yn6-rn1COcTzT11H;<>;4%XNcCyVH8}H}xvD$>L5<}BYMe+B z(=cJ{+fL6Y`64VwD~Dsk$S!cN%{Z98MKP5;U~HpLOH*yYpu=_JGHuckQDg)i47aS@|ZYi=Huc(^_w+f@D)_UM>Ip#&LG{k+4Xi$qkjh zs}=@Lji35-BLt+UAjrx>Ttrxv^(tG+L$CQn1l8Dc&>ucVw>K5mU?;CqwHGImt!|N7 z_Fh1=Y`g1bQ9l-4-bhKF-&mBS&^CX%+;OK!w9Jirq|_}CfNWcMKk}+bI7L!$-NU}U z3GkyuH8WX}W81@yi$9Lvg&=l!Hw&IvSWjoxz8TNt0nMG(OVcWBBm*9yCA-Cpa;pP$ zXK<~ka(U;Ac-Njz#zcZV#b>~)2=~V#VIt`mNEe9wJa>h$-rHLRo2E=E1}1a`8fLUO z_m9ISUEFU=Z&YDC_^q67_m+VBb23!9Qu{KCZ&E>E4G1e&qPIy7D80$A;%NVg@p6yX z4z3y<#@>Kj;hyxw2hZj5AFVCpjBRIKmYjajHQXIc1JxLo;?bUKP}hMY$?7sL)7t!2 z+wHDUTi5+=?a3$Zkz2YOCmF6=S+gqB#~D>VNgijAQUUrxX31^5@ot5!F=2uh`Hok# z%Aa@*;E9NbwTbzgMJ^{Nno3;RD_Cy+(UrWYE4@~Vg78y4Infn{O#px<9^q%-d#QoW z6rkw}^i)85jOm9-!1>30ADBBNN17H=>#`^xFEm{hda~1Tb-`DJNEQQ>?^uAMX{KVl zp+k5RfK2`fFRseX7FK4l!*`_#((+1Wah&oyfwOu1g)_L3jOnmubR4HS&HsiTVSw+q zC5}@1nppn=QL4@?gfb>;E#mzipnTfIAMo(O7F;T&l(8MSd-HdIudSv5)&-=9)#$g3 zhij84`WVCC*Dt}HT|j-4$qdLj0TtFPR2jPH3EdQ9GoLjL*OhYyMD)fjZ&%E-%(||2 zjS3%h2A4i|CTel3+Misj#v z(_+4fZ`sB`3Ep#Qu12u?8!iZ0bSi?t6GFs&wU}_yDzZyD zE9xo>8U*(l-KiP)&hyTPRz)vYC7w(-=8Cxd4mGyoA6B~2g_{Da^<0XVJTImMTDmEBHE)|+FCFG)Ke8) zx|h#HmcUZNJ^!f!bZ2tCPr}Dhy094obEnIJd>FSgs zJ&vH`h+QQ(>&$|@FXV{5ej-9mb&qYql5V4M`cYg`pAJyIapg07Nil19lDf;n`DyN; zIi36z@xcE3p*^y}U@<$nSAd{=k>n_~-Ox-Dtw%RMrBh&8GwwTDoAvWqqA|z zdg%;EEE2&v2IOw^5)rg1N~c3*+;rd>Za_axp*=~!yb>bYPgoLFz&jGH@m!~F{HOX^% zCr;FT&sJTdAVD+O%QD}^U+nn2YtMeSq3tudO{jD-fsy?D=aIEBei|5yIVx8IDjAm0*oeF4m(bD zj^0>av`2sy+1uOzM|af*vkuCz0yR{nj%iE9^@w*A8jA-m&J#4UG1NTwkx+14?fZs=VaZ7ZB zzV{k#0&RoW?Om?dz4$XX0Wm0bFwyng{b$|Q^`wEh6j#o|9^FG$4v0|^b6q5RdBu&{ z;Pg9lY~`6n)A*O)H9YJ9Z!xm+s@Oos=c=t28TuZR@6RIMQCwCQ=o!aI=#nvt4}=P| z1Lr_*ss|jv(*p<4@;*NeYTF&97k`8eVKU6gGORMRQp&%3MDC#8f6lP_dPq-B7<5xo zhc}JnbaMnHY3@o79F+vmd$ud2qVdK^Fr%6-!^2(L#Zx#h2BsX z?tM-XvoS-~H8Y{ZAT3b-+c}R4)bnKd0M?}LP5qgrjgrceWOotO%B|#hiiuCI%Mw>M zF*tUcmX?s13s9)0;s_CZXLvr&ofmX6Z|$y7D@F^cR>pPS7dLbPd4{ax&Q*<3ySo`3 zV`y$jaa2ztN;}(J4o_ZSrLHHI8hiFM=iBhLoY1@gnflwD8^kF$sY6Mk;vX-jC!)Jp zuk!>;3JXMl)JbtkJFL7{&?Ys{T>dl~pOS(TDWxXAs8fS{q=2mS5mW0(*+}w^9Fn9y zp#LD(L|E6K#|oHvu)AHF%4hCen$CAgXZ7?{Ml$AFmYvPZlz4aEh(R7Me}W0zW_z*V zB7pdgaHTOPL6jdKh%d8|*oU9tF8nhWe|}<-L6*MZqm-+qJ$%h3;#j^d!gJX@7=+=D z?BJReygTj%fq9QaR$fg1PotK0mbitDe1cn7@!icTe*R=}zH*YZ_3CcVDZgWZpp|#V zcu%i&^R$2Q-n%pe&HC+}n6yvk*dMkBxV}WOaI?w;@8zgOnTuN$9t|EY<;4ObX|F&q zC5peeAH)!w5A;D~BRt){nOvoLn>Vcb)NSdqw&U>uk5f*}CZ^V*rsCx31>YzULAO{R zh>;9jFx}_M-LJQIaf2?aYvQ;hm|djtv(R^bXDXZUS<6cRxB4^@3d1xK*(D0%36N3U zVMR*+RTuWTiL>zd%ZC_TFQ|G0#2WO$=KyBl;YN9Dy$h4_dc1~mYeD%yH0Z+#pvgt( zKb4lmaw8ATe;(5CvK%Lo*0|&DSP@VIUpj6 zzme?Q{mAfpD<$Y~y20er+R2_~Q8_f=aXCBbwZX6t`@Gx9T_G+A4ZZROfM zt_oe9l}+&6Wr8pIRz5! zdb`%9)k7;nEtkm^v(mdL9MRyZ3^y=giWfe~;>*Q#EmMB0Yhkf$r{6@CL!* zvqA$l1MG%3B*vb)QwpX>Y-)_>al53Qw0;OX584Tl#by<*>q|EWtM-G8jl54X1;6~B}v55Crn}2I1TpUQfK4whte2lB2Qhk0@Qhc+K1)a7Fm1iPH{6JSNiSw ze(fyKtv=xo&fxd9slSM?R=EuJ3Hy7t=-`zRZ@M@QPSOvSRv6$gT~KcUKhl4bVcmV@ z$^|asT>$^GxGl$B1k3;%m2gfEto#-J%`;n2)MIO$@2JX041Z0GgEfuZqn4!(lC9IB z36KGVkNAruqFy7hUF8)jyC*L5Kt%a6r=TUMEw4q2ag!uLNCE8wX!Vs9OZjS3H`k0+ zxmTe~_f#wK)?Om-;9#@s}4Znr zn@`(XDPd_5@9ebdY(nwzk_|KYS)8=GDJYSeThz?hFx}DQ1IG|xx{{FQmmS@HT4RbJ zJC08T(?#xK+;gi?*yzrQYA@~x0h`vl(tu3^m&d@P{mc3eq3a_7A88+mbdZgY@6O}i z^z%Rp=-hJbZ>0!?*(n1c2m+R;6}Y%j%Z+BkNHf2I{cM=J=$ub`31HQ3O-#|BE_ZVi zfWJ3dmMralReYK3_56i!OrJK050iPfyQ6t+Scrtfx$dYz08DdubAj>K_0&&x803$T zEAZe=?FH(>3UkV?3*;OmgNZ5u6s)Q?IB72@gI6)a2t#}i{gP9N9pV??U2-eP;(qK@ zD)q<%@tlnt--B0oV>_PjR0LA!Mpxf>>A8#AwQ;958+U&9jfhIvB1jI9;smel5I!?0 zcqKgxj$c`_B{*o#!vZeC4iO=XM3?XVnqVI-#OHUhBBuagB#3BM>g>Y%*f}ZYl;mWy z5I}D7g-yCRnA1jYGHR^a7-)5ZR-qFHuB?`+stq9+v-ZHX)bcrnn{R=Dnx1<)xL*R1 zK|r}pcz@2l8$4F(4bc}U+;F!m#>F4+TKaG?LloQ0KXK#Y>%7K8t;sejw@$1GCc+TX zM6=)du#Z-y3!-(-xW;!V`z#Q_ZcE48yI5?PcDhTb!Le6=x-(sZZ_7-4kOnFX0qiVO zVa#8N;HT9-Ab|K~}DIj;`zIzIa-=r508G7c!yw^s5+6cO)K9CfFed^&>Y!AuFOHNbC#(Q?-hRjCe zv&u1x!eXd{2WV_xIkVr6$635eHSdVnV^LwYVf~64t8@~u$wG}$b~r7|^*~xA+cCq# zJ_R=tU8x7g&#e-@kVEb*zD&Ju!MnXlD$$w(dlN0Fz)DVwgYDP}UfCfuF)4Ts>U-~g zF^2y+LV{UABQbZ3WayBy4Jf;Acs${jy*EOC12=Z4H{wjGa4_UbmxPPCIvX?MHE-)V zEfRlJ`{V~{q2WJf?Gc79!|z;OKdHF*wUKu72=p@l@N-cWpaAE#Cs{9f687?_7J%}- z=W&~4oXd+1bnx*heQIy8ZPO)q!>=WIvqj~1#hVZiy#Vdl5*6Yu9#&CHw@N@EE`q|wxgD?DZbG}O14p8jdsWK-I$OR0-ruf`4zDl8^zWz_eftM3*)C^;@ybib zd#DWApPcU4iIWuZB_jGW>|Z|tTnSA_Pyz)_RmrUbmIP*J5Nr*Ijug#`Ih)ou+Ee=Y zaz27Nw{EWqnfJ|3#iY+TWdXpeQFSXD>#_3ROvz2nUcHz+Al6)?Q z%6?y9*2OY)D$gVJ#{9L2pqkEyvc$3r2`q35z-9D?ETh}W?b03K9Yu(*O2~iH6gSzI z%ptiETOU4Y^ckGu0f4{l?$tQ^hU!7$SV@ur-PTvK)QD?DnOY%_OLXE(6t=`W6Zj-r z@z`uyGi5~Y;wKtE*w_zD?eHk@i?#ioI{X?XxfRYfa8FM_r96*Y;jsM<5N6;uI)#`J z0jOREb6<9yI@@w=iA02p#_O%EcH{SgV?0Eo-E6pgtmJdP3X8tliGIjC&7b(`19EO87ISc#OwLeWcHg}^pHaZ^^xsO zQ_^OE;)%7*Czy5Bw550zQ9TAnq6V4!M4#^>=S1EVpvJ>iZXGqW5B3_1x~J{IN z2AwI@l$|fBo!ctK2|P0ISd;6Q^hljG-lxhf*re9v8x zJSo#IapC&k%q#%4_y^N>6WIUSM`aL3Ls1*mH6VARdu}-ZsY3?{6reM=greO;vk#vQ zs6@0`GYTErtt-!`c8@N3(&P@jdfai@PZ!6{YHf>;WFVdwQ>dDiM{THS-Bc1g@Y7JO zX_WX!+d}#?Y!(z%ez`e-`vLve*<$Z%XbQSmkqUUc2KrCKns2irR`EYcj9bcX85gXa<-1$Tk@3D>NsrUflg+IPmaPC-Vt6t3|v{REr+_@s3@5~~h`xo*Q_D@Bi)ev!x)1yIo)gT+*$ zhvPq$7gCjKZ~T};(Z%Du$E01@B$~BP8o%DgpC_e%z+Va_)I_h2S^m8PKq(2dHTtnl zPsoYh9)mdGU_cD%K}w1#ba#^C@|9Q6@R%po;1tU*O}0x?y$1uw#h*ZqWe@h0;x*5g zE90lZ9*C(=^KEvJiGec$qEw`obUY0<3@CuO+;LyU$y)g7C${ry%JbPxL!!^Okb4!n zX{P&h+AS>Wou7uJNS@x_H$~q@a_h8gBRw2FTV9epAi$TsH+N35D9Y@NWkw?TsWc-0 zi*lF9%mwe*T&v-D<--udZe}JBCCbir(~y~nlEgewHc8+EQT3%Yy%)yizpR{l@VQeM8;a*qDm`y&o&RdXv}+v~^ZRdx=XHdU0ngFH@w&7nw&ausbrou%8`o z6t-i>Ks(CQf2;}}_~sq^Wk4IkkEx^^y(FcB*`wDri8_XZ3j>J5Tt8ep7HQAJ|7q&s zc*%ut4m1LmA>0gh=5c(YDU(jK0b^{RXV=KBQ_Q&!aMu@Z>^kS&AMC?4>MOUltyDD2 zSHU#23{Eq#G~pR3lYKrTUDRxvIO3=@QAp~=yJ|0p&KPZecvq_FyokRQ%5?Wby+P{f zJZ6NtQ@qMOXU4~E>jqDUcgDWI|8a++8&*$Jpi5SdE<9_oP({L)al+G<)aQkqLAkUf zYfDFKSJo=Opp=e$L?rvcGncU?fRqmm<8TB7YzCc;o+nf960LDE)8pW_Fn2!X|1dX< z7k=)(2pFq4u5$nD)P*KJqnIdiY z`z5K@NcBudy7|7MVK=4a&e6e_r_I>Jm;vwnSy#x}a7G^$ORXFS^4x_9*lJ~CM^o3A)qDwbQkQXy zK;j*ls$c&lmt~RVM9bb&)JMB|CN8cGbYO+w01p9&2s@E?jELjZfQ! zk-i0M`gHKh?c&|c4u>ann6Oib3d%;8N9yCK&{n&ObbS(W0ZGNSZ3k-Fp>^h}906Ut zyC_j6)523`l#OUmq8BOULC2Q|%0Wvep$z%Su;Cf337sWIt^NDK!4C+0A|gctUi=gE zgGz30V3NQ~9Qbx#W>32f`_J+!lf4SGGK@NgSU=V1@NLCnEI(^yd);ujSt6#%|Fhsu zOO4CL?1ghgQ{U3xOQ=OHNM{o!HC!NT-kB9%$AmJwCN#?QFw!57WA&>eZRPg6 zXuLl8iB{Jt3Ta2oC6uvqtJNiOOv(LK-#BEChm!q;&I~1cbQdd$^pTPc14?x}}dSYc24)n{y%Xw~G z3t`Sp#8np7#CSR}Qt5YvgmYRBl50LhJRr1__z;o%>C+wkf~S}!KC&-2^wcp0v1V-k$VF2J*LbT+n9 zGHQ~hoiDRgMLeqWtT>@VMkcu7oj z`11jzsX4L zAJlilAZ(+pBQFNC9)sbRKnIui^^=XA%sb`^AqpFDf@vZw{vpXW~pQUB$Z zM!}k&n=!SF|mr%G;>)nVORXW7TW-s9rBqV^C zM|6qHO0Id!Val4Q$o$Bb$TPf=;nM4R9UXaD_^kU=C_Q%?MlFHark<#%g&Xzt$l%g9 zTknaV=$_JZ%@x>P>~LYLIz?8v(bM;gw9#Y#^T`wIjTxM{I_TX9fvQbRWh0h#*c?X0 zrN@je*%5y1+?=tg&-SchF6h@Oe(1`cX@QB?CFs|M6*y2g3I}A`?>@fJ=`AgZY$u&6 z@0&E9XsqPL($q}^m-@kRYbYAAWtO=L*(8?z-sJIZ$A((=>XqZKcLK zxcYE42no;IOEf;Mn06*VLl)X=9PXf}Hwm+2EGDIHa)&cb?FP-AE9~jrz>HyC6y&wJ zDH=Q0#9^ZwdL1qgebAU5$?TMw9=-AUj;&Ikt@p?M^iItePFJCZ3GVk5>#lDVSxE0; zFZ3gJ0^SzQJ2b~jpTE34)&bY!&e6kQZO3SO%kSf6aO7%hp-@(1F6ug2i^fVvEB$cT zGpA-H&5SDDmu?)s{*?TKWT%yLBg;d1!oVpU3|BYscghIY427X;wA;O=gxvC~=u027 zN#K^+RhL@qLd)ld!U^S9k+FpG8q*5R_IikzNqBBSrN`KCNpNiccByXkcwN9o9b(M836#}E}XjaYteFWI6Y%$V9xr54-ZOuYtUfkMj;xweC4 zMt?|E{cLq`Y4Z+d;{gvs{8or}vVA~5f5sRJ__8zGt_~7rSltVHwk>$Z=GQk>$LEQs zB;u+AMfB~*gw#Veo!sNnj7!a<&v8dj@9vh?upiX|B$7*euRKAN_hT>@@uV?j7WWF_ z9yu-8@{fC6#w92H@*CH$#F-~yBiVCr;)-}Q=$=}2u^J=EP`jsJGUNP?aK^Bciof4; zXPK%dbv;~}*PvzebR!yXYt-Ybiy2U>%oE`%nZ$af|hIbY@>^1fjYXbd}cTD*<#e> zdCN5kanV?BPCI)M%l*CGqznSS*adAJOUAQwwdXWy@o8`KK4?yY> zO;yBcNSuv|msf^oVq7YklC(8S3Ump-M}}{D#ANK2Q))g##N=;ng<*<0`sYVFI;vCG z4Z}&T)K8=`h*S1g<6JZ&$K9d})7X`~Y`$XFj+LRxCi*J~pgzycbKn(cb%BQ%Q^aS3 z;v3`Bjco&6BlG(4lBHIs)9jPXM@ai@1*fam(v4MLZ^Y#l6Kpe;daS!==rvSGEiBk^ zx8SM6eZ;%h*RMyZxP9bb7wQf3wq7L<@Ro~u%BAxRVa~|jR1zPLZqaM8nx^qJckgd1 zJeO4}`B`qUkK(HeEPeA?dzCHTjXX>3G1Qh;L_9oFzh9^gD~4Vu7?^R3Ey~uLA9Dd~ zr8}MQdPBuHX0IJ9&fV=2ziVf5c}_G4vWA89PDFl)R({@Y5X??*rZfm6fO>Mzvg9PN zeE3|dWuJpqHdq}OvznMw&409^am-7N0%1>ryjJE3b3vCu-=C$1V99-V1k7V~@=oT^&PWFb?jcD<{T=n~(Q zlM&AL+)5TVd# zlsZYWh?Hr~TSMc{xRzkmf3@S8E^pon6Izz}bVjR)Uvs469x z8t^-Aei{S^Q#&mV=TL_2Ekz60vhlKyhmzgM#xZxNdH2NFw}PV4MUoqhEk|_^jcqn{ zx=0a&)CmL@vh#(vyPUuS)62;Dbe6?Zx_1?g_iZiF}o4DVY9Q{HB__` zW4&v$@K5WK?oHJp_UtGX?e>(I`=PR{d$(J*qQWtbUnLjHB_-h=s0xN+TWS#ka}h=n zA#)Lp-cK!`&KB=~cIeAWlWvbJh>?AvGD;=n(b(=ogMO_MJRUaDp0sQVm@^U+By`Q>Y!@UvAw$!y;(>z9gWI zqrr592kykhExcy$C1p9X(MK_RkC`kydHQ__aN|PDh!Eg><5U*-107zUu9_bruTkvx zW{-HlJI~!rjs@th z=X+>ZolTn`nAV($87*Z8!QMwbOqoo@^iURq%gVW?_gX!Hjt)G2l~%EwWGBd*;BWg} zpIPZg+4G|w_g<-TBeBO2f$Swbe#t!aQP23MNL*5b1Zpjrw(ryjm|bPEdcRMM%2s4Z zxV`1q^>h7*2d7r~xlp#J`xlylb1q%!Q@==7m5;iBj&}_x)vA zn*1$1%u;{P2smOBWDE=du=rAe&oP9=sjJTx2LeD2SMF7di>m|@HIrUDevRRaXU?3V z_j(zN*Ne?^)_p@#{pS29+aEivHNtQ+qr3AZ)$v6R-TUVWmDZ)D^)4(~NPrWm7w-Z* zm(6lP+-2fhh$2}Sumqvv*DDeqc^BmIqtYfdb(y|LS;&S9T3d1;_VDd0Q>J$^v}dqr z^Ievi3Wn}^)P81m3437Cj+Lr4{DYd9=Bvl5Qe#P|>w*$3Vw@<0_NH`WNy~8BqS)Ka z=>3BEUEqwnm$aH^Z97UT*2{+K_SqZJmPy?Ema5jhXtI>Z9t>D_fw8u z1_p3Sbkuh|3C84#@?xU4Ix+=&13fSPN=*)22~_P^!B_iy26C>i@+1#(GY%LCaB{f= zQxVF;LYEI}AWFPJ-g2u*1*=vS%!;Pf-hDFIRG~8w8yS(mq_mpH#E~3=69juH?wZ}a=SlTZIZMtg*s07^awevIaH+I482B*r+mi2_kkt)$b z^+82Cue9wL6k2{Xt~2Lk6SmkGX^453r*(sqEe0Yqx@9{0`2Izgzqi^gUIQiQ|Fo}w z6Rf!!kbV@$aov=DfXjIy=*%YEAh|R${>+Y6pW`_?fdmH5>wG}Nu~8&~P%%AxtDq7~ zky9oepp;FwuE9_?FW67D-is;&@qJT;g}TdaLk7*no}a2&!lig06c;(Gr-tk#GVT~XWn^-N|+_c zo}C+|>Tp{fr9P9-V`)Sas>BjBdAEwukFo4*w{weRlBMF!U^x|F;G(GPcs#iNeaki1 zrV_QK?t7wM$L$vfK3Nvlnh)dh10P?3Y_iDKa=|+W#;$o6$NbmiTU-K*A&Sw-7^-M6 zfXW|t0Ps_-Nkjxn7O^C-)DcB#?~g z;ijMe9fV(w1@qZ0GWZ5M@Hm(`59i!~9U*>9Rf0B%5I@~p`ijK%u{%X)EcqQCP09ej zy$}5M;eKwi&~DFoTqbku;&W)rgeyr4!HCw_Rd?HkwxhitG^Fa2P||{< z!>`%N!LJc;P3XuGxY~rtf$)T+@lQSgK7)kvvYC-g?ib2;@<>ULMQbn%d&s+d7$2b3 zH9e?<_bD+;vC1(qr%Rm;uf|!|7_Z-XETt;c_9Lr3IEL4JVV&m=Hx8~FvCppz^B@gF zsGt5Yt|MYIflLo)vBo8#>HKAkBRgnPgY1mjkxdq=RU^(*eUO^asw(;Pfq8h4NxnVcL+S=L>YAGlF-b3WGiUzq4 zPtemXzE`ION!|xW=Cf_J`e*2SFDoB?T&^qbmESe~UwiBoZ^9QiKD59r`$xz|ijXG| z=V`9G{rub#0gVPmv=}A5TX}PNe@Po9%F=m<8D%|W_U^=4 zx>(^qk6Vj-mlZNu2`I-#Mz;sRh;Fe4k-gzpodT7j>BeBz?!`xEa7j)#iiix!?D9VH z&xR`atH&FC{yg)+!_!uJ5+DC7XNErX8_9=XRg(b;Fh9>av{j&C=DxsPuVOEF!EsXL zeBmE&d}ys28r5@$*0@ZL*p>*}W}2s0h;064%9&HSG)J~VQj*XLG^qUk`C?=o$>MZ=Iyi&3zKzGi z)~@||>)_!Ir|55k8Qj4I7bV81OZVhUi?RgrRdK2h4GO{&P;#~0m$`8?vZv&;_V>Ny zwtJ|2p%2rzHqrV1`tWL4pU%^xT0rLb)IibK6TcqQ01aEa^&d}QaQ@PhZKZbv{+YG! zL)rc@A}()FE~3Scz`3|I=h#E$8z4b!EBlSiDh&HRSUcE03v3A*N9*?Jdw$|xV!M0` zvF-MDn`xHbq&NNV%^)4^Iejo!U@>ScI5mYs5ARI*f$Z!YN9JYVtwTMv|JxcWf`Q^{ z`TG1(A_@M~i%OO1;CpF5gXhGPLN*^PWfR1$>wEw$Zj>{?=)}hM$6gNuv&f(5c+$jp>GJk0R#_^ zKBSCUwaVk}e$9ijpyxc$iv|_=YAS!MFD(zoFN!-9dkis3-bsk4BkCax;7X;r^N+C~ zwXS{*05lK5>d>n%n3DBGhLFURZw!T>;67~4{rYCaC^cBV#dYiViSfW7xL!-|@;)q? z`@}{lNB{fbYRFabe@)im6J?f5@9N0i?X0ht(-Xjrr(WUx$J=1Fui?fL`g6Ixq&!@H z+_>x3^iBSO1>tSKzdy6k^ZP%Z{$Tg_aAuARf83YJ-rkrwdUt_k;!IfIv_eKGWK}BF z-tFUH7rQq+sDmEyGE1j`jge8I;=v?bR>N`lpZo)&9kRvp+3UxS=IY>)H9vSxfDd_h z5uEL^EN7re2NkM$4~mwyz;jHNg%q@To>mh4N}27@UK z*)xnWGlt*$3?sVh_xGO}XU=)v?`MCW*YiHlxJ33T1@neB%c=Bxwc;bVsgVbexa8!- zAHmCgSFOit4;PVRg?<(NFufkHprG*U)!$9pE(xP9!Vp`v4CT1ERW@z-=3K3I1Iusx zKK8IJx5SShrms;R#=&9QBkPVli#$N({q6YO8-A~jv7PAhlj3(~>|S0SI(N@&l6c|* zS6Xhx<+%%+`^|J}i~kT&fBL*j+-&G)QIdYy{7NE?HUr^r>y?qnGB1nh-t$#kq);lY z)NU>1^VctUWy5B6^Ke;Ltn}A{6S0<5fNfoyLErtF5^tMsvTb4U?yxTJb&vXe8vKGT z>h`?bv+_Ye?Wk*imA`d^8tA_XqTMQxhw}1n=Peqw^mqA!h7+76lnHM7!fDZS47gF4 z8f#vI#p@`Q(&uG0wfeq``IQ8eUv0|PVc&W;&#1Ov7R|M(pVzK+;!(@Or?wwz4fs~ve2lw|tqN+OL9?$4eRUo5ziE+6rSu>H zQEildGOp);hcY&CJ9E4XbTvd2h+BSQ3)rO^5EgmJ+_`gS)ad4Ni zsgiu+$0efWazs?_G}rFch6?>^2e|1q|DUX&8yMN3eVR2&oZ>9*qeVc|XNf|z+ulp6 zD>wyR37RMGyjTL~ol)hAk53%btzA~T#XkVQx8(hF|NO7scl1H+os0Y6TIcI=^G`(b z(ZQt!t+F}YivqGL?pazb6r?z8Ji&UI!axS4Ze?+>5KH_vxUALydUcTbyua%(`*CGD z=sEdR#@67uR;e}W?k2_{j6vJ%dbV>l)l1=zOr+YpngP)-XzomE$L}4GQq^-Oq~*tZ zK6u4hvhvzdHrZ1kc#|uQR)MIs(&mx78|NERUKS*;JX>R?z(wcCAZPU)6DoP>#eSJ! z!bbJ;poVMT*sVz}eZ zjHs0o04`e*7})G+@9L7wPruwKPE*^$pO_rBB#k$BX2^3EUM2@cpLOc$4iMMG(CEPk z$NkFA{Z&to9k&uyG;)r$sl2e6L;m9G%Rcb5Ebn8xUcD}E? zoCg5Z1kMQh*o*UEvKEfUSGlq7MOR z7H#k7?|y0DE#*FUYu>9t>K_kyFZNqhR#a51aj`JCs8(>0hvHkMmIP9$-8AY+wa(Q6 z*dD15_CEK|dy#j6HrZu#yyNxu_ICd6@)PLL-BtNr3U|x>>m zu$PW3s*cqMZhA((*K1!_uMg)W98!5Y*FEJqBqS$rnr|aJ5fj`cb7qyxS;ZD$oT)*}1N3Z3TS*4x+)`E!ZGK#v|Clo{ePl1HDuih}sY z`{H=l!%&@+u9p{oyu)D+Kuk=uM*0xK?cM} zP5JjYm!{c8+kD%cHgh!=`;*ld;Fq?rHY5i^JK%m!Skc7!-K)gxt{0lMTLn<5uQhKS z#b~nA{S_Qtmj1F&JkF#;B*|drcJ24%Knt?q`S}e7pz6Z8c+IY`QreJ14eJ z`5Bn#t4S>}vkvZ`B}2Q?RKMw`j%79&WsN{GKRk(wqDy8*4C9f|IGtyyb1Tk|QXt85?koWl<`KWLgb zY_Sp6G>Lb4cSMT#_|dh4>^0oloAK+nN3*|z4BJEdCQv9r4`K^gjC$Rl)AEe3Hf4ADR_wnlYTh2Q+4`qp5+=5vAb zcKM#q1vKr#1rhtMHnn7xx~Irk$-Hs-6n1?$?Bx2WH4R*YelBIopAOvJro{PZ^>xs1 zVV;&B<6@QAh-TPQVsRaY&Ggf%sS)XErm-}x3T;~}4L8)G^l+iPVz*_YKsNfq?>Sp` z%D5po*AGe0*#ERLLBrN)oM*mvkkn6y=tvT{Y{84MQFs>F*SGS?D+ob<#;tjpd3}`q z?_b@vwWx-q!_$S32U%vQ&Y>+^s(%QYJCp=90+5L1y#eh}IgSCAgnC!CcX#!(4f5GN zVFmb)T%!4tA&!SMUDn*X{gc2-sRxWLo_A-`IJE%251gFtMYh`b`Z%)SnF#cc(VA-) zZ;#yk+ww!zti&L3sD5`CIMx{*wMHIybz#K^a=(cprYkP33yQO0jo*4 zrel&i!nnx<^3?pWpqjjz=#()Q|7HtRmMiOpsu*Lvr-r)jL)BQ$(zYA%LcwF7QgW7d znKQU?$}=^G?Kazi(CsI$yV5GB+UXpdm?-uuzEs9)HEZh|(5zv%hEp6~f7|g#D(i=F z)q%b@Ewu@%&5eUBWtfC(@=r-T7*jQXNBZ_~K!sXt*^6~gP8i%;tUDR4L3drZp+KG?4O=LZ7OE6a*rLv8F=SIth3Cwy{0moB!69qFui98~lv z6aju=TxsOND%FWtEBURcyXbY7!|x?042ZUZ$}4JCnh=Yc)`&=bV?**$;XfsV?2A`> zGR{4L^*tkURylZBI3td`^65e7y}y!?k&#*vFLR*Xu}bj^T|qarUr7fY(RErOav{D+I+ z0FTu=0Llx?wZx?@oE8P|Abc5&;SyDU3TjSCGCW>8w*Rt{5u0~HVS5hu&M^J5nSa;; zj`WU(ssVhD)SB~hcW*d({`Oss4WkZrZ*&EZt~qde-#6onDI2>_J(OO;cctA z@#li>ss${`Q%XLNMtdjjBf(3{=BVo5GGp~q_)FAEcke&Zl%8;F^=7{i;<~)JH_Jcx zTbUWn1VniI{NTEx#8~FF;r0K4rer8qO*$w`U)CVta1Q)e0@&8!?OFhQ#eaoYxPeZxZ5{(Rav^LfNZ^^tE=nw z@o|T#eE_2DQoqHN_`HmtD|v%tqTfhYupmFD`S%0}--#?E4SoB!{| zO6CC&Jw4%$Uw%N-d6LjSRt5@51IrpN{af)8{{y)HR4f0}o)|@eCT+U8+1;Rxb;4GQ zikz_lsP~?i$2h`c<;&&$S1abMvJAHQz$y1T>VfpLzb(%ju(RgHD>8Ngrc1uR_KXMm zyv0y$Mef8Vb)|db?DJ~JWS|G+vh}n>)?qcPbe`$|i^b#^T6w4_q`fUiF7m&+`@UnR z0Vd{6Yuhr1*z881*}TQ>L)8o3=e;Y$0-_r&#kKHFP!?Ch7L5m+M@glq&&V&UBZ7)e zM)_KhHRj84Saquxnq?p}TJ_bEO$C^=+4!JhD&f@f?iXwT88;|7f#n$oJ}&z9tCZjU zqW)H%wy1q7L$6u$`lpSY4O+Wt1H0UNAKEAx|0{<$|CnF~dNR1(aRT$1GwyQCT4nW0NNmAs~JAvBLNfpg5P;|5DldFFTZ8|@dRfR?vhRRH&2 z31VD)+F#g>7s>1am|ws`)(5QhKK6p$MdOH;qz)kOeaFL#Rtj9E%XO9&s{GH?LTeJ^ zYi~UxqK(SG79i=ReNN`DnYGdmKu+~->z-R)2Jb-)dPX)>W#v=$9Fpk|D?EAYM%A<1 zB5E&;g@wuvp1zj>MmZE5t&T2Vi6-xOwsHB>;vY+U`2uuS{lJSimZj$1o=YHA6anJ( z*eXmEWaH>9{GjRm&Yu^1kw&`z8;83JT(#dedQrfad&iIWeog(c51GOVk5hj2lY}mQ z@d(%+^XXA7zcM2sJO(h^pk5aQg_?dAzbwg3 z{GN=Z{&Pk$q!!2W8aAY0%Ra`cd+fUNp=ul7<_#&SesBMlT!{KIxeYg-X_%~geB)D< zT)!xYdN*v*RFhiXmRd-ZoP~rsb{N^A=U!pFxU|=34im#HHMunZf$Sy44Kw-FpG(jw zPTh*VpeDC`Q}B$7K&H?KFOj!@QCHjbjSK8c{zE(j;(8A#75lv9k_cLbue}tt@H8e6 zc=d0g;v#Q9-1pnQ;<@Z4ugDHKGQP#~#o(r``-k>jRpREsZ{}CZ>VEgFZ;rNGd*@#= zf#n-prGt#8V?^WyYQ?JB&Zs^4PgFoyi0wv8^M0CPWw$p%2mtlBP`7~v4wUUTR& zrxsw~{eN2K(gvedwu9yJejpTTwWi0Gy*%sh{v0{|<>TsJDKJVeH@v>T%H;foWdxV$ zZ~vlj>&EV5&qcVnwT?nbNihC-wI+mO;yiJTd*DBAB;g8>``pu`4-aP)as%@%ZC6N4 z?pgkECy(TR3(Fwa0R~0EEJN%KFzR}8^MRADx6~g4Yc;Xl_L9l=JGgx3#XtU$zhyr` zwWAHmFCbtIv`#yP?5$vtZ5P$@|4Dc=HlVxi!#P3iqc3MKvNuMi*iVXMdTI|rR^-5X zd>%SlU(tx08kA^U)unV#^-E) z*~3jhB7a}6(+Q?)*wB)uv_?nM{J7&y|C**5sh|OL>Y%0oB+@_c;0jr}N~67yeF4q? zY|XMPb1*mc1nTA8$k4D*R8VkNUemoQ$|>8w=}7o8QHy`@zbP9S7-&;`w58}GdDYbb z!^E==yvikwF$NG4u+29zT{ zP;v}ZB6+29f)7HgEIuy7i(76LlmmRdY<)5=vT;E{4S>gw9QPfa=UB}zqE@z$RPa9q zp75PA<`{>>zo;^=jCKqRgh_Hu&*#u<)<^6yU93^H8LN#ga;j`U)Yz(6Z#e{g24jfB z|768FAV4`Z>74k#C*vvS~=69Bm-5^jlI^6w?lAxboJh-Fg83Qga2YQtWuXL=6nl+4`2oF;htm3% zt|f~Qny=qp6A$D7=l7U%)5Wb{i4_DJt>(xU`5h7@m~~mzwmncEHG{~_Sk@RLXJAM$ zb3bSX0!ZJD7aCBgr2(HK<~dqcXSq?0<_l~%wMq`(zW1E%r_i7iMDqoE&G->Cn_Ty7 z=cps3;4gtJ$mNJ6334MhNZLK#A}D+Jh9>V*c6D_5Ix8yAjqgf+1{#pTcmB^o=$ znb80%ox#7K~d4Rod{HS zl}%9Fi`D_Lrt2L`AmOgNA>IL?TAMtEwZgFeO_va~LZPOwaL|@L)jlQ+&?C5;7=Jwd zn}mXel#OrrVxs~zI~x1aZgbz?0X;!ruPL!C2o%P6;|(5g$noOa{#UI?KnCdK?&AR! z{~u%5oa^MP#v?|SqQPY0yOd6Q!$aE2d|)P@t91v|Y4B!|%78Fd1^lQDS`{FMsG97n z6q*|VR6^K}+30w)$jxq@&m)P8K#I{Uzwy<3vci%{V%5fR@iu-MCh{7h&Bi5W9W@*x zGmddn7G2pRsL~;u_B)?&hB`OsEn^_CUz6AqpRjOV`D6_5Np;iQqA@pDB4Zl(ua2Vj zf_^MAYaMt}mbUTB--EN!p?v%v>2?_AHRP^n zv-slmVPnIzqsjDR6UHt{)&ufV^tk%{gTR8>T)%$3t^3YmP1!(84Zp;Dv=~IN=C)r> z03QrT`@9Kn!x*lMMtxq8A9|qaz_k_pb-ydh`6e5v@b016RCQE^# z6_B}7DXZ&Oc433Yw^}+ty#Ygzq^{Eg{Fs@r(DSyFLr8qBe^Q5HYGYhMTbJ9yN0Te2 zonNwy6PR`4nFRxS$lp}$4{8&okKtOBphb41A0^mf$>-17JU8kyVF(bMMUF6ai5|`F z)nW)U)#el}SYm2opfN`5ytP;YOiGr|9#ayDf8t2dD1u4l$H`M=YqP+PwY(AoQ)CPW zqI*PC6Cti`52}q^ZF|MCIR%$Xaw3W+zz}A4mJQuOo>5NO+Q=9U!{y!}!8hv`*})wn za@LHS*0W|!By z3n0hXx7NLHO)U)R+L4z^S+jmyPHMy(pB+L+JOVmwNOWBAP})%Bfr z&KE30OxUtF5Tz5PiPX-9=Zf9LVtn|-mEG(Df{e_at4ox7szoLX@HRE#`Ejo*V5yXV z%h)z`nEpX(6rXpD!TS+LgwMcGX^Af3bP_8>mtgRIc*L!PGL#H+GB~Fe!_kh8wKwSp z-Dke^(ZXI$#3L%zaR@{&VS#89L+|iHkWSKF>5k2#reso2t*d>KS%EL+87rjOwHWUh z${!HAAJUzanV5Lf5dP}x5si_D5g}3WBe}@r{^08h2>VA@$(?xL2y*oyr}z%{2n3&m zY(fX#F7iM!j7hkD*uVry&zY*W?L;JWr)j|-y%ADG^k@^Me#9AD>vSrNG=r1Ujc$`( zb7}!s<j#_l*UR$HvV`A;;kdmX^bzswgf@%KW|Sg(5z*aB!)AZ4pUD#AhBK%y)Zp^VyZp7 z_3Rq$+=hZX;S^1nU3S|gQVAu%1&cAGJX5!BVTRejWrvl;W==~v-5e^kBNh$rw2he` zVaDO(;Wv#jW{g|aV!8EA9?f`&)h-tvmv;`wzEcGskU8bU&uZGzJXOo%?cqTAByRbm zA}E@^+4%n8wnL$*=E*)c$4X0(Liu;ZTqkjG=gu*oCQ_VxW{$066_7wx}e5$ zzsN2O9$z=udknsJN5vd!A99$mU_9u8+^#4c38J{1#6uT#K4Y;Ai{v|4?i_kNLiAh^4mi9g4Dt9hBa!3s8PH zKvWp(V`WHvdLIs>n{^e_7CfqLci{Xzj(+{ar+_DMj0jW$X>WI@e!#*68wP%f|JiL$ zJo9F;WjL}4Ws!`bmnW|K?Z!#kY*xwjhwiIVF7(UIuN?`X($OJ*`ygH2C6sHx z&8=^c`h$!->*r2OF=v9bT0MHnvxZYsMUXUzBVtZ=&J6X2m^F1;H)a=U6~UNzawkMP zkupT7*LqZypj8J`d`$EyfCS7GZjO=u5eJhk93>8XfG$g@Pe2sLlaZ?ZL5!vdMudAu zmI_aEZp_C5JyXKH0NMT7wXW7~)#A*<=dlOQ)3j5{511>AxLWdt18aM(KO_ti4h-w2 z{J!oIie?lr$f9kxMm^)1J{q!_1J&1DapfB3A$P7U#9@qLGY7+UV0Iylh?~gXx{!BY z-~mAj+NcoxzTb6y<7dbrO0hFj7sbza#$A`KXB-ktZ;#W~TZlvO*~VNyQK0*HC)=9q zrhIINuA+pyszl|$>4)1+D?c6&G2^qpf(mI9KX3h{BF6Tl>|v_B1G3eYqM1A}h)kt~ zdo0{8?T_-Im`FuZ<(ckJWIJsMM*8qr*}{t+_2Q}$!Fr49%lT-NeNh}|jK|#s&Kg)^ zsl0RLcSh_Z$jHIt+PFUaP_Qnxv8~hDQ|S!f;Za z`d!?(eZ`S@U_h?QADGTOJ;&PX7&3Aoir2p|r@z|}ULDK$P*7`csVs(z=N+&$#++nS zB^OMshFj0K5S~~h;ixTD|*ZoQi z2V(|~M@BirpTZ(-PBLlXx&)(YaRg@nVZ^Bl+om+J{5b8dyT5KND6cKCJMNnIL3?kI z7B)FvcLKgeOlBb+Wa4UukC}niY`DxInfy z5=-|qAwkwD#w<&hQf-gZhL1&LYTA>m=OWycEPx9|A+UY9zuhV9KmNH?dfkf}Z=n@*+kG<$`qE6KTO|uIhZ{mx!hbKZIMjM7ToLZZC z0R}u`V{0M4e-EjAw3EY7YN(dJ7T{&lSn;>~ChT$<8?rAW3{s7Ct}>yELv96|F1on9 zEYT*wN0@{@!k~Efz3Lfi*i?h=uzds-IoRxa_I8_1^I!|iu(d=|aV%KZIlT3E1aqb) z!Spx+9^&x!aA|o&8et}t!#|}w+KKLna!r)--a`E1zk;<)n0uO2HKnQ(Mr5^M0rKYQ zW2R!~DRw050{Lp&0x5CF(5Qmh8w>C$&6M(}D9@=qAVTvT^U*3~)Di5N7Uol*n;nRG z+Fn4c%ex=pe=}hlj7Umya;uOe3n9YJ;$0qB`P#Qn;cw+>!uA^!&xPLBm2$S z)>7StKicAdbsuN5bv#%s&$ju)2$DPwj+`P{nonsy)|A&Rgy#ZN!V<-mSG!^{{PYQ0 zk$XT0zrId@yrb4*ICu*ai0Is2xn)8DNTNh$yj=%A(T3E17z$6C_|mn&MHEe`0C{^Y z5*1^#cvuLO8Od#`S{lkRvW4mi^e@TmVnCagS}sqU+6&+N&M|a&_+hH*(BV5X6B3Kg zYQm?WYkT_4mkI8Byp!JhCri4GF?WYC`AWaVfk<{&@;$dIstxQl{yqbyc?oaEhg1$ZX!e%Juj4h z)lJj_OEa38IF3upWHg9RVybO_qsPS$`;0OojLZWKtzGM5J5@9L_3TD{$a)t#swtm> z;XVZxdO{ZmddsuOA)N&!rF?zXcExk0_9?UOPcH9nQcrG3) zB46yo4yt@+0luYGKJX)Jr(=XdK_F+UpzO*fjc-kfvMr!eQRim^K@=oI{LDD1N8wSK z`Zc>l@I%K*A4>*2e-LBRLK{!B2(+P7I8UK%Ni$l$Nb++d{VrHKaP?+Q1*boRBK0aS zO=%)yrE(iOvrscb48i25`qplC)#8~LF&EvO5ygck#hNmQMvct$jO0@GFQ9 zfzu0NMCOz<`T&Pj??u1!9L(IcRU~xZ?E9Vg40*-$HejF#;R)Qtn);DZ_3GhAdE7A_ z%k-&%X2XSEPQnlMPk1*#Q;_T9i#T7h`B7#8C1~4`XI9i8AiDmw!n4RtaM_kZXubyN zqt7X{FD|QjQ~N4Z4cLyqtwNy@KvM4XEuOToxNl{bcv}8)u7D#&OP6sPso&1iTOzD` z$A>>31Nz<6@p35{x>-goAYxSR%~FK|z2Q5M*(ylq;u@Dm9XKS_!axx>OXv_Ch=J8^ zo<)5BBArl8>>p8rOUwBLqQZS$hxE6j$GLx8#i&)ZF4sF(HY%3L0L(yb<4lrs==}S= zUs>9MSfn-qLF`T?`mG=w5b`3~Z)KV4A4O!e7)ZM}YRipfGJy&Hu(oBjfc(>bpm*;A zWs|kM5sQ#=X!m9e)FxJw^cM^nIRsUy2(g#+6*d9H8uYuiGoV)Qrc`r#N$vg}Q44MR zsxI*dVXF@sYaX719-mRe?Yt_O2aotuG4W4L#1{}~n{jE^eK3fz@q5_oe~=5X5bqy> zaK&V#J0`F(IxE@s@6TuNgu&StwE&X!`V!{5eN_7|VBp2RP8u}hZtwLebj6q{=qbEj znr!?p;?@kc3Zg?&Ia&jpLxNw*w@+-T|3~vv_A}Ha-ky@Nv<|c<&*<`?*pkeB-@o{d zr|c6jFl3D?cKiL;BQFlG>N@<1fzSqC*uaS7)B?Waxdha{sS!LG0APqrZm;hw{C5|# zss66Z>{rHsU=8p}lUHtBpuMt6^;Zew->*+Cxy)+a6Y!SM)f4N0ar|7%rC-jSS9GU1 z{TaxXjK=`}7TTtYuiB1wnALx`O!rE@Xl2m@|W`QCpJr7 z1*Yc#Ws{DT2^WzQ8gO>yZy4|`U0|D7GBne;qjCw)@XelnFFuf^`)f0>CjA3|ISE;< zZvU6La^wFMoWNP7RUz04qDtM0CCF^y(qAQ`fPjBk(YG@kmh2Wd7kT>s;@&||8xKjQ zRNB&!XfJlH`)7#&A`n$H5Rsb(WT5T5+RNpuwIKX)lhBVF!B$(**WUQI&gXZQqkT8p zWw2&!fwo}&6xD$Z9f_HI>^xXXS0uk>(IzesP96ehgoCC?-(|z&=t=Ij)eQZ|U4obh z+5{KS-lX{%=SJxTih+Y|dSO}@l|jIoTFzu$X~2<82Edsf3;-RCvuqT5d|2%NLoTX|k9l}|gCtsQ>XgGqzL1qDN1k7Qe6j7nQT^7J2bMpYRiOs$LR8^Y1G^b7 zQwk(uirBF{v}&cY_UgYAxaiZZq-*d%DCz>u?U%Px5lFNp6j=|;_s=tl+m%|t6Y$sHsO6oof z#&7C5X~ZYH2P?Pip9JSzIPpwpI&dEZsw5EVwc2}>o!s>LPf(}tPgJtH#l2St=O8uI zqz=z=>vl+abC+_%Ua@c8^PKS;@1Jukm>NQ*0M0QCi*^}J-F0bsXnj#1SGAupSA|i^ zbLA5(i~I%iiDL<>GnqgsEt@XBT0Eh_K+Wo>hl3NPrEZV3gs89E+zK;$--bEvpJe{w zaDWS$5(z|i#J#+A84fRVgzNn((fXe0=+M1y?!N4z4;x~2eW`}nNYoeq8mWa@A*?{m zRFrd{3(mBm+iRo-n*bf}GJQyR9lB&KtjpWczDi4IDvV9RTo^cqbM~Ew$XeufBME5L z!*?av#|Z(cbLJXaVc6N6``(hb>3n)xU++v9vB--Z*Atqv${(xry;JGDdBYYpD}E=`ts1T%Yrv({1+Ug6SpmVe$tsHKx!|j4xp9fnJIZ@xv`v6o`I03d4rh}vY=~j zZV>C;L%}#NoFW-JFeini3CJl$EE6^85#Lel7TJuNt0ecYhhJ*&9oBmlb&60By_;vl z9Uz6YZsOj@b>=hel~3!&=KJa1<91^$FMjvgp00%BQ*&JMNEQyBA0O33 zr~6XzY73tXusaKoS=nU|HjCZg z(@65PRrcSG}NLFztYq&N5?rLhzD+YVq*%8OikUrS=u7`3OQu= z0LT4yv6)e2UBFk>Y1>xn?|t^&*&3_les$DRD+Q%l;)m}I=^JWy-NbBdm>a@jPR&zG zVW>%M(p5M$Z{C33k6uG=z_B`ve`?440Dv+&MGLG{?`~=VAvENAo8s8L&!ZjA#99md zb^Ub-W(z5{p*u@y-ly7(GtP4lHiSZQrc7Utx}<}geJ6JRbi-T+4zbm&Cf}WOs^{{! z?^I--uVB{<1&fWe{Q9%=I}e}A1eC&Q@=@}v<-!O#1l8yGT$eRgYM$oZ*^FN&MwVAS zC8u`ueyGD-!-S+A0Y%J1AiUqX#ORGPsg}~@IeMrc994NR)pL9(&-3Ef!s*Yvj`hS% z+&lw2`D;PavcBEFRF@B2E6@xi~-z!ZK0y?Jj zQsjz>`EloF8qzXVa8EsnUQtmn9Ta$= zu&3sa4M%S5h0y2c?wGH#DrK+zI90+Wv$_&!J99;t^u~#G9B1r+YE7^)MGe7NIyV=rcoIzVCi7Fm1P)U|m)tz#2Yvc>d>QjiV#m)5}6{*YWm?l~% zc{-0)AW>^N+8a_tGY&&RUp3l*GlB(5Xb(1j;pA3-=@1Rhe|j6RlRR+NfB$+!zjO+T zJ~7|@)?V4OUVbO@&9rP__?!A1fhbnLcx^zex~U?n%5uIobtDB;V3iEWC0UDOD|^GMvn93>G1sI}BaFN5m5s(#|> zaWB?LEo;KZwp!7r-kZFGS5@}TLPNKnB4P}d*{m88mfhrjF zo_Ay*nP*`4*R%-32H&N48jaq3gnf1f$3lSbzw`S1X=RClJ~u`!uc<=n48pJCOMW3h z)pzz;w)MRTV$PQ?r014OluT=BfoQLdYOtL@tIVOreV}o{XV_sJHQ}AmIWsB_mu+>2 z3~GF5!+CG%yW?%9wX9MW(Uj!n#>(@xr$0(x04Ka`wzxD7j#I|3V@VOqo;g_$P854| z4Xapsj5nMzbpftOwGavh_-Raa5=jj&K>c#R>==}LZ&KF;=Q#pj{G!lr!_w+FValK4 zJ>x;$Y)&nJyYR@f(qQE9JE0!N<{@#_uV@hew!iNY_XzDejBuMN7&(lPqcL0*NoM0H zTA}zjX;AUWV`lA;;aqF!Nm=af4z&m#%oQul84+<3tt9cb($r+2ZeBNg!3v#7^pOaYhPv0(|xl=k8{lg~MFhL?>97q6#ga z{A=j~tv3JEc$h5tJ;Ij}D-_M|6NIW~#^e>*STA&Z%1Tae(2^dzW=RiCbrNJeuxu}A z02Z`hbJTpcYUIf(Ajh>2Lc}W_QS&~^EOC^aZ{+&6m~iL9^$N7C&D``EEOCKcDv6rR zwJem^=#R-OaU0FE9Q)})u}9{ps$oO7ueW`@x5}-!C>3G^@l~ML)sd4D0B>Olq6zxg zsOAe*uApy{=(-&ok!>{8bkeo>Ispcn7~>l{ubtVQU2E?@?lIaa7I>R<@XRSmjXHqA z&+2$7cHVy%ehjo(0KkRF?H(1>c&}{M718G%hCmR^vKH>BJSCl+bkEtBP5|E_Q+;kK zX0PXn$f8vXGcMa^f9USOPTUbiUz!J&@h%LC5uK-KAbuy8uFR8=8R{q&d} z83d=!qY6`Lr%@y%IYvZ1zO%T||7>Hovz9&1&}VXQURpMoHyG!(m9`MD4R9VlE406m zo^R@Su+Z0)e8tQ|eWywDkV{^-!MNvCWGxj%n{e0~S2UwuLLTu3quZJf_rz@MIUOPA1{dtiA)*Q+_QCImJxZ^e9N zc}zFSBzLY?p+kXr`g3l&R3P)CP^scF=xgk%uOhokt+Acyr)BTZx4y!Y7VEHo8aiK)v=w1@V@l*Nag{!&l^`eWd~$* zV?e7IcC;`OKIC{szzeQ?(;W1FAr_u(5YGkJBq>=2cu%td)-21>dOBPe^l7TaN+s?r zj9r__FMkbnd7UGmM?J0;_tYUWs>H(e=@*HPM0f9KfH=u(A*esPtTK}_74T6V22e1~ zGdM(~N$+%PL~mn3O1fE+(oku{%o0{?399`;^e%Au9Xa>xjpvW~t_!?VKl)iUVPQ5< z9RAw7RS`AyR<3A*;ES+c&>zHFxarGx(5WC^YvLC{U1m|lRr8&q!4TH8@ZHaETS%L( zP~CgmGjC^2x;4EB{um95I;D8h-iw!=$2ut8%jlZdQ`K+qnd|d@_J?gjQLk2Mj!%!{ zf_sO@ST0!;q=b$0;kctXDsZKO3)fW*1gC!9`&^wRw4m-;uIO_%pHhYnMjU@FL)uUI zAU*pt#fmxCD9xJ+7{9qR;|6hbDx-9s09ocn*MeX41}lA-#=bDvwHni-=8P}uj>QcB zGKbAuid8}5!S|Qe#Uj7guBx+J%uNCl@379X?1WD4%QF{Jejhh>Ce9cm-dH{yp=TXO zg7JpwGY=;KNXz-&MdV(xJzRfVh#)2UCej)-Dc!_$o|=Nz?oRMMfNY)&*0l!E1cFkv zc&dF=%X3CSMfE%YvoT)+HUI#NfEpa69ZwA<>f0X9N)qYWq3ux%lD$4L$3s)Q_XGv@ z<(?zX22SM%k^guAUblq2>lG7l2-m3W(KO(cI1`|2S?UPj1;=Jir-x9gb8mEF%&rhm zdjjkTfAfs8Lm`~9kGhXw*jzAe>6nPfY4ePqBJl+11*(krSjUxkI`emnJ2l0RsS*{y zONVUX+Q_=Bltv#u3FM$#&z{IA+ePpS!R*ReAl%~qACXC*Z3v;}swDslBhDjuip|Booi3 z-v!wzfcUZox^t$G20`%3ZncAXX$8hn2u9V>{HwAf>G^axLJMvOFlXC_jstzS2Fgzf zn|T@HL=@8zf)v}7G79?QZZ-@3&k3HU7Yi!*$4+JPxwv*nHY+a_j9N}eWHt@gCPeGP zn;lM#xlkC84SGPa;j_6C#*|0;B0DbJGR~3i?d0#*)gb# z-CqcgBei%y=n$U>=P38^K6#5IM;vTWN6s8SC9X{E=nZMJtxL!N+7v`hP zng;3d(8t?M|^sxKi*uJ%TR7^d2U3F%DU5r zF{&@VSbAbm+c`XUf-wW0z=(;=OQC4$00=?;)qRH(%11hr?wMOgQmV6zO$Y~unT<^z zxd@eC*>i4R)IZIs(583Z`s4EWLO1Ck@vLCM!NN2Cp&q55x9Xv4H427CBF{Hic$h0s zme9odpDGM-^&LEA1akC?Ao->yhT|J6=n4;A6uX~S9~=WEPb`x}Z#Y;hWn_WY6Pi^U z?1y>3-B}}oV~tXbRp}w*>e5L8vI~itM(K!*Bs(q4;joqSeJHcEd9=`x!1?hBnBl@# z-oUY1`Gw}wgAS#v=AF4HX=J%bjhLa?Wz&Mf>_O?0N}|`awg^HFv-1l$dqYD})f;F6 zIFy0<(uX*bqo`WpE=R8Ee>Pvx^+V0pq;ieOEMi+dpGdywO_TURJAkt=IThi2u9X%S z`5H0O7m1+;$lK(lMv9#W_^Y<){RRMH_^ZC2^g~BVn3w-X0hlQO`5=DkOik`Rhzvb4b(x0)VA@SR+jGy;f|!9La4VzNFwlddbw#{`AJ{yOYbc zYNt=9+N)6EAzA>OYEkcznm#hG?dAa>ypEL~Us6Vm;o0~R~)jLWhU<-Zc z2W>Y6`5W4HzO?D{MvW0n+MYYVH}pBjj{UyC?#}GQg~6+;FbGnYYcI{$@kVlW$X=LP zSW%GFo?~}G^(DVob4&@O5Rh1IYJWr?M5fxYz7_ z4VOuUUv2QupTdM8la&zZRDo@XsSZ>R06+7IOeXIi^aQ|24FI;r@K?XodVHIdmg$_r zs1XBjB1k~1ZDUv!V*nKA~iB2$mnQ&B zixW6laYtr{c*t36Q3Ei1(9%lo)UzqQNH34A5kGHk0uV4sE-?fkl707HrNLmWB>~Oy zxDq7ev_*BQ*2p{YL9^QrJ)vNVqN!EqX}TN0ldj6w+lR1vLzDnQu9Kl;p%1GDP%No{ zT!L)Ue2A_kT)z&M4%FZ_xTlzoWon>LZ;>QDE~dm;Gb(M8k)sI@z+#3xO$c7YNcSfM zFyRkto%_++Ak%~CwjYxJ|3GR8N0GaJMQojYe^n@ZgGO$}YqEpD9^l7eokZoIATsl{K>b@laQ6rfZ za33G1oeLAfGQXJXn!A80z4qq)I;hd>E>u+SmyZ>!@hLFi2&*mn3Sv3Wq-mJ<==IuG z)!F;xfOHg+u2s6gBxlcqG{;b7#F$KJ1aF+1X*`cr8F%uuyMtAkpL|kFaJmN0p3$nD z9m6!YJMQ=F7x9qT%34{x1?PeK+zrR)=NnW0xC#GKXK#W@AiwLX3mjLhcg>wxiH%Im zwDZK;h|uc}!~=sB?G}*#TjXzSV^R;Hf0+v8>^r=&qBxlRO;Y)-B)2vjWOK#oQ?+0nc@sW->GnWyWP6+CvE{#c2uVSxw*nAo%VK1qu?#AZeC((M?g=2r5Tlz2Ja|M2O*VeECAJgk-$MY9ZVQ^<$an( zbERaeF3y;N=v0vJKOHqT+Fjx{tQi$Q8-jT2^~U)xI67U{V7=it`!hM!A-j*T@P$$Yv;qV1r)U zPIlsT;pRQpMH!dYb7}#|fh%OCIG>~lhR>LF4;aeW55XiSlKPu5gC}bDe$Ku_la6<~ zMU2;ydh}Zdjg{VlZNrSjk!!`QK*)78sVb`9w#@x`oe{3)CmI zQlcO9%dA2gH^j`{SDme^+oC59(o!vDlrK@x^8+%*z8Ww5C@TN%%YKtc`<`M)ly^RJ zukXt)CO!{=eAi4boUpjjsCAGT)aW}?J}1aHXL&Vdwn7o3t_q3)52fkB&edeBZkKJ_ zR+#*!PiL}R@_HDbroWWi1bWte8I8GU`G(fh=DXKz%hcH61UMzgdBKrokUv-z1Qqd( z#uuVV>}vLvhG_fx%1zv7p=qfj>Q@f(c<=v(Yg){ol)1ibzrFL@i8B3s#^#g_!%(tP;el*C3HU7|1>+) zhF2ez2dMdxPB01r!7>G_69H!Qo=5eo&b*qVvV#aPp%VpUyA2r4R&^}|GB=Gr9qO;N zSPm-tou^#vgSi!kw6BncS9vRmR74c2nB?Uz3)vXvkC|fi>y4+nZFREyKg^IG0sL-T zV@|D}9+(-@X&scCqj^(?g4J^!Zx0WLu#zF&&`KGbc-}D>Uw^I1RzILw9X=V;LO3xJ zqHA3b%R&a0YoL_f%}t#<1(EptX<**uR61`yx_SYm$v9)CL#HoLx~EPJV7^G+4WQON zti=YvG*vu6@VDs-P0;b}Q<4)`t^>foGkbU>Iat>;rdiVRm&OyUn~`0FI+b?l1uhqD z!|{gP40Cm~x90ZQ=@g zoz2?hy2-v&wA`f^2r`{HE|IXnM3L}}9uVr~)$7V%y5a@ZtHz~s4PHZilnOz&wGdB?I z$d{N9iSa}@Yr8QC)d5He9)Yr)nlpsiaP0%cXV8` zwxU>-F}p77qZvh8gMN@4V##U+1@XBC=UOd`E9|DM#{sw*)PA|p(_C5FD^a6Oq9MuP z)R;XBA~ILeTK*=%U+jo>VgIG>mWxH6CeE)-u-8M)0BloPUsl~R z+ou!5>Wu&f6nInsz6~@>0*Njpg4Gg}1v%v(IQ1--jqRK_3&>nr8c8$ebG#Bc?Wqqi z%L(iO8-Q8%aW@(xssrG@e%-KJXQO`dSeB(F{i+Cif@*;- zhyw^dR|vPdz?;_IN=iZVhG1%^h-C}VxyNx)=dtxCqxq|C+gK5J-4p<^S9VW@BrWK4 z5|}|Tu6HbM_^OLV+J3p&8^7mhbVG~Zt(NY38dq7lIK18LutnE?B@Q5z_K@vX8F@Q0*?eTo=)%+KUR|# zSdA&AcV9jh;F$^Enx|{7g;^ zCs&TiS-wAj9OG*&S+YsDH&lHaw`3O~B~7fRT?WC3O`SBWkxBQ-wFH^8-fov^w>Bn! zbN_Me68&x2Gq^FAg;%%3p&3uT_oI|f4-h84dWC6GsF&TBr)@17s7YHmoT~e{d_Msg zqbr7tn#_r&_@QJ(Bzy2MD++oueHRUMB-&pG&fy&e3VnM*6X0{7<=iuB;v5k+-~t6T1nBJTsxr@oSw0!_Pk?REK*Bqd5Y zq1eIxYgOysv9#G>>7n9=cKo_AJF5%!bIPC`?wJ)uFBh)?BhqQ1*2;J1iH)&h@PMU z;qkdF$KIUW6kJPiW7ozLqha@^@W0;Ov(zUGS$pTN5mBQr%Zg|vNi4=i6oLm6Y_9ET9_&pd>Fen;dKAJ;Jlk9p)(cOc2Itt>~!hv z1_kBkTx+m!#z$~uc&EZi(4FFiOQISVmg1e_+jZonq2PAY+)3M}V@2}S?L0UcS z7V09;ljzaTgN>}N`%}p>PQdTCwY{6UzTS59+M&L6{~P_uq}k1Bve*e~=B6#QIZEM* zW~XQjTa{$UJvbswwM(K$p5o}?&O`}|%MPBOH`hK{_$mZibo4q(V!~;g}XK9|CI<&t?70)2i$?$aQfZ=jKfqZ+z3|KMTQjuJ1H+t}g0!98+XStu@^&I4i!!CT-RpHHKD} z{(pphby(C}*S4e>2#SD!2ofTVFn~y^bO|V(O1E^Eh|)-RN=bLafT9RU3=IPeNVjwh z%zS%%^c>H5pYMJD;R3Fi`R&4@41Z9NFD5y5i=Id=d>pUj*)(?mSMdRY%KjEPI1L3{yhfu^6Y#acQ(Du@449yj zt=O6doOgGl%=aI+ieVa|f1P>HLii^hRWMS}InR#m`tLWR>s^*6nc@LCj$ zGm85l=&;Fj@#yfT0Jn2`Ay`J`YfGALfVSgsY2I`yZWJ*>w5VvZuVM{Xrk@%K01}RW z@kd~uXmgC5@U786o}qx%2`4`!Xl0dqiR}3t>(`9v9}nf(G}oLt?3`_J9-e;ZtSO$> z@?LnFR^*H?FbKDmabmd*c%4Y2C1+D@G{j5ua-!Zt2d5#543-CZ-sIZTqQpM^u4=aWPmJUR3F5I#O0hvW@j9;zNQ zotE5UOL6fnRg*heOEc7XB}rOuy4NSw*rEEhN_!&}qkN*W5B z((PEe%jV+9p{{FG_Ft^|*BYAg6wxr6o`&R&3v4;Qh~W60GO&}!uLLl1?_f8M4Y+Y6 zz^#_B{oJ?o8UH+><+n)mFQywvQ^C>L3o~v0#t3lDz~HA+jetUimkWM4&AA@PLOoR* zkK^ucK=H4sHG-@;dF&vn-W5aCk_KHXu}3!|rjnlG+vvFO4;KUT*lpIpyV&1e37gu# z$LVcXKflx6VN+RMVQkPb(r^JiiEf0#8$Q#wY*T5EYCiN8bMd>(U3aMNQ_b7|33^~D zxBv14LlU|jjrw@4)wn>9dlpJYp)F2cEnYMC(91N{C4IF#Rl={Q8=ig&zWk_hA%N3L zqtZ*crE|F)y@b|84Iro04|E+A+`fOFEIo;{9dA*#-fV}xXEdq)Y-suf6#d`>n{p=) zlcAa_b9G_1z^-ikI;T_2qH>b8-L;I9N5Y7wy^2RQ?@rksxRuAjyb4$Lyw*Ou=beE& zIBWAqci9mt<@u=Gpk!O;`fay9)Y$Z~U2+2A2N9}m(%-2UVb{?vQp0!Or3914 zWPAHCK%B(|wcZy8*V_&*wtY~)f@~?xo;rDglBu6F_0F9<%jLID4Z+l07}e;6*R;>X zbm`Nx>Hhu|YVtnxP=A>PJWJlGAbp@Z$Ey1KiNi#0Urlo@pYDp#CUG5}O_VMeOEm(x z$w74Xk#n+q5_3!-$=7NoqMBL>&!bV#o*Gs41-MTQd!qMZfi3!A=N)>*hlQdDR-=xu zQB|C5;?`0;pSvXL^L-}tayv2w5ZOClp-7i*Ms>aV{V+*&&(8Rmj)EXnte@gP~6MHGop$R!_Eg*9xl}9d8*Fa=sPU-p^7GdfCQlkQtJj)Px<1c0)H$O zbgi+glgQIxvAHjd0up5_UcWW!(>!?!NWD4zZPJCGDb#a&5(+eWMUxl_QR<~bkYHmy z*AyqbFx~h~{-(^y?NZe6NWJTvi}%t9|2RfybGBkzF5qIbc)H_lIq`wgY^18hS`)t# za>l{i;8jBX#|Su&>u+tvh^pAs-+%qAbAZxEC#+h>AUQ4{)9x_&@eZj#Dh%C|GxQg>LZhsK zzR33Sc+JbM#GeV3z0`B{X{}x|O5m)+XR^w5)!3!!WOxCXAZ!X{hu%w%>u6p#UKof= z$09oJ_Nf=zT;Q|~i)_OyX0>EpMbO4#J>VDKa?3I2`YF{}_lr-LUxrl-5>pd?&e^P zKuvG4TO$};PCn7tjCIms7cz07!S10#)i3|HSP^#3f87x!cjukx<%E$KHT)Q(A78<7 zJcAb9m|Ip+FPuhucj8;N&W81^3BfW)H#zB6e+N=1LzfJ> z#F|?p!^znCa{!O2g{R>?82s7` zV~Nw@rikR6NJcp@=UheB#lPC$vpyd)H@WtbC^)7q93QtwUY0pAMhiJ<^d3=pYbcqXZiKVtHARi{w_FIH-q<^YIt;rGe$9xqs z)R0v8u*flAt%ZBiDBR~HufrzJujt89eL8FWaEX+HV5+EFg}Gh*>8?nr-EkL3%+Um9 zmuJmez1csE_s^S1eyW0gBdW*5;2z!nF>Y{81IXc@XClkx)i{)9b^x01g)yB4Y!yRY z^*vf^r&qF}>t}(sC%Gpa4t-0U^9Z2FjF>{-7m7%&*2a!CdPnDzkhz6VW;N7Ke z>r(gOF@;-)fR0X z@r6Ju7nS~mLTI>Va0QJ*z}2cz-z?#`ab8V~uv@s{>rLHzPAqlOg4rklTJ=Knx5K)V z$rcauJ^d928`!#iE157sVn9lmrP2`I30nU4t3>EByJXguqYw=HZZM45x>ikp2XhA4 zFCkX?Rj%LH`~8;Y|FylsfE{x?Am*U>!JlX9u|J6v=4^;~NPj#M9uy$&dbGdE-~55M zJ@)*SnC|qW_L2R{qObf(z-{(^5@}MxO9GXz+2eKWKFh<^snU;1(be;u$?CiBOVx#U z6W>?nULeLORfP{@Ji>2MQ0>p^Bdf#Kc=FeO_K(*l$V5)pS~lBM<3(3wh!jVedOdfd z_a&b8Adr?&H83=5swFDg+=h}AL3JCi<8E!JAEW;~2YS8FX6;UQp2gBu3$ks4+jkS_ z$ImaBCf4mD&Km6j=1wz?FShOjvSFL%%@xsj{pks>Gx#$t{`E3C92*fBC& zZuZt;MSLu#5%a?K_)R#5BV2RVtmRFpatetgCHHYjaEC`uog{qotw4J3@^`#>;PYL4 z?Zg~v&Y)L|o_|vFr8Bq#-Tov-j^=DxKHS<3joA4}USqTSt`B`W;-m&HqXT+)wH33@ zUstbz+i`b(TjrOEetX5wKMlzIK-U>_=48}E25aPkwiFP=4wic=gL|nAQyPyKS5R8W z1Zb`|kZS$hA24!lceF=ay|<2z$Rtg`yUD4v`xbI+=ZFSbuELSJVOJ$r(p15_SVww@ z+TpA%@jgPJ;24o-nFsc#jg=Pkj{X<2!!ocLh}*_s%20nVD+=qnp}$ZWw?Bfz<8+QD zfz{aY1aq-m%JET$9KP}?dVT6bimC&;b$cr{G08kP*Vo8F7Fokz>HClp?yydmf2v4LL-tvg#!lj0VuNR=*arvp zST@{0w= zwIYq%b6>(jR(ZMHaEbHDuoQ#OVH42${`sSvv-RRtv2$3s#F!u7rNwY0AuroG)(HIQ z?e=^F-{sult3lNqMWv+pR$g|5RbR>QyWvcJV=W+jqStEhQiY|{vVj_@V`4Aw(nhT{ zl9!Ud7smeqc||2so?arEPX@KK(DIm~<3s8LRkK@?rrzRN=Ny6|eo@!TyI*Dx7Ecu? zgD&OCiKb6KZMXYYa8++=GZACvLx1~@mh%D7vqfzqk0Q+wY+ct9O#k|=94Gh1{L>uN~JJh(ZW?t@E4ng+$WS1jd+|LBJVjeWR%OoM~^p^(YB+@H;5bko6 z<_FdXq@BYk89hvTPS)Jkyf-&* ziR$sCx}>c;mkwKSXqDje!mD7qe7IS9z8R7E&`X-e&n)OqTmCfdKMe}3>eq@5dZ01;Yoj5mKM2w}FQHX4#Z=x)#+kwoht=bpv@#K8 z)XQ0x*6~9XInxO??pGbArDzgI%ro`omkTtXy0BKS;EnllaE#c}J`vfGhVRC#PqEUW z6j-=Bnr&NFMEm`ssxEN~m30te6TE&bhEoGsZnd#u8ux~jF&vK>ZoIj-3YqgOcbIu= za$EY_H=&pFK2LwlMW6!s7{@uAjh?RQ=a3(mDkQ$p@oiayNl8?4E%+QH`aDj*)-yH( zk@J6|4*m2|AvrXZMUwA)@;ZSx^%UL-(iExd5~Qn!=fd-0@cmmb?uk*3wP5o+V)6#7 z3awZp6{w}OJA+HR>UwMNVXya$~!E|FI2M)d^18q?h*lQnFr z8=b3*9JG{^(mCDj%5^;m2l{PkYc1b9#jPpG4_S0*o9m^`e@nHTnEqN~zxpB9={lXS zm>Djl^39aAll#GSsYL8+*BIv7&P$}%s?C}I(0-Kqzw(wd-Rn8cwWRNeziGm@WPf%dL%?J{7EGGxlE8x;*%hgE1fBP z3XrdI{^gzp)O-`H_)|X|6&Z_YwOH71wtPsBWWkny$^?aq#Dq7ijnn-u_3h1ZEOjza$R7 zxPdL@$e!dRZaYp)kSz9U-&kFOiccf0#%xZq^CIz&*_9Nf+7i>+$zt!pqZU%uu2?sk z>?%tpXHD5A_{7GR`>7qXPHk_6Ob?%R_H2J<+{#-C+rvJhcdGQoa^xLM1EU}A@R%f+ut_)mcx2Ay zTDGE5{lJlnee~lQ!n=?^yrzKQ2CSYB#F+L)mpAV_-FXK_7uHt>n`KkBEaF%iM<~T- zI-`_#uW%A0e)j^>(N$JjO%8d+y0Om~7&X)Gq&To1&7SYxwNSfA$W3t2GQxI45v`lS z)aP}m+gy&oHll>J4(t_{RM60iS%*Eo5;-x|2(PJV;9U1!(l)-sDyS?IG$Kq~4QDT2m3o%%vN-f1%Kqa8W3T8WURLG;X!xheDS!z(C74BS+97YD31b!0Y%m<`|r7Ol^S zUorsfLw|?zwy#(n9>k!1HCYe;%FP>_gFGU?5e4yQYY#kJKANTo)SH3<{)gT?ppH;6!!)=YxtHa(4 zY1qvM_ZQ})h2N4ku%9B(MoT|tYrJ@FdC2d5l&zFx)YJr>WD|u%Iv*X+ax1F){Y8oT zY>+J_4N|{E%g#S_e`5EyN(qODv+!milz6d^@;YatnRHY zj1#V^aT?g7dIbk87G%wZp1-^58R?Dsk8~lmL zfrgC8;*Ae3YsL2c6~+TU+Ro=JK7SiKe11ZvaRwW9X>@JMvl2JiiLQUlZRz6akF_GdEvciR^p=Qv-QfTTE`E4 z@BD~%$=EBb=VcYDi#gfZs)}87%t=l&4P(eiSHcjB6$Au>Z*l|#+zso^4H8syOq5{q z0!eN8E9>Jb0GSjgoy1!9Yd2Hb#^hF-$b|Wvekx7F#(nV+zzdB!a!nj;Wex*!t{3}@ zdNG`dEA&4@o&mcUNaz2=iUjQ~=UB!6V_Cm<7pp8s6-P?-IsNeoS*S)mphfSfaUZe0 zdzWS-kcF@4s=mr^cs|-#9N7>Ry`MW-WBMf&1L~ZR`@8HHSL+ya8NkmQxjH zVN~6^=kjfs{B3R9x`(J`*y9lzmlP|ZUHNh6ZuTf)>m=iMQW+%c`juZwWk7uNHcoq` zm6xu!KyK>M+ry4s&XMuZz(ddT-VFECK4E?XszUsd?CgxRPs~iIM!Yczc&W#{9L(jO zUOO!rx|J;Ee0a_0bUE*o`Yfe>30P*{^zkrrzD}4XC%kkMb}sT8^V94Fa((G9?<0cs z>TFUfFZw`k2x&xV)m%OarJ$FTEx#O5eMD@X)bGJ`b7yrd%mygo2yk{s_c>x7_lB2KzZ#Sz}q5PHtq>HTpWbv6UO-${tQEr~B$~ zTZU2F&XS>kuFsz<440Lv$Ms@Y*tx)*L6g7094Pugf}aJacIM|QE+Oj+D?d}rRAYPda8oa-eN5VXJQ=6q(;!(Fz#E3|RCn=`GzFZ1PVWEPeI zJ1Fol831yFTO4wi?SuJKX!%_+S~5HW%An64vAnm5C9so>+pQWXxw!k!-lS<#8xuTx ze=bk*cX0ik5&g!<4Fdk{toN^#HNI5F5fbuW_nISwg}f8)2)8Ba(x0eaT8Ou9T&*<= z={AOcDX~A~>61oP7&|L`)QfkX4H)p8eY8!8Jr1>qjhSE6O4H{ot1g0+W?aX@#U@xs zqdbmxlPs;$J<@Kk>evUF*2>y!RNP&bSdyUP$4_~ydnattW%|OHXcQ&Ha|Y{bI)cJ) zQT@G>rTNRP=9-=&Y@(q$20?g@tp+QO5rBO4JZyQj-SbgAE+uB~Vi$r!Lc+wld(f!b zt5)XYy}B^7p}R8BQ~SfTeYCS*%!HhO!rq8QcM^noMyTk7%zBE&N6({JwOYBPjZ!Uw z=E6&KEEyT4VZxd8x365Rd|eP6IT^+9@;)M5;veL9(eVL3!i->|#Kv$z5@YL@9>xoO zZEb!z^na95SLs{De^2jUNeuvw0@HvK_H{MW=~|0eRT;Fu&WX%~k-ea&jt&YuDExJEga|4_|M zE*^E(WQVpHD59+gjmkxvsn8-rzlw~Z#%)QM`~%9D&&i3+arsEeJ)UGvdKW)^5=&XE z70bFSJSJikIPbL=XRHrwUs<}c((KUT?Ny=nj-R2+llGj)7_DcN#1>%hR&Vu@-EdI=p+S$ zj`5A4G!IGSi(J(N(HL!LEO{DA1bdT*|zh=GG1W^T&Zwc?sqDwdt_VSDJOz* z?RHpfK1?%C&C6Q6{niz@?EXg-byC2m#Vk*#;;>T2Lo6qbMz&=heoA4j^$HVLe0hOc z+`baT2CCZftgE1)RrURwjSs=-)4`gStk`Nr!Je8z{BSGi+?DaMO#jFr!?~5bZj-~U zI%*;zT1p{0ER-AuJitp4`o-6-Wjl|S_-rJf4mm7vXc?(EWPF_xZGMuDbAE&$zSZ;; zvL?ORxYgBC+7k8AX|4S$-Fv8dNJtI?s~p7FYBXQDqTa!De8GEtxxQHH`rM%#2%+rB zV`oSiLDCZaQvTdz{Ln@kEnHeP}E!qrQ+{(R0h$e z>eZwF^n(7g`U7RK$O8VmaX>|?K`L!E)^Bg_{%|3er?2Yti{;tDk2m0^fUuEb_Kq=R zI{JnmlfdpGXw~S~TA3PA2psBkyEg`d(u5Um(=uKn6R@`TViB|tVul=g%c%wW?%fw^wvM%|x7(CcYBtP<1Ozyk69s*Y?tn`!=YnOoylL zj$XbjC>2U;@mv-pegy{#KzR_mzxkHv5ScR3QWHf!e3~c*0&YQuRSyr$77M- znLLE=7LVlKM=AB5q^vel6Z&PmJ}N~KP0&xXQPWjZLm$9D$HqK!UUy_dSN6^KS(aRF zbbCM{$9cD8xTSf>DI)gXi`S9DISWhaa+T;)DehZt;*(m?s9W&w?+scbF+Eai zudW~_*^H1rhiPwZ=Ll-5iq@;mt-If02e(pknY+v;8q(fF)Q4O^&2aJ}Y zkPq2%{H37lZmG*$*q7pAqBEDC5$`kcbW1hsSL~dfUDpUDyOA?Eoo>4do%$_J6SvT9 zy{QKTuDI8P1Sc_tk1C@X%7bSl$g~%LFv=0o?b& z?P%863rDv~XQbaqinBppc#SQZ5}7r5I8AbP#|EjkSWP(}HCLBWD%f4g!JRbD)h0(KZj@>C2MkXZLG4+=Fp7OY3uRs*9 zUDiCiCS8z<_^sc)fB-0pB?k}BdqW5>=lEDVOt5)x38n08qy62CF$Cdh` z5i$3O;GYE0w01_rG4scDJiF zMXaj%Zx*czEnvVRySg58lEBBOi$^te8V9AC^q3BB+;p??CmE(rnyE&rQmEi9*H(gy zr-Nti1#Jv1h89Yhwuy&!XAk|Gh`J9fT{g68wwj~IGW;O7gtf54)~2Km*Y=h(`Ms0% z67=i5oAupduh;3+Wt6Myc(UDYLJ;LHw?4QDFtW747h&8H7|de8TnA0vueb){>eaV;yLoI)k#a# zZaOF;Nl~Yj#;IxyLj^hfSp3b>5M%(k5pt$&N2)LFXJcgUdq~)&s(tJy%`*-T@f?!0 zU*7<7|nUpQ{M&W+^?NW|s>2wE5V7hD z&)R}WR)oLtAiE$t`7U+jt7*NO!6hDxoE&s6o)Meh?C zlw+w2=+H>kV0jgnuPRm9yU{l4H{e$zOt_d zKl7+w>E@JE2_XxAtGLnMR%1Q=mS$pv|E9fqfT}@`mQ)uF1}9m=R}}*I z&hDI^_K8HFE_8>5G~?)s+qgtEmqIxx9kb@ft1s1r9}tKa`0CwA_Z(989vs{eqaZ$g z5Rgfm>7d;IALWdC=q?Zm{;~c_?sCmL4|ndLTFKDN35) z55{emQC=BKP6IV86AVKo)*eX!WkhD1^jN{46$~syy#YuJ^dk!UxgJ0lZJQbQ&tE2% zk-vOjvRT`f^o;+t>SLzFvTJLj@u0F4_h(=KQt{xeW_=4EB=O#3N|C*HW|Ab8d}_ug z9>O2o911Ni#4z72SEld0!P`{7haJoLj4u(`i* zecXL6EbHMNH7vbp!-@zo2qxi&9a1_-(gx#AB#KiLRY=E@f9@{&IqRp`4Jy?oxK8I<0r~*_!+%2w1bhqabkJ~A0aY8EPgV5;dAd#G1|LXqJp~UK zmmX%E;9eui+0K@Un)ncX@4VM73*twg&eL31SBX+qOBLOF4_H+r(7!u)I>Z}J zwRHbwJ5K_BhI!oENf!iZk7UNucCXUBxq2cT|8R;aiLu=yH1sMxL#8727Tw?(<=dNd zNmH`?XQ|I8zCM4hGMGEy`du6rN}9&OOC(q{S)SL#Vnebz!i!DO)0<;y4G2|DRkQ9% zDE-=$eF!93a1^%nGBw_0d4M)VGD4K&q!F?5y2^HOw@)thmhkBUV6=*wjRFvE9&}Tlxi5}RI zs8$pnCJ1HGr#zBQFQgpGDFu?MxS(io%+dP+Y`?GMv}O!DTl6M3VzK2XWLx>o#Bn4+ zv(#)j#r{uZjp1AAG<-k&s~Gx8F_b;REyBBIF5{ya^Dv=bQVCypZo zspE0jF@hnLwi>_8b*SG}dffNE?J=eJ(06~L?9bmnN3EcmZBn*o-%4D2)pUJwN~GFQ z3YHO(TWK0%LIVqZ!MrgM#PmJ^Kai2eqv$GEt|jani+hWI90#dxJU-z{6HDcaVb?0+ z<_FW1Su-=j;DK{Dq{1GLNmo|qYZAOQx{LdfgLs}*Vr+>dk;y;}FcB(#7`7BYZF{;}W@nYCtue46K5DU_iw}3B;<$$Y) zyq_NVee&f8Y?1WH&<8BIxsv46TFIrc5;{VV+d6BsGvw(fG)Ifc3s$|et&+@XRTe)A z5eKUKRXOxk6JEP}v_RhJwbkq^R5J7oHu}8K;I%If?zou$IY>sC4H}i@XZK=X|3%hS zhJ>M_MDrS!LYwEzPo#-^9z^9o1&cB<%? z!}-sR^kB4i2w(S~BkK8@xqIQx*TK#v5-Pr!n7Ii4UQ^_l9ajRr&oY#G6=C<*NHK|l zos5I+60zmZ>24O5VM8218D~x+hjXyDvhZXx=Su%LwyT5(FbDH@VfbUXCoa*47a-?y zk57+P$@X%_XCX6p60Gh`?;1ld;@wvj@Oy19brTcZR2cx--J&T~usoCk>eOH9jaCs? z~JBzX5TpW!UYuob#um( zZDrjNEykZSCtwJQ6OV`b9m$+JH@Gsmo-Kt9+wC2`oU@w?r9DsR+&}YCL)Vh6lofCx zGQzv^^o2qA<7UqEmHQL1=1Qt_dFf=pdY<3zIV<>=3?45-p^(;`<0iMPL}Q4caz73X z%g=&LI)8E3!TtAC{j3ICgU(t1Z^%HEF5@8`t6I@r!O^jD!_UaNqF2uwR&vhr^U76(0#bew+F)R(sMy1pH&OvY-2aBWsiU z$>aM}?d((m8FZj&ixaNWWGL0jH$l76v?~r(ZbJEDMFAL~^^9FRRgN<(VJ6lBYbKjO z{GD%$GoRPKYIi981wu+(UhSUC%Ag=!pgmkOJ!|u4?;wPj0X{0Lbc%9)!)~v<6|25P3{lx|!6Nxv_@eB~`yb*1v#NeBt>1z8_xf|kg}h{NH|dtAm&y3j zc3y5yCK@Ae2s2-){hH^r$rc7!c2F5J82EQeL6Dvv1P`tLxXUgbtZaO>o{9A63~}Os4LMl>`nXW>i$RnE5ss zKYV!??jcEeQynHRd!gD=&V6foz!OH-net%R)vJG$!i#;oiI0KLj5a7S@yiyL*BR(t zVS>AsRuj4f;3?du&Ck=2T%-%)d$C%|}HxoBGA8x7~ zo~(skYNi6JNRVlDuV~fCYk40e+F{k*TijcT0(6;ZxmmICG=X@zZ%eK@mqrMv>j|d^ zeakRx-HWr6SkQ!JmQhg@Qr_EtsBgMg-M%}Kf#nT?@AWg`j8f60FxvfZJcaA~sz*dv zxNLY3c!bGKm_aE(w+2E__5})|qx&_W|D1)sH82|XpxGk?KV@}y%3h3xIPe9AB|7Rg zTm@@htPb}E`Csw&cT0&F8#eX37w|`W>DT;kvlx&;DqeiP2xHe%&~RD5$iQc1hAZi+ zmQTUI_sEnm=0}X`uys$3^=Fos;y3~Y7Pm-+Bk)_6Kj$Sii{q%WJh*ZOfa@tX;8=t3 zx6H*7Ol&WrDFkr-{FDf|@c(km-!A$GANlvV{b+H&VM~1pr>1Hr z7#@g;rc=6&b#|~G$Eqg1>$b}PkcP~+Z$Iuz<}UVQl_G(Xi{!w<|Scr~EJM zeS3?q=FC@+C=7jO|L8kr$oo=*Pp7?w3n;6%9*Mj^ss5kgbL@iB|Iy?U^?T@#e;gZW z-X8H9`#)pMlJTwo^%1^7=N{7ExqJN=JQjmo;&S=>i~RcSUsEr(fcK3%Y&jn{JRckI z`~Ol=(Ow61#Op_q_kKq7k4&v!|D1q7=kJfanSTeXur<;m=}MjiZ4I6V^`C3-SqK>7 z@W)=6tAYAhYT>_E<*)4V->bAhGCy$VGE%__T-xnzeEsN>(yM>IkC+;aQoQ5r`iDv* zJ%an!G|FEDymyA2Xw!0XKB=*pI&pc(cn};MJfD~D9rZ-ySbU^dTRkXp`5)3wJC@dp zf2o;(l@9Q}I|mq12EiNC0{)ro?C57mA=mgVO)?2+l_W4aoM^RTEtyii=1}cdQm3=L z8FPbAL}QiTOUi&N(B{`G`{F(kiRw($ShF&vI!1x*pNtibD=N(F?E72m8R9q+sQpr( zPPe60CUsq>SjW|&PJ&x@|IoSJV{QHT$29*Z5&uWfLvWvz>k{Q`rFEWr)t{@^ZzXv_hFrmaSHY6(OksNSNe!Rd`F-kC*^kAF`~aeAFuBn zIXOAW9Tv-~PT9sl`6CnSjxp&KXK);XEpD)DMBbER5g`@dGDZ*b({Syt&RPBi3x1FA zZ-VtXDeX&!qNdA#TYKi2uQPb7SOV~Y3nXlx=!mzlJ$HlA^kV+}i@m9=KwB2O+b61@ z=CN6Ck41+PMsaA?SS2>0PS$+LpV`~aehi4c>9_gw>9f}ibpEeJ{%g7Xo%sT<1Ba$t zNHe$vR(vJ4`ie*%(wnfv+$JW&!k73n zGGLzi_qY5x7-bGFvwm7AX(+YeNNMO#5A~}fGuZymJb`aKgy~sSyHJxzo%53>(|~$6 z%cluxK5$xPNELL6BiHvV9~kWE>nq*6k%k49)57=y-(4)qf37^t%&lJn_3rPf#spM; zOcp<8uLnnc03QLbV7pL8_Qxx}O9AQ1QGKZ|~$YwD+cA zVYt9c#ea|Q<89&(59wz$SPe08^ESy{X6DGlX^$ry_4C(OCaMd;C2E$jl*lo|h}cMa zsV?9S%uv)xq0sSs^gxZZw)uEP&fXS6AG1&uk{}<31Uv1i9nGaW6UsFf6&00*c4vUc)nk9V{6|9nk5&G9+@Fi+A8ti^1H_0|uq<{^GKaN- z3}g&|$p9dY-F7;ejE4(UN%cI|Mv5`svNNo%-`RXP&3c~NQ7NF~qH#x0GXGoq`(Fy& zv3mbI&iQYt<{u;lh?-_Uqo&yD#VT9GwCy&iJb$o;w7+s%8|kC~y=L2%W0Ra*m&HWW zt|HA!qs}Ns`{RZ95EIJsTV<=3BBwtdclPw~Di+mRjub`KBD}S~03jAX!m~0`cj0um zFa8o|lM-gl*Y9+(E}X2D^1m{FbA{T%NN&8`@GUHz=6@J&M%l(^{ubi&+L6(-m$JZB(+pf29h~qrAX;`@f^9@}Y%O7#OJ^Z+Fm0_{*r3=rDK3u}1?x?IXx_1K-maY|yda{!9-3 zt6gB!l7jmW69?YMy7T(4756)A{ha{*7_B`R?afI)$is7fN7ue~bzO9CqZV|IF-`YU zYe8-b)-R=b%)fhl<$`;tlJi}5_E;M4qc+$(K>I5{j+Uh2vwFAC9iKd;^vBc0_AYQ4 z{&(m9Ye>Xzv4Qr>a$Z`~XVR{sUXHvoiR+>4?#_4OSB#i&zfIpN+l{HYr~T={M~OLa zPOfOof~qe*LgYl+2?Mn-{6s^B+ZX}kl;Lm5PCM$FCyJ+R3FoKQJG<4+U^ zB6+~~>sTHA=VE=s%zjS=!L#ObroV)Vf6v_SbS9>OyXP=gcKI9dAMlQ=%w=0^rS$fC z@+^+nKeY+<4q;d!e712w6>Z#6_MDE9QIbORBfSz(Kx6by%s}ME%zl@kPv!=%95*Q|8&})Q~HPJ7N{Fc*4YgdsItyEXiSw(kH2*OWeb8> zUwn?E@K_)aU3Xg`MZ5@q%%Id@n{M@B=_W?GeEi<@$+-KV0_b4*7&zMeOlAdIVB7^{ zOP(N*Y-7K@X&*QzFztgr<9_}Tw`1e+@DIQ9x88No_y|4wq-vH@RRuBJ0IR1+2`#;` zx0LQfnP|A#g0yj)kCZY3AtUmE>SBPH<0USS|9yFgC$@sZq1&Vgd6MI0jJN*l`})?4 zL1OMC=(99}Z7345a_^txDf4%HFDkNkm@wv(dTOabtpaY?iL!Y<0L0$cvwl5q2f=n1%}-PCD3s1|*oGV)#Z= zNJz*66n$3Ow2FFczucc`zuUtv2lYO?T*;K~Iq((x{bBM3t+F14!r&$3 z(Pwh~7)-W<7~D-7Y18?4bj-KF>iwfM5@Wlxyd>4KHrRRWxWS;)Rai(53zmeZ~pkAKUu##eM{<&C{iBs zDa2NwaGJoO>zIF!847$_3@{UmrN>l7ki>X8t4wZO8@be(9Zs- z{&}A240w(CpybKk3jGt&vntQi?GDM8UjcTt#r~G#@L=tkWsU z=x`o*h{Ls)w`w7ua8%D$Q;U6Q;2#u>7yqnb?9%jmwwb$diIQg z(lujTtylO^hl1NE5ChjL4BD`dfrA@uPS$hV0kWq91!ARb-QvZRp53rtSf zA65^UCU17%kAj0;USIF*g7_YBco+-*k|PL7V*4p}zd-KP|2%G?`BZ+ z&q0Q@pLScq6HW#}d4%8=*fpbGtVQbR!$b{glCA8-_k5J6nOJm`*LPr2G@X4wM~o{7 zIuf7kVG`t>ZRcrj*31xZU9cIikhmkCK%4r&DGH!VD)N1o&B<|sish-s+7zb|P5U8b zDVajGV)4o}vpYmRu31>r({s<95iDHBsMBNb$EL={GGMcGSgSv%evXEdNzI5u#~nTu zu^VTS^oy~2P3k!G@JiCgOnp5`a0jb?A(W?=3tJZ8a^umI2Y<{gowVLYvHf>|>WWud&}zMsjJeOuEOQ5_E&#BtzlGntBC#~p;1pe=+eoR& zbn(#<;Z;BfTyYmMp;o*Zmb#_!cl06k~=Hvstc+h3{A>4JuiQ1gERY@wyvJ+Em@fL(+CGr$t0i^&8nj2Tgu zXMcDmVQyD$j(&wkjaH>#$}%88qcgArJ&D%T*6ZL;Zam0Kah~vR3HKf+@mXtS%6NmQ z2R7A(3oPE4!$xnwYt_weZYf&BYj`W{T;o-ZpbE)+Q3(kektMO4(dp@&5u&I%GrmYn zdQ_PM*z(EA;TC5e*nn$x5IMlr@h$g5ZW)M-&{Z4i^jd!TG74!I5(0@_&ZxkHS=_}ob16S;h+o!nP0)x88!m(_n_W@ zxt88D1rDR)7fvnMB}L<9WW;7RR`xugx0oEYhKEX;GW-G5JRnX2OKm)>wb6A0ag`lM z$}t@5<%lvL%$3p7N^-t8H841+(EUmG=y)l;8$d@(Bl`;r^$=;X$_O)qkEKwQR%RyX zHds&eMKeqK(}UB{oi!^>!psGk0+L{#WdsSI!Q{!A(pl78C)yx$iyD8g-Rk^0l$ZgXJwKngh5p3E{-RT$fD}BXi1b z@%rg<#5wF?x>lKg=TuB{>+gz#xDlHyQPQ!^bFSrIF*?u^i%_FGk^f1>9remG+>#-*|-M@C1}1HclY<9)n9+B+~izegEkkEx-SVecjUg;Msmt6;u2 z`&te+TD*{Z5nu++v<&h_=eCiJ zsYdmm1r*VTuX7synkH*BYkEu^92{gl8us1nwO>IQzo9zk>4FJ_Q)jt;CIof^-7C8@ z0Q@k(14;Vxtv|K@?^eQ~_m~S~xM4rSkjApNrzc6Mw=MisD8 z7_cC=*SqvXO-$Uf}s8X2>Z&gsunF;>5@`Hx)Dj05)c8E z4v}t9xGT2i*an+rYXcTOH&E=}M3lqXdi*221NEBR3ll1MZEveHiiqg))^4dGvr2f& zm<u{PnBbw+~ekMW10~FM|lx?|_em<$d$uMu+g!88p^-&6xm}kTA>wVYd+R9i6sY z>gwt|OIHnHUo&nVu}tPEsQIpt{j`kM0d~Vw_Qw>ji8@4uZ2FsXhjGWdO&&dkH@IX{ zeK_AxtValj)IzJWNN5Hs6g+yY)QlG<4xX+=uqj?&W4Ym3Jf@kWMTI6&54s{zzMX>zxSzM4PDg{ zErHha{vzr^fMcR?Q+!R7HGSL-(zTl=RcL%UQZ|xtxOPk2DQA+LGaT z&(duHi?bAQPJTJK037$z374Ukb6GD>^`BV-Qa|aX8G$|-K(%?AyqgbXbIdcUZ^LAU zplYzZhtfV+Z$xmX_{wzXT2A?>P(wq-6iK=cvOYi&S>!&7n>8OAL#KyL$1cdA@M}|(#1(1GB ztplwUM0e=g+AjiLoi2kz9o!V_J7L3OhoIegxYtqF5iyx0vum^*9g~#vuUvk{WBz+NVOWq#| zz!~+jBfOCbU>qsbckl#MRui)owL*3iRi&hI_h5FWk`mmVa7(_`S7NQ6$6Ibc{I(Id z?wt0SSzqiRD^;bi3GRff^z^6!>l0cdzC)eaH`9WY7>%Set4ELE#X6t2NoMo5EvKN) z26NlJwZ=iPxoLG{KeDd!+k~&zpKsIKQz^864QGEpJp`e9BA;TlY0>ikYCfx7(cABq z{=4^7OC(SAR+rmd{&I4J-?L24`t^$g?U_UFsa%7$W`R`vS5pds&$)4UoV=49bmg(o zZ!)G7=vS`Qo~=%Dzgo8Sjg3akIZOhz4Q=g{9LOhc5ub9rYDUQ&8x}KfhC7PM6&7VS zl4iDhr%e>}7BFqUV}Xk4Zt%rS0}%>NqCcf`eGvq-YJl**Q{hhh$Vii=GX1puqZy&UD(=kJ_e;XKGDdy z#sFs5ovT|}#HJW?CXTh0Jf7Cf%D9!|HHk&&jC7h1{pX$`(U+|(?(FE@c!`K_Pv*iGgkGV5Hc{E>DVC>?v_9KuT;5 zU)6d73;h!l9=08=_N1AETqn~|(Y~&|mpA2u^LhM4yoxo;oF^Z}Ty->Wf+ik6YI(#I z(|@_*=&-&h01$Jr)mrVHy8SnZ+EioL8?UA1q59jl#cSOG4BB$|2JE~3meZ*#W8&4v zf&*NQ@P;Yv)7m|FB8Y;QWC=ZfBF$l{F4Y$wMz$)bVJz8fM~e^n_^EoB#J*} zP)t#$+V8GS6$sy&*uf6Jx^1+}5K8UynNE(x%1M^~e!5b;KuhyhPob&o*nIpsvyGXA zOW8DkVF4e@?EMI-d$>8W5?$k0UiflS^5~giKESleYGf7#FFjNn6Hq$kDSGzeCqD9? zzs~Q3DH`i=0KJfjdhfIT9hV2K=duDiHT`d`RxZ$k+~MimT&!^nV8Sv2G|j=nlN|Bq z=F=lpEU9EQF7yYQiR9bkIEt1 z7U1Wop;7}`Yuy~=feqbM)#I(u;7QesO{1 z5@Q1)IuYnV9dV`kksVA2Ew+vAY}Jt!6~9GLu2Jo1#a$zQ=}T0yd_F@ar)zblTyW`YTHHl}u!cD&5$n|<8GbLNDqUGrmFmCzv|AgB#& zUBNjm;I!2v8)(tPAuJh&L(TDOL~6Op+LefqyT^nyN0M~yP~u=Ka`)9#orTo~*Gbj$ z;I(#w=P-YDs36(u)5~kb;#+H13u=W{CFTVs(3SHZv|}3vJgW6n3-J&?mqmpm>w64G!Y|Vk|FkeL{YLP{vCgLS+{ly!;T!I6|ZT7fW213cY zV}|YY@6MekLno);;NgtOm5&VaAtj`Jy)G$4hxj&t$4M0S?l?aUUtrRgdzt5-KC)6{- z94x4eFP}Zv(z&zjw8YYYh&FzkkF-_5@9GolW6{;ELcQt5r+9 zJ-Hkf+j^Oyrdd+Pqe+Cm3b@_me^ttV^}T;MAs^mKhNCvSK)4iXc;hmYJ`7R8bfV44 z2m38xSi$PBG4mGZ*`-&dZ34G<@(xr!r7NHCO=oSxhCPmrkerk`Oz>e1xNaH9S*mXa zu&~xT7+WvXqf*!u)4_IK5u`lPqOnGt=guZ`%Xh>-W5uki5d=!P^(DI(wD1-0T`nC; z-xw}THJ^;+Y3bU%vNg7Azc!RIgXK8U2VBOveZ}wJk#`zFn{Y!Vr>EY07U4@netgGs zDZYdl*eAnAfO8yq;xWwS(u4Vi`nR~mlQk`?gJ`t8IMtFOB5sc;`!iL!ELrF>*geMP zLiW6}viQ&MY{sGU<-Bp48#1^TOzYIlG}r1U?hd%g98B{i%bB5}+TdYDEqupr#IpgORdf<&DF8TZSA~gMMdOwi_o2<*Qaha;@Nd>A z@~i*(x*d184Q|yU%TUsjMPCk4=8+bpUwfrp$2|*{UA!mtrw`kW-~5m$cJSaEsUC9D zLT=$2Z{9_}v>U0nLb<{`!7R^sWg(S{NkCnVLo3;1P&We9gjQj4eR2F5hb!fZc|>%q zuZ(IHh?BuHr@!0wz5#E2p~CJ8sCIbevJJ;jVFx65!{+%#1V$%&zET}Cf*XJUK*~Tu z1lyhO8t*c4razHP=Kyl2GGZ#`N&<>U;(>C?y9-(8&L(ky&zk&7Aa8{8uI1`i zZ3Cv#YL`gRypR_Agnr0WKF~rIFXOnZhUl32k>HgvPNPJ_ZrnJTH+!Aa&Qf?}+RWMV z#*sE|Yj?~}TmSg6(7V`jmM!nzLKx;V%;JFF0~Alf5^bWHKq0LUzM?N?f)b1C?z1@` zjpKb$Ljk#pZS+u~VW#S)R4wE;;>A18GVjuexZVTpi15lA`=w<+nPoo_I~*5n1m7=m zS<>1~Iyi!*wM+^JNGop-PLSt{e@&0%_Jl)#`f^dOfZ+Z2?VfJ|gwsJU-MHgWR?v6` zAP4@Oi>*&8rc_Jx{d<}hp6W?#bzi%ZG5-{~4g+^TLE9txAso?-Waq1o4tMM4I>pum z3Y=B-;PznrVB@j5VvY&nim=N!jkbrc-pJjnXL=K-a8;20%b!TLMi}&DSXY&cC_|UQ^}&tCu)?tBhK;pRwMX_E+;>H+j8wSX1^H zgIF%Di6M3#c4-gaGE0?G6^RMl^Bc4@LP_MT=_$E#Hr=;Ot!{0n;)y5QBg685o7$oWz{iGGKff6 z>U(hw`IUU{66n-(K1H<$g-TjsogwtiChJZK$x8B208W>!$PpX`vXByMyuWm#Ni|#~)qiLIH z9>7pfCpBzPfV*>}k^#N|TR`@;s;QqSxY_eGZRd6G)R|8i7>P5)<72 zd|pt^d~Nh!=}ya)DnB_bn=3iEw-IqLevc{9xC20)EjWQVe358A5U~dj*H~@T34KER*0kTjue9zE21%@4yEJLo1Z9Tm6$?xJ4}_=8e`ydvQ8HP zP7+a5gC8t^7Wh0Vh;}?K=;g~Ct*1o24<;{LUo?HvU$=L=Z&_C56P`;#@4KsQOsBsM zmFK3Ven)>sNUcMBYSzVe5aZ^t+H3kc<5ho|mr98UM?rEC^ZU5B4~$`b$N{nqj&*$0 z%^MSLQ;%0Mg;?0w*jk#yS?u)!mdoWMt%iDdt*)cyaNPOj#@x%`{6T$X5{(M!0SH4b ziTz=DW6;`AYdg^;K zult`}tv`C0Kyj!UKPQ86?WgS_r!LUFth8e4L)yJVs09^lMrsPpCzsaLIw(@RO6wI{ z*IqZ>l%>Vsa3fKu|gs))zetskhXrHeBy9C15 z^w`+sdT7H-k$YyqSoxHR0!BbkQVV|{Cuw&O2@HYwEtP6rqKX$~f=CEYS&QDwM7XqH z#`Zv2nzOZs=dz?gc#zvS_njukdFo32#WlZ2qouXPBgEl8ZS4Mv)+9rB9$AT=^p{jc zCwxj2{o#ahh3;;=ZTd!qV?=+dw5qAL{FhYbwm#PMEd9ZoH}9iWOJ1)wNoq}wBLxPc z2j(SPRvJ8Y4RBJ8PtC|c-jVciR=8dB4iB$bS=*->>n-S$o~~TZIcQMCHHY#Id1~`cx-bJ; z6E#`u*#@i?&ix8;K$lC0dotsk1*T`=3_w}k7br?g?kFmLM=ncSnu^c z=ZGHG7n?mhe9&R|%u1?n9DeJ#f%!&T5`&96nA%+InFn;++o!VG!`F?iU(a!$S{f)m zevS5r3#l{u**~mlj3grH77i(2X>7y!-`88FQpObM6wa)n z7t|tm*yhV>s;bkP)T0wHDYvYDes7PH*Zu5Qe&1k(3wue6D zjSzQC-_}|^m(yN#+0=dN5UxlFU&)j5#|nejR?kE8Ni5gKQ?GEHdU7vlq0u8f2lI4# zRCF{L^WV<^yb)(o@ovr0CmIKxGW)^YxM3_>>n5gaPvoT0Wd-O*Uu?g}Iyg-P0k0wuA1J(JQuhrABP9t6QhRZ6fs`muG&vzuu0iqBC)M34{q3Hl1XXy>r z|Cmp>z5T+q(f}DZb#?~u6CHp7#MXUJZtPno+6CxLr#p5o!d-`_Xj~xW8h?R~pjCH@ zuV!@3(IM)g%l@U>nde+)FHp)lgoscobT+}E?!v`ztMxW4N&^z&IO(4T9RG}gzyId@ z;Kb;OfJQ?5kr*!PE7hZl@JPFPiVe_k9vI&6tmQEzEj2hXkSbgXg-~_KYT8>u_ zyE#&%6 zv?}}zOoZ<+b~LNPaj}}c%q5swuqfA!THv;D6SVkVr=RfIj$zY7LDR@oNwgW6l7EaX zetahM#rdu&;W z%tfhRXNc7z6kNP&C=mAPO)$3CqL+#05#)@Iy?K*s(7{)=$lTLZa)r;y$*J|*mIy*{ zq(ajU1^KBJNi$-+;u?DP#sY8#o*R}CSDR$RG2i3K=U+G8e`a!D4oZ{#KOX_?h2iYo zcNbOVPp*Fqbt$Nnp{|xdDckfg_h#~`X_$$e3dmmmggYp?X*Df6F=M9Rtl=_DR(gt;e)4&UOFOTg>+f z_Z9)GA!Y1^3zW!e#h2^w>n3pnIb`3{Do~uzxdJ$oJT{5smS$!;LU0#JGY6Byw9qTx zymao=Lp_`e@gnYTEpJEZ_!Ald9D9zJcZZj#qTY6hx91^3K!LkFy9o+_if0z~c>Ucl zfB4(g_%ovin`=76uFUH1G8n~s$v6n|KMlD#`>;=i|Mem@a-=`auGD>d%pR_hYVh*o zk?mdZK77#T=uGb_C6yGiU(?-GGC3qm>-ea|W}tOgpjn}~+NxpTUYEFK!p$cBm<26= zpUWtVa@R&xkGD2!rZ=-v`cB{S8$Zc$P)IfXIEI&Vr>}-eC@r0*} z1-x+$w2N5*5@=eqhUl_%E>USF=*5~}KYm9ROXU0o&!qsl_i^)Ew1)v+ZM4Pwqw?Ou za|?=|6QqQf&eObqql7`5c>}U=Z3jziX0pfOqvyYdNsV9v$VlC00VDkz@bYk89T!Pi zgNr{OW%Uv6{iVEe?qa*;sd{6eKAW25RX|aj1CBWP%EAX10Gd*Fi#yMDTTlw5dO^R_ zu6^B~87H!gy$7i5b$)vU!00Wuyz!$+;*be`th&pns8GD?R7-Cdm&hYI*4Gblz;KX30h`}2}akbXcjmn8DIlH zK*~2n&ta!b0Dy?2Z+bebY#hy-c&TD5t8Pj2F8u;$u?Q%l zwh2o3Z3nqcb0t?UZ~OYm2n0UPvX^!cOnEPlO-v^-yt(~tKykHZQw2%O({>V+m+Q*9n<++Pc_GBd(dJepj&>O&^#4 zei?qtFBn&`6bSIS377q-xy5dN)@g{2x0#4ZA+@EkNxjM#;V+|b&xxDx^lbt7ialTU zS`i%aIM^=wx?F4yj;s+!BbG5luH>Fm5Bl;)%Q~Z@%nZAhl1d2$p;h6deOiYj?+2Vm zuzLpF@Gb5XRRqo!8EK#cPzb{-SyeN!vLZ`?N_T;`dii3H5=WJ>jt&P5t1<8iNgdO= z;bK%e?9>-T5fg1h4vGkZGX^sQ1_GV|ogiT_PiCMB)nXVzSVi{L@J8N5*c-AVx^_6Cs=8`w%+La#T0KcZxaj@%KuE>0sXT^o z0xqLj{K238eR=r%-f#@&k2#DN=nD^jzMWrdv}d%3T8pT+i`!V)N7Wx_f#pAiqqmX4 z$VYY{Mntovd$SUdkxMTxD*$+6GnBt`e}T{V4yJ3a#OkFVX|lxUvS&*Zos_bY%3t>gB_sBM8ffGIil z%JDa^VbHxa5Lk1L0NQ_;y?X6iqhb%*g~RvP_!A7Ad!!Mb)iVpG!X@`U@UM(yS6+@s zPNlM%*mKTqN1xMl)c`oq7sAOB^Su~6J2xjn*IG953=pY!@(*P{sJz#%WTOsQCO=D#Bu>Bwurar>6$#w>Iwe}4sMue=w zenOBW!fYRLUagD}^M07IN};Wwkg?xK1=6Q(pB%ZwWQdQ^(<@b}W);(S9rr3HTN^cR zPrExII^r~-S+tf@5cI|5A(%09j$so;tr&g5>YI9FZ~joY;uJUU%`#D{x#QX5*6C?- z5xWw^k+O#)rI9+2k}n2U!ZVaxS(-fz*!H6(7cM5Y#|tU~SWK%{1sj0YOM^%C_ZC!| z76u5m9e_dDv{HJjDTFTgB0N~BozEG!}?22|xj(1J~4mxQ9Vl<${& zk2$SaA>|=hi4mc8(ZR<7Ss&@c^VDQ#PmhCJuNoj`L(W}_d_ST`$Y*~pNEH3>BVav> zys;dwnav9(b|9%+)X=jTe{mQveL}JChZ+LeUayqA8r7A2?4-;@DW{-ts#f(UybEEl zoErhJln1ETDOM)395*s4QM=0{eYmO!#Z;O2 zCPj?cZPPoDQz@(sfvzLaLJP0om_grk+*?w=6gdTHYF0XV(bpX8j9s)J6{y_o6bo*$ zhux>GbVJ-~wB&W6j2At^)hio$(0K)A*HgJ4a?9nxsCAZniVxKdW5U&n`FPt|K$T(< zHR#sE!5}7tKnN3yfdI=hi}6aR+QG}ru$gT*&;92-jfQ#s{D15@cR5H}q*yXUL}GH} zvKw4Z#Q00AKDHq1A?PYM@~*XW@@B8Sw|BSYXt@}_Yr(8v+UL>_^>P=-nr&F!9)1Jg5wpwqDa3GFt86luUT|Hd9lo%0CO(g+k>nddA0@=o+ z6#t~lOE2m1 zcikN-v7c}n4!|hkk}w@vWfIk98}M^4V(k~er9Yh93@&`BBLu+3H#NBA-79~V${xWz z&%V6tx43}_raVZ00{@H~;lZ4w5o)g(KKv3YK~d|2$RR#`+`_x1@-ge)-6W>>`$U5$ zf=5;*`UTRSx^SW5qoT+vM-QGR>Z!m%7@^m^Z>!V(q1Vy*Dvlrda_Uw(WmvHXoSyI& zh;h)|V5zO|@UZ<9)0#VB*A7b{jqkPi2^j`k3;UY3tt;IpaJH*~pcrgNGr8@Gq9#d8 zA_`LY3k#r<-r7i!Qk=Ax4Wa$&WmTR0@+J2IB%N3_d}Q5an1ydzB!Km8Kuh36;H|}R zM4a(s>gdoi3c#6nI%!piXf`4|{FcZe+=^{maGU;6Z zI;&9$z?$>63RM3>I!Mycxe5MR;eP+^9lNRqS=lRpn&EnvXNKJYs7>1=dQJTZFv~{> zP>6^LT+f!U=Az@ICUF>=5%)pLq&5A@qKnt4yjFOx2vZLqyd|J6T#}^&s?9X373ec= zY~6n$?o}1OVu=_Q#(>x51jIM5`eCW9xABE&zwxEq?olUfn0Fj#0QZY$dI&*gD5JO)q`oD1_E|Gegb*A z1yDym1WAF%ePh4+=v5x&7y}?#r&fU@@Yr^}4Omi(hGbiZ8E}IQobV>om`#0H9BDn> z2uRjBkH&frYoEpkyA1S^XK%#don{Y+w^j|l0}AeFWXPKY0tU;lg)fTuBRujZ++Mt= zY1o2RGAF0*Be$J8X(q8%;8$$yA=(`)lJ~f~3})14($ItuHu=pRz`~zln+5?8y;Z0a z9$9WbHt4Qo(;b1$XPp@dVipaMml}qDZr$!V$poV`|9oa6k0AHq`dF=}Tjj$^9BR8c zuVpi11`Lm>an~_>mH4M6!Pt$6kb;RWvhmjva$Kyk!%igr@y}lc3?U!i>c_qAlo@^s zNF233Rn*anMaAY#uxfyb&?oLKnuBe>?{K>_r^VS!vs%&`5xSdVw@c0U;!w-}bZ=kY z8}yj!cRwGeqMen*z3?f|eO;!w{MKPT%Ce6Cd<8Wk~RfiE`( zAK{UA?tH0zboHq*kq!CsoYNT&?Gp1qvT9@Xny!^vV{k{@ef8y__veE`0*|-xZXh0fnu3}Tef(h)U22@>Va<*nVsyN5X7f(xaVUgOC2-N_ zc}ewKKOkSd=lE8tes>yohNfQt;j1|kxb4)qBzE3eD}oKv*=ZpWovRYpt6Wz&;}bJ1 zyOL7%0acUcYaddgg3ZQJ)GT8Z!YP5w&gzdtEOar=>Iu==I7v3X__xk~4^I_?+| z%-rsASertoj!KF8!9a`ROqFPaaR;uCNN9oD`eqjr?6UXZ6+U$fYW*6^=>Q|#I@3kf zFX$UHbYiq!?{hzcQ?hWaj|a>gaf0RCeGVy84;rx|j#|#f*mLTId+dKDU*S~KvzaJQ z|4CXvqRguVw+ZD)n+?l|Fl?sAdq4&j8vG%uRyNp_gJSoAiyDQRuSnNH( zIkriLIH1;QW@uuQ-9V`&FiO~~LvWD7=l$&}n!88}0^$#f7`P*@Bnsxlc76si6gdN?ak%75Sq*ul(`?hkkDvw?a z*|B9t6A=iix4>A*=aDREKiYqb&$2_nN}>3RmDcHweMIin!+oXGqYwYMt9D}P{2fQ~ z_rv;ybo-t`oHV3Q<~_;Kc<@M+tx)K;1=f70>jpeRF|ug5`i%K7le_k{uN zwjDSa^VW4HiE93y4_bDZu9>xg9wgZdr68{rlcug`csVc8P-qIbU^|J&q2<)b!y!Yk zpW3>*KK_^>nBapOoHnxLAK%$6FtQ)z183#*=Fie2)F!D@k`AJ(Yy|3~rIBDSs`2@z zf<5{*g}^|r%e-`?0kUWvFT!0{D??)LZ0|2BsJQkU`|~eTW8)vb{iYAu5Sr09={anp zu{F*RV*Hc?FJNl{r*u{e3v4T@eVghjF1Gvk%g&A;t(1Rn`ba6(s!;3(9HvBxDc8%K zq`0q6Tw@}MMjS@wCO)MjFz<7`^sKBlja?c!Py)Oo;Ny-JNr$5R>G6)S_dzL8NZvPZ zz0&35bY=HoGU0veee`e>-|aPaaD@Zsfj7@CO`LZ1lmJ{NZvb*feq=hex+K`M6wj3%^JDeD_=DCCdX`du_vP1#24X6erd6q{xn&6 zkFK=5szQW`NBP6tsk28bMYu#Q`ZYTE-l&cil1%$tKbaE8#!!iGkkz&@wepaDOz7h4 z{?aVhZ>%n;-B`qr@?I;MHNbGa4-Ow#e>Uoq0-UyuuP&_JGKo}PNr0+RH#|p%aOp`S zd>UhvhlkbR3CkI;E4`xNbT6+d3)`22B6}n=INi|w6=L*rF-!Thmx|4&lUW72T3v)- z#vP&oQ9$-%1EOC}j(Oy?f(@*-rg>Dgnp%*44ugunqyT>uHjT~Q)B&B(oHw#H+ng5F z5{9@FHo~0q_y2wSXk0>dBkuZ>6wIfl(T9GRJ^wdi7!nDGRj4V$;-Oa57jlN zW`~v;5g!jyxl}2pc=7AP;=7Mmh0lPYCT4#MC*i3cC9(L#1wbjuIz&xf=vz%CeU6=r z|5YackTd^f+L9;9SH;K}@jtp81RZqX;*TJN=SwRB2+R~Ac*`%%;U@8{Ud63zDeN+} znwg~9E2A=ADGbv3WpWU;5!3Np=uSy)aSlgES9$HFVVg;F)0n6;BigMYbjwzme(ZcT zj5zRk5YtN!(D+!dcD+zn5tX+A5j3lyE&DSfzk(L zAwL;dnk{^BH408l)u`1Ts&pp7wC&+o8(CH&l~RNbEwG4ohT&-Mskz~KatnJnf5@?T zc7lt`&orSwCDRSJNl^qHYv5jJcISmWbJzw zcMtp7`hB-cQD01z2;Vdtmnk{H`Yz5_$5GuB8kxPas`$ z{V~_MmYYXl6FpMKwZC}-UIT@klN?DqummD*@*qRh#vdvW&fDZs`rVb_Va}R{gtwXz zZ_tgc4g*T5n>1U-*`zb4v4?%o36*WPHDb;{4hki5x^Vy>@Hk>2;o0 zvfemE^-ix=XAFMLQUqr~X1wU?51PfIy631HO|+tMS`~L!iV2q$tEc@ic>Jqd^ou?S zowy;ZXw!4m!LB^$W?H&Tj#iod<>-Ct%2ID{gn3HlFtvJ)XeqCay)R(oS#g$fJwxw992pK{oopc_K>UX}Zn^%r+W$gs#N+_tbt$z&=huXZ2(Mzbp(IsChG zj2YFeT37$>9rv1f&`HmxjEMB2cA`I>KHG|v!pMc>*s6nldEn7xJBJ0V-B$d*=@r-- zvQlK?PEEcxxiTVHs|=2oeNJQVW^T6tfxPUQ7W-g9fpA%Zg+ zSKbk1AU2gE_VJF5r55cZ$$<`e^V$&!wt915+gLJQ{uZKZ%CpBE^Y}YC+?KaYrecjp zg-UDJhbQ{1N#wxlyERF-I2}SPCUR_!ZoCCbkXj&0+A{mzA!O!!_*T~^R5u2=RoJ_u zgRb%yrkHP}C6kjORz!(fd^_rxqYAcyufaw@Z7E8fm5GNZ_g=3Olh)b0QMb-D*gg&? z{P+!%VoqdpL98uk_I#VdZJ8%#S9V|af9r0X37Y>10a!PM61~dI#5aSY%*^A2-@G{s|Tq0z6s&1WqubW4=KD&uRF#8>?>sS+_&j5J4V|bL2hrzp#K% znzJ2K_ht1U#a4NtJv>O3J{+;goqKfJuWEXfS6?^U0uqF@;k~{1+Zm_P zqshBXNW2nlgxvB=bShM-0g^?MV}KE)<+_-%uf5~amG2EXCI0b&LW7JHO^u{P{xZ(v zAD$3Odut2OUb&n-MKCIi`mtdCCC*X-E=H#~&Z~~$6pz%d-uI(Jut|8y$?xiTDVYbI zVFLG0c9StMLnj!1f(@^yWDQ;vR`7?hce;GPKPHVl$=9fkY~X>$C6$1}1xf7vHKa-| zVx2jLh5ki<`rFj!dV%Ek!X|}U2o1>Ow^Rt+of76P$p;JH+Yb`FNGM4X@S^7D;>c-` z?LSDrn!%&6c;ayd;-Wt^QN#;_>AA_Ofg&R=a31WY8RIeS=Xecq#P&2d%wgOw`d=mN zo-E!f3*qvY^ChF1{2 zjlMc$Cd%LzCwF%=%=NF5*x<2CV~J4PK(QhZ6nVOqt}eGA5Zh9;mF01*uMoue-vr?{ z?uXbDIsf*oLQ3=>Hv*FP=<$gx2*>=gV=8TlU7W!E!~3tn@)Yr34OhT^wpSEA{GiSj zWxtMt%{f_%YDoH_{4LHhUVAv2f>JRCwP{(+t{1CQUpF~?Cq9mCbHtJkCuGqCkxV9Z zirsJpi;YXQVfrXdvtmKl%2vWSfRD_S3@58P?)==F9BAU_j`S`y_|Ks}X>rMed!BJqs&JM|(3Pk&h zEY_l)WeBtHwM*-{RLZ6&eKP!yb%NIxX;v@{8tlxj6^i;w-|zhSBUM*|`^^!r>lw+& z7vW5CUE-8m<;!dLH>8n(k_hlg1#owJ_x`nt*szy&BJofebML_5r3grhc~kVrNWDj? z)u-^*ASq*@`c|l^Cko_z7lQngm<4X&w-@E-rlzOc4Y_5po^2{`UKn2L7VqqAqTl4i zt=^ zgH3~CWE?*ml)*#Zt>wz0ta?T9rxxW{5SjF0agBNGxE_^iuk&veOySj}-th~Sg&_Q-ZYPt~@sR;PnNz(>cs zkKU({ihHJ__!p7c_x;nSL>3Gc&@(PyKry zy3n)d6)9pJA>g^G47uvhQ;aEh+mjBo*TJUr6vvmSHP+>a4tr(WR|bbFkm+BL=*n<) z#dsaIehk;T2*=(J%j8qX0)oFWQScUk&-;;NjnDzuz?s2B5z{twk{R)!Voq?tLHx3W zC%hlT6Q+(Rwzneb8~++6QbaM2d2N3q4+wK7nmoHNCsyG8S2h+aDGbVZJ_8O&nFymk z+LJYZny+z$aOzQFmDANF6~jf|5OomaqD4-fauay^nl6=2vT>?4o)rda)wbH!JD+GW zb!BT3;#Vwzb%KOV z%RkYjUY!4NtSpXPyFv-$Jd(Tw6v5`b#B^V(#*<~1^GYMmAnO2JIHX#nb^i86J;%CJ_z?YHb^K4PQfBHcw4<#p>WnA()t50 zW4c|4kQ9rGpGxZ|$2GFUHSbg@mQ_R9qu9|0&t6TOcos|(XhGNFK1(WZ*WCbTX2Tr< zzUEA@r9iN|Y`*E8E4My7)VJ)2t*-Na9~&a50oUh9B)9_a>sRw_>(w6vc72)4!){q7 zE2G_5b$dje7tra^5`yTH*aw_!BhRI~cP06#_cmbW$govbMxYK+S_3kbb*?LSB`kw6 zA{kr@6hC`(^bDQqc&OPk9S;6ORA_g=Ae^0x(^w-sNO}8Ct)`mFrB^>CDP+%qE7DDt zB8bQ3`K45X&oL$3y=KVNse}6$3TA-rnV9~?Ir%35@ra#<+_NqkG$72S#>tHIjKw4qp3B_ZJonL77udns5R(QSMhf7e9newKds)-1n>=gFertaHa6E>o{ z!us4=&zFgCe|B9o%TgP;V!$hpBv5ZmdiE~9&SK3#H@2%lWaWUU6?twgKF0|i#;jY0 zq^TZ)?mX_Tci-9hE#r;lR*Z~ccP{>B6SWzP!p4wz$3fC>D|u$&vsj5?OzKRYzE=W@ z&Q|iJq3YnIw(<7T7sC|YKqM=nI@``Td5DTdHiizhx~yM2TF#N)lr&B1WuFTfN2yIJ zH^Ly{88Ia|H(57Prs(KeW?$=Ijz}^m>b$%Hc|w8j+FGQ-uS;$}X{UPw3wik+(8H+& zPC4wNFa4Sg7(wn(Tlb2!TOiqMNB5Xx2aHicKUu8;BOVYG0VTBTg{!inVx;%aFAT?# z>BmJ@k{0*v^?y|dXUSwf_mjUOH1dvxuv8jhQ{1SAI)fC(DfD6sOfPVaT8(_yS|KmefPiv z0cTPhskAm>^(=bxooi^Q>STG`@X!2>m8jmjT8EpJUtlKYa%)F}xMf|R&n$=oT|k}Z z3)){(@c>^p5XlfHu`kcK?;Txu@EFxmiikjXfwYO3$(Tgb`54 zEp%_#Z>0wL%d@idATm1s31|x{$BkPL?Vpf zk2_umrC{*W*ZeMgb3P%)%f{_1Gsw1{SP&E}sckaMI&9TY5#IyYh!dA6S!+3gT! z{`m2sJvw>CxRwjh6y`SL*|$Fvsr~~$QGmKz9F$4 zdP2S|KN!3X^2Ko)@w%zO{6x_wN5FLH&1nWvtyXsp3!`qiZ8&H9%UoU<3QX-oZN0v`-W+o4J(B0G79YDn)%(2+w(FwK zH>r~_JGKJ1S~HgG23VjXk##_f8bpP8VP54Wc^z5^nV|MVZ={IJ2*;yaWNAsySC~$L zX(@ux+a4Vjc`oamGbrzz==*u3mzBgxI9@;}lh?-9(nN2&9DmU4wI*IX<76J+aji?Y z#wBMtPEfH7ve)fEQRhU-^Zm|(dp)=Nhl%=0o^s6PtdYyP^Ix5bTtKS|fz$Ns1{Q>H zB@=>d+3&G@lpXuGY1c83Z{jt=NZ}A>E2Iy2MMG7c75{=;YS@vE#%wc zY=Hv>QVeru*9-~Iw44$7zJ{J$>y@L8ih8uuefo0mOzm#^A)Om$^nF&#$;&J`lAyvU zi{vD_XvI4B;dI8Gsk$;#4{de^m#E!al?NbdPq*5fcQU8FUMxA=QS06m(wq?GDYHL1 zQ&;%2e!YB(d#RNZ(n>qYwWH{?EV3Y>9?`B*dWmlbMM4S?Ebn6g=E*aT)c_lTaC~z$ zw~0Ev9E79J={nPYVF4Mb^^sdHg?H*7_RbvavaR?L)I8lC8!{oEdI!2F$5T=25Iyx; z;%J^wlh(zEmb-&9dX*e;t7ty z8Qw>S(ezKz{sZbE39v?^_``V-DI#zT`6%WlGhbuxC*%Ci!?$2^)E?;cZ*vEI!hV{! z{rtAAKj{{7h6xhV*0dMSS)F&zUVQvM$#iNA7NZQSHj1g4FotS1b+&_6t_;X3>NQ!P z_8Gyte7c(;4I!;q-F=ePzFWr1)mFY(U>ce1QRw<)OzG51CcLfALl}O zLIhL`E`}z95Q+`f+rS>_9q--5Uv@l_m+@FX^xf9wLOz<;=QEZf$jTBMmMCp^Q5B)@;B;EMrVhX)L81d8s?*8IqH6`uHAA+b(&Lv_4B zquN#a)sMhz_r-?J z**tf224giV40@;YtJ7V}@0d$BE?b#gpPy}sfW6+hJjyh*)aziZ-ZjsdVa>01++gim zy%*v9Fml~Tj#J>yld3EClX`?KYI%YSHh-G8$qpvTZbIZap9MD+^3+ud-?$niP3FX! zk3p{QZ}ItzJTlT%mFwDS;@zc3P|tZR6}Ka)JY88ZuIf!08U-7U%Gn4-zg1bBKZXou zC7-Xyb^XsjB!loyQVV2{KZhEmvHAi7^CkZB(T;wWa!!e%U(3hV$+^OmTu(bzBAGWIztw|=zxWL&*N`2(hf1xS z2eRYBPHL8h;{P@-9oRaGS*%WRd1lpkEk+;Rb-<2ZYtva2r^CX;ExdYy6$)rsZYQ8h z0!xAB7@Q=KuMta{_UY)r*k^J;JJBYRN!~Jg_w!Sa)9#9LVNb3-Re;Ps(V;`ER*oPOg+QEPnP@e`^u!{Yv{;-wA#%uTgNlE$Ue*M6QAeH5r!vBFi1S z5#aNsqPf9wNqMI;4&p_#+K6`bgrWP=bX~gU>TZ-(+qqP&ACYWa$8q7N^T8ZJ8^b{T z_TYCc^2-WT$|V)9ZJK!+ICQvs(|7?nUf*>JoEP`46kajD@0Lv znE>0+g-Tz;e1H`JlAZGGWgU!$Rw8D*EQ_(S2naf#%l&?3uc4^3fce5r>I&l<7iB_9 z2vhoUC4!S4sKr+u8Qz3L*1?kvp6rQ?Kpm_=Ec`u1>!D7!3ZEt{+aZO{n)?w4KR01F zuo~GOHZ6j9I%FhUcB<#KRDQt-EIS2oPDE?^4pHZ${l)sOV^$zI8tN)SFGxf#cwqCe ztf^KxQ8i@jAnKE*bTL!>960SQQkd-2q2twI(IOzzE0S*EzN;6AOaZu>B`|RR{P;Ui zjI7Rfc@m!QX$WsNCg(q1F9ST#DE=BH|LM!Whf3e`xQmuX)ZD4%x6ULneYxHc{gO70 z%cU|MM83M6_EW-BI)nGM`&Y)a1!$hQ>Apq97p6d%Y#^6lsev{RpZ4^pU_snHma)H7<_9^|0?vzE2y=II`7O z_49XaA?^#HEq^^$U+;D^dTr@Ts!UtV!-YIY@JiblHhnl(%2y@#4dy1}`{wwdm5#)| z_H>^R^1WuYWu(aP&zr-Qt3>i@hV1d(rr~9(eSb>Tq-q#a(39bYleGLKaEs<=6554N^&cO_bS~-9eQ)H7=jZaweS}ZSU zq(wN{2-O%>w|7;5;F&Q?f?q_fCmR@sVeD{?7G;ccOpa)@YP!=|%}48Hy`0fhE1cLe zs^u1=$k(^jmBm8$#T01H+4%5m&abN%wG|F&anh$wW4IIydUV{@fFh{05a#iTDsv8v zV8V8=Aq?@E32q9GM1slug$3N1OpGJ3_dTVH`2-Wi^^pPdrFs|mIA6J7ZImzPuNkkD ze22PUJ^$7VBI=V=%n>FJMV!Dm0|{JI^KLlN!_BjQ=Axh2c)j9P@kCMJh1NAb6&x$U z+qOP~pe_LlXq(z!vkmYgWni{EjYM_0_wGQ%U`OZ=cr&c9fGB_{EsKT^WM;suI|Ec- zGACM2>@s*M#_~qP$}3)S)~w|JdVl=aLxH;Zf?~avW5Bae@Nn&@hM2`hqVyWh*pFGS zy;t^Ly5?8?1ZYJy0M@^$MP`-tyPPA#6AvG+Yeu?U?U(~)&Jq4#ZsD7Yt*8GFWA7bL zb^rg5b8?EKVJi`3k0N_JierQ*BbjB7P)Jrr3uP6u6SDVCRvKh)*-F_fv#h?47wM|& z{eFLbzdx>St~$=^`Ff7WxIZ4x`~9c@ug#v#+Mm;W2$lK&FzSyXjmlds>^7c@7I_R9 z`m|lPBd8MReS5@b40P^Z_n$uA*jTtmMlRcJP?58twy9%#(m?e!lD?%56sB9uRS(_f zLyB9;C z&BCVo)i05xihKZ=`Rok_dHElLrzN_o4!iqjLi}psE{t)s@vjM7CGMD)&;ocb9v`*ahze@fYZ?k1AbLz{yTgaB=#CWdURGdYo7X&;C-rMvzp!T#a{qB^3{PR%|IRo(i?IvA zaxU~?ec6tasb2#aC*pQ(ddP2^-uk`kRwHi^=xG}iCdWbX1t9}@0Xkwxe}crju;Rj3 z4y5Ui*oatq6w3A~f@NQCX}t}U4UYkKF_!vSHVu4nHziiWr)f{0xd~T8Umf$%MQB%{ zJL&}x;mM6yFqUI4+m;vOjB5P!4QKR5d%C&s5wk9mv2<+!G#uy{?O2Yq+fbZPRRNpJ zMHBa;_$x>JFvBmYM1>i+@g!ol9R0}qMYxLEozrdE6_+EMpK(tBA^49-qh68PYJ?D| z5tt15pU4O4z3J@-o*#2VML?YRcq^GqjqfhIZAaEz{Ls-#3dhrkcetXGID$maad2sr&e}TS;Q9eSkrX{`8F2xJD4D zCcc-g)|b_&NDq8Sb5dt5rWny5d~X!^Bec6>{6wnjfFlSsv&&m{DFJjdDI{Z|7Hn_? zh!sd&QEN1+9xV(O-#$HP8~0^pqIms?hBh2L|Ko4-Z{m+!rUCL;bM5Jg+!OFN41&No z=G(X)s&QPErRw0{dj`q9D)2b}24i2aXsYEq$NLMqS{^`vxj{h@vdZBDkrp2oc`sUI?SQ0!QpUVMz!1(=azUoA7g^C3<(6o1_LdUg?K`{b(9SM}=u=u&kP_c$ zdCc5qzGQB}Z~DksCDz)_zD!@2R)M&MI9z~FvAri>_KrMRTN|zN`8`IFa95y%UbsbV zU6sG>rNA+%L2lJIPSc4=%Cg+=v)U6LbN~o=otgQEAW`G>Gv4NSfu!L#RtIpS)a7{M zJEle^%5!fJQeW(NIn8S!>{bH3UfG!-E*L@ucgmn-Uqjf@-hDc=dsxstMp0yfskQqg z(OEm4f+M^J$XW45k+%8+rdM(_9OOg)$~>kqIlD3MKNu2%fs%Kn?cc=$wmFVSvLX33 zRKj&~8qAc}13rgeYqKg*6-M_XOXX`wigNPz?YM{%b0Jw;%kmq^j)r2N)D27vxjzdraEw zzS}VdCBji{yZQ-m|JVE4He#G)0zv4~wxb&nHiX(eI}oZXC`W<5c(unB>S2O^Hpp=@ zxQQSWw14|DG|55y_y66p{PPRDqy2a3GVL5>%af1C%KmRg5(Mn|C&K?%(1N@@gC#;0 z68VZ_fu-)zza|6z^-q5B-{12;EF$GRlx$})+J9e0L4NBNLCODWI#5;cqU6+6zd^7o zc=~I^+tVft_cq?1rabT$xB`)!q2!-$wLPxiuNKFFhVEJB(n4sX1@$+r(Bx6we?EQv zWYuRRjh!cgj!{V++#ggf48r)A|2!(&1NpOy)8`WY0kUsRyg&i*(ISA)tclRelByyTZT?1b-l6*o%=4lDFt>C`Xy<3XLv>g2ZwTu5laPBigTvk&0N}CI% z!D<**B$QX-^--~9OP@bKv>BhMPVK>TBS#P=fqg8W6*>}its#YKl{ zywb>rmHt`2Uq69VxzBzYq6y5Azg58qpLmY#acRu!0#)p9gO>{}nJc24B-X!{&-hbz~t3dc!d8Y zjVa^fdD|x|YV7RIxkLSM8FlKZ1ml?`z0mgIKepiRJHUzN!JwBUPR=P0)wd}RjkT?o zIJ7g6H78w5u!~UoSI-`U1L_gnuRhHJ*7ecAb1tO+^_La*Y9ZwDyDgkWA3c=>|7F@- z&N_PLpW*KOBvXmUiF|&GtOa?*oR{<9p>{0UAn|&=&$B;1ER&7Lw~gleI6=+&^CoH4 z+ON~cAW3kE83o#$Er+Vgr6{_M2522`wxj_3Tzwz^@8$b7wO3sC&kODIu-{o^hFSVO z2M6PC^veuU!;_oWO$dU7lM)bu(Cs-U(Z|Q_))Al`5A38oPvp*lBq2gI;BQ+i2h`f9 z;;}on<((sjxK-V00U$H~HrSPIz`eWDRd@d0Ov679R}ql^x$q1hf339G`#9-4GD7YK zoZ+jl2^olR4~urjB1|0zZ~EJCenPH~^;bVaIwc6G;*gRlGGg^y&%O7}J>?O-8QW#y z7K){_4g*Vj9Gmye4APfc#jH}9i~kb5!ZOSfT;loTD?3jNZ^nks9@=J$5^@i$U)&0- z7<&=bB^Hrri|6}9vH&Uff0pqy`5S5`Wrg*BJHV{OwuZh0)x}?9#3o|Q?R@t7jwh|( zT$tczue-Xu$MIx$W0sWT_Q!HhRjuuAneENv^^v4y^(Ayj4lHeQA7E-#z9a8I zT%dS>p_SY0h>(qLTv7)k$vxD+cOrmIh{RHhyS&8j=fiBslzZ;=F^}`HDYY$R*!0vD z+O+G>pUBm41u+5!X!J$$yT8v;2sv!Uf#|(KGP3jXFDd`aNjuG66~6&?8dPtj|{)6jL?9P&eOg3#z%?BT6H7@tj znT(gg-jQ=yBnY~s95qbJc?4d5d*lpl1S^@kM=UO0m4sprp4GNDNZAWVEDb;A;_o&7 zcCu=clrY=Vl)|DjEp+@ro2gmbFbPVw!T4VUFK5`RE^W`}H-=&UTQZfy%)oTYxI4up z9;l36shoIeuvF$HPs4xxE9r?i>Z`7{YsZ^Qu8$b^dns484mq_G7x!Ey+S_S0yk4>- zZt8AZPCrDtP+&wOjTv`OTP@ZW@|JB-|8opgz)k*pHy!l?jLT1NlyuXZ+WndC&>hOH zUFTR!X=wV-!k=b`?W;7Hx!tpoQi3Nzs*1naf)i?f*X@E|bUQqucrfaA?MS%={g85v z(b>3Cv>W}?!9blk_qm$lvD4Yjdzv)|riM3GP%csvn-hJ0hNK3hpeOTdcaZM?JtPSI z9$@33FXE8qPz`a-%Om62D53uWsdBu0$jr~WdCOt;3upM(MNRP`1-B2K}#9s@DB`j*s9fXV~MUA ztfS-5f3iVyAX*;xKBlFU|MJ8P{HR#sjO7zn%}FMu_eS?W^2S;dkK4;4BYJk0Mn0bY zhNZK0w)lc(@+b@~b8eddl8jXJ%E>WgNIq`(rvIi{#aa`NuiLM^J5`b1x~jP>p)s`Z zUDUKNm@N+f#8h`SwR``hYeT8*XYq)CMF`jeLYepCL^QsS*1fbDNbFH-rk5K|`k{}L zd_}oP7wTV{=~=o|Y#8>U`5L%7p4Gu6fBPZjp-W~u+=tb(z2-*R)~`>M&pEs8TP&L| z=sYB{{6iMcSB`0;cM=AMpgf1Qvo1;2#Iq3cW;CEyoxs2~l-!<(nw%Kr|(nrqrtSb)pca(@C zZrSerphjHr*I3gS5Sj4Qx1M@wcX2iS3MQJ&tAw)Ukd`+QMeD;)63tq0l%D0gq(^m4 zP!)%BiFcu`yORUmH+e1!xl001p3p$|*dv>wJtdq1InvJp*Ip;@6)k!Az49??G9|0x z&G_c?)zDoTz1>NJgpM0;dk(P#us<@_w$cXIRBEO-C5HsJYYo3XNP_i5_=Avxm;+R0__pyL;BCwiuR*P?YjDeS zE3amXPTrecyYfi?OIERqb!XTO0usAZajzLmC+yEjO?Nb=6-&wqj7v-5bvCBaeH}%u ziH_;KnTp6J+)*Gzj&}$VkKS)5;#ET4vU<^dNN%~zmd5EY7o5N&3CZYPEP%v^8ovvA zPyU(mP8QX#ps`+bK?Mj=I!{%>H!+B#T<{BRT#lFa9O37Y;ZEs#*t4t|*x0KE=IS|V z%ONqx_Oa;OGDzRi`$KeDW>@4w@RU{|LXE5-`g$m z(kDtMZVQ}$+RjMw6t(s|*(Mf`jE`WGMEzO)TC`w#@-#R8vRZm&z}8wSKI7ziH>uRN z)vji$%K7KLcYeC7jFoybPuD;DFwd>;1}!iC-v2y7hpB`ifqTPgY_an@Z{s{~8-8~E zR$vxE!w^rO2Z2!R4I5J6Dn;{`99Nz9ku1(cy-ssl>2$q`!liXT%dd&VoBFRUm06UN zV@E3Ziq`hOEe4bGub|~^HXf&36>`VBDbsvs(XfTY$g)YxKrjr-+yn@KK92JA!L{@#&dyb)z~Mw+uH{i@uTMUJ&(<|2)ZjdCQ3(c?_QAn(>YVN~i>Hj}|Js--IDD1ea`ADpmeq6aL+gv~ z9y~J1R}b&*P0)wv{hhI^odgf5KIE=+4EY}5F-x0Miqv~MeU`M`?(!a7nyqOxq*P~^ z%XPFKVnq>G6-tSaS=GTA=b`j#du3nC3r%bz9--B zuyJQyK7ug))%vq0+inh)oC>{kmL&h-TY*=E)G9TfC7L2hNLt5MKn_M$>hSV%OEjkOe6D!HV(}IcZEH7HFMHK=MLHMq zb9hQ9O6k9WadBrjm)t!%R?nGo@Uz8`5T3zGCG)vvi~1Zl|67vg?GrpJe>-rKCpHdV z-9cl%vBueA*Fk22J^;q?E%~({sgD8vfyXUl6~)*xH+hp4n(>*c+ld$+?%xZ&y9Kua zQEVqTyo3s~w&wG0^};x~gtvxPDV=cbbU7Me&%^S^VGKT5H4PKqg&c%J*O2AygCJAr zd3*S|E!LI$4Bw-0cj38*ozls@1WOW@b2SmOQ&fA-84>+Dl_brm43K7qam*rE z(fR{CJi@zLoWJ-javm%f(fE3NsdM&+)y=cW=Lnx-Gb-rmPjSEB z4$c0O=JZA+&@Y{aW9n&yuiTlggCxt9!SAli(VQSjMyP+SXG$#{6rJl?M_C>S?-Y0zGV$>xDX%?fJRD%;H`jVO^@P8bc}@_zfFZS zPuW4-NO>*NRpup3W|;cN6d*6E0bbkRWP33fR83G7)CyC+%kg&QCP-l0K(HiGEfiNR^t~a& zpdhpmva#Nz1~v_ElfJ^E~WmIj@TeEVao9D*qDTn4M<~1KuTPi z@i^A_7X>~C$FS@3;<;iI4F}z|DFP%v|q_4XU5orP7MR2_@n8D|N)miIGm_~&;D!_H8umP7K6 zi^td^rI?o8C6@G}Rz-Vu04;{cXoNu)(J=wmDkVgtK|Yqle&>Z4Hc-*q;1-@22`1b< zScUL`8QX7*1OzfIyPD4NXW-!}B~jr_$TVRyh|F<#L&Rn7Qu97x0(?-zO*w`?)V<$b zuQk)fl)N_|XhtOq_JC17eIf_$fBsmT$~FNx=?m)W6pZOtj}X2Ms9FV`XZWwu4QRZ1 z?MIa+2FqV%7*tB$q&gfQ3JiB=fj#&PKg}C>oCe13)b6n2M9~cu<}8|nhcZ>PezkRZ zX5;-4E#)GazV%>s268(iB?WIYO}=XN&i>6Dqv0wq4Tqa5Fu%E=*LD;P$vdILjjv8U zUUeU9oVvm^q|=a@hE_ax&SbF6xecTUML-$j(ZiPPm!mWjz~~EFw3^Ef!t{tovF^!x zp?l57ChHNcNS>~?IDJZUjME_87V#+CkaBfG*}mKOd40ov#(2zjbMtWqGxK9VHd#WG z563Rt{6VP=T(SCuQaX8wGG~0%nXcUpB0K(K{z=T4#J0-S4qf|Ew=u5OSo`Lc1s*5U zip7{Yx5{Sirk4YAW)(oqQt5w8qBKOr{I9lVb!^u&LrIZ?^^;~q_n%4Ic^zyl@pvF@nTAyJd9u#44Cm1lhR@kAmD{Oaw$nMzMJ}U7 zXyz-bd9Mm4%NC*6^rsUOT(Co!b-xt(*%~7IH3h4!9bvEKFZ{V*^^C8JKD#PY{};Ht z^AUd^tsz`z)x1SM_2&?SG`y<0#5Zi8gGid`6~Vige}{j+KBdZxq9yS107v|z#+!L( zyQay+wTL-r5s~ea#`6k6Q{19z$dC%L#3beFR-^dK8PNQG@==1<%7nhLT2=}6SwAu7 z>w^f=53Q-wxY=wC+CJ5-nqS(1lye;jU(&xnx-R_t9QY8JV^GXY*kO6180*NtVv<}` z22u|VC9(S{@QMtbhcp)>_C$twX`-FgG(psruTFkH5ho!-5VopN!Go!jqF+<~wFMA` z9n>7CzUJFgt0;}(x?E5@3ysHOpmH=5boO8_a#aDL_6oX|>Zf>!PP9WsY^EUbL7IMf z=}cj_zQSTKzSjlnOQ5%V@={LKXhVb%&Fe7`ljBi-=CrF)ov-NzTSBg1_wd85IA2YFqgkcvAh!CJ{y*^%sIQ+ld03vzOi2 z_~U)-*YGX}V&4-`cv5j?1mS3ix(n43aIcecbcazu#gZHY>|($*R@_ry(QXaHy9q?6 zy+GdkWIl5dJvfcg@&K*)Fhw39Jy)Iq?#En#u1g8kHCN>b{FDtrOhV9o&A}B&zNgC; zoAQB2Ne4)1Gl7F$6FAqcK(xq$;2HUO@m791AZhE0liu@cGA*5#DkaV=wtg8{Ywdwa zKeC@(zy-u(Z-NvRYh5*>3LB@2P7(UGK*p+Z*GJx;enHtr>~NK%X&1Prh{Bus?cx`Z z8$D%`66)KUe33+Wam_6{6$XD@O_K(R-)4uvUbq2MQ!vi0s;FnLKb8P2VF?{Z); zqg?M>U-FT(Y@scfjj`(!FQVt?&9io~03HzM5d-c?a+W3WxA;A0=f& zdvRBMuSOIg5feat_y*9EegwH*YewjPgnO|}2loB|fuUeqnE7j7B;?$hZZHsnJ z{-?+^PWod=3QvYE0hMjn6jxSpTi{WlQ=xhbg3X8+73(v%8`W#63Cp0IlX-h{jcAd% zm(AYaWd%qPyCGjOo8)=MCe=q$%j#uPAJ&thoj(nFU-`fY9VJR+(Ghd*)XTT`57Ec+ zW0}-+_%V-Bd1PM|OmsTr8B?cqFwR}N_R2^3&4b|1CD#>uFe_bm`}it?QGZ6badGJC zpYJcV_&;B8J|5>`U;EHc)wcno`YWT^H7}px{X@Q^RXG}yW8m+wl7uNwOeH(+h{Dg; zF?aWz`fg-#J$6i~Sn)N++u6-jHUp(y74ELDMgqjh_Qq-7-)d(&9D0(2!#oix z#-S%mQ&tSDG7G1Hgj3-oi~osGo;^qZ+9}}#Z8qngZ;F}s9x#3oO$-`yA6>8C8wClB zdS~~PU2plzh7zFokJrGa`SP8>?iPWc2E#^R;k#ay-h0@z?UVB%O^B(6N&QnDP)}~# zcTgbcby*s}G8yCu66L+n9-N*-MGH|>+L>X{^UCmhqyGKyr?@aMc^V~LHD-#$^3_4BeAC7>2LXzd=cK+W z;@@P%#-iF85jNc+@L~vs#Zr9gQ zxXZN^-0=US)_ji{dyO>31^>23ip$~!;duS6_m$dDt0R(jVxsL`MoDf6**P0O#7mf& zM`{KH7@4Z&L*{D_3Q?(&oiYp~s1Kq|rW>9_m@<$-9TELJp*)Xa>H)EYaur9|e9zp$ z$Py`Dfs)0S1gtQ_$#f5{MwI8&`dO-jPlROC5(-2Q{{(U{4gF+j%kFe;02k30p&$Gf z3^i`HsK+!)77pwc5VmXL0;d(jC-ID&-gWvkap2acZY{dBeftj^hSFV;`RcyXV;1>N zc`}jysXPiPK#UE8iL!z(E!>1#-jtGgD4EMe))VNv;vg<02xfjAXuqv%gN`RLyaYq0 z1GEt{5MC7XHH)`z0rAYomgi?Po#qB_jz53U3nR46yj-{tAmzGZJq2k5drGPE;^-iv znQ$d_BUvq4_Xi=(_0(|3@s`*8k&j)|BW|<$_Tof;omKVccn=USy`ajlF&Fn^NhnrhL*%J^gh58iG_YxC-p{5La zrdi2wbXuJsRv~mK1jC~ZURe1x6Np?DyqYbomjoT;i${XdInNEzDYvc@@`ElsKU#C3`0$PJddi436KCGV}aF6Tp_`{2o<%O)&YJ}_}qY0Lc(2THybL}k2bW`YIh z&lYdRGmuvsh-E?Rdu%$kJ2m@-fP$Ahx4ba~87r^%(SHOA=V|oaahD;7cI78jsb)A} z3@cloFKAcE?}YfTVr$LnVkA!Qf@dTuz0eJqReFXze#R6uq*OgslF)gg8CS)dk^7z| zX-F(SO#aB8ObVKKC)aeNk{_-ViNEZ3-}7YK60mq=(N zIHFT|89DFObG&zycmU#d6g>qX)4_4$=ur}E75VV?_WR3tQdHud{G&ZCt_=w{BZE+g zbFwB@y`4+MHhx0u+8VG{aJZY7$LQPQq}Ik0uE9OMqGJ*+uoUX0h8XzpBSQ0#Y(r@z zu+GYhHu3@@){o$4XENSFBpx#n0gU6<!F%`r2*EtD9H>v!KJ%ZW)jMh-iT=TeRo;dscK`571KcZd zvQgw}Dsc-{b zBjNf4jhzkCzxkuQ+(_aM{$bX(oiK$boU>4Ih3Q{uE3cPb&G`J$oX0cIKFmxOg!vwP zoh6DyKsUjL>p;+*2{I-tK<8eP7hI&ceTKfhqwYh5`y9h)h3$sZPBxIFL}L?h{#P-C zQg^Nj=3Tgf0#zR#|HEG`zq7+d8ATON<(O3H1PF0>=C{`$LImL~Yxmg<2!gIp{D_|a z^5nc|x)XAAEz?a_x*gEBB!6x!Uazr&1FK6zFkx4$2$e8UG^cqAnF%2>+9I9 zC$zXPfGzjzgM*|xnk<~2pFjvn=jG?`6PVaRuwQf{3K(Kqi0vWoxnq}CyaoQm)&BP1 z(VE8bs?)n$Wf$(o#(hUC=H82#@xS2bU9i)6=EaY$rYmjlFR$)_p`90lKSlDu!oGphu)EB85t|i4Z69_W#8Q*6^>%2Ax4AP#efOVmme10d z$lQZ&Fatgof*Yse;{0G$xwDR0WtSI)hK9a-Va6vtn1`}o$to^Rf`#qVk%?-Y5i530QU@_uiv$A5`2sa#mC`tuwUKL8G!m|mJrsJ(OlH+lA`0K})fWEO8C}K&qMj?oxeTqY&ez3%?Giy1vjiq>JF88ae7NTU~ z*qQ+&qeHdX03bFLT3_oV4Ms2-@plbw|9A&_uh)KV&Us{7b!XkIJ^mn^cCssT%IjU& z@m(yS5!9@WAt=9AJy$rPS(uU05CH6@V>VesCecR}wml;o7ry1rj>Iw(EV(M!Ic*)_ zPkH@2MGKNcsqgZYE+FV}neYL1q@@JB;xuLh+BCcOL8P}F(h3zMt* z=BQmvwPV&A4dp7S$M;YBj`AA&<}*Dv`8`$0fI=oSO)#y!Gv>ZAbdN zz5I%Rn~8d4YDFLG=Y{LXwqN?YiT91MPl$=#d_0Gldl~Y&kLYGm{R~A22lwsD5#n$qxeQ`fhAYPePXq;E)oZ>v)G^qa5+E5z14W1_J7&;+CjMY6jY3`F z$&p1Y={%PsUNm9pS%kDj)Qk?~$sp87G7tvq2S;w9nKKlvO*dx34 z9i%mft~Z6klv=mBvc-Tx2rpI!U2R`Y=ha(-wpy_Bo1?rEC?6az5&hYz(qsHCkHaF; zlsxzR8QN@ze}B(BOg|ebm{!FvGVMwP{iwtfp4cMrOojH#ESrU0mpSuNCujJk9!<)X$Clx`0sM zJtqM=)>g_>8;usrRP1n?N(d0GBi>T;^zqIyYm77jBv2re(=HM6$eW;<7J(zuF4TF; zCFu-0?}HUJwDyloCTSi?DY~!&t6^+sw!zTLC*sjZ<^KrDe;p1DRKxB;*|qO>kVbyK z$=;xDxzogfy<5c+Wlfg*&bqLZchwOa^SR?lF3^Dn*SLc9Wh@S@yYDa9TN?{&_G-CH z=URTFpuM0@%<~v{9)qkG1gh`8?B55!Zu~+K&CzaQKvxNnz2zjuARswn5IW4HXICg* z@`!OoJKg1*5J||Zo=D9~&))=fQA3w?*IV?VnJN?T8JAm6iesD10#Qb~j4x6~T%rD; zrT=`x2SbxOZO)yw0*h9kWo#RQ8oSRSug>!x;0!6-yS!+ge!vt4$B$w<4okX~ltXB~ z0&>G2LB!>9YHve~SodgFxPw0n2i)VM7eYIX>t8VJ1Wvbcf4`d_n4eyP;MTWpv3Q~c za|Y?`1%2wK^8wMEBuWY5i2i4=^Jl@kWFY7z^a{dZ@C&P}-gRf9Ml>ni>^@$3AR{j{J_jt+CMQcR=9#y1USSy!&c|98_uqIZVA1y0vMa-)RHdU;H28{w zbn7JoKh?OKGrg2H8la12w4zZn4z`ws3|0)dDW6#WM>@Df08Xz-x$ zFrIDil?QYU0*d}C7gTGIY_hEA{{ADM_1@GyolW$-ht$|8?mnKa&;rT|_8{hgO$y!X zpdr-*_+I2}R#GYG5J}8zrdjUmE~ghSLIqQ>J+I1x9rH0#$kqm|=0j@Q7i#nTpM$++ z`6vCn^vJn5#jzB z4669}dKbadq*ldFd;gi0zu?05k=AN~MP68LQ2o_8!^;+@@~?qa?o%@0s%- zayhyPnZNnu_e&M0{qG@Ksl2N{H-~=8+Hk%*fCS_*NbU!T5Tt+Qrii+Ue^-3dtrz3c zQKW1ruSDgKc5eWuhZ< zMSV8x|HOhK|T+0Vipd<$268toh6J3+z*n z;uRPLh`guh;4ymtNd4xwg?{jlf*=ZJE7D1ic=3tAAFnqCUO64ga;ZnCidqrqsP#wiY1^lF0kXgxDc8=4()RBhg2u?fHq)&gAervcnBv$Lj$Fj%Ew`Q2%rPm zg=M;*H063*QJ5#1`J-g{MITjoaGd9uNs;$d&b#5brRUs6)Y3)BNaKjZXUNKX{KBFm z(is0VBDqM-6_UYq%@)et6Z(e%X|X7%0!h?ffN{p$&FD^Fy(@I1zVhd$tx4l}0c<_F zu%}I%q}qf4t^Bgl4TP%Og)A&7Qhuyiy75?SuB;siEBZmvcw?=L(K58{oYmRV zMna9B+6wj-P~%JT39TT%|)cHYuu z)c9V1kKdrg?s>VptQHwSGCyKWnx4NB96tZ*t1T$m4 zrs4#6h)##S+)y?}2$crzR6BgdO|g+jeA9rXt>8#|;Qz}G9>ruG0cWAXUvd>Hjpuv0 z(C4jWgwb+tAf~RcDA&o{8J7dz*BM{4)KqU@yF`X3nWN@JN9q%cfzb2Pf>{nyGazoK zFqyQ_xdK839~ZMAK);svfkjhObCmi?r*r(UgoS^9Adb_DUdX?T~{Q2w87*lP=$`X$BW*sdT*)|03nzI)vnnJtWP4V5t^Q>361CCj?Uq{TqOn!$wrx0=uDRs?X2?n7v3=68 zH>ixU(jJjrtpPPK3q(U4WVq@t7u(Emo%6-4w28MC0LWomK-lfW+j~A{*72vgj6Y~Z z^)yMW>K=tVN9?y``rceL!Eq2h1vZ+sQ$Re~&6<(al%D0Rbc zFI8!sl8~~OVCj9m^@&wdEm5SQ8XJwb+f|5^ebu3Auqr{B@RZZMX}bq7$PA_a&jmyV z69oE5Isq4q5;@~kxFh+Exor&;v;sF$34>TAyix+JF*IuX+JV?omMTT%{v;MDe)ho# zG9ZdcD()yYZ>{bi{T6AJb-H{;B+rxbXwjgH)hno9X>z^TszhNB4cb1BpPMIJkFIQz zv8qIyKcKrA&WzFhwwNT^9rec1+Oh4T+(mTDc;g6n*}Qk^Mg6p|#(YB%omr-}>$}dF zVBQ}VzmNIyua5q@^n zXEqRg1%J}_1A-r`AO$+|WHdXaV(Z4OV-=^W`<6GA657XIZ{5<-$dW_*f0;@V*08vk zRuez)ZzPvNbpuuKj~oNIwmjrM5wFKoszLJeC%Igv9P9?X&XPgFPu>umkO11> zcyS(%-J-W-n{$4J=pZ}xtZl7IvCSlfC`338P7gm+I!s(}1u->c#<%7nPY@+OTmY>F zChq0Wt#zPjJT;G7P~B(w)xYN@Moa(p4-;-rImgG^X%95@9p87E@0n~^`!V(`xzx&8 z($P-cOsRa+1=QGbk*M7HBs<*fnT!Q|{71!>cN-|ZP4+i8zoGTIN}B0T_hS5rnR zB!&1dM2IdIgSLD}@K9%uUV{3Is}#%bIDBo4k}oEt-#Wi>WAa_2o|o2ZB}^N_>~=@t zbUmWD*_~@JRD4x1ye9wMwY z+zmylxJIE0xN`h0;% z;arMZ!hODG*xqAGtp6?xK;cux6Z0>m;Y%=-1)Lpt{n9xK_Ie@Gq@R`Lr5Wf6G9X;AD%>YrY}vw%n_>YIsTB^^|Wv*CjMf&}h^YDl?juwB4})yuseC&`n$V83>ef8XP*sW_Olu zPC!(VXTYGV_7Z?li&`V2mKSwDrxBpTEgiBo(wibGSkc+UJ>%;S_!iS-{4abi)nDRj zDN2dU0F&JdEiO4}As-{WrHn4NY1iuwLS@c5p?C>gCyP?J;abch6oL5AI2p8sMnTaB zTuF-)qgIcJN$YBCma{*0r1yM68=K+VLS+k3?B?xW)$?Z}r{quETylOsXZ<1eMd^D7 zx6ZRgy~P~KP{u`~KaTX*a7kt-RQ?5F)S)l^{TJ}>UT3J}PnstN>BtZK0<^0r-UdCA zwv9a>#u_t2qbFlas%2ILZFpi|^(YJCilybs59p#NPWB27I6Tq2{o`v=Q>Y^G`AN^H z410rH^o|&f;kS@Rn)XVSNm%N|C~p>@uU~she!c+ksvgPpX(P25)y+KU7iM*Zg;BQU z)yjGFEY>`$Y#*(iC-bCCtohHBYm>Ez?dtPJ6UXO;B=5UpTt}|O1$`rC@0vq@rsqFz z`dU}`R%-eE(z}P$tW9q<3H;=-bTN93L_;1fNxC9kPJ^xjuZmV;L549Zab(u+7O`c@Qvy^zx%_`Fg{b_GP991e9n(Pik7WCNK zKu$VNoG|NESVBN%^G&VW=ts5I?aD&SlKGt?gQ4fU6FBvxMb%#GuSHithlk#)cZn7h zg6L(7yHw~*ONy`*QmTT4SklS)?631oViMdGB{Un+Tad0ROJBl}5$QzXj993M#0y;`X!v8#t-$7|AfDHAYGKvV|6LCltl<@r;w=;l@^k( zm`z}AIoUA$Wry0x8Fo7i|Eb7r*SHxNz=`4Ak==ot{WPjNeNO`E^{j1Ynuf5*%rbPq zMGcm20s<8wdOYc(r|4%3b-$E@7SPYekB(0nNfoV&0jDKs(=W_@>#bw71CB$@Ov|xA zefVyAhOxzMR?|YU!}B))hx!OCcQLP;0j^r8hsIEI#;`L@Tl zK#Cmo?of?`oAYInVl9&;H#eld9*%$PIueEKT1u`vR|8vm4P>v{3}2knnD)7uxo zpGEg9i4P*Rq@vV|`Jy9G9V@)v7ZQg@Brz8SNC@yp=M39l>|5sQ6c=YE79|{OE_VEx zU#-E7CN62asN8N=-XtenDfyHfef;jCmV+LC$fw2er=1XBoqNIf%A0M|z1kiv28|4? zpT2}UqOZPZV!G7oM4^`uWu2$soCOUi1#TiGB$Rj8pbm%559OX(T!pkaUtO%2v*hN( zlSMfxV`rRSb5cDZ<#xij@*4HPr$YFpd+6>I$B7Z*G_^@a>6m@*8n zpjE}ewkMHJARSU6{_WSIr(P!UE?Kwt3rrkyc)_apkOseINN!c7ujtw}2xjM-lAA1` zn_KzWf!ndex;RW6yJ{jLIoj4Aj*dJXJ_H|8^Fo(M<1U63OTMV5 z{Fn8GJ%m%ZOEy8KI&fp~Xu}xyl$+&3x1PN}Z>WvxJ0$lhEkI1&Rovx1-A70tQxn4#Y@0Latu z&2bBNHHb0dg!^hUdewRIuNC#Mro?lygjV2$JWpU4{jFhq;sBm_e&w}{?pn~kQc?Sgb2sfwQDZD| zeIGW~81;04P(PJ+3F%e?=vX*3L!|ymz3ZT3C5Pb&%p1zY%ym(YpP=d-FQ~z$w;B}t zgtA9_D3|g}#>zy#-0GRU8xGA*&@p&WYQv8EJ9BG!L2fpQQ z75|@H?5`fS4TKw~viVTqHiXDV;U%7(Jk?lpJ5-OZtiyL6u-l@Prp>JiXkidQvwfcm zjhLGGoE6~ca5b2YI4qu z71p`$sB*ov_jRasIaX9`ESwXY%E*siPno4r z#(4?w5}93a&IkVn;_doGsECrXfK&+W;;vpjZv-0P9<*=pnDQMWvN@^}l zspDijVX2e1l+qfx)DvtyRA-|#pOpRxNk1?A*l3qG0SP7tO427u%{{9p&L!mRVgW-Q zQY4S&Z{5M2%g1wX8Mgx9>x%Mi3?ANDcMPcnxOGZ7ud&7 zL}wZ557e&(Pz_QQX3eS7OmKgv^{wvN?9N&Re3t*m*mnm~-M9ZEr*X8S?no)hCRsU!BeJsh zuDi(Ij!-h1i0tgWclM5!k<5dPgviRsNGN{SoA&ql{+{piN00lyE63;aeqZBtyIgJY5y~4JbACNt`;x&7p^>T;q4%gegWLO!u9F z72^+}PO8yHN-X9Ld7QiWe8vh?T#@wOnBK$yGr^A%Ti%)?leL}NK);|}X=On^`XHjD zV4_R1PO%0gjpX43Sn<{)k=J~f$DwEQ(Bw>_60G5sR#yo}GtxH0%f!zqc9>a%xf|zc zJLNmK{O)1V=Ak%+ZwiCzbm~9rrY3EFelc<^3}NmgG0w z8dh1b;kYGEiQi+9O~w${m^n3Vbu-hHNAUHtHUO#rk;?_3tCMmlxuxGO3XQw5dL4@? zm#z-C)LP4*{dAj|iKlCEp}#VfZDM5rCIXk6gjO;H0ru6S^`A_3-Gd37IP(q2f&K>a zrQd+qYWrcE_G#|KP)YlrY>Xd6W~Q5@T7QP8z*4+P#a(G#kIx56Nfr#;||kgaN0r`w{AvW7Suc8MO!x(sj!Ujk{ss zD@)5jXh@OEb-HCAcd9^^Hh7`KhP~_rp|pG~DV(i+r&m@#RS->kUC+YjYlPiIAw^~4 zQfa?Cg_2WvS-K8kyUK_d0NIzWB}!C$>k3kjT%kOf3su1*%V2W^Xap5K-;$emA770X z!+1A@@v`RHW{Dp$2GCn4ynReBvTotf{hs1uu>ckz8hDn5^y(8F|Fw^AT266|KkQS; zHY}J(apDjo*!Wkv3pwM15*=WaY+^6|inrr+%9#5Pr|7@R9^)O}U zAkY>)*U7%$g z7wWj?1r&qcIPnSWqv-Kvr-{r1k5sFZs1Hig%sZc_{YLd55l<#~K;>Gy=mhY?lgAG3 zRkaNAvs5O|(TNCh7c!gJ(PJcvcg9UCXW3zJWvw9m`Btp&T^fH~l*TSfPH;g55r&*0 zu=d?QjM$Ws>fYiuaEp3iE^eq8|0(r>j0_AZKaa;u+sMM2ORa=TWfP)6r(l5H2!69} zk6Zif3zu&b7l&W5{c5n*zY5GC+3|BS@-zPMV6Z2R7+&x9Usuh;S=n33E;Mdl2S&P8 zy0AaI19}Lr@v78v>V@~9pOkT68-~{1h|e2m|0xYogrh}3apl>HkzmkgmW#xEpqC3s&?wee zt2ehGdYGz;PHCSzZB{hy@;H0fK!Y%F=udkP$UT+tO=^V8@&(r8x6>OxX03=D9I|T` zfxUQ^MqC!9P>U)%1&?*5wFJf0d$2mFx{IGwkg#)BOUWyZ${gp(<|pntBGjuB&(2)S z!P0y*?N)*Nm2OB;6{_#1RVvc3+YIZ|3&##-p5pNg?$|dK_`oFM5 zd0v>?XXnid&>mM>_StXdRFM;TZ3Wp=yz0v^CZ2rjLSs=JYicW!P{*ZCwrB#8zkBc<~+D&I|1og1qW@ zucE!nvHTR6!+;PjJpd6&zDh$C~|9asJk-S{h=%vXg_f0fYa62;r#sV z67(v}Ip?GIIBlxzuq+fu#+}NbVn3PW&O@z_{g)r;d9NQ zeX3Rs{zamlRdmJu1!u$5ZiBMObQ+*+>{OuQzZ(g&^?*XnT1th||cst8pB*Vo$dmxpB#upMK{v_n}N$}mtpjhSHql+>$s|3KXk3~TKCBoD9Hy5iE}#DG^DJfSdxuTJElN{OABE(n6b(vXb|kb4Vd$A$ z#UlO7u1!x794dixC_}H!Bpa>j6f9mKMWu3Y?1XIC8`kXdxOSGv!ngL?Z7CXxTY|jF zNaG6cwY7T)raf~yISpu3d6TuX9?Qw|(Ro<4=2|W6^X5?!5F+QL7?umYr*g1`ae($o zNzswhiq}COD4^V555<)SJ9A%88!1#WJa72C(>t3CBVuqy8_UF(! zPd!e&@Q4Kj0vP!%(OlSa<$eGVDwKC>^uV}Iwlj|`Gv1w!V6L7(x?DN2Oe+$-Ygr~u zGW6pQ#|x?hNSpr``2Dlif)QNmVi?m+itnWtz6HxLt`lUjMU)`j-ZRWuIvOX*Nzfx| zKSxv0(3*EMAlh8^8nz{zO;=9#S$t_u2!ZkRmWrZy){3)jo|w$^_`E*Jtj72nV5)f% ziK`EtuIGPNOIE8OTzpk`f{;-$ThCg-|F&#)e8)tVUEKuFX8#W3#S2 zDF=Yy%y9oUm*|;tkeBS=PT8{c#7z>Z)7kDi?#9bE<0g1Rk_?=Wv|PioC*w-o#b1J< zcvh2;vm8q^eOgrNVQ!Y_EJYWwL*8a__)=HbUu?+Lnm4_YZBWF1?;p=_}-)n zn2t71!arqmN3Lg1w|w&2pB7d>ORzuS0u-ADB_%fwS4=Zza3XqCcE-u{#@E9h%JYR5 z6|p*zXZ8)NZ0P_2=85{tS&gpaenx6)-MXZ25j1MYky1{o$)hhMORh5Ao;g}| zD2(l0C;G7@RON$mUJtEKD30n3Qw<~1I2M{MQ&M|>pz`)blwH% zEam#0Se#i%cwgKas0Tg_hW&w+MdG?=kP6VAAXoZwLC|)lzj~X*TJyD+!1)M`3Wv&| z2=w0J?wff%eJdZ1mE{V_yKk{}UYjj44Wz-`N!>n~>?c1TsS!=Jyqdlu*K`~cClhkA z+O9&UaY7h6FnNe)3avIWMVAHB`0~fd(Y?sf5Bvqq?5si!}A@!5#bjk~I8nG1}HE_8O1l6nD%E#=>i6?HF8J6!v`4HOMRCtg0ehzLhapkyCX&d z<(3d~I*UP{^*;B!_u1G7Sgj>!&`p=kf<$qfyWjDSai2OAlRj;SCa0{|9mB{l^JR~4 z-+*HOI%(pZn`7wi2pnyXo#W0wMxW~3gWMJ2NAZSHODC2!>MX4C-wW>7+M43Yv zB9aJ#;kyLq71&?OLalTEg0g$1=(ULsS+7Q^CL_!Ylg7DssZ_){7`}#x4ORhTVNscbwv=-@k+Akqje7N4;qdsZ@X=^@;3v@#1sdAG&Mf{j_ zrhb7BByK?|)5~sbTwcq)&5KhZ&sAAV8P(YLPd^C$t8g3AiSc`H+3h4H+Q_OFL#2@{ za*Hg}aeumO%WvQqRCha(PKp$FkL~**ACjg^yQOU~kq3Ow9vHvqd-P>x%2|ML#`dsu zWRngJ@sq3@O_q76wDOiReM^H*ZDKpA3hnaz%WH%V-*;IH+8 z;bs>Jy>k00XMtnc4bz~o3?|Dm_Y-z{RK%2#@<$|4Htv$FiOvOG&K9ougGq{8o;ljde>ioYN#_WN%l}|Ccxux{ zx&$y;Y}5M71WkbrsjH`Ca=*{0G4wzT*+V)CtN7*}fp2fjYoS#00@^)x4&80=f0~cMeXxk5HxYO%MM@8&7sq-eKD4lQY}cGYL#7Li zq;8k@DI{O>PD@bSMmZGg^NhMhQp6-RU$sdO=DPqJCGGq$u8}SGLb2(-RcO{8@A;Qs zBn~@1N0uj6pRt+g`<&w%ClyI;R;!6!m1w56=Qt1b3t(FsS2!)4A~bZ+Of8hekMts{ z<^dfmw>r7%(@ury&nnLv^k!cao-RqcbcjHvUgeH+PZ!PW2<-!`^Sy?QeivU@vhAv51Y+zGa}FB^p%zv z``GDD5AjlguE+8NJ1Z+ULBj(4J7#i*Ur`Uo-8h~T)95g+uwN1|I7xc!p{4Qk?D{q( z-7f&>(Gqu2qMLTKpp!i_lSqx?Ne^*prF+f=>43=C zH;Zfujgnt}pM`q{y65*vJC~`w&rJ-JsxFk0>DjR};HDHMXbqgA80+`-B{l7JZZszD z!gQouDRuvcM;FK#Nv3ixUpn}_N8=O$A-vPeGkXhXY#J_tcD_c zqFY51XJ`_ir5ssPPfj*k8l;-*8gHe`0`x@|T^Fji!o|_`iWQyOmV(-bkp!8EF}svq z!HEXWcUcr~BJC=i;g8^Z8YkyR-E}UqD&=srh8^><=q+|;f)X0!Fn=)O*S$pV7TQojQ>RQVDN1%PmEQI_cqG+ z4T)qaImTJxeKGQuGV?f8E=4(yZY^36? z9~iat@I1Zk-arg}at@Q26Cf_s%0%bX!35(8X6(G=zn%WIaf-~8X9H6nV^@`vZM^gS zn2z*FU%xz+!^8-pAcN{|VV4J6e{LjBjmfk8sBLb@Cij`5HV*i4ilV zj0E?IaQC4&+n1ItixS>U$%f02%1xH_G7K$q@dunoR1_lJTj!lM<13y&iaL8)S;tGa%w+;aIf;%*{V-QWKCqFsS)x36ZRbFExEw08uGcWTFzT5zhaH;W6d0RN zZugBz10TOG3{JX4>P;OteWg3emEBu1Zd#H0@ki)Oa=P*iufIEvgw6EOz?6T$~ClPJ}P)85NS0rCB*58LA8xO4nlwYgVt$aBv zOHpZ_b_GU}IfzeG=Hmk%E6UDxJB*c1dbRT822?%~2BX71BNplwd4f5}of7A6L~|SZ zjO7`DjM{Goi+j*pyBTE(H~0N9vwu{`NT&8XL3|c@wSGOli%8}cN7a0vexR}We2)h8 zUF2JtS?!cpc2Txshel*|?x&7E7|pM7W4yqUwof=BHU9FF=M@pl-RN^4(q5Hn)xO`( zEEjJ!n)wFJ+aauW0l#vsPQ~LmA*>S0oAOemGS)Mcu4vJvjX2^wugIwzxd^nj9hTwZ z9~5TpPWX;Dj7VQg)P-HlNpl01%7$lL1X^@xzv$igbiU1qbbKb&=eEb}N>6??&WyEK z++2b2IlM&|+j?8VR`48&fquD(7^)a~NV!kI*BVqHMPOA>aJFeup6&-0pdb;c!%FSC zmnj%A31j-n=h=j3sy6NGK%l}rZke&+`AdJQHzw(sGW4yZWp~7Pv|Rhg`I=%?u)84D zNr01JbRAZ?7|5pL#-l@ejJiU!x&%9m#AOF!Vy7*!DV2kstTH0U028hV3QAnlElV1oS+XKWnvZb$=E&HHvFu%seFQ%IqddY@@l8mx&#s7-Nz_tU&QO?eQ(3{g zuQgMy9P6$;QUME)3(Mn&fj&FvNgO?jYEK;}m8UQQF$IZq0oW00f&^8Tb|!B2ZriK+ zdPGCv$F!u-$C4QlMO3PJuqVdAQ=+p(L*y*B_|rRUlfrDGAaN4{A?~|IIgdYx{AfD^IPzm~HB=qiy^g>R>EN3nK>P@ zuh<%LP9~{wmyes-@2 z+}Rj8yuYt?Qlffe@JLrCGLdTNV8O#a;*kRH`E@P|Wtyw_3&f=kP`emE#2j+gSp3wl zT!Owj&xUXNtwQ|{V}8M;rQLzWzH zLMIsZ!7ud9`YlJl40)4K{jVe;{HLwpOLum}Klu9#!BbJlfCci@eW+WmgJlwi4c`vE zuG=9BBVtB!^v!PYM@TaRt55skAaxTzv>}G+v>Oo4ULpOFvZeRoH$|2gX>fKa$ZLSB zM`Pn#$JCRb-u+vB_x-5-bRvtnjdYU(0rG7djWxJ3%M>lb49Jt!1~fk&7wErRdDm|> z()R-+hc5Vhrbvb$WXcj>R|&SMrf$meg6-F%#zK*UK#2G*GcW{z{|%R#9D*Gfo|3Lf ze;zt?r~Y&!A;O1{dh5AiLcRnxio0t|TyZ6LSRc2^ow*aBvB7l!!!#(ZO3i`ST?m+_ zN+}zdp@Dk|d(22^61ea(cB&41xpNfH{uf37x#Nd94@#-%-RLXAMV{~L8yl4AzcdRkD&`x`R+>2v3sOr}EN76Zkp$c`WL z-IyVAV9|3TPc{9rGBAOH*2TN45-fAT#8yEjOf8JhJnbM_!sW|Jww_J?44^-MHhm3z z^)bl(EF(A2XhluYu}jjf)))ZTbHUD8=39u9)Nv--aB5u7Xt zej}gJ6NeNjk*!Z9)8xVJZ%;|guNTvVhcAnpueZHoo7%%dP-Fmw$P&c#@$WlT` zu5X~AV3P!aGm~FJ6n<)l3F*2>8|nx6>TN3NYOkSb56JIB!n5`)-5zj9GJSS$*R>C* zrimPD6x;h>jxD~LVupX(UHA?ULcnS(NhE$grdb_0&S!;1RCa-)ry2SH9{udQGwef! z3fIW}x8+piU_ef30TxJ_iPYAhz)~xHAg$^e`PVnh<*3?Nef)by`1|U-Knpt`M=EyT z!bfdol4SVul4?g6jZRa5iMZ-5@QGyNas$wz8c1qh|^7pB*wdBcgR^s8;+f?xZsn!VJSe_$Hxcu!AMyvBf>C7 zNh~l`7e6Sq8UHzDJm&jjrh7*Y zo{aQmFnG+Aj~0?jw{H?$MC|WSo+Ds?NdoTYG2edtJU;^1AC1AU_qFkZ;Rj$Fu#&^b z1L}1hJ%n4TZHEX%MI=iwh2T7Q9}yR!YO-%MXE>%XM8@-DBg5Z+)6Z}BJ)ryu+irVd zCt3oz+$q`b;m^&Jf{7^s2ElwczF3*UfKoME?`?O1sUhNFKtY{7h#6^*QrY}}d=)>w z5BlwNdT_WM_9<Mjs)qVdt|?!8+_%a1G(Lw zg5aZBPnQ3Ki}&ky{O7~>`%jGSH~Q{aR3b`8p0NLItMkTx`%gLM&t~!W?-#?PE!iw- z9Cc0WN5y^o#0)_E>prXC6!Li6z7ubNRc{6qk98Ly$b@Q_sd+$qBdte5@_%|BBxDWWf$ZJQl}u#y{khjxz6U7l$W)>bS#Pd z?MI*{<|Cig+2L0d*|EgpAq>f{SMmL;|MK(Goq;HHt>=hVafA8Qo1b zLW)8JS$ zap;`u6>tvgqgZ!%x!MpS-986IqFhkuCMg-@aQ~Rcoy2sTWpWa9T%1qJDTHjPX0(9~!m zU)KD<0?xUG@tSrcsuWbEp~!S%6`(^cFgK1^VMBySm0^aVCnUg@DL#Qk0lX!n_a{#oZ(dYEw)`_=eMGIh31G_l#W}Tsd zGVOs$+^bGa4Q9G`h>Q*b;dMs9wJn(UCn5!P|slDjNvWvRzBGv@@V}jFaQ5~{qH*zr8GvwH#u&}u+uZZEXRH<@ln{+JLJf{3Ny+#GTS18zzn6iNte5;B*ddqp-7X1Jt42`4USH(F02_;uzP z*KyD-&?4&S%Y@@vB>*?AZ}~u0VbuH=TPxM&11A%?y4XKS!qX}0^9kYz03<{eEP2t{{5@Jt6ca2@On^0`Ku?#JFtje zSpRKU3A!#LI`GADdHNC@^7w7a$dqmZx>7rtSpF|;ad85KPc#SI{DZ^O1+P zwV2I=&%JqLgn*wTX#MkQ&AWZg%4P!x^MS?AU{=_RT$3;GdA`^s{I#DA*^yhpA#!v> zt`c*1l#Y=d+!M3=w`jKUZvLv)zCW(-56P__Rk{MgpTE06#b7F-*bE$B0~^dUbL@KM zZ$M;EF~TRz71VO-Yk49MqT3*goNeJ5ikA*M^t;`5ga}epJc$6FjRyGk$E{2=<8bKs zF*5+kw1W%YWrz}aXwgpx7!0Ee3U;G%a~+U`gfOm&2^nn8> zn>vNcD!|e%T02Vk96I=DzuXt-x1UL6Wbb`r6z}~J++`5l-;NTZYU*QdZ|c3_d&@q6 zXC+w7nkDVk)du4;RURXhpU{3J8t$tPP|Sk*^vWY08!7SQVE7r-GK*Z^f8FK(Q~!D% z8SJz~Dp5m74LT4*PA)A$7b!OZ{e!ogW~p834q6<~3GbkJ4CTD_(`}4zhUx+)!Q1{Z z@U1lT%oo=|;`S=TAF_hQ5aRvu}9vf>BygSIIJOhwCWj$E;u4fv&gr)wh% z{EOlO*Y7_ZuC3Gq)IaacZsr(2%47U@`N{!-;bKOsanSY-$kHl0mmn<`qKQ0gMUX-{ z(hMk#XnU0tDT9YJdwbO_P%Cz!4MqS`6(Uj=1Q)YT-D{DEdz@TQb_k%Q2B3|)t7Z3)Lv%%8r@E&c+tov1PMXw-vEjrE$i zb!AlNA*`98nG}sr$T#b1?JhK*$eeJh6RUvTE^A$gt5{&to(o>#uV8pl5;!3UMjp50 zy_t%U)Svm$Bk7l5L8J}W{UhLBg#ZltXgaxh2R1g0_=llJE*t(+3EJ_wIP#;I_TA@(46Yn|kyI^I z2*v#Xs`vVgj_hNU%Lh}lE~uO;7C>CkimW?PfJ>zv z>o9~2hZJOB>F*&3BiFGhy{NV6CuD!hJRXvW`G?$S)BhS*JkR0XjxC>QfXc);EMJduNBo{&p>#t=i;p2e zm~~Zvc+mGWdbT_Y0 zKsoC}j!b;H%M(cpow_tc8 z?7NLWgC+O304~uB*j`0f>D)n-VZxzYGU)Cj^a8c37gh6!$8vyfuzsHm&B;ry%sJ;q zm_nkAmXJdyD)?a`k{^?Ht2r2=h|E}+BMiDXl%=ks{ZKJ^Tu@D3 zh6%U>?fiSbEXjX_ZnD73O~WjdvgwLKSK<9xKH(V8333$(Oj9USEBp+a0g+^;&>Z?QBs<3U7zO%3kU9RErZJ{J8C_4{FqSI{QR?JFO1zyHlxDC!18mmyY%eox}-qIE66_$&SICRG?5zQ=O{FAuTx`D0f~5@+%%*pVN&kH*e)cRPhIEE8Np3Xmp%X?k4-_3P>#ijSWj6ZA3q+gtHhTZZxAt zCQNxhcF(a-jk})zzyipmhP|-6mA9-QXNrrU=1WWTe2wVeZIb({@jiCa;OQEGdI;v} zIKI2wU^iRCC51h9P|(RvCd~AF^-knNVD5Qg1%x~wk(F&wnNdqz@uc({05^Ek=M!RL z!QS^JsM&CwIoV*^Y!AuKk^U|d-H9$zgccw#aegeAkY#Pc1deubx@4BkrdEKxsqD@= zaV}$U_1^&s)$m5#1Y6ryZp5!U@O9`>K;sS2e9V(c=s65vu0tnVj`tH{6>8gw8>Q=k zswjfqPpp75^1c7Bq#yHt!@lrx2*KQTW(&KnTdPHhyM(m1YX`mW5i)3*3y@A!GZeID zj$;M}21rQn_$qaE6pXYdp&PL7N|p^*i*4aUGcGKy7`PvU^)iei^s-E@)l}ODSq#A# z;lYKSQ|X3Xh%Wz#kwt`bhhopyd63M8k=Y~MOu7FV)HUxdJwrR8yN11Ng(3MenGgQq zdTO_G7od8UJfSHz2C*bZU+X&NuYqtj1FZoAg6q1|9QD#OUZ>Zv*We?Rh}WpzCpv?8 zw(sn?1k8?Q;uDPm{Zh_S$!2R8Wv%mPs~NS;9-r6oQ`~QTQ0fO7xOZ>N-z%m6GiK-Z z6yesVMm(HhNtM#2*`YeW9yGBnp>lu%D1u(h3^G;Wvz=!G)u-+Qx9y@_p#m7cwhe8G zygkQ>p{LO9xde#F8&5#_y(GqZ~mmib*Z^KUmwiYJC z@77kY^(H~1>|{(mDdtAoC+#Ur$ukE_M@}>d!kt3+(W|)!M}&n=S&5R;V4se5WQL>u zVt7U0w8{;Z4pNj>j}ZmDD`IY) zLdZwG^gQ$^g$K1cgHA@HZk+w?xxDp_K?<&9DnCb8-yh-k31W^k-o-_$KtrhQ!i#$K zzYXJ2!kwigvUoE*CI3=H!{^U@Egf4e|9xRr*P9J0>V4_q4Z|VucHL0ju zPNemw%)^4#XC=k}W+4dSq@};=6%8T_#yi@|LV;GDSYDfGvU^4)Q(Vle}rCR+kt^h){|FYqnlpUZPVvl z43>cq9qC3(z2jJrqOT6UxEbuW(3^1wz7yBe05Y|{n=9tZ{bCN`9S{svU$=&#ecXYw zrYgI=XMtbaqIMXT@LUJlfJ_q7?|2s?=d=XM?fGayB4r5}_M)A#mth?i^o#h<0Z!6ZHVsa&XH|yaBW0J-3k|bg8Stl51Kn>OI-d~8utohP5PnMh zrP8TMeg1G)Ur(g}p-K5|B##e9!D}zuiw%Uv_aT{LycimnBFeS8fp@GRi2q1WTKS4zHGjDw=JxmQe!T{ zm!9uhR_L#VT_JmlvfUX6_ioyT{OK;44-bM}yE;e0j4dR?2d{>uDXw-y|7-@fH-*Y_ z0xu{&I)F1s`5ryAGcrV+dE19}8vyiqydvz($FyK~1Z@UnM! z!5T(i5UAE@qmJ$MlZvGL)C=e1N6`AjgS3v%sbb=B+g|5>MQ2zn(KsOxW**|^ zJ{Q!l^S&_O%775(Lg@ClDEV`aQzHl044A`j+Z6iM-|nq0T>U~YsCnZ`y2)Sf*6K}< z4`_9Pt!>;KP4B#DF0`+;_XFc$m_v8u8Opn~hqE?|#Jd`}(XgwFJ`rfZ9zRzJZYY<&cy(|#npVr^GA3ffDs&aBxy`RGe>Qvx=9QU{;ePseBovpQzXVX6_ z9prkhsstNTjP)ZxYg|KdxMg9Y{(wwCMb-|?JTg2C!XDOyV%h!tBThyNf5n$zLK-H1 zu|)+mp{V8a|J8570!N|$a+o$!Eu9j8W|Cp&dhk$#pvQZEe=(2@*wd6(z+R5J;>snj zS(J^E6s3SNd3YU|0nlZ~kVeUyxbt2?6tU4zvu{s)o`qaS9&cmqfO7W=n8*)!CJwCu zFz^jbJ2DauLXY2dTVzuzx%a3Bjd{M`oQCu}VCW4N8WX*WpQ~;Ir-M&viEl#1`)x3Gl~n|VVBmp$at8gnmcg_su9A(%3;$fu2A`xItsk)W;m z0@h5rFxy?ij+4oOp{4eWN4>Qss|)d4qWLQD>iFBXo?0z+H~ebwVdLB+a6qa0`yE6- z{|2VKibdn}C2avTwfc3+wo(S-+f{>O{L=D1NN)V=A+bwNwt4py8nql_Wo7lcu#1r?6G?+OUCod`F*H*8YA zZwwEja)69A30vyD*gl|cK1FWE@67S8m>5#^{`V?O72=(VXnhymoH}Y*gm-;Dvo%%Qk*K@GdeH-Az z1gmy!HD5?j6y?KW!mA6}E&}|ono5qoeh$#Gu*E|}MTNd)Dv{MylD=j8!Bfux6{lCO zV3Y~Dgw)Rb)E18AATBY0q zzq|kxmrICi6EpCJZGz+Qu zU?CsPYnf(n-m>)34qtA;x6je+#Nq%Zp&VKeD7qEs{2NC=Ym(XiKxWo9l3RTMN`;f& zu5o@^{%<#x$|DC)WfK4RjADk)6ps4&Cu`s2^J>Su`TOqosDXU3*iUotl;3*I^6M7` zUox=GlWMnPUipCqMBd;XThxEe%dD#`!k2NHRo6GOm3QbGQzv2g;+1j9tW-K0RfeFB zk`{rMq38qb6E}>*la3j)ysmX~%{rocAT#zQIaa4 zMZJOZW67V^1JDb$n8q28?#>vIO|}?*hHl1oa|qIpxF-Pdi5QbMgS-t|QgJ3@Pw1lB;@-deGaD!biq?^&zjVaibm%+^d6ZhJnnH#u)^Q<)+ai)ZxI+8O9&u;Nyh z-m-|~cx9hj1_JE+YH5%mI&2fjfZ=!J*J`oOPx;?KcG8^Ds;shvZTJH7JXYPCxnz_P zHc$a{0gi2qwWG$_87(m}dVFptSOJU}?k+(oMj52u8R8xgSGc8(YWF^!#YenPJ)azk zGR==b9czRc4DRHCfBahrMqpVGRqfl=@sxBa+H2pWCc#wCioFru5~Yho!~M3pohnz zx44)`_$Ua)LGUr3C)QYyJ^oMQ3_R(30WA54sT^S3F})s@sh1MZVUmm6fVQL-rA%*c z2ispDs1NBL7^d0F2D?lF%iuF1GI>#BBs$w+I2;`l7!(e@aDxH67ffLd43esv04vdg z8P6u>HMYaH4P0v(Ie!fFEl8F2)Y_5~YTFa!N>>vW?9z1e^yu4&xz6fy$=ug>70%er z0Ys4Mu;b}S&y)S`wz`aCI@;s-th#Bu9ElXpscvpuZ*lti>w+xir6^Y0(Hkb%s4MLE zvN4&q(~lY}vga}<=uVH}G#%R$R+Sekvo5Ty%8zu*d}A6Ci^(UCTdh>k!shRHW5X%{ zA^Imax>a8lN|lURC8&}USnxtXIxTk<@4C;T8ri2^U_w<;(mKW>Y%=5vLrSsW=y9W6 zikQ)2w66ry)0ZSmV`%UF^%Km&o|`FKiO4)n3qtJevs(>Rm0C5yVRLZyESxoK zj^hRZhSujBfJeb0TVoE?Lt~;JT6bts#wyZ@LgPU9P~!1YzKZSXH<7?MJ@~RAV}zAE zEa9oN$5?TOGJJC z^8F z?Rw4_t~J4wB`6p!&50+Q99dR;YuIC4o?-h|#(LLEr+FPmTCArhZlJeUW_r$qYwPIO zyNtS9s_ULCR!k(hFFwiAoLakFeA!4;b(`H}H}409fdyD`b{g~h4m6a>YdPU+%}(5) zykKs$&eFyo-fCobKc#B7HxbQ^^$UU(tEtC79wGEYtyly7UvbLoZEd`{HPFBq!9sRX znM!C>dQm6#JYk6y*vg9sDwa?ok2&eUntrqP6HIz^3WAqq=&f$s!Y*Xi2$f2N+y-*4 zP+;9P0h;~RetRw@#s_v+3Jly#wJ3*y>9IVt8#;-^2Sy!?hY0$R-_g{x3|2s7fak1U z^K&F5LAo>qLldnBfc52hrV$ND9wm`u(nhs}z#i=*eB2o_KHz{{!8H!P@S22KMJ@pA z&v;-bC)uY>0{PjdPS8~9)L9z?0PcN3`6(jULyYKvwpJNYlX23KihGTGVT1I2Q+ShiQxTJLMC%Aj2v$*BWxce)#xA#pp@sY=YH1P{4V( z{O7s01u-y2bLD z=Mk)2%aNmMY{6?!U?v1Dl=d6oAVkE+GFct#s!TQOWMYHN=+%&B3wY|IF!AQoBN4VT zkvn8Z*Jd5vYqp0QtX;;r%DTzUdeKxmYxN3T8)F8 z*d-Hyi#Apop?%Gp%Kfl#Ntt83hHFl{>Iomu#%p(>9Xk^$Jbdkfbh1589mj~153|eS zG4A3Wi#YXm3={ImUGxiebHRXocao_@KvsNfiRo-|qD`k`bHZ>+80L&8IueT}p7ruL z;N&$IBsC7{`L?FDY~34_ZI~X#?NrE6uRh*m*EG@BnJ~<>3)Xagy}pCv2s;yt@Yq&; zCH!LL30)WL;TB5WO2>EEcCxl>j7pbj$gaW(cs%pY!s<@~8&=d+>)38qs2DT0n1Zh2 zC}{=ZxDMr7%>MoV(z<=5_&RZMEjKyJDeY`XKUDN{9Ux<`$T*@y`wiOK!LyA3^&_8M zaeN7Axn$#ch-s=9q#9G9kGCK6U<_MTNWXL!?LG>X_dxgov+DpjF?N_N*#UNg+jNd{ z<`e*(G_co?`Af`=dprjaN{&Koc?G3itm{9P#|Q6qXWB1?tL}0U=+YTMjt>v(PCQcC zbo8?Q0}X-2UR$J!-lMI40_^ue+Shl4mN)Dr>|7ec%mXX!04ujj{|KM~1|C^=u#{uH z!!fsZ+9ciEN07apooE%6LeS02x|EgjHG$qY%QeblxGmh$sY$ZKtLEt23Ae9H%>C~I z#v|f)I)$0ew$0^r5Yu*J>||5vQ-sHy+7jgc4Z}W`y`y5dWZL>IQ|fbAeuizFc^EcT zmdIN~R;p(=Y1B$cn2-txCdixCoA2;i8MWz1nCmx5eW$2H@41_4aPKkp17u_5zN1}n zhS(qkIXEJ!i;nXJTYWvL50BLiTLSzCw)_xMS; z&ZjUUXcI~y%%JpFe20g@6+IT#Lc|z81^Dfjl=UmZI4lE$xgYdg6PO!JUt$1u)N6hP z%~v_!Tbozxg+NrUnVgVrW@12!#Me=RJ&4=d#zc>Hv}P7+eXiWeO!C1`;_zdbHoQTs zl}c|DXoL#@aX}Q^3=jh(DGKPAdq^Yyu}Kig@?uPxB*p1(hz(~-mUhRxZt)&h^!pYt z8DA9!+zVEYa57)J*N4C{*ShCakQH(-6Tc@_aQ5wm)DNmc-lyVCMLIJM1qDxut+rA& zADQMI+qF2Jus4dB%6R%tQQpmRssNE0ot0MupU1RX5+;&(ro+CP6*S&U>R)PgI$@G7 zXCmXBLAm{ubw$c{0=6w-QVNVxJTUjpRj??w;Z(c~et`#t4s_W9* zX_4-kP_L9_V?UYQbcF1@&Zz`daXOln&6_0Sa1SSnSM$U=D?2VYw$7<26Tfjv5Na6} z5(2aRpBk09OYn?2wG$@N3gC72u5i${{`Nv^eMl0&Hy7<>T@%OuA7$?yPv!eRj^}ig zQ9?$^-Xl~-wv+5lc2-74N!goINs+y>k8DEOn^KXTS!RlGgsg<{z3#(7ulMWy{``Lb z^f(>&xv%H--0ON?`hDEvZa_|5VbGPx&P#pi4dFF;^yE@UkS_?KQg0^W$N-WSoWs_2 zFpP&C0}Z&6ynbK7=*5e;uOX-A-nyoWdVsg{7bwS*nD#W?Ks9&WNvw&TJ-qss7Fn>{ zN05*pz%Ajo?rMNyuSorgE1Z?7pg}CtkM^ui5xc!>aKUp89nA6V%0<$vlF*qH`;`cg zpT2ZS%{n$p7s^Y65!2=^>(L;^=vo3=Nu-1Dv!Gj;kDWMU3iaNUf+NzuPXWt?e*9b^ zNs~XvG5&k}N`R*wq1uxc^XoQv4-4^)xZ<5geh-9+c>lg;lRY8FwdI)RIIgCsRO*6$ znEPS&@u25Rn*vpdafPJbXF9)EIJY^x=1U7^r6NUq*CPuOvtNd;d+8*ttGSx1yJ{9e*%Z2;kr$E@BmM5L5vz_&Y&f@6NkWLvzPS{dp23KK6 zbaMU12^}jh-DGW>HsWScq(xGb_6?)WjEz5Cj#^{E{IN1_-CKnVHylpbo6{h@D7&2V zW_8EF0PMcnl<5uExvP6Q5#THPu{E5+Mve3fpfTmc<~3L)U)0#hd^V;rcS$)nGpNkJrcP`1Rpyx+sf68Q8N&Hb5h-Vc~;nyzn z`V~qURyl9!+wI|1;H%j?qZ|o>%r(<%$7xjnWpZwg$bMJb`SFev#)bRm*43nX@Y7(| zllc|Ny=Rpp3tmNgO7EPIU~`j@7PSeQzooM6T~OIy%KbUrF_i#4Ojb@(|VK7*%r?GiWtP6!R7*D<$i9v+OeGkPs+RO%t~}^smt3v zFW3!eEOh)13n04$)N#I$*v$y?+b@o6g2Vf=xMjl*N}iy0mJhOq7bk5?0>Ry61gfSW z?EyNQ;^Sm@x|*3lJu@N6cY7VWtC&|^{xumibr9;#I=3Z#g%N09K3v;jD1B>sGWgRg z;O=Jj{Q`pzFr!M5XTf`lp!P9=0Yh_WP#(vbbWl&yDKr+p1umjle^m&24xr-$H8tQ= zVo3n4N3&?1Y>~q3m(;cZYKO;{(Sa>m)e^u{vybA&ErIHQslzWAO*2ERHX5rXZH%sE z-tK+kpA$}=cO|pnbaWb#X?*V!Et)I*97#D}VO6mY)?A)PI0ff6=7`MX^yIKQi}giW zOO`Q&h4>Y)IUnmuzok5oAmVSUsH6IGJ?y;w75hTOb}Zb81U)!O!@T%t<2!IS;LeAAKkj}mV-2G&z6LyU}?bzAV>VA(!NT5nR zG?C;p$RO0127*Fz|9tJ5aW(M!Yv_YPX<-zE>X%%Gp@Y<9%nt9*T73eUtgpBa*dJT` z{y8oP8UUBfK`wCy>W=qthuoy6G-JH-mYhc{fw#FD`HV@O2SsF6CHV)WMOyfn_0^A) ztDtXFCHKB!6UsS#uE&&1=fRiCU$GAc9*%vy>S)BLz>~k!j>dcbL&sXNBsTeERh zS7Gb*wQ3M5ygG>tB$%dJ&T^W!GwQ4=;PMzY8&;#;5Ia@gUR?+R()n*MPt3k!MOi(W ztwz>;L&v4MzbgG~A3E7LiuDxMk&TjY6O|1NU8Knz?vq6YJjwO&d&RZunK z!JA=q-#2B}c);P)+xkw+_qDYBH^fcgPA0dyopU?4vpu2*w`g3{+1$=P?6G96zbgVJ z8CLt%)uW45)({o>Rl0^LLE~QjMe`epYm(n8cep$#J{N}JI*rV_e!hFoX>8U9A0PoM zd3(Ffg_VF48wAklc`315y@$F;kcI?zZ)f@K9Cz9xF>38l<|5r9Vn^3xg$SX;iW%c} zNU}9?3Yp{!UuVv$C%twulGR4gG%Dt#xkWulDG15XSgWM&d;Q3BXu!<)G0a2acjqzP ztySeN?=OQo=WT3Fdc6w-(Ul*(dGIqK?QWmPopIMUFX^?yTJ<1}a!-*&r)8!kD_l_< zv4&{BmNCMYe;{2FRjt@VH`uqR?CX>jnWOZ$ z#E>AH?^%J8Q3@K5Mw`+(`~2#c@a&L`OOZ)nYqGP-&?HG9#{~lZL(iv>5^HZ}-n}ev z$?+~IYnJL%{9I8M`3S*)H@(s4{9^XAwu#YB)hWZFK<@U{%6VUV-u}*IGtqz&woRDF z&5C474;_o0>Cd9~=XZWCEZAn%by_WCM?FQ{z@PT%@+NsvI)83N@-_dVGVYR^AJuN2 z0|XH~!1@_-?&UqyD`36U&>m3)xGMn;&d&$v+KmEMMU!Su%{hrV;)loB!_eSnLZ%#a z9+6o_cbwMTDqE?(-P*twxDLBn*~N#-5gsUm7${{mni?0JDdzoXWh&%bSIA}?D$!Zl zxx95E#$t=J&w6C2sz7Oc+vUnkva5H&a_vZlj6n_|xRb?4XPUNq?VD7|$A!i5O_RB6 zqJ*!+d(R?o6HM22xsHFi=(IB}L6G?hODKJE)Ztd^ON=1Ul|91&2b*Y)IWmb&c7V3) zQxh9kEAxCd9#qLvr1r!q1(Zk;t2c#ZhotA$i##E}W_7QW!&dWpAA{=hU0RV?e;GZX z3U}gAo#l7WIr~oYT}(8VpUtfwBq=B!ob^o(D7W-UA|+`K_M5jinyMOY6zK9!-DFBM zJ~3;1y@JL8FY6T$87|zF7hSGn(YLsK(VO>D#hJ9mel9F}Z0C4$U4I8gk>e4%?UZR3 zYA6@YGl?HHBh|+Q@A4XcxBt>FU^)=cE5V|2+}b#U2M0^k$zW)UOv$i-b7>u{=-c17 zuGHT7LD$>pt}h3UVD!3kYW#dWkLz5+`LIkYJp>I;6SSor9;+XiJgsbX7U_UwH0qkV zUeTBxpWFFX%9|p46xv>5&1TJjsd@TK6hZ4dPSkA8nT3Vy<%~^<8kJ5#;fFo?az~K9 z@RbqWl+c~@XES`g&T6>J6RkycJ$ZHG8`JBQs^;LNvl*^I$pTfvOSAUX>ZL%7Z8_8m z-1_3-lrp48a3ONBsFGomQWzbBu))ucROqF9hfl$ge=D`uVCcCwWe@?aMpK^D40}w6 z`|^V8-08$#4ON=fX+0{f8=gAKNtyNe=>4sb5^Xq#*KcH}o$$csJaAUckQ5;#$KR~!fV87e zUb4yf0(;fYx_BMBQ-+)_ywOq}u@*F+U>aUk=;VvrqbG}c24As%9S@q#M86sEX7VXr zKY%+>+3!23y##_PS2~`I;GrE6H_IV&b0W9dr_u0lxKCH8Zf`WY_w_zZqYv{$Qp{EB zRir((&G3}=mDbH7$){XO_mmhW0-Aw$!bQUseu)FMdF4vv_@%y7R6#q>E1opibq!DS0{Q*a;T!8aQ(KX>)s~V z&*Ru)?uP^($$i-(7NyDq37a8&c^u}>j*=v<+Gfv|jNtovFCKV1wIvlLxj61g9KpbI!pkiSIg7X^Std3!_ve3pPgnEQ#GmI4?aT)?*=d_NA0E z{gli4w(YsZnE9nwnx!!6d=&1=Q@kCXFB@L%0kY`rQv(kcegTh{MBb>ud8cB&xL1jp z4Ytm4!F_3>B5-EyLxnUcDZz^#)Ri}Lw=vTd*l}>;vDZSmGXdEdO(fvFk66T?4sG`6 zdC9u{y5e^SFSit!+PDPCmp>0`;X*+OuM<7*3ZK#z9|H>7P4^b@c{N|ezyj=biIl#} z$TIrG6q@WByWR6-Jd5jy&ihgxL>0+u1ovCZPERtaKZaG+8?6P*0{vC0*2yg~MmYCE zW6bNc^~btf%F!#HewokSis|7?7CY@sT`A3m!Em{W>q6hG>0h}V(t;g3A%!Ejr;CA% zlNEr50)Nl>Y+;x?H>yMDwIuQ4srT;QRhs$K>$}{@E{rqadd4~N*2g-f5BgoEP|{rT zq3F|TSDY|Qf#1x42FM=lzxb0YobB;s)r~s z@1LG)zJ@bESKW6*mBxIjqy-GaNyM)Zd79n$EH6^0gEMZTp}^DlikX6oIUfv{UR^-@ zsRz+SYd!9?J6S=H#DMlfF3DGRfGfZG(<8jEDW-k8y~&kZ2Q0nW@cUKlG!pJ(#QzpZ)q0Y!QaOK;hGqEn?%pknvmGG3B|ZgdyR`vin(PiURgoyoqo` zuOt0{cCUEBl*UWis;27~9Q48-TEa;QrhPcxb^uDsHXvB*yPLP)xuh0i(!XLZiqS1OpC_ ze0JD$4#@bxd5Fhiqy-&x!zQ-nTBsD&8+j^>zwNO zjNsqkuo1=;27^!DS39MBaGv8!*X}rD8C(x0~{=cP4rby`t1I zOuI2pQnz!f-!8H8s5&W(oS6N+k-@|5lZdtCOtafuc8B4D%mGFW!#bfx%jM6hcd%%N ztekoV4E>38;8nUY)2(oWrf$xBuH#qf>2PiP?JTT*(NgP{{`4k~WeC8pT4)TIN zL@?l&;NeHGg`0ymKcorqKuN%@8FsUG2Lh-h604GNJ%3#Ix&7WN-sLXp`c*cRd1P z2CqvK!u=CJk>W?9r4_Qu@|q_SND!v}h@f9aFNj5=&~+vnBhPlU*FaJSJZOjG%z4VS zqeEs&Wqz?ljYgC{ECLavKBev*&>z*^p&f41chO;SSl`+DCSFuJqRwl_RXo(-#qX_j zW;KFfSrh?qh9uq2j68_PNF>IGIReusL4k>^*lwW`;J+yZ1+`&VV4V>mzWbNv5=6zd zIFaf{J3o_V%r$9Y@6LC<`IrjED~rY+E7+8XT^;{~M;3`7l$&bH^*MGol(LCjhHFU; zF&4g@;pi3>l2@ENTS-<53Pbc^b%>y@wVf5E<~Ca3IoA(aWm+g+If7(IdhHk!Yuk5Y zL~0525bZ{Az&rqG5H3)nvVb=OcoMd-iG2=s;QWGqJNV|~XMwEfi7$9$&-_H99*J7} zi{70oQW##V?4A0Q!T81Pm|g{wRoVH9reesTz917S9{3VXnCF8Ybs2HZ6VzQ-Ud5;7 z|070#8Ig5~3AM2EI>L$0mLOC6orA@_<<$3Ddmu2n!x+#~9y~0u z_Fxr%Y-b~xZny)?|9wH){O0_?6?1)SQker&x=Ty-Pq1EEWEj8_1F_Cw)d_k!-q>vLr-eKj*VRWygG#3{* zQ@^X7wFl&y(t0XDyL59ff(Jmv-5y}2{!&7_wCx^o^Km4FFcBczQHX4Zw422>zajty zvV{QHNc}5^2ULI?<)|5yU^P#~lwOhrA>`?Y-I<5u_@cyT^TxlGTaU5KHccGCiz-40 z-90}TSf86rSn%zq zc>Nb%HY#-ltdfX|i=H$0(4-eX_+Gb;%BJXNCEfk!K zgihox>wV&(KL6gEj>3!v*+y8B{id8KU}5oRG23!}nlxPTI(>@7x3muFpu}SVvVDP0 z@IYOlsT?K6V0vW+v_w?no<3#i9F$0FoH|drp_d?!i4pohBfnq!p2RvYP+r?9OuKgx zhb_SN4$#x%zM)a@lDh&lnzp#TN zw5WULKCx{NCqz8wqMw+tA4TR;etCR|j%2Bj4RF)6AqBY(RlnLj&O-6S-}qL5i=#8{ zjoMgl<2_t>otvxwyMh~!A`e^~eeEl3JY?uI0kzT;E$`wrsb!Mpc*Jjswm~E{Za^r_ zd=2`OKQjo=Un1J8g3T09jKy9&$5%m0MBQ%sk<9-HjDz5|3Gc7%sXAqnklXGYU_g27 zGY{vv!9FU1z}olbmtIB;z_f@C)ByCU;VFuw;Jz6p%KH%bLbyeJ`hD&fz|ozCUib6E zHZ{2-Ve5ZD%k3g%t?+9z+N-brplIBAx)}Kf)dh3U)ipJCCf=rwyab+K`pTEQ47hro zg?Rf!LEysHxOoY^kXStrfi2WoVWEm@Q3PkAlR5M(z*%_;BF3B|#k>0hfw1M;OU&w~ zSD4P)3cLfFRu%5Hsi^u%n0-%Fgy3(I6r%izSTs$g=&NDeQc3UA9+VedJ2mPMPot|S z1H1>^5ZvXNV!dk@@{3x>J!@``ypQIN*a9b!WRQlw?>GBSNFM*=wAWEhALfmWC#SMu z-(3BQho>crc+L1HxcbulADF0TUqI#kIOw2cLO?NshL~Ln>IV4q;-?1`IPKX5gf`(r z(VR3lY6fgJ0g+*zPK^-wpnkJ_+GhE_J69x5hB-X`pA8@KW*8h0~@Q0Im?ZunB0i zBOoHY;oRelhKGngJfL0$z9Zs(@Y7zqYt7=dFp$oqFaKB@T;D)EkBI-kCq^(uUXZFV7>d@ zl6Qe>{c6FmJ2e0)MUEQ9knaNkN88|$$V;}NlsSM2edJH^xX;}jz>!^2K;0bqR375E z1Ki(K9)GqlUjd@yk8XNeAYUSW$7x0$`FMe37Edb<(cWd{Q+A5lXVijEU0}#A)eBBw z>l8Hzo0JhW&2Q|tiCmZ3WFu$fV@_(f zNTZfuaaB8n+q8vZEG)ZB7p4o-UQd$z<1JP1I6?k?r`VP4r-SO$w}zD45vUkV#jOl& z11%2xS_Ul+16vN3(rfqM$7HkQ8(Bx&c5i6B;>`%oH3+oT;Ev8Vn)B@X6#FSwep|S4 zl4oI9bZs=Ql5fFlb0)!k%(26>!@X{yO@(~kI+tY}C0 zj0UrObAn%bc#XKmA_NHd`y_GKaU7U1g>|)CyX4KO-!};JZ{o%n+;hvTEZs6J%XPZq zhbX{pSzPCpqpOsqphj{Ml&GA@d(de`?tDV#U{9E(RwDdvKoCG*pX8;tNgp4B9RZu` z!h@MHW4@&$&@1_s`oIK#rL2Kc$JSQBzijNpXl~I7N;}C-JIa!n8+b;<^y<$Ws)j=C z`iiJid{Y*6(ogy@ctN#amoclsnGsiMWidf38DkauMJUBYcISKm8_0~;Z>-dw-6>3R2Z zl@lSrl?a*jrlahAw*W&<%$^W-1kcE^B*d(*Wr*mOfA{2MSDc>*W-axT$dyygs~%d7 zO=sX#+NinbWwOr1myx!F=S$6IlVrOWKK3O)Sj^=$m4+#xNKR^*hM257)O7>WANyj-{L>J=h}#jiqer^XcJ(L0wOGSqDi=nC{za<7weM zSf(;G`{Xp+`?pWcK`WC6`XVSQPIqY7?v#Ei5M7LMP!l$5_eeJ4zO^YagLdxsd}XOW zv~AGA+gEyfHWAvEBE|H*Mq3_6n`%_e-2Hyq<7ohqRm|%1oxVeoJ?l?hXYN@m(_ee8>tLN)mB+|0?@zy~?0mVWV$tzK zS<5SRYI02uwzK;5;yW?R<=G|rja-=3n?wElzplH-Fy=@i%TnrB#v)fK{m-7JS0_T2 z?bMD4hF4RYPp65(=69jql}KpcOT6?_fQltRD7#0c@i8y6)NEFGtLiWVF zD-34m`e$)1_vE@@vrHG0T*yLifyg%t+xlFmjSEj=dKEeOg5f^eJWN2WEWY zY$F^#CiQyQ3~xn`+3$JTFp6JU4?~=uR!^a=CAyvEnjU`~lF5;yVxrm7)fXA0?aZ%m zD-oOj-2(f#WUU|DF*KS|aBYk`)Rs*d9E)Oo-`k~;lb#UsLA-tHc%rLjdfKF?>?*D= zbwz}~TZ@Y#(P0REc(;j5R{fWGMJtXqtPKab1tSeK=oNjR*s;hDsiN0r0 z#w{q@`s{i4haKgM z{{GwTj}Y6y_`S-}4zrU?%jXoWRL2SeRsm0#ARg|D&>oKhbH0r$t#=m4N%XxwF61Lh z`JnFWJR6FHx6jey0wYf>pVF==8ES*{HZV*HaWX#IV?WKo&t~B?Rhl|3g6H#gdXAB% z9iQ3AJtb@GTlZAXgL!Qd`YvxAS5o#o#75jmQDcUVZPJVGq7LDGUWYztsWc$|1r&Kd zL@=H_RZjTv8&vQm)(e&`Q7#OnMlbpbkT4jWlQWE>A~xc}C9{=Z;S?X0C~~I1b-4%Q zKRr`xbmMZ;N*&$BDL(a`0>d=5taTq@zfnp!$`4p9xh1^I*b?{~mws8KFHN3XT`00!02ZyR$TE7`6c( zM!3T>mE$x^(xT2sP`stIL)`~g`5-nD(>s;M8NbDbkQQHx=R&g}%Op?LcijwlU>qNV z@yJl)xJUM1y7%D#U05Kd#Q_)h6;3Ss-XNBye+Vbor4*QiJ)TMJ+TRR<^oQZV zf^d~qx&Y%#%wG2G@rbMey)a7{6He-X13AjXw5uz1azQI}pZIVO`Y$@L;PZea;VE^W zpoQtzVERFy?bYr**uQ##*G$Qk5iAueZ&kTCwK{j10oYkpu?b@|!e0gdcx#$0Q%Zhr# z+<#w8OA}C<#d&pC;dp4EKJhTs3T1=@=qRB&;Ii|@p4Dfqyypi6G_llzSOzIeK`i>e zn6x(vxL@%e)T}^|k$>BGfKqhB4&FT3)5*Exfr`vtyl|Kk&7b8ie3@69`kF#2ktJiSc?v8dgq?27)00W$|5LvfFi|YOLn$ zjq{s1ke%Ic|APhjpF!TD=NGz1v$~KFHdi-1#6LagFJ!jjGS@-|Tzo!clu&mu?eH$1 zI5|-E7-2m?3jGd@@nAc<-!JiY72~l4ue5cJU?_n44li;MIdda6Jl<24HR=?EiV>c$ z7oDcnvIGDLT{Jjr|IT~0JYBuLy|a!N4kO)xhh7Y(4PnLdZX+fhK`IhjaO=^51r&>C z>#yK;t1@jbW?!N6Is2v`_wY}_Vvv`Nk)bX8^!hb5xz;!^KQMDL!|`Ba`EWx34yqrQ zIkSp)Y9dJr>236YY}xh6zctlrB2Bd{^%`!FyEyPX2gB?)pRpNm^mbzoa6KjZOoE&+d+sXjgv zu*?GpnAE;IgkkPSav6+lGPD}O&NIU@bK@_;ZDgRn)6&4!N8X#qij~B z;utjmYCM|wr>8NC)~dp;Q1Uk zZUZjDg;U`KS!Z{_03g+~k_cc^#F6_;>@OwfFuP|G&QsSO;M!hPkQ%vVssB z%i<35+${WP6R*<*v;F_`9)PA=MnDtYPbU>2ItFvmL&oL7pdU9taR(6b@mp#-BM^6+Y@VKKZ*b``*3D)cU}H!UR%RV<&Ax ziTR3WF3U+t!#P!=FbVS!4aZX$vjeOJdtX5HDcPpohBS^RS~RQCq|RR9Sp0=B`rt9ADY^DhAJ(g~#l7^Al( z^1`V-j3O?>ldE^afA>hpy+UG_v7eCshmppbddv<;z@T1k>ho1R7p$kd3dSag zASkn|{BU-e7Wtk80^EL$27G;9OT^0@B<3_%KYQW6 zmzq0h^j;@Y2lH#NHpd)5a0jn+Pqn{w$aaUw9_>v%Xn@G9bWOHtT>Xgal z$u!af%~nZ5jJK~gc0Z_JgXxO~tq9qqXEO4i{Y!SN`GNk{_W1gUq3baWitoXAO6d(H z0pBP6j~9+y#H4L@M%+ydOQvqHIG6sUjD6}9)vL#}`w|o*Oyn<%0oMC>UULsr@0L$a z&eX%}4MVg_heOFG5nF4IOerXDKvIGJiOlZ3pF+;3NKgLru@%7N(yQ+^*4C zOtk>dkgE7!yt0cij0~Ytf%x;Ue4`Wt6T_9`E@fy>*<^(~m2iS&DnNJ=LCvopgfwOA zZ>Sp0*0?$YGmXjueU5Ipe$rw22j&15BL5H`cD^tVU_EOT7vS)!=Na64KSQ;^g7SY( zwvkN5Tiw0w@kh;UfR3bS|4)sL742zmEtIEIXT{k%!L zUQsRKy(v9tXECqF`}3autH^Sznq++e)1Rm|2mb@vlOVafD{lw7f?q(O^lI}!{p(Uf zHJro9#RFGgcFT9r-M#6yXXueUgu`1?d3+0%i|LMht<{>(T+X|1RZ>dUP4;D_Jn>Du zJJ>W}B<#KxTz)B`v$*Si+$8pw)P#7!f01tAGdQf-0c);Y0MVQ{+-Wxk$#zBpEmYl~ zrAN8!IkT8`X=;7=Xz0zDScBhf;-}tS9oVimTDG}7cJ>`9Vo>0Y24tbiDmEsZM*UfW zy`GOh1#7NynD!rF0JHHu!-N#WJ3%dY{?v`f$bHxl@8C|#s6ikGx!iZeFj_&~63L=# z^s0qPu>bBt?Yc|NTK~`p(Z=`U?Pu-_hRY{HD!IG)C|A>eV;sc-x@SVI_%(hQR zW384oci6qYmB#!_ijwC&=HjL##b&-1x*GnDeeUWtw|R~te>U6fd-cLiol|FqSB;b| ziE94CSS~TrV~pA%7vrz3;wdnV!Xa{mdE(%ZvIjKeiryh{^z}ft6&SlsKWSSEAsF*n zEv;A%)x9ZbGiaIjY{xO^X1}wZkS9R=_o2CXvIN)aM^H?)k4*I@Smu82dGFhQ$eWW9 z;N^jKV5txb1ZR?tAc^3as!4-a@^h2%PMqPt&@XzS-hOf_i|w zTx&m=qiW+mgDE@h;;NnC85Hsq0YhMs!;a04k&)>A$TR79!C^V@WFW=&Vu6U6+4rry z`IqkJmoj)Q`lAhgZx1b42+my|@B<38lAQQ|#7*R6(7k?7X+#CPZ;5aUMHOI85)FR# z?t+%Ut<$8C1=v#$L=4%RbAT*2NaM#fMC9FE2>e)Vd~S1o`{-0)M!#Vx%^MoZZ59bP z-<#D--|wFq^_1Ke`!(&h&|aPpB;jLZ`sAHiySVG`-&Ly$Gr9})TX$}L^V!<8is3q6 z58WBmI+jC=T_3|PC>;31eHm511qIYA_QNyTrImdfUdK+-7|dSdhr<8ErPf*|V5wb) zgB3x-Yv#JH!LiT3sw|TnVJBJCCrv3AF1tp{-F`Y=SJHu=gQdFqyYVI7-1l71 zLKT=rn`2sRLPPD?&L@=gYx>#x9hYwTF`q4u-ok{xAq#>*p6FjD{V#2W6oK%@cB+LR zzc&G~Q5GG=9uZPvWWPZX;}Mh^wQhSAqzMTu=J+F+?`(~v^e9z->j)|c6y>RTZsg9X zH|00-wYZfxQq|L|&1+{T9%3P|{--_Ml_dUx2Gos4s>^QSdk~(4 zf;OBuM1^>SJ!wOn{-{IsFmd9z&WqEmK34QbQeCugnm(<5BnBwNvR{ZmRobw3xAJgy0P%73ln=E{buD0*V@Km(>16*0YgkN*IWl6G^Hg)%94{Mkd z5f}bDiuuEUrab2HX)kY;k88N9W!iv3zS@5689m%SoMKEbC_4USdZ_S{0yVGXdVO2v z%A9>xxbm4pULHu;;Ne`^Bh3Cv;Nb)gWrLhK!W;h?H&PI$qM~A_Xs9D)8t43| zDc*Cw%Vq!)VOdoX*6&6Zsj5RRS-< z*l!rzynIwrK<&W({nd+seJ`-Q!DQo*%yJmMI=poh4^aJDQy0(f$yIYPB1K|xu5m4x zx1iLZnmyLQvG0uYOwOz3*jQU#-TC_rCMLqgt)2OQqW4{A#AjTm|DfZ)*6kd~Q`5r@ z#Z_Npn1>0v6uUIMabfk-X1f4=nb=JBCv>@76Khj&P;kihz1!QfjZ;yFsDbG2@}RK7aF%l4!Hw}j2mK$*G~|JNT|H?B|lM4&wPA!=CqAf_~hk3H@%#TSHCIE7SMWc z{K`Qy*DsAYo6qw3kZk_B z_@ACHsO8IlV1yC^AX0xD#&DI9C9L<|Ta^gFK&1?(6#HT#0M+2$(>|U7pwZL8o@bLI zciO%3bTZ9o_n(12fP6O2;T7pqKr$%Z(jr3Ez*fj!42NWYpTO!b0H#N8;m1oNZShu2 zPx!j`9~CPnWq$x_8z7H~1icuSaG`?OKF5I^0{Mk|dw$zKk2~syg9QDAoGkNSpmf~@ zucw+1m&1z#B~5VQ7b&#Q6_I;i-`bwMMGZU2^@TU@+^>&>SidNY19hnUf{|DZ<$B{A zP^njM*(rX%lqg0|S5#vOWP9*)bF+0Yl2kW*Qsb@);e6DE1qX zjc2I=IZ#|2C~bF$1#n=8$cp_QqZmNyD?L0n@?03g9M59T?E)C<^B&JQ3T`WniK@fn zo*jdrE1^^eX_db;%fC6iySf@V2z9}2b~c~DNCiRx1}4Ap<|N62;-{J^=rGxM$ya|^ zLqKo-ZvbNUPfZ|C^GjFn0MC@J`VkRXV{eaVGRDm9wP66sAA?=JW##UDmdmKsioYY%Jj}6>U4G@ zFmIezdDw#bN8Gn+{}<-ep_ZIJbwSyCI06t~scT?@?&n*(?OPbp1X!f{Ym5JmGxbQA zr3zKf2RxO)e{709&#wb3XzZL=!i^YpEaILzPem;USk;wOC)4|y+AuErnE}qjj1qp><>fK`W z9$6pb;T={6&1nsyishLwXGJts{K2cC}x)o?>SQnA(!-_T!}vbRXVE|Pad-Eh@<#;rDk9UoU(c5fG$8Bypd)>eJM4Q1!%SRkKt5Ws83Tb zuLwU>LAAo}9LV=B#;_@zBE*L0wo~sCwdbun3&S3_014<7~ED4 z@cNt>A>VHE0{Llq(cttnv!m0`5Jz_z|BQ3&RhPXc&ONFeVo&5SzH+jd@cR&&>q6b@ zV1%y&rMd3W_=`50S>-3FL`r%DsrGcSGneUC%yya}KKMNzCxY}i{nPIJ>aVwL<+QjW z8;M3QmvL6lL)vH$dvtiCxP;9XIgS>qg5=bP`T`j2T z30Yz-&Jvj3=u$i4`lBUjkd$2pV*4N{tr(R#2*@i<&tgKpNu6O&&E=3_cCdyeN!M=A z=01(HWspc0Dvnw6kAHc&EgZmONQ~cLRF^;+u0&CEWHPaYO)%15AM7x`Y_NC^jPuCLny)(J}}zI zg};yvkGYqz(OYL3=Lm}?b-T=TFK13~dB`T6`^l|cM4{fmFv)#n0Y0$0@v*zvWDr+$ zGyZaN64Zafl!TvCcmtWM!VU7oK!{|TxL6eOrrV)%+99kRL59G4F-8<=NSIDTt@*sU zL|f;HE}KeV)QN~$KU`f`u<`iO1R51D9MJc*h{+xyyWu5!+u($Hk=``yRT^qp!b-!$ zxx&*0N##70K2*R(C_uN^44>wUK5*+N)2yTRi`u`6*}(aq-VYQa5C)G^B4iXlYcrR+ z5>Im@HC@imix7^_X=JYUQ=B>x+W!#t5F>#4Ih%6qV1^{+1sKTdb6o(J zcefc4Yf35TXxSiZ-iDt)YZ=TOCUKPG1~W^D#k*RlhV+oIqvg+S%Tiq|H)T`eEFYLf zumW<4AWBpt-Q9&y{)|oL&%u*O%{vMUBY-aGlAJZOENQF?-4#z(sHXJRJR8k?0`tm~ ztX(eU;bcC8_rRj*?-pjNCc{bPeu#8yzIA7UT1cYUm&QGJYMwKMo~iX_xM8s@(9ba_ zyQ@*NNpO1p;t{~49#`nO%Y5&Yk32UN4Jy}z~A*UlRQKnT8<@M6~ z;SbIucU#f31f?@2Q4;d7=-D~;FJ1&~Q5SC)_y-AG1KANI0l{?qzSQr2fC7bCZIrME zGx+7ZUyyP8ET!;-F(}!^fI;8l<25zz>-8dxVJuyLj5~5~UB4L%+T!+8qQD2|MA0D8 z5H0H}Np?N-KHT`za^1IdkE?xPnvoAeUt5wFsTvIU1-HFI&*GoXR2$H0Xjw80Ytqqk9-dA3-Y94b6)*zf)W|bT0F5`YBAl0{ z=2pZY>F})TJ^R@*U5jImCMU-Rn+t#x4T=`GEe=DY2@T*>(5B=*!{{HxaTv&oKZz2G z2qB7hHi)JefuOgpq^}>I%Gu$hyHT?&A;`YA{3Cd~+f#rhsV--VUml+k%5ctjscM`b zwV#Ldihz;mU+N6ZF}siD+$iR-Xm-u+U?c0^#Pepl;XEkqiW$(#2pt&lhkCFpRcL{_ z<-kV+N4kpHHa48+o{jQeDQ3^8n1%prCXBa@30K@Hh_A;q0C&c$Ezukdlk|s2ybx@` zC!mZws9`4Yo;tSG(ulCz2 z&_dML*t%ZAY!vD(8H9~}40l#@)Nf6Xda=wOY9`kxkw|r_=F_q*Pp3x* zcWeCExp&6ZOUAcoi{F1xhGOhhX43);@P~-Hjq5;p!$3NPqKh(wV1YH_XE$uKgsbs+ zRuY8^xdLNIGG6+c{i`%#VdmQMgh*8P9pOUNc(O_5hl&f|lHYzgT-Alr*i_O$@*ap; z-yqE{gL>}_PR0SZO)gIg9Jb+)LM*@8dNB~Az@b0&j`X;uyWEGPIXtBtO&>3%rjkyz z^F&XJuIdv#i1B6~3JHm^EAA*xc5E%W8ERH|a&?^KJTj2*%mL^J+8Cd|3}K+0Lh&&R zcV#9!M%G|r%;Ac!2=Ao)TC|52SBxwbTdkZ9?FITl+IPtcR-Gk3w}qOFwDH5XCaJXZ!w^%-jmQ5C|FMZdlm)T`Rj(N|BN{JFRXi6?u@>t0oYp%&EC%e}qBMzz`BRYjbR-m~(zWQ2{lW`DX4?M>(ZKenbNn{ZHQXlZdcTVruPaiG`A;91K1nU;@z(tc!DT|;A@ z;E#x_`K&rPvAy>;@#Hq3y;ZxS=`>-4UT*bJOJ7s!4;Rv5|L-Us?R+VP(e@6!!@Xo1?zUL=Zu3AZ>~*k5DG`yct< z-arIPq#SctZ_Bm$eD^aLnZ`a0EPestO55M(u*Zb|A7}3!*VMMW4_^^%92HSO>4*qY z1(aT*9=cLgq=Smmi%73ADk@zNDN=%hpdui>Cx|qqccev$bOMALLdm;!sGf3v_xt{H zKI(yF@3m&lntA4#XErYGe}rm_7%ZY?pp-d3BxNtx8TV5xF6^VbZojm%wF2$QEr27x zqlba~T%Zv1i(dvbv3Mqw|JFo`nAtb-+bs{4i0ny^9aRRQ^6~r2IIYQFq36t0Syi4= zh9zlvR!{%@Vr_qQr~iQEQ}rq*G(!wRqBLxUY+2-Mgz*wAeEV4lmJUqnbTV>$agJldPGQ&s&&VZ(A|9~R0XIQ49{?_S1_T#0x9g5v zZ*#RE*RfeorL~ zJkxpA0cA-orVOE-buO^KyFsnQvf!!UZ(tp;@@-!RLc1rh?om#z$A;cR4i1yIiYxZZ zub8deS>S`eG188OrRRy-#Ha2^@YFo78>-vKXaUHn03&x)K?}T}~ z@k(#;)ikMZA_HM7}1rgzDOz{UgDkNZ4Cp>v{xxqy1nl-x2gwFxnN3dt|5gjTa>)r ze>;_6TS5EJEY$wft9kf$n@Gr_@YsSp*fqH+vdz3{TG4 zvTo*=yX|Ww9D7|H>I>2S;TAJ1FG4EDh0rvj7|;AKo-uspjqxl(^0Rk1-g9wfvt+mP zI<)l1RJ5C~o$U89RdmcALo*8JQ-D|?XlUJxd-w32-8?d2Js@ZkCV()e8R?-mW&90d zJSf2Kw(}={DX8uQuctTfEe78c+_YN z+j8`jk}5~{+8HU+NaG%aI`mO-SIsF+ zj%tlWu3}z?hDl{fMSCn2I?VLPR2^MhNqHW4A2;R0a;EuCdHQ%&qt99XL#vaK2c%ym z#a&TnF4cNHS#A>OCm(kYw(`^5lfgwsbx^+V$E?Fjur+;{{?#}7s^`??rPE1YzT#8a zd_;}i#+s+)!4pBgSXugc(yvgb>sI`mj2Or;G~IV`0^?6$r6hKIIkoN*$Ojm9kUHuw z;s&6}MhA&xUF^MgHz^L*l0TC|)or=URsUC<%U-udoOrEOsLLUq+%Is_Qe6Sg9$sgI zYE!*}Ph+Vv>rlOAu&{{*ywrVn0`Y`SS$dzWIPP@;Zj2f`;D&~~kA>;92%d5^TpliPZs?sVTFWXA zR*K`wO~Ts~S}pbZcsVs)5Rf+m)^Op=lb7cdy9<+&o@`eLowQ+|pc&?dfh`S50n`dG5Lk|B@HT z+f5>8774Qz$O>VDG+K0*om;H@nv%m8g%moU48dIFWS&~_iSYaO-S{6uXjzf$(X!98 z6*UfMKh!GGyn_!YD3q323(4vzmrKD{SbbR@pAOa+5|sM+pkVMDMyWeGag}9Izj$cH zsC*^^Jxg>nJLB&2T+=Oc+y|)Z1&M=e4&S;24g_exv(}zVc(o2<2`rs~q@LMghp{zI zY@kDr;%t5-(z8ntKR{y5crO9UD2U1Zb;O%yCHQXv_}-(36^VJ%;p__pxP7oryl0Uo zix_6`<$n-a)Fq1z1`c2C4+V1$t+0Ae{w%J7r*wfL z`y7wQ8C6xD_GG2QoS1JV40U)`mQeGzSp_OY7#6iVcT}pirc}Bs1%CU$8;-lzc0w3) zF@U~d6c(2arR=YYI7f;iwRqAwh^W4DL4C}fU zdz*gI+FM|SA4xQD-8nq&V@f%4^7u|gOLajl-Xq#`T;@^?O8#x8`|87w#YJr zj8fXgwdJvbxr$5JF56cFk<#9rg7FgR(`l8hlcl*yl_`(|UbCN5>{AD^eH&ZpY%*c7 z(!4@DWtIOlTd_v0_jItZC0YQRvs;m#KHAu4<_Ny5zEhzapM-imD`}wBjxGbP+Neu- zWK1+9Ae`V_ei(h?%!P>`Q(nTsCnV3Ep3L)EalzCd*nc}^%`H{JZu$JlToq}Cx_H^t zwOxP05*n!bWARsGC59LKV@NZ_tO{ZDJ{ej4i>3jq!2#atOV-Tp-&chvAHL7T60&c% zmR48qk(W7(5}r}=rNUa<7J7a}&+fuwMKeO?UApB_Syg*7Ycq$dF9yI-UnG+%Cx%&3 zYRw9`@y1c>_b;Egc0E<4pp-in<$|%rkN1f`n=>CitX8x*e)3YpnNv?~JV)$J(W0ay zN+wGqv5gDzw_`4;2L~y!9l>hIxVxvB4`EDumjAb%x~`P4ipkS9 zH%tVD#NX;$PAWHtmZvHM8IXSFti|&21;Y_;ggm%{e%nmPc!tY+wjdzO5nC3>kX-F8 zZo~qY#dek7ZpC9r`YV$)eYeL3hw0_eeJqj!`%2dqxt&KB=4Ls#Qkip?{C(}!)@siv zRif7<&PH_FnP7A%G?l@vE!w6MN$q zrz8mFJ*Kt0)t}wMu$~AI$Th4xZ!!61@x0C-uUt{O<}szF#a}dn-s+raHjG_*6;=m4 zO$c_$^00Xa+`T;)%Orn1PA}#8e-H~4I_iuj=XRxfk87yzITW8VEs6PA(~4~hm<-HJ zF}kLFNbgqtL9D16Kiw*Ek(A^Rn(ZsU=qtI%e8Pa`oG`tH*&B2+B|YA_};en-#>B?0b~Ew`J1&kzxm9{mKxCBA+8Tk>E0Jw{_o z3H&ZoWuJNVXV16RlhZSGFriASHX6d8KUQ5ae%w=N&b8HHQS8G+rE2I>sJ4@qX^_61 zPZ<7%fg*vWB!?3XP_!84;r?rZ37b?ZerClh3d96P#srt~c803`(iuj|m-#G#!o|6S zvbx3opcK{;(PR*t{G0^?T~LhN!To2`qnw;%oW|unPl*h+I1aoW)yNsc=0?T`$ZM>q zFlUa=I6z_8*JT@ew7gkH0kO`cB8UOWdQC_^W7=_e=EdQV+3Kky+_+o^=E5Z%YWYa; zjMrDNizEh;Cjl+&xiE2KwFrdB20wgLSnJbyu1Yzr3Yac)st0ijmD)FqTZX1H7&`f# zOsmyd387xgc+L~iZrVL~r*=1-4orls{jgRr5zmzFKR5hj4#PBRUe_iGcOewC>__Z5 z3i5%cs=*T8RUO?{AziCF@W3yEsNzqFP4~Za1{4k(eEKC<)N+>V5xtt3nhL9u(HbmJ zcD1`}G|KVzi_)skY@-SvHiVGASh3CsOim-S^~UUqkaQ^vxp1yLj4`1eafqbQl;3IuLN7k&Tq79H7Vs zLf9}DZe{AuMme`?!W24v#N~Rnd82=hu8e?SsoOw70k&J-wOB};wkEaAv6a=INl;4Q zG%o)xa~?jDdcOgD2^Z?y*w2)Ky59V0d12hG*_@~sM*k8iY}eyP{9NpDE;1)uMfI|p z^r^XboZ?4o{K`@rH9{a;Y*A;obeMmQS-l!tvY71foy4L~SV?KTEP2U$b&x@BQc&Or zn46SZ32xMcT9uol!!02F5Lk2Rz~kYbKH@%wt@;kt=NYO$9yasoMT2JfZtC!?5l=s% zjALMquoE*erTBn!06&_zN4WEcd$;S2*I|3@P-1I74f_>5F%Bzw!i_7`bZHuNAOh6p zX#Tj=cyQGGI~0{+<&~09ER2SU0UVL5Zi@~+ut(K z{yn>RO<*7cz2X(?(3FQyIeO?>PQhFRADoS0?e=9EkKrrxl zg`~<3+5OYAd>`q@?c`7U5N2m@_by!2@D(A?QfV6r;7* z)Kh$Y+M@qlT3Ipc;_~4aszc%SpJox!U(m!}(gEpxm*KzS>xl96#x5@s3CdnAIjl1FA4iDs^SA`qc&`Qknr5W@trwc|N?>>p^ z52d%fp+68OZX)FVteWKM7@Gy~kMGr!LQAX%_n!+RAJW>Q=yRc{N>?0PU~nINzFJ~z zW-oOHrrXDcU(_iriX#^N3GXxICE|9S_EHzgSyIgdd9&u+qQ;#?lws z!7JViFU(nEKLt$1i>B^++sO5<-t_ z1)kY;=jA+#&0xaSaKiDM3AUzoGQ@%5au7>3k}hUaR!)rY6A?=9d>CX8y4JOQ)Y8pLgttYz$c#4Mkt&qLiA1>YjwR_aq@L$~ei#AVqg4_{ zzDH5CGoi4DUVEPSvuBMC$`9&&%E?i}%(^xPjB=fRLEf}^ z&L+^L(W$|2ruqPO@W3x3la5tWPxxj)R4Oq2En5QuF<9g=9=Qc0k;~Px;)ZmCx)X&v z6RC1_N~T~b2{BghJCG@z6p*kz;i~^m%B_D%pZ;#V2K-mQ#MCn_hp~|S!G;&bn^-{D z*|0N7&&!@t0I*`{%L`h#(VZF*e4G)S)>M7jllfl5LEpyFVI5ro-rO2~E=QTwA4V(< zf~zYMBP(Y@Q7rCD?c&dJ9)14dK8zGi`HcTA#Lr}~LLgi`siJ!SOh zMjlj*prmwBOmE@J*m(8gBIz!(TR$56gW5hs;IIV86K~Hu{CREnwZ2izUU2l8GTsrc zYH1>^r+cHRJ6g@p3;bcbLUw}yPc$+J}M(71ZzEYv- zuDCP?yZl%!Vzy$fHH01WM8bOD!*-rV=?-((E>S?e`&+pQf*EjBzT&^o>334*2%^h? zdIBd4sHyLV``z#{k=@Mst*+ec_Rg=&aK*5*g8Y&7ihzKG##fFWR$@=rR2{scSx1gh zK7O{So`V}of}h#rKAC(h9JI?Mnn^2ke`Xgrwu;y5WJuRBUi8eYu8;@ktdis?&J_UV zFarRnt%5*KCD4e8P^94*jLB=wf1Xm=<9V5IEl_fPDzoE;XT_UKsOxm`ix~IfFAk`@ zhU8EmpQKZvZJ)YAX;l_v@L9?#R(iI{eV*1gGDRo~Y2ALlS1LZ%*fgIXnB&zD^d0k-#Ha|b443r-UYd(Zx=6i2m2bS1qVYg90$UQ>`5Wi z;IoEC;9NEjS9+i{T-7G4lb}rfl{)U&(%|UcDZ#6WQU5{kBlNQ6YIxICzvJNOcNX`1 z4k^VQiCgu-+z%6yiCY!5|2lXhn3O;9O&~^!8+J3eY(g)4c%@sKq#^ys-JsYoWe~+7 zwckRW+=w09L*6ICp9uusR~p@I*l3`4(Ns$Tnni40@`&6je{38d`g03e=M}g&n1t`^ z);=XbaRu`y>Pu)!^=pQ{6Q&x)$6nMdgI66dG9cGZ+4m&;>4LH|U3IkkY? zDbcnRwjyaLUbvW5Q%XWfWe1fP9XA00m(!EO^x15At;E8+#~;MoXFG^%cD4c3okB9JWC5QUen41@tBjPJ9i81f-?3k2uO-si|E=fIE-owPh()D9ra~_%CO(X0$ zvKTAcpA1(MlxUjWw>uw4vf_{1kaiX84S_4J2CNY%?bw=tB6zcqH38!ME zil;3z0AGd@uL7M9Cxx3q^{#$G2bUFx1bSFKRFizaN59nOx$I|N>9Y4k-sc;kW5pst z;DS{#n~g-ei$JxGY;_Jq$vz1F2=xgOq;CXE%n!oX7G-Jb`qlST{^$8T-Ctcpuk9}C zn@W)@Rfg`dH;Y0&Id;ubbWlf0)KY&jjyhHI{;+do1;7}-GF#41;bQfSKb{r1h(_0> zKjI4{sAu+sAGNSR9PWps1M!jl?b#vnG*BsJ=%9(Adr;lXb;4wiJ(Ow&K@iv^KuX#MkdG4O6XpN7C|s zis9#j@`B{NKNQDkb{H1vFF;x-{Jpg5fnNx69uO?f($YFO3*$?-}9g;tf5*g@e zhL;&C^hBiI2JV`U-Z#7j15htNr|^~=KpezM!_1wuqsOa>oG~1O% z)lt`TTsG0zRBhIk=G|kMxi@NQtTyia*N^b!pcB6KtMDH~;{)%=dxZI7Slkw^~7=7hQ-O`CDz1IN4ahuK6DSA8I zt=6bOTc{FRd#ihi4NFGZ97%IRGtl;JST`@kQq60wD)IT?`S2kNv$@d}$C{O6blT1>)lYRR&Nh>Bpm(W3SJk%$-CWGgO3Szu%q}AA?1B=xYbm-nuCj3a8_}zuo8Bqt>Wst+${8+M)z)mZ5e#8R9*NKM|)H$jR7RQ7tJuOY^5n9Q`pfb=ZrmY z4WT}YZ;?tx^@#*I)(QB6JeG+Qer%O^c;r(WfPwv~z0S^-4!m^Mb;O`E z8e*O8SD0&KpOx=n7%81LuO}uf?h(56D|)Jw=`dy?11e1(2)NdIvhAro`t?YhnwR@q z<>I7yTqoLs3HZ}H)WDWZ9jbHT!B6c9U#<8KqE z?_sXM$EOWQ2Eq4-V|@&a#gXu{v#Ud`E*Stmb1ESk4*T)fizicpKyOK7cCb3*%mpVu zA@6ytH8*hF^ieKTw!TIP)>16S*XU8p@Fc)EmAc!<{jXJ3`z4C=p7M(YU{Aq9i60(d z2GW-a7L&5L6})mNCVP3FsR<06&am7Yo-?kjAl11}XB!E9@S8G$rZLE}2It$D{=&b2 zMA0h~^x~2#Z^{!!{W4pHhbsWoE|=;j0w_xC;3WnEB#N7^)SoLIC)SD#U!5>lHN#$| z&gIlpqF@o|noK(z@4bR37k`Nal?8_pSM)LSwL8lN09%kUibYK>nBETA&rt0}EzoUn zOwbyRxN38|mbV9;E z&6HmYQS3hdqyB^t1Fg#1;xrr!rCtW<#7nn^A5n)e8Fb;<`#KktrRhuBJ_nz+?I}Lp zWI+LH)G43%nxOI^1bMa$+m*kwr*oMv!Vk1^w!3yFbxnPRYBt425ndQ5LvdZjwuJRq zLwJGR{sO;EFT5V2&1Heo46n?8erp5;92B!gUQ|u$Rk0Y{ayDV^wwh1x>Q%n;l~BGA zSo~v17XrQc0&57JRA53JAssK{e7W1CTLX z*S%RmpEwY}mk}!Yt%O>Tc{G4cgfHM&@&X^+dREzlJ`)p7dM10?nAtM$WIc5yJf85v z4djf}!#*6E|41O2p}&PGp*qF7SrjG5weMsR(W01CViNsYFT(-qONS+fpOE4IHv9|I z5u!m=g1g*H$w-j=*yv4yu#c7sD%1N3$XkkyKZ*_w6NQK$p)LyrKi{+tWSBEE>BM_D zT`pZIaRVrLqB#A#6iF|U+75F|4(oAR9ZE*EACHU)r6rs{(Ma!$O6_c&0wQ;!yYA&* z8sJ8gG;G|<_Pd;*CUG_`Q&?VDjC|tFdG%qIi|A9pBBSy2cXxGN;bKgG|#s&WUxeK7Vs;_nt$+wudAO zExa8o?_a_APfa@LPB^!i3_#&y+M!rYLe{fJ{>g!I zrCR7GwT#v?(pcO?bQ*i*wT_s;mzD|^tI$;@>XN(=H-LF1PN)hmoqhr;suZS~sXs>pY*x=c`;6vbAX8<{A}eE<1Q^fnaC zHww(Qzws6<{q_v1Iq-jT9FJ{8#Xa$#`3*zf>XUZTPO#UpZ`z9G9>6lFqkv0s=7x5x zeI*v-K09A0=`+@f{K^j`H2|Wq*yrTT!Ep-5IMbW+bfA4Wr5Z%xeg+w+vujS}yL^8= zN3Py%C*Q#kKS*=&XB-RZ6)hRP4Q6`zUQECY?N)vvy7aC6C5#QyRAl@Zt;MPJsNrX_ zQE|M_bb0~seOM@6foe-iwy4{&5@5E?a8n$%@hF_OGj=)k;wq+5lEVjHXN<3nLme5N z?YA86vv@uvvee~`)d8#~=He^Y41H#Pa*_TRtGZ_ggx-A1Ah;HaqVr_u!XpVG=V#sY1EO+rdo*CSB~0vE&Xlf;(Jbc3#>Y zN?QRwMsA2yH()fHY}Dh+3TCMLOCB5Xu&1P7k9rFCaXTK6Lzi3m zCcTtQ!L5<1VjI^)L;5?@4R?yZY15nqLlB@-qv?P^(XoBrJV6L`%IgqQ74Y}^y>Kt> zsFpzNS@~iSqow;?4l0UGnUk`nalb6ZA1(~ z1wAzTIV?M$0~9~N#70~OI$9i1x7OS#yyuJgcoX-oxNrNBo{+QtVm%i%?Faz8U$Coh=4~DKHo^grE&2K49sl_zq6FtpmjqGO$y6dk>D<;B?45FvR~}`_=$ZL zqP&d@N@RkKfINOI;E;sUzeBIhP}>%db<7)NG=l8HJJHh27m{u`{anr!s_5=%H29tr z8j~R*@8kN*P612!#>iB`J8r9$V@!omJh~o(JaXkf^tPJF-xl#EQNz3|ql)qt9Etk%an z?zr>HF9@B?#E)*UP}8c~4;~4Gp6I2w~=f zHB@@V&HS(?-E34wwhCeh%?C#tPcSgc>JQW?u^R7l(q&E(AXxDn1@8+7pI3bp2PKw~p{+E4o56d7I zjA6^xiS=m`1C`x_ScaYq)}xBvd<~$!Ti(F&9#!1>%Ko)Nv8Q$FazA`A?nY-lgX6(h z#Y=5`2;z}Dn#$BPt(D)FLQ-0nWOV%8c*cgK;I!lgj;5xJ3WWyKJ#0OlJ zCRs^DY|x`q@yeXfZDPN<3i9ya*z@PMPo|FF7OGbI!R%Rcn#-5 z^xQHjp4*t-vYB{DDV`OgKOJAdg=YS`&*%C+BV6J>l%TZ8G9I;XLd?0RpMQk%URLOq zO-M9#2~|n?mCLraQUW<{CDn^(S}qHNaAQP1f%x^-%ZqTpXA>=EmFR+#%*%N`+V09u zje_JtQc*p|`OxUA-@;_xGn!9O%Mq^+8~q-|AdHV3&#Zh$82DyA=LX_?bMr|X5deVC z7fjuEN;ph0FZi6^-(F(uN7r}~x&rbz;AEXf6?G`~>GSj-4bX}A=2h=^%`SkComVJU zQQ<3Q_VlR}C8Y%9X#t|VB}ITx>BW!v*br0XwKc}4Cim?pRcJ3>ozxd76@vqbwZG+JUx%LPJmo&GQ)h4f zNC;^d8^2pc-FQdcjbJ8SEpn;z)vPC4e%YMj%&F;67p%`5N}r)+uHl(YM@ zRL_J_S13E~^1V9kkSfJuww9p&Zj8umscSbw!qva7G0E z@j5bwx;&_%h~moOh$X9^24%(FCX(G6#fu$;`AMdV*;ZD;Zlms7cNUMX9jE5ca%&a? zo!vZw&%@GN^5^V5N6z@#Phh|)yd7t|=vNt`F9_nxYb#YEok>(U6Wz=+9es9~uHk$g zh3@)0fDX|1lLsyD(a5uqhGpr07X~QPbK@pR>T2_nsJL@2?JWL`)1lOb%M^u0*_a2r zN@*J*TFP?DimYAX%p!gQdw9<6&>5{cz#tC)u0bq*T6X|wf1OK$b?e;$t&*W(N~bmA znIR!W<-Hd+yG!;t@g_hQ{n$mYf<`i@U!2l}gIPw+WA+^zW#J;uA0P?g<$CUNy31)Q z(G=8ZZ}(DNBa@0Ca@h?~pzyUnIMtH@|Ng{hDlT7iIja?RI^ZH0wKHVmmZ=<&Ic#+E zwMDnvQ>gIqr#=sq&<=pQFXO894<ly=+|rw^jLKg+P-}`sdoy}(55An)ni_qeBJ^)ATJ{S{*>89fa z1f*+#fW%hIpK<3WNUTpl#nBs=3>_ykpgL4#>*2n>4<*?F)l>0|5G~7pbw$<05aO~( zjVsD-T@9uCqlC8pasgNy;xoYb!wsNbitc1k*h=c0JIFuCdI?UcdZcNifBl^iB4@;D zzTU+jRD4f7HS%)3)$HD_@(T)~K!%t;R)K+wd!~f$dh|g$^47#cSb~ zs=jvp-pln8q<69d$QN*f0>On^i00}dMV$C#krVa&J_T}Lpp?)3TZ;r8%NE5 zuJedONz#ruBZgCC4d;J-GiO7~i%y!e70au4TVtc{e=p>!ABAf0>V5D#Po)nhqeE(# zFK-CO0?Be{V#?OaEJEGHjrB%gY1p6BDqYP|_7HgOe&$NfR;$4~Q-qL@?R=Cg!uKR# z%m=6+mI5U<8DfoO=Xe#U$WFD_^m`7c)!~;9NAZy<1jv1J*$uKBTO!phu-#8z8Vb<%z7rfvy`VzqN#QXqEB4PY$?aw3< zy#?wPMA+OdZo&XXrx4Y-zA`$4X=S=hcR-ELtm6#SRf_OGFtRB`w$5w#05yQAHa+=D ztGOxC0Hqp<`@HT39lYI%^TNPazNmj}Vv`bzTD7tk_Nspny$?Z8 z5yu^rV`2EZ6+$c}e+^!o+*Vw;)R_Uxb;VvbkrTc* zEyMEg3Lv3CG$3-~_td7qi5av__+p!aBK#l_iM80;>gi&J+%TFN;##xhX{kJby{GtH zCk&WFF0UpkDO5kGsRVKyTLzNQNku$-yldQX#DZc9y{vBL3=~vFYMF{OrzyMlm8@0) zk+u)6OMnYr1sGzqPkk2E@G-I|7O0@0HTH1IH&Pd>M$Ht!zhSWW;uQL@%VUqeEN)@} zP^Zw?m40Q)vC9c2q_HW~dC!H91w$qK9Tq-I(;FiThUKB!i);j6-WT05w~o5S)__O< z0kWg?hBjYA8uZb1`P1eG3~isaF+Shnd2m2D8^5`bteoIMF)R2y4vsf<)yGg{=6w5|2lW2>ki%}xmp+! z0)!nPF8urDkU*0uJWj&wSCfQK^?7PP9@1(j9Ui2zbj3!J2vC2+4~S}qEH#V^23a@8 z9EUK)pUy2l233vlPgYR75*}2@oUG+6b>7IY*i5}y^eL*LSb8J=j{UhKLdeTGw{G7s zKxpb_^aW$hQBs@+yD zc9yrN(2=FB%4yuiNrlq;z);ix4O#v9Gcphdd5yrPeWw>xP_v#tyZ^a4kzA?((Ru18 zU2o95Rs}*$K9OW~z}$nq$8;N|yFnN`u1M9fVC^t>E{h{@8qcS;G=7G-Ad|~PMLF> zB-5cbY$D+&CFMGy?Hi=nXcI2ADfSa?00a5~G&Lb*2BOC4qc_b^i*`dR#QAX~q$!2a zpfrXLR%!@-aB^woGS{|DdItPCb7!nL2LK~(t=Rl(f{Gmjh1{$ky9s+vYN|oq4tKOu zUWCa=F-`+sgFm182|!HlI~m+H!RG)`{ux-s#nv1?`}zLLC{9Oz{G_Jhu?o?~E40Sx zt{hjv+N>mg0dO>Wk(*j6m&(eugbS5(8_GvA2`zjTv?j)9z}5?yrll>s>)Y>oHamIB zE>ch0=204?Wd??1CWB%>=$^RA4`5l{h4JwUJ8r|{NRwOb#$0+9_(ld7X32ptPt$Dg zV88GQy|KrD)y@4R9ep=YYVC057cKJ&z;5`^W{|rj*~PA@xXWl$&(A{x*woZ_CUVO) z;}uv5*|-uARK{+^%vZra9d*&sv*Uc0r&<*_5D^rQhNANa?`k(dklcsBR+r~FUoNQX zMi=xD$-NSOoU&m+Yr2l9i+vk8g%@(HZ-acck^x1W_YdxT*|4`k*4}Hi zxTn&`+>pUDZY~l;xOm@jC`C_doCnatb%J3^Tv!Ljl0<0E5b}d4($DAe+El+3i-hNG zoHax+Rof=(2Ou<@pho08_vtd|yXmGS511hPZ$t~|Pn~ZB{K--A>xF~Ml`f1Y&L(^l z6=p_s(?z*{_7&yMFFUDrCqgG{+OgIvSFKN6h&2Y>@kLH1NzS@rYP9 z6Y$1OniLOr_F%)}q1i)AqyA}p6y4hkz^-`VWD)00$u~VUjC_b!7E*c|3l&lY+IXNgQI| zt-U@Hf~M$E(FL|_aYph!i){_B-f?TXQcw%#-5H7xT*kf+=H!$7A5Uzqv}Mg2cV)`#wG-nv#7Zj4dgT{98UNSs4^&l{5XLu#%3(@( z&AdE<33g!UWybVjKSAh8C_OxdYS2`D)nEcZN>KYq`=I6gXsbBHO!hq$h6|ApID`i6 z4)=RAOnuLzUwqe5eWQ1)K?WN52x49Bj%AJy>qrD-u!@dk~tmQxB|NTykw>#hU6*>>8q|E$RhLh~?1t1*YDKQ^uw5Tina4TY&%-ZbQ}XWPzEdrwl^23U;| z$%Q{8h=6%Pz>QF(ivl-UX8VPs)B$qbkIfUoNZv8)N-hkfmtw5IL z6wXErz+9CNk*n}bz#L*Q9%ek}s!{ALk5HtgS)axYV&pRBt#MY>K*;&xObzof5}7D1 zv;WzKT!O_}#D~*C=c3Z?k>ktlc3UuZw)M1h1=L1Ie=P&K^m=$)ikl;=tgb0@$~vV!_R2DdoCpxC5Lj;%gGox>Npme;=;O8{8Hf zvD!v7?*u7*n6l*p6$#CRL6fBUPki~VrxG@$vo$YKfcN*yg?};+l*PRwvtWEPd`h2~ z>3H$-Wh|F=%@<&*0WiRd6^vrEwktHiS6PuC1ZKbQ_zolt81ILZ5H%0&spH$nUQc1X zGqlz;1NYV4J?$)r`}=;q4FQvJ6{SwA#~xp!6%f+UX5P5t)(^n{B$z=nVhcX;C7eQ$ z9Zc00II7T+|0yYvgM)I%fMbxtJ^TF$|84BgFF`Q&hddn;3^BBQ@#n<3t4xiR{F@6- zxnqX+W|FiiDYTU{LC2On+6s)A*}db1{#_b)>%2pM*!-~_6ClYSk}yu;+UKm(>or@C zpU7@>@o;=^i-D6?m8B*7Jb)5Q6_dSFw|zY0rs-{EasU2+knOyKru=+kV%iuzG@{Gi znR_PM6zRR-cSY>`sl0BhR06RR&1#PHb}_3nvRo!$L#oYUZ0R9wZ-Kn7lQ#@(b1z<1 zHLKp3v9$Rzuqo=Ooi#n^C4Q6qX-fd#%;?3F3`B;&=+(2}K)(XajCdnu&@0=b787tF z<0+euTM2P2)pvHtq^MqHdCoH}<$GP@@pAiAyx6orRbjmMh z>}Eryk!v4a26~McD@s1%>o3`)u&e&Re-rpQMri)+BUzW9oBsbO(34Ya{@m}gjJx4% zWZB~q85Y4(I@=%(#&~U9i3fSvZFU@N$ozH+S8TrPjF4t9kbl@3KnY&-55(t3d(Fdr zbaTc&r24qcG-nK26qg_1t1y?FjSlgcBU8&IX=#4bJNxU-YP6KKHfWyEyv1cO` zW4v(Fk1OI?`wfFM?RT?vq8hQT{qC95<<$ASQ>s1ZJ9Ihe^(|iCSQN^x6LA=OpOVMx z2%!G715xA|3!4wV`Gt0jNUlvRV9Q?D|FEHqGX_EJ-ZI?T8$w`Xnh9{y8vd#L3Ws3v zM3%&_=a&wsus{dnV{V5&?#-xbPbGtij9t-a^2M32h`p@G8BDw!3?1vw;|m+T(~Zyv ze%HGqTf{%F?Zynb=huLLk@Wslz9}8RtvOi?vN#65w4AGfjOTg9x^6x?e)cA;nWL!1 zr77Wp@2?lKi@l;x4zcLxXaG*yj+X|vmFF(5zy8yf77h4OFr_#t^f1c`Sw(Q^7|rjK z<^OVmcI?yUsROp$2g0kCD-!vj{KrI5wXtqnPw^KSBk14%_WB4bs0JL7togDFl5!G_yd#5%kiV4yblmET~3Ho5xLIv7eN3BO`_r%eDmx?`R_9 z(Ec24dt%4SxmW44%o$z#!cQiEQNX|1)o-r$p9}@zK?}lg4UJmE{cX`O)nn0+9Al?c zjK`VQ_TS`O!N>mhD`b723L)jk@oK(;7OI?o+}qzC@9#zjo?rjKFHy^+yN~|{#zDf$ zzZurQKGLT*P=CO;)ZfKed9-2Dzc0@}e*gMtKOD zY_2&h4$_sK-D^SIcxS}ezT^KWptZ3vn?JTK1W2AA8njSRxvm`__TYb!{M!C?@Q5&O zuo6QRhHN z*(QN!(@M7mxVFD;^VouCXQkS6h>1KsVBG~c8L-2}RT;lxMCrC?H$$H_cfD-&cf;DD z?e*7>ew=i~o?^#3+!tbp?f5J7?mM6@Z`*M_KZsx(Anv;k<3^a0^>%U=Uo2P8k>Yy!H zV&GAG3>kb$cZ~dpzl4S4?;*RoIYuRj2#r||Ep1eZ~>Y`>%eq=`z^-Q%5472!QTvEw(d{Z39y{uN3N|0 z3E=zh9~cSP+Pn2XZ2j=|zwb2ErWkM%J+5KM;~JWjDQ@pLrR#Ph^v~0eHEE7Z*pPeu z*54SR4-F_9*Qi+mIg{3|*3G)b<{9~yb=-Nk!%z?=yS(I|etj42e_BVF?@1dgs}Guu zMSpqhmvl9ETa@p9o%_AZ3g`wrd?2~`NSj|l&P@7g$Sj>(|3GPyrEj|M`(Un(wbXp| z;Kugukw0$Zz(9k!iv6J|5vqZ+h?-qCs_Qwmy|-=89#{ekFa?WmHGzma1iJo`(0|sd z@egdcly&0D<`Zt2-THDZ(J7OL;=&YKtbG!XZ3=R2dZdJYu!?QVN6*12&htJ;IlLWg z>o(5+=ty3P|HCN6^%sME0l~XvffA5DY(Jum$rG{dKG!pU<7p2MprrlS^Opm(ecx>_ z%kLW>2R?4}vNB*+$kKi?vJWAsmg@cQ!PNG8gd2gw^IgSRIU$8&X94ZM+U({5T3_Q* zA#iK7G{Q*R&$@F9cR1^Ra6`b20F~|IuaiQV<*svV7M<3Azmc-{-*Si@Pq1z&Fc3vd z?TjL31li^O534%z1o&jTgu<K^r{2WB;!SPk&z|Fdz%q z@l0maF|cT3I%Rd+hW_7g#_f|>^Ax;?myD7Y&AadiKL7XM2QkY9VE5J9uw!dX7rrvgj|1u>FG4m&nWs*#=$6xM)mq(hmAlr=(`R51%0y{uTv^nu=!da%7FIg2>Y zb5nKeQ=Ec}XixL;!wpoJ61)vH^NIfIj7IAfL;>cL+hrzqd+vjF; z4CmH&Z>LjRF9{}2gJFw}-O6Ad>7ZeP`z^-DBX@B?Nfv+;O1)sp^O_k`$p-d*{(m(&LB>ffvVH+-r;S zhDXh>^dt`I+BPyNm3oHe&0C@Dta}%VaE_KJ^+us_i*);??iugqnG*Ly^%;Eub%2cp zt{S;rvO5-+?1%muvV$LkY*{P}97RQs?u}5O@0f-0C(KHB$vgDm8`98ho(e_%V10(b z(L=A6t(Us%&lIHvQ|il24b4VO1%QgxRLoFk-c*KDpJC_9tA1qtaHa9J7^fus__K!V z`@>COAzMbXVR}0Os}29SVJRap;F zEXKon7H@jM^8~CtwxC z7x-Xr3_}{XJ|tehQO(i5+*XhN2Id(dgXh*FWv<@!`m%FT6E=0ww_kBh-g9xct(r?G z6>8>b5=bX%z${!O$NuIOE*EhWwGjJRjz1EP{3&?x;hQf=%ey7^myaKJ8og@x;sj8z zus+2z(sEaH|bjAYJKsdG6e^G-~MI-D&M3uy;vj8vr%$! z<$d@3*Kd>8a>p@i-d0mm!`e9S9T6cS=)m!FSz5-@0JJK5L(#$cNgCl4+-N}GqSenH z+S>Rf%~hs2lwMTA$T7+~iRI3;a<@Wv^;xy|*SFpXdrZ@6`<#*gIl*VwkqtPS{P&x> z6nldH82942B0vGfGiCcITZH9yE3`=ZL=WWcJ}ub0G>{yFT4P&HGwU3cHko}m z%G`?YbP&AZmv10+%_QrW4GM66s^4Z204J*`c(bX7 zg1d=#{mD)vTs|r93A{s0>T3Ck!sTEN{l{WhsCVg2a!l4_daFvNu9!MPd+d6?aWcwf ze9~9i#A_}&>h3v_d#CJKB-2Xyvo*${S1UV975-Pu=-ofe%3Y;6DFZ!$!(qqu2e3jo zdC1#-QkELcXTi-zFC~4rMJc`?(T1gVMU9LENt{r=|FdCle?+#$bV_cV!lJ1x1!dm< z$Jm$0L;ZdKXOJaDDJe^`%Njz~X|rXE3JFs}vS#0zw|A-RJB>lv@~)6I+sw4tmqPZf z?0W{IF^u1x8A|WZ_w)V!{_!x(%a6%VmoGrc;LeaQmZ6i^!Y0~7=X!k|5my?#O-I3x_hVpdn5(UJ$YQ(gJY@#PJ;7<2hv?~tIuO(5?<)sq-zh; z7r)lmo_#q<&%BNC-y7(VW%=)?mwL*DxPdq7?j2*?8{uHsjUulF1Era$;rjL-Dd{8) za}J~cGXz!fgk@*=d@E&>Jz;)m{BhB zP~3*Xz_9axPj#m5>?7ri_c5BhmWEd!c~%hOr&w$GcKK4mB~ea@?NQCDw}lu?O!0{Z z?%x}jyNY@D-{`fyb9imW!GvJwDg?q0g*8HsNyV1)#_yeG33?YC6Z=USu(RC>pM7mD z;kS9I-{98P!fqDteA1jzLGJ%NAPg3)Ud*|yUz`(3)@foVVVL?uCf5ZF=m&i3iV+=q zBM&2zDUQ)BYaxgo|6V_kBa+lcYL9H-morZlyx%Rq!GQ344VXgI_qrT9Oo2~_^E}SS zMArPiwH}o}T74_)&S`yEK6G#7&9_5N6R)G?Bb&_x)4?jtsXG3AN-%|UGq~YlMK5Z1E5sXQpZRD`U?-R2f%Hv|;DJ;Y!h#K2;)woQC*JM2&H(qCm38T5^cOW*R% zJo{#yGW-(FtKz3n{6vTka?rTH>NUNN5m{)Yy|cnm4$kAQ%exLKRRXsEe_)MW3;}i* zqV74m@B$Do8qNzi8O|pU!GRV9HW~@@N?EsfchB^Oc<{O%a&)5F@gpWCTf}i2s5`}Z z6#w_H2k5E^h<2>u@o}5e31pOgDC{!cm|*p3#!u$`nILIUq!Nyh4lUyp75O4^ zGSh?CBn(?6r^x?!&Wmd!@6Pd+stZey|15PHj*XxzA6iO%%s`)nv7vRU8SRy!Q&3h; zZHNP=FZ>;&F#GNk01WWoP!V!|IA3X+;=K+_(oN{w8=4rem3@ROC&Kpn0N!+AhugWV-C=A{_8@mKiQZ@kRPn>Z@n@D1{MVg zSR~XDk|bgNk2wDl3;Qb!utxlH9EN12FYxXpETrhE9_)Ssi3uo*a2HEA9|gjJiVzq* zWB_iX-A0k`ymi1coIO0np-|Mi*}t7Z{w&J#%cSIEu-_AD&_z9aX)v1Bmx8sMF)9*VT&sIJiwVY={-?=oV9HEHNm zFJX4dJ$D#<*SaS5t*C?`&}BP`WcW8mVRB4K#|)DZW0xNEg@mBcY{=g5tG>6}*b-lN ziy4&3Fh_PVIe7iC-NGlqzhmNE{Do-rFh5esE5E{hy8S8co;>-@FmVU~EsE@#XoDwY zzkNlicz($1Qq8IO^2C~yR}qH8V(G-w#jQrQ?v1If^`eRHApCxz0*C3Fa4?(PYe9P7 zZhS7}rt;h6r^Ajyj`6n#5Iw!U--g{*5WIW);Mh(q0BD?pEWa!8;VmSC*woUGWDwtm zbnzD2z|9uDTc57o^(v*W9#D3UvKXoe(M_~*Aza;urah4__ktkav@CwV;}V0)&_frEWU6CM*2Ums`{7q zp*)(zL}CEBMWD@xcnZ1UuF;xG!FN>r%8!yLVRC829PK)M3ejs5i2;MHm24k_DwKyO zGY&602yIY325E6fC07a-Ig@rCP+cnx)^-5q0S^PcIly8rKd-oHH^(u01h z9(ELf#Sdy4+LT6Z5*(PZ{FFqmeVH(K_P1aD!W?vGJhR2GquvNTwb4i@`Li2BM;)5+ zzmevJ0gx6({ePkXm$}GMnX&Rn$->Krmn$b01A5b)h~JC@?3b~V=(j$zUoCIuJvx!P zoD(AyFytsY-$rV`-=+(FxGA*vPniA>M+%;E`tC(>9)qgjpQO*hIw2=Ee}p1}aN@&X ztGAGgPFVCXzGY2rYc6P`bs{Nf?nsLHsem7Y=rhGcTh+B=EtzWLZ)EV&zaFYCnDKVk z68;OniqZpNg!D1|oeHLDfPU#^YB{lp+M;f8uKp%26Q;VEqK_aO3seyMqlFiC%R6}1e`5}eFbV9(?TqeH zJ^(ufNnbgZE+^mVwaNrZDaR^V5^{zvrLSiRk$4$8!cWz-v!-pD{SS^VeJ90?(78ZD z=~UaDd-C{>B(LSf{Jp#&Z`o9LetLclh4QM=_kKU>yDXF-V)FmFW(qz}0r|^Hx+F+O zF7_yrlBzT65sxn^yWl?}KO`fcwKlN!(k&@rU*LRNcY@cx3Ov>JNvKbAn%h z_e%N0E%D_A4#Y*t0cYY;H}S#5Vh~>bOu%^cl3XPu%RV(@&Sg^VA zaxyNUQePOY=S2Zq&359+68B~nT*$YS&w; zp5akP{jHXnEq=Cica+PbiGA|Ow+QB%wEzujor%{gB*wN=#l7(fbDr4hdui$AR;hG< zmaA={_32J(du3n!|8PRzPQzg`)RxL@<)ZwX>)OVeNCN%eMW9vC-~da7_tN)X&%4`~n^Wuy2Pj5@LF^RJ zw9gH@w#ym*IqKbJ7Fa4@>xCk=)ZoKxP@`v@?+*QVvL#i#UYZ`{vHXpBqg5vmu~Zur zl#NU!;Nf0d69d+!Ki+JuznRcglOa5<6jEDsD>hLK36?KSrE1HWJzOaHCrN%`0Va`F z`tDd?slD#ru$w6Vin9s|J+2I#J7v;P*k6$IfNGdp5*M<+=3-4QgE~FeHG{4I)&Nrl zm;+zn>lgj+jzoyJ#3U*8qe@u>DIG!2|*=^YWOJRjU_l9}nR1%A`Q^`mJy9;TgR0TXFc?~YD4`Nwf zXv3VH7fqCbCtD{?a>cHYO|VSR_})6a)#@dA0=Ft2I7=*-FW72GuTce4Ir~w>2Q3%4 z`?lQXKqJ(kQcvbU6FB!Y>Diu*fP!j5kH6CL&s?=^S#ZV+2PaTzsg=IgbBv-AycCTgW@EfKctmH|uNwMk&o=_O34 zorxQ^At`3xa{DANHPF$UHh~UY$sBxz=x|=2CQ@C}_g|24%D1}hK{BdwGaUWfm#CRs z0qFjxj@#vt{Q5_ih*a}AzS;24sGUDurr&@c^}n^)ufBjL3<==cWzim6{fMfq2K3zE z1b%o`?F_(-Our<7We+ucG2TSj+MuyuP(~?Z+-hspswHSWh`8FUDh(U&NM9{U@jpgk zdm+ey-E|!Q0%j1ucBnl5Ef1x8hG+^KZLT$w($bF&^ zy^L6NRQ&9vc4OY{6met+LYxd~x zCKgJ|wdusD`LUL^@{Q1~&W#DPVNW*{(!18!|H^W7=sH7>?@tg?P-*t00CRkTeHHYr(dO$=>F14=E)v;gDonlS~!*LM^>S=o99I22&Ri-Yr{%Cn898E)(o)|Dx`25{0OZB zg~*7K%M-P}`lounvJbf*0!q=oSk~V&*YUEhIA+PHr6?qfDP@Z z?kCN_U^`d{n#?}1{+yE}R;Qe|ma0bt+7J=u#K9(1fe%5Y0MRRPENElV-)s3%RM1ty z$s80FpD7*~KxV3u?Y2P;4i!ZsM+2uc5MDJ);w_RJ9VZ-z2xVUU_JS&-q9|`c{k^}? zmI~6(>hdoO2XTUA@76k%_?=Qp*Df1K@! zo5SlwPh8Jr9D2kJq3^vhy4(3yhzkanbnOs|{LBAfov2F!4Vstxc{>w=IZUg%#IR+) z{CBRJwcv{7=7Vlo^h!bCj7cDth(>!(lVm@u4P8?)`pI&zHSH&U7QIoMG9ENSYx!Au zVU2_*!X{7>*(29};gjM>6jA`S&?HAk+#`@m^8 zAxq)&s>Pwv?x$A!TDDis%Et0P#Skxpu%`CGo!~H>1ERl$l|-HP*ch$b!VWuG5S#?`xEXz?`$#kSkDa{Sbmmy@8cwD zB|2jm?H>ts_&3l^0McI|IW^Y&x92w3cAol?T7W?XyzPC|U3? z1x+JtyMQ`*rB*h8(3+UHjlg?S64hmb8LYisFSs57f61orXN|F^;Q=}U=*T%Z;fbe7 zSqELW>=`PmlX@nYQrLST%PFv;7n-o3AhedE*TPM=>Gtd=ysVTa9+Q{B$Zd;}j|IRo zjgZ#1)+?l)kr+I&ZUpe_?>aT7JRY^>inhC|i)QG+JjbD>Cg)_+tt;spS~5P4wZ-Te z+7>Id<&OT|igbNE-z1_=b!laFVga+e)(OzXg?~itx7x_zi$2FYSRM{S)N+IPY0XFR zd*IUyk0>LG0AFON11jEAetDZ$iib+Rb+5ZLoA{GedKLx2=yG5GX|JsZRz5;4%Iikl z{ToZ@&B2xnCvb&v*OkiVyoT4{zu#azappdD&%eV3UJ1a@x^&bh+2Y>>jj!-ne3!52E1#mz>&7cT0OF2ihF zE#{~%Qz*Ypzo3;PntL*j2=`m|s%mf~|D|t;*(7SE;S@vI{^-5Q-^VWN%JpuGp;N5z zWe+yV7$g7@S#Fc9_?MzpiU%WsWqX4rZT*v9%QA68KCQAq!+#_5=_Yzh9#QAtmz!4U z*%%z7za9{^XvQB!Mogyj8FQLBG7UU> zVdulI!gM~N03J{~mTsO#s#pt1-GpOr7+TmrG;A#Uq)72SpdhYwdX`76Wg}R z3Y^nc-%qxQ8xKigp9f*Rp4M%>%WXEd?WK((cIvBo#2wv8aCT0^YoPDg)JXossSOU` z_DiNVByATYg`bB~oOsM#S|&!W`8*>Rh&^i$4zagUZ`wL{wUpLI11!u09sKM5k{va+ zx{rzXJji3-KYsWxXaHoKCuU(!?D}(3)qfm|cm0K@l7yE*Z+x&v{TFoJrmt2jJ#{8_ z4-fnIb*S3g^zt4{|M9bKH6JH3shTN|M&JuI{BJHNY!SR9vI+thMWN{G@_y#E`{QaBbxLaswC1!S$PU8F**d)tXlg}(YEfuI z7?KSZI4e1RDI`(uat6-y`DffPp&L=KiCVNH_H5yiEyn(BmiJKWTLAUaH-^w(lxO19 z>`IXq!!uW+Hb2g8Avd>diO+0XZCV?A-gl(;ZG2N(KGQNJA7s8vXjAiQy-~!VW0v8s zX|DRw!ldrWE*@YqgyH1T@H-*OR~RqB#2}4Yq!i$gLYZvJ{+%NLZ|^A$2HS_ic!AQ^ z8N0Y|JKeOh`=twd@`EGq^sjo#l94G8-#?S`J^5B60Pu9)H$jMIl7y5YSOBzN$S`NF%`?X}m72vCaYJA>V;&H&U@UR7T z@{ghkQPgD6F>5hKilN7Lv6@KS;}9Bgkv zT;vU3d=UO;&v_XddWWkpR;~tj_5LZ5{`OX`95%QsY@)1xvMkED$!9yEVG;_(D`jvt zS43L%kW(Dt=1}}Dd#5J>gt@C?y>3E`4zv7cG9JTl&?%G8)F+(vi4~AD4z?Z-+b>v0 zI*tR~3k)o8B?=!?J@5ANo7rlbjga4u)V84|Z`Z|>%I{B;>I;YlOFuO_Q~lSeoh}{N z41^Tx>ts+r5`;Gn_2A+7I&IX$NZY74Xf0IW@$l7Vx^71oNW_gep1+`Wp-}_o2O0MY zjJ)Xy(@(n?jG3S*!z-aVE?E*5CFFI%Yg*!=`&p6x}JrvQuVxPx+F4G@{#WIN@6Fj&qB*QZ!2-4f)5R`d?q?WfASPs!wCyD zu5*vZy_VlR0O!q)#IPuaRpy81_X)dLL&iN|X5SJV)q;fII};tJm0F>+Wj9`-aM#P4 zUbEG|+a)@HA{beki(0T-;P}~cig+m&eNHf)e zwJnmMiY}gfH#GXi)~;}8`Y8iIaxl(tNbgth#p9kC9}Zge=dgBRf*2i{nw1T7aVb0# zX@1kKjP2&ACe1@}1C;fP4OMQ>>4aZVDpJ(w!a)PlJ+=-cKc}U)@AtXG$7x>G+=pWw zrN*h}BaULI=vm&17A#ZVQvDinrzUI!1^>foQ;k z)QZQRP*RE44+<+?cvZyQ@1Tycm_7P-Yr!vyHctf{>X?pCWTQM6Q_*L}OBv_k=I#TX zMA288=Z_;Xq6?MoubQo^l*tt-nXJM}VM)|-<;Omk6-f4nx4fZeGKF=6^^`e(?uoV! zEBea+dChsl5s*jl@|$fV^#hy1PIUt&nH1( z@ixdTJby;mg%&dIyomLC`xnatGG}`1?qS4afe=ng=QJBga5FO|Q?PON=;zb7Hgjf& zGTlm?t`PV@g084|%@^nEFKJC^%Xou|bgV0v26ZZmv~u`|1u1fYiwPW;)<6|U8I4?T zZBoRoiKi}YUu)NhMNhhf3R5u#iV;c3ua@__zq!Iq5PRU+w4R%Z&$9P9fQ#Wy9bsYn zbxNdPQb~DzEyctL4X75t4ypTaNB5FcFvJ;LaHa{%q3*3Q7JZ^8OjCQZ@TFe1HQ_{P zq9He{l*~OiQ`gL17Z;b$Nb^~DpI4;|1Fc{p5U{8Q1^d07sh;Qz6KEfq0$5?!&Rr^+ zpX|Hyx#yuiYEb`1MnS;*x;!RJUsLxTGY?za(C_l49hag@P^$hMpM6+sB$=Nu7ik(p zpRqESB~$ndqeKtmX zdr)iotVO8+|Jw6B-Tc%7+Qyecv25`jmsOvhIUJACrJA1=eDIi&7VC#41*=iU5|*fe zIalbN2i50)salZVJamZ1AO(nC0@C0ciKrM^TG!Fa@SCAH!;i{TkS2!CDo>vlo!lkg z%p=F(-x%V99w;ymm76wVj~BLBDY3yt_QemEBc>dTEc6?#^m;AlXyn8VJlO{?a##vH z#QF`ximn}YG(ux~d7Bpr*yAo`Dro_;Mf0{%BDSgmB&vVz&VPvIfCXIXV}<8v)hrO4 zZc!rE0A-Vc+erwH=>(FP;=7+k4ZfctUcR+4l7mC>)gbv_%2as!qf%1#^I2y7k~$(M zcVYpz0(1F2n5T59&G&cE$cY-D9AU*8h(SNRGb>WVHe!s$q#W9t@@j+IqI;=mIBS9E zU*~m3hqWghQ`YzMR5l3c$RxiKTRcVJT9ISafYm|lyryqfjo#+my!O-Sm~`Q%#lP|& z*^$c$XBGtJ(Mrz0>SACa5YL^S024z1uLAF0>5-}td@eaV9mp+K&k6kzefLeQSr|rv zhD~H#)H1M}_oXShH$H#glx!f$k8sf9L7u4!?ZcsuV4dR^mW;TOb(Oq146iIRIK|39 z%!e4#+?%=h^Q=^n2A@>zUrxml=@W5_n**71`Np0u^Aa*Z9lAn}{e7lWE+NQv?D*%1 zda&e&V9BhS1j)5%PveP{ZKsxvweYvF!*zixQ)G9GAS87QKET-;9D{w={I}M`1BrCt z!jzPjO7$_oWf*SO_^#^o_j~(6tn1#mA0Hn2!u1&A4ivY5wb?*o_^@t&tS2XgnLd8F zxWe)WCU$kJygEO=&SL9eQeS2Q!&RVB*oW_MSke|2D33eGUc=dQfB_#l4qJHc1$(!u zB5j|_y4FPqb!`=C(}K~9Ur;!7-;e`IL!P3gp}5K0O5_~_Ff860J%^4SWg*QJ0nxgx z!LFe=8Z58j65cNHVu$p;L@m%&>&$5(0+k#!rNzIC*``Xa>Rd=Ty~JezME zxaFDj+Deh2KeN!}ve_9iwK1J^h&r-(gM#oRf z;pd07fASZq!u{J0yxuI3RP^Izq54-lN4O>6gpSr{l%a^g7W9!aV*84T0dfXk1(RII zyoko^6OmD%|L>4!tlFO zhwFpi6m3RH=n2E@#M7+zwaTGJjZe3#bYv}*1rof@)T)X91zI|()v=q3 zojA1lF^ylX%OrymuFKdX4+2G`T9~_nnD$gL z+49x3;bAg;c9B6fB5-L)|~Qn&p^>8hcuG<@Elxd!Xe8eRub%% z9=!tdT8%RbRQ7+XsxhlYO=J6_J*w~XFe0>ZW(a6sARcixO`Pr2m>%3Q&C8LL|cQh*UNWUb2S(7O+yd+(;>VVLMz;{n_=aF=``U+P-EP zrw+Rf@lwX#kMO+KriJ#$W8c8AKa`l?MOi-uglq7Rci|Vd;Ygf?xVPcq3?Ax2xi0d--Fo+B2et5WWQ8HJ^l9v?Vs!qstAei1DRw?>i z(814JTD&}@A6U-xrn>PWOcP=!WG6W&ovo5>?y;=19o|o%r{Ae2gu?hBetv!&y^GJ4 z&fL9oMFjY6BtP{%CmKWQ1rU2wZO1=tk^@5i*V(+sn}Q0x`rTD$%z4~?Kz!;4#qZ!* zs;inoWmnU_EH|x+uhEK$@yG$!{hFa~i<&{-f{8iLaB0*wneWHl?rn3Vqn+2aIm%QT`MbVK~ps@ZhcI!?QJ|>>MzUO5$z~B( z#L1laDlyx=AK4Ww{z(@4ENgcZA1EeXD+hZehW3H*9X0SLb9DwrynL|iT}}TPGB*MG zmxHa$bpf_tSnQAX#v*J~*cT#M0Y|ABAYXrF_ei${ z_r)XCu~22^TI1nNaR!!0P|La|_x9$Zli9%fAfV+``MIo57QTg|l$N>r26^Z0ljc?H zB$P^U9{%%*z2Fp=X=|~As%n4HC1riE_X(&!^GdRou3Y*O>;?qy)V5|5c*2Wscy-OM zT&zhicQAJ|so#3!5zg|Tebu>Flht?2QHCG;$&?vam4$f-XwFs!pq_0sXAX#|qK9829l? zq8UN*QnRGELBn1=w~n9i2dyeCg2-~qw)pG-fEELNA~2YN&olklh*Ga5rO)GhT&vP$ za;ZGxP5`yWbDo1HdKp|&v9Bj;tofU2hi>tn)=?RM=he-_W!0{ggGjOiR`MsB_J*vR zZuQ-bD0JdeJ?VVvaZ_3S3`?G%2h}%Y7rRcnpSpZ3wsQP3VWCRTqx&>4R6-zjl6LaU z6SBH}Kcmt8Rn|y(r2U8L;XT8v*^|f7)DsO21?D`CfkUv*Z^{X{0fESyb$xi8On2qS zbf3o{6@#r(ozDQ1=>+C;n&tIjj1c1j3&)Lfc_OKHr?-OuMn_nH+Mvnd1&++s))EC$ z1O#RXk@R1AtqacW>fSUspW2G9qxh_#9+WzLSv`;3h!3Zs_ z;UuBYAJsounn3HN^7gAM*X5hm?+yhiwkF{xK#)0lkcf3na~k=xMPm)ncC9$w@u{1Cu2`cQ=S3PuG2Z$3HGsKs_xr zswlPPZ+1STpe2`U|C^TFo8!kWhyIXm0}ezT?RsCsf>PuO$x+iuEXN3NMl_uDMb;8p zM=QD#NLvy7?8&Z=l(cH{nqoosH@W6Zr@vj#x5IluhZM3l3!dLYykX!h-AbC1&s($Z z?NDvkYxG~H$>E8fgbXSy*Mc&73V!yK1aJQ0=vSA)UOlw>$QM+;vczz=@kbw6940=n z>`j$*8MAKlKeybNv^A0vX=MdD!#J>1Uw39cIOd}lXoV6>nmMRx`^Np@w0b<8*+4!brR`x3Bj0bAAgV>&>~R>VDCXQ% z)C)%fE4WC;mrl^297=4|w)lcD#!VCy-a#!qx}*V1h8$DCmRn%aXM8X+1%DU#ZgF9VV{P1dwrjn*#eFuFupT^-5Dae@ScBUlu6f+2< zOKb9rh>jN&3z2H1VFW;g*idC5uCvtYfl~;Jnb&+mCr#xT;I0_Wz0maoh>v;c4ANYm z8Y53n^T&qH$cTbcwGmAot{081g82Q0W!aet8b8wAqeYt1ezxa1-R}PDV5<0~fikFK z?4;it~MYo~G&rD`-PWpBqwW|J}Gru-n&k+VijmprU+Ct5Xy3GXj zodY28bsv2+#abho#AE>nx#8N3m?wOPK?+$3#Ls%ls5Lze*-2@`DP6uW<*61k4GUcJ zQFi6$tvY%mn=#$UzyE6brx;?w+8dqMU^3+d{X1lt$hwT?$fdf z2d)m*U zaxmsIZG05wJ1{oHoUF~i&KmE;0(8mBTk0Mh_pGft6gfCj8KQw@i(QcJpt;9d-+zhR zjCjaw?-Mcb+pM<7zLM@1=|=bbV>-EoMJ^Kzb5I^G{q|c$ep4GmFR}p*T5mt@EzR)j z((RUT>HP94m(Sz0KgAI5TGEQr8}Pj_AIo0 zp~$d!@emCX*ZZza!~koa?r85vMH+QdhW40PJIXwZORQ)rIL4h_)9;2Y26sjot}ND8 zA#ph|^5is&;RHfRN&sF)t!>9B8G&5Vg9v~pYl zf$2G!-ZQsR9|(cP<<^==y+Q8fH6M;9LDddi+;?~2vEZz@mPPLxI_$`jA#-Ce`^{ zwq~hJ+*-c9GM1l9LNxoVXNk=S-Rru?dWsyhl{vIEuFs>A#JgF#ELWAA`<(_otiNuS zhLD)0qv-%!!~G7=_DE^*uaePjNH+`hrw2|>)03%03(t%cmRHPhAJx@}b61$2_$nRH zUdn^~aFhAo*^cj1j9C`BqgJTpN_u^kK!$a;rF&>yGSjEJWD3%&lLdZ(Zq)S|v1QE@QXd@bxkpRsQ5F~bm(Av) zYldDYfCYWCDd%HxM2SPi!$M*aG50lEJ3zq>(Gi(6K0bn|8z$f;#D}*e5n+~Fy1GNc zt_uSdnQcs*cC$b2(x*A2%iQ{W6T1bX-O}7F)jc{7j%5>cx!aR~9t53#FD^7*#@W8V zY#Z07gdDpO&2IY_IxS}QT))WN`l~Hg_EIRSyM0M)wNf(LUnoaqMV!Ol;``aWfI7Zz z^qbn;fhKw#tHCm4Ir4JTiw^E~31CAo(|;5c(W?+tjaJ%1SEQAROyFBSTNt}KHsF)d zr2;hEESf4>+DNTUEyaH>cLyPIEwbah+o|o>w#qkK(kc;(1L(woIGXQW2e?ghHYYj9 z{RPGc#+3!fQ`6j)o9Hj{tn!O^2u;7>9=uDy2WkOftz0%N-?a55^>CA8t?o1zwYR|= zQ2hoVwa%=6SmGWyIV(~}lNiv`JLo+W9JZjC!{BfpW-&NBkrd)AZ8G-bM_~g`OqO+1 zLhBwo$tEY;O+^~lg^o8l;eyQL9e6&ZN~1Nft@uBC8fb;aOXuG_LFm({u-LymeY3;! z&}^8H>NIBbDd;J8o`N_kwXNMi4L4$Qz>!TVy}2pZgBM^-{Mn9LHg#f)Dp@R<&vLEs zfmj_h$wIc-(^);fj~H(Ta{MuQd_8&iVi-5NCV#9icGf9SEnb8xq)Mdw#I&JN=D9WH z@(!8RTMY-kJDTgL=x#a~O~mz$s~WRv_UmgC0D9eU*Uv58Dm&n=X8&sKw(p2EW=9M*c&C|6t zia(}!vwEepm&KLL8rE)nrcJb)ybxvmV!R1`5KG|MvQU^-EYiWd=`Q5l49fbDqaL#0 zzWl{@Q`|roe8Dzi)k25)W?SWC1yc3$rWT0~ z0nN4(ZAEJRhd{D+blai%rT%7*uz$~pSyjL5O!*T=8{(53AByxT>{Nv{QnzI#c;4lM z@Erzb6Njc5`>mxTJ|U32qs#fB(<{-DSy~)of0?uPJb}*K-+Lk3VvS46@f%ZjJ<2@t zl;j(smMU)kV4v&weURy1ecgCCvbDz2qXT#vGeGCHbRqpi(#Il$fs8=JFG zJpU8GJXVQ!l6O*|p`r(ul3crFSP*O29oy%dR1Iek{`X9LQrjdQX)psuKl(c=F)|{|>9JUze-PL`?iP)>06UszqTe6VuFqUbkuR_iK%3E3 zb(ki0F4)IN6**N!l|>3U$2h*0QGwrNHmY0a)z@gxvh(1bkBgSS4tJMo;R)>=MX|d& zHm^#j2Q=j-Ce5Bh$xoQG-iY!R{3O>Mn&p|I4?(bLR_WbSic0TP)? zcxpB(+N#B>bV@|~#>Okgt;cnCW$WZdTKgEzvspcsB+*M`Qg~8SWU^4#)K+Xs*+kE% zpN#}uHCpD{+UAY6hKT}>RX4YXJ?un!&+K~?0d52VS;oBi@F$e1ybf1*n4edQFheJ^ zm8)ki`@B6Px%%32V*Fe$Z=PAJbxwJG1@3}4t8K*sZ;olM@`-@uDF5FsK(LK<-xXB{Z42ct8^LHsm1e7coOP*X))dr({PA6rSE z49B)v}%bRI^&Mi2Sd!6Z;yvXk8EV z$1kfa-n2DE+}p;dw9$phYQ-4exU%$;)hDyTzeJFRKHLmCO%5rL#yIGp823;N^AAVr zKNNHra>t9uRx%GJrbwLl9ASszm@MiDOSVCFwCX1^zkZmlqjf>FyRyj$HTc4x@TD?J zeswcj1yJDta{${k^l7|a)%4-C&hj*6NI%=#-ghgt&=_|q-5F60g1Phq`@5DKO=BJf z49ofZ1~)$`cR$Klh?9abM%w!_aF9jwCMBf z^ES-s_fH2K^O?!H8KQSN@q{cL9bM+-!Q}xx=5m`;y*adfik~-?*>BZcp9p!7e~I)D z=gP=lwO^L{Puhz+Zi~rvV4hijdPTVk#8WbZhD1~;19(VC2qI|J)8ZiD84<3H_HUAQ zLzP&d2YHaX9bZ<2Ywn=6>L#*$>a^>o-1+q<6MSat7mq5u4b;_qV#yh|-aXgW;?uDD zT=$qkyKY5f{Pa|{4NG)q_~3Kp@|4B2Ezio=C-NkWtLDIG5hiu@wez z4fQ99dBr^E8fi&z>yqW!AdSftb+eX5I4a)0%)K<_$vIQHz>#c%e1Wyb>iGJa=ZxOW zl^G+`y7h^UvW{9KOYlvo)v?{-T#}wwVs)X6btUhegH$9x`tbmG0q>v@E4Y?1_vk+8 zDYUU*tllX;$N-zG~%XzB)NAo`~XH#{}m}NzNqUzPJmf4n&#T~l-f>w zElo;QHQ*1`<J@&1~w1LtPHL|1u&`U}G-BwOH%R&n( zKRzhvfGi&fBwRgzWZM8YfP)mmk<^PPnRK9_Y?&eoH1s!*YSOj4w_m=IMQx`@UB*S* zaatNlh->dfD<$(&@9A^jlp?C#hnGo|r00p|$!KY;P6{x)m*{S4D@KSSwT)~$8e~0w zmaK|f>hhFn-Pe2Klh3MkmhR;|7&w2F{Rfp%RIFjHA=CmwTg z-cjrEkvTu|Y_;r0^Mdfr8+RcF2ERkP)Q9WQ(~(cU;-iLpxtQX1Nj2caQ=n2u@2ol7 zZrfu-4B`{rRUOLKuP`5mPFE3jKU zx@I~*xl1_C-k#{L2qN5wg0^1HKjzqYb~Yy!E$xwnKn;ZVW{~76ESD@*vv~oyA}4q% z0(WyO|kB^R&zIjIQgj{3z|P61k6_jJ3=jpr#vt>J)G0Dyvaw2cF8iz^#~ZHqr5 zxSMxR`%V_(N#6rML+UEtd$9_HX58TeI!XcJ_B$N5PBYVdSp%(nw>hLPOWC{GIVVf( zr403{cOQ3R0r##rn(=suh^(IS5X*b*+%W9v!|A*XIa^@J#nmitX~ubr9^&ZN;`|pX zezr_@feO}?iY_`(vdWi!;YC3sNmg6-DS*ql0m0bQpnQFj^U@FsW<}1J~gt4D>J00Gu7sq6lUg$ zYp;;vBE(4}4{T|f<{}amgA-mJp6UwK$xX{@c}%TO}i@acg8P~dMK2!{p?@S zpk0ku>ZD}|>&2Zv#GVdS%*p<6!!V~q%hCUe@M$nNB%VvANpHYb?ft?U8FWo9ks1Sg zZ1SV(6e3&3s`$~_vDm7A5YoaKIw0abyWTW5UeW6B6KSMxXd4@OjUUt>k_iu&VeHu% zZX?AH#|A$4J9$A-sgYdEWyNMdE}$Y|09+BI7xI>?pFF|v(6rwYZ_w>Kml}C#-z_jC z3skT}G1MK5df$(ouXmQ5_BYCWckG`Bx@1=gDgsOUwL!`+|DNiha0ujsAH55r?>xl= zCd7D(8xRFHKYaLLezai;1g(QXfo4`Un{F_aQn3$`nC~eymD`&DNBW67ZRc|fZfFo8Ig##rbI6Rb})`HSt50dvnXe*Rt=na;Zo_SgB7mJEMG8n@IX zlRH5EW=43CkvpT~h~{As8j=$+u&`avzl2Z@94D}G?^-13y_Df2?1jOEbZXVJPkwix zfs2E*L7y|cffP+mF%lH{x*k_Nu#XM02v%i*(F?SU;G+WDB`3y8L$nCgMVH8 zMAJnYyU`~+PcEwN5;tffN?25Ti^Y@ZiI4|%M4(t;Zhqd;bxm}!inICqxy8nKxt$L7 za0}?3-z0A3mFo1%uzlP>8m;o;3RLI!zv$;kMUsQPO49H=hDS+zKqHN!`f%_MP+F%> zQ!6Xgf{BT^N~J_$rz$oVavt>8(eJeF?cxzIDI28OzBC-aekFl?nF8Ap&D^Q1h4sDU z8Wo;GJy34!3JN(}2Pt!Hbo8K@UF!i-2WIk(pJ)|Hf5-H)#7ZQ>b=xP5s^0XZ-+m|j z9xXS!tf8Yfgc2hEa2O0n;oIFZLt-l3@vzfD3>PnL?WWAwL>+4PG?riHBN!-RvXP zN}9v5l5-wO`|tOC7${CZ@@HA_FS%o8^hRMb9^}o_zi{in?|GPf+9c2*x9fs~1gNiN zPM~xZfxFaCD^AzWeF;T0YcLTnkWz@YX{QQAfhB4?G%4q0kQjuh^Ay|p5(-ll3Q&cT z^c`G=K7Ng*Nv+T0p?QBiCLBW~2FAIAc+><))ygzk@2C?!mO@IoJ`-uM%{k-&@=y}E z2m|P6(GwY9s6BND(tkwwgh|A=TXtuYovLm}{+1!|7!P_%!8Pgopb?OY$sw++_(|VE z{BbKbR??C{RprjOXV?G`j&Q<;hzk7OzPI<~ei~&3hcUjoRZ~P=j(fBBDoNetkfszl z=|$N+=T!LKr;t;e`)cQ`8EOCiC5lpa;ctfMxMcr%FD^DhTz!A_I=_@ZmB0?)1N1o! zY#Kq&_YW?^%pmwCQ6F7rLgmo2L1=$s$S4blP+GYg%Io(}feIKXFm=XP$3{j*#H#vx z%d9z-L?2Oa1Ku@y`oDhT9=wvKkq|oWmQ?xt2)J$}jM0Rm*MAX{)Y*0C#O1n^8S&I4 zR#d+Jk1LT>#V#n8;i&U-Y#$PgWs){2uByVes1I6SKu9#rNYWmdXA&W$@%9R2Nc|}S zBgw$XtW&s`a*$lrzVjTwlVIo%)q&%tZhw1%a#$XkM)bPOqw^v(a4!0IWM+q4tf#@@ zP=06zFZ}7=j0(Z)S5H4UKXR2ybk{eKg>`k`E6MlRs7ZFIB2_cLUuBaN$+l%JT>VVS zmaiW(BoBCgq0PfA6QAo&Fe|yP$Lf&qB1hzO)MR*rsVP~fl1^@hm4#A@FLn1K7iC{&Y?_k* zif%R0aSd97(z|mQiBm_|31;ej9fk+omvUmR*Zz*OgRe+DR7goMzqp=Jee{N^ZJwM9 zd}j?dv|~F&O(d1X+DUVeUjyr7#Ut9HI&P&m%9$CAMmTA3KtZ%X-0E@BB2|I?cazq% z9YiCSsHgo}G04Wu&$pQIQ=-+|Hsdz5I#>WYwQwkr%riZB>12r zDO^n4bInmUgY}$gLeq$~hJhXpy{s-?J-7tPF9%N8{|dCDOF$tB_Tk zDMmW_&q6$ZN(NvQA{AE~SNZ6IFf;wnUl-F;Wb4H_u&cPX=hklce2OsK-+`-}BMDaez}~l(X)bVdE`&;q6h0+q9<`6S=$w zLqeQ{6Q5HWeOUislP*4(Bn|y5(r_CO;XaDMnq1-G%8IK*On%c)eBec{=i-6*EUV`q zOmQ;WoBkCWE&HbA{AOr$Ih9jvvD&>fJe2htK<)zlu z!d>;eWD3>-)S?^!7i4_%9U!I0T$k#vMRoDRDC5wAK}|0Jg*qQYZIbSJaWi9McAcTT zz|9*=mv}gobTljGtgiFIbfyk@3LHv0w`UfVsi$HqeLzXcFPovKGMIGp6#2F zpH@LZp+0mQ;z?Z*zutX{G}Nb^Rg|RbUJ1fpnlof|c=cXUw4R>jPxA-NV8<}_?e19@ z-tO?@C!!~OlRH&Dj9aqKkt&3@(ff}e$PKhmV)wdD5$!lSn}KSh0}oz z`e5D#F?5RXfGK(E0@1C*i$4= zeoslcDLL_@u_AVmZ zO^=klL%5;ry~%1Idqxz>-eqS0&gWjUXMDfk-}8s((*1nSIq&&;k8_Spe=8r7E!aXb ziyYNq02+3(I$oh#`arFh7}XDQK;{>D@Ii2}Hc@le5QFy}05em%YwpBzzCPSiP_2aR zee05!l=p)RVdxS{Vqzv}t&M zoa^XJ)u0FcysH%zqAXar?B|2{yLzsO(7b7l74G?Mv2nvdE4*43c1))ToDi6l8F{_Y zAWw(7VKaK1)J~vMXJetQB*tXFJsvG|FE1LP_K8vY!<}q+S8+SdJa{o#_7Czq)ZFrd zEvH+Y!S2Q#(AwRNmz3{f{o{?D#hY(UnR8?WrUWpnjY;9#LgCYbqHGLLK?KfCcXt4p zljDz4@Ej>_M5Ta9!v%JKGp?d>EdHpO<`({Ws4(+7bCZ^wI9EwNCY!hUM- zntct&mDi%%s&>GGatg~dF<^HP7T8a$iA6|Ac;f680;Cn*iQ9BU;yfCRUUUugV+hkh z&u0%RSexn+{CKR5;+)V-HNb^I`HNi^1JgMQntqbrN=Lgq#_5UoXD?>90DD?RF{O_H z9mHj*x&BQ)UsB#h0_a&wT{J1^B5>m05JKM_x&IabzFobsgPic!;2@+l71RH$}%Rr5V2!EtcU1-#K zTx%5>%!+!$9{VZtJo6xk2A_)>Tf2U>oNCSM@@PfX+yk()aBrTq49Fz(n)gnBN8cf+ zAx9UR-1$X>=+>Cki%rNj~O2IzEXk=s$~pY647CQlWSy3jT?X{&=21Zd#)AH8GJU&}3K!wHFo zS^47UU@?6mmiP7@(E&<}MAq9qU%)NZpVBi+y^Bgd z@v|BX9?<6vlzQlDDR~4MlRYJoS_qNbveHW;23J_??NeI3qU^I2nO8+pWCvvXHKaVN z?;{GLk_uiGaTsm?)~Xxl?mchCYqk8F#I6y)AYzTZxcSPbhnb*tb(jA~<+RBmhO?k&;`Vz-qDGC`lst9#Nvc}H5wma!{hdOMTfAL#c?0Q_+=bh_U`*C8; zOk|^OY8h%zztpAcDRDX`{lI$BbA3iS_vxQ~is1dI)ZKNg2R3}G^>+GMI%2Ex{Ts=n z+>2P=EJ5ot+Y7|4e%yP36t3m1>PoQH@+N=+>n*81C^fB4g-z(>4@kHTT6;z^LM-(#$=Oi<@Q(Wu_=+172E8a(nqs^^0(o=K&my;;X*y* zL`#gzvx2Sk4>${LqC{X4puVKouw}Fw@$`{**zSfV4W(*xlo}wxnW9W+YLkLq=;xh&y=bT354H#Ey z7pn}0_Lap?7Cg&{EQq+)`$-D>L1mv^pCeLa$O0$uQCko5?wK2q_>aTz_f2RCd@g;I zp5jWjo*h1v^T*>Qm!griJw0|VIeeTxD&AONCeiTUL`uDmjxK+^*+ z4U?wM)=EBHzw2~F_8nkq9l@3IPQMv+Ehta)BQroL<@0RUYEo{&&CwYO(^fND`Bvfh zGa^3(ZdAqZUh97J)G*|6#2iocSM^3$eVtO~$c>Rw>7HU8h}q`k#Gs% z3|c=C?O>xH_qqy;C+M$eIf<~ZwzMH}OtxVM2f?FJLT@yjR^G%wiS~B9ecPt-CogL+ zy_>i4{Gfle#{GSuXdp9gh_%p^U11Z^+SE#L)#EMG>W>FEtp|h_@cd-p%0~?XoE)-L zkEl36WKE9v%Az=B?E>naa~XS^k*YsqFCc|HzWpqyVPMs`;t6e_P*~fzFf9vUE;}#P#jYQWDj+bZ*juc7W7o-?l2Xu=7Uk{SRLF5S*7|*OIxC3I5ujeQZ=Tr73bX z?Ew*6s*C<~!Q=^iS#z9cBRofDfWA8eH4!Oj7lw>t*j*GFMBKq=fne_xDKv#S3=*Q6 zY2jA-eITkKRgK19p4DMk4>;LbZN{20g87+j#X@&s?-k&9oe^;-+84~D-<@can2m&pdK<%* z8w<8?eLFQ}2x+A=oM3gpx_w6G$kDLO ziyKwGRS$R3kUdt}%pacc?C)pi8*o8&s~0M-=nhYqPAT3K?S#;#0-D)BoSBA>G>1Wk z!tm|5;-^D&>n-ve{s+bSTT?swj*J)3@erg!U#{&#c<6G0gP~EZ|Fa?p=5(lJHU?R{ z!CDeD$#x4|dm@8+)8CHn{iAY|p$s_9cNcwSK!!^7{vnpTg!$L?9}vy~I_L)#!-Zje zIOzRE+uOg7uY{;xTO4Qex$AYlcW*Zx`n!HmK|(n;>pPj6>=9%5uWWdyiKrB30OgT< zwuXb6KT7x+ptIypk*T211v{YX;kqvBi(nyWB4@}#J9(Nv`clL+Msf;7>2+Gq9!cp; z6Xy`YZa_Oy$RfkRzDY^se0CPd+;ZflKHQ)0>C-v%M>w!okp!+b66RDcGq^af8}TW3 zUu?|Dh#{}f&jzSy>^2))ZHNu3>ochog3xO+E*kt(Zu4lod8j&jV({$(*F()^N8WZ&Bi*ORQE4)G0Ny4)7e?@6w;s0tJ?O*VCFgG^EP7U( zL08`)u z(D_e;n~;=@uJpGSY^m`(5bqk^YJ5P{3{PWl4;-LG&JE1E-E7i7NyHu{jxyU4@*h?# z!cfco7=gw=r|R}!2YvW+at1+Q5+@c4Sh^lzhwb%Y9M#udFZZvkeF*aZ4?3I*P|)ts zVE=7W_rD0l{tk47Oa(jl5y>#ffHnLJUo0htrn=Lc!9_bo_gU)pm~Me_s&SABF>A&=Wl^<~( zI|F^G+TPHAQ%$PYN>zYJ(?RkS;zN ztNp81p?B(|M+LhFkily2m=Wt(4pi;mzkkmij$Eue4L0ZR2_X7|8hH-Q-W+zxdQ>tY zp{Z_w1YEv&_9RVU^&_mes)i!ta?k-ldoQDm_FOpl4gKeT?tc-4aKuZySByt3UySFY z18Sk(bky>pCyn9@15`r%zlExWxYhjtBu++HZ@G%Bt)fC?6BGQ3A+2?KPQ1=TW=~P z&ZCaDAhIK7{U15`5AbXlv`iW5j9ak0R{dK{1!@%k>NpShouV`JNc2WEaME&&RK#N5ksI>7|1a0dVDYm9K;Vs-J?IguriXq0Q{exLq}6R8Bs$Y~ zMwu4PSLAWb%LmK+53z5|$^$PS|0|6N5C|db;ws$v%N611a|2*XQMQGJluk|<&?f4E@ZO{h8|3?#; zc)A2Ck<`#b6b!fT} zZFeC`^%BT{f1s9n<2Y~~eeo1HFQdqF0%Xe|z)sbg_5ycCgHE71#6jo8)7|RJ|H}9s zQknhpUBR)WNATfi4>A|1u?D)M?;zWA$Y>p%^kLtv2KhL0zJ354g5bxW8-hK^0qg~c z@c$pQ2_}#Ue0UzYF%A&>YH%vn(Q)^PwJSSlhW)7!-cWSH)#`i8Ulm^ZhYbF!Q9d$` z(lcO_m|x2H{;wpC?iyOzAZ%Lv)uG=kU1K-9d_o8QYaPJ6Nyp%W>sBD5{x5E;g}#RJ zKnOl4{Oh*ETQlncZF4d_m-lD$UrN1C zV$>L4p}S=-{DlWA#Piv!tL*9S|1><>5T!$hgOz=AD7HtP&$?%wfI1$Q>^<)PGZ277 zKmZx~3Y3x$nMiP}f$8DjA)P|mDR>2Rm`A37VDKS7pA+U8j_0QdC~`So)M&q?3M^I( zEa33rU`H5id-M)F<+z>w3EXC&gFwAJ6j#f}TG#A5KM@v#|F|d^B%MK0dLtqjpw2hK zRhzU{yU9~awdw&hhrk&H;64u7I&QSE8;HQ<{u?m}LvC66Cn2cu9!97+YEb&H!OR2; zw*XdpXH2ys)4TN`y|urR{}AV0>CDjvZ*rKPpTJ53J9&e_wI-h6sAmP?Sf{_quK(%%h!GSK}8Wx;&j z97y~B>+tB;=3pWaC#k-`pZx6()e}ed3H{3iP^S9bW#H$yz4Md&4$on)qJ3B#{?YnC zcoPf#XMJk@FA}=OuK7)mh3fAk(J~cB5t5S^x(hZQw)eTtzedG#=ue>+PqVDWy&Z4+< zz@WdY#y-8}bqpfi(oa%FJsKswicr8CA+K;!wYg*))uM0kr7L#^#peCiiap8r7aF7P zgNMk#MGJUcYnK%k26mu;%dw70aoZZ&|D_Ig6QZwjIYO($a<=;qM`o;)3(OIDyv{4& zPYteMg07!E41oUdZ%jHwg-6iUl{g&or&4afd5;YKKP_c%C_(6uL6hqKu{ZzjuJ^wC zZ{XqtE4T~5+nzsczi|ykP;(MuRHxD*;rlm|96jG+_ksV#eKpWHyJYHwL`Q2}yI`LT znVvTf#1i$_|HejwP?$pM#J<~~UrUeND{1dpq<_>v3QCD55Nf7;p1NlW5ez_10w_08 z*zWiRq~QLX&>#N!J;aBvEft1xEOpG)-|^%n;9+1}vpAHeQ-}ihf_LeJQ2wosRDJE? z5DBy5USV*zx>OB&2%5pgyiIVSLnIM6*z$+48;DPOp=i7R4D~gqaFkb;VraYisLX!r zd^gyM(fJ14o55#f9{Es=h=_>#8RBs1ku>v|JHa|f#-U>5-k4HT@YSoXZyGn2qi(w2 znww`Hb>A_iRb-hc!3v8w&|VZ#Q}l*jbxX6L?}Oz!g%IJAYmi7vnv-~Kztoh+ukb|j z#V*ODtI?-*9w-SHEn>yp9lXDAb*tZH=Zs89*m9Y2)CK4kFSC1Q7SrF-D-JJr$P!j` z0U&k(e|9!wHbYrqtsm_p=BbWpttjzzY`UrZI?dI%X{M*G)g$MDaS@Den0#= z+dIF=+LBR6$bqalw!wutkOyVR?@us-bh zU&;`;i&)jxd&ODNos~+DIBZ6viq3{uFd?V0Ad^SFB6fmS@ewGK3-;}?KqnIJ5{wlB zXdm-08N;-R>;1=9xJrM$q;8wt3=pG=vc$_ed4_E!bruyL zN%)83!=Ey7vLjzU$Ziz+RXbpx_5VW?d!JP2fX*2DYa3|$cZ{O`Sd_?%PsgI2 zt~bA@J$8L;vVIrIOEDw=*aZh&w7DREryY~gi9gtc%t5E~Be%S1B;A`FP9-rdI^(UcJO9QpClSK*dEqm$a5 zpEB`&B8TZGe*+F&KoF@H;v=)4rYjY#bIw(L>FX2g5r(a(?)IS z1xp4mne~XqU9{Z7Vt{UKMR|H{$dle^BLrzQ#8phE&QtXg;UYS$3|GM&wV?vOBQ^4e z8#Ym_IZuawFu26&dY1YY0Oall+ zLX@Y0PYTlr$TB<|PKz9QC0EMjQOX7(tR$$faG(5$8hjSY{olZcG^k%jq+F1-L}3Ne<608BS+$MtfY20*?t*tfUuo_&%mA9gTKXp~3?BCaqDi z-(tV9ZhZe}Xwp((H_*xdWzdEYhD+y(arj4=x@S`G&)$V+sK3IW!`6d8gmWe6UGFOQ zakrPf?`VmSiA4;@fPT=WVl(Z)Snb!&r}62oI*c? zfnbCk^CnctqiozA8Lo$wB7GM#M2bWumkvFZPuNR|J{1M-I%Xr%r(J~Q8PJ1Nvt!oo z4viWg^#i#H193a-yA4&Pp0w-e9fd?ZM$!xNxV~$W2VX*cs%EP`fkkW@iy_O9nQv6Q zoOW_A7tn~J0?kF_IkBaCG6ievza;S(+PP~{b56rz*`50S!Z8pF1Z~%_h*^R>ooU)_ z&N_rAE5@3?xfyTGd(%r)OemAW>f~9$wu#kH(<{S=DXe6UN%a(>Y!Cxj@B@3U6ntpK z@L$+VUQgXOeTpQ-BJ3RhpJonqB7Y6J)2{LPUd?iCL;&a!f6k*heSy)TMKJGt#)a1SuPuGQ* zYTl_!75w+%puq_-w?4lb6iLWmit%x$#%nC_^VY&+)CwF4R`PFsi{Cr$qBkvD0T_hv z#@0BsPb_32mJHQo%6o--fSd?TnBQwQq=5OPG@(nnsHmvJsm{X$f|gdG;7M~xtb!su z-4}C?O)mHQxHIIgz8X1o(t(JneYD`xCkEwHwTp}f@zX+)4;9B#1G=ygDHulx?g@Vh z*nMB9NdhSd>mtL%;yz6&>TIuia%osr^*r{Gq2a@@47Auo9D-(IcoA9cR1RDB{}3*i zFiK{4Zbc-)tOq#~Y#PXvWyB&uM}YYGwcEyBD6(Gz$KkKN$Mys}4aVr7)DZQ#wwDk# zN=kY8uH+ikbzkBMQroSdSLYOF1c;Y8T5t{#-3{it`|SR!l-M!1R$gH^mw%Sq%xO&M z#4jN__A_zxd&y_@1zADeteaXYhM2MCLi4^~k52{cKL*X7u8k1kI`5Sf?0W8;`BV6L z?4>|sDtt|+i~jIfm>gw0(V6z{Fs=j-ctFJEy(hu`P|`h3J>Vcvv9<|E_||EX*Ge}T z)|(l%#;AJcnJY(5S=}3|Mo7hHJ!F;mL)(y6uo+xL2wo8Ht-ND$cEt6*o&#J-9_*jR zs4R4LFkVKMDHCCJkkQ>?R8S~_NH=oBv+vK!DUA0Qb5Wa?iaK_R&y}#7Gw={6V2~c$CV;Ndk_^{mnEL%>FQu&8ZtZTw(xaY0b zq5{HW`<#Q<;kGI5TE!F%4CX12-W*^FUpqXn9^jV9eX$3V+a{@A3 zyq-X^b>&1)g;xznmhqtD$IBr7x9Cj4TdEo4!0?OUeviPA!UE{4_*6$WlFS96i5Q;FgFB3Hs-4a zP21i&A7=_!T3WiDnJ{Rm!?~EI6$As;_!$QE^?g#H>?}(q81SQ*ccwLFzclnuIaoSk z@)J)_SX|zBBH*tqAs7$;coL30kGag7aK0>$L^PpOwyrRCvNM_5O{%ATg-TZPnYGt1 zxTn}ZIS^G2Mzf^nPr!Vj8T`OrdM{~@w{cRd-RZ!Te!%`@>YmXIZEwNinyK@!p;sXYNsy7jghIW%1?6b)*^zPLypk~nRZrN2vOkv;b$52E6l(DFXY^72VZ%X#2th zu>*b|d*&R2j!oG`>*>UgD(}W(p`NN~M+NjmHFU z-1UJ75nLr4^lu^few3H#ClgXRr8{J%k}Qd#Zy_^aS?2`G%F7%3bo#<|JemH{nxMW4 z)@qK+@<~F%{&J5ETl-pJ?Xs{R+kL}#Pw)!WjIQnpQkxIP^!{vLlhK(ag*^?ILOBTD zOGEmd+jrh?2fwSjV7E8L{mIhl!@7>||AwT2CBJ+i*I5o}v0uV$UsXSE1p_}ynUt zj;%IaFvM3%1rNx0Jci4)ql`6`0qpVa)nH}AT=50t8Zr&7D@)iEHKbjgq1c}>27;TQ zjsK4^@8YxMN=Ou}WDRO0EQSV@licd)bLbGt(J6yn?~QKB#Us5y_A0IG!2j-!oDJ^} zk(%+K^+-JP~9g@CXS;52-((xRlxQ(=uWx!s3)rY1O7Hqw4 z$l%GYB2GNQ#5nl=?k5P-5hH4`XDDpGaE_U~HrAFDDR6o4BxFpJwmFfDIxcJ9vF|IU z9X&*QGT=VwJ>ki?LA)KT($Nkml*G$Ew2m`0H#g^9T=yJ&8J>lea^f(@Pz>J#RaRt5 zHgr@8<@Q^jNn+#eb^5!Bb@ggCkKqvV;x7b7@C3m7U}mv(W<9>M`}kxF=7$!Vp;L5b zAC5^}2F{8Z=@F)?Ua@)1$}S)ymI!m>oabz@#%N+h7Yq>v+dPUfpr=PEkbj|=rt78Gea}y~=of^>F5TEgMTz`Euj ziFPqVksqW1L1`EsVlv^O{-gSyR45`sGki^&Ln{1k@MNcHiZfMi@wsnV?lCX9?!ZI? zC6y^rRvMZgOK`R4o%eEGRMC%jl_{1EcwpwUMtwf;7;K%LiFA*bR(5<3N`%vH-^Sc{ z2{?1fT;+lGJkizP)qZDqYUQK2Kk9gFEgSdfI`V}w-)ymR%TMWLj(5F#K_S{OtKB#2L#fGe+)rs7UO4@%!D(iS)VW*q^}!7uxAmoWg$(KVRom;+W5Lq2 zcp+Do&o-pdLdqp{6KU!p_IqP+$NJ)YXslzCDc~8JhhU*ao+QFE+#9Q>UED=G3*FP( zZ9=Io$iGl(HP(bsLW+dv^&=Jy@dF6lt(W(%D=c1sy8-*hfYq^ynaEI0WjW$9UtzMn zTD7B^k$KMb>h|y9=Xr{8k0xF{ZMx~Pu9CmAy%D3p|K)?NHNPYOCiotoN$kUjwfEJ@|_+f0n{Pe8qpeK9x@Xurq(`#l3)3mU{`eh8A1J1bXybGSBdthiN|SR`rsk zX0n^B+?ewofAE@Ig@zHX(tk-Qtm>Qy7}PjDvxwUwtMttm*Tv>$o>q}g2Q~+vUH(8i z5r?hwjKS=)Kvr?Et(e@|GXNh8eqx5XV~(Mz(#8zuq<16UgQoCQ2+s?@-2HusB2kF#X@}DAG?ysD`>kcz$FQ{-0s&4FjWa>Kg72OMg*(tD4{ZYH-b zT7UBgh4BZUQ)7`;n?^Ji`7D5+*Q=LZODEsNJfeO(j#0Tb79-Ufd4ISzs*ctr<1Xv+ zakU7&>WfshSO6{Fs(!fhdG))<))rh;omF|{`_prTCw4Z+#bW#g26`trYR|bX1%DSi zWQ>SYPUm>orRpVAh5)%sBljjcM;ms1r_CLQ@T}>Qo2XXH-vb~rZO=Yt3ZQJI+kHB-_~mG<#_#Qod39cGaXWnZbQVMUU{BZvuOvm~M1rrM?&;W8@J?P% zq~67>@;M>9u_%*hvU&Yn^69AuV*2(({<^uxOs1^X2Dj!Mn`s=sezy`8Z_}ePrc-i8 zRBg@L)NHwQB*~7&Iu4jQxs;cEU7ZK(9Xhx3`)Q}R&1JP#&+Mx0GP|q0#sgy4NxYm} z;=pj;SBMqjTaG9Rku1d8`2Hc_>6~p?YZF%QeNF5z z%be+~@@Ti!Z#`!nzuq_QR1n~kqceNx;?_zLB6V@J+t+6xB0Em%;`nVvj-uYX;n?!$ zJ>k;06T%&3(^=GCt31{_OHJeHNo*Q&p->3na|HEnp$osjauOAj`Z+t*` zE<^wY2ISUcZo={CS3U!@p>XqN;QeNX`yW2vr8;mx2sOMMU6jQW!7E|lUA!$Zg*HrZ zZ`f3xNSRAdOlrd78G%h*c*0&TK&nP%2Pkc`U8|@G*-MZ4iq&Ww+DFxI7aG{~Xt7Dl zjmKD7npCSCBHV!0R4s3G>yT0PUsOigb*hFk$31$dDt`usn?z^(>mx;s%Uo&l-5YXr zsVt^7Pv+OI5zn*Fq-R(5&Ez#+%EuSEcY{6m$r&O06q8FI0n|`K5Q%NJZT_wk%Q77R zdU`Jl4Ej8;fY}wrc7rygcy6;=widwC9y8ocP|P650GB}f%%8ZuRjIvIrH|Co(|Zd8 zfg;oXVhZr16)nr!%Hi5GaBaU}7`vtp{t8=$igoFG;sQ!0W`SmrnWA51sR?%cb?KT74=(NeAc+gmP_XKs6Y%D6X}b~_q;#Rw`bw5+AwuymQAJQ zN^g2oTXPiyQx6e>E)dp*x5srEmFjI>Ciw}|(W_lCG9_+X{ZHBMOD@v^Z1MwzO^~G6 zraXHD@r0Xz2(?4AF6J|&-mfs?@vxS>#O*I{JoTS=t4O&6yiIfGO);ymrQF39;TRjY zX>IXq^=C|8-mxy96mvg8rp0jKsq1mRS_O#AM_eE#B%B#XoN;U@f>H_?k z+Ac@)nuU}ub=+tMW6+ed^PGYsU&1QqC&4sqZcxwkXU|s-bToXh+#CtL$qyORdl6h? zNH0R}`L~M!pMHut_#d+Ilh``4=i8;&?k6QD*Y#E{PB_w$NGIF^;SC&tog%bP4(JhL zHee6fiFi)2JjYm;1P9gQPSib~aFF(+JQd5nRRS|V=dpR;4wCPR)q#HYs?7!CPzCfpZ}?%1o`{g&PO{S#*rxAS57>&+U+5WHAAqS72W z)i5(lAqVP8nor=QE zrff&znYWL;W$c2 z5P2ZV&g`wKNNRUi=k}>M$C3EqlyHq?qrwZW^Zw>JC$Vmk8K@kwlz=QrY)Uk4FXmRh zXfePt0Pccp<#8B3UB$0jYOkcAX8!&r_WUMp&PfI!-Q}Ux6iVYdFRrO{B?a$nRPFFk=smjA)=%{G6oMYVBt+`8zU!|D zfsYh3Ung5BMEi&1rsXJ0Hy0Y4fqD%7yuAKg*3bVO53&f~1RggP5OKXzrYiKn^Wri!K*ckvMRX36vn z;Y;w35qfBD&s})mMGS;X&+YoB9)2jayg@4clJC`C*mm%c98=I@Z4htB;xWl()L-*R zb396NJS6M0j3;L_F%60boYSvNAibIz*s?8`in1$K`Woe53bu><`an84!)VNAQ?abJ zqoe;rA{l3S<=ApS)0Od;6&`9kJA*sh#brSx+10=l4#!6JoB`_Y`vquS6L9skODjCq zUj-M}r?Y!hzkLd2j$u`K^CD8u`IGCyC+3Hljqh-i&JUHTyDg{dE8OeM?f=Q-@A}JE z-%mHYJ?gr~{8)?)KY%4M#c5`Yex}3fdQPv~V}$r<3D@U-O(w^Chq{7T0u#&i8DXX( z6>Hxh5o7>=z?$rL%7J6YxBZYxzppcJ63mEC$)4Z5WOO*XhkdqU0z>3yJQ{uaj~}Kk zgxwoB;M2q`F(3tCB%DwHTrVIQf(nL@6dehYXmYGvq{#XwG7uOp`7)reJhBn=ILn)u5< zTw#%Am^`}!u3u{{fA#iwu)f0=Bl3rpD}5GTdY-Q0FUbj36*qx+3f!ypyIek#=c@@g z(S*;IVK4_&YT(MIz66~>e)m1Exz5fv3OqMOm&24>dE_qKQriAVp>OHFT0R){bFrXP zkpRHOt)l5ufCQ6 zrbH$MyVLWmezj|j_VHUAcz0IwNjuzN>|Eg0^@*n7$D66cE+8;c?N-D54FWe))lyxg z#SJ{2_6OK^=5pQ>jsYa)dzvQ>V9;mF#u8g}72wS5D1a;5XTZOR&vR4TyuBYZl*}l+ zj9b_uH2q@UV`ro0t?264&WSkZ-aAjRSHJySgbaz?*(*Rnn_Jyi^68Xi#5~!4F(zI( zUf#XPO!;iHJzxakIN-#bSErd68_oOR`(G4PJ-IbCKL!--AXDFBkWU3g6dOQ&X&R9uzMC7V8{P z{kXD3^CkglLqG2!CODOnxNfes^=jTgZY(oS0uN=d%V3r#A;uy@u>tzmYj@qUR#qmC zYRg-9ER|8_N~?;*X4pjR`^}(xm}vm5pd@A)ze#N#pB(@a$nq!S_KIDYz56?*b z-&Cy>W_K6*GZJh(eivpMVz{NKG3KkknUEiaZ)o3w%#j-cj>u6}v%sJRo5D|V?C$1Y z6nZ|HbvEvcF}-V?RhtiHHi0g(I9%nKrcg4-4iSZn6;j|r#Y4S{ClET4vPn>%ryGB1 z7oe<^%lYcncExAC;Xsegd$WIO6+9WJo$s?34w2GnrKmVRs2ZOaIXSyhX*ZXV-y+9x zhtb}i>Gh-*?)c&}I4!oG(-jc;M64 zgfDdAA`%FcJV5?Aw>q+N>2-7=HMgOUGF{m~P)6K!{ytNN;Tsw$rMXWCev_P=Ff%-j zoTvOfW1RU(gJlTcfN*@0CsSa?Lfh1Rm>? z$<2@hB2&tsYe$9pbz?j&f(#RUAC@ClqNx>~F!l4`4dwT(TOxTo0xVj(sXN~haL#0y zL=gbHyG7Ugt-IEX=xT3D9ahgDLKex0ZWiq+BL$lq*9JxGzTjZH2(VqeOGfWyLP_-aMi~$FKC8TLwUl z1f*#mhV@9vx(-A&K^|leF(Kr!mTa6H9#^v_zYLMp?KP_ezvIZAo}8S-+in)nFSO9M z>OHj&EzrgW4q&z)x8}0@xYMlGfhqvDsG9RM2+yKR1bYD;m3a-=-1#{eu2` z0+u`DCBp$c=~-oIUkpPPnm}}CY+dsGc&O0>DV=*i=CI79nHcdsEd_H}VD2K9CLOH|a%8W?%{3tD3qp4Cv2GZJC46beS#m z@(gU`toqtD70Y%bwsbsBIRD$uwEj+7-|962j@zmc4v6XPOgFO8c^tWr&cW2%CN3{T zobv9M_BCEU9MD8z%{=i_TmI4B3!J~0(DNdimvK3uS1#4@+SSA zb@h3X<`km@BT8eux2pp#QZ_h)mif((4ThmqTtT1GCtCcX9!Iw2%>oeFQ8QtrNvDu1 z3f8;?K-L(PLTYTMQj@7plCHm5=9lEKYr0(b;>M*?Lp6y$kvAAtE|KOvmidL-*@;tl zwF9dFZW@!6!!19#Ha* z;8IET30jeoa8>_rD;T=C40K7vactf`IA+2q?iuR;rOu3zz*40A(k*qtmn*aot>muA6R6r4S%%2 zplFc>0)rBO|ZjVlDaGksETR(%QP>75FUHb(Ow(fu&>7=`5-?CDN&oq_uILQ72Q((%ecUhrB8C$v;kLlUjM?;9QXUuUarIz^w zmbo8KONlm%0PQjci8!gj7rh9ECqK3`c5Ot!XGu)baB5^K$s|x>p zAHYqVifER#`KMx=ufBkse%jn%yGzNK4G3?f@R&KW3rujzo|c6$^au0)ShrRws5+NY z2{O~gDytxPa*R+plnTaxeZik_Y&Jc)q#f)HyzDMi2B9{DdGcWy5c)AZ57Gmtr)EeI zgr_Fv<_I6(Z6LE8JpVMnn7+*cu{!8cWgd^8mv)ie7D0jm!HPryU4f1hgE2$pWqpG@ z@(jzZ?}b2eR@suTa=kmdw&mu1;{=5YkY9Ti0h03DCUMSFscA+ai_wzCT@J57I-Kcr zM?f}1jK?^DWSML!{YfZLl~$emAIZl+Xw6$Z=w9BgJ)xUEms>~shW*psEYD$pE1J4z zx*F{V3kAx6J?y;>E-C{Vm3%jR8OIjWJf-pC3OwCo65c0reWRp6vx2(5+r#%8RQ#4N zz5vtM1b)yg@xP{}x8E7KUq=($55Su!6nf9UR4;gUhR0Yi{cAu{a2_!CAmVqVx(f2z zW56BRwj`qKptoVXsM;glF~lhbsoOJ!aY;m5Z&J^fVLDSkGw0fCxWJ_wB1Yn(lmu}N zDkb>5m)xzHnpVIgYKCp9wj5NC`7U1got72@)w8idmhs0kDTJ3rW?_V;ag>qYz>=70 z8(n_Qaq1TYoNsgHutM^fI&L=>7A2X=GH|~AdF^fx#Bg&>CZ^lux7u<%=Yq3Fa0ZW) z8@?TM-@vYTGU31I+_36Z6)fU+m5s%cpaESP!u#-a9Ub#)X9z>2tgn~mDixu;3ttn4wnc$P#z34PmT53_(zK4NpmP*T1qn_S~AcQb|_KOz)ez z%w6_4L=n_(h&k@=Wq;&2;KTc__yIulEynE^(GOFRO$>EIMVcn99T05ao^=6uFXLjP zT@A=S+q4J{pU4N99%}4U4j?Rn@Yh!mhKzxXW>!jBgeb5v{#(N;AFl=ARiu8qw6YAp z%UM@&d1{uaMf{oQePDRzfkY|1y&*jDGqZSrTi~0$%8zBuoT6%q%AeROZ1*iTW{U+t z#?ZZO(f6@moj{WoHwuS8bRS}zAVXj#VNCqZgqXW zOJ1)d_4=BG=ijED^MU|(i`CSZiI?K%VOAuz-z*Z#)gEULoK1(*Eq6Tx)u-r(J-R(5 zl`elxuN6fV$2GoN=U+nhkOu~tT)uG? zkfxy^PUv$bK0wq$##yCf3(YM_U%Op=vgtv-Jx4h~M$iM;_V1?sVh>MljtNvj2G=D> z^0mSJt$OFV((!1hMmoDJYxfl78j|enpsaoANXQM~iqGDY3OH*9l1W#`A7owA6>xB3 zd%QO5R$$i`4zQ=oTjwvfh52#A!&*DQ&2UBPIz+WQRX`EQY8i-fTh6*IwU2>XU}zM` za9b40HvrH3aNHfrCJQyjaJY^HOQ-R_LT>PV)<{fy`S>&5M9Cm9Mq^znH&4!egpxMO ztEG8qhp&@v@)yVo z=9~Y@%5r7&7x`Iz%-ENs2=}o{I96zSL3fmyl!2?yXx{ohiu*bGgYo6fn*)~nZ@*YTt-kGLKDOGxh!u?D4(n2IJ zD88g*Bxd!|@PbVS3Qe&vnCRNw6x*_YmpNvUdv0?GXAA(XiR>`v)5`r&*%kTp)%KZ( zekG6Z>h?I(#58?Nz&9vn?7yCRvkIWT7AyXB*ENu-_j3-?gnWJt&bhOUJ`+OS z6>~e=^E*)1law*Uv@#2mDN>NFt@a(L?>3hfzmCvq&>XP*4pdgX@EMe3hNMn)G*%0_ zD6{GZ;AIbFT`xmxnrVNpFgLkFDG7D?!_fdNnxbiD#cZ z2c#0HQvLJK?*~D9kkB-=+R~~o@SXtKZaW1$4ogS;j_g$Oz)3QUECDTj_qEX)?M!l9 zr#`dfGuVNOG5nch#$aRLY?=`$rN*V@nId595NCMIyIAta<6NRa`^yWE)EaERZXPfQ zs3s-K`-A|hB##wWZCMFUZr$ek2^F*w#McM@x@XLi!ei`^pn3lp@iYFmk!!%$4L+{! z0@(O7i+IxKv9P9`xw2{gm!`vgl_5%9Y)>ECLM3Ku^`KXuee-YqOo{l^I5ZtB;Sgh) z_bfk}yUi~-&bfibM|Zf;TQF2>+Ium4$@ar3X)IY%2w!N-Y~8*V+y@dx0*@j;E&46@ z=nq2JQmW?5l&0ZC+pCk<)K9JKzwPw>I=*VVvwnLhKg7ivB*+sn9JVmrKWmyq#l^+Z z4e9i(KuL+jf$#!XQ7_m?p_@yr6sf?>><5MEulCK^%dYn`6Q`%k2e-1;zO=ds|9@f84im7k`uudK700PvOHd1W4?g2x=IHeyRb z&qXsx(OZXk_Q{jPucXiGQCk%RM{yg;>5xBm0vVG$WzH*8mzh~ihlq6brO9RnvKRHn z%925q+eu2VneowtaE@p}&4BB#p1aZ+fk_eoK4x0+aG& zT*f{9^t+kAv(lan0)X67YQc2riiy%8L<_oJ8&oGjx&Qg`$yJP#pHW+sxvV(+Y!AuaR-fqc)uJQf*Zg%}Je{mA@0d*mo-U zlX!<@C61CQWeU97c+62KpDB(x=PXg)QlqM-`Eds1f2G7r(psOhWtmnk#?E7XlEjZY z`OUSaKT$f6AD~yqOx>b~l_Ul_!e>D%GV#b%bh4*M_|A7Ck2dLq8j}#y(Sg6--8(bu zmRKyR;N@Cq|9=g=9mU+h_q1qW@PjBm(;iEq{7E@ z6(AjHXkk(2`=^_%bj{ss~(~j-2fha zTy{`$vr(F-7HWln($d)hRO+=T3n0gZ9kq3nOzSJ@j{GPMXt-~7m zYcCg|J1a|NR(B<9{E1b2z((=PWS?VZ)j89OiJ(!(Zc|a-tSX+#PFh;0kyGV9W_zZN zP7KzCg^AS!+@|tT?}#b|!IAf%UYZvY3KHn1A|DDRt_){9C5&j8ZZgP1Eq%6Bn9lqP z!pvA@t`P`Jll>NSS}-&jO0?kY%SzWVa+b(@OvFb^L)JwRTU4iu_~o%Ve9phfgmXIP zbN@%(#V)(Q(seFPiGSqd)Ou6K=Cjnq)5;6g+d%?!`|e0;anLuX@mMJv0kL5}9{CEJ zVh1LEM!fYZ(1i0C`DppgD@FpQ&57UFr*-B_2Kq9J!^#76L#!q_#+`>aQH}O!r%X1JW=%uaJya~#Fkk??5P(-=Sqe{ zF43}&6QpIWZc|uhw1~ONGGtBH6`OsN1|bIL;7s&r+0Adjr2uj6hE*m715eDetg8pw zs#N=L=Sh{jv?yl_qJsRb!R|#_Ri+L6C0*&(waQPD=cZ~TTURFD$szp}Ep&RcGm`>_ z1I4c1=Tz={b3As~Jh??hL4cbjcod|JW!{?8&Tl7d4jwydk7J zrgPQFEi!++%WmhhV--MQeGdtfq4uAriRAH9zhq`JSqwoE1b9`%X}laL*g=M3%@`7G z%+JemERj5)$a8j5MWwO!Wo!Y}j0T~wpP;|tN9e2Fc635ARLIUAPMHT*k5lo4g;LpO zFsoZz=4{VhUyiZZOaa4k<67*3i^@X)Qaffw2nJtg+ZgycPXN1p6HBXAfF)?Qo*|Hz zNOUPk`De0og_7rs2Dj-hmAU`N*_+2h`F{W7k*SQNghXgDnaGmtyOOO1StfgBC;K2{ zr%hSPS_)&yZpgmPSPGTw#$;E*Ft%a{;dk9yUcFzRs;qNpU?9= z{SmQybY4j#=*VjMK%vyEtX=cLGV{&>LdrSwvdsp_|&HvAbZp_X@+tU z{g&xIb`O-Oc^Bn!=q-KDB^Kl{b@`WK981hYGvLo~CN4#kuMf}Xt6#BJiF7Z~#7*j< z{m+igDm0t6lWn__ne%ykX3n?|d5{h#s;zCVgSgOTtL;`ukktYZCZv1v5U+kx%cJ7; zmr*UVR|IKw6(cm_2uhs1a7$+nQ!Lgd%d3zFDV5vA4jz4|t1PMgrL!V!Cf7MRxm z3X1BNL0Qm)?A!kn<+fvpvG4)H5uDEcvI238MX7_C*wNOcf*lssr(g6Mm%sjMlnhh@ z3GN4A`qkDwaPrl{Dva3zLZhP0P^(XF@q=z@1+7#r+=qdsCRc@?itUh{`mud5QwfPC z^@Y?L zFj}uFKV#xj53>gH4-t3{sxB)^&7=i_A90^IMG>sGir$@^=s3?Lt{JzTl_`M3XmgcA zM_7O`i*(U%QQ1x86M8T}?7oQJ5NkFK-S4^H)t`*pnaH5oW^4WYAm0_S>6CE*J>*v& zHg{_PV3V!qu(wSyV4pb=D@bLXcc(S4_AC=EA}|gz5FpVeIiz)O@8O$0C&D6<=jcm4 zam!V4^al-078^tnTc7}S`eaGW>@V<1Y;z!9#;awU*+=3ROudGz4wX9gdxiB3=qALv zB&!F*;QR9no|6a3hgZgUwmw#d z=0u*kZka~7WLIieSwWYyzgiyNSa+s7!JHc{mj-ksVUstOS zB!Bu~#j4Go!xK%E3p9d*i3i^)ZhmF*IXG+wL$41m1JYqsup1!k*BjSlHX-?d!21Sz z*z(;m*W~6S?w4*Au(?v$?2>|aK}rjoSQI#%Hr-%$Iq@Nz!5k;xY}`On=$?&f%n3q>RO9EWc0CrA zlQUKJKAPtqoW3f)1`yfd@61IxY)(aW9<{a?y50hr_O|TrR-`S}gNJlR?{PIV0l;k6 z(OT|^`0SO)3dg$xR4#KE+lao6=-sxR_);^Px0*fLpA9Q`NSSe7vFmeRKNV@JOOFQb zYu$xd$JFWRj6S<|`_tV-ca{YlD3TM>+bdnp z15JgN)bxhj=LaWl9#oBa{Bqd4!BUos0deD0N_(l!P@3s@XOZW~dfK{Be+BccE}Xl; zWCdmO<#0uhE-GsNna*y6bT;29#AEWByZlzeI43^9VIz-_Yd^voBK`_Ma)yZFlhh68 zUKWeLQS--=G#i|lk}K055;Cno3XCq?aT%yiy5M^ke&CGycXf;Wii+etpq)$Rvq<;b zKTTp5F)Cl7oCmoFk|aq0gw?g9_>Jd)qYVpI0bCEs@*C`KBc6T}90qPqDCIY8ha(yw zm4uE=u`N)6{}D4tygM1r*<#o>Y%*mKN(M?U3AhmHc|KlXrOg)QrZkVU-yG@g9y6H$=Sxv2 zAq7*`XMGl@WJ7vCy>WpO0C*;^RtZ=6m5%~hd^|9Lf$;>0h6)c>H+^?GVGA&b?63rT zIT_>P(G zTy+J!@or^uRRoJj0e31|nDfldJ6~Z+0uL$;hCoJG+M6qGUf&EP-ARit;3VQIM}(T< ztYsuwE?j{AUHoIWKCAd=pbj>EB)wZUMT6m#p>nd}UV=?_QlA~MIi8&HrLkoBKsgsR zDrI=DZwpz@;P^d4?va2WUsc<4X3Iy0w}n3PDHTyNF-r+=-umyl@6jQi_J#%Fde5p; z%hbJdid-R}As}Opmpk=J(1?u-rA=TKH(j}z_(&5ODpc^*sQsqtMemB0T$~{zZo|`~ z5{NjzSI0S*oX&MO8HofVfJ%m4DcLAyc?aK?@5_|x%)0~crmO8MwkovzyxbA(_FPe_ z>2*K3obBpY$saZ&8904ckK>!Mg?(>UptC%eGyZzBWU||@MfoS4e0WsE_yB+3J((hg z->3Ld>>YX-CwK;Z6P;>2;+XCqTK?Y-rX78Q6Y?50$#0!@9K>0D9;TaTnAjLzAx!q` z=|2meXF(f^rxnzZLPCc3=VAS&mjtJOXp1aDDb5K_*n8PS_SlZtJKaR_mJNV7NB|7W zVaV9v_=AF#$1WD|z#h()*Hh$4fL3tReNt^n9Rtu>fh)e$Jjfk~?Il;iIrIX-9g)GQ z9E0tIZ9XYT~}Z z#`psju6D7bga2p;TtyRwbiylr>gd6b=YY;;Z zwQ{H?hP*JPX94V$tI$ZqIdiw*h-``xuKbD&%5)_kZ~{Z8fyAEbgqs_Oc$;UhESfPS zUC|i-j`E~|jTTDZGRR<8wA9pk7&OrR;t=3Y(!UiW_vNBzkJm}W)wL)b0LnSiRD>t* z`7jVOhM6X}roZ0GO!RbXV_=AtIwUV_nU-|7VHKw}G-5h=v3h<3S7l{5aZ#z;JJtYS zB-d=(ESK+TC%bmVZ#))dUt(|jWq!9?`{!zV{c#I;Dt%pNdh1IwE|?8d&89@+3p@}!i9lmWoIx;wrSkfONOHv+W#RuQNo7NmSpb(v2(iarq0i` zDRWs#UJ7|ki6$N}LE`W>(m-zPivx^Ox0)_EZ+hj=GrZ|8f`m*8{JV(ac7ZvV)U0cL z2_4S^^HVo9W+7!xnxWT_$9jwJ3HXmA(r%WIYep}k1>z>+e`5h(^C|pZh--IfCN+bz zS&`HI*4HNBXdqwF9YBeOHO(r|>S^&r$}d0F`Fvi=Jd{@RBhc1yHKab&LfqD4kvbXU z@)^1B;XVLlkKRu(n*0pX%W zU~|{~6XBfqQn-Pi2{?aM+nI*i(@0^MbnOCFz85a%h%)@3Q$T#_$-Dhte&U>`Q+7@L za5Eji`NYL(xoh9N%Y4*#m}{IyRt2tFbe{?`n8XnV z7&*k=R_4JM(eo(3Z(G;k0j<`+4nOF>@oDbOxfi1O{Z#l@*tvgT!@HlWL-F zqJtt~^7`Ni6WY=8tEo87Z@H#o@)`d0UzP=BJG%E+3SbYhTyPj&uMR*TZh9n)9cfC< z@2JhRd+<+V{dQ7d2QL%^yvR$yi{09A~K|U<3?(f>l(9)n1=F-1uV)^942Y@lyvh2;Y zWc4A)Z)7TNyvqY}RMj9?2tJgT{aTWHXg91~5#$Z3K^Ow#)KfEb=LOXuG~s%wQRqQ} zWXAnSCmLz;-1n8kJAG>N8`wfT*5xV*{TgjUG=Bf_)X$3&ixn7*pfw+@ragl)alK&Uh#n;B-f_2f{sc9N?ApuTAuF-CRp2-4 za8~VjK#pWQ$vi;*>St~$o>kGr=-qh$2B^8rcBpb&nwTY5-COEY zZJ)r#mv#zMGN55*qwG*XKSN<~w!Kd^$PPy(Zwe=x2(D629gF8-<7HbpSR+kyQ@3@!|mey41*9bl54I>eITvFMt+ z1Ax%5Ff}z*t#^S(U1WZmLj~JgGtG)>jfU~$<|Wm$)?`YiQy5o3;rsw%>%|}d!SH$z zEUdr=4tu*!OWay6uYCp;Kjn{d+@u|{5)Pcw9`(q-t`7^!T?v0{U!N6)i~`dC8wf;C z&ZD^Qfo$qCBDNe4mFsOo^WdOub7SSNp925s&T{_v=eem}i>V(Ig;$&$)ccR-Zlyry}Na1L5}3s&k4`P&nd2Y^`t|3lr8*Z+`EwQ8CSWIrS=6YDj;oyQC4DByOWgP}qjr##$b3(5_p+ zOz=Z~;+y;ay_cbd= za=3uEu6&rvLDiC(oLpQ^WKMm#DFUT0IB1NOXy&2}U*K3%bY9uK$I;mki*|N%RLEE9 zk$W+QTh6{kC88uX`@4#Hg}n7!_+J_0mrmWd*1e2P*R9SQuH~_$ni6qd`F5nU=9D*x z?{*51g@!gA&Gb*kMyU_BM9Wqkz9!fxWbvns4_Um!@t3(s&`k!QkeL}%n?3M0U#Y~{ zI((5Svr=A9Z*P!44Xk`wOE0n{PbHKF7Y=c*xX;)R#BOzyQK!PI^E?;^uc+&{U{T9L zN#rd^9YhvfnLcnrs6w&o4D2LeF#u`tl%m1yiYT@LW;$?W>b2z2t(EjuFVaxGa0cSZ35Cdp;`%7M3IJ<>Q``9 zrugxSBw^P{*MxT#@Cbt_C1x9xPZxjp+4tpwhR9cMC2I{T=Ly-@zZp>^WX;=mQIo8k z)!L;H?0w@nJNZW?T%?-9ODhH=(s@=k?O_1Z^eZ7hx2J+H9=l(F!4B?5|+`+)+kZ(gxq_;sLO2*mt_=sX3d zEthAVoe=Hpeiy_8q-#3Yx;2jm?ApE{`=E;Q0+jI3S`<)1ld2D*>WBbt5uX2+kTPlq ztFWu-)pjt(QK9^OH~uzRa1>KqBYfsW6%K>e`SxIV|J=nN4}{FyjpM;`=s0=`QAKE~ zuB!Tw@!Tn=Ge;S3s>iy_RuRlbRG=1#0$-3}MASyX=tqs{lg~h(>&Z?yVHsD_r72EA zG3|t!I$OsZU`~Ke8)MMk8*fLP1Oh^~rG>HnO^EpUGLdjdP`4`tQ?v@Yc6gSWu%j!W zuvDm07a>d}IP6MZ=9!+Y7kYFB=zjttt}I7Vj`Xe?YZazp0(2UbqTVELav_fB(5g3F z2xnGp`0^4uh}(#t=gzmWav;$|?Iq8|hV`~UFZPA!IqXeghb?-69_*ng=}we^Kj4b0 zto36}0RgCQbwe5S^hp_gt^EG^5$Y1(GwpU{l}%Ng-&nY$Nn@L6X{-48P;1Z1C>r6) zFU&x4L-|--(~Y4JFd7^{3-0yIv-Nv#Il;;|R3axN>iAOMTkLw61HQm_Tmbi3uf^>4 zC)nij6+r1ne;H1lr-L{mch1gph!^re{?nQI@GI|RA}$SxwO`%DlkW(OTK1RNbzWF& zPSe0{+{K^3S@sWq&e#%ukFTA&Cs9E&6m625;kwnTh`kTFn1pic8MbQGIL$mS0N%nl8wa_Vj;R zJ-q=@iFy9D3EjhEN`lZ&u+aGiIiCW3jR%WcCpKl654o&>DhzM`94@tpkSFKAnNCc# zM`#66!l_9YavwsnuVQfEJ_ib}ZnEHnCN9h9Eq?`|$--%;J`eX6=tJBWIc=g4WM7=H zd+TgySQZeS&!lMixK!peKZN98Su1fOI@XZz!}D8U+_re*Df<)NPbUUz=V#}DF3_do z!J=+`F~sZY6c)JCsHF^(6B{FaF^{U5eUIqFADV>n)bDNPX%VWfz`CxP;#JU|Clx2&b2YT~RI`QeIHkJmgE7S~&>i|TNV`hrB5hI7gG z_Gq4jQ&Pvl`YUxQ`s@ynb=K}rcPIqrwnytwUY1yF8hVsqcF9D!xoUR}ICk|1%m5}I z%=;YbPbbUtf!!S5=HsJmio&!ZpJ>vX%&NaAq>LuS)dMDYG=Vk76hQhJon{AsX8pje z*PG{8u2-L)wHuN(9k}G?Pw~y4R7kfeaWLo4rTX0fVhyqC*|O(s)P1d-CI>^E{F8JzAP=l2q(E13`bwe+s9 z>?ECi$u*+yM_q}f%nRVW*Ze2=Gyln<09*GBfeqLl-fIgyJUTU+4ydC7^Zz)F{92EX zivnde&yFV+R<6-q`*}zPou>0uzLbx9Bs|*2+QZpv1QHTI{{E&r(#+&%Sv{4f(hxV(hBX4*%7w!{YE&Kiet6OZk zv94S)@hp&^COCV(9@m_6&G!WkCY29Eil8om?_rxMKKJ^uqNdH$G@pSsm;rOS^HnX7 z{Dd(-i3v;NZteku2diN6GFGAy)}*$QX}O0eZj_M}hjzNqB8FR|6vu|lm<()zzQS34 zTfdEFd^$K7X}ySZx<02g{&&47;dfzt|#RNZF}nDRiZvo#K5Mr=*ZN z@+(Z%VZ?AR!0Pp9oMNSzs{?H^gzZ!&AKS$CJoG5=a-0FEmM80OD+tei0eo8!Ps_Fz zV2+f8)zJ{9nZ^)1^Xs+NIq^tpUJr3Rz}ills+ZAD#;#`y9C+`=M3l}-%IG54jBgC{ zh_~Dv^oq=my%dv<^OkA`z^vKW#UHXjp&Tui7I0YJ>xw+JXLE}DfU-*QkyzXG-zN5C zqrz$d*$1xu9ii>+slVM5gU-RV|7aTibh999m4=PgyVQpx_fvl+XxMOCXTqvA=Zb_o zihwPwprbd}Ivj8AU#TPF%x)_U+B%DYtnD!VnE%AE^+xv5)}8yLt{W&@&HyEP!a=LOS>2xb!}5L`e+IIE#`LQ_WPv3cXe-aE90 zR0H4Kk4V>XKp~VqY6V=>RsD<|`XkRfO1FY`?&dJx?@4TugFzi=#3KlPqX6g4vtqq1@A*FEJ5k2=2=C^0;yRj8VTKEpamk z?c-15Rs_qzLqWBEL*!IqEIaTgVxh%soOw`NvnLL9+y**SMsfbeSoDaCqfq#|7k8U$X{%S;DIH_H zr`4Jq3uKSSV6;yBAcdE6I+z=s}L zfg9C#HU(48pR$$s`+z>$K~Gz9AgBv+8>~978S;~Nwlz_rxgTCI^f3Xqup{H3lnEVZw=&KfK)e>rzbP60PZXLp}?-$MIaFtg(D5VsW*NY4h4a(i7faHpQgV> zxja5JG_>8@562lSP$94&XMBbh#%7;r?2IQ~dMZ+H9FSiQ!Vw3x2m94Udo*vF^-8q@ zliY$*@#lvEQ2G?kA)b?S2?jt(BDi%QL|gPO-9M;Vr;TrzDpB5#F&jWaFh&_d=PLHY=6UCo@)n~r8^C_1`DB=Faw%qa zhiLd-0bE6@yEvj1X*(d^eW)jof_${DS`x9F$F~sD{FS_w#!)t*1GkSCHhl4s^&gple|-hui^kqX09n~>$%}i|zr7UiEoJ;p0wQA!L6bum zhI;jQDDxx80Sdi&D)bvjQeWz%CbtPB28?`fk_=Ob9W4Ud@52a19!6uZ&~khK7q5qPyZS1r(F0tJO-QLkl`%4 ze10Vzm`-HdA4ar)ybHBc=ors@BbP)u_AEvv4MAOOov*fr=KXD!e4}6X=cvf8w%Dx< zc}c?if;;|04nmBCS5=Vv)w>Y1Nbxj z6(&D(gX2XLk?dz0zB8v%!?c;MFYohRaUKDvBVk~m1D#u&YtN&xd+4mmWE1GAnB8!S zLJyq&Z3Nj0FNB=(9-Am_c^)uLc=IG{*WYbBUkn4QpbVmsRh=@z0~#piq?KBKYR%M5 zkK(){d%Swfl}Da?m^#DfjT_M(9x$4SPbC}K+`24vx8m=2!#7Rb-XTKXu^L=E_Pu0F z8(Utl6RXk+k60|z!RqFt`8@1x3ky3ijvbEsb;ltef_-aWjPJQwD>bd&JkrH-aa-jzts-^nCpKW9L!42 z*<>`AW z#D>l}0DBUzI0r5mAK$ECL*CGhA)c5iSnBSBy*$&|Z}{OSfmnxOhr#Cu z7_|vMi?Cb(%%5$o{MB>$>Sk$BiQ&)X||Iq=6Ci#je6N&Q7~Ejeg=YE&uS^G zrklak@9poESvBB5Zhv7CTYM9~`I0~B5S3OW@$0_?`U^k+{@Vb}8<`XGX8+b0PC zhG%K>AP2;MiY@ROC*7{4O<7w`&(}h~oLs^8m4<*wx94&b{O8Zcpss`73@4v40%+it zUdi5TNNpNT$QFeqfv(ki9GB&g*9gGtMkVAd>6!)Olm#blL-d%@8_fp7j=V1aKjQy_r3wS zdC5(nEi)fM23DsQI-jf0fYZe(o9ldLGPDbTn-HI4S*6%7D|B!YJwAMv$a<4lv6_%6 zd4qD9!ojo~c9#FBJiy_I`=}KMu9Q#f%8$Q+ZETX&6zz$)klD52zyNS0(=E;_qEEBj zx)twWr2Glay^P;5Zux!>w4FVL%&3r0F4FIYf5Y*KtS`J~V#4f&mlt5}jdf}v4(Efu z$IjFoSJ(@F`o^R)erxDzB++gi!yt)MBDvH6s#$Tl(tN&*Fwz38vN}kouoG|v% z%H%a@KVe%s;sKiAI1xaqH%(3Hl^-h2w{EdVa3U+JlPJ!hcycJy3?-q?g(RKtw9Kin zOi$-D5H%FjQF(Xq25l&w(R6pG=+J8q;m9~U6OB;Zx-XMI%x| zLDHwSaQ|uPET7_DlDv52ia$}1DMZ-L0Bkcta*zG_-pSk76seY(>yA>h>w3>)A7?3# z5yl(b5YelcH#sZwyAAzNi~~_$7E|tXy9rH9PkpF7diR$e;}?wmjrV%(ZPi+fFWLS* zu6galHKvb-}Acr*B!JM$PB zP)_af29i3WWvep%$^PN(z8O>Cuo6bAC;<22Te9{+iC`W2h8sh+VsBx2!dbBXkh&Ug zXQZ)T(_bfW0i;s4K$0eZ*0H~2baookT}5jU5}U7RhM7VfoDIKO7f92~TznGh?@5!C zYvrge9I1;Axfme&r%mx>P!RsfC^bO#w72V&P-Qv;}VXmwmT zzz*tbOG^&|=3Vz5?M|jHB{An+-$*I{AdphezYml971oeLg88hLWJ0(WXw)GO<_&)s zdixda^g*}c-O1-tlaN~@1eHIz5X#C?z8m&kfac;mMG&jY+#|d|9sOi$)JSIqi0BS>#h(4VBP$XFqFwi72XhGmHNc z@DZ3nbYl*(m=L?f(~r87YK*Lks==9(JNE?Mrq9HKqY~Zk&Eng#xAw`icRg*e9YIvgW*eDN{f`k=y|sxbJNM}vfN>w zo{JB9yU__DZGE-)t|V2BLz;3ft*B?g zBH%#za$Q=sP1evHa0krUO8U|2|M?^0Qmd5<- zPu+{VZiD~20Xjs^5+u(EbQqqma4kLW{DUd;wU%I--`#GiFL2Ws5rZMGCcpFIxK+1~!8m+7R;BnxkjH({82CLJ*pZ{qyP3+5?Y zF&eD8t{cGfY|n2j;P}H}Vw2~xJV$t_+=qt5zen?bdh>5)r7c8mCq-bk@1;Fj!>ApPLAs%j<=r@CKU zsjcup3s%p$8mX=9_UjLs1Khb|#fxEVL6QN>2M9$O1nZEh(=`CWIZzJ>yXj6)MW8xD zBId$BRds5un;l9dy>}(b&q-(a1kQi#r%q2#i^^zCOP9|$_Q*Q&T|H%YwHe!6xqm1! z;?G8bI1|0zA>w<~x}s-%TI^GRP(&U^xRZ=@M>tJdzHb8)`Y5hh(II1^Q?-eQ0rOMSwr#tYzcYqa zW~{_ts~^fFtoEz3$?M;X>$=xOcZ~rG<=7ZnP^fw6S1)C3LDm2{HC!-^Ze`S|8Na^z zlx5$`0v%VK13-p35Dxk;u-}15*e2!{wmf(Q2<{*YRX8TD;~P$SaRtB(4-Mg$3|x#a z+?c;ZtHQc>P`1Boi+J$O)_TBJK>!cDU(Y3rD=bIyrR2d|8RgRhvti3FUFJc<2TFZF zN;S^~SnwToR59$1X~>5J`EIn|p+G!*xfd(BB_L&@{BaZCmXKD4q$h*NC@RBb-Rc@S zB}kRANp<&7P|Y+J$64f*-KFXsEvD^@z21}Kr&-Q$4^uV#oS>% zmBXQby_9)3%a@#Clb+}gpjp`okjKw*tfJY-n=GaV-I@&#u*lUzucPSI%ZagmwRZNHvqtQqG* z?LScNbYgMoML9|L=Ht25(&d6Xm;Hpb-3ID{cFwlyU7|>*PMR)ZbX>{>4nTgH6Cfl> zf}T@b{dqN3&ipee4jLNqnhKr%k#gVEb#r%1A>N$&Zq_~sye_J2g3*om5uDm^M(kcnPPdL4*Ekw2_&ohI#a%8hEs4G=liz## zQ0?~%`oemj+?F%j#1T@I*=kLh&a>~#n#A+xEo9&H?-#X7Wr(ZIhrC_(x;F-^@1Fbi-o^AZ&&Ox3b*MQSusT zfL}u`#0wO5m4lYheq6BrtcM;BYsn=Hi^(O(_%FLrFHr<1;Fw++_;h!s@Kb~OU^i)M z4=|vg4xx2}%*t|Rg@BRc1XIBG5FI82{7C%yYbZZ{takGZRK-B8B7LCAC=q``&#|KX zI>=m9C&PljjAOZX+QaZGxVZi;&F`J7G9Xz^Q?gE6$#$%1x@Y3tMdHVBmPHwg&2hw7 zdH{MU!JtRKc@ZdK3i|UgbUPcvu~fANPDCoCS1~SPIZMA6@0ESnDGjkJ1wQ2c(7o?SUXf7{$sfn zwF>e2XcO@yOKqPRmzR_!0EIHO2_w)ET(T1CH?`X}r-RhHr~NeBI{`W*LC(JLE>d$s z`}mWH*xmua<+MT`cT2r5EwaAsN-V%wS57a+7Wg7QX5tJJ+cl5qKE~XRxVuuCKEe>Q zH*GSpEK|uVaReYrSHut0fg0p}y2Cl;RgHp|F0yMvr2vrN^xM+$(_aB%<1-chF(sK} z*;5exsoH*|71Ec5GrwBmQeQbPKlysLFWGQ#eOBg+qtRqyTd6G`&7)gj-&rKqR4Qz0 zA>(A2JO-Q-YCFYqR8Aolr(SeZ^XF}+z14yBKra8QEA3sG$^QK_KqR3`AzGxkSAoQa ze5AK@U~WFgb@pncm1dHMU>dJ0O4{nL;NEkhuIR5Xe^!m!{xn7plsG_6lb-Tc;^$^U zV(WnAodZiAF3=od18@ziemKG}O939EcRejQegMq7hZs=Rpgj0{W6)-7A-`np4D*B> zL|(NJWp$m)QH7bo38obOr^d|`$Xi9n&$Ok-IAe#}vs`)jTrSnxovNK8!(Txd_N$CT zrPgiBUzIFiyQ<@+z@ZAy(+$2K%;J%RlrpT3&UqB^jQL85AZe}ieNRYOZ~`$TXn4@f zVO8`4YDbQ+s|x*TS4R`jW)Y5w>GarNZqnr~+!Uau@AD$ z)VgGt&;h#h7iD^*`0NZCorGPNu!IJ0Xk+Bg*? zBE%L0&QS*DS^z@GrdbrEiy+*|tCHi2)k%k;YdBbnw(6Wm0yA2h#&%J|{D_mCcH^ zFE^6fN)!z_0mI!Tj+@WP%`1A?SzMBv z>l&%L<^Q(*vJpLSb;)a=hr~{-5fp~XeO|y59I|5p9*h< z%k(^dz{);vT&6!RN*bWD4#r#Zz_CPQ644@@);~J$pio-UxtOffa&1V_r7ExX189v4 zD2zRDUws$1iq@Nc6Pac3Hb`2M%U?1Y%wQ1P2Qy#miVwpK>g*r{pPyYV&4{hYpr}!D z;~(i>k#0mvRrg}7UKA`EDlfxvD~Q&ZS3|Q-(Q~fUlg(LPY>4MssJ%nElF|a#!0T?`a4PQ21IUk!EZBfqtPK7_3<+fBjS+E$r1^lX z0k2PEa;t*(E$+WGIFPQgMR6Rr?*_gPK52CU_!~ZuO#*N$v+J;iQ5ZQTKKKHc07>Ar zEN;IDM0{big~YT#X2zvmc2SDVuH}};U=fsxtjtHi&A23hM@DO%&pwPS<-iFL{XppS z3vY9VX-vN$)d1&8cB*>ns5guVafcJYutiK$>Z&udk?Q2t@F5Y-pbvCCx*?dxj*W{e zvIzR=L49F1nE;PSlJQB8;an0gmG>J!aX6_FDeh*IwU`@I)Am0c*Yj;_rv2X@yWcZd zGDiW0yIJe_HEb~*dedcoKF`AJP4D#j1-21=YE8$=2=EplzRTdYSi--Rjs|nd-j?qu z{UwXb?CeXw4})SDdsBVXPMU*{b2a%79d^Yi#Ejg z3ut>7z(5OMFiS&KehxyiU&R8v>18WuCen&@c%`{*I{zzMJ-S7!$@`%9s=TCH=HUW0 zEPD%ZM&ewrIFvTg!wi*Ne~4m$oEg&gA;Lf8>;%nG+M+ZYY0vK27SBwP<$jLX`~8lx zL6EEF1h9%0gDlSQv9>vx2j&f)LH0r{=A*#bKm<5wU%GX^C#@TZqpT-@#Ny-l)4v9) z09X60f87h{V!_lrHUrRh+Q2QtAE88mVK5ko8VV`L*}j10qY2yp=2Q0YvR2(mvS$!8Wtc4J^(?xHOziK(NZ7t76cQ@}h764uS z+vhu|e~om)>cS?DNaX20zfuh#mz^+MQP2<;0pw|k?s;~nXdMq}qwu_8C5ee#KWR_x z$Q>=v|2AkL%c0Ka3$t`jfXxLi;IQiU9pi`HYy1%?2lYlIsjIVRi&<>1+im+iaMgc9 z>VMCexec>kY`d}R&krco(`5<${Hm4S7dW8oqkR#plqv{W=1qVEz7x4U=`8U4bDiL*59gWWExCbj z?jo=rNdVYo0$2=X?2OpB)%{que_c0Na8P>q&;0l-}l|2pOAAus47eG!cTH`93Zv0mW#-+yuY;criJ`g0|}Bf_6g_zlrf zmi`2sPG54z{~py9OM8p+fBvn%|H|Kg04@Z*Td=H--_NUorceDDh536~{GM~Rjp6_K z=l2MxpFf38fHtM)8#`gJIF8cn`aj*Azv3SD9(W(|z^CC#2k5d+{I$@2{+a*$qrbMR zk`HXIcFX_ljfOjAb1{kkMgCO=-qP-oM6G<;pAY|MqyIBU>d#9*m;bfvK|l(V0PM}5 zgRBto{j`hjzlJ^pDmlQI9sRy9d z&{TUJph~5t==N*L{slIZZ_yY2mwj*a={*vha{CFLCJ1GC#^WhH4*3HCBXm8J@ z+P+$CDH{Imudb_eeBuS?O57fDtH3l471Cr}bWq}_`<&7_8gShHYx(>~5c>1YzZN+d z!LH7cJrX7kW%!UiZ&z(h(^O7*a>xO&xy=Fv|GK3=@BIItl~ke(P92G(#SC;Zm+OFA z*_$U<+_88_eEhG^2ZjUu9y$3xE${8uzd^%#xFg{n=df>Sd$|4|-vxwI77$coK8uv$ z4%|BY|NJiCM!pvwwLhkDLWuQ0Q7rJ5NZbzbe_Ol$jv#++66j%UKZm2NAUs{YVR_>B zR^5)-O1X3&f1OT$z5d^Cou9)(k3J3ud!(8B#`gp#9W*KZh@*oK~ct6!!p2YMWx&PZv{Oh8B zzuR9gfh!!K6TQe`XUx#|FYjt_Pc2W@Mf?>He*Wpd!}fkMZ;;I>{4Q6bDnGGTPBrg9Ixv*CQPyGpCKNo*;-Nwp@vlt}g zy7XW#uG#DDro;cnc*}JJnq^q{&0buA}O%EpIGP zc^1@xOgMLfkw>8>C+ybi<*;9cbTqp6crp32k(oao1r^^V%Cr!u=rQjLPJ*sW`Xiv^jikD z;EB9S#=k%B&*us5>aR8C#*bAR770WpC-gqA6#w{jm~oE^9mRd=SnE^m`%BC4Z>FHIc7g+Wn^1gC!89n_1sTPk2*GPfTH;~c^!c6yCeM^$XO^XeIG)kNC)+pTJfuPGyh-Q*Vu zm8@IIGTNOcl1upv1^Eb}OszH@gmYd@bJ1 zg}O%%*|l?>`Pi(8FMcHWQgHup01yC`WO+>LYd8{((O&OQtLuq1!UUt^9&=A?`S2g_1{ag=r7DnWgz=@Lf7}WjSs|M z?+oP@7JQ<2ey#7gU;v7XH6}MmB_$*`@;4T6ApVMx)vZjkAdx_m`Y{iphmRcb{GES**=#53PXBsr z$SQC`UQi6~`Dw8q zh;x@nhxidL4(Z5}2;A(#i!ZW)Djb}ad}<@&`RfHq_3$R=TTFR(l_Y54+$8LJT*0oY zNHWJltFTXHKeyLL%axJFFV{R7`cVorU^CwruP`D_k!oA+`~=SpX~X_A(<=qVR5s#` zFWom`CB}aO-BlLtg=fLqVt14?I8c@MSiBlDWUeVqM$syWk$-emW+*FbCy?sJ1RbhO ztn!t|I6ix_?b^LND3foQ9hQ7*hki(~H_HQ#eXavH%xvIyChxVeGRiok_t|wwzf=-o zlxhEb2c0yn(CHN*k`Q5>Y{2$TS_`91?QNUCWE5>(5f=a6y_7?t2~0?x5 zqw^Jc^SUIj?=vfc-BaULI1j&;_hpY?eel>j<9+Fbwk1tsw@kT5#K3Hac*^=cg;{T_ z7t5xAhgm#`dH?ZNe@{KFE2;Jx`trgGtpAubmvDbe6gFHz+_c?^|gDP za3%~FW2N@UR+b9_RZm!0NX3BLq2+mdtiV>QN#&SO$izmg!qN^9<@Zcef^5 z%GC-$Mx-tCXQO@$+pIaYA6wdqE-$hoa>=GQv*vUX><3rPkitEjmIl#w z_}~(R)(wTyJ^!b2f9ISIZ%7A*kn~O`rV-@CYJy|;#xzg_94b%rH2E^K&*q(5#@M%& zX`=xprq4NJ%+`FUWkIxTHPE zgS`5b;|>|r>0-O^?PrS@$TjV7mar;Uh6@xzo4OUWrpWmgpZI@RrNqm&%W{akXsCug zUD~TOw@ITt;#_H{Hu7OT)4r*C*yCwnLc9JY-at6I4A8G6(`t*iS;Shr_D;&_YXQi- z;jF64wz=VkyrEZ*N(s>cFd)}Dv6%Az+}6liiJx6|m5ffJRseT5p!&uhXhO&^t$ckjRlaEBb0 z2=I+V;qKfjiN0q8%j^lyMzTuje7b`Ey7VP;(`^_6?9qaiU*=f`HTL8yV*RGP4C51J2 z=M7o-c^XU_MDhm0u3%5Aw_kfyq}A4#JaR=7+yE?+dVO%gkQ|{v)Lx}EGlIQJ#XsP# zwZGL$$)SHVmsSqFkL5%lfrw7r1>Tz`H3{o;m=(s`6Av%s9AsB23r47u6983MJtK&i zaN<|*PI?9ROyQf3?J1d4*Zy=Fo1v#?5V_y4fAtfv^kf$-dD{Ds8kR$ zP@p$vDv>d#JDtaTc?*UtT8hdx)kO3d8mAAPEs#aJ=WA}IPN`#VJgw#kkjyUk!y({TIpX`T|v!#){q{ZvS>ANoRT?I~+c_Qs!;+dADM1&#HB zJq0T3JzhpKZ|E;ZlWLHw{7p{UvkSiVhM{QJEEK9O5r58r)UdYKNCM_r@h| z;NAMb%4>>rffKD4b;tku>tIMZb> z*s8@Kyx2UKbNJkB3SVk5g2yaMJ^w0@;ny^6qeFTHv#3jmW$x+{iG}C%Y)Ai4;bbyX8l5M#IYH zGE5~r@PZbRcIM(!Q;#&;+{>u5w&NF-U@rrMbO#JkV3>}@ zOt$s9p(~Jz=)GxqeVen&(3)#o|Iai5OrqN;#mbnSDXo1^()bo;2M&HYFV49~Do6Bj zK%WycKN(Yf^y%HP5qtRnZhsO6q!ELpX1@2U7neig;O=h=H!Sv-_1lcdL3@DG!CVvs z9TszO&057B2ot@-Ea%;OZ`q*lu%^EuujyJUjnvt-^i7IK+hdexWTL+$o?*U=w8tu5 z#BhH6V^?IfthFcJzW-x_0s*tVpxdn^r>a`F*{%4Cadj)g??3TF*RZC`K4v9NtC!ZU z3e&2;LR$62O}T3alKD4t(H#Z)D1~1uJfnrl%=S4nB?x#9 z+KEQ``M6G%czkwok`xX6P%65_8iKNSrL_xFX%%wAZ|oKZ?K*dx4u%PZ;Y^cTyPcD& ze7?^9Zcb+-bA3Lrg0`IuDnrTkd2NJD^4UXh<+Ch$AMiiZ+bUjY_ z*izNRs>w7p#Qf1c*?K;ywBv!~8E8#;6mZe*$IHZ~2O}8L6S`1tG#GG6OyW zGeIYscyjL5TPtk8o-3fLe;>Q=6J2zM?p-Zea?3C6-JF_;2^lXkX>PT9Ss%WsLFqFS zuNG);K-_*j8Wm{!qMV2;POKF6maTzK5$PD4ndRZKCl&wmI#w4ZA4B5yv&tKiD)zft z0s>WW!CD4?@tYb z1d3A(9o4i{UF3^gFNhQ-2DXDP%alP8qQ4d$#_%wkw8^0qz9ce=-P`=R(N1&>gr`zD zZOPco4Pb7hvbC{x)=z$w?X99gf+9|E4R>N8N;2EZZ4XG*er?bpH*hFUE*2W8=O+@E4 za^5Iup$~uzvKUgvHXWP9hVhYjWP`INQs|R`8V$y}4haP5217z~#V$bs4nazHcoh-htgaWt8eA;1!1zTYP%p@SexbJA`Mj2|6q#74$~-#Keo+b>_Rj<#pAf zf(6WF@4=9MSy9v1iP^JE+0-=FO$jy={swQt}WI811|2ddl_dV_WeF`NAmlY`~CM?A9vC*`rE)A97St zI3Ohg4~zzW&+ZRhKgnpNvVf&VN1GiS(m+G3qW4`y2{TNrn)v>Yt)GIq*eRH_x>(cE z4I0(O55De6W0VYhVt?w~O_syb4N>cZLpf8nZ3z%3w@swV5^DazC z@7PoewAnRcY;zY>hBdriAA#iB=Pbzz?dOTm%~v%;*Fh+G*3g3MXv7$}e{?ix&Za1`t;oLGbPcf2YBz}q!IKlF?2A84h5vlaj z89m+l+lgM=lqMKKAjRI^`o=o0=w6JQT{Mh3>!*owIzbcCU=ngvrmapL0DBrj4Epa{c|S zcJYyU2nI6@nc8p2`4AmC1guteXaJ`Z^tc@Pv72-M;GZ?jS*@fHk zvU#}sW&1mfG5DlW&*IN&_8{#nBZC=~o0%SwBi>+0{FtUrtjP)0oRX)xcO4G9uWJyL z_0zaQbG-CiaU%<8|FgxI-}c;fSg_srF&w?CCCcT?zBq6hRSv>;L$)-UBLV|oV+tinP+o6Y^L@0Pk6dQks^ zIKUzC(5!7gC=+nth!N~d5T{~fs%O0fxSSdG3hBr}vQ3>c5%Rs@;Ui!_@?kKL5>(GI zSzuO_PCG->}F#cGrTQ{;l=(bx_>}z&V!a@;9>cQwUN$Phc6`*pPQF0#V z#Qrz|GBQ|7dTpKjk0E~I1x!Ac#2X<45JP#3upRmZMrKb0-|OF57lvi9hCE0(k3o_+`y9W$ z;PI8fqO!4dH)Dno$bcJW9}Vc1+lkQ-#h}Nm!A6Y}ezLdxok3V7NeN{ZU&^5rC#ydh za>u(@^NlS|JMEE>GM59@c`oY3cgTHk)HmlG_*3E#be%#9i`9ICK-VLVb`+bki3g=S zEdp;d30olpef`kVb7B}o_3tjLzgjQebyalPHp7FUKgOCSFG@(0&ibjQVZ8gga=gf$ zYaRkYzl{h5C!fKHgcU7OA#7K=kU+rtsRhAX*IDO@5d%03=x^WX z=+@qx`faq{N(SSiufX0pynr6`s)6cfs*hx#0IF`^f|YYL52fT%jz>9l1BH~0NAp|V zs~8gM2>sG^K7$T&!Plm?yEnu^r)9)!8CSio1LN--#7y97C4A}l(9ZH zm{sB7h6iqh011Uuu8_%VlAWqsb>SRzO4%^&y+s|%3bo2zDE6xdVfAzRwWJq-ACS|L zSd?WtonJ#h1!44OMu?3sko5rw2HbfFrCAr*$Q`(`ebQQ>&H*~m2=*?x^Q;%d6Gx$HLxl4#!HbF*>8V?oMB3GroH!$i zLN^l8>~ILh-(;6tdnEg056Dn<{Hol1dDz>bWPo63D+UV7W=;4`@W{61%Os)=nrn1Q zLD=8Y0kSY3*F|3nCI2M4(embyH?YJB=z4H3DwuHcH_;|TZ|pv+^^AO=pGi>7Y#Dp? zexHKJdQU%CyD?jspymcq{8F6TaL3j>y1G^16_Y?Wnw%wJ>B3JupwL(ecQb!eb*|6{ zw1xQ%B!@oadbU3D#eaZvcBmNcsn?St6($y1yYV^t5U`0PVHGCt95U6fQG@=KO!3zT zDxd^~M8jcZsLq;kc|?T$Tbq6WoPo(l5ZE0OAe+(-=_=)vqhm%`gMa(F3&Ahj-&RgZ0mdk9r)Tkd!1$6l$u} zz5#CEZUJW+x6jEBJ{x}8_|)%HV$a#lzDl>fL8NyU#f8!roh5YsB@y=MABgU~P^WKK zhX^{#wY}+GwCe4(TZ*}6$fKz1+xy%KUEr4PGIj(aW{WfVa!X;+N_^PkihdGsLKX&{C3X`OczQ3wTUE&0lYmK{<$X&NOP2<_!m~ zxGtgS#6wg+MrL`%n=2|c9dSOe?;1!9k;=6@s^M`g+@=kM?^cNVFQNOgHr|%U$>&iP zvs9%4LephuaOO~$R&geDCo;XT=vTVBq;P{s9vmtq8bShPVebd69fWCU1u>F`UbWb`msWNFDj|H7bd`p*{gkOJTzeS~|~&5wrG zu-zz0q!JsL_@N3K^zP!fZ-fwCCp7y!{P5vMbRy4A=sPsQHU-_(wF&`$Bg>wOYHZCm z1#e+`INZ(c4Ts){jTE`8#_I)?kHm|i9>vUKgICMWW0iba^?LUxaQ09$QIz-~!gs<~ zd6H`yk_!U24*h=c1!a(_NhS{f$*aTbx*3VvgXWU+iiUsZC#>(1rT`@`lBzgGE>2Nn1W+OJx9{-iV*a4dEhDR#bscNM_@zQEUEo^HyH*kseYJ|%5>|io|5y#)(yETM$`*LX23`SFe$(>21Z*5f)Y;sFen40{&a@;wqpxwh3gE7p{vft2Vrw^h;QVd(Zf=aMq%bgB>_W0p@Mcs zYE-B;A~LM=Og`*qoa_(66Q#$H?fxeoOBN1;gH$=gi|Zdn(SLDr7Y72fGh+G6o&7Bh zbb%yCr!(Ynu4CC?XT!EBr!qs7CZDoFs7)KA8Uq=H?17(4qy)pFmk#HgG0%XXyZ4mc z?>HCmn)MKv+do1VUxf3G*(VVlx}8BJN_c0nC9)&f$7Jxa&WwBGQ5o~u@-YNT(gGcd zSgd4t?@TBS5Zd$s(){z#M^fP0XeEAHzfc%03vKGjCrFEyXO^(qh65YZCvZaR<$%v`2g}T&xvxD#Gjj?M@Y~rLc7!G(O)to zCs7Iev2)v-wP#@Z(otDG>5xMCMB^AQxp`11aC4e0%J>$^VLMnNG1M&27D_J>P@gIf zI_y8UDjY&aeL5U^lgKrTcD0zCRZUEoK7{sSdW{fzG+_>*bS-k)R}7gMR^b;dl)SBY zKL8e{MjmjvT>-K^keA((Kc{DzK--!Byq_9`mFjl8DmqhN!d3i^u1E(ci+k~4_66)d ztsIe)TQh0OUGv+F@lO)ZXn+v{ZOCm5Mr@85~s{My$G(|Zfc^5pyI2jGT{jrY*a$KyC?d^iSVWszHJ4w-hT?rW^@2_z0 zukf5K9J`Mi{W*;W!N@5rl_ajzsdr{ZO@vY9btxcVTS zQwkg$K^YU~0F<|S^ZxxKdFjPuR?>j{sDy_mN{0Fhl7k+Sy_c6m0%03{1pya}YlF20 zjdM&1w2?Ar^QNu>Nyn_g$`?%R##PEY(>FHT2u>MEflE;K*4&c&h5H6`TzF0Fw?BYw zG+;Q^G8I{KNrUPPADz;gmhN+cK(iNifL@FsUw(gm%Sb1MW#L6jHw!*`7Pz-0s`6K( z`HZrcSf~IVzW;%Ezmyrxv}whPFWQrM(3TV#`h-yC*r#}nO2n=dE1RO2gLsH(J9?d1 zDa(rh8GL6}P3+dptg_ghbHCaR(5<=Jh3z>~9_^?)w9(6)rQm;>Xr(poWJ-w`cM_!Q zPQQmSK&A9#Bw-%ym4YyCJ_RU2%+HOuU%zoTebzJ<)*(TumW}~6%f#h7#v6W#4G^;#e&Aw^J3qEyF(HI7Q0LY*X z1A@U^6nQsSs=chfL~tPQwFDX>7TY#Bu4lz3!>@EWg5gJWWMuAaVrSzGnjdNL-DeOA zggMzQ3Z|k9z!TQPR08K)-g1n}G@Bu9h81q-g?B$e#xowh5)EOE=>^F+2FzD{Pe6KJ z0+Q7s77BuxLIa(+9A!77-jEFsoh2{*r0WMVy@oyAPqEH*PfRf zf7wFkyS?_3%X;r%nNQFKEc&~;$L2a0%ity7-RlwqhwWW3EA-Xi0edvnzA`Bw^L}0U zsH^A|AX;@@G{QAeLdi6Rzh3j}jf7)9wxL$XCNPeHl_C?X1~j+bDSn(ps-SYmig?K2 zSNJ~545S9Y0Uj+aD7JANkZFle@7F4az7+`TL=|SQ{#(z&qX_tW%YN(}kj`!q7Qytc$y6#wKaRb~*{FN2 zfOprYA^etdYl=ix_+zBWgox6TK+AC3{8~&&)srEKrKOIwS(o=OZA=&g*sT{!Ce0`_ zQN}?N3XWI9BvH-l%HA=xbnPOL3Ma$SY6NIS9xO6i2i8XfqUFDi2g7Dwy6GS}!U`f5 z*e8tAx~$20D+W1QHpi8>s3e6Hye5<6Q?44ifKv6|cHJoP3x`CgJjmEAV#J=oG~fz( zZ~;}%`Z&kvy{wf8^}+&BSpfGeksYcIyT(b*M-`OKUq@L>{n@F-m9@Bx`59Gr>ndsZ zm1oJZk}aBCfms^48bmE7;b>U&=AMsla1iSr>Ii_#W|AA>2PZXCpw9kI#aXorG~QpF zTtUsi1^p#=vgy^*qM!Q6>p*4vNUmy|L7oJ|L3-7xt!8$Dn=y!ACGk%_4p2mfg}ZOI zBiJLGZT``lu|tz{N(@87U?C`vui+6MJxf7I%gLX3fmB4zRD_Rt29d`oSigLfm4vs1Yv?R+P{MQAX96f3b5>haef`kk> z)YKW0aVG$EA1q2bB0LaylH!-goZZ69<>4@Ft_pqJq}h-0mT;SiD`0?+ywoO zjm)qZfD}2PK5MFKFeb*k-}mvk`8nc~^w=z_K&E~5m1H&P87SD05)lI23{NUZ`wf?z zOGe`@NaXSznw0n#$r{#-uf(U0?xKQS!D+WY^c9Xb2tg+&ro}H%E?Lj6Lq&Gr{Tzu5 z1Q%q8)F!^!%xI;4^N#5~GAeLM(BiKzbc}sUt`I>|R}`?KP-%o)9c=CbPC(HjM27vI zr`i_^1924O@k6_TB_Ci;Tq5lly71=8LHKDv2eZ^W?&v^8kX0~?^|!+kXGON&s=j25HR{r_s^>bA6?{v&zy=D zk#!H$TLeC-9<^R1V-|Z>OOvuRcBI5(<}2v_Zbv)@3ud!VMCTRE$&>54wJ2S-+CN$gtcoZ~(*BSEngEAKpbNRZ*t#3pHC0fw z)>?IG;)l9NTjN5KFIkE3>V~W(uEzdjoBW>stqukP;Sa04lI#cX>m{b;L5FgH0z~5T znWHXRd`#odKAg}ZngOTKVmGQO&~mESt7hZ4n>Lg%q<3zZNWNw~$OYvv&2l-t4}7!oUnff}?;Je1i@=@DSKCGWs*95VQ$Jsx_d+XBkS%N)YqZeSy#t0b&Uk%sm z=<2b1M~(xKdO8(&)S*(2mccoH^k7t%lB!*j+#Y_mEx4YR=1VPD<`B7?-ZoTpUM{fq zM^h4R$xT+n;D`b;Me-#iJFZ;`Zr?I^|kR``2J=gV9x zeY8-RR^FRXA#2ATRu4OQQEK#Zt<-p={g%y_Q07X~i5{WX8!o>IQUJSm7-6Ev0t+Jo zXC^K_$BPRAbV3+k@+tfsPB0^Fq?|mM^ZIs zaW)z@*5+jTs~$*2%`5PGC15+?6dInu|4lG}GAxgP1Nfcg7_X_InEY6&sj2)g zYwEOSpSECtPwEot77@gtXCHkB9|D#8(-eoISxz~8@o^#+1ESAO>?a>WEL_swe#v4U zmTX9g=@o_6^^fD*m~cRE2rR9s9P{acc>tA|yZ7!|TPz;^8O@n+`3hrY)7{jS^0$|0<)72;0rXMqo^P6qrB4Dor~ zaZX6-Zv>^L zGE}K`f`f{xs_^zxGJtsxEa3C%CvFb(4@|m+ZrMG0-H!bDm=SQBPAO}>T_ySFR;QVW z25G{-k)DM)*6-jo^0Xay!o?dEVx>My5$&GGXFX8PZfyMHPB0+0sRTHpnzs+lO7#thI*69? zacYo$cC=3wWNc2mA3!7M?!!$^`O>2mXl4G6mf|Rtr7cH)ec_Y@8vziyY5Iu_#Z(SR zvM)IzAmF^W>i9UPUGWc;PqQpeD-u*gpi^#qGPK)E6OQ$tX@^4cgP6Vk889SH)#DG* zJz1(z>I)ZyOvRzaI$%F_pqV^H5n6ye&6vkuJBs%~{QI{hh>zM-Q$S=}cIRQ{*C zX%F9r0{#1={gBwm0N(V7ephqQOJKR#A6o!MItN@4q zEn{G7)mu=pztIpyi{4ALx6zYj!Rn@+hs91vh5FNVi1z>GA*js+gOrWqMADz3I<$Au zUxNtdA5=djN`JO@dVPB!D5|N7Uwv_)xiBRR9$sFmTK`~wyW*fxoHrmjeEFhJa9Q`G z2^7~&kAUWw1ESdNr~xjH5dk7QKE!#`yJ_ zXd6di)Zq$K0bX1*Q4H9m!^ed-gYx+)v_53qzsiKzN7(J#p3?*r6m0L_z3YttF8<%& zY63o#-lk-%4LtvNYkxj5hlBrD&+@@`QJdVQPn4!xFAD(f*6(zE=YM!fuxdPLrTwi= zpKQJS!x)qnB*pe~)zcr8J^WDl^oVNzN&HZ|z^6m$T4ZS4epRgO8PId~hlgm0Z&*(+ zK_RHaO`|pE;V@h5=^NQKwY7fK&eU<(VuubTYyzuBkTKz>=qJy$^JM=#+xzbo=m=3E z=${pVtS-AS0mYNW?6Yjkm}2KIsL^M{JRprFhVh|~aKM(M9|to2*D?Wr{%D;p1VADC zGCmdr@syBzP|Yhf^r?#s%2jbCNIXH#{~$<5k3B8Cm}02rrjFm;zNm2(J{FMVlVJa3 z5&w~Ur!}q`2P~`{mJA#ZuRJuHKfd`f#s8VLuUt# zmOh5mjELVl@Waf|pTGTA+i?10K$MbIXq`dbc7u%YlXD7gjY)E|OHOwBPl2dchB**7w6>DSlYnp_5&$~oQW6AMHvETBxw9wK zLW2l`htG5KgYJ!e%xRs(&GakI4gP2wAmclM&ED3>&LyvTIZCbHdcM!M{({$0X5^4< zuxbp_zX%Fkt*6jv_RstKxUpQqE~f&fgOAa2p{g?C!7c=t@i4K#R1cFk-n*%5TwU|o zHxJaG`;zMB8p%xG9EX$DXKU$ckvkTp`X5~PyEs>BU;g2Tsyy8BAENc|#Sj>O;25(O znR9Cs*&n1cf?nmGyLr@OW9a_gpUX*0^NT- zZ2F&+SIpjvAy906MSmUBs{p>pnT$mr&400%(9jz25cV1~mfuO{p~2o54+V#y4HH?mLrsMRxbWyB5)l51WinvH0J5m4Ebn z8Td0kMl?^3%?Bb8YiG5|qDX(CZ?kAnfb~4Y{r`xmD9WNLaKFi3E*>$eng-g@Ul&|H zX2Eov3lJ?+B=6rm2h82(y<6Q!!hR@O@CXc_BWXWYBOpP4!&MRqAV#+${3lp9!Wf@c z0{HDRqjd@UPX`x&`-WT0y*)N{hmZJt8!mmKF2`^C1K&sVqEi4af{%-V94Vq5hx-g_ zCpyrKUW|-?S-t<(KEe2|YlpaG@zte?_c4;v zgWhk*uo&J@sV6Q^cM#^Tmzi78H$-yPnp2^YO9zXDUxM=e&kU}f!jvZe85KZr%-8Vh zDV;q!`9b+Gh2V0I2Xld;@rtbx-TA~uLCV>#OhpMc%WDds^{re_8IPr1ZX%6B_oCETo)S7V8}z9e#Qsi~n> zzRX~Lg3GmpQE$L~^4@A9^U5OVC8)+~=RH;#6!RQW zPR9)yR61svSobk7T-TxE{zNZZH-(pB_#~zy(A5Cu7l-X|G*-aPi_r21drO z{HsDfUnG~5#7-U9K}i76XVa|G^O(eOXYatT9`Kk>Ls!XKpX}-~ZU;KPumqktXII_O z6&xe`YeC*LAI!;hT{S-ltr*+Y-^+}+sl`^m*UwrSH8X;HcI;( zSBB?!KHc`69`)S2<7_ZGoix1UUw8*#>jP2JqJUnSqtD?TPA70rTZ;Y*gIlu{kd*mS zdli3|rLM2_+0T-+@6oY|>aT2!-0PA;>tF~eLZ1@I%o_{P{jUxRkAZ)ebc1D;u_+a zDW1OgC&qj_kae+WS)SOmv+xB=U+Ir*J|n8;4*eXW){2xJ7W= zD_|Xdb#&59{{9*66ul?v9n>&wB|BNFp&fEvE1gW5t`&yk4=Z1uodjCDxOV88;jTn(wR}+( z`gP8{WVT#Jr2K>K1*AfQZ)__$&G%u^6O4xw$4q2gt7>>Pvd1^nGgcouZsB6? z;Bdj$*GgldaIE?hJ&*0GxbX+ljG1@+63G#N<1rh(z^oAEmy`(A`HWM+`Y5^thg~G|3~F9&2hD zz5JL`YgbfG8|PfbQQGBYqZ4i0{tJN&jAM$O<5Npo(a$qzOW5aEadLCp`7JFS6V2ja z=X{5Cgs)SoX4<;64amRrvxno8zbFz2hx%_jOy)3BP~QCEo*w$JNM+60Oyf@AopF!d z%xrUU?W%QG>nzu@M`g%`g74*cRW?5<*FRsRCzG6WyV?6;TWdDXo}AY`%9yfsW1u;< z$s8HY&9n2BQ6kK@f49*7adxyZk9jX^9;-JGAk>hCZ90iJHW#NKF&jgvi(G_ETAFju zug2GJ4+um3~Oe7 zk7re>e%oDHyRo>t8ZVyUqyYMRY?vyaX1jhICHSJ&!WO@Glj}Zn)iCwsx$mcH4@dFG zaRurx3w46)p{N1l48j~g4yTL4&~eNLe@4#CyA4Ty$q(;(PRwaN6so^enJ1_lknrm{ zD4M0mIIk^@wYe|9kGrm3`pbxH^&6dVOV3A>mHr8mZCpPR7F~~-pPCNV9X0+DI+j<> z(|ynTcb1WrX@B!jl-EBOkhbDXE`8zg4TH?xmJfX&=c_Vo{4eb-HLpzqd}#>(LgLff zR`77bKf6%sOE00pr#gz3_kld}%Z@8O>%W@A9jJgT00^qawJ|&zdvcSZdG(GO+={mOcCQeud%TqIfhWh9Ew^`jV8gM? z_8hs!c*mU>e-j&lJC#V<_}djTBoWifnZ3r{L+`7KH| zt(~U#VkjF|^^l8aw)C89YJBczs}JY5KJ3fdKHt#XSg7^uP$rHw%qs#d{Qn!tu7kL# z&-5wEt5=nTOW|V*A{NRt9mq?@e&Ot1Z=;tCj#*E`!iWzOkkL|VUaf5DcbS~xaFLrc zd+v=(6TTSSw*GoG5sQLQVub!)U150s)s6awQ4^b&4b zgbVN?#xuSK%RC#1P$;t9brApY7#jgs-+j0q5| zH7x{Yh1omkaFv$T?NgUMs6B6fxpeSre;H-luv(5nPy+vqsv)e090h-NTwIL;%MZN_aet-Xu#aKNI}KN3dUSt{fMn+njS9qB|7m48Tx+fSiGjT z(~noE7pXE-GV$z2ncodjno|E};LT}nwR@Um=ZF6?Z0iC~Sq>fV1W1_cmTt<^)uY0{ zSU={PxA>mN=JRgrS5D-L$c}=A*;Z~SWuC)`DCgSYNEwu-W4IooE4kG~@?0c^5YcPKH1^+)DTAp&%6Fp4uqaDqYeOo32ukj(Z@c5s z`!#PbkHqoDQfL7$mEAGVKa^oi)5s%J&_XUY)I?|Esr&M%vRavY%$=x}ufUgUDrRAe z^A9wY_i?oDgKj$Z2kTXtHjJDLst>rCk!BUcx4E|ACZVydla+P1c0B9yy7@v|_Ipi< zH-QyxmGzUZovU2(b@|u2#o}K+;z;Yexfrf!X1-({~cAw{*D(ib7)KOS)f&x4)WE-KN@r0WA-@9YT&T~RJ-?Wq>Jaacw1O? z9;&l`)qeV8z&=+$a*mUzuvEu=hz(?a3bo>$dWM7_E4ss0*W{97!pUqNmJMNiwQn-g z*)bEm<`#F&Bh#cL^XahnNW-0myvgX&l*wr4l(fg=_uoDA2slsqGimUnB%ZlN+0c~A z(DU)k{8)$Y+;hJRUS5}=3s#F$>mOJulg^H>k3EnlX)Z)~lm^tw#tIX8{%9cFreGIcpmeFl-z-Lihw4y5P?jdj)!a?s}h zgR}g*e86E7$G@6TP|5%q*7X|S!zlm=e5{?yp=*$oF~%%k@{*|GhVXQQw&A=oX~$Xp zrwO&4g%9i^nF41YTCoiN^qyN}E|gfbe@D9GsL!JELsNsdD7$L6QZe3T=3dcSp`Uny zyQQF=ldzS*qRL49Qx*r6M8U|3iSE3W7VCwcOC@ukGk(QbzGVhDx$-_Uvm++EFmjMw z^HJIAtlhwe2igT)62moF(zH~AwI(*%!O7y-jfoSrJR55YZ+sq6%!Dw*kZ z_U`ZX*;JQcAwLo7{x#R|AQ1eM5FEXM0ENM-HF3#9KMJLe?!<`)of8O35c)Jge4ZR7 zr|Qe9e~w=<+*YVd4MioiKsIh=cyoq&eRpyw>w>&R$?ej@8zaQY+B=5nF4j8^NSeh< ztJ(E^^L@Wp9{~F|8KjXd;Pzv4%E)yGoMl#hcdI_8J`tXXK0Dx`bzVDCi3;eh#~5+O~G1w^&TuGc^FUGLeC8zhQ-3jfQidtlFeS=h{{RU=is+$HMr@~)?l>! z(1wNv+%q?I1n*lPC!^g-5(SSwuv`ShVvi(E^yuGD3HnOis$~1qCS^-s#<48_G+0X5 zpKz=bH*7;C+N1(vV&QY|OBb_#&Mj?E0DyesMk8d~ou=u~S3f1*`DuOw*~k-~N*IZ| z&-NSO%YNL=$`AXG()B-isNg)dk4UX?PR1hJC!YMxv-R!=%1KItLH2ikE=M#ZlrdmO z$tTyHDG(>*Y@QX0Fu54kpT)8jAbG)rDY|-m#(lf=+(2(w6yaiIXPRICrES+NtSd#` zC7Mj>g%QL{8)5Nrmg}=*a<8{6fBAqmz{RPSsfMRVdC?;=_=n$w;3jU@63@-cV-yzF zTD``?!XjtXm=D}>1SY<&-yhMrd}+DueRYWUCqO3JSoBq{_Rl6F&5WziW;8^L#(FXB zgdL*N&AV*lM=am3+3DW9W94+~MK@!0lzvfm#a*r3cK5y`sCX$-g?;kfz}u$Z0vcNv zcB}a})+0AAS#q0ijkkFK&&OI0l@hepb5q-%84tVugSsP*EB{;NN9`gD`^*53$^zH2 z{NQ}^2Sb$tic`@$+>dv7vbP++_ZafbMzl?UVuyA?(X~!0RkNw}?4s^>3|J0YT(UnH z{1$bStseMQR9E_j)_1(sHY5`nwci|jCTXAwl6AVb_UE1TDW-vAIN30Kfl>dKrBk`2 zo4%YzTp1GJjy3X3c2gVE(j|utrBf+T7(W*GrDG{O6uT$Tohr)=Vdb%(-e`X{n1N;u zVCJ0K0?6p`pCBmfkRToTrSkGfxnB0Co6wFc%PkIrnx5-ZbZ$d-pAFH|4w9Hz1+%R_?t6_=!HTD&|>}CHYRJ@xn8~X^@xGXGcpfWW{a$LRf>x0E6 zk?{hQQ$!roPctt)`Di|+;M1FiaEb+cxhX{~C0)(A_`oUcLS7a&*Zok{4O}%y%*I zSAQ8^Q=1N6NI=(F9go%5W~v>E+8!*o1Pb3VUW(+hdgDf`z^6R(ad9nRBD7MNV&b(> za$5;V-h@R(Q2ETXpL8^h`lFfuFR2H74B)VeeY&$wNmqc0ErnHa*D(|ofvIjdthoV+ygMX@px_*-{hq_c-=gMy6LB5&%?3X{9pU-`$2 z)0^(kY^4G+@Sm6B4OK1+|k0esX3t;BTE+{wQ5+QYYj zwa5vn2I;Jq{VP@1E;73Bx+TaOoWrAzklf#hU!C3vI`8txS1x)l%@ySB^u3?U$2j;8 zr$_=rZ`fO2fGWmyT27zi+%^6*?B>1tb=7gp_f2gO#hVC)=SWTE-gjakb}XvL3P%r3 z7RM>g-!+~I?#$T#f0TV?KvnD3wQNCA3`!75X-O&RQ0Y{Vk}m0xE(rzc?hXk70STo; zrMsjz-6@?LzGs1YJm=o;-uwPD*zC2Qm@&qfbE@jBJ@Vh~u3-Ew8DTs0ho_&uW!iY}0K7CVK9U-)Ca>aPp!q*;Ua< z)a(zf-Zxc|j{>&aUNUm9XfL^VkxOcAky&0nak~r<`zsHwWUhGDOC9xqITgwU&8s}e z@&)FncMahV*{mkKUqme(^)_)W7k(5FRPdMBYb}1CeL?1s*7@!F=;wi?h&-{P8Ia(> zKc@h}WU8v$kN@Q;gIzh(DZGYS)uIREo!5^5^0tdS#Ewsi>XlbDBO89EI6%Mxwg7f> zNPldDUvku^7^adJ3@~ShSDoJ#!(STa`0dpw?^Q?~L_}#U$Y!9R@r@a*1Y1f~b<5r} z^I$dNeP|tpoy)?_K;x8Hd1A?3uzG^wv~eKIc*0zi!8~Bt7JaHpw7+&ZYka}E!+TbIALmFJ8BqaB`A&NzF;GVH+@EKDn+@3YB)49u`1oH^aEb%Jk>X^DY; zkDo{Gv)PDiMpX$_x!+q#wMsU-fV4FFs1Qb18bF}9%-O=#X3~9V{nZ}!yOgX-mTmbw zce!8X*H`-aYfSd$Y)9C(4`BfOK?k-Yhe%;@-!Ck{Vg8M|l(CMs;M*N@qvx5Pro^!f zcpCc4Su%~2WQ{^PjsQeHL)Q}v1#uXL7J3@jN7U5V_17Z+q#TJ$MHl6IW43bH+csOc z^JM*o;RI!wp&o;FmkZ~=Fd-xpuxo!e6_DllOG^V(I6dAJL}U-o4GQ{m_w~wAwY!W( zl?QKqnJ2o1Gt?l+s9Ozxg_hXF?u`WV+gDW;TPcWG4tbyB%@Ose!V!Em3a~s9o>3?P z%Zee&`8KSW{PE_%au$v%O=He$4^?k59%%J4cx_(BG7F}OXbXE`o$1Tft`P$9=dOrX zgOhwmXez079$9$8q@p`-8GR|$wu!5s$o48U<(TU>0dP}uH1!JEt3=0fQUCi_M~_pR z(YG@As3T}1Ee)S03-CEig}mLI%UP8QvtBXp61;XKh<8-JB;P90_~6ruA3?_mMMp4T z?QcmLS-$^R!SQl491=5BBG%Y@syYeE3U{}93a6JtSq&2X@k*XiN}b9QxoGFD!1sdS zYNYcxj8hh-t|?d8Y*hTiGKF91lE)wa=p`V$)zm&60w!lyi*FQa3GqA7uDWa#SBNBe;fr>5u;XtT^loxNk;R8r2n*_|D6_|pI7L`4F7jQSu{#y%2jPQv z<_sa6LfGz08F1QoJeDDnF$WQnNmb%+Yf9@xgR`*NAKm$sNFsAXg~X5lXtL(1n#K*z zRRY<=*H)_dLlFyR+~$>tPazgPX{&-+4#vKdss|Y%mfzdor5RN*HRO?(b9XMAzcRJU zoL&@~?XaSo_}fs}N-WUt;vc*72bFogMYXg@d$d0)%i$Hm8yg#AY-!L(&WRNg9%ONi zCv=Kf-SsE2Y27)k?WtmzEL5Y6IZ@!deikKQT;i6QM??~z{bbO>vj?}}} zqJ9?F_s;te_2rc2+fnK(vEC`*(Cz!rI}q^LKZ|tPC|WQhjpbpQIjCp0TDvo}P;BKB z7#8(-p4(XVa<}eH^TT^;+0l~HrnHRL&GlCDQumf$9cFX;NZx)Gq+ZS|9!YIPj*}HI zBAiKESUqG!-pa!}rvt&uF|5-PFQZ7T${hkv$9ku@03|*dEF&Q z*FJh&_-Rc3usi>Xw+WEkhdmU#bEU0s@%w9-$5{he^b}aaFq{OShz}AFP+kU@a_&CE zNA8q9&Q<$Nct@k~%{sE{>EEi@(mMifQDoMi-XKI-cr?EUDI{DRG`a~{TGZ|HMnYI$ zPC)Qnrbj7@4`u9+WOjpUr@|lG@ivkwF*WLJiBBJ&z~$2ox?dB8*#pQ(hDKvF+8RAE>6(d?V)|T_iMEMfoYG&0b+JUB4y_%Xmxw6%&SG-i%7WLbD z=>@SO2xyM&Q^Z&R7CzZXwD`0B^dp8Q=|9iozt;h{-^W1J>=BI?Y-0UsN`SEkN(*mF z!O8<8r^S(F96ZhiX)+{Dcp+ctftI;4Gu6{=)iBy^r3d+$A>jg+KfcDXCf4WR$K zOSV!p4s*>J4Gg3;x(JzS#D}1ToQ6ccf;R?VJ=c_`R6V(k{K3G#2|^r_6kE)q6k6{e`kErrGq*n0AQzj{1jX3^pg6 z%&(4LZHf+Xx!|uusA&AXo1%M{rzj{JKB*we)`t#WbSLTCiOJ>JIVkoVRO9P*%v_$S zw6dzv3A7{Yh8F3^KXS1D51rj=@;Rj0!IhR-$Fjh z3uV20Uk$8GQ&gueic9c3{`*PR&oGotpXy6S*JKch8maF?YKy3=;))F&5;8L~Y+|XI zN@aA&kX|4Pt3#PTq=z#)B|aVj{>@?#LpV$V%G*_zq3|J%j6hQ=c2!p&WjU`x->7)T{F-nQiE$?-NB(C6Um8-h zZs5jRu-so%IIuRmi8q|@WAOFKjglt;0-DGRps4mAe&i3P<4+@_U4(_Y?P=obV7xbrj`C6cP2dB$V$aJ35?A5R0rh4{)Uof#eC8 zQ-XG44z9wu2M0Mw!l`wS_r}SG8`P#G4lL)JqlkAO9J&I}qs)|_$(pA@VUpGo&cMKx z{~;(;cM5IKmr{FSjhcP@Nw;^-u>5S=!&Px%%P`8A$_U#Ull{u1O7@+H-xf1sqvmL| zrVAw)gMoh#?H37ZrdlVz!?%=EQD=Y|5r_{?)7zsrt7gHcytX-&JW~v0VtU^j#Wp*_ zr5o}nAt*vog6p5y(SK3NgRZN&05B?xnCAPnq9pq;z96N%wld3sgztiOU0rIAS(&t- zIqqv3Nr;T?(mPN;~SkNkwam`=!A+` zk14ALg%e#;eaLNnqX0jCOG@^h8V%2N#M?)IpnqrPq8hae=RckWfi9?q?hf0<^DugX zu9oe;IzAFvsJ4jAW~mK~@qNuu>-L9b@_^(=lSd}epvp`z_QKZxq5u_mhDuIO?oCus zQ1EC|^cjz3BT3osVuG`COb577Z%-o1lzm0Fv45QGpMtDkIq&ZjxF7ci+tr9;E zK>&v22U*FKh-5kFu3rQ@Y^MdUlc8HM`&r)lL&44Vj~zH)fWHKje}ySe}0ECRCl#HFJXQDB=)odzYy17Z~bTa(60prKm5-a;4+|b2{_K==pRT0 zpu+6m-*@(>Kfm;^CH?hczkfEE9sHinxkflDvWVN$-%J1N6Mw5c`u!DuB|v|D;5AhB z7W~DA<7e^qU;pz*In*D|_|Nw_|3*-RxCoIqi!#9n{}2Ko|58)_@tgl+;UT2de(yAc z{I~N-`fFm^?-lr$mHqW3Kdb6Ep|YJY^qs#}c#^U9+W!U4IDg&If&7 z_x|yU|GLZnc$TwgNP#NgwmaSMp#4z5CHZY}f1lvL;EN=Kfk>HTIq>?9ReV~$?am3+5Gyl->dsS9z*D0C~1)3g1rw#V>SCHNly|E z^Z(s2Lei|d;TIP0ACKhu!9%|WPWbDTw%dPuHa90EqdyPQ-(L>>Aq=D{7_2H8e+dZC zg<+DAmKIe4{JyW71^4g$x+I{}!7a_Xzx{gX{r`M*S~O5u!tF8dy!xaWY87$zzP}6a zKSC4I`|Se-e-Sb6n8_NyfBP?s_>W!Hq)O%v(AC$M0A(vLfgpM4yXD!#q8GpAjdhaTy0?$*&=j&l**?-L1(oAFij|DG#?55OQjkDKP= z1JE)?hvKio{1RD{y1y0S0nH{Z(%S#xIKggs_z6saZKHTJ2iEv5Ru^1o+hWXyTo+6G zZ{-6&YA*2pAK24KC^{|-imd(nQ5r`nz4&X_ew+3G5LqH2n?Rxya{E7Crq&TAc$U`v zVKW7kuEz7cRJBDMhM6dzMi3m#3n+Cky{4>x_8-iVKpyal&1{71zu)T%jwWbz`1`@m z{{-rv49IwFmcj|RK5Co{mMgm)P2wLM#6>g|She1?T27}ppIxiO~n_ecFDfq|NNcZ-d#}q_ZNyp5`|SvXE|uRWm6eTckV|b^Jje)yy2xg z2O7P;o07}RGxZGD^_B?Z`)6XZR9o61e;9@7&CYq|uEPI+a$1}asd$h^BX)+SaoGkW%$L5el(>l{}Q#^l; zxxoCj~jO&Xu6|ScgZbhkok*i-(qW_#mq6PWJ+x!F>pinb}qSgch z)lJgiQo9d|C6ZzE2Dr#LIie-BoxK^vb(&We)9;K zBfihwX(GeB#MQg;A5^LZJ|l|x8g+v&9_@5FL%7oE$969IW}(Gn=ny;tlbsH}DJ!%w_CG$rHCnuB;adj*oB#6f#-)RQ41p z2l|j1^EoK0FTJd&9&WK zZJI4z&`Bv}s7~B^#IIao?Ggyo-PMRyW}iPP7!HtKE2P~C z2e-twS=>1LJA z-O#Kz+^uM8ckZkfyL@X25)6Y|cksIw$&NWWBo|z4>Hc;)_;wabGXGudDfmgCdcix9 z`aj%Va5n_=Yf~;B>GfvMZZ`Wz&Fq}=l3X;;C%*^3&tLyyfahj3bFDAMj_ zhBIk)L|13nc>3Xl9bKv>J=ny(pPLE#ckrMQa9b*E^X}GH$U(Q}>}YSE3^Ttnuv3ma z$u95cRWvfWgm$aTmFU-?E~_tFXUj_%eMVkAS%H zuo${OH&#Zb9p4a3Y|eI?509+J^~Xdy$uH=kX;%|}}m?Su*k7&S^7CpfG z?P-BD;0I9FhvOktP^NDJWC9rQzN>}hvd>cNu=yu#|PgijDr%Y;qx`#dc?v1lk zv^m`VFI~zc2=dw)0*QG^&H2I>Y%wZ!Rmxtd6{+0w(?rT3;&5*_X3+&UE_8VHfilO- zQpmw*T%y$|i|sQQeZDl-+iqXOzRaW)%YPisx;QG6mpk63 zU*EyaKcRE~r%C^H#ohFg&WlG$p}Q7*Y3#k~P;_2+roDaVOrmbmeO14kzUsg256I=J6y3tkCrktj3^O@#!Z1 z%QF?7xXp#zinE9f8N4gB`KRkh?_{K&#B5mGDKqm56|#T8lrm}lPBIF-%njI~I;mKj zy9XLFski`^@QWL+N^>>L9F5~SyeJu0NEI8|A7DOmOzVhvSTxYo<$zDVQ9snfWtcX) z(RIOau)1W_dEiY0>81)d59Q68?Yz+uJO5`nc3#CRJO0mdNBVlvg%c>FPs(2gyj(kO zz4h|CbkK&k71IE~zJ*D2#-7FWab z-la~x<(tMrHOd_3yjJWWzI!||(~Q-?9-ObttT-k`NJ2K!h(G>>XWrAL(01{dhn@>w z;PB3QYtczvW($M~!l_Mdle~HhYVvvwEIx&Rm{&d+!x#SA5~SQpepEpRMHZWIT))|{ z#VqE#S)W1%H-}X6f0RvqV!e*RL7VI+@g{^k!Kq9Benxe6dp! z?51{p7Dc_dYOqey!RBn#bE)Ab#2YHnRfPm{%7(2|?H-ghWk3CxU6o4Rdxl$rz6MDd z*Qr6J-DUU3mv>u_Up^*isjMoUNR^n4wB-#*`=~#5`o-RKlu0?eFPo8qfsT=hDRI1& zs{_cukIiMpg1*SUH(;W-rkRty*3vP0J&P?`-QnQ8!lOM67%N#hh&5S|)Si7d z{BC)m+aQt3ZFjA*Dm2a7n65~eqE!)1zXf>7cC%KS7l^jV{+Th1Q)0TQ@&eFx(vk; zlElMOe9=GrL#R`^F<{!Xigjf0%0z87FQ>Gr&K#yg zx_%K4Mc@K^6k|-HHhJzS^^DI)wc6meGFI0KSiz*Uggz{7Y*ONsN$v6IDnXl_bKe zMXhn&tIg@cI><7_RC_;Ls%I5)j?L#^Iy^er&LpxNQEL@>D`xVYqh#lKfxgOTe_q5m z`tqu+e0SY|zk|5}+hy2OsweyLQ=AQ|r3Oqs=G{*n(<;ftu=pb(2RAfGYgySzeBA~9 z(S`iO!(UiHOL3fE{s)Hd_@6^l>m@`FTIAD>;3Mhkachfbg9!}4G(a*diMo)%w zN*asaGryBAZ0Q5Y|4<*w$^fOxS?eK~VNY37yw%u2#`F`YpB3^W5;C%HDJt{EUgE@z z27k!h&Rm?pN0dzzCxHemRCMn-cIW*fkK zT~r)~r%h*okF)$v%zhW0+&%f4)l%QR=4gH?Rx$F`4JWS9B?!m^i0xIaH=+=?=c6j>PI@Mfqii%ca3k zw}XK*olnPK6Cxo+`AQ*1=k$$6X8`>b|KUa1(+|l$W@IDdy5yOM?_>@IYJFh?YzB9M zqqN13JsE?~?{v=e2a3sNI}4Sp!sJsb2;1>`MYefOA2bBmeU2KYC%&I!Nv%?0T@4Bo zwA-oHhol{ZK;3Z%qf^0#{!6=u%SoMK8Y-l@jyKsu3$?YiN7=56mQjYr+NdHTabDCj z=rJ)J_LY?0%Lw`1~1zoz4nTteutR5Cvd?_IF62fAxwzu=1**oH>)HUU26=#wh*hZAB}3PnNLWA zZc=|m6@#l^CCy*tULh^te}?tnf(qHaL^Dfcug00lsIdsAqrecOoweY|s>G0No4D8; zJKqK*0tkgi=?U1Xnp+>iU+T^}+B?4$S*MRM`Hw^ zEMkLB1>QbBAx?+>ky8{BWgeDagrZvDLA+EbXt*|6XbNAVh%4{K4aB;}1g|mlqz?3| zOU>>P*RYw|b@?USkTOq0M?ypGGla|Mc^hlqX@>l!4_a$nKT7$NnyxSKqiWfmVRs_F z3y9wTPkTRaK9C5cuE$_8!5(6WiivTu1SW2^)9Q0@JOs>ZjB?E-=WCZh6up4)aYdS} zdka;K6dr?k_aU04dH=6xOZW5PGj}AT!#>dySS-my$o+_b(N!%ru!gs5pF5ltyN3lld z5+Qknvc0#|!nR!ARxOb76%z|x-^yk@aCBu{Scs7zh(%sZp-tFz4Da^v%4S zXRcyU{ZITkbj=V`493Q`8;nYB5d&T0yBqq+d|w_*)EAq3VE^ql z&8MZ-$M42M10PeBuj-$!t%f4hB1ar)&_DVLa8UY0JUDozX2*~%Ta;MGW=dOPE6Hq2 zTK;b2bSBca%3qL03b}W$tfX`Hs7Dryq?DCX5#`Ct55hY&~KG?o1xbUn7QF9uH z$E>}<*yD5zGdvRXg{*gpt-a*Z&osb#z-{+U13p@HKw4<;tQ0r@q%E~dVN>hP@q4{q zwqb;E?B%8s0A@a>eX>1@d75ay|4DGsFYRTc?&47c^CFLSY(el5NRD|&Cu_3bdEIna zSy@Y)Pt!UY3+;VWXl!QYcv$ozWu)3=WIq5j`1!f4aA!q*&^ny0y80>Y;_=F}vlxll z@~y$^SeM-eX~CTla`9dUnWm~&jkjGHKCV(*kJl;+o|>fWRd9cH-uqNR(~^5v0NVuz z#3vp}PTSQL0O6U*@nn%Q?;77?tS&4!Oi-Ro950E!g|GM0j{J5$)O0NnmMQ(#jH+Xu zPh*86<}{B+KInAa|1X6(=wbmH=>9@8-S4BLiua)UwSS!uV>uhztqZ$GV1QHv_1Hv1 z-6mK#Q`(R?qud}IWZY`J-uxYTC$84qa6k@IBc8sq^7PF{Hk!cB$f<;?lDf9G<%x_q zLiiZrRL~0%N2((44&8=<5YJ>yp zs}0<~q*v+o(w(W+^@HK&KzjZKzt*Q)pPkYTHsZrxsF|gKP9NwTFOLmGQ~T-^;D}C> zAA{L1U)2A?UIpsV(EnyUX5?#4+V$0s+7?gZ758WQE&}cg^nZsscxv54_vIJ#pg;}- z%q$?uWqJo&`$s#vynw?e#iXz zr%_Rym<>r%Za@2r!uZJX$iU6}n{P3y7lz9PcvHTDBL=NxKUzGED%ZRfo@<{zl452w z_Qf=z;;I_8k;44q2v{_^x%3O|UqQ}J_mxTP!w=4Sp`cOiS;i*XtE=D54HrKL zUIpzlZgDs+BpYj?i(7Z^$I6Vc5oT!}iAc5b-a3i?)K*-s5auM+dJp7xApq?0^w8?f zzLhb(WTty7Th$ooN-yuNhpb%B(OvC1*jz2#o6Y>5)LN}J0)VNm)#%i%DF~TLm34!f z-~0*or#^ysNBg$5H)&^0dun5Wb8{XkWJ{uZdKX<4IKkJ;gzvj6m7B8sIHy#s!-?*# zi}ut7rMrv}$v)Wi6_9FP*qhE`Yzjvr6vJU+-A8dBMFW1y{aK!}V*Kg7C+|Pj)z!t{ zyHVflklrGmOKGb5w!bspFaQ0TY@(d|keQHUXE&MGw&z783ADO854%t$vyGqxms6L+ zxmGjtnu9fl^WImA?m8kjDsV2P=(abmM+c_*@NyBOFp*N2iF`of!ZtME5@hslgl!&fkWO_UFJIh2DRr~;Uh!VxjDXFNLgp#tXo$s)t>)oKM7zF&hBEgbu#+Asn1 zUHE=02M&3BO9rTN?*xm(TJzLZf2?LS(m{@ zNOHJrdym|(i$AGy9+6p=S}v+-4b>m%XAKlX3^yN5Ct)H!bXgYdmF8ltGD3-gnUO1} z!=^_rN34bq7{&LR+yZp>f5~TrO-J3~Wrc5^zn7UVmK+3{q)U_%4Y3(t^|HH^;_}x0 zm~&?rF_onj%!hgVcmFBW&ljfeqGLnH_-h-_0ZDd5J%Kmsqm=tn!- z7w-rzE=XZXd2p0wTQvl>S>EsMqil|f3^7}eUCJ7#>^A&7rgpOS;$bX1H;n^3EEk0H zw_y;)v|hRIQ8_woTf!Y^$UInuP)BXBzI_p?^$NfW8+9aOXEqx9+Y600=pX>UvMW6W zf`S3yVdC6-3K*XNT@0!1{uzuaqSFPn&BeBY48&64!?lJFPiCL@I`Ke8W3>2iXf^V z$Q}vsa-Xe@9*23Kt;x}bk>yv0hL{V}77CFb%UWgZ#uqML-C?;l)0tJWjw`dec{tX& zIxlO+NA(IZdSfE!);Xq%tmcLU43+y2>?YjAUTuZ*jU$=xa~lf0&{1(|zlRu98diuueJeR%;>n62emJF=}&XLPT)_UGeX53jB0)rs_ ztPm%?!WhI`!wz+5p|(OZxEY!iiSKFp!y>HkLH_7xMQvO!VQZnej@FHWhCY)!G?pU8|i=p5#^kJQ2* z3l~>Be8Ce&t^H11aS9PVUm(0iL!(E?H#L;xN)kLk%e{o*tXOffN$Q%dRUtcs{^fwz^=Fn zA#W;^-}rj&R7jRk;;}?zdlUq#eE2UcTm-c+qHG71qT=GdZ@YY$kYTvt6?LocXK59~ z)blGDfoa3Ay zawk=Dubq6Dq9a=r6bIYtV~l*uex8{{E_JA+ozpPnYcE7%08_vd9XFP{Y!`AxEH_{# z`035Q2r&ev?%3cLfB-vlhPLvPg=bsD-0KF6j%4enCs?vQMCLjx3~6XXAeyE-J+6|m z7V!YVX$uIW^>!N9z1IvNau07LCk9y9w32dy5!ZwBu?!rvKiV_5{5^ntlKj(%>87*S zaB2lmZhBJ&mP#CRjJ!E2N}Z3qpGzlu%+qV>iqqk$_i<(DbA6Ip4=(KCt*-jYaswsY zdzWoAFP$kF!Mu-?vD9X|!y|vnu)$E#?LDH_%SnwF%78sQj~%@gp!9U784G~jB2fSk}O!MKbG`heF;AkBATF`BUo?UDfksb--Wtm|B-x+Z&}~(>`q>4hrA{-x>s)rBS1&cjDC<)|u>et!B^tNZSXO zvA%m0jHB4{(J8ytYSopOmTi63eRFS2&9McrU=J~= zL6}wiido(lhh64Q*^eo4zRol#Gca5BO4}-uOYdBk2pGtj#&=rLA8UP6C-*wGF)mAx z!~<4tpoqKm*5c|l2(=b`bAjoEN>hTgd)$fDN6?k{FyHsDfE)?=n!9tuS?rD~n06ry zlwdd^!3_m(9xDE!TiyCYV8 zj3POY891J7#`W|n1}YhnT5&?$ULa-M0=x4mFrPY67Z7vnD2Cqx>f_6eYt6^=uQ-Mu z2Fyrgq(=ujLFk5Ol&aZtk(3Bz@A@#eYIEN2mn#@_#_-iCd>0<|9hzd!7F_XzNFCWM zN9T{3E%rdNLz>Ku7`4}T(%GwF=#+|jSfq%*-V1_-*kYV!jbzOywwHVON1PFpBNt5?pqS+Px>KFnr>A)};{%ne>SZPCx6kq~ zeik_>X;vJy@C(ngh}jt4SR9xM^tyC=wcTZ?*wGUZnr$C^&pWoaUO`6(Jb?t5-2#nJ zIa!Ky`IYN{T7M-H4dl**ER|mkZ#TgN32p)LVPBp-vCH6fOx>S_b^3r&AI6uZtZ%>? zg9xokZ}bbfzxMP;AsusO+4%CxNitLZqXkB<>KiPy-C=#z9Y92o+ zLF+O%+er?p>Lx&1zOwLW$wC~n7cNx!)U5BW7zol;Tj~@xGB{1a1rdNzh-;9ukMAV& zY#&SFRsWz`4A~4EP1ufyuqME(tbbqQ2WIU`*9t+!_q5_LDj5zOOJFL?FuP81YCtM# z$Zt@|v$N(%#wlrctD8JH+a}hRQs1d}!}R8E2vuf&xR-DhMQ)hzohx@^Nm7&UU3}qU zNS;`{xc9oxs7F{lBHv(GPg2sq&wS)@tgXY>Vs*3u`Xd1i4)Ab2$kFH1ewt8N9dz|e z%)WMz-F3x=_}}wQKrIuf1znkl@P5(MS5e~a?ts=ak!|urPpKb*p3xJz0&jcuJ*+lk zN)TNPmMGE${5bt#{6J_kQa@GS$w=656%^Twp}&7smtH&yQ0szcEai;@t48xe2h$AQ zYJMrefvO}{=VVt7rl+(yq$ELH{K{DQP>v?^m=iq5_QYU&17D{O4zotQm>$mMI?%a& z2r?Wd`Uk`gn?&@0Ou=9Gv$yMQWDsDZ4dVEY17cd1L>&?8aXnn5;T6f_ zITz!VLkCmD3)!^6GHLp%8h@W=DvDNx;MV*aWwX*PzO0eMyJQaq`QrAzwi zLTK2Q6ZotUigvs_Vm;BclED(UJJ{X7zI<5?7x<~8fl((nlR@EVMIbVI%+IDYAH6SV zDJg=kVat3fq-UnxTNY`)4#O7Tr-vALSRTL-M!lRc`*Bt0f z3wU+);M@FRv}S+-Swmz(Jsa;)%4!hCDYR92Ii}*fZd($Z!VTA547_!zRXJ!BJp2M%g7B5 zFP?Q-3CSas*ZkH}58A@5-6>=_5*aEnjx#wjtkR`V)T0~jp9#hQRUUYqXRQNVy zb%#J)BVstSv5yU@$L`%Gg@g!iUgSqMQqqM2 z=Qmen#m;-Ywa4dxa3$3HE>Z+9M(-BNcPup(03hSOP_imRSY391m6{Q&SwbeK0rS%3 zd_H>ez!{&AI5K6Dn^rZD=VO^=S!B2IWEHhn@{K4Zxmh*4*8g=Ln;lb_^`&mzl*Y`^ zP1jxf?N?;;rw^)tUC&Zj?--Txu0Z#OxnEj!6wfc%(wq43y(fF`J`}lN;Dc^!0}m38 zCf~TmMap>#l%g3-f!c^1MTz9V!Z1p)Qi50#EHM!<25i!T6?Jd#8EYHk9T&>S2u=+D z&bCVDx#iXBi-M7ER$skc%zy+}NCCaApAk0h%+-08QOH)nDLd`no32o% zj&CM?vR6pN*Jx=m8ehgR$kpg-`R9X2c!WWAI^Lu$z)<1nzsazNCW> zk&iEOI&gv3Y+A!IK0UZ%UUvY*ZW2Z`c-a7dvsU*H7QCfSAAwgdhWFy4#_fxV*?N*tUTGM@ ze25$tdsoykMcSVI)v4wKEhu2INum$Ai_VNB1s#3Z(~qd%`BXbFP|Mt^FYkOz@@7Eq zRE;pzr~^YH1hd?fG4W~?MHX*9cKgckJG$L_+|v~DEu)Ce(M;?tvZ-}KQmf1zAATj1 z)SWv11Md1F1ENJ$)!zlqFh1!z3Ow%Ggz&vm^>$v$Y&F7hrQvO-ryxEjew&7dY)Px> zb7XzjHaV;gFc69~10QHVsqkq!f40WseEhbuLYgSyqbMYdQg!%7&-t7I$WxL!U(q+( zvI4_e1Uo>xw72)8h;063Wu2E>$~>=o-z@9-o~>5YLph?W7#8TLyGi4Om-ePTDMJu| z)afNcY}8|C5Xzm8RB|i3kdEpR;6@pIHaBn{4DJi`XxDi``ZI@j19KKY-3tfPMZt^< zMIAZN9D@b+$mLWU-cDoJQJ}`GMvJc614-om7=|6lFw>D)wa6(<%N|P}I(>eOgbT#m zPx5-c%TD)Kq>IhF^h{>?G?ZMp8UJaRdMgFLL<+vjDqFnoV73nRau`KZ#TcknszfLa zYdb7e?WB$m6Al#ThSLx)pJZGZfLtkL&jU+`ltsM;2!)4ly2xL}ruh)%!n1JIeE}Uc zikwO6uzX=Vi4amW5joptlS>_oZX;F~?*R-%($-{bA?pe#B+#S^=69W;2soalG!(RZ zScn!N2XvGq=-Fu@Z;tkw0G=DcCNbw*u~+D2Sn-VB$*Ix=p_MC9h7<2};%UoIM)eB| zIQ$B7AJiFHw-G34Ge!mG_}I=XO)RWVqaw&)s%cU_=L3=;_OI2v05d~LB{Qru-G}BS zkf3bix7u`qx~eNIC{Y=P`N(?Qp5>hU5WJI9!9n1tcA9#O#7V$2@c2eBLiE)-KosiBST0ocNql z@K`rtlhNdJJn+I>`Qb;G0>c1p{4&q7)aCSh%8db4u<1@f!*pt?ftSJNx;~`GRb<6T zeTL7VmR=>`LO^jVd8M-T$yx7Yv?D5`BoA0=+hX1u%@fG z=;!ogSpBXdy*_V3PSS$n+m}s2fr(ZB`AAfqb2l@6_7mFL?+SopMu8xgTd1o`7i!)< z(*L>46LiUbUv@q+$}Bv1Q-|S%-htsw3Jo#k9G#ibohQx1u)ydN%7mmhLHpN@=mpfsr%Yz}052q|YD%E*IZR&0uvu< z0ScQ~!nLdUcSMA;0@+rTE za9%Tp&~Gy%p}auzl`|sZCzdmeB2G>@=P(cg#(qAU5-mO@$KZ#+274r7U?;kuMjg*x zh0oUwP~tDpzRL9|AUmr2jrbY7mXWA?gqQk7hSirM9v8Nf{`G395a z3#HH<$UDQjBAgGyvr$0-L!N9#lO;Tv}@o?fw^Z)RmLf`Q)H-(_Y73Cf9w(tjf`MS%4+bdTy@q zzT##FKDQE%>669H3no57_d;GTozi583k zXN4*P9#Acw)=mG#^FoG+95)8k0nf?ZJ-rut5|8%qc7@`}ptb^JVzqQyg8cZQp`oPr zuFc&cfr7SBC?9Vi^~br{G%d#$2IQ4ZxrAY9KsQU?H@@by-Sx1y2@RXt0D>&RJ2q+D zuN~lG`!4!-*kmmE-16hHPj8O+ zlZf`MO7h{|MDFjgIrazC`5Ot#*LK(T>8;5wAvxY^k?cEqK4s@i=z&7xcT48ZxhRKah+4B7Am)_YzG zYdLnd%X8JCcl^-Mf2ulBmDE6bl*6wzbzZ|eQS8QWZ7-U`vi$lfshb-cN*jo{Wam4P z>8HJ=|6OFIh4OOy2Y<#^vm!|;cVe{k>qp36RY_b=x+9_m0qmP56sWFC(qYktcrH?^jqM?rQrEDH|Q9jyl>)+ z^m*C&nLg6T6U;Cw7685VF7kRvoI>IO**VpZW(TmC2WdGxcH;reKX%2L3tv@UGqnlW z7{9chaU&Sime^F+Pad9p$rn#w`b7A9A`dt!Kj@yMDbV|3Sv5plM69WafT+P04V7MM=q5cZE#@7mY^AmD%wC)p(I$V_Ei*kQ4UukC34-b zO`ZZL_hqUf+QsnM!Dbk%_SMICVYV=@8z*{pw?=!3Yt)VCZuAO<6gRF%d?)VQFoH#7 zH?~f?-e>{_w0Td%g=T16uQ_i;%5tB?uf7VF_O$QHAnb-v=-}Zkv{|714nx6u3n{{8 zfld3oICJ=2t?<3A{nS3wt4VP8Pmz6-H0Ekp8ly0A)n}B@krY8j$DS~2jz16$MZT

y@LA&L*hOz6q}L)~ zD7PM!besHC(@Amh=(R@PzpkPd1CX~z^OIzFxp})m_j6yW)+D7ggrmpw?ZT90&2>Rg zL5*(d4%&u6kwhzxwHF4au&PpOIf<)#*Sg@uA`$_wN%aLb*J!E=dtVXmh|W=QNVI7dGA%towoQQ}ifEIM3gl`cV&WXx`lmU^I$Wj7WBI1;k zhpc0hALsyHoYZ9S`_#>dlMF7GfEujtIxVeIqvsv@1q@JFUkc&|aCbh!jSAzK`Tzzz zgis}~PY=d+`cisucB+q`0tBO@UR9(FI(>IIZMQ!zVzp=_2SXWyN07jRexaR;Z5P$X z0e6_*!Ah+&7QCOhzon<7u^BMOdImYhKo$5Pl*u&~NIi|arga67ze;HW1mq1@I=IBl zgytZV)6*VDb~rhSn|(JRf-k1-x(?SH?yr7ghZW0J>IZ=n)q|7Ahr)K8UqC&JnNhD+$C8=>js)t*On z5?&21)lC}7ZlmlL@#NJUD%syF$U3VAfaZti;02g+kQh8)y>P*O;>vw6>ltzglk`tW znY!l*MFvgDE0gXO{p?5J{sdLgKiJ*j4x~)PYcU5O0Z#L1Z6|fa`~%lFa6QH(AD3ujjptTkbSz8pxziCMGnT-taBU zWMm?nN~#s}oko0N&ekD>tT0p>13Y_BazLj5sz;c>$FMqEb40z){*j$|ROgsuz3Aqh z?zKw}pG~D=_=@A8aunURI2x#Os`_+OZs?lq%8lZ8we+iw%_z8POD`p9 zFuWhQLo(G8CUdAbAZ@!oB5mVhkVFNN4rmX4jA?kP`ttF`djR=}G8_k$1`K+jjj$V3 z6Oh)BYbOyiNMrdeVNWcm&rS$S!&x_A$56>pE|81H(IOR`}?0m?c>yOw7 z;-)MEZ7!-ui5sOOg>{@(d4)HnZ}9!8!((br+K>7pkwv&#_O?^|PkWVxrnEVF1*io{ z++ctT#Ht-@ui&#>NOll%ldv;~(}5{#;TWq^y8T;Xh350?NJJrP7^sZynp8f7oYu8e zZlIp6>1_mYGZl+^=)v^c+v~khsnvE0e6}}6XV%-R^M)Go2+5>YQo}nYz%f1nno|h( z+K$=tywS&iW`xb^LvVfwnhg=z=jM0?3#nOw-HP zpV{b5(m-V7-47MoW}!EMY5h*f-0G3fB4RA8SH4T43Xw9a5W`?&baEOF`d)$C2eCWHhwV+}Z@$bR z5jw|S4#4oXhE}iP5n%XA^)k!Qi)i&#|fh!<=blsC131UffQ^M@-r3kRA@mw1j0nD zUzQD3+6-Vw5&7gEd@z9dDC3_-cH;y62;@T3!FK{HY;02B7#y@5dN&1dmP+F1`)XV&etcuBqJjkB z7xi-^Pgqz22Il!%aMY-?$Hhm1K)lCraJi?r6)SaIFkUXHW2bzp2}pqA{mO&^ybk2i zAAp|AQldf~X{XPF8S#PI%5c^#@bP>Kwx$WPD#}%CI`Z8}v^e-gZ$Y3!TnC+r?Qr*EqFgzQW z!Qd_S2X&4x#$+GT!$b+t!F?1g3LUGmpdC*4<9tr^AHvC^1yf_JXku}4nbN)t#}Dv% zG)`cU@C!Wxc6hy-LY#Yz#`3#Vz)a!N#>2h$szr(R;sj+I9lJny_^1p>$}n{{S0~G# z6f60lrCEEn3hHzTEgnkM9R^JMFm^~up#eohh}TF06tF&+mY zh=8#qi#Yy&lzn$R)$RX3r=yI@rZO|4fd&-6BpD$iv+R9NDkNJ**2$J+WUqsi zq_WCh*?W(y<9EFsv%CBLet!S-(BZt_*Y&zyxdFQiAU7+u*bE9=Jr3&+po0T zBMY_D#djbx0(i7sh!(_v!}RI5bJuPCqDIgn*JGyOH$W=JzOO;Gszatm#$RV}S(t0> z)AaM7W^RId1vERkISI62|`uS%;yL#Zl^#=tFl6FX6uM) zjH*6~{CesyRd|dtT4m{ZJtzc$0>!$CzU70L##pGVs3LtQF4Zr1Qsoa$-%uKy03$2y zx%dN5tK~edOPmmes$|a_E<9j8dH;8LwBAtBq52`Llw||gA~V?n>{k$*{3*z`j$0JX>?TqpWCGLCU7w3OB;~}`# zrjO0<)vHrPL_`$eDsG1;WycW^s}&{yzA4-A+-s70K6?q;B*zQNax8UMXp2W-c7q>s zE@^7<1Hs{36~HPR7;rsF|(dx`BQhAkBVtn7wSeIko|xD9gvPN&2-x1|$^ zcu9TYg8nh^CkQyvS?C5{Fiil2P8$w<$RhL5169ITrH^^`aeQefx1g$hI`Rskqn`JU zh|R$;U-#NOG7outSBca~4w1Rycp{dZ#qi*zSl49_P1Ik2gyX?TOSdQP-U3h<)h_k! zd$Z=6aH8!I3}6Y!Yyyw%;bFK45m1RcQzC9(h1_^ynT35Q^|8bj~gxgDAqC?ZI?av2H~qgd6>rYE4r z2ivhfJ66QIE-Yh-KCJ}dE<1#RtYC&w4qn|K4LD%S;3l85Jg}<|etxMQ-*kka_mW9K z%~Qg@lJ(M2M8n|gmrsEmL{8`^+v~=p7{5&sp zz;B>%9%msCjL@|R`xtY#iBLwReP)jt-#GEVk!jD+A;JVmOZ?E^{O)&Ri(UG4lX`y} zE%oOBDY8b;KKW$6_a3-CnM?robvtMG7b)7=+@5(nxYfU92id0VD2}?{*O0r)YBnkL zPx?R)``ZG`ucK|oTg4-9sIYm%Zw8h9gVfAyu!B33gmAL<_eX8)br~vfSa+xgPy#X0>W=<3@{s}qOwxxYxS;6!g1^PEQ16!wS*af_{UxF4&tVBY>*paMF`yW zJ@0^+WqSxv`ZL6%N10ccHwGFE;HGf{w)8N=kJ*S)d+>sqIhJwHbKAp;nCao6doNbl zV887r0;nuqM@EF7bMYOf+lc`lK%Oh(VSx}DRG8ovk>r5ZK%1`7X$DeLiJC>oEi*oKA%b^!sk zIX)pFq4RFqja$pzWONWSLZWsp>fdA%*s3YWM&$^Avg@lb;Bn9Tu->2&eZ89wAOuJz z^dASgcil7|en8U*OI+UdF(ko+Z}EzEV9X}x?WJz%jc!nd=0k87r`xsh@h>BokAd}h zeoB=8x@(IPxp;N|$FrbULMzxdd76Y^C?Ela+zwUB)sXprwIZPQZs)J|J_YCmLco@= z*Rs)@>jByI30$gb>y^+q{xJ&gPfi6zyHG1fy4}I+5bLME!}I_DU(1H9lZ63=B{5*B z+er8aFu|R`|Eg(Foa7*IAj_C#AG+(h`@bI4)=syHyX$O#CPcMgm9CebX#dA$@TgkC zv#0)(lTdgh^qSeHN?B{S21WL9BKCo<0uV9wLm%C46xi6>SxWToiH?69_}(?)vp|Pb zeiP8#!ukByooniuDF{OM-2J?<;~h1HBjU!6cPNRx53E+k*H|{PO-}irMtfNTAaZ?{ zSat(Df&qCaw|Te@R3=Ezi8``CnlDBH>9Gr5{@W?`ZQK{V3Z_bYu)nb3VoIb{vTmBN*D*`+uDyTpj98zxNr8?F92vpfu}lf&$J%K{K^1a^}Xu z@lyYXuMu*By#9?{m@H)W0a4ICE4OL6lV9C!mKV9CkJz^gH)cQC7e z!m?W)J?!vdyAEaZDgP>PsbD;o$IEE$jN@dNAz$@E?5IFj(W7T>lv@u4{d&-#)J;y` z(xCsz0nH#=sN-G+e4pP^r6vaCUh_Z`z(H0MPO`)iKsT(7`;lM=#jAv=Lgpzw$cGd&nV7q=OqM#MXJQ~bgcusCpQs*LCS zclzmUosU?HCfa^bnO%%rbNqr`OBh^cu(K09w0ee{rp^4EfDqRg1OOM|LI^d8iLi^^e&UHZOYnG2oaSru5fU&@sP^hF z5X|w8fXB zs&iXkXP8iUz64H)nZQTyet)u~*$(3ql0vGzSKurO{0#fXj8$jF zV8)2mjl4D9o?e}+l#)HhA6scH!JHZzT4~w0993qZypp2&%y)6n+_I%4u+(V&T~dx) z&z}ky@dSzly(aZBVkJ9U6H;||tGQ0Bqo)AZvzy`Ac>|X{0Mbdw6?_XpG zbypIgVD4Xehc>=}vY?Tcld}%f1pE|o9;h^CK?1AXD5HJ*(hTFy&{8M*IJ3 zig+0qddYUX{>5fbvme~5E_c=kzu=Pa$qY?D{nE2w%|x)}Fu$>&>j}{euY@UxW^-%l ziT>5m-_r89^W3s#<+mVL#hLf|Cfr^p9EzD@qmW#6li?xk<0Ml%H240vD0kz%dY~8; zOF<#mHb3Bgxcnld~~wZlu!bN$W&H3Dg@m^$04zs z_7+ro?~B>swcUpSs`9~ce)AR zF6)Z8O?39??iXz^)Sod^gqU-6zc zdy^Ibjp4clx7c&z4S(TLJjfwu=GIu*#Ez&U;!BeU(SjbD@&k8_eiJ?@R7L zK@8um*bLygkfBNb&qZ$>FStYze!_??f<=!Tw>Ak+JfVFORcmZ%qSR{?XmC z{usPjdg|NJJr|(IXdr*@K;?27B!+`DF|q6}^v=^nWYEyPjonSTE=<5heKN7Vzj7S{B03HM>NG1itdMkZ% z3kn2{4GpD>DTFDi8>C}*HcZADi6?cLxd$XnC05~en}eS-iQvvMB99shObRm~b@}-e z*&H^&I+2(g(X{KkI6+`ug3WChXY_e^yIAwUJJn~f;_ZOSDOF??#DFrC%!X`620>V6 zvg7z2hHo&X6t7RzQNbxrqK&_GWt@{T8P&QEj;r8=qQB$GZj(O!VR3^A$Lm+iYmN3w z1uBlXmS^iDbRb@rmo3Y2Tv8L2M^fv8`_(B;B-;Nu6AX`8kshDGYv@Mtve)T5+~02` zK?8R`-$eeCZT+0^9dI@EyOn4aE!usH-4N&Dl0NimnLcE^O6MOOeuC&H9O6miyj?Mx z5&Uv6b@n+TmmrB*4xOBQ($u+6-#9+?J;2a}eXNR@lM2VGD!nqdo`eUxN`>CKNq9%0 zNSEM=Y?BVKAS#9gyvj=FFcQ&E1Sjx-_uHQ@+6?U>tl;gX7x>R|33A1=Z$6idAHQ`w z6}K&xySK2pwL0iH90u7YxMJ2ezI&hM$gnUw`s=q}Q#)&{qC!K6b2m9aH_BhhL`3$P zf;RxkSPzZW0ZdB4T^i^_;qrKRv+k6gyqo!deALErt!S7HBF^Uvs+S ziL>@A4u)>7PO%C$$AFS1dC{K5o$CDU+qVt5+(WiLHpKBQrrmr4oNUi=Kj&8TSjiOM2XEuSRWzLDlkKGYSoyDES_+F&dx!Q=13aJYD$-?#5Su@3ZGG$9s;IzOhYh~e z1Q1e{pqUhGV29(NpLRC3>+@}~-d-(&nH^Cp{T#(4Lyi7(Uh}`P#a~#Dkhzu*b&0^+ zD%E`qH`HJ^p&nA8Q~%uMzpxziY;tn2Hx`b^`g^JH|G~Jp6_Tj}HtBH~FogR1w+Poy z*1dJYTOa*xpIIbX@=@*Bz;&Q*GhkL55DTpXCp?xMsw^k9#b~JSu)gglZ@uHsqr!c@ zhoSEQ)%>K&50{rgtilb??ONeq{vLdUASVE7swH`0d3jmhUJrWfRT?zN@@yY3@d5M% z5FHAbfJ!-}fIP_pUqnL!pOu(C*@CzLFhvL1?Z%w;3H#rCh3j=qrZ-0G&3;t8^QRrU zyl?QUX*Q8u&za2l3kB zQ5S1kTSj90J*h~mLI&rV@f@e6=a+BFi4c)p8ZECJk87BGzvh$I$z7ZvfG-k5vWFY~ zbj$u3#e_7vdCUAsNg<1g&Cq8kbi)kjjhb-B)l93-6uB7bV^NrMJll(w4#9JXI{sa!5-G-w&C zS@6Cw*m~Ge>w!CitCxmtwVva=x3B5<_Qusy)$gr*8Hbrv?{r4LG&n@1kcqf?^X8q! zgqqNhobTK5AjKBg+@D1if7i61&-l5}>JMa^c)!6L$AuWspJ9QTtIXC<2uOmPgOa0L z!F}9Dx;0L)pR`G4@(MXCS^%$< zhx}grV^C|}Zh^;x{nemsC>X>K0>WdS%SU6JabJ-@X!j*pYnXkEqZLgT^LG83(=|Jp z^Pw{&;qifB9xZ=M=ZQc`8tyx=jJs?0&|1lzvmk`IZH-nf)t+3_N`u-AmsC4QR6U8JVhs zxpEyDUA}Q+CXO0wL@-(eES9z4qpa}%q2LhzMtTUlgMtVa{KCy!yj>vQd>n!1e!XHVc(~W zH7ie~U-%^Q!1dDNy|h8}IZO6JyXxfyYYil{&_Re5!`RD$|RF3Oo)9+Kp z{auTX=5<8pf40w*UUw;84YyQJP!T8Sv^T$Xin-74Q@CmDWRyWy+w3IZu6HV(qssbL zOSc_}rAvfrBAZ_2T*H)<>3%=WYq@aQyw0p>4U?lMqYmkjy$p7a7QOVxM}Hc9zg%FJ z>;p44N*NIO1eaWR;j0RMq0_@o>dwCrZGGXaWA0sDr9MJ>#)UeoB@16L^6AFhtu;Aq z%PA;tQLC?9Hj7u%2+U^B5AxjZ;Y!dvFvnrYMsVX96O+@WN=T4$Iv7JjqfX5XDJz?x zEM3Z9W!LRbX(i23?{X>W8dw;u>v~V}y}5&1$kkkE&{ixnTk@d%c+PNclXUS&WtQ(i zbBei>NefJd#_B@S&ixXab@7`sO=PbX`m~C1c^KA3YdM0x5mdzbONNzqTJQI7US5`iU z_DAh=NF1+`9fnK#7B`$Io)VWopyKy==z~-Xm+I=*PKN=i!pTop#%3^Mx~`Xo57_jH z&*xNf)3wOAxpMKHh|25bEW|jk*t<6WyhF?L{@I7L+*B7U|MARq>YSCDcbQ8hH13R+ z18)h8+V&qFdYKbtdSCom!Ta%Qn{D%kDuQ$FvhSOE)CR znK?vUD;hUJ@}qGf-FoDi)te+O%pD=iG46#P%F4Dpu?hXc@wr2Ef^k}jkMi}M=BA6} zuTHuQ@fG%bG7##|&E}d6YDzOVO)R>8)N(+d)b2}`7}R5psvxW--#(Wu9>l>8R0P2% zUnX;FYwO0^T0aMNJ2nGB;@!R-ByFNFd;o{wXumD*A&ekcfPb_mvg?)o0fW&EAt@L< zVU%y4`}Ml|;;rB`Y1da3rP$&kxuRf2v~3!rOoKfb^QWaFq_7wzHI~{a#_{y zOmjG*U4+K--ZGONz;FGJ2*O9oBm8D}@ASjH;~u6%fdQQ6QM`xG5N zD3%So);E%%f>#y&NlgC$!=i%Y8>0uU8L~y?zjiB(4HdCl+q1BcQ8Cy5_(A30Sa%Q{ zCprt38hGO60+RmFE~xc|1vnaN%>taja8EHdrARG25LgmPa3@p=M$S(8stmf#DA8SZ z$#Oa?WC|2?w$W(k7vaH^C-!B+_uH*pUF1{aQum+qtf^mnbLzda%aVv=&jPDybVte0 z8xGmO_NBVKFYvMEI&(Kmg@?iDv#hgFaQ0O6U}yFRAIwLWtaa)Q9JgORQdfh>LP^7N zAbaHLpLLxG9~jkM7Xf0@-Dr{diTksoTd@E}zL&1pR5*3ek|Qs6tKw8PJZsf*Lc@KN`x z>!abyNhea@FDMF4W@(s?#3_%D54xCsBc(UcGOd|y!f5}ZSbuZq=<5Ai=>|)#g$7o$ zboXed22K0kUhV^4Z9$`l5tM|$?$ZcNOHxcR5v_|xr15J7IGT4pTycJvD_qQ=H8 z&ic|ac<3aogEA?3c$Hj7O?{yoW!n`U=`iyFI12ua$k$hvTRIC}gt%(d!vu^`xrV2j zG6L?4^N(VFkGLnJQQ?y==jPwcE9if($uuwv_tJ1ROXYM;j=$FSyw&5?$f?mA4faLD z=vmsh19AtCHLJHI&z1I?$7#+kWpoOjZo@j71rXn|?+gB*Ty{#K{b5j~O%Lf?g~=#^ zoAWIZS8W!wqSGhG&UQ93+1rc^pI~+Ut-8{0K9L0wb5w>;%lL{JX0U5~VMQ|sa+fG4 z9A3v3MeyXA#F*xd=jY7(@?=IX6t`b&T@a?r9{sp5NWyr5Akd0RAAd`EA(&I}2>w%A zHhOtZX3h=WL1^40Cb*{W%tOO0!*`F#wi~coI5fel{8uHRpDE4P*|Iz60xnfc4Nu~w8c5UOODN5p}IHGLK zM{bUWkK2i#S|4#0HE0kEqrEMa@KvWNdbs&Udye*vv6)`Lg>18#1vAj4REM<1ffd6hLrdV(q?l{-TI`%cAuo(iQ zjq4-nwc6C?TI(sxcEfz%a()7GVKnBMW)BEC^E$ulm@iG)a1k)8YbV#ujH)N>Tz~x} zRc`AFUkDp_D1Dji+#zZl?z2*!(x7yI2?zKQPl;2n5?ezC@AoS2{^vaZ5^JkOH63i#N>VYoy%4wN7-H6dj%gS7O%cq^m2?p+_p=VWQxCmS5Y1vznJ$;HUnG{8XFT z4E`2?fAzRAEVKGKkO7^hKyM|owmt1V0^Uz!7%80JF4z5gQxEvr;R_z2M#UWPmWYqd z+Ugo2KHc33u6R?^tagB1PGB|5sCu=~{p7>teUe#|t4m(<_5RC?a-^`Si%yrE-!a&I z^-kb;KGdw~!ktEJ(wzM3yyIZ8+Ut*A8W`BV$3E*{=$P8dVmX>24c)bby(Ba|w<4H5 zjg`{My>iSvM~25qn^tRysqOyy>4G`wTJmT2mLde`l5W`#4%D@!inZNnUs zdcGYtPvl(U>K3{z1G9SbdoP*l9zJbat=fFjwWwwyZ~diwJf`^9$in>mt>GZos-Nq_ zL5VMp4HAhG)6`QsB;>^T%4IJ>Kwr{(^?54L#qy$Njw=IAs>;-C@R*Z`l>bE~05-vj+}&R?_G;WvL`u$FVL zZri=dYo@?2O}1WikK)d#G3&~Vb z7aXx2Ic>cZLaVB|uNsT@-*ueTb9Cxa?|dVdH%ri ztEH=^qx#)?8+M_BL8_(hfcbq9hoA{vXU7Y_|Ew>A5)+8Ue?J%WBY0ln+ehE|L~tz{ zXf$kXd2Z~HbAge|EYn?WjA*Y*_VXjbSQBES8%a*>Lnd9HzpN{G=N zy-g}IEfzl4wU})0*JYJ{no1Pmbz)n4PRwr8=Woic$tkBH5!$*uz@EQpTjiC6|?8r)YOi}R4 zf{g2(Z;nh(31i&xw3sAYe1 zk_~UXS|RPM$TwEQF8f*9#QRN+31Tvr=^Ph^=s9cCv{E}9`fJA0_vMVu!Jl;&J2bn9 zwKPZN&2kF$HVLIX_lfKFw&Zci)YTyQ_)e>GDsfRPBJhp~h({`?iHm_~&pnP+EaQsd zay$B4f=kg#=i5@#_%A9`S3WS(wLNw1{1D|l6q1~1wmOq;YLYmP(RZMbs)>rMQNP^= z;tMw22}|Wi+X*Ws-=N!jcSe_7vtlrzqhqr?6UioC2SinC6~>RSCaq*7p}Eg z&1&$p2C!?162vsmCo&W{-~afWdI9o<_JP5zB{Zv;$7GDo4`io4@jNQ#2yn`{6A&`l zn5OHB{YE-oCFGkq$Yh#j>9WY$<~(?a>kNWLId-*7kofU*G^IJFv2jcxX8Mam8tCrr z)ETiJ&U{DUecdGL0nc*OvzRYbN6s(v+qOn7VsvVR*Q18%`RHAW*Nj$mTig9zWi*&9BaQ-IJjp8R{_YIE1C%jklhc0r zdab^I*DBrxD13KyQ&aClj6W&?osv%a=?zYrYU}zmU!0Hg9Au)?brqTAK;g4tmS*?m ze9cE;tRFhmg5sb-U+!xmE=Rjx?@ztxgf@NOubLIAEU356b)CO|g#`1v)tw>b<2y_* zLhDS?T&t8#d$O%dM&nZdw5PHl!SSS;SjmLNrEj@n!$p?lk9>rz=n7YPR+?FJ>CKun zSCv;V#U;90YAKe?3MKBE_R^RvjZU+$u$*@`AGz0wv;}9+RW*q zMFSK!j;-vxzryGc$xv_io=@R{%;`k!lt7;jQLY-yc@x7yw7;^HYuElmW8*xZ&TqP; z61R~Fe~m9})`{+T3J)oUoCfbl+|eq@d8CJv{!^L=e#{EFXw7c8 zaHF#EYU;J|s>H5|+cnnWO3IB)};DjMRC{Hgl(x{Ekq z3UI1)3F7LaA1Ff*U9-aF0#NTr4l-S2HXS4teHEA!WfWgf@a3?O_O+FklJEp-$C=^W z`o@$u_0h!SR&MK$pI)n9NVZv!X*-hssI8c%?J>cy_1pK2wLSqGOmv*Y5jzxqT>b0Z z6Mv!*Cc_j`CPU-57eBAAf5V!*cFi`d27xB3_=$TUkn!q7qFBb21F3Jmh>EIwkDjBV zUte|_YpL~`+-inO66Fcq@ z@U5iKwT&@v*snBkEa^qae$ugC5WCE6*85f}s7wc8cMUS_-J`y7c6L`rX$QP-I@q73 zK+MA_i*u`JfdQ8CE`X6YhHf z)-Uk(^~D8_i>KX@MB|AZ;SsBqD@rT&Vm8e-A}OhB=gBVj7nwS6S*^XfaE8Hl=;DHR z+9A?56IG37?UXdZv)RrSBg5jXfm$Q0dOj*;u=UT25rYnPi!!XfqI@53#YI0DZDDnG z`Wn{uDtlRcpJU?7B|AyhmA;a#Silsen&wK^T-Kut2Che+G>iz>>0wtt<_vh0+0;q%p#tEgH{j zN3ga)_+N`Hw$<5HYH3WvyCnCRN1K*cI-qpw-kOm0V6Vb}^_S|PX#mC&O(%ZA@#kg; z%#*q~E)0&cb9uMUoHiOI)?DgOwryP{@M@STrY1lMul9xKR4c6C^VXf5$y83e;y(A@ z(q+B(N9JN|^Soo_L#=r#a*_PcnyanB71>|Cx2_io!3Mn{yw1NoVaI*U%PD z?ZnV73guBw|h%Atn_iS83^Wo?#A&=nyF-pV?Wpb=o*W4S#T+e@a zl|nl4qcM5%``pa=k~iO>03QhAh^Dp=gui{6BWkFU$wRQ0#nwN%66T3 zvDFmQ^}uT;Jr^debix3Fmbb?q6j|!|zM_=@5`e8(UhKK`gda0QA97@h9mE@njjiJNKl+smUvW<60cGiI$pig?8cOvnyk5 zbniM#8kg2}nkE@O8iYNUd&XgDd=$u)n}`Tj|#0X z`9;GQ;wW={->m*zmun)~m4QJ~5LyuKh0iE|R;gD3TICvx=DJ5wzlmyfcK<$!7fN4` zzi7~cvmqdO_=xIukzUwvGK1Me0?pOs7b_hmD{VK|GguXhz8wP?fFr_2VWf(&GQz2F z-12kwLL|edwge%c+`92LO{e(}@+;W>xek+x2X6HU zdCpm(s!D#JwQ5#9MQqyq7hY!9?;~5Krhbpbt~rK9J?atboK$Ge(4B`o;)_cI9zO~m ztG_Eimn|<0huyJtHOV)MbapPjDVV9=8Qqa_H|Z%zDkR*@UY-K7CW<;Xfor)6bcK`|E_5Wp-7Fjn!&LUOn$?~>8!P1MN(tQ3CXP@FGm`Ivk}ky@`vP3QPmQ$;y=^)1#D;UmBZ;S=wCl&Bv21Z zR7Q>q{ehQM0ua{h))$kV9=T`F<``U&$Rd9L!H#Yko#bSDVq9 zXsb}zI}+X2Cv|Lm8YJxaqz+Cr4{)p84D%M!UG&j@#-~ckGddAIx-8$da^COCiV5c1 zpn^o*qt^}h4w$vv=)b7`t|McNYiO8tWo&`VoNr!mzC$g!>AIMYV4i5hfRE|1)47^U z7f6U-)Y<3vMTiIytQroIaEC)_2f0&#n0>U@fMVi*jCyd49p%nJ%b-}Xl z>mT0bR7tX49wlm1e7o!`oYbkgvgor)lt$hJVm@;&E8OKwB>@Mg;x~s%%r1l zMo84^%Oy8%!JDlg4`xpSTM#$mVo`oC2dnCCoslqBXj(G;YT=7ANAr_v(yU%56dzW^ z)wP^-4E^leb%nSaXZ`4Y^ze*wjkb>xad$e@ri?iB4SbL@C2V~sEP_8!r|xssTQ1d> z{iLDMC|9F=+f9Xz8_`hMDsrumNnOtRW~NAOo zRLN6bQ2wiPa#JK;3}?s3vj;n)>%b7r)1^HU%+z z4k%u&RE984%U4p&khdpxxh@R+Yc)cutnl`RcAdx1{Y(4@xYJ5T5QR%j2U;OL z)=R{LB5)2Xi26jIk=kKBUY3wJmglUD^jo%C>srX=x~SHW%i&c=gSOsfC~;Qwk9s#y zx(f;f@7hNuRDyccUoa~uC?y>QY6z-?66Xj}p9y&{UVODu;s2S}=NTxo`3@s4P_(H0 zA(Pl@N$Q&efyVWsIXp-OkClr}-ZSv&_y`4>7+m{uk`u)voZ1&Q;ARVE&0oU?+i6ox zEq0-!R&2>%(s3nUtL;hk;dCfg)S!Iq#X1>hU}?9w^L26Q8nISGQtYgv2*~#)6<5J83GHZtoQ|TvJST!C<|H7Eh;ya)2$1f zq(1Z0u7b18F81f_CE~09h-aX?VJiQT5d|y)_dZLw8INP;K!IeB`u%=va^kfg=9&gV zF1F54{J?Zwgg@iM;pTctCebhQmzE1{=E~Gt&S9FrEUZ+0%4t~8x+Ze+Ua|G9iy|W8 z3Z?1iS1XT`6H*=#jEi=5Ui)>?sTrBhA~3i(bi=fZDO+>o*dW|?ZL`{63Kd%D+uezn( z4`(L>RnPFx^q`URJW={qHe)AT`R}5*S8AD`gIKb}^xKMjxvYQoJN>Uu0fJ6(-0z$r z8@Wgv&B};LS8j&v`r`=RrmY_q{VhTQ?xK3SWsnT>}Ih6BA&R`{6GGFLkHsi z*_$dp0R{2& z4@V*iS>&f(;M`9Iz`g(n{b5=dq19ES^A2xp3DI^%7V0^5=(3VS_kR8ts$+`iUflCk z9^*OxQ5^XjDm>s}wv&m$bH)`Q{@eDpKL??WVaMk7b`#lnvGY=YJP(0RhfNb5J86PR z{wE8^a3aD%wSWmsS?htU&_!2!VQ)8TIN4APgWR&bw;U4CVo|FMXW>5U zvT6sluz-k3fk$rez+SjTPK{Rchf2ZECq4!$bhk)VQKF^(VAGc;yZ56D@ZY>l4rteG zxBw1_*$CGA$@|(Ug#x`yul7!PF8VUT&_x(wg(@J>?J2C5A7dw9C3OVdvc<|OF$|I2*mHZ z%peI1_jWQ1tWo%~DRR)EDIP{8^TdlCDlM*iPjA`-s_zw#;V&97-(8P?ILL`xJloCE zwKKn58l@j#fDqiEwyU&X)&jhv5-wW^$(#gn?a4hdhfaaX$#tx=M!>zTZbq+>eO5r$ z@6h2mEYghSq1kQH7VpjU1&%`jt7j5gG}zHFUu0F(19C};ZFdc%U78x$pG>=A_hYjl z-yfc|rz;*>!EK=F>f;+ef+E&Ethl$1u!EYg;CN*Cr^1vkSVAj}(nIaU1(JoDj)Nkg z{$Y?c05l}tgY_RiMq;eWJ7e_EmP|G+bf%+pt@Qn9R|ni;w%A_92M?FN;!H+526FOy zh6F{s9j}!^NAN!{m^=`9Olm)O{=PT`TxBBdZV zB`{)Rg9;LtOV^6XJtgbFWHdx+@j`n|-$&*nz=gV#ZHA}b`pJ(v^AG;nRb7hC18SA3 zDWrAhPaaXCw^|m#+Kv*mV}v!Ihf7yCYa9k`gzPlk3DXpAQY#i&+?Qb=iOGBwX^lW} zw$t&cX6s(e?d8>{H7n6VQ#+$p(1tLTtpY)Es5?rBfcFu&g6n5|3b>fK12Q!Fs5J4eX76E9DNc6boSj$$htXYo z8rGvky=5+Isqq(f_X41vQ*t`gdI$;JeOPXF1Vu<(3+B#A`yhpWwF)P*3q;0pY_c&Z z2v`)_XU3_pdNR?Co#ykP1yJE%67A&gcetie^&S!6dvX~7x`{)r6Z2}sEQ_IL;j(r6 zeE{W#VH4J2B`^8Fa|4hoY7wJiG6o?f%-YX(Yp#vAw7{P;@&=^hn@r~5QSXu-#j~h^ zs^Kz&_)vt6?=N{B+P9V}Ak8B{l-GoI^d3-=@p;-_(4jkcTP~g|HkyzM=!lecu#6vW z%3xv_6_Bp4R{Qw9I`IwziM&s1H&!rhWXjn;>98m=cy7>>!|0GM(E86=N(a|s(rHqZ zM*cCO519F&B6k21A|m$f(pVb;PNsox#^yPZ|N1^*kP@1VKcK)mZ;~f|jIhf}oyPIwNivSA*^ z)yv)_&5S|0O1qkzPLPSbcH>35J4D9$9rpJG+KF^ zU|?Mv^`W>mg_84@b8fC26-QDlwz-ebH;{CdmQP}i0Es>Wkbs4o;dd)431)v7XxEV$ zRYi4C){p1|b|@!kH$GRS_bVr_(#oxc*p>}S!(xzhyaW3!mZ~#>wg@9=ckR;W{&ov` zG>U~-ac3AKf)<3DhhfJGH@G-R8L4YPX%S7~mnz%_lo*PWFptu#U6|(S`^$7>oZfg% zu0|08#IP`ULdlPyoP0n20P4(lS(2Gb_%N#sMc71xaEz74qXxUz-_Or8`>B;We?bh? ze{4B6*V$VlMm_lX2k3uy!C&m^#+rh#1}G3-E{>nO?{idPc-lDq1>!hC(ypmds4p~} zKNtDn-YN^1ViyU6w2(A_bh$u>);ICy$G2I7w-TznE6uwWEjwBybS^k219>`LP#!=0 z`b|PMC3Gmik2~t&nmcGqLkz>)!*fXNHEe#&U$t%5b@u-TdZwvvFv}`XG&nvEz!bLg zPBdvBMU|7}DIBdO|6<*c;7iq%o94CJHI;;5AIYJHFr&lr7);jP zh*-AtTYqM9$z*^mu-M<}Y2&=(gM`n04AYbW5P=ajAOi$bDk_%6?>b6pP1Dx*!=d;p zdWE@D%dm;;LCx@wLnz0C*{ER>sSF%`mg#UDS1P){i3RvjGIuA()E~3&6N7U;c*q4ic?Y@}2<5Vy;*@QNUiZ zDQanH$#lf0?FcOHCzUsNYm&^00O2BNfaymU!a%3=a>_l~I}>KkV_-{R1dmVLy)an@ zF*$P~;+)yBZ&oUu)#r!%-g{>fTFG@s(uOzKoayIFcIA&_hyY-oGYsj-YIM%*gl*bG zGcul*;eH^jE3}_FMlj=!HyLJW-Buc4kz4frdi+ASCN54N(EWX>@L4IEg{)pDX6*E|3@bFqe z^x9H~)?M=lzMcs3SHqoWCo~+0pJ@gLghCd>N!{A@mRz%P;$!Cf=IOA`X{Pxf4TGO)f zqJ@7s(Y@dO&GC5bi!|K=nWmBevFOpRw6gQC4%*uJ$^G8em|G8aRQ(?0?*+j7LjmQi z4ZsA#3;srdLUnKGFyWCL`y~=GctO~Z<>c8;`^)|-B}&gFM?+@QOE(nOq@0+wO&WM!qJ_;7W{$y^Z! zt?pUZ{2Jm)1%MhHJBtUtw5aBC-Cge&rlS|~go!t~o)a78sPhxua0XT0;xXWG)drJ* zR$RC@+rp3fbD-T{3n@b8k&AS+9K1BRf68vrrx8eG2EiwdWtGK|}dIs&-zjIOd?p6E0o)!owKu=e8pB+jMin5Er}k5O-$PVq>88eWIcX5SwpjDd65mkKS$UN47RXIXaNGx zGt?mHu$wKANDf4;2S=Il4190lGek}5Y<%udwF^*$g4 zI^+52ZcXEU2h#3O89h;h6^-(h#4L)>wa>kqejIV?lsK~iDrwcF=7;uT;lb=MWaf1p zlPk>pD0Je%z8r&LS-H@&$H%n`ux@5H3a25~(xdkHFdv;wNd@L!Bb{~|{SBTN%0!qv zmMmz$VuN${(U;*?`+|Xj@3m|7h<3XVTt)IDgTnVKiXWcV%ZP}7rP-VvP#fCs>K)7vsmO4X#S`xyuF5FsZq)veoBK`z6tZ4dMl&W z!&)n7|H$dswQHLm1G5s0_k2qWtc%ppg^tI_MRMx)bTUHH80X)@qq*IVNy$3M;`HrsQrvkErcdVRGFXi5S z&tF8n6$^0DadUkM2*T-ybd01;YK>9pEje(~kasAdDTY2mmRvd%`Ho?6*)}bM(^`Nt z^l8oh8-iEtv!9mVr}1c;*V5-c{D&o}<-=b9Q1S?v20_sqDAj=5lm!8F0AjB63Ut$! z=>U%SAR79Dvm(Co2gXrzwLCXF)+@7*y>|dX&51Vt+BJ@*%Ux~rDjq8#O?ECiK&2kJ zd|C#_SL>%J}p{^A4kAGDr+6;J7oN<&WkdQi1P}{CcLe@i}d+qgyBID%p+D>y3wWWwAEXarl`1)f=Yr zXm#-dv29$tu@YU-s^b*n1~*I)<$>j|y9{ZLw=6c!r}ptw@oj*O38Eoe$NaR}cXQh1 zPlyIe_>-96J~me`XftI}@Su&aHoa)8<6NcdG7T(zcX7YQx`UWybtVAEWiWAtp_B82 ztKGr=J0TNACH%wNv#9kzh?Z^NVGzTD)Qm?Uvgwvw<|+G82L{_=<;F2#Im8kmK+;`& zf)_)@Wjtj6kFM{Ir~3cm&y6yY$|$3#?2(2oTotk+WUrF!D?8V`8kA6G_O-M3y2jNa zglk@#vhT$$Gvo4mUDD_C`Tl;te>`OMe!bpjKF>MNlhORJQfN~(4Mo*BW*QCrs}}@C z)mm4aS6^T66~$!jIC_|xTv?P-0)kph6zmR(gP9g%s~dxb)QHNbzh?QNMq8W`XSx*& z^9FH3UWGBq81rrEhVF4j#A5{V&^M$Vf9;DU*!jnKwL5Gwqma$lg@m^4I?O72%-HUZ z``?-NIY{2WAT!O|;AK!}E1LlO1N~0^o(ch@vQF*IM}9BDQ?WVS)mH+@R{%tj!JY{` zf4Ne#->U#vz%NHD5C8w@(PwGA&-vM{W25XU-WMc{=gD0cc)q{*h3w!W5B&m@=o&u-+`Wh2 zI^OVAmxnhO_xH!7M~kGe{x8e*30~yJ+Xh;Sc0iKTfYb?u79FZ4{oou$0Ickc$K2;G z@n;t=_(RjlS9(&MVq|->)#VAyiPTXNHzS63ZsNGl?6ee-%ns_(30Dhcterr6l01RT z*25`OF`tq_g#)!pZX!2T8!I@=GPCaO+C(Q59JuEUSaR@r30O}<3uoib&q0F=!UcXu z$?F0)T7fyppq?GP-9kG7jA?y5$KyOLC2?Xq_X$@bUh35lVaJ~MP#OaI>tUd4bE4^} z$Wy`pHLgEMpo(7m>8<^+;KX4v^%K?Nh9Vla#VaLdR# zH4yKh7G;|$fm3M?BGv!J*cCmG&a#~}>z98lI3HpgM?ckn}Zo1Ot5WO`w0hM7NoFv&HFGKp;<&i*({fty4a|u36!OD3QzG zKtjk@o}D*YhUvM{zZktZyv>`BC#p7&WeV?5SIY&1zy9@RLtXVP0)^+iX;&)AQG0i* zJ99Zu9=*xY2w4;@EVwm=!ip|dD2PADDN2+_06=7bf&qHcHBpJ^;=7geW}JbO=U0Y#;sJodE__a!`>u6yR0@Ziv|)OG2C-Q&lZ_%)%sOnq+S z?k36EPYakU+3?^fHBby&<#fGEAtL;rSL#U(n3cd40I~vr%`v~joqVrD=JR%1ra-SS z3I|1>40`3y9U|&9S)nBU7XQdu2GKCmCV6O^Q4#N^I(U?)kD3Ts~KjwzDWEMW9#z-D+nB;&+;HUx4k29@M$qM#s8kcWA> zUSz{Q(P=-i5QKtiLd)w92f=Gx9L!WzgXer&g*vvL37pDu8yvhxMuh2LB5TmTVK@{v zw??QPkE6%>C3J8&2&Y3U{IB?AUYh*%H%I1ILb`ROx6BF32o)a+{L}7g8tmujL>y--FDp=_9Ofhi&bCCPkYA=}# z!oBhGQ#x7eLUn%Cfn8hAjtzVFodku0MpR?%khlp!9<4Ou>}1m6N<2!@)L5Qe#)?pb zpmqfK%mpJTNCUciFZDHqjpZ-!!?8w!CU+RR0Jk*{s8*bcJY5&voqCyUDdHmXWuF>- zGkF;2jB$5F3;oX|{#5gmCav-iRO~IkY`sGHp|pgob>Yep)O|(znY%D?hnpPy5R|TK z+wY+?I~8oSoN!C4ZSLT_W7=kV{RSzRAiHLA8YR2Ug!*oXeJdxd(rTMLbZ&%AX*?)b zYuL>;EDXw4xV$)vBMQv=+#tB2cjQfQK84b&rNW>bfFa0JC|T##IP{375H^Ol)AIvO zcBw`#sz-sV_$cOAonDlk0avH!I}3DzCJR9fSddtCtUZ5M+ou%i>V<#WdfAT`HZ3*B-7ea>Y}Rf%^r&kZN^Wl2&96m=9b6bkssFBFSIMrS zwttWo&hR>r?(zWu7h60>HM&_mFW7iZVNr5rmix+x8cW1K>?S}f*;%$W>rDBuwL^ZZBgDM% zsPClID8yjwekw9VvZHKEXAke#DYx#}sbiDFy-ZN}QFTsKg=p{!c%6SWCflup?0XwEsHz}c?OMEoOCAQQ8Swz zkmg7Kc3&i|Z{cewJ{g1niT)J{Ws!ZBM1&Tl^~ZbS@9ZFk5t_|A-7gDh{k&WkW?7bIzBn29V7wBC+*kF) z2c4e27x)AZrw|zxT^y0cmHHRZ*P(_|>kI6R33@PTH~RORds@*&p@qh3%(7ih#Y-1p zJ8UW9urOAy%QRVJgfIouqQ`^{O>HcbjTO5_(h%L|GNuCSNYomzf) zJ_6Z6lcZE!;8*BtdU7_@--C#+$+O3zeX#;_bx7%^hky5y>}fvOLP)_&B01%}3t%PA zqW4k>Lf}T!grL zrh-zKHbq=r%R(-!sPmSz_)|Cfm=O!%H>eHH@kTA8EVi6}>$q4guI*mfJhi&X+Z%>`gvNAZ0xqz?X|*-k)#WU<((Kt3C-oEYXk2Oj z5V#Y0>)R`p%o@G#5Y8ks6^_Rg(rSpnYW>z`ouH#i*0Lio5sghR@E-qBDl zx(oDKd6-P77`j)Mjv7FdnX7iBYu;$fl9_C&fGb)*L^LuJAc5clSDcf>Oe70E$&q@A65b2cn zi_JYPw!qDV&S^1i$lYhHbA}J-U{$@z+h_TD)*h};vv(V{V&u`-2=~MqpZP_U_lhrX zuD#nFQHQ{#f{N866fa@7^26JeDx~@W4lklB@<_xtQp;lQ*^%z5uHRky+SNtrV=gKz zH7bV31yp^j{0sbiV3^h9HqDt^Gm6SH$}?)pGo7kZh2NDdXduZn)(O0GgL+*OUGKE^ zetomn^`}ON9(Vd+D{~?7?DPlk(%Zk0Tg16AD`V(O7QvqT#WejV%y-LltXKD zic}Um+}d`RZhfF>bjVg<>nD$#GDXJ2p_qjOp9ke5jeTW^k?L}HAbZSsd*2#i#HgP~ zo&TJ#M86!DhiyBzCdP|{zJYk{y- zvY=U-5px!Ii|m5K#QI>vnza6N zb;2zw;$f{q^Tdnq7r<4%3U3b_HQVNMOS)r!c{g{|@AiaH_K_Wz1);5`XVIej*Bj6n z9iQctlYRxqm0vTZ-(d#DqSaQy~Pz22>+Z58(32g}OZRE-Snfi)1zqFK$jM}iCgAlOh6k`SbP@5sMvxt1q?!YG~S z3Yy^~^C|IXhaK`Rp70N~eVWJ9gt!VjMuX29rFZ>AN1(ZXLvl*{ zJvrno_V-KLOea>?+auN)b|%WL@|NL|S$4+azAsjl@1QE8KDL0m69a|FBe(8s*_tEe z8d#4E;~9A(0hYAAvXjz5>HKX)HoUdiHEB3~oxQ87hmgxDRW)OTuYMhcyvh=$!V+xS z1=loKB4LkaeQ-L=>X?aRqMx72k9S!bX7$M%9p#n7iEp!i&4eh`+15#~77IhTPK=+} zKXW>{A&AS_@M^4Se00;_iyEhn z#7Ue=X{gYSW8Uy_t8aZ$Z8GYL+7T3=;Cm0}<`Nc8(}NL*S*X#mDiAtzZfz7Jp_)Bdm#hJq&?q|K83tGImN@F zMDV#)u$hNU-`=)O6@gS!S=-yZwyE8SSaQilXgz1ZtQIbh6OH6&6kjgiy={qGA{Qx$ z6q$|`SzO7vUZ^%_CVAnnYz#nx&3r@!8uDD)uh`kO8s4FZ_Sr5ry_z{N(PT6vI6uC(E8N;w^!*3BZj;dwn=GQ5Bu63;`#FUIVzll=gX6r{#6(!9WJu5 zz5s!Ah*+sM)92=cm(uHU<{Q*~+4LrVR5kKpe#bEt!#=P4 z@D`m^TAy|^sD*;^gp#s_vT`en#b9QnU)%g$L9)ct!gJR>XH>EET7vC*#gE^M3zLN~ za)q#laB(?aw=o!sT*`_3cwmVDCFe=~&FOdpyvb4{YsL77qovW>2+QuwIWD^v{2Z5o z9jG2r>LeUGOUhHIIRQR|;K=^h`Xs~yN=+*)x$8tAH$E!B7NeeyGSCnY#820O_jvvU zBU#8@!V&F|5v2Rv5r&c1sl^~F>RO&mjd%YzN`IXlT#!Yb9B;D;fnG(5&QuaaNICwE zZyO^TqqS@ke17$CgaGSeyKT#6cioC#&t_=y{OOH{xS9fok#u2~#KVcfYEuI*0H>`; zd*m=)CJ#nfulCKdlgwk=_`>+6Zqz8!_0r0@35^z)h|#CFEd;ZVEN=9n{M4)sJXKia zR7{vN=Cysv5~YD zuJ-*cvQ3aIb1`5Z)`i1dDWwI)-@IskV+JDww>9!GCqCg5j2o-YG#UcfqCINTe7yla zafH22nAg&-aClWVC4Ctg-xlR+>e^i$-X0}qnH#at%~;*y%~lD?dgz8vG=S8;Ki+IpDR=9tZOzKq5` zD%#BA=mj_y#Id<}lU5ipPe~Oz^Hm*A=Nw)<`tHo(b#FTv!?f_eNG<5hSTYMMEB&=T z);ZNImpM7?s54&^>t4$V)oh&bSf}LPJTLSLd2~hDr13&(R#FM0YV<=P^a^pkH2N+g zLp$qVbV!l+z)h7bx$MzTAfC`*_6TDU5|p!RqlXa`2whvQsLSCUfF;}Jwuj4mrWxqFmr5`W~Dn$z(LUgJ@GhZX^0igw=kZ}qVKq6Fr*X4jL>s=Hlhx}IpY^f{Ug zsi4}UMBGakKK(-j2doWmh`(&Ez(!c3l{=}pz?~g2B9`B)u20E z&$QX_#epFs)e1ph(vO{+sQ$C5d;M9hkoh40i{m@T!PaGWm7=}v11EUM)N!8TZy(uJ z?%h&${2>kd*(zmC@#Bw3LCE9MSq%|S+tTRx|@30CE-rL!?c zzt<`NnchLQlfLexmB{s>!OxV~x_2EdP7)cf<_2}U+!nd=v&5GE)ldeiwOq)K`OV7o z@aT)Z5-bt%hcZaePjW*Eo$SwiOEb{#Fp&2=x}NCXcal3Bt>A9d#+d4W5%i_XiJy&6 z*K>+fucW14NeB1wDsp(HxGmfu*!=|SU?!?j2KB{tnxqwkYis`gvDxDZvK zSlIj;5Dr6`$b4-6H8ey@3&9UfF=1xtxpryOH_DX7H@BWDGaN#u`yUeL)Aa8#R~CKT zQQ0TTw5@ERV4=u|b;|7MK~?Km=m6n%8#@ydtkZ2-uujF5c$i>3J8dSM{aQ{T=deiT zL&(mRaUm~@TNX;qta65HEs=Q$w)Z!o*wXO!r|mDpJqlKGT)OYRqaa^8f$6{7)vwjn z-<2EQm{2iw_w3*K3&62grQx*u!{zRSO(ms2m>$PDh-#)+^URY_4(5=h?u0BRg~o#8P4huR(U?3 z0ioc(CThgBQuD?_LHy3_a%p3#X}Rgw&FtJQh!hVT{WH8hLT+xjx*44~MHNl{)VRG^ zTv6mvBuZ}ceTqJ0w7M&|F{{1MGz#RfymJYg+Qoy4bs49BChP5%NFNuGt(|1QHY%Ip zDEX(y9GL)NV?{hB6SwB9}lekvSM^8vyI${e9V=h*RMjy6A8rZ4P78s$f-p!`Yh*Fs@Jv6=OzVPAijdq#Tv z->x`tS4)(Sbq+GpeMp3ioZrB_Ig-x=x0CvkzD+l8wv6zRCwV^d^z+wRl_?I5$Ku8B z2ee%$y2ktLY*Ya6t2#m+x=$Ajc6m?&`r4yR`%=lT+EC|t?MIkqVP3>d5*aq)D$T6N zGuWD5SIF)TNpChPORs+je>9%tm|x@5qYq3u|MJhNp9c!*NM`sq@&2r0K3wHwvoKZj zR^&JsTq8X#B|Qh)9|W9FQdG;!+u|%$N8tiKV~hg!qaos%GYah^vcoc=_L^G)vl6>Q>uJHS-a%8X4mLG& z+tQ0g27WuGKF&+cVrj*lkCNQueB6J1sTkkcQeWsWR^F~G(611jDw@sCvLAJ*92mAM zP+s+kJ)2Z>fCT{2>3{Vlcw$T*zHeld5xCnGuM|m@Sw`VQP#p?NwwJirl)bWS8HEV* z9Tpp9(V(^2wEZEo|Cu%e9E$bq{O)1#adBP8@4m9gJo^STUqQi64CcL_CxzV|kJ;~b z;=`9+V_RCGYcor&(hk}UU$>p3PcdeIUb-hfoOF?{(AMA{H3?&E@6lpYts+apUIyDWv>g($%P~J>w zNR*5*wPm+Q`?ScDT`S#L;feS8K#^VVSGK#eX6jStQ?)+IO>gI+s*(Lz%iW$u>oUh0VeeuskI)^5lEcAb7&M@c4cg|m$HkPv152`t|u$O-Yg){)0 zuf~`6LEx;HeSfBoFTPtN%e0PtnNWu{`i6J=vNYux)sgj+bY)7b;_STmT9`d(}?dzU136Iy~Gm~W6S9ZOI z#PRpWRPeuJQvaPjJ8;TaiebqpV9W~ibU({vAJlHb1xS4`)-yR;E&;HhbdhO5_XEv5imzxAN5y zr>4*)c^Y_l;L_>qOF^cnNL@jLt_Ei$GwrK|v*M?2r?vN=K&G-NDQO?^aH%R)Ci zMr}&lrCKwIKLgk1`iuL-yefi^2OPNctdR#$(&{#{~L=^r|?=u~%Q}neq z9@b*mCxkSQm{Vd;q?Y>SXtegLX>HEYFP?f+zw?WEvZS8z{bVAMuwz?Om$!Dwd%hTk zy}94bA0+@1_o*bWeeHddb_j7%^2Wxi;{UNv!2bVx9_#OQyzxdCZ34ezZ$FV7>>&Zzk9@aTV4@+qeu?#HHK>?6iy&V!TC`Yo0Tn z28hBs@u6=o61+=7s_>VL@B>y3Z9K_kz1(U=_x(14^PN9Xi2Ya_%6ETC3t7LH8KC_r z$?1i<9FEs6Ue?6GM1{KEU}wPU>=wWIWSY-fF~#~w4E5%_3Vea5oRH;*r*wl>ep^OL ziT7!!b+pXz-0CIeadGzgH3o#Y7eBqqS~42S*P_|pQ;GKyWb1S;t`s2TDR^H@2P8_< z0J73|5>P28M10~uh}ZZ0Dfm!)8hJoBPx=yrn)&wcUp8)}H6sp*B=2|#Q9||DoE*hT zGK?{60@yna$-cHU$xkk<{PgkY|B#oNY2jsCXCZ{9pb%f#T&v7q&y0VuST(tdy;(U` zcK-`CNkvMJTwWP`GNHrKI6e_t8h8Jb*1dIoJ0Vu@$+u!Z>bLR;YeQK}0lsCjyWX1a z=B8~e0!OC>T>3m?4=d`*P8?(P9k3KbrhI_sRcGE?oxKltR$hSP4aOEKN@dd&4elk& z=I~$UGoOr@2&>$v96MJfGp6Quxh+BUlCsxQ5XAKnn^#_WTSBaMd9ayxr>ujV#sD>) zciTf(`$?{l7R$zhueXWx`P@Df4BcwMNs8gSkmQYeeLLqVVI{cb$3OO%q_iR`I z`Sh5Z9$-Z=O?IeU(C@p^rXr95VJb^-#O+2Tr@PP9ePPPRBH4qNpS)7S?9iNpe`S(c z)xl(j*6w)C-~S|BC&C!i`F74o)4Z|8Z!$}I&&s6}>4%7@4@w%8Y)dke@Bc@TkUp;Axrk;r0WO@AfyhV>q@Yh{@7U$5t+8n`Ga0L%J_8U#mvyv4DHN!0XL+s zLYSsUyi0`~Y^eH|Jfk6sFU&kMRtK%qI<8&a{`MrP{z-t^i;;o7n&^q6h^wG=^X*ln z%k4dt`b||8K@BEogUxvU?kBpX=>I>XxFlz=M8RJkxJbJOMs4_56~L_EKxi9cLI8PX zKAEPcgq4zUY_yX^W(D{qf|I3MB4y@kauJn1e3Q14RMEn}~a@t5%;|%i>-fSpni=aW#eN1*upVfVBpknYDRYf0q z*WpvU+!)-UwwU(%EJtY$f45(L$Mv+BhV+KBgqNK16r_ahQ6-kSOx-iv|81)<-7&*X zcR$;BnW1S@k3_j#i|VGJle!~lzrB?cC(e=?20+r zeE5C*;RO;*0XnKS!8GIlNY=iqc#{Uz*47T?;(G!0CV$8Jx}!QZ1N>maFstLYAdF*H zVn8LfryqGSlya0Qe8>@NB3<#2M#5%mzBII?`jVi}x=Fm(YH!LH##@(xL2Jt?9Xlem zyEe2?ir$qeD;O;F6?KiTId5b}oB|mH<_K|nW+$&U<}`Y{q^)jeGht18NY{@2@Zq(A zrT~ac&gNQq{g9($uBm56z6jh^DKXd#o(Pm3Xj!#>;in|EXfoIP7wmO5lh znU&7jP;31n5QMP46Re{b3coNSNiZFl!Wxhh;w%|*r3MY|LP&Ids?us-Z*OdLu+Mdz z60#oxwzjxluV;&fP$4*}F+BgM;GfU_4)ya!`%C+g!iT(@ZaG+5eX6*zxiWf?8trN5 zn==S5$J+ZKUp#N&w+S+k*)=k+MH^~oN}+N4+0tHJvuVJI6ChFl?Y5cW+NjqYvt(Xd z-P+*m42Fl73DHLOz$iVBHUgo3+x91R`)OvYXVIMzm*G9V&tV?oPc-zLmY??N0p`Eb zxHzP`&Eqiyo3u3+2BWv$I~NAGMw=!Y@r#lx1+0nTPf2(T!995ZfMPOMx)# z5sGv{?`Ks5Sju?s>@ak4yy|F*3k5mOk6W5m^iQwwu{m*sc zqdH5rriv;<^vHj3j^7cqwr|QNG7jlV{Y@YKBnS7^E;*vepBH0igi zDiH6GvCysIu#@QVNHCK}>l<)_q_l8h;_2~{fzzhe%OHyItQ-NgvbtVU7F~xXu8Hjy z)LiYmR#w+>PsZz!;KZ1|jJdy`sH5yxb*RuTaS@+4MZ#Q!PQy7*)w0|wA+xE+cN08IzY`^bis(Z+%gJuB;a<4 zrHr)0_45;9hnMTVjjpbaJ{ZI=c`n;8u41#MEDi6z=?EZMyY<=a+3P-iql;=9r+?vC zHv{GL?}eQk<^MA*HYo(6*wKgc;NTPtNPhAMTqjnr@89_c_41_`|JyhLuB_01E#d{J z1_x4-p36@U^kWcn6-?a27Iv-_rJhS@ISGYANP_~%Jjg&&W@l+ZJKP6vBw6l+`Eb-| z;RWl9=w59hxMPJu)sH3FAu?$DiOB@hk&csuN#UYp^iyoPYBDa}x13xO1a=&d&e96k5A z^i_%M>}hn-Agvg$%81V1(`!lsrP67yz9g?9Ne*5nAoLM`{16~c%>H9F;8lL;3lea+dF+%g(Q`*R+d3eIDJr7v1W2ue@ArJ6tsn%N_5^ zZ(1`77u#NHZL_tNa=_R1>LxDLz6Kyoh4-Sdqv!;_%VYG4OwOj(RS?Ar9{LIYE3+v&yKUpkC4y07Lio@>B^Ca)L8e6$r>y_V6gOWnUs8o4;(#qwP zvTplcd_iekl&;EDwH&;lu43*bH>|$0{`Z&Dbzfc=s+#)AxYi^cyXrdM50m+F)5j2q zMBa>E{V8~YJ*({zy5mBr9KojXFcy>ymNvMQJY;3S-3VVZ5n8Q#TK^7$FPQtzvEj^b zvVaaPrlIUNdYASamR?sEKo0q1ccdv`9ilG%o-+gE7sEtyr^UvM+rUas$fF}S1SxQM zeDEmDH=ZH!-Qhh`$*F8U`Soq)Btg5HM}FJk{z0ANpI83pvwcde_cXI%k444AqRV0z z**BbLTvP_n{GA@*qQ}tgE)!4!6Te)SecLc8%&p`OdAN}w-Rg13UY1qP>R7f@v+xK` zt*mJvTs$yo@h-`9d?e;eb}zoqkznnZdR@Mtc70 zhK;8M9Fn~sIFvr}DURN`I?f|kJh+OjjOw`F-S=c1(GzjQ_Qf$}VTO`S=fi759lGdM zUHn!Z;d$hRnsD-20g|obCG~)CDa-7c0!GjheB94Ulqv;{oa(S3T^H}0L%o09&VjuE zq2S(z))RJQZEqvD=`-)7#K%E{@#+S+Pq$TP3Sp!BxUrhnBys6TZH9NCbed9Z&O<};2 z*r!3B$jXlUzt(u)Sv_C8i9_5I-1_;cVO!_X);s08id~Gh(vh?9_j~zui^A!Faf{I6 zof-lx$7trgd)=xU0xFRB_y+%Fn-HaQ#pihvA5z!aj3t6)p(MbE_Z`&jgKg@CeFMUO z3mS0@Hh#OB=O+I%S+bFInuE*w8w38sEe5rus&e6syUDk%t=RM8K>~%igo2CW7m4F; ze|lS??ue=7nW5Jcp{-B4(YABIbgYiLY6~$*Q0QUp(n12n%VsXci?v1jtlZtwn%=e3 zHRLO6Ai?`(`At8+i6QuzqJRf6JyZtrp~-CnXg;7?~An>TuBS1_R$LR_~t9O`L?out{jd* z-tjb5#5IeHT+ejMZ{ohT!!xYwXKtlI(GU$6KX81OSO~smmVsQLO z0mFO4oV~ghm`&Sc5kP=A!hZ97V1GEDc1eX7De#EqYdCL5yTOQw;9DVoSyX43>~+ar zTgkVLKi5^ThGrR7KhVbC0YB1RQSCz0l{r}GH?oo*Y99q6WO%&qe6iSWyRnge{Bp96O5>a)+C2kL?b2 zRgU5!&6Zw+X?EaIeIRO=7V>lQ%Tuxv!d36uIlwAnvyx%m?jya1Vq?Rj?4m8Bc5-y= zi{&vs&fCAlwm#B0bbfv**_&~rPHf=w)x+(NBi@6#Q(6{2rE!(mJMvs-qwDuY z;SG1ud>UYm{-k?kzBCM&$1@ex)*I9kl)2uW+8P5(8_S=>vX;Djrv$RGMeNraUl&)z zUi%~#!!36c6U0(9KjN|4wa_zDGgdCUy6TB71q~;)>%Rz?^{uh>LU+@GoRUFW@%e;Z zh4zm?xvRy8e|}*R-hT^`W=Pbw`qvDK$opJ8td(ZsjiOV$uZrzF?lX z00ksd|6?FO%R^nxoA|ZdzV9_#kycc$eJ}`jOADDVl=X+s1%U}(7l1)%1VndQW zWSg+k5Yrpo^UEET8y;h4QHi8#Fv&LGk0fQMT?0s=y+@6U;RJ;Lsp&UCXIXr{fQMVj0C zOO~@o_3MYD)stcQ9Wqv+;ES0a^Q{r`oCpbROJphm>jaX9ZDI-@_2iX>)1p_j@CJI= zUjC)QLX?>textYW(TEjE!TIB}$G>&oDSaL$8<&0Cbcj?FQ62T}W+LLv_LNscgs=b1 z*IX`!g4<+LAQdkLd&xQbvcekowC|R@`L}mh65~2C81uzR`K%jzEu~lJk))x-|J=ym zT{{oyF>FCbk3K%5g6S&vja0uwO2&taJbU`*=b_3Jw&)!{v{(u)@pf=vI zg1L#!LTDkW+xPd)c;C+_*XFXb2EScvFA7^58;h>rg8f|Vjdhf{_xR?|P?OetW~rVR zX6~`8gAXGL?!9R;trHvR5meQDUB0U)^B74hwSrm8__pik+@-wM+{`ls`b54OUpB|T z>eDsq>3K*h5o9i)a_cHr`_}Gm^}oCF=ryS>tX&66A_C?e^9GT7ivC^@@eN%q&>Aqh zub~(&63ww8=HYFO*#NGKte*-L^EF9P#eeVbZ&83lQxk#oKS^P0U677E+ERRV`hoMa zRDDy(6ibJvX!u8x^k&fL+#7fkafkZe`;o^t@pT*1DMPBoDTGkzN7ugr2luVYgp5jH zcTR)3)!Jex&tPqi;A_Qx_OZWEoJ1$+jL>m6*BW>pCi95;>hJMDxiTBC`L~9;nm5Le zFT|?utfSYG6L$%vt9^;rd?dd-;lJHhbiV>;8AEdYzs_d=4BmL1h*(&xCNBYC<;nPp zA(+@!0F5l&XB#^j*b}dvaI9HnHP$4CdfL-5H(#re)H52b9?w)277z##e?yu<;PCec zE45Fu9|_*S2-ka1vvYP@pV8XCxW$G&x&$H;>7o(uCXX>oF zuwby!;n)7e+fM=16&q!TT{P?lyj`~5*Db)cg4*A<2Pe&kx9p;KO9vJ#B(yJwq3=ZR z$6a3)KoDUZ$qPu5DG*mylEK?A(;?nb zS{Hos$qAgkB_gg>aEZJ;>zzYqLKu2ya}&PH-%3SqZDVsK$F z6hh9V5y#MM3c$r*DoM`|umF8h-5|T7_ZV|H9P%Yc&&)wopfI6DU_Go}?zg-({YpypB?GE)lnLTfmR!1;RGS#!&O_W{nKwqOkt$aK2 zErR;8fgiEK_qpZT8QEdoI$Tb|VWnDlv|8)~IOJvf+oVHj4eXArf4~6-LvWQhg0{X6X|C7~8o?B&HplyUuNHW4r<_{;cm6?7NU9EoQ`rKirM7*# zjf19dZF^=I>&6dHM3u_GjPx~L_%}v2qJ~(3pn~haz^Qr9^67E8VO)~mXy22S&K`Q+ z99z4K-3`75Nfo~aKnz|ICf7bZinC<})=F#^bMsn^z{kG&AqOj;iUIDjX;AKR=r`Nx zvZ)yy;wxofUYJP9aWuxMqCV!oeEG~(zB!kW4jlS!tIgUUsj&Sk%olB1(9fp$u*4# zK6&~xFvi_n_~0(Br-@1KyhIar5&dXAw#GhC}xP~(78^m2x|1E|-~z(ZEV z;iABSX@u55!^-c>DiR_ph!%cUAY@3hK{oR%X%1M}fu82cYo@48a1*%i4q!mkz`lsa1mD1 zcI4b+XhkZ;012U}*lZIwO1_&$t;HTidSg_l-~V0&s~~^fJamZs)kS3d^y*Zesga@K zE0+_t3LGCHEL*T;5d?6J{GtBi~rQIcbXi* zLz!CJWHr;%(`BEcyE*dD|78SCr_hz!`TZJkE;tF!KeM(z_>kU+2mslyd|3?JFPT6I zBt3v`rRSAsvUk@JcBf|sZm`N2e*GI7H}2Lje0}2**rM}zW4Jm+X%PUiDMQW!NS?&m zghKa;kUoH#I}dObUM**ZSrgZXyGgRI;~$#zI9!i{LGceg_7daQg-UpRxfTiFu|AE7 zDfKoOJpN&qkq=qS4kmi3o}*Aw-F6BeF{H8?QWMfcvNHemXZsMoCKFlPeO`0W@PAlj zJw+1i&56XrID!N8U-@iv1V3Pp*gODKm?1K{Ms2z^J8fSDy$q|70$SkI_j*p?XJ>!ha&g6NX2hz5R7K+pTq>urJIDBoMc~`x4rj2I) znVK!uOzyE!eJp4SF&9Yo%nY%X(>P3b{qop|bPIm_I7pU_rV9Pa%aeQub!!%-NCl@= zP!N@F;e3F&vv(c_6QKL4^!tS{gbJhP_-)eUb+0Jqp(@7gqjPI9m~W-TX#BOC2V(vg zqz|h154b22n4;%rCyGr0Q~#UGs&#B4m0GVjqB+8}))OZVP-t(8oTIcQ-$@=wR15^u z+cE}odg!IX>3U;r3xI{5XS{{XGR*UCWX{fdx-w*WS{sQ7P0I@8wW$M+f3N{Gt)yfn zMpK_MSkBJUlBYeqHaY~5C-=pswv>lG!DVf+^!=Yf$H<*_4Lzzn4vx|$T<(yHMdQ1q z(NrXxOtS()k_2F_lCu!5ujb{Jf^5lfS7xo~LlavBi?H3Dgm+Ttr(x3VI`ugk$^IAL z>~ANA!XOHCEPU%Riq%&%K}iGXJK_hG^(n5|LgP8%`xNRhhC(i`+}OkaHHSc44)76D z9K@*xbz!1D<5ZI&>DT5q+dHC*b8E~09mXuO`(w;BZwmDNJh-P`w=ZO1-M|{dZDdYG z-;teptBS@?bVEN?h1F$xEqd(=%SYp3g9{gDvze%wm|Z`g%p5-{6LN#>C>$OX-WLb} z#C+H7UwwX+BeHa-vU6~2P7{XmL*HT%>V+mABMsX(9^&SNd=x`eNcV9O?)){#Yw-%< zdcJSdSyHLFGzGntNGmmY0$(>3od|hN-FY0hANCoO7D?({5|SbHUWH$P%2A&u@%GEp zpN<=TpPPG4{WHT4iJmKfHBFEs7NeT8d!n_K%{N;TI4vljWRuv@JuV zq5~b7DgKlIR9m zZr*1y;-CV#C+BxTFlV@Q5-A8G0(B^OE@LWNf9?mA=cvNZ|7E}>{c#j$!+)7wL4UBY z{w=bLhVH?3cW%0{!h9wB7%!(*Gwc@k%I?Ci#F!-WUU$oQN6c6}LxnkLvHYrv1kF_T zHma)QvNAZ_f1l>_wqVyD1rwN_w07p|Hk<5v7RW6d`J?xm;lGG!)W~pu8%Y0n5@2tr z{WXz6;n7cVy~Z!5MFJ|Kq%>T(ycfYvNtOZP~7%;_z#Q;$eC2L?$G9!%?#QF z!Yux*=X=h;t7NjDlJ7<8I%WRcrwk1Z)#zE|PsD*F5GZbIIeLrwoL8c)nk@*jLiBn?-aD6fs=dmid%L0#PnwPHDKqb2J(?(o*BC;5ei z!cF%jx&NLATGQ}~OEqs5^ud63(m_&o#pPH(kQBeQdwh<%sINdkIog7Rxv0x6-Hvw~^%TSB zmCmODt6z;ZrXM|H)fygS=I$(Owotu@N*4aZe2aKS4SIAj^s1G6;`4(CIet>T#rosL zyw$j#&Pj4I&igkKmz)1|e;-(H@Gkz@kIioaT|TNpUG_7)?~^gZlN3oRE}_6E0loSi z|L7s#wq)x|8IcZJ(jb^a(JuZn_%r(!PBH5>Ma^seaPzLx0O-S%I{&R6K;VyeiHS=i zuNi}x20*SqJl_Lf&_l<;r~d3jS0?pFW?M!3l7z0A;kI|7$d$C_BkJC~mbfBr)!IX+ zDBfEC_^3JwKbl@ z3P!Bc{$g7HvMUb+pq~Jzi5iriT}4PpD3gjZqo~BRo?Ejvor;kneD+?1qRIgGt6KPz z|4J-1Y3Jj=LI=z}GM~!Abi!6tEn%lTPm;_*yF~{baFc=a2UorozZrP+z|9OjKh_@8 zw%FgxD&>L`T4#eNZfP`}4eqqN-(z^oYu`@rCPD37T>4t9*gO)1d}b2IodtCJw})Uf z!@47d$-uwS3@iQ-e`OM}I-k{5Y9f+^4Us0-Ft>uVWXU*9Z?0HYaEnBWbfz|Qit~*n zGnn6CaRfCXp8E}F%#U3@a^~=Bvc^Le9M;F)G*$iU3;tIQzE8n?c!w}uH&sOVx$1)s zUnlxi=usFxk-q@%MQK$KTgpv*A`qrT2hI3iUq93A=%&grmv#M+kPw)zGghG13QTjN zxSuxOc)pDB*^e?*1TcNk-M^xR_07BKq`yViuj*zkky30hwTX)?d@VaGkLR2aaJIf-XHGf$904$yqMmAGs6p+7pc2=GK@=p38`Ywr2hYo|YkRJVg&lj6{ zC1Bs@IrDsR)ULGcMrAQ+7kNRp$6N=d}{Fg)3Y9G86B7EB_`^L?)~Q{+{G?vDw2`7 zxj|DNc*KCaT7cy6H^mA?;kn-2bjw{akeUNm6n9AL(93Y{z({I82HRVQ5M4BnZ~cG6 z$Bz^%M09xlx}Mf!;P6CwkPa%}MQ^zT!y2&N3IUL743|^uqe`dbm*b~k+LAI7e_>Wl z7=p}e!4gEiZ#=FsK*MdaW`LxOa8y41MxYGY=_vf&i_YF z-c0OE3Es;q^m0t>0T$r5v;-7EtKVbCsTrG)5&#_Me1#D)l7D zxSIF>ucK&?c1h!C??E)E4jLz|(^8?KL`e~aN_CD|X%E^C($X;6T9-B{rxZ=M&{EOV z((nB_t8@C|-uwH<z+xu^oD?L8peAbT#4$Y08 zT)#yHYd>QG-609I)dWH;8&BRW+|FW#!B&>BPd_|k@Lcy9>kv!e%PV4*FT{3mv^*TO z=tvh!C`)bNF?|54mr)A8)YcFh*J|`A<)(x}X(oKVS6unN2$bS!5S1j+PS7X&kmVx}y~rU1`__1vZrR7kYFG=n(v!G-$RmB!r@2k}H?|ta_=28^ z%fyUBf-M%!TA({6<}+jZp6^ysGPry&Fm+uhzIO&(xB;3Rc7MZOJ>66%ScF-b3$Ph? z0)4dEE4{uKy#p=~qeFKP9afKpyzS+qlfe6rkKBYj=epr(XIN@(?i(c!_G#{lB`@e2 zCj2hJKL}4+XUbQUK>r2Ktao5lH#Vs|GYe&e+CC11>0BwiBW&z;A-wDT`=Jw0Y|qg! zlUsvRnasBcoR(f5vBb9uoT!hO4pq(0S*p`KY~kj{EGkK;ARu0UT{xj_J6gmbkPzLThMgt^1R$;^@eqnFUN&-}A_Akn zu!9H)&Pf-D7pr|vjq&N4X`cn`rX<}3MoE-j2Ybr~im*4511dK2@7eA8a^ZA&!Dh7| zAG1G80Qpa*s|TGum!PXNC#G~q33hSlDOyY|hC?OF?c9xZQF1$<3B&Z*u*A~MDqU7l zdrlN8Fp68`zGcqIPLuPjCY!mwcjlC^oUzV3^$fa%8cc-$N0B8X>*-I>NanhCI%2c4 zGyUna05dranhoMD&UaO=h(i%z%k}-TV%9Y!a5u~ zGA2K}vI2^nGW;(73=dhi`A;!esT_7guh?rG#7St zK0g0Ok1oV~X5=&HXK-EttQHf@2HqHeER+wHI+*x|fa9ZNFz)S-g0?-(HGj}I!zf~c z6NwZHiBbBi-&ktU*hZC$c(oykPM|>f7Djw4E*Mk-jb3@~bLPXMhF#y3^sujgxzzAt zK;F^9m#(>gF>$74&oLQq4pg2Cz=x+nfF5b!jZSX?@K^}y29`UG7RIxs~7 z2+LxTuDA8FMc=ZU`J#7YAawI2%TVsRMe5q^XJDdJyHBDAEQv4YaoCOp*~}N(U5BED z6TX^!3yf;q5IgFgB6bqlbc1FCc}0+sc2M2$gRkSY_duu9YQ#lYoR-@r&pF>k?D8&M zY^o|<6aSFK=su?hDe3EYi$W7NdQ4gAR*6>$k!==HhM(YH2vNatEBRpUhG)bZ8a^>P zo7N?7dOI-*?yyW(%lXJEYPilt2&wm~Q&(>>opJSaabWp6ehTx9*KF)ly6I%#saHCV zqN9}Mu}*Hcp0y0p2oI~H>;`)CrUjm?M&=|DIl;rB?wp#L$w0bjF)$K+WxtYS|Cz~R zkAc@reBTFMt)s<3EZF`&MM>|IAx|bVbng(Z0%|M+(;gJ7!jOy#*}OnqItt=6D;H-# zw&zz-F=#niG1~F8Vr8!JvnQH|1DAC3r}OQ~%pEAZWp8kmK9ME_4c9`U#kQlt>k@jH zdEeeS;4K(SIKBijQ5-CR!y}+h`fPPFf?@R9tOftiR1g>7u0V|F=!Z4v+qs#^l==y^!8Pgohkg-(SR zZ+E}bmGXmEX9Sh>9!y|BSP90;DWUz$Gum4+?4=^e#FbzRuk;p7yVVoKER`Di z!4-d$BJ`OXOE?)sq!i%diG}rd05p(k*>T{nb+27ZrLq_4lytdf6#LJ}6N0AI-_l=W z0P||uRbShMUUFx6yb~75(ZYT5;LqA&rd{|Li$$Ks=H)(&@B+W!?hkBv0u;A;;b2P4 zWIe#ECHQBOqlLZo>dzmsUlfIT6{1dVk)&ELeriRIp|;~aH$O-|tJ*5c!0U4Al+@X0 znP9B@a#;DpURO(^Za?U??F1PWxPMdy?{f@1^j&e<(emyZBtOj}f}naw7M`3C;Bngw zfScg(z-Sw7lZW%TX1dtqOpmvsUWH?^}93{Sxw1Wuy_?XFE}MiJR0TiV)Up4iRI%xo<`jws&8L>6|N1-=4udi6>&PB%+7(9HcIA z%;SmO`8lXeF#q*a^fN%?=onjoBS3~A933CH^lP&#>`ONNmchMu|2n-07D7%-7(v7e zol8v)I<^)=4n__W+&NSMBiq}g>tE!q6h;Ga)n>zzFTEou1g4DcMMd8?%hUrrMpNhY7Ne-b$dXoHrLlDoU=$!cjYDnDNJSsC-ugcc9G{uQM1R z2sg2oIXAub(+J^7Vx>yN3RZnKvUpvOeggOM#95@{kkjH7@eUuTZy5jeclMTQ!wMRapSC0CuW7&~T_%z2 zdbYSyP07b15}9G=#KL+`|01R&q}h6c>!-sN-$2edv`o8L& z`{6S)xmR$750U27Bu~W1()`|&Lnn?SXi~*+9U#Eao~YAa93y)ouF!^NC?5R4n=`i3 z+?$NnH;vu6ULzO%yq|YDS%UX3x_w*|@8ED+R0zvaF?V<{?Y?Pdq4CP^8)<43gyqmb zoUN{fe5E%Lf(NTv-WLjXSoe0jw^lVBW}VQh<_=s-YwNqBxm>lGNmQ$iSJoTVz9a8m zjb-H-$^TX3173$o&jLrl?Jz%j6%1f{&kZhDIwuP~No{T0yZy6=heK7B;zypdn^ZQ` z`fbYSQ~ZMe%(p&_rVi~$`U?DsDGSH^;IHsHwVYfrU$#8AWIH&pcfUxWQhHo?X2)2L zE_xjeVJgCh5F>=)0Bc~MF;|xNg$oz%S@=i^>fw;avOVRdOU|BiW?S&ww$v_Ixl-3=z!EfQ z!ev(ClcVg#=^Pf5@BH?D{^<`-OLqV$5KoVcYiJ`jup$H}ozL=-^X8SYAu=whcbWD0 zGW+jkHsiNv?*sd;?&$6=*VMDCCg_1mP@7uK({@QC6|aQ*DKhEt@wI`n4g&-({em+g z;WPm&i_0&Gd^~tzQ~nBb-@+chDev^Y$$_GuUoPGrPWEoufpNeTpG}`ByU%Gl-P-P}|DmD5^Pn`fz@QO7VL~FS0)*e^Bsa@;N@2ah73iAnnNprF#-W{*L;xJfye~!z2@&0f)}boYc1qlb zzy5*#`9~a~dtF4-cv_`$I9_Xgq|@diYpuB{BK>!RP|)R*JbXXFIM~O)%D)&@!v)H} zI8Jv~Mv8}Oy?p!j(pRU$WtTkUA6XL>Iqn8}_kS7}EIyM#=18H;j9O#8ikj-})X4hN zL78`$b!!a_%ClYhq3L;ys@f@51+nqh+@g}WcdSF=$Bw(ZyMMxjin@sSsP5UbZHv0$ zGJM+nUMzjv0dOq?LH0=m{ug7?(jS*%c@Q$q+ZiRQ?_*PtH&c6C31@smKSYVm%uc=8 z-HC>klHq_S^Gtyes~pA8MYM91KLm3#>GCiLFK>GM)|k2WMNZ%I+V@d9_k-f&<1Y=l zjBI?6&X|KY*sF`KmBsr1%5+nXOI$!zAe{p$)c>4F?XE`TwO#Fy?g>aJYNs2l4oA=O z-?(RR#QuAKP3!mz^|A0>E+RU-LBlEST$gahVofj$Nik~upq%E{+b0^3ST-IJJl9Su zhJjIidM2EOU)`3y)kIn;W;e|<%F+H=7=T-@zZrM;?u2xe|2esZtRLgCbty`0!lBdb zS|Ph}^0=NSmmr#9q!@;zNy0#6*`KtHunNQhIGc`;;C;o`I?Vw-Cy`Rc+0;m9kt>0p zA}{s6;&}N4$v3Rss7Bi;j1~Xqb(b)`e)A@m%R#fFEW6Nyj(cba)q>d0Aw9fSg{}3(JQ^y$d7YC`8_6^l{S^rj%q#BrErXd;{m^GtT?g5dI>H#=SZ4={$ z5AG3eVhKuY{9l!e>q{e5*v^T;-mwOzM#dfX<qhBZ) zes5a$>mM@+N0$})^y>+GZ1>*O_o4ZokVmSZ>?j*RV$0qL>&c{DvJJBxrUS$2h+&2P z%jhidmOF?|fvgO|er|1rlOpUo_lJXkEq-uSENs)QZElG`iIWgz_i8xu<{JvKL}6Fy zLEKC&6@OGeh^z|WO_M`Hitt(mS%ZNSOJ;h!45`I|dv)M}gXKnsLzljKZ5w5UqYlt) zC;Jm3Xa0q)lfWR(dqy6c;t@I~GASX(k86w45o+PVAo`KWxKaj*fA;jh@PG&)N7l#R z-q&_{|5jV*i;HN8R#&tUZ_sYb_zik|L5y7Twn+}+)#ur%DFH`G(IjtQBfx9p7QD+v z$O?zBY75yN<7g0~;aTry;CL<4c~4p1w4^_dGN6l`zC1>1mNw zU^nSVKI?pElc>G8yrdbz*qAJ)Af;8{<`O1Iy!oX~^(F541tl4otm#pjISLwvd3M zf16EGe7qiHb|SeIM@l#5oy0y%qZ*?7lKSPO|!6Aez1(E(` zbYa0Bm>N+p?EHnZ+EHekC12CE$V+^d+pe}9=}e8JKk*)#DGdn^(u3(a$K7CitoWY@ zN7N^42CW@PgGe4Wvro*`j-OboCp|W9TYsXOUUa-4KRVHuB(xEpR?p!4WkdDUiTP9|2lGfI zS5Vnay@C&C_B;ePQ*Gb~d~lG@rG4v0Es=<7j*UnIkE&{p<^g!ZCyr&sCgqFax|E{t z$p-3K)C+=)k#Ag@CkYDxjEE~m9IaHGF&bL^DS)fAl=oqNY)e1gy1!$%Zf2=TGIt_8 zQ&liLay^S#^{#!5774*iH;_C*)sYsA2X~vrr$(N=RUK&(rpD&QW$5v(dY@8*B(rU6 zKZm??>wRB7KK5H?kN(;($l@gvJ}2FJ`INGRPe4%;G-ab0R`a*=@Q%$nTGeh5u@8u2B%!^Z3$Gnt*p6MJvYJL{I>jI zr_q}O$KMU4zIz}OP!>}nsP4zyl$TO%kv^6-$Jk6sf@0{D|$cXtYj3aZZDJ);}*vp4U9Q@^Px{Jj`@d za6F}N&$+|mOm(P1$BaaMPXBMx=LJWn<9Fv;?Fo4PIEKL6`hAE^6pQbchLTj@ljHte zl_fpPZR-Iyq@-E+!{?rTnSQCcqB^s4SCMp=V|=smW?Yf`Ak!mM`*8Q!4|gp7Dlij# zb=gnPwXbnDdCnc=b%^v~uEX z#pZ==N8c4anw)rDK3nc3SiZb4YNly6TIl4H(&haut8yZ2X}!U9JYDR;D`=xx<{2R1 z23!v=`6c~*^1XyNLN2M2EPI45AkPnK!SoE8&V_?FB(rR$Qms#L)C@bqVmQFdm~Zy3 zgyk2jSDrhp?G72gADQWgjd}@MWs|5_>nT^5tV|<*(P!@ z&}Iy|7w&tiHJ6zgtwlrgCn8EpN^}kxAD@G$KQD=VJJ!`QyBjs}_jS*ED1^8M7rxqCa3o%wH(45nG`L)x3&8_zSgOg8#R?44v$T;)x26OLP+GS{nozXs8 zJJ_V0xI=Sxp;mRs=lkEC{%DZTr0p9`tlrp_(Z!?G?&8M1n4M*|wQu8wVY5Zo+RGbh zZv9pkR=*&IF5R*U^b}_U40M#Z8)1bD!|!C%YV-dWuSPD&636ufpi* zW%G__n@yWVY^Al`KFu+Ufhb>%0m(a@3c9~x%Zye~BD5KEUsEot_;Q5i$E?C8k(I|O zhmJTc+;cSbi@)vlp_6}l_-*jFCS21w)`ZCX7EqhL*Pe_0U$%>*uz8UoDomnN%*SUl{N!U!K}| zd*(0F<`E@!}pxvA7e<*@Vm&vU_Rrcd(f3U-c$%ylKD%{H@7 z5uF6)JA-GQH4b@E@F9iU0NcRw!Z=5rjKuxU1wUM-O`b1NQdYL?a>0pL6j+wuvL7jI z*716K%jb~ThE06y->GjHneCOg?K4*6Uw^(8tZ=bld0W#HW#uP+eZwaUk?Z0UTSn#6 z`w~WY_0t{ecdXh9KL)FKfTf6phVYnsQ7(b!I$CX2TJhl<8I_~>9Z`%8jv#RL`E%%5 zve6lV!*y1t!9VfaECFJZxe)?~{CpvvnP_!exm(u%i{}k7M=Dm9zqGyQQhEKl~&|po#XO+@tXzo1CBg>;kF<%?NF63X_dVq0NrDicXE>o4wCNA7cRb z*OIor<4LY#*z?eIt(}cTtwXGV$5{s}EVnC?Xb+FLisqqb$#@6`JUo5@oSe*qDgb7# z(%-Kh9xWr{;8oMK`34kgfcQ^Jofi!9LYxF4>=?IP$QavJ~*Lt@xpMe~0g8^uhcC;a*S7N31+ zL@um<4T%1GZ?w8ilnEGtv&W-Uou4M>ca&bsSBk%ZSjfS=fLOfJ;36A5f~aLYL%%00 zB;k?Y=l|Ys(*1JdizRExMbPqyHbYgyif1@8_A!CHLzITXY^-I7l^u6I%;66^lvO z4ay8NKR=(KBX@fB>jDlIB}_J0%kgqwI*LjyVglB_7B?k_T~<~Qh%T9?eT(a%6TrF4 zFuTYiK!Q12q~I>87B9&mE9Nq-)_$sWQrQFGp8&Sw6%)xReJG@u?j5lp;m7oz_zcfM z;C)b+cM~CcF;OhO%Cf3sYXGXaLWD0!S|h5J#{6BNz(th-oKGXt&;BVe4q~G@yI=*Z zfzR`*QRRv?Q=PPOEw*&o6hy%P{7*)UFdWKNaODsQ3pfX)Q?zeisV?yDkLJz|;~vh* zgba?Co!fGA4U&$%4=V+M2L(&1C_?z6-VjI{c$G58_l&Dm5wK)7Eg_HN^{3njI4_Cn z4>(ZVRO67sJqq&R^^z+Pg{y}lf~hqQ!AUz!QNY8G^lQ&0B%*K%Z14e>1yy)?gBt~< zy`<$pAAXOqVKSMJyzvWiE#xPDW3Y${PbSf-uNVPQX=blRt0Q4f`%YNrYRa-2Vv;8y ze4t#?E|v%-)2qocTF?kXa#&(+Po~%JwokGla0D=Xdcuzqt@fi41$5IX)O`Gna7uG!m>y%nc|0WH>!?D>!h*X`=0`41EX{3zgyGm{XoRH zo|~{xu7Kw*I)Pe9Rk&-@bvUH>D zXd=m(Hr{UkA&Z29AaOujbw%qS;to)UXqNw@+Z|wyP}(72S%A0hn7?RGQwBtkJp7|x zO9TNo5%rb62iQ+z8k5x$X~zAkrKQU|6rt9btRFvjF5i`WKT&YG zdF=S#>X+0e{u;P2k)J2KZAtxcW1ph<+c~MF9EDts3qM*LK*B~44$L_Qj@c7XkQr0Z zoM-C1rkrAqz}A9*kHp z^4fS|`P|X50{>cXBQ&CiuG-k%6Q+o(pZxsFOypMe007sAq%5f`9GkY=x9)>N6{(YZ zKrqM_mz+H2>L(54#~y|8NJfYG)OndyiO6Ngwbh762Rs3bOdTwHvS?dkpY`Rd9WbX;CgtMy}7o&?X254^_;A4=9Yaqgpw*Tg{xWmF>f^CNf zA=wvz<%n1J)7m8=jym(s(th1Pv&Bp!_ZIAMa_vl+fNqk+BMB_%?TImiS2QP{;N_+} z`w^gx*rW-R4^cwRO%QjVGlh&Q|E2eCPi~)~xW$A*MqjOFJ*TpH`G-Rb8y zc{IQD``j4Y5OWvPG3ab}ci=}^h}!x2pUP=@by>+3XaPvC5r%Y!1G>WdmX4&iPtkbz z!_%y|2ALhQB;y~Gtce9z%SD2i!i9mBZyhtZ$0Cu42$5qmOZp^tq;LA>I*<#f<*|!j z@253R`s$K09W>~1PNFfvY%oU_+AwAxZNfy3mv=~WOkDp|O}@(K<1Bsr2DjMOZvee% zRZ8KQBn`_(7X8a7x7I0rnDCo&IXbp^vZrG=K{-GNl9!5F#;M~(%F?wFDa2n zJ9z|-+C{TW8ir`72RMKwHR|adEfm9VR^mEBCw_2CwV{lu9GhwFAopv>k3NmT5LVFwN7W$RpiSN&07l@KT|9Qy0yAnq(JGcG#z`` zc=|S#k%p3UtSRe3-psafoHd?{poZLj95g@}OMBdxFrwXMys2g?&cXZqNV>)-=oYla zd~xj@DCd4qkRmd7c|0Yxs^od}`i*`e&5@@Jl67C><{a*PnWGDTz;6*M4q?AliV=N^ z9~ZU0&joe6ZCGY@dtQQB5%{h=ULG(o>tNDrCp#SixZT$C2vHXPx?oAJZ7HSGKqP3= zb~N}?WE|*yTuymDRKO1k*Q;>{kpD{hA9uh(mnfad-pc!gM9PM)U(QA`3u#cEayBSh zO|eFs*W-n_fMbR00Y;#CC^{OyUx)MT+|FWA)$`ZBcwhHt(-ZO*dE;2E8X6}ifZl-D zSVS7k)xFFkEt26DAfKctJtVr1gMt0CPdBm+xf=ONtnMtY%35 z!tsi$(?}KL!stwFcqDIv6kMUyF)EcUDM+?KI9IU98<%PkkK@}VKNFP$vqY;!Y#!Pq z4)RVF%_I020@KJ)-4KK18Qx``8ZW{oezsrhO$@AEhDsAD15iu?W3_5vy9u&XQLDfH zzV~Oj_E3BFtFT>W% zeb`QwdMYcMDz;TCu*^`c8U#uQZTc|1q=|^VQA;4Bz_8H|fjhX!YVX7i!p~Xt)BxHn zI5r%(16%lbAt!@yqzW$Qna2*Dr%2`P6p_Oz8Mx=_!=U<$g1@m=Vn^PVu{M4G^i$-1 zUOL(h?2kDqW&llJfM_DmZPYF!PC=B%l!0~w5u-R6j@aXo*RREp;<5j!!LVcmBk~%A zJ=k-ys+BVRPmy6FB9ZFt?C;TwhRTSf65uFvE6_{iv(@G}@D7sW?=AjDc@bc+HXFne zdB=ngpku5c{Or9?k;OrWlD1{4bD$&XWjw>&L_#yJQEKNSCaeASpMZqx7-bRC02DJu zMe=D&_y@=iD!xFf`ebuMW1h`JQ(05U3E`U}L9pNscHs_nSV#(N%;~!@!xhNt+Rqbw z-tGI4Ro2{hxDSkWoYU(3gV6Eh?|4*bf#tCjpkbOk5VwUz1ZX=2!&TMO_3M6h*R-$8 z5Q>yx6SvUmEe1M8&2{|tNC`?+HC_K+-ER0JvzH#ZB62?`%uj5^_|rPjCi=G*$q|?o zQG))4A09ym+>H6z^x++tEc$A?QALY~l?!@sZj1^IZmfV?+!*wnu<4!)^2T{M{~qZ; z?4lL&5P9S4IpTKotKguaTCBAW76}haY=7!L*mJmJ3eT%7No)s^BN#Q1d}O|~)sErkC{Ww4FOXA9~8c!a*UD`~czeM1c1z0mQVwuH%HP$6#z+!qkAD`5w#y zBa$a`qJn!>wJ-||fpOF_Q5`7ku|8)uO2tyrBh{v6FNV@x<9`|ga+!Jwj(3V0G`lQK z;r^a(6PJK<^Nsh0(ca(=?64{7uug(2zNr892Co$O)$v55Pm!V%0(GhaY5OlDlek^- zXI;-^ae^U$E8?L5wgcN1mXp<%gly+U%=)IPgw{?^g69) z?COK~S%LR?&B`SjRy`ks6p%dMH?t;Pi_t^aO@b^ns?MJS>bjzDvx1KYTXI z*E#iC7;h4IY!RD@EG3Z|FaHtAh$KPlip%q-JY;RlR5+6@t=*iMD0dKX2ki`C+)QA~ z6UHwcgX{qajXW%~jot#JJ5^;n!E#y963jelbg&IlBDx>`Cf4&HpQKt5TUXrb(B>dU zCT#l*&|shQPr2~VN<2h_>@aP6!sC>h@p>|Z_S)~^AbWcZlFvo+BxrX8JYR2l>HyLD zRjJUn=@IP^U|VXW;$K+=x4ucJrFnOxf~lu?w#q!K_`ZeA=F&BEsX9)ZPo7i38Uq<= z|6ZV6{YPIN_q`87GeC}LwAHFC#At)i%zE~kKX4}x&~fx37Fda6B1S&d4dG9A=Ta9P zoEa}scvt}AgH|3pS)h{zpngnOns*DBZ9Ae#A&LUZ8hD%79xpWwa4sC^6Lv4GXZ$kf zgOHf?bRY6c5!h3(1!|D6gwa#&CR>njNv(9QJFIb)^!OOuk9H(c$@H$D48XoYl!LOb z3FhQ~5FE{~h(p%xXCiISi-YBV+q4nkthK5!DtBS3nhV@U|AG|#tQ=xnLH@oqS1-;RB-R) z3@4s$!GFDh4ZO=9&?3ePuruglST12yBUtkI1x|70{x??1?L{F~pJ?;fG%K%y1SmfY zkBdlJJFnq9q6Z zd6FI+s+y(vds>VIXiZz`1*p4wbVu{3c#HqhHe*D!1xpK6W7D)S7#`o@8IA^Bs;C2) z`zAVSJq(X-?A^cG2#uF};>z)}*oVN9-#p>ysd5eJL3uUMc81@YomcdVysM=R zJS+&w5dqeeV z8*AbTiWsA%{Y5?M#aBADWY=L&BQalknp`2M??xNeAVqhA%mtu9W(afI9Ux+hnrG$` zz_`#?Z#k)0P{>bgxJ3lOv@hmQc8)Z^?F2OVeT8XA8#`2HAm4SEBBCPp`O?y71HAmW z#g_B5^dU(2jp5MelfZ236*MWx+KxJcF%lQ=-*xB+3x?gws*0DI(iD}j15DNPuj4wU zM?Brr{P;-)Xc`zhYYRPr#tNxGMz_-d{21{B5rSr65Ki_#F57z$^e!gT9?G_Q>*5i5 zo8fzHcjk^oRUYZ?#%=+Qq zV7Rvs>Dw;I+LY!yg&x#+=R#0qvH-&aFm%E(N>AaDocKAaJm~;UpB&x@jN+jqCx`?< zouqj}ioQup0WS4y5=@78YkF=e$+FeU%Eki`;B8_S0i+KxxA*t=k6lnp&bM-+ z!`F{del)COwNKaDH}fm-x{ClKd+1sr}ZI)%O70P>NK zv_y^_=VYcV$hTP_x+&-2xr%t}%@ZW-SanDgpj6r9nmFEw2}77On@%hd2FiGjSkwyy zCaE+!kbH&!4m!{&Tw{Bw@JO2O}zY0>Id&e*UFAdgG>|+379l=odF2Jk3SW@j_1NWgXhBb&Yqmfu4W;C zt|R8@rhk3tM~Mz3nz+k8%>%)l(&OIadOsok)4#)j#{P_Ih7~56%yD061h*bYRTb2 zQz_A-nA>TxEftkrgXuk*c$on&?*Lam6GRfQ4_9A&{GC4b3pn1#1{x7;EFFe=6VB7$ z1<8?3M1q_?+SP)8ENTwd&|J&97e4&|5IE7GMpMYGQLOsZa#X}hifPXj!V(P zop(AUf^x*nc9^C=Km0CEcW709YK5_fsKhA)u;*=GS94dJK<3E01wu&Epg`ju z@0jB~lJh2fh3RCZr;6Rc3;-QyV=ayB$k2YbrjiPZgfBmW&2K1rCW>c%Tn3t*gZ*gQhEvE3Z8<-y{LEN9Gjr6e3DWK zjb<-791J056l-TGwydIVzMvQ#`ehsFutJv4RSq$*WQ3!U(qi_hk-=jMJ(PefLX~#< z%IdfnR)*qx9f5F*0{}N(&5iJqQvr3<)Hpf8oLF17z-ULrz~Rj!MG-X=xddWArL5r> zPdGgpV`twKsw@{EK0}ssEDEe{dlM~_(tj?c5hNOPEXaV41K~RNUrG$XMt}>e-O{h!qfAT?OUKC7x3j1T7@)as$`nH6}UZdezsjr)B$P=y47d!$BD(5QH0h zyow`ElPkQ$?X7xqGNGY}fmujFS^18GXJp)MdApP^;|YSP0$mjD7(9(>pqJ?mPK~@C zn!OJV4xV?1I?)DZt+D_mV!w(FQ4%694j~d;t+aFJ&L_KfQRZ)$8|VP~kbS_G^PnvB zu;Z}rFRcx7n8&7aoKnbcHe1U6pRDVu>>#!Rjfy!YKeQgRiOwOemnT;_O1s~C?+Ko? z2t2tF^00P_-Y7f`NHRvTT`~s|q7e3Xu4q^ZbBezDY&n}w)|0&HsC57=%q9!tpGixy z$RpbUmjxEB@#lkoORQCTLL{fQuL#Jq@>|P@Ah21m1w;5os9T8d1q)SYN39r^S{T-t zFld)m&VWoIwI;Fytf+rJnu-{P@(z|)D>kZNiOI#ieq*ExC{dGDP+k0G?Z8CJR)$RC9le^{Vvpv z!M;M01d`%!WuzzdPY^|DP2kW70RVd!_k)5qXfxXTFV_J?zrEY}!KMdnYy>p~rWI<} zNH&)dZrKNJtfL<1kK`chM4KKqS5Mfhv(ACi*8viGak40rrXf-kuRS5!P!1$BlSSRQ z1CXA#EPy~as-8(wJ!W8vI6}QJl0yJF8ptJYbIBf#0lgR9mtywr=vlr;jsh@8>8iES zRU$x94NQ+vj?pdsaVymvri}!$AlEB(F zFH1?m*g0A#t3S~0|AnA{MomT}Z-Yl3Gkdff1!W<6tQ9IqiGH~-jDQfp9M(r!kZd7= z^oV%6=`~fBH6{`Nd%Po1fSSGsP!?3})*HAVQw2@rElE6qTEc>S(4Ja~D zlog>bAQ7c@rg#l=WOp{^bGa*c2*`nLvV6fatd{wGLywP(1?BjwbOd`ev%fT}SXM|R=EW?;&*|7G02 zYFi~DA)Ik+lg)MnT%P7Fg%!hZ=3kiYjHh4(sR?zzyQ%2H|DruaY5_%*#w3V^ww)4W zTGI*CW)3bJrXG?)ershPf6>_%`kY~glr?a%p!SA<)f}5XO&z62v$v3gBBCl%l94zE z!Hf~lupM!vm|2N#?<&a%9^em@0e4J+b1A)=J|ywX2uAz=_W8E`AY!WE@?eZ2t_%gY zC!X(*hzFZ_eIbL;=Lo|0w$a5&Tq$BR+BSeQprAj*1+e}{l?t)bf6yi@y*{hQl zK%~w;Mf&!Pzz3zFbM;_eEP>Y|k$gc`18=081{Kd_JxBle?Y39-a*e&qf`jyh(N|B9 zA;ubeppOtz-n5XV00r@kBy?C&SsV)OCJc{}0n9jFa%AI{#@m0O7-^7SnoEQ{XeJ!i zZcw8h09yw0Sqn;2?4-(^w!Qn(KxY0U6FfapW;Dmxd2RuPeDat&xw*>t{y#t!st{D+ zs5yjh9l>+{4@cLI%PZ{M8SNJWBf;$~RtN;aIAmmir38V(AUamH(Td$x+EGBiYi&8%zyNW8n)km7gCNI$Oan^UKHL!^2My4`8FXzV zOHQ!;zlm-k7msoTMBUXpzn>iEXC$I8{eu!=-ZEsDyw!Q|$iNVknNbJ}{Axjk5xg$t zh;=l{FZ|ovR#wlnIYKSMI!7%C@xecXS6QME6rIC7D!cYBPbUxvnx>WClLvbjfph(n zq#<-*0Qi$kYmWdG*25>&^U~27lte&5)#%t>KQ&EN8bU7#g(?4Exl7G=sA1e>&E&bo z9Xd%1T{$Y?I8srFZ=@md1OyWP7$33Fn1ITm?V!g#zt3CrN_=v1=tin@3}WI~(O)ee zln_C$`(U{v?|~WWMiz`eUAATnP`7(OP8*;Rk!Zo@;Nvo62LGqHG8)8{JFL&-ux5lJ zHnna$Fk1_5xG|C`V6ep!MD#yGl~|}5^I*kQ&u~O6osfx$*n=>0Z~2iufSf*@kMYHR zl5qrT{o9}iFJhm7pAsojmlD8}M}d*Xi{eqd#{Vy8sPnV}sC{7 z9%+w*xt=cjo|@Q3UhnF$criHkmsiTJa{V_e3jv`|{re#9Q;1*V(^LceI)g=(gMM1` zGk1rn?^^-~?ic@^J=qRwQ=pskFC#7?3i=)p*GPA(MD_%dFYh?kEM@D2lUuGH;Og+@Gspi9~5D0$Q~&? z9msncoXG`8ULmCP@)cIn&0ug%Oo~tx>531cbAwoeu%??{TP-B)$lxHj$QC*j1`6T{ z0HjpPJW)W2kYQrjXkk=`-G~xkuRwF;cl$ml&C(#0lYGlW7k_ApNJ4naGc+eeaW-Hy zML)2X?gM9vG_jvcPuAcoervCb_2T|9lNXSDp~tJE)+XU%w@01l*C%!D$y!UR&~yC( zZ0r2ktVq;eAx>f_q7aC7!^;!6u>-`{Ud_YGm?$Q2oocoRo63!`(Cxl^mWvNfvUoz> z&dIbgq_RB8NGx!ue5+dD>OGj;@qXm~ExCh&KdLt3Omz9*#mNtAmj0nQiEsk~+IGT{ zDW>W;UFSq0PDArUuX8A{w1nOQ5*-NeMN(nab#B|@!`#d&AJLxQYq;|YyNLiB^utz; zAVBb0WModw@0 zNg_2EUO^qBCmL!Xzh9Qx7GQ#zy3f6mbxBXYa8Q@u%Uo^g+soGNH~FG8e*Cp1O}Tv7 z(51_*$j~@op|v%|cRb!mWB!w84X0&l;+YEX!>ptelW|Y0Cj94~LxPjWZlto^=6usG zv+*XogvyoWiIqFzZYw{3&Q+c{;y_Hzc6YF8V^2bX@(+1TiyoWjSE@}G=?&*t%>HtQ2zgH`1YW8Ie7IwLh z9=@}mz*KAktZ(Uy^0q!2jc=GOsdHt6>MpH@%7)+h?`6N$x}tBrm}UpKp8x6o<)(`iZrLffx0GJ`4}mFLM=h<(@b_@rMuf@1*(7^(aFv+iry)PPim{wd);m zFLCO03|GgAYtA=#_IeLKe3VyFdPw660q`neSXvs?G(L0b<+eji(mKM6HS+#XIwzmX z6fe(qy~1%W!y`a50j0<&2w`?9_g>Q4m3dH!nGc2u0l<*4{b^Ubi#=Y~y)=O^%0VlTE$TLXACihc zDMLvL!ii;);)AO-0aV+`uXX^w1yg*$!&d~Lg=4pN+~p=m4C_D;y?5blxKY>XXH7#= zU!SDik}H+qKU+oPa!a-ldE`vFm)cC5S?gF)U++NsAz-`^{Iu0b(Ao{O(Dm3!f!W75 zU6`KzMZy9&ZyVnK7y#aH$COwy|0R*>6$b@f5LiAKRmU9W6sQ zRlI8Sz*i9%gh*J~0RIH?j?D#YwZ*T`S|!tyqKwprQ){QM%U|AJ4y_UybLZ$eJ?Iv0 zpI1kEqjJk88-wyyorMd9Tl79ESI!%%jbuE)&32u+n==3UWZK}~K~fR(^!9AR&EKoBtHO@qMQlZJIEP@U60b&OGEs zzO#s6+KdzOZ}v=-O#|_*P#uLDI%7G8cH`b z|2ioY1077qUwllUaF2L=KIHJnqRM5rV5a`AQNQU2g2}ei_qiVyqx_rI=L`hr>veGq z;zNNkL;GEY`j0m4g>qDS2}e67H&(x zkC!^-O4-Uq{_|0+^lQ>6@e7h4v0kObLQ>#1#5hGV2iqE!DlY;Vkkt>t+at3wyrq^jV*?L zaD@=Wp{`n_Al@)RCVC@q<@pU_iK!hGvjr~TMw`%2VMzK9ClcLw^#q=OeX8Gi@~$6l z-$wozjk%KMhpon%>NaQpqLXCP57y}$q8m*DZ(q#wdG`o+r(*h{`RodCi={8*jC@lg zkv}9yx;tI-Y|M|ZCWYX*m=+FckP}+B`4AL#D>%_KvGc^EuSGM5&c$Tx#SA=_99~Xsb=V-q{Lx*nZUf$EEkjZayqtfBA1lM$j<> z*WtZ18GX0=$qbJ-*6}ZM?o&x1;#2Ose7{S@6SR8j5%_%Zc z#MSjJ_w+q1RvCTo7;f0Zwxw2fCii0bqe%3Q@e4OHL8I()zV1}<=i5gZ5BPo(7PPiX zBl%1KT;ph4K# z@f+KdU33A8^#T??Zq$s^n8`FQkQ@K8cbq<8ajfW~%%idW7-Ji_j%@Kr)+4o9$AM7g z@#1a}o#{xQZQ9k53M|(1NWRI*JC%EL03SkWBY7l6G#tbBPS)#qi$sMKe*W8av9qIM z_hnJX?0aB810X0-EG#?wE|SkYAH2aqXzMG2XTUTv4qe&}i>wlep6dSa)f7YNOThIN zoY~-QHFOsf9>iMpA)THPbR(fYSou35lOy0*L%=#8cvCdoK2I1WZ67RA{7@km{%YTk^Rve;F$Gn2-E~$R_`_Z$MzK_{CE_j-_Ga z0IAbDpjk5%pRa(6Ox!8=K=g+ZATR;x&@uU2kyRYDP-7m{*3rz&1_o4`h zH0d>osHk)S0qIJYDn&ZPLazeSOQd%~4UmxJIS0p?`TgJPdcTbWn%w6;XYaMwT6-VM zRDi<@z-V4SHXReB3pzrtg3reev?~z}O(KHaCjol;{Hab}V(jGauyO6c9<4v$0*1^# z-yGsR*uOga^Z}?>l27gmoPOep!D(`zcYp0~Z@!NwwX@QZS#6OF?ZE4+JtftjpR94` zmL`3=yT?b7rsk1&J+qfhy@kon7`>aJ9q>O^9fk;Kc4w_!)oZEDasb4bE-&y-oTs8K z)9e}ptGW$yKlgyI>FAP?gXlH(vK7M4n-`nLH=ve(bVMrc^m6h+j3pe*r#;a#A1wY9t)h!(~YNtVH!Ui~Q?Q8`d zBNwgu7!#1(U3S0vFcmLmXLY?J1SMn(c8L`TbBdPnF4~(%>kQ$Mx?ZcG3oxtl0C>Qz zvD$#H!4Z7%Zcbnekq}@)@2t$e)lgAPPeggLeY`{w#I(b`KR#d|_fd+@r#246^KLQ{HcLFl0hdsD?=Y z!|*x+8*b}0_YTN;UHNq|ugz9(Ud35Gwc;ppzFqjgWRxe}r~lG$-6yW#`jZ}S&0g{; zr}?)L1U!4#p&=A;(O2>}@PEna{%ar0yRip9lf>XTPhxF8eE05!rD`XrIGJ-avS&~5cPE*iOOYwPLKl2sFz_^bj?6b#x`q+U{rOaZlD-u(1kp$f3Y zQy9$I6M8)e-Fc{P{v8gf6+fmZRmagp@_PEWDZo>7Uip)tLA`%;56sQ~Mt3j(B1}$T zIRCk?IrqOo8@$wv%r9VTs`YKam|*ww){Q_@$Bk|s`rNaVzP`fTuSuur*HK_|sDLc$ z8k3l)DoxpL?*e3m7?q!=u*clTbh-oQ=`-l*xBwQq_p1U+vb(RBN5vmgKjMXp50=FD z0rX0`67SR|I{TU}_)*}A)vw1<*II!s0|XiFs+Ayno@_^eR8&D7kWAe?uX!SoKYk3N zM6A`y?I^$>knQh#9rx<{PW_*%>Ik=~*Y0y%tNe0~(O$y)(&K*~mmyikUJJNJS{wEM zH;28C+bg-AIe3L#(#E*7$*p=V{#(J7tt<>7%ROV5*6#wA&{J}bUK&?{tDXh8(!;Y! zBO%Pf(z46igg8({T&h4UrBKG`G1qJ3Hy}#`acXy)mHHkQaPEASZ9jKw??${UbX%aZ z&@t4#SAZX|PVcRp5}pOf>@ER3`I|Pe`D|KTD4%CXfO<*CWila;QvSZ~t;oQmx{ocwy(+!G5>8=1f(?~0(;E`um!_Dl;1G5z( z`rM#mUVeIz){ZbOO)2|i^Wzx{DCc1nc6De6e48hpW2X~!brTG}wj!``06^sl^FAFh zeSWla-gvXz`Nxfesi##X_>H_*YK^{20&+-$J=obuaa3k6jG8d7iQ6$A7FYEe=mHyB z;i-(9kFkk3e@p|Jh)Zmf+#FhmNO#4!1Z?DVg$uJI->OnoJv+mf;;p>!GxE~qwu+LL z6aWZm4;a4;)wL@feff=i)+*(V9O$wWK5^o|ClEf_-xLG5pHY#`y#F-eUo#eL><0Uu z-v{DBU)x%y1{#`!SFS^ZcuaiZhfDLY>)=)JRyOC$Rk~@u3opIJfJaCLv=TzLC6ry= zxn$FBh1un&+AJvS7K)n`nv-z5yqoBBTzEC8BGX`itwoU9mJAd%hx-Vx~`v^^*qdx-a9Nl zvJy;n3kCR?XjoVmt+k1#D+rXeBbN$)iNtG{wj=#Zyc%vXcUpxHTqS~aG+|g{1En7T z2WOlattMc(tXqCEv6kM*SUQfOb}`-023o=wJBjyl_{U@etkq`haurRwEI$vt#ivRBf%zp zbpAVlXuCwE*FwJhfETXn_pt;YBYcX%6^q=g<3|A)?E2AGNLI9Z39^HAUWyza7!=t@ z=qZyX4L+ss>!582(~4pED1`gZti6 zxDqUCHoD#0!hhn6JBfm&5Yb(G4xWEA=7IqbI?bg=kE3czI8PCuJZu7e3MAG+r@#KJqs!HSMk&V{o_-yrPhBP6y+N9wvR4 z-nnJQ9tufKtHM?hC6=9E2Q?~9yr%>c727@9oH{ee0Z0Q)ost)gTE>X@d%E<_hP^ zMqe_DE&0?-E;%qo9WS^hM>haYB4c7Z9JlbvALl(S{=1ZGZ04=7`MQzwpHv3c@XU!H z|E1}vsg?|Mli=c>IsN0GnOQ(pq?>UawZSN|@t5Bo`bd0d7Oco0R#(SpA2-IjU`M|s zvbc#wQ`R>Dao5_PFHz^)r(^74P5_%WI#`;#MN99`;xQ4*$$N?v0w^IE%A`r4G%>u! zpM-5Qa_*3|inT3Pe8qxt2gBz|QGxmbxhbKg;hyV6km+WLW6nzq`3h@UM(O>D2kAx*e&y6)szZpommiClpy*W=iSyZbddE| zJFrx_))wj}ZETGO36hwJkjv=zUTf`=CXY*tgtb5-qz=ClgY0iMj;!9uu@|0twVjR} z%n%Htd+;y-U*J0xOCrGM_0?X?s^rb903;bZEQf>497~2>l~g^NGzgA!=9BYznCd5= zi6bT6()RV%Q;>}SIk2}Aq>=49%b(Mrmaenjf(UzMg!b^USt-iU5Q9MH% zB((bDm6Y=gZ;h8pB}ZMu&FOeIW9-H*3c!(@Z|K{a@1*sL>|H zvqZ}Kernq8srnP7e%A$3`^$rb9LM}P(t1J62@)qs<2>4x9HuFN8Ir0ZE!wNX&Q>UB z_&*FOshqqDL8e{cmlJGOHAds(-`0&QR`NrTJd0p8t8zzrJ;6!$f?)F;e>N;T>k*Zs zM`uCcviGN{Jub2Qr3i~6roO%pWfF%D0%TceHP}6lcsbaycfbTNo0*k6mxZm$u7n96 zGTT+z1Dj$Fd(cCQ$UYQ$<`&E2zzaXz4$_SPR@+%=@9S~2&C!teV+!8>y$b*lgzrLG z$cR}bBJql^&O0jj_v)SJ3;l71=$e&Ne23BI(+$ErhCE;A;0!9|Bg4;utF>37vZu!c zO`^pyuIL9$%0|?K%d@Ce14lX*-MxnAjiXrAk~0!eQT4=|KlR>T_-T!_N6s5`MPYU( zjdgJ4#_=X@M)wc!oOi_@>$5S5l~p%qcdoTmsk8v9Nd>K<5CjX1iE1ln8S_RJz6I1Y zDLmPV#capcS?WnA7J^B0;0D4);te%qvhm1WX6wAo&=Bv3#H8vewtU`^*fGicyrbgT zKybl`$_l8yVh#A`eao@kE7Kpd_-my?Z&EDx`@f(C{>J;~UtWaCi51!We^e}i-7+)2 zWL#z0;e)H8HzU2hv18+<+4U|}RnW+;yi3kjysy8)#9vTzA>KcH26Q$vKkUjKtYNAF zeELGsV)0;F)#gYAE}(V%=1A!qWQZhwWoRG&JcNxkDRHA+Hl=k~P=` zY7TM2!x@bN$1NSiA|52p3#%03H8k#@$?(Xd6s9HxhdmUTGtPZi!f3$&_x=wl-T`Ab z$4b|{(-W$GQSm9Lx_>(@c19VgSIGjVl3OR&A`R@KA(TKNjLb=b?GlCj zj;c=gEA4ACq;T%_;xPf*UON*U(*0{!B;XA!r@m@urL6!e~Whc8g@^piN zDY$ut)9&X9Ol<91W%3y&kk86#pdD#g=icx1cIY9`g`&9qOR*Z)9{hG8Xwko}K_F4d`B5&?6__lBym2ScT=y*7v4RG(ii6cXP6U=1N#aM!6i zvA*=0B=6-#mso{6TML%bvAk95owS1}P-iLRd9+%0(izxNYE5p|8dJ}(NT52w)Tg~c9V{$zS{sdI11Q19at68>lcA4&FNJn}a=XQp77VzFB4nwnKJ#Hq(s_G&6K@B9_O~i78|6G2|fz;maPU)Rjrz4IRzb_QYSm|-nsdB530 z!J7`~Mi;`w{Ri{_zv=H^&Kpt_tZ;w{Pb#Pi-5Lsh(T1O#f?7V!o*NJ|e3AS;$*z@Z z#k&tv80+HkkJazOHLX2Cz_)Ad(_s>E&syc{ze4YC@5GyuFDZP~Mu4d9Iw^lX^ufdD z>5_|9Z1p5q4VNaHb=1z2L>TDTnZ zL7u4$H>M!@_N*Jys>&vy5#cS=ykkn17vRxE^-pXK@a1LB$BteW`@NuN0oqRhwbC`^c%f@KJ9PI^UY ziP6A2d) zsnX|mLqV6Oyhi%`VPO0b@YH2DaDL(1``p;Q?9L_0HmGN*SjIWXc{VB;caGDMasZPO zl?nqprySdiee;qvxHuDNx@P`r^5uCR_OnpDO^_4yV~+_}5#xlU$fliyuO*g3-s}Rk z37*OT>b!9(k{dtQVTlV%B_)3KRzS1#D(r;M38ESU)^jP<@)eow{p0^dkdB4rIttpg$75@*0k(J?#9CeO^c zqM-saebZ*XqFFaX$3#~m*?D~XCPiYqQ56u`s@HEn)2tMsD>s#x+W|`W$>Wl$#%_`t zM~fbmY{tRodN1~>|H>)Ou|oFX(fe~&So|{@UWGjGN$#!-er%zS`i)JT0G~=h5?U;! z!i`t_by5`UyrBd#0`gHh!$*!Vs_{Z@IWa%%(wL;wskR3{ZYSG$6i9hKc z&%6*4@Fhz{?gPz#no{1%m;xnef7K7F-}U(gvppaFxO(c~$e`>>E{=O9KG!?WYaFV1 z1LoMn`yye){kv#;c^V85Q82th{6=5>$>HnlC$|gsA!5AM!2u3zUz4k{ACV8My5Agd z9}nsRdMmc{f)H2; zAVEcML(o&n@DLMQO^uXitF>{L!U_M-z5aNeK;u(+A5pgq0D`4P_}vb*xNoS5noW%o zu_U4b14@YfGoV&gdBx<~_bm2B7bstc8T@`-wf9i+KY5Y$2A7=X7NHlD!0MM`aEZju z>OSleeqOmBAUtE;juiQGqWC^D4R&=}*Nj^?*tvESEhtrP4cL|pcM?Z7tJguHE>LT! zL!4V76QBx$TzR%Od`fw&a0L)jj|kvbN=zkXxo;OJ1WKK1WK%N;UQYCk|G#Yv zUtUm+u{In=LKp5#9z2RSIM$TO`qR5^farD$qaHm#P5d$D)z z3}Q$IbFQIlT#P|aF7t-rB5O|DjTU`9z{u10xI~;Nz2p#L@PufUb7F}Ch^vA8?bm2f z_08*w%=!ZO%@AK0)vKIMpZsuVt6o~gf!`@`^|af7^wzKE8tgYJmt69|36WDfLNgqS z>2l>?y_-InzW`8>-z!@B_IiHoyR*xQuTy&i$U5L699Ys*8TeOF>M{;Pt*jh}6sN48 zRb*dbR@3bN4zOubQg9j=3&NSz#2o<1Z$bHjb*RQp_6X;pgqi^k!J;#XMlIu0gL?*i{*G=maR~=eS z0I=jd)%z_uN<>byQ^C8=%~~*Q4%UV1@9HJe1|J6%^z1LKdsaaKHB*A)rr`1aqS!|# zm*2|N?JYUtkn9(z_u^0MK!eKrwdk&eZ?q?TM)&{E+y$Qkn!AoEp{@adDCGcq*cI~l zR(x|FWO~e&tbYKOe&B+f&K6KVBF0n}80|{$>qF`Q0pcY}UANeKd-YoZ{u!iHYpJRi zo5fq}eJh|ROC#`+18}{$>6`O3kMXVz^QHy8@Z&?yDp^&5Aoq#HAUxT?7P3%k%+4VJ zMCwId?U%mvhEHyAolCtexAB_&y&Wm;ww__e4se56eGT|#wjQT|+Ul#9ihS@k@Rj7k zec$1E*ItUdk!wX4I{nT#!%UGxWUZHSpzaC!+IsH}+T<&hn^ibvhlMjarx%^Q;*0I* z*#wK)+&q~r94->UQJhh!&ElDle0pjlyTI{lV6V}hme=(3mneay(Yf8ojVu$}z}#+^ z)C%fxmy>KWHHQ$8ZB$pgX<2#PSxLnIMP^0j{4B+tX`^WDwT}P(chD(A(=)J0N>ALJ z0HhjU?PgHYf$_Dg#*`{{B)|-;MgS2oWzAj6FBpGG_lJu)q2n=YFI$S!4v z`(7RXYT9oNe^e_Qu2(qEcCHshRb^LYJdpx-cy~O6z7EJYQui`yeFYA@&d-?&S3H24 zG3*k5CGg}mRu7IalLysEeIP{+(nmZ78WqL+KH3$C#()r{I}f;8PPy*co++RvN3!P6 zo^5xRE9nLacYHA+`k3m2l;_6+Xy{4!63kVe3XqEF&abeYlG`O7fwmwtb|rfu_t{#z zG$j{ZK|yzG>7@F~3H!ZBWAI5eVAZqcMz0~z;Sd$|$Oi3?`hHwmqE_K#7Ax{|{cpPRH#H6;E#$(_tAlv@=Bdd45(bKg9+_SA@jb-vD5X)vLa5v!ki)R#vj#g)6BRvfFpO*_A>NM`LC1)45-2r8_?JjkBM#r$j zw%ex}6pUrRs5YPr_6Z)jXGPdoq@0gL|%x@)>V6>6F9#7M53aI z?u;;_AJ_0mfWctM?cAPiRDygW;8}>YY>%O%Ntr)M)tlZOxUaAZNpq>W4BJ_Qo`Cd9 zqz>>>KIhlod5trBCij_!s-$phdwV*-eB-%g-4qO{;5#kBNvUhNzfxUBIEU6N0B$eR zq#qg{UT|w^%)etVf3t?a8Sik|QE26`_y>0ZNd4y2KHxmy{q&=@)82cWkuiB+jlxNX zJ$8Y23xJ)5mfo4Nh$Y)Xd!okBMba)Eq$b=31Oew!U-~qK8F!FYyVq)yPwLuiLLgL2 z6~m9UB>?HD|55Uwm_sP(a;#0KZA5nyG>dgde&(Cmyh(WC)$TPCt|GozlpxRgsMw?K ztYu10@c>utBOaRJh^jK>yb!SS7M^7>12Z<~(pwh#?`=RWV^F>rDf{gQ&_!p90Oy?w z&AA^91Y`^i&mX=)BO;P|2I7Nnk+8f?1G`+xcQX9S>)uGE7LC)fGF^AdUuGjfkVDPP1xYw|-sXX3-^<#3^>Qn)P&--3A zy_i8E7PA+xXK*m_I%A1=vc|_=BaJ6JArL1~Gj4-oP*+#P{~0jYRITXgYU-Sqrr_-< zkq~y!<#alDR9+URE8+>fT^-r~EnQaelq_RS#&2G^Lxr{flH5$6J%-e|%VeJ78msss zFSQyfH?$J1qp9lT5jeB~KKPSp#lSc;9}E;sJ+mVgm#G9^cA5y47qO&a87DSDJ$x$G zK7n$&kc*zFl4B{XCBJn zu(MLnR|Lj~iB;sRgWxD`n_o*etgV;B!+S4CJUCDLNAU&T+Z6CrKZ>14`;Q`quE~B= z#3TpbZcUGKyfXmI0~+iHYp5#nZQgO>=b5PNh#wSsLrWo1UE1|5y0v5MCn71=q zBKWyNoK;)Y>~LVffLH$`Fe0%kyGM5#FsCl# z`HAQ;cmyRo>Qm$w>r1_E)Qc z+nx3>cb)?bS76BDF8;jksAfPEY%GK)x1@_A|wcWBC^;B`gOFfucI)fCT zU?)FX`!*SR`=kqf?ho~4;Jd3b&)rN1%9Mq__@{cRQioprNXbrfjKc>CAWjDvNS97I zk_(C`=wuSPWDXqwQQ6T`mXxbUi)Bkf%~_*ItpJ8GJrnXItMpM>MtP~O=+X{LUSX1p z*wqj<v zQGm;6x?1iKXYR+%Q+X95Idb_^`vAZ6?QxxGX9EIo6|{%n$Og%)n*|!HX^vQV1jeiY z&8t+S<8d=@*+dw;iljTTgm1_(e4;9nqex&yV-)~vc zl0y(WC8}fOh%a3{L^OF z2$yeI`W3b#Ay1!uP%C|Uhx>xUsR=__g#gw=0@ONhFCMyh>_l6{jr}+3}r0oGesHg+|u1GbElnK?vIlH!q)2TDMQa)Co@!=f?8Lb9tF9ulYqXJ{)ziC$}9fEwhp)K`MrM4oP@mS5yVG zTMbDd+AXzC36NfsydD<6C0WOqcz!#ba8Gi@Pt~S+{D9ZplWnC-OO8<$frCW_8_|a^ zA)baXE#M64zX9{6nS8C-d4m2bu|bC$blyd2~=LUbE`0L z`AS6}I4ouN9Xf3vC-kb~S$cKQq@twIO6ii~VnPMq`qrCQ5-BnGP1MkIbIRr>)O`F_ z*h^?LRfazL1Od}`<4|FY@U?7SkV8MQ?-~9FLZ5hN^sA|vFA-_!1^=H+!0vc>pKYqC zinRx1GSfeDJ@{GIUIH*Ze>vo1ZM+t9+LHXJ;8BY7PLQ2*G8e`ewHAx5#LRsePdR&m z+>GAlJBY{Sxr16ja#ML-R7CLoghNlHNm!yt*H<27syjPk%GjayOgCVW*0~A|S{x10 ziP-IU>_>G8NAmWxPPid~xXB{eb(^yN%d8IFTQQm$&JnM{@ayDNqJ#66*cy$^5g zMp_?_{d8sbZ4$_fgHn6`4zOwa?L3+df)}|2p9Bu;ntks@E%s`IW8hSef71UkKxN!+ zD^Byg!z=Nj`!{5SRxKOs<#H?hUF?+yrjwM;lkMt5>^~j90@R+uP>>Q%Qo5(@7P_ z>$u9bOxg!S;6o%NjSUiaHYsH6;JJa>QZnGYcGlNMIRbWw6i5-x^|zz&Pjob*u&a?J zcZV&R+gl`-K~IqVZ6>0lvsl(trFKwZ%* zWSq>uXx~^m{<7DSE@{-|dGmN(w5Fk;ZCYpukaeHmTJTEfSB35wEW*<#WEJD2~8iOla|{s60pP+vJ`HLiDZx1ZTrPiEmy9R7UR zd$<=l+?UTWTqK2me-<$WTmtde3+q6YAytr$*$Tiglu2;|MPGI5ep)1>9(3nv!~cD~ zi~L=@YV@f6JA%^E_xigAZOU!Z%loA7P13i}b22yP^-0g%IG*YFeYrFr8k;fo1eF*w+e+NMN}3a4AGl%!c~~h1uqe~F{|A2_SZk-p$x4;D%_@^M`=tL z41TQ~L@ysCB~?csAOnrb05BWJ=g*9_I)C}0g27v3M2ajZ52U_t=JjLN0ZBGU)ZzK` zw*MxI)Qd*?uw^09Auq+Z($R$UQ7vMmYJxtgoCw-mWl z3=ZdRy78iMt7Y4e&AszM&thyhLl{vAy(|lzPJiRlFLl(09&=u??MB`X8`0o+hu7^*S%DLS<1E=TpTr~ZUO&k0G zWE3&}K64T)@mEl6^4=|b$|5F6-zfFdgNc_K+&~H_NhD0KQ>J?dHBN*+can@{&1JDk(YGB1h@&QEY*S-$ydKyBD0uLT? z@U#*tg5(h5Wf;L|>BP1Q3SYI!sebyZ8QE?~{%|2in8lnQVI6Eu-nn+IKx4t@^lP({ zt3WmONZWGk!ly)YDfs}50Mt5D5imqkyswq_?zLhuL@kP=R4hq>_&w_arqeUW@z zf0{!jP=DCKt!myVZ}a*r3#X?i;N`u32}{qudp>AkFpfmTYJ__!bDfLU)Yaj5sfg=u zN!ai??roa3w6=-cbKG4<_~W%X;K7y`wz`}^Ydv8wzwgT=GIY*=8&kQP!v|XV;ER2xU~sxqGAaGobUmj1R4UO6t7{!RbVk$+S^!zKZN9tAupU z^Ro_H@(z1X0Z7C52D6VeJhG4t+eQfP4ewb35hC)c3~=?_AtrekpX zZXrDhw0q2~wCy@SB21;cV~4SNlu<6&Ez4l|@dP2e<>)W;DhE0&oox{ILU8$>ggI1i zGVMpGxY6fTb3QM8XN9( zYbT@s3drV+8*bvu?-)O>S#KQ_p)Gwd(fV_rM>X(G&64(#z{!?Al5Mou_5IcH5T<1_ z7NV~3+|=~feXGq|Whcc7A0NqOL6HTe*95yY5?0@ib`vAf8#-tg?CSEjlxQIWaq7SZ z3we}fiLY(vT~dmgc>RUE4Yffs2Eh6G*3N5>2Y;1YN{pZFhH@5{>~-Mpnf#R|m*|BR zZW7h74ELAo{0iZXgwTM11@*Cdm-31V17$9S=3?4$WWAd=*aT^LdD}60A9+Db(cDZ& z^7zZl)e(0raRlM&CF0$_>8;9KR}7*%OFTn;-#%5_Eg%kRLPt${LJ)-l5@F&T=vTlu zFs(8(myP~xvZ~F7cwBW+m_XrtsaIFVA{q1fPh@u7@A&))<{o=3E9qva+S#-x?8K0P zo*0+CP%8BCrK*0SmH(^sSlhMchU@3%|B#jOpoh6bfuok{|MclcvR_;5EnEwjZe|dp zVh$%T=?JX?LSxbFc(ZcBaqnOIykzM-5|{+LZu(zb4H%_NVRvHZ0G;f(0qsuxLdB$? zsw7Pp0Q)xwQW&2)C3{e-)ww!9{tn~!oJdKTvJ2or@D|fE^)8RQS}5&#y=d$v(qyB} zt-7!&`()WRD)PA?je^}x=Aq53>ba}~^v5O8z?R}>YUIbA>yl%wtURxk=c(n=_Awxi zGs7K5cyqcaMFb_5smhnf~I#>Tx9G=r{t~gpjC4Y!UevmE=GBY#@5;CL^R4Je%>(s z&WDcIMxI%Y`-q9;*+hIo2%#RbD{5JrIbMJply*paj}$-7V`FQ3lxGnQU_Hv2;_s6R zC}n$7tDUY1JnSX|N=l(;t>CuJz~Er)mS|{uL}ip2g&w^^uSKov*aEW*UJCtEn!XZO zN940^PKo9cm@2~T>$ePb1vu1Y;r{j!j~~yOe|njqRsu{G7NdIaNC_H~SEF>~gU`>l zI(H0S8)boneHJ5VUexmEy}Y1Z1wg>NqB2j=q9EnqWjrpu^<4V8&Zq+QBD6th71F*Y z)u`^&^sUsN37>e1Ay#0TzxRoJRNy{XwfNK0iCOq}Xn22qPc)DG1z#3UT2YTVOhnol ze;$r!b{k@|7^8{^KiC6(P47ajPdFZIICiGN-{HhZY;Ukp2naMArHub`#YimI1>7a- zi351W;pvw%ii7n1ktXWxLH*;wRY3VMn2h=q08pr)^IGh0b{7U>)vn}jA$qoq_;OK!b7rsY=pok@l7xOC$Y9?nPx zW0L_#e|C%3*{zlFZam)A1)SxQ^5I3hUT$6mwL0jqHU1#$AL?kV(Gs4N2Kz?3Lv|WM zG@*^hvvAKu)FN#3#Wc&en+dG)wez-nM?M82WH|uZF6d=knG#-!D!3}%(@cht@3FDsr1!Nk0F(D zGSdYjN4jF6Z0#jtkToc)qz#k_3k+^?8!!NV64ipG;uk-ck%!TosTg9-_1>W5}t}CNB|>gycv#EF?C?s!6CK5iZVis zQvWN=a5FC+$(go&ej7q-E(8(s+>3V+_-U6F;QesN%8a-XuSZtDNvcwO?hTx8iA5fx zwwyB!*f2e@hWUq)Bmyzz+N__d_XBkP=5<@S5FOAPT}7%RjY9;SWrYj%u9ln@ZU3nL zDJ)uk$+e_n!#`~mN5|QAr)}HdEo+3s_>o*y2O|s9bB2f=?*UBgT>>#0t=how9eIq` zvN*kES)jBH0O80>?@e~!c!dl|i-rC|!pRM3gq`e3c?8_y){iDmS#{dnr6 z!q>}!pgz_vi$*=^C^={RhYJ|~ zwKjB{47Q}@lWvfZKnstmGz%&$c1Q4*tRY>r3p)um4Rk|;71y+Q^SD0s`2!`Az-SWV z8UDoXFBF(_A?mK1Gfg1^;*tw7113-47KbH5V@Z@=3c=(vA=~MR@g_9g#`B(&6cKzm zQ5QCX`OPm&E+jqZ}w)SN%M7X7U7|%ndrJwJy@T=?yDxN`}{e1g@v;6>w#JO zaV9ZM^Oqm%#40nBS%~>8SptlTBk4LhbCTX5bfR$%L(=Nkw;=U4G{xm>R@ibxp!m5c zBIO-hB3o~H&*#g`9)+>PHe+~cLb7yBsbpb*^1wykYn*8$*eY&SQdXR5ilDOC9z44V zwLG(cJJWNew&DSQ&r-M|_scoI0q{OL#;$sK!l6S)MRqJk7Z(@(ux!Hjv%xJIhq7;( zMm`HwrZ9dAUMnc#clk2~uJn#fM}yW0W)MrXO}FH-)etR0;z=XGpM1#_njOIg`eEw> zJzw4M-K!|GVxEv2o~;{E16{&2CK-;;=Wsi1xWEB#&*5J!0(xgLfONQ~%27-^{V**+ z7R3ZM>~><-&^2PVrDWEfj_)Pj%_W*}>Pc3;#1#K7q#ohxv7~$Lw};1LoQmG?+Glr* z#)}K{WCUs)kxAJBfv4IN{T^?31F!n)U_3})mirksF z6y0%vkk&IkOz9(LH$1_Rn7a$}#U)j?5u3eCOHW?;5rwN(K$0d_^H$;T4m`-wA0KX= zu5ZA8@e}{q=bkP;{IMzff^d1C+ars8+tG>J(Q`jOJQtnBY5VWsnXVNrvrxv-=k=X7 z9wZEJx49*V?bF3)4MLV=4l*=%6Dv3unla*mjDoeMgovI)>ol?|2`<{*KeJh% zH%?mI$6S-XG6oyL2QJ=2__xRjE3uFdv?M)b+1(X@(Ja|d8{|Ci?H4IyxqmgQnv=GF zU|^24YJ>7i75rL)xEK15^l0<$%W~eJsuKE%NSGQi?ZQO_2@Ok$h`n!y4iTa3-IO;q zZLusN1z3ozB|tUmz&3!0*bb*;3u<-L3OzbMX1aF>MxxTDqjAY&48~E6f-I|VULDCx z@Dlai2Tc%Jf^FB$7V^tJh-?QAUK9*Ud1~fPgaqBOymMtc(*Pn>oxQb6vyk6*G=qZ2 zk_Sf?cwnWQx%0q-Sx`yI;&Nka2IrSj;At4HxFL+F4qn*GjDRJSR(mc=VMVU4!n1Y8 z5?YlrD?q1-mc_RQz5>WzpDQ!#N&j_mWpZPs6}!`_W%o{*zpa!ttpz1mPMnfdWy zZW^^To#{5WSH{?w)l~mhu2i`i5}qMrJN@(Yk%emfRLJC`uK+ z9V=w#)q?>*HExzfhJ8&<*zFz3UHkkTg@mC14@D8}EfnYsm_sseQsOxZk*UA<5`zvN zyZ-q>0{WoX=L*Un`(GsMo13AK2Z~@sMX6m0d#}OcxaH5yg`knY#rDa0&%P77A7#j> zhjN)#&!_Ca79nl^r34ZHQeTP}JW0=X55>w!tGJ>-E#K6x{f8z5`#FYQVb9I3s;M*z zZPdh6eN|P}9BdKE+Hv%tzO%Xo$dUI1K+kGV>Xgtk5rHc>@A7cA)DRAmE?{g`1@@43&@kjRuYOw~v()rc;@4?JMStr7daw5IH4Na9P*YpRK-R`ndji6u z`a@GtWtpF-EJYe`Cf&o$&_sZ8ch%M^VdiM34|*bMRD(fq_T{`*n#0!eXx7U?$}%V+ zOUpsm6h1G`ffJESY0BHVV=od<>w8z62NwjG9F&&Jc1y@Q zw(&;~dz2&NFYB0WPc19uH7EX3&u%Lo5iQsznl851sdOfwg{H(FO#?#WqdXb^W0oZ1 zhbNz^rqv$)36f%a{pIFnjxx{xJ0D{=M?TX1C5I?i`!30gsn`kg^k;P(pO;8x5_8UI z%8Qz=>!9ViD=vcginlaq2ug{rG9BP2EsEly_0#^c?NnTrm#$fuQ){EC7c#3ZK{wQ3qu7dGKpNXwP5_D_{k#QC*%(MhY< zm73pq97TiS9kXYACS9bO9i9^>-qlcGVC!C5vI7$0FT7sjH0j-0gStgZw+X3xTqNE& zF)7~rSb{PCczDZ;i1ybz@k$Oh)RvYKKka_M$}ed&MOts9QAOk|<2IF{JP-?Umn=HM{SOs*<0XW5%B`_#&7o&Xg zj@=ll?>n=$B(c9e7mjPvgk4Bk(4NxGN9yF$Djcd5=Smp*K?L*CFmA~nc2`>S8Z0Ym z83SXqE+B`1xQ`qDD~^#|SUCq78mqCR6`Zb$UvIb7W|}pZ>216luT#2lllri_%rYbz zw5YGHiZ!)%$%im6wibl-DpT`!st<)e@BuwO%v#iutNP8>ytS&ZUprF=1p2C8BSP~$ z8x_O$X9Dc8syfU2(tb|GkPpx|S%7Ysob8M3{RNs%ej0yJv2bo&V2eKUi$BEU?k-tY z*Gf=D&S_}bxmN=P(IEAWnawLRk%w|mTP}z9zgVsI7#tkYu3F3#X$~4NE6=GCo4$an zs!T-VHc@%yfafyFYImQyOl+B21K#NBrffwH)r19Prap>0p!CJt6pVr8UUu9NW>3qT=I_;s#yIRe#WZg4aDY8o51`5px zcX2b11Sh%at^{BMFh2~U&U&lYtgiuI+iB@Sj-cqIT<@uebb;Q-ChUJ61h%!up?jmZ zE>)jTLdmF-TfHg!{jXQp5VEMhBqdb ztGCZrf{y4{I($e`U)yaPV}i_q-SRAzdL~XrXFs8qwV8+3Pu7mQt%cQJHVInh-{1Aw zQC3_T&Gvn@J!l#kDquoW07iv>xqP@G;xC^S5Rd8x`~EZc9)PmjO7E}x44L-hQykgM z?Hbx!Q?YnkY>#efS%)MdKkd*Z!8a*i#z7}wmZRA$Fah@WU7wu+Ry&6XlY(~lSI3mA zJESoM39&>cK^#|rp!(fquuf8=DmJ1lkXbXBOvCz#ixlx&K(oB3FT+un{zr&tzcG*~ zkP*FVNB979Y@DxVM^L#aHvr5gQ(U!s%T9_?DFI9)O#dA&V)rHrw~04)<5W=Kze9a( zm&Aj;3|_V1T6Tq#H&K4lzVaS?59`A9PB$LKm6XoU>Fl(AZ@IR8v47>|0y{&8zU7rd9xrTb z5|~;G8nH?|fsEF{{qv5h$5G3v*IKA!mP4rK7B)EfAwthxeJA4PuIQXuz$@szzL`br zqj)!9Gh&AqEIyYvTPhM1_b7rLAo^}Bl2&!Ha9#&Bsgp??kYdVP%jQzgcuWLW=G{{b zHgTSSwIcPZx=l=b`;INs2ztnodI#D*wLFvq2h$rj@?UpL2*wDDR#$_%i`x_`BmSov zl&iMSYz_5Qv^)qUXS2S_D@|%r{%S$TIcLs(7C`WY(_2He^qUr$10{~%z8&Q)eZN_9 z{~NCe*fge0d+T<`upOYv?JzYd|97_hkM~b=At!JF!HR(djU6x^lu)xFW~VOi41Iys zTZ!8(l%1Td)*Gc`m#Ox1wl%bVZkCXSbC!%IEkp5C%@kaiCaHOc!lkHw z(N*isvisusqHCk4l&UPr9&Ryd3P>e8ns2T^bn3+0zSidl8 zI|^M!fIX2FRzxa3_Zv9+jY1THJhnka4l)Jo~nDfj%7g%hAd}gc(Ch7okW}jk$^w`Oenc*LvfbY#m7_{_y zxAtmAr(Tl1fZZ^{YC-iZiAS>iki z*^C{Av)`)FEO(^0-Ex-#8veF~KZ=#1$c=jP(H9e?r1DvrWKFjLtJ-h{fZ4xMqg_D`{w69+C8#t zv5skzilzlmbABHdIMd;fv*xcr!M2PxiM19k#10w&Q)?GM(v?A|pS3gQYH_JCC=|90 zXYQgqe=7AB@E&)or=G%uZld{O3u_pwFC>7wVWr9?s5CSqhsZp}B=%D+V#601WNnyC zfNZ_7LmM31H3yi=MUtb3zjtownEwdRRX`Y3WI|J0cL11#H3!MH4cMqs4yv0Gal2?l zmND=b;DW-(K2*}F8@?@7X9(*PNLC(evL0UvL3_JIRScyz1r^VC39X*m?_Bl(oTxL; z8SRp(>|6_DH^`*vdh|{`cokm2n`JnOGoauM$>g*XB2=FtOE*B=j5j9_56^^O^!QDP zyr081#Kw96O$q<(_5H!Yl-LEG{G3hDfoYa?GK40lIcU21u?I_bG7>))z@1b5wV>Td zBd^4EEllV?I~~_G_=fSlml?0F#2dzt z!M+E5_~NApFDk2JY^vzrqEF0x`oe>ITADq$7mGqeoKN z(%91~)ifnHY5dRUDbw(jNXxP{Z*;@C`oBMX_dnh#fBC~e|M`K;_Fe%V?e}isIKWma z54DKXsMlE~wF*Qc;op0o@P#L_m-H-h6geMMNtvNNpJqA{SlB*-`TF+-ML#N^G9NCw z3+U};mDj&7{`tK)S8mDB{r9t>8xMRI$bvHaSDTm18WHWkMZh0&K8&E<$7SkLlqR*@ zfV*qW?WZ#IVAd_5$#ZG0oSP+_rIr5kb-!@!5B!gZZ}+3#@?E&K`%JFk|Fd^QWMvZm_idV*QRSk)Qfb-!9RBNR zkkU#BFvxLX`mmbK^AF>5+yu8%qqJk!-62V2>-xO7N+fax^>OY0RrcJ0Q1}0L7iHCB zrlMSC!%9*lu0bR#Wy=+bjLf3YT|MBOzt`vPJ~ybI z-?#p`JD<;cy!Lv(-hpl{{}438f2!6Fs-FzsUDuJm@e$ovZd^!G4c6}BD_RNK>nxe@ zZ*_gW!G=?dsLmU8!+ZB`L3fTq>9KIc{%CBe@vW0yUeQxwgISuUX3HlxtXKHDcEhfA z*CPQBC9dCwG+MHo@Xr>_y%nAb-FwDce@a{KlNod=jq`8mD)|;MdWF4fY{I;9@@oAF zzsQDI3(ap1Q|g8%gRZ3Pm`)7UxYCB!n2M+JU0basv5%dNLspAWqGO*=v|ly%J^hL* z!X2U#&6l;Ju6k;C?^|n z?LZ>Qqo8R%-mIM8NWY?)+|15HB^=WgU^)mru0)|5BAm}S0ZL^+SX)FRJ1EXtRlv25x^aEUDaEZHb^**!wMGa@xOR!J!<9f6G! z3{M0mYXuXUaKk`abiKKH)Dj)(p~I9y!d*i73}H)Bf{^zGkL3s9vgZP}5!P$eW-~{& z`3sODqZvuRd4Q+X0C;?EXA@g95T13O@ z*WVLixuq^)6%M_#5M4YXg7|Tbf8)7ypjvqV_%@D|7Njv5VC~m=a4(mR(&Ra8RQdTG zOpqB6$cO86ZTO6on=h`DGGP40Rgjm0B$PK{RVaQ5IoLyq0Y_UEa1a5ujK(Zj6k$IW zxd!5wH6J&%wg(BwEPQzT=1m_$67AxTIV5p*Neb0Gf@H0&XLii&W5TzVA)tu#J7UdQ_VNPPH%5YiYF3aJ6*9#^%Lo2nHGxiOCE^$jSms?N^%* zl7rBSt)%e={qG>R=zNH5Y(N;sio0aJo1$S%3d8+h7Ss;>^^lK^qi!W(0|UsAJTw4V z@IE_qJE4Lo#ry|YFBP3MZ9rK+Nr)@aW2_TUIZ+s%?f&-3@||NdsPPdn=~6Y-GKO%W z4lr|#6d>bQEm^oITzUPZ&MK9@FB?7kt4s9TSOgJ%KQ`e)*yTulwQf&p9XD-0(iThu z3Uu@*gp&w;2vB1>cho~b=5P!)YAD)HWxIDf}9CkSJs} zK%9WP3#!wQ?dxG@UjEF6?&2xm#=fD#EWy)hL`k_vE1XUTBzx4W-VK=7?%Mn{TE@_H z)go1J%hp?OFVI~`UwF0|`7N^6#x&q3>Gh*9trMoA*fajV00MgRJ!B8sR#0-J89E43 z+g5`-D-w(t=?2Uew8eHQ`kvRa{_6H1j(Q=wpahPOia_9x^V)5XkGNxBbM;>`n^!2Z zZ~3zUkd_H6aC(2h@lL^w96Vp!fk&Svxq+GUu5FxOj#SUlZ)UcuyF0Gs zrcP=YE1G$8Fhn#KfTaVAM*7IsP!^`iyOq{pIlPoHKhEYLVE%kxcXv0CR?LP|$ulwA z$u>0OG|nt@1t}v2S`G|Xin0H#VaLGxY~sdXUvtZBDB{vJWFJqD?hiUD9w-AnlXLQx4gl;WuVl|cUWkoY$cxbXfdNDEiM;Dm9kHY(eh z=VF?H1;*q7#5TR-C2xnK@pn%m^hI4hi#Ae3jL|*fn~=%y-Lw1t&$Jlfxfx&uqyBQ_ zfc_}7)7*^N!NS=vC$lg_DS~bp(ALz51;&)y$a(M|+$MC9wAId-ivs*#4Egpl*&m#% zI-|N7fF}E6$7UX|MHw80FG{nC{6@;2T=%A#3Dc;6_>7M9Ux1HBY&0mzih~U2TVKT( z**t|9$vhrB&jkK#W5iAmy&@7%g+vDqlS#20NpPlcsRd`@7If}DHb&Vr6A7X?h46FO z^39f4mudKyKpokKogLJR4^25ZPN6_H03d%*SGbIbKTGjs%zLh93<+StVPy;ws~?mP z#DCKWGf?25#{FpUN>~lxGimtYFd#Q5#MK{Mhy;$D{`pHRx8TnGmkU^>L&6#ydh`I@ z$ghZH9hY~BqkSdM2W=wRk`1DAZDRgV&N%%POipeNp7NAM8=tK}8Dk+gM-n>N?}(M^ znOa{0qW7s0xrBd-eV`@o=Bx1IZ$0GhC}b$bEaDW3ICSXQ79dT!sY#?drmf4^NxLEs z8KO%DC;mV1!DAsK<=r1kFD{``17RO#@_%DPQVVDk0~24WbG1_-f?1*=JD4HU5ej4? zV?ZCl1E?#wTNSzIH%mwns@Z58r+)a)_hCFCu-hzF6#vV?!{v**%BU#(Zz+VVNCg4_ zP6I=4fA5OWkqb(9zS6Vq?oIjiT&zpQr?y~4gzj993{~riVl_vy*wz2bETh9JRuX7A zDvWW)1%hrn@M$bip$9qW%R)Y&;z=2}A~v|T`O?2Kt*u|Pys?k7Grt}l{ABfyAXKx^$x-! z%p`ph+6UHWDWo!6WDs;fL-Fcx2798^fDVH~3jIz4e!D{c&6_uW;L{u=w$`cT0nrRf8ifi;FLDlAmV0T}?UTvhVfWCmmas)Y28>0SuR`KytX`C`! zk8lLEf@Xe{&O52fq`K9fU;1lSKN7DB7t1TYbxTjrScCqo7xzX)=JW`IpETvQaxOMU zu=B=_FiJ+@VCI3jHO>C_SZT)Ko3|ZSpVWZ3f`Crl`^yAX;_;|;+ZJB2$BDJ zDU>0-cN20iM;bV+besVef;?u^FGQ;Q9dcV++xO6724zrIyO5LPA|AH3eG9X^z?Wi` zn7<=a{)U8sPxB*SH)lFT8I76=HrPrQf%M1sAtVLQ9))6~HSI!n~} z9|aLIy-PNX5W0CZA+{v(zh93R?&$7zE;rfS#_PTkd7ZIV(9if|!P*W_8UtB=2623h zef$4Y_2lY+H_?{miot!9}T)% zsZAam4=}p?EFq+SphAYTs~ zeMfY4j*tU6S=JjIqZK5*UJcWz|d3Zj?f5Y&vS<2hU6238A1={suSEac7 zO|rK7rsE@y7mq^m++$%U5s$}G=;eI~PB7R>c3HX#rEPOOVUA z=v=e3GyAKY6?TmPxZe-UZz#3sbd_IRguV+g8WS=#d(9A*OmPAFb5wC501|yy{|)3G zHyQzxayibSS81aVoUKXeEJ<>J5tIir>0u(01KS8avQ8Ud7D`w=KWCtjg7pdB{Aes- zK!D&&Sx){EYyqRsD|N8ni#h7w%cQ~9aXz%`4(tcS_+t8osUKKvO)7|HEblz1&D}p% z#o$Qqu%Mk7U_^lQU(-`&A?Jg9*&q2)p_cM87jH*b*J>R1Dmv-Q8spn%3JhwJih&5q zWah`up4W~saG8Umhl(dfeKO@d5Yy@qb$e8c9Dc&*^{yZm1R}9f(imc4^s4ePcb2vh z=AB``&@Bw6P17Jo!UFFJy7OP910>%XbN3gTY)1Bq;S3TGVsm+AacLc86CEy?&j00g zWJ&A5RgA;b$Rn_8bimyBBR7(dvyY99wTTd$K!)Y>VWXHxiLeCRl$d|78)DfO3A;;D zp-SJ`ZzU}=c5MnEf#b!9BMO?T#GYqwhH=i0> zqmvB{mLa?wj}CQ|-+WLzsba}Q0z~{I`*xjF8DM~_7h=rx1?PbFF@J}bs4^i{7qd=J zIkPjT-1B3&nC8B#=91wMSL@k{FQPwxhFb@5M}vWaEb5*ZL;499q0q5tV;#-eC8rYK z&>yCLU=lcO(h-AQs}|Mtb)8aFq{5lO)@vY6QXua5fVV@L^yAB8Lj8l<)7|P&!cCUu zuKs`e&nQ-lkh4x!4Iw2Q-qyH^GBA&ITgHz5Q0F@7hMCU*Ar)S;#WsUZDYSncatyoc zGB0qDj&gaQ=z4xM{`}!}?7Pq1jEk$Kg;=N~)h=rMFDA+_1!7bA_Uy>M0VqLYVl4p2 zY_O%43@T2M9wX>~dsOLoPiW_LZY!rWN=Hm4ob^2L!*@xO;48Q{5b+gh-SBb^Q^8=(^~SCPkvn9Y5iKu zawe!zV!!$RggHVt<933S}{kruj0b?@GbV&^pQCn zN#CEWA-;iSWM+&q{_t4XJPRfMrRYyO@KmD>p%nZkw(T=jAk{K^ zEaa}A{>*F5ItG(}v-?VfFtT^$m4J!M z_i}w0k@3_pA|e!-qz`^A_DXIQgz=L0O*8+Fh()#4iaVNgcB~VmULgGT8W~OrFWj#2 ze(NJrQa1YZQ+{`v4L~0jk&soKJhs8cib@8Lk26pVuyL#)a&Y=vfO~vt*$=8sTVz1i zI@gxOj{%ZNO~79MwLaAoSUf6Wt0%IniUDB`-dITj63tm82pSNV_<({>vKkwRg0{8U zYdLuxBZ)>Rl4Q1{mw2K-1v=#GNN8kd*G9@41Ov>v&^pexCp}s- zmG=xU1DS|9L}s}L#Kg@EZ2~e*<7U!;#BzDjP6#{%@@}THFAW`D93Axj6RCShAW7zC zF0jo~%|A`yPorzZBE~*0APO$gYZWAo;}d0dI}5B5G9R}Fwstc zB+q${J`*CWMGz%%+VE%kRe9uiy0`I*jvID1JXTD}vNmY~lgEziCjT0_zP`SEn^4G5 zpJVI&f>R##m!z_g_Zj46o`E+D)^2lhaH0I1Z%TT}y~ue%B`OC=uesjNHV9LNmVEs1 zVcEGp(C_oxER2iEpZ_Xbpl~Va?hw=1xcohsq&#CFqou zi;BG@?UHV~vB^Yg^Vzj+jx8yF?)Q16GrHpsLcH{xj}ZVW^(>h5%F+=E#`@jei!;Jh ztOCW1Ub@3$v#wRzLc+#lK~}5Foh2ftH#bDIbqX3BuOg| z9-VY8z@N`Q`&G(+W zk8Ap94Tt+?tTw~YrARF~B&<&6Y^H{hpy$R7yQ+me(9b(>;gCH&wuHCCsxg&^P%RyS z#qvva&vI#BPGy;K4#|^UMQ$AwSRbu?O)Ix$n2BJY zF8%TyNXG3J+BV@9Ua@CUEO^s3Hn|Hi?xqAesFYhL9TBJ9Jn!4{?(VGSdF?lcOOeYi z>B<8>&?Pir{B0Du#6r95!fA7=XjWqjp$G69H^Y*9HXRZN2@V!yNUsvfxZ+f60q+^s zIBAanFC*|ux(uXX+xh&tyxNf(02V=q%zQDY2o9v-T!R8Z&RvW0;S`oUr|71mL->+7 zC&>@wc?e0;SL$v_rc~|Fe|gItl5o)esZZ?BwaYoZXBpZrQb9qVmhRanK~ae-EeX!f zSvy4xJfG$5U-0Qr9HMh6nBc`YF-=36kDJ?jpzQJeC2BK6F>gc2T-d!#eJ!QgwWl6m zjMI)6Y@?U~yv)UEH1Ui8xMLy7qcB9W41}KgjG7k0e$0G+$1AW}m1Lo{6oMjgVgUlK zzL&s;Uzk@wA%#Ic#np5vK>a}Vjlfa>oM}%vn(Z(VJUSN}n4g2$Zi8;=F>eIIQRgAn zviqUa1vcbOmw56uZsGASYOXD^{d$Byl)1Q^B3fvxyp(OPU7Ht)6c~@+C{`~UGi_*T zq5>}fD^BIn1SAT)uHkkzLO7{?l=W{6^{5D2f2UVlL64=j0A6Ks0ZA)Jf^$P|hbg_U zDm%OIR>=yaETQAlyOQ?0@+4VgALI$P5IQ?3BE^w$u|wO5|ter)69rX2cPe7!#LMpS37XxbF zUN!f)6#m5peEN`nZPDFz&mRW9UpXy&io=_&r0XPaq)Y=pwT`9~~|2~0{?EbTAfMe?Zd zzxeH$QdKKdZKC8fJN5lRsM@Diw_U%|#@XqJ_Tc%t656xqOL#MO2Zg2Y4xfDmC%EJg z0GrpA%#+Fw!d?l)tkJS64zYdfvtemqWA<3@1GQFv(gZPI;h2tS{a>1g$zC)nJQJ_0xrzs(0UL zQ%=J-t|zUpO1K~CA*_lR(xOQ(Szt@VufR6bCv&`s=hBv*iZ#u>8hbFUHN(gqFwJUt z2|~3)T59wvAuv~k8bwI*T>auZL}fL4rvqyd8pXne=@#tT#$%xUQsHo zB!A_{9JAc3A>D3Jn)jNkno_`v(rp4_!$vp!ba*Xqg@}_7y!_Gr%T@08VfYURYCf-x zVeG(r7oL?E^Z!L66nk^e?AkdEebk~=y4#jv82&X)Zh3Kd7dhI8@zC--Dn>Zr&C)_) z{r79;73k1p_uE7wkueeFMG&>`55{hd{NdOJR^Y?1uw|Q6JZ^_FZIB{Ql-Nh^A|+UZ zPd8A#E$C#N7A#ike%akV=zAH$2fs_ulDXJDb0pX$R*A+byX3~XKzGruN3@&SXv`A3 zgACydZ)gXklVNa)Kv^Pr2$6{E!a_EQpOGkyP;$GCG%kykCd@eKgXRw!O-X_w{gzcl z-vvW<2^264cl_S92r-ZfK+9hU2s}t(0}*+}9CyizJ3f$iTP&1KZ-5V*z5*2Aaqg_)C zp8K_is{;k}Sizb)g%w9*WiB!JK(;j$D8=~Y1|v}FenL&SG-5C-P(Fa{Bob>+PPqa= z%3q4P!t02~32S?j?03Ua7U_2#EFQWP_gf}4LW;z$;@7>QlWQ09Y`^1c#G|klUSjEr zWa2*X^3u&nL`s+<1pDa4SqWYYl9?UkuV#;OIwKJzCu~!v&d-TvYeCbz!iJ%Hpc2$^ z?9_osK*e~`bEV1ST-skYy4rSg)`=_D`rNdbxG|$R%RB>w)5OGaD$tYobzW8@{l+ZH z`wfP5%^6~q2uxH57 znlI8Y`@Mi%Ic3sRU76&`NeM=Sm7eBnAEQw2wO0C(M4X?b@S)xKC?DI8k2EPn2wLFb zFP!QJ@|^u$+j6&!Sj8L)(kwjRUsd4QV;3{^VH+U|<}D43i4jkeHSFqTCi5;F4m_lE zNNL1L+XE4EbN1JEo>}M(1vqPjs)c(>Bcc?_4=vd!>CVFOMvXp%+t;A?{Oy-}xOAOOTpzcFEGzXlztq!sum#3WadC zdF)7%1fdO7sL>M%F6sDOvHtk2(_LF*S{z!deX4&fHa)NP#lNkOkt%0$CI(@8xZVzj z#XjO1If8ExF815!a+zPkD1v_9Uc;oXnnb-Bu^6YG^};XErlgBTMk!$Ef^n z!A=d%DV2{HWOoAkuKntwdG~9G;ci^iV3-pmmewslNXrzb?#ECA#FP=h=hT^R7$3pu zt&6km7~p;;|46pY30r*R6#n_`Rb!78e6F_Es0s&8oRRdadz5-FhI{)1SNGd*c4X}p z=@(Bfw)*h#<6$1luS=^BWfq&QCA?UV{?C8hPP}pF`JH)XV@o?md);%!k8FH%puKeE z=c$tSpH_z6n)q}kLFv=U-ri3N?d%gLO@!~ZML35-zo6Q$T9Y24$nrw?^TQaxCg~%+ z;W-akgt19;B0LWEE^MuJEECQTScha<;g$AZ3rLs{j!@5iGcU>dVaw6g^LV;OmcF&M z|JKnXBAvMDO}#VOneZWxbfuo;MO~@UgmbF?^a6=R)A}FQ8$EX>AGNGj2qp<)vl65ukVa4lL2A1+;X|Me(<*^Hjvs=Q<3K#Q~ z`2}XLuEBj0bIn>^o7(ALHY&V@k;6n*(|8m!z2N!Nlecc=r9_pl8$2vu=-+PjO5?#B zXT>UOiMO}zjg}@yG+Ze@>2MyczeRMFzHkJR;d_kuAeDX>sG=Dx8TlV+uf-b)wQ8=# z!L_H|l8=}H6=XTbkc+71@=@&LS+xcXwD}4 zI#C6uzEA|sz;4ta-Am9_PsvHWARv>6q5UV-|X5;te@}TleD$@x%{|txza|X^j+k?83A| zN07V%aqwySC{IvH0x{sz3RekZgz7}oPLHDXJzb~sx%qje?elTPy~2Q z87HHoE)8C1-z};1T~Bt?rcJFczFhBHyBx(VWnOUhMHw)rnm@mw*gA>ECzm7rl!~Cn zGzU3I7;mxyqGag$dw6-JScd&=#d?V;P^$Cm4J_^ERd(Rh>0?np%m5pO(5YnC1Jw`B zIbWqL8PmJ+@gg*Xt_Jz=Cor8s3if`mF^&r)5brX-`wNgFqcd?WIS_U+xtUtm2B%v8 z_rp-vAas%@o%#S^pU;J{qX3?13n+^)Eo&yzo8JVpb8oOnIL$j?di%Ew$}$Z^Y@set zLraZ?m|&RSu{X=xh+tDufRkJa$LbH#12w3GVj1%HA}}2in{hks=QH81OCG1e^c4ZX zpF~(ZVh7&ITu!Iq_naG9DoubGWKadkO?ZU=GWgepl;vNdekrBe| ze7a8I2zCwcMmS{ZGt_ZobvQInv%YukEP#-_2SYir#|)GI;@NcwvGYBK7J6`OzEkV$ zAyJELnH_wLPQi5MnJK*T{Yh|9iR?*U2{_xdg)GgqA?SGRXm3$40k%URW(O;W{z}n% zZzumH*|yPmAxnCO)CTDerze>!6?p%6nwn;BKx9yWa(E{q{c_}7NmG_-@$=aoh_=_w67;U-p_9tfnd14HEHdyBs1~m$>X+zF#Zh}g4{2Mfl>gC4~i>{Rj z(K{Y3BEAaAF%SLCc?pV$j~+WlDV4HJJ|-84rbqwdh$n44ls8koVQp7a_K)+KU%{V8 zYVWWAP|~wZ&RZ+G4S_6K5<|NHgT8ncJ?M^y+|(c$Ne;~&q?LLk@$sTZC8&q4&S^nU z`!&$0;&oW+F)SN2GnoNbl>VtSLYjcHkP^rV{NBvv@Dtjuj>@)Akz zSz0mySGY4{ZzhQ$3(wYl(O(n2_q-N_!uQ1;wAYu}?{)Sf8bG8BY>Kood)+@zGG7(@ zprX1!HLjMWglPt4BXES~8wVoC^E{5N+IEB4TOd*dZ{~@CWrjquImy`e8h|IXrx&75 zbS~g1GQ|^^z9J&w*WA)U2Irf0@(Ri~EUp$a%D^0i@)=bKR1Be*r3pC%(o~F*5mdK? z4nd*91FVCH)8akHX3US$+Mg^H2>dzPb6zWxgzj_5u{N8#?yZF&)~!eDs3*q$ABup! zB+7(ETR@kJ)SseO>*H&%p1%@n_9y(_qp<+$M(H?b*|8d8kw$0eH?oyU!1+BNTnBV^ zUX5gA|3inOn3+L~Pmp-ek~bDHY?YkZ_EXpHM_%M{Ve%O$AjDz+`=lxvDKL|;C@b6j zjIC>xJ80s$qz^+wqUMKK;=LSo;sh~fIswA?H9oSX!EDX)P~pO0w;^YG_I5QM4rX9f z#SmY~_(s{?ujwmd1><}cDT)ocUR*DV@{Ho9L79+9MB@F-9Q(U*;EkO~>Gb2c3-d`a z+?0Iy-{7%=x;{H?U71 zR*gyG5S$GJ);YR!xx z)=Wd`N;&><-iD28k|VY0D?g&P}tJ zD&%-JrP}Ii|2|Pt+W82QwU7W>^}?NoW_%VBMwu9)#_P%?aJ&DE|3-F%oDV;hl~}cS zZLN-Qed=P)Y%B721}Gmxd|-Fm_ywqc*5`aXqo>CuCw zD6WyoC~T>`I^tBMabXDDDYn;PN#+lx04tKwQv4W?pkkbDF^enq`xZ2W0Bj$ZHh-iA zOD&b-PP`@

Uv5r@cPbm+ne)fP&%q?Z%h!#hNNc#c8WWou}WalqIZ4sq_&&MzR+3STpARi@D1tSad7p^gP~2K zfm#e-fjZVK3)p-+0Vt45dr_o!-C?#6mKx$#7}(-&Dc=T2vajPzGm=CyqBWQNp^b*A zx>9sv5;KvfxL5vbt5ozKq#a2PwV~n%`{%eTk$u)axwdVT^HIsUFxr~?VE*w=0j;w){{c+Jm{rNjPpv=?Y7 zOXT#%<7@{)xlCSGuM`@jul%>3TnCO+CmVGjQr*p{Yd1sM;8T&7jf%+m@FfvLTZ!cN z>|fR>I$hB=cv%hQA3IzNXyJM&CfEJNl4q7=p{gdI2TX%nqgKeAgu>Z+LgkZ<1-EXQ z(Ymm>+RG+UtV8O((loXF8*Lztxyxah@kU4-<4>?}M};Da zx>*#$G8`}$686~8kZ_IDo~%nLgwNI<_olp1JO-|Uqt!qMAm$4hRxLhDKQ*%2!S zX|4rcL_)07`5oXY$$Ye9IiaJr28W7oRL6lsxS+-z8ab)KQQ*ASOOJG&O_SGW73D^I z*C>HNMGme(>oqc*GTe1WV%+masqrX1rh_M_VxjnobnF+^%e&qfgSti7I*J&#|W`0^0mUTSm8L z4L)qrJo+!GO7!U`ucoTCJ@R*sH^D=4>YrohPEN^BY)rSx;=CY>qhxMlO>oo6krvyb z|4q#6BIBMAE6u~U41VIE9c^8y4P>%5W4Ab)iV0!&a5eNLy;vNlR6r?CL5>Kg-o=)> zTV||JvPd$Ys8U&CQqiCrUaGS>bN#Z0v>OhcZF|($#vQ!lG^xDA&`t1K%ft;8J`?}< z52Nh57a3}%muUnfAQgQ+Rw3^zqx+(*ddxlj1%`{BWaJF@C$>6GeZBbO%Tn{A*WcAV zM#5d4)q~4M!kr&ye7f*+B0%9w>^H~k$*ESqU7Gq2O=QmAvx~M$c%|#2|4`t~Nb7cI z@nq>&ItKcQdHpT+>cMw1=7l?{C68?An7-5C=GJTfCmKKBZq^Hxsnq|zU2ZC^ zLQO6p?P^6?&PkwT!vi}5eJpJ*-Rqki?)PX?>9Fm26a3KV*sdx8s1qUGsrcG?siFUe zO@o~SU(JLQUrFUA{c37`37)VPvxA~1NKb$U3QFJX8-O10(4h~ij17c?`hy?J3SUgK zhhkUG{sZ^#yB^7VeB*A|(uo}tS=~#yuD0tRsL_g5%V{^BJMAbQ)8B+&c~`#jQ8f47 z!s3U~ZG-s(n*0sfePP`SH_}>pt{(GE^7Xuz*BgIcM)lid_ScR|-vd3{Y(I;sYH0Xa zStopUTAkb<9oyV{O8M@CD3b}@lP#9=yA2P&%c${{4qabcoc=1PR<=(cn%Dcjvd$Q_ z(-*rdeE-yR^`MN#y9@(oEA<+wwXZ$4#6RD!s*!Wc=&M+e%yXZznqmD)lUsSBDW)2UKQ(F`ZnTAie*vDILC!sRXai~I0KriI1UE-nA8ee|B1M!D`Ko{ZV znjO9Ohk3lCe487{*UzM=G?~G48_$vE3K;e@ycub-SNEBG+*?r=FcK7HWs~wSs_j!@ zz1^$KZnH4MEP)+i`eHyFy@8tVyqaP(nmzqrSrtho|8!m%AlJpOXBlRz93$EuF0=M( zg<4ML^v|7|Q@&r!Gg?P$hJ(XW)aO}cx7a=tc2(6F<{7(P{dFYLcTuR_=aRr($Z+P( z4TOqd4MWX$d2UwTKa8uS(*3%wX=FPa;ND!GwtO-v(*OiB`ZdJ!$M<2sqqivCQCy&% z7+pj0o{q_?Z+1zC zhCC+60N=6TaCjuOh36TFPT3`#lPQ%<4@~h{4rE(9qLvfn8-HHPPcnJLtV+77vMked zLvsJqJ+f;Lizfr%J^9iN^nia_w#(Mobq-6mTUfo)tsQT0_H2sj>svQi(l@3j>(p89 zD65+5rvC0qFTbJZpaSro@B4wLVjG5^e#m;~%WtAtuC+YcuuHsZp;)x}`>~6CEoUe9 zd8@9KJMI!zs~_#>a=rCQ&O6`t6Hhk>cHDG3=@Ty8kWZd7PiMoNIXv%lw{O|g|GMls z5I|Ti^2Z}5)O|Eh#*bxA`n7qh`}ljhbbWSCt@rHt+0`%bPkNtjN^o(9{PCfJFlVb> z`c+;~IDfF|-M0()gy*&y_normtPZ%^I)0*Q#JLoQx)2!S$Z|JOJ%U^VU)_?PXL2h zkW_fzSlq#Y^6S`8M13wDo_xTjQD>bK@T?PmxRCvIo7w_^aG| zE$SyMtPbcV_&kG)Inzh>-qK%c=WbQ0o4jO$t9!R>gO|;Q_G4a28yXI&R%$?2PX;H! zq^Dw#9I;<9l*JJ=NNpc_rp1o|9r(Xgbr#V$n*2JXqWaSH(-Bh&jsl<5?ZUe^O!16q ze)<}yUXx`oSanHbAauX`bowh@(V$G-lzSV6{gQp7dc~{8!gcpKO}$nN50c3-5Ua_^ z^ldim@%8kd$c%1zapT<)cfX#7!Ag^F-(UV195Z{9e(|Mwf%S&uvMLa=rnf4}s{X2s zdgIe8qF-f#e5b1flG#!N0;sP%tAO{q<&qkg{Ct?$C_30U7nkz+OwO@dfFb2KKAhI|6~bwOoUr3 zzm+$dB*?4GVwM#_uPL2i18s|ctIy(Zv41&uSVXB>xYYIB6|rqS3YiJ@83?t0x;-K` zJ|d;9**5AC=~Lu<(zV+^zP!ZbjdXaqadf!P*Q^si+~s`&;=?jrH|T#^68K2@wsrfl zjqzo9Dd~NglRvK|G~^rh{D8mfYxvj&`PlT1G#v?&DO@pjN!YJ$e5Yox=qCSN6BY*Q z4?Cldx7;$=)xPD1PN#Yxe zUVC-duTEC#UlT8ROse>mSB9=_I6fRQJrraYtr0!#Y1rFU>HB4Q&g8c>0crZevE@5L z;?K!U?GkgHSpF^}$TvN_sBCk*n%UI3!7<^!KwDQ|7q#T#@uyRtYSRODp3nGr^Q>{+ zbXkB(z}-v)9AYXH1r49OrfLiKU8#27G+p0##Nv5x^vqVX(7mBhC{)(YJ^;MW)JIs? z@lypKFMTfz+;aQ6NmG%t4Zy~!TPw5qRc1oN=fOK}C-o}=-h(FO(cef|0dPvEpSvBs0qVIwB(GQx^24cFEVShRbqgH+aY^9k_u_irv+ zWO!hC`+?|~!~xZY2a^Ne5&N>QGR`O9fNt=&&x^mx8iu~ygX_F!fzT?71bC{_^J=8w2$zueP(k{3nBk`>h|T zjkmfE>z?#U@C`GI=0P!%cIQw}8x13c$;1~5L)Q(e3}=7*nh{j-K|eW9`c=(| zqL01J{TBieyXb4#5s=j_d%&Uzbn~&>sd*z;z@*-6B5t%|Qu@Ose|Sb?UX_l2aPdg% z9`)ddioW|i{l6SuoqVrcI3PI|{+zgI*~#xE52K5ZOeJ#rRQ#3XRT6Yt=2B76rW{|p zmf@nB6Q+-i^Eyk?UaymoO9}F;D=V^?EM|>J%dUK(h@A@|vU!T?U=rhPakhKHCOiu( z?yd>fe`7=u3uSLa*LyTcffTy^_UlM=@!Mflc?~DOeYd1y$~4Abl~V-`m>@NFJu%Oa zb%UL?%k!Sp+E&+^oLobbVab^Oi$CArQ2=4IA&y{ zfaPWtHh60&`e*Z}%W_rU`rduJjo2NU!If)$ttTub!4ixQlbO_Gv&F87LH`3AL*B~$SsQmSbcWpY!>t?w(N`V!7d z0nXto(@~k26-M8B_j>t9dmu()jc?`Du~zlL=!UddhaQ!U14Yx?DMfb?n`)g~w^GHC zB||5g-UOX!^HOrFSbXRQ*ra+V`zqrP2JG7ubm-}~tgpFx8t;z8cvO9e)=+Z$5}&c) z+a1>c~sT$8@{S{mP~d!-B2j!YCX-|FIkaVEn>6-B-OIb zRs1-kB7FIrC&G6g$k`sQZ$2BIa7fiQJzeUoT6NGm+sF0m#;yej=uLo0IMTj7%P!ik zuVuUFlt#1dgbmo)i>u1q^mb7IX)>IaaxV}FC@{*ZXVIXR-D34z*I~)xhkI_Vf8DZO zUB6OEJ~?EqAy^ML{VIF9x~^+~=-mS9s>= zU|He3xa-i-FLZoQt~ zEMXt;%<`j&vofLO#J|B!?yyLGzT#$fvg!2m@tgi;0pEsp$f0_hy5i_F+jGs6J6qWgproKX=26IRn*z1%JOd4t&%g-9R!(f?z z8j>eC+J8Gq`~mk%t-#GJiHcCbv9u6}RnGirDLYT7@~P59=m<}%Cuc+mzOYL&1#kAHJI^<7`eNd2tr(zY2pCE8XfBGyzaP0r*p)mc8Ebh0o`*i4!Gf?)6pGsf_ z{pJ61Q3oKKV_NAZKb0^@{us_b{j`r)QTYqDOb(vOzeSwY72)?wrv)yuaX|Uqca0Tzo9@H7KX$>W9H16o}M!-1WwPKd5F_9cg`ArK1_-Iy*~J5 Oj;@x$_S~(6Q~w8oee#X~ literal 0 HcmV?d00001 From 46e513c1b30292ba377aa7771125b72b7bdc1889 Mon Sep 17 00:00:00 2001 From: Heng Ge Date: Tue, 13 Jan 2026 00:16:51 -0800 Subject: [PATCH 4/9] docs: rename TakeLast and BloomFilterGuard to include Exec suffix Co-Authored-By: Claude Opus 4.5 --- docs/src/format/table/mem_wal.md | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/src/format/table/mem_wal.md b/docs/src/format/table/mem_wal.md index c8d514c171f..24d583021b0 100644 --- a/docs/src/format/table/mem_wal.md +++ b/docs/src/format/table/mem_wal.md @@ -737,11 +737,11 @@ Bloom filters optimize point lookups by skipping generations that definitely don # After region pruning: only region_A needs to be checked # Bloom filters checked before each scan to skip unnecessary I/O CoalesceFirstExec: return_first_non_null - BloomFilterGuard: bf[region_A][gen=2] - TakeLast: region_A[gen=2], filter=[pk = target] - BloomFilterGuard: bf[region_A][gen=1] - TakeLast: region_A[gen=1], filter=[pk = target] - TakeLast: base_table[gen=-1], filter=[pk = target] + BloomFilterGuardExec: bf[region_A][gen=2] + TakeLastExec: region_A[gen=2], filter=[pk = target] + BloomFilterGuardExec: bf[region_A][gen=1] + TakeLastExec: region_A[gen=1], filter=[pk = target] + TakeLastExec: base_table[gen=-1], filter=[pk = target] ``` Existing Lance index optimizations (scalar indexes, fragment pruning, etc.) continue to apply within each lookup. @@ -921,7 +921,7 @@ This appendix describes custom execution nodes for MemWAL query execution. Deduplicates rows by primary key, keeping the row with highest `(_gen, _rowaddr)`. Since each dataset has a fixed `_gen` and rows are naturally ordered by `_rowaddr`, this can be implemented as a streaming operator without full materialization. -#### TakeLast +#### TakeLastExec Efficiently finds the last matching row for a filter predicate without full scan. If the primary key has a btree index, directly queries the btree to get the result. @@ -940,7 +940,7 @@ For each candidate with primary key `pk` from generation G, checks bloom filters If the bloom filter indicates the key may exist in a newer generation, the candidate is filtered out. False positives from bloom filters may cause some valid results to be filtered, but this is acceptable for search workloads where approximate results are expected. -#### BloomFilterGuard +#### BloomFilterGuardExec Guards a child execution node with a bloom filter check. Given a primary key, checks the bloom filter before executing the child node. From 9136a774959e10c52b58d24680fba533ea386442 Mon Sep 17 00:00:00 2001 From: Heng Ge Date: Tue, 13 Jan 2026 01:00:55 -0800 Subject: [PATCH 5/9] feat: implement MemWAL conflict resolution for merged_generations - Replace RegionId struct with Uuid type alias for consistency - Rename snapshot_timestamp to snapshot_ts_millis per spec - Add conflict resolution between UpdateMemWalState and CreateIndex of MemWalIndex - Implement generation comparison: committed >= to_commit is non-retryable, committed < to_commit is retryable - Add rebase logic to merge merged_generations when CreateIndex rebases against UpdateMemWalState - Add comprehensive unit and integration tests for conflict scenarios Co-Authored-By: Jack Ye Co-Authored-By: Claude --- protos/table.proto | 4 +- rust/lance-index/src/mem_wal.rs | 109 ++-- rust/lance/src/index/mem_wal.rs | 310 ++++++++++- rust/lance/src/io/commit/conflict_resolver.rs | 498 +++++++++++++++--- 4 files changed, 774 insertions(+), 147 deletions(-) diff --git a/protos/table.proto b/protos/table.proto index 8c032803817..71ef98066e8 100644 --- a/protos/table.proto +++ b/protos/table.proto @@ -586,8 +586,8 @@ message MergedGeneration { // merged_generation: uint64 // flushed_generations: list> message MemWalIndexDetails { - // Snapshot timestamp (Unix timestamp in seconds). - int64 snapshot_timestamp = 1; + // Snapshot timestamp (Unix timestamp in milliseconds). + int64 snapshot_ts_millis = 1; // Number of regions in the snapshot. // Used to determine storage format without reading the snapshot data. diff --git a/rust/lance-index/src/mem_wal.rs b/rust/lance-index/src/mem_wal.rs index f80eb678e3a..ccc9cc3db0e 100644 --- a/rust/lance-index/src/mem_wal.rs +++ b/rust/lance-index/src/mem_wal.rs @@ -18,67 +18,8 @@ use crate::{Index, IndexType}; pub const MEM_WAL_INDEX_NAME: &str = "__lance_mem_wal"; -/// Region identifier (UUID v4). -#[derive(Debug, Clone, PartialEq, Eq, Hash, Serialize, Deserialize)] -pub struct RegionId(pub Uuid); - -impl PartialOrd for RegionId { - fn partial_cmp(&self, other: &Self) -> Option { - Some(self.cmp(other)) - } -} - -impl Ord for RegionId { - fn cmp(&self, other: &Self) -> std::cmp::Ordering { - self.0.as_bytes().cmp(other.0.as_bytes()) - } -} - -impl DeepSizeOf for RegionId { - fn deep_size_of_children(&self, _context: &mut deepsize::Context) -> usize { - 0 // UUID is 16 bytes fixed size, no heap allocations - } -} - -impl RegionId { - pub fn new() -> Self { - Self(Uuid::new_v4()) - } - - pub fn from_uuid(uuid: Uuid) -> Self { - Self(uuid) - } -} - -impl Default for RegionId { - fn default() -> Self { - Self::new() - } -} - -impl From<&RegionId> for pb::Uuid { - fn from(region_id: &RegionId) -> Self { - Self { - uuid: region_id.0.as_bytes().to_vec(), - } - } -} - -impl TryFrom<&pb::Uuid> for RegionId { - type Error = Error; - - fn try_from(uuid: &pb::Uuid) -> lance_core::Result { - if uuid.uuid.len() != 16 { - return Err(Error::invalid_input( - format!("Invalid UUID length: {}", uuid.uuid.len()), - location!(), - )); - } - let mut bytes = [0u8; 16]; - bytes.copy_from_slice(&uuid.uuid); - Ok(Self(Uuid::from_bytes(bytes))) - } -} +/// Type alias for region identifier (UUID v4). +pub type RegionId = Uuid; /// A flushed MemTable generation and its storage location. #[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] @@ -106,14 +47,20 @@ impl From for FlushedGeneration { } /// A region's merged generation, used in MemWalIndexDetails. -#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash, Serialize, Deserialize, DeepSizeOf)] +#[derive(Debug, Clone, PartialEq, Eq, PartialOrd, Hash, Serialize, Deserialize)] pub struct MergedGeneration { - pub region_id: RegionId, + pub region_id: Uuid, pub generation: u64, } +impl DeepSizeOf for MergedGeneration { + fn deep_size_of_children(&self, _context: &mut deepsize::Context) -> usize { + 0 // UUID is 16 bytes fixed size, no heap allocations + } +} + impl MergedGeneration { - pub fn new(region_id: RegionId, generation: u64) -> Self { + pub fn new(region_id: Uuid, generation: u64) -> Self { Self { region_id, generation, @@ -124,7 +71,7 @@ impl MergedGeneration { impl From<&MergedGeneration> for pb::MergedGeneration { fn from(mg: &MergedGeneration) -> Self { Self { - region_id: Some(pb::Uuid::from(&mg.region_id)), + region_id: Some((&mg.region_id).into()), generation: mg.generation, } } @@ -134,11 +81,11 @@ impl TryFrom for MergedGeneration { type Error = Error; fn try_from(mg: pb::MergedGeneration) -> lance_core::Result { - let region_id = mg.region_id.as_ref().ok_or_else(|| { + let region_id = mg.region_id.as_ref().map(Uuid::try_from).ok_or_else(|| { Error::invalid_input("Missing region_id in MergedGeneration", location!()) - })?; + })??; Ok(Self { - region_id: RegionId::try_from(region_id)?, + region_id, generation: mg.generation, }) } @@ -146,9 +93,9 @@ impl TryFrom for MergedGeneration { /// Region manifest containing epoch-based fencing and WAL state. /// Each region has exactly one active writer at any time. -#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] +#[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] pub struct RegionManifest { - pub region_id: RegionId, + pub region_id: Uuid, pub version: u64, pub region_spec_id: u32, pub writer_epoch: u64, @@ -159,10 +106,16 @@ pub struct RegionManifest { pub flushed_generations: Vec, } +impl DeepSizeOf for RegionManifest { + fn deep_size_of_children(&self, context: &mut deepsize::Context) -> usize { + self.flushed_generations.deep_size_of_children(context) + } +} + impl From<&RegionManifest> for pb::RegionManifest { fn from(rm: &RegionManifest) -> Self { Self { - region_id: Some(pb::Uuid::from(&rm.region_id)), + region_id: Some((&rm.region_id).into()), version: rm.version, region_spec_id: rm.region_spec_id, writer_epoch: rm.writer_epoch, @@ -179,11 +132,11 @@ impl TryFrom for RegionManifest { type Error = Error; fn try_from(rm: pb::RegionManifest) -> lance_core::Result { - let region_id = rm.region_id.as_ref().ok_or_else(|| { + let region_id = rm.region_id.as_ref().map(Uuid::try_from).ok_or_else(|| { Error::invalid_input("Missing region_id in RegionManifest", location!()) - })?; + })??; Ok(Self { - region_id: RegionId::try_from(region_id)?, + region_id, version: rm.version, region_spec_id: rm.region_spec_id, writer_epoch: rm.writer_epoch, @@ -265,7 +218,7 @@ impl From for RegionSpec { /// Index details for MemWAL Index, stored in IndexMetadata.index_details. #[derive(Debug, Clone, Default, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] pub struct MemWalIndexDetails { - pub snapshot_timestamp: i64, + pub snapshot_ts_millis: i64, pub num_regions: u32, pub inline_snapshots: Option>, pub region_specs: Vec, @@ -276,7 +229,7 @@ pub struct MemWalIndexDetails { impl From<&MemWalIndexDetails> for pb::MemWalIndexDetails { fn from(details: &MemWalIndexDetails) -> Self { Self { - snapshot_timestamp: details.snapshot_timestamp, + snapshot_ts_millis: details.snapshot_ts_millis, num_regions: details.num_regions, inline_snapshots: details.inline_snapshots.clone(), region_specs: details.region_specs.iter().map(|rs| rs.into()).collect(), @@ -295,7 +248,7 @@ impl TryFrom for MemWalIndexDetails { fn try_from(details: pb::MemWalIndexDetails) -> lance_core::Result { Ok(Self { - snapshot_timestamp: details.snapshot_timestamp, + snapshot_ts_millis: details.snapshot_ts_millis, num_regions: details.num_regions, inline_snapshots: details.inline_snapshots, region_specs: details @@ -324,7 +277,7 @@ impl MemWalIndex { Self { details } } - pub fn merged_generation_for_region(&self, region_id: &RegionId) -> Option { + pub fn merged_generation_for_region(&self, region_id: &Uuid) -> Option { self.details .merged_generations .iter() diff --git a/rust/lance/src/index/mem_wal.rs b/rust/lance/src/index/mem_wal.rs index 55a00b41d8a..6bc596dd42c 100644 --- a/rust/lance/src/index/mem_wal.rs +++ b/rust/lance/src/index/mem_wal.rs @@ -124,19 +124,317 @@ pub(crate) fn new_mem_wal_index_meta( #[cfg(test)] mod tests { use super::*; - use lance_index::mem_wal::RegionId; + + use std::sync::Arc; + + use arrow_array::{Int32Array, RecordBatch}; + use arrow_schema::{DataType, Field, Schema}; + use lance_index::DatasetIndexExt; + + use crate::dataset::transaction::{Operation, Transaction}; + use crate::dataset::{CommitBuilder, InsertBuilder, WriteParams}; + + async fn test_dataset() -> crate::Dataset { + let write_params = WriteParams { + max_rows_per_file: 10, + ..Default::default() + }; + let data = RecordBatch::try_new( + Arc::new(Schema::new(vec![ + Field::new("a", DataType::Int32, false), + Field::new("b", DataType::Int32, true), + ])), + vec![ + Arc::new(Int32Array::from_iter_values(0..10_i32)), + Arc::new(Int32Array::from_iter_values(std::iter::repeat_n(0, 10))), + ], + ) + .unwrap(); + InsertBuilder::new("memory://test_mem_wal") + .with_params(&write_params) + .execute(vec![data]) + .await + .unwrap() + } + + /// Test that UpdateMemWalState with lower generation than committed fails without retry. + /// Per spec: If committed_generation >= to_commit_generation, abort without retry. + #[tokio::test] + async fn test_update_mem_wal_state_conflict_lower_generation_no_retry() { + let dataset = test_dataset().await; + let region = Uuid::new_v4(); + + // First commit UpdateMemWalState with generation 10 + let txn1 = Transaction::new( + dataset.manifest.version, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + let dataset = CommitBuilder::new(Arc::new(dataset)) + .execute(txn1) + .await + .unwrap(); + + // Try to commit UpdateMemWalState with generation 5 (lower than 10) + // This should fail with non-retryable conflict + let txn2 = Transaction::new( + dataset.manifest.version - 1, // Based on old version + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 5)], + }, + None, + ); + let result = CommitBuilder::new(Arc::new(dataset)).execute(txn2).await; + + assert!( + matches!(result, Err(crate::Error::CommitConflict { .. })), + "Expected non-retryable CommitConflict for lower generation, got {:?}", + result + ); + } + + /// Test that UpdateMemWalState with equal generation as committed fails without retry. + #[tokio::test] + async fn test_update_mem_wal_state_conflict_equal_generation_no_retry() { + let dataset = test_dataset().await; + let region = Uuid::new_v4(); + + // First commit UpdateMemWalState with generation 10 + let txn1 = Transaction::new( + dataset.manifest.version, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + let dataset = CommitBuilder::new(Arc::new(dataset)) + .execute(txn1) + .await + .unwrap(); + + // Try to commit UpdateMemWalState with generation 10 (equal) + let txn2 = Transaction::new( + dataset.manifest.version - 1, // Based on old version + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + let result = CommitBuilder::new(Arc::new(dataset)).execute(txn2).await; + + assert!( + matches!(result, Err(crate::Error::CommitConflict { .. })), + "Expected non-retryable CommitConflict for equal generation, got {:?}", + result + ); + } + + /// Test that UpdateMemWalState with higher generation than committed is retryable. + /// Per spec: If committed_generation < to_commit_generation, retry is allowed. + #[tokio::test] + async fn test_update_mem_wal_state_conflict_higher_generation_retryable() { + let dataset = test_dataset().await; + let region = Uuid::new_v4(); + + // First commit UpdateMemWalState with generation 5 + let txn1 = Transaction::new( + dataset.manifest.version, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 5)], + }, + None, + ); + let dataset = CommitBuilder::new(Arc::new(dataset)) + .execute(txn1) + .await + .unwrap(); + + // Try to commit UpdateMemWalState with generation 10 (higher than 5) + // This should fail with retryable conflict + let txn2 = Transaction::new( + dataset.manifest.version - 1, // Based on old version + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + let result = CommitBuilder::new(Arc::new(dataset)).execute(txn2).await; + + assert!( + matches!(result, Err(crate::Error::RetryableCommitConflict { .. })), + "Expected retryable conflict for higher generation, got {:?}", + result + ); + } + + /// Test that UpdateMemWalState on different regions don't conflict. + #[tokio::test] + async fn test_update_mem_wal_state_different_regions_no_conflict() { + let dataset = test_dataset().await; + let region1 = Uuid::new_v4(); + let region2 = Uuid::new_v4(); + + // First commit UpdateMemWalState for region1 + let txn1 = Transaction::new( + dataset.manifest.version, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region1, 10)], + }, + None, + ); + let dataset = CommitBuilder::new(Arc::new(dataset)) + .execute(txn1) + .await + .unwrap(); + + // Commit UpdateMemWalState for region2 based on old version + // This should succeed because different regions don't conflict + let txn2 = Transaction::new( + dataset.manifest.version - 1, // Based on old version + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region2, 5)], + }, + None, + ); + let result = CommitBuilder::new(Arc::new(dataset)).execute(txn2).await; + + assert!( + result.is_ok(), + "Expected success for different regions, got {:?}", + result + ); + + // Verify both regions are in the index + let dataset = result.unwrap(); + let mem_wal_idx = dataset + .load_indices() + .await + .unwrap() + .iter() + .find(|idx| idx.name == MEM_WAL_INDEX_NAME) + .unwrap() + .clone(); + let details = load_mem_wal_index_details(mem_wal_idx).unwrap(); + assert_eq!(details.merged_generations.len(), 2); + } + + /// Test that CreateIndex of MemWalIndex can be rebased against UpdateMemWalState. + /// The merged_generations from UpdateMemWalState should be merged into CreateIndex. + #[tokio::test] + async fn test_create_index_rebase_against_update_mem_wal_state() { + let dataset = test_dataset().await; + let region = Uuid::new_v4(); + + // First commit UpdateMemWalState with generation 10 + let txn1 = Transaction::new( + dataset.manifest.version, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + let dataset = CommitBuilder::new(Arc::new(dataset)) + .execute(txn1) + .await + .unwrap(); + + // CreateIndex of MemWalIndex based on old version (before UpdateMemWalState) + // This should succeed and merge the generations + let details = MemWalIndexDetails { + num_regions: 1, + ..Default::default() + }; + let mem_wal_index = new_mem_wal_index_meta(dataset.manifest.version - 1, details).unwrap(); + + let txn2 = Transaction::new( + dataset.manifest.version - 1, // Based on old version + Operation::CreateIndex { + new_indices: vec![mem_wal_index], + removed_indices: vec![], + }, + None, + ); + let result = CommitBuilder::new(Arc::new(dataset)).execute(txn2).await; + + assert!( + result.is_ok(), + "Expected CreateIndex to succeed with rebase, got {:?}", + result + ); + + // Verify the merged_generations from UpdateMemWalState were merged into CreateIndex + let dataset = result.unwrap(); + let mem_wal_idx = dataset + .load_indices() + .await + .unwrap() + .iter() + .find(|idx| idx.name == MEM_WAL_INDEX_NAME) + .unwrap() + .clone(); + let details = load_mem_wal_index_details(mem_wal_idx).unwrap(); + assert_eq!(details.merged_generations.len(), 1); + assert_eq!(details.merged_generations[0].region_id, region); + assert_eq!(details.merged_generations[0].generation, 10); + assert_eq!(details.num_regions, 1); // Config from CreateIndex preserved + } + + /// Test that UpdateMemWalState against CreateIndex of MemWalIndex checks generations. + #[tokio::test] + async fn test_update_mem_wal_state_against_create_index_lower_generation() { + let dataset = test_dataset().await; + let region = Uuid::new_v4(); + + // First commit CreateIndex of MemWalIndex with merged_generations + let details = MemWalIndexDetails { + merged_generations: vec![MergedGeneration::new(region, 10)], + ..Default::default() + }; + let mem_wal_index = new_mem_wal_index_meta(dataset.manifest.version, details).unwrap(); + + let txn1 = Transaction::new( + dataset.manifest.version, + Operation::CreateIndex { + new_indices: vec![mem_wal_index], + removed_indices: vec![], + }, + None, + ); + let dataset = CommitBuilder::new(Arc::new(dataset)) + .execute(txn1) + .await + .unwrap(); + + // Try UpdateMemWalState with lower generation + let txn2 = Transaction::new( + dataset.manifest.version - 1, // Based on old version + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 5)], + }, + None, + ); + let result = CommitBuilder::new(Arc::new(dataset)).execute(txn2).await; + + assert!( + matches!(result, Err(crate::Error::CommitConflict { .. })), + "Expected non-retryable CommitConflict when UpdateMemWalState generation is lower than CreateIndex, got {:?}", + result + ); + } #[test] fn test_update_merged_generations() { let mut indices = Vec::new(); - let region1 = RegionId::new(); - let region2 = RegionId::new(); + let region1 = Uuid::new_v4(); + let region2 = Uuid::new_v4(); // First update - creates new index update_mem_wal_index_merged_generations( &mut indices, 1, - vec![MergedGeneration::new(region1.clone(), 5)], + vec![MergedGeneration::new(region1, 5)], ) .unwrap(); @@ -150,7 +448,7 @@ mod tests { update_mem_wal_index_merged_generations( &mut indices, 2, - vec![MergedGeneration::new(region1.clone(), 10)], + vec![MergedGeneration::new(region1, 10)], ) .unwrap(); @@ -175,7 +473,7 @@ mod tests { update_mem_wal_index_merged_generations( &mut indices, 4, - vec![MergedGeneration::new(region1.clone(), 8)], // lower than 10 + vec![MergedGeneration::new(region1, 8)], // lower than 10 ) .unwrap(); diff --git a/rust/lance/src/io/commit/conflict_resolver.rs b/rust/lance/src/io/commit/conflict_resolver.rs index c09c5dbcd90..4e55bf674a5 100644 --- a/rust/lance/src/io/commit/conflict_resolver.rs +++ b/rust/lance/src/io/commit/conflict_resolver.rs @@ -2,6 +2,7 @@ // SPDX-FileCopyrightText: Copyright The Lance Authors use crate::index::frag_reuse::{build_frag_reuse_index_metadata, load_frag_reuse_index_details}; +use crate::index::mem_wal::{load_mem_wal_index_details, new_mem_wal_index_meta}; use crate::io::deletion::read_dataset_deletion_file; use crate::{ dataset::transaction::{Operation, Transaction}, @@ -13,7 +14,7 @@ use lance_core::{ Error, Result, }; use lance_index::frag_reuse::FRAG_REUSE_INDEX_NAME; -use lance_index::mem_wal::MergedGeneration; +use lance_index::mem_wal::{MergedGeneration, MEM_WAL_INDEX_NAME}; use lance_table::format::IndexMetadata; use lance_table::{format::Fragment, io::deletion::write_deletion_file}; use snafu::{location, Location}; @@ -33,6 +34,9 @@ pub struct TransactionRebase<'a> { modified_fragment_ids: HashSet, affected_rows: Option<&'a RowAddrTreeMap>, conflicting_frag_reuse_indices: Vec, + /// Merged generations from conflicting UpdateMemWalState transactions. + /// Used when rebasing CreateIndex of MemWalIndex. + conflicting_mem_wal_merged_gens: Vec, } impl<'a> TransactionRebase<'a> { @@ -58,6 +62,7 @@ impl<'a> TransactionRebase<'a> { initial_fragments: HashMap::new(), modified_fragment_ids: HashSet::new(), conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }), Operation::Delete { updated_fragments, @@ -85,6 +90,7 @@ impl<'a> TransactionRebase<'a> { modified_fragment_ids, affected_rows: None, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }); } @@ -97,6 +103,7 @@ impl<'a> TransactionRebase<'a> { initial_fragments, modified_fragment_ids, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }) } Operation::Rewrite { groups, .. } => { @@ -114,6 +121,7 @@ impl<'a> TransactionRebase<'a> { initial_fragments, modified_fragment_ids, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }) } Operation::DataReplacement { replacements } => { @@ -128,6 +136,7 @@ impl<'a> TransactionRebase<'a> { initial_fragments, modified_fragment_ids, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }) } Operation::Merge { fragments, .. } => { @@ -141,6 +150,7 @@ impl<'a> TransactionRebase<'a> { initial_fragments, modified_fragment_ids, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }) } } @@ -545,17 +555,25 @@ impl<'a> TransactionRebase<'a> { | Operation::Clone { .. } | Operation::UpdateBases { .. } => Ok(()), // Indices are identified by UUIDs, so they shouldn't conflict. - // unless it is the same frag reuse index + // unless it is the same frag reuse index or MemWAL index Operation::CreateIndex { new_indices: created_indices, .. } => { - if new_indices + let self_has_frag_reuse = new_indices .iter() - .any(|idx| idx.name == FRAG_REUSE_INDEX_NAME) - && created_indices - .iter() - .any(|idx| idx.name == FRAG_REUSE_INDEX_NAME) + .any(|idx| idx.name == FRAG_REUSE_INDEX_NAME); + let other_has_frag_reuse = created_indices + .iter() + .any(|idx| idx.name == FRAG_REUSE_INDEX_NAME); + let self_has_mem_wal = + new_indices.iter().any(|idx| idx.name == MEM_WAL_INDEX_NAME); + let other_has_mem_wal = created_indices + .iter() + .any(|idx| idx.name == MEM_WAL_INDEX_NAME); + + if (self_has_frag_reuse && other_has_frag_reuse) + || (self_has_mem_wal && other_has_mem_wal) { Err(self.retryable_conflict_err( other_transaction, @@ -657,13 +675,27 @@ impl<'a> TransactionRebase<'a> { } Ok(()) } - Operation::Overwrite { .. } - | Operation::Restore { .. } - | Operation::UpdateMemWalState { .. } => Err(self.incompatible_conflict_err( - other_transaction, - other_version, - location!(), - )), + Operation::UpdateMemWalState { + merged_generations: other_merged_gens, + } => { + // CreateIndex of MemWalIndex is compatible with UpdateMemWalState + // as they can be rebased on each other + if new_indices.iter().any(|idx| idx.name == MEM_WAL_INDEX_NAME) { + // Collect merged_generations from UpdateMemWalState for rebasing + self.conflicting_mem_wal_merged_gens + .extend(other_merged_gens.iter().cloned()); + Ok(()) + } else { + Err(self.incompatible_conflict_err( + other_transaction, + other_version, + location!(), + )) + } + } + Operation::Overwrite { .. } | Operation::Restore { .. } => Err( + self.incompatible_conflict_err(other_transaction, other_version, location!()) + ), } } else { Err(wrong_operation_err(&self.transaction.operation)) @@ -1226,9 +1258,28 @@ impl<'a> TransactionRebase<'a> { other_version, ) } + Operation::CreateIndex { new_indices, .. } => { + // Check if CreateIndex has a MemWalIndex with merged_generations + if let Some(mem_wal_idx) = new_indices + .iter() + .find(|idx| idx.name == MEM_WAL_INDEX_NAME) + { + if let Ok(details) = load_mem_wal_index_details(mem_wal_idx.clone()) { + self.check_merged_generations_conflict( + &details.merged_generations, + self_merged_generations, + other_transaction, + other_version, + ) + } else { + Ok(()) + } + } else { + Ok(()) + } + } Operation::UpdateConfig { .. } | Operation::Rewrite { .. } - | Operation::CreateIndex { .. } | Operation::ReserveFragments { .. } | Operation::UpdateBases { .. } => Ok(()), Operation::Append { .. } @@ -1312,12 +1363,22 @@ impl<'a> TransactionRebase<'a> { for committed_mg in committed { for to_commit_mg in to_commit { if committed_mg.region_id == to_commit_mg.region_id { - // Same region being updated - this is a conflict - return Err(self.retryable_conflict_err( - other_transaction, - other_version, - location!(), - )); + // Same region being updated + // If committed >= to_commit, data already merged or superseded - abort without retry + // If committed < to_commit, can retry with new state + if committed_mg.generation >= to_commit_mg.generation { + return Err(self.incompatible_conflict_err( + other_transaction, + other_version, + location!(), + )); + } else { + return Err(self.retryable_conflict_err( + other_transaction, + other_version, + location!(), + )); + } } } } @@ -1516,57 +1577,86 @@ impl<'a> TransactionRebase<'a> { async fn finish_create_index(mut self, dataset: &Dataset) -> Result { if let Operation::CreateIndex { new_indices, .. } = &mut self.transaction.operation { - if !new_indices + // Handle FRAG_REUSE_INDEX rebasing + let has_frag_reuse = new_indices .iter() - .any(|idx| idx.name == FRAG_REUSE_INDEX_NAME) - { - return Ok(self.transaction); - } - - if self.conflicting_frag_reuse_indices.is_empty() { - return Ok(self.transaction); - } + .any(|idx| idx.name == FRAG_REUSE_INDEX_NAME); + + if has_frag_reuse && !self.conflicting_frag_reuse_indices.is_empty() { + // had at least 1 previous rewrite conflict + // get the max reuse version from each run to be added to the cleaned up index + let mut max_versions = + Vec::with_capacity(self.conflicting_frag_reuse_indices.len()); + for committed_fri in &self.conflicting_frag_reuse_indices { + let committed_fri_details = Arc::try_unwrap( + load_frag_reuse_index_details(dataset, committed_fri) + .await + .unwrap(), + ) + .unwrap(); + let max_version = committed_fri_details + .versions + .into_iter() + .max_by_key(|v| v.dataset_version) + .unwrap(); + max_versions.push(max_version); + } - // had at least 1 previous rewrite conflict - // get the max reuse version from each run to be added to the cleaned up index - let mut max_versions = Vec::with_capacity(self.conflicting_frag_reuse_indices.len()); - for committed_fri in &self.conflicting_frag_reuse_indices { - let committed_fri_details = Arc::try_unwrap( - load_frag_reuse_index_details(dataset, committed_fri) + // there should be only 1 frag_reuse_index in new indices + let new_fri = &new_indices[0]; + let mut new_fri_details = Arc::try_unwrap( + load_frag_reuse_index_details(dataset, new_fri) .await .unwrap(), ) .unwrap(); - let max_version = committed_fri_details - .versions - .into_iter() - .max_by_key(|v| v.dataset_version) - .unwrap(); - max_versions.push(max_version); + new_fri_details.versions.extend(max_versions); + + let new_frag_bitmap = new_fri_details.new_frag_bitmap(); + + let new_frag_reuse_index_meta = build_frag_reuse_index_metadata( + dataset, + Some(new_fri), + new_fri_details, + new_frag_bitmap, + ) + .await?; + + new_indices.retain(|idx| idx.name != FRAG_REUSE_INDEX_NAME); + new_indices.push(new_frag_reuse_index_meta); } - // there should be only 1 frag_reuse_index in new indices - let new_fri = &new_indices[0]; - let mut new_fri_details = Arc::try_unwrap( - load_frag_reuse_index_details(dataset, new_fri) - .await - .unwrap(), - ) - .unwrap(); - new_fri_details.versions.extend(max_versions); + // Handle MEM_WAL_INDEX rebasing + let has_mem_wal = new_indices.iter().any(|idx| idx.name == MEM_WAL_INDEX_NAME); + + if has_mem_wal && !self.conflicting_mem_wal_merged_gens.is_empty() { + let pos = new_indices + .iter() + .position(|idx| idx.name == MEM_WAL_INDEX_NAME) + .unwrap(); - let new_frag_bitmap = new_fri_details.new_frag_bitmap(); + let current_meta = new_indices.remove(pos); + let mut details = load_mem_wal_index_details(current_meta)?; - let new_frag_reuse_index_meta = build_frag_reuse_index_metadata( - dataset, - Some(new_fri), - new_fri_details, - new_frag_bitmap, - ) - .await?; + // Merge conflicting merged_generations - for each region, keep higher generation + for new_mg in &self.conflicting_mem_wal_merged_gens { + if let Some(existing) = details + .merged_generations + .iter_mut() + .find(|mg| mg.region_id == new_mg.region_id) + { + if new_mg.generation > existing.generation { + existing.generation = new_mg.generation; + } + } else { + details.merged_generations.push(new_mg.clone()); + } + } + + let new_meta = new_mem_wal_index_meta(dataset.manifest.version, details)?; + new_indices.push(new_meta); + } - new_indices.retain(|idx| idx.name != FRAG_REUSE_INDEX_NAME); - new_indices.push(new_frag_reuse_index_meta); Ok(self.transaction) } else { Err(wrong_operation_err(&self.transaction.operation)) @@ -1701,6 +1791,7 @@ mod tests { use lance_core::Error; use lance_file::version::LanceFileVersion; use lance_io::assert_io_eq; + use uuid::Uuid; use lance_table::format::IndexMetadata; use lance_table::io::deletion::{deletion_file_path, read_deletion_file}; @@ -2622,6 +2713,7 @@ mod tests { modified_fragment_ids: modified_fragment_ids(operation).collect::>(), affected_rows: None, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }; for (other, expected_conflict) in other_transactions.iter().zip(expected_conflicts) { @@ -3142,6 +3234,7 @@ mod tests { modified_fragment_ids: modified_fragment_ids(&op1).collect::>(), affected_rows: None, conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), }; let result = rebase.check_txn(&txn2, 1); @@ -3173,4 +3266,287 @@ mod tests { } } } + + #[test] + fn test_merged_generations_conflict_lower_generation_fails() { + // Test: committed generation >= to_commit generation should be incompatible (no retry) + let region = Uuid::new_v4(); + + // Committed has generation 10, we're trying to commit generation 5 + let committed_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + + let to_commit_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 5)], + }, + None, + ); + + let mut rebase = TransactionRebase { + transaction: to_commit_txn, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + let result = rebase.check_txn(&committed_txn, 1); + assert!( + matches!(result, Err(Error::CommitConflict { .. })), + "Expected non-retryable CommitConflict for lower generation, got {:?}", + result + ); + } + + #[test] + fn test_merged_generations_conflict_equal_generation_fails() { + // Test: committed generation == to_commit generation should be incompatible (no retry) + let region = Uuid::new_v4(); + + let committed_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + + let to_commit_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + + let mut rebase = TransactionRebase { + transaction: to_commit_txn, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + let result = rebase.check_txn(&committed_txn, 1); + assert!( + matches!(result, Err(Error::CommitConflict { .. })), + "Expected non-retryable CommitConflict for equal generation, got {:?}", + result + ); + } + + #[test] + fn test_merged_generations_conflict_higher_generation_retryable() { + // Test: committed generation < to_commit generation should be retryable + let region = Uuid::new_v4(); + + // Committed has generation 5, we're trying to commit generation 10 + let committed_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 5)], + }, + None, + ); + + let to_commit_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + + let mut rebase = TransactionRebase { + transaction: to_commit_txn, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + let result = rebase.check_txn(&committed_txn, 1); + assert!( + matches!(result, Err(Error::RetryableCommitConflict { .. })), + "Expected retryable conflict for higher generation, got {:?}", + result + ); + } + + #[test] + fn test_merged_generations_different_regions_ok() { + // Test: different regions should not conflict + let region1 = Uuid::new_v4(); + let region2 = Uuid::new_v4(); + + let committed_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region1, 10)], + }, + None, + ); + + let to_commit_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region2, 5)], + }, + None, + ); + + let mut rebase = TransactionRebase { + transaction: to_commit_txn, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + let result = rebase.check_txn(&committed_txn, 1); + assert!( + result.is_ok(), + "Expected OK for different regions, got {:?}", + result + ); + } + + #[test] + fn test_update_mem_wal_state_vs_create_index_with_merged_generations() { + use crate::index::mem_wal::new_mem_wal_index_meta; + use lance_index::mem_wal::MemWalIndexDetails; + + let region = Uuid::new_v4(); + + // Create a MemWalIndex with merged_generations + let details = MemWalIndexDetails { + merged_generations: vec![MergedGeneration::new(region, 10)], + ..Default::default() + }; + let mem_wal_index = new_mem_wal_index_meta(1, details).unwrap(); + + // CreateIndex with MemWalIndex that has generation 10 + let committed_txn = Transaction::new( + 0, + Operation::CreateIndex { + new_indices: vec![mem_wal_index], + removed_indices: vec![], + }, + None, + ); + + // UpdateMemWalState trying to set generation 5 (lower than committed) + let to_commit_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 5)], + }, + None, + ); + + let mut rebase = TransactionRebase { + transaction: to_commit_txn, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + let result = rebase.check_txn(&committed_txn, 1); + assert!( + matches!(result, Err(Error::CommitConflict { .. })), + "Expected non-retryable CommitConflict when UpdateMemWalState generation is lower than CreateIndex, got {:?}", + result + ); + + // Now test with higher generation (should be retryable) + let to_commit_txn_higher = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 15)], + }, + None, + ); + + let mut rebase_higher = TransactionRebase { + transaction: to_commit_txn_higher, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + let result_higher = rebase_higher.check_txn(&committed_txn, 1); + assert!( + matches!(result_higher, Err(Error::RetryableCommitConflict { .. })), + "Expected retryable conflict when UpdateMemWalState generation is higher than CreateIndex, got {:?}", + result_higher + ); + } + + #[test] + fn test_create_index_vs_update_mem_wal_state_rebase() { + use crate::index::mem_wal::new_mem_wal_index_meta; + use lance_index::mem_wal::MemWalIndexDetails; + + let region = Uuid::new_v4(); + + // CreateIndex with MemWalIndex (no merged_generations initially) + let details = MemWalIndexDetails::default(); + let mem_wal_index = new_mem_wal_index_meta(1, details).unwrap(); + + let to_commit_txn = Transaction::new( + 0, + Operation::CreateIndex { + new_indices: vec![mem_wal_index], + removed_indices: vec![], + }, + None, + ); + + // UpdateMemWalState with generation 10 + let committed_txn = Transaction::new( + 0, + Operation::UpdateMemWalState { + merged_generations: vec![MergedGeneration::new(region, 10)], + }, + None, + ); + + let mut rebase = TransactionRebase { + transaction: to_commit_txn, + initial_fragments: HashMap::new(), + modified_fragment_ids: HashSet::new(), + affected_rows: None, + conflicting_frag_reuse_indices: Vec::new(), + conflicting_mem_wal_merged_gens: Vec::new(), + }; + + // CreateIndex of MemWalIndex should be compatible with UpdateMemWalState + // and should collect the merged_generations for rebasing + let result = rebase.check_txn(&committed_txn, 1); + assert!( + result.is_ok(), + "Expected OK for CreateIndex vs UpdateMemWalState, got {:?}", + result + ); + + // Verify that merged_generations were collected + assert_eq!(rebase.conflicting_mem_wal_merged_gens.len(), 1); + assert_eq!(rebase.conflicting_mem_wal_merged_gens[0].region_id, region); + assert_eq!(rebase.conflicting_mem_wal_merged_gens[0].generation, 10); + } } From c432641be1be07bac129ed945b4881df6559c7ec Mon Sep 17 00:00:00 2001 From: Heng Ge Date: Tue, 13 Jan 2026 01:07:15 -0800 Subject: [PATCH 6/9] fix: propagate errors and avoid unnecessary clones in conflict resolver - Propagate load_mem_wal_index_details errors instead of silently returning Ok - Consume conflicting_mem_wal_merged_gens directly to avoid clone Co-Authored-By: Jack Ye Co-Authored-By: Claude --- rust/lance/src/io/commit/conflict_resolver.rs | 22 +++++++++---------- 1 file changed, 10 insertions(+), 12 deletions(-) diff --git a/rust/lance/src/io/commit/conflict_resolver.rs b/rust/lance/src/io/commit/conflict_resolver.rs index 4e55bf674a5..f65d7047923 100644 --- a/rust/lance/src/io/commit/conflict_resolver.rs +++ b/rust/lance/src/io/commit/conflict_resolver.rs @@ -1264,16 +1264,13 @@ impl<'a> TransactionRebase<'a> { .iter() .find(|idx| idx.name == MEM_WAL_INDEX_NAME) { - if let Ok(details) = load_mem_wal_index_details(mem_wal_idx.clone()) { - self.check_merged_generations_conflict( - &details.merged_generations, - self_merged_generations, - other_transaction, - other_version, - ) - } else { - Ok(()) - } + let details = load_mem_wal_index_details(mem_wal_idx.clone())?; + self.check_merged_generations_conflict( + &details.merged_generations, + self_merged_generations, + other_transaction, + other_version, + ) } else { Ok(()) } @@ -1639,7 +1636,8 @@ impl<'a> TransactionRebase<'a> { let mut details = load_mem_wal_index_details(current_meta)?; // Merge conflicting merged_generations - for each region, keep higher generation - for new_mg in &self.conflicting_mem_wal_merged_gens { + // We own self so we can consume conflicting_mem_wal_merged_gens directly + for new_mg in self.conflicting_mem_wal_merged_gens { if let Some(existing) = details .merged_generations .iter_mut() @@ -1649,7 +1647,7 @@ impl<'a> TransactionRebase<'a> { existing.generation = new_mg.generation; } } else { - details.merged_generations.push(new_mg.clone()); + details.merged_generations.push(new_mg); } } From 8ac75e3f3144d4ea544b3b2ad7476ed2f5640a8f Mon Sep 17 00:00:00 2001 From: Heng Ge Date: Tue, 13 Jan 2026 10:23:34 -0800 Subject: [PATCH 7/9] fix: update Python and Java bindings for merged_generations field rename Update Operation::Update field from mem_wal_to_merge to merged_generations. Co-Authored-By: Jack Ye Co-Authored-By: Claude --- java/lance-jni/src/transaction.rs | 4 ++-- python/src/transaction.rs | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/java/lance-jni/src/transaction.rs b/java/lance-jni/src/transaction.rs index c80ef36bffa..59ca0d36eff 100644 --- a/java/lance-jni/src/transaction.rs +++ b/java/lance-jni/src/transaction.rs @@ -559,7 +559,7 @@ fn convert_to_java_operation_inner<'local>( updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge: _, + merged_generations: _, fields_for_preserving_frag_bitmap, update_mode, inserted_rows_filter: _, @@ -1048,7 +1048,7 @@ fn convert_to_rust_operation( updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge: None, + merged_generations: vec![], fields_for_preserving_frag_bitmap, update_mode, inserted_rows_filter: None, diff --git a/python/src/transaction.rs b/python/src/transaction.rs index b867b428352..4f57bf3dd49 100644 --- a/python/src/transaction.rs +++ b/python/src/transaction.rs @@ -229,7 +229,7 @@ impl FromPyObject<'_> for PyLance { updated_fragments, new_fragments, fields_modified, - mem_wal_to_merge: None, + merged_generations: vec![], fields_for_preserving_frag_bitmap, update_mode, inserted_rows_filter: None, From 27b6734a3a91f043652b9047badd18ee2849303e Mon Sep 17 00:00:00 2001 From: Heng Ge Date: Tue, 13 Jan 2026 11:54:15 -0800 Subject: [PATCH 8/9] feat: add index_catchup tracking for async index rebuilding - Add IndexCatchupProgress to track which merged generation each base table index has caught up to - Update MemWalIndexDetails proto and Rust implementation - Remove merged_generation from RegionManifest (single source of truth) - Add LSM Tree Merging section and Appendix 4 to spec - Improve Indexed Read documentation with better explanation Co-Authored-By: Jack Ye Co-Authored-By: Claude Opus 4.5 --- docs/src/format/table/mem_wal.md | 289 +++++++++++++++++++++---------- protos/table.proto | 28 ++- rust/lance-index/src/mem_wal.rs | 86 ++++++++- 3 files changed, 306 insertions(+), 97 deletions(-) diff --git a/docs/src/format/table/mem_wal.md b/docs/src/format/table/mem_wal.md index 24d583021b0..b61343674c1 100644 --- a/docs/src/format/table/mem_wal.md +++ b/docs/src/format/table/mem_wal.md @@ -1,6 +1,7 @@ # MemTable & WAL Specification (Experimental) -Lance MemTable & WAL (MemWAL) specification describes a Log-Structured-Merge (LSM) tree architecture for Lance tables, enabling high-performance streaming write workloads. +Lance MemTable & WAL (MemWAL) specification describes a Log-Structured-Merge (LSM) tree architecture for Lance tables, enabling high-performance streaming write workloads while maintaining indexed read performance for key workloads including +scan, point lookup, vector search and full-text search. !!!note MemWAL requires the table to have an [unenforced primary key](index.md#unenforced-primary-key) defined. @@ -41,6 +42,7 @@ It stores: - **Configuration**: Region specs defining how rows map to regions, and which indexes to maintain - **Merge progress**: Last generation merged to base table for each region +- **Index catchup progress**: Which merged generation each base table index has been rebuilt to cover - **Region snapshots**: Point-in-time snapshot of all region states for read optimization The index is the source of truth for **configuration** and **merge progress**, but region state snapshots are for read-only optimization (each region's manifest is authoritative for its own state). @@ -118,7 +120,7 @@ The manifest contains: - **Fencing state**: `writer_epoch` (writer fencing token) - **WAL pointers**: `replay_after_wal_id` (last entry flushed to MemTable), `wal_id_last_seen` (last entry seen at manifest update) -- **Generation trackers**: `current_generation` (next generation to flush), `merged_generation` (last generation merged to base) +- **Generation trackers**: `current_generation` (next generation to flush) - **Flushed generations**: `flushed_generations` list of generation number and directory path pairs (e.g., generation 1 at `a1b2c3d4_gen_1`) Note: `wal_id_last_seen` is a hint that may be stale since it's not updated on WAL write. @@ -153,7 +155,7 @@ To read the latest manifest version: This approach uses HEAD requests instead of LIST operations in cloud storage, which is generally faster and is friendly to systems like S3 Express that do not support lexicographically sorted listing. !!!note - This works because the write rate to region manifests is significantly lower than read rates. Region manifests are only updated when region metadata changes (MemTable flush, merge completion), not on every write. This ensures HEAD requests will eventually terminate and find the latest version. + This works because the write rate to region manifests is significantly lower than read rates. Region manifests are only updated when region metadata changes (MemTable flush), not on every write. This ensures HEAD requests will eventually terminate and find the latest version. All region manifest versions are stored in `_mem_wal/{region_id}/manifest` directory. @@ -166,8 +168,8 @@ The region manifest is updated atomically in the following cases: |---------|----------------|---------| | [Initialization & Recovery](#initialization--recovery) | `writer_epoch` | Incremented when writer claims the region | | [MemTable Flush](#memtable-flush) | `replay_after_wal_id`, `wal_id_last_seen`, `current_generation`, `flushed_generations` | After flushing MemTable to storage | -| [MemTable Merger](#merge-workflow) | `merged_generation`, `flushed_generations` | After merging a flushed MemTable; removes merged entry | | [MemWAL Index Builder](#memwal-index-builder) | `wal_id_last_seen` | Periodically scans WAL entries and updates hint | +| [Garbage Collector](#garbage-collector) | `flushed_generations` | Removes entries for deleted flushed MemTables | !!!note WAL flush does **not** update the manifest to keep the hot write path fast. @@ -175,33 +177,6 @@ The region manifest is updated atomically in the following cases: Writers use epoch-based fencing (`writer_epoch`) to ensure single-writer semantics. See [Writer Fencing](#writer-fencing) for details. -## Storage Layout - -Here is a recap of the storage layout with all the files and concepts defined so far: - -``` -{table_path}/ -├── _indices/ -│ └── {index_uuid}/ # MemWAL Index (uses standard index storage) -│ └── index.lance # Serialized region snapshots (Lance file) -│ -└── _mem_wal/ - └── {region_uuid}/ # Region directory (UUID v4) - ├── manifest/ - │ ├── {bit_reversed_version}.binpb # Serialized region manifest (bit-reversed naming) - │ └── version_hint.json # Version hint file - ├── wal/ - │ ├── {bit_reversed_entry_id}.lance # WAL data files (bit-reversed naming) - │ └── ... - └── {random_hash}_gen_{i}/ # Flushed MemTable (generation i, random prefix) - ├── _versions/ - │ └── {version}.manifest # Table manifest (V2 naming scheme) - ├── _indices/ # Indexes - │ ├── {vector_index}/ - │ └── {scalar_index}/ - └── bloom_filter.bin # Primary key bloom filter -``` - ## MemWAL Index Details The MemWAL Index uses the [standard index storage](index/index.md#index-storage) at `_indices/{UUID}/`. @@ -222,6 +197,11 @@ Writers read these fields to determine how to partition data and which indexes t This field is updated atomically with merge-insert data commits, enabling conflict resolution when multiple mergers operate concurrently. Each entry contains the region UUID and generation number. +**Index catchup progress** (`index_catchup`) tracks which merged generation each base table index has been rebuilt to cover. +When data is merged from a flushed MemTable to the base table, the base table's indexes are rebuilt asynchronously. +During this window, queries should use the flushed MemTable's pre-built indexes instead of scanning unindexed data in the base table. +See [Index Catchup and Read Path](#index-catchup-and-read-path) for details. + **Region snapshot fields** (`snapshot_ts_millis`, `num_regions`, `inline_snapshots`) provide a point-in-time snapshot of region states. The actual region manifests remain authoritative for region state. When `num_regions` is 0, the `inline_snapshots` field may be `None` or an empty Lance file with 0 rows but proper schema. @@ -332,7 +312,6 @@ The schema has one column per `RegionManifest` field plus region spec columns: | `replay_after_wal_id` | `uint64` | Last WAL entry flushed to MemTable | | `wal_id_last_seen` | `uint64` | Last WAL entry seen (hint) | | `current_generation` | `uint64` | Next generation to flush | -| `merged_generation` | `uint64` | Last generation merged to base | | `flushed_generations` | `list>` | Flushed MemTable paths | | `region_field_{field_id}` | varies | Region field value (one column per field in region spec) | @@ -357,6 +336,33 @@ This ensures distances are precise and comparable across generations. The base table vector index should not change the codebook once MemWAL is enabled. To switch codebooks, a migration is required: create another vector index with the new codebook, configure MemTable to maintain both indexes, and eventually drop the old index after all readers are using the new codebook and all MemTables have indexes using the new codebook. +## Storage Layout + +Here is a recap of the storage layout with all the files and concepts defined so far: + +``` +{table_path}/ +├── _indices/ +│ └── {index_uuid}/ # MemWAL Index (uses standard index storage) +│ └── index.lance # Serialized region snapshots (Lance file) +│ +└── _mem_wal/ + └── {region_uuid}/ # Region directory (UUID v4) + ├── manifest/ + │ ├── {bit_reversed_version}.binpb # Serialized region manifest (bit-reversed naming) + │ └── version_hint.json # Version hint file + ├── wal/ + │ ├── {bit_reversed_entry_id}.lance # WAL data files (bit-reversed naming) + │ └── ... + └── {random_hash}_gen_{i}/ # Flushed MemTable (generation i, random prefix) + ├── _versions/ + │ └── {version}.manifest # Table manifest (V2 naming scheme) + ├── _indices/ # Indexes + │ ├── {vector_index}/ + │ └── {scalar_index}/ + └── bloom_filter.bin # Primary key bloom filter +``` + ## Writer Expectations A writer operates on a single region within a single process and may spawn asynchronous tasks for background operations like WAL flush and MemTable flush. @@ -527,16 +533,14 @@ Flushed MemTables are merged to the base table in generation order using Lance's #### Merge Workflow -1. Load the MemWAL Index and read `merged_generations[region_id]` -2. Load the region manifest and identify unmerged flushed MemTables from `flushed_generations`: those with generation numbers in range `(merged_generation, current_generation)` +1. Read `merged_generations[region_id]` +2. Load the region manifest and identify unmerged flushed MemTables from `flushed_generations`: those with generation numbers > `merged_generations[region_id]` 3. For each flushed MemTable in ascending generation order: - Look up the directory path from `flushed_generations` - Open it as a Lance table - Execute merge-insert into the base table, atomically updating the MemWAL Index: - Set `merged_generations[region_id]` to this generation - On commit conflict, apply [conflict resolution rules](#conflict-resolution-and-concurrency) - - On successful commit, update the region manifest: set `merged_generation` to this generation and remove the entry from `flushed_generations` - - If the region manifest update fails, continue to the next generation (MemWAL Index is authoritative) 4. After merge, the flushed MemTable and its referenced WAL files may be garbage collected (see [Garbage Collector](#garbage-collector)) Ordered merge ensures correct upsert semantics: flushed MemTables with higher generation numbers overwrite those with lower numbers. @@ -545,21 +549,14 @@ Ordered merge ensures correct upsert semantics: flushed MemTables with higher ge Multiple mergers may operate on the same region concurrently. This is safe due to: -1. **Atomic MemWAL Index update**: The `merged_generations` in MemWAL Index is updated atomically with the data commit -2. **Conflict resolution**: When a merge-insert commit encounters a version conflict, the merger reads the conflicting commit's MemWAL Index. If `merged_generations[region_id] >= my_generation`, abort without retry (data already merged or superseded). Otherwise, retry the commit as normal. +1. **Atomic update**: `merged_generations` is updated atomically with the data commit +2. **Conflict resolution**: When a merge-insert commit encounters a version conflict, the merger reads the conflicting commit's `merged_generations`. If `merged_generations[region_id] >= my_generation`, abort without retry (data already merged or superseded). Otherwise, retry the commit as normal. 3. **Merge-insert idempotency**: If two mergers merge the same generation before either commits, both write identical data (primary key upsert semantics) After aborting due to a conflict, reload the MemWAL Index and region manifest, then continue to the next unmerged generation. -If a merger crashes after committing to the base table but before updating the region manifest: - -- The MemWAL Index has `merged_generations[region_id] = N` -- The region manifest still has `merged_generation = N-1` -- Next merger reads MemWAL Index, sees generation N already merged, skips it -- Region manifest is eventually updated to catch up - -The MemWAL Index `merged_generations` and region manifest `merged_generation` may temporarily differ. -The MemWAL Index is authoritative for conflict resolution; the region manifest is eventually consistent and used for `flushed_generations` cleanup. +`merged_generations` is the single source of truth for merge progress. +If a merger crashes after committing, the next merger reads the MemWAL Index to determine which generations are already merged. For a concrete example, see [Appendix 2: Concurrent Merger Example](#appendix-2-concurrent-merger-example). @@ -587,23 +584,58 @@ This process serves two purposes: The build frequency is implementation-defined. More frequent builds reduce staleness but increase I/O overhead. +### Base Table Index Builder + +A background process rebuilds base table indexes to cover newly merged data and updates `index_catchup` progress in the MemWAL Index. +Typically there is a dedicated builder for each index. + +The index builder workflow is expected to be: +1. Rebuild the base table index to the latest state, this automatically covers all merged generations +2. Read the current `merged_generations` +3. Update the MemWAL Index atomically: + - Set `index_catchup[index_name].caught_up_generations` to match `merged_generations` +4. On commit conflict, reload the MemWAL Index and retry + ### Garbage Collector -The garbage collector removes obsolete data from the region directory. This is a file-only operation that does not update the region manifest. +The garbage collector removes obsolete data from the region directory and updates the region manifest to remove entries from `flushed_generations` for deleted flushed MemTables. Eligible for deletion: -1. **Flushed MemTable directories**: Generation directories where `generation <= merged_generation` +1. **Flushed MemTable directories**: Generation directories where `generation <= merged_generations[region_id]` AND `generation <= min(index_catchup[I].caught_up_generation)` for all maintained indexes 2. **WAL data files**: Files referenced only by deleted generations 3. **Old region manifest versions**: Versions older than the current version minus a retention threshold 4. **Orphaned directories**: Directories matching `*_gen_*` pattern but not in `flushed_generations` (from failed flush attempts) -**Time travel consideration**: Garbage collection must not remove generations that are reachable by any retained base table version. When a reader opens an older table version, the MemWAL Index snapshot from that version references specific `merged_generation` values. Generations that satisfy `generation > merged_generation` for any retained table version must be preserved. +**Index catchup consideration**: Flushed MemTables must be retained until all base table indexes have caught up. +Since flushed MemTables contain pre-built indexes, they are used for indexed queries when the base table index has not yet been rebuilt to cover the merged data. +Only after all indexes in `maintained_indexes` have `caught_up_generation >= generation` can a flushed MemTable be safely deleted. + +**Time travel consideration**: Garbage collection must not remove generations that are reachable by any retained base table version. When a reader opens an older table version, the MemWAL Index snapshot from that version references specific `merged_generations` values. Generations that satisfy `generation > merged_generations[region_id]` for any retained table version must be preserved. Garbage collection must verify that no flushed MemTable still references a WAL file before deletion. ## Reader Expectations +### LSM Tree Merging Read + +Readers **MUST** merge results from multiple data sources (base table, flushed MemTables, in-memory MemTables) by primary key to ensure correctness. + +When the same primary key exists in multiple sources, the reader must keep only the newest version based on: + +1. **Generation number** (`_gen`): Higher generation wins. The base table has generation -1, flushed MemTables have positive integers starting from 1. +2. **Row address** (`_rowaddr`): Within the same generation, higher row address wins (later writes within a batch overwrite earlier ones). + +The ordering for "newest" is: highest `_gen` first, then highest `_rowaddr`. + +This deduplication is essential because: + +- A row updated in a MemTable also exists (with older data) in the base table +- A flushed MemTable that has been merged to the base table may not yet be garbage collected, causing the same row to appear in both +- A single write batch may contain multiple updates to the same primary key + +Without proper merging, queries would return duplicate or stale rows. + ### Reader Consistency Reader consistency depends on two factors: @@ -614,13 +646,12 @@ Reader consistency depends on two factors: Strong consistency requires access to in-memory MemTables for all regions involved in the query and reading region manifests directly. Otherwise, the query is eventually consistent due to missing unflushed data or stale MemWAL Index snapshots. -Reading a stale MemWAL Index does not impact correctness, only freshness: - -- **Merged MemTable**: If a flushed MemTable has been merged to the base table but not yet garbage collected, readers query both. Deduplication by primary key ensures correct results since both contain the same data. If the MemTable has been garbage collected, readers fail to open it, which is also safe because the data already exists in the base table. -- **Stale snapshot**: The snapshot may not reflect the latest region manifest state. Flushed MemTables added after the snapshot was built are not queried. The result is eventually consistent but correct for the snapshot's point in time. - -The `snapshot_ts_millis` field indicates when the index was built; readers can use this to estimate staleness and decide whether to refresh. -For stronger consistency, readers may load individual region manifests directly from `_mem_wal/{region_uuid}/manifest/`. +!!!note + Reading a stale MemWAL Index does not impact correctness, only freshness: + + - **Merged MemTable still in index**: If a flushed MemTable has been merged to the base table but still shows in the MemWAL index, readers query both. This results in some inefficiency for querying the same data twice, but [LSM-tree merging](#lsm-tree-merging-read) ensures correct results since both contain the same data. The inefficiency is also compensated by the fact that the data is covered by index and we rarely end up scanning both data. + - **Garbage collected MemTable still in index**: If a flushed MemTable has been garbage collected, but is still in the MemWAL index, readers would fail to open it and skip it. This is also safe because if it is garbage collected, the data must already exist in the base table. + - **Newly flushed MemTable not in index**: If a newly flushed MemTable is added after the snapshot was built, it is not queried. The result is eventually consistent but correct for the snapshot's point in time. ### Query Planning @@ -659,6 +690,41 @@ For example, with a region spec using `bucket(user_id, 10)` and a filter `user_i Region pruning applies to both scan queries and prefilters in search queries. +#### Indexed Read + +When data is merged from a flushed MemTable to the base table, the base table's indexes are rebuilt asynchronously by the [base table index builders](#base-table-index-builder). +During this window, the merged data exists in the base table but is not yet covered by the base table's indexes. + +Without special handling, indexed queries would fall back to expensive full scans for the unindexed part of the base table. +To maintain indexed read performance, the query planner should use `index_catchup` progress to determine the optimal data source for each query: + +``` +For indexed query on region R using index I: + merged_gen = merged_generations[R] + index_gen = index_catchup[I].caught_up_generations[R] # defaults to merged_gen if absent + + if index_gen >= merged_gen: + # Base table index is caught up - optimal path + base_table.indexed_query(I) + memtable.query(gen > merged_gen) + else: + # Base table index is behind - use flushed MemTable indexes for the gap + base_table.indexed_query(I) # covers data up to index_gen + flushed_memtable.indexed_query(gen in (index_gen, merged_gen]) # uses pre-built indexes + memtable.query(gen > merged_gen) +``` + +Since flushed MemTables contain pre-built indexes (created during [MemTable flush](#memtable-flush)), queries can use these indexes instead of scanning unindexed data in the base table. +This ensures all reads remain indexed regardless of how far behind the async index builder is. + +The key insight is that flushed MemTables serve as a bridge between the base table's index catchup and the current merged state. +When `index_gen < merged_gen`, the generations in the gap `(index_gen, merged_gen]` have data in the base table but are not covered by the base table's indexes. +Instead of falling back to a full scan for this gap, the query planner uses the flushed MemTable indexes which were pre-built during flush. +This maintains indexed query performance even during periods of heavy write activity where async index rebuilding falls behind. + +See [Appendix 4: Index Catchup Example](#appendix-4-index-catchup-example) for a detailed timeline showing how this works in practice. + + ### Query Execution Query execution unions datasets within each region and deduplicates by primary key. @@ -857,7 +923,6 @@ MemWAL Index: merged_generations: {region: 5} Region manifest (version 1): - merged_generation: 5 current_generation: 8 flushed_generations: [(6, "abc123_gen_6"), (7, "def456_gen_7")] ``` @@ -866,51 +931,45 @@ Region manifest (version 1): Two mergers both try to merge generation 6 concurrently. -| Step | Merger A | Merger B | MemWAL Index | Region Manifest | -|------|----------|----------|--------------|-----------------| -| 1 | Reads index: merged_gen=5 | | merged_gen=5 | merged_gen=5 | -| 2 | Reads region manifest | | | | -| 3 | Starts merging gen 6 | | | | -| 4 | | Reads index: merged_gen=5 | merged_gen=5 | merged_gen=5 | -| 5 | | Reads region manifest | | | -| 6 | | Starts merging gen 6 | | | -| 7 | Commits (merged_gen=6) | | **merged_gen=6** | merged_gen=5 | -| 8 | | Tries to commit | | | -| 9 | | **Conflict**: reads new index | | | -| 10 | | Sees merged_gen=6 >= 6, aborts | | | -| 11 | Updates region manifest | | merged_gen=6 | **merged_gen=6** | -| 12 | | Reloads, continues to gen 7 | | | +| Step | Merger A | Merger B | MemWAL Index | +|------|----------|----------|--------------| +| 1 | Reads index: merged_gen=5 | | merged_gen=5 | +| 2 | Reads region manifest | | | +| 3 | Starts merging gen 6 | | | +| 4 | | Reads index: merged_gen=5 | merged_gen=5 | +| 5 | | Reads region manifest | | +| 6 | | Starts merging gen 6 | | +| 7 | Commits (merged_gen=6) | | **merged_gen=6** | +| 8 | | Tries to commit | | +| 9 | | **Conflict**: reads new index | | +| 10 | | Sees merged_gen=6 >= 6, aborts | | +| 11 | | Reloads, continues to gen 7 | | Merger B's conflict resolution detected that generation 6 was already merged by checking the MemWAL Index in the conflicting commit. #### Scenario 2: Crash After Table Commit -Merger A crashes after committing to the table but before updating the region manifest. +Merger A crashes after committing to the table. -| Step | Merger A | Merger B | MemWAL Index | Region Manifest | -|------|----------|----------|--------------|-----------------| -| 1 | Reads index: merged_gen=5 | | merged_gen=5 | merged_gen=5 | -| 2 | Merges gen 6, commits | | **merged_gen=6** | merged_gen=5 | -| 3 | **CRASH** before region update | | merged_gen=6 | merged_gen=5 | -| 4 | | Reads index: merged_gen=6 | merged_gen=6 | merged_gen=5 | -| 5 | | Reads region manifest | | | -| 6 | | Region says gen 6 unmerged... | | | -| 7 | | But index says merged_gen=6 | | | -| 8 | | **Skips gen 6** (index authoritative) | | | -| 9 | | Merges gen 7, commits | | **merged_gen=7** | -| 10 | | Updates region manifest | | **merged_gen=7** | +| Step | Merger A | Merger B | MemWAL Index | +|------|----------|----------|--------------| +| 1 | Reads index: merged_gen=5 | | merged_gen=5 | +| 2 | Merges gen 6, commits | | **merged_gen=6** | +| 3 | **CRASH** | | merged_gen=6 | +| 4 | | Reads index: merged_gen=6 | merged_gen=6 | +| 5 | | Reads region manifest | | +| 6 | | **Skips gen 6** (already merged) | | +| 7 | | Merges gen 7, commits | **merged_gen=7** | -The MemWAL Index is authoritative. Even though the region manifest was stale, Merger B correctly used the MemWAL Index to determine that generation 6 was already merged. +The MemWAL Index is the single source of truth. Merger B correctly used it to determine that generation 6 was already merged. #### Key Points -1. **MemWAL Index is authoritative**: The `merged_generations` in MemWAL Index is the source of truth for merge progress, updated atomically with data. - -2. **Region manifest is eventually consistent**: It may lag behind MemWAL Index after crashes, but is eventually updated by subsequent mergers. +1. **Single source of truth**: `merged_generations` is the authoritative source for merge progress, updated atomically with data. -3. **Conflict resolution uses MemWAL Index**: When a commit conflicts, the merger checks the conflicting commit's MemWAL Index, not the region manifest. +2. **Conflict resolution uses MemWAL Index**: When a commit conflicts, the merger checks the conflicting commit's MemWAL Index. -4. **No progress regression**: Because MemWAL Index is updated atomically with data, concurrent mergers cannot regress the merge progress. +3. **No progress regression**: Because MemWAL Index is updated atomically with data, concurrent mergers cannot regress the merge progress. ### Appendix 3: Execution Nodes @@ -947,3 +1006,51 @@ Given a primary key, checks the bloom filter before executing the child node. If the bloom filter returns negative (key definitely not present), returns empty without executing the child. If the bloom filter returns positive (key may be present), executes the child node normally. Used in point lookups to skip unnecessary scans of generations that don't contain the target key. + +### Appendix 4: Index Catchup Example + +This example demonstrates how `index_catchup` enables indexed reads during async index rebuilding. + +#### Scenario Setup + +``` +Generation: 1 2 3 4 5 6 + | | | | | | +State: merged merged merged merged flushed active + | | | | | | +Base IVF index: [-- covers 1-3 --] + ↑ ↑ ↑ + index_gen=3 merged_gen=4 | + current_gen=6 +``` + +In this example: + +- **Generations 1-4** have been merged to the base table (`merged_gen=4`) +- **Base IVF index** has only been rebuilt to cover generations 1-3 (`index_gen=3`) +- **Generation 4** is in the base table but NOT covered by the base IVF index +- **Generation 5** is flushed to disk (not yet merged to base table) +- **Generation 6** is the active in-memory MemTable + +#### Read Strategy for Vector Search + +Without `index_catchup` tracking, the query planner would need to perform an expensive full scan on the base table for generation 4. +With `index_catchup`, the planner knows exactly which data is indexed and can use flushed MemTable indexes for the gap: + +| Data Source | Generations | Strategy | +|-------------|-------------|----------| +| Base table with IVF index | 1-3 | Use base table's IVF index | +| Flushed MemTable gen 4 | 4 | Use flushed MemTable's IVF index | +| Flushed MemTable gen 5 | 5 | Use flushed MemTable's IVF index | +| Active MemTable | 6 | Use in-memory IVF index | + +All data sources provide indexed access, maintaining query performance during async index rebuild. + +#### Why Flushed MemTables Are Retained + +Flushed MemTable files are not garbage collected until: + +1. Their data has been merged to the base table (`gen <= merged_gen`) +2. All maintained indexes have caught up (`gen <= index_gen` for all indexes) + +This ensures flushed MemTable indexes remain available to bridge the gap between `index_gen` and `merged_gen`. diff --git a/protos/table.proto b/protos/table.proto index 71ef98066e8..0d73bfdeb94 100644 --- a/protos/table.proto +++ b/protos/table.proto @@ -538,9 +538,8 @@ message RegionManifest { // Next generation ID to create (incremented after each MemTable flush). uint64 current_generation = 6; - // Last generation merged to base table. - // Generations <= merged_generation can be garbage collected. - uint64 merged_generation = 7; + // Field 7 removed: merged_generation moved to MemWalIndexDetails.merged_generations + // which is the authoritative source for merge progress. // List of flushed MemTable generations and their directory paths. repeated FlushedGeneration flushed_generations = 8; @@ -564,6 +563,18 @@ message MergedGeneration { uint64 generation = 2; } +// Tracks which merged generation a base table index has been rebuilt to cover. +// Used to determine whether to read from flushed MemTable indexes or base table. +message IndexCatchupProgress { + // Name of the base table index (must match an entry in maintained_indexes). + string index_name = 1; + + // Per-region progress: the generation up to which this index covers. + // If a region is not present, the index is assumed to be fully caught up + // (i.e., caught_up_generation >= merged_generation for that region). + repeated MergedGeneration caught_up_generations = 2; +} + // Index details for MemWAL Index, stored in IndexMetadata.index_details. // This is the centralized structure for all MemWAL metadata: // - Configuration (region specs, indexes to maintain) @@ -620,6 +631,17 @@ message MemWalIndexDetails { // 1. merged_generations is updated by mergers (atomic with data commit) // 2. region snapshots are updated by background index builder repeated MergedGeneration merged_generations = 9; + + // Per-index catchup progress tracking. + // When data is merged to the base table, base table indexes are rebuilt + // asynchronously. This field tracks which generation each index covers. + // + // For indexed queries, if an index's caught_up_generation < merged_generation, + // readers should use flushed MemTable indexes for the gap instead of + // scanning unindexed data in the base table. + // + // If an index is not present in this list, it is assumed to be fully caught up. + repeated IndexCatchupProgress index_catchup = 10; } // Region spec definition. diff --git a/rust/lance-index/src/mem_wal.rs b/rust/lance-index/src/mem_wal.rs index ccc9cc3db0e..71bda425fd8 100644 --- a/rust/lance-index/src/mem_wal.rs +++ b/rust/lance-index/src/mem_wal.rs @@ -91,6 +91,60 @@ impl TryFrom for MergedGeneration { } } +/// Tracks which merged generation a base table index has been rebuilt to cover. +/// Used to determine whether to read from flushed MemTable indexes or base table. +#[derive(Debug, Clone, Default, PartialEq, Eq, Serialize, Deserialize, DeepSizeOf)] +pub struct IndexCatchupProgress { + pub index_name: String, + pub caught_up_generations: Vec, +} + +impl IndexCatchupProgress { + pub fn new(index_name: String, caught_up_generations: Vec) -> Self { + Self { + index_name, + caught_up_generations, + } + } + + /// Get the caught up generation for a specific region. + /// Returns None if the region is not present (assumed fully caught up). + pub fn caught_up_generation_for_region(&self, region_id: &Uuid) -> Option { + self.caught_up_generations + .iter() + .find(|mg| &mg.region_id == region_id) + .map(|mg| mg.generation) + } +} + +impl From<&IndexCatchupProgress> for pb::IndexCatchupProgress { + fn from(icp: &IndexCatchupProgress) -> Self { + Self { + index_name: icp.index_name.clone(), + caught_up_generations: icp + .caught_up_generations + .iter() + .map(|mg| mg.into()) + .collect(), + } + } +} + +impl TryFrom for IndexCatchupProgress { + type Error = Error; + + fn try_from(icp: pb::IndexCatchupProgress) -> lance_core::Result { + Ok(Self { + index_name: icp.index_name, + caught_up_generations: icp + .caught_up_generations + .into_iter() + .map(MergedGeneration::try_from) + .collect::>()?, + }) + } +} + /// Region manifest containing epoch-based fencing and WAL state. /// Each region has exactly one active writer at any time. #[derive(Debug, Clone, PartialEq, Eq, Serialize, Deserialize)] @@ -102,7 +156,6 @@ pub struct RegionManifest { pub replay_after_wal_id: u64, pub wal_id_last_seen: u64, pub current_generation: u64, - pub merged_generation: u64, pub flushed_generations: Vec, } @@ -122,7 +175,6 @@ impl From<&RegionManifest> for pb::RegionManifest { replay_after_wal_id: rm.replay_after_wal_id, wal_id_last_seen: rm.wal_id_last_seen, current_generation: rm.current_generation, - merged_generation: rm.merged_generation, flushed_generations: rm.flushed_generations.iter().map(|fg| fg.into()).collect(), } } @@ -143,7 +195,6 @@ impl TryFrom for RegionManifest { replay_after_wal_id: rm.replay_after_wal_id, wal_id_last_seen: rm.wal_id_last_seen, current_generation: rm.current_generation, - merged_generation: rm.merged_generation, flushed_generations: rm .flushed_generations .into_iter() @@ -224,6 +275,7 @@ pub struct MemWalIndexDetails { pub region_specs: Vec, pub maintained_indexes: Vec, pub merged_generations: Vec, + pub index_catchup: Vec, } impl From<&MemWalIndexDetails> for pb::MemWalIndexDetails { @@ -239,6 +291,7 @@ impl From<&MemWalIndexDetails> for pb::MemWalIndexDetails { .iter() .map(|mg| mg.into()) .collect(), + index_catchup: details.index_catchup.iter().map(|icp| icp.into()).collect(), } } } @@ -262,6 +315,11 @@ impl TryFrom for MemWalIndexDetails { .into_iter() .map(MergedGeneration::try_from) .collect::>()?, + index_catchup: details + .index_catchup + .into_iter() + .map(IndexCatchupProgress::try_from) + .collect::>()?, }) } } @@ -284,6 +342,26 @@ impl MemWalIndex { .find(|mg| &mg.region_id == region_id) .map(|mg| mg.generation) } + + /// Get the caught up generation for a specific index and region. + /// Returns None if the index is not tracked (assumed fully caught up). + pub fn index_caught_up_generation(&self, index_name: &str, region_id: &Uuid) -> Option { + self.details + .index_catchup + .iter() + .find(|icp| icp.index_name == index_name) + .and_then(|icp| icp.caught_up_generation_for_region(region_id)) + } + + /// Check if an index is fully caught up for a region. + /// Returns true if the index covers all merged data for the region. + pub fn is_index_caught_up(&self, index_name: &str, region_id: &Uuid) -> bool { + let merged_gen = self.merged_generation_for_region(region_id).unwrap_or(0); + let caught_up_gen = self.index_caught_up_generation(index_name, region_id); + + // If not tracked in index_catchup, assumed fully caught up + caught_up_gen.is_none_or(|gen| gen >= merged_gen) + } } #[derive(Serialize)] @@ -292,6 +370,7 @@ struct MemWalStatistics { num_merged_generations: usize, num_region_specs: usize, num_maintained_indexes: usize, + num_index_catchup_entries: usize, } #[async_trait] @@ -317,6 +396,7 @@ impl Index for MemWalIndex { num_merged_generations: self.details.merged_generations.len(), num_region_specs: self.details.region_specs.len(), num_maintained_indexes: self.details.maintained_indexes.len(), + num_index_catchup_entries: self.details.index_catchup.len(), }; serde_json::to_value(stats).map_err(|e| Error::Internal { message: format!("failed to serialize MemWAL index statistics: {}", e), From c7335f9989344d0f12bd8c42d1b8600cf03e8f20 Mon Sep 17 00:00:00 2001 From: Heng Ge Date: Tue, 13 Jan 2026 12:23:43 -0800 Subject: [PATCH 9/9] Fix typos --- docs/src/format/table/mem_wal.md | 84 +++++++------------------------- 1 file changed, 17 insertions(+), 67 deletions(-) diff --git a/docs/src/format/table/mem_wal.md b/docs/src/format/table/mem_wal.md index b61343674c1..74cf3ab72cb 100644 --- a/docs/src/format/table/mem_wal.md +++ b/docs/src/format/table/mem_wal.md @@ -623,7 +623,7 @@ Readers **MUST** merge results from multiple data sources (base table, flushed M When the same primary key exists in multiple sources, the reader must keep only the newest version based on: -1. **Generation number** (`_gen`): Higher generation wins. The base table has generation -1, flushed MemTables have positive integers starting from 1. +1. **Generation number** (`_gen`): Higher generation wins. The base table has generation -1, MemTables have positive integers starting from 1. 2. **Row address** (`_rowaddr`): Within the same generation, higher row address wins (later writes within a batch overwrite earlier ones). The ordering for "newest" is: highest `_gen` first, then highest `_rowaddr`. @@ -690,67 +690,27 @@ For example, with a region spec using `bucket(user_id, 10)` and a filter `user_i Region pruning applies to both scan queries and prefilters in search queries. -#### Indexed Read +#### Indexed Read Plan When data is merged from a flushed MemTable to the base table, the base table's indexes are rebuilt asynchronously by the [base table index builders](#base-table-index-builder). During this window, the merged data exists in the base table but is not yet covered by the base table's indexes. Without special handling, indexed queries would fall back to expensive full scans for the unindexed part of the base table. -To maintain indexed read performance, the query planner should use `index_catchup` progress to determine the optimal data source for each query: - -``` -For indexed query on region R using index I: - merged_gen = merged_generations[R] - index_gen = index_catchup[I].caught_up_generations[R] # defaults to merged_gen if absent - - if index_gen >= merged_gen: - # Base table index is caught up - optimal path - base_table.indexed_query(I) - memtable.query(gen > merged_gen) - else: - # Base table index is behind - use flushed MemTable indexes for the gap - base_table.indexed_query(I) # covers data up to index_gen - flushed_memtable.indexed_query(gen in (index_gen, merged_gen]) # uses pre-built indexes - memtable.query(gen > merged_gen) -``` +To maintain indexed read performance, the query planner should use `index_catchup` progress to determine the optimal data source for each query. +The key insight is that flushed MemTables serve as a bridge between the base table's index catchup and the current merged state. +For a query that requires a specific index for acceleration, when `index_gen < merged_gen`, +the generations in the gap `(index_gen, merged_gen]` have data already merged in the base table but are not covered by the base table's index. Since flushed MemTables contain pre-built indexes (created during [MemTable flush](#memtable-flush)), queries can use these indexes instead of scanning unindexed data in the base table. This ensures all reads remain indexed regardless of how far behind the async index builder is. -The key insight is that flushed MemTables serve as a bridge between the base table's index catchup and the current merged state. -When `index_gen < merged_gen`, the generations in the gap `(index_gen, merged_gen]` have data in the base table but are not covered by the base table's indexes. -Instead of falling back to a full scan for this gap, the query planner uses the flushed MemTable indexes which were pre-built during flush. -This maintains indexed query performance even during periods of heavy write activity where async index rebuilding falls behind. - See [Appendix 4: Index Catchup Example](#appendix-4-index-catchup-example) for a detailed timeline showing how this works in practice. - ### Query Execution -Query execution unions datasets within each region and deduplicates by primary key. -Deduplication uses two virtual columns: `_gen` (generation number, same value for all rows in a generation) and `_rowaddr` (row address within the dataset). -The ordering for "newest" is: highest `_gen` first, then highest `_rowaddr` (within the same generation, later rows win). - -A single write batch may contain duplicate primary keys. Query execution must deduplicate, keeping only the newest row for each key. - -For scan queries with filters, the plan unions datasets from relevant regions (after pruning), deduplicates within each region, then applies any remaining filters. -Early termination is possible with a streaming deduplicate operator. - -For vector search queries, each generation's index is searched independently. -Results from base table and MemTable generations use the same distance metric since they share identical index configuration. -The bloom filter is used to detect stale results: for each candidate from generation G, check if the primary key exists in any bloom filter from generations > G. -If found, the candidate is filtered out because a newer version exists that did not match the query as well. - -For full-text search queries, corpus statistics (document count, term frequencies) are aggregated across all generations to compute global BM25 parameters. -Each generation's FTS index is then searched with the global parameters, producing comparable scores. -Staleness filtering uses bloom filters similar to vector search. +Query execution unions datasets within each region and deduplicates by primary key according to [LSM tree merge read](#lsm-tree-merge-read). -For point lookups, the planner can short-circuit by checking newest generations first and returning immediately when the key is found. - -### Expected Query Plans - -This section provides query plan expectations using custom execution nodes optimized for MemWAL's data model. -See [Appendix 3: Execution Nodes](#appendix-3-execution-nodes) for detailed node descriptions. +The next few subsections go through the query plan expectations using custom execution nodes optimized for MemWAL's data model. All query plans assume the following MemWAL setup: @@ -767,6 +727,9 @@ region_B: gen 3: in_memory_memtable ``` +Existing Lance index optimizations (scalar indexes, fragment pruning, etc.) continue to apply within each scan and is omitted. +See [Appendix 3: Execution Nodes](#appendix-3-execution-nodes) for uncommon execution nodes we use here for optimized performance. + #### Scan Queries For scan queries, the base table is scanned once and each region's MemTables are scanned separately. @@ -786,8 +749,6 @@ DeduplicateExec: partition_by=[pk], order_by=[_gen DESC, _rowaddr DESC] ScanExec: region_B[gen=1], filter=[pushed_down] ``` -Existing Lance index optimizations (scalar indexes, fragment pruning, etc.) continue to apply within each scan. - #### Point Lookups Primary key-based point lookups first determine the target region using the region spec, then short-circuit by checking newest generations first within that region, falling back to the base table. @@ -810,8 +771,6 @@ CoalesceFirstExec: return_first_non_null TakeLastExec: base_table[gen=-1], filter=[pk = target] ``` -Existing Lance index optimizations (scalar indexes, fragment pruning, etc.) continue to apply within each lookup. - #### Vector Search Queries Vector search uses bloom filters to detect stale results across all generations. @@ -1017,11 +976,11 @@ This example demonstrates how `index_catchup` enables indexed reads during async Generation: 1 2 3 4 5 6 | | | | | | State: merged merged merged merged flushed active - | | | | | | -Base IVF index: [-- covers 1-3 --] - ↑ ↑ ↑ - index_gen=3 merged_gen=4 | - current_gen=6 + | | | | | +Base IVF index: [-- covers 1-3 --] | | + ↑ ↑ ↑ + index_gen=3 merged_gen=4 | + current_gen=6 ``` In this example: @@ -1032,7 +991,7 @@ In this example: - **Generation 5** is flushed to disk (not yet merged to base table) - **Generation 6** is the active in-memory MemTable -#### Read Strategy for Vector Search +#### Example Read Strategy for Vector Search Without `index_catchup` tracking, the query planner would need to perform an expensive full scan on the base table for generation 4. With `index_catchup`, the planner knows exactly which data is indexed and can use flushed MemTable indexes for the gap: @@ -1045,12 +1004,3 @@ With `index_catchup`, the planner knows exactly which data is indexed and can us | Active MemTable | 6 | Use in-memory IVF index | All data sources provide indexed access, maintaining query performance during async index rebuild. - -#### Why Flushed MemTables Are Retained - -Flushed MemTable files are not garbage collected until: - -1. Their data has been merged to the base table (`gen <= merged_gen`) -2. All maintained indexes have caught up (`gen <= index_gen` for all indexes) - -This ensures flushed MemTable indexes remain available to bridge the gap between `index_gen` and `merged_gen`.