From 22fefebacabad865ead0f09d75bd0ab672d44cbb Mon Sep 17 00:00:00 2001 From: GStravinsky <52568826+GStravinsky@users.noreply.github.com> Date: Tue, 30 Jan 2024 22:35:53 +0100 Subject: [PATCH 01/50] BigQuery: Decouple clustering from time partitioning when writing (#30094) * Decouple clustering from time partitioning when writing * Update sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java Co-authored-by: Michel Davit * Refactor the tests, remove redundant input validations and reuse existing variables * add to CHANGES.md * add PR and not issue in CHANGES.md --------- Co-authored-by: Michel Davit --- CHANGES.md | 3 +- .../beam/sdk/io/gcp/bigquery/BigQueryIO.java | 10 ++--- .../io/gcp/bigquery/CreateTableHelpers.java | 10 +++-- .../bigquery/DynamicDestinationsHelpers.java | 42 ++++++++++--------- .../gcp/bigquery/UpdateSchemaDestination.java | 9 ++-- .../io/gcp/bigquery/BigQueryIOWriteTest.java | 38 +++++++++-------- 6 files changed, 62 insertions(+), 50 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index 2d9c249bf6b3..c63464a6e01e 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -65,7 +65,8 @@ ## New Features / Improvements -* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +* [Enrichment Transform](https://s.apache.org/enrichment-transform) along with GCP BigTable handler added to Python SDK ([#30001](https://github.com/apache/beam/pull/30001)). +* Allow writing clustered and not time partitioned BigQuery tables (Java) ([#30094](https://github.com/apache/beam/pull/30094)). ## Breaking Changes diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java index 3d38ef0e83ff..cd62c5810d81 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.java @@ -96,7 +96,7 @@ import org.apache.beam.sdk.io.gcp.bigquery.BigQueryServices.StorageClient; import org.apache.beam.sdk.io.gcp.bigquery.BigQuerySourceBase.ExtractResult; import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.ConstantSchemaDestinations; -import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.ConstantTimePartitioningDestinations; +import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.ConstantTimePartitioningClusteringDestinations; import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.SchemaFromViewDestinations; import org.apache.beam.sdk.io.gcp.bigquery.DynamicDestinationsHelpers.TableFunctionDestinations; import org.apache.beam.sdk.io.gcp.bigquery.PassThroughThenCleanup.CleanupOperation; @@ -2744,8 +2744,7 @@ public Write withJsonTimePartitioning(ValueProvider partitioning) { } /** - * Specifies the clustering fields to use when writing to a single output table. Can only be - * used when {@link#withTimePartitioning(TimePartitioning)} is set. If {@link + * Specifies the clustering fields to use when writing to a single output table. If {@link * #to(SerializableFunction)} or {@link #to(DynamicDestinations)} is used to write to dynamic * tables, the fields here will be ignored; call {@link #withClustering()} instead. */ @@ -3357,9 +3356,10 @@ && getStorageApiTriggeringFrequency(bqOptions) != null) { } // Wrap with a DynamicDestinations class that will provide the proper TimePartitioning. - if (getJsonTimePartitioning() != null) { + if (getJsonTimePartitioning() != null + || Optional.ofNullable(getClustering()).map(Clustering::getFields).isPresent()) { dynamicDestinations = - new ConstantTimePartitioningDestinations<>( + new ConstantTimePartitioningClusteringDestinations<>( (DynamicDestinations) dynamicDestinations, getJsonTimePartitioning(), StaticValueProvider.of(BigQueryHelpers.toJsonString(getClustering()))); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableHelpers.java index 6edd3f71cc71..7a94657107ec 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/CreateTableHelpers.java @@ -179,11 +179,13 @@ private static void tryCreateTable( TimePartitioning timePartitioning = tableDestination.getTimePartitioning(); if (timePartitioning != null) { table.setTimePartitioning(timePartitioning); - Clustering clustering = tableDestination.getClustering(); - if (clustering != null) { - table.setClustering(clustering); - } } + + Clustering clustering = tableDestination.getClustering(); + if (clustering != null) { + table.setClustering(clustering); + } + if (kmsKey != null) { table.setEncryptionConfiguration(new EncryptionConfiguration().setKmsKeyName(kmsKey)); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java index 62355fd9417d..1f042a81eb9d 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/DynamicDestinationsHelpers.java @@ -272,23 +272,28 @@ public String toString() { } } - static class ConstantTimePartitioningDestinations + static class ConstantTimePartitioningClusteringDestinations extends DelegatingDynamicDestinations { - private final ValueProvider jsonTimePartitioning; + private final @Nullable ValueProvider jsonTimePartitioning; private final @Nullable ValueProvider jsonClustering; - ConstantTimePartitioningDestinations( + ConstantTimePartitioningClusteringDestinations( DynamicDestinations inner, ValueProvider jsonTimePartitioning, ValueProvider jsonClustering) { super(inner); - Preconditions.checkArgumentNotNull( - jsonTimePartitioning, "jsonTimePartitioning provider can not be null"); - if (jsonTimePartitioning.isAccessible()) { - Preconditions.checkArgumentNotNull( - jsonTimePartitioning.get(), "jsonTimePartitioning can not be null"); - } + + checkArgument( + (jsonTimePartitioning != null + && jsonTimePartitioning.isAccessible() + && jsonTimePartitioning.get() != null) + || (jsonClustering != null + && jsonClustering.isAccessible() + && jsonClustering.get() != null), + "at least one of jsonTimePartitioning or jsonClustering must be non-null, accessible " + + "and present"); + this.jsonTimePartitioning = jsonTimePartitioning; this.jsonClustering = jsonClustering; } @@ -296,13 +301,12 @@ static class ConstantTimePartitioningDestinations @Override public TableDestination getDestination(@Nullable ValueInSingleWindow element) { TableDestination destination = super.getDestination(element); - String partitioning = this.jsonTimePartitioning.get(); - checkArgument(partitioning != null, "jsonTimePartitioning can not be null"); + String partitioning = + Optional.ofNullable(jsonTimePartitioning).map(ValueProvider::get).orElse(null); + String clustering = Optional.ofNullable(jsonClustering).map(ValueProvider::get).orElse(null); + return new TableDestination( - destination.getTableSpec(), - destination.getTableDescription(), - partitioning, - Optional.ofNullable(jsonClustering).map(ValueProvider::get).orElse(null)); + destination.getTableSpec(), destination.getTableDescription(), partitioning, clustering); } @Override @@ -316,10 +320,10 @@ public Coder getDestinationCoder() { @Override public String toString() { - MoreObjects.ToStringHelper helper = - MoreObjects.toStringHelper(this) - .add("inner", inner) - .add("jsonTimePartitioning", jsonTimePartitioning); + MoreObjects.ToStringHelper helper = MoreObjects.toStringHelper(this).add("inner", inner); + if (jsonTimePartitioning != null) { + helper.add("jsonTimePartitioning", jsonTimePartitioning); + } if (jsonClustering != null) { helper.add("jsonClustering", jsonClustering); } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java index 51e61fe41953..65bb3bf11b1b 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/UpdateSchemaDestination.java @@ -288,11 +288,12 @@ private BigQueryHelpers.PendingJob startZeroLoadJob( } if (timePartitioning != null) { loadConfig.setTimePartitioning(timePartitioning); - // only set clustering if timePartitioning is set - if (clustering != null) { - loadConfig.setClustering(clustering); - } } + + if (clustering != null) { + loadConfig.setClustering(clustering); + } + if (kmsKey != null) { loadConfig.setDestinationEncryptionConfiguration( new EncryptionConfiguration().setKmsKeyName(kmsKey)); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java index 89cbc2cd24b8..21d3e53a0701 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIOWriteTest.java @@ -500,7 +500,7 @@ private void verifySideInputs() { } } - void testTimePartitioningClustering( + void testTimePartitioningAndClustering( BigQueryIO.Write.Method insertMethod, boolean enablePartitioning, boolean enableClustering) throws Exception { TableRow row1 = new TableRow().set("date", "2018-01-01").set("number", "1"); @@ -545,16 +545,8 @@ void testTimePartitioningClustering( } } - void testTimePartitioning(BigQueryIO.Write.Method insertMethod) throws Exception { - testTimePartitioningClustering(insertMethod, true, false); - } - - void testClustering(BigQueryIO.Write.Method insertMethod) throws Exception { - testTimePartitioningClustering(insertMethod, true, true); - } - - @Test - public void testTimePartitioning() throws Exception { + void testTimePartitioningAndClusteringWithAllMethods( + Boolean enablePartitioning, Boolean enableClustering) throws Exception { BigQueryIO.Write.Method method; if (useStorageApi) { method = @@ -564,15 +556,27 @@ public void testTimePartitioning() throws Exception { } else { method = Method.FILE_LOADS; } - testTimePartitioning(method); + testTimePartitioningAndClustering(method, enablePartitioning, enableClustering); } @Test - public void testClusteringStorageApi() throws Exception { - if (useStorageApi) { - testClustering( - useStorageApiApproximate ? Method.STORAGE_API_AT_LEAST_ONCE : Method.STORAGE_WRITE_API); - } + public void testTimePartitioningWithoutClustering() throws Exception { + testTimePartitioningAndClusteringWithAllMethods(true, false); + } + + @Test + public void testTimePartitioningWithClustering() throws Exception { + testTimePartitioningAndClusteringWithAllMethods(true, true); + } + + @Test + public void testClusteringWithoutPartitioning() throws Exception { + testTimePartitioningAndClusteringWithAllMethods(false, true); + } + + @Test + public void testNoClusteringNoPartitioning() throws Exception { + testTimePartitioningAndClusteringWithAllMethods(false, false); } @Test From 415b1658f5022117f2f5c161d775d51b1591ab0d Mon Sep 17 00:00:00 2001 From: Chris Gray Date: Tue, 30 Jan 2024 15:10:28 -0800 Subject: [PATCH 02/50] Merge pull request #30137: Fix a misspelling in zetasql overview page * Fix a misspelling * Apply suggested style change --- .../content/en/documentation/dsls/sql/zetasql/overview.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/website/www/site/content/en/documentation/dsls/sql/zetasql/overview.md b/website/www/site/content/en/documentation/dsls/sql/zetasql/overview.md index 03ab6e79177d..b68324a53677 100644 --- a/website/www/site/content/en/documentation/dsls/sql/zetasql/overview.md +++ b/website/www/site/content/en/documentation/dsls/sql/zetasql/overview.md @@ -16,7 +16,7 @@ See the License for the specific language governing permissions and limitations under the License. --> # Beam ZetaSQL overview -Beam SQL supports a varient of the [ZetaSQL](https://github.com/google/zetasql) language. ZetaSQL is similar to the language in BigQuery's SQL framework. This Beam SQL dialect is especially useful in pipelines that [write to or read from BigQuery tables](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.html). +Beam SQL supports a variant of the [ZetaSQL](https://github.com/google/zetasql) language. ZetaSQL is similar to the language in BigQuery's SQL framework. This Beam SQL dialect is especially useful in pipelines that [write to or read from BigQuery tables](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/BigQueryIO.html). Beam SQL has additional extensions leveraging Beam’s unified batch/streaming model and processing complex data types. You can use these extensions with all Beam SQL dialects, including Beam ZetaSQL. @@ -30,4 +30,4 @@ A Beam SQL statement comprises a series of tokens. For more information about to Beam SQL supports standard SQL scalar data types as well as extensions including arrays, maps, and nested rows. For more information about scalar data in Beam ZetaSQL, see the [Data types](/documentation/dsls/sql/zetasql/data-types) reference. ## Functions and operators -For a list of the builtin functions and operators supported in Beam ZetaSQL, see [SupportedZetaSqlBuiltinFunctions.java](https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java) (commented-out entries are not yet supported). For documentation on how these functions work, see the [ZetaSQL functions and operators](https://github.com/google/zetasql/blob/master/docs/functions-and-operators.md) reference. \ No newline at end of file +For a list of the built-in functions and operators supported in Beam ZetaSQL, see [SupportedZetaSqlBuiltinFunctions.java](https://github.com/apache/beam/blob/master/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/SupportedZetaSqlBuiltinFunctions.java) (commented-out entries are not yet supported). For documentation on how these functions work, see the [ZetaSQL functions and operators](https://github.com/google/zetasql/blob/master/docs/functions-and-operators.md) reference. From 1df9d6bba9150ff62276405e4e13d232411c0294 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 30 Jan 2024 18:45:22 -0500 Subject: [PATCH 03/50] Bumping timeout for beam_PostCommit_Java_DataflowV2 (#30160) --- .github/workflows/beam_PostCommit_Java_DataflowV2.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 58ecc37361d4..3abbe9cdc6c6 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -19,7 +19,7 @@ name: PostCommit Java Dataflow V2 on: schedule: - - cron: '30 3/6 * * *' + - cron: '30 3/8 * * *' pull_request_target: paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Java_DataflowV2.json'] workflow_dispatch: @@ -54,7 +54,7 @@ jobs: beam_PostCommit_Java_DataflowV2: name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) runs-on: [self-hosted, ubuntu-20.04, main] - timeout-minutes: 240 + timeout-minutes: 360 strategy: matrix: job_name: [beam_PostCommit_Java_DataflowV2] From 4c5ee349f9bfcdecbce9885c35ba5124066c4253 Mon Sep 17 00:00:00 2001 From: Jan Lukavsky Date: Wed, 31 Jan 2024 08:59:03 +0100 Subject: [PATCH 04/50] [sdks-java-core] Test for Reiterator outside loop in CoGbkResult --- .../java/org/apache/beam/sdk/transforms/join/CoGbkResult.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java index ba5bce20d090..ba0430e39daf 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/join/CoGbkResult.java @@ -95,8 +95,9 @@ public CoGbkResult( // according to their tag. final Iterator taggedIter = taggedValues.iterator(); int elementCount = 0; + boolean isReiterator = taggedIter instanceof Reiterator; while (taggedIter.hasNext()) { - if (elementCount++ >= inMemoryElementCount && taggedIter instanceof Reiterator) { + if (isReiterator && elementCount++ >= inMemoryElementCount) { // Let the tails be lazy. break; } From 41dee464db458fa72eeab7ddc902b242ebc894eb Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Wed, 31 Jan 2024 13:07:02 -0500 Subject: [PATCH 05/50] Make ReadFromBigQueryRequest id more randomized (#30156) * make ReadFromBigQueryRequest id more randomized --- sdks/python/apache_beam/io/gcp/bigquery.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index bba8b8a4af75..7648ab4064da 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -361,6 +361,7 @@ def chain_after(result): import json import logging import random +import secrets import time import uuid import warnings @@ -2925,8 +2926,9 @@ def __init__( self.table = table self.validate() - # We use this internal object ID to generate BigQuery export directories. - self.obj_id = random.randint(0, 100000) + # We use this internal object ID to generate BigQuery export directories + # and to create BigQuery job names + self.obj_id = '%d_%s' % (int(time.time()), secrets.token_hex(3)) def validate(self): if self.table is not None and self.query is not None: From 81513f10d60f65c0873fc2484fb27a8e38fabc56 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 31 Jan 2024 13:36:13 -0500 Subject: [PATCH 06/50] Bump github.com/aws/aws-sdk-go-v2/credentials in /sdks (#30162) Bumps [github.com/aws/aws-sdk-go-v2/credentials](https://github.com/aws/aws-sdk-go-v2) from 1.16.13 to 1.16.16. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Changelog](https://github.com/aws/aws-sdk-go-v2/blob/v1.16.16/CHANGELOG.md) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.16.13...v1.16.16) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/credentials dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 18 +++++++++--------- sdks/go.sum | 36 ++++++++++++++++++------------------ 2 files changed, 27 insertions(+), 27 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 1c5874a49c1e..de7b36930455 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,9 +30,9 @@ require ( cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.53.1 cloud.google.com/go/storage v1.36.0 - github.com/aws/aws-sdk-go-v2 v1.24.0 + github.com/aws/aws-sdk-go-v2 v1.24.1 github.com/aws/aws-sdk-go-v2/config v1.26.2 - github.com/aws/aws-sdk-go-v2/credentials v1.16.13 + github.com/aws/aws-sdk-go-v2/credentials v1.16.16 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 github.com/aws/smithy-go v1.19.0 @@ -113,18 +113,18 @@ require ( github.com/apache/thrift v0.16.0 // indirect github.com/aws/aws-sdk-go v1.34.0 // indirect github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.11 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.10 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.10 // indirect github.com/aws/aws-sdk-go-v2/internal/ini v1.7.2 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.2.3 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.4 // indirect github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.10 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.18.5 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.26.6 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.18.7 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.7 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.26.7 // indirect github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 7dc312f9cba2..60b711ab0595 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,26 +81,26 @@ github.com/aws/aws-sdk-go v1.30.19/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZve github.com/aws/aws-sdk-go v1.34.0 h1:brux2dRrlwCF5JhTL7MUT3WUwo9zfDHZZp3+g3Mvlmo= github.com/aws/aws-sdk-go v1.34.0/go.mod h1:5zCpMtNQVjRREroY7sYe8lOMRSxkhG6MZveU8YkpAk0= github.com/aws/aws-sdk-go-v2 v1.7.1/go.mod h1:L5LuPC1ZgDr2xQS7AmIec/Jlc7O/Y1u2KxJyNVab250= -github.com/aws/aws-sdk-go-v2 v1.24.0 h1:890+mqQ+hTpNuw0gGP6/4akolQkSToDJgHfQE7AwGuk= -github.com/aws/aws-sdk-go-v2 v1.24.0/go.mod h1:LNh45Br1YAkEKaAqvmE1m8FUx6a5b/V0oAKV7of29b4= +github.com/aws/aws-sdk-go-v2 v1.24.1 h1:xAojnj+ktS95YZlDf0zxWBkbFtymPeDP+rvUQIH3uAU= +github.com/aws/aws-sdk-go-v2 v1.24.1/go.mod h1:LNh45Br1YAkEKaAqvmE1m8FUx6a5b/V0oAKV7of29b4= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1 h1:ZY3108YtBNq96jNZTICHxN1gSBSbnvIdYwwqnvCV4Mc= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.5.1/go.mod h1:t8PYl/6LzdAqsU4/9tz28V/kU+asFePvpOMkdul0gEQ= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= github.com/aws/aws-sdk-go-v2/config v1.26.2 h1:+RWLEIWQIGgrz2pBPAUoGgNGs1TOyF4Hml7hCnYj2jc= github.com/aws/aws-sdk-go-v2/config v1.26.2/go.mod h1:l6xqvUxt0Oj7PI/SUXYLNyZ9T/yBPn3YTQcJLLOdtR8= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.16.13 h1:WLABQ4Cp4vXtXfOWOS3MEZKr6AAYUpMczLhgKtAjQ/8= -github.com/aws/aws-sdk-go-v2/credentials v1.16.13/go.mod h1:Qg6x82FXwW0sJHzYruxGiuApNo31UEtJvXVSZAXeWiw= +github.com/aws/aws-sdk-go-v2/credentials v1.16.16 h1:8q6Rliyv0aUFAVtzaldUEcS+T5gbadPbWdV1WcAddK8= +github.com/aws/aws-sdk-go-v2/credentials v1.16.16/go.mod h1:UHVZrdUsv63hPXFo1H7c5fEneoVo9UXiz36QG1GEPi0= github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.3.0/go.mod h1:2LAuqPx1I6jNfaGDucWfA2zqQCYCOMCDHiCOciALyNw= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10 h1:w98BT5w+ao1/r5sUuiH6JkVzjowOKeOJRHERyy1vh58= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.10/go.mod h1:K2WGI7vUvkIv1HoNbfBA1bvIZ+9kL3YVmWxeKuLQsiw= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.11 h1:c5I5iH+DZcH3xOIMlz3/tCKJDaHFwYEmxvlh2fAcFo8= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.14.11/go.mod h1:cRrYDYAMUohBJUtUnOhydaMHtiK/1NZ0Otc9lIb6O0Y= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.3.2/go.mod h1:qaqQiHSrOUVOfKe6fhgQ6UzhxjwqVW8aHNegd6Ws4w4= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8 h1:wuOjvalpd2CnXffks74Vq6n3yv9vunKCoy4R1sjStGk= github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.13.8/go.mod h1:vywwjy6VnrR48Izg136JoSUXC4mH9QeUi3g0EH9DSrA= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9 h1:v+HbZaCGmOwnTTVS86Fleq0vPzOd7tnJGbFhP0stNLs= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.9/go.mod h1:Xjqy+Nyj7VDLBtCMkQYOw1QYfAEZCVLrfI0ezve8wd4= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9 h1:N94sVhRACtXyVcjXxrwK1SKFIJrA9pOJ5yu2eSHnmls= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.9/go.mod h1:hqamLz7g1/4EJP+GH5NBhcUMLjW+gKLQabgyz6/7WAU= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.10 h1:vF+Zgd9s+H4vOXd5BMaPWykta2a6Ih0AKLq/X6NYKn4= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.2.10/go.mod h1:6BkRjejp/GR4411UGqkX8+wFMbFbqsUIimfK4XjOKR4= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.10 h1:nYPe006ktcqUji8S2mqXf9c/7NdiKriOwMvWQHgYztw= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.5.10/go.mod h1:6UV4SZkVvmODfXKql4LCbaZUpF7HO2BX38FgBf9ZOLw= github.com/aws/aws-sdk-go-v2/internal/ini v1.1.1/go.mod h1:Zy8smImhTdOETZqfyn01iNOe0CNggVbPjCajyaz6Gvg= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.2 h1:GrSw8s0Gs/5zZ0SX+gX4zQjRnRsMJDJ2sLur1gRBhEM= github.com/aws/aws-sdk-go-v2/internal/ini v1.7.2/go.mod h1:6fQQgfuGmw8Al/3M2IgIllycxV7ZW7WCdVSqfBeUiCY= @@ -112,8 +112,8 @@ github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.10.4/go.mod h1: github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3 h1:xbwRyCy7kXrOj89iIKLB6NfE2WCpP9HoKyk8dMDvnIQ= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3/go.mod h1:R+/S1O4TYpcktbVwddeOYg+uwUfLhADP2S/x4QwsCTM= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.2.1/go.mod h1:zceowr5Z1Nh2WVP8bf/3ikB41IZW59E4yIYbg+pC6mw= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9 h1:Nf2sHxjMJR8CSImIVCONRi4g0Su3J+TSTbS7G0pUeMU= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.9/go.mod h1:idky4TER38YIjr2cADF1/ugFMKvZV7p//pVeV5LZbF0= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.10 h1:DBYTXwIGQSGs9w4jKm60F5dmCQ3EEruxdc0MFh+3EY4= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.10/go.mod h1:wohMUQiFdzo0NtxbBg0mSRGZ4vL3n0dKjLTINdcIino= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.5.1/go.mod h1:6EQZIwNNvHpq/2/QSJnp4+ECvqIy55w95Ofs0ze+nGQ= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3 h1:KV0z2RDc7euMtg8aUT1czv5p29zcLlXALNFsd3jkkEc= github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.16.3/go.mod h1:KZgs2ny8HsxRIRbDwgvJcHHBZPOzQr/+NtGwnP+w2ec= @@ -121,13 +121,13 @@ github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32 github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2 h1:NnduxUd9+Fq9DcCDdJK8v6l9lR1xDX4usvog+JuQAno= github.com/aws/aws-sdk-go-v2/service/s3 v1.42.2/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= github.com/aws/aws-sdk-go-v2/service/sso v1.3.1/go.mod h1:J3A3RGUvuCZjvSuZEcOpHDnzZP/sKbhDWV2T1EOzFIM= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.5 h1:ldSFWz9tEHAwHNmjx2Cvy1MjP5/L9kNoR0skc6wyOOM= -github.com/aws/aws-sdk-go-v2/service/sso v1.18.5/go.mod h1:CaFfXLYL376jgbP7VKC96uFcU8Rlavak0UlAwk1Dlhc= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5 h1:2k9KmFawS63euAkY4/ixVNsYYwrwnd5fIvgEKkfZFNM= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.5/go.mod h1:W+nd4wWDVkSUIox9bacmkBP5NMFQeTJ/xqNabpzSR38= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.7 h1:eajuO3nykDPdYicLlP3AGgOyVN3MOlFmZv7WGTuJPow= +github.com/aws/aws-sdk-go-v2/service/sso v1.18.7/go.mod h1:+mJNDdF+qiUlNKNC3fxn74WWNN+sOiGOEImje+3ScPM= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.7 h1:QPMJf+Jw8E1l7zqhZmMlFw6w1NmfkfiSK8mS4zOx3BA= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.21.7/go.mod h1:ykf3COxYI0UJmxcfcxcVuz7b6uADi1FkiUz6Eb7AgM8= github.com/aws/aws-sdk-go-v2/service/sts v1.6.0/go.mod h1:q7o0j7d7HrJk/vr9uUt3BVRASvcU7gYZB9PUgPiByXg= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.6 h1:HJeiuZ2fldpd0WqngyMR6KW7ofkXNLyOaHwEIGm39Cs= -github.com/aws/aws-sdk-go-v2/service/sts v1.26.6/go.mod h1:XX5gh4CB7wAs4KhcF46G6C8a2i7eupU19dcAAE+EydU= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.7 h1:NzO4Vrau795RkUdSHKEwiR01FaGzGOH1EETJ+5QHnm0= +github.com/aws/aws-sdk-go-v2/service/sts v1.26.7/go.mod h1:6h2YuIoxaMSCFf5fi1EgZAwdfkGMgDY+DVfa61uLe4U= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= github.com/aws/smithy-go v1.19.0 h1:KWFKQV80DpP3vJrrA9sVAHQ5gc2z8i4EzrLhLlWXcBM= github.com/aws/smithy-go v1.19.0/go.mod h1:NukqUGpCZIILqqiV0NIjeFh24kd/FAa4beRb6nbIUPE= From 89d1c06e1eab530c4e95a807a035b450c340e5f8 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Wed, 31 Jan 2024 19:26:26 +0000 Subject: [PATCH 07/50] Fix Fhir IO Search IT (#30158) * [BEAM-12550] Implement Parallelizable Skew and Kurtosis (Skew implementation) R: @TheNeuralBit * fix fhirio search tests * spotless * reduce query count and size * postcommit * remove clear --------- Co-authored-by: svetakvsundhar --- .../trigger_files/beam_PostCommit_Java_DataflowV2.json | 0 .../beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java | 9 +++------ 2 files changed, 3 insertions(+), 6 deletions(-) create mode 100644 .github/trigger_files/beam_PostCommit_Java_DataflowV2.json diff --git a/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json b/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json new file mode 100644 index 000000000000..e69de29bb2d1 diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java index fc92a568ba4d..ebd94123d499 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/healthcare/FhirIOSearchIT.java @@ -43,7 +43,6 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.After; import org.junit.Before; -import org.junit.Ignore; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -66,7 +65,7 @@ public static Collection versions() { private static final String BASE_STORE_ID = "FHIR_store_search_it_" + System.currentTimeMillis() + "_" + new SecureRandom().nextInt(32); private String fhirStoreId; - private static final int MAX_NUM_OF_SEARCHES = 50; + private static final int MAX_NUM_OF_SEARCHES = 20; private List> input = new ArrayList<>(); private List>> genericParametersInput = new ArrayList<>(); private static final String KEY = "key"; @@ -96,9 +95,9 @@ public void setup() throws Exception { JsonArray fhirResources = JsonParser.parseString(bundles.get(0)).getAsJsonObject().getAsJsonArray("entry"); - Map searchParameters = ImmutableMap.of("_count", "50"); + Map searchParameters = ImmutableMap.of("_count", "20"); Map> genericSearchParameters = - ImmutableMap.of("_count", Arrays.asList(50)); + ImmutableMap.of("_count", Arrays.asList(20)); // Include a non-resource type search. input.add(FhirSearchParameter.of("", KEY, searchParameters)); @@ -127,7 +126,6 @@ public void teardown() throws IOException { } } - @Ignore("https://github.com/apache/beam/issues/28505") @Test public void testFhirIOSearch() { pipeline.getOptions().as(DirectOptions.class).setBlockOnRun(false); @@ -157,7 +155,6 @@ public void testFhirIOSearch() { pipeline.run().waitUntilFinish(); } - @Ignore("https://github.com/apache/beam/issues/28505") @Test public void testFhirIOSearchWithGenericParameters() { pipeline.getOptions().as(DirectOptions.class).setBlockOnRun(false); From a221f98a5f46be985afcb98a65fcec3b46b81f92 Mon Sep 17 00:00:00 2001 From: Mattie Fu Date: Wed, 31 Jan 2024 14:43:34 -0500 Subject: [PATCH 08/50] fix: fix error handling in segment reader and add an integration test (#30147) * fix: fix error handling in segment reader and add an integration test * also check for root cause * revert getRootCause change * fix format * add a log line * remove logging * change the log level to debug, update comment --- .../io/gcp/bigtable/BigtableServiceImpl.java | 20 +++- .../sdk/io/gcp/bigtable/BigtableReadIT.java | 93 ++++++++++++++++++- 2 files changed, 106 insertions(+), 7 deletions(-) diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java index 7537a16a188b..d6208be1bf94 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableServiceImpl.java @@ -387,6 +387,11 @@ public void onResponse(Row response) { currentByteSize += response.getSerializedSize(); rows.add(response); if (currentByteSize > maxSegmentByteSize) { + LOG.debug( + "Reached maxSegmentByteSize, cancelling the stream. currentByteSize is {}, maxSegmentByteSize is {}, read rows {}", + currentByteSize, + maxSegmentByteSize, + rows.size()); byteLimitReached = true; controller.cancel(); return; @@ -395,14 +400,25 @@ public void onResponse(Row response) { @Override public void onError(Throwable t) { - future.setException(t); + if (byteLimitReached) { + // When the byte limit is reached we cancel the stream in onResponse. + // In this case we don't want to fail the request with cancellation + // exception. Instead, we construct the next request. + onComplete(); + } else { + future.setException(t); + } } @Override public void onComplete() { ReadRowsRequest nextNextRequest = null; - // When requested rows < limit, the current request will be the last + // Only schedule the next segment fetch when there's a possibility of more + // data to read. We know there might be more data when the current segment + // ended with the artificial byte limit or the row limit. + // If the RPC ended without hitting the byte limit or row limit, we know + // there's no more data to read and nextNextRequest would be null. if (byteLimitReached || rows.size() == nextRequest.getRowsLimit()) { nextNextRequest = truncateRequest(nextRequest, rows.get(rows.size() - 1).getKey()); diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java index 704a895992c2..bc88858ebc33 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableReadIT.java @@ -17,7 +17,16 @@ */ package org.apache.beam.sdk.io.gcp.bigtable; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminClient; +import com.google.cloud.bigtable.admin.v2.BigtableTableAdminSettings; +import com.google.cloud.bigtable.admin.v2.models.CreateTableRequest; import com.google.cloud.bigtable.config.BigtableOptions; +import com.google.cloud.bigtable.data.v2.BigtableDataClient; +import com.google.cloud.bigtable.data.v2.BigtableDataSettings; +import com.google.cloud.bigtable.data.v2.models.RowMutation; +import java.io.IOException; +import java.util.Date; +import org.apache.beam.repackaged.core.org.apache.commons.lang3.StringUtils; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; @@ -25,6 +34,8 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.values.PCollection; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -32,18 +43,62 @@ /** End-to-end tests of BigtableRead. */ @RunWith(JUnit4.class) public class BigtableReadIT { + private static final String COLUMN_FAMILY_NAME = "cf"; - @Test - public void testE2EBigtableRead() throws Exception { + private String project; + + private BigtableTestOptions options; + private String tableId = String.format("BigtableReadIT-%tF-% count = + p.apply( + BigtableIO.read() + .withBigtableOptions(bigtableOptionsBuilder) + .withTableId(tableId) + .withMaxBufferElementCount(10)) + .apply(Count.globally()); + PAssert.thatSingleton(count).isEqualTo(numRows); + p.run(); + } } From ddb93272ab6227b468f1731d8482e14666a684fe Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 31 Jan 2024 14:00:53 -0800 Subject: [PATCH 09/50] Fix git trusted directory errors in website build. (#30171) --- website/Dockerfile | 2 ++ 1 file changed, 2 insertions(+) diff --git a/website/Dockerfile b/website/Dockerfile index 0cb992422eac..90d4202728d2 100644 --- a/website/Dockerfile +++ b/website/Dockerfile @@ -74,3 +74,5 @@ RUN HUGOHOME="$(mktemp -d)" \ && chmod +x /usr/local/bin/hugo \ && rm -r "${HUGOHOME}" +# Prevent git errors of the form "fatal: detected dubious ownership in repository" +RUN git config --file=/.gitconfig --add safe.directory /opt && chmod a+r /.gitconfig From 30917ff7b3e1009578fddd1b4d0b6b1db4776d13 Mon Sep 17 00:00:00 2001 From: JayajP Date: Wed, 31 Jan 2024 14:21:00 -0800 Subject: [PATCH 10/50] Don't use '-' as a delimiter in BigQuerySinkMetrics (#30118) * Don't use '-' as a delimiter in BigQuerySinkMetrics * Use Optional::{filter|map} for returning simple optional --- ...icsToPerStepNamespaceMetricsConverter.java | 27 ++++----- ...oPerStepNamespaceMetricsConverterTest.java | 16 ++--- .../StreamingStepMetricsContainerTest.java | 4 +- .../io/gcp/bigquery/BigQuerySinkMetrics.java | 37 +++++++----- .../gcp/bigquery/BigQuerySinkMetricsTest.java | 59 ++++++++++--------- 5 files changed, 75 insertions(+), 68 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java index f3f31ccfd36f..7b72d6507788 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverter.java @@ -52,17 +52,14 @@ private static Optional convertCounterToMetricValue( return Optional.empty(); } - BigQuerySinkMetrics.ParsedMetricName labeledName = - BigQuerySinkMetrics.parseMetricName(metricName.getName()); - if (labeledName == null || labeledName.getBaseName().isEmpty()) { - return Optional.empty(); - } - - return Optional.of( - new MetricValue() - .setMetric(labeledName.getBaseName()) - .setMetricLabels(labeledName.getMetricLabels()) - .setValueInt64(value)); + return BigQuerySinkMetrics.parseMetricName(metricName.getName()) + .filter(labeledName -> !labeledName.getBaseName().isEmpty()) + .map( + labeledName -> + new MetricValue() + .setMetric(labeledName.getBaseName()) + .setMetricLabels(labeledName.getMetricLabels()) + .setValueInt64(value)); } /** @@ -78,9 +75,9 @@ private static Optional convertHistogramToMetricValue( return Optional.empty(); } - BigQuerySinkMetrics.ParsedMetricName labeledName = + Optional labeledName = BigQuerySinkMetrics.parseMetricName(metricName.getName()); - if (labeledName == null || labeledName.getBaseName().isEmpty()) { + if (!labeledName.isPresent() || labeledName.get().getBaseName().isEmpty()) { return Optional.empty(); } @@ -133,8 +130,8 @@ private static Optional convertHistogramToMetricValue( return Optional.of( new MetricValue() - .setMetric(labeledName.getBaseName()) - .setMetricLabels(labeledName.getMetricLabels()) + .setMetric(labeledName.get().getBaseName()) + .setMetricLabels(labeledName.get().getMetricLabels()) .setValueHistogram(histogramValue)); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverterTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverterTest.java index a37bf643b7ee..0a4cd06cf8ca 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverterTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/MetricsToPerStepNamespaceMetricsConverterTest.java @@ -81,7 +81,7 @@ public void testConvert_successfulyConvertCounters() { Map counters = new HashMap(); MetricName bigQueryMetric1 = MetricName.named("BigQuerySink", "metric1"); MetricName bigQueryMetric2 = - MetricName.named("BigQuerySink", "metric2-label1:val1;label2:val2;"); + MetricName.named("BigQuerySink", "metric2*label1:val1;label2:val2;"); MetricName bigQueryMetric3 = MetricName.named("BigQuerySink", "zeroValue"); counters.put(bigQueryMetric1, 5L); @@ -112,14 +112,14 @@ public void testConvert_successfulyConvertCounters() { @Test public void testConvert_skipInvalidMetricNames() { Map counters = new HashMap<>(); - MetricName bigQueryMetric1 = MetricName.named("BigQuerySink", "invalid-metric-name1"); - counters.put(bigQueryMetric1, 5L); + MetricName invalidName1 = MetricName.named("BigQuerySink", "**"); + counters.put(invalidName1, 5L); Map histograms = new HashMap<>(); - MetricName bigQueryMetric2 = MetricName.named("BigQuerySink", "invalid-metric-name2"); + MetricName invalidName2 = MetricName.named("BigQuerySink", "****"); HistogramData nonEmptyLinearHistogram = HistogramData.linear(0, 10, 10); nonEmptyLinearHistogram.record(-5.0); - histograms.put(bigQueryMetric2, nonEmptyLinearHistogram); + histograms.put(invalidName2, nonEmptyLinearHistogram); Collection conversionResult = MetricsToPerStepNamespaceMetricsConverter.convert("testStep", counters, histograms); @@ -131,7 +131,7 @@ public void testConvert_successfulConvertHistograms() { Map histograms = new HashMap(); MetricName bigQueryMetric1 = MetricName.named("BigQuerySink", "baseLabel"); MetricName bigQueryMetric2 = - MetricName.named("BigQuerySink", "baseLabel-label1:val1;label2:val2;"); + MetricName.named("BigQuerySink", "baseLabel*label1:val1;label2:val2;"); MetricName bigQueryMetric3 = MetricName.named("BigQuerySink", "zeroValue"); HistogramData nonEmptyLinearHistogram = HistogramData.linear(0, 10, 10); @@ -235,10 +235,10 @@ public void testConvert_convertCountersAndHistograms() { Map counters = new HashMap<>(); Map histograms = new HashMap(); - MetricName counterMetricName = MetricName.named("BigQuerySink", "counter-label1:val1;"); + MetricName counterMetricName = MetricName.named("BigQuerySink", "counter*label1:val1;"); counters.put(counterMetricName, 3L); - MetricName histogramMetricName = MetricName.named("BigQuerySink", "histogram-label2:val2;"); + MetricName histogramMetricName = MetricName.named("BigQuerySink", "histogram*label2:val2;"); HistogramData linearHistogram = HistogramData.linear(0, 10, 10); linearHistogram.record(5.0); histograms.put(histogramMetricName, linearHistogram); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java index cc0722e7a8e6..c586f2b87814 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainerTest.java @@ -226,10 +226,10 @@ public void testPerWorkerMetrics() { @Test public void testExtractPerWorkerMetricUpdates() { StreamingStepMetricsContainer.setEnablePerWorkerMetrics(true); - MetricName counterMetricName = MetricName.named("BigQuerySink", "counter-"); + MetricName counterMetricName = MetricName.named("BigQuerySink", "counter"); c1.getPerWorkerCounter(counterMetricName).inc(3); - MetricName histogramMetricName = MetricName.named("BigQuerySink", "histogram-"); + MetricName histogramMetricName = MetricName.named("BigQuerySink", "histogram"); HistogramData.LinearBuckets linearBuckets = HistogramData.LinearBuckets.of(0, 10, 10); c2.getPerWorkerHistogram(histogramMetricName, linearBuckets).update(5.0); diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java index c0f470b39214..34e3b704b4f3 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetrics.java @@ -22,6 +22,7 @@ import java.time.Instant; import java.util.List; import java.util.NavigableMap; +import java.util.Optional; import java.util.TreeMap; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -40,10 +41,10 @@ * *

In general metrics be in the namespace 'BigQuerySink' and have their name formatted as: * - *

'{baseName}-{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};' + *

'{baseName}*{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};' */ public class BigQuerySinkMetrics { - private static Boolean supportMetricsDeletion = false; + private static boolean supportMetricsDeletion = false; public static final String METRICS_NAMESPACE = "BigQuerySink"; @@ -78,10 +79,11 @@ enum RowStatus { private static final String RPC_METHOD = "rpc_method"; private static final String ROW_STATUS = "row_status"; - // Delimiters + // Delimiters. Avoid using dilimiters that can also be used in a BigQuery table name. + // ref: https://cloud.google.com/bigquery/docs/tables#table_naming private static final char LABEL_DELIMITER = ';'; private static final char METRIC_KV_DELIMITER = ':'; - private static final char METRIC_NAME_DELIMITER = '-'; + private static final char METRIC_NAME_DELIMITER = '*'; @AutoValue public abstract static class ParsedMetricName { @@ -119,38 +121,41 @@ private static String createLabeledMetricName( * Parse a 'metric name' String that was created with 'createLabeledMetricName'. The input string * should be formatted as. * - *

'{baseName}-{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};' + *

'{baseName}*{metricLabelKey1}:{metricLabelVal1};...{metricLabelKeyN}:{metricLabelValN};' * * @param metricName * @return Returns a ParsedMetricName object if the input string is properly formatted. If the - * input string is empty or malformed, returns null. + * input string is empty or malformed, returns an empty value. */ - public static @Nullable ParsedMetricName parseMetricName(String metricName) { + public static Optional parseMetricName(String metricName) { if (metricName.isEmpty()) { - return null; + return Optional.empty(); } - List metricNameSplit = Splitter.on(METRIC_NAME_DELIMITER).splitToList(metricName); + List metricNameSplit = + Splitter.on(METRIC_NAME_DELIMITER).limit(2).splitToList(metricName); ImmutableMap.Builder metricLabelsBuilder = ImmutableMap.builder(); - if (metricNameSplit.size() == 1) { - return ParsedMetricName.create(metricNameSplit.get(0)); + if (metricNameSplit.size() == 0) { + return Optional.empty(); } - if (metricNameSplit.size() != 2) { - return null; + if (metricNameSplit.size() == 1) { + return Optional.of(ParsedMetricName.create(metricNameSplit.get(0))); } + // metrcNameSplit is assumed to be size two. List labels = Splitter.on(LABEL_DELIMITER).splitToList(metricNameSplit.get(1)); for (String label : labels) { - List kv = Splitter.on(METRIC_KV_DELIMITER).splitToList(label); + List kv = Splitter.on(METRIC_KV_DELIMITER).limit(2).splitToList(label); if (kv.size() != 2) { continue; } metricLabelsBuilder.put(kv.get(0), kv.get(1)); } - return ParsedMetricName.create(metricNameSplit.get(0), metricLabelsBuilder.build()); + return Optional.of( + ParsedMetricName.create(metricNameSplit.get(0), metricLabelsBuilder.build())); } /** @@ -321,7 +326,7 @@ public static void reportFailedRPCMetrics( updateRpcLatencyMetric(c, method); } - public static void setSupportMetricsDeletion(Boolean supportMetricsDeletion) { + public static void setSupportMetricsDeletion(boolean supportMetricsDeletion) { BigQuerySinkMetrics.supportMetricsDeletion = supportMetricsDeletion; } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java index 6b04ed0acc15..8b960a8c6c6d 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQuerySinkMetricsTest.java @@ -20,14 +20,13 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.nullValue; import com.google.cloud.bigquery.storage.v1.AppendRowsResponse; import com.google.cloud.bigquery.storage.v1.Exceptions; import io.grpc.Status; import java.time.Instant; import java.util.List; +import java.util.Optional; import java.util.concurrent.ConcurrentHashMap; import org.apache.beam.runners.core.metrics.CounterCell; import org.apache.beam.runners.core.metrics.MetricsContainerImpl; @@ -112,7 +111,7 @@ public void testAppendRowsRowStatusCounter() throws Exception { deletesDisabledCounter.inc(); MetricName deletesDisabledCounterName = MetricName.named( - "BigQuerySink", "RowsAppendedCount-row_status:SUCCESSFUL;rpc_status:rpcStatus;"); + "BigQuerySink", "RowsAppendedCount*row_status:SUCCESSFUL;rpc_status:rpcStatus;"); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(deletesDisabledCounterName)); assertThat( testContainer.perWorkerCounters.get(deletesDisabledCounterName).getCumulative(), @@ -127,7 +126,7 @@ public void testAppendRowsRowStatusCounter() throws Exception { MetricName deletesEnabledCounterName = MetricName.named( "BigQuerySink", - "RowsAppendedCount-row_status:SUCCESSFUL;rpc_status:rpcStatus;table_id:tableId;"); + "RowsAppendedCount*row_status:SUCCESSFUL;rpc_status:rpcStatus;table_id:tableId;"); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(deletesEnabledCounterName)); assertThat( testContainer.perWorkerCounters.get(deletesEnabledCounterName).getCumulative(), @@ -160,9 +159,10 @@ public void testThrottledTimeCounter() throws Exception { appendRowsThrottleCounter.inc(1); assertThat( appendRowsThrottleCounter.getName().getName(), - equalTo("ThrottledTime-rpc_method:APPEND_ROWS;")); + equalTo("ThrottledTime*rpc_method:APPEND_ROWS;")); + MetricName counterName = - MetricName.named("BigQuerySink", "ThrottledTime-rpc_method:APPEND_ROWS;"); + MetricName.named("BigQuerySink", "ThrottledTime*rpc_method:APPEND_ROWS;"); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterName)); assertThat(testContainer.perWorkerCounters.get(counterName).getCumulative(), equalTo(1L)); } @@ -182,9 +182,9 @@ public void testReportSuccessfulRpcMetrics() throws Exception { BigQuerySinkMetrics.reportSuccessfulRpcMetrics( c, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, "tableId"); MetricName counterNameDisabledDeletes = - MetricName.named("BigQuerySink", "RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:OK;"); + MetricName.named("BigQuerySink", "RpcRequestsCount*rpc_method:APPEND_ROWS;rpc_status:OK;"); MetricName histogramName = - MetricName.named("BigQuerySink", "RpcLatency-rpc_method:APPEND_ROWS;"); + MetricName.named("BigQuerySink", "RpcLatency*rpc_method:APPEND_ROWS;"); HistogramData.BucketType bucketType = HistogramData.ExponentialBuckets.of(1, 34); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterNameDisabledDeletes)); assertThat( @@ -202,7 +202,7 @@ public void testReportSuccessfulRpcMetrics() throws Exception { MetricName counterNameEnabledDeletes = MetricName.named( "BigQuerySink", - "RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:OK;table_id:tableId;"); + "RpcRequestsCount*rpc_method:APPEND_ROWS;rpc_status:OK;table_id:tableId;"); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterNameEnabledDeletes)); assertThat( testContainer.perWorkerCounters.get(counterNameEnabledDeletes).getCumulative(), @@ -232,9 +232,9 @@ public void testReportFailedRPCMetrics_KnownGrpcError() throws Exception { c, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, "tableId"); MetricName counterNameDisabledDeletes = MetricName.named( - "BigQuerySink", "RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:NOT_FOUND;"); + "BigQuerySink", "RpcRequestsCount*rpc_method:APPEND_ROWS;rpc_status:NOT_FOUND;"); MetricName histogramName = - MetricName.named("BigQuerySink", "RpcLatency-rpc_method:APPEND_ROWS;"); + MetricName.named("BigQuerySink", "RpcLatency*rpc_method:APPEND_ROWS;"); HistogramData.BucketType bucketType = HistogramData.ExponentialBuckets.of(1, 34); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterNameDisabledDeletes)); assertThat( @@ -255,7 +255,7 @@ public void testReportFailedRPCMetrics_KnownGrpcError() throws Exception { MetricName counterNameEnabledDeletes = MetricName.named( "BigQuerySink", - "RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:NOT_FOUND;table_id:tableId;"); + "RpcRequestsCount*rpc_method:APPEND_ROWS;rpc_status:NOT_FOUND;table_id:tableId;"); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterNameEnabledDeletes)); assertThat( testContainer.perWorkerCounters.get(counterNameEnabledDeletes).getCumulative(), @@ -284,9 +284,9 @@ public void testReportFailedRPCMetrics_UnknownGrpcError() throws Exception { c, BigQuerySinkMetrics.RpcMethod.APPEND_ROWS, "tableId"); MetricName counterNameDisabledDeletes = MetricName.named( - "BigQuerySink", "RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:UNKNOWN;"); + "BigQuerySink", "RpcRequestsCount*rpc_method:APPEND_ROWS;rpc_status:UNKNOWN;"); MetricName histogramName = - MetricName.named("BigQuerySink", "RpcLatency-rpc_method:APPEND_ROWS;"); + MetricName.named("BigQuerySink", "RpcLatency*rpc_method:APPEND_ROWS;"); HistogramData.BucketType bucketType = HistogramData.ExponentialBuckets.of(1, 34); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterNameDisabledDeletes)); assertThat( @@ -304,7 +304,7 @@ public void testReportFailedRPCMetrics_UnknownGrpcError() throws Exception { MetricName counterNameEnabledDeletes = MetricName.named( "BigQuerySink", - "RpcRequestsCount-rpc_method:APPEND_ROWS;rpc_status:UNKNOWN;table_id:tableId;"); + "RpcRequestsCount*rpc_method:APPEND_ROWS;rpc_status:UNKNOWN;table_id:tableId;"); assertThat(testContainer.perWorkerCounters, IsMapContaining.hasKey(counterNameEnabledDeletes)); assertThat( testContainer.perWorkerCounters.get(counterNameEnabledDeletes).getCumulative(), @@ -317,41 +317,46 @@ public void testReportFailedRPCMetrics_UnknownGrpcError() throws Exception { @Test public void testParseMetricName_noLabels() { String baseMetricName = "baseMetricName"; - BigQuerySinkMetrics.ParsedMetricName metricName = + BigQuerySinkMetrics.ParsedMetricName expectedName = BigQuerySinkMetrics.ParsedMetricName.create(baseMetricName); - assertThat(BigQuerySinkMetrics.parseMetricName(baseMetricName), equalTo(metricName)); + Optional parsedMetricName = + BigQuerySinkMetrics.parseMetricName(baseMetricName); + assertThat(parsedMetricName.isPresent(), equalTo(true)); + assertThat(parsedMetricName.get(), equalTo(expectedName)); } @Test public void testParseMetricName_successfulLabels() { - String metricName = "baseLabel-key1:val1;key2:val2;key3:val3;"; + String metricName = "baseLabel*key1:val1;key2:val2;key3:val3;"; ImmutableMap metricLabels = ImmutableMap.of("key1", "val1", "key2", "val2", "key3", "val3"); BigQuerySinkMetrics.ParsedMetricName expectedName = BigQuerySinkMetrics.ParsedMetricName.create("baseLabel", metricLabels); - assertThat(BigQuerySinkMetrics.parseMetricName(metricName), equalTo(expectedName)); - } + Optional parsedMetricName = + BigQuerySinkMetrics.parseMetricName(metricName); - @Test - public void testParseMetricName_malformedMetricName() { - String malformedMetricName = "baseLabel-key1:val1-key2:val2"; - assertThat(BigQuerySinkMetrics.parseMetricName(malformedMetricName), is(nullValue())); + assertThat(parsedMetricName.isPresent(), equalTo(true)); + assertThat(parsedMetricName.get(), equalTo(expectedName)); } @Test public void testParseMetricName_malformedMetricLabels() { - String metricName = "baseLabel-key1:val1:malformedField;key2:val2;"; + String metricName = "baseLabel*malformed_kv_pair;key2:val2;"; ImmutableMap metricLabels = ImmutableMap.of("key2", "val2"); BigQuerySinkMetrics.ParsedMetricName expectedName = BigQuerySinkMetrics.ParsedMetricName.create("baseLabel", metricLabels); - assertThat(BigQuerySinkMetrics.parseMetricName(metricName), equalTo(expectedName)); + Optional parsedMetricName = + BigQuerySinkMetrics.parseMetricName(metricName); + + assertThat(parsedMetricName.isPresent(), equalTo(true)); + assertThat(parsedMetricName.get(), equalTo(expectedName)); } @Test public void testParseMetricName_emptyString() { - assertThat(BigQuerySinkMetrics.parseMetricName(""), is(nullValue())); + assertThat(BigQuerySinkMetrics.parseMetricName("").isPresent(), equalTo(false)); } } From fb0762352c20ac50f182facdd82d80f4ecfaa052 Mon Sep 17 00:00:00 2001 From: JayajP Date: Wed, 31 Jan 2024 14:21:12 -0800 Subject: [PATCH 11/50] Periodically send perWorkerMetrics on the WorkerMessages API (#30135) --- .../worker/DataflowWorkUnitClient.java | 24 +++++++++-- .../worker/StreamingDataflowWorker.java | 40 +++++++++++++++++-- .../worker/StreamingStepMetricsContainer.java | 8 +++- .../dataflow/worker/WorkUnitClient.java | 17 ++++++-- .../dataflow/worker/streaming/StageInfo.java | 9 +++++ .../worker/DataflowWorkUnitClientTest.java | 37 ++++++++++++++++- 6 files changed, 122 insertions(+), 13 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java index bb39e3bd9af4..f3caa8d0f3ac 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClient.java @@ -29,6 +29,7 @@ import com.google.api.services.dataflow.Dataflow; import com.google.api.services.dataflow.model.LeaseWorkItemRequest; import com.google.api.services.dataflow.model.LeaseWorkItemResponse; +import com.google.api.services.dataflow.model.PerWorkerMetrics; import com.google.api.services.dataflow.model.ReportWorkItemStatusRequest; import com.google.api.services.dataflow.model.ReportWorkItemStatusResponse; import com.google.api.services.dataflow.model.SendWorkerMessagesRequest; @@ -292,13 +293,30 @@ public WorkerMessage createWorkerMessageFromStreamingScalingReport( return msg; } - /** Reports the autoscaling signals to dataflow */ @Override - public void reportWorkerMessage(WorkerMessage msg) throws IOException { + public WorkerMessage createWorkerMessageFromPerWorkerMetrics(PerWorkerMetrics report) { + DateTime endTime = DateTime.now(); + logger.debug("Reporting WorkMessageResponse"); + Map labels = + ImmutableMap.of("JOB_ID", options.getJobId(), "WORKER_ID", options.getWorkerId()); + WorkerMessage msg = + new WorkerMessage() + .setTime(toCloudTime(endTime)) + .setPerWorkerMetrics(report) + .setLabels(labels); + return msg; + } + + /** + * Reports the worker messages to dataflow. We currently report autoscaling signals and + * perworkermetrics with this path. + */ + @Override + public void reportWorkerMessage(List messages) throws IOException { SendWorkerMessagesRequest request = new SendWorkerMessagesRequest() .setLocation(options.getRegion()) - .setWorkerMessages(Collections.singletonList(msg)); + .setWorkerMessages(messages); SendWorkerMessagesResponse result = dataflow .projects() diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index d915b77995db..f37504692d3e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -23,12 +23,15 @@ import com.google.api.services.dataflow.model.CounterUpdate; import com.google.api.services.dataflow.model.MapTask; +import com.google.api.services.dataflow.model.PerStepNamespaceMetrics; +import com.google.api.services.dataflow.model.PerWorkerMetrics; import com.google.api.services.dataflow.model.Status; import com.google.api.services.dataflow.model.StreamingComputationConfig; import com.google.api.services.dataflow.model.StreamingConfigTask; import com.google.api.services.dataflow.model.StreamingScalingReport; import com.google.api.services.dataflow.model.WorkItem; import com.google.api.services.dataflow.model.WorkItemStatus; +import com.google.api.services.dataflow.model.WorkerMessage; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.File; import java.io.IOException; @@ -117,6 +120,7 @@ import org.apache.beam.sdk.fn.IdGenerators; import org.apache.beam.sdk.fn.JvmInitializers; import org.apache.beam.sdk.io.FileSystems; +import org.apache.beam.sdk.io.gcp.bigquery.BigQuerySinkMetrics; import org.apache.beam.sdk.metrics.MetricName; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.util.BackOff; @@ -486,10 +490,13 @@ public static void main(String[] args) throws Exception { // metrics. MetricsEnvironment.setProcessWideContainer(new MetricsLogger(null)); - // When enabled, the Pipeline will record Per-Worker metrics that will be piped to WMW. + // When enabled, the Pipeline will record Per-Worker metrics that will be piped to DFE. StreamingStepMetricsContainer.setEnablePerWorkerMetrics( options.isEnableStreamingEngine() && DataflowRunner.hasExperiment(options, "enable_per_worker_metrics")); + // StreamingStepMetricsContainer automatically deletes perWorkerCounters if they are zero-valued + // for longer than 5 minutes. + BigQuerySinkMetrics.setSupportMetricsDeletion(true); JvmInitializers.runBeforeProcessing(options); worker.startStatusPages(); @@ -1777,7 +1784,7 @@ private void updateThreadMetrics() { maxOutstandingBundles.addValue((long) workUnitExecutor.maximumElementsOutstanding()); } - private void sendWorkerMessage() throws IOException { + private WorkerMessage createWorkerMessageForStreamingScalingReport() { StreamingScalingReport activeThreadsReport = new StreamingScalingReport() .setActiveThreadCount(workUnitExecutor.activeCount()) @@ -1786,8 +1793,33 @@ private void sendWorkerMessage() throws IOException { .setMaximumThreadCount(chooseMaximumNumberOfThreads()) .setMaximumBundleCount(workUnitExecutor.maximumElementsOutstanding()) .setMaximumBytes(workUnitExecutor.maximumBytesOutstanding()); - workUnitClient.reportWorkerMessage( - workUnitClient.createWorkerMessageFromStreamingScalingReport(activeThreadsReport)); + return workUnitClient.createWorkerMessageFromStreamingScalingReport(activeThreadsReport); + } + + private Optional createWorkerMessageForPerWorkerMetrics() { + List metrics = new ArrayList<>(); + stageInfoMap.values().forEach(s -> metrics.addAll(s.extractPerWorkerMetricValues())); + + if (metrics.isEmpty()) { + return Optional.empty(); + } + + PerWorkerMetrics perWorkerMetrics = new PerWorkerMetrics().setPerStepNamespaceMetrics(metrics); + return Optional.of(workUnitClient.createWorkerMessageFromPerWorkerMetrics(perWorkerMetrics)); + } + + private void sendWorkerMessage() throws IOException { + List workerMessages = new ArrayList(2); + workerMessages.add(createWorkerMessageForStreamingScalingReport()); + + if (StreamingStepMetricsContainer.getEnablePerWorkerMetrics()) { + Optional metricsMsg = createWorkerMessageForPerWorkerMetrics(); + if (metricsMsg.isPresent()) { + workerMessages.add(metricsMsg.get()); + } + } + + workUnitClient.reportWorkerMessage(workerMessages); } @VisibleForTesting diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java index af3b3e51071f..54a3ef49776b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingStepMetricsContainer.java @@ -98,7 +98,10 @@ protected StreamingStepMetricsContainer createContainer(String stepName) { }; } - // Testing constructor. + /** + * Construct a {@code StreamingStepMetricsContainer} that supports mock clock, perWorkerCounters, + * and perWorkerCountersByFirstStaleTime. For testing purposes only. + */ private StreamingStepMetricsContainer( String stepName, Map perWorkerCountersByFirstStaleTime, @@ -218,6 +221,9 @@ public static void setEnablePerWorkerMetrics(Boolean enablePerWorkerMetrics) { StreamingStepMetricsContainer.enablePerWorkerMetrics = enablePerWorkerMetrics; } + public static boolean getEnablePerWorkerMetrics() { + return StreamingStepMetricsContainer.enablePerWorkerMetrics; + } /** * Updates {@code perWorkerCountersByFirstStaleTime} with the current zero-valued metrics and * removes metrics that have been stale for longer than {@code maximumPerWorkerCounterStaleness} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkUnitClient.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkUnitClient.java index 6f26f4048070..f9637a375ed5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkUnitClient.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkUnitClient.java @@ -17,12 +17,14 @@ */ package org.apache.beam.runners.dataflow.worker; +import com.google.api.services.dataflow.model.PerWorkerMetrics; import com.google.api.services.dataflow.model.StreamingScalingReport; import com.google.api.services.dataflow.model.WorkItem; import com.google.api.services.dataflow.model.WorkItemServiceState; import com.google.api.services.dataflow.model.WorkItemStatus; import com.google.api.services.dataflow.model.WorkerMessage; import java.io.IOException; +import java.util.List; import java.util.Optional; /** Abstract base class describing a client for WorkItem work units. */ @@ -61,9 +63,18 @@ interface WorkUnitClient { WorkerMessage createWorkerMessageFromStreamingScalingReport(StreamingScalingReport report); /** - * Reports the autoscaling signals with a {@link StreamingScalingReport}. + * Creates a {@link WorkerMessage} containing the given PerWorkerMetrics * - * @param msg the WorkerMessage to report + * @param metrics Metric updates to send to the backend. + * @return a {@link WorkerMessage} + */ + WorkerMessage createWorkerMessageFromPerWorkerMetrics(PerWorkerMetrics metrics); + + /** + * Reports the worker messages to dataflow. We currently report autoscaling signals and + * perworkermetrics with this path. + * + * @param msg the WorkerMessages to report */ - void reportWorkerMessage(WorkerMessage msg) throws IOException; + void reportWorkerMessage(List messages) throws IOException; } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java index b514dfc84bb9..64c97dcac516 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/StageInfo.java @@ -21,6 +21,7 @@ import com.google.api.services.dataflow.model.CounterStructuredName; import com.google.api.services.dataflow.model.CounterUpdate; +import com.google.api.services.dataflow.model.PerStepNamespaceMetrics; import com.google.auto.value.AutoValue; import java.util.ArrayList; import java.util.List; @@ -111,4 +112,12 @@ private void translateKnownStepCounters(CounterUpdate stepCounterUpdate) { } } } + + public List extractPerWorkerMetricValues() { + List metrics = new ArrayList<>(); + Iterables.addAll( + metrics, + StreamingStepMetricsContainer.extractPerWorkerMetricUpdates(metricsContainerRegistry())); + return metrics; + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java index 7019217b9fbd..59d8c55c52d3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java @@ -31,6 +31,9 @@ import com.google.api.services.dataflow.model.LeaseWorkItemRequest; import com.google.api.services.dataflow.model.LeaseWorkItemResponse; import com.google.api.services.dataflow.model.MapTask; +import com.google.api.services.dataflow.model.MetricValue; +import com.google.api.services.dataflow.model.PerStepNamespaceMetrics; +import com.google.api.services.dataflow.model.PerWorkerMetrics; import com.google.api.services.dataflow.model.SendWorkerMessagesRequest; import com.google.api.services.dataflow.model.SendWorkerMessagesResponse; import com.google.api.services.dataflow.model.SeqMapTask; @@ -38,6 +41,7 @@ import com.google.api.services.dataflow.model.WorkItem; import com.google.api.services.dataflow.model.WorkerMessage; import java.io.IOException; +import java.util.Collections; import java.util.Optional; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC; @@ -232,7 +236,7 @@ public void testCloudServiceCallMultipleWorkItems() throws Exception { } @Test - public void testReportWorkerMessage() throws Exception { + public void testReportWorkerMessage_streamingScalingReport() throws Exception { MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); response.setContentType(Json.MEDIA_TYPE); SendWorkerMessagesResponse workerMessage = new SendWorkerMessagesResponse(); @@ -249,7 +253,7 @@ public void testReportWorkerMessage() throws Exception { .setMaximumBytes(6L); WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); WorkerMessage msg = client.createWorkerMessageFromStreamingScalingReport(activeThreadsReport); - client.reportWorkerMessage(msg); + client.reportWorkerMessage(Collections.singletonList(msg)); SendWorkerMessagesRequest actualRequest = Transport.getJsonFactory() @@ -257,6 +261,35 @@ public void testReportWorkerMessage() throws Exception { assertEquals(ImmutableList.of(msg), actualRequest.getWorkerMessages()); } + @Test + public void testReportWorkerMessage_perWorkerMetrics() throws Exception { + MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); + response.setContentType(Json.MEDIA_TYPE); + SendWorkerMessagesResponse workerMessage = new SendWorkerMessagesResponse(); + workerMessage.setFactory(Transport.getJsonFactory()); + response.setContent(workerMessage.toPrettyString()); + when(request.execute()).thenReturn(response); + PerStepNamespaceMetrics stepNamespaceMetrics = + new PerStepNamespaceMetrics() + .setOriginalStep("s1") + .setMetricsNamespace("ns") + .setMetricValues( + Collections.singletonList(new MetricValue().setMetric("metric").setValueInt64(3L))); + PerWorkerMetrics perWorkerMetrics = + new PerWorkerMetrics() + .setPerStepNamespaceMetrics(Collections.singletonList(stepNamespaceMetrics)); + + WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); + WorkerMessage perWorkerMetricsMsg = + client.createWorkerMessageFromPerWorkerMetrics(perWorkerMetrics); + client.reportWorkerMessage(Collections.singletonList(perWorkerMetricsMsg)); + + SendWorkerMessagesRequest actualRequest = + Transport.getJsonFactory() + .fromString(request.getContentAsString(), SendWorkerMessagesRequest.class); + assertEquals(ImmutableList.of(perWorkerMetricsMsg), actualRequest.getWorkerMessages()); + } + private LowLevelHttpResponse generateMockResponse(WorkItem... workItems) throws Exception { MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); response.setContentType(Json.MEDIA_TYPE); From 4f09e62f520d8a3bd40f9907e7343814cbd239ae Mon Sep 17 00:00:00 2001 From: Steven van Rossum Date: Wed, 31 Jan 2024 23:23:50 +0100 Subject: [PATCH 12/50] Improve varint encoding throughput with unrolled loop (#29689) * Improve varint encoding throughput with unrolled loop * Change BlackHole to Blackhole * Add single byte encode tests * Add missing L * Remove public modifier * Remove unused fields --- .../beam/sdk/jmh/util/VarIntBenchmark.java | 303 ++++++++++++++++++ .../java/org/apache/beam/sdk/util/VarInt.java | 62 +++- 2 files changed, 358 insertions(+), 7 deletions(-) create mode 100644 sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/jmh/util/VarIntBenchmark.java diff --git a/sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/jmh/util/VarIntBenchmark.java b/sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/jmh/util/VarIntBenchmark.java new file mode 100644 index 000000000000..964928d50558 --- /dev/null +++ b/sdks/java/core/jmh/src/main/java/org/apache/beam/sdk/jmh/util/VarIntBenchmark.java @@ -0,0 +1,303 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.sdk.jmh.util; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Random; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.openjdk.jmh.annotations.Benchmark; +import org.openjdk.jmh.annotations.Level; +import org.openjdk.jmh.annotations.OperationsPerInvocation; +import org.openjdk.jmh.annotations.Scope; +import org.openjdk.jmh.annotations.Setup; +import org.openjdk.jmh.annotations.State; +import org.openjdk.jmh.annotations.TearDown; +import org.openjdk.jmh.infra.Blackhole; + +/** Benchmarks for {@link org.apache.beam.sdk.util.VarInt} and variants. */ +@OperationsPerInvocation(VarIntBenchmark.VALUES_PER_INVOCATION) +public class VarIntBenchmark { + static final int VALUES_PER_INVOCATION = 2048; + private static final Random RNG = new Random(314159); + + /** Output to {@link Blackhole}. Do nothing, assume nothing. */ + @State(Scope.Benchmark) + public static class BlackholeOutput { + OutputStream stream; + + @Setup + public void setup(Blackhole bh) { + stream = + new OutputStream() { + @Override + public void write(int b) { + bh.consume(b); + } + + @Override + public void write(byte[] b) throws IOException { + bh.consume(b); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + bh.consume(b); + } + }; + } + } + + /** Output to {@link ByteStringOutputStream}. */ + @State(Scope.Thread) + public static class ByteStringOutput { + final ByteStringOutputStream stream = new ByteStringOutputStream(); + + // Unfortunately, this needs to be cleaned up after use to avoid OOMs. + // It's not generally recommended to use Level.Invocation, but there's no way around it. + @TearDown(Level.Invocation) + public void tearDown(Blackhole bh) { + bh.consume(stream.toByteStringAndReset()); + } + } + + /** Input from randomly generated bytes. */ + @State(Scope.Benchmark) + public static class Bytes { + long[] values = new long[VALUES_PER_INVOCATION]; + + @Setup + public void setup() { + values = new long[VALUES_PER_INVOCATION]; + byte[] bytes = new byte[VALUES_PER_INVOCATION]; + RNG.nextBytes(bytes); + + for (int i = 0; i < VALUES_PER_INVOCATION; i++) { + values[i] = (long) (bytes[i] & 0x7F); + } + } + } + + /** Input from randomly generated longs. */ + @State(Scope.Benchmark) + public static class Longs { + long[] values = new long[VALUES_PER_INVOCATION]; + + @Setup + public void setup() { + values = new long[VALUES_PER_INVOCATION]; + + for (int i = 0; i < VALUES_PER_INVOCATION; i++) { + // This gaussian random is used to determine the encoded output size of the sample. + // The distribution value is tweaked to favor small integers, positive more so than + // negative. + double g = RNG.nextGaussian(); + double s = 3; + g = 10 * Math.min(Math.abs(g < 0 ? g + s : g / (s / 2)), s) / s; + + // Construct a bitmask to keep up to numBits of the input. + // Find the lowest bit to set in the 7 bit segment below numBits. + int numBits = 7 * (int) g; + long mask = ~(~0x7fL << numBits); + long low = 1L << numBits; + + values[i] = (RNG.nextLong() & mask) | low; + } + } + } + + // Used in Beam 2.52.0 + static void encodeDoLoop(long v, OutputStream stream) throws IOException { + do { + // Encode next 7 bits + terminator bit + long bits = v & 0x7F; + v >>>= 7; + byte b = (byte) (bits | ((v != 0) ? 0x80 : 0)); + stream.write(b); + } while (v != 0); + } + + // A tweak of the above, replacing a compare with a few bitwise operations. + static void encodeDoLoopTwiddle(long v, OutputStream stream) throws IOException { + do { + // Encode next 7 bits + terminator bit + long bits = v & 0x7F; + v >>>= 7; + long cont = (-v >> 63) & 0x80; + byte b = (byte) (bits | cont); + stream.write(b); + } while (v != 0); + } + + // Use a mask check to do less work for 1 byte output. + static void encodeLoop(long v, OutputStream stream) throws IOException { + while ((v & ~0x7FL) != 0) { + stream.write((byte) (v | 0x80)); + v >>>= 7; + } + stream.write((byte) v); + } + + // As above, but unrolled. + static void encodeUnrolled(long v, OutputStream stream) throws IOException { + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + stream.write((byte) (v)); + } + + @Benchmark + public void encodeDoLoopBlackhole(Longs input, BlackholeOutput output) throws IOException { + for (long l : input.values) { + encodeDoLoop(l, output.stream); + } + } + + @Benchmark + public void encodeDoLoopByteString(Longs input, ByteStringOutput output) throws IOException { + for (long l : input.values) { + encodeDoLoop(l, output.stream); + } + } + + @Benchmark + public void encodeDoLoopTwiddleBlackhole(Longs input, BlackholeOutput output) throws IOException { + for (long l : input.values) { + encodeDoLoopTwiddle(l, output.stream); + } + } + + @Benchmark + public void encodeDoLoopTwiddleByteString(Longs input, ByteStringOutput output) + throws IOException { + for (long l : input.values) { + encodeDoLoopTwiddle(l, output.stream); + } + } + + @Benchmark + public void encodeLoopBlackhole(Longs input, BlackholeOutput output) throws IOException { + for (long l : input.values) { + encodeLoop(l, output.stream); + } + } + + @Benchmark + public void encodeLoopByteString(Longs input, ByteStringOutput output) throws IOException { + for (long l : input.values) { + encodeLoop(l, output.stream); + } + } + + @Benchmark + public void encodeUnrolledBlackhole(Longs input, BlackholeOutput output) throws IOException { + for (long l : input.values) { + encodeUnrolled(l, output.stream); + } + } + + @Benchmark + public void encodeUnrolledByteString(Longs input, ByteStringOutput output) throws IOException { + for (long l : input.values) { + encodeUnrolled(l, output.stream); + } + } + + @Benchmark + public void singleByteEncodeDoLoopByteString(Bytes input, ByteStringOutput output) + throws IOException { + for (long l : input.values) { + encodeDoLoop(l, output.stream); + } + } + + @Benchmark + public void singleByteEncodeDoLoopTwiddleByteString(Bytes input, ByteStringOutput output) + throws IOException { + for (long l : input.values) { + encodeDoLoopTwiddle(l, output.stream); + } + } + + @Benchmark + public void singleByteEncodeLoopByteString(Bytes input, ByteStringOutput output) + throws IOException { + for (long l : input.values) { + encodeLoop(l, output.stream); + } + } + + @Benchmark + public void singleByteEncodeUnrolledByteString(Bytes input, ByteStringOutput output) + throws IOException { + for (long l : input.values) { + encodeUnrolled(l, output.stream); + } + } +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/VarInt.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/VarInt.java index 239cbafc1743..5432383f5ad4 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/VarInt.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/VarInt.java @@ -42,13 +42,61 @@ public static void encode(int v, OutputStream stream) throws IOException { /** Encodes the given value onto the stream. */ public static void encode(long v, OutputStream stream) throws IOException { - do { - // Encode next 7 bits + terminator bit - long bits = v & 0x7F; - v >>>= 7; - byte b = (byte) (bits | ((v != 0) ? 0x80 : 0)); - stream.write(b); - } while (v != 0); + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + if ((v & ~0x7F) == 0) { + stream.write((byte) v); + return; + } + stream.write((byte) (v | 0x80)); + v >>>= 7; + stream.write((byte) (v)); } /** Decodes an integer value from the given stream. */ From 5188e37ed445bbe01ce7e71ee8177c2fde8c91ad Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 31 Jan 2024 22:14:09 -0500 Subject: [PATCH 13/50] Bump github.com/opencontainers/runc from 1.1.5 to 1.1.12 in /sdks (#30175) Bumps [github.com/opencontainers/runc](https://github.com/opencontainers/runc) from 1.1.5 to 1.1.12. - [Release notes](https://github.com/opencontainers/runc/releases) - [Changelog](https://github.com/opencontainers/runc/blob/v1.1.12/CHANGELOG.md) - [Commits](https://github.com/opencontainers/runc/compare/v1.1.5...v1.1.12) --- updated-dependencies: - dependency-name: github.com/opencontainers/runc dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 40 ++++++---------------------------------- 2 files changed, 8 insertions(+), 36 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index de7b36930455..80a017c09170 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -77,6 +77,7 @@ require ( dario.cat/mergo v1.0.0 // indirect github.com/Microsoft/hcsshim v0.11.1 // indirect github.com/containerd/log v0.1.0 // indirect + github.com/frankban/quicktest v1.14.0 // indirect github.com/go-logr/logr v1.3.0 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.2.6 // indirect @@ -155,7 +156,6 @@ require ( github.com/klauspost/asmfmt v1.3.2 // indirect github.com/klauspost/compress v1.17.4 // indirect github.com/klauspost/cpuid/v2 v2.2.6 // indirect - github.com/kr/text v0.2.0 // indirect github.com/magiconair/properties v1.8.7 // indirect github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect github.com/minio/c2goasm v0.0.0-20190812172519-36a3d3bbc4f3 // indirect @@ -166,7 +166,7 @@ require ( github.com/morikuni/aec v1.0.0 // indirect github.com/opencontainers/go-digest v1.0.0 // indirect github.com/opencontainers/image-spec v1.1.0-rc5 // indirect - github.com/opencontainers/runc v1.1.5 // indirect + github.com/opencontainers/runc v1.1.12 // indirect github.com/pierrec/lz4/v4 v4.1.15 // indirect github.com/pkg/errors v0.9.1 // indirect github.com/pkg/xattr v0.4.9 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 60b711ab0595..06b251d29a8d 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -138,11 +138,9 @@ github.com/census-instrumentation/opencensus-proto v0.4.1 h1:iKLQ0xPNFxR/2hzXZMr github.com/census-instrumentation/opencensus-proto v0.4.1/go.mod h1:4T9NM4+4Vw91VeyqjLS6ao50K5bOcLKN6Q42XnYaRYw= github.com/cespare/xxhash/v2 v2.2.0 h1:DC2CZ1Ep5Y4k3ZQ899DldepgrayRUGE6BBZ/cd9Cj44= github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/checkpoint-restore/go-criu/v5 v5.3.0/go.mod h1:E/eQpaFtUKGOOSEBZgmKAcn+zUUwWxqcaKZlF54wK8E= github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWRnGsAI= github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= -github.com/cilium/ebpf v0.7.0/go.mod h1:/oI2+1shJiTGAMgl6/RgJr36Eo1jzrRcAWbcXO2usCA= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cncf/udpa/go v0.0.0-20220112060539-c52dc94e7fbe h1:QQ3GSy+MqSHxm/d8nCtnAiZdYFd45cYZPs8vOOIYKfk= @@ -151,19 +149,15 @@ github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4 h1:/inchEIKaYC1Akx+H+gqO04wryn5h75LSazbRlnya1k= github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= -github.com/containerd/console v1.0.3/go.mod h1:7LqA/THxQ86k76b8c/EMSiaJ3h1eZkMkXar0TQ1gf3U= github.com/containerd/containerd v1.7.7 h1:QOC2K4A42RQpcrZyptP6z9EJZnlHfHJUfZrAAHe15q4= github.com/containerd/containerd v1.7.7/go.mod h1:3c4XZv6VeT9qgf9GMTxNTMFxGJrGpI2vz1yk4ye+YY8= github.com/containerd/log v0.1.0 h1:TCJt7ioM2cr/tfR8GPbGf9/VRAX8D2B4PjzCpfX540I= github.com/containerd/log v0.1.0/go.mod h1:VRRf09a7mHDIRezVKTRCrOq78v577GXq3bSa3EhrzVo= -github.com/coreos/go-systemd/v22 v22.3.2/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSVTIJ3seZv2GcEnc= github.com/cpuguy83/dockercfg v0.3.1 h1:/FpZ+JaygUR/lZP2NlFI2DVfrOEMAIKP5wWEJdoYe9E= github.com/cpuguy83/dockercfg v0.3.1/go.mod h1:sugsbF4//dDlL/i+S+rtpIWp+5h0BHJHfjj5/jFyUJc= -github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/cpuguy83/go-md2man/v2 v2.0.3/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= github.com/creack/pty v1.1.18 h1:n56/Zwd5o6whRC5PMGretI4IdRLlmBXYNjScPaBgsbY= -github.com/cyphar/filepath-securejoin v0.2.3/go.mod h1:aPGpWjXOXUn2NCNjFvBE6aRxGGx79pTxQpKOJNYHHl4= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= @@ -173,7 +167,6 @@ github.com/docker/docker v24.0.7+incompatible h1:Wo6l37AuwP3JaMnZa226lzVXGA3F9Ig github.com/docker/docker v24.0.7+incompatible/go.mod h1:eEKB0N0r5NX/I1kEveEz05bcu8tLC/8azJZsviup8Sk= github.com/docker/go-connections v0.4.0 h1:El9xVISelRB7BuFusrZozjnkIM5YnzCViNKohAFqRJQ= github.com/docker/go-connections v0.4.0/go.mod h1:Gbd7IOopHjR8Iph03tsViu4nIes5XhDvyHbTtUxmeec= -github.com/docker/go-units v0.4.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/docker/go-units v0.5.0 h1:69rxXcBk27SvSaaxTtLh/8llcHD8vYHT7WSdRZ/jvr4= github.com/docker/go-units v0.5.0/go.mod h1:fgPhTUdO+D/Jk86RDLlptpiXQzgHJF7gydDDbaIK4Dk= github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= @@ -190,8 +183,8 @@ github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2 github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/form3tech-oss/jwt-go v3.2.2+incompatible/go.mod h1:pbq4aXjuKjdthFRnoDwaVPLA+WlJuPGy+QneDUgJi2k= github.com/frankban/quicktest v1.2.2/go.mod h1:Qh/WofXFeiAFII1aEBu529AtJo6Zg2VHscnEsbBnJ20= -github.com/frankban/quicktest v1.11.3 h1:8sXhOn0uLys67V8EsXLc6eszDs8VXWxL3iRvebPhedY= -github.com/frankban/quicktest v1.11.3/go.mod h1:wRf/ReqHper53s+kmmSZizM8NamnL3IM0I9ntUbOk+k= +github.com/frankban/quicktest v1.14.0 h1:+cqqvzZV87b4adx/5ayVOaYZ2CrvM4ejQvUdBzPPUss= +github.com/frankban/quicktest v1.14.0/go.mod h1:NeW+ay9A/U67EYXNFA1nPE8e/tnQv/09mUdL/ijj8og= github.com/fsouza/fake-gcs-server v1.47.7 h1:56/U4rKY081TaNbq0gHWi7/71UxC2KROqcnrD9BRJhs= github.com/fsouza/fake-gcs-server v1.47.7/go.mod h1:4vPUynN8/zZlxk5Jpy6LvvTTxItdTAObK4DYnp89Jys= github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU= @@ -210,8 +203,6 @@ github.com/go-sql-driver/mysql v1.7.1 h1:lUIinVbN1DY0xBg0eMOzmmtGoHwWBbvnWubQUrt github.com/go-sql-driver/mysql v1.7.1/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= github.com/goccy/go-json v0.9.11 h1:/pAaQDLHEoCq/5FFmSKBswWmK6H0e8g4159Kc/X/nqk= github.com/goccy/go-json v0.9.11/go.mod h1:6MelG93GURQebXPDq3khkgXZkazVtN9CRI+MGFi0w8I= -github.com/godbus/dbus/v5 v5.0.4/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= -github.com/godbus/dbus/v5 v5.0.6/go.mod h1:xhWf0FNVPg57R7Z0UbKHbJfkEywrmjJnf7w5xrFpKfA= github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= @@ -333,6 +324,7 @@ github.com/klauspost/cpuid/v2 v2.2.6/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZY github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.2.1/go.mod h1:ipq/a2n7PKx3OHsz4KJII5eveXtPO4qwEXGdVfWzfnI= github.com/kr/pretty v0.3.0 h1:WgNl7dwNpEZ6jJ9k1snq4pZsg7DOEN8hP9Xw0Tsjwk0= +github.com/kr/pretty v0.3.0/go.mod h1:640gp4NfQd8pI5XOwp5fnNeVWj67G7CFk/SaSQn7NBk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= @@ -357,7 +349,6 @@ github.com/minio/minio-go/v7 v7.0.66 h1:bnTOXOHjOqv/gcMuiVbN9o2ngRItvqE774dG9nq0 github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= github.com/moby/patternmatcher v0.6.0 h1:GmP9lR19aU5GqSSFko+5pRqHi+Ohk1O69aFiKkVGiPk= github.com/moby/patternmatcher v0.6.0/go.mod h1:hDPoyOpDY7OrrMDLaYoY3hf52gNCR/YOUYxkhApJIxc= -github.com/moby/sys/mountinfo v0.5.0/go.mod h1:3bMD3Rg+zkqx8MRYPi7Pyb0Ie97QEBmdxbhnCLlSvSU= github.com/moby/sys/sequential v0.5.0 h1:OPvI35Lzn9K04PBbCLW0g4LcFAJgHsvXsRyewg5lXtc= github.com/moby/sys/sequential v0.5.0/go.mod h1:tH2cOOs5V9MlPiXcQzRC+eEyab644PWKGRYaaV5ZZlo= github.com/moby/term v0.5.0 h1:xt8Q1nalod/v7BqbG21f8mQPqH+xAaC9C3N3wfWbVP0= @@ -368,7 +359,6 @@ github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe h1:iruDEfMl2E6f github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= -github.com/mrunalp/fileutils v0.5.0/go.mod h1:M1WthSahJixYnrXQl/DFQuteStB1weuxD2QJNHXfbSQ= github.com/nats-io/jwt/v2 v2.5.3 h1:/9SWvzc6hTfamcgXJ3uYRpgj+QuY2aLNqRiqrKcrpEo= github.com/nats-io/jwt/v2 v2.5.3/go.mod h1:iysuPemFcc7p4IoYots3IuELSI4EDe9Y0bQMe+I3Bf4= github.com/nats-io/nats-server/v2 v2.10.7 h1:f5VDy+GMu7JyuFA0Fef+6TfulfCs5nBTgq7MMkFJx5Y= @@ -384,10 +374,8 @@ github.com/opencontainers/go-digest v1.0.0 h1:apOUWs51W5PlhuyGyz9FCeeBIOUDA/6nW8 github.com/opencontainers/go-digest v1.0.0/go.mod h1:0JzlMkj0TRzQZfJkVvzbP0HBR3IKzErnv2BNG4W4MAM= github.com/opencontainers/image-spec v1.1.0-rc5 h1:Ygwkfw9bpDvs+c9E34SdgGOj41dX/cbdlwvlWt0pnFI= github.com/opencontainers/image-spec v1.1.0-rc5/go.mod h1:X4pATf0uXsnn3g5aiGIsVnJBR4mxhKzfwmvK/B2NTm8= -github.com/opencontainers/runc v1.1.5 h1:L44KXEpKmfWDcS02aeGm8QNTFXTo2D+8MYGDIJ/GDEs= -github.com/opencontainers/runc v1.1.5/go.mod h1:1J5XiS+vdZ3wCyZybsuxXZWGrgSr8fFJHLXuG2PsnNg= -github.com/opencontainers/runtime-spec v1.0.3-0.20210326190908-1c3f411f0417/go.mod h1:jwyrGlmzljRJv/Fgzds9SsS/C5hL+LL3ko9hs6T5lQ0= -github.com/opencontainers/selinux v1.10.0/go.mod h1:2i0OySw99QjzBBQByd1Gr9gSjvuho1lHsJxIJ3gGbJI= +github.com/opencontainers/runc v1.1.12 h1:BOIssBaW1La0/qbNZHXOOa71dZfZEQOzW7dqQf3phss= +github.com/opencontainers/runc v1.1.12/go.mod h1:S+lQwSfncpBha7XTy/5lBwWgm5+y5Ma/O44Ekby9FK8= github.com/pborman/getopt v0.0.0-20180729010549-6fdd0a2c7117/go.mod h1:85jBQOZwpVEaDAr341tbn15RS4fCAsIst0qp7i8ex1o= github.com/pierrec/lz4/v4 v4.1.8/go.mod h1:gZWDp/Ze/IJXGXf23ltt2EXimqmTUXEy0GFuRQyBid4= github.com/pierrec/lz4/v4 v4.1.15 h1:MO0/ucJhngq7299dKLwIMtgTfbkoSPF6AoMYDd8Q4q0= @@ -406,13 +394,12 @@ github.com/proullon/ramsql v0.1.3/go.mod h1:CFGqeQHQpdRfWqYmWD3yXqPTEaHkF4zgXy1C github.com/rogpeppe/clock v0.0.0-20190514195947-2896927a307a h1:3QH7VyOaaiUHNrA9Se4YQIRkDTCw1EJls9xTUCaCeRM= github.com/rogpeppe/clock v0.0.0-20190514195947-2896927a307a/go.mod h1:4r5QyqhjIWCcK8DO4KMclc5Iknq5qVBAlbYYzAbUScQ= github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4= +github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc= github.com/rogpeppe/go-internal v1.9.0 h1:73kH8U+JUqXU8lRuOHeVHaa/SZPifC7BkcraZVejAe8= github.com/rs/xid v1.5.0 h1:mKX4bl4iPYJtEIxp6CYiUuLQ/8DYMoz0PUdtGgMFRVc= -github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46 h1:GHRpF1pTW19a8tTFrMLUcfWwyC0pnifVo2ClaLq+hP8= github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46/go.mod h1:uAQ5PCi+MFsC7HjREoAz1BU+Mq60+05gifQSsHSDG/8= -github.com/seccomp/libseccomp-golang v0.9.2-0.20220502022130-f33da4d89646/go.mod h1:JA8cRccbGaA1s33RQf7Y1+q9gHmZX1yB/z9WDN1C6fg= github.com/shabbyrobe/gocovmerge v0.0.0-20180507124511-f6ea450bfb63 h1:J6qvD6rbmOil46orKqJaRPG+zTpoGlBTUdyv8ki63L0= github.com/shabbyrobe/gocovmerge v0.0.0-20180507124511-f6ea450bfb63/go.mod h1:n+VKSARF5y/tS9XFSP7vWDfS+GUC5vs/YT7M5XDTUEM= github.com/shirou/gopsutil/v3 v3.23.9 h1:ZI5bWVeu2ep4/DIxB4U9okeYJ7zp/QLTO4auRb/ty/E= @@ -421,8 +408,6 @@ github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFt github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= github.com/shoenig/test v0.6.4/go.mod h1:byHiCGXqrVaflBLAMq/srcZIHynQPQgeyvkvXnjqq0k= -github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= -github.com/sirupsen/logrus v1.8.1/go.mod h1:yWOB1SBYBC5VeMP7gHvWumXLIWorT60ONWic61uBYv0= github.com/sirupsen/logrus v1.9.3 h1:dueUQJ1C2q9oE3F7wvmSGAaVtTmUizReu6fjN8uqzbQ= github.com/sirupsen/logrus v1.9.3/go.mod h1:naHLuLoDiP4jHNo9R0sCBMtWGeIprob74mVsIT4qYEQ= github.com/spf13/afero v1.2.1/go.mod h1:9ZxEEn6pIJ8Rxe320qSDBk6AsU0r9pR7Q4OcevTdifk= @@ -446,7 +431,6 @@ github.com/stretchr/testify v1.8.0/go.mod h1:yNjHg4UonilssWZ8iaSj1OCr/vHnekPRkoO github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o6fzry7u4= github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= -github.com/syndtr/gocapability v0.0.0-20200815063812-42c35b437635/go.mod h1:hkRG7XYTFWNJGYcbNJQlaLq0fg1yr4J4t/NcTQtrfww= github.com/testcontainers/testcontainers-go v0.26.0 h1:uqcYdoOHBy1ca7gKODfBd9uTHVK3a7UL848z09MVZ0c= github.com/testcontainers/testcontainers-go v0.26.0/go.mod h1:ICriE9bLX5CLxL9OFQ2N+2N+f+803LNJ1utJb1+Inx0= github.com/tetratelabs/wazero v1.5.0 h1:Yz3fZHivfDiZFUXnWMPUoiW7s8tC1sjdBtlJn08qYa0= @@ -455,9 +439,6 @@ github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFA github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= -github.com/urfave/cli v1.22.1/go.mod h1:Gos4lmkARVdJ6EkW0WaNv/tZAAMe9V7XWyB60NtXRu0= -github.com/vishvananda/netlink v1.1.0/go.mod h1:cTgwzPIzzgDAYoQrMm0EdrjRUBkTqKYppBueQtXaqoE= -github.com/vishvananda/netns v0.0.0-20191106174202-0a2b9b5464df/go.mod h1:JP3t17pCcGlemwknint6hfoeCVQrEMVwxRLRjXpq+BU= github.com/xdg-go/pbkdf2 v1.0.0 h1:Su7DPu48wXMwC3bs7MCNG+z4FhcyEuz5dlvchbq0B0c= github.com/xdg-go/pbkdf2 v1.0.0/go.mod h1:jrpuAogTd400dnrH08LKmI/xc1MbPOebTwRqcT5RDeI= github.com/xdg-go/scram v1.1.2 h1:FHX5I5B4i4hKRVRBCFRxq1iQRej7WO3hhBuJf+UUySY= @@ -566,7 +547,6 @@ golang.org/x/net v0.0.0-20200222125558-5a598a2470a0/go.mod h1:z5CRVTTTmAJ677TzLL golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= -golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= @@ -597,14 +577,11 @@ golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20190502145724-3ef323f4f1fd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190507160741-ecd444e8653b/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190606165138-5da285871e9c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20190606203320-7fc4e5ec1444/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190624142023-c5567b49c5d0/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190726091711-fc99dfbffb4e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191001151750-bb3f8db39f24/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191026070338-33540a1f6037/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191112214154-59a1497f0cea/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20191115151921-52ab43148777/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191204072324-ce4227a45e2e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20191228213918-04cbcbbfeed8/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200113162924-86b910548bc1/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -616,13 +593,9 @@ golang.org/x/sys v0.0.0-20200828194041-157a740278f4/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201204225414-ed752295db88/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= -golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210423082822-04245dca01da/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210615035016-665e8c7367d1/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20210616094352-59db8d763f22/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20210906170528-6f6e22806c34/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211025201205-69cdffdb9359/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.0.0-20211116061358-0a5406a5449c/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220408201424-a24fb2fb8a0f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220520151302-bc2c85ada10a/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.0.0-20220715151400-c0bba94af5f8/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= @@ -757,7 +730,6 @@ google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpAD google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.27.1/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= From 96cf695a10156208fbaaaff1d9714fe18fe11ae5 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 1 Feb 2024 10:14:02 -0500 Subject: [PATCH 14/50] Fix JdbcIO PreCommit mysql breaking change (#30179) --- .../apache/beam/sdk/io/jdbc/JdbcIOAutoPartitioningIT.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOAutoPartitioningIT.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOAutoPartitioningIT.java index 0042d93aedfc..dc94b65cf25a 100644 --- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOAutoPartitioningIT.java +++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcIOAutoPartitioningIT.java @@ -125,7 +125,10 @@ public void evaluate() throws Throwable { } }; - @ClassRule public static JdbcDatabaseContainer mysql = new MySQLContainer<>("mysql"); + // TODO(yathu) unpin tag when the fix of + // https://github.com/testcontainers/testcontainers-java/issues/8130 + // released and upgraded in Beam + @ClassRule public static JdbcDatabaseContainer mysql = new MySQLContainer<>("mysql:8.2"); @ClassRule public static JdbcDatabaseContainer postgres = new PostgreSQLContainer<>("postgres"); From 824e06cd967677a57e0f311df471089e1b3f2e10 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 1 Feb 2024 11:05:14 -0500 Subject: [PATCH 15/50] Allow stale_bq_datasets_cleaner.sh to run to completion, even when there are failures to grab a dataset (#30153) * Allow stale_bq_datasets_cleaner.sh to run to completion, even when there are failures * Less logging * Debug * Update stale_bq_datasets_cleaner.sh * Typo --- .../tools/stale_bq_datasets_cleaner.sh | 34 ++++++++++++------- 1 file changed, 21 insertions(+), 13 deletions(-) diff --git a/.test-infra/tools/stale_bq_datasets_cleaner.sh b/.test-infra/tools/stale_bq_datasets_cleaner.sh index 47e8ea95498e..c4afabe11e9a 100755 --- a/.test-infra/tools/stale_bq_datasets_cleaner.sh +++ b/.test-infra/tools/stale_bq_datasets_cleaner.sh @@ -18,7 +18,7 @@ # Deletes stale and old BQ datasets that are left after tests. # -set -euo pipefail +set -exuo pipefail PROJECT=apache-beam-testing MAX_RESULT=1500 @@ -36,19 +36,27 @@ for dataset in ${BQ_DATASETS[@]}; do if [[ $dataset =~ $template ]]; then # The BQ API reports LAST MODIFIED TIME in miliseconds, while unix works in seconds since epoch # thus why we need to convert to seconds. - [[ `bq --format=json --project_id=$PROJECT show $dataset` =~ \"lastModifiedTime\":\"([0-9]+)\" ]] - LAST_MODIFIED_MS=${BASH_REMATCH[1]} - LAST_MODIFIED=$(($LAST_MODIFIED_MS / 1000)) - if [[ $GRACE_PERIOD -gt $LAST_MODIFIED ]]; then - if bq --project_id=$PROJECT rm -r -f $dataset; then - if [[ $OSTYPE == "linux-gnu"* ]]; then - # date command usage depending on OS - echo "Deleted $dataset (modified `date -d @$LAST_MODIFIED`)" - elif [[ $OSTYPE == "darwin"* ]]; then - echo "Deleted $dataset (modified `date -r @$LAST_MODIFIED`)" + + failed=0 + ds=`bq --format=json --project_id=$PROJECT show $dataset` || failed=1 + if [[ $failed -eq 1 ]]; then + echo "Could not find dataset $dataset - it may have already been deleted, skipping" + else + [[ $ds =~ \"lastModifiedTime\":\"([0-9]+)\" ]] + LAST_MODIFIED_MS=${BASH_REMATCH[1]} + LAST_MODIFIED=$(($LAST_MODIFIED_MS / 1000)) + if [[ $GRACE_PERIOD -gt $LAST_MODIFIED ]]; then + if bq --project_id=$PROJECT rm -r -f $dataset; then + if [[ $OSTYPE == "linux-gnu"* ]]; then + # date command usage depending on OS + echo "Deleted $dataset (modified `date -d @$LAST_MODIFIED`)" + elif [[ $OSTYPE == "darwin"* ]]; then + echo "Deleted $dataset (modified `date -r @$LAST_MODIFIED`)" + fi + else + echo "Tried and failed to delete $dataset" + failed_calls+=1 fi - else - failed_calls+=1 fi fi break From 7fb05cbb5e1a55b668318001854890649060f5a4 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 31 Jan 2024 14:48:25 -0500 Subject: [PATCH 16/50] Move the code from sdks-java-fn-execution to sdks-java-core --- runners/core-construction-java/build.gradle | 1 - runners/core-java/build.gradle | 1 - runners/direct-java/build.gradle | 1 - runners/flink/flink_runner.gradle | 1 - .../google-cloud-dataflow-java/worker/build.gradle | 5 ----- runners/java-fn-execution/build.gradle | 1 - runners/java-job-service/build.gradle | 1 - runners/samza/build.gradle | 1 - runners/spark/spark_runner.gradle | 1 - sdks/java/core/build.gradle | 2 ++ .../org/apache/beam/sdk/fn/CancellableQueue.java | 0 .../java/org/apache/beam/sdk/fn/IdGenerator.java | 0 .../java/org/apache/beam/sdk/fn/IdGenerators.java | 0 .../org/apache/beam/sdk/fn/JvmInitializers.java | 0 .../sdk/fn/channel/AddHarnessIdInterceptor.java | 0 .../beam/sdk/fn/channel/ManagedChannelFactory.java | 0 .../beam/sdk/fn/channel/SocketAddressFactory.java | 0 .../apache/beam/sdk/fn/channel/package-info.java | 0 .../sdk/fn/data/BeamFnDataGrpcMultiplexer.java | 0 .../sdk/fn/data/BeamFnDataInboundObserver.java | 0 .../sdk/fn/data/BeamFnDataOutboundAggregator.java | 0 .../beam/sdk/fn/data/CloseableFnDataReceiver.java | 0 .../org/apache/beam/sdk/fn/data/DataEndpoint.java | 0 .../beam/sdk/fn/data/DecodingFnDataReceiver.java | 0 .../apache/beam/sdk/fn/data/FnDataReceiver.java | 0 .../apache/beam/sdk/fn/data/LogicalEndpoint.java | 0 .../beam/sdk/fn/data/RemoteGrpcPortRead.java | 0 .../beam/sdk/fn/data/RemoteGrpcPortWrite.java | 0 .../org/apache/beam/sdk/fn/data/TimerEndpoint.java | 0 .../org/apache/beam/sdk/fn/data/WeightedList.java | 0 .../org/apache/beam/sdk/fn/data/package-info.java | 0 .../java/org/apache/beam/sdk/fn/package-info.java | 0 .../org/apache/beam/sdk/fn/server/FnService.java | 0 .../server/GrpcContextHeaderAccessorProvider.java | 0 .../apache/beam/sdk/fn/server/GrpcFnServer.java | 0 .../apache/beam/sdk/fn/server/HeaderAccessor.java | 0 .../beam/sdk/fn/server/InProcessServerFactory.java | 0 .../apache/beam/sdk/fn/server/ServerFactory.java | 0 .../apache/beam/sdk/fn/server/package-info.java | 0 .../sdk/fn/splittabledofn/RestrictionTrackers.java | 0 .../sdk/fn/splittabledofn/WatermarkEstimators.java | 0 .../beam/sdk/fn/splittabledofn/package-info.java | 0 .../apache/beam/sdk/fn/stream/AdvancingPhaser.java | 0 .../sdk/fn/stream/BufferingStreamObserver.java | 0 .../org/apache/beam/sdk/fn/stream/DataStreams.java | 0 .../beam/sdk/fn/stream/DirectStreamObserver.java | 0 .../stream/ForwardingClientResponseObserver.java | 0 .../sdk/fn/stream/OutboundObserverFactory.java | 0 .../beam/sdk/fn/stream/PrefetchableIterable.java | 0 .../beam/sdk/fn/stream/PrefetchableIterables.java | 0 .../beam/sdk/fn/stream/PrefetchableIterator.java | 0 .../beam/sdk/fn/stream/PrefetchableIterators.java | 0 .../sdk/fn/stream/SynchronizedStreamObserver.java | 0 .../apache/beam/sdk/fn/stream/package-info.java | 0 .../org/apache/beam/sdk/fn/test/TestExecutors.java | 0 .../org/apache/beam/sdk/fn/test/TestStreams.java | 0 .../org/apache/beam/sdk/fn/test/package-info.java | 0 .../sdk/fn/windowing/EncodedBoundedWindow.java | 0 .../apache/beam/sdk/fn/windowing/package-info.java | 0 .../apache/beam/sdk/fn/CancellableQueueTest.java | 0 .../org/apache/beam/sdk/fn/IdGeneratorsTest.java | 0 .../apache/beam/sdk/fn/JvmInitializersTest.java | 5 +++-- .../sdk/fn/channel/ManagedChannelFactoryTest.java | 0 .../sdk/fn/channel/SocketAddressFactoryTest.java | 0 .../sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java | 0 .../sdk/fn/data/BeamFnDataInboundObserverTest.java | 14 +++++++++----- .../fn/data/BeamFnDataOutboundAggregatorTest.java | 0 .../beam/sdk/fn/data/RemoteGrpcPortReadTest.java | 0 .../beam/sdk/fn/data/RemoteGrpcPortWriteTest.java | 0 .../fn/splittabledofn/RestrictionTrackersTest.java | 0 .../fn/splittabledofn/WatermarkEstimatorsTest.java | 0 .../beam/sdk/fn/stream/AdvancingPhaserTest.java | 0 .../sdk/fn/stream/BufferingStreamObserverTest.java | 0 .../apache/beam/sdk/fn/stream/DataStreamsTest.java | 0 .../sdk/fn/stream/DirectStreamObserverTest.java | 0 .../ForwardingClientResponseObserverTest.java | 0 .../sdk/fn/stream/OutboundObserverFactoryTest.java | 0 .../sdk/fn/stream/PrefetchableIterablesTest.java | 0 .../sdk/fn/stream/PrefetchableIteratorsTest.java | 0 .../apache/beam/sdk/fn/test/TestExecutorsTest.java | 0 .../apache/beam/sdk/fn/test/TestStreamsTest.java | 0 .../sdk/fn/windowing/EncodedBoundedWindowTest.java | 0 sdks/java/fn-execution/build.gradle | 9 --------- sdks/java/harness/build.gradle | 1 - sdks/java/harness/jmh/build.gradle | 3 +++ .../beam/fn/harness/BeamFnDataWriteRunner.java | 1 - sdks/java/transform-service/build.gradle | 1 - 87 files changed, 17 insertions(+), 32 deletions(-) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/CancellableQueue.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/IdGenerator.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/IdGenerators.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/JvmInitializers.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/channel/AddHarnessIdInterceptor.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/channel/package-info.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/CloseableFnDataReceiver.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/DataEndpoint.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/DecodingFnDataReceiver.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/FnDataReceiver.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/LogicalEndpoint.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/TimerEndpoint.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/WeightedList.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/data/package-info.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/package-info.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/server/FnService.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/server/GrpcFnServer.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/server/HeaderAccessor.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/server/InProcessServerFactory.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/server/ServerFactory.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/server/package-info.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackers.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/splittabledofn/WatermarkEstimators.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/splittabledofn/package-info.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/stream/AdvancingPhaser.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterable.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterables.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterator.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterators.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/stream/package-info.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/test/TestExecutors.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/test/package-info.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java (100%) rename sdks/java/{fn-execution => core}/src/main/java/org/apache/beam/sdk/fn/windowing/package-info.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/CancellableQueueTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/IdGeneratorsTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/JvmInitializersTest.java (95%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java (95%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregatorTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackersTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/splittabledofn/WatermarkEstimatorsTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/stream/AdvancingPhaserTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserverTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/stream/DirectStreamObserverTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/stream/PrefetchableIterablesTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/stream/PrefetchableIteratorsTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/test/TestExecutorsTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/test/TestStreamsTest.java (100%) rename sdks/java/{fn-execution => core}/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java (100%) diff --git a/runners/core-construction-java/build.gradle b/runners/core-construction-java/build.gradle index 622bd7048e4b..20b5d936ee83 100644 --- a/runners/core-construction-java/build.gradle +++ b/runners/core-construction-java/build.gradle @@ -54,7 +54,6 @@ dependencies { implementation project(path: ":model:job-management", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":sdks:java:extensions:avro") - implementation project(path: ":sdks:java:fn-execution") implementation project(path: ":sdks:java:transform-service:launcher") implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre diff --git a/runners/core-java/build.gradle b/runners/core-java/build.gradle index b7dd51e72ed1..499cf06fcd6e 100644 --- a/runners/core-java/build.gradle +++ b/runners/core-java/build.gradle @@ -43,7 +43,6 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") implementation project(":runners:core-construction-java") - implementation project(":sdks:java:fn-execution") implementation library.java.vendored_guava_32_1_2_jre implementation library.java.joda_time implementation library.java.vendored_grpc_1_60_1 diff --git a/runners/direct-java/build.gradle b/runners/direct-java/build.gradle index 89bf6b30ed9e..e0365fb839ce 100644 --- a/runners/direct-java/build.gradle +++ b/runners/direct-java/build.gradle @@ -43,7 +43,6 @@ applyJavaNature( } relocate "org.apache.beam.runners.core", getJavaRelocatedPath("runners.core") relocate "org.apache.beam.runners.fnexecution", getJavaRelocatedPath("runners.fnexecution") - relocate "org.apache.beam.sdk.fn", getJavaRelocatedPath("sdk.fn") relocate "org.apache.beam.runners.local", getJavaRelocatedPath("runners.local") }, ) diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle index e7fd87067cdf..55303b3530d4 100644 --- a/runners/flink/flink_runner.gradle +++ b/runners/flink/flink_runner.gradle @@ -222,7 +222,6 @@ dependencies { implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") - implementation project(":sdks:java:fn-execution") implementation library.java.jackson_databind runtimeOnly library.java.jackson_jaxb_annotations examplesJavaIntegrationTest project(project.path) diff --git a/runners/google-cloud-dataflow-java/worker/build.gradle b/runners/google-cloud-dataflow-java/worker/build.gradle index 6a15dbd5531b..4fcd49c4a243 100644 --- a/runners/google-cloud-dataflow-java/worker/build.gradle +++ b/runners/google-cloud-dataflow-java/worker/build.gradle @@ -126,10 +126,6 @@ applyJavaNature( relocate("org.apache.beam.runners.fnexecution", getWorkerRelocatedPath("org.apache.beam.runners.fnexecution")) relocate("org.apache.beam.repackaged.beam_runners_java_fn_execution", getWorkerRelocatedPath("org.apache.beam.repackaged.beam_runners_java_fn_execution")) - dependencies { - include(project(":sdks:java:fn-execution")) - } - relocate("org.apache.beam.sdk.fn", getWorkerRelocatedPath("org.apache.beam.sdk.fn")) relocate("org.apache.beam.repackaged.beam_sdks_java_fn_execution", getWorkerRelocatedPath("org.apache.beam.repackaged.beam_sdks_java_fn_execution")) dependencies { @@ -193,7 +189,6 @@ dependencies { implementation project(":runners:core-construction-java") implementation project(":runners:core-java") implementation project(":runners:java-fn-execution") - implementation project(":sdks:java:fn-execution") implementation project(path: ":runners:google-cloud-dataflow-java:worker:windmill", configuration: "shadow") shadow library.java.vendored_guava_32_1_2_jre diff --git a/runners/java-fn-execution/build.gradle b/runners/java-fn-execution/build.gradle index e055bc991e2a..5e19e4cba102 100644 --- a/runners/java-fn-execution/build.gradle +++ b/runners/java-fn-execution/build.gradle @@ -30,7 +30,6 @@ dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(":sdks:java:fn-execution") implementation project(":runners:core-construction-java") implementation library.java.vendored_grpc_1_60_1 implementation library.java.slf4j_api diff --git a/runners/java-job-service/build.gradle b/runners/java-job-service/build.gradle index a1b4aba2e908..e3d26ece4765 100644 --- a/runners/java-job-service/build.gradle +++ b/runners/java-job-service/build.gradle @@ -29,7 +29,6 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") implementation project(":sdks:java:expansion-service") - implementation project(":sdks:java:fn-execution") implementation project(":runners:core-construction-java") implementation project(":runners:java-fn-execution") implementation library.java.jackson_core diff --git a/runners/samza/build.gradle b/runners/samza/build.gradle index 4e410fedeccc..fbbec322c87b 100644 --- a/runners/samza/build.gradle +++ b/runners/samza/build.gradle @@ -70,7 +70,6 @@ dependencies { implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") implementation project(path: ":model:pipeline", configuration: "shadow") - implementation project(":sdks:java:fn-execution") testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation project(path: ":runners:core-java", configuration: "testRuntimeMigration") testImplementation library.java.hamcrest diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index 3da819ad2f37..5e74ccd2b195 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -172,7 +172,6 @@ dependencies { implementation library.java.args4j implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(path: ":model:job-management", configuration: "shadow") - implementation project(":sdks:java:fn-execution") implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre spark.components.each { component -> diff --git a/sdks/java/core/build.gradle b/sdks/java/core/build.gradle index 64aae685c380..9de7e0da1eb3 100644 --- a/sdks/java/core/build.gradle +++ b/sdks/java/core/build.gradle @@ -75,6 +75,7 @@ dependencies { permitUnusedDeclared library.java.antlr // Required to load constants from the model, e.g. max timestamp for global window shadow project(path: ":model:pipeline", configuration: "shadow") + shadow project(path: ":model:fn-execution", configuration: "shadow") shadow project(path: ":model:job-management", configuration: "shadow") shadow library.java.vendored_grpc_1_60_1 shadow library.java.vendored_guava_32_1_2_jre @@ -119,6 +120,7 @@ dependencies { shadowTest library.java.log4j2_api shadowTest library.java.jamm testRuntimeOnly library.java.slf4j_jdk14 + testImplementation "com.github.stefanbirkner:system-rules:1.19.0" } project.tasks.compileTestJava { diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/CancellableQueue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/CancellableQueue.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/CancellableQueue.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/CancellableQueue.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/IdGenerator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/IdGenerator.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/IdGenerator.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/IdGenerator.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/IdGenerators.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/IdGenerators.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/IdGenerators.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/IdGenerators.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/JvmInitializers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/JvmInitializers.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/JvmInitializers.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/JvmInitializers.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/AddHarnessIdInterceptor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/channel/AddHarnessIdInterceptor.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/AddHarnessIdInterceptor.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/channel/AddHarnessIdInterceptor.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactory.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/channel/SocketAddressFactory.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/channel/package-info.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/channel/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/channel/package-info.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexer.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserver.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregator.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/CloseableFnDataReceiver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/CloseableFnDataReceiver.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/CloseableFnDataReceiver.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/CloseableFnDataReceiver.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/DataEndpoint.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/DataEndpoint.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/DataEndpoint.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/DataEndpoint.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/DecodingFnDataReceiver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/DecodingFnDataReceiver.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/DecodingFnDataReceiver.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/DecodingFnDataReceiver.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/FnDataReceiver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/FnDataReceiver.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/FnDataReceiver.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/FnDataReceiver.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/LogicalEndpoint.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/LogicalEndpoint.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/LogicalEndpoint.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/LogicalEndpoint.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortRead.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWrite.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/TimerEndpoint.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/TimerEndpoint.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/TimerEndpoint.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/TimerEndpoint.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/WeightedList.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/WeightedList.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/WeightedList.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/WeightedList.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/package-info.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/data/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/data/package-info.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/package-info.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/package-info.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/FnService.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/FnService.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/FnService.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/FnService.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/GrpcContextHeaderAccessorProvider.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/GrpcFnServer.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/GrpcFnServer.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/GrpcFnServer.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/GrpcFnServer.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/HeaderAccessor.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/HeaderAccessor.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/HeaderAccessor.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/HeaderAccessor.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/InProcessServerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/InProcessServerFactory.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/InProcessServerFactory.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/InProcessServerFactory.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/ServerFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/ServerFactory.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/ServerFactory.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/ServerFactory.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/package-info.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/server/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/server/package-info.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackers.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackers.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackers.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackers.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/splittabledofn/WatermarkEstimators.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/splittabledofn/WatermarkEstimators.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/splittabledofn/WatermarkEstimators.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/splittabledofn/WatermarkEstimators.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/splittabledofn/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/splittabledofn/package-info.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/splittabledofn/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/splittabledofn/package-info.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/AdvancingPhaser.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/AdvancingPhaser.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/AdvancingPhaser.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/AdvancingPhaser.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserver.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/DataStreams.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/DirectStreamObserver.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserver.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactory.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterable.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterable.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterable.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterable.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterables.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterables.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterables.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterables.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterator.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterator.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterator.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterator.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterators.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterators.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterators.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/PrefetchableIterators.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/SynchronizedStreamObserver.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/package-info.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/stream/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/stream/package-info.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestExecutors.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/test/TestExecutors.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestExecutors.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/test/TestExecutors.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/test/TestStreams.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/test/package-info.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/test/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/test/package-info.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindow.java diff --git a/sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/windowing/package-info.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/fn/windowing/package-info.java similarity index 100% rename from sdks/java/fn-execution/src/main/java/org/apache/beam/sdk/fn/windowing/package-info.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/fn/windowing/package-info.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/CancellableQueueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/CancellableQueueTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/CancellableQueueTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/CancellableQueueTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/IdGeneratorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/IdGeneratorsTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/IdGeneratorsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/IdGeneratorsTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/JvmInitializersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/JvmInitializersTest.java similarity index 95% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/JvmInitializersTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/JvmInitializersTest.java index e1da00ae94dd..7820467987ef 100644 --- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/JvmInitializersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/JvmInitializersTest.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.fn; -import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.StringContains.containsString; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; @@ -27,6 +26,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.testing.TestPipeline; +import org.hamcrest.MatcherAssert; import org.junit.Before; import org.junit.Rule; import org.junit.Test; @@ -72,7 +72,8 @@ public void runOnStartup_runsInitializers() { JvmInitializers.runOnStartup(); assertTrue(onStartupRan); - assertThat(systemOutRule.getLog(), containsString("Running JvmInitializer#onStartup")); + MatcherAssert.assertThat( + systemOutRule.getLog(), containsString("Running JvmInitializer#onStartup")); } @Test diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/channel/ManagedChannelFactoryTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/channel/SocketAddressFactoryTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataGrpcMultiplexerTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java similarity index 95% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java index 137c2f890fb5..bd279cee208a 100644 --- a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java @@ -17,7 +17,6 @@ */ package org.apache.beam.sdk.fn.data; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.junit.Assert.assertSame; @@ -94,8 +93,13 @@ public void testConsumptionOfValuesHappensOnAwaitCompletionCallersThread() throw assertThat( values, contains( - valueInGlobalWindow("ABC"), valueInGlobalWindow("DEF"), valueInGlobalWindow("GHI"))); - assertThat(timers, contains(valueInGlobalWindow("UVW"), valueInGlobalWindow("XYZ"))); + WindowedValue.valueInGlobalWindow("ABC"), + WindowedValue.valueInGlobalWindow("DEF"), + WindowedValue.valueInGlobalWindow("GHI"))); + assertThat( + timers, + contains( + WindowedValue.valueInGlobalWindow("UVW"), WindowedValue.valueInGlobalWindow("XYZ"))); future.get(); } @@ -205,7 +209,7 @@ public void testBadProducerDataFailureVisibleToAwaitCompletionCallerAndProducer( private BeamFnApi.Elements dataWith(String... values) throws Exception { ByteStringOutputStream output = new ByteStringOutputStream(); for (String value : values) { - CODER.encode(valueInGlobalWindow(value), output); + CODER.encode(WindowedValue.valueInGlobalWindow(value), output); } return BeamFnApi.Elements.newBuilder() .addData( @@ -224,7 +228,7 @@ private BeamFnApi.Elements lastData() throws Exception { private BeamFnApi.Elements timerWith(String... values) throws Exception { ByteStringOutputStream output = new ByteStringOutputStream(); for (String value : values) { - CODER.encode(valueInGlobalWindow(value), output); + CODER.encode(WindowedValue.valueInGlobalWindow(value), output); } return BeamFnApi.Elements.newBuilder() .addTimers( diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregatorTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregatorTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregatorTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataOutboundAggregatorTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortReadTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/RemoteGrpcPortWriteTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackersTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackersTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/splittabledofn/RestrictionTrackersTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/splittabledofn/WatermarkEstimatorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/splittabledofn/WatermarkEstimatorsTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/splittabledofn/WatermarkEstimatorsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/splittabledofn/WatermarkEstimatorsTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/AdvancingPhaserTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/AdvancingPhaserTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/AdvancingPhaserTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/AdvancingPhaserTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserverTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserverTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserverTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/BufferingStreamObserverTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/DataStreamsTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DirectStreamObserverTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/DirectStreamObserverTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/DirectStreamObserverTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/DirectStreamObserverTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/ForwardingClientResponseObserverTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/OutboundObserverFactoryTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/PrefetchableIterablesTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/PrefetchableIterablesTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/PrefetchableIterablesTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/PrefetchableIterablesTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/PrefetchableIteratorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/PrefetchableIteratorsTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/stream/PrefetchableIteratorsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/stream/PrefetchableIteratorsTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/test/TestExecutorsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/test/TestExecutorsTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/test/TestExecutorsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/test/TestExecutorsTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/test/TestStreamsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/test/TestStreamsTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/test/TestStreamsTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/test/TestStreamsTest.java diff --git a/sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java similarity index 100% rename from sdks/java/fn-execution/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java rename to sdks/java/core/src/test/java/org/apache/beam/sdk/fn/windowing/EncodedBoundedWindowTest.java diff --git a/sdks/java/fn-execution/build.gradle b/sdks/java/fn-execution/build.gradle index d1b5c713d092..01a073bb7ae7 100644 --- a/sdks/java/fn-execution/build.gradle +++ b/sdks/java/fn-execution/build.gradle @@ -26,18 +26,9 @@ ext.summary = """Contains code shared across the Beam Java SDK Harness and Java the Beam Portability Framework.""" dependencies { - implementation project(path: ":model:pipeline", configuration: "shadow") - implementation project(path: ":model:fn-execution", configuration: "shadow") - implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation library.java.vendored_grpc_1_60_1 - implementation library.java.vendored_guava_32_1_2_jre - implementation library.java.slf4j_api - implementation library.java.joda_time - provided library.java.junit testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation library.java.junit testImplementation library.java.mockito_core testImplementation library.java.commons_lang3 - testImplementation "com.github.stefanbirkner:system-rules:1.19.0" testRuntimeOnly library.java.slf4j_jdk14 } diff --git a/sdks/java/harness/build.gradle b/sdks/java/harness/build.gradle index 505cbb38ab13..52c20e62c498 100644 --- a/sdks/java/harness/build.gradle +++ b/sdks/java/harness/build.gradle @@ -87,7 +87,6 @@ dependencies { implementation project(path: ":model:fn-execution", configuration: "shadow") implementation project(":runners:core-construction-java") implementation project(":runners:core-java") - implementation project(":sdks:java:fn-execution") permitUnusedDeclared project(path: ":sdks:java:transform-service:launcher") testImplementation library.java.junit testImplementation library.java.mockito_core diff --git a/sdks/java/harness/jmh/build.gradle b/sdks/java/harness/jmh/build.gradle index 90e752947172..d3b33929d58c 100644 --- a/sdks/java/harness/jmh/build.gradle +++ b/sdks/java/harness/jmh/build.gradle @@ -35,12 +35,15 @@ dependencies { implementation project(path: ":sdks:java:harness", configuration: "shadow") implementation project(path: ":runners:java-fn-execution") implementation project(path: ":model:pipeline", configuration: "shadow") + implementation project(path: ":model:fn-execution", configuration: "shadow") implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre implementation library.java.slf4j_api implementation library.java.joda_time + implementation library.java.junit runtimeOnly library.java.slf4j_jdk14 jammAgent library.java.jamm + testImplementation library.java.junit } jmh { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java index c47d86cd645e..310e9ef50834 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java @@ -23,7 +23,6 @@ import java.io.IOException; import java.util.Map; import java.util.function.Supplier; -import org.apache.beam.fn.harness.PTransformRunnerFactory.Context; import org.apache.beam.fn.harness.state.BeamFnStateClient; import org.apache.beam.fn.harness.state.StateBackedIterable.StateBackedIterableTranslationContext; import org.apache.beam.model.fnexecution.v1.BeamFnApi; diff --git a/sdks/java/transform-service/build.gradle b/sdks/java/transform-service/build.gradle index 702479fdf30a..1d72c1d424bb 100644 --- a/sdks/java/transform-service/build.gradle +++ b/sdks/java/transform-service/build.gradle @@ -42,7 +42,6 @@ dependencies { implementation project(path: ":model:job-management", configuration: "shadow") implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":runners:core-construction-java") - implementation project(path: ":sdks:java:fn-execution") implementation library.java.vendored_grpc_1_60_1 implementation library.java.vendored_guava_32_1_2_jre implementation library.java.jackson_annotations From 1d9f6044c7671597964bf367053a69e616a1209d Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 1 Feb 2024 12:42:36 -0500 Subject: [PATCH 17/50] Remove constraint introduced in #30059 (#30186) --- .../beam_PostCommit_Python_Xlang_Gcp_Dataflow.json | 0 .../BigQueryStorageWriteApiSchemaTransformProvider.java | 7 ------- 2 files changed, 7 deletions(-) delete mode 100644 .github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json diff --git a/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json b/.github/trigger_files/beam_PostCommit_Python_Xlang_Gcp_Dataflow.json deleted file mode 100644 index e69de29bb2d1..000000000000 diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java index d0951cdad1a6..ca0d0a2784bd 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/providers/BigQueryStorageWriteApiSchemaTransformProvider.java @@ -383,13 +383,6 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { Boolean autoSharding = configuration.getAutoSharding(); int numStreams = configuration.getNumStreams() == null ? 0 : configuration.getNumStreams(); - // TODO(https://github.com/apache/beam/issues/30058): remove once Dataflow supports multiple - // DoFn's per fused step. - if (numStreams < 1) { - throw new IllegalStateException( - "numStreams must be set to a positive integer when input data is unbounded."); - } - boolean useAtLeastOnceSemantics = configuration.getUseAtLeastOnceSemantics() != null && configuration.getUseAtLeastOnceSemantics(); From bdbbef9b54dea5f76a2836047f5c56645b6f0d10 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Ferran=20Fern=C3=A1ndez=20Garrido?= Date: Thu, 1 Feb 2024 19:07:37 +0100 Subject: [PATCH 18/50] [YAML] - PubSubLite proto (#30129) * [YAML] - PubSubLite proto * [YAML] - PubSubLite proto --- sdks/java/extensions/protobuf/build.gradle | 1 + .../extensions/protobuf/ProtoByteUtils.java | 79 ++++++++++++++-- .../protobuf/ProtoByteUtilsTest.java | 72 +++++++++++++++ ...PubsubLiteReadSchemaTransformProvider.java | 84 +++++++++++++---- ...ubsubLiteWriteSchemaTransformProvider.java | 51 ++++++++++- .../internal/PubsubLiteDlqTest.java | 89 +++++++++++++++++++ .../internal/PubsubLiteWriteDlqTest.java | 31 +++++++ sdks/python/apache_beam/yaml/standard_io.yaml | 8 ++ 8 files changed, 392 insertions(+), 23 deletions(-) diff --git a/sdks/java/extensions/protobuf/build.gradle b/sdks/java/extensions/protobuf/build.gradle index 1582492c293e..088f5aca63ca 100644 --- a/sdks/java/extensions/protobuf/build.gradle +++ b/sdks/java/extensions/protobuf/build.gradle @@ -38,6 +38,7 @@ dependencies { implementation library.java.commons_compress implementation library.java.slf4j_api implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation("com.google.cloud:google-cloud-storage:2.32.1") implementation library.java.protobuf_java implementation("com.squareup.wire:wire-schema-jvm:4.9.3") implementation("io.apicurio:apicurio-registry-protobuf-schema-utilities:3.0.0.M2") diff --git a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtils.java b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtils.java index 02419ec0f619..dd73739246d0 100644 --- a/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtils.java +++ b/sdks/java/extensions/protobuf/src/main/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtils.java @@ -20,6 +20,9 @@ import static java.util.stream.Collectors.toList; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; +import com.google.cloud.storage.Blob; +import com.google.cloud.storage.Storage; +import com.google.cloud.storage.StorageOptions; import com.google.protobuf.DescriptorProtos; import com.google.protobuf.Descriptors; import com.google.protobuf.DynamicMessage; @@ -41,15 +44,15 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.values.Row; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.commons.compress.utils.IOUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** - * Utility class for working with Protocol Buffer (Proto) data in the context of Apache Beam. This - * class provides methods to retrieve Beam Schemas from Proto messages, convert Proto bytes to Beam - * Rows, and vice versa. It also includes utilities for handling Protocol Buffer schemas and related - * file operations. + * Utility class for working with Protocol Buffer (Proto) data. This class provides methods to + * retrieve Beam Schemas from Proto messages, convert Proto bytes to Beam Rows, and vice versa. It + * also includes utilities for handling Protocol Buffer schemas and related file operations. * *

Users can utilize the methods in this class to facilitate the integration of Proto data * processing within Apache Beam pipelines, allowing for the seamless transformation of Proto @@ -105,7 +108,11 @@ private static Descriptors.Descriptor getDescriptorFromProtoSchema( try { Descriptors.FileDescriptor fileDescriptor = FileDescriptorUtils.protoFileToFileDescriptor(result); - return fileDescriptor.findMessageTypeByName(messageName); + + List messageElements = Splitter.on('.').splitToList(messageName); + String messageTypeByName = messageElements.get(messageElements.size() - 1); + + return fileDescriptor.findMessageTypeByName(messageTypeByName); } catch (Descriptors.DescriptorValidationException e) { throw new RuntimeException(e); } @@ -148,10 +155,12 @@ public static SerializableFunction getProtoBytesToRowFunction( @Override public Row apply(byte[] input) { try { + List messageElements = Splitter.on('.').splitToList(messageName); + String messageTypeByName = messageElements.get(messageElements.size() - 1); final Descriptors.Descriptor descriptor = protoDomain .getFileDescriptor(dynamicProtoDomain.getFileName()) - .findMessageTypeByName(messageName); + .findMessageTypeByName(messageTypeByName); DynamicMessage dynamicMessage = DynamicMessage.parseFrom(descriptor, input); SerializableFunction res = protoDynamicMessageSchema.getToRowFunction(); @@ -243,6 +252,41 @@ private static byte[] getFileAsBytes(String fileDescriptorPath) { * @throws RuntimeException if an error occurs while finding or opening the file. */ private static ReadableByteChannel getFileByteChannel(String filePath) { + if (isGcsPath(filePath)) { + return openGcsFile(filePath); + } else { + return openLocalFile(filePath); + } + } + + private static boolean isGcsPath(String filePath) { + return filePath.startsWith("gs://"); + } + + /** + * Opens a ReadableByteChannel for reading from a Google Cloud Storage (GCS) file. + * + * @param filePath The GCS file path (e.g., "gs://your-bucket-name/your-object-name"). + * @return A ReadableByteChannel for reading from the specified GCS file. + */ + private static ReadableByteChannel openGcsFile(String filePath) { + Storage storage = StorageOptions.getDefaultInstance().getService(); + String bucketName = getBucketName(filePath); + String objectName = getObjectName(filePath); + Blob blob = storage.get(bucketName, objectName); + return blob.reader(); + } + + /** + * Opens a ReadableByteChannel for reading from a local file using the Apache Beam FileSystems + * API. + * + * @param filePath The local file path. + * @return A ReadableByteChannel for reading from the specified local file. + * @throws IllegalArgumentException If no files match the specified pattern or if more than one + * file matches. + */ + private static ReadableByteChannel openLocalFile(String filePath) { try { MatchResult result = FileSystems.match(filePath); checkArgument( @@ -259,6 +303,29 @@ private static ReadableByteChannel getFileByteChannel(String filePath) { } } + /** + * Extracts the bucket name from a Google Cloud Storage (GCS) file path. + * + * @param gcsPath The GCS file path (e.g., "gs://your-bucket-name/your-object-name"). + * @return The bucket name extracted from the GCS path. + */ + private static String getBucketName(String gcsPath) { + int startIndex = "gs://".length(); + int endIndex = gcsPath.indexOf('/', startIndex); + return gcsPath.substring(startIndex, endIndex); + } + + /** + * Extracts the object name from a Google Cloud Storage (GCS) file path. + * + * @param gcsPath The GCS file path (e.g., "gs://your-bucket-name/your-object-name"). + * @return The object name extracted from the GCS path. + */ + private static String getObjectName(String gcsPath) { + int startIndex = gcsPath.indexOf('/', "gs://".length()) + 1; + return gcsPath.substring(startIndex); + } + /** * Represents metadata associated with a Protocol Buffer schema, including the File Name and * ProtoDomain. diff --git a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtilsTest.java b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtilsTest.java index 04bcde6a0fe0..6105208d8366 100644 --- a/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtilsTest.java +++ b/sdks/java/extensions/protobuf/src/test/java/org/apache/beam/sdk/extensions/protobuf/ProtoByteUtilsTest.java @@ -48,6 +48,26 @@ public class ProtoByteUtilsTest { + " Address address = 4;\n" + "}"; + private static final String PROTO_STRING_PACKAGE_SCHEMA = + "syntax = \"proto3\";\n" + + "package com.test.proto;" + + "\n" + + "message MyMessage {\n" + + " int32 id = 1;\n" + + " string name = 2;\n" + + " bool active = 3;\n" + + "\n" + + " // Nested field\n" + + " message Address {\n" + + " string street = 1;\n" + + " string city = 2;\n" + + " string state = 3;\n" + + " string zip_code = 4;\n" + + " }\n" + + "\n" + + " Address address = 4;\n" + + "}"; + private static final String DESCRIPTOR_PATH = Objects.requireNonNull( ProtoByteUtilsTest.class.getResource( @@ -84,6 +104,14 @@ public void testProtoSchemaStringToBeamSchema() { Assert.assertEquals(schema.getFieldNames(), SCHEMA.getFieldNames()); } + @Test + public void testProtoSchemaWitPackageStringToBeamSchema() { + Schema schema = + ProtoByteUtils.getBeamSchemaFromProtoSchema( + PROTO_STRING_PACKAGE_SCHEMA, "com.test.proto.MyMessage"); + Assert.assertEquals(schema.getFieldNames(), SCHEMA.getFieldNames()); + } + @Test public void testProtoBytesToRowFunctionGenerateSerializableFunction() { SerializableFunction protoBytesToRowFunction = @@ -111,6 +139,22 @@ public void testProtoBytesToRowFunctionReturnsRowFailure() { protoBytesToRowFunction.apply(inputBytes); } + @Test + public void testProtoBytesToRowFunctionReturnsRowSuccess() { + // Create a proto bytes to row function + SerializableFunction protoBytesToRowFunction = + ProtoByteUtils.getProtoBytesToRowFunction(DESCRIPTOR_PATH, MESSAGE_NAME); + + byte[] byteArray = { + 8, -46, 9, 18, 3, 68, 111, 101, 34, 35, 10, 7, 115, 101, 97, 116, 116, 108, 101, 18, 11, 102, + 97, 107, 101, 32, 115, 116, 114, 101, 101, 116, 26, 2, 119, 97, 34, 7, 84, 79, 45, 49, 50, 51, + 52 + }; + + Row row = protoBytesToRowFunction.apply(byteArray); + Assert.assertEquals("Doe", row.getValue("name")); + } + @Test public void testRowToProtoFunction() { Row row = @@ -144,4 +188,32 @@ public void testRowToProtoSchemaFunction() { Assert.assertNotNull( ProtoByteUtils.getRowToProtoBytesFromSchema(PROTO_STRING_SCHEMA, "MyMessage").apply(row)); } + + @Test + public void testRowToProtoSchemaWithPackageFunction() { + Row row = + Row.withSchema(SCHEMA) + .withFieldValue("id", 1234) + .withFieldValue("name", "Doe") + .withFieldValue("active", false) + .withFieldValue("address.city", "seattle") + .withFieldValue("address.street", "fake street") + .withFieldValue("address.zip_code", "TO-1234") + .withFieldValue("address.state", "wa") + .build(); + + byte[] byteArray = { + 8, -46, 9, 18, 3, 68, 111, 101, 34, 35, 10, 7, 115, 101, 97, 116, 116, 108, 101, 18, 11, 102, + 97, 107, 101, 32, 115, 116, 114, 101, 101, 116, 26, 2, 119, 97, 34, 7, 84, 79, 45, 49, 50, 51, + 52 + }; + + byte[] resultBytes = + ProtoByteUtils.getRowToProtoBytesFromSchema( + PROTO_STRING_PACKAGE_SCHEMA, "com.test.proto.MyMessage") + .apply(row); + + Assert.assertNotNull(resultBytes); + Assert.assertArrayEquals(byteArray, resultBytes); + } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java index 98c5f8a6b388..8afe730f32ce 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteReadSchemaTransformProvider.java @@ -38,6 +38,7 @@ import java.util.function.Consumer; import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; import org.apache.beam.sdk.extensions.gcp.options.GcpOptions; +import org.apache.beam.sdk.extensions.protobuf.ProtoByteUtils; import org.apache.beam.sdk.io.gcp.pubsublite.internal.Uuid; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; @@ -77,7 +78,7 @@ public class PubsubLiteReadSchemaTransformProvider private static final Logger LOG = LoggerFactory.getLogger(PubsubLiteReadSchemaTransformProvider.class); - public static final String VALID_FORMATS_STR = "RAW,AVRO,JSON"; + public static final String VALID_FORMATS_STR = "RAW,AVRO,JSON,PROTO"; public static final Set VALID_DATA_FORMATS = Sets.newHashSet(VALID_FORMATS_STR.split(",")); @@ -207,26 +208,39 @@ public void finish(FinishBundleContext c) { Schema beamSchema; if (format != null && format.equals("RAW")) { - if (inputSchema != null) { - throw new IllegalArgumentException( - "To read from PubSubLite in RAW format, you can't provide a schema."); - } + beamSchema = Schema.builder().addField("payload", Schema.FieldType.BYTES).build(); valueMapper = getRawBytesToRowFunction(beamSchema); + } else if (format != null && format.equals("PROTO")) { + String fileDescriptorPath = configuration.getFileDescriptorPath(); + String messageName = configuration.getMessageName(); + + if (fileDescriptorPath != null && messageName != null) { + beamSchema = ProtoByteUtils.getBeamSchemaFromProto(fileDescriptorPath, messageName); + valueMapper = ProtoByteUtils.getProtoBytesToRowFunction(fileDescriptorPath, messageName); + } else if (inputSchema != null && messageName != null) { + beamSchema = ProtoByteUtils.getBeamSchemaFromProtoSchema(inputSchema, messageName); + valueMapper = ProtoByteUtils.getProtoBytesToRowFromSchemaFunction(inputSchema, messageName); + } else { + throw new IllegalArgumentException( + "To read from PubSubLite in PROTO format, either descriptorPath or schema must be provided."); + } + } else { - if (inputSchema == null) { + if (inputSchema != null) { + beamSchema = + Objects.equals(configuration.getFormat(), "JSON") + ? JsonUtils.beamSchemaFromJsonSchema(inputSchema) + : AvroUtils.toBeamSchema(new org.apache.avro.Schema.Parser().parse(inputSchema)); + valueMapper = + Objects.equals(configuration.getFormat(), "JSON") + ? JsonUtils.getJsonBytesToRowFunction(beamSchema) + : AvroUtils.getAvroBytesToRowFunction(beamSchema); + } else { throw new IllegalArgumentException( - "To read from PubSubLite in JSON or AVRO format, you must provide a schema."); + "To read from Pubsub Lite in JSON or AVRO format, you must provide a schema."); } - beamSchema = - Objects.equals(configuration.getFormat(), "JSON") - ? JsonUtils.beamSchemaFromJsonSchema(inputSchema) - : AvroUtils.toBeamSchema(new org.apache.avro.Schema.Parser().parse(inputSchema)); - valueMapper = - Objects.equals(configuration.getFormat(), "JSON") - ? JsonUtils.getJsonBytesToRowFunction(beamSchema) - : AvroUtils.getAvroBytesToRowFunction(beamSchema); } return new SchemaTransform() { @Override @@ -404,13 +418,33 @@ public Uuid apply(SequencedMessage input) { @AutoValue @DefaultSchema(AutoValueSchema.class) public abstract static class PubsubLiteReadSchemaTransformConfiguration { + + public void validate() { + final String dataFormat = this.getFormat(); + assert dataFormat == null || VALID_DATA_FORMATS.contains(dataFormat) + : "Valid data formats are " + VALID_DATA_FORMATS; + + final String inputSchema = this.getSchema(); + final String messageName = this.getMessageName(); + + if (dataFormat != null && dataFormat.equals("RAW")) { + assert inputSchema == null + : "To read from Pubsub Lite in RAW format, you can't provide a schema."; + } + + if (dataFormat != null && dataFormat.equals("PROTO")) { + assert messageName != null + : "To read from Pubsub Lite in PROTO format, messageName must be provided."; + } + } + @SchemaFieldDescription( "The encoding format for the data stored in Pubsub Lite. Valid options are: " + VALID_FORMATS_STR) public abstract String getFormat(); @SchemaFieldDescription( - "The schema in which the data is encoded in the Kafka topic. " + "The schema in which the data is encoded in the Pubsub Lite topic. " + "For AVRO data, this is a schema defined with AVRO schema syntax " + "(https://avro.apache.org/docs/1.10.2/spec.html#schemas). " + "For JSON data, this is a schema defined with JSON-schema syntax (https://json-schema.org/).") @@ -459,6 +493,18 @@ public abstract static class PubsubLiteReadSchemaTransformConfiguration { + "case, deduplication of the stream will be strictly best effort.") public abstract @Nullable String getAttributeId(); + @SchemaFieldDescription( + "The path to the Protocol Buffer File Descriptor Set file. This file is used for schema" + + " definition and message serialization.") + @Nullable + public abstract String getFileDescriptorPath(); + + @SchemaFieldDescription( + "The name of the Protocol Buffer message to be used for schema" + + " extraction and data conversion.") + @Nullable + public abstract String getMessageName(); + public static Builder builder() { return new AutoValue_PubsubLiteReadSchemaTransformProvider_PubsubLiteReadSchemaTransformConfiguration .Builder(); @@ -486,6 +532,12 @@ public abstract static class Builder { @SuppressWarnings("unused") public abstract Builder setAttributeId(String attributeId); + @SuppressWarnings("unused") + public abstract Builder setFileDescriptorPath(String fileDescriptorPath); + + @SuppressWarnings("unused") + public abstract Builder setMessageName(String messageName); + public abstract PubsubLiteReadSchemaTransformConfiguration build(); } } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteWriteSchemaTransformProvider.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteWriteSchemaTransformProvider.java index c669f84f330f..8ba8176035da 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteWriteSchemaTransformProvider.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/pubsublite/PubsubLiteWriteSchemaTransformProvider.java @@ -34,6 +34,7 @@ import java.util.Objects; import java.util.Set; import org.apache.beam.sdk.extensions.avro.schemas.utils.AvroUtils; +import org.apache.beam.sdk.extensions.protobuf.ProtoByteUtils; import org.apache.beam.sdk.io.gcp.pubsublite.internal.Uuid; import org.apache.beam.sdk.metrics.Counter; import org.apache.beam.sdk.metrics.Metrics; @@ -71,7 +72,7 @@ public class PubsubLiteWriteSchemaTransformProvider extends TypedSchemaTransformProvider< PubsubLiteWriteSchemaTransformProvider.PubsubLiteWriteSchemaTransformConfiguration> { - public static final String SUPPORTED_FORMATS_STR = "RAW,JSON,AVRO"; + public static final String SUPPORTED_FORMATS_STR = "RAW,JSON,AVRO,PROTO"; public static final Set SUPPORTED_FORMATS = Sets.newHashSet(SUPPORTED_FORMATS_STR.split(",")); public static final TupleTag OUTPUT_TAG = new TupleTag() {}; @@ -211,6 +212,19 @@ public PCollectionRowTuple expand(PCollectionRowTuple input) { "The input schema must have exactly one field of type byte."); } toBytesFn = getRowToRawBytesFunction(inputSchema.getField(0).getName()); + } else if (configuration.getFormat().equals("PROTO")) { + String descriptorPath = configuration.getFileDescriptorPath(); + String schema = configuration.getSchema(); + String messageName = configuration.getMessageName(); + + if (descriptorPath != null && messageName != null) { + toBytesFn = ProtoByteUtils.getRowToProtoBytes(descriptorPath, messageName); + } else if (schema != null && messageName != null) { + toBytesFn = ProtoByteUtils.getRowToProtoBytesFromSchema(schema, messageName); + } else { + throw new IllegalArgumentException( + "At least a descriptorPath or a PROTO schema is required."); + } } else if (configuration.getFormat().equals("JSON")) { toBytesFn = JsonUtils.getRowToJsonBytesFunction(inputSchema); } else { @@ -322,6 +336,20 @@ public byte[] apply(Row input) { @AutoValue @DefaultSchema(AutoValueSchema.class) public abstract static class PubsubLiteWriteSchemaTransformConfiguration { + + public void validate() { + final String dataFormat = this.getFormat(); + final String inputSchema = this.getSchema(); + final String messageName = this.getMessageName(); + final String descriptorPath = this.getFileDescriptorPath(); + + if (dataFormat != null && dataFormat.equals("PROTO")) { + assert messageName != null : "Expecting messageName to be non-null."; + assert descriptorPath != null && inputSchema != null + : "You must include a descriptorPath or a PROTO schema but not both."; + } + } + @SchemaFieldDescription( "The GCP project where the Pubsub Lite reservation resides. This can be a " + "project number of a project ID.") @@ -358,6 +386,18 @@ public abstract static class PubsubLiteWriteSchemaTransformConfiguration { + "in a ReadFromPubSubLite PTransform to deduplicate messages.") public abstract @Nullable String getAttributeId(); + @SchemaFieldDescription( + "The path to the Protocol Buffer File Descriptor Set file. This file is used for schema" + + " definition and message serialization.") + public abstract @Nullable String getFileDescriptorPath(); + + @SchemaFieldDescription( + "The name of the Protocol Buffer message to be used for schema" + + " extraction and data conversion.") + public abstract @Nullable String getMessageName(); + + public abstract @Nullable String getSchema(); + public static Builder builder() { return new AutoValue_PubsubLiteWriteSchemaTransformProvider_PubsubLiteWriteSchemaTransformConfiguration .Builder(); @@ -380,6 +420,15 @@ public abstract static class Builder { @SuppressWarnings("unused") public abstract Builder setAttributeId(String attributeId); + @SuppressWarnings("unused") + public abstract Builder setFileDescriptorPath(String fileDescriptorPath); + + @SuppressWarnings("unused") + public abstract Builder setMessageName(String messageName); + + @SuppressWarnings("unused") + public abstract Builder setSchema(String schema); + public abstract PubsubLiteWriteSchemaTransformConfiguration build(); } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteDlqTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteDlqTest.java index e23536d800e5..4acf0a1149e1 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteDlqTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteDlqTest.java @@ -31,6 +31,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.extensions.protobuf.ProtoByteUtils; import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteIO; import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteReadSchemaTransformProvider; import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteReadSchemaTransformProvider.ErrorFn; @@ -252,6 +253,67 @@ public class PubsubLiteDlqTest { .build()) .build()); + private static final String PROTO_STRING_SCHEMA = + "syntax = \"proto3\";\n" + + "package com.test.proto;" + + "\n" + + "message MyMessage {\n" + + " int32 id = 1;\n" + + " string name = 2;\n" + + " bool active = 3;\n" + + "\n" + + " // Nested field\n" + + " message Address {\n" + + " string street = 1;\n" + + " string city = 2;\n" + + " string state = 3;\n" + + " string zip_code = 4;\n" + + " }\n" + + "\n" + + " Address address = 4;\n" + + "}"; + + private static final Schema BEAM_PROTO_SCHEMA = + Schema.builder() + .addField("id", Schema.FieldType.INT32) + .addField("name", Schema.FieldType.STRING) + .addField("active", Schema.FieldType.BOOLEAN) + .addField( + "address", + Schema.FieldType.row( + Schema.builder() + .addField("city", Schema.FieldType.STRING) + .addField("street", Schema.FieldType.STRING) + .addField("state", Schema.FieldType.STRING) + .addField("zip_code", Schema.FieldType.STRING) + .build())) + .build(); + + private static final Row INPUT_ROW = + Row.withSchema(BEAM_PROTO_SCHEMA) + .withFieldValue("id", 1234) + .withFieldValue("name", "Doe") + .withFieldValue("active", false) + .withFieldValue("address.city", "seattle") + .withFieldValue("address.street", "fake street") + .withFieldValue("address.zip_code", "TO-1234") + .withFieldValue("address.state", "wa") + .build(); + private static final SerializableFunction INPUT_MAPPER = + ProtoByteUtils.getRowToProtoBytesFromSchema(PROTO_STRING_SCHEMA, "com.test.proto.MyMessage"); + + private static final byte[] INPUT_SOURCE = INPUT_MAPPER.apply(INPUT_ROW); + + private static final List INPUT_MESSAGES = + Collections.singletonList( + SequencedMessage.newBuilder() + .setMessage( + PubSubMessage.newBuilder() + .setData(ByteString.copyFrom(INPUT_SOURCE)) + .putAllAttributes(ATTRIBUTE_VALUES_MAP) + .build()) + .build()); + final SerializableFunction valueMapper = JsonUtils.getJsonBytesToRowFunction(BEAM_SCHEMA); @@ -471,4 +533,31 @@ public void testPubsubLiteErrorFnWithDedupingSuccess() { p.run().waitUntilFinish(); } + + @Test + public void testPubSubLiteErrorFnReadProto() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); + + List attributes = new ArrayList<>(); + String attributesMap = ""; + Schema beamAttributeSchema = + PubsubLiteReadSchemaTransformProvider.buildSchemaWithAttributes( + BEAM_PROTO_SCHEMA, attributes, attributesMap); + + SerializableFunction protoValueMapper = + ProtoByteUtils.getProtoBytesToRowFromSchemaFunction( + PROTO_STRING_SCHEMA, "com.test.proto.MyMessage"); + + PCollection input = p.apply(Create.of(INPUT_MESSAGES)); + PCollectionTuple output = + input.apply( + ParDo.of(new ErrorFn("Read-Error-Counter", protoValueMapper, errorSchema, Boolean.TRUE)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(OUTPUT_TAG).setRowSchema(beamAttributeSchema); + output.get(ERROR_TAG).setRowSchema(errorSchema); + + PAssert.that(output.get(OUTPUT_TAG)).containsInAnyOrder(INPUT_ROW); + p.run().waitUntilFinish(); + } } diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteWriteDlqTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteWriteDlqTest.java index eac0e4264c1a..5afa4b7e5162 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteWriteDlqTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/pubsublite/internal/PubsubLiteWriteDlqTest.java @@ -27,6 +27,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.sdk.extensions.protobuf.ProtoByteUtils; import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteWriteSchemaTransformProvider; import org.apache.beam.sdk.io.gcp.pubsublite.PubsubLiteWriteSchemaTransformProvider.ErrorCounterFn; import org.apache.beam.sdk.schemas.Schema; @@ -111,6 +112,14 @@ public class PubsubLiteWriteDlqTest { .withFieldValue("key2", "second_key") .build()); + private static final String PROTO_STRING_SCHEMA = + "syntax = \"proto3\";\n" + + "package com.test.proto;" + + "\n" + + "message MyMessage {\n" + + " string name = 1;\n" + + "}"; + private static final Map ATTRIBUTE_VALUES_MAP = new HashMap<>(); static { @@ -239,4 +248,26 @@ public void testPubsubLiteErrorFnSuccessWithAttributesAndDedupingSuccess() { PAssert.that(count).containsInAnyOrder(Collections.singletonList(3L)); p.run().waitUntilFinish(); } + + @Test + public void testPubsubLiteErrorFnSuccessProto() { + Schema errorSchema = ErrorHandling.errorSchemaBytes(); + + SerializableFunction valueMapperProto = + ProtoByteUtils.getRowToProtoBytesFromSchema( + PROTO_STRING_SCHEMA, "com.test.proto.MyMessage"); + + PCollection input = p.apply(Create.of(ROWS)); + PCollectionTuple output = + input.apply( + ParDo.of( + new ErrorCounterFn("ErrorCounter", valueMapperProto, errorSchema, Boolean.TRUE)) + .withOutputTags(OUTPUT_TAG, TupleTagList.of(ERROR_TAG))); + + output.get(ERROR_TAG).setRowSchema(errorSchema); + + PAssert.that(output.get(OUTPUT_TAG).apply(Count.globally())) + .containsInAnyOrder(Collections.singletonList(3L)); + p.run().waitUntilFinish(); + } } diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 4d26ce96b677..d63729f1676c 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -71,6 +71,7 @@ 'topic': 'topic' 'bootstrap_servers': 'bootstrapServers' 'producer_config_updates': 'producerConfigUpdates' + 'error_handling': 'errorHandling' 'file_descriptor_path': 'fileDescriptorPath' 'message_name': 'messageName' 'schema': 'schema' @@ -97,6 +98,9 @@ 'attributes': 'attributes' 'attribute_map': 'attributeMap' 'attribute_id': 'attributeId' + 'error_handling': 'errorHandling' + 'file_descriptor_path': 'fileDescriptorPath' + 'message_name': 'messageName' 'WriteToPubSubLite': 'project': 'project' 'format': 'format' @@ -104,6 +108,10 @@ 'location': 'location' 'attributes': 'attributes' 'attribute_id': 'attributeId' + 'error_handling': 'errorHandling' + 'file_descriptor_path': 'fileDescriptorPath' + 'message_name': 'messageName' + 'schema': 'schema' underlying_provider: type: beamJar transforms: From 1a5466b7df7849c2d4305563e2deace5a6c9482c Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Thu, 1 Feb 2024 14:02:29 -0500 Subject: [PATCH 19/50] Support dict as the input of delete_batch (#30188) This smooths the code path of feeding output of list_prefix() to delete_batch(). --- sdks/python/apache_beam/io/gcp/gcsio.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/gcp/gcsio.py b/sdks/python/apache_beam/io/gcp/gcsio.py index 087d32a9e054..a6ba82a6e07c 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio.py +++ b/sdks/python/apache_beam/io/gcp/gcsio.py @@ -208,7 +208,8 @@ def delete_batch(self, paths): """Deletes the objects at the given GCS paths. Args: - paths: List of GCS file path patterns in the form gs:///, + paths: List of GCS file path patterns or Dict with GCS file path patterns + as keys. The patterns are in the form gs:///, but not to exceed MAX_BATCH_OPERATION_SIZE in length. Returns: List of tuples of (path, exception) in the same order as the @@ -217,6 +218,7 @@ def delete_batch(self, paths): """ final_results = [] s = 0 + if not isinstance(paths, list): paths = list(iter(paths)) while s < len(paths): if (s + MAX_BATCH_OPERATION_SIZE) < len(paths): current_paths = paths[s:s + MAX_BATCH_OPERATION_SIZE] From 574e7bfbc1d30b86f0ff33a01653ebe525b0c964 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Thu, 1 Feb 2024 11:16:24 -0800 Subject: [PATCH 20/50] Fix script typos for validating docker licenses. (#30176) --- contributor-docs/release-guide.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index b3b3213d9c97..559d3c0f9f0f 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -564,11 +564,11 @@ The following should be confirmed: At [https://hub.docker.com/u/apache](https://hub.docker.com/search?q=apache%2Fbeam&type=image), visit each repository and navigate to "tags" tab. Verify images are pushed -with tags: `${RELEASE_VERSION}rc{RC_NUM}` +with tags: `${RELEASE_VERSION}rc${RC_NUM}` Verify that third party licenses are included in Docker. You can do this with a simple script: - RC_TAG=${RELEASE_VERSION}rc{RC_NUM} + RC_TAG=${RELEASE_VERSION}rc${RC_NUM} for pyver in 3.8 3.9 3.10 3.11; do docker run --rm --entrypoint sh \ apache/beam_python${pyver}_sdk:${RC_TAG} \ @@ -577,7 +577,7 @@ Verify that third party licenses are included in Docker. You can do this with a for javaver in 8 11 17; do docker run --rm --entrypoint sh \ - apache/beam_java${pyver}_sdk:${RC_TAG} \ + apache/beam_java${javaver}_sdk:${RC_TAG} \ -c 'ls -al /opt/apache/beam/third_party_licenses/ | wc -l' done From 124880bc8ce22a2ccbbc0efab435f64099ce65df Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Fri, 2 Feb 2024 00:52:55 -0800 Subject: [PATCH 21/50] Fix typo affecting DataflowPipelineDebugOptions.setStreamingSideInputCacheExpirationMillis (#30195) Co-authored-by: Arun Pandian --- .../runners/dataflow/options/DataflowPipelineDebugOptions.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java index 290418bd1cb7..794dd76d2431 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java @@ -258,7 +258,7 @@ public Dataflow create(PipelineOptions options) { @Default.Integer(60 * 1000) // 1 minute Integer getStreamingSideInputCacheExpirationMillis(); - void setstreamingSideInputCacheExpirationMillis(Integer value); + void setStreamingSideInputCacheExpirationMillis(Integer value); /** * The amount of time before UnboundedReaders are considered idle and closed during streaming From 1f69271044dd89844eb5a9158ced5f0208028e42 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Fri, 2 Feb 2024 02:38:21 -0800 Subject: [PATCH 22/50] [Dataflow Streaming] Add a option to use multiple commit threads (#30194) Co-authored-by: Arun Pandian --- .../options/DataflowPipelineDebugOptions.java | 6 +++ .../worker/StreamingDataflowWorker.java | 49 ++++++++++--------- .../worker/StreamingDataflowWorkerTest.java | 46 ++++++++++++++++- 3 files changed, 77 insertions(+), 24 deletions(-) diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java index 794dd76d2431..e50a630c8a4a 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/options/DataflowPipelineDebugOptions.java @@ -260,6 +260,12 @@ public Dataflow create(PipelineOptions options) { void setStreamingSideInputCacheExpirationMillis(Integer value); + @Description("Number of commit threads used to commit items to streaming engine.") + @Default.Integer(1) + Integer getWindmillServiceCommitThreads(); + + void setWindmillServiceCommitThreads(Integer value); + /** * The amount of time before UnboundedReaders are considered idle and closed during streaming * execution. diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index f37504692d3e..3ba27bd852fc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -135,11 +135,7 @@ import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.EvictingQueue; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ListMultimap; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.MultimapBuilder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.*; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.graph.MutableNetwork; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.net.HostAndPort; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -231,7 +227,7 @@ public class StreamingDataflowWorker { private final BoundedQueueExecutor workUnitExecutor; private final WindmillServerStub windmillServer; private final Thread dispatchThread; - private final Thread commitThread; + @VisibleForTesting final ImmutableList commitThreads; private final AtomicLong activeCommitBytes = new AtomicLong(); private final AtomicLong previousTimeAtMaxThreads = new AtomicLong(); private final AtomicBoolean running = new AtomicBoolean(); @@ -409,21 +405,28 @@ public void run() { dispatchThread.setPriority(Thread.MIN_PRIORITY); dispatchThread.setName("DispatchThread"); - commitThread = - new Thread( - new Runnable() { - @Override - public void run() { + int numCommitThreads = 1; + if (windmillServiceEnabled && options.getWindmillServiceCommitThreads() > 0) { + numCommitThreads = options.getWindmillServiceCommitThreads(); + } + + ImmutableList.Builder commitThreadsBuilder = ImmutableList.builder(); + for (int i = 0; i < numCommitThreads; ++i) { + Thread commitThread = + new Thread( + () -> { if (windmillServiceEnabled) { streamingCommitLoop(); } else { commitLoop(); } - } - }); - commitThread.setDaemon(true); - commitThread.setPriority(Thread.MAX_PRIORITY); - commitThread.setName("CommitThread"); + }); + commitThread.setDaemon(true); + commitThread.setPriority(Thread.MAX_PRIORITY); + commitThread.setName("CommitThread " + i); + commitThreadsBuilder.add(commitThread); + } + commitThreads = commitThreadsBuilder.build(); this.publishCounters = publishCounters; this.windmillServer = options.getWindmillServerStub(); @@ -585,7 +588,7 @@ public void start() { memoryMonitorThread.start(); dispatchThread.start(); - commitThread.start(); + commitThreads.forEach(Thread::start); sampler.start(); // Periodically report workers counters and other updates. @@ -717,10 +720,12 @@ public void stop() { running.set(false); dispatchThread.interrupt(); dispatchThread.join(); - // We need to interrupt the commitThread in case it is blocking on pulling + // We need to interrupt the commitThreads in case they are blocking on pulling // from the commitQueue. - commitThread.interrupt(); - commitThread.join(); + commitThreads.forEach(Thread::interrupt); + for (Thread commitThread : commitThreads) { + commitThread.join(); + } memoryMonitor.stop(); memoryMonitorThread.join(); workUnitExecutor.shutdown(); @@ -1374,9 +1379,9 @@ private void commitLoop() { } Windmill.CommitWorkRequest commitRequest = commitRequestBuilder.build(); LOG.trace("Commit: {}", commitRequest); - activeCommitBytes.set(commitBytes); + activeCommitBytes.addAndGet(commitBytes); windmillServer.commitWork(commitRequest); - activeCommitBytes.set(0); + activeCommitBytes.addAndGet(-commitBytes); for (Map.Entry entry : computationRequestMap.entrySet()) { ComputationState computationState = entry.getKey(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index 5de67a371a65..e7eedcf3780a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -798,8 +798,7 @@ public void testBasicHarness() throws Exception { verify(hotKeyLogger, atLeastOnce()).logHotKeyDetection(nullable(String.class), any()); } - @Test - public void testBasic() throws Exception { + private void runTestBasic(int numCommitThreads) throws Exception { List instructions = Arrays.asList( makeSourceInstruction(StringUtf8Coder.of()), @@ -817,6 +816,7 @@ public void testBasic() throws Exception { when(mockWorkUnitClient.getGlobalStreamingConfigWorkItem()).thenReturn(Optional.of(workItem)); StreamingDataflowWorkerOptions options = createTestingPipelineOptions(server); + options.setWindmillServiceCommitThreads(numCommitThreads); StreamingDataflowWorker worker = makeWorker(instructions, options, true /* publishCounters */); worker.start(); @@ -838,6 +838,16 @@ public void testBasic() throws Exception { verify(hotKeyLogger, atLeastOnce()).logHotKeyDetection(nullable(String.class), any()); } + @Test + public void testBasic() throws Exception { + runTestBasic(1); + } + + @Test + public void testBasicWithMultipleCommitThreads() throws Exception { + runTestBasic(2); + } + @Test public void testHotKeyLogging() throws Exception { // This is to test that the worker can correctly log the key from a hot key. @@ -3832,6 +3842,38 @@ public void testStuckCommit() throws Exception { removeDynamicFields(result.get(1L))); } + private void runNumCommitThreadsTest(int configNumCommitThreads, int expectedNumCommitThreads) + throws Exception { + List instructions = + Arrays.asList( + makeSourceInstruction(StringUtf8Coder.of()), + makeSinkInstruction(StringUtf8Coder.of(), 0)); + FakeWindmillServer server = new FakeWindmillServer(errorCollector); + StreamingDataflowWorkerOptions options = createTestingPipelineOptions(server); + options.setWindmillServiceCommitThreads(configNumCommitThreads); + StreamingDataflowWorker worker = makeWorker(instructions, options, true /* publishCounters */); + worker.start(); + assertEquals(expectedNumCommitThreads, worker.commitThreads.size()); + worker.stop(); + } + + @Test + public void testDefaultNumCommitThreads() throws Exception { + if (streamingEngine) { + runNumCommitThreadsTest(1, 1); + runNumCommitThreadsTest(2, 2); + runNumCommitThreadsTest(3, 3); + runNumCommitThreadsTest(0, 1); + runNumCommitThreadsTest(-1, 1); + } else { + runNumCommitThreadsTest(1, 1); + runNumCommitThreadsTest(2, 1); + runNumCommitThreadsTest(3, 1); + runNumCommitThreadsTest(0, 1); + runNumCommitThreadsTest(-1, 1); + } + } + static class BlockingFn extends DoFn implements TestRule { public static CountDownLatch blocker = new CountDownLatch(1); From 1183b5d52745ab28552558385b6b28ce5218b4c1 Mon Sep 17 00:00:00 2001 From: Oleh Borysevych Date: Fri, 2 Feb 2024 18:10:48 +0100 Subject: [PATCH 23/50] Knowledge lookup prompts: (#29747) * added new io files * bigtable wip * bigtable wip * spanner wip * io prompts wip * remove whitespace * add gcs * add tfrecords * add tfrecords generate * code review comments * README added * README fixed * Apply suggestions from code review Co-authored-by: Sereana Seim * folder name fixed * Apply suggestions from code review Co-authored-by: Sereana Seim * code review suggestion applied * whitespace fixed --------- Co-authored-by: Sereana Seim --- learning/prompts/README.md | 76 +++++++++++++ .../prompts/code-explanation/01_io_kafka.md | 65 +++++++++++ .../02_io_pubsub.md | 0 .../code-explanation/03_io_bigquery.md | 39 +++++++ .../code-explanation/04_io_bigtable.md | 44 ++++++++ .../prompts/code-explanation/05_io_spanner.md | 96 ++++++++++++++++ .../code-explanation/06_io_tfrecord.md | 63 +++++++++++ .../prompts/code-explanation/07_io_json.md | 50 +++++++++ .../prompts/code-explanation/08_io_csv.md | 44 ++++++++ .../prompts/code-explanation/09_io_avro.md | 41 +++++++ .../prompts/code-explanation/10_io_parquet.md | 39 +++++++ .../prompts/code-explanation/11_io_jdbc.md | 105 ++++++++++++++++++ .../prompts/code-generation/01_io_kafka.md | 46 ++++++++ .../prompts/code-generation/03_io_bigquery.md | 54 +++++++++ .../prompts/code-generation/04_io_bigtable.md | 66 +++++++++++ .../prompts/code-generation/05_io_spanner.md | 63 +++++++++++ .../prompts/code-generation/06_io_tfrecord.md | 60 ++++++++++ .../prompts/code-generation/07_io_json.md | 48 ++++++++ learning/prompts/code-generation/08_io_csv.md | 47 ++++++++ .../prompts/code-generation/09_io_avro.md | 42 +++++++ .../prompts/code-generation/10_io_parquet.md | 39 +++++++ .../prompts/code-generation/11_io_jdbc.md | 75 +++++++++++++ .../documentation-lookup/18_io_kafka.md | 27 +++++ .../documentation-lookup/21_io_bigtable.md | 28 +++++ .../documentation-lookup/22_io_spanner.md | 31 ++++++ .../prompts/documentation-lookup/23_io_gcs.md | 23 ++++ .../documentation-lookup/24_io_json.md | 32 ++++++ .../prompts/documentation-lookup/25_io_csv.md | 32 ++++++ .../documentation-lookup/26_io_avro.md | 31 ++++++ .../documentation-lookup/27_io_parquet.md | 31 ++++++ .../documentation-lookup/28_io_jdbc.md | 52 +++++++++ 31 files changed, 1489 insertions(+) create mode 100644 learning/prompts/README.md create mode 100644 learning/prompts/code-explanation/01_io_kafka.md rename learning/prompts/{code-explaination => code-explanation}/02_io_pubsub.md (100%) create mode 100644 learning/prompts/code-explanation/03_io_bigquery.md create mode 100644 learning/prompts/code-explanation/04_io_bigtable.md create mode 100644 learning/prompts/code-explanation/05_io_spanner.md create mode 100644 learning/prompts/code-explanation/06_io_tfrecord.md create mode 100644 learning/prompts/code-explanation/07_io_json.md create mode 100644 learning/prompts/code-explanation/08_io_csv.md create mode 100644 learning/prompts/code-explanation/09_io_avro.md create mode 100644 learning/prompts/code-explanation/10_io_parquet.md create mode 100644 learning/prompts/code-explanation/11_io_jdbc.md create mode 100644 learning/prompts/code-generation/01_io_kafka.md create mode 100644 learning/prompts/code-generation/03_io_bigquery.md create mode 100644 learning/prompts/code-generation/04_io_bigtable.md create mode 100644 learning/prompts/code-generation/05_io_spanner.md create mode 100644 learning/prompts/code-generation/06_io_tfrecord.md create mode 100644 learning/prompts/code-generation/07_io_json.md create mode 100644 learning/prompts/code-generation/08_io_csv.md create mode 100644 learning/prompts/code-generation/09_io_avro.md create mode 100644 learning/prompts/code-generation/10_io_parquet.md create mode 100644 learning/prompts/code-generation/11_io_jdbc.md create mode 100644 learning/prompts/documentation-lookup/18_io_kafka.md create mode 100644 learning/prompts/documentation-lookup/21_io_bigtable.md create mode 100644 learning/prompts/documentation-lookup/22_io_spanner.md create mode 100644 learning/prompts/documentation-lookup/23_io_gcs.md create mode 100644 learning/prompts/documentation-lookup/24_io_json.md create mode 100644 learning/prompts/documentation-lookup/25_io_csv.md create mode 100644 learning/prompts/documentation-lookup/26_io_avro.md create mode 100644 learning/prompts/documentation-lookup/27_io_parquet.md create mode 100644 learning/prompts/documentation-lookup/28_io_jdbc.md diff --git a/learning/prompts/README.md b/learning/prompts/README.md new file mode 100644 index 000000000000..ea5d7d8bd79d --- /dev/null +++ b/learning/prompts/README.md @@ -0,0 +1,76 @@ + +This folder holds golden prompt/response pairs for Google Duet AI training. + +A golden prompt/response pair contains two parts: +1. An example prompt/question to ask an LLM +2. An ideal answer we would expect the LLM to generate + +Each prompt/response pair is a markdown file with the following structure: +``` +Prompt: + + +Response: + +``` +This folder includes the following types of golden prompts: +- Documentation lookup prompts +- Code generation prompts +- Code explanation prompts + +## Documentation lookup prompts +Features of a good response: +- Uses official product names in the response (“Speech to text” → “Speech-to-Text”). +- Answers the question (correctly) with hyperlinks to the documentation. +- Includes a link to the corresponding source code. +- Includes a link to samples, if available. + + +## Code generation prompts +Features of a good response: +- Starts with a brief introduction that explains the code sample. +- Includes information about how to find the reference documentation. +- Includes a link to the list of code samples. +- Provides well documented code. Consider including an example of what the return result looks like. +- Follows up with the user to ensure they don’t continue needlessly with false responses. + + +## Code explanation prompts +Features of a good response: +- Starts with a short overall description that tries to answer the question in the prompt. +- Grounds the algorithm in any well-known context, if appropriate. For example, this is an implementation of X, a well-known algorithm to do Y. +- Discusses the variables in the snippet, and what their purpose is relative to the runtime. +- Discusses runtime and memory storage complexity. +- Notes any interesting features of the code, or opportunities for improvement (optimizations, refactoring, syntax best practices, etc.) + +Folder structure: +``` +learning/prompts/ +├── code-explanation +│   ├── 01_io_kafka.md +│   └── ... +├── code-generation +│   ├── 01_io_kafka.md +│   └── ... +├── documentation-lookup +│   ├── 01_basic_learning_apache_beam.md +│   └── ... +└── README.md +``` \ No newline at end of file diff --git a/learning/prompts/code-explanation/01_io_kafka.md b/learning/prompts/code-explanation/01_io_kafka.md new file mode 100644 index 000000000000..c6b36b400b41 --- /dev/null +++ b/learning/prompts/code-explanation/01_io_kafka.md @@ -0,0 +1,65 @@ +Prompt: +What does this code do? +```python +class KafkaOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--bootstrap_servers', + default="localhost:9092", + help='Apache Kafka bootstrap servers' + ) + parser.add_argument( + '--topic', + default="your-topic", + help='Apache Kafka topic' + ) + +options = KafkaOptions() + +with beam.Pipeline(options=options) as p: + output = (p | "Read from Kafka" >> ReadFromKafka( + consumer_config={'bootstrap.servers': options.bootstrap_servers}, + topics=[options.topic], + with_metadata=False + ) + | "Log Data" >> Map(logging.info)) + +``` +Response: +This code uses the Apache Beam [KafkaIO Pythn connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.kafka.html) Python connector to read data from an [Apache Kafka](https://kafka.apache.org/) topic `topic` on Kafka bootstrap servers `bootstrap_servers`. + +The `ReadFromKafka` transform reads from a Kafka topic `topic` on `bootstrap_servers`. This is a cross-language transform that is implemented in Java and is available in Python, Go, and Typescript via X-language.The Python SDK connects to a Java expansion service to expand these transforms during pipeline execution. For more information about cross-language transforms, see Apache Beam [Multi-language pipelines](https://beam.apache.org/documentation/programming-guide/#multi-language-pipelines). + +```python +with beam.Pipeline(options=options) as p: + output = (p | "Read from Kafka" >> ReadFromKafka( + consumer_config={'bootstrap.servers': options.bootstrap_servers}, + topics=[options.topic], + with_metadata=False + ) + | "Log Data" >> Map(logging.info)) +``` +The required arguments for the `ReadFromKafka` transform are `consumer_config` and `topics`. You can use the `consumer_config` argument to specify the Kafka bootstrap servers, the `topics` argument to specify the Kafka topic to read data from, and the `with_metadata` argument to specify whether to include Kafka metadata in the output `PCollection`. The default value for the `with_metadata` argument is `False`. + +The `KafkaReadOptions` class is used to parse pipeline options for the Kafka read transform. The `bootstrap_servers` and `topic` arguments are provided as command-line arguments. This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) to parse pipeline options. + +```python +class KafkaOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--bootstrap_servers', + default="localhost:9092", + help='Apache Kafka bootstrap servers' + ) + parser.add_argument( + '--topic', + default="your-topic", + help='Apache Kafka topic' + ) + +options = KafkaOptions() +``` + +For more information about how to use the KafkaIO connector with the Python SDK for Apache Beam, see [KafkaIO module](https://beam.apache.org/releases/pydoc/current/apache_beam.io.kafka.html). \ No newline at end of file diff --git a/learning/prompts/code-explaination/02_io_pubsub.md b/learning/prompts/code-explanation/02_io_pubsub.md similarity index 100% rename from learning/prompts/code-explaination/02_io_pubsub.md rename to learning/prompts/code-explanation/02_io_pubsub.md diff --git a/learning/prompts/code-explanation/03_io_bigquery.md b/learning/prompts/code-explanation/03_io_bigquery.md new file mode 100644 index 000000000000..2520bcc7b0d0 --- /dev/null +++ b/learning/prompts/code-explanation/03_io_bigquery.md @@ -0,0 +1,39 @@ +Prompt: +What does this code do? +```python +class BigQueryReadTableOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + "--table", + default="bigquery-public-data:samples.github_timeline", + help="BigQuery table to read data from") + +options = BigQueryReadTableOptions() +with beam.Pipeline(options=options) as p: + output = (p | "Read table" >> ReadFromBigQuery(table=options.table) + | "Log data" >> Map(logging.info) + ) +``` +Response: +This code uses the [Google BigQuery I/O connector](https://beam.apache.org/documentation/io/built-in/google-bigquery/) to read data from a [BigQuery](https://cloud.google.com/bigquery/docs) table `table`. + +The BigQueryIO connector supports reading from BigQuery tables, executing queries, and reading from the BigQuery Storage API (beta). The BigQueryIO connector supports writing to BigQuery tables in batch and streaming modes. + +```python +output = (p | "Read table" >> ReadFromBigQuery(table=options.table)) +``` +The `ReadFromBigQuery` transform uses a BigQuery export job to save a table snapshot to Google Cloud Storage and then reads from each produced file. The default file format is Avro. + +Another option is to use a SQL query to read data from BigQuery. + +```python +output = (p | "Read table" >> ReadFromBigQuery(query="SELECT * FROM table")) +``` +For more information about how to use the BigQueryIO connector with the Python SDK for Apache Beam, see [BigQueryIO module](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.bigquery.html). + +This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) for the required `table` argument. The `table` argument is used to specify the BigQuery table to read data from. + +For performance metrics of the BigQueryIO connector, see [BigQueryIO Performance](https://beam.apache.org/performance/bigquery/). + + diff --git a/learning/prompts/code-explanation/04_io_bigtable.md b/learning/prompts/code-explanation/04_io_bigtable.md new file mode 100644 index 000000000000..44673dcbf932 --- /dev/null +++ b/learning/prompts/code-explanation/04_io_bigtable.md @@ -0,0 +1,44 @@ +Prompt: +What does this code do? +```python +class BigtableOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--project_id', + required=True, + help='Project ID' + ) + parser.add_argument( + '--instance_id', + default="beam-test", + help='Cloud Bigtable instance ID' + ) + parser.add_argument( + '--table_id', + default="your-test-table", + help='Cloud Bigtable table ID' + ) + + options = BigtableOptions() + +with beam.Pipeline(options=options) as p: + output = (p | "Read from Bigtable" >> ReadFromBigtable( + project_id=options.project_id, + instance_id=options.instance_id, + table_id=options.table_id + ) + | "Extract cells" >> beam.Map(lambda row: row._cells) + | "Log Data" >> Map(logging.info)) +``` + +Response: +This code uses the Apache Beam [BigtableIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.bigtableio.html) to read data from a [Bigtable](https://cloud.google.com/bigtable/docs) table `table_id` in a Bigtable instance `instance_id`. The `table_id` and `instance_id` are provided as command-line arguments. + +This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) to parse command-line arguments. + +The `ReadFromBigtable` transform returns a `PCollection` of `PartialRowData` objects, each representing a Bigtable row. For more information about this row object, see [PartialRowData (row_key)](https://cloud.google.com/python/docs/reference/bigtable/latest/row#class-googlecloudbigtablerowpartialrowdatarowkey). + +The data from a Bigtable row is extracted and logged to the console. + +For more information about how to use the BigtableIO connector with the Python SDK for Apache Beam, see [BigtableIO module](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.bigtableio.html). \ No newline at end of file diff --git a/learning/prompts/code-explanation/05_io_spanner.md b/learning/prompts/code-explanation/05_io_spanner.md new file mode 100644 index 000000000000..6a17f6ce2c46 --- /dev/null +++ b/learning/prompts/code-explanation/05_io_spanner.md @@ -0,0 +1,96 @@ +Prompt: +What does this code do? +```python +class ExampleRow(NamedTuple): + id: int + name: str + +class SpannerOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--project_id', + default='your-project-id', + help='Google Cloud project ID' + ) + parser.add_argument( + '--instance_id', + default='your-instance-id', + help='Google Cloud Spanner instance ID' + ) + parser.add_argument( + '--database_id', + default='your-database-id', + help='Google Cloud Spanner database ID' + ) + +options = SpannerOptions() +coders.registry.register_coder(ExampleRow, coders.RowCoder) + +with beam.Pipeline(options=options) as p: + + output = (p | "Read from table" >> ReadFromSpanner( + project_id=options.project_id, + instance_id=options.instance_id, + database_id=options.database_id, + row_type=ExampleRow, + sql="SELECT * FROM example_row" + ) + | "Log Data" >> Map(logging.info)) + +``` +Response: +This code uses the Apache Beam [SpannerIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.spanner.html) to read data from a [Cloud Spanner](https://cloud.google.com/spanner) table `example_row` from database `your-database-id` in instance `your-instance-id`. The values `your-database-id`, `your-instance-id`, and `your-project-id` are provided as command-line arguments. + +```python +class SpannerOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--project_id', + default='your-project-id', + help='Google Cloud project ID' + ) + parser.add_argument( + '--instance_id', + default='your-instance-id', + help='Google Cloud Spanner instance ID' + ) + parser.add_argument( + '--database_id', + default='your-database-id', + help='Google Cloud Spanner database ID' + ) + +options = SpannerOptions() +``` +This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) to parse command-line arguments. The `SpannerOptions` class defines the command-line arguments `project_id`, `instance_id`, and `database_id`, which are used to configure the `ReadFromSpanner` transform. + +```python +class ExampleRow(NamedTuple): + id: int + name: str +``` +This code defines a `NamedTuple` object `ExampleRow` that represents a Cloud Spanner row. The `NamedTuple` object has the fields `id` and `name`, which represent a Cloud Spanner row. The `NamedTuple` object is used as a row type in the `ReadFromSpanner` transform. + +```python + coders.registry.register_coder(ExampleRow, coders.RowCoder) +``` +Registering a coder for `NamedTuple` is required to use `NamedTuple` as a row type. For more information about how to register a coder for a custom type, see [Data encoding and type safety](https://beam.apache.org/documentation/programming-guide/#data-encoding-and-type-safety). + +```python +output = (p | "Read from table" >> ReadFromSpanner( + project_id=options.project_id, + instance_id=options.instance_id, + database_id=options.database_id, + row_type=ExampleRow, + sql="SELECT * FROM example_row" + ) + | "Log Data" >> Map(logging.info)) +``` + +The `ReadFromSpanner` transform returns a `PCollection` of `NamedTuple` objects, each representing a Cloud Spanner row. For more information about this row object, see [ReadFromSpanner transform](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.spanner.html#apache_beam.io.gcp.spanner.ReadFromSpanner). + +The data from a Cloud Spanner row is logged to the console. + + diff --git a/learning/prompts/code-explanation/06_io_tfrecord.md b/learning/prompts/code-explanation/06_io_tfrecord.md new file mode 100644 index 000000000000..f622a51a44f0 --- /dev/null +++ b/learning/prompts/code-explanation/06_io_tfrecord.md @@ -0,0 +1,63 @@ +Prompt: +What does this code do? +```python + +def map_from_bytes(element): + return pickle.loads(element) + +class TFRecordOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + + parser.add_argument( + "--file_pattern", + help="A file glob pattern of TFRecord files" + ) + + +options = TFRecordOptions() +with beam.Pipeline(options=options) as p: + +output = ( + p + "Read from TFRecord" >> ReadFromTFRecord( + file_pattern=options.file_pattern + ) + | "Map from bytes" >> Map(map_from_bytes) + | "Log Data" >> Map(logging.info) +) + +``` +Response: +This code uses the Apache Beam [TFRecordIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.tfrecordio.html) to read data from a TensorFlow [TFRecord file](https://www.tensorflow.org/api_docs/python/tf/data/TFRecordDataset) matched with `file_pattern`. The `file_pattern` parameter is provided as a command-line argument. + +```python +class TFRecordOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + + parser.add_argument( + "--file_pattern", + help="A file glob pattern of TFRecord files" + ) + +options = TFRecordOptions() +``` +This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) to parse command-line arguments. + +```python +output = ( + p + "Read from TFRecord" >> ReadFromTFRecord( + file_pattern=options.file_pattern + ) + | "Map from bytes" >> Map(map_from_bytes) + | "Log Data" >> Map(logging.info) +) +``` + +The [ReadFromTFRecord transform](https://beam.apache.org/releases/pydoc/current/apache_beam.io.tfrecordio.html#apache_beam.io.tfrecordio.ReadFromTFRecord) reads from a `TFRecord` source. By default, TFRecordIO transforms use `coders.BytesCoder()`. The `Map` transform applies the `map_from_bytes` function to deserialize bytes to a Python object. The deserialized object is logged to the console. + + + + diff --git a/learning/prompts/code-explanation/07_io_json.md b/learning/prompts/code-explanation/07_io_json.md new file mode 100644 index 000000000000..46e3099d9f2b --- /dev/null +++ b/learning/prompts/code-explanation/07_io_json.md @@ -0,0 +1,50 @@ +Prompt: +What does this code do? +```python +class JsonOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--file_path', + default="gs://your-bucket/your-file.json", + help='Json file path' + ) + +options = JsonOptions() + +with beam.Pipeline(options=options) as p: + output = (p | "Read from Json file" >> ReadFromJson( + path=options.file_path, + lines=False + ) + | "Log Data" >> Map(logging.info)) +``` +Response: +This code reads data from a [JSON file](https://www.json.org/) using the `ReadFromJson` transform from the built-in [TextIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.textio.html). + +```python +class JsonOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--file_path', + default="gs://your-bucket/your-file.json", + help='Json file path' + ) + +options = JsonOptions() + +``` +The `JsonOptions` class is used to define the command-line argument `--file_path`, which specifies the path to the JSON file. This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) for the required `file_path` argument. + +```python +with beam.Pipeline(options=options) as p: + output = (p | "Read from Json file" >> ReadFromJson( + path=options.file_path, + lines=False + ) + | "Log Data" >> Map(logging.info)) +``` +A Beam pipeline is created using the `JsonOptions` class and the `ReadFromJson` transform is used to read data from the JSON file. The `lines` parameter is set to `False` to read the entire file as a single JSON object. + + diff --git a/learning/prompts/code-explanation/08_io_csv.md b/learning/prompts/code-explanation/08_io_csv.md new file mode 100644 index 000000000000..22b75310409c --- /dev/null +++ b/learning/prompts/code-explanation/08_io_csv.md @@ -0,0 +1,44 @@ +Prompt: +What does this code do? +```python +class CsvOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--file_path', + default="gs://your-bucket/your-file.csv", + help='Csv file path' + ) + + +options = CsvOptions() + +with beam.Pipeline(options=options) as p: + output = (p | "Read from Csv file" >> ReadFromCsv(path=options.file_path) + | "Log Data" >> Map(logging.info)) + +``` +Response: +This code reads data from a [CSV file](https://en.wikipedia.org/wiki/Comma-separated_values) using the `ReadFromCsv` transform from the built-in [TextIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.textio.html). + +```python +class CsvOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--file_path', + default="gs://your-bucket/your-file.csv", + help='Csv file path' + ) + +options = CsvOptions() +``` + +The `CsvOptions` class is used to define the command-line argument `--file_path`, which specifies the path to the CSV file. This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) for the required `file_path` argument. + +```python +with beam.Pipeline(options=options) as p: + output = (p | "Read from Csv file" >> ReadFromCsv(path=options.file_path) + | "Log Data" >> Map(logging.info)) +``` +A Beam pipeline is created using the `CsvOptions` class and the [ReadFromCsv transform](https://beam.apache.org/releases/pydoc/current/apache_beam.io.textio.html#apache_beam.io.textio.ReadFromCsv) is used to read data from the CSV file. diff --git a/learning/prompts/code-explanation/09_io_avro.md b/learning/prompts/code-explanation/09_io_avro.md new file mode 100644 index 000000000000..9dfe43e786b7 --- /dev/null +++ b/learning/prompts/code-explanation/09_io_avro.md @@ -0,0 +1,41 @@ +Prompt: +What does this code do? +```python +class ReadAvroOptions(PipelineOptions): + +@classmethod +def _add_argparse_args(cls, parser): + parser.add_argument( + "--path", + help="GCS path to Avro file") + + options = ReadAvroOptions() + + with beam.Pipeline(options=options) as p: + + (p | "Read Avro" >> ReadFromAvro(options.path) + | Map(logging.info)) +``` +Response: +This code reads data from [Apache Avro](https://avro.apache.org/) files using the `ReadFromAvro` transform from the built-in [AvroIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.avroio.html). + +```python +class ReadAvroOptions(PipelineOptions): + +@classmethod +def _add_argparse_args(cls, parser): + parser.add_argument( + "--path", + help="GCS path to Avro file") + + options = ReadAvroOptions() +``` +The `ReadAvroOptions` class is used to define the command-line argument `--path`, which specifies the path to the Avro file. This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) for the required `path` argument. + +```python + with beam.Pipeline(options=options) as p: + + (p | "Read Avro" >> ReadFromAvro(options.path) + | Map(logging.info)) +``` +A Beam pipeline is created using the `ReadAvroOptions` class and the [ReadFromAvro transform](https://beam.apache.org/releases/pydoc/current/apache_beam.io.avroio.html#apache_beam.io.avroio.ReadFromAvro) is used to read data from the Avro file. diff --git a/learning/prompts/code-explanation/10_io_parquet.md b/learning/prompts/code-explanation/10_io_parquet.md new file mode 100644 index 000000000000..5fb1c821bf71 --- /dev/null +++ b/learning/prompts/code-explanation/10_io_parquet.md @@ -0,0 +1,39 @@ +Prompt: +What does this code do? +```python +class ReadParquetOptions(PipelineOptions): + + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + "--path", + help="GCS path to parquet file") + + options = ReadParquetOptions() + +with beam.Pipeline(options=options) as p: + (p | "Read Parquet" >> ReadFromParquet(options.path) + | Map(logging.info)) +``` +Response: +This code reads data from [Apache Parquet](https://parquet.apache.org/) files using the `ReadFromParquet` transform from the built-in [ParquetIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.parquetio.html). + +```python +class ReadParquetOptions(PipelineOptions): + + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + "--path", + help="GCS path to parquet file") + + options = ReadParquetOptions() +``` +The `ReadParquetOptions` class is used to define the command-line argument `--path`, which specifies the path to the Parquet file. This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) for the required `path` argument. + +```python +with beam.Pipeline(options=options) as p: + (p | "Read Parquet" >> ReadFromParquet(options.path) + | Map(logging.info)) +``` +A Beam pipeline is created using the `ReadParquetOptions` class and the [ReadFromParquet transform](https://beam.apache.org/releases/pydoc/current/apache_beam.io.parquetio.html#apache_beam.io.parquetio.ReadFromParquet) is used to read data from the Parquet file. diff --git a/learning/prompts/code-explanation/11_io_jdbc.md b/learning/prompts/code-explanation/11_io_jdbc.md new file mode 100644 index 000000000000..72b5f02a5f89 --- /dev/null +++ b/learning/prompts/code-explanation/11_io_jdbc.md @@ -0,0 +1,105 @@ +Prompt: +What does this code do? +```python +class ExampleRow(NamedTuple): + id: int + name: str + +class JdbcOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--table_name', + help='Table name' + ) + parser.add_argument( + '--jdbc_url', + help='JDBC URL' + ) + parser.add_argument( + '--driver_class_name', + help='Driver class name' + ) + parser.add_argument( + '--username', + help='Username' + ) + parser.add_argument( + '--password', + help='Password' + ) + +options = JdbcOptions() +coders.registry.register_coder(ExampleRow, coders.RowCoder) + +with beam.Pipeline(options=options) as p: + + output = (p | 'Read from JDBC' >> ReadFromJdbc( + table_name=options.table_name, + driver_class_name=options.driver_class_name, + jdbc_url=options.jdbc_url, + username=options.username, + password=options.password + ) + | 'Log Data' >> Map(logging.info)) +``` +Response: +This code reads data from an [Oracle Java Database Connectivity (JDBC) enabled database](https://docs.oracle.com/javase/8/docs/technotes/guides/jdbc/) using the `ReadFromJdbc` transform from the built-in [JdbcIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.jdbc.html). + +```python +class ExampleRow(NamedTuple): + id: int + name: str +``` +This code defines a named tuple `ExampleRow` with two fields: `id` and `name`. + +```python +class JdbcOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--table_name', + help='Table name' + ) + parser.add_argument( + '--jdbc_url', + help='JDBC URL' + ) + parser.add_argument( + '--driver_class_name', + help='Driver class name' + ) + parser.add_argument( + '--username', + help='Username' + ) + parser.add_argument( + '--password', + help='Password' + ) + +options = JdbcOptions() +``` + +The `JdbcOptions` class is used to define the command-line arguments `--table_name`, `--jdbc_url`, `--driver_class_name`, `--username`, and `--password`, which specify the JDBC connection parameters. This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) to parse command-line arguments. + +```python +coders.registry.register_coder(ExampleRow, coders.RowCoder) +``` + +This code registers a coder for the `ExampleRow` class. This is required for the `ReadFromJdbc` transform to read data from the table. + +```python +with beam.Pipeline(options=options) as p: + + output = (p | 'Read from JDBC' >> ReadFromJdbc( + jdbc_url=options.jdbc_url, + username=options.username, + password=options.password, + table_name=options.table_name, + driver_class_name=options.driver_class_name + ) + | 'Log Data' >> Map(logging.info)) +``` + +A Beam pipeline is created using the `JdbcOptions` class and the [ReadFromJdbc transform](https://beam.apache.org/releases/pydoc/current/apache_beam.io.jdbc.html#apache_beam.io.jdbc.ReadFromJdbc) is used to read data from the database via JDBC. \ No newline at end of file diff --git a/learning/prompts/code-generation/01_io_kafka.md b/learning/prompts/code-generation/01_io_kafka.md new file mode 100644 index 000000000000..e169e012ef42 --- /dev/null +++ b/learning/prompts/code-generation/01_io_kafka.md @@ -0,0 +1,46 @@ +Prompt: +Write the Python code to read data from Apache Kafka using Apache Beam. + +Response: +You can read data from [Apache Kafka](https://kafka.apache.org/) using the `ReadFromKafka` transform. The following Python code reads a topic `topic` from a Kafka server `bootstrap_servers`. The values for `topic` and `bootstrap_servers` are provided as command-line arguments. The data is logged to the console. + +```python +import logging +import apache_beam as beam + +from apache_beam import Map +from apache_beam.io.kafka import ReadFromKafka +from apache_beam.options.pipeline_options import PipelineOptions + +class KafkaOptions(PipelineOptions): +""" +Configure pipeline options for Kafka read transform. +""" + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--bootstrap_servers', + default="localhost:9092", + help='Apache Kafka bootstrap servers' + ) + parser.add_argument( + '--topic', + default="your-topic", + help='Apache Kafka topic' + ) + +#parse pipeline options +options = KafkaOptions() + +with beam.Pipeline(options=options) as p: + #this pipeline reads from a Kafka topic and logs the data to the console + output = (p | "Read from Kafka" >> ReadFromKafka( + consumer_config={'bootstrap.servers': options.bootstrap_servers}, + topics=[options.topic], + with_metadata=False + ) + | "Log Data" >> Map(logging.info)) +``` +For more information about how to use the KafkaIO connector with the Python SDK for Apache Beam, see the [KafkaIO connector documentation](https://beam.apache.org/releases/pydoc/current/apache_beam.io.kafka.html). + +For samples that show common pipeline configurations to parse pipeline options from command-line arguments, see [Pipeline option patterns](https://beam.apache.org/documentation/patterns/pipeline-options/). \ No newline at end of file diff --git a/learning/prompts/code-generation/03_io_bigquery.md b/learning/prompts/code-generation/03_io_bigquery.md new file mode 100644 index 000000000000..b13be4fd355a --- /dev/null +++ b/learning/prompts/code-generation/03_io_bigquery.md @@ -0,0 +1,54 @@ +Prompt: +Write the Python code to read data from BigQuery using Apache Beam. + +Response: +You can read data from BigQuery using the `ReadFromBigQuery` transform. The following Python code reads data from a BigQuery table. The table name is provided as the command-line argument `table`. The data is logged to the console. + +```python +import logging + +import apache_beam as beam +from apache_beam.io import ReadFromBigQuery +from apache_beam.options.pipeline_options import PipelineOptions + +class BigQueryReadOptions(PipelineOptions): +""" +Configure pipeline options for BigQuery read transform. +""" + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + "--table", + required=True, + help="BigQuery table to read from.") + +def read_table(): + """Read from BigQuery table function.""" + + #parse pipeline options + #streaming=True is required for a streaming pipeline + options = BigQueryReadOptions(streaming=True) + + with beam.Pipeline(options=options) as p: + #this pipeline reads from a BigQuery table and logs the data to the console + (p | "Read BigQuery table" >> ReadFromBigQuery(table=options.table) + | "Format data" >> Map(lambda row: f"Received row:\n{row}\n") + | Map(logging.info)) + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + read_table() +``` +The `ReadFromBigQuery` transform also supports reading from a BigQuery query. The following Python code reads data from a BigQuery table using a SQL query. The query is provided as the command-line argument `query`. The data is logged to the console. + +```python +with beam.Pipeline(options=options) as p: + p | "Read BigQuery table" >> ReadFromBigQuery(query='SELECT * FROM table') + | "Format data" >> Map(lambda row: f"Received row:\n{row}\n") + | Map(logging.info) + ``` + +For more information, see the [BigQueryIO connector documentation](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.bigquery.html). + +For samples that show common pipeline configurations, see [Pipeline option patterns](https://beam.apache.org/documentation/patterns/pipeline-options/). + diff --git a/learning/prompts/code-generation/04_io_bigtable.md b/learning/prompts/code-generation/04_io_bigtable.md new file mode 100644 index 000000000000..22ea5ebcef9e --- /dev/null +++ b/learning/prompts/code-generation/04_io_bigtable.md @@ -0,0 +1,66 @@ +Prompt: +Write the Python code to read data from Bigtable using Apache Beam. + +Response: +You can read data from [Cloud Bigtable](https://cloud.google.com/bigtable) using the `ReadFromBigtable` transform. The following Python code reads a table `table_id` from a Bigtable instance `instance_id`. The `table_id` and `instance_id` are provided as command-line arguments. The data is logged to the console. + +```python +import logging +import apache_beam as beam + +from apache_beam import Map +from apache_beam.io.gcp.bigtableio import ReadFromBigtable +from apache_beam.options.pipeline_options import PipelineOptions + +class BigtableOptions(PipelineOptions): +""" +Configure pipeline options for Bigtable read transform. +""" + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--project_id', + required=True, + help='Project ID' + ) + parser.add_argument( + '--instance_id', + default="beam-test", + help='Cloud Bigtable instance ID' + ) + parser.add_argument( + '--table_id', + default="your-test-table", + help='Cloud Bigtable table ID' + ) + + +def run(): + """ + This pipeline shows how to read from Cloud Bigtable. + """ + + #parse pipeline options + options = BigtableOptions() + + with beam.Pipeline(options=options) as p: + #this pipeline reads from a BigQuery table and logs the data to the console + output = (p | "Read from Bigtable" >> ReadFromBigtable( + project_id=options.project_id, + instance_id=options.instance_id, + table_id=options.table_id + ) + | "Extract cells" >> beam.Map(lambda row: row._cells) + | "Log Data" >> Map(logging.info)) + + +if __name__ == "__main__": + logging.getLogger().setLevel(logging.INFO) + run() +``` +The `ReadFromBigtable` transform returns a `PCollection` of `PartialRowData` objects, each representing a Bigtable row. For more information about this row object, see [PartialRowData (row_key)](https://cloud.google.com/python/docs/reference/bigtable/latest/row#class-googlecloudbigtablerowpartialrowdatarowkey) + +For more information, see the [BigTableIO connector documentation](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.bigtableio.html). + +For samples that show common pipeline configurations, see [Pipeline option patterns](https://beam.apache.org/documentation/patterns/pipeline-options/). + diff --git a/learning/prompts/code-generation/05_io_spanner.md b/learning/prompts/code-generation/05_io_spanner.md new file mode 100644 index 000000000000..24453ff01beb --- /dev/null +++ b/learning/prompts/code-generation/05_io_spanner.md @@ -0,0 +1,63 @@ +Prompt: +Write the Python code to read data from Cloud Spanner using Apache Beam. + +Response: +You can read data from [Cloud Spanner](https://cloud.google.com/spanner) using the Apache Beam [SpannerIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.spanner.html). The SpannerIO connector lets you read data from and write data to Cloud Spanner. Cloud Spanner is supported in both batch pipelines and streaming pipelines. + +```python +import logging +import apache_beam as beam + +from apache_beam import coders +from apache_beam import Map +from apache_beam.io.gcp.spanner import ReadFromSpanner +from apache_beam.options.pipeline_options import PipelineOptions +from typing import NamedTuple + +class ExampleRow(NamedTuple): + id: int + name: str + +class SpannerOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--project_id', + default='your-project-id', + help='Google Cloud project ID' + ) + parser.add_argument( + '--instance_id', + default='your-instance-id', + help='Google Cloud Spanner instance ID' + ) + parser.add_argument( + '--database_id', + default='your-database-id', + help='Google Cloud Spanner database ID' + ) + +options = SpannerOptions() +coders.registry.register_coder(ExampleRow, coders.RowCoder) + +with beam.Pipeline(options=options) as p: + + output = (p | "Read from table" >> ReadFromSpanner( + project_id=options.project_id, + instance_id=options.instance_id, + database_id=options.database_id, + row_type=ExampleRow, + sql="SELECT * FROM example_row" + ) + | "Log Data" >> Map(logging.info)) +``` + +The `ReadFromSpanner` transform is a built-in Apache Beam transform that reads data from a Cloud Spanner table. The `ReadFromSpanner` transform returns a `PCollection` of `NamedTuple` objects. The `NamedTuple` object is a Python class that represents a row in a Cloud Spanner table. + +Registering a coder for `NamedTuple` is required to use `NamedTuple` as a row type: +```python + coders.registry.register_coder(ExampleRow, coders.RowCoder) +``` +For more information about how to register a coder for a custom type, see [Data encoding and type safety](https://beam.apache.org/documentation/programming-guide/#data-encoding-and-type-safety). + +The `SpannerOptions` class defines the command-line arguments `project_id`, `instance_id` and `database_id`, which are used to configure the `ReadFromSpanner` transform. These arguments are parsed from the command line using [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/). \ No newline at end of file diff --git a/learning/prompts/code-generation/06_io_tfrecord.md b/learning/prompts/code-generation/06_io_tfrecord.md new file mode 100644 index 000000000000..7f36540b97b8 --- /dev/null +++ b/learning/prompts/code-generation/06_io_tfrecord.md @@ -0,0 +1,60 @@ +Prompt: +Write the Python code to read data from a TensorFlow TFRecord file using Apache Beam. + +Response: +You can read data from a TensorFlow [TFRecord file](https://www.tensorflow.org/api_docs/python/tf/data/TFRecordDataset) using the Apache Beam [TFRecordIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.tfrecordio.html). The TFRecordIO connector lets you read and write data in `TFRecord` format. + +```python +import logging + +import apache_beam as beam +from apache_beam import Map +from apache_beam.io.tfrecordio import ReadFromTFRecord +from apache_beam.options.pipeline_options import PipelineOptions + + +class TFRecordOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + "--file_pattern", + help="A file glob pattern to read TFRecords from." + ) + +def run(): + options = TFRecordOptions() + + with beam.Pipeline(options=options) as p: + + output = ( + p + | "Read from TFRecord" >> ReadFromTFRecord( + file_pattern=options.file_pattern + ) + | "Map from bytes" >> Map(map_from_bytes) + | "Log Data" >> Map(logging.info) + ) + + +def map_from_bytes(element): + """ + Deserializes the input bytes using pickle library and + returns the reconstructed object. + """ + # third party libraries + import pickle + + return pickle.loads(element) + + +if __name__ == "__main__": + logging.getLogger().setLevel(logging.INFO) + run() + +``` + +The `TFRecordOptions` class defines the command-line argument `file_pattern`, which is used to configure the `ReadFromTFRecord` transform. This argument is parsed from the command line using [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/). + +The `ReadFromTFRecord` transform is a built-in Apache Beam transform that reads data from a `TFRecord` file. By default, `TFRecordIO` transforms use `coders.BytesCoder()`. For more information, see [ReadFromTFRecord transform](https://beam.apache.org/releases/pydoc/current/apache_beam.io.tfrecordio.html#apache_beam.io.tfrecordio.ReadFromTFRecord). + + diff --git a/learning/prompts/code-generation/07_io_json.md b/learning/prompts/code-generation/07_io_json.md new file mode 100644 index 000000000000..6c142b49c450 --- /dev/null +++ b/learning/prompts/code-generation/07_io_json.md @@ -0,0 +1,48 @@ +Prompt: +Write the Python code to read data from JSON files using Apache Beam. + +Response: +You can read data from [JSON files](https://www.json.org/) using the Apache Beam [TextIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.textio.html). + +```python +import logging +import apache_beam as beam + +from apache_beam import Map +from apache_beam.io.textio import ReadFromJson +from apache_beam.options.pipeline_options import PipelineOptions + + +class JsonOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--file_path', + default="gs://your-bucket/your-file.json", + help='Json file path' + ) + + +def run(): + """ + This pipeline reads from Json file defined by the --file_path argument. + """ + + options = JsonOptions() + + with beam.Pipeline(options=options) as p: + + output = p | "Read from Json file" >> ReadFromJson( + path=options.file_path, + lines=False + ) + + +if __name__ == "__main__": + logging.getLogger().setLevel(logging.INFO) + run() +``` +The `JsonOptions` class is a custom class that extends the `PipelineOptions` object. It is used to parse pipeline options from command-line arguments. This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) to parse command-line arguments. + +The [ReadFromJson](https://beam.apache.org/releases/pydoc/current/apache_beam.io.textio.html#apache_beam.io.textio.ReadFromJson) transform is a built-in Apache Beam transform that reads data from a JSON file. + diff --git a/learning/prompts/code-generation/08_io_csv.md b/learning/prompts/code-generation/08_io_csv.md new file mode 100644 index 000000000000..3888e15bb704 --- /dev/null +++ b/learning/prompts/code-generation/08_io_csv.md @@ -0,0 +1,47 @@ +Prompt: +Write the Python code to read data from CSV files using Apache Beam. + +Response: +You can read data from [CSV files](https://en.wikipedia.org/wiki/Comma-separated_values) using the Apache Beam [TextIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.textio.html). + +```python +import logging +import apache_beam as beam + +from apache_beam import Map +from apache_beam.io.textio import ReadFromCsv +from apache_beam.options.pipeline_options import PipelineOptions + + +class CsvOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--file_path', + help='Csv file path' + ) + + +def run(): + """ + This pipeline shows how to read from Csv file. + """ + + options = CsvOptions() + + with beam.Pipeline(options=options) as p: + + output = (p | "Read from Csv file" >> ReadFromCsv( + path=options.file_path + ) + | "Log Data" >> Map(logging.info)) + + +if __name__ == "__main__": + logging.getLogger().setLevel(logging.INFO) + run() +``` + +The `CsvOptions` class is a custom class that extends the `PipelineOptions` object. It is used to parse pipeline options from command-line arguments. This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) to parse command-line arguments. + +The [ReadFromCsv transform](https://beam.apache.org/releases/pydoc/current/apache_beam.io.textio.html#apache_beam.io.textio.ReadFromCsv) is a built-in Apache Beam transform that reads data from a CSV file. \ No newline at end of file diff --git a/learning/prompts/code-generation/09_io_avro.md b/learning/prompts/code-generation/09_io_avro.md new file mode 100644 index 000000000000..2ccf5427822c --- /dev/null +++ b/learning/prompts/code-generation/09_io_avro.md @@ -0,0 +1,42 @@ +Prompt: +Write the Python code to read data from Apache Avro files using Apache Beam. + +Response: +You can read data from [Apache Avro](https://avro.apache.org/) files using the Apache Beam [AvroIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.avroio.html). + +```python +import logging + +import apache_beam as beam +from apache_beam import Map +from apache_beam.io.avroio import ReadFromAvro +from apache_beam.options.pipeline_options import PipelineOptions + + +def run(argv=None): + + class ReadAvroOptions(PipelineOptions): + + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + "--path", + help="GCS path to Avro file") + + options = ReadAvroOptions() + + with beam.Pipeline(options=options) as p: + + (p | "ReadAvro" >> ReadFromAvro(options.path) + | Map(logging.info)) + + +if __name__ == "__main__": + logging.getLogger().setLevel(logging.INFO) + run() +``` +The `ReadAvroOptions` class is used to define the command-line argument `--path`, which specifies the path to the Avro file. This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) for the required `path` argument. + +A Beam pipeline is created using the `ReadAvroOptions` class and the [ReadFromAvro transform](https://beam.apache.org/releases/pydoc/current/apache_beam.io.avroio.html#apache_beam.io.avroio.ReadFromAvro) is used to read data from the Avro file. + + diff --git a/learning/prompts/code-generation/10_io_parquet.md b/learning/prompts/code-generation/10_io_parquet.md new file mode 100644 index 000000000000..5493c5e14d55 --- /dev/null +++ b/learning/prompts/code-generation/10_io_parquet.md @@ -0,0 +1,39 @@ +Prompt: +Write the Python code to read data from Apache Parquet files using Apache Beam. + +Response: +You can read data from [Apache Parquet](https://parquet.apache.org/) files using the Apache Beam [ParquetIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.parquetio.html). +```python +import logging + +import apache_beam as beam +from apache_beam import Map +from apache_beam.io.parquetio import ReadFromParquet +from apache_beam.options.pipeline_options import PipelineOptions + + +def run(argv=None): + + class ReadParquetOptions(PipelineOptions): + + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + "--path", + help="GCS path to Parquet file") + + options = ReadParquetOptions() + + with beam.Pipeline(options=options) as p: + + (p | "Read Parquet" >> ReadFromParquet(options.path) + | Map(logging.info)) + + +if __name__ == "__main__": + logging.getLogger().setLevel(logging.INFO) + run() +``` +The `ReadParquetOptions` class is used to define the command-line argument `--path`, which specifies the path to the Parquet file. This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/) for the required `path` argument. + +A Beam pipeline is created using the `ReadParquetOptions` class and the [ReadFromParquet transform](https://beam.apache.org/releases/pydoc/current/apache_beam.io.parquetio.html#apache_beam.io.parquetio.ReadFromParquet) is used to read data from the Parquet file. diff --git a/learning/prompts/code-generation/11_io_jdbc.md b/learning/prompts/code-generation/11_io_jdbc.md new file mode 100644 index 000000000000..9d8a98d6e649 --- /dev/null +++ b/learning/prompts/code-generation/11_io_jdbc.md @@ -0,0 +1,75 @@ +Prompt: +Write the Python code to read data from Oracle JDBC using Apache Beam. + +Response: +You can read data from an Oracle Java Database Connectivity (JDBC) enabled database using the Apache Beam [JdbcIO connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.jdbc.html). +```python +import logging +import apache_beam as beam +import typing + +from apache_beam import coders +from apache_beam import Map +from apache_beam.io.jdbc import ReadFromJdbc +from apache_beam.options.pipeline_options import PipelineOptions +from typing import NamedTuple + + +class ExampleRow(NamedTuple): + id: int + name: str + + +class JdbcOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--table_name', + help='Table name' + ) + parser.add_argument( + '--jdbc_url', + help='JDBC URL' + ) + parser.add_argument( + '--driver_class_name', + help='Driver class name' + ) + parser.add_argument( + '--username', + help='Username' + ) + parser.add_argument( + '--password', + help='Password' + ) + + +def run(): + """ + This pipeline shows how to read from JDBC. + """ + + options = JdbcOptions() + coders.registry.register_coder(ExampleRow, coders.RowCoder) + + with beam.Pipeline(options=options) as p: + + output = (p | 'Read from JDBC' >> ReadFromJdbc( + table_name=options.table_name, + driver_class_name=options.driver_class_name, + jdbc_url=options.jdbc_url, + username=options.username, + password=options.password + ) + | 'Log Data' >> Map(logging.info)) + + +if __name__ == "__main__": + logging.getLogger().setLevel(logging.INFO) + run() +``` + +The `JdbcOptions` class is a custom class that extends the `PipelineOptions` object. It is used to parse pipeline options from command-line arguments. This code uses [pipeline options](https://beam.apache.org/documentation/patterns/pipeline-options/). + +A Beam pipeline is created using the `JdbcOptions` class and the [ReadFromJdbc transform](https://beam.apache.org/releases/pydoc/current/apache_beam.io.jdbc.html#apache_beam.io.jdbc.ReadFromJdbc) is used to read data from the database via JDBC. diff --git a/learning/prompts/documentation-lookup/18_io_kafka.md b/learning/prompts/documentation-lookup/18_io_kafka.md new file mode 100644 index 000000000000..28c22508109d --- /dev/null +++ b/learning/prompts/documentation-lookup/18_io_kafka.md @@ -0,0 +1,27 @@ +Prompt: +Is Apache Kafka supported in Apache Beam? + +Response: +[Apache Kafka](https://kafka.apache.org/) is a distributed streaming platform that is used for building real-time data pipelines and streaming applications. It is horizontally scalable, fault-tolerant, and fast. +Apache Beam has a built-in KafkaIO connector for Apache Kafka that lets you read data from and write data to Kafka. Apache Kafka is supported in both batch pipelines and streaming pipelines, and as a source and a sink. For more information, see the KafkaIO connector documentation: +* [Java](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/kafka/KafkaIO.html) +* [Python (using cross-language transforms)](https://beam.apache.org/releases/pydoc/current/apache_beam.io.kafka.html) +* [Go (using cross-language transforms)](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/xlang/kafkaio) +* [Typescript (using cross-language transforms)](https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/io/kafka.ts) + +The [Dataflow cookbook repository](https://github.com/GoogleCloudPlatform/dataflow-cookbook) can help you to get started with Apache Kafka and Apache Beam. + +Apache Beam pipeline code for reading data from a Kafka topic might look like the following example: + +```python +with beam.Pipeline(options=options) as p: + output = (p | "Read from Kafka" >> ReadFromKafka( + consumer_config={'bootstrap.servers': 'localhost:9092'}, + topics=['my_topic'], + with_metadata=False) + | "Log Data" >> Map(logging.info)) +``` +This code reads data from a Kafka topic `my_topic` on `localhost:9092` and logs the data to the console. + +The Apache Beam repository has a [KafkaIO example](https://github.com/apache/beam/blob/master/sdks/python/apache_beam/examples/kafkataxi/kafka_taxi.py) that demonstrates how to use the KafkaIO connector. + diff --git a/learning/prompts/documentation-lookup/21_io_bigtable.md b/learning/prompts/documentation-lookup/21_io_bigtable.md new file mode 100644 index 000000000000..60f564256f4b --- /dev/null +++ b/learning/prompts/documentation-lookup/21_io_bigtable.md @@ -0,0 +1,28 @@ +Prompt: +Is Cloud BigTable supported in Apache Beam? + +Response: +[Cloud Bigtable](https://cloud.google.com/bigtable) is a [Google Cloud](https://cloud.google.com/) service that provides enterprise-grade NoSQL database service with single-digit millisecond latency, limitless scale, and 99.999% availability for large analytical and operational workloads. + +Apache Beam has a built-in connector for Cloud Bigtable that lets you read data from and write data to Bigtable. For more information, see the BigtableIO connector documentation: +* [Java](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.html) +* [Python](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.bigtableio.html) +* [Python (using cross-language transforms)](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/xlang/bigtableio) +* [Go](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/bigtableio) +* [Go (using cross-language transforms)](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/xlang/bigtableio) + +The [Dataflow cookbook repository](https://github.com/GoogleCloudPlatform/dataflow-cookbook) can help you to get started with Cloud Bigtable and Apache Beam. + +Apache Beam pipeline code for reading data from a Bigtable table might look like the following example: + +```python +with beam.Pipeline(options=options) as p: +output = (p | "Read from Bigtable" >> ReadFromBigtable( + project_id=project_id, + instance_id=instance_id, + table_id=table_id, + ) + | "Extract cells" >> beam.Map(lambda row: row._cells) + | "Log Data" >> Map(logging.info)) +``` +For performance benchmarks of the Bigtable IO connector, see [Bigtable performance](https://beam.apache.org/performance/bigtable/). \ No newline at end of file diff --git a/learning/prompts/documentation-lookup/22_io_spanner.md b/learning/prompts/documentation-lookup/22_io_spanner.md new file mode 100644 index 000000000000..b7a4cfb18aad --- /dev/null +++ b/learning/prompts/documentation-lookup/22_io_spanner.md @@ -0,0 +1,31 @@ +Prompt: +Is Cloud Spanner supported in Apache Beam? + +Response: +[Cloud Spanner](https://cloud.google.com/spanner) is a [Google Cloud](https://cloud.google.com/) service that provides a fully managed, mission-critical, relational database service that offers transactional consistency at global scale, schemas, SQL (ANSI 2011 with extensions), and automatic, synchronous replication for high availability. + +Apache Beam has a built-in SpannerIO connector for Cloud Spanner that lets you read data from and write data to Cloud Spanner. Cloud Spanner is supported in both batch pipelines and streaming pipelines, and as a source and a sink. For more information, see the SpannerIO connector documentation: +* [Java](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.html) +* [Python (using cross-language transforms)](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.spanner.html) +* [Go](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/spannerio) + +The [Dataflow cookbook repository](https://github.com/GoogleCloudPlatform/dataflow-cookbook) can help you to get started with Cloud Spanner and Apache Beam. + +Apache Beam pipeline code for reading data from a Cloud Spanner table using a SQL query might look like the following example: + +```python +class ExampleRow(NamedTuple): + id: int + name: str + +with beam.Pipeline(options=options) as p: + output = (p | "Read from table" >> ReadFromSpanner( + project_id=options.project_id, + instance_id=options.instance_id, + database_id=options.database_id, + row_type=ExampleRow, + sql="SELECT * FROM example_row" + ) + | "Map Data" >> Map(lambda row: f"Id = {row.id}, Name = {row.name}") + | "Log Data" >> Map(logging.info)) +``` diff --git a/learning/prompts/documentation-lookup/23_io_gcs.md b/learning/prompts/documentation-lookup/23_io_gcs.md new file mode 100644 index 000000000000..e67d6fb02ccf --- /dev/null +++ b/learning/prompts/documentation-lookup/23_io_gcs.md @@ -0,0 +1,23 @@ +Prompt: +Is Google Cloud Storage file system supported in Apache Beam? + +Response: +[Google Cloud Storage](https://cloud.google.com/storage) is an object storage service for storing and accessing data on Google Cloud. Apache Beam supports both working with the Google Cloud Storage file system and reading data from and writing data to Google Cloud Storage buckets. + +Apache Beam has a built-in TextIO connector for reading data from and writing data to Google Cloud Storage buckets. For more information, see the TextIO connector documentation: +* [Java connector](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/TextIO.html) +* [Python connector](https://beam.apache.org/releases/pydoc/current/apache_beam.io.textio.html) +* [Go connector](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/textio) +* [Typescript connector (using cross-language transforms)](https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/io/textio.ts) + +Reading and writing data in different file formats in Google Cloud Storage buckets is also supported in the AvroIO, XMLIO, TFRecordIO, and ParquetIO connectors. + +Specify the Google Cloud Storage path using the following format: `gs:///`. For example, `gs://my-bucket/my-file.txt`. +You can use wildcards in read transforms and write transforms to read data from and write data to multiple files. For example, `gs://my-bucket/my-files-*.txt`. + +Apache Beam also has support for the Google Cloud Storage file system that lets you work with the Google Cloud Storage file system directly. For example, you can use the Google Cloud Storage file system to check whether a file exists, list files in a directory, or delete a file. The Google Cloud Storage file system is supported in the following languages: +* [Java file system](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/extensions/gcp/storage/GcsFileSystemRegistrar.html) +* [Python file system](https://beam.apache.org/releases/pydoc/current/apache_beam.io.gcp.gcsfilesystem.html) +* [Go file system](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/filesystem/gcs) + + diff --git a/learning/prompts/documentation-lookup/24_io_json.md b/learning/prompts/documentation-lookup/24_io_json.md new file mode 100644 index 000000000000..b1c2d89d7429 --- /dev/null +++ b/learning/prompts/documentation-lookup/24_io_json.md @@ -0,0 +1,32 @@ +Prompt: +Is JSON supported in Apache Beam? + +Response: +[JSON](https://www.json.org/) is a data format for storing and exchanging data. Apache Beam supports reading and writing JSON files using the `ReadFromJson` and `WriteToJson` transforms in the `TextIO` module. For more information, see the TextIO connector documentation: +* [Java](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/TextIO.html) +* [Python](https://beam.apache.org/releases/pydoc/current/apache_beam.io.textio.html) +* [Go](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/textio) +* [Typescript](https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/io/textio.ts) + +The [Dataflow cookbook repository](https://github.com/GoogleCloudPlatform/dataflow-cookbook) can help you to get started with JSON and Apache Beam. + +Apache Beam pipeline code for reading data from a JSON file might look like the following example: + +```python +class JsonOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--file_path', + default="gs://your-bucket/your-file.json", + help='Json file path' + ) + +options = JsonOptions() +with beam.Pipeline(options=options) as p: + output = (p | "Read from Json file" >> ReadFromJson( + path=options.file_path, + lines=False + ) + | "Log Data" >> Map(logging.info)) +``` \ No newline at end of file diff --git a/learning/prompts/documentation-lookup/25_io_csv.md b/learning/prompts/documentation-lookup/25_io_csv.md new file mode 100644 index 000000000000..8803fa28c799 --- /dev/null +++ b/learning/prompts/documentation-lookup/25_io_csv.md @@ -0,0 +1,32 @@ +Prompt: +Is CSV format supported in Apache Beam? + +Response: +[CSV](https://en.wikipedia.org/wiki/Comma-separated_values) is a data format for storing and exchanging data. Apache Beam supports reading data from and writing data to CSV files using the `ReadFromText` and `WriteToText` transforms in the `TextIO` module. For more information, see the TextIO connector documentation: +* [Java](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/TextIO.html) +* [Python](https://beam.apache.org/releases/pydoc/current/apache_beam.io.textio.html) +* [Go](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/textio) +* [Typescript](https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/io/textio.ts) + +The [Dataflow cookbook repository](https://github.com/GoogleCloudPlatform/dataflow-cookbook) can help you to get started with CSV and Apache Beam. + +Apache Beam pipeline code for reading data from a CSV file might look like the following example: +```python +class CsvOptions(PipelineOptions): + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + '--file_path', + default="gs://your-bucket/your-file.csv", + help='Csv file path' + ) + +options = CsvOptions() + +with beam.Pipeline(options=options) as p: + output = (p | "Read from Csv file" >> ReadFromCsv( + path=options.file_path + ) + | "Log Data" >> Map(logging.info)) + +``` \ No newline at end of file diff --git a/learning/prompts/documentation-lookup/26_io_avro.md b/learning/prompts/documentation-lookup/26_io_avro.md new file mode 100644 index 000000000000..4161c47710c8 --- /dev/null +++ b/learning/prompts/documentation-lookup/26_io_avro.md @@ -0,0 +1,31 @@ +Prompt: +Is Apache Avro format supported in Apache Beam? + +Response: +[Apache Avro](https://avro.apache.org/) is a data format for storing and exchanging data. Apache Beam supports reading from and writing to Avro files using the `ReadFromAvro` and `WriteToAvro` transforms in the `AvroIO` module. For more information, see the AvroIO connector documentation: +* [Java](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/extensions/avro/io/AvroIO.html) +* [Python](https://beam.apache.org/releases/pydoc/current/apache_beam.io.avroio.html) +* [Go](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/avroio) +* [Typescript (using cross-language transforms)](https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/io/avroio.ts) + +The [Dataflow cookbook repository](https://github.com/GoogleCloudPlatform/dataflow-cookbook) can help you to get started with Avro and Apache Beam. + +Apache Beam pipeline code for reading data from an Avro file might look like the following example: +```python + +class ReadAvroOptions(PipelineOptions): + + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + "--path", + default="gs://cloud-samples-data/bigquery/us-states/*.avro", + help="GCS path to read from") + + options = ReadAvroOptions() + + with beam.Pipeline(options=options) as p: + + (p | "Read from Avro" >> ReadFromAvro(options.path) + | Map(logging.info)) +``` diff --git a/learning/prompts/documentation-lookup/27_io_parquet.md b/learning/prompts/documentation-lookup/27_io_parquet.md new file mode 100644 index 000000000000..34ec68924218 --- /dev/null +++ b/learning/prompts/documentation-lookup/27_io_parquet.md @@ -0,0 +1,31 @@ +Prompt: +Is Apache Parquet format supported in Apache Beam? + +Response: +[Apache Parquet](https://parquet.apache.org/) is a data format for storing and exchanging data. Apache Beam supports reading from and writing to Parquet files using the `ReadFromParquet` and `WriteToParquet` transforms in the `ParquetIO` module. For more information, see the ParquetIO connector documentation: +* [Java](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/parquet/ParquetIO.html) +* [Python](https://beam.apache.org/releases/pydoc/current/apache_beam.io.parquetio.html) +* [Go](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/parquetio) +* [Typescript (using cross-language transforms)](https://github.com/apache/beam/blob/master/sdks/typescript/src/apache_beam/io/parquetio.ts) + + +The [Dataflow cookbook repository](https://github.com/GoogleCloudPlatform/dataflow-cookbook) can help you to get started with Avro and Apache Beam. + +Apache Beam pipeline code for reading data from a Parquet file might look like the following example: +```python +class ReadParquetOptions(PipelineOptions): + + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + "--path", + help="GCS path to parquet file") + + options = ReadParquetOptions() + + with beam.Pipeline(options=options) as p: + + (p | "ReadParquet" >> ReadFromParquet(options.path) + | Map(logging.info)) + +``` diff --git a/learning/prompts/documentation-lookup/28_io_jdbc.md b/learning/prompts/documentation-lookup/28_io_jdbc.md new file mode 100644 index 000000000000..13d6502c5bc8 --- /dev/null +++ b/learning/prompts/documentation-lookup/28_io_jdbc.md @@ -0,0 +1,52 @@ +Prompt: +Is Oracle Java Database Connectivity (JDBC) supported in Apache Beam? + +Response: +The [Oracle Java Database Connectivity (JDBC) API](https://docs.oracle.com/javase/8/docs/technotes/guides/jdbc/) is a Java API for connecting to relational databases. Apache Beam supports reading and writing data with JDBC using the `ReadFromJdbc` and `WriteToJdbc` transforms in the `JdbcIO` module. For more information, see the JdbcIO connector documentation: +* [Java](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/jdbc/JdbcIO.html) +* [Python (using cross-language transforms)](https://beam.apache.org/releases/pydoc/current/apache_beam.io.jdbc.html) +* [Go (using cross-language transforms)](https://pkg.go.dev/github.com/apache/beam/sdks/v2/go/pkg/beam/io/xlang/jdbcio) + +The [Dataflow cookbook repository](https://github.com/GoogleCloudPlatform/dataflow-cookbook) can help you to get started with JDBC and Apache Beam. + +Apache Beam pipeline code for reading data with JDBC might look like the following example: +```python +class ExampleRow(NamedTuple): + id: int + name: str + + +class ReadJdbcOptions(PipelineOptions): + + @classmethod + def _add_argparse_args(cls, parser): + parser.add_argument( + "--driverClassName", + help="JDBC driver class name") + parser.add_argument( + "--url", + help="JDBC URL") + parser.add_argument( + "--username", + help="JDBC username") + parser.add_argument( + "--password", + help="JDBC password") + parser.add_argument( + "--query", + default="SELECT * FROM users", + help="JDBC query") + +options = ReadJdbcOptions() + +with beam.Pipeline(options=options) as p: + + (p | "Read from JDBC" >> ReadFromJdbc( + driverClassName=options.driverClassName, + url=options.url, + username=options.username, + password=options.password, + query=options.query) + | Map(logging.info)) +``` + From 801682930e25376f471a6c733365123d670d1b4f Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 2 Feb 2024 09:55:53 -0800 Subject: [PATCH 24/50] Bump google.golang.org/protobuf from 1.31.0 to 1.32.0 in /sdks (#29913) Bumps google.golang.org/protobuf from 1.31.0 to 1.32.0. --- updated-dependencies: - dependency-name: google.golang.org/protobuf dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 80a017c09170..d6eed60bda91 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -62,7 +62,7 @@ require ( google.golang.org/api v0.154.0 google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f google.golang.org/grpc v1.60.1 - google.golang.org/protobuf v1.31.0 + google.golang.org/protobuf v1.32.0 gopkg.in/retry.v1 v1.0.3 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 diff --git a/sdks/go.sum b/sdks/go.sum index 06b251d29a8d..3a44301e9366 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -730,8 +730,8 @@ google.golang.org/protobuf v1.23.1-0.20200526195155-81db48ad09cc/go.mod h1:EGpAD google.golang.org/protobuf v1.25.0/go.mod h1:9JNX74DMeImyA3h4bdi1ymwjUzf21/xIlbajtzgsN7c= google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= -google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= +google.golang.org/protobuf v1.32.0 h1:pPC6BG5ex8PDFnkbrGU3EixyhKcQ2aDuBS36lqK/C7I= +google.golang.org/protobuf v1.32.0/go.mod h1:c6P6GXX6sHbq/GpV6MGZEdwhWPcYBgnhAHhKbcUYpos= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= From 30a778b9cbeb7b79615daa371fc3041a0dda2090 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 2 Feb 2024 10:07:52 -0800 Subject: [PATCH 25/50] Bump github.com/containerd/containerd from 1.7.7 to 1.7.11 in /sdks (#29823) Bumps [github.com/containerd/containerd](https://github.com/containerd/containerd) from 1.7.7 to 1.7.11. - [Release notes](https://github.com/containerd/containerd/releases) - [Changelog](https://github.com/containerd/containerd/blob/main/RELEASES.md) - [Commits](https://github.com/containerd/containerd/compare/v1.7.7...v1.7.11) --- updated-dependencies: - dependency-name: github.com/containerd/containerd dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index d6eed60bda91..ebcea04a37cf 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -75,7 +75,7 @@ require ( require ( dario.cat/mergo v1.0.0 // indirect - github.com/Microsoft/hcsshim v0.11.1 // indirect + github.com/Microsoft/hcsshim v0.11.4 // indirect github.com/containerd/log v0.1.0 // indirect github.com/frankban/quicktest v1.14.0 // indirect github.com/go-logr/logr v1.3.0 // indirect @@ -131,7 +131,7 @@ require ( github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cncf/udpa/go v0.0.0-20220112060539-c52dc94e7fbe // indirect github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4 // indirect - github.com/containerd/containerd v1.7.7 // indirect + github.com/containerd/containerd v1.7.11 // indirect github.com/cpuguy83/dockercfg v0.3.1 // indirect github.com/docker/distribution v2.8.2+incompatible // indirect github.com/docker/docker v24.0.7+incompatible // but required to resolve issue docker has with go1.20 diff --git a/sdks/go.sum b/sdks/go.sum index 3a44301e9366..7ef59c4f4692 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -64,8 +64,8 @@ github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym github.com/JohnCGriffin/overflow v0.0.0-20211019200055-46fa312c352c h1:RGWPOewvKIROun94nF7v2cua9qP+thov/7M50KEoeSU= github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= -github.com/Microsoft/hcsshim v0.11.1 h1:hJ3s7GbWlGK4YVV92sO88BQSyF4ZLVy7/awqOlPxFbA= -github.com/Microsoft/hcsshim v0.11.1/go.mod h1:nFJmaO4Zr5Y7eADdFOpYswDDlNVbvcIJJNJLECr5JQg= +github.com/Microsoft/hcsshim v0.11.4 h1:68vKo2VN8DE9AdN4tnkWnmdhqdbpUFM8OF3Airm7fz8= +github.com/Microsoft/hcsshim v0.11.4/go.mod h1:smjE4dvqPX9Zldna+t5FG3rnoHhaB7QYxPRqGcpAD9w= github.com/andybalholm/brotli v1.0.4 h1:V7DdXeJtZscaqfNuAdSRuRFzuiKlHSC/Zh3zl9qY3JY= github.com/andybalholm/brotli v1.0.4/go.mod h1:fO7iG3H7G2nSZ7m0zPUDn85XEX2GTukHGRSepvi9Eig= github.com/apache/arrow/go/arrow v0.0.0-20200730104253-651201b0f516 h1:byKBBF2CKWBjjA4J1ZL2JXttJULvWSl50LegTyRZ728= @@ -149,8 +149,8 @@ github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWH github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4 h1:/inchEIKaYC1Akx+H+gqO04wryn5h75LSazbRlnya1k= github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= -github.com/containerd/containerd v1.7.7 h1:QOC2K4A42RQpcrZyptP6z9EJZnlHfHJUfZrAAHe15q4= -github.com/containerd/containerd v1.7.7/go.mod h1:3c4XZv6VeT9qgf9GMTxNTMFxGJrGpI2vz1yk4ye+YY8= +github.com/containerd/containerd v1.7.11 h1:lfGKw3eU35sjV0aG2eYZTiwFEY1pCzxdzicHP3SZILw= +github.com/containerd/containerd v1.7.11/go.mod h1:5UluHxHTX2rdvYuZ5OJTC5m/KJNs0Zs9wVoJm9zf5ZE= github.com/containerd/log v0.1.0 h1:TCJt7ioM2cr/tfR8GPbGf9/VRAX8D2B4PjzCpfX540I= github.com/containerd/log v0.1.0/go.mod h1:VRRf09a7mHDIRezVKTRCrOq78v577GXq3bSa3EhrzVo= github.com/cpuguy83/dockercfg v0.3.1 h1:/FpZ+JaygUR/lZP2NlFI2DVfrOEMAIKP5wWEJdoYe9E= From f03b115830bee1e7515f8386b77441ecf8359c69 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Fri, 2 Feb 2024 07:37:21 -0500 Subject: [PATCH 26/50] Remove extraneous mocking that was causing NPEs in DataflowWorkUnitClientTest --- .../worker/DataflowWorkUnitClientTest.java | 99 ++++++++++++------- 1 file changed, 61 insertions(+), 38 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java index 59d8c55c52d3..fac56890f498 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkUnitClientTest.java @@ -18,11 +18,7 @@ package org.apache.beam.runners.dataflow.worker; import static org.junit.Assert.assertEquals; -import static org.mockito.ArgumentMatchers.anyString; -import static org.mockito.Mockito.doCallRealMethod; -import static org.mockito.Mockito.when; -import com.google.api.client.http.LowLevelHttpResponse; import com.google.api.client.json.Json; import com.google.api.client.testing.http.MockHttpTransport; import com.google.api.client.testing.http.MockLowLevelHttpRequest; @@ -53,7 +49,6 @@ import org.apache.beam.sdk.util.FastNanoClockAndSleeper; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -61,8 +56,6 @@ import org.junit.rules.Timeout; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import org.mockito.Mock; -import org.mockito.MockitoAnnotations; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -75,36 +68,34 @@ public class DataflowWorkUnitClientTest { private static final String PROJECT_ID = "TEST_PROJECT_ID"; private static final String JOB_ID = "TEST_JOB_ID"; private static final String WORKER_ID = "TEST_WORKER_ID"; + @Rule public TestRule restoreSystemProperties = new RestoreSystemProperties(); @Rule public TestRule restoreLogging = new RestoreDataflowLoggingMDC(); @Rule public ExpectedException expectedException = ExpectedException.none(); @Rule public FastNanoClockAndSleeper fastNanoClockAndSleeper = new FastNanoClockAndSleeper(); - @Mock private MockHttpTransport transport; - @Mock private MockLowLevelHttpRequest request; - private DataflowWorkerHarnessOptions pipelineOptions; - - @Before - public void setUp() throws Exception { - MockitoAnnotations.initMocks(this); - when(transport.buildRequest(anyString(), anyString())).thenReturn(request); - doCallRealMethod().when(request).getContentAsString(); + DataflowWorkerHarnessOptions createPipelineOptionsWithTransport(MockHttpTransport transport) { Dataflow service = new Dataflow(transport, Transport.getJsonFactory(), null); - pipelineOptions = PipelineOptionsFactory.as(DataflowWorkerHarnessOptions.class); + DataflowWorkerHarnessOptions pipelineOptions = + PipelineOptionsFactory.as(DataflowWorkerHarnessOptions.class); pipelineOptions.setProject(PROJECT_ID); pipelineOptions.setJobId(JOB_ID); pipelineOptions.setWorkerId(WORKER_ID); pipelineOptions.setGcpCredential(new TestCredential()); pipelineOptions.setDataflowClient(service); pipelineOptions.setRegion("us-central1"); + return pipelineOptions; } @Test public void testCloudServiceCall() throws Exception { WorkItem workItem = createWorkItem(PROJECT_ID, JOB_ID); - when(request.execute()).thenReturn(generateMockResponse(workItem)); - + MockLowLevelHttpResponse response = generateMockResponse(workItem); + MockLowLevelHttpRequest request = new MockLowLevelHttpRequest().setResponse(response); + MockHttpTransport transport = + new MockHttpTransport.Builder().setLowLevelHttpRequest(request).build(); + DataflowWorkerHarnessOptions pipelineOptions = createPipelineOptionsWithTransport(transport); WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); assertEquals(Optional.of(workItem), client.getWorkItem()); @@ -124,30 +115,40 @@ public void testCloudServiceCall() throws Exception { @Test public void testCloudServiceCallMapTaskStagePropagation() throws Exception { - WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); - // Publish and acquire a map task work item, and verify we're now processing that stage. final String stageName = "test_stage_name"; MapTask mapTask = new MapTask(); mapTask.setStageName(stageName); WorkItem workItem = createWorkItem(PROJECT_ID, JOB_ID); workItem.setMapTask(mapTask); - when(request.execute()).thenReturn(generateMockResponse(workItem)); + + MockLowLevelHttpResponse response = generateMockResponse(workItem); + MockLowLevelHttpRequest request = new MockLowLevelHttpRequest().setResponse(response); + MockHttpTransport transport = + new MockHttpTransport.Builder().setLowLevelHttpRequest(request).build(); + DataflowWorkerHarnessOptions pipelineOptions = createPipelineOptionsWithTransport(transport); + WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); + assertEquals(Optional.of(workItem), client.getWorkItem()); assertEquals(stageName, DataflowWorkerLoggingMDC.getStageName()); } @Test public void testCloudServiceCallSeqMapTaskStagePropagation() throws Exception { - WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); - // Publish and acquire a seq map task work item, and verify we're now processing that stage. final String stageName = "test_stage_name"; SeqMapTask seqMapTask = new SeqMapTask(); seqMapTask.setStageName(stageName); WorkItem workItem = createWorkItem(PROJECT_ID, JOB_ID); workItem.setSeqMapTask(seqMapTask); - when(request.execute()).thenReturn(generateMockResponse(workItem)); + + MockLowLevelHttpResponse response = generateMockResponse(workItem); + MockLowLevelHttpRequest request = new MockLowLevelHttpRequest().setResponse(response); + MockHttpTransport transport = + new MockHttpTransport.Builder().setLowLevelHttpRequest(request).build(); + DataflowWorkerHarnessOptions pipelineOptions = createPipelineOptionsWithTransport(transport); + WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); + assertEquals(Optional.of(workItem), client.getWorkItem()); assertEquals(stageName, DataflowWorkerLoggingMDC.getStageName()); } @@ -157,8 +158,11 @@ public void testCloudServiceCallNoWorkPresent() throws Exception { // If there's no work the service should return an empty work item. WorkItem workItem = new WorkItem(); - when(request.execute()).thenReturn(generateMockResponse(workItem)); - + MockLowLevelHttpResponse response = generateMockResponse(workItem); + MockLowLevelHttpRequest request = new MockLowLevelHttpRequest().setResponse(response); + MockHttpTransport transport = + new MockHttpTransport.Builder().setLowLevelHttpRequest(request).build(); + DataflowWorkerHarnessOptions pipelineOptions = createPipelineOptionsWithTransport(transport); WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); assertEquals(Optional.empty(), client.getWorkItem()); @@ -181,8 +185,11 @@ public void testCloudServiceCallNoWorkId() throws Exception { WorkItem workItem = createWorkItem(PROJECT_ID, JOB_ID); workItem.setId(null); - when(request.execute()).thenReturn(generateMockResponse(workItem)); - + MockLowLevelHttpResponse response = generateMockResponse(workItem); + MockLowLevelHttpRequest request = new MockLowLevelHttpRequest().setResponse(response); + MockHttpTransport transport = + new MockHttpTransport.Builder().setLowLevelHttpRequest(request).build(); + DataflowWorkerHarnessOptions pipelineOptions = createPipelineOptionsWithTransport(transport); WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); assertEquals(Optional.empty(), client.getWorkItem()); @@ -201,8 +208,11 @@ public void testCloudServiceCallNoWorkId() throws Exception { @Test public void testCloudServiceCallNoWorkItem() throws Exception { - when(request.execute()).thenReturn(generateMockResponse()); - + MockLowLevelHttpResponse response = generateMockResponse(); + MockLowLevelHttpRequest request = new MockLowLevelHttpRequest().setResponse(response); + MockHttpTransport transport = + new MockHttpTransport.Builder().setLowLevelHttpRequest(request).build(); + DataflowWorkerHarnessOptions pipelineOptions = createPipelineOptionsWithTransport(transport); WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); assertEquals(Optional.empty(), client.getWorkItem()); @@ -228,8 +238,11 @@ public void testCloudServiceCallMultipleWorkItems() throws Exception { WorkItem workItem1 = createWorkItem(PROJECT_ID, JOB_ID); WorkItem workItem2 = createWorkItem(PROJECT_ID, JOB_ID); - when(request.execute()).thenReturn(generateMockResponse(workItem1, workItem2)); - + MockLowLevelHttpResponse response = generateMockResponse(workItem1, workItem2); + MockLowLevelHttpRequest request = new MockLowLevelHttpRequest().setResponse(response); + MockHttpTransport transport = + new MockHttpTransport.Builder().setLowLevelHttpRequest(request).build(); + DataflowWorkerHarnessOptions pipelineOptions = createPipelineOptionsWithTransport(transport); WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); client.getWorkItem(); @@ -242,7 +255,13 @@ public void testReportWorkerMessage_streamingScalingReport() throws Exception { SendWorkerMessagesResponse workerMessage = new SendWorkerMessagesResponse(); workerMessage.setFactory(Transport.getJsonFactory()); response.setContent(workerMessage.toPrettyString()); - when(request.execute()).thenReturn(response); + + MockLowLevelHttpRequest request = new MockLowLevelHttpRequest().setResponse(response); + MockHttpTransport transport = + new MockHttpTransport.Builder().setLowLevelHttpRequest(request).build(); + DataflowWorkerHarnessOptions pipelineOptions = createPipelineOptionsWithTransport(transport); + WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); + StreamingScalingReport activeThreadsReport = new StreamingScalingReport() .setActiveThreadCount(1) @@ -251,7 +270,6 @@ public void testReportWorkerMessage_streamingScalingReport() throws Exception { .setMaximumThreadCount(4) .setMaximumBundleCount(5) .setMaximumBytes(6L); - WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); WorkerMessage msg = client.createWorkerMessageFromStreamingScalingReport(activeThreadsReport); client.reportWorkerMessage(Collections.singletonList(msg)); @@ -268,7 +286,13 @@ public void testReportWorkerMessage_perWorkerMetrics() throws Exception { SendWorkerMessagesResponse workerMessage = new SendWorkerMessagesResponse(); workerMessage.setFactory(Transport.getJsonFactory()); response.setContent(workerMessage.toPrettyString()); - when(request.execute()).thenReturn(response); + + MockLowLevelHttpRequest request = new MockLowLevelHttpRequest().setResponse(response); + MockHttpTransport transport = + new MockHttpTransport.Builder().setLowLevelHttpRequest(request).build(); + DataflowWorkerHarnessOptions pipelineOptions = createPipelineOptionsWithTransport(transport); + WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); + PerStepNamespaceMetrics stepNamespaceMetrics = new PerStepNamespaceMetrics() .setOriginalStep("s1") @@ -279,7 +303,6 @@ public void testReportWorkerMessage_perWorkerMetrics() throws Exception { new PerWorkerMetrics() .setPerStepNamespaceMetrics(Collections.singletonList(stepNamespaceMetrics)); - WorkUnitClient client = new DataflowWorkUnitClient(pipelineOptions, LOG); WorkerMessage perWorkerMetricsMsg = client.createWorkerMessageFromPerWorkerMetrics(perWorkerMetrics); client.reportWorkerMessage(Collections.singletonList(perWorkerMetricsMsg)); @@ -290,7 +313,7 @@ public void testReportWorkerMessage_perWorkerMetrics() throws Exception { assertEquals(ImmutableList.of(perWorkerMetricsMsg), actualRequest.getWorkerMessages()); } - private LowLevelHttpResponse generateMockResponse(WorkItem... workItems) throws Exception { + private MockLowLevelHttpResponse generateMockResponse(WorkItem... workItems) throws Exception { MockLowLevelHttpResponse response = new MockLowLevelHttpResponse(); response.setContentType(Json.MEDIA_TYPE); LeaseWorkItemResponse lease = new LeaseWorkItemResponse(); From 00f07864784dc71b131244533abce4aaad48b5fa Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 1 Feb 2024 21:44:48 -0500 Subject: [PATCH 27/50] Remove sdks/java/fn-execution --- build.gradle.kts | 1 - runners/direct-java/build.gradle | 2 -- sdks/java/expansion-service/build.gradle | 2 -- sdks/java/fn-execution/build.gradle | 34 ------------------------ sdks/java/harness/build.gradle | 3 +-- settings.gradle.kts | 1 - 6 files changed, 1 insertion(+), 42 deletions(-) delete mode 100644 sdks/java/fn-execution/build.gradle diff --git a/build.gradle.kts b/build.gradle.kts index d4bd0fca4fc1..82b3b6356a69 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -293,7 +293,6 @@ tasks.register("javaPreCommit") { dependsOn(":sdks:java:extensions:sorter:build") dependsOn(":sdks:java:extensions:timeseries:build") dependsOn(":sdks:java:extensions:zetasketch:build") - dependsOn(":sdks:java:fn-execution:build") dependsOn(":sdks:java:harness:build") dependsOn(":sdks:java:harness:jmh:build") dependsOn(":sdks:java:io:bigquery-io-perf-tests:build") diff --git a/runners/direct-java/build.gradle b/runners/direct-java/build.gradle index e0365fb839ce..39fc1d2a53c9 100644 --- a/runners/direct-java/build.gradle +++ b/runners/direct-java/build.gradle @@ -26,7 +26,6 @@ def dependOnProjects = [":runners:core-construction-java", ":runners:core-java", ":runners:local-java", ":runners:java-fn-execution", - ":sdks:java:fn-execution", ":sdks:java:extensions:avro" ] @@ -95,7 +94,6 @@ dependencies { validatesRunner project(path: project.path, configuration: "shadowTest") permitUnusedDeclared library.java.vendored_grpc_1_60_1 permitUnusedDeclared project(":runners:java-fn-execution") - permitUnusedDeclared project(":sdks:java:fn-execution") permitUnusedDeclared project(":sdks:java:extensions:avro") examplesJavaIntegrationTest project(project.path) examplesJavaIntegrationTest project(":examples:java") diff --git a/sdks/java/expansion-service/build.gradle b/sdks/java/expansion-service/build.gradle index 18df128b9386..6947e53354db 100644 --- a/sdks/java/expansion-service/build.gradle +++ b/sdks/java/expansion-service/build.gradle @@ -41,10 +41,8 @@ dependencies { implementation project(path: ":sdks:java:core", configuration: "shadow") implementation project(path: ":runners:core-construction-java") implementation project(path: ":runners:java-fn-execution") - implementation project(path: ":sdks:java:fn-execution") implementation project(path: ":sdks:java:harness") permitUnusedDeclared project(path: ":model:fn-execution") - permitUnusedDeclared project(path: ":sdks:java:fn-execution") implementation library.java.jackson_annotations implementation library.java.jackson_databind implementation library.java.jackson_dataformat_yaml diff --git a/sdks/java/fn-execution/build.gradle b/sdks/java/fn-execution/build.gradle deleted file mode 100644 index 01a073bb7ae7..000000000000 --- a/sdks/java/fn-execution/build.gradle +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * License); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an AS IS BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -plugins { id 'org.apache.beam.module' } -applyJavaNature( - automaticModuleName: 'org.apache.beam.sdk.fn', -) - -description = "Apache Beam :: SDKs :: Java :: Fn Execution" -ext.summary = """Contains code shared across the Beam Java SDK Harness and Java Runners to execute using -the Beam Portability Framework.""" - -dependencies { - testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") - testImplementation library.java.junit - testImplementation library.java.mockito_core - testImplementation library.java.commons_lang3 - testRuntimeOnly library.java.slf4j_jdk14 -} diff --git a/sdks/java/harness/build.gradle b/sdks/java/harness/build.gradle index 52c20e62c498..d3b4267424ce 100644 --- a/sdks/java/harness/build.gradle +++ b/sdks/java/harness/build.gradle @@ -90,8 +90,7 @@ dependencies { permitUnusedDeclared project(path: ":sdks:java:transform-service:launcher") testImplementation library.java.junit testImplementation library.java.mockito_core - shadowTestRuntimeClasspath project(path: ":sdks:java:core", configuration: "shadowTest") - testImplementation project(path: ":sdks:java:fn-execution", configuration: "testRuntimeMigration") + shadowTest project(path: ":sdks:java:core", configuration: "shadowTest") shadowTestRuntimeClasspath library.java.slf4j_jdk14 permitUnusedDeclared library.java.avro } diff --git a/settings.gradle.kts b/settings.gradle.kts index b1b03b55add0..034df0fbda5f 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -203,7 +203,6 @@ include(":sdks:java:extensions:sql:udf") include(":sdks:java:extensions:sql:udf-test-provider") include(":sdks:java:extensions:timeseries") include(":sdks:java:extensions:zetasketch") -include(":sdks:java:fn-execution") include(":sdks:java:harness") include(":sdks:java:harness:jmh") include(":sdks:java:io:amazon-web-services") From 9f8dda274dc7f7f562fb10bc9153d40ebf593ff0 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 2 Feb 2024 21:51:47 -0500 Subject: [PATCH 28/50] Split standalone transform service application from launcher jar (#30190) --- build.gradle.kts | 1 + .../beam/gradle/BeamModulePlugin.groovy | 6 +-- sdks/java/extensions/python/build.gradle | 2 +- sdks/java/harness/build.gradle | 2 +- sdks/java/transform-service/app/build.gradle | 43 +++++++++++++++++++ .../transform-service/launcher/build.gradle | 23 +++------- .../utils/transform_service_launcher.py | 2 +- settings.gradle.kts | 1 + .../en/documentation/programming-guide.md | 4 +- 9 files changed, 59 insertions(+), 25 deletions(-) create mode 100644 sdks/java/transform-service/app/build.gradle diff --git a/build.gradle.kts b/build.gradle.kts index 82b3b6356a69..7e4c6cdc71ab 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -312,6 +312,7 @@ tasks.register("javaPreCommit") { dependsOn(":sdks:java:testing:tpcds:build") dependsOn(":sdks:java:testing:watermarks:build") dependsOn(":sdks:java:transform-service:build") + dependsOn(":sdks:java:transform-service:app:build") dependsOn(":sdks:java:transform-service:launcher:build") } diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index f3d79c528f8c..10a7ba7f1c84 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2860,7 +2860,7 @@ class BeamModulePlugin implements Plugin { project.evaluationDependsOn(":sdks:python") project.evaluationDependsOn(":runners:core-construction-java") project.evaluationDependsOn(":sdks:java:extensions:python") - project.evaluationDependsOn(":sdks:java:transform-service:launcher") + project.evaluationDependsOn(":sdks:java:transform-service:app") def usesDataflowRunner = config.pythonPipelineOptions.contains("--runner=TestDataflowRunner") || config.pythonPipelineOptions.contains("--runner=DataflowRunner") @@ -2868,7 +2868,7 @@ class BeamModulePlugin implements Plugin { def envDir = project.project(":sdks:python").envdir def pythonDir = project.project(":sdks:python").projectDir def externalPort = getRandomPort() - def launcherJar = project.project(':sdks:java:transform-service:launcher').shadowJar.archivePath + def launcherJar = project.project(':sdks:java:transform-service:app').shadowJar.archivePath def groupId = project.name + randomUUID().toString() def transformServiceOpts = [ "transform_service_launcher_jar": launcherJar, @@ -2895,7 +2895,7 @@ class BeamModulePlugin implements Plugin { dependsOn ':sdks:python:expansion-service-container:docker' dependsOn ':sdks:java:expansion-service:container:docker' dependsOn ":sdks:python:installGcpTest" - dependsOn project.project(':sdks:java:transform-service:launcher').shadowJar.getPath() + dependsOn project.project(':sdks:java:transform-service:app').shadowJar.getPath() if (usesDataflowRunner) { dependsOn ":sdks:python:test-suites:dataflow:py${project.ext.pythonVersion.replace('.', '')}:initializeForDataflowJob" diff --git a/sdks/java/extensions/python/build.gradle b/sdks/java/extensions/python/build.gradle index 40729d5f9cab..a83afac68815 100644 --- a/sdks/java/extensions/python/build.gradle +++ b/sdks/java/extensions/python/build.gradle @@ -30,7 +30,7 @@ dependencies { implementation project(path: ":model:pipeline", configuration: "shadow") implementation project(path: ":runners:core-construction-java") implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation project(path: ":sdks:java:transform-service:launcher", configuration: "shadow") + implementation project(path: ":sdks:java:transform-service:launcher") testImplementation library.java.junit testImplementation library.java.hamcrest testImplementation project(":runners:core-construction-java").sourceSets.test.output diff --git a/sdks/java/harness/build.gradle b/sdks/java/harness/build.gradle index d3b4267424ce..3c50f3c8edf2 100644 --- a/sdks/java/harness/build.gradle +++ b/sdks/java/harness/build.gradle @@ -29,7 +29,7 @@ dependencies { // :sdks:java:core and transitive dependencies provided project(path: ":model:pipeline", configuration: "shadow") provided project(path: ":sdks:java:core", configuration: "shadow") - provided project(path: ":sdks:java:transform-service:launcher", configuration: "shadow") + provided project(path: ":sdks:java:transform-service:launcher") provided library.java.avro provided library.java.joda_time provided library.java.slf4j_api diff --git a/sdks/java/transform-service/app/build.gradle b/sdks/java/transform-service/app/build.gradle new file mode 100644 index 000000000000..5125899bec9b --- /dev/null +++ b/sdks/java/transform-service/app/build.gradle @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * License); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +apply plugin: 'org.apache.beam.module' +apply plugin: 'application' + +applyJavaNature( + automaticModuleName: 'org.apache.beam.sdk.transform.service.app', + exportJavadoc: false, + validateShadowJar: false, + shadowClosure: {}, +) +mainClassName = "org.apache.beam.sdk.transformservice.launcher.TransformServiceLauncher" + +description = "Apache Beam :: SDKs :: Java :: Transform Service :: App" +ext.summary = """Runs a transform service.""" + +dependencies { + implementation project(":sdks:java:transform-service:launcher") + permitUnusedDeclared project(":sdks:java:transform-service:launcher") + runtimeOnly library.java.slf4j_jdk14 +} + +jar { + manifest { + attributes 'Main-Class': application.mainClass + } +} diff --git a/sdks/java/transform-service/launcher/build.gradle b/sdks/java/transform-service/launcher/build.gradle index 0952f37109eb..e225d8da2f1d 100644 --- a/sdks/java/transform-service/launcher/build.gradle +++ b/sdks/java/transform-service/launcher/build.gradle @@ -16,17 +16,11 @@ * limitations under the License. */ apply plugin: 'org.apache.beam.module' -apply plugin: 'application' -mainClassName = "org.apache.beam.sdk.transformservice.launcher.TransformServiceLauncher" applyJavaNature( - automaticModuleName: 'org.apache.beam.sdk.transform.service', - exportJavadoc: false, - validateShadowJar: false, - shadowClosure: {}, + automaticModuleName: 'org.apache.beam.sdk.transform.service', ) - description = "Apache Beam :: SDKs :: Java :: Transform Service :: Launcher" ext.summary = """Contains code that can be used to run an transform service.""" @@ -40,10 +34,11 @@ test { } dependencies { - shadow library.java.vendored_guava_32_1_2_jre - shadow library.java.slf4j_api - shadow library.java.args4j - shadow library.java.error_prone_annotations + // This library is designed to be light-weight, having minimum amount of dependencies and is used by Beam runner artifacts. + implementation library.java.vendored_guava_32_1_2_jre + implementation library.java.slf4j_api + implementation library.java.args4j + compileOnly library.java.error_prone_annotations permitUnusedDeclared(library.java.error_prone_annotations) testImplementation library.java.junit testImplementation library.java.mockito_core @@ -58,9 +53,3 @@ sourceSets { output.resourcesDir = "$buildDir/resources/docker-compose/$name" } } - -jar { - manifest { - attributes 'Main-Class': application.mainClass - } -} diff --git a/sdks/python/apache_beam/utils/transform_service_launcher.py b/sdks/python/apache_beam/utils/transform_service_launcher.py index ac492513aba5..ca3e456c5238 100644 --- a/sdks/python/apache_beam/utils/transform_service_launcher.py +++ b/sdks/python/apache_beam/utils/transform_service_launcher.py @@ -35,7 +35,7 @@ _COMMAND_POSSIBLE_VALUES = ['up', 'down', 'ps'] -_EXPANSION_SERVICE_LAUNCHER_JAR = ':sdks:java:transform-service:launcher:build' +_EXPANSION_SERVICE_LAUNCHER_JAR = ':sdks:java:transform-service:app:build' class TransformServiceLauncher(object): diff --git a/settings.gradle.kts b/settings.gradle.kts index 034df0fbda5f..512820fce3f1 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -277,6 +277,7 @@ include(":sdks:java:testing:test-utils") include(":sdks:java:testing:tpcds") include(":sdks:java:testing:watermarks") include(":sdks:java:transform-service") +include(":sdks:java:transform-service:app") include(":sdks:java:transform-service:launcher") include(":sdks:java:transform-service:controller-container") include(":sdks:python") diff --git a/website/www/site/content/en/documentation/programming-guide.md b/website/www/site/content/en/documentation/programming-guide.md index 47808063254f..d3333c1f9f64 100644 --- a/website/www/site/content/en/documentation/programming-guide.md +++ b/website/www/site/content/en/documentation/programming-guide.md @@ -8165,7 +8165,7 @@ Beam users also have the option to [manually start](/documentation/programming-g A Beam Transform service instance can be manually started by using utilities provided with Apache Beam SDKs. {{< highlight java >}} -java -jar beam-sdks-java-transform-service-launcher-.jar --port --beam_version --project_name --command up +java -jar beam-sdks-java-transform-service-app-.jar --port --beam_version --project_name --command up {{< /highlight >}} {{< highlight py >}} @@ -8179,7 +8179,7 @@ This feature is currently in development. To stop the transform service, use the following commands. {{< highlight java >}} -java -jar beam-sdks-java-transform-service-launcher-.jar --port --beam_version --project_name --command down +java -jar beam-sdks-java-transform-service-app-.jar --port --beam_version --project_name --command down {{< /highlight >}} {{< highlight py >}} From f2056e72dea146e4133d56251e10a028f406f143 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 2 Feb 2024 22:05:46 -0500 Subject: [PATCH 29/50] Fix stale dataflow prebuilt image cleaner (#30168) * Fix stale dataflow prebuilt image cleaner * Switch public and private * Defer fail script --- .../stale_dataflow_prebuilt_image_cleaner.sh | 55 +++++++++++-------- 1 file changed, 32 insertions(+), 23 deletions(-) diff --git a/.test-infra/tools/stale_dataflow_prebuilt_image_cleaner.sh b/.test-infra/tools/stale_dataflow_prebuilt_image_cleaner.sh index 804f6e2f39d4..126249324fed 100755 --- a/.test-infra/tools/stale_dataflow_prebuilt_image_cleaner.sh +++ b/.test-infra/tools/stale_dataflow_prebuilt_image_cleaner.sh @@ -27,14 +27,14 @@ PRIVATE_REPOSITORIES=(java-postcommit-it python-postcommit-it jenkins github-act # set as the same as 6-week release period if [[ $OSTYPE == "linux-gnu"* ]]; then # date command usage depending on OS - DELETE_BEFORE_DAY=$(date --iso-8601=s -d '6 weeks ago') + DELETE_BEFORE_PUBLIC=$(date --iso-8601=s -d '6 weeks ago') + DELETE_BEFORE_PRIVATE=$(date --iso-8601=s -d '3 days ago') elif [[ $OSTYPE == "darwin"* ]]; then - DELETE_BEFORE_DAY=$(date -j -v-6w '+%Y-%m-%dT%H:%M:%S') + DELETE_BEFORE_PUBLIC=$(date -j -v-6w '+%Y-%m-%dT%H:%M:%S') + DELETE_BEFORE_PRIVATE=$(date -j -v-3d '+%Y-%m-%dT%H:%M:%S') fi -REPOSITORIES=("${PUBLIC_REPOSITORIES[@]/#/gcr.io/apache-beam-testing/}" "${PRIVATE_REPOSITORIES[@]/#/us.gcr.io/apache-beam-testing/}") - -echo $REPOSITORIES +REPOSITORIES=("${PRIVATE_REPOSITORIES[@]/#/us.gcr.io/apache-beam-testing/}" "${PUBLIC_REPOSITORIES[@]/#/gcr.io/apache-beam-testing/}") # walk repos recursively IMAGE_NAMES="" @@ -54,8 +54,8 @@ while [ -n "$REPOSITORIES" ]; do REPOSITORIES=("${PENDING_REPOSITORIES[@]}") done -STALE_IMAGES="" -FAILED_INSPECT="" +HAS_STALE_IMAGES="" +FAILED_IMAGES="" for image_name in ${IMAGE_NAMES[@]}; do echo IMAGES FOR image ${image_name} @@ -64,22 +64,31 @@ for image_name in ${IMAGE_NAMES[@]}; do LATEST_IN_TIME=$(gcloud container images list-tags \ ${image_name} --sort-by="~TIMESTAMP" --filter="NOT tags:latest " --format="get(digest)" --limit=1) if [ -n "$LATEST_IN_TIME" ]; then + # decide timestamp cutoff + if [[ $image_name =~ 'us.gcr.io' ]]; then + DELETE_BEFORE_DAY=$DELETE_BEFORE_PRIVATE + else + DELETE_BEFORE_DAY=$DELETE_BEFORE_PUBLIC + fi # list containers of the image name echo "Command" gcloud container images list-tags \ ${image_name} \ --sort-by=TIMESTAMP --filter="NOT tags:latest AND timestamp.datetime < $DELETE_BEFORE_DAY" \ - --format="get(digest)" - STALE_IMAGES_CURRENT=$(gcloud container images list-tags \ + --format="get(digest,timestamp.year)" + STALE_IMAGES=$(gcloud container images list-tags \ ${image_name} \ --sort-by=TIMESTAMP --filter="NOT tags:latest AND timestamp.datetime < $DELETE_BEFORE_DAY" \ - --format="get(digest)") - STALE_IMAGES+=$STALE_IMAGES_CURRENT - for current in ${STALE_IMAGES_CURRENT[@]}; do + --format="get(digest,timestamp.year)") + + STALE_IMAGES_CURRENT=($STALE_IMAGES) + for (( i_stale_images_current=0; i_stale_images_current<${#STALE_IMAGES_CURRENT[@]} ; i_stale_images_current+=2 )) ; do + current=${STALE_IMAGES_CURRENT[i_stale_images_current]} + currentyear=${STALE_IMAGES_CURRENT[i_stale_images_current+1]} # do not delete the one with latest label and the newest image without latest label # this make sure we leave at least one container under each image name, either labelled "latest" or not if [ "$LATEST_IN_TIME" != "$current" ]; then - if [[ $image_name =~ 'beamgrafana' || $image_name =~ 'beammetricssyncjenkins' || $image_name =~ 'beammetricssyncgithub' ]]; then - # Skip docker manifest inspect for known single arch images, workaround permission issue & saving API call + if [[ $currentyear > 1970 ]]; then + # Skip docker manifest inspect for those not in epoch to save API call SHOULD_DELETE=0 else # Check to see if this image is built on top of earlier images. This is the case for multiarch images, @@ -89,15 +98,15 @@ for image_name in ${IMAGE_NAMES[@]}; do MANIFEST=$(docker manifest inspect ${image_name}@"${current}" || echo "") if [ -z "$MANIFEST" ]; then # Sometimes "no such manifest" seen. Skip current if command hit error - FAILED_INSPECT+=" $current" + FAILED_IMAGES+=" $current" continue fi SHOULD_DELETE=0 DIGEST=$(echo $MANIFEST | jq -r '.manifests[0].digest') if [ "$DIGEST" != "null" ]; then SHOULD_DELETE=1 - for i in ${STALE_IMAGES_CURRENT[@]}; do - if [ "$i" = "$DIGEST" ]; then + for (( j_stale_images_current=0; j_stale_images_current<${#STALE_IMAGES_CURRENT[@]} ; j_stale_images_current+=2 )) ; do + if [ "${STALE_IMAGES_CURRENT[j_stale_images_current]}" = "$DIGEST" ]; then SHOULD_DELETE=0 break fi @@ -105,8 +114,8 @@ for image_name in ${IMAGE_NAMES[@]}; do fi fi - if [ $SHOULD_DELETE = 0 ] - then + if [ $SHOULD_DELETE = 0 ]; then + HAS_STALE_IMAGES="true" echo "Deleting image. Command: gcloud container images delete ${image_name}@"${current}" --force-delete-tags -q" gcloud container images delete ${image_name}@"${current}" --force-delete-tags -q || FAILED_TO_DELETE+="${current} " fi @@ -120,18 +129,18 @@ for image_name in ${IMAGE_NAMES[@]}; do echo "Failed to delete the following images: ${FAILED_TO_DELETE}. Retrying each of them." for current in $RETRY_DELETE; do echo "Trying again to delete image ${image_name}@"${current}". Command: gcloud container images delete ${image_name}@"${current}" --force-delete-tags -q" - gcloud container images delete ${image_name}@"${current}" --force-delete-tags -q + gcloud container images delete ${image_name}@"${current}" --force-delete-tags -q || FAILED_IMAGES+=" ${image_name}@${current}" done fi done -if [[ ${STALE_IMAGES} ]]; then +if [[ -n "$HAS_STALE_IMAGES" ]]; then echo "Deleted multiple images" else echo "No stale prebuilt container images found." fi -if [ -n "$FAILED_INSPECT" ]; then - echo "Failed delete images $FAILED_INSPECT" +if [ -n "$FAILED_IMAGES" ]; then + echo "Failed delete images $FAILED_IMAGES" exit 1 fi \ No newline at end of file From 34bbbda9f9822301d1186a5d524480bcd8fbfa43 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Sun, 4 Feb 2024 09:21:09 -0800 Subject: [PATCH 30/50] Try a different approach for cibuildwheel flake. (#30204) --- sdks/python/build.gradle | 37 ++++++++++++++++++++++++++----------- 1 file changed, 26 insertions(+), 11 deletions(-) diff --git a/sdks/python/build.gradle b/sdks/python/build.gradle index ab6f75fc653b..7f2bc7f5d423 100644 --- a/sdks/python/build.gradle +++ b/sdks/python/build.gradle @@ -94,17 +94,32 @@ platform_identifiers_map.each { platform, idsuffix -> // generated installable Python SDK package doLast { - exec { - environment CIBW_BUILD: "cp${pyversion}-${idsuffix}" - environment CIBW_ENVIRONMENT: "SETUPTOOLS_USE_DISTUTILS=stdlib" - // note: sync cibuildwheel version with GitHub Action - // .github/workflow/build_wheel.yml:build_wheels "Install cibuildwheel" step - // note(https://github.com/pypa/cibuildwheel/issues/1692): cibuildwheel appears to timeout occasionally. - executable 'sh' - args '-c', ". ${envdir}/bin/activate && " + - "pip install cibuildwheel==2.9.0 && " + - "cibuildwheel --print-build-identifiers --platform ${platform} --archs ${archs} && " + - "for i in {1..3}; do cibuildwheel --output-dir ${buildDir} --platform ${platform} --archs ${archs} && break; done" + int maxRetries = 3 + int retryCount = 0 + // note(https://github.com/pypa/cibuildwheel/issues/1692): cibuildwheel appears to timeout occasionally. + while (retryCount < maxRetries) { + try { + exec { + environment CIBW_BUILD: "cp${pyversion}-${idsuffix}" + environment CIBW_ENVIRONMENT: "SETUPTOOLS_USE_DISTUTILS=stdlib" + executable 'sh' + args '-c', ". ${envdir}/bin/activate && " + + // note: sync cibuildwheel version with GitHub Action + // .github/workflows/build_wheel.yml:build_wheels "Install cibuildwheel" step + "pip install cibuildwheel==2.9.0 && " + + "cibuildwheel --print-build-identifiers --platform ${platform} --archs ${archs} && " + + "cibuildwheel --output-dir ${buildDir} --platform ${platform} --archs ${archs} " + } + break; + } + catch (Exception e) { + retryCount++ + if (retryCount < maxRetries) { + println "cibuildwheel failed on attempt ${retryCount}. Will retry." + } else { + throw e + } + } } } } From 1a1ff756f868416508be1de1f3afb2afb9b7fc73 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Feb 2024 09:09:22 -0500 Subject: [PATCH 31/50] Bump golang.org/x/sync from 0.5.0 to 0.6.0 in /sdks (#30209) Bumps [golang.org/x/sync](https://github.com/golang/sync) from 0.5.0 to 0.6.0. - [Commits](https://github.com/golang/sync/compare/v0.5.0...v0.6.0) --- updated-dependencies: - dependency-name: golang.org/x/sync dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index ebcea04a37cf..493e9f7e898d 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -56,7 +56,7 @@ require ( go.mongodb.org/mongo-driver v1.13.1 golang.org/x/net v0.19.0 golang.org/x/oauth2 v0.15.0 - golang.org/x/sync v0.5.0 + golang.org/x/sync v0.6.0 golang.org/x/sys v0.15.0 golang.org/x/text v0.14.0 google.golang.org/api v0.154.0 diff --git a/sdks/go.sum b/sdks/go.sum index 7ef59c4f4692..7578e41acec6 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -567,8 +567,8 @@ golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.5.0 h1:60k92dhOjHxJkrqnwsfl8KuaHbn/5dl0lUPUklKo3qE= -golang.org/x/sync v0.5.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.6.0 h1:5BMeUDZ7vkXGfEr1x9B4bRcTH4lpkTkpdh0T/J+qjbQ= +golang.org/x/sync v0.6.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190130150945-aca44879d564/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= From c1b3a27ee0c937568141b51dbbb2059873022826 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 5 Feb 2024 12:05:12 -0500 Subject: [PATCH 32/50] Bump github.com/nats-io/nats-server/v2 from 2.10.7 to 2.10.10 in /sdks (#30210) Bumps [github.com/nats-io/nats-server/v2](https://github.com/nats-io/nats-server) from 2.10.7 to 2.10.10. - [Release notes](https://github.com/nats-io/nats-server/releases) - [Changelog](https://github.com/nats-io/nats-server/blob/main/.goreleaser.yml) - [Commits](https://github.com/nats-io/nats-server/compare/v2.10.7...v2.10.10) --- updated-dependencies: - dependency-name: github.com/nats-io/nats-server/v2 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 12 ++++++------ sdks/go.sum | 24 ++++++++++++------------ 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 493e9f7e898d..4327d5297a02 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -45,8 +45,8 @@ require ( github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 github.com/lib/pq v1.10.9 github.com/linkedin/goavro/v2 v2.12.0 - github.com/nats-io/nats-server/v2 v2.10.7 - github.com/nats-io/nats.go v1.31.0 + github.com/nats-io/nats-server/v2 v2.10.10 + github.com/nats-io/nats.go v1.32.0 github.com/proullon/ramsql v0.1.3 github.com/spf13/cobra v1.8.0 github.com/testcontainers/testcontainers-go v0.26.0 @@ -57,7 +57,7 @@ require ( golang.org/x/net v0.19.0 golang.org/x/oauth2 v0.15.0 golang.org/x/sync v0.6.0 - golang.org/x/sys v0.15.0 + golang.org/x/sys v0.16.0 golang.org/x/text v0.14.0 google.golang.org/api v0.154.0 google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f @@ -84,7 +84,7 @@ require ( github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/minio/highwayhash v1.0.2 // indirect github.com/nats-io/jwt/v2 v2.5.3 // indirect - github.com/nats-io/nkeys v0.4.6 // indirect + github.com/nats-io/nkeys v0.4.7 // indirect github.com/nats-io/nuid v1.0.1 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect github.com/shirou/gopsutil/v3 v3.23.9 // indirect @@ -154,7 +154,7 @@ require ( github.com/inconshreveable/mousetrap v1.1.0 // indirect github.com/jmespath/go-jmespath v0.4.0 // indirect github.com/klauspost/asmfmt v1.3.2 // indirect - github.com/klauspost/compress v1.17.4 // indirect + github.com/klauspost/compress v1.17.5 // indirect github.com/klauspost/cpuid/v2 v2.2.6 // indirect github.com/magiconair/properties v1.8.7 // indirect github.com/minio/asm2plan9s v0.0.0-20200509001527-cdd76441f9d8 // indirect @@ -180,7 +180,7 @@ require ( github.com/youmark/pkcs8 v0.0.0-20181117223130-1be2e3e5546d // indirect github.com/zeebo/xxh3 v1.0.2 // indirect go.opencensus.io v0.24.0 // indirect - golang.org/x/crypto v0.17.0 // indirect + golang.org/x/crypto v0.18.0 // indirect golang.org/x/mod v0.11.0 // indirect golang.org/x/tools v0.10.0 // indirect golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index 7578e41acec6..3c90d6043312 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -317,8 +317,8 @@ github.com/klauspost/asmfmt v1.3.2/go.mod h1:AG8TuvYojzulgDAMCnYn50l/5QV3Bs/tp6j github.com/klauspost/compress v1.9.7/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= github.com/klauspost/compress v1.13.1/go.mod h1:8dP1Hq4DHOhN9w426knH3Rhby4rFm6D8eO+e+Dq5Gzg= github.com/klauspost/compress v1.13.6/go.mod h1:/3/Vjq9QcHkK5uEr5lBEmyoZ1iFhe47etQ6QUkpK6sk= -github.com/klauspost/compress v1.17.4 h1:Ej5ixsIri7BrIjBkRZLTo6ghwrEtHFk7ijlczPW4fZ4= -github.com/klauspost/compress v1.17.4/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= +github.com/klauspost/compress v1.17.5 h1:d4vBd+7CHydUqpFBgUEKkSdtSugf9YFmSkvUYPquI5E= +github.com/klauspost/compress v1.17.5/go.mod h1:/dCuZOvVtNoHsyb+cuJD3itjs3NbnF6KH9zAO4BDxPM= github.com/klauspost/cpuid/v2 v2.2.6 h1:ndNyv040zDGIDh8thGkXYjnFtiN02M1PVVF+JE/48xc= github.com/klauspost/cpuid/v2 v2.2.6/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -361,12 +361,12 @@ github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= github.com/morikuni/aec v1.0.0/go.mod h1:BbKIizmSmc5MMPqRYbxO4ZU0S0+P200+tUnFx7PXmsc= github.com/nats-io/jwt/v2 v2.5.3 h1:/9SWvzc6hTfamcgXJ3uYRpgj+QuY2aLNqRiqrKcrpEo= github.com/nats-io/jwt/v2 v2.5.3/go.mod h1:iysuPemFcc7p4IoYots3IuELSI4EDe9Y0bQMe+I3Bf4= -github.com/nats-io/nats-server/v2 v2.10.7 h1:f5VDy+GMu7JyuFA0Fef+6TfulfCs5nBTgq7MMkFJx5Y= -github.com/nats-io/nats-server/v2 v2.10.7/go.mod h1:V2JHOvPiPdtfDXTuEUsthUnCvSDeFrK4Xn9hRo6du7c= -github.com/nats-io/nats.go v1.31.0 h1:/WFBHEc/dOKBF6qf1TZhrdEfTmOZ5JzdJ+Y3m6Y/p7E= -github.com/nats-io/nats.go v1.31.0/go.mod h1:di3Bm5MLsoB4Bx61CBTsxuarI36WbhAwOm8QrW39+i8= -github.com/nats-io/nkeys v0.4.6 h1:IzVe95ru2CT6ta874rt9saQRkWfe2nFj1NtvYSLqMzY= -github.com/nats-io/nkeys v0.4.6/go.mod h1:4DxZNzenSVd1cYQoAa8948QY3QDjrHfcfVADymtkpts= +github.com/nats-io/nats-server/v2 v2.10.10 h1:g1Wd64J5SGsoqWSx1qoNu9/At7a2x+jE7Qtf2XpEx/I= +github.com/nats-io/nats-server/v2 v2.10.10/go.mod h1:/TE61Dos8NlwZnjzyE3ZlOnM6dgl7tf937dnf4VclrA= +github.com/nats-io/nats.go v1.32.0 h1:Bx9BZS+aXYlxW08k8Gd3yR2s73pV5XSoAQUyp1Kwvp0= +github.com/nats-io/nats.go v1.32.0/go.mod h1:Ubdu4Nh9exXdSz0RVWRFBbRfrbSxOYd26oF0wkWclB8= +github.com/nats-io/nkeys v0.4.7 h1:RwNJbbIdYCoClSDNY7QVKZlyb/wfT6ugvFCiKy6vDvI= +github.com/nats-io/nkeys v0.4.7/go.mod h1:kqXRgRDPlGy7nGaEDMuYzmiJCIAAWDK0IMBtDmGD0nc= github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw= github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= github.com/ncw/swift v1.0.52/go.mod h1:23YIA4yWVnGwv2dQlN4bB7egfYX6YLn0Yo/S6zZO/ZM= @@ -491,8 +491,8 @@ golang.org/x/crypto v0.0.0-20201002170205-7f63de1d35b0/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a/go.mod h1:P+XmwS30IXTQdn5tA2iutPOUgjI07+tq3H3K9MVA1s8= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220622213112-05595931fe9d/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.17.0 h1:r8bRNjWL3GshPW3gkd+RpvzWrZAwPS49OmTGZ/uhM4k= -golang.org/x/crypto v0.17.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= +golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= +golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -604,8 +604,8 @@ golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= -golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.16.0 h1:xWw16ngr6ZMtmxDyKyIgsE93KNKz5HKmMa3b8ALHidU= +golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= From a47b1faa5276cdbf05c356b60ed8c4494ee622aa Mon Sep 17 00:00:00 2001 From: Damon Date: Mon, 5 Feb 2024 17:23:10 +0000 Subject: [PATCH 33/50] [Prism] Implement jobservices.Server Cancel (#30178) * Implement jobservices.Server Cancel * Small code cleanup * Fix test err; canceled state after complete --- .../beam/runners/prism/internal/execute.go | 8 +++ .../runners/prism/internal/jobservices/job.go | 10 +++ .../prism/internal/jobservices/management.go | 30 +++++++++ .../internal/jobservices/management_test.go | 34 +++++++++++ .../prism/internal/jobservices/server_test.go | 61 +++++++++++++++++++ 5 files changed, 143 insertions(+) diff --git a/sdks/go/pkg/beam/runners/prism/internal/execute.go b/sdks/go/pkg/beam/runners/prism/internal/execute.go index b8bc68dcd1b7..1aa95bc6ee18 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/execute.go +++ b/sdks/go/pkg/beam/runners/prism/internal/execute.go @@ -17,6 +17,7 @@ package internal import ( "context" + "errors" "fmt" "io" "sort" @@ -70,6 +71,13 @@ func RunPipeline(j *jobservices.Job) { j.Failed(err) return } + + if errors.Is(context.Cause(j.RootCtx), jobservices.ErrCancel) { + j.SendMsg("pipeline canceled " + j.String()) + j.Canceled() + return + } + j.SendMsg("pipeline completed " + j.String()) j.SendMsg("terminating " + j.String()) diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go index bb5eb88c9193..6cde48ded9ac 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/job.go @@ -177,6 +177,16 @@ func (j *Job) Done() { j.sendState(jobpb.JobState_DONE) } +// Canceling indicates that the job is canceling. +func (j *Job) Canceling() { + j.sendState(jobpb.JobState_CANCELLING) +} + +// Canceled indicates that the job is canceled. +func (j *Job) Canceled() { + j.sendState(jobpb.JobState_CANCELLED) +} + // Failed indicates that the job completed unsuccessfully. func (j *Job) Failed(err error) { slog.Error("job failed", slog.Any("job", j), slog.Any("error", err)) diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go index 323d8c46efb1..0da37ef0bd73 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management.go @@ -17,6 +17,7 @@ package jobservices import ( "context" + "errors" "fmt" "sync" "sync/atomic" @@ -30,6 +31,10 @@ import ( "google.golang.org/protobuf/types/known/timestamppb" ) +var ( + ErrCancel = errors.New("pipeline canceled") +) + func (s *Server) nextId() string { v := atomic.AddUint32(&s.index, 1) return fmt.Sprintf("job-%03d", v) @@ -215,6 +220,31 @@ func (s *Server) Run(ctx context.Context, req *jobpb.RunJobRequest) (*jobpb.RunJ }, nil } +// Cancel a Job requested by the CancelJobRequest for jobs not in an already terminal state. +// Otherwise, returns nil if Job does not exist or the Job's existing state as part of the CancelJobResponse. +func (s *Server) Cancel(_ context.Context, req *jobpb.CancelJobRequest) (*jobpb.CancelJobResponse, error) { + s.mu.Lock() + job, ok := s.jobs[req.GetJobId()] + s.mu.Unlock() + if !ok { + return nil, nil + } + state := job.state.Load().(jobpb.JobState_Enum) + switch state { + case jobpb.JobState_CANCELLED, jobpb.JobState_DONE, jobpb.JobState_DRAINED, jobpb.JobState_UPDATED, jobpb.JobState_FAILED: + // Already at terminal state. + return &jobpb.CancelJobResponse{ + State: state, + }, nil + } + job.SendMsg("canceling " + job.String()) + job.Canceling() + job.CancelFn(ErrCancel) + return &jobpb.CancelJobResponse{ + State: jobpb.JobState_CANCELLING, + }, nil +} + // GetMessageStream subscribes to a stream of state changes and messages from the job. If throughput // is high, this may cause losses of messages. func (s *Server) GetMessageStream(req *jobpb.JobMessagesRequest, stream jobpb.JobService_GetMessageStreamServer) error { diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management_test.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management_test.go index 5813e6ef73e6..176abb8543a3 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/management_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/management_test.go @@ -169,6 +169,40 @@ func TestServer(t *testing.T) { } }, }, + { + name: "Canceling", + noJobsCheck: func(ctx context.Context, t *testing.T, undertest *Server) { + resp, err := undertest.Cancel(ctx, &jobpb.CancelJobRequest{JobId: "job-001"}) + if resp != nil { + t.Errorf("Canceling(\"job-001\") = %s, want nil", resp) + } + if err != nil { + t.Errorf("Canceling(\"job-001\") = %v, want nil", err) + } + }, + postPrepCheck: func(ctx context.Context, t *testing.T, undertest *Server) { + resp, err := undertest.Cancel(ctx, &jobpb.CancelJobRequest{JobId: "job-001"}) + if err != nil { + t.Errorf("Canceling(\"job-001\") = %v, want nil", err) + } + if diff := cmp.Diff(&jobpb.CancelJobResponse{ + State: jobpb.JobState_CANCELLING, + }, resp, cmpOpts...); diff != "" { + t.Errorf("Canceling(\"job-001\") (-want, +got):\n%v", diff) + } + }, + postRunCheck: func(ctx context.Context, t *testing.T, undertest *Server, jobID string) { + resp, err := undertest.Cancel(ctx, &jobpb.CancelJobRequest{JobId: jobID}) + if err != nil { + t.Errorf("Canceling(\"%s\") = %v, want nil", jobID, err) + } + if diff := cmp.Diff(&jobpb.CancelJobResponse{ + State: jobpb.JobState_DONE, + }, resp, cmpOpts...); diff != "" { + t.Errorf("Canceling(\"%s\") (-want, +got):\n%v", jobID, diff) + } + }, + }, } for _, test := range tests { var called sync.WaitGroup diff --git a/sdks/go/pkg/beam/runners/prism/internal/jobservices/server_test.go b/sdks/go/pkg/beam/runners/prism/internal/jobservices/server_test.go index 2223f030ce1d..473c84f958e3 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/jobservices/server_test.go +++ b/sdks/go/pkg/beam/runners/prism/internal/jobservices/server_test.go @@ -17,6 +17,7 @@ package jobservices import ( "context" + "errors" "sync" "testing" @@ -77,3 +78,63 @@ func TestServer_JobLifecycle(t *testing.T) { t.Log("success!") // Nothing to cleanup because we didn't start the server. } + +// Validates that invoking Cancel cancels a running job. +func TestServer_RunThenCancel(t *testing.T) { + var called sync.WaitGroup + called.Add(1) + undertest := NewServer(0, func(j *Job) { + if errors.Is(context.Cause(j.RootCtx), ErrCancel) { + j.state.Store(jobpb.JobState_CANCELLED) + called.Done() + } + }) + ctx := context.Background() + + wantPipeline := &pipepb.Pipeline{ + Requirements: []string{urns.RequirementSplittableDoFn}, + } + wantName := "testJob" + + resp, err := undertest.Prepare(ctx, &jobpb.PrepareJobRequest{ + Pipeline: wantPipeline, + JobName: wantName, + }) + if err != nil { + t.Fatalf("server.Prepare() = %v, want nil", err) + } + + if got := resp.GetPreparationId(); got == "" { + t.Fatalf("server.Prepare() = returned empty preparation ID, want non-empty: %v", prototext.Format(resp)) + } + + runResp, err := undertest.Run(ctx, &jobpb.RunJobRequest{ + PreparationId: resp.GetPreparationId(), + }) + if err != nil { + t.Fatalf("server.Run() = %v, want nil", err) + } + if got := runResp.GetJobId(); got == "" { + t.Fatalf("server.Run() = returned empty preparation ID, want non-empty") + } + + cancelResp, err := undertest.Cancel(ctx, &jobpb.CancelJobRequest{ + JobId: runResp.GetJobId(), + }) + if err != nil { + t.Fatalf("server.Canceling() = %v, want nil", err) + } + if cancelResp.State != jobpb.JobState_CANCELLING { + t.Fatalf("server.Canceling() = %v, want %v", cancelResp.State, jobpb.JobState_CANCELLING) + } + + called.Wait() + + stateResp, err := undertest.GetState(ctx, &jobpb.GetJobStateRequest{JobId: runResp.GetJobId()}) + if err != nil { + t.Fatalf("server.GetState() = %v, want nil", err) + } + if stateResp.State != jobpb.JobState_CANCELLED { + t.Fatalf("server.GetState() = %v, want %v", stateResp.State, jobpb.JobState_CANCELLED) + } +} From 7f91baadcf2b945c13f8f00bdbfb8e444e449c33 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Mon, 5 Feb 2024 13:30:30 -0500 Subject: [PATCH 34/50] Upgrade website node to LTS (#30213) * Upgrade website node to LTS * Remove npm installation * Latest npm --- website/Dockerfile | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/website/Dockerfile b/website/Dockerfile index 90d4202728d2..e40724ea1811 100644 --- a/website/Dockerfile +++ b/website/Dockerfile @@ -44,15 +44,16 @@ RUN apt-get update \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* -# Install node environment -RUN curl -sL https://deb.nodesource.com/setup_14.x | bash - \ +# Install node LTS environment +RUN curl -sL https://deb.nodesource.com/setup_lts.x | bash - \ && apt-get update \ && apt-get install -y --no-install-recommends \ nodejs \ - npm \ && apt-get autoremove -yqq --purge \ && apt-get clean \ && rm -rf /var/lib/apt/lists/* + +RUN npm update -g npm RUN npm install postcss postcss-cli autoprefixer From d5aa44c9ba9eb910774d789dd4182a5d25d8f552 Mon Sep 17 00:00:00 2001 From: tvalentyn Date: Mon, 5 Feb 2024 13:58:26 -0800 Subject: [PATCH 35/50] Handle a user whose account was deleted. (#30192) * Handle a user whose account was deleted. * Regroup items on the dashboard. --- .test-infra/metrics/sync/github/sync_workflows.py | 12 ++++++------ scripts/ci/pr-bot/updateReviewers.ts | 1 + 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/.test-infra/metrics/sync/github/sync_workflows.py b/.test-infra/metrics/sync/github/sync_workflows.py index a2c062b175ea..d2400dab4080 100644 --- a/.test-infra/metrics/sync/github/sync_workflows.py +++ b/.test-infra/metrics/sync/github/sync_workflows.py @@ -328,7 +328,6 @@ 'Beam Metrics Report', 'Build and Version Runner Docker Image', 'PreCommit GHA', - 'pr-bot-prs-needing-attention', 'PreCommit RAT', 'Assign or close an issue', 'PostCommit Website Test', @@ -339,8 +338,10 @@ 'PreCommit Whitespace', 'Publish Beam SDK Snapshots', 'Cancel Stale Dataflow Jobs', + 'pr-bot-new-prs', 'pr-bot-pr-updates', - 'pr-bot-new-prs' + 'pr-bot-prs-needing-attention', + 'pr-bot-update-reviewers' ] MISC_TESTS = [ @@ -355,7 +356,6 @@ 'Cancel', 'PostCommit PortableJar Spark', 'PreCommit Integration and Load Test Framework', - 'pr-bot-update-reviewers', 'PostCommit TransformService Direct', 'Cut Release Branch', 'Generate issue report', @@ -404,7 +404,7 @@ def get_dashboard_category(workflow_name): return 'go' if workflow_name in MISC_TESTS: return 'misc' - + print(f'No category found for workflow: {workflow_name}') print('Falling back to rules based assignment') @@ -471,7 +471,7 @@ def get_token(): git_integration = GithubIntegration(GH_APP_ID, GH_PEM_KEY) token = git_integration.get_access_token(GH_APP_INSTALLATION_ID).token return f'Bearer {token}' - + @backoff.on_exception(backoff.constant, aiohttp.ClientResponseError, max_tries=5) async def fetch(url, semaphore, params=None, headers=None, request_id=None): async with semaphore: @@ -574,7 +574,7 @@ def append_workflow_runs(workflow, runs): else: workflow_ids_to_fetch_extra_runs.pop(workflow_id, None) print(f"Successfully fetched details for: {workflow.filename}") - + page = math.ceil( int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH) / number_of_entries_per_page ) + 1 diff --git a/scripts/ci/pr-bot/updateReviewers.ts b/scripts/ci/pr-bot/updateReviewers.ts index c49f8feba24f..96eaa6e22f63 100644 --- a/scripts/ci/pr-bot/updateReviewers.ts +++ b/scripts/ci/pr-bot/updateReviewers.ts @@ -104,6 +104,7 @@ async function getReviewersForPull(pull: any): Promise { for (const comment of comments) { if ( + comment.user && comment.user.login && comment.user.login !== pull.user.login && comment.user.login !== BOT_NAME From a341eb69abe38f9bef66cf9aaa6b52bc6de2e331 Mon Sep 17 00:00:00 2001 From: hardshah <72236623+hardshah@users.noreply.github.com> Date: Sun, 4 Feb 2024 17:59:03 -0700 Subject: [PATCH 36/50] Added default watermark generation interval (apache#30141) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Suggestion from PR Co-authored-by: Jan Lukavský styling --- .../unbounded/FlinkUnboundedSourceReader.java | 36 +++++++++++-------- 1 file changed, 22 insertions(+), 14 deletions(-) diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java index 99160a9689eb..39ef63c8f7e9 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java @@ -69,6 +69,7 @@ public class FlinkUnboundedSourceReader // This name is defined in FLIP-33. @VisibleForTesting protected static final String PENDING_BYTES_METRIC_NAME = "pendingBytes"; private static final long SLEEP_ON_IDLE_MS = 50L; + private static final long MIN_WATERMARK_EMIT_INTERVAL_MS = 10L; private final AtomicReference> dataAvailableFutureRef; private final List readers; private int currentReaderIndex; @@ -103,22 +104,29 @@ public void start() { createPendingBytesGauge(context); Long watermarkInterval = pipelineOptions.as(FlinkPipelineOptions.class).getAutoWatermarkInterval(); - if (watermarkInterval != null) { - scheduleTaskAtFixedRate( - () -> { - // Set the watermark emission flag first. - shouldEmitWatermark = true; - // Wake up the main thread if necessary. - CompletableFuture f = dataAvailableFutureRef.get(); - if (f != DUMMY_FUTURE) { - f.complete(null); - } - }, - watermarkInterval, + if (watermarkInterval == null) { + watermarkInterval = + (pipelineOptions.as(FlinkPipelineOptions.class).getMaxBundleTimeMills()) / 5L; + watermarkInterval = + (watermarkInterval > MIN_WATERMARK_EMIT_INTERVAL_MS) + ? watermarkInterval + : MIN_WATERMARK_EMIT_INTERVAL_MS; + LOG.warn( + "AutoWatermarkInterval is not set, watermarks will be emitted at a default interval of {} ms", watermarkInterval); - } else { - LOG.warn("AutoWatermarkInterval is not set, watermarks won't be emitted."); } + scheduleTaskAtFixedRate( + () -> { + // Set the watermark emission flag first. + shouldEmitWatermark = true; + // Wake up the main thread if necessary. + CompletableFuture f = dataAvailableFutureRef.get(); + if (f != DUMMY_FUTURE) { + f.complete(null); + } + }, + watermarkInterval, + watermarkInterval); } @Override From b0f2eebb0244302ac2315dc260536512d229401f Mon Sep 17 00:00:00 2001 From: Andrew Crites Date: Tue, 6 Feb 2024 02:09:58 -0800 Subject: [PATCH 37/50] =?UTF-8?q?When=20failing=20work=20items=20during=20?= =?UTF-8?q?commit,=20make=20sure=20to=20call=20completeWork=E2=80=A6=20(#3?= =?UTF-8?q?0215)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * When failing work items during commit, make sure to call completeWorkAndScheduleNextWorkForKey to ensure future work can be processed. * Also invalidates reader cache and state cache when failing a work item during commit. --- .../dataflow/worker/StreamingDataflowWorker.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 3ba27bd852fc..b48032677ff1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -1397,12 +1397,20 @@ private void commitLoop() { // Adds the commit to the commitStream if it fits, returning true iff it is consumed. private boolean addCommitToStream(Commit commit, CommitWorkStream commitStream) { Preconditions.checkNotNull(commit); + final ComputationState state = commit.computationState(); + final Windmill.WorkItemCommitRequest request = commit.request(); // Drop commits for failed work. Such commits will be dropped by Windmill anyway. if (commit.work().isFailed()) { + readerCache.invalidateReader( + WindmillComputationKey.create( + state.getComputationId(), request.getKey(), request.getShardingKey())); + stateCache + .forComputation(state.getComputationId()) + .invalidate(request.getKey(), request.getShardingKey()); + state.completeWorkAndScheduleNextWorkForKey( + ShardedKey.create(request.getKey(), request.getShardingKey()), request.getWorkToken()); return true; } - final ComputationState state = commit.computationState(); - final Windmill.WorkItemCommitRequest request = commit.request(); final int size = commit.getSize(); commit.work().setState(Work.State.COMMITTING); activeCommitBytes.addAndGet(size); From 23dcb7ec1d539759f5c587a6dfee357ad250db72 Mon Sep 17 00:00:00 2001 From: scwhittle Date: Tue, 6 Feb 2024 11:23:57 +0100 Subject: [PATCH 38/50] =?UTF-8?q?Revert=20"When=20failing=20work=20items?= =?UTF-8?q?=20during=20commit,=20make=20sure=20to=20call=20completeWork?= =?UTF-8?q?=E2=80=A6"=20(#30228)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This reverts commit b0f2eebb0244302ac2315dc260536512d229401f. --- .../dataflow/worker/StreamingDataflowWorker.java | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index b48032677ff1..3ba27bd852fc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -1397,20 +1397,12 @@ private void commitLoop() { // Adds the commit to the commitStream if it fits, returning true iff it is consumed. private boolean addCommitToStream(Commit commit, CommitWorkStream commitStream) { Preconditions.checkNotNull(commit); - final ComputationState state = commit.computationState(); - final Windmill.WorkItemCommitRequest request = commit.request(); // Drop commits for failed work. Such commits will be dropped by Windmill anyway. if (commit.work().isFailed()) { - readerCache.invalidateReader( - WindmillComputationKey.create( - state.getComputationId(), request.getKey(), request.getShardingKey())); - stateCache - .forComputation(state.getComputationId()) - .invalidate(request.getKey(), request.getShardingKey()); - state.completeWorkAndScheduleNextWorkForKey( - ShardedKey.create(request.getKey(), request.getShardingKey()), request.getWorkToken()); return true; } + final ComputationState state = commit.computationState(); + final Windmill.WorkItemCommitRequest request = commit.request(); final int size = commit.getSize(); commit.work().setState(Work.State.COMMITTING); activeCommitBytes.addAndGet(size); From 5494f1143827e0e6fec9e331b93c00c83d10c66e Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 6 Feb 2024 03:34:48 -0800 Subject: [PATCH 39/50] [Dataflow Streaming] Invalidate caches and remove work on failure before commit (#30229) * Invalidate caches and remove work on failure before commit * Prevent completeWorkAndScheduleNextWorkForKey from throwing --------- Co-authored-by: Arun Pandian --- .../worker/StreamingDataflowWorker.java | 15 +++++++++++---- .../worker/streaming/ActiveWorkState.java | 17 +++++++---------- 2 files changed, 18 insertions(+), 14 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 3ba27bd852fc..14efdcc5eb02 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -1397,12 +1397,21 @@ private void commitLoop() { // Adds the commit to the commitStream if it fits, returning true iff it is consumed. private boolean addCommitToStream(Commit commit, CommitWorkStream commitStream) { Preconditions.checkNotNull(commit); + final ComputationState state = commit.computationState(); + final Windmill.WorkItemCommitRequest request = commit.request(); // Drop commits for failed work. Such commits will be dropped by Windmill anyway. if (commit.work().isFailed()) { + readerCache.invalidateReader( + WindmillComputationKey.create( + state.getComputationId(), request.getKey(), request.getShardingKey())); + stateCache + .forComputation(state.getComputationId()) + .invalidate(request.getKey(), request.getShardingKey()); + state.completeWorkAndScheduleNextWorkForKey( + ShardedKey.create(request.getKey(), request.getShardingKey()), request.getWorkToken()); return true; } - final ComputationState state = commit.computationState(); - final Windmill.WorkItemCommitRequest request = commit.request(); + final int size = commit.getSize(); commit.work().setState(Work.State.COMMITTING); activeCommitBytes.addAndGet(size); @@ -1419,8 +1428,6 @@ private boolean addCommitToStream(Commit commit, CommitWorkStream commitStream) .invalidate(request.getKey(), request.getShardingKey()); } activeCommitBytes.addAndGet(-size); - // This may throw an exception if the commit was not active, which is possible if it - // was deemed stuck. state.completeWorkAndScheduleNextWorkForKey( ShardedKey.create(request.getKey(), request.getShardingKey()), request.getWorkToken()); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java index 54942dfeee1f..ff46356d9569 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkState.java @@ -188,16 +188,13 @@ synchronized Optional completeWorkAndGetNextWorkForKey( private synchronized void removeCompletedWorkFromQueue( Queue workQueue, ShardedKey shardedKey, long workToken) { - // avoid Preconditions.checkState here to prevent eagerly evaluating the - // format string parameters for the error message. - Work completedWork = - Optional.ofNullable(workQueue.peek()) - .orElseThrow( - () -> - new IllegalStateException( - String.format( - "Active key %s without work, expected token %d", - shardedKey, workToken))); + Work completedWork = workQueue.peek(); + if (completedWork == null) { + // Work may have been completed due to clearing of stuck commits. + LOG.warn( + String.format("Active key %s without work, expected token %d", shardedKey, workToken)); + return; + } if (completedWork.getWorkItem().getWorkToken() != workToken) { // Work may have been completed due to clearing of stuck commits. From 27f1c0774fd93e846de9a8b668e6effc5a41eb10 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 10:17:08 -0500 Subject: [PATCH 40/50] Bump github.com/tetratelabs/wazero from 1.5.0 to 1.6.0 in /sdks (#30222) Bumps [github.com/tetratelabs/wazero](https://github.com/tetratelabs/wazero) from 1.5.0 to 1.6.0. - [Release notes](https://github.com/tetratelabs/wazero/releases) - [Commits](https://github.com/tetratelabs/wazero/compare/v1.5.0...v1.6.0) --- updated-dependencies: - dependency-name: github.com/tetratelabs/wazero dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 4327d5297a02..5f84993d3ff2 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -50,7 +50,7 @@ require ( github.com/proullon/ramsql v0.1.3 github.com/spf13/cobra v1.8.0 github.com/testcontainers/testcontainers-go v0.26.0 - github.com/tetratelabs/wazero v1.5.0 + github.com/tetratelabs/wazero v1.6.0 github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.13.1 diff --git a/sdks/go.sum b/sdks/go.sum index 3c90d6043312..8bf9aad16f72 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -433,8 +433,8 @@ github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcU github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/testcontainers/testcontainers-go v0.26.0 h1:uqcYdoOHBy1ca7gKODfBd9uTHVK3a7UL848z09MVZ0c= github.com/testcontainers/testcontainers-go v0.26.0/go.mod h1:ICriE9bLX5CLxL9OFQ2N+2N+f+803LNJ1utJb1+Inx0= -github.com/tetratelabs/wazero v1.5.0 h1:Yz3fZHivfDiZFUXnWMPUoiW7s8tC1sjdBtlJn08qYa0= -github.com/tetratelabs/wazero v1.5.0/go.mod h1:0U0G41+ochRKoPKCJlh0jMg1CHkyfK8kDqiirMmKY8A= +github.com/tetratelabs/wazero v1.6.0 h1:z0H1iikCdP8t+q341xqepY4EWvHEw8Es7tlqiVzlP3g= +github.com/tetratelabs/wazero v1.6.0/go.mod h1:0U0G41+ochRKoPKCJlh0jMg1CHkyfK8kDqiirMmKY8A= github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= From 78d08235c72aa21f29d1ff77771b45fd175ca545 Mon Sep 17 00:00:00 2001 From: Arun Pandian Date: Tue, 6 Feb 2024 08:38:53 -0800 Subject: [PATCH 41/50] Add grpc-services, grpc-utils to vendored grpc (#30196) Co-authored-by: Arun Pandian --- .../org/apache/beam/gradle/GrpcVendoring_1_60_1.groovy | 6 ++++-- vendor/grpc-1_60_1/build.gradle | 2 +- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_60_1.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_60_1.groovy index f12f3bac35c8..b2c7053dfb60 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_60_1.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/GrpcVendoring_1_60_1.groovy @@ -51,14 +51,16 @@ class GrpcVendoring_1_60_1 { "com.google.protobuf:protobuf-java:$protobuf_version", "com.google.protobuf:protobuf-java-util:$protobuf_version", "com.google.code.gson:gson:$gson_version", + "io.grpc:grpc-alts:$grpc_version", "io.grpc:grpc-auth:$grpc_version", - "io.grpc:grpc-core:$grpc_version", "io.grpc:grpc-context:$grpc_version", + "io.grpc:grpc-core:$grpc_version", "io.grpc:grpc-netty-shaded:$grpc_version", "io.grpc:grpc-protobuf:$grpc_version", + "io.grpc:grpc-services:$grpc_version", "io.grpc:grpc-stub:$grpc_version", - "io.grpc:grpc-alts:$grpc_version", "io.grpc:grpc-testing:$grpc_version", + "io.grpc:grpc-util:$grpc_version", "com.google.auth:google-auth-library-credentials:$google_auth_version", "com.google.api.grpc:proto-google-common-protos:$proto_google_common_protos_version", "io.opencensus:opencensus-api:$opencensus_version", diff --git a/vendor/grpc-1_60_1/build.gradle b/vendor/grpc-1_60_1/build.gradle index 1617484c1c52..834c496d9ca4 100644 --- a/vendor/grpc-1_60_1/build.gradle +++ b/vendor/grpc-1_60_1/build.gradle @@ -23,7 +23,7 @@ plugins { id 'org.apache.beam.vendor-java' } description = "Apache Beam :: Vendored Dependencies :: gRPC :: 1.60.1" group = "org.apache.beam" -version = "0.1" +version = "0.2" vendorJava( dependencies: GrpcVendoring_1_60_1.dependencies(), From 3412b4f7a5bba479349016d8ef8e8212e7704ba3 Mon Sep 17 00:00:00 2001 From: "Ukjae Jeong (Jay)" Date: Wed, 7 Feb 2024 03:24:33 +0900 Subject: [PATCH 42/50] fix markdown links (#30227) --- .../www/site/content/en/contribute/runner-guide.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/website/www/site/content/en/contribute/runner-guide.md b/website/www/site/content/en/contribute/runner-guide.md index cb3b52048606..c2349ccea32c 100644 --- a/website/www/site/content/en/contribute/runner-guide.md +++ b/website/www/site/content/en/contribute/runner-guide.md @@ -341,7 +341,7 @@ For values in the global window, you may want to use an even further compressed representation that doesn't bother including the window at all. We provide coders with these optimizations such as -(`PARAM_WINDOWED_VALUE`)[https://github.com/apache/beam/blob/release-2.49.0/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto#L968] +[`PARAM_WINDOWED_VALUE`](https://github.com/apache/beam/blob/release-2.49.0/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto#L968) that can be used to reduce the size of serialized data. In the future, this primitive may be retired as it can be implemented as a @@ -375,8 +375,8 @@ combiner lifting, where a new operation is placed before the `GroupByKey` that does partial (within-bundle) combining, which often requires a slight modification of what comes after the `GroupByKey` as well. An example of this transformation can be found in the -(Python)[https://github.com/apache/beam/blob/release-2.49.0/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L1193] -or (go)[https://github.com/apache/beam/blob/release-2.49.0/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go#L67] +[Python](https://github.com/apache/beam/blob/release-2.49.0/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L1193) +or [go](https://github.com/apache/beam/blob/release-2.49.0/sdks/go/pkg/beam/runners/prism/internal/handlecombine.go#L67) implementations of this optimization. The resulting pre- and post-`GroupByKey` operations are generally fused in with the `ParDo`s and executed as above. @@ -385,8 +385,8 @@ the `ParDo`s and executed as above. When you receive a pipeline from a user, you will need to translate it. An explanation of how Beam pipelines are represented can be found -(here)[https://docs.google.com/presentation/d/1atu-QC_mnK2SaeLhc0D78wZYgVOX1fN0H544QmBi3VA] -which compliment the (official proto declarations)[https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto]. +[here](https://docs.google.com/presentation/d/1atu-QC_mnK2SaeLhc0D78wZYgVOX1fN0H544QmBi3VA) +which compliment the [official proto declarations](https://github.com/apache/beam/blob/master/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto). ## Testing your runner @@ -757,7 +757,7 @@ and PipelineResult, but altered to be the minimal backend channel, versus a rich and convenient API. A key piece of this is the -(Artifacts API)[https://github.com/apache/beam/blob/master/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_artifact_api.proto], +[Artifacts API](https://github.com/apache/beam/blob/master/model/job-management/src/main/proto/org/apache/beam/model/job_management/v1/beam_artifact_api.proto), which allows a Runner to fetch and deploy binary artifacts (such as jars, pypi packages, etc.) that are listed as dependencies in the various environments, and may have various representations. This is invoked after a pipeline From c006c5e2d53f1544719059db6c1b06ec9bece49d Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 6 Feb 2024 14:05:45 -0500 Subject: [PATCH 43/50] Add Default method for OutputReceiver.outputWindowedValue (#30220) * Revert changes of override method that simply throw UnsupportedOperationException in subclasses --- .../core/construction/SplittableParDo.java | 10 ---------- .../org/apache/beam/sdk/transforms/DoFn.java | 7 +++++-- .../sql/zetasql/BeamZetaSqlCalcRel.java | 12 ----------- .../StorageApiWriteUnshardedRecords.java | 20 ------------------- .../beam/sdk/io/gcp/spanner/SpannerIO.java | 12 ----------- .../sdk/io/kafka/ReadFromKafkaDoFnTest.java | 11 ---------- .../sdk/io/pulsar/ReadFromPulsarDoFnTest.java | 13 ------------ ...adFromSparkReceiverWithOffsetDoFnTest.java | 13 ------------ 8 files changed, 5 insertions(+), 93 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java index 9cf0606b68b9..5ea2c4968dd9 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/SplittableParDo.java @@ -21,7 +21,6 @@ import com.google.auto.service.AutoService; import java.io.IOException; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.concurrent.ThreadLocalRandom; @@ -670,15 +669,6 @@ public void output(RestrictionT part) { public void outputWithTimestamp(RestrictionT part, Instant timestamp) { throw new UnsupportedOperationException(); } - - @Override - public void outputWindowedValue( - RestrictionT output, - Instant timestamp, - Collection windows, - PaneInfo paneInfo) { - throw new UnsupportedOperationException(); - } }; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java index c22b726c99a2..78ec1a8c840d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/DoFn.java @@ -395,11 +395,14 @@ public interface OutputReceiver { void outputWithTimestamp(T output, Instant timestamp); - void outputWindowedValue( + default void outputWindowedValue( T output, Instant timestamp, Collection windows, - PaneInfo paneInfo); + PaneInfo paneInfo) { + throw new UnsupportedOperationException( + String.format("Not implemented: %s.outputWindowedValue", this.getClass().getName())); + } } /** Receives tagged output for a multi-output function. */ diff --git a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java index ad856c1c3a7d..d60ebe46b370 100644 --- a/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java +++ b/sdks/java/extensions/sql/zetasql/src/main/java/org/apache/beam/sdk/extensions/sql/zetasql/BeamZetaSqlCalcRel.java @@ -26,7 +26,6 @@ import com.google.zetasql.Value; import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.util.ArrayDeque; -import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -49,7 +48,6 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionTuple; @@ -363,16 +361,6 @@ public void output(Row output) { public void outputWithTimestamp(Row output, Instant timestamp) { c.output(tag, output, timestamp, w); } - - @Override - public void outputWindowedValue( - Row output, - Instant timestamp, - Collection windows, - PaneInfo paneInfo) { - throw new UnsupportedOperationException( - "outputWindowedValue not supported in finish bundle here"); - } } private static RuntimeException extractException(Throwable e) { diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java index 3c6c73dd0219..846e7e3bddcb 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigquery/StorageApiWriteUnshardedRecords.java @@ -37,7 +37,6 @@ import java.io.IOException; import java.time.Instant; import java.util.Arrays; -import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Objects; @@ -70,7 +69,6 @@ import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.Preconditions; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -1095,15 +1093,6 @@ public void outputWithTimestamp( BigQueryStorageApiInsertError output, org.joda.time.Instant timestamp) { context.output(failedRowsTag, output, timestamp, GlobalWindow.INSTANCE); } - - @Override - public void outputWindowedValue( - BigQueryStorageApiInsertError output, - org.joda.time.Instant timestamp, - Collection windows, - PaneInfo paneInfo) { - throw new UnsupportedOperationException("outputWindowedValue not supported"); - } }; @Nullable OutputReceiver successfulRowsReceiver = null; if (successfulRowsTag != null) { @@ -1118,15 +1107,6 @@ public void output(TableRow output) { public void outputWithTimestamp(TableRow output, org.joda.time.Instant timestamp) { context.output(successfulRowsTag, output, timestamp, GlobalWindow.INSTANCE); } - - @Override - public void outputWindowedValue( - TableRow output, - org.joda.time.Instant timestamp, - Collection windows, - PaneInfo paneInfo) { - throw new UnsupportedOperationException("outputWindowedValue not supported"); - } }; } diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java index 3128de45fde3..6db79ab69b47 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/SpannerIO.java @@ -56,7 +56,6 @@ import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Comparator; import java.util.HashMap; import java.util.List; @@ -103,11 +102,9 @@ import org.apache.beam.sdk.transforms.Wait; import org.apache.beam.sdk.transforms.WithTimestamps; import org.apache.beam.sdk.transforms.display.DisplayData; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.sdk.util.FluentBackoff; @@ -2005,15 +2002,6 @@ public void output(Iterable output) { public void outputWithTimestamp(Iterable output, Instant timestamp) { c.output(output, timestamp, GlobalWindow.INSTANCE); } - - @Override - public void outputWindowedValue( - Iterable output, - Instant timestamp, - Collection windows, - PaneInfo paneInfo) { - throw new UnsupportedOperationException("outputWindowedValue not supported"); - } } } diff --git a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java index 845d974af0b4..48b5b060a295 100644 --- a/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java +++ b/sdks/java/io/kafka/src/test/java/org/apache/beam/sdk/io/kafka/ReadFromKafkaDoFnTest.java @@ -50,8 +50,6 @@ import org.apache.beam.sdk.transforms.errorhandling.BadRecord; import org.apache.beam.sdk.transforms.errorhandling.ErrorHandler.DefaultErrorHandler; import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; @@ -342,15 +340,6 @@ public void outputWithTimestamp( records.add(output); } - @Override - public void outputWindowedValue( - T output, - Instant timestamp, - Collection windows, - PaneInfo paneInfo) { - throw new UnsupportedOperationException("Not expecting outputWindowedValue"); - } - public List getOutputs() { return this.records; } diff --git a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFnTest.java b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFnTest.java index b72fe423efb1..273a1915d2bb 100644 --- a/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFnTest.java +++ b/sdks/java/io/pulsar/src/test/java/org/apache/beam/sdk/io/pulsar/ReadFromPulsarDoFnTest.java @@ -21,14 +21,11 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; -import java.util.Collection; import java.util.List; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.pulsar.client.api.MessageId; import org.apache.pulsar.client.api.PulsarClient; import org.apache.pulsar.client.internal.DefaultImplementation; @@ -176,16 +173,6 @@ public void outputWithTimestamp( records.add(output); } - @Override - public void outputWindowedValue( - PulsarMessage output, - Instant timestamp, - Collection windows, - PaneInfo paneInfo) { - throw new UnsupportedOperationException( - "unsupported outputWindowedValue in mock outputreceiver"); - } - public List getOutputs() { return records; } diff --git a/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFnTest.java b/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFnTest.java index bb0e6524241d..33827164c6b7 100644 --- a/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFnTest.java +++ b/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFnTest.java @@ -22,15 +22,12 @@ import static org.junit.Assert.assertTrue; import java.util.ArrayList; -import java.util.Collection; import java.util.List; import org.apache.beam.sdk.io.range.OffsetRange; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.splittabledofn.ManualWatermarkEstimator; import org.apache.beam.sdk.transforms.splittabledofn.OffsetRangeTracker; import org.apache.beam.sdk.transforms.splittabledofn.SplitResult; -import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.checkerframework.checker.initialization.qual.Initialized; import org.checkerframework.checker.nullness.qual.NonNull; import org.checkerframework.checker.nullness.qual.UnknownKeyFor; @@ -67,16 +64,6 @@ public void outputWithTimestamp( records.add(output); } - @Override - public void outputWindowedValue( - String output, - Instant timestamp, - Collection windows, - PaneInfo paneInfo) { - throw new UnsupportedOperationException( - "Not expecting to receive call to outputWindowedValue"); - } - public List getOutputs() { return this.records; } From 08a717dc5c297844c5ebef5712bb491ca66a8ede Mon Sep 17 00:00:00 2001 From: Talat UYARER Date: Tue, 6 Feb 2024 11:09:12 -0800 Subject: [PATCH 44/50] Creating a Fully Managed Beam Streaming System with Flink Runner on Kubernetes - Part 3 (#29860) * Last part of blog post Autoscaler details * Fixing whitespaces * Apply suggestions from code review Co-authored-by: Danny McCormick * Fixing math equations * Linked added for previous blog post. * Update website/www/site/content/en/blog/apache-beam-flink-and-kubernetes-part3.md Co-authored-by: Danny McCormick * trying to enable math rendering * Changed formulas with pictures * updated publish time and added stateful streaming section and conclusion * Removed whitespace * Image size reduction * updated the date * Update website/www/site/content/en/blog/apache-beam-flink-and-kubernetes-part3.md Co-authored-by: Danny McCormick --------- Co-authored-by: Talat UYARER Co-authored-by: Danny McCormick Co-authored-by: tuyarer --- .../apache-beam-flink-and-kubernetes-part3.md | 209 ++++++++++++++++++ .../adaptive_scheduler_rescale.png | Bin 0 -> 859268 bytes .../auto-tuned-worker.png | Bin 0 -> 114222 bytes .../backlog_growth.png | Bin 0 -> 6568 bytes .../backlog_time.png | Bin 0 -> 6127 bytes .../cpurate_desired.png | Bin 0 -> 10111 bytes .../flink-operator-chaining.png | Bin 0 -> 529398 bytes .../operator-backlog.png | Bin 0 -> 350363 bytes .../resource-allocation.png | Bin 0 -> 131782 bytes .../worker_extra.png | Bin 0 -> 11497 bytes .../worker_require.png | Bin 0 -> 11975 bytes .../worker_scaleup.png | Bin 0 -> 7471 bytes 12 files changed, 209 insertions(+) create mode 100644 website/www/site/content/en/blog/apache-beam-flink-and-kubernetes-part3.md create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/adaptive_scheduler_rescale.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/auto-tuned-worker.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/backlog_growth.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/backlog_time.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/cpurate_desired.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/flink-operator-chaining.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/operator-backlog.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/resource-allocation.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/worker_extra.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/worker_require.png create mode 100644 website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/worker_scaleup.png diff --git a/website/www/site/content/en/blog/apache-beam-flink-and-kubernetes-part3.md b/website/www/site/content/en/blog/apache-beam-flink-and-kubernetes-part3.md new file mode 100644 index 000000000000..f172cd78a472 --- /dev/null +++ b/website/www/site/content/en/blog/apache-beam-flink-and-kubernetes-part3.md @@ -0,0 +1,209 @@ +--- +title: "Behind the Scenes: Crafting an Autoscaler for Apache Beam in a High-Volume Streaming Environment" +date: 2024-02-05 09:00:00 -0400 +categories: + - blog +authors: + - talat +--- + + + +### Introduction to the Design of Our Autoscaler for Apache Beam Jobs + +Welcome to the third and final part of our blog series on building a scalable, self-managed streaming infrastructure with Beam and Flink. [In our previous post](https://beam.apache.org/blog/apache-beam-flink-and-kubernetes/), we delved into the scale of our streaming platforms, highlighting our capacity to manage over 40,000 streaming jobs and process upwards of 10 million events per second. This impressive scale sets the stage for the challenge we address today: the intricate task of resource allocation in a dynamic streaming environment. + +In this blog post [Talat Uyarer (Architect / Senior Principal Engineer)](https://www.linkedin.com/in/talatuyarer/) and [Rishabh Kedia (Principal Engineer)](https://www.linkedin.com/in/rishabhkedia/) describe more details about our Autoscaler. Imagine a scenario where your streaming system is inundated with fluctuating workloads. Our case presents a unique challenge, as our customers, equipped with firewalls distributed globally, generate logs at various times of the day. This results in workloads that not only vary by time but also escalate over time due to changes in settings or the addition of new cybersecurity solutions from PANW. Furthermore, updates to our codebase necessitate rolling out changes across all streaming jobs, leading to a temporary surge in demand as the system processes unprocessed data. + + + +Traditionally, managing this ebb and flow of demand involves a manual, often inefficient approach. One might over-provision resources to handle peak loads, inevitably leading to resource wastage during off-peak hours. Conversely, a more cost-conscious strategy might involve accepting delays during peak times, with the expectation of catching up later. However, both methods demand constant monitoring and manual adjustment - a far from ideal situation. + +In this modern era, where automated scaling of web front-ends is a given, we aspire to bring the same level of efficiency and automation to streaming infrastructure. Our goal is to develop a system that can dynamically track and adjust to the workload demands of our streaming operations. In this blog post, we will introduce you to our innovative solution - an autoscaler designed specifically for Apache Beam jobs. + + + +For clarity, when we refer to "resources" in this context, we mean the number of Flink Task Managers, or Kubernetes Pods, that process your streaming pipeline. These Task Managers aren't just about CPU; they also involve RAM, Network, Disk IO, and other computational resources. + +However, our solution is predicated on certain assumptions. Primarily, it's geared towards operations processing substantial data volumes. If your workload only requires a couple of Task Managers, this system might not be the best fit. In Our case we have 10K+ workload and each each of them has different workload. Manual tuning was not an option for us. We also assume that the data is evenly distributed, allowing for increased throughput with the addition of more Task Managers. This assumption is crucial for effective horizontal scaling. While there are real-world complexities that might challenge these assumptions, for the scope of this discussion, we will focus on scenarios where these conditions hold true. + +Join us as we delve into the design and functionality of our autoscaler, a solution tailored to bring efficiency, adaptability, and a touch of intelligence to the world of streaming infrastructure. + + +## Identifying the Right Signals for Autoscaling + +When we're overseeing a system like Apache Beam jobs on Flink, it's crucial to identify key signals that help us understand the relationship between our workload and resources. These signals are our guiding lights, showing us when we're lagging behind or wasting resources. By accurately identifying these signals, we can formulate effective scaling policies and implement changes in real-time. Imagine needing to expand from 100 to 200 TaskManagers — how do we smoothly make that transition? That's where these signals come into play. + +Remember, we're aiming for a universal solution applicable to any workload and pipeline. While specific problems might benefit from unique signals, our focus here is on creating a one-size-fits-all approach. + +In Flink, tasks form the basic execution unit and consist of one or more operators, such as map, filter, or reduce. Flink optimizes performance by chaining these operators into single tasks when possible, minimizing overheads like thread context switching and network I/O. Your pipeline, when optimized, turns into a directed acyclic graph of stages, each processing elements based on your code. Don't confuse stages with physical machines — they're separate concepts. In our job we measure backlog information by using Apache Beam's [`backlog_bytes` and `backlog_elements`](https://github.com/apache/beam/blob/master/sdks/java/core/src/main/java/org/apache/beam/sdk/metrics/SourceMetrics.java#L32) metrics. + + + +##### **Upscaling Signals** + +##### *Backlog Growth* +Let’s take a practical example. Consider a pipeline reading from Kafka, where different operators handle data parsing, formatting, and accumulation. The key metric here is throughput — how much data each operstor processes over time. But throughput alone isn't enough. We need to examine the queue size or backlog at each operator. A growing backlog indicates we're falling behind. We measure this as backlog growth — the first derivative of backlog size over time, highlighting our processing deficit. + + + +##### *Backlog Time* +This leads us to backlog time, a derived metric that compares backlog size with throughput. It’s a measure of how long it would take to clear the current backlog, assuming no new data arrives. This helps us identify if a backlog of a certain size is acceptable or problematic, based on our specific processing needs and thresholds. + + + + + +#### **Downscaling: When Less is More** + +##### *CPU Utilization* +A key signal for downscaling is CPU utilization. Low CPU utilization suggests we're using more resources than necessary. By monitoring this, we can scale down efficiently without compromising performance. + +#### **Signals Summary** + +In summary, the signals we've identified for effective autoscaling are: + +1. **Throughput:** The baseline of our performance. +2. **Backlog Growth:** Indicates if we’re keeping pace with incoming data. +3. **Backlog Time:** Helps understand the severity of backlog. +4. **CPU Utilization:** Guides us in resource optimization. + +These signals might seem straightforward, but their simplicity is key to a scalable, workload-agnostic autoscaling solution. + +## Simplifying Autoscaling Policies for Apache Beam Jobs on Flink + +In the world of Apache Beam jobs running on Flink, deciding when to scale up or down is a bit like being a chef in a busy kitchen. You need to keep an eye on several ingredients — your workload, virtual machines (VMs), and how they interact. It's about maintaining a perfect balance. Our main goals? Avoid falling behind in processing (no backlog growth), ensure that any existing backlog is manageable (short backlog time), and use our resources (like CPU) efficiently. + +#### **Up-scaling: Keeping Up and Catching Up** + +Imagine your system is like a team of chefs working together. Here's how we decide when to bring more chefs into the kitchen (a.k.a. upscaling): + +1. **Keeping Up:** First, we look at our current team size (number of VMs) and how much they're processing (throughput). We then adjust our team size based on the amount of incoming orders (input rate). It's about ensuring that our team is big enough to handle the current demand. + +2. **Catching Up:** Sometimes, we might have a backlog of orders. In that case, we decide how many extra chefs we need to clear this backlog within a desired time (like 60 seconds). This part of the policy helps us get back on track swiftly. + +#### **Scaling Example: A Practical Look** + +Let's paint a picture with an example. Initially, we have a steady flow of orders (input rate) matching our processing capacity (throughput), so there's no backlog. But suddenly, orders increase, and our team starts falling behind, creating a backlog. We respond by increasing our team size to match the new rate of orders. Though the backlog doesn't grow further, it still exists. Finally, we add a few more chefs to the team, which allows us to clear the backlog quickly and return to a new, balanced state. + + +#### **Downscaling: When to Reduce Resources** + +Downscaling is like knowing when some chefs can take a break after a rush hour. We consider this when: + +- Our backlog is low — we've caught up with the orders. +- The backlog isn't growing — we're keeping up with incoming orders. +- Our kitchen (CPU) isn't working too hard — we're using our resources efficiently. + +Downscaling is all about reducing resources without affecting the quality of service. It's about ensuring that we're not overstaffed when the rush hour is over. + +#### **Summary: A Recipe for Effective Scaling** + +In summary, our scaling policy is for scale up, we first ensure that the time to drain the backlog is beyond the threshold (120s) or the cpu is above the threshold (90%) + +Increasing Backlog aka Backlog Growth > 0 : + + + +Consistent Backlog aka Backlog Growth = 0: + + + +To Sum up: + + + +To scale down, we need to ensure the machine utilization is low (< 70%) and there is no backlog growth and current time to drain backlog is less than the limit (10s) + +So the only driving factor to calculate the required resources after a scale down is CPU + + + +## Executing Autoscaling Decision + +In our setup we use Reactive Mode which uses Adaptive Scheduler and Declarative Resources manager. We wanted to align resources with slots. As advised in most of the Flink documentation we set one per vCPU slot. Most of our jobs use 1 vCPU 4GB Memory combination for TaskManager. + +Reactive Mode, a unique feature of the Adaptive Scheduler, operates under the principle of one job per cluster, a rule enforced in Application Mode. In this mode, a job is configured to utilize all available resources within the cluster. Adding a TaskManager will increase the job's scale, while removing resources will decrease it. In this setup, Flink autonomously manages the job's parallelism, always maximizing it. + +During a rescaling event, Reactive Mode restarts the job using the most recent checkpoint. This eliminates the need for creating a savepoint, typically required for manual job rescaling. The volume of data reprocessed after rescaling is influenced by the checkpointing interval(10 seconds for us), and the time it takes to restore depends on the size of the state. + +The scheduler determines the parallelism of each operator within a job. This setting is not user-configurable and any attempts to set it, whether for individual operators or the entire job, will be overlooked. + + + +Parallelism can only be influenced by setting a maximum for pipelines, which the scheduler will honor. Our maxParallelism is limited by the total count of partitions that the pipeline will process, as well as by the job itself. We cap the maximum number of TaskManagers with maxWorker count and control the job's key count in shuffle by setting maxParallelism. Additionally, we set maxParallelism per pipeline to manage pipeline parallelism. The job cannot exceed the job's maxParallelism in terms of workers. + +After autoscaler analysis, we will tag if the job needs to be scaled up, no action or scaled down. To interact with the job, we use a library we have built over Flink Kubernetes Operator. This library allows us to interact with our flink jobs via a simple java method call. Library will convert our method call to a kubernetes command. + +In the kubernetes world, the call will look like this for a scale up: + +`kubectl scale flinkdeployment job-name --replicas=100` + +Apache Flink will handle the rest of the work needed to scale up. + +## Maintaining State for Stateful Streaming Application with Autoscaling + +Adapting Apache Flink's state recovery mechanisms for autoscaling involves leveraging its robust features like max parallelism, checkpointing, and the Adaptive Scheduler to ensure efficient and resilient stream processing, even as the system dynamically adjusts to varying loads. Here's how these components work together in an autoscaling context: + +1. **Max Parallelism** sets an upper limit on how much a job can scale out, ensuring that state can be redistributed across a larger or smaller number of nodes without exceeding predefined boundaries. This is crucial for autoscaling because it allows Flink to manage state effectively, even as the number of task slots changes to accommodate varying workloads. +2. **Checkpointing** is at the heart of Flink's fault tolerance mechanism, periodically saving the state of each job to a durable storage (in our case it is GCS bucket). In an autoscaling scenario, checkpointing enables Flink to recover to a consistent state after scaling operations. When the system scales out (adds more resources) or scales in (removes resources), Flink can restore the state from these checkpoints, ensuring data integrity and processing continuity without losing critical information. In scale down or up situations there could be a moment to reprocess data from last checkpoint. To reduce that amount we reduce the checkpointing interval to 10 seconds. +3. **Reactive Mode** is a special mode for Adaptive Scheduler, that assumes a single job per-cluster (enforced by the Application Mode). Reactive Mode configures a job so that it always uses all resources available in the cluster. Adding a TaskManager will scale up your job, removing resources will scale it down. Flink will manage the parallelism of the job, always setting it to the highest possible values. When a job undergoes resizing, Reactive Mode triggers a restart using the most recent successful checkpoint. + +## Conclusion + +In this blog series, we've taken a deep dive into the creation of an autoscaler for Apache Beam in a high-volume streaming environment, highlighting the journey from conceptualization to implementation. This endeavor not only tackled the complexities of dynamic resource allocation but also set a new standard for efficiency and adaptability in streaming infrastructure. By marrying intelligent scaling policies with the robust capabilities of Apache Beam and Flink, we've showcased a scalable solution that optimizes resource use and maintains performance under varying loads. This project stands as a testament to the power of teamwork, innovation, and a forward-thinking approach to streaming data processing. As we wrap up this series, we express our gratitude to all contributors and look forward to the continuous evolution of this technology, inviting the community to join us in further discussions and developments. + +# References + +[1] Streaming Auto-scaling in Google Cloud Dataflow [https://www.infoq.com/presentations/google-cloud-dataflow/](https://www.infoq.com/presentations/google-cloud-dataflow/) + +[2] Pipeline lifecycle [https://cloud.google.com/dataflow/docs/pipeline-lifecycle](https://cloud.google.com/dataflow/docs/pipeline-lifecycle) + +[3] Flink Elastic Scaling [https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/elastic_scaling/](https://nightlies.apache.org/flink/flink-docs-master/docs/deployment/elastic_scaling/) + +# Acknowledgements + +This is a large effort to build the new infrastructure and to migrate the large customer based applications from cloud provider managed streaming infrastructure to self-managed Flink based infrastructure at scale. Thanks the Palo Alto Networks CDL streaming team who helped to make this happen: Kishore Pola, Andrew Park, Hemant Kumar, Manan Mangal, Helen Jiang, Mandy Wang, Praveen Kumar Pasupuleti, JM Teo, Rishabh Kedia, Talat Uyarer, Naitik Dani, and David He. + +--- + +**Explore More:** + +- [Part 1: Introduction to Building and Managing Apache Beam Flink Services on Kubernetes](https://beam.apache.org/blog/apache-beam-flink-and-kubernetes/) +- [Part 2: Build a scalable, self-managed streaming infrastructure with Flink: Tackling Autoscaling Challenges - Part 2](https://beam.apache.org/blog/apache-beam-flink-and-kubernetes-part2/) + +*Join the conversation and share your experiences on our [Community](https://beam.apache.org/community/) or contribute to our ongoing projects on [GitHub](https://github.com/apache/beam). Your feedback is invaluable. If you have any comments or questions about this series, please feel free to reach out to us via [User Mailist](https://beam.apache.org/community/contact-us/)* + +*Stay connected with us for more updates and insights into Apache Beam, Flink, and Kubernetes.* \ No newline at end of file diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/adaptive_scheduler_rescale.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/adaptive_scheduler_rescale.png new file mode 100644 index 0000000000000000000000000000000000000000..4515a9a2be8fc50bfad7a7cba598f42832a1ceb1 GIT binary patch literal 859268 zcmeFZbyO74w?8_BfJjM5H!2bWk`fY8RgQ|+Wtfp)7;_%~S&7NK9)y~ClTf+k*9v(jOix)2XA(O(VnY7YV{xcHHU=wwo7I z?tZ)M80(xmBdK&Ykc@@xKYnFo>ivqf!JPijUtZ4FG5@zGisSKP`9ArNU$ZpzxS0QV zH?|B@37<7y9ITA|}}X`{lb6 zH~LMZUlkRRxw*Ny&S$@?{azMGMswMLOCD6{(@#8M2obdg&-&5!vG~N`FSyo1MQ~L0Iwc(pvuin1Ji&e%=yyrsevh-`y zSvUm^R+PrSBP6c8w6vsC?J9s7TWe)*J=pydo0Rln)Nx`>?008cw8!!X{Hn(*ijN(U+$eDpzXvDOY^f+s@O7 z+AzT9TPqtnHmV%9x&E9Vi&S#8KGx=o1|L)Ox63nQJ$DQbzic0IeXOeL{`D*D<45t?&Aq~DCWAt1zUozM?R>qU6p6=; z(+;mHD`_ast9^>Uc_sAyds9=Udtwa1ikY%(vGvQtRo%V4<(;NyRm9ErZph1TJj%TC zo%nxOxA=>GYmtM)6(1!3KFjPZrJ#J(xHhI{GKrC>?#?hFOj zo$-)j??d+?`&LGsvWGp1HddUJjgFmCA^R(J%r!MN>ql?jp3`@qdxP=){oB0xa))-*wTO#T0S8BF1a&u2f zNlAVF@X zzt7+h0JH(hqc2pJ_M{DSKBYm%Ek zZ{zp0>@Qc5J|1?a#>nJrPfo?(Y(cvUUhLgftes00ePxgHy&kW2p+UY5iBbCd_d=Rl zTC(qccs=r4gS}XHFwKYdgow@1lg=yq)}JRexu}Hbh|6t;!}j*}aB6D}3YD&2#lf)f z@sTDPdS7Pp-Z8#!ZcZXD!W6wCs4`*6qC#%qy=#`ITZ%VW{-_C-S6!#6qobq!pAON< z5s#JT^J$*`@I5lCs^~}eN48gIGozVW{tm*OX2O35p<&4d2NM<^zU`6RY!VxAoRpNb z)9G^*V8l<5a4l$PXo&ARZspwDmv&9WqDIqS67Yvx{HSoqLvb(G?wCZ-2|xU7T3C1F zUJV=bYV93u?cm#LmrqC^jFZ~`dhm|%yYJ3j+nS)Ws?X||Tug^5?JGCFtFE=s+m6jh zAMeJrQd3i#Zp?O99_=j@+l|XjO;0m!NG&WZ2$;4I<=w3yf#N7syFa|L{HNb*Q!9?+ z^_w?d<2B@=ffIsJt&^+nH8C6~FJHd=xwDhGB+j2h1^(rUJlp5bpJnXr*AlN)x*i{H zZx7V@`(q0U360I?7PbUY8$9{-OK-O(`}VtK!8Mcg`pEnq~(l_SySD9(=OhlEYHcg=6%o5FnaFCI~>e4RaN8Cfz|J- ze8wZyZu`VXRl_zshP4cQH5=q2kAC5*#BB}M<`#S;gjFn_>alzo{pa?mk911}ZNQ*y z-MuU4q@|fHQr+GjI9gbozjCm8RUbZwA=l36yN@6BmKR36Tvr?L1ss20fBNhhn}h^? z{J9(ct&_3QYx{>;`T5O^kB5po7wTX>EUm1rs;QAAtDVOfwMH>!Wo5}ID4gKp=GM^E z474ivVBYm1j>Gt1XN{Aa`|6Dw#9Ull{N`Vu%eL~uN3!b88?cq;=Hi;_gVm!`WaO7@ zS7g#mXgyd?EWO)yld|F1D^#AXNQY;=bEWzZNl9sGcYpsABmSMB8MH0OoBqKH83A2) z6pw%0lpbEOZ%^b)G~_d=rty=(#Bs>Rad42I)b}~u+*ynEsqXxvdhMFs-W&|%(Qc;? zlP0|u7Zn;M^kW4rzTslzGd1+xrW0VY8g7UV2il1dVbCs5?|C@xQuXS`+?0zFTP&+Z z7Ac1cIclF_VT8%fy}FgFji&_cC*&z9DL*HWZ*4W;Gmp)^OMB*XWp^rGz;nkAldbzl zS68>c+D*{&3bTRFp|hnB-L651f8W9h%)tb8U47{J)F-IDS64HbnVD5?_TDYF85Xb9 zzX%`6dZ_ZWv$OO12k5;QdCr_VS`CpNgUt{<@ewr>SQN!Z4cJc3&aYm-#yxT3#7-0Q zk#DhSYvbhF=Qip%j`6nhX`j-L4=i>MlsWwO_T1t7-dN5h!v<*ZT-fTF=g{8f-_4Zl zx!$%j7C?%(*}3GBSyDpIeWxI>!#rTXT9JjUzQnA3iX)0ok5-J55Vcw9n`u`V%gV~^ z#299mmYQiTlKte_umT!e({gO(jM+GN*>(5{8e{TgP#JB;H0flmnj%yi;o zo_Cl-MU@_yvBwLL_2yL}zxZK_d~xUl#bAYPYqa+UI|1wJcHJ>!v5s|GpBNNWEX+c0 zzQnA{+c$5Fzdjexsc{!Be>50uRakR|D}j(H8sF#WPa5>KYm%%Ip@KtwtRiliqDP<|Mub z)0$pW!-yAvRWahu4F&kUUVgG?*B$#;wjzgjdPE()H|KkFs$6(oRz`?Jny^&j?4}d$ zOc);TQ6s!UOfND9vv3|j6Kk3Tlq*UqD%Pay695I}Jv<~9T_n`R#&Pu>JA~~%=w}rc zwzl)tMWm)ORd4;W+JW^-!isx_R}Z(wfHb}>5Nd@if!|D;50lLNE&`R;&rx8;x|Nul zzoke_U&BfB+Db~MT!oqcjCJtr*)!S0{Xc2PL@zsnMMqq5rl+T~a&qJVqkRAVeM*2) zr^kphKR@5$ECEXyWgv&j&SGgNzxRPFN4&~z&GFH8-TG!mH>aA!k`RJ25-y9r-Ikwz zm03-{;YZMJq4htuBfr<4Q!+GcjBnlHk}b5Y?U>fh6W_7MHh$b}l|EV{jbOM;o~abw zl6JIlerjrpg*uzh7it&|sid!+SVYh%YOY^6GN+x6N;0}<(_aTK6xK22YUj(rQonsuDkz+Gva>|N^w4;y!;G|HLZiA;~F$U3fNL&P_f7e zgV5Z6nNfp1LK1>gSiMd-;=TU}6O&6XxxZMtUU>lRtf8#WAUu{`Y2x=wTR@l%09mH& z5oW^}txvTDJQKk90udm_@HmR;G@!t7)eUq2U*W(R%%M#jK@LSl56q}qV*p_SFz z?o3){Z7tK9D+9CVDwatxjhghKQ}yxDK6Jc4zdM>rJFkSJVHN{aTx{M!_p`5W(*M-O z{*{_gn8RG-f4PgSvZ`v6;n6zv&)#0PGvUU7|N1{VFQiZIU0cDauh|~q#y9lS%-*j} z!!O9Mt>~0KX29@U_ZI~`%xs^gEBWf7t4mu@wJe^eS4P;}+-xVFqEm9e4PO531_$|i z?YD1gLmn$NWODVBYh&K;SCswC|D~2-nxwNoMJwoImK!gE@jufx&zOjHTdPotH-h^kAW{ zFp&bYcO~Ahjl#>ob4_74iFW5;bB?}AnS*U5yx9cxh@uBS&bb-7xQJjrtDMx!*Y@V) z<~~I)VvW$K{Y2>bFDpI=YpsCssW4-2U)nUnYKua}MS6dr)2>JB&Uo)#+n@%c`uU#R zI=hqDh)%9Q$H@E-^ptms$;r)KTdrDR6BVVFBVc$uWbX>VnbNPrue2ZGsmnCU${edN zM-T9>q>(;&@St)wBRse|WWc%whztOhvv(N8$W)*4$Ub;*;rF+c@YK{)*60V19x0fa zoyFAKRFJyY@jYQAz@_xsndb8em0|G(C`H+JpZr~l3mt%DJQgZVG&nqo0XJ1uuL2uN zc|39fKOg_%W4mw%&i}-Oe5Q&EJhnhn@u62+);4o;EkkDXK|h z!;eRsp`Wb6dZdPL!36wOI4cDaC8u#(JvlQKBe5}_z^cNg!t(D{Osu!_F<)}a z?5wn(#BfUgfo?GV?5<60qB&-d0wZGG?`tWr*|g)v^?~|;SvsNWbX6q!~ZK;WMp=Kj%6BCD$<1DlLDwze`pexwsmA+fDH6oC-}=&641UL-J!Je^`}QmGBHFJHc_ zLbo@8;-H?b=?cqp+)m7Hd-(ArOp7RV1~l+o&nUYdmRZ`^95o(Mo;ib!aakS;h5AU< zV5Ck`(D(Bv!)hhQx5FF$r5H?D4`@_!6wASGa4_~T4{K45*|XXc-+ zghWILF=%imq9m4M_C-xT5wNPnk;$QsL-Ntw#ibm|X{cw)LP3?*pZjB%@Lh z2wZOe`(tYrmC&v^CpiLE@BP8iu#F9;-5O}BI?8bz0K!jT1bmJ>-M(%q#BtceXoSIP zQh2Et&BW`y7H;?icGtGoPEJmmpCLe{!vP8?W%WG7lCd#O17f0gW$PHk zrcV*Hg18vyq^w`fOFf6G-TvTW)_{pg>J+|IVcY#%K-38^Yz!pfTDjVx z)z#HrQEN~_Jk`^v(I%8kAk()FLBiMHU$ePD=xDBZidl=$Bi#tbQ*o`f?SA8m_qGI& zZV4`QBcx#k^BnrX0{pVjXklhH-b`)y)FO>JPoB+hCe?!kSVSvGJ*OiTVXy*V)9J&s zK9};?x}sWq3@|m;VgM(*WH})r0Z^^O2X_Ew7xkZPk9d&EJnYz=ZcikAq3eTGk()ky z6qf`Ai>>-DqN1Rfd756-<`Y!6)u2O!H1T=;kZ=Kl1oT;^pU(ldye}`8dvgnLp&ZBr z!SpuuKtwizdT`hO{5eg8_ewjryQk+(*3c?+`$^C%LZC?f?sypsWf5QyLpU`b9%jgE zdl=R53(<@*d!CX^qzHWm2jii3hl(wt$Bhe{Y)RKi9yS4D4hCdE5Bi|1-LP|?adQ|I zLI`X1IHc*oMlh>;3x$)wk{Y1@$0Q|PzAwE8llj5R3l##O#3n%Tp9`4c9e7LoH>B(D z`J}&5O`HI%{0X!jtk*e&)+5!lum@$#&Ci`vOMY_dVkHJplIwU7_11@}6KD?U{(}+V z0@SprS`zvB^XC|K-A;2%D*@|P)zOFc=|<+`9VHMI@qfI#DLr;+GeqxF(x;~vZlm6N zJz4Cu4=h&Eq zYoH_{B^5zO*m3NIw~x*C-rmY4i*5y})UAG@`zE|KY~n^#&p@y_Iu#w=Y$CSyi2^Bi znC$v}5br_&#+h`y5C)J+5F|$?huWUnI|`YHl2>1mA=T!Nf!hhbx`Rt!%>^;_mO@M$ zCHNj5b8iyuU>p$9hO*1-4a_YgJ>96wzU{t4UD}+?zDL1XmH&q6P%?u(gJF@+cKr+0 zP7|{Q%5ad&y0~m%;kVVPLG1#j3GhwAWcwxu`Bl93?Xw_lf#|^NeYjv~YG~M;=J4f$ z>(TCv#+^GAoeP0-tVko7dmU1vkg4(d-8*hBa?g99MZI|WQUExBCp^<|{}6s~?b5%1 zM1Di~(#sB=I&Udx-a$YtdwaFtmz5!?2uw@-d&j65PDi>p;*;mEIr}gr;J_|>{ zm$@5A^{+)S%l*Sr2z`YA*5AJD7uIb3V($lkAj6hg`(Ga<<0=>EsemTLfDf$w050@j z!*e&_{e6x1vnS3rn%v_|5;# z6Jiq=ifiKl?h118#9C$b?x8o*wLp^w|73Q4Uhd{iQt$vyhd0N>#$uJ}MMgvrLCI<_ zUH|dphlZYB7;pp4J9oH`7XzR8WhjsErWe1!UBP%FB7D zq@_h)xc%1X*ZT+0MIO;1t!i=68n75Elw*)3L-IP%iWtTHcU;83YFJnK zi_R9Xj=(>@=q!C!X(`1^QQH>Kd7?UAikbqbbkIGj<%KB(X3z`( z6;)^kV3w8-9>{8IKa+fEn6KSFSaI)ClDrUIf@tYLM446J1OUM%VE)qbY^U#ASg-@c z)zZ=eKqG=Yiyvwd#2~LIecVM~*YL@|MMO!4lk(Qrp_Tc0P=WpW7J^zQWkKdujHG+h z4;42Sh7f=+=$J`@J<|b-e_r11vJss*`mXQLABz@rDN(Kf@pmS=#G6H~X z+~9u#rtBAMl_*+8ul{w8GYUhS#_JI;a{FU$Jy3E2lx{_!ZBF749EpVR<@N(%Q71ta` zYip&!I*Cn5iOkE(*RlJ5!f^V<)9CNeq1rgyWL=3+SUfI!~+hoz7Q8j z4)YB@t`^he#KZ)ja8w*egSAML5r2ASCXQ8KL355)Zr9*oIN(ljJ&nK(6`Ou~o8Ng} z6SOkV?0vMey|n&alzVZ3Wc+lIptMy~KPb}!SFYZq7)c{dA*8LV<4xUQ=z>5dBq|C5 z(UEH66lstg|H}@*D(k(}L=}4ZMFZBLIf70E>i{_wDk>*ow5R(3EJH6xc0^cM*i=4W zmQPkLq31mgm0OzU;RPV;Ks5mU_rD}y1=yT`8vyVEvIqg)_d}S>?b{T;e*NNi->|e4 zdJ*ukZg8muJj!C{1#RnJ?^Oz>y1csuFP#)*lEvkD3n-QD;41Zk86V7C+$Tto#2_8rs;rpe`H8w5jv zkT#*s<}js<2Jy|C6I4`G_zd-=cmW6iA>$K?^mogxWnm1|uI%1N)bzQSJsa>hF~IiT z9W`eW1YrNSwO|1&gpo%x0h3o`jB{mgmK`m4T3TB4u|SwXW9PPh|9Ywazxe8ilQ<*n#cA3-m@Z38OPRZO>4S;s&)Gr^Go0k~ta=2(D6`Lj z0Q#6tE-sBG#jreBKokcZpLTC?ycnDDie5ZSbec?c2On29(`^&7TL0ePT z*N-UuCVt8y**;jGmW`Wx!D^WmVTpx*vBW4Bz-!RU;34vo7%>g-6{S2u#f6>E$-^TL z8qm2LF9HWjElj^9Uj_wwuW_Me(tgMD3XpktfPz2|1nH@0#b6|q74Y_0!Oldbir?73 zI+9ip4|J^%W87JzxlMPZY`Ow!MwJ-G!&Q&q{2}3ckka&3_R!V?4AVP!iw)tN9t9VN{=?4o~5Mz0z-;h28e%Jd^D?-*7We&3NX0k}dV*5$OYA|qILES0#+IIoAbpQT+ zaD9=W1F|F{zWkss1Q?L})(*f=8F8ColsrE8|SUgkTXgAjGM5cXwf$;9*!l zHz#(u@k0RCBx;mWQir|;s;wlffyvhE;~~rwA7)aUkU#D!8TIw+b;No5?mcD%HFXh( zaANyx5?Ty9U_?Z~`eFw@hH@6fqC>up-!6x=pMc8N9rN#sQc3}(`0rT4&=i@rlE6}d z*Z|Z|^p(+EB2fw24|H4TK#%LN+!EV^NTmP3h6NL;7yvpb?YmTW zuD3VSqCj1Qm!+qt`wEskNaIlPA6yC^wrPE7TN{cL0I0{5YDH4fP*ogGK`HuJc zD11#In|60~`L?tuBDVq76nqO#t@%1=hj%M%n!#r(c9^~fy%_~az>sbPZ#61jO?>W@ z#kOi(AZiAOdz(m4L0b=T(5=M*#5>3{~&2yHD579*el z8U@Nhd3i|z*$js;fe~2OG}7LZP)UQK4YL-S3egoi&Zzx0B_Tk8x)>}-*xBj%`NWSN zJpwU}20$RIEl<4Ks1xX}U~)HrnvcXPBUmucFFT%t59E*F7A$-D2M^}F3&%Izkz$ON zII;$YY((K}aU|Qf(4vWe>Q8dp7A9fV&h#I)=fw+}4wNQv!ivq>&-~p}uy`OC1C$*U z#t0B8n2>e#GyArNx;p=qGr08dfKkOD8ovWDi%}Sb6nZbJ*|3Zu;fApB&&8TY@eEbcO{?yD-pFAh|GUA-zvq3t7H)pOq~OFv5|_x(a@kTpUL- zly{`*^V?5+Vx@3L3;JcyKf@Y$|57MqW{g2eM1u+@5o&`ZA*juu*NQFVm8xrL1tljl z00btYRbHH(tp^*n*mKvxdbIXlts>OTZ<2qmqc8@_5>8pczGUzlA!pnNE{W}jLCvi0 zXtNNZEKpMIAp|lBnW9hcAN<(z@P>RJtN=)mM4nG|#fJ5RA{bCMi(S_yyv`k>wiMg^ zzi2eW_H>F`2&#ZkM^A!@gb8oPPHaB`s0C1VGN@~R0XyufrwIwK^75=dZ+@zPpjJ=r zok^rtK`u=HA~ozuP}BuHwyaV2MHK-&Q$XFoseEwVRpn)J$V1wv{h@2bomE{0%U6!z zFTsS=VC7Vtsye`q&Oa2PxDkr%2sa{Ed1tkWR0GBd0M;Z*E={$?lNwsUa9h^>T@pgR z@P^GM8jB0d%ZeQ2u|8@XH0>I{=k;>KlqRQphWCf0VahWkJrri=7$)An&jK-Wnh zdg0H9n^#7efk1u#@dJO!*2YFrP3>FyVG|Tcp~3S&DLXnk@P9Hv=VN1|U@_Z*Tf0lj;W&DsGu`){#?Vh)xQU}An6B2GTO zR&d}29(?CO?E;<*$q4DO<2sfM_VY~9nUKdol<5>-wJ${I2v9wMV51#CDr2zF_j$K% zO9v2eaMO(*8q!a;Ydl#AtJ@y_pLll|g>@O35R=NtyakX4lMXRNnVh^7gF@zwT41}K zGdLIoIV71dzD?sYVhl_hdYRqnd3o3E3n53-Y@G-OiIA}Hg{TVvZy!9!HZfFCXvRux z;^yO1LW(TozB7CYu^7XXJZIXy(4XDhKA4QE zoA!2f6^)t#tAWtTRN5&*LP|q5a|?^i@^bB?4{zW8e&T%*H8gpA?pS3cG7Ad}TY`xT zm>*g$YNLqVh=`Ev`B1Y^5kV0>Uv53%4?rRl+9@L=BN9;@9R)#~4*}j~3~~iZa>2`i zLPX1uhR!YkCO|z21Y05$ID%w@019-VeYo`gNASUlVj8HuD9zF#I{FF93h*+NzC)W2 zbzN|tNK9|Ss{5-XIQ0)$i+q~22fhNp1qKs}9Mo_0yCjns1H`9o0KEhF2l@?9vnEW8SdUK|a8|_OJFToO^kJlCS|R+^wVzbKC&g z=Gx(^$K2i!RCRE`$G|_pyYJ2b4Nq)8d&~fEr($qAZOKK#r1<&4A4pqJ_A&1`+S?Z& z1Sz|_f6NGzg-J*zic3n$RyGF)_RKFp=WF!g#n?zMEOLA^*ml2vUZVDbp zgc9=D(NIrQRf^?D<-oB#UWGl2JcQZhw5N}#VG04q6!&Z13 zgtwA?j&@GbTuB__&BDRs~y zfckb>3W2ZNf`~0-)S3kVjE2z;<19(VGh;Nldb~?}@o)VlZjw8}%nHeBGl}AzlWUMI zW>{SVU+%Q)G26w&Q*j)OUB!C&F6wOcM-VJq@{L|HE!KgNuKW2R|D@3 zEUm3S0XatjL1^_qdwLlEbb<_c;Vk({m?yWTNAG@wf)$1s3RoxLFBe1FWA`RYg7U!6 zpMg;PVKPua4rChuAMwXLu;(B+0>(C&Xxw$n`Z@}W2b2{)HID2rf`duNwZQ&JlJ%+f#^5hMX3X$ZCzbO89ck`F0x zd-HSyAi~wx4_gULMaUkjgU$&20ki}o_^inLLgs+Kzd!r1KHDDN{M=jvq|ca9Zp+fv z7R7Qe)*Uz??sWCyl4uObQ>uJ5fxt;kphTiUp}M2bnpRb3hKD+1`(G~V#mb5ci7-fY zjd=dKQ4L1Q-{=Sn#5bYm!*Y`aMeN-5=e}!`t)}oP9E@s%}Iu4QzynoA|3>(rInjC(MhYTA_;qs=R+rBkIgoE-C{d{|{9Y&taxfe~0516VNhSpuba6@xiFho>d_S7YF(-4CxVU zHe_2Ob>rv5GB(aMuz6ud!F$aB)eNa#{}tDTnh2})b)}2p+zO;gOxmCELMAl;JKu0) zQ+Birgm1K%LDAby#W93rGEgwGhh=)jX7}&2NJ%k*T7m!m&AWHZr7HMwPxqu|WJExR zpWHXtegym*w$PddTdXgK3aIMqUw3LO+P4pDyYIZR8e4$Kw+_!a=26kyD-gC!y3XWe zD0t}-WoNy5r^4vb_wTnZ%f89UA?)et$;!)<*U~~+aCpGt1fhUx8^rq6T~9!&4x7b( zJ}%DAs2M{@Q$dVDTXNg0-4LX(ugRAwAokB>-tpq>`BsM6I76lHO3Yr6FN{7(agHd* zQbFFS2$T>5_qnqNn}t3KkWqx81x#e359TeD$B`8uYS>%`Zqrog2`G7pG8`yJ2NY2R z(llEyl@S5Jk3wKi=yfgm;ms?%9NtjunpU>iLB;;?;RE7Lh;j1hy;s)weIQd=psjuU zHQALd^qT>WVwnin^32+aG15SF!`rX=NAm?=6@0`_qCGUEQ#7@=Bt8?-pWAUUnsm(- zJ7M~4m`KIr1%8M6L;nVr-ojB#Zd2sPhtuL)is6jYXB9qwT|7=3a+SmiXpZ{5_prIx zz=zD4I8w}JgP{2+C8fslQJ&WO%V)wfOZ%R5H@%>2rQ>m#x^MC7y|S*b@m(cS#rHqi zP3sfi#99*R1)N~NOHbK4e_e!fYC65L$j1eCj`gFJ#{TgR<(n%viB@+5>^|05+GJ_y zXSN#2I^Ih#F=R{nv_-<(5^1T8JDtMB%ofY}F!!lx-$=fam-A%@$G!I07883XOJW)c z{at;%#fw>*GGINHPJ7s3065=pr` zuZ9Reg8RTj!9PKrG5q330$?IY5Fl;;I)xk$&W~kGvv-_Nzq1#<1r4Gtfncv3TE^uh5Z>6Or0$IXevR4RAEBS`$6*bW z;W-m;Q3fr%xf5p_`VZib%b$lTHnqCxc2=B57rd6}#Y8-!$EnM7TtP0K5rS`91 z<;~3AFl6xoxKmIF%E*x)bxBY75cPU%%cUAlDIf&`Jm?GOr8l&4_eQpBLs0Y!HmD?5 z9M+R3BxOs36_bAWO!Y`11LJDQbutpsjM$ei_k*ww5k08gUv2`uLj&~IL7OUK>754B znD}@>&ZKZ?4I~Vx!CS^itW;C@985BUPzc`y#tx+=4i+t>CB}|V3hFUtm^G@N(H_ddE5B4mwRpaWnhNWmEs!!ldCgO!N>eAA3 z^6|!PH&dK+a#|3`$=6tGesn9YtHQHqnZ-)YhRCk3*o^becT zFffSfX=pwty;18>z8N+AOSX?+ml2(*a8vR5u=SebRgJ1NXF5Y<^DDsteruWUI2fmg zky!A*c?dWZT0MWH_WDgWrK}yMi|%VyHqRXnGEDX-|JZzwdrix7?(TG4 z*TSZ}oDlm(6VpQvB!6W4lU++c@}hp?Yntmxq4y=EhX@DWeaDNRMVVr z`w+IjwA^jF_~cA;vB*36w%394gwxbtB_3OKYZ~T1pJNVf=^D0boE^UrW}5FJii?}` zCBd|!CKLbB4d>5vG4-+6lKDvqXSZiNI4|<3Z_s1PVM)ka;atMg)U?~jUfsj7%+cEN zcpQx-oM0Ak#g;WtUwj*3G|=rrQ6vaZ70B1vOs->Cvrgg%dwq3^ckWQ5Gk{+OJICJt zP=hR?S1FuRko@yi5Rqnpk8o-Zol(mztffoTcQ^^BDyCUBMP)A~F$m#Ekr5by8Z*1N zC=c%gjc~eSWxHM0REGBRH*nW;5U2;G?_(E@b;18*g|d6A_PZ1fBpJ2o(ceVOhUrhe&bWw89pmR{^fEu0kq(;^^FWUGuZcP}*@1*ssLj*gyy zPzAdbbaO;?C69I{vvPAKzTY^@CcX&W1+vRWFeDSAb-TnsEY|{<4={CmJ%#ET8pWfI zlWSt&1KN*zZ5z*he*<`Rr|qV^$jY$yos#lk4r8xgl0f9h|P}-8?%P${q5U`Y{SE<_5d9ozh7Kt6IYcgn4E>F z!gNQ~qDhZrXl^(ks*uWImZZer3-@xh5GbFRRC+JO`DKBg7dZ!-&n{SmK6YB@{;r*WI1py87E)L87snlShR5#e5EBU;l8X((QY~m+&}P0~hF*qYQ z6@GrJwP}nb_$I~uF{NAk&+uknzVa>_9GO{aSBIm9=A+wZ6*taOZPYOBSd8{2-UkvM z112>vQ1E{d*Mmmp5GOO*yX^>xRzgBT#L{0s$;;#U;N(5pPqg{5P#oJ!SdIWFE)d!S z0GcgLq|!v-sKF7kA@BVqbZ{-&i2}aD6CEDBs^KWm=tp-T{u2l~8WL{cEI{;-$0Ui=u{zgdImsfMp()7SzL&rz zMjzP>6H5A`Y~>WLadB1mEkj=X+^gdRADzw-S>;`IaXCH=BrSElO{##S^!&kvkftRq4tfaEdENG+y0py|i^?y%9v!8hQ|@q(vQThKjd{!C z%@^t#+OMC}uAB-SvvY%QlfOeQK^awl-#PtJq04NC%al3bu1cRIUsp*#o0OAHdh=9% zk20I1B8Y^!wz#zF?pnzxJP;tn$DIoz^T;6-;<~b85OCs6bI|9`1=1GPIGK07zRz0d z6fOmJ%mqKC%*vVd5owC$+%T6wJ4TC0pNZN36~TN?JJn%tk~L@VR3?j$(WWnrJF&J< zA9p+%b_0K&n&^v#9I8jXNqL;#i3&ElKAg8V8FRELmVkj4v5eR_M^Cu!NJWJuS&Lz? zSxJA5pVA#Rlx(el4X2CcJ))V|Uo23b6bF!bG1^ugyJ6lr&8XdieyyI_I~uYn=o~OQ z0+Z@GaS{o|$N*zJt_#B5LZbh?6|HqQ%|D92Eb8aq1KKO za;SWd35C6?f7*zdCTrAVu+PJ*$9d~KFYgVK*Q5uIMnzs(n@d17@EDtn4vMel_L#uo zzDnq0-NVC?9~@^UP;|Gex7X$MOGSAnc~ER1Z7{Y}G1>z1DRhWX2>edIuqtR+94Ou1 zw@rt<2*{o2KyNiVgTu?)43d;RoOdv83_KZ9Obu0Bi5l4hUaH^}Aw-1o;~1oDNgoa{ zR_?xCh%G&Dxx5LXY&Z3+HYHtqpRsOgg2vyg>BR-tQfc`xvp2NiaPd#AH`oQmIt=Dj z6a&!4+P?Xl%hZ?IN^P{%_?I6U5#W_Fc5L>9$<*~)`kj*|J&R*5UFGGybI2-0q$AEk z86{)!{#AL-_0waK=i_F(Uv^dZ@%$M}WpUNDe!uQ$Az9Zne8=q-1I`TTN69O={>IIv3#S5&oGZI- z8D?hL8JCE&1_u7ZQn?r^Pd@BULHF?KNr74!GVKt0USi>}YXla%JkPLLBzV-VXRb=qNH%GP9 z>Bo(o<;Qj5Z{NOUAC_km4@4XR>GvSF@tnJv8bNPceVQ69S%I4C*r&UX;4Itwii$}n zx6B5v<0oh$6|cV(Ar3#o+XyQXqC7R7jx`2+y@iI+K9G||s}Mw3WGtawhupaMIl$m} zD0bOh(S^(cgwVa~m;v~p<9Si47x%Ha+Erh*bt*{4`VP;@DhAU5@7wG-#4;$9#^er1 zwtVDr!-LqdVmMB*tA45(Q)7 z|5gu1pOf#xt*3hU!6d(~#RcBG_ffV7ulk7PtIK&sp}uy*)u6`HP&0-4v@V!G0eqox z_g+*r*32o^F01z$I`PqsSf<90-Nv0fx_{~9>O89-UuR3bp8nBu>=J>%#w>3D7ZXf_ zO$}{X!k0DDV78*m!(r=&w3w!tO0avX{GsW^v6rLM1q^PSB z%T~X{`3`>#gV@rPs-zOn#k3&Wm4_V7*mKlSjVlvsAcin$H%q zyuLU72fte={h=HbN{hnNX3aripF;27YTO#uBeKfTiiquF4*Vfi`8}X*J`j#&PNU&^ zOj1w{GJsLJgh2HNgAr2M6rDMMbC9jzxT6y?$ajF-F8-bqBRk!Frvj|912`P@ z3DzjY^szAL{0Q7Q5ecd}zAg?qCUmy{2V}Ysb%Qfl4WLWeK{8@ILhzeWQP`8(O>uGY zRp|T^@ER1OQoB4&iBuM3ByHU#z5pjAQH=!)i?13k5&%y530Ze=hS3D_VIjr%P)A3v zBPV0;OtB6RI(}aS<`}RFl!`+4TQ~@xeIjp4Gol}2Cxn+c81iM^M2FGlr{>cyhSzs- zSYhq_)PB}B{VRx!UpM86e5U0v1rZ5u6G31j=l7|m=`~_KpPIWCataS(OV$4xzcN@ z2F=p5SN({5`|+r=j87vouTp(YSHTntpm8T2ahX4yVl*>1~}_99f{ODn&E(z{(PsR~@4%?h_pQgFn88PnAH{ z7|Zz{@08+u5zc4VXYZz($g2hiJz0Ah&zT#Ib#(F+$hVKwSEXaGHyV{^WY=CRHnc5gznZU&dknPCNs;>@SNqhMk7TS& zVfPCzQfZbQXwB-{w}d!BKH4V>ZGj)}!-D&+HGKXHxmi!3G9}dfO&tGDD3Ma)6u(|P zRmH_*vv|2^zhx`p4Xg*22&ZYKOt}Jm8?3L|8~d^Cq~*LAqoJJI-oYkNGO^6cR$qMV zaY&$K8aC!5bS9*2LLK)QtN!;o3!kXcn(kS06;MI}jKt#f14lg*a!OY>>C3Gnps4N} zNCY~45Zy8XqD!l$I~=8svAiG{?*fBQdeLJ zhfQIzP%VD@qRxG7_f#yGm%%+<#3$}sm^D_Ne;|8;i11o_JHF;qHphA%%fvzaH0gBZ zGcm6Tc$JvRZHE=z zssfo%DF`x<hEJiVv4cLL6;;lq6mDBFU%KHP4Ek_U+T&(e>< zvE&eg!dA#{7C-FaieQvD1EEEJ@=!Q=M0dJdR1R!Sn-9}xklh6#YvfH9)b2e5-aqm8 zq9w?C?0DA7&X`$POonk6)Wgv{h*Yh{rybxPuN)sr|4*_GE|S9kdl5yHOd%4GnGw@s zoD{JgiTLkprT(Vo+&3TUd#*J@ueb9!R6ck16NSosi%oowe3C>4Ol0$9*Gp?k_WX<1`w)0t;5F@^7-mWMt!NlPB~LW0RR}i^#v~9 zR%#iZSQ*Jw8Yg0wGEW&cB_pRU}V@c^H4c*>R zlK37t>mC$#$!t*WFL&45ev!TGM zUwb#a9Jo@6pEQmCaQ1ad%q`iy?`)9_4AJMwVaN!AGA#=VwYRETkv@0d;==y$Nd8q- z)BK5krW)GN*H5|frN@#4bs3d8C};-W5{)3$j_0%YI6j^8# zwak1j$>&7B{V9B@XT8=)3>!L9Fx|A+_UzZJ+yWVM#rb}8YZffd{&G`%6r4uEGISXg zXwJqJgK)*s)jzwQ$fNsvBMVsbHWl;`=Jfk9_lXZK)M6i&WZMgFgX8V623L=dMvnu? zuu=7eG|J_x2~RBfARY0S9)WWN84#3Qg-WfUp>Y~rPXW10x6i)baC93k1>W*p1;et39vFhMAi{raRr3(_jD~B(HvK%yjVdR zdY^n?K=*Tdo1C`|*XJBTU5yXtyOOV;>juG!ElGaC_xp?#o?wZfU`#@iU!%X|Y=?;E zQz}*oZi-;3g&2$b8~9A79_BgGj3cit)0%(F1BsKvyf`?zw8J>;PDy=+(_CStmkzM! z1^f$5-JFsJ(r_B$D=z-62CeWWDbB73j_&F)i2*h#v>-Rwtz4F-2_w2jiNS=lpJEMU zb7ck(L|9Mbo<;`eN`atWtTM+m?X(<$wbhEBUP{!*`6DhYD4=ls^RpQqsLP-Ub8gMu z5D8=#_^HViJ(}NRs%aJAU>J?nLv=PR?l8L?BOzXvy*Z1|!Lt*x}q68vz2k{r$#j1}^G~KuOe1b7#gi6Ncm#5Ei z>;Zs{NqiYOA*FI5`_fVN_syoe^He>=AKfFfIC-@tuVRqU0Fv&6H((k#NtTcEECzE+ z^)@s%pT~3VxDYyLi;%(n z#e}CyA=;8m(%)~e>|I0E7DapEkPt*W;dZcAh?LpAhC`!mUD(+CHD+ zM2_+v85wNo{s1_fG&=L)j-i(@By7?B35T0~NC5ML`&=M@y9U>KFt!*#gwX2{W$>IL zP&@?c{u-p<87hWd=*w+KqaZ)YekQtUd4uJ`+1QGAxZWeFV|AU(^09AaQr)bJt7764 zf}qq~uFvYZ3#;pt3c*uKt8~#pbuDddXLs>r8@_9VrzlI?ZmZxb0vo>lwb8}RlZ=by zjOu5z(IB$^o}Tc<#p_QKHaeYp>3BGKzkinPw-mywXpNR<3oVIbQy>$Gy`E8XH+sC| z3o_AOzk9>V5SjU*fY88)#W`Px0qC|v=FR$T{;!=-a?)vKKy17e3C zuHSbP;-7PK>m^tZ9&k1P+I%J@`+M(2LjoT3Ymgb*Oqs*a8@(4J3zOga>cYG_Fb^Id z=k2c00OZg$i8)Kc*B~Wat?= zU%Jp>{TR-3SC_XsTi_d--*LT+Kjj_#fWL5-Ze7d4;mhytANwlQO%My%~>+_{Yftwu)@E2aB|-YIvP&|`VFlOUq| z4>2gb!GGCwLPbVktHBNz*c{wRh15rR%9^(F^A8k0d~GREV+7@o`p&c>vP@rb-mZMZclr`6UdS4U>b2~K5xyqkd2Z*Rn*mqSHTA8 zlR#q%Sy|hjC+2qNFPuZJML3vW&|krVN`~}+W~)S#=A=)Gibo(M(bd%rK)nkb`6#y_ zh>!|Nn=`>I4^a}VV6TvvuLc1m$g-wjoV)@BfI^6-fY(HFYaK(wFqA9+{Ki?7Y&7aW zF25p|XIxMQ#3W+|;s)voxex)`06b?nP#h>9N`);cXkal2Bj#0X!1lJV*l+$If0%in z7~tyX{CzR+&SxNBDD#;a(<}CBxMB4&?^fw+{9T8_^YuH{>P$; zWKmY|1@z%M9O217^gOPW)jd|>rfmA0eHym{MBgHWEUpmxE8qWw6a14z1k||HCEpOM z=M_mmkeRZ;M507a5)zyK2HQ{Y1$+Sx28};FC`80B<%TTvZoG=J{}>y8K8So?#E3OW zC(jHR5DF3{h#cU=x6{0ZKN!MF#Vs-?7k)E7xz}r5{=i-;om3kT6RQ=`ODfdv#q^BvZnrWbDiNdS2YQ#zY_;Z-Aur_e3Aef|{s z{nWPL;AA5BTJ|*6@ArxoFn~nT=eq!h7K<7doFJQ96Wjaa8;3N6lrZ-M=+G~2J6k^Z zs4F;rP$o9AjxTSMQ`= zNL~N=t2t_}*Ab|J3`Y>a773{%fQpShfQJ|C1j?X!^@aS;Nqm`oOz>48@*PzBYY_qt zK_LF928BD^833dN_dJ3aCjYTD6+wDM$W@SJQi0D2Tv{+Lf}#oKj%HL=9!A(Z(2sa7 z<)gSn@K|6J_T{QNOQe971yr7(GQ${$II4@_9b#&_f{Y+w0wG~39qvGxMJXvMa($Wh zWuT%30I&X@AW-mBz}LG&Vwlby(fvK||5H$d3uzzRUe6IA-P+!cJPPXogfoMQY>3Tj zPwp5)fw2kF$(M1C-^E4lk37q+57ZUpSU{S9SVVMK`eh<@{p|MJ z>|&Dhi_RXwyWfj9FyL1i^XV4vvrb@Lt|j7=%PJWbNk$A#Y}PMqWR@1)N}G66ERIIF z&vTVscSvr9;49&JsNRTUqaUXGYnPX=_j5K{@&5Wg7Wd`e?~mMor0>~FS>_9B=#nEO zR#THN>j>J6V{3YB2j?%4fR&3xmdi;}Op#6FoBI?ITsmeZ0Z@%FTf=!TjC*2$HEdeT zD9j}4WfJ<=b35t$BM(FcVn#GPA4azB2iysApT(rWqzm)vz-$mX77}cvVFNXPk2{m# z7B<2xO0jSBfGj3z#0VZa?jLNBBF(+qzAiBkda@2Vd;>U z!2MXxz52B7$!`YTyYKlD*LLvDzw}=zE|nuat&)_s+1wf@3#Kj^-{It)Y~x5?e2MA# zrXkXuErXsNUfEre5Hny9bBdZs1>RL`TDJU-{|VqLp7H0Pb)sowfTk9@tS6c$?dJ5C zNgF3OqzuRpqa$0W=6m13?f*<(Q<}2vFUc8v5=todXOn3oyK&X;S~unR#D9$$KdH4+1?g zS8LBs0go|+gceAb+(2ngV1j}gLwaP01Wc?14Ss3}V2B6gWt+Jbpm5!Q*_ITpp`ts> zp%*a#x)B!UPJFNkUIizolH)CeU{if5^HSQz9Vi;?4g=UT_;Dc@|(#lhE7Y#w636!@xc82<4&-|DF%MDd_V4 z{Lu$kgaD`EH}&+7I=_-%&*Lo4!Se!#JrE@yb9q#P#V>3qQo8nykJmP>V&NdLl2Xv< zfO0M-Nnn?Oy#43R^S*GxqSYDxW< zwC@wHwsydT`HHBVlTz*Viq`bpEL)l?_p>t~^fDY2 zDf(T@YeE|>6!6Xfd^Ps^rH_x1Kl-A0vL3U}kIS@`cQo@X1RWpv+U30!(+0BY>P z)b!<-p9q~ie}nVlc5lDystre`j6)kU&81P%Fz#QqvX;y)FvuVTfp$ z*v)*Wiftx){urRd(8fudyia;KXCtOE&%7dIhu#?RfV~JHN?Psj6T9zE`CsnHWI@SD zc@hVU^j&&(R^-kIXIOsZi8`m~h-pwg1^U6TNFk%5hLg9rq!<|G5Dqm#Lq?qsA3(Ip zE|c{shmuy+i&FIP*%|b0v{g5CU04~z>gfo86(jQ#d`EB^0(zOeU8(d}MyJMJk+d(V zXa5I((!L>%Ed61{ZEPw7_ru#)s;61T$9eQMw$e4qzQo2vxM?CIK?-0PQ`5 zjN`}#i?Yzc^NYNTqF@G&c3(Fm2mo){IFt^`ZHK!R`7lsYI7(1kM>wfEK48H21R3jl znHWk-KfS94-Bm$B0ef#$M)T~e+orn5)j&)&bwZBovC)vCiU9PhD3enZq-LO>M(k#a z^B=W4;7=K-VJRi+V~hdU^VkiZpul3Oi_7q+ zLa!Js(gPk-C_gMLoH&&G*w|{6D#UTn^O3%!eK@{xKqI%TDvSrc(${7N56KazCA)MsmJu!x zk=?adM${*3y@mFx_bbFq>Uxh11CNIv0}obq9Ru!;A3`&K&Ho z4p^A-Ia~R$P!K!MYuqBS)#JZ#uc+efjhmNxQ=#7Gdy@dY2zyp-+&D!DnoE6ym-Vv_ z?NbANc+mp>y}=9>LDDgvQNM0|uxR!MhD?xNnS)kk#UH~-` z#tsv=Dy$5L90u9@`I#3W1jfEtZm>ijZd&npnxymRL6i|krGEHq zX<^ZsdUmc&&eKmY$TW+F-Dq*fniCcKW@b(`J!dYJOV1}O@x#$27!AMX7@y`HCgeha z@;39Qsb<2MbknCa#k+fqP6B?Hym3syMs_GfH>l3C?#K0nDQLACQL-#bb%q9z76lP> zh)$6peI;^gLtI)2qV8e%9-?^C2eUlNJ(~c%rr5mG9Jp6?vy?O79zfv`A3ypVM{Q&d<`KnfOYfG0uX z4GuX4H@C9mQ!v_~`G|Qo;ZN5rczuGKEFj2)*a1}d$o2y$@6T8}T8b{+Uvpa%Kv|Hb z`5rz*?*gJ9z9W}BWPoQC6xgNDJ#R2a$H>1p00`Je8xzVGb8;QP_OQAq39eQ40HnBY zi2=n3YmMjK!gLDLiQfPS+Y`;^ZY;>Z_@dwD{0;z%)FA;VjG$*tIC@X~e*= z_meWZ*#a~)UA=ekOT~S(;{vcnXsmuSTn2Fcf^?3EV(Xkmcz0G__z+^^FW2NNa zb+a9Vgyd8u^9oS3HnurR50rx^O;2SB06_!p6v!ipI?Um{0m>m7xgkZNuw6ST5ahS7 zY~sp)9yAF*Kj{PgkBK;f0ce&QLgXp#q}o+1S5>c8liTF1q-QYy!^Cs}&I`wsJCx;q z0UAyR;qYRM$m%qZahQl-Wq9;s#+Nf5D2H<}QN;!>eX>=7OO$x?t)3V{y7F?K-M6`j1SC&T4iMNs@#|Uut@Os zn5x~tg{;!g2wR{463J5KfR*HBXg-I zF_8!~B5{r^QU(SMVJUfhvbYt=N@i?|w)gXgN8Q;l0i9?byfOcx_iyokj`i}CFk8KR zMVt9ZmIgbk2jaUw(Oa|P3|8&fl@X;>BlnNt^*r1!_M|E05{V;*!m8p*vfe-_>)dWF+Le-c_o(x35K4mET+ zp-D-y46Z=h&%n=|qhFGP*Ukf06qqBOi1SGIczwIY17Mi>StEH5T z<-GLsBR8+SAhjG2xuTRxztT7Y{^mWV0vRN*=?W97i>3xvn?E<00&z=US$*y|4*v#X zpeHL*^aFob%6RFc0Lmphf4rv7@jq%9x-BWZh)n9*o>gVZSm3Qe5Hd2OW*|XNj#%k+ z+-IscF_Vua#ccSaRpin#(^HMG7j1cH>dqWE#JLaHL?lLofz*6SLkBl2fX@oH_al!z zwsE0lMN?SvQZ9Tcl($`~+7%`UQ?hVklQMZOulbE}ofhl&n>_FmC~3JSu!YH4e_^|s z&qR%SN#H!=0}*R2b*LZXyT%d(1drY#*)|CMKj!l&a4-NgStd4&dJt~PSAl0hze1sE z?(|?kI|Hu$r%hfi`pqdBhDm126WHuiCL8cxPHt|}A&DO~XN z9%vC=Kj3%SJl0t^%e|geoQH$@H)N&)U7FSe0)Sb$%=~t!r6&pU{FHH1Gf*E#bD>ct2Vy7!G<1a526XvI_TJ2H$t>mAW%X zpn}L$bUC09r~k|UfbWxbl1ng6t%1v zXlu@e?a}L2!_l|o36O#J#24gv9F$=Kk}Z(k4Vf>^xOB$;C|U(Eq6vJ4ScQFF^hU=s z(7KZ3-Yx_@r3s}VLz?EvkZP)OTz32T|c-?oP=A|EfbQfi(aWo=JZ zE)_JOykyj{!t)MemGA-Zn$t6u>`t;eqxXuk%(Y_#>BCtlBeP0bMQq1xiWF|fM3A_M zlRWqjYzQ;=1O6_LPu6283F^jiEcY_#8okraTLIY`^gk8WK6|HsPQ zy|39nmdJLjY|UJ-ex<#w;K2F1wi>{}A^cbW$I?$20TGEmv3bw(E;d*rz3ax)O1srj zCd3^{@WM&K%pUtD{(ZSNA5#btXg-WJwlK{td;2X1sXq*W9dg`6?2bgH;V2zwv`4_i zlaQg_?y{uW`coe+Jjlmi84~|O-_t8|cVuCgm7FxIRIcKsd|?;SD_U%%+P1_kGR7al(RSjevex}Do#n;^3t zrCR1sh*t$s^ov^(k9ZELu%?e(z{c>~tdtclv%d_w`gim_^1AZO&>%Bg6p0EScp!=IsHMV96NAMx z>^O0lUH|;)9^b{&x!WVt0hc-q3DdjHRBieW-`+DZ*#n2iIBmIi=F^?@m>gpek1u4x z`DlFipv!nH{cqQGWiJ7IfOfk6kJ?Id^6*~Y4|LRv8n#QM0}Bs#YiHBmw>>FDAt4W= z)@xBh$p1x!38YS*W7WuOaTgx+P>(~Dne3A%ml0oT43Z|5%7?{}-@|~ChahYPS?eK$ z%K}md6;4O53SyX=naRwbG~`3%7{qJgLC_i+%U~*e+ByXVITBcD2N(*fafn!l_>w8e z<81-SDq2Uu82+Fgph+;@U7Pkox>vB%z%-U7x3+R3@oNGM$b~5Gybjh*%w_S%7Q$e&%sxK0s?HKyCAHnZ)}<({ON`B2WjQ5PU8O4MS)q z^XWQ1N}xx_)Z*geZql|qxN)=~-wWBCfaXO^E@YxWtg0d~y{I?939Mu7YzLmW(07OT ze?dkl#tEt#Y&|H%d)f7XH_e$LSJUYWFlvv6o*vZc%4@{x46)~^^_&!`{$LECkPAp53MrM{udQOyJBVF@@ zH^k(!PYllTp0MH|apKiHLWrbzx^svH4jvmO^6?c3{-l>Ww|mlc;C1zSsB?G$cgZTQ z_RFpS;kp31-cQQ4f8=xyK_y6KKv70taeKX1#Xi_yRzdS3Wg7=*JzhMreb%xR!YQMo z#R7^EkYs|=SlQI>>m)-}Dc;}Pw_5`?K=ZO{N@c`g3mTi5?@}?zsfBHwfl#d{{@32* zk*dZCPZyKZzERK;T*wvHxoe20>6cBH|0joq1`8s|* zh}%}zR1op~-q8^$tUGM#*0aSMC!fM!({nqW%fd#>5WMUnyFLc~0I-F8Dlpn`dtbQN z{QVKhr*E%y?;7T)&O^8Hx!=TQfH&YfkO?b0?KJ!t<4`9}C$&dX7~?RXwIF0uEeiQ2_w+PX!5oRsg#-U>p%6yJ?u`4?Q4xtC7!QB2f18XE8xu`zLI!$c4HWok{GxrrpU0d^6 z^MXhLFsfdJn^FgI7$K4dsnqcQeuGxs90G74lOS==1jxdFG2g!x$R?0QWU>hsdPO2I z7ks-A#RH_ z#{&cJ%d^SlL9vl>)#y37{OdeYFlhyG1OYrBI5_-4#hw;$RG?30d>*{H32?Caz*SCp z0+2;K@0uh(l~wIev4biG9X60< z9Yx?L$d_VQIC&?li68OfV;vo)6J$Zb)n8la$nUzkMBFhJ_KpJv=hXh9ySbt=m(=8KKZm+YvbyfkE9%8f?^zU;&(HVrHvgJ+CL7p`VoD8d zwg~qz%2HZh?TUb}gT-s0^-yAqr4G=!v+)yGfuie^2*H~MsrgE<3r-AgF%rizoN49D zfqwyfWW*0;y)L^Z0-go0!#%j{;L(H(lq>2SaTe_z{7oVtCOBAYlZ{o?04;Wx?U&GZ z9E1@g7movCprue^>+%ywm*HOKj~0CWgpEm?fK{*4QKd|7!@GI%Y1jL!VU|K5rGNLE zflr>nK?XjDN*84chV}Eo^%O<#1aFs9$-1$T<% zrMYM$!XSUB=?DTm$U%IQRlf3ds1a5>P_IFtN?6&L?~(tJQ@Ip;RFiO!(RJDALq`*V z5BIt1^)N_NZ5*=B;_eEXknSGQwoBNVn(Kr$^Clh5*6W{o=6 zvDdS2v{mX9++bh{zn9asNDf6qE>BR^&R*^HY_mqW{Vmd3(}~9$P*Bsb(Fm^nWte`O zP5y}`yTgWtGgpD)rSMP+@S+4ExiaJVJ#U_SXbb95%ESE3#u&y!Npr{%r&g486HHij zazDO}Kj!&uOZ&(4Xm8fRqY)@Zim+XLzZL#yc-l8#6T}`ee>BZf3abCa0kkFS7rY4Y zYslMwC48t|Rq9zXFLI~Rw1GX^x~zjEI<{ek#;(u|09+^m?&iOX4#fF&nWD8dcV)OK zGPOl1X~0<<_^cLw#*Opv=oi&wskon<^u-2|~vl;77oxp1P=E0*y?lqM-c1D>E^l{t|Q7u+4c@Z4oZAXbVX8~ecydub{U>GN$+nh zb-kfUx7CXag3n_@Ro{8(rI~D1wKT}Lo_iGf?jP5&#`8d)On=V+FQhZ{!Oxl&CJjqZ zOtTEOSJhqu2v7^MS6l=rB>AKwegHXh0n5h;ZG5pesJ{t-tajR#F=v@z1DY1O?4rc3m-9OO7JL%<< z1h%xGfmv#BFr+{pZzk+e0m{)3LL$!{u3^IkzYO9SI)HlX?aCq`fgz&4Ufx`gKFYb| zTXBFKXta*!%oZ7tM;9O0gi9-h8zj9PtCdUq;Lqp5zqz;5RMNe`=o|?~+rS&#|Me$k zpm^a8a9sF)l4gPgXg(CAe3S|*b+|$xds@uW3rQWDI+dr!$I^qyC&3;}4Fg8_eU(Ut zi~Qx|SQ{njCmH~osg0h2NMX6ehbdY`dKFp}8W77ZiQX;A{e&BtKk? z)GfbWUr{i29k7}NXiN|1m)q*_!Sg8>aDNa-Dqo#Gfeo2^9%&qElkj~EINacpS3vF# zHfo}O_2xT^5ek_3hs}y~ztf6IX;1<&QGZ-mT%^2h{Pel^&+B3znMgyF)YT&p5Y|*+ zQUw5V03>sC|3j?HqDJiKVADY|a^&&;Xm4+VeyIuIa1OTER^QI6u;`#I?jl-N76niQ zC3rlj!0mTTm*(~vokJ-81!98)Rw4dE!Se}R)Z+h!d2X~?HVj6BP#~N(x(h@wpF^x3tui;ANja4=(<|zdjA7xsH90Z_u|$8?dbiqS z9K?S{L}=xkw+2x~jEKz}3L9O3`#v11tPx-TimE}L1M!>gd{u8Cr7il%r#KZC2QKUz zgYvcLuC0zfDLI=+Hr)Ha8T1HwuZ7$dH8urz)ie$xs~)CjY8&TIXn^+h_S8A!A=rTX z%i(D!cM(??2dy@N8Wg@IO%bM=2vfQla82eHV!r=-gJq_o#y^PYd^-0=w>jT{)}IIK zunVxJqR^%(3%6)f-G}hW$Gzxyc_{H80gfW<9GYKPppZ`22G*DGjizD6v(R~)nH^8q zP4-U}6V$zc=oqw0u`kxHm-ULEjV~kuGi@J$nJ%{T_9uM{5XFTz!cRnfwa?bpc2Df; z>dD?1vk<*E%)@9C7?Zvko*l%=O~U#u%8cK?d*Vr7p+v~=BWUGApf#G4~rs->z)wS%jTiq}b6DffHiHY2q6kR@9yl?Oti8WEJS z0mD9ocUF#Vk)kCiO3=cfiQxH&v~$x9sKIORX>m9q88Hs_j6rNAd@2DEv?jq_>_&hF zMSu$J9yeQI0O<2NEEP-)&O^utxRf8eUI&;MQKPCHy6@UTx>o`@aB_GL`wq2kP?FI8 zv(hiMfq!@(n-yY^*xbG+uK_y<=h}r8E%w zh-+A2sJR6Ta;rU%#eFGktmwdBUd?@ojjJ(_buskzl?lB9mbWjs2~>q9t;lik=wJVx z2>MZae`e5QZEvroVDQ84C%XsA^p#%jGPbEbbIXGhBWu37N6u26dj}ShsrPel!GZ(K z2C}z{c#q!KPkP@}WM{=WmL}o4IW2T)E~R}#h25NEDx~K3xdt3A5?!7LS5nn&_QX=e zzhYvzt{pmp#kUH-AbV+EkkO*6!Rb;Cp4zSP9Z#Jf)+s(iK2?6=GK{S0Y5^2@+%s42 zxp$?JVWdivX#rc_!__;A&#yyOw1;*q8_BS^&+X*_vFl2A(e~z%8 z+;G2Vh?7qm;lLgGSiE9)S+iZfq#aIgw$v9ALsm)<tbVP`Ya}-(xJ^Tzn;M5zR>&?|={kw91WDY+N83>h{|WriV6r}?XbP5gp0Bo6 z`_&`0@nF$kNs-X z$30}}x@DbNdOumm_v9EgV>B;PtRG?YFFA=PPVBhr{(QgU^oBP10^Wqf?;X=vnsX#F z-#Z@kxY%C^4#{5ncsqRkXnBhvIP5_pbjMGBJ%4Cu<4jNO_)xJvqrqA<@k;Vf9f3Ja z4A|UP=-$M;Zd-f$pUyerX!l0;^)PL3ENY_Xn>ync1w055oF%ze(b@C{PB*l#)x9gi z)VU~R$HaVRtQQPp{!uG?NA9ppZsqv+PvSRfREfH|JtwY!t>a*PCp-B~CN?JEF?%&V45nulnZvG*ugX~ygVxgNQ|CL~ zgSWO2hurBsdljjQQ=epVTe!LP9e4;OjRR;f=Fw7CRyODD*K)xP5Jk_!)CJc0GbF`8a9`t?eHD%+MiuRAE^T)zDeIk zp0yNl_CxU&w5+|^*6hgWxXJ7o_jH}?j2wyVziqxM6fHFKJK5>jgKtu}b%K5tj`*e8 z+X7;~$D(Ke($DQ7UUj;pQ`ooBTlS5n(`VIb!?t$U7S?jOx8EIgEDBM|(BU2{n|2f% z_TKun4;$UW1iUAR(Z{k8e7zd;wuuXk@6K(%ooBV}DWXrY3cW^_*3Hg#%=U-pq(eMw zqQGQzbmlPGNLamFTG({^+a}G&oWl!|D&!s%g*LCDT4k|3Kt*Em1YAn=6%FYD~f>>LiXlf^7rr`4|aDrO_$3<7gaRYc-Q2 z^q<{|zL=9O_G-2L;Ogaw6p8u%%NL{ciL@S2ij`~*6R-x&;9FixY=r~R;PXQczG6_LWZhdeBK>qSo!9ZI^lk!vCG^lxnO9|arf;?-b1HYrl;GIIoa`7 zz4A-{Mo4qT1>WlHN%$^EQarX8SCilsX_zolIbpLzVyN3zQ}wHKugA~(^cq+3p3K>n zOj~a+Ej)WK+fxOAE8vJ#jd88gJY7gX4YMp6So4hTjc9{e{OJdV!-v{lgV^G``Sw1! zPRpQhbQ|>nk4i%6jWvB9mvovRetT|yDoi^cRbpjXKD!>rr)_gMLbp$~)mp27rWOJE zv$N7dpB2oJA?%s?QoqwVztJ7>20`HW$iLe?`CL|}Lcltm=y#BqdLKROJuW}}PWt@B z3TcRNh3L!gI#S5DdW{{8mU=}hXU%C&9Q+QE{_@qHKhbHW%18{I642isKzVCg58Z^a zFZJkiNtN76>NJ+vgEXV0*7}W#`LsxkPKcY;j&YftEZ!E_f6=+; z0x>WZJY2W4PW-m6`)yVC)fiXICfl~=myO;4uYzfuDn~2S)~g694k9>X@*DvAk3;{$ z($*G%8!>L*wo<2o3Mptz-P(9B;1gCojw`?dnC#;xPsCb$J`6jyUxpmm7o$|Slcp)FKRAVv zv`|^lb1pbxEfirTsS8asW>Bq-&ME#`E><+?Yx0BY^v#`naktxf?s(E>IXtV3FJjnd z_d8ALsFi#1P`N|2c`$R{uUA%v)1xEVRGs#QL0^s}#oWeetE}TRl1E%N%(~Cbi3y9Z zhMrV#NUX3W%#A+j-})M!voJj@H`jVC5=vAeJ3=|+X_{0+Ro|j-uJxPiQD5|4s}of$ zEeF{CCqp0GW%*UGh%vk;_+eaC2NO9|E&NQwAx}saVo!Px8RBVPrI?)GbLNa4QMT4R z<_aP=7v+R1>{u=f&3X8)Gt4$P>^)(V@t6ENQ2{i;)5AYHgZh`;AG*S8EaZJ@6x4_x z_L=oxzFwth_67Tnbb8>|BPsJ_&WbtXhKrk2@~!tQ-`wWpeR58tn=SHMN$=`2@+!^m zkM5Gn6X+U54!g|WTfsira8}k@yQZ3GEzB=rTlIvI*4b2tPG(FVzj*{ zsT32eii?WmMu8_bTQ>K(eE@gC0zu}RH*d)1`vtyQbGVH=w`Wl3 z*)w=ryK}d8X0T;#FFx95`F?@V>2l}kPmhx+<}*>&pcqKa>8&WtYChLqZ2W{J%GYN* zi`ivGCRNL%5UC4f)>6VVRpA~tR>qVk@K|+9UwZ5XU5O?#yHQ5Bbb778egTWilV?Bx zIGXwm4UM{v+L{%lNc$v%Dgm}TwyvFwuEnBOV(jZVY^XsUekOFdwB1bGq>j7szSZEi ziP9)+7Nbg=>pVL8%aeYxY~2QJhAixFtC$SuZN-;f4>)oee(pZ9WU7J@qawAa8)V)q zL-iK@CKT`=VZa|=w!a*BtPJVAYMk*=`<%Hv=KBpdGF;Q0!ahmeVo4EY3(pagFVB25 z{bp}9Hj9~N=T5-SSrzGZ_8`VDUY3OeE+^LaIopD$RNxDi#e3$0$p0)0T1uvh)y%;~ z>X!aQw0_Z8FU5BhEOeILPlTCU>n61eM6}z4L zO0AS!8i%{9F8`Ur5sIHyqWs=pp_NY4YWdCgvO#y-LJe)j(c^}+5*n&zsOIAvwXW@K z(*;+LEJdY-aXuf94~3rvb_ETW46%ey(t2vIm9}~}DSltaE*^4DZxPn~v-(u);wfc= zVCv52zQqa+=Hg5!rSjY_q=P?=I^9jKx0bDrI6b*bG@;I}nAK`lejwjQ?v^g)@RM0Rp_w+p?KTH&8VX9CYK*BmbRC_4!U=2rz&9Ma!Mb9A*kd-@7ux95@bcDnFZZO?S4e9L+TdDI$e`Q94k`Pb#x3~ zK{oE(jt;jXCH8VL=uKRmr_q7b0_P zUR+b72?~IJmnf{`7&sEXoSdx&6-X9i%&!9f#p7=llhWzQ*xAUAN8QN`B=d4@xJv@P zAezplhg#DNd=ZR)&FT-ZFIi#vL#*PQ4AC$CDg7}w`q7I7JjCYRg1?j^Fd{xCYpqIq zjZ>}=7U8aLSk4YUppe&E6KOLoSyJqwvbbI4MxT~q>ulUL6-lb_V5`9uBiKB~>lV+M zh>4pK&GsF0&1z##+&rS}Ao6+#wl3#X`s|0div=u(xJtw*@KymUfja|QF!e8Xd*#n5 zGUtYCb1pDI;H!zmqz`{A++MNsP-D*L(;Vs@R>XrJ-cG$P{=!MCiYS%b@K&$I&fZEla|R!ZlJphs*=s_9MM_x-&XTj9)ohfIYEI?S4DZjq!@kMQL}Q|BEXj1_cP zyT|@~uRs-G{(ATD&Oq(R{Wj<>Z02`+*)oq_ui|^?YG{#6yC773HErSXs*t=h zecHBRof`HX3qx9TOZD)}>5^*l|0<}GYWj2U_rg=fyH~Dki^VQqld!dYL4NnxY?YUn zN33ehb?{oA(A~TM{)VBYY^%a}C;TUgZ+~0ivhgO@;L&bzl5$NKlpGbj4AB}()h~L4 z$I^Ihjij0Cwvl2qP0DnD!RpA9ijEpa``q?QBYSd-!le=Ax$6A?zIk*OvSi&1M$-7cnpKx+816(aCAK#%?T8mXvA2(Jvz*Lv#g~N$=7UBq z=?U>U5Gkukr*k27W+(5TXl3?37t9#gRvC*r-hwC4lG*|#Sv^Tp=seg3s--p-UaL`7?wpq$cg9$ede5{B6zaX(21Y2!{+15Y{ zi$0E`J1>sM$xc}+fDrq(@)Rm~!a;R(_iI_~<7?_5;qmBjN>cz{p#b0c#7aHGK~w{k z>4`)I4QhRjGj*^r(BHGG4}RHmVzi#^GegjV1#zglZejPF3C+*;*B(w3f%eIqS4aYf ztA>mt#F(Q#*kTnzC%Em~9#(J0)?s^Ja}SGlRsV}CByBy6>Solm#zL*=7;4qE~t^5XS8_tg%W8Tf3Hepu0OJ@7G`&z!PUeE>e>rF>}>J-kQ0VZm|ltzRStb-PvU+bgY({%w5;)zGA0g{b$hWbJJ|Y z+>bE!mfuBUUvNUJa51pROOwhXVz0YN@=}g+2Vd{LF0}!BK9l-kwQA!; zKgCif+@Q+X!BlN7@wj?=&^M!k#i>Uo~6N8D=S(7B~7Mc^04D z(X+6$*EOSSwQ4jzh%Zj0l5fC%*SL)de?=PJr}6!wy~H*X>s=GuhGIU))wd-HOtPU* zaEwH5lHg%a?j5y_Sd*eY2~$P#Uo#X*<}@XYeBwKJFPJ-$1gV9Tsy7dWMK z;ykU`LYMXNkHTXjID)7nls2Q!_M_>%RMEiZf$=fthE}??=D+H_C#d>kM#uvIH${Yw z*?Xn%wNC>ib+LAxxKanBtI<<4QC zzb82Yf`jGnJ;Q6p3Io{o{_3_zqN)u&RKZW_E1L1b;v`+?0`DYT7H6U%wRTGSJooAG znxG^_2o)EWpU0?j2aOvN5+sswllm^}K~rp^hlc1ziaI^ zeEu%MsQpf;d9tN=qH0x(*wa&6r@wP*(KLW{K+VO~Jm?FvuysA3c;_-n^Yr$pIc_)1 zq;BaZ*Cq4*%y0McjLyYrbmV^_jeftc-*Q1@Raa5``QWKb=Fc-gIwT{}ET)o9rS_qgNmn~m zYAx4RxySiKvJW6uVLc==Eb$8|s>PF(>oCR&51wq@+AcdvxzGRGB@i&1@vU097#HJMU_W zJCx(kLov0^tMb6l&^uZfo)gW8419qAot5+x@oqFJ)RsUIFJeA&iRp>qQZ0iVieV$c!E~R;`vNlgv{xbOp4PepdWUeYP4I?xggo>7orR+A2xDP<=6hE2fUeY$DMeKAeT)J(b)#HWH2gOWo{5y3-RC~n zW9&F$q7i_fqXXWdKnUeiV<%Caou*cd=GeW~QTmZP-qBR$9HTYGY7DXtPS@pQ2t z4~P&;1BI*+mC~0gJQNaZTX4jX%YQb!TsK!znEyx|NRJP$*Dx?D?Z)3Wu*AQ}`dls` zPf(eMk=_^K&KIu8H<=Po6Rg^tMSzDjzb*E-r@i24z-kw;RmoAOgdm0bOW!Y%WrA2H?%ASA6tjf z-*ebNwJ5(Q@D)@Rrj+NSbnQ*Smt|wZ<|5S@n-{phAKxtjNhwITi;R9Yd;Vqp(?$B8Na|-CyTMsWE^BulJ=$3J$C z4K!PdB8!(i+zut{Gv3;oH3%1tEl)(GpSe}fN%;jJwm|j0UjN7lFbV$}Gk$OA37eP9 z&O8|z8OIm2{lt;RuzJa4t+KztQ->S}6T~Cm-6flsO!vgAr!K5HocQ?p?ayAX4MWI| zsM9P-TpI`a|DX(PQIE}O+~@<~j)L6ai~oI@we(h!JJfg-czoL|pC!UwkC+RO+QRf- z5B`x3W>AYLoOUYFcOelL5t|&{NqCN$33nLCgHBF-j{Boe=ucPIPDy=h&oD=p4j`g_ zf2sD=RuTxJ@8D=Zy*t-GJgitXf`@^yQy6LYiRVgPtNyhlv{&9Tp?$Wk4W%nkex{4?Dv}CgQZPwwVfNRm_%5iM)x$eE z58VY@dgs5SaLwx@L7>loLa8<xkviL*vzGUajnO6b~Eys(6r#z-dgMDm_q*WcZhW~b=crV{)ZUAQttV3JR zxm`mfv*#oe0^uI$G;@cB+o^VL+Hn#!Gm+qx8G{3>d3X1k^mYc>#3wk^o0w=3ZJ@T7ZV6KOic_bM72y>`Lzq8MF|Hk(*g^!s22whV^C;G%CZ4aV?idlx^ zDeF_3p&d#%`yPvjsftc7_pLesKPq^cctVGK!v+Uv+wXHuglpMW89i2?O$y7UF_@x0 z2x?Hp>?YSusb+;f&ydr(L9ybsLL!CE&w)wXXqp(ciz(L7AsLy_zUg7BpCA`D zX{k`AdnHdh;Ow}*{LGFJp7d&y>UrVYGf%zm~F)2s+y?Z*|(R&SV&qr$K-ptaD~#AcQK zz80uNKs4}$y|}L(ci-C-z_p(japAMQ#dFWV>0UYJgW?ydzj&&d$TYg;6K8aJ>6Y(n zbW{BO;6xan!wQ6b3sJ{wLRL|mJxq+dJH_pcLW$JQhL~K@JrjljT|)?Iv5t8 zf(BsTi4B(c<}@*KC_T^qkDE`?L8(VDts_Q@Q;=R$=hEywdQQ?opT_&<`r#hdrZr53 z?BCUkpLJQzzNWHNYI5eU6V#Z#dI={GlRGCE2yOVR^S$@$_B7(^HWe7ha>R!coR;@I zV%08^o6vK>!T7T2_0p@%t9`)_GkdInHc$1y_GjX<;BOPXRaicIUoiO|KX3m>##Qxu zZ$vg1INlwQ%s8G8&m0FU+5X=T($7Prmd*c7Czu8XPnY|zRD;mG;XczAy?(C2^2!Q? z6PPS#NZG(L(qV}8aG#_~-}i4i!;h#vcvB1ZlY{gr;5-AZ4C#i~klK6*HX|LKti32$ z0)zA?{l#x?r%anko!)pd8&on7F$$M5Yx`z$+x8J|dolb2;QzkUD!w@B`;A=2g=ZsU zCl^eQC+Ql5HLdSLk}86uk=I6t8rQlrtqHI%!=DeX_7(Zv9 z0amR~A&OU*KT*J)u|aSj5XW#dW1!<3>3*u=YtcUQaiKr|!hygJ1l8UkbF6@7q2WMZ z?SB)E!$C*>8v4^g+SV_7Un-1scA3NbtyYv!Uhw-)B<~TE#0 zp?mjJ=vO}C6{d*${~7d&aCVDX8iIy_e<0uDi#CD$&yX zpp3E~U``|Bx(S+&pl6uNWk*^UMeju@!i`8dJMEa_@cjQ50Ikw+*qZkSysITCy2ND!g` zSN&1GwBy1wZqr$pCEkdj&ZP&bM_>4e^O=?iioW*$^1p-AGdbaOA?m|ddb08knG_97 zj5c>_A76hx>8a4Ot7^>-k_}-^SFdVN;paU?UEOIp~19YOAVo@vH{G$+SgC^gHp zs4pb*(QAR66^R%Af7q0u9||PJCR?_y;5l6i4!afIWI%jXA#nx}Z{7@ZHE0A|v+D>I z{K;QNh{11d;c=Ab`}ve;nqcU9?ukST3PIkbAVoHBORr3&hxVYE^c-T?o)R}ul6MkJc^#n0^-UPaXn*^NtViDfV}% zz2T|QBXoK8nWEfrGPSsMXfGn@NkcIFeQTp@PoWX{I#+|XS$At*XN^@IhdljbH^1>j z@@VgA_M)1aNU)UCpw&)_x{Wr%^8ATaYk`1ZK6h-3Z=YHJar+x33{JdVc4ZqN-MGCk zb2fK2J_p42KTQeHX}Rm6Dw0?W&Vw$-6~@h-(3?zJ!wJ!~;<}NxSB;NQgWqd=8<})9 z1Ogsy>d3J^sU+Y@0obPRaf=@#!bGFNV}p(hy157Fr8i9iIN`Gdbw_Kv>8_CX|6}V< zps8-#H*S2Jhs@(vW$ZeV|}?{%H$aeR*R8le$s`nTHkPv=f;C13B>6e;u(M@SR2 zf@OzMy4u>m+dRgp^V%`!u;n%aR!OrDbt(g17)H7%f>#8_hvUXpr$Nj8x(#j4MYgoN zi3LX$_N4QwG{{%<4}|wkw|Xh)TJ7Me+9G&=)l}mzmm}#}kn$Nzl|h{c_B-Gce^NGT zI33c!hF9)dvV&Hiy0UM7L4S$SxeDew#xoKtZM7%eDih+ausjLnG~&*#cOgreY@dh zSH}kf`cj#CVpk(wZTe6B9Qpa{UbUixpsT``5e5k~5}&MS|G33zt7Yzy<*d#-mKRkx z)jv?K=Ctgh*5#Jsb4bwI+B=9tbvEw;b@EP2TzM0pT+8x9YOxO|x2wzmlgv zZnmM4V6h*+Y{~jTe73;+rM5W_k6+-^mahSqy-`237!*$`pL)4jKq$72S3T}xm3^$C zLPliQgnkyMVDkq>lOMaNHuZ8@JH`jeM_NmIJL}#%ECh++=P3e&xER`}n6yR)2wyHitWExs5ZP+nI}QVMd|gXBWqL z&qhu6b=i7ZuKU&Y4<@*7?$Y|vp0RAnFLj&*dp+e^Ofyh&kPudGchBU#sMN1MM= z7EgSUl6(?rvG&H$M8PKdlsnlz*v^A|qiu^va@cLmx+fJ=Je-(z6SezTrJJ$ypyXx$ z@OwFE;>d6$jHvUYuG>NcC`!iZ5ilNGQF~JSD~XKf;S;|Yc&?eEXx;wz(}BOVlsB5H z|Bg7I@cPBMo6~AxaG3-id<>Q&#@d8v;sR~kJ7X}Inv9SAHviy5W@Z2SaNCZ*xZ_9o zRLJ1q`t1QIg=H!3**NQ|Ao71Tv9KN+8{2$H<8sJK+xb6+K7w%+e{AJS-Y)?`^+2yX zmYuT4J9C@A;kKZ0&>=iL`ppN{-X0h|zjpUuOt*XVP0d-4Y}1=T;ObMep8O~tCO6>R z<#}<2`#(oG?b%;^!0~~3{(VB?8bV$r=IRC(cHP5dtxkce3Rn-4dtX3x8i8O z3#$9-SWJ>02H&Nr58H>wA4K5W-IX~!D;#I7GFE@4=e|!xcXMVNyZ&!i^++F?aL+C5 zNAGiQA~+~#$&k4<5xT1Px=))e|KLdc`2C7c#S@e zk_%K-%y~KTQ#glc?MToGchQJ-|8$R+UW|72j<9U8M&T$2{rHwI>B(NQ3D$4dM4XJQ z6tj4vBemzpIDYQqYz`N>ruEHJ>E7VXSc?@azPi)RCs`|b{vE!+{KqFbF~gMLPh_iU z4Z*tK&fx9U_aa5f*~5B)(5boY?%CkGG1p#g)M+L!mwBfcK_0B|kH$lvWG4+?#y&Rw zoMmtm{w_edS6O4Zg`A&r4JrD#-HnN}uk>?CvG_r1HcBMSG;TQhV=QZ=rBhYUOiv+; zd}R2y6=nGqr#KS#-t1X<-}hwwaHN=3-QSi)s+30YAKU$|FAAQ+#hVNS=;ELZS4q@q zt^yRE$#?FY(Vwz7dHW{943T+Y`S_z`=O|Dr6&7;S`eAhduQbf> zkE5qa2%#-ii7H@ot({h^npRCq26WLJAI^Ru z9M0Bz$RBHPLQa%?qzK zx>z3J{fFYvH2d7V2itQwcNiVN9!aj`m(Wq5cm!%PR5Ucnbv8iYdzml<@}8b zs}$$l$g=tvnmyfKe=iH;&hstuI-)Gngb}WPOyNUr{69|%*E@pm-Zj?fYV&YGAkzU6kyGEmF}V*N3DiXLRZG!5u@K<#8DT@;bQZ z)(*wg4{NF^7r|~HMn9ui`7*TcD38l;5mRoqOLC(uE&Z&a(@em&R`cC)vwK6=Xp5B) zKDyg!`ijIR?FRN9jq=)^Cfu4Y~Qb$u@6htyJbJ* zeLUeR5|&CvGXL*+lh>&YMXW4py@v44#mKmR3j9d@<$T3a*1)||DD_Egr?#K3(Wq0AJZ zR6{-Nq%D`eA$fZ4>!G~UVHFmgc_{&xFxPa`$%nX6qi6R83WhMej86e1!6U>i+B%N? zJ&iX7{}->o(HTd)Bg~&eFqKEz2Z;_RlHeE`MKXWmiavgo*U6J(;PjX2QX2ph<&Mui zS~R^s=a8dDq{7dISf#2*%O+Ni3rk)06qwkWKr#=h5oW=H)) zr{iSK=bD5iTTQ!Arwv-e1@l_O{Txhj97zwIXS;;_56bZ?3Lapm)9-qZoXsCew?oA40B)pzE{qV~tRpY?hKh-Y&B;?X2l$qA=m((=a+bu#a^-o9D8;rE zvt#Am19s26q?)IH<{$J)`k1R2Jv^a{%B;|nBaRy_q>K4U=eW|T+R0taBR^2f~ASYBXhxwAUH8uvkWGYwO zbLSEyqBf6??wKmIlqhm{bTzPsu#`QEh0K1JVRETX-G2r@n;lJ~7$7aOLWY~>>RIE1+QOyJj81IAo zp=k&z`yqo6C~eZkjXqCwDffBWof!(ieeFR2Qsd5GSQbSwASY?cQ>=fDcA3P;ZJ|@f z6!|sRn}Y8zAg?Pt8@P#*8(J`K{9!DU)v+!$g0dLJ5kOI7?avHhj4{E|vOtEJM?b>V zCJ5(NDn>hwQodeTsOzT)VuX+4&rsg)OAOTikw4F>632a0LvpKMDVpS}yN|b2s=aJw za8I!22{qLd7uI#qcgTB*tC^=n5^HZi-PTSxxRXd=U9+%!N@U3}_Ke(@dYCu1GW8 zta#06uawjLuSb(gPw8U(VmFBjGK*(bmX%prMwQ=^`SND6irVvYWc9H-UTt^J5QWjR z;HwjgH7rhxQY4D;u@;f%Nl)s8HQ!I@x%3#p5Q40q<0vlhpl#SO=#mu|8$8XGW_QjL z-?SULw0r!$-tL<}+a=Oe%`-o%%CuxR9UGxa9#4#ESHIf95$7z)Z6;knbuWV+^1;4e z$;K|?ly3ZP^MEuwqu74@PzrH(7H@l^+f6MNgZ3L-oxxxI z3fE>dgl8TE_kX+fZ%?+9Q z3K@C`)Idz#`b9{0gER{#YaEU?xAbIBB?eQXqgvKI`)%ZJ%i28wiXWOb|Lu|#AmlQK zIFbxVJUR_s<11FYM^hP4V0zf1D^)0H#3tBY%$X{b6HwwOQ17VI6ENdtd94PKf%rD@ zdyKx8i)A@%zA6SXM4#H}`UtDMsJ?OE3%mqdO}1`#_g9j>ZjCXsFjBr{LbTkLZ?Efk9g>CQ(m6v*8xHz zD@_dZ6h&Y%umNLkFoN~r?rYaKU7-I9AcsI6ogbV#cE@S~qH&9)iPpa`l1s z`fDNZUqg~yOf`ZO4LzYD@%N9T-0D|%=#jVzN>N#p9iCY7zpHy3v_*K|9k znYP$dk7NB9$;*_cxJ$B-72#5OX+P;|uw^fQ80C($Kh&&v%EKbO#>30KY5Ruz!?EQl5c$Y@C4W5W}6sE zIsEaS1M%%F74dU%_9AjhW$lSAT18h4(9o@q5Ze&CyLsb(Z06k zu6|=2{E^}NMq%c7yEtcRU9t?KXZ1#+T{kWM+(gBt@ej$USlc_XCQ?xp$I5~Dg<`*- zZ5QvTTGVXwr7Y8VT%O87$#%>!a&%Y_UYr-f^X_}HR1-wr^f8>Gm)*8tZfIx#f*|Z$ z8D~IQ(%;fRHareO zTe>5Wa(VT7zfR%!`agbrMq!gQCU$jct9YL0<7T1fWRBrrj(PV!Zk;536ZiMkeoEme z07YJ3E6?z@MWka6sk*PW_1|7bEYH(oNmF+8fWXX#jJ5qOb7^M+36<_d`E7+(vZOoI zBu7U59vC4)+QBMU29)NGZgbJqRmytipb=BzVM@2+uBd+(V^q`(gTL z;XQHlPFK3>y5t~v>Yn>>dJ8<%z7JgLF$}GVX-?OYRvWCIl7xA4oRd8K6-%@lr!Jm8 zbM@km))kW+<7_$Hxf_Kj%L*q`oFehSMPq3p&OClY*x;lm-pjr_w{DnV zd;joz%VqVkJYG~@xgpgPA5?WrO8f}5^G{Tg@7I@Wo(~fV&W#5dVPy4=MRZn+p+~&m zn(Z^YozRy^d*jbf|p0oHF`PMu-xI+m4 ziqBy_2;UIg!0L{|wCZmKPn_;6?&qG>V&{mw6a2has_8)xdqwAwOe#&OE@KU{-tiS9xC%8m19 zP>S*GUfjMrigLX8gq>8`ND-|k-%ns5@9?yvrsS`LycH%&cM@0PajVz!A$Ypm|MO%0 zz`0P+&M{o&6ZEe7dd#7lrlau0NP#o`=jA>Z+bN9W#(CB2h&?zf5=ST*!G#CsS0Aj$ zBHOfUNCry{R)Lq znhS4KcMIn~UdyPGtEizSD{BQDZRkw+W@$}2{AtY`#5_}u@0BV{bLCyIrj6>RtutYU zSu2(4BRhRqfPztI;29L3qrdEWc*iNLBo2CK6i{ec+7pK&^Rqi}&>_-WL8_&)G225x zKH5EpEwwEQ@=~ck!^~USAw@?{>c+0Px^j|uS)u1ai&vZm0pvXmU1FKZVw|Z?3Ub5? zH``0ND_+c5@>r6C)C^@f_9!?f-R|IH>P^2!#A;}0YaXY1A?NfP55C-!(x%JKJojg_ ze85VFW#^h>8vQ3otr%Oh#3EZpUadXLRf?VyXF$c6=_Bky9)lq|KK zmL?*Tk6Unrs_OO)?(Rmr$`=#;nVx#j?i?9% zmz0)5ZyhP#xbj=*iU@nO{77AXHt3)1#u_yb(BR7kq~axu1W$ePn$|e&pkU>>V>Xv# z*d$1t|2k$vO$lPOS1g{^yxXi#C(Q@RWGF28$Xtq3p*S%qjlatppMLrBE815-2EGmH^DA z7#&VXiz%nx;Ym8$1ilSZ;i8vLhQpa#>U|`BX0UQf?&(K6bHytIz0*{U?A`kJm%S96 zN!RnUjnya8c-4covP=tyoXuoi3a04QSmL;k1oGB7W`l%+!s9PMm8sKt`z`&KT~+|p4^+TgwiQ2psK<`i}pwkD9$({bgV20 zdo=hWPG%F6OJpXhU$b&wV2c7Ppkg0K$`utzb<(&Z!7)!Br{YpN6Z7OGk;9N1b=+h< zn|`n0!my%=zd$o*yZ1tlA&EzC%c;~T8;g$PJrj3j-m-Gf3NzU6JZNX}lUoOz+TNDJ z&OgW?$m`rQ+p0D;<09p#c9ll7{Z8(&UjLVpbl@ZsOC$K(+SQd-)vTLU6ypLk{~DSz(|NB7?T>&p^j|J;?y{%UOt zcVyA-pyx=fj`8cL+RrA<=X|kOcl?=2I{cqDy>4f^Xlkk!amSXzxQjoGGmL=Ep}SAz z-6J2VqdKe1LYI4`Lafq}KcQ;l|K{6tQV!qD;X#Hb({lJ%zT?}^6DcgF&bk=PEJjOdIp!(tHpMcb z#P8PwH)q=yGj{*Vpqvv6l%(H@yP>~(@&d)qARO^<#I3?C@0f+**m=D3nLW*+>;@r8n9GIq*Bn=-Cqh}`Ms%2{Um zdPWfst>3z(AB$V#DEq#c#kOLb%b{%4k?0>KnqbiZ>GZhx`m1kXbV$u7#+QCZRJ)1I ziu>R0le_rsGTl)@unW!iBQpF$h~5*v!@1R6Syz+;?4y1jH&qZ=CPBZB z^tu~O6N}<}2^h{n?zG~KCa_35g;YLaq6)k!qBmeST$TY=LQ(tv^rHTw##x8{`F5#P@){bxDO{r^q`!8XT`kilPPwX)2%e7QF` zDV`ZGmSse85=HyjA!*+`)FC|5YDb?-@*jK0M74AzRFYfC&>HSwU|uV)`X-$9UZ*U; zwffBEN4@fVep~*i6;l$K$vh$3_~7*p=6aS?&n@5fiT0lgz=_eED1p}t_7|wC2#&}B zJ6rBgS>ttQ9wcBCi|3X&w9HTNBmZPC~ZrLf;7 z!DsirhBonyY~!B-hd5ufHWL#u4q>@mF&qx}V{@MtoBcbHEe$8AU$yMpwaNS0l;5y3 zt=G1S1B}ge1`G*7v~|gAUIJm^J~AVo`}Z(553I47dI=>ysdvmDaE0n7`dn7EG;XZ1 zV$)&7ps?Le(^ljlklE9YX`IptcZ<`% zV@HS(vT3xb$-7GA>uCth!Z%;Li>CbKP9d zAw98)fqoV@dhpn7PYk2*acW4ZlP8Zapc8`(KC%#WuiCLI_RoAifRHela z^l481;Ge##Uzc`7unkjS##@#gzf)7e*L?OaRz0$@+wq>I+<5*gl3OZj#*V7|Df;hM zE={j(WqSr@64KT{OyzOqYZ6382ZiS{6GO3=9=oyEn18%>gW%BUdNuTN0wKLTzh~UH zwX9=O?4YqLi%f?~K?qOMOJg?8FPnZYuG1oupLK(hwK=yxr0HLwS~Fht zujr014fdB0jC(Ebl9u`yR?CXg$uRs&ix{Jrj1Xtkw_SZL&z%F>_Bh${!k%KdHi{gQ*Wd|)-(L0oUMM+)}s$zeJi~sUg8ltJ5>NPzW4n7+Cl7dx7xHuv) zox#V;(at@Pc)S8Flp;~hcdZOu38bsa3(WA=g@YWjW1VHKg^cs~&9>qe(UaV54?F>7 zVUiH|*9n^+)|_oS%TRKs-@xIkCY4$|MX7bZys0QXYh$K#{gs*ZeLKk4sxS1>uk@M= zqg+MD-q`g`c@v>&PP^^oD-IU5K?GnVooBo(pkOFnGddU`bTp40zjE~T=wCb#`pW$2 zlAbW_%a$^w^rnJpOnj6_29NSrdB}A(TTPy-KyzAxOiru;W!rgRXGGI+HaaHOpaAO8NG~>3&L1!<;QyvB|BM zFNrDKEeqibKUyb;?!HIatu9i{;|t<~%muyGt7+THt^%VKzRMpG&OZM9!)*lyuZ4%b z<7uxX?KQuKl*jKA+i+PV->Nfelwq&**nCsenj3ptibc&fjB)v1$KztDuPhu>ehVGf zFB2|c(Z$(OZQ~J&#G?C?Z6Ci@0*OW_1l^w0HUBrvoeJhGR%JJ-)OQqI>dNnRrXmdf zBtMya>X7VCnz4Am{_K#+ajrmBSv*)gE0n>tUq9C@DvY*r%6H(@(v|c|dXQLJFf_qu z`BvD(jSH-U!|T+bnR}6shmZoX`H{R^gW_4w+6quL4qxdiRvemnZ?@R+jy0+q(q@sl zhKdqgr7o-Lk{g>;o;Lm z4MSWyn1qFv#Ys5>@)b5dDPR%kCi=@S?leoiX!W4f&;c+m%n9=j7Kn&+&vC&OP0 z($vo#+uFu?A-Xw9;_ZvG@cLf5()rnFOA1C-Oq))#5jt134+VJ1zobLQtau$Yhtu)u zuQ?Bd<#cww+Ht5!`>M7K(cgRzYX55T2psXSysnyW=uGUa>|(R)Ziosdj9p}JH-F^@$t`>{#Z$^=kWN0h0|Kosq0Uo=>u=) zGsal3r-}2lrDdC!0t_#VG`V&8L*$8@UEuHPOJBBj)8Yg!cX%L-?mn69Krm2M zj+E!s2|;T$6eIwgPUtRUK9S~HQCiaC0!iW{%g!3tXoP|9NAkv@odlyhJ|GBorssCF zj=}}4;~SnC((U(Ei5UJy_EOO93&;}Hx@IgY+PT)XnGV9vXC1!oy+r$11~J-JFSpr8 z#@M|QOW?#7-n&&_D<3)J)+U@f%m6e;aR12is@Y4SXCF{`LmMql!=)NyuT?dobppk8 zWP*qxUEgUy)gKxR2+|bBZdml32Utbf-)MuY$V&NG`Yu~j)tPXi*n#8FB5p%Xcpu94 z%Zs_*(E*4P-v+XZ{7t7119#iRGUM+M54l*kd^}*bsCL!lqhyY$Yig!sBAO?2;N z1Zv;K41^f=_X*y~Rd02tVa5IPQA&VI0w2%R?}mm_Ene&C3Y8v8F}^>%A^7J)w&ol~ zyI~X`Rb0xk_p)F{9iyUa_&7bTYogUhBSxPw-OzReMcc3Peu=cuJ(CT>sr<1X;=)P! zCppGPvY@U<^vOgY^d#~5)W<6)bmu6Ef#!ql-*)K zfR(xz=LSESvOOUq{2J98=7yxHiKHB#5n&5VOdmp2h0pxWBg zx9SMTI<7}OT!XF~7IimKuKjy`@jhk3(OH@Ks;tK>lmQW`80~U-eU+~7x|hCf|G$7p z@MGckp~6@WNPs{T`7UrI?K6WjPwEdtk>rxSo1Hk~mQIxM&Zd_?6z|>T8}Fh)(c@5x zL2iHUl`>_gqja7qH$0*L>5UOurQiX3{SR|~SbV*iL2J)Vs4E#k4rB57>7#Skml9wA z(y-<8hATYh0`lzpkQRHoGkXgb5b#Q1iCwCg(ro+Ho~DtgKqf10h0hvI zZZ3L2aNuO$eT@u4`X1N1_kPD#06R=#NrXu9$O6WiCAwwBR7|6jtSDW4?!~4P4ztY2 ztiUtp5HB)+X*|5x>%$tmD#DM`dcE6MAv~vPswXLUn&~|pY-Q9kys+A?t!DpOf7D&c5<_Gv^0Kp!4wy3gs7hPKvIls)ij0xBJebJ;TvqvH#4=O~z_0speWVd0h?Gam-aylGa!us;ps(cp6-l@nxRJ zC_x#%?^nomS38(3_Un>(A`7SXwiE@Hv4@49Q)TsaRI|~6blTOd$Re!-Xw+wnqu^ls z$*fV^?OaH=)K2rRSiNfobc-KuHJ^x#z*Yt<-YD&rS7PA?HEZzGXX}-cn2ydOH*H z2Hify4-WDBlr|WdUAsQn(X&M2#BzM8^YGXOKKSmCaV~gva}oi#Ex6E=Rm@%nCV<||%+}&3O^$m+ciVe2uJ> zuD?X^5Jc(_wUgLeb^~s7=Pu@fKQyS&h*nqDlnuTxvK%la0p!@exVxE_l1#PWsw%|R1~6c-d!=Yg)zsUu zU(>_`-qjnVDISm5mW}j9TV(Y^w(hOkI+K<9qXwNu)ot9Fy_>@PY3QRai5rFNjL-C- zrBJ~^j=Glxz7A>`uxu;Mwy1rRL$K-EzP-|vZ09Zcie3~qNJPX$^n7)yi?!9{OK_(h zFk!yvLjjE165xy9lkaeyNwZ`PHEl7dnN)7VY3!ET8u*wcuWJLUmakQNrN&Hk$g->6 zgF%(&x^hZl{4SuOwY>I@d-%_noqZ`7w|L1ohBAPJxJxVV}?FAFh+Aen+g`D z?Qd4u3YrDc=c>X;enV_@nNGrMF&v&m`lq9!B_M7QbxkSRG}5BPT&Duf@0ZDo+sq6z zU+0Q%KULZ4Z!L^*t|&Rdv>;=QAD^~Q>ue!#P@^jwiQslTz^Gl;g|=GaDxx&3Bo0FF zcgK)eL-c=DqX#(3 zF)vba0TyX@Q0$#d#%25)>{A;vs;12|#z}t{Z|kw3v{K4zc2LotjbG`Sr+GP{T>CXJ zx6v`jVTDSSzt7_*$s*}2zC@O1hd8Ju$Ca`#$0$gQ^ou`1$^|6?#io)dTy;@Js9Rt4 z+MX;hm}X50FL4b%@ekKU_e{XJyUu`Xu{wC)=4#Q8f?G{ z`~NQP{>P138N0)wqHVw=$RUc_0JsJBym*ne5bm>563CRDIp+Hs&3_bEbm{x2<%Ru9 zQ%ENArWUj@$?#VxC>`xXJf5TS*3kGl8_knMDJ&7RtDS%APv*BVn|?lBgRW`1S-TX=v$89p9N*t*yyV}DA(@JPZa#SS#cZ=r zB;0z>EvGiL-_ff5>N;_SP^)yKD?x$Ew#WZgDZNg3-3y$Jjqwc5exAo0w*?~*gC>c%GM?kO3 zYOc}ctTIj)OOkyfC9XxuG@Gr@FZ$mi4W<8Yk+#guDG>69PSOy9 zd}-k$`-^|kJ4I7F(l?=hBSUHBu@*9h$5&65#X(LjhzUBCE!8P}zY-{J@D&E?Fl@9h zV&rwSLw0vmy*{pw3^U@eEmpLLEql7>7TaCT;|+8=YoKjm6NemWA{$Y?ia0@sXtaY` z*1NH{S%UY2+uhO$jOo7W-e_&bxY!E19U(;0gP~Dy-X!uzYa74{3rvD zi!-MldFc^!c`Os@TCau2L*tY|3NMZe>Ba!%`>_k%RTkli3A6P{=M}NiRt<2$@=5s} zjNuF^D}&>Dl-{I+rUN3~)cDG~f|XOmlIm^JY<4B7t})9AcXGVrWj>>Yu}Io=nEfnL zTgU+qmWvbTqtc3ZNo>HUi_zSYTmb!|9dj@fLxz85b+S6rSqhtxpHZ`qG64J?_HCeV zfgzwSXwDZ2YZRHf(V!WQ;JAtqY6pf{VNX}}iYIOFEB_e34_yrdksG88p!aE510p79 zUQbBx@VBurj*%F@$I987>-`5rs72x~s43zFqTsc_IpdF#uJtrEW<$N?9cce-1EJr0 z30`Jl&jhLs0);-*t6S$dGfW@0`~$!XpC7|<=5DqnfE?hUmw20!eCFe>@zOZ6Dz_SsPh9`tetK62;mbCYZfV|d~p*6G<2&@ zpgTwYFO5b*yDdn7yf5h;{UJn;<2Qmia_R8N%&q&pNj@s z&NsJpeSf?>x@psAYVlFsi%+nCUfp(f;Me@5yAJg)Shq|ym@+nNPHY5j^g4jN`Qg|4 z2sn{Q?_bG0A#{Hg=omPD*;aWNW_Es_n_zQsrgs^ZwK)~pM#`6=%b|3v@{wQ&V!10Z zEd#h8FMWGS2KEN`<=M$FvWnPqX4x3|r9n+Eiw*99cmQ=Z(|ClLI&4q^2)Ul;2-zkR z@do6S$rntz?y(U%3amQuLVnwxKm7fvwkLH>1|>Ll>=MUai#x7#Lc}xtbLyuxz-yOu z--#=TKm?{nF$iZ_vl8wo#F=Yt$m@!0c8|1HVfiQi?w?%|>;4|0g)DAEl-rJ*>4gcD zj6}6rzdO z=cF)xpC^r{)#J`27&IV5BuFPy)>kjGx4&|08Ic=vD6@*~o3`227z2S) zJ?Va=>9A_SWIff8Vwy?I>q%woZx4T83JJx)K8%jBGZI3hjpJjag%cZ2WCYzB{Gt?zPJ2TSufeZ!me{~DQsF%gQe%`7oC|?_gAxD zx#G1fzvBT>ZE$j=bUbdk=c})1G->hPjb3N5=Yi(W-`t{P(6cte2Ego}W?-)8D=Dh1 z47=?5F#4Z&&kF&|Tl|HgVCz4>UcsJA{co-@%SBg#r#>v0{G#)2DCEfiWxx%T!Ya9i z-_X%-S$;Q_BQ)_jrFvU{w&q}>d_TS*M8;a`RA%1BVLAz+B5_-3!LEo^2}?C3eLaAE6LVN3KEz# zp*A+hbs2f{8dmmo&xkK{IHIRvKk7&b0U*kOXQc7A-hOu!!@Q${i^KP>DE1=i(e`|H zdQlDa-{NUb&~4bc<`C_qFzdMnn+`ANr#B$3G~`%t_Wx9Ykh-_FY$07(jyxF zf4l1;1*je@qDF68h5zfd)=h-}rmzXTajzAX`)d0`wSyf8BIo=sd*9(Z!b$kl?^6@J zy6>hL>t6QOR=N`Ji2loL2iRo!;@M_*r^5rwBnxA@@_ire4Z_=KvN-z3X)zN6BedV{ zqr>)BLzx1_u=)x`_iWftLxBTd0?UK)$Ogs0)S$xPz6hnO#@Mr993bf5!EFAV2xSfX z2Zf9C=kl{@wX2mYiW#MCwR^VNiYEL^;!rT9xsdi*UXAFOKZVN~_poK@$b zaz%`bNrLYJ$hw!30x{*Gb~n2~$`#_03qfYe#Uv_MDiW$2E+_zFJ;91;B1ju`3j6zb z9w%E~LE;!gmcz5gKVKF&bVBMUs9n>CNU_|B( zn{dq~-8^0bHBYrb-*5My){@R+@h#gmmPH0DhGu@QiQ^s-38ibmWVGQn){fU7WJ z=2PmL75mTA%}U+KCPFob!5feY<`g#G(%7$)Cr_GFiqSfk?f{|1UxGh_*Pfgyp%8Jo zPL+GtjjjLm0D=DuS+N#n-_wI@1wZkE7kb3Uk}+z58%^k+_8V-ihR7iDU4Z0i@`Y>P zFL}3gmRNoaoY_VF_z{P(bjGgQY1hN|gHpNq3#S}iR`3C9Y^;$R-e7}1?3dR@v;*jq z@R=vPG(JKd@YzxIMo~U`RR4rQd=x5)-eY;LBKVb)p1*o2cxktTWtLH{k?j7 zb*(I-qo@wQr*c?M6G@&a{ZqXPl=q|0@BU3tp+rb#Q?ZVbBFWl&0F*TQ(6sP*Q(48l zv-C%XirtYj@=oT;UxO^*M7qkHpe>0k(xQC60wn17v5xuXFX~F~VaqM`#Ne>ktTcZd zKHXrRkHRr*z={J%gcABC_^+}#0a9~ zhkoctiYRwSnE^pJ@V&NF&E-9SBv3@(ECwEF2>uHU1AD=H|<@bhY<`E6Qlj{lc+75SfaRcKXR^vLFNfOi%z zFMP`&l`*b=aY$WPMV=l#Xu?D93^@?WA=(Wz1beF@*Q`V!{G>Rz6Qv;V^NF44eTQ_1 zpX0+n(F`Lt zPQK3<>HThQwv!PK;tVk;kKNBkUxn*v%hakjXAcg4S+M}Uz3*5N`p_xPCSzWDMgMJe z-j~rFKhAO@@r8XE&FL$jrQEDpo~XT&v>oCVwajH0Gtrg6xWczwncaq07IaAJ@u4U*3S>p#{>ry$S$gzNTvWa% z`#X`$1ZB7;DEli^d56|=DR}Q%TMrk0V{>JkML#6DJu+kfCPU!X~K2)Ke4Z$siEOY6n1^#`ubS|Bm`=$(Sz=< zMsOe@Y|WW!fI1_3ln4U@ej3E)QFmq=%Zu?MiJu85*TkNeI;Y*6I1Iub`}gD`4R$}% zQhn7-WL;p>*nNB8mZhyG5T{SUnBD8C9J^DrFU_n9M!1K1LEo3e1L zk22OGjB|aBz?DB)a57^(b~)#DZbrWb7;0MB)sl|PesSM3`$fJaX(izU?>dbsl__<7 z1!8@>8V@78`$G3h@ME@=anX+N|KF$kMbn#9gvDPU21+*3BYunX>XUMu~@W zpE+(jZDgj+s4XP`NwmoJ1tvPd#ID8J7~VEd13MNK*I^axo2@44X0o3IwU3!w;EJkn z!juIUj7XfiaE7qAm$-`g3SaD2d>KO!)WKjz_}SnPk17D((f@Q>Yy%_$SN*qg`qI?9 ziSrVZ%ecUbgNXszI0*UHVd258LmQi`og-q>v$N?&iae{ENJ)hMa74`1*^;1j4>wELeJZo0`OFKRbOiIs_&^yF%T|4wm9=l{>u<#R-KS z?mCR`OlJ6wnCk>&zq&3OMRu6t(`Sh`j=PQ_=>lo(=-o5ErwV&aAw}+$_FkkA$AVfs zy;tq|diBpeBISmMQv{FeS0v?Xk+EeUvjU@rq{OLVe!DY47sAA?(pb>zxeY_ra*J89 zDOz%cwl9&~IA|gCSHI>wW^76dqBNir^RcCqzsSp5;vt@8^nUqy5^Chx7i$z-rGMC6n&rny0~_Im>+1>n&h7MG&il$b3bQ-l_v6m6fi1+R zNW;9lzWCm;jG=)wawuI)U^mZH)(Iuf2Nz^|>Y~wQ?j=S4d6Au3arN{2oDSxeX4-ke z0B_XAl1&b0)g4`aKE`KXV;dtne`11ih8t7u@A3Kib_;FP^fNt*rFv_DGV^CGEN^{G z&;QELVZ-tXMdU@W+a4zgQs0Z21qDVa^0YkbBNXID-d3_6zgn@0m?9;^TLs72Gm{!v zYf*=&z?s88PmgiX(o)>#_JS>Yjxw&*`M=@(v16#BT*0m9;11UGa?y>f62cciTxGxd zyu-IoSX~?8&{RXmNqXhF`}S*gn7*@E*z3$6`YaSSHHK2LkoRO&#vFp(N5Hc{83U`9 zO~N5$bMSeiJdynDu5j+zeV=>Z5posvfY0cyD#&5#K|xwaIJY5+zU-G@PKd9_%I_wK z1@@j(7N!%U#RMArK3}Kq(t@4|^j!G(IjwI<$zR~c{=Qk`$Yoc<_kA-VQA?ZjYl9-cIdzbdRz2L}j8|#O4+kqp ztTW~Z8O|Jy5lL3M-ihRT6W)~j@;|J;fLujpSM52=JV%Hr;RjJYa2Rs{6}F&?fYfaF zvc3YlB8qK@&y$K8;-jK}w1#bq>lJhR4cMp^pK{dsQ{y8>8v}UDR1TLcfwjjCOaB|~ zx_V6u^HjJNWTb#Jl-9}q%o@jV?^~H0QWEgO%i9WB#~b`{h!Km72=D16+>7}OWvCCc zRJ^K>>`TT4cHOYc_R*nbk1`dJuAPS8P&F0ZG18PEfk}vvXw|Rs!Vy{3kS)%=ChYTR z0NSy60x5CV1LFDDb2Oq5((z)TxEVdz4Lhvi2i#SnP zwmqF#tbe9-tz&7m(js3(nL^I;;gYj`|yQpGZ?O5QtR#II|u4eVSfhKz0dJ zM-^5z`9+`C1)^S(_#S+ddG>2N-!ur#nc+>xlrLKYx@?HZn2-LO^{te_KrLa=PN zNT_$f-~;u!X0@)E&N7epXVwNnFhed%8wGaI`CC0wNzoh zA8He;i%*sogCCb+v(h?R%~aLkgR%?$EM_ zTjj+u96j8)fNsOioFg^XzJx$lnY});kj))+%7zbAUU6Cs+%YaDjEqA=48ohGt!FV2 z^C_nM_HG_e(T0VK&Ow~W2zCQm3-~FRH>3Y8Km@Xa`Vq_AffE4JMfoaQ&mOVheJ746 zwy@T+}^KwGHFp)4y(V&9AOa{vcsY3hD8n zUS*oLRAx9D-&dP&W;TD##Y8burRyeAeKLeJ8W6?ohNump-@<6Qv>}e3zDK2UaHc+) zpHGYeP`}&f@QVu2JW&0mtC~m;>JpPLCb!vj`6yqNAl_r8$6JIF6oJ?mt>r`JVp*rx z+HUnak1jOr;G-m&Fj*K%I~xp|1R6AcZ}&}X|7?w=0_#nT$g-y=;0Fl^$FOU;P0ksL zIwf~oqJxj0wkh-Q(FB8{%0O0Z<#qk-RV*D6EIztGCBP29#NldeE}>*U=u8z3H@M#XXF^0%|E?2N|t@aSryo_Oc#j|g1-bNBXGMD{kBON;;acH*WB zn0*Nk)dO=mB6~u%=Jp3BHB4-@$CDey1Q{fqu7zVkQ(dT}%sNTLpn-XzUopoOab&Q8scTRCnrOU} zg}bywz56q^*=UZS$66SLX|kCqV+zxqGHb3c8e8ngMWEBByg!hCf~za;pT)Tztv3)X zim2E`>YKrgt%mC&4u?aJr~BeJ%ak%Z*Avjvh`p9WOE4 z1ZfyY#S5LiJ54XO{rUgH(^o)M)pgNgBi%?#cXum|ba!`m$5lF{yQM_ByQI6jrMtV| z=KJ4!XAEVCmwONAoW1s1bI!G9K0|`xJHXDx^wa1ca)OzgK=>%6GLH}ScgX}1U^Z`n z#Ltw(FNvg7sR31h0e5aIS~)`S^FJWUc^iPLW%k;6&=UT6!qh*STrYqgPKwQg za)8OPg(oUPe`mh8dYL={z-z^A@>QKgS>UA##+mtzZ1bKyy&7ia;wV>g3O@H3q64@C zDE>OLkY6UVaKQmio*`|n_>-~%n9TKByk^0PgZA?$d_ZqbB>glb3K;!n$x02!m@;57 z5{D`R_Ac_;eQ>dsHlYy9k2DU5 zacw7WK*gy-4g0b*Csm&wo2sjw2t4FlbUyoOUQLM<5*X9-b7*<@wmd|}o;^xbn?JW; zBTTG}Q6S>~Hq+<9UoX^oqymO~z#{4hz9Ez%smaL#!OX zp%4F6zy%}%CF>)sTtR*UKv+L)Z-ATKO32Xc1MC){rPcHlKzp%Hn> zGex`+n7jrWxqE645T^kC35F7ZC%AM5fG&Vw{9Zk-v0QO5do4s$p{&P`D`oYH75hiQ zF470oNsP4%(2gj?W8SkL2(}EeNF}G@%{J}Am*O-B2B^ZAJ*+g{^if}?`xOuqF%rYc zQf}HEYs2P+&&gfndUjU_z)fgYC5v&>5(4!gP@L-iMVzQAo7q_nS$77YHKMFe&{ar* z9s`I9XU*HmK&dh6*fDhl1%;i~LLAbQ1)#;ALEy{mUm$YZvZzMCcNNd%5sqN zQLF+6fIU*=fR+b|9@zc>ufJWtVD{fX{Xqz@-CSZ?D9aW!3ZN6VTy(Pu~K{ z4FJa=DFYB7U7BA32*JRF24Ndm&iJAR)a`9xG*KI1{Z~ZxzvKS#5~ux;02PpSov;d; z0NLUkr=`Gf8MwERe9=vyR}YZcqpWZ8{hCWbPFnu@_c>vLb6_QSQ|}o_!7Tl zw>QAa>2hG}&F0G#_?ujvJ5t~)!e*<1oi2m2h3oytXmC`l$5LriUibs53U1Tigg1e@N%aM6M0iPx41^j zd4KQrIJ7;siLdMGn)vkOjf?OB*in%}!B>Y2EheKJ4YTU#WgiKumO89FH^)4_47IUo zzQkCyO*CW*CJ~22Kwx5F=^PzJ!o$PsT(@INg4=aee@?kg?K1>+1b5qtRF~|6Wsu3% z%4abDdTD71q+s*&^G{nk44)?9&zOx44-fB55(X}1JW{c>^WVRJPryzV^Raf}F@oCr z+l@zi5rnM0edYdIC!ZUY(P(SMy}1OE{#a|R2Anjb#5@ht+?%Dd$G$|9V21@f)gEon z?O(N;bUvF189aylJFtmM8Zri-XWO_`r%4Ac0$svB@jIjEy{nON8yG<-U2vRWby~7b zj7>KnVJERY3=hZP%NTup-;fy8naGdSZd@5n9oxGBd-357Eq^yzyWJoe^JDWXg|0IQe8W{S7yt1T6ky!O1u|0^QR1Nnt2_XPG5!O!eE3ZsS*vzLLwU`M|tZD zS%be-i%^XEXZSKFs-qjteq0u1=!m9DBdBjtS*=kEm) zB}XHxJc^sC2YwiKz@ekCLyw$c<^ z$xvq`@tiW%L7LPVbK8udPP6kmvy3=4;`n|fI7hRB^#%+pqGYj&LkmGU^sHbwK`$vA zPc>z+999EJwQ}CjHkk^=s^cu9UY2~!P8#~^FXed9o$Cl?-0V-n9M&gV5S?6SVt*h1)l_lqtc( z*KqLgz}W3Vw*FZ>Rhk^fsw(P94H`Wr-$`nAVnVRl3*IYT%1&dOkGQ+LTix$1Ywq`= z>C^e-tncc@-H+ER&*fK-Pxr^$FMrx$>2vb(0#RjnqJT4U-mJ;Tdp>v;Z1|STVw?Zw zYoFiF0(VPbe=3wMw|D6-!5h)j(e4jRi4mG=*Nv5G+x`*CwRz}5VZK>6xsvR)V+6sM zYuh`G*n!P>%(%F3!kEMlHf2YbwhLCZUWUA-Gld?%z_|?(XTSUk$*SHO&tDGqb<=aPjUZ?Z^qxzc>ILOgV^H6<|WX{&{~>Qgd?g z!tmOdD8XiuN%~l;&eK$>Q{|lBga<2*WgXx3@-5P%m+!M{Uc?WR46k_5D~8wvMnizM zUl{-P=RM#AB&ehWd`HBg3x>xa1Kc;!pD&^@8oV@jH}tQ@)lx`&~o#$yCi5u=EOxoC3UxITkw^*@Rh zj`#C+90!Ky!c4cX(+&?EZF8V-IevqF`steB?Fa5#tAMaB8}wdQqvPWiEvGR)do7UMJyS2 zS=WNaW7^9^7w8+#hKLc0(f5mfcIgZZ*^4{1OX_|xt2D|j5? z)d=GyeW4%;5kg#oqA8ea$|o*PgPeg$SE6KD90zRfzj-%t;FHGkDbfzD-ZpqCnmd6Q zR8hoK+A4ai!x5jQUdYTdxu74Ekn64_t`t8-zfD$Wpc)}^2vg4gTWzUt01c{)*^CKKT-%|&T)twyIQ*S^ zL-|NZBvM#jn^5Q3ZvWP=$CjQci!*{ncz8B!7z1AouLl*yS$Bp;)G3{sIx+;zFLmHk#mLU|{ z06JOwM>bU^f{6>K4W~v~r7?R%L;H6JY&g4`+L$+D@NVj(qteB6yO+Qi*5zt;8|vvI zaOJ#nJ!0R@1Rf3!jt^ce+OK2vU-R{{!Fm|`Q%>W#jl>u5YlrzuJbI2c{op3 zaO-crRV1Z8ooy{`;PndVXNME3!?h!Zr*(uk>vRYs7OIrK|Mgi|H3kwu`?{y*;x?{4 z1!+!u-m+M~7m*pUGRJ|Q=Jysen*t800*+CX&2@YO0d5k`eZ+A!Yk-JBtKRu=dw%NP?C`Uh zT}RO<`>@Ee$~Vc94fYG4#%c+5g=KSX*s@-zx0L5e#b7U!zi&%XF`j%r+8ART=20BS z9u4`RyNgn3yF%^E+_OaS{!h`8(#nZ8=ccsEX;03!^XqR8d*P5j^22p=ySg@^NUOmb zBsH^3-(18rG@PBC^OP|bpFeyM^!37TsTC;N-*l<{LluH;tz}YB;jywW&$Fm(O8!7qSK64 zhNBAHQv@R?XCH{!^*3vwocl$U7xuy|uC50>4Qa5mE!Whv_eI0JLps@zf1Q`dZ$HQM zA%%892jti$Gz?SofZN zYnKo_Zzt&;1{r&HCup*XUOcu^ITnv;Asp!~wpp zqM{$b-L2CsX&ozgiNGKRn?wQoU|{T)Fc39l+-Jx{qtgORjUj2^^Fzza%Q^39v>IZ8 z-CVRm1aJJyUIxj_ZvP98f6VVbUoE-l*nv^KFob?Vc}zs(u;S)HePRO8gnk7D`iHKn6d^=sT@_*` z=M(qcU9*akAeV`vkM~0o_!(#*AQv0I4qhNaMwCT3pg-3yCr72CUL(4eAAgREXmO}) z!OF;oj_NP3=C@9HdYu?T2BOIHLi(-)NBm@ugC`Z42)P06y}tovxw)gf7<_%Mrjq}s z4D<8>MHWJmd_up`ts2ezkn;RQSw*GEy>Zc+D9P>4`O_d-G$r~_Qsykp{NS;^KfRlw zw3pOV$w6e}L`lQJ`uRjuQw`w5{HfHFev)+V7%Dz%7EghXM%i=hN_xsKpb{Y`7BFW? zUPqGv^CjJK9NtCsVRp@*mdQ>{XU9@2l^gTWV8gq)FJ^%qVFdJGn#xa-ig!k1F``R^ zXl9a_JBj-l40(?Xwjv3{QYnFx65>)pq823+Zl?RFq|{|dLb$iDxzBuQFmVkaPkl!w zDn%vqdI53pNyrk$oxQF?vDDS^kK=PqpVC6AQ&lomr;jVkck880&WX~7VAZDQwXf&X zev$~)%cG*M*2yXQO1PSe@Y5n+~_TIlXh5P2&8ouJiDC=GQ6Zj{)nB`1Bj^p?vBzP5KkL=xfs~HZC5w{oAZh{!cx{+ z{7orM3X3PJn0`4d8Pp;3NVUa0!i(j1)Bad1231M|JB6V-pj^)^)4z z?Z7n85Kt)E-EJm?8ZzA?UVHs3NWK{$orfs3d_%R8}mij4ov zm0q|6skEyE74c^(vHTQr^FwAW9s0r{KMmB_!UFX$=Qb`-Bk)y(LHsr8$ieWXy_!WJkz~Ef8CtPe;=)@D#K>*(v^1!edwM2QNf%6CPTK%EJq@1 zluI{hedt1LDrniL$eDi%BGHA5RJj>@IJ7C^yTB_k-q8H~ zET)OcHYf|?#pO;8*%@??FtEnWJzKMeqnNl`!%PYq(1nb%J;N4-wCc_$6j_0>947SLJhw=A+>T|yC}%F9OjTi>B{ z3PgQXZrbnSIjm9AL}_UXGSEZcTiqSy2_nY&f_gC}^c}O5EM*H6yM##S#^-#zpTS1d zou@ysmgXw)sLDjlHs{D}2MiiR-@#%884bjHZJhp>LQ(=W1xknpV^pDzc^&2{s#*Ckit*}eL4&ep?E|!2wEE2F>r!3~ z9?X`U+@kT`U>rUkk)mFnY56DTH8#7I`N}Q>1Aa5gh&X_c&%YmbP$=>Gtpa!hvm*Uo=Him^RQ4FT{vH9@~WS@G{_f$OAKo5?;Aw4 z3^YXEi%3%&8_XAv-C5A!-U-q}ijb}~{$)96yOoTBwJV7NBD@q*vZ9Z6_cn!Du?k)H z^sFKUCVjg>rKWjzsO;N~K$rz|*@cGZ3H34{0rYN(Y(17(ZQMJ3m+_$b<~Sn&+vu z$7eUtk18x^_UJ9HG!s-9kN0^k4srW{*bk|+y9AkDv=rsK=<#>&REN7;-9^dGS%eEJ zLV}W`(|u=?+j-~B&CT=8V!G?v$FY?a`Xc4RS;+Ha*OZFRqfBtg|3VumOG4O}T?I8A z$AtA)yspu;OH~pj$;v_g9zvesX+LapAK_d7G5O``gX5dxNRz(1yK{XEV;46%clUOJ z5|$Iou~((RIB@3Hj=&*&yRKk|CT#SF|4(2TrT_!F)cB!A&%RkePRhsqrLnEt?4iO0 zkTErI&x;$`{|PDBCMuN?CY-snuSbufS{P^n1@qr&;Uc2B&;8n1tSXoXL7iU$7n zJRDm5J?DuW)8{vHn9O(l)=C@E3ZMN}KRDbOaZVukDe~DS+=ENyOSBPi9a@?D@kK&u zG!p&&y$6y219G?lTgt~f?#m}73#aH^-19qMTD^6XZpa_{pt&OdUYqjj4T8bDX}btacH2zv2y;0^Iq#hHf%o2GW1e_)WwL+jlNg;x5^YM3M9+oH=%(_8@a zTVGrg;X_z}q-aW@Dbdh_lfW1`Gbcz^vHM}0c~03bQC{y6M{tnp0?;uz|G z_M8_dXZe=+wF|)my8!m0h%W%Z0N27onN*&B;_iTil zTOZWiykcSRH|^0ODjxuxVy)Ad9{sv178^jN<8wq`A>@gl)BBlb3v=P1-!-V0T#E7d zcNk~$&$<^&i=aR&jS^hM(8rpwpc$$Wj&7zn&=$U6lw-RVJ?r+-!&|#oPLX}+SUa#g z+X@NTh81oNT=Q~hxo*#$@_z0Optf*yuG+sGh@+ZP$E*CfXwClS&6}S;e-3iCywoz} zeUdlVbDX36)Bq57Ua(>Z!1q7`y$N{gr$)%}BOLlE^K7@0Ozc6vc)k%MQvu*Dc@-EQ z9%s6g4=sOf@55;in^PKmY2Wm$R+~{K>c?&J@LUOot6qF)hSPQ1BIFa}|BK*dvgTlV z8;3sjuRebT#op$x_2WvL-kAIw^c-l!pWYF^0rcGl6Vsnx$9YV9qo#BX4SNlpjG}XP zELpVn%PwrI&ezv!KS`fChlFt`hsHE-KdREJCt4!v_nFP=YKx0pPOWXAoO;e;=OMn_mD>&HF@5M{-uA6XKyeNoDQ@MabB%Jo}VvyM#fBIW2Cu)`5$*>7D0?qEmqccUG;~!!o$9D(NZ>8 zzQe$F{ z%II|)I7ZSBQK8{S#(UqdwPW3$Ns}DtmZK81-Gh%I&-ba$^Ja^VQxiP5o#zLDBmz}1 zz9yL84}oOyZQgjcaGDW!dq_P#_gsd(Y*AbHKNaOD>yhK z6GoVRrR-ie{_biuy?W(~Jhi)9tpi~oeM73pjCW|?8jeGFqtgk=fiWg;2`LYYdw7dY zh6!rPL*InvMP6n8fDsSg^s`*h(&j*d!?-xP-Nl$_<@3`$GZRz0AR=#PA9=>lmFLe>tsDu3mglIl_l)*NI%DUcG0)n@$J?N*{)spG2&N z7$XFxlsIWW^{drwIV`I@sxkF~9Wl`3Rx_95Q@4X15h~||TuhipYO@MJ73{hwJ@koF ziw#|>LRXu>qPwFvq7*QJSR8Mg`{YBI{R91Ln|@NF8CrL zh2?^9bn8nf*6A1(N}&K?peF=Hf8)&$0t(>5tnAv1>{ zX_lukqi1-~!b=%}{JShMRUS*BZN<34P3mdnKZByI1H5>haJGzGUl z)euiVGqWYEj5a>Uqa6YJik1lVdwD)Z+ZXaAPvp`8@nYq?!tCN)X*Qj=F&|T=1ejt3 z1XcrNaAoHgcs5{26_u8eByG;kRl0c;T_mpaig;;w?*23i-4js{lmBWwAkZe=OG0s62_mP--#KWIo?(| z1d}pfyloKs?cl0e63L}Nc4R@_1|3x#45*{HCE{sT zR|5&G=dPALj?2foaiYY!RU4gYb#B@*p(3ZSYm2!)ez)==uRUOv^9 z!#={hCCkt`vG5Gf+*S8stv3-{W zxwPWA33kVR_ybGkUtWurufo#h*EHE4O2c-~y)FzXMF->0R?QD-8#=OneqDwph3V^; z_CNYF;tvNHB}y`fmOZCEBU>U=jX3WI?}N2}LkBM?r?tCl-7~a$aW^+jirFp{#Bdt- zvCa-k3I1WjK&%j4Q8B3=`*&2WlQ&XH2^EOv{K=;2^gVsk8QQkzb7ODCL+$T9gY8%a zX}#yi>mL2`lW__9p(Mz6`>$1=%7&n3XLaW9mv$5~I@e+qtV254%SORDHto{)e}|)P zTPI1uwCErlHsn$sl_+bv&vbrL&053%SDNg1#p5ey9z3b++ltnH5;CRmXKx3u>zNOv z%bpEwt2z$)8Ti`5KlUNYMy;s0Zv&G{Gjh0iT&3^=7doW_S!TH4KR%r;P)XF5F_raZ zUEc<4?9t4%xBG-8x4tq*Z^LUXEPqbGb-YTve6(pO3#onM%->MIC2v@F)&xWT+$p7D za?TUqN_BpRibDQW=7z6xH=f|S+J`O~FjKbjgIXWLB}tr$vG-pAHs4m9`{-@96P>kTdwG-@)O9{e>O zmXwOL8N2)AsPh%69+=T!-VJl@)37bt3TnG;qQ}$g3W`60nwm@nJrOCpF{GuV9go6;x5mC&mM^n7YVriPp%pDJtr z7aJ(BZ0NtVsQzBs@7^pU7H5K2n}$1=_qEIvOPL7+@Xn?q19C9nZS$<+UunrHiIBC! zAqLn=ilWOZ##xq#rSwRVMCq?KJ;fUY7>K(+qyb7;;0IlMIa>3Nqj;{%1|g!fiF=yw zfRtUSs;%e3l{S$@EGf!t zJFZ^L0opA1(JY+F-1jYPh^LBRz-Rx7E`7OSfb1`cj8gsjZ(8P~b7LalYVvQGZ1$2Y zr5%EIp;R&lSP9rFiUtf>GO?PAF#xZIpMaV-F>J`v*6!v7stb-(XZPP-4D+m{0G?=b zhkehOl03#@?DzIoA1idqXTsW~YyRWb<%hma)*l0w$U@C@QPgPV>a-kq&R!+HPju!l z`hsPYE`S6@_HS$lfqp3<^NC}Nz)_RSpC&a_)BAIs9rg@%XQe-&l8L37zxE}f29-_2 zsSfm4lJqQow9hCqj69QtVJakFDMrJl%t+C_?%}s!=V=TwD0ae!0;gz{3w6 z`u&&gim|1&>H{oWHO~Y<_1aafXJ;6A%A`XaqD*?9Jw(iXHb4_=#VIB3?wgBTB33p zH7K5ZJD|GbNnwiih|k7+cDr*$VK3rH_S$-kjr4c}rAT=Gx7Q_!qNN)%UNi4HGgH~k z@xe1So~@;9gR5ER`1jFY{hBogxYcr}iFI`@FUdIWnbbBtScqs~bA5|ilGo4jHEOtO zE9Sx#+m+tAiw1V%eYLE>Wch#NE-wEr!U(#IE~{htIouI?id`fI>Xysz9Bbgy1u|R| zngveWqH_$6_WDDw&c^tOpCpPN8t}P{N$C-;?hTt z24R4M!a57({JP?^bsSWEOv&RQnwH%%#X{ElXKf7eJ~xCWazcNz4X;-9Yy_Qr(IA);kGCDY>VJ+`34U{Pze zlFjwrIdv}z?p3vMqkzB1v#LR!Ag<$0%6=@mO)n25qLY%}N(2!q(%k2sj{Wo~!~{^} zc(aBvQWOc)52u)=4)^QZ<>piU5vheT+yw>lr8CcV!f$*bKe(SShwz+sQK&S~q2s60XK8oLePhOnzDKcs((928ph@ z%Zu2Am{RHo)ZgDfhzD5JhcyeT&F-u(nl3LdFP__tm}lq404R`B z0Cq`CM&`RCS20A|=o~uJO?$-k3G*Aeejm%BHX@zLxqng-1zu7${%F}!7FbNyHG= zuI7(lt)Zqwxp~YKN(B@lztj_;FK_av_kc*~Vc(BA0o`9S({Il5w|40Wqr1%Doi4>~a_+|>uO^Qsm_8l~Uw5D&E#l_a7@CyKEW*Yjxr=|8!Tl7?rMQ~B<%&`iVo zi$02{5^1fSSIKg`JI98Hu8Cde>coP8rD^ay?2N&>6MO|CE8gKUXI5aoYV?-`8B8_s z?c(Jsy&GR81Gvk;$c^{)0t%{x)G$Cw1O`1_?x1CU!Q9o<8)l6bGqU46nB0R0AVt?V z|JAnTc&-Af-Ytn_S;F*o^J=6Emnd4a)4Ei5=*NJfNP5#tkPG5jsfT<7FapJBZZJCx zsn%bH6jLTCfFdjz6YJ0re!pTv3|5Q>7u2XwgfV64qL^6nQ_{~-0+@e^n@Rj^=-Ko; zdCCcYrbxLKnv3ueE62w5_T+s7iaw$l|CuHwGY}WxU24dLP{mJXTa%2d??9Tpr2wQ| z>FWB+Xt;k0jKLtHrm{|m(XaQpKt_ITOc7rQF4op%AW(jwg%~YLb2~RmVMd#7qvq1@ zm{288Zj+(NfXcw2QVlEZXALwPC=|*a+n_2fpM_A(zHNrAni&IGNj`CsVR`|yVsfR>OuV}9=)@*p_O#7v?4o=H40$Ww}8AXVi z6Luh7Q&sN4T)qynEuDT}QPtbk@rA|aEelVRh1Rq6bI2e)yZ^Q}VsUjw{Reo(y$E{3?UQHyP(h!v@<nQus{+YW*wB3~l3?jueY@N5##n74ys+-otrc z$Za80OrhZw$Nbmhhb`*gCdbi8uWJ@sSz)F-W~S`iB`qsJZS|1EO(McCJ9q8(TUYNu z&>x(R?S@+Me(-{~R+n*LbNJL=0gR@-GFJZkp7izc5v+fuh-@c!l9qK=4)E1$@JWHC zg${T0>NC4gtz3cngwG_n2+=ke0LN$mSwLt?cIjGGn`4p?)AMJMss%9PfHzPL+PUC>FK>{)AqKuzZ7UfbO&f_H*U8xdRnC4@&l0TzBMhFE!YH}3uAmmwA8Mfe?dpk0H1M7PxO^Qq7@4hpjox4*!`3(P;Lc2z8xfh2JcxuS6qi1&bfTMA09U;yMuVg_@91oT#c5bXEec&2F$y7>BdamnN?Pn*U5iMC0{Oy)h#Dq4Mvm8~-IXJ}wa`~HQ!Jtt$SfsG zWDQO6431nm0>H;I9FEcX>E0$Y27C}G!q3oOeekxdv>!~MQj)C}LlAsaH}?-`i=ab$ zHT18=-4Z2YLD(daOdkLk0B+@>%j4VLr^ju#WQvkB!b)U2C@km)`YFvt1fPwclg1rYTd$s`BSI z^k@SBM*p@62%aKX@v5D7OAhlRnBQEr=zzi~&OFUrDr;eeFD;}EfVOafOC`10*VM`2 z4ph5tiK3eMJW2Z@ZJY^%uVlm@F-B%ODRwP?d~sp+`JVMh$bb@PAb<1jHG-SsSmOaC zaGeTs@#`>|VY?@~I-yZ5>V+|V7!)c67gvrkVkY_1<`9M0>_1f5F~}H0$Jxp=e(YfT ziJb{%VPHbet?iLvNy;?2OO`ga2TzAydEyJu@V?<9hzQIPq0_lfMZ=`a6T~$CuQ*D+ zp@D%4=iK@f&TB)g=%Vo8w&XNp*Zz^*w#$M1%wJ1G2Zo2w?o}h4XQ&Ta=D-+s@aA%q zP5J%bYX-gFC9yZ__a3KRxib^HiFq?>Al4~bp(BK*cx`KaA1dPYSP@$kt95v#t?@lN zmS|%cT8TpQ^yvtAadx_pM+M7*7&LVJWl;FblM(I z+k^X;MLfQX_MVxC#hEVDt8r}K?8-xJ%Ujuyz@WD6c-M&7AJffpEibj@bp3nq`7n1x zOAFsH&$;BX=lM`Q>J?*!{pPn9eT=vL(J{y4#l*H{>%jq2Af(s~7qW53#f@*dzNMpl zd$lRG4q+-fYomsDPQ1E-?6C#oAy$;{S-pc~BppQt@BHaMQ^CP~av~=~LEk9 zX{GTaNLzn;{jps{Om+2|Ukodib-kJA(s?RXCVuo%nU2DdqK60v;6UVzTm>Zto}iAK zDNs{nXSeIN+@CvUj9#)}1$x`sAk0@KPj>z>-UrhPFi;0ILmzR=F1tLpXC&r;UPGu@j~&z(MbSJeV9^f}B!Z2Lkuc81}jdc2xG$gUQm%L+8pA8|y3VX!X@( zLTv6iyEfhZ-h!B;#)#1+#hZP78MPNi z3eU){mRdZ_PK>+rVgohjBm5bYfpjIBHOHdqJuh`V=XPobK*9T>NhJWBN>tki>}$-`!!SsI-%L?~n9TT11Tr5?PIkeCPXW zNf%Kz0eb`1D0+^k!?zhNg!F*TA{1FmNl7_Edaa7v5%B^fYgq*iGwwvimkZ|A3|Qi$ zm>2u{kzzbU4{h4u^Z|nfF#%jsIJh*;2&R;;MaZ%*-ZkbclfvY}WTsmWZW-e#K?6N4 z)*d1g8+ymCt!}MGxrWt7h|$t#c&XBR^I~bKXogq2n(6)%0oOq4{+Rhig5Zk;E!dH& z+?Ie=pG@WR&T)=r{&=Jj)f9F}LB&f#xdNuaBV~0l^T$w+ZrQDuEr=W2U%aa}#bS2g zXY*>Q^gxoz**%5_C(LB?S*S&J+z-PV6Od(3Y6gXWs4Tk4c28Z_oLO}=XL8>ICJb~L zbel7M%O;$dLH%o-X5onWTDdsQ@buvo(1}f~%zfq3vE@?bu}1DNzoU_;*d;GU}zezml#G<$-vQ-1{?q>3fa zCt~5+4<-$%9qwsk?V&k^#Dg{q>GH%<2@ZU}+;(c`d}#J&_D0o%1z&P<@8RLc zmC0*yMdg#78yODgT`6~*ToE6a=voDpeoG7v>=ZQ*vf^I8et_YIYhIAg9}e{jjQB$Y zTkwrX+RoLxoz(953hbq2te_aw+|$T=Al>B>Hsiz?Mc`*bS63V^)(S5TUZq}V6M(K= za-fLkXAFI22F_F1clY=CGy94g3_3vOqfx`}>Xi2PyMN|o)h*K5w|?Z`U;nWr`$)Z2B3?|#hkm>Pt3Qq zoO}20Ei%Ko*RHjVIYh=9t9LdBdJnR-Db$;7mdD~qwWW@18`!f9e?5mVZ{tdHtiFw> z)#!c^JOfT^KeQ^APyh$1AE`>Z=yu6gIxu) z98rVUArORGV!#3w0$U@1@SGo>ezgjhK_#~}WiN5#&9`KC`MP!ASk{`+lV|s_V1&L4 zT#}~J_&hlw#gmj2+7dq-c7Dlf;OWg zvx;y!9vY!4t&^iaJ1ervZ2!o>3a`kjcYlL!6Cw#;vNk9y{Seb9RIHt+n7=y{v4w8a zrB}aEzjAb` z%b1b`-Qxu0R#C3l5SI@CKX7~VwO|T+szC@SCxCL{OD$rkFph?>5*Aj(Kj-?LR92IU zEF_5>#*{(0t9OaeJ$77&7mhA9iwk#kFtiOFUv!|r?chlw+?mNj7{q|it$a7%Y>lR* z+b$&9m;lh6EGkE5S7d?6!HvrnAo{nzMG^EWM#cpjmRzaQ={-Le3V7~-nqbI;_VwR) zn3{j{^U^~)jyf$XPPS5P>_^+~9X8YWSHBO9m@_`Leyac3R#WW8?}g}r_*j;k2DJ7u z#bMZh!K`;tXV_b(UNfCU#_ z@B1sLrntIys2FSqe2HW25`z{i;lDh*ps4Hv0=tJd%lunsCJ?{>t3R>9Iyn(3{K?b0#+isy&tarnpjySFk@+P!iT zPm(EG`_~m0UQO8d%pYiP(D60Slk#D(2Ewl=7BKa1&o*Bi%pS;ZmX5u9=1ek%&{Z_s z>YR&`Q|+l$x%WhaLoM}Fd#xzz`B>!>#$16YHxk02`KpcBhR!aG(5lQ zT+>%y+4jVd07|)q^8AV!zU4&ABy^RJDJvSTu*gkD?>hTXH7~D96A4gZVc)(^3B4jK zJ{Vq^;W(oqr?WZd;d2!>p3;B;(LRUUPz2g_EuO0bN4-p4&oz7NRgVfq(^VqIkZwSn z4Hb{0RT?T}hlpeHZh@#tS#NFt-Z?4m=$AI~;!P;kecHdj3wo>fE6;q~2W4-_A-ixs zX%enLw)YkY~}hQE_EL9-pWy<~d~aMM_UVg?ay4;df(u|<2tvZDOB zDWk&fI&-mQ&k!h3?$|O6BAMCTLNt`|N0JvO+;6*|1n03^aJ@m#coVTxrUEUb*RJcLkdL7t8pU zxx)GI;D8S90{HA#0P>(W2fvHt&5*zVpQQCc79HS$0Q~;1vPKi?`5`^?d6#XG#;(Y&G@FF5`{oAJlAwdA&k-RL#uLv9 zpcWuE{@5cA7O+P9oNs)|uf|GWMY4#6T}yGhNNQ93qmtdSrBwXgfsM zGP`=${2`Lk9*OF-7(jn!C`^hgq0gA~@m)i;gOsrxn!nql6tP2#Dv=5eqfJ`bDU`z4 zTc83c!U)~+TH5RYXsEd9>p)c|iQX0e|CC)@h<@rqh`0Jl0_E*ZPrtO&hcQ4)5)hbL zg6#L7pfZ|%AFy@yPq!v`&2s!uGL4DN?7#-$8fgTmwF0Nl1mV8KCRKHVU+L1oj$zei z!?&EuIxPfT1nw#IG|R6zwZq|TUJqYGJsB{<8EroI8j}3zul=i58BW` z3+a$`wtDdd5RNs3s}jc#jxMZ&&U+MzoZX9RWC_E>8G?56F(S0Veb^e!=U-ac!LiPG zFoadmuQQ>9E;#Lmy_9<0r5K^lyNx7evZ@GvDkU1R;DI6ZO`)v6wWcl}#F}NSQjV zg$*=zzF_rvs1c1*1JZ?v;1%HCK%Td%>a(oNUM>{SqFd|D=Wliz02E@X1*21Q()pU1 zbmAbGON%B4F|yF2E5& z$J#{87LJ_2vjty(;9qVp!HzRROi9V6LmT~SHR}u)z)PD_8e(|gL%zs`I2`w?0a*a; zU9*&dSk41N)VPNwI#2p`N2@{Yk7FnXZO^z=maYJ;4=2`p2mom?t-U@Qob2K#I<%a* zM{f?B(Vt;Zat~d4KTP&d9S*d6*12rZu$>Zwe<`bSAb?u_w(LV?$OelmoFBEZw4tLq zSsuDP&1*5D58i!rcTye|q}M=RwBf)@9$i#yynZAI1#}MQ%4^D+$^;tk!>6uw(maxZ zEP_+_X~a&=BzFxN9v_cC10-603kAjWA@Zq#J>EjCCJum^gF5lvv(}E#>CqP4-XDX3 z!Zr^ZSYAjnNLcQCbx$A$Xc+d^q{*t2XbmBPdTFdQAnHC4Ljo}Bb;f7HW)3BRCREX2#6 zpAZPd!bH|%jsn`AkzYFQTmKJF=K+p|AGLoe*<@!&c2@RYiR_V`6|z@CHklzLQOMpa zo9vLiv&r6j@AW@^@B99{uBWS19(p+6^S#gKzRwj8puXmERbA@~{Q(VW0E67LSE@We zpctofM03YVXfs||;nZ=FVLKVS{82sk3=rzu+tB*b#(VPL6n>(!ohK)>DtbBJpwMsf z?CM#{V#SZkg*|)Nko(x*5YQ8`m?iJpLFWH!mp?T&inq4Xvs3AO+5-gE zp=u?)s_31PionE`j4WWc>u zQI=Af!2lByT}9;UUT63@QcL!rgfOnrtMxPqbbXR35iICqkfLNzoGm!Uecn|Gd;b&$`%G z0l-!^8cj7VCCQ2}1|eA#UyOhWY}&xE85FIZ859iPDk|8=6)xdc7xZiUPKK4*Spg4O zb+H6ve=NUE=q1{FUtwJQoft%thTcm+c>xUuu$CqA;XPl=g7Dnz?)YeGaZcQ<;_-Z! ze>#a5Ipz{4yGc~h9iV8Je?;W;45Ndyi9KFnP=D`@#0OiW91Kt$^gh9SA6m>ZcVRA* zSZH4>3(5V>angMVBdPnB2G<=e-!<4KI9B)+)?q$Z$sS&ZrWyuvz%JLp4C|?=0rP_6 zSozR{0UEeP%$a|Iil-Pv7}!vgXB=)WsiKw|%vIND+w+S;f#bnnYB_MT{=h=6@`#+K z@Rnl=2CX~p1?SKVnY*?I7xwk_Bf#H;r$j)!FpY|Z!vU@@z-0+1+&9lbgi`}flF79{ za7-z^dAN?P%B1;r%!bAI%L_9c33fa36j`pwR%@O)XMUhf%3RAfF6=UHEpyp}7U&SG z;O6IryGGV1mX*kqtec|(He%}&+;9s0-PT|0lqE|w2_4HSE z>T~gNc--=ML3IB2T;9%fXKUml@0m?0ey9DfIG%!03oe}xggMcfS{}?t6OgE4L5Fb{a4s~p{%@(^SvwvYemsI zORS%w&9%O=iGrB@m}Y|8#~XFK4k7n%e=05}4;T8B^H8?yYkP}ltn}hVT%5?WNGk-B zt61fw)opL0JHavCTx0gJ%NqbFCp#Dm-8Ej4_u;sreA7v5!k@Gj6 zrEOej7b$R}f=V+9zN8j9f|FE0yY9A|NfbuzTRK+HVps7X(2hAGT24z#`vGA%mwOT3 zi;>b4DFf#A)2;;;M*qJ#W;wm`+O9j%zg0AlDLr=CwyXefgC8P(W1~;)M?ec)>z^(a z&a-j~e!<+bYsg8I`@nb49;2`=BM4zsC^ZcRU>rE%(69 zq0niHp?n`Lblp>uoX^fROxENKk)F2z=r>`I`aXho3^8_FtZ4tF(IHYh%&CNJ|FI&+ z>fR|G&p+w%M^z_qnPUCNN^eC^eR#j-l_a)PP9wM@z(rB5Qls@#wP3gylm=M>VBpHh zI1{@KpT4pm+jP-Y%rrv1ecQvE(lhEc>%_)s3V>B5exCyCJ52#G2_>oWD`h89Pj2CZ zUzwN-PC4RwU%(h*evs7NKe+Wcnx(%^aUvLE4cfNSwSLZ8ov1RauWg!JY-jK9kfc#eo(vk@0{iKF(fDruV-g$*&L7BE zdRpD4Ut-Cwd$FK;eNkgV z^5D6La6903->Kt8z!vrtH2%38uboPGbC`yyRB1Wk<@>W#YZgsbNz{!^Hq>yqHR{=U{r(2w-0s)tEXq1qBjlp zTD34m2OH?`*A`|g*LLfkjJ#ijIM3uj6fwfPnRBaR!l#HoMXfrgd4&{#%=Nm_7>~%X zwMpdJ%3WWAacS*D&+i4#=+~fEUYCu0rRf=57ucI-rlVE__@`m%`0jlITSzXrnPc|m z0*Wec*af0f#x1fE%&n}9LEXjH*IPNr64d$?9H1uoX6Y3g)YfSO$9@9qpKnKN^B$0K z4GQK*tB3dwyy3t^tr*voL*8xM(L9|f)OB<8sZ6x!pE4VHes3+i^GGz$VNqLgLO=o3 z53}vByUyRotcgegC5wz?@)h+cY!c5BK*iv?=>I$mR2tbHCQ*d0fmxn#JmM&CqPbzQ z?=aRgx$>g}1J<14wO$R_!E^9lk&)hIbwYp;oy;u|uT&cki+QZSspQ)mXVKiV8syoK zcT8kQ3iQ_^1(bQyF|OAX!E=}_&|HT(x$gEkRZPxEnH5zjdTrYVm+y3Kx=WO9S<2W? zd8?mwvi&2opY_UbxIX_XOY3#joRzKF(!f!k^zaMKRTssx`}e&=BhbVz;I+l4?HRMH zbm6z`81?~9+AK2v=idz{zH@%tx>s7U?mvhg7clBzX`c$xWVCPPDLq&1LO68iS;>nR z>hepQF{8Qhkh?nDnk?STy2RAL3sNJW(r{tp%C_f36a2OAwo0} zwBrrys->9ARcX=rpEcX(zXDtSuUQst0sy@Sab5pKUX`uJcFg4W*=ifm)q;!+wu8ff%kT0!k_bV~fe-pA_Xj_} zp$fFR399!~gH zvg@H@ka49BbM@$v8Z#;#o~O7dPhMD#7FtbD$l&YIux=pk!W5qhJ&b$Y34E2M>TN6y zOc?|7VF;O}^;0=dhA$yb+65%l$UT$~tVyiZ2nsHcYY+nUR}cSD2S>_^|V#M zh5$J0`Q?e@1xnk?ceTR{9T=viNRV>#OD~b}(x--t7bz*{<_BW|=2%nn1pNDJYbb~zsc=#i>2!Z(D(5#Ac~-7r#uQ`H zs4O~=9>WA9M59ujBegqWfiZMO*M@cn0hOSf^}0Q zQ?yce*`M&1fM;VVTZ5^rveFow@UL=O*Y3MFSN0ma`!U55!6U({m@LL zVoB3JoHdrbAy#sPniz2zo26SkYa@Rp;*H**1DOwo3kL2lw|@a)hvue9Y6{ty+o~_KV}7GH{t_Jw z5>F8j8%cvnwI`~uAD^v^Gr1tzTJe_XEDQuE^>69?_6<)kFSW2DfBxd%uldmLJsQrd zf~9YmznM|cyo#S4!le9T^fz&f&7ZBsvSP1l7FDZ*DZI@!SuZFwNE3pQeVD9Q(oc^pdrR9A*>9G=>F zqPOJFPCr?flp;9huU}ij))BdgJR#tj@N%^w-v75(!=zXl-0x9JA!QIVdoq(>1T=6UuQl3h zb)-bVd-Vj{+aKL8% zYf|;oSF>j|w1n2yueUDHUhNnU_r1!t?@?kRa|wAyM3wKVxx5ra;^IknEaJRv=hJ5Q z6EDup6hv>U(++jCCo`Y<_i8iQQ>p&*Y7DMyIVxn)Led~9dpt}2Cn2BaqwyD>Cg}+5 zr+jDGC*TDGF-j1&F<0$*)~7pWDj1(8`lr zM0?|KMh!|Z^u)PL4|30Wo72&$GQ+Oe`}2aS;nvI(8nA8A{FW=l_4*a_a!mWZF3pHi zbNvR}>_qXn-bfK+hCU)WsEXhe0ZUmu*8kwKQ9$eR6xL+u-GQm*-uv=}jotODPqZA4 zPGwI82;NMKp@deQJ&edInW$0>@`#WpT`@~zU;<5A+?&W|&?wT8PDwgmnwKv^vGumt zjcmZXpUN{pf&RNFF-n9?4rg`!QA~G9j9{J8YGG98DLvKGP4vG{} z{A8`%M^6e=o`89LxE=p01>x4Up+tlic~5vM0d6?0S@X=q6z2MzVS#k%tLWwFRN13o zkjqeJUa&Vm5vx))z81?XGIqX^rlV$#8&#rC$Y{>%vB}UF#XOr3&d7)cMFYkHPh$;x zw6q$&`S(kbakh03$O0~X-#wm@t89n}aPem>ZGB7o*2R1GDaUWQNGK-IxNb z45iaj3XJw^f`UaaZ?vuZoqY7`nLR0O<>;;6^*O9I&D?#E=OLC_9b)s$;J%^FwYL2C zhr6~&=B*bsqh5XsBQ;2+GMtEo-9q7Q>utsdu(}OKd(;p9)t{>omhXh5@*IA=D`CE3 z%{>kZG8_8mxmWH}M8r+r;Kh6GDhFD~7bJiD`0F)cCcYOv1YN5Q9xIfHH*J66uuSak4J6i>{#y!4i zP;JU#%Ub^`h~`bQwLh%r~R333_IP#Dr*g)7Z6{FThKow( zZ^I&siQ(?aX|`GR=t%Q7o?oF>B7FC>hb!XQ-;OnPX{}F&($n9>7dLgWTtF-`cet(x zZG)@TlafIxcb_fyhNe3Q-i^bQWd8Xhhx>S=4SvP*UA6dEcP9#$>D)1{5rSGm$4!<% z!N4o@$hPB*&;PEV5iW9Wev5Cf9|?dTnh_prZBNPmKk4?CXUyUq z#U6>}sHF~=!z>+)|M$#@x9+sk8H2#5gE1JO8eq=R?As_JY;Y-0aqdwp$rWqybWu?X z?_RYBOUi=T_Ym5>YCW|L+`138K<0$k7ZzT2b6Dn2?zarqa+{j5>|V;6tN%w zlCBzAGqu*%8_IF{wL0=CX=^}f4W{QTt8`sHv|%P|9^D>(*88nfojrElEp$>o-b5W6#RiG75WBZr%#d#7etwpcAO{9(%HUIIszCTl3dU>hu)d|tpB z*+BpUH*>Bs*N?4(F1oD3eODN_hhAmro1eGl5po;wL4!R8)Mt+(w@)q2cexe=2}2rc zh(inre7P51j~Hja=-<94y9j}C$F5=5o~W+yeV^fpIIQMd-e<5$9?eIohJI5X%cXF% zYpLN^G-aZDeZ7Ke=jGm29iMGoe3i|x3>lvXj-u7eZs(PC&N0tjcP)A6#MpCM*1U+R z`*HIha(^sbNAB2*GF}*s^_h&?=RN$ z_Wynr!Q%WUxX%K5$VK1HPR#yr6}?J8cWlsANKOu8=hD6?ci|0F457D+=3C0@;nz9VBu<)8l`;S35RoI{rQHEq=1>+`J?&MLHHN92qWJL^3sPZtHD<`w8Qjjc{Z@)B zPH==MuUSAss@PnpaN65qnO+PZ&f3#G#jUHIso{rn7QMmUtR@B@5gF6Ke{!Ow;E9rv z-u=3y1)nn_>Kt_n(wnDI&$p1OQW+qw`$3r0+2x98UxfFias6Qi+raqQz}nD%4eG7J z8}`F>Irj1zJZ)JUr2i`-FodKk=M1kryh~3Vcc-+pE>^R4;4=_ST6S}vZNEQ>Vam=0 zrc-7G4Gf$j%N7y)E=+RL@s%xju7Q#QL0xmns(FKyhc6*j8ovt)9qWME;1dnh!#CeH_i z)`V?0ok{o90>O$UDcfatUHLHK6XJvY_jBKSbM$|zsSJciv&9sBFZ;2<*=pZ^?}1*u zvueg73Gzu|hnmaUz7XGcyDN!%+Z4i32`a-op(;?P!=(IDv=6EwmEhTx z@F2xveT?!+ZZ58ZN%qM4VbCR{?`~oV+9Zw;aa`uj<9jHW+9_&3|E^g{ zt9)I<$GJmN(yhiEOv7z{@M}4p28tj<$OP^Yb#%uD4l4Ygf~E`ajYmt!0Pk74152xu z&kN>`rWBFeJPE!bB_7S+uv6e|4g5Rc6Ushy#O<-y^VrFKMJp&oy><5%27!S?EeCgf z?fQiZ*6(&nf5=k^VG=_D-~@e!Ass{^BQ zEpsjtn6EJ>iuQAi=1I|M^A#HGgg)?H%4_>BI+lk@YxMN;x~d7|f0|ZZ)}SfdERDRY zh0Y{INOD1PcuA)yVA0x7<>GQFR9~`(CnU1>@|gG>Rj>EV?W(QSyGi+i2dYwg=f4FQ zXN?>0>dKdTor?+|$%!=VRzep+K_d7KgadG$Eu<7v7M=7*6n(c|+;~0xY38@-hqSY1 z_Zo-wK)z8!**u^BA_cg(cfB`J~ z8cLdv^0=?dtidHE6oDlziU6pReDsVd(ECCH$B*?s?@FAWfc_i6i_2TwHj91u&W+od z<<6*)L?Un(yh*N@<(-@_i57}n4pC5JCmt4ea%6^pfn=7SSxHo^w%XU)^9Ky?hB=NQ zeg_+m4omN}CT8!hi|M-Nrm&+3#CfBFaZu*(jfvY0>upX0q!922d1l1NCMzlZ(~(x0 zT}(2k+A}+t<#qztEajWDG(ib!^w_UX3TZ)?HLu`Zoq)#;<6NeqPBnj2%=nL#&%Hfg zNT|f!KffKd9ZmDJihLbEdQrnl?FIIYV(d4HaAU3U@(7S%Wn48d2e`~3)NuAZN&Rhg zk9B!g*s(3%95>T@J)I@12HUW|PB+m`0+*)wJY)8+o;;cP3gmGpPjS)an^+rD*ZBxm zO_wE{U^cJ!I^ES8Z(6K4GHPdkza)FtiZY8mle5Z*gs@J~7$#aczr|I$LKJ6pF-l>@ zUM2B*hRtFN<<5z|iWvyhZJsk7sPq&{1_qR1#oeBXZBfDwle&evJghJ4%IKX#rHSr* zC3#!tG59-jv;Tzqn&j#+K-rTe5j~sp00bLy-3WPWU%f&5ax7TaJl&_szFx8^H_Sd|YQ{f3g0On9WiJ8B6rip0#?F+W)~n@W*&u zOwL@N+;99pVZb}~R>6e{s9^fwns;vf|9LQAVO_`aK9q;R(Y^d|T2A=YR6Dv3!#?yg zguNT1bt`$`rO?5Q?~&y(py^*~=3&8TCzJ&b7Z9Jg!S}F5PLroKAPT{f`(@7XLlg_C zY2Er4c1pZe5~yvCUQYd=7U4=Al4^N2<6a$1_ZQ_}iWskxOln?Qf(uh9M2amuB~vmh z)+wvB;k@VuU;kctE75coI79+mLrRViLkt z9!Iet(7~R8AxqpA2X#)PK91^44p)C4+nIm=mLpv(C#+^5`&ANFCrRQS9-Ria4uUsJ zaH#aOBwXk_QQ1t~DlmI@Lv1aRr6&>Vax zOW=#G5e9n?s9!Na#3WqRt={qFu|1B$xi8_xQALea69)Lmd(De7lynQ12%5i;r2xVS zfFYo&PAS`P#6HBf%0cVi=q}_SccR8$ZMr9kM+~kIHPD7F`8V$b-j`q*l@f~1(8?|I zFJ;pNRn{?5)|){L{3fFKzpXi*3J}M?3u}2vj5w8$UcwLg3h-Hfmp7fLqX3l`JZ2Hy zRQK{p5}IsF=XTXm}2Or}2~%`5!UPA}?n8=&qaB z9J_BDSnyhm+G|Y!&p?rM$>vOYDdK%SD;BQfK|9rpdU{C7u zK1dG5@xu9|RctWcDwHfVDomkVqdKDmI&A8GeO}PKH`vnPUO)U2Wif+&hi=CtXqwaI zsRJ=G#tmXZwKHC+Oa7Lsjv{;_lAxZ>^T8}4I~ z3X37^LKT;J2wuBZj!!!E?{sJ5YNxyBPO)Gv6NZq?v>RU-bM`kvk59PAM(H2Z+~Ie= z^GSuL&7Hh&H0oGN7gn}T-FvdpgicpT=E^w?&H_k-CjEyk-!x2*5yql93*?eIpBW$Q=9T(j=UT%=^@&gv4np!0Hnz&8#RkLU}jUfb~*Kp8^i z?9!ncr!A};=&vgx2vykDS9HAlv}`cn(P7FJgG4U)w)?MD)%=w(%C{L?i!PIrqV`Hs zgXvZAY1w}v%w4zc=Eo62Unp-*+cts1VAd{KTzH5EO~&mHzZ-7Ycafs}5fZ_#l5*|- zhjh61L!r9OvHu8|uqcJ0|9Og9D~FBs4}}ypDGLU;0-I>3ypJJj zpqTI!P;-M=mAam1DQeBJRQUuTJt<9PB-nY(CUVK*;RCBJf%0Qo953{)PRA4dvE2# zZ~z%vRHIX_KW|$kbS?s;_8{0Qw|<-&qIv?}A8aERqQ+F>RMP&VmP>8%po@|5W+BZc6hzhVv zpZj{%C2>NOj2cZx?e!zNsNjwlzJ|$op)K-4w-FU^0nuU~8ge+zhx)YBbm%k z(m7orHOJqW(d|+MveL-6;><1qMmW3M7{f20`R>038gVfi;^w3BWOx4 zAXIq+E3WR&`W@c=RWOy`E7Ifh9{tu$T)jhMm&MRB?zv00ulikZpXmMKEZlP{S z&%`oqQ}}Qprd|Y8U%8s&6o!^DW_$eG%H2};t1yRcA_}Ak9>7Q;Sg%7aL&+=1CrU5m^vX`lgR)&7m9lk*-^O@VwRSlL+qk z##{>tyW)f}aTarI=%g^weB6l?h&7TmFPL~)lSK%zWbdADqf)dsYhhK59i4GYSE|67cdGoMpU59i_Oy~+=GUH0S-@V&yKy!SangFGE?1m=GHrxA* zDmbjNQnm1R;G{NVI?}B?fMf7}-qx-nB?F5chg5Uyib1gWZ8Y~z=cePwgm|`mhttux z`2Y{u-CZZA@aqa53iwlK(j%4fHa)Tvu)M8i{Lw0}b2}Te`1ek@MCX;oOzv4n;{ z7`ve@;4$sosGHgc2ZX&ZqUN0i&Uq-${&ZBY>^=A7v`G}pLvr%A!My?B1tq*d1mJK@ zV1;JggG9=F}IZ*5iDgE%DZXio_?>|ccMPE$qum7n}8Jtj48V2HceFNPO-gw9BP}aM%y~76Ls^w-bS;iCw)4Q+rF$`J{=4PnP>skdj?~+w3 zqepEVxuFI~`~!x^0~ViMyWIb83B#?L<{c~c=y$Gq-u2t(vuRW9|EU!sgkVEgxXl6e zI!vQ{%FezU@&1%PR@9rquKq0Wx=J+}pXDwI0jz}K0|UzzPm82+Eg<_fB!dG~ zOzad^Zj+L;)I_DL56Be#C$86LE4PocLUh*O8%G>MLfxjXtXrN+QwEQg3rEMLliEkc zlG*5EL(6H;{M=57D8nHDnr(>QQRC?|G{AO}0e78_ME%{Nb=)cm?E@W+M_gTx^a_SM zmgL+f{LHDtMi*ea07SK{VMF-g!6DM!v3$y_k54qA2*jPZnn{zw0-e~7lw4mx7Szg1EoB!V zexvW5-NIQfG~TLbHB#r}l+jkcw*&*AUnps<;jy80V{5Q>y)u;|lJ5vRz<@-={$Kx`)pVb*3JGb$Th z=I%1IQ>Ju>tT-t=01R=!y1coo%Af)2%P%r#x-?YKeRU=}h`LaITl1)VqnW6+g*QSt zhK_*0WmvMehT^>L2TKK`f!cchNE875AmR5zOo)VfG(RIbg!8QbQ6+}%T*eF?1RgN{ z3E5^PHl-;u!PX`7gKE^avQ?BS3F4Zm32G+5LWof;jQm}gK!$fi3mqT*V=b=8966F2 z?#R2d09l|vwHnWPw;R0!CT(lNU@=~>JzuxRE1MPy_svGg0dKFEw3Tze_{VTN!k`pg zN%9r}Yd%@DEO^%?>hz^i22|VT?8`2Djss)f#KPOkbux4cdY_6hhNM#;6Vn-_q^xvy*qc&w! z@*AE?bcS)=V#%N#T7{}y9R0!MW3~b1M64KRWovQkmOpb9@j!ib=n;9Fe|Ak$=c{lj zy{mg#$-KAK(iER!Vo1Z(D?{9MBC#*ysW}aNv;LF|R)R;xCntac^?(c+^c> z?OBayUl_t(m^&__PS|$zb}Rlzc7q#29BvfuSXZARk-}cv6Tqm)s$So)OR-Cz&uY=h z6_zF&n~9+>H3-~rIfusgAt{*?*HnI~G~^b1t2GH22pB-3!5z}R;tS@IpOQEnQ2E6} zN&syB$Fo(H`SCuSIYjXt9hoUnhxNU?s*4VgcAMH)8mxb8x!H$e@ae<#ocG(n8xe?` z@u9m^H*s0j=`j?Ud`He@>}@3RW~I5zomC)re|ZqoT9I#84LK@LUBpG*^uo`&+rZ$} zLNvOs+0M^*tHcu|$v=pMyxu`n+S;FOu1F=+X_t}@2w41sIiyoGX-@lHipit&TWWGTmA$UcHD-msVdlS6y|Pvm)!OCE)DC5hFRYu zVvX?)bgU5V#tSx{u?;QcTM)!2Dq$?sDFahx_nTZUnSF_0xGf%>0pBP-qa4N)>sqJ0 zcln8Mdge|`Xk+sQUy zlIKyJIYqVAf6Gk3-vt^Pg8i!+K9fdTA<}CC2tbW(Cer)RvpIBfOVNNIqfzM?}>}|0g5p5|Xl*Hi10?Jc^I(#wExb_gmNVI{i^m zUJ{NpJe84Gd_(g%?IO)|r@`UbkJ+qAJnvm z5104)WCL_`N;xA$H}}i$-E`pfcCLwLHQ@B{)U$+eO>M3QZ^d}XR8SX7E)4@Vw#Tw% z!5LS{6y;P>vwlJU;Q-bSD{Le798HCYzh>S!7lc!8OsNhm5qRjZnsT(4LL zFjU9MfJTmp2pfR?iO3SwVhCKuHcatm)A=6}fe{90GK3L>rEgICcC1GG7d(qaeX_9O zNCu{;nQaU$%HAJg)nX7?t*Dho;j%xT;wT9fi(}F z*`bFhis|J>%W@6@oZ*+mz;j>fP4Jr=Q{gG3L!p=Eh-zMf&@6|* zrT*#jiEi6w?IQS1+;PGQ;@dzoDdx~>N57d4gxN}nC&ow0?5Av zPB8e-Aegk3vO<*C%&kcJ;UH7^`1&b)GLSjP{GHZ0XP9Kw`fC;uGUKnVuI~AXySclsZEmK6W98>h)=Mu@=b8lA zgard}H6{RHAh-tp!PO0?_poKOJYDnZD#N&SMF#ebwO5VzpG|y(qyQ9zWKV1+iTUx} zLi5TaK-G}%PO}_tEwpH!KDt5(V!aeD(OHSyqbzwhqwOdVcqkCuiKbTG$dmFin-Phn z?D{gPl;h%|_{lkhUt*d|o$L3}<#!5yfPe+~D~CK^hSh5yRDES*uG443%LWACF$k=X zdt+d~m@2BODJw*IM(*B+b|I?nh91qh~iXT_e;G8l*Ufa1FT=xmz?u;k|9W6fWW|eFD#=eUa$PW|(T1QXv)c zDZBP6U;Y^`*}Dq#P=cQ|$(`HMlf=6JbfsxC)$~1U!E0{R);Fn)R13+TSM`gtAR;Xn z*rn7|R_RxtrIOT8E+yN22>A1L`K+M>i^uX!^jz3TM;Jowk-VS0y!~oy&(T8M`|=^g zx&s#LvIJ+5l6&4~HkjYq{#?@wvd78`xfv%ME8-E7GLNU0Z$aGBl0T>O(znr1kGl5T zs68%~sJr&*r;z+^p51CQ5Rqxyd*B9P0mVEw%&>dq!(+lLoSNqs3J#lYytmocs?nwY z(V7~i&p21jC>AG{f*dyglJ1Sg4PfvdQFg>#mVz7vQ)d~6BbgZI?dHHKGtu8iaR=w0-gI&CxBYP(Wj^wuOVX`|Y=Qw8l_?RY2Id2YM{A@9Fi#jPG13aNk( z|7Z58^u>11P$;n2m>vI+DryLzpa?6zuabJ!LsXiMCTS{J{vpe|( z&F3%O+l7+O@dU6&B=TYCGVf!2*d{3KU)enO=)r1A+Ovbx-lhFN3s1p%i4Y+1A(XBD z`|YEkawcGKZmWu>xIb%=**K{PSI*D&6DTSHcz zq6too;4&|fXiIV!50vY-SzdW>>X|vJ50B`r2~!D+Nf11Epz;7mSrP|_*@qlGLIsI{+CMulksOjG(-FHqBc^W8r`8(!}pz+(Ee!ueY{U?)?Ql&|=zVhXe z)Wtvd5XJPpeMrm<5}21<6CI_{TO_4lcKNnS;(uk02>+Cv|49m;h*ZI{E*N*-h>?{w zaNeqph(D3ON{4O40wrdlecAEl;JxlNLV~R&sLG;prvh%QnkuX z=sNFkboVgoKCkBI`kwzaGDhNU@lzU3MQ$&1moW%oV1|GUN z4w8(|U0bwb$zRjIem*cP=4YVgXLz%7bjC%H&LFqAAN&l8XJ=t55#@ z?B|_ppO>ZG>SG?xze=tko zrT$Ve@+tEJOY2K(6MH@5W6iZa%Jl>WSd0#mTKypeeEY4oe_6 zUK~|r%~5mETh-Boj39ROrM{80fSry4HrGu^NcRRK2X%;mnJ}x{>4Bkn(z&2zkf^s`9)ow7V}9Y*ru-qbmAp8k{Kmoiy{SEOcw_-1{HwA9p50$+^g zKTbGTNt}Hwq)ir8NM0Q?>^@(Y6DqG}4AJFL_Mlc-V+_Y%5~@3VJ|z%jK0^}dCX6pP z@KhZ8SfS6i`pO_#aG6TKnmu(Un?xuX9K9!C5 zGzNi~_SaaQJ9tRy_pyFrD@!=s6F$?o->NbH5m9{W4DZ-RYFX=*znrL6mf ztbBMf3pyg#X}L>ob$iqxb$`P9k&`OhxFp+c0)+y*aO!xKH<5Aju6-Ci(=)w zvvG1l38KIMb{TI(fBiwI+#wVGh4guUozn16Z|&43elN>+R`}(s4;1!-an$%Gl?nu< zqjG)mFzMEdj-^GXMtB1Srenn5>s?JvO)|l5TS8@Y4~w+z?U$VZ_NLQP`zl@BjHD+6 z0tMI^tvo4bwAh2CTQMq@*1PU=#iXYFgyZHDX>nqn2hu8%lKt74#;wnWxN#=gkeBJc zpiX^ZK8J^|{c)rodXr!@_a}?{t$s7`N>NM@h=g-*SnKeHmC1x@eN0OBT5yV@g#RDz z{>;L`OGh>QSR2Joc_rGw*Noi!!O9zVW^$9K2S>M@UuG#LtRAin4y8VqV=DaeGAuuk z)u3L!=7c;wHW#FVAQ>IZXE^zrY|L&Br0lZ z__VEU!>x*iC$jWtwhmT@bRi)j>6xzD_~ZnPZtQm-6->+b#NDg=#2-$d*DNE%rmSW{ z)6)7!Syz`imigF#_C{r+g@Txvk?O`cEsf!^a2W3V!V9Y+Wovni=aQ1jmJT)^?;fak z4C7i{3!wOlaig(tcXZG|AuJRXy2{u-Sqg%t>A8)sLIQ8SsaNQ2c$_ygq+dR(w4a%&{=+slOU-Jj$3?#^ZWtM zN9r`{=f)@q=>xG);xqrCBaV|RhJ=Sx2QD%2K7n#A$}g-BA38dJrAEyY2D_xMV%G@I z;7K_K2Q3%I1v?gwRJ?2}B-berqBZjvgt-GsNXjg@CK(>HM7!KEDVbobQ$f>)m%gX- zf$+x5LKGY2v>}xRoxcq-ii8%JrYh71Kc6ax`nJxO_wETXa^1I_=2A*Enms`Nxa8u% z{pg0J!hrtf&SweZ}{f}CQdro{Ed zU5qMW5m^~o8u14&GI&*;r5?-XhP}b?>9kTCT5HuvZSwD#qYmv6XZiUssl$jqOi{_g zB4uT*P%KHr(;U9K9GAO^|E@Wz>FUaZY<38XhyE5+PRpXAqH?+H8cvj35yi^W1v|*X zcfsD|caAq`+%^+mtKaV^zN)ZJaXna$9Jb7Ox6(TR3+Yx{hzu?V7e$-dO*mrBk2l91 zA!$gp#(8TMG>0_Xsp`l9^Yq)*!nwJ**>H$C9&c(hF*B$C`I8qUex^B~d}r1T`;HjC zE)khXUBahnRkz|R0jS3>yGxPq%c|!o53Or5B=N;-q8Avcs&S^`uFACNe%F{gwR8yWf2R~?=+i2{x^a1_ z%dhw&^srF8;7y;BvmrTxtJbpG>q&+D*~f5t9$D_+Cm-%fP5&-$88sbIVu=dJ-IlsS z`sG(vHQPy_xC%8GH_7Gk?`8eFD)(6yr;sy<%NagN**nTPDhc9@pKH~UM|hdD>x8bcs+B1)8BdQC4{ z-XI(&Fr(e@^T=I1e))VY`2BqF9$xw8!{>5zr>l)~Z%uen9j_7O-t(~Y@f9be{PIUx zcd)}a6HdOn@o@bqd!XW$X`$@3dqQ#n`d!@pq&FXVdXC^AQ=ZA1|8w@QltZji^uaTn zNSv>u%wIGV_f;Z-jAYasgFfG6y)XUxkHOb=omeQQyIY>=NkJ+xrOTR|l-8X`pRAA? zW|a2!_Ndkn*9(g`{VNWQE}RbE?KPWVXa6}2(ov+jLspRM5$jT9u@x$b{Ab zR`j^D>_k|EVw!kCWm?30j0xyIG2EfgCj+mf!6g-V9SyN@x2t0u-E(y<@a@23-9!y zE1Mr0CH9)%Z+7^?bG$u;PL1oGCKZN=ptYs=>z_V?SeWXQm{Id`^Y_N?GUU`vq;_?f zRXWHKqu627YLiJwWPL$oQ6q+dvS$9Gzr}K%zSv*16h_2AN0{ZKcr%Q~r%UmW#n~Lw zw7A<&{O8EC%n@!rTXbd~WNR!Ugx}^B;$~$Zw}w;7RPPk_QT|PQgu;;XRXjVvJer5k zw6YDoU;r_ZNLiN=86g{!ZuL{lXP#UF618@SAx)=`VaWXSOQ=w1d2 zzZE9qyf0QXS5Gm}6N;~YmEGt3r=PS~J2dAIfI&tiB~|Gw{UzWHXHF+lFQ=bpWv^~73Ry;e70Y!25}_egn>gr$x- zERx8~me>m7Kep9*z|UNr7_{a7?)y7rHs)DRki8^_^cl4T?i5{X7AvAR&S6i-%3BAk zoA`s!Sd7XzgJqSCEl`LMiMZh75u@Hk29sCPf}XmQH;Y)C5qj-wV5sBz$XQe* zk1>vWEt;YaS1UsCbmHRWkY9xI7k2_ZX()DDo1YMo75DcFrSJOBUJdcZtK?{!p1*5li~H5ydN4%^mJy$91N}Si+p~a zUFsm;iwvnNa$)e+KP7E%c#vw-;RG)t@rREoRix+^Mza_8kVz)^$LL)JF9dn9^bQY_ zcb#lpx~cGGa$F+8Lr7DrE0E(*`Q75RT$}+rp4q%Xx|fHMY!;-hVv|0$~h@9yU7_sch&1f1q*o|k)$b6wq)a1Odb*So{Ngv-8rxUa&_Kdc88 zL=XHl^VJvlF0P@``=`exc&NCr>gM@8!tk$w!LzSO8>PQQFMOS~$Kz6>NkDt9C)>^& zOhM)Si#|(S2bjWs!&LF3;{#%I@; zEbnA_^;?WY4omN8%wDser{AhQX4)Oa3G*?-{_(@^_*pryjcqsjEQslS8;#m1<7LFV zKt9pa?2D6YRPXM6`u5Wu5*c4L^qSEvHEP${e2SVTa)i7#^9?@~{N*B6fASQUmq)P! zjr6@<1RozT^9l-JJv==2rV7K_YJ^R+LvvY0aMU#+aVHb5A>Uo46Mf}-u{fiSyRGht zD}H|^+u_w3*VFt*a+#z4S3O!H!mKjg(CSxgF3ZXbnCWgrUX#Jj*Ls2B@!8oiQ-w0m z3L`|)|EsJ&20M0dfN_Ix*pK-_S#_|Y(0?B09^4*HvOL=~vUyj#pb#hW#5XiNY&er1 z4=NKJWOz^^hbht(($uP#6tq;IOIiBFZpr0OB`qHn6YKGo5~WCAD)XdQCnu*Erz_ZeBNFe1BcGhEKRHMuyKmMgst8mjd$-%9ef{QA|oOS%F1LNF)+x5NjUKJ zTJ4JepwH5gV1&7>=n6?m!FL~`W^Tv#+Z2C6y!gnb$4JW`LB#;(-C1?iyK*vBCxdBj z2>n>_=F9JeHo&WFPEKULBAUyp0tM}Cg;*9) zjoiiZlD8!j%Ux$HJT#bUk~KhyAMsJgiO+y?SGAgR3Vr-S&b~o?rpv~uLn!%6x@g4N z*|tLmBOP66LE*v4n+$oGQMnfdOK@tcj`_Oz`HPDdP*zkF><+$vwraHY!vtMrHoQN) zSl)2E4m=f8wK`xCN|Ln5Df6=9S=(4+jzzOXjJ=LL?_W1jUPsIumGHa?$`=9X6d4nn zT4u4Qq$w{SSsJX{3=5c0h_1?!F0QIe&~E7H0Y&#O`6FH4!d_jS)Q<$Zy}pm+=YLe> z5~xvlelbQH=5xMWA^G}kjpBE(^ z)kbNY5$8NIi(R1AD_4`jWL7q|U{lY5(!s)CPF+>OaKi&thWXw8>4LnCk zTX)$DIBtbw>FOCzjlxB5s8nd?Za6LpyJ>%Q>8>ZOpe2WvKObpalu(CC0}+1Hk^j@@oaWoZf9>0|7}BP~CfHyU1Q z^=7Zo?RzOrk3Tbxs97nR9HgJIVI$z;n&Yi1O#E(3PBb}KD_62!5Q|Zo2x&U|rqqQRg2{rB}Ak}`ip7Ip-5t=|v7In#5>OJh$IJuHDtB5Ft>=Mz+os4|sXaTL6O30;9(-BpL z8%f?D##Kgkx*$*;(@+%i#)@WI)wulmunO^LMcX=fK>?mYywFQC8ZbwXJQDcpnG~I_b6?yh-d(WT7G(= zf9EW5W+xwPHbB6iZ4hICtz4A@s!j42g5-z=f*-S8mMoU&c;|NyvoyR>^N;GXA`>}s zkN0aT*Q=k&m2w1irhBH0sr|q%Uv+83{JAv@7ABM+H2=#F7bh>D!8&GkG?`nEE8=$D z!VsW<)Ch|2f|`$~~`0!_m$ ztTMF>qh1=Ag`i?H#RmW5E&g^v{L259)k~Gkyy$2x)IxvSp`@yz*AwM=X}(X z2<6ka1D^v9F0Pa%Bd|G0fwz#M%WF^dJBcYf;kmwc4>*l0y&( z{#1;do%HRyOvjSGWuT@GIzK-z15)$l0H#R1ZuQfg03d|PiHYLlS-hpZYu-sL`fZ|! zK>`0x)>I_IFpBqQN;Lp>G=mX$P|F->EDv)nP+GYbI6e zur(3~DyI^;Bs#T9&1%VgBfyJKx-KxT{{BiDjvPF}xP!h#p@x#RKykL=~m zR$P*L`9teSm@Zs}bPSQfH6kRC0&A=R-X9D##Kz9>#%(G$r$Ild3n-uta^~Ar7xAnZ z_l}IMqSscaUIx)kQUC&pK~w56TUe`!k2FVGJ~EG4KFVs?_1jxE@3b$3g**pbGId-q z(XD$!HwzCI@bpZ{)h?RIX4x;WN88+(M%3NJ-^#~Wi;t^!%;;=6buc<(nGLpLjr|1z z3Jd>)s?9q#o1X|XRvh;g`w=`B48j62HZTYVY}Ez8yeJ+^jY&?LjD1NX^6Hyc ziZRuz@|Bh-yD?Htua%UdGkuy$6=|Py8FgJaiJ-Amarubx;+aq>AQN2S;9^U>P5z)> z`lJ8d$2FyY{)wJEzS-NcCbQLI%zK3$8Smn!V@!V*UW!K5C*D$137Bb1;`yPkF8tM? zqB3^Je8cP!4FF8>*9%)(Gfb;e#N9`yA##!a-XNe5hZGqIScuE^kY&2bB8tP23&t;Ci_%apZ$yF?;-oOA zHBn8~UtIRp@m1&rlC1QpO2Vt9c^NZsOLSl?H%I3q8HN|mbP{%0t1=JKBVMqQnw9@~ z`JZdR(<-SkZk7{k`_W>t^hKiUY-;Y~>{SoJQg3MfuFS|x{}tNmvpz|&nbh>Tz0a%{ z7pj%e0wJ=Rrt127?I`=nZ@Do|oh9z`2l-5K+wI7P;BWPupS;F7N4CMbqWzS2x#6%iBq?ekjwj6|LUHX3ZTJobKtmnVNnsy!BTT2&4ZquF%r@54g972n=i_H zyAk`?*a?>|%Lv;$RZZn|vWqgR@XbmmGH`m)r5Qo;-i_b*RGP3+~#rw z1>OCE;(tPEXZ)zC%f?p(T`(%+jec9a+tHE~K8rrZAJ#=)pXs_07j%=~{C4PT>QVML>q+d2q zonhCl0b6Y%B)(U7JZczF>*9u0Oiu2F5cq+!wI4(}c2e~iumCMFBr7gO>P|8-u4#CC zML$&_g$m@}*)lb1DwN^!+T==Ty*o6r@XFb|v~-$0^YfwfzbS|*kzX-7k3Fsj_m9M)PEX=n2{HU}h~T)b6BPhl6mh3BzAa2Bv!a0D5WkXY*U3n1|J>1S z+QMIE)<{=?f-rIcwJ$_ivp|Aj$FwBr*Fj#1$1ebQa=^YpA^N0Y8Z@pB7%UF97%Xdi zA1V$Zwpb1XnD6^IM%oQaUy|3_gryt$ZK%a4fOG#QM+;Ikp9Re?=o~47QiCQ37uY3I zf|#!Be6jzImSx_*pOaK$pDi3Aoh|TimY0vFj~bUFv>XLjr~i|&S^B$8vV5-Z-HW58 z<}A?0QqQcAp`$ZfE-qFJ!yvWTo6HAgRT&_dKXXM? zfC*AepX;3{;Ib#0{q}Q7036aZ9Ht=D1MOWh1oX}?6bKg0)(g`BQGtIv1$E><_*~(- z?oEmrtGd%F=L##a_S{er#z+e|tU~}}yaRM`@8R}>CT&cv{GB>j%B$d%wb{SythoU z8v=5eT;vL{?!eRzK;$Bq=}g|*i|z7L^>!ih;D3Fj>Ma(e>#KqqNuDoOU8?S>&siVS zT@f+(0_H&LwA@tiEl#e6m5r)niSzu{JM(3L)a=P=?n&Bz!rcUR=s)AiIb~chWOK6= z-e>yA17Kt5>)eNuIIUC0a)VE!c&gI|O212@{xr{0H6w8=0kSbBQV2X-HF&A`q^W7b zE%k$B{_r0?R#u5Fjf-jX&Iu$i=m603TK2Ei#9gR$HEx-%?4M|l-up;MO0Hf!iP6s6 z0t(rree%T;!E9(v6lM2C$D2>-e7UY=jwB^Bp$Nb-+QE61Aqva4#Ppn=S-PUUdSRHF zyvlqZ4@_k2sTS*&fN?go+ecqHT~}-woq)GKT0x%vCv?rNeKiBX&hs$;0yv z#$dTjB~K0BoJw|T!S5w#Z89=tT3n-q{?H8|^Lm|AU=mIsV)48LIU}(+Jv^h@_jN*2 z@MUG-rcTUkO(SAI`S2=FlUFLKAh@us4EcKv?aW+?AnEdV^$8@Ca*1mTb*=NJPiFpx zg!J4ctMxRZM|M&9n_DDTJ_4yMx$6wQN;qabu|-0aga=Qe(TnDNsJf5caXHU40R<*h z7DJzfiO!`3kMxQ1IztE5#hn;3x*c0`?T%ha*xFaG9dtOv(+;cPoCbLc5-2W=BB-an zz#&zbz58P!&=8T|qcSp;ARdZ{5mySp`KojQL6?72MnYy-AsQC2#&@2Xg*?3USZa^g z6IUp)TfTGOxT4vfaAz4IgF{2u6BmhH$%}%X1>^cl%=iA6zm17j4v^Su?fXYuem7-m zCijY}G-u0zvMxDyB?a(D(l>9sGNI-faOGd8+Nb%7K<&4R$h0!A-($0dXO%c=ha>Zp zTVfU9GfcY6wUn(Sw=fy^;;OsllaH}g1U&wW8{=@|PoIQin>&>kVydlSF99Fa_aYH+ z3CAE4lucv(KnGIGcu^tY;mmlbU`09BvN1759{1eap}@nMHtUvGR766-=V-aLbcD<+ z+R$SsrS}4WD5|Q4h6LEq@mLKNz5e~B3=tOtA%TJx+vRCceJcW89Yk>Ufi}ryx55Vq z(%_j&Otdb;AJ~J1FyeetP%* zy%h8)ZD?J8+;yQ$h4QlW{rmUHY54LC&xDdvQVA5uo6o@PZs44z`F$WA6+;kO{N;x< zU@N4TD{sC5a!HxCOuYG#tjdI*3Iv%7BujcJdKs0{Z{B1JItGS^OUIEZDWo#zFWa5` zhp;Xkvi59~x6Yqv$rlVd-HLr5kLe4C@Ti22&)oNpKChKtS!*fKY!wz1Sb&Sw+2?_VxjWWBE&RPO7*iqZ|%0n`!-01VdmS!rtoV zwc?|Sxy35z#FsG}tm%^G3|J{_8m4#FFX_zaMHQTPL(^O<_ znkRdc644U^1aJ zXs^ZIAIIBt&jHL$9-}lVwEL9GcCACIVd^C*snQ2u?i^_!E_FeB#H19bMh=j+h;%ok z$esc`>(BJ}_tuu*ljU6g{UDTEd!yG<#P9q^HS1q?nVg;K5QPYmH}CU^NWUWHm<7P`pksmJWkrcbueqo;0>!p!w#hW60ITg$tl(02Fee5M8cOs;&bL z`{r{D1nX5AIGzR3(P$muz~sH-&XW?avZ!4&9@ zR_gY=dYR(yiw@Q_bSvQe^phsltQ<6?x`5J69mu)+VCesU`wX<20DBJR3x&&*0v7l_ zzbF_{(ypo)jPJsQKsUweUW=;EVx(lsUMjw>WgXGkZ~DlNZo{5K(669pT4<__ zA=IowE*i6m0#a}mtxhE97EicGYCaM|SzTfvt1!qDYf+lC`iicfzDo%G5A|P@*kgl?A@Qp2$8dRT4k!lVfM$ z5;y1`82*h+bxQcEb+-4H)+_@{+lIR~XAFo~!0GzEVe}H^3`s0#+|O~t#GZNwG9-8S z6MyF%cbXYlclLi6#=cCX+KYu@Pu9Sa3#Q1(NhK7ED93iuTO$ytk15v7E?{U^gJ<`h zWNn0n^5gC2oD>-K_sEy)3XaKhT#2xTlOQJ%D0;bhg2 zF4nmO>hDiyk6SFPRtkD-bEtLi3v-W&aEQNLcQ~tmof#!XTjl0Ujqc3+(CGZE6|ltz ziv4*iY z&UnqevUI2{Z&G?9ys~*9;*NZshot8#;r0HNrN%hYwnG!nGHRvKg8<0RW@Zv4j_nMx zDSl%W5y`dd*e_XoC_&lsd)4{*qy$*|YR^xP`|Z0qfa>QJDqC7I9xc`fx&2)f1NArx zyW!u(dg`nFnJ|!+l>}C`DI0+-2)9POqTgHZ51MUv{^a?1wXhFjN;Cl0N8-p!p)9=3 zaSLQ}g$qD3>F|Vah3!d+AVyjjvF>17!>$XDAmDI*6mGrLh>4 zY1JCKfXNT4VqyW3IvKkj(4FV#cXyVRl@%B~0!>Jjl`$-}Ihx2WL!~?fRJ{0sjXco# zn+EN*vRZDjm2H2f1EKuy-@e^E21-i)FSfX~CJuNqJ79||4g=_I4jZtVwT9?aUx6sz z???i8JrDwXmF^Yg<;4Nbsk5FC-iyYQ%NhD38ls5MIff47FK-P$ak0*a!I6R|$Qs(G%uOJvlr2m|3=aexa)mJBWzQ62!2#p|bM8DFSq-6TMYV%WG9@ zm0GNAt9Bq}!oV#s>Q3~e+iDlnAB41(qGj0sErFWd_23;m0Um9NlcOO)YKVB@$W8-O zH0k}^NvsepEkY8VQeyNka-%xy6BtPunV|6ajEomzqNg`}7-Y4Wh>hsXWBKYOq03uC z$&4DOfItO(B)EBmc{f|;aJ)&oy}kXA&);?ZyNA!Kp5O@*bZ}sWgM;&6ehWR8269RF zTF{Uj;=kp=FzbmN=pQE?BP}U8Xx~zAeQn3kEX~IJn%}o_H3C!GhneyI;bJ1nZ)9YI znvrq8!uGeb{Rc-u*Z6W5kbvPSMb+(o!D;?e+}wQr{b3xr2pNO|8Fp5H`T>THk1h-q zuk#h8&T_^a3p+6>DSyri24ddGZ(2j{b2(+C2LVwUzSyu)6kN>a1leGZP3ddRna&B*uyRCZFu`sgt~ zY5u)N7S(BfDeMP<^f{3LfAia%=jhE3&9@*m=+Gh9_VQIvMtJ^&U=%Q6U;sybFZ<;8 zQYMz99}wI?!*0)fwL=?e&?4h9=MF z{-68(QER>VI1$JJKS;@dAngJYD0?wT`1}*7WeUI}4)P?Z(y=5FK!bj^Gbo%2U+w&C zvN;&Bw@^#Ixw(mjiyQL0+iNRHMYtbS)$2~}PjE6Yd?Xkk+_FBIFR{1OeCSs+A%dD} zhD}V2O7*pg8dH3?o%zo%IHDgtv*fVua@pCD^$r{Y@O->H3*N{OhGsjNdb8J%z7hVq z`a1(YI02HWpM8$2`f`t39qrn>cW8=+`<&zJj6-BQ@HT02FsyZK_f~ZZ?8@rWhb&*^ z9s}6zKceHl<9w}HxNdIIA~wI!uhGuzqRYLsLqLk zmx~s{O-<`_%a4yzDf1P3(kSbJ6C&E~iqCX(UvKclubGM&_FE%*oSU4guXD(gV}$(+ zJM)mv01A#?OYjd5|7)Y4#ZQa16(5ku7RCnxH~Sqp&EurLu-|1-^wM{IuMV6Ng|P?U7RS0}7Jyo+7u>-o5J zxE>1mM_tG$U2 zdLD1~g?MA_O$MEuh5s4dmrye>F})m{!*S;8&g$P^Mf-VdLczLI=TrZI z7JGT1@tHj4T~)W#p;1yr+0GE>hs91^fonSSxS_3Xz8-Gvr^DLbVQ~CArpF6>^=Hq3 zku4bP4bVz{(;4=1_!%-^r4w|#+G(==EdgYCqSKFei%y&jjp|jnUpTX(imMmQa^$dL_r)#u-lYCk&J zB>5pW*2@3rH2ocpnYza0i-Z6>eSiD>nlE+$yx`f4ipT~St%KCzdKe1oP?v+*7r^lU z_TJB+!$l0}NFc%IaMFEk3@%K&-gt=ou13K1_RdE*^z44pf-7CngX8XUKM1520kB}U zfj-~bj)loW8O&&~;Q$S)_gBfN=~oqz$6$aC_;PY`1pH1#+MlkESKIbWvL;Jaa!)`1 z^TP&a3~*;cW@q1l*QUPN2qQlQeI$io1-pYy{38d#bq=tP{Godb*gh-(kWJ6boUJ%d zd<3~KHUhLno@`}$a4a(_*j~YDreu9!GkuDcK;0r5*m2qbN%qrQ@48<$do!h&FJHZy z0@5?HZjLn$St@im9b1;ijE%wcCR2_WoyC_3c7h~yKEd(Mc4 zickpHkT!t!Ishv17*Ps;u=@cnj*!mpTw;OdP|e$NhJ{g>a_@oaNQM3Nbg%Q}?2R4I%C8JM6{3ED`N zbCKZFkjn<-qQHKw=i9_YTG4YuJ(gz@E)oVQ#y&kiKVe8n2o`)fJKV?_L#NZ=s&av}aPQwEgfpQ<*o8W~*gfT}|E3_MsyB_RnYvU_(k%lKggb59q zH$t4aC{wpscW8IKJvXcW+h8`PW>^^f+cYKq572Y>oF}@|?`D{7ezlN-zMer);3TA{ zGo+4AOoWN&#}{E^zkK;JJ}Zk}E-EfAOc%^zNyXXqfRV3;AsN}|luM%B_5-;RFo!<` z1hXlbnB+NX?c$PWUE90c-f6zsKLjTbic3ii)fyuG`gJvUpF#|#!C^3u^zUMdmdt_! zENYcIcpn~t3CKq91|ZO;s~wvj32?3fd&UJ zA2wZ^vWl91rEI&Wadx1fz%RGhhpAipQy>E%+r0X`(`?tgbh@#pBC+)C+qb?PAEdlV z0t&6ejjP-IX(HMS6E*?|FsfbdpxMp`4m>8r#_;E0}^h`Te>9*>H2#`~EpPPrV zS}>Z5bsZdHJOpgjSL%OGUMdG^hKX7u7oT0SZAh~Bn5=OgkI3H82#5@{N#jc!eL zMA4Wj{g(gH%_U0H2|C)AR#GvtKHyIO>qTkSePdU^`ur0q&7k9VHPdF@>1)=X$LKwu z2t-a5-dLVoqcD8gLfXQd=kJW`Pet%0S=N147PwJ>5P!O*V69%$cU}sVh66$MkKFRT z`04)sLt9-Ax(HRzqevxgqXQg$iD!lh6xSW_Kb{uhvcJfWS-?Qp%H01NV*8hb zjwFo3EGogayg^Ak$7uAw{oqdfNUBZV2a=)=%m5R~1^6+|5W;4>YMu;J(j3xn6I>n- z98nh82Ex!ey-)dqoGDNHDo%Xrh}^4g`@%!xc>v}B9~U?al?!B zryiur(i_!lovS&m5BVmQPue2XO)-ZEJXuN)Juj%E{qWbkFVLYQVxx(9F`(m5A^xuj zez20kw{L07-ZqBrXzSey{Hvr7+BiIUk6$?QMFCUvu!o_<5jN`UK?3xt5UiR#jogA; zq#_OpIv{t@=f#9E^jxcDCdVi~pQo9$%gkN{yGh!pi13mW!1~EzWD#phvw|jHf zxGpEZNvEc!+U}1#>n*0JKe2gOr61NA;C+LNEvn-$Dn*?^odeJU5s; zzIFs?{k$QxPzUqBfNU;g;sTmY25kduz7Os$`_l+$19^3ISO8q0#R8}(P{3~VNSpe{ z6%hh94D6Y}?N$^|69>l~Og1Vxox)keE3<=tcdCpU64lmqVimj*TVMpSej66V;b7Jh z)bGLZnQ=l-@Fh=tq`0^#@r*$O?#X-Q+XhLzjC;(^_bd}_!m#JChs~%1@YSqR&($mpx0!~;k z)t68>;sAPu6LR5w|VttkqGdL-#dK~`fR9fezdfX=mS`F zt$ZrEfRo~vk%tHGdZC!#@UdHg@Co8UfY(O3QWX|VmM9TNlkmSBOJf}Y#Ss(+*q<+0 zd*C5;%t#qbe9}3)puk`Q>UjOiV8pAlSq+s(riAAo&JzPWXMtw?7MJFGE}l4C)-3Er zZNRo?1DIv=yNhscZZ1434*w!bK|XJqQ0jcmqNRUzU6+-WRmREby!M~pGaQ_1^<>}v zzk(`VO8#r+u>!B_<3z9R?rWI>-F)o0>i`(Ip$D&%qt`!&2JL%sXH1NZ9oG9`I{~uoWieBNK0Q4>+2Lvp-DJ$n%)m@7N<``2u<2>_Xfh!p z-rw$pH)uWw<)wAbyUGkH@;Bc3WGNuNCj^3pdoBVtD{a`eTTP+RD zl*p*4DNv&I+LHLeqYe0n$0sFG|15;SjD61u8TJuV*W3^Y+3N6EN5Ff#ojb5y87P$3 z|NVRei^6{kPFwGc$i^2J=Vy|({#|mBZ-^}`AdTTBOZVzi9dGC59qS*s{1*JI}GIF^Wn@#52j~Gh)F^Z+)zE@3w}AKiHrp$n%a*jH+)Da z)-hOBzWhAz9yM$(nta)Ah30RiCI_NU|!SR8u215 z|KT^9(I-dQy&HV~VA!J#d}%Zm>R7=T{PMEF&`t^wr2aTK20oW4eLIE=R+d?E(U zPr9(+*ZVpkrnDqbmigyHwtnlLhoXmq{GKCool7A8VactLDtxb>JJ})^t3_9x#TWdy zX?JSQ89_S4T!?Ki(_-t>@5__*FTD_cJbVkXJZZ;1ovXzT6?*3IT=89dKk~V|&xp;R zp{xN96kjaD&kxRBpwU;m)sEEj`uNSLo{lg{7`I3Oeo?S}$|D6z<=O~nHu<{HR~EJN znC@7QnOrktQHE95a_;)0^C5mPz7iwLN_w-wj59S2Jt&J^?i|C)S2V;Hv*o)m_C3Gb zUxI=HDr{Wb+vjOWpK-krSX+P}$jp=|oh@0{<`)%7{#f(+ije)&AGGKgL(>sN$I!@QnvX#R=<#-?h@Yw@784v`F1EL84G@Z-8|L$vJU{-+IA{n;| z*dh)_7#vts5;it}-;?@(Z4+j*UE<=hnnecNWzfRm&9?PMup2wMHUW-C z!KSR37);OelUH(D8rW(3?OII$pv%h1^@pMp*K|yf4qu@*{)!F`hKFteffPo4{sHc% z8)AtZ0F%19NWkDhIw~rv_>`2G4c6}9+o(hIHaN7PtE|tm2c-Oth|)>34yyKC8oy%fAy?b~G>kN8Vff7pl-buS!^F@S`=uW|J9Aa zacrxIR_`Hsxo&RjdH;_D$bpdJVgIqTDB9bZ(h1RKb=2Z{22?)L`s)|fsCwAyJ%;?!q1!!aL2K{wR>+W{ zeV-c0zfAi9?+yhelTt_s4)jnEaBZ_M{#8aJe&PTLHK`$3NV>33N73I9k{c z={Tl#-Sd@G_s2=%H=CS)u`ap^A)d zH6A$8?t^bEYM?)mH1dIs4GJJhP=wxHpTq&rZ{Wx+$ZQ~o?c>9vkq#b98tR(gvk(Xb zB*e#q4-^Sn3Q?MQgMkctZm%`m z=6%b7f`S4qUqOE9x!UAbXqFHH-t1QkG~vPtqSP?=HD}l$(sNb{(LRoI?J|s7BOx;x0dUGzyPB6(O@SF`IHR6R+l#cNr2}NPUyAq zaWRtQgdc>j=oVYgN_yk)msm)yFXr`NMWi1DIh29T<8>cu6+wajM)%GBR|8|>lz%>a zpY$W-@1L+#9-~T{Pz9oR17wJ&z~R7%}j4^`GykUH<% z^GYH}`mErv@Rs;(``xw7z9k;+uJ);5-5(@!o95s5h_rD&m5%^2WG>%nB|E0yXCQfL z$&IFijyy*v`k<&ViIa)dF}PC?c$>__Fg1hTQBQXh$m`zP+%U@%kG%34UNSHhLflCm zrKD>Ikr-0qw+=%L!Sg$k>Z2E3r({=z1R}!B{C-h08fR;!>YExg_s!%9Kc6zppJGbH zLy{q*724lda!>IZBUT0rDry?NyMPitq1TTLB4dFjlrK_Lkvir276w*M+Yf7T1CUac zE{%t1SQimz-hsA4TiXw974^ec!5t^@uSb_aRE5)hGlHJY{&yF6>ySRf9p0PU^MQ5v z{l?+VG!v7nPHnAQMCKkVx!+$D;2fr!l+T6v^X~~!705!Sh?!t|fb7hYIsY-rhHG27 zJYPG*79UJ?85lnJ+>go(`lhgmiK7-5wfC;>;#;1y`q|LTf20QhU|R7UN#b+U#mL#+ zxW50`tXuD}A->{mjVUQTM8VFE4G^de81APJr`KnzafdX{L#Tbz)o98%jyGoHQK)8U zpgOBJ>W6h{yS1aCql?^lsn^<$AQ?PZ^bYNA#I4RIJ@4K=cw%<|I9FmgXvKW?zXeyE z6i!!$hlVzS0E-(#z6JCPC{`#D5fRW800a>bfjb=ozCG<@$k{HiV!$j>cEEk|J35HK zvIGNl0JXAy^%4~Vf_iacL5P&V4FRA_z~?~0V%Qa~kQTV{{_W>uwAzLHd-RaA)}Spu zm&18nbx8MaVIn)8* zyY}7hfditEvth-edHqtoqh2>9@5tF={VoMW)Sm6*%~)+@B*eA%xg?&N^sSrZ$B!H2 zOvL{$?t>4#n>4hvUBQW93Wfnyiy*KhTzX24{l zI%d>gxD9`yTcmOhlkS_%XXP9x7`#=ycT zW@=glj*7dhijNx^2$PTy>ALrQlrfJ{kJ>~b{9=dMb9Yc#X^0QaX~K*7FC5~|I5|04 zjatFf#<)u;swgU=B?K%SjJrQX(9_z2S%c#Fs{Xqk7Bv(DdaXTT>i>XO9kg~5rdS*_ zurKW>g$_KnROTciWnka}Tn9mD#W+)NR=gMkQF?CGn9 zxn&$1E*`3yg*Xz75dQ`8_%d?~G&&ru?;Du2Cw5+4z{pL-lJXB-u+0_|lYsNTy};LR zez?Mj-C93$ud`nh7Cak#6&@8O;q5I5z~U4LMg8WDk7KqRXmDv`A)tk1>*@YP2H+G} zMNk@FrQ6Iz62}$&%=i8KcQ)@E2S?5wFbyq8w;H1z&m(69SmI~EFo!;$_4V~o!=OYt z8zrb8k~?vom4mNQq6ls(4FIIi2JS~dC4~o4DkpHjK@-c<@x}uk(CUCu@R5rs{~E`{ zV?eQ4ME^-Qhdh;N`l>f=lClt*EOj+m{8IjYLN(O-Wzyz#SysA_A61A20KFoz1}1U?gJgg}*g zus=c{aT^;dBR>C{)q{6GTXt;C$40Sdn$$)XYP?wfkS;{>{_dJHh6Fk*66F z?bM+i6#A#zGpiNvMy@795F+u_f)$KDL;_on$m8WJ;6Kyc%ND%)I>!1elYA5zbx0Zv zoYer6kyuJ$VPVOAw&-3YR}2S`Q4gBgAI#*ohN31=CLd@GZ(XGip=$x8(4d19fOlO0 znyJB6-VaDx5crWpmHx}a1x&$-fvu~J()K7yt%`))+_)NpPAUK{z-dV5J^4f;;)*w- zk~{Df_pr`sz`Skc3iY%RpGB-@I5-WN>K?8IxypR+ZagwjBstDCV4-AnWiyW!X#cfVc*tN$=C23izYi zN9nFxo{DWN2SLoFISjp^MPY$#43Gg4f56@5R%08!dwbG@R5(Nz$_+vZ?w)&%#u+_6 z_pp$P?aU8X<~`ajI?R67X)IXqVpN|+Ztu&DwDH{1h`Hsf-GBJ@ekh2?0t2o>8Po4C zVA&tRsnci~^cm`??E(Z1pqo*=W?DY>2(+p|LloS5ak~h~nv8uHSG&JW>#0Gzw#%zbMo5A1uY4eqa|)rZ%`cp%@Pd$0122y>lC`*AkQD)pdxgY z>~4<;i+0>zOCF94fDNQfir=HWu z+F@m%>`E=;gixUPHcYVk!V4>R)GULEAb20pp^{C^78_(e&y(p345@p-(DZbE!N}9Y zrc9pmLM|%43BZW!y1MNttQThOTYNxHi=mnUH^O5}e(G%xhF9{db#uiavF-_0LqxhuNXS7rpFZtL)ffL#Ub z97Y@fdWXNyA5r)AuePoL4%vL*8^D3i-y1uQzT&gh4vZ>seaSir+5P$_Qw_*?z*B>R zVVTv?-xrMWsgTn)I75L)Hgadh!@=w?NTlyORktr_PN^+yg3&>M@btbE!50bqp7|0h zfdKV>Z3wh!1%sQcfCee)+Q-|ug7M$-w>&}UK1@?QjfC$O8#1{NB-d2_Cnvb5rE>Lg zJWkjFoiBT9nSfwUaNaNh1LBDgH!gxCw8KhTJ<;VIJygMHc&g7<^RR{49T2^(33RRz zKt9ywzBvBZdt3?gi?MZB0+zJ^5zr4D1x3Z~+@! zMfmprm^urns?#pq9~uM!Y3T-~yFoxyP?1oi8$lY8Zjdekr9)bzySqe6L`u35q@<

g?+S;KX>JV5)Pil(5YYjH9j z@g-c;I(4~sgiBUF_+y-xy>>1xRS(^B5?=P)tsk=-sFR}o(cf@yejryaKizQ>c}_|V z@z8TybBIq>RV4S{AuT(z@;J*rh}xMqxNrg=7h)ouzPoXo{0dC3GyI_RlfMZo#f)w- z=gOSg+RzgNf%Q|ptYwo+F?o!qw+dGZSe^%2gb0M?gc$Um2pEHgE71l=7yCSO*Tjua zkc!3$L?Z+VATcrV`ZXF-Y&hIdt+E(F1sY?r!VGuL``kv*CONvRi=+uIlA^%+K^8=m z!Jf&ax&HG)*L|vCL(;wDT4|COk23sB)=x7FS47}LG~-R^U_fOzbyxY1iZbkIRCAYHyywzJyI8sOSh=F}`hz-Wd-##>TN zU40*tB;JWWxfp}_Gf4H3b$N;6PIr-}p>ca-BXmCdg?r3wY@Lht3kCV`*5QzE<0^Z!nV)F;<@)K1>;Ktfsa3biCJD+T@YD`AvQO3c|>!rVKW&KW|?$) z{qLzn7r(?Y^}B48-nIit2p-{lg9lg3L>Zqe15v68%&Dm<3Q5N?<^SPWgSh{JFdTvkH^gA-%tc%GY|z0(-024i2>31%kR%yH^Z7=rS86x! zrC^lV3_P@f5PwV&q)^RhVVyOSvUf&qp!5xJUo4PMA181d|2nm6>241Qw(q5Jd3Tr= z8|x2NcmHMo^&}(B=Y2w7ZS04+b!zbN`a@~YQT_G*S*EGZEa323?|F_a-JWD6SVYXGIS&j;&n*<5k zz{O44>z^TTi=PV$&MqI6AbK6(pW2bl61X&wNr?co+?y!TL(YGV;lg-Lv=0eL2zv-Y zjGz>{fjpr=P~88@6m2+|EcCemC#V!mN*w@L?JoonEaj!xBJ-J$^Jt!yQ&g}5L;lgZ zIxV$y1_4<3$~6tF0!J6!T}~EP-FIYX@~5xh7g}XjGRP#k-W zb-U^|=kw`+H$misTdJm47^#G9JTG?Jb_j;9lfx!djHm97JMp_)7kLw~9pM@tzfk-o z-iifXB!SyAn5JSQ_QqefJB?D&Nd`B6v+E|r zP+Gr(fH_I@Bv$`9lA|kYA_yGYZSNfKHGC@ows0Cej>D-cqwntAi*>f#cWutg6ps1u zzxW{ONafAkF9OqMCJQFp(A5!`(|_1=Ko^|kzNVgDHCZq#MwY+?>Clg>JW9{e5r(Cf zjQCdGH(r})s2WV7A$e-TTW|z#zvuVCtey9kfJp`uCE^L)Z$D-3X7XCDznw;8WrtKe zht+OCH%3-^PBwQCgqO5jgemG5rLF|b>-gY@KdS&#u{||fD%SNFu~2nG^E{hC`*t+V z!%Sy`RZYqfs#qeo1WpBuq>?{n2uS5tVtqkUEU@&_j3YrkmGzq#m(j zH6kM4)l1ROS19Mt=Z$Puu?vzT9E9TIrI=plb4T>Ub;+JjZCR%|A0609-?$N@lt1c; zO&^WZk=W7Dtt=D>e`q9Ax7PzC)dZ+Mqy!FCIQe>5=2bI>@^z+<>dP5NT#pdyeaNJ=?YOQQ-T50oii( zEjvaDHgY7x@prX9b7IR#Sf}h?0@q}nlgLc^Li$R_s+H<7XGNViQ*%lK9zI*z=&M}1Ugy>inxhbOF8v&zS(SA)U7K0qt5K)`gYSDJ&r$1s zDE@JmhjtBW^WXmuk3>kHE};GA26+3M2noT$2%VkeD7}(E>hL{9Z;vTI10y3UjMxF* z=_Gz8=`h%P0B0{KWVXw%9Q^PRzzQnYK8zsnVOg)aVRj)))uT4-nhM&@jkuUL?j6Q+G-YQ|5ImHK=A_Bshnn_#t(hWW-knP~S z($ApN4!8|&hz>wwd998msYP9}-+UyyQB7-GN+9iX?+x-)d*#^N+cLc@51=}0_)Xv{ ziTchO=EKIe014bvK>HyC^bdJhf9d9+u9}*^PU9n~VUEHtpH#N^j)Li!yRHO=TM?}? z8)?qJf3N_~D)o{j{8_3b1`z(C#K1y;W4~J(GFt9hJ~tG@$LmxY>ilRklqjY?Adqac z_s{^^klhX{<|WhPC@pI?CGl1|hfaU&ANdRr);FEMlc_TuqB`qHDF(troq9|!8Y*WL|T_$gU^&@})< zqS2-9##>ilBbZmNN98lR-#?^_8@Qa_ZDKn^*gX2s+tu|bO0ihL-$-j9?W9t3Ett*+ zo$&eQ=f~6aj%*0}Hr`2#4#eV1 zGy{dGZtyLpgl@R&h$tp)Y*_5eGt0=(-G6s_iatU)w)kfGYdNuItGAR!v36i&N^QF} z4&^b$6AjlU@8^AEhFbV=rW|3SFZar4j=-1^^h+RU`Exax_c3Il4Q0`nfsx`zexah9 z5qBl^Wk5RytAnQSbGivSL*xK*lh~=Q^%^mdDn<+j`Kt^eL>-ZLD;GQ*21D@fma7GP<}% zZW8+TkpoHAWWFfW0IjVJnD8iQDVL!+KRsK|*oQ`VVMt>4zJ>-_MP=nafU71zR|DJm zJ>!O*0}FJeBjqL!q1vuDeDJvZou9!AUK$upZftC{dWg1$ilg~LS%nf+!aO{fsG=!A z^su^8#R!)&=eJ~n{Syf+=^?-EzkIwCJul}NuI1#SJT!5=3*51C$F~_`zrhzVx&s`Z zNKoDTt^%!KFrlT;B&gug`YuihP5pol?C*u^?e7do?1QZq_yJ^xDou6m{2j3n{t$s! z$Ky~nBWNK%OedlqL7!kvZ8P*5(*YB}pXZ)9#f{S5Cccg)@L%fI=NbV$IXPHYwUG&7 zJiz>;{4OZ>XXF>=;b6zbAzO>2M`MpoC5qP~b+MqoQL^W~ch1ZQcQ% z^U=37APJLn0%)VeXG)(VwT9vzSUFKvh)r z^zRPa_2exGa8oYyh42=8LodV&g(H*6<{!~OKrT3px0ZtCUE|rl(H5!kB6=YMdl+wurh*#Y9urZale@>MQHcm0Rxfs7d1FL zC7>%E{>My%9HR#{11HN%Mr1JIv%w8mxg5&%=F4dVp1u324;<^W92b+V8oz@2Sq=Np z>BcXg^Zd3O-i-1V~`=7<2S#f)MHN!-26+86XQ(#8c^l1@vFQ4fipoV*WvnvO^;k|ivLNydq z0ZmUK(X6ykyN4v|z|_kQ)^RwREr1lrUz^FPA_W#FNC`;C`li1;k~hxB)>IyCut< z&a@_cRyPBP6;daAllAaexeu2;Z?kQm%!uJpixS>{=kJe74G?u{S|YDp8L)&%C=(`b zu4k(Wa`e5++O60yjQh+awBEkpG?Wx(l$_a7z#UCZ3aFPgujP}lSey<#dY?w4dZp`NMW|Fu1+S${r?&6X^_1v z63ti6?}Q!)3}~U1=353tz=J|lkDNF9esa;$)ul!lg4@s)sKt_BJFNjoJ?H*Aliuj? z*BXt)uL^1I7B7BD-v(y>#hb`SSCyRLwN;_&*W0}91Ta=OU5v==TVfb|eSO~WsjF!W z-^ce>pIdP|UdpT=fRC~j45I7n7=Zt)=xTHCr)m@my)Qgsy#4#F$0wmgqiP9&;f);Y z?hpSBV%`PB^1fN%Cgp9VLXxsc|981EZJ!<#v=@mTAW`|y%iLTc|8~Q7Y6Tx$Tg=VO zjG#wBY7^iOL^8CWo@GcgtXv*%OF-oWtKbtd4qJ6Q&5j*bCLD&%x3RaA;>sre( zO6PoBCK+m$yzbuK05dJnc0WAtyM@iWfPr$OTAz8sZ{bz<$4kfzBVK0n47d!&hrSUI zj3g!8^!3c~6vEG!$=5(*c8kMZ+(V_rz%FQXJ8;P2R^PlhQYRL~v^ZpiHZ0I1u2!Zn@RZHPs_DNDGw?GndqGY>R95y5H0 z`~pL?Y_>qh@3&bVE_W99HVaETwr+kq5I(#|Eokvx6tj8=JwpZ&Hx?!TFe=;(+}POl zN^GwMppAiE_QAm#ms}5@i|udwWt@+Q3DUgqZJ;YS%hiwbi{y70J`FC zAX3{Z{_Kj7^<1OFOw>mXAFi0EQMz`zoIB7Mr{EiFua{ygrJg8@2%dTi)^^(A4;rEpyo za>(~(J6OdOKZtm*Oao?jjP4%kKX~4a>%D`y-3J{P#hyw7Asp6c)srqD!+c-gr}}=L zo^J7j8%4@}&Tsu{>ZO|b<3aX13N&aP4Vmdf}jFaU!@RYD<#B8=|QJq;S;fqup}I=L*@{0&5ganzocbczqUOsn~x%+sEWz%B-`q_@z9X{WX_`9)&7=Id9VlwuETI1>d;zH0rr}z5pChZBd zMd@mG76|lCny&5~{jstI_ja&Z%Z>NJnh5il^mP2VJ`!U{RqxEz6*iC}JK*kqsi`mh z#M9tsW{)=o2ms3oUtE~yE-6nGz@_=@d1XN-$`J{8JR*S1W$^l=s(i2BhCq2^ZTW-r z=SL7u{O;@5NB18*$O3vYXUK;GM8b#%H-A#-R9glH;*y8Wqjrx;KDgYFPxDjBOd2Lq5S{Z6>XTxCG3j;UWKye5PODTNzV$76U9(rOkzmXn|0cz#{pMQ!59EazjFvJk5thFjXvrCv?8_P~=lZ z1rZ?4a=;NFy?*c>N7z4Q#+p-H-!?TjXOfblIXP@5Fz}I3n5iu3goe?fAV>FNe(|kQ z-1YoZ-(rmE50jF1S^`nYrK8R+o(G#vtf%#MKUBH(O1oOg)1`CV z?dhvLvbq&F#YkBJ9(TT~t|Q8&LV#o-&g4FHlULubj3*KF$Tc|2FAe#VknSU-We*5N zfmZ|ffWI3`i~H6u`EMf|*|8wd$#WnTM#0kZE@ELd!JsF)dah@J2uN;>`^?(e4%86d zPjR@@%9!q+=HeRoge&%1a#+4tphJDDewGk2`!BKZicbFg_QZug;Q|HoZW<)KhN8Fh zSqArSlSVf)gq{z1Fc}F>LuF!K@wmpqWF*A)Cj5tn2dxEAEv9$VC?4zP{ZI_CN4Wm& zW6SN@nji+o86&PQ!ZMIjNa<$81112$S?|0w*Am^?D!u0Ao31c>3^xw_LvMWEC%o3a zPq<@BTmL3Nm4`T&PeAF$wyd|^iuiX*toUo6IA2ALg*?>p<9J9LX?&%02Ax6FIVLz$ zUT*V1C)>Q!go~VL#TRRaI+AmT(Y>uSAC945(dBGF2bM5<3bRq2604S*tO@$P?lo}O zcLC=|X?^&>aO4Lmn*z`oO6CX01aL)$Q<@(r_d>jddr_|V=pRYUoQpy zD6fu9R^~mpr=)U=0LStE{rkN_6QGQHe^s~N^oCZwaWoP%jnMMUO>Yo~5I|9(`XSQY z-8bxLUv?U(s($ZRwc^dzLhB3w9sSW)pT9K6i5)osXWxE`hA#0)?|8Z0pf=n}0b=Do1X%~yWS;2-hqez(gwnT$PW)GB& z#lq4SIa^z9U@k{sklA#)6pnm=eOB~mnmt62vwps|(uba4kwyAyd=2YZqZ++&B>pp& zb5LfY$o_S!HI@Xk)Z?#zk6zeyti3dbV3UZs1Y7HrSC?TJEA)8D%g>z2LoTFY`a3fx$Gj>>rF z^cGM49c(I27GF?0F`x^NdD0U}vcwm&ms|Ud=tP04+Es2nRe=#L?I_bQJ9Z1k#jR)Q zCF;MHFz}J>-BQA0kr5K@3#JCsgT#u#y&cIYuj(%-2?JCPk1m;nU-;19DReUE0M7%% zejZPfh;i`J_9tG5??wN=!bq{;HG_5 zZeR4%un!91sTB`MW*JO{e!V~Jnl$;_Irja9lAeS_KgAV$IYG6FcLzAZ@>UOaPkCS)L(WwgO0|yQ0=BEb}_d3s} zjrQUdzj5f9mwj?2gbzlrWd88&mrG#lP~Ie8h%id(`rNW}zST;zKAiCo6uJ;>gNs}1GU&kZ!>onj z|2hp|pO8=Mk9q8Pv;BWtrA#2uKq^{0Tpy$15e^T+UEB-v;ewL`t|-AU%8}e_Lj#W; z30bKwL3X|VYQnPUfKJ7YyOGMVHQuAnsugsI=agJZ-#8prx^8bAK0x?tn71u1=d*YX z8Pt%mI;xCytsj{5&IK6H6*!W23qQ-?k#o>$mZJs@uJvuTCOLMC$rj!pFyMw;@iI-2 zAzuvStsqUzMqA^>tvXl#u9U#|2z)_`imIv~`0kO`EXiRLptryF^~t)q)#s#+{=Wa_ zO~ok#3=EZ#<61BgB76ZYxnkj$m-+c^WFk{3??JU3G2T~XExf-~>NcHl46UI)$EOqD+} zFc9EIB0F5@DC`^@@>}C-ax$PwBViDj7l3UJFeZRkBGrO_C}9+7SUqZ5%y>1V5GnVc zKN4;w2tq*%NJ9bK1|(%#yFG2tEGR^1q*Eu<+5TC4EaFe7ST;Of?x zVl$pGX*38dSZ`4LCGqKs*7C#+dNEWUj*p&zNOm>#qqXlzBT(@_=Z6KMQ};4(0I z0g`ne+EL|Avc(9aJ2m|Q59owjZ#)zfXv4v)`TAgS@FX}OWbbH`!B_^s8^dFa&chED zCnkgSBj1zEG&R8sb@2BM%S3%$ykbOmWUX*}q63|APsyU;S%b$MtYf(TlYM%J@R|7}>-iJ4q1)e$c5+dO#4Hgi;?Im=O&8A8icxW5 z(>e;_EzpJccjZ{)z;Ghfh48(J;9&D8_YV4b-eey|P+^Rp;#1<2=aGuaiszEq;CE3$ zkMxi;aIBT>Rog>g7uKg$aMU#YtuSW-@k)Q3D!3EZtWjMs}9ig**3z= z@Z$3q?7NHriIGxc6cE!dYA8HnLokd9MZzGu?a9x0s-n*~`hZ%Io%y>I(D12LMsG z{zxI2fA_QIw}{Ft!^o-ff@@UkmL!u`Ya4cMB9Mm;BT?jdC-(BZ?Cg=Pa0(nIx3d0q z;&R4+lnxjTVtH9?U^?eZYR0qh**T9n4ZPtHaRabZK)d?2;DbLj%LGbl-WK=%UDnky?Be0`0o(F?)*iIN*c3Aj! zbjTr{0!2lCMEY&OTmz4X>wY)+bzutuG!pgh+g0HP?%tXd{Cs_ z4aopV4lf4~_@CRtcto?&xp*U$tfv}f_&^j^_S!ZdX;fK`-GDE_7JAJl%Yd$a8A3x| zyFSZ0UA5jUw1>)%Lc9{DBWXMcvc5d<3-4|>*YY(gGudTOY`&S|vh?QlW$jkbpoD^u zdShoOasDtGA}Uo9EmwQmunDzh=T~$$$3*bn9XJeNQ=9lU8`IB7; zUe*{ti&FnLKFgo%q2|Y+6+=MGoY!?gVeCok@z!jk2r3HZs*sYRr=u*PJ?yNch8f$I zFv-%)8E%<`F7KGF#b(4SUTa=&ceiRH05nT}w3}83Q#(TC_)D4lXdHu-+`m^}HJhk; z5@i9TGSCt*(><;*_}=-q8wZ@Svjd@tlmIr<>d$%U`g4oAVwV7W!AVCD( z$G3g;Mv?{lXr???U%lmQgHTkuIBxCq%^)Lsb<50AD*Mw79M8X@&sCQu_m?-mmz#V) z(B~#6rP*8{;Qjm=$9;6q9|5b(`4CHPv|~4JayqkdzU2FZf{;<4e?S8tfNx(&maZW$ zvj>Kzpr&I4$~l-38(m-myc9%k;Hi4J6b<7faPLHbRMlo7c%?(d4;EGArxe2eViFR9 zKYSnoaV;A**`2~%jy|E^s9ba5Ku_<8A>x`EuAT$01Rzix>3@B|d;u1BV8DV{pC%hZ zB&}wdp`XRuV;4v~e3HbE0Z!nkzGGB<5vG|gRkyc_JYVN;V=+q#c}MTX zqQIoQ5XwfEP&HPfCqXUaO$`%K%L$CCdKM~pwU7FrbhWj6#Y{+U7|m`kKAF7uMp&_4 zQBSy9qqkR~L9Jpi+mB2t(v6e$71$vM2kIe&g6#N-*u74S1$5-fQwpYrG>!6i0%+O+Qo{( zAoE)7F3|g7uTYHN$ z=l*hV?QB_+tL9rtvPj(7&&O`);DNxXteKmPLV~XKd|o+J*4y5^#MpN^t#wv{r&;wn ze?>^@(^Qc`zUzdO9Rg{SA-t6i0kQveg9)9>`yq??rGf|x3u+eM-inHKdzDN1kYH&9 zMHAdS{<4ohZ^4XDAmRlf(?n~^k>TC-%wseaGZ}V8;x-fL^q+`LkiN=z+u8$BZ!q;6 znB0%PzKw=}D`M6s#&ER)yuNi;!*4$j`u{p`qj25-v`)s#&#thnQ1WAH{JTYZO0Ynb%!=<0NdF!|;HCtiOg| zMMwp6!Od`)mg9#2KP2aoYD6~%LE!-k=L1EIJ`QkeFvLct_lZJiW@&t5I;;={NGF`zuzuF ztfMSYX^_4pfz0EBW_{AJ4z7y-6 z?b$~|3Lf~>ftBMUiiExa50#C7ZyI!6KtTkTT&T@}b{RF?Nud$s)yeT1|N2q0=2bRx zaZVAD*~c;7+`Cq`6UM&5jH){}*4ahu*4qPH`xq3VpNW~Wg|src`uVA* zBdR9#tao@>2s1Q-QzD-Z$|Tpx>4?p|H@wt}Z!(^JVIcg47yqv6k_(gTU4RUs+U#Df%ieGE3=<=x zOy{|1=_`@##4PsxDuLb$9WQD3LyI6LzQ|OKM>{qaCMGE{6-CS;3Xjxc&1&9$wz0Jo zaB4^paon#UOK8e;a~j{MS2^`Zr%!5UYGz$4(H>70|6r9%{x<0g)EUB=Si_jxaj2di>jfW3 zk7lJHKIvVVG9oc1X{eFyFq^rze&Do%Z2IhHIMyx1wzEWbYz9?P#$J+n`dR*N1qedk zjSs7PYJ*PflNQZNDm}Z_=Uo#KuH7#Z-u!6U`W@}k8a#cmRo^WgquSSd=cWi<6^~Ro zzW1mn&(^Ht>+sM1?`}VJaj4(l#VP-6rmz&Af+fJrJnT3>=U}P!YyIe@efgqW39QF~ zaWR(EacdaJgz!{~%Oc9&T1Ld_D^-=vmMeYAlwHdqCmuET$%!44D&O*m@em11J=J}e zdGD`)*F}bW>XH739#*H+S&gL98A_$k>q+w|VxC_e6iu8rM?}=(&03rcLKs>*#4@^0 zw*|+R)vm)vC!5rZN0;}R-E_tGMoFCuaASNs9(s>{!9mC|I91wRlZ6gnb3ZE0_PE-= zOck9KPU0}{IhzYG5|7|lkX`&RfzNGCb)CTb^l50iGixi+>hep>sy!@bF&o_vkMp}G zTX@`xkN$GWN?#8YHi7SV5@Lsig;>I$5RgOuPNMIoZ6kT}=`+aB5;O@;M@6WH9&FR( zv$`oK&ApW{7n=9@-oTg@Txu;A%VSO1R+q~DbISBQm3XsM8QVoZmTtE`L}%N8b*y^0 z@_oji`owKNQ%1+N$}jTBF=!Cou+8>d0McdOB_*IcKxwx>|8|Yk6LpEVFn1 za_0KH>%NB1CgYJ`ZKdj(OW{wc$a<^$J62CU*9P^N{$3983gDM^tjk345i!wu3Vhj? zl00#UNI7PXk2Culwf=&yV<0!w>K{Q~@6^bhj?Ek9wdTXLJ2o+5HY1B1-!E7e02Ek< zz0Lm!d_*WCpC~FWLTw?iveT5oM7pJf=l^s+i@rzS1{R=nu)zm6+_~nnw6Z$eGPq8L zp@rN9BdV)*gh6i$1=Vk#+AWJh6IBWckk&2CyuSnP0ji5T?En@1NZWCeb}iN z>rSMz+f9|(cMq^mzvfe8<@5W*P#czCKqOztYe$hTrJS|GvpknOQCyX1V zEj>thP{`xiDJp|-l{0Y<^KXKn-}@Mouc}yc`YTcmjkuh=$yq`l6b8g{3Q&fRj@%~5 zA2fW*_>Ng|DLqI-xHd&|{)0u#_S^Ff+|SIlqrXjUY_?nPTT8={1x3Qz8q8g}eQTyE zBXX794}v?}*W!hDel=!D9RD$4C;9PfV#YRaWJxq+5ao7c_lCXO?QmQ{?LW^+EK9}| ze^@y2b7NPDNeODliumj4%cM*{#VI-GawZR0#HO*OI}xsCJ5mpx>WRk}AXPBZCB#>H zM8)MST+MQWJ$d?RvVuD=b>{4E_yY0Q*Rq6~^FMoB39NdAvp??FwdH5{gt|9hJv?!P zU95GjnSAYXO$CEpdgnUqN<ZO_!$q^zm6 zn%6t4yXZlAHO{}+wvB7=BEKc8Hxk^TT2qyB8_&hgog=Jc4c{?8F*hKKxjeV?iDE3g zLHxVgI0`?@!+MqkwW;v}v#U-*&Z}C3&9T%-aRqBAV@vYk#W~2Xw5;Ka&q~*%SYtWO zdtYCkSa~&>uAaPjjcKNlIU`z0B^fi7Xr* z?VDu6HE?a~sp9)=XK5}-o8P@AzR}Fcal>WT?kQozdsF#B28J$#;_=o*G>5np&EZnM zJTFB~xA7vFK6>Jm#o>FC4i?j*qM{X+1l^9lIJWDDC8zgQb70FX{Mm3PKZU7_1MCU= z`TOm_&=B&oXJuvO1|M7pVDf@z{$3VTT8)x7b(&v;+6(_d1gZm{Q+-yQ5onubLAw3}lMV>mMUP5kEa18YNny z>&-#P%sDdT3(uqya=0mP$?MwCmU@W&EmgcQyD9EQ52d7U{k&9?{PUK6xGzdgU)7I!znEa7iejJ+@&R+uroe0Lrfl8bwr zD7IP4v;Mjkr17&Z* zywKK-Me)ayg|pY3j|-|4EkUnLLk79t+nD$(VEJO0%|fYNo`nQy(4nZ5PK@rJjGC#Q z{;>CQdhltAJB33peL(0KW{OKkAR9ln;Mb>2YQ zU!i@a_vkQ+}CQkJm(f&%WqevF^6|8T@ICZ_&(J zzprl3F+YD~Wn1A@lB=kcI!bA`O4Ny~=oF|R0-p`q1S$i?NAuv`RR8golY?h1M-hwQ zmSdfKEOXTD>>KB>_d^RI8@ua6v+iMlTh&N<_r_VTIg_=3iwuX%73+|)2YwI6-_qru zDr|GU9nt1EA4p$dmo1!ktuM#kWV}AskI>VREPWO>N~Lm|Fp>4==eDC=_<#soZ(IbO zbb4+O&goViduN(Dw<`1eDgTh6=3tX`-2FYpod5W-m8`7&5pesJ4u#uzKOT@70)-JOCF4eG)^S&WC}6W8}Kv6U=K?k zJfU^^sVcc$HHu_41=!fu%SflM)p*T^s9%|yf_9b!m8#wFJ!B+XTiLDu)N-8qQL;mN zeu2_*&_`Zm_vpCf;%w!%U~=P0xQN}nLg8RLPwQ2DIjs?8e9@`?^p>D@k#3D8jm!3n z?pZDy8XO}`%48j_`N;vEydh)J*rv5>iw+WTf<5dq!{16fR@Q4lgVV2hwd#!e@s*PW zLod%y%wN5F6{~rsvIrx96(F^T5)8bP;EgU5tnJRGZ7cA&?_bjbvm^z)Yah&l;+T6Q zsd|@|j9Y+eyZ+uC-oI)L`kju>*kIfExQ52_=ih(7eFNbmf2M1BXz;O)+`ZJEG-aUb zp&}YbQw~#PC|)h2=%W}r_>na}cIuxC=u#(9AVq&POMLw=&$$O9=2c2uj75Zl>`&~y zAdTz?-;eL7w11PjX>(U~Hh}O1OYW_ELoq^^Rz`jio19x~=wPUI;nVmzERN!wcUz~~ zf&C5eQPub1ZSVK%U(O#}{Gt2rVV0%miAF%+n`lyV=y_}&SjlOVn<`|EEPbEp><{ZO zWY3Ce<&Mr+d=znYFNAA*jhS(t4zIGlGbiSAj*?~}EyhpR4J>!F4c_S&a@z5?cS&fd zu?sr~^PJWF?{ylxyteGolq>=6PT}dJ?}5&7T$xg@8sq%wD+Voc*%jZ4ry*0r0aQ)> zxeEt<6NCxZd7izgsjU?t3$N@n9YrS>&QFOfegN;j81r$5(D&6HgIF^sPbO7LPG=`c z(?I9^V88F}v z0_dv7!F}<9L0QZrrcP=fm?;OGgBVo@$6CocKJ_*eVFyl4zIsAIRiTqQVp;Zm(|*+2 zk>%E^Ufo)<6}B%%U|_NcZtUS%0M32)l=cqT3AG2wc~b0)J_CHcdvF(Af{Ie-IPAUc ziW#og>@D8=P?coI?)Uc{#md3Hc>GG%FYlR?p4xm#2tN)uR6P+(7V5So7ayoWFGnwM zmt`;gyZ|teI|f2VpktAD`q!^I?@M>&AX#$tgL-0od=K2VMj;Dq?D9?XfvXy#bSY=`{54f^Tf!f zd7|V6&aa4wh)(xx6+(05bS6F)7vEDvWFD(*4Hx*1wA`JS+%WKGRPcJ+vk3{GI&~Vbb39yY4sswy#eg z{G!d~_|*Xg_dPRt&2?m>-u#=iRkUN%Z>_dXkv$9JIe&4dGnB&An}6;Z=ex?0Dj{1s zuYWK1J7?GV32Rx$_Dc=V{;q=N|A`f$c@@dwcj2OCI^@arEjr^7bK!|Cq1gUe$XY}C zb1ok97OK@C3@CIgGA79`Ul>r1i(dLvR`p%{D`s!?6q92piND?}=as-O^G;y?+dr{= zu~~TMSK=#kbH_OnC%wa?5B2D6kGbpL<%rY$zJoYj+xloJqx_=y7({>{xsQ`z?Q*mJ zv+R+tCv-TKZVC5W=QHRLu9qb${zBgUwlVX=QI$r8UkgefRv%ofU9tyY)&0qr^WHap zx3!Lo!x&oaxOA})V=s4gmeE^h?5;cLfspnQ?r5g^xjX|_MbC@mi@FZovOYdO?x}XqLLHywBQ}!=jAuq?nr+i04Hin2 zd3Y+GNS~`lWPbiP;9+i=S}r}5 z?|sho(pJoKhU+?Z4wD!w#p&JICXXPq=bYMhyy-c3f^y=zl?%|5qaN_KZb*5Kh<$)(L65yAiSWG zy`Nu7pH3!O^ds~!kH0L37tEJs*6Ucz8VNcc9M9@DI^bUV*+$npl5v~ls5kA}H`Kd! zQ!1aoqE-B{b>z5uLRBqgo!{T7(Erl$+khBeaNI-){MzM~p8E_PACerArEtf}B5!(Y zd5P5O(U;*NlPIwuc$p?nPj3#sa#ZuWwuzvTh=+{OmHZ6vZ$M|CVjZ093>{}d)9({~ zeC!Mv8`_*?`tHATC6_{&yuS<0Sd}*SK+2~j`kc^&ZuV1Q_<{Dr$kGv}D9}HjX+5vT zCFe3WGBQ$gXD2d~FO;n$V|$TR*_orAe)5Fv#hdzkjPsT?=bj}8?LXZAP#MBvyM(hX zpU&7;@KGx|f3$LHn1WN=u$-;VgdmWqs)3YkVIbZ*Ay`~&&W#KEs! zB~iIHTDkTa>8D3TURU&K=>c(m55o1BN&+%N#;>z-8a|3l$8kllC`PK6>c`Fg7S0cb z(hmO*{2gu`#PSrJ$LM~z1-AEVUGyg~!_(lxff+8Z!#u4QdPx@9Gxgo_3FC3&H|Inf zYmq67>PF(8#e1osZJ@Vhb_Q8FDH!jgKO-bdd0e?xJE@2FkWshQP2yx*eK#h3=Svzb zU1!?*?_&)3mPyG@{d!epIT78zY#uz77nu6qZ0uZ4knGtFoFBx_3Bh@2O>7ig_%$g+ zCQKQ+^V!a~qybE_2ET9n{KcAxIO^+}ZQ&yF+J${rgVPr(r|gl$ z_buI582Q3;avpUaXt`spL2)PZUlVh}mi_%B9j9qC<&p%FcOF$V=o)!`1kuuBK3`}? z&YBNX*Qop?nH{^TDrHy>jHD3_B$)4(Hh}wy&JrJGtC++B5}2r`gtu$M>oB=15d86 ztWXzTywp>OohiF#sPW(y%3jp;4wGWZneb3hap-OUcg$wCk7jQ5)cV0|#>Ir)Dhmb- zJWMpZhTp71r*)Cr=gi9PcRI{Qe>x`dNh>|p`Acco9Uu>My3OZ3>*^urp6zm5X4*q+8WMG5=eH83pHGQs%KF3EPL-(qt=vi)#@e5ct2{_rWTb|m^%UDYCC2vseS6B&ZKf(C z=lV)UHkKJ4Gv?4X8)2$vqKoLrSRC^8=B-{vPzk{LV{K&x4ft7xZ=b2LySq576Y}_+ z@;7b~9JA(ndNg$I8w^Nn7QGG766(!Ym&W_B2fnSpvyv>QVkx2Ps3f8?!BS9p=@OpjF;OfRoijBV99 zY~kBnJI%$CQnCU0XYO759?TCJnE@2pHdEy(923sCG9(I%f6g9@xs1FKuC7s!WrjbF zzr7BI_Jo$#4dB;ntmfm z3)}*{fo_<|sH&*!5gat?ayK3Rc&u6fhUL&*3f5U5C(&R^Mnk}X0Fg6%UjN=A=_}~Y zW!zG5{~p97C!bFzEm>5^{v?)|?r-^@DBI0DP=v(GQ? zbDhK7FQ3vDdNTgp#LW5f9SuR?04P&@%0v2O`Pv@c~v^}T5P!^2M&KL1=%8T~y8A$uyMzXC5O$1y51 z3zyQ2{ANnM-NGp=s|w9I0re%JDfrL-Zv1)`R3?Vs#hO$l6>Wde4ymHLOzk0-fO%4Z zrziuHvQg#(Z$@-nT7aDZrYtNvAtAvI8w(2y;@vP$27ahitm$mpZXJ^M(2k^}C&&22 z_{zmiOiCO|0;9LF9JV|-W0ul=kU*n}PD}DpV8f1@i7GgGYhrGS2r0r&aHHNbqaz8W zAq%4uX2lPxig`m73MnJGeq^Cq|>i4L4A2C1YpD#iV0E+t%04&8-*ljFvI< zEOE0cBm^d;X!6FrJ60;3o}NG)X9$R0xTu$->7Nv2m$$UOo)qLvbOC*Cxk(>h6Y!Y( z=hjTZZG zwpUCd>h=O2hej7Af(LFF;tQ^K{PunH32@gTSxwEkOq>4J01zSuD9Z!kKIz$HuSft# zL32rcQ_7}}35Sm^Mfxj)5FvNIq^~hY*5}*O>FDTjV0~er2G|+4CU7cADVR5d5^`A~C+YC3xsUeCuh~Oe&7dGA3W2B*b9b`d~pU=|y;~ekpgcFH->W%vc0u(T9 zfw%({NUEoe7X`nZhZ*QyA*x4H?>On6M5+xmolk4S^SMrVQ6(@$f>Yfohi{aG;*|*P z29qrFcN6NKT*v)V?4F-PEU6R5|L!r>*-Q>a^fWjF3n+TX2ux!YzbtpxiEn&%J-Z6(LZ=_ z;VVf1W&~}7wf0O$pnS14#Uf%=Sx;?fmCT!1&*@(%>k!1kztoLb#z`rm)JGK^E^>w3 zikxy~KD-M5MulkirS9cUgAAb?f?_vwUBt(v->oq~wtZ=LE6n_j3T?3Crn zw*4_f@oLod=n)i=@ksF{=TJ@mwddU5`(`Gyh6q9<=k1cV8@u9rp{=UC~RjwLR?G7zDPJu3f3;x{wceg4cv zFD+7(org?_V9`xhX#A~@U?Ft!{Vq(qq=VQ&mI-Rp$U~Ws#zUEx>PE(9hX3d0Hr@Va zTPW3WTv6PG|8WG2iBhr9$w|l3Wo<1jyJ5ktm6HL^9=KV!I7k)8WlXh0jOWychY?T% zv&#NVn_*6}7zJd;G)KILo0(Q4a4!pL5ELhcJO1?dzuG=`gA6CTn6jOln{3Y|bwuGh~ea?gNW)Wu6hIyVTUE{0W%GMzV5d_>wKJp_sWnx)!cxomir*FTU_XatS~6GgxBMT$RMQL|kFpAI-z0KfY~R>tJ1 zy8smv;e#}#)M_UX4vqqwfT0~I9~~oZ*t)p%=*4oZPtJ?w+_wDn;t3P2Ikq#Ton(K- zW9N5T%ee_vRPtmpVazM>eIp>0Ky|_|sj7ouClGg`TnEdnkM2av{Jln`z4A5OhWy<$ zq2-ic(up@AH` zxq>$MIfV`N+cURfy|^KC2opPtgxprx95rUg!Fjfq?)Qt6BSb|FN*lF!!VVLsjG6*W zAqY_;xzqa%Tv>EIFEm0^SqrO|Mza|$=zm+8UV$zTlzMsV?+tUztQc(0#5a>_8Py%W z?Y<%egpP47<#mwCrGVs69uw1_@Rj{vH)3R%RAXOYLqCs+69y`>q(2CBi!`p9g}v9q zNazX5^3^$T2W$2$dxY<4CazF2zbqz&5e4ZNWb%qI{yxwTt%9B>Isqv_21+Jt{r>U^ zBBY9pE&msrwAM!!WSH;Vp_Hc{cE$uP9Xpob+|rD|BjvrNiZsMyqo z!r0!b^R4Q;Bqk0>M&wsdD$tAymtu*7{xO3ILkmkiCpS0C*4h*i{Z}-X7Uk&98R%ku zYM+G%8Eb%PU`88B&i=YOt_{y&y4q6#9Syi>}Ro8Fe%`98ow^8}z93 z?#A{A>UrdTw^JW2?IpdZcG|qf-Y4fj$HnNN#xOQW|Hz)+u?o;n}z7_wV0~ zjEt3xbn53k3I%_|M^IarOQ#lO=ks!N;lYq#iOIylAqNiVK_>yz(SZXCt2uGXz%=G% zK|z74ni_Ick6&Qm6F4*qM`*!^pWEO!1e25p8Az@>vG^Cg@s$M-fWww}L#e0bcJl4r z+h=IVFYM#>O2v!%ya5meb;k+8F5+qq)t?a!Y*>V8-Fyp7v4cH+h)&l-NkUXl)ayu` zM=c)`M9EWsM6b78I=Z=7ahv?QwcgN{js&raX%1HOJR2g%f@pkl#X&kzk#?QNg?LMm z`qDrkKRnOgH{HsSDvLT%`TkPMAEXG;4P+cGmmd9NTiW^{1dQX$CIl&e&9SW3eLk)2 z70JRfzP-ch%Dw(qhj_J4ZPW9Vv&mak4rx#wpUCFrDiEI`D5{+eb=V$>;IKcu7s%f9 zbVTM~@m2h|bE)!|%NWU&FOGS4GH0*zb zc<0M-6<(__y(SGA929-r#ib?yUX5@^E6@u^Cntw$umytVvM8DPxA8l>;d9bn&K*x% zuPY7YnLp_vSto~rdy(()o{s3jND8m z;QWT{RpI$;m4bgyiNWNYkh{=-=3Q|69OK9vLYCK267O4I>@Sc>sLlANwSM4N0s=!C zk{*LWYf_C`&E_5}Gx1Fis*Z>#Q$wYD-7BNbQ@HQM3jg=@;ULe5rN;GHfJL|5oxFw{ z%0m7vW~Z+=ecG+M07uhVIb}pP>F5kcWRRKPHt0s1XV(yKOjjD7iuW*?dj|CH9_J)q z)Yiy;kJ1=l&)DDXoF85$E)*C%_zQ5{81r8r@vzP|O8P#>PxuoPaz^FfvaIxmoVeru z=x@d+*OhW+3)?fpeXRGrh2rjgUXW-gxd7 z^Io?d6Shj*xCx3Cw<3SQa-(*t@5u)CtR<9pq?@(kk(6dmrGFP}TcBtasa~F{QZ9AeOt-Qr+ zd#;XNE-*Tpxe_97_L~?QZyOK3PX}{PPDaBU4>T5xf3=s!U$!rIwhigL>pwwtzS@vk zaY*GMT~ssM-3BkGW*-_3>M>_}Rby8Zf(t`i#tp-+<9ereWQ>+z$1mmRo}yILiCbaK z_bJD3;BsUK+$--Fb??P};^T?b)EvWZ7Zw)mr!~$0%Tkv^g8lk+3>hY zc*Yipd>+ZP8Rx_c%>oF+4+$t<+w#Ljt@KiAI>FS`(pPntMSE1ZZ7aOF0yyDIU_A4n z^6Das5o}dB>i^=ogI5lHuMFIp+yz~E#IF82AWIvjdUP`cEG)iL8@8twEufz+SQD$92;J-x`SA6`@-5L1B9cK&L@c^|IMg@2+! z@R)V+!^kKr1ioB4e#FjVG8)U0h3EIGYHEh;OysQ(@{ZiWOrOQab)Q}EVC<+V+g}kLNQ4xKylH zi%v($FsxeC%R8j{G28lJHV76!C`s2wyR`$ccQ5&i0X zu1GS7eOCO-Td&lQjkQd;IiHEuw`~nf>_+R*uM1r{H!>4hpRqLwb+ad* z9V&1!{;j1Ne9ki4Qzy1@W7u6}M>T1EMsccoGMd&gf#O`g;>UmkXM2%0Y zzi-f>XLq4%#&DZRoJb9=o=Cw6Cf~bonxVjaIdNq)?P+!N#6vMoz{@gLT7w6=#avmG zWc~n-x+c$WL>qJjku>7`s8ss6rt|?IuSQk}cNHhF`WVF0bnuuz;-iNWQMmH;vDSPc*;D9% zKBN2iKGSDvRSxVE-F>2W16BcQU+P;A;*SnfO5cYrIPKod zy)2Wzpw&%}*iFr@77{6UX5(%0U&czofCMY(v2lw;kq3IBKxQp&kvHa~s~aG{arFZf zE0;UzygaGmo%vIh8G%`B%CLST`qA0hi@WO)uQnjnMRKqZzVk)t!4$(bjEi>UiJdW*h+y1Eounj zrmqvp6o0_)T(>76Bm@fs$-+abH@{E} zk&*fwGUTocye?kOF4E-)uCCqG%9ibc^0 zQ`g;H3?hGe`0_20TP&2EYFO&k)+ag&ns{YGnmrGJgS(G!t}cRlE>yDykR;5^O1UbD z#VH$*A>gJjG571rEExplZ~zUWV3zyisQz=lYjpm8HuoR9@socTJ%RUMDpF5PEXx;%8T( zL2Q(;>98*=-I$>$V-wR|OrCf-~>ms?I1r;1o=)=1Vd& zaZb~cA(^oe9-kqZ)8UN3`-tCg7WAF6APVpRCME6rD&MRCj&%39%)&p;H2_O^_(%R~ zVfi7j^=^YRsLoVeJW}*RL+$e71kWkJo#Q`(o)IPcIii zbHe}ThX^UJ`EVy(2^JtHjt3l@x`A!Ni_$pVFCfm$4m7~4Iru8x0|#9$PvfO%?RT!t z8TT7s)KnREY#ve^olZv{|Ze^?8DC!d`= zD7O14{vg!|`d&!y4QA^x-aoF=RK|i<%X|r)m19PG!E$kCVBN}O%yU)^2`&}2S zU571)yrCqOmT))|GqaRaZojaAE|t}61tp+c1RiJ1A&^6mItXHH$w?f#cdq`fN5LI$ z@BKw@*TLRoe0&`6wb$9mkY)s}uK-lLYwZBukh35#Za9f|OP}Kw(6c9CjWgXM4%mSv zaO=i9X}UWZfFJb31!&Z@=hTMxH|G?G;ql)O>v7;8@Xv&I?ux8K_T5lbwAS-O{u*zd|={9$9+qV0ALWX=EF)vH@@$eR29%D)Y++qV6?BB|b*s!3+PZ3n(dCVD;=j z=$u*E_-NA6Z-+ROq$icS2S}A^_L3&=8biPn6>JA7#-buQeW;WKeEmpunsD6VwJ9g(>ULl^L#u*EJy_a;mLkOgU&FkvT zr%@FvE5_rOn7)m$v~_}ZSB7_?)R3}=cdvmL$XozVi3GKAX&9*a{b!dUS7#2K<@vlN z><|u}id5f>ZpHAY&le&-E`T*jggUD9D$@|3`M6+j#UOTN@(0+Yn1$6|S}l#iXg~2 zx0x;T0rRoBygVwP$B=wLOMSQMcSlSU`uyHgyjYzTY;Eg+x7e+2Zb@=n%i(QAy>qCbQ;S$h2|e^h6BKQUKPpU1CFheZX~Hv?}6;aof%O#mkrW zHVbqBA0%Ya`Q#b}E^UbnFjzOaFo)0HZ~CDtS_Nblm=yHK1S*vUg6j( z^5;45IR~QtZ@ejwGLn&H<(AZ?jYkA}7b&M-)<`>C59_y13O;eJvX7*8+G%(k1?hFs4UiN+gS?QY?S5_wfja6afi*O2DxTmiJ@Htj$Z8#H zw=n!Yua4$JK_KMxpC(cy($q^x`)4!M+q)9M|0i*DGs9quXPq@1V0;!{u@%ohdoHXO zxr``ni@&oVXPngkKDk9Z)-_W@)nR>c*u>5J8;C~iLRMYnWSo95v*L-*IXsJKD~9#~ z$KD@XK<(CsdZOQX+31TyxlqncI99 zTt8sSYa%lhDWK4J>Egk8QfWSU*;+5V#(yL`*YII&lYZB|s9109N}RE$-%%6#aNvX9 zwDD8ewY~S$by(2Yb-t8iQ;C1E2gFy~54`nJ5^hL8m3p(fBF4ze&lsh$)U2Qs7Dp{k zy7sBGr4PWh_WxGeC`7(+d7T&PA7>vs2p<06+CcglwV;lcdSUhr*!(CYYrp^Y{NgFn;oPAggG=I8rCx%>GsHzHCkIk zmWQ>ayfVJ(&mWxIPt8X_`&`35jBeq8huvI(la03_lDo=kQ&4-^*x5}z-hIWh+&#OR1#0={S03wMdl1^BmeEvHW+A$^F_0Ygm+w(_WdJ4|ce<~!uj-Cp~EBQy=_ z2;K7Fr_Eatf{^i@3s|hF5^m;nsRh(-=JH1FT*%^(>yuPSNzMMJ!;D2i8={o37z?)5 z#4(Ljb#oUlEeadJ8@}}E^|7$vw(tk1AI1Rw`>BjKZ{qOr-fe~34R7Bp<=vi2(7*uY z`4Bp?7qR;TO^Wc~*%iJl>1_%YOPe*9i{87BUjEtK^Gknn++L_mBMZ0Wpj zxX-ZcT_T-g>~v3P)~H1}FX&?q6bN)3Gk-1RprgX+a<@fRU8si+?u%okk*`CWX8~Xj}vs z+!WZnWDz{zK@`Rf9#m1hR8jA6kq_{rdPpPWNJC@*M^1wIWasjB=k(g5cA*|fH(G$s zscE;M5Plg0%;0BkE;lc494G^9Y;1qw`pwqI`y(Lp+{!%c*1H=F?pdGQJ8wlpkYFR4 z_(Ohver|P9tZ=nNG2eD9ten{(sv?v%A`zV#`r*@ zZ!Dp@oYuXfZd2`*q}q))-xYA6;t;U&5+n9zIC1KZu_3!Sb?LLX;#+kYT=+3 z+orUN=Wo@a!=Ju}Lw7=J?EDViv8OZj2(%vb4aYO-n7IeEB*s)~Q?1M>)!H|o(<$3F z`1vl1MlEe=_UdldvvCZVdwNm=H_jq=aBm2rB=&- zbrjGafTg%jUz0%iy&FJ@{#RVFX6AM3iI&l=tk2zMGgE?c$i!sqou{*=NsF6#aOj)l z6nf8F$#0&u?OkzGu7;z9|GsMjDJFgt;ZxSsbDQu`DL7N-5cVcdb8~WLsy@6}db2w~ zly8pkbN3aKOA(&HmT|R>Eba1Frc^b4MXd#shx-~D*WW5w3sBD+wVcYPcE%rHd^Jg3 zDFh6-sP8*4I8unr2W&$0f(CJ@l!(}Sd9IJ+3~OdwE(5Zn*B=OemY2&Xzwx}Z|MsaR zgz@h|dtd;fdGm~@OT6hxhkOU-HQ}(ka6gD}ym1$U1AC<+WwgKUR^ie)TnG^7A4o5K z)4OZuhPxElcauWnP#{p!VD0P6VtLS9wMh+HH=W@>X3yQT5;P3Y&fn~RC&1gFsXNz?&O7z8YmG_|z{;M3k+2layl z?D_H?&v4M6aG00`uef#0Z`*j6I5t4!ePDDx(rw~rSN41VvnZN+NNwV>x?iWfzH#tG z3GahVOr1w-r|g~#3)Gwgzy5P&FUlu;bP7#|BQMeZ^F}wcUos~hq#u_Ug4$DlgpQgV zF+75bH(?Mrb(%31p)HqAsZacTfM^mVt-euF$y+`$ z=X?+Z3DMifUo<{^=mBhV1pZiXXjN^4eF%;2ELd4TNK~wTH7Ef=}Z=5BugS2AVIp z%s2q@+{(gcYJAP!XOG`{y`Y-h(!r5jQn6wXPVGSjy^b-S2wie*vFo)t`-PD6*&YqE z_PO77pr2jGMO*_hWsc>tU0>*-Qwme)on8Bf`AZ<yKfZmuR|9Eht3XRSJ3ZYEgkRKy%l})4kA+c( zfTleWU2;&e&r7*ZAJ z$F9_^p1wq3>f^b5SxsHnn%A~#=b-=Y(!X$EKtdpkV;dAnFa<4yhs2FJ#~ozIy!;i4 z^hTZ0!v{AlO%GLvs+A#fyW(@L(Bccq&b9dstg_stZxYIlB4Rgwh8S7By0^)33aZ&t zZGq}jJPiZV665vrOGUP?2qJ-QAW&$5Fr?d}Mvz&t!c22HfMWzncblv*kVrbVp(3}V z9X!9KTkV~btGJRd4H0F(+!W}=wO;|fC*v>AmFgu^CtDXc5=>d49%=d!QL(9p2Z_#g zL~S?Rd7D!GJ(W`N-|(*%3!Vv3crgryw1W~E7%%|OhW0HkyS5xKu+nQz-|S0tBKv@1 zi~%;3v=In0U#z|&rm(T~9rcF7N5iJD{GRH{qNY@@>jC{(k?gazM`9DrpEPLU zT}GHe9hk_7aPMW{jXEe#Y zmgQcMnF6~aKgsuy__(+@2`Ghfu*`FGMDHJV^4iybN-CLHmrkR=*z`7YP|zBOK8IE9 zYC1BmZbtOO>xCoD-e#Mrr}2!_ZBsvw_xoA^^R4`;>AUuR+$@o5G1G^XKRJN20yGvE z%P;bOK&d}dB%pY*q-)L%&3E`ASd244Xr0S%wz40OiZ}mgeVSg)XXEr{Q}0rzrFPFs z`HvSJag}J_$c3JqS5wes@~92)OOb#@9;a-!stS$1NoD!Q5Ji6V!9dtUGP$}xQs`*$)>yKdlRSc(50J&*PAA?Zj4=hcJo5RZN<>o=3@Bd$L8%hC`kgm{&x zi6cD0hY?yE4H~;kt(aq`>y3dJC{H+)v6*0Xcq(MtD_w+8x$KUy56;gSNQEzaV9>_k zvm(6i3VX&Px|{x>{;j;MeaBXAr+i@lbAhJjpPx-0ku87u+!U=fqT4ybh{o?%u^dQhW5XzP_&QqT3Fk$9qv?@l=1o z*-9$8NFjC3oxgSO%<=R!0R$fV=MEMCaC&9<#)cjglkVepYa*kDdSS554qMrm(wMTe z)Rfc{EhBo&6)m#tFFZne@2z=?l>^d84*xW2Ho(>{7wE~XF_!POl$NXH*T%7)w`$S+ z`@M6LaEdH;Y_nGRSRZAyKc9RvF#x-fUJB5=g36_6-#-9kE+v9zIc?hG4!dx0BZXe< z);_$LA&Gh@e8S-UDOk!>0)r|Lo~jtg-wad&>xN(u1KAS@ptF4_x{V29-5gFE0MZX1 zfbeiVUiFAS*!k!||Fd2JUP54AJJt`F*qOY*0>N7s;8?eV3fK#O9M@Z}a}hTI4l60z zTGxN_;_3|gUe!25ho7n!X^to@INNt!E*Z^%Dnh!v5~JL7TzR?tv`u$PLTKR5MvTGBBA6$r<6 zfzlX=9z40c;PL^ur-~;O=p)*JsdXIGLyWqzvT|;GJoZ~D2*Gg&me`)tHSRA>Szf<> zjX%EToA~!ll6Y$i?ci^K!RI*+p(>3yyRCa0Fp!#4GQ3D zqS%=?u>RVP`sf^W2gl!{rV;E#Egc$Ox@CDs zjDUWcznT%9YmkpIMOdxSQ86}dajWot<7mI*kgw2YZ)&}Bjm41t)mz`ME^p~W)^{iasU z*f>VlpMsnF*ydST`B10Ai|HM8@wu^BFZ&Vt>wt+#uBH^QT@h@@}Px=tZF!rrE50l555+(coCLV_-R=gS(6+#xfn8N=?%JRy;M~Pm4laoPQ0ih~jd|`6>6Q|o_30AVPCS+_EetH9d z#iLsU%goDKN9dvh*tjY!ju}>zPZ^9!>3+GF1p0kI6KYYS;>1W6td|3SE~%?P39dm< zi`##ygtfiVr*~Es%F$D3@zvq@CGbN@lb6 zwj+$}mo=cV$tpi0?1v=*zFVi=e*JACAi8_yj^5goD?L}_yHzi1t4xHG+O}`f64+ig zDSW6}a5Hw^Ln327ZNAp6xl-q4YY?{X1u_`v!`I4^ZoWEDN-F+TePM9xSDqx=tVRN!wjEvd%(=@W#l zTJg5Jq8927-xS*e0p`r3hO)AIOOC@Yk%r~BqSw{~>GjK%!SDijV=H&6t7Gh%f|^{o z{CR*~WHpP&#^sGt!L5E5xouP19>9z<5FxKq1}~|w--#F`KTH+EN|Fh;;!ROqTyAfc zXoZG&762%fMwU17>)*hK=kvE7cOHhx$@sT?M6j!id&BN*3}P?B?D`3%vwa+3S+=Dn zuyHQKiA=s(*!lRg3(=+YU3$mfX4eiEeN@WKFS|ui+Q7RI57HB6_tO^K?A60FYgS}F ztM6BTIt=-2@azAweqX!+bDMF@-|DW^C}VVv+s653C)c$`!#y6KWlxfg2GOUa2eR#+{|%r zsY^QA7fA~f{-L6v-{{Pq4g$RYoc^YE-NgDnFT~8s8V16O)*{muu1CY^>MfQMtgNhZ z#w>D{N=jl7?ODa=J;Wl~*ik74IzQA#PPSJ?7oZu0-MH+?ZI>UBzHh|`zq8Vn|hu^{D#|^^ge>&N9;sBTog+<{Y!1iiq};!)1@;8 zwn>t^sZ{vI@Uq_R^I^Wq!I_re@=r-5A+fgZq+99w5##&O^)^B972*h3Ioswc=VdGB z30pPuQF3sRV8We-`uYq7QrQUZ9=*Ym6^ui`@*y5*EyX5>u4N5@s0LUOJ$RYks}zhZ zARs-l8%*Q|bJvVc`{mk7-C8K?75J`}%jQFxo$GQDYXIFFNdh$p;9_`1r;4Aa7iFt7 z4s@!lUgi#aOlq(V1H%%q(OS7AuYr=)JCW3z_SlpxH>F)DmqK)nyuo3f)6BD|)3m78 zm0hP4?D^@Jd$o4+{HIl-j;-RoE@C&n>e{tE6H|M9mVsX)hO8HXSld)~6(HRp=ei>X zjZx{A!O+&BZTvgM>uNF)f7J9d6 z#jXS+s3Kv5Lr=$bXpIA5zIPPcjuJp847ATc@+=@K1~ehlRXfyNa?#6#X2g;QigTS zo|uRFPt%l9OT~i$u%!qS1LA6$P&^6_YN)K_qu6`&E!IAs0@l}=GmNdnVC4p+*>t$! z8BIB8F%}dt7MQtRDF0BTdj}N)paVd~lV8XZq&&04CmCztavlBsyHC^bL%57FkbN`t zkH|U%k>2kWL{A%2i2Q-H$B^t6^dlg`;j5YV^i|9xk-yiUW_VeDpWf8a@uqpzJ2Di= zHo&Dx2}iv8c7J;}d@L?%TLT&@+Hv8Y?t`ar63@qggkJUR?bn5^L8r5V7GQAAYqr-K2N z4^T>VkfUWbT+pOK)iQD@vL?3 z+PEZuVk@hv4!33aRQa`1z|fAIwW;P?(I=IZ?>BPG1mTixfAG5Neor0&)E2++(>cwg z-NZ1wa3WM7qxWg)sWvDRP}S0bG%h?vR8UfqRfI`OJyPt~Wnhnpo)RP{C;w|)ex&lM zyC!1OtUv~XQ`xNUw*Chcpt=A@w7UhY1{H!ZnASH(c3?OPt^h#cNfFQ|S}79{|NV5C z3Vyb-GZA=e4a*x5AOI!n9qN$_+aiqU zGIVg|Dzn9EXy`C;$l?;mL2(sC-;jk#IRv32-8kx(+DrQPZQccWOoGL@44B728f0`_ zoF7P>ghO9VO%y;`lwVXN0|d_h;+)Yx(=)H z#(5o?3Kc18EfN#Yj0jpin4St=FL^L`hg?#JtXxK!((-C6*)B5BZqouYvg4Fn9$2el zchUz_gNH5qR41h;JQa97MoT&fJDR`Iw;V6K`biOaDjWrD-dr|jy5=EE`-QK47FBMb zp+sGaW2CFQDAm-u2U?Wy@3f zF>-49s9wuyRBUF{)DUfKYyjA__jPHflQe@-(lO!W*UfVQb*w{?kIZ6!zS4%bu|puu z94OiSUSZ%}Yx~x{j0AatO%V*Hw?}oCyM<^|QUF61P9MrGOERvGX;;|qNB^zf^0V#! zH?wR`lXnVRG*ZJQ^!C-E^nGS@&w*}Ek&c*`FssG6o=D4lS zOPx_8b-Q`{-Fi2qPN0gHzymY{Faq>czsw@%NO%gh2jCg4u<&ir(>t8!AONqcs|zH1 z`HpOL0P7?ir#MgUL;?HfAj(+;NPh0^HXmuheHuQ7YXDz6$Zz18wnjwx2aa!Zv|JgT zKOXT*M4~*ycmjhh;Mhw7+kYWBYA;NP1TFVLimSl+haThGL$_2BHv*t72f}h9Hr;>$ z4gplw>7pJ*MvB7XKIv7i3;t?f3>Zw^_k_o3iSO&fTmSZFsh;EL3W|!7A{+*rE>B^4 zTD}v?LbSiq$ZHRQdZ;ZaFH+9Ujf_H>_@{?3@m7>5{&&LNAO9jn0kW(Yn7YsQ;zjjy ziV%ZIZn`q}b2o~YR%kM?_9j_YyK_@wnrer1$A{SO0()n1$35CMl2AO!j~~f^sr{LQ z%U3)CSxJzmOCWmma_1=yPmgiG*~wEhSpqF+Ffht_ zFxO92h|RfI!2<_>-w25ra2=qAHT9#VKk-0G=18aB$PGaAe~yccE{o!-pz>OlhQLQJ zPaZJmHqLyXEAaUP-Fp%9#oTEfQ!zwA?!~;3L((dqzVq8`V(oImhv$w#_IA z+LzD5DmRZHgQNJOHU*}sj+wykU2tO5iSq2h0PVYJ7n*!lu-tlYf3Nq{;VV1_g;>nW zDfDK&yqI}VQD&&6Toq~O|D%6sc(cET`>irjP*JlCgaJSwF>c65>?I-k0kGx-B@pmZ z6E{G|3yA0lmoXrBADZIGC zRiU9pgO~tw|96FyU;GN%Q>=_r(Et88D2oZ4Zf2`Ep&(|Wy}~$+naXZacoAs#0=(C6 z>O?&MnwYjzW72XmlGh8)l{Gfy|58|dg9@n}3J4v44pKNIVofwCIEpDFtmRB{@-iE; zpqBe!@7fz04k-SEhtO`dV>y6$y?u8EFgnn^^$zs;E^SCE$!f;dT zI9)xvQq_L{1XOs77m*q|qn(tYSs5KBvbstEeC0@xJOG*9>0BYv0+)gDiHYbR@yT2l zsO><1O%iDYO8B;u5FBtEX})rCKRNCJoD#hA11e`gtQ(ZqtA5bM4q6Bs&qB<}qqkY? zL3u{yh1(Ud+nw0ADWj(x&#;N+-yYn#28es#9%6qfxI`WJ6i7qKq?ssWEn&2Se8j+F z07r=Xn<+~1lcy}Kq1E;YCZuwONN|ep)9B6xt!$~hy`-L$Yk*8+Unt!T0JCDD(%>lB z3TWEEr5=RTK-8-{(%zv7f#j3F1@U*d$Ow?K`4ghu6XoBdS*H-l&r$VnW>`{TUN7b1 z^&8dM)b@YuB92iOj*YLNUcE4JOtfHjCZq5B=Q27=Z4Xo1h@FHt7))g^Xg>bBW zj_hBZ>!O7y7ge^PrYT2F6dXxvP8;l>4@w%!QARR}JcW#IYh(E4_fkE1{e{st3kx&} zoh+vU;U!I^n!f@d-tT|%+o>-}GY#(tB~tvBTVPZe%gYum!}d7!3V68A%M!ZO_>Y^8 zW;Te}uuj#zL{m@F@5x2#p#}UeQu*+WSRheaSkWY65qGO!Kl`Nws;JyIolxI1 zG8ri6PyVGyhZ~WQHRUlWV)djy65xqtRZ!EtE=7jyUvfy8Uf6+pvU-**PA5u_%KE1C zIRmPl&2nDcyg}H$4|f&ecR_iE`->w3;8c!E2zlc{7dUVc89ykvB+8UB_IlZGsmxIgiJcY`O{XUJv|fV6o-Ef=Ab)&A!UN zl>_9nf6_VDHeXM~WT65ft^EV@uf!&a&Y^@{$vmW*{qL*C_~zlSx`KF7!UbtjVo12Y z2Kd{BUrkhpjL1nM{z+n3xqJ7HJEzob@VwCk^DtXpFA^JnD|mFrscyUKg#r(k<&WT} zpd84CU>_;t+jXG)g0~LKsVRY}!{(d(1G0a&cK^+?CowUP8g}D7`{3NdEA?z`NKyIj zywpopT|NF43A@HW+shqLzRQ6F<^G>$a**YzSITdwa^MC2CI;R9m(>7bApXH-#=yW> zxxBRpE-uC(ryP*it!QYz-f+4aIA!kvcLv%v;4*|Fq5Jjv{y};%xr4Y|1GCKl_x`qU z?-#GiJt0Wl#a?vYS^}eE@4!&k|A^`yltKL_MB)faq`0m?LqF8#{PIg)y0w<%%0kBi zY#KEf33^<7J|E`lobvi#|24_$|mn>iH^zebdysv05kvyNJ4v6^OV6dStKH5)EE3UGp|fWm19IJIx}-ZaQBf0!vH2PDJH9&PAa z6p!Z)A>_aHKo^cdyFWT;h}t#Hf2E{3Yz|5Sb0W}T#zh|i9yM^zs=Lm9-SdKP&T990 zA{4OH1cZM9&=Zqb+1SLy&LIN@9x@b?)~2LI*f@?5FxmlElgwx9q@8v~*6ToAAR7o~+StL0IFv5w1Sq4I%p~>S$bgOD=g-bADQbt2rAPwyP zp9=emaOywb5ylyJ%114irO>Q8yso>>R z)ta<6Bo2F8Ca42Pn)Q_VQf)X{UWMVab;>Hr0l4}k_PL4qFLGP({S9D>>R?L|HxvS# zY5=DMe0VJMNPvCL_{7_o89~j^E)>Ap004;&7c;x6u91CJ0h3A}65v`NjE?xcT;N8H z5#t#fHu0-4jOh4ej99Sf2f$H~l&q+7)F%mC`Y(5q|Dodc85K<=*tGx{#qN^5E)&vG zgo;5D4>+cT#MuceihdipWR?_AK+9UYyw9p$oc;Qxkhtws9RU5vI`Cqxz7XrWehy7G z3#>!rUyw-~dU6c%9zCRF+-WwGjkq>F#c%OHxuAB&A!ryQRBB z=}zhHMrn}l2I+2iH~05^KNtf+F0SHSYtJ>;F`t*)Utg&8_W`vmsFZNd4cMR;(tPI3 z#Z26$9Q;T74_54paWM_(Cc$AF+dUmw>>bJ)|N0s9+BNitEcrweDVthe1!YC}Hl26D zL*Yw@3`Qn-JS$Ef91&W(-jY z82q2Z;8DU6PNBQOQ|>>%Bf7c}_B~E3Or-Xwm87(+n4nPAF}OCSE>K;mpG9zARxGl-evc|fdU@VXJCd<7RWW@2TJW5 z2vJIA&IJK6XLD3=L1oEWb1WbBJ`pKv@D2e>IK{Cq2W7;1l4t5Mo$dz`+@;sC0hr6HBVW8}G|go?ah1)goV=wgv5yjI^FR*5)P0*uCaHmr2~lGD zUAg4Am+6LqK-n4Td@8;?mUCY4ldwDEXV!NHz-+5E_hLM3){g!OejrDF1#U0dhjqgq z7F3W<7US70`IS((?b(!pT<#iThHOIawjBWoUzS?vUuJBYkEI_<9$~if5)C4aKU&7} zzpjx;S)puw01b-o+@^jC9ja^1LUd4~Icx#gO;3Ejka%TgIh*zmdSEXjmByfZ=; zglS>lg(PXS%HjNkX%Y5Y6+JMi`Qca|b?MTGb7lnHcER}D=^VzP z2J^)%J{- zt;#Jl9ko;down6U;8H9+Bz~(4iuIrPOkgzafyu@P$U-jG@>lvzW?0QAC0p zzN3lqmMG90q^bLagv9z$6}nI>Eo%9IBF!a!VHP>pzn=eJqiN!TqUH~VApW%R?Kks0OA#5#0{{)6LWJR0J};3^t?<5H$h|P^j@BtE$MUP@e+IQM&|?D+mDwX zDvwv-z>J;tBTQq6@^O9g6_ZkF)e#~7-9KDTzHfn$Rh%f+znYA&U57!W;+=InJ7Ma( zkL`j#X!X=zFti+h+*_8}sEk`G#H3~e4hl34yzOS-H%Zwy1x#pm_eB_g6AsRr3-ScNE)h%x0+4=fxYm zy*^(3ResG=2bHH|y0BEOq94$b`~fTtEmtC1JX63cCN|a&?63n#`u*!(pZC*&?q>7B zub#G>m2iMVMn36y5il7}@dMCCL!fYj0IHXns+X@Rtj6$QZ7;^zSF!7bYinnxd-viM zRM4Ho_!Zg7!5cLNxcKj{kHX{Sc=rLsH6|-hba^+h71?5pENW|NPbehW-7jN zF}V!}ZGj_XZ7#MaHZc{hWhzkRfO9>^VN65|g+#hqe8*QnFvDm(fW|=lJ`o}O z^q>zW)!Y{e6R*7kO;}2Y5%epa#iRZHyD%c59sBj+O5}l$2NVEpvORK57!Uq$dw3!| z6Lw7Cf#Fq0f_UAie$A-^;AtvSc|$|;6|WoZ*X?0oj((}DpV!tg%FCxjne;U;|M@dL zuraGSxq^<_uggFJti1x7b!_raB&noVVu+>|Y%+Ds8X3_ZnGO^XjyMvHe@xfV{I+$B zTk>trHmWL38Ws;Hq)=4Sf{;og4T;+!&7uZu4zN)R$a6-KbMcDXhI6}w0#UA%nO9yb;#aIziK;jwiSb zP}rq%M)ZWg*=36p(TDv@7skYe!20?5574ah3Jbq=zn-!s848*_uqknwDu}^s0W38a z9Z>|g^bc9O3jb4(nQo?eoRNb+zkWC<23*cl+ zhNOWcKG9GQPgVMPTum*Zd;J>R0L^dAcCacdY>f<1DZ*#~7?PNNWX6#aJ4ySCyE1L) zAXum3ld(o96?-fuG-ohAK-Qezey5?Q4FE6{V>dW}hWU(az(EWfsN!-L7XdazMGft? zT;S=9H2grn)UoiH(MO`tw?JC&P0>~}GmbcT0l}=}&*v}_Fh=pR^eXqrTaEZ|PM@h-k{cbquT4DUff*oxxEr2dEwE7MmuamO{C5%B%e z##=!a;xzM~C2vpb&{1|^-YJ?yB{LLwv9cHrU}-SM!CJxXhWOzPL+9Bx7o@`S#fh^| zxWmIoB&kqCxdUB0;STKa!>@j2wIYZN+)xnXhq|FDTeI!VoF9XQ#Y_zgEnwuzTA>Tp zuzK@+#XF%o>87~BeZv5J5SlT(xV8B#1ljg&2q*69ErYTKuMp+Z6X7;fWEji{VTD>l z*TTxR#iR#?cn8EN+6P&YyX9jWg&0KymolW>Qfsj=rdM#D zc2}(RxaSITvEB_-yT-W*BMA1a?2ww7dCUFm91ko?Gk z9tYw1lJ4v9OTL~x^U50Wd$e>Pi>c#{@yE~LkcKgkks78?U4*YL%v|N&9+LB!AAZOQ z)?K9nk}2GOn1C=SGo^6?6B#8_0J9J3WZTTE%BCg&2|-lrMz3KPR_u*QD<%}?S~VYQ zY;3CQ%dAbp8v77;+Z2EOz^^R*(f}YR1ZN8eEz;$h_!5x{Rv9$=PBAH8T;Y=vUI$25 zyz|w0)Z%`=-K?_P6hHFlc#OBT&Magp-;b^gaU8htDt+)s3}6$_-vA1I5yOQ3 zf&nTuVWPjciu+j;I3VT8%*JLQM=bpG zdeLSRxCHu(3p2ui)w{O7{*__jx|{A|3o@tHKq8D7*uIGcmERBQ-3CbK8UW95@H-+` z2%FY3nD+aB^uw8g#0N9wpFVwpB0wJSneGt+;hq5HY&VDS^#z}Xkuh-IQXGu*z&GV} zIv9p*Tb4UQOn*oB0c&vB5L3$`2(gRF9{OJK7x%ap`&}>z?1E=Qdpl94_dTt^<0(0) z^*QoZ9Ofl-<8zZi=`A<UOZl$zuLFJ?C%D@frN|tT@Fc3eKTvems~PFAr(Rq4fjzV`uO+~ zeC}^ZjU>B2dy3}iVg8A$HoagL5dI{~2pdR-<8%oN(V4SddRDp_-S_8&YfWu{IJ|X< zAS8VSra!}}*<_6ZmQ1x`IRLbQ>k%v+{UwR?ug~vG1iz2%-qn2o%bn>SQq25+m(Z8D zx?We9zz1#nhqe$xq5K6zu6f_ItbTJ7RqO?hq@Z`io+zO2dakG)TDKKS!i3EU5SR{q zo4JTDq^U^d)tedsl;_L(_wb zVx+t>s+Sd|L2`vqQtG^h{rC*d35ghQC?=JZ^j(OTI6g|T*QY9|s7AD=Sq@f10_JN1 zMw_@OSV0}60NMw8#amEG!2y{!UG4&996j6i?L;GlVqC2|q6uFKy-LFgXD|Ywpqg>0 z1JsAo1EF)v1S5E){oA9PH`ANcj zQM&|i5Sj!1VjDmi(NTg0hj$=1KRyzKp)4>qHswIMB+f+nz~Sr`Xt~9e_&#Xz~BTlv2v&5cj?==L)D~DQM_G z0xoJvH5%wQz~*``XhOii-i>Eo>engwN$_xE7IA?Nc9u&c_#Wsppq9oMK|h(($1_~F zXdRlG8dIrvf3;*vNW$noQVzt22`Bu1KhwE?O?Y_=WQ2(n-)wS5Lbed8`b8}XObZZ< z)KUVK0CY?xCL1N65*m0VLPQN;HHhVHS(Kj;CDLBuJ%Rt^s)T{614Q;!yt-8*G2!6f zLCy0$u* zrW_5A5aU?4M9}*xa|fy4Ys`s7tTl1LaauxAoH%cf6fd$%Ckt?m%4ad>fkky(C6 zgATz~2TV6y9IO%LZLlkATkG+GlT5aR(*{OjTP`q6(k|Naxy2VH`>eUzBYc^6S} zFl{4J2mUn&gr4_QcUdkm@kt-wsoOlkdR@r-bFI8-O9{?7XYj1u^<3iISanfEgaB)h zz=2Kzr-(7(qfR8AIono6-X99$y|SnFPg|&KY}a&Nb|}dJzk;ci5^8ESY<*_8y2|;N z@3TZfPDuyR2p^@TCe21YHu`Sq3@5bCJG<_RsY**tg5>2`I1R{x$nf?4f$A!+{E!@Y z{(EGVUE6BJ;RhKo63^3rf<(oBB!$AA=LPbAn6%B1j3u^TzB2UI*1=inMa!622r66u zKumf?&u{Ev2VPRSvJ}#++RxDuGK8$GPN(Xae~GetqEyJdy|l^v2*Xh_1=0J`eNF_3 z4Ra%)(Ki~e0xd__{Bs~K1R{meLQ7AEM4fdu+HA~|Udz2;+pigPIJ8m|xW>m(8321t z$|rlJr-~rnZrss4-+FS+jA6#a#rXqpuMkaFT*byaOaTJqy@m1PE3i@df&mb~eSOPh z7vKBvn8%3x58lc5ZUy0gD7Y3zr2B}#y9Yiat-V(>1O~)LgEdzS<4Z9)xhN_Uz_NTj zrdJsbAVc^LY zK{eCtx#s!`g8}Y+gmx$3S1(tK^KB@Abv^RF(lDx|3+W9de^=5mrb)G)(nY(c zRz5J}j@zf<9utrQieK&2>G=WrY2kxzS4GtQ4gZKnx$3LUhMpb)%;5oMupf-dJrJdz5YQYJLVefM0EigfZZ$o@D&Hm>qv8caRJiY z3%$Il4`hmp5@q3|KvwYFd8p1#Tg}Iv0^2iN!n-1Z)Xj*vAmZxPN9cZrzP93V(g(V`FKVaN z5wFXfQg*RK0O0@HEH?t2!i&Wb{9DNJ11G7i4rUa85$GyO0%*7YTwVY@ip(1lSEdoL zm_~$wz0rf4_{hd9f1(kL>#~(BrUX zF%^%h%lHoA3xUHg9d8dBPI<}DeeGWi9EK8nX8^_maMUdV6kbqaNwKgMv=S=dp#Gx3 zLj#&A;Mjj~6v`y>%0qj|q0m=&(G)c_m$`#t>;=dU3^A!wytT7;u~Yk$Bp6{Nj4F|| zCyo!b=Q$Kz1}=tr@9k&ws&Y$O->L@l{Vr)~FGiBMtOx`gEP!bT>H~WPobP9a@VJO` zIp+7E`;cpR5A~p$yKE%SjhuzH2@Hb(!S;&c@7eq$=S*c^Qv;U6V{>N7x%gPHY#5lp zV1Fb&V6iYt0(`gC55<2S7KjY!RB{lY`mV#hjg3u8vMtvj1Ul0;;F@LER}+}Dpg)t7s7N!G!wT=aX6#QcAA$w?#xk>)Gt49>}P3j20T>ROVG&&FbyxtFEoqMu6N zU>f%U{JhkSr2MOMk&_Jw2(=#wR)ip$<~4w)1DGv=G6LKVer7Vhf+!*?Vzi*sAei{1 zFnQ4`9S;BZ6v`iD%tbbru;w}GB*VlL0^lt$*9n8EN<@UCtmwEtcvY=151E*O#UUoc z`}CU*3C6_nJX0KC8-2nM{W6Ula;(XF58aV4;8;HiW zP80}=_g|%pBO(wZAz*wbWu}pG$QOK7U+~^S<69H($__EpDGposAY#t<*lFF_yn)E` z-ydn%vh%(bPTxJ{5NZ?RN<gNR|i_li-~)!Xtiv~A(?_{TeOem2#8&v@n8 zglRf=C_L$kF#Hp!Hho#L`pFh=b}v0(T!RTmU7E^QlUL7VZp6W+IdEC;0ILwh+U@V< z6MHc~72^K01>;{{SvNvcGux0Yvj}Mp%J4P<1Uvl$Wrv#7F|oP%?kUI;;pXu^12w-# zKTTA2tIkT{Uk3@nc%Vi`I6AgH)4IT3ECA52S4+@yKT5s1SbdCP8|8*d9_kB( z($SAUqHMQxiy+k~bkKSZ++&AZlm)OpG=Dyn=B$H-=Gf}tmdP6JQd=|Ok-cwSZ?|!xmkf6H@9bby&0;aRy%o+~(grF{ zkJ0>ApL<1}gMu0%Dtzc{?_K$AJn2pkEM%9?%@PJqK3ptNL&nSfRaD6_ewsydC&bt` zR|gwyB~98aai=XEeYd~o|E=nDbhDknJ)W6d%{RH0OR5wHb*X4N6U5j$(d6O(ZU3oS zx}7$3ym?R}OH!`auTVAYdGa_8CaM9Cy5Udjo>Z)8YeErST9J0581A zAKbn`gRM~j;s-53@YKF&T;7Wr`#NAP4ucOwUyyPQIuXm2*4IqZ?@;fMQNsH`lmrkU z?E7&|UxQQu$`Bax=s=gV`sdffLofKlku)}@1i4qY*+0*}XBUz@s`ClH!26Sc9|a9Q z0IOf0td1?zS!jT551^7j{cYC;=TxfrP@vPpAmRua8*jT%m4p2v@Fid*jRWfs^4{m@ zNfnx7dz%uKp3}Th_$&z!rfEqjHUq!fE=x-~x3AG)18}?^VL{-`!L_yww;OxujMYXSdNet(&ub)Uw7U~h z6L6)~5*_ypwQCK$-bt$gYZF<%#D#k*Iqr2)@KSsRIQ;zlwgJ}{{5>>?UV|!|>CC&; z=8}i;VDJ~^(F-1^2(T{<5V{aRyvjV;3L#)4i0NB9cv6U5*ZfY}yi_gD*R(0{$g$92 z19}oLw+&hGz_U3sGVAZx9D%U?T~Iiu5d3gR-_x8Vcyy4lG-7V^?6q@omUCaNeo-j3_w8hH3kHM0*6!BB z00^ixA#Jwr4=PHFe?P0bmwb-8u_XoY8s}i@eJ$|F+46iSkr?ZBwmPPc$oZ!~Q{eSj za-tziNbOUQ$%vB^K9dAWz!GlQy(ujGUOj4vf+z-ZQ`3eW0ZfH6_-9(#rB0p~1JjzV z+Z!6DBht82Xi4chJ%=LBTPtMjK(~@MuVBFxiXw=_Mw^-zAix;;Sdo8v_hn-8SSAe} z!~u#obC3#vTP#rJ17D^pFU;4bd~qQzqvrcmK2H2a`&ph|^k$kfHdQ~Lgqi-^Y7H9$ zjgWHb_kZVPMjBoFXT_tCt3uCw%L0?-SW3fGqKX@~pXkB!k>{ui&0e?gTe2Z8!~7`X z@`I9AN;u4%Ph+gNO1`US$!%wx{pDv!VT>vs{w)(ziZr~xl2M#NmnT#_$Q-p4TJNGn ztqgzSRZX~QeQbxoi{vD&-}4v5k>hr~%zp2+*YiG-oRQF2<2f1`_L+Ku5`ueDz`67Z ziGrtJ3`$q}$8$84TFg(So^O~P@-mB-K8Y6(9i72-4XmXb-jfl_Pl#$;AHEXW#8{PV zB;*`a;@MXP-_)$uq_7*lL6^3M($imNZZz4W4&B}%ZvDYVN8FVsZEUs3hWJ|m3JEV` znJ4vU^w;;k{=+|J3emTGBw7=LI^CU{1b1?tE6&BHh+|??RPKU%-E) zQ)hB4^?zNL`5)50@r5rmR;ppI-oh==qYsFs37v$Fk;*Gwx^taCCn8k~XcFV+3BG1g|>Nmxm z`7ZgV$C>0yw<(wsH8O+msYfPrX-+q6y0u;;q$~bCb_fz||646Q9Yt3hZBaQAJX~f_ zVi64{>$z4TE7YBqwLVd&9QTwoxbLGvK8m3^Dm1*~vy=N#E{=F9y6GM2^;Oi`Yw}0! zNc3}o>ZY!PE?4NqdW~X9(7*A29BJHroe%P2JiFp-W6PL3^@Sc1K36vkbn{}q{#}kr`h=!ifyHy8>cD(X46Sott~s$9I73!icBZioP2_ zYyM*QyovhDLm~a7?jcvpF&J0MHz`Tg-l1cj_Mf^?CX^8I^Wv%4PxZM|>0dnN zIJG9$I88seZnPyYy2`cK*Ek&UpyqeztO=Tec3sBSZXcz&n@)K(H8uaUu9tRb$?HoY zk6r#0@t6`-E!P}j8~fM;qRU3+x-Doc5e;P*b?3LAW9WlCePNKWys&1YZ{r3Uh{wi) znFn&zI!5FMJ1{KJ}&X_4BTc9+?L8fX7Nz}^*gpZm2kSHm8;*-554#=N zi5mCPUdlpZ)KYAv+bd-j>yBYbog{aGm-A?xngLu_i>l6(UuIX+$yvk*15e_gxk3}-2bn3(9$`e`PCeb-@^EE$F?j`SB5k)B~5 z(3ldbbTXK;Cd|~U#0pyNxq&7OY=eAK1ZUccYHII_631j^WdUwDbbBb-0gNY5kc0Ua zOF?!8sMSWmHwYqv{D8m%Mj-LGFvqlXPgG#XS6N&8Nw-05wb~frl}iSuJ1^5mj}@=! zR-RYWYA@GUljGx^AVoFl7Ql=*&s@VBHQ)fsD z(91elja6xPZ_7A5JPgBy<9Jk=e$CO{97+e=<}3HG zyu93u9S^b#Mc1<_$zgYttS&zH^q#lK#!L?7ikRUs4Pkdge|>i-h0j8B@?N!Kb=da@ zU4PoR7gGKFVcBwHB0V$jJ74~X_kbThu-+^#9*yT$Jn--pLoCDVicyZ=7!jNh2}&&T zib$frTvpw3P@m<_!SVyH&t#}NZ(#M1T7_wEjPRC; z3wEs4QuFuCX#9RtX;s_Nv{ZtRS`yr7+W^SB^@qN~2t3*}pXa;PpFLUPxuj^Jo~5o8 z0&j$f??E3x06k%rfWXw~i-gZf0TT>%AMLE0@?AJ4L4xD67gsb$3d@sA2hcq}NDu-? z$0x-noL~m(b;T65{@r|ZU_=uFaCLM@j|BctWyr135=0ngdDF>&R zAlaa4_RVFoA}0V8rD@|niE_w#FfXZ&E3&t#YeIP&3I=BEcu4*vlLiiCA!twZEuqiO zEeLILH<4I}jGWZPSZtcino|mur zTSkqK9?;BeDa7@4I6dBTC}`^>6t%(j0iT&Ea>WCa)xm-#zDPTDG z9DZ(QS*NJ@Lko!*`|Sb&6CGb+cus&IPEI2UHZX3CtdsIj$Axh2`2#I*Ne9uL(3pP# zfYM`);3}9!?1Y)DD>5V4OYy$i1;k_{SaD)w<_EIZ&*7*mkFE#J1Qhk|zR#I;2`76i zY~-Vj-UCDNezKvKkCt&bw>>}oOE{e`u7XMuMLjp#12PwQB56j|Dn@K6K9gW%(+J{g zsB$aJ_GaNt^W^mH_pzf`B@cgSqrd7; z^foyO@QUO8y$${mh6bKR@$cEm<7E5l^UHZDkz3C2`fr7|guEOk&x)+f$(i}+X``|c zBFml6?OD;8@ca}~qJ%1wrTu%3pzqJ8{(Cd>-RjyyjGNbijF%L^@S`j!w|=4*?)ts- zZy;_b-Dl4{LKA_F{#ah7Ch#U}_J{h+BP{r*G}((+l}Wp$S6a#CnbD6wl3XPxLr6zs z`W!nfZI3PxuX4cD(1|QVL(#gL4E+NpZh_LZhX8fY7ZLA2LOQE1L|lTEy-AftokZrIVd_7E&2U+L$0%IMEi7Bd8yuofP>XH)vpZZG5VJcX+@ZBfBA z1Fp`qz(`(?liuH%Q=rS%axR?2$?w}2(*t*x3|To=6VjF;-{d|x@Y(~sBc5LFr-WKr z8+pREa~_$3@KpDp&qZJ|6}#xs;$Bm^b`$8qx5)e8gkxMhK%7sg_*`#62ZB%n>&KDp zmG4P zBTBtZN!L_L@8w%$2r=9N$rF1uRYX(-AFw1wYDq8oulv(zqu5rdb>= z-%#hGyl7*s#P)Oc5t(iUYKm84YF%a#)}04eXu|t(McV);$FJ$bpw2fPeWT|e=w9*p zO!+d_mfw*0!{Z5mcKL;9h#yZCnlnK>!7fvy|)gF{0Hb;pKqUXZ>5j}5V=xSgSQ$!FJH zrA=m?rJ?(H>{^_z0@C;=n>KvZ4!^S+23b|WJL&fBbo;eu%j9~_xVzzmD1kC`Yr21O zycp!@J^3+IqGAhJV)kGhpEI>zBqJE=XoxEP<9K`OUf14!UH-SMV!FLG&KV{1N$Swu zlP(FfvF`#*Bo;>G{rkk;p+BFWIr)=-v3|bF@ZH7ENG{L{eo^N>UQ<(vU!HhV13D;+ zX%0~&*33~%<2Az-VXJGpq<2)%7lbr+k>H%pRcyG&!;emg>sNu0Y9kz#dQF{a%2%x| zU$%C_qeG^{JFnhLG@Y`4@-OqbO~j#&9N~yY@>>DuaGuRxnNOGaJh-EZ#R`$c+%8&jve~W)0s54by!?q%Mx;17RycI&^pD%wi#GZp zR(jPkz30a}2Hp0L>z7xL7h?|h`^%i$v62A_G@>*ZpGBn|M;4)lQP2$DR?H3o835cR zK!XHq#f~e1nVA_@iRrLv!k1sL!95IL>mxvmQ)50&x3IVvPw*)U zqt?{a)Q}wyeF7R#Vhwgj(_eG5AVOg=rQ-hjH>MGQkmE<21{(kx1*mf#RA`FNWZ3C! zT*7~(<9g3k4tPwDWdQSS#6@5$t*J)6;TwH!G;yqG6c>BFo=Y>j$}@yqa`26y#&F+m z(X6dSOSKyUd* zFYiY`g`9u$+h?Gv($fFwF=7X>16eC)QKB+Pu9Mc*$Y%dZ14y;R-t zx@_MHV}NDF?(KaN^6AeqC~$9kq%g0uoFUUPkR!yQc+zoCs~UY57lozti9l@w$tx5Z z`wx%8kA5hDcRXesc*rwc%E~%%e?vn>4QuDqFDnk6g<~+GlMBDcb!%<-hm7nOlok1Y z0!nl#jqLF3(s5RngtZ3=j<_N_2xlw9azG;myNu)C>ITC^z#VFjg6QKHMKe!qi7Yv{ zK7Y~kV%zS!OGic7mm2s&dE8@^UkIW}Bx^KJ@4lyV-USkrS!DY-(eek&K`Z9uFQ^GVFEp z(dK=tlgD{9g71D#m1i{cbHX_4n667EC{TQs+P#KW+z|3&`*g2Dm0q*sOwE(}Z7z*S zRG5sMG4XJi9Z{kf82`Pb`NDD(=4tgpU3B>J#qQuBee|4g@gjrzvGDWz?rM3owurVh zktK3^p-D;O%<6bp@GcWDO47gje2`(FQqYN$|G6AkMx#DLRfff8<$UEjumKrI5uf-p zqTztV$jOdJvwz4g9XGt*Ka=Mges7`zF@b41jt*=t>AzYg1#!Xv)~<+2-Z>U5pKckM z#AGaZbhZP<;1jVojU!Toql%Q{(@RyDMvV#OTZqv?Vdv(R-9zs&y%{Gp`!eeX`V}+s zt78N>0sN5Dy5N?pK5-2TO{_jKY+QL=im!cCeL&xxr zcNVe{SPmvreNO3izC4AF)T_b5F~Jzgkq>BzDeJL;U(+W$P%f=0A$nO=`OL($_kyO& zvufhJ{Yf3E$#edmJNKMvE6W9!8uKZ^{e_nx7D}`aVpG~0(p?vy{>RH;;`dckv7y}I z@_Te*g7ypsW!g{hg;zgrUi^W~moq`5S&o%1C*jf7d+uVO^<0V(ArWzQs7_gFSJgSs zl~?IdK8QQ}1NPQv#uLNz*y*2}+f_U4<}OY#-okQ)BF637w?r-7Q<;RR3eMQr(!P_V z`AV>NNiWm-kPMsrvJgjdqHV=Gv@_6jip&X0Re(_XyWtZSIjVmB(5Xc(kImB$Rw6J*Y;j?jt;=2C5b zsm0^JxPMB7%AoYDzC?up;aeH%a)m7C7iTB7T&W=A<%bRNKOm%C=V-$WnDaSsA(HX*k`puzPk5PyH8n z{O+4}cm!A0hka4Pu@2(he>D+ib+(m?CW`{0>rajmf^uHJ_&Eee<;PfZX1l(IzwJky zu5xP-waZ8n?7-aZ!XLT(+ak^S@!2FoIeOMke7EK-F&aAPW=e6f%=Uni>c@id!ZPbG z}tBn%ymSyp3%(|h*VwQ>1dT9`F%{yPCuhkh|C4j5?K2RN*qk$=s z)%8>l1_@78Q4tmJn?Y{&Q|8;^$jHd8?d_*AqfwvC*N{{Yoc+|0d@@@>44|mDK9yp& zeVA&mAAl19f~+2Qf5 zO{_vtb?fX-06xg@i`xIARd5jij4MmlV`gDreU;T)KZ!y@8cU}qF<(yw^a5pfO41_d@$w?-ePY ztA88g$6i-E(NPfQ|M-hVFScRjDWteCDz`X4JzgSOBzI!%;I!uGK>JH#GHUn$EuF&e zXDdEuMWru7Vj3=r2FlWWIyUcAO9Q*09`qf!Cl0OrUR;|E@g@D&}&4ctMn-3B! zQ2CU$du2jo{dXmg68eAx>_gUiUH;H;Kl5K20d0)J6N=Zj4hNS1G z=8>6`<|HrUM6{@3DPj10K-nO$>6Kt!`+X3D=DGcDPT68UwkoXhO>vT>BZsE8o=-S8 zn}(TsT}aQRSneI5;zoS2P#X36fYEhyL_$6i#?c3EDE-f5P9FT3x>i>4h_HsDKoHTq zBIA+VXL$Q*ueuf*#?gTtM`aDo6!rGv5jiL)ZqOZ|Axgx>G$jQrUDM95Z$jZt4VD}OK>(SP(5o(yIRE>J&C3it}*|@ojl2%MO&=#1f z(ZQ0mnBRGvr>2!$34K_c0iltYl=`JhEbUb~?xjD?#N!K4x=>z_ z%ApSvF_h+eqWc^-n%Z+j$v+K*ek!zZ5FRTYHXm{Z>n9^Wm!<<=GBBY4Z(z_=wG2ty z=@a2a7O+JLISm(CnCI5g=+^DXOYJ*FhvWn-VrSJVx9U5%gC5|EOEYIkoaoQxKHm~0 zFiNN{6K63pE!aH29h6b42#*p9+3>efrJLob(;q*e86FZYoz9{8Oy-_*;B8~=kTq>S zIQILn&a-u@&OoT}s~80qNWY_ii)fCj5$_Rnk^_AOkRpg7wTb^^A+He~;6?+zZ#$Iw z3;vTBQip;2oegchvne^zDU)%b#T#s7S7-}rMP^#NnOXs<3}cce+NU^%eM}0~y604H z+<4efCONuCxN5Sqo#${#E-cpE4VS$-D!j*oSj^BjsdR8Pi%%@jNu&t{MA7v2Da}#S zGIVY6byo6h1#AC$>sR?536+PA`*X&q=_a`q&x-HC! z{^Xp~SAPn4iH(iB-Nmf=gy=5eqS zM?Qbf&a~?pRKclR-e8joz4+Fn{n{i%4Zhh>N+EFI@L0-JFv68H)Fa>iW_Tv{)9kc@ zn(p3%hrv<5bW;|_c+5!uDg9Mf7bt6kh7P1PcDW$p0`J!6W#wfA0K1|I%h4PP!@~NA zzggX)T$x9G^WC`|d%jjw@19cp+6J9x8xiV^rtJB{XsRH`cgPo41`#Pi#vVDT_LW)2 z2sXQn*vRlr8SWuTVbjFj5@<;DUEI69I~wez4A4|;vbY{Zmyqa9?Ls;uj^^_WD{r8Po9!ZByZg(!xU5)R*#kayu%p()|eqf%&AhuY!h$=-EBGBFii zWFJNr`y9W&!z&Mmma%`IpdfVuU0%@g8#RY|-F|zOL63>z@=bNZDSO=P*O#Nf-eQa% z)n@)$rp%raR?J`c#X$@=+`cea-LQ%HQCQxZ`mX-8k489Cq$Ae7f0R|BDKAc)K^&~% zz4U23=}%BY?DhB{0KQYPh_mNJq^FY0JL1{p(c!R%_>2Q;>?e$PrbyZPLNA6si}_r|mHt_o9HwB-dVI-(I+i6u%EJG6(J9ZggOI{zdMgsd5Z7E^|? z1;3Ou`Qn~`jQYw2C0RtsI}U2RiV1>-jxGnd`4!bha~pn0${%?n^LoMSTB0=_TY?Nd z`q8mb4n20?snXNQU_0^m({?)){MC@rZA7~T;Snhj<|SE@6-Nc<2S> zMA5*DpvPkI>M^*utl=QTE&k`s%-7>v_-UVFr*3Q4jE4E|w`E4lia!shUK^5g+|CN4eEkCHO*zTaeW zvdqZTxbm*FB9+C>)1dvn086z@bIf>mOj-tAG&a$KD<(B#6gUDvsO4OxF&`V1{MvVU zEQg(awz#OFd)Y-C)*``DE=$rrBZJ#<;oGM-V5_dU_#cz4W}7!#VHEz5UyqYasx7N# zd5{@H|3c(fT;3uKB+4fGYL z?@mB_VAO}VlkUYi_H?9Sr188mn#nK(qO_Hip0C1B=ZLQUca4RRPGaaXsG|7mq3iR+ z0Rk#WkN5qwdDln%ocMga^&Mfw{r zZ&0IE@UQh!TGM3y{ah?!VwNK#H`XYqnno59vlGxPhtWv-=F&{dG${L}?Zy!?q;CT- zA3BPx|sT}$&@b9yE9Y(?ylcS%P{ zhN@OG(ws1J#S3Y$T1NTj_dk;&c#DS^ySwAoE{3eQza{G?*Wbs+C&p5kLDYS2BVopG zlS}3<*rtRwtHb+;zreCL4S!7@y0*jlj<+pJ#==F;OAY_Lnhj1Y2Xs#cE_XDH!}(yc zL&DXfRu%y6ntMs29*zIJ5d}(E%+KZJQO=h5>YU@W`vT5)&~EcmZ(9X%L@H*{!w~?| zVseVX_gtQSmY?vA77_7x^FI?#&W8(!Oz;;DKqBE@5~!Y3X2H2+Z{PuLPEsQ5ar0Uu z0y{?fupDs~$1waQ0YT8KsTc?D)oM~mnfO?t1Jwd8N;I?=WWw_TdkJ6ZnNvzckPU{U zW7qn1_O$u$KcLIds{u>hVGkg8KsRypfVA+;l<4Srq?{4d~?NOCq+VhG6* zGAu<(UlF3Pw|rl61uO}-b~%MmAZz6yG2g9Bf2q+9TqItr`tL*soRn#8zZs&1k?Nh` z6txR%`w@=N{e#5}^68Xdjb!nT2q~$92T+UYCi^$SIqW`9jJ=uO(XHUH6Q3q#F;lh` zN1i}i>c2Ip`_WWy$U)lGSDDr%nuk17_Y$dr`Mz@g8s7LDb=*&-@BYKsvOapAXGmKq zrLmZ?D&_JqR=*0mI-)M_1;<+6Rpe7$e7QGuR{LY)GD@iy{%y)|8_TBF3(zisYWw+ zXxrYipZI_BYj-S`!hedK^EzZ? zcdv62#03lhv?{KsFm1W8vLAV(mF7EZNk1glV@*wA|AzK@5w_vS+gw=9kI#sIv|i~D zGg|KEE*xc>Y?L=yJ99^*LK=ET-HfpK`#jw&4{apXlW-v`_0)T$3vaY-@vStu2&XL9 z(syA%^IkO~-UN59Sw?SWn@Xu{8G+x&CB=;^Wxh&N@0OOAT~qzpIikipNrpzkAO7Tt z^3T%2u+f%tpA|4Ai2oj*QqU?lU4t{%?KO+Y)&N`VyN!na1}OiH*Ht;j&hJ;FMks+ zRY^xewnR_$%3q*%KVeoJZJg=-2W|HU<{!A{M48RnWWg?Z@d$w)ZX;`-!z1uw;HXLd@G)F{ z!BLQn*U%I=`hksZ{n+E!=oaoNmGcZyh;ejJM5c1Pqg_X$5jC{!BY^TR`rs2P^kUEG-WH??<#Y=Pf8s-G|8yt5qD%FmR$N*? z3P#aCSul*vqcsDoWjO0IBU5H~a|Puuo+01m+)7&~uDjL0OjE~kJf`cBCWaV!-crJT({#l=xVH4UM4yICL~0HC0tsU`+(+Gc+_bS(TMBmzS45 z)j@{D(#cHQ;Am?!ku#Pe@Xd45W(8SLry^RpKLdEcw}22Y_megDJrO4l9?)1f z{`*&1DP)g`?6UXDO15l5$R;6McJ?MCdy_4DlS1|$$qw1sd;j0}^Zorh=Q*9z!(Be< zey{6#U2j!!mjbGf!i(oG2}bF{+s2G6ouCH*CR|6j5JYtiBw@Zz;I_hr%fC@aY_QDI z@au0=GJ4inuWNQ_p!oj&a|;R#aQW*NW_K=9-A-cSWBR}OR`(@2S-=5pE1={)1;?j= zTnzz$?SV*0@IXVB^!%{+VhZKfKY3}>o?NyE24WWYDb-rO?VO|w~ zPwvI{`VXf(hglZ)Y#v96d80Q31NX08pa@`R=tx6BuQ#*s-~~={FQlXAXsh5ZeuU6V z2HcmQZXoaLw?!T$??c7FbhURFS(FInSykW8%WGI6m+-vGYUO)GWNuy#{=PWMqb(Lz zpM34F&Nxxlr4-r2A5puyH)h2!{HPc(fES5zP!UB#5;!eq>#1P&xgv7WM+b|y3cQvn zW6GZh{GnorW<7tXHvB@&nkS24!R8{m%S$=dyu6-`^bmi=Wa$WveeFn7rJ*Ple+=W| zUtiUWWCDqj*1_KX){Kc^Q|FlY&oY$wFl*BNySzeN!iB{ILZkg=mouTvCaab#?vPf6 zzZfsvjBb6#ZKQh#lm^{@TYBY^#-k}4JgzB!*u++i#p+FbgiXSf4)A|Rh77Z-*svQ% zLw{s+nh)UxcMe-{%lM#vbZSg)`q$@&)VjWo7|NR!I=vfA!%&5OqVP8_Qa&U143iXk zLWf5pR2&67Xnrk{x+ZdHZiX%Fl~q3Z&Mwn33RS<-kpLc`FDgEuh-rIXO5S;58lycq zH>04X?74VZQza-Ji#=sqv@ zQeop@IJ9ZwCJ09dO_jd7W^!Yb+{{t)J+!vwugfgMFC_9dC7|2d+RA9+qsab|F?GOO z#g?B-CiYRL##a(?qHcIA%-%cETZR4WXF;WoPI-V~ifRV30o$$Wob4}d-l3G)q?@b|<0 zHPu5%J{SH&T41E?WBc2-iEPWL2bwiGNE|TIW6WcozE5>mF%!G}nXL%Cc>Rfz1=L z#N3gb z@3fIjM(VvYHXqPw8kR&z8&i_<*f+=D`=+FjJsU>-=H#*KMR(I^0|3}tHk~nN#w)Zf zOW2lpTG!Sod-mZ8Or&;lRk35{XP4o`oGk1HY(GF4AUJ`c^C6JLBU0{acLH~~%x4bC zknG%f>cx9)DkYU{qaC`aC_GGK0*8B-APj09yBF?Ihp)oP_4l)d*1l<&)+?G5Y!i~} zJ@eT0_ypX)%}j04yhdO*h)T;gIpuS7yA|2m8gFR4@(xcKB~&j|Mf2;==v;%f?(S78 zQk4%EdkjQ3BVkV@#F?^oyARmIOkWzFu@s;2MLbYXa5dPuRc9l*;G36~Vjn|NDIxy7 zsBGQdd-7G%ClmF2wk<=v=@$HM#$k?p)#5+Sj7?XXv5QZcS7-DkrwinSmSa_sY{&~U z>~>F*&n|U`mOt)$+?PFG=nT!s@HBgU$Pc47gxy0irikde&N-7@N$U-ose~@g|EMJg z6eH=^hxAB`?7=Q(gG=kY+eVOUIs+LDA2-4{zHMMc^frFP#+nzEDyA(L4wvlx-CFdd z3X67D^ZR1QJZ{YGoHfqGR|hoV*Dr7Ev5-VnzLxgp?G9sjr>7Yi|G@BmIKcZUCUbjG zj^gL~tFOyhiBHdNW-e^Mrkc_8XDv{KqFD4)8RuA-o5vS0jQHHX>WyhRiz_D5s#af& zk56Vx)?XD%EODc?WbtQQC1qP%nEgWa(}k3ZJwZTHiXm(FU!(0;k@>C~4_p^5Qb|me zbl-cgg5i`f*uzYwf}bE_5i*DwUw!=xs9}UgfRdRRQyITyv|-koSCne-sxz!Vnb#h&f+}D=chhwa zBL44MW~o1+78A4Sf4_NlMiKl%v(}ni{R$Nq!{7Dlo70kp0x@?A1miS>zfR;oQiJiM zvpq&pzOLxe*LV5{uLMbcx6pMPdJR{wKH09Ct<`2K)~*d!?F|3&tsLC*(KN5Gw%+Xg zFy|QgJrdGwZoK>5Gbblu(yJ^Erwx^UrRcq7?(9m=m;R1_>oK@HH=p0}uw}d<2;3wE zL2={kkIZw_h|O~MK#}hgjVHGIYs%moh+1rQ2P1_q+St$nb{R~~eRO#IlUnl6Oypa& zc5iUYcXfwokxrr5BIGRp38lRuJ;#V65hC8fn(MSZ(+yccH+O28NQBcP)!>x={my5b z+Que0XX|eq+THV~Miw`4zebWc>!xXDkb&A_K9n*1DW5Xp9@Sg~p!O{2m2O$Gu)0i= zAm-4O zdO+LX6PrhC&*2Sz6&p4VPgn=g!UG7CW3rmaXtZ*n0A}TU^ay+i^YXTYsGyFXXl5N{ zE!x=*9V0bniKMt{GIbOM_J+PzynY6oVmM)t#8R`)UF|9IgqIyO`+OkiH{>y3Pd0b1 z>+X@Jr9*Ew-1AHI0!spJIPE}oF+sQ@3?d$}!Nz-JhG|gx+V%6Tfhrgxb zYw!4qr9U77e9u1?Y!e9=*gL~S8#A@FK~q4^!ukb%h3wIYiYm%co;%=2dxT}fF*R6I01B>J|F zo=6FpE%8Cs#U&CdGJO2}wJ_+9d5H0T2Ny~^~MVUi9kgTVI>2v*qm*`SiP?Xz| z*=--q>WP%si5h&H$`xghFY1?hX=2+toM9#iAcc|QI6w59n)1(!mnrQo5SSd!9Mrk~ zO4&y;djY(}kPYx>-+?k@gtw*?q1Shog5{WaAePD5{<4UJ>h(WnVV%*wESpE zgT1$H_jlf5^D(oV=#Gm7o%4GRq|`OLkG?rIcfyWTN$Irenk6@GR@|!7f<{YUb086E z%&GXcM5cgv$p@{;efltM#y5iVO9=nn^c7A8m5ta6SBpgA3I+0s+WrFjvHzqef9K`8aecR6U^ko*^yHj zu7>r?R==ogSM`v|mARwpUbrSFSJf|B_g(q0&1-!bsvE%>jKCJcCkzwAWm1NR4o6aE zMPW>Zzpj>j9q?Z{N8eOfOuu25mQwyg=-k-1_7+v+TWJScLIm;q^9ykt#9EqLO=zOx z{zG9r28PiQccsOg#Es)D1ApwBT!GsuF1oyX!k%AoT3*Xk72Vs*Z;h$-uUo&%_(TIU z8a=%kS3HQZuc0-zD)J>7EtX|W!o}#^I9mm>;_Rc9{Wq_r2Q?&+d*>q9uMF`R%Bt~b-fnp@pvzj z=952%J#Zq&X~vqndj387_Aqe@r@A5+>R=PK&EPm)ie92qUSC!DhHJbP?i0qbvOJB& zW4UnRL&QdYAk|d#)1Rf!wFoScjIDsF5)ju^ITn>z=v{ZczCar=rjs}E{ctdv+MR>Jr)oyns~ng zZZwc$RXAQKD3E$Q(|dzPevRZ(zd^kDW6Bk4oLfchQ)j}`xU`^F+?nONtb#BUe`{=U zG)$cMpiW}Ce0id_~`WJ-o#9q-)e>*h@6;D@%zs-w~3qpRL$p%O4{<;_i{hu3{ z0H=#ij2igvsa`crW;yvS1~2kqMDLQx$Y~5v+2E2cri{P1$A^YJF+4MMJug5Y`v1?- z*H)m(5_g)E6s?H9X&Oksi8|k1y*|Byw0e5lILI4PkkG!q4{RX-rPqHza*+2~*`L zXvTIK#z!i$Z0f%G!$`_xidRt_y@p{iE|hOX(gJU zCN24h+9%%;djQAo=SAZq|I>GsXdp_5i?Z-h|5hx~>Iqebkg!O30z!6|WjDEXWb{(z zhdGzh{D^|O7DwXL*ovsKaZktr&Wk9xzKf+e{Qw9(Iu$L-?1VHQE-i5g91%i#N4>|R zeH8Mji81ISte6NpxA*BJn5w2{!Qr)Nb=Czj)hcfkB|8CZ3>$p^*Kae?32+P~o^|U7#wU5vX|?sl~3*d~qNW;U|jFHi1m; z*ZS0vEO*f1gUyXbM!4`3ZMm*Y*4Kj6%ame2O~_@6GUPFmVRvr0+YTpVDq@w=t-dg| zjpV8P0tU&VE#d9EW>tc>KrO{JQKr5gYU!YC{0>!+}_&V&iDD%9}UY?~vC`u?)pcjdFZ81(vY1KdG0zZzC3wJVwYd+XZo9%|D}vtmeB9_9%f!#g@h z@*+GRzch0!>z^e2T?sx#=;%(rYHz|_xV;R(@h7mj)mP+#3x`l_tT%(UF01tIXu7{w zwZg_f$vJi)J2ePr3<-qE8$AIL_bL?necM^`6qQ3xjhbhYarMzm_9(Okoktc@I#`uW z>yPdF#?6ORl3Z`Ur)UZpWu%~D{qkm{sYt^km|g9RdGKGYuVnTQeYT;wIiZgZe;A__ z^}^UMo^twjuDNV^vw{A6Z@pKUZh$lTb1Cgu=I$?wY#rBl65U9-XOGlR_pkV+B20CV$B;{W_R-yNLKPM659A#54jOSixbFMzVXsLn{jqhkZJ=4ycv@2g`H7UuNl*gIP1>1v9UZS2<~qFk1<`O52BuS8xMsl>gG+_AAGbFEn@cbRMkoim+!+ zw2mGu`DNP-RBfZ3LUf)@C_QYn9|cN1#*JCuMN8ygxG0~vdGoQ`Shj)wVkrE>vor^a zqsJuRy9O{XMFUAEexIqm%Aq`k8gPI?NA2R`0#GBHk#c?=V^d?eRrddAJ2K8S1JrxZ z#!+%2mnT!k)OBt3G^_^YNlU@0RMD50UaCKpW6Pvfo}}7AHh8S-_Jpa6T9DH>{I#>q zdDxTStwtd|sKcnu-VNuH3dHPTK9@R#EP4Y3M5!?hrntyatU$!kAF`hU;q!9s&Xk;t zf$*EHnYMzZ4nq!-Jv5(Uy_Q7#Rj1GmFtQ^|Xb`9Udw*O3mdONPXn~uyH$Kt$wt6!0 z!@fC>)%1_Qd-RWGI1ujjLz0+V@R1XKRLVWF4cU>`57+RJ{^$N}304$7Lxx@Yq**#9 z9Nmnr&X085_kO}PI2w-hG1nUS{z7OQ1q|zSHxLIqyC7LQp{%ML{Y=C{qn4>xvo9Cf zpqK#z?V&cVdUwXuRws&ggnokOXIie>$pR+E_Jw`rPsyLvJm&m;9s6VPR}88Xl<}(t z^ps;?m^3pno(lMwS8Ipt3v}<+i9*R?=)PafB1a|P821)7z{%li@kH6f=H}G}zcwX( zy5!aq~D3q7xrnSP(jVNg~cmz#t)B!OB?g;FMLY{?4PMrx5OU2HLs>;2CPrF zeC^`iS$A)bbYaKbt7&PIw+3d_%{bu!G(^Bh+(YsC>)5~;1k&jM4U4CtZ_R7i^K+jC zUt$MPgWU`Y6KeAOMMzH; z9F=G-5k~+wS^qN&$8Vjhd2Fo7JdL4AA<=qq=hkn~-WJSa{pUE#d5yO^uyE`xDX!;4 z_p-!^$&^$5i^Q4(EC?RKLH@0MUH5vJ&S@@BE&oiy%K*PRPKZBop3Cc%p0#9J$nSxrxFf{ED;h#^W4VK{)*c&aoLqNJAARaM|1O<6D^mO$0YVuh6I@ww zrkJ98fIran7|S)@)~O{Atn%PwJG$R5?E82!kDhraw0a2Vy}^rZ{;B|{KjGZNB1Rm& zNfaxGg=j90JI`21PMr}JdNHlM$6W>>Q{NoNLP_-c*W$Nd#g1;5V8Cy-Z?MH2#!?r3 zOKUV{<;u8?6Y!@a@Z+7(4UxlLq=tPh*eajxYJL8_!~49G=q70IKtx;XP&@Xn8EH53 zFDkj+YgJ{s^mjQ`Ud|L^%703vvceKG&93rnS(?OYeHx!G21brj-(@S7Qa8VP6g*DR zQGd9NKX)pgQ2Cy%(m6YbTU_p_?C*7Fk|HwGTT;ovwQqUH)@paNg&s;+Ul-9DB;E|$ zD>JQ=uIQH5{P@MNbix2)bq2Gme+*H#y)Zae>FMs%Ro~RDjT2WO(<^YYeUNu{%YN+N z=}GvP9y^z2Wvl*%aXysOI4o`7+d@2UFp|?L!293x@ze~bycaDvt~{&+a2!sid<`4! z01sTE87F+HTUslXgM?8Q-H%I#CSf~t*8_HXr;3lcxj#$OM4w*#jZrl1j35tgF?%Ka zLS{o+xZnoGk^EhLcLI9I^P8|IAB@7u4n3l(zFo%N3S<11R<>ra_HoJO{YC2O9bo{! zMM=kk^UmmWnde_rb3Qf6Q;wG=Is&`Qh}RL+y%+sU3wj_tS?kOnE#5DA)ct-PUGpZ& z-<{qxBDjNkA1N9IDGkfNR)nbTT>glEdWi1epPTJ=y>@dc?{;iZD28}p^~{Ys;__=) zZT`V6_&dzOAGU?OL(M22m*Fm)4F0U!kiT=)Q}ufmqm&sRs^y2%T!l`-c%LHi9~6GQ zgKU)DyB2x+q7N3#uqErys2trFH+?z47SG;R)l{kGNVOY?Z)i)bl)0SfyrknCfs+ag zm6lrhKgHdy_$Z$6d-LbP+y9t~UjDJ8_yYQ)$iNm6{+RhIX2~S=Hj4CVab*KxjdymL z-y=yrfBsxIYh9&nHDCL9!(;TQx$6DAuw(?;8O*)ECDQq_c)^;!=h`gdz!|}>S@NsB zv8Apq0sgrgYnhJLY=sdEeEI^m)&Upw7mvJ>~B%3TUVd_DvYXpCjagNsI=T+ zS%Br)YFLhwXVd9Dswf6LMUt3=U#9jMqZa%+^Id*1k}+v1|po9XTYyZ%PfezhNCi-nTe|b)V8_qwW zlTPCy)fp0_KMqBSd!{i_9dS+=b`^ULrwAEW)892X1qX~SA*SHzbsmU(h0{5|);d?! z6hhO}6X<+#3DaZ+#|T^=xoW1K2xVFMP>zfl8Y#7ew4+H9?P;qj`!L8t+?pN7Xn_?% zR9tF5&GB*z3so)NFwXF}sr}WY0$ckd6 z`I)o_L`Y-K)`MF|s=8^#&+uPxFFqcRe$Mr%(}I0r9er^NeWm+|MXsJ{GD-4O@357;lg`z`G+eea<)C6IqqnsjNZIA zP9C#JU-?5g5_a}C=^c{hhyz95D`8|&U#~mbT=?1nWs}!W0SAMcK|R-VUb8A7aeKFV zQ>db!fa{!B#Zo9@0jihn)fQApu6m)T%Pn@-zA+f3ChD8%{3Th_Rz>vR%+K${2&Q5P zzdI!`Qxo!G{nd^Z`2Nq>$ZMHlll{5O&3tK!ep?qJ8s>*M9>wpx0^rk8^lLZ8p=$T+ z$BEcIzorGd_rnRy*RA&&&1_8j)8E4AEdOI%%Bs^X(6a|V7bnIC%BRmy%of^SwSjK` z3EF=89(KJqyL_Bx13A^zNBNuQ&chNKLI-`E|E8%b)~bnlg$%xwsRU+Atv-24X%oL$!7#5POLzEr!zPJ z#LzTx0y-FSb78jTja|B|nr`Y_UdOUw-t@%zYkjeL2!YjYX4j$$fS88$vBs{-P~W0J~w_|4!rzL<&v;$?+R+N2aTPIIe7q0 z=cKvYleeCg!ZE6H+kb32F*;u9^+x~NW=||fdBtA-m*-ctP#egnBQ#l3A4(P*hAv@# zd$w5ny!XLCv8PYEUv}z;!hmonEpWCq+Hu_$BHr{a#=4$Q9M;v^yGz z#4Kl?>;Ze`pD+{7k(FYYFASZ4a`X7yn~$RU<%`AMFaX=|P%u2!;Ku5;Mn{IXCV&NQ zq?m4pzRvc`2W8qN!G$1IzNj=Cd^G32DsHMEMDTI&Rk7hlyXpgpD9qP2syd$(#Z~Wp zi@@^9WK*R}leexBh-fxl(l~$hYVpVs0NY`pUIr=}v5*Dwo1Hl+M-!Ll`K+?IJD!Im zcP`W`Jx6(R<)0<)j~HqcPky4oMbfM^xTlw%4i+W4FQZ32JJ>e)HEH?~V4+OiwM{mv z*I>sDt|!{JXYq6Hf%Yo{?b{CJhDWXg>COT!_5AL;P#2TAB7plncVUi`H<%zV8kbtV#_+Y}`rNDh z?+JEf=be1@kKk~&tK^WbbQ0MG#nG^NiqA~3o^cEqY{#__0oz5Dooa> z(gv3E>?y6~8F%Gnmyw>E2AZIDWo|+fAwaoi0K9ux?nxG zt?C`ez(*q6#6sGfC^Z8ZG$O4$&#*w^I~F%j`9d(uaqZJc#D{H7=5Wx48k%B+#6DL-4BPPCR?TArb!T z@tL0yn)lF}dx@ik&_4PfIdfDl-U4QsRc7hiyVluWD_n6rvxA5G^GKwNDnSU7Q|-Ae zmpS5knnkvh@t!{m*Xa3PI1`^co_OUd2irTzte0u#LRn-+%59~Ide@tNl$rN@$X4Jt z1SK`K`OvP+ENgHuM+2<`@%ueV5~5}7HM`ni zq>__r3J2lrNz%#LEa$EztKn4D+nc?M{2~<~NR4Zoewo@*#^K(jeB<&PfT^N&suFhaertOaz$_X~ z4_BLB_r+rJogW+ziwJf6Wq%d(#_1n+QFp}*!qjvEmKwVF>*$O>ro*FK_sP}!KgUok z{l&?%9XNfn8&i+X;W5yC`Oi#bX`!ZlrK&NBv=qN2EoGn7pnz~oRPYSm- z95nXM+~*H)&aCr`i{S;O2G)pGSGZ^VlVq(Axfn_dYzF47OCo+<0}>)iuGyK_XhC(_O2ADrLQK-aMi@E(5?e7|IvN@h2ZXlcg2 zI?*jX)b?atvH_QLVkZ{;h5idK@)IGV<;>)MCM{Om_4R=EgkC{8|tr9Unk6ScpxQP0;AGJX0mM`oPv-GE{r7O zkwi-;P9%`r@0_3PD8dF6>NNs1?p|*}0Gtwk5G-8`G?VdKqQj_&V2r(+qJbmrpX2i~ zO=h)Mu~F$o5uNXUSEOcLK4Xf*Ptu;s|H`!597)9f_EOW*iQb1nMK_bs?yk@G&G%m4 z$Aa_9{`ES_r*owtD)!geu20@qY{$gCLTVGD&<9lJrphYh4SkCdnuMTS`NwV zxoNH#khfhmO`Cu^zm{*CbDfuv*+yKke*RA$b^2%*-VqU? z9=b=rcxP!><$tuZQI2DlxF&a2>nrY#pLpqXe)^#YLO z=4dl_RrpMA<}ETfEm=8%`8}Y5>hquIhOYh!DuLT#hI=2V=E=r-%n&l5Zu}KfFyZo9 z_}g0)$GBYU2Wd0~C9#qd7VA2i28zp`HFXL8;WQ=j(ynld(pvz<;Fjo-B z)^dI*x)OP#xDg%F)j9ey>w1z8B*3)O6WZWUaFl9$cUbd&V6S*cH)woI+!d>;1VKl& zcGN(l+?E+YsG{1J@KREq?hcD0nhU=cEQa0c36P+g!od~3LJYe~xEia)5jr+J4yt=b ze%BUK+{rA=P4pe#j?}~d((I#i_pGK)PmGc^Z8&?HJoP*uP0wP$a0b~yi_oWa z2LZgDSST^J07#4hm!pCL79PFA-yN@;i^u@7>sKE(3i9eU3X4wLD6V(k%tlb6H=V5{ zhmmliR!;Uk$xCTXU^n`(PFt=0mC@`(Bx?B*)G_w6juU(5o257D6Gtr{si~179ML|8 zKjh_bJv0wUNeL~trjpb>>GS~f2*{mm*1y-{B8z;aw)+^q#Y*+mc3|tIw*7tj*z6BD z)~%)U6MPh%>DZ{DPt?xU(DKxA7R>v6^T_vC7^mH*tGPjvG<%++>FZW-6lJfI@o3In zqC)`|E9Ks+J`j2~?5gc6xZ63+*SvwFQF~32`U}D8qMyRY5!4(#rr9=U2Dlsj>~Cgm zsP0@A-;WSdp4Mc4ZM9jb<-$&~JkaI>2Pmq|shDMt1v8%()Sg)!p1kQg?Q=V|=(UEVERLs)jFXMdJI=y`^T^?n z;w_$qzR&ypu80L0r+HZj86M_S*J_H3)lcPicp%uOd?#?#BWP|`ro&Z>PjsojYm~?B zxg>notf=pp6$Pw~C_%O#4l>aZqxT2=Z+grec3Ej&mrr^Cg>pzXTEX4pzN;3yNwM{f zW%ES3Zps*#ImKhNL3_ujiL6BJFpgVCUx(HjYN{ZBYfJ^*S5YOs$hY+pN#gUbsyvb< z{1W;=tF1b~JHn~Eenp=h#@{b3xvl)7J&;y|Imot~Ro(DSP@A{aZP<1{2>sr3V$@eR z*0;#JnBrBG_~@JzA>Gcg54$f^;5eAWf7?7}y)n6U;lxz|*``w)9{;9m(`R3kY96mkrg>evAi^-LTdtf~ znh#Y5Qr$>+`(3;q-DqOM? zkbpA0gq$~j#)Is*_Fe<2=fEOOwh02ft`HK$b%!;F2EF-P_u@h=ry1=>B_u~wxw|;r z>r^Q5{lnDw-c?xC-z?ttZY<@<9RGgNi)>1EC=l4^pW@*o_FXY7LSDkJ;edKx+&xSLc;V^_&# zoKsh>sd_e9FX?$8A0}rC(t@|toThyl*t2ZJ_u)c$=;}d{7<%sUS|c~&4Lr<9K&1hE zOQ6OU?BNs*cSZObkdHmCa^44M6q!IQdDGSLG~nrzxFHI5i)@vg+Yo*E%TEZWm<0fm z!@`jhf*^M6BuHRlQU3dDGg_ClBCQB(AYzle7t7vs_2aqe%Va-zt<|(EG7&?D48rG3 zIu$mCi4^bayw?uJcV{ko^ty^ z&w%@DQr1p{Ac84%-Z5$)`oX04EvXw%VX>&RhW16kJQ@r<(KI;WlI5D;>P3t1{hvx& z#MS+DArL9W@s}17f=mz4GWb@@q$pFL5BrM!Yd>V4M$Q_}teZ3mqmlU!j?vmc1YFpI zVS91;ci9+Rp|&pWU`2{9%t)Rl2aXo;p}>d=bCsYF{_^sWslB5{D?Rqe*`=7k<&PJ% z4h&>)kIv1ZWEB$-m=(s?%7pP*tgq&GN`ky3vNB&GW_>NS=rfj2X%@7M)Zrg=pef1& z9+wxJpI*^VV=jGrmwblxuVp*W#X~WbqZyv`y=1ts#Ns8O^=~({{rr1d%%@}VHbBlL zlo=QYI+qS$q>g)m+Z*!SLsgJ}l0;?{_7DXc_fVVDpa}p)(=+m0;{z=R#8p$NVB4iX zeiLmB*HVN?dYFV@$X}n~xfJCKINt(x#L)KfTM?vxaCb(jzdJ0entVqYw_EyE(N-kB z#Ta+Ba%_C~=vHec{h#N_G)e#jhd)Q=utk@2rJ=LylP6t2S9X=OW8?W(`|&kj~K&60Plmt;-Qk6-zc3V@o9rr@jUiH^HfDQj{QC0gPJLV85CKZwc0j zD)Ze1F~sAE1$Q!k;k2ZP!9`k<8f=g z+E?@3NcI<7wKuApHvs{t2vlO#+tCl}U*a_z9G-vyej!3|o069|9{9V5+L9n!y_mld z-aAQt|LyF3b!D$Is{UBMau~TFVbOR<=aNBx61R@iqzR-#*)4_hUaxV|HXh$V{=q4c z{k7ZgZytj2Z#?+{gRJ9uwrn|Bfuf9$VD*9d2@;($+&)r?+^j|M&!gz7KL-Nyg-Ge9 z*R7j(I7r^2AHf~9-L$RfzWW0$cpa)FVr+gF=$Rz%*O_!$amFs+pl13?I7qJu%d*(Y zEre>*29RiXMCCCmiq9S^6rIAHzSK6AoS=I~DVa*()s)(Q{E8%vGZKDEP~Q_Zyh6aS z3cl1D{Q6SacE&QWCU)Bv4ct_d+=`>Pzh&Lbc{v#$Nmntl1ZrvC+$$WVXQSD4z+mdG zX@>DWo;lK9qyNqD+i<4*=HQrnr9ek(3yBknwr0|QzSmzB=v4mWnCigdI_bqs=Q11g z96jL1${r>DZR{2t{Im7}0Z6q1KM^Ea+7oOo%;IYY?nxZn=_~QP?^{zFGp;viR+n83 z2XBu0!}wHXv_U(`@$3YL?#0Gxt3kGBtCo5mi}nNjt7LZe_EAR$-)cM#-=}S5FF~v% z+Mn?WwTbwtR4OY+T)N5tG5W%*?hp8Lo=iP3>6hg=w!EVf^H-<%Sujt{u@t>MKeNU0 zI{xkpAB@!*V+Ff9S4cw{W#NkYT)I&CDy#3?Nc$%C3%(0iyz@u3fGS(bUd__+NT(HB ztL2p@n|o}~uk?!VpK-MZq^-P;d~te4O?W?b6ISHk=V(-Qva`!L>P0Gm*_K&KYpPqI z*Ajj~W8qwIvvT!J>rdIgEn<@TB)xKEB-jNkP8g=R$zixsx3^;Kh>$FYz3Vr3EIZs$ z&eb(H-0HMl(J_(@!TPZO>w&tsW0>&Pn(v2q!XXC(`>*SF|2?!BUoG?6F`${x$R2Pj z&gfOn0>}LO%ps20>qJ!-lO3%p)Y?0rl9M}uw&`L&oc-)(LCC#sjoI^}N9)Zj->fu9Nge>Pa}Uv)A{eU(Ir2gJP0Ev1tmW3wfu$xnh1J$gdW1o@<9>Pv33QU9 zq*}B4M5K2f#!Ru*9}>mpg)ak6ALnxedC5q)|J_?3f9UhA%zlO1AekMN3JDy!{corq zggm{UKcu2sD9lUY!YxUIN;Js9=9-VC7})wc{qBrI)Onf5y14f}Q{uI^A5H*;I#Nvf zH6dHUeSeE@fdK~ls4xZgU#~-#a`G(dP6Y7YYGBIo293qO<@fv)afQioozV(A0~(U; ztJh(h13daXtx|6=Qut*C7agg-oM{Vdvh4OU$_Mw{w7ujAOrnLWz-qN3GE15jB^b8W zoflJt#XKT*O5^uX@pB`FEC2PD5(~Q0+??wC={N{S7D#~@Sq0&a5l>Yu%a0MXiqtf`OHq=VvGYh&j4#IHBwz;doTMVqs23s8^@`8?Jf=*i|8|^VN z)!LdB?%a3^k5Y-}=gO8D709?#Xc;v)E!vd|8GHG4CW3~|^_6at+IisWoh|W(E#C{1 z`|~oAA2rT5o!Th&&Pd~vYu~<9Zp@F%MZY-RyF~5m$JnQ(X?hk-ium3gJHSAw=k6iA zn(in7;wg=i>2^?!8OX?Fe$BiqhW}&(pBy1N!;5%05dV&s!{t(DjJKxDdRBmow7<5n z<3>@mXq+l!vDOKRuOX%Wf1czQh=jHJoo^SovLzyy3)ngi!xC#poh#HMn@6|V&QF)3 zb9t+NclCD5_lTjP`)7lPMzAPoDMyPLd_Zv4)Gp!aZf&gZ_Z~Ntj5fBbQ0qNvow3xm z9LU!AjV6whVoh;Xq~^9;m!WY#9)E@Q*XHrr-A+S9^2OcT5c5&4nX>3J$MZK+I`%(l zn7o39fDgl>%>@H<4>hj3F4Dez1>cNTaOsYiC}^f?rT|0%>%3L+63x;7>AmgC08=TB- zJH(il(-dyfYUKjTyNWM;MZ--6#e)yGZF^ruT|*km7z`~k271xrz7SE;jE1vq;}YCq z2!eFH@Q&`C1&g60H?Tot#mnjl3JH%C6AIjZjBzk_@LDU{-O~!8S(|-P4n2)z5&gf#sQy3pEH6alL>$a2Mt`oA2 zivaH6M+iW;U8vcyuz%b@=yhKS-~ZwJCY+CE=^i{{GmkLKHN%v%C29S1sih)}`x4ke zBDD*<_&*0{2v`PIVD`ix2?K-2wR5hFUT3{;>Hwo$zu&_sBX@C35Oi^}W7g_-7h-8Q zHaGFKg5i1r#N|2HO=iS@5NPoMd>xw2uG}y@$@ZNXiN7L#WvF#OE7; zLWzrq2ee#1qwCP3t7>U=XNYz3WFF}bb4w2|BiR&7A$=qrwYl8u#KZ6hy9wLm7XgHq z%KkzmP;C0@+r7IiK%oeIX782z`R1wE^^Q%moK6JWx6|DAriWAMMk?#%3?c1Vx%0NB zM{fYa+P?hYP0h28V3hdE(Ia8V;kpMIEv?`wt3G65I=6JmJ`+Y?b!MF1?68h@BOK@4otz_{9kM3Uprh(6n!2s4RM|Zdj_xmj-cXOyK&6g8~K8}=EwocGbP+2wJ?6sHgg@uZLl{#M``~d zPkQ-*E-RzgYv1-2+F6BLUwINsCqiD_<}jPrR|vWLdsmArQdgm3YbV(GhRk5w(UilE z3!VLR?TmPJ&Qh|-Kk}N;Wa=l~FE7&7oJrJ?3W3>b8bGz-6q9JL=p_I9Fy#MBfFwqu z?NwvE?a%sm=4GYyLfrA3rO#^S!*%aWgR{^J4~VueG+xAU%_@TOO`f4X*mkLdJ;K(q zyd^@*%v5D6rudy(fLOz=91@d(!3&=+;n-~1m$eB;vmUb}E2AuemZWAK6PvX`w{o_Cz0N)kvNR9;D)WfD27i?;ee}~q15&$z z{S(VsM%#7EJ0Z6ZKz-+w<{hSnS?bATGxfxELz_VRABPY6M0#lrO2+J$-DRHOP}2<; zxAO1ncn*Kf-kwf_NUu&lb#Pnxcb(i~2tO)(JO%4i``b}^&I}zn<+K1uV+*bv5GaaF z5yXHzFn))%2S6Z&e0dd>5y7;*&fB%qFCpXv&RBAMw+7hEQ=ne^aq*DYazp$t&(3tuPvuL3urh%TN+wVfI+YYzz76Cyc= z_qU^XdtVi5)>x2_Dv(R5$~^?)b9#JlN69n^&j-r5f$}-4H#>B^^vRA7Yg9B;+RGl_ zd0Q*E>v+me3A|4gF@CAsYx9q5r1@@bU&sEODwr=Xzc`R=!`C z56Zm{@Yd$_C(eVjNskrP_p$O-UWUeY?os~X)^jexyl?g4J<-o0&+R|c@6T0ist`jJ ziE?0I{jbCYc8tzysahMIbPHyVb0go3HhFwjs`4 zOPOKYmI>@@`V6v8rUE?i@&Ae)jAbDFgC$?L6Kg-PJ@G0@MzZH*nEzSz*FXyD0{ zreHX^immi1MgoXop_lKS^ZlHOl*#T(J&;HQR%;;Xn7t*}hFkspcH~=cD5U}I#!%Xb zD~uoU2itEqur-!o)%mN)9f(VrG;X)^3lhbrC3{3c7@Al|`N`sCCiPJBd*&@HpuOO= zVncL&)+Yy{PLx}4wukUIV3q0RNo0FhWWA@m@y%odIeExEyE5&urFZrX-O`o%JS6Mr zexPSLRJ(^2A3yWY0f^=Bz)|Y8WL9OA@kzBUV~L31@u?fghK9ojCB7{Po)0~+2SyBV z%b@XT{eL`NWmuDMxJLy91QevC#nIi}-Q6ijx0JMkh%}6DB$e*b`~{@DyF(hJV{o3G zb6w|yUto@1+urwn?qA+JJpEBg?Ek)Qbk-`bx)#hW{>(K6E<6%=P& zNu>{k{P`CMqkA-!^7R!21q2|B*X1_~;r)a+%y}h?^05N6U?VcB5h2Or1&Xh|&w%B! zu&^+XCGZh`l{c{qN)1Z&YqkaJIA5V5?a7XV2gat9kJ%83`0>RE{X+}ICR>jQynkm^ zlWN@T$sTZjO_5l)xz-sv2Wp$Zc4pEMw)WqbX9*-g@Y&KLjA5+~;Ad8s~gH$o|o3;v!t zD8y(1NA*%e81`on`=AES6rgP~mDicP#%gH2c0%QDcwcmS%q^CRIEwsisuKLv{BXVw z36nx3qkxgH`sW(25_4;Wky}jvbN-H?`Sf1J-I4R~6`J11XUnX!8@FHk3fkoco568_ zJ2jqdp98@{xG_8?fVUQ8w8?B4_mW=~DbY8RAnD`iFP<3E;H?0!Sq~_Vb?-6^Qy|MH z!FZD)oDQ7GtcN1Opm*-`)aa7z{D#SQR4gnq`!|{{9u@F~*@vf{ABU+XwEFv}MO}f+ z56@6`oOH1t_|AcnEF*SxGq`icZ7TN4m$|(ht`8E~APYue!YwhvSs|Ds{PCa%?gy14 z6ERmpesr#IOT_=W$IXFd-KVUGy8!{FR1yE<13529CJVkiMY_UD(TxJOCaSexKvvCa z#}n4Gju))vJn3VT4n20gl6WZR^N;u3+6D&Ubm5D>13l*y1s4CjGoLA9Fae!_cxn>P zhL4ms`pL~c`?wo-WTvH3f060!6_JF}`)=w!{NsV09{G`Jpu_tD64*BJY(WCsFVft1 zum)dm`LA9Nq?Z4@X)qWkci*-nT3=IL3d6eL8bS9jZ}68NG{4#()j+=A+33CROSem> z%}5jyBfzFZB&@t}QhiIU{eMv`Px8=Jgcqa~UgC6=dr*NemQ*kJSl^@W5fMTWTvmcFidzYV1xQond-weHe%Or%-t&?5ykIYlES_&q2h5VVAD@V0 zTZ|WQ)EzigE8zp=2zVm7eqqsWn|6cr_YF`Z@D}n@UDp^Jkvb#?`3EP94SWU&UJxD% zBsd*+15#G1qnassYyFL=^TrtM0W{r{>^>|)%G8hV*gtK(`=Y3-%X}}5p5@Iuyg9aX z{H=gQ3V6uDp#mVG@TCuGVGp^(Xo;*m;Rp7yMBI8kATjNtjN^{v188-79o->lik2a2 z?BDG?&pcLPq^SQRsoOKPS_12h4i+O%1R9kQ|4fPtIoelz6A->jU4zA>_x9tu8d(8~ z0ncF)kyi$GgubNG=VXycw+GB;u1`etsc}7>sVf|wT7;Ic#_ePWW|dSNEoJ|GhWUo$ zD@acpZx0Ma_{t@Dcb-cc8X8J)wFB1j@wjX7(KclW;i7rW!_j$+_ylgjLW2c|9J)yU2R@zhx&JoNi!~L~hrTQvT8@6UODZ$y z$$#uw)()N1HeTT_f1^sn@)d%NflQ}i6r^bspcf|*k}GXc`^NzC>FHa!udiOZdc8g3 z5gA#6x7zPt^<*}3@;V3Aj*6vR_@7Z9FS`vd?HCDO9n{T;++m{Set+hHb$ub<)46|^ zM=46tYB-&LMgB9GE_}A89Xli>gzva?0w+wW%5ladL#UKF^SOK}+=r8cgTtlMnRO!Q zZrZOEr3mv5zy5eW)Gg4B0SQrB&&jE&_zd)sR|0XP(Wl`zy~D%Zk#jFx=cIRXt8+;Q zX8fWwRRx|<8;9e72U!P3ZQq+Vzkd32*YE(uiXi$R zZzEpvRE#R_F9gXw_jTsqF4)?SQ1T8#=hlL%+U0;dM3`ejW@al{Hf#oC&9a8!Uv#9*UAxQ?mI!<7 ze{xJu)DuAL99sNPNcmR5yogv7>4~umejEyP?i`j;+CLHoo3aNNgvLS#<(qYVtcy86{HQP?%HV7MiTZPbSt7)>qS{S5Br#fP3Q- z$zgcWwJvWgfGMqRR^6ngZxU|CR~*&H%l{Zp|73{ZIYYk!7u6R~QXAkz9**(im1@3L zu5_ZEu8Ioq9J~OU19hEw>Y*DTUYVN2V;=sPJ`sF>FLWs4aTi3_J0fYDOr^d1mh)L_ zvsVb^ceD_TpT$hUe|Edo{e8$e{+X(4r9Slk({QI}uu;E613DMm89O#yX_;^R*kyEz zf#uQ`=%1dwJwKix(@#yUY;2($9GMtaq)H8%*a(F%{#|K)Ud^$9_ja+3GWw_@hK28l zkK0;zWRGE~?oKhNXXDAq62XEh{}vuI^v*f(F8#;|R#@=FMeYX9V%%g?{GtLVY_C2_ zyd+{1y9?CUe*;1ph)GC1diqC3dNfsqLlD+bB#4#;Vk~Sx2|sex{s=$APH4<(~pDlM=3^Z2x!%Br{}! zo;K|j%|`;@Y_isRwJbT3CHzvKHC5|uUPq@;mOYRJk!A===!k&$QULLEK!~XiYp+vn zhaWm4)DwZnmw|g~c+$SS1scDy{kRUicQQkj)3v-ez>(_J3xeT4`ofQq0n~>D&SAC} zS9uJ9lz_X>rHOHlIQmau!M6dP64q;Tvnjf2`gw6s!hMM1>(n$(L+)eAyhrowwUA0_ z6V4s{%tbxB{oS)vH9Ct{e~8MFr(pltt$C<52lOLRn-nL*zZePa-<0PpB5xuhN8W#0 zDH#4$%9JwL3+(7%V7_#Ixq(mj79wBix5a>Ks6L?)*tS1d8L-N@In{s6V}%t>FymD1=zw^(Y2QGT*GGP z=56!I&W5g7+!yOCZ`8m~aIW*CfA-g}hg{M+z$Y|&B|Nke4rnZ*@LV&+A;O01`g*VM ziPF{oXwZ`ve*}&==rd3X@;-xyTKyQwsrSC@w|&}zjgq{Q84B&M-_ah@L1&4GRCF4V z{IBdkd)kj@Uu@)lug0I-*-R|Uug?%}nVCg!%E%QDKa!9S_sr`*9c~wIE&7VbwgHE* zz>TvCagmT}WT@lPpBuTvFI%yrw49k?243sFZA!2flfxaLDHtvDqPe9gQGLBy&!ER` z{NG_9>!=1sZBN$>F5ga*(YcU9S&eVm_aiu``W)wLH&K;!*wv0jj{*+-o7In=&jdT| zXiV2aVb2U9Lg$`uKc0}OwhmK~dq_H})A)pXf8c2ectr2VFX z@>%7c)l~r|I9l>oDq-iU(|khoeazug@eDPOXNA#=U#+vf$rVLC->c1qd`DAZ;t~lS zxN8sRuJ8wZY)Tk>Pjmd@`=Qf4W<^@&f{+RjGIoPD(eG*1&#-G<s6M*Z>-)wW zCJ&Gh`;n1u{m#jqN~V(NGegz^aqqC(gZUQS?qFE0^osK`t;L1s8`YKchy=725uT86 zjn#3^nlYH{Hyx`9@;Ac89Ii)w`jx)lo+6#xkUoJK7vf@Lb1|~b;$=LPFv)SS zx&HWmrqb6PI)RKvrhdm2v9n|{gi3D zvc!0~42|gUGyCm>LrV_oDYm-ZUSd5%0-d$!;koy8$8+zV$dY;BT^=n(0K@N-65R@s z#cfg^P(O1>;&rBQ$e#5c4YdNY*E6e8w#pooUTzaGlmZ`ar zvF-D}{oSwS(+*jVN_f@O@o1p*0*fNIPv4>%?(Hum=`YxrQ>9YLGlWazP5rj2F=l+% z$ALxgoY@mrul!1SnmgUr<5LQ5SFI<$^!wo7{r(=sWDr}|25>s0hmi(Dm_2!0zo;3& z^@M)Q>e7`88!DE?1+D;E8#!X;KVO3UqQ$#6k-d~mosQkpn!XPc!>Yb_BvCr^M&#j#x6}g|>4TmI2YY<^$Z<~2pTiP-qCCFR z)MNsyqm8SY4qn=P)O$YSB2gkIlK3S1Ju^eQ^g%qSIGK|8AuB=fUZJqV)9gba145%_ z`9GGpGn6pSuY=|T;+0!HsjB!g$pa&V*)xUrPUX`lN}9WumXWXsLN6C6Fzsh*=8kg5 z*3CGm)u0fA11Inylypk!Psg17hJ*J`37v|J#e*q$4ecct(HljGJQQ^Adcu3P?gb8E&6e|9g@)WW~e7^KUEJ)C$x=J*fol-XF0KH=rrQ^EaXL)GsPe0 zPbRGZ073q5MD$U}5uD05a-}1#fpXvs%sVql$>8$i9&%rYQ2+kn;ld%SJMr6d&F*M2 z!PzoHUg#!}bMyg%=9bOydp3ws(@cty(sB+c+){IszUY5w2dLxFh`sfx<1WAQet3_-o%K30%H@o>5<5`~A+W=>=$_42TA zws(-#mFO88FRi%-paI+o2T8ZgKpGk;i`V8S`xTnc3bStQm3QuL^e@O+X!B3}PhGX! z#(7#)Gt(;78BVRBPemPs)%x0Fc6mW_GBPh--A9(p@P-)2mFqjZfurq+1!UdtjCs$}+`4GXH? zwfbDxWqHoqW=!1cVpKmYRCl+>8fI=ZOb931J05FlAU~YO1VWIOPXcw-)gd$mXgw4S zA=wAb;lC3SuSEP>d35`Fdnu6!@CZ%@o_S5g+fd36+xldVf~_ zG!kzpy58EHE%zi*HGbYXSb1Brf|^btpAt%;@|w@S;6KN^1rthwH^4N^`?#{~+F z>uW>D<1c*?$ixnN>_Pz{6)}#+q|Tzxi1YdbY4BL8H99@G|UjG%plqADEJvDriN zW0U6Ae1$GuU+Z~9QZmf>il-O!8Qq5M;#A?jf_LY;k{a-7B0d+&BIWeU70g|nsEhrX zvlNTywFG)iKG5~v={Wqp6P)JzNt_zK1W~dvGDcMC9k`^Q$9DrjlONC;1o-&=J0215 zuz1p=q@|@HecB~DrvQEw93+6ByMh*%6A(kYx?j;gEXC3HhjAJTAxVpLu+f>xyZ=JD z!(=q9rLekVs$$KQm6Ji_WRT84}m=}pDtYJwCnX8 zINnfr#VJhzHgy`yc`dV6D@VhJfg(Cp-E_<2Y z=B}t>0jNjBxnT{JCZ7JNg4bQIaLi_kG``JD&uyCG07(^{iUgBjnk%K&ktRJJM7=nf z=5*Fy&5&?CeHG_JeVabr8e}Q0G$twEm&WkuxcG)of;Gb9Ku?`(Be0I^z)ge)Wbt(lVZ{LGB z@hsH!eWnaA<^4-01tb>afw5rn?&~sw(HHsWH5jcRh{A!ZK+xQv=d)JkoyU;XG*_BU zeY1djF9{QNH?DkB%LBp3kdQdNASct$V7^C5;Y85xUhKo_vLu};rGo$bhJ!zu`$XR0 zfCW_o2TYrQ)KZnSL#ILwDirnc;V*Mp0od|WuH|%_52(*9^{w*y3UR2{!}6n%e#9>7 z&SXr>l;pM6_jUOMhbYsKZ1lPB*!fP9FgbA5M>WzZ2>kn2RWPI{Bp)G3T_;zqls+*K z3=I2k!s?gPDs_W}7MV)qlxuAJEdSds!yBgegsE|~$Mc{%k3QW|x z%VcK5ZZL$s{=RaC4}f7cRotG1EWY~2I%Dkp4UjXyW#Ke^@hSpgNT3v%)w3ZGwE2-i zsYLD$YOoz`Jw_q%eYO6e7>pFuAOd-#Ronf)0RuxYRJXtZa^!TQM>EMX`VHqdq<|CY z4ezsUTjYr8_Wphar`gIMonGT0vqBargVQ>?UOCB91b#k2b}0-F4)HX~RPrEgD+$pm zO{)noiP{hUyPQ2v?V&3pteOW(IBI^QhjXwCX=LjElVzRIumWaYPsnD5z$QW6vY2n^^P zT@0H`y5P^0>UaK-4E6j}O7Rr763{F&s)6@$kdqzN=sEwHGRI0FSE1YI_z(Y;xomE* z&!=4+J7!C6Gqy)mdSLJ$E+Gr+q;kp@3O>S^Rb7XBZV%5yl{pS{ub1Dk*JpnlzGeG8 zq!`_ zhlO<30S8tlPETNt3!c3`qSOc-1>)aJ*qcngr!H(y%&v}>NP*ZjFDR7as7>vP<*mG- zJK@ayvV{hQs{*{0l>e6Oso6-Ic7~QOA%5{lNY!1%j_-c?!}B}%0r2Ta)( zs122eVpnt|i;jsFwWY9*+RVgUO%^zgnCjCw2ht3G((qqy6{Y_(S0YtDyI5cUmT~jv zFZwoKd4Ha8+?(-~`i+C~@u)!+R7wKVHcu;c+kn$(Z`4rjKP?g6K$@f2C*x z;!?Qn31S3~_zs^e&(8K%S64fGRNv#Bq_5n6dns~l7K+o5Pb|K)*Q=dbD8c|C!O~fZ z(1(8DNp)}l6Gv{Ni6&x?&%Ka!9{NE!PkyJx(;IF8@nf_jQ7(0^|4X$!#6#FQ*09Ya zN(An>STgBWp`x#>_;6?T3X$W+^n$R+_&D*4qaS4Qv|&j46E?Y!K6jscaFeHpExZZ) zv-{#J=#$k{#{;`o_xkROtmk`C#NN61U^(s63fB1xS>aFfblX(r!B4Dov)9qIJ zHIkJ2BmpZxx3XlCu31ieS`9v2SHACT<4y~>K3PLPX%*3HHL0-L3CCx=2DG-vU$Nn# zp{?&Y?^1-mCr}M=u(q5?lr1vD7CYW>TG>biMU!rYNd*BtJ{4=j>Qj))oCyY|HSpqk z0}tO>&4|cEe_~P+E)s?JW*oL7D=VwDj!x21=F;-b3fT9C)3_tR0+6_k?Cju>zvMJ? z0U8cWEU4lh4vx{>wg=xLk^7&It^B&b66Z+W{I@kQS7WW3xzU!J8-#_0)l>}*>qJjn zKz8DMJ-M*Y-&<}_io6WwW|QO9N?_o;N3H9I%%NU8GH?cj0InS(9}RSVuqDrh_w|1l zT=&ah{{T}D`6Y9Lhz)@B=_|*$Xe#0b_(m?fRk7Tv|LBNo9w|IV%+Pjxm72kr z2Xc$M-7aXDeA+%l#h>=~hnB;}>dgp9vhw%>0T#XA@z_t6j&}c>Db9I^@&%TU$1_wr z(ei3e)*pp(PYML77OS-=1wWDT93}CsX~X@TFJ;jcpnLaM{MtK@kyo#qZsTPk4VGd5 zomPw8nY{{psAbH?*GJ;e#t=qJLX&731rP&FjroSKW+CfabmHRr{ ze>0|mnLIuA_{rR|^`3H=XVh3m=00(`8Jqd7f#xbsJWozCV@zz-OVxj>t$7n>Nh6W+ zHBhAck?_NI(U(E)b(8lN!4BWi)lgCCccTw};#X}rTgc!O><+7gO^`3;Ij+=Lzl>_^YJ(rJ;T4+2Nd{ zxlR8ryYP0(arnKW9t4l3MxcI9S{f1mJ-XP1mkm(lN=0Wf+SoH4tE#XQsR#1;WAuz@ z$9U|xxz4}z=?>&EG=mhnjCb%n7yZTXW_EG!_(G7nd}VI>SD;_Q=0G;&O*Fk9xQ0G~ zHg|P(JOmu+2i!RAE35FQuANxCC!pHXOkTckjc2QJ4cHKN8eBI%fd-t@UqxBBml*9X zK$a4XSc7Gwh%X|d{6zPD=J5dt7z#SPw|E&?Jpt)V=Lh7rXrIS^1{`v{!gM?R$BikQ zdvCXFKE!I~*3e+ttZ-jHpSf6H;+WB%A(GCy!?GRO81eVw*M9##kHh=x91nHt_E zo5cpp(E75%wm1*0A*DhMqdKQ|VC!4N{k_tJZ4I8u3U1}d{NVwbI3G|UniGdA_iuk_ z!M3Un^)vPLpGo};A-61*AFhMf^zOcsV#OBG{NF%;ELZAzS0@Dt0`8tEGwcC`!2IE* zGxLf(5*UaArxB7|#FpOP{ORcnxuHNU?<8_QD!HPzUlfeUho20k@TALIaj3Uf@l;Qk zom^k6aO5cBe>&vJtIU(UEyZ2i8w*p2rHG`7&-@_=oPv~i?^}OMRAssid~kCOZ2j`f z6SuBxxIb9)Wo^gdi^{gS+&Qpk{k=praBycQj^HF#wTT;k-U+6bnd@J)=U`LP43Z0^ z9E4w>X1*4sMynMNF12;#^1a{1)|FQOl=HjyulTSvO3s*#g>taGAPnsN zh%f%%i_o?@QCSACTWMytJY4+&(%^) zmpAs4;IPhpcig#l!9izh>GE))$gz1NT99k*4jiU!;rhTD2cVO?~KFuMgKKn;Swr}x8e!qbpxYj*!h8E!B>hTxTJCD0tH`s zooeF+CWLv>j6zwr<(pf2cK+!4Nw(Tq;Ca`@2#v)j&Z4Q6ktVTQ|NZD0*Sal7hVU-^ zsF+dwlCewWU(LD^f8D$7_zI3p6siM;$j(%{h?$eCt!E8IIQANqm7ZqPQEHNUfZ2o zg2%$e%8RP+9aj_)9FxOuYg~+=uT5i$_;)2mLMoOB$M`qADhM0wqk8|CLf-tVOI&G_ zSOxPcVV;a75wdb`k>ERB6j)OVj1n21>JLFVE@Rh_qV{AnhOy~#rZ@}JaJQorWWn_g z?&EmABTKlS^}p|Y8|oovBJ$&wNpG+i?$=FkIRJ$_szrMWm$itL6`+~B$>6Y#QK|IK z?%HSd4u|`Y^)L@WNf(cKTL7FOsI+pX`3IOM`vgI5uS@z^*gK>ZDJ051gGbzewnqW7 zsz^vMJtHA;Rsm&?_to+7C@9eh3f;h7?6u+O1+eGSRmpcOtBc#TyWfwtgdcAcd30?v zW}&y?oK7nr{$Ir*g{#ES5Z70pSS@rUYHDhu8Y?USMRo$d^r(k8 zivllCCd#}$06tz%VFTq4EK9jm%4iVFp@afk`9fCo#8Pe(hq**}jZRobYw@8z)A!BS z&a`~y5(@EGj}G->x&v;)S&-~EbC)KmO;W2zJ{Oj+qT)wl6xgU^+XLCl6~#yGWNWmy zGqkDpl^pN3EASf+#T0Lqg9j_-3rbR3^j23^R7Y1N^cvQ_k??)olq(1M#$Y1YN^Pp- zTl6Q;3*BtS1K3Y0wDa^3KUxft7Ly_5&2e^i7FcC*;t%wh1O5H8lapQI!Kki(!tp~F z@*n9D!gz4DjMbVU0+reAZZ7|^k^wghfORt1-Urn)lZLr$FaB9FkWP$n`cO#0cif7-hxXYd2A!X~6L1GNt_k^$x54uOSrTG|a^& zN%*DX@rh~E$4r`)cI!Z9*wJiWD=Q(3VL{*B-7|4!6eRiSY2K=_9D#rM(rZBlB@Je~ z4Bhi(B?0`7JXcdc?PXcp?>WGy;ijjh8!d8cgTS)^tZPdzASglmBnKg!V#Nm7(K^V< z7^hh=uc~Zny7k>s5y|+w)XWP;Wpe$}2e;i1cV#Ls0XiuYue_5_s~zSVKlF>=!~e@G zKyO7&M_ZnhVaJpof^N=vtD51l~yHDLWC4?Ky59{-Sw1mI(tN-;HtiMm{J z&xjgdabJelW(M1WwyafkrsnvecIH%^O~wUkMAAx2oqx?zSzl4;yDD%moUU^BzQ8#I ztQw2+=Uh5ljFk1+k_xN}XNF8_t-Qj9?JJUGAOrE@l zP837fyqQPmA$H9Jih*ponX<{eIntSm)k+)U$j`*_Jos1Sr@SdJW#6bm^qnh53DL$j z6k~EC0Cd{iVjGSE;9Q#Yh-Ct*@XkZ~uq-dTj}M19F|@lbcU*?=z>dqMgD0&`W#e^} zKFAa0G8PUlgGLAKdK5FGGX=9BpT%(L~wgfJKZanV{_0rS6wNE$sK+Nk08c zkWan@f7@5QE`I0xqrPwOg92a~E9V?$hFd!iH77*7*2$vXd}|B0UA*rjyKxgBQ#VeP zdKzX+uYHfLWovvfCtSDvHgS6~5Ik&R546!S({dx}Xlb4Bw%=d@ zKm~cIG2m>7H(m?h6I+GV<>6-M45YD5^-Wt_T7ph30a9!biC-*K&1ApfGp%oJyI}YZ z{-iP|=QC}lJMM_knGyjW-YcY%k`fPoN?vfOfm0kHa^vGW6H^ZCmM*E_3Bm^SvDC0^cj3KO!^@AgIILN{x_ ziQ10cbCOWx3pa(9>tPj5uk`RM$7m+R<82ZZ@!p3S8gI;zDrD0}e1v9GG(_8%9^1t| zLim4dCvkB+lLtKLh~>8KPqo56^VQ0Dl~u4kE2^uw>TKlArx0VJdhU*I$&PT6#PoQB z{2d&mE@3{=>`$mXD^mee5IMjka+)Zm;T`yof9huZmc^`1bRVhdger#YXc^ z%G#wohCe9=cVV4r8u2s#(!^(*Qn)|)l~T?m&vGr)C;fUnlc#w)JY$@ty*8GLwuN}- zT6aAX`NEMYHDxo|*#vv>Aj!|OnSRTHQqh#0|tGF=86g%j=jBC_jB}b zei%J%csmcuyPL>RCF)*%3TnL??aeaXw%**9^85QJ;>+?|?h#-|@7`D`WxLSqw6AC_ zY<3bD3wy+S{u&8T=!r>BEi1fudS(_&1k+4)Od+nbBhe}D+g}<4ENk{9wpJIq+r7E~R@ETeU8+tdzXB<|~`CsU-%nqjA(m+2LOLBT0%#8%IdyRbz;0_Gip5;v` zyZ1TsRDgDcO<38V1zQ3~G@g#OKCk0a-!l)#@rM`58w6WEzVR$TlAQd3_K4zGzVJ6XP;hPY?$VUjnk>;U*-F|3bF zO-&8OoVATnc`8N=z8b^hV129Q@l}R}!mC%JAtD08H=br}_wtv!0$!)_gWQ^Oc)F9me(A-IJ zObh{Ih^#GkJq)4zFi7yUPw=oe4rG(iPnt8UbSLt6a`Txrt7Myw{_zG=`tQqcAb=i# zpWL)uzZj@e9E#nzaHBX=akawkgXb@(uz*;0%Hp3l07 zVZg~+D9(X?#xq|HQrtJln|=uNJZbNmTFDM4a8nFPo2{B7u)TbaZp7$1z)+ALyL#<>rz=g*QG-#x91C3~8%S&v6vRyXKL*cqp1Ab?uG zP+=0k=V4bZP!~}Dfc^EIz&|NA(yg|p?OI>8BugNvnm5YgD)<7nrwHfPvj?xV@8o?z zFjHDwJWkqhnAc!<3Q&D^M#lYE_0zK9UGV)mB6(+ed%hdL-TTN>8=q%{PkRl>Tfvkc1;f+<8}+?qm-5>ULcd4Z!Vz>lC4P$RF^q$&LJDKBgf~goz5>bc8L* zAPIPP+~42VIv%ovM?GDw2)JNex_#cE&3rm zESsvl_0j+Vc^XxjqaZ{+zT|@resn{p@1q^ICT~}O491puJQE^(imFoHoL*9Ti^5`E z`i+sfV&b*)W=3Jp8-k3|m!+G413`}G#Wj8r>3qQ4@lO+WZ$ksRd77Z2ZDd4lHJ6qT=r&J@zBEm`2Ut22@3-hHX5BZXN1 zH206KH86nCL#3=eyYR#c+>jOR0Wu1tJGA9>{yAIeG=0A=uBt?0$q#Fnd7x+M*@gLN zi}j8qh;;83I~n+)9Mq%)@ z<9{X+wqZK`8Z>FLjsiFpOv3Y8soL_4r11xe7`hC84P`?%YbB^#H|cGw02snvrMTA% zFUT!p%Aa{ZdTh+qxuYTGcQ*C=JwV|e06WC`^WWP;>HHshUmafUO_c~HL@#e;b)Ry> z)P&^5sA+EbU~9(*^ODq1axXj6Z&2*0^J`d9whjHx*5Y>0&A$f2y2fz9`JeU!_!VpB z8=~NovcHdL=qwv~?-+5k4iM&lB_79BwY3S4w=?a0?te$Km+soN*BD>v@O@P1>3_Ge zjd31=CvqkZU9g8RKx9?qU2m-7C`RO}F5(TKeX7>zi2f4AQAX8e_VKyYC1^R;FJEK&*u^LU)=&V%L3%1y5JU}s@Pge>Qicc zWlgpKlZD{sIfgUVN0mx=(e`My~KeM zPEk=Y&^7Mh5ew+c1^|B<()R%hCV2)+?wwpWhya*_Ih$exi2+M6(hB`a?rTRxhyXCT ze7f06cI(_eY?8atDb=g_Ql50Nk7ejNdZ^uw;&mweU+NhFC)oRL_AV0eLwG_X(qV_= zQy|2mhLC9^nR9!5W<3%nXlw*i_n2nYZ`5(SW;L#h#!e1}LR^=+%W7w}Z2bTN>U@e( z4hol*Wwgn>`@-we)}5*1HV;q+YOS|{)V0xO&Te`VHpCCqv7+}5xk*7lf>*vCuPV}q zKq@cxrbH)W1FSOu!b{5QMEc^zi+0VM{~vpRpX~v)mJY?>ti{`4d7_r6gYUdV{RArpvxSfyr zlpz|WjvV_`-!}U2z)iml+bi@JhDK0tnoK-_KdXE-jU+`BZLQp2nfR!&Pff_0l%((T z`nt^o1#{UGPevIwoY-LeU$&CuECcD3LKkEk|9CS{Y`S0}1Rr~D%`oX#F1-b7pEW!2SYoNkG!gQ$Qcg^mfGRXSMl#Ee09|W|c`* zA-mw$Ko_fk9@#bd_g;){T%r(dzxPd9+{NKY)L!J3hB_WFiyTrq1}l)di~|jBMu(k= zHzP?DPk%L-f(kHJF=xtr&G$C;vShlZYQnU+lZB;X9A*C6)27a z_w~s9UiODeM06?_^;BqzP)^-#<;Sb8HMs7o=d??4@$%LHWp%`MAc)7}lC|eZouq=j zz`)=G@s?Qux4ZLH*2yf!?$MweR>aV)7$%Wq833~45V}fF&w3Vy`-N9gGIjN!8s)qe z4#c|A0Z1(e)dRKPdl14gr0ZFWm^g^n@$wLe=7>Pt1{%!&U}U;|0B}lLun2A#kT916 z$^aB2PhC{NeH*mdSAm9X1JLibT*Noyimbc+SNk(DxpW`byTXujq_aaL-U6{_(ul|% z6RmRE=vNV3R z&h#JvIyfT{&>*Ar(UrFyVx1*TpX+jor7=PQH}3YX^n0 zwm=>O2tg$U90Bt?KHxMlvh&er+)*UrtGeV-2!CkOvwZU#Y@7LX`z!c3zY7X*QkKqB zqDd_fg!HZz-0C#&5<47<4h%F)@- z2G~{sMU@qv>m}D4-iwg1wmRlJ-I;H^SUdoRs1zW6<rX#8hLzUQEjXhxWN)8sQV&A zSm3u>#KFZkl%VPIB}%v3X7qmCMjxy2ysU6`oWqn){Aa<)JAt;V&MGZ&t-hB#yR;^q z|Ni}xEH1q#JsIR2Iq_=0TlzmuDTs0azVTwX6EzZ_#q)-ti;(gIo*6>Y=0|}-2ppD^ z|J~2&Yq3dUdUB_D5drL%hCdw7-Vg$p=I@hz@QA8+_MFIbKr5ZB!zX-}F>$VYi#t?$ z!Mcrog;%fY_!_^(&vBTW_(_gz4Ec&8`@V|>2ud&g`Ezvp3mH?{=%Fs3ciex;-4K4? z#66owzv%yzeIXC~HIR+I`qPwVgUx9;t|>`LG1?^8fYG^!VV*CErHVs1r)O0&5~@IRHa{1pJNJN7*Nz7#D6*# z-4Ngo7SmhH&RY57v{bBbVlvEZSbxvkc@@zKF0<6k%ptJ%pa!p$vV6)1HK4DMz^S#X z)3JX98zGh~)-GvqnBzcLTfg6-3K4;nyx`|Aw*gf`H^AZC4!_Kf;uuuL!pFyFqN7{w zp$N>VA2F-8!G#aWaC#kTPdV%nr}1Bu+~7&aO<1w!L*y}$C5>85DDHpIVdz$@bk&i- zD%<+Rpy#*uM$O{>tOs+Zit*CM6o&8%8e%1aZgW=_leTYJ1i9i0ulT@tF6;yJooK39vMl__@8Fb{&t{Yoq2a5H zn`CLf|JVkkc?Zj_cU}zt&j5<^k#2u}@~pwvYa#d^oDC1vmamI2BL3vQLjtb9v<2y% z2&h#0MBaenO86(LKAHHsq1|QN-}^7V5=9qsUQksvZ2xqU_OFrN;lh&+i;^i}F>m7@ zUtas?Np!&s+pKe@#_SE301TWuG)1n^#=*8^&f%CX)K@6(L(QCuv!Jzu{T`COPUbK0 zUx(@KfqBC&t}q#2+GCkw6B0!WEw$w;ARGDMq47$3NS3DN*<`TQ%b9C?MjEM z`0fn^C?T;3pv4)@Q)`7KWUfJ4Q+V@&vBt|t2(<`iu^W8#=R5I;CS9^^dhz_B-S>RO z^%KNzSIBijWh0)_PVE=gk4XVViB0WZ_nsLM(_U5X;cRAgZspv;Xt!`G?qBzkvAyS4 z74U|72F%`%Bu~C-WlkMd0JyQkD&APPJ@Gqe=CyJAWK%0Jov10mP}(C3oJ;Uy{& z=C2ujpl6Xo(nU`SG#=&w>VNF-!k+CSg{rcP7Sv{=M zKkv*`n3e})d$BK@?UE zbTyOaihRTy52zMt9MX*5kXPaEKe5#(%+}Rqu{J?4H1=GPDKX&NdVs9b@$?@!K~c$L z1m#FOQQUihBndFRjzFsHjFDSZQfKRwIxE^!FqFE0g5)Yz{eA_dtVdL@sZN-H#V93- zVHMU6XMW-WWHKv0kqGJV1z-+5emw&*Gw762jfk(32Oke;B5wiIUhQcLfFecj4mh8h zfm(6tDhv9ZOSu9KaN(E%%O{x0#hN_8;2`vHy}@1Ndv{r?Q&NaqX*obJJvC+f@gBgV z<1liVFrM_79Nx;Mu9jB5;XT3-js6)CkX7rLZ0wn;gV5}NjHRB5`Ns{4$CS$ln~JAZ zc2f+~)6;h0*zGq%&TYEc0oaIR?emx9N>IPQKSBdM|NR7rFqj6}GQ{(Z_Rn?o&$0Uw z@V4@sn#VKl(dm1~d|9`M7kq(s`FCH{^K$F5K=zsXg9tPhgo^$J)`*53VI#lQ@fIfJ z;tV2b^GtLhkQsk(#o|Xo6!2wE;LiO{je7a_vaGgc*s`$x{8{tjM8muQRav{oL&5#L zNE*<6qI|PXfoeTn0CtIchsUY&uBo4xUMR1nkqJ>&1K`Viyn1n2pd(YL_?7O?`pQK!F{K*n-+X{4W{lRgbi9UHs)9hwpR* zS^w4udFJNK*+Q1b>Guu*cV7Q#-w;aC7I>_Iz?f&^g9sy)KkjpU+25`;ekl>2acy8! zgeq1Xj+FOEp0`raP6NU$kA-ok9AUgAmb@&J6eGvbn>9^#gT09zF6=x5T5>1OMJ-0x z@pN>R0L7>y+P>RNsu{n6`Gp3<^+~VwdG-~5K85Ue{zued01*9lGg2gCp`!B^VAy)& z1qp=4pX3b`(y~cFE;{-$!c@BusY3dM5AVrm2q5j&7zWTX$NN!xhya;qKm!qe%Snn| zlbb8StJ}C+J|M?n1PnLEN4CIQR+Qz#@TBs1l>xz;P7pujD3VDQD{$4MljprX=iYxh z&@ja@c43rLN! zFEz`9!DhL@|1k9xKvDKx-wV^Z5YQKS~0*=s&>lLZ1QUf=gB3N!lDty37cmiLKit z?UtfIfi`GlA>IXLv9bW$Zr3WsQx)3@%gVQJ8DS|4yU)y4i@^a(_>>_LYR4Em&qyaTy+|?7T z5|(qz&8BPwTgc4?BA?4)xE_v1GlNNjp|0WmYt6*VJX-yx3R)FL9PNGz8iN}ZrrwQ{T8X@^+Yd0FRkg+(_Wk zA%EEtnUl*0+Gsr4l8muTdz%UEKy-GAU*T`F$_9SGhqMIl6mX(K~l5PQU%jX|U zu6c*atZflAI7E8WZQEVd!IR$f!P*^~&ueU;jF{qPCy$BiTviXTd_XsvoLhk7pS6EF zJd6XuOXGL$zd^p@N;N1Q`x@fRI$~>0zetBCvr6Rmwk)^*z-NSvS<~39)Q>Jrp7dE4 zuGz1e)9+0`e8{dRbDzv>L-UXnYPpKzJTdHNU1t;SJ0I<8@v_rkLS;hzm64hXJndW zu8YiR)B0za3>)e2uRBsIL&--C1_>DyZ$GF+55*8`(H_;55zewvD_RbY&6++{8!%NY z9!z}o+Ol4g(6zm|No?C~JZK+z{ELsl5A!|26;xuP*_Y1P%$%?N``xs)=lskZQ5 z4+{(XH&t6%(Nt|7&tX|%+;uM$_+)go33$jaV@M2W>q_F z=@o3i6S*Hb)C-FhXcoUcmzUvLL2XLGXp_Cz^iYt##IzoJ<1_2;i)C z)I5Lx?GSUST{ru`@p4Xp84{C{dL}1zC@3gOYHB{U`d?QF`TLedq-?3g+HxI;YRYP1 zZ1C>_ng`wJ4?@epj(IK~9;J0PcXzRPUj4tXQqu0j9GCY6Q3b=^2^16>Q0H%0jX7P7KkBY(-v(jIZB({`P({)WgM7oo$Z>ys-~ttp$|m|vhb!xUHD zwEaNMZv8%Tw(Zlzs@%$#c%V_}PdIW(ve)wkv9StPp4e{MDJzB~=)*YI{?)^@v2T_; z`I)_f7X5|XjrgRizQ;o?KqxhvsH|?7$(f?O%UGoq9=lMt#40qseMHz{mU!~QN?7)G zGu3!gF}BsOuH%mjbQ7<|9ys1bc}JiM7mUoHffG#fHW#Wqk<%TMWo|)LKj1gI1$12K z&j4v}NHr711IeGJo?#v680s!K;b?EaF^*Fm{Is7gWu|R(czHF?kS2b?8wAeNFnO75@Lzm*8r3zrIsE9#?{iBr?+>B*YxPX*vCoco?iWA6M2{w@ zA_rYyOtEACRirYv`0Pl7__V=Y=?QJs>?Z*d<-S7W=d z1Kr=4adtj#56F>jOJMuu54Pa9-kSm9E&k)@xhbk2ix6I8L+M`Kyk*6XwY;~KRa34E zm+>Zt8TR}lqAln{id~WbYeV*Hu@4!1ycy2%VId4dTYUIIfJaZ3`-FQ??b>+in02T^ z@o$E`i0@@1x8;0vXXEiPwRHGcD_kq&Nl(VOymow_$T40C33oky`HBze&d=GdqNX-k zx0QZmewT<(4P4f-~<>=b8~ZBQsH8%Hjr?#3nQ$^r17~03eEK>KGx32)4s#rz=`z{bLLs|EyuRT0aoGh>OH)QQg=jwTS zM=iP3wE7l~afO)~8h`CaI0myP@<<7_8bj2)oBVpz?}Hv;{GZ?2Y2H{jZS7Ii^HJsB zmVe|cWy){U1lf($geC8ee3+`)h1pE#?GGA`;Fi@=ilVZz&>MH28oW?zivBLpF1PeJpQnF#GM9$0g^Tcd#@zS*A1uh6-aC+- zeqL(=(OJIzU)K+4!cz@|v}M4wdF-Q7nGQel$LP0=LFJN`b;O;IVa+yD_PKNfS$%Ro zUkYd=c$;jPrRAa@u3pG;rGi3L{BV@I497*)K^S=d^3hTuy0OgGX@^bo8nMLBaXk_V zD|xx&t25cO!VeF(LB!sEBbUZQRyV(>QtctJR1kl>`?=PaBTK@Fey?mcJo8(MPnj{p zR-R3Kb~5YXjv6TlY`^jD{RQ{XVfce(tGv%29>k@8Z-1pXYVnmJ@WI6M63J4eNB^%K z)3jXAPs0^=CZCjc%thnXnK%$q{99k0 zeh}+VxaY%~Y=e$g1c(>&$mP~#wc7H}@b=fey}d@ZwmTlN*}L>t#{tF);QncDW=4hQ z2QXSSNaeV5X%W=a)NjkRAmhf`*x1;S${`IFB^vr}4>ttyDS3XaEi5gSD5r_#fBSY4 zv2m#kL^1jg+Y(}sM@zd2sp9ZpvcfAon)J8}hjPKeBVOJJ)IzlQD1!OJ#la{9m+;Cx zlvx|l?0*z1zq8xJ%Zebp^&k*{X+se@6G?;E!sI7@^c%GO7kU?rdh)?`Kp{bGo3PZ@ zYE@K>P#D}W^=0ShUrR)JJ^C5$i*aa-W5*ysOOO#os(CT_|-yq$ws7pL|4^(gFj8#!rNz0- zt(%NofNUhn!he5Yb6 z!(uAYew=BSU;Vs)-|-RSn0iphyBs%9|J!$|e^)!TfAWC<1O0s0l6ACqZ>1Xx9j)Oc zG9?V#TdNkxcf&F#$0t3vVURyN+qRY{Or%wqpd*z1aMF)+W5ehQ6!6<-$@JpGhnq2P z>?V6`zeeLNyL~dcP&%M#&%)EM66#%4i{@T5`tkZ6FrD@QZ~aY@9xi_A9DZzm-TV>5 z?Q@--W$~*-kPx2zcaO+Lfe*k^*)*_CWDn|hcK#}`l_^{=Z5I1vcr=}EKR&VH7#AO3 znx7x4tbF!RlAYReP#2(492Vryz4-Kj6fjKfpx|;8KwE;}`wDQjzFOb2?^+}((K4xWSzch(C`L29m zSYp}O`&~V7QiaKO&?O3vvmK@A4U8K?=3hFm^{wel{);*3(y%lq`bdeJf4}Q|T^slJ z&53W*C*yRTMtd^~GVeP+88Ft9t?at0lh9O}fA%V8O4oXCJaqWEl##Bs9%y-o~leJE~@C8_L zbgKJ$WZw=IT=V729Jh7%!!Z*Pkg5@tz7Y2y0Ad=%Ixhe}r2|#olN+nJUkvzjKcd3% zf2lUzb|f8PC=!)EX0}|B;`)tk;h$emf}rU|bwx%{?a_#hW{9^ZSI@b{+yjtkpPozb zysQakUbSPW_ZZDM2_SIMe1wGV=TYj$aT4-bc#KX#$$qd6000`Jdz_kRgd& zo$vjB7#oIk*sXRz|6iWbMsz^?st_=rwC8vA>n=)+D{FrU?j^7S>(Tv-r>O!CE9oCu zh-6-u)z)jh+im^>XBX1&<7aWISuZS~!qBP3H>iDLk!_V5kofpDI{KNjJ32l&_D=$t zW047HCeO(p;<npoa0gU&+LPm?$3QmnvAIlE& zh*M*QPp_Ydq<)5}LnQSV_z_)3E|HN--LpJbW%mM-egz@T%EPcbVFm_e!ebW2^!ZiY zOO9`GK~@baJEmemMQ%3{D)j)46Km_+w-fS}UTFROm(JW(RwgxC$w(7sCgr96M^_JY zeo=w)HQSgip>0Eyev-3n*Jfq4(%?^+9zR`c>@vz?lcOaSl;tA};-d;LDI9w$7q^P< zUv3{wDs_w7rQ`I{#b;BCSuUwnQ7(GSsN_CbP;s;A#&%tW3=mWRU;LWRP#MXp^{Pma zO`g<{n{Y?{*TbY?=7WL_G6t>U%0f;1CDv&64dGbkhvi<`i!~3lXv|zNl&M!|pV`#7 zK|k}Zsvg#~88$Ke*)mM(;CmOAKP$M>z&f%6v$De)E~xk(@=1zBeQw|UI1;d5MR}c!3D~6p{SE^3L@D=O5?M8S=?rK0dts2*>OK5q8V$ zU6jl+WnP4nQz1mw^5Sd+$QWT8I(A(w7)_{rJMju++?y0;$2@NAN5=rrQQ=^e2@?kR zKwC7k<$m-Ctn3RuK(hzd3^CRsGVYCO=D+h_@g<%|1W}4V1UE!bg?j}_%>}Ni$^5CZ z&61X_M(I>9&$j4c&~fDg@)K~KKixvYcJXRITih1B>zTWVj&x*QEg}lTpZI-T-<(~k zkAJ2Ux?X&Ug8a&k<<>sTi)x7?D^FkiWAz53qU}@Q_q4(bJ%6avs-id+r<=)~l zY69B^0pX%w>%ijA`<%}Ge%ZSK2aa6QyLxGe{dhpj-cHz;>?Ou)GCL`6RQ~I{BYyI4T5mR* zMWM=ZLrctUzzoPC5|CI3A3A1naS>i|@N|3qWB(#7s>|ytCAMZ=8!xGm=>D1;skeGx`)8jwRPAxA|b zOH2EQOVzzDZKF;L9R~Xn(5od`@=d2E7@ZB6_$bbXE}~!D_G7w}V|d)WK=#EkqRKe0 zr8;k9FXm~~K$yIl_4-8^>8Mrw@p*t8k7fLB`>I;mbM)^KiX|7Ew!?ZsdD<}sf0JTf zGZ+3`i#z#EDJLv~H%*h)P_fh0IRD^}Q7SMOFt$szv(LVP1s2LI$?bGsX!7g%3Ivw} zAaw_EhjhB`$54>l3 zMEs+XUwe;rnG}s*I2@Gne!WMbG0G0oY$a$)4^OL%7PWYEDuU_K+(qAQW-dPcqMCfq zi3C(^%sn)ZY-=hg%u4>C6mb|}P-XZ-dG<%aqnlRQp0^@ckALS3P$@H41C9SL)CZAS z-y0CLpnp1=lwVkQ89^rK!Bz)uIAFK5Xzh_ChP{S!lNr2Az=ce(Wf1&lKLAHX5S{`A zB0}txO-iK=E=5#Slx;%6*=7R2VK7hljrJ=q)MQeXwBDyi^o1BG6x$$=XE-W#}DE>`&OINvrZXGzvOq&`%X|d zHM!T9*@49;0>GcaMr9%zk<3d2D#8f*4!S8KcsaT_^m!i z0?`_MEw`kekmvG`ul`O&B_`q`Y|hZxxH@GR*&nNb9L4vqXbn07L&ckyO4`bi-_EWC z_4cGV{0hn6)Hs5~91nk2<%LSFjf_08UBZY~V0 zkEpw15Z}dp3uE8F53uoaN+91-rc4o55>;0V!6#Z7&Y~8%WC+H`pTYN4QTdJ8Y(w^i zr5#eyuo19X1!O9>R^8OH%1vTnafyLf0Nc2h+RgPHk~vkDI?G7j6D&*xZ*@dwotln5 z6^Zg~-;}P*wTp>&^qQcQdM|e`RGl?^Z9;9d@wX$aH{n05!}Xx&@=|;(>A9KDY1L|} zWezN{ zAvs5)v^`wn!ZmNybILC_n9qpe8r-b<8VxiAh{`#76r$pjvdZ5$y;sp2$HY8&KVAXk zR+_0s%3V8qkk{P`Ro9(>E3l%eiy6V|>0KuxTS9nEJV)_z=^cPDf^`@5fjH!287*Zz@VkT_NnoxHDR zW9N~2B_XEj?cBNEx4D~X{ziz#iZ?fpi8B|V`Z}?zcZ)qhG5AETpC4k)l(#mS{_5_uh;T9HCHRP!I z55i+mF||7OeGgGQ%<36Jr!Yf^1$7T`rm- zm{ArJSIE~ppO2Xo7lDw#c|uyU-={zVpYMs>+k2BiTHswtmz`{f2SyGE#6k07PPh0i z@{T_~%Omv6o1$4G4li$sKk4j1k zxuGmVqLh7uA_NmG3=@Rx?>PSZF&BtJXK5) zt}r_rulyMK%0a)?s#O0CPtVG&XwfC$&z_7Auuj8 zjB!5euVeHEZGJhpYBkz|Y{}$tv={FZe$o3t9>1H;+gEl`vABOG+a@0}qF#hIOm4Pa zeEGJv+r#in`t3>kr3g`L|4nqEV-z4V;QuHLl7cd;8NxmS>I@ zlNH!6Vi!C6x+JzzMeo`?*5}eLr=v`^$68AY?d3I^OF?VjS)ZyRv z$mXwRHj{t=wpzodi1qI^WbfZsL+I4X!imrWO`IQa=fc*j$OCD5xldic95=YEfWw2DolkV*>|o-RaYW z6Db0;4N5dSs0&yOd_3^3L3q~4{>oOSuhQWG*Fajm`36>4nu1A&AWYs82t{e?-Cet% zwY7gUtexK94kSH2+AOrZKcUOu^>^xk&@?)?wcJFY?`^W|R|5yQ%lH$&0C7m&+EeVi_jKR0kijIeN=2L1g_6)AOFm&!Q+po)aC4sOic8; z^O4r)^k|5}vE04NP4=tm%2YkIopu9R>tI67%H!By#0t!VwXOo-vHm!Y{ErUZ^VoRC zTgsUVwznb0+{H2PSNZx5W)vxMKeuq`2RBQgjm!DUR2FAGp}^^3zO+2?EWYp0JXej* z5_C`}GBF}N+h>2%jDSOV%?{7HYG^bM^6tb`BnvLI3YqB)${Js_-w5sQ*}=z#>=c)G z1)hUzQt_Ej_60jPAG4uLkVQQWUYW?F&fLL}Qb9+9Ogl++GPci4ZLvl-l`4tqUEd9l z;~-y0GT-#Wd|xCuDOjV=`$y+*nK>IvjuDPZ9DMjZ{>y0QtS$?gH=4M)9P%TFMmL+E zk-51zBCV9d`;8^YW)HD4B6A8kWWb*YBHUcmH6&DUN~gly>ucT`j+>Tcn7_FxUrSbA zDugFNOhO=YFI`#rB@Nxeg3)UKbn{)UWLZ4+cLi)RZ{7#3F+6x=^L#qBv$xl?gHp=f zod?eBSN87*8#Nw+K|ge7a9e_Skd{g}0ww}7y+6d1{gFZkRpylFut8C#{+1b>#Ti{j zGFD3brw6&vpFj>hSoKSnTw|1X1*dvuHO3k;at7jHIF{Ly>BOO#NTbue&4~w{1Kb_& zk#B#~uv^z`n3$Mu#4fUcEk4p`FgSJO1$0I5Qbj`bz%P(?*CXtfuX!B(29^gg@7U!_ z`xdXKf36}fc1HKZ#ALH@+5jKim)e^fP>pdieM<*~@j>q=VQ9#NMM&e^>Fzk!Dal14 z$r$z6*0y-b;*>}B&c~|5ar0uME>bcwvZh&^^HC)i31VWxfT;NR065SkLd7$ZSy?vMcZ^CA)u=7v^nzW9a|_W7Sj#1aQeZ+fq0JI1n_4V>J&@zrL(1;M{n583*4{XYxoWg|>(LN;-bhELq)6fG@qD zJz5p#Ovu94V{pWM9sDtZ74s>~f znjR6gZ=ZQkq-VNL)ct$po23ZR@!6*4i|F}iAV5kP((=7 z1V8feupVv_Wza45bn>(drtlE!KFMjr@ujmMs-3^Bwj%dxaK<*iMlK283}OAeU2B3L zssHDYCT&-p)GeZ`Em5Q59uB${<_&#?nO`CHwrlM|oo4rJFLcnX^1RpSe>TQ9d|>j* zO-&TMI@MVqr#smS1lB(oI~eXRbOlVr0C8p|5V1YhV)ATW-!lnomDVOB=6V)jt(b19 zEpJDT5D2qtAw}&%6phrv1+9l0#iD-}6NBwtMZL=_oNQInOd{i7ggR6lspvi+^XcJ{ zRwXq2vgnXJu})%SvSetrcZ02GJxt-WA>CZzIVsisbs>?nvjLkSIo6D?U$K<3_hhJI z>FIZzX~6USjz57YrcXbeUlMOu^KKtsp^!MWd1X)4T(ZS;jWVID=}!qQOqENo{^_8_ zpWmT&kj6xw+!S11s94?-m&xAlw5(?1?P_W{AkF2xZ`1vgtBte``y*zuATxeQW8C(d zaAR$pU$o3$ECVAAFqSi6y-5|mL6z*pIZuFQ;yPcxmY4<6MQ z7w-fzOK2CU6VM8i4Vscc`%HD$psDgV36cabj&`vB_nf}ibh^66Nkk9Jp#W494g#l^ z>gCk4l(n*G1VliGf$i`dEQ62^4hI8{moo=0wk^1DRtkG@LP`iJ~aP517G9TTKe$jZ>%XeA${TY?)DrIo9a6sK?}ix zgBE2oy~9es*S^23MZ0!-DBUPlU*z1br6YDOVlh!pbnWa&;tf&=#`bUp9zkqxhM7D} zg~6;E>VEO{2OCLPh$Su^|I&fuhmKo$?cZ2wBY1R!i{GDF5eT^;y5&b4PFXmA&v}Ej zIAn!bj91`K)xOB6$iR)){UvRHQYUhBE5PXOg}Jvh*BZ5DI2QZ@$CvSV+1^7YP0Vc@ z8dIj_bcd}gmZkob5c9%nERW+>`fc!l^lSvDofI0sDJPNCc!L@WQPWT&e4UGX8F)Z_d_sgOU z3}C=5J<&D9s)`N{08GUg0QkZG_{AL}wc* z^rIqe2Vk=xX~$+F4H=$295i)MjW)vb26q{736%f`Gv`e1H)!AFYmzKnG{MeXB3_`$ zNZwRP;7LIAMP^R;m(~phFodwEfM5Q$%#@hfxB6vxeruZsFH`noAF`6S>7~w;b&1f! zRlB^_f)w#+aA(`ceBzR%y|7t8DDaW};rIIHIoU_wH(r$pYfcztwtgA^meLq!=*M&q z{7>aep2DkqLt@{H10K)*%J}Xv`aXS(ShJ}u-!CP`TyT;pL&kL**cJ_%THd7h0>jc` zDMoH&NArJ$9Zzx53%I`#hN@#H^?~@~siwMVMKxP+^y&8)_W?~>D=iwU4XsjP(Hb&J zirOS#wbC`Z)CY4P1@^5fDk#hzVp0W5-3)G95nW0CFWGnFJ!yCwSiqISNQ9BzmG6{! zV;U3E&R|K%^Sx>g@AP8&LmF&{uGmp5xZq&C=mmJZy4cmeqIfVzMaYBq5#$`_&Rbu| z7WVhWK~(T|V(FoHyFvG-3b%OITP&L58aQ7pCRY=GgHMk{n%Z-a+YRqm-PxH8UcK&- zDOAE;aOrXuK=!7-cU^DGqOhbv(a&Y|@W+f9$Q*{oaqyMCer{!*I&PHSsMr_s>9{jo zB<@!GMItj&KFttVDY_M!vbh^Hq2y}AJ<*@b@6}su@JF>ssCE9BB5gd@5V5QsxNW~K zQJn>TC`^|$U%n|#Q_3-G^sW2}_dM&2KEC2dg{#$_EG7NN0P~5bzD~A9P>8Ja)Q9GFYSQ@rovp0dg&)mh`Z#%hi?8}@O z!-&G;ykh*A-#1ly0%fu@CtX?Y7@Pu7lDDZEhj|bTU+5oRZoghXyatMyg++lzIpl4o zdw#$fc~Is`n2oWUmi6)J=T6ZBzEO+#s!EC9Mwi;fZyMV~ubb5y=k;oGj$DBG$>@(e zZyf-H3&dDgG`ks~F&i+fO5ji4uqzjxVR}|hX15SSW{B&jz#o`gCOgx~&R%n2mgv-I zapuAKm51c`gvJXD`(A+qI;eEi9M^w`f%kd|uNDjL?X7=57mtIP%rpI_Z2vy!{^^RcGN z@km`e_=bhDlG2Del2&evCprFuacXGO#uhZ+WsgS6_pdpGO=_r3Zwhi!1?9R&)MKwz zy*ab$PpEx4+Mnw75akkWNytj?{tp-BI~Wqrv)t`KeX>8SY~LAqTRf*;SF{)bLU5+*SH;UkJXeK=vnSF8}vL#u@DLHM%72p7#}vdT7;B|6_yen4XZ-}mLH z#o(g9qwBSN^NM#X>E+dR!U1{U-*3)u9{yIz(t5@e1{T^45*@qN$j4VI1wF zZ|_G(M2>hO^n31ug9lZ3+t8ZKYg>p`5T0O``48?n%^EU&GjjTVk(&uGWpd?ioJ9!#ppeW6C%rW&$g)+oh3b&^IZ1w|(~ zvkl9~uh_XnQl@5&%V}`-a5eDu=e0V4m(Qd97y`?LC-<>ig-x6}KU1jsO;8#!(0pn6 zt+Q)sIG(CAbBOuYVI>8udqaIX{chTpi6fK>n#)MNP;{KaDpA0F-bPehp zx8tZlbhxVb<2*H%gfiiyFm-H@@s(d4!E#V`yx;FS1{%w`PFXHQU?-W)(!Zgdh* z)VkDHL`z|)7-bF0f8upz5JIq36X2IGO7O#MnArnf~Y-^AC`_}khkIc_;I~ZhmnQJiuLS#ys z`K(GT(F;lQYR}13qdST2WU~btab;{5;Ia$X<6T|qMVD%PQ+ko5yHuzcJvnd=&ID&*u2ZM|&_xc3CbM!raq%>utIg9oO*S{4FY*Gf z?jmz1zVX}-A^3iG0dzz}SSn=ip#*J%2+$CT#Dx3_o6Zv8h>`{n(HCmdvvfg!7>e@P zkFZ>O<;B%Q194UQ9v25!xha_V6h)su`+`Lok?Xh0z1krQ4BKh zJgx@Cx^>PNcz=kcMZj1y=jL&IUsR7Cn~<&r#V;Zzup7&D2<^C`Q}vFV<(uV&L-SDV zKIX6TkyGhwP8)d{%|^4I4wB#zS?1FD@l3D7wdmR7T&3!zEp#_HY{Le2-71hc9B-8D$4qjK!re;B6 z!jLbiA|WQ*vU!;4I*Rv4CpGSTo9^9~R>l`z#B-r@SougsZ03FRbv#2~QGJf1&k(w) z%L#j4G)TGA6a96*suV&0hNZpJJicQbqUdsp5VNvR1$d%NX)d2pUAQ`FuzzygK$ZSE!@eYihW27O7?}z}zgDu#5Xzt0&e^&cua^kZPIXC_V=j8ob z0wmb@X5HIg`x`f)7j5MH0fqder!x?B#pHY~Um1bSI9labPok*2GjK{u_#npbAz&pl z2LaC_(*jz&KLA(-mn3y656m)E0Gdkv#|4Y5?9uNEvBb*Ew{b{0dL}lBtQj!m;#6Mf z<%Ye`vm)$PsBNMrV4?}jaB@1Rxu=x^uoNe18}shbyIgeGz`?uI)kK4v4&Mfwst;_P zJ(88(e85|SB*M(xOai#ffabhCN^aedw6fwyWSS#yf)fbiV3p%_xX)x16pzV}@A9|; z*yM`-OtEqF%1rwdRzALq?uQ30Y6~UK7u|?Hibpl-aT!Gn^nfH4SA9~hQjj3b9Axt+ z7+l8R1ar;7pfvPxaUZ0LNPdZ1%5mZG=Fq=_=&KY#zHPVrl?#-#Cw!Mz*U6CZ*rQ}F z=jhu>n;uO+cYBJOjvplNxQ!ns_oAO4=8&-Ut^0KyI59E59Bf<=INngPo|&(YfBc;E zt>qhIqxprx$_)MZ`wZ;t7VknO?44<`;!;CC)=q_k+XTst`r{2xf#RRXT<3DvPTvWU zR}#`ilpi`34;RR*n?R~#+#5*5Yxcpu8u&36AT2?+-PBht2u4`N@Ar(s$Q3_lVwWH5 z$lUdkdzyLHTdh<@^OM)n9o#BgXF_8hO4|)TOM5ywEU|>>7X7VGq%_uD_td~7o_;Up z8)^NdwiAm&FZauR|5?oC+LuN>MN^9phTr60DvYve9~;Xlw0N|1{GQQ2Ej7$I&--Q% za?(YU(5g`c8_(-z&xo7Mft9Pwz)PN%&D)pO&KNKuilD}4@S}UR*d3cPbR~{19ZnOJ zS6G9Gf@#dm%m^!m#BreilLV$aIuJmrcDp(6fm4HIoEs2qQBeT@fC*W7(I+t71`6L- z{7D`b_E*`F%(&18-FMkv@d6gY^N9v9N~VF{Z#u7lv#<&sm3{__=*W*BeLxvb24>3Q zQYzBy{qkP^d!2}7d%tKHD*sbx?!?5$HZSgEnnaip ztc+qunG&3>-?F(JGeIEo9a%ZL2bR!XF`n>0Ht(->cK1rXsTQM51e{eX1r<`c8}-q* zbx{)FR*=Z-ul~Ir1V797aG-vI`Y~*#U+-XW$Sch3V5&giW^!@`&6nWJCR5wjkGD2& z385p-n{i4?ONc{QQ8@SeYN2hcmK7}(|9|;hRc>fxj3=7n=$WyB#kI22%M?1;_CJBpQh9%y_fQuz%sVXXGnEIdipHlPJ`^cW zrt&=UI~dKta2y$*gpd3h{)afYEmHRD-6Du!}KN=k1B zGNJ%!fbKQbz5o5=0+S+ei_0u}%#sL;ZlGC1Y-#D#2_qMw0N^8D1dP{i0?T`4xSqPC zL^!oU$`Ds@QV;gHnfl*1;g^0fQiy7#(M#m&IUWw0QZMo4IkL!y0gWD1{M z0vw5kNd$lv&=N)EQW;4Gm;#1`6r5s#ZFAel-Vml54*ZHg$$IBgR zu;AIxDfI53mwNFYcd`M2hqnvQUKpGff}&zE@f4DAZvsm`Po)*WUI{k!-VA{`g}IOi z1Hg!2?i-P7892LpyKv)x>VPH}_sYRJG9?94V>u5DsgxhbFvA6JS;PfUr+#nR!af6* z@QYt`wNh`BO36I|PR2bL{18px4E^Vo{-gi&DK;4ZF z7RTja4Kr&H2Z4j-^R~KuVC{r*f5T3gUr5}Gz2xRvu6OXPTWWjyP{`fV9o}u$uoZP1 zf=wD>Oz4l%RR*VKcU&mbLf5ex0(wKaEZ%(x^`DV{zHrd7c%k2XXPI^T&Mn)CNZgl3 zE_9clNvbuGRSw(E&e7KsOq@x3BX0-KDgS+Y;AYd!o^1dQD9%R(lshn%(iOiXNIL9> zH0&B&nWIf4%lblbvO z?TLd)U2hrHoj(>U;qy`f1od)UCx47*zW0^p%4pU`ZN*@Iy2k(kMr&L02&vxtyF{!+ zedxIP;~W|K5l?9ZiVDu#{1|oI=PE*&P|>TG+X3P;5WCWZq}25#VUqrkO|zMDe3@|1 z5DVqXi7K67x*sr+e1!*_oOS>n{DFD~-!Q`#U)<@J4S=$zd0Yzd)bFddy}oC8hO%@z zzdUN}0H>QX!=!WZb^!H@K_>hG{PjC6aSejoU&A)>a+{vS;`C@MLNSSdHcs0L?DE?H zjt@o^^Dd*^0R9W?=Ff5;ZGeqFKYev6u-SO!4*iXg{n88+a#2`JPE@NOFl82x`989) z$K_PdZ?Yc{yPn?k=pL5o3_73s|+j zH%Y}p;9yApsWuT07d0kE7mr8iC(ByI!YD;rj5xfQ|3n6lamv`og_e7-SB@-I7}2MTU0nRzg@Tk`49E^ zv-0!*ZQnf5&vYoRX=tDm8nf)T)KvLtR5F{>AX*;i#m-_vQjE%*ts5cj@6=>GX7T`;0+CQA^!H@Gs9CJkm2h=MV$&GQqO!2O_$1B<_3C$NkP_}3r`)$t@ND3w2<++AL z9620(R3hmS72@nT+gOqSeRD#`3%oCw*a5x%UCts;$~cGto%A;|lBxX;JD2lkIi!*R zv+_lvNN>POLp;;mhrq^W5IYi%P5;lT!qkmYP0?rce{Jm1y8e@vYrw|I=`+2BjR21e zYQSUsa+rxJ{%Ixdxrq9E=i|N1nkZ@&y=g>^wwSC(G(V{q@fxhzilm(~*=k7v1S5y4 zO!`-9{&s|G>mYrmNBsSraziq+bNtAUnw~wU^%!c-Hz;nVwOdNwF$g+A`eI@uW}+K- z4NTi(kA)9tFeE$~a5}9j{dessvz03N@!V*YmTueVj5-mEVz_lXahEpGk`x`q=irO;zF1F62dKO9M=6IybCwu*$Tws(WB@r zRUpJh0Xbl@brUv&pcg3wB5h23{CtYj-$6?cpnRdG1TOuC^Nzbm9^oih3t*fMRI8kp z_T3U?!Vq{{(jQe*FFYm>!KTzdj9TDzlPB8Ch7Usi1UJ1OGi*yHT=)R-u@3$*=NT_- z6cY^ve-R@|=-u{7KrzenYjkve5=^bFhkxq|KGTk< zwW>!$ayR=GY&&24X>g@K-Cy%Et_n{u5aI$piW8*9t0XJW3t|%1s@rojwHeDEZsv%uz?Ga5FOaNIPTNQTE%#txu zpE2(By`OgJoohi?q!Xo3W{8r_ta=vfu{EI(8}3wd@oBh76bP59-hpUtuBK)vLL^WT zaeOmZUg-K6m(ogxulWEO0UmX@2Dim|DfK*XSlaz?kcMoWBQmm&M8#mBUQwlud^i2&%xYWDutW zg01Z5Xf_^I{Zp&&pSod1NcSw3uI5Et^D@zGCZ=ofMxn(N&csevN|^6relnP_If<8< z6RmK%F9k6K(2l|>L5zbO`aFyFf?piIJjGJzKy5+7`)A7x|CY%H1$z8I0&yJ+AB+A= zylsiF2#OA?N;X(fLa;%#o=p8_*ks$R-|1^Y#u;Ag%<>*26 zCpyoyF zdLVv=EE!ySH`mZ6l46T>vMF5SgI*ER^V5??5WjvB$U zencAgNq>;}QhD8-2vW;&j30{mYGb*1vn+WqvdCzM+l%?f*ZK6)Fx;2Xy(=DV19*DV z))5oH_SR{12_T}x+YOX(E4_QZZ}B{~yw45C;830~SbPWb&L7(9 z6|GdSlXWB+xzWtOkIxT0r}g7~FxQ!r-9J#k7UZlj)@Ww+=;PJE-}f?dDqk$mzpV*o zupT^B4mUz0Z>yX8RQei2SEadZsqBXX`BQ^7uG2&$33+J_6$z3UQ)0KRUud~-+uh^W z(>}86wOC}$rzx#My&Kw32w%APi@zuavXlISsw_y{h=lw1T@NNxJA z-AoAfytRQ%CF=43MJ)t|ARTD=QN&(=p2oy2`Y&@=4qQ_U zqP3`OAi){WYX%Q)?Ldj1hxCinF+&@OEhH)hT`O z#Kw=~o%x*>d|o#BzYTyjzAQ5((Q}^lo34uha$xb1 zL{wDp>v$ZzVWZy;PYHwtZ}Epkv-1mUSKwB?o=g~YUcc~TyL-!K{}nbn-x>PjJ4x*t z3Jt9duHjyNvn_PB5WfE5a-{N)_4i`7jj-UW-J~~y8*PRNgqH!%Qm`GDe)X-hO(E%V z0Z%9KQwV1uXl^+B7Y5o8yuH89*z_Ovae2f%@GvBrK$>#tId|O*u;#k&~12 z?;X&rQJj!qYE;zM48UqKNThr2E^Qc?DFcFd;x9@S{%YZ;ItM=fp7EP z$26w^ofpSW_{htCc^C851F8kGZ8wXRvdn^S6kI(m^FW{St*hRus8zB2hG@NuBmMg4 zdKueYs-lW|>{#+IY=85mB-uD(?*y`{LccE|{85tsqmOzPTE`9>$6W|B&#Dmcb7i{~ zFqjkH!-G&g#0-=6dCDJPBjR;B&T;HBTPH+{BvH?%3^J1Dc*|h&khr+GC;W#n{ySlA z3>7K9j}puj1Jqg8dO*K`sPbFp+%wK}bRIYd4`0l@N8^uIo{cQ|)^q1g0Tn|F#uF82 zOc>&1Gy7F>L0*BLC84L0#1yF7R@<&am`eGEHG(bx8^+CgL+Q^J=yjfOgTBdeXh(G| zk$LRZd0N#fP@FFAO}uI^;$J;$-VqUYp`j8gBPZ|-({)@a)B2^!0Y6F{#F{>AmOk<} zW7N97p$rxj1?{0U9ZX4mUX++C4qD8^Hy?~>M)uj__#5W%#b;!71jU`+I&nplRU(2C zD-_6eB^bNZ;)GA5jafti82{^j1Us4Uhh6y`kYcV}hDaB0<9Z>{If z;`%<*RL^9)9@TF=yNz+zP8J^)BJzsufh?+~ba$0R=mHpUqD{E+)N~@AU+N@72ml)g z#}lZ%{)Du6Y1Tcf>Mz`H=BXF7fMLn(fy-ir1%C)wJrm+na6Ojk_{}u_{L=Y3HiDcy zwvp5f3lVE6r=Wm8J~6R3&P#Q0+kLy<`%P;0V#N(?QQz`rr+eKZX!P&&zpv=$e(yM8 zqMYgL@7a#syG^#9YqQyTRFuAa=bUV^rb z4>h43p$cKn`y(`8FNOo>o?v<5%$B}Ex2kUPBx;;3zAC&`R~y(d?Ko=i%*QVOB+WL5 z%WzPAU@rImSsJzYnbPBMY{Icx@!hNFn>Wub^Rd!;lf1#w6AesYuAwv!|G+#6<-xFn z@?PpM&H8^VT?aUo{rf+*Y_c6AR7gm6vR6n}g=B^7O-A^-wdLRMyG_K31V z=Kp@)-~YW_*PC!U=W(9o4%p|XAk&+Jj9b?^nc1avFz;!IIYcj+6pY8XJDUp1p#1Qqj5pZUnh z2yCX$1UO`Gbs)6Uqz#k@M%8+?kxzYHee^3y4yQ$lB{E1~%(>zry)mK-K{|atNC87C zyj)OM##^kWN0E2CHY0W`G(Ruin|R7ZWM}gD>*XRxb&`@tk-kWgDGon zlu_qKHSH}MKJLWtCoA^JrLA9$H79+g$;#Z<58>`>h(YrtQyeWtmd?9+IeeWXZQY*l zDBt)fcr?&r-)dWX%783E1VP3&Uf)=0hY?>M2?`Dg-=~P^3jqAA{CgCe;!LCv*c_T- z_sR0klN#H+;zaE2(w1|2lsfpE$fiufDJdY!=$+LqDPC-p&1QwMgSGFHXUyWD*$b8U zjV5n=O;e|5X*lx1-(!n8D1_Xm;+3BdX+PJH8^U;{&P*gvm0c@LrjV9P!DCNmd*77> zbFBK%=yT?^vZCG9F6Ic9M75mUECuR}N>P$>ISN+tCqzMBMFl;uywgy^Z*Ck2SW3jvoARoB`9o`xAuphl^pKsq*gR?&M{6-sLMf%Q#q z;?PmIHg3KPd_0QzLK>0-?IuJChSq0Os@Y5R_s6N4vEXAWN1WJiXsr|R|KQ#iyIDlS z8zLawy0M*E@=4S2{QWLD5AD8~z{n$M$<~I7u661AW;n_bky3JH4KKf%i;g^cyN5k1 zaXmm=Eb@z)^gOq}xCJKy?`x$$xsF!h9iQP2mrYyis05~?g;+a3-uqPX{g{_X_js9dC)?5f~ZK##M?YZpEq-Kv2E2h5hN?pcH*w9Pp ztb0kdGq|Z=41A^`0eNU6Dd1I%=Rth96e#z>c@7TWOp z6Cr|C{%(kAUDk5z_#A$DS(VtIBddeQWZ#IonasG`3geGY=i<9cqBXB?FPCLWv|`5R zr!uhDs#@QvVJs?G4Sssd*4ALSt24#fu8lqYvRT?^b^rXaIFZ=TOh@nBOY*8IY5vb= z0_)cHaf=L^2*Tws6`4HPW5jor_V@RHf~EPaV?{|Gp4YU?@O`9T1pA7ZXzs~@UsxG< zGye0IK??D$mq!jTVNg<08LxbMODkwad45paa^-dRPjJX|m3J|nq-Z+MId(B3*BX|2 zvDKDyM}h~93gM(!^!+f4gJhYWl785p)(4K2qzQ-?c z3jX(AOHyJFKqdir`Tj4Ks5Qqr4O^bj{5%vCN$^;j<#swL`=DUP=isN;(AV*k z?n0+HTkuqmQ`+eB4TiqFl0KGaCZY9ki zcYlr8o(N4!+C*Wx0FLab-~KH47Dbg(g`-M>ONHN9UZIt7#fLQDVFh9nqA0qndZ z2j>1PP&#KSZex?TZ4qZ?eNet7neu7Vq9~Dd1A{YC7N!Uny_*5^v`{MFd8IwlZ)O4Y zHQ9srHSeIHqD2L`BV}@!4rae2kGo?^Urk+^YMGri7NEn5Q><3_#@;d-*H~af1AaMb zAA@;m{S8W!wJhtsrOUVjxtv9-^pI#;cWF6i+31C9k0@a=42|vmYE2>R7{z;z7b@lG zuAHgg3T7Uhukr;52&3>rvORt&QOb)$*d1-dxCAZiV^>x_*7D3htw#dKu%)FS*$6g5 zK19EmcKm9}kIba!+SQ~V@27;UX81DM)t)?#jqfwB&62-Jk$8glLFyFJ?!c4*GESFI zH=kqIFTGOPUdxF+KgTWLP-jGGYipk?mvI-Gqtt(xLsC1BPQBAr6o|93v$8JkqOfpJ zMh$|l4bQ%*oF}S3`}yV9yy|IK(kt(s@+~2`3X*@fn?B%Q&D{c- z+fv(l|E+@CZO;1zVujVIv7vj_&5lbNcT$SnHmYN*b;#ePTshDh^u0peJYg^H0Oq#JgTm9)zOWNwq zMi0dte-IBWw52G{ZzbCYJ?b#Y@IGjR^^Oq-2X3c8|AqW0;(nOF+ttQLbTDNvX)|3% z4mDMhf$f3)uu~Jz0m*mC zhCXyiR3~`!Zg%di0HTYBLN*cljGz4heE?F{t1O54jN7r%4y_mf{_^^adzTnA(Jywz zcEK`=a!B5WuJ>mcTTJVv>#2T7t%o#*F0cZeHB4q!xN`OM^7*thEwzARvD{%^CKYTw zB9O-j9>pkNxhNoXkvc}?R2|k+y{z2yLLVHqw5{)C*X7~MPU`J`IWzx3%12KOx26;Q zg%L-!+RV|3R%{9Xo5ih5UM04tDrPb1G6eG-?XAb;1Xa5(bVs*>C8pc?iD1iwM! zE4J|yfhUs+^A_rTrKrfX1d(O_<0uWShZy!OM3MEB&TO|{r@$RA<7kXw63z~dY1R}stG8A%qpThS zCGU}JyvV*-6Z&mGU(}MEx(Q})8BlYY`^SEcXgf*T5dWdf$0G9J@qZIRpfSm?v$cb# zVZx>P?I~B9yiixprwVkw3w5^(?r7pNAOqW>?X(hky2j3M{tzli$ypS=#@@0^Vdx_k zbnwK?tA0a{FR!7kgKAPEOsDxt95kps&ReqAh%)4Z*QHM}%TNIRKlSJN+H!|jP4R0u z3S{xD8&g}aduaE~q#e8hId}!73x_!RFQdYs&GjDqRh?JF5hz_6*s+MPb5-=zY7bD+ z)zrFtZYKeJYpxLAt^%R_xD1HMMTZ#2@+gJtB*~ekBas`Ryez{r??Jbohsf@|)!9*7yQa zjVy&8O!)esjgmxTW#PFE~x50FM@bEBEACns^I(Ag57VdGQZ%P<)T+(4@}m z7w1j>aoVr^QHCr<9K@}Jm_=%&5s?I$4D7*VR6Q*dYw2uuF)t38@oE>aNJa#IxGMXi~{jz7G#FcrKpm z=@F!Efu`#>HQRQ{}M0pPJN$mS%p{bO&! zNBbFJ|DiBkh6e7h5S8DSTZ4@~Gh#MJ{T!4f?fW&bCr%=9WW;UINDI>ASM(Tv@dj77 zM;{YDaYd%NS<#-DiXt?$IkQ#vDE zY7mFv&H7$=SJl=&n|wO(b9A+&EAoMU)R&|gqY|b7dSe>)yE%NG`xdZ~WN|dsscwZE zi%Xd{ff_3q*XYprY~Up{t{E>c2hUK;er1&U#pC!bjX?@7RS1dXn5{#{S1|~%gpWAo zTt26L1NV)csasYf98t8lWSW^D|Jws|V=N+9KoA6AhxVn9%@_wV5cxNJ~qbfPGnfZksx)Z}>1E6t^9Q_dT?!_ z-8Mu~b68FD>Q-P^n)kK86SS_e+_)$<7NB>8q~3W1G1(iCjO~7X{TZAmbx=VaA?4HV zcGr#Csc7i*F9Y+~vHmfGB(gK$-s@{uXo(gek**!SQaNny@ndMns=5x)>|5~c^+72O zJmK?H4+G*c8Z@$L~vW252`#c@HHMNteVb>`Doa`|0+QdacD&vt|XROG^ zU`y{=^V4PuyaO?2b}i1LsR@j98ov2A9fLoH{D0Zkkux`5FrhLB^SYo!$4=Qb5bqR) z?N)qe+<0M?d3pFE=)dw#0pS;;M)&r!{~ahI{OEUy-?PX*)0}nvZWda0kx+>H2*N?#qU&z z#t6Co;7nOu5`rD$;im6^5YN4>%gRa+#A^D&qqfdlu{N;LKW_S31nb|4nFOT|-SY>p zIVgTSm4NDYDM0B_XsEk%rmB#lonf5#Ihnl4rOrCAq8z&wHeN_Ts+s7tPlecBrzac~ zIq=VBbG({a)j={=GAs0W{z__%3v=>u4yuq%)-ec;# zyZY892EY`uuAPYU02cMynfo(twKkzsH1A!msX(Ok_aEHgmj6K=v)a9)-CxKNVe0D1 z`Br)O1J50t4vmP<6qMmyt6@rFbF)$$_EARc3JeXKyJCQvF71SVoj+KPmX=%E!GqVN z#huR1nk^-w9B-j-?91%^>mH?iee|Mxg;hVsu@366{gf#L!@>D~TyDd?>|POn%p-f2qcHqD><%^Es&o!L6$Cc5jj zE#Vy9$1dJduk7>c5#bcb%NFxYttj?)NPuhRF?Z429BvxnZjfmg`ea9U?y&4ns>{?2 z@~|uS#Ln^iJV8Yb*2a5rwh~yiY09}(P!`>4e7cW%>OrraNxj7?~E@$LH`U z=9OTtD9@4f43RrhT*e(*C0*|5CsmV;j;jK#$|ne^zNK>G0}-*H`K%>B8r+=i_S^V`$m=s5JhUQL`I$uVy#vqCRCCUOa9PV>2Au z`=yp;F?O{!_aDd1_U4t(72M_Ah3>w)kEJNX(_gZwK$dpNCpTO{A=18vSaaQ^eNVQ; zq@-6rY_go3v8t=9HNd<5R`r?*Z3q!AR$@Xze11_9Y_b&=gAf~wo`gM%BZK6jwoo+J zTtjVj_0alj-mHu126r#!Pd%lGaL(n2S@KZa71=N|T`}O|En!XJKBNv0!eGsFks@r* zgyekhw_=eFQa#p}M%IFE0RgIQX!rm;4M5mSgB2qaRLBubK1asK^P$Wf$-TbGd~tHD z(y%lU7c>tsaF~+n)j7=naZ43SiH5mA4w~7=Rgw?`T?OSugDY{eC&`xy)-VYQ3lD=1 z?t?|F9IGk++ck*%pH@pEzxw*#CupBxW*UNhy=$<1HRoaBBk}?JGw&b1IfaFh2+jiK zh7~0M&K% zafipdq4DC!-M7j^p&2}y`zJ0#!jwe8Cl}|t*2mk!@>M=(xSt?kdwDJ&vN^>dV#MM{ zg_)dOmCrN0e}Px@sueu$9^HOcI~8tBdwjGQ>-*?}v8z}bHg7NcZUdU0U0$xCMx3ae zyA3$|ruNVUstgGg1$T4@OcjwHdn?8pBl$}2`Jt=h%&rT^2Xyoyl^F**PG%bDsE?yx z*V{4j4PLUKI|q|+MKys;gKe4c_Z%A^u)P1y-pwmrFQv;+`e4krM*iS~`u7wyiNt;u z3KbJ|=cnffB%~1p(PeK=hI3B>$=~#XvlMS=Ob5@9FA7vo7J~G3UO82CA^s2}<@!)m zBt|hme=a8xzpUXbuUwm>f@;YL4b39HRCFgdGxGo;Fg$Hrgzfqo1z~Y&ANYTINGslE3oBLB{&L%5A>A+qYq`tut8h}!133i6>pIy zX+I1rt{a)-mgLb_jRZEJ!2`V+5pf4=YE;gcNeE!7habY3!+IHzm@MD{Vc6zd`x|!c zRbC}WQvu)?PNF2)6NcuQ_G$p!zVlL1`b@$3o}bPE&CC2p3Pf>EqD){w;-jKrP3sPj zld;sj)#%R$nCY&E();h*xE-{U-&wp+k=gjkyr^gey;yqUNY|~q7A1xKz0^aBAnD*r z^9LqPnoL7q+s{FOB*_^5fMN~kTb7qi!%?ecb_Hlbo`w1WG|XQ)|EA5NV633j<-^S4 zkYoLq>UDa>qM9BqIdrL;Je(c!FnaL+K+9>!T}zR7HItte-O4FdmGySbts3i5h3CT| zNF|$N3>|P#*L5@9LCb0f_m>{4V_SqWr3h=L{WvL*1{*<@Z`J8aynyQsiVr?u04Y>h zD55(wbxV%Uv84DY0w3Mz3mm>IrQ#j!WOz;{Vj!i}q-_UNAW9)n`ql%#1V%Z4E6%@m zxG#ivIZhadW|gS@d;0jF^H4H%3q+&YJ)^}Ho^@yQ^z_^w(Rwi7hQ~ZzqCtubsX&>> zb(=!7oi)6kIcjBGJ9W!$S_IV!HM-1c8TLYVkV1mZXUKERr(FaTApQapQwJ6U2zfx0 zuTw5#cL81U4rrsiQzlZ?KJEa2F8y&B`kicmn|)f4M*y#JQEdKwGAI?)>=4ItHKB{WrDi`l$j(%UvXg>2hAfcU)<2d#t*HwI9zhn`zR}9urPSJBk$to#+uxVffAr6Mt~SVZ?RCo zibf!@`vB)#SX-aS=RSsHcp?=i0g@dO2O_8^-mFU)?;CAYWBd;?Di z^4NfM2SU?s1Xf0n6tu#nh=nW`D)UM@*dfN3(g$$utF73s#V~~&rH{kgo=xsMoisau zsY~Y!W!Bfz!5#<%)4%1o@Y(hqoy>A9lBWXZT{@n+pO~r{5{$6-`U^nDp|Ca{7e+W< z^ZwLuBq<4Y+tO6&F&cRN!Xjdvj`03+&+?-32b!8!H>TsfxkcPuv6hP1RS-aIEL4$R z6E32>1D8t718N_lCWF{_lb)&G2d4U--B-(W-X9i=4?NrY;+87DuVniDt>jA{pPO*? z*u&PI&rKEnNAummf$3)WAzNEpNDi4JY!*f; zJ_nBgXwx{~eDap2q@v{5D7vm`X<8v#C9qHE^8+2qlfu9jTJyENh$UhhR|ZCW56-U` z`P81aiI73`o%k^zy#nz;K&rv;4RSY!MpS55Y*ul!$JCQFrZpEmnB#kF0sf>f#^o_Y z1*R}Sy-ye0&@%G%ovUC`TB6AJz(2x)P|PU0@%VL(4!JD zWa*q$Tuv>A{qee=SR@_RZOeKu9Mf7}6g<&_2u+bDfAM*V*UNEAxw-sUSpCH;_-i!{ zxAN+lJkzM*dZ`QG;(nvLWK6o>z!Y zq)Ke^zi(N>zyL5tQ`dh%hLU!Ft?HA}t>moFRm%cqAfwK*Ac7wQ^+#SYKUF~*cDP=`M_WL6UeLWh;_6tw$it9L2GucWJXcQHJ8AJ?}s zIIWJQ!46&zN@`kTp4X`uFw;a35E{*X3pZkEWJ$ENC@KsAioh{zsSMZxnQOgSjU(8Q zvISN|N?Kxvlfbe!y=^2S_02}uxJ@MRGW80u|O);86Q0noXJ;`lzRFF zu6FiNTI|RXpsy$9I(TAv%w4f(iGr&P@{*TraPZ-f$@MfeAIXw~xZ~x;A{RS_qfYUo zP1q>R!NU{MlaVaZHf-_jMVSYnxjV2fG+?n#aCZt9Nf882A}o)})r_J5;# z4MZQN{NXS<8d6^fHZ3b_pVQ(JoLgP`pA>_rLyEF<%QmD(O-5SUpBwknj9pbt4JOL1 zP#h;I!1ztWjtzRwjoB*FEwQ%VVdT2I%9!}mD*g8&z~J@8H=V~-cVes? zXiDwgMQDYdGIQg|8dvK{%b~WDUCs;hH+@S+ECePX47?2BW2BPR&Ce6jtBkWQpP&I) zD0|@@>P3ALul6xsP2~@fix{I*Beh;JiP=>#DD4Y?+$eq0Ui^{lXS2&U$eM4e3uPj$NQh{<-505J(x63Wzq$%FV@j_z=WD20mZ-X^94Z z{r$S^RB_ppHQsOT&wpS90{kLjg0iw$A*9SPi8mgb3}#CEPV78|+27YeV`a91kiw8 zs@t#zw(Zy7^YIgkz&1ZPA&_;l?hsGsuoSOY5OARu(q_~Ol|2|CWNT;K7|i?L;XnWv za_}HZxQNI|$FIhUMyGS6U*g7g0K$+9deCx=i}$;M1`ew*{COqp-WEguA+`{s{!~%q zu-S%P?O9b-l-1R)Z=8PP)E#`X#^9s9rR@<7c!Q6Z&pyuwJ!79}@lt+%BNE*t$hagb=Tq`IKUJ?zB0gUBs4Ule(A#6gsji zvRkOjk$dWtc5IBjHat7*zzz?$cI3teG;IcD^yZNZzFxIh)zr5mBH!C!iN^dXeqiXY znj{OAQ7%ZW@Nyu89{O+DF@JxH4k6vBWrHR>U5Pbik*}(nJevc0(5x{{*kOVjVr<04 z6$B;K`#u#gVz{+J9ABcpPylXBQ2s8=$kjt#TYZqh($mHB9>cHe{;taH-3s ztcsAhC6lJMKKq>M(7}_d` ziqv?4=ZlEKLH#7Xr!lg!s|mFzuGbMBGp|snV~a35Hw*W!8|k}W)}Vk$on-+5h$hUpHx5!hf#W_=aB-FzU(&`{nxNDY;Q?B1C`k$I%x7DzNe zV+`CJgt)Tcvx<445o$5s9TWPt_S;>AYh_}$tSkympn{@8jMa6F8oNf)jiA)mNa8K@i*-AWj-n@R_f$UwcOiAbkrXfWR5V6De0plHmN`%12cgfSM&$5ZXe($ z-pxi~$pyJGJ_2cWcC#M+z%p^W{-b~h{@3kH24v(9a@C*5^OI{QBrO%^=&%MKTiphZ z5Hu^V@`*OA6Rd#>&B<#GQ{whFo8xJlJ3kPf1|-BC`rTR)Qm!fHn8i4?kIOA=@k$2F zAZCledY_);!nf_hG=S`GAtndB}~PU z>FG@0G9;N1g3Py^k*g)9?>6fHMNCv9L@2H)ir3vistOSa*7o+vZ0o4Z8ViAb`W>{X^O0y=j|SzN80C6_!(`mYThBY_5UAO(o~a$f_3_uC zv#ZB?slvN=#mkS$RGDtkTn-fvOcqM@$J*fOJee6^oEUpMXubRAwl%;~peix$KMsE* zwtLsYE5jb+wBX&!o!eqJ1AS8ksLejyJ-2QdNZPR|em*o-=88u^Oo%=czS*NX^+v?FdO+i71l#n#JNp7cFjYJ-T|c>Q6l&>Ox9inVHZ!sCO;gg2}uVn zf6f3Bae!9#8DIrdL!U1C5dj6_+V-RTdN_|&88J5YDAncrJrQW< zYb&d{eA#QKod}TtZ;;te1KbRqGJv`|+vg4LtHuB7Rw7g~hb%zZtf8Ud4=$uN7qL^A ztbTehc{U**^wGG{eZ@ycZHgktd2Dfg%bCW4ItLxG6TW|nFZ#`#>lrs@{KkJJ@~QXriuiccE!$ z=HUwe=U1HDsR?Sn$yfRaLWAYFh--!YuB4bTtU0MJb)Zrw1cGJatEvtVC)iS4dr5#ixpx#JGd~X%iz0(d)St7uQT}y+}i<1UPzbK+ooYCJi(ob@sRf=mO?^UOT5L z3$UIuZo;^=0-3!lp9gEMl@Vbz^j@EeQ+`e!c|CJm8xm+hcibs%Mpu)`X*gNsm&oxaN^_-YBusrZbq@RTI_&Bz+G%dlbrEr&HCHhm z&y&9*;!3%<0jvPY2xN}5TkvI;Or}29itS)Cdw{Q)HEKoG&jG@Zb0ci|Uxzg&JsP*R`e!e1%V!Kcw4b|pv+Rt(MIyqxgRJs~*L_LADR(;# z$Dl2$TX|=VAQoU=;v)(KwLa2G(DIvxW!rO5%DB3^8i?tD92>9=EI==h`nn*7P(zE& z%=DjqEH41Ut$;F8+J-WDf`CIAf$n;2k)Rg7=l!XMrdR`e9waZY>Vdr3`N|gS7q)~S zjJVw(Xo`jPhjHnj!(Xpdc-j_YC1Vgz8E957g&=v z%*QCUzOJn5CHp5;x8r`d68?G+_&2y^K0Y*8$^0)>-78T-kyMW=`ts7TMl>i!$2 zF-hn?bdox)?Rouj3ot{Y<}Q5yn1b>N(){?gP`W4(VIWu&NdFd~_ZEGBL?vkL@3w_2~S4K%j<-j$uMH4iX znVcrL$-|6jNF7dw)UBnjCCiJY1=lxfQm}XlIUS)zpm% z=*#{t>^h~PZ6p_-Px~R;r;7-j1u{7r{nW-UP5{V66uqqV({t-Mw7AjL4?)7lV@Ni- z!jFPI)|sq&jZIzt?|KM=`^x1A;&sppO-xBWPM=>}0sst-kg0_ZBcU#1ty;!n;%cZD zc_rMTKqpkN3TOu!(}Fs!5hFhVi#LofIq~b%p~y5SC5#7ukw++5w%x#enclrJxVUX( zQCQ{UgpNJ1yW>X3n)n%hDUeb@`(2YX$qZx_P*o63fxhAE=e#V%xjYRF#l(V2&z+P$ z+ym z#+I2db0E}7l8)`biVl&Ny`N}OjL{059!v&o>8jFXO7Vr3k{^gsjwaE5v> z7Dr4>tkdXzOC%;xA*apqV_FET<5?DF?2y7kNt4wa@2(-eA`Ii0D8zKI^D~eXChA6- zW<+-H1DUUeHi|4{QX=S(AqzQ(8H|XCK!`9Tqg|)gju8kSe+Lon=LE=HMiwsT0D^?2 zCMhv+*X}@_H0ep;bO$A=Z~0BG79xn@fezF5#y3FH8%Aj{l&i-QJ~S0 z1eqfJdMB@x?25}j4^NOSm$2RlUU}BXk9dFo{`EKulY6=9O>*OG&tYX-lw<5(%oL_0 z2!`jvmc>P$>G*nFdvbb*`%r>3@JkS=jQbh>{5>F4f5=CKC!`-uoJBf+zf|$r4%1qc zPjE+y-H+_z<`LX!i3<*K&qt&LgoJw)a%YKGFh!Uql&{_a7YH`2H2`$fpatBPnlz*OnP<h`jy^ zhz2mD8vlLwS7fR6?A*HbOMfc@T|(LUmMVJLuJ%d%B~$CX$2ed-VNBPpG@pk7H}L{l zj)pM%0)n=p@7$jofutd@&pdF*;&Ywj;x?$KuwZy5*tPOJ93^1pI6slqa6p%?r7XH1 z1(1)7Yp;Y~47pR}yC_^a2WFR&M&<9flh3~J&T9_{5Z{q}-Pg87nqA+lKz)JHeM!1- zG_$M~mD7guWr<4qLeFOeeo&3)w}6>}ns@K3aPp1Mz9>+|502$g#)97pg?Q_^lYNM= z<~gEp?}quQv!~P4Z|&k@pzi-2jVKp#nS@G3Sge2*Lo;amA_s!QuAE;VrgIRF-POE2 zEx{@8Yj_tJ>d=+rmpgOw2QbKdUGR`gXx=_Rj~F&V;)uvWEGPWb+EuC2BmQBcr)OE=m9p$aNCEcc<-x6w5>6^2*bWy1_K$WXYy5{UQ&m# zwr%WZ=Dr|mfYKeQd*$2<%NRgkBHQ@S7tj;CT)X}}Ug-mS*Q?SRlH|K^cA>+}Dn6I5aHmm~jR{)9#DWsN-bM6py*`?>57k=k;j zwPLfB%R48#;wmg#OkH7EYf<#_ndw$?^2gZXJ}42TQ52es7@y>AC7A0)p@?%j;`?fI zrU!#S`J@oAhlP5l>NUUp;Yr-jm!N#*{Qgp>d`>G=9EZ5D{ka`TIu)}4vV+N^T#JdFmxgm%DzUx`_@iBIHLBwg^AD3%{psLHe z{)bQ@b9|jF_M_>JBm!coW>sX%m3zl5f{{gC*Px)R18(H0Ct9Dt%2V!6p;wFxqX9`N zv+Jn&J=23wII=yhi*NXTyW^Q^kyp);N>Cw60t7Lj*YuGB3gYpY1B8fKk^#S4aPb6E ztbV#1Xb3Z*gk0|os%wq@5|JuvXoOr&#tCBmZqmmlL4MIWx1OcX~O4cLBPN%LX9`DXHif(o89D>;F0~nD7k`ylC2A@ z6zq8PNoEKjh!>!ia+w&5bxSlRxyPL>E58|+p7_RdthLC^ffW&vyBNU0%j6gjS-N7f zjEr-!9bmws2c>=-?m#J#0*(O|e9dmuqa0}!S>n`LSjyFt$+ztV65(7I9eH>qP0*C> ztY6QN0)7tm3Uzi~Lwo)_D((;OIrs>l!>S<84{B)yC`3z%$f{pnoE^GD=#JxFpD=`D z9O0=+p1Ev-9D3Ica?_-rKfjLbHh~&~I0NM#Kc+dDc76bB%k+W3)2TF%p1$~e`9Xqj zV3Yz|4D9YMdQwaBsPL~ab4Tl3jX&>m+zQY^>uyQ{&um<-gXjSN?Ik5qqkWF%2Q36( zbCAE3QRL#OzJdEZP872wQFu^LZ1Y1FKjlFgY%H zf2i1^TSH07S#?t&K2aj?7B4QYseoo8b?@BV&-@lPy zyrg=45|t$GHsesngUZ^eU4|Dx#U?TaK@8x3Ie%46CZkX6n}V}sL$hJO;hiT3C%0ku zFQSh}{tfuC5D1pi(-{6ilhA`nL>c6aCavYoQ*z{`$eQ{c0FImTJWy4p#}EGdg%#e@Di z+%A-%&sCGM8D<&jQY>ElvNwt3Cyzq$R(2`s?Y!sC^(F2NT9y=IOx;Mjrb(y$D53oq zX-~ilW;;Xmh@u+puOtL=L12GQ0@uo+!ncM-X$t{JeHsh}If<+-6$IT8dESNlk0lI% z>Qq{xJdG<~BeE07!P3&>215b>+e=R31VrBtz(@&X_}9M*0m9>tbL=Tu10YsN!x#e& z-)o7ofMp?f%^6DREg6XBV2ayA4T&Z8w=zeVs8?#wL8uGdaTcz6q71E45bq?Xk$|)6 zaxGF-2+!Gr6VPvZs}r`1vn+gk)`A-+Q|{7EolGD_JS~yntQ?DbSQGbSErK48P@e5+{wUM~7?EzqL^-_X)!Xyy;cl zoLB!<;|5F|1*|aqt-ET_FlOZq;D+pKH;D z_WTlNSd)~pvT6uQW;dzzqHz%-U2t4K5IMp=s-@-~F%_J8+B|-)f;s;5 z1tU=t-F@7c_-<2mb@j{%o;0<^>_{F=6fkxspj|-uL4{&KoYH_h>>!v@;iwFJkYOXJ zdQofXd_iG}jL0-|+QC-s+zO*g#&Bm6%nztc_xPOMlUwTBH1dugT~TZ6>(Ve+0bAF# zD1Gga)ocZl* z99!UP^!}fFy|3#$zAAcGfO>yAYWxn*=lJ`>8g@2cH2P{6XG*Ngc=jb;r84K-cn~11 z6%|NdB5&4a)HvlqtVVyz0o_o~vn>Nbas}H<=>}eCHRfY+68mG)G2A(<>(|J8T;`v= z!29wr1BbD&@v%TkpH7pfAUy-a#hZID>HWCEM@#gdVgx3|pCBWYGt1q#TZb<3OHQmr z1dV~abo~5Q?B`{tc?1H3YgkgJNREy1`r7?=@XZ`X21y|`DH(Dy(1`eunnynj-0A0= zTmO&xBq8q%ShR;&zELPB>5xj@0cE`aQ(`ZB`Om2lz7I@kZAd=^S{v{$ zvun}Gk4Uh{3)F%SHrwm&IoTTnV>gz1G#-p8S8kpyw6Kp{Z`;F!8x0&0+W`qe-p)H2 zETgY^x%sk7KFu)*CdFlk(X*<0d7gi`R(n`2Xv7h z?zijYM$R6C>d&NMFOxz?Py8OZIYv{=F2kZvUVII9fzf+zrj?=qO(!J3i&J~9VW1Zv z#SuZFl|{+<`%-4EG^d&emNDGB4ru6?!2co*PP>{9Vnk7A{pQZVp~3WYOCjc0G?RV7 z*8Qvpnv^2FnfYU{lE2(VM+2sjlP}DlzKWACE_?rKYC6j5rn#wS+;E(tQ0+8&cDI_GKek2&AbROZ_to}fIdTQwDb39kw-WMyPq@m)d&>>d{$ zDkqha4yoF|Gt0*UOu4Yq6at}o!IC+8;JXY8vSto}Qp)%ww%!1MP3`2z2NvTN0?>$i zWZ(r#BQ61yA1osljqrKmq5lA6K!D^TAPAz)QiV^!We_1#2b$Ih+c12X&jQAO;jkp) zZd@LP_3|(!u;K@xDt{1ZP-$WTSU?_e*%Hu38Ynn)+IS4 zck+9#$_^;|jON>eFDh}}9zLYv;(BlMn7h{mMhz7E^@`>>a+HOgT~b=rK0Wcb<>gkt zU%Emm@*I>s3@wm?qtA`|p-lY~ueJT-m$O4dSsX(Pf8EK}Zwl^tY3b^QBN#Yhs2~yF zmeQS=9*S3fdbV33%*FMj!=(IcOcWO9ZGnY^wg*3}@2|2YR4@2_>RlR8-ggzT+-5ua z?#L`rF&oe_`ZPjeIQnQK-+XwNij6BK zsf*sghcd!-6o$83F}6(%Ip3}uIsA}@N0Z#Q!H`(1Ctv(kA{SgPKf8il3)nlv(jtkV zQtQt^u+U-GuoVJr4;4hy)1o+4rVw%uu%JaifKEHoLgh6+OTT$jfh#x=5C0G~IBTnD>v}bU@Z|1% zPC4v&_;95W-zb?EKW33C ze)R_$&ya;EUJi7)fVD#U!HD)FP=aMQR?#X1VgRtgLJ4q1M4nk@>4As#6|2kS(42sX z3lPtj&+MH4K~4fmtYi@AkB%|&S8|W{AZ76#rQlFtJT3p*BK-jg`Uj94Z=$YMEKVb* zx$og>CaSqku`>_5RY_LDaFg86_Y2!PO}5PHtsCYJpVsijaUL0oVZLNEB=azp%& zgU6{wk=BQIhd68~a*^~)$Nay?{xMltTve!Z7by%iB_FE!X*_C*G@4R0n3 zt`+{9)i(_6Q>p8sxYzA(()wu~<|S=IVlW9h>xOe1eXP{opSTY;K(o(7qNs$ULLh2PONJtb82O`jmGddu} zOrA_$rnVy*Rt_hSIE+F61L`Y`xbFwowREcl@AWhJJ5c`M#w&i5eZih8R|z~@ zo`9+hrEYx-#>Ecc8Ja&)$OreVB{e${LK+a_BP=Y@Hq8cU+t(F$h`@mEITMV@F^haf z#rcx5ejGhNZaSe!lOH=Nz`@$ESwU!icOUV3P_G9D~ZNth{V%YrDIOq%^>CnJX8JfJNvRahvhS{STJ% zxPCAP+pJ+me6n@eGErrD8DeF~M9?ItS9znrjRyo^n)1_ueAuLr1Q+O1Q~oQ7%JcDx zOl`Tz2FHb4>i;}~WUhc7f)ya=ni^^VrjG8@vD66v8Y})UwGAv=46s51^rc}6SD#nG zQt&9Ws3x^2qFJSlRv@yj^fMI0=#DQnc^1lu{18k}Wp?ZOLAS4lJr@8*2|iGl@D)vV z&p}}|d*gs2B=N9yLI4Bq%h?zZyWw-P&|nj=&o?$SFqG?D9GIDzeYYpN#K(7uGHDE> zh>55pv#5v&B{lSr!*YUDEN#ExH73U^1=ShvQg&vaa z5zd)(jtuI9L2#Hr%v0b30}FQs_Ewu@&l&$)qA_20PE4Gx?5UNGp2Qo7j=IgRw!^OW z+4uMN>cFf}4u$@YgRU5+KWePhFcy$KyaDk&M4N_73jXZE0=|ErFH0*j69Xj&o(g{m zOZt>})(U@ZPMLbJ)>iVca`nj=Oa&O^t2Q4eRRF z)>fJoIVle8{SO6H^5nor$omPH&--aU5+{{oHM&V{CPc4+ovLP`r?=z=f|hxCc_~)^ zwl2odY}W2@=c6*?LGRs5F0fBd(8H<4+kjU{{7e8T&6CyE<&cSq1lq&94fSViw?B;K)SEJIKD?P|?OC`-P?fNWWp9N=VpT_t8|}iY$iDD5 zCUQ`}g650vHQ5d#4*NHICI%-SBJX!_;al-=iKe?V%BD}mWpo+fI|B8KC!l5Bfw<0j zNLy|NIOE0IoG-MmbD`Q?!621Gde8}2TDyDzy)QkoGY>R8HAf`B4w033oiMb=fX9gw zoowi_OoarlK`UL|eoG6Ejn&+yCdOx7F`d)(&Pd2o=Esk#tA?>k{~akyuoycV<{@Qv z3OTuiK6A_MpKVp{I}4!JU{(=9UkhxLf^|?~VepI;U-UP|%4ultJA0ljk;+rrc58=> z*=$^sRzIMUj~8;C;*3{atwwBhy^c-|y$qSH=8F{M{{qP;CoI{F{TR_*!=DbA^rmK8 z9vJee?#M+z{(}7CrOsxE_3Cj3rBD0;>|ab!zEKOIKa?u(Inw&#$C)=If2#*=+8L&M`>Q7a zvjFC_uU!+*UR)j^tSTQ?$mXCJAOxEm*x@Sb8$DL}D8u8sn8BH*7_Somst8eW9|~;g zp$G6~F>!DVa;9kY&wb%xxT zEY+>8U^lShv_BkD(g>0(p~N$A*^}9wnb_csglGVl&|HJ1Oqf z_pM)w$a|%G-k622EljFV-c63@~^HfLQSj6YUYT~U?hwmUBAk@-Y{(G z`rW)Y8U%)B&S>z%G2SJqKTG%eoi(hMqQu*BET+qyRNNog106A0UR~WAlsfpatXlAx z-2G@nt|)cuQ(i#I12L2M8#IaKQ`I!Fhv(cB-4}O{1Sf2SJYn~d#nJZs`{DIn8^|@# z-Z!kZOFR7mH{bc{kxz@Egha#4x#Y=(Fp>^|DaED`#lp%OvuOGxRc-ZAZSS&^sQ~>Y z)LmG*(bA2Zn4t*o)-*N6Zqru~F5%HMyJh>mG+@=k6L4{*((z$%_^!bWo(t}4F~qC} znyM#r>R%~pmuM+G(;wBWwtx2K4+sNLk3~ZMR5m8J z`(N3nSUL-+DEF@mlS-E$Asqse3ew$zNDI=bpmcW$QqrY#3I^TMp&&?i zD@cQsbj)|={=d7HYh96Hf|>K4-#L3fd!LHS%h^M}*7p4+4>e9g`Cmz9SMdo+djX|6 zav!4B-vWjBjB1yHiV5r9FZgB|yZi)15PA=cm&oawrYX?&Rn(Y=Pa-OXt(g8M{yrNW z-Y44sT-rmI9u8N@9he&wjKBN{!D$i~+k8u2;1)*UVFgP8Wn{ts!fpm+#8@a`*c)O9 zqh=Jp6Tyh>vPR;u94$9(yQt9BdUn?d{C3sTyFvA?>`sWYSZ7vh#mjk*TZj(aUia9r z`j1)>BR29?_1`=v93$lk|gaCiG;H6})ExcLG`ezPY2c-P`KH0&9EE`_VL_Pg@j_Ucoo`cWS3R z6n}%9CHKvnH~%TJyZ&vA+AMWNbn)U9YO<%*ZHPIuPOcsqz@atke8ajI`|Koh(Z?ij z(3Wq#`R`W5t%6rQ=`^VtKBq3;C08=Nd-vrLEPq)yR`3(q`&dbLgc;;xOGM&Y$`@6F zrvggcyL3kqKgZ@6EQ$r948gJFG1R<52)llD=O>acHP8c+6E(1Puv%4J?P>shlM?R+ z&W+ar?`V(?O?Wx~h@Z^txQ|%#S0fiKKW{_%-5G?C$S5P>3h zYCf1$(gYo{<^&+FX$Hs{wfQ=;v7la#g1i;gJT^_nc>4U|Vj_O%V%`3+$-}{0zp8>m zZR#%L%Uvh75G63)_9+s?nlO<1pffmM95@8t`oVuyUuwj#a1wF8HS1$(CYqv@AW4x6 zZsP^1*6@~(vrk~5XH_Zutsd&>CVp?a3FD)#g)zG#y{5%gJ;q1Fdcj~2U>WTt5Aut_ zN}q+$r{d;DlXX)0ydxU%Bqeqe>$?dy2bdjQhru#pF)W&1hmV=DHxh11x6;7pn*<|% zqSm9Y<5j9EiCGCjY{!q_aPSI4kz`KqQ-QJpCv~LNdbYsH(9+F|tV2BB)%a#4qQWx) zoY!peJpq)E&}+9|evg`2EQoi-nHoI6--N_?hBl9AyevS3roT8GFh@N&3fWv<1gw4# zprK#&(IBnLG7>Ep%SlX4;6+q zhSeZ^S+s2Q7j2RewWsA1SWth>lr2+W3&?dhsI5y5)_#$aNneracSm2Ex&JO{a( zG_FLH(F!2AhdQ+ZJQn)8*^>1WSfvv8RYKA8ClBHz#TkY(_7m_KL(KvXxP8>s-Byu)=`woOFqUIRbH-v(Vm^4~JmMZ#12rw|NG@Z9UDa`b)jmflyY?28 z&85j}pTm4c|BXk?t`bep_|)c#8i`f}TsTP@xUkL{v9}yp%$ALMR}DkzW78h}v|S}# z_f0&tN;lz=TWXaI>pJyrSb;zxW_UIz*5C+9>RVYK457yw8y=Q^eUtDm3KWX>{m;G6 zZ3+{^_O`2-IrZf_USvPQ&r|A%;!#Gsdj8~nM*>Siew&nB{OQJMMB^$|VO8LyN<-B^ zUVk&zw^HujtHW%sHH{CV0eq=FWA@Y;F5xDsHZ=|nOwlYM3e|z+`{8yW3~=_+U|_V z9qkP4QL$05gQeKHWnxRI)b7a8#sBLXQ42mC^^@Fw((dZ9AoTZ-{&|l#NB{K(ukxRQ z#T7wOZdNsC{N`bQ;{CjXuXlR&Hm=Eh)af%3#=m|%R2=p(noLyc-I_rX^LF<^JhBcW z&`Z_s(!9J%7ht!&Yx1C(G<3nIart>apyygRCLh+|=O7Cv+o{ld3*wt~kcM2zZX-b; zE?@=$mo^NH8B?s-srm_TTsZtslr2gP+z2KV=YNvK!#(j#Hpo$q)#uDq?R)ax&Bg=% zdz+ujG+({qAb!NScJBUaK_K(#i!7(pY5Ss5e0CmN6`OxkV_ z+}Z^LgWI$r&i>@@V-3!D?Qq0`R5(x}3xV$1~Y0 zL9CIl>$y_aeS-K<@#LSshO+rvDFf}db(Txd*W)ZP#RMfzX=pA=IVzlR2 zGG%-ZCG@G+HFa~IiwU!6{68uii_~bbYxRY590haIi0={0o-4`5i*iH9=SX6}ML|-r;oRZ$N$_ z40MHae0g&(O0=gA>q&K6PM!x6yPb5AJp(-HFY1Woo zpqOl4zrtlFVIO1T-iS)qqPv0dSj*n@i!LK4%m{+8yv*$TuR{bE=$-5k`o!Y2OQ2`H zev9U})BFT=>4R$T{nY`M-_e3e18Bigb2_^5^%V%;$S17(d@o24>-lFfn4fiTd5o$u z&)8l?M_{ZB2#0lze@^z({X!4umvDh}!t3^%k4~3x z5TI?)O>J)c7*zVoMJyk$udlxjK%1XF$C;qAh7*8iSQ8;qU~>dp@;DIlJr1LBr!ew6 z8gKQyxJF}y>E}IB6mY-7K~};z;L5F`b{XcI;Dkn+%k5rl)gl))r=Tcuf@1?CHGu^* zl1R_+@gl+jbo;#3CDOx-5akFCbt{(M^B%}Q1H?Bk*~w)6+q((ii(=#E3>-F%n-#yN zPZRj8#uLdX;Hoz~p^f|6STM%{ORPITw^q4Sc42uhr>PpF!V^b_Ow;HHoR5Yvz*Pn3 zC#i+Uj3I0NsJ z{_6$Y4eR7!g-8>K)H)0oOdOJyF##eA3+HwS%4}#soC>;6lOBy;Y;UbQwHa#Mn}w|e zu&h9Wuc51jonrQjP+ncvC_;%d=7HAz$Qx6f;f!#)gK1*g<|i|cGj+(=aTWMnj*rqY zEF>fy2)M_>57#btqFDg~3u;^MHR(@2CIBmJsI#Mo_7Q-+pMH*b{dn#Y&N^!PkW&h7 z5#R?@0MKP+UYH`s|w&VtERHrQA7bbKFkd%<}?Y>z!D(^ zqgSA3QN6skB7Afj%*Jyc=-liRy0CR(3uoix?^r!Bb98Ao_f7A+nPLUcfjv_(rW?6i zA+%RmoZeh_$|Q#zcLZdJ{G)VD|zTSE$?r{5NKC{muu- zQGwUIekLuvbkd`ZZ!4;rX9KLZ~N5A-QNtym(rP>2~w)N-1}LXe2ZH}V>})wB?p z8}yq=!F*8s`%=)&zera$M*r=BqTXz}6e|!3BRsl|K@0>G!>5CSoxDiRI3$(SK5+VM z+m{xlRrEw2lqoPeL!^Ia2R{%@_DkMn!>pycy83d^E-Er|Xky}nf9f^8UJB+Tf<1c1 zZ{pu#+CV}Pg5){Dhe1tjAEV{!*x=0gv{gDr7lr#qKetCPyKTyB=y+;Si6|xS&d6fz z0a{#K$n(jt8+dK65vy3%H;k}9m}g(RGhq7vxNn+Xg!X;r`dFJVL?<%B53x)5O@!z7)C=xgN?PKrRuC@wOr^WNNCH?(mYvN4GZ;^$eC8lkE%Wg0z z!zFk6P)FXxYN~$)aWx zZU@~&)hQnUnJ zh|g}Q`*%F4#Q?%^OOg(YSZSAuXQprPHs~xiG~1q|i<+VF?iruOE4E!6Hetzga-t|#g05ZP_bcj>cuI$G%uXz4A zH)K22UWNzBu7f%olX1maC|fCqT~(^%P!fksi8 z4u*?7$m0tV+2Cvd+HG3zb%w`j{MdPH*Y+342n*OG2E%4cXMg(wmhBep!I-Bo9d@sO zL4h1neutyI6X?7xfbv{_bvzref4oU13$w?tC2-wfbUO+C4!QNG7Vd@+bQALd(=XRv zfTpV(;5+vaL|}!Zv9`BqZ!K%pcXFY4+l7LXV(e5$7{G?Z3-rdv&Ib1~LqSse$SGci z*>KTD)*xT=Ex9}~Qj4Oad#~%lkw3=sL8!6|rE4?Kudh4ypqs;blMltLomYw*MEQ|{ zW`Phu7HS7F*TQe|IK$*9U$}X2aPcaMxFZ>Dx#@ncuqdrnvTdgg));(ELb^9*5zVVs<`=+ucbXnKPyxBKM%gk-`z6y4?4)zu*HDV9t0< zTPLTBxxZDAD0n!h&qW9|&{g0j=oI0Oto4u-+L^S?p=HN>0G%6KT&rW476?-cJrU?; z$o!GlrH$kyq`Z0gO*Ucl4%k6*Kj7dMEN@hcz+NGW^V(y24um*X-BI8}1p}){6s32j zb3bZAXL#q1qKY3rywVWIz|g%99xK2kxHWHmC&$aL_<#dcO~{)dD`YwF00%SD7GgS- zUn!&CO5NSLl1Qy$sYqHx4!T6ZN`m!5$?5?h3}%+lO?XMGcT=+PJh_ZaRHtaWFNHLe?)NQ6gOzYkgB-K;mW4vqLQ_vx>ambX&O z2Yx7=Bo`K*7hg*M6GlOqVxU3cZoPCSB_~I|Ro{uYyfv z?TTZA(Uj-Sw$OHhi@LeHrOl{GPEw5*S}3zVJl*o`JffJ7f3Wc$72W%Od*bTOV`-)- zGeN>RrjVo2Hf>szS~JEj{LyxbmqAHn+u<9^!+uu*Qt@O%+xvOWb5=!xc3FAlx-*OY zL)C^@-+iSJ3x*Cd+x{eJev<0PM*cV6Rf@L@WR5RVm=K!%_&r2hU5PkF2~U5H^26QQ z-Omx%Jp=c{CJ~aGQV*Fk`@=qEd5^cg>*ykIXI}#qfeslY3oH64$R-98aNCCK(x2!% zX6u`p|HvdSiGEI1n5L!C`(JRqi)Ui+KV#{7_^K_-_ksF!KsE=r9!lkO8?`6>JMrUn z2gE-u-iuwreZ)0k8c3ut&O8lCCd(RbuyJT@Yk96x^18X164L&1t$2>#m1>f5PUz@P z0+JjOaIYC}*7*t-H`8=E4P}4|ZI2dGxAra%>C4EsQNkR4pRKi%l1lM8nIo>zz)c+>8+mKm1c z{Z`t-OE?^<72QCHnFQPgO;562bj4&)ag*WMV=t*Ivh{G^h^tpT-|Mni_nRGJF5ky+ zb=k;f-$3=OS|)LolSq~17)x{N&FPN&)zpK7?fbkH7nb{htThBH-=D8fq4{okDgSkV z%G)JPnEQA!+$()RUTiI1a<=3^n|j*!j`1r=x`D1(lmef?fa}9-WIaT3hnA7a6#Y+f zkR2@#QV##L)Gz{Ypz8LG;QIsI5t8MDNaYF;H}T!aM51Pp5tMIGpF|+R>#zk11Mt8f zBNrWLvF%$B*FLs0)z}a_(uRzA@95}2G!H-D)^iYt7W{eymYRle#PdAgD9YXPW9Rqy zYlw8W73asy3{2PsAg9IPLLo<5+rSuCH7FLnl*gFiSNn%78)8{vx-?8mNNX+jnPG>V zC*Hje-~xcQ$RsyNCe(wOVfE`v>zXT`y>KjnBJ#p-p_S{nO7Zi+5m^PnFu8Jk?r&=} z@Y8~0-Z&^Vh&~*|njm)?oBXG^iY~=Qg5JqnsWby_2mBjI_7^UaKmyXsuBAty8(=$! z13VCR`5jVfOf6FJfsbA3A_K?(q1*pa)U!wLNI{G38-zvmlCx537RW9DQXd|oyJC(E z0b!>+rox!^BX_LeKq;lj+oLZZLW7UOg9S(eq<#m3xG-JbTy)N#2#X8926|Scp8-VZ zwKHFc8oZ`f~{!h^!Xk1?n988|B)Y=nU~--9nyP&+Q$o2b^Gij zGY0BtsGugC$EUKU6f4(AnRW)m0j0>FgW*D3B;CUp)h7BG4J|T4qHL0ZG@{*{`}O>a zd}g7h1S{CsHXai&%VZ6<2p`;#L7o9t@c%u3x87Q^e?tZe8UKVb@op16GLNF>IRra-HWdX0cZe4x40~&Qbf)id%Z@V0?!-% zhXUVwx`7F4*1>JF9?Exb+joTn3%G?_4WJs|Xr_gHKe)I0CgAg)4ED8otyeZei~k0$ zteJ%YkaPrXF%HhZ;s?phn(!ZT>lz(yff~z+}$zCebxPn)n)88tOpoVm9`N zP1ADxB98B5nGPtFg73ci-!#SX#m+r6V?QvN-OXXUbR?i(Z6j!?ZFQ);by=C1EAH4} z9$>!KN#vI;#rMHP8x7?*JctTvoZ4Iyy_WN(N3f)7``1YO?UtH2VQOEp2d%#v7~7eU z;X%j@oC+W+Kx?1#IUhZRlCO={onlJ-BN4egyq6w zWN2+K^w9VVp&yVy!1#MtE$c1FJkb#6ldZ^vUWC~pEc+@gVCoPAgL6z6+BW1}A(4ql zi~?l5sh+Iudpd#~{uelBq+DDAzL%=Ag#bJEH!#uLK^xMbdh!u)o@;BcOp*iPEn|Ee zu-E;^DgJ*0)H!JqfXuf5O`WWLrGe?Ah9_}z!;}NEgl5CBSAUN z4b;f^X4~!Ryuu2F`?}yP0k7HXw9P-Y64Hy>hV)T^fy_!zlah1FrP$w|2NRc5-F}Fk z7r}Bf80DU_Dj0*~Pm%|o6XSKOn;5Yu{siM!pmdf}hSCRsHXbtWXBtZ?C5_}%HS(X~41 z(Zl4*d8igF^2$XwQ5-MOK%Imm3TI>hK!Tnd>V9}07Br2Uz0ME<^G?vk5sb57P*HUo zr&qF>zV(9?aEzoj|1dQ|`Zu8dTNiJUAGuJ++K1h=>%Uri2o9OVesGhB@Ra!RQp*aY z%eu%SsJ*Tjw3FULk2lQwYc3>zLh%Npqdol>LV$p`Oo3=YBK-)2>PdrP)k16?cT-OvQJ?A zh3nm?VC6?NVTh}sh0v7%!q;5$G^AG= zP*>>f)39+k(L}v(Q&{Z!;b0Rz&ARzr54U!QQ9dBDENtqGh~FX`)q7ro{Qy10txk$> z)0j7utC_|Ni{nGEGwQ}E|A+%w;TIx*Ar)r<3iVG?t}Qo)_wQBvwBVFCf82W%QhzOJ ziucge;39LLZ71a8#mlCHZ9k!t5gVc9w|tIpa<#uIu(JSA3K-hNE*BA3*%|md@29I6 zjQxfAbq!IU2Wo}aTVls41x~v_bSsJtzv-t>^gq)ukM}y;@p^tS`JWYP-!L7$EAQ=J zIW6^@_Z@?*R0IZSlcY%77u`~m76RI$Z1%)L!f4T)g@0(|)Nx{Bq7l)0W6ju%l72i2 zoiu)3@60mQ$Bq8aJABcXX}ZDe^&cX2F?tKqZs~V?H@90Zf-Cge{ypa+g3?^BpZi{n zl5NB7x@V;X?6FQCG`>*J(C}Zuma9D9rn7S-d zkU^3f^nBgKiqt{yK0twLBM#CB9?|!+*eO=+PIJcsvR15lJR&u@(&zJfWbeXun-jF4 zFYX{#9WC#1lI(rkPoX1>id$00(0+!Z)N$1wE!dW*A623A;@7QNnp5_@+Zao24@7|D zP8NZ?poI-Y23{=sorI2VbO8EZzw^C9n4hAc%|eBL-HVMD8W-is{W@FH3Y^`3En!)-)BzP_+qF_g94Sdk~+*) z1OmdMMp~5cep};}M7ve~VxL}906ME-W7e`4P)(}&C!_?WD3bH);N-hhyrY^;p!6Qi znVOZ(gs#Pu!m!M7F|2b()++*=nx|(4U+kI}MiYCOOc0b~nu7sB;XdCMl_4<^_=7-d z0r!vVS5%%~^1=CaEq*GpCEIum$ufI+4W;^7qqa}d20wh=Y{EK#upv{SNNFy7XUisN z_B5nX^Y)WuE>krp!-lmdB?pn@F*hG)}4B+;9-TqFf{WCjL3O# z$HAwtf*XjMnEGh}f)~D4;=yGC+?w*uv({n}QbIF#Hh6YsVFI7`lNom%^4Q>N5{(|y z3C66?R)-Vni}$@@;tl#(>WFr2L!_Ap5-dT~SgKDlCTuLlMx_XrDxjp1{HXRFfDV0r zB}}?3vC599pb{72jnHlTI(BdYM=T!B+}_e=E%hyMgaCce$Vnq82{re_y+Lfo?tQpI zS$!}+Yf2#{|KWnCG{6Xtq3{DEDEqZ_lvYF-29gi<8^0bi+Xuh?_C6nJZzhxaMTyN)jc zgGI_a9d^hOnPoDrO=#}x_@<8=mHf^yPRov8!$=kesoXV%c0?14Xm7^hK)@Z!^#_5M zcOuK1!qlF0BA;=qtCY1Z?F2oDc`fjhEd865)hNVqD*d@6x~Q*yD}I-(UH~rJ@(oev z8@#_t!yni~8I=+BcM34bqv_b>Xsb0Eb3n%?5||_SgEkc7LphXuG!S}csrN94QUd?+ zhW(X=TniLsUjF4(ls7Gle#B7X!}pI&!xk&w<7>4EWJROLHdDxj0J+s1MtbG5$t>jl zJP<}>nBjW4j}Bkk4`6hs@k zX{;#0c5Ya&-#3RAREir*2`oc~{$2sqKFnzQRfBDRTP3^B4s(S!m{Cw9?Wm@-)O5B9 zsc(Xcs11)6?|4YoDu}j%qN|(sHFUCVV@IO-kx}HYQ@CEQULLk<>quXozoJja6r(LK z%ADX?JvljTu^ydkVv zfvZVcKs8yav?2;PMNL!TMt$5I!~1GN%JWFadbdgInLqvQ*FYy6sYfA@ymUm;U3?`n zfT{t>4XZq2>9aSZ70>~J@!!VHj}m2OW(EbNyKX*=4*hViX6rsymvk9>2ew^R`Q1|i zFheQU&c%Ei2pcB5!a>Qgh!(OL%GX=UioSY@z)ozrfmO)8rynEY8MQrk&X0TUF+4(B%$2@_H$9Ux$+={l6?lb z-UGkY!js!q4GnPb#UNEPHfdE(cLaV#s{T8i76o-bTCi z>eZuYR;}t>rq<@ZRggb%Ai&r(ZD{O#5u7w&dC{#}XyaFuY#w2`K=Ep(kv(zz+Myt-~zQ?tC2xEoN^t?m^7iokB8+9Rfx~Rijkd;E30;yVo z5{N$={YX`q*{P#B&LbG(J9qL`=g-_-XXi0cC!_+>ff1}^Izumvt zsWCHT#$sP(vDLY8r1c<~A(lsJzrfWV=CNS&w7S5Mrcc*OtFn#Bk4B#erw<=nu+#PB z2@e;$riX~KGl!Tp8^17}>#Puq{icVDiuW=ZK-vY}vO^5KEI`~Ev)eVXm^O?N5foK} zh>P)!_+zqYE#|u=cd`c+kqItF;H!qsuo|eCtf6ceWZe1B{Tl(MY5kg)WHvSWKy*g< zyxw^%kX{H2uQYy}gadR2mCH|kkKUFr{T&^aYgkwuGJJQs%bGZJ#v5_X^XIR*>HYma zgmlLn(Ig9B7YytP^7C0|vn5?)rH-^MB|I*jZOzB-w-C$Un7>@aR*GMuV^6$@K&&b) zr9Pp*U;XxUTz7>gisC%xboU8U`}t*{LQu#+Sop;z3ggUnT}%}5$%+K?hl^b+_GN~@ zTb+-rvkNl_W}~`zBs06F&QZHb63%B<*>;pf!!OJoNG!IJc48*IcPOL;q~-(_iCd5F zHf+EugBkK38e?s9k>QQILUj)zlmO&cZXO<5ATaQQNPsS*0C5erf^CW@V#H6rgY2bP z$%Lt4Y0xbZBM$l^OQA3_`(6w0$4WlwBCHSVu`H<7jyy$Nv@Q%VItk86E!8dFJLx{` zYW+tN(U%g>yXVE3$ndvzR_Sz4c12=#HX>U%G2~lRM7WY(wbFT=BLhqg*J$eYzL7I? zKa;r}RXeFGF;?x^DocbwnJXIM82H(eOWD4fUYs9E_v;p)Tt@NS+u8FBZ(XA$xy`ld zV9=UX|G8|SDhMO!^gUnlaReh*{TBC2b}p8f|DyrVPUfov~X6%5(-rTEVzOpnB5a{OtSTiIbUk z{xS)uL=AiQM)-b#3hHA)f!zM%8VEiZ27}ro@V(^$OW5H88|f9bc_&~;g^ZVA#=6># zy55fs7y==&|KSuEU>V3rNnx1Ldtv{1Tp6@>5}^NGY$8OOLt`CU&fcI50ci0$y?f?< z2Q6}xH9|y;%r-`rR_9q=6TeU(76MPrS5*Ia%?iu`NJd;aXu&$3CpN7^Bm2@9=3PRN z)9$#jqMPeqm9z=W>|A)uM%OrRC;#W8>YPa<^~02gL-Ds?{gpTCecKT)cAGj2KD0<) z-hB^`+sK=|;L$B&A6*PGGEyH|jf$X1e03tS3Nv(Kn|uAim=)@U9w~*kyUjmS-am2< z2(a|%<;FRe)oAs~W*r=uo14q<;cHC)A-v|I{xDOnj&n0_1gD7<_qs*mR6x0LOD4|5 zP(fRdO&ax@)?!;SNjmK5BiiY|0*$ZppPf}di99TJ$TH9i-W&>#ifl9bp*!kUc@Slt zNWGh+bCa#4SWbrVUI$AUVcFDypv@dB1ODuqb8pIy&vEw?3=abIr+#hw6HoPYkt%f>8anpS{{Y z+J{>$urn;md09O?5)du9CEnfB%fZaa9R8~fWsS+ z2*M&u?z_wzj+9RU23OkNK=v$N$H~HZM+P$)eRM_)#;PRpQ52%V*NW|YAwc# z`X*!UTRqy65o~Ur*D=JOITvg$Qtzh#51rvhLiet`jw+Yik$`!AN9N=`zM{DLbu3@M zri$rpQqtt%k9N+u?_U~`ziDc%6U=Rw?w=?Q8i{-fFQv}%nk<&IzBd6oc2;(3!hy1R zK4ZBk^%M<(OfXuxehD50Z=iugV>BdMJuv-+lI3^&jN3hN^x^&c_s{A`C@CM=*le_Z zz-R-TkE)dw3#jQGAL5YN4GJYaNgNU|4qw?eY3IOjy)$#&9iPXeaG)v|DK@)qlKnbs zpCG~Hfpa(Oo$jHbk-JQx|Ky&u{IDL~9Clv}Bb@=Q7enZgOQzOm#YF^$bxluZw#R!- z7ozw^lCq1*@dx1pm8i(`s;MB&zXuJ4y@Y5A?`NvJ&L4vCPYR+(@MqpykI zAtmNJ99G-F4kKKmLO!6PBHOS^h2_GbTky#m#23z6CEtB}`-XMP6b?=l5r0KV>ke$p zJ8y*azhPsSroE3gr=rq)!QESKW^Ljh;oFW$k1;drz?AJ%s`wr?L<(`BQZP|6+^BZQ|JCv2!;+!<($9EfH4v&1$i@~p<^;#l8qO1GP6{^#p z@c7?|UJkAa<8<*mHov4&{bX{=^xD0`2(@;1`WKpg8a6+D2GS~Fj0dHfuxcdhr%zet zDvz%@rgq%pOx=d%_|0z8&lQp7i*j`+Vp|RbnN7LnE4ZKZeBPjP5;n0CYtK2oz&eh! zP#w^A&1F=)UO|ski6)c z>kxde=3>@c^<^;N^~**MzR5w7-o07}^Z2XN%O0cRuhCz=aK1Rpz6*Jo-(hXm1Av8y zvN9g<;a~WW4|R3@G3k^g6z_s3aRwzi!wwmajzklK9xlLp4k)rvbP155Jgxqv7xp?j z!jP;}4};kL7YldqB3`e5MdoVCp8TCZLncTf(63u%u^z*J*XEF2!3GwwwYO7Gb#y{P zLc)Pjm95`-jrsFEpW^BHdF0QZk6)yJ`m}VwQA*0r&OY##Oh@Nij9-`-qkbsS(y!q! zt7m#LSPaL7g+7)OxD0Yu$vWF2j$KTZEtOvtELTaxFY`2GTRtjQ>Q`ek$az0)DW4Ya z9(Q9nrt2jd?|<-5E7meXc0-tU`b-8Z_hs5tdJ6Y;e`Q`D$LGxH0o6Z$$ZN`FNBJud z>TjSH9`t;fa5LH#O&aca9`-3>{-dJUMCbFmsDBy06)p&~*dtJel05U0=bCsn&L5wW zwexeX?6U*W`-adZ$LID@hsjC0k?3K3@^+ao-uM}7d}OmvkmLrX!eP&PJhmwUAQ*@MGh#GLBiYrd|jEe+Ld z&wWcBJGj(R?W&<=;PJT8`5;Qp$}O6$?8~wBB)^g6>g~v%hH7`YMwb%o)cxI3=GFm>n{pk}e0tSbr zTOdSKMloM(@f&b=8tEYk`{o6gsbHQEi=AgO>s;d=W6jHieJZ81YaWDB##35^1|d z{VCbtTg2@&N;+B|w7K|&5P9#v{C(OGfGau{O^)k!@$UAa(-#`X)&4BT^P??R5)u;R z2ghJ4;6X%mG#OxE$y-`lmfCKqVcG33-S(p%l;Dv2xJ4`Dw?@Oz+ZzIav=D2@`7N8) zOUCM&SE0w9gfg=YqZW8}zh!cZ3bFmJymL!T9I8w77^;O`^fS~9n)TQ32w-*d4Pu?# zkvqzd4H0qqpqO4W1K8}C(jz9fseNlO&tvrCq@FQGd)a687p zo;YvXgODNQ{@;;8TD#5dX_?Fi@>uDr$jo$ea1fYx$Iu`mw|~Pp14jx{aH(6XB7R`2 z>*%nM$`ebzXyyLkuyywf!Qk!~i<@GX%IsIH^wo2Td|Xvu=RL4{`bZ@ST1M2JWNbP8 z{PeR(F3YX1YImg5LjKXb*>ZDf1mTGZgTU(G?M7t@zGFH{j^54K$ZY55;g{c+LuRM_ zf4=8lrLNb1OZ8CM)5j<3tH|K)Q!p$ONlQ!T<>W{e6cmJq;*w_-j3Y0}WffL^w{zJd zi@&&vyTDSgsH@zp^ZMVr7nyD;^-GZ)`fL(0U@*p`;$MHK9CsJ0nIB9>x#Kf3IA6Xp zxOa9<`4|)B)~&<hf%; zBJ;|fzd7IL)RUU6wG##gfs%Nd>dAne_~!5DYvi|Rh!qYm>#P4!>$80KvYZf{+WtdI z>Y}eW_;YiqQrXwfPwZv=+U=BITgy8?eeLrKnn+Ac9Gd30?9nKNF}vhKZ~aNhg|ZTy znkr-H8p0<^_xABoHPdvpLvmoM!Bbax9sKbLk%VQ73_IwUv{fC<6phW;bK8Tu6heJ0 zpVIaAA{4XeA|YDBehBKy>JqHEc)dn)@VUKxou zgcepi?0gk?SaUR{w@HN5%&t&>Bo90A$j^F_ z=lSBMj4tM~cN!*G2?PLg&sf|0QI{&AxS`WZ;MD#i&Riy-`KvOND3^ z`h-UoERmgaJ%-yiv{NkV&^_YAP5b^b#(4~XEhrzS5LAO^Q7!T+>A@>+S?IltZ|2csiMwTRA{s_JTL1B}#mIXo3(kH#b9I)IwtOc$m#@_r(~ET3lLMWL=#g(Bb&J zuSXW^BN38-LxHKW8CXaSmN$;? zTOaYOR1+zUw_kizWhJg$Cq=fWg2q1JPuYg2QT>sAJrrixvvz33XJ^;D`y@n4cKp#7gRtU{(o;J<_V?t^m%fE}hH3n58nsfC zMc+(fSK8m#`=E**g{h$zem>{IhLkzC_=JyX|NFw3S{d{yulek*rXh82v|w zfc9@g=Gn)|3dSo`?PgT$_gkVrSz6r%Z*sI)Jzg8sbG61pW8y<{W$viV!tHDI72$C__z3N~I_>f;EBpzGV;5!QtT} z6_tF@iEl5q(mhI`;I}4v_Uu{4eMjCapUm&4VRYZ0^D$4T^nKo|WaFZKyD@#%B_Hb3 z(>S|B@W2IkV@hCp*~}Y0_HfAWlKaZ6{JEqCYK>A%>~a13nu&3Xl9CfFQWR(n8QT^o z*(nLtHH=<=`q-tQKt%D&Iqv1xf!}WpNtW~X_T7cO;I`UN4Ia6>{1o8t{agfR;wcSn z%^1!;jGH%p?K}JxGFrK*o&2RqhOtvtEaTtT z(nJ&ONX|ayE(Nb7dZ^&^t;I>TKla(8U)fd7{@~#()Jb@CrOfe(NJ`tY_;3nZ+oopD zE(Px!fq@KzJQztyy2Q6A-?U~EPJH!N?!8B)!bu@$_UV&Z$!As}wz#=q66As2Jt;}Y zyrd-GNExkimf8h3(k78)x4Puhhm=og`Uy+<@hlN9U(wXK;dA+ux3RgH>PFwx9VX7D zy?Ee^BZWmWKT!?y6YunVdF%){KIAcV^suwt9h!`i@bj*6nY2INk!+iFX#RGpNQ=y; zMq(bgYkJJyYKQU|JC&Z}xqx|`XI z!fH>U}buIQG{ftqse4&nDW8;hnE@}%DY(D^S#qy2HY_|jZS5Rn+E&CZ@$-izVaF2&df%h@lyS6m^ZT=+ z#XNbZcH+!^pF#|#&1@~#om5UUQ^Ykg(j{Ph=6W7O;Kq}8s3<_tgdwPs64uql$%{)C z72@RhZFZyY6M0?(62_H0b?!z%Asv(NZ%u1`#d~&;_hL*b0nO)7b%MFn4k2OkA+|`m z1nt+`cLt8@ZkPI}sFjwv9{-}5F^^y(kh+&}RbP8`p-1!0y5swx7vK6Vzgs&pgU?N- zahRC}$XeFKT0=ri?0iUhmM^BW#C^n8=^v4RPlRt!{DfNE=h$+0q1hKwa-eVherW`6 z>wT#NW(mZli^S!g?D6g*p_HQ|FLDF`A!NIBR9iq0T>|^`DiBc)9QPpF*cn(4m3|lQ zOUugvFEF4;BdB0NkIP*jh-5afsH6-*ZBG=ar9?3`H$RAd4km~>P!kV?-EQ>Q{ZL%o zQ+4~(52uKjnCF0_hPcHTz?UL_WZu)_M5A1;=LcUOSU|PU(Vz#v{y< z#3iu-T|G?#CAmgSBE45z>>OoxIhm)Ju%xkiezG%HzPD(E@04~D{;|)k*!t-u^}x@3 zi&^H_Fht>s&e+84-r?as7H7{7p1b_aJxnTwa`sI^S|WDNu#D!w`eF=<16svk`0vf9O-}Cw-XWQ ztf(t+)6CYdntKSrgp`Ga;rqs0dq%#xIGD!9w#C4Cp7{}@z`c&nePignV^a&|iF%;g z>$7x;pi{aUu`)FV9+%uQD3;eAAJf#(kxl>piX-rChOf3N3v}i-*&l2QUaZ|B{k7Qf z%t+lZ1_QeP<(1n>8jGe((CAJ?luBVVc1k~glG|NVQu-0vVKsBxq}dAh$i92}7QVObovRbxTUsQFgX_EG+kH<>^B|Y(bT0Wbncm$D$W!rDMQ| za2g&s92cBoXX7qZtz6sHfj-^8rG@%OkNN7`RH2<}`#I-X8nKg0TtmleFH5VnN;uRZ zGgFbBb3in7&JYx!Rzt~(Jctv^4AnW9Sog@P`Q0u&{a!fc}vgS76V zB%d$$C84rfs+Q8&w&JWCopb6BC5<%!0{z(9BZY$UYqoh4SsX`k*WuB-Od)UP%z~@z zAhp%L_a+W^cHQYHm%erwiVBcBCpq4c%?&${^#BcY@s3vkxHqY9){iB+;p7t6HH;(!dnShGOzYJjN@s< z^Xt29d9s^bScv!UWbNXYX$K>duCaj2mkW{?JzfpVV9&Vi^sx0$vvMKV z%|Lr%UE%^utDNBM7-!8tuW4Q_`B1FAw+9UUvczcs-b z%CphRGd)8HTs>BODO_?23J*-%P!=+TU-w2jo12@zdtikl>b}iP&-?C9>791SjsEdT z=X3e&PLj^k{FiBpcO@uduIr0o$zn8_U+{&bQIueN}OfyZUJ00;f(jPGLH)s z1LYEL%^BU#wNX|1Kce0Pp6dSnAGax)DcQS_QTE=u$dNrOWbeIa$W}_(o5;!vAt^+5 z9NDtT-t+vgbAP_S$NxNTckbNY4&LW_UDtD6-JCXWD3;wnE#IWbTTdjtzUUrGv%!XQ zd_u>+@_o~p^!km>oQDHXs$aj+Q%KAAw$cplonAP9QdjOL9SfKmsSp4B#htCA_0i4I=LH*7>N@=g!C$cCKZ)_&{>EaR^SK7gWZ34- zt@ZPC@i%y>ZxV~k?6M0BBfOrswl7ke1rK<%oE-WAq}7qw7WUcqdo5_VQdsY>)rz$K zS9L?Wx>TU*?(FP{1};B-2x1frNO6*za=ELvC8|PZC3>`e(gVdzuAZenpsTC0qF+&b z_o|kR-NJX8L@a8$bVJGUCA&m%i3o}@O#2_!ew}&gm&D`o6Np9_k=N-T^`Fs}95Jo! zB~C~yJs$m%c`BgwO)%=)8_i}0#Or9+Q*rvv&^dR?1JSPgDa^Q@0@Ukn19elulQ!h8 z_WY3zx{;^mUh*8-_w{AOkFj2{-8v^v`gYpbxY*7h*sTcP>#0}SmNymsqPyO=;rpp^>1E~;cPuX!l zCuvH(aV)d2_uSz9d3jSVF(su_9Hs^q^UlA+6Vhn;8bCtIFb zN(i#k1ZnDc4;@9RGyA+LTg8CI8_9lKindeZmJ{Dzs5nf>$pj;KX49NBm11Smg!{Y3 zyi%$uupCZY7xwy<|IH*!RxoI3k&h1*bcNyPp^jO)%j&5E?s5*ao?bdNpkQvf}HvozJd6+J=U4sL{iYkDnhk`QkFoTC2~FC&clYE1&mD zPUJBxA{uKbNj+2kl`F&>#o%VTyW(a#q#!WI_^pC&#zIOJF@<@~#blf)@RC}YsYQ)| zhHyIP+gl=;qDn$ET{dG$7- zCveH=ZR3xsEK!ai?D+vp)B1o&edy@O1FMam@2?D^H#28{o zkBTBdD=7Gg85D*EmPP*fC_;nmYf&paX0NSnM3a%^NId$X=zcoZ z`GvnBjt0dWUB*NejVPDCiVsQwd$SQ=zG>Ne;$Nvk^zmY)7QTLCq~I!7qdXe$+%e~= ze6ZEV@ou-R_Q-IKncj>HLREAwC$k%8bllh0c3mQUNsa(r0tv4iY6b(1)sF1!ocFf= zCB@6hFNGf`%rhcb3+S{;RNQ{o0xxG71}jufi4Jy7ZZEaDCVYjlf5h?67YBLR+C<~0 zo>~mm+hweLqU!o}mz6`RWA&C(sKeRIhV6N z0A!wlxn^8ZiTvY}V4g>9((|m)~ow?=pnE#6In?ekXm;%e}5$U+XC&<@ zMUBsWZn?JHYY~K+00g^FuYc%gJh{WdO4TruQSS3E=-NQiUu8yIkNc-fzm$_d%C3sx zo*T;s=ia`-y`ji;q5Bz&!k#V8&gG_Ssd)PQnzEVl?gOP)Ts&!*7hR=MpK949XT1L0 z)vhr)VqB|kIG9$&_-N8}=vBUuB~K92`#dkVB%08v<&Sw5Y~|{zpI&pnTnAYJJEt79 zOWz0cy0Gz?0n$Ys=q=nWmq`<+TwF?P>sOz9dSJWofJYWJi_MZK0}ZMV{$jj%dc4P1 zXV3okVW2YrM0O#DS7@AudSE($@`Kf*eA8y*I&Rb0$vk{*J(i=b1lW7jeQ|__16%Gp79zX5mE6Y={SF+$+gc*)BtY&{Q0x(h#?R~@!MN{s+ie&HUqX#Ea>!Jyl^4A#(hTf_-4dmqOd@RiNWO&6oEa{44!cw8|mh0nB zpO>6|^cbok0U~#IWLs1i9{I@MZq9X_eey$itcDuHS8+yWB9UofWb>KT4zo-mtvjBu z`*{>b1wmkU!27JA%lFC26knQ~6KEg-Y5U30@UTIhJryiHw}1to|E8K!fFa(4g#XOR zAN&BQA$Kp&w)XzU@M=NU!&d+al=by*fmm#5i(c5gO;}j4K!>jS`%CaP?~?@Pe@oV*0&((G4K=Fqdx;gi9Hk0x7jMS*#-?NC(yW&X=h#x)tcun= z(AX|M$TTyK$nDB!^&XAp<|&myzEEEJOd5I-$xl>^DSXH2sj4VuU<|18wIS$i(ros;^%bp4DWo z*55SIy}Kp0#dV|f@|fS8?eBCb);eFJIYqZ7-M%Uv9hTDjG-jSL$-pkEmwvxRvKS0V zhQI%pbPcbg19mPBWnGKE&x~#HKt^$V>g|V<0r16N%PLJnQ$_duo)f2P@vbiJdDF%pI-UKA+lS`B}eBe)vVBO#@y4rm0RIo6AHnNM&sK+sjk{vjztPeDQD zUdUzO!s6l%=m?!hboA~!54=yld1D7cvt1zKiO+uQo)n~P-sCp?Ra!k==yg=it)}!2 z9_L7FPRwmR|0=0Jpzy-Dw-)ovvsZ$o+faaW~J*Zzlf53E>E`q}C>0_rF zx_YYj=Nt4ww6iw6x$b^cl0l-b`!6DxWv&yQtgLR`+C81J5=}dpR8%pQ%|6byY9VQH z>7u(69Arn5rnu;-a3HlECyaIdYuDzCH;3!_NLN`J?hc5>X1o{bNSIIR`TE|7UE%O& z%;h007_Kfa|1ra25Sl1=#w;whsyJ=6f_pdVZ-wktue&DZyz&zs!T@5lRooXDK)kHm zz+9I&J!K2y=@xsY{b{C+8a#5?6m`q8Mf}=Z(f|o(R9U`zo96N{X=t*L;6ySA7 zBkZW~JD+|z`tp7_I@J5st5?<&Re7ZWoy7UNWr^Ru#r``Hbt64Ffu=T2zv<>;^j%Wr zgxm_l7p^_r5n5l>g?&V(f5g*i8pT{5AlVb7u1-%E^EQXO?)9p?EPtYAFI4lMziQHh z68g-p_6Vwt)CeIj9&76ET4dwszt@iv&-P`dy-9t>?$xk8(A;e5?>cW?O1;?^RAon* zoILR!gF*%f~zvs#F7=*_`o5%?xsT<7NEFP($j*D z&u?pM%gBczc4d$Vt6sUa3CfdS{6-&Zfx0Ip4VJJVV3wZIZ-VOjZ}JKzX1Nxor}$b7 zFLRd9ZCCT#F8!I35Xs_{PJ9@|S%?K7!fh_DYiYd^X8#=i@uIp#r)|jER~X!A{JP&JnN*MbkrKJY1Z<%T*e{g_59VP% zt|Zez*1LDNbSq53a%|cKNU5rZ#&bx*`Eq%>&U)t#ztgnD?(VKeH+3r9*Wy$_cR`M| zrT=_$WI_T-3?=V#bl!W}s5yN#z-XC)r#9&Hg9N__7UlSD(64=h#TN6KGoam(?ghcv z3bKL$DO?g`z;)BqZ$oezR$0x{o-Cg%7fGRCH;>T3gHk-Ls#-LYQwxB?p8v+$Am<$2 z%t~TmW4{O3;Bn!@r|`3sIQv)q1%9BA3c?RZ1AWlBOGi9i4-n(PCZrUq!!UJ>Y+A*Qu>DD3^1x{!Mdj9i;EaNHXmwci0B2og7TCp+>sf zzy3Fvsi&`&-D*XJ1289G8J=b2MzKj|+G=-%4g5_A`1WG6@sEPB)rUn>-#jX+Q>A}p z4q}Y8^^F=eS6s8=5+zjwwhbe{%tuJvNIqAa8L`1t1*C_pXCPTs(o>v96Q{H(jX(qy zU0PWZS|Gtult+IP_^A{QvSM>5!0&{wUozi z7zyz?j0+IcOQ^sJ>NVv-={4;^`P$4o1cFiAlv)G1F6~9J@zZ5>JRArQaId4u0#2}s z!vfih`D%}}M9cS{8^*OOgKL9c({U6Y^lJ4fCcS#;qx+$IJaND)-GC zzdQ=KTF_|}^(srIR5@AlRvB@3!jSG<(1|KJIYV&F=~B(l0VBcYkt1<1!830qjrq#KMe0knmhKy z9N!_6V+@}Kj>V&PaWSXDBxZLu3*BY(_&55{r%+%Ly)~i zYvkuQie24us71&yKwQg5X2pcoRUb(cnEgS zCx|!MJ$TCqW$hHEb+5^gx+j?3@9snNy_3jpp2U_Q6w?)>y6jl`I3W+t?!A7N!wr5l z5&=$kl?-v3HoUzhCK85t5%(^(otc4PAt(CdV@9(%U9E4nil`H-{CrmPm>7AG#d1hY z65v@!T+Sz~h>Ggd2Y^NjP+Vx0_C ztFO%M_3c-twQtpoi1!{>EKhIRSzqmL%pcJ{d{eft*M8&-DKE$DAu6wa={P)prMp9C z6tgrpK(j(7ilxXGxHlf2uK3RBY7BAQT&m|5`Jxq0-Qx1{>-XTT=caPR$25VK?+Fbdx!-p9ccGUOG-SlHI%+k%E-IsQB8NP&oUl5EZE& zeZBSQQsMMG>F7>%Mbk5->i`X`G{%!+sSlINWv@57o#k!xo_%hWniiw8GaIhDGj!Dd zPyGFSQ;=zDBI}1AOVfcLrJhU%G1#@9odL}O#h1vj^}=)6ZGC0M8p6Lj26E&jPbW4! zO#@FX(Vzs-eC_ZyPrD4>jU2S*VZ1iRq{)+Rcxk5k;OJDiSN^SK0y3y(BjI1QK3Au< zq4?XW-k?nD35E@nUe;d1CjSG8bElkjF<6Z8(X2HSx?jqssXkE6b#-^=wzOoz#Rd!2 z)8R_;H{s9{IT51N>Uvi zgw7kcKFO4q01}}CAvQL8A%d^HYrUc<|MS=h+X8HAdp1`2RN*`8J@;)4d!e+w!pia% zOcR2HUv5k|kweusP}!N(4FDN(?dt%{RY`en zZeg)JzdSFWZZr_agL5`Cv^ri@t30bvTl*H8`0-j-(bBl=NLsPFgKG~-%=#*#Fd>t#gR_JDH@OqGKTA` z$)otiAFH12Xi0=V5HPt@pB>(}@z}a=@Spz|uA8ssP2w-%5acYeUHRQNVn!gVYJ>ZV zHHnmF^;e^?R8z$6i>7fIIK~7=4aZrVM$22;lt3J4GVT{2LyB`;@2*7G*HsWuio^Qz zz-QDW<99n1oOTM}WJnWo_zPz9MfiPwopT08d8*OF9K@jLZW|^h{=jo@e}8|pJ*f43 zm$3FgC|a#&P;koqG7a4&kkQj?1O!N4US4)?`9~ute!Q=>V;u0DAqi2T3B}`RjHSA&2;ctpl{!u zsFDo!W+}dD246DKry{4*JM0&eteP~tai6d(X{yqzdu}y8qhYp^@>XW{dN+H&ym3w< zncj)}(1Vzg{=mHki+E2_3qvzjSY4UvjnmC*A zQr7>7Zs?63^{dNAk}q0pZw&ph@)JP~dhGMIV5olE9gtTVh5~NV$e7`odiHbBG>C37 zCsGq)W_oID25xD8aQYJp#aGE_xN^onY^a_Wz}YV0Y3pRn&&SUcY&ijiXyCYo5fy!P z->FrRO8ln7C|5G2DbQ;!vlzSEb4uG^dxdE{{ZBt2)SmQGCr>!K{W-ckn_YY{*(9uG z-XP*WrOEdhJhO)zD$D6%htqyhr@ma~opr|Zl70iNDEgiUyi{#B?&{@AYzN=Jq;dL; zMe9ztadAXOjQ=)eaxK-4I_xT;5BACqP3;d9hrk6l&|yVmu=m0W>^|O+!F2lcgvHBuARr?vMuQ!m#JLVJ49c=Qz?$gKL<3^W0LR0 z#Zh0E;wBovWu^iRG>FlId^6Fm2Uuun1@^ORv?7VXzM*Swqt{m7>qk;QmwMBZSYO8Q zG)BQIAp=wk%sqV3xi!wOFzGdVZeNxoOG>nijADA9?~PGYQ(MDh(DUOFY?af|aS@@V zuK;Uk5=v&TQY?_+a% zE-*7Gt=Brz8NhXPiOM3fzTa0hu5ozWo6!?waaNWcGDGNYJ)W9rN6+hw4%B%H5;|k( z!@55GLQ{134E{0}7ObF+?yCZQUKqGO9L8&&qLuemLJs5Mz0X;Ac;b80gtIgU+8L$- zdG9S))C*o{pl)_v4L#xC2twWuLY@Q!1RyAILF$^z-dD!)P7ctHinJr?BxEN|wawn{ zuIr`4?&0B~_so8y55rMbYi)lg;Ick@FV7_}tAjdG-ShDyqvuEJ!vequ_?&IjpP<8` zKYYl6ya{v`CYV7YaS3m3gP__CGLPzRomI$?w;ufz0C9>aF60hofT*w<)(in|58@SL z-_?Trq9W8?+qz3i_`&w^27v(l$zt~#`-!?<17$q~MfkmZwX`4`yFh-X9==bx-oMM8 z8V%f2`0(5Ba}Vxjm6oq_pI+)V@r3WNW$!ezQXxL17S<+W+?2~EoXdH3&w7CIW9PCk#ewJt#$DQK!(4lC~+;8%UtS|X#ER_V+(L#!aaD59^W}pC|4I-xP&UZW0 zrT;UPTIjZIEtE0A=J%`MV{Y^*UQ=YdE)w^Q*m9su{%%4Ge7ty2b&Dq^OHcoqg!9Y2 zkH{vJ^K-(H@bN~x-Dou$x`FchLql#LhTe5<#HJN_rP?`~X%3z^;nDIViJ7*%Kdha~ zezBm;!jHEYDhRlnaNt#3lCw4-^ds-ux1?VZV2YEVy*MBeawSwVWWSzYusk5f(f2`Z z;5!_7{$H9^;}bmls-XNUX~;t3pML&KK>dy=Q%8F|I9SlfEFwOq91aKBD;tJvm9QNe z`6a)%+? zRFhX8`_;zs`NhQ_z@YS@SA*0$P7x3gI0q_$f8^ygk{yCi|4Hl2vKt#y2L?2mSy&*& zS88PGoJWoTc<1BY9^O@$L4DVTv&9pe8fg>9=crU#sx6bSg2TcZSVvc+38*{5NV~%Dd}2Q|?-7sLvmZ@aS?p2X$U(d7h98AZG-ng``k2-zEH@$NN1;o|G zx*YQDpLdjOFjV)e6F90Xz{-PcLkArCoBB<42gov{c)CrY?u#pbfQ2!Nb{iI=5Zk-tj*C1Cu9BC zjFMUr>*lT(L1M|kgwxH_@@_0g{=hxngC;jnmH zPEPI^lGHN_Xwvd*sKH;BtV=Ola2%{DL7zy76a7)7Rh}~%d=!+FChlag?nel=-7+;a zG@L_6Lo4B(2-iz|{W`9kkU|l?{MHNabACZVlHR))n~8%Sbkfq&!0demU7WAhQfUb% z?5_C&3*8_IG6g(y&<(9jH3_e-u3A`FVAOaY4laZ{Y);yp`g{b%?UB{l-RRJ)ECwkl zDQ+RSQc>8_QQhq#4YxZ9XX}MWy+N1LRqE0g=f-Ter?O;e*AI3MQG@q24#!%L#ur*c z%GZsZ+i>^SkqI1+3n4*lqyDj(S$`-lAqL9_Ti7fIU(-98=>)d;dc)-Mr`n3->C>m3 zw5ZnaPPpuJTNlzfTV_7fg%}(rNXg8-hE}KXz^#D2fZmBm7QYHH%FS^-1aWNzNhVLr zvMbG-ovf{$`_B5$15njL6R7Bx&RxW3{BHme!Z#U}kf8k5l8=v%R1T&Qe5s55$71d) z+x49I@Bhblvcoz59)jpMfoDYW@LGV-+6@Iys~*K-c! z=;)>6g1N5lO*S?*jE^&wWWlLxoS&+zF~@4{Qk0*gb_lLrQ$XT=;+osoP(Rx2XkP~% zNh7S)NpKNqlclBHaFgobf5f!O&l)2F*%%Bwn@G zmZ_DHI+$g^8Pk^*;xJK%sp(WbzpUHSlARrVetv#)59ELHZ*RzWPio@e;pN(<4nHkr zzY0C1Q@xu;Lv?SD187IT8*SnCzJ^~7kwO-)k+dFO2Di_?rb ziK4>Qg&S`N@{Fv;Yg|4sSuEcmqb6TL^wY;aJpcaLI^o)Jx9!~CUKBvnvQP@~5>V4bug&cSf}Vwd;75{K{Ief60Lx9`E4 zBJ>bfu3cjVtrApi{wLkw_RFV&mjI9@w3gK|m99eXrIJnHd{8_H zru9i!Xy34BWO|ByJNR`ZdqrV^TWPia8r0EbsVv6uh?XEMwIe-QPuy9MM3V`A=yKRp zzPd#cT->iDz0IoDFKV3@8R8|PbxyGG`L5wu@y6#H9BUG+TleyBvEZAE-y`F!%icG_ zy!oi8*qU8tLV~s@+o_E_HuHe#l^(0_s_lZ%d?JW%O`i0R`6)l5+cn#~&bb z8B#<2$4luKiE1QC80%Z`9Z>W7r`#`;e%c_8iQsF1P3Bkfs?)2tZ#zKMb>7Ldy_J}e zp{}NORk!rU%+ZXp?G&}kKKrh^aL7pXSQYlkS}EETt)`&_C{HpXkCcikYTtdKhxh2# zBR^CC)XBFl^NjaM5!&H^*1UbsFPVX22#TGo=RF-QV^1+7RwpSCBs!?-)Kp4P61#9W zCp!EmYhfiXhb2O*5*P?_J3#XX0P6>c_2%Btv@hWzDD+ZzmDi?dIPQ~eET$kbeel2p zo$NB7$jm}!RL}gWdT@9sRtq=oNZ|C5G?gu;q9S{qvN+ut8p&VIj1h9X-EE$L#<0ff zh%C>&MU07PJYrn5>{LyS2k~$wqykL+RlV_6+SfW~Z zz`OY53FF1-0j%C#>Nih^s3;dw9>5pN(!hU2^AHU`_^>E*qHu!z0;#2jf}3| zUN&y-IA}kC1*H=AtgeF8wGa;7-ZMkKi8+KR@>;W{~Sl|dP+&knvvNesS&R9BgSEBHut0*njn@qkq$-HZp&6py8j zfBEtyy$!eb2&(K=dguOUlA~wlwG5^GRpU@T4MjX3J`79%T{gPwB&HKnFgB+3@bG}= zu?@79=<85)MNN1(153~v-B-VX_EiLc^hX&xDDu?IYC+Mwsf_VXTelgnkI3GMz^%_S zXFgx*<~Qp9*mGwx24+`l9Ch@6woG(3J6i2g@scV@atrcmh6Y+|OVt)QdWuzV6vyA!2^1HUSTVGj%8D}49T_oIIhnh8fr-P%2P z#Y#5yvBM^;H$Elh2TQ35MoY^jy@qq}EBJOiBb6ong3m*-zDzoX!(a~c7=1K8OmR9o z9v>~GR&nt0=ckg1ZY!AMMLbpZ_7?fXWnL`~J|TJ%nsCyw4PyVv(ef`}Xw`=e*?EJ> zqBIBp8D^-#{h0t%%gvY*3P6qU+%z^ z3u)5>FJ_&U90?0B0U^Ksx-}l%yU5F5At5O$E?&`2b2A35z0X=3*`$XjlNuPD9M5Nr z+(M~k-{Pm?vGnzP}DBf3N$uDhjumDG}EU?@{rqmfE< z6))jFX!DqdW|TI!Z+X)kdFLME*KFiz;#J6BwFd@%QeH_3Z-y&?CP@&#p=UwbyTE7{ zP4xnw_xPR*5=gsYWJHD1){Q>Ka)=L6pZjaF$(2V{AdeD3hkdNRp#l4`wPc zFCrN#C5{=x!x`cKzWu0UxBKpFAzprkKBkLH)Ia-$z>9PZ=X;<27V!5g(DQxQ56e>~ zo2X-gE^)j>r%TWD<>e1bwCcI#8E*GRy?Dl9-+fR;`|MkPA1x!Lj-D(^Kw!`4A86JYNH@22h^ZYe{$v%!V%%(WIJMl@%tQk3x=2Q!xTW!F#H zkA^RxhbN^_aI0C)QWit`=STggo7OWUdKKRFOX1&iod1s-8ai0Hxj@Y(@_rc*8Jm;I zxR2cs;I}Z>?sL?91i|~92Fmf-&3o|(xmkwjAI~7(A@E)X(d|1K0Bwwx8G)5q9L$Ax zpu~X?0tnxg$%eiA)sxV&6_=J`!!G1s=2@++tveY;2!bnDaz1{Hg?bC_gM0T90gwi< zZ-_md`JlA;@zi07%A5~nC+n#G7HM+h9V2O!iqY`N z)bg=kMbIO$GgN8nHcE6;za%>de{=UpjzGw1`XP92pi5CEXamhE0WopM^J;{io^pk0 zvqbtwsl)y;)Fr`}H*4_zLOPcH6G znW09P%Ex!D>I?VIM@4!k0#b252))@lZk{cfu|2B}NF7g0p>_$H`m&(1C#CM0cFuqQ zaXZNWF7HEaLR%b*_84Yi>Gj`KAZIdACERL{PCkqQp#zs|DRo}Ip z!e57A`_5&NGm1s%%?!vDwtV@NI*`anV&H$zVkXV%l0h_;SmpSnF@1zMvU_fyD->Je z>}qz)bVNe^3pf?5{B%vpR9z6vUcbww^8}{+XS>M1 zm$QGfG={?tG9uRz9nGt*m%h;#H*7gu=d<^TA zDViQ!%3JR@b?<^oEJC5BJ>+sw8JxpikXBw2eGFnY8mIkR$`;RaEA&GaKNK;t;?NT3 zrGITF@e@NKDLPD|j%Zy})V?UnX zUJiQ7blMR&Fz^HiS3nenEPoMqxYapOBUe#;%TD2y0c~g|F7%c|?)Ae8X-x zYReM!VCUfAqTOKh_U${N&<&!OmciTQ6^8k(tu5^eH&jayS8i@@lX+KC2R#=bbctW)l1j=_uSjb}iFep%I{24t zLs1iZRMEJ`Y3el@FG1ZrJl@;^E5&ewn-w}Dg2Zae2{-1P z&Y^X2?1I~BtIA$#`p_tTdB#-%7>{&`|LcaXNnh+=2g19?(Lp6%Pe*6x2c_9I?C!{X z*7NX^Gq5Lwj-J~8m*VdDS2befiTY;!M6G2wTIT>$X*+Ze-n_P<}Qsdr|Y+zdI#| zjGSJfUbngeS48&^jSul2U;v$-G!bXFDj>a$qh&p#gzsa1CHvwLV|@?~A}u8I^5$>L zXH<R{-q39QX4kpRdNCb_Vj|wNI5=A zqanDW=B4Kovitu|RPIwU-kG^($PGP+N|F^5t|zQ#*jzKp3ah(#&J`BuT(|mui|Isx zY3);@_(OhbIvwjX21U^e?wSpA&C_4_sN5*e}B3t z&0>p-?^*2AtWWzrsm*QPz54dUXF|Z z#D;f%H;b>eljMEb4lQ*L8782+$V>R@s7xc&8Hp9bz&&J z%lXO-26}@&Aq2#n0YUc+Lw}g`Mr*EnAg2F3Iy@rXoki=3GBUXC*?#kU8!v9b**esg zgNp=~`UVEoL)iyu*W%R{hG#IH`hPt{WW|^sv%PZSa&3x~JyiN_l?O!>Si}5a&)(;Z za5>Bnx}@t$jB2AY8&C6(O9%H-d__tn zvT~l&pT9L}vrmAWsQD2?v@D$sps$uS`))9%iaMv3hle$2J3!IwKWdPe_$1ayD|D=d zZt;9C&g1jB>+eWo%FeL+Wm%|MOjSe>b_2C`X5NMcicWu z%rDHH^kz9ezSW01*=y^>Uu?tA=nLxZ5v>W5PJ$Il2ODQrJbN?IAJCTI|Bm~6AOS-w z7LIr38^HYF1ODIbgM&q22=LMFUt|BBAO7H4E8IgX(VP}%_7>XzlcYf~MriqZy3f-4 zU_}2pV6V?W&Q>%ASrPwxyhdT6gqB;yeJ}a;yE6dlU$)gQv!3W=tfz~~Y`ylBS8??o zrhhIOn>5SGcV@+FyZ>631GnRTfGfY+K>Qx&hY>oJz$@0Efu0JX6{HYddiID#`>yY} zc7Eb;TPu;4vv6ff?&Sa#%s4<5)KnT#RLJrg#0{<5D2!+*{J6QGTc5{}v6%A=Bio_% z3W(w76=bp*e26%T33KhSUDul?@pK03roz)*3ANtMdEd}!BD;}BR1P$xIJMIha%kwhbiIdxnlkYc)n(XKzpj=EF(8^NU zeW|p0oV@J8q5raY-0d0V4#~j6F@E;vTby^ngzqeds(mDl=6g4$ybz8pe-e+JazBwD zBq?wE9+`j(ZS2#J#GkE)hDg4*$=SScJcHe4IwGvu#_aj+@usb3K6AM3pv=ty-3v(7 zSEMhaB&`9f<3vyz(?MW5M;J zvXP3PEr7@O(fFn1q(}`FzvgB`=ez6KPLoj-*RfSqqhKamU4^z~fs7^gNkl4j%?rY| z8HSU>G~+N`ZRgpY&JUG_svo(NXt6H{5^&k0zk#F@?YBU0na)#+`wQgb zaxKy{ucjs$LPz5v+-m5^jPxGbkva2s=4AiB+wVpB`oj#dH$|DQlA3-1WUmYtkpRhs z%3wiThW_`^hy|O*(P=K~D(&Km4eyiH`rjkBwSRk9nXfk8y7>Aydla0+B$s8RZ^TI# zX%fkm8xN2Jo|9P73~K!(snlq1@0|qMtJaZ?>yC+ElFnZzP4Vwx4j}5|bT1Jr6iq$aX4Y3p1?g2lW?~J!7lX)0KNVNWq z%sgQ_?qG^;ap2;gm7DM&(ee&jIG*bm6#TG4wZ&{-z&>Rq_TVf%lnjvc%Y+k& zTne6tLFvx`O;nQSHsg4%B0z+8B4@Bm45sd$yaa03%RFWI&!xW;i$5yBsFZx=x^dXZ zXl`%T)~_1;So=##Q<;_p^2$Gn{o?(#6@_>Q4a=Gx3BsQiJm{dOP1+$V(W9=E|2g_B&$;HjQER=o%yP65Gg)mF<3R!XW+2TVlP9-j z8|!U~sVkZs6Z$(Vn;(KrW?-;lhhzq7>u5EMf^LJ0NahLSuFo=);2WXhQu%-r%Q}He zI-3&n%SXV6cWR^6Qt&t-B$eC#e*FkA>ebCCJ5$prsR)++@_W2>~x6MC(G1u7wQarS<)^%p=8RJU z(oGaET@`vUxx!lTw^a5)@IIZRBox%zFBvg1To|0s=;b{rQYS=r@%44r+(qJJM@%Oc zLX-XzoS(d8`*^Hd&iMmU*%O+==_IwQ`o1kBH(BgV7q~EH^cRZVG?7^`mfQW=$4s^f zxkatG(`3NB*1XF-eIU(+-NJnLZVXHSGmYUvw{&zsoB>>fZOa)kGxI*_FaZ*wU zoTcDSjIeKRinX_78xOz2o~OiAkS~%(2k9StHsk_tsy? zzGLH+4-Q827qL|AN7ICsmQkTt`P0pVlw2}WDF5Oz#kJ91H!M%_(@+t&jhQ331e*37-ER z3$04__WxSdFwsO(!G!zvOP9J;aBr=i)2<~HF|ozUXne-ws1X*R$ll6|8rlm;xWCU* zA!kCYR^byohcoK7#NwmO4EeUnPJGk&o(dZD*q?sv9sf&60(26(MK$CqhP59%;Koh+TRMu4gXgD?2u>L!8lA@81K>&NSnhmh`jLodXAl@kK;v+Ps0bPEPYF=aA?!BN5fm z7`rBeHwGlxFexgEp4qxB=T16fKkFo3=;7HDsRv0F*m;?W=xo1aNi!_IHTsG~CZ4vQ z+RDETg4l$LR%SDI1mRh2gPV=C+wEpCk8ivC!x3~OlHr#B^c7UEJVWxd`ve`|g95cL z>8IHd>rys+bnIds2}*H=M?u#`Z78%VjeII1uxiSXOh3$sLg=n>poVIr)-Jj@>cjLZ zd*X*`t(!WjiGBaj(B81d&yt6|OujoLU1eK*rz3?dYNL#b8IhS6WH88lMT9gi8s@L; zC5cPNPz-de7@bRwT}^uJq||?QO|%jlYPdZcNK3Gv0#FW^ljKqngo_khcta>ixsvjY z+*4p5T*#A@VEA#D#= zhz0;9;2CKaDK=Xs#!!D_x-QPbSeZy@t;cjUgF&r=bxjb*ps=QfA{awQ5iY^CX-Rn* z0=dGeY#9I7j4^{_+x9nmdGnzmj{}#Ufb*9H zouK$X;O22R>`nrk{B$$`PZ*$b`Z_U@#$(c$4!F(#dtBOq-6qTS`lY}(UsA*uJjCT) zHKYyDEOFz9!Jf0K3&hvI{#_EN7>}z%UE#WT8_L#YefNaSmS0oD54XmbcBcM^>uD2# z-xctFW!|Yt$+7w+q;`!pLFwumnyk*fJ@V$o%r%G3TMNlXOf@7JFNm&0I_(2Ftqig*r<%lP$ZJ3W?s#+6Cm$wDEb<9d}SIiylkB&;i(}?CP%E z{5!GJxz#SZEIH{@PPLR28EEes@kI~G7oyr!WIMdBp%I_g5pYR`1NvfBK`gQw!~Ne@ ziC7aXie}*yb4%8p1jRDQ#Ahb&_k4VOv<+x9`}AF(QSK%TysPE$+*E+9#f!6~sq}Nm zlF9PN`V{NhNvHABFW|tIb-QaDnZk8Hef!TJ7+U8=D>M=a#o&YkkDq%>27&A~@r|^H z>o6dFp46LD3HSK`gRh1WrQb41Ku-WW0#XjQI{p*HLyITc@ax_Eu+UH}NI^77@GKpP zZlB8rtHAe;4nDV~hoIa)tRR#Vf@Al{91lH{r1OE*b?mbXi`V2+mB?{>q0N>vM|2(+ zSZNItGxusxT__ z4Im!xh=Ke!;xBQAi@C1izA&5~CIo9D>ZhU>_y4Rk5F*AUVM4lxX?Tc?+&lHE~`Z~EpIdLU|z ze}~5M+T@Z`{e(@W+b_Wrevlios&>W2y9f|nEN96gjVz|@82mAah`NNQa4<1D&OJBe z%<8ULe#0qpGcjwXep*XSwbPr9N>0uOL0Nb65%Y^kmO955xy@2vKBUI)Mf>JE)8R)!eDt~y6pKt=&aOYQ2ZG6R#DFURpo4!1GEr&#~94f^*d z5DyBL8I~X484nz_wb*&lx%PSckS0h&e1nqg-!!&EkJ-T+V`w04BG&*lWLFl=q{hv=S02b$ISa|i={{4R^Ir=aU zN(!>W2=-i249~+W7#@zAWb2ZQJwgb{9B-s{SpLD^0e7fIFh&Nu=MP+fA53AcIlXP6pj!X5m_Yc*G1^oi8kqg%Cy|g@N#+ zXqaj*zdvkx4-2Qpw>@XZhFpYm@K7Rr^2A4BkW>D2hR*29T7@1O8)wyf&*!ie7IgUP z^0vCcxH`$}+I}f{X!DHy%1U3CXf zJ?iGN8*Gj3W=UXFRA3}#Py+14f1`Fq8WjuaTPg^|3XGuLJxRQ+v~u#8AS0heJ1>6T z$y%t>Get*Is<^nQNcw8(=|y@0c@m5nCFz&g=q_tJj0fgdXAuc|rT>qr_l~Ff5C4bH zaqK$9F}~|L^Tf2^T9a=L*N+rgX@D3k6-G#ADi2 z|HF%M7jAJ&NW=r{k{-BbK&@{9M-}r^5!|Wxpoq2hKg)}@mM5j(p7aa;$u}Wmd=`hJ zTSB88!TTY3ndyHKKRP;(*tocEi4`~$L%WLqp$`f`_d$NAl@~lN*Q3uvwM#&t=&7VS4Oeh?;h&UhOwsClya#r@y`6qIe<%ZW;d-Y-b_q zgoQA$Kw8@>M3g|^laiV!%|ggK6@rOCAYRb z#l^Ba2lY=RzYU7iK<+|YVNnk8Ns?o<#iJM)I4{W!XI(c0^43Mr9~!;FdWi$<2h<5P zahCRnF6-YxVZ<|J!sx3St^z_4=P&7B@!GWfs-Npx9`z>=yg7K3IS?FXW>?*%4kOvr z|DFB_(b2`NZ($RZu88Q*K6*rl#f4|kdj-97jc{I3tdQ}Euw3XRT+C^e~w$Wy#r0sO- z7;-DC=E`g8+#d6o_;>Tp*u|#sjlG@h9tVCdu1PAhKX(HQ3JU(WFLt3V2Qz*gnTkpB zNqli0h})1T!Ygn;SGuo|*5O=uAiJP(u;`KO)-Um*p{F=I4V7TRlRsVm9kb`}BL8bt zu2}s%CtS5GHe6;;RtFRaS(h2&>E;OerniazfsHu4(cnN@5D|8qEr6;exUK@ohpazi zh6NEq_iK{N+b7pXrI@pqB(j=+1+c;#tU1#y(c<+zEQ#WhUshOPD@ntLS>*e!;yE4y zk865Q)f~=bh}m}LumUFrI;Lx#5VWWU><0#8C#0F*&iPpUizMkIO8<=H%3fNn@2nxk zb`n$ZsA(ppurM1uxd1&-9Y6g^S9oiiG}9AQZ~*?$_ikc&7z?WC>v&_Ae#PGqiBi*B z3r^G4On-5;I;DzH)}nAS%u(R#s~UXhzd;Y%9gclQsF)_N^L;E#Y~qp*UKiDEjT?-C zGc4htz`e1Tm?OW|C}v65nv&hR9o)&)LZv~vJoBXFd`Rt>l{3X5Az0iV{jV|ml z?QDuqbOv*A0~Z_{SX1ZGZRbUuuO5?lh@)U18%4X4u6EGAE2(QW@U4kxijI6 zySDy*BbW^&GF}4VEcko#Uf_PVIOBzlV393!;j{nyy(RsN`hr{)+a2R${`~dv(EE6> ztl*hB>jdL$uy>jKEp4)gdH>NF=V;-(?*!tc>8CJP?#1=S6`(7=b1npDhLYrnO*}k2 zylY+o_w*SzSQfzV&XSBu8#Vp1s0@mcws5rnmrS3;m*S^I}Mf$80)u-m^6l zh4>jO)oeohwN%l=$gn-HV{v0xN|PvV__2q%d? zM4?;1`mnlwAs>HCrUnDus?BT$*J%0iuus>Xa$t`K4Zz;f4@M*X*9kI>kvNl0rLgQX z(nt4Q62Om0jdTeLONaPX$Fvn6a;-BFqp!98+s~rUL7O!~f1_c;H6W7}!!lfu@Q|r; zOU7V3D%}|F@})n*KG^*qt*ZrJRI$rq21hL%!5^76cMwu6kpShfUqQ?=ju$oNn?NnM zDc;GbLI7jp%|^PgO&5Uv?m&j;9UY9Relx$%?s`|vp6>no3kqWc?cqhiL|dD4sVpj_ z(xa9FqyYL&{Y#p}9v7i5*If3td$Fy3RF;F^lE_ql?hViot@)j?1mVnD3(ev=I&jzIr)CUXfT z(VRr#7!k`6Ilm#=izps3pg^gl_1H?$Y~^W7VD?&}%j^v?@gC%0`Qu9l_QQ)0_g>hC zR%HG5hKQ~VHs21Jyx6EPmC(xY1v3z$$=RPR! z0j%qA2Xv+zNZE-pMMartzXiS-tlyp0n(wpK;PRRzP1SB!fJ{@`=LC;cNX|0*h7Z7a zS~V-o?>x6!Y1oavHF`Y_rDXG*gZWpj6`4PPBxSZSTdZK6Vo`D8UeY{%DYaEICkG7> zCU{mUh%Gdoo`n5S)wvhX;sC?Qrm>pi^2c4N5pUBKUdK42I>onpoZwkz%!5Wsu#V#_^Xh;|a$rCFj>!W7# zL?t$e&WVSF2!;FCNrr<6TER?eGLr48wA@iYe+nAr>^fHaM7_XV$~pgk?&af)!sQcq zZ-q^6@PGGt4_4h#oaAXq6|l;sfMw;lD1 zJErzr#c#ttKPr|EduiM)Nl@}k_KXI|W%YKg1?D99=usC67fD%zf(e>gmUJb$i{&h;xoWSN7r${WsqtENR6my0aIp(;wa>3yR>)^( zMR#glpu4-t{Wfq7jEkaJU5nE3yS7Z$BP5#@4&`(W&DOQ>^4Jxp?c*H^GVH5pDz;qb zo5r~ao?RtfoN}h@*1^0zJogsbEf=eJyT17Nh*)GqM*w6h#mDe)T_9Y_xQ6ij z`&6CtN1BS}$zjFQj-RT_gUIXGHOI!f_SYZi>5J`bw0Ircbv1!fu07{Yrp#LP zR+vfg3;piy*7|zj`p!%qf6AvksanHfMSXposm_GMU+;SYx}WityvUIQ1lAr));Sb8 zg2N?`J)y=OJ_fU-kxB9!2%sR(htsQA^6H+QE+JK`q$}gunmG*)_SfZ?oz_z|M%+e0n$2x2Lbbi zy>ky-EyBK0Q1>wj*^9kRaZOHKL3lUIgDGH1MQ6XeEwf7~e_hqZqf;r~JodonJ&$RH zEBsLQOPcY$HksK<;XPKQI9>gFJt!?>0zt@o#iEfIqyjkV+jVCyW1s6DDYZT%^}r%q zbJ`9PsZ%DnxM+;{55U-SM9&B}i&^qyC^1ORiCY(t%=Hpvem?KKfABlgpuqHIAC4kuG5wEK~ z4_1kpomh<+Uy4sY;DZmh*7J~%6Xkq2aTbZ+*@hb%Et(fEwhoxgElxP40rco{cB&bn zyK~53hQ1aO(=^V?rDNtpF&m>%*GUDbf-|CzI0Z>pBBY}Wu*}CN-3gafas)wSIjwzb z8qnn_kjz1R9>H8{XI-!M^bn5*GfQx`H~CWxdwKM^b<*y zw4?RU4<4~x?O{E-8I{kbahKD#c40NMa_R~*lS4*X zG%tz$s&QBpk)xyL&F5jnouv;-h$}oCuil}-(Lw``gf9ASQ%{@!k_>sirok`!>% zdJ58;EL7fF@m@a+us6q-lsZv}c?2HAL#1{MHQqZ@FZolPBDVm{0yBvY(54yI`ul}% zjEv)i#$ek@&Bhi{TdTYVyAEFU#Dsa2e~XV_b=f$}^wq=U)}`qP9*}}x&J=jnVt4IM zG;iBMQ8=7jOvf_fI1)=8aL>6*qf=Y$)@Y$b1L`zbz>(lt~zc;O8g(hl`LpT)u< zcdNtDbybbOqJ$M&mJsL^6Q{u=SjPPdt0LQQ@BI=kL= zzAqJz#<17nB#gI9A*e$>>M<#TxXqijJKp&=xOcnz?EMh0otfBHF?X}FY;av2Ivxxx-9EfZIlZ8}kGMLkZ!3VP zy_yv!9;52Rd^dOSRUx~&!x3kWe#r+93*5c3XP9{$I2A6iPf*dDMQ@j%&+)3~kyLEJ zC14SphO9dWQ z*?(7nL`jH5!X<4pSnj|EIUc|QiIn|#_=8Nqz&P+6OZaI66<}PPe7rbDl5dA>^OBjr z3e=Qf7zTzE1qlpCduww*7n^GW3*JxV?@FuOojdYPwvRS6n#VkO23ndkz{4KMmC9~m zm18x?_3PT|*bQ~W;?hzD)Ru|L0ZQ&wqgHwj!Vq;Vba8U>^2y>Qc5tZNUa#69ofZ!5vMQUnF zuQ~WQSy)g2yv;7d?;=?A>Ntn}sqbYfWpW3qeZ{k3RnKej~|163S8nVCW6RL#hTc-%uf!VJsm6z3?`xiJx~Lj z9ZU-!<2P$DA0H1sm13m~b-jCU=Uu_sGWO%Zs{O5i&ZOa_a-X-3gLsy zz>Bx)saF3wmZr9zK3cIOEpalt8{RZ!{P*L@@A0z$Q)&BKDr~i-FikpHRf2wDZutGN z=>N>%Y9l}u2Oy_LXG(K$1+W#Az%Coi>W=>Zc=Q?GDj#e`x0OETJ2f2Hb3PLZ^G_N6 zsM#K+*17zVi>Ba1DIW1epZoL`n==v!W&wEe=CUVjh4@~t^X7!F{3Ptl;=V>nh!3s; z+Wg<;!PREEVeW<4!}4Hh`qkNDj&K1%LEKe~^QaiG!TuZ+=9E@?0NPo^)ek2m#1ST` zlyk2_2QrqL8*0nTWpLik77aqa2SYx;9$?J*FW}{)f8R57HT7v7D z(RlJW)fT|#o!3bz!0%=zt2<(;doa279H2RdFC3(UF2J;gRhhhI-7dcD5zQ2TW~Nbi z&Mgg6+pQLnZPBi0Y*r`UIbT%0)>RT74NszzB+1g33|A|NF4Kbx?Nox9PKxIbwMs0B zeFy7|X-#y8OoLO{B~sl4;V^JM8UC)ZINa|v5pY&>M~+jX&+9AbiSo5Sl*7eB|JATQ zxliv;fmtUYxj2+`E%h0FDi!n9eHh^ajZIlj zPCh6;iq2^#8Bb^hn8YlJ;ZDlnM7dGK%6k(M^xY%-lUL$8IsaB{I0;+IU40|wm`p%e z_+v&!(Dt_5Siq6E#AtK20VTM`P=bim)OX_=_Hgl%5?GaB|(e*gR6>8;4^@nL_vS{N#Vn9=c>i?poy2%7?U2AJxv z6^;4$_mN~^B&e|!V%C`R@q z*YD>bK!yL%VK z$hQt6NiY#e1HyWV<9G_dti72UR8VDaeJ2w8AC_DI+53+hgO8#%eqJh&as2w}R#J1C zO7Jfhw*MUz#MOp;B%1v%msy(}#@s!#&$8+$^VJ`44S&=sO!lDWoxW5faEgTHr6*nV zs45-tZ0pJlMoPesKFFYOt~~p!W$b=v%N;+zV|nqU2)nq-swx$&2)t|pqZ^}ICOIs9 zssuMiO$5#_hA{VOjY_VrrhM%j**X<%bC7!A-}*o^@Ic`S)R#%%St%i!IMjEH*b4m{48OWP4m=rms87I>(ImwOx840% zU*FA?sS!n}`TT+qF>6bGcP@RF4ITR4r@$f(BmyjtTbjZd?o=ep^PY8XY-|SZFD6Oa z{>c}~H?9aWS;Gb=_Os)?&~rK;s*){7iRgMI%xR5qIjMi+OLcitX1~XzK4TAUTYXPo z?^p1)g`D1REJ0oHwKasm*Z9h$Et|&VbeW@vU0q zMcbHUn!xWwWG0l@Z6fpgRP_l5#OJk2R9sxd6ciP?u06$nvwV1i6%j_9A@D;7=$II4LP~s~RrU1L z0TqdnJX*>==!z|MIqCVY&q%0Rjv$-v@pd*Pcyg7Cw|mA~&m8dWw@5q!Hj^4f9V z=k?RYONh4uGTeAo9#lHHDojimb=1QdO*=&%9%K#sIcCzy;!gDE&*o7aYTzB*gY}yj z_h(uDd7k)D%g;ySHJA1Vp41QIAoHu6>Ir3K&8yr4bMMm$+`%)BU`}his(DEtlWW3a z{N&3Hvl*7$d!PEp@=(Bp;8~3TC4w}7;6fY;92rf}`H5QpVuTKt$lu?85bGz;tt*WF z?UUgj_g=gT5Crypxdv2#G+qM&HJBuFaPsjH0AOA2dmt7R6qMHgh`ZaT2>&C#llB3yC@}rEpxJ6!3A)@#Oou*I90UMbM90rt7g+ zz|y<#>;2xJ`>D~<7?7Oc5{Cl@_o)y*U}X@+uGj@4EEp-WfmK%vkjgemCC~QK<5ByL z>pnM1tG_C=JMNq)o{+M#-n{Mwo;XSd^TM+F7_7*_?* z6C6TtUkc$Oi-Em}3$t$enh3A@zvo9lNPrOu;wUx`_R6Ej)UKRk-V9$l0i6;&*+4!4o)QGW?}POkto_eITsom>lFer$&6MEZDU~xSj76 zciL@4%4{*%x*;$uJ3m^7SEM?`<^U!n3H@;bTXzT|#yb}IpK3wDY{VlttA^piW3Z06eDXRASRE0{EE;TG>(y22RR+i1_F%IsToBp1!upUY|0Rs10cAEM}}5uMPXOcp$6OrrG`aRp`m z6W`2vouj<6He$5#rOSt-pogoYC^GW+i18rA%&dx0=?(3pwRPH-h(W`ylOtH)gx56M zV^$kp1X?_*_dg{0k#&8uGS7tU^n*uTf1aIe0&(TO=iqobmkk%A1o8SN-pvbr-iPNj z)G&`) z{5_$VsN^x~#`PF2VNY*WMoCgH2I0E4w(Ea3j=iF5A?x>;D4rHUTEL#!9i|?^oPPBs zEu^5f0K=oZ@S&QFj7*xx+L&gi<~5_QirNlh=D!9a2R)qlVSpOK)3U2}?wT=EUA@W* zsgDLF_@kjZw*>BeIk)Vpr6zPw{jbO3Lomk3(a&QBvw@<=vlmAKoLMEm3^a$Sn3*?^ z)Klx#!RDKkcVv?Cm#15iD{0QS7JdboC0`ATrryXPu4-sJf_xM(7~<&4x=sgPqhw&C za}5kE*cH8>uK^F6LD(4G7Rz_%<4vX#`@i(K`6OU#xs09X&z;t@%DUk1t&Xbc=tNgU zh`xwYK4k>Yhz776-TO2H$Tc(AFEt(?Y!v|$^y`D~l#ukQ20jvIZx%^O&oo(0H-)!u ze{vkhB4+l!h}(3SZ%j4q@BVcx0vBpePfwg5+&=isP23SU8om^I4xUrO7nE!_uvqNB zTIr{QXz>0JQjfrW;J@e84rJ``vnCwRtz%3@u2M zWA`mC(1`XB6=V}iR+Ati7DorgJ}fi1d1(=CRoqNKPsNEg-eeY>e0X~nrYVN3@Z~tg ztXt=-lsHArkwsalny&^`ufERel5BJde)_LXY&CY_iR+U3UCNgpm2r#$wY7!inT=DC zGX>+Ex%r43>uU_&n-d8M1qU3wii)EO(jNs*!Xk%TU+sj0?T zN1vXlP^g!k3E;otz^+v=kTk&U)x%3$d-B%MlK0^7kQM=%Ip<^awLZeIw8T?1 zMqSNPBDNg<%JlYf8(nN9J6~_L*V~s1ZoCfT+Ly&*8}0vo<^Rq2GMs@<>jt;;fN+J7 ze9}!U*7Cjlz(^j%_JIA7EFOC5a7V=KY~b~)^Ot%i>Vv}{P=o}KycT5^w zOyLK^teMqS?Z1DLd1~H5fZpxc`P|8h_(X0RnmbUx-U6l2lm0K)#>SK^9p}_{cPTBw zhIZ$6p34Orerdt`-*_R>?bSbjY*?hD7GL+(a~#6m4rUY!l5dOt-0H~mUlU1>onZaWSg)6n!J~;E%7bVQQphWzX}l zFi?04;h|5XMA|#>uHs%h!gi`9g_kddyEjkx>eX|*=E@t@R>@t_i2F!C+Qj_YYv6(T z?~yFSUz+QLmBj_}$g~@81lzV=4r*pa#B_?!#_U_j6~$}uhU7;N@XZQdcW!ygpU9xy zENw1bc%}6%hQS7a&SD}f!fGBy`Ku9}~7e{e$ zz%uu7sQ6`1uhm=-Y=t?ks~_(dhi!Y)es?HMEPLF|ol&hoJJ<1MAW~E;&(xkL6SpNm zK4RcKdiG>wbo4QZhnwG`ndvZRpiWrJ>dSi|{p$7W z;|8(waKOt#vGxiAFdz53@2#3ld8JJG|K8Z7`sO^X{^ZYBCK{U3r$e!2j^n2{W&{NU zj>SPg@DbR!@5h6OOYJmbEsv77&t_A%$}(^}!3uf;3Gg-E8i!jala}uWw98{>GFq^b z{^yzOZ`e{WsdW2X-oJnAXZ%IAfdVtIA-w2$@jBP|Ik}&j)uV4TZC?Jee)*}*vpGDg z@vg`lekC@ItlQidU5JWi$as~Qcr=QO6C5Sp<(FkS(nprL;O&1DvYT&w>(fn;JF-nvC-C2Shf|@3VH|FuCW%x#FSYxT(^}|vh8D(+A ztVY%=EcWf$c-imeb7*D?rcg>g+G|T&d@mdOb1ypoen0+f1=%&5>Mxe+zxD4H9@?s= z=4Df3wA&`{UM5M$m3xWDX!Ex}dfiAs z7S_{yeWWn3)waYGXnJ8TuGfuAe=ZwxIgAQw>@&2s+Bki8*#5mfsaL7@Ip8_h70{6( zx@Owq!XKQ6W4_9LC1bHa_bzyq=785-)}?JGQRcA7{NC;t3(UEx3?}TEzr`ibIgMNL ztnJ?Cu|G0avq)Eqz3R3Q9oVrp&|5l9=vvjL>2?+rqy4mpCNAuFEpdKsZL@ZOvHOFo z`mo#swYknze(2X+oSoAz+1+<^fP@K$6OrW6z%HqGBBH&{P zH4pN_?Ub7^lZ@;af2p6#$H2hg4Y?neSXwN_x*s^!T)SkDpG8E)4oCamrMNs;!ku;S z5r}~PwWbEgX=ZYAvVk1o+23n;xPs?Y0o*fXoF)*yetyW-omh2%%OU!+erTw|^UYnI z|MTp3rt9a@aB!H7JUJHbdm?GrryXsHFnjd2;={G4c;n_m6FP(lblV&k!LLvqKhpgk zj^Wmar^WRN9EhJ}1iV55($w)g)T&!G2%9P)PDOSS?Do;Su}{3<^I8a&L2gTZ)TOxz zxM&hMk3;01Et>DIPnNsPlIHt2HB7INVSch*ek$qhTS<#>T37vc|KD!YXp!r0%f2sD zA;}6nJw3%SadE+r=ri%-tw&7KKI0B^>D!jQRD$$Xi^4{60h2HItShpJx?e+ER{|PL zW)^rQ97Y(&e{mq+9@Fi=YM9wtbtf`&mNDFvU6=fak%%#;N#g7?-QSeeXQ#m4j5;P% z%v!QXejr9%j6rm1$9yE6y@dU;To17{HciDU5qJ}^S_&e zd1h3^K|2RWd#L<;f&OCt5+kA2v6}EsrIT}cU+$?5eY2XJUWp)0N%73JfAID3D07>? z|9oF|_`&hN-|y3tb1%{M^f=is>8~q#*R>5XL1o>5`S-j2vz})AudC{&?0N|L>Jurz z@(XU4=wi0 zIGKb>S6BMbog|d=kT^%tD_QQ{M{I!!77u+$#_GIgLzmAcZPezInagH`wFxbalWe{5 zM)|G+mD_S2VGGj1l$A2?k6Hm=&N7C6ZE$i0yIbzmh3PYVVeR<$;h7lQYeLnfy_I$M z03QX#A!4w51z-{Vl9mq_9DSwvbgmkEvikraPjbOGqz}rTs=#RgT3rl88l*c(kv&FoHt%IckxpWNa=!P(Dd-OqYBH3y9C$)V)Kb%|SZodH&x za7UR2=n0;OJLd(Q04T(dhK6p)$%G09TPbRq$V)R!n~7&z1c=uHUB{=O&lS(!GS zDgPI~KqD_NU#znHEF=uP_#tSgdqk04)xP}&XSOjOzJ`;qBm!~ypM2rFnu(FcpaRjq z>h$49?Caiig?q>*f2#67cx3kp`yUJ-2AR+9uhU|9qu&vwpWe_T)M?{mMh)MRWEmFv zutUA(OPt;#;biKF_7ljEnMfaw^?S|j6d8n5!Pp4}BXo`g zYqXm=Nefq>2`POMO|{Qmr^xf0A_x?jDQU-NQkn0B2&wkpUg`hrHqQd5Ep0-f)e0W1 zBuKx_;dE=JluKE;u-GQusP;5_?#6@fL3V%0jSn`XBUuBxYL&~ytT&RNhID2v|^CYeXU{DklDx5f~J4vm$Zt(eG4+z(OzaG<~(7 z_ulzxIru>)@B#mA?&V)h)fMJ}6*YfEYR%8m)dsz0N!`|Jh${WrgptU?RB0+Fmv1R8 zfv5=L&Hp(uYy{4tVUJ_a2PN2Onfm2ErI{J zH*jr;ZhbR$?%5hU9GrFj`^aMq67%VkRfkPyP{DmV&07RF;t>Yfq*)OxDv62Igye|o z8X-T&5yZh!aPHj}@5u;tO@3|k{J@%r@nmgk(3*U~@s5tp{+WdYb5X#V|5wu*=9;R6 z=2I)V>h60HFYIUStxXY&rTX=`5>eAEZkn%R*E`TYo(Z$v0hK6i2g!+jbw#B_Tj%`8 zBV&}>`QQ_r*%Zpdrc+dgNFkEj``!}bN7!fo`A4TugzXksb^Y9Ox>P?|qq3_RY3=FO z_Ppl}KI`AK7Tl-Q#9&0jwWj7HRtHOVeSz%Y;U>SfF%;Y}7ws;6yt1HYAjNkwY zz0k|e+4eJ`rO#<;I>+D{`4IS9gEcs4#t(y@87?IKovS95{Uy7)C^d>mOG}&5pO%i{C6W7}9cenZ>9_Yx z)cpFVIs;~|Jugv;%PwRx@*c!HSfpp^8}gsNh<7gaJ~#xOqN7U4a9@u2_a4$c_=tIgfC&EOS?+z(;BdD;YSrUJ$$r ze5Tm?ANol5)rlKeR7VAKGTooLx!?UxQis7}t3j7`QZ9>~UXh8i#;7qB?yuE<*20L4 zvPB3kM6-DlLxBJbSrF=ARA~7Q^`buk21RB@mi~{mFlUr{QG^IzL-tASn8CjszpYB8 zB;uZywmBCC&c|Fe@9?pGM-y5#WF4QyvZAf1pNQaV$q-^t27<~zJ|sdNK#28KMz2^s zCgN0NKIAXhZZ9e>MvZ@rXxdMGOIYT}p%?OcQGZL$o!rR;V5z+4~!24-{l)4$15cVBUqf~yHV`v>FdN~!QOoW zEYr&JUb=t&c2XsWld)<1yV^O^h2E-2L=pY>0_47zysIT?=l}QjUaX(`ReP8wNDAw0LM|gpTL4oc<^43Q>T} zWjsC4{s`-$YW(gwkKGO^N-5Nny?CIHtkC>2|sNvcTqe{>zC?!SiT^l(XvQ+w{$H%@#QKYfcjK^vs z!}_2=LtQMmN;*`8p6bW0{rFTurUJeSL7x6m&jD~7(_jQ(J6v)R2cZK7aj`joAmogZ z!*?ACQrpNVrNh>dGS@lnV`yY${0_G-NuM%Y@!qxtV_i@HovZQLYtFhJcN*&q8^+WT zW)>UA2d-{VnJ|c$HbP%!_R_csi3T`BWEv`T95R2jyMk<;MF6T1>b3Rne6ck+_5ROu zc5iuk`4?f)08>)mlm%s0v?Hg4G1WUUF2y7AWGx}|EI#(|TwV9Dp-M%d%eMFIWHcT_ zP=O4wZXMpbq;;^Xovpm-qE+F1&4lq2)tp*YhrXuPg?+o^DMEMMV!iZV@XaA}Q6H|pBP0666V@wP zK_UmjhU)_p$+@!$GUjQ^Hcnsr=*AAZ2QP24te+3X+s&?{)yAWoNP6B1ecKps zBY|Ca-&&JcK#JZs#?}LedB^gpbfyHZ{bgvq{pMDBI_Gsf#Q@6FQIEiFw+hE3DQljZ z`^{0xcqu6`&Z|4v$K)EFH@+_$eiecO)5I7#SB44lpWa{R2q3?41Mg^-!D|eOOj~za zH>omWWfT*u%Qr#6MveYJp>$Sr4l&pijxnOZT4}*u&h?rJMM{nz@*J}^NMjsM{||eP zk$9nzB!^xY-;*7X1XX~pZo~@&?t^em+L1ESixG z$!}tpZrx7YA1i#p0h;v<8KOQ1;fY3dRO}%zJ&kbf=u>%An|{rmR~ zbz0jtp_IYVaQyctiOQ?0&a9@ZDk>hrh>Q_f3E)&63iBxLD_7PSBt_k>t9baYUHm21JX6$kF1 zZd>~i^M`g^dZ~nPL2Yah6F;L5;`=~gmLJtLVT#Ej@DU|Yja!Mv^Q%u-Bck7wERS8o zh#VEIcvh%Hi1Pl7JF8z>3BnIZ*Krl@soy!v6yv9tdYH+`OC!ougPIio(w!* zi51Jx6gt+zhhGRU7QO9Y=kp@}uogCtesN;9l=eF&prDGfSi=p2PsC4Inw0Bg{Rv6< zVaM@+WwdWzvJI*1(BBZ4i-9Wc;jXmfqrUl=_4Pujse^-q^!vruK-DI6UmK&4m#;D? zQH2e2c5ZdFDim&|N%#z>XU_;Y6HttDZUQhQkANM1JS$gX+l}XUwA)njxc3SqnBGhj zBghaqot*|$u-YFkSmR1J5;zCmDgG_19_P3Q_b$}KTEwKJs-B+Zvd?q@-p9$SVJgq? z|G#~#03q4-Rs`mC_=7j&6yeHv<$Z|kaXhUW{Cn-0)NAfUHglxd&3D`%BBE1nzgYk0 zRX9I7Q6sNPnhZTzpw67ugUMz+O)4Uc>lb|Nt}lA!gUBOVy}a9~pv=~TV~O~u0Kg8G5vN8McJO3#hRxW*0;s+qUUCztBqcylCc-O@ZYL)(^ItSvxJ z)>k7)&z*q5iSt&di=v5AXuu8fe}>%Nt>{8ZCt7vtF1+QWb#z(-t&GhR~SBWV4F zKVB}HWm1(U`2M$$2-WiZ9gL>1q)x@Rk(4;p<2?slC5CmkcRDYKfksX{@5L3{=+f6E z5;{*WN53b<-n*ouiuWArM|LZo(|!o=#`VA}{?*5Od*f=1qIj(xFJwG~_nRZ?J(g`3 z&Q`upjQdc*MMo@dV|4Z4sD0XPiSY57DKYq4SX!xU?hu$^754YWzrRk3LplFu&Uq_T zto!|O??>WFFFL+&#Wb_#E38{`u`w}7gutDQtD<%p>yGIFb^-OkJVfaDK&Hx>v8Itk zf#@q^TXAr(m%7qLh2XVCrj5y53&q5T_~b+Yk< zK2q$y9|))eyilsa{q$@>zV!yogd0tIIdX~w*(Z2|N0_arT=M*SN#r>rGes${gT74r z6_y82+^8n_xf7~mLz9IO;gtC6$dsHo^OR81x;2~N#~)PW`MQC3Bu>!WpckZ}bF%)se9vVK!3iPvD!mJXVXhoO=Vfh$eg+ZaUeKfQ5)n1U9l$DsVtfG&7+^H_L^AgN1aWE{1JvMVSFo__U@yQwvM-%dx{Em z{rME`JA%*C)$%F}?|wpWY`Bp6GcahfNTKI@HKeu+9uQ%#k*yy*Lb_Z;Vol@=Q~iT< z%HKNZ(h6`QeD_w(L#f*NNE8$}^Y5ebeZAI0w0Pd#q@L?26+K0v^BS-2^YPW6a>NQM z$jUOYwxk2H-jw(1M=gMXM4GYN%Hj)d^Lfp!P1=JM=unIgH)-Wz{4@?xuu$cdc$)je z!AvL&DgIq<`jNT7wsjlQ`0$meMI92wDGR!J z_(zQxN>QOS;Q$ceIl7n=;fu=XeOzLy>3Sbm3NT>?*T=0Q6uYn6DlNwB2Dv z;J|fSFbjIFauCCoAz%T9Ye49>5w*((DB!3S1@4F5I-~%!2 zKZ%=_I7JvjOS0+r)bg&WfZ@E$I^QrxWKT3J6vb7&LDxuxN`K5*Uj;vn z5pmYMLk{``iyV>2v}|JDp*BoK)=L?TI`2Sz;qU0A-Ot$(68J@rv zR~`9;!h3Y}5Xl#MB4jJgNxU0j^tQ`&{L=6D?jtw6A1p277evF5{pZ$5DVb`4smq~X zuGhjtW>Gx%r2&+-Hm(Ie>(+4LJ$f%m4FcSZxgdfGKxB+h{T!77mJOXESc1=H=(Kzs zga;=ZsMRB4*}z>H*9|x?>Jb-PBM&-85r}3^;rn>~K?G?f61gP@Jt>M4RnJglq&%(mw$aJSuK}Av-?J2kPDGOL#@4&)4kjD- z1!v2Mu@MKG&WByB8Y6P{S3fE38cJUeu73Js&@p+!qvg4owqZiuve)k)FQ&&(lKxTr4&jk$r76g@4_`Jhmyi%xlX!Svi0mm@*Ae zD4qQ{4`;Ne2~MEf)7!fNQk%rTaPGf2*qWP$NcN`9Bw5_H`Sc0b8oU+-R@u7a0E|K> zcP8NiXaP1}9`@6z!2eJMF4@4cpC2mbhu4IUz)=DL@9zTLj>Vsw|L*{`R$E1=6Xd-N z#TU+4xbfc6_?BLze(!f_AbEgarSgpA`C{C!$Me5W?h50vQjL{T^47{uwEGhCeh6Ot zy@m0+$a2uL0)@?cvW`qyna+O*1br!)lOujdre>bkbrgLWvy%W3_79${8akKGyjz~L z?`ZgPDqjlzvI2VO_N=YX9kK-w@!NHvnmAB%`W(hDzt3Qu$59>kkYkjWy@>=S@t?m~ z;!y9F_3qZN(qQC`W~EG zcTfHPUO!-p=*Yu18J>AIa$uWl3C6R@pzr?5vbD*n{XvlB%*3R=Bb8{pOh`sV1d1c{ z>=N1frhZh3B?%7w^;ocw9k&cB_m+fHfA-vA$WI2%P|7x%Ekab|weU2f65^?3S?cB} zq6FpJuH$R9p;!DB=BKX72BWd83x6H!;mz`1N5EhTb8;+Rez;TL|Ahb+@sihNGs=gH zdWBjEm}J(9Un*QSM6X`qoovS~k4*d{bl>FNr(~i;OWm+1Tb}D z+iQJehqPBToyj>fb>`_7KPBR{MIz6x92@SsFrtc!ycdlzooM^_1 z7Hjrd&6``T^ab8xiQeR(x8m@go>~DxK3nl06&<9yV#BoAjC))gzL}Ysr?)rm%9!x7 zvNKkhtZfE7(0O@KbOe~eL7eaY5>c{-frV1W!-f}484NAX<$gy})?ng9LU+L&G$gI< z?U(3HaIi3khK9yvcrRdzbkZvp|A`gKCW8*>>qWs#WNRDei&oAo2V=CAeuhQ0FmkHTZOt zPB&QHp7KpIAcB6j6S+@vmrhk;?EFT`EJ57;dGDfm=d~W2XXp0k3=w+-q5*F}1h@(n zKiOdvcC_lijN!G{1B4sWUm0Be%X`M2O3&S5wQu?*nP4?qCt`*T1AA z8V?Ii8V;%wrh)Sq;DUEA`jgjEn z!D2#gLW~%B+_tr^ezw{PIkBi5lKtOLhKMCF)Lv!;?7W?Q+xsz_m^fl}8Gi(In*%CT2qk!5O<%6nL z-_G_kcD;fyr>}pe?!*uLJwQ$06?~ND{+GXl>c^J|kFCp)jq~-9;BsC)JNDCnONFQ? ztLM8RIdXOGN>*yGLISV-s|(ebk9LWW$Z;CqP%jrBfzt>>o+<%|82a?j&G~@PfiVPl zSrmEqf){PXD(YHkLAx_I$Zyy_6VFQ_uNJXiNy03^ezu4RLqSu}*I%oV+P(0N5K@0K z6;x%(Xp#8!DLFOH$>m=?J~Py&!D4>Rbbf-{I)tQ#229%7j;R{Io-Tn~Mz72WP$X6N z_XT?bZ2bJgTe~N6lMZK<^dG)Xu?S>3{ge3Z5J99W-Em!sQ}WzA_UMKquES{>eeDE} zelTgyf^hy-CU=YcSB62=m5e&*?CWSFUa8I&I{Kq15~)~7S}8;c%l^CbyleG~YLkyg z>6Aiv43_N~n4K0S2i1;Xd~kDfQ^>vLzqn+5e@hk4%ECxVZa_4!JWJ4r0f6-rsXVo? zuvpyQJ~y|)0N9Iw;g(+{D7}%;FRa=Wa7jg8Em0sj%@4TzX9}W8uu^|nr@WSU12cX* zrarll%bX;Txic0#j}O;|s6hPlWFPTALrGqqLD}N!Du#)PiLB32Y18D{eLWWh2!62R z8mH-!Q7MC}ZlM{B0<%F8bT)*(Zc3t*-CeC2MHHkGh_`ZzV$C1ME`#EhSFUDNbKYq} zZO`|bnputpQ!)k1@~Lc|F1kAE?oS3@kC+`Ab6Vkcw*}B0i%-6Wkk``fIiLb=+YuYYP7dycG^y<+TQt80jl8gAU{Zo=8T#X4 zuj{vG$UC8#B1(to6m?WJ7&x6Q&!^Qzb$M9`H``mNcn?t9{)wui9Ks93<2vO5G{IDg zSUo_b(1;?{04Vmyfwy1q9dYV=o5!Ty`|3881YuTsx*7WUnSJ%A=gis})_1?> zfbyUc@TjL-LD^EQYCqJJgIHLZ@#!6Hf4tpr&~xAfMg!89q0;Pf=sHT{8fyVMQqrvip-3p3_4-B;>D31%*>DVV*WMHX2sQ=P#WuKT)se zeS^lZA9v$o=LXELJGW_)7v}c5OtxxjL;x z$nG4qMXQe;H2_@+ortUDR~40LH`$dy)bP3bVePWYE!Ree;L+Dq4ufeJ57w&p-z21^ zVL}&U1g5g4yZWb;q@d59Fs>`Zy|kBHnpH7Z{x|DlWKtoJH$txF;EAm^wUJ1}LHk$B zWNGZrBN-bu5jGVe*(L|2;u%y`{%4?sToDkA| z^mlxf&`X+RP30(l_tz4N)bDmS4XfCPAc-n9tzm&k)Vmu_e~cD}-cPgr4uiHUwYMq6)F&kXT83 zx{Q_;^I*rB7h>I*(ybB|dNg4Em@)n-m9{cjN{|&r+-4Fr#iD2;R{D)$C3KEh=SEVL z2bnneXDC69$q{(3vD!{htAB(ddPA4dI~po?y7ZiDZT*CZcx|}CFnMfS8}#$K42fwf z(Y$@DOS6N72rT-aSEO@WqcN=o@tpBX<~AypnFP_vPv>1bu|U7q8A1_UC5X;z&3Y=0 zhZ0en)+|f&A1RYi1(YIT=?bQWluVe9%RzMBZ>wb>34c|D-@%dH-Rm`x5Ul58284q-D z(=E>XO^g#EAOT>n;YsKNSgNp>E5%B?ikF2S$>e{|hvw{kDVJr19@H4v9vZ4!r5ppte%WDbnJ$X$}0(w?>(dcdWBw`*kx&))5 ztsZl&H0eBkobbU3DI9=XiuX4)wL^LqzJhpld*y3wc1S+x93IvI;^6e&{@yP^$0#y> zdg0I4l@6YBKQnLt6*?K+#B#w=$ZEg_s0ZAHD=pRiVe~ zfJ2rgJF??y?*v;-lyTWQ04-XHw ztBbRAX$w&FUr0F2e&CD521CyI7c<2F>$gxJ;X^am0Xt~FG9*Jf(ibe)8)s0gd86(b zz`_6T7X@s!9R+g|P$*?hcVBSP^(qz{`jC&Hu14cwuSfq?-sRIB{{r>P6y*YHR4t-i?QyHe=6gr1ywAZ_w;6YkxCW3919BL7ugWC^z`T2!>gwtpU@jJf0D7cP6nwl$vD*LAafXD5 zAsusN2E}f0E^X}XDI2B5-RQ6si51Gy5f2C|KX)jp;qaa?oszl+pN$0Txl+TBfOizG z=PD(6o;3AY+brRaKXf);LDX<}psR-*?TD0`(_yOC zXfbCx!`O@Fx;jyeA87niYpj9~nZAKhwhB|L6}4^3qPp+*qLx4Qu)a=)&c0elsy7)E z@cr0SB21!l=$qYM*no2VGJv6pQgZhMQT39%3E@`GNxSj_OG-qciybsC_qNM%(6pop z#yNtkZV?FD2CMuI8nr5(t|szl;XLH$e8w2|Z^1P7nyI>v51>4Bu0=el!JyWij<2Ai zexs)_9#my5L>2XYR>`3j4>~vyD(}e>4fvo(N?liNKTtT{nk2b#0|PnVgQ}}CYUe_+ zrpD^|pCFi)zrVMS3WL5(<{2VBp7^b9Blh?r{|agzTatcu`i?rUY=Fm3-j;Gqk}0K3ib)fB|6t#dpRS z3*{MEFLQ;|-x~)ris8CF$3g-D$6JfsMhE~CcqOSmN0LyqqM|W7|KzT^{V-Ow`u-$gz?tN3VpU?4u&;1O1$G-L2Jcp0!j^TlOH5SR_p%tzou z0NN1~1klEFTfn-IlgW-`YNcnv?xR=P=O&y*aBdcmr)^`tRVM7>Up$hMgjAN)ddF*P z-0_Y3E#4%^Mh#}3v6@F}q9ELb1cd47={a#2ZXiVChJ?$UVD8`cD=y9p%2T;qjMc+@ zfgk1Fvqt8#k??nC0ebO_ zc^+u%KNbI^pQ0_AMKJ36MoP$pit4w!+XgZs#ZuPREJlZGW(H?u!SvE>-z)we z)qktPVsf6vbZ8*Hb^etlOz1Vcc97t6eo#x7uf)J*&|pW<;hw;6iMde6(N(O)to_>R z?m7da=;tRSs@$fC_P!Psn0mM2Vd9>63=pkAEA?Iv<>*2xEQ*W|75>*}N_0+^`HZXtUkuKs>vN`Vj0s=WyyH|-Upyeg39$fTAfW3^wFJ%hrQ z-yY|G-V4|HA!`_OPFc=(iEPc1c_Mbx94)!4dL^-J*t8sG`UxYxZzQDiwI6!@aV1i- z?{Hwc@c4|4*btc{DdyRi{P5tFdf>Kb6Jvb6+NW2OlAqLw+V6R62E(~RAbTvmI=eea zoeI!1p>VCcjFyjCs9ulG?Bn&wbZgdgh zc$8Q9hvbxtChA^ZqR8wdi1fz*8ne419x@D6jg2Xc%S|7a8RA%h?Af$^Ui`B3vwY@z zld2`i3+w?L8v-2iCpH5nnaU;|-G9%BAq1}*IUO5l=z6E?DZedq$1-78^Q5rUL+2o~ z*HR&WZ{qD8OYV{&cQYu# zQMugOhxOb*e@@rKj9qH zshUV(HMoBAKppQ5yCX;6-pNc4(1EFqoV*+f`2m472A*`ND>P7*xLs16h|xQDEbbvz zTM*!cNbw_lmYsvbZKD4CQz)>lZA(l~P`a2Nl;cchQg$eryop=p_22l0*X%lBj{DYb#aj3J?aogW z#oSr)CKQp#Kvz_PpMU4yjfbhVj0jLq*VfFAkyuQ;O*=VUSKQxU^gTj+a|F!~D57Cq zW1Cx;xej4bwhVJD4aVj-F6_(m%xnH2207G{@1Z-tJH;H)-H#ms$v};dM+T!<`eWQp z{HytRf*pC3e>S5%JaubE8k|_wP;Y7mp`rzcp_{d@*r@ZEC^fjV<{7W!B`|@DD&U7; zrj!<>Okng}hdjqGBw6~JQTymN^X+TeLiMDumKIr{bP66Y+mnE9 zh4e2eot@Lu&dwX=bLah$BU7f1j?Z^0RKv|>#fL|&mozytJ!Ot-dcSC~e{+2IdG!mX zsP_cn-EBOF{doEi#whQA+g+(&RK2#|tNs-t1a)=Tr}bF%NWJ`_h_t`C|6EJ;3PfL_ z4L&7SnMTRWd+-Nhk86&_zD~jYwD;X@>l+txj~3)St$8$LO1Snc_rBT(V8MFo&)+)u z5J0Mi=?NQSH(5rW!l-pkKu1rXZ%~GGD1ds$=$+B#^Xw8yM2iw^=8fBhK=V6qL^}ak zgo_T|TO7zc^I=p2L|0-Dz;ZlFQ7BsPW#n4hswGm^KRt9_jBFa@91c zw4erWt0GK*`jE5;LenopLj?T%{A+i%3v^)jYUr=E9RwlrBXnpmx3RNj7G*=%&5;dO zEmQ=hZAfS+)D9;v4^Pj~l61;viCDS0aWgV9YTKG= zN++MWRhwkm!cwJWKxD-+{q&QVqDkp?Cc*6frK%F|5x&5KmeVgJ(~b=@7fbv0ZeCs& zy|muPK9|>`eb)?-n)Q`dsyn={PQ>wBDN3?NNJ$A6xPDO-+El@4k4XaQ7Y<3eAJW%f zU~|jZtUBnm#S#Iaia>zRi`3oC(_MIhQJIaFoGEZV5@`#A;RIfXz~0fO;atlLvXNYQ zH6tVPcnCOP_%b+%my?sj2_AmPFNg&#N4}5Z#|Qs$bqJ3zf5^juGKI8^1xG&MDY?}{ zB}x-FO^lC7p+u!?TOjn2I>vOe()n67Yb#RovGVc6oKSQ^p;R> zlpJ2*v#kZ)%&J!2$gE|)wLvsVYtQua59;u`vl|1SLUZt97Wb?TPr{!7{QT~K=~K&x z#GHSKPhW1YJh~ELu9tQEU2mn1=_^D~Er4@(xdO3^*nDP!J@hOzDG>GB&5qP9ObB;D zC`=wiHtfsqlXI?1fen;y7csm|&@6XV`l2EX5Tj0J%ZO+pf{iEXE;sUs%Cf|81X#_GxHIdD!odNgVhB%0VAbYvf^ik*LS^<&kbd|C2ieDk&oaQ)`k;P7K_QqmEypO$&JmmdPA>3zD# zthr+UuNmyP6#N#EP(m)-Pgbv7-iSU90gYcC47AwT*xrwcQ;-A$Ex^G|A3uV#7tP(> zJ+KuGIs3XTb>Xy}t!1wL<7)2#aXPY(aGGzKc0^lVPz}z1-?3Pfw`chf8P;+zC@>TV z6@(fRS|BRwdYL!;dj^C|LCh*LCwDF7#qjVjaOk&TIv4}nKQdSfqAO0nQ0sm!B-5+$ zGxPpJ2GdlTEJ_8DwaJ=t^4!r7Yr^gE^yR9o(R*ipSGNLg4x zi?+YNFX_L=YlQkLA%PeX1eh)i2A?80p!9HNv?rR>+GtSSVK=HUC#d8fu5N5pgL1;8 zNop`IJXZA3KZWy`?Np)Bw8!WsKBvh{@0_YKHts9M)q~<+8&fPSG2Z7|$m_3kgeAOy0?PK_jfS+CBR@6f!dxjEnO z?4YlIqGPCgn)`j|%sdH0&I8V0o0!lJdixAKxsjXo@?(SJ>!;7)4lu7!wXq;|G%$ z>|y-dJe#3m>)V&oi#{n9B`Cy@Sp>l9Z3`F(!KbeTiflkIt%NhG;jRSP$hR^>EWlQ} z;eU=kAe1m_a$bCY416<0VCyxy+(%joA;J+l!>@up(CPBEqx@4=AL$A8EJMqm7gtzC z)jY>Y@xXZ}p8i#J3$>^>7m#@3qBu||NJ_46<$XQRvyk7LESZg`Y(iu`GV5r@lsq3# z`1~E1WvZptdA){|I@FHW>}r!H+1M24^q{5->ZZrE3?Pt#3e532-g$) zNw6Ovu$QWaX3fAPiTXkjcrhMP&{oLO0hkFoaSTgaUL*~7+@AO}W(K3p>$#gwD&8GE zG_F2Mw=%-EB|(MX82dUBNZ*1-sKQ&`y;N1?g;BiFr+aS6X|2gfl;*(zN~MkzyvigC z%ABO(;4TXoUiT(B&`u7dps8fq>@;c6xt+V*PVy?}U*W@_(Fb&vLU1$h3DYQowC%~a zz8R!54=Hf~vk zYoOe3Cj`U$#`h7Yxwm_6;4=tuoi%<&C78if`G5e}W<>Qy{@82QgYKu>d2gg|I9&(q z?Y7~q$S4>%@%|Xfgm|qmvpO5rfY}coWeCLh)}%+5fBrfXE)kY82|1&H4C#;Q;QP&} zfZ$J0gHJ)`DK3y!K}-rWwJOhy`qFXwdIH_(kQ(g=xuA#e^=-yiWbW12{IrW;WZVlM z-!L>H2P;vNqj0`mgCp2`FI&@(Tw#3z>}kRUGmUCG|9!a5_hvZ&Q;=7Wv~qL9Y|-pr zxbe(!F;t(h>7o^dNFbB=24}v%t{D0aq%=a&ldk&lTTBpOcTNt@oK-5)-ay@ri6l(X zt4Iv0`$VuIJ;wkP@&+ikOP+7JZ~CHvEmrMV74;jTBi%w?@0G3jQRW|PEDp1r_ZEHgrmuH;-sJn2AGvo6VmJ{8?02E%lfLn8vvhBS zSw@C-!S9d&61*Ir*}VfD2U6idNH3iS-oW<338NAT7Je=aG46Ygk-{v>J!}PGCA6}0 z7GHm37Lo|zaFv>VC`#SW6?)nCL)oapXnw#uHzy;MTWl*&GhEoXXKUlsCS}hnG$y%w z@!y;Pf~JBYNxxEzik21+(R97RrTM}aGMg!;%`(7detNckC`c9O=HbQ3OUYRxCT& z5hm-L1#cM%3KJhIt*zF1o?kvqIZZYLW+;gn^2pvOzZt{X}AYFhcXFV|~ z4M#wL$f(Tbx8B1L+|+^85KUqvjSul4HmfVlA~KYBxlO+}$HnW0WX9B`PDIT+C~Nx( zzcc2=yPJPB5XH4wI*Z^m9REn+q^ylf^R2@6Y2f3hK1KF+g^nuQ`$+~9zjijPY%lC% znc;3)9Xus>KN)4_FnLtyX<@((87$@NywQ=mI=5Fn((DD|mlywR#tmol`~ulRHzj~9 zajK!9XIlo9g{0r9U`tDjUaQYjWZdM~*qFn35e|*GQzYcYEy9K(WARRy->>StVAm{2 z=C~{cWDJkw6QIw}r zyDx7|7WVRq!e*nw8@;{Nb0r^HQl-!vL7D+QkABgD+#i$T6OmHfv{m9grf+8WT-I`7 zK7$oK_Bg$rCUhv#(*!V6>s{zWTaLzUQr*Rf5S(>FQH<#*n?u<>LGrY?N2%FFZ1kpE zcd;8NgA4>hK;|~OL@ZgNL_*4@h-;2o>1DzsfbVx)=Ha<@Q}2&w0zZyl@x#Tr6;6Gm zO)h{|DJIld8qfxd%T;lzeZkMmukJ4{1T9NqhXlcrh-y}~73@oJb9o8UF!Naww%O}$ zf4i_BL5($&!bKp$!w^zZP}stcF8>m%#v;I3$s|Pbd9GRE*Ny=-b;8LXjJY|c*bkCo zdYZUJa=smnPB?8^lu|JrAiDl{jP}VLO-a!z@&VB%xT(=F`&Kd8?;wqxNKJ#-ijzi4 z0^-?fbe3b_S1Qmv1cMHB)pcAeQ#|;|EQQ4{dv$$u8GjnLXGkG;P-fW_w~$ake(e&dCyN?drS$b z7y#!FN-L{|+;7AG$&tG_ywd+#uyclMhsO< ztx_h@WVIn$2%<%1l;N+yn*`E9H^)y5+5bv^{TcdF$M#CM-RpCjnK{_fuyZ}8eDR-} zzhnA=g7Q~c)OVIp?fcgLJ~~#)yS~D6Dy*!za*(^lZOmE5MK_@1U(J!o``dSvIb(w2 zl&F#P;JmpIFT$tdZqt5>noB%|2*r>di#Oxc9#Z3ys3+rgNCMyW?N3{Ypec0y%O7Ju( zqGQt!vIz##j$S4gsu1ZX5tC#y(MBD{eSmmqZw6xg=ad{xA0=6NdGVFeRnf0ItRhv> zvhX$&vy-u)59MhF3v1s%e`sb5nRku^?RNrIR(w^mIg6~sl$0|MT49o*NZ~Qj znKDaC$bKK39zp9RU19btM-?k)eQ{;ODosln9^p1}reAU@Yb=$Z${*X(`)nvqhx`m> zM_4&@@v?PZ8Qk)qKxbrRgjef*@KIPzN{F%IsTMli48_Kdk5p`kwyh8*cBuC$va>Oh%0|>GT2_3h z6TZq+CLtNaDZSeeGC64Is7evbKE{5iq^%?jZ>i#sKr$x_ci?=uMB=!Jrgd#37OmE#PsPl$uF2q3RTSQC}e-~qX0gg+-pd#>B? z)XfKwb)oPrs7jP>JQRi4oDMZM10({>qTB{Ek!$xc41=Q zX%9cLDYDQEYHSfOT=Qp|C0cPrU7sf9T_!v=*afRHx@J3pW;@p)zie+%^P;2-C@qnH zpyHfvcsP5m;ByN@sZI`b$wiA*Orr8ub*kfs8jFJ`=(cic^aKdKt-E>$DjhX+$RzF5 z1g#AHffvB-K#oSp(AjAa#@ONtsvhY-n!0Zbd#8LAi@Y_&eD*kB$0bjzoL0-|JtKFNd$ zJ>vZ>OtnpH5YD4re9y4h*3B{xganh}co@ z;J=S*@`1NWGZkc(3ZT-gyuW&=n+4fSsJIL+!O|%|0edfRCbj8@>_)6=m}>Y1X^VMl+-rg4kMVT-#E@d zhsm7{S%o;yPsiTRtL6L#_BTYjQ5LGcg;ECT_!g`(>^00C6%!?RiQ;dBP!T1FtSXZRWZP@(=Ov7zn^XBE=!k7F;z|-N*Xg$3;a&dN2ZTa&bjPLiC-d z1~IzkojZJXL*!T3u*BnNzn75=@y@(L+8LlBh{!%`(je#3Ev%Cg(0yN+w@39}{7yio z2KkV45}Q7>gtm68=6@$Qxg=R$)_NQ`+4SCgYoKn!E7bGVS*n=^m+%#Hg^nw-e8pIk z8boj8m*nj^@W}D{Cs)i+JtAkwR*-r@Lo0`>RM2N7dSgqAJU2zNzl>VkpDbyDQ&xKZ z9iMJtAwU|LZOt#tapPj^9LZFo03FdNkud!`>A+PUX%Qt99ebzfE6bCTG2o1OXSQV0 zg;dAfHOT26w&s{;=Y)JNaQ2$mBp@fJxMQ8zFtK$+ZmRrY%pqwOxBn)Krp}P}W()G! zZpsb58?qABcS}4kwc<~#i86MM6^f43$jh?Weo`z!<@`s5r68qV8~0Vs1DRxvB1*+y z;?gCycC+Q4!NcwvQwxs^ILBfZ)!!ep4YV8UPkm2~jZUcm4EDSiwwuL{*0{$=ZpFd)4c=Bx|q|C1W->Z`LyPw@!eB} zMY=^2^WLGuIdWP)t@CRZfF_3r@&I|B4~E0BFdQ<6#g1T@Q9#+Ac z_AI&R2O6ZM@@=SFclcSG>1z0iXP7>aOyr8_vQM_5M!}0*#Wd@(o8u62z^Un)U<-Fm zIFF(sWh7t~CO`9^=uY;oqO3W($EMhhzfw!~6LT=0&B$?{CX8hOm zfw4tRx0QReNnO2IlDyyx;rp(-yyq%obs9cGai;(7kY28tnkNq@o_e9ygpqYE>6)%3 z9l!3%bS2YId~T+%^;U*4@#HMsXnZ1`(En4~*0*=;uP&pBRT3e}u++6s^#HAO6%myw zp4jv-^sY=yBvW!#RRvQ;T31yYOIn)h(oBPLMn+diX#eSENmo)uZj?rn`dm{Y==q{z zRiF5$@!#B^b>+LiJIB14Zs~NC5O9(BI+sxci-<6g?nB&<<88M4 z%}P7nc%+TabyJ8c5EKG4l}4 zc&p1X@U=b?8W6eJrf~^?uDSz?*Kw zgdk^R)J;i#^y#EKj5FE@;42zv7w?uCbg!8iLUn{e_VWFCu1P$V!!9Y`eaXC%7N@~W z${S4M&2l~;J-QSC>@D5X`r5NegB!p*)5pT18I`>8RY&fzC-r61UHi0*^dX1eE6_8} z{5#+3b(Ws_WTdtK>)`owkcAJOr%+RW*j2y20jHOmS_8SpEsjnq337z{$Ho$oIj6{= zVOYUo2+4RGzZ)0c{)Oe@uryf{@U~(Y_Vv1>ZgnJA?Ac~dvFNSgs>5OmTfIRyf_J6 z7TWG0B9|pSo3I|ir{rCxFLQ1{(K&z&l)a| z`Y4v)vQ3+=SHC+t)!}y!UrJ7f^Q4-qU{YXdMg9a^G&vo~ZKzkCnX2zu;e;m@DP;yF zIaLq!)y|VoI3APJP-~T`#1<$kmu;GnQ`~U1^&Xk*!#wOF-)59(ows`AHVH+Fn8x-{ zCOI+n(4f#uS-T~U4E2$+;={2bf!ec-*GW1xQ7Yo2QJh?Sp)8NcD(G(3OUs=_s7{w> zcRCaVjdIdb=O@;rm~?f#=Mvnbm|wJ29^3CvNy>RV`Te!PcYqx_IN5Jyd}diL$DfyO z4MtTeEU&zg=tM_^8^hk(=`)X6nD_u)&D1|4rU3b@#^0kNp+X)O*NS+0aB&a&BrBw_ z;l@Ya%~s*-R}ZUDl$VlAPKusGW7jVV61->sYR7`06+4mZY-_8U1OkjLNV27IU^(yA<>f6GFW+L9aW-^yI`}K?w3;(#;AUf z$w&JyLw#vtzsn1!Yl)CV>)$lXw*;M~!~U*u*W^<_?wY1AT?YL9!?-V52tOCZciHFm z!*bba<+o0xxv>MeeiTt7?3CHe8V|SCqvh1C^IWgYFO&^79H*HAt4f-4H+^KRGl-l= z0);|N$E8Sj|8gX)ELf$TMRt8+B-3gJ!m`tAADMpLmvT=zR{3q_ZUT92Y?|&9t?YZL zMelvF^uIX9ca6LJC_l`FC0-Df%ml5+dwcWEqsPL051c9t?ASBxzE%s(kUe69-1OqN zNss$UrPodDR4n|>MBsM*vtZ9-I9^zZ+OG;CH8$}j%LwJsiq-N17DV~u7 z{A|t491&eVKjBeCgR+`~B766nqhP}0EhFoinzX}5kNP2{K%+#*OjlAOy{qD$?S0&# z3^^a=rHivJTxxMsr}EU$?)c3K90c;tG__*N03*1aTucna|0R}`aJ=yG83&UxFx}yS zT-SprL6Lpel~KdH6BCdba8p&&PCg%EsR=7b4YaG()uy)8s~TGDozIoD1YBDEs-!qV z$2PNlgsG!+oTd`l)ozWH+SdoXdx`qTEcfphWBT8(OY&b6kL@6!=bJVWJPc%j1tDIm z12?w7LI7W8(1$%}sg~@IDo{suR}fz21A@1kluwCS&z^6B0PY)Lr0?|dQIXlcuZ13Q zN=lxZIHTiUGr+EpQb?g;!$w7JJp$qyB@cWH21-h(H*en5H!vVwU0X}yj84hSd~#Lt z1DV=;CBNY%t*NOA0o6>P!DBq$UC4#sbsr)#?}&TV=B=^P0Zg6tYS z2)Ot_lk0`wh`cd}uRRv>^6lv-uuO11;N01>y?_t_sjq^x(T}X1B>JzhLGHvJzpP$i zOxEkZCggj(rD9%j(ui?^5es(!ViYL67~6{_Cg?3e@!AJyK+pLh1fe1?DM&qC+djNH z-@WEqCkp>g)I&8{aWCCvE89hlgJ$R9<_rad@hvKQ`S=bAT6Bze!0!ue+%ddkqz=U@6Hd_DosXJUckaK%{(l z3`!_T_TD(>X%GvhUWJZs=dTZ#J3Di-nUTIM1$|A*nmkKK3T*aFoLl$+lWHW^d>}-p zo^DQglc<8jmF8>dDmV&lTV`8qe-CUJrd%2Alg#AB^Jxvfh$-hH(Fb_*CfmKhVYaBm zK=qW#7HX1iHa0_xg5#H;#%x&H^RR_8oT{EEIFLV%v0)-siORuvX(r_>*aN$Qde~Xy zh zDq8Y3dYFo#OXAbvf&OAjMUoGP(*h6wh#J)m4sVyB3Iy%@F@-wk1XyN|Cr}?aY!Csm!Mlj2CGu&#^y-ttEd4 zJqmhM+^ZS7RRLtf=n-zZC6>e@PD$#>vOlcWB|7FI>i>^EDWsj9>kntmw5&?G`+8hXl495+VV z-m1;^?2&`VPTn`T?^$$fq9^gxJH_dOXu;HBzrjY<^6qe+Rk-rfaw40`ZzjmEcz3Tm zN+(NF(L}eR1Qx7J$DUmW3#me`4I9FamEhCU7;QXC^XH-&eWC%f`eXP`vHhy&a~4uG zw`A^#_nrSsIwCEsJ~|w+p%ITQtsaXIN@}~#(Kak0U6ub-d*nqb;lc((a!mvU2Ob5D zCe%(g=%#Aa4*qG$Uswrs#S}BJ?OwTvC`i~TAtfr<|mtx#-B=0 zb+FC@f%aqPCAsBU)X0a%!>`94-RwK+rex#4XIJO4^!83|eE4}uablC7l*ZrPdtU0g z9PXP2gA3EsMq`tykf^x1-Jjeq>ilNNhk|3dC$13Luz@Ldb#>a#+)&T^#zh($f<)hi zqEIfT{^;UuqR#KpOtN5yL#p5Bnp!W+V#Rd^Swn{pD(h!Ww4MhFCk@5iAlW>ix~#sd z3#sQ$q4A<;83?v0havkH@zSh zP$2ZECm|*V4>UwEz)-#0`Fj-ukDZd5ssg%!Z}pE2A@(0)xS-f36c(~h&(5}s((YAV zz6T--*~Wr}OA+=ZrDSK{cRHaBZ^vwXpbI;zy!I9qx5f&wz~%J{ETVT{{e@AR--|8C zDUPEO3W3V61p7==l9O+%7Q!^nlC7i0V_cpG`Rt>=zC=8Rmi1eSF5|zA!dDL9^oJdF zw=}i1471{{!Z(h6kOdo})yJUtm%@O5^yky=_@~`AEmqaoJR0^|#W;QNHuR=MVnAIAY`;Djw5M?RPB{IpY`#K1=U6gX-- zXg~c)h=AO0Rz#8U`+hx}qe!0h%^$xxT{4 zl6{dh?{|3TFnt3o-Tkw(=azqq|GBOJorqk>0q^%_dk3+m=AA~Nayz^ABulrjIOc2^ znAsBxmPm(sg@x-Z9ar#UqU)kAzHbtwyiPh9=ky63Q-{IcIB`L>FsfiViBp`pTeZ#7 z4V@Zy6NY~-HkOm(M-*{O>@1q|TYLK32;Gv`-cMZpG?F-(LNSYbno?^%~IWu`ha_Yde>}dxr|0k# zLxiNzDHt_4-bJpD317EI8{Izs*6_pxxkmt$yE9o;NUgi8^-@=qC5#S?TCqp+Pj!5dlrr%tk_jFo)d!+<>g={{~Bt`bO zH@^KOw<3Bua*Ngato`1MBjPq;)Z6xkR6m{Ceo-kZlwGl0x*?HS zl#t36UQy_*nf=o5bbi1`=m^qE)Jmhe7x5DuUdf>yOp)HTOxlRFF+SXh&M z&?y=J^+g$$RrJQO4h*$R7+0`>rXjQK>&UCvP^^0@LuCCVmpz@K6~cIt_dXE&Ud>6r zd61+InGQ*n55#DPr(-DD?l)Q8a-k8^jGwdo5UXC2tc68R6&=>Wm*Yb_u|{^{P#5>FUWY^P=_$; z$@iqA!eco=2dtzP9Ew~&dgarIIqHG@YU9~LTiS~jOhEP$%q!l)Dhz!n6aV$QLU3D) z6CyUFKAiih%8ey4H>V``JIqqj45+7_tFw%CrkY!mjuGASBBy14Gy9i+a{SC+;JMP& znvisjh8k3FkGlCCbcI-l?RA=Hf`%*Ew>LG7dg^dcz+FEp7a7R&cX#3cZ>j)Jc3S!N z%P?3DH{aSnIXOvZ)ay<+S!Ni4NyMNHkY*vM#>W8}K-RDS!@8mNhDjlarIl8eoa(OF!v1wbAItHzbq8A|edIlMFzu zhnpJ{F!x*Fe);|L)At^dKe_6P375ZT8~<+$Uk#t3=f7*qWuVt@BH)Sw!zO~6-B z?T3r{^#jPkyn`r(@IK`f#{RWkT}S{I4UF3PQM0T&cr9|YH3@r8$_mO>AKYfsC;;XC z80_0}fEF-tMnqOXV8l5ndQ&5^Gk%!xFT z`UE7$%b)yuaT=j&;OCLpCzMo2fKF-lDnb2TZ+v-{1@5bOj_f;|2r1I`jeouDY_m2m znOJb+tM9~AA#W$)sWtP~uhU-~Na3bR1A$=04C^B@E~%k^^P@1VP|&+Bhgk{I*xCoP z)IK6hvfL6RC8d6{+yR6lr=)BTyRWLb>Xg=?-YJIKkMcxgSMSFuf!_7b{n4n@D!Vcj zI@bnRA*D@hnj`f&xt!7UE8D9pTU(mj->W!S&T^VT%~oc_ZEh;Mq=&@SEW02@xC{tL z8$qhb^%p+C;kv(9R#t}NcN2j`_IS6sYJP-@`)gA=5>NE)D)DY|Bw3%3Xs z@orN`>GmXeHKu;&GDT6leHY>87r z8FI9(n>#&3+qragu5$s(UH`Bf!F$ZD@oQVd@2D*61=Bux4coK>&#g>l-t*?C*I5d% z8jpFRL1M@ZrK@BaCWokiy2#bO_-pG9)T9U6tSs9LhG~?FOJ5Y(=C6YKY+i`p$c<&@ z`qk^V9C#)2vXpgVMYnOrd$i}?I^qDs2K{>~tdFN+TOf01BXD>|yrW8;1^1C~O4kk{{ z|Cv+&gB`E^0B9rswAOyq6fM0l zukqp%J33+K`abpIgXDC>6G}0M(I9XIa7oWhp_l+pOhQZhWEIgtg@iYx>jtMH?L~>{ z&E6MwsYg}P2_-DhPxtGHl3X_3*PVVG3vToAI7WiCE3EYGjg5cluH)-Ag3`4*YV53I zoAf>dL$!X-cP!r)FZNF9_(7HC$RXogzd8KpTD-QdZZnkV|B!UufmrunTT+?`rHqul z$;`+e*?Z6I&24YtL3VbwkUg?Dk&!JkJA3b)^`86p{`cgGZr|}a=Njiayl`}y0;bIR ztv0L32;?d39jjt41YR}-CTgh|8F#(!n%@xvUnn`8{|JE|W|O z_ijGG4iqIYq9$@X7f(d?j*rJXZB2-Eby4GDV++ETCk)uLH<XYFuy<+@!fGUta0U*SEAt~e zDXB-&sXPh1|JF!hH{HDC@JPeytUKCAAk{B(R+YcN>NxY6I<{Cw-BieL5`prW0hu-O zZgp0ZmLwT)jRs*#)D%4E6|jFK>DSYa=R{l%pCEKo2&TqWbd%w!x)^qQ=3?e~JgN$z z=}h@bdDvie?CW;l1*^;m?9@T!2Mniz`SlJN8QK3VH%3EIth`|E~e0Bbp zN`*6Bh=!1dF!p;Wj@y%AsDiEHHcWDcg~zuwz?FALtnJTp)*!>%RWdf|zy6>_kp(FX zu|P2y>#%+C?5SAZi!sa_jiS^AY(-u3U$JCCBnHdBHnhT)>)C+oFS9?SVw=%WvDANH zgzM%yj^xLp^^~JNzMuVX4>PFZ4VB6AQrC-H!M(CM9CEh*Rk8@lq{e|TVEL#vgG%1e z5_SFPZo#L$!C!ykjiq64R29PGS66wMIi5(6Z)>JiH#WG*saToEBsf4F1w@pj@SyZQ zbxdTeQ0e`+dx5@mw?tC7fTns9y*D&;7p#TIlJ_)>_(hT)i;jxSOEA4m33o_hXQs!G z)XKh>VB`LbOTop^wbBHduP>>-<)bMPNja>p%_l$${uW^-6K*G;$Qm3cSYpQA?=n+4 z^=UD6GwbmCZ^?n6Rv4*wuw$SBGYE3ARkBK%Jsa*bqwV)3COV(KtCOb8f|H;s90!a? z>=q^rGwJ31{OQ94RBPW77>6v2zmka`k-T>j+&etXgo(u=u(9-Z{%mc{HSCH)LaKHa zU1Sjr(kY(*2nlgdc=k==^IsaE^u@^>EmXz|N>Czh2>N+P+UlOEyu#8WSH= z&%d$(N6#FL+;C z#ir^?oJFb~43N!U{0D~w_x&s_^8NW&BwhbF{MHrUd%Q_$%fb2dXua*A_5D#2eNWJ| zDBTQKu+Q>cP){2O&(n0f`8}9E{`xGs5R$Oci>C00i+RRn%Ber#5$vcX+i&3~Nm)*i zyyQD+;%W#tzj8})8DgD%ugF2AQiw}+MAFV^-A=@ywIzsf4jx{h$JNM1eY&BuF_N1V z)`N&Ao73g?S!H?W-_JU4a^lkHk{J2GiSdt^x&rY?ToV)bi)EtL#FqLjz`_f$}vsX8n64wgCHw1D)QcoFq#FF_>iH~ImOv>vGYRh zuD9z+HdGCJf|m%XwBmb&8j}(a@f560D#vveKQHKeT^)}j5YB8G<<31l zJ@s5MXxA%iR4AtHm_7Q*XG)X?} z{~1KQivx{f1>EY`3A5-_V}~^Ye!t$px^~R8>1+An&mG=Bi{v%Y7QrxgRO^avp**&qfqwOLKGIFUMOq z?04`je#SYwX@h^V;~_)M$H13vRQ7guXIte1%=GkOn$^2Y&LOIM86gb%*47NVuFFsR zHEUwn0}rls#|PUc-HOs3e;=@2A|gmo;>m(QZCq8=^=v=v=Z=7xz%BLJkep;EuGRjV zaLuZ#_hORFe;~`i1y%vuLojH=u1~4Q_4z6?CpR3QZG&;$4KS;BvHEanug>b(DC)=c z;hX|;6fd*=Q&=TMj`GrBP$0$Yx|_BH3k8yT&UEW9-u|yr{vfcfdA(D9GO>bEQ-Y)1 z(5^rK!X)|Y-D3p^!SOoxfJE2=9Y7hwaM5$mk6pUmBR961fy0324wm7gq!T_Em3p_mD?3 zJZ_`(IM#kGKU)0IJHy(4_}q4d)veAN&UDy8dvTi9K{(v7S1)vc=P)z?w{*gi@SPhg zBP0=}H8eqLLvAol#&4>%*b|_jr0i@Toam_<-(3C{uY*5+ z{ly@hGu@nY#`fpqM9V?SR2SdsSvQOEJ25{7^w-Ol>FDUlP*PpTCJJ-&GhYR>suShk zsiEsBv|uj%tru3VL>Vh}qemrWWB<`d?+ZqCJlkW@$8TmQ1H1D3qATguuXSjQwi0F}-_%vQtxEeJV6#YI`zzw>+iq`c+b>N-({!eSsvuNd`vw=tLgX&&5Mp_M7eEJLP>{ z9Ty!be^>F+tRrRyjv$>)PmL2~E5%5TdRfY{D#E)*9h3 z^M52wp_#mwQ8^+k6!;cRzz{w_@ugqSwt;qZP*hgf#Lm&+SPOy#TY3667xvayTe4yv zU$U;k{NYZ{cWwH5B?CQ)_7^|)~;Zj7QPCmH^^ypnwd3TCajvG5D#9<^3)7z zZ-xBZ{R7%!Q|ByG!3}LC&g(`Mt|v5MT`;oD*7D5FEWS$HAlY0 zFWxo53*nl$%sP9tR|?sd+QNC(4cSG6mP*qPs^+G#(+Uag4|#)Q8ReJ@;@5y_;^Si9a5=r6oIm5l#IfTmC`f z+mpoWEoBwqfx1m`YKiKpKCr;@g{tAh;d*P&--)cY!JWfjRVS$DOFXz4&h|Sc0z5=XRi$>hTdvbR--cbAc9ZfatUYDD=qR~JI7p>mb}1z zW`vT-t^LD+r$fB>-_E)yW9XlCi;{j*FQ=0E!e2`y0DI6R=V#p)|GR$>M~_SF9%>i1 z(DrI~@>|Wr&oKl9)Y3^~D~5la6kXO{hp7n^7iaG+gv4;st#rl4#yn$GDZA7>ovsa2|fCot{I#@KL$nU;1_)r68_kS({QM3v>VAU~g#rwty^UKo)t zujQPr-H+gBdcFix^_-9p1v5=C02*I1)-GYMEHCfR2NLtX`mgrJ!KG1$ckjpT-oCzN zP(v2aaO!9Joo8JGJBgeLS(uq!P9{tae`4}2fezzUmV7oKfQ5Dq^*R}rI@#3B-;Ill zE7NWE;yInR=Y%W|wl1rYT=^ij{M*j~X&Hk1Iyx%q46yI8sZw`g!Ot_gJ}~=CI&=$#!bhp#;s;M*fy&j>e9maELUUJb?t|i7?HDJQax2aI*Elk7zYY%&VFXI3+>A+;;pw>epq%Y`-_#Ee zh{!8#_G1#TO9>@|i59E`s_#;J80tg8TCZ@E$}U~159Z(HBzBC zvOJ}VEiHgxza%8Q5W`(vUA4D!zi4o(0Ys@>f9by47!Rm(nR~Y(^Cb>Sv3hv`LwpEa z>SWO$ws55jvfMX&&cLQtJaG!nFt7N)9J(I-&0LF;#wtYx9yYevo9ol&MIBah@dd|m z?VKv^`TuZSmSa`(4S3<|M_KtsjJ4UNjp%QyP} zX$1cTBbrLJ^9nW-1bCy$1ziCMl(F*eB9FHn$`Vh0iy-L?eUTY;3d``QcI$U}E49Jb zhRL~vb-Wu+j^o?%D)2EuwwD*M8hORoC>T_f5FD+4tmVe@@??ED(9M%4A%TC~00?vQ zzNAIjw~?dA(lKNS&jKYYedEazr0La%W>?PIC9B6}BT5p==*nC4zD}*0Nc+7GzqDc1 zSo}5io=hToR6J>z$mE?Y4cqpW4;B=d%Cx(F{=QTSHf3gLw~MMaiavFqK~0lz7a)){ zO5qLnz!1rpRvpd_emR+$-I8lY94>sm*fc4_vT|%(FG!MPY0mDfO{sE_B}lV7saueT zIk&+&_(WQ?Kosq_TPWX7!RUxPKu1SuiVbeVECRX*=Qd?rO3WM-RILI1`sJK6Q1Cj! z#Fz*_Wd6>7LQ=!iS7^m*)^GLddB&_^=5K7-YmVo=h-Zf_a$jN{|H-u9B@%a5;G`t# zLR42Lj2>&W_|o;TYHSsbC$TMNaZITVr#t3;x{pc__Kbb4a;cfD%aVEGkKT9F_io;= z+BPu8xh6CY_-WZc-4_f&p_on3lU07_w_WVL>{Z&9f+w#)h{+XfJKbcw(YKF*Mo~ao zJnbVT5<0a14>N$I+r`25l_`_j!Ut|AvAoE0wjn`5q&t>`yrGnI~Mz6DcG{;Tp3n`jd3nzWJK2&Bh# zo|W0}tba;yo$I%dT8ewhCstc4NZ?B+Kkelg6-2C5_BLw+5p$a@CS^`>`r><)=g8NJ zn|N-08a#X?-=E`)(+#Qo6!uu)CR5A$}QHy0$eSg9@qKG6J1AgM_ctqNv>x(+QrQ$h` zNi>tCF|4aF3=+Jq#`!MEVl4M>9z7U`B3BR%abnqB?@YUNvzn$jw)r&vBVrL|pQLIt zw(CA`Zcy>;p)6ytAD6Z8LtjBv*+wz79lVJmKDpg}t$`Ow`XrdKA$>jG2gQFiHIvo3 z_~c89OmIoF%zjZdt8g=`K>&%AsdGusvycVT!AavCttR``6uO@N?9GFog!q)cTB3_x z{ZARcgfQR5m?R_a!fX4v7=zNs8^ucog+GcnzE?K&p55x?b8d}YWh=ZI4~_p`*0*sI zi;fI%c;GGnJD0Ds!fw9ee|;n(@Z^@1H)*dLN>H#>FHwBiI_VRwNka%sDxLSWt*TD}r3mkJ+^EvjJmKbf(1Pp1t;`aK5 z993zwQ8qGRWn}^jow{N#aNftOuC&=KcHOLe?7bV^)_pf9%>!q-#(C8a7UDvh?FtB9 zhk=C7sLjNXAl+V8#?1C&7fRArnHrkFUqED>c1!Q!fR0BJk&Ja~Bylk@$SqIfd))Ug zT_KR=HuCV&vFjp77=3~9=x%^{-wO*vS5w^?K$5#IPwuk^S1vz zj4K{Gs`MP!VbH3o8@{5`ZA=|*J`b-yq3K77WpljrTEx_ zzsUFho${SAVREg#0;AjZ(YXg?L2r%F=&*`y+gG>zp((k|%5y4Wn^uH^DinZow!nfIY7a3FL7{uOM+Q zHcKCUPU6ez%8DI`iyJ3Mtc#*_@foZn1)c62c{%xfIz~%eK{qds6s;#Cv;51>2^S;{DMb2~IROgdFe`GEUi|h_ol5EBdzk8oA$Mc!F-T?B= z->;6^ETb!!g7yP9Bdb*Y@aHUym@sHmP!;zI&|qmwAorp0bHE>pI`*K*Yo=IUo;KfX zA6=U|euk_Kd2if10)NFRQ;y0{httA<;NL$Qo}y!KK3jO0>NIJl{ahZE&mRbLCQ?$r zPD1-fk`j{Y-q`xl~_jUGrO4%Q##S;(H2{7-#5>2)YPX$6CY;$niR!Jl2Q?! zGGgc{>lO7QFK+T+rTCt&LoBuyZJ602Xi^__=eKF-Q?{^_4omyG7mxfWwlIfQRe3qS zSXFQ>=vtGDuUt=3gym+i)z_1LwPb>dV>)7(06#i7Q8dg}(44c$tuaBNevYPmMkI~z zeZfpSzZ_@Bmsj7jTlh68oh~rq<13{*YW7XZe@%rvZ0Pe+;V|`=|7sg^x4cnNVbavG z%_zT9AN6#!L$lSt^~o4}M>Or@yOPt#Ylqb_)@yq>R;AoX@Yz{pY2O}ZFNFnvF#nIB zh)JD@^@~;F{ynjbY27K@RgEVP#MIPtv+VSLyA}Mp&LKN|F z&$Q}};D<5R=ZmS*PVqUzWoTid7FwJEYEpk^Hv(!5-e@fJym=qhwsH6N&q%5(FFvs_ z{BJdj74-w*TpLug_5<&Nx}1@4uKFi5Reylw2+sHgz1$XN-sZLD-sRyyfqiD);%;`$ zszBfB0;a0MG$M&2TGuB_?XL}2T4)6i3kiEL->gvHKgaH!s`kedjS2egnYgDsBUOFX zfeW)@u~DeoT{mAM7W${Cm77H;D%FENY_i4B=w?}(+2>oV`z!Qm;&SK;*Q!71mJE0i z6Uq{E?d)Lt1m`DUMJcF^N(VYlR#{3H3`U!R5^n7xvQEW0-&b5D{<%a}D7_0`$Uda5 zudZsBJAL`>A13uqQ$)-Z=YdkAgxO6w!t+;)=mQJc3`^Y>HM$PxS+^7sS2Ig3mjn&= zSA-blKx}E%vCeY-Jr(}&;WY~j%lSGwvD{iO#`t{tS?fdGsnH4(GQs(iDQZ;}^Qi}@ z?ap)GMy#KNX9*_rojhN@x)_x%>l58yj(>X4tQ{r=`kq9ADs941kre?x6-Uv8-#n3O`B1o^Ks7FraS_+wl_8;ySd) zbq4}V>Vm<<#rJpCrz3)R0Ps8PeZNj-F_BRXjQ+ugLUae^R*B$PwHTd**FULGArxX% z4u3p2FlogNX#y`f#jDN32HdmtjEw10upKJ1DYm2ArnJzPW7{TjAhW_aJ4{F8iy=a?E ztz*Am^zIAZ^;*0M5n-5zeR)q9w8l5E#QU|$Z7;ln2E4I9Pzj3Ov1<*bx=pXRa`m`7 zStIIKbyH|r3mM{|;W?fSmYT)qv#}UsKwEJ@U3mbRIf7P*hCwF<3e}4jFL-up7TGsfi;FjJHU zQsGyggI}cQlNKev%`2~^7@PSl*$H&rx7l9W_r8HjYiw>#1H|59vuAC5^8+&R1@d^U zR!BRw1%$I*)uUDA_70CuyB*l@M9O4a)Kld#+bNm$SSef)2AxtBMViX8x^Uh8M1s@n z?PE6k*^Fha29L+m#>@X$)Gf6dy;$IQU0Akw9@hRRm&0ystK@w5kWtUfj5>ke(D<2Mi|?IBfj=8g>IlBX$9GOy*N0I=$F08oYzqbDpsa`CLfOi@66GVA z^X_a>m)kY9LUk-*<3@e~*HuF!%HQhQ7~lit49=;OFxBIfr#4jt9anl2Hclab-vv~- zLnH|f{O$rFibj!5cTW!+BaKsi%4)JbMvly=@qI$qrAV}N-cMQ#@sq0}c-mR739aka zDUQp52-PnCP8`xcGQdTvJBSSEHxFM15WfhEC-b95iAHNw3Sv87y3+yfWzF~xRJe}F z*UitI>b18w!3LvTEt|rnTsb@N>(`otJQ>!#Tt`g}mIGaneG%o6!F~J#7oG+nrP&sI zAr#6EW|I(%RMj(1Nt4+b85xVkU@KJ?XoImNWfOlOFnEiLIv3?5mdnBSHeuZU$Gl|p zg@yZgmweROlWX7nhtV~?|2c@o0RmasI?woJkcFr`(wbZY&UZ30O=H{8n z4nyMjPgPOT?;;m9Bl&?~+G0H!83qdi+yfr02YD}k7A-Y*uIBvQ4%;Ms{-C^+oP`@V ze9+R~&fCgBOWUqs=Be0qNJg7zX6(x}zW$-6N}Hr%q4*qia`_OJGYlE=u@I)MWxBoMiJ@NNQEJ0_pJ;LPD!5{}PPcUoW2!{MT=*&SI40C)^ac&LxwUlu zT?+iW{W|vSsqi;pev{Hl<3ne%KSt`tj(!5s;&X)e>~zE8Y9c!Hp(IOGd7widHr46Y z$lg;}i6$V{rjJ^%@t6olaLv}$tjuij1YKxaQJvVt%*L_Ctm4o5LB+ew6l-0rm1e47 z)-T%?obM2eHdXbeWn8RTu4MR&#aE7qPRe^EwA`F=cQgHgvF+e`D?h@HZ&qnhLY_1{ zlV{tiSJfUn@JZNUT2r~?iTmeZ3&BqJaHSoQa_Ol+Dv9Yz#-OcLrp?7 zjg_%C9PMGN@`Wyfi`e|s5{U74F(-Ut;y2{t3Hqb7G4v0qM?2%CKWh_E6>^7_<@kq2 z{K0K~r~mbAF$J;Ro^m5!c>QiF_OTDwgCvIZtr_vWDx;mJ?6ety!JUhzt6E}+p9wK3 zn^RTlcnZ@OpX@yBrY?L8zlb8-DJz9C86FgbX8=tJY?LHJ?SDdUCS@PymlKTv3zXB; z+>q`A2UW0OZ&JVI5%mCZ^Vct!-isE^2lkJ|a9D?KN+vS<>(}4Wipru=a9Mg{#raY0 z(?UUsLNafPUA+{e*$kJ&;Y#I`L;5$La44u2d}Vp%nQz1;>{sM8%7iO-DaAYVaZ^3x z7F2cH+bJ5(NZQwjr&<%aV#{WVkDy02yikxhTvfb08m?mOVQ%k=rslbF&`j`+y8`Tz z5%b2hx{AbRQh2^{oQhuSkF_mX@M0(J@-4EUQpTpg1vlrIheJa(C*8E17uOf+a&mjo zn|j**@5x#r>)9Ss*LlF3UC)(_cKzDa(sHDb(^}*Cawvz;BWJQTBfI&fdwkxnvr5+} zyFTk8?%QgR$#{|rM>Mu;+f@ImoD+Rx)j{LF3cm%;W=o%8OUjK}*%UfiFU$ zpaz1L2Vfv8vpLEC3|w)Y2M2cS748gbMJls)J@iuIoBcPJr>_G9UyB+np_gY|I>>Ym z1uT_GqwczK_L?_3g_(Faw)t<~;oaByujM;)+;Du`AH685o}~b761);sujsjyXWxF3 z9*z!>RB#zJanpC7np71H_onfsnVFgKY~*HoDW;sET|d9)n_hP(;swhJhkC`$3)@;P zR1z*ztW3WwuPn`n&h833esT`$hVgM$S$?$}Ys0x2N@0WlmGh z+sa)YQLKrD$!G140=US&Cloo!TyI!-z4H#z;HsJ+{u|7;Ni7cUshRhI!|mIAJM(ed zd>p774sakH>G0u}A+FfTqn{s`Xpx%}kf!a}8> z-{B$5Wpu;aPKVH?j#oq)saL8oGB6keYu!Qe5{0qxNFmCKF=)H}cJqH@P4x7~2S^7G zN^gJv7|(lJ8XDFMDC;WCLNt}mntT_v@tN>430KDL7PkEWbi_q|QVqBl^eNG&A5i|j z4kT<%jKg=o!K=XIbm;wDs!^0!cm1*J9G~&>{n#MJdFP?GzQRrgpoK49y?3 z8QE52clm(k*p{e4t}q`o5ZT=8?nqw@{7D zseM%5NA+&F32CXY<$azSJ-#bFH7RP4vs75k$sgDwsUA04xM@q)J(=BKG1{;ki1xG7&MP?wReJ2=C<{rtZ(pYEZ zW3-JoR9I*TRxIc3 z;A(A||CJ@$MWk>bfKDVVk9&SHWn({>E`W|UkB1THy7o2jpbbZMskHND4@8S0iST)4 zN_i*8@6<6O67C%eYV2QPCOPkKy9X<@FUEv!Z`_fP2auEg{n~+f>_CR96E~{)n)Gp` zOq9ZQJ%FgJ$-SNyQ3+*6T5?n_e(U1&?+0=cRVt`z+25BHm^ZR&uHtG=7RCywu5Fc* zteSDZ)@+M*H$g>vH(F)YvyAI}!|3%{|K)>%h)hcx(qyA?(FX)K(uncK$toZG2b%+- zyFw2X)wE<<&Tr$wsabXS`7#RZwrr{hj( z-VP!0M}H|-wl(J#VkcR9%e52ZTSNj2+XXhIOlX8a9JS~E9shA%n-IrEQ3(R}&%des zRdq+%=gI?H%QeRDy3X+iGd35J--rj78_UBufCQw|T0KIel1UT>d+GLBp>9bG9~N4G zj%RFjsyDCX!9i~Gp$tRgzNoRWaambrX6vnw5b}DEY~TOE)s@qvFL88qo@@`ZrOET+ z-p7bad-A+#n!T%1fdKB(FLRALe-Fpc@=At^Z#{f-Fl^%Mpibs>BJ~L)n%RbxJ`DhJRqs+H~q8@|fGsDlnZC9`axC+H88PI1Lha)!lGz+1zY&YX1P{i=r< zw*06(!^O>gQ4Pm9Y^&C{vt#bL2>ZLDZDm1zIR3z6$6&j(KFR2MH;gIX61N5sf0DdL zk3Dxz)15PmS*|S9WEuTq`$9wlyY{cU&!*F-7^8zl8~vkyMl#yzpKdp)Q|~RyP6^*G zLNaP23YA5h`S%@&pvijp@L@u7axeJw9UR{_KLDwow!FMt9sC^eI&MLbZ*t_q1LbAz zSsC3S^7;U#E>fX2QeT0MMsHpA^e_n6w`2!SZkqr4?V`xz zx&r!8%o%P+=(7h#6MJyQDzkNK)D?3@sv+O4Ltw-gUAue4FKJili;?njXL zNb9@9)12Nfnbsk~nOY5ku1%J+C#1-)mwT}!Jrec$w`>WGa)v>3q@Qy-~`c0glCdD zVqWS!WmVnosfvXL@Dc}w*lq1=&44$3>-={KJw}a+Pi1t9QDhoLD$+9!C=JId?J9$& ztoe*yWppghLiT6fca6lw?=u35sfnJTaWIv2xhBNm-eN+PQ(1sV6j-PEZc4pos--FU@7)KFEs)p)CD>!O`e-;9hj^u3y7i zM##u>l)XinXJ}sci&j6J?aim`l(5w|Wxh`z{qRA*r|kkz+A`#DLx!I%%uALk?hAGj zZ;26mDlM80Ci);i{d@A!e176(6j`1T4GQw(>;I=BhN2!v9h}jTck81@eg5$--1$w) z&To!d_V)JPP8m-y0+ru%gMRotr2YTp$hjt-i>-3+11+aJJ53kXyi?YtM$o)@fE-uH zl!3-27-s}sOJ2zMP2)2 zUw0*nk{Y~uT`cNpL3m+7Fp9rMwr(L-?A*&fo@ABJ?E|(DGgfA-N58j6u@f00N)&0& zyX&%R61TEQ%>Ss%XTG*4IEs_G09auzTGzFqM@PYg9aff6XYZq8QaZBUXWsQu;(`>RO{FM-`7pM0dM1-EGu(cg)4(p}!J>MNFZ>|dZiji>k^?l6K(Mev7 zlBw{xlWYFZG7++*V|(a=AenV+mlTV?uH@ztI!nCmaB-OkW0gTqM;|9fVHM@(k9XFh ze<{RBEbOn@|IRgy7`J=%TwmF&+tFYLKY3SqcP6=S3b?;4<0^Lt}gCjY)&56vX{ma z%X)%&T9q4m_DkH$Q2OjfUr21fcGFp$8S54Ux43Ucx* zid$DUBJehG{cnOG_R(>YO?HEG$q5#_b=KLW1!6A#8GXjttUF@L-z4?x(ND}os|7q= zxnz!rp?^*bnCjVJ%)p&<6`MLKAHaR`!yuw^VpEe2_ob`wCN{r3Q&QqFI;k?yN`jB4`|Wd3oq$=foI+39+&5nc-Z)@mhA`|Ks!S6`_8F zZKq~cQ%BfojgzuDoL=t+K17{$c%x3&QZ&V^W^_GaR}Q+|ns{YJKe48h)4P9)F0M{} z>u(kH6m4~o$L0>dyAa;v>?Z7tT7d=CML}=LTR*^$A($eQQ>-{XI=cT^2m@K>(!AYo zJ8!9aNm?-^R5xR-9LxIWR#!Y$i56M0NE4&(TB>NDtR(c7TgluNJ^Qz6WnlDOBDq${ zjKfSOv|dA)@KB}G6!B|XxFp3kTHow^Wpb#+MG_c-k1n1BE~w9*T7;J9@3AX1@G$EfH825_@wau$!AWTerV zr6_QaVisz!#R>eKFI12BD3KPW*B@niM7&Q%=~PB(mb{s8SE6F)3eqMBWf(LYWv-^y zPxg8e4*yJlp!e)7Zk(=gq~uGPq;QKhvOCf8C!Vcz<|{DYIGc%zeH73NuA~sn%_Ey8 zMb1!D)()E$#t$Ojn8DxH9-k1MRAA=?N4swn+Mf|&xPJ3toA?XSW!IL-bx&wNh`lfO z@usdRZhfTdA!IQQ<$49bn^tx4Hd>`d+8i|P8|>myZ=nSljUxVGQ)Y4BU$C&GB>m)>qhRAv!r_HzCe^s_fwBdIW;XTPRcR?p)yBcq zTz4xN6XX;WTBD+{mG{+DR73-Rwh(bUQ%aJW?VVG>bn@%g`C zG5(&L`$^aPsTG~inKCRuE463yA6qFuDgw=PHuhG8eypW*3Zzf zJ$Q+ShlbXiX#Yb1mjm9Q#{uD$qBv_3$WROC8Am+GSyNB7ZM_fvlQKGc8%IA0{IShG z2Um|2CZ-(-d`2nP823(u;7aC2y1AP-E}Uv5szGGj%PN5v1SW@$wNoe)f;cP;;~I zq(;g%WJaXmXOY?FG5VS@yCQABUq-%{BGoq)@r>r|AL7Lh z8!}f-#?gqFBEDA!_)BbB6;{F|a($9-jij=|;3(KQ!jDwbj?Pnb{tvt))@dhd$<~Z1 z);3z_kE*L{my;{P#8~kSM5KkL-+?AolT#+6$n{aF?d~?w9gWiJI)OzL0h(Ra`qS|| z#tM+JQVPvJIV=8Ae9HLB);+cii|7OJf2~0myUl4xb+AAqnMGKB$t)+O1Kx7lo%;J^ zpZQU}jg(YWn4t-SC+86wDtLz#MS8Tufcvp1t_?Upubt+;GGx&Dj_lcB#R_9Qow{J2 zHER1~lPHC$`cKmhx_KrxiAC#@)2FQdM(B}|>~E4Sb!EVH7_(9F;zL=9Z)=A=HJiuK z$yG2Vl_gIE?xIz!|6>#Rd>d6#UOBP6eZ>b!jNb^ycqLvU-+H*Bin& zg?KO`dfv~y%Y+>ym+#^mr9*vHZ8GrkWbL=#^71lkQXk^#`uCp(SC(7t0(*YrI+Wygk@k)@i}ucz(@x1Y|b zYTe`g_i5{I)9cGszL=?c24NE1>kHjrqVDCcurOqmTQ4|1GfY@8h>!GZ7w?~%s;!s|1iZcKb1_NTrZu?ql8HIw#Zc5Oy41PTz&v+p^D%?JIQfOq zPaPxoVXsKWIl}aT7kso4B3))Bfi>VfWTE(Sqk{S|IQW6Fp<(Xn1td=QS5&Zqo)eVj zxw$&lNU|Ap1g`~b#dC_*Y_irvAN%jY!ZNDhzFw6aFiRRK?ETgkzKKgHx z_smu$G)lPX4S9F>-DPLbZKp^AFQtDeWv@ilRd{c_#8w%amBX-|l^-^Mh~hG??Tj?o z^MJeTebfbkm!|k6+V$=iRx?;du5ev8@JK!G9N;@rXFs~QLhj6P?Ra70;Hc)!E%k-T z2b8OdsqN_+1nI@m`cLqLNGiU(DEVm0+KsGtAWtACg8Qsz>Ja-2?ai0%XkILFjJh%? zy}iAmv=i6c&XP4H{+6dMo?5dEIJ#J94BC}4+6AnM^vRqiUpH{mPnz9`yr;_}SXSvq z3YTS4I0FGBUeH+7)Mo_G(W`gCVX@0`{lm7W-1~gvxPm+OxKCQs*><~wc%-a^p*j%0 zPmc_H9_HoLwX~oQ=tSBe$jsQ25~8A7A;jwq);+uP#A9s+-b5Z(=3AHf%!JM2DEu@J z`@zOnB4La1oQZ#CR^opuT~^uqHvsz{zkyx9X_sl}u+K*xF}9X+#eFm=#0iOrHTXDv zeSPNH$4F;ELHIsQ9}XIRsg9}zH~a!vI^IHP#_<9Vm{;!c_hCH|iG+)r-~7CxeO zr&yy>Vnz&hM!dCOk*eG-AlllQ%#7vd+xq^~YxL|L?0GY9M%o|J+%~Aaz(5F$w5%)A z518gT8l?Sq;Oi*wP2FbkMHjp8t35s-o^S2J$0VDe`0{IDz(jGFk=A6TKc%4K56lMf zg2Nf<{)4INjO)+0<-`ZXUb-$(mP8`w93Y;589wRjaaQ~hbbvIr zV1>&hG7r4Cyal}4+B!d4ePiDdx)9Q!SBv`jK?^ZXTim0AdqR=95>;Fv1x=~{L-&3C zSRITJ&+i2+Y@PdkWhx+egv(9mFh)T4KO=jwzGQ&&g@HMfn%bOUI1cA$!^T-b@wgXZ z9k>V~DLo@A#hhge!FejEA&BF(r?&(&>!ILDi&`XK_|B|YfNkURWe8+}pa_$SKo{g@cCQTNP&^ z#cA!`Q#pj-**u~MAeG1OW$*jahURlOB*iUSTBwX8!4g4E_I`QNf@oL8CFynr_d8a# ziq$dVd!_j@1GY7uoG1zI1)HN65@DIk8o>@V&G<(vHE&2_r7pvL3FKpHO0wS+W7ZGj zPznM;3b1eSzdLviR}va~d49pU_Y{lPuoA23tG0(o5UbR!+bFm5RbeE8=^cd|o1=jn_fWi@9B z!gEFf69qL#)``?m%oo%ie8CFR4g8vF-dx}ERY!Y0Cz_%bke>R5RRs7gDm{Opwi5&2 zcdp;fTl6z!OHJs*acRtz)$voKiw6?*JKvP{-)buIEbEa1(2{p*k5|R1o)gU5SjL#L z&QQ82uKG$RujYtrR5ErxV92~*s2FZ+?8cntaUfo&)xGf9@?XGLBFqCgu$EyxOG0$? zPe}K3G&8FLM{OB^Mkxu21<+roP70lCsd%p6PJHdC)Y6okPcUI^dnYbYZld^ai^Xc{ zojX<2CG)p$l2cYO)bQ89-Z%Vbvj_@nJCze5#J zJr#R!Yx`Nuw_1DZ-8tF)VOh2eR0(V#2h+8bzpV>iZn5|;?V=zqUD}J%wF!q7+HrU4 zBg#ZH?@}{cJk#}BdU8TT^;ceu2qY@X!Q;KFVD{^OX_AwR%IXAw}(<~ zo>C1Y*~+aPaR@$Q@0qS(>d>B}ZEjg{3u2sE%V}cvvVIxzSu$Jr?Y8#toG6N3rG~na zQX8-;A;#qikm9ho_B+#z>G|3D<@07u+C6_+lFL$wEVAfGD%VxX4^=WkISii=iwu!( zd_4Awn7B9kCZzIw^6Fv&3nRW68pn6)MF*{o=But zx%AT3GT)V@hqrNzZ!Q^sipxS!12BEAFz;mlH(lths}8Hor_VojBM597zm4#>U9 z!9jEbjRGMg#%nVm+Zy%J`I%QfbKx+^#lpc64`E8ss=tKhyhg9Z_jU%|zKsu6FpP9j zgoK2g_i@a92P3>U=W0mv1E-zou8a&4{rkAz!7_%GQaz-W`dX=MmRC*Q-YK%R!7)+FuVl&r2ZeudR+X>kzSJPH zkR)OQg9o71xPqzgFe_SS!Gs=XVPnG_c|D-x-Heg4uKZb4P_Vc+|C8;-MidzW>>eH~ z2`0!|=mmbm#L2cIwQ6f3C=TYl$0Lf5?)o7o1Jx81G8lLr0>*G2e*}+3VK#;h-Lpk7 zzaTvw_E2~mpoXm3)w&FcQ(k3!%!p;Zto&2&%!pK+bhR$=(0t%GQ||K0{GA(h0`Kq2 z*2JVpUSAT&AxMY|c{!=wQ#Q>hP^}x5<21eAT%Q!*kByB*-eDM>R&F}zriCD>gMsTo zbUy*8%)qYqEdcGyN~Q~pOMX5SNd|a2ykai|MMUoG)~>z`yAauIR+%b>f%V*-r@%n6 zLMhLGQ4OIuNTB!^>FCs<91HVi1+xuYj8S|lGMLsF6y7ereGjST!=|Xs%}v)cD2tb8 z`()zvJ(XdBdNd*DZv)Cq2U*7XKE3GvxhE=8b`5$-`_95_ ze!$x(ooteirfIRWdf&g-wWR0S%hC)~pR}U-O^AX;V@B^2siwGpTd^6F`FqonfdPni zC$|BrF1ia}HSWjaVR`oN+@OhVE2woDvzChDrz`xQPo6PPF7N-IB1{Jwb393X+%TBi zSUyIrghSD$s{Kjx{3pS+spOZ(Up5k@5<}euuqO1FSnzQ)ez|NvS03G5ET#nK#5&u` z2|KN)!yg^6fP_Gpw{hca;S(otY#bXC6k$}mNu&RK!v37D!ol|bVAA#@e^N_kb?OR& zXfQT*8Gg$VDI9u`+1?K_Ub0K;&OsX|7MkD(d^#sE;fHj7wapr~yYn=~a2t$`1slgl z=anHxcgwZy87nfXehXKtZt}kpX9jBsLXc3`{?$%mT;B8mQ4i^Vr1Hx;o3Fg~(FMDY z8O`sYxfc+qDQouu$E~7T{e;b+Gkkf%B#j6U=|cScL-z$3euV-a&4U~9eUwepIR)ms zqd|sniHFf$ZAsR3L7;pt#;WRc+04C51NZL&421N9+!+qxZj4u_R2t4^GY<{tz#h16e2%13@bV6Pq5l@}TsWAT&3fqQ3*iFPPt)G}YvV=zojj%hg9 z6`pg)+0|EK5ia5zQ-QbNC8VXpw$dSX* z$h;DyWDdh!FM@=j@P$S+5X`qaqH5BPi8G(Kxj#`Ep*G+)5(u3-p*A=l&3Il!l#J7) z{bF$ur*&F;n4~VcYn(S?a3}73r-Dk_>6>SYz(Pg)n+GK4TVhWFNU}z$4|i05K61*4 zjWRmezyE|Ced@%dQiBuxh_i_CpFAsVJuXX;VvsHo>wSJ`pH2dt`_bBOv>)C#7jw<| zZTmjqwr_hMGB$ix)pmS}$DsAAp`k%n8J*iF4Pc!i_$_Nc{JcBsc{&3v<`Ve)z0X6u zZ&^-O1i_3muyW(J4EHa2zfV}2ms#uRq>6rOvOMN0)~F65TX(3f-MElW-0e@TeotSZ zQQE9wcGIZxipl%;0~IBe!+P?9=VrCzcjTA0Ttn6{(af0CYdreyGH6sKf|dUw-@Mjw z)jBHJU@Mr7pvGt_V!Fz{TC^8CtmEvw!bVBp$HG$!)3S=b9CHf`B>ZK9cwN}vyx?g- zy!so?dj6PbHFz~{?e#D&t}YUNB^AH)&Zw)yhmukQ`rL@syzO(?TDLI=U%*1U6$qJL zHnS10?t$k=7-&BbE4&qnf1Bb$>LL?3T}!MtXX-hSyP`)%&;+nXwviPWX^lCyiGpOk z+`_^(c<|-4wZp(gvIHQBd#&5u25r zodiEDe+pS>J)nr9e1terJwRaaSotRSkqb;p_MBSa9sM zz|j^2e8#nG&{cfDtk`E^0|QbTT3UH|d7sHijYmXWxZqwTM)`aGe=MB^RFr$y#fK0O zX%Oiym68tW21yk~8Ug8$Qo6f41pz@?8bPF6q(r(yQY0j#zBBjz*3xykpfJwN^MB4c z`?vS`mE8mH3s6l_5M5ne(6K6bc{Q&$dcqJDNo`+$#XP+AvAnz+T3VcpK@0>$EbKW# zjb;MzZm(1nKuc- zIyDSlpu2sW~1Ez0`_2LUclm2(-I>N*i<4&vzMFLO(S< z?A=Vhf!ez}uR2ygL@UcjDkEip_lOkp-|@Ymc^jFxO8AOZ=!G*O@0jHfi3_z}{HkOk zAGu%;ol#}eFvopj8be<9BvM9UowG9Lu*+IivdMbZ(Xr7VUY{R|H#84!=o*CA@a@(@ zWB5J*&x|=}Yr#(gy;(Wp$BNxb@78qDz3k zi<+etV(~~apv^x3^H_5xkU_ISY>+P0z^uzc49F35jB4tDf60u@%sSN7>yiYCVd87d zR!v51k<0C#=pBToX!Im^)}`$0q~gQG&AQ;()I4Dw;K4B$<>BRxO^S=&v9ldAl_D{A zc{ALs%fh52FJ<7yM^>shisI#wBwaWrmB>WKftPsuPWzTap1lgyZj7*)*llLH8(RYP z7370{1pkiNNAzAUDU!fL4>j<)rK6O$L7OC=iBHpM1n z-H;5K&9np8-}4>L4QG*~-#U@`$(O=&2Bzlu9h=AR^A%8dMw7N>NSIT1)kxNVwg1A; zeFbw-0&Ioh`e)=?kEgxI07TtU!~b`}`8bA|BC2iaSUT49=)S!u)HEX&RrIW)x0vP5 zwp^Z!S;!78GYn2f(dqWWqaR_rPg2BKG6_!MGvnV1TZi}3oTXfa(srmOn83KZb%UB>Gfdj0!k}NVq z5Y5btAU@ux_TEHL&#pWj*-mi>1_Ht^bJ!WtHs-0Db<4BC51W4r^))WWY9yJTp8{l%aw~IMIafstsWU#v|@MPodU~$6-7mD@VY}=zr=el@AtJ1WXltV zBe-UhqBm90sd(sa_TZ>?hYvEilWcI?D9&Ws&WPU}OTlV5`1XX#Ew##P9G^isBL0&} zbtomLqMThFo_R=Qk}i|p$=(ysdhJfafP~>f_M7-$$lGjF@|R*!reW<~Ug!}Yb-%|3 zcSFZp!)Dj}AWg7;OMh?!5`6B$s51gqe%&T!Rl5jKILN*c(%}byA~AR#7y;9yK@PHg zGWhi}27|#?vIHf27aIOUWZc*yOuKisK3xH|J8d18o{n8O76uJ%sctZJTZT1ag0fTr{K_C#41%d247{@4}!6=q^cL= zcOf$-$XBBaW%#=A2QB*HGC^;AB(829r2OA}_0F9qlT+}B<(3+mH%V?K87Zf3)X@l`oq{4XA6z8=NeDgP*2U-}4VXW@=O2yv7EP-3SpE z8w>TL~HEjiA0n((l(tcj`dtCxSu?*<~RXzwr)rK7FPi1WneadK9DI_Xqa# z4`;zi52i4QU_9#ZqQ`~H4J+!t6L%yls%U^%!II0x3S!)tkp5j4QV&ZW?+F9tPp4cJ z{3pqj+&ux8X{6#=QR4=OG7-D$9qhx`i^pdpB~O|pDZ;wpLrtwj{Ddw)xV2jFObwUR zV8+KgbEodIac%pp<}>y3Xch{-S05tEFZ@uv>f=Ng3isap+Sr-0Ri73Ev)=wk&ra6% z&z6q=Y3e10{GO}ga}=5t4_dVqW=W|hzmmkd@pdjQq3kxh(408j1mEjwv&@Q1XvN1b z9t-5rNwAi4?wvyJmu&Yr1L?vY8k^ZRH4zD2Cl!}jimt1Us*}pF|oJWW6n)0j}kBXBEEQ?2X zxt{jWYRGcbM#)n>{KOR>KOM-4mx9@t&qT7m_HoyY2-p4A=$O_>Noj~kw<^p1Mz`(Y z_6w?7L!IOiiy8(F{u|Bx(=1x^>xeIdC`a2*i^&;7W+Kde2+;l={g|zHO}VqA_{1p& z+dtGj;D+4U=PhwFB2|v_`}S`3;ZQpS++Y?v;Gd{rANXj;o!h{Tf8(w2Rvna;Vk!}_ z;|y2;y_4~nVtqZe!uT$7`6R!f02%{MNqEEWLjg|@-VVjZy+A-QO<+Lwyv19rs;Y|H z{m6m}{913$ckAQg;|~U?FZz&Q(k^}O2X>=&lggaRN}$zNp7}t^5ec*vLJ%$a-icFY zNQN_zQ{M#^0u=q%NU*Djt)VFk!Kr?{SCOB=bF&^`kXAbA z8IN;694P_^2wpLP9RLaQ_RI~r|cX9HkJgwjf^^22mzWfiU1`_+@CMfm#nZdT_Ojr!u2fe0W zuH;B=dqrLM#ta}B4U8a{w&-|1 z0;KHEeP3mJkplh)Y24j#w`mf|U%;(mU+HCx=>&lD&!`HQF-%jRe=6@79no+Mu_@@w17g{FRs>Hs#cM3t=9fQOBQ~c|TlUO6pYW*IV9yXh6&P@C4_1 zsp}+_8r9oV`cI;IAIq&87+_M5b1qb+;7$Z8anWp&V+)Wf&1$AGkiqWvj}~^@mTGe3IF1X+dRDGV$^f z1G~Pwx@bh34+9hdC%ZE}Ac?zt9?xIcdsSi3{DVSs$ssi;D9CCv0=FL$zp44pz2Zc zRJTS1u7?7azaj{pI?0xwCdrA>w^VKaPDYW2bgL~`IDV}CWveC#K+&K;VYj$Gm;k{D zZuK5CUr=s`mwx`VV!f?$)GyL(Ww>{U2_C@Z(&;4ci{cONd=z=(O9k_$+ckz-chxm2 z1WQPd{n@ID=X36=7Nk^DG#pNlAJ#KKX-zN*Me_9ZGM4rzbY zBc=FB()FX`tf3iSZ~8{N_i(e^c4ux``^mop6@G2^?< zfY(Qnif*UZw2a3yRW{BKTc!oOH|sop2gEXTh1c?JoitY z{{QG2uy?TEBIiqkFt!7v1B|K%(7BusOxZ=~R(vo`Q_mjifmxOxv;j*iCx#uj)Fnb$ zvz&=S@TmJoN43CWz$40=rhvQoy`;Pm;!ggczWSEhW~Oq->Cz+Mv(Jz`u;uQ}{fF9i z^D4%>O8Jw&vt&u{+O@=K<<)(E_>v}kdnkWrI%kS~=9$K%1L@she%qj=kjHo#wYF$3 zBwr=cNTzLUm>v(lg6OU)?QmYM7? zRK`FYO#OWy#dsS#^POPIUc8^(5ZXzitDI-~@6B)FexJ(pHb}(?V|`}?LqLeY03bKp z*+uc@ML;De&kaO~vQuN7NZN-~`8^{7En&fC-_M1XH_FiGA}$s z#Z!2$OUuiz22v2Iknq*yvfYjJ4Cia%EmhtOU-MDeDJ41k8%~yeF4X>4A{>NPa^B{r z%d2mvwSqyoy-c7l|M$L(w&d};jJ<-8-Rqg>axYgscypcKnrRCvT^MO6<|=-b;O#Gz zi2U^OMsIU15$m_mgQTJ5iOIYN2XmI1A`kn|-mXOrrN)g!@!s(eGU?p+Tn=OPo}pvT zk%F_Sx~dAx7hX;EXCa3n%a&6v7Jg4J(RsP>)tU>e z8P1Kf7drzw7bgD%N#x?ZeS8O=)hCr3fMMeom%)hTtc1-%*HEF?;B$8Q`iUOlTgh6< zW{tBn;roL-3IIctcwV;iiO+v^f3-$aduh0;wepA%nIa zZssTaXhCX}^AIwre@6qA;tRAtRN|gOa+;=)?{u;C4b> zsh#70pREncf}hwjpGA>270`$(O;gzkU`kVH)n1-Z;FyQ!$`9@RQQnqZ{g!Gf~pn@t$(Sv1CoDdW1Pm+5+)r5MDx2 zhKD=CuEYc2jB|a-0%s@#E~&?W<9*wg52P4-p`#*;mVL_8$ZPWR_rdVW;bZ%72`=pr z=?zK_S2%Y6rK6P=PsD$Ns-AI#42gM0x!YG~1IK!|hVh~7-FMvfV%_N&i~-cz=of7S z)|9moomak9WqE8J7B1WbI3*L#*4Y@0%>7<3vYR&}Q?jg*Z%BXT~DG zUU-<^JS>SCof~4z%KK!&6IiG~nLTgw(l$?%IHF@0l-Vzm{U&Yz+*FV^Sv)e5dnz;A zyoMj#&{FrHJVui3n^@A{X^%ux)x0=pQR*gig!Bx4@{_Op`#6!|qm`+lIhRAo00Ymz z?<`_i@{+Wh5`YVx9j`)gDG|XfzWgfdzZ@?&e>yCrpXcpCA8ScV0wjAQk2jIq&uKo! zK3cF6?0tY8G0^)wWZ_?E)bH^W2M`!9U?~0uj6O&s^)Gs~|MDZiZNmc*^w12Tx8`AP zomObF?njIMUd*un+80L5`nu!GozE}--GBqaTvZhp(i%8OZ_WK3n4GPN^lr)alN@k* z{zvn0zRQ~pl5WdU5=YGC<+`v3R89fI<7!FMo$Sw&{%#xW{rVRAz% zFMx7OgbGw`ND3DGIn*-e^SL%-KWm@Sql+wD2?m}ng*eWvZdrvBvv%92z}@9_#A%aQ zU|}|ksKK5%?FWdOn+~t|I&Dcc+UT-xD{&TB^*&%@WcvfRtBvP3-8zkQi53|n)F6-F zVT{-0c#l0-qSRJ$#IE!=Bq6BC6F># zoM^(E(o80(_cm{CqOPl%prYrmdAf{ya!(PlNl71y&d;B=i}Tb z<0_7}+c^K3*}@jOEnNC9%s=qFvcc=mJS{t(WDfiCN-uRlpmy-S4BGA;{o`i3>P4c(i3tWcd5 zL3-zon+ux{7F39|wuB!bKsX#3y##o^0FB2^rH}A~6Te_&3K^u$F}Uqaww0E$3-E6W zJAFs}`Gf!UIcLzK;TIE*=C5^b&IwcfK?B;4lg+L^y}ScXgwJ+k+Antdr<<6R^gg>X z;Wn3NBh~`>*6Aiss6Gd z1R1(zo{_br7zlYJD)BfNGJ16{MAaSNez{nU*i5%Y#+UpT6IJtKCE@oKJLq6U2q;>X z3~e$95J*s4A_82TAq(Cm7dem?1Or^{%dRTst8(Z}!5n}zY+UTKB0-Z%xbTb2jMePx zT+&h0?wLA0ZW{7$<96Gv7$b9Kew78DI#quStI*a)ASI2IXZrB1{;AF1Q9M{-GsbhyX$xPDk-hf@}cje6gTg8!Fpvf1wnYZE@wY_DM>?jFJ&f4bM| zPqSF+G=$oVMZ}`C%ZA%+oOlSZtNScIuqh`(WE2f{JqH_G+NCQthaKAPl7q)4exQZh z+b4yAfbFKsznneotCG&Izmvt*WBeW>N8Y1Z+Br2&{g4V>+&|M9BU5h_ofFsoDNr=Q zP!uZax?2$?s=lO{*mOM6Jr@zr2Alj@JhwPd90tH z#Ha?wvYN?deN-DY{6T>rk~aVzy#X++q~{OJZx29Nj5$~Pui9NeX)}s#hU8eY6#as$ z-+y5v5pr~_5{B@$hwK#CSL6WU;7Hx{Enrnhi~cLHgWkf%P6bR5!W7LRgBr5ZAoc+1 z0Rc7u=;$GSv&XV=q(QtpDl3^0oe1I^wBr$b4|N zJ@;j|J0vY7rBM7%fse+3+up5DMOC)nm!0}EkDO!cSi&IFmhU_bB5du&t0EASWx9gc z(ifS3CA;BlelLEKo}Nw?-T!5!i<|P%{%Q*sozv_u!k=5r-}NX~4zt;qzS707uqaG< z_sJ*sIVLb>))&}5WJ>dNc^&xpjzthEtyfv>HPm~DNjxcYy7%`xrZm-b2)OR0ax?a) zsk|%|?CueIX&HIjh3MpmGS#6)s6fg$!2&kNdDj_RKtN333G+DKQA)Y*O_Scr)v@bZ z4KWMot#CK`T+(s&IPTrM@1QDo3EyY=Gav*UV`_SOKdceDe!bFyf(#>!;A{*Qu21NTxPh7LYPX1e{Iwn*Gt4nVH$*`8l%X z7PS1m5Ma(rg~ux8+E41^KERh;RF&@rMia;>$z18`^7Nf(^E;20yuNTzHP`-G8+Yhl%@NIA zVaw<0&JHkg?Sj3WOY=Lhf!-Da%rf1Td=8EkgKWb-SvL>M%AQwdH~fvKGwr~~M4@FU zEFVW25)o$+soj+E-XRdkr1S_7MRj1exlG9GbYjB{^6yHTxLEH<4iz9{N73mw_Up=vR50(K!1l z-+6$uJIQ-xYxa@URtQAMYGm9P-2XrWDtAYmMdr)EiC=|%Tf>pr5@g{{_;h`eY@J)~ zXlQK$%LehLzl%zrb%~S_Foyx{wcL*r+s{7=1x*`-Kd9ONjI^wl=!N1g()l#2PGoL& zCTvEMyEK%2TJ*`J{=ASBMo}nh@k|&y1tnz~ypJzAq;Kma!$;-GkxKuCCp5Kf|K)-8 z3CPf9W+liyF@lJbQvfubT`Vx!Wi|8J!U3yC{*q;Yq^zwLNxDA?*VFY8@_Fz{61ja^ zjt0fW><~~(Q(@=_y)A19KJ7a?K_Vf#VNC?Lhi57!Y3Hj+6{54X5_+|_!uj9FnUn}o z9v%%dqPK2WM7h;q2@H6ylp_Q=@D39nt7*d3cS-jrDFcmy)mKq*{no^pW$~TQrdCb- zxM_NyZB`7m5P!g|s8u)yQQzXrcMCJy9Rj~xM{z&1+kB||cdjBAN1-=lKKjph*Oy+` zl*aBEwL{ z02_E$+U7>ywc}7Yh`@vh2tAn}e}D1elh?T`j5qlA_p*`Je3;7~lvaN?(bgvANIs3P zyLs`DAr5P7Y)t3rQ#tS!LQsXbfQv}p2-_>qCP}5&);ymIz79_DQAImI)>)!iTFFn_ z{K2K%!QZRVbGNNB%0ykO>%PVvl5$O41dj_+H0<`w9i3fX95*o|e?us8JiF5}P`>rC ze;_z?bv9@~B1El_+!FT zb9|MPnKY*P9~dO+n^=M&x)P6k7GFt8Tfvoat9_ja32To0l zykfX48N{E3ryx4BcL;c=-YT4?u?T|?VvvM3+*FGBg6kxej)5RLN7r1$-}nQqW+D`2 zjAHz^n>|Q-x-!$_{c|t$TrxV=^!bgi+#a7`BJAvW zrI)b!Vb}2kT!lRD=Q|sZ4g8N>TgKouRZF=O@Gpr`xM+7AMyze?f9RN*$U`PjDjXIkd^W*{)4NAmF_V`ufM>V(I@CWQ4%{If!JzC)#1U zO{YWz29yX`%(4hk9GhEz@D!O?SwR6O2R}I=OLFe+B0M}iKYXCyJ@50`odJNIo}d3V zbZJ>%7g%Cj7YN~a6hfvq0C0q~@ji!jF+iRSyt(Nt2AM8pXXjNOJOsFehEcby>sXml zCLYFz1w<>gWw>E1oe(G~waU4fOTNa(KzQ=o-V@-6w)&bft{-wfzlmXu<}m%fUvC8> zo+S6)6YBldz*zr6D|Jb86A}Bas3R|-s?zwYZ+jIJd)oA9irVI{3XJNh$jXWA1r3+w%kYS7Fj zL);PblxI}SnZRtZW<`rHd-r3akt0D#efXQx0zRSUK&H(~@S zx2}#97{7~a>V6**1Mf34B8gAdoa>MX{ z&rNY8kF&LK$G3{*i}|H_rc?Lb=VzDlRBtEgd;5L|&z;|MTs4+KL7W#OWtOmiENI__ z)=d8jep`2Fx2ZLjY!RRQWJw7OrQ`3kLWa9aA#wxSE9nEAvhwaC>+H{iCAlM=hJ^U- z${WCu(TR={XFBY`ckGa1x@FO1d5L|xz|M(Z~BnnO!#`dvlIgl%`M)TL%@_8qq! zrU$FT#trM~)($^36^voQjt~r-uXEzAESc&{hupS*?%sKUdRw_6Em2`k@ILlz+=nh{Sh>LrN0eZA)6_OPg>?pa7i*ZZY zgM?1_?#%~MvH|mviL0hZ_pBLZU-~0LdT!~*Xs^ICG$4<=wt?PimvZ$y4wf4VhY()o~<@Pl%`M<2hoXMAwuC))MtSw|g>)B};!5 zdxwb=J);=44*6#9cPnBu#rR111Y7_Sa|=cS-*p9Xzt~~>HlTUqmRddjT$35-`nTPy zam6AcepDtik=I7C01aUAuXa*jQB_IvrJWodi98$CNGOeBuA05nwYV5RDr(+13cLm- zH6hA@A?<{L?V5n=z*%wwzFnJ*69b&h0A#>(I_M3S z>+is=!rCB}S*QDJItvxwr}}#W2t?CNL`iuwT%Mdy>R~TE5qJ{*|2)18=kb5+3@&9b z#CBktSCb$98tMVXF_JOnAyJ#rrxKY*P|sd1(}jVwN(S6e03B8Xq(wY%`>hAt1uqX$ zh!q#rec8etXVjV>;)Z0UhP(a7ZMRG0&!%=WzsK1kfoAxHEVxKbTKS&C2(AnQ5lzcwO;3i?{_C*K!b;YCq4soV1`~m(e2#+`g64&<);%6$YBKUm^T)-x(sPgeRaUD1Hymp5bm|Vh%SUG94il@r!`%@pP zOAeV(Upc*{kTy12;k>DUqa^KG;M&Cu_o|972iPp??;?TP#sPS2^?FF-Do?}EFb0;D zpg?%iB&byq(HEG$`T0yp#p4b5q>SokU~t!hlgz0fj`T^DbexW9>gh!S<&6!jo*d{X zz^sET-hzYbhd_oBdIC&>72JM$R6X=(gAw2Oa(!3Ge9{}3rscl)$QT!uY5~9&q0_BR)oour@0vqf^a}ALt*x zq5c2&cQa_s`sd2%;m}XdUp>v|?^Q;ZQjJ5ps6F^7I#w+WSK@Dd?*b!%ArcLMTW#xA z|Gh8QSLZ&%p>EMX7tVHeb`0)+QEOn$*%33k&XfM3us@z4?9G6eApGhZv4j4UDA zAYaM>K(;m;0Y-`_6Nr>Tyr?>lh~S&*Mg~0qzog*);vSQdbU+*GhBLB2D##lDEg#RR zMDIUi3)xI;TzCBkW+IneO__2`MRa#L6OY*>14hD^X>K^Y`NGDviISy;nK2dxDD@Uv zF$5sxxrk8jsBtNW9~f5&57&RmFH_g_2g+&T9rM*A3{vGYG>w zhfA0{=1rY^nXq3sS<`HwgPxuDMV5a0hGyX0rW6xJhp)KUitG(UqRrn`Q&YDA=@(Ym zCjN+sjv#R{l_ym{)}8YJ5JTe6M>e9#P^BCAAiO+ksUyahq?9yKAV08QC9#rwDsw|K zH3aHcxo^fb3}RxqL2rfs@Sl%VHz>-qD#zmh>O~{>Ikd|J!1sa+#-ix#xh45M9_Gj5 zVq&Vy8~XF+Ow6qG_j%D=+>v;@HO~_G`eXF3+B;>`*qL zFBnQv(}H@A=Np3pp}B!oG&hPhm;q2b{uh@$L*3>;2E{p4`fgpnE@n&)F=c?CTdy@S0Tc=5CB$8D;(Exj7&|t^&Z=E74)CPo zXya_?7dYOGRpQ$i+?Z%0DY|QW5(_o~A*!youd@A&xI@8!h>9o&;|21a_`h(FSHKZ3 zLOxX|%-{Tg>0N@w3LP--3WML|0DiVZnoyHSClU+e3J0@PB-sO0K%o&zNO2p?mH4r7 zgc2FH;&cl&tjVpXsF{u@|n|=&t{+c zmzFw%jf0Vk3m+)fp3lFsqT~li5t5ek+^eQ{Z{Fx?j(6xtNP7;M!5)IOVT5X5*>Jsr z^IJ#RUY9o1fJ;{b~`LVNv?zf z!Fwj1OG-i?4T;vvNx9u+vE5VZ1~(}Nc?!4)_j^_QGc^eO91~tDs-TrWKB0pgQ_llc95Ig#4XuwIi(jW}>wgn1f54?86VIgQ zv0EE!c9znTh=8dnG$!sRn-RKpP1I*3UGw8fsZI;|udMf`gM#OmAB$~$2UgnNW8%vjsAxT5qv}2w-*%j&$230w z-mIvk)ICg(7MC(cty{O(o`Z4;O=v*Z{h!j9)biKXOb3TGc-`kE+l$tA@&WUqWAa&{ zlH6F$BawM{66o84_8p;lF|zfpc%`Zs#OX&Y7u%gHB4Woy)$5DPhu@KabL`LTqan!AQb;MkX(oaSsDk*bPfchs3F4+TOo)d_S5V`$xk*@ zJbEC&Kd-SMl#iwg1MA|XKGLSa$R&8yv#@=b=iyqQthF`hijcjY|3D%A#J)<3Wu#g* zLub!cJ5(nF#*?8KBt+9TF1hM&Z^D>9{MNyugS*CyC%&tzOhCJ?n!3rn7_b?(aicHv z5HvM2Z#8kt%~ufr=6a`JWm@s6q$4F*V2X)On2hV1VrRrCH!X`C1vbHTVxz`|QHMow zWKOB|SSvPR4tK7qX6mPT37Q$R94v%wqO45yeb*(|s~Gd_CoJ{=B9jh)>d$jc;AiEgHgE0|G3PB>;-M++5xF%l9~1VW)A){ylmB~qq+_Io#0&s4W+2n<^?_RvL_;*NX=kFwC&Z@Y z>Fvccb(p#UCrkvj=&>7eT!I0*g~c}%0=CnE z4$rNx)BY!rcS00xB&4_C!?}lF{&}4ORw!eDTdy5fcm_bu3DdrADzZk%hOq($(t@U2 zZx=MS*$FIQ2$)A`AkwDgVhx>0NYY{W!QFwq72z#|=UReu%mr1a_4>c7b z7W$9f_ZnD!IqG8&mP~_8zJ%N^A_!_G`fAvOGS;KykWLa`;PDmjnOI_&?S1qeg9R2; zp##IaLWk4N8zrWmf_G6NwrtQZ(m^Ov^mhXqn9QZ+5nJuQ*610Lrge;ELEJ`2#sk8Tl#%}EH(}Uy`$5E z_ho2UC)z}Q9>03~CbPCpoWshEB8Riksr#~euV36F_2kMeRUk^2>HN%U=@aqOE@jb@ z#qP7v?YXAfK{I{v<;{~kGB15AR89^&7>TSauw0_6 zX-~b4{EnC(a@B6jFM?~1rPrU4%jS@46<}qQ&9jiP53q1Y3j9I);Lm8a;B$2m!hFc0 znjQ#SaC1sJHnj9$S2-E4IZmC;RJ=pt z9NXhy+Oih3X}j4G(|$r(XY!}@E8Fe}+U38a_Bz^=90g->

X>fn)vN&3Rlc6zLNsYo&}WU^5>~*h3S>G2`6{C$Yxl_i+KLFq6Sx4j8G%WxY03 z(Jt-k>%mv>HsA!Hv)F3OaW<6D;f1QuL9a1#E#N-fwyzw~7EsFig1}eSGJHVGgn;_$ z`BzV20h`I;@%MoFpu|Yh`NEd1C0I3qhS-^Fx|7Rh0=IE}Djie=n8c7Iai-7J*&ER8 z_u(a=hYAWGonK(73@fmmfJv!r-0ssPVbo3N(trH8DX5bbYs!4TNlcRhCx|e*6NFVD z;GZW@&ss$oYHMNn&;{1~Ac4r34b9k)A3sPMyu2j~9|vKI`yi^A-m^*EVaY&)=6TUs zFfX6Y@NpH=Kj7W0%%8l*sa~y&b;nubuUzf}vNBhkt%=8ddV*76-;06{H;K+JXy%uK znyVl`;~bpbWYIYdY_wClKdlrk!Iaw zGYBYGnNB1~>#vbzA ztbR(UPF98ocEW0GJ~%Y#>nr%!B@q3bIIb7<4r|13?-x$GKEx%fcZ4Qn50e*vuruGYFQd-m|*)R z1BE)6M_{I5EXvbT+pIP8wT#{GKhp&awJ_qTm1%_HG7|%H;VdjD@MQvZ3hrGRso5o) z6I%{41adI@p%k@uLJVFJxS!bEzd6z(QQm@w`0@Ni3dHarC>_ z#9hOKtYUv}SfTaQ6Ko)nq?n^xk4Ksw-7l>aoP&y2^iaUP419KA| zvnXCaHYM&CyA(^EutP5fHet~*azG_KHw*|JG+;>G3M40AWZ4A5tsfnh&_+%G0d|m4 z^%T;V45|Y(}2*7z-;FFPPXeti5rb8M?R13&+} zUR${#6s3mW`;X8cb_(vvEuOHYo!Wod$Ivd<3;UpwvOLbZ&cLghHsQSe6L&$4$N0pZ z+1h9A8~0bkMdY^6PY^)#5LViddCCgT9@&t>MjyY=2Aa2>2S%0}dw43n%y2{}YJ~9; zk5$`KE6Ejyw5B(m8_~qmKIG=2LK2A3d8X5!t8Zk*Ze9^!A->&YZTb{Jr`(z^jD05Y z(T%8pgVe~0@a1>4u5b>9gQjhVA5t&gcWfJWIvfUBvT=;I{bmu*x=+V{+Lq~2Wz4zF|(<`@)0;2zL&T>oc~giFY24Z?%pzKxQyo`S+9#K8&mmm zVM$UkpAl`qY&!#iZXK$3x>lPze^!ew^QwXvMVDmet9JJcvSEnx#tqqPV zjY8>Yw1BbqC*CFV`uUmdet!U)a>H&i%S zQJBexI#*To$@@?s^Udr?Z~r%6_JPzU2Y}jn9D1b!GzS& z9Ci??32(mHDf%cKlZ$%9K3+cLE$oEHd3FNgROGK4R$CPB$2J{?7g6ZKq9k^FALZuP zux@Q79uOka#$e-olf>YS92Z<+F0Q?lnX9%+DsI@CbjJba(8~~2(+)o;xPHN#66V{o zUS3}9%PU;IH*S#{?Mzk@pf9CRvi`Y_0;UNSMhDoi&^I&>0v6~hPaP1(OYK2Iu)zY1 z3urpYjsUE|9#S|K`r&l;@IpN{oCjWh~I>rplsi)2l_D(Qi+Ry zPN4(@g=Y4R|L|BOh))FQH?NGan2`Qz@CFdz$Pq+!Ya~|0)*wfCA(+km;l?#=wf%GA zL7sNZuat~@Y>E3-!ubMgdVW1yu`y&pXkaNdPepsLeNa?V^XvL8?`_}~*ltX+W?l9( z1C3rId89eW)3wp-&lwt9|438gF@QPR=q6{|`e zA3MNfJG}rUsrah~mVYRyO?X=iD#R+*X}(p?kOHi_Wv_Iai$*PRdhd5xzsD+P})$s4E%cwW(AAI_k&KwS^sF=uCUv5uL<$CJ~>7Cj+OUHGAY{VFYKlW_Kd)185-hJRJJhMS2+y)x#) z_$liazm1Vpe)~5#!~I55qCw0c)8jY|l19)G(7+kP5FY9IyN<2g3)Q}2rAJtVz%;7l zy5i$Uz+N8uj^ru4gium8F0P&*MURQ`gVD%<*rcC_2v}HhURWY?rmzT6m2IkmbLa@} zv`4A15>Q&~Edptb7XmoC(r)hhc_q7YrO@rmN};w(KcdftayUixeql#qgnyp zx@;QnFO+Y*xF>|GrH4izNI*}J%S2FrsOy%JmUVXdz|BeUvr%onnPE(in4O#JDsOgL zR@T`UgT0AuFFf&6qsA;6mPk8ld7_Bik1vJ-w4T1}Tl@6<8%3e-BLM|EsxBJ=t-5Ey zUnXduC_0nllHbHmv*07+roe6RLZM1p3w;|`HEMG3HcXNI{dD55*Qj$|G~H1)1w$)^ z<A%@r4EVRkLIwKY zsdXb;(W-RYLr>hQc<0pL*orC%?i&&$QkaRjgu&?Tn#$p}&)LmI8ucUXY=;}TCQFso zau{6ivJ$BnJxHreRV`L&Z50r0(vP&V)nKGB)hU?R?b6hVmY^Dc^6ebexYV3jKEUsQ z$BZ_w5`AAbypl14-wOusiXTQV$UdEOqFl5)K0Oy{>oy5PZSWejnwtA}SphMFihNd8E~Xs~r6mp6hp9&m{# zPoDgN1TW<8oxOtFuqKOv1Y7g*+Xv1(-|(Uop~XPo{EW%T8aDZ28F*aCtlka z+_efl<>ewHX|$R862bT@o$cPp{*pgcNVbJZrzzth?MOEk!gKeRoXwSn>E^8fR6iB_ z%B9`7=f~1^WK&%0rk$DJ{KEA(d*I@2tu?$BPv{~0=(GX5>-hNjU)F9A$ezw@3$LpQ z^5kl&KV+_Ub$Z9hO1ZJ$JIO9${%Ab@`f#2=$TQxrFYjn)C;pAs=iX8oTrynla+g1! zuk0&sWqSA%dVG$4d+4P1CikXPpeRl6sCLClqz+5*{o2k;>j63QNBj<}t8KRCufrD? z3sk;6Nyh!&Vv*DvOwG_kU0`d-!0a@Ur{nFMl*Y09*1-Ll(E%IUjOZcRy;sD)-$se< zU64z(9J+ohKs%Y-RX61!2iUg*P~f_jjMuG3-I3(741}DpRvp3w3NGg|eeQxlL?Tx; z#lGqM;xb{4>*O-DXQe@pv%kOpV_Dhd#f`714Zj<=3DErj&2se-T=NYYqFuCCzsOQ< z3V62gTv6Ta&K*4m6oja2J#SSRxfDs-^)XQawLsE`y^7^rC$$a>zAC#8mUPPP@#6E{ zrsG)U4{6+XB!jrhYhXQO_wu=Y-2Xba61`_Iv%Bi#nbU}HVRd0%#FxZZTlxD(pC9gQ z?y!)Az1}=~WjWnoIim&?VZpRi#aLHj*ownj&nLq7!n&7N(f#^}yNeHD?{MyiJ>J8Q z9e3yNo0HK|L>%r6*#G%^pGzxzDHOwjg;&FYeEAWoI zy9^%H-)cesCXubN8HaeosC`+Wy#un}wCZqW25))M&<5`ly}J$UUUdcUwH%e_gItRd z8i?V(Y}P@=lCidCM*Ps!B(C}Oj64Mry2X!nG8d8P4L@LI+R!Pm5XgNvj3(0@XOHxm z^E3J}Y^4$?#6r|ZWXbrJ{@`U&cbP{roXw6pjK->5kFXnr+K}hcx27!7O;C3J{xzRS z#eapOs2jkPC>5q!)s6nnfAszxM=9Qk#4`Q?4>VS;@#hr{yy!N2UK$#@&wOkZf*9MQ zALG5eDkUEApUn*BA_X=Wyf8mEdMxtuFZ!G0*UzZfy5cwJSf;AT8ZFZH2vQJT zGX$9=?1#}!>FN+*$O+8;4WO1FTC*&tl^ouP=>Ansn)|6KT$(M0VL&?>H({fVlP1ZM z5n!{l#+3GY$TQxI7_@HxY`P4h3LX2rjV%%gapF1lrMi(!UGF77_7&rebeD9O!p=|a zBZ;<&)qv;v*Y!#8yY0^5nTN2gV`Y!U#3Ju2l!qQ7ZCji?!MlE6(zvBL-VdzrqtaWr z-&qZPzaEf{)S#2pO~|Sk8TD=36%F` zBa4FhO-l=Ru`*?1jp4+Rwa~0aPUqmrLjsj!8NHmaf9Y)gukjpXLayD%b@yO?iT>-6D;g>d3WFvLHY zUOq(SA;Ttb3_&Rg{_Uva$XoU;__8~|8wx2 z`z*^9Xo&-b;^z<#JOB)GDBq}12wb?c5PwV4b;1P?6U7M*V4`=l7+1^a63bjc7npS% z1vA^ikU{DLSMab1M@*LGulGc#{LkWgM1xGzjVTT@PaB?&?=wn#sCV_)&}lv<2%2c)WKA#lT`0@pS9Nv8$Z>DZS@R&mTwdMSM_>QL;oam-Mmg} z%;`T|XtU+Kd95kw(mV60@|8dz=9y*FAM}uSXjTP(K{%8R!RIi06u|7?0`q;hhIFKWC)&BEp`5oob*M(;#TKUbY^Um;H zvvp6##+*q-N$ap0ZyCE9tkZ1EC|YAt|Kr`nGV=M7mOuyf6_+_gU`MMQ_LF+?EF(Ko zlS;56_L1M=!u9Ixf=icD#dy}qV}${oz+Dg!^B-Ir z2Ns&$Z_Vd_&iL>i-*#+2=Lj9!n7o4F`|CjH;FXb>GeN}NaU1O;=Ez^-7v+dlMF3g; zE1x;_>+V9S3EVGl(~NI&q@YL!LaO+nER5CaXum0;hBsk=zhWk6_(%Lm6LWNl{h45O z+o{3eP|}+^lah>8Yt@aI#uVP|QhNQCmxLQRKKGI4y>^Q8TceYi-sQM7w^c4RMG(ei zC-|Jb;!_zr&SX*)|e&|h66r;~5FG>aATk>O;Cq`eaD>{T)3$r#D>9HdmCkXUSu zq05MwOK1D;yfD0DZjGnLaAC3SWku=AV(7hN)f*pXmpuE~{V@}CCKSkG&08xZytSU%QKuTLAW{zTgDD zi|>ZcR4#Pyf|${q-h&Qq6d>R87OzLUySr20zq>8^Y!G2=1wH%=D_BIRI0x~X)y-JY z4d2&`5$99)iuoIuidls}%Aw$*xQ$pM`hH|lTe66n&Y1jt(s zOd4?uIy=Gx^E!~;TWbp}nJR+%77;(P>ABL;uNCzv zuJq~mWXvASR`w@U>ORD!%~oE;mDTRN!%`T)kStj| zNc`w;(kIUh(;lrKy+V1Jw3Eyb;pq2;=g&#BErpYJ(wNfD){ahuTSZTFAI*O@-?V=2 zJ)jNu$;?J_RB1T=62Ol=fOB~MA^K$NVf7uvfjL3o_C`LbEmK>hp!YsjZri+qYdm(w zh@wbhk(K=4FB5Wb@#wXffMZ%aImJ$VX?nJ=KiQW629$QXHj|Qy4x@}NJEBGOcw$b? zh285iq&tK`!Q-^4v`){ayrm_}WM{32JP+--MZp4fLN_U5AS;gm7%(ph=bj1DZcPG*~A}kEVn6);s+Bmri z=E()Nsa$lvZJtccXqWx(&@V8|;Q#$~Du{BcVz-NqdEth(j>Fyzm9F+KQ)0X_r(8gT zn^uZ^(Y?11Sy`3>WX1(}$F>{f~Qg18|IlYbhUWO@g2 z-ELQ;zZ20-P^hUd+h}WyD&lKoCoKu(b-r!cKX;A$Ma1&dZGL>g{C}5kMBEM>CkwAu zV<^3TyhKEgT>T>X`r^O0_ok+%KKfs{ZEbCd2g#s5Q11v5gG=D!pB{vWOG%mb{b0z; zJetx{)6mm9OCn!lT@Iok2^{PF8$)*%#5JH2MEanG#xp(Qbi`-TPk5;QtBU((oP%

V_FG?*}!{_#ED5dVjz$)VG7j^TC9J(2kZ8qO;ahpgY z_6lS;{IknieOYukEV50>m!TUe>Te%e9B93aV(bC=(;~2ED{MdWZ6l45O=#96t{OaA zwdeRiEG3Bk_~t8rqAW#e!?w$ZZMb`T?TC;jUGzwiQMXL?{+C2&X0rZ97(&2=`{Js} z{Z)-F6tc%heCNoITd(u_n{>s@4NqE|pIOX3K*3xjX#^eAc(U(@EBEacl2m~cTRb&&y?hR{esD<{c+b=b>se)X8gK=*prqc{|y4Z z`29`~uM^iT4)h>L+m#etne)!+&o5Q)-eOH`^23&)bJOG3!K;1WHaP!BwqJ0|i|5EC zFaYdYcYcwsTGHjzs0FKsu9e)v!nZ&HDk>_nu(IZXpY7M%eJ%O{y!^>-8W+n_4#PIZP z&GIFaTNm8Nx9vCZ?<2(ND=y0}LQ_Y@pFVv^PbWWVW#!~7uB=qPW(T&+f;VqwdwlyO z-7gfG;4}&m?G#Lx%<=-Sals7}@vgC-skVg4l=HPi52L3?a8$@g@Nrv_bnMk%@2^_M z5>LMPy8d9-KmU8bSe1G#JC;*5TMQHW%aueDE&prcL4q`DlfYHSQIQjA@-6hB6p4SY zFxw^HhcLxj@BG1MAW+b)neWfjVqQH#71~)IK&muqL5}Jfeu%Cz@yU6$*zrS3(zm&V zez^R*PweQ!a4|97$4SZbMjKi(72sgYttg4hYI!1b7M*z3JT}&L$h6uQZ6U41m%wna z-@4%<#3tl(%8^`o{$99`DrRVN3IhoVJ{o^EH(x?};jMIBkoe(+C~?#~u%e7*ov}Ye zBK`QxrR68Idu4z-2U!uW(LTZ{#$F)0^dv?s&Y`RidwZs4W5IoF4Hi#JaaG^*TU#?L zYTDBw5#s=wWf9g(cZiJJ+q1f!f3I4i579ngk6#B(kL5_Z01}5$)BAUrBzEfo08sdw z%~7x1&o7s5M?w69^t{=PF-DZpD~XOhJN$NEE}shX=ENj_%r&Q!o$!l2{B*cqfjekbHnn|6rB_ zu#!sX6PCN5Qd4XkD|P#iJ&5%aN^4wgIHr*o7IS-<+UjRQIJ(X?9_@&Y7XRqHB%PiB z9;B$YI`V~3C~c&j&S%>2Cja}d7spr4q}|)TWWNxS6-lv_p3JpDKo8@jEEl5)Y5Ai1 z?thNw)8=y^0Hx|-7b*;7sTf(i2Q)C$WgS#-Kd{AOGBHr*z^YloSXbkRg6jI4ju z`(Vb!#mJ@LanejSTS>V=osv@grSk^w$VbG_=tDBGz5J$loGHH84x_ZZb=_Th9$9RB zdoat6#!|~R3cM6do3f=2!i(yW_Po8oQ1p}MFpYA%- zDjPRY1>D{de%eifk*HimCYFCMOI1!4@6R94Nlnv&ze`B)-3gF9YZ{|ELv#CWHNSdL zb=>6pKa|rO0jak%NGs3Z$a4&&ewRIIDxA>Q+{9|ml(Y}Zv8WUFf6Yx?pGMxdEJ85A z52!K0jNOuPjx^4xBqjdQiS#OKWF@f-ZLOXuR<<1HpkkE3Ht<|cZBLn^ZKsE~@4aQ= ztSeK+W@Kapydv`DOPYw~{i6*Up)++D8Is7Cgb)KQdNj;;e0=Nz6+k@p-#KKXNjo9O z0ilC<^ytw7*kX-*H;XP?$`C{5D>4-o)k5IqE~0x6mw$6eo^d` zuNj*;dmY3-92Q&Cpx>opZ>Gj{tXl#7V2~i*In%oM6KS{T)n+s0Qu~kL7W>0!$8q-` z(vL1^Vh65%-*DhwPxZFgOAkhRDogbD)7Shy|64@d@TPr|bp=x)Gaxf!I^*zxz9*&d zY{z%6i07JCtD~ZgLB^LH8HCH#a&-$ADwtCCe18K(`Z70VRVXDz8Qu4^BJDfV`pC%A z?aqgvz5eb?HP*_wA4S2@`ssOL!_%>BlZcuu$KKqq7x@VZ6N-^BsC>+Px0W4hYu#S|CK>yVR?BZ#DA7oRHO+yrYjE5?RpJc z@qzd~swdyakv9$4XVRG7Yp+1;aP+ykxznLjTx~=#7Pyn$!Su1eN^^ToLx2RPC1HeG znXP$dV&WNql&Y#Kxb^Zv*AtwWz!?=47T!%9csuK$drJ``J0i@xe=B)D5(HI|$U>(- z%+K7Fa5xa#g#STJK&osQNT;AQmebQCf$}wk@F+uttYoRlhaDJ|-cW=5S|Pa;z1cp( zdtQy+yxV_l|9B|Y{*0>m+Ox?#qWcpG$!mAh;^;JMINgmE@D6qMm36`{KyqNcYWy8#`w=Wa^Eb&C|z*)hBshVs#)hfOxl!Sb-X*1ygYU5h!c&w=-4 z<9ygXq{EA=l-RX%xn&QLuPz7w^qwKPfJOfZpc|69fx#dg+#r$34E(qJl#`Wp1&lK5 zkVs_-8T06w)2&dK=^S)P0c+eRaGH}_mO|kKppORonf#e;vd2!8eN^6*#clGDx5RMN zgI-g-!53P9-Tu4B%xyhNiTEJF8o2nv2=cI;7Wz$Z|LJ+{&b^!8?X{|Rg>(|4@SU+v zDE|03zgvzy51X$ph>C3sqRSzdG|i92^mw!H@pul0u3;wmig95#o(B3JEs0 zrHl_EY#n;qkl(R5R#tq!deL8}%uR_xHKj8vAowfy`QJvt>0ihHT(v*oW5|_NxW4$d z?dp3)UhyhU;_ZlCuLCcEQeo+5y>Gk6-{yp#uOy7zYijL$$%zJp&}pP*o&IaKmhbtZ zX`{=yvZQ?ZX-2BFlriSer~FwzYL6d2>r^;DP= z>F2oak^;{qTf?WmR@y;=g3RXLymJ-zB#PLg_4P+H`q{bMgbSntX+?Ler>(_2Rizb(W22pG#i#Sit>8%WJRz(71YmQ1CtgrN23~-W0v8W>XP5L3muj* zQJo#_B0mEy{jQf9fl*{3U`!z-8=nitS!0@@z<@iSz=oP2GhSpbxTS>HJ}nWM(keS9 zQN>9}+nbBQE<~O~!BA`WmWD;`B`I7%i340l7T0~NEiSF_tkNUgym?aFuLeNeM#h;cd@ZJ zoRFGz($=E&tbgEBGzFzZDNWu6dJyZ%K~9m-Ap>C;b&V*Y$kuV@(IfV#?7_P8G!Cse z9$wGnh-UYV1m#}gyjDqD`7lvOg**Fcat2&C2bgd&EJ$@LU6+>J7Y?xblx8A@*gPo3$2&r})|!^32~p;b z)wS1mab>C{mWHdT@2RP;SFT1M`!IfY7$Ag!w0yX7Iuqp2G^C*wBQ%KK_zy8ZM5y{G zc^M8>-}adhHEnA0orx!{raZ>Ee{a!*wXR+%zp%fz*h*)TUG264aoVXNj0OL1+G&X;c-YDxGF`$2?PG_{xkSt1K( zacL<=#dPGfT@gTs{QMQ4v0F!ooDrgoMH%com+XMyjtF~2Fx~(|Zoi2@MIU~`00Uee zJ|tnssa!eB(8YIb93fw+?aq9?BbX5bq3{TcCZZ_m0d-$U<)%oH>YT5 zNHIO#W&HRyV|;pA#?w;(LDr0Kdcc{Vsw`_)v6G60R;)~p%koZ|rh2wxfiPBM&)d|0 z_t84#<3#+bkku%snz)`JiRI%&)!YVs*84_ZUHOtu@3)#r$ej`H*R-ABsGOWUe`M7g zZKsv8^w}H;r-*TS1E1}zwPBHe@*GxtG?gY}06%{JJkxbMHGu+S%9U%KZ3MzN_2Fvw z=m`VI`KX8|WQ%80waoL79`jF4Cn#E$)>K(-WUE3$5{M8ObH);f)~0@hzxKNcdVK#* za2q#&qdA1zB9J-^Zpi)~y2qR;DW21}>vaT}0pbjtkVay5kaTNIM#ch9`E;uc-kH>F z0Fkr-+X%wdzBnKY1tb?93kpswEL>e+=KeOv9_b(4IP$-oUs?x`+tl#P)KoW&n)#QU z$cR`?=)^Fet+NEk{uG_6+p+N{R8#ED|NTYt8g?Ch5;r`UB%BOMiaam>yX(|`VTFjr z_RLmifozX=im3}hx7Q9oKE6i8mO!L`+2E^S$g&3c6R>5OzFTFjSBFE-tG>5AR$_VZ z3xrIofQ77&)_bMeQ$4x3JnOFTQ zfL)Gp05x3PPZdI8mrW(c$rMBOnnCH+otKJHj<98ZX#KRdrlzV~TVeWzkRl7YxrhDg zMEM9)iDE3`BSrOXl zfa20W1D1(Bb?W^jVCS$ac0((hsFBFHv~gOoq3iRtBKVmd`?C+(H-9i1C8U$uD4l7g z-GgNQNm@t&TL$O32UCSC6wPUlFMiin;)w6I<1w!{aPhqZdmy$!MeUT}*N(m@FXz(H z$8LeS<4?{l{OTLfVN92k;^KqyVBP^fMmnFqO>Y;sej>fZb2G+AWqJH1Hqy;A!zl3H;0>yUBQ@ zKhdur?d<$ni=B6}oclz7(fhdlDeRpQFh&;2TtF}{j(E&}c3g!qWxm+zgFyG_)UOb= zZPCECry+ul0IXbM&i*4pd>aiU=77wO8{V^vuU4^v+d-P1o<@+(M$pi$oW^S@(O4IJ z=SBe5$LR~RbGl+9hH+aE)rTEzP?o|%JOR@U>4g0o=g_2pgc&p`ng7N zu|Dn2L)Q5sn1mQ}ShTyNLP=&+c(MODBoz0b#Fbm@WE@Xjy2iFzkfW z@wdZ#&Y1p}bbL1-0UAlS`(&qi=&ny&(w(|dk&FmvI!t;^71XRFHQD@Ku0Ks^pj$~D z#wM}r`(R?oO~Yz7QoSa;`!pa+ zo&nbLuv9jyP5tNl$0HcHB-o1W3mZ~N1AY?EiG%lN#^_&@-yR}e z6wX{L(TP^(^L^f4tHo79KTTA#12|*^DQ;eb$31o~m|MwhuFb}sam=l2jb2ByjMgui zTyYTsE_7=zh9&YgelNLil@$Xi)Qnbw8jZ^dLMkB%7IY(pX^l`uhPWy21)jB}Zz3i> z;%3M{iTDK~bWC*d{njRbAm5NU%JN6>B~&a5LGj~9gDU9)jTm+07k#l*2=;p!OgFl_ z{+eYE1T$3HxJ5xu$adk(n`YzW)Vt<3J~PvY5TSrCtOEeA9Hm5YUec7{6d4~L6CWnQlc-Z^91ODc9oa499Ux)3`THD#;DncIcaqAMwsazlp)^4f+ zk<$w~-bL{LAXF2L2VxcyDPfhzi`Y7WzTk1=PcI52Fg&xtH36JkfrX zwYR?^aZA?n^QJ7|I)vsb7)oKD1q8vH(+ym}JYGA`J?ByVMc`hgl+3ZR7j*js385Z_ z9zRQnPQPrn1l<8!(tmahbv{Q-H&7Z{HtGIGo*}!##3Yo9G_Sla(?<05`;kfQC-V82 zZ^F6o@>?q{o`<$h5_VWMyr?(OHrzvgYkqHVC~{!kEp>$H(C1!3r@TUZj0=D4xQ%*> zLg9q8**(}x?W5Df9%OvuV6urMZE1Y9u(sWR2wOhk*$u$NiS6Twg;GW(86GvixF}b8 zLl~0BBG*D}g9^nHQ@Zm4+o$JJ+B(T)3?8EK|845*tUXFzvGWRIc>^;%UQui5Yc*oc zFH}s_W)jCR094hMeR+5{0c?@Vb@$Co9&XC<&%FFgqJl+<0_>-f)T0EBl~Xj^avBEP z`dLbVKL-T{z{=T_R!v|KGAQKF?3PrO#Zsrl{UdJ)4Ub{gv=j8_&!$Zwgcrs?!+yPU zh-O>r`GTKRzHovx+E+0OUFYwjWNIcYZ=KJ+6<69Q9!zxm>dDVC$+ml21yyA@L+_W$ zUogmuBxaKz@y1+6D9QrANbd^4%jx01F33?fjvecL!S7fb^*tnJ#>arIM`i#~mODvP_%Og>c#Ic*_OE5y4S-)bT!RTbhdKK8d-9Q1x<>s#^4)_+_tQu&Kjh={TLN$ktn z1Oo0y&Uf`EKTSbpH2bfsf87^uFfcS-gcfrAm~o?iBi2;+VT%9>g%&`xFZga(fk=Uw z-af|{dm8ULXB#DfU0h~Ndz%f3%Zsjsm+5QG( z2y2v+BSbSgf@p{EhI1uC6abD(Xl>M-&9j5QtldO!>iIVoZSwMc_k9kdYYZ=J$~rKE z`d$7t5JwJrkYaIrCdVJzQI%vJ*drnCyC8&06-BWiQynzwS<1c@28r?{WERkK1U;o) z2A_Z{lSt#b{;@}LA}5W%NgDspaE>nbuS)I%j|7u;>U5=X zstti*7K%4wi0I_h1U|aTWfzl+_0jP7vjpAv4Z3YsM$ZSrpNWVtJp4*aJu3bh`%gnWJ?~j~2`JC(;hJw%*xA_A-CJ zjF)fDJk|@3w&znfsa)Noxku^w-p#@5OkVUXwqGU1z@~isV3aN1pZfEeqJL2Me8&>!XD}zbLiNRY8;=01J$CwPP)@ znr>f>&{JK7?d4hOQ-2?;<49Cy0-R^p-JN1wLW>ceUwZMi#$6?W!L2#dHxTEiF7TW2 z^57Zit2S;C_iicERtzx4L&6ICSsjfY1Vv(|21qVoD07Dg%Kg5>o%%6?s2+KyLm`G&$SZ(IpPpRLg97q-(Ru2xjPPJmTck9aDg=-H24uSB~|wENpC= zGO98%G9QwYCw}~p2hj8H%+eSKgcXRC#+JD!_vG>mCv1UNP2)Ggr(@Kz6*sz!54{Qk z($OHDoK(WF-eY)Vm|JJUE%IOA}i4kwOU2p@hcdw@%Zl46;*VQScMz*n;RQ4edrFh!K35J1K$O^bVL#pn)pF_N;Lwop&d07)Ay+zyivhJzR@0mFK8f<* zw+DSRFbx;F5H;u^(TDp9;-h*r`a~^jY0d2bs*3hO4PEcKt*fJ>1<{~jK8NoQL>N}P zpwvR5{wH9+RUjT^@a~%UAILx>3EY?wBiP7uTIaW$Fya%3gvh~!RHoYJ8~{o~aK-zf zN3+lL=8y!Ts6cpr5CTRd9O2)1f7{MXK2!sub8s7_3}huXqu|HN`Yg z7%uhZ3r(&LOYK?Un|%v~59s>I7o*k$_3ev0T--_zMi6Q5Bup6AzWL>$cE|28R-EpZ7d`$ zEg1nOaKPxj9v0_(frl8#pqW<>+My0O)&M8(kfj6Be$G?gGzL zPmQ4Hj_OfyrZ|eLr`|p&Q`$NS%0&WZ=)q#f0ns+qdZ@^G@|01t(n91~?kj#-ii|f% zWt@>dBECPPX?`l!3@J%DCWTT0*_!fL9%xTzNqj!6Du=OkH|byZ(zosVt=xgc9cChY z6yy2l1Z|<6b~%yT*`{3npIMaxfB||p6ipe?k z3fC!8mvC;z;7oV+2%w8)Pvp5e@I@R&baG}On#D- z-7b7g!4*-0GQw~38w^|usE1$wrvy}b0kS}-M@s?(k_59G?#PHFVML?f((TO|{@vFQ zZtk|*FoRg<7#JGbLYL}~c4AP6V8Ga3Tr~0Z^GlNm5+9SeZS8IT`1x}(NDf>fgVGHg zFE?Yeh%Y7Sj7Uk>Z{zMu^tKi{+9sY0wV0yudJ_@JMx}NuGUBuJ`In7)w!|D0Y50Z+ zGKP0F9a$nP=OoXX4ZZ%$Z~D7k?=)0fKP8{+-LEueaOpN8jP(_(5ldZxSxt5>oaz4o zKMN=O+BnqzcD4WYm0_AP)A)9b=s|JERIM)#mGiPeTn2=;dwD7hi~f)d(=wAw>2@@& z$1)7mYwSj+kQo&nm1bh+vrd0)68H8At^415A!E5*?d&dp9SB9lu1-7nOTm`?e7FN$*?;Z)CtDT-bIfYKf0oO8iW7G zOF?wsEO!2@Q(m2N-$VmlXKD14+!*rI$EM4_(_}?18NL`1q-1IIT$KcTI@^5DbFj)@ zA5DD{MT@VjC?6Z;82TsnDqQ;Z2vG60u zzyA(KS~M7(;`*cnGp@sWUcq6rh8uL=_4RZ7E^_Je)U)~S*?Ub)uZ8yz-xQ#0SSjEf#U@L!9Oq2#s9 zCl}y;(5e2`lj1Q?-~vl;99aZQB!@{mLB`0S0yAY7LK8B#unO7}zf7OH@5cVnOtlZC zyIMLrCo~Z=`>#YzpF5l;;UT^4dU?_SG{Q+ukR?<-KCTljUzxVqNbH6p(sus~WHx(2P3D@@Rxqdsx+m^zRwIW?~lj|y*4es=nK zUs6S7=-x1fH%()407m4RSn1Y-=jX>U|Z z=m{b|+D;Y++44&x-;%=?$hHBRJ5A-=K-0mdx2!gIIz5i}WpBhLQ z9XLUYG3|-6v%@TnF263%0emoq@I60EGPID_qb83>PC^8?E?TAi;t~%E3=dv~OzAw~ z|9vYN^n+b-;NG*Kq>JScMRBebOj%HPn%(#EI94|>k_KcdTYwCUG#Py!-3t9$SMN(w zXzVGB!;fJst???wxtE`6E$4@8E-P5ZiH5R&E=5~FFr6EY`cE{lH_N#iRXq8C!AmXB|(zqnXLkTghixT1}JJ}gnJi}s$vu$JoUc4;G zJzSYu1C?&Vkl#T<`Q5b9wvz;_pjhY0(PO*5yyQRHl?nm-Phm5 zM>dYqy@bS&BcmKakD{Z#iYxO`pMw<3SI2{QKY*JI`vdVj|NO8L!J=fio!>ixJPp%H zF7+}BFC#{U{Q1qI>cQ$Sq)4y)A7Y`2dHOZmc?gT~Pl-&qQ$2VYJz&#q!gKzUjf>0r z==*N^#s54<_s`wnvxHbaARfVPw~Zc~nv1^Mk4$_w84+*sH=VT2MzM*iA`u7$JiK8w zKvJim5MgI;D2a^fFO2afw^q*4*?D8PacLbM#r5@CSz|A_A)$^8unWS>v2=6VD17`) z>dbrygz+7n$6CWv4dlr*$VDY{bc35YL6*Lb8NUcB*L`4 z95Lc-YfeqLPo)({?T^Fh-tnK0zUr6%-ZVZQ+s3sNUSs1cF>zcOXqhl_t0%Ir<&71x z{en-me@&63IGgvE5}h3Qz|G3xGC7JePZrlV7yj8Kivs&?+gJ|vDbGJC{vLJ4VZJq2 zNc-Ae`r|4rpVyk&o`E#UXP@!4>7tO(ZN-puaqPI3{memsg>axp124`;A8+acd;->{ zLyz5~`Qn=~q}c58XuWPN-rK!ej-adfWci=7jQEj*MlSjBVr>LyAvpC4ayfeIdU00G zd&KNLf>i8+;l~#i&&F2(BFK@3Gg*B0Rxk<(jmUZuwU0I-dag-bKPzc6e5Lg7()1pP zLk2AJ3k!cAS>KS*=Zt~T00vFhnj2AX+u46mEP_}TB1(~SU3gI+Qy0-FDk%ZVMAfSe zKwMustVwbM@nHlV{wavNVNbs+2?renioF1O4KiJm^h^Vir0Mbw69+5^p5mp7`tWL% zs9HjJfO0y2L}(~lWK@)n9H=>hRZA|zs`XV%2qFBT1nrBCUE=^NU^zkck( zFTZ0&qoE9FM?PLOQRE=rm>c3!pNUtses!AkxM}N`yTI?0Q)53onfT^D_Fu1UI7Xi+ zN#qtbg=3D2>oDH>WjTHMa+&JBFfR0y8m!opaYAof84lL-8KcR^6*V;QAE|`QNa=?% z?p1Z0=#Nef5QgsRab&-^nctiEzR>?riwSujI9oVi2kNa6b-TsNlzj+xbvSVYi|_#s z5mGDE`VOb^tSX)DO1yc4^nwdZ$5O690|Y+Jo5dMT=f9#5vJtqq!D8;t@_)DF5kKD9Z|K@EaPRxF5OfwKSc#gjcw{2z7G?e~#eGv9MJ@kU|_(iMOdYxFF>h6okCsE;YdSxq1q=%JMqO zf^kW1$^!TWO0yM}o#TX>7fz7%M%zqiud$_VQQv)Pp}CDQ@)^!q3aoK$)2bhVje>RN zYPhR%jqgY9A<)aefgpX|@(|`Y$AbB_qO$hTS9I^jZE`pvVQlAefZy7b%xk! zGf;@YHnEFi@Mk+MaeaiHpRh?#IP4=4AjzR@#_GZXM2H$H+1J&vCQu%$^UU+d%|#cO zQ1?;0_`mDIV(?`ROP8GvwQSZRlFsRU3w@?!`LavQZ?oIQW$gxxB7|l9Wy;BP6*8`b z9}PCQj=u4Bk4Pq4&b4HLeG316AMBpVWrsuQcq)kr;m7$(v<%;pMesb(fZRUbvIrZY zb(vxLz7ISnOzlVq-6R%j#T#Z#+h_5l)3J&=!4SYQf95ecryVwuPo<3^v&x`lo3s;p zBNXgJf|wm|#50CK3WWbwls+SAlUI?^M=B=7gn z$}nfO92xW=Ey(eRyZhq(6_MwK}?l-(u&CFMM#q%!w-k7PI>Rv)tK#OS(s=K(rZT>y__FcP1RzH~TzKidb z5ZjP8b0jhl)d^)-1@mb0E2pTndLL|0nUm!o4)DHV`DR12Am>aQJ#N$Cz^hT0QXnH2 z7xlT=SG4^_(83!MIT_&|K}QM;ef(d~hnXmo=Jw1)lO{6xa;1YwJ$wfh(FPbyx!-3nPeZbTx|0esvJfy_?>7^Y+XuzYeeRXjih?&Ei1 zHD~_scoTrz3vN;vV}DM3Pfj{6k@HSF>gTN6Zt0G&T5mtyN|}~uvQE2sm!ACnc9zeQ zFt%}NZpCnN^6e}~!1zHA7ylB;Y%hs^ifYN^@yX&Xx=GO7aht@gb#c(#=hFtqkj`9# zaGd9nI2N?H$`t49ZP<^53n`r1Yp1tF*QDA9gWD1$t)RSeG_=xw-`;GK^rRR+TYP{IYG>(P&+I~L?k6dQx^_gO) z66J@C0?Q*DZ~~8{ll`2?8AK$m7}R9SVh0MaMNiW^bgWrsswSpcv0q# zvDY>jn;Eur-x?Q0$NawAbj_vz_oCWsQJGlRanSuFjfsIQxti#r*T+gElfYXdd96mG z_N8Fw+#{baKM7V?{P%b*vv~J=-Qvl#uv241zB>^&uX7Xyw1;CSJ{-EfwO-xyI|k%& z=yM)D`Ir422r&6Px3od(A)}|K_Y~8ljE3IBope_BEqnX?%RV zvpa(D0x=vPmPFoqDyQXZm*d7L3gLYN<&3>OJCp{FmW+b%R;afs*=m^6AWE*Kb zn6_q7NccGGa5tori%JK9W7clvXn@Rn%1zp9#zU{bTf_MrF_Sbb>ItyqUIlcTl~?R! zdan;`dHsrDP!qqrPm<#uYBg_?lY=|s^xOXW8L~pc^V1tPatgaNG5hz*xbNt97A8A0 zS7UoOaO+^Qty9-LaQ|BCat~dFJZ^?$Ia~^7;*;Mm+}84Fwo}}RJ2e8UmR^54FPGWbh0gR zKdw(J9EmhCudr=sO;ce6>{(bI?lefHpz0De zOSR%5}?Wi}PUj{uy#1;PPdpn(uzQ3qYQ zRyX~+5C6@t0d_$FI*3pe_pP#!$a0vTv0!Pok!UOr9(yJwg_0+q`JArj2mXj+LjwnW zI33V}N~^T@I5Fs8L_p?1s7`uI05a@K&1JDj@K1dF`KNRzXSc0@|MK!a<_*;5^UYEn zaHpZ;22J}oXk3OU3TulaVZoJ?!jE(Jz#?6pq57dNeeGRZbN(pXK4`ejUM?F64c;v!Ty9cEagb#2Qo%4?-O1xX+9VYk76Ppwar>| zFYY~9#ta1y?hBp_*!J;XZet^iwmNQlc+zy^cP)#0EZl`5jG^ft`&^jGkci(;ATnvg z2L&1|xZH%u)$YK;)8z|M#%120Ki-7jkU359$kHO~2r7yBUaP-NpsO%o{d;^q9qFr% zCiZ59rHIkM{_K>RL?AZurPp4qbmR+`!9Vy0ysxbNaH~{2?Q~jb!;imQKOmX4sjSjP zWr9~K5PN@OmNK{N#ka(PH5RT%4=H0RRPTfKbnuWsnTbA6K6)^cH|YwzY2BB`Ou&Pl z2~D-~Goks2F*G zD)mW#weLyVAYK&nz;Cx`YP#nbZG@~?OeZH^FRL1GJSz|{qaKb!fB|k z5^VdzufbTsy4rAtQs#kXN?Uoc7+T<(f6&dH*KvC4RNZ>0SlzTU@$${U{|4fnn-j!> z!F$n%86G7DJVKIAUyL{r!v&7I9>kgftjRi>zAOqN$6{Cn$EK$zB?0XZ}Kv4h~% z-op2`C0rtGtX3K}rgQ$+og4RP+N1klmNM>0M5)nlNI1{6zeO=UzFwR>^P*Wu;g(4# ze7W%ws=PrDz>Rnys<;486qS`Jh<$%&*L>%`I&dCLv7(3aBcD!`?&9Q#u|%_S&bY2Q z%3Q7gV;H*?AF(y9*PQ1)$GSZ+C8`CbDS`%V^*Ji7(#fv>8?f&;EF6em!IjoyXrCnX z9eOB!?bW^HQOXQTOd|QKte$1^7$y1D)sgXruxGm zw-N7~s_TQX7acmpv>724f8`&B``ny{#!_(deRRm`U)bzIm6P~q9y+H;CE{V^6?u38 z`kbcA`HfvyNQ=tezxNLAovgH26HkiXziT(zb&veNf(%7E~AT#13BIo{V{PW3OjJ7PH_ z050aS_-{;fT5k~1MDWjS+B$wa`b{BR{Y37)c=cj{+{ zHRPy@i1+W1^Xz=PT;@``L1AMikLpVc{QVLUGhvk&L1O9`CEQ1?qHoLk#LF8GBUI?u zw~*4B3JuKqJDuEIt)GLn)I@OPf2x?u4#WB4B* z5U*2u^Vhz5!^kfF8`6i7UX1*UOLP65=djcewi-va;2liW^5n<)fo^-?Nl*G4XZ7BG zQy+Ky+UJ=R)ooZ=s$oOBYY5rsh~)--qz}T`Ha>~VjE7;?faAU5<2T?JO)Z>|{nrGp z^#l<%bR)M};NviT9(@HQC05m|JuoYO9nQ_%2Sfiogh;v>5;+kfrG5)uh~)-HOxdXS z?|8X7Xo`X{7a@HoV+z(-~C8K;G?mS_-IG$A7 zfE6DzC5}7r`G;U9$pJq?26zc($UV1UtzLX1Peeo%uUG^!vKGbApzksRyIvJr+3eL# zZ{TqUYEwCgyUc3ANT~g~vwa_dIw3y*1*ntF-Cb}Iy29&%1%D2r#R~Lt01eR|xQ0yI zfxn0#LYg~c1RCXHP}d+MW@-jX8T1AM5VngnKA_NWZJeEFC>xWLi6yt!zs=7_jcvd) zmE!&a)12%B8H51>?mhS3J#(15MGOhHz_y;h!3Xb4X#6egh=5uC6*0vfb)qd&F*kcA)b>fhp1tDdBJIhv z*DzCoVu!Bo2TgqwMTX>V8~*Y)cZ>eD8u6XaZXB5Nla{$+eR&8*ZxD0ZojEBnMNO{i z@$QBxFV&WWt^XsR<`vILfi}=G4tDr0m;U}bL(^+wqG*!PZ&Cjt=>cJEFBqsnLNcjc zk|_ZO_g{1rJ%a2J7GIWxpXxWJjuB$@*;H20;c0f$q$v~gXb)5;BPlodV7g3Jiq+uoa3ww_{*JQcy7b-I%8verzYxBKOxmKkLvGzb ze>{CNh@_F%OkR{qbu0z}nj2b^FVbWaZmapK%v3+NDOI5a)?kqP-L{qNiUsVA0RcE* z;g&OG$(x2!oj+++Z_D;4>}F|ZJ8us$z>0CNXxI`yKIbS8Hrt!IOcy*BzMd@K{n~Gx zBe0iJwtmYW79OBNK{~wP_rkrD)cgg-rJFpc9r(#>4XVoV3IVU4`|lWkLQyv=UsYRD zbNy*7=y0ujCCj&PD2Wde)|Zp$bWE6(-r3K@H$Aw|G%*89x`)mNGyO&{S`W9BKy3W0 z$QjugIkMmnq{NXu9vu#IUdOBF-B_|PKj72vr8fKh$)3Z*VYGhv0Bh-72Y>8FuuFf& zu(Xugzj_n9iZ-b!IX?kKt(%Y5e?QEU+LyI&)qAe=I;wlEgs9#ltDmf1d8bOu_dJQx z*CSO~q*_?(;wr5X;i%|-pGUC%mD%N|uvNCd)^aY7uPi?s&Elm;ax-7Fa)v{P22Ycl zv-*14JK<{K`C)fV)Y!~IBf_9Fi{#ZfZkO)DPJ0X0;G7%Ap_+EbrzKjyFhsHlHaQ%A zWN$nnVgK)rx1`zichNkApf#0@yU?hGQB4H1kGgncY;3LR<_@1o*01Lu{oDH0sw;ly z)_7? zCXWUYP-s?rHP$4G-4uV#{@1zDaUNjyBhP8CyBZe1n#Nl%Mm1wIgsJ>2Q85?wd%Il= z$Rw;7TIO0(&GdS+TYUdLZ(gU574s#hH$dydzch0C{CA;i$2_k1(~{%y&o%bi*~$;Z zL>hX0&xad-x8bJwUGaopi0;)G^r2d3MdJL~b=NX78vYme*!^%vKmGr>It!>M_ciLH zQi_ChcS$NBEeMiQ(jfxU-6>K6(w#$xNQnptNGl-?BGMpTQqm3IGxvVYT65OD=fJ=) zy#M!!y?^`2{a9GH#fd=M(EZN?G1R$CGa6b2e8ZMKW8hMH$uCp~N7QXz@H!pr3|7d} zbR)m)*X^ozoYzP3$CgE<-c#cK>=k%G=rSZCAnP)uiv$o>T$(7)_4O5L@j#cP)6dXt$^(5} z0{9ECZHB+mJ{Qh}MR?XcQlD90e}<*oAP)VY^(>;_90Z?KnjDTu4`dfZ%s;q6+MuT) z1888=HVbz;M?uPS{T+X$Ik0NvOs3+_!E-Fb%yJGHLk?OO%|cwf+u-`2hv%mwk8%bm z14v20;Sn-f2)9=eWZE>&T?gW-9;JIh{U3qaeZ1Vi^J-36cyEt&MOW(ol z2u2&JeDd8wb)dSb0}YO*w9yR=VnDM{9?28@uwe_#q!x$fwEl0;_h}d14^^E?(z;=K zL}LD)^wNyUImOMd$D{mvDZUw~%Fq(RkTOk9G;GB9dyW8W(p8k(Uk@(4Xz=HOmLk4? z`SZlbgSt0-Big07#p|}oP~BHNWeep+Q0_{_lglzGJNU2Ay}kK72(0>{ab;1L?VdPr z9xLO0p7>HCQYi2Q+OxAO?057|l5v6wol)Z!Ny;Ek8Lwe}tuF>Xy-=ojQy$&4vnjkN zrtSzYKy`$_^i$8Q-MQ%kKZs%{8P*tDu;2!wM)<)EYId7M7xuAosID4mdtGrS>Hf0Zb!>(0~Ld|5{ zl?nFM2%6FkaILhOS3?LTEGc_>!iy_YO84)_VfjYNw0+(u%YhMdZq2i(_AXNwz+1I9 z$|*d>hK-^G-Yyyg>A`EgkT-+B0PCiRF3~sqT)RL>V4<6kIUyi}_PK^BLjsZ9hhLGF z)XLwWYhcT!ZSL-zXq&Xg$^6wj>^qlELs+CDc+|ZN z9_YI2?=ev9h$TpukL|AH>ngW=x<(jaj3~VxxcHE?)Z{ncmR8L;{A$x(1pbHtzMaO- zeQ{5L+UCQS1rM*XK-HPu4OKj$b!RM!S%MSG;`aF}rEwd##(yDqSq#ly4GrtA81M_) z$HvxD#x9Ao-h_x$-FoGv>rXZRrdWQ3dLdkif7Fm$7LC?;pPq>_kl(ePWOBSbC61Iy zz0Q(Tk21*Aj-StJSw5L3GB+^f;`yXB_WSsSEIi zclDLi3_5GKBz$WWKqa?1>_hOMT%-)N4pR60+o#~TD=I10@cf&le@x9PLS#nHB@}p%fVHLTt+{(zXEdM_EBi^ygG*>cxr7^=qY}3k$|j70Be@Yne3+jULCa<)?tV z%*h2=i!pLbfK=W!Ru*4JHA8GQ?4DhI z;TPa>nznR%x%D3v27Mf9ezfMD$YQ5>wEws*`g8bm`B5?IpL2R8ljX=WrwWl~T2ZJTk z6(1ro+iH$?o-ikzwhkT|-d@*-85P_RA}Iu-fr;Vw_YG3E_;uq`gT|w#8GG%py{Z9W zh3b=8;Nu5HQ3&j{(OkvSfB_u^xLk1K{kl~%wYI&z0+Ya7IeB@KF9&S-_o3t1gGUKT zGLnP?gY8O?Bm3?~7&^kZ)kYb{mm)aN34Lz`m30`OYLw1^=tvn>YbeN>8rbui22YU< z4!<3^+F-#v@UWAz4K{53z(zo_y8+1~AP5@E->03CWNnai42=ua=k|@eGALL7Ld4%# zyZP{^!wrv9ACkcTqm?Z#%p#W&1=(Z=tOtrT2W|;2#Xc=6FK0>Vb(%0s*)4PMi7GffrFrE12%U~w9-M`h zLIG#iAuEjoE2w!DjNm2!QR3^&=;wSjgZwZ@1w~5QM(HEt&}e&-YMch#YR%#87)J~08>dkeW zV|32kMHgJZgC&y?t=14NK;4apg+pxlDDtPvM8D>=?;&%IFJTVM!~v$3<$GqelI4Y0 z#m%$dUjA!yGS=d(o7wmkqRd7LN94PdRdDpNLAR=E{%RhL&ctb)oKJcH)8Mk4Op+aC zmx({jG=QlPokh5!GPwHnf{#Gv7Kc8<;)lfK!{M?me6E)EDhI5K6LMZ@trJu2tTzVc zqq#ke#?)5_w$TnREsZ1X{_Z4C9DMhcj*goGN44|0Db9U@jnYv$ zoKL<3kPb2y2|zm7_`|9e1r1z;B`8MU>?{qXa}6I6l8_|GF$$h zKGqsX+GO*NsQ{Tm4oCRyCwsoDGyfV{d_T5sKH$WApOf>|$XQgMn3hxG)!JhJ8n!Tu zIfik?b*S>En@g$6#Y1rfH+~DP9SToV(-;`yfvEAI4HunIZ2k}}LTK?l0YIZU432IL zXg3P>HKF@f_ zbQ#4Y28-kQg`iv>^kyG>U@e6ONn*zEgFV_9Sa!a+>=-L{9xrBPUrZG0j8|O641Rk7 ztBvLBtDWmB9}FYzH)I$nOXZlT60v9!C$0N9Mq{5|R(2T2-I%cC>zTb+5m<)3N;l}+ zGu~{J)%HM19`7ccdAAGq@j)y{ZzW}vj9K;|3qZ0zS+u>w=Al)|7Q4*;Ly|pM*z9{e zm*N#nRSB2h zh+Wrh#WKoovDZ#<3OssLJoWWoE7o$$)nWM__*Yo3Tdt3)$J*5 zv^>@xkc7r+f6~9-TglOKx{|&J68&X;uG;JuTHgv^!R>~JcIOWy*O)Dn+RqMiHuy;N zzcKKW`!g~jV(TRzzXZhVN&!6bBVC@PbVPbu^7d`<_Eob|#>W7G) zt_BhBWes5lkr`j1?7`klAN&3@Db+*|_k*smrMY7m(9~6pajx!rA-@t4d32sNsIZs( zGP9C)WjTYkYeqRHW5Cp`uv7eFQxjKqhGN0U@w%ecpD+1OdF@!i`Ar$YvaCaY|2*)X zE?#0Y0qU||<}(XZSYsR*(sHnz1Ft8r)&iQf7KLSrpW@S}=6_J#kdOStc>oh^IW zAIB&UKSJ+B2JI@U#QLJ5*7p9%vJoS*tjTR95$CtG$X8bN^pox_A7TZ@Pp8r3x7@SJX6LyKYtM~ z@><`SC-e08Vj8?@U@aFSrT#^8c|N{+gq_|3zZakHp&zKc>(74gbfQScr8b&b6s}Ct z_Ew6%?jt=-e&Ne!+w@-s2-1IzBt!X$qf9J2cW3i|d+$BUGbU6J+<4^ z6#nq4Bb@hIS$Oso{GRptcz+q}tjJ*;JU4r`J?EccEi1`O)puK>?O!qH;4c zI+k9rQFl`$vkw|>rAvDa4U_3^`f*HD7{nU-v2qRyt~ry%c1BSg@2rRV-Gj7R4nvn= z(YmpO6lvAOi16?wPLVH*?8@1Uv*#LGzP_r;2D@>;#4`sPFBXiqc-JnDQR{~9ciYs4 z?)|C`9JI>vXB7WutaY_TK<4EjdZedygwEtOY_IBTR$SSPD}HZz=Zc?`<(S;SYkMnW zdYPP(7yh_U&kPHeWexj#xVcqpNEDv!Ib?VBwD{1ov#13QPI$lG->Y(1U6OhhfBogL+qY$Q3m|)d)=5yTYqOl`chRn66S69a^*Jkwxe$){p+{qA9>A4IHoKv_ofF(&QAs&w9yby2=H=_AY{Vddgd>b(S%mg(Q40%Q~Fv zuVoi93wC;`%X#?OG2dQ*JZ|r;w8FBZsbi#ivY8NJbQkgq4C;+X#ExHWy;IsF7;K`) zc=}2=Iq5v2mY6$5z;1>P&a$2L&13;3l;9-zl?wum8Dwtli@Pe-MP!Z)U+3RZi~}nNmMA}Vju#yTi%Lru zfdn$PMCm)S3u7u!K&C!4L62v2f;V6U|Wld^tHbPAp@^8w!n=tewyHmx)?h%O8$Yq1APc!>(17m`3 zWCXI-t%}N8*KhyS5b?B&H$y}l2s}Ev^C-n2-t2XTfgXj;ZU}uM;ne55#lp#qh-hk- zd_D39Yi_P0w?JRUbwlY;Pr(hg8M;<+e?4Ofq=0qKH==2ykB1J#p*XV#h)B;1I(U|e zng*m*7|Mqw=bQ7EZ=n>A^SoIF+2Sq$^%Hsgh3|LC#8$3l25jh|634FAT-wKnREK&x z&CLp_I}Wt%Yc(>WSI9kwk{Oba0l?a^F+7zI6?%Qv1;S52oV|yp!G8RBM%i`EvbbC# z+3X@9Gz~3#Pyx(Hlkdix&!667)Xi(Ic{XD&oX|mXclei1X@z%+*_xZzjUsAen`}9@ z^9S}zIo3R+(Z&zcF@y4~e>BpDdR|CC(8ciLo|?8@=N8t`^9@?UC}^nKOnKssGcH?4|K+0#hA{|QNdUp)f6Vq~{zyE)ax zrzNJ_G2C+MK5l-A(vRS_Nj5zCUg*JSa4QYHj)7J;;CUzd8QbK1)126MTtUY=au;41J<6 zR({^EIz1Z{pkKjY|JU2o^I4aTm7QJoO6m3@PNg~k9&~ldz=uLelCqa+AGzBZOe2>c;HHUurNIeLA>`R+w`11q+o zm*(3$Dlc2&E>knn8wlQL&8%THGDJ!WCh)`&qIw&NC$(c}%ECQC9d~VN<=$-B5@afe zooDy&j%DkRr7_pd-HHp#zxxFz3#h(r~T#%5gWI09(H`NZJ@j%pq?$MPsAdyUZ0k`+q_e0OV z-DG{B23!BV5M{HW_+z0_4<3yS3NSMw2#72Rj$%jj(2ImK<_a#_XHmJB-{!xknJbdNHYk-#mcw+pc7MM!` zkpKyeOxKe**LT6C%uRXYU(!943tr^pWW34(7u={w&`Y~9#11?L0Ty8K5@2<_iS?4B zP$e5&o{eqNBrn_HK457@X8&||M-rqUG=3Kkkw#GP5W&(Or6>3r{^V9Fd%5!K>P}ro z_a=%yeKLo&&YOSVF)sdUx6tEZ#tyDR_p}V_@a1s!Pe!2HzW&V*vi6qe$?w434sIjx za$RLl{<#(==Fs^TCrf>sRh1Y?X9DZ#^k5m=TveyJNIAMR;j^t+i(3HF6b9#EQDdIdis2fDnLefLB8=H0kI}n zQ5-dri<6g8044xpUyXy+VzA5fE+SC~AkZ~%$oXjcJdz<_r)Y}3=d)fP(sVJZ33G#D zAiC1uj9WW=-@*{d;d+%t7u)$USg>SU7sp$pNwe-UnZo=H+ zPidY!fZZF$%LKzo+TRk*hyxw^+m*((hA+%8bSt`O*wkPi4S_;>?JG-{l)xew~!{kT8pE#ss2Jx zE*CHn9@2MEWp68J2ThOU``9tD~z0 z61kGuQz|}oMAQzkXXGLko_Ao65jPSJ#Brg=mrE@;wcK)1`I1aKCSiok`2pY^h1=U- zn=^)cY`tMj@ogN($F3q)CFNRSPV(`NuW!LGYiME-HTqe8`6I3Iz5sEhfELM_?ts%LgOlne#@&JXx=GwVU+bb-CTQ|U}N_fMX zk|<5k|3+?9n}KduD%XDy4gd2W*Aifo`{-@<_uVBA_jLM7doO3C$nQ|)LaF>8znXrJ z3dwaap{lmq#&s#shiSxGPKU5AO$;|UIQxE|{5}R&Z`kyurQJ=nBi7>Vs#rSZkR5n* zZ@0Jc`gJs{2p*=nab%<{Czf(1YOGmO+xQ(<1B4nrd_&J|QBkk~b?u@Q zcc~YCoX}laSd~sDOZ{VEnH}Lit}$&Oj-f;uQ+91IpTH@Cw@;6Bpp=QYyDa6Z$C@Fe z6|{`_(*^&AKE1z1vv|Fhz!^!yd&k>Ms7YC%dPFmlJhz?BuhB~+z$g^JcUIP^L@8e1D zVv+nKhA5t!;O2P*k3;ds3(yiEJy%Gtqv84Fm`l|?D*ozYtFoEzTJmMo{Be3-<$TJQD0OwgahC{r^m*bz4 zImtSL`rVQ=@ERzV(B;>D>!Om;~)4;jVEIdlndtYfe9Sk~%gv4i1(F?n^m2oLO>l1pB~t?0J@ClHAYM`T9Rc zX`Ebvd_iSb7YM0lh)8i7Za|lj&wWulp(+u{%=Dv-fhoj?0*x=FoB@o)JX1Ye@V8 zR>r&P!vA;Dz4MMWysf>ty8m^vi{V4R7MA|BS~RYUPectf%3zd2iAPnL$^2pY7OUB& z!*+`#v(PN-$}Rj}NW0S!DyffWyeIuLh>J<@uk@)7=!w^AKJR51+DJ0{J9R+m(TotF z>91T(5Us(dvLm?LxPQ#y!d3m?l8|7ZrpNIJJJ5tMhBNc~PS}No@EXlG*hc}Y^UJY( zhCpZg^>D6O!eaox@y760Sk<muTKDedFOqiz)p+r z=kx?Jq~nbugNGz{ov_BU3Ul=a;j6H;!zo>gZUh>1HmO*odVP*-W5KyIJ7(8UpwPN$ z^)y^;8XrC-l>XwZK}%7_(>?A9GrP-QMS^~}9v4T^`}+&V0TkSPybjIZ!iJ;xM%TJl z=LJ-j1YO?-#M4u4`}Ym0V|VdfT$WL$WLhswT~0aI>5~gg-z5A!HZ}|f2v7`qLG^_k zZR<3i{VhdugU%bsFi_v!K|$t=04&OH||cuAtUuQiqpm0mPNK z%1Xz~69RBpLP4ci;iH1fa8%NI6w<1Zm}JCH)@@n|qXIa+)YNSA^QYtiT~hzp@PXpX zsFzff;5dWw2@CuVTugSO&6 zXXUFZo!>srN5>9)S?$2Z4r3|;RUx-LG+LBNJxVUwB02G0-5TgkI7y$WL$l#zBnWSZ zokqvH<^s)|@BcuDx`hoy(~n^aK;Q&U8XQq0f8CxBTzc{Zm-&f5i--U=7gymz(1XMa znCqZERqZ1tVCs-wJ1;%T%tARF!NLBX)q5K?Bvk7i9B30G6S5|DNo*JY&=Yqck3lt< zZ9*T2pAl1qQY?2V+n)19OE`e)5r^Tr4VhhY52QD&_zaK=si*(`2l`!UR(zJsfnTG1 zMf$|3arP-Os&-LnHWCK$;}d_{z%Nol0p}}ILa(Yx`ENnTFCKM3Na4F1)&Mi~gRgJc zPHAE~L;nB6arTGVwMBlt!61Q0!9=~DM&ZVx<&9e-wwhj**9NrtP6 zGZ-wQJ8QkzSh56DfC-%ff73yR7rTG28yEsPaP2z{N>?w^KSDC(1E;jOV}k*w z)?nhSIu<3bB&a#ly>?>)D{kE)p2J`R=)Gf|zRsG74^` zL4+9g><*W|(6I3#lV)_B3bGV!vR=wxpIes%H8AtTz?8*D#`@9>^83<&L zVq!2IksxT#TtrI?(*u)|09B>ergOVvE}QG!E*)0PTK0V|XAgVMW?wQmcX{3-Dgb^7 zDB+ujrlPU9zDEBC^v5-V_T4Ht{Xmf z7b=T@GelzrRbIgEV&~$nXHiiR>{YS+Az|0~qHn#i))^ZtUBW^VMkABhiXw`1ZQ5m1 zp=}1;?Y^zPm+zD^cvc1BtTV`_f!pbe2%dLSGaHNv{XFi!G4qB5d$m8QcKgZ4PgHp0 z7BDeU@?K&pH5hoh-KdJm3;=s~kmz&#b3w`PEZCFJ}}8f!11tP_1;}sOmqh15Wyq4d2nbLr_GlC9%ir? zNck+Sf9>&J-^P3kAQZ|jK`_021;6yM2k)JeaAaKZVya1EV{&t|n3Sr#|o!K z85Hu6rTSa%d4+&U|Cg=omlE;{8R`Y*eB0h9SaU;>8qcV%TB2daqHc9pn5vRP{b7)X z>+6d|LfxSaw{Z?)MtS-a0=^{+P=gI_jp>wwBn{j=kcdSVX8P6H-)PIjW&)jMP zxNWP^gP*Ini!>)7z7{nThcYSfe07&S`4U`I(DY?CA>(&d!!$rdy^IYdJ_KgP_L#DA zkAy#j+$vBhkjtXMoHwDf#_Eg#IQu;0NcKbj?aggxHBH@jO?i3{vcOuW)XI7J|AqWzxD~MSpT)NCEO@3NCB*;~6WU`~(0M zkB|gWFuWir009?iS{eTK>-%vi5H&)#ecz6+V*@$=2J0>EAt^~BbQsoB`^DCL(2FnT zi)XwS#ZrXSiXUP37RI#Wn>y=iPNU3W$v~7ZGiew$L*L^oJR*>{t6NbfnPKL20e=ww z2YJd}c)=8yi=R`#Q}OKlB|iM55ba_m#Wj)fq`WhRQNG0N`Z;Lj0jO9cnIXp)&~FT` zX;6=RGvzDE!;ruu6R6a^wEwXnZz@lZ7_e-X?b+>#>t6wH)(;9y_%+2Y`_$4U%Ym;M z`4mY!D4@%esrhZQ`I%AP7}Cck(>6v(V;uREc`|bw-|Lv{wm|x$@Z&VWcYsBP?8x1G zbSg9{RZ|sjjEU)h%xXZ@|6e<>3md1NN5slszx5^KN^f5q#9!p#_}{DI|93fC?L3e{ zQ(-bQhoL=fZPGt=a}ifyug8!FnO%Zp6YFsNpzB0@onvPAyL$DkDl<=w3aP4Axnk=G zo~_+mrC}D@PuDWIv`JP}TS=`_+MLX>U3@}<0>zTcjN905KFuZO^OEZ^yL-i~!GS+W zJ)J7lX0Fei1BT5Jr18%>T^z6|`D6wY9PUhCx!D0pR*4?Yx?*08?(^r0D{mD}QC>20P#A}z%)JKrAf>%ZR1p{31AJiXfCiL@SdneErI)uP&{Svj@z&O%^N z2s&3vWbSCkn4Ye?5gX{zJFlVMT0HB(hVFdscF)B;F}sIo?0i~P;-SMvY}eA<=iBIj zd=$>ak&g#QOk|1^XB^6 z%D=$E*k=&x9#N27MrNd zWWY`D`JptGD(XauS&*+8`5?yv`NNCpKS2&4mg zdi?vPt`uUDI3iTar=dQE4?ZfEa%Nc#((EH27WHnDE#uPtYHk!*{5X_abvpbI4v4*#P`<#8W zqKbiG_(0O)1#fVk6i;S{DZlDV&El!a7$rrJ3#hRXb~`jOe->8J#^ zU4InJA|}&jSW9|FC7|Qiu~$Vc98a=^%0> zr;pfg(+OPBb_J#p`pr*N<70_}@1!L3++m4lT!dk>FeP8z#(@QE3U_SxayLo1*|B4g z9EXI0$>Xdiqz#Y?nVw&0^2V?dz7`N1xiug9m|hK!(h2KP6uTxJZ){TR4R3xLODX)N z3yNc$hBA&D9~`)CGPLYm zc~Ef@aMR|`@1T>0?K5`7n;@gtL=Iw_rn|S&N^?3&>M#z<1|3iP&p!_wbGz*o(pYqVkO_WcQIL%4c&sTjP*?w!371&#pOgq9Xl~1bi8;A_jl1eZ zycLZ^|C<7L6HsnijVtglS0@~_*>j7GX0(V((X2}pU8^ITJ7UyP5I{%-c_j56R+M`tz4Lp$EeOKm*J+E zNgY_?OYJdN+6nPOKYsqi1JuqOz%1uLjygF?=VJvYG313KO%ibCDi`i&?;IGmLPgmR za0_YX1SApyiNv0P&%N)tUXb$d^vxzLOJ1{U1gb)$h4?50Zy{Gz&}}}WN6=HNIIdk> zR;2cGysXJMJ>QA~7H7492=jNPtjS)beud=Hfrzr}?_)-3DnO}^M{ z8O*|@^Stc$5V~4xdyo=Nu=8f_9*Cm%R|oE=@B6Ga;Z9hcuTSsKnNSM*T~`YI)(~!M zQxWYHP!UotkN5LC?EHB4k0Dy@!ia+yi=y^m-f`SXn9B1~LU^iL=+D^fYmyj&Z%(pQ zk~~QZGsj{}-!6}Sko;U3ldajHpfo-_XRnyr>dO#H5T7|kaX+`LoL-gZO1bp_ALZ`d zyH^i^QA`W*79QLBjPeUB=?*J!{LYTPB71`lxr>NHM4sj7%#_kL84D?n=S{wbjZWR4 zoe5UXr#F=gGe5l3)6o1WvX)!jtwdj-c|Qg(j46ys&9$}~$)0u}y9eeu8*(Dn2GVJ2&Ot^SZiOzV)Srd0E8`xKd|hV@5xqH-(EK z)0#R`d2;h@DJj6pAq8`F0sL@lqE(Zv#gI!n_K{1t5kH3@N~7@8hi5G@r;-&*^f_U>l9*@cO>&irvKyA( z{t!sToX)<%DPGysN%Fq_m-Ieqp}!Gh^zEMMTgcNKP^ULvHPwretIWGvZDF@opgww( zWLo6~e8CBxx!h7E1=~<&H9=c`>k_u-gh@A4VzVB)A3SZQ5eph@mYJTt>u`=UqYTBb0B_0;_wOwi|gO^id@UyPZO1RfP_#-jU4{F>d@aqD z(l%qhazZ0mcb!K0Jy#WNe4Ec{pq$NfC0aBjW8J7f9p6ofnK)ycc7Z^LldEP&^hzNc z^#S;_n3XfSzzOC7mtK`e*DHRYHsTYA^hqT=a5x?k3hEDJqp{Z@{a5^ERfmFMhQJK% zsiNoylVIVfy?`3?a_!*bfW}qHz4phSH?&0;wDF@3Lp& zSM!E9Bg+WRrm*mvKlhxnj9Y!=&1H4msgiy@*+ZC%zIzxQ+qy?{bs$cAUvu$s=H>`89 zz0KY`xe-SZhCa7ahP!hy`a2#mhpk?Y*jN>no2Wa87gBl<@=q20qmJtETA1jd^ltyI zp>2icz^+xcl6fpPxAt^ZecpKiHwgS5is_k{wAya&O4Qrr?L=xe#!vYu$9vvjgN9)sA*3jd%@4>t7@Vcdk=8WodkX_ zI9nV3l$K}4SNOI0X}DeLC+CPPT;_d21N=dFQj7P=|62cxyocyCkTvm%;!0k{lCC-M zM)2gSlc;p!M+2#`?1ynRUSxZ*qY!2%{=t`8BYp&?EP+P7z95zR#XPIhHiR4Ki6615 zGk@7IfkVIvghx?KRt|QI$#Ta0>AfXI7)=ryngww95nR9car}V-0talI0p#%zp|}`4 z{v`E?xWC8eC&9O%u#sp2i98N~MKm z+LdkV2P*W9Ic5va1uAp-eS1}K)M!42Q*LabJ*AwFljXH*dn$*jW38u38i~A$;qhkx9PoePV_8-KP7BtS?(Q*iYV0uCn2){7pKz zGDl`k?MI-BdWrw*9xjWtZ&k**YD2DDSo`1eO95f^da5;W=>si-!6%; z*WRGlJO2A;UVX$mo&gC^P^S8XW*)K=-qDq?i6J8}cegxecHZrMjAmc0cM1~nR&bHZ ztEk|XYL(F^6W%>RZBP`b`uEMbbHKtN<#q>ZAe^(Z8{l ztE80`^P_1O738Ycej~P-5E+?s7aRMAeF8yxyvB;Z`Y1?TK} z4USDZeUK4QX^Tp%8Bs21GJMb0k$JRVzI?&MIt84B=bIap&@X0gH6E>q*oynyQQB6O zohC_X9Qc^&v2^*!W*4J%vv!PHV+$Vj7-KrloHvE0+K5n1JH3@oBG+nK6 zkMbEemK*+2>9u)x0w!$pDsr>Z{yyCQ-i$KH<(u?$bw}l?50xb5A72lm7iZ{#XP*Nd z1%NH~L8sOU%X?rFkz(kb@N{Jd_gh1$)*^0@;9l%4)!?4WT5ZR!fOAz=Vz1q#F(D`^ z52f;wcFJFErQxPNcy$xr<2;LnrOKd?!E_BaG}7BBaKFCBK*qYTWH{bLK|E`}BO1|F2RO0>MJ1#b5{8Ko8u=iJtI&RyMv2i$a5KoHntAwcLANjb$`E`fBFS z3`yj#UpgwZgq)?KG!Rl!@{E!xz#!g~|JZxY-htYqKZl7!r_Wqp86z;u-#ld-YiMRP za>SaBT*bI1wYbC;QbHUjw=|N14kcBM$4T-Unsin8AzMX5>uV?r-_*4&R}3S&*%mmK z;GLs*q?rV(k;!Lo_jTUAkw~N1h%-eO|YF z@YI=IqiZDOK0HKx35rPm0Vh_(ifC-lujkreAIq(1PS&AU@~&F3{yF_Tq`$;2g-htB zKQo`equ+n+v2!gYUHBlt2%UlNw`Rh?66|wwSc!_sOkQxa-wAB?KWfyG-J*bsxJk2B zC;beDze;XQFol)zpyDr0dIBNW=@x3*=~-WI>e)oV31{rYsoP;*4FcHi6|BY0%ccO# zl03LK(FlB)coSg6_)OcImp`3ITJ_5)dYRJQ(`Q?FiXVRM>q`hH3@300;oZG^UR*lL z^SQIe^`~*<@X)VN#$Iq)gD@)7PH9of<7d30hIBzywS3EfoQ4b0+B&NT2N3mx@^3S{ zTFc{<;NA5KMEUZl;ZQ1GhhCal`&*E_1JT3y#Gii_xLiocteSG`7~a&C+CW$Hzdo)W z%e=1gi+?)k`uQ?TYx_((`LhQf);H_nQ8V?oPjQ4&KYg-?zzk%j=QclY;pKR6+YOS- z!5EbOth0CGFMoWzq!@P?B|`y>r{B+*ReN5So(?Vp-=Zh8o%!-8I4NNa1MZcZVe}${ z)s+{g^c^?lp1o^n@kS+{TB|YCf`n-SWzk(JJw3gz;K}qnVG7GXs;%tA$=3{){N{E{ za-R^5B*V0@#`?gZu}LPk?vIp7=55X30phL9>-8~H_f*r|{0XUM6c15@rLU*_)P`(y zrH4>t^ zbW9D1c$%+ZJ^~5Q)%RB)R5IIt2_#nyf(a{3G^(g@tKk0#|a9v${o zySwlD500=M59To-P}vjy`^q*iMS{GI3Q{EWx5fKTFg@+poj=~lTra3mOrk{(ut+ae zBT-Rq-3q);08k>qU3A1_Z~jUS7kI6E;@A7)Q^Fj9gHJ@MSu&M}AuF``xs)m)`H#?{ z98Jr@DqZ2Jp3&0IVHxwj~*ii;(LkYEh-gB89a%Z!srZRqmyOS);XdMt52S) zPQFM#C^7P1t{g`w>)2^>R={l2$$xi!wr-V%(43v}UGkU?UPwjN2 z{Q#%4?)qHQ83%@9{g-WDH%dC|$o;jxXK@o2mlner_`abD!FfsjTKGl9lwEs*6{9Yg zRMQl>RQR=jc4*0Kk^Vceva|k))^E$D(o7pSuw-ff@u{jN&%|5b$C8%DZ`{n#m=nx1 zw?CCrvGl#mbAm*yhv`aEi^Y|U7<8!AeyP!;GO+BvdY%a9n1u~*cnQAJ`<@?zi@1<4 zS&lUiEoW87$KzxSPA3M!FqpIzE)dA$U$*(~NrxRJ|4^kk&TAi~qe<`wI6@V ze8<6lfO~3bAj6 zv#Tvk${%#u8}C2maiTW5Hre{3@}+7ulJ|z^=8&|hIvX?Z=4;Pbx>;eAXpv1xln>k1 z^X*kfs9&@~1JlFLDoQ96GmcD1$~oB{eD*yMi|19>TEMq+VC>FC<;V`1W^P<@+SpW42lod^we4%Ea~;>CFFbHP$Lq z1X2~IsS0QL16`ho!=hUf}TuGOW8i<IEdC6C{3lIvkb0muGfV zzxbIDipXpy9-C2)0l~#cJF0<{WF!ar0!m2pn8!y(V2Zr1?#2gyXD@e42BbA4!O{Z( zICtL{AqNF!@RvpD_FMH0Bu05Q8K7feOxP7t=`6u5Hh4 zd1ya0Q^TE7$-Zfc7R=4*6KE2n|7|pYuB4P5!Bc_ZFU8X_#jXW0hU4%g2aD1sx8F7S znr~Gy>@*+W2pPyOE%){qvDRvANoz9SlJu~kkd|La8k_x4nfJEm8HqZJ3hG=nH1I7= z$3`~Sv_wQ)Y3b7k*mrmTVAu%GQ-mzK3L+0f_bDr8%_JfYZ*Mdg^FcP3QJ;+&&5wrX z_qbB9bc3_9vIgDy#G~SfVfyH4v82)_hw|XT3HRcl-xV(yfkj~*_U2?h1Og?afdpVj zmHgcTSvmV3Vf&xxC0=I2v2x?)JgHVh0b}$# zODkJi#J~c0b4eMjSN70?@P4PDzSbmy>(fbyYVoW##x}dh)XK{C8%a16Iu@775^xm4 zH+cU`H#}ZZ!K8`Hx8^VL8QTjCK>v?Ee_0~RO&E^a`Va+rw+HEcQpWO=X|f&bmh@r6 z_6%8ybf}P>6Kx|O+rJFMPBylTyEqs7mHz1BMiqkPmos$jWtDcpp}F|&8>QwfG11?9S*MHR4FWJ)knK64KF=#;V5_H z%e`)>FM(+t*%2B-xPrisRr2Xe>?@<7qhuK#-7WGB>yqc1Nu0OZTvIX_7rO73uoV@L z{&-}_im&kAacpc3R~oQcfTxT35fiHzh-R%D-Kzhe#R>Z zCom-RtIoTEVF?C<|AVRb0H;E6=)w)B!TwWU? z%pXVnPzqLa)e0+?`{D}9Frdu^2P%Xyn%nb5p?n$`_rRA@Sissm18ih;TJ7OE2c)x0*JUdB=|93@6Q_)8x8#hv^aL1FK~)u`JEcv6+(Iq^9;6TXRdN@B+tm)reFOq}cJ4`DMA zrpbo4IgaZJMNm=R173!#$PP+n$#o|*=q2J8Zce74-|OBdH>$Xv`SNyxF0M4Rgq2L5 z$CSDEOembla#NM(3`NG)+oHTBZ;KHL>`j`VbgDZ^mZMo7X4TcI%>BCT*1W4_Z{kz$ zRA%ea`Rwgd2UafEbB|*eQe%8ty6HW6ANH`}NfiE4>1E$Y@s-y)UgE~f1lXs(kHbeb zh*0uobLKIzjwbCKk|Y%7LTQ5h7Q|HYJtC8@`@CK~6-F8gDaaQJNs$LHGGZH>3gnOTAa_h~8a`>fzTElm1a+D58Xo=p8=YFo?2-5KEu6SYp)mek{_^sX z&GPfC(w4tPG1|_PNf6BO7*D@rEAys2Es^@s5teyQE)P)Nhg+r9WIKgdpFEEz0Vh;z z%}XrbK@&r$13{A&)Pgf5W7J;ONf0C!U3Wwz@CVqH8<%y3jfYINTV*#RsOApQC%n!G zKJp9dxNnd*xOUl%TdK_H>lkb;WLqoDS{wn(Fv?=UFJJ3}yg zgX(hY)ANFOAIgPyF0YzefujNn0RZGo$uW=1dM?2ht`^ZAA=3a5%dP=iTzg>Lcg&M; zmx$0&YA9<%5Xye^6xN8}$S%nHPTTnd60rU_{wne1l4lCCUMFqWb6#Bmpz4YDr^#=0 zZ~_y`qKh`5Y%!*F-pN+)5`JmDAh9hY>COZe{Dse*JBUTB1Ah6>{a=cYuV~EW! z?wzv5puG1G)OGT@`Tzdk9z+?^DDrpC?gMOx=n-VL{=wn-%pl2B)XT%u8$T zz6`ooOem_CXC~4!os#2e|Hfrp4PBMT;@%~^YKp=RK zNhW`N(4nn*v*COj`CkPhY2@0soc4QdydrCN8sa@oc2qJRK#$ZT*FByF&$Ga~i^+I` zXMwKg+v$aUh9z~!bK)EjIeQT?7RT;R^!-opPpR1^eNOu}U)Qsvpwxdo*o7=Mjr9|g zLT`+WZiQIq%k0cjct_VCjo-Sl$!70PC1Jcf$TeyXtz@wmzn&Nl`kskqfYT#NL9?jbe*Xx|^!--OL*KweWjr^orR#Jf0*_}_+@V)N38 z{j3?8?g=nGICF0S8);#G^N7`HqthW;o}+y+2?x3|m@-vd6T1u@eMnwL7}d|7W88cs z1f!-?U`e;;>77#%_@xJAPr?6j(yza^K&*|0l46KEj-kjsZu_x4uweP%-(QFExPBfw zLHA2NUX6I#=$y$ht4H5!(-0Jy4%FhDs*e+U*&RiQPqlo`!uH zr5c!G3l1?_SRXH!3`p;EIP9t+%8&4sXq*&ji6GAt|C;aHgQI}~#D76qjK-9CQ#~3~5Xy)un6@CY322q|@6N8( z`mB7MY0s14Ij;eD87+RzB=~U%u*9UlV(&g(#1`{tro~VoatE3|DRk5>A9d8~L6dRm z+HW#=T#}jix-5#z)NHGF-$LKH)7Y929neD6S`9spH^A!?`NjkYeS0krlotrX71xwsqD;JfZ4pU6x%CxTPzh!$dl{$tfMjK0ZgYqhF=PP#) z{{!!r8xq=@n%z*qj@tA`PRiq$n=g42-RQu#Usv&b*jf(XjCUwL zVd!{hLJFe&t#ONtC|1|Ki`}_Yug~{pbPH`eBE<^$<@od(9#?IJQwIDlE}6QsC9r#Z zOvt^%%@^WGZS9qHK@yknm$;Jn7hy^cYJ;=yGAGNTs_XiTz#TG8uZDQArK;ahP1^Vi zsiuaIL8@ije2X7a7#3#Y?zXc7ZH+9X#Su7=_qtf1D1F=Pi@Z$d;Ft0#9>%J-!JR9c z_4RdRxZDW@5%nU|w&AzA19q_rM>Sm%$c9U;tc(Gr&{s&7|MKnIu0Pl9B{c>j9hbRV zQ0;+>L^L78?!eLKZ7h;uEtL}rhvV~(yGMdo3s+HySZz-AfbJlZz!A+Z1HWUv3^R9HEPU-VJHZF}^P!qr9}*GQCy6K%Y0o-_&D<-%ZzZ z1JM7T<-M*`5V)jL4b8Ak(v!X2_)Z+tD(r6YbOr#WZ*+qb(==#q`Mh2tCZwT?(ds3J zjJj|a(M7f{6>V@XqQqGyy`;?vT=J)!5b*)tqR!_N6Wa$B*IA)O6UDKtPn&oBBr-~+ z6HnIW=Rpbwa5*dlCDmk5Dbg9S?a(i@m{v-qW*#+{M!8ZXMEh`BDV;ohtLZz}r(Sc$(!b1-_Z#HDN;7d4 z^xppZYRB;7T4e3vi8Rt?6Ie$z0BlMg?ZMsCHB;b}XxD$Eo{EIKDAp#8zUE3!=pZ?` zy=rL%T^0DD&wde8Q6GQyz6y@$DPl&$sA;bfvs?q8C ze2@`(Z>RDAS82{cQ3;VvgZFA)0!k;i!-b%~BB(8HT01ELl2^lj>|yf!zn$zREM=v* z<#xZ;aIEWCM0r&T%A943Z9BIR{I*u1VKbvM_@K$?u&&*@Z93wK(s+4D;kslqS!>^S z2OB*B&Bp3P00onX1R`uhPr#YKzcaoQ$ov7CAb|wu3EX%~N zh-2A|upqMN>3_vEy50iN+;{VPN<%2XYA>nd4Nkj`dA3(Yv&n{J9c{;@WE=^ZnY7za zXL%9V2?ZYBcS}-P+>aBp=?S@?2er&5yF~9dxMJbdCekEp=9QzI{Cj#^Krobl@g*n( z?}n)@6=U(YsF3K7&Nl6Z)-+}w?D9N;J6KuF1>Y2k&AloQ7v^ zwKc0Chbq3CoqN+LhP;x7NUo^d`-^(**UH3jBmb~bE5n~%geNy|H0*ELARAx*)$e%j zU_ab_m#4DjK5h%B8w;%{Ml)mObPEKw;x>WQrw!K9-0K0BlRb8R{yQ%X(x@#ZOIo6R zBHma%zqa4r4NyQR80S9Gt$y(c#AT6z4BVgRC#5<^hU)sepjI zV+bROZg+#5He{PuE3}vp)fL9z5Vhl4g-E1fR{FkXb z|KCY#EXgcW3qHos1y~?SD65tJ7)|>VUCA0#CiiXl8J~*W)9k$QLyaMnRP_H&!VhvQkWSUU zuxq)&0y-6(^7^d-z@L^<{EhOVFMM)&(>5Ka2Ut5I7hy}i9~mj{XWV4H1w^pqP*GtDcR8QfI}Ul;=kyqj-^uhcTI5E9A|uM{SRm@`arwWECDni9yKZ;{$7O{NJOwNBCiM=m3bgxTqp1tTWFE8;peTVFkMW+3R z@+_0w4{WQz*}&GjJUhHt6&KQnMKQr~R2^8?LbArwUK?%*EpUagI8mf`6QbE>KnT5^ zr&$>){>ar2*|Y`OyWY#@<~JFIw~)TH)L^E0U^YhmZ2_Pb)crd^lWeg3*=%?unvc$M zx-QCry1UNvc(9|k(&jy+=Cp0gW=L-iRC?|w9&?*+H&kiK3Tjk_14`$E)GRxT3QWnx zx9l@Fc7+F!{X(}81gV>CQ$wlx-dTK4Vkr%bb)YXmiNfN&EeI1a99ZAX&Oc3Qa`nHO z_m9#`(?T4mTQ_VPS}!hFzUR=<&*=7E*&oqLJ-#pUugyLfD&2^GUvD2*t+kV?7wg}G z7Z$R2SGUPD%;Z%!I~h{`+yy7h%g0T5>ffTe0DtT;%S@~YNn<-vg%Wd)7iLidC3(p0 zDzj9EcK>~87dIfSEF2j`y#3-y#?5lSNX9nNU-p{NzQvOZ>O*h<@ z9AD9{kZ^NU#)NhlRVSKAF&q7HD%Aj7zRw8uz9?Pd zPvT+A{*HFumhW6ILDLJ83UBV9?m1v8zA~)d@@y5PcB<6ljDtunxS7{RdInsAUeWc! zF|2JF))rAk<^=pDxs?o%o8~$6jx;xc8c3;a!KBVlZ_W(7hFUZ*fE%hmiRv9V2-I0? zy^?>P-O^UX>?eOK(tUFGb_+->?7p70Vm~QE`*1{5-*?$mM4_$#M_E{f&ZoBLl5C0j zGDg-old-RhHOebcpDuipHi5Djhfr507+X+g1chwy_K>H7J@5)Epfz-84s5!Lno1JX zs^1$bzIlZuR~U1_n1ih25&eve;nz*XpI1Bj+n<8g_?dAjrrcv`BmZD z_AI)zFq>Gc2DAFLz8X^QqNxPm3Z9TYjLjR+)BQ4P%oV8o5ikO#%nxs=UN^|URg`m| z2zvpaNZXOH_|><`94wIU*5B0e!*i$%!5S~gW7))}#5?$GcFw}tKBpLr0p&HRS!hpL zL2f8nfU~W*_v&lK@@@VrJa{{)dB21E8YHxBhWZI_D1zGT#zr8l1E_D-cxmZPYtoqU zn}4yzSmR_IDqgG?ALE`EtLGbqlA{c~3Uc5S|4$SX1(bOTt-@DB_=y}YK9tKg5~Ge8 z6IxhPENT6!INyZfpW60*OwPTumLdF&1=l^MbITKt2D@F-|I0&s7RP8diqSrb%XPhT zr{Ob4MLf@jB(aSk<8D{$(tKU+V4InclLEqE{%UrSd+(RcKgiVcMbe94rHvi(8ul|O z^{wq~+jm|$-LmmUf8RRyBj4<{jewHj(7A?wGmg7voOeVbLdrx*_wT~hb``#Bz`8DxhP`p$|N|!u}sEB?>nvS zCN`zM`89*lVQ2> zMbH^4e(jfgok_@WEsklx4i+&HdmY|NDhOCCHoUUN0n1PwmmI6L2-HOZg@(rlDgA-LjDZe-H zwE?EvIeNHZAbCZ6hRyladO1>){MuOnWMSQ-H%4vNjgsq)2ZlzPOP1)lH^1L>0pfIc zY^1n5bgI~8jTESj0vMa`PWcNPc5woJ&nCa5&Fc+zD%&02X9j7^P@JIAeRxX;7hkg; zIt7X;BgXPp?QvBj1u3VE(b=%XwCC*QV*+Gy7p~X8n@kNS`U$^xzx^a`euItDiqtfJ z|4gjw5q@;!1SkCoMA7V?dh{dleC*IOE~w4G1@IM3?mj0 z_3JOjQ93-Y9YWe7;HaT?tc;XgG0n=SR=dw8?k5g-5(y4?4IC{z)fF)O49YmoqW;Xh z{gKC2m6`>B+u*?D4@5UQr9&BqS^W~}A0td`0D;aXA8w`_fj^`{PiTuVBzaPZpG@X& zLGI7BBKF0*irVVJ9N|Ao^B?5?1ouKlZa$_|_Ta=^Q51q5tjuubIur{e_CCP_b`T^k zR7ww|sqv!>?&I?Nuz;Ead<9|%GYh8;Yt*OEJe5OV9(4MKfv3Rayh!L3xKTjy_Lk}+ zRcBdcP3JNeC>1vfuuyz#KWBX8g+T_EkB=9?_&`x;x8=kT(w2r;aH|iif}n#c2Omyd zT613qFnqaF$3NAj)n|)`J-tn@(rh6Gj1+VbVGjImq_mNljT3$lwQ1g1yI^GZ(@v~y zH^xr5Yu7tSPAj|3GFz6A(ea2m5kfjxpi-$6G$kLR$gEQA-6~)784M%2aU!CD>XVvt z=K4}f11M1uMA-rePavw{O-9KxJ1++!;B4jB-ocMdnmWstXZJ+p+sQ2Jw@;}#s5UIV zy~ZQ>WbJJ8@8=2%QwEn_DIfs4sF~CtIl6Udf?P^9wZ*{eV*mNYHEh{HF^grZjk?`< zjI>Xx!8i1_YH^x%Z{m~++$rGoD99bbR|et(9%=z=vTOf+K>!EKN1~Lu$q9lLpPRM* z_XSyCvj-c9AEh0yWFX+OH*{Z0i9Fb7N};6#99U!e?r^At{ue|4886%^nYDpPmH#W< zyCcX4@u5soCpz_RFOXf@C{V)-%3G`Me|PQ(vX2~6%T~7qn&h?^R4yvC$KHtTv2EIQ za*8)uNTR<*oaBA0@oc2f)Y7ue@Z%-%WHFC)6`hR7dDRBbTx==Sj=!L}F1lU%v+Z2! z*^D=$^eVX~I0~QnU5Qahj}$I}{K&vQnM*`s;&JKjvXZX0c!{zKOMORAU_hlEpx$rG zHL5|#Rt$=K=L&E zr#_$FdL~QCLa(C4g7mGpr9f+S4^uyD#`$@B>?*BW8X z-bmNSi!rTM6LHYze1Wysj80(-6+!kOfwmu#;j-GWZ74zVyZivGB6Dmm zpuS-_Tue@pmVg@+!`&51h$uKpf0U+0VgAu55lfL@EU%n>7SqjcGh2UH4Wp7vuUx~- z!s<2R7A~{f$Iv4p9NxDSu*tcCCoY!Nk~_fAyPIXLcd&j^O2yRoB9}s1EIIzlSW%p5igTJ}y{E<8 zxq=s;u6%^8>(|_@tCxtpK({_PGE&~Vd9BBB<8LW7-vgwQ`T7>dldAQc+^T~?sbVz- zepr(k)Qa8*1?{5dHpj$hF^M~tt0Zd9y^9VU{*IY8;gOUSs(|ezrm#wXXh<|8puIVpo=IV2bYvvzgtsf)kb!H zK^p~B$Sce7#{V=goj>#I*M{?PiHtCt>+mjg<;aNsF{Ssp2VH%zKf#71+JNWy0)f_| zFd0vpZOe=CdcKO(ylBdDaqkxR5zZcNal?W4mP*!2CQ3&_=PMQ1uV8p9i87w*hif&w zuk{*EHSH(;o}Ci?itg4+ZO0{UJs?XKY{x-|1_8NXoco%O3zb63hr{Joo-O>Jdm{Mh7CB{D$Kh4@*>@nL5TH0KN7rZ7&;)DA3Br~`I(v!e zi89Ae+4y{iZK3)KwMX#iA)`WjEmfW&LfQzaI>CMoZ7bAKzv12C4Q#>stS|~w<-bBZ z1xHLyZ)EnJpd3Z*noy8o=j05Jso-ODWZCj|t^|YH4s3_uHr&|l3TpY@A!nf^s{Z2i z3(ZRd?Yq?;!a}$&=^{auxamZiEluRC9YXEU5y??)kxdeLC%kisRj*~ONQprbsNQ>_ z&*YU6h4W(_j3mv5z8@#oOSlhQ>V(ZLa!m6IN7}q?zF;Qi*PPgtYhZ}sbCYudJri>1 zc(!6XPLbM>@$D6xw$s-4XDe+H&(}9{fYYTKnygY$`P@f&*Uum;=X=gzdZFBpL8^!~ zyLs;hc`C9H&;Ww%1~3ItfNRS~ewBQ#9af@`egh$=f$fpftLBi$s{o|0>xZ&R&gc+- zl(PU2c+2S$d|HwU7k=wkiL?hpPw^Eew7jMr|F)$JT?7|-~N+xov+|;)g-u_{S6zI&U$Jr;q>tanO>nr#U+lrI?rV$ z({>}Z_-u#dM2!IP__!Sf{r(Yel)-VmKX1%YHTRw+`p8}C(ZVn4#cKHJ8a2HMr|iU6 zf>A3wbdSRZa-60+x^x1j?8ePm^{%?gQL+kHigyYo`IZuR?2$M-%1aOlgJJh^WasLX z*L3duyKgMoX||^)`>B+GS?X0jAR4t)`R9)ydSq&F`q01T+q+{FiI3tjZ+a#sT0Hv*s!tsF zsPU&yDk*3AyI%Lk65W8$_Qn`Zh}-qRbDu`&6cc&|oEy#YIzqs#;oJBh$O*!Io@v)r@^IbGlMI)$?GX z!rlGnIjoVz=S+O4lL-f~WI{lby6s;tZb|#Fo1%|Wi@Eu>SU$jooNA);w(pOVG5pUb zn8*&gD581Ny#_ve2!&FiUa%w8&L6H(K~N9&ka=)d-~J2C(qGWNPqiyNxX3kRdjKXw zK~HT?IgW6;QYKXd_+8&qJA1}0xd6+G7rHjO>+^;{n))H>de^G~d5+gis^8PBh9|vY zMx_ns3X%-uBz9T``_f62MS(BMkOZY!PUTf}r^bgMT6LKZDmNPkEC5{iZpYi0=;LmNcb2Ht3f z_O1*>`I&B1(Nu|@`ju0*JYC5he*XI5?5nZgV6!wIP5ySlEkZ=Qo=o>gdosMFl}S zbYMYbx-c`XCPx)LTxhUZ9YUiB-vGP3|nGupe*eIKg%OjBN%=a89_06FWf=yHj zA@TTrTKzEPTQtVgCofes*^zbX9I_w*y6gC2K_7BSXoGya^~T_pP94BZVzk~S+dewo zU9#M-S2+PX!_juvadBr&_BpkP<-yocsz$?oS77K_SSgp?&8#hHrC9EZ06ZhAf|%5J zoy-Rop*=KH8?!dSWt)yS$<1;Wb9x}S0?C!*62z1uIln31J1`gFDCX`Uv%mBGQb)J? z-7|300ndlb2{7&<5lS(C`_E{ObcNS-t$~txTIq58u%A=zUD?y(;L%6y&X<%OqvcB? z-kzm#JC_9!K&KRW`S+-KHW-xnyV6W{X3kCUVWS0-*!qz^)zakVUzs$JbG{{u zKU>!#HlDe+xaInzWz{Z^vq+PU+qA<%X!gblDzaRq8r%;qASQ_elf{%(B{^tLGW*g7 z2Q2kBidq!THBU`TMfh(q*E1^7wzS;r#wJN}7o9I_wfa14%YTsCR-@xj-{)9l=-=ZU zTK=T--?@8GthC17dm&1-QuU=pQA>{^;U{?aHfyD-{}rtg2Hrd_dbh)Vf0nzc-H(%< zOGs_;0)NYGNywBDjT0wJ>#SE{*Sj;~*~PYVV_HQ;G)Q$5%!V@b{$2Muj%g#)xkVtG z??J(i>nJQJwV&AUBpaT6P+;jYkYMoF~F3*Uh5~U6R%L`q~ZytEJRcT(+0SOtT7&j;#6K z!3)U_bnWcW@QmCsYietScJwZV9^AVMLE1rXCi?i?0IMLVp5d(ml74z139UZAmFN?Vub%EF$ryXB@&X#X)J|=2BY$jf+Sdl!=uo@$Q`Yf=zZp2F$P9MS7*`uuznR-MhPgN*R&5 zDw3I#b8zKK}EWQGbdEQd4x!2ueyqo$-3d^F|HR?kvZ0anDb)r;Dx z{KmKb9qiXd8u@PeKD3yr;672VrYoAV0?V)V1vB6vU*D>ENamy&XI1YcL{Qhv z!np0`;lt9q&HE>sylnBkDW!jH!AHQxICWnl5I_cFcHA#}5fUn)5J5{2#KCE*avb})!+4C6KA84_>(nOS7iFC{MZ6g(#2 z3E>2)0wI-3Qcd#|n5=U1Jyu_52pQ8>l>))a?+WTmSEz}2Nh@ihiSPMABQFOJ`X97p zGrf9Q==3W&tJmQ1Om0$#cOeUwc{hhgK2vLh$Pf)aHRo>`J*r2&w}hp%0I~%lgq9$9 zoI!<7OO{F`NRR&T|PZz^k+e*Eq6(b@K zRMzqc1o4jjDXJ!Vz4Z1P?UV`v@$vErARUQTlLFw+j+_1?yYoHZx60N7(Cz>T!a&4+ z_yVEENTmE^yvXPawKKWni?qHPA`22R1hDf;l_$igAM2jvTJuz&gVb{;Y3U%E(xK|1IhMY?0Ddb&Gn-zkuOABnn`!I<9 zM2J3IxJd3gSF2kd>~qIx!1KC3pGE%KU4h6L*@{~P8kN1=4dT*k272U_tI4O+ynU;K z-0)fC2Yce^K8pOQ9R1QbQ;Xr5HEZv@U$!ro_Uy^-oFeGRsL43-$7XeZR7|#w%-YL; zMU!iwvpbvPxsznSmOws_Y;6RAPE)E_u;7p&GWp*Z zo9XX-F!M6x`9C8H_WZ5j)qwHjq*KnT#m4(w^O4)?lZT|F*Qc_#uL;znCp-&1xleyY zs#RK(`@DP)@F%K9?a28fFW(Q#+i91Y7|~IA=jUhWapP%bC}0f9cUCR#DB#v8Z z14vu#K!b<^ad$K4|DL+l@NuuAY^lM_1(PlHk;9FCA@0wqw9*OvKl$hd->$?>g?BHx zKxTu-{?~;f4F(3?Q$c+98Z#?v!>yW-vdZ%QX=NQP8o$5k>D{K5Sq4{E_Q)c*YIvnh zz<{iVj6|M`f31yQ>@FKg><+H-Fi|kTTvsSylH{6kG7#oP<%fsNuOWixVH~7~7knuC z+tbza0B#J$7`<~XE_Sw^u4rxeLUI()6%8vF=keXZm$Y<#c>4XMc%2R^Cm+H~wj$O( zL&yS{BFY*Vh~$C{dFlP|C}Ws0W@(OgPg>{GLNpH#eQc$(a_DY((_+^fp3AsnDj}-9hX--?<*Np zp7?z+QorMds_(h;{)?fyuHawH&$V)%qB`Ns1l`-+qrz4_sqmFE*v~kFa-x6MRC}w@ zcCA*U^QU#dLEIRS?VtiRmIFdR@+tPv+JEw|Vg^02DY8^&OS}g{8OwE2qLS3lI<NWntBe{%MkI6{6UL$3Jy94d{ zA`xY*cg5x@!zE*qM7K)8f(D?cSzo!$f!jNOUodgNb4gbVllk0NL_s-)Hb|{BMb&bR z;%bjE1{T2nO!b8L>4M$s~qLwBZ&M(6fm9~u+9t$F?=12l`LTZ z_x3%x3@DuLFI-rV@l56=nt5qBJ&yUruY=;DGp;}0eA-=2F&UYBlvm!~*O@vo{~Ly? ziEmkGdyGq=yf6}eoS-3jvspf-GsR8!p^(m|;WQ(b+b$LUhZYl!G<75-N$HQ~sY6Dw ziZTKb$YZ~3xj=jO4hJ&ZePCeVNu5sNNtxNw*dzK+9oD}Cr@SKWm2lK}w7N%Bt>omn zoZ#}L&@DLO+XyM)IbX8Tf0a`|o=o+vx;J;Q=vyo%YCa{EKk!r_Aoz z<{@>2)*-RgjEI8Y$pjk50^7rQtAnY1qvKbD@OTweiG{vKtF@Nsp)F6%?0Lfo)t1ZwzyYDM(N15+VeB6l-BTX#@1~XU2?R zcpq`wp?8hctno;I#m~P*oY(bULqxj+H6h7v0?ZA2HEIp>o{*(;vOsZ7OD*$_3dL?c zJM*em^ne0081k zFcRC@eH7Z>O;>Q4LIQQ^Tba_d`Es!B8T8}R<|?Za_%}QeO0-IYNY7TU!3qAz+HU$d z@qf3{9Lit7gaFy`n+P;&d;#)&!~I*Zf$6*u+Qel|DMidWFGSxk6iX>BSFe17>sMZJ ztN4{~`Cq`KIOon=*&(*Uiu;+Nj6Tgme zPZj*%Dyr66@9*4w%_2PExp#}QjI;W|k^h0@9;m02LRBC?FeeA{{gDnny~?>={{uCF zE0)NbNtPH7)JmLFQ=3Pg4Pbf6klhrCCIoE|(8#ds0WE?=AoOuQF1#sGKC*;w8a`8; zbZF-vE4B7Aq^k@zNNT_<@^5n<;XLt^RMo7;Z~4OftE^KS$R*yX_#61)YWGWjN*~AC z^$`aVJOef-CS(8uvRwH7>YcSQ0jE@K2uSIQJj-d`SI*+tq77fJTsr7;t03oVWwTB?d9(jg-W%Eo{5$=bX zw4DnHw4*kU-gf!nT=KB~mo0NrQ!=LGZU<^OR>yT^kO#NJs&R*2;e$7_4o<%E1lL~# ze!(GLFGk1hdEhHw*cfBWEyk>o(@v1|oVl#DqnUG(M$=_32?7oWyUsHEO5Xi5!zpWxXq&ENm6A`pJL2djFQfIdPPE*cb^1GzYCFdK?o$Nj z#HOef1NWb1#}$(UBkIWP&BTLTSzTQUm=6gad!3fZ*jczzLDwAm!eSIp3e%T0YO-r@ z!}CNeu)AUD`{U;0!rsy$W$al#1$!*5)8ZDK$X;;MO>ksIbYeo**2TdxzYtHi&j>2m zMXFYn&#Tm!F0_-hGc{c-1KX;xtuA(V#6ku7Un9g$Prcowd~zFizNZ+XxNMb=y;}?y zM=f&PP)Y0UT&BA%R@I)yI0(}_N>4AqC;M95{8 zg_}F8mg}JTBno@ zipi;-URw!}a>Hkn%yR*S{I5;p0=A>zd^FOeo`B!ahppYD$Js>+OS;@^kl$?T!X1^Ld)GfQ@Y?43G302)O@-LJ*oQsQ z%*H~mP{3&k;uQb0^*ss;fJB`~E`NQl5B^>5DI9YcN861EuVQQ+$Idp{#=6SqnnO8v%&O?rKB9^`8MKr&nxg+*sTq{^1X zYXyP|oYUB00Xl~x+;fQx_$cdEbMou-7k@GPCni7T(Za8_r3yXBv(28R@I=2E82o`b zft1pmY4jN5i(qgb-ko#cg_x>l>!HoH)LjCRmke?x%o;ioYGFaoe($Tl=M{5@HmFD4nX zAc2(;lVY$7}Qe1i$bY;BpqR|8phFIo4JD^p={3t_q)o|ne=n*{Blq@=m# zCP&C+L`2D!RHNbqXbX8=7bP1zlNVPV#ocDaL~E`x5eS!z>|D;y$@{J^T_OzaHPO;T ze`b2ddCG3syw7Oot^ecqe-7j*ipy2;gM$NGcuP;E#ac=B7DQGpU+e9t!};4DZ`08E z$bIuh6QQbBGc1e2y~a-oux@CybV5_7T8+b6#hsz1zwCN!!pC>8OL2?mHdIY5+4qk( z>=)U6k0gvkFB|YnN7|GsR|5TCO*H%&VoY#M4~}Q*S{1p*P;#pHmzYcEav=vPU1kqZ zA-~2%7a`o1`EK?FkDvi*so4 z{&@F&;-vNO4ef2oj~Ms13^qL!Ez}NzAmecS6!eNa$)+pST!x|XHfiZ8S{K(#$=C^A zRUw#LlkRK4Id_27St*ovoi=q~m2)TaV)i{SAaJf4f!pF0QG?3X)1rECBF>g^{+&zn6HN>a#l&#CP*(pm1!#~wb9=S&j-t6YLx zcibp*3ry-A3)tW3=!P<7+c$6_zHr=V`DmeFDSIV`#Z{k+&zdVS-nu66`ZW3aN%$KU z_ajSGpQae5!2UVg|7CAIl!|8114AO(UwGrm%+3y!h@`e!$hvcd0HbLI>wICWHGQrV zx=fLj$-9MD%MHR5vY8*iqzf?y0DZ}S1+<-sI1OE3JYQup{spU2grRTvfcZ+Vr|5x0 z+3Gn}gjs4J6*;3!Y_G8!I=}R$`BQ?BAvGa7w49vpHsgIi3OGGypV)kXDF;goA{kFY z7@t@C6~dbZ^usiXy{zfAHzP3t3f$_o2DA!V;JUCfy#zs_2bw?I{{v+K4w9kVZopEe&LC-1HFZHnvxrFiP3_s+@}Sx$iw>0AQ~% zI6lVnI3XPc;-VOz=zs3tz!j6KpDQe!SY4FP3IsZ@v4(zGC zR|^`49%5!>qW`n~YJIr3#Z6QicS8;jzyHq+9O+HQ<%&u#CP@XsIrw*%-AX@3=Lu6c zmJd=;{c%WdERy7<)H)!nVu)K248ZG)Lf{`1)7EI%jf)H6g6yaZi?}2&qH_sy7=D1O zWWq9|{%1b+UonaH1>C0+_wn054`AfrS;FckuNu<5^qp1$enOy5+6_nr6(Uj{3WnlU z%b&1I4~%kryK`?iXFGC@p?CYN#K6Sfl}e%(NB6NW)pGz5NmlS!_AH?wV%;LQC@S<1 z*Jj6>!{36!=Tn@(Lzk-GLj}m+`() zAkX{d`7b@7bTe|oAmJ%K7(hi&T70|qRIH)^k!8VQ-{VY9%f{ed_bYm2 zY3b^)*SKLn7b~fjzVEzHa%n?DI>9S0IOY*;(cx38MvcnA{etY57%q$LWQxL56)7V< zyFyv14e#!=^M+mx%FJBTR6#jC&*i}oGWru4iKW)4hu>Sy3X`kt@_05Zl@<&v^3wgI zrpq~v1U%PheCRtb&SoyVYT3@W&6fhx@8?_NXHr-j^cqJ&?97%7^_8u6_z+Iz+}W#8 z&Bv5YXt>PLv4YJ#>oQmY+Xm_N*&YeWo}$J(iHw`&k^ZI}T|l3^T%E2|Lm`q2sJF8V znZ0c|Pdp|-inHs&4_CLKbUbH>(A91jY=z!kXNMbW>+AYQn-h6j)e*qMBtzda@%3nN zLF1L1Vm1LX#><=Y7xO(43}n>d`o_j7#GSoiA-T9NaI2iKM1lqk8fBl(&fMte>5;;x z3E+Pkzc$LiXc4+xljAv_wg)^^72eXGD`iLzXp7$w)CC%g0!eOwXs8G2FJ$(1CFJWH z*3yBfETou`naMTrkMp1KPW;-#)1Na9!t3T1t|m+92e>%){Wt8h3I^M*O&NZF3*#!9 z)v4W>UFn|jiW!nSxee=O*q9BH?biQ!ZUO#5Fgf#~rm)0USQ_=I^F>OUsMUHsb>C`` zn%KF4K)4=suEeog-Dl5`T}Jx3kxvV_1eIiY{Ks^?FR>rh_kH}&z5IP2-i6li6&uh0 z>>x{71otU0J}@KGEC=y<-rFH6K$$2{C95iKA9q>7h1YKQG4NUO5JeDWCmY1Djhuo; zFyIn{ux7IS5`h3g@B!8ughR`T=Q62el=A$Es0Vo;MNt2r0TJH#R_js)1P|RhdW!<& z5FD;BLGxwjlBwNTSZ)%r2>$g{JZZ58Ba&_Aq|bJUhB@6s#C|jBeNr9p3rKl!YF(@G zPeZmf2q!>hDuYUWV)PYTRI_<85a1BN+RFNH-5kjij0>X2=I-GhV1(sd@g5``H{acO8x9ov5O}*OZ{TOSC;YxLu^y0}}JqEi@4E z5d}Xe?c`vRKnev_wIw*?$F?T+{VBm#1ACrZsxZVQv@3xW=X;LtdJfkA6iSjYk@8d@ zf$MRh6E*O>3*i&%U6QZ}f{e&ey;-{?P%fiXEuTmr+J!A-!Z0Ie@Fxfl)4P3Y3vMG2 zkf(SyX)ZA1A*|oOVbcDifUOeUn>KYepct`L{7SNLUM_xQ%L=Q zMp#I<1tFr9%v~_ZQ2KpO&lYxe`CoI8!zg2)+JIpelbfz$h!{`2fk$Ifvn-0iy%ZqEu{;7un! zq*EmPjo(D6$v)^Z*zA|3kwtS>KDvShG%ZSKWM|eNPq|!r1j24Kop6B zA=T7K+2Uq#$D@mj^?+4NBIL2%;2ja7J{1+#x-1-p*dQ>LEq`0_{?Bcj!ss}@?yj93 z#&<2kD(ku3vx%jyCmWGsoCn#oSZ$E*cw9XAM?)}RJ`CgI1>L?=O4s*lFJVdxcmIpH16{1>bLIsZ0bi7WD zy#9Tlu>DD>dNI|(5A$EW=dL*WxR&G9`TA&){^EySo$@<#CVoRj>*L#|#AKOPI zSo(|N$tW{BWN(t}6{3*6SF%IMCVQ{%y5H~nI>$Nh+u`Yc?#F%ouj_aH#zM@<6gy*R zcRcD{d_J9nP{-mrA(^mmpv?>YmZe@Onqan8z`PSk&BX(xM0#k>sbl-zO5@K%#a*>g z5O(J=06EagLj1#8sjur*R)?4gSl_0DrtN6R1KbE?YsfAa!pI1-CRpw@s;>FrGLhVsZtRW9}Z8n|BKHvmr>b>v}Rd;bjDFaY$7&MZQBCV$XmAPI0fjaD65o`PFKfs~7D>p&E0Lu_a zzSuAc!_eH{YFFb7lhbx=#`z63HN@>6K1xI{hqRjcDtSL=XB&%KxY;rb3PQbvq6U_k zD_;mfwhI5I#bSD)b8Ab^weNk{l3r6y&`Bc_$iE&l*3U!@byUTPu@i8bYM2KrQP}?cK)n@XeO36gF zJj}do_(vqs99I6ER&R(xq3{xEzQM>bZcE?5?*9Ea zqYvxS!5}?^yR>pksPbg?>cRjfY$pWsh1mJs6YO_Q7V{7dvJ6VVmh}Utaid@)Gb9}p zb++V0nRI%WaL48$K4qb(vLB%{E=KvD(;&u&SB+8L2O}D>1IxAG2z_}y+Nah77T#J= z$`F|~gex4?F2?I5^6m&vVdmD?ry=q~=F;ym;F?-#)+6gC^e+z4BW*G2 z=;NsLy~=G^kRUTcNyDj?-Pbv^f3?dLK)k#TGlc!lFBQe1pBxQsg-y!?(Dm60oo_!M zEz#?NpXq(NoZ5oLTHl+scf>knw@TUroO1K4r>`qs%09qa+C+c$IKyJKVsWGKD7`Lh z3w*_-xFmF}IR_qroi_IGC^`BbuvA0{L6CAaS~UIRyw+JsEzZI~yIaKh=%ICCirgm; zXgVicJ2pLz?T?%XMbOJ(Vi}y6=*qa-_q_MQ|4lTY+TtA3(E8E1p zZC$}LN6JhpxAF^jl)B8BsIo2pw+JyN|7^y(ZDwf>&@^{rCd{PhVL;Q@L3%skAPwTF zpjVo%6N>A?i)?34c?7eXhLk^j__tJSrc`V|$*_adpqc_`#yt`TAqA7#zY_;r2c!h@ zH5wrTMxJfp-pc)95~J}4qHpX{0CVgvs0{WJnN0p2kmCsE=H_7`Yu%XbAV&e;59lrt zzGB9y^2pjK&A}z*Z{_fVt8e-!S16oG@F3~Y{*`$ z7R7(q32E5)&kt4(QtXQV_EQEJKHs7Qb-0Tps<;}|$}iltJMp@xkaL(Ww@^~r1|0ie zN|KDpnT7R9q6k{qtf%2m+XDcB9?D}GythiKik|Ojli(df+EbGh88~wRv0PP+jg2g` z()-KJKoGTiTm!;RcYD*laIhZFrj)K56y%7XlS&2L7sP>}Qeq61$^5X-rmo8xKM%FD z?Or&TLBOHXY+U`POy^+lR#w`8Rk@OZ#zns18YDjaM^rM0S_^BrZ;m#7>Y*zzTy0X%oi^9cd^&-%~M-v8K8VUcaBe|$0N7~ES9&1}HnX#m< zC5(UtxwV(9XI`=uqKqURNK(OQI7^qdPFN7%_e-|ozB@o$gKiuS2@LTb7&g0NMV7F92P}SVti3hYOys(AvfX&F~ z{QRtniWoh&i9jHlAP%W9U8X>&ZmGte_|r(1&=W{2ol$2uS)z{3Kiu@Zv1e}Lh25ra z=Y4Fy7N$#|xK|A0ua-)0LdHQ$ztiB;Hql?DyUq2199gqg!79ER+1Zn=-i`hEWF%GB z9MqF}7x5Tmuk=5<_pQ`M3QUPS4-09WeLswk*=xQxc6sp^xY)~Z5X{4Rm>aPf1Y73z zRfh6cE$F(wn|Et3-Gxn-dXtaKzfX z$q@0Naydwvg;D1{oOOU+oU=c4Tp zw8{yE3q1d5A_aD(kP-_$Zx7^MBio*xvx_O(Pa}mIoZujfw48@CukGU>>vaQdzYnan z&L2Fz1XQ_+y@RZUb3mYlf9w0+gQrUgZ_Rq*9bxLO^w=@}f9IdQ2z##u=cfr1Hs;WH zeh1*<9ORz>Pxc#q+pIQO{h5pCcD~fqIDg*Gl7 z!CT^(HJp%222>{GqU{<^%n@gtfk$W{w1m1<+IY!_Ah3uYz4p99mKS`2kP`3HnDHdO z6cz=@nh7x!t9P5dIjH?q;OyrMY<|m<5;M~ZEgrjxZvEFA-Lj=D3*$6Il+2h>0w-ix z*uU>;#he)S@huIc`DcIre%#H0ZNR3OQFCK_Qn>Sh-3h1ltZ%|A=OYuKsgn5|&j|o5 zgFqDka`@`p9&QX6x&|6YaM%pH#|fAzN(s0-|CiKIvZI!VX5hnPJK6eCI<`ChA+%zb zM%5X7Sw$SYP(JR8Ck~DTjaUsaFEXd~SEO$w*-$_-ysc}T9XAEoyF4X_7OJB5`(v;5 zWzucXgIU^IQ9_1X|I*9Pg-t--%>P!;g7#MW-fhkafw%9?ZG~J#J9y6yOtmXlN{dTq z)8)X<3z$B@N&Uh#3c>E60x_=CR#pW@{Ih$~DoHFLRgqR-p`b~;dm^g&X!M2fYtkmE zqTcFRe6V;Je@2`B!>)#gx%(@{QfR^aPB7STL$J(3N#Wy4TMYt2h}UyAxenqVRqZga z)K7Au&kH-v44DJ1_~DCdqAW@60U=kVh~S=CWs0ay9ew~QhGi7jbKf#XVU&0NRHpZmvTDy?8S5Fd^H#;A7cJvKVrv; zeDkJ2W(>?CQQEQtSc!*}cs(Nr3$rU3*2dm0wj1rRTD|;Nfy}-tD;N5^vbv=X@VCJ_t zqR*^{Ewh8NKi~E|N5TegJfOOvb1Dcjo`%C%>;H1DuJoj0T9I%*_<9BrDsv8rI(D^z z(C3Yn;c*{dnNQcbZh(6i0^qF<=fwg90Bd5{qM~=2SIVXGMk5i^z=I1yylr3u4nC?j zhTFbBf7&ihHM;El%D}SGrK6EJ@LGEOWubtleAg@_)@S8!%b4lu*uxFSvWV5nqQ4w| z;=U~J2O|HOax^I~^Co_eO=7U`89v(ioM`Hv%)6{XDBvvl_JCPpz;z`Fdy+xRW1gp& z4l+kBODzLyp++DDB%Vd9nJSp98J;?nS%EzQU_rHHT%B7W1x)|-W*aIx*^kIar4>CN zh+~RkdNF`$$?>%5R4A^tvOQ`nQ*m)AI23YW+J{SJ$f_FS?&HgBqKApYO+peB5OEQA_|_%oaIwMJl|zArg|SvTn#PQF$V<&@y-R{e+BaT3BsO) z(Hjv1HokWrNa74emVAg$hU@z1j{k%6H}UaAh;P zZXQm(R@7BsU>4SFf&LUnbo($=ZKHm<1OP$E1O}DU`RU}WV=Jzt!<2lZiHFBK2m?&g~mt?nB0R{XXUDY zY1bH%kjm}$@`Qc!<;&@fTzD8nhrYg}dtp4TAtXdoG|NG0s`W6; z2xh{zNqKJ|9#Zv$!rHtwXhsBA@nh?#$ktcevmh^H+5jM3o@c!~_E5N(D(l4a*aQn45?~;v=tHRATmx7QZh+ z=m@B9dA_|AFgGuNC;p&PiS{u^ZK{%N`N5sj?tf_onB@)b&pJb?smlW()`mK+D-;eJ z;2mkYH9X^G0@^4T#V*VPn?YQy4c;VJ2m>DuUNpMfRJ@!q1c%WSDZR-w8PeUb~~>f24#dp3wy#-PF+SBgt-En=Aeu2*MejegD-{Gy^({SATI4@DskF zf+Y;b_)vqZ)3N{fnpnemiudfl?*xH8KW8hrVNoN*Xk`@@nslGjmyW&c9V=7hK(?7r zwl2Z_2KXFL_~?&k?|=YmWo6ZB5+z8FjHB?vNgtUAUY%DT>BbSV&LrFm-3>9h%{>2h z)-+I;VBUX>ZfBJ9W)@TK;+T2cMPpZ}r~se71_jl zYm5|Mm+u>KXo=a{Ze9)0`7Xjoh(rbu@ubK@OCu$f;ZdqeApRRxFWHrqD&w1O<??t?wXBqK=lr z(7OC^_qFOBG~&~fJHq9TIg;$q{UAEmQvYqh?^Ug)aa*^4BY!t*mqE3}S#wdu=y zpEOxQp{Poxf_K|IOa??i)uN8=I^QD~wrHL!LyD@nq-5~t&!2JVW2I4y!G3ohiVVI_ zqk&kCacfXM4bv2O3zu6C zQ5#iGAfM{4xL!e1W9Q)D`_HAN_tMhRj90CDT5W5t#N{%C^xy`vLrb^=Mk%7_|4wzp zXz_3-Y}7OGeITecfE5Ri_4_&4NvlE8OV>Fs&^%uwrB3Myxu@?VPk?z7F0Abvvqp%9 zz8*dNf;@KI?P3TE4aHPeQ=>Z+Hca=}X}cHLv4AW2!21iB5D>^FJf3c+*)1I&9i;~o z&A@>##mLfWn;V+;N7*GM3iub_^+62ZNFa=I`%)16`KXZAUl8>f29?kWUlC&gQe8Ti zc26r{C2u^;Af`+ONaAit1J!JzH{s9|E+%bop6RKc1)?*sFV|Qdyy?o_#_ti-zNl!z zFcF^>EobC8#10)N+60Ka3(leiy4?rynUl}GQt~*HEP!9f_JUmZ6k{9+$(o5 zd>NTuBq=KkB=x^@;G(*gW%&gVYnp(%mg?Idu$qA_&%S{b>LT4rE}~ZD z+!p&1H-h0{@vVc4Zng+-G7gmPW~h)@Q5wfH_99ez?+nuzlRorv8DJU9+ySLRScvZMWCd=@stdh=Wno zsPS=r-hbmDo)J|hfQ4J2FahC-s4{eH?P>P3(oOslpD$@4A1F*r`|KOB>_;)Kbb=U` zusb?v`?fr`r9jaB z`LB>mi0w9-zZ5NA^YUJZCJX((XZjR+=;`TM+m(@!7>}T$ z4E&zLIhs+@F*{?}Dhjh6U>vxG!+4Ltadn3-i&0ce(#&z-VC^!Zj@s@-S^WGmw7~M$ z^5M}g{+*pES?2ljsVM_h0xF-w;+W$z8qTArqiOeym9nD{l|$bhU*3hPgeS9JDhOdb z&23UzmC1RFl~5eF&s@c8Ayu+%s3VoK$gOd3{h--~C#B_OEMWl5qTy;RfoSd zc5zYVN|AR2+2XB613en3o@CIHBXVx-Mfg>CUZqV4RW+53`1Uo+i*LK#%)&8$qCv$Y zg91@MUMn4^Y)^MPY1)W%Y^Y|u_T-^#i7_3_4Ez|q@n5-Xg;n)kq(%FpXI-z)3b&n& zBm%kZz&TK7(f_Uq;nqeR`5ByO{vs<_8AFdpN`2^hiNeNVUJ&Oz`|g*gMorLyzWg@@jAX{c~(w_=}lT?gbaVw&lA5QmL;FU(FXUGPlzB zdTSSHmW$c|z#v*W7FO!LX>DzrncQTE153!UGb`)!KX)*=xVVI{1wb-y1Lp=@JcakZ z@~DBYci*M&Bl?Vi1*h7{>eZPM^OJ2}|9a3|kA!mJDN+rVJCsdZ*?Le1VQagcRAk5F zX$mwNczX-T_#TDFU5asqbXl5~J$a*FY5_mE=lM;_0t*}*;m7S7Tdteo--#YoyN+To|C-~70e3+{YFap5BmTRKo;(ohcgvF>&aN7huk`U)in zpB+@&p7jl8oBH-#UH7^%s8-goIdElv67C!GeZ8J)pD#_Ds425!(@80Rlz3y)%9@`-o^24n*_7)==@Y4C z%oSHYwAbd7^@3bMQ;V%em0z?fnM0d*vu-UNxJP#pt|aqa>1K&FD4EZFpHZKBQCrYo zNNvKC5mNQWQjqJnOf4~NFcp&K-+(Vi^Yf~&0Z^bfqwxnA;6T22ffBgQY_-dGlumi-i2SxNuubtU-LVR7is3LY2Aqs zU&TII`oR2-{-+1OE_@W8->%8E#@yZyfIJ+V1~J9XsBDaio38rw*$9M+8P;AlyFJBH-`S=*PZ)IUlbQR z{o{sTUM{XtA_c(%RV3oiZPwczJP!~jTKoC~BDZhe2mIESML~#O5Al5f(67w23lAx! zU_Az%z+wnP`X4BoK82gm5%t5U3=qFnRl)R>brE~G#6oIv|I|P0`K%Pu-@Ybk^-(Wb zBPv>!`#eNFea@>m7%n>%f(^W5jNDgwU6cHE9_Kr4KPmP)#4^8hv5mN{@WpB6CC|Uw ztoDqO{PKmj=Asu+GgCG!n~leIeqYGj)qd>8omHE~;v*;XjZPySD9!FH8~L~9ZQ8)m z->3MmJ?^7W@5>X=DR$w-vl{f@w@4LXB$)r46FFZ{SYlOQGQ8@s^N3uM{P4a|)t^cr z2aR<0VM3UdM2<+lD%8XeQ(HP+^y?8Ot~N~+XFf^w!rw5$+C+0-%)!skhjaF40v8iG zP;BZCMiVVYv`011604U%WhwyjJ{G;Y5B>cL$P#^i2MpjNA;25injkgkaN6A9Z@)G3c*c= zh5(fU%f4|Aehh(jLEr5Rl>glU-%ETY*=NMJ(R_AeGmwuN2!r&ox9atyrZ!3jJ`3yEtw(?a&7ny^!z8K;F(-j1cJ?1I467^K z3R5N=DKB}FLP5~9FVF|^dx&O?M2>XuREV#- z{Nc;?9BEHcAp&tWiV%}3?aw2m`vKP(Lq&NEXvwy-hgYROR%kvl|NR(U6l#6;jf*r@ z>pOl83eSo=gzn2`4X+Y!+==a8Do>REO15@=(laL?b3%j_f9M9Vr=|!~RcZudnc*5& z9G>2fqF=d#KG|UUsqz)nQoB5VLYmkDheDOo=Ndo8cxLl$D87yOO`D6ym{IQ#pw+*{ zuE}vNmS9-mwNZERByUh~pK>K9iQ*W?H|9vsE@ghFke8!cf+zbaX>2yas9>~?R=Rcg zDPJmfK@}gW0mF_eQC>?Bw%VZUq%BvT*gs7U|7r8K%`dT!2t?lGwQ#%o+zgkvPW~7j zWdg>10s->J?K!t-r9YMOOI4}oO?=qytE*1(itf^kY1=!~5dr2X8wYNSIrqYz%NMNE zuG{K`H@}UD(ngzSGKldQeALaKbDMDFC+FNtXEx*_cHrL(murQ4!-*?PO(4Sf$M#?f zaf)JzDH@z5*Im3LDiD{eZ$CIj{!!ijJ#+jjrbkjZ-K~>);s`W)6KSt>Y!dlfTJbu2 z_#t`QE?=cSIzt_uJdHC8+>uJXoMCHw5x8UNck^PpI5TyEmh;tZPvGz4@4wQvZZhYt z@yXDNy$ScY){kkwLq!YT8@yHgAzJ_X0uNyd=|S>ceCdxX*e64%Tj6qsJjuix>&Azw zf3eK0BG7W3;!faY$CyS)a$;}QJ#QE#Nu+o#kY?s%;BH;QVDivZoQf>*Rt4&|hrll@ zR~V@1h*9)I5!1t<`V znxSiS2F|53u+fnUfd;|%gM&C(gS|(6;AD{uYZg*OAU~gI5~<}+Gj{wjXLuFS{B=SX zi9}&(PhEX23L42WnKfCSxhgtO==E+a+B=D5*=M!88|iag<4R;>Ogosfyfa0*abSkU zCwP7ki5AOjTK#*v6-}p#fuP7kYpM2(v=h(kIfqTTlJ(BGfRB!n4ZaIp6=s4*q1p&=gSGU>+PiWz0l%zgHSQD zXE9s5nB(bFX^te?XsK=Bo9%!C8>A3m?!h$tGj_=GFX$eg=M5_tj(}zeX}J*vlg*bQ ziA1?iBjJDjmrH#wm|=y*2{+!9qgMS^2kZCkYZ(_C7PUZdJ3%3tQM=+%D?G8h7;j&Lg9h$#V_t^1~HSKqK6)tt;Ag+g+lSW=LDpYQYrWIYD{ODM~&M6QN z&7UH6NQ&OT#%uwLia3mRlB2UD?}l69gcvXHlT(+vdWBLG>iAY0t{3)P&^eWB^+JR6 zsEJ>Lk*bTsM^UoJEVGnZiZ3OPdF6SYiukYcE?&8it2mhmiOLXn86{BcgjmL;mt}-# z;-$2S2KTT_CO(QQ{J^9JB^FBG?Zcp{1_2^j!h!=nk&IwVYX2d_OwGq8Y{VF+g5xZl z(f`_<0zGpkFnx%l=-Em3H6Ja-C#H7)g_-69vzl62nro31bwpl^I4uxk3=%wJRqfd* zq}j$v?aY>$Mek&;V4lJa*bln?C;>HZT7o*3VP%stx@#DJH~P^BC4!rcdHznh5?Q*R znEzNu)IRL;$ZOZgA7%W&wB{w(in@Y%0#45_az`;6l#YryC_*?(OABrF8H}ZGdkQ|z ztAA7|sKJ=U=!ae7zau3|xM1xH)<+?&m$(V2JO<+jlqSB8CtDJ#EO% z>zj`$7iA(|ZW=2XpFeb*dOTe#jULu3u z*Smh20PCiNgak2h;`;4Om?g8a%6Yc)6}?086?oqwbq<~?a<5Hm0~1qI(_&BFqMaRc zuuC(8QFH*vO{ryV|Y{dy&Nz!0Pe&*}?Z!fR&eY0TW=km4^`s(p2y^YtW>1gzok#~e_|CUd!Zybta z@#IB5Nl)zm?i!W1{(&_U1Gp0Nb7p}&yI zYHn&8U|T&ARO)k3h_tMr_c~9Zu`06ue!5nOK%>asV;&koPVmsUy!7!B0%=X<0l9Px zEp%#&PDx!|(l8zX{)iL{qz6TMT=^%{E)nl8EAvOJY|5I@Zr{C&)K_Pi-%Eyez8kC^ z@WDbxC@<9s6J7zKNFhv6Ub4uMt#fus3=y6bI*KU#ur^r#{dIZ@opP1-9pGP1ZPF89 z4nc;cfx&D?|G~3`#ys`HozwMFgxl^2C3APJu=MBZ4u`!1(rT6)HeG{$HD@9A_I|#v zFPwDmIu^iVRi)%Zmsw>%ixIN!ar^BkSv~Ehvow&=*ncA!nd^~j+2W_vWJb-*D#zXh z!x8ELL+HZ)0;QsPVaIa*N;@3KPa>zhJ2+w?I>5?Ai0-ndqFIwxVQ_4M{!+u*HD;Bp zUau+sI0O1I?Ov#96(@KK!Sz*&&}&S5U`vsMn$|r+GNPTP^rPq4G_~QMVc`}Gft_0~ zL*AK0^LnOHP~FH*(3+a~n-C+Jj{81IjATd)9_0QwO>W5)?6g=em-|a zF{ncb^iVTMI@y@N@h5j?4zI3kVh-AHb^XQMmp8M(GVewg!@r;EMoLuqGV<8Ko{RAV zbHD{fJ9P+8inCG=`iDa9W>nS*FWj7=XB=s{CNd~#kC1oqb3&Av^EGwy54$Q#bF<0i zFf$iV!CKLfx9Lh#_o7YdgQs4Ct5UUotZ7no2%~?!9_M^xHXD&N+aRgA4Yh>5`}-HQ zp-MLtYW&6cWTCTUdJcsq+AQf!jmo10C6+3d-Sr6iwimgZN4i;Zynl>X@H&jShv!@} z$JV9I_V#-#MoUN6!X=>*V|W z-ET@qNTFQU`3X_)aCuLQKoiMgUb(i-CVQR{o?jigh5b9x`!{slcZlv}4~x*Ie|ka& zEK{QzEnw9X$e5XmHyPD%aNy=N@&hiWkMA#{NU!lRkC`oR%aS@`wh%49h&YY*&!%qf zu|dfwV;nO|@sTohTF%3s+YN=Y+CXH|L7`mPd(RA?Z#d8=sLKiJ=TI_DOa&y|#}!%P z+Lj$nI7mY1f5NA>bTFXT5$JTYULkmPcxXwJo#%dD%c3Rzm30JKxmb`wO{M(#*!qEr z0x{>ZX8wru$BGr#j16>a@E0T@u@*+;fJ9+|rCy87w6W5N1RBY~vYu~jNAHF@g!p2@ z!?0u_e0Cr3_v?Q^2WbRE7(XN+mLI;YiJqEJU5Xyu3qEa*HQO5iX}Ok zRhh&8No*muO@h?675M^!5!R3KapKbYLlT%Cy@V7WCtL?j?a=hJyrd*LSl}VN z?#Rg6{!D9!f zzNuioswgneoIOuwt!N)v*So0j`|1oN~n}% z*fd6e087jQqOGF$n##J1QYT!adf|aLy3$Kj6cUnkN@ScL2e&tr;~s;{W_2CgDs%LY zJ6Slg<`9MYh}1`kb4!rCga_50ukZ=kesyb|u~zY`e@8BI7Y5D4L$^LFpvKUeaNLJ8 zmWwXLl&{0kK5?#`@j@_bG{TuFw!^N*s47uM0YxD=!i8tR?~=k?7Q}(@Xutz6;r|wX zu_tUksumbm6+FI1E7EsH5Dn zH{w2>CgWbkekH?(!(XJ)!s@cnYWkH{I?biUr=ov_RcNU8=4Uq(c@%{jbbIwC{B59oc*ONE4#Q1`<8CP8Ds<|g^WIj5J zd$wHW7rjR3tB;go2~;Sx{^r*ANsK8a4{AgTD@15B67-DQj#gLPgbbwn8nBD1Upa7Y zyZZDo(s|C1l`W0^L8elHA=?TJo~i||yMY|_dZ+*Z!mK#7?n=-@7;sHq{Dr^X%t0A< zbhmMh7CZ6nBI2aXmPJakL><|JAyt2s8(&5>^L2qK&hDE@dQs#`Da`>ba7|BSm!Vh; z?cm#*VR??&oXVDb%!@99tT(~XbxpnOuwsvuT86G^-xYnTok^M_(fSg(@xT6R|8}o9TDY-WXc4B8)y4a46x6tH878mG-qy6?)AuOry53k)*%3{Ynr4Ob<6nW}s47YFO83RWlcM}IS1!3K>e{@v3YILr+0jQaai_80gWgt>Je}%F$y8nI# zpXvq#39vtZ{v6mLf*UXg>3Z$7X|IY$xW+JKc)vA?q~NNlpm7M=wnecl^7{|I5H=-RGF;M{(PzG;(}tGsa@P);fv7q4?Hit z$)_Oc2GmY{6BL+%7-$w89@BR2)ARGB zYRQ@H9`lWmZ10$&6}E|%>$aP6{3PFJzb{+8F<_>Z#H5iK$sEgS%yIwWhXNOLi=y4w zB!M`qw9H2Qu&!1ymJxbNfI@PsVJm zkDe#&W{yZH=Jb#5Aq=WU^aK<7FrlQbg-sV+Me`NxGv$CpOk*YStCIIoQUc<>?l|JO z;c^mMgR@~cp~)vg+!XNtl6Wj+D`@7|xKHc+K32TCjYhuW;_@{$JX~^h3)?tR`+5uu z=Y3gtkacpgQ7Qtd`#uFg)wt;(H*l~m_95_ky^8Cmye5k|efF)}Xx=^8jjnq04^QjB z0mF<|3r3|(43R2rdp!%P8}MZ2h$se6O%x~!lR~C@ic>l?5OVE!`>r=Fx#*JRM~t!` zN`G{Ig<$^eD*NuigX^x)Lv+WHz`};EY~Yh>zC!)ojoVR@Wm=E+-3KmyKI|!Y4)I*H zf7fl|T!_P3wFDX)x$}k>B4~f!@J@qvkBo<=Ry67T>~ngJR#UF|GOez^p}dir@Zbd1LCmj!WYFazWN=Y0qwrNm%AXc3*P2;v@Pny(xI)=ncG9HIf7OJob43D~qDIe{HOM{W|Jxk_xt(uoFV2?+_wF6g z|5{sbz{|tk1O=dHJw;02sAriSN7Cfum}#L{?%F0a(sP}u_z0d!jFi5^^wNP(fP2B{ zG%(R%+!k7$>91Wp8XEK{q34aZqC~t|!3dKm27;2o!Vq>kW=OIB_ATWm77+;H$|r0z zi`8v-Qixc#*)yTLK@>ZXGX>iNDWMF5;pab(AK-}MVBGKk=MY|4gZ@99J~;^o_Tuw} zqDF^Ix8nXR->Md3Ilf$tEmz!|xdjdcuios&e2}KkDTHCRg0?C=oD=o0$ zWQ-e;JC#cp-}4+=cPS_%l?AievSfLeUOx4lhtb_j4*cW;X>$2OK8>7Oy>!jU(H8MF5Wsmo31@HBg^x~^#{C%&V1};vmiO;R{e1s%^mVcAZDjG;X2reW#`j6=jJ_wBe%ztHshaZrs`dPl+j}W< z==QK9poe*))t3eZZA9 zhvuhuecjINpCURIMrI@bJN;4yp+zSgMYBZVs`y{7ncJh?>^z!W<7>kUI}n^pOhIAp z;9&j@@PMG8_Kq}P=w=Yy29N<}cQagRRcne}pprx3Vx%!oovXUg|zfn<8` zVB`1|r|0!fyz|QnCZ2@IizxfM*HA1+=k8e~y!!pbjXhyzN6a@p@jI}QMVWfmgOX2bN z*5Qn89=G6!DDr!{4ycIi8fGaZW(#C3W>sfi#^fjj;#m_m)Y5nW>~giVw8+WH{c-y? z0sFae&^Mn<&0e;{%Z}jfWq@$*>4n@C`*E*@&~r9o)1CP?o9YR3VA>8254ZT;#L~9C z!e6JY*T0JX-3f7`XCbIBxkv=S5*6Y%5#a|6hAl8HA#G8FL{71-udgs*>*(J$DzLyj zTyIK#ecwwyBcsW1kR?~hu)VzOJkMVB;@%`4r~%afhL2 zQ)loP{0bTqH(2Os!WhYGsOtVAuzagK~i7P1N44b$WPS%e2b1MlLlX<*VUa za%GA~f6SG=DCE|T-x9@mF>&p29@D9W@k1{PciD3=C4?t;Y7@u?lwG?_l@&BQTvUw- zp7n28m_6z@xx45W9>R=9pA)hpMdmL8>wUn$Wr@B_rHbRfyf&jCCqV<~kxZ__CneLa zbDw=aa?u~ATp?47&oGAQFR`fJF>qR?orxu1=06LRD|wcyZZh{wvCy_$H|t>+uMH7X z6(4p@Y>j$k`OLY{f!8(t&_ztp5BHTs(P^<&U5Rl?^ZtQslj?q7H$%v!qW`lF+cBlW zvHT|_@T}3NcPnVE4-}OSbrE}}MZVRVzj9#%etGY@b*Z2*u6%_dgy(w(ehEHKa~9^)YzTBT28x(fB|d7q)Gl z&6f}JB1vWxoS<{Uc|7DDwjJzUjaB3j;$8B(lkC@ecu~40%{`7o`XTOr-ULwo(K@S-V ze{t`y`;+l~PI@A{Y@V`lS2E|4`l{J9jWKAJu-;M#LYe8tcjx(f7>>zcU}S&r(tY*# zTw2}k>}ABalH&3SA?$rm!3q|{UZKI_J^ z0lPnC%WIh*NT?-W=P}ibG_yjkNY-gXRFfOhx7SF0nsG!Oz#8AVk9*7kgslx*neB?Y zCB6;uHPmB;dgN&12i_z_OTtXP8T0V-_3GkFuHf6u9Uisx!E|;a6Uf^DW zC;z#6i#{j1Tq^agz8Vhf^cSC-0C}EhV5`M&um(;r++S-E)fLy^*QwzlyI@?GNqHi& z-*LOSJf5Rbz>P`Aol*}V#y#1;m}WNW_;YXUyORyZDrhpo^AcyrfTfo=V;RHxVcVrH z&uXmj{tmM9rv~>bKcTy@o)2yrEwB)>JrXJ{4f~HnFcBXj1Y=+8X2o?sL}h%WZIlOIPJCZFo|6NW-iCjk|sMuAjCuc7_DBwgIXw(wW+n?F0nZi_1ur!Kz9 zWb|_n>cIVxt+kH*i)Al!)*r_YcnBegLM=1s#8yCH1jx~Lb+ox$Ipq~QJ20B zJ3#lNUhZxp8L5i3?LS9f!uBdlvuK$CXkm;mgn*%r&A)wwIV@VdZxr$n?3H&VcgQA& zIesNgo7w(u1ZQL8YlU_JIjmWeqQ=Y&K)I@wOot2yQvcjj8vD}oQc_t=HqqC@!!Pi< zAn`1rG6ih?zoyX+KV3R--8tOmU1wd9%u`8YMDuJ0vT4N$H7o;Rm57D@8dha44R1z_ z+pkI}<4yS_2gkC|1bQ*GAI3970}I=zDc%C<$b3+EMC0P{_l$%8T>o1u%%e_3Z zi*0^UsPv9Ks|M0Wdi- zAw<~*M?P{0UjtYLDS?QOTTPAo0l$@HQQ!?PeNogaq%eF|FH~hBL>4ocfqq0N+jdCC z1VWdB1O9maUSsF=y8=I&1F=9xx+QB_I=0T0vX$1Q&O)RyC0VQMV=!@;sXw^Nn*V?v zJ9xo6t#R7UcRgdTR-`-4}BB&^>F zFw4PS)oCfoq5`^_3wws+TYtf$-U7q~*Z=b@M(`E{WeJw1jix8Fxwj1@muQay`9$XU zLU4dABI3+ZsTyOU zW1I+z#jeJS*JpojE^M))BHM@AV`zQr!d*po`(bj-P<-JrOoN4zeq5IH4fR}DVhU#dhC^c zf5(?2sAC~pvuKdN$IbwwNr$;ZldCjgsEZdsP8eSR+X=DsquZO}EW}@Bp=u1466kxT z_zql=+&dK;l-gg`P~rR{l^69H!Je_?GOkK|7+**n5v2ZJtVF5Vh6B)5ZR<}n?;YGG zL-Hy*L*?vwpVewVk1_k6#+k7Pj8pCUrt^l9Q31x#KWWj90oX+Q?lmE;`Em_}&$bT- ziGk0m%%t&>D;Y*JxZ0*VnE%GsVefW3LLAB!EUG~V`JKE(ed$%Mc8+#&7NfDkpLQ;2 zp8yhXeyb0QIGE(#&$ejN_IvJ~4X(Su_83MaXjEd$ReN3mVhU9w%#wN-voo>Jl%>ma z&l5ni)))7e4&17i+;~=XQ4^*m%3tKZW_o96$s!7^laame=al{>$8p=gm~?giR)a0` zC*D8=j=x(-$4;w7ozyi9h#3$7N=vcCmEX0hICqP^A^B^}wiePl%#u7d<P! zo*wc0fjla(BS##$E5L!mr-x z`Sv@!JcH#QOTVg4`Hxe0eB2j%FNToxUBo{kDy_H%8O^T4jtI!4ow15SAz-5k+?ypJ zYHAO+%wJt@T_HA~ub_L-^u0JZ0eyZCP-g4 zB0r+C2?+EeBu?wndbA(#9~D88A9swhMlJ2X1c)mIj>D5zuP4&0(ws^Qds@2Fx@&;e z!_=!Ss?QbOZqB~2>HxG|h)9uZc-FZ^7cVvZgzg86$Z>$0g|Dd*)7MyFX^lCzFaH& zj0;MXSH7HLry@&I)8d!dv~Is6H@7b#nGxodYO!z;*&#s3g9zL3mrN!?fYEMkQ0g!$ z+|=(4Hxo!UYfk)%`Eo-65H{#4#g`#0rX9)%QM1{&{SpV4QWrCXeZakH>Z zj3p_TVIxmB|K|r<=y|Fn{6))u<8{j}G68Fuqb)@~->gfRB7}z7FTIT93q1c*Vv8=- z&$3iM*^i!H<942gBuc{lDzcl;2={OLDb%P`ec9R)i6_|2X!Ys;?a+<(J;|drgVM|Rgkgt$!>_#S2A@l@ ziJVHwF93u0J@uwsUvkj_0)-tM_ok+Dq~Rezw}b@ZpPsC!$h!X`C618Tmw29Xy>;7z zAfqFq$H<;P$|+Z-LDdcmW$3?QM3WRT3u;*wzIt`-EeZ_{jV?St2-#!uI%CA$2;ky^ zkXW8_kiy0jp#cLc@M_At&zuwNexD)|-e}rYVW*T|gU6IBTlTEM?pG16c=(vF$!)2A zl}g&b86Y#2aM$pkHBdIGVNG7uv-}g5r}JvJgx`@ zzUjR?za3p8{2PRvSsPce>4OXsUMTzbSs!HUV&>tI2&pyxjaL_2us43B3(jwd{uzOQku5R81h@A5 zYJJnh#KZ5*%XeY7L|_vj0U8!TP< z$Tf@O^Y8oZIh(DvA1`)~(fN7~F5NN|`%N1*8#zNl1uGymn64_Zc?P&q?i6SK+*Z zvE&)>ePX4<3r^KoAAr9alDC3|PD~N?bC}Q)YKNr^big3701SdCHbj%!f7-5Twalod z@7$446el82XBn{8@~I+NsH&^GHfTbEULnses=-uwOcI%oB17Pd`}TQQ(9yFn16g_n z$kPAA(|N~Zz5oAT3(3mNEL(`ko{?ErHrd%Ln{3gs3E7(@aS^hzv&qU%R-`*$$NiCP$|Q!o^1a*7K7!a5gd6nQO%_}#ESlS%f{3&u$P2Ee z(0LZ9uRn6Th^qeatsD5D%u-ppkrE@xmLJ;HO!yqAg_K*gff9n6FkJ}C#j1W|6(lC) zDU#Hig_&zQA*KNOVU+^HE!UW?zp%u3NQNUB3(JO&LVvAq@N*{<&6=vOq9jK!+U2D| z|CL!u7|`^%NLeD@1WNpXI#>+DWu{-;Zo;(#vqvxLLrT~WYgoT~@SZtdZQDSsT-14K z;k|ESI+GOh*t!N>Md^y9X0g%Gj`T0K+Hi5g_KSGywvs96Tn3g6ZMd#9K7LhYTL|NF ziTvOYx}B}F^n zl#l72UwFS-FuIBusgNY)g7vvTLd8rqmS|ae(Ly7r-s6{dh&}JVY+CSh(~oyQa&t+> z>0dR0wZB7>11oK$epr z0V+7c)&dX`z~Ot`jR0T}A@Fi;LQzQxjAR|5LU4joeg{$Rwu@av!$kHO;Uy2ZsJ~i) zx?SzIG%}t*#BwkNs64FO6eL4?2xbE0xeW@APP>}*Hl$Gi3tI^k7A54h`W(nQ>k_20 zNPYEhLkxdtHEMYADy`YD1|I(R13#Oa_EbH+bZvSylU>mthSPjb)ALAtqbH0UbA4nEYXu&$fc;?ilK``wCQT2~XXp z)ngtL3#d8=oqwM6;qNb%#Ngq-2S%ugT%GCB6`bt*!?aVNGPs!zn_G})HZXx^>Lr-3 zEGx4*xSH}~&|!r3v|O=XOQY=6{N@RW%VGy$LbxhL(m{k99dbuR?+TIBJ6ZU(B7ikvi?)nYQl`TC8F)?KSx{X*it(p& z&{?~i3SBDn7oNgKkQ#?Azj%BLtr)6aj!-8#gWsxnp7 zAFT*YD#F-?pR$4%Bfv6QAw-ff2p{WQy^I2ivwTPn1vrv|2Jg_fh8jZ zU491~Mo{zY3>>0c+wQ3^BL(1-3S-IK0So(~=Q1>biLp{?vh?=+l$)$Mi=b#~vfzeH zV6nn1D~nntq%e5=g7@4UulydLQNd)-@565LSXR^iCR~ifaT6K3sPn9*g1*m(7%4t& zEmxRy$h@B|z&b;=jHO6W$a&4>Yur3Xr_F^notv&6T|z2|bk+vT_NUCa9Nt!5&~To7 zWI0N)6^v}?=vDQ#+UioZ3eqU>i24R=VsO3i10M<;T~OO-VH}NA*0%!DsTU$Ad%&3h z!vt#+N-(K&Bbx){Yx;ERunNTL%S%gI=aF!EtWwU+%|X#53ZUoR4!Wo_8HUD@YBWDk zdqjgmKWe@^!Kv{I29lUOyX~&gkki(V4qXYtfa&PKWKRe~h+jw`3&LkXRwa-?ZLEN| z{j&R&&n^Y7@YZ9r^w$ey0B?n_`Y&RNwRX)H7hOO$IcaWlk^tl<2~muGD>uF&(z()u`M0bTy^aVnPD z9kQs|*9ndKZ0ex099qGl-eN5!MDaibRRd~pq~6XKC!q0t)`8+n4wsn&Gn7}_BXQzmW918SB*|wMml0!UVT#ko(^BpYSUi0|3L9XhT#Z*ZRAH26 zA{+wKR@Nn7C7~(M!*yG>8&#G|?nPg#HlgSHz~Sjn%w^+3c0TNV(~Nvl2}Gz6%W zz0l%N`73S)$5t*VQvR(%d?HWL*IMQKgJhpkGqbp6^8OQzA;15o*u|>7 zZc-UOBnMX#lsr)0P*;Y9B(4O4^L1zjtsu#QHV5kfOrdgEF6k;q!c?G@_b|V~94-^f zECDVqo1RPOckc%6nkQx@@|h9b?Nsd$#Ccm9@yW= z{MG+&AP08jDTMZ>#TKu@yASe;R`8`*>lRhxia`mCz?E4$O=2!CF5>wGKpW*gJT9&714WBto(9Dl-k-sDvTkVD#+CH|A(qbXqU})Iy&3(aZf(L@XBzUv7 z4ueu8|1-OKJyLL@?F9hOcG!pY8s{AmVK?~TUW0HP?k6|V!v7s|a62Je5kzH=sLju2 zFvN~J+U=pKza7<%JduxnCe*j^H5@A-XWIAg9I}9M-+~vC+|(ovE{&mn zQ+8IYm#T_aGSOM9^l$@d;qg(G*9PcISTUsAqk#g4HjQp+H#4J4zlbkct-z(w>`{bS ztV;D8Ii@n0b`Y5^&^!i3s8Qul3tmogY<`<{yb0qF?kgEq1x2C#B!wh|tue`UVkwd= zVbg>DFgb*!MOxQLx*by|oPdvUmSQ!cUZ|-|MIZ}YS;%tN7O=a2ipULQ>YRia)p>AVU4g{6!x`7 z3wmj|=V-218^8YiL6QVoi*b|hqShNP<%W%#GDmMKd@`l8Di>^ZOxk$2Ld!{IZ`8lK z?ecB$dYXJ}??SZ6TfgGiPWPCi`&ojT%%5=yLw~ZE!cgMH_&S6UDVcrI>485s)%C@T z28h&E_<)H~3%wcKUnJNEKLjqE@k8Kc+wg*T+=4qLTO zZiBup6d!bR@PJR!jh_0F;mjV@z7BVa@xwcCU(0?dgOz7x^RP#;;v;GWd|da5E>Zcu zvg{7$w_dip3BANnZOE2FfZ9u#D7tmXuq9dc+=clDh$1U*5Xol*t#oBJZzfTM)DN75 zX}(7Nb8Q240Q^_%b$coZ+%^zD)%W5UF?B#N4z9I_1^Fs-u1W~#zi~kV4K=aI;yxr% z;Nu2OyZL4*2ZI~-D1b5>mIe@7fpR=$6iV!1yxt08a*o93MD9Di=%LszHq4Au>Uuzz zDb0%6XrgiBv`GBgXoe~SFP9VEEAFN%24y>Awk`+9p>G>R`b}C{ABm%ZrKP#v^hsWT zj2((Ft$Y(U3o7h@9?xp~pTLPtyY#vrB9yq>_8y^gJV|xzx-Sd^N~8ftrW0x`aZzhy zQNlZzU@IYn&@2w*XKPp4)0n5ZjL5ip!8S6A6inKPAKJR8`ahuY1Z`~C!JbdEVUB23 z9_?|tjwy`(+LoQU9zJXoyFBjW5uuO33%(222cm2>0#vUlwDrG%?t#2v+$lX^MY&0r z(u4f`CwL+s#xIL=V8Cqhdf}sHjL8&PlHvK=YY(2+4yai-rt~m=F85&d)Qcx}d`nwO zKlq5@0*Yj=YFgYBVw(3F2d_c>!|{pMMm6RRc@+4k62JnCd84+9a-Hw2RfQ~Pz;Dtl zsq3+8Y%fW_(#?Ql;~NQ+BLsJ4+HyeS3?D?;Goc4uDeIJlPr=RQP<(8qmMzz+WqX04 z@Q@D;{^i(ej%c0=u%|#}$^_jC{!8@5go zEU|e^f0+n0%lgG#v_J7B+iI-BKv0%ep>a@-KAm=E*|2r#x(JPL0f|?Mu6Ph`&}17Z z37|@`;ri_K$NB5R{q!IGgz25hO6I!|jF227S$6;VE7|Mb)a4bmeS=*-q@eqJ_Qvl^ zF(HlQsxZ{m8et*zMXuuR!ZFHv=-3`S_H=)lG?^9BRIj~S{PCOVr@(JTA&s!4&ijXN zp=zntd8u!}_c0%>-d!+EKd;%Cvoek>Q2{wLYy;C&^l_bY4mF&2aG+I&Dt^U9?!z+H zvE@^)oLJ7`*H_hqd~1m1R2B+xl)Jg$|9{{TD6OWwh3>IX^$ZPA5o3iUf!XM);AMja z2uLc6Q9~`xkDq|eWj6m%y;_A~W?l038!eV6zHZ7c%>An+T0=i6M59t*7g#sdGXG`H z@z+Gu-Dd8hk zY-=Kt>0_@p+OByWV<*>6;S2?`fQQdKD^#6~;N=j3_LT~uGeFo04xp)icnacJTAURN z1Aqn4i{Oy>+J)u^V5ZS_CF^M3basNdb7c7kD^)LheIX!x|XkJ@-) zc6vCPd@J8Jg6(%pV@`u+02=RQVTefNm#sY!9DNV3g`_F=pd^8r>$e&N8@hH#&2?M$ z(7{5&OI=&tdvKbPq(xI%XZP^CIMa)5m(}Xc1Rmp@M~5^pBQ&!2rd#-BhWNo?_YY1b zh`#Z8%$sul<)ieICoie)5n(y}lw;a1FgOo=f-4XRVxoOI0lN~dzfKvalt`TtC30{F z5g&mIcvr^J26-2lkRNL@d5D+%S0CehH=Ct^cTLq*?KV(n1B^UE`ZOi2y5@&XF z?$axJYJ)5o&wLtOb57YIWDq=PaAXjj7dXKt{@n~$8m6Z;7Pr^49C&qv8x%f7LRa}M z?R{4qWY?tK&@UQUN+Yl})i&aaw=%^(6-h&Z>xc-}VJ6x}CcDj5j?IkDF5#LSeD8A^v?3J%9iU2%dW~=j zhjAHXprD{ywEEA51>rgbNI`*|`D^=`D$fi>4GQcoWe6o!Ur!$)%COt zsX^9N9-QWtl~Le?gjkfW*WSRF`;atBVRqbcefQ*>)<0_~-`v1;J#mhRkCG*7;AV9xe%t+vk<@fnB@he@s->B37cn86|5(3R7N4E&VcP@S#pVvyzSf#DjDx> zd_h4${`IeeLr#=o@Bvsb34mk#KRZfXPV8RbRS+l+cV?<(nQNSSWd*|4%d(R5_Om~Z z{CCoWjd@cNM@NSpS!7LjxGCVidL2i$kU3Ht1zWSF%`CMdSdfDIw6@y#{lx}Z&f?Kux`USaO&T~lR^UM~U@FbtY74#1 ztIy>d84Td)6-6(d@JRYXP% z5*q++)OY!vL;hO~2yaf|bOTfgmR`&xVR!#N!?As8*Pt``%8}~<3@cEdgBZ_I$kQ>+_L)#iU5)-{S4 z27-fa&wOo5?Fp~VznRI1DrdUB?iBu3p@Pq0f_Rvh$uFYfQR$>Ub-Jf>xYDa~p2^8c zxPHJnenj&l6ejfFT96G{y8EA-$SXl!mR0$$Adgy*`ani90)5p2)i;o3SfC`Qzzuqd zpk^!z%Ths2h&-lgQhHQa;>N(Cdzo`S8p8Ayxe<^UJ20Lj#*?bajp5A?B_rE|85c+) zfF97)2wVOp5c`b8-#nLFL_)dX-n%gaZA}6R+85kMX(1xqB=9-#q}qWKA(*O&)#~_6 zO^v8&g0(%LFB#NsRySk7Z zZFG)syhRvIPhEMdp!JP-F>J5?WDrc{Ng0=Qeu}3{bYxBpXVB&zwo_XD3)!^d)gfJwF;~CH>M`*T`OHC$Cf0%WcY<|h+tHPX%3!cu+b*zv9A21 zEg&fOO4CnPIHIyS0{(^r?}2U-jx$POV=Z(WI89V0!pSD*H?D!(WA6|!nofad)Xst; zurb(p7iERa7U49)kj;`eSK~NPr5a0O#`%*!Daa&QG{20q5j`_aj`{J%-wxXEa*w+C zi-#hYrEIW>+Xf&ya1)Tn6GvSIrJ>@2@4wx@Ar~1LdDzcss%(&4Ckpm{*N#8lmk#jA zk;|KqY)keRIWy?SB)-EHyg;;~yZ&p3st8HCHHz&(ED6%!a~&^>EFY6Qd$ z=f^X8{#b8~AvsDmOHHHus@5>urA}Rz&SiIYkDf$~>Cf@2q4=O#gq{AL8%AE2#*t3% zZBR>ex_>5S<4jTGRjSruT05+GO3@Meo%L0bNx<9l&HrD@vm@|wwigxQ_WVD<@hB-w zZ9vDP&m*d>>(EM!5OjxDT_my&?sdWG2_hPKB(v^(##ezt-^9N)lffG z#5C-TX{~yq-rvMmzm0QpdRh%UA;bb88lI2L=VX?amuC?)J$x7zfJcOUT_{JnLC71V zKOu1FM#Pf|&3WI?+-G7#FRd&rylRgD90cY2JapVI0Z+2{<`(-=RKS~&5H!ECUcoSA z`<|6W0}u5z9udk2yb$11Bm!qX3fTxC__Yq09+QCE@rP(n`KoGl38Oo*8t_FRSt3sr&lmNgN0K_ zZs}g&xyi1SpinANCg2l#F`mOtWr2h7iBLK@AT5=S0ap=mB-J6@>?|6R4ZY99*Z3S6g3q zqH&_SYX>dYkyu&kDE-T?J}?t(d}-*3xOy*^)v!DfoRJ`Dc89Qoi}5DRv^#qz8ky}& zCaJHzC*k`CYkOD_Y_3{b?F#~f4jMPu@)>DfBO+HV!juTEm`Pt=EdcY0vyhzmE$1pq zsAW=dUpzZ2=1Yc22FS$i>NxT`NxR>7e?Eog0SvTkcgf9xNB4ja><9m=>5(uq=w<05 z`QLK4+Ea zXzXvjp31|fJg}v}`>nQEZq6WhCU_t+^8HD}^Ri=GMXDMqqgjv+-zUAEo|!)AyYY9q zhNAZDuROkA6WYYaOGGqyyxcj@`{MF}+an{lmEfrRY-3}MX_}Shf7!>scg#+g>;##r z(>}5K-Vd&yj@4JS@cqHEkN zHEMb3-s$jO6)E&%s0aJ(MvRVw!9w5y(nieZ@!_~(qhCv7H-9k!SgeiAj0AnJXa(M> zM5lT;5+S>W2f1%fzCe(G5oZd>Pu&G|??gPpTL<^FQ3I*A#*pBe9TTj$gm1t6!tvV? zFVSw*^%s07!QsOaCm5>yjsclS3>v-UiU;x-O*dA^S91DH%q9R_xOD^p>+cS%;Q4>&NGL_CP~@gDMx);~uU)(r1BJ z^+7~{n}P##2Ksaf%q>qron0DMEKuuvG+$YXFzbFZWmNe*t#l@VJ_NFl(_c0?V#gAK z^UB+)<=7IoX|MsrMpx(F#5$HW-F5yp?1)TX9dkp0tpnptV_q=456=WUMyP#!tfXle z**Iyz+ez~DdTmHYYjHDW|LPNRw% z1ut68=xPV?voRD2IJ2OTfVHF+mx8a7tanB(7ueV~lsz*g;Dx3=m^u6;PBn$=w(226 zXbG$i?@1anrRgyf%9@_gNh^ShLBb<+`qHZ3dDe+&y1}SPwsx};WvWai2J0WQvYVS~ zk6ZRm$d_*R-1PXT(E-Q64p;?ZK|+h2($j`K8Ifowkf>B6d+12`t5pM9+M#BWIkdD3 z4Xt?el-jd_57tqjpFas8|A@@+3?Blp@5Jl%W(t|NQ(Kagq4zZd;pVooLq7L)ft$XS z=}NZNfAedGZM@BcBGd4Acb9B&KK!?Wy@e{EI$OVBLC5I#j~P>H}PaXW=r9rB@(x=)D+O)ndNoo2M_{n6ad2z7@%{ z^2N0>E7Q-*Q7M=^bpG1%TfSa4sN28C3c7vznN>+lKs=GbH|zFQFzw7{CctwsU$@n+ zJhKBUUp_MZixGt>(pU>s?nUM~f}KApm~qT_R{>m6vN1X@wthP`^ey6@NW3@|n{>fA zT`RfOP_?0Rb;`Bi7?ilPnI6=>zT3GPJ;h)QjelV$Czz5m{{Ec_*~I%`A%%)ev!s-8 z>C)q?Q(y>!`O>UkWt=UJ)l5LEjYJGFd}uw=(jCi@kE?F}z_Kd{Q2rbuAeLbIpYQ;M zHQAU+s(<=J!8BaBKQaTBckm?sxA*UW&$$#PNc6!UGLSF&@qRoDNMm$yWmxHaUD5oq znSbx3Oxy4F@Hv5weG34+aWGAN@6_PN$mF}5Qm88s-fd}a!xaf~#r3Z*E46h_BV`C+ z*`8PQ99{c4BD7&0=4}tP?0UOK>EM%W!4&`-y7{jKi=6WkS=Zob!hHUjDGXC#pEk^p z-7NLbXldC+P^afFP`;89BQKnkE3~wj7JU;`72cCg0@m^f*N{8rLq9P{v+uxS6U4~z ztKYeidjy0PX1Qubw91$#!inMNEaig3I@c<@n@rfG_?H-9fOzPdJ(w>p5DN4_LaUUp zkMuAoy@}x^NsynN3%Uxl0KD%r4NsGaKcSDw zV)-I{dR5u^MU&02!&qhE(RU9YCxmw+rXh z%g~#N;3l#;N5D2ua|sagEV!cZ*hh6Sp2ECO2^V>$C+NA7A!Z?P`|(X5_qvUN9QUH2 zeq;sBPbD8RQ4j>nEU1M~ifQk0k3fM)hl~{xbRBQrB4sxL#+|i??A=>t7-)!U=<@mG z}~AL*le5aAz5dmO?Z@&I-8)S+{*vLhBF4Bxh!v{W(0~*+bbiHz;9OBvxiOmF@BGmN zk>c!26J`a+YXPzYo_)~E!YUu+E)cz>^CHTDoc)j~1C zvr{|+qoDP-+3M&!Q@hi16n%_by~o+8jv^V*$eemF*x}t)E;u(4ma<XH1%WAf`9y(nNAa+FCEr2SYgHwT3P;w#D$r#3@1drtkYyvMFe;BaPz?A zb=D826XN9>?TgEa9!f3Cn^@2wyze50|4Xk8c16Q7ElhWzi7pEHy9PLnTaQRW=;XL| zPiqznIkh!{TU7YhG~U$XvXMMm){XveDD~w42ZGTiH^_r^=q ztpFIhm`-8`fi<`wfb!%~mkjIlgwrk&nRs{su?g9sy|Iw^PvuVIUR;qod3A^VfCy zfCs1vFBL$S-_K()(<#RD z&eYy{X>1PjmYbgHlk}fcQD;RFM^)c*Du)*-t-B}X*Svz?0#ux4+OB40nES) zEoTH~3NP*xVIgZmfFK*p0Q}Mj#QBTRSzP!|PU}#ieX$wmg3{yel+URxqVYpU1X002 zs_}zmXClxHwm@W1ygcj0hBPSS7C8~X@eGfM*ajk?6{6OoqfJ+PFuAzjH?#r}^bch> zelQX?<;&~AIQb4#Zeim?pNaMJM>-as9p$yCHMB>x+88_2g5h-li#m}pS$1yKK4)+h zDpe43d~kwaO-Iv^pe9ZH`GFbX&il+?#LAB@=}PDUZ~}(;USgf{Nn$A@XD-FYiI5(v zES&-XOzO~9XMfUO2Z*7ZMh7V?U_oB|+2@8Cv=xwfsWiesJw$wHju9lcF9@Bpac)zt zSbcb2hIcxZ>sZM?+D1{$hA{{wUOL-cP0%YZgGB^vvVM?I6xW_{A@kE0w)>Xoi#>F= z!=iggCraxJ$_0~j{WJTTQ@VrRSur5GXrJAQ>Fw{kzRjO9UNZyB7zWr=xO`yhE0!S+ z4e-I+R*P?%dKjlem7^ckZ3(2t8HXC!Uw9uU~;Dq@(G zRmX8|?pARsjfV_VD74W!q45Or{@_Y~W96u`8eXq74tx`!F{m zRQRy%@tY|0^R2)hD>D#A8ur8V@rU-}faHH0?+hMz;wRqZ4`z6Tc?CD6eE##56%RJe zAx_sD-k?&vI)!U|pC=f7h@ua?&u~*b82xX2k@L2kF=rA|r_KRH4wTmM6lkATkJqy} zwaOg={wL@oe-dx{Z%1%YUOX^5+8*VzM`xmaQM|%zWpT!VP@A+&8!teU&H;+|u*<3j z0JvL>cavKOzTRDcbchh?oHe)98-bD22KqTF%A0NxX8b3V6dWQ`e~SvNozb1y#yUFA z8$PSD-m+1o!bmloA0<_@P_(=VN;PgZ>~}cqvv!Ictx@g^5p&%-Y~X&qcZieh;y;`< z*Z)d8NjD#VX~1cehsSC@>C`qsrk!)T(x`Z|5oB}nx!Lh%PJ(H_JyucwWc7esG=ZWK zqTUgGR%j(uM8grD}$?3tA&Wf`e%zCwT+{5BS$S+y=$2S(%)S zjLd5HDN~xzMpXb(vjX+=MnbogrsilebxfPg;}W$jt)ef6nR&UD3Sk?tsh%pj?|^qF zDfl+lC$h1@q!lYyAODMud)sR0NUU$?G5^#0g_xH#i}?G_*b*Hi9+$6dsyVQVDwx;RWttneo zofU~M^A>0*m96t|BibSRB71Zhq>IYzq-0c7Ph=wm@LXb4ac_ZV(1fwC_JUpCV_Qm5 zaG4${%aN~oaP~W%7u$W%xUZT&0n z_^IjPHqkW^qn*|ma{PPwE{}wC6~T5Ok50#)=N)+p!MRO?vmdmI`j=e|lfO+>8{dS_ z#(+iRqC@2`zGMYPW4S{s>1D%5tFJ8o{^wLM#Ijyq zp+LgFLP?QZ``u+4rQCbVS@39$#*D<)O!8Hr*qK4ZO5r`UFg=;61B}N#pHs{rlx7~9 z_&E^ubJ|s=023f7CsHsJIi@8gC3vq7kU;|@B`yYX@c+edgYXxT3STgh`XK?_x7e%v zrq4IrklVxrQXLRU(A}9}U{F;B-@Vuh0>D6|76Og*i#FBfPgCR+ z6=nC?3b-3gs3iP+hyrK(M5x*67yDgmQgF9d*lw#y53RTWNbPII{m`FQ2|xZyW~l+1 zCAycSrE8#5;2dPO&@l#PV{RYxgELq=JiHyEQV2qvH9(ri$-7zw)$()H@bJZ;_H*X= zVQ(GlrLytuhXz?`0eJ(A!Vy-QQ4Eas_wIo4f65aK8G2Heb{SnxgC-xSvQcFR5RcGw$sM0T(8PLI=D^MxV0k?gg_`) z;NU6;;zL3EtCu0}X~?Ve6V%AX=KiaTvtv~!A_(2v1|tlOT4Ns|m0;i6y-;RND+x@R|Jvz5cNc1f zVwYxm34deBiQ0rNb>tQ0s@w)YM@#DKF)+eP)#;ONt7(91STbLiI70f#(*B7CgIJCra_qG34E=7CQjH0um;3Y0K$pXY_r*j>oqn-9Jh$HB zOF)?u-AO8yrAAU*&qM#S`ppr9a2~_)b6f#! z{Er{CBq3pKL$$4{Wbk$Y4{4PwgrU`h?Y_!*^{Q}MXP87jTj_Cj$6qYeI!KWvW0OWa zlB?@OwFCt3mmbFeO65=5tOImE#_qa$=7X@vr6pGiQ-|+=sC@lc#S~X{w!7!N){7c5 zRRWIs`r7^U&9yZm7?)|AX8fP#fTzby6$xNE13L?;1ZK&!&uIvU&vux#yBc|h+MA|K zm_+masu;H70sXzkah(d>B|ng&;z}@R@=@jMAnyN@+rZm`O3)1CcA|J-cV;AYwp;SC z>U5EZ=9$Fr^WD$j2u?z;)YrdVHhp!QQ(M9&V$3AYkVUu zN=ocuj^cGng~L{;(<+E*?p-$b^3sJ_Ab_AbLR+(|X5QQi9Je8X$$x_^J%R1v8N7@P zUE`8O2pWZ4zhDL4aY@9Qcd0J6@1b`+Kzti8MH<481Nz;#qf#{MY4h#6- z|LBm=>cDV;{O*<9rx6Ud(0^4#$hUxBZ_lO2#E>-9w0}7VCR$L>uhJ%rL@{5{Ly;}n)k9*|so7uFMOl@s zV_k)SCjq#Aa7zrZAZ^pSfz$MapVDkRPV=m$8_}2`5!nMK$@Y!ylH8r&{i%&168>G3 zg>VmE?>RLD8O|Xf@z87-X7-g#6xDpOzd&oKxO zPsm#492~1DA#}mXI-?ht&oyO*;P%#QxTit(cLxU@mWP$UEGhDnkGVIpNo9t86L#PtyF zxMg2MbhL{67X@ks!xcV{lCUZ4s%Q|0_`c4yg7Fq$wgKwsh21xY^71%E;cZFKqyn=J zude>QK#e|~acTK~eA$mh2`e8#wY|UMYc%4Ts@3|j@(p@DSbw{B;TZCQ zivnELSz30eN|ov^Nw&^6P3cM2^qsXAy_mbF-vF0@T8FDsF8*>|__y{^Js8eoIy=)X4SG>UZPpe#<^jDu-}8}Ba*I+#2NY*fwZ zOvL+)bcAY};PN+hJxEad6hCBhmQ$Y>e{IYJgRW9v!?dAT=jJ{|)y`k8h~$}>Sli#O z$KiB(VEWDPrab!_NKh;)RQaPdz9xx($%F!Q2BybhkBT>sO>Y?p^tILTeR8>B*KqDC zk%bSrRB?;_6BEw_)7n<|2{5mN*Z|~0rJal^{lmkpGc)&mkDBj_zB@*OfIM%$l|X{j zK{}a+%%vN8f9QeEcpPA3h`2}q^wN;ROa|_~3k+c$m^qgF!!8ats=YbMJiQb2Od!KS z@}6MF;$_(Wu&BPXos{M}C|xEe>rUFpSBbz!j3ahB-p3EW)5xCk2!E11dckmLli@w` zh*Fqd{98U;cNh88lnAXhMVhqI_$hjZtF*cf&E3;`nC-R?O zWsL46*mABoDND0=Z^Cu2qH8=mh7RqPrI*rCj;H1NXWJw;j;l51+NjW?^0Ce?mh$fMwNh) zW?6YVg2CPV_{CnJ`5W8b^NYj9p(*UQ61r!x8Ks-lH8b1p4;5}u08|wUZxBBhw@Xzh z1KEN$sdwkxsrL^b-}KimYyEt+qhH$w^TBi(f8j?EyL9Y>uj5|~9Ok*|eR~h52D>i( zuHnwW_XG0i=~%JS(x~p_)vJM)JaBYcCo+3U>)}|KezATS4(}MUP^Cnx9>15yI^#U% zFjg2Y1>#KJHD2oik;mNWW{x(&@M>bcL>szF&}z>|Ui?0oND%;oCmNhK6c5_Ulb`O0 zv=0)%kSvHfsE4!ki_7e1X`vy}#+D`=)&YLVvl87aR{#nHt@QbeRcWc`{4;*n05)he zuZ0K`oVnV7^F1qy6f_z7 zfq94=XXh>YHPtITFl$K}4j(t6O_m)X#6=S10Hy#pd*EYctR}x#z5Fn)1{V^jkNeb) z!#~n$>?qG@5sVT2=1c&A3t*E6$ezX>%dFXQhJn=+%#&GGE z6Mb&49+5{R_AF?e4zKU(`xbML*dYl}2uKOmrsBYc**ZN(!sw8kBL1{qy!k|A5W?-# z;JY8eJ#rrj4}#qQ2+4rtsyzoe#WuVYFg%gSdJLERirY`EJV0)OqHx53zn{Q}L5`7J zi4Gt8z=iVx4wxFD7%466w|RQqE3KpSyAa(Rn|^=P0RQg8r}4xvsRybB{PSEy97~nf z7_K|kzjX0xH{>7$24oZ9xZClB;4{z>etUy#Ou_X~px%S4LDaNl5)KdJtkGq-o2v98 zgX0m1Z&R@>O|cBWW((XF*%{b?dyXKjwsOgUGdOyIS9lKc>1(y(S8i0u@(RU-2bzhR}>t73CwI zX*k3RWF)w*l_AGvmu(lT!=q!zWk%p}0EGywgS4O93z$*+fPjE=*#z0B^KQc>i1(-P zks`(r_6aQ8Zu$YD?sRnLca0Q(X%7g@Gv81O0~Z#_H8W>f`*`D ze45Kb#ZCeHFdM?l2C%;)G0%`q>JEow9H?DahB;@q^U{rK*4;AsDF2qeu(O&(yCQrt zHpixYsoD`Lq1hj#Up%LDDAK6JUj42Cgvk&`0fPk4;)8Fs)1uLsH(Gx2ok`Mbz&GUU zN@4nU&jNihQ|Pa`GcA%L3g{(Rx_+b|AjEp;iDe>XL>rUYz|`W-EeA+i#k?o+#L{gq z!3lkJwm1GB-n*{9P^JKaQE-xdWgnd}_NpKiZjDD>*;@igVQZSI_T>QeSmw89Fd~O? z59C@Jky40JWB$K^_w*ngxn4^Ju3D!PPI8>Uv*e@Vf9XGF)o|9wz+pHJ3xaA;A^>6? zHl~~aZYIxA{&)>-Ds5Sen_E&zsn2r_;6mFAU^u*jn32p=9nix_9vFkM$m}8y`|RB4 zZRO?tl2s$KjU(*{k*Ly{rT&s`BeSX3+GfXL1irc&b6bP9HswJfNZ!vnDMP4slLDy* z|6d{ssw-3;B5OW@Z@1k{io`0vD`F&#+pi|Ik57F`f8;y+_g5Lq7^NVv8$~s%0Of?} z=Z|1MBM&g>0gIXaUKi1n9^f}Lf+rcVNFm>5fFeLvaT+p)VAHL7hm1&&NhXM-ZY6Xh z^$`3Q9WufJxIt@>U>|uw-ndQgZRzO;!9`D*_0&(n&`vLh${4)#EnY7;J>G2F+pJK! zY#6ZkAuzqXd#rS+AhxR&cki^}>{RLGB=eVI1EcTtjR2Cc8!(H{8JKyLaf}Vu2J#0Z zLM8@srOUx(XH>%>9BMf3q8UuE4}w|MSOUcjNp`tHpxu!e`2AN^VGzj^coi#!r zvh$8;2-^j@8W58wDzFJJtXve$5cbW-k00FT9>M*Hx?Jo@sr*l-bOCvsjGHFt!LQ;b z$N4^6J)TN#%?Bchu-2c9CQ5^VLywe3Ad3o*qZl47?Fo8F_{Z~W6%9Sc1i1t}ePMAS zbsNs}Wk%gDZtMrP!-GkI#ovT{90PtaaUbp|^Kadxd_J=@kb}v>d=LFGzpNq)x*AcO zQw!QBcgd#fv`;m(40k{D3goSPKDqdml)Rj#r~LDa&EzQ0sB=SHx<`$b$2Z@;UC%qA z|Ea&&I`$?a3%!2-J$4~yXffth z&1ZEzy4B}xcXzJ&xuo2O@r}>jj@7?iTYZb$Lz)Z|U1YW#NAmjA3DN3LhKrGZWgxf- zBdh%0dTj$xK6?FqX+HDy#qoy1G)Y-OcY=(DqKf>%Go-yk>?D3)(Hqg`VE}Z$4o3 zc>~AKl^>UbFW5A#{BNjHb0QngsWuIdKbP-GMJqRk4Okco#?5a% z!JAFYnd)8WNs-G{0bg>zMfvR$Z((+(B(K}gVb*OMGO&76Z58KF^u10Sak7J%gZtMJGq9 zah1{F^QTeox`Ax9qEZ`r-gh=={Tt4fZr_5lWji)oEgxZc zxrdQ4*0Kf{?Ux$`xe#yzqCvwM1^qR|Z3z4q9eo=~b37Pd#E~5y{(X5mzgBT-H6r%Q zJfk_TkG(+O+Rz3S*KnqCPw2sIs=>Lvm9a&8!eZ~(koF zg^UQD)aBS_Sl9V39DZ--fF-In>JQHy>=j5Gw_JAmfH1w^|$@svI zZF?ngE?hU#IS!f09xtBTdv>Q=Pc9O0WJD=f=ZyXt*Uzaoby}>V^MNcP1eLW7+chLy zvILyAV&>NQJt=%;Fdx|gWLT1PPSgL@`+Y0J1qSa|Rp)X|XQ#hcx5QF05S+8h$y|2% zTnw8|TucUv%Y^$bELIa_UKaJj0V{pp^{H7{H-Rd)8~3LUJ$QCw`7;+ok>6m zO(enA>rW#op>;KiGFduLf+C^!1JD+bEWc>Ou`7McMOXG_QC_g!o;$FAQ`Nd6qz4No zW@VPNO`!w)V1;+hdsfW!sicF8o(lqa34Nl%jB};ICi}IAXL9+#qCyZrPW_8y{BGLu zNpkadLV^ZX_kVoYZ86@JR5*^D8U^^b$VCqA9E>WLY#+3bEW#1KYZ zhgq0&jh+sQw=UgJ>@QPNrxOgMc6(lVrNJ1N)`};0Mz?F4a!4PYaxm<02 zU*IBfg(Io?qy)U>tdJ(wE)%Lvz=oE>b05K4(Dr@VrIrqDjg3L{*W0_PWO#mCiaV zuRFVM=<(%ZWaqK89_+pptpu>(n^v3b;T7z!cRPr?TWG~WEP+6thqhl{tn;&<|6~Aw zCPam>Br)8zC*}%d^0mRBkW+w%`GpY{eQxq705k zW3E{j)_8gAkwT-MWS%lR(g)wK+zpY(1%!w&0Vo3Qb?+2pz^~nLPYNe2yj}o;iZB0KQDHsaY$~(%fCP zi5}3VxYWpdR-tU>MgNU|JbRi+6ZC`R4g$ni8kqKyo|<4D*YU4CT|hC1$}>EIH0JEt zCag=>%62s_p&9yYq@2wvQA-7tF!K0-=fGALo_8>^6a%-AXMXd~vA6f*o{p^_NL>cm zPSqwI)-H-XBZ~X4?G}*A93WEmc8DbZ{*dajYYp(RKIS#F1N|z3tveh$ijzxFlAn^1 zAOum=fS8Vq6BqBtW`P!)O2KeqdiqCuZGT`1LHHaXb|tbN+W!V{&LeJaSjiY6Cs#@7 zA4LZS1Uu1DUe})!AzlDD{$6n9TqN{%nw23J4)O)96awgZ4Kv5_FGkl8!5)O6=(1z@ zjjqD?9qfM)&?JOjh|1{GCC5hJH&;xYJTRX8{i>f6?UT`SMmmkNTrmHNB}@ zIJ?usuU^4@Io3zq9ck>zzI`@cF`?0);VI0tVl5R!KH&&|k`uK5jxgYNfa&ILMP*$C zA`%e#y2+l2ab8{x`=rED|QUz@H`Y2M-O=m$E zw^mN5$Kry1=`q3g? zdkjfX*(3jDSv*zE_ZOxqzWGlEw5|eg49%|6)AHKu8(RP;vIiLBE-+1y5IooX<^lJR zn?cQm1`X88Ej#v&#TxThiMn~9|YfPzn5gjFf5Cp$)!#7mx90&s4C%lfaKb_ei(BAsqn#8rblzxtPpgg+*Lt3}N8TNs+>YuqiY% zzTgTv&VZU^b3s~`BrelFDi~NH4V2!Mz1;@0*H5L*6m+4LQw19H&=K#0A=^ zJLN2JxUZ#E;w!P19@cjFcijN90@CUJ0Mw^2p#tW7I4ebF`2J z(a#St%5%4iyzr5=8_@%M*S<%Dd=NjDq{;p-_6yN6bah=#7P6Cyx7zl~x@2k=$G}FK z;9`{38s#UR`5n zPikI%6X6q2Re_M>)6Fb`d9AkH-Oqn>Bu*55?Rh_-=^*gh?U%EXv@g?#|A_u>Ut#s4 zkv=qo&7eTrYasjK1Dfd82LVh9&b&8bQ=vZ#`AZi2E=0+740&txkagm$x%oV=aWa=J z6ncv@q0B7cT*3M+CipXFcZ7~DzE`KFbANHi&=*6_kp=ZC7IH;WJrWr&_N2Jvg7SA< zu~zBR2{|!i=l}F{Jnu_j){EB|d;Vi%&r%_JflEU{kMMZa|4LTwyAHD04NJW#!K62T z{M3w$9X>A<4BWb_!`4~guEQOh8){B0qcDX^iRKszG{g*7Q2Ago!dSn1RjEJuM#W0B zJ+G1ZfUX<81tC}DloTqjhRO6># zb44Ak>`^T$7Nk?xt0%Bpe_!=J@i#ezY-o-m!z#t$6<3@;uENyYZYC#Ryp7?HDVf-y z)de~^>0J-$;EB}^b5%dObAEn)6{QjU`R}pcRGN5dQc~1TONe3KouQFskWx~PZhlSe z_5F3lIbBmvFd=Eo0-BIbyoSnFkC)yb=y|ck#Z*ro9RxwEXj$o2L?_fA8rMG#{ZiL9 zAGNmkg;OC~G^V4&AUnE*Ng*`iP7sqqNu#MQ30x9!F$#;`-J~4g4|&5y*4f~tqSSpA zjfB#DgQ@icx=f#Zqu08$9j=RgNzO8TJJXci^}TU)*Yqu1W*&^FXaa(lqP!%&}41O|sJf(Q5Si^G(#u+uB?)KrNOcMSHha9NAs zjPHG;M=+LdLl(E+QKCge)HT#H{ZZNrf@letF22DV#bC7$@Ok4bgpk-qb~i83HkH8f zUE6D!)|i4Mv~4)_3;y*ysa`*m{{6dqbuDf`5TEP0>$@wPNVIp)y$szZnqyPT$-^eY#<7lz#ADasesNN&r$)VT#0V@$hr>6m64z_w=f zI)2>k{KNx~A;a3f&DY%vrNgj%z#I3C7eyiyWKR0NgG^VPl1@0n+>a63L$5cB!=7zf zF5kCw(nn<>j+xJF%OaWz!8M-ae6{UsF1}}#e$JSE9%++C@YVVI zYF95Ta(x^j2Gf3`(uEp(2;F@Z;P6{N-YQA6P*RXOjdAyP<;8-s)o!xM_W`~0A2c!Z zuDVHz)~|7En=PD>?bdBly{ao7!_-tvcqA2U$lc3T@%Ng8xaU(uLJc^g#K$#we-~D+ z^Owav=j=Shxw*GhAaQIW%6)P2y=0-LwdXVDI!KYH*+d2Ay!U@gceX18m|EXTtc$%+ z^1HpQ$gQegzX*6u#D_~mH8{?a;N828ckB-@r&TgSh9_!+Welxt)Mm~%u;xCv^wjUS zS&K~LV>v=x$W)#1Y!jJE=bna1iGqqsBX#=s;+j0`qYDuR`GZh#0>cNW2wMU9Rn1b;LVes8_C3Ta=NOB5G3aM-X`{#! z!(__};EzJ5DBrx~ryX_;UsFXfx>=I}6zdE%4RMmuOqa8a{)p)#+CYoPm0&NVprDuq zC}0ux{=3k}1B`$N>=1o3bZ3Vbk)8WutB~C|Yz3{ej+|oy(M&mLY(*pBOirNN-=aEKD}@q44rNxcAgX&Q9n~!E7sr2aZ%GYRky7yu$?w zIty!WZ_|2+KV&sv&AD`8>HJ8N>-*aMA4Y!0MYql^Wh+=Ij}#hpLQeAoNCbcdvqBGZ zTOY1Rk)w&G$gS$>GyfU!T9|X+ct2DW@uaH;g7YpUDe6`Hy7lX1esc^q3!91wEZlSV zz%6%?&bji-Dc7FS3H1{1sIrxlKWP6Jq=%&Y?oJy!74>t5#w6;aY}&y zO%FL{nChy`eH?`=0GNX%a3Gll?+9sPU>~}W9zZ^jZ}juHKqKFXFPr5DCsQ29+>Y?r zC$~T=Mn0_fW1>*$L{)h$hloO2!MuCBUuNU7^>;7HwvgiVXIoNLfxT-|$icfmzxb2G zl(22;} z;*kVA#6as@mCyYY?DM@Ibvrb^;&ey@E7CPSXikBo3?zi6ps)clBfz@xG4RJ2(mDq3 z<9Bcw{)N}P?^pzf0uc!6Bu@O#tVDiZnQdoH#f4WQmlM;2PQ!DOP<^=Q{8;a0HQPWt z6ux*!4(8v{hd0#aMam}$kss&RDwImm!b0>7p&8$PVn~|$3FEw9IQGdw#N+&Djoe|~ zDs-Y@ao4CL2`tYT{Wj~ena>k89$%oA0l<|weVm}l?8}iEY*Af5_4w8VRUJNqmJg@x zy|NWHjm3SwY$A$b&ev`ba39fVnl@#_rAPqYd|%90c}}IA3C5)Et<#v(6vk^JiSmM9 zr+f;TsR>GfWXjd*e0tJ(A_}h>`RLC^ZE<&C{ubRE6!?3paxlBo6}mTop#~=z3@L7C zSQ_&ucpKj?W+IXO=&_Plh-0X~UyU0m15mD@0Ox_zrB~&B2mA#{wAhdzVmd&N@@I)Z zU5j#HBurF6AyVPi#*4pt;b9c{?eEb@UN4_+xLERlTiu88`CQn}3%b9iGe6+f^>N4+ z1j0d~K#>`oS&c$o%5FmuF&SE$ajj8%N?* zSx%OD$sgbZK#Uc;-sixwm@ZGK7vKJ~(|04V$TV@o>TKB}a(i6NgGx9DbIS@|@OS5C z>63G!N734{6)~pYZOR|_uxNKV4c#6o`IB2L6LCICDmZQWK?Q3cx(v!g990z*F?aMK z%Cg(qj3vfGm@-2(mx$mP@&-!hs+1hdOP&(w#ovm{ZDz#&KE^O#YF-1Y`P;;MIRkt` zBrTt3rhtZ#FHy=GH?4@X7)<6m_QOkUd(~}=ZQA+S*~G!%=o<7WLH&m16)SgxtV$_# zlA;;!jll(+0BO`U6b*+g^d(xBwfyV9r(FRWw5-1!9jXn<(A~Gb}PJw$gSL|c& z-6d*d2tw!?WcY*)?_TvJAYNC)9L7ICo;kDrr+L8-*Aa)!^8t6&OC;(2e|d6<7XS+} zrcWIYSNubi0Lt2W#pC;Iq~_=C2h_!oXi@l0`J{XOU1E%4R=jXDc_h3vl?o!kF*6`3)NLgEU z&8-!etcFl_hrX44T|Oa)syZ)r1ySk-vs4ul`0it*Np%yMxLDDoo+YaRQmceGFNufZqE6rWA>}H<>5ZTO$s|DVY9t68@a! zYmd>nte;sFO=_3A4L3F}?Q8`}Om)u6)wt2Wx372r3PFe=UnEXF#Ldjigy)3d$5WHS zx^4dNM;#5f9Z&W~z}u&=6cX0V@H)DAc!i{iZ}G-OT324I7-C|iX#wDVsa?| zr%uT9Z8q8_^^$D5`ouoL2WU_{GG@h+04jYT@pk1CwxU(t#7e`4A)3`7ckokU_aK#> zY!Czd2H6^SQMI)5O**2NOt!UFdTqK^)0EL(ZyILg%c~Pw{0(WlF{h= zciUxIHU;7%qG?0N#`B0jZ3gV&GfzzZjH^SD6SgEqbG)$o40z2lT&({jXDM$G@(%zX zfVGUNoFJ!h~HVJ%3OKz+^y+p!SjV1RAI6mx$F=0+L4e6sbI=pU6)2o&>xrZw~{ zVj~K~r}6~k_u{eNS?Q(c4Is^H1(;;(>8;XrP^mE@_VVhDaw%F1uVag;pyR(!DG0P&-L3Q*y)) z;j2-)x&;m03n3vlD#JHCvsD==_hYzXf!l#Y1S?s|Rro&Fv3>vL8vF|9l3$d0R;#0+ zpusC6b2iEh*4K;OyVRNR-bI*+l8hgZ5Qioc2>gQ?j3=+J$s*G8}eW`tsZ+2^kX!pJuXMPcR} zt6C7tX~FyU<4`iLFKzUgsKx!Alf}*4W2{X|Q^rO!6KrJ91G;o0;Ri#*;m6jlyE0!-gOw;zE zKgfO^zv8iShD<{w2}UKg3A8?P{063D@>gq><=~!L>HNDqOjr82X#kd*>6xdeOHa#1 zER57$Ly!aeT=?)og-WEj1e=yKrqH}<&F$%GBh2^UrkZ=Iw`99#X-Hd zUjYLow4);`zkfM{Y3i8Yn{9NRIXy>LH+@I?!|-J;c#r7hX(3*!-hQeAnf7e9uIvGx^KkVgWAAMBBmss0&#mbkK9NuQmR^=x$ zMM2YfOzB`xx#~@mZujnj=~Px4&o18K53sX4zv8boj$CBq<1FQ(<*y=oPCs+-WvKWr z{}E9%)2#Do@kHzbp;~OBlI3roSeH+_oS_i~|7fQzb@)U)VI=86vfmh@E^Ycu{}Eox zFV!tqdLTdGY4s@+Hvmp86epD6UPsKF2>L?^dRzu<~5SKUdfo%h|=h8;w z@w?OHG#XM|9o33sVBBr(Y*9|d;uREZhT|KL3L_^i9jp1cEs)wveEHTn`TFUl2PVfv zP+G0xx2ln^S1N0?*(K1s0I84W8{=hX>gO7u6#}t=VU<4#K2e1AIelae*?C4wtU}rM z*F*9RDk6}f8v&IO+zBCBC?Ih@#LqJ40gI0QV5(>w7o4s8{b}qU0W2M#d_O+#6Jc2V zyva6Q7lWDeMVU0)7sFG)n-m7su7Mdhr4`h1yVEbhd5~sN5nPvxW`^drreZIrr~FZ1 z$uO_bhuPJ&PWTOV-v-_X{`2vXEttVoT_qMRX%l&r4&2$-@s7x}%vD(Uo2C4r?gxX$ zR*Qxj_ilU+}O_i+7l{w z$UQ`#M1Nl$dLn%Ge8+1Vl6fOphcWYTg`(@44^_j4I%U0anfHQrW=@cPyI*on7@TR4nLWgdxucayX!Kj4_cl9c2(VOo^qM|iL z#4DTfaoEQIF;Kkkp0G|l)Vz>(M{VMQY10mFMJ| zu^baox+Z(^;{g_!2I{%I>kZ1>Ro;7Bi_s>yq-81QO{Px}U-C3k%5h!}=%2lWiaqF3 zG7jd_VI$u48hrWIxJU(~E*Eb#*FPh94v;kXd11{`nz?R2xP#cUmqJLPPkJ&!l9TAh zYrbS2;@?%6p=5YNe!WMRWou+-r^=?af;Z=GK=T=8q@Ti#0=YkUfo^Hp!hb*aFY>0d z-ew-3%N5) zA2#SRq~uJK!;{C?efki#Ps?p)z<6kMD^v@yLqZ$mIm6(SVd2L;p@oMM(77bnh;(^X z(zCXpxTpFtNzccpMWWh4ZZULXeZ>ljU&SLC!j?j;7$7udR{YWRIv6|Yckjs>dEA68 z8AStoTvvAE>bzmP5-6q2`5^8A?ta&RiN`quD_d`C59V(BY@Y8LtG@FZ_k=`IYh8OS z;E<=2ACu2;3?$4~|04D`2e{AFwVXuRV)FpcgD#aL_45kRL$*)$e(cDc6Tc%v^W(7F z;de_YkD0gg6%~m+k}})#07siV7jrjgT0Z1AFBZO6pWvR;DMj&R3O(;8!q=;9R55N; zhX1_V77hd0BVaMa^|LRl_ko9aVB_ZQ9~?Z2Xr^MPwW}PhfrD#ODV^%Tf98d(GH&rc z@5`B70TW=Xu=T-H*Z`epWzNDUZDplY%BC97{I@o4hI!{}FN#@!49vQikbppcvX{96 zpnm_?8O=X2O#NvIT*NWbQujoSdG=%8pdfO-s;^F@&DltEA~W5aAysDJ)^Xmb<_F zNTBl3cVuA&_!sEvbVEj30)}>eM7=T9@EiTSMJi;G5(0J7dCs9kOMg+fIOEqE%asn9 zF$uBFJ#v--U^~thz=#me`^D-OW9zkg1$Q2Q9$&5fLU?akgH6i`l`o(1VLfLUv>dbU z<6B?ics{ta@B_ZyWY0W9y%8dN4<~j5sMsp4z z0Q!2U3FJ%eMuY~)hTgKa$L(fANy!ejq5A3iNvDm{n*S0WdKBmhXF&NYiu8~nLP5PU zTT%dI+wQ`xh9Km{gR^kEuFtIx;$cDbIf!APgSm1r9i5Q+cY*cq$kP)fToNK<>rW=? z`#0Q^-oO9Z!BMmzake9Iw8--m@1LxvM|4LC3(8SB?dy)Lg&d}o=n2y%HqJRhi6X#2 zT3d9n{!WNx+N;-R3a0^DSDKKR<>aZ|>?KkcAU`-5OfdVkE?iigkPMXni32!FjCS6k zuQK3Xs zzNQv^BZSTUG*}5yG+@+OY4HF}h1jj-UGM0m8hz4Kj@Yfrby$zSS2092_`3NQ^k8*UFwp4=93P zX*mI(lO3&>G24()#Towu8sXu^0~r6T?j^nGc5)8*l6JjQzMLsn$kpkZ!9)nZArAt1 z0=?q>JYafFEE#_;VN+&w}Y+NvSINP z-~~1wC}?P)c3;alasMLL5qCky(2Kb(#gXWa;nUk7EaQ14Wo9Y{nr|jbtBtp}5iXYc zbhv^zSeiWeEQ2k&gOq|QztlhLa}gQr{wTO58bOiJ2D+SXqyZ2)C|qv3l$1C~C&@ab ztm43##-7=Ej2H8tV_JkUgN<#6qO9be`$mjqt^KH6U5(0J3xnmnX6|~Q0n@iTNi#~bTbsx!SKxMvO%D2rf%V^)7?bH zh1aie;2tt0Kq@wr3O`(W>n&E`{;2%;hqyzK1P43Z8|q`v*~3OCANH*{qR^|w0!RF` z6*xJQvap9x>KkGM)XR5cSD40~+I^y-#5(Uj2!@uN`hU4%?ijd!fzRT4N8I?oOAZjl zt1>SJlS4j^+j>yw&uBwbYVfObpR2{ROy5tqqmrz;x35bpWT@K_UOrsj_qGgOl$m>4 zHr1RqGO&RM4>u)ORt74mJx^#I?!=0LXZ@!1I-CNOk@Sj029Lgng*RUcYD%IZsze%EW(uP(( z`3e@Y(4!-P@MM4>$ofdm7n9b^I&67!_4KL%3|Fa9y-V#&JAWZEa~Afb>mDcj9$@99^;)O+j}1E@d~FUvd5N6nulfMFW)HGVlByT^_}p)PLTN6ZZ4`8{s^wT&$h^ z1~~&^*JN(#P{BG%Ss(<2NQ!b1$`j_#ahk>PP~3}=5*1I?NGQVPcyH{kmH}XacfK~% zI8v2YQabN+1PM9y4A!gn7elf+>cI&B@HfH~F2G~{6M=q;U*l5=aoP@4!iWud90@G+ z|3hENyTH}b4Qu&*@uPN??Wu#h1cdM4D5{Bu>OH9we-%KMEHIv0Y=JB>!OA(MgJ$b* zpbe}$-{8*l^bs7Dr$J(57 z$&!*er1sJgQgt%NpnMT7r6<~9u-<`HaKJ26wGb-rKk0MYzN(sD%htqLlVA9JTTP57?od|+s%F9WVFEU9B?7q#nNH%D- zWIjF-@085|{Clpyui-eEg(%-xD2%Cnkdg~dv}0f16f4X%sUzW#+&gA_t@-V@1wC;} z*gjaY2bvkMWu#LSO1;N^H#jjVWKXU@clgSI43h%1l)`|cSBu6~MGv^4Xxx)uX%;q= zMBTs=z)(Mlk7SD*l%mUdaQ(*}hp8}5j-f?p#?uvUP-2$@lyP!JEwzAOCxWN=F~c(m z5@w^M_NeP%T&2$3bAxaNhKbWyCcgqYOyomt@9ka}sLO zEX@B5nYR=LfQY=gye057c0H#>oLP+D%72;2J?cdbM9faj1l;~!x zWrm=AMNE|L4Txvv>|_$~f>*P9HR;-O?W@sDxA0`6?#cZRB_#)*^Isb+qJ##T)B%Ll zh~yUH7?0G0vf+3kqJ97XuZ4+$o+@9z%r=+-obmq-=1t(lG-}q2Hs^qeo<+j7HlLn} z{Ij#e%7=%0&KGbV<5fZk%W-!yCJjjk71c+>r&&ayw=#MS_eTInb^2AaqwMBYJ$b>@ zMjkWjx11h?wfJI=I#0V`4yH|9l8d+jvYF}JXRtkokE|_%%)rl>FJ6&(Z;L}q6@+a! zt_I3z#Z^6K7V+va~A?PW{fW`KIp|ac%)lxj-J_5SqvF1h?ikF7D&z^ zJob~Fzpli>xS^b2n|Uv1LaSvJhz+)xEOk56K2UUQTsjdSz;FV>06(LSEtl)kPu%56 z?eM&|q<8ICb^nYcb6sv>x45tNk7A^1{M0=S3Jp1w0T$C&?9GOfx{sq_k=@#94sVs{ z8Hw@FeO`&U+E?1q{`A9|<&sRxaOU2R z3$};wWo3Auz;hZId9|6jonNC}(7!0u_Wt#kR(kL4t`lmEaOhCM$i?!AznSsoo^a+B z&Vhj=qEo%4c*Fo>$r1#++RCS`uIyX4Z$btce+~VZ87hepYQ$ZtdASagbuqWWL>r18 zV^+x4)EXBR)VG7fBwqBu%>j_RLEQ7IhD1NvckVo7DNgLpzd^;2bnBztO+cX`utaNu zNe=x0qn^``Cce^JZ$E%}=PjPWR_w)qp#{Xsij4@0Ev(h=f*6az$=mE*v1&lLK@Jqw z=2II`Kgb-*9#~Ol(;3peW1nI+bU5cmydnyFFej2hYz$?pM7?V`1hqk+g$S&nHZ z#ijunO=&-%L!xDmpNnZnTJUoE_bNZS64u-V#BF-{1uzjTOV*y^rmOH9rRhK6ubU}L zwO~mBiT+&4#GnP}R6WJGk25}^)Vj91ey%)|hI=3W2~P+C$?gGCon;92NCZmy3zx#; z6_dlS|86ED5TSg9!BB`7XsE~%@4Z##5cz*}#{KythD+xV^ChxtAt@5zv_^z3{~#nV zzN-!as4plOAEJ}}^$(AEnf}n~kq$fXuYZ@Gp5`3?wR7;HLT<#uu@DtieH^T?@LMNe zD(ojOH#A+7$S*{qt1)p$Tb|UREmEiy?S$U3&aSd+hdQ8!TbnI0Obdn0CtE?C7LPl@x3y~V#E9Ki!vwG70Nlh$q&O?8DW6} zwx&=Q66%Rl3Sy@~zA&r%+B_?e_X#$S0uj*T9B^I~!~33JUWXTCYal}WXbI9$z1@5S z^|b+a%MREI086kD2%PKfK| z6Yl!-uerilWmfvuX*8|_txANy8qNYIBAi$ttof%!H<=<#kRZtpc8!?SW+NS$|OojfYQj<~^WkqM0yumsp4?GNQ0#U%W@xz3g0(EE?e`xXeNf zDBH`Xj;fJ0w_kOCDj*F8_J5yz<3Qsg3sf9qog|iu)s@Og0{xWtwhyi;=DmbSgi-h0 zapELuxQgJFp-okt{j>EJ<|3>huBxNi3`#+i_ zMtvxeX$>;(efv_=q5RuIDCob;*IeJu+Pn!)5{ybyNeNo93>@lw^y(;>Le~7ID+;80 zkn8wt&Gu%y5^ZW*d$ST{=~(kGjyAG_0b7sQ`0HZNZgy@p@#QlF(mdk~>b{ui=>bC%eDfRhj3xy6@ZLK-U60JUrZRD{-{W z?fzkar!1fspDk`BbL&zxlR(lpUJ^BFJ=NTR6XOc@tJy9x+Tqvoq`XW>z#672L=77* zOxJvNIsC#&?LeB}2<9R4BwnE|cEkd04S!6VO&?+*8Z7{9eQy*CAPgA0hS26s@sI!7 zlV7OCuF$}!YHy6($1eNs*Hh@vEnYF8T&R1Kt0K-$sox6+px%zV~d70YUS6G*6z z^)p|w-6}i}R^lOVag*GcPX>sBu0zXGp%o(F6{uNaqLeGz{xRDP=qaZbkLm&DIId5& zzKPv23avqmFrIzIF7g)3-eC^3Far;yMmr|}0Gz?D@BK|E<&|{hi`9_^9Ud(RdI4^Z zoVm!&=UkR8jRN3JxKUQR^H95*3IEj@d>ib-7s9wHNDB6gU*H;iXha=>n7xpqa!1kEz`k zk6rGfZCg$(8Jgq;KTq2JQO^o&w@DjY{W#KR$q>#bxt#qVMo>RI`Q!QH3dL)i17@3DkrIIur`u@v{_QX`39zz8sr|dWG z2D0`V$h9zD{Uj(@{(w6{qyCz{ym+ck3Ioh>$Up7`b&vtE1=CxU$?0~2FOt;_o;_j& zn}jDBVY(4)QYL4>34BN{kO7xl*oy!lJa>vha07JMi~Kg8U?0wr*MCD;Hc(9=UqTh= z0x#kIJpTodrMLX>52VM0(B7LwkC6f$zx{QfG_cKDUNE}+p=H3jz{06Dc4 z;UQX-xP%*T1h^qSmO+;&Fz`%;;L<+pc&lw*yYzIpR7SAqAUfTy< zWo3Q|M-nJl%+{4r^GG5AX zTt7%s1vT1+>($p$@T7ytdiU7xOD(N^<3$g`Ya0xzFC6yGVnQtZaYeta2Y_NQ zVSc|ami)-q)7bN*FG%9K)9~yfxFWuO904@#(GuSW7Hx3!ya8|?|1%HPs9+p5;r2ZR z-BJD&b0)IZw2>Hu_QHj^40vSdrO0UtY{{ptp0WW#bn8nBpno8K^sc2K#>$bp;X-&Hox28Tbq~-gCuneR&7t8ld-psYB)*yFuE( z8XwU$yOF87eso@uNd*TXs6v8gLD|=x(3jq{=>{5@#6$%FacZO3F!d*b*O~Du1{z)Y zL&0>|+yru7X$6KDp6ijg?LC)wz*m-T^9IK8(y?^bOU;bwGQpPt5e8tjZu&#_7ALO@tz+$?~0D+)U=MjTt}1kX??(U2yQCg02Ae^cq4N+oAFi=t@8`mhmrS z+>_7_rBI8Nw+c-RzaZtmnNbBhC;a*F>2Em_D#XK?Mn06NTOSqfx+pQbz*o2WmnvKq z@DmK$><{_>w3t`YRM{LMjm`yr2=MVWf>*p1&d>lND6t*HfD%5&J`ZvdO8H_?FF)~XcYjrtEA%Jh#GSS6}##EM29U^LZz#wRx1h0vn`Y#{;8{yIi(L++7qa(w!1 z1V8DM_OFBq^vsBDdiCy*{f0tSS0KocIOUyjs>;O~5O$Q)xaU<0Ag(B(OFdKkQ~pjiVvWU`_DCe1(jkh_agC->8p zEOv-W&udX)he0X;05T|2w=A%P>W?3$OOMsFvD^fhvXq z_n;_GOM3eN&GOg18(+=&G}v@*#XW)A;6R3Z*59k1h5UvmA#;p@^0F!z04C5R9Jk$Q zK)Sd$$+5MWi88Jy2>Qx0X zL(`5E=K4FccTu-LMOSv?@)* zhyuLj)-eEnNEJ-sx`O4QqwGSrc2Ix%>$e|^-@xB?+{XNN^2cE+>F54?lyo%_+ltZe zngXG@2BzB^zpo0T0_N2FY(4Ox&YamL?ZwvY5sw9Yvtfx-FAHq41_=W{Wn|xGD1b?D zTNbODs%1!J4L3kDi{K45ttYu6a#h2UIT}QdS_3mVl_-{7eO%9D$MnOilp%%3jGuCd z=^1=pW0usF)8%4N48fa>WopF|*Rt}-tc-1g*HVuKUTgf6j(68rYkB1g`lXgbU472M zIu4UgOb0&^eoP0M_2$~qpW8{dz$8jm#>3WKuJ}J(qWB2ruBIqC+ICQpeB}d)fX~#< zd+Z!h&}34IC}Kh3ss7fh|9CRx=k%A}dCmnRW@rSmr*{5!^IvZw%!{iiV1j2LwECyW z^brA~^g>dM5k3mCYL48`$p4`rx4g8NupmK$M;qUUCJ~=4 zdETU>FVN;de$CF*>EZ%NkCqT0-zsl>m?3vi(`tHauEG4W{?WUd{>D*bke$I@pDX1+>Ho)PzGuWk0jg4&YP8O`$j-m z%cMOZ1^qVVKn)2~3xNx1#aHd2C>G%{!z<^wS)11YN3kgiQWiXO1Bnk%!|)7^{o;jA zd%(@X_6va^AScsyxE{$?ez>Tg5@&HA$7yKwA&fc~UeE%*iBWG_a{9i3mn&>ga0h@x zK&0T8f#w}P>Zo?W-_w%CJW9ttg!E;w@x^sPTkuiO!#l<98~UdkiJ2>V%$#H~?0vv1 zz1{eQ&hk>N-Lx`wW_if8(Do&z>hs$;KaeY!-;@Bwtu#tScqU7?5K{!GC$OwzM+gH8 zS>lJa+|Dih7WX@RIxYEMG?_7ePCNTk{;i}7)~D2wSAFDs$q&)3^c>P~`jM7-FW)7{0o)gW(zO$$9s?P}c!1U^RvJ ze2(Au}jpJt0$N)>rz}o_q$d`;Ao+NK(MDA-HUyDKm|BBgAdd zGe^Z)q2(VtsX2TGiOaX>uEXCmf6C~0Z*`a7UhXc>?_1klt<_`Gxy}qPyiZR#10XYk zO=qhrMNppxXoLU2PU9R6gR3wn?4rcTO0SOy*gg&Ue zI35A}-PH>x0aS7(B)7;OG&eQMw6pzp z+id$1%zTyNR!(neFsoPlD1>$gDOo{Tso z;f4k6`(eXH@icpZwtZTMKQw3it*K9{6Xw7wOq-&7=eOs?G|$lJ*QE(6pK`w&<+p?f zzzgIFXA@ZN5D4A{y6#zb`RF8X zj?u(y^2hIA8WUUlVQs{i zuw}V&4%CUk(5bjcnP7CxY`e#@VIL{4dUx92MDzH^+Ahw<70JhVrltDiW$Yh;e)BmO zl@Suqc58iNUs^&`B%``t|%$s2HuObNwY-W%hBN&^Js=MF<{mN zE64_(u%R}9fZE^_Aq5f0lnsn%5Pbq@O6C@?SbFd9iXh__q5Ik#E($VC{_RPyO9Db$wXbveGQ3gNQ5&+80n zP9oJgd287HDvb!38_lrc+XZfqL&sP!skVapc+VS7gCf+SOlIukM>9orkm4PSl5}bf znLlkw{7wMHqN+~>^~8xxT+aD6Ab zUNJ?mf<(hbom5<+I98EH-uthV$hJo0kvI- zOlhw=!c(*dVs-SYoS7j;NR>^R?Q`YOICTjp%#!y|DL)bC9TG$Ws6<#~WPRA9&rfcK zWU%u(Y(6r;K5ikb^!SWct|Y~MW?U3Re+j~@aPN)I_FgfnbEHutxh@msUtc6X)vsGW z4T__yU5n*&i=8JpAE(98RBaLQcAXK;3s3it$}Mj6dOjymKFaoNrW37Sz-T!Okv1_CJK*99Cq$45kUM?OW_!I{Gmx zv4ivR;AefwWW@fSRC($jkzy|GJkl49pPJ}EY1(gNC}=N8IYbxJ{<7Utoz7wNpfp@; zWXJAluUOy_Qw;fA%PpyLy~9#Rr9Vph9Sd4fSiFz6^3#hs)QUzRQUf94UN+6%aKr-A zU+`@KcB29CT12p3=dnYMQZw*9y4}{Tav6YrWIabMWE$qpe;Naoe8?ry81uzL-ZF0& z$&Vo6Si7TAJhAFD^w`u}dnT39Bv(D}j$W;=^mTri#`ZUW-%pY!#y@sKTQp6r?1QER z7`Zo0GH>=P=7i=&c;@Z@r7zs9Wzp*y4*?ZXpv66qLKV0tSm$N-Byl!r@`r-X5-lvnP*8atq$})YESdKeYanpQiaoe< z1v(?l`FeZdwUjaRi!+zW!Q)FTyBk2agOmkh(gau9m7b|KJSfz)_OutbOjNekyDNUR zA-)|K(tFZ%U!&_jynj%r3GwK~)-7zkvfkyJ)gy}^bk)$C;0M?G3o#>>LPcFwoX9>I z9&}wMc+VgkhL|BT>?=tw>NluDeh+3VAPt%upW{Kc21qUt0r&d}vuxqjYl;>wv3UyI ze?E1nM&Y1Mc{PM@+@3`+tqOf!ig1V|`eO9VQ}FdY2l_ZJHm%K)nq?GxS|E5?%ya@B zOlwe_k8(5<-2W_@YLBm&QvYr9WRGJc+ty&awUvF`?-c~^_$DvBxH}8|qt^*MixVF%JRk*$9Yk>W7JrrZopOWP zVrUy5dSY!)r(*ZkvU@v9rhxE2?8Nvg0%TZQTOWRt6aR`NhQKlQcq}Iz4j}K!8yd3UPrjl|zs z!~je;g8)s|rAgQPmhUe(--T!llXc#9NyrxCR`ab)LA(zBI)%BUu!BtBlluygs&N|$C<0~O_2MLAeB*{ z77rl;`A{d9N1EIn2sH;ZLmsi3d*93eCq&@$T9F92~>cOZBj4?8Af zKQBwBdcS?C{_ii;iPkb&Y9uRr-Y;K>!(cFY**ySV2Mg*;;~a}ML?D8jd9N4vQnKJP zEQ9RWjzd83d1qj|=wEdQai8g$YqV9S6Btx8N);|G+R{XJV*E#WxIR{11R4WoHtnz$ zy?U#+TAhJWmR=3^CjU?YU`E<(g|2lyE*)Zt;V8BptGOg{r?<-dRdM<^w8|kt|C>~Z z_?lh=4?H2>F<;lI00`D7B7*u?fc*~dw^1GD(S9uxd$;J0{)bYTedFrCdJ-Q&$OSv@ z6P>W=E;58$QMA^&gG-@XVaDN8sFz(6(axD!`gI%JQ}^l(+zLkH(TKQqu^pcnzRLK> zX0oaWFn+l%5Bg-n;YIWe-PF&VYm07#YB;#TW(`cokWBgjDSR;L0SOq(3ooW5nbx62 zIOaKQx#9#~G2e4*$3Aw$Ze z7W9?NdhuS~J4`+WdUcatNlG8|Zi{!pHyWtwSa7RywhKr4$@rQLY&)D z>7Q&;^b2H5$Czj%OWoGA8D{A(PSp39vHlH!;2LYw$NhNL_jR0oRUCnSJm_n0SnJUSkXGIwC@hOuf~Dpzc=9w(eQ2rb%B zCZ|}W>+#NfDEOfg_k#;FJguNYgO%}F$0E<#Wi6^rH*hY(O_kD%XN!DZXVE`-(uCvB*@e(B$O>vWOynJLg z&Pq%HmxT2|%}4eSqF|+GATI^Acd?^nfxLF^TRXt+)yEjN4)dH)6J99A+b zF@QvmUir~s46navZ0P9sqH=GE7FSBw1$msg5Y=N3?@K3j<5Xf%J=ZiQlh^xw(4`4hOW*mD9227qCO=7J}fW_%Z*p0sE%KPmtii zrb9)~5qyDh;`(=BTAGTXP!Okd>?6wu^jmqbA@#2`%+vZ#=MGasz7%X*vuTm-(t&_b z03rrj8rykXpM2WsUq2pgGs6r9ADP)ui5X=9K?V~D4mBUW(XM`wGm$w3s$ik(788Cj ziEvsAfU*%#l19^pd1v0ZF77X_AkYBL^TpiR&-M@72NuaAmgX!4EKmR6uvGW^70)Lq z#MwF?secb?rEfrV_IxN$hv&cZCm3k%-dXU)T4I z%K~wx-MC?Mr$?0ChH3r>bw4JLIsf?-Q(TFV8r;Kn75}f`u)GUWRg+ksha@r}mX}4k zv`fY~Vc>@1sVHP(K%zNd^;b3;S=uQ+JN@G_^X#HC{NVm_hRAwoPv<30aU$EA;B8@r z9MA~yV;gZ0e4lAdoWk-x4-(Zm&=qiCA-z|g|8s#Dcf5gL9RJ#qza7D?YrF&iy1qkh z~CwuT%-!(v;xg;BjcSLQiFoK1PaKZaxy?nwRyqVEALl~+I?dkuh&JT6} z{tfKt1>JXi`cFQ8yiHQQc{lx&`V&ACoP^hNXAIazAQuJnsagv~h!mR)&V@$%RmQE? zUm}5oZcJk(QqLXtkxF&1&fMp5C=U^AIXgA|L4m=KG7Vqx}o!Q%@#oh7?B7sdH$76-@e^xZ5e<;Bd*f=Y#@}v-oLX zWsgnEcD2erNx2v;oZIslGIjNz04j*UxA*;l5(cuesXy(DF^dVq@{THIXo&XL>fP-B zqv<^0x!nKvZ)9a=&x|D5Br>u>$PQWAD_Pme%nC^e*(=H3Bq1StWoKrKkG<#rb)WC= z-{aBaoYN7X5BL4PU*o!-mosL`K0OeB9+Z*D7_%nx!dfEp)?VJ-B$zI{ieAf|`ur;s zUR7pysA_JrLs|dEo>SdA$ifk#$t(X%Yt$;t62Ne@Ugnk35oGo%U11qg4dJ;fj z&PPiC(<9)3!j0ittWYv)eY|oIJm-}i%+&?ecsE<@x2RxfGCGH`pGup+4dyDE!{USL z9FTL-V%MhNzs^vri3L6Ny$sf$pbW6fY*FCq1_9&Iq$)_EKqCm8l)6E9!8r6cmlQrh zEGiG2Z4M>5)!F}<3ZL#>k*;?fdP#N#F*do%Hlu4MVgNq^TxE(eAh|FBQvm9ZPOsix zZn+#B?AK0MimFMC#YtsL24O4#A>c_Uo`PSY)nZiV4Up6Xqw7JNYWOJT#!!gvQ@6PA z`Ve3(q*89?Y9qChUguZscK4&=;+V&!bT0A#^gqiFIO66Y!^s?;u80p1cndVvDP=>Z z&uq>Hl~q*AoEJL-%5(~XXEtY=C;@wO3z8vkQW-j-LjemJQg09Imz~T88Gn6cL5s)2 z8){m+Hkuk(yo(v1>couriO&Se!j7WU&{#r7XT&AX4x`L;Uoks322>vF6J!bpwqkmg zUO&HbYx+`WdS@v8-co(~ECgB40SoRDK|~)B z+m_GP0Dut(k}-a<)_4&C=#h82T=L*i0q$PUBl>#8f!PC$d1Qf$h78S7?#ZsADv-7f zx9>b`u@;a}XC(Uq$r2oTthR_i;Mrn(uJ+4fhvf zC?GxX)8iMM!hBZHAQh{A4iVZ8j9d5vU$@M#Ib#%ZC>8s3h8%&Y`gsCnU-MHUk(-3* z1YU&2?oA+O9LM~26p!w`-DTDK3!V-IQ>~kBF!|a3nyL^}sN(?jw=zfDn9~ROibx|z z_0|oYx(9WGH&zY-uMIOG7_e_M%sOpHviypNn-5m-Fk(9h5K&{S2LSf*$p8j zgyYFt24-@rvOvL@`mg5j_NmPXD2YR|P2mT@oK6?md^E!^CAPu&{2h{!B3L_{t&Ix03(0@E6)Z;($z=zp?3OF#ihzC_w@QKp8y` z{9HEpzQdw)0u?cdZ%{-jk?x2v=sdTPZEHtWg5SH%DNhQZ3nOT&w}q2b&1()>m$R1- z92^pj^&Fc@m$lndugpD8N2l~|w01J_L8>b|4(Evk@1B_4VBs@*~*1Ta!%M3;<;j%u7QzyutU;+MT zLd0WLkzfPI`Z$0xrD*20AR_r?i9->hd$VW&R1IsJA87G|JsEnG#0fjBtVEDO1oC^- z*zZ0f?Y+pklvZIQLmt)$a~D)lk(XzEmkUti5B-7aXB`7(_z3Y(p&A!Yv_OUPYtNFM_`r1oiOwr_tuI7#-|Vh>D^@+-#Pd;#6NTn+RqI`} zr!K_=#f#FOBL(GawU&vXql0-fY`@c()C<|Gz5iyUTd*}g4QSNL)5ss9VKg$F0&dCH zx73d+@=C*^CrtxJe~5F>PG~p;FwJ28 z`ufxzrtqMFK-SUW1U+ohu(|mal9lfXkKRwtSP+pzlpH5t5DCpN(8P&g5m;qdG`M#7 zOlSfVg@#tGd=sD?y4`^+p4IAbjha+^R0P$E47!EI?{U%x15BV_%GC1+K({^iVRJJP5tHq~MvDEiuw4N!t{874Gyxp6$>JtnRXGjV?t(C6tMBRv=wkAN zFJ!>s67nIfL2&)agyNS| z`)YK^D~4%iRc9mAPNQp3>a=7xq2pkI^1|w>>}&9nRI{|ukuzWa@bB1vwEj!}^$+oM z^+t@TM#we}JZjzKXbb`42SiZ%7_gi3tpF_}|GnfxkUau&tm-3pZ$RM<4Y@8WXz{i6 zA=iCsa2)mwuQCYbIG(iFeDqNJXzt7w$*R=nHTA!u{;UF-x``hz#QqmA4bfC$z}sYn zC+8TV_uW>DRbU0Qyr1fCdNe^J6$FsAfOwn_&t@*2&%)E^2W|~niq6pSt-riGmz~R$ zz?e9r;Mla4RPsP9^62OplN%M+olN6%n!U35mCIw272zKX=-K$SduSiMsSmzm)Sle? z-Ob(kf{U5p^b+IMnzXM|b);L=WiZ01_m;Ti{V!*+8AuoH0XWy&&ktw%5Gstr*T2{Z zG65KT|8Y=&TciNHv`V9CTG|p#U{c*m^JJki_U4B>8Hm~M5J@u3>JN;kM*+hV0@A5e z1)kakQc#HefMkhXvrr&8Y+r5r>$HUsu#srI{k2gSi1$)27)II(8CKH2;O~a(p#DqS z(8c3{vD^Qx6dE5z%(x&221;aP$g`>xL5qNE2cToe-f0c&<(8rdS06LWLf|oAYQD%= zWRL%{y&KCL5B@G_K3PHG!%2#N3cgFwIcW2tTwht>--P+JE%7esi=4EsU7luX-Yyzl z#XXu_2wy}UI5<{|fwn0}_(&-0S+VXf(;5yL&kB`V%{G+tto!(C1dDw3wb8;c_B}bE zyMb_be_UzbwkT8URiD)eG}8d)#18!3X}VmXHc3O+1QC+Gm!R5( zCnn(E;g+_n2%|@qSCrVY#;K}pPF%7qvA3=hWenbSz>tV~KaLQ2hX4BvEvRJDCK6yDH3ZupCXtKx8&mi_&5608DU2mRK7of~L8D*rHqSf^6O zjQ?D(YVbJ>u+0vGlPeB>YMi~aJ?^2lMT$iOAS=7B;G~`+cojThaE|kYM#2m8uC@dj zSc@(H{5adVse{B?51Umip5h0mF?-hEA~!{TK5JVo%_}K`fdtB#Kx;iP=6g&>0`yX_ z$-GO07+4FYq)P^gzi&WMP_HVf-UFkaXxV2OIRRYFpc5N3HU8iP39blg!263%0Rog? zf|(QtNSXwZSrj5=yxSyao@)l>ywg2#J|PhicpB<0T+PTVt6&~Y$1ZDyhV%!5R67uz zz4Er?k_Tcvq*oaw=*G%V&rirsXaEX>FRO)xW(m_4q z{qBPCK;FSlw(*SM(-5kk^L1~K?=cNib+kaErV!st{_lS&G0N_GpA+EE9Q_gotIK^) z=(I2Lc8`Hy)e6gaFJyUubl3>_3wgY&f*$JtK9K630qpI?@N?6BMM0d!4i9&6gmLk5 zv~Ic;v|Oi?bPhNL!<}$`wIQw>S&bH}ify=8V}O5zCQV|r(2jKTl(2tjM!-E6FtynE zGujaEP}6lY!SE*{mj0}gY*aaKlb&+dh*UJSx65pxk<;%uTLf+03kWJeTEgm{m}}K*Wki8gx5GjUYY=2hiR~b{(*9ej?>Jkk z2WwQ2VkmP!GJC5n|Dc@*z<%<%FX&$boc%Umb*j>Ot*v!bmtyY3#Q0-%>-aj zt5xQs+Wj^dv+~bVrA5&q_vCG?IeQSBWC+6sTfsWwEpgwWb-E&_Pq(GswU?ph%+N)y zTEtUiZ=*kNX!xS#^$T$XB`=Y(j6 z?~Y%&fFsmQ02&6c6capr0C-Orid|1&sXhpji1Q+Y$;r(8brOw=U;)ZlMQ9m8zy*A} z?Qc)vjsr>n_?f*kmHU?=9f#meW!QMb#tKs^ch;aZ*Mc=$+~Z~NowPYr5Pjvei-JVg z9y7q^!`%u?gJDAS4=qUTElw(xqC$%p^FFY`^B-DqDiNA5yK$XhmAU<);4A$BsyZIZ z06y9{%Bwt~sj#^pF!|9KiSh>h0=xE`;B(drx%V0dCjQP(JttL9e4?GJ0>-F@pWDDY zv*4S3Ay%q~6YPqxd#v7sVnRW}BFzg%c!1;^7lKR^#Qc=?Y7|{}C&8^AGp7qrT3w;6 z%1u~x0xYpT&TK;*4pXA!NcY?aKbG0zZj)@idLIp4jF!%eOSoSnV&s5q317=jfCHSx z(cK(#exPW|7sBnXCO~hE%o=wfBka3n3^F~4Z)+|)9wB#8B&3YctAFx;D>QZ_xop7Z z1H$)0ej{??AeqeNY4r#56fFnzXiNFwcLm*jEqKny9Il)@OhcqQ$Abg@x2IDfM|;cB zkNmVphh7>5zPJ>-mVUPP-9$3NG<`F0`t8xm?S4)IuEF4$5Yx-_!NyCcg`H+<9p9_l ziTKhw4K71be$-xa3oTJY{LR-Bc0(ShE(8hFAiHmXY549ITK|7ON}I$1KT>&bw~(nh zfyvsXUmB;r9}j$UV;u-E7YVFDX5i@{RD-%5%JF@yXCw&V0OA?XfF*A=AS1pBRH+(A>u2em z_ktu~v)u7k(`RO5uRVc7_I0?=H+J~!a&NKk^Mv~Ru&tlA`E6T;(@U5!v6jcp03sQX z{Q^=TQ%hT))%Ba_Q`Ng7HvsBFc-wvJPF_mP*zE3m?s)AcF$4A+T|4dtD5{uFN>F*h zz6XTjYcKdyL9|F*u$$Q3-QXUhEMQ>aEi3XXgnJO=0q!=}(%Tncwg%;l9p#;E1(N|$ znD>YFgfwNeZMuPYZS^2X?!}`R za?h3mVW$VPgFa)D6Gne9--2TbMrSaME%xE}Pf6-dQJY;9*LHGB%&zo}V|K?`REiG4& z{tdkH25TbNRQ=~TxPV=i73EDdq=!Y)6%c&wuZU0!~t4}d@fnu3g^5#2RV zHLYr<++U$rfPkm0Vh7TU;tGntg@acw*!c2fbkb|B=nS5hzD=i9<96P_Gn)s_Zq|hM zg*=4Aq|XC&ZqN`7&Mp`yA<9zZodIeZNp(d|CTGa$Xq;tPdm@J+m8ot5tp>o4o+TEr z;@Ai!gDVI)@%vt>UiBBe#ukv711oaG!Ss7o5bZS+Qg9EK__hKS&H?uNfR99+8%yDw z#sA?Ek;4M!e>W~SFV7pT?|*Lot8Rd{-Z@vp9u6slJ;1td-aL%$u>zY2X52R<&hG;} zSDqB)->GGhh{Dh0CE0m}E;sZb9_}C*!~Dd*QU?<@@9Vb$^tSc$-lUf>r7K%dM?6%t zjDSr`P{SK*F_!rg?!lJk0J-z7U|&BSG-6<&C9_$>9P9RzHR%4pi;UQk)^6Q-`BRz^ z9xGbKZ+dLmzGevM$5?VyVMF_Fm-)JHO|xJLfc3j#j7%Lz^(5U2LjK!Z+9`y8NhbI|^_5`^5 zwPuO>^&OpM%$g!O%VGkbT(GoV69OO!!RjexH|CsQ&8Rs?oVhov1W5_VMvlOt#NCM7YqNEM?avhsQwz6+>LN9Le*9-!ovsEGi=#DRW=p?%MLASMgZ8n|*< zPLo0~4K!8o%>)SNB#+p*x1zl^oHC_)A6SEaxSORZSzVKRA<@K~`egHx8~EltjM}kg ztVqC?<>KqpvRk;_pLVzy8P5C)bn^x#RJ+2#YX$oDiTnST{=7T~;Z1@->7(~;Oh*RN zFctD^`kGoT4IQW}bf92t#WY9g+~crqN?>OpjHHS}3e)U?c8T1ZBtS;h+{yi7*4XR+ zcXo!^{yY7%Ir%D4#vh(PO5PXvJ@lUI4mhVV+P%9MT{Cz&G0+|FWQ5-Sj=Fmgy+C9L->aMHzkIN-F~&%-Q)LNQBuNn>=(d=f~1EpqiYM!waddZ z@Wh=T2Fu?(AROlbD26@@vKmID@Ywb|6RAF1zBl*g2K*?HI2yFGHor3*fJnd+;84aF{%Ue3C5*p-P>uT)gl`(4u9vYCB>z#ba=8P@ zB{R{Yt0{EV-Fy<{U~v^7jdX=n=oc6M(C@L$rQ@9vLs8O#dyf98DyreXMa zqGXsdfkXn~rIafd>s}|xe7;ZF>Ku8OQywbbqx%kkHtHxeJCfa+Ceu*p zgY_7p<=bDfvQ@ksuD%Z@Vj@qts{v&MYb%|)g76=3Uev3CPo6oJ*EGeAbL9~!P)xys z(l}$y5^c7Kan0`)utC920jA4O)~TeBgv9x4Ns^@JKZ7jv%ODyVfzUpwfKyyK;W@0~q9V=eqZ#)JWPF9!*g6!nXzh1i+$E0iK%1>Db{Z z3kZJvM|(yR1${JFD0cw&53>F}tZ13qZ%=Op0hn=YQ+nm_nMa*1X*byZ1|6P|%2eO@ zLDh8^_GK^&wGfhgZ@9C!loxO%JQz+zSfs=5ou8fZi2_Gq6P(S64 zcPx_#Dx&ah<~>n{4mcN)Cp`efY*(K^o6&zB|DK775n_)`}yz>8NMUtQS@pnIsO1R&6Rxoe| z8a@2ppZ@#myO_(j?#@3K*v4&3fo3AUApL1|^TuczG}H5YevPAN)$^CZy?$tOA(uLB zNP1vehZo7leG?N&kh&n&dIb9jrX~W!7WJQ`{6ES$A}KrH^pk{plL-FrdPMc5NZBPU zq_m&<@P3`e0Ho_bDik%+TqB_s0Kw0`=o1-vNB@y`4x?Z55RJP@3Rl_f z>Mf%vh*U&G(%prqvuY@UQ{2Ta>`#VZt>vne#|F7D`v$on)$B;H_#2hO{L06nY#JOk-o1fRs4q%*zcreo% z!au!m(ZXyE?6c{kDh;|#c0y%lT*Vg*q;g@-;6$;?nL>d-0kECmRn=nlV@sv`xCSF4 z(I+OuM|?dxI1#Nu%?{eKlQs%d9sm>x? z^Fk-+$-87;9b81{?WX;812D=6H6R&g`KCh8I&Juv!bpv3Vqp(WItE?iFikcv}0Fb)^aiAdX&pLbycQNb2Ly{9gTJE=OH!}4OaRD&aNf8!u zj-;)Gk(TMzyn%surXi?L2w9(X!wvz|USK-};C*z40K&7W8I-P=Vk=Mm6p$y24Qb5% zRI7MCld3oeq*cV=icgf@PKo8 zf@1FZK3j)jbQNbCpsGL{k>XAVn+7}Nlwr7c05u!}{u-5mqG;aY^?j?Ut4qi}O>|0* z8y^}enMO5Nq0IjaqA*^BhIA1DdIF(r4ndj2r0PU76!c`@6#@Sg6;;rOB@MALD3|ZQ zRQo`{`S|BwK18xxetyO+yLlT*U(8><~Z6k?=5FfnBX$x)Fvhhv0i zo$(cxQ>(vbrFfx7A*UCot$VFi_Fatj@An~!si#XR=Gv(Qfe2c$iBNd5^wBM<)s>YWAn{cOnE(vk_yLY{3Y#y4V9q;P;$5ZG{s zmz5>LExakpY`fv(W$Q{%F-Ytt_HDiM_#Oz-`@mwJyXM=!EP{?CjpwZG0=p zMk3-aW4acPpy8U)m_Gyp0Tu$NhAHC#@;=b$Vc7Ud^ERA#w!lY->Z>t-@x!*t-EMHT z1?4_yBj}3b|CAtIje;tH*O=%FM)}M{a9jNr+U>ZGZ1&gD0Y<3%LLzkLu3w@OZDInZ z(d4tgJ*9A~Sg&@L_E-KqoU{Z_TrIY!9$V>n3Q&h|m>lAWP1n`Rgn88~8h?3wmf}H- zHYO?(F*?td+A~PErQs+Py~H)J-1azcWOdbmt$TpZq%S2$(npS~cY= zN=t;3$RAO>_T?Sc4~!rBZTGu~5+B|1m5zI@K4!*%4G1Q0$8olY%NX~Ec2ug@-H%b2 zU)#u1H*&g<+ooBVMwWKV+T7j+LJB^Fj~$m{|7c*xC*S?gE?8dQ zCRsIYYcaTIr8KD1!vsW!qQ-}e9{92)wTuo1OL0f@36RL(HnpTIHgI$ooZ43Fv)&Oo zd4SGQCE61&TR-q=V*ayX5!sg8V9Y|TRN!AKpOt|aR;z5q5FD?tTX$fDLYhZ^k;Y^H zHb;pfP{WOu{R0P_uHVFYl;SvQ%@jFbmfqi&k9DnHOO2Pg6(YA%ORMs=%_{ z$L{`44?5fIiYo~G<`!sYAigmTxz-__0Z`rp<9XxL=y1MR=mF8DUV)b( zbyO0`evE=e*F^mtHk{AGmjqtZZ}&$1q_wD&Uelb;1U^e=WWMi7EqKSpfROqu)WmDI zFZ#NVq?yLA#?0lOkHo%cH(!YTo_U8^`)%bOkMik-A@_~aozg&JXZkbTe(e`^_VEpCk#>NsX@=SJ@dJx|687OTy zi*(i)r=u=7m)+iEvGY!?SUN#l)jv4c3}r-l^C$pFB#)Yik&LCUJDr?&@xLMKenfr< zz}V0@&LT>ao)mkN|gYJbouKA&5ZY@VZAd@5o) zJ+FZ^`1B=_yPUkdmVpBxf`yD_uf6U!w7T>}j|40ns|E(IDh6=5hS%hRX;$#o} zXi>fbBQG?Nl)+fQFtd{VjsZ(5cR%9}j3ppHqLMW+!;$$82=4~z_Dst@hUJ#)$G@l; zhozuE|61;NjKcEH6(3}V#;Uhb>g-s}T7FDw6#=nWQbp6?Or1r!*I1^h_5NBV^c;TC zgSwH}LAG~jB9!>?qekoH@$qv-f!GRuH%V}y$c9{GM0pB4h-xds5`+Z?xn;544-?~L zCovJaY~k5fxb;GmEO%XB8Q!zgF$)uH#$*oqh`&L-ZpoIUUQ!3FsjhBWttYR+cFC3- zzFd!ZGXtjmspK?1UTveH!FAI`@qWIks2n_hq&%5p4eZQRS>h%#Gb9=}VI#oW0e3Z+maGOFor?57mA@os))g8HZ9+3(v!Y@6 z2WHquV71MGbI)B=p*yaMdY#aOkNUl3umV@BliwTfGUfVl)>%{&%9)AtYMBLCP=BZ! zXUsGAe=i7W%#Co8nZl9E(SI0C(K$?eUVrhW&C3S!d&6pJMz; z(p4@q%p8yTGV+h$Fw~}6zmhdfsY`D`c59gkJpSu2Sk_YvtTb7iIZoTz8I}E|{)j6KftUo7qXOGvy z3B8yGKJi8UC*%y+je4A?g-pgfx~_Nc-FW1)nS#t{S>N>s)Zb>_K09|~&#Ti&6Qi3f z;P;%q68prKC>L!8B7Kl$z*lhY?W5gcNPP-V>1IU8ilkJ)d=8kdY!KMxHgR~s*#p#i zd7)$^)(mqNH1g)Iu3EfU8QIx`((L4O(CITEf}2o!XIz*?2I!EV$>5E>68;lil2@8p(Zjwa+xvazuledD)5!HDrkf*r)(ObrkZ(M=ws zg|rHJWa!(~|6cEm{^n3)`?7eZbX}i-hzKL!6UfMaYLdWs zoV=DOq}1k%vq*%qeA-`)jL9Gd+VIc0T@UgMB`x$dD~;X5FF?x-1K8)t41>XG_1TE zbbpx%%$H7lBx<8sCp1eTtD9ftI%iHlx%~9+o_gL$2URD--2qEI<|GBOJ-5dNcR}_G zN^zidu=!%N#L{>L-e(;Y!Cq>af~Qk!);s3t-rrPvw&Ipl+u(ri@lBP#-9BkQ1wG5B)cdZ0y!*1RoIKT>H z69_sp>b4@2!JX^i>I(7Fun6`Ph$v)#ORz_(X1Ubc`a@Yzv3(|T(A2MK|1&lzYpGrl z?B4B!IKlQ;rC%9Cz^%vvvf&}BPd7V0mD~Eu0+fPO?T+YV&4U^$Ib1U&1es$;Y<=Y2 zTW39;;Fqxj3XVS_#w{#3=yQ2jQ0&XV;{AUNnUpq1d{R_x-eY zQn-ro+HE0CMs_+8|H$7bHu9)+#wAgDV&>DqZWk(|oc;8xZDJVYQ2yj8r)8oh0iSlY z_O*Uz_Qd>8hnnZ3x(|_G%ZtvS8gT+qxBwsD4`i&3M0)V-zjt-LxD9!r(01Ftawr2{ z7U;>|S=P;@h>D8c06x*5OSyySU@8Lnx6O>LUr&YKr%U2kF40QW3-f)|F`K^hTCh*Z zmUT2-`CU)-d4H5yfcS>wF`le_?|qYL?H^Sb$J>udI^=kA*lvCH@xzwq+#zGcVm7Bg zC;wGt{D5^+Y9alH>7D#ux)@m?HScusE{95kNJw>v!rshmAo4HYn7oWxZ1)QygBx;t_8^jt#!;02dwVo4VpJFCWz+*34zhH1` zk%eAGAOo*1ze~!mzvSG{qBKG05evOqxm{_E$`>v2UeQ`^UfrD z`!*=3vxSo5Af(f{J#^eI%PXoWq8f? zC1tUrc-+m|wO=-Tq$;1wbd?E$FoNc9W6R_lNZ*V9iE`~ny=a`(e@q@ct^ik%@{b9* z9Gy1Xvi=wgZx{5d<%RDRENM1~Guf}+i9SlX6JTOhOz`vI=U4ZJFaqi;`|@OUQ=ct| z`)&j=#VD8y(5$W=nAGS&u>`_^cQ0yK!<37ENR_0I>ypNHw5!j&zVm>ks5gw|N=4aF zic@j;RG9^~9J}Xa@4JlG$7`Y_SG%zYzm|><#ZdqCyqbyoy5TxkygN=XJn3`(WT+RD z?i}(_sPYXlwwMXUDlhvVlvDnBS;<=T$-lz}!bbS$RQ{G0NZ)Ztz+}DxpE-02SW7L7 zcElLT>8H~fRM3eSC44zCg3Hd6A|t8qG50`bMKcT z$4RyQ#ILO_r-^FBTMryS*D(1{*}I+PM>J{dj?%vC&HW0t6K~Jk%>S&tVjpajns&W- z9ofhobnH(@5+cbRHO_Rz_f^9WtLeN1 zKW&;SXLy?=-kcph%!tN#MEr{)z4hxvE%mn%=0}8|&hCKK?lzJ*1FvpXJC%w7X;u&^ zz+b12ND@5qqNk^gK-aT3x-gaw`M6>7*^|y)+-U2Kmlw!F)X-}+r=(-s&DGVNM^y|Z z2=a6!R8j{6C z^~%h!B+pmXL;%3{qHe0YySoy#i>wQgImO+sFE0_sJt8=g$?QJ}9X!Efz6Wrrk0~h@ zuCCUM1ox*Lv@0Dd zPE<{6^i~OvDlh)(hEv?Y%*U6?Y>{&GS1yK2^wYGBZS~?Vd-zXR)?RhL(w^(|TGD{M z>K|NWi|_8Ysl+|03(IvbG!&9zNqt-q&%GoPEiC894p3#dW!H5$ANVjAX-QX%|1KQ7 zbIj6kJ<3Xt@zH0z6ECPt@=Isvq;$xIBGoD!_MPn+cb92YELQMO5@F83ux`#l+V-rb z1~041D^RBFEiesQpH?KsvubcXp)G>cv`mcmG_BPueQVl0MUkA_8+lB`Mmm84ae1Ag#@ae$Y~} zr9f(S7o0_9B#NJ$MAu;N%FKa;4J`<*0bryRevJP=! zP|#gUIU0!F#bs`jf|FJ}a@h8B+x$r9D7+1Eo?AKV&Pt|mV1XC zkr>8C1Mi&u35bM1EJ-exO6OREQ_(Tx&LB;=qhRH{=UNMc2IYsO1bZhh#_Sg&Ec2^G zv&GoMGUA!r`xjrpiH&>oZ8#&P!JqOTrmYhMRzoO^%!O2&uT>V_*|YBL>o}6ASpk$(c1@C>?B>E|Q+<1C#?219R9yJEl~)r~JLkw(tHTvp zYNmll0zh3TlqTkzp82``TY?GS%F6c_Fseqv!jY^`SR%ZI?)x2BVjS{9bFz2U;3+;5 zZF$!JfcOP82Rs9BTwRUy(mdzzknKN)3B3qK>z|0d!+Czxt~tTeSLh~NjN#8_(lf*_ z-4+OQP5+!eG+9Lt1i_r7^z;$M0wH5tGj2$NL^rZdBm98?!5!+Kqr%i3sF9g6Il6wxb2)`CVYW zLzX^O))!FTh-OGw*Tb8qW?*QY-`)cMe)VcS?=Mvp+7^7yIj4KL*nzxV=>O&L!6|Vg zwqpw_DXlKUMo5N2yn&VMB@m?t5Pp@S8APhM-Np#vzR02m5m^8RQsQDes=~Ax8(2Qu zul2?d{&Ll2<9?ANB=_z@#=(DE9+;TULEiUDR>T|mY{Y8;Vh7(XNY6=MUH~iN^M~(W zBRd&bGgw1hGDVt%7UO`PkC>Ww+n8jZVow%gZJYM;|XocDsEMs_8L8Pqi28eeCVHB3Xxctt18*$ZmR%H zgmMQ}@1=pMn)wRu6(|johh_2%bM&9PevlhW^W=C`aht!Njwwl@!mczSY@9kUqI}Q1 zgG<_27~3Y84A0|af_c0Po_9` zS_O_}!lF?ve#}?yl(E7Hv&@1vc^DG-24#Z@*WAva9p&{Jh=C{f$DMM>x1j6%b=N(r|lUbD*3C-Dh+?bA?kVWZKk3)yu_=w zsEx==Z**OK{MR$|Xmf4V7C!`LHAQ51)@$8gCLb9W2)9#)f9&~OeMs`9@dh;F+)Rr9 zYn@uZF2)-m*SG6$MQ=oA1R{mZy3=4x3O84GN_f1B(W71WJu;m_ms%!_DBGk~UgLQ% z?IhaS5=Ciwuo(g;OkmUN?|)rwxbFoejric3Z@y3_pb)|%}F9Y`D~{3&wOpq-ze z(?eptq3`ix$?82IEx z#LfX&(#J3_MH@2lhp8u)Kk{dY*oxj~ZvbL-eJ@;wo%1#GY}`*LT!4AL1qLN-e_56$ zIKxH_T?nxQ11=9QFBr$|;$3=09*PyOVim?1^FZP`u_PpjAX4l~rV!!!RTc1OE9H;| z%LG94mYqHG-F`Ib?(h1WUFJ&l{4cM+PRWZWHjPd}hPI`*=+B+M0q^*&cjeXrnZ!bTARxgY7h|DPnRbi* zkw8D)rvbZXEaxL(de|ozaC^v#7U1P)jHD9xSvEZ3xU26Xv#l&Rz*hK9<>8jQ2kdsL zc5&{my)t9#+$e~5yTaA|OS6!cAxp=R=?Zj;3Nf<9>RH0GH1zv3$h^n4DgnRvhz z-Sn*Hv7;cU8HVWamq`^rRZ8$e0dXstS3rQ8C8@m>4vex@+`1xakXvXbrs(Lfq;=aa zl3B6feV0EqYXOL!fr{HNI}6@44jSo80VOFBaO!>RMi%0`1Pl9isGvm6i3k>t81Gn*5kx zNb;IpB5*IkM>>%c&uaB&<;PpzL5o+X7W-x%cso-sCJ}BS+ZSzyVBb=OLQp zEVS2}|M_U~A`&%;%xRl4lBx`fbd1Yq&R&Z^R*G@u6jVr$D#jO(P)a0*}DuzD$!vqAbS6wf00|QfK`inz0&`O_^MF2k9Ajn+N z-MEUWw@ROMQ7O)GuGP$L6!z!he5>|QS0t;!12ms-3_yM}%q(pA zZ*XRDe*xT1r&NhPtrP{{a9<^CGqRCIrlwqb?K!kR-|8mst)pH$ml?|8ei^3qth?Xx zSkFRkN#*&UJ4wykLn#K*(q!J&S$J^nmSq}%b7O*1m6D$mGt+|@02Fq^+foSCr47raNx~Oxr%=+bKN_19mnEg)jy2RE&k7UF zlp+QNWayMkj?S&wwcmwMi-^})mOp5pN&|ZZkFm+I695q3HBFxjRg9^k)Vukj^wzmN;_@={7Hr_JC3-fCViU&Q%87r_!s%YSo@NCX_ z%*QY`QKcLH9KQ~+&OZP>_>x<(z#L*Oa`ONAUQ@9EH8w@_zdsOLzMV+DXloxT;{_XE z5Rfmr3&8Eks9B;X`UIvS?|TydU^px#hOw@!9*{h;a_SvF->lY5t%)?arB@N?g7zjy zV6O*@pI{#guTAsl+kNF=^CY)M)-Etqa}56>ic0NV(jc(mVM_GAme34k+Kngd^Uh-v zN^W+kDfVI>Ra`-3*<3AST0xl&y#kD7S|J-e!ZPgk=~#J%|2CLvjC{E{;8uaXL^47j zsA)V$LGK)oOb-y%vcW&*)s^pN78QCYr&O;UALWozMl7bQp7jsBtT$izgFoNdXNJoV zytmMnc~p3i^lau5yMWkou$H}}j_PFWw2ZzMYqg9xv*n_+-*>7;)&zexjR~F*+q$v zy|%*Cyk8^BUIYpMNGJ{TPuCPQAVzJQLnt2?94Iwa806s)M6b;fhB{RnD%qqr@0Lf6(+yJ)2I-j=q z{!4aFokCB!gNi$MjPs>E(8j9Xw3?v>HJg5JexVadR9$Ngw^R@dO!{pnEaglGj|iUY z5SS)*6+a=DW=YMKTruaMx2^$VGXeJJpD$STAdJCH{A1_RBmW z48<75&yZARr2R%HdBwR78gQn5fq~^b;bi@#$|(mh&cjMZk6*k${zbZi^@}k(P>x|Q zV-P~Z9l9g2P(j>yt^U@o3rB|l>?ZZ-B>-uVvL7Ek&F`Bh7ytQOncoL}d`9`6>&=&^3G$5Qn zwO=tVqeD*H6R$HO6Hk%VLEwWfIqn^;ne%f?idQ}xT@I0kWU*f@h$nRGMu_97oG@B0 zbra|=Z!#jlzPo~s4jPq#%-;4bEg?0vnoyYgKv`3){(Cw(xy@QvLOi4XkP=>*lUOSJ z+~LhZ`zP+OY}oNO_|Tm-?XynF9g-c7w<5+|8tm~jNd~vTe|^Q(BX2b4cGaIHFK9pd zH(OE8Hs)Wn=om&F*kUp!=Cit{HY)EhD3a{#t8;1Na(OjUbZNsrDEomAdw7<8-$I0i zGcPZPHqupCkIo%0r!W62D1ivUV?xCI0Op0$(5Z76S@c-|!Vk?Kf)#MkR=%4?Ul&)d zSjFM<95JQ8!2OkWey#SFdUdyDZPNP3gA-nYwEHA~pKGxBm6wvaP%Q;+NCo)??o1{u z9R|HPrZI6gDfPOxkGr$9yKE$2X6P>P*~d4$UHm_ZhTu|KKob4O(Mw+WO)n`G)<1Di zWud2WgP!I;qRH?R;~qJxr!7ZEN0M&aoK8?FFu)pq)7|>8tGm&2{^V+usA$qOhyTk9x_t6j~ z=XW3}&Li|Pn2&sd3~#VvMJweT0m}jJhaX73w~qV#j9zUqM74>)mIR5_g)ufFw}9y~ zxts>@4PE}TU&0_GH}WqGRAgdU{ryBAT%SeY5wG!iX5kw>{dY|U7h$^g-aj=*^`3gE(9ou-J z{>(<6wUEFVqg};b+Q+bgn4xc|-MhmSa;A11KNb9NC$e^~!y_T!eY&nd!H9`4O$UWm zK;!64;NbLyMDZ}4g>!K2kJrH{8C;ey!LHDg4l1RwLY@CQevoj4O{_8{_Fer3IFS+0=p>2)QWncLIrQ0ZzA46pvHk`T5Tu?A(WyWm4F`?w zzAF0TYp=ER2^G}Iu_8JY*o;5HD`5LY`G^AYe&h60|6Y(_V}2;O8$<$g1t{CS0*$y$ zq~f(FE-VAXAy#yRz5TgiVUafn-V0q3;d_V zPxr`_KRIuN4wRwqly~7XbHwJ}Rs=OeGct8Ci-(ua@bhJ!;jr&_XlyFRExxD`_Ch$5 zlXz>p6mOiqiowjTmcW_kblQZ^=6R8hBVYfTqe~>3 z_K{Gs@`p}s5KRE^DAS*+V!ZHa4MUtP`x9HiB$b#|`UmH5{0YGB47N~y?NJ>zvKm87 zxUn5l@E^l}Hi{p!$A#S#D5TX6q=BvRus%a$O#}=vGPN>CH;4bU#n1rfVVdXA@bYS+ zp$2CppBikXEQH)XhjK1y(o+Wy&V63H~S3 z$j01i<@JpQeRCpI@liSjIaA;}_^vS~fd;u2CS|&!zG@)ZCWIbj8i19V)r?GJUG3H9ZR-Y%gg=s3N^PN#x1njU2bugeo$8J z--8>r_H3uCW6>D_pH8|Cs_2_7YdY@@JRG#G`UO z$1uvd4&#J`;S0hSqub7?+2xA9sC#p`u+3qGjDbTC+aLnVJ_sJb_~Q0Y4Hh)ya(@vc zR#pVh&&|WfJPF5e`1ApsK@Bb!vMbc|{)y7s5^j~|(mdw$yq0XaHHC*7ee7HlAw8y=0xQQiwT{Pkmf8-jkC73w^1 ztM;R;x0;qg^S`cJ_B^KVvy%uV{ax@v$8dRAeCU&ng)Uvcnpa9Nx@lq&svpuVH}k5G z02hP6*>Cq9zKNqWwt?*j-RE$Z*qzVNb=0Ht0y$j{SKt`q{!<6oT1`6;kIUcz&w0v9 zE=G&FeihlDjX3YspLq$wXm6K zm>11SPP4Ju-I*52w(b7Prcxi;blF&%v-?l&cXb>E;jR=+=7zZ$;JM*%@cu^vQ@UZj&pk>W_&)P%?tXLUJWnx7b85aV0IePQPBdL3hOVTQcSSg@|8+Fv9f; zL=iJ}#?gkethkGXvE4}`lNL2^zqC4Qn0{;J~g* zJ6r7Hdb~yADzXKBTgVP%nvjzR_G@3pW3-m9G7B1!$A>;0AYlv=>;?pWA?PY_S|Z)OzP zE{LWgIg6kt-wo#V4nP3v$X0-tTA>*F2Dqkg$0^yn1*gLd-~a}ORBnsE2gVZ>ay0*3 z5~fYTcIEK`VY=)>y6)J9Bz55XYpvFNMHg3hmK^g-=-yI;CiKOgpzMkCf^{E8ZI-)$ z2tVpLofLN+0UkUisfUiatw`=dYZJL~<<>pai%oa=gAtHv%k zW_o^<=*M^7+Am0_swd>GCRma*y@jRF9|26-lgSY;S&X`?~Jx z+{cg%ct~&YnIo9~B^G6QDY)4E7#SH!?b$r!=HcmrJ5LPW62r{PvypA7ftiiASrmWd z^}>`1=@U>y4svmwKr;SqeB7w~kDc#ghz2(;5Uu?j6EN6-{MUU?&fuyo5P;+ zZaeAb;0Fz?E~LrcR>bs%CCo2)7ZzchbHKuT^t4o~#q@ky_vUrYCaXDAlZnsV_oaq0 zw)~kTWC$O?*r5Jqda)9uhfF3J8nZsFm?;Bn-zNYB@|H4xlueklBZ0AA-SQ)S0*ys} zTWy?b;c(X^huBDXcmYp6WrKfksLJ#dZciYbdj3fN0Nn>he>`!PQzq2c!Wh4k`3JP# zs1F2nNb1l-fiYP)wu}0sAp;Wg(YGAYU$XdEaUKyPxgu$7;X2%%dyAL6ksh?yM;EXA z&{7xMWr5^1RW6>rzLbqn7e7#kI|#r-J5SLNKXjZ{$f4^;v6snBY_9}iB3Lo=_t2yF z*BA+jI0CmT=g*nDM_rJs3WE)Q!RL}(xX)V+-MPw=cD8r=VXInC8 zJa?2OP|MxzUwxVdx0$8c12B8=riXY}!blXpC2i6Gz%yPli48*nk_+H^Jno?2PKZ9q+IQ!88PlKtrmWVSW4ckrw;i z$;DkZl4?EE4r_D9w0Hf>+hoVyIqj-`VN=%E;4L-elUB=QLihTIiMp(TfA;F1>);JV zctn}U?@-kfCRacFESU9`;NTEOJHW=9T)Q>_3uVB#z#ezTT^wY1VO<&xneI`HgpCq6 zo8}aZzsWLczZ(KPZVf_nIU!+fOOt19sRlr__!1F!S!bIzMDm}Ay~Jfx z(QvdXj}yi*qzAP(Qo&c9Hkh=^o`;A^^Q%)_1qP90)9-T4@4sO0dolch|6e;|q=rb* z$jH8JFQ6bCB|s4WWS~fl87htJXt}fR+a~3;M5NS|9oZq8Z1>2Uz;vmHX}!&4dJ}`m zY(bJJtE1tMCQd*nm!3ELxAgSi2LrGEk?Nq?__yzks@c)LYx~gCd{xeM9k=J_{H^7Z zE3v7Qr%^1Tp%=4LWZHf=C5L@YYD^5yZ@pbNrm-xSWOrPhx_Q{R`6c>pva=10bd$o` zz)w5td%lH4L|mPfUoF9sWpZ=KZ3AvDzaWnG8f-4jfh`LlHIdzLl^39paoH@ZXFvQ4 zpKfFo6%(`9j)tqsMD%-o-E*W4eb@Q(b)%vaZxIlgIyV+jDPTZVX})f1&qtg=%r<7)LZO-j6nX8-vE%2FQ=bR>DZJpHfF zQ_`XVJ4l{V8YeRR%Z{RHFmMKF!tOSK+a|`U)nmu6Na0eRn!#w3RQd?Msv#BWE*$%p zZ``Ry8V4NF&k@3=jhnkr^K!1$2a#eMr>9DRT2 zQ`-x8P9gJQ%bg&+;P3gT;)%PMs3g_lFt!6Rr(R2Lw5=@6H$auzffD2+{-r1%j&t~> z4_*x<4_38b_L*k~UIgJoF8B_dIY}A6aZjBdZwb!&(Z?04Ies5sQMa9r#i?o3z@!BU z6(`yF2UkE?K$)AjbB}`ji@b0gG#}JL%VpP>jSzK_rVo0x+#*Hdb+E*M0)y>ufk9dE z=D5we4jM|h>utn(HZFnE>M)!!DtWDF4&uUcoBHQ|$fja*-#l#s$2yQG@B*P?ejOL+ zb9D&)h`Dhlx9rMakrF*=&kLns4uVAUQ5OtQAUL$p$nm>1FFBx9M9B-N7cKJz| z9rUMKt>U+8!j~t92Jc90Lsa&M2SsbAn{^r)sx46>X!vH6onZxqOmKV1v*Sv(CUXr;2)WFVXHzOno> zwB6)-K}_?yP!!sZZ1_J^v)_hf*FvBBNUc75Te-7a^s0WhwOknc{SXDmG5LE?wGFB8 zQ&@sn6qMX(6t3UI6@@cMR}g+G3_hPNe~E_NF`L>uAPW;ja{FwPSWBRsxx9eshcgUP z@otA2b3gViQt20fl)<+i)jC!xM4LrwQtYfo^!ZlZ`)6O6&;Q5aSS zps>ZvwfuL4?rp#1SIvO0{7$!0?t;|CZ4I6{-qNQmH{U3&y^$_D5#<;s3GyxHHV`LHJ z-U*MTD@Uv-q@QGSBt6jHz&Q*BCls7RknRJ%C7x0k+{Oi`;m_MCzzZl=2dhV=_xm>G z3sMPA5kd>s_sQl0vh_XD;55~8d^1FhdHM?Owl?sgh@4%BigZLO4WdpMwc8&jkI&Do z6snhMFliZS!siceV4&SYYZ(Y8{6MyTKbga3sm-e_64Hj?K?KEOQtbQX?38ePtb^v9 z6KWNU2Z^NzuCL@QoNgLgAg3-6(LXGFs*1T=!v;80orFFHk5@&!4w; z4ZEt<3(}#JY@aB5Oi!;)ekSP)*QB_HPmDD&a9V2-vi|^r3^3*FMQTP)ZHXw<8chE2e zxA~UmC?8&NTh8l^pyz|yIdM|q^I?@&Eo9n0H`jOWOFX<)XQbqKNj`avNhLe{y==`K zd8kP6`p>qU0fW$siwUvfppls8bM^RBZKf~EW!`4VO8H?()SmI3;C`-|IG(!4IoO-= zSg(4@yR1+eCZXZadDFcQ@~KxMjW3oqjV>_wVWW35v@*fd&^`x!6@gHTkMz6r)~!9{WF=%-Vye(b~@Wxe>|anoGQ;*rrU zMq_3TTb^3~An58;{U9$uia&|*xAx`;?B=7;(dG=dd7J*efFafz?C064$#XP6(p>j2 zV9yID`sebfuUbU8yC9i5}p+t*rIZ;qwD_W9~V{(a5S|F>3HEwDc7y?LGyZw=}1DKYfmS$o2THt>BJ?eC8F{8{~urBjSAFWx{&1SvZT7l?|TX z-d|X7e}*&m;Ko17!_75CF^~#6HBrDI5jN(fFKIG{zr-J)dj~H+2Q6jS=;J^a30K%Y zI8-Apsdp}LgvhMfZLbCCXgalOTsYR!gRl{T)xhAy@q6NEy=n$dD0_x`tMZn69<%J0 zbWUP023^NJtg0-B@c6Yx|3Hi{9J{&RECw~Jzu39ZC^O0A!A|6GmAJv&jUh-r7-ytV~iKygU zkm%*kTYNtcy%3ddvX>6>yv<;sK*|qX6Ctrj+H3vyA8)z%;L0oGxNO0@3{EqjvC$u{ z23-Ym``-9EP>%96h3}&Xw#~D*li3}AI=tI=-cSkQsHycgryX75PhzNL2@uBl**trx zCAx~{7_i6tPFCf5LGHHW${MF@xhxSMnhYFj&&5&r(LL8ZNiGaImb75KzyJCdY|0q# zL_34mk?l~`hy=h2EalDd2;M}m;yX;rzZN7Led@Y5Is&W5O3T4JVr~OI58S6wP~_vS zYxNKzu%a+gCc{sdtLbjgHnZzM4jOc$#i6;?*FlN2-FVDFIrV*}@G}nX>lz#r)rW49 zzK{AR1(Pji!RH&%MZSUUI_8)i&9K{Gpf}?QZSJ15&=Xjvgv{q8YYK$+S&wy{Ldbo& zY^^E8nRvX?tZdV>g>_eD%AkhIsM}BTv@}{-#Aao%7 z=pyjBIyokL8i9g~^eZMU$od{UEdjrt7KdZ~4Ae;&g>X00{c|yWS!vK=&9me>8Q!Jw zVRGmejOz!|9;@-i%T;?fWW$VjV?^A3J3jRy(;mK6thw2hpbL6pe3jyVb;h!A z84ntP;Fkl5{1J2Bw6RqWE|^s1lmjte_S-_|FT1xEK^qq&3L9>l&Vr!SfBSX3&SgPJ zM6#OU`R!YgYO7TjlBgWvN>6MxK4?Op;9WfM^EL$CTUrGrW)c zf4>m5OeoGb7E0`1tQfJAKR|B|Hhb_IBTz}&6z7;DiX-l;jRhmgJr=A!C8~tJchR+a zj;f9|zxNt?Nyo+fy8|AXjpA$JaC}%{TYd=Bczf(G*7Dq-92~HG*dZ;>f&!1;NUkqI zF>60b**Zj8d9deLWWo4eKs)C_?VC*-)#+x%=>M4ugpm9xRQ`i+l8s>w{q zY%H?Wgb4aMlQa$IjGh!`KzDdI2zE_xL9EJt$azVNgmj-xhyML!0;exvxC^LW57`d<{7EA$ENnM3 zdnxG-<_cH`1e>dFI`vm66=>b7U`qF7j5mSnT3l+T+!~a=6R&CNC-PMTuJi%rMUe)@ zN`Ad6pwY5OB8{ma_ccFG#y2xeAtrfdK^&vSnKAGUy|Pw94je{Ogm{O-WGmHxXftghfn3c?T= zQ>OiHd2R=tZIC@G?o0wK(9-_};1xI%!>x)n)gtz3{`4t8tYNDq`s-fY(4uPz`6_s$ zElZ^r3^?15U_ApXEiDdea3ejdpAz8yja`5hPiMwW2BT|`eS+4tG#S_JdL9gp;YtJE z{dosIcyq?{ZAiSp!#uCbb`G{_h|MMLN9UP<;W`0~@AejbYYn>T%%z`sGk@+i;m}wl zlXMC6D$pv@DM1$Wyp!t@Tsxd-s*2U3&ra}Fb0*3g75vj0sg>V7m&fkFf!mnfM=&NT!cR0}GgA8= zr&p67fC3)NWpEuk88N)xvV{H#JzQBO!i@}H`eDxD;KEY#oc6XUE?H!Ij*qO zbHcA(xZ@Mr-c-vG!+B9YxE2>}B$PeD&)aM{k0>J#z6s)WTa9nYuoTwMA&Y2m@H8*- zD7hSu8X5}THknWpO-E)wmjL}vN0NMyJumVH!SNs?Q)Ch?1SDQN7_$_ImgMi_KI6H= z#o@)+@DLmrji7I5+V~hDAb{j)5VL5#SWUH}lP3V~HPRV3?`Q6qkC!|>I6TD1L5Dih zV8Cv`PTFDerlUg~8NbdfT)HEH81t%ARi@Vs{C#9IUSkg>GqiJFtGs#MI(~>4 zoT_@tuKvYx8QUL+aIQ?{%dA>kFxlm~Vc&9@lkNI{mN6_ChfJA&S$3wA=LXw8>@{7; z577Zq0UHGVq%OcyG z70>NngUe9!)uATLdDA6|0|za;EuTItgUKl@b(82pINok;P(ebG9rJC)x#otEKXc$+ z9CX0p6)TBX7iHh>0rmDGCeJU8I7Ff9_!tWZWGRd z8<^LfCd-=fx}6gZ#Ba^35}~4_fqo;mHOGB^fB*61T^emxv;U+qFrB3TPVTPXXczj~ zp_GB^(OoOociB_c1fY2FD{m`PuGP|>c-gowr|=$}MO(dT*|;U(o!>a*cuKZ` zYszy67Q#v&d%>9~Q{8Q+#OBFKeFE>88c|4Gfa=U0lt=r1p!MnQMOhxjPr^Nf9MSiD zPsI(jh+y_Jc}WCsK#1v$AXM1Bf!6>wZ(wyAkRpT<>-9;1`Zv&vVW$j-b_ItRcS7g$ z`naJLgcUNlqRIcd+wRMjb6w;?l>~p%y9e@7w+{ifxT+v!PEP*2U>rz?5jbq__)zj^ zKIr8rfjn@r;$;*^Qm;u}3i-|2CV!<6!h=w6DEH$wnXtKYo8{)}|E=1_TMlb-XjDyx z3ZA{^crQJnw}5j2d<+RO`mE2^Jhf95ED>Po z7__Ka1G~OiWy;fCKi;p<34+oI?C_w$s@1w5H&o^PuKJ@7e-=nIzT4G;*Zm#!w(^>2zYV^;=VGbqBX>6F`nO3_Rs0h3`I-1oKLBMY#=#)66&%=nI30vsK2 zt6f*!bndXrw&QivY1FoLgB2{Sr~AON;*hXTA>v314wG8?!5HDBfhxQAu{JG`1eY@& z|6*wT!||aj$UlG%0D?^q1CEP(4KP!Zv%2Zk1U(Q?ae`HCNAyV&BU56!$p2ugmpuY1 zFm%i|WkX6lQ>z3ODO!rA+#inm*A&TcIC=1F`b`~q_1{(9z-ez6!wf?GVwHDrqnUCW z6CNhLmHCUE?BlOu&aw5@hOSk#y@QZioc$T43vTj2Udy zU_B+xRI)5l5HV#37CBg(*y!xLmWtoMG7bn=8 zIYEnN;IhI8$Hu8?>YIgrJJ#YGw+EAd@16@PG9d=L-+$7CpkN>?Ng*(+Qh+2J5legJ zqzv87+CoDC6!!b%im|CjoUfo#w8imxW1WU4aBaPML^dww*7+@zYf3^cwi$ar;AM$8 z3d@jG>ItgC$pl%bNL22QPLviks7AVK){Z${ccRzX+N6Ci%)+!#M?WdHTVvpmdBuJ) zUl%_+znQY(R9k~$LUBfeKD^B|>=&J#fl4~}ZvNGv)xeellDUh>Y9SAUvU+L&vA+WG zvlGlErP}3}p|kjqtJl`@oG^8athiUh`?n;ugc=dDH%0JWFCe71#p`}vH9fM&G4NOn zk)p##!gE|8uM4>+?*cJ36+sSgyb=W*utcs{fqKW5`|_a|MAt&;v?`KPBLZcLT9((w z15nDrsEVH8=H6+u)(?FkuVr_ecI+HO|AcS{0Eh_GFeC;9l0r|GsjqqwN&$S*6hs+4 z-guvXQtx-pH>dpP)>j}0wF8#??sn6aFR$>GkObxvpN+B!MgEvg&P#H zQu%80FhMt~o|=TOq1)ywe0Ah_^#vFw$RG4}SZEKm9MAWIp>G)6a|j3pxVRjLjnurhP7rL*`0623KGhHAKm)A;tT`4N z_t)OKt6b&fD%bQnGR`~|xLva_^u+z+3Ox_ z4b(>$Re>`RzQ(m1kyBCq&t#J4t_zkwLkH3sP{Y7h2N>%I z=q-BR#GqpM2jsLBv5}c(0o@B|0Cv1;WsfTK*iH-$$L`#6?j~T7MHdwDgQF!)?w)^K z0N$Wwq@{)gJ{Iig*n(OOifQ|?*yMlwAjxS-7GlS+tiIW#>|;Wr{O;z77bn@6+oy(K zQs8Y+{b0SXQKH#LEMWK$q;+ICI%ONpLx@>tAaM(p{~#cq_AIEr5#?fWcXdrbpNSi9 zk5JKaM#Sn8^jm10+o5WSdoN5=qPaMy^q(*G8uv|t^hCe&0FFEz=>=xk}zfThB z1PQ>1Q}^uryw(roZBtXajQ8(@l-}IkJxRS7a{Rzi{d3pj1wRFVo4`&7{ITtA2P8NQ zE?OP%a}UA~F$5Up94pVAU;q;$f;UJ*;00)W8(4Qe&u118>mURQj(+uM18BffN$JNW z(_<(bT*(Ve{M6UxOii_&F&*eRl?+b<@K_$b8ZCHzglW#j1k zqLgL9p8HdNL{&j;U`yp?!coTg>z)eL-rq@&yvFZZuJN(smJA-!IE2`XH~(G z6P_%Ai7*S#Bw>1ar!2`bHXBw`1K~fxY7^ICL?r#pRN%9iafR~x>);m9QL8Kb-HCF=JGQ5KevqVay6!eq6@oJjZpxQmJ$7MjG8{R z@40Cy-Ss#ftk|BLfxNbd2DwtOepsmz{}9<_uA6kTXB4g+^oRpMQ}7)0KDj!yKX(`j zRJeq4hHP^+3&-O@fFiZ@0R21=_JSF=(^v68%q#!S`NN$p*2s#JvnNAeV1xx%F$(}O zEW6U-*|Xz8&jW;#Zd1lowVwO(1aKn-rCftyW|Bli9w!%J=d$~f`*dmqy^gTJ`-*AHn*K3jI&-#NY(f12W zt}06P+cjkrfH$o;C(h@@x_vdruFFfZFT>~k~j70&);+sIjQW_quR zwKjCq+va|bhgOUh}9b#AB^=g4` zP@3iGC#X4yg4hMLdKZ#U&;%5FpaTCFdjXHS3Wx*8ZHUtczf9NKPC;|~fCvjd?}mK5 z2c$crVunZntom8~j3->-K(EVyM6@DtsbTkGOkj;axwSQ{AB+bIJTNXm;YH*CcC!s} zPz>;Ihax7=lXeV|cDNYU6{MhN`uRip`(Kz1XMm9*8ba2q?54Q2%8dp=>@W;+oB`;| z;I6lnVivOq-yuh$ZeG*i!4uHYzeiLtjRCszK1LzW~KsRC-soXxt)1X;=S9# znAKBVNRJK1aI@{^{U-?TE|aDi02hlwqD{{v zNLu8q_DpBYb>!k%zfT2D@)B=bYdi9EYx1)btbeF0gRiW~%TZD*Gjnw61bfI3JPD=< zx-W&DApJG9V`hCPy-%)UAuXV9Ef{+0B;01NpjlEIQwbj_oj*kc3)B(2;)&c5>j?XV zXNNi(8;t`yR=jC=gIbCpq6lW~H{I-tf)N#6*4KedeMxxz+x$Zi`#?fCiqFN%$EUHF z!j!~J7iS~7W7`G-YLaDW((}%@njVQcGx@-h**zOK#xAS>z zyoEH(m%D!1Db=hFH5!y5Z-<21+pX!Zri}4;(J~s~N0q7`C#bG`=fx_mW^7xsa2W2Y z1^OUtGqtg0z(fjC*fy#YkC^KL04DD1YVFpDlpgqF<5=FhPjbW!McB16Ey*8xq#B6B z|Ak(B_fOKzErGhh$UIOnssq1E?c-Q*9uvH}4*+O2K=LphTSahIB)Cev1EZB2*7QgQB<*J&GWE>7y!v)a_2Al-@29rBoG9* zLnVP;g_YdT;0qxb90&l$q7KL0v5s ziT*3rX|c@Y-GQKKSh@NOX_W(Bri*h}-+LHKoyL zKmx1lovsLCmM&;sX+JjaeT`Yu;^=~LdpE@A(cKpO!--!(ijwTDD7USg6{(?!7b$}5@2Nqi{uKX-3`YaNt zQ^96iqQNZVX<_v0;0oQnn0yPL45Cv%3M)U5rG00E5DZZ<=7w3WxSgT_n zfPJ{>=wuBEYo!PE!;xU<$ZK18{gk$@sw2;P6uTzF;K?rNVp8pr` z1*nE}34=Gq5Imh=2K9g2Ji}0}QU+wsv{8%xx*-BwwR^vbwNa{V#M&PFB>bjW}&}sTZi`T zLD2Mq!nPBw@G-Rn6ZSFtcgYa1d57NwL*6hWw$_D$ zYbZq0_`4nk-baxOL?ope*RP@O0^mVx7O?tycjZZzG)Tv#ste|i4PR;)T%3^l;RDR; z1CUVZ$Z#S$Zwo3J`29PCQWSyhr$_4N=iYP84hf$wFNrkBW&YT(gvHft7yUvYlWww2 z2Ep<`e;)R-Wz&!j4>Jt*^qPK2=K{Z=#3DszLaG4*j&kw`pK2R2K;oJExAX3BDY_*V z*Sy}0Kasp>Th=e#2NxTH|YGU{d z5OPPr2vX_YsG#p_Na5~=RUiW_189*yU~Lz@7&U7E2v|4N1-6VQMMtNILo5QAgYw9y zp6|3>WsAHXDwV9%AwrVPYB$lt2%qqaH>td}(49S5o9RB}n?DFVS0$3R^H}xV6>Yn0 zIXW(PY?ep`0ySpNhDM9OV;*3 z3Q(|6!*KmhIuZHLL<%t0o*Wz>W4A4o+t0+|Q7M8@t;Z7-*KuE6BeEMM5G^?ur4im? z(dW#m@dOtM05E;@DORXrt@|Se1R~osjrP&>FSXQX@vN{-ml$L;rWcY)c#f~=8sj-1 z0{DxP@XlfT)ITpck(WH#@YF8dAdDzuSO)0gm7Yxlp3<1R7kR8}(1tOOrvYPlCND~H zr3A)Hg06ST(sAl7o^WI+%GA)ygT>2!Ipd`gX3w;vcj3L=x(NVd9FRz)3#MCn z>E%h_8x!nm)|q9I4EGmSuJaZ}?iPnJ4N!K1BDzjGc+S?ZSC$2NEt6|G&y4t;h0qlDgKutmg@V!YclV*Flg(3!!n2+fJ*5{A}yDHtq*7j&rn~@cN0D z`?iOqHkDvdP$uNPbk|Yu=+!-_B~R^UK3OE+L;)k&5}TuEBwYMV%m2mTrz$-e+mI02 zZKikde9w?I`A%fv#BZS_La&7Z5Hw!+nAd}$%?J<^yEsp7yWi@fe7>K!FgrrTH`{2o zW+;un^vh9G`r~_}d}YF_hTW{flbr>=F1U+vP&l+DQR0L_)V2m}=5`S}pJr;3qb8B` zNHBj;2MvT91#2IAfbwVwpcaJCA~kQ>I}*Zwb9}mutU?~Y@!inKCO{7opY`4!Je$_N z26YdH{!3`oC;kvZ2sYG6<9{349P0ybo)mTW_J#qMZwYQ~Ufn(8Hnm+4O2G6%tMusc zK8Rl>pc&ak7*)nB<@fZ;l1`RL+n8|BgI+WmEP-N|6!|VMI&ax;W%)WJoOH6iZ2sL! z<#`~#+oQijR8Vy`uvhQb{B-s-IkaX%aV6s9w4CIqlqsTNe;%Wo+R^9*p$a{LY1_Xe zLdFF74CpMDfCxrnS*?pKddj=jTQ~l|fX_a_2Y5bEC4%uFKq(7FfVIMHJ|zQq97`~o zC4!d@ZRT%sdKb9LYyCl}JHj|n9{>c1RX)Mu_3BkOjPim%Fi|cUwW@CSx{^u|Z<-3? zal!EogDO)2f(c)G>7@tY>vu1wBTrh|%E_ebJIOy; z!>_YQ5B9s`$kRZ2%QMk1=Hqop z*E4T3Chk*uf-t6()@7<6fEI|0GUnyM0=G#1V{}$VeWnQ0V&+29!2~>sfiLlS41IKo zef9R)DKRDD_;JB-y8=0rTJewWFJ&5OoaYiln31Vz(a#-#VXh3O&fVD`^2*J`5-mm= zF1OL(;O5ni^U#CKp7CEhN z`Pt2-@kgX5_%aRgnl}dvH&iPoMLwt#0-B-*C17ichnFplZCvfyr8itvK`+1k*M~Ov zn*tuvBZ2C>wz<$MK_TGF$kEgEojwU)@q;IeX?iQ00pB?mhGk}FR_-Q z*1(q<|2yO1;%v#l`j=+bu-<$KZ~kL4yY#fJ6l`6!uGMgIc&*b4c$m3Hn}Ad^7nmTS z6uhe(_A8GIW_Qy+dY9bN`-p4YD&UaD6a%sd?$7^CpD5$|-hq(`A4UoJWE2!*na^C^ zAHxYaa+Gs!C3>Ff0i8!<@I9DBt*ir4f9t1FaVqg(dQ0BnZ;aqKUnP1U(F%K}seYXa zKKQG)WxrBrE$kfirjGS&=zWj$=FlVTmFsT{-(;OMsP5e*ndVW?JGH7;CCIdpNF2CF zw>xubNQd$hZa?o}vc1&2&w+Q}hk0jtE8arf`v9?nC9&Mc46C{X`0S{z)&4=@)dVuDP#ac)W=CVm=OE8N%TTla`uIl zh8;z6*C1|sbCjO!Cp ztWSw*2M~5Ai%M z%3!?I%OEq^(#}o^0T=@g8>?-B10kRy#SOD`oEe5ULv_QM-mNsg2i3OSV@gHroWsJ$ zqO6yD-xj(whDA7n@s{fSJ}C=TJGspGUb=1?E}h+VB#6kvg!WqHYHec4s_j#5nYkle zwkIymUeZzAj;wFlvd=3mRYcSm#4zq5isZPQ{dd^DFlYjATvM`E{ftW**1oz+j_8QE$%`RC3E)_ags@60?fy`!Sm=UjlEd-sho329#JH(t&dbGS zROWGkvO=l+ad6K6SbYF6^etx%W^FKsOdlZ`K0c^mpA+}Ph4iBQ5jjR}lfgY&C7FA1 z9$3H=t)Cg?65W>~B)>CABKZ$Bt&9XxpPUPZ99U#I+3&pMiF;;E14ER8iJeSAEx_q~ z=aStg>VH~wO4pBm9>!1&vG7$L^L4x->LGe9PueF!dRA12?HRv<*pV@zGQX_D8*K@#KBgFm9}Bc#?HQj+ zVVi_&>!Qo74NMgOfHz3~1*JUaBq@Cyuu@)u%isUZ$dGqTMKK;is7_i8^*~GKkBa8& z$cF-6U0T02ME#$7Y;ELZUPk66R& zTy&nSb<_Vi*83|GgjyQq6Uas%1oS}1^MuqOpj)_|eARCK4AS^Mkm|tg#e0Rt7*z7W z_OisJWlHY99wZJUV*457!6b-LkX;*>hLJkeInCZ^w)pt;T7^d?1@RY?UD|(#!eO7Q zQ;wP`EdF3AZI(;U|CN99)kZ)6#y)CW=pHZF>DB9}ytt^3$o7`0Lu9aUNR-Ym$QX`f zrB-QcqaXRN{N@M3k6NMccV2c_Z%*EB2#6Nt-j}iP@=62b*F!>L{svWI^N2w@>T)L>AWDoB)@xi8PT31u31P}SXgw)(Y_PZ;Xkvr zdXM@P6O!kV2|A+nfUm*cda}gpvxcl5;j1!$W)n(W`pf2JN+AFM!9)j1W?OzrU3&ZC zx2&kCtLs~swP(OyVMz4KyV;1NuyIXRa0xOdtf0e1awGltD^Vl5L3$h)fDxAY+Cp&h!Xp_3DSV3Wc|`gJI^ zZ%WW@krcrlAGyu_zYEzW!@@tR>>4ykQ5ybbVN^`n0aNG5cXaz={YTTJ?VR4xtXbL7 zsoSv+?lazW&ta||@c+D6Yr$P|fBg8;cs^t_*9MbKerf)kdR6j1Md-xlGJ-aNJ1t%M zV7x*BKR64G|J^*hA4k7#VH+Umh61m{0)|exdS!cvh_yI0f1la;hpv?qWL+(~Hz9#{ z?WBMSD4_bj`5GcRe^YynZZ+ySQB#@`oR0cc?Sn_10b`=km~*%Rq~Go%Uboa@&Cz9#fX_13c$r}ZvJyL9&~-TpQUY)s_^IG1nwhrP zlLmEVKl-a$>ik_z5=du)zzC*qm-pO&k1AG94ckLj7UZaq`mfz!B>)UlZVO7(Br+0Tuf5KnK>iTu~yL)V{?n0NDk~ z8WQ6Lk?A%n;|Q*L(exll8CFEpd?Q`dmu zTna#Cbp`^JN>g(&>+dlxfj8IBJ zzDv)7V3MpgXZXMlTEGpkRnooM!&`$lyb%YNfaWvdOtzm+kGby9WDUSy+3rk9RD$Yd zBG4X|%Dm~)>HD5=#sT+o(IYKRdS?&=U9rJ@pM73_-$&fO3v=GlPbKSCbt-dJBuhQ0I;jho30$Jwn=KEt$NI}=IQp2{*8(FwW{QaTGJ6rltxhPvXWpUbNq&-2$mD5R$qX0Abxq$Bg)M$^!c8j zU;I5>QB40^!y+n`28(6qJ156`TgO8F^+nit6cr2eNDalj@$Z9fqk*jpR8WYgnR1D( z<^SW(Yfo9kAWz}@cjoVrtL+w*_b;y7!Y5^NY~knBo<#m|(@y{ok8_7~@*ZRdCL%LK zkpDJ&5gw{k>0o$0aNj^d@ka)7{FzT+&KOZZ6){xWbzfxnCbaM;V?`p}Q1VliWMU8W zyMRusSUP?dI)5VIh%n+LgL&07LjB{NzKVfn;7E&b&h`W7-!D#DD@AfwODaqN-p(fj&-n4%-1E$5OSZ z3i5al>Ibc=8RGhim+Z?%+WGFACNn?9A*+~TC1N4=LMuU&{HXJJ%Ng#J5Vr$JgI|#h z;P%pryb(GmQ#MbN$p)}D?~YKCZuJnmRuJ3ch0ZB}pxuslaqFli^Vp6x9Gpymi3CYJ z(BI1VAc;)3+Ej;did3VV4+1JC%?_EM?UoYgM>^8OaXWd?m<*6aCM5z<8`{0;J87_P z0H-FKJS1Kqtv*ZAI`)}$9X<4lKB&~_K>}B(JApx;n@=3M?@WIjUD`B#gouZsqVr4y zz=Z+jY1e|dKiHXM@V^>V$OVAlu+pG=z2YHctD4iiyHPQ4FBqm1vX~(qtuzsM5HmUe zC@prC^dKqIAx|?JqHlqiU@UYfqLhz^AIP=w>9ZD{k5HW5n-Y67JVgEG`p54~NvOFP zfotJyshd}QdZZL{B%mBrOAz&A$}3&Q1Qx(~L4Z)!5H}~O98DiL;C1>}MefC?3c07> zf6VxmG5o@>X#`7t(^$3o!6gkOLq38%^QKMASJn{WF{ z+d<{O=b}3GT5i6Wva@Dbg;VDfX&}Zblh!SmEA!e~`}_!Eq3w#LSY02I^1zZ~`O1YTt)=!^A#e9+y$6YbiAKw&@u7>9^efExoI1lj5- zgYR5hSQoIfHG|u^IV6o`K;EHTeAlRo=pcYMMUa*ClgBGW?5zhgJm&X?9jTTxLn&bn z`c9Q~cfck6@oI19bk?I`m!1|}Je(6ei3;EBmWK7CR~336n(eRnZ0)H|&w;W2`LBs+ zXVAZQ4^;|tiupFt^z*CRRcCC~aaI$^+#1$8WcklBLv70ET(0hAcz!851r6OG z>^r&<^&r@X&o-V+U&R?RqnLnK0|`z?jtkIc`yeeS7813(3qr>|=<~5atqoeiuMp4< zF6nY)EMm=@&JVT!Ky4>55fK^jTf&6}D5W;FVXl#60r?9-N*f1QOrNeAT7Di{CbTZ0tEU;|6}Aa+1;fI!3aueqpmxZ&>ZxTUU>( zmmM|-=_K5-yEDhWT;Ic8YCC*$gN?XwtybFpsy8UJOv}nFS_J3_YF%=WKZrU)cR> zodHWFMva+f6_S%1eDB$CZR#BTmuZS<>ZX;^H7xo9xe4ruHs4tP6a~XQI z*G+RRmII~H0KGu{rXMGLYmq4z`eIm4gh^c^ed_rxKU(BqNJ-VwcB_-lABQpRWO+O6 z*Y{{pgj9M5uWml3tIsCY# zdpn;0YR1hHK2~9M1q8tu{vv+g6yxZmBZz!@py3E=ZWj70gd5o%2s-?#a%F|MCBD!w zdMp&2&YaRxmX&vMPEx!1n<{%ez2^(4x?Z(E+ZTM$dMnjuSy>)a?gikYM3H*;=f6o- z#;9Mm>1DPQOES9Ax?+q%icS;MytwlwQ5mCYRqBmjJ*W5bpqM4)Ns=ro`!&zGM>!@e zy_Iy&KKuJgX!)W}DzCX>;9~CdXYEnQVj-!`mcv=mg4+#*u|oCeJFO@8g|3Ddt!-^P zn?3g{e9qmGWy7R*4uw5Bm4=F@scH*FlfEYtk-s|XhkpUx2fN>`e%&b5`DM}nVm8P8 zeiWaW$6`L2Lux^Q`u}6;D}bVUzrUA8x+IqdN$KuxDUp(v6zT5nmX{c6`9AymzcbF@I4kU4?mf>rpE?N6Vq8JbklNCn=d)Sa4PhI#)hg`sJsX_6)P`qw zfb{qc52b={?NH6+h){bWtE!6mLM;mdy2x+CZ!mrzT*4J1GF+rRD{X%8^u3EZlI*JXDwfzPEM%;NTQ+&YM8|3lKs!Af`J7e=RC3garK?GS#-$So?TNVeoGXz8e7SL;+g#-yuk7 z@V#i4b;^o-K)Kj$`otO(8oLfhXzs%WhsqE0EP zyL25Fq3S%S{4P0<@eMsK9lJq0>ULQ#dQ`WN=N}q&lXh}DJMvs^;q}JjKa*kR23wXk z<~LQH=!$gu^}gaNadBMUQmJ!J%YBu$72g!3O*jM`=Mz(5iAVgA2zC7F{bNJD;m-)U zY(}Ace0*!?W0LUYP2m1IV;fxw8VgECS5t%l42PV8!tQr40ue^U?Lsb5R1A>iqz(=a zRx~wbM@NH3eS3EoSXpy!=W13-fNC6)+-tFC|62P3;pNMh$VL5B|NPra57}W_ZaIni z8L72yP{l)E-aRz|Wk37!l;(KdjTX~$F%93;1etYa5AyF89s0RN88}ENvgQMi>q=?4 zj?^fjv^B!Z7Jf8DFMf^4bNno5eJq??`5~CPc%V_UNR5>`u*2|cL|INQ#Wwz`)|~)$ z`Y{mvK>P9#wdDuj1swN31Jqb4Q-|pn*SBHQKX_w)8dFgV8=36$=rgzxVafN~dL)C# z?=N*+@0X^^IRB8s(l(oX#7-_UcF*cObA$J@+0o|u`+WM*n?f%6%~-@3kzHHP&jj{I z1kTW8HiUmc74HmNn91XqMl^82TmJOcV}q2qB`%&AZQhtlxkFP(Mm;GhbRB&@mujN^ zKlPHv;t9VasrvF%7XXX=%M@#oSz zzUpQAl`!I*xX>c>y;lMgx_;M2l<~bs#W;D{q>+qtbl{>1l#A{KavMS|tL9kn)S#;e zxXB}XpM|Uyr6J_DvG2gY+%u6w>%t*oSH=Qg14@bALpG*3%)^-b9j8) zku2T%<38EcCT8?8cK+^{$|1Q&M+&6QD(|F=xr2kt{F&_UTW3nEzt9X9pF#3G{xgWm z{Yk{44QkdFFywxzs!9e^km%PRKR#~{WnR}QvuIyD6Uj?FJ3H%{y}LdxkByC$sAZOq z?M@ooC7h)#p7=w1=WKi)ec&I_ctsLp9~2UOFvEHD!gGkR=cp~O@d1~hN)kEm{}gMU@RTxBLr%Opalx2(bv$MXuq`_GXuM=b$n#vLBrIa#J^chf(iqp2egTo&{`V&He%&>lP0a6o!{Bf2g; zd9<@vEm&gda^}7|q_rQI`+UANr!2}u^y2aY&8PXgtU`!*2wz-;Q$?eLggz>&Ji?xI zp*gdRL^4t9WISp0qBmCPP>4qS&j{`hdf;7xjDz*D96z#4g<9c8y0q+UJwiK1_s8T5y90`uqrm0oSeMhlHk9~`#?$XNItEP9$XIsl`;!2?>SQd*4qZh zGU+hXi9+cxTcA4bNb%@^ew$H=h={lV)d_#7MzT#d_9+%@PSMuV!gm19xM-z|ZG8O0 z#fBsmRnALdfKwrF)R;>E`JP=7clJA$?l9c_Cz+BRLL@g0ULOKVu~ z+a|P9T=uO3ER(^&#I)NT&2zt;(MttM)S#_@I#U)ICVKvS0+h-?-Brm}L~2 zbI3eA^y+JaV|7b%PD#WpwVaim*0mV^!lm`JZib9aO;JrPI@l=6KGb=1R#>5#oXt5r z7Sekp7pN|9--$OEiy3~-wWy_Q1R0SYL>KSbz!JeG2IomJmn0fT5;d|^QE~~hq~2eW zgx>Wl)a1NugbvB@jZh=dkK_$R*7EUVoV$JO+YzFoM8$m1HdLt7KVer@Oe09q)K+sE zgN_W7~uU$sP2=x9;r-0bWXs2*iaWKOVeT8oc;IF3Gp91y^|J@_30o$(19JAs5hX zg~6z$>|7-Na&QVI%4$hFP6oD1p|2|uG4{$RbPOFtsgxlOoiZPKbRdVCGpD;sR$#yBcp(28H;F66= zWe8$u+IBvi*g$xgBPEc2{FBh^VUB+tAWS5i4{6x-Ox}AJEY#0KQAqkI(pb+VeUy+a z_(0v?K)GeeMrE2Iia4lMP(1F8bgKt#PFtbIZ&^-6qw0x*bj4j1yd{YqrWQ7iu-^%sF ze0S$Q!^CLl+2+ZJAZ8Kh%>d@O0f^uGy3a=y+du{fc6wiO4(A9Sd!-jWxK(`r-UF)j zDOfzQ4)BUzKn-iaQNt{WfaZ#Nct|^3s`m%@Snb`O#=(|Kc#rB05V@`E0I z;wc1ifc-(j^JpoC`(59CU&iBgT3Xr~z%e_4pgIy<4w^Dj7g8WYhIZ<2=ykBUzFMA`^k18L4EtH6TmChvw`l0$ zpIFVsGd6{WpYAEtNCnBEF!OaUHkYMpS@T*U=e}AuXJF-711Cry=);`>@Vp7CBr^Vk z;ncrh!uv!J-z-5VDiEh{*y8D6n)7jkga8Y54``dq!JL7CN=OEjpy|L`*Q-=aq^%H7 zDy?T4;9;7RQk-<++g7T{3@))6!Apm#Io#~!O2NenS(}Vn14_q*sY%-T)dNg)vdK1{ zotWsWhJ*fnXVnG$jIy?uc@|f$xKP;Z%n6lF=kV`|a-OxzdnuNK*T1=BbZ;L$DP5qP zmP`j~AF;Nm)O6A6JhDG7Shz$QiX?pW%-FVdve(<-hiT=EDk;Q;(M@EPu7==<^EVtY z+<^x2P=uFkctBtg>WC3ZH$B6Kna@Ll zFchPsL59Zb^3L384VxO}4?8~K9JD9=wc+>9qj}))Jnnq=6s3O?6K_>qU5sNG7aWi8 zOmpPKgRM{)!^39Kd=uxbs*-0(XcReTkPULn~ zLX(E*vl+9@o$oSR+q3Yl=}B`u2XoSl?>KrN4W${#MZqfmTHj|Q@p0q1eVs}ZZKTzAkW%x$v@#iH`67TdELDgw?aLsOnf1itaTs0qYBx(5L zx-5T16L*F(HI-93I!)Ha^pY*iPAX$#W2a?iW)A+R1fM=j(oiso!E-_0cG^dwqpkfh zEhLMsccKSda7n6RHL&I87K{z$0KBijeFVJcYkMC>UXDSm(W) zu4s?)t?$w;?-LU}4bQVa61k3pM>5d+MPU*kLiwE6q`)9j9QF7>EPnDnUf^$<1W1S! zd-xZEj?yhAeiEC!J$5d~8AMQ`PN>l!Nuo?HNY2!JTZGQ7Hu-{%&iC#>X!%k`hPzN6 zHW63QV>`u>Vc}A&CmA>Kd6~U(;gc9-j8Oc+B+c*=`fzB9f9okOtI@$Ew(7Xht39^4 z%{wMW&&m(tfz~rMv@xEYdgr3zQmKLVBgizzRsRNTg8;o(Dr6jZQ#tERK@PtAggi;i z!E2UBKllVmF5>5d=)=OlqRBv z^Tl%@(TqGF)RUMQS1G_G;b08XRVDF?cLegbhS&%aQYQ8b)x#ougUZN;JtwKVo~``g z#UFA3H+XXGLA)+_%q?qPe0UZ~?_SvLe1Zq&qC??vWzn3)p0Qw@w(VqTEUJP6PyPnN z#e-j6M`y{i)P-#?2Syj+b;vY-*KRBX$F;GQ2uGDV(BlQ02PwPz$+dUX3cE|>)f#3v zc#1XOW@=f)4eZk-?ka<)9lBwxz!pKz{eBixeb!b?-LpLN)7zGwK=lUBJ#tF5s1-Sf zkTX|J3QL*;Vfr-8Ve`@)&-RE%-ZrY#x-Op7y;#WGO1`Q^TC`^wQ@N zeX-NvRq)kYtjQ>3mYU{Uw|w3*%+f$TZE!Z{15qM6L73Rs)Q_F3QfjS3k=NS4m2oTc z6kPr83HAaqHY>`ylNn&EvbpAVfDhZ(SPrFP`pzwhxZJ2zH!$#e)#vebNlo)Gn6)JQMEk`diUa681zTa zFOouL2lthnV21X=u6p%W+KG`>Vx9t7O&Ux^1WjjbZRLZ zN>u}++Bj2|GE^JE%23KJ3jg_OjIBqDu*yH)qy}Y@*w;$ErQzNJM z$g5U#w4LZQ`|Rxg=@UtrZ0`rtJKJ!q|76kJf~x7%g93zP6yAE(geH`~sta z{;g_g#>(cZET*ZuERFEFF%FCLjpV}EuSH>=bfHTkU~@MML{F_#R;6a>^|>1UM8u4Hpw zy>0P)xxKX&eccAzED8-p^aah<ba`3Z*%D0UQEcE_BZTRobda_j6*3l7lsOEn& z-3AC%kal$BaBSKp77~}JVvnkEG`OdNa!_|oR4|9&qSxG5w_qmJJx?AWsPV>Wz=aMK zeQnieM_ao>!AtDZG%#c)oBiNcR9uUGDDq2Za>-Iw-SdTi2+g1>@>}nG#qyQ-z6Fc{ zQZf8yHC@|j6|5sYv?e2*xNht^m23tr>Nwu<9Lhhdn%MH!Hu5y2sHee0o3LQiFFiXU zI7simsr0tL$e|JBOdUwh&wp{nYd}d8s)<;XDN{JJj*4IO)reJ+I*Auuv7|~=yn>Yz zAJ!l&-ajxHKD8}xr$v>v-0TY~(V1I~nMb3ho&jp4mAP9Zm^3s^jd9AfL(G^Fe@X^7 zhixtUmGI6>Ay4b2hinSf9t=nKSyX&UR&me%uCAj;RATI;&~eM!-#o@R%2n}&S$Jy> z6qdA1BAJ|Q98;QTDX#F(VL#Hr*tB?{=;6aX%_{wQ#jLudlm?eIz1B#Yi%7ca?fZ_; zHQ!rmL$Dbme5!<(`$s0|#9t+oewfPUDy%kirQVMhVBT^~Bidn{e_tb#T#yst%oq7l zM%uSv)4SH4v~Z$=!Sx;t#c>>ADDTq=ta?$Lef`I+KLa?^(#pEN; z9*tnmlhTsr@XoL5cojw3U0nZFP5bD#g_}&Jc8RHR`kZyG5ctiZf1679JB(HRU6y?` z*vf#59Wz2E@q##BwbE9UHkz1D9H}8gq ztyq!_@d_kL&*6QQY1Pm75m7=HFNXqdF1EmJI5;`14pft+WDq7RJdU z!IH17=y4>R3U+oZ;HJwc{ipsv!qp2@-7Dw1-;jC1_Fw3rS_8^tv2}K%5tUKsJt! zkiHe=HWIe32Luagg0GLp5V$MnExRXthm|*20>Kq{?>*dn#=7)~n4C@9e|&A`L11#r zkZ9wm4x=1XJRN>2qov25o9#s6F|+55>M$;jyrw<>O9(;6&-a z`KLe3=_+vJfquKcFqQ8gS}dp#UWyVj0K~XJ`pA!Vfn(eOuoocQygPEeG*w+?;c6`C zkq|_4(6&F`+P8vOSg=GRigZ1x8_bFcMY2&AJHy!l4ujPB@CpM@Jpv>nlU;#`2P^0` zojZpOlBFXD%5%e8jm#Y+2@=*va3y;GX78#>WFmIrW&ALV`Wnjv>WvnCr1|FcVkcn~ zcyTnqX*eyqGY-eCtv7OXq6eL})NiXd} zR~7r=dq<`nyoq9kxI6$dahEKSplgGH%AE>pXtMo3j+haEpo12M?4@TcHG|~p58jl9 zO!wCAZN?-{slKqS%Sum@I#2S{ty%GQ2!PNT9P?j3+DVMY}koCdvCTq7##><^=*lO)zTso!x(<_|sL zJw6Dxfx521n>D)Llo61bsPt`S5-37IFiG;vr|%hQU4FZK0_@Pwg-1}ADO8=I+j=ldF8)w(8~ze^DYTJ{o*+Dyi`k7Ku6(Bi zvXzdSv|N2l-zi$id5MvfDXv6MZaOJ`c{MiiUjTdM99{hLL_7fLv(DT6BhIKld}jw* zv9y#FX@IN;-F(Cji0NK0{QLLACqBDLYKeDuFp!{`Ej$Z-fQ9(n}S7LJP)z04-PD)XJlv# zv0u{P>jId50uZW!x~AJf@i;eO#hT2{Q?CSRsggE9FWU*uMrW`mb$ug_^N2TxE(QI< zx9#SWl|gMo!;@`L$10qVj7uul{ida-cU`hbK|#TY{}w>muU@_Cxbc<*%QWM`@-#H3 zjt7C_31+yW#vkm2Wai*d-}l=q%e1&YT83xMv$C-Tc`QEmgLeahHtC@_^rE779?s@Pd-A=J&h_ZCvk>s4A&Q(?2hEWRw@Cb^w>vpIGaHB0Cxw@ZMZELBYoM$u zafR9#rubOh#Y2|8zx&=RyXSM_g&VyHZG`GstW^W1!ElDsxcO}}|mMT+= zcc>KsjgOpqy>3>N8R>*KlqlW${4UHKxm#;%*IGI!g`-~aoeSx(=f$_5Y5pcP>}bH$ zT_%%wW^zt~r3l_`nhJ`3hOD~R=lVqwDTPm{k7nrGj1kF0mR$N!&}zw6>co@OS{4hHHeg*rg5br$*4_QI40Rw5pSd(bG)W(&l!{pqAndTt zoeK!$t4bVA=1}iu6dgWZaPD_CNbvG@h;|X4DH2{f-u}12RJxh4B6M#&nIdLZ)rfz{ z;djxy&#?Kely_mLVi(pFlYH-WGM*R7I!PP9VTspeVpYX7uNO{9$U_r{M(Fw_S;QrOxyrzfK(=fkhp2?(kLGh z?U_gRyewkeb{cfB{gc+wDmX>XL_CmrgY1)ABr|=Sb!VO8xpmC&->%TFv2mwhr+G0D z7XSp!Z?d)Qe@GiWMG`Cwm=v(URKspPni~Uvx5Cn`aST}&ZRpD$3vNi4uNp23N-&Wk z&)(i%)Jn1e?l#L}r{gCapLvdcibsxG``98dr`S``-W@?zgDw5CUt7IzyDVJd*=a)&rPC?`~nrvG$ zzaWsB@6zzj(|c2KZLt7}TI@=%Aj%XeXC$)v9c%;WfWzSW3wY$N%D1GbnUcP?p#KW* z@Nsv-!S*Til>Z);I@+YRs~`ZSq^C$ zuto{Ij=U5VHVXXg{QXkYU^qYjCfEJ$88K5*4D8zp$VCBhaqq58DB+19aIcrH>eCDC2a1DU09>J= zrlM@Z5@enp9{{-Pv=&L>2!Q}(j>7)G@142AZYLlpi+c3SgMl;j`qy)arJP(xjd;M| zMsJ+U%o{x*!hP2hjsHLTBb(QTj+4_s{l*Lo>=3D+vtheFAg7QPc$fFw=?G$eJ2u2U z_%f7%&@l))CLORXLRCv93Tydj$G+>ZK9#%{efjd$y%$=c8p{*Zmn@O5v*3m=PnTsb#S%>0^l82<_{_yE* zW)*~eaQNJV(=rrh#vHm4DrU<&)p;&g`+g6`nU;iKx!<-)|)yYG_xgRQf ze2&j<@Z(@wCX+882yM)~e5B+mI5gPRVHPexX=YnT#yqGIKjaF%YS`1#dlNYv-{a#Z zhFj7b!k0jvO=h>j4VdZHM;e}~DZE%Y`G5a3g+i6s6P5TPCyNm)(l92t)#WTYJljxE zgblZLptp*313RG2JMbK2TXXiq$B@@FN=u5b4gZiov>B$xp~y$BC~ZQ__nzOPTF9_P zt>O#2w63+n`SAO{#Cz7EE%U)`?bhHIG`XIyL! zSh~>>Box)x5UE5UhG-&#H`!#6{qc^AC=md}QaLG!#i2%c`0=`EDj4UqnFD6^EzPQ` z{XH({0BiB`VcaI&z>33ZBlCD;HD2*%&4=o4!JTwkpZ*zG%86*Cf_dKakyyM&PrU&b zBLeCpYye(?`iqi(`Fzo;mQFP%1aQSSh_pQxE<|3LdS=2nJ4e+(nnPDjYMD%{nnP=X zma#qlL$Ij&Gl6VR6PdA=glFIs5)6Z9`DH~NA*LNwPpn+bI44`PJ*^+2ZrWn1| zoQQ1^YcWU-0Rn~n;TC@ggNyydgHPAnI|^d&OjjabCvo;oMjT7)iV0U!Qy_Y+A@r+` z^}W0nOJFj+B?82Bw;pCYBxsGcKek*dBM4DS%==-HW>GNN3OA&6l`^N&;*5 z*PkP|-CQ5$xR$^MvPM9qAV7fd1lAKncnWxu%~-Y{LKa9$K*WQJO&(6hYrO`}qkVg~ zM`t6iWsm;;et0l54rC5Y1M1?JIz|D@*h1!i_4#IIi>ljEgV)~b*Z5=8q?+$CHl4HW%#wE zLGU+@79}IhmD_H|tu4!Rbu9)6~^PP1k+~k}zrym+x|zIN8!w zfN|1@pz*vUAPC9tHVw$pDGv`18~^R(oC3;v14Q@20KpinRidyCU!EQiAdZ0%zA#;+ zi4KZ%H#n5TMsft7{$$zN*#Rb6&clO0HZG3aaY+bp^ip4Rii4gWl{7RI!M_gq;o)&! z_Lwza?}~^TzH)MO>;o|L33y-K9|Zn4xcr^}l{Ckh-JM`qsQ@HXK-2#T10po2^=-x* zY-{BPGoJ0Q8m@;WGUd^ru-?S)@$NC_05Iy@IYCc+bSxLZmv*YB=M;Rqac^UQqn<>q zQnqIK*X0qm;R4$KiGh{WbVtz6mW?R6KDM+W+}zU=J>T*aeTp#LdgYN$uqz4C|H}dT z*qEsq%jdaIdwX^SyB(6G#5J&Q5zMlnDfMWiO&!qG&8;BoIS*Gac~fz)GW2ikJXA!+ z${?lZp9Zr2p<~4xl~OyM4Q{`hWz2K8HQY!|T?YShxpyu?;(`5^Qa&!UV0ZuuOmyUo zbxn{G7(k_NS^6z#`rq-z4IYjnIADM%8lhnpw*bh2y2QW-gvNwIdfvHgZQB-Ued~0E zHmeB$MKg5rpOcr*@AFpD=y5RSfM$H-9*BXAQQ2zA_KMF7wuMM5ai-HU!Ry9OTmRhS zhId87NNss1&4DTjpxR{NexvJWvDrsmP(njd?21bESFVr0*wNyaQJRb*_s<9>vX9Pg zK&jVy`^H!45*bWD8Jn7~wJ?$-qoEEjKB(h<9gM|K;yt-RE(;0EdjhhcB4=ceH`|)5 zL}sMK>J21&py7E{H~5W7#ndn@d04e zxw@N5)H0@kc9s?Z%0+dxckhwa0J#FJ@a1Jo|5_no}?@GT& z4O^6;h<#^*NVGh*%Y#Bhzlw0(J!95U@ND~J7U30s;b7;<)G>mI1lmwX1HJWIOl+6r z^jwa=<-_3?J zGnK|MUmX>+vH1hTc@OBWF!02@aL^UoDA&-!!k<%^O|Lin-l498$C9fq{N+U_^gCYa%K8@pK8Qd zwI@>2o3{o_o@J;iC?}`ds3ZlSyWFVzT*gcZ4)CcY*Y~bBgq(^AT$`}`7KM#mn=!^( zsbK|UZ)n?E4$el{M{}6(t!WTV{;?-Y=2a5pJz>sP33+TNdtay1!INp=)yD;7WS0~b ztyhhnro@7%SIQ<0@ylua<9oE=C@Np{=g*)2S?!R7r)snU4|jmWBy}W!9*A+Gkl?G@ zA!b10O)42Ewi5IdkQ?;)FW|%SPgPG(0jj_ob9iQSFVIomN-}Lwy*d#~nlxWPb1RG0W8EKNlB8WtHm~quTn69grj|m%9I&-*{Z8sN z(h?PC;9MK$z&{PS8^>NlW+IGuN`A!Pw-5@v#Ad9QCl&bUQtR#lZ-B+I;y@qpz3BhNV<06%N`<4O+Dg?%KA0Af2q7>1IPfj1OB;~P;w!Giiut>{P-7Y}GCPz+Ji2&a}0+Ynz@&4Zp zaPXfFvi*b)h`>|56M(sEpjdCdJI z(w_|XZ#;bc&Ggkb*gcv9l83f6S<8de;jMlI<@s56X+osI>-!X0^9u{a=j>c0$0d7O(1i75C0v8BfvDn$?zJF{a!|0n(ch>=~3`Z)x z;M=$hK5U`N(Zi>f=V4bS=_Tk#?H*i z5q?|fj__-Q1(}G5M5;tL#M}UM#%aVm-GE^%Qn>j6x?|s8ue3#iqYHWGHM26~*efqu zVm5uUKBxn^rL=hAogXVvOxOBVadWHCUOPhgTCAo}Y~&|bpSZmwA)TNBRQ*Om;3IH~BT9;{E?wzd(`kevs~O-i$ zEQ#*EHd8}GqX+|2odH4IpunT(#O$i{p6LCi_9h*^&it)?>x67Ut`7pb?^GU~EswpC z+Aln-_F1dgF~Jk|aF^%pawKliu?Ea?xwkA^4~|GbgDCtLr;-I%&<#%Ql5KF2R_`^O zCGSzrXf1B&B(ar$Zxz3)*#UlsRF}tP9`Sw1;ZV`^PfY)5kX8BGF_2k+l_*(V&Kc6M)kX3u|CEVcqy*$5C*;dG|i z|M$Xr;NS|QYKGZHPYkFn=#g|4Kh-Nz@$kW;3A#oaL8kQ+PrnBE0YyN$SwVy`{f4pc3jV}dR8)Rc(5iughA`&ucPy=l--pkqm3&1D z5NLB&H!%yp2&~fpiN1aY+$3fmDAOBkZs1PJPP(MSRzDAQ#KS*x`y~%!g=BpIC6V5e z_xd)(roKfE&C@CTBO(^x`?^wf|ICTd!AZKgJSrUz(McaVQU@HMAJ;TYLyIuN$=Jrj zis&SaNn8ce{M4Bc1Z@5&|FOs}Er6ke$6t2h_0aNBqyVomj|LQ0G-NAXFMu>}Afu@; z#*d)10qj7a-6Tznp1kAuNu6S=#xo|)b0wxhpcoLf68aJJ)gp*ALb+raK-11K*~~jv zi6h9W5^Kh_$at;rHuG6sI-}+#M!Z{N>za?rN8{CZQ3LzoAn)Bp`bUM(Zx<+#(z2?? z_4ri_61QNQ1VI3Zg&)_8k0qb7V%#iUqWjL&zC6oeHAg>J+OGktqgga(!qP2t6Mg3S zWYcuc2Y_ODEN>%qr7$BFaRKhbtXfoS=9`*yt~oy77pkDFz9`%f89ZtDk7Hd20-7Y= z3W+}tuqt-&Uw|>dtVX0)n31mbV8r(8^JyaRO#q4lK9yWCHz~U()}9_3WIX$O#Xd)v zGz0Q25TJxAF+0IWi@X8WCMO`~r{gXn>H^U+E?}z?d{+q!c;`Ouh{LO;Hgq2DngjR& zGYfvZs&?lhi#7usWdLT6m+dPpUYvdLWK&VZWlE=ubZo-du5Se9uv3em4Op5lLNB); zj=LN zcCK}Q7s(f15>9>fD~tc-dR5R#9ck>j)eX#Q(#A3`0+xE*$Ms4j%jKpoX za-On!&*yap!1R)0q0^-nXZ|pIPJ$10BF{{J;)mixr0arrL#kn{RlXEay2mk9i-$r7 z-+wtTIy4kDHIadtT{@Q9X0<&4&YMk6O@XmP1NvrP?U&719{|kbj9Wa-Pu4mc*CQzc zQ$kNC`T&J@67+N(1hVmkfIt3pTo#v-e>6x+DLeq|0(AykC5lT*=(kI5v%%6SJp7uP zniJ4_0RzsAew8!6I(g0wwwj!Nl`RWtWbgCn10t6i%5W>(dWF6Hl zmh&L&rgc=j971TEVs{MnK^W3XP3vhB@us|apA$jSLb zP#QSB=@AlP4YYvzv;#p)9^1lBC5PCH!{z z0MG;`zDi?-4ZnmkUIPGwYV#43EQ(-T15HHlJD{Hcs@c@U8Chm1T%}JG)7MjsF~wKYCZQ2m%cdO_SG!f>j%K zIB4m2}>Iqv86zDc$j73xC+Pnzx6?<-?4TL{NtBUP9*E9C`&&nCW zRNX;{P_hvR6HrN1R#tXwe1}qviX5=}f{839ni;yick}wV%x>y)%wmDRpaPa^kBgi} zN2d3W9iL(}(}c+x+{n~E_dD-qKgTpG-!sQOV#F6#Wew@ zr9Ew3`nvmP%Bsul(q?G$;^K~bVR>-!uwhj_2&_@q1Cya(0#x{H_tl!#FxV3!8}xLy zs$8V0N#p#d?jwCR5YYv{P|sE3wHYn_`0-=Az^>OAka|ghvApM8%oxnj?f`oO{?UNY zp=~z6(`W zqGW2s3jq174p{Lk7=KIcuOd<4{;;x(J04XKD8FKI_|hToqIXu6;bA532_>S-d_yR) z-JnLMCSljd_cY;}`MY1|mPwNppx3$8`g1PEEJp-0Ba-j%Jk$9ed5xuTU_iubTUo^o2ZWR@*oBp_12j9h&0 z=hi~?J{@Bl&6q80KJsIizpW0drKe5OqOnO2*-o+>c|0&6_{cwx2EA3@1gmw__1f7j zK30_0qJ6v)U!o=hoH zNk#Hv+^PmR8sD;Kba=SU^JiC=5zkY(|uP%W>t8Ep@F zWQvs1Y?OxWu!=P0{3Y@5fti}TnfIUH)PkLyk21oAbqj;&s)bikeKQtD#7XMtd#|GA zR#G41!2DAxTR&9$P$cFfp5-(I;n1SH1W0`3>T%R;>+>I+`_5{T>)Gt!PF#0JFEYlZ zvA7{#|B+OEJ(LaCj-CL$7!5r=Qq9u3Cf^FOBh0jj^zJ3z5pv6sj#M`Gwz7REha_rr z3j*Y9gGEz)4zY%_vvzV8K-56~6?lC%3g$47_453UPQhKW^uZCQGide@Apae&aRsyz zsGvCSE;0XT;yN{Y_)iZ@h|2rX)6Re@5@p4-r;tl8a?#q}nX7=^YTSIwW$3${L2%1% z-^q<6U;kj5xhYzbEmZSc#3T|uDtca*R@bm-cb1XQ6c@0VfbkgbF=Z9v+KM@QfaO>uG@z&-P;Wz?D*O>8G&@N;x~w^iG__LG+9Ky%Z>!ju>M_sC=Mp_1|hEwtjx*WgauzW#S8@|m3N|8U?L5fXM{ zHNOZCt^tQ#5YR@t*$0ivJ@PD&46dVSFk6@ozj*rUk;5IZ&tO?v{_eD9M(A)lm9ZYq z?&yt*QG@Vb>u=bqdX?R#*2eK1ZTSD%2A`wJnhP0G@$pZqT&;Q)VgriFzsdQTy0{NJ zN9&q@6*S3RzPNTg8lQfAW+pP5vR z!n?I}(w?hOm0s7o7k{bOp6nwmsvHXc`&~`v!WG?%41`~>XxvV|cbqCJ`i#i>Muu+1 z)oK2Up$@C&TP@c7xQPuZX_NXM z>7~UCfjqKwEBSFxG{ZkFLu@*kB&vUZ%^CMAX&{}3|2;a`xI|k&MbUhl?au+iWPZ(r z+&VV-Mp{Mu`&pTGfsHH+a3mnle~v^L9VXcD^ZY)*1cUPM>xQt3Z8 zKcuP!V?beJmdS*0l%O;j7|PLc1ELAg;bfBhYckll15pulyyCBN-vR%4%|}FMv7)-6@Y}CE0FlGP_wU53`4!KHZ_jaf7yF43}FiEb)j*xL~TGG7eAS7~(QhO1_zDs&?>s5xP{4?BQ z4Qb7D6}k6hmUf67+41hp+Hy{zm=#;2@oZREo{T9_@RvyRon3{@iR&SxhX$tl+?awj zp2gqKmXbDX`=1aD>(D?*K2@$s*AHEs%XHVJQ-7icqatYyqX)fj2JWAHjDo7kzlOy= zv|sr^#Snmeo`CXRoPD*+;z1q9%cv|wK-L>IV84y@bCOiy(%lV` zQc|KwNT{TAH%JT8NVgJF|2p^k$Jm465WRTMIeYE(t~uwkVyz}z$Uz`)P{wy?(hZu6 z`+Dsy9{NQ~N8vV_%xOwX&#ffwVt?)6{XtQDn|HabvvJFOlJ14c0-2IJVwxTv;^5N+ zwRsg6(bPr9OFu;Hl27&CRvVn~oV5>s<*nqr=BFEvh$Y%K6oqtg3 z=spsu;eNhT4M}c4&yTl=Aqo$pP%^|mgeFfagqvQ1xY2lRDO&a7lHRs}7Pl6E(&SZ) zwT8#v&;jjYJt}^C?8>@H=jxp)w0FzekMzQ0d89R|gi#UYmVK_!jzr&-MrBpTF^3t& zKKNiQyZwC&i|(>~@Ht#%&IABkMbb+gYd z^%~dQYq55c%h$;-;v#Vz{_$K+r&|7=#fcOrED$1EuP<2aaBY z0ZLwln9FTRTMqS{T{Ceo$i6VWjc;F)A_eU|=;cht84*9ssJH3OiCY}WJjAE-f}|jy zWBr=tlaLFPa!wxq-#J(@Ah8y5SwD?L! zf{7|Jr-qLRzpd4xDt{x#A4?9qH@uS?T%{3fBR?$V(jUdln&Fa4X$_~@-@P}QQzKc^ zzJx$7*TJth_DCGTQ5wW3H>s{!-droQ#$iGB;Odn5Y#~ro*iUeU-g`&CcQ@K>S6iAYt4^ zO)^>_V7N>4`A)#&=7xpiIg}3nbA-Og!VJo8bC5QzAf@+OG2{H_-Jrs9$5;4?yGt#j zLnX;GVf)|Xb!wgv#oY+Kvp={+w484DIRH3^O9Wgt*`=Lq_o%^f6BJ0h=urGrb~7 zJ&XiO&I1(H0%$L7SN1Wn%lujv{zZ+s9|wsYR$5by<;Y!2h!@=pd7UzsR!b!PhZsQO zcyI52Ft_`6T@L*Umti>qS;s-Z(FC!+QRLH54GbqcuJ>lD%ZRwod|>x`?zrtiD;}@K z&#%wdC(F#C;rn>8q2(9&sI+r1OA>Q!CT}+TcVd#gF8=K5vXMJdsH1BV{yRkkDjq3f_{Rha482;=(zQ+RX#9B8;2`)< zcw+ICq^-Sbh?1uz0s!FOX)Ot6cMe*gRpm8S(IDHw8=AC1U(Kqf=y+{RLKMj0_cvE-+%nLyoj~ec3hfqu;+Juv!I}$Tw)f4&=hO-S5k4B`Q%1F z@zriOFi!8v>3dYHPhle9*jJf%@xI@+xQ!roO=sp_RZ}0M_0ma7AQs}wKtSMk+6X$r zYD!Qt^`CT*KK<6WjI|;#LV|~_=@z>@tjG5HFx_6$`uQlU4@ypHUpP$pBL<{+#k$Cy zsf&6^jZP{IY3a=oQbYNbCD^?V#sCHbh49e1vRw|{j}fk5xYu>+WT*g4gU>47g3_`$ z3nrHuJ}oXIxSPR^IqZ#sfGd8k1<;qj)(h5#u!+Cu#5k!mJ}H-$OXZp zblYOWpYsv@p!Ehdb(sZqfYFd^W$kw69+cq4yW8s#n5uGgZ6e1t7a0^hu$(|sGmN80DYr+%T!^Z_4MX*G|BwxoY zJ=`!x$!h-}>RqkL@DmI~ITu^^#fbz_Y8TOf{3-`xu};Fo3=*F$4ERx#k`zxDjxn`I ze5Ae%guv1M{qr#}zCsR^d$71;QQ>t+4)(}{-h99eIhZ2^YVoNzp2Jfn;Uu+VS3!!R zV>wPbQq;_k0I?5{XaZ!ed79+I|Mz+R;f|fuPZt2=-x|f+7;fxPi_sQJaK1r@6iQ)p zz}P7k1M>%l^euQ`qPoVQQ zsbj`a$oKi3_~iTKZqy>)LLdry%$>(R3`;=keQC8)dRG>XHb4Mjh6JF5BRxopt zUouR8%!HkDMh{U;l&j{pcycY2rfZaLP9F4W3KVbj%I_mEV0S!yg1Dj@;z!RJZ(i;t z;esu`F<=;>*ioUUelB1?)0La68{3<(yJkSS*r4o`DPj=86ce%bnt1`~Y4LcE%EN0p zM*KWbE+e3;FI&L<@Pp7FJ$LVGV@v^R#}*dyC9eIw>c=&&Q!vYX=({iy^}6q;wrn#> zomZcQ6EHs5bK;L=3ob2ch zoF44KYqt5AQ%AOkYsU!;eMh*1feoi%|} zj^)8p$LSWFZKpb`9+ghjzWOhQ1oA2P^x(e;vfNe2veUt}G>T;tdZ_^(2(S==ADfs! zTKx(?f4g^Rudj8mq9pI(5UnuhUt4Guu=$F2n|9 z^MO4#I>98>VqLWq_UjxvuYfw5Po^_&JIZ3DEz=Kf^r+64nnaa=pX3>--nYP9Vq`yh zTT6_@oXaHeU|A)q029GTjQtp-hqpqOw9IeV=Pni=jOpov&l0m_x*WqQ_9kJnIu0|b zI(8;MPOq6-fJoSljH+EZ&veYEn(q)`M){rM8mwS>Uwl;wQmn`|jP5Y&-G`Af6EXfG zJSEyeV+gqO-~(-n$CcJaH7?>4!XF$0j32bJV%cD;bFo`@F!GF+BJ8He^doD#5IJO&y{cZ@4g0w-;@M;x{Fy>&hL{e^uh`T_4MFsv zCxTT2f491am7(?vjd)yqa~*!U;;Z?GOQmSHD>ijR4S&`%kS=rRsZZ2c>97diDE4(fCj&>#rhRkXL2EQR5`}d{f9D= zl?fVOsVjdLM!C0=^eRzk@T5YDqFap&MzPR8lZ@lS#Pw_1b0p<#&K{n(b%ue>l-Tl+ z)lWu7{sUnW3c;q}e10HHsz)7?VB<_nN>5JNzj{0x`nu+z(@(BIkCB_30BACFu*fOm zeXZ92C)8n`aqIEU48L_ZF<2FPN|09bKs|(9BpK@Arcpe3Cc5k22yJ{{vdY#2|2jk@ zKVFN2IK$yvH25w8{pqiCVY}uEuFh4T&@cxHw3Ei9?i{a;%WsB%ud;IoUZTIeAU^E9 zM~bCODaU_0Z{c=9-YZWOD7lZlLLlnDJ&)on+o~R+3_fEJwkDL8^kD~ zDUdfcrGqSx#flt8<9y#@??&q*bgky2qh%@?xq#8G+bJhT6z*5~V#qV<`MdyVr9a+B z@!W{}I3Km#A^BtY#`5)5HNJ^i>pZVc&~kuVqs@Nm(YWEa57YNukgLsiM7;lhNE>}o?&Rf>xN!n1TXKkJC3f>)OZJflXV2D8SWzcVp|8k z&3zrDKLzPH1hSdf5evDAnL@0AyQixs<UX-2@kMSti0CzEA?U@n0D#Vxr6(xuxQ! zSs5jS)Tybdhi$)g_xCG#de*II<&VJoqj4b$ZxVb5>0VY*2|fKdZu=%VIRql45Oz_M zIX@4C&Yl#EXw8SKef*ZK!7|@gPnzHITOon6I-gjS1(3sDP>^nGz2kCPq94eFc%=3x zJ>-y<&<@dDq+g6&0UufRL?G?dC?O>UGg%#@K=;no-|sSFI9qG2M^d0{Nk~dA9-UbT zq#?ru(7@5B9929j)#gY(g`B3+O+T=afTu88eAjSn5|_}&k1?EJ{$@GSrS<9;h} zz+(Tij?OBUA)xhQZq;wuu<3~;a-jHKMt1JqwYYi|9{+06jfz)Vg1zcdZ&Eo+(sp0I zQ1vlQW^rdEjd;BJQ-htOyQ@7>3rbc{$M5Dk4P&In8ySF#jQfLWC*+cQN1J^G>0{yd z`S+(ypi5R+iXJN-7Xck<;R?vy!RiGHc6N3g2*wVkI~*LX$eSI?oA}k5~LJm#}x*0h)6;Bq9o_#C?IHTia{Q)b;A0;ux zooznZS4)UZxG2XL_-|85vaOtNI+LBJH_>{%YY87zhkxVx5zC!R_zFDA5v_ot?2438^nSo*$qjR3hoJ3fOOlNsa;Xp2KU#2=e_rL02g9ps;s zLIP+Yk#obW#)t|G>I4RdR_?9=G{MXui}rI`kpD9Rf*im2?R&XdOy{0 ze0MK+{J`_k5rWg@HAIC`E3RAX6LwR*fcd#eeIK45U-XBcxH>vHZ0rG+;$)O8zsH`v z&H-yPb;IB-il-K@>JmTWXaxh$Lp?-+)P)p^eUIjedjrZK_gDa7?$r z7!lc7Gl2pEzLq3haa+!UliC)0RZBCGuaIw=#akt}Ws#i?M@0RKsZ8)5IpekN#GZ!r z$7k`cQ)%J-2Nxd}bO;XBvsKN4EDzNnx_1!}t zjH?{LHzpI5%o!XK1mSGt`lME%>17I?{C_jr1twBJ&=}S@G67JWw#2-N{!!`iM>n7v zv9~WD+|88pI^J%9kI;WWBVSWJj`27inLdH7Tq&nJHKnPF6kin0e0g$uL!J947aE(e zrMAMpU%f;y_9pusMnF;hgF426(MfL{iKap?yIs9db2Q)l4&$Lhnm*Rt1@+468XMvJ zF$M9*^ivw6V8>=TJ$}l#=Z3|Y81&MI)?_PDtWhTL?>Tvxz>%BkkazEs=04u0Xuwal zZm`z)1b>?`y7=1~EO>oJw5)w6;7CiO#IKx-?_~gE+x4yj=0o!wg+11A5wy6qwJ!dX z*!HL6=TTJKdrMAT*})Z)*UgJHEzU}L=%E;)MB2?a8Ko|LX;ZcIb1NRj{wkF+F=NVT z>g&Hni}4|g%XDjMX3(maiq+i=o3>_Ii)LSHe^3E)O;c|v;~W0%IGv@@CtMD1Kega7 zbnN@3;%#A8i0vP^;eXD`RT0a?3AH39408gG+9L3R6@_Rnr!`MtoUshyMJ0K8l&_Dt z16vrqWf*`Qvv}A(X8=aSzmo0s}jc=;P=@x`Zzz6-^ylq>SCDqLH z4h&|VZ2$a*r;<;SoZMp<-xLf-7%Jr6c1!B5lV+IT+{V%5OD~=J(nP+j5<{GI?G)Mq ze!u0`FGJ1gG4spFsAt-nfV(V_B9_40ac1( zQ>X;Of9_f6=xV1LTQunjD>Ii#B`U_52cSs|5&m6UZC(t15l#JdhPsW%P&bu?Xxe)9 ziya5dKL-vm_(OVa=%`7|eR(ZG7g<}`Kf-Ga80Rhs&z}EGB5SgOlfR6KQwKRR0EOJy zg~Xmh)|B$fXMrhN`ni-!oRrG8mJ;-UIy#Tod-4dud0gYO?iT+_getcAw+oW@bJ{@( zoF(cPRAYL#Hg3jtTDW-*mp$wh)(UoQz`^hk@7^Jg=yIlgBpE|BR%YNLb0kI70Qz#C zcLWDYTO|w?3d?wc6yBU35`;24dR?xY^S(axypXS%^%WXZCr6K_@Nz4cW4`eY*hwxl z$Rn$S3FKt-ISSUbG5j?fFbb40)QtWpp~Lv$t62*wa!go?oKv71IRX}U1ZHh9>$Ggx z(?6ZlxITSF>2rc3Kda^u=+tpTah+zdC-h;Jx}S*Ah!IS9GNe(g;^m5q>;U6Px8>}> zin9;{iW4^E_M5}ObKJSn z>?GB1z=oUxB;e%UB{NFfP}qq6POC!#Qi&_2Ts-?-MC&(oEv*~+0-nDgYd_HV5vN!t zi+FKz+7P4zzz`T%faU2Tzt*oC+tmKt@G@T>`JZkCYJh!+rJ$|SCmxw1*WdL0m4w=Y& z96z$7g54|N>%VKBk<8q&mjW?IAt4sBaVdH&4tCiN5{3EB=P-Tkp`OJ(RQwAZ|ZL`%=lj+rkQ*+qwESAKUe;q72 z{ByP0pzCgQ%FEFEpVbUd#1gcA3;TxJe}yurMh4z>*~iSzNU@q|A%>fAs4D& zzfql@Zhjv`r*Le8V)EOGIBZ!iq>YdI0@=a`{g^GT9%D*yE?yOn=LKc21;nKaycJUq z09q0R*Kv)gBkaBwCwFmXckrJa2QPcip<>P)JGrmC1SC~Yc|)2aqXN~>_suuhh6)uPh zks6(XMA!1T!s2v)@me1t$3lwI zM$qrdThG9NSyYq~))Xa7myJBjfy9y z5iX*YX7f;g%W0(cX)6OZRt^2PIyOkcc8{no&B$jiBVg&$*O(G-xtFXDTRhI#xF(N}r8e-uI`;f~m}LvL!2 z{FdRkn-(bL$4=Yw?l%RJvp z2Z~ZK*oA0pIamg#k^Y;G*npTH%NU9zRQ)3Og=HyX=^7$z;P#FDp@f6VtN?E2Os-Ov z2fr*T6nS8I__dkee!S5ix$1mF%pNLjmP3Bi|)+;_JxGHwo&@2&ff7qKxnfD z+-S`NOilE&l262Rec+-7kzfAOmet49OqRJqjK+>MDv@@m?eVL8zaSTrl;s>jqmi=D zxV^hv?l4rS6wB4S!$N%wGvElUKR$b-x+%$Fr5i8L1+?5PF_x?X`Ac6Td~+1HikT8Fr@33!bWUWU2Rl! zsY51s&<-oVwA*!;bCtM?9v)Q!T|71E1G2qUvQpc`glQ~L#QAugOm}#Aah+c!!lD>v z7~vX*CY!wc*cE2~0uyE%kZ@VCd7e9PpK?r4!BR-?A z3H0y)xIdBo(zab9%tTGU*iG_8(69HM-eLw8wbgcg+c06UpzjpO);dnNBY3k(k z;w*~qtatLD+W0!=h4F;{%xg4j){jAZZ}B`XzsZ$7wu>Ce4uanVZ*}ud16riz5Ochf zfM#8C2Uhh5%fIxy)tT?igNwX&mS7J{$Nbf?=rf_`hF`cN&&$-LQgCa~VK$GI(wZ#E{#ZRf9No{$Mi^B&6@fGIFN+qs z$xgJgc2I<99HsW-*qBeZLTK<`B(!a?dVdXWF?}VpABJ5meX|YE?XKW~IQ`ouCC4k- zD_49svw{E>0`J;{js;ANiJb)`^ZX5!wlR;YW(xiW4Rvo6x|itoigqTi>g!MYnX}7c zcp(Z1|DQh}ZO=DdKgB{^P(1tgNJvB^ucG2WRQ4EdSANew&)lWxVa7Q&Hg@qxQ&sH1 zofZH`Sy@?4?$N%N|8mWne8WbH=80M?i1StBI>BQ^Xl#0V3j=1SKYr-Kh7BAKAKh2W3^n^zu^Tzxd$EZWm}zT?CWQ7LLw*|04X8P+@4LDjtMM&WHU?xa>Pi42#ULrI@wGG1#AxsSu#>+$>(W+S?=Z=(=EaKPFGA#i<<)hyV zM*Slb6x7ZTa@c&+GauT9eZM{Sj~LlRkf{AGgtA(6nDBl9rT!j!{;(u_+r}}R^Yv2i zC6j@T95Qm+Keo~h=^T*x9I{yEvWMu;#7IMWekaYMwaz*@G6)T|XrqI(JK+}`_@igG z!u}ikGK(>iMWCWv+q};QtshR;@O-^!;oIhkTX_pL85T2utq0toOWu;dWB{+3TlxUv zvG1ZN*BTKZTN^tx5D>}!qtc|6#Joqri%cBw*^U)vUn&axF9qU(j*4E`Tqt#L66{`maEDaT%g|&w(9NnoB(P@zj@vN7n>Ki#-cfLn>eY@b9CWT#fC` zpMgPOx2TAN%eV?3Iv7+4bS(UFQYTx%KC*a4(rDPS5cy@SyKi%KVZM_|@bzv4qz5hI+<=QkiK!LDi zt}_=#pE31@(F~u0O$fK2uQ?u&k=x9Xch(c(%AXprS-NqC74vG}qGIAQ>{j~ zgt z*;`JBCl^g&40e5kgDPU>nZIsF5eRh0rzXY>%(y=OeKrav)N<=H8c@IcrDTm_o?Lz?E&TLZ$yDR9>8tU7 z1K~<}Z-4AWWBJq7XE<#S2O#-Oni=ku*7#wgU^R)Hou&0Y8_>;^I(FZF`J$^IZ?N-Z z#%=MtpjOZaNsq0qjy`_hOd8vDQr>N84{k%w z&c)TW)xj+%w+h$Z&A+QmBaZYgj3SW;e%Lcc{t}{xFqhSNn7Zu_b0zI=YXKo)Vd%n} zZNDucu~X0u`-3K?r?1~)`32bxL)Ax0aA~7M9ht>8_k8&vbdD@=fUL0zMQ5w#%6f8mZsHpNLx?P+v*(b~NdJiu+f%ZFoUtYmzcnj8(Sv3D&8}d?v zcQdaYUgpf8m~w5{Rn&r;Al*~k9biOo#)K2EBj3P|#F$=^Tt12YZRW2o*e#R|W%^KA z;O{ENxJ1L|DG1WBnNGql@ShdJt~>$?N%wbJ>DSWmKQzVI;0tBMDOAxTLgdr0jf2}5 z#7Nc145aFIF|KdrS`-?2t*z-l;O&ZUD%08ANHhNpa zQEUg5`vr;+y%4wsl(Y@kUz?iJ%l;S@f*!7rT}`Rx@IP+T!#msz2gpx(p$>h!t?dYW3c%C70CN)+SQS9; zcj1JLQ+5vxU2kN8xvY1i6jAPd<_zI?GTU$e*p6WnL$`vN2wi2rXL%o{y!i8Mm;C!U ztHFCMSNoLlFZiLi3-VsLauwaTn?^1Gm3IaCni3(eA)@Xae{T z#%yeHuGleok0qH#b>L(Z!TV&UDn;$G4X2t55+wBIkEitPi0$@^HK&1))>pZ0ib%`M zzwz!7?PSBTsmd_LXr}x0`R&H${O;Qf5lYp6(% zPCRGy(a$xfS_;C%}cBu z7p=+Z{Ul>Y6j#OE@*H=;BhdWWv>wUjc}o|3rOW#J8f2{r3pfHNRQ>tmf;44=r%zm5 zyt}uT^!n?gojnn`xvjq+)P?9^_|?(FmW`aI9)G*jM3p@h25#@idvgs_KqWoi{S|e5 z>mF1VO?ej5kglyDmHf_7?wOHX!38+L-nz^YeiYP$nl%y~H6(e@ex}Zf=sv~G z&o!qe6+8L76iZ_V%jJK=Oy4^9s;Rl|%zh#I`%T!e!UnA~?j}K2OEC3XCQk7H(@&rm zx$Shs%U`m(-sCp$Zt`KXF^k96ociV_yY0%xBkuR;M@6IIDV?U|Uu!5>`{?)#;<%Dk z4x#F$Dk`3JR3*oD3IFLY>`x&CjPNJ8DT1_1UNFPbvHvEo$WOeNmt+dBc`VW}0Ut~l zP9SHhBKA}M=exBx0EAiy2j8iRWo^?*e`?B1EPYd22!M{EGenO7tZowd8k_i2V+njU z-Te_mT9v39f5&y#nmTq-+nn>-r4a4Z6cUL9ms}$j<(~emrQ;uf0C$(C!hRVaseDeLANOv ztE0P&Sl+mA05%_13{40q7edGc-MmK17jh=HocB8T3ui(@H=X$f8HfW3N(kL_L;YB2 zf~J2fDZUO_F@b%9&~d}I$N?dIRjXm@QkZ*EMDAXJc?sN--;yKlyM;bkboAn1$YBiD zX2H#93+fCaBa*t>PYqfr?JGf07bUkep&z-vU6Weez*@gQ&_8cNB3eER{mmR1{D^Oq6(&R8;3eA zcmAz_zjony@e#wp=$M#Y%M8zT)kz!Qj{pKH5iybX*;T*eG+ug?XAA~9_bWi@BUJ7* zL8wSUjs+Qp4bHJnL#^!XLtw9RpP5dgEzhHqr2z(0s^+&Bn7r{_k1(`juk;=VefOjQ zGA6tB%TA2tmD9(g3~UHk^}0Bw_R+FQGXm?gzB~_Fpk=EdLwl|+%0o@95}ob66-!s* zL2=~>82{lHCf?E@5R(ojO63DR+VaOUa@#(Pv#V6u2&;>`71 z&tO99LL+LR4>Cy$&B9{F_9hJzS@v^lum1YgVn*bXx-ttXNmj9M6L@^@Uf_kb{cmvi zSS|2f6GLB4N{==2H#m~v>(abmpd}UHOQ8S)Rebwbz3A%uOANqL0Nog^wzhVv!AlgV za6g|`oqEgEnDB^YX~G4+fIk8kf14y3E)!`4xNX8uvLV2)sEC%Ho<2M_hUBB=R8)|_ z85d5s|FQ#LRXhB`1!E0>-d*3*?~G02+4)Nk9?qiy2Ufj+RK1{5>xL^ltw4_%p8bj8 zpOAUICGw(UO6L{=>ez0}$^*^zT_3FBpK}qJY-nH&+%+@YgsWJC?m45a9fG!FT^(2SJjgW`ys!Z<Cx zdX0p4C+=v#x+wh2PJ%`YyEVL0anNqsvB*9KlO=@z|BXLX`<<`xR7GF8f~7k!eE8=R zC6DJ!UmIwL!Omp#GnysLw|*}AXRmIdCk6(zqOeD?dqQ1sm%v8S`ODXY+9;&hjPV-% zxi|s~f_f}nR^o)Wi*LLh*9{ZxI)Ew(`ba7|o7k)w6fCk8HNU%5k5vU4Lr^+t04DT$ zo}f?{^{S6#K#lhRnC2f`!v&8r-SvUH&Z+mXk8VduO~|B2N|~0l~3}LJYa&tEFCo@8(D|wO`P=q<;`TR%%(WvV*EJ6gur8TK-QXY;vry^CnorYH3pn`g^c4uYQY5gohpQ@q12v|r3l7*sI!B4kg+1K)zj(QSB_x`vo z#kVKUNfa!SbTUSrav4mD@;rRbIjAk=+Mg}7gHUT9QHP#9To}N>=aGDXk#=93uEs9b zQTm;SY^WUeTLFUz`PUI^YjKTMj?;SN2ENkxHf=H&TfV{*J{!uGi`OaJLa=EW!kBAK zKiMdv=ni@5;}|MsNdhdX4&t5|RX7~Qf#Cxvw*laX1Y09BH}{V0v6j;_h!++W7xO>q zCG!TNRv0S+%r1=3i`ssp0hJnQ6Y}(wNK$|DEFO@6r8MK-UM zuMYWaS1K%w4Dw2D;D_XBcfU!^2(sV<(=&EMOEi+Muu_q)ZNy`$N)Xa|>UjVU=V^Q= z>B;3ItI-LD;id}u*TSuJR{>R;_}la0+;HChH>Q^0m11e=eK6ogB~d7QEK;50_dps9 zmK>bw;AgLJpQgE=&0(S%yIf)ACa9V57Tfdr-k9atCc)|Gs$jba7~Ek1*|~zl1~C*4 z;V&>Vq!Ks~&Kgonl@YygUg!pq| z1DO;0$OC5N?$^-|tSpWa5)#mr5N2r-CH7zgi3~0vByrFC5!K!g{cHCy$dU?-{tR@o zFi{OrRwfPOWc1Eu*ydVwr)OjtPlyxb(o+vv!7Hz-qR71~q=pN%>~C?_YLk(c;_NGjim{8X;v_LDZwz0rtTeIl}dz8mPhaOux=@SVM^ zj;wOKWkE!v%k;}$2WQzz+m7pU;!tCIm2b?mXWL-~_fk<2JhdyN@5lQQ-{*tuydb{R zq>y4p02}ESuCp7?3FGoM>BX+HrW59vSmYG&8p0A3_82wLb>wu2wRKQhF!O%sl~}YK z2Ies{>qq_}_0+hF-N3@ag!4Y$%+1Zl*N@8~dcC+&wpDj`&BL8uV0CbsiKYJ4ptR^~ z0^JSaq_+p_>Au*QyfeZ)loM9HSQJ$QBHGu_`{`US0b;c2v zyPcy0|IVhZLklwcr9{WV&9H7BKU$xZAJWPg82ijKTAre$gM%(Q>?C2f`iC`SZqd_K zHHG!Mn!a+F-%vHDsh^VbezZ?__uMi?-XO>E@Y8yiJCv`1)ZX?NFAN%KceKVI|hsXi<- zW98$EhXe5;INy-a)9a=#7iZ_Z!oqNYFkoSYT|dwI z8>t5ay%q~bcJ}boQ+KbhS6!b>>qX$$tXJ%Mlu&$99~_LHF7n)E^uDjZRhQO}+f#K( zEvxOEV`au+qia0FHS(smp%j;K%`B6 zxm%Z@;G~B0OR}1CQr>V>=$Ju9WuSS}B)mS_kK9~J8un%N6gcdpZ;Lx7@z5!}Va3tc zpDusGM)YUfK0}GhO+4b)dE9=t;-&o%Pz|3P#Wn$_lx5wJU*%Y2R*7a`A?HU46=yyM>7H z-NqF8QrWNXhX#0a3Tw#x<|{>Af5yDtEjIXMUGInJV7>jL>}jeo4a$P;quZZ(25T-q z7dOAO(mN^+4mvooN}8i;i}UwK%-HpCvNH>EhW!lC?~8Hzt&1ujFz)2Y&KY{QuU{pF zmBY&`C2ILFS&!q~J$Wlu+eFuP2V46K4aV776-7##QEgJXMz^ie6JHe-%!rtO#O+*=P`-?SQ*Eb%4^0rlw|1ef{#% zQeapZR;6Z~-?vS@!^0d{3FYqLvADH$@0;zowqA_g!v|{W>UlLaZ(&3;HMhdb#`Zu@ z&ziIUOhZl%fr5g9j)_T1Wq3ufwRd@Ch8@U=joJ*pX>s2RF_>DkTMHb3P&qBhB41;7 zQGq6>riNo?W_G=bFJDtnPDwfcRpXc;oYk47j>4R|cH)aelFNF!cX;}~u=?0t7mc+b zMWaK5!)F1F=NRwv72W^pXqjCyj?M2JQ2l(y zm0)MOQbd$nJE5#`N~FHcvJxuzCesdeVhsap(AFiZ$nfKj`Nf39L?`$pSPSTAYd3eI zd8SW}>hwHlot*iXGoRP|ok8P1l|%ao3NthJse5w~oeidoh_-G~h3uon9u&-Dl{?m} zpIjVDpN{%Qy*Tq9b>gT*=ib~p^-NA9@xgCr&sWr_#N=BA)-g3i6%gy{`YzQc$Dh?X4WJ^pcj2t_NuU zGcz}XTI9fWqhF+Vc79_|2kyuuSUpqg`~>cfo`HdeCsuux_89|y)f~_Zl}+9tBWs6* z@($e~-CFh9Gca>rX58A7g>5}mpIP9b^aSwI$RPltMoe;Xb17O`N6hITS3!XdjcDP$ zs1Gr{F*>#a6+R~@CYf35RC5zmK9h5bhGfZvmM;+Oceqybv#wBS&!nE5u zRYScShiISQ&W9bnqmu%U#6yhl2RLemWFab|DTH_Ls^1pm%m{2{2FZhZtV~qvTAA%_ zWZq@piC`W=j(AsI+Us~FKThWHOw|i~da0?oMeJ5{XGhokCs8);UP+ff#u3VEcYo20 z_YPR_^4wwN7R*Q=@2e(MzpvKTm+zh7>Ho?}ZLHL&VRrjyOWcEuJazVK=y~$k&7OmK z*5Z2_HazBP1rDVMV}UC$Wtj~1D5kJ{cW5`!(`)aDqEzea@0{!!H3&#ePw!1$A>)b+ zEdG-C!VB)o2(=z~X)G%W4Y?*bc3`NA8x{F7UD;gMHm_#=w=ji<4Iagp2v{^*>2*>s zA_wBd_3PU{0MzXxSG+h#7e+v6MO&nPIL6LWIaZ%DC z!&gzI5w#63D*CfI-d87+T=MbJYdeJ|kE7pJ{DWO=I-Jc-CQos`bhL(?TGNb|9;E7L z(C_~3NcV%9p%B8!#kB$^@;t!EgHQOKRafkPMWdL7jctaMC;dYNDq`{7$JuqR6R(7a z*#8!k4P#{@c}{P#oZPPRaUS?MGE4JVTmaj&nDs7C)4hdWOqblPw|O^PHt;69g{wa3 zCL_9K)axhYqdi|^te1xDB)lYwr(?2{ebhRs{faihPMYQ9!+N?;sIxbj`dH#R&4BzN zAijp)**PogyfkGl&r?!T8U{3m>-JPjohimx@s>vZ&!+2HJo7SvgCBQqNzjo|*eTII z!2Xj;B|CQa=j)j33tOAtUkFR4pR}}**Hw*taru(=5zAOMsOw{4X2e7Hny>%{(X@>j z`7q@W<)*hDYZzN=OTl`{m;4l+($*}T4{t5K4bee+>FW@mQc$rIA2Mts;Tg77VYpCF za9bgLbIQxavX$>v&vP2qf??_QpVW+-(Y@;dn8#i0+cA~L>88WGN+tXB13NA(Dyx`nA8VYW= zj=ZOiQ(~4pEW+KFRaV5|b<7}NHahajH2lSRuGI**vPXGXrVNzI*P2NJZ6}+1M&78; z&8>Ed`|-bvO13EDP($giOx?gOYd;>5QlY95M=uM03Nt=kXKp zz?*s3XDm5$by8sFfbn0`Qs=S}2+0!+L@V7;S>=hw3`>sLz~<^iD3+tMOU<1#BJJ_X zwGRd-2!hR7DYT#CJtWz(ECnd2MRT(XgDuZ;%u4;8x$9|!?3HW`c*ZX{k|jmBxVAqW zp$wX;zxT~*&hWicT;%zHvz+tOhTieP#c1$Fc29qALBh$vR`qpbB8uU>KMRX_{qvM- zSYL7NG8vx$L-b{`ITl-5@g0;Ks$g%&yLAO3yqEv*M=u^Dm>y z%hQvThNjhT`S9xlN*doXFs!jc_}DUmsVU}88_36UpPE6 zXw8`%tuY($qYYz(2ew`k2y3A^IgIew?gAf(y0&(FgCc2geJ||Vem>Fv^WUH9lc9I- z-nmBKCFYojxWNW10Yv>iJAV#o{AIv(E|F&^cC*JX=SAwuD2Y$~3&Z2r3X`Wk?|V)A zvWo1!Bz0sr#iUnJ`|M{tt0WP2MNMf~T%XxN59^(9;&_c{z_a~9^ekeY`5|5O%9AYN zLOio*8c`zFV~iEg(Q?BQ<@0LK+m6b}e!|ky62_#IY>G^DL47h|MKmR|4mNk)q$TK9 zD4+MA7A3J^Q9Y+&d0-pJdQ$_A3ISyg)7D1^{j=lRyJS-YoNCZ2@u{b2V0j|{)9<%N z*2p&X5)Y2!-FHTHKg2mTPm_E$oisd+N~_L(l{1F^s)_T-;AuvxSU>L={1H-2H^JeL zS${qHd+BT*iT4K{OG+PQ5>=k z0!QO6UCxu@sX5NV`nmeu4zTr3@AK&55VjpdlngaP#?;8c&&(Ia*lmZk+*8=| z^^W`KSy(K6*Yq@w=tD}Amo6U$dIxh03*YAa6#lgNRHIu}rE6(tL(NI;PR!;cw`k|M z-Nv@0OZ4XW1$6XJ7SMhfRe4Is;cvnZwYGUOdZyykGX53+bHUT8I^30&-%cz7>5l@& zpBt9NzUYUeNkY6!h)!Er(s`4kFn89}=-T>Z-#@d0oSgXesnL~{RZ9`${bG}dN5Z|n z6ZUfzYn=(rTBPFu5+A= z=Z6fZqbsueaZwWJ^2F$it~0?DDB`m`aAs+4BLdKe3dP9!pF)&st9FvX|Bqgp>}Wm4ccKLcV7pbz3&zp+r>2A)pPxs2eB*U&oaNi zKXf-oOO)YS3{j0a^}I)x+ziL)@0XcqMYzlJ+`qp(q~pKJJ!+apy&=PHDW{5dDPfr1 zWj|$?YetaPpo3c$|{wOSD=c2&N&CP|iQ|)t1)B5)IT(IfB z+>OdjNG$TXO`9b3**kWEJ z&O}Dto1(8Ef=~6olU>iz+Agik&iSEKJg3y>3_of76B2Zpj~P0rd*0Z@lwLpO?PQHN z*AC`VbJ-#@bHG>DnQHWseQk@Ckd@V?c09{h|dpwnatZz4Ni!x4)LAr>A%K^jun94XLgcm64G-74blAQq?Uqc}Z;6`0c=h zJz2|u<<*_X9FICQX{Y1bD;oAHiRefoBHAr%{Map3=$dh56>*Jf=PL_q;T zKmid%q*PiO1VrhO?vn2AERq{k49J<-$A8YtEUY z=BVx}BtT`c;LnW`ADuSzI!@K9?dLAh0Zpk}!%cE+OJAG%+jg@%8n zvk+1;a$xAD&`Hg}y|CqEds27OK(p-a;DTyjL3MNTRB87pg5zQS?iBgR!pYAdag|el za<0ep*+y%2M9zLh`3~`MM`mZuXJ_J)KgNx>{#dC##`X2JqhNMWpxsLMVXt`pj?iYw>70wj< z%}jHT|EfB9%GH*EeT4Euyvbq$pWj29OoQvwL|a#9uRhEP zXGn3P^oz{yi;PXwp~EtsctC0?^~%{*n=L`5!I=Bw_Ak@daaOjr8R=KYTsOv!YBEUA z?w;fn7c-(`V8Ei(u=@8nh>D0Pidg9{&=E_s!rtuPg^pW)d!j-IHT>BnwgeR(Opq)_ z{Ya97khE9(0_nj&=NP_1Ytxr+LoOatx?mcOj9Z0%pg$z#_{5L>X@GC&>%XMmcQG(B zL?fw&a%AvQIdv>hiT%6S@!zL*z=nhcZFkCk2X$inK180lFy*ruXy_#l-rAs+_~jT|OfBnl0M z0IrPi=zJWU+})Ym!Z4!;Kh}F2^P}t5cf=)f`ZOFb)c{LO}F~W#)my9K`+$uT+ zX0cpo16i5)%*#(U0e9XspaX z9jQcG#g*SjXTeI5cgo2}&S-ME4k8b}9Qk40#m_wKZtqsr)DqQBVn%X@7T;;_H*VXVRX2sK0J#I-_+DE20nn2m%ADY) zs1+3zP4jR!mEDC2(r#nuPcP22v&Uxj!q%1qSfQc$&M+G<6|DPwfd@8way z>T74B#x!eeNW77UU%aAS@qU_AZMFUyO9&;J#@TjZNQKM4qIJ5bhNZjmi+j2GIA&7s zMBK-#l_Yh^t~u|g8fMc@LE>V{&?%7oo?Q5+$*bL9y2D@NXz^Jyd$A?AOi#0KMOBE3 zOQpRZjdbp^wd*#!rOzKdgUQ0H!}Wp3)%ACfJVz8x8=h+MTi^BhXqc8Zl9vU`y2m(i z|KIuKfAEij@A{M+B00&~*}s8P*?V_)w}DUe!KVZm2p1O@eKm`+tE&kk|9on=>FKh( z*Id7prIO*IWLNO*+0WbBoc$Qv8(aW~sFdj3$@k=GoX}D%b&m1A>PlHg%;z|GfGOomKK49#YNF-Ldl5{UNg@kT#EueR9m))-aE_Pruw67 zPHEUGK`!Rx5#l6Katz+Gn93M-y%ffm4{RMNLVSIFIh}T&eUzkni2u7?Trw(_#l>%P zeO4w6_2EO~(=!o)B$+Xi%>L?_(57k3HTz)AZ_~=1bnj`iN9x=^?O=yVbmQTM!(&?YR zv}}J*mB0nD_|DF4160T1#YHk45@)aJTE_!O+z5#K>6tDZ_LP{{jnL50@Tv0>4lIH{ zIH8`-kd>7+6Gsvd6db77?Ah+vG2}|Zk{s-334w>Z3>Lh(ezp@12nx!8MIOgki=pzh z4(5}xTmU-WzJ0scU>j9Tk_z>3-G2|;v^6KMbLPG9`h$u1?=-nX`EnRuIkAz1m@Y&k z7|r8dPwriOZdNAEm(fntxf=5|GQbsSuD)KyllkYE(<1l_42Sw)Z59{}@%SvC; zzG|*2#YRF46OhR^xbIe1|AO|Ffm(~l)U>*@(?`8CVeXqxi?qN9i9ZZk! zZ7ehVbeU6-$ZXZm{yEf=*)F3lx4M{gOomvE4b+5cg^3J#&YfR8Y|l7P)0189mPANe8Wv-N424#%*J%dxTSOt zeMcyya_{{*Yn6CJ8pjr5&r(Q^hm-q}QH14QX>x;IWSk5NrrUmGKsMSFV~32%w;z7B zr9-)%`u+BUx6mRN*Q7mgJ-p{l0ym*9hb{ruWkF4eLI5$tl~&My;aO!eQ^#pxVFAZC z7H+3K*iEF%1}Nup+iTyMo>1eG&{@q&Ba#PC9F7wuHW^uy)cLS zISWii@i z>;OzW6UDXuh8G|u(FO=dy!XgOoyWlk*wVWGU?fXDLUZurW}&-$;gY1a^*s&lPhY=j zb!46jyqhflUj6;kd*Lv=naW2-hErdP zJf%~1GkKse+0cjQtD62u{d8IsYi)Apwu)4DJT+n!EjUwjK<4&ry`NPJ3F}u*w&ptr zQ>0R4(N!-SyBDfWt)<#Ty|+f6F*>Csdcf}Z3h5^Jwu*Z_>LZWf#ZvOox3}i==Cdam zE?A+dvb_j8dzUT-0C}OhTwHB+)VeNl&J!c|@&;-#{+JTV@gf70#JZfn1qI3XU1^8% zsOF8LPWxWrrPOVGq2L$z?)R3@^cLTK@H?SOp{no%!?7K%tZ*xZ8+Z)>7Q}&7*g@$8rJ`%VVb2S%roCpV6DGuOgm{}(1L=>w#CF; zn&LgX%E+r_k*-^erDYkHQWfi5Toy_rYJK=`Isq!j?8AhQtU4`sVZRLC#z;ZXAH7mK zGHX5r?g4T4?;|MSo{qJ=wilAuFY(zluBE9M}O$@GG3sA zM7P}Wh2wXin1HNo`-x!<_x7{8_lBXwTwb)_q~57L^rPcG*?k+ z{<7NRr|Fh`^hoM`ohzy?RFXs^zs*XDU2kwMuiRd~q9-eMQJ9Ps4oi7fl!JS3ZG0x}zh@Pi2^gSQOKA~>_}9&~Yo2_r z&QAF4`9g=YeSJuIL2ZOtEJq<9AJupMKQ(s}KC>?O0)0}H3bVEnb@RJ6jd%MweYd=9 zHuTFEXoxvXBv~Fk#7|3fdP-+5_tW}A6lPY}T19sPdl)3Zyat<^rpr1di{$%JGu~=; ze$E>-!-&=E0_*63HV{}@F)kWl;%SU_oYM=MLXh~FZnwK%%Zy18q zxj4dpmA(NvbdAXk(XPK74yyBM79*QlA7nTmWLRyLze`OK*skJm2KTS)b5m|%djOQ|Y1}W49|@9sHUSb+ zW4~E?)9BGWiOQVgX{oNRUTwE7`*SgrBrMb1%9c1-h;3);{qzLu3)Z+VW1i5BP}kTM z)2k#!C5l;C@P=%T(tZ1m1`JYbbW}df^317Hjt>R(p~TvmlLqn^t#+S@mm5voRf%DW z>qZpP%7P!v=h(!d-+>NSua7aHVL=l48Q(UJVSTtg(8^}3Im{9&(AJ+ND5sv3G+wHU z-9yCwjGZAxcc28i0{%y{VrD^OQlp{sTNAF~igfW~fsbqs_Q#)=e2a*{)l>Opo{}$! z4CRz9goyRl9C%tPY$^WV`0w*h-aY1>KDJj2nr%7seVrev9Plp;*02XSvo zF=@%}c=C7#WpTjbevM4{49qTB4+=6OO&~Wn7YURb!NiQW==3S$Uy|gma1<}4XPFDt zm)>RxI+`dv>aWlY{FBLg(6*7Ylqk2yzKn0>=qRk9;NxQw>5}R8wY6jror?r&=WFd) z9Bd`iTZ?vyoqhs-X*>mgTm`+uVxsm}ID{h#bAj9}vf@d(G1D1BBBu`xk0>m?sBT8t zK3rNd-3pK;oLOC2@^mZh9P!1c;xmxErg`-vsJK5jI+;N!mXNtaM7yLdLaXPEO@>oy zs>mWh4M(P(>Y{^A8~mF;DX9RW4QRh08Y3te3%uo_OQ+MnHlfd?EH7ddnxxxa}PT`Sa`Tb9V{ zh%7lwcs_?>6yJ7E$GtY3$ph;hfxM8FRu*8rc?E4T?BoG5lfWe@)K@r6jy#oUFF+J? zZE%{+KwE|>p-Pz1ZVrVoWMo!>$ZDh8Iqy`ldAcVWvQPeIz9-Dgq{1Q&NSAX`yOpn? zznbie>As7MMH-Z4-(^s_)bK}d#YzveN%9F20)I_RN}}1Cxk-j`|8`RroF2%bfR9EO zFxmf3WW{)8c=@B}l7AT?lgsM}iW0{iO<7FrbkkibRqZ!1C_KuH zo>;4WDZ)jgWyXV4SfmYayIh{W2y9lftXFU-nCK(F!y6id;~i6qsS)It^D;kNkGykT zsmuQP82)H5FNn=vYZmNY*w8q+x*{gT$Vj}Bl9I~pIL`k-{@)Xlm+Yby?^~9i_4R+@ z&E8!?GL6s}oNJdf&?H^-5*!$FoNAu4Z~=%aOA?PH#8^Hv*7*SBOhlNZ&fANYwOD)@BxsA_x<;~ZD z5)KXyMmuRO7D?UZnw~fP85s2Ry}-wp5zym|WKqUvmFV^%`(r=$YD%TvXnC1;rZj(e zCT?dzYVu@IM@ISdst@zYYb5IIQp%cz1-!5NabmnPKhzuI=CQE_A}E0W7jG|S8!CL< z@e78-zr0avk%o6MbNoNoKKXc+JpOYPDQWU^F~n%tago5T@a*s9KGX zB{rr+N4h=#G;44jk@yIq!c~qh{kB?6-I1L}X{d~y|Xb9(~^xlQji z4j$h7(L#-Fl89&WISnqqD3nP3)o&!rn>lDdqk3^pR!YsDJutZ>V98$ETw|C zPnk)yc@qD4po-SH;ULEk1}E6_rDdWs^p?DklN+3*{hm_&v za+y+Rj2irVODsPs#XSd7xwXN<%dMM|GO^eTAMz9i_Pwp&LLTG#D+SpkPV1T9kRXTV zE?3gOk?AE@`7(nZXbSYy}y9!QExa6y78_ z%*18ms38}iTxM<#I9PBFl)HBQHW#B{3V4`!J?$vHS;K`l2q3K<`*lamS2`HHs3 z?}*KP3hq3t#U@TO{DzhI#x|LGm0?f1VFqR%8CluR$}d+=)j#)!beFjYG(<(0jfU<` zX=pr#(u?#%wILeq;<)ilXu;#)c-n5$$Q>K!k!{4Bfh1COVpQ;e(HwLRpUO=__Z-Eo z-OsjVlOOXCf1B;;4HFl6&dxwOD>g$M8oUx9O!|(x3zma;gOKHJo<=0}2}8LGzDq;- z9Nb*8Jqfc9-7c3r$8Tla! zS}=Ng`Wq*<&}UOYORws~QT)j`&k+p+tHn$R4{nXYLE0e^IXC_-t9^lonXX*E6nJU9 zt2QP$NK$t6l_3NV^~~@808N3~lXs%y5+1p-1WTxc1Em?#I@B3Zsv}(SDD2%C2aiBLP>cmvR)MSHFS+>ckqI7iign|W(7xvYgMo|xhDvdBOWil(vx4_~ zF~0cc<%4q5N(Rdg%nX>!6K$+<^Ly*9ZpZQDw)oNe0-h*e4Ii2)4#UIV1I=fS`p?%H zLLqs@7w!yDi@+@HBmC6)(`t%ma&U0)#fr)<1d%mCtwoTC-u_WG?swA*j15%o*X9!}la=7=-X9Pq9mqEoMk>t8 z#FJ-!09q>C$G?5uUFR0J1JYk?O}U1P%_JXg4`)tYEdNMUXWJPI3JZh%q=!Ng6b}D* zL}rtdtQl234Fy^g(^VEx;ijgi)&#?|^q=;i=5!rd*&^*%tb@gxB(E-_?1NN(THr}x zx!x82T&hzE1Z_uWFaLXTAZwE~xR7?Y!)O4G@FzkZt|mo!&L@I^eKj@PgBx8pNj4hyCH~j_TpLGPG#78m z*_xsWX;hvgWF$dK{1r3fP#YWc#T1gDEdHY2p1b>BggN)#7>WH4e2LCE!Y^1>Ssf|f#ZLs(Z=P3vevTP{w3X8~* zlw$G^QxXkDy<`P+OEUP0*;3V{EvuaFjx`3)yQA)_$|#jS)o)0rmxg$F!h{2VcCS#G z`*_V7jf$x@vqK#ky9(U)Z>)YapYs?P@TkSmihhs_;LV;}Lh|;&Ju>O|F=L$ODa`Y) zaOA*q+AvMfJ#M(R`we%?mxLfa?omB|KnqqFSob~>p!h$DyrIL7J$AmwoMJz>E9%$& zlyPoDZ=3OfdQmvy5F!x7#6;qW3Yu9e)l*aGUCFt*6S}@J4AweYB9^KH*s4a%?GOxd z^HPHzBCtT2wIVh5Bqk;rjTVxC%c;!V6*Bk{TVa4{F4fhmnvJGWQm!<696^DDi>m{T z0Xl43)*VO}tQ=o0c*h+cAJOfNav}3D`n_;Qt^%`HD_=pnWl`a&;y%9utHT!Om#S~n z!+Gb?;vH(V=rkT@(XXOu2C9S&;W9=GBhD6#^skJ&;r#hC_7Mi_tvqWwp2&RT$1IkA z!(}4_G%m1krF%#-2Ht`;xzgHkn@6C4$h-RI&xuNn`j`)(XZZQ^48*pB>7wNF3d@s} zG^su0HXhOd`OtjcAB-KmABycz}znbbcn+8WfFgAIwT9HLGw z>I3*S4xOsS%%%*}Ow~w!%`8XWOk&Xx^JeaD8hG46g=Gjf4C=NDjBc6pcpTeNK@YXC=7ON!PID zc}kBPm_Cpi(;b!|@#b$=BTT%K0PL-em5>8H4R96NB{Mwu2=9I{F=qszE@iM1Ec7Xl z$xiXBA<(?+9UWx?B2ssCJPB5;^}@3T3v3t?^#0&Nu(!qLt^Q)|_f!(GjX>XQ6Hjj+ zLDDD^qw~IAqUY#y%~MkZNT#6hA*Hgic4s^jrKoNny{P_}D7!bA`iG>rV&SAjSnu{; zk?dnZneh@s&;p>QN9u2&4b!vFG*fo%703qwC9A_o$U$W9W1^4`#aN%);AZ}XDy!)` z3r*yb>^?+O3quu~`J`l3=r)HVS!mn+jfv1OP_2UIzCewo#2koQvg2;fz`!8mfpQ%F z3e!|$qRpzU?=rt@SaOs;ZzZbhd`Qkz{Af?#qLHN`Vmf)|e0GLV=YzgYXI8t?e7vQS zE@4|%ogoGWu{7e0fsWnz-!c#VpciocO#VNBta-6+I~HtLYxcQ=fmpOHU?cm$q9l@9 zYC<(?XXNNQ{D|0RYkG?Kgoj68&>|bQ*4%EKmc2a+6lb1dK_@Kr0UH+Dv#Pg?pF<2ZiM5wb zXhJT_|45`bxWt$$DckEAI8PC2Bl(Zo)A~^)zd95-mEXu(2`a7K)+Wn>%!AVxbR7P1ei=6-`yt5)PLkF;I2YJ*%&A0a1!}jfPL)|go(usPd{B;I ztPiUgWt3}dc;d5gq=!kBgh?#>k`Xc!I5YXx30DLyxJW+Fru4SPN&v3_{A!CN@PBkJ zk7${gz5*}^S_8T456ch_T#7&p`U1kjvO&4;Re_WWMb{w|RE3(x#_OQGEPAj+lpzxu zhI-_9ab&R;?-^ioOh?hc`*HVO{UJ^W(e3G|?-z%WQdXGPn85E2wCjld1Q@cg90CM^PJA3LR=2p@yAe|~PSo2!kgB52)Lf9)sVCk_R zoT<>J8XMiq;->B+;^I#>*73$9zK_QfjoAJG#$+KTs?bn#CG8F;ja_IHV_Nh%u`4Vr26f7qR>=m`!WiTTBpKyZirCXZ(PAieT$5Z{u4svJMwr=v(n?r1unFE*w_TK z%r~+ryzbx+nui|OiQR9F~{(6OCdm@H%h*eT| z^550kAs6b=wx92Sh+elSG$}Cq3_7T<9GP9_%eVcW4w**78YKI{3dsg7Ey8%JvWUQd%I7gtQKs%GkM82=U0BQRSpz#i@z zPe)vRCO0M>!B6_|InShpl47qXjAm^#pN6NVoSKn?k9b7(brjqF}KSaI$l9T>hTC2W)5 zcM$ejr9GM=*|q#-XYR|7FN>4NM~Qt4uM?DyE8o+Im0Q?h36PeeA9czOee&a&QI0%m zZar0V#;^zio_Jw%vkMHdpk_|jLgVo-eZ$3hM)b${6#=foxdWqgMQMw-kzX)vG(e!r zzlJ4(P`NgS!$oVdZ1&Tezwqjdx&RXUEjC=g^AUb5-R?FQ-s02O`^%lO$s{_@C^Z&s zN;~_LM-CXMa?N=3($v0$wr)y5twdHbG*$B{-z561^N48V?;ea+41DumzeDDWR*gV) z*j+w=rMGHK%xRovLK)B(6DPv4$7xx&huUztj{zrCw>Oa!lZYi}!SayD1oj&NYYCgt zpF$G!OM~uM?VWH7P6Y+TIfS~gIIN_l4_T%FYfW(fs1>rZVlnX#cTTFt%n z$9#+kNtJon+KZ!@2w*Ov$%SqfQF^yXdNly5{R*5nf{+B;9b-bz_wV|9Eo)45*Arcd z4ORj636($qu!n-)MInf)x;l!k#A?6W8AhYMvESG_jB5eB&ZxdKZzKJW%-fY|(GwiE zvb23`wk90EjL^j|{99=oKllbO%j#I?4Jfk4L#0H-R{BefTHTGMwA7@q9y z9qV*>&2VrK4zP^5?h@wZ4bK6ZRrR4YRdRLa~Gj4VR0Z+bsZ(U0A37 zj3eEX5=6l87cNSmPWGo<5wlwkoC{CI{S#zj*KPu*HZ4_wi#Lr5E$4DCbqRTqmr!5XY%1d*lvs#N$Kc3hrYhjk;CVsqs?5TNOWx)u{&Y; z{vUFUEru_J$1S)p*SFX{QNWX`vSme`-iZ1wwrh+(f#v%V8> zTKmCYDmO1F_u|_cdr|Y0zJB2>Dz*rVQ&}WAvoG>HXVr^GU4KD4GUn3Xoa3!8dJSaL z>I@xOztlp>FMUDeJaoXX)=3Ht%|H1KU=x+RusZ$ToKauUu2GX-3cqLG9kcslpyl`U z{ZpkxBgM*{3mZEk#i{J(7F88~FiC6<|S9?nUd4_39Jc-vm9?Z6{Eoa_9 zqxBqJ^9Idyt%qFdXax+V#(7YV35Kt%+e3HVGXIfycN%u7giQo(ZO(JLPXt{ zm~H*L2Aol#CB(ek-&8b_FKR>K;Q~cVURH_gjrKkU`q~5&rsN=p9c>_3GsY`Uo=tQ4 z>oTs5d%lRUC0N`ol8yUE@*BC@VVkHxt;zs``(d}n9&E*!Fn6x^ycpL+2wmJ8KQb~} zOmQH=7jCj@^grxC&RuTDe@lyDe1K6PB8U?b5)c*wS6LYMVSOyLmbSpZy1DQC!e8`} z##xGA49~+7 zukI4vc8=Qh$960Ez+-v&`!Amwm2c_l`x3ojIt*^NaY9lZA)91#zep-CD*fhgzu=Ft zXoSTeNwxh-Z<08$aHjx?k2iQWm|rl|(keeBAMfuhoPLx zKjPKW)`6RP>84_n2vpQ*O4XrcJe@`4bsk}U_(%Gzd77OpAKV2Q~q81N1pqpUFs6Dxr!RjVu{K?8>{)dF2O5Bimxc4na+ zyZ9_7_zs}i{M69E2N!%TXG7{keOu9_1Jl0D=3jbHro0GzmvL_26_Swft?y`4MLkfe5tLURzoJ2w5NX&e zlPRX}Msqf&)vD^`%ESHn^b-vsCMtBe=*phFm(@BXM?NM4Om!jVlSOI{5pmIyv&kAr zrqn)p9v&@;Z{B~y(gsbzuabvvG4hm-z%K~)V=@|=pvK0xzb^O1;YhWE)NFe~L9l3) zysLB##IEn05xJ2_plT`ozYBlL#M;{WaB~#f3xVaq|1!z`jKBIb36D!umSh1ok*=<; z@lej!By!wOljN|-8(jOEMOBo;bf3@DAGM8#enojIXt9eQGhJ||i{mAqOK#I+t$h3` z4dWW0)fz{20)Ns&W0TV8C5?6Oq#!^o1FVVn{l#8TYrUJ@Ka4ieQK&#vrb4fcXzi_n|G%sE(o4e zt`IaT3IX1u!`I~&X6`)I`8B$zZyZS~q`|}5^x~^h z&flU8Z*0#O%j0Py3*NT(Ue$)GIJtcB23?nItJ*?{HP&()$G6nKHD93tkdz?ORJ1F0 zsPgO7d|luC!`hr%FRjw@mA&cN+|H(ijm>Kil_R|YM84c~G!$G)v*!uUr01~iY#h>a zzyl1Il9HCYKHTyFa4mmLPVR-a-|6aT)cEiBKeNe$mF8;rw91Vz^jp6F%_0>3RcZQO1BjtcC9*;^Q6@CeHvs1m&G*c(Do>Jf&06aE*ZRYT%{Kkrc zfIiW1IdFM7CduQVhkc#ficuAwYp@YNMR`+x9#si8uRCm^VVxzptTDAwxMWVd}$P+&u` zT7X)LxW}!exLpok0JwoN2S|Y3=15O3&b{XCG-DzN(^sK){r)4L2YGqzx6vH}3$w4` zufTe>uSV-%3jrU8q|mSgsI)|x?V-=@4p5Q&51=ih(&L9A__d&QR@UV}oZ`8NWQ z0Lc8c_CI_4x|J_gwXFF?FQoJ1gEi6c?{ig@{_4f{WZYZO#d4|$SOXu2k0dPkCKT$E+teLaNXXF<(T?% z?O1gf^oR=8VWi9~D9(_SZGW;#ctIfqq+6i*rV|(=&3I6DL$1nZ+1%F^Q|x?TFzM0; zhvj+pha{c(W1059maPZ3kUsg zwiM|@!Lu&g9V)Qr0!i)*Yn1{h$t}Elx8GSgH~^{3)1Azdprq-1$7^8$H^%McK0+M> z-|Av}rudn2PfORG@FAo?Dx1d4J`yF!zy;>tcDd0JO7_RQETLM?!&-tV6|iR60N22n;G? zR+24@_-ai8?iqBj(cqVa{Y#pd*J>pYjo@&26O*jLjY!Gla@gvCtugDv`2iBTS8c2SVzNuMC5yUyNg^S*@_GT@S&>@zKX(vRx@lYo7~` zq+WbJCvz?rurd8)b1gLKC@Pe11iREd zG=R0H2lf;zAPPj-QTWW+@MhG71Ma!ici|R-xsWygkGhC#?gIJzysBa%Yzn6YBjT3~ z6!l+;jbmS@$M-j7k4ZT?Qu%$&bOGiJ06P8w4h~riLrp8BSBXMBxjF0eM3AUp&zRGw zeb@;0@%7mTzN?E9L!d9rX6nwglK6DnFri)=B&>TBO+xA;j9M$;+&rJWQPQZf(*`&j z`~v~d#oZcNMh3G0Vmqa!aP^~qP_n6Cz6|K&z3BeOT)jgH5;iI>u6Urm_lM-9=2un% zL6qlIB;_h&G=Trd18;*T7=lI!KpvRkyy$;=rudwWJd);5 z?ob|1qhIxH$O3qLy$jhL>d0d-_&3qT@6Mx5vIZXHhF>ZrMPq#Ip(!Xj zkYFdeW5}D1XzpVPk6KZQ9U2uc`Ifu}v_2!N5PpZhy6YR0HhP$)MxG-hjUyvQu2HtR zSORNTO}9IyLpeA*b~|-rQ$5yl3ec@Q-Y{_F1Bo^iM0nOnnrRO~-`wQJFr+%^nG-A+B0a$M1d-vLU?N zhoCmiQXnTuz*Z)lslD7s?qW;GRZv(_%>pg#$SsthTZ0WR^p#1;Sugcr{k*7E{>bT_ zO6}1?lyWnfkn*mOkh75sW zg3xPus~aiYp~WxoX+?UzwfENb&eT-0`S1Pjq+QCY0pd~!RAIkKLZl21lg;M<`D zgcjt&{Tag8A5wVDxmTw2qrtR1-(&AqGSvc%QmG9qF$sl>-?x;r6e|UNt#Uu_Buqf% zvs`m#CM4(hQH02WSmLJ^w=>RIT`e#m8jay}7uBBnsQc5p(7@!eO}6s#6y zD;p$R#oN%{=(HD3A9}c9r1YeS*H1h}g8kNbxVY~i4+8+r#@X41TRU@C1!B>!U}Fn4 zLU@Oug#cgrlAiwD_484Y-PYJwL=Y2M^D%c77qGKjRw*adckU~ z4I;nuj>tvcGjV@PrW-xbl5FcKb8P*6rDTPGE>H6hd~g)nB1LF`iRhfq+KU^V^> zvQFgm4A>13+G>J2Om;;*zF^82G zSoV|Mk5!=8?rQQygDEF>P3<&n{m2E8Yw#)I&Ye5+i;KTOTmJIJi(=>{|BK6C@*zG* z2*d+}WGt&G0$uuaAq0auxXS0?0fk$>x(Fsv|B(W<*~>WMZzoE&sc`K#C`)fCX}Z z@?HEJJ>{!cfgsf6vRiAcb3PD}c&T@RgQ1K1xc?91H4gbjD2Sry5Hv@qV$+BYio!QfDAA-EY;nEuN`zlIV&`Bt+j6at?3n1 zA2ekNN7@1Fyu7K92x0K3lMoxaEfh@H3smD{76XjF5EIeJXOAH&3q)Cf8%p9-hJSp1OtO;Z!Gk; z>1Q=3@^EG*x!i1o(2$y1ZHihdS+Duj`t;%qKpZR(Vo32EJ%k@pyn2Q1=y)>e$$W!R zugSoL1ibqytgzts@Zbfk4qOcXduXviW(bfJQ1lD)^Gy+y;{Re}KKWP`al0PV0_NE^ zJ)Hyu==I`A4gAo`+8UN5e51J~oi5~)^$ez;qwXgox9eZxv0ew2ey@|WKO})>(c zFDk9`a$B_9L|Xn4?M5s1*C)MAHfDseg_Fmd|4&h3(kX_tq}u02NxFTmc}Qm3ZzGke zc@BcThu;$O_9lonRU3PIdmlLozf$bUdmVQCdTwQ<4LHC!5M3Xv_8A^;jU)U=g0pT? zK?e7YBGL0PI663)vCK%cf4CmiW@qx{*_80PrKmY~95kMgHI=Z?lm1c2pu@^xF%~dQQ zf>a4GdUyf)x!*w^c9~&&b$J=fY=~9wa#ZtjU>2GIuu*pdU`6xHn= zLDzTV_}2!mZN%=QH=sauJ73Q>o34q5KQI7FmHqwwYY!xDxIT#?1FX8bx)^qH$r#x^ zxepzMTy~$;))*5JtBHWI_S3vPT(HeU)X3fY_cKB2S9kHR)@V>++YjvBHK?+ezf4y&C z`%Cr^oXt>tdAO@q=LTM=vY3B@-+Ha^3T32k2v!ZGNlsQu#J!1wLlyD$D?cP({ps$8 zm8L6ULIObf?*^i6i3Efd!GzIxgbDxJ5E)kers;RZfUpa& zgkiU?z7eUqynT>3S%%tvTsL5zet+mz@wcVyeQ-60nA^|4Enms`_#yPeb&Uy9>;jk>&P84d+^dQ{?8`NsE&62S|$u7hqpjl{>i`J`SWpQ<+Z-= ze3zHmCm#UoaGR+GQDNKiK1 z)HH9l9l-lKQm7#gW9mAoM-rJY8UzHi!V|W4aF79ck?IYaBRpihg8HD`v<$YL`=)^p zdQaxXd|ZT3Kd%Xl)e=M&Ev=%@i{9!s+2BQzGMBm~#Nl8QTCv15cv$@M&pO~L8GU9+ZM}lfa z$;vw1ixUk=d6|~8zgSF1FumwCYOIbn6o}X^Pg>uAjTt)sU7*%&;X${#fJC2n;3A1* z)NZB=n4Cd$WFWh*XRw#=8ueoyZie(@IQlKLD6yy(nP4woR`IBoURnW zDcI$6Lu;@lV)`wtx=U1Vbp_Hx&&wKhWWdejPum5ElH}?!8Ni^$)@rxONl8A{CeQ>Z z$-bbc%QX0C4}K7;as>=fC8aqpqCz(5xK3wpZx&n{h+v|n)zyRD_b3P;z-YRLl}7PL z^f82Mc;zAi;VT)LJcz7d#Mu_REKbLHNdr5h_(3B>O_D5)+8XK!py$GnC>=q`IEv z%}edX%2ku^3QTY<-4&YK<%oYKrtf#rV=(Yo2t*(RO};yl60V)zLzs`s9-Ao8r0~9Y zHQ=Oe!K=EV2L9>&8l^#GKO|$>2W0W3(Fzz##=!yPDpY~~X!KwSWbbCJ?+XKSn5 z8TAt8@CQMyvKF(@v!B75;F|{RNexZSr?6%PaV5hjUDjNPhmS7=KLd;&D3CC4B0PcS z6Z|7)Cl^lJ6Re1!a`4(A0^Xoly8)ao5^;D!jq&J1jK0U|JQ|=opk*Te1&K53Ki+2# zepND}S`na~PpV-uJ*PR`p8N&^4usro{Ktn_6b#kIhq>1S~U_2(GQ<) zO`@p8fGvMz%TK)AVjv0UB|V#nwt^8dwqQsTix40vyId;4bqw4h8wh711# zi`>ggH}HQp!*FEI{P51a$2=5yMa2=Vf8XkUe@$@n%6gUajaT4!I~4Z-Q>3)w?z4Y* z<(uqO86vO2WB2?IDn&iF*p|SHvnyJS$KzFKHgk)qw(ml2i0MuQ#w!v{)3IlRZE_ft z(Z4>a^NTOOTflPPeKSRxzD`QPag%EpMueg-liEry^~oR|{|d9r{LW5TQ4s@tIm@6W ztJ`hHKnQyPOJR0)^PF+ceY{BR`R);Ez+2BaXF1}4FXhtGw=+BdayBNC6i)1oT5lmYWm;HHrnemPQ7(L+rg+nSoKMJZDgpL>* zb(^IQoRK1}J~UFdMu6TW0KR<8X+r`1W$rGAyaMI(=ROEZ4vHQgA)yEi*Bo%=!?4uW z-k#m8`{0^HG-7xESbIXvR=L4~B$Da_1q@!y**ZTXY(iwDLZ5ST#=Y{mwlgDWBS`5H zmidw7TWV{wcT^xC-~G~!M&UUxQ-J@`Mt`}C`()o2l+Wbr6J&|sm*mC7#P|00DZtBl z&R+rXq%YGzg$6j3H+(Nl0Agu6&EIizbBkrMe%>BN+CDluYH_kUEB(+9dTUR+AK#|~ zFzoO8NT=!O=pbqa7*n2caZNy==znLGU3T8fUD#WHrvK*6n=hfESpYizCvZ&s-@ku} ze4fdFBB=+Q2!YLbQsilxD_Dxa42YpOo7B_Nw(b~F~*hns9SAr99MMJ zN6`xFXH$a62jm=4J})x?ET?XVy7+QJn5m;=^%{PO@RD?DDA9-s@NknHCU+o{hu6LD za(G1vG^D`j99=(mMq+Yc;n^)ljn%^l1D3~;osdK!23Ho%<@K1{yLY#sd0&S6vJEUX zKr1uC@tCNx?7Pq+CNr5OXlWzPR4SS|9d=DDkvhy$;W72mQqUe?pn`0hxqq+r+I(;g zc)=)!^n~3)558p(1N!>t1fY?+N3Ta-$Ywi8BK=a7+N%`f+=9Y4>`7;e~c5wRG(g@D%e{xlSTq_ZHifrFd-W|L`fa1fes3Lc&$2uMYE z+Tk&;b~$$MqdsgPxYbt|0a-scg}M7$ib_MmTR^@@WQRl#BIh3YR1uoai79V0#%4S;=B$M9qyz4Txh4;geD#|YX8m0c(=9tMZrrp=< zy02+ce(xsQm7ics7>H4+NoeDa4MF+qF2lV4)E+6-6Gp2ML7w6&`}zAj9?4#fDU3)7 zCDCm$*!0I)U4N{$_nm(l@*KoJl|R4UIA|z#r9bg)MAbiLQ3t;%sUDf}Bs$|D)+Yu< z2eEZM%4=Z_@ys#0rd2mx-buPx04^a>QMVAl;SqMO>}m|m`@Uh$6yzd=?PzBs#2ajm zp4Qh_va;d^>$Gj#wvf2EASg7@RV2Zi1>DWf@amPioo9>edg4;zJnmqREzYF9fl&|g zK|oNDIl4{E^K?#<@gsrq`7ogGPz<{n7#A}_s7dYxtW63$4}V9}n1QkwI0at zBt#??x%*s;m7}+E^YNLs2eHimUNBy_Zhg$Li?sL@m2ckof)^4rd9U&`A-Flqe+M@Fq(blCSHN>p zR8w25Vx3rZB6SBQVFvKUg_(-XOiY~SkJe5_O7T)CkA+!kL*INTh?Aw?aar7mxA>OF zMf%*O3oR{?5EMJ4-aBWpMPH2_{-Zx1lx)g9GMG$#|CD3NZ=`w8zgZe56q<%JF;zn| zuAg%0_qKb{q$$nNGE*nCiMcqna$j*vLi_IyoW+GQ9?EpW&%Zob4$R6ratyA6+mhS| zu&D+fGJ--v#V&>?a}&cKJ+cvpFK3Ketk_|<-xtuUy>K`Z{87B2@3PANxHdWQu)*j5x z{x6k*tQ6g8l&ODYJHP%ZNP$PTv@aPWl2=q#KF2m}Pk&<^?HUB%X`=gprzQZBWY4Kl zoIM?#?MQ9tPO$ypm*?u&#?P;ec8P)^w%lFY+8T*6Rx{h^IBtL}+Hmxp#%uFEKf{be zH|GKk+@;JWel|(~YdTjNCvWd{@NT)`DU}KPTbo$xNkqqA*)B_MiTRsv`}MkNrweA# z80YBs$R1%7#OfG~I_;3WdJG#I$dEspbU|>2DDy4HqGzS%Fsbi`VOj{Bm*i``{v*AB z)HDDC5>4r6c;PJh|1>V%8t|-BP@p5q0Vr6X>E*p&hHq@q{R`WEi2xfeh!+PUV7Z~u z@z$eIE^xngn7D{;xwfJEtg@zBsoyzITI+d>4^Unv+sU?HSba=ePE>N2;f{m)KD+;>zX!XpBm51Y<;1wk-cyytNRl4nvp%O9z0<^D^j^@*Y3)R@~hZ} z4{3#*D*H9+74hGM5+b_OWjgVCx{RA6OZ36>7VkZI(|*{;AcMmv_ypkKJ5Rq+sW!FM&DJ>=6O!a~A6GR0xcL1h;EjUnyFClktCYbHD zO)Ic`I|vIqWA5L-zhqU6RZIZOdNHp12TK`F}sAV z-_{j%fBR156;5ZZxN{~$FV6u4$PD>}P%F^0N~(Z}B%SPN&CVt7ZM$|U;ol(#ltS{J z@9ZO99bG9nO(2=8ureVg#{F!;u`U16H~P)Jf%pb6=}rjB2SD;+R&CjxJloyFBNt;# z04-!8Qj%Jj2q&_~PH@g=?(I3gE1}WNk>>SArMhNIS?-^miyJ)Jdu&<#vqmp*tz9}X zNiVB9OJ6)F!9MUnmwr;V{fc?V7Fs2?nm>&nR3&AOi)=0aag05?O}gV>ejQ!QU&oN) zQeW-j)5X_@Vg|qMs*o3|`cqS6-ncN%s2|v@Jw5%u;68#0ShJaNDlw2Da>V~9xbOa7>5O%B7;*-C@_G3V zel!(CqI?5+JlU*@E_v&w?wfPmXEjx%h z#nD96C)X{)G0*L9rd~;&%o8N)N5HMydP3$tG#J#80k^TGY6}5AYEl);l`W2U{9Y*_ z>l5J?6bcpyXobJP^*jY%Q&ClQ8jBai<)+)=h6`X4lLp_CX;+_k5e?-znJep>&z4$I z6(<=ST(qWje&Mo->12Y{E_!-;;;x7M45{sG(Lk4Y(k3WsE@7yS_5zemEsKXIQ9oA<`gzoFg*s0nfMvcGKAr>3LPbNX3Q8x1omap2^XdoFfn zUHWrGzm0}HcB>Zkprw0C$fcl4XlWfX?Ex=GY~5NDWGA-i$k>Maxe%Ah2JAe$_;$7W zL@fP9(ZhR=xSJIlCi$Lzao;+2%4Koh9iig#pNz6{C+p0C;D3AU?lD$-_m1{``}P{b z=@_i+E&c@)Z$grJPTJtK^>E*jxx1a86a{2ng)ouplG=wC;aHTIxb4iDv4?PXesyTo znl-0Uta{JKEt~CK7y7Syxw= z0<1MLa#u6e>Kg!P#$MFTwKGxVy;G$My2dXL#2V+PcHFUcU3_ZpJRJ8 zrtX>O%3MBZuR%65da{n%FGenwbl%w86^O1%9ouaO0gx5{B34yIpbIX=<<3a<#~2}h z>{k%-u_wBWY#zx=ms$<#4UBKi1bU8MJ#>NKv`&UVc$uCQ>`C$2nupJs6{tmSw zhIs$0#_lb{GP9EpkJ@sflKcJb4WVPES{~fIenq4cO+893wu1-hOO4clT@sIrE*G-! z$S%SZ3s*2NKnF%_N_5rZO+D+}Q#67$+zQi{mKz`R(Y8ILROi;8{8v*=0wJtcA#Bf) ziwrM0SQ8$((Ps*O^jO<)p_k^qLX1`Z~e6@A4;?XE;pX2a$PKr4X9fGMAlqCG(a;|w8=Oq;n5|SHY z_j0>Zqz8Zz(ah=B7nC4b-^Rw4g6p@cZ&}$ZDFUX80&z<~vXx7a+<&jfs5X}xDh7flAF1t)Ljoxn4O{3~v{}zD ze-+!~@skbJzHR6EYyZxWL@i5|b@G#KMiBo{5`8?(Ne;)>!TtMn0x+q}O^6X2i;0Bz|Lo8I#JP~r3Aa3~9 zIV9gx>dgPD&h?9Jl^(n0;L!Auvu1U_2fDS=+YXnvgT-(ArRIyl(trmOEDo6#8hw%{ z^7{9Q#H~*n+`IczZt!D{g$u{Kf8NNdUIW7jnve}fTq#DYJ{rG@$m-y(B)qJ%O9>zPQqA0M=hP|&b!11d$B5%DFnVF^H~%i8N`Hj z>F2p+06?_VsKIaTRtlLBOiw3c!vDD@FT8OY{Ndk5CRD)`TYbq-TYey$$b{9dKuGuT zhyRn!LDpJUd@|7djf30Xh-BaHjn{s9#w}PeW{PX zJ?7NYw)Ki(A!ae)OkGC1^g@f|5inXXSWHTOL;#ALn_JF>H?(n{qoS`W^RM*Ec+LNj zXJ%nphQk*T4o4L=H8DA$y_S=clLE`yMSGFWs4pR?+^-mA$~9FpGu~$ef&s%iy!ljb5tu-utqUH)mwnFN%R? z^`;5KeVXBACre-YO7YT458OWJcG%$)d&~MUz37!1A3yv{-sgVvI?)nHqNW?}{g<-8 z-HTp(Yt+w_*I^bch?kO18;4@rLoWr%ivGFUA^F&)*5AfE9$5{_ChiB5oPX#*;0*%_ zz;^Zv^V5F{v3GXzmomT)s9k$)g9o5mAx3BLf*`$+exZ$frwLrW7Gb;y)}Zde5j<2b ze*Ode{HjPJ*MBzvQ)fZ-O@Xhcb>~-`RP@dfQe`wR|EVn!ky>1>wls zr?PhmiIU5=CuQwawv^{jJ{Zo9dC^l_O!FhCdv`awirtIKrKQ$Qlrt-5iP9I(q*xXo z=8|WnNusM3-yr%^wxxFI1aFC7l=(Ha1I0&9wg0eSXXo=@^MvtSPGG`%!}Z076|$T) zpT?gMvRLs6Xek^PVPYk0Mc^J;*{@%A<6<1|m0FtF^ykkXpr~7N4sOaZu3U%S?{>B6 z%gpCQQC3yw$^z!iP->wy$+fQu9MuNg4rz#$< z`|oi`b;GS93c2o5&%f|<`7F*1=Kn4vyPwLvy|*7c_W=u~tdVx#WMKgOMy2sgbIio7 zn!o1va%ogY6vJWG(0h?NBO7moWq6Cl_3Lgl{4Ih~qNdRw8oqs^ISR&U?)qz&}*mS6Rp}aZrVaAeVe8TSN}r0lAly zp0R_b)&iKF}4a?QQE&b_sG(ox|It6XisiGbLZs|M`$=nh=Z zYCOk2b}0W*A6KKgFZ=%Xhqtt;G%YP>QtjhE(?_2hdOVi1M)k-O=H2wx0nYx7yA6M? zPV}kDVcs3Yu2?et{VviTRB9){IiDI}!P#w!;6AOOruJiQtbEO5;M(_g3f&rPUh)tP zH@Jx39j2z6Ko4RD+8vwZd6u6`2k4;pFYNmusXEat45;uOIP3A?@TI+EPW8o!>~>-l zPD8@5_L%~Br2K~mzG6q5R!Clk4;&_Z8l9ClkJ7?i5$1ebSy)st6tmQNQsl%7riJHM zR9SPk9B|pEKiSx08=0B6u)Co$H(N-5R;T}gNeHOA4RNVf2AT~YS1x|KuM-xsB3_z) z2Yve51LhL?_JR)oE)w+lLCJ=HP4eZh?$MwwS6E!o2$G~tvMT)L@UQphNuAyiQ@9=n zzzdS*H!clX;2HApq35t2XlvKwXV0Dm=v4v14#+T!9)h23K$Jh`y{LXC`m!(X2E2@F z0g%N&M1e4zX8SY_s}XE#Qa){ZFzz>M%Zc#dN_i*30c(7 zFRcCVli54JADKN7YIGpI0PRn?xxH9Uw5-HQ5kxIvVacqOLYOH@Gg(;KcNkl*bt0Xg z-IDBQYH<{5%(CTkeb(Ph4_l6i&rF)(UeZ~A*uy;cYpe@N-*ghOhyU7cODg9R1+RtB z;qtXV_*%ll8wIEKx);i{yD$zFoYxd6F&K3#p1solQsAQKB$s|~QZ5VqnNPc!^~#JH zA<5z3=MRL`MCi)5*X!ax@}daIfLaI^Za0Bjugg(;^zfMd)69@59V`$eE)GOSL)J+7 zvwGc!i_6!hO9)U?A8Km8Cq)|(GP@G)nozW0@PSkw*e11P%||lv^+1$5=!z+T*jxXf zOT~s{i@Xn$YDAbs^>|6ryZL!@fzCmg%R{6m!&0Q6bD1(=y0?B#iua*?Hl#p>4F76^ z(Qfd$mh|gyx)z;anZ~OrQ0A}gd`OErni_Do@yHVG$4CASv+(y#lzMcQ3FCV{w>+rw z`5Bi&AEiIBD`Lq!L{FW_paxG3`jq`z7tv?(X}lq_lN8jzs^4RZ^0_V_(NjU`jj`XG z2A+<%3KnEq5;CS72;(WFf5GevnS;CBaRl`Q#GyD}YxJT-T}F1;l`l;4?>>)e=5DeT z(UB8SnDj107K0&fAml&F_o$Qftbx`>f`;_ZWE%eeg3i}WfOF6=hoJcb^nGy4?dw;f z18(1b3{);P-J|)-!1XIqO3Ha03-4yjRC?d^TApq9wO%dVZoV`1fNqe4wwF$MJtN{p z{@Yrv(ETA@51(X6ZLIBG6Uyr<+upywB~M86^jTkheU~^>>2{X``c}KF)r29Cly3f1 zaL&*4ccIb1)V;_?*Uy0<&-H&8K|H9rZI!I1E*36;;k^jiN0nUCy{`D5Fx~ zGKvCrE0}6_aq-sUb86*mBtW%|+eg$I-JG@2`F& z;T{H7gw_XDIkdTNkX{-Got8W#$RUyuJ)Nxb#!{I0_c-+5{=cClzgJP|>7c&-K>9Rh zpO+$Ug=Dt3TENS zGP~bpz3*5Pn{nw6Gm&lGMm;hCOfW>zONBA%X}t4{qP9S z+P4=B%NHf63}Yw%FRBCS0LcXac8$SH71GOx zCGx!l;sex<=oZGQwNlNoB4HCR?VX*eRjl4Czx8#&^jlazP;wsp`P=1-WEutf7yNb>wTXN?8d{XZlDDc z#Y`X$=+#Dl#jwQo3QSeiAk6c(SC`KY@s?ogsd5;A783}Uz zZJn^hM?YVUmG!;bB-M`=DvF^BXvQhnwGl;kT(MycWv`;vE+AQivjZF0(e^~)KTk<3 zs0F(vfkvo<7$azeQque^zoMwB1MTaYNM6_l#b>??O|&rqgx>%0AyYzi#p|9r_!rZ6 z-&b24vp-P4`@me}a8I1?%okVgHl4@6({H4=d2Hys_Ih(bZodT{A^QOau9noAMW43| z{P)%j-Efm=Da_S4f93Bg|Eu@a=3NKMzN}JB=-aXBsLZMJYR`8b$Z+jBnWUw6VC-{e zr{nyzJ=9z$p2*xJnh`*GJm&$w*NW?8 z?B;GZsiBwRhYe@9G32fxa~Y_=$vZP!>0^o-fDjc#yNQSNxY3fVJct!{l}l+ZefEw+ zQHvfZ)DV$7nLS!5w$x5_(|=!K;&lsBhVN0#KSLrrwdupCVx%lmy-Ole!?l)|b^BY|=cQutqO_p8qwbMne&jYgm z+P0`L$zPWo`Qzbk@Pto{vajKS*Kc*LC)bQx3Pdhqx^t%680#nfbu!El>vxWx?8r6K z!xSH$2XQkds8mt6$6cU-C7*8`P}B+%0eu0SssdHMRpZlDs7OQ34E z(W~a!_sL=Wb|GvZPMo;M^lvuY^c0{R=9wA5)0RJD#gQzuzz}M&M?mA%?*8)i0`*Z*D(V-Ptm^ig}TVa+$#N z8djT5@4INdmiL-iSWH!1)8N`h22P-zaR4bsu;y~Fg%9ykQYUY1xCqd%qN1WC#{_4t zz5r8G1XId7UtW(f<`0N`iYh8~EiaJ}aM*mIY)R1O*@+B+%*v)7Cm|qetr(Vj0}~B@ z!T+AlHC2Q*P}F^#9jSn5Y+%1c?0BLC&eua_IQKSrr6wI)MRSH?*hDg^OK`u*&=$ZO z!3(b`xT>YM{5FFi$2m986a6?`7iDJ7!nShx&z$_qmpOTZTYgE@I_eCUaU)PGB{dC+ z{u}O}N~;yjyX;XL-jku7F5IB*8JoKpUikM}ZIfqg>ue0avYmI6XW}0H(_!p`s=2Vz z*Uz`~m3i_Fr?@54puLR^pNAPrN_jk()U(-p#A1VRy~qWqiK?55-qPE0$AMbIuxF22 z@bTRPC3Fb&CQ9^?g!=FI!~z)Ct>TGXSD+uLpQ$$GJY-Rz zZ(E=msr0Yng^+$!q;pRqeS5!O=tLNMlX8e#sF_^CpMU^Xxr9a@D|WMmi0WP3tQ7OS zj$FG2SY`9h{L3r+lSm!>B|ntNn51PjW1x}IYAL)+c$ezSvG|Ec4lFliPCzdJi4YN2 zBWpCSF29Drr|tq<23*DfxrB?5X5nGKUOowr2#A9co#=`nA_4O+R!EOyUI5^SoMuQC zDv{oI9G%!IcSPcza$`N$UHX+bhwRNx3ctUx5w#jc?Plhe!?6pSZ+aFaI%`Fp-*Jad z+~;NTKQRzpU7x2`NY#IS9oM%>lWQ9g`zx4(uZb2|IyV;q-83Y4H9$%#p+{?G{}#=A zHVE>JMxZQrw(~0ie}shR>QL$mWLO)R3Vn}t?S2B1A82NP|0Y)R`&coCip|&qv__n@5+9UP=2gi4y!-%V}@Pp_n2&ArX za0FF7BizA#{I{0+P?wi})l_p^K~WJbx*Er!8_4m9Z`_~Y!Cf2s=n?+a?c%!*g|Jh% z_RE@@_uq{ea~5L(&x^Q(MYj)sh+-BUc8tn;Q0aM;?m^Z@ntS2BBq?)ygS?EwSbqb@2F3J~yuXWGPI<2)J3FNscq} z0sKY#8>-VSjErV|mh9XXWG(_d6BT8MIc`$dE7xF?^+RDP+Po2xV=IuZHPW;%+*=hX zWXhIiSaI%c;vWt%qc=bYAhO%DiB|>2t-|)2%;?e7(%g81Qu_TmkpWChyUsjDX2AY$}}M2LDMCVAC7M3+>0x|7(f1v zUocGE^)lHnbc>z^ncY3jzGfvwF-#Cf30%54djW}`O4A-kuihb+$lQmo|FJODQ`OzQ z2jDqcw`Pg3<(G)ry2{E`z)Wf{p-#vc=x?tcE>U##Qr1$7=s5kXe?udT#WDsWf+Nh* zqB+ak2c2TRbT>#>OONrrq-FRBhy@d?@Eok7K`s$?z5S$B_58WEnn$)Y{y22epnKwY`7F)aCxM51t8GQWu^}i=3@>N335AFCg=mbUAghy%8A}+l9 z^zSFsYreFr_)eN|suvG@`*z`o;eEA~ydjm^`UD-xT49^C=mX{IW1>Q4>Yd5(vz3(W z7IZclcm2`t|K@^%&{snTJp_r|h=%IyfB2?f`?>d>zRc^aH$IF_w<^y1$F0(<`l%hg zqRt)RLkEH%#YD;v4GMqm=NRB{^)4Snsm=^WAmd5 zN&LGV#+~xIu5LV{{d1hKJ~jNywWgz5<@I&5$UWo5k>jEB?$Jh=ea`1oo++Nn_s8$8 zw4=7p;+jsJkime}l4G1sRee3Zr>E!0uV2q$n(L8xPKkvyUp?s3sc~dlv&KI$G}uO(1m1~L#L^SGyRH)Va5V3|zkb;c zhdA8B3)}Cv7lXj~A-G9`$LANDhmx}X=5L*7ajdnVCN7$J#BH79EcM>073ahumGbEC z@qYnUL$loHQzSG!c0NzKiqAB5ZRkbwZly&E~(lTf@&o>t@~$Q%I8jq?GjSTQt*mzEQsZFzHXkp;%I_c zn)nH!A&RQrWFr0Nmd;w06x;FrW-_4@4yZKoVREnjt^V7U8YUQUfVWd|K}_hM*7-bT zuh?19JVdmltMP(XRK?hq82J9z^tpR&=l15%n&jj>k8R`u>1VUmq@^h+`c2T>VivNc zjg7iPKQoG`c`;I}1uOQ4{eP;4bHGUzSzm(-_bG70kC*voKJpQs<{8b8LH)BGgMIOc;!;gUT zjT8sUJ8XUcfiOi4iQ4GuwQKROJ_0l&Xe4R@vP%PLDu3|iBRtUYZ*NxYwW|MzvMF zo9nAo9vutC56QD^+((29z&VMR-7#P)$kk*Vn9xM{zU335g$U{IG~VKoWhUqXz{hQL^$&cud@u zxw-D*p&j=H&DMdPsOIe$1;+6H?rb+W3_VH`kwcQYK06<4O#t+tF5KjdyFTd+s^#{V-ssrT^ zlE9T3PC;v-uu#GR1N`{u$xb_X5o|=N9W2by=7Fg8yycgCZVDNNg}zjP{Re7Ld%P+9 z)pzKsc8WQ1fn(dx!NKu$7O>W;Z|G|{VA=r9LGSnPgem?BBqXUDH412qegq1%=vOcr zw2qOd0xbq8xsN#uBd=|ak_kN`MLlO|$gTNwCE~#aJY7oElS}6-QbmN!>mH*=e|Eln zvAEOYG@!)0pFVwJ3bR>88k>e|@yi1*ID{cGfGG_CmfxW&Bx^lTU;^^+$G|(tu#3YQ zc%bh-FrdNrNZ)cm>2a$y;H-4B5BuCXtD6sSQqbxz*F}jI{U~ zh!*xK9t@hPog$aJcsnQI8m>xq2kkndG)2FFM))~U?~GhL3_a(?AW&?uTiO7d1L0u| zukFG+dHVF}G)k+T2#i8i>FvY#gD`$Qg*+QPR7|`GK^+IQ%~yY6+MyzJzdVIFKIi1+ zC4dHr;L^PA+U@R-$BPA!B+Fw6X7yvz8@js_=9bJ^BGA5Ypss^gMK9dTpjJKpSF*$s znp(K~1maD0Y>|@AI#-P0)cyPR(Nbd7)<~m0jrV*AE}a52O|IT>X#)&&WaywhRDu}X zh6Riwj&H{!)N5!ZcnPztpV{URCyu=)6334NoM%3}>K8|DCZMZ5Hjv)NjO zHhTCrNP<+Ls-9+@+hxddJW3fxp`C^i0>Uxp0)0a@0WfgiEnCwWacFYx#S!5{?WdVE zs{;LQuQ^)^^3tW4yNi0p2tCMnT)4nO0N%Z^SQHS7<=gxQv$=s z5MBFGyKWVTgZSTRE2$^9T0<#=0FKVhk^~`-H>j)EkEDG?eSzbzk!z}muwv6%&1Rjz_V8aMFajz>~+vD%qkUm15Lo3ESFKqF+2(NqIzIP?h`*55Fz_AvdfG? z#PjLq_@HOjTUFbUb8YNjL_}w9h=mhv)+(LAkeEq+U?PYe=*MeWcs(Z!{Reey02U zI%25B7k{QC2yMYE|ApBPM(M~^0jx=+%(#SqMqiCR)!_ON$s29PLgsHcMNDt1rR>K> zc9blr%2%-mKEoDd8C(JGb1x39TE9IJC!?YKMs_h`zCC<>F%HP3of?8NncM>Ihp#|i zKv(huQgA|{NAVjpVTm5heeImfIMIgiRP&}GH7ntcK`|wUBx(YsUx$a+$B}*R>nPG{ zF(E6+5EJm%_O^=(qU7{m!Q3P=id7VzQ02J(6O9kA+T%|;a0W}yIA;ks)(Hy(rJ1w&lWxxLdc=p+1xB2^W8JT4~hhS9huSuE5oMzpj+HDdU@w!3!K+JzU{8&sw(6 zW!GL&7jO-gF-$_5EDR>^%!+>06#`i_pWR>lFYms7WkK&zQc?mAQ1<1&eEm%3CdsXc zN4__1tN?&!(?GP{{05{dRR-B45zB&TEet^!#SBC~wU#sFa#_ zNO?(gcXy+3Xlv^=^%SJpJ3#&u=%lAW?sn1#^D3s13ntpmvpeRXzX-5E96cT2&!;A z*p%&E2Ae9!%g68(^Q~LgiHpAy{)aZ7JVx|-Xpg!*$Btt}$s|CebOAC}5&R{b$SZyC zH-7uWJe4*xH|Gcet}qq#aSGxXLHW>&>xy-r{sL>WRqM9B#zVu5*1GL{AK(y>D{&`S z=1Go98YKtT{Fv@rNa4`n96K$ypE5l&^Bn%Z(5*b+pfSenX6N9*KAYx>S8&up@g$G= z13Xr;x?1l`yhtN5J#o;$pvLq6J$4f;auT-r973~V^`}(L{S<&0x71SJ*&G$J{Is`o zHJ4)3ZdD`2NOtaKoGTe*G*+)^HhgNdmK{dp+bDatQgUAbCR=gg=tDD36As9-iKr3| zGPW#tRDtEz-kgny<*UYLw02*B5UC{hg^_&qT8dzROjzPsGt`ZE!Pj5BI7S>fa3E~9 zE5`_$`6Q-M1RuC?=)tgvTPsoF!8q_r4Sa+Y4pCT8MI(}8W4uGHFcy!D)h~{AJQrOt^xl|0u}v8Xd!6Tjp*M@4G=y zlY;hwon3HgX{nh3W~k86>_!npPkVfMNQ_Dn#u)SXYGa! zrg-NOJZJ8J>oOa3BQ8t87B1rZp?g6iUa4toZwql*=>x3*ekpEb z1T{oV+&g+q4FZ#nhWjXmfUyWNI}RJKB<&w$Hk57LT7G0+pOuvr;0txBN#!<%{eT}q zzxFZNd+tA1$s21N_k({!A*ElRw|di_I%ryNB1z*2_o3=J8l)ALC=xkB4Z&+gGDU?{ zQC%GXg$GoFwg&zYnbiyw!R;_NL)l8yP&=_L5&Z!>wB@&N(*fDX1TzO48%?ZQ^r%LT zfPRgd4J&K>z~wQ#nyBKl=g%S+qyRvRx_Grm80h^waoU@kQKWT`mz@gOqHexXKBgh~|c%|K*pL;i}`btj?@Bc$5StZWS~0^>_ZrN}~zwt`fMG3nj_)O>{BJ&nV# zH7M$)jI#nzJ|ds2VC2#MhH9=*EWXTjY!5JTV_Z*E=^N2i19h$O94oEDv++snzm5;> z0Nez*ix?^(zo3~yvZ%!SUF)Nkl7P{-s6r;V8l#Gd!Adwt3BKcxdSzSB0Y#e`i0>O1 zsDf1F&-nO0q#8(%G|R5KlL29Tzh`ZVi`^j(ai7soHG=sChPhTcPQu-?DRC`Y4d7Cr zpuQvo4QUtf7p5^*g6Y+=1KqD*zviptCkP~A;1L3ow9~W*%ZbpMXk(3pSr;$WGbC1z z%w#m`2yg-Wh z#|Ih&h{OBvmE1^Vw{?4)HwDV=2Q{}JO(O@7cIsZZbg=;dJAeyuUO|F{8Jl7E&JA>Q zn|%Pv8-LEZ_|{T5IW5n7$*U31|FGBmv{UrJO57wyA6S|dIddIlz_2FT?Uj2`UsxCL zv9nhUO^F;n{3qZSQn|qpfX!($|F}Nl&`-OW)C?q$mA;4N4Ww-XEBBHQxktk7!f*&5 zx?FyH#3Kv$Yv7Xqz!q>VPz{>f#NGGZO8^Mle9*;MMHQkY*+;QQ_)KXtEV>*^@r#X( zMFF-Q7%T%M=v#eIoNdRuVnle_0{&EINgbJ&3m1!z(%aw9j&hTG*HIfT1SN|zcK9u2 z=-6PK%SAyc!&Ye9Y2M)l$@!|iFI~qD`fdIW)P{DmQj!+H`F0+i8*lcaj%0(bDmtkz zR9{8q9-_4rX&CwKzjSm2;2P--kuI5bGh!5j3>^ati*3Pq7#I0cAMNAeQNm2w7l(tn zq`mls`mH%P?#%_atj|5d_?cVL>Kc>5BYHTqk)tq#tOZL9dCa$WYPtOVp^D$#STtH_ z$A+hPP!nh6(XpcmQnc(ErgiTw}!7F;RI0Po9L)0)dAV9O;=b z$-szC$|Y!CTxm}Xo1kD*q3>sGSSm5&`qBT2H~ed3`n_3>?I(b_ykb#|`Yiv^vOfxp>dtZ2ZOkeU#aEhkc%&8!yCbPA%g zK5FyX2N*0E%wE*0gKesk7XKzD>Y9#1BJNw&%wyIMtb$CTQSxm%!T<}NPE=sP+oY2w zPR~#(VL0^woJI*vg2n@gh5|x2W_Ls=2qYKqXLJa`AY-=d*g>WlKVv;7KHql~tWk|_ z$csBsr~r6;ikcYur7b={S17v4v?%rR>b^cu$<%4iG!P-85wLj0w&;c^5!!#yL3f6@)g-}0Oi7{h9IP-tHJiA#Ut|LmOuLfB4rzw#`f+pzCv0qh^w=$tFt zF6xnC*1oD>`84fZM5+fzZIxi_5!H$52aNlx5OwRDHLQsSL{v}F+cVpRk$NALMZULh z-*y`QujbhScMHlf>;=s-i_ytg6_#A7n5@l1Fbt9$(MS?Lp@HLH@oAJeRv>cF5WK@c zJpfV$9~@?2Iq!kMkr2R-mbrZW@ywpEUAA`1%e16kXNBky;okJO%`B-OcOR zH43A7->Lzf>Hx~`V>A4sZ7nDx=_J(5RZER(Txf{Jq*4SITR5%YUO%MNT}fJsD7XHv zmH&kI&kf!OzvuG^5T1cJ?S~!u6*6AlT*qh(B%-U-Ga8AubZ8kdNsBi^V5$BBP~`(U z-?5Uv^bi9yOq`3iBi|V_DAon~9F)ZGLqj#-ui|x6NDEf#5s7xhVQKLiv}uT>Waa{$ zZWFkN+R7gTjC=Ob`$R{#I~GNX(NG8=fWCmi?P|215{lW5A+>iY&{#yh>Y+i@V+`MyGT|?DAyStMf;8{pgiC=VOs4PS+flJ zG~!xXwLRMj*7OHi^Nwd-NY8x|zi76aZdDI{XkP&M>)z=Us>6LLp(G z2^KFsEe3cFrJZb&mf7dtN1-U_KcVS{)KeLn0Wx<3HA*G6NblTPjZ47injtPK`W!-( z499o`0vd^b^Esx~VL%56NeQ(lSi8YC4ohBz!;$ za}@KyfE8?c_`l0?lEl+3QNG21^1@U-8(eT4!f!x>Rm(vzAnsZKdEnlNbp(Nu z-FGyJ1TfQl>#!2u^?G)+2c8eV(LWsq(#6y^vnhD&s3(K15#RyU4np2!H}7^3`DbX* zp{lF912QrbgcpKUqEfql;|71Nz_}O1d91ngp$&MgY{8ajB|6{D*U<)Ms4W0S4u!M_ zC3UKH+7<|RJ&nbIG9;s75fJZcmvgAP%}gU5mdFINi+jlBj^q7`@VoN2`8pGC1HO(_ zP0~#ovuHe{#J>QG3W7?RO!p$FqV20);wMs-7~D*7#w#cpcf+4Nd4lBoE_Vd*XEK^Y zy*rtRQOGM>d|>iSW-CyB!YC9W@O9J7*vLp7JZzqcePgqVSdpYvP5wnrO-VIV$6(7E zD2YNZmiDjo$I!xm$r0k9N<>P9v0Z-_`4V^%lN&9$5MU_qB7we>^pMCO(A`y{p+w7o zF{VshfWvV*u1t8Z`-wcdaxY2FYNM&V2*Xasu=^tdYMwkWMZ=b}KfuQ4o26&W8DobUE)0snF4NY&Cm0KEaIIvit6=5t|^ z*N5+^<~oDuflUAkw_JaDdU$NvwW|h~l-sDp?N%o@#10rHK`Y4rZha=gP9G{wREH1H zPw1)gN9qrW|Bm;E?g@&(c}S|kxNU+o-Bfo9RZbMcQ%X$~|N7Uc<)?w~`y^`Kz`rYl zw?eIU_Cfqdq)B(w0(=2^)mq(pKZb|z>Sq?RKSEwC?8HaR>&;92Y8_CpsC;nc>}iXWBH= zw4*|8TCEGH&NU8^N;l;a6pJL>W1l^n${%QXp4eWfpIQG~{1$s@^)e#b&=28EFInUP zH<541qY-NE^tu{w%SNd3lTpeG`6|Y)BOUsuXKXo}3Eq#(FQ;AwiAHsrV!562o z+uo6`M~O8QRGGN`Rp?MrlIhgqB#@P7vUul2I;TD04gK>DKnFP8gyz7LwitAk#BYTp zYCqU?0Kzg^)cV|pFDJIP&yZ3NXa?9jJUw*yjn)N3?SXSB+1#lXY?Pfs2n3^S#1bUp z_Hi%7zlqVWyA_m7^$;Y_q-zu9*M{9kH!giPBi% zWjJ^7-z6AfnO59p=&y;yDLu^-N!|)TYhv66;K{mR7xWrwOcjg6_p11?^7 z4A0vXtsC|R9egK7b_L>p9YC6o;L{63`PYm~x_3}56JjIcI01XuG{aX%aXC1EJPeD+ z(nA6SM+wZTZQHa*V(-C&^@1;b3Y`-}ot#b`%36q(d_Z($hTjf*<3ai}q?jvcRNnXa~7%XS}9YhY0+h&fSaXo>oP37NpQ6jga`S<2=8 zW^3jEK|`#8lTc@^H{?-m_IrBvXLA@&oDS!V^H5J`M7AnN^J$3cLctBzdggW56z~bz zCSDdOEfh=-H!_S>v*^e+O$)ZF>3LVju8?T7q&ODutS9Fj?mY?Z9`Y@4qg{dT!{|ci z#hmpV!YaduQnIEJC&S`@jg0UG_a6=Gsjav>qPi?!P!Zm>kvI$h_JIzX*o>hTvt`jK z3>!bC>T>S*t2*hQ&&+jB{aQ3VVy%Lryf2jM8~max?QF+bOhSf z$vS8C&rA$#COAukP$?8{TgY|6yk|J|{f9O*ld%ZWAzP`zDR zJN!GyaAHsUJyA2SqJ%D6!sUO1VSmT-QVKM-K;@gI$O-K`( zP`hqoIs)%S7Ked)Rlq~wN>_!+O$Xe&#}4%!lyYdyh(A2=Mkqt|3tF+qfs`6WMN{h^ z8e!V5TF~j54n{+pt>fnIK_!6Yny)$ANd3EhJ0BvD5;0y122<5f zomz?UUKme?g@pKky~xVS!Yv_UN{|J_EbD-B6nSn)!q6@+t5@SgLN2(+&bw(1h_WBZ zl*)AE6JoK9(c@eupExyY@Zp1{ zyLr2!7rX0riwOlosegTliBYBzAY2S;?3RV(YY;YTb22Bop(z)F?-UBbCcfIAyHb^s zJT9^{G<fk`_m=X+%!AW>|fA zn5)*3ox*Y2l|Fc{50=!?rlhtGZCLU?K1hClMHht`G0tl#VT7qUo`){C5$8s7_UC>8 z8U_$FSPqBeMk3idj1E0hmfTsRLi-vpBMIkCb*py6fXUL53&kj5FrPqp3;Lwp`b8^L zh`L5nV{{Z4QWTZcqXPr36;=Niof++X>eq0pK8Q&ED^-A$s#SE^5Lli?45tV$k}T2TqHC6J z3aLOrmH?za;-KLJ@`H1*Lu&#c=>KtLelSGqP2esJXIq?VC(sN;BYminLMC*wA5(s{`lTW}_6Zv7bN5F45N9e)_$K`F5e24`zbkDVVhk$u0N()k>6LM`0eXh|@eLVmIrrlC z>EZQkpc!M74t}uNC!;fzvZrxhspiVKqhfFVws~rW$;s6U?lNO9emAGsS*@aH-maW& zvS?+-J;l#^IB8y##wu)o-H$}gJMk@Q@XN@=z@r&We1nz;$m1c;e9;DIyDy}@x4;V@Ln+Y16h zkm`+&4FXHa1O??(djxX#qr>x~1&H_93AO@_Or&`XC?3#<-|%F0TuY!`BHjmBl0uP2 z562f<2#amJi`CE!FaR4t2MvF~{@UnZv^t-j%C1M{A9b0J-~wXy@q%V|%u?S@B$niP zfx{>5!eHC_wQHZ?FhGPT^p#poR+m5`(X_C;1i-|DSIoSaK*GJXDmFNjfu6b&gd62E zhWx^Tz*ke3Vu0gN7iTR)IzTU z3drc?FUEuC?t_(~$$_{FSN)$%-^W3c7to`UwLJ(Bg+#ALJw&F8AebROR51JCP>Q+baO)vMR6uPGTW&YE>Giueo($_|M-_#V_7Nmife_m_y|i+? zV^dOZpj>`}dZ7yKSwLW5B``wu$o-+sB9FO$1YXL1uk)*7v6Qv%#-Xm&MJIYEUYu6- zu8#jXBYtIf*J{;4T0l3yvE`DCGzYPYy$uE{&zT^1!NLN%*>AAXyNEgog0c5u3L$4O zt^(nTAeO(Pv6IKavUhO!j;0!;n-O@YTIF8Xh!;JS4Ao$x;Yko4OAVWN;D*5`yV8H# z2@+0RVj_AVfrTq>4q6h6UG9R14t*M*d5)OJv@;jeBMJcXK#Hzhv*s`Kfw04EK%6Xg z|I0yeQ)uWRG%B0T^Yl=eBwHn*n){9mA2kI}3x%SOfK7kjET7Ew(_IshnS!Q9B2Job?S4J{`y~LAR%3@U~;bNRfn`k9{!eNJ|(Bz_#HY9~GcWGy=6H^YdGHWo__#Y_t92N|Su7&s9F zE2{u1DL!=94@bxDL7YuGv7C<(A{z`LUM}5yKPN^emMImu$Y9YiFAMjQE%4LMonJyL zlbLyv=uOalL{a?i-o1;NNka8GFoQ}W%BCmUJeAN$@KJx9Dy18eI(@&-+3Kv|L)nRV z&r=^lir;=?wJA_={SigCbMn=pu8q0fLEg9NhK`(JXMSmX1|6lW<}vRR$d?9~c)HBJ z1o59Nh~s#xc$cwmegYoz=se?kAH>3oQ?(B)3vv3cz^9kh)@~6h;Kd->W{`L=!r=q8 zVS48B(TB2lQWXFY<1-duxVv5&X??2rODH?{9K33`0Iy{L1-%Uka;pzq=gV|4hs8U# z|35`UGz7uDLqj||*+%@3k@#Vx$XTHS`{8qvP76R!-qzcH4Edvwx%}S23XpUk(-T_j z>(wyp$My9k-YGO1HCpY})rwGJY9fC+iQ5+FXOi=T;GY9kP@aC&?8>j$;X+b*)ZFZ> ze$>VthxV#=;y!1-a2yCa6NBBq)zoU}XaVBJFs~mnezJN_>(CvM2 zXbK$%ZqI@D#t9Ku-1ye#Ms}h&CCSGioui$#0GM#s0uyTreY}- zOa))=elC#r6YaJ4FW0l!%(lW?S!%5`9YgujQ@SA$L-x3`1Af(}+#<5??}P*out%o6 zs5e~~%I+Y-QjEzlm3jJ@lf=b8tDSMA+s54^jak%PB;l zn?*%M(->b9oX_G{Y#Ls6=zUojJ%Uk;?FeFOjvECsP^`$}OUXZ(TcQz077%pq;=A5O ziN9{mnl*>POfhWPGOkoeRD+5hmod}68F&gK3UR^pVmDzCp_~$@7QGn#RU}d^Wh;{D zJ~1(|cZs19u0Q7H=28p>P#@^9WTB_@9CA7B(zbf#%G`@kEQDX$NF9WDrF?Qh$MS72!%-5q3o10LQ*1>QArwh{FC1m*3G!O(p(qRAI4Dj6P!kh00Jdn83 zGzI4L>r~#+yX%dv-Cu`SWQ0uv@u*|cXtanxAvN?SHw0L26mXvs#ZY^DyX^u|V&IT$ z80|+2PDs)omsg;vAbQ_lodF=30$ntjn&HK$V<(iDo0}_Tjp!HD5KS4}6o3nP0em!! z(X)$xw5^##rDAN>-r8!N7s&=_Vspf6(9C9==-B@oUIcnHb>_BFLilo~_Cf^I++k7Z zBZyfU%>mI2Kn)f2zpFtD5C(7y=vYakLtNXknvGdOGb77KAT!2%zikxb(M9*`%QkkV zoGstey7`W14(HBjSKiz?sj5episfuIw;q*yE0FZ-I$IR~4%Cs_yP+bcgQ{&jvo`)#H<>UvcbVJbXA?Mmp*^y@KmbPzl05u-WX`bDUic zYfIg0gwRI|Y=wEs;7sn|@GQt0`=_fsoCE3je|T5`*uSEsR6c(`Azk^~wh+0rFwMl# zTulK?JmR55sRlNcd|i@G^p%qc#y)LEwQ=L&_aIp=H$_B8f%CLD_&mKCHB?4HaA087 zADhI6;a6rt6?2H-gh<4tG6RDC1poMRCHKGtFmB(jfhw2+oEsuC8udvM@Z%QPC^1lD zU5tK;uxVb_N%wc}-huV0w>1wj1^G$zIr$7usU#YzvOpL?Uf+kQOVt7=^ z6^ImB)_?6%xtkZiuynZQCwy!RNTer(7XcYiVkk|i4h0^f?>?U*_Zl)W5-kDU*ab9i zkZshb8pw)^i+8$`kLp9{DsHFYE+z(z$Ts}o{a^#K!>XeWbAo|OqyL4MCVJx}<2AgB zkz2F-_3I?^;fQC^Nva&+N8rk|q&E=*Ye&bcZ~9#)#BsJbw@5}DH}W*VH-AEoC+NoY z8lIjg6_ZfdK0bv0B_m+BF?56=3m(rw`AoViAd+N$9zXe&#u0t11a>mh;}_ayPe{&Zu| zxVgb)tZgfNcTz3e@PRL&Oo8KRVKaA72DlFh>b&vA2C+&T1J(LB+na|Zj3O)l0dh%v zL9>vW1l*wYfvc07C-sHc;rCXV#M32ew(h~v3UA-ilJy-k%#;K%guYp`8e5h%G@7C0Vxxk%phLfJKZqid`K_r zqF&kn?lLB#Ov6FIj1O3cXRO8xJ$=J>ra z1=i{$_~LT#!QPBYq{1Q%$m5hI+exIdpSFJkoT(IrG|nxFYLg2})Nq!RqefDJVt4Nt zHke@N54W*4>l6bd185N+kBLX+v2KbS@o~B}PY~$IfR?VC$|H-972&-8|PXLb;e9&g{v_qgrYrm70@%-FwmB!J% zrLQEZdAyMQ&UPl<2##^>+%~y@9-eF7t>=z6G~?m~mZ|HcX$=}|m(<{MK#nTX9- z6u>?kx>**?(!71Cy({1C4KfIs3CeBKAId*mef3X|BAQWQ@XDW{w=+APFY!-!kOXCw zB{X!rP#*T-gUP0^7z~k3BvOTOsN_F$$nIvrQZK zu`231@0TxMChkEU#w#H)d@rcgZX3>|$JVfFZ^a2^e8?;%l)L`Hh_&X5PF?4>riw40v zG&D5XI5R0-875_+UAvA#O$AWvg2)52TrHL33+hk%4V)96SMgqL-ewWtx7F#nj^B^J z6<&-=<@SeoEFX`vJTj0@Xy5NZ8@%;V@hdi_Bk5axVwI{ zDA1+E?c>E&KhyGfJ(azd1ooK}(&hFFkCK9*;voGV)GCuE70|Q47=yM{fL@hx=~B1kX&l&r?#w)bg%{{A=mCK1wB+TCq#Qgq9hv_~0NaH>hV`Z(!=Zu?i0UggtX&Cm>Ow3q*tuS`Iz#os1qP*n(a5oQBMb8JIXOGj3 zRZ>bZ3;@knE5ZDLUqjz$?8iH03QMSa$ysvs1u7kLLN}kdQI)a6d;W_&cKut?;Afd9 z*p=f|OCMe4b*;AGsk*>a=$La%-qlp4x@}LD)1`?b=}-KAe$mYVehkMrr7x1%m<(2#dXKzj{p{Ndf4dp*{WAc46N4lL)!=MU!k@l~;JW~E z%uP&QKqVLbRO*ucGizX)Y-mvWaHohJeuy%kHc?*$ho%rw5*PO$tLOZU?M;@NVfvYq zkd_>AbH@^d5o>rO5Fa8YK>t^ppus|n6ENZ1aXqdE#(|!ds0ne2XCiKZ7!6=4FpknW z49zMI5ICW*_~H%*B<_bLf(05nj)v6N4)^J3;$Hqj|-646|b$z;S5Kn9wk&pl^HrmR>Dxxv0(Um;}3LB z#6=fDTHZ>W6P3vGvOk~>GHnA2y7Pn;Knqh{WP}J4=q!xCK{Z;6t0+Y`cDB7=G9F|%a0CffPvv?(WGwFAv9sG*)%4F~x6#atbeZC_ zY}YPI_K$l%JRkP7#WSVKUoo<$iTC%Tx%|=Y^WPPpi<&=k%w9_~ zuKw+o$+@w4MZj|h$4P#Rxf@?SwaVsDjUx!Q9*3=2vJ|1Z4Y!`S4ai86XG6}dZigqR) z`Tcki@22r(P7#gvap=SDTNGn>6GrzGhUywQO2$m)ZDuqw)^J?fnoW=&2#9x_MS;H>0 z;%vp%*oJj_(|QtiDTr2x&;<6q9yEoxiwDC0m6}BTg%vAHOv}&^9=;~`zmlP*vojJY zc2m$HID#F2*_FPWr#Y@WhLr#{-&+?M{ewbYJ&?3RjVDwYzw@UG%0+9a)Sp%l3N;%D zv*I&RSe>`RFrCSOxX_H}Xt*=b3m_Qj(m#8%w-+NKfP>MJ16?A)(oja{xSYoF*aT(- z-Yja8)DI`)KMXny{vc?i)J;rwK>u^+M>v(!us%ar@S8RMs9eR4q|hHvMDAz>t396j zCEM<3H~v60no3XdVDF?go04x;S%3R@xxK8nssAmR#=TE&W~v05ZxoblTenwlz(prh zerm`?#x3nY0$6be^Mc6KZ%QA zc;6d{O(8QSnM+`GzSOKqzkm)wGg00?JV9FH+__z70j#0g!UyCF4_rOY|lpa1YfWyV6zmtmaIo>AFRgCt;&30dNSD$52!BkLr`zVk@2!N zmW9fj1xbH}Y(ERWID1OD_vg`>Wv)xvy{ngm-nKC2&k7Iv{m80RcTsYix=`=8K~R$Z zfF|?rq|ZA-kK(~n5WN}nOc)6`yZ)gUuB|$FCDC$j1c#{Eu)y#Kg$Rj-IMublAdbo& z0r8rs`=M03%6eSmrZVC(a4b;qpJ14J^)nns60=;aylIjZ4xa&1lpXNcG}D@hon8}a z#E;-b@dZ{A5wNoL;UQt`z{4o(xhjv^11G&a+$U{^Oul}`(Sb@bn_d~d!Sj$w6L<-W zQ4+pxMAndunt#hujl3gptN$yKNO_1*(GwB%t#{!MIXY7?R_@>-e?B|u_&Sr-Q{0b| zg|7bIR8=<_W7sow#nh|jSZUOB?(7w%stLJC_6F~#-uZe%bLH=htxJoIZQpP4F5Hw; zlFl^gU%vdFjp@lnF0b|E$CMQ^vd-ED2J?QvIM*b@+dFKZ5PJ#!Lj@%&^h@*l4}*Y2 zm_xO635++|cg;*Yl=^UNw!P}_L{0&jZ2|0ugieFClX2|1bHefw|JQ7=ma3?H zFy9+}{zVIUV|TOTFhe8e1J|)agGu)r5T6fV2T3pcFTw6aieX%n?bqkhVz@-N*YJrH ze(++%+O37PIAkW|Ftj#!C2?p-@M52FTQ2~L!T^Wtf4*^Y(kNr7(9gGdAv1Kx1r72B zG}Bpjn!71`9=P)*y2bcg)5e03Bnt5e|5Xp6&vJ@*dbKg~zxx)B{czg!()EC!>1~P} z9(zC2Y;HWUxkp_7FY?wgE#=0W5BK@a#&4s(|0r3uopF=ZGnp6dMis0(9~>$cTGU#F zmT>bO0D~4%9E8;?&;!4!3wWPMl3*m?Rh@yYLW~cMnlf(0mDi9%pU@~eM(mztorrRv;I*s)zQdqmX@d>B)yeSDEN-g?4=m}##UOSCeqi;T^6go-Lhq;s@&VHPHQ_p z?cfStdh1%gr_*{WEv#RLT?HpWAk+&(YR+>oq%GH60txfp5Whbx0%N zKFo6+TMT&^h^5tvZD)*tUz9iHot=lPry#|g22N;z$g@-2CbTFvNze&mqGR`(JSM8^{)aPt#8dSN8TK@L^ zofC!XfRzzZkXtP`2fFAl?kxndazi(8d`_|lCsPhxaWofsIJmuVLsVefw!*eWF6Bk6 zhe9L3{lJtZ|C1VXwX?H(-2CKUeZ=bO>Pk#vSd#iNN7oSz;91}EEcn-{_1Rcb*O-;Q z;FP;rjZHp<7Q_HjMiihVD_r*0yf`E}+4N)Hn$QV5CZE^H5V&7*6e4z|gu-jHE;;a) zwPo8sFiLy>Z;6=}-#&c8c*fzZf43`btfxGGvB5}=*~o+&fsu&8~2M*?dFI?u(u zHRxAJW%<3{%qw_>)8L}>v)r&NHBY5zYhK8xLIF;KVzds4qF1%S2O1AH2T`d3g2Tdv)m?^N59}&~=00V6Q;r|UGGd{>FBgqsL z#!;J&@^qCuJ7Bv<5P1ireE4KP0tFW~ND53>zL^N+kP3Qtct>*Zs{@;PM?3@gfXInq zN2CkiVV9GHu9sjK5Lp|e9lk70<%V?*$r$En1mP@2oY|I-K%ww`nVsy{?XlrvQ z81I^4dV2CoGuw<84Xb9)Bx5D6GMP)xGPcv9a&A}dtnD+RW$Y_n3?63gr_OZ`S_-zj z4j|@F6-ST{LQe^gh`UVZ*PL9qA!8#5M?h+7lQF31LLQk9*@yCenNBY zS*J$oWy+JOCQO;z!-gA!=$LY`31gIEW5D&G;pPS^2524F6ASvi&Ks`~6-Nnjd9*zr zL7i`sj9>c&s^or9Ol7CN?jC>q5vxJ6ZfwGajh7f6*HNc1U&I~eE93EOUcwrKR&&CJ zAictAgV;i}K4g3s1w?bJH(Vtv5FO5k>kRQQ>|hZ7@pttYqVNU34vHkDh~H(>Q!Rgm z4Mj&&x0SuSeM9TRRk*`sE8YKs2CngCCg~(&#Z8v$+~0nb4vAD z>o@VWDW+@4J8rjf8`K*J=r8i;i5kNX< z*c;$GPtuNjW)=Dy=jTCerv$X>*ph@^OtV&^CYFN#k3RPi{kw4TAPF@^RD}mT?EZ+8 z08S1KCkbr(-$}sA#TA6PDJZ?+SVU5gh4>?%Ib}t&d`ZV#F-Ui_&vgc%f3~hMhgX6jnK7Ev{lSd@Hg)7uz zu3j&Alsrux&PV%1->7|CuezSh6N;sd#}e-Da#O$D^mA*Tu+E660EK~YhZ?HAPblah zYGa1o4-N{N&6};h{Kpef^xq$Tq<+{ygNZ`t0WN5aLA-;$cTkp3>8XhBx1)}Z;&>>~ zDqV?`epGo@(b4-?#__uB;nPG%J`aU{zhmhd^ zmc~sFc}{O(TUjOpN5EhEJz&%O`BnMD=OAKP1;t)cQ$s&}pNbO6Ey?L0H~$~_I< zp=g{)D)2`b!L(C)&1veugYYZr{O$<|xL2+c(D(`|7$=ZHO@en|VTQ3HNG(1V ztQISjdGLfv+BOQqm|;G@|H~Cp>Vk3kmp(~=-84|b9& z1%fk_C+lspWhEXaZ-0I?_GAC^E00h19o(Tl(5tju%;q@O%6?)pLHm$_BP&yAi{&ZN zBWRNLFH*7OFN%*xyRPT)L*YvA^6lKI*tdy$<2}VKtRDBQ8VikjJRpH?#JCAq_DiS` z1I{VW1*&6UB>Dkt;M^c2=!I6?F3GT^K-JZnG0hzk!WuHMx`s%u(^$a(K#2S2@+Pp#cqk*C2FWpszBZ8^Y4W~PsCOoTE4EzG`69=qkKfBz zI4`6Yr@!Oaes1lv_%dHM(|nHt81R>17Ex)C;C8$XxJ_b_IQ0EC1v3k*rl8yqLavRo z0kllKe+toc-_tGJ-xk; zU``F+o%Cdz>LcZX4qI0S(d27-6*2w%J@~3UuY_gfXo zQK`QEEZmW1+|0E96l3fX!r7_5t@(%_6tXN@la5g_&Si+9Fr^;6GaxK)8}sLt^?OsM z0(9=GUE6{;SKq8URS~!6uk1yAI;SfK&lN4Od<%)bS5JJISi=>c;xAupu;&Lg@e=yr z@A9;F(0pxPn`g0MJEa!5!*an`oial9eFc{6?8TkIja7V}l6lwf?pvpM^2F}~>0cX{ zn%R?E_3t-NJG~5PrCe(Mm0VX!&o-AZ^@Xuyz-m+Kfu^3GncOQq_AGO6o+o3zbv|o* z{#U8JV3JbJ&&c&1+L9XQvWOXF`qw>&?T1TX*Zu{+hggh9Z)FI_ZQ)d)4!n zjKlGDUnV3kFz#Sc1|}}yA*7-0f{ZH?CXNjVnD!qaxCfX7l(_=yc1`wdrOVmb)0PvF z_)z?*UsU1Nmep#FA0G6EnlIV~1XBHFxb&uT;;*`Wer;(n+AT_RXZ+_0XPcF`X}awF z_Z>~T(nAf8=;<_$rVH;Fg-n_Q(9M-jC&L?AG6&NGd$k!SIUmNaBzs$HQdo#T%AMfV zb~vRt(xYqls2zCQ1h)J5u13#`9h;2u!EzjYBd-iFbjU`9#u;-${K_aw}1cq^`J zJ}|IQjs`%xe%_Yre+{u4#6K`n5qsfvh<@=^W$3A!Eee^j7k)xQ-H)QajWoDwpy=AV`|YtS2Q+9T&>Zq4z(G9ag(mfXZLAP$OLvbd$rF zC*p$VDlXn(k(&uijT1a|H=%u@j634P?a#+P9pcbZqQLZ+H5bXmZvFnKC8ryIJS1Cm zI8$0m9)?M$Mg8E?dNEAD%Amu-%1Q(U|Bv=1fezf?3YG7jKC*|_jj8X|&0CW`;cv`%*pup^!;Y%G}^zcZMw}p?`@RTRW)9z?OD8M~d@!i-#xUPPUcLCJ(;R z0wsWT1Mo7e3MgiSp+p@(ijNXto{*Z${KE&3*uNfdehE<3Yoj)$xN){iU1Wq1hn%ox zd3d^H{sT$Khe80!A17{Hzs?`@kc}%cMQ^M58x1xA%)KDn7|g!$d)uwx@n0x-Jp0GE zygJ4EKeHSWU_gdNLvkkmL9#D#cP}IVlvqW;I9~+@ByE}8`TxOgxOOi!r8f1;s+^d7 zy5Ur;qvvMTaaOmm&3t|1N~|06&$_HW=rI+3p>eA7ATCznq>P#S(MQgGtTwP{T5ymkjVRo7 zl}*?9&>G*p&_iolC-v~Is5&KCwu%btD_S=#aI^5_IyW8*ecjA<-N9Vk!eQuW(59_- z10Vx*{L{k|xQPX%+UxOhqrj&t=Hd(o&!EuzQ@oD2ePLrXcnmfo)A&cW_A-PRJt8vLDwfvC#dK@on3_ zo#wv_MRr)Y{?9VQznC`@wC$PY!qF6>^{n$8Eick%+QDhG%Xa*rN%v(x$V0~}()|~v z&HVxeu9P_es8u(}qtMi01By`!L_F?-UFleVbBs}MeYyh8(mtliGR=-76M-vd1!}`; za8mD;X%zN$)xB)_WTGPF@7d(d*NrXRCw}UuHl{{d$t{{Cpe`d$EVKbQglUma*$dPH z*~PF-flz>t0}^tG1yDti!4-(Q13Lk__#qyxKJp}qzZA-qbT^F*jBERXlAEOcfLRwh zbm$qJ#t3D708bPs%rZ#m(Dn6&*=F&b%}1Oo@S^iLi)9xjV^Na->oYCRf5Sxqj++lY zWgvmj5TDmDA(JMc8YMA*yu49Aw246r0lV;#zTveK;Y2@96gwDV^%UJXW-^e#9zfUU zpCRq53&Syd@H@gJK~_SINYSaneLz7zxX zzl2-9cCz?9TFkCG{#n>0#792cdDXqSWQWrZ>!NJG`Fy{pmBKp;^sYB%*`DIx#5=xv zH)E6`MmOn4*BRD_3wpT=XQ+}lE2)^yJ80frd&(?s${n2AL8&Y3 zlyxaK&CC=Sl)C;Sf)BiLwJxfZJUX#SwB=NP-)yd=!hW+qzg};57jksekNK;=tIBb} ziz$>z)9A+X%2DdvTUCCHx*3}K>ITQXdfGHtRCgy&PTek;h-IiMSECu_;(7DEe`=n+ zB&E|wDsnA&xtg*|Lhof~|J3rr=05utnm1=EWN%&-LvBX{n-5$;w!?cw{yTEyWPiV2 zZhn073+ElxR19&#rx{g01zb9Hke!ijUyQIZv(vO-tXLhlKdGJ75zRQWxaBxS%c~j*WeaT-PH-tKi!=#Ae0?! zoa~p!Br2te;^-8QnoBrVH4xtenN~KzZR5AoZQ3QW?&I$o$;9>cYdQXAZy)4t5puNC zrC+iYps1boB*5(ciebmGI=^2>T>I+dd2AhYvbd#$ET_&$sZKLbx0spawWx=;Z)s6D z;r}de-IF;EW?3ti*Avoe;T2md)VAslx$xZ@mFV!vQXC82oKs>Cq-VWT8bkk-wte5h z=+dqnVb}0b=4eR2udf|gv8xQkp^OFg2?{eZPYb0Qq1q7yKn#dDN@Q+KB@Ea+e+AYL z1}0Ycdf~w%Q40tVgMTmc#`FdFt`tcyEtDpR`HKa%$hzC8O3huQ&<QOTnvBFPNMF^hvlnT&vf`hqxg?HfGejOa$4#pbo= zx)%Aom7=`abkD%SLPqyHsIyG$QdBNmx-xDZ}3XMv0SSCExIvBd&6zD zT|L+b+a>kF7e41ruhng1W!L+udBCkuWZiVx35@~UF;9%C>yGqIZL~46mno0r4Swmf z`L>#-S)K~Q1|&S_LMHlBAU~-8ZzVr){-9BR#E{dHAm6;HRPV32k9Ee1uh(mOZNKv3 zK%5pu_a+WQk=MHO<_BV1Xl%1;F`tp z@Xme{N0C*hKeg zxz(1sl~<~rzvqH#b+My`+jDfD9qd{4ICk9Y*F{#B5rwELAJ2U(aM}HBu$o6zBKu5A!rPKN z(V3FoW@W|ZJd{|;NuWExVf{m4&wxKPZIO z57$q5MFqmbIl=U+p*-`44FTSHAgtVk3B$Y`qW7tXVR(;DBA>Mmh6-QwV9^<%G2ICP>$COJ4HrXO>~+0Nu#v2Vz~N*rsM_Qek|%Zv;wL&J3rgKvx~iAo2+ zWC~(8fSQ!Hw?Bv4%O=kr6%#lBB8DNj7R-Pu>gvSSfZ+b%ujl)q*iV4)CE_=Ac*c?Y zB_9!i&yFzzO4M+|Dg#hmC*$~&(&e&I8I0t})(=E91=;N!2W`vCMT%=j3fGL^TT5|} z>2#NyU&W##Ba_$21P+LJfBt%|pO54ikbKjPOp+l39fuKltnTFOd>%junSBanwm4`E z5aX9d-kOPKw$klV#}?PPsZH+hjA)#mgiC>DCj}$e(CvBd_5D#9C>5=fM5&@g@y(Ju zG-Eot{AhWhs_<^@6F0>IpK@PO8SdQn$dP+=`i=q}lgD=E^?KWeZ3>HweOo+X%ypwPh)tn`pz0^kT~UUHR@+srb@Z^5Xycn;YM4cW=YUOrzZ;n{dN(wKP}E zk7@Ec{ylamyOd5^o6|pt*)a9ZAP_7HBQ72t1o~IQ_)NpV@N!~l!m8`thLhbz8um*| ze3q8F2koDY=k;~SvWD;GyLT_<*rSA+wQECr{3!;XOY1cs=UAuIQ5>A>#$cWLNU(NT ze4;DTS-qIiZ=|K<(?%v%uZ-Is$95*Q+kGx@mktm;`LUtl(8K8=kyFWx>b!iH!cq!r zj7!VPWZ}dk?tfguGi}UjEVS=wy(2ktblR8?T$6mpyx_kuVs6syKqo3d|F<&!h^KF| z>{+L!guQi}-FoZaD~O7nf-#%Nbz$A;ZZPDiCmkFTDG08ofbf9K zxyC(?qWo+4ZZ=|hf!bKep{wAc)Uf9)0so# zgwp~=?DFT^+#Xzr#PUVee+^^H;K|a4jDS%ID%IMNEhZmsMSNG^f-;?x#NI#)90KE3 z_GpJzgO5u?)=Q%_5)4I73v5zSffHRr((&zR1w)2aCPoEid3ZCWUwm6D0*5aFN!>2Y zq{N$j1h{w_lNrwX&gj$d{F6I`%oLDdFl%m*(~!O(l9RZB$h-__L$Ak+WfOH4W>Qsb z6jcxVw8^UJhC^V9^WUY4$hH3(BPKdGv2yt32bI>aFnO%&8;{vj&pq;Hq^~$-!?PGi z(U7>#(^Vz;dP&2}R2Tu7yhC^J@ZEP;zkz=GYu*jh$aedkvOW|MV(TKryGNz|Cz~=q}`vCnU&A&!uFHNQDvq z&1|}?Vh?MFk5aRmnQ2(B1+kt}xJsFbE4cH^u_|8%wC!x`8=^hqMr(P7$08*E+s&q; za@uWm!yx+J`T2zHhmV@%`_vBq8Ef>1yBl?>#QSd9uPC)J3^3;Wlp!q%kAX6oBCU5# zHSTVYnZf>!)d`MZ{k!dNU4OmltayEfhWF)_W6xSPzD+Y>9rN8wy*4G*p)$?hgig4X@M)?-&z5OecVk z&dP&^H##ThQ+I1~g)k>)H*wxM^LMd(gOJxQpkfH45WhKdk#wr42d)>}h^=;A=*d)Y zP5E8cR_JpazZomsyo`Y2Q%`BT$r%3^T*MOoC(k z5mq(`WC#D7jIacKe<)J>_V@313E>KPA>QZbbK-<>C3&JqZX^{mB+VK8w~4qJvN@8{ zhFL*)!(?Iw3UfLz0U&ubm6Yx^4c!OEjoa7;iZdj$o;kE;aJey=2|J&V=qT-x;%QuP z{oR0%yoe2eAF{8JZG*O*bOa>Qt*fgG%4lMnb#l5;tG+@?0Mz10t_i_RiWsA`;o3Y| zfdIC6m>ghNd5ma}NW5ueSKTvvW7FCe1krIlrn93kCYu~T4qtUpef2fe-ZsN+5B|}` z;h2V*0KkqQs*oa(`3PiB3q%)u5g{t2s9=0q!l)v(BF+$!>I>)XPnr}KkqJCd8sjcL{#c?Ov^PZEX72;+cZ+cbf za9qjQQsvnq?;`Jy8?jYQbcSoyInNmHPfCVbAi%s>9Ovm<=E0!vLS4%D1Z9pgM6;zDhsnnKi&_S|F@dRlDhquQtvVg z>=LD&8hqc?1y1zE3*7{cCPn$1Va^s^4J`79Nfaek-1nKHAyzOKN$`>(A?t7SnaN*cO(^_|_I2%X<)2w6a-CDO z`g+rT?d#b1G3AmqWKPM`dGKjSC*9Gd7`^qNujBQm#k9M^*Di-LQ&JdeUaj8grIgO= z{BylZX+Pz`#IyBl#kYryVvWuUB^6!)l|I~*#)b=kIQ9YD>Bg&VIXEgQ;hq?^ZX#)A znNv<(ePS)7`^x;0vqE21A8c&RwBCt@i-ASlkUK%XL|l!)iG*;M=qN0BVULrres}_} zjlGl3LBh6SP^1PWXRS*YEVN%C~&JYNuePbIfyYObg&mA_%{Z;8IoHvEOG>6 z%=R|FHq}B^|G#NYTOb;N4yO`r69s}U7$H%C*ngLS{USz>K+#7UX?Qe(b+k{JoxqeJZl2{M6mt@faGwT8#krjA#-kF^;rXEQCS~K(f*eRsEOpd2#8Cj zF|~*ghk;^YMVtOPNHxU%30&KkP|W}XBt<&(6wn0+)A`!nY9N3FsvsO?&s<}TeW|-P z%zn_j&cDUQmcH$wgt@zJK5r>~tnL_Nt~?X)~A1kr?TGKwm8o! zhgpBV%X{*rS|8sry7eA1GB<_7Bbddz&%Th7xZ77;Nm1%G+$LlA$>)w~{eOQj`JC?2 z(g8M1Mh382VG~=3n~85uf=tN#pIA>`Qaz`I2O>4PNRRrh4UG-FS6qleBd0@GfEMj?U1&?tC`Y_qH05Ga^sd7N}J7T{ZZqUSQ|j z8GDsbfG1&9J`=zx&A5NRMg#3>DtaCszvSeE@^vBm5;k__J{tD;P6IHb66YmJMk9$K zh1+00S@%j|nu2wfL%!I9Azxx64VMJNHRpO4#tNUUjyGN&R7%-3^~B|=Qi$4iJ-Bg$ zFX}9hPbW|1T1_zsf&!!%O$-0HnjZ2rzE4BuW@LGMOGtJEbc$5fwOj@Fl&kfFLn$a) z%z|JWc=Dglpj83O`K@QOYU96^Os^=AaA6=o2q2`#@+YbT0!%4Wj?8VpXkZ?soVvQ! zSrIg2|E9zs?Oc=ghPyl$j;{?+yvVid>#G(LqWrZ*{5Kod{k$KshWFIsOIn+rBmGbO zM%PIg092r)AP@}6yU%qU<3NK&g$aB-hl8<4iT776cxN>dfnW@l#r}=MI1?gS zhK-dd6Cvg&q1K2Mfkm9z)6)}LGcq_3|A6p&Z(we9NX`H%{d|zfZAm5`nZSsP76RdA zWJ8icZn(%CF$C-^H?0U4(t%O7BfIz(1yMBO*ugNWw@rTH5d%FvWH!i;57&fXs_F+| z6pNdbXlf9Ocm5{-zB_moA~;2LMcnE;I@+4G=j$OQ!)8(L0oKqK;c#QH^K01Lji`xfh401lw6G}si>{J6{Y_o#apudPyMls(V5SW1UYk@T zk>s1j3O?dMq^4YN*Ohl)toTe>^x85J*C6yymd(U*ors8t$+hYO@_!o*IA{ixfxkoNw#f9puCVefi9sIkS19iUcnNxRpn0PFt zcEw2D4q!h*=jYLTubCn5L%P>G(V+H+0#W|BHmw0Uq2f z_L^3Fnm0b2q3`*k%3I8BJz>V0&t>uB?U(Fl)z2@zNuKW-y3LC@_E$qfL$}wT)aF5F zvUX(Ep4nHJ@!HhUz6lx5iixg|OAGWAS+ZYGdmh+Q%3L#~Frp{AYqnsL`plLLu{#~Z zpIY7r-#)tU-B6%FZhna4iQQd^XB$ml3GO*{T;Of|dQ*!-T)`fUvIR~iN9^Xz{iYYb z|Jm2hv2a_Lb%1U6kz%|&X1RS2lJ5EfHA|av$xuqp1{3$}sO74t%C%}&?`^(HZ;n5* z>885r6 zjKZR#aY;!&X|MBW)0$B4uSelMSR)s_&wlH`{8fyaLA%a@Fm&+0{kVnImo{uPN=!%) zM;LQzYAT5*gvWqiQZn@UmAqm16raZC=BcIbwJ211PD7Dy5(qA^wY6o(&?pTJjaxHy z>PP${^;YE&A6Bi(W=aIZ0M{wkuVYp1h0YU&_+BzQAS+8sMepZ3p`mNtojH2?`lZlM z=8k2z|2nB#{$Z$!1Y0L0Bes|?fVPfkt2jIwuDIy4cN4mW*2b7pM*9*_1jnY-;* z@`ExmB)(Nv-slgK78Vvh2Lb^m<7c=xrqI$~9se*iRdHl>a{hR&=iSPI#V%`#xOEhI zaViTMOq6$RqHg%$9FE@orXZwvWz35hhSB-#M?Eqa>$|g$ z{K0y#y!gaGLQ}_N9~;!=BnNu~RpTDj(krY>H@$K#c5P&;S`e<`I>$xfW2k%nVmDpU znOgRer}Rk$JNrdzB84NVm>Mp)$c+!|nu>jC`1ST!j>4hbk!F!de!q2?QDth8?w%v6gjG=J3Fel+-6^o53gG8?xRa~ZsMuw9 zSF=%p(a4;_T(n%tx&KJe5=z3gp&IhRnaS*Uxb{Q+`Sas_p{JPlUH&NRxnEN}P*C>a zyj+diljngW;V5N>A1@u${^eWbaIiOZ8^15-g#O7qyL_a zzB7~MqFiUMEy1O8E3og>hMcTc4gL-{p4bdE+vr=|^8*_xM!Sonp{A9@y}U2)7jGBt znVs|os(015QErz$R`yi!@clk{1#jucn<8Wk%QjC?eVB3P+CbZ=&^`Jn`I4cGN8Dqf zF1q?3Nq?5z6(vFhMqTB?!^3y%Ib0sSnjJmrCAw+QP*3kR7SZnO3xx+?UASu&!MlOh z&`lXpL#H8P!M!O&E<4hTK(iwkEI&n!nFxMgJk(_s3KoB-Rl%~2!Ywo;q||(MiVd07 z8m6XPaQ63p7>eqTd$7+E3jdkkO%D*t!TpIIc07c+SK(A4prK8q%QnJ^A-Gn3;}bPm z1rv|IP3YX20lS0rLRjzT;Hx`Og+mW(TzK;UrYqB&IB^1sqN~tFsW;lJHr;+=a+L=B zo+XMCfQ4J&q=x#b`tqB?&j@kL{hgW?*V5qB1B=mlSbynu%0J!xI``fwbJLHty?i1f zt>4kAveEEfx^&56T;R!6Y3B3?IBAXFS#I?9_NFj2Yzeuj^^u$U?S&P=q!T+Unz`Jc z?ss9f-g>ee*7zY8&!I!6k)vI1|7lpM&%PBnawJ-7-v=*ZeLBUqdfz}>)|)gC*J@;a z$!15p;xt_?9nF|%iw(T3a?aq@(!g)h4iDH=!awoIIs&cNpO)g)aKM}xj2P2z-^`#b zMX?JPk=|`~R#pGxR8)ck6$7ZldZFjFLdCZ{n^Qc%;5lD5d2rZ9N0?P9ttqZyIB!P@ zt>%cR*F00IvVx1Crj5U7ze5d`s@j!hc14E`id1p*JXel=S1dd!9J$1#pL9RA{n4J5 z!l3_d3$jbzoIS-K*4z}MV>qzvGHJW^S%-;EgnF_pFF)HsX}BZ!$4Fk3a+ZhcOY;pr ztZnV>!8dl&i={Skg(S-g@4uw2Q`nmkX~DY7GiH-=3&&Mq3iBe)b8X&>$6SjtPb8f; zc#7+2(Ck8$=@{YW$bi6p%X4d#x&~cmmdqxa}51LlH zv^ToP2eyiy)7ZuA*yEgcDJ9T)uuyGwSyevCJK9mEW2Hz;L-=wFui15r?}6^o42fB+ z7Qb}E7H_^i(-b6A};v#i9(XGQqub{jt!o$52q+uAV9SIckgU%Wu~ zLM;7gw;1R5rIG#Lw{PZPY7kMTu@h6z$ah-x5PAF|v@Stl*Jlp@`~^C?i9J*?IkJ;N zFJieFj30kk7<>*Z3==H+NlAhPj7Lq|9N+!&a_Mys+DyUOTlBuAf2MqSKGU(@xTLpH zvLY+<%8AW;_wF@3DLvVGswtu3fKy9Qy4QMb6D}^{kkF$ai9XlvV88R`a$`!Bbx}$l zVIK}#+%gji;0=BD?D*BI5-BMutXsF%<(PZYZr*$YxAKSnhjMS*$2zW7?)^D1u;1mk zp{ADB*Q%;z8kyH_5OK_FTRXqyI901sBFxBI0)+V1*%En!MPP*enqv zuk#;(`0;ZuFfhQ)VIAKmhn1C`Q?!$)&FnJ^|nn(g|ZGXFsa}W6P?V!K* zJLP44bMxnG-w#)yLpkQ5ykTy2Qpd!-Y3s#cVZJPsF#8H;^l!3ldj~uF;*3b3U8Y>y z_QQ768F7F9E><7skPb*1Jg6VSlHy>OR6Txf^vn2!Np$!3`tPSgAIGKkW@XFP4ri5D z7zh7i+4aVa2fu!7LW0-j*DD8#mpUhM|2Cf$*}c2JV7kT%V=dT&K1ZKRddL<1-M4*g zwQKIv|B8l(n*rw>1~SZxQ82Cvw`;|MF4CZ>z^!HBCrY8>Kh=+6u!{SN<}C^Ks;Nnf z%1v)*ZLWOG6_#k;wI{oyHk_af7-Cdv+s^2EzgVw2`&6lLbu>>~o zJ3hECpx#aQN)&r9As#6@rSNSF1?SUgWs#Rd3 z?J0J#daXP!JOA?mUb@0?pof7n6(;sKfHvuep%3RmsQ-gZ%RG`iP|DXlvP`kj==^0PmzzGPG(!l5cXrz@ax zAYi7|R4~)_+S8@i`)eBwD?orPAn@Zl!Ym^^n!T6*x+%+C@_NL!b9?2$lkWp}sdo1G z+jLzRne9BEB=q02BcUQMyT5LgF8DH=emOZUO@N)QzOJss+$`0y#QkJrOOyWDvsx}L zdvjJkywEw-G&xQyiDoJPrWf!fM$>U)c-405h{ld?J5VjGcHLDlAPuRN(F(ErteExe3eopx zBf5-N`C=d+Qn2+mi)_?eLrJ~SfV+rM?!!;6L>?JIz48^=)S1bgv0fHI0uggQVjk;) z%*>dzWjlBP9s#}a4J5)tof!}ENSm7ODDvv}iGqPHEE^aj-?ZGzK!Y(N)K0s8C*55g z5Nsh^39!SxlWILBe{PSbVf@uV1n>uj7xe!olSdaWUS9hlmO8et?NG%fB@me;WFRLb zBz$q*Ven~erzD}Ac)Pvk6Jv)YJ}KwpH~9yZ(zSU1(fSFCQXj_dfCxvAMSP<>-9K)U%krc>@^n4*FG&n z>7)2^6gU1Y)27fogt@ELS{x;VPOG^S*H;vua(TW53PGj zL1Li>S%02Q<@%-lRd*^0U+OC=IR;{0DjXb&+`Oybiu^WE;2oqRMERP((hT(<%z^V| zVsSByMd-;i<;bpd{>TZ!Ad`+Hy2=og0ga!#vWm)8+EJ~DwrqWV=9&+s?3E%|TxAX#6V*<7$lcM-W7fMn{H?LhrKeGT~oW(FJ1 zZ4ab#+V-fvqCSatb;H$_9*Zt&2Pd0r|# z-}Z^vJ>Ttcf>>^VR^0FYT8QokYxSSJ)@0y8-vpn{7_1vz-jRVa2q234G3du`@DSm# z>JCGi0$?liN3$$~fSrm6#)gFNX4&4%+Ik1jl}y2kvu`F5ptApw4$HL%j z;X!%JmWgSl?GT#DhqUROm%)lzefbc>{ajlIKAlqSmUQIp)F=vsTj*DQ9T-~9hgsfh zuUi)y{U4VtTOZw^@zd?$AP{YKwjXenfaErSD-I3?Rp@4K695Yl0~yVN&@S zu>SEuJTQPMWH;~}jED&Ylm6GXV*}OBN@-93K0X6kI3rqy%r)Th z(gA`HL2^-nc^0E^c92W_i3~WO+8&|Qe&JEmL@CqWr>IfP_fqh(yc!5kb`=hhnXXUr z?;pwV9ra6oU=r6Slu|-yDmims#GEuC74qs+?OZg%aW2w)3q>*Q`{kLJE6}-w77hvr zvEfmpySYTvJwQ8?igKTyOb1e?x>qHrH-W~NxcCp*(}DgT(y046Ku z*UElrp{Pb3n(Zjb%-vd>$TzO>MI?6i{@}E`Fu!IqY=qb1K)QBRSSZTKB=K&KCSZPE zwyD`mXd_n+unwtvdN2`S*xcUzuw2Ie3A`Z4E9Qnkv3G0!2!1W^pj*N1%#q*gZvT+| z9?YW_zq{Xl#<7lxq8E`y48n{Mh<$cK&03kM7C9%AT{)m*r$?uHzUHsTE2sJ0@@|sM zd5VysmE!&tKi{!CGA+hF*-JD&w*BBIZDiGrHJeN=F)1a5+mjFgSxkYB8DwM3gYgKs z>udwUpM!$~8}OdDyKn=~!~)8&+1B&Ji_g%%?AF$FFzsG+uL*amI5DHNi9W#@dcQR*BjK9_x%x3$IYKAVY}g~(Men%Lxx^<3Qdnz3vT!(J@#x- z8%W4mC^-HWCy0I(!R{xcb6pnraIN zVHE*-dmfko0NfFLetw?OQu-``F4PU$Aqc%op&$-2S}>>Cwtd+@H~{D)s+zWK&)B|> zIpD!T9D@P0$PPeD0QBSH;>xUS)@ZT-D+@*lt_j&1}SWv-6RaZ+9MW9)yK@@!wBa{3fE7Vvememe_O2VXOzvJ7#&JlY^SUPYoX zzolZQOz+Lddq>9Vev@Q3O33MUsdqD?>Yz!gOc&X8a8YG90{8Zvsp^e7IdfLSOcV~9 z^4k0dT?IUSxokzTihhD@hDs221x6{Lkv0Va?Fa(yUo3)7Fa4ZOmf-n+NPfQ~`MDVsF;)6 zp7!ta!lOU<3N$7L-kRPkX)stX4;O4v4BcLH7DD%7XC|L@8(+vw0;K4Ahss|xn#wdf|s zi%&~9-32a^%8-@knC)jYR(x%v+sOXT%A+jvi?mz6JtSnSYx~^0fC%m8Onz8P|G4b% zBYi3392=zzn4|#TYhfsmLI;5eQUL@9^V*l(ZZO*c0sur1a=HM|K1k$wVR^aq<8{v| zr!m-HOwY|>qrho^WK*CQ=sCF|x32O!V+iX6blZWkv7JDCUB1rOeX^-WFnO{b$d=Jq5DThX&9AQ%xVpt z5p^q~t4z}tb52&D^7wSpQs3F4WWIToY9F*HqlBaD^^~FQa?N~wED{1Nm+{CIV_C#h zUiA;{v8#gh3-Aw$dh{MEPkTHnYJ1MKzkD$UJCeTKSr>!lBk&ZThZ_OE#;u(lWi>UB zsJa1=RH0S6r%aNcSLCSwzpG5|dIj}TJgNmwfL3zIbO4}JZO0jv$NRaq{JgwC;_vnA z0U9um4PX-V1yIq+P0keQ9~-hyq#N%qIJz%6m6qWt4y8km;0?xL?ekn=Sx70{4u*kFEQVlpPKdZdHZf{-KnHtWIhGGT;cK46JCH`52sTSD%rtCIfx&1^DH#Q;Tv9cc1 z32d=&P*2`+o_YeZCA^z(PJ9#^w^U_5912{3ZVO~#!Tk2v=^0F|p!6fW2HVLDfU_%l zT;awt+b>zw*^cQrQ+i)w=Eo^YFZ_YFv+uAB5`}V|qFbDtWME{}2b`;6@4v5BT^_8u z&8FOlb}l-OS3*oPP#3AG7&JAfzd}XAfy7`HjNh-0hszli+Wj&j_Q5nF_v%-hOzu6pA%RB7Q7f`p`UA}RIUSUN2@3Ub z;zG6V1IgqMT9|60SiR;~CrM2!TJCh0WfS3K5jmEoBu&14qV8%pddKVh1Z|;61zTxtHWJS*dAA>k3NjGAzmyu ziIKJS%y$7asDo4yHU{^0$`MZ08c}(M!hVEbh}`SI`TZdug)In|7#nPB!`>RKx?FzF z&7imD%;q;(HVN*;UkmujnOE`{^}5Y#zE7x)<8X-yxL7+`PQAc4+j=U2J^(J}@kF1CSpvSY?Jy z48sLXC=l;25V`w&#`c6ikf5eJ0My}%uCDwLIE1P7@mCf2pp0}uz?3F7EiH1DKtjqD zcFE~Tg4&$#W_W7c;~;%{@A|j?6c5f%Sq+~Ay)tSH6uWyy`WuFpx$)`F^w`e|DNa~W zOsbwE5k>Tr2*ugUGV;PZu@^h11Ex%WXuGiuxjxMcMt@v#ww)b5QSjLX{ie6aDJGyf zxB?8w6`(Dy0RHsK)ev;r-{1e={*vFe9{2+M*BAg$)dd8lM1!G#fnF2Q=PR2e-{Xoe z(#yV9Lqi#<_unSkdI6zT!`iIYz2LGAoM%{AU0IP+ZU-t>AIZBl2_+n$b0CD!ApW%( z0hSmJ4vr0II5EL z5-KYDbuw2@TYx&@S&AQFfZjR!Xx=9?a&H1IbDoz%{l4fQ)Fsc%_8`9P9aplIC(z@B z@LjZlX}+4{tj1)zzwX&6F6 zuHY_aV9Gk%e2$d0-sKAJcm`HhdX0ijKiw9-lPJNz{ZKeb$gXDw-Td}$0xv7yw}RjT zn%`}q5@W!j=WqTMi$H@PU2O?{fnCN=_qsPT(%1%VuE1(Oq?l|0UuhJQdCtPpryn;l zzXYK$9Edi)rv^ThanlI95vEy}(z*OL)gZLeMUqo|469wjmn3;uETyqTAIW=)4o9%u zD-y?kS>o?sg(6jSq>uV$A1mkbmnz0s0N+cjzLLN4V+6NBgU1`F=xY(%s&`Cp-qeE| z65w_3UY6~-W)nDoYbaoHU`qW^6G7Uz0t)4Yqg-VlHDQ#q)g8*?=Y&_An0^w^6pz=4 zB6wBp;C%RPr<^Ro&B3c@xw}5R)~vRX!z!_C+wjV2Zu24m-9MFAr!Pfb;_;3S`ab;( z&f_;n_l#+YQvj{YX&#h!{bK(Elf?W6%%y;O17m~;)L0w+*?y-~1Tk+ESbCCxXa1}Q zpAlT&LG(VomwPn`m(tjn78q>u>90Kj7wZMU$Oao{4xqPyN2{M`>MMvb=u@m#q8#ER<0|NSpApAY&BT+bxDR!)vfZB7tTRi|-*qncXqr zt6;2gMuo{&TXJbIL~s%jrI?-AuwTRve2c!#FS7WyF*T-Y_pHGQt}kv$tI1)Z{%#YQ zf}ZvKaJ-yhQF$3Xrx32cuFRTn%R*gS{BmgonIY+bsh)?Fm=ZfUa5pymohw+pK+6C57EH>9+KxPNl3b96L1dXh@~X@_gp<5GU8jfwyX8!3=2C2Ex(Cvn`o zJ11&t?<{+6FX|UDRB%;XfVciq+s3K&Q^ny!`Pz{lCYro#Y($4mwblCX0M(MasFr8$ zG%{Aa3+I>F7t~mqoQX#^_UM=W`&(y^xvM+nrPGQSn$YcS6TorxSvun)$_F zi*^G`%I?9j4cjFZ>7f;o^V;F+DGNy5A!|gB>khJ3h9~&D8UJL*(QlGUG$e`jPrT6+ zb=KZ=A6WhZkCcA^)XM+;J4WProwa%{m+k2B`;H7BLQNi)R9AKU3EGsk(T%;!5j~_6 z#3Wu5qOYN4SC3tp{~Y~fP0drg^AP_ytA*eXSG4dggF8S zw1}=tJOR^R@;{~xW+vh}S*{Fj;_LQHp$mtUHxL?X68$4X2YD|B?@j0$vn1{wF2*jJ z)}GKN4vg1Nykz#58xRHtFp1+E1>(6aFj$*Ep>ptjU<&~_(MB>mLu6!#=Y1v@*I#}Y zj02xn^$0_l$d z?NLaja}M%Pnia`G?h|u-UN3OAb;HteB*DEVlc+i393$+SI&9!Sn>#mVVaJ`KY6#L(_lCc2a?)q*P(^VC)v56W#t5flLQORO#>a z5K1n+qsW0DrORysR1TbZy-|J?YhA!Xv*O0FtB^lTZK7=RqkhPam+97NWxJoE_;7mZ zW=Wa;Xj*<3(b9#xLq{LwsECwI#48%Y=j$;FWoa{wbH|XPoEpn{My0t@n?ZG!3RX0` zJ4>OWTCQ}@YYhPIegJeeS65fa608~IVTlZSJXf6Ew7sOQy*%H5wHT1Txp#_STk#Ns zIcXapMgJf!@c@+trVUrgB>GCvLS)gwK>tRG@EV9rZ^r~5hj*t6(7^&}9kBJjg0o87 zrUMp2hlBqL;szo}_+!BYl?L4RpxFS?-p=0M_3mF%1K3(e7wpk91J=rhAM)F;5LXAy z>X6)A8tt_2Z%WJwdCh_C<%g`|#-JHJuL01a&O2wugLRN6vO6}psgQXyYD!&P;;+tr zeSd1Jj55uxo}J-dn*bsy9SLl5mb=u0iqvpv>hpy`ZWOX{bc>`4{fk;19NaBKE&{$! znhLQjzG6=glj{x>>-$C?ZNf67OFK7JZb#D1x*`P`b{qm_LBMYS++o9jHah?eHpjrg z5bgf;0|O2fC+CIxP4xeu;d%og{oM#-0&x3t2P*G!yS3K^F>pVFfIeb;b`zH_j8Nb( z4(QasK3?p=*YoHAXfUM2=~_EGd6}r<(JTIx7V{#5BCE*w#1J4|l9yGE=PwpLV*vMs zzMh$hrn_j=Bu!N!wdaGOA-0pugVLyzHw9P%CL2Ux27|D)+=e_{B@9t0tGVo4=Ar&d zKh+XS93<+H*P4+*iHV71*V3VORAAbN&wL+=D8A(;68n`QK%qnm0|uxws6Ti5iekvA zvh`e+VzLZyczI(~4G<8%u%&%x)hr82F2_v;r9iBLnk4aeH`sL=9Zcvc;iRU0ue7&g zT&TlO8IcCN2+)#tFU7{>vJto41&KWEDMBpl-2qNs5A&KK%#V3U2^K8jTj6ypk|5Y3 zeY%UE=fgEPru1~ZpBzU;UOZ6Xy1dCK2E8M}#4!I`-zOx-UL#k>>?8mCV!#ul3vFs_0B(($l5PPePl2pHeIQ zETVW%8X!Pgou?x^^lS&RTcGrL?Ru(0deD8xpB5_koc*YL>&V!x@E{Cgq(#2&>Yp~J zOi&=p6BLcLhhsDg)owpMH&G=xjqT%k3{LYX^SBSB`$c9`B6O|MaA%Ziwa3a*(PESJjA2iH$y;CWzV zU}*ow^|^ud!!s0WqzcD?2GOPT6@jrJ&Ez>(?vA72y|X7N)u>7LS5;`IuJu#)r!o>% zWImEpCJDlz=VXel8)_OG}b7u1APr17&Qk1uE3^Zln8=O~q z{!2Y@;}SQZTDM7lo%7?&ew(7N=a1nS+2eO!_JcX>5AqWI{SZ%7p!&l5H%=K(Du}^a zpILB04ki#xIMmnC@aE>9Vq@F)Xd3F1Bb%J`r_B{_p|XG!eR6!f9TXb({~_yP$#HO* zxBjw`M=ruE5xscfdl5?AD>ee<_(|BDj%@EK0uCCC)we%Nu@&8DOqf%hWy#wJH{t9| zWqfVe$EhadDFqCGyLQMqoLp~VN|8L{ZgnUXXVcaabS<+=eLPo?#q=4!qfG5X*TosH zL+c&xx|`s3Rj;I_)}j*z|8}?C^U{$M7^mdS&EN3z^ULtR+^F?iF4NjC(F$NS4sJQP z;pl3S7>N4HR-pXIrcQUPeC_Sc+aF?pYN{|>SthwLRnfWZ9GMl&>D~7#w2cMn{PDzK z+)RbLAC?^cr(fcDd_iKNCA6hatYrl5! zN*aCI=3bKJ#H$<*CU+5L(A=oJ>DZ0T{gKgRm@hJtc=nU7?kEs&1GA{{XuhU458;PH z&g13_1-}jo@4F#AzJKw`QFU@62*4a22=O8DP{3(_5DXKHEG$CDXbpNu5IQ1=wA|P) zsi4~yjfjYz@)sK{_7@-M4B?I0YQ-Dd3kvhXV9K$QTZmSzsHe zJNEgz(la##jZ@R}}&x zQH+&ia(ZX0R>xgVq4Gy^a=SY@D{U`9*DX6WhSZObOHQ_vw@NakOv3qZ*ZZ|jm~#q1svG5{&i-bI~6P&Jt|0i?hTA@_i|Aq8l zyc{d(sKco**GXhSB}9w)iL(JcUM+_K$rIVmolp6INNd-|mEhBK_~PQC?6~Mh3C>uW zxb)bNiOjdPZ^Fc-9HEOCc@{}Xa|>t;#t+W!9}iiD0v0m!dRC)!3#h^S*josm+5RCw zb>l9i!l7fQjJnUqC_zhf>LV5`g|n>s(F|1A2+kZxD*=rh!KHwaM%L7Usi>{q zvaZOiy0X#RMUv6-HfrDQfcg+rvWjShXmo!C8F3HiBRRRmqH>E)aJ@$jq%mL&3=QSp zD`aIgpo;mFR>y}>iD@-q=jcYabV*gRpeaXS%EcCzTV$&#z=5i#P!}xRav^(6l;ixJ zu98_US^iNacprTze?IyY;8vmRe6e#^;!toi~{ZWUsv`|cx zf>lt+&NHA3i!3ABseo7`QBJ;AKa72Hc{Yl9hyWE%ll16va8$Geld-Hche0I8Mm;;b z*4L#3Hb+d+zQn&%R|ATRYY8#93&BrRTJv+aiAEy7x{|zRC8}vt8nDp-5%F-4g8aL37wVbubFJ5D}B?MmBPiX&E2|P*Q8AIC4k5x0y5LMbokwg9@b0M$AEkL zmmTi~!;q(!<*CF`Pr#|0^~2zWkraAAgzpm(!*lV%4Ih@BId^~1jtB=S}7nN~A6XKM>@U5rJOF@n)lI8U5UqzT>;H2NXO>om3RNDv6(@nd{l z%Kymhy@6Qm>PkRNl&tba($bdw3UDdM17K9){F^WCcEFZ5=4ZOXaIF;rAz}UX@jswh z$j&bgV@E_}aC8hQNMBUGCjW#+>KBhQZvMmhSW@<5pePi~D=_7_@O}qFeEMnluAr;NPVHSJC7yNE2oRr-~)k?vHuUPiI*QnG1uCto81)P;uOIIi)A$GtKJtn6J<{_|nOR;)ZE_QTQ9QD$Xq zUkgXqMWi3sbie<@o?CAX6R3jS^I}9K3W54E495&e&6i>^9f3so5tBB*HMbjX{2rg* zLx;)~uE&4rfA{1R)Aya-AWVt#PR~Pcf{acf0N&TpP!nkUW-*1L9g^A8NBn4WKtiACzo-I&Q-PV~(_0EX_*%mNw*TFy| z1m>6LKl#`Ak=YE*09l79cL1^)6BpNIRkz{1Zc<}qV zVSIUc7%0!8fn_%kg7z(?zVQD1`Lp@u`C%A@DL@i-z|QO(EVY!seEBaw=IV=zPJVs8 zx(l}yuyz1WMv{ubF z!Zg*c>bTx2$38M?6hhI|89&a|-o2{PzkcCwx2why*-t9vA!KRSs~NRut#Jo7f_LU4 z>uZ=gh+$Z?gSH@BCsD45vDl5WX)dm~ zxF1M`3TASN10(NqvKk_7_|gOTn5@v~H_EuU>Xg9o%d#pS-jXKVQi%!ks!Te! zxsj|zRnf%lefNMyOp=`IN9Yl&Bcoh8=`XLViBMI;LwSO+>*j*z@Y`_=DVXJae57&C z;QhLtc}Ni zzf`QzdnV9hDrRLBMWzmaF^GU`78Y@gEHlcZSJs_s_YrQPvY*?x`8+rM-ozY>G4u>` zX`e{T%)90 z23np#*L}Y@hOo=gk*_ztNS6XEl{7M`d(!@708ynN=$x1cEn^a;BtXiUGDisR`ZB{E zo0blw##u!(zTC4PDp}Icf5BBw#)Y!)78P3Mva!bT$upNlq3!?D1POWlm8GL17}z;G zU!_nPkXr0#hVjc(CG>Ui{>`9r)d}pnOJayin^OfB?<30V8)<}2yBxMU3H%2c7oT0b?3&3 zmAo~N*q`a>C*GmVPj%ZmP33;z>aHCdAY_mKfo(!fY(Im7I}ogWfrg!*F;-kg8u5!Q zCUqER>%0IH`DEWLw*bozZ6-<-xf@0NkF`%GDdj;YnGUWqVc5>Y5E2#BgMikFl?L2i^QcOhl8RTy>7=)s)w~TY|-Q&fqoQ! z+JGCRI6)~#2dI@ma(pYe5tEw}yYoUD3=P$)lP&DgqMOZ5Ecrwx0<^{*Tuc#>M?s6k;UJb7PI7R*d z2U$TJ^8Gbz60-hZ(OW0guobj*HZTegMvZ!PBQ* zJ}Eq#26e4xmX<~Uq7!Kxm&t#d{z%tD zsXBG{#_(A%_p(b-ASI6x`DbFDRV>KY&ng@)7>eRd+zip3{bAa5xyRPN-p{ae^J3G~ z(t-%bD!{%9a4^;ZD<;x1&dB8!SHx6thCNr-^HajyfJZC)uv}M-@~Y2T*g)w1dkn=E;SPW4(4@{4vw`EwwPAfDAVMR+s|d{8QvU3-VA z5tJm1V7ux-s!-QilcbpGA^!#_Qvv6b$8x^?7q@Iu4{MO+ZB*TWh&vdq!Tie#A!z}hDT=!g zETg9DW7CC=XxFO@N29Xr-0 znKLuqBwt1V_AciO_!b20MidJ_39Fs?7HaAOeom<*w5&X!QG`(SNWp&jGBZBE+}X2J z{K}zRfma|cF`{~!tna(IneoN-$~SJJe8u$k*);dvwYnHdt=Cc4XtUp!I~(wI2b&vq zl(dvbfO!$bOd29+nbUDzLFcLH69+Xpzk2~&Yl#JL*4qdG(F6{1WkAZ>7(i;nmX3r0 zS*4Aij1~$=(tdL8SdQ}VkO0cfHw%j#Kr(b*y`yh(I{CLq{5@|9zcX{rFpT|c6#HbI zGa6Y&#^j);corKt0Jlu?ek8`qil+uv77KVhL)D8z{!<;>ZozU{?nb%N@I`H841&S$L6Ki$7% z%91rL!o9Hk_rDinT{JwnQKRcV5iGCdcL^xh$ zRzJMzD+dvHWyZm={L0!NL3o0$8fHJ%xag^IqvxV&1{f-_e)6O#$n!B9au604TXm8n zy#Bj?X>j}odt!P+)VG}Z!OhLMz>tdw46x#2j=5^HJfH;#qzL1xo>#*8=G=JZ;+acN z5BYV`OzA-f0SZlkK`7Yhr*QP$>O-6?P8Vw(m^lD91sk;S8*QTlhNLX*ro9LD{{`CH`b!t??IlX_}*QTZjq=ixH z-cdVvNA~>i!w6BUIU#J{?X}p<^RIin5A1NUR%c(PxyJXdvl)<#6C822jVB7!c)mv~ z*65q&!M&`+18iV0|PNjmWbz3CM z2afKLB7q3@7$869?E-WDhdRhd`)6fB-sM_;;lUghRHu`)1nbBFx=39SAPmy1lE$y0e;}pFI4!fJ1 zl-w#Kha$0`5b|pAvI=EYNZANPF+S}d0Lf{%q(HKv!R~x3X~%4p_IPNw(9N>{+Q|-3 zCxFHXLZ8?Mq-i0VMPnKIFkXziHnUzf3|Nnno3B54QFX)w%1x87+|T4>QPCcWuc^^h zq1!|}|Dp122oAZ;IKZJ1^!v7Xw@1O_f93f3?W$3IPF2pv8Mk~+H48?i2XBfnhYe+b zUVAO*V1|`qStX2iG~nnt{M4xU0LSf&;QPJ^bb0gnd#{_OHCqs{^;ubYVDK~a1TL&D zXugugH1i`>Gjf`SXoS}YO!gUAN_c{!?Vd$$?R+|M8 zm2-NjB~~t?CnHuHA_9fpOz?NM0Z#H2&mj`|zsH;V_0=}``dh9V&xJ;|j52ilGcy6{ z>$%^K8QqJd`o}GU?MYglM)^o6yPpwn%}WA$XvQm&`VA{!_cAKh0c{F|Dy-_SG&({k zeCxAX)=x@1w@Lkgi1{C<3gYenQNh;RPy$%|#kCJR9_*;efKeKVNOc7yP#2J%>kgpM zyXWeR!Fy*PS(xHK*wj@N#!*8 zIu5{}u2S_DyfW1;SJQMnr3zFye`bv{+Mk4GmnU=ZZ6fXT@@*mD1xOhr5{=<*PQs^V z24ct5>^o95erz(|d2n~ZU7gq--uIeZlQK{fpx7>6#?3d{qe}(e<HI9z0U*0g zQ&J|HSjH-jFpawDnn$F&4w|3}O)hU|{+W3t3J%<`?C}Z;cqt`@vO=ju+~R|jy@VVA zQ_j!WauIct+q+O%?z#`xqgh2g3MDx}@N!%#P$n&*U%|YevLyR)f3&kob9fh3fJUSD z;6yhSs?h8kWnPK3ijMM@_L~DDP?OQi6o#^j1-MI=7DAUcsaXXhZ~2fxxdzr3`BMk9 zg4B6K+caQS1)L4t3Tlkc_zVUk>|4fpN}3wM(l~!eeuV19BuOc!K$l5(+euQ0n;DZf z5Ek`U{u=PQt+aEf58)>KXMp3m!UxOoG`AhfHO;^=wI<&6WVFzZ$Uu1?8_zBg0{yKqhnt#Xl8C zlF~w2L7_l`9N*jkzF5mM$Ujt9CbqWNgC5{>9M%_j4`Y_Hs3=*J$Mu7*qbtnBT(TWd zkOb23#4z?Q?;9(8?@jDDKJV^27=pF#mCBHQF6|P-VM33u-POH=L;W`k=N~CQrNL5Q zn4O(ZPzV(N^)nYaMY-1$tW2o#mM9PJnFdDobtA^!%lDEP0u&;!^m2=g&>vA1x#JT; zL5U#3;O3PxlX*z-YjI;M>V%reT)L%Qy<{1CEx{Vpk=xf|=EPbIgJYDOEQnfu3Hhpm zw0Q}#m6FotSfPlvxwQ1-CmI7`_4>u#I5{T9%}HTJ1-+0^+UTePaI^e`qu$2XKd^JP+*bafca=u@-TUzI z8Z1nt&e4jl+hq>uzP)kBGGoAep*C+f!j%1R;-w&QA}tp(nE++pUe?vt0#)V3pQEFY24{vb{7=;7XQ{}^fips`jzqd={3Y>v6;AV+`B4;wz}ksS%$I6(0j_)+2`$zsQ_3L>C0tw9EYRi6rdf8tm>L^lq8Y=O`2^BSJ8oEw) z+4s-OF2MQ+tjsW}rAhunhov@d9@;JzD7t&c?XIe1%gSZ7hN~$pe%KZl!1r9CMrh}-e)CQ0K8n0Ud2>dz#$B~Fw44E3%qObN zBQw7fn!Nyp)nWIn}hMq|r(C!cP(u2zTkl*KZg-!zv->9gt#HF9ftR^VeQ3Jxqd)lTpH z!EqT&KrRCSJ@D^nDl-5>3=koppytL@!s#Cy0Hb*2%O`GtDH}9%Qx&}pp=v^pkMjaBd*~m0kn&t=>R{J~ppL~LivOmcYZpG#)F~&d9l8AG}qKA(1vKm+noTWZOOShmY z;V@Px;Y)xK_ILNMkg8vx7|^gR^tMjE-%nWt-+61ti78ldhSGbf58!*JOCHNi zJ@kIdnr{?~SV`cU^Ys_)vpZ-SI`yJsq@Y(ZLqb6aZD49t2M%Q*rKE46@kJ*>nmR2l?Z=^Wk0(?! zH-eezAy>TUkirtFQYrLJC9gV;)0Vyion_ zOXDrpq}lQ*XcXRU_;9?xKnmE}1EN6+8bv8v5C6#qa`v0B^x6h=@duNQIOJ0p$1kd? zWXe_`Fjj#^uG74x(p4&i65EmB0O+w!fSF!njU%m9olK3H8*jEh&$3 z6x|%Wyg&*pPVW!y82a4oMHIMrcqR)J$-#wK0DLgukpY*PrLta80e^S#aXw))b&l z<}mCiOxTo$`Uvx2iuWLA=zGVhmHw;tco5)w#j zjuPh9);H|Ox*u73R=%=zch;?%-Vb2QRrf&uJ`ArQ;3*V=^M6&6Cs?GvzH|a;o5`~( z!e5ViSw5t3q4`vvzj^GS@V~zak6?G&VzWftPi84|I=8P1723Gfaz3UcKryg(w%Z;y zvs`P5Za!p=#hMAI!`1VO?8w|{t}@xUfWJLIh{W02IR_kRh({r2^qjo3f;AZH>hBj1 zc5R$0GG69ypS*zI*#D@W-9|vn#Kq0c4w&Xf8YU3Y;8N}jzKp<=_wcvhK-Oh5K$}!( z$0wio`}>LjP*9(`Jk4)x&VzlKlofQ#1&7k7gWsm6xdtaT08~UMtU2({!U()u=+-dY zvuP}{u*|E?3RJHOoz8`UyWEq~Ft*Blb~CR;RgUm2y)=65i(#Bl!+=2JC{!N_Co~Xx zR=-RZAtO5#k}X?05Wvg?#){;|X*e1UIvg}Z9zrwc;>leA&Vdye(nTd8`}B{=^_`&q zy>KD#RSpC_sV5>i&?r43h~#zz=`%4*G;?}DOW9ATG+CM*h_=Ha3q;t0$Bq_PHRO7y%; z5aC1ckc5*o5`d4v#{=MIQ}g@>r-WD;;oPm$$+42PhbR96mz- zcoBMTu&@NSY8emJi#4j|O->3eiac zd^N`{6ryf+21(jp%%~$)n0IbTOW{ig)U@nmkNksvR;`__N{6C<&>R^8?LmQ@}He=*r&IepEi=x&B7OW2#rtEP) zW4nFNi9W|!{WwRrI(iEfAOb7898GI)qoDG+KmE&~uH zrl)x~ui!!VAW<*{@Z9%oI2)p_++b;|G$jgl0f(13;5Df2TPmuGZj-oMBBe1F$ff8>gD&g&en z=VRROw>xyknBiR;o(2r#onHe3a^AclN92qY4erV+A<3MZuEA!Ck&P`DV1p(wfg1T&GMI*ofj?0+0Wjs+SB+d50TR9}HB-0Pn?t~e!Ad5&@G+h;XESh(R z%+e)XeeG8ne5!$uTX~0qs;~Aw+JlWvVnKwVbxkj?6zcW3%5Yq}EPZTxN?JPpRq ztng35l&;UF&GFB4JNo1knRtuqH;}kV)Mlqq4Lw8F7BPO_!G_ z5{wH_r&_D;dyCeECXlgYTKWFrCf4`pHdneRl#x<#N&5|4ij%NV<~Y7H&_jKLnPoKe zV{jW)&g2UA%IKmt-9OxjM@K^zKWCNE{45`k-nm@!b8?^}>0jAIj9>o{cwY4igkLGw_7!c1 zR<5x%Y0&sEI<5e1sKXUDcI1I>s~#X5@Bs(8%s#la!Ym*+3wS#@s?31={E6rspW&l%ep*EN?YAd}VCJy0 z{wQdG&g%o9!&e;VM>Bw8mkvBWH|WBS>>jhQ6qLD`3b@StkiLhv*p`^#_U*O8{k{ux z5Qn(<@87+?Jf;;?(muL&{6ijbkLszlCxY7Hv1T6 zeDf9(2bSKI{U3@iC#k!q#|@}m|0+$dV;tT$M_hsw!YBLacDa-VS^fF*ub@lboNan6 z;>I>n^%fn6&v;wgI$D-JqNm5hK&rfjJ>P1y*WCT7*s! zE%)yj&2Jzt(lsVuTgygW?ChMl~Sz{@a4MSbnN z{(bXA?%lD7k+VgdFMBOF*egJfG_iT23Aeud&+J(4ym!Mw&_ZYx+3ca^jOtI_{%EjV zO}w+qS@-?9owGyVHeOQJn>Z9`wVwZIZPf~vqffDl{kr+@sBAkg6F)K}Wa3Y+6`#xn zTa1*o3jOAzXrkk9Yn_GUpT!%}{hg%QU|*)`2!|CiuFemOcz*L8_c`0rQgzszsYqk9 z$b?|A+Yq`D^BxaZgav>bL^^is6~|6mJi7v%mG%+;=_;Uum`pDPQ;Fw`I*Ucq)OzXs zw1TfaopWN7QgQ)mAea2!obuHT4a^ROgn#hh#1A{ITD{ZE%m`;6(1N|;#8@1|qgAUC z#4fbxYY!YBaMTbuxfvgpK)FC3{4@!-G@ID(*}Kod(^fJ>vR3+XrMe`G(&pvcpC3R0 z0jEaMa3Lp^V8cvwlL0iyV_STnmWHvjowt`oyms0>xKJPtzE~4L`GUVjO5Z4iomH;D zH#_@8%Qz7v#q|7l+rTM#MS;)?X=$tZD$?1cCPVkA&?tnf^V5A%oj?4gK4wS|ON>I%K2G=ts{6;&TN|C}X0nyzn)b`?g zF8b>%OuN?G8Wp=Si?u`~@SotW(Kj|PJwhHgVw zKmTshZkW@x{2zuIatsYw}3x#Dm_mK58qxKlb*geEBI2KcWu@JKC@^nbFLnMl3 zRi-vDK2rd@_~jF+?^Z-=cVcSLxleV)5-0Uh3UTp0qJ9Q5FcoV&e~$W8Q!Dv_L)3(r1|zYuAgYhKXRNS1B|o6TBW}bJT_BMud#fV8b=c}Nc&vqW2^F0%d{|} zh;-j9acO?BoKw!$kRpX%JAyK{)P7d=M2-KR&X<<4>2>aD=$#k@k!6sddp+8y@F$qQ zL(bzzM#qZVCk1kTrqsKI?NqcxQZF2yQ~rAJ) z7?b`;K)~;ITUm?>rv#`(n7BXwzkfj2o^E^Swkp5M@9^WgJ@?PlZ`?wB#s7HEt})BY zF%J&xlfIbitLn>{G+?12yY=H=;F7|}lGzHW6M`MwTIU&nE2^b+l|KltC#-v@+fqnhq)lulbPZnJ-jC?nXMOzl1{LJv08Zib&Vg9S;5aT}> z(~vg-84W;p3xin%?30M|JHJbahrU#!59YAm@GZ4$vlm|aW`mZBd^+p!R09FZZFOdn zilbWpZxAz0@TTnWz!EXm1mi@{tV_r6>qpO@#{G#F3{j>iVf;$GvX=inw9OGtgN;&zoF_-UX+Fxy-^V@M$J|zC+#xisWpCdIw1&DBMWE6Q zVj;*Bo29DJb9fSl%@H{}I9m_jkP3W~sCL2?o~Mez(z0#@HdRqS7GYn43_coQZTc6H zuj%&0CL8X>k@F_F{0tlxH1y_i+9DbOSg1VyKegcb_ZLR(F6H7;|JSP*hd*Snw(*x4dF;7yav5$QLSVOz}W)6iHV-i6|k)FzH||+QUN|ci;ng z6AUtw9)$N2i=gMG!!&niSJh&e3DK8%3JnY^%R-Bhkz4XJqUHq+KF;#-xk6)4YDQL! zWKC(^`@6Bzuw8_!3-ihs4cdx-u8Xp5K4pmG>lsNIC0}3sr&+pghF)P(47^4Pv@X`(ydf&Tp7A*Rd_d_A>2 zhSAX3mc59YBQcnnQBbNYoV43G@_FG{J2AHwh4%PpJ6Fm7zlViMzVYuEmlM`F|P-TKPTA{k5*c8c4`T)Ap^q=01kNC6)x*6}jTdT+w0V(R@zd{gpAl<$i@o&i^SQn~M^aMp}84{ERH9ai+kO1w~I@n;K`fkA8J zSqY6_PwwV;d{^KTpvwN*@7!dG@_KicmJPe`)2C+w?R8EM!@fop$py0)RhcHfouI{` z#0m+j^R6)ZWBb{wM;pOsD|XJ;z`Jd->3_CPL$!MXk12I((AgDk&A zMZ*bOJO1NG7!BsFfZjKuc53dGm~jR}H}QJXp2`5~jmve{85LdEwa!|EYcR)6&O3{u zO|?(#znZnj9WCL&J9^XS(vd;pd?p5Y>$b41<7;XJoP-w4c{MFXelr@5G4z?YZ4q5P z#4Q+B?K&SKc>NTU`-}1%hph#mp-xkWB`$KVl3<+%gq{A85gU-<;E)MKOl5@0s}KnV zwj}g8bgkOPZ1C;(2$~0Yrc+5z9e&@`@$ryMdTVs=aAb@g>qD!u#i2>`f=Nl=Trn|4 z`r<(#+Q2aW6Yn<(>ZCWdLjB>TTTxAzj&mG^_h}v%l@1nENS0K|t;H!A2<)Z#+{SVx zF4)V|gzAc8aevN_Fu&&ee5H}(6Whse>n*~p<8qH)0sN@PFFijfmax?LT|E|5+If`v z@@bSdYXy~6MtO8XIp*I3XS}qrYIt{a5kejyGcMm?!Lg1gjf7i_O-FgH(s1e{t zCxWLH*zWpW5OqyDcT5SKPEzq#LT-%=#BJ9i!G+IhQ85E! zH?MzPMN|`vg)>D1__7=ST3l@`ZM1&-kxo_d?&IeaUR%|kbo|KK73;lVO*vf3{=fbh zuouFiew$w9+9Evt;pIbSHHwE%e&(g1#!>RKYgTxlf8#&eGn98SMn#Q{-SF@$t9G&a z;d-3o_4g4VBS(H|NF#q}@-}PoPMR#naotzHb8_O97x3?HI;UYonf$7iwJR{9_eTS7 z4l#1~0{amC?f;UlRQ0NK*uZ`Uc{sn#K#L5U-$m$dO%E;8k@nAD4dsbtLsfvV>1yiE z?;cMSW2<@k6Tw8d)KU|ADdFoQJ%BK)JYjK3W35yI?|gE8amS z8T78;Hl5s*EuAE=HK9LfY688%#F|xFpkL zkHA9u{W-zp6CEN^Dr(_sVnYb99DizUOb^55H@yhi3usAT_esNs)ez4CCXB)<`+)&P z=@hFYUrOkJeEe^y{vde>bJQ}?e=)-hHHm%8!q*@yuMqRBX2Lw4f8y6$bZT(h9AJck z#2DJpkQz2SsT3=6!yV|2>UcGwv6QWxOnT#3RMB`pbwD0oD;(KxB8U;2!*8m(x!>;U zz|mr7=UfC8Ou@*5$$eHu6&*H!tZ9_0fW4G$@SE8YHQ%fcO=7rPBM}5wj@+Twk;g)y2lmzpurOs%0NF4Z430scb&taf zKT7BW9?X}MACN(BUB>3~v}Te{fq*kH<7x*7jvnz%1i2=l)x*kExxBz1u;_n z1M**1jtOxhUf4CJ&M``R{u<|0;4v{pf(BPF^-Vk(HA(COwTuqYQA-~7P7Hg|{X2Yb zN$=*_*?XGtZEU9HCKP>@l{4Uh3HJ|y#1qMLeC@>1k;_|vcd@PGU@8x4M7)i`GE z--?aP&tFMO!EFX_fS%4g9iAww$ag>Bi0za2FZq*FwVJOG}nKDEv2$G@8Qq$6?v zEH;jv;@V+HeyuWWO;?UYb-VHjtE!@!P0JP0m~-x~wbWj}O`ycbp;LYHjulOaqJll} zm8hah{MH8vsh5(|t%&$2IF2_qo|acgk@8_$&k8iCOx36JexU&kl38up3LBP`D(<&D zF|jlhVG#}ThZ^MMKY~IzmvPX)LHw{^?&WAWM5RF*8cHX?!`*YLfm@XUDLfY=Gt2G# ze)P8js_b%8%!E!hbR#6`gmZ@XI>EW|^0vN*J;&`H@E*l^TND#%2Ci-@9ySMT=^M?v z5Hv~L&}8I^M@G97!0lAJa?&D4FOJj}B|fNNYWc=YEMz+9xMgDBMJVXK3)2XRxU|>h zeGhU3jrHXF01{60=via4gc5i@#PjhWJqe~EW&a(obdd>198Z3Lw1+A!WJz_sR^#Pw zPbyxE2hCRQPfdlcT1OdqHqO|2v$CHFb#wf8+--=9%*VWfI!w*5EzQM1|C!_a-ocJfD zi-)5Uwp&{KNn)fp2>H(>th4R1od|0hU^Sfdy-Y1DIp*^)<@M-dk(x26eSx z7r%Z@wfrNMSuSN&z-6{aflv;h35*!uFwx++7Y&Me;|rXZ{*21@f{GGVjxRe0S624L z|1;se&DjUn5M<4DA+kSQ0xu1JgXxG+=ihlhQV_KP@*>Qk5yo+dF=FFuD#xvjSQE(M z6m0vCS0Sni&dc~}TxcWZ)f8lk#*(>?j`K|5Zd1Lch4urzW=duK}Q490sW!o z^93nOrH8WTPoMySfi*kZh7>1W%=#||K+Z~sxp;E+p{H`{DGNpK?TMZ1D8Eep{rA1x z#W^0W98P>F&a{-c>>55k`SCOzgt+AccNz3J;8n@J#-GW}Jwri;Yiz+s3d-w31Rj)2 z-pd!OgcPw6;A@ykW+Z#WUjF5t@sx|j=QJxr(R;&Q7Q^=0EHRitQzSHxxbtQrh+&2z zucF#Xf|=wPOW$j^sP2hs5*SohVSwdc58Z#iyvjSEUFhrJ%igbfic~jS&ge> z%xFTtIDn-W3VG8q<)f8+{(Fy>*?rakP1^lbJlN)SOCX(vtPctR$ODx4p z9sGDq;fL<$yisi~=o*3DwO*BycYj*EUbC*S4=%$1X8Iu>nz3lM0HxvZytuT5-tBNq zzu3$)Q~h2xmvlnX>9xekh^T5pi|XxOMf;xRMP}ZmJkOI)tMXD(n7WwA&3n>lGV1!C zZj;*H#?zC=Kk{2xD=VXdFu&3E(|Qx2pkXHAOZ^@4_c%TBUoh70kg)$g5cK}79m>d| z@34bSMQi(UwTtf5fq&lUAq$qHUvt7V>)3x~id2&h`#+UDDOBh<#K_~)+g3a5mue;@ ztKTjjDd)G4&T}m7;gilYa&L2aKGdZAd0q0YuHIwg$X;aDsg84P| zuEn@+5_$B$9)IWW`yC~DX88+JYm)6=H&b74@GVo7^j}&LxShK{M9=y*4lsB9{E;T55kzLTF~E^WpjPL+wuh>a!0iv)hZp~g8I@oq=$ ztkA9mBt1qWFK_0T1#bImTFu*+5}HzsYH@wT&t{ioR~nvA^F2Ke-z^AgY`DAqQmOxH zkz0ooR`)V| zjR{#H!uTr>wTN!tYo;8<2kqCftkvSd+atOIf%?IZ13J1oj2tp;R&H07JX7?;AHVt$ zgmN|Nugz&#oN^mIByRrYrfhQQCq|`Fo#tQ(B|(Q@;mDK=7tO$a>VlSjgVx}jb8@Na z`@;*b=ux8h#+7|B|J?+$t8vZjnu=E?yd*C-_R6 z_1LIJZh6jQLIpvsI66H&oGEddJ8mT(&g-CdxV*(K){OrDdLtxt&Wz?i>4s;2y*vDZ z{Ja*q{7xJZQ6-T2_6-tPN5D7gS2Vkjo zuN+9!^)65gIM4Ni=@j|wATe;LSH)#NwlyGn@X0*4h4(E!;Sy#Zlb~v3TkU7f@_%Rb zqqH$}%#TeKjVlg+J$DfD@41T;pRjVsH%_3AUGP1N-NAIObM8so(BDlIv*VUyc6}d) z))O2c1f7qWA0nrPp8fI1`)})R-_nOlsW8p*q4#2;>>?+mAdsG3|5<;wK~G9fwvzPH zf8$uVad&R`u9CLK^@tei=k}LuBzc~~Est)@Cf2Gct_!6BrXbkp_zYhb*N`hC!(?7Y zsXp;>z|!H>GKu$nH3p!j3Z_CJk0?b)pSt zJoO9)&8*EW*_v6&HWTLDVRQcMu_(F_nz5fd#YW{cbn4|l$=}|A=_FwHN+;mfk14k- zv+@Y$sJ^)gC&bi#5zH*Q6+0W$g-v2lZ7cI#`M|C$$&+g7VUaTO)227%K0Me;ogvv> zxiT$>Gg12ojT6p{BG!Y{YSu{%o|mvCwGxgUwwFs!ld~4BCFM_FrWzd(`C8Db|XAuo_yzrmZ^7>+u!dmFYMzXzJtmiV>$G z*AGOgmi81(@$yjhPOYM;_sLckb1G!2$hn6T#JuHm+{btp3c8;+ZJZv@PrzV=9y=}H z#>q__Gd4lh>Bz&yMczHN+HKFR0S_)q*k9MB#}YDc-@D?TH1J2^U7e7VPD(kucw(%F zK%x%4`^Fp)2Y+DSP|X+vC09 z`98j7>tR7qIv+D_)g>%DmUxFA(6bBMHazK$N$4?EU_u*@qBChqJ#a;n)v9c9g1TBp^OM3Dt0bVl-Wp#iYQ`hB5s}>W53Rj>PNufN7X=-e=Yn(iL^tc^O z_G7Nn)_ZU{iVZAX`J$%Hy0^W)?|81bJt3vSOC2N`Si?Gd^JBDO!1@_@lA}xQ9DdvW zshX^)y#wPI9x~UN>abGrDT8WN_GF7MJDhM`c$!g`^iaOO_It{yTcDG{l2!?yzIQEg zvoDwz9lBQB$0Ybv>znR=eZ5|!Qv|Z=EpI&{JM1%3GCX=OMUtIcmr{nx{TTsOSM3+GsSU--4l1;X4!A{hY9rR}VZ? z(VmNZPpF|8xs|iLESR~$4x#!YK+US*=|_&;&%ZhrTA}`BPF6TKK-Us%wy-F~xW2yr z;jq7k?Ma+lBJJ`^P@}`ak}{syhPv=K;Q#@SWBJ4rwb%jH#mqN965|GbR4%$=GcZ}t zT$TjlElZMVXssX%ufG^0`_V{ssUQ9zWAmXZ@~mK)MX3HWIFn66hqvwdyrX~3UtZ|; zvp6oO+n&eJBS`u;@2+)-w0)*#XGQ$aPj*?7F(HJt{5CPJ00*x!?e=+6le4VR2PDl$ zQ9bx4-o`8$$x~I5qQ_eJva2@%X;*mRon}vcvVS2xg-A8l2m6;AQncNvs#4P}58VD5 zeb~zmdoCx3uguh4j)~%_zJ&u*TH%@u-cgZ6ko2yo&D^5K*Vb}7w;k(%9tZ*I#Pjfm zTOgjOeZvfO9BrMcA8Q1Ci^6qZo0iEeYS2>VG-<|=Y`H?g{ek&r^kO4v$GWAnove?a z+aJ4I|Du2EbgEAMMkY`mBg-%#a;Z!o{x`i}Up>q7zg`hVHig;qQq>Y+Zf?}m`4Xr> z$1#NXD(HMFI_^zku{n=xJ-2m`+Mj$(xVT(QSIiB)e>P&w@7|2F>x4`dSewwtm^8%t z?msM&+!mMbrW%XsewAl)DEXhvOvA6&I}FzrB+NTo^ACp)v$q^)9vu`zrt+=Jyvb6_ zq1NoJh5k~ot;4IGIZ!%EoU5cI=2qIe8m%RJdbZy@LTwPfS`B8U7JIH%sb(fhcQ`g9 zY}z&w^g_`=r}C?9hw<8)+Cw+J$pE53tRnO9m-BS=v{|iGm4)M&In73cc^X-{UwKIK zE52?Yf1jL$)xHgIydnH!(kNiKr@8rive!xVQHSPJ$=Lr zD=W0p2?fYOqc&O(O-N(YMdT16iT_JZ{NQ)$42euHgZw;QR}jgNA1721!NxhRU3W2B zL@>~to3bH!H8b#{G_cE}_Vc!FK?jBv57||K1cJTmAUNOMv2aD1Aa3FNqrd+b+LAnu zetN0??bNm>mA^axaZC53!Hj|l`$KJP^xc=4n~WOidy)AA6uWzEMA_3AcQwo?GLEKW!D`b~pO%YS^bzBVw3OXj*x)LR{;qw4@NXt&=@oMn zasHG}8-9AMpttx+CFM#4*Yt#FpH)0QUO>uy;Q@ZKPl|hS%y@yxuQzSZ|9W@^l+&XJ zSdap0uwk4$^}3TLm2GT%nK}FMvYW^>KbKrXi9xAaSkcFeHlbY|VT0`HCtQkL1f>Hn zib{*Z-HSuM+AtDD!^tkc{?BJlwQC&BE0zq->_Kx&SJn=(H}d7$M;zi-`VCRkS(%Qj zBcr>U3|S)@)1%rnsl!Ftbs@S$*567@7>lrrc~n&W=xqaL^Oho>c3JV6ac$g}4Ni{c z+T{q)%*s)WYt^!tdr?oPP*E*Xc776W5pB_2rQ?K}>=UERsitcf^fv)1+Cyiz{w)_D zm5r*m$Z?BId6mxMI+kq3Bso)RuuV-iQ%0yBk4l3MR7NZLTc(u@q4V|{loT^>9=aj@ z({hwDRTD|ACF*g%duOt}ok@Wd$2Sgr(k5WEAMIS~{+&C>;rhnatmRbu_v5sYK)1LQL@ygqr`L#w(cYr|U%iw+j2i=@$7#MwQ{#hOKx9G&}T49!x}<7Q#aZIqZO zN5>y|g}I|av6VZRI_vI{!0L1+5Z#{~uI^EJyx@m8*t<7hE)UrTg;GxF;5G>eMu zBA?)>@fqFe{iZ4*N_jtjOLGql_C_we?<`No0uc$fi9Wlt>Ke^1Q8;_Y_jx zepM@n6?Lb)ASh?IDGL63g2Y*x_NuUd(JpGyDx^KntD=D%d{nupmPd(bFR;(JHatBV z&&Ta#n3yba62dcQXQQVbZ6+(Ukv`YC9r^`~&i#(R_3Zl*Q3Y zh|&JT`2IKW*?-+?K4TYdm87Gjr+ zY~oATcNISKBK@+EHf#UGP@$4XJVVIREAsA_9IA|zzV7Nlo4D$`jtIcZaquA*sZbe1<`m(&2qrp}a)3Z!He;SJH7rqP)$jQkG z>RdJ%ymg5K0}4i*Q`E2KU-feT689zODHTjTKOkh`ziaG*G;2NhsN*By#G}a6o4sf1 z-?Y(E^PUIZxIo;TBRZkKzduWT_POkl%8UeGK6b#Nn3w6P*vDZ`N8LQ-wa7$%8KjHz zUDCu{rKM!uS`$}_O&1BYn2>|BJEX*;vhTG9=Kg(R74@N(Ot(fuZuiZN*5n@S4Zt(W z=-i%Ve3LDss?T@*#^;irUq)kznp3BSBL<~OOlu{~^5e(zQ{&KuIgSJ9m)kGgQ0s6O z&aJ~An>ck^5F}Zj?fF$0dCiCn+=@_FyC0Mb)7=gC%YT0m{}AT~s#IiF z{>F99liYxZzd^iX-_{GpkvD}KkqZPUh$WGo_FmEYkf z65q0{jRn$;3l0vBnLxIi5QT@A*UiBL@q#l*m>BNvMr$^DZ(e&N3E+WvpDd}ur{ej{ z`J)}%FB^Q@Mqnx2KS_N{MBTdl_C*cj^A{J9a$MQ*?83eayzn`k^S`B+uNx|7rivW4 zUuq^XUHTRNOtV`1<-wmReV^AC0Sf6a?T@iZanMJEv^_Cc(-q(G>c2feGp2Mp^4g=n zF!edt?iy!drfB+9+cx#vC-y7x5VUc%HuFjp-aF%;|6SM+QyRZbmL6|OWHq7=Y} z`ySlD!|2?+?jm-WIb zS@80Wd~vf9mr{@A2xL!wz@bJ-nyWJK#%pxjss7#@GuL^(kli8mLXuQ{Y8YCke+xLftD0=0C ze%Vj^jpK?HNnHot*%O>ly0Tg{VqRmX;deYwK07muTs-@bsS+2_MIqc2X-6l^kVlt0 z{2bcghxcByX+tTqvqSx*U}EX?6AT}czO4cu8JNGr7H|X#`Q|z++7-mmgW{_-vRA8a zrku*s@(U(%AmRKjva%d1e32|Gv)hhq(Z#)R(tes(R8=&qu$3c7wNkQXf~T}KUaEY^ zPy{1uax#LZ;?nBPmL)^pTg~bvxALE=OY$N5xz#`HE0nrTN?`zuDQDrKmb~bf2xs7H z7-XOp(Gm}BO39ePSqwvsjqMkuMLo~x(qIbmefD>kJeL$--rJr%S$wV7TQecmW=-${ zrwpxa*gKRMY4o{SS)`nluS{47d?()Lp-QOmx}XzsBwjo5;t~>;)ZK1hxuYc|kvy_o zH_MX#O}>sb0`cq{42mlGgH%hcUD;)-IW=mOlzUS^bX{&#tj?Z1@BlMn<;bhiq9Jo* z+u7Fk7Bg8d#m0qjrI01JdicoWBYs@j0-gpD_l9RY_Y4$Sx~;xfL`v3jq4!&6P(BLy z{o?0RtTgpp&x~Uvqqw)^!Ma z$p}^6@mhxtGr1b=Al9&nAd**Qc+i4$yIxn>RN;6O+A9@O4+9X;6QQw}_%mq2XRJq) zEOGFQjNY}$Uc+q$D`VqnV$#Uk?GW$0NNoqw;x!9q_7|L|xqK3k2LA#iI*8RE6vS45 zFMxC8`zBxh<=i^12Z`;zEnW67z3;nJ2YI1%s9c`mAN{H}MO&GOi<>__lo`d(YbNAXrGIK!uqDh?D*3-YYOmFl} zGuw-g9u_o&RI52jT(MYG3csXJyusl^ml?hLy7IZe;2w|H@-NH8o=uMwEC*6dvG^vZ z1+7r6(CWuiLVwvQXUb`#69(;$xt`DoMxEK~(-P`#QEBP56I5vgp;MA4!6ZvN7LQC=A&{2T+3F-2a z^`M@&DYubSF)5=(aq+Z6xt9q}Y~w871|I{4yqV(T$IexTsAUn$yN%AWA7<~jw46BY zP3nHxwZYlg+S5jwlQr` zM;_-Wove>qbgU8a+#U)wjFYNn#9HwToK<%eaj>~wY?dBwKK2im=N{Z5nQ}Y3Hnlue z@usoG61qKb{C<0kq^R#7@!jj+AAB5&&)D;m^E*is#=^4YSn$54 zNMxD2P>;`+s5Y4j^e{~u4eTC^!bhCNM(5^~UfV|p`iJ(pD+yzo)ofQ+ORNi5aX@$6 z`>nn7pYBd;$N6et`q{F!FUqC<;q9_F|5i$u0`{q~TH{W#kZ*8A%e(!eOE#e@6p3ry zhG7idsZamDPSi~IQ3r~CElS;)Zi^f>(X`!6=@Eq-#=wye0*Fd*$4f+~@&QmbA*Riu z_Jiw^6ap%aX!-VGw+)B*jgMs@9fN`eY#Cnd`7Nw~{Hzd=&yt^fMX4?>E)a@vL{fSF z?Rh=|;qfSA(5hY^(pRe_emMsQ-tu*M!FK||W?gLi^zuH)#90c}qm-Etn8|(Lw$Bji zhtXn)!2u=lANQS)u{l4B3_Qa?@Lzi@ zzF5?n)la($QN-te`SL}g**Nq|C-BUqnKIUuOg|k^)^s#(a^RYBB`Ow2_^~D^X%PMW zv@V!AB&Yhs{E>4;YUxyae{v*s2n*)biT9}cT7`ctVaLnWyZDP$<<@ETX?|2b5IYtJ zeloQDrt*W|Q-U3`4C#w4i6%38#8}Tjdi$|~?{IaCL@!{D?xzb7sC)X%xU)a4{Ke2~ zSVr@7er1+9?6PaNlpG0nF~ONn>%q_TTQ=;dURv-gkx<4jY@1)c7t0)G@)e#X%f}>) zX4I?Omsb_uWN_{{B#Dbtbv;Usgk}4Ze3;%<%|lBL1p=nj&BP8jiK~Ex62!-!SEv*8 z@W27fpuSqfm~;Jl4(?eiWgp3Gmf44Tg{!eyDu0i-S48QF}!zLX|v^WX(%Tp6t(VuPFJM~)ee6U_e zI<)mfE=wE1vinHa^SwQ?N`_UbbDdU*OuFd4!n=CDC%YdTe8{M+mw# z<74{FGiZ{LXBW6Pnwb%ykwT>lyt z97_5e@4ru!@9JMFrePU71;9KXMjj0Yltq#f!>sc3F~6#9SSQJgz9{g8 z$m_N${!O@3#DemR!v@I7w(&ZU;pS}gMpIc6Q#gM@_WZE%Jrkeo$^Wg#uX}yJ}gpjDYIS$&r1`26cD_h6KC0! zS*Apg?1NP~9X4!lGqp(SFGg8`8vOQypIuygo^zGJLl?)H{+I`)1^s(Y#W!)NojAX1 z5X)S_IagE>ubsv0g)64FEdLU3&l0aDB(4dgW!_2(j4WQqUXLm<=ebgN-?Dy7?rK+X zX}-OMbN$=w@u~Z*_kBaaSa_?LQUwrg>38R;39_E0?H92q-mCSue=D7w92pSy%ah|@ zcU#o3ZO}-#sN3TZZ${D++3zgVlm!hmdXv%Sj4i$u-%TZ{F7HT9xF=zm{iWm99taqk zmEV-OLt^BLEOCCYt1Q!0Z!V2U4l6MU=Reif54dUHx{}hNeUSW!Wa-1oiHZV_V{&Gb z`e1_Y_R!7PQ8|`u@V&$-d>J9ftVYQjm%!qL3Fc*|dBMgSaqU0du{aYyYqZT~BgkV_ zCkV+?KB!vLW&FdMH7Q?lAlb~R46uPdZ7APJ<^7~RH+g&xB*MJ9ZC&^?gBM78T6oFM zody&UVPX7}?033ee`Q2B8CEk)q=3=1dD$Ew)&=o%;?#=l5x;)FZ$Wqb74*SzTBNaN z)tyC6EZpaBvwDqmZIoGkAw%jOJ~j(IJrpW2?PSuf5o zS%$xKL|7V}8IRHwVSH>e{00$^rC)1t$MpQ0E+!fZX}p{=6QNB#na0)NHgA*TbQi+) z(YJ7h{LI#t8SyI!HLmsfzds-o8w}JD!IQY+0#Dh!f|2Z)w^4FM=WTQY*QWn98-dju z7$~5uyH^O~IpRUnR{NnR>XPe-6zD$c%t8%i}1suPorqnR;LBvwj z1RPfm<%*EhPsiz0!aYl1L=ZP}MeTP04gTWEtP4y#{uWLPUXv{?G3Wh>d#+{$OZgn% z6xJZfvALwWpj2@tQ6ixZ@7rA-m>m6Y<4W8!w=aLG4*SGSbX(9-Mq?4a$8qR|_`#4c z9B6E61`(bTg7vT`z`==kzlb3$xqIog^LPE-O2+=`*7}ox0q)Nq%?V=@#ZlT!`X2xF zk5U?uAWR+a&--HTwL?|V)EfGv-j{s4hoTtqw1MP7l5zzCo!$2A#k><@FWfluF^BQ(k1*$D&zx6b%CH^X zM}sh7Py#$BUb_oZV@i&So=aHWPddtg*!|mT=KQ>zaCWIWiw1j|n;r*|(t|lwkx7Hx zpqlbpU>3qU;to>cpyYdXE1q;%3v*@TuwW@G_wMsI_pkl$Z zX0s8=TmReg*=z{2yldlKj^_XTz28E+zFF4t|_S{yyW?0hMz_%Ck2g^AlS4$vL zqo(gg)7$Z=jbo-0oJ1H_$E!1_wwbu#4-a^x%Zr9f)<~C5!s@(iieX1bTbcWQ3^dvU zP?jD0iteh0ERnXN_ior7tNGdC$PxTfHUdXv=bmKgBy3F% zK2c=Gp{I@g+>T=AB^rF-D8O4+W!9t*E%>*p5n|vT&Y+lpX2J=kF#|$`=GtZE*>&Yp z+a8%XzN`)8(w4L{Z`zYHO%F6TNt>>;}%apeaMVk#bP}2h4cu*eg5QOk}S+V zDBIwLwg?+OO8=%TM>zD?9!e9^>R}VUx4)tuMfFa=!TyV1sP{_W7$-gwM6BGz~Qaf5He6u zWK^xm%Uf!x!Dh@$-5XRzM~>H_QDV19Z_3XyHtJa>yTt{OWaaLPMg4kt&;5w3yYopw z!&{j=@V=s*zV+YLUQt)0{gfdvLt^wSxJ)|h6$Q5|uHwH3Xba^*1{!Pb&6$^eti!jY zkev2u`CF_HP>d1r_D)WeIF9?SJDb9BYyMMV(E-u!_lenm(nYa0-)^PyCp^*aYs}d8 zcsy3%_mJen_OWmXI26F11~g{sk}hz#ow*7B zepTChzpt`eQlOCQvvHTz{Y>vTVucihNnsG|;7?ARy0551zAkmHAJPu;u&Ii~-})bE zsB&aHSAa4ASor^FI_t2i({>9Zq9Uktr*tWu64EK%Qqo=0tw@)2 zgMf&DfJjSsN(cyuNJ~pe$3D-VbIx4X_s7gPv)!=a_rA}1*1A_S1wFomfSb=_##OF? zo|tf}oZ?!I(ub{=TuWoJl59P!dDxhQY;b5+!C_}p+i{1r1#z`|^Rw+_$w`b|`#sA& z@t}OB-H#u1Ff7#P{jg73jYBhz?qmd!t+g7+oj-3C_ZwGsP{BS7BQkA0)*N7Wyd&T4 z>_rp-G@1jkPYdleDmV5<84qxkWjB<8fBrepkN&lCR6x%8y}0$?9iDnvP=6!^w(E-X_{C59tv^p#3|`je7>lZP+TiK>eGJv72mtQdi1C^70pjc#E|%!7 zlw@dT%dt~+ZS8(8;^xnAz~zU+G#<~;o(%w>qP7j%c~PuJJR zaF<9LTxHIKN;aay~&9oH+#gpaBz8w5ed^AX#%2@RE1STmqZEaz#Ls#Q#p zDW_dXCeW2kvP{}0y9qtxN~gwqz$M&| z|8B&o274M!7PVqcXn?(mq(o&aCn^*IPUjLHGXYZ!jAwRo znC*Uecs?4QZ@b$1$D91Y6Iluu^xq8pc1B(y-YPjMg@YQ-w*Y%vzgS=iQM^|IWFViP?_ z=W`5*Du@+8EA8&UE1h!rGm!XM@6o-OXifnhg{t{ZwJ@shx?<@HP{rW26Y~i9)+`ev zuG?`UySb3>sNMNZ`=VOxbH979si7c?*6hW=DlR(eh-e)I)&ved-wF@3sLLa{6NHpe zsF+UuL}Nvcb^kV>7?n5tA$9z^!9WSmv72*cVkEw|P=vB$ZEMbRs?(K=Zlv-0^1*`Q zc5n`8(ij+3LeJBq={M{h7>wC)Vv2~dWNr4n;g+HMjA2`w7neGIfQv91yjy$q$_5XIFDhM=L6*P|(8+^eug#`z=}@3p+pbGP@8?aUjxhmHhtwNhvoTlKeZ+EU&VB@2a@Z50 z2Z1AUB7EIlNMV)h3r&@k*K63UH~7VW?eA+_M+0{Uyp#L+GdtDXB#H@gRxUq3p}!9A zbSzXahDJ?h{FhJfmT8ZNi*k@2F)`r}TA1tgtbWFc?BXQ5Ke6c!ha{YxbCS!TyIfKd zok>Kv@u>)qLZmTo$_C&8u+b9I_XEKObRz)MMpfKR?C&dY=R~FC?=(vUg^6tW!=&{n z**|;*3*iT3c!`|)SKxx=YPtAB&KOiGqKlBbI9(>L)%P#x;|IQvQ$EF)zAYOJ z4Llu{Xl36h;Xac#IT%k8? z7;Yz+Vx>r(5)*S`2|ez|0RNCVnRU8sGp7EVEdzc`!8_-ku#uHl7rXu{!=4vr2=pN2 zqVj=d(x3%0OLct`h#&$BS+D3Zaw|EAq-8LQoyj{k?(m_$YWvS6VK%+v0SIRlcL@)$ zTKsb9r7-XvQ(Gsv)!sPrTeaq-^?tiA>gyUO{U-`b(n-G_cg4m9$%yo_%#<;8znBy^ zsi3AlB7JR#qu^Jjf5vXDh9SvxcBAX>)`=6&5<3mrguDa&an>g4J0}VG=EwzMT*|2mEQ|NwUkwH?zwrv6E(H7S{%LGDFPM7aU)7-8fbK7A zN9ux-NXr#QS!Chmn|&;3S|Ak2F5{`VqK+Y({a%`1T6p`nW|4uWz$RVtkPg1?OQ(?K z;ooaKC~qTK-WPEB%i555l@4Eq=u6>9NOC17yq>u&qm#gx6OO%_3jdv_o>pKNYZMnC|4>j6 zT=u+9`T1krXkB()iKX1#t|NA$heDIKA~C(OTl0_UJ!iSm=K>C}n%*3B6cVclBn^DH z5gFGSd)QNTwr!!KQKsDhy;OR~8-xAVE6kt3TtL)%dP|nv1r2eTPS)URf;O0r*?;}d z<`dE5!us=PN{VZFnX{oLoJ*JA&sP2#L@n+)x_XL)?F`_NUHNic67o}C-bFEYBz(|X&J@$5^tVO`SZZ^ANnUlb+J}$q5)&ai@s{6X%P+NVz z^gOJR`dakM^*DOq-W`j_f(}0P{!AbChC)f#vOX>=ihbG>L)gqK#PD}IzN2K;jD$wa~if)7MV5gnkzuE}&r){*(YXz;X2Z(Dc zJNubOFTb1KX}vTG5fLZS&yJ{=|3YE>lHbW80A_`g-xSsQWnavCA3)l(gUqR^ePx9X z4-YRoE)MV%P%^drR`LN7r(ppFkkvIcV{mBC*fWsE;Ks;6t1Xx&uZ4c-CVDiYKa3w^bolk*mXDf_?}=YNA)tQi>_8++ns@Y~6`!QlRl zHYkZcvLZ4||7Q91I7;#Sd7gfm`s393eGQgmCc-HEsP3NmeXlGpVdcO;vChuUU44{2 z=#+7RKlgB%3X#^*(t_OSkoy{9%gD+i@;%@&0&fu`gqVWD!rD5#YFH==tjh|8xbUSx z+^^>%Gb1C7*N$**Zf+=pH>$k+v2BgMr0|!(5X5Zd$?0h;RM>(P*uLm|Idh2Z(U|?= z!v~u|j}t7HZB8nQqH*=|BpavupewMqT#}NCcjjT9@A--dBM$7U>v)z=ymb#d2Emot zC38cIFLf6u&4qRDg`w-Y<{7!|X1!aqLDE~<;7H8@|n)bMQ z$-Rl6*Y2$BN&Z=Mo1Xc+|14H;N%hxbM!0pr6$DzrG2?`rDg$b?&E^y73;k8#f)sEck4&%KtCg8Z>t}3XK&=qi?b zuR7O6PXx>BmoSi7FXZ+UZnX3K+6{&3-~f5uZsIk9hT_ZV^w(gU?`UcLq(QFm~O-pWF1r5o0r(I){_D;%&=hJbk%qzDfxtGAyJf~CX%J+2u;MK; zK{;gDx5&X4v^q@R#e`ppS0Gb_{i|U3TfwBy;$w|)BMeqnEL86{*Cg1sOuV5wy5-h| z%ZwU;1#5cPl)ypW(7a>*A8y>Ba-{)SN^un5bCUkMe?T>1d-5C3B&-;aSKyXZeLA^* zGlmH_qO^7k)*xj}CCf(Ts-=@;k+MM&mC;NX;CRT7z<g=EY9uUtJkkIEY`kzngJVyBXW(*y zjs~_44iZJa*!WmNu{^ogU|J2Gl@v~I-*v}}D8$B*v%&;xk33sCL;Qd#ABoEo2Ur$@ zW5|?~#OAmjnq%QZh-4TIM|GFaQ)OAsrqMQdKsMtS57#))3sLbQBNy2I)MtBqvc$`? zCI_Xh%&17abbRTAdXTG9Dnp?rG@l9ly^tC)%R^r+DRDdbN)Awm26YRrl=m}R3>k8e z`54=B&+=>!`E@>yu%N;3*>L6?77S!Ys+0oLI1y6ySI4tRjT;$2%Dr4+oT+!A<)8QC z&IMcA=~!IAn>7c$U)-k)?~02f?D=L-1HttXecy5Un94t!AT=;d@1j{JqD+S49LFH4 zs3wojSVrDSYP+7aTI1wvAvNeKrWeWYYKnIUcTfvM6Z|pr&N_oo)bG;&BXc)&5mWa* z@I?2$g<4sIUfW<|FW$d{5<|kgkI~sJN$lhCXtvIYF|pryLQU4C3rF@+dU@>U&w+tu zu^x?XE3cJw#6vaVSb)Q*vTn(}!GB;_;vj$u=FEobXiXxQGpVfK8_D4XjdV0-T)kN( zcGKioPN{u#__3lER4!u(%WOee8yD}B_y-N|C4?Bp{li07-bc^ToAO|8drL~-Cfd(_ zO_X!xFSN~&d?5IywESx%I3j_!1@#nx2iKH7>v_l-+IP1(72v|JRS=zGdlN*heJUL< zQoMxO4C=9i+k?17W2xoQF^M_?v{Mu~!nrtt>ZiWesoxNP>V5cZ@_yU9-%obj2{gc+ zS(GayyerI<`jA|P8Gkf&P&Ys&w6w{i?B{S&S?d=oEUnzQ__bWCX-BI;?hy{B-@?#j z`IIknb%3?vGV$c<>Ef0#LI|if4{;+#=jN_@b^d!+=DgEoyB5C)yjq-_4(e+C9kUg+ zmbariBAEZ}&GNWaLbKg{(zxJDe|FPmFA~U0+(13^{jon?t8+u(ByH6?!w@Nt{9POQ z?y)3{g4<2L*rodC5#dM%B{JO`R>`Bh+4s z;uqEii%HU?f4!=E7$2v#_y>Uzw9-!ZPd%26z%@E!p?ZjeBx+Hv|9#2ljusnBlGF7r z%efR-q!CiY#Kd6AeB0hGE+ZpDOi5{JViM#mpycGl-rU@bgzJF%G!B2AR&IOF1w0Wt zi8?#;?=$CmBk>S^eA-Rk`iZLKsZuBQzm=n7!pGS4F-8Xa<~--G%BL>gb=uY>jFiUa z?xxT^A^u$V9pS{eOA#Xji(U|3f{jkk>e@iGhyX_;m`7>j<(h)`t~KC_0~$^*#6f=j zQliFgTi*X{+KmA;K}c-;_@NDLtu{6`X9e*>cW!!YPji-SHtCmD`Z|Ymt2_xQ(iR(m zVB<-Hj?uhQgF&8?GJDw#vFD!K3=4dm)_$D_WF&`Kf;8f}=K}_*9 z(*TwQY?ie9NkRSA)B+?<*a>h(=|7XG82`TMWM<0M)8~~IG}7rMppZBo`|X3}G}%8P zKMlQ;m%dL(GQ)#V8cgGZDr`z#8-y8^TF?nZOznV)qJRlti;n`Z5-aJ7l^&#OG{1Ug zX$BsJ|>Xv+bU(BXa zI=Sy{U@aX_akF4(!%;TD9JWw!+E=QgMa_Kt89up*R%D8qu)#)|8VoWBHTrvQ@2@{I zaL^_l1K4#bGU&Pi@-IQ_v1786$XMnK0geu}$i1(#uwQV^&YD z{pF12nPa0uw91U?Ex%{@kruGDfr|>z8fdaDhLhIcH+T1f_mWW0R^OQzPa=VRVE4lyP=J@z|^p zy3v|x&JcwPcR#d=$N-ChH8q0IT_luP+!b|+1}$H;}A?}eig=qXAFM|p!Sr#`*8K36Jt1CM06Um|Kpq;}*?~JDbtG!HYlvyF((|@i zfu6>n88a z3rv*XrEKc!Q-S-%D*lm3e-&cw&Eh># zVrvHnkgNciVQr(f96oI9xIB4G=DqEyk~dzqI4X%%I2qr`?9|C|al3EznD_0>C+#&) zLrjbNNiV4h;_n0mO}mR^e7#Xc`O!j^D-)->mifRqg#7%G#5fnnwpGOQm|(SLO6K^Y z;@P8prKak1?FVy>(Q{jxH#@wZ|*BdkLxl> zE`08EwyPlr6c6&+^5>5w6pIdczE!?M21qz9vWU=H~($%92W2CeU~Y|;FS4dj_bjR@49N@pwv)Xnm=3tH5aCQp4>R zLMf(n_`AymZAoU(zC&t;v6z+_SKeM;1F80-Np-fUzQ_*dUb(bxcIUZ55x)&ZV^i4n z@7mf=ebVcERQIa`nogA;rxr9fXQ!kPg0CV?#Fq~c{Dg;VBUYeN?3T=V-h`Y&=c1QN zoeQ$7rZUbyTA@&)O#8`t#atjORF^08)4luZdi4ms;e?eX86!A2IA-9a)pPq-Lm~mI z_8#;i9>Q7e40WQ{dOJ$WlrX(}Mj-_#Jd{uKpX0f8a)xRt8q??bp+LT`G_&^YuzC92 zxV)zCI>lHnt zMZjXRbgCH*JbJemo;PB)G^nZavp2;%y963%Nt{(VPwfhn6IN(F2I%{>jy8UN(190kl zLx=2UFg6k%kh}C5M9VBUGWR%cigwTYTrh}+|FD8V8_#a5ygn@XV7)LnUM{?)@VL_9H7pBpALysN_HHBo1 zL{8q30QQKvu_Z|Aca3_bSym|0_i_s^_4DK~-{^?F#`*bN^MDElUXV-%9sdbE zI<)evbU(ocF#srtd8#tt4gC2@hM}IE4>%1O{*J0oE2iK?+4KDs9Y)LJn+ah(Seo!a z9_7QAp-o%*oFNVl-1ptPdD@A4yn@V0G$?%uo7E<~Brq!ueXslS(gZ)VTM7~Zl4dW4 z7&<4ux_e+1r<{eTr^EK5-;FRvMv_m5dcnyQ;1I{!RPORupAt97pD90Clta~1!(WU3mbT((m;jMN z7VotAZ)lV_Uqz&ZEpLRTXw(Z9QW#)pQFb1Q!gmC`rtrO6!IWlWz}v9T0ZfB9c6gTt zat$=I)nof(F(Z-yc1pzmzJ}tp{U#GFlK~d*#Fp#w?LH7-hE~=coF3*31fjx#s+VAM z5)Rj*=ayoSaj9BD~(6Tt#|qCF?cDG6=HH8Xi*5` z#6rvkmJamBOq@J~5(yTTo;7Nzbmy|K-D-I8Vv3NghXgMDnDG90MJzvNINxEAfYFAb z;6`cpM7X!}Od0HO^fQ|$ws0W>lx+kyW1RvP7J{Z{$`Br!pugMN`%EaY_Z_T#w&Xs% zgvvj14_5G^KuQOEy^tZwaHf-YDpK&$D&ft@5Dt&*GG~)_vfA=>{hNu@)Y{Rc)uwA5 z$o}Ie)HzZ>PMl@_w1@w6LnUzFq~%0J^r%>6AC7T~b;scaCv-J@y%s;glled~=x`xW zF2bukqKS0~F-y(v_&e9AceS1PY3_HAwv!J3PmR#4jv$s+lYaC{Uv5mwF##BeP+iZEO z7S_21{=D~wLyMIT_wKO3^1WEm^Ys&RVYFAFya+)2tgo*xEZ)hrq}j(f5&dVoENs-r zq8a&)JLns4^F0oshi1G_Hgc>?#}Ori3f>_TZ=yQmxS?Qj^NT(~ZL`6eB%}2V!j6B8 z-RyAHYkLMauTuEO2bxIKF_(1js<=N@zz>LeM$ggH+l%zb0ldL)g_f#)_oQ{p|Hp=# zflD_*(_RPWucK1q#ghxcmA8_*@J$-CP9h3CQ#AU%!2wa33xR&d;}rBu(7k_KXu_xz1Phi5D&PRPUL#Vz+=V&qWefk z+V9&}mk+WKUW6M!N;vv%K^&Lf`CHv-R#t_^bL3F9IN_pO-=0e3E=ptXGY-N9lO16v!y(~Qp2L+hT1Hh zb%AQD1y&n2hf&)k@yscU(H1Ps#%!f@h$8lMoQYhd>DA<6=QqfNB?AJq;o{xt=e-B;W*c>f;ePqAw*wr7MQMFo{JCyuL5JX8!qzq zK=-prdoG2G+voNTx9<}ZQlRHUJ|uMz$uv?0U91>q@YZfNCOBB`VPQcyLBGsHKlEZ2 zac!4pRH4Cy4>LaGKk~t+>{+T@<)K@83fn;)og4 zAS@buX@FUA3JXWK3O=pWl)y$)PiG4o-d(QYuYD``Vcpv9g@+S=VmvYJvUm#lOS)EH zpGU09!sRd4*yXuK%Ab!wh(n^SB66x~hCEKRA!A%wG$3QAHJ?fNe1#NNxxJ4>Ne?Gp zD(C4gO+Is4vT@}uEsC1rYwnN4ONB{kT7*MN)YyD%Ojfm6`)p3bj&ubg7OOKbhr`kB zd=_GLQ{(YXCr67_`9XN{DPKb`?6KjMn|t+(nl!z-fScN->6Wdz45=~3zx>(5u5-|9 zTW=o^N41dsKk(=ua*1|&xhAFpH_t_9=Xw7Vm;G)cs&&UJwQYA4FLqiKNO46AF9hp%&}+dN8IsQg zJk^8^a*xc07iVTLi4}2G-|>sS%?J@a7MDlp)pwfiMJvHOnD}&3T;d~+jg7`Z?3OgB zg}cG#7?v$jiG04c=;rrZ6sfW;yH2Z_rvn>_8e?d~hB1Cfwd@Dft3j?${ykT8n3;fm zicDd{?2BG|E^u5D6T^JA`p#ycotj(`a12(aT(Q(at^Lw@x*g`<&3M*rFul^zWhZS9 zXG+Q%0Ui)POw*4G1ASU_q4V@371W$Z^qez>VfP_kzLb;GV zVb5zdd+kHbC){}c%tz_hB#fE~?_IT)534=s9IN)9-#_EL+^!|$&0p3#eR606&8Ej1&!&0qV51_+WaL&PgPDDv3 z6A~PXR;5CskW=yEimB+AFVybjNM^K$N`Xdt;9D^yH%JzWe#XfbvXvFWx!zD=%Z9766eug}*0 zT}Tf*F**9cp}z%#9{GbiWsi2c!u+!K$2`cdA*aN=x)8I5PiUBiCN5DrNi&F=nOo}m zv)gu_KpLK_+@h*IRb!aK<#>kayCZ7Y+i7$aKoh3`&PNJB~1=PBuZ(pMS9p)CC^#-D4 zmW_qNN&i|Vc(2x$&Emo%NyHcXBsZEZH9p{wjEBhh-i3;QPHEqWC?=~mHKn_znaf5u z5y|*QUpNh1`2l4?jqPO||FELvoCWZc9{@!K`pMq^83f+Y$+y;(XftbJe^~lmE%L3t z#qCD=u2>|#!V5vWhrSNa?Nix{ngZ@jKB6^u@+ZA;XFaP6yx@P_{7)Tnfu-dQNkvq? zSu2RfxI5dNPiwjfseC9_O9)EP7pz8tsNacIjL(4F$SI+nFHaR1}xs;;&md!i+HAw?lpUo^mf-Tst= z<>d(;@AY!xMRP|3OT(*0 zTVCddG2AIOcfGo7W)a=HwgPOZ`;p(N5ytZm7hN!zzIqHB;mY8f>@RhLN%0H#f<1X_ zOSQ{P!49))n|*=__ynlJ!&FRte6{R!{p1bX;BGqJ^FsASI9ZF-)#ga(<dQD<8-U;5(bFa4WcdH2m)88GB@5I`sEk>7ksMtg=10TAL7c1|HXX_m>RS@Qr5bN$Li>UpH)WtHFWX-kW3DpHEBX zi1IFgQbo8i_V|aTK59VL19#;Xql&{N){3#ou~4H5v!g@y!}zFSnWAw{J{%hSSl6X5 z-Es+C8Y7}{gBuGgtBRwxCu0jC67iE;MoVT?4d_9(Wj(Yi!h`f{x&2bs39stn5wjkD z+s(O_4t>1-cIyBr-n+^wUfJrSCjDlza8Z`)fH5K%g;(1V$yh@|L2ftB4ZBYn_6A<| z2JU6=`h`L4B?)NFfj|2%Or(IU01=2H)SzY}d}+0`Xhx=Fmb930s#-MhHe||vWn#ce zPX-%pGeN4@Ag{4GB%+@f{O0&3)@;Ci9m*Yl#%F8-#w>7PMKBg^w@AY{`@Imf?ugo; zP4c(>1n_#`dt=i$77i|W2%&r~Vag8%#iK&fG*I@@Z4TEYaN)yFJZeS?*mv4GWPmY9 zg8>wMdzyQwX~r-*S_K)Q6)4uUyQiWAPBChHd6Nv90leaSFt)okC`7eJ0B@;A+v*4Y zz5oZQROuK*3tXljg})A}>z6NAHRVb%ftLmCUJbu*$0dzV8v(Ff{9-%qlg@XP9EOr*!D|q4&1KQ&$#JdhH__TXPgQ8%rW6z zxk3aH07S5#wv6)`r_809#M@v)ZW6N7G`W|TnZG_k>eGOWfbc;A4J``|VImGl_x<>JBx>-(DV>2^lF1Lv)vx{Udu^&pl<{j=ZsdkJ^Ncw=kJ|F^2vUGt z4GZe?;=5Z?EWya=WFIoE3EV-y%8xQkNrT|#j11v⩔1+jCwVO?ccluG)Hni%rBp? zV@vk$fTJ*I9~V0Wb%hnp+3D1+t{`X z#4dm%1vFjG*~UB`+l& z8-qgtrLFIjPB{$x68NoF#=Nh z<62X58j_UsG+(=PAK6AZ0%1GIe<34xc}y5O1}?X1^kZ_b{}t^<4Q-QL9=QxK7dd~I zXRLIX<>N}fhr$RZ7^+X7_O){)432Z%nj9Ju#8BIX{3&cszFD8Q9LDk`4dK3FR%3bGL!}Ul(cKs)d(n_oHdiv2fHrZ(pu*zGTMK_4!3G?y3pw zMe~J~*7KI(91<8H6wOoXedIMfQ-o)c_15c8TY+)fNMG72?gJ*_U@dPK_COE%F=~ zr)Nn8EZpGX|20ZId)!fNPcj}CenN$$(m6wf{v}kIK#aTYf7R*0R5^ykR{ko1>zUle z)84AF5uNNwo&d)3%*XUs*TK;{K#r1Cl z?4)i;ze2P$c63PcvI#0l3Ki{%ooJo?JkofBxwm{#nicqku+&@V?6lKZr@=WSHZwr~ z&xfrTx^WDwiqgZ1slT)cf?L-+y%vu}p2_+iwj4ot(aYv#Ho~(Y;)*=^Za035w`_=$ zf6vqMkE1x)-kYmJjO-KNNnj#FZ1$7RJ)-IU($PDT8F^QUN%+KvB1&!Hw0q}Gr)||`OSM`LUXW2kL8i)9yf$Nug0a4?uCCmFzg-)YiZ$~! zS(=x3J-Jg~a1ei<-2v9qw#S7ry|$y5S6WU^&PmhU39XUu4)1^7{D9LJ3=9m!WMp<- zyVuQTyPT)9^VcaV{+k*xt4}*!)Fq6q zPWX?nd0+65TK#X@Pc*F_IzY}Au>$*WWXZ+hTh-c%sK^*QG6MZ_Y2WMjIC@aCm)YQR z0JRhTbFX^o9337HyQ3$&1W{c z#gsGvXVz_*iFx>3ZH80PVY*af=q1bUc(_wn6I5GP+bw$UP)bP ze!DNz*Sgh-`6!K1cjX0XlrZLK91tm%Q7>;ei1LDZQ8*W-WA>9&o_UbPp%1BXGG9nB zX#ld9uwsBPC7Ah`m11hBp9*Bax)*l(eT#=aGu&LSwDC9o^S+B?qglGw()4oy_n=^W zGi|&q0g=F_k+?ZsMu^eP1w*so+z#w8fCUU-0r*I}7j)78i+aARD+rMjmT#Nw@?uhe zAsA@A_il@PVdE}sypKQqTTH1z6E38AXs51$Z8Qi-kv$k$vI3amWx8#zNn7u-q~o`1?{WfSR- zIPgtvg>z=~pa0oO>xsjPSdhR4X5jZ@EV@6G1>U_}alOI4%P$r?LW0gIk}ULM6jh&U zuk`qm;b4rWSkAPxUgT}Lt^E)(W|y0DX(~@}$M2R9l$(q$UjCXfcl3__cS70jy*xu* zGE~N}&~bKl7;cE5MFUF6ZcC%f^*j~PC*fc_gD@IvUV&CG(c+5vs4Fwp&;I|+4^CQ8 z^25t32-Z>`zpFle(*`V6)wj}!Gcdk;?K%kA(gJW(pns`Y19g4Mb(y|CPJ${ud zs=K;brR93rVzo)`TYnqhS!HHD@M0K4_MeS_cV3F0+G43#`CcdcUsl|PlVZ{$w(sBn z!%gDI;S1UEP9AvTkfKs-Ty9r242hqz-MYd?+VQqmZ>*Kew_XlOXt`j z5^2?8+pD1BH>pDZb?lt1=Rlg`FDL1_^>0V4{zK>_n`%j$jbhDsifl=9jtV$?M%z-OzL^J%WZ} zuSbB9tYv$x+?1^n6#kgOX~C%oplY+&J$Wi|OlZ0c7pp*s1@a#y2P=Z=oME~E(glw}Y|GHTcoP)-atf0$A_|NCc z_YVrn=#L*Vk24EO4QsZu_)8PLSURNyWZ2R_Jnpa0;jU;zq1(M3zO$*J#rw4r(wZJ3p~c8X%Aof3IC}a}(`y zxSbPh$)sv+ww%C*m4`8+v@CtW{9dz>VxFcBK0a8! z2ovE$$Tv#GzV*tBOH!wlDz#IF&$!K!b6iYV92mNJgj$P#~QhB)h&OkWYtfkAh5$n<1#4?@*YPiV1L+Nv9ZFU8nkgx+5u1p zS#elQtP3Y4%6REnKj&5H#H*%aChcvd#FdmvP_xwwCCYX4 z3no<%djd%O@g*g6ubT2D46KARj-$W=9Eb+WNGiVbp2(B6o$lY6>%mvQ#DY)6&Mz+X zp!E7d*iC6QudT8=fyFm?0{dNfXUBD|-<&}21NVhhj`yFTH~kPn(1D^)h?i+KeVF29 z(*iY#A+dPB$}`=l0%}4j{;i1ZP0_E51+BEQ4Q6@&N&n?J?~eqGal5d>B2FA#yr}(Z zH&xBw?#<;*_MPXB-_0P@{88~e;^M`MQr>>XI>&P3759WkG`FX3Yz!Iy0pvz1A?^a! z&Gymek%K6e`8P-Z3giPX#NW~x7&oUKY~4&U{vC0&WTDA-Uv*2TwCfooMd;PnaoG2; z6k74|*U}a=u@l2b8T#(f<Z2|k( z&}2nT)T`S0AE(D+q-2x(BaR=r8zs%fcmHj5I8FV~3wR$&by?C;v$n~kWpeShi}CTO zO6e!W?>Q~Pf01k=rc~)Tv8;Y;bFTF0wc+MeOX7oWc6w08_&0h zR>SACIqIX<3)S=)J+g-|2aVa>day%q*K4xy<=XHqlM^QEd$zA$pNgKcAqW9%Mh;GJ z;iCx+7(QSXi{+O3fONu^_kWYAF~Ove0g+E2Z8!q}Ot;b?6b| z?)^yIGT63G+m;}zhwKkuKmrk*TF(CD+PAvBQ)MHB1`^_!o)b+JsB8IyQ}kE?l>=ZJ z_IXcRRJ&}dOtQbM#4&sagjBT8P4K+z7YloeV*|+?F1zUH5Xha_#9g%|Xq-fLXu#8s z38T>URV2pAo}zR5Ks4Dyt``sfBEJq48*ZcggokLrK=zkU+j zjeFmWh`QAaRD1?+MI+A%dI`cz`EBhh7Ja%*! z3}4HipWl7f14s}^6_a2|M|5dSmodOuRUod_P7&P&+nfBUEqI;-F$Ad|t$1LoSfq)S zCc;QykoRWHr0qXI*9|Rl=%`UQln{TIA%~}8xyM(W{QJ>0!|OyBaDl|Lr+l2EPo_Su zAG4MQujiR*1W+)1%2GKm%9;ER&s|vR=yM>cySfTNRbXkN@y>xZ$*n?W>N=%s25=2WNivVBEv`!BqK&Kg5 zXtWXj^Kq^OMRO8``jQjjatUZ{u(wQ{&Fi5v2TyX}r*duZ>z<4X0PGC{QE*9}KI%Zr>B8UBBZ!!c=pGY}!IP?^nO<%O%ky7m6UNiDLE zU)gwj=+lRns`b*~QleirTszkZXn9CsgEqY7@sY1554~GGVpv;@vMzYRJWGyE8d_-l z^8Uq=iGkgyP`lTX6MnP6d;~;sMsBECF?hh*Y4%WkritY?b4ohAsFaZGV`BhxxQ5U$ z6yz;>?YXDTui))vjeh6%<=Z!pR4Q-t4FRGEENp%T z^dg__M&b!{fC3Kypdzw|2su@-1IPkylE^F|kPTd*!3Z`FkG0DUBKpmRMh$^Q0@t%v zGW+?KZ0NRtwnLo%s075Nod1mh5*+KF12^50Ohbkcm*tfPZnK{a3zZM70*~(8P99Y#bc(Pu+%+znU`Weu4%o7}aQ`cfwn ze3Y=vTsE)lkSZkhK6gHLYmi7?zb<)T>niY&^HgzR-|HQvG!jZUn1kJLdoU)@DD_&ugY-03rKha?B>L=-EnIv3rvdJ6v}21C=c2WbUSphtCl# z@R!3boCi4-xNqeI-#bg%{Or*<+2d z>e_&#^8-Q;JftF@DTKnEft_6i=m+GU44JA1R$+zX&kp&)6hK=*bg_H)=rcU20X6M) zFVtTeH%_=A!;NpmhLH*Y81y-Em1ku;JPqVol?QY~7vD;iWK8=aC1~Q8F1dC1`c4NGqg?|!lCSMRa-`;XD(6bvZin+IygfW zG2F4Asl5sM+xj|_WmNhgy8v0iqC-i2D=MV08jY;}ZzJO$3^g{SncsSleUeC<_Vr0? z;Je0uf%y<{sL`K4L`O*-crV~Wf;6G<)*#feof)7Q5#;k$g{=E6yT7poS5?zCrqEw+%C9Lr%~p0 zTu%<#z6=Ay#*QZIz?0`!Ni14YYIL}YNPf0&TiDpUuVlzGMRWJbG=1?8Tacx; z`I5=!xp{`c0l9Fot?oK9Y4R5+}{fsV&@n^Nh2quRxQM!ea(g z1nilhssOxkKR?)?u}-oi>&Gyp-b@?|0_o_{0&Q|XpQA}=PsdQ^T*Rb3O^O0725PtZ zEx6yp1rv$29uDRsv&j4sNCdk#Kw z;4v=uo(s~6Fy0uY;+1v}=|XV?6M!YaZVITY8^(ArC;(m?1EXIl_B<)*X*>a}q9CcN z$^*d;<&Jo%W#S0>@%xH?B+xP@3A&Sh1-w~_s;O@a)^E-A9mZ`r;j8)r1ydVEv6KG2 z&&yT2EIBopv^bP0WZ(7Zzvn)a!SC7gBG%JJ@5Kc|D5M(_Xv{`8uy4gf8WaRK7`A>` zYMFDuk6r3}(2}~Rq=Zt70?l8;kQr1Fv5@q(%dv`@CSl;?Fl`B@%w7BtfO{b`B|Yk3 zEI~Oq6J`j0WiF)1fJlu(IVKVWmco~rZR1~m?pb~&ikExiF!RX& z7jY9R0e5H_^Y;ZCcE%AsklmMdtY5tCc=|K-^nkT>k<9MzvRK8HY`|sqE)Z)^_5ka* z-!ZsFqW^n%p$|J)?D9UO&LL;^qr!NkjgXxC0OAZa>8EId7XqU*n83loAv(6?kQa_I{W2Lu+K`kK zL!t)h3c|3pbZi?(@Jb;0>HKG{4%^q3abM%0ZK&>?URl7$=U_KuYBL&*p!U9;KpEqdX~?pl$wgk zVnq)8q=t3PxoQvO_98$dyYqr4%e|w8uNG_IjVaeoKzt(6U%Y#MJ9frqO!li{j%B02 z*-SXM1ntGyiL&xLEd@`-Mp|z{TtNXD`1bMf@q&#I)J8oJP%p-ZoIC93UEs3c!rpRn z2g9d12U;trn-MvAOanZi9DXp{_+AJjf-of(5?Tbc?Nusb((Vs(qFq25_07rH%tF*bPO0HFfAio48NQi&*`#6}tV5^S*W( z2mP?ey!j`D-v2@F*39T&5X|&PG>BXf^s%ayzsW%A?5{0%8atL)`>w8&=R9ZA%X$lf z$T1nj*Rs(hlZL`D@21ewhKABs(D{wrB9@?2@vy?8&x=!37_{(Ao?34j6l@V2;O@iT z8Js-6bYs9{JAF^F}7q3^ETBBE8YzoB|o)4?x%+`!NHWgPj#E$ zj`np;i3Y-lbtH zO{>66#iz;akHcah`bV;yHbZ%FU%+aIx^=P2n2!t#;%fM8G!+@V{iVh{)0u-FdJtm^ z`l`R*RS`ubQ^x4{&CcHBb$`(rXT33l$3n&u`jedz9o5+GE(ACxB8s8SPa-hfjw4F7 zw9SlCV?=h5C1pbd?ZG2!N{Ys!ct!ZmZSvvD9E0x@(SZ-~ieG*AfOu zs&WxY)xAI~+udwUW&*Su`I7eqg^J=kV_CND z-dPXVY)a=${G``!HE#3u?D9ldbxe_A7eSUfuFM|OZOp?%fT^+Ls;G)#%);VW_rYrC zZ{gJH{nWnPNo$rw#g1dk7>}s9?3eMy_78E;AFT1(z)J(EOqqTJB<7gJ8q{625!gh8 z7I@$*<{x|ot9^>MLDi}7untkZes`<-hmbPd&bU~Lz~L;ZGDK|ua!j8Y;;==ZDET!$ z&aloZBSGkIC!O{EAgdzhYy|7te$Y_SGg4e3UUHJ{*-Q6ySCd`MS}JkvOqyzqD1DBe z&Ck`P)%dAhP}mu55G0N{_0qht$2>{b4yR6$v?xOfE9YAkLs+q`-NHX9JXE8X%;CLR z6<@gz4ls(Gp9C|;>hv=Lrw!7-w?3Hp2;bhK2T7a=_5_FxpXe6)dLHz1cvz~bMG?)sKX$TFQ zir@ILoW$yE&+@7T$v0==ulh^ZYinz*YrE#rZ+rkFq-;%R?;X`2W8DqQXGAI54fq5L zD=_?GyI|&i`L7LQ|J#Eoqgvhn%3&tzn!BtE)4O9Clzzfie0ADpGghRL+nv-MMDtc& zTl*eppv)nUNBIMn@kgl=_!lwt4RX2`CyIjosOY=UT^N=2!+c)opk>~Ac>e0#C3AFH z>g`dR+1$@}t`o$edY&x!(nC~D_f{ zX)Pf^VGLloy(at#JG)PAw!W_;ZDO4q#c>&I~2TZL3CD(gXubm^Ce}{W2bptVVBm+x_T*4J1&1uShgvG}*4dQG5@s>mj0gfG!H$)s$!c7y%aUZv@gNf{ zOJ5+4=M88PAcOzl=m;qWFVdV08= zL&y2wxwl7C&g4qk9~Yu;9-E$~9R5gmrmtu|`gAbcg{&I5 zj!e3N64W8VxL-k{JC26JXES!9`^a*D61p~pVd9HsMnS~FONgzBa?#0DdYsy2#O1Ys ziV`8)(b|fl7~rQVB$cgWVq$ArEA%r9_FV?Z_4_pmtiA_E5I`$Sw{J-kq8bgNv~Wpvlq2VBj?)cKNa>N+JWQB0!{UWb}ZYLJ8*pf4+sm}hB(nKbJsnQcT% zkYnxs@uCg0p#kHAP~|NRf~naDO&sV=ck?9`7?Z%2!%ct*#O-j=GkSzIngl&@NbaPK zyk-8}+cI1iC>4TpB9QLUNHQh-)_*=FPYq9nOU2o_-?-_ ziWfDLzm)NTE^f9wWoD1UPAh>;LM(bk*h_YaeCDj^J_SpsqGyUwsY_lPsxKNeZG6+n z>#4|-Cui!cBqxm=CLzSE`F`6Cp7avF2nog{IRa@f2d0LvXHe<`+QbdvframH>osff zA)a?*b_8FbZdOsdg{E{$Ueqw2Q2>pFQG!6D_*t<0_<;)tMu;YQfaC^0XPgoTR_|-_ zTm!>xPZ~^_5Qe8Z;phO)P>J{xD~+7zh!Mo2J@A)+8TdZ;l`=Zj3bI$`uEsDRsLK&;4I+4=wKu3lDIg-qNNNqhM zO32T=L#lX0~ezdWii6~^`wb()ZKa9kQ!r_HcYVqV( zjg4ew4|EyPhL5w?tAV@^baWg+B!&Hw+L zI82yUPa*SgnGb^$DrQ12{uL81(belm+6k#_bl@qXlMJVeaOFcA6cp^`K^58em0mZD zq0i%{+^K?pzbt;J>UmA<$tEvy#qnK>-=tZ3rHhYL^$BKQ?SAn5A<@?dNAz_Ere@>} z-&bNJ37aIbu66rn9!caQaO-;5awj*R|I_{OG_rbBg!bi@grb4rbH+sC`hR&bOJ{!k ziuxX*2iNY3OBEGKU)E!XBcJs>*3zjdkSv9YLL6s&M5U`z!2Z?!N*GJu4>R4~?|qp_L2<$FB3U|q z^P(DAbghT?Y)B`r^i(ZMCiiKFwt!X=ZT+_H z2#Ow|_(w2ea>f;p*H`*ek;biVft(5sxJOv=o(0r>ycF8|?T_ySZWP!QL%K|u6TYWV zw2E>0TzMe@rVNh_7NoZuiTZusw(8k@H9WG`VUe(h@j}GgtI;0AQzccBH9fnwJ_e?9 zWVyl7tfx@v+&mZSX{Ms3&UIVQ7N0lkwAw}IPV>^Bfm%Se~)!Kj*CKH_pe{kq?a4&@|A z59Q0OgGnW6+e;i6doTRsj{d&w=9|bxu)<|tz|I1qI=Y;aVfs($XINwgX)-cB!@8%q zweh#ZEZU=AjeNfIZ~0r;q{|`>UmxK7ex=xJq(QtJlmpMP)flUg2RhQRVQ<0eCoTl0 zgH;PXPky)^te|tQE93baDDfudX%^Gs=>;6$@>9+nQygq15UmKadtRub7`# zTZtE;3wh@GZbq1d|K6nI;K>vn3xF53vQ()=s4gWO&H}_bJ1``@i>Yh-7_Xk#!Cj=0 z1B0sRCd8#eU8qfMioU2?xw?3%hH3Q&xHSB6`Gxu36z(JMb6%x*KjVGg2V-h@!v5S&nkB`20Q zGs8UMN~W{Ho_+>OYCb5bGG8;dwC?Ceq^Jw1!a0#H8!@<%bABf7uw3?9X>J5g%akH3F!q)-Td zTXupS0Cf0d+y$yu7TT6H@Tp)XapEQCn^>T4={J!se3m<^$bk-K9xLJAyxoX{b169Y zmbl5by>BYp*#n)QGhPJixd#R+3jLo~8~UziH?wX0uI~^#2?DpJ&a3g_o?3xMY5WHq zS9H|->>bBhJ?4egmMCSifW~!ZjT2zXz7i~hOI(c&iA@4gn1UL8Ml2c`HyL>Zc6Bex zJ4A6-t|dXOqTq%%1iO)uV#5J&M~`BCBTkyaCSi<`K12TLjthdB7)xdmP9WFUN8H{f zx9czN9ltq_8q-A?JDKO=&W*S5B#%e0&!m3eqZMFfXecfdz zvfA!%ZKHp-)?!P}ug^R}5YLxJp30WE&g(y0S-TleddOY%CWgs?Z8!I}oc3Y)RZ)9s zpCYBGlmZu{(~zd1<41PqL&4lSCG)(cYn3FcPkOTGwjH0b9m~uA8EV@MdD;B3hm}8< z^KSiS5V@w}$@U!?V(c#$;lqFY?F^QVda%=6<3k@DQB)ptCM?Vqh6R87a@asFy6!<5 z9x8ljh~3S^%=u z2<{ZZ(DVVLT+zk5UVkOG>`CxJ11n}$#y-)k;k&AyIIi}DVa?h6HinbqeE2JA(S>0#Z_LJ0XWFG9YFpN zg{96+{Q3}=`%UA)A}4Xkok#{Qe5M&0jXYsHd;?bPpI@tQhfTffH4+aD2dk1gbLh0O zOrZvN0HQ;NGlT0&5LvmB0+RA|8^*z^8 zta^P_)237-0oh8;ytzdv!dxJAW%~D(lKv?fJL667Dv!+hSG+YINX(q?pucG2Gh%-FaeHD z7)a#EU7dGXF~BQZ-iP+v`@1yos{y(qwLm(h@B6kza~DsYJ%R8&wneUQn-Ot35^pz> zRJwVy4PG(+i_5piCkQ@yKU}FtbTU%SE@KyE?adFTjYaV-Od|L*2kiIM1mHG$C z?BG$0wxY=e*Qug-Hs~jTZ-3L5OE&^KrGh8TJGN`?$Mvpn9l{mHs>IEdk78P_*obljl7{{07}ks7QCgg4-$t6IvC#6}wjL7ohT(Z4X@+p)(;awNg6RG~o5E)PfO zVKCjPBv!J%tZ8~G>=tnvQMAGEF$nG?VkW_cd;0XRhAGn#0b=Z1TGcTomAo5sS2;zs zfc%K%>vQl32z-NatqAA@c=l<)Y<@$p&cPhIER4<$)+s%=Dv`xHkOb>cznloMJAlOU zHA_NhG!hU;`lEMqX)cR|9_b;UP>#Sj{8sk~c9ZKvx4phPtt4BE@E(u;-jfRNc;GQ0 znILAJPvET^)>NISxWz==*(Rb$-rHU&pNNu=>QEvM%U}=Vy_xISo0=eHnd)ma?uw6H zFWYg5jvG;Pg5M5^78UZ+S?nQ?2AzvP`n^|FPy>u6@w>dGGp1CT-GHz2c~8m7cAQMr z>2U8gf-FZN#(Hr%{`XDuD&qU2aCo0itkP6?qzMa-st`a$Qe7=CC_?w=DiKQospTf-2C32z41*5)A#Sahi!LebvuYk zIF6xDyZXA>Glx9GjWgl9@YUG7;9~sZ^F}*{AS(7}!`iGQ25-F|0lqp_jrWo9`j&*z z!nBg0`w9-bMRD1nI+Qua%qKP)JkPr*JvvHM^KObuN!`Ru(73GSvmx=LWthg6VX4>V(3YnkMMOCy_}Pi76>AoV5wwHbEwU?F zC_4Zq_iuHyI1M$%RH=$l@NNUX0oe!uOUU}?jm*yt_noYG=#n>LBE5V7ypN937{P`( zX488ln#wuEPnuU-963=SB;<)YpR}F3 z9|dSLdpQ{s=nJgaMPs>r>Lq83EArLUd@oz2R8+comZb+wC_b6I>6QD zrY`}ktMEFw;~3)l7OV6W=tqk*6_Lu$+R&H0kvmUAjv_N|kZ;ZatN>6ET!QA>?vroK z@{ZLjm{-AI!w#T>$&RT1K^>rR6w`Ul?f7;%VMQP0yXhmxcl(Q6Km_}_p}g99HHCQ zs*YKv8zEap7uRQR&7R_cZ6hvHP>d0TcSHBo0WOGjzJH<8qPz>Q`>g(W&Qqaq^kN~S;mu`P z7hr}PBC6eWC#2XtzW*QlHW<-$bk~c}4yjmAY8RU7> zm`7x9UnIqmR9&4Lg(y~rzN-ATFpY|i&ChXtByeWKk1Vw(1|S~OPhKdFA)$A8Vk@_8 zGp9ie`lg|vIktCMgEdbfv3yRVn{Vq8e=B;Wj5*^pGWJ*%AhQ*uX~VD;0tg31AvS0_AO+n?eF6f4q|8KugjN5@yM)6_os zr0d>(M}EzU>?6$1_6-a~j|Ym!C>p5Z4|8({kYBmK(!aRdL&HRg=iJMX6z=lnf7`yZ zJCyQ5bF4tR>d`M-PWHQfIzgH+;;(P~@@F)yUoorN@si*AY` zG-o^%?tIiqf9+_`)dypqpJUzs;Uc*ONFV|QAm#Nr#iKM2+REvSTq5d+s9KMODh+s| z{zYYWIvw_QzTHPfDcLpnC?M6SNwPq4$<~x4ARmeCWl8OFdN>$@Z_IMR7H*&v<(Un`W!%<)NUBaOAM5 z*RrG2%$_~&x!Tik-K%3aB{Wkyv9IDCOq@zwbbH@D&COa9_T|d+^%JGhs+o{c+uYV8 zdDo}!-uffo4S9MZ$a@cWVK!9+O9bS%%+5yiTXa}mIF-I?8PI=wx{Pvgco^}D-c(6V z4Ig4%z!|wQUOEMIKGN|G-MnjJ^QRQDwzh0ySG!TgnymV6vo0uh^Yy82iDiFk>YdeD z5*jN{LjQ%_Xq5IiR#eJ#n_q+YPd}G>mDwb7^pRKS~GX$fSOD8Mm=RAS5IC;7M7{^imDMlR>;l5j$<%~flc7-AEBE+=bd`}D^8M> ziT}|6zhI3+79E_xCAHVBd7lb-0J`qXiA)%K2kLWZ+u!&nbJUU8YnIDuB%=Y3D?fk% zA&cY^jCf4PfmYW4h}7~SHxxwiXEzZ!eB??aI;-G;8>Af+-Pb?-6|PGHP;?&mbHLFTtkNcAIsMzX z$|772@X#NT9pa$1!>0D~*TXy>!87zAb^Ph)+iPpNWnDoq=5w{~sO3Q)FEa@@Q7;a9 zQzF(w-1xo(o4soNass8impc97jz`1eJK&ZEUK(0MU6Ht6!<4j_MMH3Ddc>8_D zgIN|+v1EvJ`bTy>F;JKG8OR$EIf6uLIT~!%);*?58gUWY=-4fao|KNH7zy=I8rtkrXPdKC!Y>n zZ$myMX|$<*as8{-w$|bVGT=O;~n>WbHS;mIk&M3V7$ItV!^o5r>)2s8Yc0T>1P5P}YqO zPa1^IvEJtjY-@dFd|=^E+#;EsGZu1$1gUtTmA!wpw6|-+hVe4m*hS%w(;Xsxra)kpE2LZ z%x{%w;jgeo7E%Tip8b`=ET*It%)TLTKB9?Qxp8!)H<*1Qfe~e=|6_Un6c37@Fj0|l z8D}c@vQp-x&m8-{J)Gv=XynelJ9$%^;Jp9#hXIkaPVd|KBu-=#6l!vO1N-JUZAsr% z_-8vauMbFFB0TqwM#_NrEf+qg**dGHll!4*^yvGS)ndAPX?c>T#gykON7(%HhQGDv zlUNUu>_2i56Mk9sczFGdlZNH@=U#!`Pca;A`P}*!`;?ab8l%6sVujYIiQP?%V@&Zh zx$~c?GBWLrjBwA3K2s@md>y)QBzODbpBL_(6gf`AOO1diSAcx-Vi&yd+dh1=W&562 zO_!8jP#%iPEf}NZ(7!uvTO)iqc`gX%jcb0Xvxwh--WNc4JwR#y9RmWC-K+cHp8u>d zF~cX?$5NOE-k1fDPf~8GcgHJn)W3(+8oH#uH>b-<(33_9omZw*6WWOo0FV8yhuSk7 zirV89?9*QUSm?Z?aQESh-iuK+`1<`lik+7uujDG7-IyMynrr^KjOg0gEgN0g>&`hA z*c`hvyqULCASQMw?5UEeOqcxU=L5WAhwF{hotHl)&OvN`LqYssd#U5d$ev#Gv#29Nm=VeJYFSIO;JRUwzSU~LG zj^j!6)71gvsghbsr{8Z;*bS>A1?8v|MGcMG|DD3@j_6x9`Cenq{-QGQ4HG5&agSGX z)S3T>^C-lKzIgzvYD>8tTnx(T=2Llq{{%(n|vvQtSBdHHzbT#~UW=wrzy94?|&DEOoMEuKRhVZ9gBX-EQE zUyrz-A#ch}8q6>XN_=6})UiZbQWzJN_M1K9wh4Xa%2vCC)T)ZAcJ?4CTKJ@EwX*?9 z5z+D%MVP2g_4zCHwiRaV0bJ*$M;o_m*!SsWP4Vn_fVFV z#J9PP7H +!2%3G}O*F^AXA4KLzLNmqV}CH?w`X$N%t-AfuX^?gg6hhph*Z7 z&!;ouNd+H+^rAxSI?koP#Ak;oeQ$4A$>C!FUwR`LIOBd<;1Z~s-cmm1fy)Y5yp4Za zAj%L6pHi^`7Olo&NzG{qP{nhH|Gb*Q3e1f)>Y|a(*HnB+c6I@5HQZ!y4U(2#4sS`_ zPvRj4rwAilLxCL`V`Vs`Ue8dpbOMx0S@tHVmKw(2`!I(?^SSn<{5fv2GCDUId@94H z#D!z%c=YQl)W2EVe}r5$1oK_EJ_A<)NWPYy59Do=Z|LKkNWotausYF-JHU&BE1MuR znv)nvjR!kDoY5&a#Up&oxHEI^>j>BOP5;kj{@3 zby9n7X>7acAmw#@%_&H4Gv(_c3F%x7Mr8<>U!~ZL2IznHTEb_h`F(}Q`}V+?b{z92 zDgf+d9L8e2UK8fezAaEmMQ?rlNaEtJHuFcCIRlPdr?ab>vmXyM(R{thyZ2|D5ys+B zPC$`r_KnKo`}W(|C=PNLhh*K5?Htcd^c ztHfw-K@nT#%Vj0198H$l#+Tvs07DV2-|s|-Yxp7!n#-R4?s8M2^o)+hQW^OQO!D|K zB$lD>M`;^VIxMiGULUT~)fr^BEk^HNo$ok9cjdKVAQ;yIB-?i09zQ`^hXL?WhI}&S z1Vw1;Hn@XwaR)xCQ00Nl6E&s!{$X#r$@kXzqJ03MJpU z5d@1kaz2loGFu9k!Y}!Xk3t{IGN*ScTA1OE^FrjZlL*V}(7?cZ|Aob%(7T7zZhu?_ zD?g|l;GPWI@E#oDSMAJ=f>;Kb@%>sTq9QxSg+govl0HtV0?(a`1yPVp5=BQ{%cVV% zy+$@WX82s4a@Ksm3j5t>cOzO=>F5ume};~5NRPD?W1K?Bs(kpX{zGXK1!Z&*xE1f6 z($Ue;0>4)IJRD&%2%?!jV0l3_)1lDFJ$v&;%7*ZNj4eiGvI_61Fg=3B)%9TbmpbfK z9d^7pOlY(5-^~GtxjGuVGamY)VHRQM2*J;Vs;P)bs zk5eW%(Xl+q+OBb>pKKYUh%g6ea8kSEo2GqxvzQFo zN9Jauriv_?Df5BGjT%+dHbe>Q4c2Fb91NrC4@J&`KLG+~z6)K}>gAdMZ&=wMz_H7m z&;A$ubeS7y2%z2L36w8yt~ zBgjFQ0~pt-m#;^Zf1uF4T{GkaP-oU9i2@U*x_~T)cI)aZlQB4d+46XvE0FSfiK$Y} zOGAOVr6mo#(I6-K(k%oS6cEt6@ndszL`r`sT*JV}BCvfI2=-#ljoYUm&)@sl-B5Q% zDT?V^{sO@hQ_ukv8< zyA~{PwtT>26O-yfe|@1^TieP+x=Mqd6CBMJkPK*h6~q(tfRE_RA1&-)KcObZ7)p%QkQ)B8~LWBG8m>)LN*a%|Rz zkgthA^OwLA`2Fts@XOmf$afoHjQ1T2NGv~CHkX%=uhMPIX38feIr5&I4$p`epyx^?2{#cEzi+-kxfEOAJrdblAfnLO-!g zw~$gp zoedbR#??{3Kd4PzSvm{OdAN~*9P}zbLPQ?*oe#sH=AHjWy=@;PK}(5V{F4I59nc?0 z1{V}$P?`(UGN=)$(cNyz`*D^DyS74pb>~y-qYuK7psG+80O2+!JWsrdW=?u{zO@4A zJif}5-U}=3pBg>b?GdY=?_kP>FvJRMk0fPll>sPWJhr)Z8x4H@uomZFu`DDiD}=cD zaKXR?S;zw^$Seeyjx{vX=V;0%jXDYgG>o3wG%@Z(AZ+IbiBx5z6+0BSWQ!3LG>D0t zHrV17v^|EYbqy)PaZ+uq^fj3b5*_s| z&44PQ-!4<@RFv&zL)q%gpnwI)0_GeDU8T)x+zyTTC|(JE{D9*D=OQr1CdJawPOV@5 z>0f_c{qEP&N2$s)uA75DFNTAF89;$EOwHN(HW)8Ll|hIj%@Mb?W&fY;K$=x0y-y-r z6EOT-Ef$!4a8CoD*Q9~x!}GsSKzh^;&7(2(QbSHSFquI>hIta7{=^=byH0r{LZXy0;ebui? zF1k(~6Y=BcT*UWDLx9_j?lS{3GguIK&xIkF4_pZOPRa7@zOrTQ0HMv&WX&x~!o?)6 zmw=EZi6JIdtF&wN)yNal&Z0?EJ~9PMXLy5vE{f~ngX*0SM#Ng6r75r$Hu*&9T28Ac zoqmP{GC3i{7P-{P`jNDl@cMy$r31}Arpz4>zyinv8eB@(+p_+6us-xTP&sg_o45_4 z7}QbK>ITT>qe>&zaX1=CmOmBdO&dd8KT&8m*}VSX?`Yn7xGupP1S$k(OhCj3)Re2I z^qo;o(5v6qWM80`ULX{#Pdi`7s?q zY*8g%19IiJxwt*^dQ?T zQBf&<%@$hVwgEP4_oHmuxa^PBF-nY6)|cNMY$Xu(WnFkE z`}!QS<68P`D$kUVgj4WG9vvNR2uZwy@a%ux--!yTipk9zuK&am(QFZXxjZLda&lj8 zB5M24ddqqIr*dxGHBH^^i|sOQF=4|91+l8Xm;EGK&}#o{UzYer@yBmZ9pi^=epijs z-}7y0HfT3a*qyB&`zFTs5v&4qDa7?J|60{O)QH!=g*WqbV7HpySe*Oe-cmc!!$YS? z5%sQv&oMVePP}3mE5e(Y6b7^EW}I&Sud@D%M)pW76RzoSA?gmeXCN?<9gYQ5dVX11 zgeGejGz19j|KRBn%Bv8*_u@4niW;=T|3u#-`CTY>$#RELxfQyQ;F*YU5iqoVEkrda zypPkD_;Ar5Uu*@~S2}>g9mmfpolov#=6r)Q;)C;2HZ$>v<@xtx{-MX^0kABo?Vy0# z^GDS)+vn9+QY)EniKN=N7;~xX`h1x~KTe+zF6xpO9Y@yO?Ki9bXiyizvzq?y&(jHk z)VghH0_g%(o7{m-i%~hfsb**XeN_Q;IJi$U2h4On+dM>y_gNDxR{yHvi7lagwwv#Q zgGR}(t(^L3)>a6lc6IYFpcQ%fB-b*Em2s-s`w2MxD0~j2;5w0@7y!gTwZozurRb*k zQHtoY64iAIN)~*`=1D=Gj3kbHPWueM{0U_PvERg$a}P%n`$g_9{UJ4^%Ak{RFFD;xPRnE6 zImWR5_qEu0n&oaEAOv7Yat#Z4k?svGqhKy&O^AiMOAfwnrFYq~br2J2GDNMpx> z5H#sk@$w`xBU9{Mh~zdS($rp^h|;>cBoHa--mh}$y9=mA`)_fv*`~`S>|1lX2Y5an z2ZV`@)gRtCM_FEC(145TQFgws8KV-?`1|ZB;0Je=#%)y#szICx`zrkKEAe1EQO>3U z2PTlDF6f{_5}igxZc{@h7>_`%Qr+MwUSh#X49%d`!;sx(r2kZ8_VNZ$(n4h^1s?pD zR|K%6+HnK{A4?DpEuurVe8izLOW1<4>48ZG-sz)*0{)|F$$g`0Vk-6e<&kY*n*ZJr zb(I|8*wXPeg65O!s6~NtN-NOta4c)(qHAoJ&;R>)kI3dO;0zv`8&?Ac zANUoMN@-e37`xo%^%)*fVba`E0Ps<&a9Qg=fcL>~fs~dT-eF0DGv9(?D=;jB(*R6j z@LkJfJC0^W+$m#ieKo*>s5LykT8JQA!+#Iv0Z{!P=$4KB>7dDTqkV^9=~=8A)!YHo z8DRtiXh6T%-TlFM6O0sZTMnAz`{?VX zUZTf#KU>Kx)7qr)MLN^n<^=zz`FuJyISFOg-&XfCTYU8+*Z<@W{jhp)FdM|YO=GZtXmud{02j~bv@K-ja4A=sju3|1{Mkg^5LG2c%JaBgzVMps6X|yj z!kYx|UsI3M)AY*S)A)-Qi7mn7jM`m|m;KN9-s!Vxx&D5y7bnmy(gqqyDLp-k>AEOQ zB0L$HPzHf*W|GCA?L!$tYHv{INiTif&tIJ`zu{}1%9`fpmm;*fX1lX?&wZ_)ZeC1D zWwCJPrpHZGAJ6K}hx5Z)NeSOaDifRpJR`A76}q3;_8s70&$!2=sNrw!hbeh|s&021 zRNs%fSa6i_av-y$e28^3mP|qO!Zy#6-qh|aw|-{zc2*x76i#HE0yI`iIG2IXY#eBG%Ri!G}V=wW_wvMY?F3ocjB z_}_!cF7c(n52UYkPsz&iG_xBhg7&4}xDd2U0AIB#raQ0B{{|@LFU^{l3Z9Se~zidp)bSKn&M2NaTyTNE7GJ!($XL!$c-mtdVMw(+& z14PgY62!e)YN85*nwq^ z6sa#F`-dOXx4Jw2MyJXI7o&gX-aA|xy8?A~r;Fj-bo9_JqD_8>KR|tSLlr165<87` zp`e3AIwGY@$Gx6Q8ORiC{vXY|^78U<5{||}E^vJT^!2mx&A!cd@SY%{u-KHG27{6WG% zWp#B_w+BCVKpJE!_hw1uBS!9FDtP~VrU-7QbvwMIS>3Yt)Yv^>5(5DwvT%LFU+^6w z|9)b0Y#m-{GNNADx`=6G^H!Sn?N1RxK``S$eDvMo5!nGO zgI~`t!yQWsi|3@|22qwxGMby`Hbjpv{;D3OjRX#^E`E~2Tlr!dwe;Zu-twRQ!R%a5 zf&iPP+hVExPHpv;aed2#GBF-?{@}pM6qA11gyC<{W5fk<(DZTr-~)hFSg7uf2C?-H zc~<2>gg(ciW5uxm|N00Q$)q<_`(xetC9q%frt$21-?;6gWMyQ$6;jZ) z+?V7GEpi}D#d=Ry^LR`l;WB|cS)V#WR)sY#KYJhmsMF%Peh>$xxw0wd$ATYv+8B7x z6Z)ox-;O@Hj$HDPp4z*bjkp`klt0Nq9Fk3Qb>Dn!6R1?U+)2i`^G#icP&IYgby^MG zkL!c`AYCvWci=7RW$qW=Bb&CoYow&ucOn_pS(m`i4HR;0C1D)~N@>3=dq(@J%gEe# z)T~L)+%`RFq-mXeIj%)+2GWVZYfv`eHdTg}ySpJ$NLelv)4MEvM`6r{0VFxB385Lj zC}j}_aCpb)!~(tp2v7|Owq6bMdr^??+u%TcZ?u`pNelLYXy6m#Rz%cS#qExH0!Sfw_ z0>L7pa_9hFFoz%#AFfcaTLNt~!HN|NxDH-evYACqT>CxWDZ49KRST~>yXTj}M?X!K zQy)hlOQDOO7D6v%59s`nf|BPT?eyHPy@wQ%2wq?8Pc?b+LtF$zvg~HkH2!xV@M5Tk zCG5Z6O+eqf(*&KZY;FIK1!xg>8#4ebD5-PDkm3BygIVd}L=A@OpS;XVl%^|PLn|DfS>d6*!kUjj6a7C!+yqH{F)X8Iomm+;M8wE!NrOVrA zdQ=lM15)x9kNU(?qpwPyYHRo8Da7e~ea*5tS%HzI?<;4(yw_Ac7Va3G!8ZClW}5n; z5f2jG*G`(ep~AjF_w(#Rg9b^wh6h}*((%k`yB(4M0u zFfhm-W}tk>Jic7tPV+vRLPNdr_HxrO3Y0+IkdvcPI!+&_penc=Ux~MB+ks5@<7n98 zp3`lN-!1FcQD#gm)9=%$mb2T{u9$vbF&EZnZ8`o+rH1#^eZ97>mjpZQ&37!&_yO)B zh)hv{Y7y7Ppf72zx+&zc>9(gEF3O~nl2T1|Uq9+|?1R?FAM<=mFe0P*#+>2xjeB03 zyf~SxUJ%^bSZznrL-~rCM8fD2BUNzYlYfY`=|_Wx7>I9( za-(j}V^b#xE!-95gj*K{#f%%_;sK8Kd<582p`T)C&1HC)N1hf6OOwF%511=G&vRZI z67t&EhYepr*TQ?+2sRrKuK}t8#+%oSa-cSxTg(H(uwEaQsXx%FfJp*6#-=M;5SPF* z2Qx*VV-ZX(u=)6ChBH=&z*b!*8-d6-1FiZnJa6n0VDr>x;eqvTO8-WJq6{xJO>K)- zp&Vn}0stc$`>>}K00a9u5uGu@*2FC;KAx8XSHK>Mf!nVREa?;@@0A%FHw&aG4OeT1FLX6x?gFsg!1z#=vw{3)lIc-aR3!+L<@2j$k)zJY7jej+57n zO2wKI72{xYCJpTzDjJVfX_8gZuJDQfNe2m^BflAlLrfo$TmAbA1U(C}(%754x2ikM z=Nup=6H1~&3iIs?=`cZAnp;~*ovzgfA|;T!g7ydy8DQpAn&2M5Qn9s>SGj4h+cP<-p1WH#m9X8A>sR?!>F(ZQiu8fuH8 z$;yD2n*8^zm}?kWfMg_dD5hVqKF72D!VxmS;%B{Y!1%U-(oC{szZ;-Rjh~?kgf=f+q!(^e z1n0y*SUx8SIQ2cHq9LwiWxgA&K3tId4)YqUp%Sp*A|rk3=HEUM5fOnuMmqS?kVU>o z&nR@Y_ag(h0EMlGM1Y?Ezp(o(MfqGjfb%q&$8rxAs+ZIDbrCM|u*{#((B(N1i*8ZY zDo-u7znNb)yXzx;;l}w)4|OfIapq((6~*0ifkNM-O-7n77Ug`;FNAYFlSJe6Hng6aa9S;P>%3X0`N0D6a@_hQ5r^s&KtD` zR_~1>gD;S;Ia+P+8P}8f6NyiEy>H>o$~`ap$f^B1t%p|6CrSz4KED>Erqqc=* zUF#dBqJKi7n~y))nFbi@mrLZvQr@EWEdKVxS~1a))%i^0LGQI#3&LbJQSsR~l-SJ< znA6_m6coE2O=BmkkX#%oxI&67ydVtgx;n+XmH?q?;QDA`wbd{kAZwxjO=!rR0ksx# z#Eb0ue2BYb1jmp79JN!9Etdj7zF67WU93JCM=^pdj4%2e8JE*>CmX`A504%>cN>fg zZq`zWH`sU=>XQstPjd0)iq$_Mar_K# zEIeOoYZVoP)EZ53?RvN1|tIlnteX;jLUJ^9_y!u(Bi3X1vzVPM2 zyw!ZIZT$8n6X+9pbphymcmWA_a1=-2p!>1Auk3)swwr%EcOun&GyV8yqYD$5GW;$6 z_E1zbe!FqL(^Jfp`7N-v6#V$`5&-}P7xABB9P-+O)`qp&y(&GQcssaSy9!nvif_%s%Y>~nV64}uZcnxk|4Q65_AMje& zDS*x+J>{)Wv{&;tFnK_>jGH#6{X>MGFlE?HMD16_hbrIUyp3nQMfU*f1n~E1V@obl zxV(V(s~D3Gzy}baq|2&QYIa)<{Xd${JCN%B{r@VJLN=AXg@}aA$S9GK?3JCp_h{K6 zWMvg0Ssk)hR>(?{y^g&d>)>$C?>hJA`}^bmqdO<(ocH_vdR?#Q^?W|AfzdTcv4i3; zGBgl1X(ykb5K}z-d}W0U)^JGrQa<_&*8%=Dz%W3-`s2_SFymVar)mz%299aJiW7&; z*G*HSfLxIxTBQrZdn&oYhTfjfsy+ag2_j+?iWG?|5gHU%r?u6CSyVZArC+3}DFram zLCy-6;8m$BP$9o!(JCNIh!1F1Ejh(or;OeDr2SE1mYMODhc*QZi7 zGtLvP$rc62c_F}@x>&ReR5qzay zMR4mXNtSP*UBD?czKRSgB4M8w3OeyQK0Y3)ap#7SdKTD|BX~_o-KGSv!b%$su5WB? zIE;OdV2_n@E4p8*iFfIkLiWLBP)!VlU)?7?hw^6$gStV}dsORXN0uMu3*X&Ep*cw#vl=&n z*)1FeLJyg0lX>RH+CJNpUk|ZIjFIEVU&F+DCYcp8NLkYrW-Af`%nRNRV5yBm4(TL` za9sGp72YvyQ$~Sn!m1apC8x9X(l~68A0D-y4|1vez<$8|aIdMhNR@@(ZS;9g_xAZS zOumTl$Hco=m>nHY>$r*2eNy73$CL~%pDYYfCTElYz2UBh>G{OVQrku^k2<2gX!KBW zAs#yOGtx9qpS5r+P{m6&pNc&nc%C~ab7&NoI!IIBSSIrI7Y=I^wRe2psro^5uM`{G zNUbLyG-*$c`h%5nwKV5~f5IU27=vPK&|tuI(-i%HL*CzCa;pK&1ataIE-7Kp1DZKM zD@Y3j&Sa8)0RZRhz=TZ*uiJYI(;W%-r z5PF1N>E`t0fEs@sawn=^j(WDs#$+-3C_k2$go-fE#?&vPTwU@~qDohb;v?~c@cRI> zw<`SS`s-2$6#dE!QtlTZYa>VQ<2yT^F5!LUN}cE5Bb~;km!}7rebHj4Ddl|u*;J!U zJcU7QNRvOw_UrEN(c3S8-RyLS>TTO^GH%_{x`S0aReXggkgPpWJY@S0YFZM%dEvWI zZAIn^qbv=|-fUe(42dm&=aJnn{=W708i7G!| z3|Xyo8TT1IRI_CKlf=>DrLQ_WHMN~>ICm7JfhkYvP^MXv~iKa~?9o#S`CR#iL9PnPa896gL8ab^ncNb77QpAP zni@W2xA5`wV3rI$sSfCCYtdQPv6Xdt%bml#)w}0D@jOloQ5A?KxyGA&w%Fk5=}TES zov4JwcQ<--Iy$nSmOm+=9?Vmn|I$ik0oiWCSDGHzNr4-aBd1(oB$uCHM10{K>vy|5 z5FM`bt}0(L@&EmI#=__k`^Bv$G3YYg%h3WL9^64Fwf*YJKc}yFvbDa#2Y{H;HKI2V z5f~}{v+poEr1*MR+Gsx$KH#T;FZdp$8B3>{Y%OhCZ{(1jRsZt@Tf1%4;u`D*OI&IF z;5GyI(+duXhk50Ya}r_YtV&oRIg)qP4mmRM--;SCZ~ZtN_rPie9R{c>U?i|){(^}^ zzt<{&hVm*Ot5Oc~3B|bDnUf-txkLt7AX+(!ysiD|*U#hty$b}4_QcG9aF9+~YcTWvPTVN&NNjyulg#&_OXdh2!H$H_Hf#qsi$ z6d5))AGc=_Io7;BCqr!cS9F)BQA=m3KFl*9aN#~?HdXgTxwXFjn8ifX(##XPEI99+ zptvPueE9>8izzW`PoeQEEPHzJfWcc@zwKj!>(nLQ5A|+ICCP1zQ0?CC6F*n1)BOy> zBh5y0xLDv>z?f@m*iWC*wctErq!099QVDOS|K?pV9Dcyp^BvGcfNXkRT^)u#kx-p0 z!N_kVm%(rOQ%~ycFw|T}tmgw$KADO7Xx#CT;}>mSfRWXIeccAKxOw@9=XPI~3YJVh z6<0(^#L|Caa~WyAeTIkjQIQGm4Aswy?fKh+!kC%VYU7+GU$8>Oac9o{rHWO|r$2Iv z`iTTfd}jIMyG$=IOunxk^-0g&p5ArlH~6KWpAKi!nNd1Aw?FAQaJ+#0E}!Gvhtc{# zI)O0K@`q(xxV3UY_qIOpz`qlS(U`5Vfg z-j_QKS_{Z*@pGv=%*~`?{`~p#3s|)31x9&={tCUpr=G)z@+V)BmHKM%OJSX;@pvVT zo=A|+dkl1A2%@CcY-Of<{=5P1yl;l8^E>B_9@Bra#oIHD1Z+UUHkC!Mm-md=^V;i_ zvL&h@th#AqBsaymzDL^(MWUg#X5`shfybREs7q(inFK}W@(q`{4Pk~x_;rY{q*WZ_ zGK>RNo$R^N_};156rd$Wrv_TTU%Cm2CofM(UoGY5=H}L_+vw5(;Xnf#x_7RDGen#B zELTn@4WG`pXQvVbP;KcHhFrQS19P}0Cyhl8ulqperfZ=LDq_%aQ&m4OPrFE0o$l!T zfeEmYKR&xg7fwX})r?LK z#&&(o$sKqE9-;#6Z%&gES7gjC^vk08M;)Mt`T(KFlJMC5$STvVr63^SA$1`HxU|^_>#H-Jr?D`7Ro{t9&TBt&-FO{fpFYm&(a&7NLY;* zvwW{N)h}?3)g6PD59|1cKqio+vWZj@<26JcL+VV`X4}^onsFe&&3#yCbA!MM|_B%#4U+yCpzR+tZ$7~`|>Qpd|V zM((|LCP1686){!s-n@PXJ9vcS8dz}SAMU*`jmX?1E_^71?fWuxY;o}OWW%DfVV4y( zeUbq&qtCUQM@yL&uiEeLEd}4_fB!DcBfxvu{>7eD0Ca9%s)nN0?+f*0JVXvub=B8; zaR?M&$MpBJ?jo;?+~$t2#?ZX6q%9rBR6Kq$<CMa>B* z+1tOIWHTv`HmL9gd9SS3GtAs{RHvoO1V7DJ^@SBoD4~}ZF@yh1XG{JxuTX)0ADD9v zXKSuo{kOJ8w}Mdhy_Mj=hBgzbh3|jCC;9pH-bF>iGgH1xjAP^DH6Uba7Ai|#2hB%n`^UKv z2T>CkUXHl3@Q6E61qBi4+kiy+{(MjzKstDr4kP|L;L+M@YHAa%9gOQRU#;l=an&|x zK#b+1Jy28hKvP@<%$^(Ki{bb2tcDbsKnbU+v?uD)iKNU_$ddwxkhDPZ5-6fYJ zui1iWUI#}upN3+By!6TfNR=7&8ED#?JV_iK|NdDMWcnYG zTKA2t-QKw9cYHyw#_QaiVVZC7f}|I&}^XhMwoN=jsa-4db|;Z_4kVF*n<`~6%fZ4WdL z`GKp3GGm?nPBrHV;9G?PDUNS`=;FP?_BTcX_qoo2=41xNvqM#2Igj2;N+yAUaHy4VEF3Jt3O=K!1oK7IeX# zk_iVPsbR-xUATr|y-afD&%o^$JS`=CG9T6&AZ-A&Nx*y#E!h}ZFy?1@(t2Z$y?9^! z^_mHjJ$W`1$)BjgmfUr9FAQPWD{?fkckva_E;F%fa|9rlK?(=30hTP}N_=$Uj#Eaj z(?4ShlOT-W|DjaHgI&9K^k=c0`^n|ZWrsRZ>z3YAGu5JlXm->@m_TZQVm^64& zKderkEDkpyLF{q>iz9C`IxmACmH zdz7y+3()JiEM@BEwK8xy!8roR9rzD{Pp6s#spejBJQPf@BOX1oi7+pamrDF;`S+1I zONR69K!ucp%$W;1)c>eTXanF?j3(E`8Rz z5hHVK3-KaXaOvEGYXsu61)~l)JF)cGRMI-tf*(H3aXhOVJgO4NCjV3wT)4#xKRc?* ztbX`1Q9ITx%mll+9r(ou96Y^-W z;A{2^-6_67Ey4T_!CU^J@vx)B(Tn!I?Z`3m8nsI4wT96Lp@L={O}A9jOk3UbX&2CPg#y~v8yC>wAM8;;5r^T_<|7C5JZ-MIz2y4W&{CKqMLUxpZJo%g8 z#<}D+S&9F2AerW(;cTXZCkfqFw%WXr3zxMHjN6#s{?_2$KZxNR{4VMPUU+~V#;e$% z#2u?ML@&Ii04>I?Cf!Bku|yg;>mCG7R9uIhKsqpJ;tC%Kv0#V+@YrOAF<6dsfwN%L$tu)nH9P!G=$ik%>M;}u zE1~UIG#vZsl3OF?WX%<+)pKFR{qB9RJ=QJgiVH-?sVIO9K-4j4gfi~>DaaHB!)I?@ z^#D?RJn{XSma^O}o#HEGGCs7T$@pNaN!9BbclDyLCs!nVXo0o8*nE+BCY8=)wlgQ@ zYm6s9C4`m+n2F+g7Z7BUJbPCNUG;h)(R>g7H(;b!7)fCB`Eh8?3@NBC?a)eVW|Mr< zjOL!vCF9P%Nqk+{x?X{n8M2=Q>2oPdGArv8N!Pg$-)RnJVm#=kFp-u!Bj9q~nB>Uh zX!~G)lXe4lZEY>kI$x@5UYr7vH8;_-Bn1lz3~^~2A90KV#A zR9EWC^q53Za?gd%wxvA z!H=GRX}tY&956G2hZfFGP|m-@#cbVBDEzIJfsh+Mw|XbSU_bgg?mEo=n}f#M zk2%!#Vw$rXbZG~sruzDWvF?z&y~jd6V~S;xJ`@Ry~uOtDMu$;lEYLrqpH=YKvSBh0w4@sg^m zc;fNmRv|@He5d|wnOVashov27*|5yL@<#elD1KrlsJcD#$(t}%Q^%7QU|Dg(VX`P% z9wGtf)>gkKm1~DG!*r!;n_ojV1-feUkG1AKtKH_A_o7Noo?2S#!}~;Ml{0^Iq$O={ zB8O$lu_4}&z!Hv;w*5|uL!!HhvPE<~=6m8iSJFGx(2F%oevU(#xQq%b=o)04J1cvb zbiMfcl*tq8)=FXBUfrBwm0iiBvr;I7xJ8M(OAL@l!@a4}O}>#>@H4(AyuP*`-%@Cv zh})Ppc&b$RGS=(q;X#4jroP6^s~TRv3+F$q&Es3=duX)#X3bKJZ}!4k5B|%7U41Dl z7z<{dTnKwYr0bmc%I5?hWCVBhN?sZ;`N#Ca_A7F0YEs9Y>Y~BNB+Pz(@n^Y3jhtL4 zteKyn->cnvfBuf2O87_N{L+<^l1CG>v~kRiz)0MZcZ zkvw&~;N%&oB`70RRHk5XyFPi!pMPLl18yQSm?LV8l)!2D=rvt=*Nb7Re8A9xRlLUN z84tD9pK2~bxsPyu|8LwWw8o%(1TVXXXjo|<-s*RKb)xljHSy$Rk^U0l=ZmLVlWxQb zd8(XoeeYCrAClo77*#9=P6hZ=fb1R=h^-8a;insP zgx@^7J0QmkGn$wKx6kMrAHAMk(?JBaDhX8YU0%%cXxRyJJLrcA8Iqj-o*-1pU1XM*qDA&rO)ntAn9UXgR%*@Pe`{<8Y zYz010kaw8)9-+)spM29$R5x~DWJCdd?}dL5W^GlxS8_pn$_K-UJzTa}A3?l^_A}21 zmOBnZWZSE2&g_x#uNB#3c2eDr&?RjSe_$C0wl++~1aZC06C+>}fI5G#tdm439!@C0 z$YfD@XrH6+=e!tdeC^kQF!*8o_eY-OtG1g9e*-B`r$n#sXtzK6mQ>z8sTyeiK#^D* zj#0H`bf1VIA>=R&N_YrumXClxUn{{;#LA*{1?souLCyhMXpV_8fHpjo?2O?M-MMfX z971r9l7wUfo2dXTu2JwF#2;uTWb?%LS>k=h?^Ph}MY`_fH-nmlH8P% zYyxcvSyfmvWnk+n6xgDt)T^s$InNFBOYZ|>g#a zd0P63vGnlkewrJ)kkZn=y8p*D$+76&uLzI5O943kJHdW6ohNgDuX+e46$vJgmd=PZ z9g*TC8Ypv3_cVg$s#m`j`asL18ub%T*B5*bZ5MEq!|Yyg-G@N=0dwB(06}b{=47WE zFh_tV!Gjmvf8EH}Hq^&*<3qgI?J!x50&ReyHtU7$R&+oU2P7c;^iB9jows35&bOc; z|E$pxU?>Ao0g=+f;NHfX<da#MN1>>WQwoNH_d_B&7uctt zzx??AL>BYC=0)ep(XKCy8^iPYnka8Rhg@y*Ow)`D$A7u*U?O834DJm+vSG z15^@G=&Z56x*N5iqPV_77M|;-k2er1a%40vyVbShms!+T`yif7zvGo`mE4jN%=~of zv_##px4V0JO8=6gCUrXu2x%*>t8I%Fbw2Hp!Ss!&_bd@-^F$({x?nvEucE{qtnVTL ztuMr&7S6o2On!ehd<2M1cGKs`q%?ZyE}Q%7HZ7AWwYk->2^d!*Z!e9EPO-vgzd|T) zzBThs+@k4J@!ns%3Uhz&CcP5vHdRr<3-X!>4IXWDn)vgwYERu{9+FsN_|e0u9U3H( zv-|5DK-P7tOuw1EOi{H`2)@Nu>AfF88>L9t25n?@YC^e*(E+~A)pe?+p1UQr{lTZl zV)$QmH9Eg7;1kc7!$E%=&KD<8yUeCZt?0@_P2=sc7YQf`{8yLKOVcKg;PFCZ+JVxE z(2h63DDd7?ntO=QDhOwG-Xdp5SndbG7G^G-NO3WAD;mCMO>#nnEIH-5=5IcIHA}kv zG;WGI;owx{v74HM7i`Z`VbLYWT$#-1iO)hrt$yv-i0Z=$2%mWe!)hYzh; zf4o{)IIf!37*<_asJ>255_uFLH{py+Wn(ef+1Lul-&_T%Iz5X!NV1ap<6sZQ>(NR^ z2)BxAE1Tqzp+Pab$%6wYL_?h^14$_UMu}tz55#^<>0M}75WabZu*KYF>!Th8p-@6^ z6ijLhKgGTqncPa=9M;fw*jlBV>-s)2aGS+z|EOJ6$1m&hE0zNi!ZtlNPIx7JLzCtL zrsRAya_h^C)a-~iSBVtcCnrzM?bpO9$Z;oaPjFRawdw#iX;Tj>%Nu;IDUci$B0ozk z85AuTvoT&3C5svDSBULk--I0s3B4|Z(|S5ZP5$I=PT@BtQ6)JaW;F*sa+CQqOYdhRBrLaKPvn8D(G%;M* z!=#jI=TAIW`aS1$oTysGqy&Nq)$sUSuod|5`k$72`YZu(e6?dpynu_@+OW-Vm-xiLiK>v7&^i#u(3jds}TuD z2yW~0(N)o?Bz`*G;>kmtbRX2+ZS9YA!_Q&ke6!tZ_+<`;ZQ3*HB-=rkXKSo;*ta<_ zY+A8!?LIIzXG^7|yJ;-MJB_gS&ad`r4MWB+*ne$0FHz^;ei66#Ua#OS_6n)81(ID| ziaqboHKByA@TE-u5~6T8ptj=dj2cOt8jq||Z>Sz0GCP|B`U4Oi!6e{D#0PFp)@@|x zfIGAVea)*+HL+XsgEt8p`7aU>+-X50L@yu`Hp=E1iIKw7jg zwSzhEUDb3_$!;6fk{g_n*t7|?V0g=McS4)F*8Wk=l4&O>f!Y;+a3adgfh2fE0Z91( zM|hHvc2FV%L4nV*weXMR;HH z>?{u?I9I3)0YwASezUdE>^J?I1-hm-+xJE3@m(AnlY!cyeC3Q+E?SpgOOT7zx=}-7 z`&R{kc962WUIG6$L?MbQLwJ(A9gYqy2o5Ifou~XTH{Ws*D>J*R?q8>CJm7_?1e=XZ znO`Z7UPTE!LwSptd&XE*8SE+TR5OmmHyn?$V~M8%c+A`AVo|jy{MH zcwi9ELPSKTC`X}3c%D74Fgy92vx%bw?h zT|Cc(V5RzH%Aa_C?OPcfx?{X_+Bog)ay6}8fgj=9zzrgO&Or8nEzG3{W+2J&ge5Vj z$XHCyYlGbEAv=mluB*%kgGn4?@v3zu4BbM7zPrb$?szsJ@ic-`+39L{cPBh_u%4KzDsaIuUV|) zyONtM4y{@|q9-*a#S9Kt^ovz#9eyda} z$1+L#1-R6Jumh!N0A_sJdbG?6h6GQWIXJAJGU>qIPhF@QKP5w_$~MrA6^C|acoJ*n z{hf3g3HJZ}wO|6xR{4ZXsjl^E4e_Qzq*tc1=ObD(>_j8a9p1<$-|(M&PbMvj^O@dS zk3ZAOHw}5u`*GTxugFv?e0{X)w~uex_~W`=@lC$R?EafK|C*-w431D=ZV#?+*riXJ z05UO8-Fg`@F0gr!p|&a>@lx#(CiT_KuW@>Q%;!+Yr9u};(T2saO#6gB;znsPZiS5Co_B)vN~;yQmZ1peSsE|J$KS0Q4%(OjVHy!khg@3KG*qN_~Z z&UKVV#Bh7u|HGH}+Z%J9J(>g2(?+#5 z1v05)!rcyKU#oUrsX&2)>7|@BMzOn8_lh+Kw|2K%y7`Vq8vQ+odx+a9>GRJ*gfVHe zh7fpC4!3&lcjJBuDF>OC_uveQ`?8p^#L9t47PnCnX53*1zR`V+iD{~*V*k|I!{hrB z!S5X9STaFAog#9tI%6W%H~mXgfQwzm2*>zH3G`T(lgkr5jE8Ba6Xr`;!;3Ep%oN`9 z-4SAZ8!<#RXzo^-BGzSPpvOfL4t2JI`5s8zG|P6Le3~20>euBdzB0$(d?M{Y4vuQ-?#P2IZrYv{u}E7rXGny5eBg2crKz1Of8Udur*dUpS~ z4Ze6jk?YX}Z3mK@9G3dDR8&-xm7uc~g>&9E?`?bgAa~Hb>GWW2bKJEfLqQ^e3hGt@ zEqeEIQNs%Ly5>@uz+&yW+l0^|iAN|w7B{Vg81c(FV4gOf7w9JFs@ zB!un^N(V1_ZgK2{yaNBtS#mfbWRS~;k8hq>R7D#U(~?OaeAYwQz1EWy;&f_UND?zP z1?W^;{BuMQFArrw(a5uo3__MGGKVvLOCqM!FCWM+@B5tzFAX|PlaJ>ZaH?weY-%zI zSWU5LTav+@M;*?^66+@^N+r_N$UiK~x*VBU_|NIz@CVW-**71Wi!#tP+se;h&fQ!a zhg)JZ&rF#vP>|=5PKsWL2-1bwdt*tVm5DnM(GkOZ^R?-+{tb;R7y&=?+cw@&G)u z!56u>xZZSGQbHKu1OB`pGQNS0?eEzbm}`~ipgan@Lg1{?i1v-c_H8M+TX>XjL_aOh zcG3~I&#zyT(_g@+qd!L&uz2&8OGMSSdfOensk|!F_qo+n$2vGNuX6#?NVMrMv zXd>_n-LRd9b(d{=^xo;H@LifC3g&TpFcQcR0el5)cI{TlID?d%0;p)`0El5`EHzp` z3Xck!CJ%|{@2j=jb(m~w9HU*r<_YK&ZS_2_;f)&XD~%|4$FXTLb%e$4MOId$Ba*5Q ze~L*KX9PPMpX@A#hUbbl=?HQ0XP3b)!(+|lZ^8$>t1y!si{ZFiv=BND_dK1$;Mu(-wcY8Y11c*X=Nt3 zNr+h>fo=C4#s?xH1WNr$;{m`oUAu2q)lc8Y++*u58v%bsJ6=@act{B?m<)W33p-bm z)?~Br3vN_U@W%OXH*_T7B~fz^50CP744|1K!9jC>0zHG1m0|BL^I1l4w?FA`;{Q5i z;EkbDmlZ6jUcBInVcw!c%kCVe#qd!-Le?ak(7^hE8lmcUs4O*c4v#9W*O>@dYM8Y5 z<|3!mHT5n}NuLU6Fio;TjjuUVz0}YC_8uHT0G;kmYWN=KDe8TNn>^Ija57sqrTb2ZW%CGZ8^a?eTSr80WG^!`niz zdS3u*H5E#`vQFBRCt`S5Jwm7OCYQB*_Q1s6N6v@wU>4FUN=>1Fl}6;lP0fOTiu|b* zRJU2<_};fUh;tciRQfEY+SI=ExyOL&VsF=Nat+mM@T*86QFuenNRN8DmfsGw|Cr;j zgQz0U_Zs7)*ghHMbm+aLIqss-_@&W&;X`x_T(1{b85ybUxvbkdu1MIHe_W-1IofoJ z?%d%9Y8kDeW5tm5c$QzS1?k>y6c(-4L11oXb}!v;Ot)=$SGV9ygj3Z!4t-$dX~N@KM!z=&UrF0>*NX?%Zt_G3eOVY*LrrhflC52ESwE@0 zgY2lyFlGrRAHkX{pHVaM`-Uc{eXrD(^ea>x(#sP~WogLm#rnKD#Nhk{>t$RwonnE} zMvR3y7Y8i0H$L^s?99!jh2ljL^4ODw zQZ{k!&cnkFNlqp&aQ|&CIrCoKFzM$`cX^d^sa*jUQqw9J7C$R2ojlO~=L7Snpw#ZGi0=Kj zgbF})M*|Y5x*}Hsg+Z|n-Z#00e&@Hhj0z@@Son(#)@Jh2pBJb#)Mg6sE#D}u+86zK1PiLS1mG{DR_UlE(qY@j1)xAPd$x6nq@V{a~>?&2xl-F(g8x$@3YLC=T zhcos0D7j7kGnt;mpeEOGswgWf<18E|vtXF)ry;v`;I0d-?$_#7 z*d;X$+etVsU5Od|IJ()^NrAlsKY1yo>j$@w{B$mVy#fyJKXKP-cy)73n&>6 z2k&>cTYvx}`nb38Ic_r8PtVAEkkR-P>Sw_<*R{!DT0WiPz$fE|o5u-0s0A8xDB@`W zJ|3_nIZg3G4`JRe8Mv}_jb6-&8mumiRa=(`@R#}JT>Wkz6@GoU6}jFr>kYmKvU18c(nacqLGeLa znP&(&+f}tl&xA8j9ebWDcm8`{9m)h=a*A9x#R)%5M%sCL!PA8WC^M*_v?xj=2|t() zr~f)sdo@d}Y5YK)Op5JmgLb1P=f7X28i0hPIwdYOy zyn}*5!ZjhP)xTb`9DlMWo$!^{ElRo}KV#MN+2&l-R3Ya2MU&!u@`%qVSG~UHPS~e!(k}Cq$?Y|1Fx*;z=Hr z#INEoCc1VbNx+ylGB>hw{&KcfVblY*o7U{E6w&Rqj`)m9lwy?FCF#eK8zOOnNs1D> zIaf1nm)?I|Pxsue;e8Obl@U%x- zC*33b4gUJd-92U#$>EbtGGvnE=BAS*Ae!LSw1dsz;>b=M#iKGU|5wR*5#<$t8alyXH$u~LJw zv^0%!U#Wb2c;2A{k!?^)9M;N8fN)#*wW)`DI;BGJu79bjv&O&vHk_*P)YbOR&Ty9+ zBP7^-CsjRmE`V(^Wdt^skk(mpa5+Pl$!+V@%pbpOb~N48HTFz|D-{9H48Ekw8v$8( zdW&^`d0-CLN#EZR9BYJ2+rOeuJEW`|zT>DKagvL^j1ucVzv1(EUZC3j$#@dSPVxkw zikS3(bS1XsO($+gCJr-dEYIus>yJZ-7Ta#?n&M`5og_WS!}m^p@@=v>V%A|%Q|CC9 zJE$3(;KP?x1f6C4cvs7_3IjzJa`Gono|sRyWQ4S@l6N<{A;Ph0NkPQ|QTCfok$le1 z;foX9P4hov9{unZEQs18rLU2o@`KaT`XPAHkhuTY!?%0Q!g!uT5-uYF6V3$blxN)Gw}~B{=LkP1VMpZ zBI7Y$A~$sV>UToA;bpiEn-3DkQPV{$-A{M>b)&_pDk|AD=xBp1DCuW_zS=0WDz~^Y&#{K=V+DO}l1QAJ-ZxAhi7k@0tcWMpv)IsHNigbc<=!Wp=ZD z)+21D=A|m_dM_Z*e@@>#&Zh!MWt=$ts*vd|x=$LmQ9WHrdGLhe8)sF{{C;O&;N>X- zpdwGc9Z?V3NkGz1AO7^02{pUL13qH3ceK)OjB}532mRR>YtGejMWf>EtI?q;q#cTe z@d`X+m*%+!?^49%*UHVLYN%@4dkVoia^Vn;^|06aqlwQVqzy^prG*@E>=O3Iap=1-rTFm;=ah6?64Q1egv z>#ct8<%59lRuh0`i08)_p~#eek5^OO$%1rLgL1Bz3~%DJ-ha}M^9LigaN8Zo#Npv^ zTeh2#3z2Y8)vDpmvwmehzEQbVdHPeVkyYU2NHOC@MJpV(GpNa=%Z%Pe$2xm8CN%j( z`pl#9--w{C?tLiNhWe3uE&mk~#hrN=);!lL>-!#GFOshiIu%yb`mEL@zk$p4|#EMLWkFMn6@`;1_*5qr;~+E#M} zcLqPKEud+_DqQ^}n{~uj(7qzH8lz4YAv@ucP>q{iFrICmZj?o7>sY!eg683Y?tWUL zK}|Kt3hQkol!Qh4RVMh_61l7%sIp!?*w?a!CT^M*ybu=JrYy5Vff_0edOCs5$F%R3 z7E#XXX^AZAP_spR=SeI?miTYtt)nMlyZcH%PELq%x>DOS_s^1Bm+Kc9HJH4yxG~^R zp$J=189(v!&9TOr7UJ_PSEQEBP`?UGJ-apGEQV^pVVkW>-6t*YQFlM3h{(s+nX&xA z@^~9I5*ix8-H7y~xpIYD3^a*$`GiVvQ^55?)y&1xz3_n{3yO^zlyEpf*iF8#UP&@+ z7aezkX|TB?Ahxy-HPtAc?AIhODzJOH9zAWMU}tAXCK)bn%EpS2}!G6n7l!RZXklC<9FkFM61 zV9z^CG4P@McTu0Qc}v;ZY>~ zk&w+5zuVp85P(zX5$uj|UZGY=TSwon>06S8pGnp5xP9hF5}?yOyTAOky94!@Kbhke z#o*weIq1JFYx|LqI?*ETPU{n!Te3rD@9^b(#e-LpijtXBpduL2QKm9u<=#65$^XqXYFHUj z)r6)0{=6V2!Ipc6vH3^g6Sn=KLPfr^nscM>{Ziu;1j{7ZPcde9wN3$|K?l{Os;X+Z zK{2aS6Xu~%qpi1)xHu22Bw69ss_zeG8o zW^^c-uz`30ce6 zUxw_Ee5OZ{SDBcwu@8QMOCAX*C<>jzr|VFbHfY6fQ2<>a9WbVFF~cu1`)AXGJiGhf zjso8gJe^!(A7!Phjoz>cA&~nqwlM1Kztw&-cwP`0&sR#5_-k?Dx?M$W_0AZ5`>tT| zNYTU(dO%>f?q;Ap(bD_tkIY4{(t1x|?s@lW6S0CjThD8!tHo9%0tWI%44VcUGX!im z<2M}2F3TOYrBm+~vs?9ZLcDLtU;TIg5U_R&$dQW6)$=Fm*KcRx)fRA7Xu$J!Zg;~5>lN% zW>|o?%-4uzbLl4Y~1&5Z@ew&*NbmG|_Qua4(vq6o0Gfv@y zU9LPYE!1?`+V_+c^AeMrd94V2N<}dV;%YH=6i^LZ8^Y+ zTOcgjFXy4CY1@`7!jiy6!O_Mf3hL&wIYZNdf-FzWs)05^s9VTg?CL zcF4;1gnAbG2DbA_Vb;q(YZdgCEYUmF$2BA|-IGW($Bx_@&@S0Eu^S>eoB&@P0ZbfW zcZDW5?UlQS_l%gBZLl+#iS4kp+NH5)9gIioHZrXvCyJ0)V)8#JmBOtL78%xsS_)tK z0U|!$(bZn;jiu1Hn{$!4#>{ME>2tMiNL4t&;`b6)`~v{rTvAN1wMDqhp8ZpEb`;?F z2EB0O9GR!;7`AMDM?9109MO>7_j^6+!T_*rjz{8-v|6cBuY<@~s@GzJ+2S6!fY+Dx z5~pa?wfx>TI5UH9Y!cH>q0G=MgV_ulmc7TCJPW;{g$f!zy^2-O54~Ij^xXM1=vZ$M z@G!STB!E7X_sn=|p_=8pvbmFxIGUy%Dp? z0G*F?&wu(`=x_gND)`PCSE?%b&BVq9lI`XN>xpv*Uo9L4YSuWid_4Zm7p_Bp<3qd8 za>OxoC=}8FGH(nkCG`z z{2Bz9MQVYXJ-4qv`(2OoY4A5BKJG77dGb)kcRp^t8tr}9cf)YEgL~}lA&n}brPuW1 zcE;FBaV(X=TR%{yY4aI!7o3}Z(*1f>YOJ@UI}PIi9Q`g-?;>cKN0k(Lxv%cr@Lhcx z5iJ){%3RoPx%|CQY$mwm>zpgE{=q}i$A&hTHtqB$jee~5rj3vTCUx+WsTy_InrsUU z*KZ7f0ePO((4I4!s`rIKjEzUV&SijS?vup@$n2)}*E?H8Ddn6X>6##|;xq$*;YFbT zSh~27zYq*udG!CI!2Cwl+ky0RVW46JCf>ht_YJJ?sD!MOH}<6sNq8z7zCjsqJeU;( zhYzmReF^Aamj0J)0xmI}OTg|w;e^KPJfvVMxhQ)wO9u7;*)kX;;88`)>H{sMyKv9I z;++}Vfczy&Y!BA>5I5rKfN(h+rU@-yj zgR;_a$hBJgS4(Iw1^Os2a-EY zbf@We>~(f0w}O%r9!o@Q1>-Oy8wY6f57_6xO~S8aSLx~f$1A1J+=E(2<{f^R7K3|_ z5WWS>`6y`{=EkPtzoIRD9CUHhXV ztm`PMHUsR|=3OMcI(1YtP8BI)|BtEn4#)C;|HrLlWR#H+8bYFwB#KBx5|UjBAsLlf zSrL*IvI$8@Hc7IwLr4grLiP&T{2tfq{rMip@1NJ}=t#HwzV7QfpXd2lr&#N086AP0 z7ft_*{FC}L>|5sIl(0$^ZWs;FHr1;&Y9%tc3VYe?rN*22TBJYG^b;}2#i{jO+bN%8Bi2f{uyj(I!7udN&bjXUfFFsqb_uX;Fm-#!wJG6t$-48bL?kz$yMdkF|FvTGu`XiVOov!`wM1cOYf=b zE{_((8yz}R*pEf4Z6gJj1$EMobB-433l>V@Q!FZyV`E}z+y0pIb|cm&=G*OBrYSwP z+|dku*=V?QaS6^H(VhYPOWm)I>Bj)ng_sVvNa@FiU=&LHTUp~Up4Ww;so*_|7K!ihT;|mYFQJ*PEp?*!h2Ses-JK=&JYcqM#J=(PL zR6U`OucZ#h*|~qZ8M!_qy*df1@krT%82zbb#??|+Tq*r#Rcf9+Q#Wwn7#K*sU zuEVj*9qIKT`H_C+{_6tAMxBz|EQpPUrAI6mDIcBpTsc6F9_MohN4H@J8kUd0+%UJT z_shh+`(iE|-EcYM#4}#M+$6V8bG&&%VNdv%f9EmjF0LZ6yb!?|Pp)KWC$(98=*7Vm z${W_e!Zy3uW_f}6Da$%H|FI-C?fjj|Z61yNE2>$sG{*T+v+ws{9`;$cF>Vlg5Fepk zaxTBSws+)Jo8`^`UXgK!sAkI5JrcOR1I=i$U#8rtJ@RgH+Y=b^`q!q-%TpefCU^aH zTrVZ5uqnbRfj-t?X#KR*TiskwqK*)z)y$&)n07VkKraRvw*2modL{fXoIQ!SVZcZs zkUqx1NfmAVuKN#9T;YErmIaSvEvLee37zk<@;A$Q63a2pWOM*Jo&eUg6=D)}(g%8| ze;3I}NZPC}m#rx4hiNQ4vmRIBT@7A3*O0T({~(=` zsOp;+zZU&!rAO%X)~bL?#-mRi3`zHTL&EMi@lOVSeWi3n>(IuHlW$0O3&!rOdL+Gn zLa z?Vd=?&d27}oyydL$1BsfQqCoL0CD=ea>+u1_HQFuncdy^%fX#_&RX4lnf+eEzobkHwFA(}$|7q+8w);|9WLZofwamH?|ii4|lH zE?a8@#BOQVV6!)MoIf0Ut>f&!Diiq*d+ycmBOzw>_2oKQGn^A~*qTQaw3%*a`6Na{ zefl1{j}>fMZzh)Ou?p*>e}{b_3e1ByK}r$8y>qE8{qmx-ZB$g$PiP4%hVR`S3_S4{ z&-}gd^i(44pV%a`?zXjdT0uMeTJvJp^%lbU0blYhagL9hs}n?W-LGH2ie%-2g+C{3 zt(ynQZaTZ+Q2864?Q%08K5;(`?~z1I+MX@%qjx?j^XHp(#ou3;OYJrfKKSEam#gH| zKefF)=U;z)t1m4l6f!ipT_~kx*!i-1u0kkqthyu~gQse#!`Urm)ix#56{~)SY88E= zdL%0mi&{yfo=WsiRj?1NvSvU$zmm#hZf54wc%&pC*mcrmrs2nW#tsICd8{9co+=vH zwC)ugcz2E%gx9y{;irI#Qd5u-%WfdNnx10l@vP7LsQ3C6R{Vv&67E+Lj7`-{&o#1o zrc#JCi@YyB8@En~j5?eTeLOG-WPI=AM19+5M-Zjo_0$%W=xB%iUy_;WV&YVW>Gj@+376 ztuvNR<*n?`bmSxnSE=#nx|pP;nBBE2D7Ed@apa22=knU5mX=fqD2uHavuD&Ol=Rh_ z9cOw{4Rbnb5~_s0HFcQWCBazp`hmNAv+pOVNEa3Tp`P+MD|?sC%%3hik4bBLjzP2Y z>K;{9U(^o^aZ8*zUT!BYS5nD8v62hHbHtWnuSslkC!!)e|j03BUFBfh}7`Bk@t?5gMF?pK_V` z!XA+(#LA?`pWh$YZgz0bg(U5CRP`MU&VJSHcn5+VQY)J5MeRPlx0I69pEa;RSpvz@ z9tm2(l+fILJbv~;mYG2friq;v_Dy4t=6<<*|FN!n(BI$cmI7RVnui zFk_CMExwAk_UhjKuidAl;wj_RyxJ1^cU^LtCbv$XU?#J_7OQeGqx+R%`d5p0{SBFu z);s0xXHs!~d&@r(YBU)7!Q+VVO{69AuT2&4DDAmRfqNQy!6(G_YMc)rIM=haGYF7x z62qIzV#f8Ga83@wXfTY}uIX)C|05x`B6Y6a_dcBcF%)=Sq`g%0aOCwbP;&gB-CP=- z<9UkoF61(9E(sJ^3h{G$*mP|xID3nx-0hj8WZ)cA$$k!DEj3!rGWjtE_pt(&IWss#j~gEUW|e zcNu6+yV0lAc+AHQo?$h0cVYTmYY;r|^|+>#*8FNgT!WuZHz+y63CjLAV>$c4zu}mH zzKR5%`*cO_25xJONA13jF#LWYZEL(WMJ~X6r}^r|pdR=6pRyGnZbjes`rPcf#UR_I znCuh3JL5a!vspf^tj+oeCZ)~d~A=Gl>PKna#Y(rvklM^k`#8Ord+@s5mcoFGI{vFrav6s&SlStP$)>7 z!b|`4`1|=$cV=DfrQIyQL`;tVO=4I+w&!m3PU8La6`u01`E+jg)oNtzL`T!zX=+oN zE6h%){MaVP{qV1zj9Y~!F+3nYWpj1%LX*!+wtxr!dR)7Tc5Et{P;=j3za_I8!li6} z4=Elh^!@I(G~T=1U4K){JZwr``Lw&N;^)u3U7MSfZuh2(f6T58++Ade%)@}51Kw;a z>ZhzngDE6j<*sKR(323@DarhltvhE)-xr8I9ejL!iD3;NYvSS8rJ*5SG(!)&I-87h zMWm(u8;2jn9vFDBIMJ2gp=~htyD|Is-0IxJH4F{k#_512EGjY~)21!4%2Wi^QdDdd zb#31$MLeAp7wUJQ{yFjJ%3DRoVxyw5JI?T*=Z<5dLnT2<&9NlOc_m8dFj_6Nz6+cE ztr3Gv-C5AT#q}a};UOCX{xnkl~pVu^G^=eQuu|2eDNQ8ai8^^_%OxCFg zg$!X)sUeel5}*DmaSt6%bp^;JcPhka!}Qr)h}+YY&xRuTfpxv{3jsu)}5z%z~jPkELERUTUP42hQM3!(@Y?LbH z>ZHUDO0HIFE4A&(IBcpz^=f_A1;U?x*^_ZaJk}cHk$;_UG>zij@4^KJ*7#US7M|VF z>@?lH(Jp=mFUWlQ*K2i=_XTjxpN>7iLm*$0&t+y!exzFFp^{?Uwyf^dgz3RAnh~)r z-tuw@Rf>W-M)fmT@1${CT&i;LBv}ZBRHlz{@z)04-DE}WYRP!%2B){b8?TUHJh@gB zW-8@d)Y4!j*|}5s{T_fbVs^vZzIKxZnlYtldaI?r*1iq!4wnH~Q(A`#Orfr7()qvG zOuXFi(g_V=SL=3oUF%|T!O`)ikkVC+6EAKg4&0~;NSaEx zAm#)jY7_6JBfWDREdr%0S8uNUyEA3Fu>WDbf;2scmkegLUw5JkZ+C`%c{R5}NpQh! zoC_A>$rah|BbR;PxCVOj_0^64Wzp%el+F@Y2Zs3Rsn za;><$*nQ2#FL?69;it`Ap__-E*6hc+YsV5`!_~j;h=__-R#a5<^86UQcs{fm2*rCm zcFTAg8%IY409JFLI{l9I6irfpf>6t7|05rs%+1~X0$L|Be%|8mo|Q&bigNxVRyR6Nll8eT5ZR-7Rhr z$Q-8iO0p|G7^cka{FAf{SQC+XN5tIOhPXYn(j(wHHOXHTxYr{M?@{9xCi0E&rz7uw zlHr(J&7vaGgz)`3Omq@9RzG5I&HFwZ2-NtAse@j2{SNvJ;-tcTti*la)cU?^~d~MtIXxa<2oP9ELnVAe&6}BITy_6)+ekAJG6;0ZI8n@4cp##{3L|5VtWMa$gm(ZO->3C zZyjlI7qFZ8^w>jVVUC+h8Swy4$Q$z_Kt#>s=VTO;Jn~T}|2v1ED>f^2yr! z8a)fc@|XLvvJRig?5wa9MLRj$bZD8+yy{!5Fp~b=ikRZnE~IOws0)P&KhM=uMT>sl zMA@jyo+?ZuxZ7g%%efSN6TUrlX%AW|3WE zdxlk5yslkJ*Q^|IOzKE$7z-F6DQu@9YUx5DTjsnU1ro{JytA^##t^4y$UM z%f~rGZdPiEc&ZR(f<}>ODh9_xp(h|B5!;X`A4M*`?Wd z+^p>AixbG-KNxwm&7x5Elr39n)}W<`D3|fv`VAbh-7eJoD9*Yu;FR7+Vd+peR$(dO zpujt3l`5nluAHt}HR7mgE6%p?5G!Pz`;OpPP14jw9PYC4w%j#EU=qqK+CXbsQegsUUK*S?A-zMnph+tu8#%A zzte?~smfK<_Y`?epTiG#-|*$y9A4u(lcb)QogEBT1hIEEMfvDTeSCcSH-B$dJI~F*g7m+=TGD6c=om5oMH=$NV&9FWfdt|)Zfk2>GXMEMzC)JS z0F1&G*gN$Hp)SKZ<*O{RpctTeb|`vnOPnC{%cyp>2;s?zCfWP&pD-Ik5A_CjtaNlC z-{7|w&RkVxy&AdVa|G`|aEJwjMHk;c@$LnA@UE;_TGXY?F=8{OT4b zx)&S01#1RQJzg9NZlgC(D?jCDm}Yd+=8#}Emq(LRR+V#WP(j#zp}Hg6!b9l}Yv($T zOkt4}tOcrk?Ip5uGsnOt5ho#{DdP}Dfn)^hg^jxT@D{qSBjOMeG_X?a-dJUq_}EJA ziN&nohzRCEYU#%E>d0KQmji;1KOf%~EOS)SQWl4{q1)vU`6HjM^;Q#cwZ#Cc&_IH5 z5;8lcov{sFF(DFBlN#g22yzro(XOrH8abXTgLaFLtG8@UE&Z@_0mx-O)l;~~zEzF@ zUE&>ks0;O*^KLd0aC0}JjZPwl9tOBJ=_4t~C1@C#$LkQHudLF4`d`Pyr6+U9felKm zO_$gfS&Kyvts>+L4Ehaxg@5q*npq7ljm}LIS#EcTh4<3ZjmrG-)%4Pn6*%>kq~iY+ zw5ttHT;Gzqmc6v4MPGC+`*L;{yaBjq+HZ9$k*ZXKt(MBkZ53!bPMvCk)>RoF=~Y&1 zVxP@C>hEmIQCB}mAiW5|X5WKuG!4AU5OW_HB<=N zrqjKd(@wM4q4INS?Ih@??%VMH#Zl+Ba9@y+P9S-0PNv#Sp_}i>byz3M$ZSyRn%mex z+_T?rk8K*djUBBw%O_%=h`nJzpv$&4X3A4RS{Z+|KbEHvpz6M8RWh!Qc{EV(o{;^eBR!L@dEiCU&jJIKT}U_q z4hS&YPkzfZvkc?|jJ+5}+LKrPDR&<*>OaBsyDr@X4uK;fN>-wN$SO+v!l$&Z-}CQV zTUOq0`$SFQXsy3Zm~ZQxtQ+#90L=8sTCXx$<<5A0$G2foEp;DwP{0R?y&N9f5c+9xXl%M=Hbo7|S zP;b+xQ`d!<0(cQKFT}<*dF}A+6O@edW@!TeG|UH8qiW`T^p1%(&U!=|7keu9QAd^S zwAl9SX5~0tJJQ~Ns6A9V(>Aj&_NYeIHvoD7+ydxVKiGY66ss8Z2$nAnP)T{I^`_OL z;Mq>0JzNS&TDvZoVcJXU5Tr?=(asOiFMU*Hxld>EIjzM7F&Mk7^6xO~QSq4{0J>tD zw@p>?bYh|wKoki*@&4SZm>mjtbcH>jfOm{1;|OHE&2Qdq*-w>g?~_}3G5fB>p5!*G z8=|qTZ#teIzGAwsb;5z!|K^`Sdd5p;(l;5&G!%VZfF>cR5Bb+A_gG+lP135IxeFoe zt)hdgaT4%;?ilK zVMofnZFp3dpCw>s%8aP~y;6np@S1$*!h@^)YvqPh`h(I|tlF6giYeN87FKNGD(Fl0 zhJ3k^A4OFcW>sXtP12r(WE#$Bmz@c{4fX=Ho%2c<%&16#9A;|Q-S}iB#u&hR=cEJwWo zN?gYdd!ysAz~|2A1vFd2e@KW&`m~oy%(h@@jiv}NX((D)+$$i~v!WFC$sfqgdSi{l-o z9t+Zd^RVfV*l0)O$RX94I6tuOc;lkAK_=_bkF#Xg!PHVydVz42{aJ^ebxmZP(9w*} z$^FEF#6QGBB^*1Jg-=^XOyDeX89KmozudQz5Mwc4RZ`W@toMmpiEY4Fr`zNL%KLwO2jWKn!58QY z*VfibjwX5$XG?TU%qL{*{PN!Yx46YMtK=G=@-BHQK#CbU*?)-1B0crRZvcenlG_s7 zES9VJWh#lRIYKf!mEC4ROpB3a3kQ+5b&y{2Ckc`A|DttE_L(YQ6_(i$=~b%v-LJZ> z{ub@43klg-_OCGeE_7YL=bYgiIOj9~EYcT{80c++PDSw=U4m~jzt;G|=1K8&= z;p{-O{DyDen78w|h!pAv_S%-m!$C1~`hMT(&HIlTiHc1O9&Zhc^I~TYJg%5oA`+#6 zIJskfx>cx8@Dk~lTxr4@6a<*}v+dZmgOb`?zM$87;lXKq@dp5p3MFhVq^``9Gu-Xs zDw=OS%kZ*&>^p!vB4BajQE_>cDj%GXM$=GU{Qv5IQQ4yKEz`U=kSC4r5*c#x2CsY5(MkI zk}{9=Q;Y{y*}#OTpDEM?z9S|!YA(elm#)75K-7sJvgW1S#uMHq0v>mSyfv~97zibG zS+XybD8I@FHw=hcJD(a`6ib9PY{hZG64gb1PMuAk)UDZ8b@y{2CHh608iIGlumH#` zUfT=4{7#Ejn+roU2LWePxiQd}u#hCTo#$muHB5JS+w35PnXf%bve}F_T;E&hg4JpF z=idxV?K8E#x-Ak7(?#9w6FGm-DHx|+vHY7N)+@x;x4_8TNMJFJJ!u`W_4ky{Fb42p zUV$dyZ_VG>$wl9_(~pf0fgi{9xpT_9*;xLGTB1&pN1L2%rekd2v3;vp!YS;hT~hP? znR}L-E19=u<+Fjadv~R_S&5C>VN;6q+5u{GIfeT1=jMGz^4jvDHo_#3|4R~cz_4W| zlh%>v$R2C*F~+en9w9armNR{$5eMySJpqmt%iD?BEZ7EV!zM*Sx;^aP) zdL`dYuzvJef@)TMeFfiL)nfevyDs3wuqe=HThw2W;wp9t0jg6P0dm66o*p&%Lpq6< zTT9&ND;3NcGj|=%@mUtqfDS6wH<|l*0~Z;f&fE>zydTj_z8wK5&3MU(Ci_*33qow1 zlQh?*hWEY1pt%3I&fkjRoW!m%sYlD{B>p48j8t{(Jk#dJQcr}8tm+4fgTyTZ9&iE2 zbj$Xa@ul;4$15tzRcnQLZApYBAo0p=5q1YK9YFlF-Z+RA@5h55+^CjhSUKak>j^I? zZFDL)C?uy?zjn*(=z`q1B_8ui>^}ZMpC%8Q1-K^`^o5W3viLT8GJ2+;VTSrEOP7q4 zww4k3J|=g8ThijB|mVwW-W8cVRlK!iSH9QD)S?ar;xW2h>m$@i#JQmzJ-M*BIEOAXrgW)) zjodn<2h|+u3|EO36Y)M3SgDlTJLOz@jqs*?K}EU79*yCl>ONG=Cs6~XTE^V{g~Ft_ z=$ZA8|C`||?LN{LTE8($X*Fij6xU4p@Y(dWNf zg^1*tr1sl4g&w>B#|kwO zfzSRiQ7*x`TpfK?ydg%Jk^pupwSxHU6AxW zuDbE*mq?O&+Sfs!O1d-0sQ@pCFYI-7F|xJM_twBhorJ(7Q6z97M@O4=55r(2^Dsz?I+ciT8L=sLdbI^wp|R zgk$0Tr1p*nT)&)m6{-Z*n4<4YJX87I&}ZEqLW+aA7A{X54G4>>mf-OhGSZ-t+xgg% zJ1s)<*;7M2^flHE0-eu(9&^d|Z* zNRF~wluO%;D+YYYwlN)(Q>7$7yAhwq5dUE2;71$&277wFif02C#o|rCnW_q?ly_Y$ z?N1bLM5X@Z$K2b*b2jv#e48nMd%r1K;Z_TAcFw3c#0AR`L-oM+(@Wg*t=Q51aV<9xo! z+&~31Jl&Q|mu{waj5qOc^2VHp7O3PVb)NUG=Hd@ailte08;xR+n9&T=tjPPBVj;Xvq|wKwHbw!_~7CCesyC;e9Si? zF@s<%xlKM(13U#p${#%0N1yLgKNNoT#uK%SeIZ}=D~kBT+~DjR=Q5m?Kj?KIRK}fr1|EILF2xYX?Lnq!cCh{9f68&9vyC0bkXO?KYB7 z-d}elouOpKvW898lUosHP=O$2Ed-eA-_tYF@oMw=2^#^iHXo>!L;qWKhh1ELy;E-O zUUonckUu0tBsUQ&J`l8ZWPYT@y)Gi?Kh!6=&Oig1VK*bMF8?bkZzlRmqF?+k{1N=L zTO#WKn8BRC)l8RiVAYw^Zfrn@eW#)Hi6U*?mW$)OYanaNz)_|0u1*DfRZ=jZm+?3AujnkM(?nwZLJanU_*O3}aNHM}{&c{4`1GRwfJ^EofO zmI@=;wQGlyx(*3kp?El%(-8ejNk@G7SA$dl;`BPXdBmY+N9XE%>I(p{&!~kL_`=3M zC<}ytwkbyY{&hnyT?+yLqAU}XEJ4p3m@c7VF^#V_Wgg}I^S4X8(#@;z}uii+)o`>_0>`_JUyZ>k7HODb9q$In#6I}9f z0C@+)RZ1sZ%h0Y9{sn|okU$bjxPXz|eG=xF`v`nqZoIxJ5A8g44Hkqz)s9prnIb00 z-o)aQ^|KEs|26De;I%A#6V`S;3tc4O2SdlhDk+}!S@j-83?48l0BTRx#vNb6`V*4= zGm?*lC^ootbWV{l`lUAPz5s$2XaxpFt#=6Kecf`4KTJ5WGi|OUUSSs@!sEzfO|sYp zZWkgvi^=DtFdHyE;YGB4`Pcc? zwl6$f0}j{p*u?5bPEfM!eKwNpkE?Yr= z)=+Xvo1+9m>l=g1}^fhDeSH;RU@gBR6MYLwgq(zNtd^(W! z?+oCZE8+6+)KA8Cb9i@?2ildA2+Ia|;&TvIY6)71_ab=A#b$T6JTPY!-WSFdBYcd? zttTRz3FW_hl2O|-W4F8?S}3&)>_I*DV`g7;1FoI`bE8$E(Q~Y0tTDyXlCL%w6qs&3 z_Ruf=^0AFn-0pf>RjxRs?ZJ&cD(NqEv%Xpug?OJyS5MT)08&wKkoaYp{YGMpmACRm ze%gTuFgX@`{-#|3!;+Krgf;6iOdwWe_4OXX#>J3LnOID(PAy60MqSjhTvR*cA3_Rr zyI;ni8@pI+KURd-j)>e~bzOb^gizHAVut@HVlHOSoT z$eGaq?G<0#dC{TyXJ1%=KP^05g>V&OBWSA*koO0@KbQ2XrEO;!)jnwVZ;2bQElB-P z<5cu|VU_`cMIn@ThZNV@js7p@nQaR7V&r8qxCh%5I7pJ^f}d?ee|`VJM&HrwsdoyJ zFJ2m^|FVy2RgHrKWE`B;}4g?5Z|MHyw=CGaZE#T*nhj0=q5XtehOmSKJ&R+OP)%8jog?p{?n+<445}8JiZm#Yd(qx&>z0lS) zZ{2gK<>o9c`(R~Bt3J4-DRsydxEl6)jmA3JBanvSf8d_Alxz<@Mc%=K;jB6P-rDZ7&Olu|l#nQh|0wfAYO~~l1UBhU^<4}Jk-Y>D#D6hK_JsQOpV?TT*&V*7G%SMRb#R7yWfRy zVt>lzb-R?hqrytE_D=b3>}=N0#FC;4tJcA#_fPoRF*h1dRqvp!^?~MDSF8#@6DHSk z)ZKw)PB(z--2gW!MsH~187=DnTma{U_I!VMjq&(5bN2%iSF_t+b~#Ko8vE_IEzy)? zWW^Y$7+7T{m#o2b!8EOtGk|VY;^6=Yyh80dQ}I@dcj!EzJAj_>h;O)k7Fv+X_7Ni~ zwwvz<{N=LGW@^H5Y+OIkus>mk0!izHl*C{E+U+>{`>k$^$K9AmYH(mAt-U^@7O#vi$B+UM4C#~RjWQB&5kw9eTq_F-9(FJf&)lA45nxFKxfu6wWK4El8**4<38JM*>%hl*{NCrQrpj0+6MsT`|9d@48^S#XFoVP_$y{E|_L?zPiGSKw}(Z z+2M~f2xSy2NRgns8Ah7GnWN7@gmb zzVvtg*KK|uv*we!K|(l31df;fbPwj{+1vCdsA37PYYxGqd%K&rn%CmO*+YG^k~O!v zEA5w+T4#!{n5Sw!n)?}_CR_62cQ#M;gPwg0R~-dYWQF*hI)x>_oT4mjksIq#5_NaG z&MZau`andh^UbG7R=5FC3G;Z6fd1*q1*^G-O7g97&lUB$k3d>1tZC%fox>I!V28B0S+4?PE)K7>WBmJ9mQ0w$-V6SGp?qsBZ0rya0%u zqQ78lA%R*A)*hSV&=CtnNTh2I)Q@xjEdd_(eKGRzaUnBuurqp8m~t3{05X;wWS|)q zNT# zjHIn;$H9W2v%p6HrQc!oX*efIOMpsYQ1_zZ+dpa~QqWy%3gT;vQdMNOn;&*KDG-qn zK)(!0fo`*+W;dBsypMeDr~1T5tFrTaI+I`W89mR4fBBx6V4sCeo+g#`BoLBWX0{#T z6t$($qe7Ts%~Sydi;!}wHc@B@N(WJo89vf1hLjh3L|ZK{I2$^eLYBykiJlhA!sR1w z#yK8tQ;4~jq9`6xX(Kv#?>n^U1--cHL1q~>9z`ZK)Azo&-a1~ri~YZ)&p!6=M0B(( z18j~sZz?LfmDkm%N+LFZ05`M*)zhz_XS!E|Bi!PfrHWmO5M+cZ-k#24Ro|Z_oUl=% zN$j>s?y!8u8*NqgM2G^Qcaqx)gR}(zO`|56$%N#$+DlESC;3KXo@CtssUi>v;+xOh5 zC|_Sn1;tFvJJ=i<0h?VAAtfS=G?CXH_<@;TyjZzqBvMa=^v=$T6!kwiY##5xN=U#6 ze+4AD|M5z>=kAXt`4U?X-eZk{Xlyeohf_P!<2DoU{m~M+w^qTOfFkhrL@M!aXDs6;dj|CWE(I18?N5jP7_8y z7*Y~a{*(Td0yjzDk@?ms@!;R#40|JQ_ss)Se0zOUawTCPaF}*2dE80k!!JLmV}Y{Qdg7z3lyPZV`u+`YD{i4 zQ@X^*w&San#6XA4z(vwC0?funccMV5_TDH zt7%sz@azb7a2&8BZBr95eAH&N_qyN#T0|ksrE9^P(7^h1`A3#o;G%X(7?;ebN&hTK zh{Droqa9GR%J$?swd|IrrNvqyuWP|VjRh%c6{Q_EVV{S_mOg-e7Vi}{tGWw`Jya~r zD%SI)(2_wk_67NjWU^WyW@3rKI?#(K6gj~ zm+C%GmQYvYnOmubRoHRhT%=-C2YTxgTh5lYA=WOmITFsi^PvF)?bHk71ss$c+`Gnv zEWIvLfP z-NirMK^Z*iHciyb#Qrr#GMp9Fm)XBrvY_$xCIRp-5b>(tBD8sTZj>sxCR|{UErp6+ zM&feGZKkkiV20F8d79mV{_J(PWysNRRP5Al%jEW3(bRQV@(xk*8Kp!_!MEP^u-Si@ zUi@M4PC)0)cWX1)Bp@(@0Z=pPl~xuwo?}1lE+5f>#}e>>;&vBRUt@T^W11!0iu|=p z0vi%j^)r*((%LM)aq)ZK3%7q)u{=uxs7`O1qV4P)Gqwgp9HHB!h(-r_g;zsrJhB;5 zt2%d*-~5Xs2vxj_{F-M71#5nYW{QtS_Sut8!R?mEx#WU8t~#wj$+voywzYg8$+JJx zq+3S-X~X=*Glxx2NwF;3Wf!_oBL-la8xg_6sHwtcTp)PQ3h4u{n_N64Z8q zxzq$#jQ2)VaIf!g7KXhkG$=~;9FqfUN1y7P@)(}Kpi!L3d}NL3UhaoOz1NdSygf6p z5dLa>?0*H6h+1_D(@a-L*LqM#fBXN;#9ETrmMb+b00Vi1Q(=NrJ_S*Mtol!ZbaFNf zQd7JqqEgz1#iu4>!Qe@0LHBLneOkniLSoGt86?6%tXxMIRlGqdxS@-Cay z_%pm}QU(>L%|oUgd+thGqh>r7GCA%b#^ba7W7-u#`8HlkFM9v(Cn_nXR`p5-p(u}_ zCqKH<+xMvb%nZAcaC7eC&X<17l=1ZWZ63T0cxJ(``q@V)=J!!MpD;i}kKx3uN>qsN z06x=Pk{fc|5(ibD9exg-jieQ;JqMlqbe!#?&n^7M0EQJjO_(`^_V554-xa{eK3iPk zFmSkogDS0)o041->n~*BD&{n= z>RYmC9eJ&ljt$L7WPmKn%FbajTxA=OKBSE_+L&x;F$=zgo z!y7JVpU@MLz(poXm+A=8HPgNc`mA$Py4rMF?=X%!OuDy0?5iP3-^g8!>=TC~1kcOY znzmlHl_)U|z#tZ{)(Ae>vb624hRAp{T@6G`fY{*!iKeTguG1`tv8Hs5Z<>+uEgZZ9 zuASwH80dCHdtAwYWC-%4!dkHjRVEYK zFvMCsAXi8F({?6RDn3XtIk)D^R<>rQ_DRTmPyUHAfsFk0vzyVd#mp13a3YrgB0xXl z&t5$1i4%(RY9Q#|UMr1ZQj>gi)v?) zgSh0c0|S=;M&pm>fS!849xeHw(^cRg-wO^HqxagAy*XWx(d^FiGyK|Zx|&g;_U z)v9|jx6fJ>aa6Cgw_INCNVt_ndt<9JM2qM8x!e26$h2IwC%0V1!%Gs9i}?!lw%*yg zhW;)LzOnLYGfnB#keegFVB38khzM}-z*gIrsnP!*KTL>%5){NaYOpBGxQ6CkpJT6+ zz3TsE`*K}h>*0`I0mWJi_*))QHQNBroTmaS}dd=J7q}!Uc`JE3v28?+u(2Bzd&chXm1*6VN0gennLp*|1lN=;OAkXlbQ z(<^S}ztlS9SAALQHkUaSG%*8%@|oK3!_*Hwy{b`JZ&de2Amod2ehl>0kT!tjg>N-i zPBz`@#eeGOaVpNYtbll%%{Wsul@qE|BqJ19$qMzouVw7h5kLZ|*iQ#INl8(yG=y|8 z1+gvYo9d}2po1MN7!~K}vG!BR^kMutW!DOnnk1G75y40E_)QiG~n(oYWDj5DIL(1?-iguO#Xe zgu!ZHLOy-cWy2rc_^Khm9}8t`s15UkfS;o}VfIeO&>1O+2@%N`0St?SIr`|cFtK%B z-1dyOcuk*BI&$*xN1V46ZEGJek%&##60{wdJx#=;5t}{{A+mD^B~g3`JwR1e6?z~- zbFp>#XE}Bw0^1Rjkf#7bh%*k ziYfo+CprN`F8nSYL^KXz%-71eLa3kNANpPNmX3I)8Hj|e2kjExa{hBU!HInj(!!>; zPV6U{>a5Z&&yf{i5x*Aya&=T?Io5nzU36V*59902uHypYmWI3I6&72LYLtyBzvbsT zbK4|f{BM`hmyV{@l;CzlCg9djn`@0$ew01moZXET9aM|YVm}>cAcY$4!$4hE*42`f z)c;pg+6J$VKzcj`<+F*U<+yOy_JOA_NC;CDj}+D!L;LWJh;UrL^w)-(1Q0wS{fo7> zy|9ZTkV82%U26itiFg`hio)a)VpURnz>kMFhc)aL9ba5nTSYo!Zf_H&34Gb)+`A#m zv?{~qg8hsae>+;gQIZuKP1c>J$e@$5=9fG|fmZdkm*_ivkP2C5E0J{9MRka@MWZXBEdDj&uB$mzsz4M5G8t>5h={{`lC# zm!6rx5278mlcLFo>G9wpN>^_1L{%Hy%um%^@B4N%sfs`<0B1;2#~V@hMTA{8s+UJT zNIu3FTT>0?vkD<-`1Eq_1}=rl-cay5$HZD2{!s!}!l4ivy=UHMEjo(+Mp4w5UC#%- zKk#5+ei}2{0qmi$6TF&8@yUtJbfd?nQWwg*kU8DI6*y68ECvMwluYOInwFm-cnBU& zikY}=ZeZqI>Vx%D5G?B?yIuatNdp&QM1vJ_l2?c637H*GD+yQNYaC4sw%FfZ__}3? zhFD~?SaK*L1K%s^KCXazdZ(_4#N);rDe7!=um_}F>Iq5MDs4BJ8ps74@gVk=|_ zL#dfpgfaqUr7eON_eAnpx0mT~gWSuqlB=^6i#?j#@o^_Ie$h6oHY=ue#e$P0)F)RR z;Z%g??TByY#5&1dl5Iv}98`+ZdVj%InB~{3w~^u>#bmOgLj%UwxAy^}4brs$;{fu4 zP=Y2!MdvMfY@Fg_4?w}aH&Z)o5TQk__!~rH!R6XSL&zA6#YZ*L!@Mxish$?d*8!Dq>Vtz-N1a*fV&HZ16S@s1kL>dqfFe?ko3JKww zn8bKRz^*g%^+da1lZWIL4I$Qnf~!ct8ZxGvr(U~UHhf6`^4zqHw{q}?w9-EBuX}0p z%x$MXTS$#33t030jqKm zQg2NAr^ze`zH6tKdv3WLA3qW^7%a=}0po);ZL>TCZU1BP9^x1#8Z=_}nCiug1atK< zdGP=6n^6RgxggFCVbJBSgYKc_yGcIY2;a*7tJpdLTjTh`0SaszagLLyRa!+z1!fW$ znasj=#%rI=HN5%jcc5`&6=>pnsuoM`6f)VC)0r57w;ID*0T^bMO~+Sl}_k zE)Wzz=!WocfPg#0U*BU{2!sbH#bAzuSlfkg4zn;)7~)F1jX^WP(_~fT9M`QL@Qak8 z!Ttq=ndfmi!rz2>3qMZmupC6aTq?8Z7g;g(Tf8hs>0%>pc^og0P}gyn+2WmFidf%$ zSJw?@{}H-lKb=epqrQu<;JI!YO-z8^5$_fQ5k|d_u3x4X7ZqQ-3x@GWdfVZ z#Q5|eJLfb%^M$XLh2JeVsbJ`(px94+>d4v;7j0c}8En)cfu2*sKl%(WVLY}fHF|uU zDPP<`__e170)_~`M!TiDAj`Lr$v^Rqv|eru$0`L>yvg!40CStQ&Wv5-L5&L)I>z+W zq;-sBTg|<2mBKIo44N?v@td3uJ8WE}XYo#-`7=wc&l+4B{taoVT3KH$#OEHyUES%c zm}xduEd;L}2EKg>v197bNw4!%ZCvZLiL05ohbqCAz;xw&9+J3{G&9NV9LC2?YmUbX z-}4SStRST`8bN-iWD;#&r)m=k&8Z{Oc$Lf%r|LZbfsn?BHVlL(?u9>eOD3(ifPg>$ z%<}H~sbJCs-z!YV;FX6s4d#?s?_^1<`1`x_fG>N>KhjJ$s_0cp(M~nYG&#-*4kMts zSQK6lC|^q3ULALGcNsk0jcYlq5`IT$+nFqNbc47^8W~qqRh_C&g0uM5Ci=o{{Lq{5 zud)saYY#WeC_K8Y`ak{|U8^QUsDv#Tdwc?V`=Uz>kJKG?UuoQGzfUCG!G>FbR6@Ov z&Z5z(K7=+h^IgxlaBntZn_wdHMOr`VPP*39Iv)2u2`xnjuMykBh=LL%2|JZ;z8PJM z?A9p~!dNXB{&Zkt+UfrloIz;h^UW?UEv+Bjd}26uidIji{AjpzLd}D5xq_-&pNrNV z@=c}&HjXli#oTwxlYQ@YjGttRK3O-Y*LU69iIngds3vs#4>d$4anBn|n(n44|4`n0 z?eyp;^fqDZpAJ5Gpv0Eh=f=EGS-i_T$93ZT$8oj$m4ty9+pY-gXnb+C+%LS{;{ zIs#w;Au*bTPpiz5)CJLv1Kk`g)c=N7JX zv*U0zQY%Pw)W^~*w%sa(9TTSE5N8C0`hxw5Z%oz_Xub6 zfXEs8!U}#F;&2>Z*1aiVqABDcVZu>LM~j5#!=6rBETG9VOOTLKfF2L$n;sX z`g3gq$lEQ7q${mccQyIcFmCK8Qt&*NJq?sG@Rojbvem3BxmRi$3lBk~n z0Vgz6$&is{Xu(hQoUVNz-u@9KrN=O7;cCTCI-gUbO4JmT{=`m!gx1I^Fh|{!^RtTo zig6Nvw0}|!tOmSR_k&8R5a*{*J7Js(Z)l$xJDZ`AZ_&?qTjC2D4UQ6pu=x41Un(fn z0ui?{hJKn8_pkK@g$}|HK-cedwTIhUEee6?#CQFFM16TYR(trZCL&Zw5{;&0CYeIY zOvWV1JkOCxh9qPxWX?P%AyX0(kz^(zBuNOF62e{k{C@Y|Kh8NFpYXo>-TV1I&sytQ z>sq0k(!QN1(BGC{leko70N@zirI=}4$=5!cGuU<6c~2(qiz^ozQd}dJ_%_6S5%--C z)zD^mxLQs( z5rK!{Y7w+~B*4>Pz!6t1ll;3Uzx5w=6#Ms(wqJB@V^+y(_&a~SV@B+#|D_9)ug0MV`jxCa?qak) z#r{6oRZ#QQ1LKhyrzf{5`vo0Iu40q+D8-d8duQ4be7lSrHxdigU;o)ZP{JH|ru%r6 zx{v1piZY62C;6`qCH=ow#o8tJO8xtNM@Z}c6`n0Igpx`T*?-A*W=F+GR@=lS|EKN% zK;EVpiI~Qrk&4G8U#UJ7T$AUcX|k?>EZf&<`_yXUxV_b{E2AwiyaM zIRb#QR>xE0aR0RwK-V`kkKW3iw^hf9Jq&%zvmT zj`F&)S4B$z7dd0AFx+`np5vU5j%0<8q1kcB(_p`amRBXEz7ZckM`VL_$+wl43`;Bf zp?W^@d%v1F6p_EHk^t`nkcq|j{ijy*?@Y}Zwj$YHt>b4UBiDqK$@1>ivy*lP#OV3b!DNr*l6|N^YL>Ue(p}%r1DPJ5HxO8V?nS2}0P>738 zL_-hWq+Ux_Z6IHx%Z5qsz=LnM20*Pe*$6{b{iyk%M!q;J;}hXMY<}T+{TysjX|{Wh z2D3qn4M-$Z?%X@kJ1d@}hzcDUy`g<&N1)H(olg3K%*$n8_>9ETJf%6(V)xnntd8xF zA>T)%pm2xZ~XQJ(w z73+-X$`_Ub)nzLon}F@6OMA)kR$g}4JZ)h%bGiM5`f#Uvq>^micu`c_4`j(gR%sGM zcWy_$(5I?xe2n>ml`MA_Y?-TqTI&PqZN8u*;>zbCArBmS8@XvrCwsg?$TzEiyrfxS%jvGG*dPN%s0LRI zZo;28#pq;>6}R{{<#?4%GfhVs7|R&_&a264{O%EDpt#RKeZPrzrycw z>x;afu^!{}4RK}=ejb6_%ldYQ$<=3;v&r2vUKmr}@wp&I&z~|SC!8%MaC6kg`I12N zHrX`&yhqoc|MET&pIxUV|NN1qm5@-4owVzr0R6PPE!LJS400ND^vjfdd1bFu?lq04 zHcuEUj_kvf7?X--$$EKR!SY3B*P$^??CSc))FH&M*U z%gOzPX{_S4Ylqb&=6Cu4evT7&owZrlra$R=kXd@IxPrs8E5t#Gw#{%e z^z-Gd>>{JxgK1K1-iyy{-fl~U^6tqivelK?N8Ss}9X3V(?3BOtNi}CGM%ipQ6u+Yv zI(yayrp(@CR}>Nbx$oOxzBVVZvGfb_@5e3X!b3N^3;*dVk|64$H&ve}qnBsT{ellW zNh$_g*a%=hCcKcQ;f`gE+FFkXXPmDh8j!|5U94}||Q^t;- z%R3S>OpUpI*hqJc7iDx$GB6!7oyv@v;@Z#*681S+>FGL2;&(o9NQXNuw&mdai9MyE zWL=4j5~XL>e@-zsfT!Y*SI8zY+w3y71fq>5JZZyEj}3YmD_H%cpEGbeaZg({)WHFj?KG(c@2}#?k;hD@fW7Y%(II?0ZEmylYn2bIuVpRHpOtH4I0(MM#L<>+l z-gfTHFx~%z`ZRWSj6{N4tW7m-UC+6f^xto_X&m`_JgQZTC-xM52we!PTvIxJ%!^sc z!jR2~WA(oKy2Y<`PP<;DReVt?0a?(fG(%O{t6%rz3fK}`v^Zkkk1wJu6ve6jJuI>F z0zdRI^>zD#)4w_{X+GEHV_fd-U#;98p&Kd95sdA?3?16A94m*Vbj|#qN2zIc%btq) zAi}Jpw?TDRX6`FTwO{YqAU($Fo25dJ6tWdE1uvQ?{tGks$hKp1-u$=_BNa-AWMRHX zt(8#2H%0u#rMx_p0rlXND-K*T$k{BgtW7wcP3neWcsJ*aa)Pu%_^_o^wGPtNhv)JK6;Qu&fMt zI+w(^LwMvE@0i+SctK9!IX8I^zP$2ewuluGP~}!lNQ(H#S2=}Fdn{e z;r46CT$j%{;vRFF(=-bKelV#-D)0pjIws-M^B2p^{^*JX^jk@5&`G#nl8F%UJK3wb zjGB~vONcg%^+uoQ1NLqtX{gAZq%R3_61s#BP?uAXK4>)0WnU0${BZvVC!Msy^)F7h416fk_am1zSp@KF`A&uRYGYZSD5Dunk2F zcVsKoa;D~?&;avsrDYFpfbj)7$`TPW8gw}J!I4K4@5X(7^OgyTKvJu*U?R92%CPV> z4vKf0K_dJ?nswWr6 z6lwB=(iENu`|LpL7h!ZCFO0D)SlGukJK7*8>k%*bH;rxHDT%rg{>^4A(QWA(`HGe< zcsG{ZiggPMv4w1B zcM|PV@zsx|LLSX#E@m%80diqDKJaz%<$JS5mhP*1iJpMKomaQ(D4*dl;EA=hK8!Ww zFue<$NiZs+4Q~X!Bl1f}f`>h$Zg-vvJ$BfNb<^FP9?KOThQ9Gab%<}oG3xwXfOfzB zW?1H7Uv^-Zd>FDTWc?7u{rMhIn-x>PK3Re2<4nq3!(ZMw@u`SKVSxPRDt<$B=6=SF z;CD3RO8#PjVKOxe!0%Bej6~$bqE6BUq1^s{HT`~k&req|xyBPql%cqWlUCLZW*t#& z-*-3BuxL&WxRCWWFh_O1iL5z}&#+lLL6JKQ^^_)zor6vpDjqSt1cs1?X0!Yj8I&~4 zpt*f$wc@_~V5e83p?vq#%9^P*+XnLxbyalDsLk7!@8uRJig%_dFqIT1e2HxPZX=cQ zPBT8H(L+p+k-X{K&U*f}fbF?=@r49W2i*1{9%Pe8DN;4^;9T!M#Z<zKRy@#QJ>V*sS6XjRv{2xkf#sOssne5J41mQK0rMQ}MdvjY? z*$j=@*Nd0Rzc?#jlG?b55D0PXR=u3ePW)`-N2N0{iCCO{hEtF=nxJfb{gLSS1O9BM zCT~LHG#Cm4`9$n}-T9x=BfuDTTE$8I(6a8{e(JKJiwiYZIBrp8pwk$B+XA?RPZQRa2nzI4B2%ERnzHh3yX z6_ih(M8(-%{FHFfhRvLqnAm5V{DPWK|F_&IWxW{SBth?N;ZStE{bRx|OvDyOBJWET z3vjUDBk}UfxLLlYm@OFBDmOKBN!tcP$XSYbyJQbFW=TipO;TQellaO#Rg-p$eILdG zGq6;7nOl13$4Y0HZhLz2HV-oST(>@cG5Wj$>)FMZIBwC>KeM2%^xPKJjPyE!i~L~LX;W6Z>ctVPscMue<|no<*o6Oza+sw{ z@4}qlNmD$5Ot(&>z?g|4F5-Du_sG|*bdLKOK3| z%4!h5QNaRNu2Wn@iO0~$h~+_NY7?QS&wHd`-fXd0diF}6^3v_=s83=^F}sx4ZVWst zW8EfqZ&Yt`-CaPHCFY_AU4UGEd-r2usXbNe&EFS7;Xy$m)LVrULHDzubee9an-Lw{ znQgVsWoeRUoLAd&dZd{bilx=~W{9Vp9`c^V!bGv&Q^E8yMk=Z!2Qy3~Y*u~f7W_|; zcvMqIGir|+@sMsyHWRoSImspYa1TjNJ(6{l#s{3qTWp`YnF=vtY_nok zqkjKmp-0rNEQKtZ;-BQ+2;FNmM-~U*U%KM?AT=4(qo_;TSxUO?*LqZ^6WFoC~ z4x%1p9KAmnjENsNvRsC*aP32PPF1vkXpY$o%U-|M$v4(iK8d5UtDmVrRZ=KnD&-*x zDHN&~1yo(4DsCMaRkzC8HtT3(et={F{8y>LGCUo9icu8=Y(H^5uUWGRmt6bVZV&p5 zpvjPQd;K$fRW$OnJs~*!m~bWp`WE}Fhm*6jKY~v`H~R509!b5-`pAhdbN(QlXz!X7 zLyT9^UeG2I*7jdK7h9PSzlHvgfSTaJhHr~|l*@kq?`VlG7afKFz{upgcPB}#u1Un* z`a)9HtFTrq`Z(}pZW29H&ad+ev#}hWJt5|Wt(}E~N((U)>I?7mQnHza-@IN7$hEVG ztf-aF<9%s2B0Qa*IK_Uu#YJ1-_LGTOWzmS^_#ctBFNq(0hBICo6@4-o{8o3nO-*rV zg+kEnUoE+*6x0Q|`T1kmkVq;D^~LKx-ms!7!t?`zqb4K1-)QYr9-Q?$5J__E{ysuN ztjA4GGF@0JdHwqJU&ZSV62ISE!1P57x~`ha+`kL3I(zr|@OjMB{E4BI;o-(-4xge^ zzJKBdld!UVV^zO4x~OjJLk#4adD>e+E3cSwa-l(|GPvTJU6iLsj>oA>Ttvpy>&)L2 zUb;i%a#ky6_o-z+XpF+$71?r7dYey1EDlOZHRh1@0PB^rJ)aH$npBywV&kJXX4%&` z{*3hXg8z0`>eC$i)H2^64AE`1Kyiv1Oi;+|$`rWzMEEoso$werj2!27oCI>fXX<>o z-dFvM!n8{U2St4yUzFYklzZsEJKaqrQmLZ`H$we0mZ-z=l5vib;Hb+?=lFQ7Cq!x2 zOevIJslbQuY1RB599A|v(pweig zxF!}D?IaW)Y!i82I3=gDQ<;P2LY5|G2DIXMlx~)v*&p0sm5aqfJ{jS}jMlk~Hiwh3 zyqX9HUheBgd!Np+L>=1I|7mCNJ%G|ebZPp}_bF#$2t&clb_-|EXrSeqJeroSox9Rw zEO01}KP!pV>jKVnTtK#rZ3sgNto2npxH5K26yG!!#IcME`b&6^Uxq2y)FC71Yc6NL zo}+276H1klQ^Vkx%RGiFs|cc(y}|CKHDQYB=Nc5KueGtDnQ5P|fe<6JkuStP4R2 zvNVcI_T8U-Q>go^YjLjSz5~5RRdNM+tgamD%ARNrcgjQz9QFd-rkQuYwzDOD_Iz*_ zO{J%=f5&T9N1k`eggYFyZ!B<2f>01vTbcxUWEKWK8?MAZ>T?Y4oeG z%>CnC72RXIz9gu}H}iUb^I&pLWSg1P)l9eLq%$v#)JD14OnXXD&rDzl*jCMD?V0f`jIsR~5b*$I7J zxgA%Pdk|;gHJ2S}R?o^mxw~3aM@%N!TlLz!@*o8u8M^Z=8WXXE!w(v07$`mHZ|&{) z>YCO!4fx}t2|3`+jrwhDWx|n4maxtha=*@{+a+JA-#B(lfF_hLFDqj8diBk6;Z8hP zy81dtUaP|dYLf2|)e_Y%21e$3>xxn50gb#=bxw$3Y?XI?U)a^dc>rsuIM}ThF8j@S zB$dOs7PEC-YM;h)ZJUy{=_X58dD>H)(Nn*EkEg_9K z+#9c-Y?7zPSR-OEtG`XI>4eIhoos;y!uGJG!!>)@1jk>Jy*H)-vyz^=#TOY7yqD9a;l*>3e_?EVDRy{AgnOi-y*I!p z^jj)PJ#QmxjIow9Z$<%AD}ksqzD*{fv!7^#8&1WxV4}e}@0W2NA%tC>Dtt^Cbz>%* z=>`}DA=MMk_a`k(bgq1hA{IGGIcjyUac%13{HJUWd|IoXHG0)XufK1qAG_$+rz%nX z>n};MSweUWDXDy;&5EFQOKbjJZ87o9*BroW)NOlus-`)jTgukm&A38GCbsR$E?sXn zb7lxz)4L}vue<}xdYIVIUHrXy<0zyU4_mfL3&ofL{~fNIdO_R&<;B$>?=%m=IHKPy z(}zRa{9lbG-L52^kiqd|hFqUw+yN=aRbFBTSac~d8f6Z_jxhQb`9(kQPtv*a1aLzQ zo+}*m%9PY(@m-yvLpEu>bEd{CB$F!TI{fLy!(l@<#>VX!K@(irDm3P9bOErb)KAmq zO4G?#&wuJOJXtC>HPoA#=v12U*Oc!kqE~*p4%jSQllDE2?1MQ!>C* z0{s$EdWooi9g{avH6N*1K4^%T^pS(SNRU(XS#I^Rro$>Sy<{uzn0m49v{q2$|8dPRG-@yO_I5i^6d9@wVvsme)sLgR@S+`Bl5qdpcsvR-q|uK~tBmeRh6 z^11+Y`&r}Z|EVvn4P!S5rQ@yP+m_hdFLLY&WFbMv1A#|$2iL&{?fl++JMz2X;nc7~ z^7f%U7xJT1is{+LSoKTC-C!MFGP&zt0ycmGUsI>ynLHIbZ<2N~vDUY)^O$cjM?Cbw zIgCyYwhnJlBpfw(F^>z^?yWXx%KXH(7Pv#mV+bc($0y_vBpwZuYkfPrjjdu8Ba+$T zE;s>EKuw1R0U>FgN7z$%CYyk;qUXk6H!=&=e-3j(W08I7APRoS@%HN@tOxd=Ic`pL zfEZl-VrB9|o3*c>tj>3P)b_{1dG$nV*fM$)Wk5rYxd{*W3QD4ZrKWSJSUY0mVSuvN zASc7oiVkqYE3Lu#_19ceM@Wvvf8e7J5_DP1v{BrwvpH1nF3<{0vw5b~eaN-!58h9) z(_=hjgp)GWE+7h_J7v&$;NQ>0v;4%WqKV@6%&kwU-ByTk>YVGx=_&Jh_bgJ9C}evX zn}PtzsN(}FQq`-^Y$yh^EF9DO)h^Q>{}v=<;Znv~|3!A_v6A^(px<9Fg8;cEUB>|y zf>=_=OYmMJmA}y|>Lfm+{ow$5rOBHxZc^n;GWM`MjU(2K`13m$TnqgKzjKT(Y^>~P<`$O{FY=^VI+=x< zJ$t6!_iow2*MpuTTqAmmIi@>wiD}1_Y0jcGOk90^WObue#ZY z>m>!CNOks#zXvB!HovRBc>d~NPR%sg&Wm#vKzmk+0Mh*hv}%O16v zWXBu!M%NjoN+dt1emv0_wL8hnw%}Rq2fNhGazQ=^{;;3Xd$s=NlI6xsxEbEw8FVd$ zedW^$i9ROzwfTzoK3(EzQj5v6{V9oVf zv*gM)CC-q@(~oX1%;*gw#fBkT$4{y1*G9bI#?%$q1+f#N>R+{6oB3-C1Xm^FX^UkL z7Q%6xu#2v}arA%3sVvQ_+{z>*b~UR%UjO|T@&tX2{_6S&Kc3aqkaZv^U&ExkkxX5LEA ze#$12_XmQ7&U(aGsJm?!w)ztVod(^@?KLw*R=BNl=8ox{OoJm=&b0?#lP0HdP zuxjDawwX>^*-JKQXP;hB>024%p-xiON*s8Vb;?=kFazT|OafTnHaWwBz>OlLWU`TD709x_r{`90^@5mQSCVIEp4@2lkTW7JLLGi ztG+-3o!OJU?`ZNHEvBMV*_aIH{GroHE_4*N~g57876 z7bAT~jb`-mi}0Z9gv9HTPRcV zX#{kA-?>r+ryt(n54UAZ$A51sN-Zagz37*izZl%;adt<&DF*2e<2LtN*Sgs;aJ~ID zTsJ>;=n_Usz>E}1_P2M3^ef1_oGiW=4D7!%)$q7l;4wzv= z7)VVlonyC%*f(dO42b@7m4i+M0~YLpB>sN&lR%MKT3RYXmR|b0O4fd&*sPl%olqr7 zESjL8kPW{7E>c@Zhzty@gl-upAuPE5B%RDNMN+?Q6`dcF`nIaeRkOOkR!(uqbui*6XuR zE{evp-W;Ccn4D)ZDr0PFu8O$h%CovR=BtF8;ldnm-OFW4=fZY*jXZ_nWg7MCjR>Pw zahHF1#K8IgrKF;{w-r65 zA^9`pW=MD#5y-gY=3mo*K+<^r$}k+IHR{KbntMO2=WE+wfo|UHsm_UEhcl)nSHXqX zW8~7wBM*35*RwfFGOk3NUm&Ig>2{;uD44&y(WjozrymCPzzR^ZF z{W{P32sX4iqajeW5^C}09(ZO!@s5VEfHCUERVT+ut|_G~&C^naow>07fq$;%Bv;J* zja@Ptp`<$?sS)a_Vql<-RIr56{1O#u9Ig*>etVEUV8?Q#PI?d>790*~4Uh+z0y@U@ zp+V=L?a}Uj`L2zT+p6X&V;E^|d+Qs8eNb0g{Cl_qFP)j`ot>~IPpp#|I;WE$->_zj zSDfGc9&ZDW6j#LC$s(+9b=c%bQ;`Pj02^ZFyer#H$1+deg#6WdAh1Y~=irwcqb{WH zl-bXzlturu&qfFvh@8<%_N^W$mq0}>EiKsziQRnD(VU8kSj2K{xwOxtGMOSIuzq?uor9vVc2lgxqSXYO zTu8et>?+=a!js?iMAYRoAdpH}0&f8UBX`*c*pv^;crp}xbNDKu%p4b@{?8HIh-!&fcdm!|6a zG$>>#1qT83{JTGvJN@isp#)9+$6GSBualfPG=|c8jX znH}$FvW7|KYtPlgpa|S4BvhAvu3%QQ2;nX#BERik&o`FcP7Q2`I{{NpBLZ$#?rFbTMI>dFBaU0-sy`|Jr%LHPZ_=w;yDZ#z0SXi zL4cd$3pid?QhGjMZVJxQ4Zi9gRS*iyajCV?)5r=ztb9_vf{=ydF(SQNs^P9!UhIS+v2RIn64vyZcT zR5#K&3(gOwTud42T!@PR!toK@Y>;-saGl0L!eM}*m_@iOXsdT*fWwA1xi~=;Onroi zURVsFTrp|1uF!<}b2u9+S*Bp+)UTNvLmMX{vP0ui^3rYhT_ss#jg@1nl4rT8mhrN* z`*e?{L`zlfT@hy$ajOc)Re;(POP) zy(ER;lFWo2wT`3tI2HQ=B?Fz2*tNstD7u1I6BwV?ZF#1>S3gF~3NNRAam_NP>vBdk zTYy9=6><-Hrxm(0pLK~$z6pa0$T^`;cNoa;qj4f8pd(bpvpv*bU=QmaylbrGFj^e& z(M4CefoTabo`Kj$b@>P;81ArTaBh)#l-LR1aM#3f-h@iQ`#AUUY3;@qq40ncxt9sB zozyS1KUlFwR3;kEq3?+u&j&pc3z!w5fr1xI#}1NqmiOv8I51F*&By5-P&_U zQZ5%3b1 z5Mm<4wWsY5b$FNV9Mn(1%lVmh?^k6ZzkdRF`REPN-<8u0)B#t!&W1Lp9#y+JAKCT; zIEz_{=s!tE?*7!`;R{*%X<+Gh4Q+gk`-0`T#_3zgrw(d}{9N#3eqrnKycuJdUYJMa zvADQuYkDWds@^}=mTPG}R05>zs~uj0r-ua)gq4znks#BnjzkHBS=uX0EY z)BEhEF^0X)pDtsrqL44bkA!x&ECzto%lY%4L~U2>l7_H0`zEz7N--x+%11Iy-O4|5 zb)<3X8Od)X`V?cAYY~Y+i@QW}=3}Wm$ zoiuOAMNc^He6~om{|wK6g6R6Y)U8ICH^&Rx58P)V{jU%D382~x+v#J3w-^*VJD%2_ z-ivU8O_-3Cpb9=4-A{46hWFET$8%E>4woc&J}bo}J5_T<;miC9uCx2! zzFueh{P2tK_aWYhmy=R-lE38a1&zf$@05BU>^~^k$d?h_z}=ofr_@57y?ro3+-TU| zG(A>I+L}+ooA1nz>u1kr^+nt<-L|$o*jzp2Z$83Z{HylNM#!C?dM_UU1v9Q(AxYI! zoK^E#qb2eyh!wUdHdPa-kka$>ZOaVK5~J9IXdq$nS-|!g`^STz;$}%6ducx_tmVi`eK?zq-O0p~b9eM@hULHDt$N zrPHw)=~x~l-pWGd z{OxIlOaU$`R~m~8 z9qGF>m8%WP`~pX?%$7Qe2f8<^f=ewxqoE9_)KC7=xMmZ5kCYd(B>$lec=s`S@s#RO zi50};F>_;FLp#S2dzujb#Hpg5E{u`WD_1_bMJJngxr!tDX)kC4=4j_#Alf;TSZVUx z_JKGL4fnUUT`(46;Ghek3=4pa9m2>rtElJbO-FU_K}ZR`Z<}#FydC)@t7;tir0=+x z+B4ls+y#k~SYiR|-rQbJO0gk#&5-0`g=e=^mp!hjonjLrdt z>=GcwQ`N32({|=&T;RI$4f;HNp{cP@v4#d(q5aKnSf2k`+-X#nppY%drQf9{Z%Zp? zzx)@PRQ)t*LeHoJ-U6@Es0 zH!;^7g3ywnBA6!Hm#+%*x}4UPbaxP9yyCuNE>R4GvO6rVeojsrL$3S7(P#ZI5=PEp zVM~ILA5&3LeT)|jBtl12&ko(p^17IzL3rg0e>U3jymr%rSS=+wD59&vCKx=!?z}`n zTJ0_Len-y!JsM(u0F@ZLDr$>r}Byty-4RnJ~R|2Lk-S1ff-$@-Y~pyl6AA*P(! zZJZoU0xX;JTv?m^OVbsmQm$@F7mPV#9&9b2;w)aeZ>sm_3)7&6IL+3y)?<(DyLedmNg_S zr9!=eG@%d%^B)fDsk)Bhb#HsTKI&3PU24&x!?+;Q2VR}hdU{XkpH?|fll>p+0b*VkE{aHc*^r@SwO6PAZPg1+jHhv%o`FIjE_T^=UQ zp#-oBoR_+gTvB~N-}tl3vT?O9_d@KaY59k|YoCb!@U`@6@!->}sPr3WcGQE8Bb!vA_VFW&R+3E|X-c2nhH>xkv+k>h6Lm7c z`=h{OtnKLkA)lcU)jHmMkV3`5_W5S0dV%mhWfsiXLI$T!f-V5>iJlHnd>kly1+Mc~ z=8l1cU!n^8mgn;AhNw-@kxH2e1+MVEact*o@8}@{b8+AE}@Vq;B^@SepN0eT5v z$sH0TsrEaG!4@fB_z=aa{E;ntcXZSka_6(ydH7&ZUQ^tzL&C=xmj7(K^pz_IH#Rm1 z00?*v*0zXV>6(4n0s0VsNckS3-Q5+7zm@U-qT62ae1cvQ9t_Bjb4u!F`T|p3oi03o)~8`wP3# zesE`mg<(I$(4T(q_TM)mk79WkKj7q+i(tt))tj6YjRk$%q|Lju+(x%GJomo;K5NAU zolxktLcb5G7eK~`4a$9snJ7vrIt^)Z>McF~Ewc+;>-Msu4D2`77s0OY=sOUYG0(G} zGE~&bB4i*wKb_ogMwF))m}#9vEB>6A)D=4diQGbhGEo;#1~N0_%V>wL#LXZ z&H=Hm$vlgy7@1;3BrU>yMsw9cX zOs`R1Ra;$$)uZrn3p-mO8(3%goA|n)kZqfJwkzjCr^xt=x0#>hARPdY1AjEx5%s%b zfmG-R=DzM`ZVj^%Vx)K7i_BF5Nq6K>KOHjC`Q-s9BfWB=VsrJZ3;;R)sADevvT>F!KUmg?R$=7KEFjo8M|i zr+#f|#$)RuYB2G<#^2KXht15b7phW|rK>$5VWf0)Fq^4q3XK z{EWdJ-xpyhiCG3(393uI;ZM@T=wn+twNQ?P0CE*MfPtVqNxU5-76U)>~; ztiL~;4WkX*P;?z_#7WSz0nqVRCdG*S@_P7y*S?@Dw2CO+P6M$8Igguvs6U=sEXs6@ zQ!EGrYPbeGOrIKhVQ>roiH@p3?`vd-8V*9T@XCWMExm(Cr@B01Ccucy1~LXFW@w0psD5Tju(8Bz0JeYI6rVx`e~kpkz&i58vR9&7=uug-V{*Kn-%p1G2_R&GtBu znoNW-B?8owDSptaSv^6>7{1+I^$7DK?se@l?o$IFcT>E!bz9I{>7o_q{!1$0Q zyp2L@AYW?lM*Vq+{29wAsLYYGw%fpK`7%~|klQh0;}d0fnP=N?j95kJbOZzhWU#Wu z{={iJ9GH&+utwT5J&*o-Y_hh%rnw0|7Hwee_ecvT%I0)KR?s|=-@^L~_-z>pnzFYx zWhNDaozu%m+q@~ptXDu^rgtv+YsRDY2|#RR_R6$k#0WGeQ>^%xrHtVi4a&xOIZ)q*Q0JvG0W8c zWlzq3DA)b{DYMq#ue90p9S`)KdD}v$rNwL7i*B%UCdFU|ys=*RykUCz1>`}VlOz4a z$_au^EC))hKPI0`M7eTm*f0NI$r799BGS7P5)-dXn9~uM_tqn%g3MHmwD})2cyDlL z<{g5I8j)+*ioB-_T3X$)ZodySSA{~yz=*nk^Od=+d*KuX2?-7eQI{V@qXCDVXRlg1 zeG6j$&p`Y4c`_q@9lu6wXrO0-6c)jC-~Y(<8=oWqt*r}bvCb)JeTX0nP*s}oTvWqTQj1`|Ap!80i70joeKmiq zc6(;mRmQ37&_U}qou7r(7xYGvvI1%geZH?@B2cp*5H6U zE!HQQR6jj;t<-<6v(rgbJ?#WtQ15J*LMh{$6lkip)bD!Myiwr0-#5;}6gK8krOrul zvTipTPei=n7OAaMJ`sI9mRB7Je?t`dUCiiE4gt37jTDQqS-AcFVPR~g(BGrVnd<6% zKehj@*ZFh~3@9DjcPw7T86xG9tFw?}O}Q zeWZaAUi1I3!C;Q5(HK0Q@!Ha4X?b~9M+d@y-idPM6n;$VX#5vNRea5 zq?=h~WzmG_0k!8h)Di?)*VWx!@*&^<>Ep|yte1Bm-$_DTz}w&mUgBfg7KgqSekq2Z z0qN-s@}KQDqm=}MsdsD@0@Ck=#31n?48+^!=hE# z*!bUVghB=AMRjFbId@`<%^|6%+J{epEydJh zqG1NBv)G}1Cgr6$lR!=Owyf+IZ8II19gLl@kmroHxqMG8c8R|rBie{1c+i@iMh&W8 z3l@fKT`mYOq5R3p$l4EbIZyvb)UF}?f=?-u)-wBU&c1ah)tjL(y#`c?C)TpsI*QFM zWu$kHv|r!lM2%roU7c&(eOt|Mk9PC(lF_5HGILQ2V_rBuDebwYPAp>_=J)kBB&090IN9N!~L&xIM_1~>$i zRJ|&N)licPTQ=y5=bBUr$Nrz3-+pI*nVI1fFmGs`0_0&^xb*k%YZA&`jS4v~ z+UmgAZ-ibM!R+h#IyEuATl`Q+19Tr{X33q!eDh|nZq2G+zm$PJ_|_jqdaV$*0hxN` z5vc2X$mIN;WhpBMxidZzp*KD@A&E@@)B0CpRw%*H3scw z;el`Km)=U7xIXGTf4n9R(@e=#ET~ukwLbpOTWasQrF{g=2z#}$2Peb7wu#T_b2r88 zBsqRx_rH;rkbz=4Jwoyy!iMb@h#x0%9<_ z(lfA36^^EO{?!xOBeh0~Qebw<;-}Q9LL0xGf+V{4URH0pSLxbeo6EaWUkSf#A4}k? zKlIdBW7jS}1H)^B%&GZq4?Nt;o~>0DC|vwTKJ2~$C+hY4QWkz`90z3U zCh{`UIbN@~iL+=OB8Wh~J-uJ;1frpWHa=Xa^c!i)k!=>IVmVW{cRNRKM~eWn2?!7k zt`vMt<#?o!Qykyw`EOp{tfYT_hGN{=^@p!cm0ey?)?n<2j(EG~p#mMPL{;2gNDd!z zA9T*z{sB7^JuyYP12wvwSebvSOwZ*ub9aF9@A$*4R4|Fwwze60c`$Ie$Ya=G)}v-+ zn5T`UP;9b+Y=^a=e$CxFBeQ&NpPg9II+4_DBFv~@@uJKOVgM(RCq1{lKx2_Bo<<+& zJ4%90awv1hR@O0gL6eDv7y(n8JNCDC-Gx%B=gySPW!{)06^>Sq6$?};DR)^r=8;a3 z$zRY_@t9(|g~z6>-s!}OVjP;*RNJZdp&L$9~i669hx{dTk>!`S7onVJ1`S7vU0g*^R?G$@c^BC*}}dn zBX_j(udJ;u)@;7GrCe)RGt|Y(So>=}WyW%n=AlP*jqnd-G2MI>J$uc2ePQGb`R#GSFFk5ITj7A_u6s2vG&Yr;|~PK6CB&d%Po+SorWiNweA3PSUN z@p~c4JD%q!HjRy9p==^OgvVqvXV2}Wd3N`uQL@jsCzA5=^E0xt{8fdALeH%XB&KZK zoEsQJC3wrafyGf;3sLs&L!iHs>}mbB2z2P_ipuUy%NMt`IbG2QqT7RX8%mr8E$b;J zomcccdp`STkW-Q}XVhO{iBe2mlXn+N&Wvjp5E9b1P2{E1pxadzOm6FQkU~~Hk0Umi zSibvXlBd0AHd_;m7ZkzUDL&jM`P1c1!K~O}SC-e$ z<7kLFr6N~ZnV9j1(|0-yurc$bBRL;ju`e7T0@3EnCs@h0rS7ABg0F+qKRf_s!6L$}j7Zvy z0w1iU-%Tgz3VSP#@Xc^jSv5wT3J3^bU|>2yqpXtqkK|GN!64SCzY@>K%MU!j4DgKY zM|(=`7ZD}EYFMVNi!fhlv^w@)+Gp$jI_crK3Mf|qPG_we8Cb6y9_M&P8JE8Hik}BK5k)&JcR4c z$jE48EB=nAcfE7urzxaydCORfc{5WgSY~?lg)|nt+Naa2eD3E(>Dff{#PHC<6({cx z6ZnaaPW^(Kh?av88|PlW@cUH~ZJYRhsv8sMR`}P$`X=XQSoblQLn+MU^!5C|wTdgi zoScAAlni@J&jY1+f-%HXa%93_@Ur=;4>c8O{r!}c%Xx4|?p8f}1F|^dlKz*AZQ`r7 zTkEyt?RmuhLb({!egv=N4%mgr5wscHEB5mk0Rem5xnrJW3*nr_=gcVKui%=#!pvcO z(lU7z5R2&m_ryZQ^M{|;e~t0ayw)fy*7G+GvBtI&%cf+=N5MIhVJ|Hb!h)jY&+(N@ z2b+DWWybF_6UI9s2YIt=Yg0b`uwWZm7rB=Er}1}&_0pd|&fwn{0Lr?cXce{7Fn6ov zvC$FM%sZ$>?m{#C-FDa3HaH#c zQ>GFx;Q#8CSVBU=XRN6bt{Z=eH$NwltCgp%kfn)QiZWBFAFj_+(KT0Vw|>xzCt3*< z)%?oU$K)wj%4hO_t-la9oG@KoamVJjedU=jxy3RmpCo&+BnO#CPOHMbD@-*WhK5TU zUG1cP#&21GGF{JCnm0eHJjm;l6tiMJ;xjVf+3j5$)Lv6j^R%6{p&@_wAW&4sEG>Km$I>0vaVBZsPUM5!>hL@_}z(B&rO#V-W0A< z&o1>e^|(i7rAv#JgWi7@yPE16Qbr3Gmao{as2*9oy&mK9&~wr`Mme^BHT_}MOpKbx z-|U%~nrxpDm}VQekM1I@ZGWQ|;ujIYMHtP^2q#2Uz-I}_yL`m^4mLiB-jGNIA@v(= z5#OTE{GaVJpTG4-e@;z(!?d=`2x5Y7EAYL&e|Df|Pw;lm1CW{0^BR0?DXgyhS9x^t z$FF`$Dyq)UpA}3^Px4fdX>($A5<%R5nsnK)huEUQ#dA=fm!3SZp4NL3GvL7gA?rQh zv3}n+;G3ofMN~2>t58xA677s=8reHpk-ZfXLJ=w=O0ri-8BvjBC7Xn-Y_hW6<5Iu> z`+VN_^ZEB5&-07>{(i6PJdg7@j`KX(`4fU_jXvtdSB%y5{XYBnAcl;%mm9n{Jg8U} z)-%RnaLZAg4c#11zZpmg#C+`qEs^csgno`h{f|0rf+JCdo%ZRe!(a6}{`OpC46i{03|JZDL7T4vQY| zq5SXL#7dV6rgWUNP6DGvosZ{o%SAD5WotVwXP^6>MtYAAwMI(Mfvt5$xf*nhYWX$+ zkG?P+Jr$?+;$GQ4aq%x$TZMr%e!|~9Oc$;jGjBr+eUe9!wubZI`acdK8~jd-V!p8m zr0)i}*9t~o7x-gRsX;}!G(Xd_ct$GJIKDK}$J-P!ECLsMgM$m~$@t0dt+_1dSF_6rZ3kqo;nD3!w{Otm-@+-Z zK{b$}ws$ie&wEK=*{or=1DzUTpg}u|c39QSku^GVs9KitHl}LKEB0JMYuK>sVi{pq zA#D}k-e&e`@7sYQNbj=7#+gR#4Gmi`iEKAQjxc=LBIG}PqN~1sJ?`jum@VL{=hl|4 z8#9on}UjS)+V)-?u~KcDB-bgg#rJAGgWCG+6wiBplGE~wv{evZ>KDfXPn3^d7^ zw4X7Jl&etrGuu7ek$^7z=)tybV+@ty4cNoWuxAyj;s)2D204`F#W4T?fS zH+4OAr`5rNL|ATJOSdL7w-sE5);>)Ir%!LlTCMoY@dDx3^55|LQPtJgU((b(*Zh2U z=GLxKy}5iZ=nhO2wrv%}*l?@_U!gbJsHv#HzS zGopDOvxq>1mBv@c)8NB@XOOvXf4Yey?8}B#x>fzCU+DlZm-(?X#prR6rb(bu;t15(WlD%a;xVA4tM)hD#Tqr zopj;cYFbwfc3D2o#C6Qj^L79Lsv^O-YO8FkL*t*a%&&r->&z`xny5#OT zytuGN81ktQ{ia2e6>d%v1)u-qBU2NrKIulx!Ag`y5C=+E+LnrZ^a)jBzxNq)NSPLx z)2WrIxF{S@(n-7(-|FkfVU4Kc-A1$HL$(E0`2HBR69Ez@Y4MtoQM9g?X|K=Sk-7DU zHmvc?zbF|bcG%)=DA)V!tgO24y4Orhu;ja{rsjR(m5kM1?++e4I0A3UDC+C;nwpgu znfDHTlXFRxXJTv8%J#lZ#h-g+U1TTm($oli%B@?s;woYklpwQT=HiuH4FS-a(m~vU zy2b0_5_jUt&%FZ#%g~dyj(z}7LP|Z5%5W8Oho$8KmjZSO)x=j!JjNLMm-(6Zz7y-k zzt&kY!+kk571=>d!2@H!HQWy*o+2ck&ZwWB!$>`lAQPviq3t|fd1+Tjl`D4xi|ljr zhD4{ysm?L3g;1Fn84uU}VWfV=cj>~Wz_>4Ky2d2W?GJvmFJa#I=a?sr{$i+^FV{tg z{|s45ytVt4%T|5o^C=oJywhFm?Y%B_?4)m0wcul?2*!mw%2!4tn9u1{^()%tGoHFK zIvPU{oz-FL!}Ggj5JZ?0LotRsL9IJ@NIqPMXx1Rj!d!aJ=x0mfu znP(`23E|WYPimX~4?RXu1qS!{@wI5)Ja6TBG7#4Yjc?kGo+iofs(QD8-sXPrPGFNP z%2{#iviL&}_!u4joPpe_Ir?YS@oCA72Sq^5@(Jou)E|EIZjrEU_2}UWk-9IXZ)p11 zZje51olkV4d4SCXEoNKls_qrs9wJ3A`yA~?aw4H%>y{LSkXG4JtJ_li(#=OnQbQ}rRVGg+9YD7W*z7J`jB*E})ww5{@qHpjU% zC#`CLtz=VK-6YHjj!Pr)5jcsvCw=ne2AyEvlRI0`8Fx@fXfz{o-XxQAW2g+5LHXlT z=g!fA;L$F=yOoSO%;c(5%*Ku{#J-@?d*uc=Tgv=Q12%8J9NkRr1Tzl}g@{{f4t+7` zc15NULy2N_X>Z=|i?9a`3yJUFk^dx^Yi(i@FQfkPcXPgp{4;2Ff!$xWMv1TSFq3!Z z{8^o%sn)9u6ksCEGYM9tB1QlgDGXr}#Y6({lRJrv)K0us@8iX|52OzmjeYM)ZSBny z@%r`|UL0-0a)Yz7I!Sn_rBcw~Wl9_u#Y4H)Wmlo=55854H-}9}*rz`ix@sk@T@c8e zbB?X=-@R##vH3yrMW!yLDeD2E0TOv*C2b@|))p|$or}2`4wpC?wgN3s_15CjFM=^@ zSyspKBK(j1%!_dccTmB+vot)wX&E1%VvHjCYReL09=s&@Ul{4T698MQy~vX|VxcSu zXF3;Z-GlL}1dl(>ZnQ&4M27gW$d_na5%+iYNsij-?3O>>OE)n8QShF_J5u7v2i*@F z3I}W?iI_nJr3B@vB&HZZ*&sXV#u1^C3ZYK?{b|$5QY)62If=su6k;wWSurIb@uytB zw!*E8AqoG?okbzk3I;eYJYnTy#M%3iN%h<0Zl59|Y`D`&d`d{k{|u!df93xkbH`gw z#wGLW^Z-3&zbZ5XMOJ$`Mm&^)#LK*+37Ydh9%dow#P?Y&CTzT{|gSW#Qg6| zmBKo0Y$d4x_V`fYH`v{>pKRy{t?6uP-Zt?{0OIlmW5VAkN5=7Ay!Sl-dqfR(_$;9VI-Y=2AuA`tHp@0oJx_#e{>6Be-2+3`2Z4JxK zDkK)!|9!^~YnECr@O={%ma3XN?2pOji)kl`TZP-(%^v~=(ms(_YF>o5XTSZJ9`cSz zgu}1h-(44-O@y17S^nQQ$@ky4GiF@67#&_6^#uz;m{0~LJ`gza=4MT&jj=UXn%j^( ziTX~G7+Q9vH8t7Egh!*y+$T#T%<%kmx_(NYUV0Csd70^4l)oP?vlYy^f7@^m7R|Pk z^^BNxqF1&jJ8@8e$(M}f{t_-4R2sxHj)aUYk5@sN*Zp|u`R@fJuK)jj*~|-KPrBn1 zlB2=VVp?}Fbh#jW$ol}Y`HDl>r8Gv^$jM2Yl`B`;VV}u#zviNhrR6v?Ti$;!HPh2Z zjtgNKI9u(3)F z5^cE90j<9m&SQh*uBHCxMGg#d!>~PK=z!g29K8oMH8soJ7i>~IRn zPfSf6*tc&a*nd_bX+UDZB(VHzWM}12%4J>zl^Z!M_qc`PZoDo%4 z43+Csre38@`bG6V(dYMuVv?#&(jc11b}{)`2e*)aIW^Jrw;n(rm z_>dHb5z{@ca`h|AB!?4JVlK`UJEv4wB$?I3o-65}sLQW6$|TKO3{8mz3x7nOR5VQ? zxRC$O(Bw2Jp_ow1FqBmJ-59LTyy%`_nHFRhHOy}%(1ZMeX%i--fBEz}kir2=< zvbmNbwNsn5t*zy!9bUxuWDX6V!TY4*%a9edpDDyBY1vbO!gFSAEFw_7{COm@zXvD( z?p*9{oSyT#y1FcuUj#AEWGyle>b}5U+hM;-Dsa}F+!6@Cy{`xL*kA~VGnl7PUV z;fwijayo8Y_hO5+ily>1P&2d9QvqIWfT5&?u9A zE<${3c5_}491J$7*_9fQkHS(?<&qRr&U3r!*PlwLP2R9ivoTA^^x6a^NwJKeC;T}f z9L9xARe5A;B&==jwy!|`L$X9WB!r2y^Uz|5ztE9^V6<5;-5wo3N8pPbJOkjdxgpzv z&a0BMTWP&?J#E%*-usiRy@y+@JH2dgXYaEBjcbShyL~ze_lP<;P4xz$i8T9^cHMv= zx`PbUCNEdhapl~v$;PAcA9ZFv2RuP15HBmHi0niQu9fCN=J}!$gZ*1?ReYr% zFg_4lGZUP!RfNS~K+Zq4c~G;V*eElZQJTS+BdFA)RIdYz!MPx23X|6g#M?>wbI!0> zEGLu$dT(~Q&ULcaH*7)a&^OOb!0V2>2ivaa-X}RJth|^kPO)jwUWGKYW>CXrMnCy= z$C~c)uoArQ60(7crq=CW3glz8MKRiRtag4T4JQ){MaE0^~OkawmVY_etZL8_f84 zE?aP!DB!&;BCC@5T3Mi^gB9mUiAp**Nm@s4YQaUHcFGspXY}9WU-Ye;J4`fc z#Yf0L5x6sC9!9z?r2(g4bCR50v}?a-i=Yu?WsJMic%8D|j=)JdKL-tpoRc}o1j zJd!h|%-cd`VgkuQ+>lTXb&Z z0;icn=(ONb5E2Q@#O0!nzy8g4xDM+$q~Tyk{OuuHJ{_oM=54-YM3KjI7T%=0ySp0d z$=9uBhz8~^lNphM2M>NA^Oq3ed0t_?3DM(7R|#3SMo>~z)DH4cFq{-P-Mh_f5Ci~8*+4KoRA)=FJHanS{MItx8mMVV3!1wx<(kgzT8@|wgE@T zC}XIA?rYz{)Up0uijo+wPbR!*d$AT1P=Wz%?{6i@x;n=5fM96d(vJ5v=-R8Mj{NXk%e@{OjN_7% z_xg);tcH7b7~@qQv-wX`lXA(6)#hFQM2F2=63%WK9e?+Ad-vzM2lHZt`TzCn*K621 z^9Rju{(5+*$&iBT0;fN)j!q4IhMRy4c4QE-o%i6_{WI&Ou>edf$($J&&)W zrIq=A0r=G8r9o3FzdRV^F;YwT-SCv`_TP@Xf=bQ77}Yd3mcF`HV+VAH9hPJwj_aJo zQ?X|z*QkMpW)(};dcB952`rIz{je2f-O3Yt8X{$4UU!A4lG(qAy2Qs%uz?h3x{UvX z^+z2+%7D!*vuOTHEIwQE--9aF2ZxZbNA%?1fNvVW{@;7haqA!s&Xpl)A%HGX%~}Cv6MadxPc zAqlvl0Hs3~Us-wjAN=r`c6ep%K=M<5KC;Eqj%Nh>fIOTzTpSAN<`30y*nK^BbfxEU zs+ScPCHkHRiJ;|+3{g}3^E&3(*`&$lc^O)-_Z0X~N=R!2NIXOk7`46L7sA->TdMa* z#d)FkjAXp?Ji>b_M$HPs%s|wueV9&*r$<_L?YbXbeT~RWU}Qn9sFT8s6Wb5t$N=Rd zkp1ksf|aJL16)3kVY6jjDlzkXKIz8;gZ}1+{UlP^ez)f#^HM+~h!PnQ@%Y@xj1!+} zTW#$}Y?ysdg0YG4Hlhf?U{uwU^}nlFw{i)67(c?JjIC zoDo0-oJW&*umTh^=8^oN5I_SXI=*#;HZoBU`A@8P`_oi1^CrEKe>)AOjVH_`od!G8 zb|Z9bt=Dv&H!bS58M5hw{?W>FPW;!$Wa?W8f$S~oM!zf~aH!pJDk)NF_1WOtEIM9{+ce%? z-bv(b)ceH-8EL>;W7u$>22X9$Z$bDw$PK{L^d|K@5P}!(NC=vsdo35g|5v{5s8jqU zbc7`EzjQdRP#Lkn#KlIudEv!Ttd3n;cxe(S15h=Q0aykZk07tQBSw8KfqN1UNh8I! zXcCRp=*#kW-~KcwXKO)Q05BtrBJnG_&>wo~x5E)cAM(6eT&CNP3MnsQpr1LB%R!7e zCB&>Cb8DVHt};j6J_evm)*|0%(Dotr#$%dlB5E?;D%nCqIyr~joi`?T+vkMB??=5#5-sb!9 zxUB42+@U$!8{$i(*&-`~2YgFQhMwx`LIMyd8V-G21gpYN-NSfm5!PrN!+SyUV%{hl zQJ;L``{#nV%|-8I5-V&_I4c{oUOHIc2V6 za{RVeungXMhP5<&X&RBiUT0wOvMi zHSd+`HNb<_2vNnD1{1kAx);8KVnUkX8UajDTO1X7h^nGL1M!M}ji*shLQUUm6wo9n ziXi=fna#AAn4GNp!`LQ3`tj?rgpGI5Qvaq4DqvC%j4lf;>*~d{?FX0S3%hQl)heoL z@7tpM+I$CEWQj%@m62gtqzin6bc`gp9Ks2S=jbH>y@)$C4nBw3_hb6)3aS6aaxHkxEgNQN1X$mM}1;WIgmbOy0kRT*_Z4uUfO zX;)Xtr8tLn<$Wpew33TNO`3I ztAEXKOGpOEi_Q%Q#SiT4rc?KEy9K@8AG>skVJ)_i>N~q=EK$~y7;55)TzMN^TdXPd zWrWlO=$fZ?9i_g_fh$#>qk#`84>IvZZQRtSe5fYl9fTMTPFi86S_b$$YS z7L9uo|5vdJo%uU>Lir8tpZF}eg`1k1>~N%JAo1sZp55FA!^6%Es}rpvv7rkLSr42` z%e1t#+%VB>E#@qy9-ocTAH-kK1jbitD8RQNRNsO25ETijH%DY*4Ay?I6}+PDo|yNEqpvg`c}q4bYuXdjscXK#^@V>!F0&O)sdxo zZbtfEF(E6zC}l~&>m823hVKDRhUYRZ%+(EpHuT=h)0Ef6*3i7e6)HyA_@%eESB7DY zMCGSXQb}zox_#4f+%+;&i@1;a3v)A+L_GIQ%lbYmK2ulRk^R@di=$r!X=ZCG{|L6s z=hroSuzlZkqc);bp?#c}@;A(=e|H*}BLV_R2soAmaFJ6&0yy5tbW2Gd&iveL+oAy^ zFM@Xg{QL-@Cp|ZHb#=od>;89SGTr|Ut$lhi4k8!j9VCcG!)JbqvTYqPN;M9-W;(nmMot947~KLO$pla2OQ1WxEfPW9<9!lYhpFoOcX^blL=6H%>-By#v*KFc*mA&z8#!~qvb}6z zlgN7}ej9jWeLxQ|29=(~Um-LEPN)Fx65&n(oqf-8zsGm{jQmnU7U7~IJqLmT?K2%w zUThs$`m*8&vRZU6+-^f&_5giuCVg`S1ZDUvkn6S<{;=8W#B9kC*_y0Ljh>j#PJsyd{sl5zx&`A z!xUoWr~vGI8f$J_2mupv)zeu%Ht6O}stWP(v*f-I0*L}J@+{>EIp0)ZC{OO(n5!|Y zRf~1)MeZasP#_zfRyI?O1?KsG# zpi(^%Jfecp;Sv+!GLUe#M1m%C$;=VHf9F_o-@nHVomo+Vf`Q}=PZigwJ4r%nBU}mc zTY$u$UBt-)+JmB;q==^UBFth-J>GG<+&~xL7*0E4=R0v?8QRQ%?L9~*6&u z5|~Z0udnaeg-vI<8E_jU;lBLpv9$y71$CY`nG1>P#LHadZuBm+tyPG0uEQ&iWOT6^ z%A3+4XaIC&SBlWy(N^T7*J{Z^N7muYCOnGXg|ZJ`)1^`lNfV*SIL%q4vDy8tEnhoS*hf7(&W7$JeG1 zKzsw@m{UrMEmq><_#PqsFW{Cm!Q%bRy{38n%4Gy};LljS*g{`yWCiI=Ch>!q!3fG#`TD88_WSpWPoF*= za^Sxlx}TuKf{Us+tswiq;LlueQze)wq>$_4!f6iWu>7O-#VB|p+sb=Ao{zDH*b{Dg z?-7g`Fh{UT(<`NiQEiE>L6gAdkRC7sDD_F_A>mzj-d~N{GINbI7=yOhtwWlE4{UHx*ah{_Y3m`MFDejkVhHw9)j21$PMsz4W#SJ);pE$|^b zGd}1+eq{!|C)Bw_EJoS2$&M_P0eB~j4Sc!;b&x6xSP-T))LQLp3SFUcG5I3vZX+ZK zf-X^+4^)bS$Y?o=38eFH|2yIe_FaP`b1-5^4v|5I0&IkDmVak zC#6sOA12JF`sdvw^#k^)pFrPSL8bewe_bZHnY1bKGp&VeXUV?J?=%6+ zWVpr_Ha0Z%-N4zjK*sGDwZp8Bz=jDSV&NpSSgC%%8!^{n0|?DaM1`YC^PdUJ(2jEO zbVBWZ*99jm$?C-(4PYl-sI~KTotVi(aBaBj!pw)xQ5vd6ygV-{&auqUP?a~Y>aK>? zs3?RiB&DU#?vJcoHUbU9^9d;ROcvKTJ8^k&HjwhEcB4ArW45xg;w#|Y;++zJ@w#=W zR*d###MECxd(^tGlKcRW0mI_217h(riS|8T zygHo!{@Kf>m9&cnHiz=2?s=T2-yStZ?{ML0s@p9e#*J%#@iBe4rJlxj=T<}6+OHSx z!RED{=HBw-QXDjWd1j9u4F`CxUwh}Ncb8+OvwDtIWktB?RbiSi=ZNv}Raeqs>bi?vv zo$2xNy_mec6s3I|iCE`+bt1U1Q!RYyyj^#CB29N4dGVW%(sEG&DOzL>@kU zx_#YB_ar(xx&TSRWv#($6EcW~G@naPB?uy&(BXni!8*7f8JE^aPEEa8@zb;}w#t9U5ETw+YOv|OG z;t7R}45irF*{NMfeT_F`HPCQhx5R%GDw(=9^}BrM*@j&EjKNWY?g4Y96BLt@<0H#N zZ~R3X>I{6SdA-Jivu|eLcM@91_XabGi!WH!eRmt*M=+ zEaLEc7tV@ae_hf*L)x-4XU;fG4wT;6$U#c<-mfub`m-X=e-3)lvg*9Qy&nIFdwTl* zH`e{d_ja5hfT3r{=qcY;h{Qy}c=CAMR_9PZ&MqGc2Rgh_77xprx-YV7D}g&(cPw?cHP>G+8i_hMv210y3Vi#*qcaSERJcbh#V z&me-{+J5lnizaGZ@%FZN<>h`I9UTRZTW}s18RFyPMXY~ql9rZE)GUa|G_LRH?mm6_ z@+Q34ZUE#L&a;!DgB+5Qk*{7I%ra~J{8-3X1UCpx#oF)Os0am&qdA?GI_d z-}4-|ikSaU@Uy^9Zz9J`VYjQ-7w0D={o!uS0%T zRtR3>@#Dt_cq8ZU4<9>q;huiuWcI^qLrw{ah>9nNI|l}wsz*7_`0Gvls(oeJ9745h z<-JFbNn4pEctK8}Y<_li z!3nB^=AN^$@=vBwm-aQJJsWHrhYf2SGFQ4qXR$0dg$Qq+Yd7COKzOTVfBZPL7acpd zfB+p@{=k0pHl%UvQF)a*k!MiOjC(76{`>|AmfgTTF}B0skuVq(6yO`9L&POAH^2Iu zO{r!nS-K^(E9uz2phP!56as{Xum1UT+>lL9@EF3(yV|^-E15=&q)_pRZpak4lJ0@V zi@xao@<#_snC8Gvh~pj+tQLA!N~FzoVVvEc?TC@%Ku1Rr><}Np%F80BrP$>zMQ{?I ztv_%(`rcZMkH4s(a7spI4GbRde7tL+F?C3`B@ws?n1_O`z$(6Zk$_O)z|v=5QX%UT)A>3RQ$!&%81u* z-*$F)-wp_%%X6GEKtP1Yzb4PomO>aL{94y>F{CvLQuSh##jgnmvfm!nw{p53&BDS0 zNI^UV8Qj?!^aYqKzKEO+<0|i9cnRO2K2mr5!i_+$A-1> zjJTj#tiY+`J!nn5yV%^^ z@Qx0h$>Fxjs;CndR#u$C!Xe4(Z>iN@y_t8y(|5DBo{ALGE?N%2_Nlx)GgA($74Xa{ z4b;PkM|r(1xbA@=AuH7!^2;9!b>fB@;E4Y2ucvcczfCw5_orD9w}hz1HqteGiiujN zglwD#Ti90|algh!7FdLjREUH0#RTZySZd*HnZ z@8KkMFcMBd%!?Ofb7*vQ^vL))rkK0q(0aoi5RsV3&&$UritJ3pR(z`yu{G*rQyn*# zt$HYKzh7Hh8z;KvjkPhSv<^=5rcIl)_CJC=aNm?Mb-MM>@Q=a4axzu^-J6Sf0ED69 z@*!Qa`N@Htjnso&TwDeAPdJvv@v5s9Le6Ts2frVwARyLipw)PIx`B8%)YFSBe z(UJ?^kt0uXEPHO03)OF6W%UaU-Avv!w${Va zbcmCa^SruxInwbvFqg6zvA;J*()QRfsz}!bQGDzjVq(na&YdIOGr+r8>dG!9BU1zk z0(E3nf|9`8+?)y3>GrlB`0TrZ;;DZC;+;Hm=8}ennWKh+f{d)J>-gNQThtT)puICr zMn=4f+1K|pQdYf=4xVJgmntco`GuVSiYfgZX?I2RILXyzBl<}CNj;?p7~93i=L?fi2vE=l28I(jjKCt(xNsw)LM3T>w#Vbc(_tGC;24D$Lj#(lax+QWNPX@ARqE+1O)}j(p?tO z8y>m2;uv(a459NR;(|cV87v=O^l^2!JarEKF2RqfT9!1>V9QB&L8$+yWx~ z7?LejLl(72Y<#>Aa6&cSFlbw173!;sde0b8T2{8!4af``>^l z{L$5Q>e{v4{{H^N!bBk_iXK!B(6wQ~f8EQF{=U=Lq-YnLBv4TtZF1us|=_xa5*avsd z34MJo^|v-Lsj2t88F;AcOqVFF=BbYeKwoed%7BTg`X_Mvy#S%otos!mriSG3moB|H z+y0}_9cnlgUfX-XF>`Zst>N!^2d{q*hZLfitk$^8@?Y{<9yQvxI7CJ6qbp}wy87-f z#mN+Nu9zJwa6a~;f-ykG^7|V45*ezf;Fyh#u!V&Mi1jKp`&4EA-@j^oBaS$d zE`8hvI_hQ11eyaHO}cTgOT)ljg@5ji%^+{!vxSmR z>0gLhhQEv$Hvj@Gap{91BK)Iwp%)7E#e2NUt;po4e+OFM)-zpZ->-LP&j>BaefSwh zYuCMf^JWZ6fky4arcC1!(!?R@DuGYV&BN0Z$kld5%W%;k9f2hmO$T%R7_d9UUE*CSd zd2ltdc6K7IZ*6%{7Xmk^a@{g(eS@-H`)%^uw}Wq!HA-?Z19~jy~nP zYwunHk(P+W?|EUQ?%uq)1c<%ijWr+c^iS|_ocLW=LWq)qDuu)y8%EZh92}#QlLjbQ z#Q*d@+r-Ft0fh_-mMw?O)Si9O{I3ct=AtJQ$>VpPDEf^VJX{%ocrRY0Pv^(g@N;*6 z-*$wvT{Ai>iwPvj-_p!azUClm4pP6P1RX$_L~;NiKM^KuddJ@%^H~JR5CdS3A=}xq zPoI8Fk9L9x0G^_q9BlSVNfE$luQW~GKbS`WW*fyAsGZzdwY9Q z#Df^BOVO;v9l#;-Xtjrkl3+S9D~e$0J2)Jy{r_A+qvGALb?ddm-)n1wQS1}M zIn??#4%m15cDF3k<}s8aOPh`MCg00P5j zgaR-usfR02(ZVtvLYh^;WKh*{x+AB;12)$aKUCkfeC0~|ty__ly_1d?ULPGD-Ertp zD9*ci_m{0s&dxcI61Pe?y&l|hC8ROe{s0*f1@;A>f1MjKR)Z_M|M)REBLnoSD=co@ z=zxaw473h@N9W++`gh=+WzU{n2Ux~+;6M;Wm89x!fqyXQcxT|jv_u(Hbk$(vRl~)s zePEZRcEh(GhqYuM-lAmf@YfjGMU`m^XHr(1Ks1}r)Ou; zu`PqT9N_Tv>(>W=2{E3UY0cXUi6M~lf|}ahk2-~oja#o@A1>gEi$Q~I7s@Shd#|d>{luex>&3>#mPMz-%EB@r(Nf<(VZ(m(#q;MQc)S^ia}a6C zs+}Sanu46r{>b|%l@GN!y+^f(4ocj-j=sK9#GE^K?il{=pI1fSS$sl*z)UzcM+B|N z-NUrjxGqt<>vQn)=g;@)d{_;Q&rM}ttz;vPgJ#|% zZU6%Liwll_CI@F@XAwT9_n>OP5xZe&IjC4JcK9#@J^gXS1u!bw$oQbHM$?_a#{;Z@ zj|91pX3=?C*rb7?Jf(lV=EeGFr~Nvxhl4{zWIE+x#E5{BqGB=f0*M8tG1KGY<5yC( zsDJ^E#nx13U)RKtuqau@h<%UF&OrkslpEYgiyxmY+Qua%b-`4HCV6)SR$&QaN?u!bU^}k;8;?IE(xU&n)*5hu?-@1^*q1TUD=4u2^CK zQTVH`a4Y^!y?xJ~r0PFnQ2$_N7iVE%*&=N6&~&%~)xJOL;bJ^__bYKFl|1*5>YKgV z9>2I?^yNc+sl=8zLj$Y8!Yr?t_;{2&+KR^axsJ?S^`Z3VeeKh`DPXLJJJUl#lSBkav zDVM}wd6{s)!bSKA3L~Jkl8+x(T)lb~_6FZZlj7>?jk*eZnHtxox=g>=wI(w&Q|Vw9 zK6JY6um-+}9Z}a|jA_iVUZqSD(U;pVxy52!R_-&-d?}j!aEG zLtQ8$CiXSQ^6D?+gDRP3$E1`gT5>j`Q&C+pfv1d&jiqH|5Gg}&;VvjDo(7!M*VlK; z{MpxagF*kw3$V*TOSg&Q0`otHlFnuTy?J_iURF}duC&2M-?14b6b< z$tp0DlDgwOqv*Thw*x-BLGvZX)L`$0BgC8O`BF}sL2 z;0d4Bp&9Fi_$e!3E#eSCK%3*L2iLHaa@tdP{w5qZPU+1nk`_u;DTa@Wu_40S-@h9c z_RLIt8fj@7e9+$C^k(WABfP%b1qCdokKFI$5;}6dQ%lDrR6)s8f&OgffUcaGn zKXai0GPUn^jpjmqzTt(Y=@ zzf3dIHgId%3n?qXFe#}dZD1_aE9N_Ab|H}OdQ7XJVA{4eUWZDW`g29WzLk0F0%<82 zM9*LL>RR8q$WJO_{&tTZKHQ?FuKq|+cO}Tok9zwBZ~T-E#)-6WNj;5{{7h-jA|MK+MGJ70)z(zD(ZwAD^1yzCp5RI5UQJ4@sA@*eGTsipM z2)hm%V#sXnkE!n$+_KK<*EjDKx+VK3OL;_<6^lVCeX97RH5|xgXmH?SnDBbmI9?Sa z-iyjxaLt~-c!9LBc|QZg?!zuDyZ3G2p`()z7tv1Q*4p3R`$J1s&-?CO&VAb{^)<15 zHl2R!SdKh?`Er+8-h?7)3nE1nH0o7kx;Ej?+^Ci{fc%Y&aJ^)h*k(`UAdlkb*O-}n zSHQdp8j4I!gsMzRVNgS$=|P`&LZ8+{C4y+DdBK%C$a{}!)(E7wG*FMKnjo#n%=}!K z_H=?X|0`1oH$d~RQq{Y1`oaUC9i>DZ47#&! zbAo{RJ$2AzM0qf2m|bR8A@SN_rYh0;0BSMR6hDtvQ@@zbyZh{?G|EEopn7^SX%3Y< zcJZb&7gT{mu*SIJ*r|yM{gXYjcbGSO0&u4DXK?ih-4@ck>t0bVT(O(3^`@RhH=yj! zBS%FMQb*2>)RF?)~Rw@=W#Psqg-hnE=jZqd~mD4+kdEfoGRR?Ym2@&*eUDigfh42;1t=_{;e^kKyefGWVVwd3W)n=VF+o-%RBV<;?qR>3+p1Vd0k6D=K zvVH~bf`OaauJ2`Dc09{Nd){@thYk<{1|XG)j0Y@@o9rVgZtHfRO1zIH?50wrIlZwl7S!b`YlIfw*IG(J*qDytnOS{oENxx^XAfq z7{GSpwQzD9Lxgg7hLGR3iRa5L-lDoeaygMui+bCy?Ro+{t;bH^2acdPj!Jq zW@gDAC0ib`9azsUcK*VJLVT9S@qRY{&1O5R(l|W-r~~WbmyL|v_&DI~v(YpqO18le zi8WEvpO!N?si@52VEa&>X=kI;AM2I3=S%iZM{0(hywRs}+e4xpCAEx#0waQaV(T9c zTnRNeD(xH{7uy}l*w3+n&vPI7xr*gJ>_q#Dlu|?GB?x|I_`S5g`Go>_1dqs|7 zn7970)Nl2I)U^x7Gsac%m#X?X#l^*!zyO#A=__JxyV*|wGxMB__|Vq27~iu1!C)M6 zQ{cmU6%)GG^;%wAknN3^Lk#&{QX=Y}Sl`WSV~$FRyEi>!xmD@v_3txdOuu_R3i%HS za-5RV47ThhF^8|nuR5)*@15w)Kd+sB|ezl1fDpI5x7;vY^M^@Zz%qXp>D6X%s zpUmwV%&tc73gy@2;CXdFm+rFos;$hq9+HX(a4hp{&REcn5u0O-xMVg3Egr5x)_Pb|LK0P!$oC9z) z15GRkj*Z2txv(P1dD{q3r>FZ^ks+qQgd-U97iD=~^kVLV)s=-J{Yf{CPaUv$Wyh%DAN6Xa z*Vp52WF-3@6`>yUcDj}yqTASGnX(puLS-7cIUQ;$961BOosSpKyRn%zeVevfYbBep zLFJ>I-!yO7m7GsdCar3ZtuYFyk(}PhnMifn*4DOZa)cwNAt;2mjZC(Ya|1JQ}h zREL`zH`{uLv1f)z?iEQ&cYZ{3hR2&`(IS0qHxGNNR=k(nKJTi!{Sr6V7tUI1)>^(_ zHd%6gzWPpV%E=z8_191B4`LkKuaF+d$bF<{!=}adBrykd-R%d9^1sZ72rO&Jab{<) zh%6-?jgUeXtMbRue(OqlH|)m1c3_w4wp)vjZ|ncDvN|VDm7M&tEcC?r4P%-0(YXs^ zZ$dTF6)aP8f78D?JnS(}zv@+daG+gFDMhK81)_HE2_JcNS$XLomCSeS2E*fwbXKd~tdB_*wgr{_j`dYxvYu56T* zL?S`X5(VKx_$H(<7;M2kK%?M)B4)jlpZ_k3A_T;R=tpb*N)Jo_uEzQw8{R_wG;NtO zYBImNP^k2Ye6{#ssEa#M&z%r{8NRF8E%wB&J>oi)^OqS$Pf5}bG~G8adtzE>=WFy+ zc~pj4J$l4qP4+}m)cq?tbOI&aBEQrIZ!I<|c}ee6@uC8 zZOJ)c+5M#*EzSn>O^m;*1diS?bN}gTeBx<=1Rd9?G-r>&rzEz_{u6Tc3|8`=E~GSN z@}*_(6rJAb)uLmXdRf;yk5&C`=)GJ=VHU%C&xiW4%&%Nhc|azIpKXy1qn%H-8^Ukbu6lY5sxT>p^Z)$f6F+jK@wfX%>D!|< z;Z=Q6B1-yYJ2}`~_Sb)JcPmV5pB3#&^_=4mHaqd#MAfL{_A2IEQ>Cl;AKu+ioztEv z$!5Hx=fUT}1I&{O6=j!2v+9q(jd0j0)@L#=fPkLWI&pF;%e5@s;n)%Nm0MR|H~s;u zNjqp@o)>LLX@^9;U3ZwRfzHE52^ipxaIlFU8<2y`ObzyLZ9AysRk|QXL??Bp7Kr81RgT* zhmw+@z`!DqM$)pf9OB~PD0WC80!rr0TbsevYu7SJOZ}e%JeTkfPdu&fH+3vnthvI^ z1iT!`NlX}|0nGi`O$PmKDEK#OxKo`M@|<6xS@rpuWMT3XlODduk6uTAe7$Xs#&%e@ zeyT6teOfFt`sEO3dBx$_&kq8kADIhI|LFWi@2gUIzoYAw`LJ*3GwU{Uxl5m3Hm!0C z)!l-oy60|2m#9i)6l)HQu(4R1{T^x!10($=V6|R*?j9v)I(h~MZ$CdSSVKS_+~Vuz zu$Nmd?e+8O`qh^ry#mL00~hykUZ1r$I@Mh5ll`neqc~Z;T51>9d*bBe z#2mveSA&0p=FkYUDUcWFGEXpp{w$)9kEmfAIC=W?*JSm>P-eFraX1JHoSWFl@|}v5 zfHrpT-W@JtaSIYE{z4JLJI-_zKru1X0CJC_AF>(+q-19Bz5k0TBt8F`w#1%P?4|k< zsaB+-pn%5~ACDQ}gdb~ENZS>EzfQyb9@h~c1B0nY0ewF#m$_6Vnf9nCO)vWRRs`*e z>>v13p4j9Spew>PvDL%z;DZl`60CpLbL%P^U+Wa22KY$S6VmY4UY?_pEplFHZ7i=1 zPWLSpOPa!DR@0t(6&X>rS9i=7vfE!~tdxB_)mz+Tx)^gM!0fl-p<7?Rd`aMyg!mz^ zrL`Sbv8X}i%K?R7lNw`M6^GrT2}%DvV}H#0TsEK4#fdU!i@1>aXjZp#@h zyzO{V;l*f|Z)VF>{8}@Ox@8G`2W=Mh3Vtu+bLdu74|?< ztn}8zRnytd>2K(egD|3lS(z;oI6|Nrz3R&4HviB&F>`itl zBD;)~Ei+{AolW-Md+)vf&*QqTKL79Uc00SBor9F)c)gzE@wh)whbdSYFey(@&xL^p zY(GBS?o)j3_?}0%&bsSFUv*_LwkX(LlyER?61yo|CaE@3it%F0VeKtc`%oPRicKyb?~=;_YOYI9qRDMd)c;^0B3OMXFuNOzt-i}=Ku ztn-!OS{M;!JB_EOC-f{yOG}HWs5tK}*H0I%kijv0D>Jhuyu$RE`cra8!3{G<3%AUM zac-khLeAgorgR78#uuj@v3))E!7*_+`un~$@CS6nWX_eI{4g_5b`HoCj=CvOAHw+t zPju=*=%stYX^mYnFBdPJ(7JDth0z!dY|W-v+@y?8Rx}vnmE3i9dZ9}*f4}bI!t@G&`^Qg-7FdTnm*OMY9fw|leaP{V=^}hm{@+Rah3$ck5d_-w) zcOb~L8Z93KyYpGpC2#L#?B(QVsWLMCNuE1;jZ^}~KVQr14pzm&{JJZ9BFqx4OPM_H z^uiK(p3db+F4Y{{YAGm)LdOQG6=atWCvmRn!2Ru=9auFsfKC)N4jfjKA_%Gkp1R$# zv+#%rC1~=X4S~%SB5H*XQPj~1fF&RZb@DAIVnFr&5ex{#)YRu?ZvStgeCk2>Z#7AR zBG&?yP2%32)(0@?1U(MJ(@4)R<6&9n#2ih3d$pK{VlOZSF*eGX4r7(vBBx80^0!xm zbWL!2RljX3DB#%nF(ldqvHm6`Pcziov^VojQNsD_%+}59fZkbi;9O098pL)SBIiEC zr&0OQc*8p#UGk@&LH)fg`#{?2lITpq#A!9=XKzRO&9H12T%W=!1umG1>7$gNKSO@} zfDYOG*FoZ*0lvHmbGR3A=fNQwwWa@mEPVQ3oOK6a;|2;{wynC#&^+9$)W=2IAHZ5Oimph;iAeXDFQ|5JaLVQ_)1NQR(5rGc8!5!EIB%I4 zhn@dR_M2d35~yy+hn-x1->ISDbG@KPc1%dotD@3BS(O~WKKO1l86Whzi+awusLbD2 z?K`OLlv0L|`yDnU9XBfU+2dTdFc<9>8+hqkQ>6mq`UdFg|8V^vCiN+`WwbfjmtLc& z%+61gQixy`J2+n1CuX`Qrjw6zHd>N@JV)AQv_sNSaThc@=?W)h%E(6R+#QH&fP{we zD{MK;hH^h5QZ(}s9?*|iJ2*5nH~SJ$@v}qGi+Bt99af!&V75@OrvY0z=}`Hy8G9#>l(EiR{37GS~kJoC4cpLISg<5bNJOI?n`Q3RbzaTKcl9Pl@0}pYFatk)=O$jzvS;zN-HjD z4m;uJ6)f~o){Sx8jDA&JceU+XRHE`t+r=B~xF}eFFljz1@|2-$y6~e$bEK8&51YPG z5c4NAwyPDsYevfJKSv`a1Y#8)E=MEZltb|FW|`Dalnl%%%3!oixpX?}d{T|T2*;zJ zI4@kt-+q|C%te1uv%f(3_DXZ&Wt&#)%pm!4f_c1I)u17&l;q0KqbB6@?Q87*ZEqHf ztS+0(lP+Sf=Wg&W-3{Jd9InDG&`>9s_&91t8R}@Mh5i_Bt1Ng%5z!*J$3PVO6;ZPc zh8^l1$p{t+uKBx)&dv4htl;8=-TkXXcJawAt};Q|@W6z#XQ5@P|U*mPo31u)|bSN!W(VFZ2t5#`J1By7TU&LW%hP{99O$!eCjVvRB1?wzZM)A z9}0H00WK6`LV^3$qwA>Z&fUAdh(=mo-t;Hdu5)Te-S%IB#n&B@XGgD-JW-~bG%L&= z$|T%E)Mt4e!^aA0S>gwiUiegMstL&{i4^RpR$_VDgH_+&c#Izvfe!OmmfvG8E_4({ zC;Uu`TOSX2#ZIf|;U=b6P{4FgHO^%kH)*H!WT5Yb`zps3J$GES;<*~VpGR*=@XuKk zV2^RG0Z-i@SfkZKujLyQB-O(;GLWi}SPZ5yWd95AHmt2Agfn2Eh6p!7qmly;Ei4VA zl7XCn*eU?6K@2h~I?%uRogiAqe}5h7{S$WlWJYl{_(0Z1?wN^%l+?(%UKdM}+G>}wjt$Z@jre9OhqZh<&86DGLy|U zDerO!c;-#8Mfn*{+P=e{lU?ACbpX_)*SM2Qy>ntvD@ZT(>0lDbMvT~AszM|bL z=Aw=;gzXYM-8_nZ`Av}GY;}kU#7y|}6amt&)hbMmj+izoKcCM8W%gmASOqBT4=hZ> zJm2(?A1kVSl@}6SA0|oD5u3Y=g3Q@?kHuSMlO8W$P9uk7NpbP%xv0lu14b3x_xjm! zah{;s%>{Wyq2+{}xq0SV20UwsPzIK_h>}uUTiZ;p>cIBx>L3S?<#-z`Ii#OHMZ|LO zI|1c1EN+nl5pnW?!VQ#5ez20yW%>V}jF)O+{?~Gms-leB+`m7DE(>O7$slAwy;5F% z%qGy?GBX|4>w8C-4lP8tF8D828cBOw&UBdFb{W}clO(w`+Ri(8o|>|EQ!)r9jd1AO zG6npMrk~e1{^4@Dq>=lVvEn7;wHUAFv_QIUQ>MUDZ<5KSu(B>0g8c18OM%KHp=jry zM3#;7hYt&WQ)#KYyw{Al?kDS){Sb|ojcsGPyC#_^-;b{8MNuOCBhRcl7pwi@-dIt- zhiG-b6KT0GwHKTg82>!!I?%+EM*f~#DKcPLF=lPDPSjbJ*k?pn^aIN~&lfxW+MRtK zyJ#WIh`fxHFn$&nZ`z{-UL-A<2e>5w!}7 zBG^R_7yP7BNo7$XvyDyEad}bLQuB-7VSG?(O}D){m0ISQSaxEu(sSnylVydW z;Dr9zZP9ydQe)+0ZANwnqXwCeQTkjj-_GFE)6mRv7??Gqdlozp-ILR8^E6Zm7$@6qT*vHDnsd!q2Te}N+X!w zaq#eJ2L?$0y9B#GVhM7{lY@-~Q1=;LoS%UV1^FOrYYy17(J?T1gB}f$d&9b63*_F7 zjSFm1|L-oi_QK=;e~(YbR%-U?vm?eF5Pd0de>AbhZ6D>2k606)?f5l}EfVQMNNLxv zGEUB%(o%G7jT1|wJ4CV2muI{V4fl(p*pnfna!Ro`Q(dIbl!n632;97As%2j5yp7c; zi7>Lzxtk_m*O-PWckEs5rdcnitWq<Qmt#~D3y0K<5>o6v$B^io^8&x7F_whXq<5~_;+jVme>ufAlvHJj$s9v zm?y^^mj_1Q;D>MMn%CBg-Cj$rwf_=r=8Lglre~B;u1=Sb7Xz)C({VG?jWE*<2k}zc z!z#CLzRNaLz@h@b!s*#sQRX2lSWv<8bi69g0t>G9xsp5olo#W$FyobejUuehmCT=~uuOfssh<>lbp_Y& zJuG)4+nb8Uuq)ElSWt}el+CTKTXE9sNMv0m}>`up*V z;rBOlWL|O);@&XdHKefSz0}=Jhoz!4`2FB+#ryU8c76Q6Mk;HK9eklrl&Eu)8tZNi z_FHP!6HwZ%FYXaI34FxRR|t5eypBe%>AWk&Pl5S2eOO+!-P##@tkEHR3~FKOXEN~Yn> z`Z=I?;LY`)>AMx{)0tSI`&#T2+>grd(MEn!htOYy9kq z%+sgk(|9=gI+=KJKU3(^m`ky#jd9K~mJ=ivElM!Cd2VRtqvYlDf3Yy#6(xW8d#ae* zG*sfync44_m^ZR}fLpt}94$hR#nn9Eu} zD$Biib#&tjznZ2C0XkP;1cVANh&-bDbuU(4t$uo{)1P0K40^5kbwTs9WR=xF?YFMx zDa!uNu%ED#;IvJDNs*hB5-?yj;(+=VcEZGK)YEmrK>3Okd?eDPekz?uM@RW)L*4u8 zGVauHO4`K}uLUXvR-tke3vO2*9hi5p+v$+fmJ~kDXovaOiFv-DCC~S+@3soI?&72z zRCBW&rq$xPBe?KXHgR zV!0$EIbbMs^N|&FsWmn4K*2r%cuiJT%->kcZhCtsb6G<#0-!dkDN^x z;g_j?`yDwoFO}JMdz({*z=EqQw<2(y4j*(zLri(=6m-NfR#>#W&*JIj#d1XK*eY19 zCx47`g>}1zo%S#X%5}c}iqYs^aB8$EO#74p4&&l9v`72Qpr_gKBSt}cEdBCjf_+v= zNz;HHp}TwGLKuyQc%ZMS()IjNZ96z8xw%t6lO}%WsB`#H>MnSANXu;*m}`73w5X_n z$n!e|Q9nI;OUti}qngrMj~aA1Ws^Yd#*B_y?$3O#nmsK$@U3w169ws-Prk6Qad(5JA zz9)Q>VVwwMS^Bh1ZLVdnrx*)5vRh3*1-a*m{8kiwcge9toaZdsOjxO0EfGsF5%;(1 z8~3?v3&P7gvY*u|EMOTcfB$T}bPdDPTT*pS_YsvC6Yu=hDZase_tIrl?yN|v*1*pD z3zxJ9q(n>H*I!4I>a#AnTSy#)F)=gey8Ht8OoBo03|oBIy3~ZJhFD^9!k>wttT|=+ zgMFBc`j(Np6|sp9Ts`&o^LshaX#R$0RRHRPt^$v;_|NGxsVU=P!umy--JVfp*L$7w zr|T9=-9CnHUbQJBeaeSH?Pb5~53#O{qb^}%+k%f4?AAwRw!#sp zt0W|b(Pd~B?rrWPJ{T+Q#c!X$5n8->dbZN`>L=6aS-E{B$sKYE4|bwKZKG=Z{v!}N z{$OM@*KZq|bTbcYNTzw}g#83QY`3b$eeaEgZ^Kk31IvvSC+~^%rpf}Nk)wr?f)?>S z)dx}s>No9+=~}=DUA5YkWH1kvBzk>*Y41uN#$O&`H_xC&@*V9}kz2uJzt! zzZu*iKlxkzxzI7rnA?HI2`az#@mD@uM0!9i^kcp%kL^IOd8Zr?&m%nfO@tXSE~@r| z7xuOYxZ%H5Xk!o!BW%QAaS4x|wRH|d$w;B5fNEAIyO#B(7-qygocab6#@uf3RI7#y(ZNEpTkv8?B8=r zjJi)!eazd*6}IcA)2q?4Naj?OSs|Ae*KOw=PJL!>cqDK*juLTyMpSSA#%}k=ry;ZC z?nrrE74o{WDjbo35*!K6nWawugwih`9`5C0?%d<$K|0p0&C%gDQi3Io5 z+Eu^L5{RtShD#xjN2fiAZ9Nc=+veFd7S-rK|Vy1LATu_W@#o?yM z5E<^uK8EI7N~XOg1-5t+3sC}cFXveWyEAvR6@^qj;rAwzc%=l)UqVJHo*W7Y2taS@ zEHOe03Ty5CqeChq8@aorcQ8?u!!PGMRXM-t6=GeOGILkD=G@poeS0^P>2`8>7#ySc zwBs5UWi&k(iI7Kp_@pX77m4@SvWTJKPjh<{ctClx`@7-LWSn6g-KJsWaW$Q7@VQ@k zK3>otc7ZFvdZnZujnom>s_0Itb{cq8^m)FsxhGUo+t*>i;4{JXFrIreu%N9f^Mi1l z#qcEyi>9+<4Yl1OGZ}k(Zg4s~p~yxGZ&mD*&(7T{up~_PD|O{l%utQ|>=)b+IHzs$ z(P8?M=3{heqSEFsYu<_Frw>ZY)*qX6CA@H*aQ}RsS@gU7h+$inZOUz5J)%l~?`)1s z;v>9;$?+)Y{&@z8whbyu6x47SF2O&my_r5Fsk% z`uaKE6IGQkX#A0n%F3FD^Ae0#J|I7~ovcRt5giRBX(Kpaz|HgWvvcpWO^IDLtMrGVS3C^WPp5KM52SqpPrq zU_I0`z01gBOCo{&e#ma#ZP+o8{`<2lZy;sf&**kGPY8R_`fM+K=db}Slo?=*xpC=Z zSz@ZugL?;Ebes1_m@n{@?tiqu3)84g#~iFu+jN2#il(^sD}iNV!RBT#JFUAjfg<-y zI9(F_)Hiy^SYo8J;4MLO#z~2QXVez1Nx#}n$sg#-ELFN*S?y0*sNDv0-f{wPAUQw( zgSl-&7uRK*W9+7}6qxo4jC_1=*g-pMYV&4lT@Ji+<8ms-N2VxTzEf&;wfOn-Pny54 z)fJG@3TS3WA5%NoiP)HWIZu<=ja8XCnYR#3DIlFqdMlRSUvL~^6we+QIVU_Gb>KXx zu(1}Fa?lv8Z2P0b_HbtU{)=Bdkrz){=kC}WvtGzP-(P_UObpHy)q4oUWI5sKNOqDe3K&nEvwZ^4(pSK+W~3bqVg@JUmXI;zN&# zk$&B}ok78S`{?MF{YIc+Gm+mFtQdgx-k=hCYHZ9TRdq_Vv1yf0C16H^(QCg_Jw(RJ z`DL)seCTMs46%jxn`7>;=3rgWI5$~543E%#?AXdGZ9Y8Mkm7oDr!*y2Qm2UEY^MVw z&MeZI`#|lPPFxC1CbNrz)M>Zha zf(lkil{~ZR#VmFn-5%*CB|#%h}Tcz&T6?5nl+b+;0mBQ9tVe%{)lUK`mB; z@LIqijqro8>grXr;VC{3xGs@%R+XM!_02dlojE;S*II9(Dc7}cgX5J(JbO9Nc35w2Ee8b_OuwS{Dr#&r>owXG-sDiz) zN5|W&`CHTte|+n3@Gl#JOc%GUgcQ6kEa@QzcQTe%9XXds%3v@13C#YW=N)v>XDoPv zdu{DI$K3pa8L-GWzo=GXNL6673jP{^X+w%!=AVwV0-epOe#MrqX%eo3#gB;=jU*jC zK<*eF9sN99Fs+mNW~b9W#%Ffu>Q{xlO3`+`pcjX@!FEj8jkkQpTZYvGMl5!cVONNZ zttLZ<9@-t@tkw5F^qzP>+t=RjL*z)<&_p;`ZiJAAVNK#?@~RKJj?DGUBlr=20_9Dg z48EVW|j zc{Os`eM6I22Ak!B>zB`@9yrZE4ZK0RGbKgC;W5Gk@|KqFo~`CK zarcqZHLt?ggqkWO!%IE%SEyQYFU})?&bf(%Nb=HuC^#7Z?0PWfV4wiLWKcgQ?Vx+L zMCc`yEl@MDNeQ@-?>hGqGfHdc&8IJnE1Q^@M4ggPW7ti_#+8;vA!=`+GyDecC0Jeq z!HZ|c1H-g8L%Darsx}3jlxYxhG{V@dLN^Fz(`KPj>FM+g48!QBU{?b7R!cvxND#H) zO?G`049gP$006b|;Bh7UB0WqR)KW7t=Ua07)ZO54|I(SSAYb>#M*dS|DXWOm$W3w75H7THNzT4u)cxGhW1Em_;FD<+1 zk@27KTmkw(uP^fd78(-_|JK1YC~hNk5$Hw1B^uB@00uig2O8!u3R~%W6EdV1z7JO@ z_P2!;%l%;q0W!3-xe**tLwfxUgTZmx&O#T*A%Ewuj5h?=S(3D>Gb!7fswd-S)SKmR za@IOI9zB)ZtMba9wgV6gqa8tKmJ7Q=w1M#qjltz}j3kxS+P+Dt4+GCd68ifG#%*lMiEUmS$~L`A3~|Mdqx|Nb z)$=IWE>7e5z8;~4d8fS-VUZX1yV#|^jDd~6XQRw(m#biV`!l?mtt_r=a{nje-Bopq zm%9~mAEY$AWLK&=Ir%3?kJU^wrun>FZR-J_ zb>i_20HV}^VGQ2$oj*DdBg7xfX4JR6``W@x%jjjN6ZUD;&_&y|V%jar6uvNe>egX{xBL_#<{pZPZvvIIhyH1+uvGdI8q05EGG42=! zAu29Q+qbt;P;sQtSSl#N(5Snm&TG~W5833lwLETZNl;eVsycTDElQk}ot!v+z&0xz?T`PcZ~)t|MFgU7=eBsJpPD$1n{LI@MfAd8b?VY z*dhX|Y0##Ez4f`a_EWIg00;uELipoJ%g(NP#eQZ31sZtmu;O5dQ^*<*;0*A}{-zoL z%NFz)tA8`IvloDf;9w5AItO2LQgGenomH{47MGR|GaK>o^AEq@gP|)Z`9@B})YQ~Q z9(F8s)m2U_L$A#?j4q?1vV1?aMOjPBG~%n(i#vd*VTMr>YB1OEbEf71?0-|!(nLe) z%gnX@wl@Pjbl9k7@0}xj$Ix4;G4?H zrnQN$j~HiqyTqRxu63U_7mDXOs7gvoPJf?MeKmPq`gE`6)sy(aQ1M|4;*%iPn#6;@ zw*^_FYinOoyRK5JnjaB)iOP%XTb8}@QX&yZ<00*?S#Y)sMN3qa(5%1z13%_NsXfLo zv>Hxssk9>H$vE4@pHd!Ln_EFxI0fZ>so!D^iW#UVT{}!Ka5If5LLJ%YtKRj$^326p zJU+b=8Urr4+}zy3?K&=o_Cb-I&WGV`>t5+QfAp>5q=KmP_x~<*mA!Owu||WG0Y;&9 zb_s1{ANq`sj2=j6}RKvBe5O5Q(Zw`p~(;5BiyRFPm*fGB_y!|9BC8s_At6KWuNvc zo9tGNzni?l%c-wV0L+;rg9YPpryU9YzM!Op&JdCb`mF{Jm+kFeyo>#Lg5aw%s`n$Y zTOa*qbZ)-X#rs~TwQZEM)wlQ3<@S+9oPghY6`HP+hBF6}uc$`P-g8>S^lzPQ7a-E9 zFM19Qlc=GgA#mC1Zn?I$wr*{2=L)I{on9RiJfO8-9qioS8|bGkPo!(fHEhx;rInCO zAh{j~8mX)-R|*PQ8xv+d$l5sk%$93C#_y+^^@okY`z>_zN?>suI2aYvSok{%4GE0V zo+~QWx?x~-p01aL)Yb+kB#>v7*xXT8qm%z`WU3UwCWS0dRqo~Q+R+OO|5pF~=!pIp zSk{a8vbDF$zjtqB7MijopnGNF1`CyC>*Cv<-WMM4@O%gPilN%fI@KOM7j1I}G z0oNgzIr{ql=2!)CNS8)n%M7DCBV;sx>Gf zCm!+e@eS>}CnqEzKGBiAGcckJ3d+IEdhvK+-X)Nw)xpD3XMNBXai#%d1W+NK+|9?@ zik|f^;}a5UdCqN{3Bdz{Dx5sC+}hn0&mI6MK>)z%EA9i|M_E-BI2KZ`UeW8HIX-~# zFp@(9>^tp3y?-KxbF%+c%Ej#LKhsSTu`;4w_URcLr>1Z zv#!v%YFaSTOUbD`l5^($v5ApL{JqH12n_sRDe~@S$|pu2NK@KnMfUN_g z&>tzz%^-+Rh)d{fF{q{~6xgvnJfxEn`ly;u!hM9f zTc;#a=$^_4M*OUmcNm43g%TYSg2j|e61313OaOdA5(&h_55j*yW@ zK2c{Fr34%3{e8*+1uIJk6O6DKQ6k0uA(z5*y8SbRn(m7DUVf$hV9uzxMf&3n1uSj1 z^tbCJMUjw8|E}e_YSwCv;KA6FA~~eT@c1A1YlxxcTYt?hEoUFYWK zbyc>dU}B7x`F6No55na9N34~pEqv3VGFhZTgE^60*wauD$T!*Xr zqPGCD4J1@9E32HfGh4tfeAHeEX=!FM1|H%sUc|F1Olpf<902bz6yD?>;O`(LeK1NR zOd?>g2q477Fc0bOw0}43zXc+iOlZ_EbP**W2A;H&%#b8&r?-F1-(8fqc!nOxPk8#{ zc52R78?e`nVXRk+I!L7SY@PYZGiC8V`)&KW_Ho@7{(xA#Z<11uKU~MH5t+A{m_EU@ z$vNuwF@o+Wm-wVy(t^+Yfjdozx*@$tN4Aa&@;zrfjjZlPSrC0LC4t;e`L z_c%m*Q;ka6v`WpEI@%QLT#tcSL;w5hXAIlGa*ij$yNj-mkm$A}{Bl^suL1F{nQ~p!T ziz2B`x=EEQxx{-luuO;vlxx_ypPa*Cdzkoqco6aFlM-F zg|u95v~6UgGN|723d%bjBlWK3I~NL}ixVU!&%eBmU)d5w zJpmoFLU#OIwiv7Mhx^mGUz5JUUelC!-s{8xz6_7-ay#dQ+GES*H$$Qwow)szF7Dw^ zR*g8!`r;mW!;LGVESh}Gb?>vL~*I{vtCp*+xto*}@2frN z4_&kTmwz(Fx_JXXub&mbCQ^8K=!w@1YtT(MFPa%eQxbKaUoA_Kli+>(V7bywT1ZHG zZTQ8}Bt{Gkx8FR3g4^`7z<9;V-d;jhTIAW2+P&R3<1Pm}<$!7e3h?tHGtrSE!n4b4 zgAH+WbMaJs@7LGMGJ|AgyX0cT?=k&W#d(lRegwDCCOv#$#R7wO zc*_u^CKQE8-KCsPSS3%^Bur&w}Jc834I|1-h_ClNZ|ahFgw)yKVd#i(pe8N3RU)C z5gEXjEnf|Z5o<~x4N5V<84qmdVD6*8&YGAw5l}3kt`-$RktdTeD3RY`fAbaN_4cQ# zb7-%cMPy^gAI3f-LNRgOc^Rw&6ZBgaI%qe!jIYLpXjbCOHyR;5)#TH5g(u5BpLVGO zz06o^v@O%(@`b+e%iir0GU$vhdO4A*oFi~x@WUQ|po|E3Es)A~q92%clY5DbzGGeb z-2g2gX5ydmz)`bDp&nv)&{g#2veE;uE1wO6X0vW@i^1aU*vB%?f1^HGQD<5a zr)`;9;jTMM>}OAc@{SERhA7bFKx|@qnI>?a6d_c!KNhwPW6i4|!F>y&F|h;9{i~Bu}$9 zEqf6W_VucrZ&=#}jrOu*`sRhte~3KKfk7-G0fsG1pydFUS_&MiP&?Ju)nNe26OL_M zz(NBg^=ZtbZ@|^l$@N3t4}h8h3JzADP>Im}cWa9LLsJtl!|{=id-MpHt%<f2Sg!wpn*;hxVUXlpdkSjk(mL;pKBsRR@RMZ7IlwA2 z4JXTM&g83|yt-1k&^Ld3+ms0Wv0i00QN|#rqP(jcDEFX~b<*Kg+EgsZXQOH3ttYPj z6`lra_Ei|@=t#AhsacTA6WPjPx$?fU?!-6>vpIvJC0?MqhNfiCz`+q_%FerjH(u}^ z-#nJH6%qO_9(PxLKM>uksK6QJsTgFlYfbM(T6+Jr*FhAS;lXpeGu{zr7260}UJC3` zf7MdL=1lzFq(qgJIMbagEnQ2C{BEy=eV4y5HRmy<5+?VNWX&FDRy?=uzRu(RI>}y2 zkZz+8YocB9@bpNt^a@Ys#j&@(8yKOYTc3??{o-uU3<%p2pGA5cw@F)FrgD?yHp_b+nYq1YTJY%X+lWNX1?LiT z>%6+heMHa9$%Ce)Mgq_(ep~N^S4$a^oX9XqUqmohu3l zYhrN9w;W7(b$vS54z1=>QAsr&cwZ(^icMoytn2zHoW{L+bKJ#hz3#A-|0|)e_{IK6 z;Y=^9Rwo?h)SOr)j+SLveRpKzvI1Sd_yW&Md+YQ9UmVlZL-<(_1e@Fpkc62VZ0_g) zpbUG{9FGfkWMywaV30QhxY{Jp^^u;*sj2k!zXoO;3=j&>VPho) ze53zPhM~(wLK|QWf+(**um%{ig^VXRAfMr|NC42p(E5WB=(LL9oaUjGCy?boGqYB~ zwEs(NY|!V=t(NDkR(FAnWB`yN#4yb}?&Jj+8Qv)-fPMaF#(tRhuiaLTOUNNY!LQi0 z@_m26(d3Bdz0QbjuGu{**KF)xGH53|RrIyC?3Hq?XunhUlD?Y%5q0<(eZYi*VOwGZ z6^>0np$ZO){Vq2cVdLK{!?()$=X){C366xfVoN)_eq-SIt{o_O5w_F3&Dsi8b!u+lPj(v&r z3rEkKt02L)Y4z#)%OYm6!vb$fuz(v*?iny;vu<7)NU}Z7$A;ZVNkW=#I*WrwygjBz znXWTdP?4@!PBnIrGI3u|btf+VKtZJb8L#BJ>t8a1=ms^mfvG9RYh4mS@^Mdh3h`?S zN=*VRbu)f~O#n*@myr}V@7hy_PpdC!K4=M6QC~bCx__wmHDi0v)V}SsTS%Soxx%ll zy^MnVWkqb+l4aE4(Sc}dhm=kQIVB}I`k7~`Dd}>JS3e1iUM~8z7GqtB9tkSU|CKlx zG)T=*jgRItct%<5xxJJ!IrZY+0%6?T{7di@T+wqyTVKa-{2LpzUNAr|r>fF>M>hC) zSCpE&rP3hr+S;rPOJHCzpSHHl)CJwM9&z+);mGv#Qve_%xETNsTW?N^#@+f>2M#G1 zW3xk600Qp?4B}@%gBt4fzz`cKM0G%lfJPJ>rj!y2*UN+;=K^qJUXhUP2rD+J?19f4 z#88Xx0A&+-K=mL|7Ww?}1S3(e=VNtflYj(S0BQk?R^e(o`uek(mtlb0)t31slN{x#kvXT{Y@&TdNc?S3`C=6IcGXMi zMWw<j9E{`$Z@{(nz{qarzk4Q7SQbBab8VklcarA;-qx*dHK089GdCCUk zz36qDGjT;&!Gc-8uegF(4Fv(M_uxdSQ_jvYlcO5*Az$Rlzjau-{l!^Ra)r#LDsk*+ zrIO>pmJB2F6Rv&_9r+H^E5$Cdv&m`sD??TnrWst7un+o9W6MA%1LgJ4F==dvOr8hx zLMMG1!fvWTiJ!%*R>;wZ-jXr53Z>7Clm1|f^2#CTcUO>CPYSd!pqk`c9L$*^X{V_k zDT{ynQC0oZ_VEgsR&NK98JMaW7%1>shzpyBXFk&?zj=m1=$VpIfLFa%l(;_PbA$Bt z*n>&*7Acx{x&+Fc7Lj7>mH55(8#2ufTw_ zyE;UStT0SmA&#YNKw07Xf6)VBT0vpaGpm!l*4A7gBfiOm^ z4+TuzR)1AEKNsOB2ieHfj-Kn*f3j#H6~lg||Cdf#ICP{u(>NDDfTSJ#^{Y5YzAyGG zmR6RRBSGZ_yzv)62lA&92!|t^6T-9sd3+EN#CZRHAXS+IvEw0oR5B8W0dpEzzyO=~ zp|X0BP$Gc#iGv;DyVak6O0QFm|67Z=DxgYe!3Kh9*za{MJ05Bn_~?lJQCS_E4iTOn)HINr0zlFYOIp^UoElu9K^Cg3Ae} z)GUWlft9=o!dm>o1+MpKyM}GqiTaa#K}`eY&()sQ7g{5Hux7R5KBSe!vJ1t4#Vj!D zR|(to_|9)X^hr>G>e`LEEleHc-4nJPH_OTAmCLE<$++F_+kF2fy{=HW(5plf-?wnf zXA~W)BhQ@16RY(8j%Y4cf5DIX=ife64s?6tq4~a;(jY(1ywI?(t? z1cRCtsgTh`3M@0``rLb1XPpLCx{4y=;(tim2fDgKfa_yB+jI>Ds;tV`g@ z=&2{24>2>?`u`J4z~uqkDiLVyzJ@1&PHUUWk@jHawTmvG677#{z(|97E!&)~RY>c& zgf1kMy3!v{!Hw6_@;u_c1g|}__U}^Jw;O`_%O-Zj474A;H7R7DunV7Yd1GF#?#Y`P zl=vKsO<*>NC(?r~ENrB4KoB-{8alVPrw&lBx<(oxQp=?zs%7h5_)Tzo^jFO}E>~%Q z$P7jch@l(y_Rvep0S)xO?@J^&;3We&Sx%XI`a zj>MKCuM~V8km2p{B>?+B(WQVun+1LujOPJuz69tU=pJCHvITg_O~@A1vCcg%ne}_q zlgvH$`~T{wFaPQ&cJ{tO64ZT_(zBokH?M%6@gs-NmpQG)Nf!*=!2YnQ8+}c{xH)|2 z-P29i;2;NAib;EY{-%G8sDo%<8???fb+BPOs-zjmL^RQ{wbfwgq zwsnwwmr$yVXCmJzVEiEW)fr>@Uqw;ZuAo1c7s zW7n@%8S{car5_!x?~`Rk2bQlTiL9uHdChd%pQ4(}Qr6Tn8K}5Ds6uU-Y!)-6W1`JF z()ffEfU{PTFOGN9!n`#yNcEe4*3P5g%zMfOxfrP*zlWS+zw_c&{}ppiFysM1XI3$r+za&dI86Z4npv zm-oDD%EGtAC$QxH;voKFyLJEGMiDE=wILNLiJXR|>mMUms1TC|v=1Y%A0`THjMewk zJVQQ=JC=f@uTG~!AZL2US#oUVQ{50|Svnp?C8y1C(np^O>w-I&KV;PSiii9iJ=;7f z*zTKO>@4i{Dx3OOj^Ij>!h4ONP1@I{&C%UuVP96j$EOaNxnsLafWrZ6^r$wddpkQ~ z%(ddP*GF-Ida8U~)8;{F#yuyO#I~3K*Aa7R1qFw*d#w@BQLKAY1Se!RQ)r5!wX6;; zPPmblmzy=Fn-1JAFn-s2yjn;qNf$n#hy59IY$m7<^WKlJq6tY}bMhaVhgT^npMnDT zNFZ9a5;pK}@`XI^4YCoT&dWhJig54YooWE<8f35BVPgw|ZzTwAd_R4ng%uj?kAb=% z4#JrKuKT7J7JTGV009jks21Qm4OO}cfXovlfDmf+>Lirj7l5{iCj)u{cr;KjZ9;I~ zuxyBpja9D!RCwo~gv3Np)t4;qz={&7zJ_uPe9Y^sii&u#yoi0KLsPR^A=B+W$e7`C zy%J#K;`$0dL0&71vb93V?~O=e0U(c`LF*k75(3FBa0&haN*?Sx=YUr}1DUteU@$?J zr{Jyw&Kk&^=YgY!1Qx)+wi^thk`%<%kYwNlkzU9#08oE@Ab7(C4Yfsfo=G3GTr__@ zysB+rS4EP;H@CMD7V9I3u6?Fi@dnU>NP;m;j4?JR#d;Qd)98$uL4LikH=wt0LUZ0K zWh-pd%UzdUJOL92?!MrEZ?f^F|J|t4p|8}?L{$1DWsxANWbnoYbd!o`Xps3C%^NpA zdHI;S8l2M9rcqCx3`vgSzbl;NuLh%jY80m40*=C9*ZLrqjdqBgkL&m?@mU(p>4@oMCuG-G)zxQ7-h zZ%OZiqTleYVC#DyU~Gt8!+|PUv+Z|L&!u49U zDd_6Tb&&;h9q@OT8i>7@kPNLI*K2#UnVe_NEhMsycDb^KsDYoUO?8pC;hmxFO0XgD z2kmREbbq*?JRlzPCU1I_701lJeobmWi=*dtzI_G?{DYaY{@0B%Mmc)wZ=ABVDDdTu z_{(97G3rvD>Ty?58)P>8#tn2j){q}AabuH{Bef*9;)5NJ^6B>FQW$udxa-dOr89pW z&?~~a=&&G06h+Z|Uh9P;o!dJKMV3;I9fIn*uGfx|zppZ5d)lv{y1Rp={6&_B`-Y|5 zg<8rs5?uB*l6IIejeQ8cKFf;H*xRd*d(g1bVVt>a)L_csFL(c(TzVt>5L$k?{vIS* zPGm%R7zh+)!u?{gZR&f;b3zT87-wgGGBPqqwE6-MIRu3@x3v71 zJV?Hh4i$HBC|vsiRVp}gp8<(=Edzi+u&M5WNif`|xP*i$AQFPyOCn(JOG`Hizw9tT z&=dAZHWh3j)a(P3-}sYrBUjT4p6^EhG3B#cyt=c{(U_20TvEEZNi4s)NNj1n`iIPY z>`^RqX?eg!hm@lw;P?Igt9CHHY6B4fHIM=r3AqY%lK!+X$AB+<2a+m!(v%ocaLvq^`Lgw^WkPU&joQ&U&2_}G$ z9F_<9by*NP54j}1U>S$s{G-#&I)v4T&?4Zghk(0bNwUU|V3HsM(Jd@DUqIoG#1&kl zle*SZ=Cc;LN5KWfziuF{ zPhz1p7wwCtPl^~0L2dz;o$AIR1lMn<|@LYu&dLrp&2Jo!+f?iHcA<&fGcIbeUQ z&;%=;Z0^s!LeH}6PMMj6nbSC$&G(eJ^=iGLvN&X-;`|C!gS_kce;HvlHH&{^qLlP!gE%*t-sKD&*X;tHUY{x z<=)5v7kH*%*^ESH4r5+?wpHOd*Pk6HjkdNj!O5LBO#e!f-hb5Ed4UDZAlK9sYmfaW zYPAk;1IKKhYt)8ow($DS&{!vwKCQd=Jo&AbX<2%8p`j!&^0CHdJyWvxPW@g6L}xrb zywn71(HRHbswgZ|A>Nn2J_;pL;^5@DcQ8=1%s}Fa)a&2c)^86Xl@#2sH;;Rj9BMq4 zmw!WQw)c+eqA4b3st-Z@cdlm5^$Q;wSnXd5hijAi!KJsUZrh)Mz*y?Qy4@KsUGWQ- z`8r9w>*1|Z#uyR2D`pRQIEh(#NdcLdi4T?Ve7hIt>&}j-k9IXad&_}T#u^%nG48|t zJnP3R;Ab#8jN|}uug7=UikO#|J+XjwgxDJ4MKm&EKwLWzf&{rMgXa}E1O&;j8-&;J z0TBKma0PUA3~&L$Py|$Sd@z;;1;H<^qQI|TuR}hHjJ$j!q>4di5EPTJh`9~_D`XdT zA}h{9GZ`nRVkcHaPmT#-GyNjL)(uF6nT4x!4)lC*+2z9b4Ag&b-yU5d-@!A;qG%8# zc9~vc*cx7tjHwj(kyH0;!;6se$^5xb43c7tBo8CWnFutvZ;x#bl6DLrT@87!Ak>T+ z8dxOd1Y#=98j)}raJY>SC08BIQy|M;Na7P2$iw{x!A*|m$VhLFwBo0KM&2$&4;ho2jqPQu6~C{2nJl>fP-&y=`-o*(fWUgO7aUM}Jl5IC z$ylCHYW#RqquQzef?6jySJlx%w043+_#*ayRwsVao1HS{$ZD`H-eZSEaEC z4;Da; z&U4D`Jgo|@ri@x3WG61GS`AKegO+)`kl_fA#6RO*(N zNa?dS8~+^&8<$>lnl06yzZ4@=vTF@bR8n;eQ~K!dY{tzwl&8Rc_V#10&c*38+}a48+^$ZAJU zmDlp`uAaq`f-BQMX;>fJkp;xYU+e1bR_xN>ohSGp)nn%|r+FK(`Ct1HffCeH**AWd zi@UWj!=a<$8Jc#VOK7TD@p>Wrz_RI(#ltJjAFtmVIDTHKHkj zB9H`M3^_g)IV7-qBjy+mh=>Hk1)=_fiPC*KmeWb9*2CRwuT`qlpUXryyf>eaEekj- zJtNc+0rM}yy@B~5n^n*dJma~QEoIJ#;f+>S`FX^;owHM0CI7Cx&$)xu)(Bor$`}zNeO39QEItb9mS=^pumH?|AoE0Dg)vWj zn@b>2VwvCg8He|q{@9C5e5jRr`{}9UDz~%Jeg+;Lt&!4_x1HCj`JGZX=b)z9L|ZFT z6S~L!_3+0WC#@xx7zd-t+l^7LPp+Xj-FM@B-rcA>j5e3qBVY6L#FgDu5}nN05&e2u zvqQ{$*_(C$R{zf1GgmBS-6nSVPBE8vK&p6o1hy4L!n4C3LMv>Gav3_A?!_XWGB{2AD-SJ1E&M2VI)m zeZ6cUQC*P7HKVwVYgP8b2=`#9e_LK;w7O47_`1*?&f!lbMrwBqE{i;-TQj`0xQjNW ze*{jL5m&VZ{8w95G1#k?#P|jcZkqFco8)oKTB0D(MJ?Nw#{%n&z^yLyOY+rvGkTQz z9h#wBxpl%O)hHT=|GczBhVhoy1+KMJ>-5&soW`CdJCRG5&I(zxKl*q}!Sia=$S*Yg+#m<6tEfGeJNhj=<~_=G#H*CD^)#^lCG#5rK+nr&BxzTPZrN0ypXkSV6KnB%@I3)_RNL- zz)n9RAtVj*YFkZBO)P)FzV9(5?G=z`Qn+4skbBf1qDJ$qVE*1uf9iceN}vpnUi-Rv zxFh!hSj2ek>}e&Z>8}QQdIjA@j|u6p^YFKKe>gP7Z1_4P=)hQzP-c*$A7_>JlcBe4LtjcF*>2bKDz?>C zXKEXL?X&u<`Z{~E>w5R$MW)sW<@nNXZ1nyt_ccspuf}i3qZ6pQc<3Q&u+_qRk0zhF zMBg-j`1#{vE@g}5u6>syQj$XWzQF2?@}>56Ys!@iQ>XQMJeI5MCojiEjxX$R^PkBz z&ClNA7PE6c{t`zyZ3VP|YC?W_J+)!>J5`FLcU`rOlz*xBQ0tk+Xy?6!-cWovTTDkTDa=)uO^Dp*K-L4qrcCl#8mDp4sYt)U_VQN_H zojY8$zja3*nT53b^E>tA^={h8)bjGu(sXP{dmO8mrRKZ?T-9AGo}s&QIO9Fn zH`M02RPD_#pfGi*>UEV4s9+3~P0SeY*qmo8D|Ph=%S;{T>u=RrUzq3<`pZmsN_Uu^ zzoa}q=STME9R3rqi=oPSQ#bjOQ6<h(Z(lW;YqRKrVjdxV4de&rbQ7zP~v_Llx`VBk)mfW3Tt zeAQWjJeue5b-lvnhWn?h9U_dz;QnSgGptPk7sDCl4GjPaWbt})Yb)6cPW1H<;E>`y z858W-X=i8r`Kkm+gXwqS+G1ET1v35&glZ5y8{!Y56(-m@KuPR3&?N~72q<1VQZt~} z+|rzKdqqKmze@f7pt&;bq0)rO!A6!CMbCtUL!|AA=Gd3Iy63p;-Xgb$YYxsVy&+c} z;c>+q?gy-Aw%kcsefLVT3tJ!sMZDTO)G9MF4MN;Uw-q_Lu$Z&&Vb=_lNV;kft=vmX z>rrS?plot>L*jM2`R|NJy^7h~^%I;&-9zu~zwJ`e8MD)n6T5WgX{$<-VjfxV?3ugD zCu#X^l$~{A$CdO|cJ`UiZ0?rHhCZyOjm|V!r*Cjq^H52OqKL1*v3FLMaC&;W+^r?D z!VSphAU8K{j3$+`SkL%iQ`n!!pWkmQJ2t~(9nlog-nrJm)12diiyz;ti-Ul>h^hQG z3$M>fUvkqw^3!_8ox0)mIggS$F+@UlX)0WyN!!QJg6CUWz&%N?AH(NpO@!nd=J`4Y z;za5;-0JE(sd<`LNB3)l&-&aQk39Vuh0NPNwYCP|AD^0HLs&OUY zto@^VEc>~Sq<&?%xnwFs$!S@D)Fqe`&$)QruhWiy;y2(N8xa2!oXvf7M4tm-$0&mxs0T9fZo*FYV=`S&BLv zkKD3(kI!(c#nB)Up^+}}h+sadiBdwfyLT^#}FYMGS&_7PYzry%kw=kXg0hg8a6Gp!{Wm+z4}!=KQHv<$kh$# zEu}whZfSWp5wdAyGW-;V0AYJ)gdr^4K(dwm?UmdzY|NI19FfRB(ZrtX-i<}E)`4L< zSeOo2eGR_j@X+s$yaAex812=^UW!Khnp|PqK{X19Ptn%jkLn&jy$65?S-XiwMA)IQ z@t@_^$j<;REFV{p(sP#Q2e3(L4>8#tXi6qN0~mLl#RjD(L+B+y-FFTB?wJ6=HE*@D z3p!k-5;VDtHe_VizB6mtex|ddJ#!`0&*L(jQH5oFgce3yjKZJ2cmXRk8q9@N+i3s8 z><7$i0}-zypVA)A-qAHU-XaXRdhJ z#i~nCyWJ-nG@KXjz~Rj`dm+~<;`@@9d~?$&^)XHPwuPB`?b)?%r#Je=h@a6L@?y28 za?fNs{j0D?&MRFZKn$7&RxCO^L&5cO3FZ z3(F@NZH&|HuhvZ}hYc}EXBc-Jc-m5bP`BrWQnC2)$Dk@I-#5q8MH%I5q5>Iif0>NG zwC&E^GT+xE*NqeJQ{Gp$Sm=dX=f!KJ1w6X3xb&m42oY*S2)wGmoc}HMF+u6BsyYGK4 zzd6;A;A4pr8j>6ThtJ+^2<=|A2tc{SUiXCY3oL{i`zBsj$f-Fs^ zVX)zikGu5Q?)gd}8wJeuTY$)66mk_)dc~{k{AJF9gC`v(%+pGuwp?Dcs|pvd29{KV z12hU*HrVexc|yyn`sO^AV}O$~8t*M_{{Fr1(2teZ+8H>Imvunnz5AmgK{7Lz&r#H5 zTJl{NSh=6h8`enuUNVKfE7rjab#gk&5<-aV~v%lCNANT%|AOD*E!&*g+ z4dG=18e@7wO5By(?{2lQus}=g47da&Hf$vehsgu6vznvo5c&D1K5i5CM0VpIqPKSP z!zBr_QXu=b#Y=vZ`>?9Ut-+W@WUcX6Y?HvMcC^&7xyUCeiHDt?{Y6laC0dFYBh

N0Vaxrt0)5gzZ)*KmEytXsP!k|qJ4ylY%_qUKu(~3!BQrGyA)7g zE$#YtO4oajRw|CjHk`2*{Y|m_R`iHMNAT}lUMu#gcBxJMIIMKe5rh|n77Q! zW>!#FxQaLUWAP26tINez-FI9{n6NaB@2a6(4)Wa^Y`KvpjhSQH zgE*$R5GQ&H^8;>h5#tMV+h!=kZe5N$rMgWf{2j-ea{cqi@3H3~5nYAU;=&az6%QllMRdvTF#P=ze zWUblVvGRSZQmVs^nYR_)UV42@TH#)C$&Y$kQ%&kBpU{g4d@~OHG&gZnE8EuVw5G4+ zO@*k6BTt>x)_Fu~2)G;`<~}=B%#NyFp>3dJ#rMT|Q7)ISQNsi5-|U7u@(cciA4}*g z@UswauPAP}E-jPoUz+LcM>Q*#7MAGcwGq3&G18ENt}XJ2f9ig8(muQGvN@rgoa^2l z&S3p9+~r(LR>&?@fLr zRjKQs>hp4F_3C&p4zA?G)gSMS?ZY`emvp#!DE#qG zUQFLsh>YZ%E^NPuuu{^uM^Ld$Tl$V)*azA*8%pJ)?^D;iEMGG)q1Ey^(xs_U(5|+I zd3U08HUDT)h;p4`l(5q^9vbJBx$uG&2h;@kepu)9)Zf1dO}|RO2(o3by*(6_Lmei_ zFcG}A_kaafQ>Q{R(=S7jX*a=+C>#e>Cpo01rlw?J6S8=XF+VuNBdGLIc`w)eOf&Du zk;d4FS9NQKMKwTu_z7>I#hN$je5humWo4^UdBMYB&?Ikp-W2NVaHDX=1ni){i$Ovw z1WhwDvu5m3BeMRovPT$+?55v({LbWo?M^v|`noH^mZWZ`WE2$_Qy(nJY3QY667}rR z-5X(etD>?3mSzVphub1$b9aYa(DKZ`AXlM0L-|VX3{TW+>z(Ak)qPVftVK=4 z5gwjDdnSRGi*0Y9Xl{xQbJSX9!1MG}in%BzYYtpq#ZL7RB#u3rhJLNh}$VE)VA^X?4EEt-v@6r?`Sj?7JUvzD^lOwyb4v@ zhK(ByU_5cs3)HD%mX!gBSYq%dUhz_bKHOMr*^qacTB+y$`uPEIz06!Xm8mQaGcH#_ zhNVioyQ@@+Mot-+3@W+w9DL8Yc=1=?a$5w?vx9;3d-ha&O$2-TyXl=aw*=&b2&>NvR(KJ)U53bHYmY^`6H zH;fgjV*|0CZLvDKjeCEl?V(3L`B|3~t@!D@>41W}u8nwlerNK38Pem^-`0H6rO!I5 z$6U{%M^<>x?VKGBwXapUH280nxQD+8B{VjnWX>D@wyT9Uvz$gy&ttutn99r7eZ(T;N$R{c^CI-fB z`f_j0uhXX$HZiR3>q9%n+Pa=yvMKrC2iqINPgpKt?UbzDOlyWTHO>Cg!Us@ zcMOy?kp7`M>zuFD6i8D{P~a1rz8|009lnzDP$Q`rHhnHGE`-IBks-P{C{+c`+rEYf zTVtFohQyL24S#YzUMnBM|G8ynXIFIZM1k-b1lU!>0Hu%lK-5|}cWPgZE*t?4IB?|1 z<|Nt5cX79VNsDK+6(x($U{Z5k2KaHH0r&;Jvln63$Ga%k?`uc`1nG! z15QrugYgywa7f}j5HJ~Ms6v9ki!a3Tgjd$V}(J28 zf@#r(v1CP0#*EJ<@)@dOJxvxZo?dAJVQi0je(9EgPf^dgbM!>0w?JakO=vuOy2PLM zJN#q$3KK8evCLQc_dDeE69gbl8ExW5|>Q<)R{t#H-ZdHt~`NJptnws^f2$Ru-6>78}U0tVQTP{6nvD3fG zC^=! zZa)VPH5!Y8&&>B=kW?RamS%SYhwR2gfx_*KMGXd0553yYH+#CZJ*kV}J9N(MDj#u! zIPuLii^X;zk8g45X==x*NO}3P`t5so9YSP;a0J{?w?)fyJlrU9v3cr3&eHFrhAU29 zAXOeib#-heTg;JfVZCY7jbn80-QC^Cv0k|j3J@Hc@IT)>(fuL&uizg5-3cw@30jq_ zrzu@jzBHc;-YD)8aISo&=eUid(9XkJU8wOl>*5_ATVV!ei4pkp%16)ZkU_f46ydCuNWuH3zvNVD97ZMt|TVo6O^$#+Oe2|EF zAX<{A2L?raxTK@nAha;{0ys+{Nl2~8fvoq7H4b>LrzCb>Rvf6kmui zQL4>AqyK==7n5CT!EfG}!3pPGYNhnB;4a!h(VKo(-yY`hr#>7eUfy+Yk0pF2Utdo4 z_V*VE_hOdAuC+b(d7rhO%l>WXtyh@c=I+t1vK`@53#-;(B6F-t?Oj^hPk^Lq?PHeE z5fgy?Qoz&-li{D&Z~q$PDQew!u`Gb+0(`8%DgQ)-BjD=!^ZPOW>vl#7Dt-TWv{AJY znM5B8jzVpBP^BuW#SbA%eraqBAS>ODy}e&>H}$Nn|7viOb;=j6{W0OpN;{LF3>4rk zX|h*+k9Y3bqqK%5ezs*n&(Oy${2A|bfc=24f7D%CMj{VsDx}|S{ersarEb*w!j6tG z#TXe>v96M#UR$?sFM<{_)8_Y92$Xwl&+HmIx}}iag!F`kh4o%%yM~%rUNNOXQQ=0M=0d-j;Hklqx-?M%nn+cyVb8$*j` zGX^BDe)f*2A9z@vLS%twDc%m7$^!d6%NvrBu3+0=7)vH=623r?&cyS$_WZ z&Clv*ndQ$90V%1fv_T;m5B>zWslQ@lG!{QqnKp3QtJNjs<^%3}n!-4(gJ5&uGydGS z{?5Xb-RxZU=$Fr{V94x>oniWxE~Se`{C4BlFvv+(He%5b!E#6_CvO0%PSE~}Wk2W} z78KOq$7?V8D(;JT`DTu7k3yDOTAl`NU|?u=u)o^ZMR9`Jo$Ipv?ChsCr4sHn^4qGa zUd1cpRnvomhPGVwx`4$8i)yOqa}bczkTFt}$>9t0H27CAX=+1|8qu+)VQW|ejuaVs z!U|SF^VE!tg9i?*6%xvj6s97?9<0<~;q9Ot2Sktn)eX`54~(DUbt$~RCSB%t=rs76 zc=YI1fC9cs;&>4p46G;T t@et|w0h0S2|GosdlFz&xP|1(jq5B_ZVHC2=0 zx}h;Zc*H`9Pq)G+$mV)ftaMUH!O9vV8z#}@9^Z_MK`2#eb{_7<_bon9+MkBk*x|fz zJ>9_L`i_qSx7a2RzFfQezhfSZbOKviP$*a3>}y+B=Eu3=Qy;T~Ne4%bT1!`P zCG25)gl%SzSUbP))=26@orvNj-Epe@+FP9sRTCXtP3Q@5Ao%4kO*7&fFI$&z|0@MJ zb}OzrwVw6SDFvQY^66&kAZRlk=Z@|6imj1x34ZzV7*=_a;oil$38K2Hs`5r-m7AM8 z<^6}g(8it(>(?Kl%e*4p<|b47;=Qfcy^!%J7&sNaU@A~n-?_)bfzS5+EtA2=A=0i} z%a16jGh}q07kj9xL+xMZcPxF%t2Oz+;HxKzYdWUeQ;uZ?^1lP&&~sss$IsL`1emcF zRiBhl&4K84l~pQUH#oFjwq=N}8uIMmsBltH$rpP3^(oVn!}dIX!X$L(t6v7sbAM

79q(~d3O8Z8RH^UoW#2xHpzgB%w`vVI%ZecXAyn0jt0Rgq&G z8L@!$`5lB|WDMfw%_AsWKu2}=^%Z{@IY^Q;8o)2jW_wy`LoRUqu35Q~=C(gLb;G2) zq3E!-4?Cma($nHr{Rz5naqPR-3%@n^bmqdv%5BB4H*?_U-@tt-qhMw$0~~Hl+A=px z2<5nyZ`lc3fzL>CVFNEmgs|=dMf_Q~pCBylKYqM9vl^`{oB;)7vL33m~r<*_84wYFX3TV;Q zPn>n(>*>nBvWYLyP$Btc84A0=Yr5@MHY$0%MDWM5lRc3UH z<1p7#F4V=4u>Unqh`q%4Rx3*qlDi-wiw^69n^hAO-C#+~5H7B^G-L_-5y!CWF>fB} z!ho=biYLrbcNH-)fmD408@JzMW;kFa%@04X^?ch|P8ALdrbQpG$NhtDn-vx7Xa77t z`efJ9`!I93$<-~GJ;U7BmDkR#{k}J{Uw&nx<1#1OUgW-vUH4r}!TNw>^(M{L5h-OV zu))^X*C#`pWKC)0WDw?XI#!=bYFC#6Ia8E8(&}5jaZaaO2BTetQd`PjVN0M3 z_6VO~$$fZ^<;xKTjlLH11lz*W9NOzYb0vDqH?-FL;*0OAWB$7EQh@vDr10Cfqq`LE zE6hJB7n$9p+kLleL1|l+*MwPP_-~Jo67pg9oGD^DZ_lOw_RhWcdVYj-p15zcC)ED% zkj!?M`V9{>yn1J|;Hi8=#jLQG-+3n3BbtwMBs3fw(R^HfFzuUD<=Vilb{6v*tz5r^ zKd|lvLA;zVor4jlC1rBh|0pPfBAuT-qZvQ%D4b|i;+7apK zYxRN_h{$bkqVK-e`rLv_qpSPoh{Z^QroFk8>8@rKDZ^NXzP`Q+wda*JVZtK= z$vXKkUZcg&a1}Yn&0Z88#5Y%RUt)|x?0qFDs8TQ`bju6#^T*%?*V7-`uaKz9)8RCu zO>8Ma<J7xIGBpU_oWz1DUbEub=M|LBi#8MM1- zlB^hdxjnYSCl4lFY%+d#jDztu-&=Hw6?-#Z0MC!kcSMuxhM?*nm}PlI9B{L;Y9V$< z_Ma}C5eGZy$D>5t|_m;6}s(U>^)%9+T2XGMghI8r=>j;DZkm~*RE%` zhwBnAH~?;zqriG7dmep-P|AC0#+~T`I1`T7>{hVCqN1{rW-2%AUlTiORGJ5Z6zr#HP8i6<9Ra4W){^OMRG#JI%%FOH)l*F4W5`d)#C$Qht z3$9qc=sRF0Kq20B<*CyBPVChH$N2Qwv#}aMd^LjVkPP0z>p+ruk?L1N;!aR&klQ_n zqzAEWbaeEqZ^|9zZjur~#|%_?_F9DojbwCMe3LKXoPojVMd*)Eh(71WvZh!005x&} zDn%pYCm_#yaIGL?Wl_6_rSln9>=hAd%`_{(X*upVdjuj%i5SK0(vBASOM+zkBid3Z z6ZwV{N19WdnV}n!NN$R58fba1#}il;zE6;y&PW?*LCc*@@1{#oJwpcKeRFFWQg0SNiJPyQ05o; z&yXVXQwgT5TtTZGvFtw6hha`sh?OAyJs)R#C4R|0ljEPwe_Xw!CQfb-J`=%exJPUp z9Pu27-9CLf)mI%}J=RlV0x%84`MIRTV~_LfJ)AdC(WS^gb6u{RWL|t8oF;(KI753TB=pk`1$~3ymVbIT#9j=m!282? z+*sTeB2h}ySWI zeB+V@6oX+PQF1C!^j83+QL}2`whAIeAe4gWp+ntN+>&a9N}cn=^5r*b1#n0lZ!=C$ zN!fei#BN5$U`Y`le5)~?F4WDa1Y{9H!m@nm3`xIQA4UK7JVMU$NIuqcKHK6EQ9g&4?S4$~eN<54>7j zo})rSE-jG^qH>D!SRx6#%4@I_bldgp-{Fl)Dcwq%CugkaDI!Lj5?KVzb|G|pfnyl1 z!cL8RJY9Z};7bcLBnFY&jF6&#{klec|DA8T`9wnXV6$8@!aI|=bB_G{bq!4~nGM(d z7cb@M-~cv%yu}6EK1{m|`|ngaiZ?7u1|MYt+`@*{NEFMSn~8Q65R(e>)8N` z8hBCK4!4zJ+>@A-J!{Z}}jxBngm)npc?0}?6oWdaA&vH5`^2qR9*Ya9C69j4rOvQOdu>k2J zYEkOk6*=)GKWtb<_lg14HR3$Sq%}TZG@0B(7oZjeT;BX2uJ7qL9*2f%M}|xNL-cN@ zkP#EifdEOWT+p}gAWuTd1ofnt;P|JSm+eBpE6}HMW?36!%mCL0m3-o zY?A6u>9m9Y9`2bR|0;`pGnXki9<8NdWMs4*Y?c6MLNM+F5aNDbAcO~716d-(!bf-V zUO9kdg((PY5N-Q{{du?l?UjC3ioM8yyqV>^bOJ94V-WaZz}T?~3p0>OKFmC>!nD=7 z#0Gf8mxC|77b4M*Pp==2ugE=~$))<{p<+TW%6%U*eMoq-vgY>Bh@3bfHZA%9Mm6-E z`j9e-EA1l^gNc*X=zZby+l9r&#W9A1q}fH!T3=W9Bsrr%cP}gJkA!Lq6BB8aMyp1# z*1oKa8g@-2+d~+PyFx`qtua&u^D?88jh{c?2Z%vfn*7#}P(U8b{{{*Q0c`QAG5J94 zwBfE6LXg2&cM+Oy9>{2XH@iOV6Gl+${fJ~p#Onyu`|%q_1^3s1`XwVA2l^ebzFZnY zCxq5ee}9#{Rlkk=gSTzov&VwzRTV}^#l$+A**Q6>LuILYY0XLs3n?O+2GDP;tk*#k zHca2z?ZE^5%7k{wl5`E=*d|fIrk-ihO=$i?4nlE6K;SGw(w{%q+bA(xUhAl`6@P7X za&oN|WsB&^lPq?0X&PRMiKcl&=r+GVn=HJwrA5!cU|p{xds;?@%bx)|toISN9oz*6 zL13oMm>mxeZdIDu|2H!{xFdYvFHiH(-+w!%+0Ux3+S?RtktYqO{OEIDpR22V0HSxa zWC2awz}bpHd5p7PHk)mP+euQl8{OA(61P|jCywr-C8j~h*Z|bhV4v+E;yM7tGr-pINk`;JO>~$}>Jb zUTWIZ!Xn&a?g225E?glU-zp}I7xm#$UOay}LD}v(ZAfUSSLy1krZlGoTLLVQ2?$sr zRqgAhbN_{g@qdVQ%Y_T&`?Oc@4U~)|M%o1A0=pk2lRFq<-zOReXbPd|6F?<0ZUbQ9 z2IK$=A_gO}Zi23(3M>Iyd;i@fBcWS=l${sHq)5^QX*35lUJ$w?jg5`P>f=zSK`JDE zxla4~_1Bh%?rA{=gt?-=7X0Wh`z;oh_w4B@Z8BVqk!+~n^uaH{d4b8b z;1k@q=Aa#fEcDE?l2P5&r)+I)!xS?}jN>?-q*@N-gp&{pt-H-gM=0LDN$-uBET(8@;Wu@Nf4ZOJ@)<+dv1hdAE6y*cD@<(C2xCR|Qt!C&Ck$qRuoUQY}> zgY_h^Nl)kd+}Q%cC?I*RN&%21KQs9U`K%Z$L9cbpKX{eQA>FhcdQ`NVPFq=x(`e!@ z$Hg54Evk>2k9IfzL%b)VQpLy*IEGK~)5O|7+|m-q0ZVGu0hy3Qz%G?Y^0XAk{Ku**7)3x##YZi-elyL^_S|S%s z`HgGHHa8+!0z)JfF6MN__B?reC{m=Jl7d758%xa2Ly*A3Gvv0(4|ISxS_7oBgQ`~s zRWlL&B9ImfoZQK+rmLl2*V4iOPkjueoCB@-Z*8^*&_{kvrrT!TbJ(>AxbXuc4P;QL zr^JVT`}VVtBLh%UxAhqf0M zhR{IFe;6O!X?PjzMnSwn^smMwMyLpQu1}Tc!&?=K6}0W}=;RZWaaXLC=5GP(5I;<) zpB-9Q^Y{_n&Axx|#K-EwS`mm0*RNlHCpHg|X45Gk5SpWs{{b02`?>HM06ZPejMb;k zoh!Z&bjZ}u@EnBE#FT+pj(|o(UX~{U4Pb$o7&fd1g_q+qJY!3IEchlZf9}~9m4bl_ zSSRY(?y6=(Q3?b3o%XUIuke-*>ExZn3x?RLCf~(HQc@D4@;Jz`X_<~cy?gg=V0vRB z_o}}YK{ke*0f%Y_rcj1J>$%+KWQ(%*m_eC5h&=+Cjl!LZ2X1`aYIR!blDG=%8SR^- z{l1%jJuffsxhJ~_=VAOUehanRYOix&FyxKq5i{O{OQdBkXNSv*6Op1rGI$m?_tt$C zdWaxsFS8)qgEfcS1KAF)Jv(_O4s5Sr?dRqeUqvMt2iA6_6jhodEOkrwQOi-1c-uI+_Mn;Mb8puGH3BHkS#F1zgpMG$OD%j zH6&oQ62hJ1O5hcDDlUqz|L{${rH#!iNe=$gjgM-RGZ0tW#P&OfdaALEJi+NXT8A92G@ z-ar>>Vq;qeEQ~roT^$~S%+KyxNNj8@x~HNO&M4lowvHuh#SUogez9`lvPhrmd5(7X ze1KNT_$ELrC7-_Dx1+9GB*ZahA@1G^95+CVgtU>Z$QUR)$14qaIV>zjr0BxLd2vgT2==rVL-b^1 z)Gsv*{Hz==ADMERKKb~66iGe|`zbc>-mMW)FbZCQjHTitlF$qqjCl3;wEGy{>>Irr zur#$3GiG`itRqIb%X?fTe49F0Pb?mHIjmEX9$F%tX|YCrb^`^-4}DMz(6?e+ zOP^4fLmj9>-+TC{e5q03X{fAwQK5EHs;6Z)-Y?dgHC%sTx~iuj$K<1Hu1;k7*|e19B--R<YhE8nw6-1#IZ zgE*)h$-)vJvH6G46)T{Mr$8&Z3J`z{{NX`~-uV8Zps;W)+_nMbLM%$(8TmhbdI~SR z1rlK(XmDCju~q8idnd34NXgM4Q~^zckoQSi1v^;?>`yZ^aTWF}ps|U5nWXe49T(;# zRVjqG%$q+}3ta!!g>U*QHs&4Iug#35JIR7;@Y>1i)qohXl5_jQgu`%KTS( z-TW^;&72D&+*V;KW*+PUC;({#2t~EI`_NC-Z+2}~7qTcfE`(sMm13MMGf7b`sUvH{ zU9z8<8ubzHef^lXFV?Q%n)bgo(%}Se{`%CW-~(rpdeEdcs(U3(xM2VnkFbMKx~FMq zN8}ISwm1U#aRb2t%0v1_N>Fde+C@;6lGYdT7e&7YVXz25*3!xfb$*_FfKehQztPeD z_Ewt_Ll_veNp2?9fnW68Bs|7RkY2S1tMOt1G5LBh6 z(;7GIsMA+7alL~z93U=Ykgf1h#BH8-c66}dL7wwrsf3PhPtf`r47sQ#X}Fz>{?UHB z(*U6Wtht!&(-TLIJVjoHjDV5!Yze0Y9X4F=z-YFq1z>FI1zZ+z1#a5f7DKm!(q10I z9Mm3gl(YY3l!dOI0`T7>kWLW=D_W?f@#yN%tGWIy{vJZee@JR>D?KnNyd#QqkL=o$ zm8@z02E!qW|4#84g3G$+{wogVe$uh*JcFIh@k()Y z7<}4D0ZzU`XM0I2^Wc*k=4bx2*q$#BRTJHFi|^+`QOk=UNsnSf4u2j7A#)}jr>X6L zFl2lW>Azn@4$UesH*nUYBs0p;S>csUeDL(~F#=cA8;K9<``g&g!=b)z_Nk)G9{CPDE&0{HcF@R&Y;GJ#B9kXefSJX8ZSV2Bz);I4gX2&EbU2;MbU^o6VTZtq;oj z8KxnC@rA0DAe|2~q4z?NCCQ$l4eSMU7W&G=H5iaEfX2UN;%9HL6iLQvhkc6UD zdko3kv6ONXq$%fRW!c5V!tm_Tz*`lsm30h8+y~&>hQ6Vgk`}y%Suv!6`ipGcfhe{) z$Bqvz(A8FN+uPf7KRNZO9tBx)I!4iQ$#8A77(>}7A941*dnxXwjGo67M}k7qF+;=H zVts!Zwk0v9LNso&9Di>GOBZ2KzQgBaV1E9JlFon0sk=xaZ;%|yP8mX47fv|bEWV1u z8;H%vn1?)O54hM-Sv5wwY>iPYon+uc|A$w}Ir~YBn{bPI0>zm#zgw)eJlwMGJBMC; zk$#-fYsdWUqXmO`izh(~Xx!T69<9P8*w{!!Pz~7vvjK{tlO)pjAp~;x%V4g>u+8~r z*>>V4f%Z~Z&mX<7PgyC5+y|e9ZjYE&L=Wl)kQ#o6wRfYG?lC$E$@RDE0+^MZ<14vl zVNs2$hOiExW#q)>9)Nr!E1*d+1z|FA0%K?IOPFs5B*m=#kBqr|*DK|aMm<@0>){q% z|NEw`)mQ6^^!5d5>B5vosoGEVfti^ZbOnk&n6<#|CV^!2l4MIXyjz50LmdU{)*+LUl(Jpncyg=1qdQ0?Vy-3Zqw0GJ01Kr(si%CNWJ6|G?O6FAoe_2L z?_kiG;be-xBSK?lEN>_<8O8RG5sllnb0^hZA0N4U-!(Nkf6XjwqK!v#@4k`P@g~)c zEJc3J8eB&dzKjp;cApXa^8MIIm&>&$_k3ATeApn8pqL~WftEDYCLvQ`&IDKokT5kn zTZ1V}=eeIdA&b6%C~2Q7CZ~fUytr-}ZZP(5n~1in3zkUTMC0k;z-QI_)$s-~fg*J` zf*QP<2zDZp2c2vjQIdmUtS|@lSUA^P)PSqWejY^s(^$YvWTmE!Z@}BpAQD>kS206~ z`o-^%(k?zjTH<}s(IKXzgI-iG(9*?lfyeKEIvUqRtfJWM6%gSnWtU{Pe{cU%Lrh|m zTACi$4GsRjWVXl6I0@{ru^&dh1zlej2oiU3&Rh0<{`^U}wcCHR*5RpgI)ugLBnSy9 zU03B^;U!OcjOi6Yp6kaPvC@Jf9=Z0OPw5g9IxnTkE>aUMdv)l-8fxFYe%x8?|@$A!ZO$I!Tx+ydYhm{C{Ka%~Gny&~U}z*2xV{U@Vn% z*nYYkHYk?Mj}+P^zdC!Gxv;i?j>Eg|w_a#)jy^Tll2(3VLlUYKVR zIq}4Y)c5Z{V?G#NiuVpP<|sO`-mnXwgh+HKf==Vg$C` z6;v!5mJp-=bnpr*LUi#2MnwDlTFu?t3W>^tEiStN)?=u}e%aWkkF>~H$#Axdq!oPr z972Re;ET~0FCVM#Pq-XMDX^4C&%)xZn&>qAA0fP%H|(Em?Oj-)w zBsO~&ti#UP5Ed2|q7~s*OSub_MLNJlnWTy{go)j{E#Yw^@P8hM_CD;U9zmFT=K%OFxygfs&}9t|9DJs=3Saj8B57etTQR=7_c2a|+l)z#Y}eWSyoHpnF(fHEQS zFl@a+l0{8pV78N<-W6YuRKK9C&{9psX$3h|w7j%{%D2+GgeX9h0GUG;jdzw075f?6f;lj-Cg`3~!U0C~)X)1@pO zU_+Ai0#FFr)oj2d5b5-Qk9LHgG;x}LvXsfhu$wD{iMx3*dmX4g^usI%yu%*0*1q7R zul-H?CM>iNKc%I$HR%{tGoMDhC6>Bg8}53T zuMhjp2;#{{UMqc$X44i_Zcr7eCXtz{_iC)Nm$7*A-^|Y-jC*I4Ojx8Q(H>jsB#J;ogbuaI?#doR1g-FoM zIE=H!>6fXkt&Le>c7PTd8XAxfQYc6WB@M*}06!6Eo9;is-TvnDH zry4Gk9bjN2I6B9wIbnqX{{*5h!h)I>0Nx=T3%XBZO8rpU9Agu0yBAXEhK49THmSdE zFg=?3KJskO{3f0S7jZecYDrI(e{6xmhkqT(uS>4We^N3D7d;rgWAMt z+i|@O4U1@$mk?S#gKY$?q)>A1V`q0qZmrF<)CXZiWbf(#o}iMxKxL2esvh($g70a_ zK0$wjeiY+JAu8XK`tBVo7A01vql=Jk*76yjPuQ}1E%`Ix2ter)geMy=R*7g2?ocMO z7;7y_FAszOM#Dp3(u~PBjJ|LltO_yuBqF*BMKPK01_GvII{8q-M6$>JR`~JRO;E1T z2pImWMbFg!r|tP$JynksP*dPlzh_AzK*0|@Da^6rz&CNLzkhYAWb#kLW80zZr!lv6 z4ZjP3Urta?*a|7iRfv?Ul~uY-4sD}`D$plCUmVf`CQ*C8-2&HB5GObvn6 zA0%w|;NYS%-%f%Lp8ajWnl?UxTAHlRfOXe&q5k(IioFghoXqyBKuLO~`)obBRdY%P(;Db%)mh{0Fb$ zA$$N$a0+VRiFtGI1yG&sXc72#VOXg>K96>N;Kb3W9s;OyockF_L9B=IQX&)+YJS^* z?4PvH9b2FZ7dH>b0PX)F??3!{?%)0o{9R;JR1_f+r`3sMk5m%bh3wHnR#ryQq-AAq zS~$r{Mr5RjjFU36M?=|7rS<;Zk2t^A=lA;;e&_AFKG)}Tq4)dsdOnZwcs!2Bad?aq z!X0Qe`iNV>=g$|r7{oq)Z8l<5kN45coa`zt}MzOKoNhD>*@0D-jSL5r3Je>CN!HJdPn);$ri%32u3p#FIO84E(me0QekT3<_fIY6c6IF3$#CBy zyUpJ5sf(a7)GL^H_B69omrNzMhZ$yPd^oG&-x^kz6LUmh3xIb_)3 zyJMIARiYg8eM-vfi7l>lrgbJziv`84TeYAO$@;YV{LoBIpN0(^0;*B1_cBNKCNq=3 zW2INeAI}Ef+z#t3Bjgm4QUZut@a46G5;K&uPJa#c*t6w#Ku&2LoM?Qv9au$zXjV^h+^h(tcwlk zMSgdSR&vsPUvf70bf41zz)7MA+|f4X_R_`_XS{3OYfHb9{*%5xT4H+xCAL5xY8s8i zAB4*IepH?Ra$-nzPONIvVte+Kc zzL&(08xU4>yVE=6(cr&sNK`QiV|5YVNJuSmWEZ*_I-jAy97tVE`kCdgYl`I>U5l); zl@OSgW;e7cE^GYR45iV8uJ}->cP574}yBp=CP~xcD1eEVuYEpd?la+Pg< zD}w~tdhT3f3?C5-o1$*uf_o|}ON8sASi0}N;p)}?qHY+fv%awr6$hd`ii5N-#Xt;W zQKpj3ZZf%(KNezp74d8$=|BY%R7wo(V%1nPlU$0Sws+=!@llrsO>hEBO$ET(8Z(fgV(;5!^QCYdHm^I zu*HI+ACQ9KCzt+g$z8=1E8mmhFG`4e|p9T{P6MZ}dU-qwF|H&+0{+4o8Uux|Pi_g@UBo(5N;j8bQXZ|V$ zVuQ8o)^+`UdFiLE`Q_9#;1z!R|5{CWaB0AW+PT;;OLG?Lp=ROp&QEqxy}pgRP6>{B zeXHxfVTYrp#xK$OhR<})aSuEDY65EtZ-)>1D=s)FC`uHwW5PU)NNJ zj5~h(cr2>Pzo{)iUV(M|>18;hG(_)nb3@jw8JO_7zGPRl(bx|?!fB{c=cz+@#bLiL z{f81%cf7wN77dFR-vmmT2HXjp7vp05CipKSMs$zkE$`1S$8Q3kzr9Do#l|2bBg`X# zZ%eDOZQ&{sQlY(q>MM1^9?K(-`S4p@_l=1bc`AA~4o{dzl`F)dw6V_pn3sXG+)$$= zo+pW>JoW5Jyt8pOY8Jk4&z}C>KM9*RZ{8Q1u*c7y^>iqP@(ND5)kIHk=7`=h0(bA; z-ww`yzC^~;HiB^czZ2&~fjJX#wy+ET{yPF2vB^g(X%1iZ?$#fTlMGSDx#L{(_;I&) zKQR;fxJSo9n>3;}`Rw$Q<1L=Pj~f&HZ?B2Uzc{(X)08;Ys943MTddgUI$GE{P7A-{ z3|j8_lWChm25QDq&{$dgnc!fGsv5WY{ehBw1CkthAh>>u=`o2zMufzsm8y%;_iv1? zdA5wF0;l8EU0|t{@^IC$pNu*qV%EDIU0=jFbJI`V$LOD}VLdqnuxv6lge+@{uF)U2 zVf8J)#q+-(M8&V z5kt^GFqh-~;=`X;5@z||h^Rb~iuK45-GR(SE*3NW>cq3N>d|YiMBJ`y{o_hueS{v@ znJ9^fh}hLU^v|_3{S99Id7{@QlB5U)66U;!TB-d@eFHIw_LPN$hJHK}?h{1^$UIpfgBL={QB9&rVfJdhXW>V^C0S~{*wZ0))_pC#K~M*W z8r*?paqx1C@slU!n%Y9-^mugLn9~39hs?|y41U167k#ntyHQ(9|5a^=`uEdq;7LQ) zt`&UJr0dc@U%Xp&@AKy)hWN(Pa2AjP@aiZTk)i}EkO4}>Ah^d|)b^^!;gp9=D;GWe zYF}zY85*pf*D$Ee{Is+$Xj}^an_p5rjN$nc@!nqeQQ6f+!M?sD#ya<~s1T_t3NNlK zY6hl^TfmbITK?9CanKp`QDT!oahir(el5*XKbLl+Mw@ie=69pXqlW!hXzJ2_(wV;y zq5Re6i1y-s%9g(T?H$IS(5HSeshIpH`qywGiL=qFK$9u${j#Gw3DY^J%j_I-Rja_gUKgl3=o9olj0q5#LmfzibJS&X7HQs5N44JI9M+p3yfXW;4?w1{}3FrWC&q>dCtz zP$8|itfi^%w{hdhEjPFP4Z{svPG3MUYsY-HbkxUf9ro{^Hv45nXS@k!;0Gd564f+4 zrZt{m4nE!1_B^?F%a%QB1LHlzd?18eMfC|?Y3l392%$WRjHlJ`?e7DCfZdru7G)Kv zZg#&@OOPMR(-+_w&_pSS1Q~~T6fs)EXIi77>5baHj&B|c09JSCC;P&M*0Vl$lJ0oO z5ZP+R={=klBSw?dTzNcGO7Fp3DmW+Su4gn8>!FZ~l1XCuQ#UPTJ`BQp~A%utw1 zVq#2W_<+huf%;C&N_;?Stv0opK(6QHMu1w#xHW>8%n47!1CnP9+tJa1Zkv7MU7eOYeu-d1a_UpD{D?|0II2@ajZxTW~onV~EbBU(cq> za8yXE&HMNNi(IDi%ndnR$r=cxgO1F#Kz!ZcU~25LFFoNZYmfS(-+O`9z6Vif7SMb}V02!$P@iA2YXr>Vqavo*XBAi*Ix#z#$*ru& zyP-w*h=NdicQ%EO3aNS_BK~E?X+S4o;y#!EVj?)Vpt_O&@4t*THjpQ+8{>fT@|qj*)?rlose2?n3Z=LI@7RBAZ90QUWu_Z+TKg zVcT}?WEl*kS*oFLMUwn4Gql?e_8{g;JN5nhU0xG|6k%L9Xt&nymd zJLEbeWAycnrO7kqK8?NyN_KH+kL@8LdBL~P_}LwQS`cqneL+Je?En45U-MqYy&hO2 zb(>~{UHhXzNQ7WlXh^BYsj?gD>J~LPsApx?iID*ipxjp%ObBBK30Ay=K(1##ut&nz zJV_>_lLa%8sn{nyiNrGQN{2(ZQhB~?#J#oN?rS&PopJr5mI{XNZ}-lB`(wtsF?Fw< zxZAeklz-&HK3#mi4oL_v)1yAMIy^?FVZ(t_r|!=;-~%1;$j|Utxl)EsSwsOL*%VFD zA;f7~-o8SGfg?#4LbsD8%QjHHVM&!8nGR?zQf-i03w>W&d^C&BCP_iP!&3zCsF_kCkA<< zn&-AHqPKbdy4ofN$OF*!0p%O4M?ITR1Jlm08X=j71nw~In69~=@tGfai-8l0KR>ZP zIlf6IL3GxQod;?@`FX_2u@eztLd=`@mls}3y7eqH(Xi+Z(c;J)C@m6y$%q}N7RNZS;( z5AfJS!`Elgdj+-H43GzuwQ%?$Fn&CASryNG#P^xOjEcjnIAU}Ox^IdDM%Yq~*?bet zW<57>+7*wwI;Q(~dHZGD`k8*^orI<;8d3wZ=N)9qN2~5&yanuFD+Y7h!dkrclU#|g`OEJ ze_SoWW5AIwAkdp^tEU81y;h4xE0Q+1`-0c{D43AJ;s}%juP64l)L3KLvLGZgMSy9y zZui_PcxC$F2;?3Li&m{#k(d*2-3mH=y35Ya)ApKtv|&-iZN5Rsbmj~+uunW;NJ>7LN@O>a6fzau036s8Tb4- z>dA@eYE$;D8r~3e=d7ozw%g>MPEX5DF8%U+@3aB6qo9{CnRV&LIW{}o zQBPeB(&{h;fz;%fxr%iGri|+kAI=`WVR{{vK4)V^YfRdd^*X4fk(AbhOA$c=u&{{6 z8L@^|9{yVkV7Qppf#I2Yz?}~$JSZ)*TECK>nJ3ybe4)jOnb&j8i&oBySe)mZqSI;l zGxgP@=FYJnc>hcB_crz4g*v{c6kIoMXsh9a?`o z-z2n7Vj@C%%mXXlIx%yHl3#Xn0wM(F^h48*{{!Be z@nQOY_r@lpPcBj0YPNpc&YkO?Mf}yj(?$X|r?YUs1r=+*ewU9VEcnx0tZyykWk8p? zjJF}lxy5PZeE}KlUU+pBkRN?j3elyMhT|yo8&-=RLv=Lr-_7izUq?#|j=w*ur)Iz{ zl5z(hbV9H!!V^AkRQeZGALb;m?d@A?oVG_80tUPsIPhM++l%_N*4S*$hyh%)3)p1D(Zb1 z=Ti0BM2@HcW6MV*t+V}Hyb?eoo*8b6l57V0XVI(3XF`98m(xDi`{hTrOu5>zZ{NPD zk6ZyquM|G$CkwrZB)oX>1d}WecO}Y5DA#=CMp1U{_07F{95+0?7_)v2|fGGzac^vDi)Hycp)aIO0jid?fr@UN$u1Isc z`sp)gW_3Y4!uI)zL?=)oSsKIW|>(;KVOVNL~u?;G+1oi4CMi88^zFKm< zC3x833`9<%OBMmf?<0~vZy_h&hF^eoD88G`0M(C-L1tP?_ zNB4uCCQnCVlmZ~t=(54FuWS9f!_i{HonV?6KY+)wXb1kh|lu`vMHlCFima!K*$ zaco>mMsRi!4uS@i)#Jc5j0UvEwN`@g{P`D|^k%e-p`)s3hhW4H3Z`g>&6@SG^{W@J zUb*L{88Ocx0!2oz-gCeiy;8i-&K`so2>rBe^MRcy(}4^`ipLv|1p43P7S08R=KaM* z5V9o~kJ_+Gnj}|5K0t#C)w$&d$3$-^U-)_*&|9-+^Wwg1M!D`cA+O&LZdtFI*L?N8Ow1}wx(m#7wq3xz`(YT&{Qy+Hh>mV^w=al?-{SCE@ZVYJystq4wBxziL^Ey{-|5Mj_>|qEOqeJ zZ7cE|j;sl6*8S+XwCL!)V-^exZK3rKA#ML<{Rd~y&@`8f)KXVdr z2ZIKdHegnj`Lk%lh|bgMA6{H}>BPHB>%*qK+_K}qNb9WT!#q9hqOC@C6)|}3oo>1& z(>KuBEcx`P4Gvv~D?9&(z$YIcS>3SaPsE1dJ$Fp>PAKj7B_}6)xdALn1;kN8kzp<) z3ygp_HawxNY)2Odu4cp-$Du=i5!6KUz`uZJ$gBWEj5spgYIXu9u?;*6f1BARexpkp zzt_nM_8d|#;YE(ZaG zi+L|N(e4Ram<#JeU&y>U1(V6U_ZEpXouzO^3_>{~bV1&bhij|Z(|O1daDmkIi``IF z38zZ4J_9gE#@slN7O|;xY;2p0II^yFI{lFNsgP|qxuwUsO{wXNx9&-<6?5CGi%M?A`-fdiSwf|G&ae&!NGBoSenCB8T9@lzvYAJ%1Pp;>G zNPozLebp_rM@LXB@N)J=MnS)bQcs%a$Zt?w13Wx>%b*(6746HlfRMmarK7X+YG(5q zHEIOiUclVEj=ug7UXFYdj;Cy|U^n)*u&@U-lR{!*`asZvil8RU6hki_KK+au^Toil zOk^1`OCWPV1V1^l>l<~FNTi~oVp9G{Adq}jC{~v_rR_YK@8WZ40pKJpJd`)Ne~(O` zyVoB~P6~cABHnlYq=SJ@kGn=s8ya|FSA%O^jT?;798v0{1tOL{<=Wu#pt-tP|L!iy z&rd=g%O~QAs2eB+8Z0cVvE-OxCpr(fUW=vPWi|qf0f3A+(02#)(PONl&B|fWdsC$_ z|GX;eL`nIt(o#Esa`HxL?!is!1~#VxH5$tRL($4gmsckzoeiQ^dZqTX`+YL!*;CW)9%2#EpKwn9NGU-UR(?+;M29xT7ws2+c{EJirqY+NiYhysv z7cqS{37*jJxbm9r6>xEKE9-jUEJ0_Zo{9ir;F%}^@DW{&4Pj1{5FSqC#gm_PsenDjU&ti53N&27&bBix|81SQeQXk4tj?{^+oV z@ItiLoU$PLSH0dli4F}>veHy@;P1M;B2YGN{||5X3_oXo_F-_{RGp=H?ax;pUzC?S zbZ@U?!O?3mx7~B%-I;AI4ivVvJn{1M#+B!fcLV&EeK!12;Tcwhy-uzd-L{Q1=0=1+ z86mMU+d(@cV@(JYSv2`<;BghEb2k%@stIJELW=0=iw;>`*-NvnPt@X0$^3FeL`&Yf zz`^<&Vq zDEU0ojj&L9FRl07%uhikIDqUO>ev0Eb^~gK1FLze#cYR3;oh-JZ-O= z_v$mECbNH^YD^KSfO@AY5PG2Xrf6^m;80XbNg#82MDSU(v9@OB$g6N@`rUY9i<%RaWgaDkiooyzcJ! z@I2bR`#jx{7Z%-qUHH+*^6ahR0)xv=UphZHU_NS1S+NMXUVvpY14~GwORAss*N*Cv zg6})+|2p(iB^`BVA$CxO+~zX?20R(~Dlj-$1Yg8&lqNg3w``ozq`8I9eu`Tfx+ZiT zk{$W4cP^NJ;SWjGflNg93LtmNnxZvcew4Bv#YGg*Ao_KwAZ6pn^PiYCA<*A)G{4Q~ zvc8hKf%}+Eul+u+Pjen}^!I~NF5C8>DO%riZpem5XIBhr9BS|~g<*>gYi)E0aymr7?Be&5{H!} z3F5bZ60Ht$%lYVfaL3RsyFm#flU1djB6P?&d+!yd0k>bRh<3Q}PkxU+82*1 zY!3x|PsLT|HvFNFvz{*_=cR~U7_^3=XDHOD;DsbmY8I1o&;LNxfz@jQtju;H*2VJi z9@xh%gk^JHU(6qP&%UWW)5EYq7XGPayF_`V$|1-L2I#S0;J(wyHeR> zyU^0%l?0*7%iWkp`j=mS!PyxeBF>fCF zN*zcLapE!Hiw_I@q17+V?q5>+Y|u9kV2m}7*ysWtgjk|n%P{l#me%v9UK3bJ@wK_Z zob~(S{EI-#;vy_ zPYiuB(WO`H1^rd)uO=_Bj*V#9$o_QekkvzebUfYfhhf4hpHsRjqo3rI$9FzuGt2ek z>zuEXuHWzu-=n(e01$mhh=468kMBMuEz(EzG3gTo>hqNxpEF&CWD)my&QMcPKA=IAi@+D<`W&K zbkJrHQD=0Liwg6TdhLyB@=0nyB{C6JL*i^)j8&v1v5Gd}MyM z(3&c)y!sIbjkEdAj2EF4m4hHL?O3Bg*7td4(Gf}4!wW|^LTQg5-{2S(bnnb6&}pkQ z{Zfn9BZb)PV~D5HX3-~V)7{@I-j)@PY$HIs)xDE<%y$8Qict@Wikk#9Q-~rnp9=k^ zO`Fn)ClY&2K|f6z*zoyu5aAQgT;NaJkG|fh*N@k0+>4Jrk1Nq$x@>8eo~W?2whq5Z!<0rTe|tt(1I`5(a?(#)Qt9APD+RzXC#I+O zO7#tCRTf5BSC8v55P79wiKM#g|Ni^+U?!QY&iiRbityX~fGDkv27N?%#-vzkVGncg zE@I5?vqp6-lo5eh_DwRjF0;G;P#?ZW&EVUGp>_O0*<3h3y@9Mjy@TpEUngOQCN;3p zF0)Sc-X{3W&du0xZ@T%>FH@VZjDM}3;ms^{Fd`RNVe^68e}5ml(%JetI^K=IGNnfW z4buYWgW2nQ&u8!6z4#*CTXh{sb4ox5|_$uS^U&J=}GH(wPx$75=pR2?|Cv zq&u5J#~SYtsI028#KFuXPk}J4QuSd2Igqyz)Z_gzfG_}UTztKLaKh@1xO>Z(%<6zqem~ znjHed!dk!}`g1zB)$G^CS?&gywEaOByClO<9EdmTw_p>H#AO6m8 z+O+1~7i_+qe3ipVm^1aQ@)GAAZWZi>C4Mm=I9RMcPHool_gn9}K!scAuwPr1Z*!fG z9g9H|A|K<1t-U69D`@4t#_z7ZR!{Slxt>)5ThxTGZ5Dzda-K*qErD_W4k z?*Aa>G5qG@=2nj)t|F-TsmOJNnN6|q$g}*`KQtsHk${1Jp}cqj&=h_wuxV*(ssmA{ z%-YTVn2?a%Lg(tJx);#nIywXZz+}eKoMz0}?)DDUUYz;y>$%A_x1(9OJ>!gtey4{= zP$|n26x32{=`6&RmIQQvY=Jq8Hi)Ew+NvPNuz6J*bR@i%dZl8Jc?AiFT zgdtm6En2!%`~zb>@|{7uqG(@pu>qaYwTVw<@r0=fUQJ|uM_OS^xf4;VQ&c%$#`dh2 z;40S!POpbU6K~u+rAks3<17@38&L5NoHC_cW6-eezTlg}qB5eqLF{}1SK4!6!#+wQ z&wC8Qp+!2xy!`wD*P*X5L)P%%nl66%58Hfag?KYQi zUaV2qu4gPmSz3ACYvbdmPXo`MEo^7#U}GcJ&Ua#Mp(mUi9rGT>6H5<2)gq6nczajc zhHjFB6Ov#cU*_S2*4?}75;HuCa&?H#L7d9$=VG^~j|_xba08tkwB1LZXwhP;I+7|W zf(P()Vrf>q-W^m{6l$TBHs#zi#?S!m-up)SJjm^?=%I3vjv(5yK>B{{>OYJ-?aaMD zXI1C-Z?@*gAGT{>aIT{dvmKdit+zMssA$}u&Ey2aFgapd#$p<`Q$J_l zAIevD0{%k7R2xHPHC;(2l`-OMipyGF(WkMH6FSi4)*!8R=Vp)AM`Tvl8yPFg;yOm) zEA@^m*(1Xcz=5Jm5#l*$^w9#lpxLi`X zDk3m&(5W)-A<_1&xSnVcSCe+>m_nexuHU&+mzz_TyVI@uS=pEV4(x|P2*mXp%1g9! z;<{o`9^P+7eXL6SwrtVCmnM{7_l+E~w#t?rYEnfA1%|jsoEJC?7M^cqzZM+{Xs3Og zZ=H7B;Jc!Mf(s0#gqP>qr_pnt1xRQonGaA#^6lm{DLbLBub+sYl`X=aXAgP~uzjIH zOr&Px#IFJ8ai(GnY)*RRpKL!htPP6BO*!qTf)Xu|)8PW6+NblNz1z13z{Wa(M~VfH z(0e%+>J74qzllufws~_9e%^|y6!jV3u`bCVoLpoc+F!RJb?QT97!&#*pfx=Tvg-PV zM|g?)Nw8YT7^0LeGgUt$!6~-D${5mZpU4!Mp$*#T^{)_$tI_LRO4hJR<;%PIz<8*N|DP z3h&hgCt)D)q!=tf>+f2k`UWpal#p2VnFCMZ*qGVu*>A7(fjd89Q1ZPaw+H$7X}+0+ zcie*p*x98uUcalV7njjl#+9horYCrB)*2CE_JRO!2 zH~XO~Wrm0(!w?iSiQ?)>Out_J1=`M@JJ*@?>rwu*6YhUv*O0{b8FskqD^u9XM5r)3 zfJl)6@J=BKeytw;xMWBje|kQSycD!D)gNZ&xti#uw|us|ph;bO^^%O6#4T$B?aq z3N-Gdramdk9|e6fao#`zQ{hi)xdGD6aE}B$$P4%WvKbopJh9l^y?3uqg5U3Sy)9Nj zX_wG}2vSE=GnxK^)nqk*a|#c-i0L+d7)H+0KX)@aOfqgWg_Rra_)t%{7dQ zipu->Z3?fEt!s^V#sowo;EW)StMAW=Z`%b}5=*;y(N4$^w1J$)*cm=EDMV4zCY zpW1$!^rhQrvMxxn09aj|i+YnPYsrTN1f1IRVg@%-p?KG?#GwQ8sekK1*cD+4i?d_@6z1OE=mFdqU082ykNc$LljE51iBX4ZnUBDd4d zcn6(i+LI?3A=ILu3c)gRwYtSNo~BjXZAR2rsDEWN0Cg{am+W(mLh9yG{8>-3JCRLV zn($7Ic2FL|m^7j!*^MU%Bu6Y-aYlh}aQVq~ln-l&mssztg~s*T*_$w8y5Mzhrqs2N zS={7#*+p5`-EaQGU&VV>@t>faMgzg%dtXii!k8!bb>ojb>ex?6aa zYBLaAb}Vl4?~3T(-(AdEa^#|~&rPLc#0}Ak;D!crznd?^Xy@EsHR7?z`De+;CJG)S zEJ`&7k9@b66vUvE>_?OaX8IKme;!S@U)U5`zuN!|guna8RbI>{;%zWWQ$O!}-MqvJ z>J=BeXiiUGXld(UNR2UosqE?3eCi>UHRIDk7IjSQR%#*T@}^B%=)?1KAGXf@{5c9- zy7$stZHPQ6n=#v{MQ#XYloFGM^%PsaJ15!`DBiz+?}s5czToQo;2YpWc~DHk*hsSk z2`w>}S0RF1I^xcGy}{1KM8IriQ5}2%W+NPATvKgT?b@2Tk2M;7{OD0u)ByxuTW?oD z(wbTyV(!2Cu&S^Y0+h0czcy~zpl&ux0#PetkLr#qL4$|2n{Y1Y4PBlg64}*3gC8tg zK)$-ctW;iE@jQkvb>T}YtR6nPz<_l_M(GTYvIPQ9y2p>@FDq*c(A@~mfr~KYR0?db z0386by?~rSNa~4$u#L$q%PAVO-`Q!no|+l4+taxIx9I`Z2IzOTN!ba|>4-fz9DfCH zm)hjssBETzOp3dpJ}CZ%jT@b2&8kJxx3;ml!Sf*d-b5(CNMreR1WZg zU#vl$ox*dM=Q(p_C1IyMhTqUvhvInGQn=_D+b2>a@#n*0Tv&iay)dFoX-ZpgbDD*P zf!QJjPdo_O@cu77taV&mPE5QcZdi{dmJAq}pL6DL9ZjFL-P(mp=47S7z`CO>HrCqcF*%Sdtcx`7?WGyC15CP8+SNF~~ZZjdiqg9(m) zn9(LUW)<~V`KzD}Zq(3>3moY=SsxZ08am1;m;%bWXwcM4OY2e|izg*{3*lt-ldi2C zZgaEi-lJ1T)ksiL&;5Mc?qHP8+={AxOP4McgEQR@&O)W!z>=4`3Kq-fdywvw>DEg5DwCd;vN~$em5{t$^hfZ`}%oBYBpf)cKj(WKUTmT4+A!#un@*!W{#Z5AVAS$;d$|^M zhZA0ZqWgnT3FTiisD}+V_Q2*A$@!L-QXWtYIib*=)K)p}-1+md5L0-vHU(=*qMGZ1 zWuzFJ$M;ok+Hx=3qpG40DAO90JMuFqNTTth7JUz-H#Y|Ss&)VV(@u`8 z`wOG?omi)sj(q9hbHdVea(;u5GlecVt-FvuA@r_eWnF zFtDPpTdhXQ_@)G}fp7XQTU3xW6ieKQc^4MUetE7gk@A4cSCWONxkGi-DO++;O>n3S zKsmuAs@6SwHo{wM0Pv=H-;ex)U@E*6*($@?(|-B#r9;cf#v|x!ODWjof&eA58LELa zhhI-iQK+6wUG|XcHMP?l3c^r=>kT}zxv8>jdFUp`LmfswygNvPt5CD1@rSnqDpzE! zQ6A15bky%!N*#}t)31lEncjQx8(o`TPj2{CdE@pR#1(iC5tdh2xaQQU&XNORKIO%D zl;%Yi6=&Sk5D0k6lxAdIOfwr$Sn1%<4K8^Q)FQs`LVQ}pijfJj>-@>hd-tkOoA$CsJ)uQX zQj$Ky1E3s%oJL2c_x_Y|{$unMX1-Mh^ep#d7oxDr#|4QUM0M-euP@6&H{2U}>u6Zt zi%YxJnA61aRjtqch7C^9s7M$Zs+L(45v@|xtb6k?*Ipm)Y&$|x6L5Z+Db|0H6k^iS z7Zn#f(WAA=q|%b~3M?qwkB+h&wMgmvsHp;cU9fca_^NR57lu9P&)#8f{zYGa^K}(l*U)QM9I*#Z;-U9q~BYb{TDRohoWpxNE+2rEr=*RCT zgl2&5?q+L(U=ZdMR=qv)T1p2ACO9D4UJ5H|(s+qA86%{?@5;)`TKAh)I;bjlzN>M2Z1(JzN6jUC($YF3quS#bAVRE0^yvA zuBmt3CGiI26dPKvNPA-)+_NYa8X`Zhp~!kBMEhzs0%pe%I=9t4ki8q3jSKD`v3oY8~4ItuwgaQFoVo9?1TaKgNUG7(K~eagnW_KW6IQw$t9 zkVoVU#)+l96|qc?IJNUZAiG>>%vt~bYVL2c`h#79rewWb>pp$s=A7DD;ak{K^`U9( z#jnf8cKYS}CZ}h|`}cM98m>1U)gzBCNwy~bGRLr-e~i7V>#ZtRNhG`~sU_*Uq{l|? z;Jb7xNSa`i;Hk2Mh=Ql0amY^d#zr-fj;|xa-M|SG0BjKIsmRErOGEfO%gm$4D@t*C zz)tRmLAmjcVZvQe6^ibBb*EwNbrWxg^*!BTujil>K99$oUb3#my1jQ!mxXulej>7N zyn$UEf2Yc0ANZun;5mpW*HOnN;s%%Ea^%n7ZhY<9L==}V5D55D~hF%o%$_u zn8PlP(Lv<+rIe}UUElZQ1ynJ!s_TNPc4o*@4tjU*-aQYVrgf)I^;ERNKNL^#PnveF zTN~xr$9_83jyP9MoTl4j+lSLOeLqil9H4Y>)BT)Z+0%2vmA!_bAnzSTv17CAUV2)Z zbgKCB`=hT?@wlMrgIbb~DS)e(Q~sK{;`M+PvuDp1sd?`e-wo)N*AS%#+S+OfB7gpT zAV5B_rY7Wv^dYiCg2qBcfjJI{q92+S>6C~iAN$lNpvm3lj@}Tfw}R~9(r7@w@7$_H zsAk2`mfU}49!5 zYE-=leMtLCoxqZqneb-y;LDT5ERKiof+BZBOW$l{U#s~jH&RozfOmH@)QVH{iCS4Y8GZWaE`?hcPw5k8DuZbV2%xC+qdwx7(*Lp8x!`3O&s& zhRS6$fg0O>o&3p!IVymSD~p0S#N9Um^b{%vFUo_ z2X$&0ka+WWFGq(2O|KtsAvLYb$KS`~Wj!Hs;PCan#;A~qBhO5y46wFVp5?~{OEM?M z>}DvP@y@mAR&#m}24IJ+3#7%ZswlLTYLf$h3r-}#?#cGXkS>FH5Qi+nz<8X=U?Zvo z&{uB1QuSR{YLQf1PK&IO@~Fw&7`22k&iOt2yzV>upx4lPw;%g|Dbe+4($+5KSleNZ zwk5&FgW-!gJCs;2_no>E?cluxoY1@M>?b z1&Wa)>&<_YP=ktQ;ILuC!X5-mff|xvUR^vXJ?VA>D6kytC8uDPU0B=C#?8269FRIqvODV zH_TFOB&1tARO?Yf6dVIbikXY6N;(Tt0@%dtN6!hiiGsIL7$;JknKC_b5kn0yOox_p z%k02(=xt?i2UjHvL8N4Kh2XO1-X3ALm1f<#ot;|Jqw|=Wz6)OlfWaE{E-*Cs`)T68 zccy7KUfJl>xF{8cYmBR7(v!01m%ptzWbOF@NRwv4NC+n4GgcRw1Z801)SlptlsA5SQbXiwkfO1c-n0C=g+Gzme$-~^`ik))5D7crfu&qtwH+2w*&S( ziVLwFlKA3mPD|ZSf7nM|^+BO2cjg(@mda7bdy*X2Rx3qrgvQ*DPvgWnq3pU2AnJae zjX~^4%KI=7$(AN9y=;k=2AU7$wX~zR>>rh53@t6XX5rvC*urv?dD?%?lN+Iqy^)?` zNcr`g@f2O~Ev+ZM=xvnoty$66X9kTo-M6Ipe3&rM)j(O&J;QmrQ&q(~>95d>yvJHx z;z2@~v(gUH7I`*;ODn4osg5uS9DV9S4MBpXKBkd`t~qhQt-XC-)~H?bEeXXF+b7l~CtKqd zeJRo9HxXbGEM$)LBxy@PE!G1OC9%YUkaZDVj!k%w%3D3vD<1i(;#erb0i$o4oLKGm z-2iNEfwGK-Dwa+diDoiYyde639@py&*O_}fVps(k-8|LBWcrnm%e{VPM5Rm;IfuY{ z_tHlXdA1;QTC-a%JD4^iKw1Hk%cOtw^N5zS=ggr?X=P-jVpRtvaV;Q8frnZK++IAN z!Bqh%_((ORLg8?tUmNrOW+W9+ilo2*ztGV75>3IVuOkP6e!EVLUBpZd*RT;4%uU{( z;LAR7GmB21I3bN#_jzv(Iz%t&z1^e+)L~HP^O47o*SJ<&YxLvj28#Cnolw z)$e`9t>%Ue_;)1I1A)fuU9G97z!h*kcryNEs>nA)z=A5(9~X6zBKHaX#ts~@bLALd z2@`>ofVHRd{8Q>&`TI1mWe*q>m_dKo@Va8_$;^|e;pHseyg8^$y-uCd`N!=sS!vCS zxifTam%pPnO^A7y7-=<$dg6UX8{4Jx(`&VH8*WJn+-YCWqjyOF5rpz z|IagPEH{M8mq$|@7*Iy@gvRL+hq_$!6BOKCoUYeoB-e?mk4N@nEK~z@W_?EESbJbV zbt!TeK%R|sm3qrP$jeswfOSgT-AQBY*5$y`lVZ!xl!r4?x>=QSo) zsicNkr1;-RG4*=h<%-v@2BM6kEQ>`=WRUZ-s3^;LO7(Gjrw{>wPB!LmIs(cP>0~yT zxgu~Y%TwcPqVkdvBpiv?^XD@uv)fNTS4|h=CgJ6ud(DkHX*H&8!jqxZ1on}_nF`aI zt73ky#WDFg@0u~v@3jdfW$&O(${J_eUAi9Y+@>P_lM$w6b!*gEqq?EXJf1!_z}*2e z1Nkum9tG^&*@#&!c5DX9P@KmKA3WO=xo;0S2#mOkehQZD*H=@YA)FGZJ1O$yxpJ$L zZr`qRt)f6>vK&a{ z5!C`vkSK$k9rv;Y{SZ<%wL!1yjG||-r}X(hbNFi850km2A(PG$x%dXEt(;Gr)`NT9 zKp`Is2;940KbPwPbqVtKT{o1z!ggyl?>&=O5Iy&C!|& z3oEJ{wD3+qVT4BKQzY|ASq_A+w3jDzf6igE7Z5^UFs-r|q zc{%OLS{4;TZ6%^~IekO=jIvqa*!4&pbe)OxX)NxT_T1>9Yon^34d#6M)P7P_X}kN( z9!-IQ22a;e({1v*9vM~roQ)lsNtg%`T?J?-`XCZcFb43D8cX`y&=d<6Pr&)Wb&z_w z7CX}ElM|DZv$_pdW(x*J1@`>n;?{VZt%3TD0*HV@Mg`EBh*48KCX*y<*ek%Ff-rnY zeEG6zv!3%4SC zH_=HT6>BDHCK8)^zmC=C_->o(L*T7KgOfzrD$)~*-zLBx!3haIVanT0xq}oaPPBHQ zI+k2kSq6vZ-_ph==Xm)+HwrNkj1=XLmf{FtK|+&6NDiPJ=j${W92gC%zytRx1fcXD zOlk~3!Y0pJjvGl+v@~#BYh=s=cWn32Ost3tE6YC~*-*JC?D{|U#`zz@AJgk@A2xHR z_u09%#czj6iX4v<)Z_rlX3C_56cVuB-{b!+=;7+>3KJ`;OgRqSd-NF1`mCBYYkKwi zdvo={s*G_ICE>%yjDO3MhsdkVR~$(DCV>brtzWA9;Uh;{V`NQI6h8ynl#LYQE@(zX z9LTTyZ1SLGi5xu%QcQ^xLnQg*&Jjumb%rOAGpi8tPW}X-fkuz%JohN2npGX30cTc= zU^3dC&ledTy%rF5MBQ&EPAlFYSZ%B*Z90h)B!84|pjofpwM(il)$gpV_wS3lnz>y> zt4_2$B3?phCrdlUV$|~#bEKF{Nd%zR+FHBF_T*}fpAGNKIAiA4fi``Mi*LPp^{PgV;ucsJ9dMEtIofCy z9Ty=ukUNuvIt4J_5+R06XL#Ljx9c8Y_J&|B!wiy-sJKN6L<~scnj^IuOjeR=pI;gJ z?b4-7dp-`cwY6dBn%J#_l#SjS2Ok+jhm+@F3pXY+UvzjPS zXCS2r@}CH1GQ74*=kw>E+-|>*-ULC0Bw8XJk<~gBd}8d)Ok!p`i==SOQkagDHb4?2 zFq}4md`MRwA4#o2bTE&QM8%w}KjQtP zZQf`Mb9Qb;mmLuuO;u6@DuZ~??p4*W!wnb-y148MLbQv1B9x=Jlm*L@@KGyd_YV8- zIq-kg8_6U|NJ4e^!S7=cGZ;b+ktYWO#fw-7Yg?=%v(_i;XlpBwUhMC?OkMJywkjOH zB@oTStE*Pv%T>5O864~X{j;Z9d6e4>OQ-;ng$uopN8WoCUVB|R;g>E&QQ%3W!r=_w zYfH?`5*#^^EpFryU-7AIO09B}=5X=1mpw%IL;q?CZ6~`H_(a}8-Kx>E>igI#s^a$x zT_K(gGF;C;TqS=FKkgSK!3G4S?dq!hsKdU0?-+JvW$~~w^}2PFkb2>XQ-|kAF?fUA zNoEn{)x$G0hoh&|`x*UP{Z}2aM)#~1EEY9iMJXhVI(i&5J*~QS9aDCJ^C)FyzkdB_ zG$)rC^P-%BNB+O3VQu}ZT0Qss^ZxnNKl}ze)vgA`|L6CGhgN5?|M?Z`cPiEKruswt z`48{f)t}{me^*iS>WA>Zzw6#Fs?4@pNzI{YCP_I`9=odcDBr>i_ep zvURHeRvoDhu!;e-3KW$B1|fIS%^>aQ4r(f(Pn<-^;Rg={I#l~t=nwu6Lts*90GzZD zP*CC=eVK;$GPxFvV>RUO?C_+wz33Id`E{4;Pbz~nQBe?K0}Y1$q~Q*wPy{iFN^{1s zHW~Xp#_uQ2Hsxyl_m6jooYr?CP|OO2_|*fg&Uj^8#~<{^5h7z!0xgcW+~4ZKg|tKr zgCmX{sjZ;XT#>M?81c>Ks!HH2SyTrt=TB7)pD(*txw%>MzS|bRPpTml6>8x2+cSM< z(-`bF_wEa@rrvM+2+&s*#jIJg9!()xnSroch~uZsEi5d& z=kA3u3hE~{1tlN0s`W%#Mu#A%ohnH}AZ^OZ%9|5v+I@2k-@G|*(58LO+4O8@XXpQK z3F-xrkD#dtH`_J0ezRu#Y{R~cMdT<}nOompE{SeXe#|-Wgml8l($R(9bX{w^?0|jv zkrON;qqtbSgZ%K{KV{D6)cf)8WlVLIsVHDvR9##$=a=8-%g9`dwkkE%nxLSS2hyS9 z8UEYSYET-g%IK?3V?qeS|Wc~e7i-T z3Jxk&F4Ez3tp=^j`qcZ^;bE<81`p1l^!KPJHU0kXnwC*bUZN>~fEv(e%fz=U;7r82 z5y>t&q9)Qss$ngNhPJka)YAeIvZm)IRkwMEf0z(aSfW5RpA%+FSEfdtsrWH|IBn4J zmtT&lO5G_mo!|tdH%BNiIt|(Tpm=*grev71nlaxlmMK2v z-+Hktrq|eT%Cihw1=%4Z{LJ}uX-iPgDKI{84BMI}f-?Et3K15c8PNjoY*BIP_g%{S zR1pO5-CSs|?VJ3AEcgh}Z~flJHJD5zYyB4FGVHf@u+T>HPBgEDlG) zbLVk|zz19aO+_jA9uwlHy;3G4VrHWC26`9H_1zM)a2Cebqw!AWTB|59=n;86H%P>0 z`^~+Z((6tNrRpCvVua@P8#ibXH;Q}WSd{@j)6lxnX*0W$1euJGD}1KKo)K;J>LA6D zmBe$W%=q7^(7F0@-DPX8AO=WhMHqE=XQzELEy$4I@f@!fLMWX(*VUxJ8PJ4WDi)3O z<$I=Iz~~6H3%q45pERfMEtG1Bh{hDsCrEVydPws=?n}D>Ay6rT?AUyaLd+yRr8Eu9 z^faE9q7an<1b=@OR96A=mT0fUS}1NxJ+>cd64m|bV1<4KMI8D{&ubl(2a1ySylcXe8&$GvzfRhWT`x`7Ao8P*7M(m7n7oRKa zuC%FmX#%t7h1%*t=Fr6qEy-N1rg6y4_{^BIvqoZBtt*9=3~=zTFfvvS4u*uV)Y#J2 z7o>?HF3%`mDR}&`aevE2p;VV$JhWl-GLW7ky`3{CRM`eEP?@k2HFI`q+yBA?O@e-f zMn@e+fOq_Re}UF2CgQ-&6p38DGb2tLO^Uil{9a#^h_8h1rOkMHTVmg9k zkvqpv4W+D)

UKF*jns!Py-4y6Qu)Vb}W&X-@mNe|(PtKp`9@~hqc^uit$<%v zl7Efbi%0__GC{G|$vdD?8&iNl%aD-^f}0vGS00IF{`&T3*0n>BMx856Hmz@*_Ya@~ z9605D%>BmOuo^RFYmqmj1o>ZIbWrR(<2*U;1pc?&eMi%ME@EGo-0Sh?@ zP)s~^!-+$SF~1RoB0>Yn<9DkquW!sG9hNsc-=^$Wcc+`&vURI!{M4|_%EAP~ne-%# zJCg#mMY>C-ck!d6hnvNJFH&z{!o_i%c+x0RR_Bc9|E^0`?qHC0*p8we?%E#^=use>lYW*I6j;Ggnc}g@RT;hVKJ<>n zP#SvS5`~q)m_G6H%=`oU_oqdVZ3pftC>CtCj9%pa66Gh~_!d;gp`I=-E_B>tJhG>( z-Cuu2@L$I01fkEH(RI&uuby4H_fz@)d`;zvVe17TPq?qJsJg}Dj> z#QJQVkrojUA2@3;uS^bS1r{=5ZG%NBqrNIKC(DOQ30TAW71X?Iw2-h}idDNvwo8f&Ww6MEU^{7={7Nw&;AK7k0VVoADRhah3r zifyt1XY2^| zK&!n2QPC62XT7kEwqM6+^04&9=mf2pI2ML`PZ<;)ABe#JApbdG!>aiBC@-%oi`?DE z_B-;`e0cnp&6`^i7HcYCUH8mnh}q~SnWY0~GZ{GT=_xq(fF~BFrlZRYzQA=53D$zE ztb{zj3-=#+tEqUD5wBa}DAd+Dh4^>eWVbnhw z5txdl6k{oww)=XH>>JL~lbjCA|1XK$y{vq5fA(6s8-~h)VLH%%0IqD$eq1bV1!q|H zV^9Pg_r}e1XeFN>MsDA?SjGlaT28RNdmOB7n6i(#_R8>Hks)98~UnLGpkY_|BmXWpSX?pAD z##gRx0MCt%v?D9t6;I+?NFj_yB9`053_yKi-C#mE^q@9r55dK(bv7Wnj6}+G15a0! z(=zo#l8X-=3i%B=rFi>h3hE~*fcO(ZRB_SaSfopgQi$=g+;Flo$9dM?vCAB;`ua>7 zeX=`yTtxNk%oUjXsx|$ONqSCxWJ%8GH{p|tAEtSf93Kz-!?#(@?X+yWZ27nc7T& z^=^BKU7}Y9Ui-UvbaRLE8+_s3(J~1s2dqxRJH(rlvw`A72v0!z&EISM-(e~95@@2R zo{U>>HX5i<&apu+CpI2{%gXtc5Zraoh{)sb88SszBoh3xLh&1EJboXO z2a}EWLXCCykL^gxd$Hia=4SXLl43`BFt%+}Y`;4op^3jAU>G71EuYR~4bZ zj1|#J%N!RPtiPsh0`kl5(0N?a4K9j`0y^d*16C*+HZM*kMb%mr8;4qM>5Ab!nSx=;JM6vZ~)s_hvSHJ1L^-T@Xo@;vaZ3&v@ z3^)tndUyn}o`tg%i9{mtf9B0|_XjskXH-Jc2~FmsxW>SEV^`U_XWl6Mo7z=IsaQCo zVrOFIsjLzkVZc?p|4(!0;@9*2{{LqYDcfSToE1Vz&YOlx3LUl%O3ESPgR*9hi;7ru zGUv0AQmB@5StN}NGpE$DOvR9p2&Iy~_bc1%_aFS+KHKN>nW^5d!}Ywb$LS(7BWWVl zy3oRg@Y?7B$cCxX=Lr_OhD=Un^9U!D_of?Z3duF}+<(BYnzLY<@w0QdDZ-1ZY~)Lb zIc`o^^Qy8^Q+9igFHd^&(Xd68c8kqUB52Ck#jex4xu=U=1}E?OwKB;1j|psX@?Q{1Eq-9y-E;Y$IDF_Zwv3&EJj;#{ zJQ3rROq>$-sKxyJ`fnjWon3BnA)6?SD|v-f{=6-W<-RCDJk)ipsNM;0zQ1rV&Imfe5j5-NnZRNF zh^k2{{@Y+IT4!|)lr$;qKjX8Nbt8m^#ebk|Bw(whHr*2(`*$%hF)v>(Vtb#T+TUmV z+c=8sz*)bySDS<+6*|A0n{97pwuK)vH6>&*bCHTreaRz zY_Uk8dJ=ss$zena?JEoc(i4)Jv&9w6iAJr2BTythQsha{ENge_AaV><)O+56BMTlA zLA7@CX3U(;jGK}(ahRRm&{?7g5Hf@L zKQW!!iOP1^jWOkm5*>d);d@qsTyLJp$VeKttrHU;gR!*MN_80gjpA0W-R$3hd^2%V zM-Fp&&YsM3TCh~K7sL^-FC^Mqw8a=tNZI=N{P|HQMj=^^+qW-*W<9OCrILsd{>eLc zf647o=J116St&XL)MSJYxoQ0fx73+}i15XH3o|+%_g2dug?jbW#|*jlSnFkawy8l= zK)A&brgn4d`}Xato;|gagHY{eyp;?FE`S7=nE zH4Jj&7WDct;6%~AGNULwoY@DD)iJMClhJq{kf$q+YBSz z9X>jOpLFJ|S$nYHi=CN1Mr8&Eyg9q?EHP}@u=>9g5`shQn~I`gs$xAJk;M|W6t1`R z^95#rx{a3QoE*CMR!oUGe9**wheE-BXlTDm(loLmf#QDjTK2KM$2(u>3Lh)JY)8->xf>qlN3ax1{en z4`ZMA%RwM#(nl|SlJx9nbVS)F&uoLQl?({F`3r}GdVGee|9aer=4t7il`6<9WRICe zBSDlIq`^xiN^ag58#XT_#2T9=+qUx5u;=?GJ|@Aa8Ihm}#=6}Iey2Ze#NcDM;WL~q ztu)O3L1m;O!Org8Y+&j~XE>f7a760FBo7@HefW1`pj~ri(o+vMGa_jx_I-sLAA(IG>%yTKkhgzD?8pbgH-n?VmEqr7Gq?wfB8 z94O!nJ+dm1wGy@#TS@~~Kf$v-Qt2V2VtE)qfiZp*GZ;oQI^X%`jX?)8DeFI&*Gv!{=5U2QzP~6-mYg? zhdFOV`03M2_dW+Mu|=-J3B&vU<>+}p-ab9`qs`Z}?a;xT;Dq&$+W@eiLoj4cZ?v&2 zjd6Pl))n5#9{1*Ez^i_u>;RMO00TC^AVD*RRyLZm$LaZrlP7fQSZymPjRRrcZLrlmh&n0NIh~nNyfVAb{B0QE+3^IGT@bW(LsVY0JaZmCH zPPQIGe+VqDp)iu@)5ed&D+`<>=!NLm1d*o%Ss?4g@FpEMSk6y#@9T(9i0|WAmwejA z$w^lvnLuKxpEz+dwtpHZtK;Rk1rd?@(b&7J<{@G%iswlB!@~D11q*Xx9Jihri}TVT z@!t7wt68&+Hjjh7xdME&C#hWGKg&+s^CI?uXHN_K!9Q4Q69+2s+Tx029I>>xYysh} zrx%9AkS24a&v4_@AT5}^n&P}OE^s-jYb;fX3S)j_mZauFjU{*Gm7X=!;q zz?ER*1IBw~0>;e)kQvSef3e_8E|bg%xJ4y;hTKz0Yq0^yOt2zI57HpuQt9cd1PU=I zG>VbHI3_a%t1v_b<@eSVl;)+S1fhyFh!K$2UFI*0 zw76u&r~eImfL_Sgx(8%!mY8f{rIWr0*vgzCKku8brwN<&fmz=yWVrl6`u78O~5M$!l0#J(Hnr)VqkNcOPD-`8!EMKN=YL zhILuKNO*)!0$3Jwp6N-c6}E1pTm6@tpHf1_;Dz5$3kEM^C_5d;Hdt0ySC4|a)iLh> zzT47<%4~O&!&t_H-nlG)2r$8KWjDR(}hIH-P1QEap0&MnaIn?;!%|j9Ou^1Ro z#@%`Hq$RC!oA&J`38vV&(wae!kf+xP!Ry}NzQ~`*{JZ{b9|W2}+A0haADu=MC4ON6 z+QIZOS-EFDU@=KQx@%g+58xtCX8Y}ZL*Cc5Yp>HE)ya*AP%UAzkvPJ1hje)VDdJ~u z9^0rMh~47ceZ$-OB^qI74sR%MF1v;hy76%r^A0??j2_`2h4f5u2M_w*G&YEw-Mi+a zH^rg)9Xd?7uMi5w8b#b48FDq%!)(Ciy55pd!U3@sU|Y#( z&#@|*qh~&NvUKAktHr#>#-juq`1{w(z`CkVA9w2#Kh({{OXF3(^0+uVKU&qqIleh< zU2$BrpSd-cruCN&UWOc|Qyt4A%ezvx3=O?*PPrF7!2*`};X{3&mb^g4zrBk8(R7m| z=@W(vCGP9*Un`;;`R9ykT~*{#m9oB?xCCw>B-V}0;^knjno9aS25Br?P1QAIa1%~1 zIi=UN*I?m)V4M5*?+=l@#{C?xwfD?^lG+IhMJ7hCV{LSt*vABFQ*}A@-t#y(E6zeq zq;uv^T;Jbmoevr*`gs+GYNmgE@s&Qw`wsLrH?JIaJYaQoDI2^4ZhaUMNO9oY(^TV5 zEgr-%`HUJc1);498~J^^JKVJuZT*%><`U-x9>NWQu|oGjmg|n(G|1D~LG*jWR>GMt zeLl6bRme}S)&r>8a(W>x^qtvH+c+W6p|TfftrAjmx%PWW(Zd}%GJRJ}Sg5764r_)5 ze3+jJSSfHfdYCzAWJ9jjSzf$9*YuBo$&-~5(2K%yh?qPI$Zm9KI9hxD*4KL5qoi6| z|F7!U59jVvzq@aej)TrTE#+BhX=e5)rt?Xf<-ve7hSmq7AI-3t61Yq(j4;+eXJ;Fo zU%qLL+wB(=PA!9W|5lajpEWu$iZiNX&40i4`gG^ITKenj`cz#;%)XfLGxZhlN z)Qf6`mr@FfJ*;Uhr!4)N%sb+JDEaHJ??hxf{|9hS zRB6&ZBM9!&z5B@WK?cliOuNIi*Od*BFrat8UY-(7p__%AuYLQ<3kgSQ`m6?EU^U4$ zhI~ZXh9kTZ=m746$_)=ZP0FlSd8g%seyOD}%>Be>YH(;_;yt)|mn?y86Vidr_MpuT zpwl-2A{As;^7tU}a%Vvu=72X;F7c}oe-b~UV{uBfQBmoWC#!@j;mW8m=F6!_ROJFA zDt3*t)p}We%aB^))}G@8feq-rLcO2V^RAP3R!GLN=CySYE7u^lgzn{gub{Eo*woWt z5`1@QO%UH&0z`T5IRzyAE~JN-r%SP+hlb6#acxAL13+0q5pzFDh?p1#I7-;=IMK28 z^i|3cjreeJJpk%T3o$qMHHe>R;3#Zj@PRN%K6L7{3hKSDPLqP z4!z~Kf^O_Do?FzUo5l{l@M~SKl@!~l!16z_pF)-%QlgV5LXb={LPo$V_e+U78WNl0nX;0y`yeO_fi08q&6{md?Mui}W@hG*Cy$>#l`uca_yTwGy7e~v z-$ME@_?OcwTerG@1|&L$T;;nU3QB-fA*FG=UfI~egKYt5#v>w{!=W@`kQ4kEK?7vY z*PCB7kZviv8xkuz4-E>FSQVOpY-o||^e3i2SJ7dM`;=e|x+Hw`ACNu6^tbi?pUIkG z)f--BH<(QHtXp~-T1uaia26d#;-`S-ba#@Jxqg@4l`rmA!3T(%5_&(c5WWeK%+{11 zq1Mi;iWuZ;Yiswr#+?~;KI+o?mLE6_DK8xJ8nJ#}z3TGF3&2FYW7K{Tgs1mQOiZNBAAdp)BN0R(3+U=n z`rzKZaJt$!-aUbmITbzvMHFQih?)0lx70@QL|PRmAys`^M*r_@ekQ#*Aw{rgg`lC7q@qPm8P~e$+PZOZ)(ohD&O_EQ6(`O04j_ zSjJCQX61$4C?@7ox{Ua z+T%g{MG_a&+P7^x1+xYG-zeT8PS_P(YK@?3?@7mIw&Kt4uCn}sbRqB_`BBI>ax8}v z6MHMGLtb>y?F0xVqzq^>bGU?43Oqv(D#lBBjjJn)M! z?a*q?v`L#=7ZDMm)|NSS-w=hyiY@b|ukh$dvZQxaED!)~)#ZT=l<07Z9Ms|_GZS*` zEf`TH3^DOAa3nAU&&O)>Qr3*bphCFZ|1GP`wUvn11IlT*4V2xqX$d~T9v&MRtMn8~ zr*!!rNGx&>({*%tH8K6GZOVvFrh8gqLW0b$QH+ADHNZ`{r*O~?uym%*t5u6RFk?_s zMWIrTbM0wIVOfDyiDIz|I+U4N;yW~Ks9ZCKxQt~0*mVEZm{7#E&Hz=;A~e@FwxbYe zDF|jGjLJQk;JN{4(S#mWM(ob8U>Z*TGy>@&>h`r+$B!Nr1qC|?0H4nRD%$(`#WOxi zpcv_RQPeV!2n64TV0ii}hor$5^S|!Aeej%kWwZn@JlzNoN%ZsjPXUcIY*-%an|j(k zNuRQ~?n)}_9%a7HttLLdGDP41%VNBK^?RQoT7 zvP{Fj6Ba#@3X}~?@kvXbKaq|kf5~csMx5$7GVBnsa4q36bP!Nou6{cR?@#jJkk^Ow zM=@Px9Qp9RCPGGFEP>JLY;V+{ytGt#^K3r!WEb`L^KSLjvfwD%}I ziF-P!f{Wc$u`+pVQaRnRfy!YLgQWLS7)c5+-N%pp!d z59~j{fq$|2be>C4JYs)#>#sqdK}b@oX3d&m%^MN;lm^Ds6cg>Fv_&|+?3u?`aP54S zzW}6PTZ^#W(ea6<%tUu7N?>w>0ydIe;yz^3+_^`v5v_h1%GlQU{0*3~S{aGhDlnE4 z(Q(B`yrW2K(ze1r95M91@S~ZZ+NbcbOPMmLyY}T-mC11X!&*Hcy6H{$QS!@ybt9~f(NMc?=ZmzqM z4=;}|$_%gaQ2FF})!YpCU}mhuDj@EpdfHK1T{4t>Mh7yg>NzI5bGYkrQ8a826*_6h zO;XR6O!=9D*MDaLrnmtH+M+iy`?c)mVcJ{$cyKBA&b<8CG8d=K+Cz|nilnHwpr9cX z-c^?6WiVWn7PYHR`R;x;#rQg)T)x%&j?S(jrg6S(Rbib*6M!<6W-*dh!0F;O%D1Vd zjt$Db3GHGxZtRwZtij3ULh}CBn?=Kt-ft-3#g5kkZyp*Rz=CF1r(Y0a4!LoYs+nwZ z&y(R+ij>bS*z|-9KBO5tEqdV$*<01^pqYQq@U^YyA^q4;h>4C^CO}Z9z|L0NQkgG^ z@c!X`v^M8IWz@l?q=_EVB-h|RVc)v+D;x{Zs*?%8vP8dEV_P5bzwZ2kF)o#M*c7_ zAr~H7PyJ<;^Wjr{K%~nHf1TDJa$|S;T>G&)N0bqr^@;AFWdT~+JYt{l{0E%VO@18u z=JlK;J$sr5$GllHXO4C`c6w{Rtn*k>CPR;j~~x_T3XseK;+P+^y1DhxJr((Gsv+?)07fr9PT=P zk)7vg97*P>e$U>T>DXqDUtp8(wczW2>JrH(vx)+HLKb`L=<1HS)euU^rBV=`Z`cLP zEDBJBX?yFj&QGB^1J-zO$zfjS9r>DuWC5e^^RZLVV zui6uA1uq^b!nMOzUwM@eIoKa>by+blx^U=#gXZM}M5m5>{fkF?Jk0Xn8ipOadV+-` zdF1#GB4SncIf&1SrioJ&&Bki@R$~GKZdiVVt(32nTaC751db&_0#SkKGOV#&P%yr2 zzbsh=1{3ye@J6n>NG8D8q=yu}JTQtx8!M4r&73){KK4>|^~uhkeLZ3C<0PULgd0)x znh8~s^3J~;5PEgzkmgEE*^-vdtAo6`+tt#&@rzxoGSdN=CFfPyL<`GPxNiwhnmF7^ zeAiWy%b_N}dHoy|Fn1`HGaHO(hp4^U#g(zgFb=lkN6}CWh8oW-GzQ6Nj@m#a zgx>_qPrO%A=V0ck*{2u7NaKt%`rTo z3F1;o@Zy6A_JwnD=lu*s`QCx6*-R{~Gwi!c*!n_!!G^h<@aQ zByyUgUz0F0ox!X%6^G)NdkYME`y^}*x{bM)dLF?H*(B)O4_>5@(xt-aa~uQd_HL4L zFB?uu8lGC_*Ib6FC2oM%+m1&N(seeY!=X8UjU?mAuoKf8MfxlOnj(_nafX1`W|ct5 zx$U5j^yQr6Dw{_EiW9uVjTshl_7oG$Xcw2}&z=_Q8uvHot}0qTBGM29G>0Qv(iG6! z3rr4WgEn)I&vhNKa5xq`3>Eh!>}0}U4^|>>3;VWU0bllyB5Skc3}`BWX6X%dD5JT- z+6qiYa^1n@(FCmI%$CGl@>wLFOY-u#;*uMg-C(efW%UwJYuTxRzvsRZAE|0w#Kh#$ z|CML`Pz^<`#8jaX^zCKOilq}K{4n2lY0yqj>C_FT62j?PUc~-!(Dq{kkMG>s5xJ*X z{gcNiOH~P%C}3VYLjN0UZ1~aeJZ&DB(j8Z>vH7 z)(@YDv&#!KZyo6^H)Z(}1Asj6BZYumbFF2It?iLCXel^@X}3mgdI)$B;4?Q`jm#XKpo5+b@l3Hfp{K^I!qV-#HE)k7tY$0(Tp{h@184?>Ckf_t zDbXKF%q@UKnST&$m!w8!p9~)EX++Q7TvnL^Mu4wLN(nudYs%Q({#UqcfV>h=B^&C% zRS>?dOIj<5n^>ikLZu*|8%Ecb$|%1NT!BcQHm=i=39}J;JB{+e>eBi8@6D` zNh^SLA9rl#x??U?G)Cbp203=V4z)u<{f~U%9|Ak@sn6u(bh~l5dCN9Nj}4=fXfRfb^jyP+ zU~y}&!&!Z-tR^i!2+@D^$pCg2nI1sRI(ujJyD{RLAx-wsF6uB%_%NAs9FXf_ha!lH6sWd`}*4@~S;{#gT` z$ax++Z9SFVfEicxd2n;HUZC}?S>3A_FWql1+dhQS+WXj>>McD*(pBss3DwvC{By#e zr7+Kg+7#&-2eUu(eKK2KD zAPml+lye7Z@M*s7U^vc!>*oMRF?B%m{4clH~Bl__e6fr_zKaSbADV$Ui<;|F(cJ)IJ~}nK~NJ zgXq1)LPauB#qbm*ol7|}3H582&p-D8x*XeB;6q#665e6TRxmA9c9gZQoR+zvJ|N5D zLM|(POoY+Ah6l8mfBya7h$CxjIZR_<5_0Ek>dt7hND3X6OyId>2oehj(xSgTRK?^Enk-)34;t;(;QQWrRGfq5^@n z7n>E-;Q}{yk*5tNw0QUCL-4d-PMOnRC9-`1QogsSRmd!Zz&%!AME$L<}G?g;WL#sX2mQlax|;gbYg zj;8r@3Y?oUbd0re_|42kM+wcD%j0a)wQJWgnuSc&qLF>{0*QoZlo;)E%}c3x28ag? z;Pg{~?APja6o3n+##|Ct@8W3mqi6Gj)$_xGx+wY;?Rq>J0A~o1Dz9gUV$#FBqzw!b z5#ut04nrbc8n}?dO@5mXGLpeR8VvM0Ki*d{{XKlqkX#*D5MkfkK3ZR534g8Xbn$x{ zq9~3nT>`D0`yoq>gbe6M5bX8SOB>89=J6Z>N*F;n$Ri*!PGux!3DjHpe!(vud@ud{ zR;$nx^D7u?V%R?x$r(R=(HGdGdQ#211CAPT?GrLkD3J)Vt>>i7pO{ut^A4CPriQz` z)$vGbCF(?;iMzaL0nbs{KL?$N;tIr|GKu6bZ5d#(D4JK)@K!ZGf(Va!Yrp~jSW-pH zX^L6$s&jdAd_r}M1<{1!Z?LKw`_Ds8NQvPz+mLBfeF*EAg7xBzWCzklr~U7h?sigX z4;*+pV~2I0Ib=YM3v_58)(TZ}dn5<}p`sB+f;{0W1$Z@A><)HIXa*a=V=*3_x`Jr! zVEUg81w#K_y1R51@5D=ayuq94v*XAQjWA!xU)MrdjNbQ5?BMK7hj=+Ssx=Fk^ zJworW0T)iwTuNxL=)B|i>;a1rjR@tWF-LXiq9ey#PW=kEa($mmXF9z&+7)D$c3B4t zA}Gd^I>Z-{a2x>GQ3O=c6#O>y@;pZMmhA?PF0_zXT?v~`sqme#KN11 zKG+P?R&<9n`0(<}C5Ml$D#I#^%EzI&cHKJUqL+uBdut*e4Y*kRx$vT4kPGqXKUZ{= zbsg`1T{mnLX?KvleKJMj8;EYSg%9>^Tq}%#*7V|jKa|u2XJaAr4s2LlTPy^cG&{lp zq39@N$;D(`T};h)B+_!-Iyd8yHKDpXI<^=9xwcJ6EKZt8qA7q4-AxYq3t9n>lJkUU zb9D^H&1w424G3Bv8>oe((KIpU0y5LmzvO1i zLAk$1BJh6u?Z-nSYGb>GOURkbXA&w6%&~3kpq?{9H6*i62ox*t_b}B<1HZ^iS)VQe?0`TGXx-EYON953l>|NZs3 literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/auto-tuned-worker.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/auto-tuned-worker.png new file mode 100644 index 0000000000000000000000000000000000000000..a0aa8baadfe414661e19f755ca9b11fd77a3edcb GIT binary patch literal 114222 zcmeFZ^;?x&7d4D6N5#NI5j}1|ML?xN%HEWU2+{^fgGe`Gi-K;tQ4x?-8a82qNJ=*< z-O|$USe$d7>$~1xzJK85I>*4x-uJ!kx#k>mj4{`v^9pCy{l)SZ0|Uc4>e-Wu3=Au! z7#NoAS-le9ad@AghTlXiPhGaWXm-QW=ISk72KlR&H;v6KjSa5tvDUq1VPIy;$0N*h z^w1uCOUs)UqP)B&|9b(C*)2WZJu~y4@g|Hn&#GE5Fs#2y{$CO;8D+q*gn@y2Qd-G2 zxWC!PR_W4w!Ejk|#H7NOB`en`Om)iy7b^W0#Qpk8THrbX9GPS-9! zT~%LP#6MqI8*L)i{P!1z^6zI||ND!p>4W8~|NBd~_5b?^28RE)B#ZIz|AK)m!~btE zyj?SFpJPAv{Y3=wv6Uw2&5=T_j(ioTu;UrtdSxCeO8Q%q>@!Q#*RNkc z_ViRSHa3oMuekcxnl)AnbCX;W5?b`3T)O>O_lvuCmrprL2=Mbq=jLkp>K@?WIQ{0$ z8yQ*Ii-v|Vo0yq>XtclminjFlm2)KKOazvSPillXPxk~07%98=@oML4*tFP1amWWo zw0TNO^xOOD8VojNinc#H;T;gL!#XF-d75g|Y~@E@aos~CQa?BSuDsFh>+>k+46nA1d0QbByXWd3f)-_jjjU9sAE z&b^gUo{EZ!l*0L9L4N-IqM~Yrg@qGcf=7;AFfcF}u{}nMIOV>zr&imgda5g|^>yDe zECsjwV@t9;<(VCwora$uZe$TO_U|kWY5X~ph&3GPC=zh|^E-s)Nw9=d)^7Xm7lI}Z z^}2pN^?G3gKQ;~Q+9_n}P04JIH(ERYjVn>VCQeoD?UmbtCJhn({{GMU<_64MD&>Wm zBh1_KdvYdQ*3k20d2^K9w{k0LYBnoSUH!vt3=G~JD(bCSG9+MB(_N!(Rvnv|+n`fl z_2#0$mG=zt0Y^_=jFdANpJ@#^gmo2avArIY)@)HGKK^OFr2gFWI1As6Cx?n?WnGnK zE;EfxJnXm~QLhA>Y9w3NkVrhgu3!s~TNW1MA6BvH_tnI+&hNszygL14`-=cRJ?lTY zK^8TtdOZC8Zf;AdSW(5;8p(;e>jMLwWnpZ$Le}jRB$w&a zZf1Vr;zj0LIoi%sSXgcvCB(9V+IVR&O-4b%x7}>-`Ae5lFNX4M+HLz|$+{g!KPx!5 zx3_mcJHhlF7h;@FyG*x^Psp3M*R&`R=zYw(q#p~E)f*GbZ~Oa-*t=>q^)%xPCMNNS zwk>RIyoK6laAVyqi`8LI9DOiLcFi`P6gC$`F77BZGn1BbLd}mq3ZW1vW|M$tKuN5P zef;>ZEt@xYzjWuJQJ%7jDGLY)#6(Azca{XDI!@i<*2p-|<}`L8Hp`e|i%9t(4(hE;jhHuYF6ipM9vK}?z11PZt(I~Y z*ZSO-S39AqN@;wcQ6*L-v8J~=*6Jf8=gEl`h~p~{UadKVWxAVU*`wHurIGIx5Mny1 zPT}B?p~b|;idp70erg-uf8YQWU)FE)va;0Py?ckdxumE_v!5JfeK%qi@$H)oHA!1f z_T|f$mJJ3`Jpp>5-)DyMc*DxJc`Lm=B+HvOd#hY9Z+<%Vw8yS`9c?z3*>{wlKW&~W zhK7df+8?dWImAUrw+pUYw{Ci*h__?h-m2;Xaua3#wJLtGmCy5*uOfSK{%L9UdM=wn~=_^3{Fw;)OILBcsgO zv*)d?t-qQ$jQ6JUxdTVJu70_KV{>o*<#Y0Xpu1|VaB)Kqa;WSdnDcz@itB^fwv{Y=s?$oJMBoU9lU$$N! zS*t`s8&5EIW>Ab=x0RLEG%`>s9?{IW|MIgI%Y1~aI|KA&XlbU+F=n|_ z9QyT>tuAIZjK0sF?I{dnFfeXRGnp=Pncqhh9k06hNkYZRDM!$_E}TZAz3o%FbV;D6 z)DW3~tmyccZQ9FAY<8dd{{6dj(V?OWw)x+y25|p-iw;r0eEFi5ZTrrbTfN3O;IX^= zoh%UY?BAU6+&%sK)A|8B z$+`wdonHB|$<)-(Q}Z?uiP^vf5^hXCLCb0>AQ) zAQIJShp08H*c`8rP2fv0Di4?5TkF#HR0!XenCY+7QMI$n+``IgLQN7hYca;|Rzs9c z%zv7j8x31n+fbHf(ik^0NH3E)cP_nlhop0M>Wz{>CZ~aO5>pZEg{U>>lKXPTDrE8A zb#JfA`)=8`O?|g#0d}pjvvV$R?>IJnT?fzp{cmvt>$o^M3qt*ZgN^5qK1`AlV`2{W zAzcUgJb%7XN%GT!_353(0nACpD9kn5^CR1^v$ZCsFI>1#Gvque)`yJrmZFMEs}|cA zk&sY|RFzCooou$zE>990%=kpp$!s+ z=!s)PD5Gq-eUs4nx}guzzqa9wy!_K@x{X;MN?MYmMA-NseMqffmxWxp^-qtE8d7(1 z!jOyj>4N}B*RkfRE`L6;xpivW_q@gm#>6Hhv9Pejy}20GFwND|C(M z!Y=1OKb5t#w7zytA@JgmQuw+B^1pmhMYKDAz0QIF6kEx-0lzJ;&$5bT?MJjFa~J)^ z6ezkZGe2x$R!^2DVA);HQ3SZe>ClL3dRZc@OmrZnGA<({gX4BK)HdKl`Or+e2Im6~;S?-Jq5dVou?9l!VV*2HK#Om0XnXjYt9Mum( zWimqcNKb!+L`(DX3aSc9c2;i-wkUl^QRNK2dAxjfZlbY_X(b+TGGe~i|A>5=J$71F!0IGi`^dA0O_-JI_t8@G(Hno0vW)IhPXR%_3^4e_oMm z?_Pv-&UA0=#57Qx)A>x~2@})}rI>Zv8j z^!;60IrgcSE?&IOS(e?-u7Y2=KHB_0(Kr{5;Mi9uBjANi2F^LKy!^;iFrAnu|H+(RNVG1tL^G6eB$<<*Mosf^;{;*v#~ z%0hY*>mLu|xr?yC(lA@+4A_;>@@DpadJ;dN&v4d1hg%%q7LNS#%Bsz5!3KRJ89<15AjO<3`C@+6;LcnHFgQWbp zI5}D0L*H>9Mu}bhg3$u`Pj~A1eYfOVOq9z)!)|>$(4ZIQM42jh{dSk;-gr&AGTMYU zl%C$$M5cyB)Z!g_!7qSGQN?<&qie8tU!lAo$AUDxy?ST2^Pj5@bJNrI4y`D>7XjdQ z1KIg_dv`xSv->#(UBaq(-l>8Q&t|W?&Hh~PQhsN>#C|FZQhTUf=Z={_X@_K0G|cZ2 z?0)sr{T_1p-+Z`Io#*W5P*7MT95YaIH&ne2+L70ZdRm#JTSOw-;OEDC2M-+h9)01J z+qW^Y0mEWnc}l1eY_?6?@dcizXW&uY$1B(GZt69+MitSmb&`{li%m1P{^afNA6-{> zSw2|!0*(KA{u7SAb72yiG)ojUE*>5Q+Spj-xxRt)R%M_wQbtS~Qm8;3$s;2p{gadT zwrRbc{5iWzK1fiWK9Cj!tP+`&bhP({RvwE{zz+J>(bPmFe! zR9^QKkM#H7EhbRanr(U?X%fC+r?I~n(6C5D`+!Fn)`Tf&A^W-uEgw@t9IK; z?K|U9g`12HGR>mW#_-J3ddBbn+I8YZrn!{bXrLyKpE%Fyw&C^I_%V&9bonDkj#wu} z<>t2RZj8XMOx?f_m87RcNdq+Hx9F6c5g5WApOBE1ksyol^1JyIV#lFZn1TmV1zEqNKTAZyyFR9Y57jG?5mt-C@zX!h|}|Oa!<(fpAp!yb!+eU zGi=q!dYtKv@#2$B$%PhYE3hl$?M6F$-hN{8DrkFGq%+T}m3%}8v=cAd~0_P(fKVD5CiI_45>mT|Awa(aJK5rx|7I4WO$nJrd1 zz5<1fhf4ag{;A0M`RUrc-eSHo@5TXN>`3IT*TB=NioeZ!9LuIBhY|tm6t`(wK1bIU zRCP{0LxX3=qOqb*ywf5G_s8FAoM$_H&F1>E}fIs-XLMpAep% zt%1^z<76|3m3c+Ls(j6w>ZPBTyUNo(7cQT_e*JpT+T+|S)K1?uG8sm&%F-r)qDXv8 z7x84aZ)&!zlyAtkOBOV3if*=Ulm~NB8Z2^$20<)@kHQxxoRT z*{*nd_0!mNU!p86--FdC{qBy#S6pinG-HQyXB3vLT>Gr7D_MObzv=*s1W$TZH?>%w9&Xd$LO}h@CX^<91Wqiw=YLU;QUwI^X zt#la5N!M~3uMTg2Kq(0kQ=wU+tQ(byPb#6@?6zy)C_Ywxg0xZX3&#F6JPLsFq)l?0 zutVDri{+S@;Rl&Po%j7K$9}v5y&dTYd(f3vn9>Jobi#i6pA2X}cQA!u_C1B#Dv?`0 zliG2s{jAeyiBRB?>-PsUyLMBP)Hwi{icJfX466^utEI{~O^0ciKI1ch zQ-^6VP3c}T906&lbG<+AtWT@mapc-vbisK0RMa}Um5+$CY3|gGF9iigC4mB23-dGM zv!i9PXiCk^&F$x>y6#AfA|UI`^_QTo<9=PcT^44vXUt7a6U0OCq0)C&Qb*Dm%&pBR zE7X{U>1IcZvwkieb~mPjj-#!jt1%sPU?IpUhWsq?z;m7ts-mEPZ*| zDyAwQ6|FZZWz1(I-(>Tr3x2Rii3D(c|c{lkfH%dhJDoJ~G@^uu5Fv?Ada! za}sY(tQW5e)oTe03=9mfkzH)%(BN~-@Q+IvR_$-T`DtB)(`2(%SBOo_%a_N2P*a*U zFYPCh>LoVgDcZYVeEv_H^~-+$>mJ`Psc5nz-JxwN36s=LZ?+Cv$+RmPFoWOVz-Eu+ zOZP^%>l*|u%yvoA0@GNI4E-z);OiKj!h;hqY2c^MfA*Q`cLMUH5kgW6fg3#0V8p9= z^TP@?09T-E8*u>JKp`_NFP=!sLf6l`%nGXm{d|00BmGH4JEWx@4r}a`NT&6dxwLIP z&mrRQdE*&fANT!)A09?5B6y8`x9oB z<1Syd;aml-g4J{D9)!oU&E0n-_wJQ`b^2+A^eR@J1z@#vq*=!1R#H-m02QL|DL(Nw zU7)#EnC^B&?q*HXqgH=L4MC>EERnK7xlS|pHB6Tlanv8if|dk}Tm(QwCC7hg?g>-7 zavjU&Dse@N8Lx#lZc2&;Nh!oIi*Wh?FvtCbyg?TU;9w5 z>_~bIW3^gj=A;m)SJQNLi7+#|dffKXNlre#i)6*|IW9d|{u)BhpkHZ-KN=yGqiz0W zi*?RIr%>Ma5~0>6<7;nHQLPCn){JPu6A9hQqY*vX;&2Y^iGBO?GgYxFyyQx`2cz`L zLUFHz!T`O1Wbt)tdH_;|X6sYtJqW8%=Y^S8qfhtOsb$=J0Z{Di=NC0LX6f)`!-4aX z=-|@}rnVo|d1#u|!$Bfpue#m9+Ri$|o(#dT)cg*9?h`EO!QVAo<_zh#Ye4bpa6U%~dOP}z2yCvthg4sJ`8zvebD^#O_}4ms zhRw{(ud=d4LF4V;zvSBBU8XG*{_6@U-VBVjOFvsub*BeyfFmzROW!`W+mrRPHN9DW z)7r*;Amu@U`Rd-2|8}-I}CEg%l zql~3xN^fs(Lombh2p=w$hJvCFKkeP#!H2n}ND~0+q%rP*)X!5q<8|`*6?PZPfbk~1^+x&DmK)8d)4l8Kp%CvDkRXpxa86bP~9;k zJ*jBf=_Yea6Ii>Xz*BPmPxw1PAHVZaVa{1v_C4$+{zpCymKu`%CM*cBA#&~WY65c+ ziN|BrUZlLPzF-t^RB$rDXK9#xR|dCYxa^Ux?NT(!51doXk|5lq3$l8WiYZbEt+^?% zhvg(0wgUNG^-3oRd$@Zk(I!FgKcGx4>wb}SfHr%36s7Sz%_WX83LPnRWc^s9KVw~b z>WZ&N3U=+I60O6FA*M%V0aVp`7WymkCe!rhtXCNs!u|G*Gl3<2d`m7x5x!-2O`GdSOpDXD)#^>uj}0!Yi`GhQk0#2jKuwr%Ywt#ae|qs z^QLb>bMOJy$%h^j(PzCV2@Fz>hsTm6pa`xqA$Qu4v>Vc+Habf`Kk512q^JBBTz`Z{Pk39FofpFr8S-=Ua#nqfALoMmiHXI5#)svOZw;PrKJS%lsXB%b&KPs=Bh? zA!_k^9v&ef6(T-S7B8<}{`u7`D+n42<*YLt8989gb3j>qD=DcilU#@+34}Lij27|x z!?~_K)i3J+PRQcGa=ZlTBavd{UoF+AA z1oF@=+hY;gSLEHW=(3BpznQ6F9bQFIDiv;9J3iQ~?!Jw;1^^=Qi~F|5MrCDXLv#Y7 zv%f!$SIF|#-+9gk-iQUs=37w_fj$OMia?`cnX6I^DlLD1Sk-v_8b4EK+Wp3Xv}M^e z@*SZau5_1$dF#Fv7uS~_InF`{-N;TQjkhH*5w@)o>EcOG4y+<$*g9jRRwb-{+ArPAX~e33ElQz^3Z}Ckx!(om9k=JT&MR0m6q=JW=2_0lm^)$OydpVcaYJxXe%K zfoC&l$}pb>q8hj^At~AF9IsfZy!z)Z%W`Q^U{uP~s=?g?hKtIDNqy2OiLi6v>IfQ2 z1>aHE7IqAz+|`0c;py&I)rdm6RXMn_7M&{-XaqHok1HEejjT{WyDQI0rb>Q-CNm9M zH2&T7f?fciswX@6(qf$(J_V--&J@esP8eZHcRd=mW>iE#@#4h+6Bd^na$JYK7@q5} zUCf(l9`K3>fBOix9C8|h&1t}}CG#`I|eN;*T z`XJ~f9V{aFGmr_Di-h6;L;&B2w=O(>MJO!#No8P!N@zN-BO~uSwG?`BK4>1oV>YwP zX@LTg4|Z{mrPZ!0|Ena zMu)HZ@dQ5SSzCvY;NV@ck&ztz^SB0pF}-{7pFq~MCY&frii3jVpbD1Ky(2`_D#j_8 zUs+SrWB~jP*iCfj>!w)PRna~vD<~+GXYcb6{)j+K0T(upR#=@Dhm??7pG0zo^w5K% zj-+eXt_5bd3|vrGHv(1huD9{WuW@V&OjqK4i`acUG-W>%ueYdn(jYPH3B&bMo?pT)HYLXrn3? zkPrscymP{i?6h7yiFA7A^I4lQw8X&t$ocfN#!nr$7AYuP;!B8pNBZrdqS<3t2CbDK zFduQPNz1cx&lGb8{=>tCN~iL54k~CwVK&i%gUoWjlM;!(gEV5uWY|O}9xzn{5l873 z`L1Yw?egRHD#PJlU$TBZ<^oTo-)a>GI7(;^BDO#!Cp7XvtpCCe5FActvUgZSyaOFw z8Jautjq6T+wSFeO$XAe{|7CRR@uNrIv8%^GfcAjg&{wS8?Rf?pD_%KX zZHx7;tx;4j4x;)rK&?4Z6F_0T^>yzwx^n@83JSJ|EHVTVxepkJaOpdijV9^HC(cRg zf&TVKHH!vTlXuGZWJ^R>14f*bq!8-{gDmh(cRIB}sn9lCdK>{G!M-BPiLW@k6HzoV z(>-iE2F<=3)JgTmGrRVpy#TfWXSQEZP&q2ZIuI8tE{+sd;?V`GfUC#Pq&U_(rX7g0 zW}2?*-a9tn2EeD`!te-vr^k9;b$9EF13yFmt^xJhsjIvP)0?*VS^_(EqHuvS0{ zy@Z!9h`EUhqL$~J1Eg1#Zq};2!VAg53X6|(j_0Vid9ur&k8=nLstFj%=)L>k6fdF zOG>`%`x@KA05S;3WTo;Xcb=@XiP$(XXels%`junF*q$s=Pe> z{{8zL1qp9ddQ;l`+iD#Xsq9V2U%x)CExoLzmGJA=HB$A}(@ihItbowHq^kO+qCye0 zTx>?hWmHxWT$eO8LwfuA5C~Cd+Oa`mA|oRUS4i#ebbGJy32r67Xp2`cwBRGG=MV?`d3nzPy^-S8 zWzv`?4@Q$nAXZkXkZ*68>CI!+mDJT^!C*r(UbR9vh#5bsluj%Uj~>12tDi-`!zCba z2{bSi?F*i}#l1s9;!#_=gmzsH}v2>m@Dkv~_b} zmV%6oqK1ZsU$i#o=!w|3Z_ne$Ado4mss=cPCd9=lp)W;yrgZ-Nl?5~|^#xWIRs**( zNmGlghZon6`-0NO4Y5-w`|)`@cI+T_F*UV7lGXbAjmeFI34o1E^Vb6{JR@8VbVAZ5 z>ytEER;Gdvg9WC4)EJon2~bHx!?@NQ{)xHfF1fJ>ZNs*kckVP=5Rj1c&w2{W3aE2< z)vngQ7U9&<)jfbW2lUV!bVvXnw_?SLh79vpSTwiXT5$jUZE4ZbXOVI);{BW?65zA} zl{*)jGL~~AfjzkE0opdQQiw_t45g)MWy?Bu1>=@>i`z#NRi?Aa=#Qhsj4d}WuL?X{ z2nM&_)QE`B5@{uw8Sv2{vJT>(KogUwgEZ|@7V!CuQAV>VLg#>lL`$yoS=d&NA3t6$ z@_AQTXW&PT0+GZX`-%QH(a{N0e918}GRVSUQO{$oomybhAz~gDabWbFh*?nII*WCh z2xgMGH1l?JW|aiLk`D1iPRoF( znZMCS3-N2EJ32K?OSfQuz>1-QTRVF6XnzlA9I{1G6lI&_^h~v4V`7qAm;;AOKYlz; z*D4v-P*pW>i3)71o@ElMbp-(sZ*OX`R;=uMq;ttTqZW4&8O>si5rA2k8|e z4Zo)xkxG;S5Gn^Kx|8wO=~{Lk{lW-(7WNm8JGtH>SJi0f@DA z0EuSKgk`=xEcs3T_kL{hfPes~V{ez)ah$!zBM~gkY$0^pz~E}GYhPdA3FoW+45sL? z&TlY=7jMGT4IKa0WzRHEIXY%zjetKQaC2h9X3kIi*%~bOxkTzlod3J)dlEfKeP z@ZgP&#ZOfdUhjh34_>TA$XJ&bXy^y9eL(u#nDEX63EdmFMdB8aieye~Lh>X5OmVL> zwnBx6x;zUI@W2Txn@hS;0NS$QuOWiak>)>Vd>CZ{ujpbKu;B@3h8-Fm9)5-HyxDRk zikI~q+2V*1CM|!qY7X;j&aM;+rS6d%-jVKHqM1eOr&s4_B|1r~YHG$KvxmDzFh&Nj zUGIFy*yJC66FlBsqZkw1cNGf!%q|nR@=}X5zGvP@O7NYE`S%{XA5st(5x!NWv-Gr= z6yKv_Bt5a;`aJXU4A^N6O#T7av!N$r6+`AVSdr#HauA2j{5!o~0i*%!U?RG^CXAtcR?fYXgR)Ei|vXm2kM~lx{1{GINx_Iv+f4=JunsPw!%F zXG5lWMQN}esd!;V$eMp^It_FV+xCAZmcF?;eGl`l)D-Hs$cTu#q7tizqtlyFGf>tf zAxwyTwv!8@#7*V$cX=Uc&_j3DCT{+dYq?Ut#AQ^ECK;(9FCH|Sk80};pHhc&)OMIM za;cx1_O#79^XC=F$L#}8C1LitpkNg%{e@auKwE9F5FYT*apsdkL$@iYWk?ZV4uIekd=w^zQ9(V$lQ=t&rU~^gvnz0 zEuGmpR(;Ad$F7;a6zF=27-ou{K@b8i*nG`(9gqe(dVvT;Ie*S`ItVJ~>xc-V^i4EV zOO>~Go;i=&L$2uK#*{aG1Q}haKC^YZCc>>MlLUA&kq_;bf6_XY-QklL1I|3Z8(QRos3IEK0DR z{NZnuijuJ8Q?$`l5y=WC=B@~Jt@1j5SbBqno--;htL0Ifhkky3vEkw2vTSrn4aDP= z*I4XnSvgWZ`%DO1*U@;BPNzd@@B?{Qb;a_`kCgn9l9ioIs(MDdnH`VTZsV}KF-RGT z0UaFjR)8^(WLznlukQ`E#?FQhj%5K1+HHNSyM_nW#)x_ecM1zFF|wHZ;mB4j^3jnq zaP*#tN{}v^vJi>FMjC-XK#_cWm8LI@O;;a@-yA~a>lzPcObZ-3t=AmX)*P%|?C@jO zl)p>g%lk+r5;j`KRI~geE~P;%z&n3ySeO8p>T<3sw)-3SgX}gy!GC*7u=8SX2D1;8 zq=W(oGm3hOm34JSuCyo?Gn24z)(&iII3`$N3!vb{BqS7O)zQiwS?$jFv9rB>KR0)Y zL|TYVmROlk>8QiCYcHLcAC2DFq~}&K71CoYZEX7RD7T3G$6}3DogEzq!1E1$zv7@a zc~TLsJEu?3nZN|r#~aJ9xu7?aA22IW`@OU2r(AZBy;UeZ#LA#_S!w&L(PLLl^$&Ar zm&l(-HQ$(!nnneZgjnqquSXLqjYv!$mqi|O8htoAQxPl~oR|Fi$4NITi?CS(UoB^; z^0CfzIdbtKK%d5(6#9rj_(p&{&O$jOjZr65(jDH8LS2j&jzPj-VPc@4bl+zwNs)5h zAh(UZbPb>0k4g<4f3{|hhdP03qC;*+x3NX48VPrnhUIH-kEF(qGHt7*h>==#8s_sl z>4}7LE9(BvUm4-y?-SH5x)R^0Gv5^4R>X1Gq_@G+qN>4^O|JGr=dS_{{s6j$oMsTi z-BD9by*1(RYJl_}x~tZ31biC`Mhfx+eJYK5BS5l?^+Tt{kF{xOk?IGSms?mnw9h`* zbyfCB_{?R&Z6$<>2>H-??0Qz`ktH?0&SL>r zUS`UE-Sw(enWOV%m7QNy@=ji__U?o5BlcU|d!uc^B*@JhJl-!iAFwBN>ULcNJwVRS zrSri>K_l^3FYWX4&^2q;4A95q zngi(t!Ap#~ns0DdzcZ-6qUIMs7cLn#EY0goC_@uO=eek;n58tiPb;xjVq1Gxmyp3b z-K8cRFBG-3c2~uGoh`RcJ8ftp^r*D&5!Odc>WYU7vl4&T8s_&$GzE2ZbS%GJPdj4K zd?U+~L_D3%e5G$m^FryKq|Ez#t!D98vnKgmI&B4OEh5F%RxbUxD?Kv7VRA5@NEXrs z3HIys0J!N};-|DM<^#lJORP?1m|jVH0EXPm^l&Y6r`ku2C=qpGe~Yv?7G0V7%Q@*U zfoNwmt`1%n>e_5Ea3euHc*zC5q^97}605Va#g{ER)kJxCN)wDMK6#3?`ZLfo>p#xk zFDx$ZzxCm8QBr>mcc;I#wSiG@GK{M8n%A9+{7?F2?mnV_%_UU*Ain@9UDcifoY|sj(Y9Zq%DTY!#ZUzK>Z) zsCiXT>!Z%>ci>JAh>6jQSc_2qC&n!Sw4B!cGA`72F7v;hk(P$CmXgM$@o86Hr^e4& zzAkeMd3`nW2|#?OnTb^93i#hOlh&~7;jzzN$ubq_8pyBxyK|SxuU$7nYP1#thR*0E zt}CS^CC}85hZxIa5pUw095gDUPG=MB5Uf2TMFHh;oDA2XlY9;T?HTMd)->&|@Ea?a zJsuJ+<&i25(N?@7#n;swr7zhPV%3?j!{YX5^A?Ft(^FlpAJ)3o9bs9&p}&3CVV~>i z1I5i2rJ{mXMRqToiZ#=o=5?VCTde4MG_cK(!Klk>dDnf`DDyRs@|QgfqQA6TZ+)z? zL$Phrb=0PSughX|=1Wz&uBvpNW|o?3PV!CbAUzK~3!Wp7j^%PS7C4#(j;1F|gcR_O zHysJ)_9-ztX>ly5&%5+iuJB++8^6oEBhkHKXqJv}zoV(tmG^YbLs!uBeUKI_;83@M z4aK_7S$^eiu?M3CORbOj73r?(`2H@smi58th(~RJv#XU*Y0awc%N%>|hm>iY+cT0e zUuRc5wM%SetDo5J)*vwzm8@Jr&Hjq%WzRoz@041NE`X-9{w~u$L{a$o!^9VSdzLJ{ zdtO!b02Ofi$xZ4%|KytR;17%FvQ2xWR`Y)+(?1KZlUsl5Ek zf=K$%S-?D0QqZyxKIYj-IieP)z)BWXay?l=uRZQWVy)4+2M%n(od*K2*I z%;s5Yqu#WIsHT6eEmuWUv0y-A!cu87+s4ID{lQii4oV1N1~G(bL~Pj&!!sngN5KjdtM+fFMyEz;{17ZgmR_W9>0eo#&4 z@#8Yy5u$6wp;GK}F{<>giiOqA7_AfRO2N9Gj~^?7y>q8-#-be(o_0MQtex8_OU6i` zC1iowO2wW=L(3&5M$G+8&w{R4v@zr*Z#k_VuqTVU9Bt3?6{SIIuGri&rLl0HIG!4Q zvnG+cQbhftrDfk1y*WWs>F4MLy1To>CnNaMFT)`5+FV>nAvC44jxZQ7u7lFpIg$A& z@kKIXi2oxaagZ}GY=?CZ85>E7IqTNMFU>>4C9HGG`6!6t__T2lWZvTzfZa59z1+*O8@k4?uzRGW4 z5j$-or(wDE#U>??Vk1+I4N1E0cmYNuSU+r9%Dr!HCI7Et85>JJnZ@fWPX#**m*kOW ze7_~Z3eskmg*-UnxGrP`Jg~+D0BQxA=NUsa(1| zm{*qOK-qlG^Nr_~0HgHcqjOlc*WXz@Ha2#j#-_#BZ=llcm+D})2c$tQQUQx~2s~^m zFnz42JHn>T0&^XjAJJ?_E|r8TaK>>v_-Lw`_?KEA%a&=81ix>0A! zC^09J*u24iLG{1pl{GzHt33{9F|oCqZY(E`Np^z=6qSaeq(k!Hn=VEvTtdrPo(w^M!7NyKlgaKtxnYHFWaYU~F*Hg6w$f<}rctCjju&u?vgw`#!-`<(%s-h+yCw zzeC3f4|ACp(GSp7C5ka+ig^QE7T3K7S|M^!D~PlD?%lggyxMvmKby=8ShLz`vc!S} zC$6Gd&bTj8@TnnOS4Zovd-g!^Ga8?c-?8juZ`05^YgUME#}8|N)rp4o2K6m2*=1R$K~XTnd)8C zb0A?6`P%2%vv9a>Wif%^Mj@Is2~Q7^PXE<@hxbTGIMX*Bjh{QVLHmTB)S3-@?k*Z? zfv^8dn|~W%4bE0ze(eC23`gDR%!4I>7J=TJI=}qkMa%$^5eu}w_tyT|wryJH}3;`wTFma z4TA=cPC%4gz)Lv6R+S`TP!Z%pF>JK6MG+b+{A;ff=Z zznJ2DjX{8c5Ao>Fxv5!r0#?w{4PgwHZp<0Z-w8Ij1~xohvwc)-OkyU1cVPV+IBseD z81R8b=L8(=?`n5Kj${&P-=GPbNfv(YPSy`H7j}O=TLR4a)W2A?BS4oFu-XKZNk*WL zg3zJSDNFdZdU54zWirOl3oecqKqZ%@L9eva#DwTHtndO()0U*%12@qk;54pKn7unn zE&{E%il&thjFRP!9p5CM%@A`6jHRp&0TIe5# zcJ&GlM|BiFGOr9z#w*x2`XFK}1C&MLIl!~*zy>Kx8T~x)1`{qwm+CS83Le;?qi<&r? zJ7Y?Q`_74vd*3Mbe*yLN03>LkX3LA^qcRnkXo-gHQH$)lQ0Hj_a2xtP6|YNbkn4OR z&xIx6a+{GPxNA(n^pNR5;t7WLp#sw%F+lHqWU0v*n?sJa%iJXzcs@^#M%g50c8;F6 zm?@JF_0b^m49>+~^kK#Rc8nlucL{)5@03b;QE5#%j66PpT|cKOI5xi-DA z`P$OE1VO2}GVGZ;yGNF` z)ny^4S+@>)=r{!C9t3GM49x>`v&i_w#pn$(7QUn=Q!8_L9=cHrY7vP>HknY&IE#`j z@seIRQP6@i%^2_6MSO9;$30+8HF-wC2Zc@X?!^{}_UDE{~NauAVB)32f~f4|N#4 z0Jk!k=qGa}Km|mc&U2m{gK?w>W1)l{fUH8HV&Zg7LyIM^ZdTAy>LY?VFy0LbUU{e` zhX_n$9syDQoI;Ez4VYMGrxND^m_e(7)Y>%raC^^!%6}gx!N`$A-`w2X{)59O72Ua| zV$%OCs2P0`pU*qr^p3_^v(VbXNI7{ZOvjF0JET2!lJgklO}m)ER5Q1v-Y-{cPO0MU zyBGba>jPzz)OFUfiYnpGJjlitpQt;5^6yubGX7vk-gP8h>yZ;QasUCmC7Rk!<31q_ z&u68_U`71G6bYyEFd0uElVQ}qme2YjRl%|vgUkh;WpOywgwg{Eov28#unU3`??%$C zf;aLFMhiiYtFMQp)~nVL4)|48SFJv+6RJD%>LxK2P{}@N=(ndyW(s2N7NeWue}3En zW9>sBPD{*S*2bwOr$ESGw@bKuus7D+?IXqLIQvdANl(0Hw0`B?C|dI0Hg4O-NrmEQ zV6`+tK=TT+kDk=F%TlXxjWiv-+QoYOKA|8f&Uclx=MXphn9nu7xkv63zpG`&I}}ZB zD|u}@n^4tGO}#g-yH0-98nX-I94?z)vtn2&4xAVnIVXN(GHHcYGQ@GDkQT9wNmzV8}CKMgSxML{^$vs_~2(aN7(pXV8`-nXX|h5ab3oAo_K%?h)W;VlUsUu$tbB>Ze5YxKG!V)w*d7-+66-~5Gbx&I z^ouUcjt0d;k(e!fizoW3etdC!iOheYos}H#spQmGZJR#aObs~s@TyvbKhJwPw|y8E zAtSFjTG7&->ndgPpKPfBC42@Ven|oWuS>W=D0ROgQ=1G7{y%Od-w6-PsLQd(3?nmx@;}YVw|rzN_9-Na1dp-?}&Sa^aSaA=B7j z+YaPooQXDl5&eoZ8p01)0!?|Kem`K)0=L&Q(XEyF^fed^Bj~Z z^@m7l#8Z;h6B*R6yzV~ras+WW3Eg(8WZ8TEb%~9LuG+DPh=^{ibS-x38%%S`$`B-zW zDn&x>V}YfeuK)E)@9OMQV&Px6)=bUlu8dj=`;uR`GS%YP{%tDv3J1+E#+0k37b8o$ z{RT@v*~bbK3tuJ;?dEv$DN-#i3r>*3klzcnZsbp0#_)ppm2`{Dcfg#6Nb%D@AF#P) zy(hJtoeCsqTRzRbqF(;}0&K78>c@~_#L2B&V%Wp5kj74?{^@Q+R)2oPoO$=$2@E%0~m{g05Fn&a!xJp#=sWq(Tic@;aK%zN8{hx*Saz zU-+6sdzM$4{rxX&vex8aXz5*kR!HbZ@pcE22UHg3=LQPkcRi0(O>E!mCH_cZ*!T$s z6zX)?m^l@^DT}jU#Mw$ED9bcwTp5xufwR(m>E3tYV(^OyznhN}KdgYgdr*370Q)d> z1rSc5T6C3Gql^q#K~lR>78*|uOL)wcP?ylfr2FghBQl3ara6*JMP-54Ldi>Hc`{Fu z*}2DCxQLOIoQ{H}tR*NPl4dXL!pfL9Lz}l~+$N<8$mt5_DLEq=#us4oQ2@#%vtJm1 zyz|Xa&DKBS{?$!V93mZ?w{+y4$;~uPVlk5crx26O-6@-Fo_u_9_q^@fvt*Moj(4zr zJDbtwu|}=anr`*y=K~~*Z(XATJ7$}CFLlqgKk^mw|=z9OU3thPUH_qOKc5nPFQCx$QLre(Eh zdZ$^&Dg*Ys#d*dArUOG{f8tOgM_CwPRF$~1xYL_nA{Of2MX8%+o+6)s{X_IA=)&a8 zmWv>6VJp48WDs!VGTx*YgXK!ckGsJ(CxGKga#7^zUB-o9P1)02KRfAbM4C#Tn+ z!$T%UDK|D){@GJ z8Sx{wcA|6xypee?SPo^;hcc?A!^hOym~NJ=D+=4C6+*rngE5SOtx@OL$e9II$RO?3 zGbh54e#0dG{8&ahSg;7db!2*rK&nM^xdcx6;Pxzd1)(!7MvF!4*Ta2_bG#VNsQxqD zW`(WS`>tSjCB}2DFtxG#se2M2qY3X$Es=j^B$Q@vTigHa>&7=!9RSC z1U*iR0ggC_r(m8%?8*o95Bb8uS+7ak0bz*D_x3CAP#_8wwheI$V;al;Sm@S&TK|l= zw1}g(O-FB{&C6C(b4O6l?^}kK8SY!l>Vx&_}Ex7Be_b@Y+X^(7G`?g>$D8qGbwj)jj#_#m4l%dOW*h0h0G>KXSwy1H;Kpnpc*5kmK6I*KNQ5U)tmN zZ`)(&)}K=V=uc+wFTRdJs_W{eJz9zB^;QoP_f~+}y9#3c?0>YLwfWed^~?F1RRwC7 zp8Pxa{?~mVBCoEz3ueOISo)uLa8>wBk-}yV()9ep^7qr_t2dm6|A_Ji%2EBZ2m{}FvhcX@2K1I|9vMIs%R;2 zg#8ap${4w2zfbr&U=4HhjoQ7r_s^1rdLA0eL&a>T$G?oNRKzoeo>{fOK8k^y9e7Xc zSJU88aSvOE`{e(>?~|*snt2bbcW7`3qt=-12@4#V6lc90Y9X3QiU zgscZ#+OSvGppYCrgdNq_x-f5ZGBod1YG<8C8O z#fGx^+DDLdhyj;?(-SMA9=74B~&>$`sAs6@TIU`8@~DcP5k~n6TPjVD&Z_ zVB;MAH$+?!PO#6P-L~%4N*pmfjEmv+ z&|5QS#X}6{p$hJ$+M`su`I5tXtYcSF2^AM)ED14znl$$GRk0P}Bzpof&+6wiiQIULa~xluarjOAxLE^V?BmP!g`` z<7njRm^^4ji29=CH_9E!ySaOII84jDEX2b_; z->2Ww4qJRe{omvq_h=YjDcZWAedf{Ab+dNIXpHCnIMQmls;j|zD#|~VWUOGJ5AkT) z8_ayuHPz4r2qF_;Ipa0zwAtqCWHKD_q4w_Dy?<1ccwihkax2KEvFM>4B(Zodog-Z_)^-=gln+c3_7eI#8Zdy-uhpoScDy^&n>iY(D$9|KyheYeE25tk|Xz z4rY>^RX})HH?2EVU;p9ba((L;X>KcQ6ImKL-w!xdmQSy=3atgEZ^#)n)C!EAylI{7 z6r#so|{_x#gk zqk+u4Bi_{9m*X&_?y7TR+H32;C3P+zzZrGw_Kp>s4%yi$UVY)BYGu{u#rA*rdJk|c z`|u6;u}3meWJ^XiDO;3eQzCnm6|z@06)7Pz*@`54L^jD5qU;sQp2_CwaG$+(QHWESw>(x6dpAg2S>=%fkVPIur@lb9Ednn; z`7Y3#GRl`BYM^!F1~O!;nFqfDq__}{s97s6xh|>=Z zkvah40)f!Cfr5p&DqSuW>2b0*O?3U$5~Ty4QDAmw!6wOU0g4{P4F>TUXe}tZ|BWeI zEB_iGoW*db5WaBALxxd2h0KUqutI{9`^?%9pbbD$ifsKgkXEL49>R~eIQUGz!P%n| z&gkr)bx#ju#30-2{jdNwljr33Uu*+ZWZj{vsMrm^_MYFk;Xc6C>nQz$nToRhp+>0{ zM2YuKxkMedhO{&_(=>IBU#dEhk#JV+SAV{i@#b@N*1LwJwfEV7ZKphkI!UJ~S{K$! z@Sg=oy?K-RNtj(kTR|!E)Tv$c#6;S6b#&2duvtt?EvlwR4tEH%=Ma?=1Pi%{YXB5L zdO@Vl41RP-K!WTnfKJ^6UB?F`PeJU5uI_BXw$0o;(#|)5u+MS6j)Dp4JOxd-&rNId zwshe3j_^Q;gCNBai(E2oIV~~zeu=l+x~$*uQ$frs)E3_ZlNunWh*ZRCqMYTaqh1i= z2-tK!l&D|eS;tg_sdo9=!;3ZT5HXV70?-JYIFw+=>k$9*0`4jEj_n2<=2l?hLM`F$ z6!8m@EdvYxFGO3K2Dw7_lUXJZ^Garji6F^wSQp8 z6jGw-bf{xKLEXX2#TA*FyjYMLt$Pw8B`_QTCGXcn30+7#MaO)xkn-X*9C<%rRU75% zYG|;qcVaT@X)C!MI$zp$SPc<2FrTgUd*fC`N%%E#{VIRo?402)7`~@kX5g5I+?te+X(%L6yMPzB3H~Ssg?V#}E+( z#spyC(+eyv(@@!bP8vwlk>^8_Mno_T$jX8#DAb~uy&#a-hE;T$fq|iD=s?sA9#7!Q zFsc$Ohe*0P;a!H=6@}0p&oOiN1k#Z#>ONO6H2M~pwzR!Q}%a<<~0LBBEP{|ZhLWzZ#>dM~Q zZ6p?6-$lwaz8g@`)dz(9MJH7i0!!Yqa+%7#nN>bFB~)9OtuispsFf}zbCIC_BgO<7(zJE4Hc<(h=?BX`bAbDls$2RH_m;G zNAdH=HSoVfif>5dv=3+E98xPd9e6^7TnG&=Lx>OuIld7b3&OV?2*zh)t#ZDBK$xp} z^xh=mo(Muv#2W=P_6S6CAl00|I1&5;d|m;3_XUZD6I2>3!Jv^^$R@sOZ(X`WtVS|dg0dMb*UBJTU5KVtfnX}! zM2LzA%3{~U0W>0W*n-6YE~k(nA%1Ki1a6x93t*HI2xSm@IXwA7IVSx5&*2x-`ilvJ z2~D?4BC+-oIUiw%&1qgm3=?>(Hs6GZEeg|lF4x}Oa0JWBS;R6IF;qbuFDIMAkfJI` z<3f}Fs(_&rf`fgHo~iiLjzBh#8@6BW$_F839Vke3!7AlxmgJe={yVq89KsBPVSr3w zdY@f;B((shDUxr1f5iLbJ$>-^L5^$0MGQ{M4?qJxh#6r!nTDwk4hNHPUVcZMrocqb z3hKhM-`+k&#viE&2$;3qzsTpQ`QtD>ify%I4<>Yai5NY@reUp-^x9|tPm(i8H!r_N zLmF{^x-$haGrjDwKF`_;xdpH#-oe&n35TsUM><&U4leW{!hIygf%}AaY1v(MFa@Q0 zOMsycsa*yTT*7!1(m{Nc3SdT=*Z>Z277*K2>Pz=;_Vv&~0pUF4YNxrS^1A0CAV=yB zP}@4x=cFU5u%4b-B%nep(ZKjsi=0&n-}2%N5)^?pVs_BBI;vyJ>f*yL0Ipbbdk5$B zak=@~=8-x!!W@d*KnzDwsIn_%HI|iN&L~HTAtwJ1=25ExE??9RD6VcDW0->n-mNwa zbadH>YXk5LQjq#HBr<)iP^dc)@rQ>9j|aNE`UoZ8@Y9r(y^CH4X_aqI>w-&(Fa=It zro{p5$`pLH^V_c>#ON&*$>4Y^mLDNukVQ8uV!{lf)lh_Yim(%gJB)E}t12ly*f<_N zkY6ug6k%+>zH{p<%Rz1<=bL;RzxTx&gv{YE>@obt^KO97e?lBvKxuXZ3Wyo^jExc1 zrv;$+i1{lLMFM%vnT$Fgy3{AAE;Y z?{__4%03N$>ZB6>!)cooYa~ z=BEqBv*O>AwCO`yf58hR-X0}ex+){%57R0Vhf-vk zAc_8gkvZZijc6mEKc6k~^J*g1Ajac1#a3#dx$3{|AN6>cTdom5OrpKskA3XuF&qRR4J6$`z?Oy>cqwuFrnt) zXxU%#l!So*a0N`R!It4YSfH9E0c-LAd<+n|$_6poF<#gOkPgB$fked$YR>@W-|loE zZ*#@{n}FddFxr3WGu85|@67L)l&%S5{r>rAigkJsFk)=k3#?&dN?Z^@V6)Irqc8LL zOsopp1e1P9dnBJ1b5XQ)mS|s$9Pj6o54z?FVy%OpMx>)8)hmVe9a&#~j;g*)5Y+HX z(1-Q)?|90b-Jb@s6DyYSrVc&%0`lF$A`_m$KbgTa2hjPiRVqLaV+sX;6}fY;%s;yG zWC1V>L1HNIgu06tD2=n0@CT$|k9%eU%eNA)+LOx(b^i|8gE(!c<257SoGVVsqqTN- zAwPHWA#kC$T1|9fzTMdF{sQ=lpwWzYA~Vr*COCpx->(oSEaSF3)OKv`mmMD;W3~X4 zGk@%hBsrM6^ExocbZB==McSi@Q*;#0wwLcpf2^5agnEMn%1if&CC;}`iP9{opFb8) zb3A(+0u{dji5{Tq9bjBUvD1fJ5ZztPlkb-vwDY`qKi9+vYhWUR^l6vDE$-1&^vT0V3CL7_A#i3>iql9mqAn&E-8IT7X%~9R6XLl28>sN{uz$j7&CokJ4OV<;Q-zsSc3)A(%aAqKsqXU z4)(ToUzoqwU~6J#7Zemc-iuIW^v55IA-|-&cKDFK3$jNKiGA_=(Q^+Ty~_Jvrv4z{ zb2vzZ|Ie#DpnyKb zpmb!?KVnZOw?g?tjtMxlb+5CJUTU5fAa~&i8~W3*U#Nd!_0h1QxFH2cSh1afg@)IT zZ@p$6dHjQ){=rrbI3(i!&$m(AgGY;e8CaM9dp2{y_0PfPA~*ieed8QYF&JGIge~_U z^Yt8Pq{D%obIcL@-!tBaJv7x1IXjT zK|a&-AD8NcYNmV_grH1EMBe{BDS=nSA`j}(|GFmV-oioW$^ku2^p4(KveiG+`NoX^ zW@IuR49U^#`k#I39MB)(*b7fBC@46t&!MJuj+a(gd_32)qxuM{lj8U+g|LG~9t5QS zzMF$m7x-H~WzI-WX8hOkIjkYURvDgJQBm=)hd(Urz5Hi00+GOB`3*Fn%o(Yv%>QvA zAQFeC!tVPY7dmM7@V|(0hjZ(&X7PVHvIh(Aus#D22(vsW0RQ8OnwpwkeqmHTnuEFs z4v5tB_^2##Ee`6lC=~MM504B1IS|>K{%hYl{DM=bP94-_3b;7`wT2Ey242CslfJ(G zzqW+K4*~)sF$_UR?O`eXe?1Qo=s8rouOFUu2a6T?#NmB{vvvvjnHfx@-j85nhuA+N z`80r7sbEcc>Ahi%9YPF4ETjPj0_q@m)j?e`qV<;=P~2c-*MPTU02r)MdCOFrf>_~? zqs!{tIINn9=_5orD6e>dU-Q*J(2F7ssuj{t>D})a4(5WJ;?pa;M!MTF{7j+6_* zY@ytO>o99TJs>&C#Aj?OR&F2{L2yNck_J3cq&9_s^sFlJ?}7Tl&cU&pA_8?j`7V&b z{ROKQ4k(v`sW>o8;no3&eT7SOwsX;;2e>ZX@biTchQr&<<3u9X^B9j37Loi7yzAbM z?yr})yqN&|RfHvo__rcoE(%Rv>TIgPwO_sqvDpE>#6q4!Pfw2mY}BFR^c}<=^oN&B z7DM)0*d!1b0~H`Sgpmw-)k`)* z!idQ-QaS^7iEr~RtQZAgCWjFGRDHH1z`#2VPyszy#|Y4az)Jx`Jq1}5fc`?tW-jzC zC^&eyUGr!^^hH)O1!G_gRDmr)kFY)wp&}{|^%=FGK`7a3gd zL(Ks-!Zf7xRt475($bkN(cu5&S`Pto-v{fXNr-*82CsgpSpIyX48Z1(K&(X=H0}Tj z9hj>jR7{8;`W6s2b}+4o8UPz6H-Yp>EU|VIs1WzT#T!AMtKvCYz+W{?BIxr*Nf#8+3{};r%Kp219@9>&m2G$(Q7M|kKOF1~RyUt=^ zCtN!$LUG6geOx|nW8%*i_@99AsQWj$m3pKkuN;H_;aEr^F$e5NwJIBc4!wu8v3VCl zIF}_4V66a5)(!0eSW3#hp1Q&lGK0lO%J8qFP0Qu-;dY*P8!^U4Dy#^+SDwk0nK z&*_F43{!s25V6@u8t$AUcnR`RdCQ9-he{=I8$Eu0NbLbod4SM!4xCeY4Ic1m z5a*$2KKKXB=zXSv-`xpx%(n=P(S;CJkIVoOr?Giw@bW-F9boJswm%CzoL;Ao1`CNs z5T=Hhen+CjUsxcrh^a5~zTtk5hlDw-8sV%8G_he=fdgQ$xZnm`kvl9;X9UOsTKFyC zH6abgkTM>yzozpXq5%_s^XzD6IX+%V`4Q66I+Q@ndiwz)n6JX5p#jYfaOZmHZe5aB z1Aaa;n7LU!l{lI=o%+Zi@Z*b;-T)gJW_D2N{l}pRa~uIiUZ{DUoq6d;fD=OBk;?R3kTO5n*YdPAnAZin&tm1V>(EGz=oo)ukSx1 z5un<#$^(D;f2ch?fyZeJ7~X_~bI97ip6pPVmm3#s={KL!u#M;cq5wjE$2$&;`pp=3FwQ#V#_^&(*Ry?yj zsHOfLRvdh+_R=?S>iVz8fdJzF^Kp<93JxxZe?=Gv!m(?7RI>ov{Kps=UHlF^*zqmU znJ?;DG!V`BNP1uxJ^d$41glbJc2QB$qluGj%w@n8Ov9%CFy(tmE>r~N4Qx{tagxiA z2BQH+XSnM`vI-Lo;w{XGVhnfQd>;iK(8fV?Ybq4 z6AhY>>(8-`C=m+Mb@nDIy+}X+sVH21&WxY*jWZr{qU;29!Uy(loO&mMZ~)2 zj`kiW$~ZxscL@Zf?0m#>`Uz{pu41`Ls4~g__pb@zz4O>0HV`yZGK%%<#Bc&C-r<$j zKtK>)1Xg)RYruEio>gjl{iK1*n4jNFeUPh)8)}~VIJJRQ0Zin7#_hKH>6rb|4*kKr zyK0^lKHGmfTy?@w^O(oK{xT-!f;<#9jup=LlRkg0aWXSoE{Hj6@dY2fHt?~w`4s9t z#I=Wa6GQF}x(UQg(6qv}q!mY{q2c-Ew$&c@Pf7xZe;fF+|^3)<1`SxJos>D$)Cyy^4h#tka z`(I?Rh1)mB&`Ht18UDP@KPR0!eII>^Tk3zk4`AIZh6CvVw(9do>Eu98RvTJUeM2)` zlKrCRCm;7;3b@zsI{E(@o2V!jc?ibAzbnuViCsn8OGXva6FDlkEmHyH=%eD|H4s*J5XC|#6GorR@eXF!Mn}3N#nE8}3v87}Ii#ynvAAwOTo+XEXwLb)DWv@pn(?)l1#PUmz3$de| zRO{>gX4=}8=Yr|!VhldnHAdmph>7c%;ytn@w78* z-Fy1R-=r~6P7LiU;<1Xp^TlzOjC_5zGjlPsZ=jMTjwzX-gO&?Fa58~yz&9i;j%N2^ z$uMTOhDO(*?M`_^OZ$2#`D%}@as(^jQuY1AcU@fm%*EbeynGpm0Ae{xC>Ofpv>L%a zd>75?0&3PG*@5V^Z==FfJv55>1!xSR)hht-|rQ)S$OWz_o*tMmXSnXJ%(jdJ?4pbe4I#vEJeaM#_Z@*U#mt^1k0cJrP7 z{!q&`@cRp$mdw7+_BjBTplD~YOW#%AJmCy&JQm7{%e*8&UjA#Y$8?I{OhOn+`=dhH z(iMxnZ923bis>KZ^&nk8=G9I!wqpkt?r_SlK1_WZ8cpvrR6RoHiXH82a^ht$O`3ef zqbavax)L9h6rXt;KWFBa^G3g{g5n>@GBPIrEacs@_-kpgY%~+C3lzB_`wlF|g5v@I z7Y7!cNh|7HHiqkRr&i>84H+KpE|kz|;$NYIpip$J{g})mrCZJt^w7>@7Yt~+ zVDWuW?`meldq-u9JzaHtS?0Z5>t(7pb<%IMLp_19BlzTHWxO07m?EPj!B^Q^+`|jirDS^v)2;_@l#|+c+;L8=R zuBCMtAvh6rM`>KUHm9Rc>kx?c@v2<=p-Fce*Y&L-qx4gkHg616iU>I)dhXdf?M7Pi zm_(u6x}fv&mb(HwLe>5JFchoD_iSQrt{4skD&(H=i|;;O`}tH&S{hcIk*rqXjWV=1 z=qbcps!yIo=F*HGmTvFtXQ#(Z@`j$@@_+HGsU3Woce{P`o42|KK20yBtSWI%q{haE z@ODG7TGEW`!odN4$QRzNZog&m1~0jA**y(dB+zIDk~v6p*d2OUA`>vpgM7(mmc_Ahj&c*?6&ffaTEK}F-l5y z{!rJ5lB61+FH4oUlTD~ZLjAqW8hBsGOyV^d;{SF3MKUF7UQ0%19}QX{blJG=Hv%^7 zcapn##;t|-WV)A5yEkBz@g9?$+0k>MDMfGJ5TiO(wzg_R!CU*{FI3)y=f2zW#B;)# z!VXchQOW(YF@lfdO1>;a2as&2+V+p5t#XHk#u_DeJNLaXQJ)2no+>@gJ{PuhePj6J zL`5+`P^>%^^~!Y~9#8!E@!)_s(rbG^#>10b?BzZDu07{&H^peDwkpjme5Ivw+qn`N z*w-taEOuJIZC>Wl?K%7s{>C06UmFt>?6RxSnF^L;2z!v}<%N9(Xa?GOM2^AuocxU5 zx81D9cdW6}UFIDv$|e40_BqiHbL>-9C2SZMo$ecL588?q(}w$Wx-RbG_|nzQoqT~& zAVEOdEcrWrr52Cwc4B={n)BRlw}^<}7L@iDgN^|@$WfUAn-vZ9!+OlnjYH8{sTIBU zRcL=E^7M?)gp7+^=7}U#ovI;8@GRKhD~WbwT)WAwUaa|^xy5z#qfa%;_g%b1`vY6d zNug>2B_QHHJuTb)4(DBN83&)f-Vpx|Ugy>^+Quk-GP%$qei@1pPQ0b1T|L?RGq>KR zVv($=+75ho8hWwt@$O?EaWZ+ZQC=8y>5$mdPmu$iyXnVVu9UKhfm>KvST#BAueJE5 zR(q(g=*^CfN=n!!7P&j`w5Z)W_ci8cl8RDfPAO|%njZCm++e+S!FZu74uxH|xAeXxqeyu=@RCR#H@z$pPz zg$!;XyXv)Ylkr4KeP%cDeT#0)>#HT07@d2^(-VrE>qRoUddaPuE3&XnFzG)>S~YZf zY24Yy6ML0wGQ3eAbBDEJ+bG4q1P|; z(u!PNfV?a{mDYMaB@A7JqolvTQwC!ZkC3Eu-p7$S3)XY+EM`lO?)Act7k)qLiyYxW0;kB86m!ktp^ThJNc;A)y2z z*Bs~y(Piu1Em0Okc&5e?R~f@ezrBY9hUDGGh#22qx2pAEi6d$ai}?QFl~gHLJ!|eD z$Cn##Lo7wDwfCbv7w&2+cLYU6?PGkivI1@%M9Oz^qjNqnJv4B08{9D0@%^L+RoB6d z2^YK)%+k4>9^0MmIlmc!c6pCC0nXb<> zL*j#Pqk^u*+b>U46~6_d&76MgNK=0mUpq~KPKg*X3-VKs@Mwjyc~Ye|ZVlEIU~>QL z>Uy+PG;GjrqNLj{@w05)Ap~o66m5g{PH=Ec+x0}R#qj%;NBL}Cg`T6#4TVxgA@IpfKbem!*B+O-&Yz^{$8s(n=aN|lhffzJK?+G^|miXt=Gwa#b1 zyJ`=sCgh&>KF>}x&tJ(AD_)E7>u4y|47M9OJ0r)C#gx&6^%=T_>TgY(JEOWIzZXid zx~CF)o!qWH`kwj92l}-RYFfXJHT~;L%gxUR+p1OidRMf6y0h$?3Y#3(>chKR{_q`kMZ)2S9T4$5GleNs2#WJ&8 z2Mg=0g@s@3$SdxM4<7~$ZkW41t$Vto+YNdoqmg<_=#OeWDv6rEpelY&Iqr9oirliH z05teDg4UD|&{u?Ph=!KC6DfREU@tT^*-Qsc>#sXB?buBg>HY@hos4Mo?rx1Mj_=}8 zX2A5Jau3x5x@!*N{vtt$IK6$;86;gc&5n{Hy!-ESa5qX%?+NVBrN1{4vRq&LyK zciS7on-kPDoM3GtNp{id&N8ZI)SHve$Fo=N5tku-3^;GFv(!rtNBae0IPtXmnUIL2 z&NxIoe4h~pU9GFU#K;*JsL;~k;SpG_GyAKGgRNwD#4iIy~d z?(~^pe`ZL}XLMUX1BcOf(Y6*7;Csx_B#-t{|4|UB^!M+(IZoO5JnDNLZ#P2pot}UI zn#$~3^IxVX6TTku=36YIAA^ImtSLxJt`rEgA+0iUM zt9AXcZ!WuB$Gjd6{l{v)3iDQ-oi@wN7jeYwPEVru_A#pfO7uNasg0u)Rcsj103e(| z$KP$YfQk;#W)T!~pm}9=w}~UC{E2<&Yi9*+N;+^LBiKCR$^^LW_TcFLJ8~7_s~_n| zPY$hD-}(&&{DDedl=r9?<#f7pK ztJx!F>WKJ=DmR$uFlIDJ^Ika*f6RSenN_x4P*(1usb8q~U2ZeR} z?7OtRec_EZ5m(mc)SBFh8`C@M`JahuD%4G?Bi!YG-=5lvNY_^@q44uNzd(ES&leV&? zzaOt!J^6MM<4K_CQ)0SnAkka~3E0oReB-`qA4!*m0e>F-vN2$e*6oYNhk+}{W9WuL z2t<3W;s;cSI0kBTVtpU2D0!+j?`0WOQrkjz$S-rLgqN@0?@*O|w)xup$!{^Io@IEU+s6-kG+_@An14&k%Y_QhHRj=mICX`;LBj0aGBCskFTKqd>!AYpj-5 z?V*y~C2O7H(<`-)YnOJWZ~YAWo~~;Bti}%CoVxzksR*iU9gJX_-jR12oVWC}^O=ff zPKAiy6*iJunBuL9X}*imc6Kw?>i(k;M*NB8hg$ZJg-VNTJH#(P5d7;R9S}J2dv`0P z*{IpyveY*lXJe?YT^?gBSm-)N>}@qnE}bvq(cDZo)0f39v_{(S#E>e}@L zV8c@25!2Q{`y0%luGrojeX${sf`fcn3!vAU-fGD8P`<$QKLgr=9B*-M?vu6^E4ZjJ znCQU|6HHrPo9gK3eA!{bf5dN_7?@ic;JmfI_;oQ!xVD1Z{Cy zdN0PFwrFS)^g~SZ{dZ~pmw}HDy0A#;Sqq*hW#z;%29|}L>@*E!dfk&PqbHjb_}sMh zx?Z=Qgr3RXx>5SAVp~d?M8}T5)x%E+=6e<>WEO9}^ogz_WJJwn{XP{q*Nf$|XEnVf zO^Tg@1C;WYMJ+9uX6!UmBm-N0G8zJ z)KmxFr=^w2=Glyj*dj}o=j>sdSf{Yb;%<5e^HFp7k_H%M*so9gbvov0BM`n;9DF#vO49N4>Xh@wNFbLi+pDCVw!-i`!h z#+QLE{P>4L(7k7GvLnvem_;$-15fo0)+cLS!P2m}R-z`iH?!Z_jlST2RDa&~jJ$>c zePLHL(XFB6wN0KdZ5#SAdDE9~a+7aJmq=fU)t7n_UnN~>9It1J-WFAFDqBc7EoEX$ zmQBd`GfU>mjg>bgye3p!BbR9j3*5@8uPG`JF87;DIb1A9H_%uHyFX$KyuZA^Q*(kh zug}snf}w!yp|T13h3XZ8MG0~GTPP?2ubVuh5MO)Q2i|7+f4thWw3!kwu`heiyR4u` z^B*hEz~ZI9P<`WU@bEMa>9e55iCXZR$Ol0cbeG2h)2#*|BJ(x9+DZQmgdn@^H48nE z3)%_OUMZRBFJwbjCsf*!Z@sC4n}4XHkgz3@O_DdfROc zqIqvhVL`k+R*ZEeMs_T_OQ`UR7`oKu$~O3Y0>0l_r7Q)0m72WOWDqsr9pl1mA;$XE z=-YDJcaCd}$Gjw`(6{`B#3C1_-Y?%dqOlz83YA|yI1eR!drwL^oG9tT2s)WJfMIoU z$6WEQq-G{oUV34Y?$l1cr}f5c&>OPzkiRPvINGEdt@qZSF* z3v)OyIYO{heu=o221-Od5YuEIv&T&2Snrrcu`a1Mm#xiU)i_i9ol8G2^s4!!y6fE~ zcS_-M200rP^Ax7Mc8huWm7VzXOW5HBSS4d*PevlAuyhSkA#FxUURw41U!p&U&=jk? z?0!ry)YawiXphG1>+ai&yfH7FpdLERDrC}8PiBz2TrQ`Ix4og6xcsSaFV{ZckKL*PqF_m;dL*Y6bbS z`i0Q)e%JTC+#`MWj2oTz=`8X&ljbh{D(|lK9tpu6?@I_zkB^w-sHXX~@v{lzK~;x8 z0TW}pn$gB37b2JFyb0yMZizv|Pi39rw785KXP8|o^3pgOUzoTg-c>X%S$A@2v-@to zaXT(=Fj9Pd>vHSVk2zDHjVPS?&71a^@5P&KhJVe+%yVe8mvmg1@;fuXY2=7H@BR7J z=sS%ab5s_o*xd?pt#!7s@44e=t(EUJ21IKbwZ-6l`@}D1m~i>T1s>elsIPd0ck=&~ zPu`e6`}2LH_s9%)zN@X;-!-}<&+gc0ZogI@tg7wj2@&3_1aX8b?*41mmLD7z4e^G{ z1w`~-1)aI-?&SD9$yOuZA;E#-z4%G>gnLsxyPNYlsQCKvsRiePq|)9r+R!u~+8~)N z?ZJAH8-MeKrQ;p7q8dMRTvh8cZax|17`8pH_vMzMTY~(RIqZ~kk%s4A1Q0SlxvS^1 z_jeQ2#hFWn&`tFrP+6>$pEl9nzL*5Um5uU~lF89qq_ThAD=~r?eECerS(PD%!6;!H ze^Doi;zYOcY<{lIVP;mN!Ls;T@1G=bGB2%=+(;IyCb323MZ`s!iyhHz#qw(5ZqcLx z%M}JLkCwRZvF)^QnhY`VB5K$ z+XF&k@;m6K+Qf8}}L z_VfMfX->N)$~$3QzNXSbwq~x}6f)bhV?t7cbDd?E%lpI~o;bIN5cXn}j}la^I6mtjDWiC*q(7hF*FA0B;aWBc< zJYQ{DJ%r&5j>5U3vrRO93I25-PEbn+owpqXu6U+)y{oIXyL&M!L)rJED;C?U0JP(C ziXLW{ESF10Nq;8;=FcuPfnQTWhrL?#8kH(#v$)3x;VRvkaK5|3Rrp#iuhsN|FR}+; zKi6@~ko~2}^MI2cWlk#Do>%Dp_PYu2b`6Bri3vJZ(5D$?FgIN*L9k zrECmc^{ml&L+koEx!GMjOJ2(M9~Po$f+7EWa|@|pREp@z(7;6O+8N3C!%n3?y@Ug_U7x>9BZ!iq z&c*#rlJRHHnAqwf=Wg1iQzvmMYHPx>^+?PVl04Fy-ABx9bE4*2#&zk2cCrWERb=0jXXg%eb`46A7INa(-pZv9#L zp=tP&XHjJaE4S;JNy!d+8hnE&P%mR+b>>7$$``+wK*`Hea`(#JQ^ot7yW@C_&xuX( z{BCES*A1%YQC1O-#C`&ZAG33Xis{8&#fW4J>4DW-4=^Q$4#nl}VZw1RYn=Am`Dn zmI5xNDrVk47x*$M6>YZOZa-p@*Y^3EluI+ce-`?*^S~*pIPJB0Zh5|tVv;tFAgkBt z3-jrRntbTq``Ga+w@@>EVb0%vHf7CXHpO1%>Tn%Hb)GBqO0Eq$;p`IUg* z+NyC_Zj`IYpL>>n`LR(IZxmx0(Bc{?EcH4z8g!JhFHkpK)b)2!zN_b<0HCZ5Vr|R| z9GDJjaa$v~^@Nkeb?pf?Gu`tkv(4eF##}R6Uh^(N1Vu(r+6<|=ceiY*+(jPe<>WS3 ziiSPyg%PG=Tk$&@z^AzB`RdMyX8FU2NCG`I-#_h+M%}E@w0h+fXZvT^#(Lv!dJX+< zJmu=YnWguT6q=*HANes>i+V1IQT^i29=dOXpNl@hJ}~4 zw`gv+FLAAIJ*E87Xy%>Mc%oZJFyo2zW5=+hjt~)4M4J1rGbp@joRF`kjpm_>qBq2% z*bDECZSuxk{N)L2&tqY2rsj-h{E5^O%=YcGjjfB?9@~rGse7)I6H=64xS}0#b55`9 z8z0fx)iVttrc+BE8<-4UUkF>{S0*sx`?1!9IiS}vl@8Nl^D_?(VXlqi^nsZZfG2@>9 z9r9~O`O?`2-~l2X+8>J3dje(j(?ane>>yCH>5 zD`;xHW|B}#J}nl4`Xw7>zmEA$o_dC$pcm(p{lLb?o zXGJT>1h4X+IzJtrKUC^Reh5zo*;7gXu1ch4`gNkxX zziVDlowr-5uZb7o3Khw@u|NBa&G*~+;hN8xSd@||Q|2o1Wx+v%YbsAOJ}ruh?f+pI zo7WzzM&r;|O!3ydGw+*9;d#J5Z~m$aTh11vFRGneR^rziehnqvWg9(RjycQ8>JwH4 z9bsDwalS1I6dSW?;=*e*HFGlQDj;PV6ilw$r16b6$coD;@O5S{Yv)J^ejcf~#UFjs zIpL((kHEU@|51Y*%^iEi2Hvmy_D z)jC)I>?qmdq?Ww5-ZaP?+E|bOQNrjk&gi*of8qsgvp2Tne$#$171ofmCa+TbCdPxk zQM2ayRpSRMKN6hfWq|E2CF@M7;s$T*b@rs&> zY9X!lS@(Z)nfZTPosinPYp?X|$?W*u$o%xeKfl;=J-huLLqCBe*9W-(Sfs`rrr zKTclqy?9mcbbv*CZ(97Hgq6MB+FZX!9?NUpkNJGAcua3i`SoPivraT%G$k=?dOa^r zw$i+X>MuJpKh?X)ilU$pbLd-pa{lSwE}h6dhoQiHYnb|RmvUt=3 z9|OwoY|YPku2jD)NEbi{nACUpwOXw-s>S97`F(%KMP>XX{$>3e<`p_rSEf&~BIQL1 zLzFPu({!-7x4`GJ9LAhw`7DPEr+{0*MFO=?(ardi!6p;C8vdWVzv#1;oz^#1Fc<8- zt0{_Ans^zCs7ayaONqWEqh_krTda0Vl_GMYONz-4$;CCzu99RI)^P3 zg6HeKm&(RP;WW#<+tKhs|1-*;Xdv#{P zhpSpVS)W>;Ox@B*3GcjlWwia&ugw&v*+0)pEVBxT*BR1()z+rTty>i zV#3n(7|+jUc%RT|FLPW(qm3GmXCN7bu-nZnJ-50nKGOt^? zgmc1|E%83;5>>}@)*lgs#NRMRcCURJR=Prlq82pqrA^_J*UzjoBZD2%m!B(&-8Q*XHw7{Bb$yf_e0&5+rpZAa z8zR)3Ik}~|(__1|ENm=NGoC)1pSvTGph?-{7Kv3;Pl?6gS@SNMk_*MEx`=Dy`e*Cn zzQoJ;tkcXDxBVzy$v@k^mzF=5x+>vz$4#ZQ*glG*G1$RkRz-woIjN)`cRn@%d^4>J;B@t5N zqB!?9F7;}7&)gYcPtkN{Tw+`Kv>l^&PG2%Q(2>G>L@iHkzC<^Y?K^>&g?HFX1{E9p zqRNqJ(EJ?4w3zs00!pth;^bDYQ-?4}F1p6K|D8<%U%fze)(~`}0pG__dH~ySW#|x_GAZ(oBo*fxM31;0U zq%x|KD;5n{G`YPL;Y?JGadXS&zP0z@^$+E=k~y31C@#0jCNuPDmwD2`JCxRV6$`GT z*{C90lKGL7SvvLaR=Z^PPuoe%^wdfg;8=Ge zz55wd35HLO|8-vhhaD6uz@c{%e~fjus^WT}Z~&!L)3U&a<#s(^sy$q3wNrf7)QaBO zx_EkSro#lE<=m#vkyUShnAQ;t*V+Dhs-U6$@a-IJkcDb{t9|$=-JhsGG zW0=Z$S66*KmQ3doL~e6eKm1M9J!9LA`)QnTOz4kWM2~X8B`S62fltn)VRs{PtR97J zGGD5nJJT3CG_)M@b)tcviVL;5@8Oa-A4DhiMO{~D6pJZVz}s_S1L&_sUQVwbNSYAv zKH@v`!{Ldj5OzK35lC}g9ZVa)e9*|WgZA!s0Z+Co9|qJuM#4jo-h1<|78k|P!--&w z4SN7@pe+Ut81e8KxX-YF%rqH>8f-dZwSVC`lhMo^7_Q{)CXi@K`(T0 z$dt2M@C5bHROI}wIf0EK*)Tv$)$cDX7n`?lE)RFUGz&*>e|zg!axxv|uvp4O{99PB zRp|<5)*YiR=JlQa;**=LhJETxHQ~=jJ?pC2a>|~2RkIMe$P+hcTMmf^Hpzt%YM;6K zBAhI?z&tc&cWzX&!r0N_qTSXnE*?&--(UY!4mSidMF$bO-^V=6 z1HKB8lQ|i+k+1s4+5Loh(Pb>~h1`|Wp((7qN>tWRJtcT-RA1j8ZBXz;A6F)K{@`2F`#iZmV)^#i}T))2j|33T!|-h11dlM$*f!@~SPlZCA;{WWjf zmm!&^DVF@nv*d`z`}sC>GDnnuzE1dEy|;@WMf_d$I=AC9`P*`EW~fvk?z;Fx(udm?A4ys@4f&h1o?PYe-4u{! zy_aMnACSKK#C}fh=BFicRJ^Of=1%GIHL{MFpB^;*cpqd#mq$6(d9v>sJF|YN^VdTe z<}wkACOE z$zl!Exv*sLG8I^y0ZC{!4hr2A2i>4^XjudpJaYKylFEe zdg`=D3PW?z!24TLWviL-SLoiaeY>4!9T!;cM1ZcmmiBw2-$#0lZ|R4kk`PVuJhrS<|+9SCZ`I6p8TjNjg4aGbB6Q&nP2IZ!z283ZE+I4DC|lh%EL|cXpi}r zrO!ETQ0Zw~cY+b6c>PWsQ6m<+8&gh`kPY4SwJ~x=lo;_KZ9{gwl-bu)D^Cb&D{t`l zo~FPZxwB3zIu$?GdxlFnuHf3^M)FT6hu<&w9@zHsZ0-#{kV?AppkQL`s##Iut0@m% z?q9jHOf3FePb&UW2*?>$#qSZVT`2oP`=QzIc0=~#K%yVaB+3Ri;-WbFt<_G6HTXlh z2}v>2lL{Vf4wcqKvd>y?Zb;q0jl1-MLR-7WDi;IFm5YPYZNeh7O1M^g?q=f#8Z3lN z1i>;}SJlXq|s_64hCn3bn*f5&h%!TFn%Ao?$x@p-G2UKSa0 z$J9v-iu(R4t`Q2=C;lzILj2{Sl?Jy^@(P3o3=+~Uy3svhC6R1WYkbuo2W74@b=sSc zULh;7rWx>9n`0XXE~%}Mc}SpaFyVs4=h}N@qoyisc1*Vd&V{Fj;F~|QpJQaI3*)|b zu`raQF@!+AW`9qc?EIv z$)q#wUbVq}7*Yj}C!UW~%5YIWHdRpcN6W?ue&Z~7%7kG?-6ZyzHuVyjy&Ij`wI}cF zM%jGncFvp5YuS4yT~-{um1sVZb0Yh@^Fh)VGP$8Z7=ddZ;9o}h}lL#8oO_X@e{DOB9Q%=93iT{~$^K@D33w=B?I`vap z&n%ws(DWEM-MXLq%V)_id{IW`nu~&}yf{zw)C%7-4#Os#mx9*U2CKP9CTuZHR_+P< zV_+|(y}_}UaHHgDR$~b!>{$u241d}-|CZoYH71jQ)D$xd2Blujc$XTu_pQXwC!C{< zn80!S>c@&Ae{OjOi^DetGx2Gjj4&Yu8yrc2I8cr!y>dZqNXF5m*7`GTr+<=RyvkSr z&2_fidG7oN+7EXkCAdsPA|4twk=~`RxaVIMm&5bB4db24-_eh>M5{Z0=VyG%8-iu# zc?qbxg-@Ta_`oFn>gx4p%*~s8S0mmFm_L8GI@xK^iXKr^^mCAW{6g;@4ocL8$Qqr( zs2aT~U^*26%=8Z$_ji`Q6Ub?t?gmy&WiXe|WpB^>JE7QHJN$PE84FyWL{yv~kDkOu zMJn8t5l_7Ig*S+@P&Y`X{2k+!Y@+v-X?L+p~kK?`1vW_Fpg&*%4I zs`^RJxVvHL0HxwDm%u0nNG%(O;y{oo%*ayYoH34v!U7c@M_y<3ESiipvz!3viP9)D$lbE;D z>a|;TN==5K7AqwgE>vq9d=qa|2BYZB@rGuK?sy5upt_$MOjO?VF)kl2n=CHeWA8Vc z4ZaZzTgC*Rbq+qYI-@F-p%Wil3vYagDlzQxNC5_93Y+B{EoA=po!4as@QH7b_$;<% zN%0>ds5n)xO0-0IVXQa8XoXowGc^jSE7@*_2Zqcqsa58?%$5YnZTloYWGfD@kcmCo z%UyU=eQ{}_mN{Idn~}zJBYn1y3Un?Q7@d{1_D0&&(fq7LjyK{iXsTn*cx1je$aoV{ zgHJUHBmVoO|LsJmNl&Aw@LHS&yUFY5deLjz)n7( z2C>S2L=V>eVko*;T%xP5zC45FbL+{F@@{~w#mrc{6m7rRBSiqW&WnR5A&nMsNhRa4 z{Ch_5_+Aa624Rd+7k>AQCcd`lLBWq?c8Acih>;;5eXfTWXmy?FRc!rwvOm9=erssx zE(fGWpUn|g3v#4I98ET_qXX-5nlJ$ns6Wx)ukp)!6+ffIGSYr_oR_b=TDCrYP#G4v;Y(M7}Jt8oaF;QBg5P*~# z4lZYl2Dt7?F1$!!fS>UrRJj3}7B3|ZvR+4*j+>oX{uAtKOW3?u?*av{kA&%c%<&M8 z{r%EN1x>;x6t5Iwq9eq=rAx7p$Bc>>laL;ei=jEE3(8OE`QEzHE^4LLG$?vvjII3H`EiUrEcjj#Lgz-*?|`R0a~ za%)zv^XS~7!zAfvl^m3%#ZkUt~t5* zrO7FXjEU{(s!BalGT;u9E|yRr+41&bpdU|zHG}`#J{u%enD`0Ho5RbEZ!0dAQ?QK6{_*oo%#cmi)fXpQ?oaiGIZ7D$^v7`^cNg$Ai zTqSZ*PXZ9Eh42v5hxM2f0$MTBfN15Ks+#!iL4U=90%Q2WkR>AYW@q*H77|F*oV`EM zxSiiwm0K+LBRpU5K$TTFZ~iJmsIKv0j>?cDVG=r}+mg-A`nxT~XvQK3;hXv8epPaT zVO`FPiryJi4M>qfg!mU4{F6?}?2961u!5eqVNz?Cy!=B;K*~qZqftNA9!G^a=|-!& zx{qaqjJu|ZAz_M)TaW2JKy-F{v=X!1@&f_ba_(AaY06?6mvYmmKKBLG(RUYo8bnFi zaTf6ZBz5W$qxRWaoM`CI3u5zec@T<wIT)v(j7LH-gnj9am-jggn8^>zKT5PiO?Uja!^}I;i+Q>#T zr8n-PUxSVjMIEiuU(MuqL5PkwbyI##Dt-`{KeneHE;ZouHm)uvUtP37F?2~5y!Wjw- z?-eljutcgD>0?872K$-r=$M;(L8rV;+-Sa|`yx6uYlPxfBl*l8Hm8oL7Poi#AN@Ug z5F`Npl^2QEO1-gSzC1?^lq#z>X!E*+7z!y>VOLH(-aoa122Ac}V3jKQbVaf8HnT#x z|86{iDlcoSgMcvaK}q|V-;Vr2Dhp|om0P^?&y|YZDB%_Q#ia8qUVWwcZB}8_>|4q= zc)Gmw<;q!5?Q&r4OJ+wFyNT^O-A3SOC3nAm_CcDw6em>Vv3WAI_x_g3;d-xh=Hi#| z&b%0l0v+{aIq{#rd#c+oJu3hSP%D52%!X8YZKt~Y>pM-Ykz$iL?P^I3~0WaUrbl{>}Ep&J}Vt1 zR<)TzH5sCW)!GpMA$>#GNa230Wc(Yxo`#3j>{cUFTDz5gjqZOY!E~`Rd*K<_Jqxh_#FvJ*lYPDRb5tz@c2rm^H0;SaeJZ>TJL|E z2o~kOLs%8RhoQ{)TtjQ;U!L25suICz6E{P#2-8EcSt?|B(u!zRdEfX}!?eECn8VP_ z=XTjWBRoY|?TsyS#4x%qEJp%evmP?i=w^By?-O#iuOm*Zgo7(VAfeArlm!K<0qco$?y0>x(xFF+IpkBg9tqF_j8%-&j##^8Z1Q`EX^IK*ohrGchsM z0j|B{>9|fwaSZ@z6F&%F0o3P5+xQkL)oqz-PK5igu8EzG}JCMXUs9sZ`pT}4fJ+V)mOmK{^poe7S_Zd(hy7h>dZ(FV&hV( z#82NwkHu>v%M(=2{E;s_GCGbN=@45$dA6q^-h)Z4lpdNFm)(mO&KotX!i=}3ZL^Au zYgF}`SRZ#WVi^0w|nE8m11jIQACH^sJC6CUi<{wd>Oj&+OiaFAcfkqc9UeEMH`axefm z-yj4K$JE?>86>rR(2*msFgG^=t-uQq)o}=t4kj11wxyDmnYSGTunf^ZVMg1!?!4;> z#0i~Yd%pUu3XE>05s)A)_hLuqQ&q@|er0KCZQ^&ZKRY2+5wY&hPfillqa&YNC=48dbS3isO0D8}K`NN4E&nwJ3z7 z4O0&o1J+Kt-?&g8y_c%!9nt3X;k>8>fow>4_2H!u$CSYH~yj z{eFJ}d$U$GA-|)5fWNf%hER2emNxbH)ZPF3FCh_BMy9Mt(uv%pb;99!j1mRlj; zqsLDuf%m?89nLOp3l7Roa3l>s8dF{kkt)%KCkvYlWfP)P=w-{PKuH3eRdCr{Hf<oFI@C zWVWCJ%(^>3b(OrB1mR9xPOGvv0I5G7{M@{K=Tuw4>vw3joLZVY=|cOHYd^*GPCGWD z#drnXUDuLYbks}uZt|;+nV!Y=-S5TnmZaAvlkBjN_)dP7xrWM_;^Kx!NIB6sQ7P4O1Y->LW1 z?+UXB0rmrwTuV-05n&t-CWNwPKF|FU#-p(}oXEH-3Qw(TY|&R`!|iXc5q0$2mlGHR zKz_Gv+HaZ~rpB)Vlssa|(xG+zfve@LFDy+S+He@44d)&I)sm51zyb0ghjpU`gRpzx ziCI!}alvMS-=EeD^M%rwj=DAezQPC;%j;A=ceE;!4R_l`d-|3+o9roCp-xQK@j5zs zl4=}%3A`R{XGW)!MdxZk`LArUQX4*@;1ZJ!-nAsVzXK3H#tyo#I6WCE)^rd_43$G$ z#T2usbleHWJ6cU4!qfHlY=~Bg&Pv?XjAZu*JvUQgsU-aqnL%~~wlfkYy%A1n{dcez z@4Mg>IuHLyDv_c-p5M`J1}&>eUMU{PrgBSzjv7cVRdotb(wR+Jy!!f}GWL zwqFbP>~sW1zG;0!-H&c=`c@**sTAN?{do!bW9E0YJg|q42^qBL!~hvz5Sa7)K{&z+ z2o8{Ob>#yQTH#>8wE}pNoq&i5gy-Jin$+fkjBB`vkh*~WT5@+pD_}-Pz65wG{G9L{s7xS>Y7P|H#fD+i&3_mT^DYpCmf*A7wdD(P@@2{Oc%~4g6G?nN0gkZNt@mF2^B&lK5{zunlB^tM-~A$K{`Z^)s_i`GgUB-d)6MIH&ba zE7Psqkm71*vW&1&L|wMWyRGc;H(gUbThFKwEj1z1>rP{kSjN9R zPo}1%<_51KGEmGX&f1AwWE5U@8pn>>yIxD%&s=hOuFzy#b6sIvi=?ZHp+oI^ur z)tdJ;JbpWDO3!qPyUEb_uRc*hUZ@pL8jM!D$x>}|f9Wt33UCP$lVzXwrJg4{A8lB) zH-01K9eI8|hXuM@c3H}&GLA~_2zkCfHBIsc2N43)%J`1pLt-^u@v5!Iq^@Ed5!|q> zHJUaGYYlCnv1V+p_(0oCY-T0NoWi!E=DfA8lgo?C*Op5~$fwOo3 z?o_*#J;q@QUz6H;wXEFqzpLZF%RPc{*?GIq@X?R}%|UER{MIZ<@1WUXmDwk^bNdB$ zJQ=Bw=j<_bHd|K-?aL#KOX*)TVAEb^T^I{9b^6Q4)H@WYA4*TjbIhCiQ4_Ez@DUp| zNqBXGRqa>hkn1gBOccV<8<(z4sH|ajs0Z;^k*M6Rmq$laE%z5M%y)%-y;rZtpxKEE zrBF0+l`kvr^reDIKWin|GS$!i%Uuh>Pui;;qbe}1f9-e>jW2`_{2m_ih*FI zz%U2d7G4zBq241MV*N!gvWOtU7Z8U0Dl-Mk{Yty6&>GIyiK5cpgc9KqGElLX3{8jr z-~goWGd((%2`Die7(U}5{=QtrZqRHgUr3KsjNJ|YtPLXsR>?|nrf3IQW%H~q2#bH+ z+;&lfW%idoP!VQJC(DlYSBP-nT%2eu{=j)-(GaT;MIVVS1W~K_!v57k&r7zImu zW|bH~#${Jghp(AQ#vMoS*WDPKnL*d$te~Cxoo17PEjQ<_+t)qOmpSvNK@%JEfeo>L zhd$sMkMiG;3#abSUhflYgzq;OoGnMltsXO`)DPt-MwE)hrEmBN;{#tUT>GK9hF?Az z0pz5lz=kZD7GwJov>|~tG_cWQCa|x9We@&KRmLdnqK%;@x*_s6rjV~b0Z)VLBaz&} zTUyBCq>+>t)g_!&{T}C6#p~VgxlqusxXlz1qb!b0NUMC=b@Fp-)Y``+y3~f1k>EqS z9Wll4%8yYhxCoBD6p>xsUuiK8DC1X6_hQ1TzF$s!-EIY2k_%+W+1{{_0WOvNQiM{X+5OfNc71#w zYc&k1J2xb1)NkV%3an|`{1oqqt%?peN$H2e7&9)AiclUF{ljEOY`WSC-}Ucom`zzd z%EpDI0SVAefmGAr9KSmf;LqkZ4!Y{-IoX7~RgaHKX}elN{mth2TPC1(ZxnSg3g2b| z>7LTCd`e9CRbyCW94$C5Dz5}Rpn(|F)~1&g^a|GE9K;go{5U@Mv!>tZC>fVa@-?5p z5^!}BWQgm-L?uKhjfW_KLz`AaxLT6k+I~7$x$XKWaRifH30k3SC1DWBRcT$^zaV{P z%*CH5z>jQrC3>~|IVLre1)}dO$g_yzZ{KQ$eq_7|MK3Daol=pG2A>_~F7ML%F=V*W z&EdLbU+hry^svV^oX%X%?#XgEL>vw7>L|x(bM}PvI|*>&{4?|jRQb-5J3a@kRtDK< zPzXzMYLq6uv#Off>jQZ-gIGq1*|lKSrR#ke@58QuS|uey3e#7$C<|tuo3Wemq?{7M zNr++y(6KWF@L}lZWE=O2*QN*2O0XFv5O+&IO-W;l?g{oRcjC5H!f2v+jm=7=jf8-e zL^GB_`(i4O9OAP1@%C=**RoPZAk59&+<4CUYaHhH5S(wX^hP)`SM-{w2fVu0D*4Il ztiA`a)2@(~#}C#%KQxA|)XlY(l!sr(h@vQF_AcWy`setSc~Pyc!wcv=PtfcZrl%D#FpYBpQVcOXJbp{D`}ZawN#88S{qp*Xy)pqa`)A~PlB9T% zS2Mya1|Y_o69kt!0=d;J8>_3*HhOBbqM{TKhqF?yBiX=Ty_D41n||oTgkcedL~Q;o zK0Cfjt-iqYjpBPRT6Z3|R#nX@u>pMAc!n(6oKlk5D!vFyN?fUcl^CSozuZ-Gy1W+s z-9hm~J(9ME2f6icFL%j7B6r~qk%N_rnI>Y%s-TzGXMgrvH5k=4r>f;1J07nnRv&0V zmVUH9B9(R{rT>GJk;p@62u?<5%!s>j*gmxP1r@^m6|wMnSmM^G%%RE8$LkA+J>ta+ zCG4($2`C9($_?Rd%XM@>eq8ID@G&wz6>k>aC&2fi^2uXg?jgWt+Bb1RR~yX*Ma*$) zRv(a||1+aF=<`QFbT}U5e=}WW_OL}hB>Q>#0xt6f$m;qHqH!Jy&5v5!FRGTUx5_W1 zIt8^G>lU45{Gg`YSpF)0hE8%02_#23dKJjH(>YRSurPuMWL1BII41BrV<~J#lW-Vz-@<Nqcdyl%fRmA8F^bka9VVfTFlV1M2z zL!|V5TzB@TH$!~&*F7qwacu%anB>O!!sM-6%Y%T@E!tKzP~t59Xam7%JFyOXYl5x1 z@7(j$f5z^OqZvK8%PoD?aI@`$pJ`q$M28y8sv}Sb&{h1a7q~eBjP~W?lClmEIynrI z{IE#4kw88i1i+-VPj)78EQYeXKmr&_#-KMkOR*{=#CvtjKcT0*mFRFS1@)ZNBQnDV zb23Q(iQxI{q9H!(=jUPW?NHa9)9^r}<8@ZS7llhfv=H5D+1pI-XT-_~(#cJZJ+G9f z>9Pu?(GK!5_+`~j1*DUa#y&%!p<80LuLveSi>BICqGerwK%FK)6Gy4Bgw54RldS49ozJW8K*%K~76mhw_9y_xF`^hn@&xb>n$q4bJkI-O+?x_xI%D z?A;?CI^3UtdTeYgejQKubUR2#L_R)Upk#R~y?2L81|Ez?IG;{p@A3o+*}|#{WqJxVh%g|FZM=OcjwMhR*A~>_VvMcF_%;V)aO?Kj;ZU=^+v7I|5a{cXnQJCB^ln7cy_c{M4G4n&-#HoY zMG5!o;Vh)okG#A$=gG$ZPlw*U*>H7Z;|<7RaS+Pt?eCXt^}TiYCAg8QekLExJcjQ7 z?CSVv^Cn3r`0S;jCg1?j7b8~sHi!ObMk~Z*$5}k&Bw^LtZ;?Xvndg}Y1e7>hJ@2dP zSSIIsf0M3(iq}65&e+Hvg}gy&aAn(>xLZUthX}rT?!JRYGL0l0Evy!%suo`u7yiA6 zQ^X@a1vPJL)fpJ#_IlEv{@!SY(lA~TaS&D7#@an6eTwTSNgy#h;!!!+`S;Achbf9> zn&;M@c;!M~Rs1mUj|SUgpa*cA2la-I7vJAOt@B!#}Wn**@)v_t|fiaq;3 z(JC!7drL;wy&4fAI3cw-b44w|`9P)6L0^HV_&ol9UHT420I}ShYvfOVYV{tp=a)b{ z0H>#%8a-%=KvaM7#0OqhJP8pmXFPBvby^OL6}6rdaxuczLt#-pC;2!b1F)CX=jh4D zL&S%>)7JIGWvD7+!Nk!J8Y+gE;}EZQR=1v@gix4|2ob^gNjSP81jZlSN@S!6ho({HaHY}n4FUkq}cAopS`%e1B@VVFQs_Ui#^ zc%J-K1;gZdt3l+CAz^1HfS?HLl96enjX`R~0V1ItHphjWhGPH_AypE)?5y308AS&| zeUqd6VM?*B-IBh7VvMPltS)dn>U(`Q3f|7Fs_6RS(FaPfxzM9;fBt7+%(L>2(8w2V90do230w{arRP;?1LO&tm_sC#d7 zJ!Vy;>o(M_toAne-s0W4wKP?}muc}0n}&uaf!jV={^dyCM(f>w$F&^GTKH+E^RITW zsfN9>tM6JQlu1F6XQ=4S86bTY*v^;}Dv*hkB=U=m>~_2pjLL75xqFg@tbust>& z8cEuirb7-t2|r84 zpRczZ!IbcTEAPYG5>%i7jn#C^#|bj+O#Mq;M!#5-i22X)H=;M}PHT?Fv?U(=((GFShgBGUfXtGP||9I2q<;*SmdZomj?&!Q5dn4x`rd3!BG zUc>YU={>)}WqQly9zW~__vL(E*nvf)0XnO2V^mcn&nMpX^Nxi7TJmy$)Xw$Lan2GZ zW~}G2hsJ%j90s@j{>3C*^Vd7tj!yH9wJ(1$>#kzI5?kMkINj-TzgTSl)T;8NciVk- zSeRp|)|Z{;b}m%vY3xc=OlKBab6P_fFd79yhK3^-sWO z^i=6wTSKsKL7(+3k)4$$E74+WPRbgbFqP(FNC*Aq91ltT(E)rfzVxYxSGlw94%-CB z!{E@I$oq_Zd1+Fd&VM!gju_k?BXr7;TCUj-#9a4Ld~f2q1~U+ejlhKZ?lz#sN3Y5C z>mL`gP`f2z*b-i4SJ!g-2U?XIY3_F0T1=|1saX&%iS9INc+kJQ21ks@5!$b8(FpO| zEfsVN?J<4FJrNV#=dr_7^F%~heD>&k;rg7efzl4|ducC$DvrP4Cz^mNQKC=7AA|;>Y$?{I7 z2R$`}fE`}U2ku2B*6h3-xNn8O64nyQ^Gq7!;g-{8cLEKa>fAY9KG_pgnZ8Pgym38C3Vm#Ufq)4($;C5nRcXHlog; zM;o&fA}l(p+W$IGsP25He0x$L3-qqyjHFS5fSj4&%N;Sy&tVItL{LSh9>)GL2kUhy zmpdn?Ps8t$P%%9{2|W^rb;Y@vZqcU*CfKI_m4k z!0~ftrTN^yNW-SJr5NL4la>gW$N1+T&t4n)@B%BKEyhv=hkuG8UKEShdH9|sRN9L? z-onK!pHql~%`D*zk|ABn)bp76XoE>S2Wt<#S+*3N@?m7|FIX-V7RtoN)C+4btH0$RhIZkU&GoAw7Pj zw?`rdHG4fIo+OB$4y$JeStN;-uPDZ!)uw0dDVno-IpwtvV<1v8YnC*_=AlBqOMGnn zj-T+O#g<3p>}rHe1@vV2mrZ>puQjLs^oN@kAM6yzb38CHm3j$>9o@E~F^AMV=K0eYDLP z2Yr1B`0oEFRsx;o-PwBl7t!R;K%U6y*b5~PjK}4$*!HAc*^iXjq8$V@16TcX(aXpP zPE?Ugb7!P^Wke!Y+{)cF7`8!Uxcex>$MqZ|-@lz*$QwZnW-%#-{prF-$9k2^ zaWBSZny$-2?Hib?&s+BRzN`4p&HM0QH-d~)FiX8(kX|*F7v(_>$gI*bP+vAoosHUH$#djW-yB!eZbU zw&SgmoDY7i$S_+Yr;XWPSv5IERnnbO!O~}jfh}P8_mjxR^KD=TIhqdCW=oxhp37Gx zLzkK9bq;-b$~%p1d9BLOcgf4tCP-Sp2QnARZ|pf~Kf9tnea*i5W`Apxfx}%(HmP~6 z`~SQjT)GQ=0~Eq1)B9og@L#7~K-SpGBPUjyp%e&q?9{gTN-0_Anzxt$wEV}*ZM&19 z!Qv+bn+m1nCcbax$W-vsZ}Y?K7`>POL7$sM20Ct6F&IiMN`S|w%ntzVpTjU2xpoHK zhbHqss)`N}D@*Ubm7fBEfSD)Awv&*T$7nL-0DnMf0{*}t;t&wUNFI@H&WM^Kfvox$ za&qY5c#@;k?FXKL(KSj7e^=rkq?o$1Utg`hJjfCXo0^na>j;8ViiuR6!%y-JMYZ!@ zf-9kW9X_?Ia0XQHU3kOc=w1UN*eea#0=`hwohjA^`RSd1rU*h^$mM$;67u0K^@ z_xn(li1J$5nkYxf0P-nJ)R|hrc+2*Ei%HN`OAn&%*SLKhWyJw>8|95(ZKsnwII-X# zx!g!ILWL-1FnrGt;%zzwpA0Z#+O`6xpd2g2^3T(gTi82VFR#RuZ?DRA2qdbiK&4OX zm&5l~y2;)MyeD!TCI3!dzWmXs5;eMi-GYHC+NHVyK#s##_%VNa78iFGih^xneeg{s(ArUuUS7klxILWq zpt5w7Tpk{!koX*`*^}pbBnPLHtiSr9>`k}=PJv+0{LVLrN&O2B%Wxs}FAOM#n26!T z7BjE!A*PCuC_^Bm3LkJUFaOXjnj{~6hAY!JXVm2vhtB;OaS=zEWc4Q(_^8YoG61Zs z6H5mM;4;IQ4H3IdBxIO({~)KuYxk3KrM&s>>R{fl7k6>i8BAZ0`RtJ9JrzVIKS{Y? z(?M5CAXWA`v6_+1nWo5!|LW6Cx&qzlR)>ojT}9};HdV*Anr_-P|bX2JA#Jx{P^X8!0>vFV{0xK*8LMWM6d4ApJG;bZiTFe%IvMpn)i;IovQGEr7aqf5UTMV(fOQ1S0Y#{Jra^HrO{ z$Yb}h!-?hG%nQQu6tMY3?>4ToBE-`^zK!7lZH32Cf;^Lff5F%jjJ8m~x)3lKDG@DJ z&3IgJlD{)tetpq~@cWbNyhq%-vw@RcCX@Pac4xLQAXFBs(1nyyTUJ8KTPdgSB;`7@ z#2($8;74Te6XU(*Bja>sjfjW{7zgYnjjVZFK#)?t!CV+=VaVhNGo;=evpT27 zg(A-Q8$#`z_VpnV8+?XiDI<{pmcnG+t&m(XlSgN~D6uymp>jwrj;+}puk~RPS)z#f zPay;^rFYPdF2~#&;GIj zXWZ0VtD?GL+j(+<4f4JDnBf$Q9Arb^wA(v)k{o6}jV^2({FdMx1*}_QW$JAusiRvW z@9M3!_IkeC$9T6k)uR=9L6IPT>rpLrf9%};v8hS31K~0?Pc!+?{udMf56$}@{Q`LG~NT@T8E`9G~xinjiCcbNN{8% zR<-pw#_9I>hn|&ZR#r@aL=df%!M9Yq;41}YEWgut_ERGtAV6(Qf(g%35vP$1dJ=ll zcw|b0?jPDUF#A89b{j*)6!HlSYLtT6vI>ZQ8&xuV`oijdgdo~qcM-$yR;b2@viw>? zHwHCqUfG0CVro7~KPt*&%;D^9;b2GprJxWfqn zRqJypWZVwMZx%z{A8^DTJCHTbMc2-IV)pf2rOT4ZNabsVVh=M&Q}Qc~Fbc0ZAz%_! z)cN`t4^!`*&3%bFh?=TT5ash}aQE#e0P8K3!*pd35%tF2?doZT3sZ!!t0wU{1caA1 z3s-pX01Fon*5LT(rAY%6_c>O+Pb7RxsF&}D?8K&dJUA!v3SipQnp+ zSnz3kK~;G_1K7g+W{hwtF&t4x=v1rG>;`w)OPcGtOp&6!7Fx;d$HQvfX~VGk!=5x< zb=8MMS05M zL{Yu3tS)R~t)}sG88O zrjCowZ02R8j0a`oKWrUuFyIapx8fxuFNl5qDXb{50ox_s5pWKG0HpBt&g;w+8I%Ky zb8Z3ep=TF$j$@9Qy`YC%R3zb1|B|+mp~)EK;@&%&3a9Dd{ghz}G*l$YtV3F0DWq5u zCS3LCRj^?%o^DamT`lcTAiL|>UtD~~r)MFgIWOwc6#a@h&(=;p`+g5@I!NkjQwg5d zTh0R*i(R8(*oNg#{Hx!Rc~?4iAOVY6Wm!6MoUGLhpMW4c+)7% zebhe#kXiAkXbtL!JuUQw?? z%787E?v?=+Evsnk{^He3jM7&<%j^bI_0RX`moHCuek`1SlCr&r4A(m!lq;xMaO*Gl zpw7CI9dwf|l?QMJM`CelF{uaw+V-ti-zY<+*?abD-4XQX`d~)I1ZGs||Nfje%ZKg< zS3BjOeXh?orz&ZmM?MY!>AtzZ7w{Mic9Z+R4KmQlSTPmH@{j}ys=nj0wx@Pm4y-%a znbr_}{*`belh#B`COr{JF}w8E3wDUSFX1BOGhoc72Vb3LP&pIlT6GAG=m#roZJ@@e zq&Oi;L;^2l8!59ZvDGa_q0=?JMj7o-HLTKh*p_+KnUO4Qznjger&76R0sLO0oexuT zXA8%c!|VN}eXkvDnA;xdADe$fjs76p)F75-84=9HL=v?Eg_Q_GzbMI&%c}EjOUrdc z55qj(bGM5~JRIkb{t@<{{jsS7k*O7a_r8j$oCpt{9ZJ6&M}0s-efY6KZfh7w048l# zocjn9#n+&zIm&x_)M}9WGmFJURJkAF9YP?FD{E}l*P(0J;fGmRy^q)4hToN3=R3lm z?Df)}fN4J6;H^%tHL~VQ0-C>*?fl#8YD=1%!|G&(h|r@q z@Cmhl9&Wjrn=4UE;J=LuDv`U9v?VG`L-UnPrw%1=lk>j5{0MU$M{wot5aIFEy zA4j=hbaQQ6@_W1-JN7wN7Fcz5m=p~Q;7SeH0qOgckYi8FY4utAF*Nr>8J~weT<9HM zWjVArM!f1(##g&(e?!^%cArDTHs2*CAjLU7|Kpa!_QcV}StsJmL65WPoU+1`SD2B4Q=;{JIl+ff1!IBahguf@=&Fl8oVct%hO`zN z@n4>qcYH=h1Smds=bOnIUH6XaqsIHi#3&OC=NMTeYm%yxkzZq9TsWxn87usDnFQof zAOmo}aPV1EVQ)aw{f_0n%|xw{!ypF7-kLX{Gv>F~7U#D~4jmE~ue&wyEDH6zpsA`E z&#jW^WKasm=T+U($vAGKS0F2}ye5?DeB#HA4W&H7EP!_g-DAra2;9v85mS=stA_vI4*#Dm$lIiL=5SaFr+v$18H8$d0kQLy3<|LUIcDVRd^m;}n}b=ArH*d5dTKuitOlW{VlD zbY>SP42FE8e=Vjd!{ID%oRu7m)tAEL2xcJ`L(raRYtJ)0Qx&ay6$(HvO zK%r$rB+>Bh4Z1Bvw?*XSIPB}IfMRB%^IeiVQgry^Kefb;KNZgo!vgYHk4(aKO6779 zzA;zkef)2`nf2JndZ2;u0oMtDYkdNFpN|C`s6pm)bcTHkiFOGgN9zqYNP~zpEM}kb z=*ICMSmSn}4_p_BnEHy191F~U-tEZg``Og*-;+fpCx=ITKXUps04{*TpIwnS1ptn>4H&2@*F2sz_fXu3Vg)T%~?K7ZT~fgD)KQFjj8q=2yZ0OOZee{jt<$;5HYO#G;d1ScnEROC4a> zKk&Oh_j@elQuupdl<*SrRy!vlPcmRMVCS7NCcbf{pK|ASF#k{!uGg$4&3tUz%N>%&?eBXh(Ii|5X4ECgfMJ8l2ELXsCt|N&?nvZ(4qc79q)ZtoO1{ zfqrmo8MqoOK^U@-%clR*kC2s}@>Uc`-JD(1?b(dw=$A*J8~os_TImeKM=949cV9vU z){rX=u1ADbH}XD8Zq4pojWsht7SLyppUr$PmBKTAbNwPZRI;~z0|S3VweH3V=6CP2 zYJ=1!lmls#9Hm14!0KhrZ-KH?s%ZfXh|96b4B}Rli>E(Wj^ocP`_}k2zp0PW_;Kk&ET1yO$ zvN#{vtml3exxk+Qh|MY#YM3SC;qiE`an7JMXFOJ@2JyGZ_CV1nQ0{8~fEov?X^xf) zsNdy)QKWuxS#Lw2Ij^UREHltY{*nGRmo3naM+w>S9H17r=#T(xpoy<~O*X{G_lr+L zPGU*;m3uF5E@(gYtyUF;&&A(pEiZEWq0;6+DqkXm{Fti2{T)OM{BZcBpBf3?^4s7U zEAY9pNndPUG_zd$o$LT*qc7$1A%sO6sdgVYdPLkEHf?A_9Tt&sCGfgu+Hv zImy0Dk0&jAvPlEvevz+GA*Z~ha-w5+C`m}#%ZmR0$2}9k6-xv1{9RXJpb0KdoY z2Kit18hcEG9~uu;3!yWUA~HPu$@VxVkRgfzPtb0NF~-HJKX#ZiJY3Vu-%{Aa*9X$k4Hgl|@9)3v$K8ED&iS0rbzSf4{eHckFPWM3@oMnU?Q7sC%U@r; ztsQ?uVEtjvu7ds5ov#(SL^Zud5;tw0YYY(+rG6D^)nFbTSXoo=pfkf#SKh2AiClkd zwQfwqES@hHDVWlV^+1pk?<0YeXbwJll$!`41WWW0a>NsI`aEawHr%0_vnYFvwYU99 znR!Jmh>7ILxwVKeC;46Jm0`vUf-bWx#6Chag>e*SH6`)?>WS4K_X91in$ zc^L$i)2-ME%C=vkhUWH)jTK=~^SQ|Gxj+$=$Pb=Yg#t=JPBxD-X@A zi;gML<7wx0zx@zi=7Eq8(+#)1&~^HS^|dL1lB>J0gunwl5&|c495`p-NyS7WrPXB+ zHmx4Z@w+!dw}9(oVeGZDTCMnxy8(aWf|f~=x79(D3&WAO2AU~M3H=Ko23MCry0%VB zP#|V2u1NJHOH=nR(7)aC(bXt=EWNascM*$#PLQ%ve!ffABKt&O+(*cax9Lg+mmN94 z{7L*GMA|PP-wy;#m*eWOS~|Z2h5WNumNM|miw5vVbgTX8BI6^NqH~YVi{$fL!O~zG z51IX>V96k-C|_=Y6!|`(mP}ETv316oSE26L@vejOpI4pI?fPoRXt)XxQo7Bw!fb8B zr{yelbWj*t1}Kl^Ei9PWwKCBFfC_o+;6E{7#SLtbgh6Uj@zKrVJ^AL zC_z$wJ`$4k^72wVWw9X_;TA%_!An|ITz^^r7Z)^Ory8?9K(H=NJ(P;7ki)@^NwoK@J0U|{>vssD?@ zta{I4|5GwXJcL3&XU*_EIVY8fcS%w|Y_K~dZn z)gfAf)njrwk%n=nWBPJPM0Xr>H@Kh!u9)`Ni|TLf?ed;mzZj+b_FF+U8JbB^zTE^0 zXWu18N#IuD!v3Se891g-A<4k4YZhiM}fa=Dy_C>+TgUZT3HgfD9v^X zc6kFR$_(pmDGBHgb!Oex@8*Nw8!K?}y-Au@r~yyBB50?q0~Kc@UJE3Khs2#Y-bKX< z6A>$E05XT-X3$d9J$)ez|U|Y?A;r08WjVpscQFSY zu}EJW&7;bf^pQ?K-?w>B2S$RL&jv8sG56$$K*D3*;59pP&xQ0{#w={k=HS7VM!yFdq`&9Xt~7twJyLN7-tBc0hLl2`R~^q(T~xgBj4g zB4s$4js=&(dsR`lTLISM9#O-eyLi81(N1;8;-WR9Pv9DRAyS~Ge9p}DS(##B8X(C+CNhMBF7ew56BImqWF&o0iVeSGB7Zx zJKOEUSk@YW{rtzE4EW74wUqSfx}c8&1?td85)=0F74T6&Fc+RSKp+C_Dh4IPyw$iz z23d7Iqk|plnwkod~h;p^cPJ!5WqEaR~pIVm?M9ym*r}N`->Nch-mBzKH>{$sP(!r{IuchUXy8u zJBPHKB9%ocHy{2;nV>x!%p5r!#6~m~nN8aJUy4l^V~+2xOrspd4h-YO|6skXAjKj& zPWpu4tzlVv3|})ESo}b8rb)%b2IeBxe73881*G|!ch;Dejn%p^>*?vmzj&=BZDd## zTU8&Rq?IWKgD^T5B4T0zyD!%;NK1^hf4!;?%(1#r<-z-3RbWtYaWT+UmRMl)xL_(j zRR6>ggGIRhIZYB+;-0G9dH-lAL<20=m1z^VH*TFM_)pdONslx<$$^)u;~YGooXsFSBKy=6}U%+ov;P3gy96&X#qqSz|t z817Y}sq9OD4DxS?-Zo+Rmk*D}DZ!V(_=o}6TGSeh`i|BtFG0__kl=F;0Dry~_@KeO z`9!Obs=Vbw0Vo`qQkua5~pt#h3*zyrxzXNGh;0I$|O*jGBiqb7*SA+8R-*vr}W zIqNGk^BR9}^gF|)ghLa!l(8=((MO@$C}YNSv&PEYLD9;7aGAyo2>vF-fkyy-co!r{ zlEd8fn@?7`bISh{u9PY({uaG7Dx4tSK}Wwjm?KGeO`>B?IuObbXGo6{`+a_>H2F5= z!zq)rS!!wUaY6l_lzr)4NKLHdf#5(|@8>~Ux9~z0#cQk?;x50govj^g;yQoCO%+x@ z6;^sCtkkjDv0^OD=yiQ(0SspSCu_AkKp*1ZCo$iZo&L1drQZWoI~CS>hlAPAfzsCcSEH}$yd7w!r=eIZCKh0 z2ngPo*Q5^A7CQq#HVb*yCg$ z1_S|2fmeLCq?``35@64m2EY||CfyMwvD#NN`=;K)TmFeG{P*Iru9ss?BEBjT@VrDG z6Ko1=ajdxq1GFBsC7q)%Uc>jw6)s^=4cYemf*B~Kuj{R7VnafXG(R09NnlrWVLZl& zA3d;AV-}lnzHmhT^C`0Zt--nT(-2k_@CK=*L6rqKU{@!nbS29EcX9P$!V@uZlwc>4 zKRwu}3_AD2cxN$9SWukv&=52EE$3AnYhxbDe?Sfrzv-0L-1Z> zGduL{3Uyr1J4JAI@B;7dc7$I^@Of_!+g>}hB1X70w)MY;CF5LtVzvR;(r!nOnRp_^ z7fMa9-Z+2g_CZAG6z`kcpp@B|==7Mh;}cJNGD8)fKthL{yRN<;SUy}Bg3oNbH2%Y< zE|NaWEEVg*f1f>;9QpS&d&M!E0!riiVDgGN#{VXJg0Umi%2sr8un~i~DFa{?iEj&d z-#~lmdh_Y#Z6W8$t6MQ=0AqD-tN^L@XodLahF8ll z%QfBv14BbRln{|-BnZ<{vA8TCq(FWJV|@Tq1wQWN3jZ@Vt7V-7z8ed?B1YnA66{z% zsz{qNuIyX=6nY~Nyf@Gqc(`&XtyMVmmb`{Iou8J5c&O?2aI}oy`wt&IGyx!y$zS7q z>~I>$JR#tCG%-0T6SlwhQ=5ze&q|%TvcW9L+?|joCc-x^;xi#`bh<0E9^Ry&|2IMbp_pr8>)mFWmi)6<8lKFM(JeG<2mTTTs7}f9B_f_Cq zQw%;64Vze>Uz&Mlh(IzC=Zmyn&kWMppLeo6Sm62ukfAnMDuHk#hduuV_GMq+$EK`z zm7h`UgyODyjo;e2dK^qq*&5FR4NiQ?DNlABPULI3J6-Oae7H+UaVtv**X%j+AN$bX zhD2^}Ums{wNqv2z$pF?H#vwDtJIIMonA4gn;zXY=;#9Iip8~cnAfE#~!p8b|%)F4c z@QTW`8RZUpfQ|*(`mX`N8dO~kSAEPBY{tePdkyAX$I?Tr;5+xKL?^BQUinm5g7uP# z9As_)5DN>FheU{7_SY>baz;wGzT}f8ayWBORak@9@$_alv(lk|5_{{(pVIBN+o;iZ z529+%Pyd)Y#Irq!P6b3+b$120k)nmk+Y(A91fXo!#x)0qXe`dx*b?3jj(`3JpFH;Q zbDTUv1rIm`HN~kZ%H757?ajaHy?(XAqfmP5725f(z7)(j&_IPS*L($da@aS{wV+ldP%aWEI=1cIu* z&O|}rvnjprh-}tbFhZ_N|N5Yr897ps&w>vB1ae_dNoFGZ(gZ`mj)SKC$(fhzdN=ND z&Cbu#g3QeKI+#1wVz>CiKgJUyF4$!O0?hfN(q!-=b@IBzp;MY>EH-$s?iV!d}N$zw|RhX8sWaKuO@q%AN5x;%=24FL+05d`Y_ODMKW@c3V zd<$Q0Z*{as5CiE}wH-Nzz(>L$Lc$!d4Wr>PMq&I~n9ExIleM8*muWOHVluZFNPz)) zz8E~lcSAj{8Y03{OPOddC5kt>4*D~%;yTX3{+ZPebF+;}6;WJU-eVgy9Y-cQVG_pE&&(-Jf zl99=I!W?|rB)wKfHD29oZasuVqt!msR91`4P4%AJKWB(+VIDW!EzChp2iv(7YC*!p zRF&*tqo0A_^$y88=bI;u0B&4O8U^su)nl`m?`dFt4_Bo)i{7EgB`ZS6vqSw=a z+V#&5)SUMcga1>s0~VV=#rFM*11bbQ8Gmsupq`&UBeCitnK@Y-UrN+oD>K1vJk0*> zVtF%$0NR*vYzam-+fNvFfY{<{JM7C-=9ua#EP>b&# z?4FGx`1I;lU^J*oMD1>OzE&>SD-T^2<39_e3?JFw=%eSx*u`t64p)X-K_Sh&=d*Aa zl^sbBC#~ZvzP|mCJ>;)Ww&ahZFy`xGs?HcN?VAodz4=&#;SD$x6B8*hWvkEmpV3OE z36U$bwD-ZePzUICpZs!if=&pT;<^h}B1hLElujs@r(0w|ss2^-+f{e5Ykw1s!ANcv zNYz#3L&3B^5it%@cM_a%4j+6=e`a_T6@X{fc)loa8x^H?llJrof1n#8ItGdg%82QZ-c2Wi` znalnUOW5P&G9>F`FmR!rab2#yFdgd17FX@1a#ITGXqd-aMqLiN^6!H{^f!aq2F&Wk zkbDQ_%F24Z%P>3W;~2isQXhUeUEb8y%m?jE!9Z~ec0Ism*M)UlV@oZT`n#vH%Mw7X zK0E+^yD-M_fx)-20fZqM#9ThAJKT!Mkk)A83yrGx=uL#^!mET zd-;^qWH(gxF%M1d19ci^$)bx=seOv1_gG^KX6DI$y}hj*InbaaD-#H?Hiq*NoJu-X zsF_H6=YEw&zEMkHmlfKEW+|Nkk~97_o4^VirtdUzl^+GHJaVc{c)d|nr4Uxd)8}b| zogRHD+;R_9yw-}p{njMBKRb|0sP|t88UzJ^u7)MR;n{yLN?82}I0?DnASx=cvHSoI z7sSsDvx!;IbcayjyN4Oc+iTU}7#fVFmAA{gwT(9Uw`;tkR2=f$04^OxX^G9m zrRjU?hbDp0MON?uojTUu;3$}6n#>G7JFFR1OmzNojXj|N*T7)TU#_p15>Ux;<3HbF zp?+_J3u3S_GUkiZ>6tH}hYCpiA?3}KaW^eU`qK+G($5pwdJTsN+>ADP5Q+R{ zZk-2$$KxJbDm*RPva^lv=YjuC1z4R^443@#V*8|SN12_lTk45nnN25+kYtLKEUCip z)=wO=lnO5*|080@f+}>s?g2zX_~4SSE}(qk2cJ$%=hyIjy1~VTC5vybuUBThL3)a# zCsMyLDZYCh9!~dhp6sDADHKU2pucY#4?6rT9J=OqXD&wgOyuf#mLAWCqI1L&)Nq9# zFJC?}G3mE`;jz^u`w_1Olo%0i#p~y?Y$D)3T`_NEK%F5uC%H#5^xnb#v=QXikOxy7 zJ|M?4S4$>g;>!Q?Cj2*mGnVY;amSfqaa;aWdd^fIpFsKsmTdolL)keV%?jYaJNZfp zTP{mlK+w3_yN_g{f=mCj-{`HbdN!U=_Ne{(2?+5zwst(q%_Lx&R&|`3qtaWhb+>^0-i-<565po;!T~{ z--WJRx{^~k+T8X1m;Mc*qu0TxKN9Fj^A#e=0Gp03F)8V5o!k9Viza#q!*<5BXhEWt z5*RLFTy;K@zXj*!WXQkW5MptMbx>POY{fDn*swW^4j9>UoO z;&D4ts~o_MCT?kHvXo)TSp3AU&AbpQjA<%Q#`Lbm)+IH{R>FaMP?{+rKCQS}TYp56 z$EFfJH`$xC+Eu4BUK4Zt8zviu_n7M?tN$F2B^jcOF23|x9T4SQ%y$36N%${}8ES2m zarf}}T52M=59UN+_~go%S{yuqOi5o#YHA*VC!kONc)SW1bLfNaHJFf)mnW0u3t)*l z$vn&U@>n8rm zW%pPFQ3^MVoyWnrz%W!&U1g`U!i zaFZW;t+vr^TKh1kNp*Ir&Dr}(p={!d^=nU){!XiZAAcLeY3%nm_u(g&v> zYUZJcfMLKa@L=DwSRU7(D%;wri0P)afY>!{ZqbL~V{1zj-iWkm*I-j8TYQPj(h{&? zszl5no5I_Av;*h4o~CB!27YR>Aj$o6A4!caEURgtVh*Y0pv1Q{-zjgzdi_SZ*~NsZ zp#HY8i{6W9tz+F*^^Ie<7)esbHletYY@Y|hT79GCaRE6vdH6t%F*(1rH%NRLf4XG^ z_|LY?@trZS1yVcj_7`sKHaN$1j)VIxSy6xe#HpfTniTL>?(gH{<6XSrLre{35Mk5l z(GZIp7~y9qJ0j6IGmU#Fmu73i=b>0uC!RB3Zaenjel%!n?DDTwUw(`}tME?ZYx+L=Uccz^uhQlgXY++qHUcU8Z{fm{P-#^6 zBGJYl{{tu|=<{6WwV4kS0WP98;b_{MSnTbNqZY!n8DgdvT~9p%1% z;j3VK^r#at$M6#7vfNwyoIy!LxfTb`^k+@*_-*iI#<}L=683Zu@uWNPP;t_j^OnW@ zi0X290LeR|n<0MNi0vOnJk9_=^qbx(dA~duHkq^&MlL_P2dK*^$=X`~YSoX}d`(fBGf$CL&c1c) zkY#IO+HGa%y3Z1$)v4Fcce>BzX>n{T?cE0S-_$MrH9bwVvbg%?SPOJnXI2V3&;2(E zH*TwqTuLupa~e@h4O_}FnjYP4Y_owqC9!r>Q`^||HFvd*sVm_pa#39lU58P2edJ$S z)`g2Y(zN+qc}l7;DhKQcyMF98FQfw%8y^{Lm-h|g17re?oh?*5y%{|5UI;*gjP-{? zup#<5qK_frXo1zkr{2q+>s6B>-ZBo+j_<6c?=VVymv-SN2IG|f;7a{wJ<_`=;%n0lZItIqew!Zw@++qd-s6C4#{AcA*!j_eaaW<1URcJl zt7?}T3!9F14&JZ8lul00Rz`38hSFmFh=3o_)UC9ZIG3j4J8h)ugjhv4WAujS5qdxn zhW;l00z*HOL?0o=&(tK-6e1NT8+~!sbnU;RkJp<{xRSch_Ht^{cvuLe;vW#WGzuJf zjHFm#adDdjOgKF`Rf}c1IhPZR7ca`rLyg_;qm}c}y;bzGGg`z670Woje`=Q#6;-bk zLt>uq{O>Ilgz^}5H0sy57PFlp9Z#Kq;+(B$>-r3T??Uq424sDiZQ+T>3ov<#^xVX` z8(0Wib0Q-7Q3H0HYYqFSWNivF&qYzy(N4ppq7{m1$!ymJJRW_qyyW^-&Kq=@H6>z? z5NadDf`EzJs~O^_qk7wA1dz?PD|$|c8&p4b>e>GOteOX(h+DWg`tE*kplb7DOH3NS z@~i7Pa4a}*(5XsAyo{TS@?J7 z8ZH^Lq0EP18J$vZ1?v)_SC&CzmtN90`0zhCKX4m;S4y0QKoe9J#MC7+LUk{mjuKzH z{jfEi;Jo5z_!VnkSEc1S8}?w_3fo<5^5yFHgj8dS$kx~9wkABmnPP1V*I*-+8od(o zZ**C`-tCRuZ|zU}nIpQtio5;rb7#ye1P7Nt>Mr%l*~4(>qBnV;_-uCw<4bu^0Zvhh ztb$3JoNm$4z^iNf%{YekG-x3Bf;FAnxiKNFcvR`~VovOGZ!p?hRT(H$wG=}@r#Zvf z=35btE(AEroUE#8!C_7>{>%~wM_(i#Xn?QX}+uU)MHX?_(=-`1O@&3aJ3}Rr$qEHsz`2qtIhgjU0xZK=aF zGf;3=)Z22y(}gS40>|7aP3&L6!LC~gFW77=w6gC8bXTc8MI^g5gzS9*|Hks%XY^hF ze$ba|xfK+~70}T{7?3=p&$9>q`uYJi7rX{3LLT;__l~8WjCtZ~Z?@4sY)G;Glv z${e8>w-)DH`d5D-rMDylN0lj>;}>;*4-6jFMY0Lo52!$`E00c701x%w94>*Q@t zeNwVDu@p?Jq|`plp33VL%FhkP6U@ORpxE5s;AXm-+e9S|0YvWGt|1eT&pQ&F1qwYWVXqGx@LqIs+trzSo<_%+q-yO?;BwtOuy*VhZ za#;gc)C54K9ln<3vxmDl=)GrOT$Goe8ho?y5EOkloqic&=j5PoGfu#d+|ctVc}Jh#5Y#sMV7hn1fO-9Ywn&AJ)}sF@Y_@soxYT*=j-y5;<#3~`{C zulKaxDQ~ITWA*tXIYyO$SQ7ksT;-T>o$Y~|d?>;*0=&C$DDIt6Lez6Jh)ztW1Kc)d z2a@DdTwNu`dwC$wWm&vsoIE-%LBkQ<%nWWO2{+{DHJ1Hu-t#YS&zQpiCisaqRx7Ef zVoU2#vsA_7w4a^;S|b#3ADvEz^P6NOH8KlSRXP7>zy)F;Iu`5+iL2*(DIoL z_1LxbInCq`nevnD=gA;KRK{cu>SzYJE)d?iSHHOybsaNhB{VOAE0Wrp%Zqw9vI}uo zCI)Ve>3Z@38Eu59_GcQmc-(lW2^1H5lEy6#pbqvb-s|m^$hTGap%9th$Rr)_u))oGJH-jXW^Lub<`RO|Pc<(bFynIL;OI54G;h2#3)>jqO$w7ntcA|gv)bt;@OOL`8T`-M-DRy z969=d2CDspe~T`#W6g%Kb)rI*_xESlO>*UTzJMf(UmK(qu`cFnWIQc;Ntzjo_!64I z6yaUQ>(Gn0;J)9ZVHZg0F2E=(?3!|$-E@Wr3 z?%|ndN=< zM%2n1Hf6J6d!YB8S;`ro>etYfWhFo=yV1NQLew)ZMODn=Eki^7DV>q)fhjKv4&80= z{_XIz*{4U1F*FRV*>=?_1d?AjPk~Iry32(Bn>Lj8mEPzS*E!Xje zs@^RX@mix!yw`jH#pDCFQ`|t!^njdXGzY4oSbR9>ogUf;zSk4mr*G6EchYa<|e7vfZ z;=z@f-2iyu!tq#Z=Kn4~JvwIZR-iKSW_DRXRI@E0G-G1?fjf?>jV0>>qU}$SCX)6P zR<-z(nFzFN1?UEtF@0{{N3(6x94PUa~GAtYl57!{^V&2erUar@NC{QT!8|$MKrUAAu*U{);=<`e2prDx) z_}BRbHTbFIyU(jsBUhLiQmOv;NWqRtYsrLwhMJquha1CgFp!oCQF)ddr7Eyp=LK&zFby32SSoA0|fjCBs0QL4^y2 zK*y&FsriWar|Pa%l)wTag$OBYeVS4QRVH6H-6se|TbHV}?pQ)gX{*II%TpNFz;UW> z^}r6hxL8^QO#DSKcQGL#ZIwGhXu_HKhM$nIxb29-ka1C;=U=7Yue?QN0d3zsnR>is z@f(H(f!-fylt-*Qg|4z%k;!*LVL(!bCTdmO#1%ULjOb^$xHui~GR%A;PwSuMYk_Jl zs>z?LU)&#~Cv1NP_=_aWUob*0rwO4TY+eKVyuO~gPMbDz8Fa%^j?1JTK_~$%hIIQb zOvP@tow)PUT_}ol(s`EgXMIh}bGG9R9Ik&R-OK$i5dpIF5hw13iOm|?v_J-el=jhC zQZlCQ{sXZudn`q#IzFNT&BNx~(G8w(H_dWDlIG;OLOTzj$|I{CgzViO8#S`$Cs0rt z)NfDx*!NfKOUiKZG%?+dz26Rv9?n6|EM@gR*JYTEn=?UzRY~DZLhv+sXhd+`eeghk zYd0SP(OYVrU0BD*sJ>1c@$$TO{p&W_ajMY&J1MzMj;r0@WDh;dN5V&w*2VHgU|1DN zD$usrxigYhwkvMbR|+Sg=Pt+h?}gXRrmyB=sY>pAB1h?qzqbgM5)?@(Jcl49hVilp zTTcRc$F65^FcqTT$|OR#4W1oi%ZGSm~$Nz!88JRNnq^_#!y~PF@xdlhDF+4;i3{O8eUt_R|b5^FKaAl#W$vU zN?AIN6Nbx6K=!pL47c7pUP3mn-wM8$5C_REE7dy7etODtu}J$hp@gu}EnGB_!YvZq z3`DHmwLWuBRJo_F)kis{-4D{j9@4Xv*PiQHoT+oP*r zWg}q5=yo92{bgOuDR_jV>h;0&oQ(s&r{A`6lM2zemg=27Dx2JrSo5ARbO=^{84IFf zUurhev+od09kpCyOMbTgLmo_!>$+IDtV9tm^6VNx?3GQo=;YSx?`h^)i{21nRCwJ?EZKB ziwGFXmBWT)S(ptFLJ{6>b|-aSzx5HPhs315fj};FX1nyeAFkbJ`V&WP8PyOvby7#M zJZ;(GIHKqvr8pL__Z)q!^ijL<4y@x2=)w+#fpC$tK0f-5y9MUinBdEYvrDy5YOri4 zVao(14m{=tXXBj_HNG6j-X$x=eQ_V1j+>@O<3%Uw-3dCPa> zfWS3-eZ&a%+QZIk&0V0EVX7$EvN$VXq8fkfeN^Z9DgMwl8v)W?(s}35{;fH*GOFFt za!ykK-a|LwQu7grx$Q|Yd|SG>8DgVj1Rs)&F&w8m@p|cPF`G#snE$pcD|suh9?hWv z2got0$i>e`l`L_#h^hc;{p^=W>-BZs}JX>xdx8*HWX7#9scD z)!E>k1oz_+y#Ex<*T~=cl&Lr-GqDIKkKulIv@sPk#Y-=7YQA~W@#)xJOX{MQFz^u7 z-__sg$O(EhDM87&S^gsV_w7L#!iucD|G)K3(2TNX_pvBl9^n%;^Ut^x<{Yw;pOpfh zFHC?6{$~vjP!`!wTXYLk;J^TKcPwGUBZ*0BtZbC{@l?raG@AbgHx=OsyXN&=$**VgW1R`EMs54(#y^UUVP-9;$S0Pm7a_OPK2rmXR z@G~Xp&L6;AXFshc_i3NJHhntr1{64?;55dVc|x}p=DVK5bRtY0ZAn*cnV+|8OvE62 zV;LoGwgX^e^d~>k;YF7g>YYy7wfzemIfB*rRxC^?oXF!LUnxan9m!^+w?C2Nm2$Un-lV?Zh7lul-rL-aN{KGOAYY;mjZ`S)+JWyeZ-iz%bsz;_ z^$yX{8`79sCv3^=pA(ADjBfoZnS~+O-ndH{fCf#6x%VxzY<2{5{MZ5}YuiDOk)f}} zgSpBM+H%IoRIZ< zYPj*sNjZErEqKdZtMQ%uEMO61s605ck$9x^5R@k5prHoxasig^>xQ`@-ue{7uTzk| z3HoX-Eu@W?(ylY8{nw+!wuiQr-`jiug22x_P-|Uo^qGu|%hE-a8K@LN1N4OtD3X{T)8<7}5Jn^*@dvn^z-K zBTcW?l2DF2^89L~yKw{EUXRWlZ!o&@7N7iKOF62Jjl=(5y7L;6 z084C363%yzX79n2ZysNE0DfV;JpvQ6z?cB`o zX3+o$Cq?)>&&_^GX2|LXmk17nh?ZLS_iVba_2}hUcNn)}MV*6wC++VJBqD18lu?@m zR{2uEgjJ{=_r;x{t2q$q{DW&K`3})Y1`w0{Z-x0u4Cl*m+pR~T6M{2Wf*M`GFl*=J zPhLxR3}1BRv^pPdwja^*PXoyE0+1<=xdHAu{e+gX8BR{Y`GWVw z2F`IzM;vBq7j%OZ9JiboFeI4gUsPfNy7DM=C&^jD>qLjo->S;-N$Q! z_&tkm%?7yB#ov**i39icDH(GjWmZQC=e_1y4X=KxFnH3=4uyM&P ze?h^J7k@?xycUwR=6bN%oZ|}yL0<0OWsw+6tm?Tbr5wfVj2pI2*HLM#sdnKTtXC@4 zVlpxToDN7&+m^2=Duh{m|Jt`FQkDN&)Z~srz@13$!tI0`Wn6}g@>-^FJ@eT<1%h6I z2kyg~vcwMcE&kphTgZyF1Ssx+PDN-Zua1>gfgp5<&d4>9D=p0?(Xo%_iNH&K0AhnS zABY8rSrIF1Sv&~X3SNyw5IDsbni`eITB$hSl8aMW!0G{FaV_2l-q-$+nvly;SkEP= zKYh$8EtV?7XTpF2~m$clj+Xx1D^DLer!uP}>S376sR~+c16UgaB z&?=kr0zDz)s3Oh)Uklo&1C6j`^GC7iW5pu&-vkH4FAwhF+)hk#@dEgDY>V>L^ird^ zthKq8tNg?OwXrE}xwk36x4%9GYfF!)%-M{}<2sZa;wf>!P*neCzh%?L`3UMhWh-Kyt z6tPy6S$2a&m@&lB#5C69bsl)_OOCZ);&K!xEZV5CGk+AcN*G4Xy>CJWmjnoO>Ctj> z&o!C_;=Z^jvCc>`7CDOa;er_ShL_dqU8|iriWnBDOEL|!xWtjcnfXC|w>Dgj)%m#? z6ZiA;Lt#r1ILq#pKdm8F(!2%!W7tBS3KTmL>z`~j4?ae9wzH|hMy_5h$zg{}Ow`5z zleL6MGKcpu@5bICL2-Ru-JWj1m>V#+h3}mezkbjjzRB z!ewNR`yQ%_v@pp zKCO?Wldy>n-FLu|`!4Iu14DtEcW^CWn%7QsMYHURn`)9V;aCK*;MX2RF@cx0Lz+R|#`F&`uN&-Eit5q)H1KyyHcuGpOlXzX?T_gmk;5`F) z(v_gl_O_v;2jO1-+3A5a$KlGOn6p4ydn_Csi65=e%#Ncc(_a>`ECLNC50}n$Npm1r z4=M%m?CGVH#OlR8jXQe#eK4TM%@n8d@t3!>y(ItTboELjLDtXrWO=rsi1%|JCP2FF zBbn?Ra-iAee(JdlNnT2Fd-*i=^PeZEfJ~sY@DXhC3i&9reftrOrj|Gvyf5Nl-%=gC zB!%<1A9}Y0L0>j$$-lGQR-;gSHk-_jccuYeiYNyz)bbz*?|fGcCBERa5$8yN{B!950?i1@eUFK6-JxRF5f-7z@Dmh$}O>;&?5W}nqu8F!5`Vc273 z-3sY@%{Oj9&Bw|b9V<~h3L@^d@qD{tAzTk0vt2jZ6Ncl=W`MpZ>C7Lj>(^WDa$vL; z1g}*Giv0hIC!!evK=$fUx6C8;qW4b3lWzEeqrW>7$z^b=s>b&4$+19)?b8?a>^_9& z+dil#YsQnMbf+Y_wuNJewAeNrxDn1Q42z#R`s9yEqm9zciZs$)#D&H!i_$pu1jqpk z`uuFD|9jYlg{_rBuAj@C6%~6D<5r%0XF9?mpfKLL{4=T)FdZnzlRt;kP%{RpXi@MX z2;@a4!PSb%45rBYOb_;21jI6fU{Q&ohhM&L1vXl^nQ5tGgXpr|h{bN`#+$gm+(n9T zY1@ZbMwSEpecIVLfLn6)w#c@l-yf{(z+|<+^>OUR9Xx)Yiuh*m_OWaB6t*>5%tX){ zfAw24l)4lRm+@M+>g0vMEzh??@!O+>+XX?*G^opB-4fX#!7DY0K^1UM$DAaJ|FLZxyfM~H_=AKj_Y=QTUV^xq81~b{< zS$vAc1nCsKSH4{ZBWA=SZ0DEMIv^+T!+Pyv2~VngmHtgn_lksD=o4=cAvWm0F^S2} zm}60Z?pdAE$StaYcBw-%FQ`OWS;Kq_2Qs?ZWwQfmmWT0GweX&j;r{ZE>$NSrhc9_x z#Ue32jc+);>R}1OAc$$PK$27v4-SfyVa)CH&uB5R_|KjT<{uzsIBZT1)8`79O?(}O z1>V7CejWSWu@0Ks(#(nR>vi@vXK z@A*ZXMU8;Z@JcyiyB#!w^?g?TapkkVZ*plWV3yxwTvr)_KAG z){q;J@B3xKIl{n=&wtapejl{a=Y!=T=`$G=y|j;st~oen;rWTJM3Z7{+?+ZvMVXOu zTDNiwxkkjt5cJ}k-2b;l&b+L8WbwJeL0Jj6p~iSM8-jgz7wt=Vo5RfslfKOCq@!yU z9>uAIysys~(y%zXf_f`(;Y%SxV+dw=bJRK=DcNm^P!_Q5!2E^>AIb*EW`F7{ViKDXW|EF|@rmD8JdY_MrZIX0d*l z8LGtNsq!UZyH2K^;-EQ!) z5wz>v#f=|gw~r(@qJ7)fckZHY|EW1}?x>tguYr5KtW!(c2M6vFKGri@NZwFZ`hAd! zH(|#wqS3g?Pxkacu2D6+(Su_oh2^gWw3;b&JM3}4ExGdBs3NL3{2PX=AgfZ>Pc6&^ZB&ev=mQfGj+C(x2W`3lGz|vwucX8ZUI9RpCd+@@eqRpU{ zUyQ0o^!@LXTX+3J9`qS!rV)CsU!5bn%Y*P?_cC%`oSzLXoL&CfCWpvJGWePKdTo> z>N?zVygLCHw%*T0HTJL|pOxbxDTd3$@oI?fiIKD=W^JD2X{+4L z$<)^+SIc~Lt_+BGQZvv!r8W=dfSlXU6}{nld;QVH)H8A)Gg-zegaB(lXm3mJoKqVq2Aex+tt2y2ujTt-x81&cR&(+TxSN6)im-nVZb(lmtcObs3 z+kBp2D0ii|v5#1IFIiW)KgWD7hUHuf10YEVoqWl)*P8qu2yROQU&9H9B;u-qX7txd z(r%vp)DExzFY?zaYFv-cT@4|0M7PmLoPIlr{AjL1_XD(Z3jMzLNaenY%Gvw%XH62d zY*N_ooHkM8I$PS^uvUL~{6)!PFk#gu7j6I5f9c3C(|3Q%&?7hZK!0#Ab5oJ?*UKfv zoyfjh_PSPKp);nU5R4{_pt5y+sN3V4*53{xfo}LR|N0JG=?LLn8Op^Q&1>N2_XHz)jt|l`{?JH?l(=!a``V&P%k$hK zjhldS@|ZEimlUXDd(0xeGwGBfT6?`kbIRv9X%uGqP_HsBHUP9 z)a$${G|}dLBU~=_PxC&G<;dyzWwNte(OB(*gxnvpk(Xd2)qNs5=~DYUj6IUdqb5A@ z@ur@vD(@*X*d4R17=L7cXJc6>hSo+!!DE|lh!GS#E6TIycbrFg0{8O`ly zN_}>{#OYTkx&Kj)K|!6O_4TAlpQhUG_IH*k&+lb81S2S3@yYBD|Caiu@KR=UY>XE+ zH<#6^5woiaWfwpEs%gD(zk6{=E!yQvr$_=h?Q^5qC)kwU&|m&5wkbx1>RL%+eXX2h zb6{Ov5oCY#gWO*>V8DF znOjn{q9N%1IwkbvL5BU@Eppc9X7OcY#9vJ!{<6@Y46@RG?V%OpE@*F&8|^$~#VTpv zORF-k*5^H(DmaVqjvu{ng`5oE6I|ILD~rp|Yw`WG>C-94r82+HpV{1)&5@XG+o`LE z@ZA3O6z5cAGN*j}!Z`?le4}&sBhLR)dHIw|2kA}DwN>=pMb^~_QZo{s&c;jh=9fHv zWuYwrU4u>ShHobnKa?RUd}nM7%-!VuzLr(Sbo`1@_aTyQyPY5IdxL@tyP}CmTQvM} zcixyp(RL$_C|A?jT2Cw;@}xag7x&(_PGK_=;Yj@TG^aX6)Pxc)+uO5uGgctOAw#t$ zdhYmOY_4ib`%xe+(Zg`d*ms%|eK^v?SNt+m4o&!UzZY-yYOH$yb-|rav2t>97wD*% z{7AjMu|RVrH^tHqrlqANP?+BSNR0x)hCSEOeo7@i`SS$^NSCXfmCM!^{xz?{Nl!nI zuG}HPLh9wa5{8hUUBb$OXJ&7o`kfgoYoxQfaZ6NZG|;=C&M|4${`?*Fao;?PhTowp zbvIi}CX->X(_BaN6=QRCCw8U5@+<95jr_1j`rGrIAH~I9y&{q?qE{{jHYh&9dNC@N zb1?nK1XO=6DA3SIzVRq9upWp|>+9-%?$?&r5>oZK#}IbM4-AAa-e5 z`Lu;j*rfCxJA1x~+Sj~1)p&hVP8&vlsr za#QdZCLM)GpvC{Cwj7=(aJ#_Dz}7a$wR)P0-Jejx#=@djn@LYkA8(v3V30vFG@{`8 z1XBG@c*%L_Mtk1fQ7o!#h*5lkZ!dI$GhERPHu;>>VtmhFzpMG2L_T zGJ=Zjjm@PM@LaZ;!IL8L%qg5nS6Z)MW2F3ivfoaJSesM6y3PLL=ed1Q2l4{dCMQsq z+8@Ro=R{qWUP6^&b#-)Nf;JM6&fb^Z8uoIJmodj0l9G~s;9s}FSC)a$;qYjMlUIj8 z-eSeDWq)DSWSXai7KNhhC*LR*tSrUP*)QJAcq3>ytm{U|PfQFp&JtU3v+l_=HAjX# z#!NBZec6Q7j}?-8Q)Rwoh_8J4QsXy1I^&RJZ8zI{Cf(s|=OEamh4;u@QGhMADAo(b z{(EP!9LB{&&)aai_Co#HeKWV5k4g@LbzS#f!;F#yN&6`lN`Ky-lK!LF%&r;q+co!N zFuf}`6#8OrbI-G#H)WAAc=uJ%X8P0@RF|9&^MwnGgtDGw9+#2Zr5ng2f_;2;Iv6;L z^g_uQn46=(Fv$DyTcBN9Dc_HQf#Iv{c$U7~`7Uk&ry=7iUf!q3bPpRf40(18cs6nS zQCm?em5lFE(-dF778H(Ua^zP@*G$crw+hriYBEP3VXCLO#6L9m=-u@1tl<(Ri=9Q8 z^41@GGf-Mraf;hp>F}7fL)w)8{XcnM^Y&e)$YYAB0TMzg#0Ur}yJ$Uu~W2WDd zz~Io(QkQto`1~H|#CE8ffcKZ$b`A^-WRH%Hx>$O2OAf2W1NAeYn3EeCN z7|dmpBNR6c^!0suc%28?!|ZXH1@ZkhdeeE>f)|;dAgcKl(Kbo}nJ0v<&6GR9M`~EA zjU=Q-ZN+Dro>QeuyOuL^XulSZdaO%!7I`MbXC{>K!%f`}4c~HGOcmf0a!#JER!Dv0 z4_kJM_?Og#4HNsQ-}IEx6!(Mgb}Um^N6P38$?AWzBcSNggQjP}bd|$j-?L1AZwQI*t1Qgba$T4*xL6Ur0T3 zz7uF6<|QXzLb!JbbVMZlh68~r(<)a4n56ZI`$?BZ-SPhNMY@(XXD0`TM^CF&^J{B? zj=W}~`p{?vd?|6HEMeUEy3Je%VeE-T^&UD$rAXpJ!}gBatCgWbjM;{F7;?Ep`=(!$ zz3`D)j#h9=Zf2}a7{t?M|KP#_)8uP$=!uB>z!9w4VzGSm-S?sA>3z?%BpK<9dl)Ot zs$K>a+*P`1J;-~ZI8^M@$EOoq$m|#JnB?1+vaVelvGXI?wM2eCb!c8E`#o6yM?genoBly@r1`E z8a=M)_Psdt)OXOZ^~vMK;Bj@t{@+#s)0+wxN=N5k+AUux#s~z6{Bl4rGI*I(onKr_eA0?2N$ zv9aNHFr>h4q_Xnrvb5XejrKJBIUQf$8lHUV2MsX~ua{5!R)xbWL9Vq0?4riT;#M7T zCudj$2=HZh)EK_8unhB3E)op*Wb9y}VmD`^EN|TSD;pD^t+JWH+}f;Ed5a=pf=vN=}CSCa=C zrr#X6N?&f;v+bJbsFanjhq*LjzI*oS(WZwkNSfYensK=5bfQAup2@IbHmUMW($#+F z^(oovVvtLiS6m8sYXmkprJE6OM1F(@+t^pR@~B*q367 z2Mz<|n`pu{hZVPMX-wSsvvq0FxM*$?eeAi?CU)1m2;I)~X=3fLqx9j#wDvdcf*Gfe zvH?oNmt8xXR(2m&4dR2^ILn_;bHr5$B?~7g0RqStj@9+Z>+js~Yl;zD-D#fXI+^pc zB2}Y{89$ykX16h*W)tsKEtu?OU3O*PQl%zPP!Kd;X@$Uc2@;y6g?wEJRYRZa@VMma9&vMW*p zwKH#EyyD|(B4cg&jJnmMGuxs8Wn@{>Q^P2)NrbvG&M$9syUh42l~6P7o<}^0H%UuN z8--euUkc z$5E4yt$oR-U0}RLD=|pl!~chpbNpJxYtgq-%2K5}yQNLvV-MHcA_?nfLUiPa^o6Fj zMP2g#mD%C;g$(LdD@3mPQgd*^t)t>V+qSI|7UZv1FU%<)lcef~EQ`P3lan(XA^ zoP`E1_DcN(KKApz-#*>eU0}oGd%{&SC{eT%?1dQJk|G}bD@GJr37ORKP{@c6Nn^i{ zZ?ETU+kypA#^>mZhSo?!+B)I%ZJLU z*Ujr8ubhYN$C3Rw0;}`8bz8I64+L^2!*}A8>epG6a$m>r+v$5M_!{6G%)ELOiFYHI zJC`m5l7b&t@_QVT#U6LDImQoCrJJ`%*y11((n7 zsjk5vwaFs^I9H)>hW<(OD_y^3g^^+b0^JcEo1?m#)-aMJqttiPeU;cj(#H;LPtQAsQ} zAm{+&AG+=dObiv-+3G()d3ms%ABk#ddBFa>k*pR>sgdwMRTBroX&!;n5ZxOt(@8~+ z;YC!J;b4?|+uBY+Rh5y2g~ig^TFTU#k*ew&yRHvc+cQ5ew4e6c*=NVtRX=Rxm{>?m z61}MP&NBms=WI{{hxN^X63JfYsN%~@nkbKN*6ssp zRkd;^`3+QQ*^^O^oI)ie+SJ#?&XRUn_1-0|b!0cD$w_7u1b@-^I^Ltcmc+!tiZEi$ z%sKD+iaQJ0lyBelwwCbY`GNT*(hj+rsK13RB^0lfKHHfrr`$W~KgL+UlZj%;3d2;A zmyJ*w9>iHOxoZ4yQPuV#zR)fi>??o8+V&`v3~eEWH>MgxRd$s(#a$4Iiav*pBBFj; zpU+pn^KxA?+Qt4$%xI*jtam-GK4EBCdDIP$7w%v5|o>6zAofWTFpf!R& z{X(G~ag#W@Q`Jq@O3DU$z}(%$w*+1G=oE>|lvo;bv~MIUancJ&`Qsa-y$ za&y(N$g;2}aMfdJWo0nn|rS&pU!ISFmk-LGHQBgtLwqOqK4Pt$s+SlSI#8>YH-+?hx$mM^Obo z;p+lCNoFKbdm5AlJg%JACJNq$t&+R$LDQtC8bQ!5t8-kQ&3~>g^Xrj(EiIr^y6rpY z9>w8Ctu)d(Q!bQ>yw#*R*Kd-k$D;k+$dyvuT?m!<=)PXgLpA$MeE(%PzK(r0LO!?G^gmM-r_(?PB*uh}c5cTkS#$-XWYXSAx55%*;X!n?lv&5ZWD>Wts zA}p@x?zhfpcOohlb)`MUi8{J$qwuMQVZ%gUb!*kufF2wj@aLVbNosaJzlJ>GGRG-> z*LudrpEvQ>@v}PF6;-fwj>^%*Hs{&f2mA7+wZ6NpDJ~U4uUtkQyoIbsr`){fovMmV z9G+=CzlLsf%-auhjQI!J z_L#n;N<4ro0Y)wL%T>`ZxZ2WhJZSHnYMWmm{>F0MNcLHA2itDVzh6v-=1J!)b#9{7 z(l#ev>!#XX)7dGy>T(wPUVYFk8uV#$($d0zrDA`LbiDSczN)Uy98D;VF#Kko5B_rT z;(bx5624*zWV2vzuDHJidNMQ4wykn5nEV#E&XD(1~+J=5rpwvtH;Fz@z~nys#_p?Xb7`L zQTv4!(e2{4w()Se&iy!vroZF-$k()1#*#}@B}*64b*&tKt*EU-I|Y)7)j#)PFJ(PY zkeYp-m%N9ax|FV&KH z(ltxhm-Y5Mx`juTJYu8Knf;(NvG05MB0tBLI-()Qnk&JBZ0jy#(#7j?yWY2~B3yiH zPojhA0TyqK6cGbU%S;ilM6V}LWT%~;KYxfhT5-$bU<|vn->wsM4R(U2Bz%2pRIS?X zac=2hZ>C|oimLgIiobeL2?J5zO9w~meilyXqe$Vj7=eZkQSvWW%x)rS09 zwNXq-v2U40rksIXZwa9lbDT6@Y30{mySRj$@Ng~5l+JOL%y}WwtM8+Jr?VZK11>bN0@oM-R<@Tkh zNGHVto*J}+p2{ySe*PvRf9>tNhk$6qSAmEW?`6yUCLm-LL?Z)Wbn8@RNUhDN;hN&c zTZ4v$%&x_=G*pGq-{SVjEZnH$gq^*{v~B5Uu-~@RqAurrzbzo@wuPBF+F21JI}7Fi zF{ic}6vSsIj2CF|)sTW5tQ!T99@G+x_J;@Lnfx> z&*Hw-6HV)ZFkuH4I|RmRdV+aB1#lmR*i#>>hdz#!6^=N)43Rv~Dqr~|?BYN&Dq1rX2R(RNRGp{{HpfLt?`glxWKT)~G>WZNT?l#T}WQdJmnSDBr=}a6|+&ln!Z}$YpL;dd99>V zHou+ysoZ$3Rss#J-ic+wzvU)`HF5?H)c%60Ha*?CiK``r* zIH$k@c`DFs-ru2UBz_>?yzu1e7W>|^Rn8P?QZOY6wt)C&T&^#C=(k~}*5T(2dNyHa z3lq0Do$Co`37TYWW6A$?6<=6|+7LnCWRiQ7xngzK^eROD^X6IYubD{Xn!8rZh;_OJ zRwKx(D4fdQQ*1dHVsWae8M*=DCUcNoO%&=VG*NK9vvYNJsR-p78w>a7b6-f)17mP!=0wEA|ee0fV&B&5gYe8fRB1&Y-tzJgEhxtimkKD*TI^ zL>7|y>t)NJ+MjKYp_zrBOe;Xy2KYUnWYL%gjC)S`ia1frW2QtxFE2QHdt17O~yKxL){wVgdKD&?etA;1`>6n-QEKk2t#UQ?kpv`s9lf1B!AHsW-MmQEmd77AA zsYCQ%d=r&OlXAR987qpuD=`@woghuijz4B=uWm)_!+8Kaqfo6tqs=z44jdaGM*cX<3}2U3Mrq_ zt&qP7NKORR^2yPQ(Q_-RSBl>}o1n&qin|diV!|B67dZIr5^K=k`{;nhPg!5tpvVtp zWsco!CkvCms8%<_fZxGXb>!sWkjFNnv$RmVpkVwZ;nEIdfLLv#O6{32!hm4{zgs$A zuWux4_Y^*(WaWzH>%PAy=`;;e$yz?Za=BthJ{-2dBdz$NiORp@_#2w9LI7f^5XTmG zGAhnmDu(;-SO}@yfGr?JwbJd9kbzQu4_eh^nB7rd-jZq$0}lSU&2_{H(<#ebuis8K znDn)``%A_mkCn(z%_Mi~W)nKJypnLEsVWVFMZ4_f8OL4KxzBMT7Or(UgYR|_> zj)hSCmVI{k!afM{PKQ8EGI?{ddCR_`CKNnEMjO1t3Me!C6z?(Nd@ZuZFLQG+64A}D zrfPU8ge~f!+WwMJ;a1@Gc9xmM3;&GoZ@|Oo=~S%6z|VE$@}cYMFBk(W_Q`?+7JbBz zEbmn1kgfG2YkX#7KZY=L8a5U071MGm7)*VJ!?1A%J5%0_mU+_J8fX1bZcV}A$Zz@h zM;IYWFJ$one8H{v(NaH-*ZyiaexEC>E|fG)oE zBwGFNgZPTvanZH5W#+BVHz**KidT?9ktv%!Rh@h(`)VK*O@iNxXS#x`F$ntFi0~nY^i8>&&ChP_fRWgA8 zi&Gsh)cI`b-z~!VT!k((d*!*+&{}P?L zahH)#5!Xf6UrmLwv(IT6PZKsTQ60f3IhDcJ^LJ5N4`5 zt(xn?v47wQ%&jBCSl`f88M-+cpd^HNYvUy`;XOsPni>-lw!9&1a{kE2AzxoNJ|HlI zdw@ZqR|NSy)FNniI?k=AVz9RG>*!ynh=>)>*T9cQZAC0384(s3l>=tC$wKa}G@vak z(L?oqkq?3~6_%~aB!`J2RCy>T-|y*7lVyc^Qo~jP(zx|}dm5QLR(6OkPSOeGG4Ih+ zL2-?AG==L^jUi1@H2O2hwXvyjo7^7LTt==E%KNFX2&WwV$ma4+T$>%MT)(e;j1G(9 z|IBK8N5^~3{!Ld31i*(UMO;R0l6EHL_eLR=&sfFb%)NE=RI}Av;!I4Tr#_=X@rnmP za6tkVU}5RkE|PA-887Mgiv@pYF$hy({H=Uu%cv!ka%iQok;u#jAKL7_t6J%D66e>; z*thD{ZXD#4^;D{ArX;Ebll%!`?wgxAZoy+Ta1gH??Sdyp@Qa#1>sgdvk#PD8rU^3E z)WB;ShK3d^mc+hq%)xzE$9Id;W%b04*12U{WXep=LNEppM>3yM31I>0J8z!~e~0YX zCq0T21=Ii!cj$QnbjAFZ3tg->xB79%2O6EpCOaDHpqifJzgA%-K6-!Psqfsy6H3JB zoJm=>1-!(ne+)h2;xge#?ZKzRF|vkKo)U3$yJflXgm6j)ilkmxG*9qWw4i*28*boJ z+=}(Wd}*VxM|EZx&hpo(9N6n(O#Ikanm-#cq!L%`?ow#M#S>b zV}hK1i;0T-v!fv<1JI z=9F#bS^}znLr>;gIpp_OE--75wylOrYzGB}j;=Y)lRdxdQpm1g+z?>nWRo4SGg;B64||C__&z$3C17or4PhZS-8v|73}+APHwFa&B>jJB|}Xbst5#JeYl% zbaaGt?^r(je|v2vZlY#ZV`t>8sUP^`Lrrb7y+?@wU*T7-&zEzCt+TG`#k zK8+M_9(;}rc!J>9xDk>m_`fvl;RWrsvEktbkcOp-DjdpuFrR!0ba~yd{95VBHVBA| zGFrA}^-wxm=0^v6K-aCmQ_>FcM~C6`ePp3=8wV?idfVGW&!&tsnBC+}seh(&2TOq9 zhdd5UZaXsP;;KI^HTYH#VVTGRwWOJ_n3wkO_J0S{R}kKrfh7N_1Q4|BEKxxDAX&zN z++|LBGM4SKMIrX>2gzj~eC%ELp`X2=koZ}_}k+wu@H8r_bp&^`l2=6KSDpe%(E?(Y#| z>jhR7mqe*U-nF1qnQ2VQvicFQU|txQ$u~1Buh-)0mi$S!HTiT%_WS=!Pj2~CzKrgI z-o>#5y$%m2oa5HZ%l)Vg#a^qZFl-=Z#U#VWTxQu1^ALPtjI{fk$fre!5=9*#Ss9}w zcf72$$?DG@YmbZ~)v7dRl9euhE79|A_UOG07kO%LDgyJv^P-(`FUT5wUgTlTj&iorPnep>f@&aH^3i@5mQazYz{)u`>TlH?Io| zC-HcBdDWOFyhy&7`R&}5^6(%Y>&{CDg&={wc`h+o-D)?~z}02qFClXJgo@8PEoEG*pipUXuCSHJ3Zc zTNX9dFtkO8wgg3a{M}F_viMp=QZN*EuGKwFu?n83?e0#R@4kElC*xw3lmsyeY_z<6 zqW1NjAp+wJwQ{jNIeFu~#3WusP^|$6ODu<9+r++eOzP3q`+43I9LOH*!;<=^ksg7u zwY7c2^ke@AEOCYZxFdA-cg_8@fEu{$cj&E$L>TLBawv7RdWH~PlcP~6q0bH2#Ja&4 z1Eq~N_B?G_!cFvHOz_X3?v5n5FE%4?u*m^XIGsZ|fGM)JT{ZWAeFchSM)q(h%)p`9 zqTq^AH`u6?DE@GB0tGxsOmoD_29iXuRD{pg?+`Y|Yq*1vCa3GOnliN%GhpIorXW$f zz41S|;}h5TX1|+XVcoT{zL|?^1UK*K4>$Q)pjf=RdiG3ARfkzsv;zH*p3cbJN+Id- z75E4CXRHkm-jvuyS!ml(y#-vDOLu}dwd4aoZTdniI(&hSX6g0B7N5&(@yRzz*1+_L zHe3Ouk}JQ+g=P<$PuwkH|HDqPO`ta!eHZ|Pz+zB+xov-(~SPXQ5Bw9!#h?&r4}l# zl9s+A#!nCN#WQIado+^fL~*|v#3-et$H>C#cQ||>{z|1<9VxT1U;Za{L8F<{DdScvU6Vk_CFfhcrqs$bavv@m zYaH~z9?C}DI9`UhnAn5s6a`m5y~&=O{8hBPoRAqbDyWmj!|wCDEm-yOr*hJ

H=E z2zda0vpDlibU#Cr<_4^GMG8%zi-a0|JfgAecs8!@AGZ!S`(h9;!nda1WF1TZZ8Yz< z2ye?LPB(x7{B2%^t;w{I^C1PI75s^dlgpZ_mz}8E0RPQ*2-W9u`LEX2v7`+ZIhP8Z zhgFq}cd$veJc2c&yxoQciw|yiu#70+0#6jT3>u?--TF z=qReW6G7oonO9@7zt>af?)q)ZiX>UO1Mb*!HV!U30l048Rk9SB;ornBIwen^koGO$ ztP_>zrtYm--L>O7!Hoq1Y@FR0&?=WmpxFJ-ehEJ?{ zHYkWz7-i*3;5S?7jR)XjWluV!28VA&-s5Sd1)DnfWK(xt(*k@C;H(H4@HC5{#ZS*a zfu8^GDj9VmW49!1@*D+3b?RK9SZuTcjw9!=jawIWx48@?QuiYskmNH2$^dv!yRuxC zchrLrx_H6)prMb219S%PPu+sN2M0&y;^;20Ur9XP;xEWRiX!~K0WS;Zn)}?%_u%EXX5&Bp6n;KsXbUDeN>r!L(NUcW7H zwz#8xHkUfYGdbN!G+asMnOCuFMWuRCgJaUpFYc0$KeAz<$ZojTEnQgY)l4IAk>r3$ zxTP>LczjR>$wA(8^Xk(<@#nfBP-c}k7!t@c&Od@r80=$6WqKvY1pB6;y36x<$`z{8 z!P^C3*6SG~PKYmZCc3N>A5XOmV&<2s?=oq1kggL_62{-zpKjXuyC@Hl)PzMr-gE@| zbDmrmEY-bzQowBU)z7fjq|W34Fk1Y3d$or3hi^Xhi9WP_&;!F^NdE3#6d|7QrF^)o zScC{+#D2wk{R}_PvAm|-*GWsjiXvlwjSWq+T`rV(9e+_mc7Jhk0Yj8rRxPz#M)-XN#p}KzF!arTC5BeSnn(?? zBqb4@r*|e3q+Hyqlyp5~uk}STVd2O9Ix8D#l5pQ1MW!*SC!6X3%{^%_3LhUg2eWq7 zbL<1k#-GXQLV?q?hBfFh4pwFw zn2y5Zq=<;ZM{V!isZ=KYIyvey61dr)tAJG(6v|rT^nAE6b}Gas6?$t^B+T3yxHE#+ zz$#BUrXT>M)cAOGy5#|W6In$r6rc29ju}(jF*1TPFv|H1psyEKg6k}#7OGs`sRYg= za;E7*WSH+*RbfTe8vLJ(J#e_&(J_h8lZh z1yV=qqO}F0K4ib3Xmt5K0qy2KCaH9IbH72nNDQiOjuW{1SIic4%2Y_j+jOu0ntV3# zRS;B;35^~7I%Z@Em)*)&4+2m}WKhWsbCwBsZGFHiS?|Kaq>hgBpn6ADURGc~r@mZS z{lW9?)9)vK-wK#Wng2%gSyprk382znzrHf&_Qr!-N7-UyC(`FKW-rR3Ve#|85F>;$ zcoYZvnGAiG4V{w1#i4l3eIahcVr~N`FBxWol#|4pkrFiTOtNv=O}oSPgGp|!W8=2W z_~|z*CcZlp_M)Z7&7!5hg`uDE2$v%J;SmS=wig;!NX4hzsjO^RlW@N7KaC;fLG=cc zPZO9$@P_px?>Qlj&ga_kwvb%9VB#u%R0@ zdfKcba7NjCwVPNMPR#yN-_Tj8Zsyn1m5l_QTB8^prIJq2Hk7Mb(#aRm47|CZ@g)_` z%a%)Et6n?Vs#mm&SHn(k!}tNXMKaOt)6WlBJ6FG>6I`12L&VZq7)uLT{snFuug4>V zc+dJM;Me1!FEuu@!(?%&0AjxSOWOyvtZNu#+6uXfSVqK0x2O`HiJ~A|Z~efd&2#|fxw1m?S6>eVfDXKp|CG}@h7gzzN zlQzSp_3<`7aZz{jF(*-nwdZv+Nk%^qPxMr!l7RYMj|eglzO+mQ;ZeB{^a4Ju~ z9&iqxtXn)dATS{~_*nT7Hq>rB`8BeVW0WB_1Hj{jhc7Pp`DM$m7I*mi;+mObf(|rN z9!z<|zmh?pwKAO%!RoklH}%58)q$o{LfNzATq7eyHh-WxHzC*GcpeC={%_qLvn^H{ zd#v$)JV~F9b~Mt0-d~_|-ch_PrPiZpp9IPwURm*ETnTiXS~qwxQ0=XN_ra({<9GXi zmb|3VnQ879c}r(zkj3Kr6ULH#gEb-s#)JUt@hIaAk*lI^2ot__7gVI1oFXJ-?cK)x zi!O8&CXxq^1@=sMoAKX{_nh?Y<9wj2Pj&KR`K2V!ytTaI^n>`JZ^XGgREZd+$H=*C({%a%<;SkO z1o+jGv_DZ1M&>tG>O9;}jN!7?-TjlY{UcLTLD&+3;0a(y&?ZPAzokx`3K`kx8WkHIuFQ=rcvzz=N+3QvLZt!>H`! zCDfin3qW-d(%+c)QrejYVi?wz^(~!;AX=e4ky{9{9ni=U%=;l>@qp{nw zEf1EZF2o0IFdax5S;`)7lS4&4%pS=iYDc98Y7UR$#w+0ygD1CX!O=kE z&~1$XMoRR=NG<(3c#mh1CG^{Xn&bM# z3&wMxP&PGH?^R%--AosoueX9}{_@4zCBI~L5IiyC$rJldn?HQ@>J^BU0|50+-^?kA zGw2tG?}F#S3Vbiaqb1K0iv6~9juto;fO20q??KpX;`v#L(e2Wrvqhem+k z{&1NBvP?mWg0flCa8U{vB#oI`=}NQN7mSZfvLy+}1CR>z8IID{b8pv;g5w4l9B;F@ z6nLzyZzR;@eHzaQ!OHgtGBT*|OfVlECJPg<*WvqfW@0#}_Q_Og+q;u80}W67v#dup zfboc_D3LIPI1c+DH2o3Wr7M?{%)xO-%2U(9<^pqQrsIn%0~oav91 z!`?7WOk24oRRmZcfdJS4bCcdPI2h9@$!Z@+e?lNxbqiubFmJA+R|0kx5cb0QXTURo zAn9ls_x|G0K5W`#5(+t`nO45?R3lj}ENsS#848*E4|O7HY7O*BcIK>OOY((q&Dp433jNZHD{vjs+D{L??2 zkXo2S|K~Lwq=-d8iM$sXWPAcjLbC-$(clN%sydUBW_JAJ#7%8B2L^an0ud08H5#@9*fw5*VT9Mvvk^ zzp3Hg(N4~{Z-4jd0q|9Wat;XM*X=xkU;r69M5eTqQ1mA6+JJ8opdRDP`w93QFq{Om zp?i+G5IaSx8xcUK5wd;)e9egiZfxC{!v!7GSa3rjHq>8UegB(Ii5sv&91WWyc9vpSlQ$9`|ERPt zb=c3g0y-&A!3&}}g$p^?r{6>?>VIEVc#6FO9K_z@RPfO@Zqi@lV1*x89QmL?!1OgU zl-i#P+xm?M3qYylrE(?rCAPcUtA@|@H2)9}ME>szKLiD{0W+|pGZ_{{YMVMZJ^+m4 zu^nZS?{d)HyN5elMANF~<}B28TzC~&gOf_)i`^#G7j9C$SC_*tw*0$DNYi&}v#R=f zZXMgfAwJNMSQ!HZG$cH7J^SwC(j@zOws*{w<1hN$w%dS%NQmqt5;|UgRQd5^FXsO}qo4Fknf?8J9vGnhiwysy{IpJPUs`@hoM^&#ASDg!Rtm z&n)hcl*FOe)OY^z4?O3Ri2EF$Vz=73apzRk4%S2`;OX^I8yf?IoHL#M;Kb3MoH!~f zDjwM0-rmX8LFxA7qeADuCT#l#O*3*EJ&w^A^_oZf081drB8>c}9$ ziRZ5*T+)sjyeH#=cJCfEY|7eP`ro!#TwP7(NlHwtHfLh~0I6L9bl#JXb!Sfsaimoc zLuwf4J9psKqbwS@;zT?$bB$wA4j3CwN`e_#*mnPa<$ZTN*6sVZG^C-r6{(EOG^B)# z(B#UhR8mGmW=6KisJkt4scaJ2qs(kg3KyAKQRK2IA^UlsRQLURzTfA+=lSdL`o3P> zU#{!(`JCtHIFIo@-p6q)9DT(dU&s9_*!Flu^>t<0Cb0Mvh|jd6{K=CiCKefe z-1b2>m8yKr>19Hkw)clv7b{_pBFQ&Gp#(0o`0@<_Q(* z>n!B6C;GDOGBY%#NA){F6&MG3uiK@K|KnVLRf9f{*;9QsT47!9=deYf?Diek&zV2g zV`UfSG&FM&xL%=`d;B>pa?Ug@V66D^h9tE|qEx)OE+0MHlArNM{VN(8M)AkwZyI7EKp>Cg$|m>(JW$a!aJDn>AotdZG0#>K{G z&}!vmvi>c$t-A4IWwO@UNt10p>OZ>mrcchFf1Ey1P~Zm9X&W599~c;ya4MrKuz4bK z*qT_cgFPU2Px$1&}jC*?{3Cv>Goi1*u2#79nxf zdjkdt2a(iV^%17>mp6B+tE-=6TGRVP8ImsznKW6xTH0{A^ve9jR2H|?)Kn$v#!Z`i z>eb$I^gJ!jT$On2m*Pp9%W|*p`KygGwR1)%`=5k+*+r=-$^SNBnoXEUKUR~Gw<=Sf z>g)z~cM`vGdx`&8r@CuwPL3?M$;&v|>lD@dlVS7@?azIVoT%*X*GW(v_ov4N4cy z3Hoix5-)we#(qCEGzpKcj#=H0_R!AEz35l;t;Aof8fD-@q0UD(X(k@sz1v+4jD_C3 z@xF%3)8e(-e3XrwHy_o~@?W%Q5qV=@zMOsV02tlmum<2u{_^cqEn(L8hoW?7Lm!)2 zW@P0+4^=syn}=&gB$hL{$8~je@l!ZCYqD5z24$Sf(Efs-vNeAq8GqQZbLVQuz5vt2 z53Uo1t{tu*N};0vY~J)(cao;)YLDzmMefT-L*W`%S5@)9ySkz)%r3iR$uCXD&$Duo zBiI7Qe8s(s7^t~CROXu(yqJ-j%{}Sp=t$&nYJx7`jhXJp%*+zZbG{vz{O0BAG5Tqy zuJ?w%c+uRsbGv}L^P`a7vt9k2^WG*qsVHR2XO_aib8fbTgM&wDu^_{C>U?h$J%E35 z8wGu8)^yrkSxMI^XHKi4^;u|%k>>F2i`l2O3+LHApTtHD{rIu3X=J6VuMX|o@6B<$ z|GP*q;#zwS;Ca;3(_{9@ z_5V;QcCfSN7e30&bh6jVl*$P^>T3U{|nz)QFcaUq+;#W7GUT>ddU3FgWULVqxLSf9%*Xfn%W_ZQAp8 zWi+Ol{Myl3OOBG+iYv^={jPccHS~T{l2K)(Drre@HZp$=%27=@Jt`_HBD#FMuS@q@ zB{EIB%zl}_l-M7flY49`==#5UYeo`@1I_2O`*?=*z7U{9g}P*^?fm?=(!qG_ zda+2h3ItD{uU`wz1g0Q3Mu2FuaD;}?T%wCEKb<*TXUI(o9RegRLTK2ph_3T&#whYU zXd=V$uGy?<`}nMWVPO6~QyY5c(W_(2WCz?nfi_h%nPyrWi&iVx!uMrmwLa={F{GtI z7OwYoWL$uPdcKh=loe<9Wht|3TkftJmYW_e>+Z|$KfZgnl^FZj__%;@XRm%lm?Z~Y^fPJc%PuPl%!TJ+W%fIRGpfN;y=>;0WC}v+cE*dHo1)Vj?wJ%!ogM0 zD=jU>vsC&UhQvSub_oo9zc80`$wx5-tM`owi)XLA{oyYP1CN>CqZ^e~F&foPO`L;6 zc)+;4k=*{f6dkmk;72p2uji5{Q^h|D3a`7lG1@k5;Pla!YE6jfA-s(RL1kpP?R;Pk+m4^p(5HAM`!zWt`1UmqT}qi_X1maH=N zwbq(DYlHv&_rydI$`Rw3!U~1?W4-?}hzbY(A?kQ~vQW4#JPK#zncr9a>uqtA{z>l6 zEC`ZJXa%EAz1zipKPJzg$L#1$W1NG+81j*&=^wx5n2eBU13zva?ZoF^1)qD#-o1ON z5!iKN<|lSR$$b2yWWr}GFi6A8uNtoH2vf@rI@PYbYvu|J_dfhd-M;7_QT~2aL*h;`AY8e0=FvX0sI*Nxv-; zL(z#pH1%_lbjlI*JGdi$@x|#II^(Fqu7e=c;XnV1s-uGP7g2~^Z z62re_lyD=GSx{-qFneuffEiheoj3`!nvFTdHRoI0`?(q-wt-ABcL z`+jQb`C>M5U}0d*{+o7yr^ZhS4h~+l0PvK-x<4;q(VvaGsudx;a#2iFRN>cpkMPFr z*|Jq%{~VZq&UV@AQCVcRN0RTii|uuM>cf)ihdaGJ^Ssz z-VFceY`xX`tlurs<~#PTY~k$dpOsVi6a>Mlt*-t`Sb#1GI$TC}|5Hk5(5TOK3$iS4 z>-05$`LpcV7cm3(p~RC~HA{fh(tV=CrN7?h-)~|DZjucQ7mX?s8P6t6=$W7GGgXrk zGhpc|Eq6DRKohyK+SK#?boQyV*$*jPpD{a#R(L$A)W0Z4>@x}tcDFtJc;3Km_7l)K zzcDjBrF7Isk#YZ!|5&7=xp^YLv5`>`k>5ttP`>8vnEn3jIhlEkHq>fR8QtBkym&<9 z1to0ma{rrMn%Q~5JzE?%9}L#f_ocYcniOcz-+&-G-`GX0TR2Y}$laZjb0cd1H-Vek z@-n)0_M*Nbk<{Wjf7Y?>U+c*ETj~w%zJ z9rhU_vrBtXTY_d)bbqn~(n5 zlh+QUmYdv6QBNw=;Oj4|B$g4q5IJ zrmjDen-YFY%tK=F*oee9T)M4WF1=GY@G9%<75{3Td1M0Bh3G~IFK2Pn|D#&Y{sA?@ z)6$Mn5t|Vzm)3?3zd1)*{@cNM@?<~tsE$rAKINZZD?-2jTyxp%ZS;5iiqYnL>FDXX z&;P*JHydP60|Qs3cnshpKW=~jBMVMW+59o_@g+^SJu+Gz>44`odl~*B;w=%cT!It@ zqm$g-u3d07c2l?6@BJg}wJ^V3fC#eMO9P`w zu*c`p-0CHth&bM)`SM8h`Wx#WcUDH3zM~3sb#y3Eaf_rQ07hqitJ35$XiUdmNI&FT8q5-@vZ^oIy`g z_Y7aLGm{=t8SW()i-=s?yllATG+BC!B^hEH_1T4qbkZgbHgRIp4Z5HNv>%i&SOF?Yq$J;Vn)$RIynS7!5Om1akw z{M56=ys@t5NsAX;4UlzAhT;G%Kl;iQGxp9;C~Z$OC|zULlyz)qXsAKfm9&*hj{F29 zAO?0TGc$9eh{$n(1T69smaEWvFWCOmVVH69Hc@g~HTjGKfx7VWg=loMY5Br^TqC2- z)QFy*3j{aWbZ2#!!@>nuzmHGCieiF{kScEo*5FGPCj?#DU4%^vDjPKZw&rZ1HU%!4`U#&GcC8e&6nOt-pBmURm zY**@XJ~YyEbRwWNK)<&;K7KQ8%UcD})d;c zo0yv)M`^uX-FbFwAMr>Q&d(#njXshufkBwi5Xy&^pBaqy++_A58x?vS_6Rww)mBl-rgm4v}!!c{?6x^EE%hujL z4s9K9Zr~bg*xTE;U2Zo-XA6Y1Xp>oveLc!VA10+L^VKn#*>cA<}sRFxNy|eG!BClQ&HrYmX@}E_#z;%OHNMs$f~+# z18#d;+nD@(J!l!UOsXv?C}5FqC2Ip%Wo#Tnu8gjw$;sQ%-7`8qUIjPyw%Wh{8vM1R zy1I!lz7HNgj7?8J4iV<~rq$Qkc^FGtvvzHoaiySHR!meGruvWDDI&_prm<KS}P5R)_co z$HlPP+S({0CuwQMK$_~&Qx?l5*x#p5pOj=8utY?{kfH;_7uD6(+dk5l{3!In37D9e zC@LEme$hi-AUW~CG?P(ddq_hgysOIuP`!TRMUmwTqhn$yp)O-FcxjJ4O(7B9WoB^0 zp{c7YD_M=boSdA*MMam7hW2zkH=qdw_3j1O2#U{9^k2n`f(RU_pHsQO7tajOQzJ^P z)GxgjdcnKQ6oL+ipS=1sy@vXF-cYkjwvnNs7ziVLZz2FKEsr4Ye1%f1KUd$vnZ@Zr zuMSbGsYAH>#%6FjYN$@40gG~JT)y1Ael&?hXw#Y4^s7Z#5MF?v|MyIDs^rg9G4UA9X8I;>5 z60u{PoScX06-2&GOm3UBqQHu;_j+?b_p$6VwEZQcH;u(qPMO!=#yoB`4UL89Lae;4 zut2DnV}}oWuejP-(vk+DDH0EpvXhXMR2+^Ql^p@r=^4{LXp;Dp{pzMF3XL>{Ab7L z2`eir0LsxM(5$%?nAssF#Mb_}(T0nNEe--%iKlEO<*b@qP_UrE4tIa!Gb{zrA={R% zj+~nj6(tnQ8Xp$6-s*69eSOv|OlX5O^uFp}33!}ZWw#i8R8O6XgYv-69FV4$zPXuM zF2SiBDfZ6hk@Vc{v=6qCd&@iqM8}(g9MD$w083AM#QUD^hYuerMR`em*v|gKVE0zZ z(aEVI!>G(m$3lynFX)3Yk&y>rRb!KqG~rB;e#<&K3Gd$@1m3+*u3ymaYmUQa=w%ld zmnx$J=+!zJvsEYyVw&Ng1`n(F<#lT{dEQ&Ke0 zujZi9TBs)PNlhFE;8{xaYnjm!?yS~xjvPI@l6j>chlyahk%(G}k!^l6_bAISj#h5& zuGqa5Hj(~!wplmZH2SH1rqO6e`?CB>_7^K{-ErVC>^`2_TcX}(k4(b_L&YO7|F9*z z!7;S5r-NdfgiYQ);HWcVG!ncAS?B0<*da6>Irll9yW)(2Tq~pyp4{rnbZH{xtfZY-^adGh->0$p#)B5?9+ss9d zl+r&qmWZ)y;iy4?pqDiGZ_4Y!UbjvAErOm5*kOF6&DzNGeEW05`X!adhmRj;lL{5> zy^@k5EhHplGA*sBFpihxvZ}WqF=M4?x zplNYN;Url_1Y6pzS(a~6zqr!<3{9wH^cB&QrwqqMckGDB%gc*qVW^bIfc26-TVj|4 z74z5gwZ@Om#?QO@;{N5gDapxj-Z?rq+i`W5Tr3aL^rIP6y6e|3rsr|HsGV^M8gI6_ zB2yMxnkPZ0y}D`_8Qj^|=b0-0&bg0zrrylwiX;coA_d1N6iVAqBe_#FV7j;BGa&r&a={5t+u8n z5^fE-BE0OoKIla#A9jRD-O6WhkCdJHSgFTle&m41PvwtJ=U*vTh2kDq(>4Gk3+82ioH zPHPR%DFxVG@KJM&Q*TMYo6%X^-{@You-(E(c4+eAKPVA#Kmo1dA;Bj8mP0WAPX8eLyC7ees3%4;HPh1V!h%hWRT;QnM=vd1%@}?52P+^ zKP^4+vT))Dj0Pi9dx6P;HMTZ3`+}7Md^p9f)Y=%}uo=I)zSO+;<=_6^>kWRmBK;O2 zR<7L==)+!l*hqR=sgY)7XKxRKsC>KY8MO_i%$}J8JH~4~Xp(a{D{q~j;*Jz6s0;g8 zQStcDhLs#Lfa;w z#jmqRg3mB`j*|6vL+RX7!+OUv79wL1Oun@~nHxB^;%z{we@Hd1++o{QrPOf7K+kWn zEq$S!p<(>spviDdX7mour?d~p1^rGRU=1jHTIXfJ4buTBTyOQ4x%3JF)23+sxryf6 zLeKknewdeEqNFGG<--JggMD(vk(VWQ&o?Z*X+14HS2wctoefKRYU)15y9RH)m>-Q! zx|n@@x}LB8Gm>OQbQ?G}P>xqnvHz~x@#7DPnti^;?lltCNi!d>5qW*W%AX;Q}yjHZ(TWt@Fv6k3KlcJLeSZ zBgD)eHuq-M($L=c0@q+_9vECmr#IBabS?SfSC-lNRD!;YHhtBYuVmYYRQAXOi87q1 z;{FlE1)k17Ki@F#uQ^;?T6zqwEvzt~*!@zsP%BsdK4V}yvi2If0SP%HaQ<1A}&l~l~fQMLbDfaUMTw!z% z|E^WWZ%}im^Y4VXI1NOlZ7IF$I{PkI1ob*0lF(m%%Ubl7m_nzlfwbZBz!o3>cgBnc z?Xs5GX!!UZ`a4#yS#w)|8_L2n8?v`11_cEbxoxYwsH365>VMwU;HLCdNm@&il$V6r z4Y2|9;kcAuyFTSgmYr9SDIl=2W;fwg&L>TV8vYb0-dBinu8pE~}nRLmQxd{oX z)T4Oh!BK0Pl%h+%JT)K;!3qAu%KOX#RVnoiCHL*Ex0$JJFWH~bxo^A1mf-M5CGT$d z*R1j47!{jvI}@2It@zwexDMXAH|-`uEhV9GBMslb_k^YT<(a8@1_lK1;w(>ykJs4j z4D>l)xy@`ijbgo{B6XRd?R}}yZ9=LB_wUrzM$UaHb+T4!ZZuGRghQ=Qecv7sU5#h+ zLEf6TGDB6gcw(pGyug0}YQrGye=eLqU;uiX=xL@E?#5Kk~eOn z(|&5DHmVxiMMWRMnjiq%wo%|#rvo&Qr$>btR?cN@cS1%nu-Aw?(0{3coj=DLW4WZr zN%3&G?yfH0@nrb?VXbJLTMJk1FXkmAHw9o=QoIQt^Kb=VhNXr`Y<*#$lNG zqRl6*wXLW8yq~n^@anGd`%r|AAWO_e>8$PAFN30_40rlTH1t>-o;oKjB=(`D<#a@X zC|`|g;8|KzN#>1|y;U5WCC&1rmes9DVGEY>6~RpC9#gTF!c@t~t4&MNMQcT2sX4q| z3Xx;jZ;;_vmMZonp)PdGmMz}hYpl1NW7KJPejRakT_^oh!~-i`p@8%^+WCGf#@lmV zOYszGZ;gDF)~xjRp7kUuL<-*9pN(dtNZV~|kRRBRU_J0sX}Eu2P}cAyU*$XV3Z}sB zeU;`ap>jI-ILC+<*Sd8cLFyUM#kAc)VjQJI9-~VFwcfU0vgD7kTsk-^Zgy?3FtxQk zsJ!G$5HKY~rEJVf?5kFtZ(G7>_+GwJ&aWyVvWd$@Y zf4y!JPz<^O$;80Pfth5sRgz&BRvFqKZGBkmTn_t6SD#UPm-cG^p~Bdco0OHoEWJflbovOAO|`u3DX^@YKip#ZkHJTVmn# zpz)7Ar!wlVY3f}*26WA`%?0_!16+$sH#B@MluB_`J+)hA+ofl0Pwhe!%OyO5Ka25S z-^yctVE>@}YA3<36$8_A=(aJ2_14ytUwTAsbh9IpujWGk5KNEX*A*-3%b%X|=zh)3 zY*;Tl=qO=!i@EJ$yo#rkdBAW(rhIUy-mtaFPQ#6Xru*<$x-jdv>?QmgQxd<6p7Foh z;;Hty`2G9T2J>v=>eyo-cpN6!&LFqT&oGG#A0J(1k>2q6!%tdFWQ+mtfH?^e^|!2| z1W4POytV!5aewgaoO5M}F{QsgyPT9a7E7jtY@$}^gZ7j)))m*97cwbrR}rQY(oyk^ z@q)w2Vt^MO6P{9r+&pP2H&(x;fw%*5p#v}V__N24_Y`v4YqvQw_O(u&oA4WUmd%|U zZXjJT`9`ADf&=UdyN`Z$kQ&};F4zuOwTAF#{nyKx)$f@6?wjBEdA#D=#2O{Tpi-VK zelhKc6mdZ7`%Jy$#*8L0*`x-5V~oY+?o-}{d&BNcjkr#0Qf)fIZg}dP7i%22rgDSH zo53S@=~{^nAhEX_0Vy|StYw&U@8r70b4GGU+XKhmO6IM|%m=l@6rh|)&B6gT&uebu z5@z)gz-@O~5s9&XCQAxH8x}88B3~ zqhEa$*?e=l6SE1R;?$R^SQ({m|F!?!yjScy^iRr&PW|L!?E50K@VQt2`WBJAt8U}? zMe9`yHf`I(ih`Z}{a#smhYqzIQDx}=^3QVqtj_~fe#>eO!=NxR+D^W{K}9ej z$q)vt^mHIyP(38Rd1^T0oK`Ngu|r}l&<9@=MhEuhy#+eXOmjxQ?V8v4#(udack|xU z&dV8gp9IG!p6-1XA{`u3N&#rdLyQk$a$1Mta5Qy@%J?mML-IV{^Zk@Zs@GG&yz~E=8^ebHk zlwEM$AHo8|>6gEm2*3W|#*{l;Zz_#VBqB+swN!0N%K2jVfI%H?_bz>ZCzPiD?k^pe z(XWJB1gx>!E{#E*ptv)XI5L;$``6;8M3xpQFnAI@cMYG&MIyjlbN)ckPG}pIVT(@IQ7F8p z%Hnf#IxCd7R%1ZWMoJdWk!|M_-z;su0+~;abWTpkHg5z$wHT%pcFt>8P4C7zAuS{0 zL(@~u)0Ld|jVp#65Z;G4wsZ7m_veXJFY)t3vJz1EB_e6a1YAP$#^%c{p#!GB-M zK{Dz`R~`(mrbJ%Ruq}Fd+B4?EsaG72aq(o9P0ZBPNPhS8R!@z+d#}>ur=0<@XX>aH zh!NIT2cZuhXN!7W;6Ua_c}lgfhIig=C_m2eV%h0+Iv|{9wR*~+;Pys0A)yv;&5svQ zei)6!o8EYRZDS&rXc%NG`0{XFR(pV{{@X|Bd3+!@65tH<<6|C3$s>m*XU((fm5Rbn%vVRi`f zbf{Qwips%Lu+tdttufPjs52_Gi#`A6wY($Cg0oM1&EtI#IJr3!kh()kO{l4$TV+yvcH>2{-?W0 z&cL1~M|$w#{(Q(b3fSnKjn-v%{Y&Q-cy4Q4I&hO@t-=l|X=-2Ge{^6k70?zD2r_1j z5J;89=LzN!njRdUEWCf-7t1K%Ai~ypr}i50Fq~^8>|&D0GEK}pkYZS7mo4oUH9;%} zsD827I?om+Q5t;Xs*fL|>rxCAuW(as<)n=(_$fz#$DgX6D;NX>BS9};xA@_fN)HaP zvw7`XT}SUAL-k{_-_B$jOT(-LflQ~~8i z9D8ACIkZP`O~hro&`_H&WPo-zC9<`kIgL2b;%Xf8aCSZCZQp-(C*>#l>bYnk77OxX z6qbOvkjB+8St!!H@$6#jfduW$BR9BR97_6i!6P6!I=I{kS}6uOu}f4ofXnaY0BeVC z4sD2<+^9aM^S}#a z><9-koZKQ9_&ntZHlXd<*J58~svT_i^ZxMs1xhSrlaYi1T*^7(SlRi0&(ngRs2-@f z#;zOv<_^0l0#y0Ycj`8v%|6%Ja(d5VBv5~Txr2}<9{TqP6MB9(9AlI%+}tr%rJ;@j zez|f07{s{ILtS+ZgKZz0qsrZW(?W*=xFd;#l8Y)dGC~2R5ndZsFX?*visjBpUj_w} zH;df-ju!o>WjJ@kf~92z4kA)ScgAzx4Zb(xKHY-=LIoVl{JyMiky5{HdkLv@s29EY zdQIlAej2LPOMdH?aqi z>)4MLtJ>PuvDi}v$FBu~?D zY@T2Qb{Dc62MYW56G+GKf6$xjv7V8~O|b-zNfLcX>6OqkF4XZ->X8L5OgZUBpCC$= zWaPmkS76t%@rKpdMaIpUYcBR10H7FUnv{E-N-}q36b0N6Og0Hl%<9@msk!+PrwQhx zsx+37^`TU=EI7YEN-bP8P^UtTz+>nj_bJ`(cP`bo`*<$&m-%s4zj6tSPw|8m5U}ms z^@m&a5{dmHHzsowebl69(oQ5-o)O2aq6B0axkS%{P8TXpU4}}&*qi|p7ZS&C>`Y6# z7pm+d<03KD1CmLSva2>7f2}h54Q7mj2`?>f-xAlfaA32g$9qyp;;(t4U6RO6Q{rDO z96;gFk~lv>U2b$AKI0wg?mh?b)9&utq+0N<#DVZdA9iFoQdshId#Bv>2G}7Cdr$`h z!cjJEzQg-nBaPuxQ!Zp)A~IZeWkW!z=bjq4K`b5mx0Gd{*n9dcEFY4mH%oqs8kB`3 zi0VTc-k1hBc#lQnM=PzdKm>Tb|1zjRfl_3IV0PW?2ASL34 zv~|8F3ss!UP?(QYpDkKvvraI))dGk_(#?Kl#I>#x@YQ}Cmy~xB^C&xa)-DsB~Z2S#Je^AA`~E}=0}$e%oAfZ#gMs%cBzyFb9O873<251y)r1a53myCv^ycq zwXw(Ib~p|9MALmmbYc%}z$8XlOWC-^DjOmS7-heZ0fb0$6Dx&-vH80LrrqRCeBDY$ z;1K@vgW%v^T>*FIKAgIT{Fu(7N9A9NEF6}$R^mcA1We0`!SdaJ+6jhpe8{>IPlPz? zME`YV!s_q(_L`M&7}z$SXgRXf{Z`rLya`wpv%w%KO@S@jMI{J^wm3{C7z&vhLAs+O zNc1jjc4%FSZYa$U^oKebl$H?nLh@+j^}vEvpoHYdgCytu7>xPh<+a5_m>Nc?E@Lm2 zxwP1cyE7*M@uB_*?srxsxn5*!P;g*C3yOK?BV-DDW_a|QF@y6m9`S$iVED-I(=679&0Al*V-zOL%HA#AxZw6$$^tXq^UUWxS7 z&sJOPL>K;Gn=M`NH`}6XKN2i1D}xo&Eg{C84;kq8l<7MJtE8MZH|I=e#S>z1s_N_4 z3^|-R6XpWV)16!_VjXq1U>-@2Bd$`1AB*$ieXf}l%tVD;+W0*S+`GvZCi(KalAlmK z!Sn9m=SyE7Xm(-yz;1Z9X^2UX5h{)eBHdXk*?bgnHnA#ZRw|4PE#m#CFeAq30k#-^MY7JVsS`q2(OYp4HF4S#D$WR)x>lBIBkpF6g_Auslop?sP zIxqte-tZ(#EOUGPo4gz7Nk;3`!5=KT{(CmG8`7(>^*+m5Is=1t99&~o*=jSwy6(?_y?F-yNMt3 z0|g&9pPiZ60T5>KuiGa0r~F!`POl*{{YJGq_SIhKAXap48CuyAPuuG2!6w^UoqCJ> zNi_ZWi$wYMO}4&7-iRF|hn-BP6*sNfXJ(B9+>Dwt0^oGQeXh|m492Xx2KB{kn-3`~ z7k`|1hFdbc<(#dMOau)6Frs$>XF4&~CT3=LWewPlxOOLG0D)j9uN8PqDhc#Z3D$*L zlMm!Xle!}!5^!mYqs?Ffo{%U`95qc)@@84R=?ymU^1FL8nrrafzFqQ!py;N20zC!D zaxrcjiQ+y}0M`73j^|xia<(IXtU8HADY*kL7oW_%lt!Emf-AWGe8o**1mX=1(#vYG z)$ES_Fv_Uo&oKS`@*1hZ0c9l$ZBt2nMZyIzel(_U5Ye%mKU;b6oEMCCIJXU|?&oWm zz`F60J&rmf%0^+~4t5 ziEKO~H6t(yP%B0;nO4G`7!qd#_yTb_CEi+G&ur}Nm8r0Dd|u$I9ZWu=)eC8)_HghL zl1nW-i(Y_@#2LeUoDGyqQ=@b;V|Jw3RWHS$w1T*FU^sMOJXZ*zN8;G=rMdEu8`Yu$ z0spiFynML_0Tq%Meq|t#P>!g12ItmBl*ED*B5Qz_i{^JPO~W_E0O_G`h1|-?Y});u zogXsthPHs))s0fQS|XTNxhHrwmJ*V-TZtmWj6j;Il)otXv6@ygOcE`!HI0p%h3y#h zF%S4~f2iE$EdEX?eJVd;#8A-ydtx7PWbE?>IdSm!>*gccHoLNcJnDADH00kzVPRo! ze$a&8I?IvL0$$yUkGMe_-zqQbnG(Fj6Y;LRK(GC>?kE3Tv55EE6%X zs$CZ3jo#z3vYYseTy*tXgUMw`iwdzakU<26u+IV%q=0W-g7!#2*<*NESXfbsHy~;X z71$*YMH+9?mPqF38{9RBQHZmHo>C?R=P=dvifvsd0TG$NS$Zaf3aT@t1 z+a>dF%@e;agYiJ*5R#cS-h~9;K)P7z(xz7>f}Op+cV*K7cTmuyN=AU9rKNRg;04pn zK%>7|)ym;_qRAe&$#75I*at6{i0U{^Q&#Pu!)ld1voz5)Gm2JN*BBk7@1( W+HQe?!WT38l(JuKU*ewASN<0rEyT7)nY35$O^JX%LWRhM^>+gdwD*q`P5AfdOfd5Rg;^ zL|RbZ@!tE~f8c((U(VUjdiGvt_5Q7OVjt;il9Mo!;NjtsYiq#_@$m5FaBFKIG42_B zwXA@9;rkeBLh-66n7`rS-Lun%sT#ksI(T;{iFS}-h>6a}q!G&}Ab^(K$>H2Vp$T6X z#yryz7c9FgW@AF@10y07AQpp=0Ew|6ds~fd>z?!5=HEMh7mn^uiSJgAo;>sRmsDI` z!v>!i_AI>kK3i_|4h96`vpdiy1w*Q1Rd{PR^P2b!l!3Z}_ zLJ#MjT91#fsC%}xJ;{RSf=!*?`K~;*4%v3^Jl1F#q&IJ!Gcg|zx&3ny$&Hm+c6Hx= z`_xacvKHyG*4?%2+AzKnd@}xYP6xG)-puysel~YP#>4Ul7=jZ0GIZ6Ik7Kz7*NUgs=?! z_TK$s)!(DBM3XgOfW-$Z?mnX@>npn!?6cz%wVp(&d^0&N)=$P=bAdDRRsP5GWsjrk z?%&_ByKm=-V4lwG{Qan_;sKRR%%A~uLRC7RlGYm_beZD9Ccf{T;n`x zQL6uLB{%5sn@IQNSB)7A>d#iH!HpRMPWK&s*|QUvXSrGh{yqp_YyE0B??T=krgNPA z&THu2Ve_)P50hJC@@*I&gVn2oj`Jm#5HLaN8x=y7#`nnWjO5p<5(k|p6U*x>8~xE8 zn20+aY@wGJ8W2Cv^-)ilcB^&3Hr?*1QBmzKKLLd&gMkfqaH)E`N=QGK6YMh@BwGoN(c#al&bXA1hMZ zOHfzX@>aosqxIqBOq*|lN>At?<`PFfoS|zwHWMFoa{fJ?G4PLO4TMMZ(Ab27M@MK3 z6gEaaZLNjfo+&BLko5a>_#Yi!ie~K`PwR*V9E_oWW$#OEphtDXtmMGN&dwwtj+ti* zE_^zg-Im);s?6G2z66f_Q}%Gt{*5nhjwkXOr{n4fZ{7-IU4>*Mjo(PYmeZxOE~*1p z2}rL(E|0}lh3&Az;k_gaA0NUzpRs@Na$&^BDtXT*3Ae0FXNML?SJsaTC`kkA_dYi` z6@_FEYKfD~?qqu$#`ZP}#W<|@A}FXi22uM3_nxKEk(26;ZSCfT`3Y7v8X_DZv+4|% zvowMLork~@F+0YbcOBz0%dI#?r-Zvsr^!nqOQJOVM)^xyITPjjLaw$m+*f@UgqeMo zoJ#{FqRFO+G2y}Tjq=kEzl7a}us!={GN$-Ae*K4Jiv^bHCxkM)2x^S<9qD&Q7r>wO zC(|c}C*CJ^oHtD56o_!G3m{P{9Svo)V);F^OJ2x%W*A0XRhWp7hW=nT(<#^M4$I3V z-j-puuodC9t&~tX-6t``w%4O&TXA)DgF;G%h7Ez|92=a%aR6$bhVV&$Q-VB~ud z-wy4q|NZQ=>~T?N6skG$S=4$E=U%vW3{T`6?0~LfD9`u1bT@A37ijOT?2Q?+JgWKOw)%d{p%HxrQY2Moe$&RBZpJ7rQM9 zh=-*Oj&??u$HzVUDDCf>k! zKyU>}UYXdjOdD=zbnm^-?gV)Zuqq{-s4p=FS9R73rm{xHjv)F1rM0{ag@WAge-Q|F zS>rOCYWB)E7B5)^3FHAA)>w5hf(OF!2^BWL30zuWx)>*%kf%GjRK+YlYathH(hC+d z1yr`))XZY?KYIxkwI{@fng2fRx&7O5F??^tmVR@Y!&*42#LhmcmQ=fFb%PUkt(@KD zX$i9f5j{d&aywd&@Kb5VNyjM+9o=nglZ}QTfv?jwdO37yRP7zyw`Py8+jet1)@!&{ zqw;3xsVtUaUUasr;w6x#Qc0+qgJ=TD0aEe;F)Atrl0A2|lDI~ClGLPp2~rPnxp3ND zZJ4Z4n6Whx`B#iOm|YkMI2dibzY}*ASMChzmf@|A)M&v~4_Q;1O?wlX?Su(LTQW!A z*`@~cp2vPw%X#)?n-x6=a*W_f0zZHQm>*lW8cRzdC8CAW;xfCdr0~}zmIDp9gqy=m zH>n?+$j$_ZzxUHD_Okr1P$XlA0H`@_5ch(8F3B3eR2KAQFLZGOhGHbU%1GQ`?NKVKUfykHjV8sULfl`4F!Z zvST?V>GP{PklYfG>qFZU2@&p>ljy6_WM8W)DDeX}@^j@V%fbrZ?yRRF?BMKmsA(Dd zV5npTugWW9%HH&nK)gO4d*-Y3qZIDq`2kL|>+VHqHy=-(vv;l&?VnGg`Q6^~-z5y* zl-hBdI1^AD9=%R;QM>~(94}lnDS22^a}uc{51^)D_RXD~ofN1oQo|s$j8Rn(;W@gG z0vxao20C-8ZdtRs)sTx-@`_}WF-~ZvGjpx&oRJb2tyrbWMnBZ8-g_gIv9h*#@5Hdm zWxDgl9mD{@X*;PL^$2UMUR3lN&yjPEypq7#Bxy>GT395}IzgBn{aJX*Sau|^9A87Q zxwc!GibppDx;IyOg*qEw@L^RHQP%qduupykg(v*^X|W7*OE4)TkK)gcve8CCxk>L!KOQg4QU*F#TG|h2 zLc}o}28hN2Qbg#|H0> z0OZI3(-R8pdmPxVq7g?@{Mypr&?9r;$4NHY}fkOg&6a% zN;3W|O@X;h%C3jMzvg;8i5s)z700OB(a#n2Q&NgkIFOvH@|m2BwjZGwN-fB;U7Tw% z@>{)=OO-kIT+|8sy()e0^^k%n67a+5K2O`?D~;CwoPHdYDPI^+h~RL=a~uQiM($4f z=6o+H#?aEtBRAJxQ^@Q(E(O10pK=#7Vt4%R=@l}QrjtNU0Uvpc@rdrB>WiCcp|;z! zDpXGajGyU%BJVmwkT#))E%8B?KO&s$7Zpjai9Pjx2Q7tYA=#ey^@|w3pU|r)kDz1Q zv!nJ)dML4wE|(tY;jC@pt-T-kHr}XmE9hnC03bSw@L3uSwh~UM2?P`L4UiM z9^N~s`G?7G?d!f|c)6P`y7mVn=kd4{*sOV7*lQbb7&(xJ@|J#()PLDcHjK|f>pqB$ z;r!;Y6{A8c<&CRhg$K7I@3;n-G5k?xGdiE!@JG`Aqy4-z$b`DNn$q9P%|b?qD$9^m z<)s`rzEDDFVVjj_u5mK7&5zHN0X;#BMy;f#t=Hih=LD77mnaf=t+K>|M1u3jgj0?0 zjK$fR6Fbs~5|c}d@dh|g<+%1AB(bY8qMxM633D->)4Hc6RKi67Ci zs{Blxtko!~>)4!D$AgvdvxA@m)uJiRU{U8|?S(1D9^7}cn)}h0`k=*pqQd8o1=o~_ z!9wqnbOkB$?i_ku24$z7{Vb-$Gp!R*;`$L>rw6DeEaWYwb|k!8&>0o1?d z>)vus3LzUZ8Wu}@r4&BK*2siXONrji$jH0X;JZH`I{(-XDKxo+HLe?KhYMVOdl>6{ zXMQ1h6M7)7@F`|qN={N|4+U_B5sgRn0`|9`_nk&e9+PJM4EhLcn9n*JY7^d3wpYMj z!Odkk-omvDL}}dK%rZ-{zIbu$#YczMnr@-)DRUl6QieIO@&EQNY!YVxzWItuGU7YO zkcMPmo?fmWCvWz+ueQ_avumatw=OhxsWr5`v1R^o+4@r`vhR=P@&NB>D>hlgFc>lo zd^wnzkyx$-$Wr&zP%4;_vtYE=3sn6TKE*_B^frU=Ad#mPZ8Gmp=8f6<%07ImTBItu zNrai$QNTiP3mLvyf?YL9l%u=#4pTGhHyC40kq9E6Nmfu21f8(rFBsdeh{&nD4R`)-qj%>* zuFqF4*9j?d06S}_)S`0Izh82nDn{LNo;5z>O*AMBpGu$*?$}(XV5Q_5d?R0%Q8raPESI5tOt>GeyQhrJi{H=RMuif8>@7r= z;NNpzFN?x!BNml#ThcNdoRvW2A2aD?NsYL`ry#K`!@)q6fjN3pw$*N(%$aynKX_ma z5jUFGk+m2ZU?ReogbPU&ZFRyM8a9GdZ+_M&LAHDC^+g*J*1>)&Iy$G%E6Yk8yX4}C zN-9p?PF4OwsR$waH?U)DbS{B$LxW@V-4M75T&nXY+%)MO)g4Bf=W);~vOH>XSE`e_ z0yyR_($CmcCZ|LGP(>xF6-k*t_Eyke?*ZgY3G%cj)x-smSKxsjy%yeMn3w>)^*5M8*4t=Qth!Nw_&ug?+UbpU{20mG8z#Ari ze23{Ysks(viQ=!~0H#t2omYxV0!g$NAeuC|bm)xOaKaldWd=b;6(ySFM_9dBuoex% z6yU@#(9#z-NSM$=dglQQmNp*vU`TpO;2BC9^WJN)k^TffeTgM`{ziM5s5sEnD7e?T)l94LbSYR99YtCk*LsQE|+Q+wYS2t>q#aYyiE zRs>Vk?S8L1_F*&)J4Qcaej8L zZjRYvjvk$V0Q-y`f^^wYWEanX5Hf9S8h?QAJb#%51{2Uha8o=yg?9qDO}?DC_2G5V zXAo|DijQMN(mPWQwK3yV5hp}i)0#Y@ezn=bj?A7nD619)2&nWLX7B~5OS5RJ0brhi z94HliDjS35Gl^LGpnM88BFpl8OY)%RQn~?-R_DmhYC(ViEPO}Xl?JxAuNojqGrDX< zD^> zOPe74#EcDD^w2LQw=RJA2_R#)qqJ>7nxHG_LPW~6G=P$Ow+Kr2yh6~3y8}1j>;7h> zc+iRKea{372X~3^E9M{Ny8PN2>82FXL&FbYGaSvN(~*f3zArV45R^Rpp>lx%lO9D@ z>VT&;{#N@F6He3yFtJMvj+*|+gwMHqJUhnJ?L zk&KazfkNjk7D5W|wO_9r5*YRa5sg&&ERM^~ZhvYyH4}wSM_3eI)gFPEM}Kbghu~ox zuxRmakZefEH~IKU#+0ZH<2(wc=2FF@7mNmgr+WTsK2!ww1oKS^)(MQ#qb)mJEPxqW z%!S4zBBzB_F|hS;wS!?ISIRj>)3Jnop^iJK3 z0*gP-;}7t7ci;DyUXy10-N*3o#OC>G%LidXbEVFX`evZri-YBbDWhUm7~myy`ca3daA83r~{WqS&L}pp46Yl(Qpt6{7N0$iV5+WEE*T;3!$+*PYBE$cGMn~M@ ZYaZ}7i=EfR;C{W}X{+nQs-ZT>{{SpyO9ub| literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/backlog_time.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/backlog_time.png new file mode 100644 index 0000000000000000000000000000000000000000..5b9080a70549ab2276eee186e7f9f872106f599a GIT binary patch literal 6127 zcmY*dWmr_v+65e7=%G7?7-6Iai9w`>ZlqIEx{=PI1wj}Z0cjYN7(zu5>5`OEQjl&0 zq~V);e|*n#|Lk-2Iq$RIwcow>Icu#~y4osa#7x9EI5=c#s)~9zIJkV+dkY`|wiUru zP{dwvef3ljIJKiJ+c-G%Xf;K7gTUwe1^A~1)71S!5OHW!4MS06P6Dg7jcsuxFB>%p zEv6oyq@Mbj8-Gv7KOipw0WTB+g%H#E-`#CK-Wac3?AUR)#?qz}krQ0sm9!p#KR;{Yp9J5-@F-|eCw14wbX z_(BLsaTPdbu&;ZAgWSBZq1N0l#;`50hY6PpZ6frBY!oC6cwM%~C=4J)ns8Zj)9$Wg zIvKG!(lVqU#L3}-azvd`QrCNpiz5=tds$ZDmw?xwLls0(Q1+}D6j&Dq@Mzr^57@|c z{Ib#=-^v=WfU&*p!Ps!2;kAj$=nufq<6im$Oh22ifLW6lv&ZLRbB~twh0v1`;~5#~ z;?J5-OFN@acKHJ?*Rxxt&lehhEcRB1VX+CslT}aw7eWrddUQnMI&0N^oOAIg@SE{= zt!d7hw~i*WsMId7NQY=aC&EnU+Saqodlr~Q>=SmEzJ{32X{vK6>#3RYF*{BGL$O02 z%(>S2J$4xO8k4;bwU=_OZK;sDIqF_y3)x$tv+(LHxh8Hou6!5n^X=}+RbHlo*|Tke zj7fCBGVE;HhTu9GX$d$vD68+Lm(TkE?y>ajME*0~?Q#;!<`YhYhKezSzpmnk0tq0SbVM(0?O8xZjuipxwj<%ig;%M`r^-Qi28PWH( zmaU0I;^8-ev(<+0Q#YdPl#&o{p*>?Su`84`c=bD);>+2rmF(v_?qzH4%(dx83?cYv zF<|*sdgyj}wtxCVoxR)ZegQnJocd}w1{gqU_T<{%h!PxZ=-NT3& z`>{E8l#`j(A=>#{MrO|CEfvYRSJ#K%Gy;|EE+!FPLT6@NXg$iNYwBnm#TZ#)$;-9$ z_4Rj-g+UAP?BTp+k-72t{&UU;16*%3-_FY3-~8!&&G;wbCXRHx@8i6O)Te19^A`oO zF#$_Ko8@5<3*P-LcemHy9c9$^KP`uQoNi4Vq(~irzU2(1>&WogP^qwoHNB=JOKnfX z9u%cevxUv&#>4jOFVW;rR1y3oYyIpIFHc6L>df6Ry;K-~WxZD>5N3NxP^3lRitiw~ z)E07f?rSdK22&%jZCj)8b9BTBV&fV^R=9;&|=+yl-vpO z@U`Nh_FF?pAs zJW1=_)vo>>{0hET-#vuut~4)2x-Hwi2K{vE#Y~fpDyah^?2%d z!U6 zI`LCnC-e@!-QQ(=i{J7r>7YzOgE)(#xL8W$9{qW)sSb$hTm}$P3ai6A`6}~KUnnZ2 zduVxD_ZpI-b*Hl9?qAD*#Z@7XM|W1@+36f_vdlgBxc#oSD>WL!u1+b*USGH0--VoC z?bfxHraVzkILLJ@M9+O$+xe*q&H8Q|T}^}6*0i^z#5$!k8D)JWO&OFjSTXzU^oIs| zz)Q1sSfM?i%x$)|fRvdEVHfo~_-rTB4|*Ei&C;dYI5N9>!wlnF4m)4WfRuad6S4&s z#lY-|=a|gc|ERPE?>EoJWE`iXk1l1d_qfbBPjJ)WiD?Eb#=U(8FhGV}ASS5uaV)$n z{PwEFZ`N|QIN5qmp2OI}JGO1WiP=f^B2}1Zq9<^0LUBAwwKPiqt}x_)(uxLRB)<8i z3^~abG3)v5cH`j7t{~%REd6ud3T7ETmW2e=)LYN??L0A8NCg{&{TAWB18}qeLGsw{ z<4%8Cu6Y8IiPPyv(psuF68h-Z&ENk-07TumX>qbF^PU36Q^pg7n}M=&T%0po30x_v zxpKqAoQUhr?Cx){%<<}<1t-lEN;vG4;cT@^yS4#j$w0DrVH_EKM|fwpBF0XHDyrjr zZy3E1yXT=)+Otir;%*a}T0YyR#5!h|3kjJWV)$R6$-ElpeB*Ehlmm~ zX}Qp!YhNe7=Lg>0o-eO5D2EI^WO>?$dHw70{mn_Gr8Wnz$WHJ%{3&-}jx)uOTrJj)Oee8M4VX2jdbkpSp-2W%z~n zidM*A+@9_AHkrN(d0hrYRyR8XdsvXzGi|InS;d9$X8w|9uwc~FX9qL@;Www|WCzEH8RVaA(Q;K(Wa&|w=ypES9>MiYF0 zd(P8OfmT!<%F=*uk2(id&-F=C=mw=V%frShlg7?5~+AKxfyQ-1O9`T zG!}k)h5b=YHhU~S*;g3pQoVC;;`5>Rt{@iRo2P@dBn@D+h^d;ODbhVks^(FA)ls=a zfs(_-D_Px&)2A)oxsAzOvUv871U6K-{yU4i-T^;+A%0T^*fF`}8VC(vr4Qjo$})nl z|Ls2-oMF=EpeM!$c_0YXL`)qv42LAxWdGQIbUjAPi@dM>;?lm+5I!a(gO1cfA#v6K zYP4LP+=l8%B`V*v{#?a>H+kZoOBPhqpIoM)yFXnq9h{QM9FH1A91`P#Y4n1nmZKD& zIN9bsWVWKed|l!)_Dt~A@SD_1;tB|bkt&*NSQpiN>9Oes=L$Abw>au09l_Vi>z4hZA2na#j67&D{`m1*|PMtUw z1Ri(=IK5d}E>k{x%r)bRRMIZj7qG{&SRW42Y!ES`)}0;FoLv4Z3G z!XpzE$lW3$0{SsuTIG3LlR)# z)bt|T(ukN&HwRT@K&?=;$4#qeoi=Bc!*-+ohuPG-m$tlqB^9h-Z$ab4A|2dx({=KL zjP;2@_46rgnFTB}Ox?v@D1WiW6ixN7rPJ^vjt>|V2CfnvU)#9I9(6oeZHx-7qC#ZS z%tW3xz%5Kk{C}6+#9TgqI4tGQPxRhe+Bx2`%_oL+ZFj!CP76T%qfDKz{^z%d`>CuD zNQ_zBT)@KLE;(!N#JYZL#dEW=pH@4(eFJu2$V;L_o{mT-R4K%hBjD|&7LQI5wlwbz z;w6twpS318U;do`fQpfm=QeafD*6Ab0=}O%glQecUaVJ60LMu|VtzZAj#aKVPdUkb zms<+ZU5eEvYf>dHz&!N7^Z=eH$V(eakZ+qH_#>&*lbyXi^Z=~~Fd7Rvb{|2P|Z$QVo>i>)Y_2EMqSz`vA=iKAuo&iCQ{Vh6r zw_g0UG%yqtU%FM|9}omVlqlmF2YFv8ctP=%E>eEmcJadvCmLRSZ$s8&-qri2{PmqK zBC{m39haE<&AoUUcnBwDXWwVLL1Z>gFK2bR`?+jR{Idn6_G4EI|uB)23lM&A&^;>vbxLcu{jugg}C41O^rDo2KzmBk## z+C8O7<46baIgfvU&rDY8y!&vQDw(xtUc0WO-e6!Sm`}(m?v|5b`W9xVJQ@>U7R)*L zqS%((kXO&&lArQ7eelVML^dhe?bTWOsmw^8m`1YU&V9_YF9sM~3DIPF{&Ym_FaS}YE?u6wZynZxB+DfUQxLW{coE7XBBWdr&fmlu z&N6V3*r$%_4JG{j_*w!5bx_(XGyw~T{Br|WRK;7R-=uj_5(bHb?f4kkMJ0>c+u*;# ze2_1{`0dh}PSG|x{O&-ankY}wpC0EGCj1*XTyAvKQF-D{>%y1FI@5^Ia%+HRz^LRV5WV^iT;OXU7}SSToED6Zs=!N%K!P@^m1IxZnuJl05cL`FPGo1Nra(3y zIX}fdl~dM>EwZ2Gro;WAuahGzKQtGx_Y=1JRMb+8i5|B#N3-4O%yQ&V2ScQ)9xJNs zWNWa2tOqqEVEq>oq?{mp9sF2`(|+YVJ6CYNct#i)(L5 zXrpfOFT(ver+avN-x@T)&YF}~5nWvTtsp~n#luwz6qSU+II@s|u;f`3ydM3F8n^4V2D6&c2U<9AH zL1<&_k4Vxg+2kMD_Uk>;NuOsE(YwT268ZqmLHN~x1bs3|x1!Q`;Smqm$v&qnuPg6T zxNexr-b^$mL_|zOMjw^;G{^^fCd%~a*frI?7*dpGF!AQwVh#XcaD8<9Apx>MdtlC*wzAANbP#aj z0>UblkrOoU(OR>Rc%rCxJ&&YpfFP3i4l)2@r-wk@t0;!* zxXB4<{L8I@Qu$D)_}AU^taTf|`?(#mDZmDuZw$Ym@iVINtY|gD=#nE*1{sp864m5x zKpY=atV^eFMLqezc_q3_C+4WZK`Ol>k8sw#?YIwBw|qV;Oqb|Tr1o31U%p1$Z=a?i zl7BlMM#%(o{!!M1_a(PTw?EaS9I=5nxN_L#8Rv9=T0Tc>MsYEc`&gxK>(IbuIaP`~ zu1K;(zPd{C)l9s2qFuqF=G>2wcm&;yYBeZhFRzEQ$bm(OS;$g+Vt1Ti%CT(LXX%Xz zzbJ($==m0JMEdhFNx6jNalCh-US6fICfYrBKsO10n0;dM;TiMqjfG$jVf$Dch!ZUP` zBhegM)sIw&9B7YK6-)oT`4Jr3BHW4A5@QDvb@5FRQYSWEg@-zZx3|c-z`x!;=z#AM zCF$-%U$W%RP7+QWI$1lJfY^6IW=tch)l-@E*JMa2XJj3ztd0RtuGWSUnAk_Nt^wVA znNg)6S~+}|TsG9gIgVF!hOmqA>M$Z#SN@G22=(6qG=$wI5W4hxZ?uIC}8*KyYWYl-<#A zYy2gBY9N0X|0)zK+AJBFv%#rIxeZe(9}({b!f!MwVdQ6V355!zWJJL2QWmTun)yJ< zwHxHb4Y=X;+;0fG{znx-11)ph#`>G4dN*V)Y5)_zFc?lvt-|eY&}Y5l<{KGR=UDzeKAORP_Q&FI-va#a z10n(b#$l-zfiHZ2BXt;F^(fm09v%cwQ$^`vi0w{3S+j}BpMItn+{C5DBnhiRNFr9( z8wq5HWWqbeWZd+zxr)ePfp|#=$#|Y}omhD6GEpwV$taoT#citF$vTM-wY2VhK03b| z3%)wv4W~NE3mK)4c}?}-Z&c8vv!RmJS`E$ES zmM@hSbW~D5P?hhkCRpikQw=oJ=j8ZYs0T#vy}q)JO0DJX@3!qFh90dZ>_x1}Sh$~8 zyeS~RJU^lKUmYgfnQI)fY7rFe0OB-=E5|^jV#ga%CWcdL;M3{6^!L z+=+GbVl2gWuFP&TPhP|!&O&*5zVqU6sP{t*Vf_~`U3U#$vk5ipJwc*)MJ=MMgN&&7 z?7LQTCmX1t_0J`}Pforvv5I>zU!4vpBz_H=zw5rJiDgm?o)kvvfo?N`U?uqgIPYmr_7RSp6s|g{5kx2 z{I~pKP2n%uUdM^;>DhD_N9RGBH}8|({~{Rjw(6;K9-=K|4_hx4Qo?(dn| z`j;xJkhT2a!QV}0HMT|JC%eaTyaPvtA+SoUNC9daeCDn)u{-*9Bjm-{9>pV|^)1R=En(Y0M(r7poFon!kYXoJdf3 z0=FKs#P?l-2dkRCv3fNxk(@{DFLvW|M*!`;l@|5eXM0h9&(+@hcEjUKHT;U$p93C~ zQw#$!a^)0&d3yZAeyuzF2*%Q=&$zTtu(;$o{7Cw#UPoDH-yN%BEm}jE`8p}(M;%dV zE)4>QT9%@8#KV=Sqk*(mb@8 z+*f3IF5>FXTG}wq^g}=DY{KW~Gc(rA%nCBY@#g}X;{!~#AzXPr zJ!JTPPzD;@vq6nbxoe-26HL6(r*<4jrySsnYH!&)?XrSnt9UHI7qIEc?1-I9 z{7|`|=x&^?vFj4O>>l4*XsK_xReL-tf9`R9_4>K37Hj=EkcE%Nf;1VEFBi^Fw>3yBlSnq^G??0=F=Y--wZ*Zm|pS5aK1ItR<~w{UO4d;1>h@;ns@#r{p>GY&tS zSJDMfjDYg|AecwyE#{mY$J4M!utCWpiWQXt4EHx|Zpc=YsirU@Rh}*sTQq(_T2!0Y z_SESJ3Uap$a=$shUV=1+?^EVPh4>b-N;f4QkC!f-({b{M{Ol&>)WIG! zgCw|X+n?%5Cs93k z{-_lE zQ00R`Qbbee0yK=>OLf`zvvgC44EE3FWQFmnBdylQcLKX?QB#s%ml^KeB zNzWqTmFFp!c=PB3A+^xVfwILbgMw25)6LMee3vNW)TpBo373H7IE`xiBn$R`D`9C8 zo^KaW^el5=cep6Bc_tlY?RVw(0Ns>>L-4P6Cf-)VbKN|__ z<|R>=TXxEEB$R_YldyRKVk|9^1s%8<`xD3L)*N12x01a%YCwy7Es+%P?0Q{&wMYZ; zKf>KAY}jdqa0q=a(|u`ax;FZLJ2pZf>4O^#lBLcO%8OBD9=No}4+|w4Ag}elXjv*FnG8ouwB*&B06Mzl~k$(``>6kLTOJiJ$$&WmWpnV)dZQY9EAd|^5E>&*D zkpQdX{)8zz@S0tx7XJIo*412hp{=Jccy_yiWqfuN*ZVeXyLKRT66^;0dv(&4R6i;$ zc=6+wLaiic+dKOfhbhAz#oNTCWS`})PFEZXpB&d75?j|yZXa-pj85Bzj|jOuFK zjxXZx`01uJshU>zmwM@5GeziCrp;Kj@8_=>`E~otawUGXn9<2~qC-%Q_vPa(qMD)i z@4}gI&t{yaw*EoBJ8^jNuT1NZ!Q(CHe>YT2vHypKAw9y}ynikUR_k*0f!k z?)_iy=Rj{iWc3{5bt5RI9RR6Af)`=U|LaW)^oD*mng5?{GXUL&CfHFk{ny)?+aQa# z$#eew2V^bV#)m&JS}ucp5XPNQJvv}x{IxXaI~M@#8jumy!Tjc%ys0k+ko8ks9^7HQ zloB%~Gs91HWu$(-2TF@(M(LPZ;fzn#<%HxtO(Szq#PO`lGxkNh|u+K9c^E~ zU>ua^Y4Rx_&$e`DaN`9u5TdiaJK?DVvO@zaCh<{pfoV zXQb;n+Sw28?o1k^t0c9|m4h)p&WO5L`2^GmzkzTvp7&Il?oDGS2vb(B^H6qz-0=*L3;Pmw>RdZp zsP?F#xJSzD7by%T>7UYK&ag+)B_&Ijm%H%RM6lj;y4uElkZspL{kFqE@bye_G#?Q^ zFq4T?>~43Lpl)AV8!yAZwTG51af8QzfV{llzBZggrsAUE0^biDFjO9$-AWy>L->Fn+8pv0{i^3+jSWLu z;ln|5YZUD)<@yOS;gKovJz_%4O(T6vvQ>_SrHmEa;6iJGCPR|&wzBzRX&!xj%`LHC z90fqbnEa+m0GVg4i83r>1sddw8We>(!hVyR**yu&*Ura_|Ze#Lr>$Y~5Pv;jaM zA$+f6TIi1TQ?IE5D@B^4IQH62#FR5P_5yL)M=ZJ$Jl!7Z|MCcBeJ6%3=j0mCg_yGf zFh`Rqpb9L&LNv}V1*gI?TQ;~lkk0faa7ruaXp|e4s3B{(Qa2{v%;V}nr5})Wgd{gW zKoU2NnF|2BJI8AM%7)XT5bS~^V}9Ik$mi}j(s}BuSYeiukXtBPscuE>rSej3KT5Tk+Z+4{88Ws#YHRSr+RTrpgh zL_1BSbZv0fpV)BlYf`2_0B`TSyKP*?Td7$%E&u$|cFwU{Eg}A{ERofF50Te5oRLn? z$`=ngWr94`XKG8>>B(CBNtnpYutmZA_QbOTnxa=S<&%o*{9Fd{pi*5w>{p*O>%b|K zqTs!6kGt+@-nsv1SRh#L%LpO0ti4E7v5zjisvGFtj%3TX4Ig&w=l;me0@KhWCSDoYP78gX+PF1tmFp_>X=hMFfw zk`OHpEf!N**!J^K7G3cJdByk;=3-5AFw|MV_>JD(X%htMuTjh7L7qKJ@u$qkIa=ag z3(N?U&c9dZL1YJxLe^)XqkaD5zO?8AG>F9leeARUDt zPQs-d0o42;1qDk`rX*Q*(O(RgcqO)3o8CQkZK!YZYn)6VXZ6 zIoE)JkxDXjh(eti9khsn2WF0i5FyB}8NBPj614SYBo{lFd`C5cY5@>Q>nmM?-WaXxZppQU{U|*nZM`RZ^NOOE2Uy! zujLM>876A8)9Uf*yZGU{~TW=4SX zP(y0-bG$p=0|}y#8#$j|SKFHedOi1JE_ypdzj-;C$}i}gaw6f0=7B0Llf@1Zjfio` zq)6y(QyeN`3N+e+l)}|g)K;AEvFyDB%z>Wpz*<*`Z9#O3|LOkHG0SgF6sQMIs8mG5 zFxN8K(%3HDJxNxFqQme zjDZ|gOFc3U7L%Q!Y6&7B!!WzR^ib?8$_KJ5!`~NeYi#9=`Oyw}n>H*~t0XJx{Cq3c zhP8pM{(JB2F5iwYfI#pc-tkwW>9)x|ZTk^p`>F0I-s01cA`OR*a5selQc)<9#YiB1 z(PHttVX(_v1H3LW`towLN*Cth#a=U3&LROzuknEOW!W#h4CL4t?1gc{4C$%~RH#5F zQbMMekLk%!zvRQDzU<1v4Rhax=i)fu+P1V)iCpg8lF@t_t2%sqZ=pMAy>fQtWf4~D zyn1DFr9ehnjX1paHVTcI#euQtFpJnONcK%Ko?Tm=_d}`;*<{bb6*DLiQ--k;eXI?H z|0B9Dnwb8Cu{60~n~lF9dXe#aJQ*D=lECY$QfhMw#EFsqqxVFaZrSC8JoI)YDj-3X zDTT@7@a`^D?-t{bsXH&k=wT_Z>R$KRfS}^Mxea#R{1cZh+|Ws0G=o`T zPB(bB7O#kAeE0U7Jotfae>yV@V>ty5qWe<3aab*$n5HhB+@j-xAEGQvePOGSeK>5; z>8PiCfnq#L+d`1ReU%zejuB>bO{XvHLd3zVBslTrr10bI9EdQzMErM>OZ66PM*%Wi z^1FZf+R9u)d$&F}(c4%>HQs?ZgaT$U{D#nMuLbH8;_H)vo)uORimZFlp9yRigmL6J zoNSy*Giej}Kq{DJ>6_;2BLoytEf-D`+TH~o44k%c*(qVE7yhKqSnJyVx=q(28UpuO zWM#Z9^l+0lxAAthn-J22akuq^sgjiqWYzm$5&!qT6);bi9f-;mBey7yp><5%^Ck0Y7=ahQ?MFt!?>gmbGVjo_DvUY! zR&P>43jXA#^CfAghQ=f${EkjAVyM%9P+N;Kj4(H7a^dwHIWqj3 zP0a~7Xob>RLLPu<2@)W3M|3*HtTHK-KqfTF(rHsS(NQoE6$~C*SV(eMF)t=?tCTRJX2FhI;#^hEBT| z9e_TBg~G+Kw?j#3`5*zf-A{_b9=tdiBi!nJgu_?H$}jA?3p+hkRVUdX`eri8)ywesiMlI z3WwcAcwYx=eiusez_VS@nzVbVAc>dDgGSfwqIpO#;9jnDj2nacdT z+dYY+=7GF}m}e~A{%}Kj(g7RQPEKwGS47fZ2$#NLFdDA380o_slO##Cdv{rSr9VyZ zX=U_KO5b40&rtE(yRI|5lAkyiG6FXzMN-gzeSAOLF|smpfS#%{_)go4@ZFo7Y)Sg9 zG0xcd1Vyt8r@y(7AHrUo&G(W=_zk8R!nHg)OqnGto}i`)r{d~-&0gwxg(a#=K()4S|Ug3NXo9KQdOsUJF{3xpj(&A-Oh=|?l zmUE)5_kNptE!Bc)RvbPn)%P>pOjABvHW6|xZ!tU2P}0M;VwMqKm-YIlM!Jyimi<(g zlp+Rr>fl^FSwi$=Ozo80YIM4{1_V)VSE6s^{vcXSR#~0aekY>X0ujyIT<%$YPT|X@ z7ul`FWjrWj^bscwh43!5W+)Nn1!h;n@55BQVDbz(BNLW%Bl9BW%TC|qE}EAyDN#>0 z-5tq<#@FZ}h3bffp<@K%Z=v=GSm4i!I+PJuX>&ykflqi{$WmfYf+hhm8HH1(nvGe4 ze0FX)Lk757b(P7>o&M(c#XOk(+5E2fWO_vq*Fq4Z%tuWAq0)Z5tlxV+?6r^JO(*4Xnr0AX`>T?pY zx)EWHlgCrl)O2R@$3J(WDir)k(%9Z&9x{?(YF_nXGN^i|qi>I>@=Z0zV6LGIX;D$K zj3`DcKdn)-=Uy0$05g!eIuBYYtN?FVEZy;4VOP~8_n!#iHEMCdaMWxHWuM#sV zEmrZ7+@F8kwtm+9xwc=H>|;}lws=(SFhq@BczHgRDL%!Wjet*W z6WFU zL=LrrB-w11!Xgq4j}gLQykH9Y9|?(RQy`_{c#J@i!-}}-yNm(~grJ~Fk8;s+H8A6k z!rz)Hxai2|kOO~q)o_;f;rCaS#X5)cU@XbrVVL?qTh#_pB>ppwiOp-b+w4aT0}9>r zXMcS&Fq?8B@^i!C3n@T^NP?bzqFR6U(#@$QhHK*|Z>A49flTv5J{lrgY`!KVU#=Kd zgO=bY`)lsp@B|Tb%ZsjrgGZaHA`p~6T95=hsea7@0u;6`z%#v;styzU@A_jv+#@dj z>y~Tc^#vHkXQor~x)abTf$oZG0EHzQtIhO}*o#*LlvlK1*=HlW|1n%Vz*sEZuT%cJ z1nvgo7$#gRYW{CJjk_gvalG%pO9LHXu;j-LlV-rs$s953$Ii16EJ3FZGGgpdD@Kz{ zYjKyHIofObYnv#>%r=zo*T%$vcu^$4z1D?+MP9b~2i+P&6bKkjkg5QRrc3lUJAhWx z;5T-qVI!`D^U}M3O--O~`7F z%l3k5d@mvcP%yI)i)GuNisJGqaJ80sbtv8L65a**t)rot(B=nYGJCQ- zbWD0SYu2Mqu`g-ieDWZi?8v%m9ZIz47Z z>l+Y+Mt44R09*4$cYW!NQjE_Mte)dCQhs3Y=KV)Ht`lWx`3>pz`3D~GWDUR{Bb&X& z${0y%+5Hcgh(Cv)I>Lj`x3}xZr~nDM{yirk7ViVWov+92ZvuP?W`!3mI8$UeaQ#A& z5CIC_Bd7bGa}7Q~bgZmzOmoGC?*pWvSnQwmk__0SR@kr4>U6dDKf(vIZWjSmmzp?0 z=ZW~m?fdw{G#X+6qlCz)qmye!)pSK>uicTch$3Fv=_9y)bJ?p8pf@a*DaP_e9MxO_ z60gN9RQ|H`o=s~2S&4dvxH~GIwNBi9>aFvL;{&Kos~E85ll}#y!K<&?+%51^hKdXz z>hwYcfWGN)t#gl9#_|?J`vEieF97N729H`Q=BjJIhob13M~PZNtN4(<_n564mZ`RC z8Wj?9+FfW-Gd=cT}>=QBD3{Eqgo(IHy>caUm1WQFKNv980TMlcRtqSf=*T~m0|PA zifn*WA;>)+x#d_LRcr5#-=^!66;ExVrfq@~@Cixcm8s>#mf)#f3Q!eVJX4OtR#mtYBV#vs7-hV85Yp zAy6F!n2CzpUVv9D+e2#kyqZ(c=~V2}lOOLu3v_zcupMsG)%hA%i}ki0VL4O30ax-4 z5@6_o7}}Lp0Agw4qmkTM7xN4!Z@-#b=>0zcK{t|5ZlUC{F*y-xMA;Z^Xy)1-U8tt6 z;`cbZRQq2ouK+_#z8mkZjTL&oCNOt2Hz-|OIpYyGd!v`{r;8GUvHk^4U0Cq*pi8M~ z^$@tsZ6!Db85Fe+h{C;IZ9Asbcj1qIZ%*A8<@p>hqyb`n?ws{QcH-$+dCO6=U4xR_ zu|RmVJs@>iq)5Q=8B2i5?mIc@0#tT#a{Nj`TCL+5#|+U+?p0NXU{%!#r-xp%b$y_; zk(Y8QB6G}6ge?wfd~|NRAHz}=C}K;W)J()hBRQ)r8wYcXRHDc6l82C84#=}$-g<*B z^>OV6OvkGaHLvai`OJFvx5@QU9uzSVV%Cr&`e%-oKyWj_g9=v0l&q$IH7br}Xb#%3 z6qKzgPfRam)6@G9Hj)4cF)?rF{z-iF_ypcvmqDjH-cjy2J}lvEN4!yDNxJl%its7B zjch9~vRj1_buAXxV)VPQEr7p#BKPF%mvGRw`MJaw&S}(4kC)K)EK6gMZ-tScQVU`A zXKRE8l&So!?!P6Ij0aC-DUq)>LAnjlzds}|b$np&kp|px7e8@^S45tCWmbt;kaikc zswGNAq>*o5y-)<`bOng82mkJef+c}d8&&;Y0CStoFjw!Q|Bl*8LHqdVmuDXY8szQJ zCK9B_BJS~m597o3DffZ<@MiCL9Irngpj2_a3;MSd^C9q%f~1@g7*ED0^PWu!V+wy4 zk|$k8a-G9H(U5Cp^#w&%tBe~7m*hgOUUX;IqllC`>koYBA>y}g#@FsOC6kl z>BW(1Ggfp={Mqd4`I&Y3g! z&iv<|Z||r10{oiYwX0SweQG@ws;VsW1dRj@1_tJdoGkby3=Gm27#Kt>6eQr2X%y;X z;1{gJOPLoi#Y1FUFfddwa^UB$T=jRGP-|JGoLcmVB9NrOPcTOqmBC$9n4$1!U@{!! zcB&=pF4!kjLnLYv5sYDBU$TgJ!@khG$A=<(4C7YwZ)5D%g0dh43w-e$Zg?pJ+If~ZCNz4d)&|7BBDi01&b;x0f+m^7vVqr$w3E~ z`bFD`n$za`F+Dx^-W zWZa+T=aV~6QIk9Ve9ru1pn{Id?}@@vxWTQl`$)Kc9Hz)&Yvsw1u#G0Bvo2gXxb?>U zv-DW=zwb6560l}Ll8u^{yFH78rVEOy&0p#&N;i8oi)2+pdB7KT0$Ztr$zqg6jylyG zTF$FVa!D62yLWQ8s@36ei6mQZX1-DXyZBq(LBQ6?fX*jsnt4Sp+pvIvr%a-iRJ^v; zHwv8iq~LHh*%2rnE+P3EP1~=t*S(KpGZkGLq2!YaW7hcyv>ZXc7gg$#|88J@XVQT2 zMM>|n+PL+<{54jQdAEByw;|>drERWGOXYX=V#zMrLsw`kCsum!?q{%Aob#-8&Dp5q z>@&c=sh7O=e*Hf*!LSJju$pWfkCW8JG>YR-!4cYyn_q+TpwZ&qj{|F?c;3t|QbaC4 zEphE~OyALEnBV*0qTo{3uQwX`!7FaX1G6kX?Ovn@a`tY?hFSrdwvy_(+FB5OpHwoK}KL=N2?28*XT&pS=FIk|rP z_sN5>lj>p4`ZSuk?(P;J?n1~(7sZ!gv535KDv(jL;><++_?wR+x0bPdE;oG{M7QL& zYN1hhlR_Zd2`%$nk2xrXzxG|C&sCzu!`+3-&2ELt46u#J={s#u)Y}!p7TFT7J`vWxHgTW%HVAMOrXaO$lo=}PxeZgnt26ptFXYq=}yWQkQ|5hktK&(o;dlp#%pA z18dY=>?HX#RK?R6Ur@5sRbe^(1rJnvVG>;Fv&L*7$KP@_Xnx*!+an_u(|o(eo}apK znwO2TVySEnrNl)k|kx{9mc!~Ma7a*5lZC2kaPBV|Zu^7;Jhi*0@x(?wRrtxtVD zOGz*C5)fgE^Mc#PNJ+-S?MC$IzSl*=B6-zaZ04=$^=M*qWv@lfcE?oGaO&*Ppj!vU z%@iaGfS6u*buhA8eojah$8d;56|e|Pm8E>~I}4$M#uI7dv75z*d&dze zLZx3JCefmNUe{xp`eQdjq=?RoUKft*$>z$x_T$q|BW*lQ@Z~7`t!+GJ9b;4HX!Wl8 zG?Y-TOpZ>VoDs#4d>U1_g@B=bdL+BiCB) zcUq#hGa<*oTfI$xQZw-pbXGimm;s@mrT-)Rw9etwUao}8?36yLdZ=h74#o%94$`RR zy)tX!Hty+9^GitlMAOoY0%T>)Gw%X?c$=srhMW!~_=-A!l+yU1s(!Qi%erweldog^!J1mV%C9$j| zRo)gq^&9vu_T2Aw3^FOM zdSv*j@IV_K6z8RCB;fNk5{>%}pL+%`;J=hoc0(uJ%z44r7d{UcW@`B?$39p61|ND5 zy~j)ssZbUDT1$NHcGPBu)3{1Xb!&~?=W27&;If@?6jOMYqpDtq>4m^@ zok(OmH|g6kCXI%f@q|&@awH>GE}Ov3GKl#7QF;^`N*GEw=~vJDlT4M6fM9ZG&HBFA z#pO8lR+N93I2uCpN%}#~W4H|Us<)#c*TDWF>lQT@Zbk$<(J-xlSzXh_y4Q5Ep!Bpb z(dM{$$WWb17~LVCMaAcOYz$rh@FP9J&K*@+uCl;#zzD@w!(R2Q`$5!RqA!9wN?{mG zdZ_Ejo?wEIi(N5ZGwQlkfJFXDy}33WNRt%4^9|^iqkNz1eE*>>VIoP~Z<_gPD-I4RXUf^u0$YZ)n>sFddev$Z1a1`2 zUFmvo_#c6V3U|pP)d_QCEJXH}vMsblr(Z-yvgX`~03rqZef+*Eqq(;s32;;QHtt8^ z1|l=|Cc0bpx*1)UguH|O_V3hLVh>~46*{;;3U04l;+(nvG`QqXg!ehgj|xjD=5NjzSzsE~H_b$`m+rJ%VA+OVhYBM~jjA6yz{ zS(_?bF1y&-Sf$sG2+Sx;2%9#gvhz5bA1jcGfj-uip=f_ zC?r|Zj3;EGFA%iOU*5N2d6&}eQ{z*&npD9@|RGwFE)V^Cy`S3FQ%a0I~U$dsC|)_zn+##t)d>gzEA!#%tf)XuRh{P}jmV965dvE0?(qjmme3EB& zw@~i)cN>QtT7nxznp?6Lzc6T~9 zrth&=HOeGv@LRnJ0$OWP7EGSguSLdkV2Wlg78^pC#kSUocRN0>0*f^E;&p z33c<2bg@WBg&e!=QfkrgXzeyb@GK*X;*>!<7cp#BqOl_udKwp8Fh{Ulw?vC>@=iP*Q@lQyt3c`mjh>zgT6uoWduIwyc}4{7#%hu<2kD-XeT~y z#!E>%I@B`Sts7+^2N*Q@zHRVlmjjW6B<(mS1P_72x1nlxw%xt&-vn0SIqX0I zQI^wxs!w!mX7vcKswPK9DuxIj3Qg7xy)?DnxvCJ_2MR;-y-+F6s{TUhwjFt(UQRNA z1810SNLs!21X;&<;Z7RAnu;c3_+H$T*}TI|u$$~2t}{oo2hf--&w0#^S(| znp?oPb0(io2#!DSbQ={YS(8krNs~`8t+>D5*s)Np!bBZl@Z-h#;u?BTJ^KL!SJSj- zJ=StJ+u}58*s+BycIekHmZQTD8hCntoY!DxpKoUyDEpEp&Pkw zXVFamPhfdTG<40EhG$RrpN$HY*JwuRLN5u*Dt_qOYpOHc%qTHASBs6UoH@iTd5|H~L`fkSK@Dl7Ee{?nqI%~Kyf1QH?r7CU ztdJtraio0{-nmZXPQ$w-d_$ERZs=!gV3AsFkCW9y;Kbk^W6CKqDI&P2?>UgV9u_js zHQxAHy#R{g zIRPDS+H070_;#JZ&MFO^aYJ5CP}iH%U{MyJSaBhdN-@bydDF({a@0l0kfvrQT`ZWR z46QF@?|t7rZ8O}|nabjM4J<8G%|sJqqoxUs3H6}tD#Hz4+R_4q39v5> z9AY#gS~n^<*$&a@-Nn3%K>{3l8`|>slEalwl#gYC`+=W^pYi{cX1uBpzp2P&(rFiU zQv;!`*&2MD|ACm3`}HL?q^7M9Hn1D zqnXU6I4e;5o|%FZcwa#{)NgQqoU8DYQsk}MQB9s@TQ=f4ii4Ta0QTkw_p#Iu$HD>Z z*E-Fg`r)TL$wB(3n7ucmR1=L$i0qm|fp(D8RRFKCt~bzGf2}ShiriHhM)Y1;>}s`B zq>o}Iq!>K9R28+PHl%myeAJ&vCUgMdeQJU^NkdCumF4?*!l9k?HIRA{Qauw=*xG)c7&*pjZ& zN0^b8fmO{VHwcWQ$<6ytC8Ve-xiULyGUUASsI_nC&{V;jRPDdeD0jf9I8NR`RmLhz zW&&$T{Zht-n9=xDL~Y29{B^^ArS~whC{oIlbA+@=>f%|kOXT^>uGCYY5U>=wT=Gxl z*JPiSahHjZ(A(K|WqC%4^Q~K|guCX-pY!A3Y$!$dR98zd{yrW8-MU7GOXCIwkiA4- z%QcyWo+J$*7PhheQSAFE!XT?zG2&Zc34)a5P#{?4-#&>&wP-0&y))!*>xrs>-)Yjk zET-2%T1=XP)jpr^Qtqgs_32q@*ABR2sNja?oF~q)HAi9rnA}iEPBRABbkj)Gx592u zn1icTzuTRbpqbYOz=7Cju_zjPIU!`+a1p#3#63*t_=!CdPs$)=l@Yyyd~ zd(V*R;_gDVC`yM5t4zjTF{D4@k|wByp)g%Rey*{q+AqV7)a0(~ZjA;7AB!ewtoGs; z_}Ew#ji_g}3*Ct%SUaQ(he7>f-`!A8bG@0H_&M|skB0&iQNsyoY_8E>U{NsS_TY82 zG&H($!kzf(?I(^qu@b02$X#^ve|9MtZKS(UkRlC2L^33x`q)?QA%;X`*JwdhGmj+@ zn=l3x^P`bM+tS@*1wpI-Qj;g$wjDKFij3VF&BO&rt!<`h=qCp2ax^<*kzhZTUR$_l zMC>3-<8}_0ugcaot2&;qJy#^tpijQgb81WQ$mVIcW7(s9zxk%7{&l})v)U$pxM<#1SGs9fo!ry1;52oN<~yn$?kLJ4&|7Y=u{>vo$egk#z&X`zkRP zNo|Mn0aQ>wj^MA{L$5P%JUt^7oT_5ZOH2zzG~iCafO;32)S8Fztb28F<1jp3*vY)? zWDp{MLvfM>iCE~0)0&&ig*o%X@-$q1)qY|<=Z-$IdkKv;7^}?SL|hcuNEcw&lq^m7 z9?I1C;OYYvz(z+xoKnrWU1f+)Go7gK3~?)tgvT9H(EuPrKAhA|MrtwL=rmi<@c=}j42CSPp@q< z1U;uoa6_&Z3azZ1Hu;S_8{VTDjy;d~VmNEqqD%HvtK>7We~_s!PA89*xIon$fCEtoY8kmm!t@Pw>`uzwx;UnAp`D>l@| z9IPG6rD4%#C97}Nc4+R8=L1Z5_~h5)A1e)?M-Lx{l{FEk>KecF^mG5$Uj#VTdSp{&le1k%0aJfS>em(%C!*55 zsPWj~2j>c|h(w{hO4ugm`rXx5@Jk>_n5Xg$7+sp_8 z3Ek%u3aXyCZl8vFFehJ>Jt_Uhda6maVmCljSc2WvMcLS5-y?#qfamgOd>ESU`>`V+Qe^qr7rLEa6tWIhrS9(h=o?O*;GJ*=iSzNYHb3gR8-I#g zH(ItvF|5Djn#xIUwrqlOAcQfebqbZF4zrre54n%u{kZ7DvikcN5=Kmnndb+L!N8-_hg$%GX?1TMvBTmCT%+8uDIY& zF!=P&5`rq;-Gb^PQOb0&sgr9suamtG8V;Iy1Mc4F}{bH{P96#_XPU!RtYMtXGeFgZ_kQgJ^b~a3X!=YmP|GI%D{uDy>E_HA8`~>Y6f6UOEu( zim1-&t`xkl(N!OaBP(B~l32do{mg6h)B|Uzuc|Uo@AC&e6S+lA7bzU3ndkj1Qn`Z! zqzo2c8+I9s93argGX!i+JlHTER?e3NP9}Auu9EfTM2J#M+!U*>F{|zHp~3iAR$!2b zY!-DUmRcjvvqnu@c7+J66RSqx97(*1g|?yJ(s>RE znu{#usYZiws1|hu&;C9_;wZ4YKlJ(`#mi<_+7}j&k{B&m$R{G-w-O3Zb znz})*BVKk16YU{gWMftjrBf+t%w^352fYn@rL2a_RKa%9o))borz(G}fX1n3>O6+7 zV9h=4T^fYX3mon-CuHr|1li4boYkC7j6v^cl8u+$K4t2;el#DpjcW5v&W6MG?LWNI zRMBu8=a~^VjSzhUccN%m0Mm7oBKt0*ZO%XqzhHjyXkK}ShfK*n`Ly{1&X>zp@!P`= zVfDH>GvwI5o+ujnZ|!o(33mf(e%hJOm;oT%r(Oz%$!qqH8M(7OipaemgyR;vBo@Ph7x=_`2wR2}WF=k;+b{6=eOsv87 zUz5oYtkd7Ho7y;R{|FU3YI@COJRdc}c%FRJkI9w;GR5LUgZ4Z;6DU54+U062CFsRD zXEj;j{GG;>dZTtafr|(m$FA0;h*lb~@)bHsFjl}yM$iyU@|-v8vs}&Du8Av#U>*P| z8SP0J>HZZx7{q8qX87j4PpIe z&hvzds+=OHlh?djygcvj5g1hi8*3RSgnT6v1#6xw@=6*$nm3Q z;UUDRCwc|-e!fqB8GmIW&Li6sp-H(uIRwsGE=9Uf%aT)% z!46(7Wf;!6a$;zb3Z0SMOwOUg;PC0k-nAI;a0MTsmKYMqxZy~6pw>vE^dsUVn{+=F z2`VQVs4aWPCV5*&Z#2g8rM??Ay41qjTb_xE;Iz*6!N|(EEjBQJ;_zG-`9xgYpwA21 zXRag`j7TaiO3=3(2x&@CAU@-bEbB^r<}n9A8TVU*04iEe-JuHM_ItT+fcxaf+|7~L z#sbX>x=ORIyyOL78@){{g1AXCaIA3nL3SAmjoddF3spBl?s+uMtCbL2z%=&pzDy|u zV;L+gSFjaom7iej_}Y!sj}XPoM!2>$WGOO)T`+lwN||6hY)HBSCRVl7S>Y3-drh}k z8?-I|41z2+1R|<>Pf=|)-GeGeIIxrvbZ)ix7*0XxK67;2P`ejuMU18qom5yfPx zYemjhTUgApB0MQ#ZgN`aNxkuZH(y)aLs4l9VSLf+QL|DSFZ#J*-KMCeBI88=59}vT zhDzGNaXc&%bp*x(LR%fcWuI($xEHC~o-tQ;DAp;Mc;VsAQ2gs2x5eP3s;`L@$E?(4 z%?X~EXQf$5keqyD^4!s!)mI>qT;$pcW%sIchKV~=tAF<^1WDOoWwaB4CqvaYqkhAN z*v7Y6$`hJ}`4#YNCMf>3#oPT{*e&YD9ytA_PjQtl9y_B z1OD@i5p-TQlzROl*V$*X zxQ^uOY9{^Me@Dhh-0f9!Nlt@neIRe7A^n`{S7&1jr;Lv-?W(1*?w68E^pQ_J>6hg% ze0$lAhC)BnjVU*DUmlMR-*Ct2YW_5G%;e#n{h3;hPghH**=SryofUn~#n<>w%Qiks z(UVifazO_6qfqkVK;p}}os>y0wRdX1#?6od$iS0(6dGuRF)#Zj^)2et;qYrKM-qGQ z$uc`i0`K7!7KiZgL06ll^bBz-dCpXiAVkuO&dLl6yXE;c^3%`Jo~FIwqSCUx6E z`F%^UKO=1?-_a5280~P32Y!NlaMDyU7abc7`9Ydp7V<{<1KvVj9NGLBokSVtiW@o} zO$fGn2wF7F{ZEl=J;l(<$YdDcIEXOfT!2WUegcfb+jwzh%Ruf}M@aCIOgfTrwMr;B zE@J_DZks!AhNe-h5T~sXDxGk9!-bYCDwM_j%KKrNC{}&@r%S{K&Oqw3GU*VD%S)|E z{lbd04Sk*8sm_~ViAw%8=EEU|cGBPgi>W8tqoXUey~0hm6guDTax1&ytCQ!JQj^xRV8T25X~!Wn}SO9x56LPGSr~{-9H%7RzUV7w`Y&_A78cg7>2KMc8_hpzj$9 z+)~bBm)g(Hk%+V2+T+LOlgbjp6-72rWKhirAqL>~M@(V)8{*-L|QSI3{zL)Hk#?=yV@m!90!$9CVr4pu@8F3xE7 zZ`vW7L@zIY(kV9ZxUVLnNLzqIb^8bOD0X}*Qllku7V646`HdX@BJKFI$xZt> zF?V-@8%f6@)Z1t*>0(4Pcx0D5^Gb2%{g0vqC&Urj4_{Jzdc~eVMbxrjUPEJiRNP+_ z$B!|Tp`@D*7Z&eydB@N*Z6toMo~?yu9dSy-I0dFdmGHB1zE$vSuOJ4t*^`(k+(nq{ zo4OE-K&Pb3NI|BF8k9)F;6oM>sbaOXj5jwLLSh;0es0tNP%D;pi6U8=VG5m_@fr+j zCR%-#!v|s|F{8V1GKiC1Y%tWml^gVx-=Qr zp^Zu>vat7iLGCk z_vw%19w9)!eBHT;k0{!Bgnd7?&fqax)8~XjV3WOE`VOG+)lpSi)n;AwrHNE8Z3oL0 zJMM88Bd3x}2RlR361=-ug!K5TbdfQubwS0kvPwM{4jQ3R!_ZC=Ffy2^oT!WU#U#I= z31Rwsos0Oa{axFk89d0STuZn~nCJwb0Q{NRk)OgA%uwUNx!~@Lkes^1yCEWFV|C`e za_~Tfo|IOI*otrq3zRITzWR#F*4N36W$|jrMS5pbw_LA+ot1i#%yjuVHeD8zxmMUs zRP}&SG;qrDR^?Gw4IZv1`$ZV#{c<~7W$2#$D4(>r7p45Md*W{&-Nxr(z0w>BTMQkc6qVX65JrP`c$EX zKNSJH);U+jezNJd+@vY2YBK8)4c8V~ypi8}o~FL5Xt@CCBJo72i4MGZnys+)BrTdc z{YhO(Ki4bYK7D4j)%2iX*AE&4bc&{}s%`FbL{cidL}lY^Pos@u5MQ&fv_tcXLk3-u zly;K4vsgTeU5G<{ZlRRy;Q<4~<7^|J&2glcYBwV-=tAQj3g9>I@9M!lFzVSi}4-t zNas*Hk;A=KG}6e5AjLlsZUl!=QLPFQ76&s})u~AEfL960*g3PR4*9xF;5)b|Pee3B zGe{Em>D-5D;8yP}jn*r*SQVu{l}rz4#iDsNe%kArT9 zr^>%IS*cw2N;6;cCJq4+eX8GYn|pK2WYWk4saEhc9%l z0(k|iM%NR)QY3bJ(sl3;p7w1baC zM`2}R`>7q;QO-xyjU;s5U%IbxRI!oEv7d_f-O%jz7r3AQU|BRd5;8MYB{=0YE#2=8 zJi{7pFE8Dvf@8|71&zylQYyAT`OKO(-*FY{)lxs&Pc>S4F6r5yJLg}io4+IwfP$Cp zqNC2%jn9;$ALU_KKE#0jypG?r_e%u6`GS*RlcO82iEppP?QxBCatgaaN&gN%+Php; z{$5SDsNWqH1N?rq%_~i9!lXAy2bTATBG6iL<06`)JG9i}4hNd+OQa7#1_)ao?0B6>9xB+Osj9*zpaK$EZ=Zn6HJR}gx3fw|;N$b-#rtB6>HH~s zCKi+#jIxj_q6@UltVU*@sK)EpdFekb>X?leW6qdeGt$mO4E;8$T|yd7nmB`AqNv|9 z9nwY4KCgezEhrP&kn9WcSI*3n@ zB_kkqom=&6(E=w;G)8AVGx1BrfIif!NTjkZyPzvV_rIH?@_u(yEP)L$4k^9 zzuFoU$eFLYsh_*$-*Cu0ZC>}1?NnpSS?TNzxO2GNj0|M77!9Xt@2u+hbnis2BP|gR zIsC9%N7D6$jC1a_!PQ7#1u-(&`Sw5{mdVu7bF%DhEtj(Hm8oJcGVHG*=Woen;_z0U z4o@U0^Y7Qy;PMgK*GIe(snEJGyj@9pJTK37QeruUH!mUzq`bg_YJ!3!=}h+O{s2zu5wtP(i&!_W7gkBJU>Ut-)F}^Ilfn`%xTk$e z{}eXrBjJLthh(i7!QgBx2}^9jA`0r7s*;K>fSE+_V7jLxL|s84X_&iLR_aXq)PzmX!|7Of zop7&94sDr$p`i8y1XC*Sd$HG4eRVNoSFN0NOwmOp1ULX1qbp^C!)F%^KoUXY$ieOH zXi>ZGAB?)G_qQXA9xt$W5cw$os@{a8Q0@pDwLRE0mzb}whuNfQZFt;rfeL;m&Cn&`X;m`+fZMc_rpj+k>r4I?L;tP6N zvq#cGqq-UhK{5pG#xon9m^<7Pl0Vy?i+Aq4~}L80zJ709z8 z&Zr*>DK$a3fl*Dl_`w3vRczWxU{vIbV6VSas7 z<06>usvmQSh5c7y1x3&(;SX`m0yCUCdsMpq-i9g>vv@!0_sPp&Iz4SxE5Gd0?SSp2 zfkiFEdhBdHeA@}{Fxu{qf+gQb4Oy#XKdC$}hhTHC!F{b_&D{WPjCU5j6WF-C9kpnB zE5hb(Nu^twFUc072qpruSfPCye>a(nE(ures- zYLZvT?AN`Y{!za+mtcZY#?~>Ltin>CM46-7>F^Oh(p5|bYooXcS-Utn`kAPF{ZQ9t zJ9=zdGfD+f`C-J1FYO$!NGh>AIhsE$SZkk9bfASVvbAy5BqP!z6+6 zLVkuR1Vyui87+!nu8hVye}e;GqPX30!%aZrWY@>I7_2wB<|a!!T6@Y})f{~zy7R0% z)XBwQnY^RkU$%=v=uw1b;F{QDzKR{w`J7erfOe#jq)Pf z7DT}jLK?3IZS-CIvXn&nNQqu`W4Pm#5**R z=R^hwtjA>PdE|Vg{O>8<6{2Q28mcnv4T2_4y2&ATzWjDPn#z_7+z&(-L~u&NwG2eD z#didkqS8v8BO=~*n5&2)C&5zYJTHuMpc{5#UEX-bQO`qH5pGH5y7B+(pPRLc)g3Vs274 zV6xy>x@#|Zc917XIcM-p(0202SwdvNi^oO^;#V+hlK3)66B1Rtw0m8s$PAfm!>=xn z{D!r2ZK4?$vI`|4dlq0(fp^$}Vjm8MEo|HE5Y4G0a6IG8RbEZ#%oI1%KBeE!(4UEj1!Ph~ z?!=Nxbp+j4VM19FUD~M!7~Sc2YIXrzsoACYf)K|_U zt?DN6ATcY6mGzDOS`3E4ph(h#&Dvqk7Km88wzcjm=hPb^$ev>jxGLf};XOetfh zwG45ClI(qm@qxXFv9%v)SfBwyyR#wST*q3dqvf(P(B+EjH|Tm8Tt(twl?L)6rjB|N zbS0}L=Ppf?f=vr0$*2n+!{lo%X0$8TLaQ!psdL#>Ao$YrRIq zYO35q0_fasE-hpc%xrh4GmxLs6w`W(3vi3NkOjXb*=sUm5sTTn5ihgGJbYUDsC&&T z*(Ttq8=RPyTcl%&{eCrvQsJ1LC;ljB{%eI3YJeN z&{owLQupEZT~z8@kayWIoSVsw2+xu6vRU^c2kn61`su;8I{p`X_exK)ATA8T#@lOq z*u-an0+|IwVxu+=sk0FQJP|0Bk%`FEtDH5g6!a`e9Aq4PuFG-MHO%B#1cF2Z99)$I z`nMsd&G{$oGjZ8iA3YEbb`(lF%PBHIMAX>i*yyhG#ZJ^k=)>c30qkPhc8bRH?N^0<*Wmo( zW+ND*^j?7z%(d)PtfQ}Ig}nU5gctBYqZ7PLP>SxMp;Q3vF{_YgRW)~sM>1)M7Bqs` z%&XK?1*Mq~X#4VroUqsOWt#1|H5uw4u{pJK@D+T~B;X5%MB8~8i3a;WwRGz=`GW+& z!Uj8`fdcR8zv3#%Bvd|E=}8!!jeF6lWkIq4ICfcsMDznQD>cCzc>~(Bv5fOdC7Q}S zQ9N7kwtKZOuq~7{qhQ@`ad?g2O=DJb__%0Dz{JKdt#^)7X;$J&PlQ!kK+>otyA_mQ z2z{7(u3@ZhvKP+%jD{tuqcB?=?Z%vX+vuPTv+tTMZzCTl)9}|xc1+L~>AftUbM=&F zgloO-u0lq3nJAbtDs@vle)4lv;}HtXQ+V{>gZ%L9^YQ2`#9C<-~8j(wYK})$>N%rMG)VL z+jEK~Hg?LHutb&xp`6XsR~V7jo$p?}6V0F`s=|iBcGEnJ3ug)t)9)nm<#0b*kzxFz zWM-=0*xfh)psa&g>7?H#3yg2iZ9Nw$l0R->IrRW2I^8{+$8<(cq@h0$s5g* zx3FPxA<%Qm)Eehmir>lOR4syQeoUl)&oi2|H!(pjReS8eqD|=3mP25`+f-=FUlii_ zmE(TsO`(T{m{>6igd6IB;EIOn%V(@pg0D@a6O&TxC}s#|0gRf=zy91Xvxe_+WWzmF zfc8?AFyGp$QZJD8J_sLyGD|U!TF6#Uf3DzAGLSYaZEDO=TaD5S1HjK`XTNusTecL= z2)cd-2!uAeBXEVzk#;Q*1-Q??9|?tg^0v-KN7(|j0vKmLj+oP-e|Jm1V7*&Ffha2Y zejInFoIP8(kK{1e%YFntP)uRbBw;CjST?b}Ck|wk_*@=X$qY20{|5M^L8S}3?4;J7 zZ?aqL*f~4vimmzHKtU(1!K{ir=tXTYeSs`X5V}NxJcPoaatDjK|X@ zB5)Y#%e_GpBRv%+Ei{Ue-3K#3`5rsFtiW(H27D6*J^-(kELXxbb06R)d3+G2R9D88 zssTE?hOKq1lK{4XFdRBa!Yp8wi|&m0YbH2}>SO6apL~`QJvjvptg8j~(MP)!e9PF~ zNbaq({*x($7)Dk!QoVA6iApqM(uIm$kkki_^IL5NJ8SJb&62a2$a1ZJ^a2cwH5!y) zXpDZilu=7n8hBV$xi_VI&6eJ<0C8@r@nQ0RNB6tZ=D_iLqz3~%z4c?8x-?tUo%2AZ z|F{`bn+TOHR;R=q;&%*z+_S2xQ!;qw9*>T}FQ&B}RTcBd6^_&DhBMkt(H)>tva$hs zC68dddG*-U^?AwaPhrR!+QSLhFaH9!eZxTJ{*zqvd#^pQADOVceK1mj0IPD$`(}@y z@=3*0V0U#NdBH!HHRrq2yw0aBYUZ6u^#UR}dO^bBJ*DG&xm5~=dAK{Ssy zA^?pgLrJpn`)}#X8a_uUb$L4nqlX-gNUYiL52AGI3v8;2(Z_z*ZS+~s()r-WKrSH% z@S5QVLmx>r{pUv4{~*{t0k;;*8=j$vY{{v}vwLkR$PgM1*F84j|8U{7O!#x3vEl4& zU@^!ZxvPA#!+UvuydPLxk`3H}`((qYKgH>nTxos#Ub<{+!QVVq8i4x!gpU$CZAkQD zPOT*LC)LZ2$I>bd=Hu9q@Im9Nkw8NIqz`WWI>n8~mVc6_tqkD+#_GxFBkdZ;&$@b= z{*i60Q+=?4ByBSJJ2(F=QAN~SpoLzy`axL7t90#;1=4^-j5fm{myb|OF2<)AwIV<^=Cyk zO~0nGyf!5jp5^i*!FmA1?%yWsi?E*9+I+L0x^P}y6G`Hl*f2+z4s>Rp%xdZYR|E_M z09P3(mJaV{xki?UIhWpcZP;Tdk6QorAG7;S04DbPi${OA3vd)ou2&hvpHV-~ zt{=Er!nzd|;0omfJlE_eaG7U?=GGE$&RTFkB$rmg*_DyWY$dLHRlH-A0cNk`Sn7hK z6Psz#Z_5D4{otjO%m+{R9!ZLr>P7FH)9c9=G3&A}0@8vD=YLLNfc-bK7(@!yS-hSA zPvp+KEP8p9TJ{5$l8g;#GO9B>(_hm1-TL{sTYu=#pE4v!1jMuwRV4a%7f5hZ0~CL} zUHw=8I>Nt<3xsaOf-Z6$kH5R7;q42+0_dlj3jR{w|2?yRj0@0O(Nx2QtIz-L{*6C( z+(Ydj`tgqu{$Tad z0ZZX-+n4&wa{kv|{55Xt3UI53_FP$&`QN>n9$3oTkMrTb&r|>3gFSxg*9F|2QB`YV zIsDf<{@)L`Dgv52h|oazNJ@Lu@_$p9KR*pu0Cvv~GLi8&7he2V5&uQ|{)bTitBC(e zyMR#qUq$?n!uhX9{2xa2FMk^T*CYOmME#RH+t5mE(yj{99GVkq?vx>yCz234bN>Nepn32+7f& ztDr$8%9l2Uj-w|jq}7JV%1pLO_6qzli#( z$Zvu9MLyoEm>t!4_b00HcwZnkc9hU|fm(VPs=Ab=8E(=uKTx!Yy`xtri`2Jd%s|3N z`0}6eZGspm(46=T`Tn@S1gLYaXg&opNabIVBuZICeuS)o>PRk339=urFzO{;PyQDl z$A|I=ig5jO1}?YzV@_MRt+h=%#SKwKJm|lz2Zn# z=6x6u6|vF<6r|UHBA_4ygiu0+&`W3vB0>ZXO{D}-si8>=NbkK!i2_nAbfhMLQl*n1 zARR*c?;OSV@%%o#Yvn^$mM&(`J#$UDXV2wL+6}=;9zI&gkIK|+uUtsB;|3N>y4<~* zJ75~-=^g&w9``>FNd8(B59l6Ei+FzY^!VNnfLslK51-GO( zXMMb;a;5_C!D^Bk>F6h#?=`6;Nv?As2gAj~SI5CCg+9mv7cAtHt#kLjdDz>xU3Hv4 z20O}r%1(a^%*KVgPw{C;vD(#zW-P}1t>`E~21MT`;z)r-NXqPOiE|TzX!x)hp;`a_ z>0`b+^l9a3{#v^9H*an7$C-`re7K75>B7iVXyKh7!%}a}%E`+kUH35u?*^c$CD@$- z4Y#*69>tRvTC8(GC_Bb=#I6L7Vu{<*jI`N8cTeD*g%LI#$3itkxNh9VVrogh+Lph| zT(;+cR4SLOcLjbIs=zuv*I@Wul1tZD50kJBV*xuxeS}{|y4zc{kP;+}GXuEarx^9v zPw#yIQta1C&%?r%sIdL4x&0Rfb3yaia1?v)fZC_na9q#aCo~7d;QuqVe-Gu}6`((e zP}$HO_DF&kuiN2R|MG>Oilzbr3Y9UDCrr4?v!T3 z3wgm&o{~qu4{$zz zbgzUZ&g4Xz0JuLR6>%bH0C|H-NjcU)JM&>t#AkU(o?WmWOW&cv9Y=xOfy$S~Nd}ql z(2qR}@2DjUKgK-KL2T}^4&@G^;-s;rsded^USlgT-22TWqriwei6Va&{r9c`F7mNe z_Rf)qgsEIrmG2L2FSb->V*J{jiz#?4*_fM72?^O&N-m~s1pZ7`3Y!qV5)Zu{81$fc zQdc-P9mOiHxnBHM71y&+%7jC>-b?YUAnMjHr($(f7@Hl~h@6o`n9Fb+CFaR~kq<5dKX zfX%TDxr>YxFMeUp)VBTJz^3qLynhr^j#6`_xOgXG3Tba|o!}pppwQ~1M0^bVvrt=Pav`txIfA@Py+6S& z$npA&H7NP}tA)5NR5eG8wpQU`=qAly0st4GKu2%9tnAO?*e~%4QhbA4EQrng;7NK* z4GRXfLT!20e&l2a4<)2iD|O*A7Z|GoJx9C#NL6Fc&YHC;7jVAH8_os&)d`cjXLdQ0 z#0zazB+k+LAchknkgxfbO5~U7$K3|aaKIYN+a4bMDOYZIH@dqY8Q*VcDkLMf{tBY{ z2$B|FE5O!uVGy`Pj14<#clm7CUVbpEHbwqveWkv9EDeRJk6Il_kQ5L^W7^59=2V=qWWalPpZWUa zFDdTv0`bj2vD@Xy6^)E2L7wv1Ah`D4_i^mL4ASYyQ6vbD$o$6zyuy)(iH_JAMP}7PLJuG$5eR>yViy)%E^!%F&N9l4J$l??TI z_IQWNe8P6rxfx?GBK>m&@p8fw(KE`RN#MMo43c&#`<#}?THa>OhwRTKF&#+W5vCEs z3cTi=gcWK@K)hJRQ!Wq72{bK7~i$tLuQyW>E`n0^$y(V z^0pa5zLSef5uwqlfB4%{euvdCHNarF!)s7Sp*}56!Gcux_?qK&0~gv1Z@WlkklT5U z+c))m^H>P5WA}oLwCT|Zzu2#bOe=-TJs~gX4ok2V)=A?0&9!`w&M^Y8mpxJ{dr0Ej zy||LEuZIj1K=*n{ZaNj&ZJE!;?2a^xdw_vFe(OK$K ztf&kf2PF+HGy#8YX$$*KUw}2C>y7mflW#3j=Y#9>gE>qaBxfdGxjL6wV5J_cRlxEglS2FUTQ{F9BmTpZ{*LR#+WH{-))88Xi=1|pY9U$- zP5eV9AqTd!^3Dy&7kygK%O%zMyBjbEEw0XzDaHHfmX&32PeRNwaIs`dsa(d1YdVb}H_k8W7OK6@oVROvr?RRMl~T6M9=xltxb-D4n?Bt8 z+oNT%|GBv(=cLcW6v$W-T8=tOQPW;4j8&XCMr4QASxBn$zu`E#cfKWa>|rIp~=R3@SGV1nx~0Pn{-VKW%7wieJ?n{~Zhq#mU0GSC2D}0yk3Ws*12F zIh|q-+dX~XZwXe|Pp2K-#N3#hkf+7EfV6~mDp;UdrS(lTzE0E*)F2g&85k{T>YZNp ze3E!u%A0&YWcyCCQeXAV=I6@h3SOeA6I}FY7C;~zWuMY0jR=Wi70>0PO|^58a4uE` ze-s}yOcVxLu}Wj96F|LsulcuH?redtp;oL8Q$l?aiE~wkM_&7@6rBwO40A8kA9VEd zy=JO%3gu~fS}jzzzR08NzAaXE8H$1oc&=-R6I{eAbiJ_=j<${2u4*6f(#&X~^`Xx${t5?miM?2D{CYirE!>7b%0>vi;E=-0fLSiLV8( z+s7Rp?7x=yhP;5^fUz|8NX|DY)R*1x+T2P`7F2qm8AHKZd}H9{jb0%zc8MM@bWwTb zd0se1&%T&TtGGCOTMwo?=rPJkIE%CoU3=#ymM#LuQpYMa@(=ZKJyK53!fcA65!mVk zZV%Edk!Mo3qpI0^aP|poUH|gdzY&*Ugk`&DV%k}s%FDlhzj z8cu3FRUQJbEC}xJbG3e+!As&A9OY6tODh?$L@xn(LqH;n0_*$xQn5h{#&WQs?i=XMH(PE%U%9OV%8_!?9}?LJ=~l%P}Z>^Ttbr zhe|k-wxDLVt>pbuLQsjGFCu8rFZoD{U*>iakd2k1L+MCjqsNpeO1;87`g7k=z5LQq z1xLQ$I*mEFjdM;fL7V57w3j zX*(CA+DA+lE(?MCb;hkD^OSvUX$h0`S^jsVBS^FQ$aVdaOnt8xg4s-0Zj~Gw``Udo zaVSfDv#I%4X7Ke9aHdj2LBkeDR@O{+iC+PGsl6hded1*wGb*adkLma`aE>ws(%z=T z@LvK1Y|X5j$!t_YN%c!Dz!8OfP(?43yCkjcevZ3 zl8mJ;Hb@tZ(C)uQK+r1OR+&_Kl0BNB&`T+ErNypCRgO4eDif zYT)fERHnnyoQMu$#gEbnG7M`C>*n$OfM9^oP`}Vz|1RrB5%zk74MQqoi_Q+Z+ajB< z?9wEKCx%YKLCU7aj)mKJ{(6l3OJ5E6k>9>+I*5mO) z3OjvA5cbQ|q{uWBVKS?mzh2=7;+qEbG_`=Y6Oa~)DNFWecq1~z@@|Ca?Ey6`gSQ>6 zTyg%2bMoFO*T7j@C5^AaT#6xiamM|S()8V(${>g8;`IhNo`uU!Yw(p5TH9<2k}UA( zJb+|W(YG)t%CA~0f@8<0W?kd9@j|{E*=o~6Sv`fyH!%5^k@*rX?Va_z#=z13O;GUW z{~WE}i99BV24)ghcKLR{&?D;!LAL2#($}GP2&w_#bo6^nKr4bFdZ^Hh zAOm{I@@;%vd+Fsspe3Xh81LG2=I|95SjC?!xA~RZY^!E)@$vu$|FoEJHgj8#UTsQF zVIMWA%aCZtWg0>u1;v|%1G)D_B#Z8eJPdz+%aP&zN~l7}6$Y(pKlo@K0hlinrz|jl z!iZwTR+XQa`9!Zy5z+P2upJUsAbli&a}p@<(2q%d?K4t+5eT522B5@9_68P)=Yrmih@$|O2u<>?&YQ36jlZ(p zv_Rlk-Yz|5V$-6(@qX5sBarFXW4eiGwQ`{2f%Iq|@i~z<6pjD&O7Wlm8Zf(Y%uZW% z!26Hn@gFW86Jq^< zQHQ(Z@bS7&7jOn97WQ~NA{pqqEF73B0srf#j%FWb-OpNL^k&1)4Oc*5@B9MtI2nPw zLA|`dB%>X{)o#EA4ukCuG7|9oU~J|6`CBs!STwwN>&D{OCH0#uVNIg%e?*bf6{AOve>a&N_M%83%|JXktT?4nTuIFYYMEYu+|TAUyEKQTzE|dB zMk{$eA;1brtAGWQcl&Exc=q9VA9*!!a=6x8 zC1#ONB;UHl@74S}FHtpRb zIQh^>g*cIya7^~*RaIK73J?xYkRJ>VG+G8MZi}8_4N}^BFQ#$tF6yR3Lm|K?0tR?n z?k9#WlKD$q-w%(9+5;O9xsS_7$|pM3@61;jaDpOqQx%wX8=Xu+QAN$UKh(hWya^>E zsstd3Qv-OAT;dA$$lMsNz$yEzA7Y+7_!u~~OX{anX58)P!eb&eC07LB-8YrcXW7vU zd4*i`Ov(gpxlB7y;Tt0!z>l)zUvM)ZO>Sa=eWiY_#R1@kK>LU`4H5QJQfcSbu?(?t z07!#3xQeta7RGczmdYV{*X>t=k?cRK0&{fXGTAa;0f^NguPpD=yfDVQPa-vNY&3|i zioC-{^!DhzP}=OyP%;youbFMB(hDb3dB^~ZnBz3v$$S9^OUnliP%l;hDa*d_Zw5;$ z&6&dzYDHJ_;S_~fPV_US_*qnHb8zZP%^TLIrn8rJ!_%sOZhQ`~x&G<;0>A)OD--zN z+ll{MDHp{nX!x=U_4K6G2Vwf|jq)?Zs4OCEOtvc9HqI`ZSdeZIzeK`8FA!n$DES|G z{JFZ}-l*c?4sV|pv*nnSpYckny$a-&-+Xg##oHClVG_*DwxjYPRwj)=s7%tOvAbFI z@f@8Xjf?c=S1TNM>Uy3w)g^1wb>na*%TQ7V-tH58Fb$tU)uEP)D5yn9=!a@$%B|;G zFIhi-aye5R{pN{$%dW&=K857`+l_lias1z8{oZq+9|7vQH^g}Kvw=z)ocuKFEC*J? zCVttXnNKX>CR=N5Zk04=doPVMo`_QRPoLDa@Bo*%5AE1wO>G{wtaq0B+!(z#EDztR zZmGk89GpVPLxvLj-G_T{($HYN)DRmQ?_%bkm-jDg4S7d>LwNq+_C%FH{-x|_|2zbQ z(Z1v~V>PEWgVwi$)>`3>;Rs;I1LV+BdZ}uvuakX0E|X*9w75aKLTB=+prPt`XTJ# zNq)#F4zGV?Cj97SR1YfT_Y`GjLUsG3OAob3lRM0hoxdP0c86TQKR*?&gKO&J0{4?3 zsO*b6i78l_sjs0S;qSk{I74QRc=JQ2+hZe=P`dRV7QO99aYM5x|Fpc>eqnZs2zRnw z{;HizwroSJElAQ@P^W)ZP^5V@mqrVF`{VyVs8j4#-V@1btV7rlKCb%P*|xNw#6t9e zR@PUg+_$+Jk5yCi7$O#ssw1Acd=z*p|J8(s!wE;+sDsy>^?Z;=%}{^I+aEU841>lt z$+6+GtF7xR5k6kuoVtBF&)~XFjnuUZs1Wd^#uJEUajC3ON@7CMYv~3yW^K34Z#KCA z6^Z>fLbvq3JHdB%d61=_@kAnc#g^{6WVpSdXG<`AsxcYCroNINYg;c*%E&_CZan;n z`jkkC|E^lb|97lCL!R8cbF$Cj_s;`S6es(E_8RYE?OIo619LX8L`~b2urS9%Nkyr! z0G34Aj+awJHSrVep~nV=95yzJ^?TC7B=4ld(smXdy%QZ!SQ8N?A7G_(ZlT3K5hlL? zJNFEjvd-S$8uuZ51(nn&lSbg8YQTLLn-vRCcqrrJAN*h`ERhJ%?rHr((qTw(pM7l` zA-8GNJa;bjnTyAE@UC3G)9voYFx#(#0nW=sOp#UkdI>!K zyi?G`8d3=bR=|;Z_h@4KS8P5@<{OKNZ~0w3N!@xLpL3ppT5C^wGKhOX?zN7uokqS? zFa=OOg~95W?zcR#lwGD5S#>YAcc4wZ*jv6Wso4_`@oL}Qsr5=_&Z!vl8PJLI0b*Tt z=go21eKN6;Vbv$bzHbeOjq^{ZNS$Nw>vY7h!;QDD<7|tVyao}V1OwWj(wLE5s!t;U zPA^m*uaQ^9R{~T0VhbmC70nh9gF{)4+D3#0!@_GUyAV`NNnS>YgeNwy93j*oU=Fmy zi*1=rB0^Ae!NbPDnuux*oL!! zA+GkJHl;2;^ zc8F$QVd;LR;sI<3x<&fNow60bzQ@mzT;eigkC`|GR{utmN4K-NBV@$)C+8Y2?K>38 z*LjfEpLUTi75Ng+>OXW(>~M@6+ZlVBeYi*T$zeNK*}#o1J$dIEczcA?Yz1v-`b3k( zUzcrn$XnGNJo6|4QzknTquZy%LF&nd9nRQ4RRf4ZEWTz577Bu`ug8X9D5rr!b)j|& z0pfCEh$Z>G8<-Lwku{_pts$jEEDW2JNZIVw9aRaxl<|DyE@1F-5h?b+TZ~_VC{F6r zywEi?E>*-=$)z>l57(9h2`(eTBY<)y%2ti^{*i?0wLE2me#jdj(F7&%z6G73y8g+! zv~E+~CdsLJ{>{I`^uX&28YcGG{_^`1b3gnc5C%^O?TPh#ILa9y;GiF~TLM`H*M?z4{52tQ&3sb6;)`r5QY=B~&K8wZx|_Bs0O4p(jrnBV4>S4%YGWcH0JlT8%7z)RvxuWVu49Ei_0@ULFX^yDA0V+!P1*Tk zBZ~B7Fu1hqv{s-W|0T8sA2_wCD}?*eufsYp>%-t5NbP||^=8E5qdtkVTZBaE`X=;- zw0Hb{6>)ywi zzVVrb!O-38Hy;jL7U!ZPidn`3gW$%OCzb^s{Q$0{2xNYo`6x1*7!F{&c~t_8^q;SB zDk-1Q@9Qn~^7fciI)E1iGwv(+$VjDI8uTkJVI)T^hpD+F39zf^$2hijS5_G9Z4tj z!=>*No-&(1yI%uvCRMK=mIReM<)1R~Yy0|BHnk=AwKdg*L{sjq!Q3f*G=%4j5wgAaJyc=OepSrqiqGQw? zhR3VFJoHUE`4T5@;%&umTLW%p`5}wt9iqYVYa$RVwHBZq%oLJR|8n)fzSpX>4t$fJXCiLH)0LFX0e3MZg(Y8<8C+Gm4I7Y*iw}J z`m(G6wGYZt1F2^!t-5_!Rdz)PtZk^VJ{>PBW*4bfU4J=ULi3;E)2Y0>oGq%yel*Ch zELM`|T?%dkV-qUfAIf7q7s76rJXPF={gtXIP`ao9AlP-XarSrAX9`vT$&Jg>XiFX| zP`-<{`s(b3Opqal?e5?qX(Qp_DQ~2S6}qYf?_w=iX`l3pF~Xh}2NXkr+Cy^f20k7e z>!steNT`;_w+?em3a~?6^u4_Uc50I_{j@>U%q0=InWOe{rma*)x^DXovq*A73%h3) zu+$u6HX~$cOk%Oba{~;3*@q3p(7~*in_laEhZm%$Us+G$jOcH9017eb}2Bj?w0O zt#e|xm&mUzHH=P{N6!1rH(%1K>BB4EFXyA14Vm6i#rNEb6z}S|bX`KU*erE>i1%UG z0^GqWMHh>Yk({{kBJ73E!A=$4XU5!7+muL;xtM=Tidi{N*@JC}`G58M_%69YySIXHPfnxaJ{yf9~Ir206)~rzHR{Fvs2BMNrSL%+5Oha z@hh9Gz&`EA2aAEsSuJD4`yE}w&K)3nTEib^0n{4+9pY`&_v0K!Pe|nGLT19WS+Dq) z=Z0TbGzh-@tmSr}kJ)BtF#Zb4*W&#X{d7>Txd!5zE1UsrN zQ3lKV%RW9sPAykd54+c9wi@1x$jxX5W|f+)TXYb;aJMy~CW}uNBX7lY+)B^Vj}LXB z(hj8CnirbW(tP)o3%oJPZ9Pa@%PI}-KDvv6g|F<6FrTl9J`XIAc3tmP6-%z@2JdF!;PR+-pG|Ah7aMT2=zAiE zidD>ssxxA>*X)5c{E4P>q8DmRJV^5Lp0CpatK8}q%p3_uR3+~LG|xP2*>OHt07s^H zex*&sSGRoTh&SI$m{O5nrdJQh^7=B^mC=<~>a*uV$$S_=AjHSb9KOH0HOg%^sS8}m z%zr46YTkXZV{E0z4sQUi%;WNILVSgES2{o6aOycX`SBq)b|vdJ3}C2+s-rq%!^iyY!JkIjla$4qKO$5#Q6t z(HLrQeeuw0S@uIwLCHZKl``!cuIn$_1J&s7T$iE>zRGdu%{{SuH|_9W>-QF4q?Ylh zJl3{qdT~NRO3R8@AnirQRVD~;AA?AqKwr$^yRI)aMV_-s6$AAyHnUw{mSrO@d>%nP zao+;l{L@e|u+&q`-RZUadE_5vQPWGxCkY|LsayM$HB+I9!Ma}w3q#1#Y?m%+{w)xO zYPe+00&lfoMp#g`P#vFZg&Cird0g^4HW#! z?I)pKH|Q`O7*HnglI?BgUWU@Wk8!HKw4PTIb~m%2(T{*~Bz?lNYlwAS7-fmXMS!=vHi50gtoK_nsJRXCWQg&1qh)mq+| zh|$CJOqLPXDvIV?B=VWL(yBVE@&f$|HaWB(U8GW=_@7Uj8r9V_G!CMp1F)AQ8&Nsw^&8l~%XvFGjQbXtM4LSmVo`3sy$iIz_(025 zDgxdB{Z6m3Cg2f|iJyD3VQ~VTLwt#k>0;f19=HoFJnDr;H;(vP_|ZShpI!PwbWdu& z7Wemlzh%DORNI{?Bg*b3o$S_Qo1cFQ*X5{A%mVL*$ivYGI7>b~a1%FivKyg>s#O8) zFaQ9uqf1)g(KM#DPNzgKM8E`+PB9*=op}03U%$TKWsgzkC8Qo-|D)VAug|e})Yf(B z7$UVrD7P01swOGh=~3ysoNT?e`NZ{rdS0aKu*kWh)sYL14`NlpPKjRb)AVs`DL0xT zK?V}Rq5iwqNVU7R)5Tje*A^;05Jo@awn2 z0@R>0)f~2liBXj$uKU{Gc7T0$s7ZQjTkghVP+B#Ku3gI}Z7suFMo~he&I9Mxm$k)a zhU{=T8m?V>!*A`mS9oc)Onr5K7QgQ}x!zykI6pI-nwi+uAX*v$2k-E>ZJXKD?4T3p zn-A;2q@qLb-JTn8u-U}Q&n$oZO@id7aX11eqZrg-w3dKfib}yg+h4gumKr4>w`i;%hVM_vDZDTt<^(sS!mF}X@X`RgM}X@CajX9@Y~ znJO5w5j_-AT->HZFTn=pJH4PF45sTYDNVO+?5Y+ASbeABiqrj(Ee4xA%SSE9(2 zo=eMP5^i4X46264gWo^CBxEIO<}R~}rVkXF^w}PV=NMX`B%iXWiifdEqGjBOkYbZ} zU4ZF24o6q1A;_$mF{J^$EU z3=Lp$IxYB`449H}bI4zJhXyKC#46#-SpuZ&MIJ8Ca;!AXQO-b*w+?;{d5pKCuHtb<> zQnFPSVTZK2$d=uWW9}#dNUgy{*uX}Q(gp2z$Dg)^CC*I}d}N`9DW2}G!}^A1%5$j` zb#;8m#-T!(iOPpOjT^^0rX}BV0pTb5AyA-}9y_K1g7)bsp<^apDZn z)jp5TI@^&n@i}m=%XxCedEXZG-5b15YM5_V3h0H)gR>e4k1+mhEhY@FT4>wN?YLgcX}RMOIMO-aF%f( zF73v+hX)&j7Py1mWjH;*=9kf%mW|Xy)^TQ!y6ZNxN|{k+_X%Z{B!DkW(ZtO*neAPc z&12lX!bhDhC}0)kspMyq!M*$9xdXCLt8xR}gY++TUn2Nd1s7O?$2m&n2|H_GxPOt{ zL=5A89&dgu>sqRqoaU)z)IEt%#V+Sh?pgzEmr`FIg^6EEj1eLmVKU45nIj|oz;jV3 zt4c9aM=9`pv6c^KB#rq_W8#jNabkaYS(mlkYH)`;ccF;wG?cS-(Vbg9tH^gVlwSk0 zx3|{44m9s@&1`X}`6uFePrFieWH4Ib#N-d}4oljnCO!>Uv>PnB0zy;@Y}IWfmcaQ< zy~LBBd}!0Ac^`0&jV>-9!@c+WZk=Q1P4f|5QrcGxN~I`MAY8RDyB*6_2k$sreOX@CESpRXVuH$N1&iW#*nUnW>qjU}Feq0LiY5bR+cn zd#uk#!US8Qv_yknnN&OTYQ$gT6nfVsvj8cIf62>`&0cq~?YYu=`wC~)+)ZIcv4zmA z_Evu31kq_@wCZhZDE5`W9Z}ESwWm1$-95Np)x~2?TZFYVA>D`_9p(SE(tp1TZ~YN@-KZEIQ$LGd|FjnCVjT=Fwz#2f~IZ#teqYpf~x?EG5 zC@d_O8EO`CxBBLyF(cG2yC$Z3(#dJ4Nm0kMnK{#2UVl@LwX#S1(vj&O#WETm0BEk> z8aeqphCvfwuF8eWPIfAe@f1Ct*4C$qdu1$wp)Mp+3p`9=WaM-C$j6GgD@;3|630@`xqJ(qlP{KdP{U!o+BT#$_#$W+%T;aU{j6mw7S1KBI32zI zvc+cQaHY6Og47&1ztX}17a6LsffL6Y$8`lc>-DK5ddS4SoTMA(z)lwj{P zJbSEZ1IAFWb@%B1dg|gEM3XF*Y#N+>3ah?lR<(F46Pe`7|1zxVcTLX4%MBhC38bblqQR>Jq zniqjt65K;0TF}$0t#pE)6Aujh)E}+GbauVQ@BjioS{kXwr`glOe?Pvf+DYSi0|W}T&MV85Otuc((x zWRdnKHFmoWQ?#xwS*jJYDYqcTqo_vIuzJb zWlKG{PN(Fm+gb+L-w)9iD`VzykODJj>5vy+}`X4@n836FS3FTiC!Ka-DM8gv$3 zbdLjPWuFXniD@tq`nTB+ezl30-B z8}3UNoUS_rJ*7yc8vC?{UE*GFZhMyOmmz`8cK%2)Sw`c;??AWNOGqxik0z4USo~v@ zQB4HrWY}`_e33|gcBW2IYlT)yik^pum%%RQ+;M{YQPVG4-jDMHjdsFx|MpYnTN~JCSLXkLJui}{6t4aZ$ z(!8Gz2QG(|6mOclQeo;?k@WP}CVR?U)Vd+L{IMx19PnM)EidCJ6(Zxqy{$?C5l2Ye zQ%$#F;{`E^)w=@{2_A|fMw+{)gN zH^axDp)`nz4!A0AIp346%uK&Q`Q+S%@|w)#r~s2zV~~u#nE_P*<^;QO#?8*` zQe#@Br~?rVDtE4GSB~lw5p72JnRwo@D$xj$UTr}DU(^ta6E?|HF+9j}DnFaMHJVpI zpU2^rvtjc>keuMW3HgelS@*4Sby#jZ&$6srKvtbqfNehCl#AfPx2StEq;>>utR83p zwh+G8T?no5WAKvPdwH_+KUtujFz^U=O<~QyS_`#Qx7vd&RHm~ zS^QS`2=%dt8Hm>ozBmiArzA`{8$o|9J0^G6iM9lTK!jGi0~e zc;SnoSUF~x8=A-b26^vQHea&Di;Ks26MuL$@JjJ$FvnptgdSK~kgSzq#$3(OD&fPc z4hOKqDvK1n3*vD$^h7e91}Tb{LQ$vGHpM_lSrn<QBAK+$$K)a<@p4X}hyFY}`v}+lrk%F^q=XY>v_5)^KC4W|&)c8-V{ZG5$s)_=P zBCEfUt%DmPvl0}7dK(>=VW>cuKLrC5*3yf@>_Jbf%wb?aT6L>j|C1ri^&)rp6g!0P zL0)CIMOFPQbiNp8$rPp8L!X>I_&$_BEN-@Ys(he=H?`_~ zU6&lCUPo?`4G&M$wXQZ|;LVIq;r94w+0-+9o#}O(L&6D6K<+l)EO6>?l+j|Q1|Z$v zxbf2O9#k6D>1WyWPwVu$7FTHmB83JX)@k<6#lpt~F2Ne-^SCr^F;D#0uk#RLQs0Xi z+!5PGv_k2Nf$gk?sN{@JI=4^53H*`fDXuTZq+{P&%Nmz!ZgwQTVz+E zXXF-GR?vOVIEuZWRR^p~rf`+zP%)^QwY`sTGSpDUFwLAgD|GB{iSuHC?8I)SG9C9@ zW-S>iJF6^RRykTN+en&nnRdI{~5NqnZqs zCx2p8>q7AC_b0X57N!HjsSl#Z&{`8R)@$Q*KIJEifdF6ZF|-vbXr7xyAItZnNI=RA zSlk`9438E=SbEIp$`+`SnBxM+T5pv06oz`0KPZVqKC+l197C!rtXw8+xw_?2h|Hfl z997@6u6>MVf>DC+YJ09$8Mm{4w(fwm9O6f-yC#&@$XQC? zz!k$J!+vM>%KN2E-d>}#Kl$qTbyTDdtG&CcVFnWVQ3CA+H}$~o8CVE~lSNTbx#9!X z!PmU0Jd^FhseX54HLOg>+)q5B#=UCgo^9(jXFrYM!%sSDIa=T@IiVy4(_9hSOE5Rp z`}*B*o=VF9I9&lbb9H;zBSr?jylkpT!q-=;xfso?-oumMiGMVUwHOP(AyCU~5 zicQhFIY%%SXZxQVk9Nc8b52dvqrZogy48hmBaxc^wmLl8QL^h027hXe<+Utn_kr{{ zWS*D02|uzJYaqcWxv{1-hAd~lnogYUwhpEFtDaJi1j4&u7Y&Ju{8-o+1+VBrzMCU< zf}arm_R=HuqCxz}QhCxI(2vkv9qV+q71jfMJaXwb>H1?!SNQusFBmsunu8pRdJP(dZJjQl~;s~CXVW^j$b6d zjxYr5-_9{%552Z5v$EMw%sr^C^4K7liyRh#D86Zl1!W8uJsxs5E+tZ6Qy zV>OnslaUTOA1m)fJ8NdhWazk?VpFO|#n64B>t(?O`KD_se?>k45CC-9k{t8r$liyS zp@eF~ozH8Tp+ocC=wF^^hA5oL8%}vItZr>k)M*D|hR7NxZek>UKK6BC<6MPKpE)E% z!r7s@C;WL&D-Lhb@J=k-j%`KX^zeFE2VaC& zpNpLw9C){0^|tXE(Tsnr_2zN~KVb`E$GmPS)TgnsmZqK#S7v1`_z%trXan-{%3Pmk zBFh^ia||LEOFEnGQPm9t+W%GKj0d+E21#XqEK+zU91j60JU5&T?Is64Pr)Z zXe+j9G$qLjqPz9iR}|!m}3L6S|kiu zXi)hp>S7%REm=6Z3tIEvzOh0XV492}JVdf)pL-}=1rbjNz8-wpR?4r$aIERGS)GU5 zY@S?VzmxH2!r*p0`che=V(ZDOr%I{23|v@QT3(sh%RY;d4kDFB3~mX6KFPc>uEvhL z?ArXbRhspE)8XM><;0DknC-8{HyU{oTI_~D2QSb)pADls6A2$H2m0I#uW<|VFuDL0 zdg2x|Uzru?gUK$x_gLko_)PV8SQ8K@_rZ=UyOF_G%kXrTvRs+hrxTQJ{=Q?^y66hW z&OMvW=#D9q`oxRigV{Z6G7Qq95$r(0V7)nm&toQ*%ZygDB&Nzadmk7OZ-STr!KfwW zq#}%HA{oV($5xCcLv~rYIQ(@JvT(GJu^R=oiqP&z-gXK83l)~Cw*J(0+ue6zj|LlW z>oqL@!>*_Cg<|M05uew+0$eBE*@&wmb}4xCj$FaAt$aXKs<_bi&tKS zZ75FDtKxpfGJKn|zYP(v)wawIqA&aC^RSJMmg#b5qlIUty6mE7_6sI90BY0uE(EEH zxVNjJ{6hnkHHKJwr{4O7=U!N86}bSM@yur3D%^!863%qo(L%|Oeej$2sJ-(tUMKeU ztqU~roBs)~1qlFx>eH+2|EtB0(~UkMi8uQK&R9320q zuif_kgKU=yDZG+@)Ov4}1ecU-{*S66;C`z5RbG@LS9g(9nbG$Sdek%{%-on(oV}a1 zceM@S;EK(sG{A3o%hh<2n7{w18t)z}YYFZ5M;1 zRvltohQ}CCHAPtM(x!k|bvRY5s>n0tbMH7ljhjgEkb-u>sVg}qY;e5H@ zd$?LuY&aR?bib3?mtF-2Q6E;LuD=RNNs@fFLJp8he6GSp=!dw zIm?@Mr5^0b=90fJw6Q^%qttupuGRRYtYOw`-a$0fj6tbJc8K1mrp;?3?t|2&*7Zdj zBCCY8!P%a+ea)byiKnL;;Of%*?B;u9ll$L~RzHA}1qs)Pr;a>5Pm&6v&^CMh>E7G9 zrHn4Qni?InzsiT7&29z^0b8M09q|!B6wHvXDtc!QqS>y05;m08<<7aJB7h;3DbM@>!*l7Rdttt2;6V z29kSX6KzdBe-8T7>BbWbgK6C_)C;adD4(@^+$3Tt-sr3cdws8|vo1dOOpXa@C11IJ zac5e?b}mv+yn%fu3#d>y*k5PN;iH67uvvqJO;E7SowlN)>;6T$OM!Ej#X!EYHe&AG z$*U#&Q0vxxYznBZ^FprHPNSR0pkUlayF9IWdCs%*eWKmpOg+G$bUtLGaD7$;Ifuk62YW=7 z((dcE#a(y!V_Lti|ME5fle61ru%!57iA{W0wY6%~zbK*)tyr2OwMeTp)ODvFP9HB7 z$Vb0*lb7q|s0FT2ei~<_UEF!lMoB1Cjd6cXTw#WRos!LW=%TvBcos8k_Dngts||%# z;vXy(&vm0~qju|7LDTMZIRhS1V2`d-xO|4_F~+?yU$1h56X5@fbpa5I_*uj%^p99~ zsCa15j+O)-0=Cjmh^L^5JGC}a-dtNNC~x)gK*jb71g)pMi%>AiD7v_#e|Kj(R!o); zdN@0=Y{rkkWC4|kY6?J2n(<~UsNq$)zF?|s_mzl1G31_y+?Cp`=hD)k-Y95>%>2yD zPk;XA?=M<`&@e#Zrk_qxK$|C zSc{Nl>`Pg*i?PSp8DvT%B-yf!r6T*zkY&ccN47+R7G$67B}>`Inz8S{YwqWBKi{YO z_aCp9W!~5Ox~}s)kK;Ixlb&=&+c`f#SaI1R>4dD)0-=$H`3Lr%( zj|Ji;`!3%hAs9 zkSBJ?@vN5CJ!;==%+btC0AUEPKtZ;dw{q59atjq# zs89x|S{T$+*X`ZbclM?<`I~U$E$SsuoyF|Fs2TF zNd3xB&)&YHFm7u`n1vII^K`5K>O;)nVr44eKX7z8xIz1sMq2p$nN|; zSJ>h*vx5o1CwqJKDg5hpW#$35#AmvoIT8q(B|z0OOLsw|LVL96eK}p62TEQqURP{x zcTz@y9eH-^Z9DVRf3yHH*R=g!Jq^*MsW7Grr!5iKYcb4EB?6}Lx{F}u)f6g_Ti)=Q zFBbS>n`zf_KV{Gz<5N;{!v~*0&ESi2APMR9K z`>4_p?grPMT#J#|E#a)+I>N*0bpK}LLBOXW1$<>z?w}e!&nJdA@)9^N1-9k6<2ZiQ zYnbgHU|sl%k8{Mq$q#{Ca_vsIVlQvNy>OG+xO^Y|{LLGa;P?M;qm$m!F^R5ic8`q* ziO!$_SQAeVn;T^b+}Gx_Ta11u+uHZ>Gm&Y&6zkAOmhYlACS?{QxJoW8inmAMN<=d@ zZ%&`|UY34*b;ioyit@7fA7SGrxOHT%WmEoLeKLYSU29kH72lw(9Gt|4o#>wLan>3+ z(`R62c5D0!lg?$q%sUGYOLKa^D%vKeh!5V3&Q0E~NL?9z+pzh{d(2C-XjJOOg6JRr z!T2I5Ar^l{nUbCFXk&JVcj>c_zNeeT4Zc72 zJs(LWkuwFeBi*+8M-r~JdsA>jKKSUHGyUJ+ z`R6IMKqPu2^7jIvX@N4~Y~V&9NxhNJb*t{bWQC(R*`eWWXNmT?`Neoz`p-Q!$ zXB4}ER*fUO>(h;DIwe*W){V~(KHHEn{qbB$Vq_U-O**SzH2wE8{k>UxE`dFEWp!wx zq&>5?mZJTw5XOH!*B{?znS<9Bp;pDb6duVNuXo7nGQ+D?IQeu*y?NY$chPeuUh{=P zw0lUwA;pZeoiGQ$VXq%by=E8G{uh_<=MIV{y$Q&2h;+$pA{66>U8|w%JIj8l_7P$m zd7~Vv6V`On*<_ke2>Eo21zBKD!8IgKDEy2 zimy|$^{CJbQsJpVnvKtvrmHZrN{4UCm|YNPc69Ksw|mVotp0ZO{yK;2sk>F9b7Izd z02&Umz5RaK2A^2h%lVovCTz;sA2*jM0)u@wdP)1QbMVKv`r}}EOVYict%Tb~%=F95 ze|6azm)5Q=l%A!No@ylLfhl{3m=*>7RP?3@EA)>y?*1W&H9z)-8C1?%nz&BYU%ZlN zR^$_sV~>Fq$;=;Dx#RQieGcVSR0A4KRo@QsNzr#Sq?1(gLivvDmrL{3re7j%vHbBF z{@yyZnc#p4r{03-;=X@OU%ooz_Wi{9QzWY7(mMBjqWWSmWt@KTFRg?572epcfu^X? zF27*3bPiu|qBK#kM$cs?IW(~>5g4_U`}7k#1eRvmEwT zJMD!IVMJmUa=$L53l$rMq1*}r+%-1CjH_l6pAf^ERba`5{w_0SVcq8;CfV-THyH2g zFf5h-Eaz#R!|yR(qKb@=-r7f(7epur{m9)OXHInYCNLcKmQcO=_1$=03=TOwzmhQLyyHHpD=QL9<##3I5hpWY+h!G)|_PABL z=!&`Qgbij1g=jeDttw2*1h(mop97?b4*;|eiEmPB*$s!^7i4VYwF-F^N@?@O_dKT~ zFQK+BDWat)2@BHX^%uF%AZg-M@;WN1q`Q&4ovT;nE@LiyV!P?s=Z5p>llv~~34FXc zor0~72h|DQtBzXT6tZOhl)wMnaQP&Tv}TU0Lb{5hhow(s&zPa=lc4syc@AO+KVv(W zB#>fR8%!&Bb+GhJGq{LhF?{?N3*YfowIEH(cvoMTEu4-bOx|z%YJIShvU50CV2pBo zW}3cl{i#CVgNC9ccFn2Ewz-l^A<9{4lr3ua4%ZMihDv5ttFN~%LQHh&A|L$gu5lCw zp@BBD!F$t4o&g#1n2Gb;y96OBgCSxEtup7eDvH}hD1qR_A$Yt?Pa4Gads$aHF>Wc& z_@$|A<0*qgI$zVwPaPIqhdvxDX3q{AmcQHQ8YjCAq4|n-5vJd|1^QCN8cvhnAlKqo z_Jm}XL^fk;HeYMf);bOHBtpA*(b@-6KYrNu+U->>+Ge+`)FJDl5$Gjb#E}0{X7_e= zTOOEYt+)DlHF`z1e))Zco)?v5dE%EO55l%8XGA~n-S`x2^5tBfTG$PE)0CKGv3!Lb zU~;?cm{HM zvVU=qK)@0Ce~o1W*=Tx$=Mv-V=M#;713IFO^gP(Bgc*^B1-C|{P&bxb zHnPZEM+Q6#bHizx#P6Q$D6535t2+FdZF-VOlAEfsM{HiAlKcGE_Csce3Po|smQoHX z(H3=ja0Vs;toh~Nc3N4N5cjAq31i=_Twq;B+zvNvINo0(fZng(@#e$QilRPV?4^*5 z_eqdGanD6#&|W{2q}r1vnF|_OpFJJ-9FaWQiC`AqC=25(SJ*@e=I5&=h8qY@08K`K^gKXW{eE3=g?j%f{-WBrm(;xy%S=5>_Y0UjtHjz z#Ps+wN?J`MFo4sQeCn>j^cAC`qIGp^AD97UVVn<9j3ydKjN&&Qr1!(9$oPy_M1aap zMTOy5Hs*wQiNgAqkSp*1Bz1rxA_-|}GZ=)kxN9{+7I6i;joB9@;LuP~S zg_#Ter4=;TN(wAh;O;I5mG7 zUHrr7jvsH)1M(M#<#MR44Qk(Z?lwn%C?D2?%3k>6R%Za!Lh{d?>(WK4IpJXoL6b)@ z7}RP#HP_1bYTjC>k(kjs7x>ZBvG|53gRu-fpK5~(nm0PBwd}u;3+SE-Z=VVGE)p9kx0uzXl8h5U%3M;Sa5_D8NjC;F06geksK|j>;Y3lk66VvftUDD41<;q9t1I=L zyWZm=MS|I|=E*Kzf{KST(r+&O;f>L`qAD8cu{g9Q^DWLa$$R9rX(W)oW(D84I!pU2m7g zdi}i_h~z}i{=&SaqZ2;uK0xEZ>yXf*~;~DzjhSJq7W%7zF#V2trH+pu;Bb8U_xtfh_6qA(Bvf&8wRFHuw8T&`vg*M^0pa!3q*lHZ*TTg= z&LMp$2s*Ae8EetrYXEb;Ko zoVA0}OcZyMoX#W7q!<`$Wl#iCR7)~nXlhfb~^}pQ)eEM0nMNP5RpWO zSYWU)^}o%1O>MH7-8*Poc55)~e(|H@7OYcV3Y9N3`#R;|Wir;}Wx#VI3{8j~>AiT% znCwu>xVS2|z0WJ5VFE6rx2`aqd@naRZvJai<=iiBPhUg4D+0f(+9u-Z+k03NL0KEu z*{}@01{;V%n%BS#pmL5N^kmGxAonc$Fh%4y!Uch9IX}fT>cjS_M6SQebn}&eeX{CwCnfJ=w`@D#$5;PUH0i;-8+k;qFIE3ayo3T-vp`FElWGCmfes>4UKYm6r~kAn)Ns_4 zDOVCChjOoA?d6}}$SX5rd@_#V;}DsT#!XeR6<8?xaQfnMV59R&)gT4`8ZWC5uWemS zo8T*wSHZTO?bO+&$emLxwnP;#-<#s3eFFdrG6C;c{s{Nl(O|eL1UmcC87L#uR_m>2ozzh4hEL zr-H9jyzbwT+N$rJb8^WLkW%5Ske`N7a5yaD?oTx|#*1+fK}m>)x9Ta(L#}fyze@Bv zoN%+U*A!5#;NE-_TfraM%sUmgbRKb#O$={ES{XXJ_N>@9%0+B8lo$=!K6p`N#MOHD zk0d&46%d~SA7%9D^@Sqk)3@X?ZV0i(*L4fOl@2+iKgm~es6b8UBNzl9&a}r%cr1LS z=5*Zu>*CjKFn2RR9a2Z{rWYT@zcR*~$<{0P6rM!8vvIOGH)-3v+$*$>fizH%Z>1?~ZYJ8(VY;3SCNku= ze6{;L7SROB{x}t0QB#x_N3$EZkP;{1X^r?W4fI13539HlSxoh2a~%pCRqb4NRld2i z;|x120f#VLlr*bU4SKnbYKGlSqWEn|9AH_!#*F)Hphw<=;3Cve5!oNxQaOtmWT7Tp zk6euQrOVytTYjH;pAnyA5+9aNBb6oHbRe=OpZj>cq0iE#aaMcw3+H0s@qFMyLHgs) z+NTRm2mM8bm`*l8iwdmr#%i=Nz)u)CwemD3#GiMbZa;ZX;Awa;(~es=-lZ{#71AOs z0WdCiBXn1GCQkL?=0cyHbAH}{(`1vs2({yEXL8N&W;*-!{`#gywh&~UF zWmSqOjUs|i)v&cmbjSN`)wahy3jM!Lj%TAOK@tF?`2%&(|Gyp#8~?8-pOfkIF#(G; zj;yjUf+C0*aqknQo4-^(d8LG9ly=tE9epbtS*2Bv_TPg_KJMe(ad`i9<>6fSD}sUn zF|spPpZ1)NaV#p;V>_-+r1&#kev6%2(Bb5RuUAfX&mQ@=gjq4kf!5A7(5Rgz;ztoy zBFdHxQXJS~U#^BR5-cHMZyqa}2uDFFGyPWULq+OmXk|CL#R-AH%se`SD~BQ7S^XT7 zNFG!N{T0}*sQ08x7veTWtoIlbdg0vnEP~VGh}zJePT8)G&x%smb@z@<_ar*~s_m-Y z_e6Ool0aX7p32+kp^q(9N}t*AcO|P{x})5X2!!jEx9 zXVgZl;8g=4AIfQ?XjD6`^82dihGV|q`qBCU_`A{Wp z#@q>$iM&Rormu6vvD>pXzLid$N&FIvFIg5WlBFL1{`t}4~=pO;*EO!5XRt!Qdk zUb7}kxYnt-H$&$OI==~E{%#hB>Py&=TAw|w0-A9Ps$k%J^1GGEV{vOl!vQiql6C6^ zCt_&HtXZ4h!Z2W{#AFjRh8cc_E9L9fjYcpB9M@;|8q9SpI)xb<+Q-Km_7klM;P|Wg3?EcYutWWL(nNNL6&4sK^UY<<9b0QsyNwE7k4 zq=+ztyTXRNCi^42$CUQ44u3Ny*=0O0CowqJ2Fx`FFz`z9N6cG!HipG6OI?2!Iy2S;ls{lLH?u3j51 z^ILiUM~}2oVvjUFdAj~^Ug<;&*nlX7;dA;&dgYdtc`?3@NsLdXqq};~cf6N}IJ&(s z6UfhyG0k34nK5b`o5$q(Y;?{I$?BBACeM;og34R8VJE}>{d>-A2-*bBSdGh^SnmyF zc}?__wKMV~(-7ue3r5+K!K$Tq#QHxrHn0uJfL+vYw1U)nRiZNMP5 zQ?)IVAgbdSk`W7axFK9zh=r`7e4$V!-UH{a(mNR{$_&SBBeVjY3)df8{Yw6&_eWWN z>X{=cuY3K@Kf0qWWhwBD*bH4=!Rf)`v#hYQS3NFm$q_Y?8l5_>i;badzI+i=#ZP*$ zaF^Q|n*Om$81&)p{RfTrLoXJR*<>FqNVw?9t=1TE6J?(+Yj(eN{_%EGFvg~zID3-Q z9`k-(y+K1Q%kJZ>ueXp8?OsL&uqVm14m|R7#{hd(`iAXMCoZ$qceV{X(P0bGEOOujt2H6FLFP;1~Zsd#6(#u7niHy{M zp%`Q$H(eCM&oxX4)k%GPRSMM(ZiEfAPgVoRg!Q-1&c{pJ^?FTJIoiAKYd-317%@h+ zW@%zaLa>D*!r{E#p|!zW=0Mzjp5CexS|~xb9tSn?KsYI$1HcNxlnucM01-Wl`PV*B z6V$4(o|QFbAn_nm`Km!j%F^W4)-06eX0Nw#$r95h`445+f4$cdK)ahrS_ztzS}I>5 zhRbE29^_Yj`88R3Oc}&+PZ444P#?*ng;r+ib)9u<&c^(}4;S36#u%BG=)>Qt6Z2QK zTI{b8O6_-6H#sTL}bUnUUM_ zt1_dWP>}xc=Z9yTn*(qSyBL|d%Twa1Zv6XI{WalvS~tIhb7j39e`;w1VQ`al8jiZK zHV~?J@R_}6=_O0QJ#5f-U#~^$J;dyo9*{w`;0w#%z!CGZc<>~gs+q4 zKn2hqQc`(RAu*O5*=1*M`9M`c7MLeEfQ!7cz>|E+5h|3 z;N31^!hDZ(;fo8hVHgy#!|J_JpLntV_mk| z+wF?~oWTuF&zg*>CW{>PosVg7ktMA@rcc%*9mv8h&^Vb?r+CEnn_A7KK2FhlB(cn& z;Dyez)0*{B_fS$;f@w4o0wY0UwpvJiast_~jUmyWR(Ag?DwMNE0bpXn;uxqZLY9x8 z(9#loi`pZ~yL$9%@%91t61Mzr(+liqiuFjN&Dxd0@TY`<(EUw9q7Y`cU~j`=JBfeL z^Y>3&d*S_91@!TDU9+0QhPhDXa(Ij0-q!3UVGwn&;vRT1m+tZqxcPJ?nh4#$`Mf|k zT`FGaJ@9XI?~}hu*qB(V7#OX709EH-_b~(ZqksADI~+e-^UbhQI>)K_mrm~&n|k-9 zSN0Du`Se4;$3T0ruA=!k?d3&wy`AXGw>EZh`5%^gn~$Q8H&*I4!4)gx#aj^$u+5s~ zHSHz^Ie*zq&tXJyRdl4ZQ-AQ|=AYz?8wZmb5N#470erti)-a~Rmi_NoT%zUz2svtl zd1e(w{HveUnC`%;?eQ^ls>lq(CkKlDVbEFN>5#mG;NI(1QsS+-y~lO`@AppUrC#{; zlS}6r5f%Eg&wOuxkUi<}!^HBEzqqlqkzrnzcc-ej$F3?kJgFM!{#fBN;hTk*if6x< z?fjlm42G4O$IF5~JBYSvrtd^qyI{}6z*`YKvT!_iHqpdI6Y&ebW3JLC7_m8uXHID}LBaGq?^x?dS1S}Xbx%0p#DCQG z{fZlx{PFHI0eJsM95;Co`uQ5vu+Lg1m41bBEwJy5r$qW{wHQqtXM?PT>tx@ooYH02 zy(Z~!ZnwmwbYTV{LhsrAgv{nZ9uqM`j}&Te@LI$0etH)JPL-(W5JoTg%g(scqsugk z%WqofeF{N@^H`0y5|;WNtd8_zb^cK=u8XV*Qv?@<<6se_2djiF#}PZN>M?RFu!=D^ zou_G1@!T1h3PkmCSQ=F4Aqd(vTV!Wvaz5{GEiq-@XeE%zB#)DPb@)2XFN^~xt$O!& z=7-&>H*%LB6$iJAq6IS)e}g?&e(m{at%E8Y3DE47A9ZE;StdM9p@}@WYuqQ7el*G{ z1de;w(;cu&ooWMA4h7SHLu|3 z-kLcsghiA+ScKLY%rIAZbM5|{1$~qR~p7*YT^FvvZ z8<2rUqxvD9h@&k`PwO-l{}RN}SJ!9GI+ggXswE~Rc}GaJBClT*hIhwG24wcrov?k3 zW&YP?zE2_#DeSE%^h$!J+}(fzo)MH3b3JKlmaDa9kAv$a+-E0$olrga?h4#I<_s2o z)QOUeV5BB)&PrGE!VAI&3JhXTe*7F^(x6`f<-=gDli5Bf76{vxkF#TqxeW_1ur4Sl zB8v~F!Z5&}T7JhXhFLZpA7;oDS;eE9HsfER+lAizwfwyX_)E+Hr}nkxLK=j360$LH zgT;4Yf2mk)Y1pSKEI0S<_Xp|J;>}M+B^-eX5;5WuVLl#irMf3$_G7sT9AvM@#`7f7 zBK)eq|HgzLBzF&qx}_v+scz}HqOIV!op*+Yc_!irIIP9?nssJElHC-C1S@bVjh$R6 zyK_>OF1i(yPg$IY>)lI){FW=fHauy=?nDV!MD51FLrX@eevyzDJGjYw<}4|B#?h@XP;xskgyPZFzunLD8^Cl})uZ z=Uo$@q3eo(&vj?NJ=&jDZ?_@Ub9nak1nA~e<4*rG;1uc=Ls!_qBKP!>2gYEcb%o9# z_VdM9ZsYc?F?W0)u)GY^sC$z&>YW?y=6r!oK6SPEa%R{6wufsr2}-&-s)a-; zLa;_HX4P9Gg|XN`jKz~06Cn$AtXvl=b$T2M>p$uGGY6 zhqt>o&13xLD2&?=CxGW;-xX6gwr|Z?pO?N{YglWwBGgn`8NI(%!QqwfOM^ySWFg{A zDi0ubK;e_n`)IwBH=LF_d~qGdU&&4EoU5|L(E<-@<W?&V;3rRp;ABT38XJ-itFdIfQ}2dByqa1nwh?OWe`3%{D(J8||{ zf2p;Jo2H6>+8SLrB|B79?Y* zJ5N5IHmJOR_Cg*c#Y%-fQL=9QDa*PK9Jp+P2?210E#uZ$*nU2YZFXG0SF@?EE9;ho zU1!olh0uVW{CZ1OYz^vHg^7|(rnL?<`ljxb0VlaQ#ceBP|INkH8s}Ixf*|S``OI zEe+XhrdkrP*tko+j{WR^BJuV1K!J3Z0?RN$OzQ!M$nBeVtPIf|8dTJq3 zo~kFCG5-FBQ~8o^sOxPvRQl!9tb%*qfn#Mt6=DnIzV9Bd2Bry}FP6?LgPKf@DMU#a z9gi|6ngGoKf(Wn3ubJmN-rmqQY{JvVUee$F{_hq;EQwqFw!Cpg^XguUnKeZctjpYf zQZrAlUMSupK&(4i&?~>? zLNb+FR^-71010A@+(J*W5upoQfSIyh1&Rv3UpaU)ltFO9m%4Xy#eNkDWn=b_wd>DO zi+l*YeY%qGakTcSY|QoOWM-3ctX15Gp5wPS_l(u6-yhB-5~iHWJLo*G{>sD4kJcOU zHf-cX7A+ZQ%#QvHcmys}r%`&n!_1`iT&sJ+*iNoA(qQ4Ma7Is|4!7b>bFp=W8^!c+ zh<8_hN`G-iXKz@OuYdRDQUB&6w{aifL|zQXDs~*s;R^{aOk<9P#D9=x{inV|DwOcxYS zwu55)^$I`R2=YtbgO;?u21f>}tn9+b?)tG$;?cLS-PK8(d-8}J<(CE0Jp(UGXX#S3 zbOZ}Ro)5j1PEOy{%F!xwPr>CpU)qW&(Z6?hNbvqI#b9bmo#zX#`BV%&6A}CUOR)X) z;j_PNdm2}{0-d)$$`vy1?Cx)a#%b<>=X^Q~)LeoJ*ws?A1ZuO`uStQ4*Fb2rCNBK9 zSNA^(WaV0tLezBZ>HTnW(6GvF7@E$X0u>4%RVh}VBbh`Cq5}?ZF!j)jTYbMqGPvPY+|fj;;$=d6al1Aot3hJhX9Jy!pi=VC8$|Xhr1Y{Y$~U~elPVJD2iVL3`*hb`7F|*-(PNHamfQ?4RXse7@0w@Sf(Ca zP2!H{kckJlkJX1}Im^5;H=jp$LBo=PRaDST3*GW_pnNOGO+|DY`&x~3x-?~*0*OEl z$nqZB$?R4H9z{MKbNwQZUW;bY$Yu9WI^IEB5}WYnfs+(OEu;1HAR8cQDsacXdfOe* z_3o{mh;fB`t)ic z)ax*|9zMZn%v_$D^JpqANAFxtSv78o*uh12?vBdVUAjW2BcNEQ)ghoqs5?16m|w(nxpUYnu4Ad@#d}h3ym)IiFGjO45ZczX zYa`1ubc_^v7u4IlH;1c|=RxZWnwDkl6u=98uz}XuB8+N6QM=l-`he}e$GzbJW5v>K zZ~t%^*pFnIw;i3QVgAZ!`3N%fd6`K}!{4X(`T)d`KZ>|G*oT^(P3(OVzWS?5sJY+J zoH^2d-P8 z?{dK<6q|!iY$R85KP<%B6|v=r#%$+XR~VNH>~R~|k72ow*={k|cLvrC^QBOQf1M^` z>gV06O#AnLGI*{JCON2>#d3M9HfEF!Cdof0to=SJ4?FHj_il0;_YJX;_}}17`Z`X7 zlwL8yZjo!}xOfXy()cO5FOUi*_3t@8?rA53Mm17}Wz~7qkfYh0$c00}il|lRkp{`0 z3KgMFUQm;;Zt5*^dHD|NCbMcO!Fe_VRo6W5O z03B1?t11M{XG&=Hnrj!=wDP2XJtsX5yW_jnnW!X`We!`HTn}L?l8b#I8krcgKjr$R z(|t2pvCy%j`_*8LR-&XQ?`EPMTmx-B%Ovg#AejUWdZ$3ONXD9-K8<z-*9s6nUrMh8>iC(r0OZ9;>(6Y@VBy_(#F}3)9QoF7oJCL zbf0jygLi^^2gcXY*34_DHk$ORD*?{o(5b@d=xv4Bu0KE0WoEdU{CHrXsfLj3h|#>l z$d2*}K9e$FpyDvy&%cO!-uvThENZI3)@s<7u|SqwP}dysTzey4(k3&q?Pw$7HUiF8 zfc74AQH)1kyyF?TxpwC8NA}ZWE9Hv+D!l*008sM+Fze=ffqHRsWryO=3K439x6>r} zX<=7SZ?>D5^?kfbm<|`yne>N%8|kUxJz{Ko$LMm9+v>sjK{0Mjk6G#Nh-w7R|2sERQ08P z`mGnVGSzW_FWqT!2=RzsPSDE4TWN?+953a>m+1Y=L?g8gj*+ee3e|ARfJRMw%BRE%_{k>td2>$(d5md-Ku-gm5?MSE@F)b}JL|(z)K06lVarN<>oY(4@@X0KH%#|fNF?c>2PX1BDP z(_{Jh?18u8N<>QG$KT@iJr|TMH(n`$N_#7cO(AU#kA<^1*Fb6?eNXUx^j^3Y>C)I; z9olrf@02F#K)bSCirj;R=*hId(9|saGHESoe<7M({N>Y?A1P_a#7KUu`LK{heL`>3 zEPCN~Sv;49`gGBg4um1`mdb&kS$Q5X4~{bJ&Cqg^bk=5(a8kEy-%VgFlHFPSvhp^P zGkX&^|G3Xy0@_U%DWC_h1EBiLF5bs25r+LMyh(V!7(LnWmu|LKXN)*1xPceIR!k5y zOT}>1N4m5&uaCcNY98M4^6l*iUIn(4+WtI;Y}Qfkn7Y1bjj21oNnM9Pa`TeZwz zM(J<{3DEV>1->b(ZjtXZXwVZbE!(KaP4x&yNEl(@tA1%gTW+z~*<@p+yaKX)%x{== zWL|fkiOJ%U{?@7n)tU2uXu7WH>ya*Fv`SnM5f-_y;d+z2W%Qdpo0TZiN>PpMbwsZjaGu|&3~JuI--o_~Ly>vD-RwgbWvhP$DoGNy)Ap5DdYMt*f-z4k`H~jbczh9a zxsr}Z(ujiRp4;fjo6ir~G=yX-OkZ1&AYHwqgB^VHXFEo^EVs+#?NoDOX6`}GAP!$I@E!fsV&C<{h^Gm3DG;(p`Yasm1$+<$ z$%JjxLM_2ZUBWMxo951dmc7v}Ws|kov;J&^cdpZ%kdS|MV`M!5*!Ix*ok7z(z>3i% zW1eo-69x;Ch4a#z8->3Do5N=%*W%t9T&4Z~iDedr zD8_Hj1&&^kuuz0LKXN+5d2iRxf#swD)$U`+d8_P(kBq}Tdd?*r$WoCte89l~rvg<( z?yqSXs8pe=`oC$hau7K7*e-glq1r|mcA&yl9fBQG5a%&TFlaK!%L_!(X3526qSG#h!JKYaHf#zb65OPCvZ3Cvjh|4I>#4zObIxI4xB#QUMH! ztiui=*=)u33BQjW&lm|bcHebA@4Z3y!M zDtIbxXZp}bpLp{pwV9uT1o6E_6P{(#N>QhTUI3ftGVEh$)tewOq3h3@oy0RS)!onO zDIw<~D^vVKwo9uMu~88ck&MwD;WvOg7&v1V_Lp9cv+vN)L1tr2(k5{+xP;c zfW^3&IDWT2LvrWSq{YD{{h>onUQXfc-;uB(JiG889Y#T*qe!5#_AWJ_4*KExhx^rr zEv{EG3!+a}W#D05n)JqoKJmur-Kp(&Y-j{W@mkpedxi13`eqpFUbaoNaByMWQJhD_ zx_e<&PBIQiY

sb$ki?Lcgvca~+&4-gzx3r)2tx(6A`V`!=Xj(9al!H(mm^m_=md zwAq4d7uL?yOq9LlhRIiP@mCdJ+G{a@qwG4k358j;hWlEhx14TUAvI;C8BgOJTx#)K z1!_qS@w3jzHs{P)?%A?-9v6vN>rP+Ye6?CayEH!IWnSo}W`Eqm^`404?F_qQ23_MK zR265^=9nKRMI*imIHcVHH&NiL4H^CJJLPeK7fl0Q`Z)inhz9SFB-&DSugJAEKKw55 z@6R`!RX-Mg}&+2tAIdce=ux{lhg0aI2;`dK77Ah>>>2OUjnhNg5?j z%&Plc2-)78PixF8M@Q0U#$q%SwPvDu^RXlvJO`%1w+;+c5mwVUg_SOO@7CuG-)Exd z`Zg-;)R-+1TpTJlboX1>b4!xoxbj`?L^(Ds#@HAgOD^9g)%l5yj9|)}OIK#Mr`D30 zeH(|~KXAp^H>SIt|6hY7VERTvi}DsGhr14R6~5oK|CPvt6*HWMUJ!>-v`Fpdf-!)J z8mM*PfWtG(y$BUtjOdCm{#EgM(*?~3d1$eiA8na3TOqtRo2(?pqcD$1_$r9SfyQt5 znoX3&ysDN*nxw|MWCHr>^?S~%lHS+Xt6Nfbq^-B=Z8e7sZA8^Q&Qg@b2TkxP9!)8! z8_2R1PeV@mx+@4fs^~D6iwhd6YRq5f6jZ~Yr7%eWm>aI|v+b?dzbl&_Sh=JPMi!0^ zK8oe3Tx8F0JTat~TkBsD%+<{6ESEp2`DM0er4HC5P{kT>E5(^cc(L%W|;jjriG zr%;iOPz`zcsEAihH@KabHTxsdlAq`?j&pdNue{O83y0CbtI=mLVevxc`^P15@G<0) zJalOzFJJgPH@UEgRPAl*S4NM!(oRYe;J&PGN(tOW8VRlWHp8!wV;8CgZ2`evWm>Of zs;wlPRBdf%Resd)1Ez*+rZ;+C(Pl9Y1xr&}cro*T4HJrHNiyt57Jqbs#YY9 znCBXa&l=3+QOZNLr{WA%Bgx?nH9~uNYdf8yZ0Q=$DgP3*__8lgAjAh0_Bdy|cWyz| zs0fPQ(Bis`a!c6NKvAt#bBFV)n2U`uK&N8$*sgILcV1;!v1{x|YJo*k?(x{oL%G~D z0l1&?Ma)4lJb%<=Ss5gFT=?{nl4iN)`NnKEpkNup@X@o9C7~_wg4f>ZR-mS;WVi01 zgCKjKcWl)(wGPnT@otBOup|UTmknsfv9LjN1G}{1RxR#0C_x2ZSY>$OWw!xpb#F~e zMYr&NRQ`k8^!laUD;f)vn4z`QD*T_4o4VJ@aenDFDsRBhsQiN`Ca8Jd<(WtL0 zI;`t&1CRIY9lvIg`EPX986aP`Uo7Xf{mZ&-DgF$CT&TCt{Uv>!bN;|U>Fu_m^1eWW zAGMZ7pI_2!FlL+Sw=4|?#%{!R@m>t9pxD?RvDAxI?Z>;GBo1c-$a!fn(GcFHzURBs zHogE9pvBiU8e&5zO(L69l0|qSZq7XC{?P*H0~h42=B-7I$eR~S0=mW3ss1M$*7qgt z8W%mcvkd8A_6C%e9UC)P2_nI3WLj{1jiae-{Uw802vS}uo?1v*9*h^ranO^^>6UfZ z#^%oIUVP{}k^x@x5Buy?v|&Y@2k^15dB9mxw((ZJ4qm5 zskt{xzpAfH^M3RkDR!=eiJl~~oh6phpc zcU(WkJq}{{K|C~h*ZrcAiwC-lF4lX{z-r&Ac9=*p2t*V5jQYcA6`P5X)Zp|17_%ZM zp*^-N+VMd=eJ#h1kTouK!h4#=*Llv?;<^`^u;o6TbIoj94gSJWk;`OL4M<1LJ|M!1 z;a1ZXxs`+OM?GknUmJV6J*Tkj_5c#2Fw zt#*iV)_-I=@d1PFX4^hYR=07Sszqr>4inXz2(ptFH#5Wn<06Nh}mO--@# z!msqt9Q@Big8LOuUL|i|7r`112mD>C?eE=#?p{z%A+7k!AmCXpXyhTY=NszsV<0zH zQJYI6CW=$KUxZPgH1tIhTuo+qmkd6#>{hGbJTjxO2AR$?B-Pjebbp0hzCZ!goNjZ;xt!Bqv(mLIQGcYsc zO@XsChe;A#MqeQh_W%_~7&fTuTR5vez`NEhHqske%A;;qW&28_sD-89Fj`4xv6^>^ zlz{SJVPmbWTspTDG_WY5mbIAz#0&L8S|K;oB+YvRXXZE;ZKh-yjXi7kAfpJIdfo94 z%!7{^h$wTig{;*Ig8HWbBNo#YQc&C_YeZ_P<+Qtu1+Gn*_E43s1W~IBXqD~a<}0Ja z#`AmCfM)k~8|1UVp=6vJH+pLTfbjK9w;2*(EpD;s2=xPq;5{_McdmqP?Lsrnq$CD| z7Ww)+_^YkC6A6pMOR9mHzZakwG83Ezy(DPL1Xdcu>Xm5fj1s{W-`N!^kAc7H^bexn zW+R6!brUCFsUUH3*opLGO!^oY4yFp~&{$xIK+P}8zgQ-odEqkZn!?_PX#+Vh<4!30 z@(s?~SWH;LJHFqPxe^BSDgFTO-2nJv`0%|mU)Tdt#zO_dQuz>erK|qVnKazae!U+3 z?j^HbU{D4D^e`3xm61Am!bxvc!pWxfh+@EB&OZu`3WPS)H3?FZ17#l1t-MSH-sdIx zYE2mwBBCm|6g0pfj;xS#Omhv3$t|cC8faG;!9|0vLSX#26X9iTQ0`4JMN3`08}Wsa z^tk`eWk8t}&jFf%)%RAnwv^+<*F?E zStgNppttz|O#Z;#9Dz<8URFfiWc8Vex3l+lSVbTHvVavmLGE=%polTZ!s9M&_~j~f zoV%#Jf&+!Zl)-*`{Nw)6s3K_~okdk<>s-@6N4cQe0esNsx4sEtP0P%0{eNt|Wk8hS z*C$K}64E6t-6h>fNO!lOh;(-<-5{NVba%HyNl14O2t#)ay*K~eecoqx-;eMG?zzug z*E#1`0IR9Wrh%28Z^;~Q$$ZOHvv(FFd^yzy;K=YeZuE%Ue#yq_zp;kzBZc{1s~LR3 zh}*jfQq4Py>mcK82R{FaU{t*DX+&r1W+Ze%?6_he>xZX(($~7#Pk6r-QkDLBIihA4HGenqXlmGAex*+ASGoA7RNAU2i!7BC zF-mX4|7@Q04`RYwh}QC62tHNnaj#4=|MxfdNCRvOf4HzeOC_b@Qqw9MgzJ66%t3ia zKAT9#+agw75nkLiqQImlm!t-CApXNbHB-dw`jW5Tkjbe6dYR2(K_G5>ch1nE&~44X z{qREV_DdrN;J!YFYXhi-6BXi@FN)q@qP_3@s7G%h1BCyKDvUF5zYh?5^Z|fYfVGf+ z@`jWzI@B3W>_5AmPuOyR4B*mG<0_hta_XAYRmrY&y=Q&W{q>;);>}8!|AMcViPql^ zuZt4XZvRUlIA{9bP@+|DRp!pvqP(ya<61n*yIcPZ-%YO0^CME%BQxEJzEt{_xd9+C zTV~$a!`ND%4NU~hs4Uix6kN={mn8ecagpb??>#hq8nJGnTvh)_Bng-_dfBh_=;e&5Us8W-?)n5_JXKG<03|}$M%27Q9u;)-d zc~CzG5V8Ls=NjDi|GKP80%$l+_-mvJ5N^OMgasAS>!<6}1u@d5ZijmG5>xEe6&v@B zuwezJwxdmwz&X3+Dt6iFg2XPS9INefzCc zP{6NwO2Cw3+?c>Ao!5p?IZMyo{pZcp-g<>gn=8JthIL;9N=eYTVAhz}f$e_RHhbyn zW}CR)9ap1aGlrY(N|U5uEcRyOl|^Fzq?b<46>ah#-%~$%D-@tDP}JO2{`)pH4fuD_ zmEQ`+!lH}KM-xPfMF?3CMrlfEc}-J1g96XRl>cAT^OtdVqoSw?4zT&6NW3|uWG~N; ztfO|63{JUODQq#4!;=uoV4fQU`QI_!9c_`jFMOYS$%J zpm0;?DxA`VvCj%3sP9h(M+M|(Js$u^W+m+Cya3dl!{aRCnr%>VL1DbxDSz@&8t6-< zI65yi@N3Ow22sE*C8vLWG;oKsjA@tasLxj!=l>~-FPE>UpYa;6YPi2{*JxG|p8D{b z9KVn!%5=*8f|4(f3B714YZf%U(ezr@@-3aQJ`@Z1&hei|Eu#vKhuxpXxj5um9(yvs zlGkeTI9UVQU?S`FE>R2s*l7Z2klfmN{JfhvKF9LaU3bnE5XEM==)v|9AWS=O+A*vmKp4I)D!H-LRE{w@`7=h0OaE{Gh<2 zE;mbm=bYb8s@aTbBeie&E_Q#&O(N(_Yd3E7RS&y%B3y>)jQ+R({gLcT!JIizsRF#A zrp>#~^AMU?k-CbkJySq8J=VS%`R$dn4st-J?c)2ptx4Hw@;H&7hU$Lz*M%=q_N9u? zs9+p~4s~Al!-cx{R8U&4;Jn!Qi&)UTStZk8x}w+nn1hBX?){SH1~;f6se*X7Li|Sr z>*~%tRjU804j}|)8^;HcVeLp9jWba9+Khu!>*;2Az~sdA2;YPL(hGak>&K67kQ;lV zbP27@5bc?=u%_Jb3n>QIEaw0qt{1PmFURmbE3@34&9jvBdM~G)Z=EsZY$=tg%%E=RGHrW zx~*fZi#`x9oaSoyw1!Y_{UUlZg9&XQpXB-=H>%VMXRcD}y+l{(P%E%r7+mV)Yujz9vW&TJFXW4Z}@w%x+t0I8TO{{CG9PoUR@W}L~-aq#|T)!MWVEmc)<5*iafwFRl}JRf3anDo;3OOnSu zqxX|1NsJB?pSGd1QwAgtm)x&_&VfVFwR8-3!_wgab~c$#@&JPVWMpDDoKDalL&bi4XBtjC{Ws?SJ*;J!1`#v4%)Yt!#S|(1!p|2F>e;1ii z-Huv0pNC3*asPR5BImVE&mYoFT@x?5Y#^xB^>l@^O zqojXdx%79a&2+$=sRZKk6Mv)aosA3e9!f4BMpNXJ_cxDG4aC+Frn`Hm%p5Mx*kLvr@q@Prh8nnZ;A3!PRST`tz*?b+LBli zIfyRbEBy|4gkMU?<;AGHd=lf~1)u)NobQoM;Mc*7!v0P=lU1Qx1NDs>iMtDbT{E$W) zQlo6y$*9>8I=F=6wKzq+vE?zBh8^urS{1rpVko>X-E8|k|0jW_7XY%gW&!|{0{qwn zd6&8eU^~|c)Uu5)!xQ|p@3~cD%D^7`-^GPE zJ5CSLMav}c{~}_Es7aG+8w(+=)WoHr0UV6aEmw1fiL3CzOONI2r~*U>KvE{)->hff zhi29VK@VHpNn;y*Y+meL>YcNlfnQKhQHhsNF3HbNbN#X8ehrYU9nXgtUiT0b^aZhb z+F6dtKru(ygm@kfVwy>}J{W6TajWlIrsfc0K3&7dobOJ1bX{Z*bSF$XjTkp!Hk&Sz zPa6N?>zUp5Fd<0Eb}Oa6+l}e&#+keIO|1f6qLIMEX>ZiJsmm3w*$n$%Y`k$5fC=a} zO=ef~8`%s{prZi?h~-;xe_uU94n{XqZI|GB$2+*Qoco3fMRR;|#zPCc1$bB+ zHYCkMAtjaGk|M|=j^xQIPDmY#h!R@3q z*FvWgz$izzATaR&$Z?m&f#g+#Rxd7U`ic1M3gmi6SODh^yM#NHyvzcTT+BS@Nm=QU z72pvd<))9;XIy!o*>9({3oRe}g5V3F2aDa74L5qg0my> zKXTh$gB*e$I!a>YS6hae%Y)I`Ti#W)?U#vO&YW5S3p3q6hDXC63U2-l2{@)1V+OQ{Me$d7csaVKWa;o8NK3)3S4$x$x9 z{_E8HuS4(?Cv!Z!R@hh>x`dV57n$5o9RLaohl{E~08d*3F4IJ;)Z$49)%w={ihZ}Y zd2%++#GA?2{QTcQx}+Rw5dMdybl!X0`VgNnA(je0=P$ZK?PZE^C}Pb$$iqp}u2yio z{-bF%qX4_wj+;anE~(%EQLmTyU5oqL*G$Jnr@_K;dGm1>^Qnsn3iXV(g1vgEx%{0J zOUx&k;eHCg36~@V^it?+@zjNHSa8V@&9GzlR>SEwgi2)1iF*6XBsd)T->Dg-P9O6} zSGY$P|3$9>wegpctdtb0q7n;_hQy)=`Y>>n(5bc+8Y$`)^PxjTJE2uG&1$Cf8IOmcAZZtY+8Us7b`N+IXq^|v zd<^bSF=4G}*{wnN@=i#ReWr6aFN*Jp90aY0#`^Gd_Wbzq_TUetEfIy^?V0061b?v4 zNm>S>2KDG?*sGF2EqBn}OfiWaq@PFvt@ZL&OB?DM;rEV?5>Gv}&1rnTd4=iWEH@e zjuPhJskSX>B*&$X?EGsQwC+W3d$Fj@0+kcPhP^@w7qjg+8<2U`-Ua6w7GN1vbGj6j zTvQ{ipY9?*6wy|pxb!sMA}@#pqSJX2`G0Taw++ChJo`&Cs<#Rv!Kvxy2}u-yRE8(( z^rHF25^UgYNTTut21#RbnVRiRoWcrI`Nl-Sg2*-%wI4Nyse6iFbF?wCM)CL&KKyKL z0xD5k+I6TS_Ya<~32`U}r3q?1_7Ku(uuXBbADfZe<5mCoL3QibmizCo%h=+S69<7+ z<{4)xKCxQ<^x7|fMIU(o(zeE(@c6o(Vw}HpaAl6PrcN@cYo-3~yC(7N$pmRS1QVao zNE&FYnPbz;;kD4jepRznwU|%e@|=;`14*OsOvaFg=lQHeQc6Q8qWtgI3=dqk&BuH( z+}@`?65+s=$Vuk>hf0~lYE)!-)GRLO1}*&|L_qcaarv13hChiUipY37t!R$`=iB#+&eLNUXMc@~z7!|_7giGujmoZU-f7yQ5 zi%1eb(G#@cbX$0M5KugKew~R@Tg10H3~{g;OH;s;h+IsW_Et$17Y+6c%7!mYPIBQ+ z{$K>{6@$7auYp}ZiH?aEvu)J)LnMvam(lQo1pFAChUnRnar8?YhBRIUBoH z`vv=YyWS=AN)vs|qfJ~jXFv-^X!#j(HyP)T?--lbHSxqYzKV1G1or6tCvVIb;gSWH z`mb#}VHuoKZZ`+xx5ea?&BYaK+52S(p66c>Q}Sv)=%}r{UgR@!T|(+7gJa%zjYMZ_ zb4Z*rwS0RkthiqK@wQzL$5J}s$0^XgqPFOPY$ zHuzhCc*DTjFP{&`IzVqbp$Q-HL=1@1&VD@@$|8BWCWbS8H!4LbsS8hUsSF`9EEuYd z9ftzi8b?5e8SLP@MH+)bn>`gJNQrXwNA#roy&zDce@=G}9CR?Z~t6@7F*;i70;z>|k zuv~63tSuq5=8f2|SOJ860D!8yX`=SE_4k zS!x#=@;v%v_IX7+u?#*f>BG_nKsYOhF=)_y7b_=mu)Iey8IkHv3bNeKF1D zK}|<@QIE$LyrO8Wn)Da0Y}oqZ z-0XF7`9;%lg7gm?CHZKw*jd`$&yOdW2Cesp_3cy0TyT9tCkoc=@(kgwvI3W+m`%O@ zm-(k*uPIp-o_A(6d%xbFI2cY$tA^Rwx3e7rItrOZX1FUrF%zn&D#B&tGQtMIiW4^x z2#=R!67$Fl{I*-1EQu2{3$IOBf+G=nr)ja36vb#t@SWDu&THUHmg?-Q+<|EMV#@|g zuGE(S3ppO4Q~5v_DfcbxGW8}0W8`m}mQ*Eh9ds_~IwV!?VX#W!PENGCTVE;-O&-^Dx=JaEY#+;{(98f}Oal?O#y{&C`9@ z5nh$(-teXv+)toItSlu31~MdUZ6Ls2nsW@JZNKXP3^)QY3 z0@TDa+d-o|IQ%(IOJVvlsx8I%hU#DXr_j<*m(mVuH!>GQJi!b9T zNA#_YQ_9aQ)k64)nZX{0QlirHe-(5q)ppQQVi1cf$3gXRdk+>IKZY?N!S1ELQ_^@# z1l6mm{piH9)V+6k03OE_FI6Ylk;`kDJZ^0^#*bZb-WSvubq~bTaJBD2tL;its z$d0?t$EF~Mz=>!UhS-g>qvy=d?<*OC%aja{|Cy~->3>bT|LuI^)Fb!T2B%cTtKW^vnwVy7%eSpuxzG?uJQK7)ns@m$? zJjhXUf$)@RRyF$?*wdV9pg;-%!wyeHQmgQdaQGD~1$iyp$q*i1PfejNmAkXJ-ls6K zDaQ&e>*wbWlJiF1RzR{HO(Zo};19fAqHdT>%g%l|>7=O!(bNZWKoYE3o+3!{p0|bp(Fb+reRh@Bw8%1C(%;V*B{Odm zwb_HIS2420M)ldr+b~{+kvfD`*-YaK*y0ewIQFmeA1~YaGgY%d5Zc7`-|8dhA69~?Dl%_*|z@gT)~66 zA{(KJ=XJ=Y1$Kd)lz7$shB5^_{%q;b(Z9=-ZcBxfP;%!3a}?a(6SYhA=gG;_)(u_zFd1(euSmQHfJREd^g`s zQ6>3BaN`f5VQ{8HSL=lN=uv(zg`r>bq`iJy@3gx(S9Dj=$P?$ffaM_|B&#y|6{y;5 z6yP$g$TOJZ=lizNW4=D#2(AP;&jMio7{nK0IIE-_(@>)ZcrQgjqR>6yff1ocmkg*( zEsdrJv;UvU{PUm6WDr2j6=6EZugQfZCXcBWYb)**Qza9y0X$#N&1?@aV8M&EX8-cR z^-}m8PR`>I!@kOI$XjDtx( z=tb#i=#y@d!lco(VHB*YlC?lk`U;NM+jYUIx-$|4pSv*%Zbbvi-F{n;%5gZ$X@FRB zd}lXQ==pXd$Z?0^bqJL@5_vh^nXy=p%!q1)zCTf7EGZ}` zF`v%6`cJWCF76JhUl-u?dXPN%*qtAk9gf_D39e^}Mh9G_pRRj|zu{JTl1_VXp>THI z6Hg@t2092LZ1P}Ni*V%MbVpm3(mE@EZb@dpz6{+FKK|GK&!;gir@7O@{yBU6#BjXZ z{w)p>VjTn9kO8RNEOLpuvB%KpE=FmD#0qyW;W`9ePRbZjY{j` zWz7!X0Z%UB@$|$2pvuHLVL$tDCdjTTBx9^oJcPIGwRv=A1{SUk?S4L>H2bC}(jo4o zAR$dcJ(er@6m--fyy^o9(icNhWUG1d*A7?N1Cgx&+XVh9HR z8s}ksuZc3p?x*8%k?Tvx>oZMFy1Xjj)mKdzmoPOazSX{i4*$*u$(79Jy(5`cWM+?;`-2iST`1u z=)&ZJ)-_zXdPe!jBRZ>a?u>kleDwd`NV=5(Zcs_Y@zO$PlFm?CGT=W_@Qd@jBPmXt=mAlA0sybn1n zj&Ujk&AYDXA?aT;fE(2$UV=8lX#H_72eK*Eo);fh*(3>@rLs<{TE0?j!!_S55EN`GOpr!AVuaAr94EdGYJJ9{5kqj(6F z%Nr&Ml1!CaR&9!0s>{SF-$;wCa zbKFB=XEH^8%w1h<&t3}e0e1!}z6L(#I0yp5QNy%K{|W!YzAjiSmD0%jI5{W(5li%4 zg~u*^zi3(*G3d@=nhe6KF5S_K45$7KEuIfi)_GD`04}J=U0%P=>l1#kd@6{fO6VyS zLdo&ed4{8*ECyJ$LtH@jMw(uB7qDE`BXeZ6%9{PBN~s-A*}0n&k$-dlS6hml^HP?( zS;+{DSiazvJ~M|xmc;sk3THrNC=U(g&#Od9I~Ig3$}@xnjf>Pfe}r>A7QR{G#G9Hf z$W`zt7$X2sA@{ZB12)&zjriaC7*VWsa!#L`kc|LabmavAru1nNHb?;v6sfZ0h(B)t?kt1jP+;5KT0~+Up2D(Wdom`qMOUJy{Ap%uJEBdhf6f0MwoZV<+p-3GEQi4S zGV-}kMFY9xQkHECPq!$a1&po|tCBw7Aygm~B(NYO0WFdPA{y1>k{~n-@9BR1ITWwz zPG8_!STp}~tk}f@r!}Z5VCx>&QLgI6#BH>8YDLC$OO61;aO`$S|8J=UOW*APe;3Y+ z1NN!8C!NS`b#ZS=B_^us>hqWB8%pYkGjajINxNovzdSV6RHANm2VMOWV?9hkF|=7+ zobF|jP{>Ksb&|RM@U`XPd?d=0L)13i@!#LU5OSimHdt)`0^Rodz_Kk#Y{%gOf>P}s ze!1;_UKG1^>5&MqI~Bcz0gt68ZJufjCl0trNu4DG(go%Visgv1@-I)t7-AD+lH!zv z*VizL< zrYZ^u85!E!_?Ys;^eN}%rW1+y5J$BR6ylA0>(v69@UL(W29_HhMUx4z` ztk_VdD2<4GbG2%LmBKtP!ZiL4*de;vPP!$#KY^@-vF8N`xPyjpAKt^Yfq+`->2fF0 zY&w|_?b*}9`ip!8lgXdJ#OGElQ`?^AZE?N#w%up*k-Id0kgUFH%BRZHQI_#gsb(=w zk*hvU`4I!xA(`GgQ-f|e%y9FmaiqHesaP@AEYDF?(aTu3R^Js~2&w5pm(!D`{$*4Z z?VMDA&Y=5O<4@N$8xQEA+k_-bUxYo6nS6dB9X~5JWMGp0@H-1rZtqHU*aqAB4g|Vik1*YVa1?BGR3V(x()HuVl)Y)JT?YRYt3)>= zuH@$2V&?Rb$qo&=q`S_?;)Yh;&wP2(+v55A>AlCkOi zd1|kAF^J-;p8u`&S=-=6T>4iu#mah>G#)Q!b;2+U8VfWs@4LG_v#<~;)o0u8ut0%pcinF69J$yPp_$h(;H&TH}zveHhMrZXQXqj zl-jY+=J(fN&Z`|e&5+g4Li+$*71ut#)B}AnI|~d_F+CBzF-mhAqRaGuU2+XDLc)kG zvugGED+7kf`cu@3&}gkI+sw?TbN~SlQ!d?=!ew%@0;d92I@V*;O}Y^^%S0+#co3XQ z7%Z!ZV(+Wr1Xp&BqHd})i8Y+t6FCvlDUF*-d=JA+oJ+c6=j~1=(n9z&1X4dccPpu9 z*E`#b+1ss?MI8Pozl=qN$fM_vpO3HXW<`#Sj!{D(rO-YKhn_FMjBx0{|Ne(Wnr|Sd9Uze+WSu!1Uri~S&3ftXCip8Ma=;USvPCQtyfJ!Y zOatK@&#QnB1_LT4mRkG4u%Kj`!zarTB5&4uq{}}1PGf&^gaSoed?$K}8+W9^AMm@* z$TN);eAop+QY-^VyF?pUFpOp!p{4hsF4yAuxIKM0g&Ted(L zSm=?~caRo{PT`=TUInYW+uSwHA~ZefBE?lfG{QVFDX4@)1q&q;_E)qy1WDJZB_Ttt zndLK*zymbOA=HOK1ba?@D+;1nJ+78TUN|i>@Yvrn5Z%pjSUc4W1~NfOU;JD}RQric zo_^~am7*sYgfC24w)}pAd&jG$sFNoqZ7q?d%DmP41lI|7pmEBD$fyGuV<(a-argCb zYeH!CK-B-vtfq+^#9H#%Z>>~l*6nnzvsxjn%iY#v-`CZk+kkv;huGA#vb4*3ko(%G z=>q@TRnEK@4eN=z^@{g;j?UHIk;qrLSwo~{*8sdg+>%d%8 zPu#bBmwha;w2f9<_f2V$QabJniYf57y6d3rbg>YM!GdMX)PZdxkHEGXJHvL0#6h0r z52y#JyYE*|z{3ZXONe6HSB(IUAu@FY5mwM>Q6LYeC$61FhV;>>pp3A$D8-dQ!o{~Q zHG93+?-drl4%+z?P)0!#kx5Dp0Wmb@IPgVvQki~PdWP>RXU|>&Q{QU=zl1Kkz1h%E zrje)SZ@N-&(oB#LLg}#8$9o?zx2l-2U|9FcK{V@qnLF+$4B^K=hKo&2j$P%_-Ico*xkoA0DnRz@QV) z3Y`96u@}#O2CV&|x!GU06-LiG$M=NK38IOFCLxsK9X2$-(s!_x-dtRe%+lec8ktz<>c3V%q-+Uc^W12HaQ_D^`rj?0O4(2Gw zunJIZg*bh{!KYI+alzZbG^*r%(n;}IQv4Y(F(uf`G77}~JP6ZB@K_Q{yC|T|1`*r_ ztGj}>la%g@8{_WM0VvwuzSS^D1-^p!@Ndh$h`<&+hSi1leJ=eYkkejjipV&kUO~fL zmNhOELOEQVSK45fG#eAnoMriHP|!qD;#I`W=r67Nm*p22UX~J{!t?Zd&84JbC|lF5 z_1n5N%U(2>H$|&Tewf(xaBuD0coJra$AcQ~pL|ZjkaC`tVAG%{{Pxwf*LDF*df&X;hoD;4duHm(fv^G(qR^6SLBKGVMtRuikP#}vKkxHdtl?j{1) zv=-k9iZp#E`s8H6IBTK)G6xPjBQg?-p;r8ENk@6hr-6J052uGMxq&~-(t<}RkexwT z@clbpl@n^oZYZ5l^7;!o;RO)?IEf|x!1(}sEsipnZ89q@{&k8EyRGy4ftK^X3@mZ{h2f5%%`rC>3EgrML^y3awZn1*T+o^J>5#ZV2jCgYvc2csOCOHpCxkCU{io^GA z8ieTrV8%1h%46`-G3PTOW0GsAAi}T(JGrUAQ`aYITVa2-)*$jq2?vww$#DaizU&5T z`(3!>{D|>bquJlZePM$*EvU%Y?=tTtVvEVhC;h}}B0D|5Kfvy$4&e&_+wA(Fc4c$j zla7pQQ=RIS)`{1AqC@EEzdmd}J;q?KuT0h2*GmWXl!o|k#qE{OLyV#|2biHc&1Kz) zzj9tit`wu#3wJOBvJQ$IE{bL{v)TqePaJg=V81t3Gx?G9DlNviX_;N*?sB@HbmUGF z4L=KK5>6aV60YgtJhhuO5^=!@xN2prDQBF(T!X>gz<`<0ERAnA4X7u5z`z-`HZx$? zeW$%pAiSCb9OADjga5N6I*tQOmvtUTE!&=qe-U7g69xvGm%kw;r108|aJ*i$Byuo4 zu$j;HzzhgDZmn4qItV^_-gqsOEvFj%G!l~JCk`Ok2x~gsdmR^G{dv=Mt=D{Yln)!P zMaTFT%vardX94EqR#r(-rW;SX+fHf^kiM9vfJ2-;o@C^1s}DXlJ}8m8`SCJ`F|g#) zgu04vy)Jhb)f!Nd*pni+e21R$FwMeW3^EneULN#l9P5D zH*exkvG>oR&f9MU`eAmOZM~l<&=OaWdm0@NdouVNy_3DRXb`0G_%||0z)d(G|FYWI zdhW=KVb*H#=s;64&2p!WhON7rppDxs=3^sXPySiR3MZ?BLY!jleJjS|_SbT~GuVox&$szTWMvyQC}&#_O821ALEi ze@dW~XT@p&Lzu6Llh_Q9Go(v~%Qi3KGDch_z7>612)w}X_XxIAP$_s;^3WMU;UoN= z471PyL5&tlEW}Zw0W_8h?05}_r&}mThUk~(Rd586@eI$196?af3a=~JO9P>gUHoqM zG)9?E{v&}gFr1mSA2r#`pnF~d7eWK<256cI#zdQ;i%2kA0D+0o^07_&4QPP0FNui8 z>cCXX;b}i@lC^bEb&Z^4H1Tz(1L2b$^Vy}B+`U2))N{|D#yLUPsc?p+n!Wt&cj_=VK3(r+i- zBJK1pO!;)r$-*v1(k!)Xutq^1p$KOeUMjT<#24@5ebn7z3p;q(#+XGvK2D7r^?t;6 zusvyTH9*4h{aL4z{cUoKvvZ%g9keEN@{woWsEKA~T2+9^sqwv}(prTjkBolNk^2}P z9u?F{@^j8cMbkitkZGoQYVU*RPQ+}(aIH~On@H2^(-EO2X!c^0<#-O17|+l_*e-5z zs6I!uDeR0}t@NnDwe$L=$Cw|l5v2ewa83!e!k2Eo3N=5bo;E zCUuN4P7*#_y|`xE?xwQ=h>%1+3cD@fyROW1vbkpn&>VUJ7srDtmjp-AM^~6<(<0ae zK%&OlcQT4L1=1b?YJ<%KW|+Y%=J1ReC*g8cPENjJ#VQI zf!v?2iacW_oh={3RMU$~~dwQgQx+c{cNZ za{+o702_ff#c%O9*5CL_CMo0{1G_LL*6tN&3^nI_4F2F4*;w0*|45jPt!`!-cL3(- zYl=Og*HWee2YbK{0_!!UujBMds)2Q>+7tAN~G1G`}57c-(>Ph9~AB^LM2L zu`M&{nS^tSwv1>aTAn^!q@ZGi({ActL`3B!BoILTL;;n}ChQ0CM=C5$B&e3wd#9n$ z#(n%|nmX7$^66J73#_#)Yfecjo>_12yXu=uMjLUrq9h6*ZQ67S;o*0Z6-qQ1gCe=n zG2Xokul93QndF#CA1b)(7W}Q-Kf~E+TMxgg|%^9d2?Z5!{0>OMZ+VWn$8a~aC|&_NUH(hBwF0ul9D8KLEx)ez`Rx+ETbFf_-T zdBts96r%V{Ioog;j#D@cuVmMD)D$zWA^-_?{6iI{Rrc`o{Nw;T2*L`*62Bd)fgzLP zhbi3b=4Y7&W3}ab!F@Xh{HkmIP)FfgYVv*}7*|A#V- zM5m&r+0 zadTB2#uX=K5KW6%LPDzEWzoaf11_qA$4F|NtSVuslt+```V{n19N(MmS@I&`<#)px z&?UetiTlMsdANQWx1M?||G}lymoNtsRFo^$=2m$bOr^w7rcLu5{GjW`;P#Gl$i z&@0<7mLMq=ck{;=y9j(woQjmxr*rmP*AVA@_~wtyZ5>KsS>qzJL?--9;H^UC?4&P+ z7TaRcVy4ChYU#hvual5+-q9YQNDCUB=@XoN0Xo9EG3Pn6We^+~-TEzqBe^m$lwSCT zV(>jWs%X=EtWal0lTA)q37UJ(5*PhwqNOO8kk`(dF#U`jJnu=`f?rrR!??Hk!DVbj zHUo#J-FDX&xn?1^l)Ua3f{fC zv-`~umuTX8+QYDCs{QIG!%yTR?o-t5OMR0!zF<+@@0tXEx8aiyeXWb+fIRzgT5*3S?akAg*BaJltt52yDxQ)3%OxVbk1N5t}Liq$#A9-CM+oK( zK_{jOPjDU;vk>NGxw}sT&JiagxLz!JC{$uYKUlZ{7kdGXs&;mGj*bw2=MguiVch3* zBy9?Yl48~az3crxq3^(N^c_$)>@yZiDju*Ym$q!7H!%W^Vo;QqF zs5cyxCT17AFPa1Z-+tirwpv0N9b{(#gL(*NW{l&V#)T?HAwLyCb-6d`xYh zVL>XxEKN~(O>H~%L+d-k4%@~WO8NMCykC%Vdx(4bH_ScIc1_CN#)9S8;3TGtBH+?q zTAx&aiD84yl+RI}XtF*cT@G-)7Ptdkml1%t<$ZiF!ra8RO0L=j*%H-;MvbBVi|+y4 zpO}zbFBj75f2Hx}9Z|S#Dgb^^5_b^+x-!pPHbeQPO~MP|NV(~|?Qe1`jIA7ft00p8 z*JNZaOJ(YPmm~L2B`~T#XFI*}KKSB# zUKD9gR;0mzc$q5}>#v{|&4FN}@KI@6R!&Nw6*KxAn3R}&Kc-hA7S(5tzez>~#a~?U-sLz)9 z;rbGGF;+E!C2w*KuCx}x-pug$q}q9AAxPE>_yi zYCblj=bBpu3mr%^7xL8Iyf75=9v>4&ysB}bMQS71iKQOMs0n@!*5sh~Zs zdQ2C1I&CP5d^Uw1RRS{ts3IIE;xH1D=suNCS&|w;M_7{Jw1@Ag{VuptaX;xC=D{pO zA!Z3@Eb?00+dAG-X20edXBq$L?+>~pHbfy--=E>{RFhJWc^{ELtUhs&M1sZskLy5> z^1PnuH;Fg|iF@7>W---=XUqlJTt@MnU0^=B!~2@&BJW#dPuL1#fLMW+2|ljNJ73|g zY;NTEq5 zt!vQ{eU5dJLkg;cSSwH?p8LApeA$m52%*hATq^MQ!m&^b4v%b8KW$hV)+P=wSys9pjQ`+;Wg#(_j7Jtt@%2fU0%I-kvQ`Ye_XQgz2$lx=zl|>?UM9nr zDb45)J@i?r0n5&w7eAuse~9(&gs7NFF_0 z(r`P2L*k~<9E|=V`Y+QId4Lt0=2d-Du+U=b4B+a=&KEAl!$PF|Gb?>*d< z@8^DK0BA)J=1@l^F?1Nl)AmR4(7Sii3q>bw=l%y(-xN zw6%+<9cl)#I#i>?ri&VF?U7I`MvPcpsMe}cTM&v?t(YY-OW*r>-rs+H%*R~!b)Dll zkK;HsgBS5Y(vfr{dudQ<<0mJZuNuciNsH#SM|Z+@{3eo}5NT^6b-UwI8VDy-1y$V~ z&R<@@BY|xq-|^~v4-GgQS#u?dk3ZB+=Okrl9>hoth&IP3ymlK}Uj8c1o|gBCpE#)V zq_k>uGj_QEkK!76MRMSF|9;!m#a+k&m_j%85z z!WctbEzRsvyA&-A1pX-A5wWUO+36mpJwk$X;3t}EQ1^gvw)?yWUIvX5lB<2V1(XQS z`pi~!-X6QiJ{|Pgdknj*mvvT4A+I~G?DrQ4JJIlAnhO6N9vkJugH2OsiTpNxF%QyW9N$wQnl8`Ni%^bOAl zcEOl(uRq&)7@jj-1~3aO&<_;bu!?HIzk>$TLyDY=g6!A=1!Qa`eP`}g*zdan9eUuoM{fS$I_4t(_Xxl^lkzksMghFUB1nYsfS_Z33>JjYrm zOu`7ivcWBhuT*>V7~y)=6@b)=Sx+{As9xtlZ56V*zu|XCthXeEs9x`5zW_z6e_VH$ z@$TG1YS70|mM0U;8lVnl{qT<)c84gD5FZR~`gcJ;|DUbZkW~sQDiMvR-Q-$D8u#M+ z6T8a8%6DvHa;m>b=OH#}Iv4_x8 zT>gN0Xqsw+FA*h^DknYGjp&~p(&LlCDv~|AHY6zy%p_os0{d%_d-X{Oe6zI7NiOvI zMm%LHt%qVjlb{UppH^%P^W~h5axmM4l-Q1M^ipJdw}kr~3+h z+mB6>T$&VKlW5nU;{(cgP#GeeCc5`4RsvP;Ho2=iT#s6x*iU(n20bQ9scx|)-tk-u zR`D3~D(}3slIflMnIVqlnr~EFIl$Tfg;h)a$_un*UlRu%wV?)bT1m zCklRw8^Y)tPdV}^yMHFivoLI&0tkOjt5MemArNxY$YP)f1j1h7mgf&WQvx=hjY8NM zUq$?S#Wz|2NMF;84<%|lu8UMy%M5iHh2?9X8+2ce#R{6LnAhmwnC)#S+VDy%B-f?# z=k7hryV!dAuWTIK5@Y9_B)+EXA+^a$NCJJ8+2pqw6GH3)W1d|`KIAe`=*>570w}|a zTBm$0F~lP5`O5WAvW78=CbU;6$;U-vQJ1#jW19zGS(GY$cxbPFQ=FEnerOPRic(K? z&wJJYQ8g6_SWnTZvBK(n*G8o9XJb#JP}fo+YmQW5nWpu&wE6^x#pWUlb)S!CHTVe1 z+bnN%p9C7aFGcQ0cZqysqt!B4Z=Q@8n~VF4Kf8EF3nea3tc7VNV*MG!@;l?8o4T~1 zoWM1;^WAguPj^hLz~$wSfO2awO1n4IZ3?drP%y^#-2vmZWfG)+N~zDMZo{9UcH4@j zSs~;EJ4UnbL7<77<=!X0(VwNmN)o;iBimi8A6KVpbqt#GHBoLqcs}YQC(9Xq1ojr9 zrio_wW}xORo-G@kt%n!XU+1(rlCCC&z65u7W0f4!Ehm zUNN!T?L9U#Qch(m({yR&32xKvk)PG;zxi>Nhf-aK26IriXvu?C6j8;t#53D8p0Lrp z8n)Lpx9-K3l-FE+w!M$|Rh<0dJ@4}w#nKOgeawHZhVE_tdVjvMgT>{C^ycF?4=Js; zs2pj5pCJgVq(ohk$u$Fd_31+CtWL{x+KYeQk-iPdyZ<3>Pn~bSupq@kiVFO*;pydY zMIFmCHv9s-V9C7Js9vD zu>-NAsR{fpblu6-j6&$#ZI8*ZhI{HnB>KIRc-l^{{cGG4k>sVve;ss&mBJB+PMxZZ zbyOnH%?L8yBOxfi3mP&MIZlQ6LY$U*6K9lmS2aluor#X*lfXyZjIBp=<$2bR&7jh7 z)dQ2Le$k`39S!(70L;W~PfsQwqoIuvZk}B95%C8jLl=l>@w?r7}(bXblVRdi>=%j6fnUpQVHY}`lZD38}QBSE3 zn^qS-f$;*Not2;{RZgQSZ(a&ZKTgvVtsOjTadvKm{L=Hq#8pFkaUP?~d_noxhh+^l zB@%Llue9W`%L+w(PHt})C_9mIpffEemfCeL$Q+4f?{FT`;*I)cD27E5D2u<#tg<7H zeBTyKO(o~fY^IbsbpQOzU*c9@0ITbAex*R6Uc5Qc+Z&-3 zJ5Tz^osa|X&umGxIrZE$YnS5r6KOT%460gS1x(M9dFTyPBYY!Aajj=h5DrhoG*nH) z-kfEp5rc>FzDoB{Vp+VF>(R?ImBDUz`VX21XH21zZg<)~;>XX5jxU+?3(@;5W!BU^ z5V%zyK;IQ}{@GkWURI7P)T(6l8(eKM(TKYTyyWfogc0v=)pk~=kx&ctub2q2WB$?x2I;`ctom< zLMf%QM%YSc&|v93B+I*zR@fINFT2f{DCAHeN4y2b;V9(;NYl#e(4D>kme6# z^+)ZlojJ+TAM9Af9r~26RJD#XNMkk0dwP~rC@{R#*2TK)JaojBPK!V3fm|#jgpt9C zJ53&fYZdjEd=u*-6;de6)eOz78goKl2GraF{k;|gyyqQDeo4NN=_%C_H&Fbp9z&oR ze7mjk769IiJs;^JY6Pxg#*#fOSDmONbiXD^?o5z!J!Qj+^&j9yo1w>H{C=d-`qIyV zA#2-Hu7CC_YvYOA@phme2^MF*6M+Zg7c|Q=Y%8IXix07ZOtmt=`OB7Ind+%n*}e9} zt^&9N!K1CH*>BJzJcSQ1@I&=TyqoogPq1Czw+YS|VPi@Up}5 ze0Q8>ZUWUveHHWY%t7~;Gl92F?6s?QOeTdwFRnRJLnSk1aN#7jV|nzI;Kf?vk#t^3 zlo{|->JL@7Y19bLn*GI^FRDm9Nq&9gazJj}CE#Ji*OrBen4o$xIF_gJQ7ao9$Nz}loYBrTyPpv%mW8Cz?h+ZUl|-`1UV7!#oxGXt2EGH)Wajz35buUYE2|B-OWFWG%Jmhs-pQ;+{utdm zDVR;vWv)6IfcyS&rBKn`~kV6Uxcs!Ci_2>en> zu$in~+VK9HQIK@##Su+5zDulS1=*| zYM0N^e6XypxTR~(vgRGS2JIFFX63o>o{f$$T2oW%XOoxQ6S|V~tIc!;eZ@^i&QcH~ zX9@e%jfDEvx2aQ2|EOQz==@O)U>i!YZfz0kaFxX<(ck!m?!07b_^EXNCy7-Tb@9|X zItrXkMu=gEwIt0uzMD@D%fy^@xkUC;;xw-Z>&sNZ71nAO%@i=Bu~^jPSfI2y!RPKu zPJ-t|qtnf_qk<|6Exw7i>lVRg-t#>Q9;c~*!E)bW$Ukan1GcX-%Uh2Oa^et=VxB0I zV&s65#<1YpNwIrSbMfUL&_&UMvQCevQ#+cuFtb*xN&JDlgud-5HmK<>$hjwu|c*}}I0b^eq*MgGX8=R#BIalBiR z2{yT}+P)}f8l=4Ki0Ma~BH?gKNKefz3*wFtRbjIyQt!#M-{UmV5T?9d&J3;T6lSl0 z@tdwP3|oiWFCIl2t9^Uuz*w@q@lMTdP}p98nLl`zhv&`3wc6zwb1f}K2>v!M{8TRT zrT$o^v8BZPW`w$B8Jkcww=9Oz0VnV4esKDeLnV;qic*V0CNx`)E!O6gM2b+$v0J5I z)kdE5Po%tX+LSU^809*ZOO(_KgfG{(wLqlqUDM-#9f$fL5?Nxsd{mri+8o22HM}1< z;I8&D|L<|jheX$u%D%bC&an8oczCXnhR%=Knw)mYq!<2ou`SGum~qmJhIlhk7Wv6= zqSV}T&xIFj<(;G#3JQa_?JaxWFM}@=8E^Y0QBhHGMrgNF8mI z*!Zq2L}mzo$tV(H^Em5~3EAnXmN85nx+*1!9{r{et4931pkBv!=jLjTyd19Vqg%H%S|t~~#jKH* zf8o`-E=q9{CjJi07iZqLk{uOlK2$CQTVEvP0~nDW?4Z2jX?hUse%!`?;h5fvr$>|s;PAklM*t$&+p0y z%yfVF8IBd3cTXs^IGGtm2JUv~o@Ki54s6~n{U2V3a9$Q63s+2K^GrVD`BDBAn9x=fz! z6s4kda8Ko>AmWUmI-VS+%jBsS%ubNT(Pr}SP#2?85plk{V$SX@^|1{U1)(& zk2Tgt?)`!vcE;)aZaGU!1!So_?74ZbMiKXLb~oQaC({)aJTftaF-au?aZTZ{r0&}~ z0W0gU5?f&zarGx!;`P9G*b7+2mS0;D+C=mSeu(6|Fm}3|!^eBy($e+`FcCBwx3P6d zi>sm~@iC|51&G>-S2OGuwLFDbbPQ`*k^|CSiGE`8F;5#!F`a(~o2cm>Zy%@lYwa<3uKDx08N#aiW&e7V%Hmu$p`7OyGyONG@O@0$f5SpEcJZU_+3n<4c{~ zT<2fQTd?U5?oOE{R}ruqr*bV^jPeqha>wa~Zy=1|B2AskmC_=b0YU`2@Sl%fq8AHv za^~|tP3wCcWc4V^FZxM90hswcZLZFg(1+wO4<+4nXdz$U?uz z(YP88(93ZC74LOgmfI?FHG#wfm++M}LI#@nIWvVSal`$T1wR%SX)&J;0Y0sXa;j4Z z8M~#!#Ba46CRU$I3W$Z-;TmUxiMVPy8Jvq(=_W!vBg$$w>}D=vlS&vyeg~i<8vmq# zt+uz_JCDnueCJ`4?#fWfQbv^lJHi*phPs6;!v)h0vAm9Gxp%R$E3qYRiz%|t!pRv1 zqS*i90!aCS;Ym9dszY_sWkuSm1Bu_|i$5n8<>M}}raUtp(fg?EjyO0 z-nrtt?6$f0GjZ1^DSlgvF=kO_MzM-ZkMGPy(v%qTE-qAe&iwkmR?B^T$g$@=@dwY` zh<;NVPs3H9D?R5|n05uN?uTR%Upetu-|tXJOMx@Q#(|Wa5|E5(iIXK!UL;J>Yfc&4 zg^&XtAIP%)Fyj8&!bV0`E>7DcEYdO`rFd9eA$#YlLy9>^s>!(Z^7?hb5OQF*S9f12 z$hq_r-(tqHRX7nRUmcn7YnY<6-Bu_{Yk}1DuM?*0dsn%Jw2tpB%dIKpJ^$XXxe_~( zs~J-H7RpmJIwPkI3_LnmMr>>y`F~&JYWxA{x=R>2S;5{E3tN$rr~_zqOR;x3qY7~( z0RQlT6NsUSsO&!t%j6BlEOZT(icnfr$A7=gJKE#ID9|}_?VBb!at;JGm3>A^IB?8; zDLWwudQ2w9iJkq$xRs+QsFi`0wR@zzpWDqM?u`@kUBKOsw=5cA(FeO>%%MwZIOxtv zDojWOMvi<1AMbty;PM2Cp*d21>wq&NOP^#pm7W#CHR4K6W;XLWw_E7<VB z%s$td;0^6;j&JD)Y$DFzwftW8;LDt-CI~XKxr^s|Vl2B?2pRyMPw6@=kL^O+YHfC& z->U4}hq&a&HNX;yvHd=c+v+3c1R@iS3RsJt4CBKYcO>rLxy@;6S$ZN{>^e14tViF6 zps&{o$hq0mX(K)qfq@N8+^PO@!+@Hfl{QK&F4DX zV&F@fgp`e)Cutp)O4IR)RyD#MOR@8(M{)~7sTIcn?DHVyg}ta9qk-L^t@u#}3sIg^ zUnb6^Uz7dQ39WA$0j$q?YZj~wKt*-+3yuffu$%2^16o(ut@1hM9-|Lq4``XFG&4Nr z3+$1gc~m&rf7O1D`{>1Agvn$xi+KqBEm^pBu}$L=Dh9w)XV2Ca)0;cheh7ZNcQ*7j z=1Xrff%x>7^$d^b$FV~rnIFmM^$EUwjs<|Dtu@t<$Eq2>$xN8MNjuf zeQBNy)VnuF?U|EOjEs9U2Hx68*|t9STEObJM$4Jo-^~kp-^>U!r>tGcg=*+1_k8!u zW$-kv{kroSzI;VvVbDP4g-6Rk^Lt@d72CCli7U)sR!_xi#l@7e>C`i774_*2m3f0+ z#Yw&jfj)Z=Nyr`Piae7t`bu%{Ee4;+_w6yP@s5M{jJpX(_3ua+mpQ6){t2RF%$V!B zA~bIp2v;ih2#KvO1`juci8I~4X7xPXSg`!ApfZTLn`EZGY>$`l~4td3r+n%=SGWOA6KJ2<6OI@d{ zTtl>~MMKpttFH-$hF-u8e*gEA^eC#vDGFR=e9kmtcfmR0Ms=2=YI}=ZMN@rJw-8{) zvFphkUcHj|L>DWow3*-taSC~zE>7Hl2jVq#z9r*dfA4RcOW%09_kOth;Ae!J^e1t~ z5*w)r1ij_{{jMALwK3vXv-y^Im=s->crrDbxsMv|H^e69R)Qz49X(abhtfwm0dIg- z*e>hQOJ4u@*6C<_yrJ}9WWti$a2<*MCaeK^E|@3jcjqsDcwgZK1OciL7UwizOgm^&BrLYRz*bl_j%UXt(;$&Ll=Q{n64b3}Tiwv8M>{SbVMN2nSDL)4l+`%H0FYSNQL7z4{4D=@oi zD$Z%R_fKu#11|I7K(``-dlbL#;fFO|$$=GorVO)1kA1p&wack&A*+s*8m^{|nvleX zCSGE4*o+WYdDE3=xeg~Sfw9;^E>heNhYxn!g(MZ@Yw+~jo`(mS`kzcApRj%H?C1kFUMc&%yEWnr7rBfM5*TT>Nre>BA2kV zrN4a1g3;_IH>zui@MwA&7D5n9b!pZAuy|5xmsYek5?@0lVA8X{{o{s|%kYym9(NOJ z=Q1=M>oZ-#Hu9Zq@9|FNb|P&R3qCF8Y~f8AKNw(XAf7EV3a-1w7d0QH;-^jjMumlKz4canW39)88=eR>qvz8iSG@7VC*&nUybb4jvd0NhKMQ!t^ ztzdJU%J~mhJqCAwfAIXlPOZ;5^cT9iaOchldndMx2r<^{=x-59@#sOK#y%)<`U3Ub zE|0dYy6H*_WbK{)caPK27y+#ER;2Nrka$#j?2=1|Y(Uv!t+BY&%4qk4$d&hR$;l}m z1qhH!9$#fj%uKC8bxUNFbiNd$_s{pn6QEkHrtRhWTfLVLT)*)#mLfC$xq}UU-AV{Q zWYca@{wHc-No`)@*`=V19whBc7Vofrhz&wK&MqXm;nqz;V4BBRKq|aFf^k+)oj;Tu zX5RxUt9%giTdM_fy9b&qip$}4#~sGloAak*(#~`T^$`?jZBZbod9qTRy}HE4djo(O zx|Rh=!f|xer+20RgUW{~3?87kCkO%7fgUY3ciQecoFmol8oaBAK%>9u&gokIo zlh#T474O|vb*o%eRixN?1^0?}B_-wy$)L?r}Zm@At;HE;ZY%~J@)jbd% z-0|!CBRlg>eYkx?m2)U%`FDQ@gwZXN{|;u8I<7hNV>CiFOi9=~r=r#qAI5 zgY&K2vyi7h>)`T}zN7bN(d+^Wr#t4xlXogc?omo#gp4bBoS-75s?E`C7D39_9r4dq zJG{Hc2@PDHQ<58C73^)YIy~)rXM>-5Pd(0VfZGGPw+J4A%6qYl3IJ>+B z(yiVi+SO5(VzY_X@qofM{mio`prE`@HWM{ilhU(nnt`25g}eTd4`s5xq97lj0Z+4- zwoncYWUY^QJ6xTO%_OGVzQs97n;z!C&otU+&;(3bOV@SEvY)UqdYj;z%~s}iy;6Jr z0Y#Bf-T5p}=sHw(KI_%_mye^gB^Lq8iCtS=fArZ*H-5i7P4>u)a37q2-Os3p zm_DPLNHHD^kFQTcxg%hPm)+kR0zc1_8=3x$9VoY{mN*l!vgWX267EByFNp70ZD zjKgs}ULpjYle?+#R8dfuestDJ^X#5zcWKFCxM*QJFE5cjw@laqa7!|?qzyk9_oizC zTjP=khgo(uA2COdJV89D)RVq$l!Gy8V`jGzD9Hm_n`&C5aqlO3M<~t^Om^*$7NS{_ zbqK8x{^Q^sR(_r8d0m8RKFfSXiKu57^{99>F$|gBJ}yB)b0JskcKfAe=G=DA_iF~X zJRQf=2!0jSVJWZ-QpgS~=`u5~_s5&5V{0K6ckRPAdwB|yogVc%t!EzER6%Ig2G{Bj zGqnLuVMy9H*?$yK|FP{Gc*FMf+_n1$N7Tx5F9Omn(jhp}b5e6rdVacm*~{zBnGTQN8`9Okm~#E>?Mlbux>M<{%Y~oLLC+0_$XSvZyP=a~GrQWKTus#0vsYBo ziW4_0?W-^mxnL{D`K_@39J~2Vu-eX#*=SV6NXe~w>es6tOb6G&M(&JB0TOjYTH(TV4K@FJ)^%}bwUAN;jb@G& zdinv!%&v5K1K@4O%h20}rT*vM8VB>DWj(Ky?DzlhD^Y z9uBUnBleFdW*BD{EX}f!?srkb2FBBDk4bCH}jV_L%6KmSDhZx5_1KU*YB}i z2|7R$h#1wZrEh9ulK|y1?upK{e7zZ-_$4X#ryI(#f|(Ziwt01-v#HFC3yeMDY$MOb!YN~x6Ah97iZzF`gfHcjysBzmKTUNln#et(Ap zTU>rZOa6#V+m<2jJPYQ|-w07zs}rd5C3;Kb>+;qJM^nJhA9|*V$N8V|F`osNoN9c_ zR5lbqb$hR3gp7?pTcw<#xC@s{Li+S%;G*4Mkex6=V65E%$aiYm8%~^a^?Txf#bKw^ z_JdZ4fpR(JWMf`DSjO)J!@;juGk1zU4r?N1xEy-`kp9WY_1M?1{gI2c3OmA+QjDMM z&W-w})Hjudi7}UQ+~4U1_>PofBHT1$(Q2X`ExQto{uFRO$(rm70k%Y>>XR9)C_5Q2 z`fd!9kHERqot4l<7kR zpe{F1+u_VIVz+6~0Z=*CLj24@ z@tj2~cqoEYt<&!gyUKf~N$Y6R%10 z_yaR3QBVubx}V!6B9-|=l8y9LM*@@f_y!XK=a2j|8F#7fV7b(?yO=^BFPRpv*dC^i z^i-m&)>0FkwKwWkCbR$=B1`|j5@Fz!BX33ld>!KlBgr?rZ@;_2`&`8QL7-p`#d?r; z2&1Ye+8qMqIE>xi#Wkhn-bQb#`L2bMY+8;fSL7y003iEu#<9ThAhw@0*qCl^*cjrm(7&5l@GOsdya5 zgqMTSCgVB(8nj`7C)bytg_3#KDa3`!&3|Kd)k}S~DX*82FJm8_r|$=`Iq?{hKal+3 zHIoOBc}Kc*Sec3P^43)>cur~~5FC|M)0;GKh&b4X){p1I=G3elTnqZ`z$3V_8(BadNC=TO)P2DcF z%X=1FebDlSQ1i9Q`{GuqUzm}7Gr*3|KAhE=J`9*4++pe-+E9p-?+aVtxVR?y*24V_ zair@y+Zso$FwyTjzCne zyMQa$pMT?9oqSIu2pZfUd=?jwQWvJUy@3&MxYK#ykQgKZ_iQYdeTLD8CZ09S$* ziExA*WXDOk`x#Yn`G!&9aZ){vd8aqH$??K`KjGaiMNeZW9^B)_WrjrmD6%d`t377E(dw zAIiWcsW*&UAv+b}b?FaujV{HqXnCVEFwD z@tC{oX8(G-3K+7bsLv1n<-ju!K^K~9*a~&Z#~Vl5zd)*L@!-?bFfa&O66tSL$Tl~E zUU3J5D7*T9<;^=CAO*l2uMy?8Fzo00U1rJB=unT(#?}(av(k=T5og}JM4?q~l zT;>d6ZmACZmbUHlWQzlu9M<{3CS>xhMO^!!FbBY_u&fFQ{8k%4jp7@K6HMxT6}ev) zk^wh}@K#vnuK%tub6Hg5Rj_f?PuMKaQ5_i%F@8R1GzkxQn(Oq4oV7&^BjYs#vESEF zUv{!9Z8H)fYt{RaBaRmUTcpz4SN^jnA&Xb?=G%1K{RzWjCMIB`PDi3j^&glFzoOsv zd_^w}D8wu}k@zewE~^suI8bT%l@^APkL2F+EYDW`@HhD^#hMnnO{>SLj>+^Iln-QJ zTQ{7v$sTjJGtw0i5>%y_QqKSvceP&MPZtQti?N<=;-{erbqO`7p}a3OwjE}5a0mKw zFuI%i-c)@R?h+%yLus&}wVOR)hjs;R{dglLQe3@sibG|nRIB3aYJtm6_-9zG-a4R;79etsn3gPsaxfDc1Z0d|`UZ=W4H*H}qv>e4uA+J;JNB?r#%YT5NUj0hs|Nvgn+| z1DBj@@hDpWdHG$1zCrESDAS1yBokaR$GpAt_~qxwsKLH=-l@T?mG_27DaMN#$BL(A z@aHr8Z6@M$rS0ym_}&pPTx9Yf2Wj@XAr9maO<2dGmhzO=DdgfWFOV0Q^`*^f zt>?cw;y0k2jr)}L(#{%90A?@+BiWrGJ9WjDqW5fE@}w`!-UHe-Ppdf{5!Cu4AVA1l zgK=&Y2JsIhv+irT62e1}F(Onj7S5+tj( zG(&*(b6>KWk)UCT$Y}iB%fa~AKpu2uOUGF8xRh8?)tNzb6sxahuN@T@g@h=xJJF{K z5EU>;=a8i6Czk6v&bKSn0uo3^LH$O+f=hq)5x>i!ZbLXKe3Vi!5gg6T0ebyFNGo;i zEanFfPnQSo8*e|Mewv<7mZWbt> zqZgmw(bVt+&EJwq|4;AqkG0GN85hY57kpO7NpNG%^IDB-2dVWv%2coyzQ$xSrXrrN z)ap?qa@TXd?BSKeruv_h#LL&9@K;%$@Ml+yF3vZ6e!b6&w{MhE*?0TB$cuq8w*Y?W61ne9P<4wJuL((;0OAM1m8UpEBHE?|D*E1B~Wj zrr=X~T7awX(0XuA`OhUI_3}*Cmw<-F?9TGLDOt!;7op#WPGJo*ZNERCO=V@;yJ7I~ly2c_2_5MpsKi=3NfqTvfwVaxq%g+lZpy zGpW?w@hU;m!(70Z9a(B~eGb!eEd#oNuJRUnk1p&A{~;ao3ES_w6T0xxi2Pa?g1qG{ zs^NMeob>3^M16$PtI!W-?8NGo*wNCy=GMd)j{W>WqrOjN9qRSR&)%nMzV!fE8`Os) z{!P9An4SIyrpmh>Wya_rIO}e?c8Q55_}cIu^IpcCx@1quUA<{gCG2sw|7~wWH}&2; zHAdG<#*ftp8hgES7}!m8UyXTx)#VSGjj2oe%MO(a_}VcL6Lmq6Wx#NHDyd*pIkLL1 z|2o`+78A%2cY_jFMiM3}I(2|6uca~7H|F(H2@M1r7`YxNzgQU5B184e6RkB^`t2Zp zGu=bXybbShUa~wlsuX+P6*tfEXV7Ckt9bFS%pNCT*wCYu4NY z`d$Bh4$a<(%@0TJeW4Z}cDGzjdD|>bdn8z!9&|vJ@~z+wX8sqwzq2FiQ7q(f9u#nb zw|oR7Ya7Vxx>X5Cl& z0T9@HwO<|?SH6oiQRKXNIPRSp*LUT*gNnOi7N&#)mtkQ`ZFE@v7>$$P7RQAo7P948AnkOS_0 zEA1?m3v2(IDcLb{Fg*KYVjHt>HCba|OodCxLJzuh+?W@9^5#p2qg8tG;xlhvr@hV; zuUiq+=E1EZ5ga+Ab8h6?di`6unvAC$!*t*6Gz@laoawi8Vm248XEsFsxyJI`>g=Uc zFI{(xt~!88_O}c)SSF1%y!9OO)Bk7_JiHfI7eD+T=WJRogU1Ew8ztE7H()Dk>N~6o z!r{>}P3hndq=UP($Bt9UKiZz#0{@&JZrPi?q7Yj4z^R_%z2x8Gy=YGbLOAIg?2_O! zD0{BO^5IYGLXr7+p6MZzh`V}!ml{xBwha`dq13LunWZn(Nh57b=-Ai_Il5l-rht!5 zSgFnv4cK9?O>aX1YSkLT*Q`>8vwkU0rwcN6=2YxFx3Qm4F1N}|h~lUoMG58+?7~U9 zVqzHFpt`6=D+K(e?pq-t*&nca#$Z<;8~rHzWFQdo8YXKf8RkK)%isQqc^09R7yo^D z{DC5YBa{|DV_%>cE_2HNI|1-~r%I1S?OHb(jYm0tbL?h&#}}!@V5>!>$U4rS+G9L- z$ehEA93wAh`F8)$+l;imU~uENeUY)x{pbgR*GGq~CkvR((<~h|qU=xR7G4?Y&q2pL z;E*3m?B@l!QiqIRw7%>9=d9j1oqFw*Jw)Hw`$iW~B_*Ox0fU7alZXkW$GQ+e1Kv?k zaDIT92OOmsuVk^Cie%(BLD5EFByQF;eNsZcCfQQj_}1jJ`eS8fLp|rLD6 zWUkaFKjj4ODK+>_7IPb!QoAF(oE_g4mq;A8^~42{l^D_Vcf_zzV4;3h_RlH6-x`KY&Ovz1_Zt5C2ep`|Z*4&+1bBgAQO) z9XB098+6C*f3us^w77gibD2*3+p_Msw!i*=o_#oz;`1$E{u%BYC4YKX$E;T6nS&_a zNSksdCmKAShr-DB@H>aR+Yd_-gPir79KD*G*QkT{2^?qI6YorQs$HTBTj{+qim*&{ zS@`xB*soRE2NSeRynhGccjCmXW#(OyV{M?=E^^Kl5NQlGpw;PcdPicA3>A@2AwG0u zI7Mt$Y&dyJv(G1G%(tA45=1_aI>b)PvNWFM2Y~sNuj4gj))hfpAF~OsV!IHT>t+kZ zW&I3B=2c9_;|=sihwV#;kYr;s3fiR-l2s z%&I=#`FF~|_wDTvA_JVFQXq019QwS9+04{M91O~O%35+Euj~XrO_JTbzE^SiVFa+o zgYLA#!k1_Ycg>BEKUD=?K9<}~DaW-zf zcV#)nPJ}l)9~4>Mb?(dRcjUa{_3cvIL+x2HUsV2D{@}z|6oO&UuJetGsF4)YGk>=5 zHcrDHW))C%%bA59p(K_P`9R`TF<+>KjmFX7Oc~8asM+pA{#{l3wyc|Q35l}HqUkyB zZ9J<)^9tzzzExYfqTKXuu$Qcz`yW~AlgNJzjsnJqix>Ab-kn@+ND<%oJ`e1k< zvqihFRz33s{mqJdUy0k!WgCd#r(F$%m~sW?3xf}PW7-7XLxz~(Q{lNcqu_dIsuU?i z<7&i%JA#{ET`#9KUW`2R4+)@hnmyNiI$9bJW74t1xbcdX{CslH?{MiEQv|b;q}xNc zJ6!e>v*}C)nh_VUOI0y!(1g_^< z%b1GJJ>F?%N(vMtGzIG06xH&~H=5n>w=d358BekJ$;tyS&&ejG5pN8^9CTX=(u??~ z2(r2pg#=l&Q$SbSCEj0-nU_y2kcFP3jmO4UNK^GEdBPjSV$D0|PBe6gt@(m3=GFOC zV0E70ZP_F30!kl2xkAr0ozP01@O9jCS8vGtsw4-kp~jD?Y5c8uX6IYP394SeBO~0- zb8Nb=CDsR%;7K?yoJGKehTheqgO4MW^Aq%6QF3_Tjz5-gsA6)Wi+)agAxp@;=+b{r zOI>1C{fup5WZ7R$b;My4A@FmUgpDmVb2M|2fbW9oxKB zK?wuecx_gHOpQvJ^u^7@{mJVF82M*Ul=>`~!&B(rA3X3z7pi5KxkOsqriv5U+NxPT z`5H)uGhZkff(Gi=avGgVEIb7lW&UL}x!sqeb;FQ4%IWndQ9Y*p!DENhvUeSFe(@K+ zSbOY$!U1Hq+7)+3Cl}}=qT~iYXFZaF<9Da(lYOW|#Fj=!S{)oY+Y_LQW6^4l{NQ(f^r)>r zZLJ+>02XK?-W8X*B!Q<%UFYVf(R)92>eHPVDridX%;xl52+JQF<#cZy60im_#n%9)fk?5@_;I!X&K}o@if-UH_Z`zDp)596GG1Jv!eo4U zGm2Hd{duD{dllRXa$Ag<%ECN&p7F1a>A!}}RcxT^%Ztq%dGD6LE?)D8wHIlEy65WL zKt_Prw!;fo^b3>mrZG^Lp)5sRw+Rqy5Txxam~)}#Vu>2?8It8|e<<~sR66reeL=K` z`^sE8sW7Mf_k_kfh_zD=o#XZkagA=992Nesq{Ta7o^Ye@LRv+UeC*O~X#y?-mA`#M zDx*E7kJ=djLQ8J5@E3usHD~w@WwQ@c{vTCu85CC+bqOQEg9Rs8f;++83GVI?g1ZOz z5Fki`Yd7wVyF-FQ93w^Mfi1iYlt_J!kJFdoMrX3-2QgaMvXY1Nv|~|) zGq~*biGS}T!>M&Y`qL!vq3^+~_5Mw=g!}nj^*ESGDO*YGlNEc3`4m>331OKBf?8rr zlTM7cQe`duRr$vYCXlt3sIB(?&DPK6DXr|er``Wa$^Ji;?J6JP;-7k8+y?yz5XxVB zT3RG=eem>UKR+IPZnoK6=5fvDpj&vIT8L+w{G(329I)Ui`xm}Y9tmpb%WyzxoeT>~ zyHjS=j(A%unR`4B3dJ41P#3FgJsUKbv-~Gd3a5}&*2Adp@74xjt7gKyxw}elWfAnL;Ee&JpLqD)p zg@xl3UdC69`6^ov64In{=qJr>zJXDmIR7JS^Q#1_$nBh<b9*wswXq}HCT=m2sf2W!Uv|nlI#Fj<5_F{rh)p}FdIMQ z8LbN(l1wqlb<0u^R@jIks((XMPBR%u5}uJl?O!&oWV1~a>32EI_kV88{~thI#UMNg zGk8z|i+OX9eKxqRkB)pS42Fxc3{9+Seg?a`?KhoNUg@~}`23);k%wV$8D*sb0y_pwQXiel`%uoG{&E2NkO7!m3%%LR1Qltp?pW4 zq<}Xl$uv_AvqKjLuFBmU{Q*}W5?z8H=UMsM){wHBy7_eDMk|Xp)#Or5$^&ayv~^B_ z3G;SAQsT81@=(K$<)igN=SO$D!s5P;9Mt~2=a&0R3{B6&h9hA8+sJ2ko#Qr>yYP$P zC?(Sc&*7BmwS1juamJWeL$x7B^s=lBi_g%fs#;q(9&-M~GbXKa1Gmfv* z!#B?|B2Ci6MfTj#Oh?QjZ~7@CWOLqD%k_JBb*hmH^&~>Oq)zZtX zXND7n8!@NSCmCu+?VyTdq;g%0a2Y*Kq)pyJf8co+^0G7kRTb{OC z#FJn9<_A@Ls(F@P0yaXaauZ?H7X%<{&Dw&Z65OvmM}2l9u5Xs>YBY))#2^h85?L7e zpJSP_-roOTh~vMPIn9dzxJu96o#7e!aevI|(bo^?v*ep)ng9D4?w%N;TC(Kk6g92z zFbUW;TPwJJmwiNask+{2&I+3}yn7S5YOmS4GwVide|!yPWL7Fvnu@!_SNe578BkAl zA!9neYh`>{6tMfZhHmHm)Htq13 zX z1}B13aVK4_Xz=xQ<wFccnpf8Z^#ndyO8E-Cd zd1F-uM}UF&ag%Cf$9o5y_gb*)pTmdTkBmV?zWZUe{l$B}FuP7mTDYWsg@KkhEit6; zNCVMHD4K-6|h3~rTk8HRqXT=;;V-J+dcYa@%YMZoB zk;IFj`}guJKH57{%FrgWzE?Kw%mRo1b7Lw6(*!G&y|$J=I6-e)XdZ9MJR0)|45lM?qLqc6dfkJ`R3;o#jVNa)|4cY zvf-2+1n;JlB%WDTBOw&-*-hY2^D)(IsA2$h_Ij`?Tg1;}y`TU1-y#^+Vh5|5W4XM? zYEhY2v~Iy*IiK%!0K3{fR9d8Jit!vH2%9NMU|9(O3Q(Enyle=mL8O2~kv@us{$^-_ zdQQNYyW`S?<#aXg^Llc*$kJzX_xM)z9eJ+6$CuB!jUtVUKA_qSO@k7xqur_N(aZV# z@a)ZwA&Pi^K51=D`G^d~vC@baQqK&dTS$QH;@oIHwNmaJGH=1#N2@$zzeQhv_OXXl z^b%jbqOshYH^EZ@*dmYyVCZrtJhATwH%_p-G*Uc)lOFAlX--c`#b5-NG^*cn6qp$HkCgb&;|^1@n|*o_ za*(#%hdAKJ@BVNdn9!8M^Q#9d+byb&@tpjkv$J6}6XOen{=#f~2hEskjq)vyM*2E< zptlJu^iYm4w?RsXDC8VD9H!x0A87qaO1m`dOSX^`U|2zs08)N}K$H+3|B$uG` zATJN(?Gj0HF++vD+@a!+*~u;?cKB2m8P%fAX3+u80xwi@L<{Qaj^t75*OyVO@Y!<2`CV!#n-CZ!}Ek`ITFjxE948#h^j6MggO$x zvwa}l7MMh^gNZW1g1w9ueU}>vgK6EwZMy_=q_2Swp8*ceD=IJP%N_n3cpCqy;0%jo z>K{@E5g|-Eoac)ZfSY+!<4QV&$w8HfL6Wk;9L3oE&*p4d=@zN$H5ZS9VC)4m&*7g8 z`ulVoD)w4F$4~pi#*`n!9)mF~L(YqQ0Z+pN=b>R8v z&@LN%Jl^5{*BZFjfONIdM<_!UTOgc*I|3*>6N0yN1Sc7gTBmn2wY^oFE2$?|U))gG zM;LJJjw~C3J%!dkK1C&)Pw78ZmQb+b5LwY*Gk7_6HiSg!uwLMr(U$d#H0|TA?9M8h zyR>YCJ)2pupDmafDxcI)TtnS7;(5QzaTk97wwxYT`dL5M^I-Vit&;J$c`Gnaeb(l< zD%-o16GL-o@E)7AZHZG}=_shHyo7Bq2G!h*(oYRZ%EgL-<@N^U_Q& zeIV91h@+u&geq_g5oy(f)W<|gZX61b{LwQXffw{P`A1=ybSOi>_Omozks6n-8)#1W7UGS##Dh*1TYGz! z_Pw#W-(Ob=t|Vypao4pgM)SD<35Pi9?es1JE*mPZt5# z4T(>Fr$)ocqA1D3YNP(Si89~gm%MC>?@>LCE%!s#7HG zG+nKYtZLX63M0Q4dHa&At2HOmG?s1|9P=Pnu?3Cnww^0B2iA6r*CVn7gsr&tTyM$f z%|07c{UCqn!~OSlyxjN51u;v3q$09&ws~ERMimGS%U49J92kmMk&$l1{R8 z2)2zBLNQgz*6yH}1kM6)KjBt_XSQ8DI>^Oz2|ji$BBIAlfVd&yj}txW&JgW;Pk*+w zBK=$cp#3-AJqlxbU)xr@nvlxT8@!SMO$s|L;(e?hYY5%p4QQ)CIqUQFptn2)x zJ2|(P*j+&gm4xThdgA}olY>D9i(P92M4bZir<=NxGkJ`wZ38dHh`et4d$t}@%tde7 z3D*wAEz!S`h(@^*4UVnr)kf480xNCa?3UwM3%8Dt2!+Mx(T269?BFruO~JV{v^C0; zvTQgt`hU0cTDJn+!jkwUArv;r8W4Ytw;=FWgZ8w+!gtZ{K5`7Fk$Ga13rg?vImazf zBz#8H$?GP?eCqAUk7JCM96&-_FbYUM{DX9rbm_qzBwqB8 zt8A23E#8Yz{1eGx7C5M2O`Ph4vmwiZ_}8asHmJWmLpb~$oWG=mvF_w5<#8V5@TJ*w zh8G?Lx7Isc8`ebvS45szM6noN=`^m>XGte%+|5n|XCY)e5dOZcKTa^s&oOiHlkLX) zczKKB5=XoKa9(wzMp^RF#Qq{1Ac)X`vXTF+ovY*vQc?vdm}Fe`uvxbn8^a~?oxAsP z@s8MR$jr+#*pov>owC}TJ5IAQPi8tL3x@<@97;mwDa>1O=I&H$p~)m-g!3jK1xtWG z?wM>C@}gi}OZn!~umUn+2(dqc7WOR?IH|1~=HtW&il$=bRTMLzhnJ{fSa@$IZPeIY z_}EIgCY=}en;(&;ViXNG6nd6zlBPZ%TWcX=^etY6=|dWz&wpNx-OE-$`A01mFK;!E zo_`*{X^2NI%2m#p<{)4dyM?%pwEXEZRzvaR4$c^ecraI>12nj?p-t~=uHm|`w+${2 z#vVXK)5HK}kdk(}PGJ-t{S**X377;8^olF3mu9Xb29N>^m3e}-I^P?6HQaKEPJqD!MY8KAG$>5MAe4zvy;x;aBd}EZ0QUu&BQn$MYC$R%D2#bL9ho8W zVB_$#P}`y{v*^3nX?NJ8k|C^Pgzd?5Ci@d5$Y>M$^m8ofWZP+H<^@_5sR9mbv$%TF zxjnose)0?`NJDTwib2^jpML1(H}U8HT0OD|q=GPox&gSEZZwGNkv-YrTlDo37y|K? z=4Lny0f*JUH=w~YKlr0#rh0Hj|4q9?{cSw@jNS*LM&Sh&Qn_yx{RJVW>Qu%vK2zeu z47c%L=x4}ho@q&+mb;s6-6Zz%=iT0=7Ud#6iEAN97UqVio)u|efb+`Sxerj%G&a}g z_}?}-4(+>VSHrH98&Qfk4ik6Oij>7BSDMz8Vz9OqH`=evYG}g^#xMa_6FrY7?lf}MO#{l)>le;~UR^uVr z*=VyaqoCP`x@|X`m665Qcj@>La;x5OmBE7$S6>oz9K^)qd0LXkh3GYxHR?dl%6{#j zb1u>wC7E=fh!#BizhP{Kv*CV3IH^q6v}K2Jfk-3B9zu3$Y}S$=Z4CP8$0QK1R1x>L zNF~I4jM%%LI2-I{hYiDkCAh_O${A^(1~tZL<4<(>%EXf#a`s)B4QKL@av&q%Q~(V@ zt>TN~7_37Yt7Jy+*tGS#l+2%@*L?ohZKob23k6tOLl?ka{2Z^h-tTn48c4pIU$| z!9(0J1YX8~M;kvm0cjeYfJypru9C}3zxFFycF@cH&}HsW{$G2v0J#{lFK#?G@U|6~ zHIK|2x~;Fz&}Bx}(&w(LJ9?+Y|L&Dnb&Pp|on9?fVuU{WSOjTAW@1sWBWv&k|6xm& z0bHJ=CKuhZ5W3c6FQ^3W<1q7w1c(3a1xTB)CYp#5q|2GEE`&N3&HDcHvvn*Z*Ti3v zE%jreD)drnSJ@Vn@BaC^_xX5FM-tQX%lJ~)+*q&6B)-tyTWeC2V5*@Ays8EcCu8MAarZ+h0OOAQe%RDXT}s za!q5Cy9riz6gE=hu=KY9`S`qr>>uyfvqSr&YWHf@B!4D0xxY8N6BZH)-w$oYk>b@6 z;niR~vQt(73W5BxN4s3#-HN|>vduo2Dq_3)KBq!i`1 z<2X@XQ*H!;huz{C+U+|D!^Gj?(5^e#3NV5SSE*mB3=&Yhn^}cAH232X5{SA6 z`4~q^5W|EQp~WLb5&@3$)gNNX1f~qbo*%CBZ;qGz&d_pCd%J-?3jeEY*D2?swSgft zhfCQ+!7uv`o&na@b1lLO4>lqcQ~{v`-Xr->iep|0p}#>3gL`N7BZ#_SX{J=u7BCHQ zJ1z4+;JJJR`(F&vYqfbDCxJdksA45g4KLJfn!z6Srx{AjOX71jS**2;Ddn*mC2j@cc`bBpHi!@fpG<`Ubw!rNc+Q4qA;AzBx7}Aa zzAOfMLxDf4A8YPi7r)^(DeHBc!@n1WcZy6dIU6culE?Iy%rsDosIdVn;Hzzt;+^<# z!*yE*v8qj9Y2M5hTE%eERZ6lk`70c@z(VsFp2L+J%GyeJXGc4{l*|ujbfoi6qAof% zwbriis1CGD;a@g)rN9}84iYL{Wc{OQpQyJ7e@PX7srAc4`}9=`sU8;@WF$b_jP0f* z@!_KWEMzIX&SUAKMXT$fa8=EEG))3Xn~Atzz+Sh>_CET7d5|~)e~MzwJ+8dZAAi8z zv}-7MZ4_rIX6-$(ym?#eh{=#R?yiG1`108`__M2?y&3YY6r$}>jXl}&VvS4)DQWdO zA7t$DOIHKP)>bpM>|Et1f}c!6 z>{cfpejL{xiAN9V64nj>7U|SUVuok&%IR@UTYn!E4Jw0UmT*dAv}xe?PSiHINyoLU z##YT#w?a>XRx#ygrtjV7s>j}oHxU41?C=0*6H^v-gDn>&8d)weOsY2>}IDD0`{ zk@7GwPYB>7-Vr9q9~ln~kuY_0u<3;eve64ON3CKjD zr<@HzZRrQfJs}HmGXE8^gF&bfmZz9C-%)o$@A$n#{1DQx5zgRsi;<$+af_sBE(vkp@ZQ}-Xh5FxOq;b zN#LZ#>T2VM`RWYdkPDl8PNbI66U3xSIwXNA%nLV*mv!if8Pa$P zKhN0`Yssi;cv#B*$sv>3K7FgZW-zh2x;yhUlO@mE=_x0MP*#*BVk zJUcvKy}ImNu6h0dxY!P@BF4&UY1zrQh-LMY)(uHsD?x1@%7*S-mM)8FF{$mw;Z zKoZS`W@!!}dC~aS9@3S@`gUAmd7domTSjU8jaEN#SlAm?O1GIlS)pU0RJgy`#;lKK zHYf>=F#Pt!R4rJ0bebU~*T5qB^J8NhK*Y&A#zbmjHHwGh%BV{{L@Viho*u&T& zK8++XN~IdPMWGSI`@5>W006)}dPxQmL_B^c!QrGT2sR$KGum`>qrmhJrM{RPSJmt? zS!v%0yqc8NDptvEF1WqpKu47{P6Bn2DMB0V71B8^^W+ohNDSM^&KkoOk*w!GliiGM zc`Z=11rg7qx}PjzT~{|gt>X!;Z(f*TlPduI2iNm&3paSgw$8OhX+Khw`udKd&LZb! zO2~-V<3i5q>$DJ5Z7m0?5xYb>nY!tJVMfoBXS_5ZIlIosCRY7%BeB1xa>(NEt=Cw& zX&^PMb(a_9q!w#6ma)xR?4075&-1!PBY-^SP6I3UA#B1$I<{9|pXR$HoFs2rRWh7= z`OHbFHnBR&F}qs4F zZRXf2CDO$e=YBc;eG`vh8roX}F>XsiCt^F|F>J8Y_T2_>vVGhIop6agD7sbFMm!ez zY1lK*mP;W37Dxv_^W9&UlE+_i?H$~(v-Lgp6PwBjaynJk85@v7HW!-BVHp!ly+Yyl zGv{Op24mu1m~HoQFVnB1(;m!BUi?q^8??zZGjTDT!pX9_lQ4 zjkt~aWyEeIgE9~-namWS3)>-|S|s;=B-)s!40hD*uN1`WsmHN8LCYfZ$o` z>u4x@H#6HE&2Sc#ea(iXgkKzp50{T*r+!jZSzd|KW*?5lux>*|&{T8xj%z;YK8XEBlwlh31AF80;FD&)!&9KOB3# z_Xr*ET?wzd{hetsv_|r=p#MYUA*KB{#DMd^L`AIO&q3~^^gH!lYNFUnJ{9m(H$22^ zJvI))EY14o(miQJcAQy`?ce_LwYogm4Rjz(0*c1i8 zUr&d6w%I(t0MlGjK~wgsO&_;D0?U`wmmQyP9y#Ffku>aXyQ#pNe5w~DJzYF4Np2_4 z*-j;TGu)B4AEgnCL4mC>|D^l37z4lst?7F?#@FGcTVo{Dfx z2^Awl2poTO1J!Ji&%xOCnC9U5VStj3k|@dK5LU%kR<{-yBOujaIbUt=A?7B{YC<-` z_x+s}wg32;Y$gZ|CS7_MS_OZ(mCCPbA8DbFjm5;ze-@I z0ji4rvO*?x=cj9YZ^c{4ltg^5z(5Q;lSZ-Yw~G)I3h=+#gNchSP830%!af{6RQKLm zL<7Va2l`@4)F_0y3BLCUU@^>cnm)K0+gBxLyQza3{^Asj&~?F+s&K_*X5AtI_Z{Lk zB4(YRt*QpLFUQ_rma%ElTLt=xW64K%vuiReiHDjay z1}vp0NT0^6MM_f*w z9EAUJFxS$|KOnaSAU2gE+OP}?i=4BZSm*I^yI>^`GfP}@$Vwenr<=N3grN}DvQGkNz zh{PdDWG`+M>iP=Q8ttyucA5?^UGBtvzRzevd6Cm*tu8f&Tw%f)d+Qz+{wwJ*iBQzs zA^r8oTHAWxo2OqEQ{6--<)8pCP%hdF>4I$B|D)UIpCkpLV`o7>iLb{RMH_ukH# z$;2Y{j`vkq%kFvp32cs)7`Ow^QwSKPIyF*YePEAmPNSEa3Oc6PgFd)XUt zW!>nI)aIZMJe<^9$scW>?Y$`y6_vw%7psH*t&ZHD@N7n*Z=4l^6_T0gJj^4|2}Ijra-gy+xfrM+!16} z5IED|_Cn;76XG9^vs%j$!rq?I4!_$e2p6>Q*gj?IK<4saZ&VB6?X#ocRCNfnxTHwi zJ)Du1hC^!gP@_=6=l0%X+^x*hHVU7KyX&PDpOsDS!_roxL$T9D0FO`SH{}`&0VN<@ z84s#>$b2ysSNREUo5(~NuSn&v;;+cJ&Ok%{NV*#*`S*NIG~)aHvF{(*2yFetsxN>L zLfj|aT8Gv41^)g%2SDO%b6_-&wj!VR1k&YkVk)SIfoNy(WG3xsd^@~Ui;l=+Tu2{$ zV@Gop<*k$aX%`7VnVRl_)-tRj^oae@4WRrC*g|HiXv{z;JEnFy;nM52??ia5Me$&m+#QnFcoj>=2WIIPy~O)74d0Fr@XShHvJ>0XOtFVcdwT-Ri9RO`XMt zqB2~`1nc@;05J-fd*ug$#JqXDmrOG!qQBbjBvs^48b;HvSRYF4z`>lf=nOyJsf^Ks z;<=^j+D?ni^FIr)hNTVyY4=X3JnKp+VlO|g1*9j#h=;LxY$d7tV-+VwO~@RV8hLyQ zm7>0Q>rycL@PS<1?hr!1RtkBaaJOVI5ioy-0blzrlKJBBq@O+lcC-XnbV3I4*wYT! z{8rHGBjxruxnk=s!Sds`UV0Dd@)Ec1szK;4cbURYNy+OZ-z8dzlHU3^1M&R8-w6m* zz1tiE-+HMAyWl}aYKp)#e9PnCeRH{||g&7g! zmxqyn$*rl$EsChTlnH*bg|k`+^A#!N7dX#Oof-RPTJ=3cz0-YwL#YME8|+AaMz~`y2oy?F&|;UN^3!(BeCYfX zWcv$jrh0R=X~q^!*$KtE~@7lDS*lZ5Ij1x zf9-)V`&0!Lcy4MD*?}{CRlswS>x!pL&*Jiy6|64yi6uCnf=V*K8w~@VZX=P9NgAHR zKs%=Y9$EyGgmdKQLNnu5lXVB<5j9 z)yirj8_s*x7d^$MlZnQlT+2(VzuHoe=*N>b=#z@z+diQDWGULfkQ(wDBYBUjRgQ#n zI4z^^4NVEjDx^D$fb+j^Ci{nYyeH&F2>Ug9ajyJiJ8i>n5g<4_TP}u~(W2jUA!sWk z(kYMTYHj)b7Es)&SWZ8ZfSJe zNIIXB{x!;P#wS*vOMa7K9xNd^wa_1-qH0OuV6KNr^Xm}9h5Pn}$K(0$?qAkSav`rj z+gpu&CV92TTc*|5l&!fJNcHEpUlWkJaQub5ew__%&wDMbVPmx(;mpuUrfxbj>cg$? zzW6lSWy{^+BggQxcx-Voz#6m$sH7<#M(qtmDJ3|7Y(CJ6V{@n%d$} z&y%UQpQg)bbfXOrw)mcU$5qjGJ5!~4sy9sgXY8T=nI_Rq@d=t>3<-7%vwS=Fv9kTh_j}p(0rZO3&_1~T-Tb{Iz zlw-?ZxkE*)I~^X90id4d@{XojPtE$@pD4N{DZNsI2Bv`LYuHZA>3=BRx#K!TomWCM zntfnDL9WVHlQcakO}kTXb5R-7|02Yh@nQH(UQ-u7S?Q5;H%;qYpU;~XS4a6uUF$!e zR`0hcMLfbgY&~(ogs|0tw_;$XsDD3h-FvzGLpKe}3qsTD%RXXiP0sw%%*SLbMuSFr z<(CE3s|_JxJoS6j!-WDChKv$G&9HGO0l+?kc`KXbLkKd^@D#D8NycP zjzGD34N(@JjL2_<3rhVg1rh!s8&LNuhbX+maKE(fKb-c-Vi|SaM&$9R4pEktT(V1v z!g2=FX#EXh_k*SEDEt<>SfQtb>mwV^y9SZ^LzHjC`0%_d{+DB}jlr560UPL@aZ*b* zM>v-l$KSK~`N^S{S2HbIrx|zEmbK1}3d}a4Y%Na;8MltY!L27dOQk_j>k^|Dx6V}g z-Dd$3EcJ)0{SM#21R6PgoPv7sfJBtXC1>UF-TxF=ChVJ05aINJQXwJ%wfc@&w82M( zwu}3-k;9L$3v97hg+uGud|LaasSM9U+s#_7ogP9Js|pf)tPa9Pks(*hS9{ECZxfE- zDPZj~-xo`tSs=Kq#_!=mNc4#4e}%!rJM=1e43tNkGuyfbHG7sH?F_ zHS5mWTg_P)v0s$3FH6o?+`wTn*iiEnuKIxz&?+wZNe0T^p{b zK&e~hqYII5qhV8X4K^8n!eWE{E@q`kA|bc$9S%VnEIc8}!decm9z_9MsQ=imn(IV7 z$_uhk4!R@BZn@dJf&7gj(%hda+;oHg*Rh_eh-TLVNWmm{U&m5nO|n{3qIbq9P{lqJ zogudKGlX~(qt7Jf5KFNP0%nEBUS_{z@?$R2tEb3_m88pqMd;S07mI%>8XUn5oq!5R z20_i`BtJYni3`N_^-JE!FkvbBmo!TWQlWzG$uy*zs8lv%QrMsSU63k3y?~Csodcn& z#;-4~o`Fy2vcAne?iq>FRoju++zxUpkJr?LwCXeyq+C{SSA6cbYtDu6dCG&HlcV)a z>2f`K714$$&ylb-6zTSwbsH=6ZT}7@fA(qk@)td{?p>yaUJoNqNdh30;Fykus@}Op z;gMv7p26LeayqZVs*17tzN^2&O!>M(-}+8aG1G)2UHz;Kjehqs_?Q_K8Js8Qx&2Ho zey-<`k#R4c9eyBhU-}HS{vikQtHykimR@PU4wpYBbw}C|w9#lW`*uQbXLK0`JkS0$>X825^ zYIpEO7&uEAJa*;aER>O#%q86To|gSnmGw7rY^DRbBBJ&Csa1&AMNhRkc1C&=ZEw`&q6t|Ko*kCP3wOFB znu|=KPFG&9z(CG-)u)V%Wq-?cCmddESq#v;tG|{{HG4%_*FhXU_;~anH+eglEZyaxhE&PSDOKn;|lg?9IR02VRTBMKL4vSqkLZ7n4f#Y1U^&3&q$`>2!er+oO7S`Y?M}2 z*N8u*nAqvE_u`S3MZ=AH@&`UJ!VK z6)^pO-^b7Y!loA;{`a>t0XgbU{FDc&xB>hHA-P|%V?yk8!WMwCzgOD!50NmGNbH6xepcHa?EUTUK$b1&nV0%|M!9zU`T9_^Sf{ci{Q6^x+kH>KqlRL5p?icobsn2? z=9Ig0=l%*pkc;Q>BG2DyvG*Loiye0#SqE_ZM@LK(} zwh(HpOnt=A6}qsvC_HQV-(G;fjPYb=17SI+#l@u~I4l)xVOo|yI_(c;a`!KH3QHZd zF3IkCiZ!=qO8P~k1OXusl`sKX1DYl_3yGmCRslSJy~WkbYK<%PIB#GrW->`YlcqX{ ztr;k@Z-K8?7@?qk67OpJ0Cv>urLQM{W1lBUdQ$w^l?bW3c|AVsvikf?FjkPQvE#GP z7C^;O8XcO5YLecO{LQgxgIPb9xL3+{EYZNtL{&f1KiC+qYhlon5Qx{{T^+|=+g!*F z`FuEQ-<+^;3DK+M`wcF1$-j zRyj@Dbj(;npQ|AM@zBV&7c=)6{14%Sgvw%du_WR*K}<3UXj9vOebLijg3`@L_t_S^ z^{>}Jjr0o=w1@X>-TqkRNOGdI`K@7~|5D!tTIDfYIHXWq{VX&JEXNgi0wdn`0Brt? z&0Gb^2ePDcfccKaN0FyxiPxIm!Ja%KncmEZ4BE}{JIMv?1bA83MgR+5kEPcZANlf< zxigyex$Buf84EC<-CyqVMLPY8&suu2m_xN&;P!DuTA*`&3Dw%(>DQ|5wY<7>{Er7@ z`-#Bw1g1^`!mr&`jQGAO54%{paO<*v3%{_EM}D`w+OMs+KlxtK&(TbDBe5<n*@qF5&v)tFOhNXdTvaO8FIz~LdxWzVjzD<{4v!5E(Aq8u%pR#l~Z7`MGM%j{l`{U!^mG=z!! zletwZstp=fJ}?aX?)Y6^n(_{q4|jUAS`H@-;aLuLXmJ$>qR763W7>0c`~7T!1pZIa zImUf?tKz4VJ8xH&q_2j%DpZ%fDU%sAXpmDXKH`1voXJIO2;9P5^L0%H)P76***uwYoxi-LU;ZkI zG7JCl5}7}jdwHt=U@VB3Rm~VWd`bECo4_WLul@d15|$UcTK87Z@Gt5}T@**jPnMm3 z&W%vzzMuJ+(nO3(8T;@1RR;JCdz^t$+;PUSc_k(0yXZmwoGs-k3TOf#LGKd>(fCAX zMN6_5@*<7jj;+Ba(rRDL2po_iOcEXxzVBj`M@rDmcI@Ov#t8*W-eJMb)hjG}|913_ z+Mr=M12u@JAx2-`!@=3C;~|r5p}z91YvNb0Yt)~G+h$!GsD<(JtVfyL@QX=PFfaRSdZXi#l^L4CPMw?D z%j=2~wBF(Cj>h-%+Do(7Mlr=@qa`ef`Y5Ri=l1i@MgaU^Ok$_OZY*)^lYBvCBy?HV zqbRz5&!T|_#x}ey`L@uonF zcf-tj=s`7;*PC^2y9aW)j_0BKXp)QdpLigKgpOqlV3gH5Y`B@$pfAI&TW=p6w&63D zt7v2W{fIsHr~YrI)xFnGr6>!;!akrT5j{3lenP2@!DsxUrP)5d(I=Th&8G1e>6Q3N z7jW@;Lxhat5%((%n|PIFJm=g^j|0B0bq2T&&v?^~<;5_Yo*d&!7z3|LFp(uhN2_{V+yvBfzf027(KA9yB=b^5C2g@z) zrLLR(bg~-fzY9yYM?e~}(X@Msu*BzI%O-%dKl;4!y2EKZMbmoTzcrC~Bl^evUD40w zYICV90|>N)F{!J!Ar~s@1~GP(5MAsL*%l%Q5dUFRGW}IWY=KV*`aXybgh!Uw-8$fn z7oIDQ_sQ{k+dk-=UXq90L3oP4LCDlr{#4Wekf>zOvaGYUS6PrS7r>(~lnxWm?=Km^=#* z%3h~cr>W(m-XOkLL9pBpA0AO5CgrBfn(u!8+Wh9vSBOT5nh~WnG$&hEjgE1l#0BxE~YIQ$lj(^A+jAW2@b z)|(_-3!wBVINAkReYonE71QaGUp~3y*pb+sav%RlAPAEe$8x(a0s8wCf`t5FgPFvt z=pZiJ1{`ylk@?dEL~E`&7#D_#fUNG$w(&x09bosc=k_vE)7A^JmB5820!9j~MT^WY zPG;rcPKBjSLChJ~kdG}V^`6FO1K(94!YXdDHvNSS<-E%U?LqjNt$-5{OzX%_emk)F zIH>Lv5j7G&g>XmV%jr*4@1Bu8I{d-_ry$5sI8CBLwZ)&(62+~E=EIaBd?_gM&6(wG z0wxHq0t~nDG$i3D=@WtY!{{jSu}?O(SQSO&*FP}*f%Q;EZ0GBVp=925b+6+b`+a=W z(I>Mr9D^2*U*1EWFk@MAlW*AjU zVhc~O|4IsNP%*f<79fkdW*${!-mScA-LBOuAReW6i|-ynMfoUsvRTcLM9nE2G!cy4 z{y?|g`OwLaQlj)ecv<82hQc1-S@;xAcu#SG>@|%Wj$g?R)ZW!(iQ;STV#>8T5}hRz zMB78kj@`JI;E4BvdVt0s`mA#O7>m*wE@-@#o96@NRYQdPvUzb&XNPI_-E?2rhUVI4 zw)-9x9Pg5Vz5o2SW3kxK=Gp34aB_mkk5u-kx0G^f##M*n$YcK}b>P!6)za*w$Mfp#M&v4Bq*6n<0!Gq5}MqUqI32

u-6kEj zlQERJt0Lvhf8uCI+~#(Oc?vi61XeTooSm>DiwOW5w`12*+F{@S+TqDmW^F5mQSAc<3?=k zr;b~9a(8daZI07mJKb*-+Q4HuNjIFvSmy%+>}qfa!9kyYdCbO|G!n_YCX&hX%0<@Muz?GX3OaLl##_Ut{%-jn8nVO2&yv&dq#NO=oer~|cR+5oseqf>oJL{$#ySnM37H8R zoE_l1;C|=ACKZWQXhn6S!yt6OExYLO5{{-NYTDU+KL4p!*Z&-eKo-G}=x*b?eh%xw zriVN*51@>h)qh8}9&`DH_Blvm3fb^KQx`k`&PNb$z;4b)JO)F=6^~3?t+GFHV-X7v z$t%1M=j$(jvc{4>taya{`>a%brwdpXIxpR*q_X}vaKF3dG0fM1YpKy@Q!ES zxGgpwrLK{z115M%_w%iU7tgF0ecKawo@c)^B;9_`?(N;`xe0bMZ%o3@v=Tl3)Uu*GlaU8q0M_9nviZ7aPdxJz9aL~|tfdi`b;Ta}194{S-Ou?g*Gi|z zG*SI7(-CW4mZOa0iLU*k%soI8<#GQLO(Ik#GymqF%kFVJHkec9GSg0jH4FqV2ACj3 zH>&>~%$94Gpsv4p1D9C5J~`e;4k~mF1ECo!^55M*99CuG$OT4ZcPEcOgb8vxD*^VY z8t0DCzY3G5BZQN|@Q8bpvKAY18%C?0mhOAcmNuWn=<`Q|Y7b)gMoWY6`L(Po7abi2 z1U`j!$E?6FZ7e@#$Cs{2c}{Jmw{@B`9|*aqWl>LzhpklJ#>{Njce~P4?3gAz|8bWPnhHYLdA?mMM#lWTf#dK)s(+6% z>FeO%*kuMz=rz*+N7Grhwb4dvG`PDvlv3QKxE8md#T|-kaR?3tin|6V?i6=xad#^Y z!QJ)bJ=ZzEARjXG%*@{FUN=Gcl*X?vUcOxJQ;ohAEM{xGE^;BP^p0?d&55GnVPn@n zEe~*`E4}3?$RjU=02`SDxk#d3J7}x1KpaDPJ>`Dgck&DJHmv1l)t;-@^~!#%-L@tb zg|T16$c9&(OX7Pg&)eped7KEeI`jT+C9~d<{q}4)vc~~{$An!86|6wb_MSQClQ&la}j%qsg zC9?39%jz=@sd$zy8a&22Z+!S0HG8C0$KjA;OsphA>6BqsZSSs*4Zn%jjzKOnDR%p# z<^zW{4M0*U=Ze6R$!OUZ48WG3XLUWG5Mb_R_mgb@A&P(l7`etH0#=Q(7xqVBs)e~F z^`mURhSbJ+IX(0eJkY$F(5?b#G#6P6P0mcu4Tbqw{hkg>z)E5*9k;h!v1iFk`vR2unj*f+E*z;DlN5kHqGR(&hQ2`Fs7YLx$F*rZ7&04%KRc3bJ*&p;`19AEG(g0?kUW(yER== z<7(=?oitzfuOJOJ$&D8Bp2QLc*w}Z#+`cl+x(}~`Q~qAd1vSgRr`u>{OR+WehrUi# zq|scv*vTikF`_ag`v2_?2{cRQD^g%C>x2({O8Z>%=MT>_LJxLFyti^>nBlBb;3WF5uirU%4n z$5O9c63dyyD{2ced~;PXykNe#%_rSKLsu6V?hSbyBZBj>hSlGGV$s=G8yqqCiVQ=f zKsAe@H-hp_(|w0W_y^H@ z1*qO7y{|6kc&vY}P+BPSWqq@yTC>81){+OmoFQwO{R}dECptjauF`D<9;RjlNb&vzD3RWG6fRYJ|5y}yjq7Cajk0p zq_$B#Eb!7dfV}xbcoEqb!M94)IPiC|koangfNfWp;hL)ZpQ{J9Q*;GMVR~E`{rx7!HrBBG5HsmleXjPcJ=hc=a#40-nmvO zmN8Ny27%2{aZr}1)Ox{INTTW3kIC+$aNY`d73Cz%VU;~7H;`1VpbTRAszOl8p9k25 z-P<1c&iSYm?l7EB@pzs3Dx-ewOGZpDfC;CV7u|qesUDnz*eUokTr`DzB5@V4+fS}l zWAu-ILhQknuaomZdgZ~F>tkP6=%e&5c~847U80-T){|&bz^_ro{V2BuI6w0ZNcM@A zlcQV(5lHbEq?&;wdxy|-wFZkIViN2hKP#J;OO@UImMT%S$t>cp$u(qh}KMOA%CU{=uSS;l%GKPv?@8I7245H z(ZDN$KmX_5` z{!81x1z{PpO3GhRI>ix~xIoFvF1{XiB;L-%uB_Xb$2E3bf_|GDF_s8QctmX43z`^T zsxxtlwp)x))sAAwm`a#DnXWZ_4 zE6i?prf@f$Y`W#f0koDw#+k1B%u3G->;*38q1~RA?a^Ie8IiNwuftk3hG|yH?;{$? zTWwwb&jXBHtyFy0jYv!|niFW|lNPp8=Ps)#fW7wY*mHJ&7)@we+hjOg6_=lJE6H2H zz=Op9(okt70ttz1Emy=>Df_Em*+2-&Od&-WWDqU zY}f`Ib^JHxZ>g>vio;>NNny)|;^EkJCYbrK=H`-ra`Bf+AgvE|n}5dwhRAQLhX{+! zbFoU%DUKv5Tc5eK4jUM*p?G^dXt9oPy_?^7z5jQg8bxJmRDpGB*%l0N-xb{Q-R!PmA1^7r zAAX5wGfHV7ZsCzU2DOxLwX;v@FRi@V7=t5@hO*Ndz8f;;fGH0hM~>=}Af9#45V3Cc zWqIy<*AvsGZz;J=U<2^;2`c^YNO>(s;>xAdo3<+*E3t& z@xqExHG^jA#4ZOCRi5d57U|SII4-7LHmys_`sVn}yVl9>68<%zDs*_yJdT|W%R{}3 zoXFtOYMHO?>Sb;HHGkzYq^Kh{-%qFZ#B(HJi{sR|KJaKhI!yEnqgHt`3IXyr?n(15 zvHjUbEw;NOz#OY#8_z^duRlwMCA$_|GTj7Wwi{|Jupu&AwyL_jM?)8y4GRIdhU_=W z#u@QENrQ!#byT_IzthaX61_A~VOe!}@hi?`R z;z|QX^*cAjR=yGQcAdY8BXK8x)d+ZtvFcpS5~WN+pQ{USCl?N9W79~)tTq%6&jQCsC!U4Y(=#czf+$9|wO~!v8l$b0|eSi+=w1mfasnlZ?xjn2(~*ga0_`8n8^-fAc*Wf3O}L}P8%rE|QV=$_`C_%9X89MCmER6ot?M_CaI&9k zge8M5#AP4MW(vRzFyepT3f|Xt;%1dSHyrw?H_=)skVxcjC!ouFCOzY1i{CmuoiC49 z(_r5_uh(Aa@`^1x{+bmN7@8*XHnsQLZWvdP*-l$ki%xaLi)$Q8!bnL;GVZ}jcgP69 zo5aJiD`dfp)W8R-i=;*>prCzZG12Xrcu!7+7B*_&OQW@wAjps+#5b|>p~X>+WHkh; z>Rp&`Y#^jGfJCJ5m;yO@FXZ=8(R_6pqdq8iLI<;mTBhQIP5$W$XF9#~ z4wTIdP``l2x(0NDE@q=b4^KjqV`pvW%e}qhIoBt}-m_7&jcFXw z`6Y~xf?XLU-IdchX#UbD>o3Dc{9?fs9WFdV&WT$aVIW^uMI*GQ&;HK8!@=~H+l;0% zZ~XmOYOdDy65rc!_m&Nb%05HnpQz7Ct{!7@;#F=ol92ImbjOA9!uI0a?S_Cv+mC%n}6(ZnIfV1(=vba-r>7yMDf@) zG&_6#N7{D_VzV^l!c#j=u|Abg>IIFwj8D#_hRA}HYS2$qiBgy(ox0wlhFQ&t&O*>D z=yFC3sN4c^QzvCt#ItB^x17kz{4RAEl#}TZI}|%t_&#)ZY(A{*V%Mnuk|!1fLW z#O>C*I4`5O)u{Y7Mgn4uq^-4@Q%^gcjv|YfUT>!{+e}VU`O;9qFj&3cA*D?E>!Iq8 z@YPeUD>o;PpUnpeo^EyhD%IQ)cG*0tZ|zpAQj9LKMgvm%B1|53+&P z5-m*BNvbOx{=AP=6tYj*ja7jq6qDYXEjXv7(m(xw$r$fy4LGn@Zf>Q2S2F}gp;i*l zA8Kn#FNlC#U2I9swb;832{r7LR3=UiXY$%@2V&(4iHnz6pFXW1-JFEf4FN$2qIzGx zZF7B}pwcW+@)&U$*g)iZN+Nd>R>oh0Dgdl(IRi zExFwp{mo_NX?DzK|33Y=GdgRuw&AvhCF{fI`Oa~@soOZr%XNqs72K|CLLd5Kdt}X4 zw_&c&H`2zgSKAF?s~|+o2WptS0=*OCQmQmxvp+wULUxgpzP0@|K`Du}!2E!S63B41 zQC;_Oe%!Yoqrvjq$=vlcA#MABP5i%h8}C^1efS>~nFadC3pZc7K|J1978_oD54ecI zWM4xIj>g%$erVv+sN(ftXlhsKHRQ1;_FeA{=VUEZ8Y)6F8J{Fp(8CCc?D@gpPf4FS z;Y$J5G@Jz$n{n{=u+Rg#b-3i_Fp=*2Z0G%eVK@!LNv*w*-QYKF*TL2( z-rEx3htzi0EZ~Uv2tJT2n@3VupG~UW{g;(tTe{Bn55_Tigp1b4?Z_mYSia7I266dn zwHN*H@CJ;bC^4eKK6jW46e?-?4=0)Ze<1%ILe57z;;>uqD>VPl0+ed2HgaAdT_B8v z&xMO6^`NAF$s*muI#K}O7ZXm0Mn!MW4R+E^^-Y4H5{l#oplqxm*@UW{hB8${lbKFP zV~|J46BB_tah4*@pPj{pd4{N|)C$Uq8-x5qFWLB8-YC%+!$!MxqhF?zNcU;|AH%~4 z=57*JXT>+*0|njJsZ41_GQ{VGv#}Il8HC7nml(ZvwG&Eu0#$ zF6e9Qx&-&zeT`h{ikyLAGCgNN@Yd6FKRTlW2Q=7-WWJ~=*@Se{Z#=yq>A7?4%URL0 zO71;Uczju>FsW%zj5rxYCDp*UgqnGI5A_v20tAJ9Bx zkp*mkIywDKRPfG}i2Wof4)H@t;Y{}?*nruoIoflcxfZ@`NZr(#t}A4$^ZXV%*@e_&K~a+$-ia=MM&^d@Y!(#?nS&4~cIc^tH0359zSxYZ zf$*$RlB~jhtYj7d+Y1w$F=7x#_PN%v%4Uv?GmJ71HJRtX(3U~E1cpws-Ql8VrSK9s z;4CI(OXDmD?I-u*XXu-ng8-W{a`~3nyY&6np`(At%aG^kUsd|V0ZGPxsMe^iX~AOR zA`C#Uv;6&D@+Ex&s#&f9_YQ-cj zdi)J9A31#53|UU-_VJ!6?iffgqWilQ=5xr!*4wdVURs2F$^BeAJzZ2$P82Ml%!Eq& z4Y(utijeIr4Ez^@8^85-fpVSB!;$vxK_X&9I^WJ|sWVm!p*g6B4^)VmkGbWJ|F{27 zocL;S7`#Qo5>AUc%lrrQ{6|0!@B}`TU^F?uwLhZW;T8hs*=>a!m?jX<@L`3pxmKm*Imdx=QliQ^9+Fr15KMMfuj`ZfT~5{|ad~kG<#ec* zVY&7%!D-}NQ}R1Ym>Lp8FVdL?GHYaIU4nWDK>QNH4Y1V879{W4&LmcZ5`$re30M%T zXs+Rqg$@QJ_3dks<*JHCa2fnMe%uw9f~F*VR;Iq+`E5TUhd4n&RNf0ldw+#h8|pld zvsS~HD>fJEfyQ9fs>zVPD})@__ZmHj#W)fHe4at98~}B7xl!D42g=pYJtdNO^^&lm6YUg8ifj%Augu_&JC#i*&Kvw!Cg- zH2g!-w>LT78o7y@538YgSl%wq*qX3#W%k=oGj(?T^mB)|XOOv8mn->mD^+1t2g+08 z%!=UE-OFbf+6QjpToa>&3PHJ7tNxPO*{3_f%$psBw$~$67Xcv-HhxWI%9B~uI1OcZ ziK#5dw&BrUSI>%x@9#Sz+nXhty}}~4g#SSQ_8KgXmRdM`JknVXSyB}+_!GTfKQ#8j zVF|~>f_fs!@%w6_3ti1tHF|c<5Bu&WgH}BDOB~#%UKR%Od)q;6EB~%!^NlE1qo>3UmAqKUsfFpRGol|c(EZL0tBAG0iq~f5Pjogy zE%M%nv|i(yUqbAxo1NE|>5}66Qr>Jo$hwa^pT|FM0PoPsGwdE7n_n0F&c9kY7OXPw z#yocgSsM(oN1FM~MDt}eJ#<3s@(PXn&b;djxy6bst0&(x&m9q%x>26LF z$6jeuR7=Nt;I_^H;6!TWgDDG8E-ZreCK8nwSwshB^us=Lbsi}J6Yw#btzdi6@LSgx zM8fIGw_ma)a-nxVB`P=Kle_$NmC(6FM@yu43=%Szlkm+Fpu=o$u#d>Shf;V~0>P!! z1y;&&m+NOwJ(j54Qhd%xeIbVnxZ1#`@cn9ztV>7i>EPEKl$asLzZ!10jJS{j6>kj0 zm2d+pep_;v>|l@1VeS#}uyyd{Lmd|K_YSj-%oxyIws;nR67;;6_8qfdbWty13a=`> z>_#U)-2KNkT=dv^h@{-pZRR4LC>jt%2bgnN`GX7f=4`?@1K>(^1759Va1lfwdznZi zmB~~v?qhJDCJ}<5^LFni{&UmSL+ER#?n&?!&$oRZK`2=GlLGW$#l%~o|L;2)(rY{|Y*wQok+;wAv7j3e8V@ytKwG zoQf*ZO^(ZiO^#dcBa}P~EH<1@y}hLoXqIRzul&CIyz$nD<6}4q5uwu{h7aq8ANiQP z;wdHv?jXuWYQIJcDI45jjS>5@j}&|!^s7&9b#-g9u%FoU>I_tyQ!>>?{Y)e20|Q8@ z<+3^wW-{ywZg$-KoDjGl22%spVd4Y|_;ns4@A z{QVRk6jo*5ll|eQ1RX0%R!_0tFh4SCXDn_^^ihLxv|oR)FFN>0&y{|3(gWxaVlS?tA^CoIuL{bEe$0Ag#6 z1sGV$2{zo%H1N(n9Z4o^C1umTkmVl(WGs5QpTyf$8Ve_)%L?CpPSkH;bT1T za7Va@wSeEc=MwRe^LviVP6}XmX+MlA6@M8jAXu&J9Pqjh99?md94?sWH2f47o}Sfb@Pj$ z7BcurjhH?ZfaT<){G;1P0G$?HZTR;S#Z4M~$$S|!V3~>AKz|rQ9J@Jxg4@|vKc`82 z@~Hg0Bkmlr7^p-3V4!F<1WTQW!HJ)Q5e_W|b}-ygzBMQPZUm%NrjsD}{b*M{V((j^ zY+Qr2M0=BcK8{qx%yK0u8YwY^LE*OmZsfLQEc>r8G$KA!j+)Tjf0D>>;=j=C7H5>_ z%hfy>QdrVM#JGS+mq9cUDV#l)_v)eayU?;I@hq!}LiO9Dzk8{DjSs=I#mZ!~(_;_M z=Q8>G@F}=H1vp=S$YCZ$w87Leu!R;@D5HQS7k(h+F=-dtcr3{bp5Yt{2s3&X{KYv#$}J{HI&3PZHJkBvbEU zmsDsqa#3nZxWyj7A=lu3uw`?~=waM%hl77wldZaw>GE+}ck>I9nn!wWgD|7-_f+NA zc~hE)KE1}Im{DQeJNL@MN6-Mi?P3etYa>Lp29^BD&-P`wxfnS4Z0iL+{nx(u8G^wS zTIOuw)dNd6GUfIf0X$qILH6c$UNLxci0r^|4yQ#*9D9d}IM16lp33l61Qs{emoZ%l z(x>E8fu!%}TUS1|%Hn$O4#gvwVA4zg=)AB(Gb`57@RWF=jLydD5!Ui2p~O@8Gjdx7 z>*~TNM!CqqZU{~%T@~tze!wpI4vSShOxB!}mumJEN#_>#1^yB=0+f6VCYSdcV?Pb#%5n>tju8qK?IFb&Puzc%8-`n>zqeKeM*OY+i zu)aD0GHJAZv|Q^5^tnBf1{4`^d^*AV520))5`4RHVJ6AC3^MPWO3?z+=seUTHy6Eowac7d%T~Qa^}_fFQ>mRGcC?!nSYvj zN2QNAs9z*mkV!>#AFF}ME1V^EM&#QInFPRvY`kZ`O&HTuQ))7K#X4XkZvxBBc*}ji zN@c#v|4y8V<=8EGkz7F}kCo~5f&C(r{&$b;J~|?{!)`x zx`Y%on4tz0S6iy

xPFdth$lkp(%!Z2%YMt*ViD|K`>R>_ zB=X&5{boHgtz4?%v+W#1Ta(Ib%T)G|2%pKHFPZ!;%npj`;{|LLlG7W9zapmG;zUl| zQJUbVU|0rwVKOrN&pzkU?mK>y?y!D;AskFc$t zIRR>Pq(XGw=S*E@w#2D`vXi&ic%lCpRtK?UScOmy9i&6yb+N8V_d>NB)>XiUQ^g{DfG!kJDazs+fc9ZAryoRp~Sg?Wr};{-;fy* z-!UqZQY@XYSw64GbUcC;QR{i7H5Wv8fWE{XDjJOqnRvBGgaR`!DubWIfIHXc<>8CL zo8czaPXkG{J$1hULbC;-XXqnkZVeYGOF+pHSkT%&6fVN5c7_G@HD-UPs&0+b#9R* znu_G|s78yA@(AgJx3&K`SOa5QI%!Tde{4uPf1CrIHffXHqpY*73jtvK3Nj*~{*ZU2 zao_81VM|7W5s_kdiV@29&M7_E_c_;qL%A0O^BJgPYY(&UjtvE|&}g|n>29>Uvr z+h*+gb~U<=tMMHaw2(wY%jE>C^5rJEj<-QAVO5%Lg9Gm0!y@Q?5w zFt-+^y!h6QYgOuu8ku}+ma`1J@?hE6m+ZD8TSfsD6U*C^$Sx-hMcf8xX#<^-HZm~kN?{!rfu{u zr!P`O-TKOO472kd@?l=UVjnj!lxk)Jg z7b9^tDo}m=gkTMMj;BeOy!>{vXTIa95a~>z3+E5Lmlv)ElXdcq7dh7$XX3A7zS|j( zKB_RrDXHM9sk6MV5z|v8W{>&6TG4u|nBKj)4$u7R&f|CG-1(u`U-zl>hbfqd&FCwUp~hldng^(_6cbzlz?TBtOlnih zbL`;VjeE+Oo-PxS)A>c&O2Q&BI9`q3V8C2f+GWz?s3GMda`~>9_6c zPy}2;4&jVTI->mVVX2- zOv*@zrD+O0h+cKO1enOW@`X%gRB>;{DKA>#Bjhp=6@{k76*6(k%d92*K$kt*H2d^C zAD~{vWC<%3yd{{O8!knJgx5M=K;ogc+Fq@=RS7?q$zj9A`f%AkCEv&Bze}7dEy%v= zaVQPf_7|U^DQ5}B2@bHD@M7KE@lnNUPq}P zet5*NhoP3hw9o!BncR-eT6FzW%J+1;RR+gSs?BrVo1V>pD47InzD8O9^MFv5#(!Eh za{DljA0lTlsYt8ZTto2z*0byo!c|)2ybFl4E1WR(Jq{|f(fau%hdTF+ zS>^dTDgYf}eB9Z@3|1wk#b?3N79AA$jGTbE9q5-uq!K zk|r=oii2}&I+R0v-7l{m!$|7nF4b>FU%WqUKx@UCsbv$2d){JJMWq)v63GeU2tt7~ zP1e(Wp_}MT|4|$Ml$~HB(4!?DVNWx*HU=p^)l{e3a&4JEq01dChvsD^v7P_+2?;wTCUt6Sdcz37Ru#1l&)wX;=9Qd zygeM|8&?;WYQi`J~1oi10~A-fxF%C3BoY^T<@ zndiCkQ7#o%7CWtVV8hAL+_);$9@zjBEsXXyNA>0av0>%;o zpD=YYiu|eBuH31K^f- z5>JOL>XHM6l&3@UpX|;>eHd*g>iRmTQF}kJQRl734H}w*_=R2Yw0$oeZDPt)JND@S4#e z9GcKU*dN!Ip{whCsqbH_DwKVZVHPWOuMS=2%!MAyoR%a8q8NU@|07W>3#cdEPZsHu zbg9M`TdjxVLl?bt=XGq0DfSZ-m44sa?YJn7ByyEh0gLD&Dhk7Y|JtM{*E z-~D6k#@bLrR9krW>r~%i7_ZVLhJ%5DfH-KV_H$>lM|$@jhjKs}1%|JC?Kgc&vdPzS zUA(f_UvbkTd&BuJeICI`f}P?u_kL^53iT1IQ?)d2n63sUS0({aX-HVdTX3@R9mdC{ zr4DvVrhXT!623G@cz}{BU*y*>u+*lnaGDZ(92O@Qg3U?F=^s(pV!%|l$* zhh}T5?>VJfuV<^c0|9mpeJ9f2M=go4sR7gl7)O2knPL_0&;RCXWt9$zi`B136aJva z*&p6bTh7}}zc`hU5gg5)tNendRb0av-FoS7i&*K9C5O;s|22g4jK+r?o}walU8R`1C=;S*{HG+qo$ z_|2TK;HkjHHDCl=Yrg0|^*j3-rnzs>Yv)sY;J!`2WiB)z@%>l!Y|lPs}p&(R)%9yIo`>3 z_bJHuSuxvi67I@MRaZx}WAJSWG)Bo*((HbMuMBpLC$IzrxIre0D3E5e`l$m>*X6-o zOSz(?d!X|xZqz;VVz||^jsE6ze0~Ft3Dy9xJOx$9>I>@Yz!pM}y&t3}GZDeNnUvQ_ z7Hf?V%lq|Y7b!Ocki7p&!jk}%Q!f8|+Y?j)5FkbqZ{|&kwkG&Rd?4U8EvGC4xiDH9 z0BNDmbT}yP6g~5mIZb2pyKI;=8(rF{jE5}oL5nMoG)%nO6SQ8%2@yle{Fc*N2rI^t&8{|EXpF+k>7 z@ED3PGX<*#ll*;vnw~?Gsq$&lu?}t~yX!pw+FYSR!VkGL4)Y@srZ3IuMsn!WjnNo} zXjUo{j3nnpUuCOMy|zVr)Vzgh`-v^{qiiK_Uma9XX@K`RA@|U;Ndov_F_u1E>W3Yr zOej_aJX4ajUJbVS&$_x}uMg6u`}LbC^^!d)+T!lT@_XP!UO5py#|TpQ$HNvRp;lL` zhb*oa(b>7AlHaGo6kL$FDl>84lU-IQvx?_Bh9=F9xJ%@NMin8VYL)f>EWjR~S&4fp zMReIySQzqS2e#mU*f!6%MO{kr+Mx7K^i%8rxD683(=!6^+IIKrE50qgRb5K_55RTV z1&J1+ZsDap_9{t3L63{^5R40yJ&mAi_JYASdZ}5CXSr{d!z|Aj;(=VS9L&(PE@Q6D z4bFrfiTW9oh2ShF9Mgq{H*GGfZ(PcIJc%d@{G*F!%3EiDh8TnoHalHB?8G7MS9@D7 zeDb*oS8q_=)$PB90zlkWP>iH3;H8O3;A@~NsP%U|nZw(;ewjha(f8HA-eYZbXm1NW zZjFaS%MFbFplbu0MS5n6d~*FxpGLL#UK_uQpY;c6QguyUlq6@v-qNKCfFIDc!i>8+ z8EJ$-RHxYTG4|$eQS-o5Iqz|qUw(a=co6KqZupTR;q!7b#f?A(3%?Ip+K8>4;STv(OwDQSXlsRNv^c!lT5 zFA}m@jkkKUBYY@N0I6FOfJ%Hq9?9;f69V4yqtPrluK@?li!Mvg;m;qiWLu7&Py0bh zA89D=kZ+=c8aid1C&{}X&kX^|fp?y3E)kx(uMt8s2Q+oJuCgIAJRXJJiD>@fb=RfK zuk*1ei^EgVFWH;GL8$x-ug|9z4LAAc73n$?}%|jOef3{2k6|2MLW{S*B8HY zjRr9fy9t{~n+?$B#RoIn{B4NHi=&US+aDM#b`J-Xg#%AH!itwQw+7UhF$AKH&eCor zF+*QLWhG-8f>m+f9vTlzT(?1?VZ%~jLtw;UQ5b=m)T7O2L@`)BXAjjp;*RIPuZWjb z1ZuOY2q!uDjgL3UeYHSCL|5+X)$5_~DwzYa^_El7(}t==@?lBd2d3MNd_ zyipWt&ISs%HZ@G9Y`j6Oqx3WS)YFR^|6s3t1CN2310?1`a3Lx>5dz%_DU>qQ zYqHy5D;tqWCMP#Ds3m?1_bw zdn|Xqr#;lC-M_L2|4^JVIX&o32I+YOqM3MX72lcaa=1mctua=#<%iHV3lHPSHJbcf z8rAN&5&5A1Y6|0`k-b})zgtI@F1Nd9*m&`!a z%-%rL4gt{&^Q4FrFJ_tIxsul`N~gDyzsnqIJ$U#s(T>y?k0qWmK-hfVdzkfp#>K%g zWDbso7}*b`~fE-g}KE0Dlk6X*3;nbY zKS@KZwH{42CyA#0lj>w0O(<$(uzV>z-7*p#Bq)%pH%cGQPpZg@p_8cXN1B%mPiV6S z|H+VEZ!tZ4@=U|t_z|tSqv1#swhW#H#Tq7AhUAwP;*%rwXmPLMiZ>+Zn*j%%e68)_ z^rfIV364V3%*edhb3JkB){|e9(DO64UzD!flD`{ckGV5~ zPUers0X)}{d;bX|Mrn3uCigzD&Up%?Nn*1UzocDD_f4iFM2W~6T#K*DB-34QZ-^et zIo6;Fj&!y3e6wGFHW+!l3rtV>X{Y4x;M^Mp)nMnOvKcD;(1MR{gIc$Tb?iTIGDO1w z+fTc0i?kLL^A5;pbxW+AELwjVbIKjAOrE&ZZhR&sxor7U$=V2%&r)qlfz%8Lhgu?kKKQOrU6bo~x_b3Ho9UbQ@NG z-lMFu*siTa6K&B8wfnMYM3|-O0XBG~3luyq7jk3KOv;w9SjQJ9(00wb5b)*?9nQIY zr3j%T@0T-I4sN0kLWSmKT%}TF**N;CKU(wWfK@mr&h*~4h&-c@0Xv=dLJ z-y(WDJ?_Q;n_Z#zDhV`uj{zd%%`X4&)c&Yi;<=Ew%ZDU%V&t6wy4U7U1Xn|k1z83E z_v(p&FWeqpCo{W3@xHbx#c9r)(E!4f4^gpvlQtB8Do0tkdzGPNkM=7OZd5b*8B-2<#Y=z^>j~6HovAV`K`^Z-xgQQ762gyT{ z2vfs)z#R{`uBVUpOLVi^#kddO6vg548Wa>`(5P)*^VC3M+)su+ZIdAr4_}Bge#MDq zVwMJ=p88wU+Ro(hQFQ2k!c=8w-At!I#NDQxBxCL#!ppZ}O8| zkPk~p@{@t%lX9kG^xG!8=Y0IAwcsk|wceUM>Y>kjtN=%y*h4QMW`2R>pJ0BTg_Ozn z+knRx9@0n?+F5+siZ>bu@!vVhj36yFi?cb%D0CY^8sTvjr(9i;pQU1?eWCbmlHzNH ze(;gRvpuKF!!3tqlNo~B{MpK(?4L$rpg}cvgV>PEUT4rhohK8mM86dsr;fA61x->r zK40Ys1E&%1_vJrChn4kQjs`f<4u5{&=yK96-%Hwo(Ja`ePs-Wg0YzbY`SmiWd1fVb z-J8#_fO6M8^dEs&V2MC`qJE?pNvcUsKN!tOU9j9&mjFatom9rY?>nX%lP z>)GGRWnpusAUyHvQz5}dpC>fFmWB~cSp61wKL8Uv*d7qXW?N~Ts6$xiN@m%P|E!ny z;qT;#&)e9%)tM&?8^MSisfMgMT`E3d9tCGlB0K{J{!K5@S9^vS={Ul!e6f|f`5`90 zxRb|Tpw`YOpm{#jr?wi8FIrcmJ*LjOyFZf(GxBr%o%kVo!1G_<{oe-fGWfT&kJxZP zQaS-<&%Bms#ws&H4E4L;j(?&u+pS4XJ1?2>(kVC{m^4ep-Map*91jfg#%5;SR*McM zry6`!BAMK1Pxc6dRv>@IzyJF=&3|){jw*ReJ&;gFg%5fJRs2c*x#E&A@=olhfGXdp zV6sZ5H}+cwdVxq{fR6<>^tS5m$BC55TJ2G2%&0%}o!za?_M~7#%VztDH6^-F9*JDP z=@J95HO^LDYZr+oIb-~$s8AUE%`Fy5^h4G@DZd+@ejiRghL%{SN>x^MfNghUxf0B% zzxI_Zfd#My7||de@mhZc)0R))ms| zvce!EhBS$BV+1fzOda_^j=$tEZQ)AM&YNy;@?sb%x-doCzWP($l$366*E35lAomQK zc>C#YnyHRbh$}}&{c#f($hqe`7Bxg~f=Kru3eQRm`m@dbcO>C-QxvB5!ED#Lo5yhj z*u*D;d1YJqhe!Nxssnnu`~&`P;lv(BL@ zXf@(Ob)O=%|07wIWj~%(G3K{UY!V{kpMBi8Ael0DhyptH=q(Xh#nQu)YhQN~0FD(P z(aNcVAFRrTU8;NSLAk*#zx#1e3r&JJaERBk#!@+;;X)pEZHltje>%3^j@yAQjnQ!;lO2A$ik`EO_F zyr24~yi-I)GaMUrcyXRWs|5Es4FvzogYiL6w?_|jhUL4yw;b5gJ_ScSQONgRNV%$> z|K>9kP^PRB5;=TgvGT_aCKa`#4adpnvtvz34E~)yPqbTzz&li7^DXhJ1;nU>T`>FP9>H7^(uj~e3tKz%7L zDTGSSmJy;hTS>+5@ihwtN^s(dV|Xawsn0P*nN}~n-n-;~*XCMK=5(lSEj~gT-~(tD z(^p(I04ZHgd};8`cYl3VeeL20D(Z4FqmM^qU)v|Jsa<0%l~maBh}De- z99!%lCFGtMFH;P8y1AKTuS0gDGH8mD3Onh3Cn8KJ@O2X5qo}mQPdjgRyHs{1c5CL& ze;2hka`Cf9t2u8)G>Gj4n<}R(u_kEu%xd=1{k*FGR)}G z&;a^=JNoq=Y9q3KKE1Xk$N8^@DraR3XfO)&flod&)oBXX3U;LGzWKS-~ z7Elz~OnW@DuL=t=b38ZDRzasVWBbQLC?aphWOJ+xYfEh$+f5KP8OoLPCn0fXKlnSJ z-xpAfkdQJNPwLk-(HLn%+=oI&)satJ#UALTtj?_^%Ku#hCTK~4n!=48r;=?)v3BS> z&U7}?yKn;e5FW=)V*GsaRq5PNR8l+AWEz5f(P#>kd4t;({~4yO);HqPq-qrZ-=2*y zdmkPHg9e^t^xEYsNZUnGEO`RM!S}k5mi@@4PEKIb_uQd3Ym*E_7P=hT13dMyoP7A8 zq4P|Az5k)zVm+f z-oG%j&)#c2$;3g9D2K`J#?k|}&I6J=miAsdMj{=sR+y!GI^t;SnaMHyAQ_Bhe zJ{U?HGe8QryLVZ8bo66>lE_e!Vw&MkB)!=2u|f!l_5$J2%kW-JVdZ(Hloh#-*t>sz zQ5sj6$R1x4M)^2-X36fqNDmIxwrXmUs^D*FYHJ>;AS`X+{(-l*ejHq$J_}^w7WMvq z=ozx>%p4bLLI&N@l}!Ud2-dX5OQ`r2FhQ9%?7UW0uXoyr=^AKkeSx>K_d1rVmtWB| z`xYEVYi$O&X>Q%+o=Fl1q4<9kZWhB~bFJA15n;w3Vc82~(3ZQl!zkYv7HzZM3z~&7 zuE((9Y-HgJ9T?(xWn-~=xjX>caer#qduZD8U*a!b-nyS{ZMk)5y(|KL8X5ju>!_Oo zIKZ4S*%%Pqvnyf~1?Sz_Nhd($5S?!~R!`GT=l?tNPS*06;E;2hO&{y!_>Z)d0mA@$ zsECj?(+fqlyV&Er@*rv*^>Ei(C3FyGeh_zpD#*l+te9*RER7*Ya~2-wZ3PdjPKXIQ zr7oK@Q2WXZMyo$dEM1?~y6o?mtkIk$MHHv!540CT`enoqFU%FkrmS{EJ5Iyop65rs zY`TOlS*IXMjFnN@r;0ekbv)zUm~;8Gw{!!Wz}Nzq9;;lK`(qN1&N%vngVQP<*VTJ7ihymw9IV2e#1Mh*Y|~pRfzV=Pmn)wK^P!nRhA$*&iPHLMH`~SZYo- zz{cI%MobUfgYyxP7CGc}gDtc2*ZI(w&X8!;ZvR<5eH|o!aCl}5iDH2j)vexEn&K>( zvLzjIHzjfnguHjB%0|}1mdd=D#s4=u%+su8@RaMhvTuM_;>cd?WZNaId%1r=5Z_BC z!FouwY)k`_H-@IeVfZ@L>6p$(LCQdW;HDRBVc%wOU*i!4!S?DaDI5M0i`B#)<7SL0 z47rI0`Z<(=xEa|>{@9gRH5NVTCCh5LE6F^*^nSrl$cjplFVCFwtX)8Vh2mMC?=Qai zZ%Wwb-4_seZ_7UzB@N)CdjpY$t~|T3f2QwWx*u}j2=dU_Gfe6^38@%?g+Wv30nc8A zZbM}_m242k9@;?>$3GYXy^OxN?VvNy1@v=^=P^}5R}9FCUax}e$0CeABP{SN-UdwX ztlX)u{%pK`nQgqj&{Q)oF|=IhSxlMEVhXfRr~HP=_}L{8w$(bov(8Dhyul%+DAQR9 z{Ju3A9BA&FplqA67^5_NLXh8L1QO_(SYxMJbnB&$xlqH2+3jiRRyGQh`60&jejq*v z&ceZF=e6q^`VqgoGPiv!D`*KnXy(e)f9*x^1*rcm2%4hbiqf)>q{+wbHj9z2LR?I~ zcA?FIb3_wo)Y2(S6;*?KO05YQ_ogOt4_Y~qSn^nxFeL8va29Dyuvecg_Jg!?MAQG! zj5BkS1xXExyK3E0KfWW?KDK(!aw_aGgp~i7bC*K!;$!>xZCLL8*jPkzNhh4a8aXWK zU)i-;xhucI8-LnLkkqHS@!xU?fekj5f+4&7U6&CAH09iKm{gi=;Ph6HNS|O7(7D!B zL+ezThZTr{LT;F$-)HHt8YimCfEcp8Vj-e*l$vDZxp=Be&U^1!d;8&12hL(7O)%MF zwj`0|0nl;h0Qs&U7x{@41s7{Wuf?6q`u$~sHw!#BxI1%mu0C zeVRb*o{COnII#5umt9mt3rRtiAA#0hY`3)J;0$V8l!j@h085EJ^pcZ3IPX36R$*yv zbJV?9oQ{*1L4+dzt)-^;W?GA&(HkrH;wp*7?J14U$GN#rIB(7>scai&aQaiv-X!752E-{L`Z{>m-R z$EvnqhOFb?hHxnNV(8D*pH+ZiUBhCxN>D)zkS&c7UDfG0X{|jP7=AUb?*{|Jh;Jv%>TwGF*QtNM zkpyRH@E@}qC*b*f)#UYo%suEFm_HcKl?N*LnQ}QW*rW{qmWH3!t4_=X1SNOL(jQSW za+E1_LV7$!$j#Z`Rk=rfL<)k$8|WF17zXP$GC0&~s+1QGYhW@h+$I!PQCIw3!bLWB!aZObJoNZ$=MNrC?&rsehqw}~b zL&nxtD_oz$9gPAgKe}Tj?;inh3YcO(BE@oFN}>BM!JRRzK!{-2(6)V^WF9w8nV&ND zL-`jsk-{K*D}ov0>b+CDVh^bpeittMtJL7QK^2OE@mZ#!nvLk8(I;s$ZD5;I2z z>s>EX|-CWA4kDDf{6^?eB~1GyP=8 zA+RlHpbPp@XKP7;i*m5xp>-K9)Y(~lM7d|Ad;b;v{xUR6J*4lNyx+$Fj(U`k7xef|wR@)aM7n)~c` zsZG8fxCp|S%oZ)hOh=`E9NdZ$kkp>b02rRVN8hz52uS-dWH4W3=)*rlWiU}9K#ZUl zo1!>zo8&7^lo60WoSj%=QZpLxPo?}26K06!ow@){0Waa$LuM2{?M;ZM`mw}aU(KTQwzAP66)h_mK_upfh5FKsw_Wo z@l*vKf04*Bh+wEpdi?z^zj_$)FHJr2? z-g4^#xkZ#}N$+~}ceH>;OPbnZ!5dSp9+tsy=-lRMeIhlej+I(7TN~=u|?FwA?B^7*SRltkHbj z2_>27D46RWbN9o0AE*b&JKR3aA~Rh>jxJE20qaQ3iqjIo{7bEH_+uyX-;%&>&$397 zJp>{;BC}LseY#!Y|V{bfcux&BFS=W5a zYH*7Kwv0O~0c3*Zl-92%)oIl+tMImBXiD1gY29@j`oBU-!nquBKcAspH*E$iK;-%3 z|I+A?AX;!*fz7!A>_`On4SWH0kOv_eV)1*fAWa6E+E(G`k`mNAU*DFY#QJwn->Ap{ z=8OyIgiCt3l%ORi$d9KWlj2=-uf8`0)Y^MMKJ4eyPx4x@Rvu7Hod93Y`!_(30U@Co z_#m^4tyM=blpL$K-O4S=F2B|1dA<8$rU`{<`hS z8lp?ro-E^E3nE0uO_jg)m5UTJMQ_%mO2VT`7|M=~45fujdeu{LL(y?l4OUI~a*W|aSl%=gaR3*x_vjWp9Kr_vJdu(gO1AQc)`g7!N5iL(b4L;M4 zL3)G*XLAeq+C-kpnj}>OF9)#y<)ZA-A1G)4eAF^*rFr5&D`9MTf>cckb5J1`&d! zifLJ^C$Zg>Utga0f#Ej?=xvyqhEj|V6H`p|shAX*UbECMivq1w={^bqx|#o?YvxRk zX@(R?;)8dAr}Er=wc-x-YrWc(Wi19F^MCDyiC%QXDl9Hy(9XB(`TwT{nB@NW5?VJ8 zp_E#wD)sJat(*+Q#^{>!#}_V)S4^_wlxp9NI|WiSmYG>9D6zFaY&pO&SOq~xX5y$W z%>gQ?gFuCG=y$FGfwUNryL1-Pc9jkx{SB6dHhhuNa@Lc^tA>7Vs30D&i^-QFEYoN& z6@3bH`$7b~*=tbi`;gKv&LblUQA)*dG0y-)Rzqjx*m)dV!H86AxzN26^ZgxZvQ3?A z^ftB+rkenF&@mtnpNxDwwOq%;W$tZzd4TN3;Y0FgMSHn-&l6RJn|9dCjh}=V+|fns zLsPav1_x}1Hh*kd3|1T4RZE1L)BnO5zj1-0MDIq1=>MA4G<|pFma+|9s7+tsh&^L4 z5lqAf_3Wg%hj;h(rAYv8qp`ClF0QxsHq(t~AwC>#Cw`lrneuKdd_>iETilw;T+FSA)oDsP4 zU=D7~B!_2OpELogWNXYOK511Mp;=uYmaDq{BP(#4Df;!^x7fV>J7xhjJhz!1W;+34 z0USMFyEI@&2*t%LeEowx*^{3}@=rk^PYXi39S+rz*VCpkw>~Js=b`6W$|MWYV(~+tN|mKnzqo<*1DPOu>(T1?yoiG#wEwtpy<|NgcG2*CuHTm=Og=2z-?*7BXpj;EH+v&u7HzA z!gyg1kek%2LG{>qT}dFUW!h!Yipy;N3R-i{_6m_7l}8h)9|H-u?e3FYlq(-#d<6?X zbl$)*^4x+UoiEJDiX;qHy#n%Zxs^Dgq|w3=?-e&dobDI@(cGQ8?DZC#3&flz;{opg ziFE`e%Gdiz;JFpKU3Lh6gs`YeTNLKN+b+kyzl+-HX1)GkZ!wjJg+;>Ee{~rIIEYtQ z8vQ!7cjWS%bNkMu44(-Ars_MkEYoKVDX`STW)RfG+b|VC2so{E&3^!|JteN~N}hbt za@^o=EO&%yJI!ChIV8fA;%psny$R>Ao!#hCe<}MY9q@a=>FDwsBKx^~2MQ{N2D;D% zOU1+0^vnG;m*4ZDuqys@$9=e89+Rqf0MqVTjc35iC2j|5>bs4to=JyG5~w z|73R@pot#`S^^6%shlS=i6}CwRHyo-A+GtU**tgED`(KJW8Oh56>+VPes&uLat#{P zl&Z=FjHT7!I4v#b@TwMpSAdm8)G>_sjI6l05ViBjL|ws95>WFT0qOGmj+n_+A-P02 zO2jnP`tFoTC~#7e5zC{#$#77)iHrmTKVF#_W`=)5DgIOWE(cz!PFXYC=qzL1C)Hdwya!8We&kdI;3Az39Dr)OD{Bfy&>D=2?vo_PV721Ece? zi<~JmOKUdt$-~-4hD{(Dsu)tU=J`0$>4)JEXO0OPIG07x(Lp!U(;=xa57&P)3wl0R zn9hYo*0`Mf`_+j?HxBfZN%kH@BSaXMYf@lV9Y5&&WXxmUZ4`6hlX~Ld+ueThx>+#Y z_r+lCCw6Wq)2+7Zb>t2&wn^W(^+EYb7ub}eLnU=t$(?i~=o2rXF{3|QR zFR11$wUhthC)&_C6bn8&rWn~3+_saFQK*;p=j6SBPnq@XxbV18vNGJYDUC+=7!jPJ zH#^Ckp;U>qfnwkzes!L`W}XJ?xaE2C%fsQ{&zLNH|q{LHcOn^SF>tV4W0RBK0!|VoPEPXhtW(Qs_%bINx}yiuQ@<`Xh;LiqecvBxh=bxj*62?ofdkG`deJTUEl=l$A_#b~{KH!4< z(jtSr;>A%1XW`HnP)I=o53kRsUZibXPK`~wg46qX{KI_>Zzi+EgHxCrGH;LgyFxJy zPF`z{XF8d!1F(w*{4FxlW1v>Zf2+W7MH`8sd;X)mlji?YKBPW7c!7m6cs-SVmaX?5 zssmXdZpXFibs_I7gukO%7<^RWmpE=vXN?c9%2sIMh@e$SlwB)*h*R^b3L_)9=CTPQr-bp7; zp^Wr}o)@VF3`o1@S5e`kyJ_uvOeY$=H(Qq+|Jy%Qv%UqU!0?jP7y14nM7QU64i_en zGLo8fultWF_pNGk5&nA$QGIEgrC?UBTQDJwUQv5!iUGp725XgG>W!xhzFWIsI887= z5-!nkoh-OlRmC1;w)Plw&qH`>%4pkeL5?80HR)9%6p=8M{@84YQ1tbXr(b(QOtfnI zqi}37n4%!UBeF7Z{Fa2lwSndyMeWtHHZzV&{G6wu&2dGBiul_zj3;9{`XaG=yi z^{2Xx2>WG*4yCS7=KflR#ZQf^>v1cPxo$^Z0pr|w{hFstgLt#$0>=ZeGQVN2o`7ZA8RBmDct%m^Ou(T zZoM3Ox_hNFmv<-m7ZgozbmRI3K044OfjT(SF_$i#|LLr2L>wo_VU+LR%VqlOHa^Bs zy4&lcEk4Mc#a5%T+z;6-q(;q7L=fvL+8RwMZ)o%Ar0JBF_;q{1uB-55!V#l8(VpeK z-YXHCt|jvrj?3Lq^;jOD17|(_^bU5n{TVXLd;qGU`!Vl@ZQ;C;yUJq*8}-<6-9WH) zY9l9|O)j%?{IU5AFI0vtc%@jRoEud6UJT4G&l=pSAoM)l0D9WX4&MU+*F3h?JKvjC zHyaE0js5A1Dz&^*5TIt!F|_9^2f1<@w~&nRo;=nRCugQ{v!9*F#u0s^6zDVY-WIiQAWto*@H-m&z#p-N^3~j(> zZ!8qtk^di*-S99gIxIa^YvN&CnKC;@ED_Jm-*Lr>e=EH;I57<4CXPIRp*d7Q2-G;k z1>Ntc;tE@J(P{4VM^Y8KUhXWgw_Q!j6v7;xCr)#nc-7eNM*N572#_#n1g3P|Y>so|`^$1Hem)}feyT69|L zeq#)tZO+jze*oQtLX+iugw2d{Ik{ZQmkE0IGh8vtg5&zJ>Q|ueN7{fAg|Ay!M}Hj; zksvH~+ee|TL{ltZ$Ai0t&o0ZtNoOt#S@5YHQJ=^|Szc-*8pG1c;%>XrjPYZHq&qz{ z&Ar}6RaBnyKPdc6hzcHRBfy>~H1p*k!_qw`0!PCdS-;{qNp zEnRo=7QVadz0dk#DHBJS7WS%!h{;a#`l!2{9H2Q*2-74`6(IKq6?+s)Y#|gNBT?*h z$3rKdm~v@mihKA?v$f^qEEUDXo>-~`>Rt)U>XhTm!M&r1^!7l*Yx@K5`^C?bs1LkD zLy(Cr+!Iq5z=-Z|7pR*j@08b787jNVE1`JdbSFB^D8 zD%C-eJ~pjpi+~yM`Pp_9!NOx0e!=c!JQGpyYFvzODFW=!pdkPx0Y1Duo|-RN)pM)u z#Ot{d=**srG~O;=jyNQF^q^vjdSLiqttG((hiXrMdb)qzaCO)~MllR8mc>(i7VDTExZ0VCYq5pr-KQ~dT#mNDyPFH3m*d+dCdmNU}n^)(U24% zNl6dUrHe-nafs`eUfk?W{IcBY1q#SArc1SXft1f5Ak~YSA4xK%7z4p{HFXw078~pg z%Z%|&rO|`AGUN|?BbM5Q{a#a1(|^ufZAU9i&gpr{>9Og`R-2Awg$p?_z45?%_B9zX zJG&fvoD-RRf&3(U2uB*CD}`w?r?W5-Wkk1xG`M|J(^bHN%;lloy(6ureUdn)H8=fF zkVGGXWC`cE5J)5u9Ld@sI%r#X7v4dqO5)(!hXq__$QXp<@Tl9b@9X&g-r_JTS{~AE z7b{mYdR&|flw;=P&UXCz=mAWiNEv?GEkNZP`|Ad(5O~Ubr8|?O+rRnX24Z$WdTMlN zy;xHS<#q=Us^l@F70{k(S7t=dLe_oy)BrbUJrC1~c4VCUNthIn!zOK5_~$im)FqwJ zvn8NzwSoIswIqh`-?~4i_1q^NS6J+HzQRlRpf-wg7S)&~&R<&Rw|mmgdXo%Gr}eA9 zi`$XOL-t|yx*}V8@6~@=o>38<0C-YSOf23DoUPBM@#0&Dg2lJ7)wWobZR5K1!Vxde z!+5F(4fa=0f4a=hbP(+a%hgJgpqSJIUC-aKGV0_w?7H0p`z&5+6wHQOvWMM+X*66F z9xep6UdyqNKUNU_wkN}k`>dcr==7`=5V8vhA-BtC)_(&2i@ZV(laY7PG7lb%K3+vg zM=pv|A61naHvsr{pS7PGg`|g2h>oC`QYJNFh;!ksw!S5kP2}8YcIDp2R|7&)r3XYo5_N!ccZ%s4E_-67G&=iudsQV++S7x z0=}b(f1SZ0DNMIlB$CeHnXiqpl71fjIx#$HaVf9txpq<4f9<&|%>F~~Q#K;V_g+7& z*q$$R+~Y#B!K5E3V7)SV$DLM z0ZY=QRV8x<0J0(-R@O4yf}Y5N?(wS0V}tz<9Y>-8XMNHda|*3&s??BEi7{Nq-Y&{J zgH{LM;9oDL?Zy|*T5~xh#NBYEEcppDlyilZ%~8f;FOSG;`WmkY^JzJRNx3aq+V7ysSy+bB_rjV47-V)Dp{p= zx{x7fn-+2YU53yz8~P~QM*?(z%Sm4jfe2Ke#+C=h6}vIPyf@3#MW#)`4jKW^z^&Cw zFae~D6*wNw-rEs#T|2lId?s{2bA5l1ncn&1gZH8cDj9zB{bABr#GIYP2ay?|UKA1d z4o$3p87Jdnt0{>@b&W~?0IKCHPU`VVqI;yLXVSiaR zp0Qhb=AF#u6-w~HC+vHExDF2NbPsnP)dKU_Qd^(ev^{;tKpmxRsOTDzNR@z3NXBzs;?HCTaow`6@!_iiTpM^#t$y46_HT8NGaL|5-){*tD=?&qRhIItn zKf;4-ZBJF|nN)`qpMQGa+buLX+cX{&x$`;^Ion9oFWXkDzYU<8DR^M7F*vtZ`GAj8 z(ByKyxv=SR-fZ;j^0vOztMpF7schvpWTy_+8he|2ZrC>kdOR}h#c)_wQx@d8M_W8> z_b}TrJWX^!EE!i2yjH!$R34b5F@J8n-m&Svv79c5bOxOZ+1-t<&NZrr4<%iL z?^v0!NjM0ljN0x`565_CJ`S7%5HPwu^cyfjShdqoG<@B;nyxu5Tm|K(s!5UOuo1@>31k+Ij)dr(Kqf)p>SOcwxQ@KNIZ32DIykl4t^M6>eO+|8Ail(!8``YACRpU@ zF{__FhT?Kd$IIblx1EGmiq(1IVv5rcIzAhod~j_Rvqpk`iZjI>qKlyjHjd3$ZnAFh zX%x7MP%6uXljfY76evVeMx(Y{DfCl_U~KZ}T2u62s`QGA>B-KiJ@`_tgpqVJin`Fz zGa}mTrRODhqKk!1`IBeax4#=_J)goWi%wfedTW*5ERgCleX^MyYwsFAmQ4@zWdP^$ z4A`i`pFwter;k)%(O|iskXSLcgm(JeY97OHAMXya%f0>MduR9-xuT zp)+;=_3KM-ujc)<(8-(MHlnnvbK{v}g#e?wgZq{habbVm@Nm3=cWoE}p<3|T{#q~=(tl&p_tEP&8A4?nP3QtV^U{rl818w+cidY(&?(xV|z6jSPyVa zd?D(N4K-2sy<;XE3K=??zw5;pHYb!%+V#>!;Ca%&qo#}*3Q`2A^$P%SM>aR;;Ir_VC8lbctZ zs6Q(OsagN8P#h2xuLU+jc>5Eo&Vy-3G7oZt`YcQEuXx;}&cD_TiI9yY{;G%Ttu$fcs^gA3s|D#N2-E zT+@-phpVh=LXC)PIxcxz^vzs3G$%{b8JzY(#x-v;#Rq))e$Lb!lfE>EM+G#q!AF=b zS;uWM;@eDewiQ&ng`mvJ7wAp|wuRp?;fyL(&1|lFQEL?#c|PG6g|lv`7qPXVZ}WV< zR>y0wThnpQT}tB4Mn|QP9RmLDK3`N)fs5pApI`L(!04>$RU$#|N-eU{bIm#5QJRr# z)4Jd~b)P~jXSRN}fy^8}Tdt=-%43;PWjwSq{W`3oFcI3~HZ}O8I0TZm(e)4Ssq>_E zPM5E~M<$k8%JHq>w1)V3Fd8rQ>aN|lHiXJLTNzR|5Bh>5qGX5}&eIj3Ab&f^^!Op1 z3N+pw8Ptt5WF9`5#lWx5t#}x%uvhkaTfdyE*M=`_mG{h4N*miQB!VCXhp#QtZ7}}% z%{jviC9j%A-=G1p44~ydxHH%w`ejhNRVCJSir@SX?jv~rz^y!6w_Y&n*#e++`OY&n z3R_Q>9It*3Szm2zyT6`PBy?hb2UpNHnerX*?lC-E{TYV6ZO*pKG)%UtJC~1bl$aax zF;00mc2YIO+FSHbWg!T*x3V9-@4doNwqpeJ)6#&-tb>lByRou+u^n7@p2iJ%&b$><|&D_ME zsUCZH*YRf2kK*CPj(FQJIQ*(5H1EaxnB@IzXVZjobTIv(Tf0+a!|z4(&O{m7VIuly z_?ev-NM>Mq?_swtMwLK%PcpN%sE+%Iy*b0swm>GDw`0)XDnnKdnt2Q7NgB<1zd{Vq z46Y34^9Z%~S_ONszEwOhcL!`)c>}#e3`ppN}c$7dz;XY*oz<(4~Vj$Uu_s5x-X^$P`GP z$gpZK9oFsgFQ4kKQf|C{SsM@>Vbyc89M1sH{kl7u=9!yn@8i_{ki*pZo_ccBSX*~d zaLn}Bx4Pl#^R&m=;jhI)I?_a-Vja3LbXT<&0Pnc3;~|@t&?GKEx5aACl4}6@R=uNC zzl_KI$R|Ygc=+djxf~})ly9+au-0@(r~M&x1rY90KT_T@sS7YccY6kBbl*&jLihlA z4&$Ri8<7~CH2(=QPw6XmWE28@yy{EmSQebL7Ne95wR*DQEb|j)SLkGJpu6$nXG%18 z+&H1#ROd9PV=xk&YV8!*P`oLkm-F#- z;MX*#q{nbio_`acZg3>o)X&E_f1w%AW$1`}JGQbdlIjI250B)%f#u z<7?%SqyPZ;{#tRIOAIEvdBjV!AanREM{z3>HpJVI$`4@ua>OOmuh+YJg_nW6Qz^vN z)y85@6(z9B5)3Ulbz3+(R z18qv^&aG_CXQJ{-JnraFexm^Iy0C)_ir)vB?y%4m5Iy8(SGT* z!J-#Gm?<&k2~!z0j%ywU1%pzc>j;zb{4t9@Wykg{l@SkTw|pqyfa7F7kmI93FN6}z zHLFyWDp3AWMsJwiP4?VILu$Z-23Zl=U+8(=hSf)oy-nwI^Zj4O!7qwha0Du9gPAkk zH)omzcX#XJG#8}VQ>%S$dx46#~?fy`?oDrE4YZ7wN|tOoyg;R*avgg#gDA}sGKH~Q9Z=P zR#pZsn}cJ|rax_mPueE!93eQYmXy~=3!E6F=QdZx-xGxjjtl=ypi*qRQMadf*M@+j z!$A)PeEAMlEA!{OtcR#;S^>b2@vBr}PWPR!0zK;G8So}@j&%1FdW4x7z>zHzTP%0W zxw_&APGm>h35BXwsWNCa7?i?umUPcu{d??oe9R~tVlnTR_P-UNxbumnYT^?*xKk+@ zJDVV`;IdAEQtwFj)mtivr5!|1NvsLPo z5QNW&a+efim~0plPZQ@J57`ks2+$E&cQPAA)7l&2sNIbMSb)`mOhF!ik416?(BJkk0;5>R6xmw zm8)i2!!^tQRQ6W}Q{zg~?>WL+K?Rs#t6-hD`9MuCFxP(s4-e1Q0x2okMj}zH*V}9MDNqW` z^m)p5Wqo6SouJl3tIc-=M9yfYZb| zBH>*dR67TpWPHc$K<(Hb{rFoC{Z!%GDexKXW@}p(TNm#<#6NtR`wi!Q@TlhXbTqW$;sGOe^WL}bJii$3u)e^Et9mPX)t-}#95 zs*iooIbi&Qz*li%Z32xX8(vW>m#oOxn2eXw=>D(3VgjME`F8b}C>)x=NkzQ;u3S|o zh5{~WCen5@pODiXm#Fz~PQ)b0^~`z>ZY@1M>f-@E9wYc?P}56>R@iiHDMi@Q7WUVp z3UEaSk%py=4^SSxx`+IwV+&(b#7;VEKE{D-)+@A_-n#fZ3a;-w#JIy3SrxcK_WVrT z_~$*!pUu{^{;3=%p-v7+kQ;33HvXcs*R^k>3*C#xVu8>2#}h-Z)T!EWX*BRyp3K45 z!SN>E-oz`q(tcM^e>1;$FQF*W;j8kf@pki=JK;ylpZLo2kFzC~MD=t$aS?*)r{CKI z{aRVP8k8of{P2(LZA441_|u=+tZENbiYj+muO2B`(A)Fhd#~G}ALmOgNA;HIC}Pj9 zbf8kCkTt#7jARCf7n{7a{Jyqmyy|T-yWe#v+<50WLc-M!vm z!ji4C=w3HtjOl5&U^#+6Snet`Sl8PNYHIU{^$}wqqQm-7dyA3A&kA8-vE6;9tm83S zvDbaPk>*o3G=5wiVTJ2)gxr{E>pq?03*S>;5_ymdd|`t7YhDL@{athdTft3+eJ+!Y z(hhK+HE0Q#)MQyS8xjk3n(CCk9;;MQ$3AVKYE@Yfv_M@d?R5HRk6c7Vl>8G2CE2d_ zMC9IatxnJ(A@}*PH8F&~7wLl`Pk(o(MUwrt{!PAtQeg9>X+MT4?6bGU!R?wUBGo7C z80F9T5?T$mlmNzThlr&9+W|oH9`e)WW#UW`0+;q5)Jpd{(gZmvvHCLS{u z&GcN`65RJWyr4u`UVjVxP7;$ux-OI*jA(_@BSl<~U7Ee(Y0Vw;ML(EGIE*9-8C4fe zD85?OFD_8r?xuTr)?vnUd8Bi0ccr+Xg6+~l9+=~iXx{qEh~*Q8&bv)7UAF;BNcB~d zu=NX?p~8)q`=1zgWI$tknx9w%%}@u8IvOHuzWw(bp*N9mUT0J!u2$o3{tz}0>>3X@R9Rw#jbZj{f`%=qZ14-X|rou#pHk(mQN$+n80+Du7W@2+d>a(%UIl#Ac zH8Ah|Z7%kD_9tw+BtBMfQvy(F3@$QL-jKtV+clYBLT1J@LHsX@x`w0jYhScs7hxC^ z8jSjHKX->hTVfor9lO`@!lA>-fcmwXcjVDI#=8czd1F&_S+5fxqL2YEL2^H64}N~5 zxiSxvfh8%R5mhtV#u6jSu+Axy@B^kOe{rAbqdh7w5zXQSOscv6T>4c6u~?HBj9o1}cRC|RSKG;Fx53ayvyfY=UtrVD?x5+#W^rzAKxOmvvvVw< zRVT7z4yU%B9NXLBTVehBCaI~Ns{@(D=Zpd5j%U%X$e7Pwe5gY@4*nFtKC?T4Z;1?xGOZhV{XBhzgm$MC@H+lMIQ*A6Q z;eM3ripBb+d7+rLO2MnjiYDK?v3c8T;PEB*-k;?fKSxKW>lDEl4yx6@a6})QC>$b; zM6=}ychGu`^6e(vp?I_LA61CuadUi+MU5E{_NoWFP_E3ixWN}71{VUYE}z}L2!F_b z=PL3Jo(ZH|6Yw*$2aPwV@9J14cAb`POcb*oKO<%v(H}`MU_*-My%511$^n=93n07^bw?C9(0>Wuy!2X}N1e7fu^&bH_rYIt{c2dHQ)_e#=jNcI6G~Ct66fTDu z7j6;M8;kr4`Qd%L!C9Tgi32D7m`Z39@3FRlDqz57-RAKfuY=?3-V$N~V2T+fEpS=$ z0-V-0m!rgbPnbSst?SYxkISTd$4xs)kiEgja_)Al4DXfw0n#{ci~PE;BQaek3wJi8 z<5KHJeg2C;DN{A8&{?a{c7#}!l@tO7vBtW-(+o;6TZ+h+4&))WnYt_jp|Tun(_e#u z*?!F*gwXF#kLRMT+m7Ons^62sVJ&SYatdpnZw9kXCM5~&G@$*`lv;a`HMOvx-8Zp> zKODE|Cqyd~9frc<$Eq((?>@oPAbfW|oMkb6$`ASd??mvG^)eB%dJeuf>JnYlU!1)< ziw>WJj2UmL3?Lnr1UKqP9|LtLZ#?ga$yHwJF$m8;j5khhxr3bJtE`7r(i|p#{6x<{ z$FzFr5K=^F9imU({BZI|@!~Rq#zFRp9WGbNH|wq1iD(CbuMm4N!h$YQNyn|StGO6; zC^GKcyJ{1C8mb% z4wXqAb9e`c?*xR*1&7|(*XsU$xk>H>gkE(1baU{d6?rVdgf|r0>HwZJQQH9|4jcjuMHV zN~%>x10^yY2mCfU@GE7!@tWgxK-pz-^*aMM#Iydt)d!#-kV*b6-7!zhA zD)n4BgvG1H5{+A+!d+fKa~K=!-vujOd;Qq>^g^lnI8Q4wM8ug#fo#y4?hux!edj@? zQ+ogKI>=ORv=SRXpNa2Ldv_@db_sXuBDp4=NV(7kbX9<>RF40=%?bm4anh;}yq}UKZU*LNUvIOYR`1SL+mxs- zHMo)ZfQFhB3!G-cpBQW?)P|W{mhX?@dB3^B29w90Y*&)F_iG-m3oT5QHN|-{YRCNsJ(kQJ_lOhe82N^& zuQsq7GHfIAlVI>!*AwE}d6sfh#2;3VN5k~XG)^#8;A#_j%WE26&C%8+^!itHl+|6t zX=InK)9c9UHW|^}eQ}dYZO5_WaRs)shG(3K$9hdGJK_X|%GQL5xIt`va%3eI>kq^7 zphIS~PiL}g`-lYwFhRLCI0f72sb&S4SPovY%=bTuj~`~jrk`@Et3LP5P-yUUT=#c| z-V<@>Jo0+x85KtTdR+=vDlMw5is-I$_g`v8FTLTd@H4c6!lh!RQ!7@K)=YS5fJ*h+ z(&F6PLh?L$|IB(+#|^rx1kIM+rM14kxOG~sW0CSCxWq&tdYZ*8J4-AU-+)w$?d7JB z`ED+5n7|{o`<}!jAGlLj-}8NC`ozBjXY||Zuy{XIKT*K4$Nv&>8Ch)Nuic6y>Hd0` z#grypy7{K8r?P>Pe99;7{l8TtdQK=SFShv~4nJMV^yVj3d$tl&xs7&6qf0YT*TqDY z?WVPDXp$_m;VB%P_fd>B=52&Y%Lul!g@{9Qb4SF9HHw$eh(a}px0ZnT327++pCEbg zrmYw6)KmX2|7s>IEv`COrrnt`w_y3g?lspQDgB(ksbKlc4=#pqN3zH14X_-cH>_z@ zPU`v&cZ9I_Ivw7NQcrPB`pmDNDU?3A>G|IbV;LhcS?zLgEZ1Lgmkxb zcfIHNuJwEWf?4a#Is3l%-q-cH@;22R?(uRcEU*VYY zBk-9>lvppn&d+X3nMi;)LwcA-$$YhWtIt6I0kHc)y!f_LM~VJ@Ed>zQJmqVrE4Rf( z`m%m8dbCi7O6YN)fHTeqXvR7A%E_N4Ct{FuO&(_VVs73n)BUj_wZyqP59|p3H{l&V zDPfr{-*;Q7P{v~?f#5I}D}}{$Xrbvb)K4GLXBa9-O}X3Y<=Vv0p;G5-VnoCrrRVYr0Kms-5ja<5nIwu=*Ml;VJv^&lV)o_2%BXw zl%80l!a5APrmcnAydL|5E%|$eJ8u?kq2;GX=nG2n`r_BOYkjs0rXv8dhrH!cgXByL z8)3(2A6ZEzAa?%MzG)ox>OseC8~457BkqA#;|VXxtOVg@`!pBsPqF>YZtiPfL7OIL zaTLFw^GJ~|;LtX!ez;i1o?0nTKt7fe{~ei*t)X>s

!%_(xd0%z-km@?HX#RLgM45r?u96tNC&Fqi z_W{lzEs??)3lZ|rPSj(?SMrj=DvIJNI5949ljt+HlW^PPStPEVK;FnYFd1nH>fv1+ z0>`A0OUqFWW06)O5bUZv8VbGwc%t;?9fQT~AjBA3Tuuxx12-@v=NoCwx3W)_lutIb z^hMPF3|jLKSQqO;mj*;BpP)XK!?5NdlI{nfEx_@8zO!rsEYtC>^lNNbKL~Phv(lUh zoZi!LKzF16$VKaAA;X_!e!rboQ-| z?>OW+`m<}|d}|nxaLi}<9B^CtES^=90u`7|1VG!h+MOyz&KL2B=U>;#;=}}(Zs5p6 z532{;0+=#(%duX9fS7kgs*dmRV&la$j+em6mZR3)bADw4Iw@Xz4)0>*#w0o(!ljk% z)1=lG_Sq+}TCB6aTmUfib>I$+sic+nnlo@Oog`h9cr0jF@oh9cN_O1alN-FC$(gS) zA7){Q>-vPr5o^b0Ca-beyF*v9bZ)}*IC|h{onBk(zig$6kb<7ff-W~>2U^ja4unHQ z>w}2}csaogl|9dw+_k(6)@&5Bn3mSvL3e&CL6+(n`qzYu7@= z4Ek%l9fJrT?!F*DUBNK_ha3lUVMdS|Kb}Tr6XM3tkiMR(j$V@SRv6x0PfQtkO}Ux3 zs55VLLB5GmTTtiRv6H{uY#Erw!+Dlb&iB zuu`dcsWN40MFJilCe zJ;;+$r&(mQ+W@;gMr*;Dn?+XJKJsn*E3(;P3|0pn>Z_-|; zXhgrJ1m1~?7g;J?SsFD#Om9%DMl^K>emc^8f&|=Tc5!D-A-Lz@lUo}xVOvVP1{OH4 zI|w;)cyD@ZN)@abmx>us#62#;y=9u=cb|dC@w|@?8NFGPnZv0{c^E2VdPP!XluzLK zOIxSLOksMEPkdIPd)lkd#(J@WdzsTss_r{Jl0I>@Rh6-SDtB!J)dxJbBM*{g?u{62 zJ`5()GEuz)E=OWzaxtbB7nA%iu%~8{LyFqFHrDZ{kGg)OH~IV)JqDl5xY$N6zv@$t z*;EaMXB)?9DOwKQpse%`N}uBD5dRY6UU29W>3vRd#?l-&{_(Mx?eS#25z@(vo+H!S z65bJtL1@;r?zJk&{LvjD8|P~c@A6xRF%OL6&h5$Db0B&Wf3DQady9SI?^mwVrSDM$ zmab%c0Ss3Ma5d!|nKVWI`8;OO$LqOOO!#{P#PMh{w13Xyhk83kUi6Q;ImBH9C$P|O zN*XEpF5>sF`>O-vyKT|+X;Vd+QjD`&%VfK;nwL>hd3vSf&>hQmxcb=P^3V`(gwCbJ zQC$C)%SGilx;>HFfCLSns-Wvtu#avw(y*8pzXo!^yYhZ>K4z3k^W}EH)0`Qm)hJ9E z^f{9~{LvNQX!!Hig-ObCVUUR8z87rN#K1^@R`p80rLXTG;A%e2*r?y$X1m4F&C;Q2 zc=^6gg>d{wAH{?AR0XrUe`!D*^HuDq06Z>+kjp0H#OYsU3DlkL_P|5li{>x7rQ)yS zGHVxOL`&H)sBV}(EWc27@%Zh=24hzu+^o3jwP6W-x(NXAhT+d)+cO0|wVAl&EE}L{ zX>3t~>Kgzkgu5%~=#WU=ZXQfEW7CTNo6Wamb-4fK_`zRw`8$IS9Y`%aHrkPfUAGG% zSA+b~DCb={c5sAkHzQnRo&%j!nvjV?vjj!OKW!NR%ePFqNl#GmV;q#CdGXF~re5~8JYFi*wC@wCj;@CZ`8f|3Jy=ieLyRzz7PXOhs4STjVDPK70SNloW}0+EN)8d?ZU-;0Z-ayHp3?Xo zu!v7xd5IE{PI`5DKlp_VNH*ZB<_BkA@Xs_>>QTVMd0^@=|6*b?XmP<{0=^nLptMd5 z9+LP1v3Xk_FMX7|V;IyPTS|p!GU|bJC3$LnKI|Y}n-~+A_+A`e%;U>@7gQtN93tW- z)`{!!WnqNlR@%+N5bpH)ub(W$vitS%4n{1Dl4DbBBm8AvYGVRfUuJ`w_sf3dwdgk# zDHMl*O|5-(-7nRd4W@xquiQIC3gd2o{;o2uRj!STc$1yhio#}>fxE-3SBQi`;mKYn z{^Px(QMvio98w2>Dd?X(O9^ri&DOa=t2F(|q#S9*DZhL+TXOC%aKSRt$cSSAbuL%HH~@>9}n>Y8V*>n$7u{Nkmi#-m!O}TjX9Dt>=N` zxXH)qQSaE380qGac)Zi~V76qHY{onvz5O2ZYwJnVtJxPr4*EhG@y^L=R%Btn$Fso_ zF=DWv9Jv#Ma@Gy!W5>AUA0dsOouLEElrte4nBz_OY39?PBWK~KRKFu>w?Wtd>a)P} zV;h?ftVQqGe?Qc0`ZBNH`}9vS#IAGNOCyx(BVtHr)4MC^M$LSRj8*Z}+i!u<2V9fh zudDQ1)Cc>B=8k_2;tx zne=qe$dh(^DwGOf90W3WnhGQ0ym2VkjB~!J@^7Z%42uRMVnEU*5nDoD*yryh5k7oa z9>!vcyTXA=;cO(mW{g0Zi3#uaI*c^+z29b>bz%mczjr~JIVyw5^Sa&hvG8^W z15^PEgyP=!%$TldtxqI_r^dh9PY;$+3Wmp?%-GoPfe1FeuDF}=@J0l8TK9+X515C= zFC){Q7I_7vY6@tlpkxNCcUS5O5oMYac(&a1h4@@SN(3Cvh^{`G_X;rVWJyIo8Of&3 z4qXrhkfy215shEuUIV;+A@`;~`IJ3nep-%a0cZNVcwWhGX6t-(2C^HFHOz`!p*DYA9f5F2nK`>d>_h&GqyDO`r9zI~ zn%Z1>+v{HaU18;ggUR1-BOHiKhtc;1f^35$q?f-U4i&<9qB#pZmy5oEAwqj)p$SP62Pu z_x`bhwL|34u6$ZfeGvck3+PcO;2y{R) zMO%qqKdGzn;$dT-(3kn-SwzWlD~jRop$?sfsa))V#tJpshzS1s^#?sw#fG8#^yucL z^fEW|*VV(w%(tDel^a+{Ln`wcP)v~jsk6TOY?w4S8cq4{nKFtXwYON*%sC!8VtV$9 z2J8`xUgo0U7(tJNx3-^zf5HpP7g*BVeFHr(Hvu^CH0wYrZBa94>w-4C7)(#*V` zNS|$#TK}oOFJHS#JX_@T__K1ofHtC_T=QhLeejJ>E#7}&*wwo6&7WH7~zCoFLhr2ol=k(c@+&7<4 zFK4;-p#fBvL|?|2qSD8jI@ai|RzVV_E5a@#!-Q2uB;L=SB@RP3X?d3K{Z$ve-UEmk znv%q;KtlmO!uoezzh5KX?^focia!eyK6H38SM+^DPK zVwBj$4&T}mN#+DmQKu0602W22$9rQHFlyWIOsKvX5gX8B-oX{;Qi{%rQ zG6H3Z1@aqbMmNq0IOyYkU(51;8p1eig|ozyH;Z|TxL(qE8Pe#PS=`dg2`~?l@@1m~ zftbFxgaJ#x>u{5&5OSDY@lQ;VvUXT?;G<)PAD7bMmg;S00s(5POrr^;eXaK**C`c_ zd4#{5F?jEssh@JRc6aHg<0Niyz7c@FX3^6x{|YUZ%#PXVAITQ~@+SwR1*{TO)pfN}c>ZgAsUulzqzop+aBK1l#yv zty4T>!c!n6hmk6(7!OFE7hlf_!it!S(TK9Yzop+@tarh6yb>~`VyY*RDEWircauP`}Brxrs8XL|YmVEf(D zX?56<{J5}kccbT*{dR43%qc9Bx32TJ6G!K4S{9Vik=_LtnVPwy;h{RV3dZ7Ov}0RA zb)SfSuEOy*R1k6xonpxPM+J@I)ds`kEB2xPvF zQD0AabGrYwHpelXi`f#o6e-*_rD`!xD5-Cz+G1z|T%ZSkDBR|a&+cDeM2}^X$%&!7 zN_UN(mn$LrdEyU-&s+3r2$670ot-NJrDc3GuW5ITQW$;?WafAKN|c>tMu8vDB<6D; zCT4XZ^BDC2C+>G6-b4p&hFaHAd2&AuZ7D5_^T5!P2d`eE-q4qdeJdJxy~cQ3OZRrK zP+{~(sj1Ahiq|$0NbzVj*VM4lpT9QxYvgr%uqLSRj-QAgcplD7s>VcKGReEi>( z#=J`GM#z@(r#m?&CVPOJ7|7HT3vjPyI3(PV zm=x2Dgc&Ri6DG2lroAb_oMo9 zX}IZG7HxZT(Y)1lVXd*ueqnBBi1<;|)%i)g7$7g{i2LS2EpH-*3ws@)Di+|gcK<?qT-+FLFJ~jJ99ovcZCbB@P5Bb5h3x=LVhz$*ND994olZ@6NYyf5{#J@+pNgT1|xxwx)_c{~q6^Mu$+~Lkf3nM-uwm9`^SbZ#(=;Z`cp;2atrDS~KSSW)z_deRfq~~Jbnnr{y0n&qri_(eJ zxLUqT#AR!r9cO@N8^8?wqlLcqf>U9eIW*=zlOdIPvvT>dFKyE`*D?Fr`xevcLpxMt z#~0ahdbP)5@w$KE0GTJZ7K-cJg09V{Q1y*z9{GY#mB;? zry!Y9;I^PturO2a!8r{3_N&l(6KT%xLT^wG&nm_JUo!L(Nv5$kQnK+stWyn89Br}1l9Hlj z8RECPrwNG;2zZs2vl1G3g#v;kY9QSEzv%#ku8)!=^_#~wYW<<~Utx)|FBx#tHDr&A zGabnkuumQgJB#_kG8`T6X%JmkwQFCb6yb7CPiP-W$hhoBw&9c1sF|35Qc}G|ODpWS z%{2c^xf~s68PnC#M*s`54Ro9~LJMg=yqMK<00+Le_~BvqQDCd>ZI1O*;(AvnUl$+^ zhXAvbx+1fgJPk?!H5low^~u)i(@6ppKzIim{ykA8&X31hKy89j?a z!7=5Om9#_8!wGj?2ZXO3>)X}N!VCiJA792o=8T5q>BC%&m<+O2WapOdx6>%yC&V4E zWK=f1BtDjX2%k4@An;>z3)f+!57+5>@|r*ctq9N-2v!?l6585t_e&$~;_QvU{RF&;>`33AUdTj_ zr25=E=MgA(W6%~!H=`)i$vDmHoicql$w;GmX)zmCDI#9i^aq8WYgn*($h7@idYuqp| zMMM)R!Vez1kxBeE)wlyfW!CJ20bYZ=F)B@v-OxrXEz)@TtEC)InO1vx>ge&MYE!v? z>!v~1?9Wa4UIgiND!rt?F8_U1ycWxDTSWHeBuBkaacH>z!K9KdC!)5QLnV8< zsQfFIxMG&B18InV#0e1lNXZIHOoSzd%;bK< zCmNO0VlBx-42S-u(>e1)g$`qZdR(ligtxvv>q?9c7Ik#(RGhcFxNPjvBw{g+3R#bb zh`55bX^r==ZJGOZ#WVy<@~Zri{1XECprmNp42ZVOkPwGAre9u(6+8LZs=<6Te9BxV zuV76Lkdy&2fTm$kHEt0Zvxp6Liy-uDVk}@&OPfWNitJ+vJ@#U!O|{tO5%+>ugf4e;W6O_#gN);|TuxMmK&n zXGB%4UP2?___ukkPZCW}T zNx2a|AMT&Pf51&wQWOs3((NjaD`H);PZ z4Kb+}u@hc4J99C6wwjnZgip9ayBaAXht*cuNIA6bnLCnc5ODfGM+N+FQD_})+ZHBfN~O>dCiZ7nPD0GJmuK zjrW!n7idXcVzk;l(pi7Jk-O!6w<|0RE_g6b;$o5d+{1P`G>V zWV*kO%Ns3B?yXf7LHI?Q*6PUFcI_kiBXbpcF)Fb2F&c?=J-(Pbg+s(K41;_+wGj{> zMooGC**RLQkJQjwNP7$U{Y0=MgH!h!iY`a#X(mDtXqBv=Foec+q5wx-O%;frVC3|j zV5JtQ`C+NoJ?JEdZ1z9YcX)O^4}u&uW)$No{)mUJIr2skbH1j8lN85DyBP3s`~BqJ zbGyW)!EW+fQdu+%Ge8-^F7C|OHYE_(J{Yn~!m8W*gF8FACG$fistC{1R?1A1)26Hd z%pZP(250*I{w6QJZQjm10=`8xx{b}Eg(h(yxRh0J2N4jPpFATZ5xOa%&axv#t#^l} z8rnR&;{a*G`S(2ZXF`e$pP$mXWCc+SzwL)k6!LBh(-no%-}0Z@j1pz1-DI6DUax&B zG8kPGf3`-}_jLLp(7w~3d=nDT7#%Z9k(!>7%d@xT4vDQHX_zUp?!CKrZw48ra`HL} zQ~$UGxCucg&tsRB*}f0RMc4bt9%QIU}@|qqnNIH zFx-lVjE{cO4{s1n1wGn@$##hrMtksP;e_;! z_n;d>O~kHyf$x|iFPD=avkpvaJtyImS~h*a{?4Cc9XSDeA{XE3df#Z$Q_{)r27sVF zpAg#9(U;&H@NRE5UP=&%~-R@u0sf<(6qZBb`n*g~w`q9{{T&8cIK)KIL+`=P-EHGPuoOZ5YAM~ZQ zi;)yv^TK+~t_YT11oE2xqz*6L8%JZn^rSc0r2BB{>S=N(UUgl|Z@gJt*>kv1YYfO8 zIF}IMABd;r7!>=b^t$=|jWjh8l<<*NLz2nu7a8odql}s=KRmCg_>ndfy3R*Q4rx)LQV*v*u_frU=y?`3D#{P()(l3vcfxkJUn;Fu-FGaw zY8^$UF*jxUb~kPNyBU?y!}$j4-*|msWolS^W`;>5V5g0Y_pB9hkvNG{cpp|*f5xko z8Aju^H_wVsvAOvDwlYfNkFhdhsvqBJIDEg$)1~zDSAc#b^KeyWeABx!r;n>LsN-<^ zAof5Q)E(Q7WH~4NC-w6;yNBmb2e2m6`W(CPya~TuCJ$$i7XrT?ZcwT_N($@J2c{{i zXO+==V1Fi)BUEYRJ{C+toAdaO(JU!0##G@q=Q}4o^ToRxNDoLvr0e$jpq-3%bP4D( zEK7CFz14fS(J#re}KPLTiyRyo9NLahz z-Sn?W8u&DsIgoijdEh8n8}*@Y-g(0zMfACvUU2LjP|+ZuKi&{l1*%;g$YBrm0Zv(T zoDp@d1t=v_{>pIK0v9$H7NTl}pf!P80{^g=RR{N(s7qv?QL310gL*vU)khPsiSQr6 z-`z|^w*_E{Nz{GXhaH#{vVi9At#^G;-7Cj z5eV+e+X*LVGwVjUf2!0=#K)@Co940B15SWm45z=dht^xQeLdeGh@R*F^BN%uQ=ZJe z$Bl!aBdK#lYYZuXY3(6KikjxX%fZaMlzS?6*^QN+I1_|n|D*=SgE16pO^)j&Kl()S zo5?87ubPg%ekX-2x575K<}%ED>q3NZ9laqk%;M~UvW z+>rp8*Asn6`%kxl2*WCX)@bG=qrm<}1^27=rLL?R2^pMU z%50_8pFC#M*4cn@=DUx0Iq`MHOgIN7E9<=J6r7^3;Yd+#QVuklq+ zm^fML?(>aI73QaUI&~J<#cyJ~1Fb-wW*(%9%l@J9UYjuHP)>j$E_DHW4pghmZzVO)nZQP!&ayQNCpT z4SI~~8T$p2%!!;gPjSPZIynF^P~(H(?+|8}AwBTXVDOJ;>`ER%o)gAaZef zgWa)&MDfEsLvK34F0xbr_ivAAo6o~_7;`Wmul-V|pPxYP#i><$%qo9A)xCN!+N%wh z1TzQd_xZ6m%kbD@{8TZeYDg6VoeEHf4v`m8kKNP-N7x!!&hmO8?6emuZdz;wLZVwS*PU=qhyI=WQ{|Ru>OBEodsJ| zZ@BemhHj)Aq(M@;Ly>Nzq`N`78A?E;~|b8;bKXw3}=8J@7>BoO*n zFajBraJ3t(g~q(~o%eWYi@N-odGkmH*7g0tIowgA6uQ7P4F%qINa14M{l|Kb=_wHR z>{V_A)mO+%f1qxq377(aW$4|w;9bU`eCJ!k=SJXjy%4iJA#D4=QJ;IvMcb@s|LyjI zrr}$O#mu=YWr)U_ej@F+-rWGcn}+=s;3({G5G?NAmfeYoHk1Q2#{2j+wg!F~H*djX z4CpTi5frX+-NNrWFf^$(5<8H}e_O4tHzq9GIxRN+J$8Vw*FW(*To9 z^FVC&lVo~F8;%+@#0^&x@{QN_ZD{Zule#ECn z7F`;0cBpgzj)%!KvZN^`7V$&4l1*W9b!%A#xcDw1)fRfDk7tss!lm>}BKR_XSRNFo zTKV1mvMJ(9Az_;6%zZxUa0^{}q7kpMGc;M_47S4vr#~C`NZw!XVJPLYvbyu8-%4@^ z0EABG00pq#7uAI6D|Mc^VU)KB+ZyA#7x)QLJ8>NbCqZ{Lwe__|#h0BtfgGk*ymSo%G{4dWO_NkDrq!;*4hgK?C!D z6a9xt+|e67wiq>i<^U7g2QGuhV#$C=I_s&XiTWCjJw$<3A!NJcm04em1~0tCxBCWf zJe~Y+LgXi!1FZ+k; z=;s~KiIbUePUBA!(D>5RqLFG!68P-m~T*Md~ri%K5y(RA&>SQCkmLvI)n+XUhv_jLv$Yg{Sww<#Tr zB%6ozOu)vU_RIz_5dm2_dvzkCOjfV#-Gcp0lmo*+)Kl$c5rVkw1e%@WSO8};Q0>AD z{ob2v@+7;PNIj(#d8fW9V()n2__>?^M$A!VDDmVj^H@oO4ip|eyHgdd>O7;UMA;HE zf~P`Y?C*& zG7bNPjC$X^x`NNLu-Qnnb9=Yn5 zatN%k<&<3yO<-AwXvN=()}b@?$G=}%B@nDJJFrZwBVtN}DAHhZztVInK`j$?!VSSO zzRs_=$Ko4ifLx_T7Lt*F$S+a23z-@I>#LNC{jb;emu?ffR_P){pWha7+Aa$sbHjch zI!2!(3OwE0tI-hE9npF>ZDG2ROGPt-`^?yhHoEqoJe?jpw!O5}Yn+8PX{z28JKMO& zdh2wY#&q}13?Mc6BA+||h8iegSUNA-0n9rOb?QCnU$v#HuPDwL69CmI+RhqjI)AT7CSp#whu>un74bW-P_+L0=Imkq*|zM( z_KDPE_6Ccsm_jh8E3WA;#OkX%9y$z|Zch*K3^S913QBx((3~`=VQYB7Mr7Kdg&;rS zeP1;ronuOBXtx0F4y3_hEfEo9do>Fo--oJ-CYvRc?jwirvEfobOj3CwMK_fhtYI`X z84j`D8ujkb>}M8bdRvTuwcjk`450KF<{} zk4Lr$`n;&};9G&}K66*^uFU{XbX0w4={5=6=jyBBPN6@O$2fq4gz1ZRmIQ7!6O7}P z`Nlp`yz}Mm5hYLX@2M3kOL5n}%Y-x9kbsH}&aZ`0;}_X-6#A_3$qqT>>(b7|k0Q#n z*&7Zhm?xmBbJV2D%-*>Y9Mt}!N<+mm(CR8RncL`XTmKszymqg%HA<~stz8d zcQGapn*otoc#Qt~_aC(mYHmMK3kO0sxa&HDD8`CTa86;%ptz(woa3weFJ2~Ntv6wR zWKCV0tv3Fuo|Uoh+0B*dlohq~~+Yf`6u~ zHh>=R{wnMFpvL8yw9Nb#m92RB0iLj0+wi2^`@+!)b9-22+y|+NHl`6^=B<%Wi^b1r zS(&Wzad0E$OA;&?cg1xEiuN-itro0t!l9~bD(LC+AsreAJTMyiLAKyP_vM?y_NIoB zoF>H?cSvVBjkoZPz-bknXru&eESM8)-t7McB{gwE7cX)PL;TfU=-Wn;$aETkh9)ax z`R9srx;BpdX7;(UC)Mhr{ojvA^3@esHVPL~!Vi1HlcfP$bVNJUvxJ>YBlm#Ztllkr z%rt~q1VMa0`-N=Lje5EoI^ph zHPzul0PRZfU6MiB7}74@&p(TffP{gFakN9kg@l;S1&;dc4>+m@rxvPKT{9;ST*wx9P*fCUN6oMZg z?r6I|%wEzl4GAf=9rM z8|xkN!vpdvpVP1&7M@xm7&e?!$#|WHh<7I01V~+>^hhsFV%)lsBOg_jy7GCBfn&&D z3N^C}De?}=iKo~#g)%$v(fK=oQcItlGYG;&RTO8gBJ##HuBpgxWrd42#G}#31(sPV ztWl7G=+XSEk4cZSX`kp_I+eVb%jia8?K!ylfC>#I#!(V`zjO9edde#POUvk}aNTv? zyx6`$^|39-ZwuAMdGnKZlM?wn8r98k%t;^%UhK{fZ*3UNyFYyCGat&6Z7BIFh~Lg% z#_zumR9^BUu;yBIl7=lWK&#6UFjFX@oxx*?S9(EnJu za}a%yLgKU1kjCp|8znTvGpr*;CDu&X0mu3n9w--F>u<3IoMz0q{tH!Zc^@XtF<>Ez zs%>01e*ttG0hP~IJe$!zkC*WNBfSV5K1zU^z6HFQ_Yr(F{ ztUT3C1do02lGtv7i-6|rhi_j-LZ9luRlucWZpWFhi~Z>%z(^Hn|1-z%Ke%Oy*l z0SO1!4Ka?@&BO0c3E55JV_a zY$x|^K@5Yt{{aPWbUXa3I<+yR5!lp0`VcocJ2M4a zNf7&|cXUjnrwEcZq=x!&Ar@1W#%kK^ckg}WO{i`rs|Uta5}Cxo=6m8l2K=fcMeUmQCLWRztY9??jhn^3OE(V_M>|c~H$ux$usF0O} zXA1v>3A&9It8v%^K2ElIoG$rX5nsm(9~((jT?g{ zx8OYo>t^;5s+&ztp+^5H(G){Zz&V{bD(lupzgTqH52?Xn%Q&FO zqlJvmuA@}1#$*!Q4!)-xbO;eH)35tFr&>SLu91DW0WT-UQlZw8+6 z);c`S&m{)33!~*y=zS7y;=kGcw9J%Wg8)QkKie^*#WbtY&f9WC5`{xY5xQC^2TTNQ zNlrKgz-?klT8k_?Zl*m*x$T@^j&A$?=f%+L*>xK@s41*f9k-eg+U6F9YwB=LR@^cCS2H{It`{fD`xFSp)zmHpdK+}mNg9n%l_Q?6HojNNx?8S{l7#_y;< zoS#>2^HRj59Imc|87qFU*ueSUCT2)ZTs-eMqgKW0U<)L=tC)5~@#YrneonPtsvvAV zJQhGIwxj19V||G;n^sgM67!osD&LRWoj*rl0UmZtbBj?FP&VDjjS z7q`>?Y4Tv*L09#yQS8;=NxL54!f%$H@)to1050QxNTcmpEcsxaa~WFCZm+xV9P39{ z#qLff_MhmoKoX8Ls{~bJ|A>sgn~i)|%KJy_a6zI8O&yV7G4q2sQeIxq`@lBj;26v0qmlhd>8l0>;?vi;dtg{G%K24g+d? z&E#_q1gA)E9>1u5Zmb@)taRU>kZS|*8GpN@@N)s#VpWT{+rt$@TE)E1bas*qf{a z&iYsh3BXFnc&8O2Rs>$5N@XNSk%_7scq!;d6Lz(4)_c zr@titFA-E0;x;TZjXcNaHaVyN1@mUMrLGJt+Gr$Rt>FCIUPCO6Ke@O2S=_VlnM5rq zwq}XMUQ2eE0nMmxNWzCV5p^W z{#w=c2@+|rBEOw8o{%9s5o^OW{FOKJS!3bP>s=EGHB6%`^<%& zBdOx8K!VS10%@>FUUTb|@RY?k+eK;p`4em+__GhmhQ_ASq!Th%Ym4uX^n1NXS)%Jn zXBEL*b_D^zzBHs8kR#si7a`H)cL<{?Ik2(-rxEOIrE7)?@xQgt7b!KAc}^q@BBiR( z-e>Pn?Vy!eq_#6YL!kf`_r3AgKQMx0|Knvni)A%2NHPh-1lXVr)~#rlRN0hZ)%tqx zbF;g;ztrliBDP9N__I_GuTAKtc`o#V5MeGYgx|y?0P90{Ub%5Q6}ISE$KLJh+ZA+y z8wv{JF224ewOr}&yPE%x|NQ0gtCI;sJSK3ZEx-@a;gjF&QHuPIKM{ip(9X`*5(-tK zaZvM23F6Y~ps!}pPV12F>N<_nGkS+(1nT)pvXy2h5ZCKEkZ}q(#=ZqTqv)s1V*{xT z^}f6~cM3G2jL|=Zq6n4%kAXy8bOxe>3$@3z>xKz1p-h;Gs)%P?&e{)Ze93=ktYGAZ zHIBgB0DOc!(F|WhP{NC1VCQV_l?2vL)tXZmY&5MaB$%8w)*U_gbu9;p`|O?QeAe7s z4U=Wh^0Uh^1G*H<$KjNng1T)#h0UK!Ov60si-T8`ZtZsSruALyXu})Aif!!kSvw3b zz^EbrGR!}0W}e~q80ny8PgNZd7NH}DT~ZO-A|CL$byUzbmcXmusZGN~lBJy@!pUb0 zqJsMsM1!U#UB@j$%@1oUPYTqJLkG!4s9mz0!i|lR=6;YdK-GLcGlUL{n1k`5d27{g z)Ku3HMl%p?8~_HZf-@Z04Rlr9QE-tVZvcB{*|lK1qdX*j|A$<0GLUpeo{E`zE=P;_ zt9kS^QCUh4Er;mS_7^H_k)z5xFXL8vM$(27Pvzh52nElJ4=MppfLu8+3I7Y=HCimU z+?uz#IUg{HIl`K$TFI@HeZlpP^E>YQ*2Ha~(=u=B=lMp2FV`gLb14WujW>9@;Rv>~ zuvuAO!1PK-ZrbImjL)bP8CVDxU^hU~$y_9@cbX>)h-rHP&{KVZH!{ite$@bR_+6BSCU`OCOH@Sg`ok!y>O%qs z+nM}vx8pYnYim0u3QiWN^6Xg<)@1|&Swf~a<%1#(R*19)H2d@o%%rlA^Hyjb<~!{9pdfTa`N|7|)5X>YWWFP4`;3?$ga@ z9nryMCvHLe1sKnwpO4?!;$&P-enU*aIkWz`vrWLR);-2;durde)uwa329yJTIQyh> zTFk+3P-Q@Ig(vuhdy6f4I&=z)I%qcG0z#KDux%#!#EVN58dixi*YO@p@bi*xgZVqe ziAKmZg_b;oH*Wp&Nz-mG=)=ULa~qs7)Ayj{+>-sb0y*NY2^j=AJzZ0S+H6XWFHDUE zkq6gXN~mA^KYHy~!Z*Fbo2Q6!wqOJ-kQ}~E@788+CeEtZtNYtiYAp3qU2aKI#bj2( zr}8?no9TWbfMZGmAm3?z56rj(Vuk_QMzqS)^Xp$y(J>x5WBbdd+kJ2HYN1)e9)qrL zP++|Fu?RLAtP59tG?XE7bI3SUm<>y=ilriDK}{bcIslzCUTD>SJh)i#d(^MQoY>a& zb`ZDh49evfV1?b0W}TfTL(QXXd|A)F-or8``vkE4h~{z0A#Ic*o=R!P7avEmckC;` zVnBmjGe-Hm%;0Z4p!exT=cj$DIx|Mq&$C^3yHrL19q)p%dkH3FE@nZR^Wjo~xT_&k zcSHgX2UHrQx_%UIs4{x5LNFKk9X*o0I<{?AL0}YK& z*4`Lf6Q>?xi2rLr_5U-Pk%TZQ*sH!{-;ad^G;w`+> z+DVmA_~ovwp*OBUSI0PFXVi%F+s27)+y4SrHrM@8t>TQ9w1-kSBL@4qo*PE8M2NXY zCdl8ce!0~NK$Hl6NAq{|4~#p*;ODPNI#|wa<)C zR#4hkpE9Y-)i+VtMPk z?Gz^f&eTcErAlL|>Mlut0dK%w#cA7vlfp0Y(pmR&j}+>$F-+cW>KW_6+XWA zl}+c}pAeyyOQw|(4xwcYfnwpR;S$8rD(5p(?i;>Rlk`2MEchx%y#C;M@-jR$GNkl2 zHX@{bK%ax!D?!Os!)oW~C_oFPIb}04tB%oH`2GOJ-;9ZRut{iD8C%2W=t*!+DTXUi zbIx%zE!DW1ZW@C|gEEOElp@Mmy+E|&FE)0WaeFrIB(6MiTZ<$YzFh8MMhd>c=$Kp2 zR^--OqfMIF&JDVTy}KRiV!QRCza;wxv1RXJnXBxDQA>;xmpC4a!l8~AgKkI;{Pf7| zK->F1kxA84Ny24K(r&4~21FU}Za>Yt>tP>tc+OK%rRu{#s5Mphu_3lm=yoEZ8jqXK%^uGd@V|rT3GQ z%h*dM(qEu1YWq1c{oC*MACa0gd%X|)CDPhtkkh?bCFMl0+FfNIn&zW~Trkq}H=;Ek z*u#7g8a9jI;4+>hA*XB|dU;N^^Q`u8^=4-=`^0X)vEJ#R?$y9(>VL<{*wj@SGd$*aT; zlPY8ugO*sKRmGLOrNsR&CcSH|@F4qfFxeAO*U$!*$p4CE?}FBny&J@W3I{MCN;WY_ z6ct)Vlt=+v-P6k`i~@ov8ZxQSRtNiBPa9a@h2$!O6WjtR)Js)C25t1X&|GBlFn6;uvScKnXw6gC)-sY)4_AXsSSZj*oKWbQ9Tw0d?_y zSIwt$P@3v8<-<|uM}G3f%lvAv>GDo^#&A|NZ|(`aFmYaUHv-K?Cs@9sDWautHxKso ze%X2HXouK;BLHW+1*+OU_F-`s(3bB$K`+y>nf7jWLD2hE-y&>ZKOArx8gjBI%Qtk^`CAOy?4axue9V;m4J9!@X%K zYo!s;Dr{}p0wEF~pBn4$8dyHWmf>fD8K%&1JBDnX=t`^)njLtMSwP~)`fbrMj)F>n zAGV+?0tY`I&EveXr7#XtZY~4~_>btr4!rO7i(rVT;THCO_;SbtLRnZK;q@uUHh(y6 zT4*0Cu`X>DK(|6flg$S@VIgcfQtxg)&N%E@M?FWQHF79TWm?=qz&)#n?=~O%UF2b; zU+-wy<2L$464I(hh5|c9-`jjtAg+qi3Squ*mpChL=Q{oCApmq@0(~)T_9P6V3#c`r z*=CLJYy7d*wpcDgF8S@aX=~GmxAxW=4D!Iq#sNRZ{l3+j^&vddSzuszt_74)YXMq?Go>;D|*hp2Xv{Z zCU&WL##EQNGPJ&Ha^21Y{s1B6e&D`v1PXVyl^-!ef02}OBFdfXNBfJvrwzQqpb1Zt zaK*$;yWWoY*XcHfp}WDK)+<++Y^yQ^cf7cuz;zTS9>w*)p4PGLk9%#ZoZ1G*T#kH5 zeN$`;iQgJ@zZ{E3GUYiDa9YBRNcJ>{T)*lm6LX>|=V&LEbA7t^_$-KVnj_+Qgo_dp zDB>5xk%`r#ECayJax2Xz^5uKlpCD<>U4BvWt{M#z=}HqVbd-OGtWp6;rpuv z*pq0NcUDK?UHw?v7JtuoQq702Yehz+<9w9#+}DQ7mkx9JPGcD$s4vtA8d|vWLKG*( z%>J!Q)t`B>MI8h)8Q7ayY*LNW`j)*YDLru|Ssi8E8s{fG+V2+N$zBwokkjvt^TpxK(AX-`W`@HhGjiguBeTAIw;c=P=uDaRS1+Q zr+DTWI<4t+)koWQ4#RmJ22t&_%G)C0h(cod!0-5$vznh z#JZ-cv<^>=#7{ZN}2(@ zsPw|M!4WP#>UCNNlORaOXh@g4-~K@2;x6sY5;&2L$Z;!)<>cd6YKgYX(@+!~JTu-w zlkk)pqqZU-NZ9dhbi8m!p-@@I%iYMxYtAvkX6nedl-M>X;o{X zst1pnX>_i}4l}GgHyR;3XLi>ooR}uX**C4m4|p=}qh0_*6f9N-zs2#`7ciB)U{X?1@{XWcHYi6urIdRXql=_~9)xlvzg zeD#opPBm!q!5yl~>2)N%`6ZBcr5!lTlz}nY?Gnu0(9A%MD zKt+?Qm*|y6n%eT+8Oe~4w=N;Ys{qe7@($ctiz7!Z30VUdH>E_hr8Pfc=8{JWP!+#wWOI%MbvM;d%7VaN7 z78=}{#M~e2BN0NYA|BTP;O9}&b=RZj^p8thwcl8wgE*vlhb!~{7VpBCI_xBgZKUlbiJOjJP3b6+G*s)6@)c#){ zQZ%K=%>F_GyX?M0W-Y+CKb7ce&;Gy^OI3JCJQnHonrrFRVjU31VTxkg-R2^$#U7ew zw3g`9TI_~;8MmGL`S_^~p;-I3K&IbbuWT}omJ}98jmZn2aI3XgK>LN?!o?=-!p+c_ z`a{cB8dYx(uPQ`s#SQNulX!zv6&eaDrnelTYTHq!D+(-HH>YWW!U;Yzk&K z+;n9xoABZ>)O}+zR&2|HoXZ6C(V-Bg*q>k1ylAjd|6ppnf6x~!SvmrkHWC1b|A!*{ zZC6RFtbj(I!qx#t>UZRUPp_qa5U!E}!y}jK%bHgbsTQYk5tkYCT9$q?Op1f{$4DTp zr22*&H50HihK&?6S2Bpi2PC6Xb~qa$$%?NQ?hn2rQ0jY0MbeU$q9KM;ldrv;pPW6; zjMyh;l4Upn1_Lbs$`P4zOZbl2zVoPj!mHKqwlG`F@9U#&P5#fH4>64~5Q@dsvTW7g zedYN=iB{p;ia;XGdOk<~;D<9$j2lhH>)>iQb^2B%YPWQCS6p8 zr1ExpQpm=GmmgT({lJ_G+M)TC@%u~U0D_5>ijeCT?h)sF8hqQ!yu|YQtWnpMloZ6< zvsz{9zTd$nX9ObXW)YxA#>#_8rjqb6J&Vdr+l#7x(*9ybRf5K(-?=9rcdyRGky<_) zuEYD`|Fi(WPQZTeDV6^xPCZ+bM4( zWeRl_y)a4#S1r9J-z>Y4JN6m;y9UG0o3TLKv!BBQQ}cV44wl-3B!bzwtW}uXveJRw zm;$gr@#0JNCo?*VvZb}I6AKt&XezsF80fUBP9A0X?%SGKx2L(>J7POss@8sPK*8^z ztJ`N`(lwM@A^c)F&sS-!F2I!O9vjc9N^%$Wh*iCk8&4;z0TU0{16D2zRULvGltW2N znRTv0iOL?gXTNkTpU+Z(C9DA3$%b=^^W+6S_xn;h@(+Cjib4N$hP>389@9!({)EM+ z_c%h%EFp&OPZ667jw)_}8F0=qFy&8-i*`V7Q9BuyjUcObW9JB1_4{WFmB}HX?PQQN zFi+5@AetPJ;FRU%VQHlpPvJBsLh%-ywx9W-A0#L`vEXH{_GcuyoI2pcQ?qP;nBR5P z`~B*xv&}T-fIsd@4CcbDnb-31l&2Aq1q-E{i(2{eUfd}z66A&04MW|uJpoD&U(uDk z%W!*P(YKr!YP*ZDq%cUWdKWVkM`Va*fr<-tUSw zJiZ;4*h{*eY6m&9au9gXw-Om$GyoC zm_B|Pu{}mJhHQnxV8FoAPAqdrt>xm>vJ zj%H~!+L->i*d;6qY{~BmM%YIcw2MR0@Qq_qf{C~G6HA!-|M3;dF9Y8w0i}HHQ;j7o zw-QTp0BacH_a!tIfu>8gH@Yzhuon_g%e4tS^9NfLUk*pRxw~F~?}1JhiK|%A(uhr! zPh_S5vx&#h+iDI{VqMC(C(RWaj3n!y`daa1MulZNKeT zY-sSNu>v%emc|nf1~=Ab`zs@i#?~nI04l>ejx4k*%QjDQ~X}mGfd@AbHR*r zUBRyJOg)(&Y5tGCY>B-P1oGapA>po|tfzbgB`qQ0{-{;uuN*u7mPhT=3i5Ac``-Vh zDpdT12qQl0uJ5|d0UTeOkJf#(3)c9h1|DQFWPA#OwXJv*BcR!J-bn^=tXO;D%E$+E zSLo;EYWz8^UW+%tbvjG!i5z(8gr6?LM<4#}wiMV)3e8(al#jwoZjeU(r?FEw4DnEi z1Ucw-~%nnU#*h z4a03GKpUGg!oWr!M;h4DLOEn|ixG8SZ@`CO>Q$MA$Ej&AhTSW940uimg)U=``N01y zKJGp=bA!)f&{-VwA7x+#qvmZzOn9OM$a`zc#CRR`Xc6VToGX$kC*Anh-a8Q zGTu%j_rDjb_hY$8zagBS`QHyHXe+QOxN>P`o|-?`Hj(b8`MIn2U-jnt&Gk@Ae4o0& zNh-DKlCaFSt4y&}z|}P5V21v}U}ApRQDyViOM+T;HTp;qZL=>oYV$)xz_&jm+?01; zb=gBjV; zqBo4%yr*>}_8GoUqSRWr+Sa!qX!Y{HQ2$`0JQFayYiKaaX(-_<sG) z(j>S85{yKE(Fj%xB*9I4Yw5c6X#sOjE9vSN>jqR|?`D+8ZWY>x^N|lFh`Q|+YrbfL}gO2DV-kx>A zII=x2O6srCCl1Paj4AbcM#*~WE~Pjrh%()>if(@LY$V@_$GoxGJos=L_i5J3Omx8U zBZJp6{hdR>!502d;p7u0-aw4y(XxsE0(~i1bZojCTO@N~MyOi}6t52(M&#5-ZsB+WYW$t;6i_CbAX>fn3tslS`V=h}ur~KV*s#L*r*1C4dSk(UYND6PsN|ZAO z;=-szTSF8Q^w0#OqqAi6Ra)nZ`@e~+>!sv8ee#aI8a?}B%`#L*-W4nt5Ly-KN8Ev4 zy`iV|=jIXH`JMVp>O+;RSOXk-!R=5J1frAN7NMH(U{#V?OwzaS^@D&6mRAuDTsx7k ze^qbld%c8xxCzuu;^~>F&Y~{1A60EGv8tj_@zX9}R4BlZiBbP~qGUong01I;a{P3q>~eOB=Op`nIWOCK!ioGScAMv0kerw1X-4WXw9VW98sst#j+-j^eUNxrA~|16{0B0 z4;ES-Oqy9hf2l9ldEEi=vIX1Y4g%reum2T^d?rCuX&Va0{E9+^%7&{;L&lbakmCTt zg3nwGEpF5)mYAsrV3Ls{o=j+2+Ca7p;Z$;L70HxSPL#V;fNtodtL}F^N=4#SMP$1qB#C zY!0PzwEa7Zqrj7^XqrI(!gyeu@c3mN^-kvoiZeI+X3pNUR@Bp*Y zwk5@+g~k4K4op{(HOnOU{tH;yp?cyK5gU021H|in)-hDZPvr)$VxnXgQzkm?WorA=v9a8A;!zMQX76rqwPCJ@w>{H;~_YKBK#jK0^&zYfryneSo-+yx@8c==^2nX@g z`(wVFK=%wrq!YIR*Sp>Wr@YKx*~ss!{3p(ORA9-d#V6umWD`qoHI6M-@9 z`%q*o*Xmy}p40G(w{nHUMsf``I-YYBHnMpnv&EC&2s25-iz!Y848^JB2WklOJr09% zUsKwRH`TmSvXUFxz>P_FnB!zN7kOz6toR%UmJm35X9d-%xW&7Mn}u|NE~`ZjRw_LN z>X?ba3N#XZdgHPhEcPyuO0iH$;H!o4vb)~bafUV=%VM!8*gl7}wt1O9Vi zpe?0K`J4UEjYvh^uSGfwkqsyrR7>T+U_&V4WvIHu$wB&2+zH51A+9g4F-WAC zKb@g*=X;*se8(wx*!zAG{zOfdp7SOc34(!0bSKLWe4U^qn&4@ZL_E>-j`-YA3k{%G zr6c4KDs2K?D9>|c2@^R|mc;aYBgR(=`7A909?yK(7Wk!J!I&lDqiqo4M$Ne=pFsaP z|MJW){b+x0&gs@$v$=vYCqeJ4&@~MWj|%j4aMj0?x)zwdZcRe7&X5%i3*M91X%l6$?m%Iy9aZj@dXC)`9RoeA^h5M=8JHH?mSilWw zC)(~0VeDh2)4Jo=b3S&fU~zC*`or#9_D!W3Y1yZrUMl(N^S{uMC)Q!uYMU3F!VNu9 zQs2cg3)+vC-+!8Mns^Hb-E^MQ^}R>P(JGMf7)+0^h>w&Bs3fu57lx@V{->8{cwUe^5pq32JY$ z$PoHVFY~LssSxC2og6@HgH9s~j zau@iTanuzw$=$3Z#LN5#s}NNg+m)j~VqiLYV>0|1;&%=tHBNe#NGI>w z%;&`!(5)z19k*C8jM>(lcso9|iiqBnKKi?w`|i&KV+xzAFu@%uKeAIo=-f}W{DU}q z3wU5-X`6koIYri#xuYF!2zu}wFIi>8;yl=f7~`>p{PvOt*TB&J$bdS zHkGmC--Wqq?sDYbkI8m(Yp;4}HUIoh4h-v`Kx#%~ta9nn4AMqkmgEsLS@**=rg-a% z1UY!r-}4}=xR4m+|Giq*s6I&| z5G^#Tp!{ITP_3H zEWk9L-D3~;b!+4*+&lz@!>d!Y*703r8Y9fNw>{|TT1WV5PyVwq(y9v?o&(Ddv|&9q zfe{0t#WbAHsuFA`V`vj0pEtVEz72U?Pgx54-&MxFl9CPplsyDNL#1U!U$^q9bzVpq zO`|k>B8S3Bm>*QBayF|1e;$Rqs>Qn_Asv3}RPTRy9h=FYxguTgmKXz}V@(SzhZ!Nh z!#1Y^^4do9E5%B}=n4ztjt%L3dwK#^nT5bo51vkRj!&O|u|0@R^k&QQRR*$# z+>$u5f%~X8S1sV-I9zfbzR0>7o)ijumv~up_wd~$&<%2F+kN!W{mzQE^C%unDLS-u zW8CE0WBYA^szK%cGnp%;lZpxH*(J~@j15(^DQ|x9XZz)FW)*ZWQ>RGZSH5H=5a$W} za76g(X3BHn!F5Jtqb#FUjy~t}F>>)QpLm>sW2(0pee54_RvbG{Oh3H;z4Ei<_K+PI zO*M|)o!)9(zMapq@&;U8HV-k80ij7(Rek#p%SDEcb{Hyo9Q9(Uln5V!6rJnPga2JL z$6~XsRo}I@==9(2wC{#8S|?l)!%!AuoRo2`#RpBdwNbD zyNY#5NH#{?TG){gPn`pU*3%KZ-sfjEjCYUNhAhK+!jm1B%Ehg z<{uC`6xI`(aJcAV$5S^j$@|^o5=aO`-4z!fwUot78A{_ME_*n)M)3yt8rkC=vS#%$ z96vMx-L6g77c5%pOc#)vg&LFX_ip#&-EXwnI7SB^QS-(inx%w{<>|!jJ8xLC$7{-< zI{|Q8W6!BZ-vF`}u*M+q{wvm~8V7}LdF6Vw{NrD8E*L4FBM}FgURayf3vs#6Cr3vbNr}!*#-BcopFW?^UR}q z`13v4+i43Bc(DalL25rMH|EoOO<>)b!%?rdgm$PL1PYwxK1}d8mEk8f=ndhpf*3+2 zyfEV4GhND23AD4(*HJS5c(tmST-f}{Z*H`e#yAY=g`_k5hb>IZ@Vbc^{!{TxLFk;s;U+rH&QHM6%i=Ngs5g+#(F3fWGJUy z!s%`=B8yHrMVW4uG=byXFFKFohb9E^D&_6OTW_oKmH%PtEx6j?qHXQq#i6)Ead!v~ zEmjJFV#VFvU4lc4ySo>6DDLj=?(WXbd+v9}cmF^}Ml$x^Ypprwvsy!3Ry{aALnxW# z|3V&^K@gjW-8PTM9od9i_|^P|&19Is5WVj3V&-Pl>aWIjim1`ot_~G&?1>mj#tK)C zG3VEbL39zaA3_2=t|UltSn(^pI8AH6+WZg|R9@Ol-e~yiErl|k&aj2jc!{>Mq4Mi@ zeYR||FrdSj1utTzO-8;s`YcMjEA={LC|!L3sTN;=b};klEc&e*rJDXcmB{RkShT+B z5L?RfTw&o|7n}-Tmu){vUer1o3BQ3mC+p?+HAdyjFh6{iNK{Aclkk)NH>Yee-2*j4 zn@+TKu`6po@3YCfM}rBrA+FqW)B%>6Zu#Y@TY4$C(K_cgqHA4P3c?99D`bz`GARYZ7#0cO*JYPKXV?KgpWiakGr}uX9 z8#-SG#PrO>ZAb*g zm${7!#7QXzFBa(?xM4;0xeW{*jtaf#kXq&TnVRLf3FpxWMs)BdVBif>qI|W$u;S{22*-37AI#sXZZw)-4g{FHO?*;_eSwxDwdz03sPsvd z9sAAg@H||`Gbp4|t}*x2w&40!B6Y_SgNp{@-s$-X0$30^S#HJ!iYP+7HP|qA#p2TD@-3B<;DcF^&AWeOW@zJ%^b7KAAEh)JM&# z_s&?Xs7ezYW)+tn3a1i>>=k zKu;kG4mo(Mh1dXQ4`zNLMGxtM_vBi_8mVZ{o*Z219RIhLy~Z?U7I}>pA8M!Jj4YVw zdYHeVOZ3gxLf_(&D)1VdVcw03YSZ$tL!8ufaF|Mih|s5M)#E~{e_XcuK$b40-?whK z0*u|aP_Jw=3`s6jILsp0;mEh=;n|ikJrxuEzODgLU=UXzs&7!6X35{Fa-H4Uh>jjZ zzCv$Ud|vNor-QX0ZAC{$ZA784n*Nv>`!QR2~P+- z6V-+TW@GCcW;b0k%6yhy{PE$XzMuL{iPUXeItX|WL<27B4Xq?6Ygea+UGMR$Soa+pLpnKV;BQ`mY3uFEkUp!PI zp77(@60G?M;-Hbq^E=G!>Fff868-9MeH01lV;nl=WEN@rM<+|vyPXxwfgpUe5y(k0 z_vD@@H!#R}+F>ABCZT{E3E`;!qi#TVOr>l;xfv%fj2MTt>p{K|a^bb+_hW|hm(3qM zPWADkd4gWB1JTsxLnrjZB(%-)>2j1d>Z8%K8j^78#nvXAiU=ESv`B6M_f;p#if?;P z3%)`j=)?fMB-W%jPk*Ifd8@uOp4(cL@iJ`bi(UK>0K4v>+64dBUp)Qw0phs+kMw|6 z9N_8OVUMFd>=4lJiZ8}VE8c5j(2seY9chi{I+OlTG~K&}^~IT}#uUIsUKE0^#V41+ zpI}EQ3*spL*&;)545`AP1Sy7O3+~TW{zUzx>eV;nr^CKLa)GP52fb8$A%&g(Pnb*i z^~K<3@XdvYmVj89K=JsfCuyZVJM~Ss&fs_v8lxtjJRq`4z$gjmm!n!}@7|Ka0y}${ z|FN0x7XXh&57HwOTBzlDn{^Gk{{}z(H9Bzfs?GZa4aY+;>jv|aBo63xJC6^d9e~k~A>C1Rt zOGEzX^$_Ja1m_C+eK#&(p;EUfj~#^MnaS@KyS#6?KMR9_?hRiY{8aBS_TY4S`Exj# z8cT&qTMVGga1(@llg~perv0n7RJdQ)=)5Hfa^J#P`EH$ILy`7odq)oO5?Onj$@#aZnC z=qi!w^bpUms@X#Kzg~bIpCGj`I~QEbwI~PUv20@$u=i?^(~+AHFTvD0C{&6*k{1=+8`=Zn+EGyOGqt^L&wGuo5-` zzZ$6F?Ipl`(X*0o6D#TjvZx^An+P4R?S%1j!1h0W|E*Aj8GHo6P4QlVKjT;EOFV)v zDx2SVRijFmO4NgJh&r-7zNVwY!lK;1PT--a^5fGBh4J&mCF-UaV+<0`v+#apO7-l&X z$_tZ;Y3o=X@HJAnK0KXfmHezmsbIHgf+qgz?;SI&Tq4GZD?W(iNXnWmZ~X~I<-{Zc z&C~Q-D}hh&kn!s*GYt7D*8nk0c7Sr8i%{vp!HRVCKc{S%>k)G5cV-xF20*>l84drP z8H-E`8}6I)J)5>rO$v|As^_iF3xL zfGonH0a;fjiZg&fVTZ_MEG2}7Rb3t~6QwUc;Q*_jbwXDlP|mGmCq2+gu^^X>$J&9km0#{* zvv2i-9eqqGphWEZRO(8VuQ*-nXt8^16{IV$34= zt-uzNEIRw~-PuBO%VA7bXbiK-Y(F5$=+VgcJbH;cC$wm|5U*)uMDu}M<}cX{0(H@3 zifnw#co^2L?FTiXmE=VbN@YfkU`E0`*^2a!0p|R;kZ;w#v#}?y{<(+I&S|@>wJH-zkfM1j25W^FzG>OL=UiD z8vXA0D}3SJmh5jc0wRhF82B>>b_*w_g?>wW?@{%yo$hV@s^bz^swn7ozP;pcT9Bn& z^xe|p>0N=}_eOuKDbn?&27mUU z_P5wEglF6jvsqU!!K>AdwVpLbFy;We&CZw!+0VOI1P?Pn(0e3|^@g9zoad z-JLMJ(rcc|0GS2d$_+MCs5(f9LL!`?_QCpuK*)ou-fZftk<`YAux~uW9x{S{^@dpD zIb;V(PC**_%yQ6-*+YZeIP)ZYYlqE7yLKzqP6qB9DbS71drqM4JqF>0Xfd-#(TUm_Gw^tsdeJLJM|J87gL@VM%aD3noTkZ7FFf00 z$T~qy{MoH}BVYPN3PE&a6YE$hr9|xpC=RBAQ81f`;Y=zv?)p?netnrEb*n#W`gT~W zgSnCMjdr5o7oi~D{F{9HYj8`Q4F=D7=)s?4*~P|7W+=9Z9db%TzpoO-JBJrXlQosa z3iHYpWt6vPnr-oWMTZOy zg+d?R=t(Cg1F8-(xetfNfE5JXrUTqd z4Nc-5D&UsaLZRO1>LHug__9HpRPl23}7ON zgd&AKEx}4U57py%N`>yzM=!+}13&*NVxnM=!<0b1clO%q+CC;C?a@o4ll+qlb%m^$ zH;BPOg9k2L#vCDdeN`us4n)^Xk>fI-;bKVU>|lksFeqgb72^wtw??4=M*~m!XlhyR zSqf>x%z@(h#8Svl*?uCB+~-g6M3deQ;}G(9RN9Fl7XK-o->OU#v%}f#d7b)`7l5J%qnS z<<=>ZKdl`VSgOF~aOZG`y$|;u1M!BU_C8O%aXoP3yi{0^Mh+S~d=q-m$C@6uJPGnd ze33C0@-BoY2EW1Hu<#A3oq<7EIMoa;kbu-S2nC;w{FEU6pBfzD&6$-4jkBQrA=D-e ztxsVSdyBFHqCIf)QS)*{tn^;i3|L1vgQKwEn3V7Dx_D1UQ_gJZ6nSvl?mbdtwYnzP!|r zLT`zGydO-<2oOdgl>uBc`=>qoZpTk&drr^8B+Q%V4SLR^foKrtk50x9GSm-#tGA{; z`kjN^G;#8N#22E^Xs=zktNome1JfsGUqvZtrMcfRMVPfaToQ)&+`8RWx4=C|O0+!& zEIFAPd%vgs$|*D{k1}@_$q>*;B*<=0X^eC*}JDOKVEshXk)2g1aLTu#C}nNdV{7*dL)RtmLxB9cnyJq1ZEl z5Y18UrFKCbi**3lLGx$V1tiL-kKEK39J>U%mTD{Gv^g1k4f%(?%2L${F?YtriNZTw z{DDV^2#tl?1yVh5RvjXMA-xbAcKU|^%UJH*1>jFo5H&DqHs8l+i@%d5r7vj1P6LK?9%=huxJx(BAFY}oC%Ysf#E4ecJArgqxl)?CXd3j zO?hk<%n6as7}WYw`+ z@hj$pp1g^b>xX~vR>uoVuzIY++mI>vZ{uTG zG*Y4>Zc#$%PJ4exS-e3lo64|ig%T4X4GvmC8njMI;KyE*Krz&3ymwAv@v_1qY2Dmx2r7^9f@1MBB5Qn=XBiFK7s&yNx4p8!m9R_7A?p? znf-^%IVTKJRHoLqxse1rmJ&cb(tQ)3J4H;$%pSlkdN{sPoH3EbA;CP30LoeSL_)~( zzRIAa+0R+epo*(vRUROxhyE84A=*yOUrYeljKCryBf&wFgR6ps36`Z=%+pSKFzcly z3s!;J#8gxarOXq!t#MdW4kM^{*fV48oI`IEVxdXdrqFyygL5FEa7{P$`m0U{D%wg*$p3Q;S(-|J~YaiTlLHG%^KTP`evZ5CVb5$(Q=*32cLx|*B1~O zT6uE`kDd_KAMZaP5|OXbJR5Aju}WFQ50NS12DYJ6@ti}}UHRMjH3V{=aB6nZ{d<9L z=WDK>ikZ7P-)P-f^fR{=XlQ#!R~)!glI0sNhzq@nH6!neIp&}NGV5~z9WcnRBe;`% z!%sb*J`T|Fno}Uc`8xTU z!e`5HyR)1!QZNfoQ-aoWz&e0ri%lO;s2aC?+ka`3Y2U0cj}Fj>iOm6xQZ71l3uc3+XCQX;f>Gnl+*5nbAV1^l61~IuZ!Z1=q6nRS z+P#Y50qsOGDvT=j#{qE&w5b%1FfK&t+Qy|xd0KPV(|_w%X1Y3}!qjw1Y=tPeE+ESX zsK657#MPpih4JaMARWv|NonF7VQmMP1fu~a;l*pgEZuN{*f3|B2h=C~?vw*M5bsVM zR}s@w?bPEERJK?GsA4*sXd4L{y@{YTARuu_Eq}sK`I$- zx6n#-z;DQ39DrK|6{NNrjk~i zbWT6WRI(wxIb)Fs;9AJ4K@GAV5NY^rp~_Tn_Bp*iV*OZ(QMEh=RR4vz+`%KAR0NANz+ZcNx1RwY05d+so#@1{&O3J*}&zQg?ZZd$6y zuw5190i!;Rzaac<0PSq~postC4pFRP)L;kyTrdh2t?f?m`#hsU5~EoLnd{WVuNK`< zak&X6lV=09O6BQ@K^-s`_zuT>g4hVyvVL96!;`_>8-GUJCX5oZEL@wtD%l;2B_j^Z z*NfYs0TPM?-!eSz3nn~5D(t)cm|Gj6!#i6^*v#0pUpnwr2LoKTGo+I)trDSZ;MMsm zhjPNbqPP5~yJD0~%?CoJ)(9=$D&i6Z**jFn4(o%G*<%B|1B_V zCVv|M#U8+$_NqaT8sL&FMm>U}7d~BoA=@`f2Vz1pyd}M7cJRXi)FBrRdd;h)%g=F*5 zso6>8jcBT)oPmwrYEL?e)>JJ$mGKgWX$&K}1eWK38Ti9oeHNwdba-KS>y|!H2EbTE z2A#F>qD_xe{-Ynr82wrbXH^1`Rro+OM1Wozba~!L#~|QW09k7%*}1?o>LmGFhVrd) zE0YESJ`BdcMB6;oi4e=@opm7d?p3BJXd@(;T}ayOd_t>!W-ZmUJM)Q_1j9!0LM3^Bsn$`vhrgQB?a z`+UVKzm0~v)RM?Pr=CP({Dd-Vz}m+;p1nf^iKV}La>*-R4;MASxR5;WvGF&)L8j}< zJWQ&+5Ue&Pt`dqdRMl1rw8Ceydf-;57f}75(a`5qc2D@T)rOg7RD4SgPSG}z6|gax z3Xd{CR45z0I(w;|Ne|}J5Q=MuEUiuWnD=b(O^(=mMgl`CfUaXRZj9atI5GUISbAlF z1xL=Wha?0smPU?nX#V=m8KmRV>&;tA=KW$WUWURZ!dDTdRYF{8xN#pb+pZC~%bH1P zCIwl)m+|E!N;l{2*S0xHxb4GhwD37emzMk8ScP(vv2Kj(*tUB2&+9)gHq za76ooTZSbZ4Rb_oLRZU4u;;7M9JoANzN<`qQIL)v9ujuroBb-0iTcO1HwbV&RCfNK zuzS^#;}#ByR=H$VJ287tl7EFJ5QPT7Rqn!pbJoy>Zljf2#YWJ$o_+|$X|;ziMZjI6 zT?wf?X{Sjnw;JN{H@VW5YjEfq`4*gnaMjt$cfH3DxC1CLKiDNp1tU-KVQ}1dpp&8*e-@d%n?CR}dyaNbaV5tw{>$*)t%9E3Q2^hJ34 zZ-^W%rL>chLNB|CDESd+#dRSq|! zo39QFIqhqp$am)cxDd?bByP)Bsnj;*}cl|^baG+2o}j0!Bq!hGG2A= zaa$Ph;_lXS)5NQ-=H(Eq>X0>{>a*pY?pG7^Y?y86&Mr!UZ}_k`Egfxvi5BtsG?pB1pEA1mL{^6N; z*+b5$*Cmc*Pit1c<6;uB{%#W{p^_!*R5zPUWRlRs2YL9W6yN|9&>sUB&DvR!lL0x0 zfQ(7<8ShoXF_`b&UBi_(4#jKy1i zsNUJAiJ~>dLWt!Jg0qX-m&FhAv7&ylBOVue=Tco^MU>b*5$nS1zUz>+T-A^V*kbg7 zjMLQ}Sv#7j)PITh?@ytSElujRh0*rkzNQm{?N3*GZOv}-I)Vm9yMwi?QMZBjZ_sH0DfC^7_y&_!~}{E zM!{sT$yR&EtUx-ZTc2jII>tS_a%LV|4LViAeL(|%{WiO6=+Mr| z9s!0Qz~^Q1u;PiT%3_Z<8c6~v0y?1gSjJtb7o((85d>*AXg(bH^q1bGa9bY*p;>s; z5=%PL{k136dA3>-767*g4Phg5ux|irV0sl(q_5Y#TrS`(6e#>NCSEdDeR`K* zEQ22eO!U})s0=t=L#xJgm_D3yndU!e#X?+eBcc&$-YPsK_|~Bhebb>1&9I1A)6KP} z^xFNSPS47L;nV}T;yUVRt`on_4I=Fv{_IgLGeadl@~$6Iu>J{bawcOIq`UW`N;&&` zzQUBku+rqZ@QXyL2C>B_?8w?A;Y_qdq(8DDR~xI>tb^~Q6R z6P<;Q+0@qudE3)Sz6@ghkGT($Z2NFWw$tK<_nz8MqHC(xW{FKE8e)JCm>2|Glg8uxVkZ~$4dH5NP#jc0ho#Bm3*IYjDVnPZBx z_zy!Sg8GyMOjpqH__DB<1~v4x`LB>I1PvT(MBcGoQkAa_4ZqHI;e@ZWB0-m77ai}r zCaIGq`M|X^S}_y2SYexR7Lsqwkl2_SaPTEtqZ1k9H_IeqLm0zluZzl({Noxj_!@>` zNycI;GoAO>v;eepwBgz&|84teLYwRhOzB=kcVHsz{Zlr{^+ks&OBG=xDPyBJDqwcU7C4h=BP5A)Q|M?@(B9Mb zY!T>)k+N^QD36C_GCz!Qg}?P-Ct3Jt7y{rl3gJ*g2rWv1DD?-KG?$13^1996*eUPA}RJcaOW?YM6wTt4v5Hab#u zX&DP{ywIAFdlj3o=K`*H3Bw8epVgo*;cBMRuOU>CG#?%2cSI9hgTyalgy@~PvBS^+ zGj6wg{?b^!uk%Pxoy41q{CeM4K{qH7C+mTUV~-lMJxrefVvsyR5q<6cR3tMIy(&ts zllyPQJsF%5$3J-WvR~tqULt)B+efDL3fTh%3FZsa+3*~&S>z}YZjoNr&dBWC^3Fmd z@_$^n2b}QF=Z!`}gTtPI37*x=a-^JsC~W&eAsLf4XlAor&NEYX@YtPbr^uz@S%{;t zJ4ky>YUwnruTx>(dG{Z)S&fGYjUw4gJ)Q?CRw4iE1rSKmIlF;VsZb6p#SM=Zv3Xy| zyY(1wq1az_Z;NPRzpYq8!e;nNv=Fj=O(CTFm&Kcx^NUIBsE;Cg zP_gA_MTj5m0QW8#tk}EQx9Z)rK;jPMZd@A zVHYrJ_3_a&XN@1Ui_tJa=ry?-t1t5#jb>-5S!M&Pm|Qn2L*r+ifTd=~qPRxIF^@VU zP#_G3C;9Fy;44(CEAll<)}@pW@EGVlX-jTgvBWn$WHY!-F;A)3Y{9C~xWyTCqa|Ga z4pGO0*dxh*#+TShY~=?8ob5mbm>G0n*afL+00x1OM6!$*uN@)JzNL&-YxQ4P`~Z)^ z2{diCS^!ruy?mzN*GT3kB?+^U806cM#FAQ#9(YSBve8!jGdrBKJKdw_&U3N zUXTjEx=8ODG|_oR=EZbSttq6SbtCTz$>$Kz^8~|2zx!)370IgCObcw-flUJGZ z*lhe58!^I5`U!7OOMx8fA@csX6Y8+!Fd1%3i)^C2+GG!x9N17gy+kYk2L}~67|sF@ z4b?CR(5W;(DUr$;WS`Lc0dJIckN=gm{#^9TGBMjbZ9ddRzhf(znd1fEo0g0(axNf+ zb7SfxI)lrsF?l$Zm+_w3No(YFR=?1_t%Bto??;JkgSmYl}#6Z3Xdz|-E*-mzM68b7Y-qhJE@SD4J!o;T2W#~ z%T!W;utO)Cwjfa^nb1ivjDpPxtZagxBadRsN8IyFIaYW9$~azyR#K{qSDL}bYd`~2 z7Bw3BAY%Ar`hRP-7hSJcC>0 z)X5-smmgx>Ddj{n`m);ubwdfmPdxK4^desMXZ+lr{1rSz_i?@M!!sxKloyp+;3Zdn z$?GKD;1Lr4E=>r0fh{;?MN0);)9dm~e)hikOUz=sS;#4++1;~%a*G_Hr@$Exa4=eAxu$uu4jbZERwy=GG-N&=ae~!Hz zd>`p4??r+B4AIclI@SMpP5&kH{wF(I)hSd1AQq2io6C5A<;{OGGJ)y%vg9w{uo=mm z2Tzl9VQmpY?29n&%)P{a^b-UpBdZt0tRx9PyII8x#YodEX2W>Q;pbz89!J8Lb>{Y^ z12vm2fJ}#pXYT#6G3hjWRR>F<7@EyyL<63z4O~+UmTxLDFI&un3`8MRvSR;chmD*g z^e{@V&2B{3o2wNk_9YydcRR(ui&i?DrU22kybvjcGl=SPS7+rU!1M}`*MZlM{_fMy zNd!SBa`2r#`tX1(m2|)gc67t1)GJVuRrlBChQOsQw}4tjkX>2fP;8t}hh3zL^~&e4 zdRW>=2sD-0oF16dp;Mo<{`Un%yP0_0Zo2;OSej7JBBZ&#r>;shG8Y;o@_t;i8BXGG zp+W#s#c>!7ZYq24zN%CV51hSb?C?~^t1ShvmlQNkE0N1q{y(5tm`F`V~P|Lvw_t*=DA93B{ACUFq#1+j; zLMdM^WKby${YD%mcpchAHQ=HDUFwykvQ4pGx3C1O-z=k54-R|=R<2T}_VC{QLb%>Q zT>kD8^oUxZG%go)%2LB~ zCmiAH5=JWf?OT;WXiT%$({7P`XcaxD1kNkl#K6f*ZT8PpS7nFdok2G-~{avw$4iHAi_~veY|bnYVxK zU-q9MBZ(m96#dXwHOu6SMpED5n9Ddn1ow`f_8@Om***vYLckLC4vacq`9r`|sAPhL z)g4mA+<1;rtF+JcPU@#+zrWFrEB-UUD1w=?y@R4m}Q^w ziyAbfoEUW0_nMSs%^pR6?7e}tW-G_c94Yh?tl-%syI!tYe}%56x#R?7=f0R@&bo@I zi^4f=mNl*HA~+c&;t>1mAR@>p7ve7ecHjpvsGm=3EHU1ZHh?VSYoSUpU@*&MtF)}9 zYq`^=Z@=&K5I+G}H;e1xFv+1lnLc633sTt73zU<#dBt1ImF`LBmRLjMU%p2p-Wr{6 zbDh>}przF*7RW$PF1NbZ>XGI!UqmL;BL$*a^ zn26GxUuSIr2#A;Up!})>W^WUC&ARcjes@6edvRF86nw{(UqN%3O@)xch}3}mMa>qJ zU^bu`4x@E@6(3v*|B5}lW}#FRX{0jo#J>s6ggj@FU@!te3>lXR!*_%!e?d4+l9@1Q z_uh!ss8Xkmm?mblBKc{@#X_;Z3FlI++QK=N`-}*ow@b*#oUM*XQCZp+H9T|&mapPf zea52b5yJeZb8fjsQ}c5Z%@aW}C4nmQZk3^h3bf;ZVkGmuBSE#&?HNLEAmg-nE!!zr ziBy^)+&fyJe9=mxqnht5`zpxjE=1kz^0k*Y0ZJBt)EYG`>PPgV`0$U(o_78cLhmXW zyt``Qfr4j1HhsB23uND&wcR!-)o9L57~5-u)tjGS+*gV+nOL&2{6f0$r(7qmlzBZ@ zhJX~;7~?}K5;=|5*2m)|Ekc7_-DTe}(y0^=YO=XJ&d;#uXVAgfJmNh!SW2g)r;~oR z<>-(83yNsn7l}U-8k%_~Oa+g8BT~nIS~iXh;%U6Ua5Jx0MIjZUow+hB8bN2xp4d1d zI*(kJ^Y^EWleRV}zP~Xhj6E5t;R=e-Fg*p4%B71hBHHIbFcJib2$tBfd7dmn#5At7 zrrjEboj4t?UOUzd)1_I!PCe)%dy?r$FHtdC&5;NH^s?di*$BiRfNmY4#PR$t5=tW* ziVj5bUy;s54)P3$y9C{LCaYk8`HLZKOe6YByGgX z4hfs<0RSejS^q7A<%>gJj19~vDpAv9#zQ*tu)O@wCJ@x zXC27*l2gCN$t#kX%O^C>oPwse6OAI~Q-oDJaO1CG-kJ^QL8EP%!~Y4fz~PH$`M&c} zBnAOoK2zl{b6bAWWB68m?DTHJq|dk;5MToV&3q+93MZoiY9H7NQ4=rdMAnP_Kvy{U ztxl&sPt4`K1c7wR*gP!Ka~4bVM8~NmZK4DU>TPug_3b?|E=)TbQ*L{*^@uH?!GHQW zPPG>-2;hbaM$K$VN2}2M8uD=NBq{<_r5J+`Mpn}s_y7@qmqFc|QU%IGnH>RoO6!Rf z8$oB{uT&t8Z1&TUE3O(%$F5BOj6HmIz<`SuNE+_Oh?XP`{`1dJm4K&et0fLh-$3i= z^L{AyL>rDj?mvFztzWc%n;XWSiG;~N*ff$D6$eXVulW_YJZ+^GoD^eK@rc385Dzbn0S5ys4K_6z57G@0W}g!j?? zOVLi@3=!)B>IqR4^?prKL{nDlx$!>P;ub(-Qz0sgoMKG7F)6l4n*O(2xp9RO924N# zuP@-KB96M(1ByJeMEpNg?P-+|{+qc%Gwj(bHC5kY-3`KRzrcmXO3rW`JNZci$mdOf zqE9|Sdls&QCnYFfleXA*=6;RqI3ww(b3U|5!VGU;r8!aP=;)MyvL4l2t<8qAZKKr! zf5!B-hD4dpc3k?2;_s))b>_aXtmytxEg4kC;Clg*$6fAwemfc7BsbEX51Z;w&eqBc zxS0e@?hygdlqMC~bp*A*H_7)SEXk?thjr{1iv4xSuSaK+8*iYsy8VSB`13aB73uQZ zUmWEQjouvW#^&L-4~IuK!s*%oKt8+F+Gl(a|9xzF)>5M&LY$DppY;-L;3;1?U!*rX zC|%)IK%0qHkc`~@_Zw6QJrXX{;i(E_+GSAli)PYe(_Fkv2%=Z4H_xBuL8< zAS1Ch{*(C#hYZQNlr}oogRDONG&{B3ie`@G!QXW50m;jNo*T;yQX4@Y)(c#(YiM38 z>SsNxG+~OY?+yy2K+R}LQhB1P{@}5F1h*SXp@Rig8h~U&Op;)(?q!%5B+%chR~R9; zcf}i_q^uTBv-uM~OZGu(eYZN6e}-K_B&uSKzxY6`0q~=h4Qzb;M^li#@*AU8QRG{G z`_2$xwh>uXv5lcXD2*xrKqFNK`OGEW(j5#%F@(zg0usEhk=AKglT8mXoW7s6`6Wy% zU*-Cg`mjeo;W%xN*UXGEW+LhC_@sckSJ*lh)m_exaefoAw*Pe`atN@Urw6V0Ai6$x zSECHD5`WMu1rzT|6FA^zQ~u)#>kz>gOw}=4?M`^=b-`=63yqb>_l*6;E$qAUY{vOvq9z%bTnO%_if=DyNwmqic$ag9Ai?6-7+jF zA$rc^q`a~FS#5I(iEFdRmCkm(^~*XE0E-RK#pb``0f{8V%(GJ=5vbkNM*fT$74BZ7 zh1fMnKVDFv4*kP_#ZrCmPDZr>V~Wy&eWG1^d~43u@ELVF^2j7&btq0T0rf_gLsV^j zj>@keTD#uH7wc_Y4#UI@A6m>ZaTK*U$w4hn;6y?EBKa)b>H(Lme-hPXVc!ZI^RZ?5 z*V?-~_97gEpIdMENd(^zy!kD1qxY5KG_BH47SqIvpEiFWUd0pTp<3UCWvg-tVasdsZxg_wQ|L_TS>1O&=`LWp5@Y8Dir{gau-%Zeuiky%v7JeNPN6ME zzxQ~bLK;^kfvhvQD^i^lnS(~*5eps--U_@PExa~RZEhp8v!eBB{6=DPO3Q~Xpd&ls z6L0URTvzfSKdFUTBvS5wRe+uq3H>ZkwO~g~k?a!F%%b6PUBD(wzPofY)^wRi7^Si~`k*xuqt&8fiRmuJGnBKM5Jw9QGxxD2+UXm|JfyRA_Zr zZ^%0Ml&-ZJL|wIV&d~?p@Kle=k`p}OSl6d`mPRP_TUy!6!OJ;;lmMRPy1pU(sBhx` zhK}>AX2V1IVE2FKiih=5uQ-Rd){v}ql0W(IxHoN}fW_gwwrgs}+olfgF2f)yYT z+0Pt))eb7iFn)8-)(f?-vUZF;_dI8O$Ey#+7xOwKRo&p+@pT(dS|46Nc^`CQyR;LC z$XNBbKc4ZU+Mw_A1gT&EM&*z70p-F)-VLvBkZIQBxi*gonyzOssoZn2%lgqh$yDOU zAH#js4x0}psN%zSlQ4VQ7R$P7ooL$s|Gr)=f^gI6iKN0J^0tM~j$!7?d}UH6&;iCh zOxHcQF;KK9k-y5w(}3I!oyg~X0DwGFoFJ*do!seXkD_+p%qvmvc>%4i}@fgW4FTYyN%QH@~ixm;#EZYv<~hMK3l zL^`y)%JO)7?Q?4<*y-8l;GIqCg|23(l8rc)6rjH1G!Su=pE%csb+|2dwe_X)o1|Cr zATGGZ{?O6Wnq-yXG2gM_$jD>W$^yD})jU}@!JzDNY#JV1ilxTnZeA}2tG&A0`x{otVn5pTH* zF4r_mnkx7*Oe_pTLP~cn*>Nj(>rLA8oj5 zRPvxTpoH;6OcuQZ*&8X^|3#Btf0cosqf7xC$&eNx|sYrTH6MmM>1 z97az*fp7hKUI$6tF=BDSjX@2gpo79o6jGms&(78`N_B03vovdo9J|^66D~`3 zGZ;xY1ptE;_j|>{U}g|&p%Ap&NFe|8PLmjGrBt`TD49?mPWdo`J8PteP)1WpRa+N5 zfeNUp$6N_plGSk@CNKFVY}3J9Y{A%k=n8tO_CpzmXU{$*rM-S9B|fjJUFyg{zF=+O z@X)co>!Dj{L$GsyBJB8;BFZKLh)2Ebd<*{_uyQsF6xg=P_-iLjQ7PAohGi*w79!UH z5s_UKJ^30{MEiCZlf>|p1QsjsIpfMOqyBn$Y*_205rZxGDx0YVe*GM0EDQ^iquf}4 zjeXL$7`%J7zg`9x)Wi{X%Cx9;%uLh{OmB7t)Q_)X%NG_F{Z`3^pZJx1z!?9V>hIpZ z*IYyaf0osL7OGkvG*^RP%2mvp@NGm=}3{{|Dk^j<@0`xuFUZ^NhQ;$Z3oX;c8Z%DegDpJU{Hw?ufaTR^E$=StxOgj z&KpGGNc+(qtl=83AAt=JYlHBrFnt^kOKQ`sHa*^qZh&<9y$3l6l|SvrbqERxe)Tog z7R&uJQI3*OPc29nmM)K3`T&@EdEA_sH+t;G#ZEP)vVf_ZliDZE_r0jk;gzg5sNz=l zRfpPt8DFf+nShzBzwB*9V`r~*qiP~YHXf3uG0t8!1myvtB^f=CG`%ef7Smw{;@ zS;CtQn{=|9i>mFOVXT_;<;QoJ{JU3(BGh4~?l}CY@gRYM?YQRX4VzG`i|xmOk7YBp zoiQyNx2YVfc)k~XJa5JM|KiEu0x>7Ag2O5#RQJ`F(o~LV3Lxm9iN{lC>grejd#TGb zp*sfKG%Y4{uq362Uv*2+5vL+pJUaFIdI9%9&%gKPN8etO3eijt+6y@_NptW_DEGz83`{NmEMC>$y_5jn>H}-q$52nE9u0#=V znw8{L?&r^}E#*te*c2#mgFHAdHHd|*aGbdY?9n|J$1VPzHlk;vGF8?!JL?fvuU~eB2U=j_2koj|lYwUn}4P>FA??D3kEdveQWo7{!)V%_y#y@g5LW<$l*k z6Ll`8NbP>2pn@%Zr<}j{!C4j ztY#>qq-*f={)wO-!H=7w^SgFZmTy8a>BC*(_0iFR8I-$9#O7B zc93~8^~Tzt(JwArkdcpn-Iazq2iOV0eN!(!3C@=@)VuK3)1~Xbz|4{*X^n1kT5~Dq z8^wUX&0rrdbSL>~7R^!g!40*q_QJmKKu6lB#XzfaaTe=v62I-^<|%JRSu%2X3Es0*IQOvgJdHZLu1 z70(;HgB`r#s^@>Lqa3?sGt_HUL~7ccAHSjK{Px^t=FDt{kQ?(2IuSlwLI2U?=c{u~ z<-_S4q%OGs&cC~!I)OVVzR{)%Yo?sstFE=N1zFS{c?2YOK9&bR9$YtPg6vb;MI~1V z=Ls+UdeQoZ3C!Evk8V{h{yyS^Sn9}0A87uE69VX8LP%mmyxp&Fgt%h&oIxxZxe7`5 z0+}oaj`jc#Bi}OUl^LeM@qocx@95 zU5zPaEj8;WpdNj{f}Hwg#0vCr(d7Tf5C4hQZyL2s`?N(r$5^>Re)HwqVq+~2c`XH2 z9ho1mJT%8td-HKNN!fmUQPO&;$v~Wwz`)4%wyZ&iQ_)UH-R6I|0Mk!Ti^w?}e|PKQ zQvY;q+u`$z4fygw$~%XtfdiM8^D+VZ1oz2%sAzyN{=Gu3z8&p?yv~jY{1_+K7Q6V{ zo9M4@0ZcntQi(t^@plp+mFz2}P{B(^rJ|tbdcm%o7yh-#Lwkf!u{NzVE2jp!$tpQ* z(VN5N^fokv8|@U<{cgK&N;o!Gl)F&XF}8IYlfrIOXfO`m7Wr@~Gc70|g$8wdIEf!K zcDT6LlN?bY@|7_ciUWc9*)AgyX4$|`JL>t?3j2(Ct!>C-8ZK5cpPuL{zwcV&E_iC= z!Qm_-yA=l>ckZ$U=kdZ35zH#fJCqHwOhL~wIHs72vdobglr}Dih&e4FPfjO&kEF=3 zKM3(0K9iGJM->?GWRY?_q>z2mWPd)`Jaa&Bm{Wh%$|4tAE$z zdsm*qVkRY~K8phBZ5Uz)Y;@kNfqDuujoW`4uWPm^FPy_ctysvUCX9H_U3945JE%}!7D_>1FOw39J+IU7{PTjk|QvDGm0}jV{iN`=O!z=)j z@4G*4qm9Bd)3DAEtxf(qfZhCi(ePR%k1XoLV*`lAR9Ij~1^lwyS+!iQ^KD0!CN3_X z!XJP4Rz_M4CKSAl#SV8(wHsmjySy^vs5oHiGwwNULYle`#6iorG@r`l`Pg4(&2oQ& zK(QKE*GedD%GPEQnPQUCwXqY8=Q{X*<;@4ry z{$HecM6R|fKF5?;J2Me!nySSx7HAWq`|?WLT7A*~SY*DTsqPqq^Mp0Rj!~fVF%8N* z6i})PUt5gmPxzI&*f{*$EEA{9{ZChqz`eCt$2P^Sh-sF~ z*RutO_61?T{1`>%Yt0$;@_*P(*F=XRtWFx|oTK~y-e2s+-b*Li2pA`TH13!D{CI!- z&jKs(pTyWBI@tMmysR-QgjCosY_!@o$sFm@4*B92(FzG8)FFJDqGI{qD_$h#K67#) zQ(}*_B!($8v0ye??%lN0(n^OQ!=L!T-IxjG>x<~^vm3Aq4Stb`fTu?c z7##;t7*OOm)57<8O0@1}Tv~I51-nXxzaDmdGZ9m_<#TrC#vnHN`09|hu_h4>0c31d z-q%`L4^?cW4A`}cw_P(WWKu~HLCHOP(QG8LuH$;ykzvRFpacx>cHwp5dVv8FpPzHc zf^YcHU*qh3@0=`N{&%ih8FyCP$=(*a64_sHWw>%~7vKhN;3P6Qse?>g+17Kf%DT zj~Wr>^B*as=Q<;(B!bWX)c*eNkaCS;)MhA{wvd%08P$b}R^fc=NOWVF=I%}#>nAaH zCh|1sk4)gP4~!Olty13Ssds8@53-~wB$yc)5}RA+8%DlbYs6)_eLGYku31ZDWz-5Q zchubP6->Sv@XSOyX}hcSxvRe>yto}yZqa&R!~!KYT(52v|L^+8+R}Z-u&kG~HtautKJ}9RP!~i)PGZ@l%jhr6iki_hL@>$YNe-LgS4I42XI9P3 z7H&@-xDTnY?O-OYH~*X&UrLeIT5Z0D5#txvSli5#A6~3os0Ek-^6~$~E3UO3q$rur zgK!wtCtwqE$Vg|AQH0)z9o4V%Ru@mc31PEs@A}Yr4MW5UU;3|7z7rZ`ov_}2K7ee+Ph}E<*ybW$UX3rMd)#+d z36hL0wGt@w&d6D&K|b?Ia%Fw1Ws*LdCDZt%CXV!}GK^O5j9a;b_lsd0kMLrTWMTk( zwuET^c~#e5$Vf07L<9V&yj|OE{!ISY)NRZad8u1Xi;#~hoGzP5CQXDu+}R!7rn~s} zXd(f_BIQRivA$f(599*vaa4sQ4;Oq_ZIKeHMkQtmuJLwpF^%*j5)Wl70Hs@TZ0V z|9j_zOedSfkDQyr`I-DGhSNT4@m;lEcW#TlEpp)J=OX!w?$zxi`+Dvv(0=&Xkv7Dr zwuKUu=UWnp3O%HBnd)Wv?_yWr2iAQQ`Xk738ZlM8LHXslI~vz~$3iZLE16FaW5%af%XN z6#wo7%jup>*No^Eq*dF*a`nO>gh2cRT9j7!-apm3zBtjdnxNgREdsi6iOPlzZJNPiD?_GGS34QsB{N^^Iph96++5WO#arFfIXJx#iSp zMH1ZoZz(n{qUSvL=^hkYe?@P#R^t?nM7K=Y*QnjDB20?BCQN$`If8ts?G>%cuoP@? zypPz`h(RT4khsWC(QbhFz+0a?mU*sJ3Tc-KHw39Ho|5H7xn-RmdNpSbURr>488y~Z z`-^zyhN$Onehd-Lwz)WO+?Z^~+l^BgjYdnOX2ZUFl(nct>yI-5Y7>#-!Ee@co*&zi zndxt?p05AN1pn{(Kgk+F^fm^M2!7PFcKdMq=ee9nbWJICly0yH1St?=s0oru7ng&j z@3}=mP84wpDq!|!wmC zbOOz_FFYOma3+fk1`V^Mh!G)W3kKc>IL$D5-h&?mpmUy~yv2sM%K|P->D@83w0)w0 zWsLoNty5lEHKL%j5GAX+acu9)=5%@9M(Q7T&c$`VWs0P}eYfL5{=DIJ^D^?^FaF81 zF&q>=2L4dxW&vt|sW_(P_=%$CFCVr7QFmz2p^JQXCVOKC>2x~rKWC_XH7^^SiRy{2 zNkqj~))2aL45t(tvS52#6!cwCa zL}|Yl4XVHLWOSKccTbMjcffS?f*}%FRwgmkWM|_YqHoB5`Da7=iMnQ7gRp8n$>ilJ z5F^MQ{^_s+RdI{>{;pP(ab!M@oeQLV(Kt(-hPJAIvpj}Ry*bIhlyrW{0!Nng+6|hJ zIhuv$Q|)dSt)0v4GbNM3Y&a5jt%mX5cw5^Vq^oby3)n!qh;B zHAI~zN=h5^a%qt_jPo~~$LZEw*^CDy*Zq~!riSL%zlNoOAeYgnBR~Em#!RmzpNSl4 zO#87stj)o+t*;wb+(Hx$*@0O{J{pE*nmBO#cS?-WkO;xjceHz3|9Pv-yRWw8O&CST14N_#?rc@uqd53mHVTXK3Nsh zb&Lyyuaiw`8Ae7!;dlA#o3K04bj2WDCPwwx9OnKPay&sX&maoH`wtS8obad@rZbAP zug4caHA8M`(LbOs8kS5PL^!2mE-}eB{*lxa-#d19tw^Pblcu0F`=|(xv{!=%eUsKZ z+WyZ4uIhh#VHwS3pLJ1Vk(uT8HTA5wF$G=6-%%NIUe9I9$?PO7u}rb~Z=j{u>&T9? zKxRy_A=_R9OIRw&1GMR7`H4swIX7Zmybr6) z>M`2|%eo8Eh%wh&rs8R;Wm@ts;|5bDLC(6Rzze!FfYpWPMZv5_i@Imp!iQn$pz zHxXJGi(X1$8cjDb*%K|Anr=Fl315j!%D-5hQ{TDJ642fVxEPh+oicJ6%*MfjG(Z`# z9&8N^9e+an`~LZRjus?&BF{Gry-IOuh2JE0yPxb@xTBu1ip|o`wYZoTCzoekC8o^W z!+K7<=ce-FJ28!C+nL3|7}Xr1oTd~V2EvY6DNk6-vOL-F_0e($1A>DG3&F%){-nAR z|5dLMYbtq(y{h=xkN3+SD)v&uS%eAqpTr@8IAUXP5s8t4-Eyz5orTvdVv{`uD71Az ziY6e=%Vktf8ZkwW(B5RqD6;<+qDA&hi)kU?dN7d6q~ce7`f-Yty~4^Lq#W>~oTk?! zh-pDY$chYd=yCY$zU^Z3bWs<&jn-s|?&~_ShNK^kN;vjCp2LE<=I7bYL^#Yxkl@Cmb8AnUNyTvWw|k6^#tLlxvd=e!oj+j$*CYOE1^C+OMp=+I6YpjWlr?>>RF zk95y;xE>XD=(6cgZBPNR92z-las9tLNr8W51VQ2v+!?Viw;4AkT03iw4=7IL!u#6( zMmtS}yOlWvkyhuRRMpc!QMq;C0~A=UKLLITyev%VwvkKh~(& zJ&dw**uSBg-+X)II|+P@*LSTvp@8GNl!=3B)++~{XAo=mTLi^>+SCy**Z~9nxw=Kx zP8y2tpusvq-L0$}BFu!69s0@YnqXmrZox~y%z~;3M9cR#%aTD^gH`E6`fBZidQlcR zNC_zYDijJ|L8?Ti))`8Y>+W*r;(c9%)C;tg!6n&FUpv+MCt{tt%ynRynJm-_%*Ism z<#K{}H#b1&s1{K2{P9YGx|o*cpO;W2ekgz$JcD_BQiGrOx}k}(JJzYrr+%QE5cUzc zVhehUr`?4qYTw=ErkHj>S4K?1K}Tle zY@5FQuser=HR_S(RgWKp?>9g0uOffRrsZKmjEUC!fXpcN|96MXDC7M_$yk~B<77_@ zA`YAU6Yc$F74mNHLhg7lseHSxGSOIdLe9b$&TgT>I-KFnM6hr!LRL7dt3lsbbM^>5d`~k}#vH zyzsj-7rifORFg|4=;8DHX;otvq^>$6J2m>_nq;g+wXqO)_p8uSb|hyb%O+p%OSf4f zuxMGjcBMH-(RzQRb!~W?35b4J;R1Eap=%pTa97!$7b z&&Jr85be)@tqOin2&?JGDjaPZRI9GMeT$YFz4}8wIe-4>P`S>_HYLYM%71)|=67JP${_)4u%Slj)t@?}v^l$zwzDwn;kh`ex&|J_AS({Zbq)%$$vhISlr zOVlwzSM4&tti#?*QgqJVD@ThjhLlqCG?>jKp|OqA!V@*1;)Q9rg%v=LzAC$oA1E0( z5^=-Tk{jGa%vRahfFx}OQ)O*3eXNB)32G!-jqQX{M&OLi9(ut+Bp7%xOp3Sq+NuL-0yt2Xch$GwPH@58JCaYdKCE6x3#?A`aM_Mv^w%BC z?|cqsDm*q5^*_^Y4rV2Pbexw0MvhfD@G;b^RH_eJ@?uMiD*Er!1L9vU``_&rdi1dt zhVt^=>{wtarXf2iKnjFE)cd!jvw-@eQ`i#yZ-%yemIA2gf|zlZ<%a)f;KEUSm=-U~ z8?y~;MY~{i>qQps$YJNA^sHXC|E>*a8$}p{SUfxmwVHHV^B7F0f>}QGVjGNAO_$^N}R*qwt%$dsJdZ{Bywzw-Lep)O!f?AW-bC}217m)yf}x4NIR(zXK$bP1%d@uP*>`oiUO>RAbRL?J-9ca0$Qubi8@tUDXtVv zUdL(#k`Jyob=A2nHn4=2`587h&KgIW>W~lMXPc0;2c{PmI*{Z~Td#peoZf%rXkom(v|BUMVmuG{Vf6lxAyKT(MJ`Rsp_N_(f`>83P z^wtzQ3~}#pn?tRH%i;T zPun$$1}uH#pj@7ovs{jljRXHAZdSuN?w9#q7h0CEv1tNh_b6e=`|Ikx|0Hh&sWLMX zq04-cf)5m6?w1dZCx39T%whEpK-6>f+V9C~+x=u6;puN4&N-sraB$9yVb1P%9?pgk z<^;efS=e3s79r#;-@i8+tR6=X3?3|v{UN*BjSpx2+8fK&c7(*7#=5EWs` zweEu<#Q2?YKB5yb#;xu>0130ClF!~Py53wk6>-j|;^2<657M@o?M-N0rZ>yFWk4VTsv$o+#(0o(rKKNp9#pB5UD5Z*4 zRl&R<*SUs-b$%()Aq$r6#cs5!LylBFncB!ez8j(%eZ>^GU#`fO@OIH}CmPHv-sx5f z8~hE(rf6XeDfSxT0swaBo#KvLD9S}{^Q)HU+uJmPP{IOp_Ma@1?;9n06jv=fvB)m7 z_$Os{WWYHXrBx_e_ieWi{=?%@z=eDPUMmhv&F1=nwr`x^02XF^Tsl zt*NoQkFoI63K%q~F(=43w*K-54bdmuxn-5R;VpK;Z)U;T0F{rY#{4?7CtuUw_F`ar z<2}c@svQaop2nZ5ZFdDFOo>bwkK&2Hv3v^-qFkf~4h`*wu~1h*PQs~^IBmrDwe~}$ z{mb^jW*cnhL7r}E{!tNtSS|w?{7?u!_29USAL>u_mlp}tO`a~@_W5Ov#<}Q^73aH+ zcT1j)wg`D;7m#LV(FZbmWj8Di)$XRbi|h!uf11K)(Ac5G*?)s=3}7bsR<$oaGC3To z&y8!VOUP!3dSXukH8v&SuMgC9Qc2lagXEoN4IA7YPE!^cubOsdPqkUDm)pjM-?U`| z+Hw|ep>^@%XLVEcFpHAWB7t9$_1+-s>=}e8EsHZn30jbGtxMk~YL@q*1N9yS^u@aM zV+C%j#H2Ci8THNC8zb%U%-mn*0|P#B?IGR&25KVO@}mS#dM7#eNCzs?QnTKr=#?70 zP+PrM4$Q7%Bf)x#_S-rM;VhpM4AV^Juf8u`K=Opm#imhUaLr8}e5<6HijT?Y6+~?F z|K}rvF%ox-p_xouy%G*%ab-VeuD+?i!1ujBvXLXP zpI#SP^+sw(k+o~J_+AD#Yxlgkd?aw#sfr=7=DzqZlN!KWv>`UL^ur6J%LHerfD1GM zdFCOIM7`)fEJ3dkhFu#Q5%}+x+5db{=wh@_eW%G(Z+y4QL2LF^eu%Lipx5jjDf@g| z?YXa)+I04M?!BduE|;EwC5y!Q0N+Hb|9$x&U~V~@;kA^4%Cy$cA{Rf=MBuuq#=oyv zyT3=fZ0DpN9N;~K{!lS^?bYX+K0*ZwD!%pLG=*8TPF;qo|w{4K%jZ!0k(yp%;m zy33{gZ3G)no5x_Evi{f@3Xsxzk@Y)0VjRH+*-m8*iy001=kJv2 zX}mtcEt99(b9&d^u@e9JvEBb>F2Ow_@IHNR@~1w*GQeJF`3_0p;ptpE7r8V~13>HC zd%x@aulrL5q+Cll(Ev`Ja7@>Kp&K~1h80pA@QX0`nykw5vu?0pH9ejxkW#`}KKOzJ z*Md_c8~TjMeya~Npxe_Ci6ZHA6?Dze%muTXDK~c={W5K*Q_oGh@&R0zbKY6!FKse! zEcd4M&bTJLZht8`)$2eFPicEV!<6!9)R>YJz)1WXrf08+StTjK}x9kxMvjxJ_2`d7u?UNRC&XMx z`2L1H3QX4zYi!>;K+7UahXtu?G2w-1`6X36%qi2tIskne30Rg>!gjV)JedMY%6r+^ zM%p5}@o$8xQdH$*99XF)|M)%;F4O^*8?ANnP>W~!tM;gnG&)x!b8|eQ-4fg;N7ZSe(5=bq#He5ls7^$?&W2ZO?Pobh{hiZo*`vwSo&Yy!u0K|ed@ZUp@VU63Zhv#m7 z@|eT#7LsbCW)Ccii`DUoX76=_NYGF`kXn{%SUd1th00gr2HCcIhx0psSkg-jx2)+A z!`{PXE=D%(t692)??pCHH;H!MaARyB<9wJHZY+6So|EkKGG{R?EXW36b3S_wey5bo zsj4o1ispH=Ao6h3&()Uqfmj(0!Q>WlD(uw9+rdn$jN@cG72l9#>@y798n|BJf%k63 zcHMSHPlD78Af3|uB%P%gpoQhR%-*y7AtQ6 z%=>~g?9+2n9TH#7Ld^Z;E4PIs(eFBa1walj;+}F19f;IX zt58!^4}u`{2Q+W7Qyq(vr`3PNN0-KRlsQHlR7r>SfW6VelHY7kt4lr-?YQ%Dg5D{8 z!X`$w!QN;*tep?00HFvuFOYq9`)i{qXf@Z26)<}@F{7AX6ABTI%8wjmxD2&{LsM|l zYkD`qP(qMW%|M*eeFH`{{q@ZnL*$#~+x1{w&XeAU7PI&E?mXGp8Q#$X1YevY-~-I! zuQP5T&*}K;?nc8nnxl7nyhLj994W#w1TPX}=wICnAosyp)eYfmeb&Jr%ksmGp&(9f3g?ho7h0k;hf95xv64|Lju!Gl(O zujQh$fjF&D--TKy%FUi{uf4spsywI<7h1oTQ0XGhwg#1Tnt6W#-!FC_Qx?a&pAS0p z$quvCf7)7TMEZ%xWduPv#x;otWI^=tD?~c3q$!Vcnv`-<60&1Ij3XZ0Vm=^d(3#xQ zVBc*Z8I}J%BZhkKh~Dtqwon8W>E|j*ltK)G(k6jecx9*l1_Lgr=K)Uy$*`)GJzm9Ua zl})VW8{QuG(l%i|yQH0eMhLt118}Kyx>B^9>(84TTIp!nYU1Y%6n9g6OWY>5>RWDZ zB5MK-ZoD{RT?6*P-u3!M!WC~H75YA=FF#buQTci~m%LYegOESU9d$r z2>n=3w{=hYDH$_x&aq53lH}s7pG)i-Ya!uism1TvP$Gv>Qa8JrUpI2iqMPzSiLgKo zXYa3Fa;Mtej7qApLn#Zbumjr!CL|{SnDkyFbzdRm!G&$^fTKnHjfG26J_HFm$~!Ix9JIdXTsL%iHY8&CqqeQ~`I0erH`s1TmLX+BFD6*d1mrOQK(t2i(9f$a+Bz^$3B9oNI2;Ts^$I z%O>?50H#)J_#6DZ;aTFFZ5p2-{CW4F8r`3I9e!V9Jc(=ae1`bd-X~MTfk**XfPA7~ zY%g#J?*QCdZMrpJ;xrsPJ4TeYF^A!u>$iySMXtKv3qgqI`BsHuwg-d+`9O^2vV5<( zQB^6@dD>XcMO_Vtlmll*ubhJhJe?)##e>(@!pLEoG~|lh{GsVGSZL}-L&Lvnkp`2T z^QjAZZrhno&~Rnqr%nZZ@z`@&j6&4V`jsR2a{ltD_2RFzgsu&|cb#qAov%%2UuSR1 zSbYn!4pwsf*y-@l92ck=B3c-lyrPu+6WVFRnNRzXnh0T)TC+ArxNkC4g@-#{YF=#X zvkE$pnO<_Vcz4G%C?I!RdLA zZ#%qaKr^+jzT%S1Z80H1FyJQKfS!YB%@kL=x-bzS7LYb_n&~1B-M&7M;~e|(cps`s zl~M_w2nVy+Z?ih58A4S@13_|+PtD6uGZw4Z8?+DeV;{Mi!-F5}YK0ZA0$ z6#rBpkgN)Fu7t@fj7@a~^nxj(nhkCY+MUQT*9y~(?0o5rj7?3aG!hIEjZE3dK!(-AB0%Li@y>10uWvTmuYMHV3^!Rp_rHl!IRvdGbYv4&_J)DYxr~V|!`guSIfmq6Id59E3d@ehQ&m|?PRsb?%0Yv z87a z?7V9rGXTJaMY(wF`w9#dv!2cB^3Bh|I}rIWy!t;~eAURVBn#Cmz(zCj0~7@zx_#h| z%AEVtmg)h({KsDGgB7_sX*FI^r^2XU0DnkftK?0_5M-Neg9jNo{DZ2{kI=AhlRs)7 z7uiL)QL%w)9bu_-|CW({;i!{9k_r;IL{8lRwhlD#Cy9n#FKj}Yi5IA3{RRXrbjlcW|?(W7t5Kf~%WTk8&MxzMz z)^hEIvLye`OOt)9RkRe(?n_KO3Yl%=Sw>Z4H+4Qsd)E}^KCHJ?aiT`o-dT$~HFkOX zzcKAA*$6a$*!;l^L=>q$(d^^ydw*Cbo%o7iHe^^Od+nA$FNDB*W*cCuM9k?|;=m(1 z+01>ssjG#L)P1jJZ9+7b&F)@jrJ@BaHu=xhg%PtU5t2ApSE}Y_)*Z6KQi`qp)4r~I zg~79EAMV6On}+Q6ZZV-GHMNfCG{k*cep65kCtE!ZD8qI;#?0)N!jVO74SSY(ckwZA zcoH?cCqm+{fc0UM%mUxQf;&(LR1rSN@68yff9s7(Q3WsF{F&g<2yTREFmvAIq0m+0 z6pivmtOc{BJ{Y8YQv|oNB2vvn{wA_EZS~ryi|aj@zEI+MHY((79Q!(|)7>18Zi7xx z8NPLoa!)(U`I3ha??9X4i<@vt$c9FmUmDWRB{bRMtGIm;nzo!D3k&@=4sl2om7I3=qh7$nBgffJp=!)4wWP6iE- zonf@>lUcFjW@%DWHy+?A=1VsT6_X0y1>6;QLKJy4G=028MDq_6bJ*>Eo1j07dNacV zEz&m)OteMOeWXCt6_)$ZZ=`~3g&`H*swFFLUaGGNR&42HCK@2PmKfae0s zssF@J^Z1Dqgze;Nz`BD{JlT&uCu}f=>d2K$5q`#U`b5#occUoG$ZzbveFbwWjeM?Ss zsQVQc^z%iAiD%VL%Dy`-K+Bi}a#J*s73uF7DbdW8Q;IIZ41=gkl(qHwP1sVOz}lIY zTz<4ao9QV1Ei}r!?38a2L8UgR#=2$MRq-75wp=j{#VBw3a95MGS?L8a2%A>3fQ>r9 zLss#46In*SwWRacN>qZPc=UT9#c3?A`ddyF zo4h^jpiS(bpJ9qnU<#JgYI*S&mO7h$$#k=UT4E3eL!qF(>B$^)^Q5umFls9+ zeY%4sX#xqNkCL0~F1vpyHnnOoLs!-#CDezPl?oFj-gX5*UyHxzrh{LxIdu`Ea^P!_ zq3e_*njvrpF#V*Q%6F+{&d0d3f2VxJBCSl5!jJ4Fo{m91zjW5CWtTWM-p8Sf9%=A# z0BBMTi+rt}sa6cNb3;6nLj(ygu8n0&f&UYOm%6P>oB|~M zY>u#mY(~laAW9a^C?FB|tl>!D>Ig-yi=Y zduPD4z}?=?VN^R(vUqDdNh@z@M%>W;amN~9OOT&+>-~aJEV@}VIy>I8L7INssQVuF zRxgv@_}BPe?w!QRCajdzSfzCq`D?j2>_$VJIlsUR_%yRZn0^QHSmVyB1jENb&j*Umc&q>rJGzw^35MAHNwfno>Lu2e2`0yL$YjNg=2%61XD> zL(Ug*JTvi@o3+SqdSBDEfmIWxcI%-xr{3jHZMu;beGD~Z9e%po|E&WU==5mN_c$>U z`VVJ_CI1&STCG`ROF!2PbN_q_KkH=<71ymrl_zsHrSaX#VKqHBMs>VINho+AIs(988Rd z;zX_&T~~PI<<5BK8VFf;HkcF zll~Ev`N&ZNc}<)hVR-81tXWt_=iUA-TCigZm2f*7{%NOw@zZW;pFBFR{aI$i6xK!g zL-IoPdiH~Pn;Bxo-(jQY8%p-n-d5(s$>;a)1xddjm^dG*nfm{;!_b6{VjiXpydi|6 zdV`?oK9Y>@{=J@OyPubLD&w)k28cIg9(Qsl z#NWN7a5{Mrnz%bR!fPLc zf+jrYpR(_1Xk`g87pb=(@t1xV2dZ@oF!krCW7SWY5`G27?Ul0|DK?pkz|Coly~6yC zvF+&Y6aL~I2T&5Zuy0{Mp_r|SSJq0*k;sCwNzXc7u?r;oEI+CgCPLV6v~j$TXDUME zqR6DwB`$S77opxbxef|UKX+^7oX@;Ea>1!9m>e{D@{4vOW{s~@o6Y>U)EH+tsfMY< z^Q3uv=a{UiqB!61yhh!A==s!dv|)aFp^Ctpb}DD1rsZ77kZt-f#7xugNTGb73p?>+ z7s4fhrkeB}KV74PYX9*|5ZhkfLkye-(Az*!E&zJ+Ureo!068D@mU4&-|Aq)tXAD(o zBtK~s_|B8T%vGe9B!gRnh9ZL*~k^(pYlvn-0I8Q9{iT1VcpLl;l+UaL)Omv$E72W zt;5Ss5%~x24a1eQ%V|?>AmPpq@ZPz&7*`*U5;-23JDwWRL?-@V+7`rq8Lo`^Qj(cE z5A?Z2Up@J0J1Y?1O{C>^EA5-8)C>)GponD0jvZHM6fO0B~cg~YYrzK1+#Z3oG zzI_If?5m2e5b`0(f}m2gEWn8TK9q=VuQaJGH}|_9j9i04o^&RMILMW4H~#5oqK$j)@%bpOW_VMUdfiqznmF^m;1iX*oGSCyLkN!+Q=VWS02n`B>)-4 zE>YY*TJEqk1#kAK$le`;CZwTs-j!%@R;qvZA`^W0B|(3sw^k6Zg>tTZIz zSZm!Yk4dS723VVXUys5b?iIXsy~o)(Cn!K)KbZPOi3h~5-3UP?HYm`cFLhm|urICt zX=#@+{uI*eJl`MPv)V;`{nf%WeVGk;q*1Ae0m@BuXjH+XH1zrn**6EZ~qv>1L(aBk1K=e$$h=$$G45BK4p@`qdwPd@&6fw6jm6ln5TrhIw9rGnOP5c&TB+_EcSHWj|4}_#N3U z$T1&AgNb2e5@aDHI>8StkXU?AB9OvJk{;8< z4zlT?u9(CqfQ7U?ksO%bVGBQdfgc*ru#TjO^I8`yLAx*>_$dr1{D>b;lA?+5e{O#$ zTovT_gU$=cLKX#9X!hngIGWL0!wAE0o@~38)3f?8r=VBbKf-(~RX`Y(wlEcZNX-7k_@g}%VH{Fk|1YD!}n|dvJ@&hR# z0B+mC`)%b}rgV?QhSDGR$^;gZpy-}7K1Sx3t;wZ+JJq(CZFW;%?$b-td1%=SsoES1 z-VCeT6t1JbbW3rAeH!L|>**?Tw-8V%O-{8oyt>4t_3V9A~V7d4iGhCCQfV|D)+F*rICN zE<8gwNQ0!3(%nNz$I$hVN_QhLbSX$kcbC#FUDDm%ozflO_V|wXAHd8#`@Z(I);d=j zcL!Dt12RdCH8%#0xkzJIf)l{72$7nZ=IK7zSYRk>!|>Qe_X%~c9ra3!qsUfjDWdRGY^qxJfO^PTUiV%L?tPXUOnVe-2r*&3PA=VogX*2i)9c<4kGcH zy}8UBQebQS$f8&K6%i7kc{Q4+esFGR#aC#slTPSnwYdK?pa$9G{;XSlb#}yvO4RMG z8@*bFPiN#@s%y&$s^!LBW#7$PsjYDP(+yk78=m zd+z5BbEH3jSypkNPS9wxci$#utcfIp8N0{_wKhcf28 zos$;s@vySJ9{HGr6-0@1f!RZlIP9uFbp2-y`~^NHAkd@EZmB-xx&JJ^UPS6tK%hNS znI)AQC2Qww7~}y+>vm?CnnKTt4^~@?_{J!u-1=f(2WdgwJ{X?r&{Ey7M3x7BTgnbR z|8~&`&BFGVr?_1uVha7J&37qRQ$FLfgan&(WCk4wSyh=8pwKn%NGCzA3SL2`B1{w9#grO)DCCrl*d!e|>Ae&XODQ z7~?f2L(BeaP;Nx&#UxX&`XIt?jKWiHKx?e6pF=E+SEHl=je|u^8(qG{d)I(K55B3limNYo{E2BR zs|)*1GL@+`{;|G5KvL(HlPw#4e4}Q#PnpxKdc-b#{ayY$KMJ=@t1)sO)-sg*6bu*r0jR1f0sKv+nzx}_SyTQH-A^= z)X!2u?xPZL=ZfWkfsFOQV<0OHXw%uz#k-!SFum-tc6|7lkWZbxl$vWIko0?abjq=r+dl1K>)kcw9 zfN-_L*0IOvGegStL8XLOJ6@z`Lm1cxOQbh|wz&i-2UCt0&jE$t|4!*fygjE86)}@d z4^=L|(;;N_I=#>EVx`CLCEh8Ht_Dvzna##2cN9B9cDYi$aA_`AKzo0E_&PYSbo+R) z4vf29Qs1tY8c~)oqA7-_!fW%3Rtm=W$t7Y#?x==)7)19Py^4=CP1^baNf1xMY$4n{ z9j5N964rDzahxK~Wt{f8)xd2=^PhAmW6$$W-yh9YU`r5w8XhWyC}A(BzSx^#RgaXl?l<SYK70t zg*14exgo|_1FBN(<{Zq#mx8M_9|5D{)RigH^TTs)0?S9!!H0WyrV+iC@DB+hsrIea z;4=N#%sKSO2Xc?)AF)NXhItAH8H^PnN&wpaUwVb$g&Z+={Ss5?CHGQc59R14^{bq4_3JUC|Ly-kC1e934-j zQ>!6^JtP#z^3(H{8$HeY`^wk35 zLe${wchN)ekLWy)j98B$3&Nm;T>lRPP2xV3*&$Zrp*2&Ssiv*`*-;hcginErdN-=+ zI01V1Tl*T_=3521W)xQ7GXCn%82X|mU?uqR$v@trl^)R_oBn0ZZlvK=sqeUR8pbOy zz@dIJ0<(1|IZ8?fTwc1!ECtD>k7lkcF2 zR2~~be(^AM;#R}=}`^?EL%{o8+5 z$a%Q>UL;nVXaWs0e z9@%V)s1RV7a`l{&7MJDm=jh*{8G6l36O*zwz-9L~Byly0x&1a2!PZ8+y}T`8){Lxj zc~Lx_u7P6bF|uOy*SBzL!Srt6hN|JfpYQUUFOUts(mwwnzNBi>l^;h|-EwW~2f!=6 z86q&+E#SzCO{;@JTGSuF@NNb2Nd+@ZZJ4WWd(5vtNYi1=C(A!Z*TMXrg9jF>1978; zT=IK)L`E4OUCax2GB>-7-ygrE5-M4B3RNH*Z5TWyXd70I3Jqbf<2kR))zG)znCi{j z;4>2W)msm-X*4;u#ovHhm<^h^`)>*A=+E{>F`*iAoy_V2+T1D|D6W0U0LsS~wl7Hv>?7UXoRJ;5xpv%=hV=xb6TzpYy7p#0(b)Gj*qp zY>j^(ueOESx4Xnwls&F;wZrnI;wRf0q*%>|h>)Mj~x(o=09h-%C9rCB-TX#h7^Kl@<@9R_VY9!$%&KSLo`5et zJ9>g90E#L5Ghra-5d+N~r04uhV{%vR>+JOv7rq5}9EdY#K{y)*ICzHW%$AS0z>5N9gS_ZUOET~F3R$4Q zKj)?57m8PE1PUUM48aEtS+LgyLB9<4@~haohD4MeSMjrNy^ zqjeenc|9484~jEK(~g|AIMzD_9)0QudOO8BneW8^1FQ@u)5wRikC!1n(~E7qVZ__x zQR<6zZ28%}H!n6Id?B(pkILu2Eq|m~`N=u|vu!td17TB6v+Rb^p(Q&>f32tX^{8KL(stLyTy4?Xba5Dj~17ElQTrKsLh?aE7fL0z+1rb z((%@$0vjy|TJL8s8>1R{<_bzBWHVF*?hH&yatMrnrYYYC#lLdI zUllRCml?aQ?JJJgusIWo(lB$W5T##c`R~JD9Y~OJYMcTb z<9Y+1>lsGpfDw2tY$E;TMVKt=;o7Y)DOY8ZbdL!+`$GrK*7fXO_|!p3h!yq4-8(&v zpq5p_=QH{h$HO<&XNb~x$x7x2zIDK8+agnGbF|y_Jzk`*%Z64eOHn}9ARp+* zk$+{Eo451L<+Hv1xLG#Q{p~YnhNW(=-NuiUZg)BO@a?kg&25Y15>zA>q*BG(3|~-9 zf;|(CXH6o40TC1E2jdXXMN77U^&MwE)7pq5vfRA`vn#z;w=u~iXhGFjpO(g_N&qiu zl7YLfip}m@hw(>LaR60@onv`0%t-xSbF&5T41ahlxRBK&5ilsQ_;y?FJ z*~j7E$i5-HkYFa82!|#hMEpz2dvT1?WbA1oHySzRtFX}u3QHb14+ zeN%OF{$7>P?sd^iILo*pU()gR)4Kd_ zDmLI}L(X@MFa?t5)NNr)PsjF;!ICVKR5_{)3R3ijPX~-AfD1zskJ%-_Sf4Na*@TNS ztL=_N>8sd6Tf~gnsK}`2@i`e1=E?fz$T;fX$xayxyx_~&OC*{4(ugMkkZGcwD2=2J zi@JGQ3z5DfSSc-WP@>ZXlm#NSduFj%|DN@DVeFM_F;^+aKli5a?g0=NH>fGqqs7?} zK6gA-!~4Aq^P%L-+3x&Vyte9Hno>`{(T8$6k~W;m#z`0Le2?wGpA%`Ko$gG?s#K=T^2V^b ztJ%5@c3(7QgZ zN~7EBTbsH#BSKBvj_VfkHb733(S?R)Rd6nfP)M)(#CeA%&4H=?^>F98CjuDD+`c0E+G z!q$^q+crp+F#Xgk+ISm%-Iqa0Sjt&j;~vdx=4?);-9Qe2>~MjsW7%(d`Ta3*`YUJA z`}8SwL%&EgjnWKUU(C{dVDv1~SNZAwkTQLI)A#dSx9H2|x8!X42Obv*I1v0>bL}WN zYT~n_MOs(874AOBKg1mL>qTHp7sH;>eDb0&FC#)0C>6$Mp%xm!xvTGQ5`r#c{v*mO zC2sdS*QhM&Ju=cvum>dfkk+M$n>wKB{&G+Kw3_GOdw7rKQX*xb7dD1h;^~RA5{|Ve zKo|I@jDv<50R(pp`rg$gYjz6{YhE7OHpbq+KDYv5&x_z^|NBUoT+6B7ixX2_;}x7< z6~9OyT~}`_=Q*a=!M$py%lMo>7`}^Faj&}8Z9amDFMR+ak=O4sU1KqZKaO|;YqIBK z$q>lQ{N11)I;4cMgK{H6w0#`7Yz|hZNOA?!Rh9X@T+3T)HY@5}Z}7EcD|{fsWxxLo z`ly%DENu7ePOap5Zsp~1qrcd6D)dXG4c4KDC>HXX$F72ebNPp}M*;`kznFzB1r_wv zw7G>N3$GItGb!%sFJg8fGgNFhJx2YKc#jp4&edscM!V=@gT3f+@GVycjfUiS1V6 z&4->+F!5uDHKyxMK64%-98#I?RP;@wb`30aK$i-F8Djo4cnkp#C4QJib2injH_ILV z@C0YGa3Qs&i~RYQ`}^V{{N+rxbgHFx{Y!`n_T|>3xc=EU$$9HhA%Xg1Weeb{{IN9C zejAeR`n?NH%!-&6cU(+TRSXp#fw>x6Wde>?PdW~PhaJg?Uub$Fm?Hkfdhu$TAr_Te zX3bBu&VFqb8)D_cZRIwIK|P|+7?4WE{C+L`oMWukm^7M=HBXKUO1ktKE#Upqy7J*b zV4@?S2pkviH258Ly7ZJ*S)2qRFh~RqBmu9=O#R;$X|_D4JZ#jBBzFuM(gWtL1tFMD zhsp!E8C%cj{I=R>n?2bf%T8ibgD1>NgnK*+pR0*}c7C4GV!0&4FQ_wR|0Vs>EFt6< z9^On-D6gn`qf7MazChnL7aX(ATh|nwmW6TD&!=Uj07?0B(x_v!OJ?a$t3+pU4mmbjOsD+XjvCL$S z)r(ZtJb5M_JwAbmjj2S8waGOm%oQZp-H9oUE3s|2(v+Z8rYBXRT~XLGG&f!^ik*XN z_F>XLt^D7{Eo3Z_NjplbR8~l|(ci@FU$7nn#b0YL;Xw$6wwH=zmf)prH`S(VuW5NE zOqsvr&B+Xn6#^6VA7($7(}(%WwCA&MxF?l z{oThKQdgQpD4|2Rzj08Mg#|gh-8cTAdaKv8yX{Jc0is8W99*T1lZ&#Y7vv1RlY9(Y zrfpP-Vu?yq$b&V{NJ_guVGGv}_ir#oV)R35Ve)ylYDJyel+=tCdh6&MGJS5b;vOTk z#5@ne^6WPahVlw+vSZjct9=m%(OG=|GL>EQqAxE*g51zdW4opsMlQwVbQ^yo5UyDj z7^EY!Gn9l~0!{}rgvHFh?`@O*EQ29u1#i&}WQXxfNzDJtfz$CuaYC(R3D3Fd=h>Ug z_56!?LcOQ6*5)&12KaDLy2DV?ZlALd56YqwRc<3D1N8SV15CCK9^288a~PG%l(9!!N;k#Eyeld; zPRp(p>*U$e_uAgPBIlkZJ|5KvYL~ijVpg?R*mE3yD5faXg(mT`XUJR8pYY|x;K%Ng zW#qHB|Jp6>*UT9eB^=GEp9Uf_uATQGyZ%QOu4sfg6Aq7pQDC?9v~heEvPuROm`nVc(>wBCBB9>XrG@VD3-v3Dc-R{o?&098619X9RntM=ZBL@CvCa{h(l zt*66if$R@kju%o*g zrc+mn({C=}3KJeHFb)eSz?ar|eM(&Gc5EHbpKp##E-^;oHb!Q)XU*JaJG{$If163? zau4aDpx<7w?CPg0?9^G=<|d?YD{RZ3QlH2;lW8x3E|}()61jM4mp)=>vP*}1Jx-q~ zwd`XY{fDR*YVD15nNS1Devn3&@1)Lro2pFc^46P;(jYuGK~-DO zbZ>?gD88hUi@gMR@pMsoJ1ZMyu*E;U3fl8ZeY;*5G{u88a^kxLySoapux)rUaA6YX z`cGtJ+3%{S!SzMJO-&4=77+-Us%Q>6w0k-@G|{+Kk#Xl+81 zD4no6{EVz4Ia4yTsG8tI$R*8fdY^*=_5sQnkw9;Uxr9AkaJXqDVLiL)^bmELQ=KcBz1&{7*DGq!Gko zMd>ukh4tg$Gv&G9%|VxG#lpO;rGg=b6W9?qf3h-VGu@-UYQUpt*ihQJ45+1?JU)DN zB**iYzt8#Qbf~Z!&2*&?NVMR1{Oyx7!`&p--_D?59@Ecik1>{}-; zW^sT!f$@k*|3>9!8Ft&{#dVrBEo#rq8-HuRy)mdl5^DjFS`e1~NSes+S<1hiNtYFe z1UT)tzhqKHuzjaO-!+LV*jLPX%=Xpi)hv*A#X0taGDP})A)#OS2aBo9viA4Gd4D@~ zjalv%IM11$js%+&9P*+1VH%%ZUg4k^!X}CjO(Fz+>yDj-LnjWZ_hY#$KtySD&~Br& zn#2%W=OW4`l!5AHe`FHsC8Xb`26R}#*?>f!f)Qw_yFhf3Lx$y%+`W(gyCDK(*hM6s z-Nh51K&h`K1o;Cf0YHdlNwP8NLLYxk>v?}i>)eYSYFVMw)D{GwOAUAT3^EB%6tAIq zOM3`g7TnkQ@X1RJ1Anf^+u4q@2AZ5BC95^YE7(L6wTz01`bdg?R%Cedd!XAJDI4ge zX9Xl6@JbIQvZ!l(cQ1l^N38lZS%k8C)i;7-g2p|K`EvYe5j(=r)0c;m*sw3;#M=wE zyny7rxFPv=*(a6#LSN`acC(OFDx@PKX)RgY23v@uxO-TLzx_WFBA1EXx0Gx}JQ~5_ zPfe+#qWAObB>cB;x-bfE*MCgc*fg)N8Gi`~F=L9sq)rz|+TBZgEkxq^F!_*RB@m(l z;46sOjZDu+W?FAo6#%%Ye1TDoG%E7ECvAv)PtvuoA%9dUQag9gkUC$%Y(Jw zE$~C13Ls+N5w|d_r#rFd4b;8RfPkNrlTaH6z_0DkAE@j1`sg##XbJC^H>*7T2zY6% z+*jdCqf$WR4HZGT;4Bqrcgv4qBFyX@kUX#Lyx7T71NJ42loGoAvi~A!AT(89npw2> zaA+?d@#hzR`O4jFwE~GBj(H?(cC(*Qt-@%cD_U>#RLJq2D?0;o!Gko(@RhyDzknOf z-UVB1%d~Bv2@*{5(qd|uU>8StqG5wzeQ!NJ%L+X}0UX-lOme&V$ z5%pq!QjkV{ao?E)XM6}X^O!fQo_>BXUox()+Bk;GuEnA_SfA#fhn2oP0H)AR zow8If2y&%uL`DRubb_nnL+=~6<~1`4h|h-ZiGB*Fbk$<*iQIzGvn}5B(e!sn2A>dq z5l}901<)tL{D$-eq%NMwE0sGsl7TiZoWh8Jq<=guy`s4sg}0e9z!fyoq0{tu+WZq( z7>GBcAb(F+$AB!@;*{pRb!g!ze%<(W63=Jkn|gf%DLkK6!D?F8Jezs0cy`#Dxj``eq1aM`(@m%G8jG9XtZEHdKJkxhZAgK*Lwc;i7$ zFD!Css}OHdV%ehFmbdg|Ru(k3B+ZBn63Fq1QnHOTXr6`}Z4_m&H`97qH4AKtQ1qvJL4by>Quq7E6~^94U&} zTM|^qof)LPZJyYtr$jds26P9-f<|M*p=h@4{Zp?Jhj z<$>2~IjVga_wgOc&09ZAt|pf74ob#EN5($_UQGBk%C4;dO*2l(PR>rx5V$jk&ShPa z;{n3nbAU66vCX9r_TOnP*W!wdfD8F2L>5JF_mhG)K1Fw>|9Jsg(;gE!H@M~y5$z1= z7`5?rJ`J+oT0$ypT}Z=x@R4D$K25Bmvf~1W&th$ocgeHmB4c#zHlui;!A3xCHPSOm zuKSLQ3XuX6EEBf99x{n$gLzz%TTno7$httudwK)om-@57O6OH!LAA+0KAuH1B}0fE ze-N|cjYpy!+H&B$ai=7(grSCTV&uu*rtdU!1CFQtTQDOpi683@(j(x1 z-I4uej!2w&2bi98iS7`x;W~oMG4#$*eKX-o;`5UC)uB^ERlxa*g>v7WBF=IRPaTiO;#Hw^=ob|(Ai;WH7kOZvP_aG*( zn`07g^dbSFEuItBM-0}>kurTIXK1A+mk3SrMLopxUz9KB<0!C>c%`LEUq0>Wl3C_G;P zo8Dx&(Iw(BHh$S>0=FTurXaeq5z0WLNK&3b$MZ(@B4yW6vy+1)iG*Ukfn^QIS_N1V z!84Z*no0JQHN=zQP{d^K{nec{{iB0#rN8f5Yuc|wq42?YMm+u-mz_Z~#H=&nEAFdE z?4O<|7p}0MwQ?IgVb>eZz6x)5>T_o`(tSRbW0um!cYBm$xkd#JP5u5ja_#AOxyxh^~PO(d=e1bPXzzXsO( z@@CK}rxS-Ve)PM0J6r|WdJ!jL<+X%qL5#$($@xyanz@NJQC}!#2|gCFlYsL!%HAY` z+g(KT+>=DyTCzj|*2@o4V0zyVJejW*MIsovb{n8gLhxUKN|jiEvUHN}L}z!mE9&v= z{*&23dm{OcZStmy!i)p_B>Hj}WPdynu{)YBotH&ISWUW+c~&|Cshp*CL|?^@_VGMSUzBQ6T2yE~vGlgqKDCkfTIVar8ZAZ@AaM$_64&R7e3VV&NQ} zX4`hjemq7oZ5zRfr3iKSt^dVf8$MYo64>AwCgR^PXpGN~Gkkk9+C8~YXG1hoVI)dx z?=&NyU^!nirUrbAf!#<|R%XI0LO|xvSHx@InZ#t2=RW;kjK-Pq#OkscaQqz8TFLF- z-^e1V$175U@u{&9x=!Wp(YME-S)#AP3FWHq1^>pH;+iO{>WgKBQd^9+BMnQoLzX_9 zneybxp+l-7i9596#Ga9Phd=)L2(McTMitAkda|n~tj!K5ES`Wv$CxzMXRTLE$MLaL z=zcQhvFXH}o;zd|>R_{5%>z&dF(ZBWlZP7rVf$DI@Ue(}Ao*muhRk5?Mr-$nNbSc_ zt{0?-oz6v<2h{Tx7#&f_|9U>?$r7-BY1O+iTcQFnRSk9s@vC+EPGGe=7;^&RKd z`UDNeZ5T}446~saOjbZK;tq}p z(4`fp{fYb^3eUYycYVWnF+D|Iu^R#6=Hju!VgZ{lu*bQks4Ij~BPa=oUqoZz!wVB!2OR7xr?fUT|rJW4`ykE$&8AEhI&$$~+ zVuLQdj^gbWRv%2)TFgOyE&$7(>el-kRVyk8d&b;1bmppsUk=b2kl1cxiHW)#4v##x z@SN}|1Nk4_N2gH~BKF?T@}Od7U+@}bYFr#e3;W*45i*DvcE(}NzZS)Nobm3Xel&%M zn&tK$^^cwJB=JHXrz$I7Ha^L7Lzumft7@Oh8v02dBC~cme(Uf6o@!Wjrm?}uTJ1M$ z9(jdI#?M4yaPnGjAl%8Hn6%3=>_rB?v_Y><6f$ta$F*ow{;WNA&>J!I@UQ2YLagQ$ zi9hR0$3?3a^a~LFin9A-$pw(Q!^qi4KIaDbD8L^Ve;)sK`^Mea47oTLOqhSUM)WIk zX=?j&A`mUPfZzCA{R+Q-VtqN`bPVV_5-b3s3UGp=eejSOzM)U{m{{Hue{w#p;}p*# zh7XS}2ooedli^aXEzpIw99i1x*$kPd-M!6DP5f5na1#75L32#sTK;m7dZ#p?Xt;12 zK$PjK%Re#{d60)0b=I5I&2%16;bTa7QftUMY z&+`Br%&%frY^)t0kCwU2Jirc}`u1{Z(4~&@HEa z5<;62$ogLp{z5vycAR=hml>)jTNig}o13;|M6j(IuFlSf(%SD`jRcB0dF;SRvzsS7 z!G&Fn{3!Fgln7|tJ^xOazIS8ykYq!A4YR;%*wOos*E>p29pWQ%lF{a)ak@pCQv29i zt0}xC2qgwN&&1US)%*BYd#?Vz`rt-n#^f~Xo8F}7LkH4GUpWE&WTF@OVq#v&AP-@= z`*VyB*n2>39fzG|AXd$2p(t~o3siF7FM2Rz-Cr7|vI=d=P{Kx+Zlpn`%)0O*VoBoM zk)*fbLgS#foZChNvD`Z~V<_AX_~RbpY*0qB-0>dB``nq!y{S^ZwS2oqXVoA=mSX)z z=NnzKY@D?R9lgbL1--M0n3aL1%?X=%ZDxh^%4ood*cl5Pc6d{D#Xmrq3$}HKa1bpO zkL@B?zKMxt`XX~|A}wN(AlX^lvd|)q&%P)tDE9~XNiXKlUGzP=rK_2Q^|%VwBMo?n zxck0n2|!EFN0i^3k0S&mU{|$U0Qo-{?APqCw$NU(!cRs0aK%x+t@SP?F!{g#CJy7W z5{|j*Ma$klQ7-RR*Oml7#}^z9SNG;hoQ5=p)RY#LI-q>+qm5~0Ty*QR5T5@wticUt znmh3p=;e>{isX_F6eU%Wo1*I-Qs&15bzNA22+FryjH{;zq9pJ~l)jYNLO_3N88QFs z>obiTHK07}eYia^6wKin)vIL^N688@5Y*i;j{L=Dw@5@cf@p3GbF#lq!$8Q&7=CpY zSXuK!X?+~z(xSt^6< zG$D>E=W1pL2*Vf}RkI*)>tFVGBg_^a7|Zxp{Yy*^JJ^>qZhx@`FmBmL0Qd^g_8luJ zsH46}dLa{tpz)3f&Q!wLJuh|8x8bbUcsK41wz67R7*h>2{OxyYnoS(Z%)1def!GYv zob=QpH?BR7z5$RR@4&YsV@>|#3%(j6cKqV2Q`@Q0lR;X@!0}t=k|-Gi;*n{5W6xI$ z7@&9sEEb)PV|@FeBo{gOrzgMT_2wqm9ON(zR6Uj$J~kGE-!W0YRH(PUHaHXsHM{17 zWqPwdodW#*NF-wwh3_W+1;7R8|M;4d3ZOTxZVH|t(wL$>V&4^z$u1nQi+P;4`>F1! z-)z$vQ!DnORF``M8(EW)10s`*AlPWBIXMqmx%F*4iET}9^l*veL+*-Ih)U(Dhn1F-Olao*^poxxK|=hq9Zl&;e*9eu?(Kkw4l zx)(DP!Tz@PV83mrzb9V+s5Ac&$z3c}2DYL}bNE(q9~Z;|N2og4XWiw+hCECUxzhFU zJ{dqg9ipGVC+{2&BGf-WK)16BkPxs?#ZoPEVLP?+v}-;NzTt@4ZX@h-8hMyz2aktg zk6)C<0TTk*jp|md`9hU4)?(u~93Lcsaz|BC&r&Rb?HDmlx{|*7a@a;q{+3>z%o5oK z*MDoyK9+N#5hGtKopC{WjXOkwH~qrY@MGpixGGNI+6dpfICB)A9dWpj3JrwMb7Jv%hpXQqxy2ZTI}U#-X;q`*;X56u?}-I0R`T&)<7)$KAwCzkUl*N5#lEo#XPBy<6sTtr)O((P`k+>4UEB@^ zc1$Z2U#RjRxBjFxv!yvTX*}rdM$PVajcuop4^(7p7j$%lwN`yf^rNE?WvfXBu8YKSI`H z#|798;Qcl$qiOEGIsFE`$IJh=2m)$s($z6l3MOpGLW!f-2s70~i-d5xGii^Dd`|Ds zkZBg0Z-Gp&Ay@bds@#s(M0$hj=h%*l3i6-yc>WgxGmU-+7?b>%KBH7bk5uyJMrmJ^ zk3hR#jsD2`o`TQ+;~l{<==4VfuPS)VF0-w6ogy@Vy0NerT|8 z^k1^OS-|!+@V`{I9rP^PPI>|5{e`}57qCr?WSaY}F;T1avyCf2a1vK*2ev{Jzfz2}OS5GjZ;QM`v97JlyEyvB z<0L~>R(c~_-8goQ+)r}H0N&gCDE`0?TBXSs7=-h78A4MFJT{Zhc8lXCJl6RE;vdDU zRs2O6R+`-M!0<@)Z z!Ock9EXURjvpIBOGuR)R?BM|>2RDEkxMzS=`gafHl+)#>BZ0*K(@UWNZNq~%>kr~G z`hE_016z_$Vx7nkr&qm|M-XYfEKBlF#PCgjSGA(Hd(#+v437T7fxYdtso`Mqt1{r` z2WQ!uT)`eq@zuYtNmg$W<~S%WJp05xGB)$H+@4rSF*LVQhO~v0!g1HY`A>3&Q)SG8 zqnaS=l1HFKy=#G?-$w5Hn+`@VsQj7+ISar24%V+CgEGv4I9mDLmD%7fJlOQPIBiQ0 z#1M77ue}8jk>-;wEU$hcD%J9yyk+x?uIg$0N!eew@cO;NuU`EiI1-*ys!Z1&d(B3i zlRrF9Ea5j*JsHu<3EH!E)dR`Zb&(64CWenaGePmpWYdeHIdmG5Gt)RN0&jCS+;@C& z(#oq6c5M=sa-wrIzjyyeC8X>E8-N?LmqlxAXGU(INIhc2sIIp)9aUFJPIof;!3)jOl_kQAQ@Sc9SMSYxv!Po0IAD^FCC6WceENk$u3TJiw2~nNHSr zU1_x2^2}RjvzFOr*eW<%8b!jK z|7fO!NSMpxRR0&S>9~8M#e6#%{8PQV{YME<)xu3rWj|@)eNX^6KCBl3R734`ALT)V zej`c;7Q-qK6^A6S89{uWy!|s)q{7Et9NLwDauW;?KP%v2DXbY4WV*wf-ub)co7Sce zgsu4h3YYvVX@LKw-sHOIA(6$YNF}OM3pAMfEjw}Vbz`e922N#LjAb5V0rax^PvDMH z_}Ses=WS}GJD9P`mQBs9Lg(Bg}f+FDw*;NAf zxQzn{K@+=0679p*=K-L_%An;!>|!wymg4-GR$F!W|3AR4f1Yi`CG3&~-%8S3n^=OW zbg@rtI_R^M>n<7RzV#~CURKnZ%^Kj}(STFo-}n!HsHD2tO&4|)v!-I2Q%JARC01!W z6B0&sO0QClQnyzC&MZ>4)4WxROtl~89SwOpq-bSz`P2hUh#P28om`dG8{aG zUp@PdH2#eVeJIG&Wu-i+{(~J08p`;0%|lM63mGTxyY}A3iUXs;k*HhX+NZ-IB!X=| zu=b`PrPr2u8&UA|KQ0F~OePm0Aw_GKI?+Y?*=Qkw^aO`g+0cs50wi}gXl^Oyi;;q- zD;Y+vZI*z$0&YZv*e}if+3z?5IXOe~{1rKX;Rk{QuAT%sSi<35#`yh|csR;CDM>s6 z6+I1!HxfTo@JUm!k#xh-Yh4^kyS66)2_BKpv zZd>+-Q&L=3n=d^rqA1`!A%7PCJV9?9DW(J^nPI1A?dk<&5dv3sjqh)dt1=a;_V8X= zk$p`w1+uVx3J3+9#~86exCXjoQ^YWNGmolg5?J+G(TyzI7wKE`XWL4DhdbEAdLyn_ zNZZ2BD>+i@B3V~>7aS4-OexJgsW?u}TI(hv5cNO17LFD7+m#-iqI74p=Uu+`pY@Rt z543@{ce$%BqjFh_S*yLh03gc+tOx`aq0m(~q7_&-rt9kUoOaz7u4f;)hyJQyv1OxKQUMh5T>DGZ;9aH1-&IXB z5DUF&a*vq(3tPMGKs+hCe^l%KyG>kTn3KV#6rpbZ-F;mcWnpL8S;<_w%VY#F(b(9x zizuZ|$9^C;3_GsD&hR?x{CG(AGtHqbrGCYI=~rPMz}d{grc@Tm^i3@s=6!2d{djr> zAP@=dhzI|1uvt!c#>Ev#ct7MZYRku(@yL@PA?51=W=&tSgtf|JU0*}%>}j#vX)zth z5zt=1mdUn}If+snn8;YNBb?<0k5j8bie^2(q;j_tKwv4Y`rh^M$5aV@iSYZQn#6OK z;=F(;%@=GD1+-v1k2`dez{%OWPLf^ClOk?Jec4yAMqU?6ZhR6SYwY& z)=gI8t8rNI0>vAaT<~?Lym~lrC^4n=!eii6QiByHI5;7(J~>>`&S2v)PhLS{o6e#X z&t^(ShmbY{2gET|Z?E0a`i~D%i@`R~z??=<(eiscfsTwz5)CYkq#i*y`K)g+wVR11 zneAH6|6Emsc8Y#xyD$ey9laSD-}Jqc??i;E=sw1|75*5$%_E}725v{K<-HIh!)&ue z-NL~PSMzklEar@(s8-!*xL=>eFXP90j*I4yQtbwui)?dzkP%0JGSjzK99@SByZnv} zC9HD8eenE%r|^iI`yO-J8H6=bJ?$GPOj#iEkRI!9$tZ&WIpE2fVnSOt_0f4Ge;`A> z^=5;8TI1IJqv(sutZ+7x5BJu|dv>1zhO4q202NLl^N*<#hy$k8#gih@0?{>3iySy4 zQDh1lOs~^M-a_HW zA9ON-b8lk>b0h}0z7@k+ok$I2{2|~hAy0qfZ0)_IdMz`{m&sw_6GYGec}{)?HvXBbj}uE#A(^^dpjsP z#OXiAN6Q4rRlLQ=fY^C_U1;%(s`ub%(gz=6q}t{~r{-!R_r%MPraruqwwidg+U28R ziJWxP>(RnA5>VaM?OaibLwf`D86%OY1YCz+*#6R{9;iwdU8{?fql9sBK6d);jjC6Kx70HZ+#q=%<@@uE)?v`{}LUqut>-W zRZ8l^=MOUFa5CCo&D-fep=~^-E@vM7YaSIaUnOmAZsvPE@< z8bE2;TXa(pKQ>prHYBp|f(91T)1JNLp8jl){joRkI-o^(-ADS1v7zlNBUr03*12KT%Vx*|C_gw(GVEGcXJVgU z&kovelcD%IfAEi^6@(?YpXU2t&A0rk<@VQS2R)mzx(~)?e_L-0fv{dNnpnogPy(s@F}@cnz0O!u$QhD70( zzbhVAL3Z^&iwjDS`~`aowa|2}v*T>UkT-}eFI`gEUhZSk{A>cz>?6$OJOkG)6Y-;? zk#1VE`v+kgRjME=uj-_;LNoXV(@{g@>C#sqRB(WWS4740Zq7!m>GtM3n4&s&AP7UzqYxq6MA;1J<1Gv=?3VGE%%fsp4y~D z5CqRYe9ZY)yXDQ&5i_Lr#^Cam#mAc52?=S)7;*D@*+us)3TzN@PttF~*J?pyEP|TE zh(q-H`28n!nlz4JoTtA(WS#=co6bse-x~19-k_8m=X8+Js7wE~`dVJw=Db(k!x=7N zOeZMSVQZW5XYh1@7`6+dd;>KmGLAckA->rqGo?$s)%@HwYr+^Xi8C{ z5kn1tPFj@Pec^>Nv46QMYxp&))C9cTTPC^9*#(fMlWz&UFD78NYnn6sW5IRagJZV* z+i4~R`U5uro8 zzb@Xx#sJ`kWxtw&=HR`PT&q?3hU7MAQ&&)c=w77G6=t?Vs>4bR#J! z-L0h5Ff@WPNDoLT-AW1!Fr*+7l2S9Iv^0p47Lr4EN(|kd@9pmSz56`7e*lMbzV|g( ze&VuI(V-(=(XEMcpT_O2oxp3ekrR|pR-b)Jl!5bdD=)hIeoiMx3aER9$972%w~zo# zzQzV8$}P7_VSupf{?Zp9Bj1t+BdeI6ftErn=ic*>232LTWc#m%+jx%V@k(e%Up6dc z`XXo{e%<1E{>tM?6Un!j6PUTh8Q0gLO#`hI;;dCEZiE0UXSN~kHnHAj_Vw~;69d#Q zB*fb&3XfOjOGXbQ!7mp_ZarpFxgGoNsi0{^uuA|dJ$gMX^vvEh+gT`tNMrerhddTvjfIC>Al&ZxHd zHTk%uJBxMD?c26GFliJj-kxJ5-s~-a&$==&+xjTbqiJ0zUY?*MdiwIEE-$^qctEW! z=XKUVK#Ij%qMu}!#HRxzkZ9!3J$R~_*~7pBHsY^XN3#Rds7WQ8BYBOZ=5lQQj-?`Q zMX#dcy=j=TIb19pja^7OL%4|^mKP<*ll;8-A1r|T?^f)a?e1-Bd%{#u_oZ+RX%~TydfdKM91Ol?4buGQ+Xn0z zK0MldSE+;t=wJD1{%tkvUOJW(kK$ZqrD5V`aW)VWW}tfx}ml_~ zgyy5*CBvEL8KEtw_FFcHQ3JoEu2asTGaJ24D!eeX4@DiQHzdR$gx#0N@dUQUh9~jU zZ*oWy&wmC=Q3t_wOP)c=vy(bcinE>4lL~pOX>M7M72KS09?73p`!rVohDtmNRHSj;oc^=#5p;ZqYU(g+@w;`~5p&D7D? zBRuVTQEk?-bbG#;MDMhojcT#GbwUo0h11`9?|k3|2?us(#Nog9zDMpMVMTHZBUdjk zr0{+Y|sTU6{4 zawDBa8vA&58p)9tlNi&QcS0sP{TiViu+JDOh+6yiLyV$ja{Fb0WY62tuR9xM!;GEx zRRrA|;1)eYij7Xf{9k_2n+*V31J7nUkr5cLjFo6!(+lEm3J(Rdn;$Eb!DlMA_eIhh zZ>6RO_N?OzKjdo|$11*BfL=`AdUXYxtT}fng{`r|6wPf2S`Pc=)MONzZVu7616!3X z>H)ELmC`IcpRSxfcT4D@h#c4Ooqww0GZj5|xt8CpfeyOh4?O)gFN~zdJpK$wqz#7- ze;SM$%wL}v)+yWA7&W#m{Ka6|yy9Ciup6a#9jzm~Uhvnr2<7ke;rKWSeHAX!c6IRA z_uFm_Y3%tDtK)Cy%0a*x#WGoS+2uj&*mKXmzhuScK9dovi?w-=GXEU#3g5{n0UGE( zGJ0!pyE`9E!Nojt%UXk4isenKg|}H?6~m$%`)h983!SUa#Yy6Yf?MX4hdL5|;-(##2wwkn9}*ME-Xx zpBGQ(&LCifs?&%Hh6HYIH#(lhO4Fk^W5mmVZGDqgQ)~ARY!lX@X(d(grmN@0%{jI* zYw4)h@>WYws72!6=wCO=5@2xLiF~_wFAckNZXVn(<8^?~FtC{?mSd&JOfCm&W+BrL-D$> znsn{P`r1%{X^<+tAgJ=PwwJR>b^=0^($HU4Jsj8aa z@fCL{ZK$s!Z*o?8*B{el@0 zl8evqyhb+?EH>oodf!xp}jo0XNIMN4vLIZ^!1njs5HBPS`xH{bV*B zb0bU-j*H@1VV)vQHY;OkZr@@fU8?b28)tpI0K-4JHf&8WU_qdVvqoaJKk9TTJ&#DD zH*=}(z81IZ&8ZkOi{Cl#R5aaTet@(HJbIR4yU_mSjRU`2VB<{K1>Ka$A3JBxs>lK@ zLf0wVZp~B>$^KBy(~=N9+~WZx1$E^0>sJ}K+RWiQA+WV3jo3fUfBkZ++RgN0FB_mA z7@!g)w1r4ifA~JAMFSF0^HC1XoBZv~&CG6wX8}em@v;fFLpjKQ%6JbQXk@u|Zek(F z-cb4LlxojONx5PD`gGpXMkXR%$N#B&RR;l`p(GRah;^I5<=TBFt2QT$_%Dg@7vz!$ z%%giNvFdRVCevVD2T)e0awuXh(Sm5;Z#DN9*A)B1_q#X`sOt_>w`$nJHEyHGsqVd2 zL?H|78j*3nBL0voPYRdEX4{ASXW`v1>WBA;ahzU!0*kR0*rZr+%AC=XHvb+=i-Zvs zCj*XDLU7r!tY1NSW)>KkvjXhBp5KEv$mEnmJI8foQ~ha_kj%%-P|Jr7Yz>DXcVTw{ zmFV%MgUj%6+BW1 zypXU3`WUr@x*q9fOOc7P#6n$(a}wNR&C6pcvJP3lj4iIJrIMu?sDxbr;!1@z@RvZT z5O~T)e0n~lE#Wi7CA;yHCO?5sf(Bjab&E?m&@EW`{CE%u3UxErdO90J$Z%B1VD9C% zMs3@L6Sw{r;MH9;f4$?Y#yVpM%s3VpV z2l3xXtMF5Va&~w~Tt-|z#UJ~@_Lb1-mD!DGIMY=xnP`9U3($~zgS^2D&7XlD=?w<_ z;LD1z*7urk0Z6@g(FobU!Dcv* z$$iT~W!!>dMbFt38>oJ;6$4W^h=9M3Z{<6Q`Ef-m8$Ppj8zp;sieXqgekH3jQF0i} z-`w=3@Dz3bD5d3RMx%qtuEHM$Bc2xm0ep333_;zkomTt}s5(&-v4iX4tYYE%PE%*W zaE_+HzNG-!OhIEeT3{9FseEctVb-LGKcx11CWs!=S>Ez27?qUXX}93i;>EW4W2((| z&M5d|5{Zx=XnmQIu+pjxG8`WN(;t{b3T0n8GGTT&iGgf(f=QVYUlABNr$d8cVli(2 zVDK?|exl8L%G~*m0~|2}Q$0c|_uOiTT^q+OB!Go@qvY6f*U|Sd${`u+&Y=lH)^gCj z*;hZvr3&nc-bxH+y2rW^PtlovbqwO6cX;2AK_0QTLueNxjd!mi1ve#o(b?4ER6 zu5Bb)E`x=C$q*sBP|~(L8T`u7g|3SyQ7ouOsP&OTP$%=zYYa-w?qlnejkn`;<5U!; z!5^@Aey0}(Y^TqQ5Vw=XWqU<=#gsgQest zsGqCgg#3v;;>+S~cwTp@(sP}#znAo>D1g?a7)Z4v$)8iizl!xo{B1d{vUlTbV zirW-Kntjr|rVR)YVtBQ)?UTDn-d$T%Ha(XSZBZAV*SO*w zVMy#N9_O1!hw%*2cR5v`+z}Rx?_%(euSa38>7jhr_ky5FpWyF z2KP&g4m;;x-O50J|2AruqX8cPYp-}6;u zh)5s(9K+yf@a?7L#mnz_8B1Z52Uxh;>JczC(@$zxBKGkEv2ydd`1B@70PlU?#}~cd zq!5;hVhtPtd*5wcu4($Bky3crb+$j^JZdOt4_kM9UDA%-ZAr7Z#44^&EBFuOIe&f} zmTKAh$D#DAM1k{mz+a)-fh|&DgDlC>Rd7w*m*jb0znhA~^QCj_aBe5F@hDvy*L`4F zUQu`USK@TqeK-vuD!2z;ODY^~l>JIK7=P~hv`_&(dfY3%A&)De9~}U4u_Rk|?A+F8 z3}i$66`A^$8Yt&>XGOP=0GK@V$TcIAx1QNX>hMb1@cT55A#IdjpBNIOzSawlcpto= zr}mv&ds1)`36n3f>j~=a3U9{8oSJp`yXA)s{4wik%E|MQCtfpkf9eeFIK_rWvViij zw#R}VoEJ{C2xTP-LcrtM#-$_wl@j>a0W^jCrjD!$RHjjM;1jH60kaytH=}|5e62b5 zg3rBs{AJsCTP?W9|EVWEp5;UH+?(7$(k?y{r8s=DH*R`Cg7`vArDV_@F4D(!Ah zg*BbA^q=P>b9qOs~Za2!`OL9`X4)a#*EQ4_TfBo?14+B54rui z%M@O4#Ppqu^gEl>=%H(bnI##^^Y%~rIx^g))fNf?L%ia4T=iXWv#f_kO#@H3wNHm~ z8?)QLe}qMgdd~m$)8|Gi)9X%t8iAIC*bPGiluJy`o9%`*_%XeXY(mel>*T_IA2-DD z?d=-oC*Ft;7g>1vKi%84rO4}~e&i=#>=NU^cGcQ_TBq^2GY$!pmu74nOgE3(DU6K_ z<}el^VmNWNeZ1bT7{*cdgpTQvYAr~9Pg{jS&@4B#@p7=~G1ZvBFi%63i&bF%_1c06 z;nu(b(&!Z<4t%Oe9_NHAqu{9UvhE5KYCD(nbq!@(C8ljcxyHyROO)MBpc#_>QSr^P zDuVUc+ckjPTS%A7rw$pi#?we3kMjyHD6iN`m1+7)uwlGljC_MN*FxVun*D*w#k;QL zMEx*TFV>!pqPuVYglLd*Y|89o$Q}z>CQ81yna>HRO;tHX9|pTl-<=`8=Eq*Lw7d%{ zZ|SW?9eLjOQ_4-XPsb-czK}mnriS{pD*B{19Qc^jNQB^vtI~Ktk_jR3b|@&~{fiD> za-Z_rA`h&7#MslUBm+nK??qV@XFTb~6i7`h{Rn9fgXxL#Vp0}qq4pG;S4g6hG#aKmf zQ)0ErlbQJb?V$dB%73%f*8&3Byk!=_n*1IE=P%P_k>ofdloa=I+Gb;I948wDm-bw5RwH1kt>qZFl!EU~qhSy^zO2!L!s@Mq&j(&> z!qJjZoz<|Z8JZa^WUxuQttrui|66BJyq%HJzI#i#>ikrV=4a(JNX%{k_LWya7-O&`%q!E?{H%YTJyBeDj4Kdv8#t&Yx08HvSt3z#v===Uv zN8+Dw?9VE)MD8%ub>40lzMnI3r00B-mDP@C1&kkKAA(Dc_J9t`F$~_>9O*7iY|tYS zWJp%cjl=JeKqx~?dB0!|m5#ZuSFa@MrL2Kf2S;n@S!;?K{f;ov2(fsnM>`TYlXUyl z)_J{}B?+%3GrE;2rb+EN!aF&>>A?wV{m$0h@GWJKCX*11E>A>9W%*b1K+*zEOsHh; zCmmBO-`hsBuc?ngzRo+dE(UpGe*irC8sjf-0o20Y;Y{m`5K)D+k8Qukn$LLh^qoZ( z{Z&v))d~Q^SZd?j3qL?NsIMjrGJafL<|Jy0)Ld5Ue@oI4HQHE*eOO+btV)5SJvOP& zXR;9VV=M1ttOQ`f>j^g%6U8%NFT|64QsdQK!@l=;b6ViTxWE!`ZNHX#FGLOiH^u_W z7$T-F)xRA2Ww7r9B>svP@BC6MPkj58d72krf?KdBG)S!0==O{I)tjk>PQ}l?jn6V2 z2wm^2RQ)So|MkPleu7(FUUYI3R~n)npj2rmr7wI@71}0mNo)sS@mG}wCbb3!dL%`% zS;)YHe6I01(ot0g2Szq+ql(Js=%`p#H1!jWPz2GH%8Pdv3@DIXMs*l4sU1!|CyWvP z78A^t4x0MJulunkKw%;2=z|=m07Gy5cGwU~vFE%MRk0Wz&$pwQIRmCllc0#{$dbY)So5?fA@wQ zRGpfZrBhXMcl&`2;in_h@{ZQgGK1aNrWUfmhd9-KUtl@&bhVraAPs|^>iXv*X7&)Z=zW#97q&jcJgKqE@%n<~Q5Zt5nK3X9kR=-~`-tL;S z_4R);*#Ca{0d}T^T^$dev8X$ovZ|4vr5rR;W@6T>C%YHC3o2Sb-6S^~AR7A`wsf#2 z`o%)GK-rJ&J7!)I3+ON#^IeOxNrz=a;|QoK;QoGUz-%>lv_!F z(RdVoKvv~Ya!~N?(fGN_ulTeBJ#kTrp9pu>BJBPkq(!^2c5Z6zaQmm?j(ch<(wIvw z{@4aMIO|0nhr>#q2$aoamWvwbjHLQsn{7g7k~}cP&di&yf4Gc~u!Mx}Ok`2%Mj?kD z;0kh=5f0?}k4n||yQ`weYl0uBF7-@huAE0h5!*#$Q^SWO;)%d}f0$R7d$lD6BI&w# z;rD?oo66rLa5K@_=p=THEQCojbAVv%OsvtOWs5rGt8jeS_afSfL0#CU(29mp=70#h z1leT@ppNDLgLh{*)RH6KHowZ8M8r>QxB$j&4Tzhx%!_QdZ|C3tzaIL5KGOqCes;;+ zxB0&6-9oaHvr9)m?{Wf(yLDpU%NIm3Kkr4UiUXG04k;~3K0-10WX*fg*vKP6dbOh! z;w+CgBlH;7RO%)@Y?cV2yizwOPuVg{qP%&+_Z(_lKQ+413NLhGkwwDPqpq23?05VUbB_dGe!ZriYe2 z=eRj5A=H2Bll<;|U1u(Y@=4LFo75JY`RIxjYAD=dPEFArHzy&dm4K>*ESnZUSJgWZ zeLoeO=n7$KAOxo-NaeZ;gYrW3L^1`F2W5M-HYE#rD~w6k*7x;u_eNEJWk06Nc`3NqnlC4NqV?<@a~VxE++SP2b+Sx z1z9oN!!(^KYB&ET7aNg0^uD7g`Sa0pecSFg2(Q>Tr!2FO#?zlC856rqINtJ%3P3uk zhjBEJoz~7wk-X5SkoQNxdeC9v+c;=EJ~-TeOvbl*tDTmM+%+T2zb1`S6p=Ie4RuXx zUNrS-%rh%c7a>`{n*u6D&5OuAvor*~)1G5r&;Y55sPKjh)0S?95T6h~m4_N>cnL7Is60`>uQHVR!BU9T86Mq?J(g+wL{!E+1=$(W2^sOG;_8HSN?OBb^OD>UTf6v*lC?M zJ@`#rGH+%NIQ*}Qn5gp&(bZpwLV{Ke>ZR1^D~f#?M!BH|Jl*V9OM+@5|Ji2B8sOTV z=)Wy(yWflpJhq)?%7RLF>3WVB*dPJY65?-!#@G7(KJj_`$IC+-FTYUxvCIh+h?fNC zdz0Cu{Y!oAqP6J_e;o(pY*KZYofUk$xpk{cXb@tHU+zuymHv$O#Ih_)=8|*MWZE43 z3+P7d?3QZ`r*I)~AL=S(^7U{)KE8LtYqiFvg|EE%ybY=P9zdY7I@042UGv)qOsXUZ zLEwrqx*?%%n7J&c%;|tT0TGiBq8vH}IHPTZBCHSB>1eV&aFpawAA8y5Pm{U2%6Xx& z=t-O-yP)kLPF6yxat&a54>{nBlCpA|&{TY~Wlt7H92_E(#a1CGP5+22Z}hZwgKlY7 z=1$VI|Dt}Ws=0|1CgWZn=j_{}WHj$#SEy5-sOss=7;edObJ1OcLz_=HVyU1~MxPqJ z$p4C0amSqApp7SE0=5-z34}l?q@c(S=9@-3vSehkH-x59g*^f*jL3fWGFxY?@z?E$X+3)#E`WOD{TvEHl0GSu8iZAsfvs^PC{jJI=p zwsi{rh&At3EuQY2d6A3d{~4ZuorOJl@+53_@?zF^M+THa^2*uOuO^x6&Cgp*%WN+? z0Q}w|z5GEjI?UFBie1bM$NA`~%V0HKLT!!T%WD&;_VmcmhvQF(FEzT~4gxufBqQ-b zssv@((#4;{enr9ZJ51odGP)m0K6PFd(rf-ctgcBe-%@f7P&l1o8fWrSXv zGF=vt6C^l={)$~n_ScLDh=EPvQH$TJ=fD-$fAZ%hX3U=FR>GI+$eVAnbh68Twf z(*f8+NMGuZ;qc+{gj4+pg)&{3*z-p})jhLGNgS3^uZi$VrJv6XQv{cC;qs0*7!qo4 z*U_|yy#SNnf0O`_=}opnUZl$lYWEI3b6}9jd{?M!Mc$}#6*0~Jb^MwgDVA19|DCr$ z(Mp6HxnY-SCo}t>fp7ImLf1~QFZrT7Q?7(S4@5#t@OU5*xVFV~w}h~e8kxdD9QpWs zTwi|P0=V&Jw$Mv8XaeuuPqsQ8vIb~bO?&(;8<8o3L>B;~j+m0r6b3PZIS6`&55m9p! zeUf%9iVetpDpdR<&@?3QgE8URkAP-Ig@^OeA#x+B)z|Wpjq2Zd8)ECR`2kN|bfPMb zVkk=_i*s!RAQW_<+n*uaP54wTuvNfC2Hodz|O#e6xlsyS%yl)j80_3v>oa zLD?1NE;P{foIrt1zVN&Bh#|P>7+gn8k_(rh%Vwxl85C8dnp3A%Q=n?ZVT!si)fLU zs6T{p=HTL~Q}jXqf4+Q9fFsmialf~RgK@U!X4bFa!ku70fa`8Sn3U0;Wn=vuy~cco z`us#{H9ob;Bo!*?mpfG;9K>~`REBwQH~ruRb3rZGXNHi;N)?)#+I5d?IiNyMw^*mb z2UyvlNW;GRUrMAM0$GQ`s;XbYYUog%ow0FA5dWQS+i-)1mSbd4Mw0OuL61Ydide9! z(S9H)ViPz`(SL5Fts4zssYwHJNpPGJ)tC(C2e@mxIjfUy>ARg!6Jq2+q1yCYh4t&6+N7W`zun?IjPlY$sgQ7(4IM}lIEhHhy zQ|;S9rihl~HQ?+mT963|ls(>>thMd0KxbMJ4?ZI*`Ywie*p?=q)x1Z{KWe{@TEW55 zW&fH|xXPcqcK@=~%q^F^Ve(U62k`gkGpoe|f&Z+e|C*NKF=+a=fxoKn81hUJmyehw zSIJv~C{51G+^Z7TSaV&v6~knS=?vvz5;7?~QcnNinJ5Gwgghh42}=f=Zrt~&EI;bQ zr&4xRzzstbeTdpAVm77rFzH&n>~9VHxAza*?za;uJuxvNgJb9srQDn3A+qcM$}zt_ zo?|9wJI8yO)N{J@D+_TYVPgkBIxT-ZQr3|s0iMQo^_z9mx;g#-1UH>mVYX)G6 zaxM7T5M1o`H9d9szZ#YrCcav<7GcR-g5_P~pv9OrU*AvKxO^0ma3wR?g9j|y*`@7nAS1$B>zd77;ZHbSlNm`*wBV~MN8y25=NG_e^#TT%l!J=%*?-X zTZwg<)cDta|MfMZH+K=OJ^AxWu`fgqk3O=HUwE(dg|S;wPh)o&r85Wy`7z-3gNDErRnwS7uev?Xv5{UNcOHN7z^}S#H)Q8`dSmtFlw0j=8GR;Z{ z$-C|?u^KlwK65Paetu$W{2%T3zscTb90>97vgbGI&JDXM&((JT%0EL^RKF+|1Bs|>b zH(ytfZaYO_Pfeaokbk>X#jX}i$SLibKht4l4EQeqdh2~!E|N7TZ1>mSA-E6 z#E7JQod3yWDwP;eUi2o}c5om^(tjsG`tENW z^(K!F6I!RomJSc}F6zNWGnJ!7$9_+wZn6z8s76cbR8~1^a&(`mP;EZp9^-fZhMrfO=iH+-=j5nO zr3ryHAzLke_{SD_z5Og#F+T$->Bz)?)r8TVmpS_xKx7#e_f7#DmKw;5FZ>-F>a9*w z6DNV_nta?736{jO>pYpiUrgWjM*&^8YtqWRL->%2=`et6`r~db>U7SY?_d_%SiO*G zj~R95zOv8-7pGrKO+MZQwgN)McLa&;m2#@GvJLZMy&Z9c-NI5mXr|dfcnX5WI(yM- zo3r1f1HWJIGwLpN>SH>>HgVB)<_UU1soU0&IItjIQ1sKLicSp^Hv%7~2Y%c6o&DK< zg_o}QhNlYWHupf1UTXt?w{>rV?FGq)9yiwLZT(kuj+O-aqRHRHIHdD~LZVWVxL4~W zkZ>>`rLGeAiL8kuzO;AM1WDxj0*dba@lnr#X$TbljL$&jLbWrQ?S5F}`6}DR^iRZZ za7OM5vP(SU!TT5iCAWY(pfOCOJ;<^*b{2I zm)R#H{S{sj_l%exY)MX?(D#nH-T?_Y%ch0UZSWJWf4^qKFFLk}Y?p?u!YLj_&Wi~r z2zqSPhk=|>Nhu`&dqdtLzxK-hJOef*RQYKbdJkmVQ9wlo-&R=JQk648&rtLwet_Ho zQMVVx`=4+H$+-fw%_2bHEXyv_wZVOhmm7N~O)vDJy2%yKZB6l0>&1p(25|aLeeJ<0 zCm)pX*l^`JdzK0kHEO@s0*ufd^e^)VG?wRcKmUZtFt4Apc|Nx3+`m`)f`FEg^ynAW zyIa8!gJIW@LTDj@Yzx2&_CpbcxnH~6c*KS-H_uD{7%e`p$IdiMxxyx$7-O#MYM*Zx z25c9Z?|)NHzqSczzR7%I$_W?q!HCau)LCjkt3Ur$Awt%GrKj>r8gEp}@&h-Z#VH|6 z5wQEsDxt4AKw1GF+FJKv_N@{k{8nG7{|O-ZA;gU8k?Fj1w-HC`^~cd0*}{3wo;C!a~CM>616x zZ6t=^+}EXY;IO+QU|VjCNvVg?AIWi16V34Hla>9>Wte2J0f94fHCZgA z2binGZ)e-s6%25SA19gq)(nZKa$?KT>>pO@&e)&OMC+U#UDRTRIHl(_?`_qP3V&Hn ze67$uW2y*HZzxR>`@aO!ADg)~anAy-_QGhbbIBW-G~plUdwbED4xr9yL zIY+j~e-%(7ewlDJo0fsgL}rtnqvXtWm+Ly0JI9NPMOdNJ+GNtW=5smpV``&Cpy{MM zM<&H8heuul(Yp4o9E+t=cd28xX-+Sbnf<$*y~lG~_A)}CHRA9Zq5C;Hwk3t_&|VgC zT(n-$JFuc_K*4fesv;rVk;O|}6uXjZW!4Z&*G)`F%3VYTV{ww8eP;~^l>#s;Ly!|< z4Lj$))!s}H2C6d~+uA*ozc8cUyou?wLBR**IC#mCdyaO;1z1eL<9=gcp*5v(EUtm- z8=HH-vy_WA!LT~IpDSJj=HuxeUkK(U9qZ%gSojmaWzsyCbNq1&P!`ne9ndrVV8x2S zR)BmYp^Eb^4M3G_%&Upxm_GkqLFaHEaIN>6ax5||ZvvpILx5-V-cvKY;>W+qP7^=g z0*>%Q<)os|-;2Aq1s&~#QLbVVYih!kAH4-Dywxeo$-pm{U&@DYfcT6`VXN^I8ZPl? zMNo--B78?{&^ufLy*5^2D$<0%J8~PkSe*v%l)tG=Yer0aP~JHh(VtDbiWOng8(`QvNK!|pCA16*;MR}5_!ViWD#W}P>&CTS_hyw36C$-d@Y zr{Vn<%Zh$O}{y^^>7Ziex|$cve4LPO-NdTAhpjz5KMIi5797EY^k4S|{l`BX*5HOFrT4X{?*|+@t_1pD0HW zDl_Gp0aviPmmXt5zr%aAOIyx2He_%wuk$2hWJXnk2$Wc#74Vo1FT5!bR>`?3L;bzz z{uAKDP;>Va$U`*5Yuf$vICx7m_%Y|rQBk+M&wQXS6M(mr7UtMn zuf;&}Q3`D`kL!&qEsg*OxWNUPRflv+66pf6=OpD%P*2dXY%S07Iz4twVvLBEXxqN} z6wRHpf?TpZec0+9Z2}uFLmuS&-RcsQ;*TLJWv8?7cA-F&zCh-CUQVi~S27NnVBM*N ztGD8o4g)2%x36cPHV%7C6%sJzg_yv37&SxJsgqZO73G(@I80h?;+5cjk4!MGjXU)Z zN8C!>DYZ^Y!q(#vCpLp|I3{l*_F`-of+{vwg4-;P(n3LEkkd|Mm#tT1ct(E|ni`lK z9uz~d1&fw$Hx6hrMT>J-466LR_Wg{MEw3iCIVC2ffo+3dBW*O(AJ8`EIa_ckwOg+h$;>NbBPJA9^VgW_t=vcejQuat zR-qhz>%%LdC>DZ~b71?DBU^Jayw{Wv21D70&24)4487ueu2_Q2@2vEiPUpYH4km3a z`9t`bO-`aKC^ypqmgJ?z$!k%B%Juw`zL20^+}&#nI%nh5g-ph6$vq1q=tIp9R*HHz zIx5Vm3cjg#g5!?Xap>|YUoWkNYl{M&@LqaqVF1-av<1M|3Ez&{E*qRT{iF9Z05am; z3choGyslrjYUbN{@!T{SzmOsR?kmFm9XI-Hx1tQWs(t&@|GGlr zyd`+){`+vdiH)1EhtZ^b^zfbu)jd5BlCC_>hfJaz;&&fbg<4FLgq%i$C0R0+=z1J+ z?m9YmNMp|O1ObWq=}@}jY}HfwU}v5BqscALgNd$9-l*&^B?zPp`GH3-v-0sJ9!AQbj$ zpAi`T?H`37UXRYh(Pgd+IIT^EqP=$#j_GANEWdmzxAJ==jRU*7d@cnBD^r<2t{M^; z=JI$q&$&HlRSsZ*_vpi-_^YB&=wF9JkPKXUdatfYNWHrJUi78L>dK^N9qbT8gsii2 zyZZox^9Ez_$PhzT(s=*S0HvKeT+EKANhXzFDjfv;(=&6QshN5IDB_-fcL zqwB~m5t<=RWmlGKJu;fSt>xIH;Si0Y<9@9%3}U&EYLHbd*e`Ac#cpx-bLsPdd#du5 zdSIgvjsa)uhw)q(un`lY#}jbS{eA#vgHD+h&Lnt-*Fj8Ig4|Cg9rqSGQfjllbS-x! zIVwfcg2h4)Jny;I-I?{oD5$xs{T})AL3l4(6wmFE)nI25&oi`bD*soVoBj2*$^^=4 zePJl3dyV>tV-P?(`eQ*1`zF)c*S-P@4a5rIt zB-m#BMU22VKFNV?390~b3)Gz^aV2?OeBxB?U=|=}iK0lwAdUZyiwkM^q(5a^bO}fG zIalL1=185GBzXZgO{hElK-n+YAf%GXPbT~Lt+>Ga9lvYOhGbC5L4YzgK>nwgY35*D zU!pf{_p=_~ExW_SyU3bQs6Pb}-@=Iln^7q8kJ*{gD_9TQ*|Fp~!Fvji+>q^=a{~vE z0bw&x(uYduxF`b0X_kjQYr)@%h?LsVz>z5PT3#B5lBjs;!~tmxTsC4t8fg*l*0of` zoGtjcYIC$50`TM`6Gmh+$s>Z-P@Gc7D8n!thpyIyuotM7_qO#e5uvKphF{|Jq|{v) zr{~VIY}D$G zCiwTg8^9njrxCsj0PJe+*ukhXEm8#~HCgE8O?i5_1@w&oY2U>0as;rR^*}>}UGIF! z6@RUF$nbIOoAlG51stX?qej2pn7Ow|{u`i2eHs^N8pmEyjFvF0aU;pm)yLy)Hz}x~ z3&)qUEyus_)%^kZesez}2ZIQilTTi(W5DqMtVKw75Pi z{u?37horj3-tWh4)13{tI6}g_5S7_f^JTU=#k;kE8`*t4fxn^1E#g0O166&cTN4!_ zR1it)E(Mf9Z98vjVVSLBgy+i~JN%HkeD)mVbiUOunku!>0FJUE%lKyM<=XDLb%?&} zGv?Qp)OCUv*%_D(D0Zh$;Ev4Mtr1kOr+vHPi7=^d*Mv+C&tCMjsgiCrWqeG725vJS zf&`lt+Wz;n=}GCUSdW3bgLH`-6h@;$kSI>C;fac z{e%;GT>&^q!)Jh8HTLi;I`S1CA=zLcWyFqyU1GyG!tV~Vt5C@U!~yi{UQ^bgp%-R6 z;L9;t*$O^|GwW%u<@BG>cWZOKhvPSTcF=Xh!PuZw#mFhVMm(N}Y*2AiJOMViFdv^% zCw->c6f9jVf`HHA_0Oso5*|OJQvL7?XC5v66K>%1Hw&KiUY}FE8Kgv?t{?SxLX4xHo&e94XqoaYg6j1zn`p-1tcyl~n zYUY&+lbDU>C@bPc!vLUyqVmrc@OA1K;Jfe`qp?y)9;Mk9xUv&Q@^8m1+a8H?H^wP+_!EHc zM`3WNa1=dKKhT++zK{R@PQATI9fwHNKq;3XaqtHu^gDnjiATx1woKY+t}<@~u*fL5 zK2toN;0I zz8-i_EYZH^M`{`BPDX#$o#HWk%>mcqiz<=)Lzgs414_x6MR-mq1;QdzdCS7Rj} zyb9)j${=@-pXUcJdO!nM-2KzpiCsPj?f8YYF$ZSax|LKjqwJ*PiMAUxKo4R!<~{G` z2B0tXu(&kQc$zi>MmB1ac6Iw8zjw*gf9pQmC`f-=P;$|T(GDJXoTpZM6>{J1S=Dpu zb`eTYV{_wG^9Z!lfsO2n{W4nG#Lf1d=3b`cfO`YP8?#aMFZ~R;^C765#fleux*-Qg zF<_$IbKNM^8xgXUvzmUa6CUF4>t|;xqxvn|;Zv=t$2R9l6U1;>D(a}#JZ*95A(Np> zi?z>>&(fs?x_wS%6TTX$^_-vWz%PzbZ3Mc;^Fg8&ieA?o)?B{1AyyK13vhFd+UbdR zqPJLzf1R3yv{zYjS%cQ~yYb=3C!s!?x@glWL(xnRf`(De+XaoMIjEaQl zBl|HXU_AwYkZG%m;*LVew-yEd1o!(3!w)sK-v!N6-P?G5eA=hp(rk&sG2C^p^xhC9 z$>G}Tx5mSAs=Xa$YX_5R8pXelc`poRWHjt#32@n`Z+h;oaB^4pKC%X&dZGJ)+>zIn z6G5M6{JAO8E<0dHYGEne9+k@(FQ2#r(1wa-Z?3S4!|_2!`5PFC30=6P|333SFX4kO z-o}~njLF(rV%~wLWIa(1wS7|g)z+O22o^j4`jhi5Le>F8V{^kS^ZF;7lX`SYrrt*r zN9%Q64bw*1F?5-KD$>l=&BTt@XEWz^r_)l+++_TL<#C>;g>}=4Lt%>XrzS$OoTk-` zm@!jlVN)RWO>M9xdZ!-qM>->5M3M@SqU-2j9UQ%ErQd~&)&tt4Wzf=#17*Vh!2;Mo z*>RR+g=bx)OT;WZ^oa|mevqu(EhKnF{*$6mqc!2RRf|PB2ldq|RL0V1m8MY%hls?R z2_MaM2t=S0EpiXZ6baMhSChyjtG_c%lcMhuuS_SAsl8>>XPFjN2cmKcz+AOJ@9?ig zT?Bn3{^cS5-YIpL_3ZQbwW(Bqo&U#>82K7+W1I(Pi23Z4=T?b^P|CFUd4-=D2x;>ca{g32&N{OfT++XaTY(uJ;pSM#x5Uk)Nw8*5jd<}3IBgIopn@`57_=kOCt&hNU4all2T)Ugd&XY`bxL7 zFggvSLvnOT!vJYely0QEM)x*y`#rwz@BN!|IA>?~p68D1`dlrKMu^+seRZ$f6}CI9 znNX&{=B!sdn#yj17-A0z2eDJ%IsRFkBWYGeF^M@D6siiy zHE!p9(ZdEzYV2@rWKD}%E;sgia8qls*-=x#9 zmw@wql)u4_06t{~a|S9l>U&Y$;TzsD0wmlVltC`=BN$l6!o?=6OE70@;)U2ReB)@R zr;AKOe(Nwkwn=c*#mDoW03@h16ZJ%F!1|Qv>=Vng1-yYK2Ng6ImA6XgV=gFZf#jUg z=Eb5Z!G9~%q(A-{u&VCdj2)AbrtAgo5qZ%__?l2ZSwlaYJU;)@&$AALd?hlMZpL(S zL5&fatYo?lHTRY+cgX77qDOFzwSB_PtV#^QjrftcJ?-mX@RX z&6{3RfCt=` z?yA%l%LLv8vEzd@7QcX-+LG})om;Jk2u?y%$Wo$G0!;e5|x(Tqb}612=w-iJ`FLLejw(UyP#^X4FTa zH}Cpve+WOhJQ_+!^>=M#P8}bjb)%Np({>N6q z%_hEG&p6Q6$mjAf+yn}+4}0mQ^W4MyBp1AI_{RM!vv^F3kwYoj zsPB4NmECVO@rqC=V6^1AJ&6q}Xa@jZ=Q|TSodnbcs)B8Z@t$kfR=Hbx(@^9klhZiinR0digp7i2*fE+?0vHnNz4H=;M-hJC~o6Oa?Fc_eW>(D=%n8 z4nI}|q>jPGc1pO%3W-W+6#f2fj00yqwo>ASwG*F2$6qp zk&)PyTCQ%jR1U;BJ~iAe+e}#!3GOE{cgL&N5FqU~q5)Axy5a1^C>tv{$-D4h)Dt=z zrs#}PWVK9Px4`S|m?X~n$(SO$AOnrkXQ`dJs1zAmgjExbsr~W0WVic4Yvr`_&L6Kzs5y*?}KK!(|&Yt=Gvx=yv0q8?D7$uu}>e73KLdMj@z2y$sw#%&y z#LT-dx`7>Mf0!r!PMe#1#Uv6dF2vQvSIS&F8HI!>?%gzru)!yuOy>aHj!gjUHcM~z zV^cpq!G!__z+|e{*2lQuZ+}HnRh%=}7@GzGMhzLbCg&B^@H1esNpVoq!Wh( z`Z2l=IccIatm>{668FEeS1XY$Nm#fN4H(95Uo<;cnj>Vbz;nanf?9br{1aR8coEQX zt^t=W1|5B~oS#A;oLuWG6rsIM`tP^~idg6KCFz0$+6mAPk`}c_!7_fIFH5ilm4l%U zTc+(Ey?=rjJfD0RH*!F`M+jSv9Tz4}3NMpA|3pDcgJ2KlvH;4p-ipKEn3QtCo@L|e z6|#~bcH9TBBal2j{gK_-P8f3u=6&yi2a;*9LjnEue%(QZo%o=Q|&L>VN#z+q3=6G7~v(6_C%K7b2gmz^fHB#|i zZ;O7k;>^GPcqq}%mc!D zGXMvqj+Y7k@7bv=jWB?R@wuvy`h^uN(>~;T*pWb+7z@Zt;tjKU4;# z!#sk-#sKLuYd2+i=kQLZ$s$1DLu6xUtNt2p;vxw0bAI|Q4QflEvAW<6=_iZyIpwdu z-!Tb^a_G#s-Jo8GT49--CZ%T3^V)i2pYZB7z@< zpl0usCrOo17$JL*e@wmTao&uC@6{-w<}AO(5%6ZW@hw2LycGk~w}x>H+wtSZ6&4&M zUY5t)MEIk&3foXWmsRXWA65D0C3`3Fm>d<@SE&1^Xu2^8tE`+HWo>LX`p*3)#;`c_ zy+ASKl<=k zIRiKu(T28#mq>UAe!@j&0npWt^OTiqr5)HXJHN8Tp&P%?I;wQDNe4`uBGybibi0Iz zAtr}0Bc|iit+!WW%+sdIGSkL_-!%nVFd54?MHl<&XT$kf=*7lyfkC@;aG*45gW?&b znvpRZkf@^T%FpW7eOKhPlfW3v@;oOPy2=PmQ3YPYS%{sYj0Su3?{HGjK40V*TUA>F znZauXV$H`?vYFoI_MBJVv8)XTmr8gJINm)jA$>{{^1(Hj`>|^6I_>6Nnf1n=rw+4a z*>Imtap+!Yacz|rfp8h#NE7o?rx+Rhhk1K>IaOAvWfv`wUhVvb-%%9Fgh9NSzc4)_K(qlr-CCXV*)v zwa2Th%2vL-93cS{@JZ|L|vBak&=FpI{Iss9-vK!19yQ-@~h8=o@ zj2S_!jG}-l&W+yx9ivQ000117h)?*h1ZwEI4--}dMiMJ<6T2th9$z6z?D7{i36<=+ zj?Ii!vyXFTwZddL68t|ZMd@?}Q?+^SyD~)aiSv#Z_lJpntCC|8` zAIK?VkK(pxy#BoA8k19gsOjHrt5M_czUbD&f*^u(`~?UXRY8SceFdb&MT>&~vXw5) z*NoVg8#I!BGu4kWoNg{o6p2$!0Lt1+x8)J=wbOWrQYJkog(Sf48M&sR6< z!~_`$wP@13W0{4_{|_JY#b!6Bnvirqe^xr%*JEC(i%UdX#Wwi`XwWnTEQ`DG*d5C! z3D`4U4u#O=`Na23q#T`C?Y@VRPud04vhtPu0ryu&R^5L_r_$}rSG|5BJBemBS6sO_ z(A)VH=h`K5N%aWLFPxg;Tr9ZBEqeTwSRPbZM?VIs=6Vo`&%%QE$IE4ZFce%p%=M1a zcjCnY}b9q&l(@ef^71P1nLL^Cw#P`95aRN8ew$PC?dgr~GMD*P< zk?-Z4j;4E;XTpNhRhu5U{1vusSgQN&!P+B7pE-lS7mMx9=o3K#ZtE8v`H%Y~+4?8A zRJS9bT^IjIcohIvZ>-6sVb(@bN#_H=_0j7_zBuuk>mr$uYuVT8+gJYnU7QJpeeNYkvmFkjH8L1@!ksU2)j+o6S`1AY!YEH2GLr z><3LHXg|T5`L@QTbb;o}tu=V6i96{5kc0YrBop-yMX|QPm3=Su^1A17?C3R{KcuYk z$0KLNZ+ZZlDn&PI;#@h%XiCxagK!lfq%CZTO;N%{>DmgQ*ykw~1+vPu>)r4l9S<+= z=R5%??;6L#cd7_A+|G7anlqted`R*cX0zG6Nyxfr4na@TxS^dMB`sg-iugzj!Gd0z zkV_Ok`IG8c#N8p|=W50(n1PNFxj3cFyBmG9{X2=PU9Lgb#!M3KRwsktlSs`;*UQVn zK{j3~70-w{t@+OJ;)^@@!lDJ1*N+(0D3aU*V6~+NdQUQ<=l{Obc9hAT)uGRsY!6yo z6d#QNBEnD0l(2Hs7p652b0;Cw)4_pg)u~5hWYS&i2#PqaIz^Ji2qML*Z5XG?2sYhX`EotC=1OKo*r5yZukS#=VL60PN)~1y96UT+> zzm`h~Orn4JCPcDZ{1*#VO&dwVuBZc1c}2pv`m1O7s|ZuRHs701)(xJ!vpq~iyfkpK z9;I5(ALfp(Q|#iYxsjc@NwUU5;f|8^=DW5E4vR4V*Jw9GW){+adI1eCv7myt=2Tao zOvmb0M*~dsUF3Amt!f?r z^T=m8K-~GqYNT^u2QnKrZk=kXchjyvlI__wYdb;9TdrY6;})!f5q-bpyVKrsmzdq8)ALBUOFax>MO6;*BqK+Rp%zCi#&BnOqlsXVgGT`Xy;)JvVV(V6H4b;qsP z;@N@+0syg$xexG9+w4ytHh)onxyRRhkz)KEtSbMEyREmvl12sGSi7AR{kv1*xK#Mi z%#WM$yNJIK#KD@y?N3(HU8ix;WwUp`hl@rn$>W$LJ)fPhEpGx%eT#i3&#%W#PJJuPPOZc?lwVUey{Qva~p9;l6;? z*S;{BXW+kIrs9Pjf)e5xmlMw}Hi+OUYQOfSvaJO2UvDL!5l~LjfNU;HmZDl6yHEcr z0bw#b+bse`tHy{VkFB!D!yyT>&!pb%qy4;(vxm+$j`%hQL=Fm3l@jXTZvD4#i;gK1 zv}!qqqO3V##6E3?cl0z2w>IMGb|%QcQR_l)A)!v$fgZt6&K*W>6p0TBB&vRqy? ztN&A*BN|&ub)CgI=O)a_4ib-#U(W$Z#vRY&E)xC4_J81NbiG~qRS|)O8uAD{c=FN z-7B|Y*FXkLJZdN9J1=@ERWHEg0_?S9Pvr4YtQSu+5YY-hSvHXUR{VUSAgV>6n@ zKaj2MF$pyf8_(<@1kX~g`rXVkT+2PAT%X?|eeWw|tf9e~0?;%s!!c(MWba^CplWZTJQ_u7>l@(0ZPnHMc#I)acRKYIKzjjnURG-1ISb z^5W_V?7jgzw(=J5Z^t0s4qF-n?IgON`yjsEv2RxnEp#wZx(}f)i5KO9DZqXe(lT~j1f{uFzO(FbKSH|oiby^p$9IiM4&n#@L?6uyt`oz)H#bYxSQ_X~*KtbfC} zd+7ly%Vah3)~w=mTvj%JL}MP`!N7F;(}*;u+?9>xyy+$6G*-6{UyhxKj6yn~)~p~v zfLsX!Os)%yNyXWDcx9DFg(v3N(^SttjJDks&wYh%-;c=bq<7r%+ZTH2n`MN^?oT?K(kzd)csU z5)zhIP#KJg=Z#z#@z}Q)mUah#c$V#9fXLz~*~fx{m48S_fFz}`xdhPjvnyrRDbi5~ zjmCkt$qm}I<-kzdjkN0fTc%%yzg1DwO*?t?MR0qbfMh)A`WygFeFzzr+~f0G`^3fL zjKg9&g0!*KOUI-UidufLx130>5<#5cXnt6)>5FZEW%$Xfn>|$!Pff4Gx(}?Xu!+nv zC6LTvgc0X=xf9;EyC?`6-qE<)6w@1h7*97OEq68UPW}v18%!8%Sly}pw3hW220)B$FEI0IRRmt#Rd>hMKO#`OcNjwa5cNm(ytc6VX|9?UnUc*y@A zVSfK6{7N}=@7W!B5GXcw@OA8`e!Jw7#3sN5xEOP!Mx^k|qIOVl%8cY~MI%a6!lP?{ z<}jjMM(_#Hw${cVT0g?K(e&(F4R(65K2EQSvjd{9*S}zhC*kE5GMK0fotF*0ch5LI z)%~a>G;%}8Vr}src*>GagB5<7;7)%kQUylM+Dpj%-cK{a;kTFH%1zzN;)Ro0iK@oiJRv4N>ppn+L@k(P(Cn9 z0O=iH&N6>&u!X?^;9pAMaTwd^l(DN?UE7tz3$ey^a3v74Ah{8zK=!g5sdKV$@sOtp zeceU<`K-LXZtZhlg4<0G$}#Acw@oHMa;Roh{=DrdAQp($mS58fsW%a9zj(wyM5Fnu z*nl{pC)I#PL;?CInbx=TEg&J6Jl4u?3HFU-P)QD9eOqN+He48DGD)Y(Ra_+eW#ji}Cazr8bn-J1Vzx6~i) z+>habga?z!u#rFctbX?!vL+b-blBW=S4u;}uxVAsSWks`-=S&XZ1?(KvVdYNt6`g% z;_AJ*n23g2%BHVYw>!X(kcXU9W}wpJVDzuxgHQVhlo-j3;Wxf+l- z7oC_7OGcHPG)~Dm3d>BXmmFQ9dzsPIf|ourtM~975ZDTjOnGC&TVucRTMi_F8_e-- zm8{#p`zv9t!pjVTkY~M?jAJW2WKqADX*BYtYkQzg6}ss))%NDFD*27cHum-?8`Os$ zrJOulwu$OTNa&n0%|-Y{dt@5@%q!GJNLu!I(sp3p?JHZ+LVw+J`22*XIW6O6nw1Qc z=!PAQxEddMNNnG0_T4KzQEp_}2)3r&YZ$eSx#JmhpXL2E0hR1OXX^Hv{?{0Q9?-wE zE>#eLJ)M90Att*cO#V?e$6T?ULVw|AfBI{x&osV3)5QfSFAf!}XR^;t~h0W!|3+A$lKrjM_-;L+bcj-*18gCoVHRhI0*V zvyvm7>nJJBJjdu?a?VQXy%x_|1X%b=j^#B<$>m|dg+X2Va^n%NeHRe}A9!8J7kAMk zyA70afg|>m!OzuU?{$WWfbFi%?tG)Gre3V-k$sM}gdmAgtmJNqgTeg)$f-@wr>_RV z51;%6_sdc<(lRhc7`kIN`hNR@l+EAEon!!HifkI(0>0MQK*-;xUcUOJA!!rA0$umo ziL8ire60wzUaw>k%#E{Bx_H1u?6L_s?2g%-xr`Vtp5WB8IJ@DvHB3hyaO-;N+%T{u zg8n{V7kOsmV0iWFKh~fs1DD^M+nWr=Z%)uzje@3rV}%GnG^cID{bCD@S~gQ=(HNu9E}Uz&*UdWgjf(8tjRQN&M#^)_GVOxRYdB=B^=LUO%Zw6+WB>;! z%+SQVWp{~6^Na^DqJxP22#C0hK$Ufq7WT=hVf%fsY8nG4f#3 zs5nJNW{=T%&s6ne)o#7j!S!5h9ED<8$9uN38SFx$F-Qm2Itb{epWY5<`d`1nKWVD< z5#ow`MSru}Fg!?W8Yxf;zW*TSz>58qo=84%R!ejryW*942ra)YRmrp8x0?y7+sX;1 zx?!^Im?hX`M-XmJoyuuJ%!cA?`}U(NpriuEPB<;$Kz2+s3H1NZ-bt)Ad-rOEUQZo9zS6Ah7tiM)|yQe zCWn6r0$SkWou+B?oN3!M$a03Q;uWerYLs?QP%R%T!ecc2c7%yYPNj%uj$uxHBZ-f0 z*smX`sDq}1xv;66(j@k8EB~hjNL&_lVFLy;{`yHRO$C^6eO&Y-?rYnw&mMwOu~ai2 zxX&iyzydpdr(EzAvN^y7%ocRxrm2M~t7_*NcHa3GcSPVCzLbhlA>W16ZBY_7O_{zQ3spq^sF|7u zmiYL^zqRmpoR|yhO~6f<1iZhGPGA_Zj2clJsN~ijqU+A9Xd;1V}mbk$k){rE9t!Z8^u>GP~q^8N$6Uq zBo!^x9C6UE%*ykuZiH$HXK+x5Rh$T*EtmsjlO6UI!m%l4>EJAD`gCbuA=LI**wy)t zLK_TIPl^Kc-rVEmVHQD7s@-rHX;cP&5vX|!1;_49xn$iC%z zO=_%T3HM`rE#0v)H_Lmj^edHyghq8I!N-^jWM<|+@-qX{kPh3E=Rl_jxW zgHrFc11s*lz-mC9Gg_ExmVO>@u+wO;UY*2p?L>wjVg0D7!$8TnQhNDcI)%Q8n<7Vl zX>s-&*o{Ngn>0!C{c0CiM%fhli#%E9hJ`<~Sp-D7HKI)3d}m6vl=ZTOTloZY2Pg7x z^#=k!Tg%k<7^i>r)9D+KrWoljlMs!octuM%?I8>a8(=k9A)TvrDxomDNFx9FsLOUN zgHT@rl)9%qmv0-)-sR&g<%u;jHk>u#Bf#fw^Tt>O=ir%*aSvpYe~VlZB*K?zo@~z~ zXM(L%&#>Xq17hQ0a?_05qsKJ(JH}Oa9oyI?$M^b9WhuA$AS^aL8=XSC1Ae%g&d|=v z%Pl)RPVU=$Zw7OEM!q<~_je-5wpmLzdv_pvNk^Zo?IYU5rP-{&65qCKxaRb(vu*he zN12J5=HQHWMV?IH#}&dIt&Te9&*%3rW7Lf^IS(pjuu4EOl1JcCAq`{^_L~lYrB$6_ zZgnl3+@B)SOU_1D>7L9@b>A|$G`CC zDmg={z|VsD!3eJ#ODqGswBKY&zZ1aB@@Hl_a&vcMFC8p#05f~(f0749{46$*5`bUT z!Ed91K$V7aNk(-_Hn8;EAMNjh{r4RR2mi}exJVjs80)%iBR-O6Yn^sae-1Bvkg}+_ zxp{rJn!WZE7k`!%_~MjZ((!^Z!r%YTV}(560$ug4*sqH#FwJT?05f!TCsx*N?G;d- zO>+nUVt~jZY~J*j3Wsu+JPUaLlo9uEE?brEGtS)WukQIG|25s|<74}alI4l*U&8`( zb*?7BN%`+G%t$okinKPxdTF8FE6{NJy3{@KfHJ393$1;Dq1kH%Y)N|#@2krKp}ong z3Uv-~pz7@rvZ*&x&=jiWu>N4;!%@JO{w+n2VJ7)9KdzUoC@Sy^dXMt?<73jAcrJqx z4+T`!fxR_8yPVxlEKX2vcfn8YcIh#fR{xuMwy|WAggo#Xk@d$+A;IqApX@(~k8S?NQ`~Mn zY(Hqk@|2CXVHSJt#$7d zMe5ABR;^iP2x0J_@4ZEi`7%N+;fl4|u(Z+OKrC3~mQ903G9}fdX;Y%T=v99>b;#qn)DkqO12}yToLiGQzfVTX1HKr{VY1a{&FJP_q&rGdttE z+@P&t!GT#3o3|nIUjtjJ%Rn)e2JV74^Ob3k^ljsQCV6Hp2ABJR(BzzGE4dAPrE+@e z(G$WH8O~}A#!+uSvnjm&sY~gFRH_G$x}0^?pIlU$G908H$%T;e5C}DhTYNs}K2z>W zq58Qc1dyK{1%B+4e#?`DtiFF4#9J3~n;5V}@Vdf@j`~^I@1mv?)Ew`W*WS&!;VV{_ z9aQ+ftqSvw_Pj_V?sLp2NP6wsPkO*?J5ysjrm|_q%%+R1Efu+vk`i+CsjJw4TL#dd zz*Jn?@*uCbVw0XLu2xT#CaZ1XH7!&LwAi)f1n)xpW3L-)NiDAu7+mIf|F`f?`ON)i z?_Sup56T*8UW6xQPx#3V%ba#{2^x&rJ}=y-OUfPym3#%|@UInmzk3J%z33x)FGN;4 zSoi5&7X5J4%zgVW2+{zZXKr4$+%)r!<8BZ9 zww;QOTl0bYS+v-!RpJ^STNANh2caq0j{;=|e0|t$1IbP0Aa?LCr~p;DN3f{?)cY$D zT-x?NM~I&@2Oc&3Bx%_7+3>gGXOv&eGC5iSy;P%pHUW2Yja9?*4{N+63_lLIm9t+l z#4wp6Oo6BoGUHv^2E7Kumtw#!Yu8O_w6f*UNVHh?W|bOoVup)dO1CCk^u9rfZ0+yX z!fxL6x1J3fFIdjXoOAZ618tK%p8QHuD9q?c0`y~4o?Q10H2ZqvH{xd;-Dkei17 z>X91HS?g3?(M{Tjd8fLsj#xk@_2=)(-_S%O%$<_W_DkwkB&&bX>a3J0$2B+lHYd%3 zve|aWn`6|j;nJ?dK33eqaqK_X(mg4X??GPLpeNvj zo}BkUzT^vJyN`t&WKVm);3ooSLB~UBK`0Gtgro|MQ7k{+s9zVph%s@W^zj@xyg<)_ z%`~4?W9vk75;s=PIx;gyLX#79C5#z>DGb9AvFvysx-s>#hG=S2hp#`qc zC!)s*wEj(lj}IA~qxw4!H134&n$0I6zwv5+%tUSO2s&jD_-(h|3lO_zyBv1T84yQ8 z?wKcCIjR{1-*4m{fmS1Q1I@k=pOdrq*}6FPlG1$S?xQJ6R?ton72(R`%V>Mdxe<)71T7In|v{LA~S;>HXzH63{JvUH`?ZYXbNK z`s)(6m~YXs+mI>6VWm}{{P&c9hTB}aG2Dx`fFz(|HEr9x{leWsFSKq<_9sorBX{;f zWC4GIZ@2T<$(Z-*1wfA4#Pi6x!)U%j65Dssw2?~$cL)*HQ=;1gg&65Q)!q(Lf!&vM zm_Hj?#sx0TYgUJ6qUf$ZhT3nkjSJW8h;&up7reZ~ISc2mncGwQ`cy-eTVBcmM5Ix= ze{a}f97)LEyujX!#KP=qeCh2y)W>QE1XO_q+B<9SiDGDRwhW9ujtuXiicKQnlm}9% z24t6G7XBAt#d>8wz&xQebvC`)pO{M2lQR}T%=|0t5jc3dhXdM+Z&^uFysC`=B~605zqU)6EkH+C@3al~wh&u~QS*cFXxq;r>^*^Y zVd7te;cq(q0O?LE%c8w(!VA)z0i#m6i=ZmPlCyGnL3U#3?5e$rMd(_eR}5p-I4H`WX_ z*ElTlPG+MsGE!wqxCc0rE!6}pghI_&I1YgNg2i--Z!qSflCRoxR}+b{ejTwi2f%D) zYxsaGS-1FOTp|$XEzJ;Zu9_Noz9g>N^oJ3suGYj~bObX#wV?@`bG^+$L}Up>C|uxcdiRYlyyg zNAJ&0UZNsMr}BLEc=eb4MsIwC_trNLs^+~Wm|d3Qv>gHdg@-_k4{$ip6S@2+D@Y|D zdiOJ;s{QKo%O62|BtiL00mwYmmRVac{c1!g9}pDrvTNOtQL&@_ScOS+KSQEu!Dgj1 z1hHC~G5Tnk`IT`w3TO8fZ&}svo3a@(T{&~4oP-7Y^UWb;^TI{3xk4c{;0#nG7i@ih z;)>YFTF49yI_kg$7Dlr+TEiu<7wh>NoZ&BJFZ!quNW*J6aX*P*iR`(b)K4s z&-E*xhePqx5O)zvhVUA(=37(2C4~(B7R{5g_A_*0>5+QH7q#D2{IEa63uHub5+_GZ z9Y_94VJ;oF-fQ)}%Ur-7TgC`bUf2P0`KlT7xj$2xhvs`aW-<2nb#@#CmwT6`vusg3RqSfhN&l8)5s7Isn`>lW?sV@FsL1n?mML zwwSA?l#UP6`>9g%#9HKz*?^%3{pd|;}AI++2X6;@YLtZy3X5Z~? zEr6zo`#d$@Z|dntL`~kEg-Qcqrs+4vHAM+ye9xY!Mj}7rKX56l3x4pkKmajfG@Pl0 zynND=sufB}f&hRhW5m~cm|IK;v(n8A&h#f1%BZq9VcLU59dk&!xKR=1`4f)kOfR^wl*q@We~FazU`S zp=KNrdy%r`y9t}7ftQ<~REPWOM{=*n^g5O~Bz){sc{v#R`9Yodck{}?A`AyjOD^Yg z6YF0pJsUn3?=t}`#-NVd16N*T0BZN|m#5I|w;2E0+BqV2+;XxbS{I4I}ecM&A3cH75lVa(k0hXgL++`{KfhHRTIGzKX<_ z3{K!K59jA9>W-GKP4_(2F$?o8@M7fBurJ1M8FO9PHqx|LH|>|P|=Q6M|kdxcj7i*Y%L;7En9f)P8I}11AD$QGhDSe^^D&2C~F)8i=bkC~%*JIhJ=e#SJE zCx{tPXH-VVl*In%4>H^mxP6=o@a3;7S6t6`r%hl+2JE+en~(K0iWrMqp+9cSm*6Hq z7zRn|h%yrHDO&?_K-+?nJE?;oz6SZ*zw$n$EuxS3EHVA zA(T6+b0Z{8*#h6}PNY)RwX_eAS$j*4O(}e{p6*`}u#(kGMAXZTTfsw*&<=vFIw6Gx z@#oYzQ~NOcrI8eK|0Yl6i|}+}9gA@Ki+IP2a4bJe-TuFL>*dGJTa>mk{|y>lBIiMTtAj)%t*RrFGfDpTrEW3i5);1jJJjJ`0SC z@&|ME%-F&%#H6r7mfz2YQCoHB6O41-JLf;823dJYu9jD)|aJyVGiP%9F zk6PmOT+Wo0QU;?Lg+qboQqkKq`M*>PPhZTFSu%^*(KxrB{J~q`(XTWC)%*GEDgJT| zN@}R8A@~|_Ni^r$7-0ANGaVj9wuFbxpSW!Jii=Qhy)Du%d`uQo!ndvpey)=&V4gCLN>wtvE zSddF?kv3~U)|KfJ*LJhZVzY?jRQbF0m_w;#Z`IGCJo~xCZ~_e&@GA|yEvjBN$p6J; zljPrxb=YQh?28JA8Vf8(3rJ$6(`8`ZyDT+-|2mgm^NT;ZNxws3NH;=%bnon$`JqP2_mgLeZ(J3eKn|o zpXMiizsUMVlf6HbxN8i|Z}}773voOv#Ac>F4VFl8a|kdGRi19sArGh@U0hRbMbE*~ zGGzmy%~f`PEc$HLMwo}a|8~9_l8HT9ue|;?C2I(!d{%FwdKnJweKKhS?i;x+vAJ1E z=WUUiUY?c??yt}Dt>(gnt_5hi>@TGG0cJZA&PqOxXk&5?K1Q_^nPSAPu>}zZp2GR$ z+RJ(e!lhaiXI;*8Q4sQaoTUDs{<-$B`MS1I1tQIVTbASOyU0xqJZe5nyos2s)7C1D z%}VF;7cV>8&Bv4vga(}J6lWvu#Kx{ zNfN~=!*b4-t5eyEC#a-2rHS5yMnhs-lqm8{p$m^|l9tXSs z=pj&178Q3-nY1jSYebHvng_mSF!y&;QcC-yKl~EM*Nng=dv+I*kPLC%UI@r|Oly1Y zhfr<sYE|MncBoF+`qOJcWpv=vTZL0+YH;o*La%hg|m zj!3_h^rsG{?-rf?H;s%zg5HXrc((YQduj=KtMj*Y0F}o-8E%-IH6=j6!^sRlq0}x)4jKXE( zn^!g#1#)jH4&Ad%(#(wvXi!|KSb7bc6MKID`jjM(p&oY`L&#kM_DJaK_0g-3UI)L> z+?=>krM(gQ?Mfme`h)zl9^Ax~o-px_D$}STy~O2=;j?6?(c{|*eYVgh zXXCpy4xyTBsSST%r&F3>_K0d7Vdr39282~EYoXN}pcBEW)XlZ+8|e-!iDDU4$K=XJ zzF>5G(jc#SU{SY=W?_Tnj<7qe$>AhHsa6ZZQRMgu=y%BGOHe0^7>@bhewRi`l@^)dB+oDi~ ztj^65HoSV3D!i#x1%U88F-8V$kS^VHN9@Gy>^RVKkeiYKK$+!T@9c1W1|@Alre*>B zj50uxpw3wDleR*wZlkvOJr(x;TE?eY^@cZ>&(ocw2V8%9|7J8e^_zFT%_=!~fh2bP zNz!?*vT&~6ae}LZKPLo3o!6ChNVZHu;W)(yCg69zJtNIIwKuMFX&-VR$~!LA!04VX zr#YK3rxIC{W*Xwx%!4N8SlGwK6h4tnNe1lenFQfl7mhjAVO66`9=r( z@Y^>0MXDjp)zdL@S5f3(`|A{spGWqawIW#VDwi+Vxs^(K`R6)gNqytd!hTslEGqf9 zqdZJu)b=-|-Xdp>!%<=9_OtKOApr|>E$HGSN8c z?JcLui_ba#y#nsdrXpsg=N!$2*>C2ucx{Qmcil#enF`+AtRMKpOgCO8J9=Uk_{&?y&SJwUD zxF5gp;U7A<82s0qc+N3@RI4Twx!`Qx`~&yae#`Xh>!+G})MrWg2poKhlGkHlyA^^v zxYok*;0`${;So;TN!seUtkkGHH6G`L7wFA_RBcR#YfebxA`5!9bo#I7oUZNosdxu< z|LiKLMDoY)|0W9#Yt9Egn+yGk8thaRXbLB$W;JnIP&0Z6O{F-vZ#Pvpgc5PzP&pHQ z)-fNGcFb4*vhrEt(dmF7^_#n=nuTwYN1wI&qyJ9}fD>DGC%-zm#*r@*AVQaAZ_g)I z9ly}?=gk`5KRAjwX_K__e@+bsrDT6tBmHhr^X^V$G5AG&LlT9Qqbk%#g_VkfDkTql?zjc3ly;YQnR(ForERAFQj zYV^{{RQn+F0e)wo5wTLk^J(4ev1GTWM-tLt{f2M68f=Yddpwwg&-4fW=ep#J{9+Y> z?}+*ZRHCq^vu~x9DBo^eSI~i*a>~$G9|O0VZqnvkYST8nkoaD6BM-6u5ZLrh{@G{V zY5jy*=mR*32Kp$6)0-HlNdTgQlrVq6vd0XC*V-iORPIMI>6KXGaaLmM@FXj-6)Wsf7gcRmb!vk2 zlioRGPdWk+kVd&uGmAyXafH>nn(!PTsBA||V2NFzer#3{)`4Bv>E|8Ei;_IBpyXle zir**JU}%^*i##P-Iw8h+@+0^|TZsurcKMoPq#0b&;6Nr$@3QJL4S8V&Y8_iJM>4RW>0QsV7^{` zA7{^r*-BXVl%sT+ub4(nHrkGvslL>})=XEopWovb)DuYPk-Tc|Q7~Wh92nXzJ_;-V zx*Z8ZjpBO8*MROzq7CzBcb~l>Rw(h=C4-Ms#~%88Zb6)2JK{826GUX^JIq;OhrInh zaRL*|`$zYAM#&xU-DW8L{G^AC&maijO3;wBRI=w=XI8S;o5+-zQa+m?-mgF4vqWrHMMwgmj?O&T`Hz)*PLB9R zOw|@+q_+<9j{k-HE)vuJ_3oZCKlt}_7Ow@>LA@azlUm)0=Ea?1)fT8-fX?p5sVwoV z(FGyQe)er^_UM9_;D1>rmL`cXiD9o0xPkMGkp>wAns_hN6os9?nA6IW3)?~x> z0go6ds30IHp@QT9DFLOWrKLs)QX8SP(hWBPf~0hpvVpLXQlcQ;8w{koO9qViF7Nk! zp6~s>|KT`xT-SM>`HL)lTxz7;vF4J>{F8nt5t+X54VsKDX0KGmeqTW|4r$!pdB{{ z$uALEbhSSISbJWek>CaRrhj?=QuS5>C|EQvQ3Ka~k-u`;lEC&lKu7nOri&IpBO0uH zQ>wuT@zFSi=)iJ~a`b3@TFJ;GgJ~~`_iBGKn%&B5auW1c%*-4pwSG)XId}!z=SI1X zR=&D-x1)}M{Zfsb?VZ&-+G9K7X@Cy8{U@y2C;N3n2C^wn^IZ>sz`EIX&$h)nS6CL= zd<>N)G-6P&W-ewMB`LXz)^Ob#hrtqtrVI7m!P8*DC-oGOuL2l_EbpFw82PZbz7}#2 z{ifyAo+8KAdt&_h+d_i@N%6Hi_}BhXRd?O6ab&Xj*RSUZXr#5&7K67=F1};>>Xth4 zP689WVPQY3@+qIWy=2k7HT@t+Yf$&24Hkt?Ze5VSd*<5Ctu^};<4+2KoSAig?jGrR zXcE8Q7&-?lq?vH);y~+A3!1g1d1o>HhO`U@ZiIqIRL!d%-bvW%DS35;JxgYkHSG;( zl#jdS-*jB|(P<-eES2y8H3HH)wloRPm>>K0kdkTl{|H>IkR$AF>r%-LGP(kJEHc&y1G5EeJ=~?aWI8+Mv$pI4SqLN z2jf*;QEq>k>ZSjk8)Qma7a-9CZRJWir4^U|T$IDXOS6!cnwK@EA2={07pUg?Qhh2{ zb%!QmRtFJ87jSf$HDnt_%la2)+V=}HWMBJ?XsqM9+;d<*(`$0Z^Xm+-DT4d$H$;lS ze;Jkg4apwO$4<641hY|$Xx2wyzvkY`QxJ1n{$ppbwg#c48A!1ke8+*6#e^Yczt|** zhc~Ag4V?1woJNuOUDNucixOg_WfKa#I%=Qv95+;+g#02_l(@l8u@!%DWan77I@zun zKse?C#ny-AQ}yiR2Qm38l3g1(bzb5g+yBn5m^7&E{{d53r=D;x$zJtCJ6g`b^Be5y zpgyhaZSS4$bs#rTzGRKoCVR!|rzbf@Q%$ps6JZDk*`P<>=N0rv>(dRQe>tk{Qoqg> z;IipN4E&x;%17eoaUW6p#|-Pgw2U2=A*R*gwNtN~+Kj*E(y z+YLIthpTNLKrJ8iuO+b3Qn_%%T)(^krf$7#X)h6#aG!6$rUwQ5cSPkZK)(;KkhqKR zt1VT1bC>Q!s(J|Y)iv+@fk-i@K+FI;_J*@^4Kk0H);Kcf8xxW`m$^Rv)xi11^F^KF zhB)(C#Q@=-X)|UX*JE~ivyQFe9#TZyG!6ZdD*uhErk%n3horcIg;Z{RY2~7k{F4D7 z$_p30z|YW5JvxZe$r`ffkY#2~Dp_^8VR;6`u9KJQ6_lbN0?f$%)jYQ%g&MXmUf7_r zJWVt%<_B-678Ahpe2FL4KKxpXFMajWjmTXKuF)HyLk#H`fAxpAA6-dg#hCY_CH54C zH6R&Ad!!u>PXOp$bg>iy3O{m~+a)d<9(-~6G;Ye{t1eskA;_`6H750+2w0q()IWEg zye-KAs2{R|E~&}jVAvcKoN>1Y}c`y&VBzT$Ec5pa|-9b3vpmVqn#&397juN z07)3Zr+_AhY#x=LQHdt*ZJsgvOPbKLKbxu+jTHmXQ-Ht{=3EZ2TNn<@BB+_jO!Z(7 zfGKvKk}2m|{LB-vEV;SPLAk5vfL6RIjC^@SV!|rSYA5*F5LJfOU734*HjV1Zyo-s9r!}rWO-S!2+Xh^%rTPUQ3k6#VmbH5X?I24Vu;d(5+wor+ies}Y#Lu}YwyNTJ3hX}-U&(L)TxLYyE z5AeKj9A3Y+ao8&R6GP$mfrwbWb^F3!&e3NYjxuY=CYhZlS^V*i-RJa?BNpY=X>uxa zx}aneH3IIRA{P>uZh542`*Du!#gTzxdaJ9&%3vv=gP6VJyD6J-Hes7_IpBF_Ka53n z?fBi$K)iB3VU|JdQCaw{#HG+cC={dS{3Cx&rQ{tt@lDoDYjQXExtbw6=tQB2BvR#_j7j7LB*Yc8q}U<;BOeT8Eq2_=V@h!HoM&T>l|*)_6;wNBNyTBc z(6U0x&}tb$ndK4rP42&2#Aj!sf!ObIOu@yg8 zAU(j~KhhUdsS^oE7R;e8vy2V|d|O9wDA-^CY^4%xwbko9Rh&fj(6Qz%Bv%XeB|!3f zL3!;__W8GA7}stSoa?zYQ`F`q=LJ9HOJxFEA~1yhfs_Q##>;jyvN|rXGeOG=z+7u4 zPNO46nWsM__6?HyNt-g91$SOHsnQ+wF!+BrM-`tB8Bt%M<_2E@e>D|E+TtcpV$n?} zb9sG@xJBVK-v?~On{aGu_z$)UEKg($GMclh51G9gOz!!f&2P&h<^z1{9~O4ZKYW#C z^8fn11BAl%ndiSMCIHSsXn=<QJ|Kp-l|n!KzBx7bED&S~ zymepr*h!U3s-KF1h_@kmq}$>OFjP$&OJI*Qq`r*O@!(vl{&V-n(1)ZcIPui&D6YaD zL*nSw;=dEB0FeQ(o~dO#`B7YIep$M0c2>qu^6ZPl*wsii2a4!jZMg-1n(P6aH6)-5 zB)lzHLi5QC)19bQ0-^83-)$j(r~q-9Jz(*QQy5@fUEXMM=udP%Z9Z~1-|6W~#wKsh zd|~D%h{Sgk)7NPpjylWkih0z5WG?;Nbd0?@I=_TmG9twAY%%dFt#5x_sbVW`XzcEv z0Gs)3{4m|u5+c3ZCVMYtcAA*B*B6PoBo)kj5k(4djDfr%P>|$Ms)FHi&-!MMi;Y;e z7+im)p^DPa@N?1BvxoFsjlnDxuqnPNaC=*_6hkjrRhtlEw?F$1=|8W1t9d>#{N9^M zo96#$7Pl*Qo$8l}a>V(Pr(lUCR6dzeUj5|WJiQZ`C-@>BxGlX(eA5<8|pO ziZ3K+wBJ59$GKenE{7I>KI&N5r6epryKXCrZr&$8JE|P)h-CwDCzE)xlC%2?Us&t| zOW2fgu6^JTn+B41gUd|R|Cyg!FY?eH2!7*CI6oXLy{Gj4n5(fw$5{HbXEY2XM@wc1 z>?{hQ3pI!BfdAkFv&VV4m%9!u9;->=0uWhXid6R6#L!#~>PF`)=+`Fn=`RSNA4fH- zFqyk5z&7{O3(MT&3SuGsvf8Nv5);oIuTw;byc41CM@yVvn(>#8S~D2$&U@M}7Qznr z-`Hn;5XjBOPe-&|&HF(m{S;J;*jVmwUbDl4DnaTknMVxSv(8@8s#({66O}Cm>N%jL zG6z^D1jBXqMx})OZ`d26U8YhaX(70&P2f;aGG$?U`ceG>q1>-#e9|ZCuDe>59K(kv zJFPL^PrgVr6Xr63cO zDs$R{sl}4cO_q2g#IT88S+FF%uEM?yh+nf8q<0y`FFZXtCN|ft| zhnc-#k0KRh?N%<*TW{9mx90M>MXmT{1vq>}PJ9!7`fW7SVY`|0bNMFA@OSPQquXJd zGdBBrr*WO)A5i4>?u33dhkg>CY;Evt4>*5`^v1U?Ia+U(FoJ@aa)}1W__m`hrQs`r z%2icN2iUP50ZW6`%o%_6ye^D@-Fwj(@FTvZp17Km zw)IZ4YMS`(MA^z4S&&;4U7LxwcvAN$^LS{W9Bs4Pk(=um%%a(v(bwXv=~ZXJ<_#jhG(L8%n7vkz3%m+q zuFUpXNmp25VyBZ;xZ=EV1M-nVw|=o2@mEKZ%H{W-u8^ZUX5ZRWJ~>JAi8~;r!!CEe zN8+{fmPQ5>^e0dqZUgfjh(21_8#i^kJ-P(m)h#_^eXmf$`D9|0f{V3o(mqpoacinv zQ>5)0=3C&!nZo+dJpZNhQw4bQ%;2BGZT>fzA0eDr{hd{DyX9+*tDm2&MU$&Y{$@(o zv0&TQPRkv;_v_gr38Czs2em4m+s|N>K2skt`~X znQWV`53pUb7+~-O%nvK=u+!*G!CePr?863aB)P2l5clVZm6^gndT=wO>9q8%g$`9V`NSwu#JJ#QdnX(2KUrc)t(l8zx4bLDCq+FkhdnJ(O%Sz zj(c8?C!8*g)wWi-Q%%Pn3sQy`&6&yDg@GA@DBe-yt1I&TjouK?`|?x%ai zpS%IR>zmayrxS<4&1-+$D#?~F6KN*`9ELqV?`=n-ezF^eI(c9G!N$7?&eOBpT3`=} zrZ;SvugOVPJhitoB09C*&-wGAlAvHzb_N zy<%};6N{UxWn6=Y?WKy3;C}cEI2CsX&xzZyW0J=e&6tXm?r0gs*zGXR*2#mmq40Yj z?Xs#7y}75*PAOz?1B0)VRinL3q3ruBr|Nifk2+XS;BKR;9Z9JNd{l(Hx7m1cP%kgJ zr1jfsJxB4%VdSoOh8t9Tq!ydpm9+TG&)g$pGhAHFPhRn?Y{jzY@VjI}y0n|y{@|?Q zMN0LKrjh58f?ndPTaWs8hx5uk1ikXIr^jUt!8qVwqd;$RWUq4Npt&Bc|8to*_~NEi}X(F?lbI&J*31 z3!dJc`zv6h4Y;1T?d9C%pJU(UVtCtqc#_Ax0*1^4oVrNw|DX?B`Q_^q*eMD9Hb3&p z^p$#D0_sn6PKV9UO*peVGL|n{Ha!0PWOi`=sjau}Ezc_L%0N7$OJfD`cKxOI3#7T! zna}7;cnw6NKlq8ejL)}V&*h-0DieE_t;Qbh0-EaLo-`pH>Z{WN6S&IB6AgocqXpp7 zMV|~J4Zir+gFOQ9H_45(i>-lSi~w39yY}s2>{eOXj=Jw9;Of{e{vF%xH#=CcxFJ?# zHw&TG0)^kBr%$K`POr+17v~fElZa;xnYIWjxB01oYNI2cUB8 zCny)hBnwi2q2(p~Ua%cI|1tg1JYUdC&&a%+WBZqDmredUn}Y`}Y5bX=(v1ApxuIGX z=`XWfJ~!;{U#J6?e}Scc{I(m-E=)6-`PiTT6DB)67_c6sj^~Yu@A28kht>Ppizz+X z_Up!7)~*nZ;f$v7s|;!e9PA(gc8LofkqMn|IyAelX0d&ENR_XYg)ot9G zQ=BI40ZquYTwjri^XR0Jd2&|#6UH`79XhpVX{64qG~_W>;jsSaN#|%9bxYpg zH8ix=SzYz6+q17*bZq?i6x$?%{mBnI3MnOWKiF?PuH!}X@#}JrRF7(T84QEmD5e*j zTYpv5$KUYKy+Tj&HugjB?N|PETU;p|sN`%^a``AM56xtsm@@Sgoy<14oGGQbhFXEF zMRCn`%?uJ1Ng6yt5{dNxB_PDs6(JAes)_Roh~D-Mr%m@xVTs<<-{QG}|IJ0)=_u+b zd80f;2bY(9&7_gI1b6LTFa6@$OEYaDcAdVOl8m`pWdj<4kJ1esH#^~!TTWPwI}@qP zS^v&Gj=Kjky*MXqH!(Fz+VtG?iMCH%V{pbYu4d+~81^6F(n0uDAhWp}%Eu^=j2LCy zMoU2O+?Q%&Z6c79l`To*Xz3QY`O+_lpQ-}bq~m#tn39Zxcf*^b?%8cRNg}r(vOP{h z?6%Q`}b~-q0qJn1#d8DCenAvU8av=x4jjy8>$B{5T2N zhg!k|(<+*A674P&tq4fRr)%8xAg{JzGNR8=u;0$rW!=uzWaWVMx znb|Iyb-dj2iEuG*aaW^bevW;!*j&UaOPWh6igsA4i|FH;zxkxIGs$Jwq%Ay8!z@F; zFr*}IOxjr<(jbWsj?(vX&p8w0P53K4ti)r2$DPWMH+I313Oi$e(swFcb zrtoUgMFAJIuIe4YStA6}Zs>9<bfl-YEFxlBBo`8l=`#Qk~83 zq~J7K&}_Gj-n8W{x9_ePLdtZ);a{3_fR98lFQs1^?xJXHToQYH`2+73w`)dBUFHreK~j@|mzeZ1_rn%xHE&ErSgw^7Xc zLsU_{_>-Jc;Jw5j$b3II%^U6}o)svTdSp4_4KEJSVPs!Mr{UGxxXaDVy+RcOBqvGA z(5L-l_~rkN=f4kJBc#Z=480%0N0s+(mRl3>i!)OvLnPrGN%nBw0^bDUqo{5_8O=#&hwwA7Awt&~OgYieVBesk}7RSgxD=l(CH| zG69@*lI$y-sPgu@mNRy0J~!@nioKtS>0MB|KIynPcu1uNUTCR9)o-b6_}C0u74D$` z@B|7(dI1j&N#$^saIq*}MG?7TbnaBu#u^sVlRvj$R;+|6Dwy5ot%mIsd3fs$nh$lXmDO6k$Q*Ut_BZyhA{1BG>4?{$}B@IN(=A3EYP! zB90;bBxQjR*ZKo+V20cj2VjhQju}@4GT}AjjI9&CF6)jt8xvh;4pEIveijr z9sUQ4jQ4>`IVc$GHDLC+`Mx1G8^_tT<6`TGCB8&m}F zbeDY|zk8Q1hP~qFHtFYRqBa%i`UCY-otOi9q6}5-UCEM-k2!T;+AZK$k;@sFq^TNi^LrQ-H{elhmzU?o4qOh-f=zjHBfJs3o|Y! z?Q21NzPtRCpa`iw_Ij?Ah~>tsGkh~~M3MY7o-bC2at@mRUV;LE5fX`Nu0P`UcL(Ez zWw)mrAb}Foj{?w$b~~-17k4PeYdfw12|mOQqfQ^5ve%6^g32jeK~1bp{rGv@`^gu_ za43831!*_Cb=NDF*&5OcGypx9F?Duqk2LZFrbcn}Mp3`-=nP)W&tN~BKlgAjN*kJ) zG58QTD)|Y0J&BeesLgnY=N3=1xg>d3os^x&{$nqj@}r}n;q1dQ(FT>Nb8| zDh4%QsIfL!LG_#;&VjCNaa9L-SuY+B7@|oFEL@cga$?ED+vI4C^VnRLZj0|l6NON_ zD$g508DmRcuypddlx*v9ZM&r!-D0UyL0*I`-8Tz<&-e)z5I&{6dXrpdp=XOrfv@j6 z^K=sk#nt#?^;TV30aSY zW;u+A$_Y3g!bV#D7O<87tr5!Oa$|<2YK*j2>c%P+3$%y9G&Ba|u9YFxo`kA;mKkE5 zo>cpWt(3otKP(}m7Ap6#T|dnNk!w`}HOcQk57FApGC%w%0TfRKS87hw8C&O8qjt^k z3mXFi_e3n4ZdWl}0;rdc)$yoHTks^~9?a!Sc57|6GPki=uP=0`80Qzg?2Gwe2tbHd z1}_$YS%r}p`$ZjJktHuXl2Y&(e@qfCJhu|AL^FqQI^qO>P z2pS58-z0om^jaL|IlIKtsdO%al5H2zW4T0-63MH2Ef${dk^<6Swb+8p*B@)8GENOA zt1Y&mOhw$aPh@CJVDkodbwjtsPB=qenm55trL-bh&Ynr6@gHiD)D$6+URLtUp>kF0 z2U;{)v6fgUJNi3!tk-~9&04|Y0Cgh@#M8nw|1Gg6aK!&%a`FIwdcFJ z`3@a(NPW(_eF22|atB@j2c*RPx3>_;jOOx*5_5y|%YD>iSYKYByWctf z*nYWEGBd`iTdxiH(2|Ch;5wStDD+;Y7(2R{ffbR2fb{d3=8V zh`&ZdjBBCl@hyj*x$~lcpuHTjTiuUf#}p$I$&I2s(K!3ti^Mhu)x-APzy5gI87&jb z{7{dsL_oc z_x)xnQ*mU6gIvR{od^pvod56$~~{UxCl&quQ0%if9zgh}}|aPkUBI?5Sa* zUIJX>2jbj7^JPy2!>W^QTHKI#G60OD^CN%a(diD1nV*1fkXaKoOg4{bDTNf5HLIXP zE}Stl%Q}qNfAh(O^GeYlhyH0uOR38jmo8smj?#v068qu^4`v^TVUsYb%K=@*K@v`t z>2gYMjaPDugVKGX-gS+Jy)MuxLCma1x6&iFCp3{QF+QLaey(QL{}#SvJ@9R{`Iv<1 zEq(1e&YoM#{C6~{^UM`|%Yhyl9(q(&>e3b9VKoc52DJ^@61m+uTEsn2`;vqy;n$1_ zy9|a)j&QJS+IFwBOR-pem$1NfSZ1lHCvelsDq}gQvdo=0<})$I;^`VJGy<~Qq2Z^I zIvHMRvOhQP=Ql7qs@>41!(&Cx+)||xYcn#Ic&}mV1JTtrSy#8&;ss(2TwARK%z-l8 zbjmUBJr}4lJL%f9M&Ozs3?XMiPrg8Ml-&i=L9dH=1{A}ad7)ztQfbm=+=jTI=Wb>| zk=$BUcU@2l?|QQi^2;$g)y$;;bwc`U$;o=-F={gX^i$H$lAm}1Z*(x!@3&daju;-; zee{XE)ii_TVeCb_{d`<=%Tlj>Mq^on4%M+DvZUSQ;Ugd(;{?lraRxCg*&KSi{ta#F;?=NBM8RUmmDFK3IGJau&G;Hh!LXY8m$N;`}WHVfKCN!nSBtV%u=6Qf|9^H z!0h54@dNj0+Dx|^IIP8Dy1~Ckj^dk)5#1&TP>+M$^5W?I-}Xl}MPf1*R9D|utK_K6 zh>RcJiq8oE+Ay>{t0u_OtmHYhqp+OIRq9-Ugvp!rmruAgU57=qT_nsTq#LXhWSyZX zRWrn{nHYMa&&x_CQ>p>eT~KbP1(BX-%ZK=tjlZBa6u{;jx7d6wXDzkchB$rRo)zJC zl1OJPa>CynMJA2se#~ED`2v=Oi>gC#i+*=o9#~blG3ykwKHbOPxt>Qw@cYc3{qLah zKZ%qUe4?4@U+_+=rK|yz%%3L|mC1i%%X>5OQp)7RtObGoQJEX~WKs}UxRh-l&p|_) z2;%7n0#TEgSgZAu4iw%vT&Ow<5GU9KR{y#AU=2{o)VIs)u8u+qZ+3V zP)UP2Sy(&viPqzo>yBc-$ksU5nOQTBS(Kl&sEoW8b&@vX4+fB;#gyU6nCo9guH{Y` zYaA#fj5O|WiSKaUh;U-PI+>`M8TRblSJrt_$_h--5>?KWJ8*?)E{aU8D`X$H<`n?Z zK-ZDphv;&#U#xcMpd@Ncx%+WuAqD&fRj! zRQj&oBSNWpy6Mebq+mHN(Y2gR-DI+GJcVK>JqL&w5BHOXTQPPDwG%mWPsV;PElTtl zqLUaEX(@UQHx<#5J>~rOKye75AyTv(tHH%_AGaOx6dIvQZhM>?ni+#UL!a0^Y!>U<$XK zIVd86hIiHeeaO8EqFSJ7I&&16bsg>pD=C)j`6KSXA^h3M&&kTHDL20M%74xFPm($L zCcU@YZIe%72KlNkKJcO=AMLb;uv#I`so4?OXfhGswt3Y^0 zXWk;D;i8skX;Xp_Z}vp41wnG(p`kX*dc@)Nv{H@REPW5ly9Fs<&~hu2-+!7~pwlX# zdWQeqW_7}A#h=gv4%Z;o70lyQe%_<~(dC910L*boC37~LqfZ+gZR4IKTUFNVG+iCo zX1-fLm((``oX3nYH}Ts&iOhh|dBUIY#l+WX21!ntPq3Ozs=A3nTawX%)X>8;Vd-+KA*I5mW)kWk=7zHWCW$eZ*h~Ie@f!dEO;h&4dcG* zZgJvKidr+sXrF{yu7vJ>=w<2M;>P9;GZNL%TKIk@naZf31%7C5yE2V`_3?EX3s9Tm zD4poLzcoS`!FXv0NRTU$`>2rrCCmjk8DC;7lHZPgfUuT~$AoK4`{+ zcX{MnSJGxv47sjg#5v)=C4Y}<$&jeOg+g8Ds*^@wef|D+pTg+G;rAK_03Qa#FhNB!`hFm5;3p_F^Y`-Zjcd%TcL$Q@9ItxoF7 z=n0CNPGvr+x%v9apOTjxom^Y|E?iqnVRu@d z+k{AtvCuxabAyY-`0lsF>pM#iOg_j$+O=G&rZb9@8HJbSXWzEoyjQi;Gu8&|8ep~? zB|;YT7PI)Xv<+x@LJ_^$DSFx=;+u)A{CWc?8QTvE%8+OOILf zfVz!=pHM&W)&RRh!b^nu=|9vAIgH$FY zLf49fD~h-QK0@SsVjdSMsaWMCtsa9Go0a1lXHG1ABdOfBe(=lM+BpN9|HM7>8^+Xm?%5Z&(kxhvp<;t|_)szS@e;9+ zhBb%Gm!qMcfH>SO2gpJxwMdayctFu!fO;sHxqcYq*~_piP%g;cH{JpQ&zl&r!v+xg#|YZ-$69L%7`B}0@$6Q1}t zb{(wEO>f{hnu}R+Jgz^u9Au4O^6Y0x-S?n=kh zWknscj3eng+w-22MEza%Lw-I}Z_EY_LP5Vm6R6qFIZp7i=EbT6A zR-3Z{n6)pvt2_T$e}(851Xtu{9g%B6nG1>nN(v?oL|mWj#z%ZLee9OFOeYI>2AcV> zWx5-tb4ftu*Sf)Gvl7b^oKm!vx^_N#{Nxquh{n3x9Z1>SG4{6~$;IwSX4uv7+VF#l zc?jAeGeagcapfjCw7`{zm!7vV-hf5;Z-l)W)~G2>0$RM%2}#ze-X}{nx$X;O#WNH^*AvrK&oi!{bqc-F_bEhi_@j#M2sGs} zZW$sxl0R>~6f@&Mkka?CIpTAMQ+q}fxi$kOhPn;U;xz5xE8(@bm5>^|)lr~ixU-{I z=x-QH9Nf-UW-f2gvIzYU~;@hw&Lu+u0y$f-uytX z@;&$!;B+o!?x)PFG+ozll)2gQi}5LV{HTRoS329D z|Gy3YJ1~`6h$P1{Uw#o~UyuC$Ved0>yYwH;;`ML=P{SUk;f!tGwS%4GQw5AD__cOj1+6C$faJy zF06bc)d8M0rL1~psuZcqIz!)eAuxc|0~HN>ONc2wqKL(~)ZwUq4lkB|d% zTRFt~%-eyOvTI^I+k3njlPmZ@J`(NpF5dc3KZ;LW;984Sl_*x@P#sZmC%LNheyNPF z$;T4a6=~Fo%BveVzh_Cs%eW*SoP&9}AXd`c$u@*f(Ci>!!mFJVq=L~2Oe(98_ClO#;oAjT`9 z$|6-e8JQ9WD)!>t%~>Vd*%~hcPr0(+EkwV_SB!9N<+T+9O1mC?r!L+e8{X>V%(oeQP-Qu!4XL!+5lk=D{z1dBlheRymQvz2 zIw0H5k9~gMtjo#eC7n~S>}Yj#fz3fxS&M^LSAwz{MG*izQvF% z94PFgCr?md!V~pft_1K3X)eb79hx3dai%PFTu|)J4>@lkDMOv|6RBx`!C@o>y%WxP zoh%zQ;m}z(&Xl^*4@43mz8e!!T7vYC0T!I}W<^b?tRohw^n1VyKdlqg6nHZe^NWW`$FqOw%vOT?%F!Dmz)3u4D!xtZ76@N)@7=S+GVn%$hok{ z-5*FUB1OPaYf>|Ydzn~XFF=YQNV!#&%96JjigKdLHTRR+H+iHADRZ9;htv{QQ4HnG zx-e?F&Wo-y3$>)lbve@ax8_>o_WV%ij`>bYn!5~VRHyqsU=1Up?b!vlLi16O)a*;_ zixBbjj*R2&K|B?(PKUL=DM5Suj6(-hIN>V490Kk7HH5$ZdYocg# zK`4o`ZAWgGM&daUSmh+YDlvuSsX5?ALg@+qViqK+0oKZIa zW?ugrc7amS56(ApJcbm=!J7@;5BGl_qxXcgjhXiX0L;VM(WuV#v+-1!vE+fp=x6gb zbk`Ck7CExNy(*@e>EiEqs0Mqv$-d&%`Q;Ro+t0^twpBvkFX7&wNn?Vcv|@}}$r#z; z;<8@Bz6oNUF?KhUfb96m&306uvRU}<9&dL3@<6LjH0Gz~76UV36_P)p_e8n&5z{8i zl6xPfrEr@XF5o1cwsz`7cruc4>=5UHgadmq0%AO-sdSm(`6Ad6V}-Udz8E4&Jzii013&}6j%2~ z$AbJN`_$uj#Ok;VY^kj)%mdKUcu$Oa2P{t3p0>1lTAQ zm1t#V^sseqdldhs=Hpd*@A9x6A|*7LMxEL?ljdz93**(HXo9D|&~yEqUWZyI)^M-D z)gryO@N_S_>3kYMy0V+yUXO*$b_rXh)##i+iT~3!zXM~$!B2zve)8uWEj=jtL3AcO z)cOiMoXGVI?bzTD>n&M3D|>P;%=c%`rAc-8%*oN{waf+z;zsgyvn7p?Yx0!3w@9LG zU(VCt_E=>+Ra@?9pNiIjw&7(@!(oFm9ktOmQF8rR%&4%+VmrPz6~R!+hLPc^eIPw1 zG9=i=Q!R&(w>SVvsyaEOg%e~4EI6#t;1w8PB9`{BM)`ZGr0acSW1H*SrrffRGk8S- z(N;$QX$$|fGo9v zt$cZ60h?0t=F~clDe<(-$PrWn6OaMl1OVcX*E=l>7MN(f`EkJ|lC_jNK^fh3Wr5|z z!7gNXq*Fjt1QA5{X!!`^;Ljj-zNy?cSwgvUWo7RFT}D`jR3n4Qb&FjiYWNz6OhCF- zRD4|8clIc+HpV^pZBVD3m96#@v08SF=_2JEPNxU=b@h7UU+s|}nu3E13^biGGg1ML zFzfG%^uX z$A*+!t}gvP1C0BVAj+?7>`$|Ny|$RYMQxS7X}w70dr_v_*Ni$lj8;$A|7!q7!}e4X zc6>B7I1=f_XHUbIm}lo1kcjCerWnQo&=O6#mMhk{49Q5wzyn@LR&y{QnG|JJiI{2j z8wF&*+4}gQXF}W1@wE*8lW;e+Anaeb^l^4|tf2>M_vWqE-o7S(BA%D-m7KB`nAf=n zqTNBW7EL(9%zw`l|Fb=JF9e=(!>}8|G@8F*-ak(}LFiY(y#-0P9X1Rqcy+mKw6nO1#?`4QAs&im0HxvhO-OPEy6fR@Cd=DxV|U1} z{YtnIluT$Yz~B(S-YzjvFwAW|H12$rVXI!nGR~J`?QyeT91&PucJE-Yi1rVn2p5ED`Yk+b(6^^*T zb9bislNp>N`YA8s-0X`H0(P=DuW;z+@bo!h%e2nK+sG5-mce}*M=jf*C6xqx_bQ_; z{_&2xdYK#SUKAH5UUtG{pw#-sh;*Uh(fAJ~OsXyNfaHmVbLL<42q5*}d_xv~%o@Y5 zcU%1R@XWZ8W8q|L>_r6L-95zuxXVq%+pqkL&P?`6(**sz_->Ef@wJ*f46v^fZ zGsK!{97?v0wj*Y~(O{KQTa-=8a3VuV>Ke41TB#~tTfwGXY*4B#X4Mva;{yoQHZV8{ z;R4;0O}O~$eqg88W>fLCT@W?H6f~O1rMU*Mgo^qf&t?I^t?}0qN2N2_NgGhDDg}T_ zWC7PMY)R1wFC&$pzZ`e`Y*4QEq#e`WEh-Fv*oNYlrMXVEJOh@l03y6WFy4{#}3flwXUN!|_hqm|CQJ)YCdKnggP|Eb8q$$&}y|AkteQ^u6oc^Z%J{{-P}XO9ECewznEU|JZ3iH9}}5|Mu7( zw99TKql3BtYnDzOlyv(%eRcP_%5?G4!<8u0hYbHTUEbznn2M=N=D9(hXP&1>mM7ls z2o$ODP$5cTu~jwDN~j^c=ZockumFtpAl-IVItr_{Hm)mQ*4a4wZ{jkNGzn{A$~7bj ze4@(D0c_MVL>*|w z8>*Nu3{P`~3sp9PfJoN^xRm;pjKOt)H(i?dZ#3-bo8~`^tbk$a*Hx>5C^lc7@e0!< zi798%&-%qw+kOinM^nc^UwSCN!}OyS*YPIYT9Z#KxwG!9vg^I{4AE;*kvvo<`Rp3^ z>Mn34R9kISSqk`hdeNLb&3_?`%^{YZa@mG&`yKqJoBtoE%dcjjhgh~8%`%G()A?LF zhgu#SqK%>shAG^v8DmrCKK3V<6{a<1 zfO-{kM+AcV5`SA}*ui8Y<^6#Nc6KyLV%mwGJ9wh`wH9=5zBHs6z^8^u;9!MA$3LR! z%k>IX=C91mrxR7K2a22riFLTNhk{XrohI_#$C7_ceE+N)ht3wW0isLA$J3+=VCZ-C`VY-s8IOrVou) zWNWS6V#SH(#U5Tc&n1yUkxm-+vTy*5Q!dP{<)et5alc&r`Ts6S{)0B!qe?dX+qdkc zEX$dkM!S>Z4ENNV@HaY$nJR*{aRR)kj{L9XA-1QXI*gzKW^O{F5qG2xaJPgEDO}-V z@5g9a`t@fxZbMPV1H4rEuKgfIKv3#$oiVwUklR$q{ehBr56OXI95D&K(hL45MoyW9 zS`|JC=@)CWW3A+`dG^83zSbWobj-6&icv}BII_&q*UR1|fG$iFv(~quk7J7n zhSBvP<5|T;ws(R`B|t{kM?lek;rIoB-R37YKof26!gy7yfJ{98xhGBRpE4QRQUH-m zi8}|96SGTUx&{e3ZBefvKdzULR)v=B=_kR4rNp^_wtR+OW81dxtGPirx-2?u&4gJ+ z)f{X>?WCT(dn;+;EjNXPR|ZwzxtTx;8vBr?s5az01Q@MtU%*$9 z+4OZvD@k9TaKBjJ$4%Ax#nx~s7}aMqgEJrbitg# zjdtc&pwJ@l7f6E2o%6TgOVdUevy4`QFqvFc$g#SgnCZ$-D_%r}DV_hqt~elVtz|Os zy7FRU=+oO=YR@3Rb6d1J7ZfPixK=RaXH|E&oGm&>FFycqSf2swMe3>FfDq6cmRmg3}EcM=tpBstM2p`Fs)pRR#he4*0bLm zk4REH!2?7+vgM-|;2Rm52^*Aqj=qDTxp*^KZ8ui$W|IMk;tfR--NJZ?epalrz(uhE zsqodP!g@16W{%4ccT(9sml$0i)SK1ly`|A>|DN^-IJZd1agKhgMoVj5ZrcPB82<=n z4crpJz8(0E|6fY_yL~77K zM2S=UeSZIuRJvzZxRJMvCq>{OXm&-O8p&)?NyasCYS4&%3tSuxf_&^|0@uk>^AY?K zH;KY5JQYSBYffm_;Ko zfLO=1bfPzwKE-8Qtn7gEBky>b-h)N)@~e>X<6;JWBhoh5{=!e@?E$$fe-;s2wLKP& z!EZi?-($1Nkn90RH^<8jxSw9V2L$P3k*kq@@_)9c(i*)sQ>}xrzuy)Y#=ns#-)dnG z05&L|tJ!bKY~~AXx;}WGK~^I(Y?@1C4ki9<`+Jc6S~w=K`-0HBZucyFry~Y>p^IT~ zd0w1ChRk}`o~9P1`te&q_H;DJ~Yxur?hk^AT8Z+4&B`#rHGVBgVIQMcl|c^`+47c@B16${Be}w z@L}&~uf5h>bItiSrSgd~(l@)T$U6lwn?7Ft?2p27MVjS`xR2O>RJe;4uYmcX$}iSH zR4f0{tjBW_ls{Z+?b&!)$#$)e;B$xTI$#Pk%x?t5KWQDC%D{Ciq+m~n6SPUI&!u0z z+diDzsR>o^8MY$<|FzGYe#EYR!(P8$;w)b+nbWtGzh_S2RrYkyl2!Jd(1&m?DymY( z%84~iT9uoxU#4t6_^k&!@@7R+Y89;L8B>o~M)ef(JW)$uBzOP21@|8=jx`J!pmp%^ zJopQwyq`W*?I;R)_auwapCz>NkZOaauI$+WjJEWbn*k^6N_l~@zx`;!Y1zaSCY>Wx ze`2?N=FA0LfjU8&OIb9pp6d)RQs_u&7vBj^$svvfAf2d+=_tZ*5fvhwN@YFmvMj!j z^z1j?99oOz$!EKwYvzbnl$^5l*4MP90zBCzy3p4SfYOelEU zMMFQdlk@XiGH0$CxAXg?-jP5EDt)-XqAfpDEJ)?@WV_?^^U0l!NKbRu%02HM_&Lx3!gYM`o#*kiT8`zy83(bV;~ zFxyp>lOytDg^wiV^>w}&+P}nT*yNi@qeSnJ}^Gu56mrng5W7mO5?6j16@lUprc9 z*DSa_ETgn?r;XCc~lR%6!ZAH9&Klmx*BrO3l=;x z?Z|40r(=w+L#_g&7*r?-SE*JB+sPVi13s_sCkSkI9D-cLFb!Z-an3 zF~|&X^zCMvc!W-V8GqS@q&wlH^PLe`z1|r#6wbf%>74Z}$D*Wg?0))%QVA38fpx97 zJogYvzaHx3CUl#6Lq0R)DFUQAp4?4Ni7TXVW9vJf8*+=WZnu81%}zPscK~=toMA^T zU+RzF(tAJUisJ$cW+Z zDce^gHT5V$eUD3g{(R6H487JG8DD@=ijck)Xe|Vp#Y5(K93maA0NqzE&Njd8u(y@1 z`q5(t6;*FOLE!hjWc!d7!U=xCaCV%?oAIQT_Ad{<_{ce6%JyBWp10jA9lQnV+!QYc_@QWg^CXvO!SO zlQnDJR$Oo&Zx?>qHFvcZb3JXMHNH3OcO+C|tEOq9iX4~jb; z?lT5cVQLFe{T~uzP8efFdTLi=in)!sr=%DA(mFAP7p$q*>y|f!i!6EfdR+9RkV{K! z=KH>to?|K!LHYaQ%H0!ZKm7mGu&7*6l9)2-LSbPnZJMl?*cnJjNJ^-`MazB=*vjzZ zs=Wa>2Rc)-X3~`1oVOG)2^+IVjMhskP!6%i1u$Dyd`CeTPN0ttest;DwItOe+3R@s zMkiJwx1XUp{L2P}jA{sl?T0~h52SDJ+&kmIz3ZT^1;tbjxz+WXDa=YXu9B?GhTv^} zPo9X5mLyD-oz-AZCKjD`fsk5S_q>)aW3buXiR@*ag&jS#tl>|nAuPM0q!=JXm`a^4 zX`VZc+$P;oNQ%hANJPlpd&-rL5@g3qjDY!T3>10q99US-cH1Glx#_7piUh6G2WO?D z*VAQXWtJmseA#CCJoyX$L4mgUK(F|&2;6o$^5OOz#{9DRS#2&5s&%!>Kwnq+_U76W%q-%iW(dyVcTvhv{Y*-CATuOEnSOC+ zF-!(HD|?q;AWvi$;hs+esP8GZ`drGPa4*Uiu01}nbECfRHm_Lp+1*yOQJ4}Ui$m$7 zCDwG9a>Knbn+3I>2EszfPeLi1;eO8?nrLXPK?2$Hq7*;}qm54HlK_+{08D`|t^x5Z zni!_nZ(Gk-!gme8wA<;%?!)MxBYrB*X_mR9*%f0bD$ngo+E^|_FDG^ea*(YArTUK% zXcj_#z5V@j9r@F1Yo&KBlYc>B<9XnKl`7Q6d-n{#!3r>qi|@vOK5f`#j?NE6|Hw*; z#&6bq#n~cciqmr51L5|$a*Icq&~>^$<6(V8`EbLAVMj(aOX;O+U13jLOccgy?t8NS z$Bny5l|~~jTlCMJR^&kqKr%MtqL~GQYL(VnZx6-1C*$1E-R1$c4L7hrv}K@LsWHJW ztuKm;oqHtWvVJak_k?~EL-!Z~c@agMGllz7tg*MOlPzW16WconeH&lR6@7ELdCya2 zQ2!GTps()9_L?DQ{01x8VVOxcmqco8xy~T`&1^K|Y40I&jo}=fd&VdQ%6Jf!kK6O) zXXB)EULy5~nIN-Py_x}n2XU1^B-?iHa_e`sCBpjzd+u^EQuqUFw#E3*Kb>KPc5|(- zRy3a70BAlICK&Qgz}?l#=M4L)p%|}vxqgeqkEG{E^S^>D+r&=t>xB`L+g9oE&!PMn z&0ns8ILbL9^cBVks1hEqLH*gv>Q22Z|*_dCSx>K%AhH$ijtp#uLMlbMeg-jlhed1(Q3Gt>%I=$j4;-%Cf!2Ec}8gmvnI z;g@R?lP!)eR%(}znGIUKO7P^aUFYm4=DAR`egnsWZ2t!WjEiY+{XGY>v=Bp;$7D(% z7gRl+jWJ{{t$A`uDDV|7IWQ;mac?g|kc`Jl9H?Pu%hvK`GpDL&_7~s?@$J%ISbjg{ zUR*BxqOajaU5$g6;dRyL<#F~UXuYykjH@bD_Yr5SQ_ryfwEkxPmh0=&9GCdtVw0O8 zJfk19b*adda+xk6JR>5ea8;FC9|PIMM@@5)H{MHqET4G}+5YFyHKPP(fSNY20w?-8 z+9h=e?G-8u`PmW*&YD#05Y1GlAM;lKLrqQwMSa2OaBt#wvtCY08KK7*$=5v4TH(8MXH){V=E1pJ^#n z5)+IBYmQ8d@l;&qH}Z^-H7iV|52$+}@40zUfj#&B&!^sa7D&RGm}eZd*UEL}n?eb} zhevxIcq6ees0@|Y-m9&~&!dHk5vja+a0NWkbiC?A+mTN(!D z%BPdqSe!Wg(bAeHTv^}jhYL@G$CxawXkO54?bxYj>lzon5ub4;M4gY zCwZu(MSibkBkyVbVENQ_e3J6VYhK`!;EB?GQO`M4a=e;n7N^yY(9M5=h<(VR{gK1+ z(^lW0blEFiPtDtdC$=l`f?c({vDON|6MoF@C%@F*Ww+AQIQ5P>>@9frCbp~ozZ8)F z2O96;5H*7zOo}r+jkVuRQI7ML+ugU>Mn9z46RL z^gs%!RHyy{{73%_)_GKjcJHX=t8~DV?mM!~OnH(`FURzTst_bS!R!t5AC&$qAFe1j zQoPFmQR+kM&4smgtC#yj=dUYEufu?hDJxeh?xEaNwk$59hW>yxm#B8K8Zn=r1o)Kb zZvKa22RLyk=x48@pd2UpB==$1PsN7^dG=^ie~>FM5Pqktmeoaek~0f4yQRXo$9{&k z7jSM-1v43s$8Fxf+K+vsJSku)TO;u*5EtrTDSqGU>#yN?YRZlEQ$t@e$5a2Kj+osr z1$0j%V5{yBK;R9;Z;9CE{%C%vBTMH-QnT7+?qBQu6f0nLu~KVd;8hWckO5KbVn&rBMX|-ey_cqgIKwu4N8p^6Ip& zYqbo92prgfP!xXl@D!$j1OGHN3kc8?|t zxG;}_QaA~}$~MYHbt@{Fn|<$nZ0&FP>1eY^Mmc`IU#6)kEon&3!x<^8_Y2sfgPb&u zuR%u+%~OC?w;`(a`b6gf5JN0`tUbe2m|S6ewm?UyMRgX6&ZrQMh$Ex-HzOf53JwMg!X`f5#>9;H!ye1auQ!6Q6AcVvS@OH! zz??N8{}fN?NNEy?zgYO+=VcTKl4Gh0i(5qPm8pqX9Ahzz}@JI%^XXF{HljJ}C_?Mkmv$;q9%D#I0 zr4MPcq5kY1<4o5B&cF;!Fzbf8-TU=vH5f^*Gkw$gk6E|>u&x=w$$fwEUckOA5a~%t zmzSO=6%)#3rcLn$;-Zg3t?iZno~a)KiKw$ZzpZ{YrS`tz@_*3jpd0vHA8Tq?7$V<^y@D;0QZLrHe}nvN;Z`Phm5lGL zt_}O~=4hrth{Z`7|H2dGjiL+=cWIy(@DfkzG`u*LQ)n zhpGKEvca&D;ZocTyYZxIAwy_jocLLNjg&oAkoCREeaH3ZzvP)?8T^k@wg}nIL=oC& z`L>v;$`&6chGlV#5NuB@4W)3IO>Ws!hbo)O9^6Ojc;4_C8Vid?vd56CLUfyuf<%M-b)Xl;FiYwK|~eEodcC@ zrELYi2y|Ic_zQW17S}7%&p->&!-aP!l-RkeJNjt!Y~LA}AP?WE(t!5V%?6Y|Haz%d z+SQ;5lrx2z<@%%|5#-tOvo#?3^{Tb1%w6@l+Lvm&_;Y}b_8%@lFN@C#Zu>~|xU&z-wX!2>pLZ(OQ&gRAnK6_t`5waYCfjdE zyynK*wz{$i5vw116K-5mO3(Pvr__$O77b^_EDx9#AJiG6oa`6&aGfyV;qz>DzhE4ut22o}K zZUcwlY8XdrhTgm1^c@(S=@0mq9)i+K(EH|ml^f7SkmB~08&&8%O3Yyw_WMRn1)i>x zj(u$Y^#DU8@}qz9c(qw3)E!%*=ZV`q>?S_ci*t;oP%_HIDel*}3cB6!g_|}xK zD0vZEyk+zQu31@kA4qEr9>9`Q*dcSjz#35F$R;1466!L`kpEhlZ!6wPdB;Jo|B$6* zFs{(w_CD)ktDZ=hZDTv`K**bTITp%%Ts4bT|91q#761`iZxD$lY46#heN-`oL=S92 zYF8ba+NrjhR=$x+G4=(|>1|q~mzUjSPXx0p!e*DawLVOO{Yc(NSYB7_`rMe4%w=~r-Q#qJqWN9NeZE0J z4SLcVDSTz`bAxaGE zZRQfXnD;;0VSbE34zl{8jh56i*73C4HY>dRzt|R_T{w6GI2u^L)!v+onY6s|9yvh_ zU&Pgk!&!yaKENI~GGzB6Aq)>9Dvj5ZR{Bx(z^yq*o=%mZnc+tqjB#(5N;sfS$OK}Z z@F_PZ*^GleL3+{!`C;GV$B%Q7Q2`L28%Md4yPmpQ&Q@p^=$pisLG~%<-jlEsZ6wG- zHtvSyY5;$?JY9Q6PDhc@7Q-tE>|NbtamR2$=p$AzaX2J%LA;WHu~4;*c*=`v13=8P zkBHUG^R0YA+^zoqxzmYi?#i3BgB%n_1q_^e{AA7$)%c2b^(;I|H{ncmBs307q)~8p z!1_~ovKbUo2H#^+Hjq^WS8DLDQEOK^ntoThJX#k6R3Y25B?}~$V2dH&+o4V&&tV|_ z+Hn#Kf#m_^^t6*AWq+uC!!tLEDD$6b{K=;4;&``g28sqSxcuDp-0hHr>#WR(=6h%Lhney~S>UI%ZYVNwIQv!8H7JcKBTO&kSUd zNdotZ{#;C+J(eZeYOg4-+-V6K9S`@3Q%&iq&iMxG!ZA{&cv?`IfxIkv0fUaMp8G+v7U^ zcdzYr;$2xUqKQBz66PU&ph{Q*>yDF2D?%rsGij$oF3>QC%jClyFzzT-YPD8lf5Z%v zYB$o2e+uy#M;5%?X`t;^>P-^v#V$i3@4n$b8Wa<2A8GaWBpWV%R9fHhV1bF$bpMa| z!(^UgMgG7%7=p6qu17DaB@8{z!{x~7nscN5krJBn#Qky`%*1wKl1d5fvZXOp z78k!`8N(%Nz6Dq4pau$cdwGowzHNo>r!VTv3aTFRpN33fC#6gy=3sQWvL_=Tk>kFN z#hVZb&z}XaZrNLEpy)a&bwKWV5~Oq_q}FV!J1FrfH_JR(TAq|r&;>%^EE9Q(WHobD z5Mnmm1Qi=sh`%an`8S^2+w|9I!RZ30<(!MftLiICc&2`fQIQ1lCKA+q#Xqtj|e&yDAnt?MCdj3(GRJtr&1-GN&etHNu=pB) zT?1sIhmbL!R5Wo!z)ZpXSVi}6{df3L^mcuIDrm(_?@gJDShoDGziayDpg2-7`Uo-k zS)kKbtbV|=$VR80+l7JJ#U7B2x+BpK;0Wm3)N`U1Jij@o! zpa-##@!GtOzTYSn@Ar%`Hb^MNYyeKdZSB5Q=`7XeQ`6$d3L1*}PCxnc&t4m?*r6*i z-v3o^XRzUVee7(DfWD$itg#e36+Doxup>ID|61Ac;3epX9a?}61`}Bq^VtzS{uuws zuU{%Y@hs|D9Qxkzyi)waaU*h!ky!Wd7%}!fi&Ehk-=!$Ea(*+Hf(9B6*$aEaL5ye2 zkAuGFSMB3F6QK5TlC=N7rC0y$a8Rgv`_1zG?Al*!kTRsig%0!Qy=G67f@wWq3x>5M zaqS<$e0%eg9-6A#NP<@O%m%otTFCokYa_`|qBU*cftPlNs|Fg(zoN zqD!!1W#I?o6OkumeE1MbhWnU&LwI!?zVKt}bcf&dotf*q(^DuFl=Y?S+w$zV^jEN{G;PE)Kc z21XR+^moB@2Z8Ok!o3D)>kCoK*=G0AyD<*|xOam7?S=IQm&HR+_j@)LN72^51uu?v zmecB=!!(uM`Tf@8ah)F5-N!tkCsT(0G-Y@0Vl{s;tb#|Lu83C?7uBukoxZ{q-F)&K zCax~;JslZ-{$moYXn2ZZ1_k+Hff339_Pbg66&AKk!NO(}H@NdV6<#w!sAM~Lg4Z91 z&XW;ZL^F!$A#0*n%yTV#61QyI>Iq?(NALxgC3#p#phWgWC_ zM?!z14c0w-V{&I*Qd$v*(6g9SjnAP7&o%99YzyQ-Jkc=(#0m-d;eS6CO=#2^Mb0}o zC`F#GDfZj7jYAwVr?p;HIUnLy9cOFISVIn$2q#Xrs*eW1NvTFg#o-1Xh6cZKSVUP> z{OQq8->Y^N9)`SaSilfC=%&TB1MCLFrlooRLYHEVK&Z3;|0lti!<4i8*B+;NLEj_f zXd>=Tk|;&T$NSm1*4eQ*%>i1>DiF4e$$D!kyQ@Vinb2lTYgJT7iZn|5jnTpyZba=j zPRnuDjk?oWnPb*eC=lfr$Fjt(2Pw)>VLn_I(RM^Ib33Z)Jl43qN>g+#hMc z7hEI3*%l7(k+A&!`w@|&M?zN5!~U_J!VhMf+V$&D}_XPQSTBtw5HfzRXzPDya)tXGPg21>b#KZI^zYSVl^=^qiDd z9h+CKk)&c2FW=o+gC+A$+jodgwOx#ruYsbg=)4|ZE0#p5Ue6Jm;+e(6ZB7#`p|g4a zK!L^xx)ylaQwj0S2V`mwrGVuTgUk9beYAZu(;GcaJr`mMkc7)r)>FCp@sbNBt$1HK zEZEL`c7-U?H&EgxAMuz!px}0UHlsn#UpM5jIYbB(k`TW9JzxRw${N5yAj(3VkiCf7 zS&?Rw+j@P-?j0Nrg9kIS#j+-MdSk@}`BiC9=wl*3+#`rGZ-kD}GxX=X9;RLrvWT1s zX{d#1LLKz|V-gz{a`Nfx=G2a`yQH#A4v<#uxv+iF25&sVnd^*uAYxnx>a zLvj`cjDKRrR%tBh{75=#=%v3ME+jJW@bED6P`sj4ImPa4BNY4Y=6jIwa-$NIVQLdT#uE z5SSEgr6d#koNx*F09~0Sma>fTQXR;y`*!U_sQCRp@=qE#Jvf#&fI12JH2bG`ZY6c3 zgKtKBw3K`X)U?jsPlM>@IJ$Wj4m%%(4D#>d`DOsO&X-KIi@u;54w=H!12XiG?a#Zd zJQ7x7o>~JB{9a3wBy7 zIxJ@#jfyubR5|EjaWPs_wK)o+$9XIt&KRnYziK(hB2vRxwGn|&U3t)FrIYl%8zR@d z=3cc#7zX*IR5ko~L!fUAIzd_uH@=u}LDYcY*q_e*nHnKWJo-~gDVBqQhH8B+(7 zwWJBU8!H0IByED0SAWuT{|7?zNGY7Q+CwkSyz;Am3WSYo7xTI(ZagV31Rlg*3?v8& zdd;T?vaQN2F0-Du-W$#9d0)^y`kC?vqZU8m!s*q6AIjx2{3v-LNwR-+NF1YG1r@SR z-sJ4vT&uw#A{mVjyX7i7=MXO(X!i4pbgRCQRcleQ6bYhy0H?&I zWG(fVPgTX9Eb&fxykeP(yabV!cSJO-@_egk~vMs_~$!gCNTx-=SRKbT-S8b#&6Z19a^ z-tgBz+(OVO4*6vTjF))`7|jJ)Z50Lg#V>&2MC_;wh47glBSa<70{cg@n1S!rbGBB) zS9!l=dwb8$H%Cb|?j-lU!OTYID$m#_^S8wsM?0=_L*V>KS(j8|uvLB|a3Phsu0>gE zyS?Uc5P#fPdWTzU7QrQfUy6C%RmTQNY!Z+?$VT!?q-2LvJbL1ig&^~v+( zfvC~Yc$kWLhWCH(jR5R51vSc#v$~3H-^)D*f|w;pUS|bNMaIK4(sPeSSk#+E`&p=s z(4=iP_;!<;EjNt1O!^52ylv9QlZ@OntkT^kxFYAK49b;8@f~+z@bvEd8WUMzR0jDZ zKW8VV^EjWkybDyAYXbiYLVkX+NN90{YGRK@P>O!~8cGPNsX&)Qikv5fQ`u?d0{K+`x zq|~Q3>CDc0r^PtNG0kH@z>d`{{oBu2b9oQN7M1>vxK+zP-ynqh-^`)=oza3tZzk%;r*S@9cxs~R;a|$D`14~-zon4L@0y*lV$B1OMhSeofa+)S9(=bl zc(Er-<$1%?GP?}s7Xj=E;$AZREer5t=ng6_?&crIs!Y51&do5ay~&=>M!%dWJZ*#r z(>%*!0~NZ zE#;PfcT*L0p#LK_f|VGo!V&>b?X`--7((;MCs7sZZW8i#%E};LZ$w$iy*bz0M+jyU zN5j&fuh?SVLqM5IpRn|U7Pb_bpzgj(*@+`$MmwaUn@_Bq*QY&A4*t2@M&(OYrjuFI zM;=ut)1@E{tCZb!cE$w}hyP&|mjNqNERV*3nG4e~FJ{?eRIC6jlbM&S31?MH`T1$9 zt%U2}Y&~|C6S%W4ZaD?tUa^nIm{A#{|07#PdN1);y2@jM!wB;bAdQsrTdUOJ6hrWR z+r{nSVAFU_B2A=5X$`JfibzO0{HV+6CWe20sR2%Vd#;DR^|T^nS3 zXpxCl#k+k-=SN+|dr>-(%DnOM*SuXYFY8TwTAS>+ZA5WQx#tR-8Uvfeu~31*VU22! z18;4U!;Y$I>hq&V|N0%6<&aQ{s=VaN^l%w(;`aA^uB|o>Y<`6eGD&HL&>2O4G0<_@ z=zaooj)0WH!@Fd&yS!whiJz9?n?ImGTqK2s4~`fQE1SFHTfk3p5ef3)%RS3N)-aPS zb7-R?lU+~w=&$2^yQdklK12%<;DGlB_(_SrCXmnG8`VS{^rLykQc5!Y={$tjob~uG z6&A~&zWtaUGRM zz8D#rPbEYSP32E3=ZTpstG>}D7!9{}h6i=xIUQh8M+*@25P3u<#`6gogkMGbc+j+! za9-pcKZg_r~Gih*|O2L9q zB^outC?r$xUje-4-d|bdfD77acQns1jkA`hOxr{Zv4S_LSdE9# zNRi1wLs;L)YNUw3WcgsE$@q8!HG_v()agYWbR~{x^6BV{>*gNvlyJC*kg0PJ{l!m0 zw)of$e3T-`0m$gRzhjfA$ZcC-+r+dgRki(MFb9ckN9k?$o8-Ss9#H#;;6gyv6NpQF z&GdEYUcsR6TO;06AyfJdNtI5mXnAd@mgVvhslrT?>B#9(rJB}4p^ixc@o}ej)I~`_ z@#9R$F^=zBTG(rMH%D)MncS(9ZYn$&EcwXGDwwOkmV&TdILbI!c{=a1Dz!0E$bcsdivh`daU zOR1H)9LpCILhE#v@|;#!Rtm!c%ReV6r?q-`%|R=g{)z_?xg2f!6#q|z*`JddFPz+w zFnkYAhUkFIZU{R|v^GhxNSxBp$+fbzhCGeWHmDvyWUFQbB+8TZ??z?_1zvGEO~ITxR;e=!qq|*{KRcVXtM(Em{Y5hnkIxz8-T%pdQ>fq9 zc!`FtGQ0>`-pe38bRz$$FwXSx2GgnxLd;QlHiPQ zBA^n~W-f8!MiWT4jo^95<|3LUc{)UUB)%biq{V2}&9XrwuJDR6kmg%JTmE&@@j9`# z{z>iiXewITgO9iNR*!Z6`La+yloNse`nK!u6`C7#@Xw#HC4(HD_@|GONT+zMo5_3e zXW+6S!KLh(x13Xz9Ek3wxq%b^UW|U0kTGJr*c(s1T6q0vsm9V}=dI0SwFOi>J%ES(A0+jjnZa+=Q(cgM zjqIN*Ge+p=SM-%B=61<`6r+#v#uH%PVJb!LxgY7{=~8+KsAzyTQJNIzT6a-1783=N zi3Fzu)oYUdW&D?vZx)znH#=CYj4l{;kICdKwE=br5?HIDX&sfG4!y{bzOKEawCrUq z1n0XIZL@Q6E;Bl1_ZGYt)Q!GX?7;=hUVE8xf3J&DAQ}l^U5b|Py)+?KeAonv-Tzjd zN@I8ViMo)=$~MG{z}r88fjn~ZHzw9&6sEyc#m zDt!wblVXA#MBbLJ7=rn3=eq$fASD6wy_nfu+q12!l(;{!5Z=!<@*gh10=sP=S;csq zRBLB2$4o0d!e~={F;R_)#CI=*W7FetC)_LKi4yFRk4cH7c9le+0}OhI3?M*TAqB2BZ=0MLgk;d9XI~IZp>J&BktiJVaZ77LdU4w2)5z+&h!oW8EWAIdlxPVa2S= z8qHKZ0!v3U`^D%0}R0i{wT8CvZ+#B=7MuJM{r z3^Dz*HO|7NSWN5rIZ!p$5fs_bgX_*japBxdBd1Ips?arpzn_)+8S=F+=atW|_PwEh z!bc+2Pw&E`s+N>U#N#*lScj7F!AkQEr(2|yEeZTFW&N^a<}z&b2!{R?iM!!+_rn$R z%l)J4(-Do-w;P1^8kf&OqGC`Ze}KhFSvX1_#bVdGhvG705{r_^H|JF?ElrGk*bAAd zgGgkE7Y6pqf)1L)l>Ry0b$B$4boKQzHe)}#clr7RonsLW75+aT*-r!=oBCKEeId*0 z?^g-wJqf^Dig5B@YKyq@ozTnD8CSPHz+g)g{LrD3$E&iI>^hqX;F4Ym-%U!PwZc_6 zZsKtb6|G!O3H__)<(yOa!QtwNwQ<+5$K{sl%JEJ>8|~cHkU#>hw`&$KNLx2}NBTJboyHSQ7zThG4V+jYac;3{8ljk(7zVEa@x3 zXY}0qsltfy#j1_F8R~UlNtVqs{@VY>aWTTVJL;)Kc_hWlgm;sc`CGqZn#7b}z@f6+ zD&sM**#fS@2>>_&E7{?ILRHJp*`FNC6-iVbTtb>vH0w-m4Y9wmZAF zx*Z#t8O8Hu0jaRf8gaew!9{&c6XL$wo>&M(S-}p>Ybd1fMy%YPwkfkXeGexcWm{w| z%!zrdyu!2@@(FM%d)tycC^lQEGtTbukSMu(Wl3j0yE_}B^zDwpzVE~c-njUQ9MV#o zOIBrAVH<@ajIUm*$$h$ro_p%fQ*ZL1fA{V8;D2r~nLoI*jFtE%opBkv7`$FV?YYBT zB%>4fSq)!f#ffbdjJmSwT%7J2P}>6(dpLq`RyvUTQLQD4a0*YDh7vwZ=WQeX^&-_R zBLn36zD(`o!UbX%?cvv=9Bh6ng10C2Z9_odhoS}Nb9o!|+A&taVrFM1cGOGA1SJ=h4e2+Itqif4Gp5?IxHm~v8Pjj!^AX=a*T{ATn=fxVxX)mxS)i^%^B z`ga9I#H$&L5wQK_C(H-kY>3xH6(uRpvjq>Bh_8pM%?Ct}M!*=S0!(lptrjSkd<=$W zTg9svthT=otYRstK=B5Hah*#qjtxKldfj0EhCrd_o2J1~)#1tzL0ZaBSwhG`hz?eW zfcI*meCT1qE} zFf|`!Wx6e<-UiSxy}psZ*j3YZEvzlrjIl;5y#dGd^&Xrf1@ujnwsZZ2!0%{@%e z#ZfE4RF~JeGkpDH*IU_~$rWMjiJonR8!IbLO1X9i?^Ec$$k)*(H$g9VG-{_mb){9h zkkibx?XGdlW95duq~_}%#a!v|s0nPRgCB&a#*y2Y+9Tq+5QA}#70$|-1c|tuL);Li z1j2qMW?32p8gwbF@t*R3vcUg)YujHTq4XjbW`^bP)d3_=Dea)+bLB2SxhzK4K~QH{ zcfO-Iaqh)A@UtEQ7PGJ0`AvJ{=qL;v9kg0s?~USQfa@6XLdc-%%U~@_3y?|*%M-oO zVMze%joAcRm+OFJpS{1>^+tUtCs(s|84b0Ce@s0B&E*+y46S}t?jSR ze3g5>&G+(H+n{IjsM;=L>S0SGqvdUR{vYpUY8rU+koF}#H!c7i2InD(M?*lfU4N8F zioxOXBDMif`05!@8Ys_G4}O~AD_JccbNHt!EAz}{_G;Gd*ZZ0; z8sw=e5Xn({1XY^wPx_+fEa|0KDIb~N$lI4_`<>xTk_GPQ;_{dJih_pagT)`8)3-^H zasq$&*~V3Y)L}sfUomB0)?B&^8IFq;8Y`UNSU0XD48ot!ZmAmn8A9&9XS9OEXhW_7 zWfu>Jara|OPK(pqaY%>Zc@)7*myfFoUlN7}{PCbkZ-L@NPVb#;V2F=w*}d~`b8tR8 zAda}6djuHGLxY)Ov8QyJMUhXwIQ@9u^$?eB^G|h-eexC+fjD-Oj#KxgCLn5gFva^7jIIr|ef!XjSd zj(~C_rixm|wXNAwpxy%4OCzxL55R6&bJiowXhF?ORsOJR9mV;Ix_`zWtQnrUUwGHfiyf~J+V zkT&ZBUwdgVP1yi=q2tk?AB0Su_Hx)YCJ7jeN7s9EKz8yqp*&sB)kW7j$4Ln@yPOPr z>VKV_Ywg`Exo&nBDh0jjl7+D~@We78-S(9Rs$ zSOxD$uVJR^*B_Vk_^)-ryS`ppZ^iqktZ9Q;;9@QNk;IYxD51fun)^%>Ahlf@2!K{H zVJ4B2gZL=eOlcNJvd@#jdHs68MvL4A(a1a(;4ek^lc3+NlWpl7Q{;jKV6rYkW zE^@sqAM}w#>4Nav^mShIQ(`BBOjayza})udc&t3ZeY(}Y;K3*&+e`&}roMYEg}ZKq zv!T!f+1#Z2ZadkU?QVUYqojaKu-%nMv2>%YLik@!J~Z-)SnilWQ#QfT$PO$tb;_EA7(w)wp%je=39(urdhX zFc7^bSj(SHf(`G=qzFVSQl5XiE{T8W^^PEaT+U#7sxFsYG9n9<5b#;wTkjNZO9DM$ z7cM~%th%HCC=^Z3TS?6={-H`nZw~jns77Ms*tJw$WTD=tGp;hBLZ1fzY=V`gBT@0H z9F5t9d0towZ2k$0b@tSnkko4$Jesl_h!C7KOSN4CVM%e;W2^@TP+hD&jyC8n4Kq0d;Uska>R46DsoN zO}nOKK%62IpTpMc$rW?qVp3JbbDFQD%Ibh1!A1f6(Ih&X%Ie4>NQF|GY?>X0PWc7a zL612HOraL+UEg*2BlUdvug!DgEq;$_a(LYa5GqOss5_d9#{&IBF8(%|vbL*b4w)r< z1B%J|{Bv9qI)g)gMVjohxirzJfs8-ZMN1y5h})bor-0c9FSkU1ri`1)QRHg6R2Psx za~tYU7Q;0k$otZENipMid+pNexY06qz2geR1d+6DcBWG*1IOdKA!R+Xo&ZYJO%zvi zA1A=&b76nYj>J}LwQ0+W-0A%zR_TgNT+NMiQT<^7FUp11S4$p#^>|$^%OaD*R8s=G zUqM-MMp-!x3U)OgHH)OZb?c=_+_JTeaz1L3sEI<|-8rf3gmX(E3cB=@k%h6VA$>a0 z&Vmf(UZJ)IK(BkwbLzBgNH)Ajuw#tO1hwtK`DlClzbZ0t0+jwb0sAgc3DAt^X}Oo% zvN+LFIBZf4=i4~nqFU6=q{E?imW+Bvq!mj`GOp)3BT6*fy%?FFdAVyR$5Zwak~^mg z=nO3$$IL(GwTnx@gp*HF!kk6+v{uV*>!n4c&{WAJB3WK zPk9<1N)c2#$83UTem3IfDF0}tM61SR`~3go>b%3L?*BhtQb~8it`x$F;~3eIm38dH zF%O~$A<2q!%u4nM$L1V+kF0DO_FmC2vf`LIj%-F8<=0W5q1qoIwm$1C{MBXXkgrzrWl3HiXLoAl}M#CYbAI=_r=F9(egI*33r0 zXde)N@_`-EeX>A#^Uibo0P7=YyKXT==rksFaS>d(CHWV^cvKTV57201qIqdrt(>&L zx8M)Hg*-^w`op;PgTT_FJg%dr$%A+7F!F7G$`{&zgwcP_Sl(1LL$}Hoc@cxIm1@UG z{8D5=x6|q7nw?_F8NC+8Tt~X6vW*>j*TTqqmmI6r<8sze@wMyO8-}Ub6TD;w~8z8b*ZVR>#OyNA( zq3sKz;8gQT^j-tz%cN2!GZnM?xoGu8tgaT6XB&{x0|1*QB-gqS4B5G2zw83tU2yAX z21C<(n`oTwSX{D$vJKwlB5=nNr?GquHfUS0M;0wgm5kRbLsoIFd6c^?j-Oq&fWUur zHlpa*1WlJL@U>x7l&JjfiS<}BWCZk{w6!RRPezDP$4mpq#ftGCFoEn8{%+=J+y$4( zFAFeEF$?x2&b?L+wVL~pMnN=?Nl6$aL~3?grY#JbQyLD{x0u(h?avJfZXkA*8vT~4 zj#at4vs$qk48b-P=*SbM_OETelAQN4%jEx8y$^!$izUv#TewKhw1A7*(;V;s&#P}l z+$QtHw;`v^%~s7G{&<2P9^AO9x;NvJa#5Nzu)=c1c4CNP3DjL_WYQ1+IYolB4GyCT zEty*FpV{$d8Ng1IFZe<9GN_kxT#Tr*Gn$MRpS=5Otz@HGc48kG4nruGVgb;10qCsr zyk!teyFemYHD*?SxznwcBXZ!}r$DZG{)S&4Ut(dH*%XP&&9r+W)-x@0-*fRklJ}o| zIIX1QDl#VQyGQ3x)-RJNv<^bB+7Pfy*!oQB!9v}3-YV8F6%^*>W?9m2v_OhoY8wco z6#J}aKk2F+ItNoaxUN1BJ;zPZRZ(=h>P1mY&Dybu427@_BD~CroBqdh6PbHn30HPo znD@Q3ux`Jj;Eb(`BIu=Ix^)RpZe_b?6D%j`I2bGxqjDYAELReY?D*)0rL(P2tBha{;9mo-fOF)fmDAQe-1?KCR%@WZPY zul^4PBTq(z|9&Jf*Q%V>CcZ2I;%PF7aJ(meZQla@Fk7}=>>t`Y9r8twI{@R)4AdZMMAdqUx`w|6bQ57G5 zra){gf8siSF|i^CS7YW$$Bl-_6B{r{dhhTRL_4Px1GTyK8AvICrxf$iuxY^^EjUBg+eW&zAHsHo%Xh@ry<9m8F67Q-?E5%_@E7 z*!!h(8=rP^x3Ao9n>nh@t9vr}8x<>m6aJ3M@kQSAQ z!RCaoF#xjGEF8-@IY$MJC^EJPhS*K!_6;p}x*8MT#!rQ-r z!|#w5&*2o%_-FvqQxp_OPb-OCurXENcFP47fx~k~C75wd@xkA+$LOK{ANoHRYve&8 z7a=)Mx83}VwQ(DFqgOx2Q~X^LFO5DQvwoT7J@nYFDsArlGZylJ6$a%_3x%XUac0{2x%tR9%l3Zhah+E|90 z+z$jd=}ksmei~E(UQ>HB=SELxLkK!RWW z+J0Qt5Miu_9nxg9HxBGWRk4v;zVYG^;IfzvoMm+C`(+BRcJmYtRCfy?y!S{qBNjJI zQ-~3}U>!|n%??1{r$C6A09xXrOG%uVsT<PLK07@x*b57O zXES}LOI@GK`)!7!EBNN*pCym}#PsCIj4f!|E@)aY!FYI;KTck&Zc=X`{MBg5w07Xm zJ;C0*Hov}_iR0+Mwm~iGDEr#J$jcr~^P;kBPk#PlbC2M%y7P*kZ#TNKfG_c6+r@tk z#7S9|cDy+H;`ui9y`+(T_7;@f)1MgOw>@H{H{vce$ z_ruFadVmPd1uAoPRRJJqx~mPs0)GhYj8KZ*8t8+ieRph5Z37^BU>kKeuR`;rySb#r zqBeOjBgO|I9Z6bi(Dt_2Xl%6RpL8dCi-Q=H4-7_6+d3aATpAP|GbK->ZjrsRRlkCI zC1HoRU}FDwHpHf{$QCwxoiEN2I96)(r6mDI(t#L*A4$Xm?(hX*pcOEH8g#Mo0FG9< zWYbtibFYmIp;u=vs3o+R@z(4#6l$D=gz@fX>dw7Z%!xznEjMVX?;q4bWNLdrKfyEp zK3kdv)5)$^0AhHfl^V+cP2)fpU2x$ODJl<>0hJvkRcL{fDunoI`FGH1f&#Z1%z)=v z>SUQqy#Yy8w1cZxl=Gg&9uKiMnrBkdQkS229c6nR&2RaAd!%ye93ibo@G5a)>N&CO-9mrre!b4#-5Jm5PBln%J`uje%IfJ`s^Rn?rH<~Ky6gtjGR zLBEbff=N($R^Gzo?<)Vd{z}EXZ}M(2it%Re6y*XUndy~q#-lrj<3KsNSTEu7xC|s^ z7gxXOaK&leG%!nVlimoT<>!i$Jc;A&^9U=JIe2RwcGY{19(}mKU0UIB$7)KID^-z< zqW}Cjvsrco9JF2g!o`Og5VCFpO~Cu=4$V_H{!%YsUfabb!OANF2a^iml_;&UO^}rK9`W9G-UF7RIWKmCOQ12xecJr%?OB1Ok+{W&cKAsmC@RA5sZ*CSDq@btz_-~q&e{1AOwh*ABE$9$lfVwvToPLUFE1A31( zevRTvcQY+y39UE7hJN2>lf@i>G0UP1N9*Wt;C}})rQ9>+qu(}v$1F#Sy0w-1nB19) zYQ97mtdX^NcF|y#2w(p9*o_b1sQ@zYEb$|zM17i#()0#52nEm5da05X0Lia;qEQ)Q zctTYIwv{BjYA$rk@q1m-XCJ5=CM%>zSL16~?F{`3)5 zkPkodXe)UgGjf9#Be=Tk)@8oEl9;mnd5Dr8?$~k5kU?g00F<*BigZLrs7BO1|GAis zn~{ElHk2D@nz{Bm)byE{Pg*#v#q;2KRSmjwnjj%XWs6Ei(A~L2_ceIWj&z1fH*2Za zCN6URs_v7ySoecTiO=~ojl@pl%(Bg_%-TD9!M>&O=P3#{d?*)f?#d*Lcx?K`{rg2R z{u}j6aY;SOeY4k|yr(I)f;LST##eF^RkE+_GQfC?=)EWD(So+eQss)H0oC46n{X}z zA0~pwkN;%>S{3zBR#hr0c%Dn0Q1Fe45ok6B; zF#DDUn_~>p(QosYJRp-UH@ANqA0R0~-=kyN7bu`?zY&Q0hU`C6l zOmu99mJ3mcsQlX_JI=7FBvghi@n_kALMiGuVbV2S$M$rrcOJHjAd^Bo83{ zu$Ygu30E;kA9i9cN*gt7bB|4@9)D5!$g2}~^h`yv*e1r?NGv^qW6O*#iPx3k|EV9j zw6El;>ku^dnA_ny7;V)YKHnctNA@`rJ7@u!V10$7xu!v}z=$wq0VG$;stio95#HVG zuO>5^ZBIYvC&m>PiHm0TNWcd+7-;cjL{%d+dufdWECjJO^_Nbb(o^+sE1fagVIi{U zUSGD10@8=br~0Q~6p7SKYCcZQ=npxX{-Bmc>FaQgEsolGzBAY8-1{vQ^9J~rBD;qX zrFk12B97+L@^As=((y7QwbaS2tc6@b4t!D7_4+l7`Amp&qHYh`#%z~?L&U0NXPs0R z=w~+$zng)-7B6f3Z?5D78Kk;@s&+wcerrsvF>W$%!Pm22xI9a(2na<2mcqkni8+U` zYA=fj!3E*gEF`H*yp(dto(sWa8}&&;VbIpNVZIo`cU#(}pdEQ>w?*MI^F=;YxlWPI zo!#igTvFyKK)8^mBcF;&z999Rm+}V1B50u>PW|rUvgeSwMbW|`ZlM$&bOJ_^OPkSQ z0tO9>uNR}3%?t#QpG
    bX`QgdJ?W1$k(>@{YIJ?&H{)-HkB(cSLSSH)av*97i8o zSJbq0spXjo--dg1o7Y>8y?jAJro$h#0UH^h)PIA%sAp!b9)&Q=e-s-U+%R+Gxwk~a z@G%4H62iN|2)+_2LQ_E2#rUn9!(l50U`d_<<&1STRxJ6xlna@Bbp7M^XvZxOYG(x$ zX`a^eIbAD`^Lm|;@dj54Dg7%%0mc$hV6#C=aixi7gS*>m1`pGGvyW~iH%Z~aw4Ehi~$!-`(6+RI(|Xu{RpK9hj6r16s~y-71& zJjR^`G7(YJJ9jyPEra;a71 zfJ5f+?j`nSW}k}#?0<|rZj;|*^RqOK0^i2bn086%ZmWaYKT;~7Doyy^Xl1E%VY`)9E}KeJ-3Q(N=`a+!9P5bDQixJx_u<`G z;9Iy6$p~gZa}uA_rg8(cKNBi`*&6Pj3`9_U57W-Dq5-=-aN#q19k~sfE;m#kDLq-# z^gvLCb-Zq<$ZCj|E2WCO9h5%?bG2BWlQD^UBE)~+U8Tiie+hubB0KAE^rGB8Y(7?k zcBYuTv-p|+!cVFgn+3a4&S6r?oA|EdV_ z4cM#zisxy*<$DfcjI7EZo&OJpBmeMXiA-HdyF`7Dkz`&{2&C;zc-YMZ<^r)fCdFrV zADzoi5R7HV7mc_Gy0u}$qa1~D!KlldD!VdfGd-a&cCRl6r@b`0nMoQ$7v6oNEU+Cw zMWydu`fJ~5p!)b^d$K5K{mYYLSg)#y=>jW)Y!3Y*s=vOzU2tybOf@uHn$z??lB~2{ z3&#!)bF#CkUu((Ph`|(7y-C183fgIv1l=$k=0hL-GDmj0mP4z}O1-AvNl|27o1@zn zRetP*TD~rNwHbnKaMu2*dwybo17@x1B#pyVY1eONe5c*j(>j^$yKs->ys^T*m!Ej# z*Y@_`4?T$!yzjSeI}kpF!a?hx@yuUo4rl+O+LpPs*T!+arYhkvgX2D6l)Ovxz)}7r z6=Wu+0`Xg<3hagnQME4s(2R<3FlL7mA4wg!oB1cmYzOP^I!&ZM7UIM6Zg9b*+4P2q z`6XZeog+C1 zZj}4$^yDIXKNv_5n<{z#OBSN_AahVhLL%+yPV!teL}>ML^IZOQLb5_ipdD- z_UJDMnW(W^e`D_9n`fVwjo`EaZFp;|Vwjn_&^B$wHQq~i>>#eK9bAs2$m;9cu$)7q^q(|Jf>7c$Sds=DiXv)0 zx>iy3bCS*tVi{LUt(u+S()mbJUgq=i)tBO+QM*IsL5X#@Gx91xtKVXT<201 zeX`dXCuEwfI|cYTg!*YkS#pe{ohJ;irW%s6ag9iu$(`4M)svOluv|a+rSV@C*9Q?z zcsK6Qt6~*jJ8_1xdc*flkd!kn@!}LPJX_p2*#6e=;Ai!1f zd~JesH9(oNQ{r>CuR}{BhZT%zGzAx;H|n}j!-PdS{*aDbA*mQC*Q0&!4GZg=b zmBr}d#cLF3m}$R$Ox~NsJ~#L#yB<($NV^D`e)yw!OqZ$peux)QXl743f3*Rr4j=KD zSC^RAeWGa6leF0(@ROj*#XY`zMCCN((!ahDgtXTf5{nF6-Y#K4GzvYJzIAxl%X{dQ zg3MRh6h*n|?R&`#YJaK)6v$jPRhQKh4t05_tr5(m3v$e0n?fK$H{vdA3Y3JgggbT+`s*TgGO2hGlm)^^~K8s#W`{iH5XgG)43i#vY|+k4)*a;fr#IGsbo$`i{l_K?DE4s2bQI7xTUsGd$mLrDtW(CS)LoFS1^(2O$jYxNoL2AB&LON@d8V z9~J{}w2AF;!=Z+-J~3RJVTa{M^E}U3BK4p+$D}r!_CitJ@r`$VW51?9-8)B_ME^qS zB=}`1qsE6U(Xc(S9!p5_%Rc)d)OS~xp#Fy_5+``73)AJ^1I9Bq2WmziSNC(KGMF&MC z4DTtnsN7h4^%?P0AX6*i*8r2S)?8>1kNTpMM1(Q32zSGt#ZOh_H&m4QB)zxLb9-&9n5QyJK?gji!bnU7(VsNmt0$n)fxRX;1?VV zToi*DbQ9^)#5FyY3Q(N3ElcQU$Z^=(ViJ^MrUhMOzmiNa;>>yt{X z4f%*`vd6=hA^)y1$){S{5gBLde>-UZR#7k2C8HtZma>eMq6mSC;f2V?hqFvdUBA%x zstgUj&Ozc3$1$P^t>n?gl*05==nSjxZln3ZGDrAZgU z6!dhQUo`(#Ht-mVbn;cCeB_Ye3Au;4Z$!bHV^-P|S}g1ufgF|G&Uy|?gc%w)9#_2HBHrl(oSEXX%47;QX_Fs-G{>VWYyH$ zSBnY4k;3yaisl{}ZjVmkJgu_DLTS#Mnm_&|^^QN?I@>JP&5Pzcpr*tyl(YBJ+$DberHZS?%+xs|M$G=dQ*VT_RK%HhJ{HiA68Iba1TY zD!QU*Fd#9Zb+b|uAym=Y-Q~j<`A~bpkm}%9rQ+Dr*zifm{kmb7|5cg;?GFV=eXM1= z%Xhn(?JnG8R%=a+;89zG-YnoY)I(?DYp&xP=V*+_+rM=PxQz3LQb38+>yfsy<$bQ4 zCQtFg8Ie63v1NW)W3xO_7Nubi3e$LC_qhQ;q!FEYo=~BZWMbD^={K`*Ik%y!9$ZA+ zwb7sNAU;Ef8y#Hz%C#k9ORwS(qd856^tQP90%7S}G}jLfc|Xk3Kn~1kE+14e>}Iyd z8X9zSMS5+dbVDqGY8>}l^pv_EdgzECYSD>$`d#=IKi77WxSB>AgCs|nFdj2;!*leU z>*rlV&Z%-7)b!Tj^^g7V_Dfv!|E7UK)|;nB##$u!Vpp+2+^K2;6Bf|&0_?Wl^Q^qa zo98^Nbm~lXK(Kljz91a*_dtf22Dl zY1mO;#!vSoHohXFNVtgfazlC9h%i^v=iKr4Vq3DcrLkRpcJxzYXI}e_CvdG{v0t=#{R%Rzz7+G( zhBxyKj?KhYuPKsRB0j?6ES8nfmmg3N=)Es%Yg*p_)qsksV$V$ahB7xhsjo)56iR`x z6Ia(J=vz3nw5w3(kI_xgz*vi@u3l|FyWW6cJ(`tH+MIa;_|N#qqeUnZV8s~%8UU6% zJpCymi@&}EG?!5+cc^*S|6+^eo|c5j^C&lgOWuRoe~gLKX?v8~o!!pAz`tecY)KyU zrUq&}nTxvpD~P>ca?sw0Ga(i;5W+Sm6UT#>A2b&nLY2lk;!_dVrMu!G*r8~AE9}1z z4FHoDkOSgUPx{ntd zBxLhbX&`&Zj$5w~C*DKz*K$LS6q3){EPG`n>R8497&w&4flV?d4?S#m>4%&2N24L97sA$%8|>hD%8>^dF^(n z&(R^~)Nx)Oz)K}KGZr`-A;@klU{P&p)-E3G_i_N>Ny&CdSjTd;AvVa%Vxz@S1q*zB zaI%ql(_)sRII~mDMwXfKoL_+&LK)yyPw%~P+V5$jwX5@9mJ$>gHqx1+sgz7jckAz@ zC|_gh_gZRAU`!IE=&H)oq59(l0&zNtid=sQ=`7y*PZvT2LO~#fq&FGrx?&+|7L)o-z4lOC4FfB=3B zCRVw9sdtBGZp5WQSv9f+4W;E+`}cxy-oH`C==s>*Iz?rFWIcMQmQd!Y7_QhcUlzT0 z=UYxRQFZiv2&}~DjradT9IAQ=9XjfH8_UKYjTW@fiCvT!MIAtK;jDm!G%i6?poO$^ zGp)yj;Sb#n`Qo7D6IQb93M2XADn%%wsY~*4(jzGguv?M^WmbABa-guf@%mvWm*bu| zio@6}YfMuEEKMlhZtV!Mw}FLq)C>ox0@jOy zAdv@AW$JcKMM1(75um$pWcRhhZPO|{byFppv^PM3Jf8=&UWs*UjSp!zTH@hH&m^L5 z-;C;-mv`3N)@k}3lrfMr=C%l=<>fV5fT`Hid2h7AyiL`wA{ zudZu`b;Cm?3q{cW#0SV`h-#MUoof|;KThn+gIAI+)3LW>FDu~|<)-;5wC8hnF~yf8)`s(k)^lYa zD*dIN!-Qb=!G!HOr7@48TBixA_3OJ>F6_|er*as!O*^-7Q<(+L9+Jg6WPT$X4B_#E zLfs>ZchM!lNjVaju^4eLrM|kN({i{uu8VuOWr|tCzEH(ln2{1Id*`L2+|$A1oJXOj zh4yTuj-~S6ZJt_|l_yhl|N5_&Jd@f?F$|+7f1$>_CmmfcoRY6=S~9s67e*Uwl;7LV zdvdEMz8^y9vKY!L+sX6y>M9ggenhz75e$AB<2X?x6X~@$)hru+uKyMq%unVs5zW3Mp zA=cY3{0dkR4a|QWGuNWQwJ8iKoBVUZ7u48(uH0O^B35SUveMn;*jjQ4Co1}VE*Y8U8?(=yB#mSuZaLoFzqBwQ(T-28Khxlr z1A8O!V=<%1>9rydvYuZA0A$wd^Gri4cptgK<02PlnP@|-2WWLn?}kKOn=Gd@^G@=q z?-i_ka73{=IdqyVW%%}_qW3Y-Rb*<4ef2k@cUF@DvA>Jp(kHcG2HsZ{Wkf1rD~W4M zEf$fEe(4ut9PLJ}z9fB+Qo;)V?~DCM4F*sC=VHU<--krZEMAi_p)ho#dHb_C#YSm9 z`$t6(`AUt925IZ~!1N%8Ki_a^&Nm(g@ao?)V(u%|bei4=U$dCwO`XrBI(I=6jilu{ z4%^5)088>dCtu@RzQX}0LF4>Yq8s}tF{9GVb5)e=w;Fv0Y3>mJRK5Pj+q+s=660q24{SR4Z<=hPZFg2W{!U>JrSzpyGRZx}DWGtnftk ze+A5IvEZ?1&cJ8?b0<{U_{5&QMO*d-l&-45m15b`6mNsLhR_SGs&#b8S)iHk^{y?? zkjpZQ7=;FUy>UvZXuo>;q#N1B9KDcgm>W9ud|4naO$*Pf4tC(2bb2%d9GkTOw1HrE zA&G(zL|$IH2-q*!-cdneiQ22PmvVsa0Yc=CB#VJ#R1(OFRi+-n7D0lXkerS8U6bj1ym z1*g{zW^94zg6Y1i&YDM+e;M)^l@Y|On#)qTRCy;mGrz&c7G+Tp4J0UL?)le(QNbOW zQ^OUuaYNhJzlioqD3<>Fb{UI-NEIi2d`0^2Ff4cVT*t1GDa5zyeu-Hl24cRn9$WU6 z5f#hDQ-F2JusDs)8k$voi>v~l3Wgm9!INs0QM5(l#24+H(Lc96?`x}ka0jA;A$`O9 zO60Q`?6HbA^V(3_Yag*$j8csp41<;qfb`Ma=Kd7e3067fvkw7Y@`u}m5nV(v2d?~S zK(VKzLUR+5>Ki7tH#8#yELCGB3koM}I4@ke?RRvrAgfDO@mLS>%6cxyl7MN}EyBvT zO4}F}fjExlw5`SWH_r~j*L@W~X-S3eZ7s)p_B&I*|I{ZF-8TMfyGMw;1E%b+S(v@_ z1PJu5;BIpw9mxB!C6*5Y>#9)6KKsjbgb&RTT9fVDuN5y?i+-G)aeJVOL@dE+o=uLt zGO$zgHh(rHaT*V)tVwZ`?exO*?bBMvamP3dA?jThlfBNVaG(3*wEe6G zCJn|mXAb`n_&i2ZlCH8Ev@0pzL!z?0N&_=8EW|x6DY1+v#)nJPOJcuI1GYBSpIu!l znSi=MbM>-hW%Pay@5Z0QlEfE2{iyM1yd<+LP8$En1f|cu_0qrly3a=%(tG`F&Mg+8 z5_1I$9jxC?-_MQXp6RmGy<|+_+^XBA=35I~@Y#qkNdq{Z8Ap(x^HhoI2U7pY7mhIW z%Kx$eHfU3O?hAk1T{dkfsP1CfLoxrn;>PqRCwY_aI9c2-e}A4b|3qU}NO!D)BKkB| zh9fs63L(XA@A5t7H?QVE%&md?bB@6jwkBO0%@S(azQ;#86n2^6VruMYu*q2&NM6&( zwyf5DDTxPnmuIj(XK%P(-au=oBt>$(U;&EljEqY%J8Y{!D&4ug^A{&S*BG9){*~m@ zcV0Sk{op-=1H$*z1!gf`8Z$4mAuDVelkcQM- zEO@*R9ab7E7qse`ypvIl`=!-{>a!_EK^8Mu-wNyA46r*%CQ8+Lwfy+0>y8_IAt%*@ z?4FC(5%3mz^?d3{{G7x0-^-ejkv?JIxV}r?HXQoxXur8_pl9qzQ{j?Sla_hyCA{wu zdRht^;kBoBfsUSu-EXpKWwx@ZJ{8Kf(8QE4YQnMpli+dAvy`Mm;k2 zEAfleL3x*&u>^whjH7(uP!>VPL#(^ji~F-VM#GqQPRXIA>#Y5_BvCcKmsUG<%rtzD zWis*%>1*N{(jr`)w@mRFs+`IWY~RyKgmX!2-F-5dyXcuTPu{x-Dk_17zJ)u7;OIqU z6wK|-W-Z&cvv~K2ga~*MsaJ>dmF1lLhX4hHnOMc@M?Wss ztX@+czNN`WvLeoPc1j?X>@A0~I4LyD!4|Q{(Gl9_l0=nmyi31y^#DSq+m5uDw3)WU zvv3glY<`jZ=$mmV?p@V005FUPHV9`}^xYj!DXJ)|DbkYeoZo%vM?LvXNq`!5WDV{2 z^;Q3`6PHcfmV74M3fucn*h_nH&Y#`SqHYLFiB*wixoe*OxMCo{+BhZqG}h*m>6PwV zu9IhKaOYvgsS7=+r17Q2h=y>Fbq|Z~GFSE7TO$-pKVDtRvGjFkZ~VS=%DB{lsakIz z2>EOQ5-af6mLCW{4rAlSA|ze|u#oY*)1Gvbmh{#Lr=4y44R_v#^)kbD-Kj0i!Xyhj z%!l|HIpW^xVDU6Fi&wogGg7AR@`sk?B47qV8mKH2NrNBDd;9JU;}^jWzEa&|unBt+ z5TSXS-dYn}G!N4|GKKr@ydn!g<_@LS;4Em)UE|zn)4i!;0?0KW)}KL2k?~F{%FA~5 zQFi|3NdD^>t|T?d)!>3?DoCU@aU}Qq&r%7b+SrAY^Yspw59en6k^e3Z*aq~-a9?bq zyzKwpJshPJ^aR9>%`qbW+Ux9ksC>tFx2NLX^IHnYgN-R%nW|*-K#e>_7wlopF+0Ai ztk$73pZ6WcMYfTKf4<7=4fln@5^a*c*<8#>{^=1P_zGhG@aVF;Yr@FJp15$nNizTI zr<1P3no2etR%O5q<0-8J=NJ%FsN;DLB~sBpY+tWCVWe+BVt=xg&lH|EXJAijus7~Y z1!fef)HcWoAfv3DSo40o^Lb(DbNg6IZ{_jP{xIkjnB{&W@2tCH2VGwJ4#dS^7{3Ps#ZCoF~Y%Xi}elV;4jAg?PnO#MLy! z=JavkE1!kXyk$YryZB=Ecw9KV<9wI*m5`aE@Da^fM2-Ll z<4OaTu~4f`#aJ6*X?vpLnYUCObY~~2`0`<# z_{ttNxQyy?rQcGcBdg>_fUe7Mrd4<_>KVuvv{Oo=Ilk?5S!{tUfYX);3@CK-h@(*V zp}^9a^)F|wNkY^4z+)bC~0i6`d|V5`&A z+!jduVkf~vU{~GyG7-s15mpNB^ee#siBMU1*==66xbM z$X3;_r(>HG{W}gy4|<6s=?Z}v7R(M!g3`y4wr+i;78_)_E{N@wqd%7Uz{W-*bB0s0e=;aWY3bZ~^5w03oeW|wDo%+7WmycJu+|{q7bZ8coptd=E@Q5Z z=t1+#+S7NX4_B135*9jxg#I`Mv=2ciq`)^~Y2p*<;kMbynTdw;oo5jX>QQ&<%+@`2 zqy?>{z2S)gTz-faF)5cyDvEHW_Z&DW+bxSd{3PYXv*mEYOTp&S1TSh;J`nUVPuTHu z`zYpxNllOSpUsNM*MR8&{qiP}wJ0amu~AbSP0Vt5wW)6Aem64QY$xE|xN~>L_R%8( zexNP+=w+NardDzCnWFBV#fAu}A%=YPX3Cmw!s~vwz^Ijcj`ydSkRzSqO8&o^mF7Ud zs5!z915{a$%S26v>(qAd7T!wNw-F{8Ldf$=gu!8l{mkOL&Yz z{qNPqGB`^nnaqcF)7&7IC$ASeS*sbiZ@Gvp)A`+Q)8F zS!h)o@=fuGdm~-raq@UbRO%am3aTeW^$J&)njhs^)?=} zl&`ckpRKzOJWa^UYhQ1N*X&&Xw%&3oda=+m^6>lwP@PnMq=sGgtx-PDivDE%{i)V* z${olqS9cM(F-bcb1(^|&Jnp>RC2x5(uZX8>g}k4d!bqfg{Tj&Fu5z5hfh1eCTcFu% zaGQzONbVr1sRl188D>5!;(cA`XX3GI7H#vCi*f?j1m_q-v;^ilPURl(Gi=1VU&`iE z!+SGWoH%JURg&`ZQuFt(DgS+n(2`FP5tA29#>~$TMja&Xs|Q|eNq&wtqzxyvGBv>D zvo5amgm}`L%}F&JW$zk}_^NvFh#8Z_P9{=Ha;Bp7T+>TfvNQc)(v*UHg;-dSSAJ5 zMkErcQmZzhs=mjY_~QZ4bz)y1Gsmrt=AYrnVM)&E<;V!gYY3ZODxzuEI>94sQQ^nh()jucV-dYEcNlLe}+hIbMxdiGOkG zZ|8-VK($8QMQ2*R+pM)LYWsWTD+y#~gTO>aZuMJLroI6Rl5ZY*iP2?Gqi6x=8F7bb zI~y*Z)6HMcO9<_Wa(yDVFXOh?vHI>X=YpGEHT&UaGE>H|aLK(w+Vhpbnyw&k4b(ii zNuPZO%+ag(#X^SWO#Z^6)SzNgw(cy%ym|YUx0DktkZi<89v3!A^rkZDxJ1rbN}1#%a$Jjb&b6 z={T&lsyDyY`(r!S?m*T5Kh@wr*$w#fKdNBZe^L)N>Dj(b0qki!H}OTi2fSBrCNJOoz$wH<|9P6`(RfqD&X7EpJ-S6Pkg(nTQ)aV+ z)>LqIS}=EBcM|z7D?UPNlS;a`o7Sz4*EbacB;b)@G5cLKnC_-}5wb{{1kN(#HjtQt z7Dom7 zDry#Sm)~*PFLT4Rt*7RyRjuF9wX8PpCZv>EhEnoQon=qo(VZvz_7ud+RfTZfWaLN# zv*(zC?r&eJkyxwV$J}-B`0Y`Lt*GUZpQ90W*$UkZ(B4XZZ=g zI)?DPQAbiR8u>GL-(IN2T(AWm{P>Nc4Vm;o3-5c^XFBn~ww#gZ5;npLIy}F~i zLH8Klsqz9aA^$CfePPEA`W>1TG>f5=mxVKNH7unkz;&T$q=%l@&g-t!D)f9%8)V|} zZ6J1c%UI*Gir_x29k~gQ%M;24p%l`VGaCydiIDU+lR3dsUj}*^@1^AjaV7p0=|_CJ zt;Gs8;xUJhZ^RiQuvg%EhwCXh4|w4Bb8S+erFyw7 z$D!MQuT9jyc^P^BZI%+f9-T)`dT4KwoGu+g=&J3>SgmEk#7KPjgXPv9?)~v-@9}Gu`McZ5~Pd0#H=a{y~qIt@Q zoj|!Vq1J&{BUNELgcW=`+akt?*7mcz@F#(X5Ex?LB+zfU6w#c$L91x;2y~J`TLvm2X3%Y!7_acU27PP2=OMSt8?M&G1L} z&Gb-32ZQ$g(dM|V6$SwzA6ODpT?TasO1rRuaA+aq(WEz1G-7WXt!OJgbBaat4|fDm z{E%Iv&cIp5(w-F5b4WHI;b6^3f>fEF-CPXB?#jI|zh5|=x#9|1qH0BXhQC^HbotVw z96)@Rn`xQaK#XG9E3exxbplhhNreFVi3fy^*ERK*$kP@nN)JmnfP|{jvmbZXB}HHE zqtgZ==ZZ2%hY=em>;)I~w?;JFrtV=Q(-+*Ecy~E1%aJ5P8NnsGe3A*cf}{{T#3%T- zxzy>^DkED>BDn%yP=z-9wgifb223rcB{Xcxc$*-)&LC%&gL5n0?=rVhYRF!1tn}}x zyHq4R&6Xx5tEqkFzq12c@K|S1d=gtY2%F+!$Cu4MRoe82*)GFTh>JQuWbh&T?O;Nl zI!_;y(a(JICj&$;1%adyeX@CarnDKdHq0V+ZTR zAp2vW>XxRTugq(Ix=Vd-W)B&V1I(?&=|})HsxLm4vb`G3BsvpbKvcPw`n*2`IY<_u zU`&la&N-}@K!+;7{8GCR#emkiFZnqokRcKp@Pey=SNVua7pRct8$RkrN>}ou9(7&! zTmVq6g@4I5%d`{vibuu~D7=0>jQp3o?ca)U5i&h-=7Xs*hh%mFEcz42dBw$N@$xp* zvO(@zb&aLysSQm)*C@^mu zP=<^we?1tZkstk1q4`Qy`Cr7o-Sv3JBxIGsSvwH{%HCE=fz?LRJ>SaumW;oRZ!r=Z~%^;3)LS z?3U!_;2;7g;#RKHpS&k*J*R$I0!|1O_?V#>dUn~1dwM16k?E)>kbi82-S66%n(W|E z_rdbur80N}bRnSks)-lhdY~~EAD4uys6?{I_G=8=zBeVlQ^TKHWTevLcv0zRG2KGO zd}H!*b9}N-j|>djU^QjX!?=0CZUTIV5iRy({s}w6J36DECt0*an=-l@KVD*Lpe7nc zZR#FZq?X$_iz$}0w7OBZlh9KUx%uD*~oNC}!Mmv__Jxiks@QFwcY>h@mJS<)%WwoOr%7%kVq9tQVesaf1p1x{=C8Z_ylta|DTd& zFt5KM2ONIh>Rv>TB`+`fX-gqiss@R>t^vt=7L@cdAUj3mm(O1u2-1Tf1JN{GnR{+x zD%vlk6UEFJ4ygp$7~;o#Rp-g~1=uvZCHO+}D;I?nRu)p+(UGrZX`8NiJ7pHo$oWgN z3>E2>J-e4O9Fv!R@93IDutNAP2NI$Uwh!!U2IFtV6#W83)*H=xlJj~$mNPJ$4$qkr zQ=;HJB*ogk;EakYrYQI{8&t+qo-;DN5I1BI@-$;Y+4KiEr|`d0^`dfyt#}OP4crZ% zASdGmLeAM=$AxsbZcy%zfV?06*80?RM%53sd1ikgN7A=zttgl(@)t-D{zGoOLLgL3 z6LD+3k~lH#XYpyA)&QERkX0b3P1& zQtwy?x;2f(2#UfSZ{ZUQ)9Trd+M+Q=orguOLbFYZ({TjLW0JABVYDH`2{Y8Q+;wN&o4=$bq1RHxGq?9p6FN!71HoJa7OwOFmdv; zOb*vI0)M&?t;2K|+}72q_5rKi+q#)Jaaia~ypC=d3*SI|9DaZfxfHF99eslyt?W-8 z9@4@JX2{kyTALsT-p;-Rd!g(nSvv#FHg3S6H|??Fx`+OQ9W+};mA7rkTv-dF%O4nK zaGG%tLSEaNU2iqOF$inET_Ej2jT(r_H*N)^sz8!LAH<`C|8+ac^`S0se+aie{@w)i z?Tx0HExVqq^gxl=IOmM+CC`g{nB66@-x4>$VU{);bj6E^W2W*-E_lgSCb|UF!uGzL26I|I*T$6w@V_#%$bwrw*Db71@U6J>RGAf})~Y$}iB{qT<}s(O z@hly62f{VMj>A3wqIPmkA9S;qe>E@tVt9I&lY6dKDM8X^2)LUklgGVF}XACZs z#=7yZ%^-*DTN%0Ls9bs1{UjLhf4BjMwRb1>syLa}5iD-8{+I zU4G4;p17Ht6z@#)9{3(k(dsQ9Kk(rnNR$gJskg+zN&huu-Mw2YW zO4CL@*RRwjCZ^e_lNLaQAN>k}m_Pa)&su0{4D1`788k9Fesd1&8sZdtsLBRW%M8lc zo1xkiriq7#=Be7M z8RZ)%qn5rqnc08#9|gcGPc3KU+4d~WHHzm(<`?q&AoBNIDC3C6H^_1-$h3B|bK|%~ zlDY-iqTvrpcwjibkU?C|a@YbDb2^XOu%F{je55~VNLGRXG52~H>n_ya&^mIgX69&A zD?}7^Xp6u)!2Iw(Q^D^toL`8Bs}6rWOq+gR1NCP;p92A7J!hqfo80LE7td1JZ%Ss# zC=NR@qV1}~x#j6=liB@TQ@>wT@pUq3iEXaI-^~imMu5#;u!N4iRA)lQ?Khji{+$9AJrKJXWJyVt?s`EbgiULbW)hh7XYRnCO5LRIA zWAoC~U>=lD%aRD`w?(2^DkU*4pu?c-{^cfEEd`D^01lve0qu|LPEWYz)cBMa0M=rD zBzy9*$#z$QIA6FkYG{quwr!b4w+r3xUxI4U3thUwHSM-R49lP9WO<8lBsArHG-?R2j5_g#XMzm z2TGdaUi%5LyzF;=9#+hn?N z$3`c+M2}11K)h()^)bzPocNM7y7EM|#VX%NPKHV$q_$8HQpAs!T_X~Ct%M5#L_t8wW9 zYdv4Lvr2bp_Y|rK5?ZhBk9j*j-KhJ31T?O}aMUoG$Yg0K%T@2XLpgkRZi)Mkvb^f= z6aL|uJl&Evn!=3dF0LOkyu`ef=cDx6Lu9vFy}b3!=JsfhRpUZVak!KaeA{kMW7sb@s@)95bwc};+tY7qP=AS8#icE<`256qkGVf4rX~*^N16-Fkto5w zZGK)<{C5N}h$cy+LI554m&nJj7Nc3yd{=@gHu50*hiBg;$nF;vt~&A$>Z@z>_ZBFe<<$6WJP$IL`AA^Rq-P(giY>| zFf|IXuJlssERyS#V=uW?p>oc8yj&X=T#v?+P@T0*d!E-ii0;D06{8K(w!6$H$1hZH zJkGUd*uA4a6pi25<$WY`4_~0*NAR=#kGQ@?Yj#7ig zrvRGTE)G^uKi|9q*7SwdGR&HNmAQZ%Rq8lVr zy3jaQx1Bp(->lNyW|f!nSqt1#4Hqg>LdSK&opvfF$RG8Y5H7gg?^CANpBGF-xZv~k zW0uS3cCPGuGQ$oOqRGa^AUs!i5?Sd(;>FQ=kK-1qWl&x*W>@-b-H6x}1EG@v?U$q=w?K2x+Q%s3br(xUKAheRyzFXeiM?oA$!lIvC31|*d(_-l%7quq+#^q3I)j)jR{1bh6=g5 z!66F;qcq|_D+Tc9zn2?nQ_iw34$`^mi0n#Lbh@l-hWF0)w{~6#8Rnu8fAG*zT#65H zB)Cl*$i?f|ZtXD~viaCWSTgtyD&8R<&2B@;pMuCuEVE)CF)t6y30?5@ZKvWZO5-jc@-8;kX#vfVJsypCn%=)4|6 z^@wsx)$E^TUh~qdZVSE43mRao@E?XOxPyjhS+xI9gY0XbJf8vaXaSexY2@xG9cAW{ z55zK6pWGGw$u1AV(2EcgwRcW`0+Md0A~=ebGH{Sgh8u$f$diR{)@ zs3(b>ey1$RV%9!jP@w|a12mVnLBSpaO6HI=9tAbq#a0>s24)X&2+`(qf}3tuQ3vFtUi7-0HP?4 zNa@n(kTt#HM9NvmT^4|Xg0W~qPw!05??K~Qp-M<*KS>QP+EDZrdGj`ItmFg5T2M1d zzEq=JA{RG|^%x!v;h6JX=(83{&7kJ}vfHYZn)NbJOZA9BS)2EeWp445CiPb?7+iZR zA%P7qvML~Z6l*eK+{8-6ku&{eK zNvV~2S3s{%6l6yp41l%?gI=_LsPa;~{R%GA)mPwzT7!aaNT;Av!V#KiErq~l$!puJ zc}>Iy!%KTYZ)}r@OTp!>UE*A(^~1e~o3+~uUL~&*GG)t?OP`kh{KTEX*kSDF_vB== zwC!-v2LX;4#k%VT000;|KW49PX8fSqnk8o@ z-JRg}3&7|-Z;hR+m%{ZJ!VN#*CO}SH%1Ph#42XokBEP2*)j9Xm&DGW7IoHT;JM;Z; z+gig`&KPTrlpE7YdAdI6|NQ2vXjuB?TbKSgKmBE_K*pf+)ar(p%|NOSy$n?j$ z^@wxPl_|x2&g7TBo2h7bk-P`BZr0n?;RhjS&*e`&28eOpWKs<10Np}aN539r_T>nw zZGgQHj9`&B32szlM3l_^Zg{)0tHhMZys>?6fPTM{0~gZM3T0gpKn`uksj_wh`=MmDNB{^uWI58Q)SXjyRO?avGy zz2m^vj_4{<(y){&0?zU(&iV2Be$56i`^o&7*FucmvQz3k4eA&MJ~mkB)NWqa2szB# zOat~=A6)&Gl=q7j39_qbZQ~lHuWh{rXt?*x@wh>xH*n5LJDLr1y?7h^im5Yj;u?MU z>K!PY7{3KXMgs#R8u(7g{&)Iy3=uV4J1D#-lmeXZAr0l4(_N^7Ja3dp@lNn}y(WMp zJ8-Il^|0He?R+<~Yl~shSVU=2KJZgPVTA!rh%6ic)k#bK)zF6U)LGsTe9pD_G!bcrNC;jss=-ostPX;yM6pg9mY zsbZ?inTZrXWZag1oCGEHIYs_3VZdhP>P4R$@u>oEbCWWjVGYepB37TbF>Z@-n?nbUmX9gubNRXPN{TB1EdCgKMD1%F7Aykho$P-LYLBE-oLg^*so!0r zi_8yICS&Ks1{eg#firM{Uho|U=jkMcwAgMVP_ekK0%vrBSvJ9XEKb+3Ao6**{ton0 zTQC!|=5K=Phn!hqurf+Yfgld?Ir~E9x&nZyLyC+h$CpU7R(ttN?^T}exC9X8(|UDf zO57d1L=@~lyb<-4h=?Jt`sv2wRh?^x^W--?3ndq%u-`wZr7{he$MQ15sBbclnllH^ zOJ=HTN3qMQBzP`?$;5n2Y$X#A9@N?!Z0oirhj;auWv%llFRo##HbNG|=wsX;+a0#q zY4@W>hTwJ9A4Oj6->WD3XXWsp6nGolEnDAZonO;eF4zu8tr~7Y{q?h8Eaw#!0{f4K z=5PubfUewlaC+oA=CeQ4n<$6{zC5|7Cr6?z^U7o#?#f(spS}i>F{&MHH2XPsoPJzW zsGV_vZxy%Z1zCu-7GMuZx-hpi=9Ht@cVB#jHT8l|=iui`QE5FFJ)o1)V|ORaFfV6( zjYKgNyU%uOo32DM9QgqVY-SmDHxOSL0xa%!f!|tMIuehHWU-eOj5IVs`Lz69!K$_nOGpjH+jjTyk~X z)@!RvMC9tidnh?~nT(Jd?>O>z@x}0cNuP8invAT6^3}aB`wpdEJzZBCstXvoVUjrI z_Vs9K293Mn4VrCJiU6_+Q3e??(zVWa%_&8<;Tzy_ZdH@Cdf+LvbkAZNX=M5nMrb z05^Xm5EvF*z2hWf;l2Q*mU7hSr zJo?34+v}ES;=49I=M$}$!?{b9m@06kHTIK(85hZN_~;yzq9k!d_H_&BuCK7t1VPM> zcwY|ojemt19S7JZzQK0mh#`cWn}PUK<2HIbZ@DLrxN_Xz+M+qd*AuEU-R^7ZXAzhr zdXii!N_C!!S*QEy0(GcyEo-`@tO{v83@qX};Xr>}z-v4|lF2gDMm`_8GG8Ny3z1$j z^Z#W7%bBj67CN}|$0!CkofC8l<==S8)#cu@a8A6AXU|Q|Vef%XA0Ny`Rjt>r4u9Ne zN*387Y&-#{wTZ63qO=S6-Mi@B2Z47@+BiW9%%X80Yn6MonEzk|lRhk4RHFO9^0dRt z3^Ju`wma8!7{b2%-lbnz`qz#hDR0#cpX05L9k6UW)Eh!sU|*y=nx}GerGB%eDx5X_ zzy;zSBCo@7lB~q*kjNZW3{qUrhC7EsuFLA;iwPDa7{3U$)a4NF()=(@k`Uq2%%)Ss z+SCYJqt{3fQ|pDWBH&b|Y}*9%!y?V2++~A1BKFJQZkpv3IW|O_-jNWgH9oS6&lU9D zmk>V)X3k$Tw4<)oFZJ$D#C(mIqxHI$Apx>+VOg1Bn-}h~L$RjkTR{br3v)oLOf?`J zLP+>le*{E5CSOt}?_BfB?gAlP4amP8lVr+RVitar%Oo`l!>4^$#CbDyB3z0=qj%n- zw{qSj8&tY>(h4>t+){9S>eEJ1+j-$2hkvhJ{m&m{bOEsfd>VT1PNF_)nE_&lR-GI0*Pc<367bZ_M zF#%}Y^&4YRM;O-l9MX-(_rk~MyzWDy3Y8vbc4qX%d zE<letEs zh5<|cC|3q-!8l9_B`ed`?B#S7G$7Un`o#13yPqGM7Mfr|cNFrI_nZT}L+#}jMZz5)x&&-by%7xD394^%> z5#*srJNVfTx$%1#Hh?kwqvisjSAutNL|>DU3^Gf;LWA%2`FGqIX}(F5ZdeLdQU<@{ z7n8n6Yg#A}v?&P`Za2T0;fviS57L0~Sx6W!T8WkSNon08tsyX*!@hYjw=B)*r@G;T zM}#PCsaiY@h46Or7_cFh9>TgQ4_oJEE4&3JezWdw?dur?vQZMO@|3(qP))$FqM1d< zd>~C~wTbs6R`y<<7XPRk-GczW#>2&`$-nvHfwf=F!*Vqpe*=F?xjw7)I>TLi{;^!r zK68V_IyK2HC;=XhxTw~=91XKwcezb1K&qrk#4$kmim(w*b3XEu6>(f66bf8V4}Hsk zxQlTF(Kc>q8SzuSK{kYYYkU=SOgS&Kgf5e-F%;1Bq^-{Pth~|VYP-K%fvZ@0;co!ou4h|75WKE8a{b zn*@r@NiA_A9<^EYl2ZketDAhJ(dr^MBS_J zs$p0aP>~HtFoxQxTOT@|0n<6Z8<;UKY;HVE3oo{0A7LSosJ z3PbF1;SijmnSIS%)l(LkCaIx`+)j*${VR91m7C_zn-98av|t8FoBTpc1__uxjFL?k z9&Ihen~6M}u_oA^mYRHt5Sso3yxgqa%pWNSx=$m6>htkpYm?kOMDyV}_yJ(~C_`Y! zxB53t^3}1dHLumUwm^>5mmZ67hBp{i%Uw+Z2le9f^5f;&e(@@Ujp?fONxP92kR$Tg zR5@vyRXQPoH7*t?F4+4Q&Q{INl03lT{8}|~?ItlR=jXo?D0?tjl{vBAo(CuU2w?5O zI=4E4A^QRJ#Z^E-(09~CAyev7xUwNR2W*RGfnlNa3z%G5)OC9C-WWfwq~MWt6Xz~g zL~?JYAO=6;BL4Kkn$IL+r}ir2^ZV>eX|5FXl|NpNMl!d`rnqu_8AKY@UsjI>5;W6X6ou(c2@w!t?QZx~)3|{28GZ zN17i_=*yNTZCgfgzNOrcDN`4UkxZMeFip%~>O%P$9Jcyl7Tn*QezbRq@SGK3N+OkJ z`(r$E?ou~_<9WTBYZ}-NDDEgy++&Kr2YU&y=}_m+IBIpEhB!lwI(~V3ExQufIQG*& zd}mnBQP|useRhlzOn7KN?VC~>EdR)7ijrL|2Y?82SSr}beu4GzWTnF2=lg~1XzVM; zba!GIZfc++*Fltd+#iHmT_vWpDPWB+KFVH^d$U)iT5yK%5T3{}a%_z2m)!3C-jwF| zyWDlW>4=7u6%o(|<{0Ur!>yc$xQ&gb?1%@Ph7T8}foCKZ7^`Ff5KS;z-<){TWwivM z7a3q)U|n*NnpJoW9AJTcA}Zv@z(KDSflIJsDu18>yK{T2)^Volc)ly9;9~mIyFakN-2t3?r5L_#t#Rc_V~_NPapWVkYKsKhPj;m7bw z+AgK8S~6_UyS3^0z!dqszFZR>B;Xb}ZtVji0Mw&|pYxpHlIcQj(DQTp0jbnT-GRM2 z{w5Bw+RJoo<1s;mhO^`)*Fs+9m}K`oQb&ceihca((@Gu0Pm!z1oOpA(TL1YChk}yR}Zq~ z;X&ENfm4Wb$6409e7!&ExcPJEqrv&OKDz|SQc2M z4p3=X3(xC4PbC08&BSHZRA5M7v6k{>b+)K+V< z^6v=Xg^SvQ_m3nT;CiY&AI+00>VsU(9C0$o;Nf3y|KaJWfhL7KbF@ZTgRFG%2d3Rg zA9q?{^=o)Fj9sm50-zrd{c!1_$6wwKWw;DHmSz-lwtTalG`t<1AZ4~oF3ai?C#ZOb znngqJ&JTiVt9r7CJ!*=;ZCL?|{T-48m+C8P6{M+t)5t0%jM`|kpP%P^v?9v$bTdeL zP+4+@Vupe@V0q2$NLgc_EV6SjrZ@8Jj(mIv#}wH> zqO~{tnZvz|toUmV0J#@@_nrG*7=+X|UC>vNM`zv(siV>jsBS#|#bKg9#WcTg_3Umf ziq|@t7^C@%)cCX__8Q$_N7Dkxjt%=}TGb*AYl{J`-h_tVH(L|tH@K00jza_QBrB0j z-#o{YW|x9}Z9^;HLjG8OmRTeC4>XD?56HkSjc)$>72&tu4C!H)g_NK30JcZSf+4Up zW`~>lc5#0y$^pnZDP##WH2db0Jt2qh5AkDVzHz?eE+@O#bRMx-?F12f+LHGzB#7IP ziRDNYzC`x)2b{QvsdVW#g{}MV)NkWgvB%P<0(J4sFGHDL`QowchBldj!y@gF(}(u& z*J`d=>nP&xXXAq_q@(UAr{L81)XsZMn79v7nEAvhJoq7Qeu>_-g&2`|mC^g1ep7CI zCmDlRbk2~~LA#EwvgUZnhuKgNOS>I@etN~~)4biVO_3>xH#@Vv5n{#rgHr9`tI`j% zB!ujp@M5H15qwzF}pI>+| zomIFPnPr}qnB;_)>yi>0EF3{Hrg*7R>qf{b2C;u&ZL0*08{u0AEX91t`NO08XR_KZ zkIQdn1utV($^@$Z=_&rJF5-O!%Asrh&YpwF{dJ+x%j!ez%{z?{d~4h=52XeUv|_Y^ zuG4FhxPO0fwVcBf`}u0Y&GeI)y3@k}KMG9e6u(HP3-0& z9j$uP#@Gk8-+@TWNttDg;DMYB;CKx+Ro=)2YR}bVN1F*?-UsfrC~qY|ag8L|l`5At z?@*k~jp-rAe}7~3IRT7-WdI4vlRm-1yY}6e$HEwzjvOTLN|>R!vao*BVe^u^SyF-Kk_7DEU@NiHd_Y_Uw*>Ut;5gls z?xnZ8NXm}o6<;x?j1({9RK)o}9Mg{E>Y&!MM{_+!S8-C2wkXooK&2kHoN1#$ZtAr^ zBoH#&1aZro?L-oeNPk*kuJ3Xokf-O(L%2$nH4DdjUOP4&=FLkoM<$4wchDmm`$Pw% zdX_-R!WJoxXH6H-_H#2vTH&9}LJcjWJz{yk#7%7&5;OnZyLnT(74k-#_ZUC+Pve~Dg6_Y1x&5tOjG4!Hj8-$zr z$S<8O@C!kAd0A*iV&9G@^%q-9aS-Jv#6?tq7dnn5Lv+SKwTm_R``F~q8j%w>v(R?C zvCsR+Zf%sc#h6YB(zbZ3cOXUdS-RXT$I)}FZ%yoB(av{EW{E8oNdkD{&mz`}5Scz7 zz9a>_xOW1J&{!wi3g9q>p}Ib8$8TFZDC;WEwmaRx#YRYm;`3bRHo#xcFO%S4e7>vs z>|}$`Sjm}5Klu5k3J*F^VGnMTkxOlUzFO4-x@r3rGEcWOoyXEKLhfCx1-pesd5lt9 zt&~-LB2&HGprD~Y0zqB)j9nVATgINfxkeFE0}Kl>Ox6Sf602ctUh}k{->=Yzv%$iyxdpT)Nx>7!*Aheh2KD14j+fmiu5|iR9i`Nc8jWb#9B2XH9}O4`GXqda2}c= z{Gr0yje5)hs`N;P8cqr!uwGwXe)NgCP*PKMe@x5z<@1H_m3I$>#zZ8|*|ilE=@tYw zM`yIZ2?zQD(;BS=)*bRcs@$aOgd{G7`>^lJ{9pN5L>#PkBqQ0_a=`m*;3y!V4cHzh z#j3n!wtgs_V$2iky3!a)FTv)_L3h%+AckM`zj$3~!{?-50@3JMFq$C{LN!Y^ZV+VZ zITwM-Pwk3fA5fvrzub1PHHsXN!hQ*%jNwx0rzwmVE!z8?7#WovNnj17oZ9oEpYBe4 z1eDF@gHtchwI2Q$o1*ykC`=UF<1Dk|*_*9&tUUQJ5ml(K6P*}#24x%P%}Jp>WnW8Q zE)-)&D2yG%HY5!LwLV)ehUboOVp3$IHVuq+Cpv+c=e#$Aj!#kp&-NMMxs3YGLOosf zo=-ksN>v%Ywuc>IC%2`0{{E4PeM_9Y!QG1GVZni%5x#nkwJ!yjv?QX=Y8@6sLo=U7 z7hTgt2jpxBOVZ=g5jBYjzNUlO;oP4}*!Xc1*J)j!2qx#lsBy=QZowcS!Md{Z-jsw} zZOHNp%~X2V+i)J&Pwy&%C$_dRxBs6G=09GCz5L)Q9J1;M9_>n>I-mWDj~4Di!ZNw< zrXLU(L-9&%1-1Y^nGwybh|XE=5!nX8b~utp#ErvtE)RJbkHk&d{2ErFJe7A)fIMK1ImcrWZ7o+qrPdKzW`6_&#sbkc+ng zVX0&o-+a;`ZL=jjp084LQ#?Hsn2qG98}O!wUZN(q%5Ke(ySiR})qb*0Py?eT^t2@) zr@5^8)RXw5z1MzTkYZ#`;EWf4#HZomKtO`7-=B7sB=n}rUAs+r=C7aq*vMbP^^AmD zp);e*{8Ax_0&m!SPTZJi40s5IK-Fw0H=9NC)8#=SsjSi>ul@C%mJ{*4B} z>mx|~2Z3F;0mUwh)s??)y(22jTE(mIWj+F(eP3gbMR>nQg|!wzeO|wbl%Yt>W{P^) z9m?!?V4jHU-TYPcTb6XF8H`AGnzw9X;;yuR#0r^=a>8CJ&7sW4N9PxQ=h}{U-pJ*tqUU`{5Xzp9(=l#664!j0NP;lsNvc3iQ{!H z*7)~LW+NCl#tNfaKnYFkWQN4pZvOnty6a5U&%=k|W9R^6g?R5Jf+xC;{?9QpyW%GC z6T5jO!%U;v*~Dn22O$r&_A#RJ-I)0%;8Pe4{J<^Bnvx)38uK&schf1}b+74Y*#FD* z?7*gjENH^kH#pDuJboFADHuT;EspPlsm+MkQi`19z<3*vqBKZ{-*+;($%8g?`#_4a zWICDD&O0}+uBAJ#+BmOMCUV3>tGAGb_@i%0%oDy8x0k5Yzf-Yjk7{S#|L3KW5xn>A zabu8KD}s$O?CBrhxy&_!3iFk6=j!X6PaGlsq2K%`3RZB?5Fv5o0~0S`khsHm`tIeM zT-meJ6P5Zl^SQx`*E8JN)l@Z$rNC%ow9lZQ%w_}6A$uz)R#1h4^j2Z7~{+9MS z^TAsV$rB{{tq2mMS_c!oN#=ZFz@eo4DLE3xz}!Bj z9)M6(CP9mNX=`@cb41y5qB1GS_sx~ug4-oT9OvWX)TfVZcZ6R%{BO4BuY9gxUlq#n zXO4bXoejj;2cLD0YJh_s496MGR#Yqtk77*kyokP(;#>Av`00f{&o zBrTR?1rA~N)p6yVzj`Ru=Fq17Ti$@6wo{26LFBg4B(rNd2Ta}uMwd6uUdX?JtO1y% zD}ZQfMlndiq?08Ro6>~C##csy2xBJScIODcRTnVR&HxUjF*JrY_wb&>ES}ro`n%5d z&z8?a!muDkz!ema;6kjFIZ%rGIi`@UJ2}=W)WmX}-1|k&K(5bFQax^n=yU(>!Mx`F z_31Awt|3F?!lnU2iG4ztLIUQ|z{iOgoPgj)PX7;*mn~YNdUp$M$~}ndQI-~&PS|nW zZV%2W4rn?5eav?%)Ayflz^itqcM-lVGhVcIY=eCNywCprTP`1Q2XNB-NQP6bBKSN-0m!MWJs=+P)X`o0Zxp{7_s*R@Wn^Nm=1b-MtD61~?6IZ=n z4^xx0;7xDS_?(;bQP#Z)8WiMXPj1h5V!r3jCUS2ySkrBGvk!PK*5@+ru9g~~&((V3 zT>bEBrh*v?h#UXs`}^cgIoRy+X4Gptf|B}=Z}J?GC5UNb%V2d|9Z>fsRHD9aJ|ADL zvewNBWmOQ8j<#e95A9c?IDA@NGH_*f$5DQvIz@~8exPjN>Oxbw5I9(q@H1z_Ae^hf zrFOMPZGQ%r&XI`)7z%9TgUe*G=n?~$-5rB<{&!Au9Z_&iYMdL$Tu0tN{Sw-dm0f+X z%2W#0P#0}OVC#v37y83VdW`&b*W$S|K6Hae^j38=`&G;T2Im0(V;|yXrx7UIUqAkR zw1JMpxmRoopoycvuIfwo0#yH1Vwk9H0DR3s?f26267v9+puG2(BFzFUIH4w2jRx*b zEw6?6g%0sY+^;s=nTN&_P+6{R)Ov$p){y>@3%`Sv1AT8FRNi}V1XpdgkWNG-^RHWL zFjiJ04(G9z_pjydC4&tCkIhH4`&%8ZzXEU4C`u!?=0Y?{%a;ZMENrH?+Xw6`t4K>$ zZyhEwn=UEJyoj&lM3@gBj4j(;{&Riz&s{!{(t@k8eU%w$$3d;`cPeP#w9&NPFU?ih z{?TBd#h;&BQyva=t|-mOA!c(N_>5QzyQAR-rJ2Fi zw+vcOc5)gmuILuZ(C5;iVok9LCfvr-D)L3^>aMz`sbZ_86c<91hpn0KNyo(YvO@qj%%=9cajssVx6aMYGn6o!2%xdg(9ZF z&?bdFH28>uvVeK5=V54nIVzvM6!0V}$PMFA@XRW2PX^b*PfUkek>0o(emnom`gifg zEU=zg+u{=#F-OW(oos$(*FPn!f84E%Ji^BA^3I1#v7(xs3g-->p)D$kCV>Gus#6T- zDJbPzea522(L7s96jBUDO~vOZwonn%-KCFybuGw}KQU|x953%hG!5M1&yd7jT%jGPvp6pXni!Y{**rac@_w(5WW>mo=%aj@&ExN1n|ZE8@cS7%8sWu zb30-t<^FzJ-jEwJlrXK1beGYvu^K)9WdS25#U6GQbnxKMCS2Tv89cEeaaa=`_EG)Y zV4dc4zG$i0E&U~Jq$Z0e^E7Tel=`rB-(vw&Og(p}5=O$|Kv8gDyJHpdA7|A+1;D>! z(kQ{CEh&YD3CtWtRxK7{M6)_TM^<%;iSgih#=GL~&=hsQ%4l((9vqmcAFJ%Z^L++x z!@lk2oEkw->{3wtnN5 z`EWhbvpg%a9`3$Q6xI`V-Gp8`ZTBq6f!R-?V!D6e9%_e|e#37$E5hRmOpT4;Ut z2;t*Zo}$0#vcbc+N{#d%oqU+cJjK7HuDoi$cY`}pcEDmYWxV}1gW>-%T={o!x%cg1 zI)GRH5ar@c|Iq)gx69$QQuWDs3K(LE+RV6CZA6bnx8ij#OzIZ1iX`XCj>m_Aw9fSB z>2l-@Grq|uZrs#0BB!{iVof!Do3-)iB&cef$qhKO z3XS+4KPa$044g_lS@4|k@HG}pWc@39_>wv6^)gJ8k0^XRZNDOG%j3s(o1Jux^fYAI znJHmA%`G_JIe3=g&j9g1M}Um zO!e-Mx#&XXU(wPA8i44Nayk&ov7M63I%NHQN!^ckPtw$}HQCn}n8_({bb+N*zXheq zD@?Tt>F$Iunt>qy`Pe2J{@rEz+`v@A;zs53**;vHs0nX0tl_Dt>y|=eCq`YY?kv*) zp6upq;Bc0D=A(3R(!uKW;+_GO{+dc^ay3?isT=pu#^A<^Or2~WEmc(NbtZ1tU|CiRvk|gi+s!bhDQ*byQ|cZ(2tcc2){ye$QYomAy1*!t~|(HvY5Q zgNuB2CfgtPLMN9tj8+ZOD2widanPtCZ$e{*zxLfOjbs2cm3}u!pe|Kb{R$U@7Lq$1)el$MQ7dpV{2YhsN~j} z{OEgsBKsDv9%Aw7V(aoJ%DGG$v7(pDFMOh~tti-tHWR>=&}wFvPnESxb|oamaD@KS zj&NgIi)zHR8I!jSNTy2|sN8H$rh&%l^G4kE`EQRUdXOJi3*JP)cN}#Gd=3}tM;&$X zqSK?bOlBW01BoC56Un@nZq7S)SSNE)l~V=2sO_qseHRmaCn~4G;&6%ZZZKK=B`>@s zm5G;1slWxE-B~HasDU@blwH8(CpD}>_wa0dk}Mn;j$Z-a0MM*zT=^AAVOQd|9`FbFE=kCE&vTJ(f|B&P*%O!$ZXpGb@;9aceFd zwq4A%l37ZV6VdQ(?(_z7$8Y7N+}wsep_V=4cJi9H?O=|7Wx4gQ1&g|YHOY#guW~&>LIoEuFvepr(aEbgADooF-M+DP8GmOxFI|e&hjDwrz z32e{PYdudLGxQqwL@$M$Im*X$i4&(r$p zomYMu2aG(fnMBjd6?>Dn)BoV0+O^S7@YoOTe4c`a1~M`??d@TV&w&qkuvj~(9a%Uk5mx||th zg){^WJ^rr%okvLWt}LlBI`#kPIq9%d%?VFguvEUx8>j!9O4&c526;|5Cl)Pg3>g*i z~h)hR^*J4nqYN1pOqB(<8oXN0G(-tAMAv8NnpM=frm4Cn^~wa#T+^Y zu|AGmuRqwEABqN7odVFGrayjuev|x(?xu;cZa@7$y8wR_g#f1-N!S&4(siRHJH7h+ zjbDRdi(HiQaG`;w{KRb-0`MIqFUbDA*_sowtw0>oyVafkJ3-cGyx>e0agsxp$-$ba z7tN2(-%{eDKcohkn~@f-5!T6G+LPpk7%0E8mE0_RU@jLglR-=|7#%x&au^LaWP7i_ zq^3a9ft{^7YjPN z{bfc1V_c?3rIX!%E0fA)F+Lo-X-{@f>|$s~w;BI3M}G0cAaC2((%yqse>`p*g}<-! zfz_9=v(Mc_>OczR)cSmMyALcWC7)&HXWiOra}x)i6AE~nu8IVq7p8|S4}y+ViQI#~21ywl^*E7PXnYqfb_K>>Y?=lg`wA1Xh zyG+Zl333jmoFLx|0`Ck^CoX%Dm5Ie*}xUchKorrqpQ-(?@X( z`cz#Ch#cJl50~UtX_(d=a-NJ#y{8#GKr;68tGuh6@ev-|;bx>V;2=fCcFN*#O};M5lFnoBE~0)jk1^)R!$qR% z^Sbf?du6+oMSoCe$a4A0F3GttjOi=WOlHkj3t_A(=95dF7K@%7eGQE%TLH*nDln8bj9fYOZ! zh#;NPIlz!AASEE30~jdXB{g)1bb}%xIW$s(bc}TOb4I=2=U#u$de)ld8u&xb=Y00r zXPt0Zzq^$O-CKP9~n?;G$JofTzD`1a-f`*h{fv}WS|xnNeJ)VbV2VgQ<~ z7EuseK4#zge6@aCt_tl>1i}lV=crwm7}bv6+-PSc9pWuvHg1rG(E&sGb@m7D4Ia|D zF8ZL_WC`WTbFF42t30~xn@>k_6Ba&jWxBwX;oz34;p&$&M+}S?^mb<3KiHR~{6WMr zODK(6jQ^}|`NLDI2L4*RrD`e`@{OVwA0CL#`n?P-W0{r6 z1m%?uA(7>CfE%VoYck&$hDG$@izx^8A|V7d;tk9l1VOhho-5UcwlJ%tWm9rCQGhwB zdZ8#4UZt!89#e2jJ~K|xRGC9OcT}_guxumj(%lq*-S_;Qe(Rs?+^<9MCmcYEuMo34 zq39xDPU?K4;fl)zn&^`TLfF`e#0-z_COyOdr2Uw+CBuHuj2C7xODipX|GCP1WJpvg zZeaEeZbj)B*mkQPD2cp}&=m=xp#g$J=8Q!XY|!FMr-lRh7J$ThJ4Cl#d6c>XFoUn6KNuyG)AEB zNjE7k%9!xoev$Y>K;`4(lhYdhvp7iZZ1D34{Jq3S5IRVGgbWX8314x@RL6Cf4&IF( z)7{a$cGR-wL6=u=HKcoT{OW)pCC+KKa8OVAm<|4MQSOS8!YOAC;$qCm`SUbS{Cp_Y z9~VpU9GE0SqOROj=AcuYnyR2Y6E7o&n8da}dQ(QjBKJGwjutCFPz9!P)ho2uRIW9S zU^;k#{!kq~ zmefMDsIOnT-Ructe4-DlA!#_9;SUZ&YLp?<|?P8pw%ryptK>$kLy}G{b_03yv;h zJ1KWkEq7Z3ZC;5jo>u-Sv&>BDid1o%?)r;^=rw;2?Do9%OFe+20tZ%EW>bE&WWouc zSh5*apV_Bn6lhv=C+4X=cz((*s*?i?=kI2%G*lRAk@?yYzb%#I);8kF}YZejO8TZa{kYL3{ukOir`6u*LNENUZc5H-Yxwkn$Rl121ID~#2aBAxYn*;Yd4MjjD zP!xJU1|n`!)mpF9hGzEw4D93I**Y;}kZ7{Jy4w3-rx!jzX`C(-lF_5BAO1B$RM!$5 zHWZVz08&j*xLC8_c)J{+!WOEPIiT#=#31Ba^Jw3Q78TqEQC@Ru1PetW^L;nQXQ1Ic zu5^igXq8csL)JpTNC9*{^dyUg`w(R2utlg)*QD?l99cDPKNvGk=c=Z*GM=W3GHG^N zZAx=D&XC|!AM@872rPfB`hN-a;e=N_YQ`guJa;Mg(3W`9GbjIntPu z{6n@p73m9erwii|u)vgxhx}O3kUQLGy8#ro0y7-j{zwY~xtl*hHa4PJ1qYr*aa--6 zVY`eWsygU#VOU0_J&Ku+p=osh?d+dN*ip}%pW>B1nYsn?ci`{(7X+Js~5 zPo31tqd<4WPtS96Z-kOE70Qx&_iSjJg~W|C(phQI?_xv1MUEEKPt#|?Hzpk}I$5mz zSvL~f%8itHqlb~V8rF>7vi##b3oZaK?Gij1YLnrYSA4Du^Ri6Z zHl6u-qr;0Qql^vJnjtMx<*r@sgSoAsNn_~aX{q{kg8ew&S19)e39jdg!#4#7M9My< zh1Rb*7@$f^wo4ijN>GuGh_0+_F9>D}@asAIrEkXB;=r(5DHppqhB&pw=B3p(v-pfg zV0`K!=*1p%z0&@z-0xH04hR^i-&7T7YqkVsCzf`9^vk4H>Gyn2rlPp|h=nqOVK_o% zIH&6NIjQ!B;Eda5B@M_DEHqdd9aFe3Iv`&rZSX@_G51~n{=CY<6fj$K6Pq%SxOjj9 zf@_V)X$7ug=B)snqHlT+7>{M7hTAA7%4VVkxt-b)BT0ziEzjBmPG<-`e3N$v1-n?@ z!2!ylR(OY`PFX1JV`J}9;~_2VfYylosfO?g*8BGpp+fRLpI$TG5w$&Pu>V5*<#o8+ zn9|a%Bl;D9zJ1l2b-tY?w-dlhogjP5wezLHa2UO=rk^K)w3e^QGv_2XY2WoXkYPx| zeovmtvHprX&=x+JHj1Qo&{hFDIhdtT*HHd#dMGZiYjK!w9r^uep3{ z)G}hx!L(kdEhG;+Z{pJi$pZt8l5mwsrokGQX(h>``ujV&SWC?Zcj~!B>U8qlY`A0Wf4$f`H6*ShSCySX~fG3ZlX^bBWzee z$49`_Z3i*94$3T~!_Bz+6;XC8N988r{2a!L6Zf-I?Yh3KE>J`|hoDK-hyzYpWzAti z#saNT2LHf8R&z@ z?1h}#*@ep;3&T5P8by|56*BwGcR#3|@)@}`x3DQZGZ7EM-Hh6^S@vlCyUdTJYk>jq z%JsH81Qq_{e~5drm*xdWL=3$4_lFn3`$9XF|9GmaIrHsT)WbtclNn4wuO zSKp$853z!w-_~!s&mGwvv6NY%A%3X;>_A{Q5|hSsz>+MzE@W4c@^!!SnpJ6VNy%&} znqboPg;c0E2>_bsmPI84ox@F&moRFwu2^?iB%5P(Oi&JfCoN~!sJcRQhM(R?wsbpP zg^VZ)Z!LBIKKVujaVCSl8}&Sz-HJmq!E#7ruKPMB^}k8fhZB$vDb;o!G*ZYcDCk7x z`^u@!zvWkolhU_4iDAzv&j|=8piaG$Sg&TJZ%7+3kD@J^UFg`;loWBJn=_#TCF~i+ zsAbAftU!UCuU&b!Q(K+hOBob+2C!_vvz{fp5$YNjdvb@X94?S=B-~o~<9;Xp3WyY+ z-;1n=F>cdSUV>1d7N_c#WE$i8ANKf=zf+~em7?_>xt*b5-3@Wa#^LZ4dn^V4lG7%a zvgPLj)43gSqd)F-5;dsVR8Y}j3#D~x$h6HVJz|pRh`2H44lY#iiRq%A;f8?|OjJ-t zX_p?^m>H6-;w{?&zoxAQ&`(UhmCCXptd2ZUY{laUS79G|XSm4(^hCEMap(TXy)a`) zj*4Wsva%ge5~Tsf_aRR~i3x4S;ROSshLWq|vLJYYgS2=0TTiJwWyH1?t59DL2uKOF z6<;rC7`8-c$9$C$7&T2gBc>z$IZ_4M^Ktn{*$n=Wj#kq$_P~~aSfWv?D88Dw%Q!`rpRlT0(Ak&f;LwVHXcPJ5abr^p6f z7gDKK&Nmk0AqTe$zlE6gz9&HS!h1fPlr9QaBA9;?ov-5vHmpt7I1D~2?N>nsn-j{^ z6Cw}htxG-9S<5wNLVSC7W{?ffKGQ*;Wk@K#4bhI+bQPji>4Zf{ab21*!N1wqP8SHM z^a8SS#}pdH+Ml8ay&ki+77z<-lb*!!LG-waA*9~ma@y_3-IUv0MkIeE=gnilIXZr! z{Fy39)TD@E^^g@?4ha}hHTPi7V5%aoXt6WX5x`Y~ivnS3%7f(cXWVKOB}8pR&;==V za6MfD-}Ms$K%N~F-xj=EDzM^1e=6_p8cCGMSJ!%$RjDBo>57%1w=v!RfVFfyM%BOsF9&x=i(tv;TxcpK^&L`}-W8(wOIB zusiDQ!HXiRr#EZc%Sx!q)Qb4O3qSiD!HU6_#WTndEtl%sKAs1Bui~7oeg3SoxJtnH z$k^T2h@C6ufGx|Z-6~V=T*)?5Y8_%zTo@*L_C2_9AmSlg*_{;9?+Uk-{@;?RpMmWV z30Pb4@z)*yN10FI#-GP$Mx9p8%T^+3uV$Scnx=jX2vi2M)lCG&VNSpH&%C{26u-lkBKV?`qC{#o*m6Ur8t*Ui zmnfJLDu;knj2&7KY|mi@l-Af?GXk_(T~~=nxlJo_@ce}c@7%YsO7*3MmL9lwxihGB zADZfO6S~Cs1cs{&G*;}JCcSi17y{fsH}rx?z+pu%{*FA^rRa2Emj3E+~<3T+!iSV+j70{ zo;_7{^dI{8pmX8;n|WqmT;5&8GkWY$xwNk)o&dWl8ri5j5Zu)UENoT40&zhjDEYX` zT8@^ym`Q^3(Sa7LfT03v??t9r6t{oM-X>Wp#WP5qvPq$9HGzhLx%)}Vg2u9207Ugz zxJ48#K*jujLR<3y%sQAYNJWBCbHt0~YZVS`&GDz}?W%wJo-Zb5dkU~1*yRT+Ckg#h zu%~ytIRiYm@fd){k4#PZLii=xtJPv=Vkwj@BX3T_A_IaNR*10t)d+V5Uww4NxBopaL1zf@I)5j zbQ`>$hv9r93uBY}Sw@k}*DQF-)p||q_uT2jn38T)k+Ts1+Twjf;q$P!Kx6kHwy6?m zlyBh)+Xg%+?jD#61veDjD{fU%{w~6;U&#!z83=@DWkmx~0)1A#8#jbCk{tt*mkbVe zP-g4d(>4`W+Z-X!38e2?;rA}sJ&)SkZ(aI#HV1sKPy{v+WA-=vFN{sx#yRHCT`)#< z&+^cQ#oSQ1jJp`>L#m1z(oXI%FaEgINyfoe;K#^fC6^NC7jsMPW&*9su^!7&%bwdH zvHciGDSxYM0QF{j1l4gR9;*dFD{I{o7Opr}VJpYM@x9^Ni75sUI4VNg=}Nn9M3S31}X zl>4_Cpd<#f{z62lcUTT|Sbf!_?T+HUQYhfKwcQPSCiQB#$k5&0SkzG_MmA#EU06b{ z(_?i%s=vTB`JseG@JN1t^A#@r!*VFt^O4&TduAQoagPCK6XL|^0OI)q_nI5D-w&e~Dvgau&_hm4dqzYI>d9BLAmkk+k zxN+pKr1Z&Dquz3^R=#-QAgqrHSnZ=kk1*CVyvhSida2+TuGB3Lx{gu_7RNgf`Di1?wLgtn^8sT7GeD`owdxj zz<0BHAH#h!%Tn;ZQp}e{9yT??Paf1bbd=tAS)Z)Awh5Y2Wt$%x)R**7E=XIPIx-0X zWN6v3!#eXYRTs0VpUv4 zEg{b{(C|C+o<_aOW(xEWlFFmIBVlYs)wD_v#t@ zcbG1#DO$MVO*Q_FG&%V(6(`zanhgSMtr|h=B$4(6h}f5-U)A;N8KctP?2F#>e^T13 zK6?z*x;8+2a|QtI#60hbkLE-^TBlT2u&sEuZKm+)G5PrgXu?nw_SjckH^y$=m~>tn zhsdZ~U}ID=`(FJ`eG71l)a37PxEXV~E4k6G)LZ<>xR#9Z`zr{kS);4X0%{^|L=q&}yoNdD{`yvFt*2{K zv#=c7bZXl1jlCqiN<;`$7*urc>4E9Ek-XFUY!rWVEjPAnWv>&>O2WZ}E?%zb>daP! zCzJR}5c%P`YrFD2@Vc$)K4_YhPfM>K1&cryC@d@JP7>Q?w;Sg2hM=5isrH0u^hbz9 zVag~xA**wz1})YGkj&syrH4D!f%H<15O(w#IgmSs%d1}hZ>e8OE+&V`%H!*W$x?%! zbVpqgW#2Z+)oQUP;zXMSEONu6ap7uU=4jtNEX`OxrL=t;5u=hCY%MICY!2bvkYibt z>t7^<)q2y)Nrc1HMPx*{5w8qxq;N69lOJf)r)1pFzc&$TF__g)z*Z18QX)0Musm|G zk!90Wpy8Sv${?a6kp$l#L;F`pPucmevS1XysVro^r0=7GB%MI;s9ji zgRYAlexHv4z}9VQ>}A40Zi^q)1}aJSsYCA!TD20xqoc)Q==Y^~i1unNlWo=(F)GV+iBoAb(Zi{O zTqC*NF_>@Uac`uqS(20an`;kJrcW37w$^|(kOG}=2=fAHg z#CT{^=yn6%RoofeHN7-+P35YB`X(%4Gnv<-6dOr73}Bigi(cJh4~0lWyKkn>`>S@u z>`A43eRmF8>=ljMz%8S!mYAH>*xL&)F#}Tc z@-dSbjrG11abC~E&C3em8na*{wi^O~lL%xvy}N0P0!G0)L1jsvN;~V#jfZ{@rW`fK zO(K=#Y0KuH!(*Vyho^w12D*3F|(E!ARG_;ACm>M{NOG(@-TrAvS82=UlIa}55saDKZ^ z5Pu|V!iQtjVAxK`!GnGqaK{+R6k5ilE`5cTO(|JhV%eTL^@}=-qfGFN#hnYue#5bI zSU63|GrF$#NbA|1g!us+`GF$?@18x!xth^_+scz`7#c!(_x-Mn1%cTR{yJL;%2VavEs%>H-PU z<&t)4g(h#H2Tmw!0x^hTb_Xpe%TXH2sU^nk4|1M`3h$@r+0^)sms@CTe;`$mcF0dZ zJbbP|>P8pEz7-E#Ce?+=&AaMA|`M(*o4DsQ=pEs~1y~2D+ne1yb+t52RVnf?c4&`A>S8sFtB|-Zs zz;o6=+K;hlP>{0SgE87{(p}BB*KA;6J5BlFv448XJP|g7@x@Hw#lT{cM>YZhs7f7U ziuP1)^WnQoq89>ZJCPTj@s;Z}rXBI3%PDXCkdE8>R)rO$27I#=@ntgT%i+%srd*8F zZ^%Bd1|Y|w1`ikajrXDE-@V@VFE@!MXd=;aQkIp%@DLs6qK8w{sqd-fF~G-<2@URh z^0@6tul+JWV;9bdZd2vA#HV7(9~^=tCB;w$@#7#9)YX$;g)t8g@a!s@g3#`?v)V)% zQUBv9h5P5>T%TE&+p5d(Ff zYa)FAF3T!WY`-Atg$o-w1VWc0L?mPT38+cH`ib=09Eu{CN~PY003@e|wzK3E@li&5 zvymPSyRGEKT&O5Oa|mV*_TsFP=@m;x9RJw9axTW7W ziz(Ul^>qLSupBKic*7)Djq=`~UXIB}A3Fjx)gWkP($IMjN}N6UXg*Am;&CbBVs@$g z5J{R{HSJZG(Fky*F|HgQqY*AiFX74NtOmVx%8o|D{o2W8Eg8LGyTb#6@(|{<0QBBh zU5)}!;uIn!Bo9iTo1FhI=QjtV0T=K@iUDJUeW&=#j^gg+(UYgP=OoBLzL59hU{efWjEkap8KcOWVy)WhKORA&I&Ud`6C5_^`*gB(FmD= zH2@}ZL<%274HeTNw%_*1f!;A~eLHFQ8&^Jxp6%%hI(`!GI56}&GF=@jGdl(_$mF;T zg59k5^OOx{k%nQQr4e|33!bMC`ocE*TLpO#2$habd9hrS_mEWMr&k^wH zIf5{_p#QB*@FkuYOy44J!mTS9$EF$X1M#H=67KO*a(5e4Axs&*+SW;A(4t!q?A8M^ zX*SovxsJuM7wg)q`V1<^JY%l0rt0*K&)!CWt#2H2KXqx2@P-0Mg(gL7Ig-37{qq~7 z9IdK|$9GA=Ss(ROMai@!C3~<#JNpFODbFq}Q zQCRf>{Hq**PPo2d3TnTY<-f=xN2p{gmuY2PtJ_Vfw4PvGc*rsR2w)gd>ApmP6e2q- zK|H;WMW!vXla@hr{J3j%FRJ|~UKb#CMLC)}k=m(x)HQD^4YN6${kcSL1Wec62Jz;V zoL6L2AMTZY3v`ulEBQ+CnLm)}ekfZGs1md46GV2)s{qVNfX+%Ow^_AC)=gE8)AywB z{wqtz5d~K9l_b}zKZ{~J@7^I~_OxXtV9=|IQGYz8ecf%6nD$9We_Rf8s80mmO}3=O z-Lc5Lz%F-_Frcn85b3r$FxddI5v1#XRXxSOxk&u1W3Ld9uXX_ei*f2`P28goTbo9P z4PGa-w^3;=w@d8pHD(&y-vIz8O9mcyuZbKNIN|}Wz?~^TXl79G|4`guzO?Hh`Tlz5 zPnGeSjzd!+fRh4ReJVu)4vVVjkwOQg{;Wgp{1pF_PgoQHmSdj)4lLadCSFKlde-C+ z55~>uR;Qc5%$;ugdqzaPQ8!=q6CJdiD6}XI1^K=`76;@-l}XI*upU?JsF^0Y-*st$xy3w0-{SdcZhm3{5|%?bY7ZS7Bu>?oSYF9X zNy7lQ6U5LH+_|;&_rC@-_9YzM-CD3#!SiTKf)EIu_l-T`hJW9QJKrnzW>9)jX<%Es z@OfhSwdDIg%-Q`rclA~qrNv;wvFZlUTN~AAk{3nXdn!R&IzPLQNZO}eD;e(8GZpT5uK-Hl@#3QN5>c<1jEF3eu0=QjkD&xPGB-7zaWKSiwiBcm%8-y%WNyjvA!}n!BoRLUf%n5DqoG&p!XQt;Z?#ZAe{@O{|qPO6rno9QfG32I_Db)jnG@fw+d2=jeuXWekA$wrs+!kqMa zP|*1}Tq=U|Ueq{OdC~Sf@#qKs;@SfKye=Y3pqb^_pKkd_Dm5M){GnwUZ$f@$*G%wr za5@#(7Zp>|1fA%$49OpJZoC+ZHcpOyLK;tIC@PKU`tbY|h}n2AUR0~_3vsiDU((`@ ze39evpwm}bQ4`8sD)j(WSZSQ53Cfk)hyq9k@;@hZD&pdWt`r5@W~Y*!=6+7I1gI}S z&QtgERYW;~d{!R>l*o8x_%Ib%>FGRIq;4w<6OJd%OZX%|hF{*ET0!}{o4Ezzx_PhbE zeJ4&f2nVxm>H;-DLUw$T*O-h~>4)J6yeo2$4J=scRcfJcdiCN0$?xU&)Ab3V{%$js z((gJhRBpLtfioHwexhpByQnQZYUJhW7xjA99;*Q>35Ixvu$f@SnOZ0B8v^V1+&9NP zC<_|M3|(Kj@2yJnPrm#M!xFwGVkW+a^4K6m3}>K8J{$m*_5ln60af+o`lCLOK}f9Z zp?TXf4RFv(Wcn|~?Js;9oHN2*gyT+bH|+F28p>3N-7QNm7I+mx?n=TT>Oo!$xHZcI z3S3H}l>9cgIYvz7+&%W|%De0*vZt56vWN6_*3oaX4~>EZZE@dhqs}bDgi$f>F!JwF zLia2QH0y*Rv#x@7oAtIsQmir;Xcz%=$9f`r)#i`OPV;6=261yf75!^cGjYqqFrdkf z_s2Xj91HT-i^_@8BnA0JX@lA|NeBt+9xN`sD3agKiT3jI(=A3GJA#wdhlJ^P7tr6q zyv?d50#nl$3b}RkX#wcE$)a3)@djvp_zz?g-T%IJ;jluk{Lh9bF1nsyWl?vnGL4JT z%dm7;Ht3c(fN*(4EVi45Pro)J#yMy8A7*kM1cFl z_O+0NEmEqQnY+B!+nTi-jVI=@2F@Mu#F`)e?a7aCOmB+K%r%baS`l8!BJJ>#^yMu`v3&Jglhh!Cv{`!jS9Ayz==*zFreiAt6j8v& zP?lTR^>SCXKg;XKk&xlh9h}z_i){3eXD4^UnM&_S#ntbP=S!Ha-E$VWCP4-#(Xkes8B@Z`P4q|(o&&C zLuOr{rKrvHlI zyCLMtpCtPXB78e7oRgQLx)h8s2djB@9&mhWT2;IIpZW6hKp>%wT|LMmdnc7 zKhK1_O^Vu{aBoWk6V>EpijnVv*U}pRz-C8{X5zXnCKUm47nqpziUzqdLPVG)J2S#i z<2HoPYNVj~Z`15_=X6OL>EKPfwYGJQWl+I1kGmKRW@b7azS*P=Z@WbiE9F~&0#>iwA6bV+}; z$K(30S@S0%D^x|h@gO|h<%$AxNOjBqWvjmeyO2VhY0y8QuK2sJ0ypUlH3ULl4@g)C9rQZWH|G`i7ez#N`uMjtc=(&5L+enfj*N8r( zPz{jjJH^TW2<;Xz?2nor=C*%wPCakM_|$*E%q4I-6WlpvzO~>%P>0WR)#w$3oL06x z-pX|AluC3wD+JP93 zZ(7Lq;x{&|3aknp-?R_g$7B)mg1$z8z&;{o9xMT(cHIB}s2y{_@h39Bx{{#=!DkLV z?b~ye^2=5>P4QK1>~2>CI;M^;l?arNQ$ zKwP@^VFgo@(P5JM+Ckis!F!Af!GAsM~Y01q(kql zwZt$*isj()A{}^P3894H($wNqh}<<5_XoM{Y>gT9{7MME=fc;+aZiOwZ>N5x-4KA5 z^s~p%haU^gax)%(eROdDB6*(ygke!`^dDN~7KhpJ_o{`qS$K!uh5tJw+_CJP^ORu)0%js8Rc2p+tp79 zN$s)Zu&DNyF8$iI68!Hf;kH>3} z%6r1*_5&oI+Dd=1nt~D?Xy1OCCBNSidRRTq;Hv9($kY!LNTur*O zKJl{u%qA92usm2iYFp`czs7;mDJxA|S-5Rb1<6ExB;7A)GMe86mx1H6vlT1g#qs^Vh z{=Cq2QKs0#c2`f}>h>Zln@}O&orQ)04J5X=LAhZ)vh1Vfa2|AHctXfIRk85rZkS1DH@cdQIwAyL_29tUTfyc^I_AnAKHae{h%X0|5jgN2QQOKm8IwYS-}F81>$!?&OI4{M$J zR2KXiENQc_hd#_`Xd!Mx4d=7mAtnnCc@FbJastuW*pK(R6Mjf)CC?)^C`F_nP01%I zZ&qEV?{_vGzQ$6Dfcg#zYHyvWrIc5Wtyo=V7}i(9X|3LvKeo|vD6iz2D$Xdj4R1J{4CxaPi_!Mc`uh z!r(>e^Ud8!sSzyJ47!fN6w|s+p=) z9b2AUO=;L0rt%i~lB1DXE~1bt*lWE#nYmu?oU9Z@J+uqSN8A>$2d=_~CUmvF+dVa|4<8E_7w9Exw^FI@5hL4xh8!>eK*RM8zTpGc$ z+{h{D&4Tm$=u8NXvm-Q<9FMD%H7in^-r;U}K;5EdBpbOT{PJ|OHQ;=?L;)Uc{bsV! zT;wk_p=R76Hon~y^_mAp#f6^sKem!>4xvy(C&{M7_Kv+sK39@Y)}zlWv_{Y&_AgF}skq#QNXpz?Z`hK3JXV|IF;EVoILo@(ZbAB+07g znUpGFal^+92c(t6>#JgdP}-?YGix54uy2#a&Cyq}LYmX$qE(qZJMW|ui?qfGLhcLf zS9|xG^{!>!(BaqAmy9lJ&c3YyQ8y?nVgz?M-_;K_3x_-G0tAeA7A0_{=Ew5V3 z#_AO1{@0M|HNmS@NOGHF{AG%bcVCi-Iq|3Q6f9rnBE0v*;8Ct%hia;ZTV%q<@Vj%0 zJjXn_5wW|kGg-ec?=`6n&l!KQoN(M75`C+$G*vLwPq;Zg&U-BZe!c^6Jd< ziH{4Zr|2k{;17?5tVgBIlIHo6=13e2+L7ruOHV!+eaTj6S0YF`HU70|f2{!p$W5_^ zp%45vzh3WmN*oWQ&#H}7yA0gyulhO|YU&WFn(FaGr|LM%V~QeA>X0~LbILM__1o%g zWYuLp=wZ|_=hP-g5xU#rawt?a3cjWvi_Rs0jI!pY%a>j?%rmn$AQn+Neo0bblRqVB z^-{Ng1pjUw;^+lBXLe&yMYJJsgH(3&NcKI(ll(HVSFwzDeB3Q~Z0djg;#2CUeC=Y? z#Tl8Ar`-X#8JKi>@M$#oj8*6BHMA>I?eSpl@z+<5w@k?Ju&{1^l#>$IJgL4G5E9~7 zmvzkAh7uSlnwYl18pS_nMA~&81tDbH%QC%}&e0XnhgC({hYiF7txnKkgHgDFc2Y`_ z$B+5ib56K&<*-fCYE{Lmo(m4;s%>Uethid`7G=|+ zRozHrXN38DPGQI4=+Z-{MV{BOuFzB&7ooAyj=0(Jykc@M=%~_hO|909w*G{$v%LXZ z?4lFpL+6nvA_uwg*-mG&o3+{;53eDb8wlj9Qk~I(U#0(LfH$kLfdyVuOuPHr8!q9S zR5aUOpj_6^XAu{9SFvg-PpmE}GkLD!%1?P@E1O9+5#U!2pm%KZUS{e+hYQbFLz+%* zXY;w@UswuFotV`nS*E?7pRJq&$YpdL9=?TL#YZ;(45FMrx4j7t5M`M z%3-v&9j~Yz*ZKGZHby$B_atqA=-<NA1P$*&*T_GJt;fR$#y2`6G0V z0$R=4%vYs<)Mm3fBtI`lx47`Q*3ct2&9ZJ$Fjcuqon=&|vc^Tqgx^y)^it6D92B~O zpCU4OQ1yuLAK!8F&Sp6-uBwWa$nLve_UUT!iq9u6c`Tyc+vzXAhsWsUaXHn#1{a%i zZEQ6N(Ti*b$v1cvg4Ql=BW{Z-yXlmqj))Nn!RBrzU&a6*AhR zaw)tu`|3H-G31J-PreDXaUV|#&~6rNvaH9VAC4+QL>d5CYrQ9de}hXKd+)2!x{b=u zJ@(h){hREj82qI@Km9MKl3eFjW(3#RPB2 zy5TXW`=&$ru>75Ov^PZyvY|6O5!qX>5|YsE(d4$XzrKS!--qMv9q*yDPlvN@|7~{< z4aBgV1amLr|FpYTOYECKQ3@pu(QM>|ayaJej%==&mw6JAIBqmtzBw#&M(?Rp9K4xp zSx3w&WmI<;y7WOZGJRJIEK&-UrSlqRL8uY>EdOSP-Vxu26RQ*5OQu#d`$Yz$G6&1; zS84LyI!}tHrDG+&%7rCbJF%W`I~>b4ZU4P7tp(K(S=vy9GyRSQFvIzj4EBlff8I4a z&IFcao!#U$pIYS3O6BNSpAnP2dgj12`Lr{kW3D!BU#uzBR7!QH_90<;TfX4wM{_Ge z3asO+mn|dB$emc5vd|_2qgHP^6+R>RW;9&R2k+!3&mukG%kOdoHj*}3M43d7LTRSX z#%Ru0PC_&14o-RgbYxa_**CK9>dJ9%pK_w+US zk(4yzX2s_0-Yln`|6qgnI~tGFcFXC)FohH4Wque_WTWzgFah6a zK$Feh^iUH)5aZ_!`ab=o)g7eQ=J{bmdb$uC@}Dpy<`S-Il~7%q%r8^ZXSgZ5Db1i+ zZ@=SOiaiZaMwV~q5XjkooN}&0;s4sG%s9bD zwMd+E@_VB?!kJhSgKxZ__BO(YUJ{$?S+iMCv9WZ?*Q`w)D952+rzy7eMyE@@xoXvW z6|((7Lw2T}f_uPWASF?AT5EYOA8VR&@u+&GfwI@sk(vZh46# zCn~Q?d5U^o@&+|m7(=NRs=Tze$J6J?>d>4^JYLRijSc6v=cmQqhG+CX$38~mHq!qZ zNhC2CNexT?-QOc&!P6If$Ca1D*)X!ydmFl{Pt5B0V3kQ6>O$;QxV*fZ>alBdP`e&i z`+1t8(J^@m*mL9uj$EdKxw4B|Gh~B5&^f8v+_Xa*U+BO zezak3KdFSO)?iQJ^9hPePo`5YD`IBBlSxyD|NGnl>(Hv8Qgi6lC_&klg|fsvE2aZP zoHK%f4yhgdU4Xmza1{M;*$-SDzHfpgLf)se?tdPv{RcYM4XNVYrOfHKfE*H z&SLcpuybMyU^HGe#8`pIs3z0thoB}(O{EaKAC(`B0|bIcQT58>fzGv7T%#B6XAeB> zcTsQPR912SUZj|4@;Px<-S0*E9`enDRP#lV=<}R@{5y;Zn>Pr36gQ^H2g?ytVuh4q z`Wl%U&|$@-TBz_KYawR6mc|!Oj3#Cqo%x2Myp{7tc~2h^AhKQ!WURiNsNBqrhy7>9 z6-dF1&yVpt|C;d>hL>xMJG?_VuLG7}96YQ86V0y~hw`Ri)s)RN7(^CXFr5Z$sk9H( zT$J}HZwlZeYOxhZy(vVLM46yD!&FE|VOvp+iPf)bgh_>5WMuyePQ<>y1HPJHsOH^o zU!BNw?^-GVnoM0gT(`5SDWMk4U*b`ch?d(!N(SXp4mrMwu)yM`(g0yzQKVIyqqFKT zx%WQLi+nLV%H`W|-Lfg=L*Cq*Xli8j{vOyMUXr!{DCC+Mbepcw{dFw;Ifvh!K~Pl1 z6DuR}Ypn}?{FHf<)#eBU02eMx?FAwos+oHI#HkYXdBy{bGlOu6=!e^z18nOwYv;XO zqwO_kOBbgH2DlLDVT^T~ligWATT$>u-h=eul{4fm4ct3h@x6m%X!;S2-Be01X$0G# z%^``kQ)pWvuDhKzE)*#JEKD_>E{IN9kLH+(2yc=VFaFEaTq}X4nLTn6RQ^4@m`@M< z0=!J5TR=~O=fR$f_c+Q2fs2)7wp$>(yXGsfLQ0ADzOcP;C5sl6EjDK=3f6`Unc!{D zd0+G!D}|mIvV`3o$*gKWwO#V^fEO&(bB?SY2OwLI$FGoZ?cJ5g%-d!eU)MA@rKufu zEZL`14a&=2iw|j(IrN-&UeI)<{JDYswtE{15D^g%n&14#97(H5B&<(vQVzyG$adfn zX->1=Yu5&k7@XSD=A~(*PvD&wwjx!iJE9|rl@!Nt9DuPV7%9< zs`C2&NWa>*9lfTLq&n-(ipnyt_1Oy=sA22UJw>h>g5>oRmIJSUd8a>5EZJ9D79C$` z+9LjTI1Z-%__-mB-Ijc2lnLQENuV%dqczMsHTR?TsTy^p78S#-H6>P4TKXWY9D-N1 z?bjDw1?LdA7rqJ8zTEI@R1jC-%i?1Xrdw?`=fdjTKX~1=bTb7%nfm|Phi;F{5I=)6 z=ba_MqFHVE)dkQ_AHQZnuE5qd6aQM~eZ^-Yp3i!_C#OGc_PN`p-`|AU`~E6gIV<I%nqe)yT#oHA+EBxfmIzs6z7=w7-TFZ~_VitAIfl;@68@6d-C{ zuw@(z)*vT3NEpLK8iRr5#`2=99u$Y6C~{y(<;7+<)Twi{6dJ6i1}CGq3)LG;8rf)$ zLN@OKXh>w`94Tc~O~~d6I7Bid^IgAI2QOly&?hk6$S&?yq7}LglwKgC@?2# zEkB0hZ&W9<1YE-%L=6X40;4k6(iB?yB6|;Jj6%aEHazD0iTbP0l+XkKtAu~$ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/operator-backlog.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/operator-backlog.png new file mode 100644 index 0000000000000000000000000000000000000000..7248fbb39394b53ce28a6b38790ead0130bb9736 GIT binary patch literal 350363 zcmeFZXH=6}8#bzlieMp10MP-25hV0Mq!*Q9p-Mst9Ys0_2uK&i7C>wi>4LOCC=z-V z4Js`ngkA&;NSEH>+YjJ-=6hes%&)W7S?l;`66Ps;-{rcmyX-vHP*Xlc%R;+r*RDgC zFDYExwd;V&u3h_-X!e1>_&jr;2LIXZd`|O{{o|7`$)uMT_4JF7CyOsB+3J?kcazQSY||I3Yb}#=)BU_91<01aL3#qPivI~ixQ?^U|cRQp}ao_i^EQ6l; z#BRT|90}_ar~j2@$eYxN{HqJ^R<_a5s07z*cAohM>ipP^Y^rw!f7p;l&Q95Q4c_2JR7?hH?=ie*yP z*W(t^&r}yNI;9-Aw>60Fs4f-ieU~bwZ0;U@e+^j8m(#%d1o7;6aD;6J6ThW}DR2SH=r^XD1zshq)|7-w#y;gkF3Mr?j2E!awuQ=@W zaBb#hGMdRx??BLil#RBAtis)0ey2bzV0pVI*I)+}7HXg(Utx3MkKI@uO1T~)bIuNI z-9@gynDs#Po&Lwe+3ycW#jZWQc=$5x5;NIxvF~BZK!}`=M<8t8vvLos(5$||{$%^o zscQUG^$D?#R&uy;tU%f9*V!NoX--^0k4LcVXn)TE@$ik@KR&u?PsZ|z9AlR^xdo7R z-w66W-dJBrA^NV*%oN1S zc1a+j4L!&6+GASuEMjht;tJ;bGRrCm{(I?ecKwMcv&$hc2HSgmIln(iOI=1(QL!*k zp!kf;_fto#X5E<<3`2Hb3JB|uR_6s?wtl`tvh5Yx)WN<)A=ow4#^CE;ta{a zKh9RnQPfR8a?+()bJ%dwLx*f2@FTcVS#G&;c0{Mb!(-Y)cIMUasPTq}gZ;T(x$EGM zSC05DzE9Da?q{kH_a4!TW4Rkz?|pz=*-T(Ej*k5qzr2A z&&oYoc$z?03$gCoTFb!bUUHlySu1$rb;_qhqleCayT`U_BH90lGkNmlklVYxOm9ji zig^uv*XD)?qa4&K*5|TvzjcnJXT`Fve%qfS+G-$DHk%}#i+TkOi_b!M^NN7=x{*({(CudI@%}{5 zQ(18H*;unU@A=CX(T++3#QKH(Ml+GEb#0R&KC_9lg&$??%fm&7R+P`=bT4FDPuB`o zNZaMspKCW4^lY#F$}P9_d8{H)5`|3rYtR_j*37}`sTk1>^*A_)WASo;Z>>4HGVS1x z@+OQ_nvc(5J@G6X%UK(dva_#`B&Eo$4gX>BMDET>B`*Kj$prM!K?nGXY#W8k{VY1R zZ$EQcfoG_GJn5{WZD-q7-QzhK(;zvTF>DSr?W?{}(z|Buz}e9Op?%ZFhaq6yO&Pdu zV6sM@vF!$TzFy`w$I+JvbHp>Fk`k`9^*v;hwWGIN?O%xwf$+M0J!lEeoZ>vy5ZP-{ z-wo)AkTFo!%<0MU9f3gKFZHjP#gB5bHU&B-r<7OS%4cG7vGT{@3am2`AK#p?FEp)= zZlvNv`yCos?A=)FWq{vpkeV!=O*dXc5ZKmr^zHKSeGg9D49#`zc4;kt7{sx9bm(jp ztS>c1X57jO#3tf}YnqC-Nuk=2^+>~&x=EjflG$JfFHS?(cJtvDeFxQr9#4l3lY580 zzGt1!=G%p@6)1TVs$*0%zQ!Rvk8|+mUGc{E2VT)famKDdA#HdjZo{j1zQd-|h}9H3 z*J@m0U97hzi^#1f_!M-zUIiMt=+S>HF2Z}V#n7!gZ1mCOOlQx0{&2512xe+@OajrQ zNPTb1%%Rr2`v=<<%4EaO_cQ0YwXnCNA-~+G2MFF%^^YApm_%g+tydqN@uii?ijAOd zHm{%WhQGg1kA3k83X=GNCsGX&*Qfdk!{ZLVp0)0rXOE7*&jbL(0C!?t`=OlTyHenL zm5z}wqDhkD$80;%=Nv&ckkcck5ZEVrez&rT|O}de5aBj~9kn znlI#pc=O4OXof^KMUMI`@;kU+)jov%{Ak`=Q`I)crNwGGC#+Q|l}Cm1%(1>?nh^!M zQV^}*I_~=6*}Kwnd$iD%E^x@KF+ow!rbrw2y+5cNz@9lcQDL)=26jUa9T+pL2@T-mSO2?SzwE~=xbkt@bb2vtHMv_OWe^f8Q zTO9!gr9e+=&xQB4`!0q1evZY_%;R#ZS-N$nE9)9-oj;4Ur!UxOeOU6b5H-vig(!5q z6fn|m$Fn;^;ul4~)S%nzOGD~JW|CJXK!xhvPqeQh9>(-6lrJQ;0z4NH2+p?v(14a4 zNlpBk<-0a|ydp5KbUHMvt^*_U1%SM{pe7dkF;7+I&GpJ(ZG8X3^knp+D!1L=*n{nj1TpK^6!k zDv?rY&KllsR1|5L2VR;PnLoX7HC&*+N{KMIaS!yAS^BK5cEX`_%8A%JB)d>F>(5jg-{_EM#A3>zRk*+iymhlj?3ByBU+^ru|~(7C#iZZX3&+U{{iF?wHRHvfl!( zpOV*VQE){Q-YBSkcb4+K3ac$9q+kHgTb z#mIeCc8P>`HXPNvdJ~rH$p!6r1%HxuP@6mq+jfMtv0`jN}fK z8d)RJX-JQw>HYaY3>m94H3R$0gXr1QlrPy7EY=f&TCrw&GK_C?`ZfgORoz5xKZZB? zszA}7ba7-(w7PvBNjnd5MNz6^u0>zarl@dvB;6vkA;AnK_z|MPV zo~?OVpjad6Ajpds1PUwOa5Feje3GqA>EzgtGJNWhG(bkxRaTm?*GiXb=GSU-D zPm+C+>zrOBRZiyF^@WOx0rN%{fkE7fuo`Zu;Wsbdj$9--wws41(Gc-0Qx7+2E-Jdj zpS~J~Ke!5o$zk91A>X%*cZC!3jFGi9>FlWS1+BPcBx6FF_3<|QSimOKMLNP2$tg6? z&i;`z0eoXXkGHh_31IfJ>5ST} zjaWjAafV=(vm^2R3D`l8NzY!XvJ7pa4JlGxZf*8BJShR*1(CSpY;+Z=^Ngf#(=eh3 z($ogNK3u(Y-`#7W20nyVT4fhL8f@lCpiBRjGgaEsx4DFiu1w^O&NhB50%7S z0duoj$+0iX#kP(gAj>qkXt?^T%x_Z6?J9;TE}N&G^?WAoaS^x-*Mud z>Yy)LNRws2t)u%_14*eIx~PQ zcqj&q;tXuLKh+(WAGUM{(1F;VsV8oAZC9w%6RsJO9%)?1-^rMMfz}gWBZduY5}g6u8pqP_Z^^-c75bh_4&q z!yq1rRB@w|Lp#UdLyI7e#pE#tvMYL>qhf^}eRM@+D9(31XocVfYkqw0J`DS)Qj@;O|4bfb@;|7D!lbd=A}du(+b zz2w9bLj}_{yeX-)t=1phII8p;TZGwQm>`S_jFDHz858ybW>8Zh_mI`YQrjIXo`AdI zMZ{M7u732bqghuxGs4NX^eHIB`^QDLD-e-PPp)#GIWb|I3Rlh}^&Px8RaPBlUv;ma4> zxdhBLqzB?jKU-TO`b+U>W~b}&;;M7pO4rx|u47*{vG(0V;3TK)iieX$27@XIMMJSs z5K|;Bx>I{_oIL86u*W`x_!A zLC&t(QiT=g_;L()bf8yN8TpgvLHR`Cz>_DtQ>Rw}0W$9F5vvk`8CmP(Afk@|rcB0K z^KU1TpAm};oyV$=e~=hNeZtPO5Um4vF{lu1>;gpP_23!WoL;^2`2|VgWn<<%UHhy) z(*!AAdMo}!cKOHH!RxkwxDP+SD)4IE1?7^sT6HJ$T(8&6m|mT7(cJa) z5fpjScFL3Kbvexfp=A6$+xo2zkdNhNf4>gWl;&~g;rW$36uoE<9ofVp-6F*xNw8r~ zs&`54VazkMu7FTM@gi8iRPDeSpTQ;T{B9XRBmHTCd2DHhvd{ec6gO&}0gl1sYWo4qoL2@8SBuw87ney0LSkv&w zyCXigS|kN8$ddDoH2F_oFkD&$tTi0<8S%vI>N`nB$9kTW=;sjMJ07n5=nQrS&jMSt zM7>HY?JZsq)UkQ&tOqJXqdw9l0#p{{r;UIT4Q6m?kskfL9595z(Q@)J_T{F?&ZY4n zoXDJmf$p9X6TpBnX!XPkpwI?Y1W>mT`9@>pV(g%Rj)?V~Oj;yo)kPRbjdC3fUhdjv z`jA|8&0RceRfgxm@e)TLP46>Udici=9~2mcGMXOz)dMHw`x6lEFb{-w9k#PN+n}De zg%MsjEq}A}$o)GMZfzz|8f8#tx|8sR}AnN^iI`U8e$#&Z6u!-OTXSfGa!!^5EFH2abb_)rtbQ%AxI9$mQtV$Ql#Mc~tVs2DLEr~H#0 zt4=smz!+WM;XC~z?#B5nFE28$@#hXaT$#Eq@=59&)WHxLKs~Qq4dc7ai^G3)(o!Tv zD}(e)1kw4ON%1bZ`h_P^axsmJ0{()q8+mUuealX_HUQsBEc&V6* zYx)7y*7AOU1B!4zqV85AdsDj~?Nk#@<)tTuH^{Dx9S>(pPiv0hraCJK!d7^?!fb|x z46ov1syGM!*7Rp?K{1zr!w~4>OU;YyP}Gkg-Zs{;=ZLhB_k?p?dPy@mGw`miaL`dE zg}rYqUxkMBuZ{GiqBPDR4Dx6L2+?F~3GxcVdxT5N7p%W-fJnkoHFpJOo5`|;hAMlA zm|GyndG;qKP_-l|29EHZD&a!k8pk)OHMJ_UpYEs(eSd>H_Kun4?3ar+#Li&4>lK>g z>1{08q*55iW_mb3R-6;2=KL})_|7?f?}uiODRvxL;(T*i2jmp)?NQ zT11vL^alq63w*32cr>#_fX)oAyskDpUvztA4ehg!gW&`!Lo#@?7%Wvjy$A{-T{+$6 z1wDE$acr$HP~(^P1qJHmfiZ#K6Gm{s9=|4aIT_G8@DY)P!R5mQ3?oreSQe-l9sHgQgp>)tF^kbsZ-EU8%CO)BcLzM&AYeJv(_cTu9P9h(*c;q~^l+ey} zMa32v2BtdE;oGT`%jfg5l4yd+oQN5PA890-6q-O}FjV8nYd9U3VqnhQ;jg$0D_w<{ zkB$R){`285?KN^G@`Z^OH9IKnWF3E9TDoRh^t|pecMRTOr3w*Zzb1=K^j@kpRtsbj zRMYGOh9(m94e37KjaDvkFJB}GCf}<480$IlDnZnMwD5-cE@tv}fMG|lHa{$prK9RT zorTw*N6NMo;bc>XXcZ5knf;!)D8v8o{D3zDDqkBQN?4Q9UaYoXnc34^(#l^|x5)@S z!ifMK{6T)5aLI3X_k6XGTWKFN?5uYl24oP?7utJ2r-0_J3@EzV4m0<)f0SFF;`e2O z=V8(Ot^+ZsxvoVSh_=t_fQ-4KXF_SG?sGFeAYk}zYxvQ}axaww@!Gi7erH(nqh4cx z|FzvbR99gQ?YG_^3STiSI%G2CZ|OVVcf|Kj5%DFj3SnIY%Xd6X%Y-%M+)a(fj6I`FC+Coig9M8AvT(Q#aP|X19 zpmk1zsD&T%lLJAC-EC5UiHG4}13sGFJXNCPIX6#!!cq;CtHRJ4VMiu5r`b$^iB%vAejTtJ;W&mWKZuh#jkXNVtaVW7ziEK z+OKiq{d$;5SL$MP8RtOvT-!(4{Pdxtz2o-f#se}elV8lPlG?6Qm-g%FgC3y8m;~D3 z!Nr^;(<%QuN8~0PLq_l9ci*v?n=Y}mhcMC-B92{k+`moZ05S2CrBN`C3kP*M{z-=2 z4=i<le*amP;z-2^ojCt^4ILisB%Z7sRt}Vkt@9t6R;gXC0vF6r*k+#}Pt)J~I)S617WCY1gUYlyDN9YfFMv-T$?CWv zR@!$+Pv~^VUAS`e<$&abqgP313YY_1>@`)f;x)KEwIqCkhZ?m5Cb26&Yq2DiC)TQ+5Rr$GLEUHhvK|8Ui1Wbp!7NNi^5bPWTl5M%mIdmrpvCujq{ zHO`~^Q!%}k#zBeq!<~ui-sH;bvTkEjK_U%5mWvd=A}>d20ki&b@v*ruW{?MDP*dP!pZ?Y9IH)O#TKYZ0L2T*T077B z&ffL;j2aC`uOX@4%j!PUFKmb>UZAw$mRMuA0#cE+afcLry|B(+GTXtKYzMKLnQ9r%v#<8Q7r1rMV-IC76ZAEHoka+0?9;F zqe=c=#x>~F}XWni=q_M+lj-*4x z)+^jf5l^VOf_(R?){@k~i!}<>zl-V~0A*a=PHkfxs5vIl@PT3%6GkSV)<_YREMDM> z+Y4HzA<34`x+fd=jW>q)+`h>i>Qyt7PfMkEBA4b+y{iH#!ZI`;ln`%f;c0gfh>4N) zp!2iQ-jh7-jKlgUn()O|Snh2Ru>Fdy(n4}!!d|Yikn}mFvi238V1Dv`b`mfYSWVNYfqQr3%%Zcb7P}u8w;k*REQj#-6vUS16Q5{#6 z7BlZv!;|u6GY^b^E1nQ)h99vNgGwEqteKn;ZuNjB+FVcDAHVkvaDfD^7wB-+Q^{&7 zCN(^YZ$7{OkCzD?O(0X$8{!nXz|&Yh2|}wLMlMtNPNKCmDDe@obs3;_(4OSh<8Gmi zdt>0jMfE2bhwvobw)5>B37Kz{%sQEnT6~`RQtaB8g@xY}HBNt)oGcnOHxvjtH1I9T#TXjZ@6I#9-5`u5 z&N`{6s^;O68Hi@gpex+bMASi6(Z0VETa(ypx0G<&K!SnPfG7{2s$<|?Qb6U?q%(X3 zaj-St<>pjzVu-AV5?xiq%42mev(^T|s@t%i)_-v*Fu#iekFt?`)H*RU)s4ZRVwAe% z5G90?)8z%5%_M6p@)d*4%m}dD&bQ{*Q&toLYt7{Gz!6Y|2`t*OgUderlbrJ>`B`dl z=K@<5cTGX$>9*7d8b(&1#lEI+q_+c5i}f^LQ5fV59=ssl=o{yPnT7uV)&KRukf9J% zRB~@A;d?Nyx==%1yXa5seTAr^`g5IpHe^Q9wMcDhgm%s25h2Z@(@LwarTwWvMx#2p7|GY1FW2iA}s%+9b%k}Cf@kysesC0b2WnIAG86)l(=T9vQ1#$v^ew&%( znPaHe&=^Q&tZ}IY$0`%5vK%yN3kd3HBx_?8fdinj z+80EBy?|&A`d5+VhGIQg<^z^5SauhPG>Bg>4mAsC?Uy;mFPZ3J=pOAp`0AB;sYrv& zLRU@h=%7^kXOVt16p(&^nL?z4LH&+XkOyDM3k#v_)#3V&cfA7>4=?sU;)uuhc#OK( z&M=We`j5;rflijw(W(8bmfQtPgHeJDC7^J3T9Fh%^MD{n5Yj?wVQK`SF5p`Yd1`8o zH^SjClubf{dH_P((1kRcsw{)B;cdI2kUg4_+fY-5fPHvWc{RPok|1$?t<%25uCr62 zk_Qo>s2JnQKaBPk_czZ8Z)@${SYN=`AkuHVe&O`y&{?48Nz!pt5@O@|eC|*pxRG$XiE3bw@gXFkN2w zC;j=9)M$}b*zWYy)TT@nk{X`CZxG;~aIF~{ig{v}l+dq&)OZX-J?1!gwvH2fKxWdN z7}ay(&dEF9{MzLj(RQ*I&uD(;#!jEbAIfu zRvRLZ99aMb%t|2u-G^p|^`sz{^E+zuMbxjKZ8q_AOLH?)XYh&8=kk?_sJy_a(MOCP z`$4nXAmw-}QJEB*o>eZXXr2OwBM`X4e$1Y{@+GtiJ}S^Ps4@%}_S(jf1slOuht1@HSq9LYMky_+9cQI(p2`Nf z+GZQHDK3c7vFv1H%W!zbsyV1FM|}kilZ~5pEv!?Il;410RS;evDz=;Ix!C#_12?E>j z;y_4Tk3Q1y@dz)Zt_OD zJnQAzlO@CLx@YrkXTaO&>&QF4&62isrF&9M`45-o*}*5$)1PkF34aV4yg829<@fAZ zAvsL{D=6N?W&-oJb$~4z6yLT&gK?dd1&_Y_OgGPCy}V|kdh2AHM*G$}Y)oe$(~;K# zT1z&|Q5^CncxsOCOR)>37S+kF zJ?=vcuT$YV2Er>#prsuJ_4HJYX9A5^IiLoD%)_5P;}SV1AUBr32wwcaL}2F^Ye5B_ zOE$Q?mZ16=$Lix&p!ImX{!k|v4COq1YkS_%rATHR^zny5$HD;P>aFh8f2;|7m-4O$ zQRs!Xkvln7>Q2RcblV{aJmOyiV7w?4z9ZpCc)L;*mLE8P*u?fu9segr<7Ul1N%z`_DH)Zw!$n4Oc;lJGk z^rivMTYy%X+u_y$=}CfSMWA`Tfn0Z~)AA;NQriPKxeaLxj6tN3ePU;#hu(+~rE@1e z=X}PcicUc14CKL}Jx>VaI=`3I=7yA1*wM2wx2g^d|9pQWE+_Y)%$VsghBxyPDEd@C z8gFymyjLE|wwARscWi!bUR9}apbNyM;kN4`st|U6g(`EK13O@1%Qto2BxiFa@tx50 z52Gj><(2(f^d7)xMYEN!n^M}wuZw{S^??dI69%Ca<)2?*VgdrmzIjW&pnq{OfJJ){ zI9>Fcz5d-=4rYL3AC;`Z_o4diKfmCM9gwnpU}VTYUkV=Q=LZK6-IkamL`lIjg4uVb z`tg>H31>w`L%F!P27S+K|5{8$B3R$a-QLj@URZgCj?bV_h+prmf{FFZBkes~s*X4S zf@8a1rvL>R`*8t9tKto=ZteZxJ@7W6=;xx84#7ll0~xscyXkLjD&7J-LMBmEn9^(d zm~^vE&gVZiuMJam;6l$r-AynWba-Rm__oxUrNC2jq?}H0{o49(6ry1qk21MJN?Z2% zW#{|U!C2tWh7Xwk$Gra6VGjn$r&l7=;||sN^Z%L+*Ej`^@W6IN5$JL8IB+oO;k_-% zGmC=tX-a9XP`c%dBDhGvEE2PQ4-NS0C%!N6H(UA7E`{z;LA@+60I729*fA?>Yikcb z^z@+~J-tmhrDxZG#=6Qwc`2Q22?_;@)BU5Mep7M(?vzwLXhuZ37e#Kb^)Og#pE!3q zB>?&21}-XpQF3Za*I)L7ud?hVdnu{jltH)jjw2dxsX0&l{{8zbBO{;g)3t@S=CuzT z^;=Z7)E`s?)*m6aD_@<06UN6t2gQEr0}{2SG|F>F+67ejWiPl$fIZTGdynGatDn?S zI!gBlF#~(P=XhBA{MJ%#Lp_3$*qfBlO6V^t9F^98sqx0VHRl!xRW3ruo``y#Jd^~b z1l}85Xs|7jgO7nko}PSZPRSL26v!i=V=&h=FC!3F2pXo?-jr`XTkn)FfcQ7RwJas* z;p70o&9UF=#P&K5f^c+%n!+M?Q}KhxN;7X?e+J^;{b?P?DBR*F05i-p0 z0J=~?=-!SDKOTawaxc5Lt@XD6P)SdVr&ZxMDi!BP$ZF(z-#Lt6CY9p-(=vbma*xE+Z=-DfQmbGtmHos|34pe)l9%Fx)qrDZkK!U z;zd)1!-8|b_U`>mp(CzgFW#zA60bLdB~uW(@RliNL<{@_$J`P^c%68+Z;^Kc*j_wf z$w=uDoRHYBeY6tU*6MjMhBNR#Oa1#UDxH8pS+{-C1PQD_Y3{~&;O|XyQvj7G!9oV_ z4s9P%t(1qV8naWnCn-7q5)eQno15GA=~E9ukREzocl-NF{`-|ew;_9QyTQ0^56!?8 zum{M0QQH5&pJ4xLDRx<#`!^{CE3FV@q-A3RuP~Wpp3v? z2V8nww@6#{Wni(TX0JpCbM9j zN|P$%I}Wb^o2ZNi>l{?Eq}skm9(>(5Yr;p-EWJSBs!u8R+)jwgfVqq$WfoBq;lVU0 zAhSlBZ{HINv=FKwK+!6F{zeOK`mTBZng#g9^j0;-wpGA<07&kcxKb3Q=Y25*7fpCX z25+g2`6T%IPJ)h?5+xN90_)u4%)oeIo6dl9G}zksWQuf#7Tl9~3p2lM4)~+sytDx& zKw}rWXaXXxRbSP8=)|om__vYSO`Q6bRXu==G(xQwO-aJD0D-VaKG|;*uR1Wf)LhzF zLNT#;2gSTP=bY^Z@IM|~0XqUj!ICKg{dZFOX{=Rg%Y*%&I15s<*$lL6-2qL%@{IC3EUGNtz% z%&P1s1U7*Qs<*ye+}tT*Unhp%muT^B*PE&}!Fx}tcgIqC#%;h6rtfUOG?5SZa7~Jh z10_Ve2{8n^vuxXz_y9VMZ~3I=0;RXiDg%kgHB4+9W?cx_>}p{NFQw4%UkPR5hK--9xCP@+c2d9V5)ldn_bk-5Defi zAwdgY9|#ML;W#TACEQwxTwf}LQm31_)N#MOepUsn!vfDmDRKT{?05cv3cqePd*>oB zjj!Lo-|&TXZQ}_0Kj*?i7pUBIdGgP8=6h)-A*yu9#DW#~XX-9#Fpmh^`MXOHKv=WY;wgVZOUt6Y%BRZ_yXK26cd(5DYm52pV^lTsv~c| zAe$O9pCefxKs{1e+b;O|XqnO!J6-%E9mmb;WJDQWpu_J05uI%3%nz(pwpfocUI-)oz3&T~ZV zOP;ZN~8~NR1|Mvt(y<1kV#2q0^^HVg|*nNRWtty9&%ptkj*c=_|NwL_Th`UpZ(tC zn(+nv!07b(S5LNX{m0p1$WvBn&f3+sVylo6>`8z0u0|*QVu;S5KNVT>`Zz@m;N^2_ zkTLV_D8IC}F(_!sw7znPcjVh%!M~{rGAsK@Q;TkuP9={b_a;wgBUe2^GJR6wm*$ww zfDBH_N?+Zv>jD<3p%Z^uzg!K>1M4diu*@!1Ixek4lG}bLUjYDN`SiH#Z|!hfKZBav zPw=9HMPn*oZW9@L=5yfnrc%PA6nXGfztS2c$~;xlk7s=GraO)7b{o5{1fXLno8R8Q z>FaMieOpx_P}tX^j=^Ft=FzHFLGP7zqP5#cv^x*5J^{I_?*brU!@?yVtULGxKlz`;5d`!Mq43!r4swlOa#P*%Hz@2;C z{$vEdRI>G#|B)7`4HQth>z9KwnxN*M*Hw=ORugwXWp3WKZu@<+Mj)y#R8Rco-v2z- zB%T_BN=E0kjxt1G?xLu1zkYkGG8^OJI-`;qf`*0nl^Rmgo<-8?{|;$om5g@Bgn49Yu^ zQE}=!HvYhqnXmYn>6bf&ra74)Ha6N)E1NY~3lz8S00mT@kUf-kAq@Y-tE^a325LH1 z_svYOZBGORKn!boNXXxe!j&|jgcAJ4w1bw?Q@E%I2rU9T5{$#xer&~h8^2iDnw5e~ zfd(k_-Ywb|in@rUp1y?yV6yoHZXr?ihu`zJGOyu~->OcoT77z9ipEI5f;Fch?+jODrbrz)JQ_>+)?kZ+?q% z@*8{6dE{E1Q$nPTnOn+RcgBkW`z&)5rI@+AX86=OtteH_yi~C_`N6a}|90QS66!VBptM9Il|XurRP zpvFS1W~0KZ)$9>=wM68Ua3>mm$jeSE(BJQywNMbObV`N+=C=+ z^jM|DB&X}S49E`t+6xja33Xs;^Pt5J&+~I<_;gXl{Y0tml@b%J0xEU)c%Jn#&bIL`I8IDK`78zzpys4n@25Ao<^VLjvQbpkLegEnd4@JwhHGGS!k{iycV@)e+h z)^~P03T*)sp}m;hX=>-+n30kfCyPv?vK`gLrvspybZ>l^$Wn)D7EF_Th| zg5MYv8M3_w!*-s+Cg`QJqkZJyso1zZI-76ubA%cU1;+}EcLpE7OuE~&mk9M3)7RJ< zO&mfgk5(zKLjhPsx{YArQXDD?i`5`IyJ9bd64_d-3kOox9B?`Z9rC~*HdN-)GUe8} zw#q6OxdB?;R=)X;0$SZ;f4cuRIxjW-#8ZS$ur)jgW173&X%(kYr<18NuYi^u=y&+vgktW0bYDKAJvjLTwJDnUU<-uaD z#qj1f;;#aDG4wxsXlK_3gK&7-%jA;!_tyx!P^+10bE#@(1QX3r+inw7U3(fW(kHGx zAZa9jEWOy5H36%2A3=q*v6p$qUxLjuF*vsHl%n7j zKm6oVo+3*+6)zziICbo^!XLVf@a5TO^9))a4a4g19l_gN!!|w0x_qLtO{;r_X#tsW zfrOPrGTY{KD|9RzKbEGo7wftkpDInV?^#?jz@kYKHm%zh`fntgE&n1OnS%kT2dc*m zoKi!XO*803!C)I1Qct`bK`LTnR9<8wk*3&Vwpji9h$B}? zY0M}wKKP1laf;+ZR!NGJF7E|(;r@fQ;0V#H(pw4RTTsSa;KFyg+EUbasj!xJrr>fp@%}<6f)!SuMqPeO8cJ=@k3NFjFrx_ zGZpTT_1X(wkQZ*zOY(T^a`95`fpaL->k&e9&Oou8YDK>84W-M-bl2c}3E!R|a}zq(pN0_Ke&wjApFkms`Jsw(*X1C1c8iDsSqXgAMzg}X&mJ2DJ= zOnXEzJMA;tQ?q>p?tqTwFqk6`F!4X^KpZ4xj=cQ#QjBpEvHne);)lPy*u85x(Z!}o zh57a-X_vFPC)oUAu=B1H|fy) z_f{p76ziVi*Vz8(ZY*=?{`irAr2h3jDC~veZUS$2id5B2w{GYTKDEyb66lrGamskIwfvVvh!F?5MI7S{)%mJmq(U8_`RHfBy5$UD!PSK zmamr0Ew@S|Pe?t+haEi|Wug_Tm64-kijyQ2M@9_7Kb1SJ_Sfa3=F<9!9cxCntlt=G zp4+4}d$_=sP4=5fP>6$bdnHk!i2X@sEybterG>Om?L31SCoz!xf22EaMY-3ruC3XwATaFO%*s%y$0`~jT7*zCCO3hsQwPg-P!%< z^b42DXwJcjxf5%n)L+bcT(p;lnpkk0*YQ{ZluvA@Ozr#C?!2v3OXv)gxH=te`Bez| zO)UMQ8E>^x=Jht2bblnk#B73QH{l0ErqxkxOwkB+n8)!j%)ArI^tD%vOBN20l1x;0 zJUS!S-aNfO+NGgFEfx2p8KJ?{$McRyP-fG$g`nSpoEezNq@epl)E)P)cI5TerZij> zcex<%lFm-(azS^I4()=svb>8?8y}AhxF6Y7@wmx$aQvocndJGrE<6bAak){ZX~m?P z5$cqF31ZybKI0SN+tkMnP+Msh3NPKm`y~yFMAg3KdKX*#F7hr`#NkDlBSAuHZ%HIR zXw(Icm3)^_d@4gSc4-fqJ5{^%9JwNlF!jOEzBtAYEf0HJC5El-;g8`xoqPb zbXy+%Z;X8dj7zY$o0UAq->^m~GADPZc4GKt<)X5ii_`I0)TGgDBuba(2s@z%i;xm^ zv_5K!^|X7jvcLh*zkOJG#v((l(P)LyhYT4i-jIMtq>$6-<%u1)*A*amlk(fZnb#DsAH?X_-~>S)#X z*IeFTkt>r)!e;CGY+zd+I2zisY#0K2i^XQvPO@!eVlR-=%#9_^!ihd!f;ATjY5N2j zx3uBs4<_Y4`t2OuO-!gc7~X>69CFLCcTZ`qqVjZYSjN(hWjrP@yuTLHF-SfYt=7sQ zro)|mHYyt#W1{&2acRTZ`HG9bs_N?KzS@y}*_a_D+_olNvD?G#3d#aikZ1B@rPbw{ zowbRP>Q=Xz6M`YL{nk=DT7{n}+!B8Hag8LHg0ZT{alf@zfhgMq2Q@?DNHNzq34@A~ z-1}@~U)1y;%?J1ZFdShg9lA<##MqeiNSdZ`wYD4zBbz1xvULKi1w%^HMTxaBcxHKS zbfq;&fZzI=B!KfVXmBC`v&+=+!@{3F7TcY& z@YV=8S(>dva#?a6rG6HPJdbf7rOt;Og*tttORCq?u&J`J`8`uzvrW*HW)8yyI4xT;KdONoa{b=x1yc+z;NT#sdYe~N(qX$VGeA;9NU|2{S zCp*J*GgOwO4X&v}$SoxMS`e;vY&9lL#6tFBx6RC&&iy&`-t1`&)0{|TI@Z?x%PsL< z8tttX&NBFwc@mq28-@14nFDy6cRIHG+T4$>iM73Ws4+r)%Ph?WaT4GOSwAbcMRI|k zsFPBrALB_h@4<>T=V3dkgpF-t0DS@6rsatXR-V8z(p9l2O>$rFs4p^k=WpBT9xt?B zLRvWEBOp!=-73RT{!v)KHkJi6QvOnlVZYKuuZBwHe)wt6xq_bVau%&_Po|*^>;&@Q zRn%UaNaSnlw=G5noi}hUXm&=`feI;9AYvj#jkj77URguDjISR(8yVs1F)=f#3QCGT zy^_6Y<-U#GPd7d4Ak^u+%Nk_9bBRd_#qPi#HYr%e1`Jyd{B~IL0o>UGubb15sB*Ve z-x9<{RVsO5^>juX@(lljB&@$-S>^?l%Z9F};RctsJKcxkg)A&1>hM+4q1ULqI^Kzj zkq!`tqGRS6`1uA_j`fCy`UP8MQB1 zJKdNTN7V~wTo5R-4Wi+MoDHfdwe7D=1m6pyKD^EP>x0b80Flq*v^^)0IXszJQVx#B zDAOeShe;TI=Jy#`f>F#+W*X|n5Po89Cf`e^t2~1|bG$c~MvYLBUT%}S{?`io7XNbs zt2!koU%PYj<={Q}qG{E1?6sT_$@9@%r(SZIrmgmjW=^X&YTLC$Qs13+PD5XOk=<;Y ziPOTR6)s@som4aHJ4aFXlYla@$M_HrlI+Wd;V%^IW2*-A9e0bo{X(P8XOVU(EZ1k~T8PS`_oT zWOX4X)=%$w9hAbGn#93BFGn0sM1<|aOyPwKWoJAsbEGX<^a>>XK`oA06SXu=jZ$&$FNVV{ql zet9aP+LhMFj4qh@@wL5XocH!5e1HY>Yo08>JMi|hJ#1FwsGZ$AwIG)3M-$#LGx(py zXrFSV;lN$O=W%M%J@ltaj6mcyuHe>E+{|0koacLnd>JMHGOhe zC700N&98TNQM2#b`al2mG5hF&pikG1zq5K%s{dz=ZVw{C%Tb@F?0ag)t~YBpYpVI* z@SX10`7`vhfy?1%yH4BvAI82rs_FA>UnsVsw3R3*h=@rTNr-3wE!&Ym1`r4kNRVj) zhM`r40-|CaOfew|gb^$dW>{er6tGH_1Oy37K%rHJY-JV@{e94WzwbHkd8@yFImhGk zJomcp>%Q(!Dsb5B>6}?>Z)a%NcsQC%t*K_W)GUk~d6^e~RS;O=-PcSJZ*}2~J!*5< z-Bmy>21T+KwB~m1`g{P-D#fEI)IgKA@csYydH(a)2jK1bBMQXU-8k>qrc`T>&=oi# z#M&r!-{2z z0mAd?EnUH>>Kqt;0r_wnbN{E8@Dy(x;LjF|2@c@mZv}gSIXn@#&dxkVRnEw zF;|a3#_U86k+tWr1Bx=1{l0kN_;2nj78$x6Y^SzDJ(eR>4X%Ud+8pZ;hS9`&zQ`sXk@f`I2`zXlB64|{l= z2E%bD+$gop2u@z4zlS|Fsk#PaU3zIKT9~tLDYhhYCAoCmP)D7Jl(=AMGk}#9i5v}_ zk;S`ZF-`b1s|?;^tDWKtpXHzhfeAZ@CKb=9p!J z4@>m@OIw5PT`k>Sck}Z}bX8@p7hq02J-YpW`SpLy_s@YnM?kwdf-LA=MK2gWMAYR^ zDdgNV03#SfeT&gLJH;f3tn~Pnqjg#~*XBUkmZQdC0{!~0R4xlD!j?#KIDLy&-pOVN zrX1&Gk)q7OfjnV>HoMQ)xgR6`8L^CfEcxtIh8s?0bMGnjYu;_TBj3RAu0bVD znS?C_`$7k^9|MPW_*%+oOq<-}uW$vHO{oD?b_s4I7b`2N-qY9FT32bIn1ed+luL{t z2fOjF(s`DklZTNuqj!`xjh*014KqV}C>W_=iW9~iTM0RS;q$9RVIWrzR-MjQ*&_Sp zXA}RyhR_2Fwl8E2s>J9(NC^u(r8_yrZHADe zj7FvqvhG9MKX8C`!ywXsa*2dG36U=`KVct1S9)48*m=4{;y4qxq!Ceq@@KwNKF*bz zt!D?d!LJ1WfOB}Hhd%Skg`((IfV1}ns_gj3#WvFE|LI&2+y^IV1^j3foPb-?DaOGH zDj@}!`ZY_M=J*jvu}gs#i(`qewt*bQylzRgq{C{TB;A-w&0>WZH@v^6Sy7eZn^SAe zFpIDa7pP9}(2kU7f?S#mH;z}XfRc3ywcym7?}6G(_bnr`Pi&mx!K>|nf2p;_%l@Ca z{C|9&EEmO~!6+Bm{3M&d>{l3QE=fMu*;o%z_5AyI$%6!}VKvQ-O*TOF+Iw04jRxCR z-{&J$QaA#zd>g7g{TCgX))`;2i$i&Vb5D7#e8~~ zt0y*j#lDvL%5gOAfXnPeY`g}AqK#kd_~ZFb9cC1DnT}D}ro(DDEVXGd{i7zh|qcKcEMTb(I8Tw3eQPoXzsPIF@L++Dio2w<@?$ zQjh(3B?A}id&M+k|d`LgJWPF;#RnH-Ot6?8u0iyTw3|pN(~C?Nr&-1Oicy&DcOhL#_PD& zjDh071X?1{ zy`s!%12-KAA)(D_R73WprJSUhh-#SeV9ywalTiWs^u3Gebj|zBFrU%<*S>@$+*-6? z$*8Y=ui*W3m@g+`&ykr+U3YS`j%Z|l4g!)kVB@X}(JP-!{skMGp*>4Pj>Ko?$yu z$lTcL+YyCZ_Ll~@LA$%I7e{x)Gq@aTKqEO-(JR+5H(v8N6NMF&YPJM1D`hb;npb(c|m=yIjl?gYc0ACEq%&J5M z?kQ)7){pIG_=Pq0^2|OR^!*;0>oD7IK3n_W@*0q5&dBu$DHZd+q*#9+A1#b{$8n(? zIHP-<7e^{`+6uh@n3mJ|s!| zWm*qHpyT&Y`)t;h(fLeLFhvmwpLEG|Vvc~lU=g_$L6yF6)q(1#So&b?P?^^W#a~H! zl6ir9g2A;7%SLHh0@*kwa7*Xcz)?i%RVA<`4W@W}*e}Mennqp!*^Z{##EAk&Cat|4p)xiQLKUl#9frJ&8F$Z~Y>z!% zEzCaRVRD&GR^sfe96NCDBP1fzs~{-4sl+(ac4ki)acX>d&+C6u&Gk(WcaHu0oBjug zXh+pHlgtzgxdK!ShPP9VSE+Oit-+ADg%dEOIJ0k=e`b+l;`eDH5}HX6?_Pt4@xnuo zJY2#iMG^4GaE8}h82ef`C>(swm$r_&rQrHkIrG2NY&m!fZ;x-V{qP_7=tT!Yx^uIGlL| zg#tr((dh+)DWR9r{B{cL=-1V!BOH|_Y={~zUity`(AK~?tw*r7Yx-E1ZzK@kfg%cY zH(m2LW@Tlo8r_XzPf$U>;sx#!G*U{IB7e0Qs$@7YL6Ijv2T7E`7O;20?DqLz{ynfi zRhzh@pM#r9=nC7ew%%K-UGyAk47F@i|Sj66{>_rzfm^-q844;DvKfVm0kW zNlW|W4}`XI?>e>XL*QaF$YlE#Y^Zu)Lq&`|Rb<18MxD1|1C2F~S?2RciFM39Yt^J` z=3>Chb0+otEyZp~>$^cZ1V{)y)(! zPuL%**U0rRvqg@%l^OB-Wl7b6l~%@MLOFo~nk&JcwkH8wzH`s1UOM<{X%Ki_-0gpM z*^LGxUT=|*Aim+Mz{{J1+Jk2+e4FNklrw|(YoiV5rA!vTW!On<@VP0vu>qyM_DIfu zr4$|I9z`0{G;d)=z&k^{V@kxa8pR7)!Y*$W%zU(5aq0!~HF0(42d9A4(hXpSIL0uk z)rS0vd1#w)dFOOgd=bor7c7`EdZusS)q#aqL=?S_cu4ZuyQEw6OsN>AANukvl!Iw6 zcZ*=B+%VuoPD+3H>5(51vDj26_DuFPU4iIQJ`JW=u1cEL1{;A+|G$<1HxR{V9Zl54U7}sF^$_J)n zj6MUxMwnjxyy7ph>1@y zx#GM%G;;hsa3ztiF*UArKcZ{=l76J0YfgdF-!0cQHxjB#?goJ|{`gSfSB`rSDT6Q+ zdNf{U$tKl0q8TwiLC)JdmJ}QAay7PjT-YL|@3G%ah1*iW!7d#9PJQ=XyAX_xruyT~;pR>cTG#>cJ)~tGb5R8Q zIh^dMlQeA1%$+i1c_)YO@A#)BJZJ@+J4mj1`adb`U%DR8+U@z6o5H5;F$nwBq&c1? zm!ybZ-5!<0wQ{`LSNAfRI;CF4x;-^U=Fa87>$M?p+(5uRG4*UiWW7o`rr#H>h9Jn7 z>KwxLH;+^$wHF`k1+@gMra+zgBEnI28&$e`IHHvJvK_d{dRWBv3_(wAoLnWKCAo1& zAex+*HWRFHTh4ACl?NW&b8!Q&)9p2mpmEFD=@v;o`z_Gn>ZAh&pnD|ixpo~N}9GaalBZw(VU)?4r*z~%Yo zu5*#IYR@Bw>asH&bJyuf53(v~%dn)&0$=yXyOwzRdt*G7Oy%2Gbc#>+j!d6&OC`>< zX!RG1ob_8iF(^@HM}aE$<56GiSDd@-xm>`jmP=w1^I496>sB*vjM6bG>86c_1gjyv z00q-~9ZC_P2uE&gKYi3CGc5p=6dP#o$FZ4`Y%#I#tL_;bKWX|qzJ~|HBs0?j9 zShH;ZTnmOtu`UNTt-bEs7VPuGVW)tc_Nl>$$ZGujKV1Q%+~Z38KUw2>0sw7p$$llx z34fLi2@_gN2+&v0QJ%cYyYW|l9q??4sv1rec9G5&9cjk`(Wi=20O4Q|f_OLcDsAi# z>bqLvZ(-NsW*AVXs-=R(Emcbg;QR9E+w?{3cpfb1^zuuSZzLA@3#y8nrq${AHM+xH zD*cB^v$A&F==%gABElB-wC3TJ+A`CPr)){t^soyxXqx-O=WzT#`}PeN2g`;?2wHA% z2q7l^T$fDEHghw1ZG5ob)0eRtBx9QcX=q@yk$vjoe5HupYr7@_S+jRT=a?t$85WEr zj$O`lvzqQ0xiD!-J77c=2c9UtqDt}03HWsmR4Y!3nljC*e+rFCNFNtIKk}4!yGhR^ zbmO!wTN66gtjzNaJZ^#lKV=Bda(NAF^?J zl=%<_QYw6;s%WgAVD0y= zdMr2Oy_Ga!_=9yFH^VBJ48hrrsm0(IDg3uDYT5@SBq z>GvH$@VUK$m8`uZL7%+#`3@jMFUf^GzW!CD_JI3{+ITs#uac2uQmAx}%EBObF(i&f zoywn6bWRtJ@5tzwMO2wO8qJ+m{$203WI z@>_m;4?0^ zx%ij9+$ZR(0N@JMhnQ1ei#*dUUDvKPlfX8MC0K6Z2$*>&{F)W{L0ORf@5e9qOqd|5 zx+0fG8H77xGrnDRq!0daUrrWN8mdx`A7%1>;oL>{k54DPoClb4$LeD3ys!dm2&ip1 z;Kiji_b?PG!J>NE}|KbXk)20xEX9NOCo z0z&VB-+rL&rG}=4d>P4=6>b4qH=nU^eZk*gMJqvah(7@!HX2)dPg7aeBzl4S5SbN2 z`Z>6x!!#h3quq7-isC-ECkA5DHW(r7zNKW353NAiN{<6WD0fG|hayHcIbCkiat&J_ zZ}7QZx2kY4#y%Tn=T1rm7}@c0B{7LYK@PN)G**QI$)nKYn6C21X;~Xh`fyFO{FhI= zDe92yVKLntpvmi7eW`bZc+7UD3A(8elVY7^XQK@=4K`!G7p(g<@jIlU%&T3l4s1EW z+75wC!6vQOG^oLN&5=-|?Y!hkUvQB%tWIGlTz!zT7g?IZJ9W4iUrTei3tk~+Ld^74 zY7ya<6BU`vRq!#J)f|JN=l9ZkK2^K2pEn@#ZKT=PI3tX<-8T7(ii#PvtYIjCxuw@o z-ZyNqg72C3)W}3`P10qEVu5W%?5GKtApYC_3eTu#j}Io@+Rq;PDqbseT!oSFGKmJz z@RmIwb$eB^*^jfr{tx%pCDB0a-EO-o>V@X(Y=%Sj!~9iDlDJ%>&ZTrq79SOz;%@?Y z!fNT}Z$HIa)VDwve|z&+IyskPa+dga7??ge{GD7%ex|ZJ;#%`M_>M~$G)bpx;t2kM zbRi@D1=B_$mc|d~^U4?bQ))s3$np|8y`$*>QFTJhFn(T3F-QpBO!rtmI2mGeNaKwSniQ|Fy z1{JFyiT@h+shsiv3v0k$FQ6Cj|MYUWi;}$qwo^nmoL$UdQvK5TI@yz);H*z}AgwR} z)*38>8Qmi`4YlbrI-LnV8V0_Nd{hmvr>z$Fzgetv6!`;5H=XkChG~xL{xTT0mnopt ztH`w8M)>}LC~~oYFyCuUMmNDD+YWqe6uCE9RA^=RE?M`2_re^uz26o!^9d;b?ei!O$PFJi`_ZJSr5PktK{~};QZ-cAR z1-8=VXkASd(Nf3?qk%39{~nDwo;~ut{_%{~a&B1G^-_S9R>{-82vCIhY6&;11!~wm z22CS#UVg+Lm*h6~`R+#?qYvPQmfZG7blnRFrP}aV5tuA6-EaJ3iPz1IcqMQ2T^v!8 z@XLKyloLV2HCB%$5C($}Vz?8y6L2zZv&J}abF#R1Aaqw2YNU6Dy!KLl2&}$lFT#v!Ys#{W*Efd6)*?VamNo#)+Mmbxqo;{FlC@eED6W1uJ6>1w6b7*LHG_RuE7}MtX*oegkyW zvRtIYk=_bY3&~yHGULkO&4_`vC4`5SzXX4pYU}8bn={M*yEfjV+j{RO0L$GCxQ}M_ ze(UePf@f*+&q zUC`6=!PgyK;QGcRi2+yo6P>-wx$FgX*M}XFV{>6-vPjLbtgo)#&A3#(RQntw6Mi*p ztg5EEM;{m)03Xj!^t)XjR*k1y%2Go=R!mP-uwa2I(+ML3cwUx)$5PFv>CcFf3Uq3n ztk>pW^P*ewr%`d%*~te4(rD*V!pyFBJ`HpjA{;A!Xs=ay-*8cYi2Y&@XIs~~tn;4I z0}ys!n8b`H3RJNt7|{gf!su)fo@iQ_Se)F`os7v3y~sInv{ z`&j3siO!=%?unJlth=a~V`MH!2FAh!-=@gu>VJ37Sz4<7${O)|B~+6bI|t2-7t%8A zULIMYB&O|cCsdQj{EM8xQ)4Nxv-y*E3~BgLjIUtS0=&smSsb((=}IU4PI8MF<_iudV3Mf^YZHeic@-0H#8GJbd^*P|iAc z1dr;+w6-0493X#Y@gn~xHghd2Sh&hm1FFG?iY3X-;l1~t%AY)FH~8v7!|QR6zt;G-?}!G_gothPhS`1b+O6E9bb;1* zJNLG~=JfNOIq}zXBJm!@OCjTp@nt*3$5FYtlmu>di-=Us_w*j?b8T!5al97OW3h_@ zPutCfh5VLBFKDF{FY6!_!eEZvkNKN+zuRXM+5&aOjENfzBVdlHD9u2pR3P)We{Asj z_EYBH;9og)6~Zs`uTe_4ySB-x85u9>89bKDVdP`N?U<$_WlyD(J;i}>F7PWfO{mJF zq(cGJXE}jZg`%LR&3oZ7Xu-?MA%zPjx2*k`O24E1CS!gA=0s-Yxyc$MF(y2k_Pr0r z^+lSz?$>lyF-?K^um!5w9Wk-USjLVf!~WQv!J%>`?By0oWVZN>deDdg>$#Fy#9=yK z4!+{snQN0Ycj-SQ_X<#c4KoeKzkCmwExHF~^xWW}Lo`3~m2oCHpwOo?K;>~cl$0dZzLG@Z6?*A19Uk-63e3%FTtw`2dG02%yGm6EG}NT&~hd zD$JbhD8XV;mIgoGyJ>yi&T`7$a$fU6mvBlAT>Au%s(Jv(k649MYL#lP{!@I7*D)sK z2NJGxy!770JHIeViFR7a$fEtMeKpbT*5J?pWZe{Wb823ts@puIeK=yaR+Xbvaevqp zqq&KGrf(7JYQqD0;RnOO_PCe^K+D5<2liF^4F92k{012l=n48t2xw?MBP0U~hx;g3 zW=>Zs-I3dx`-}z*l**bV9>zoQ&$A;Z%z~6W{m2xGr0vA$O9EPwL+>-Xh;!s?bS6!T zyqOs%pgG-Z>H^(@O{1DzI{!zLC)Sw&RkoBrI%M*3QGZ-Mp+66A2k%*I^CuS;W^%As z_rUsiN=3AiU6(Efg-ur+v6j4zfnp?%@9#w^e>(8NjfqLR%7uSn?EXXhM`5%*I|Tv# zc*}3g#mDM*?x#F60180fxb0Sf@)5_QVra+deSA-^5fJf%C~YV1AlY2J^&rm?k?{*4 z3#hf46*rTlhe57c^^0wHo?OV;dAIFY(!Jp|?w<~E$JJlA{+;b>ob;T2<1p-(WY~$_ znst5kZRw1AC2!Kvt&_==(UXbIr2fONX!P1-ex(;n7bEG)`dH|7bR!8B-2{+PVV|G! z)-UGI2j{h(;gcm25=Cwq zGJHhgVpggUv-ENxyJyAU%(?D|_104*kv+N*K~>YkgYj}aW)*6x!nggrU&aZ=+zB>p zqb&UQHXOhfC*{w7ZBXdqDO(&|(0T{HU+6lyO}%gDZWA9%E{O4HhuwrBE$NQqhuGl~ z>`Ym~VfQSD`k4T+d95D4hIN=6utPCVdxbWF$SCFfPJL*tXOpiSHnF&^C?M<)h1WoX zaQ3}%u}(Ny_`LVjS$eLu*jPm$^0DCU@S&RdUbA`V(Foh&!8b-7wU%SX;e6^C6e3(~ z+YfW-5dT9=rECN2QTyz*`_~ZC0{~aB6%0*G(o{R}Tljx66~iZFoYa{#gv}*UU;ve9 znQ|urq5?r&88-z}fK#I8Tk%NaTh>O$fzimvhn2}EH5GQn(z+owU?6> zAqOG9=h-?*fKv9__l8!)4En)DdCeLhM%k4ymRkPg1**GT5E*Q1a0uy&DqB?RZ4V~1 zFNWJ)0}BD0S_bH!l*w5uF)ODo{bnchIgaqaQ-jrLD9vAVL^A=gVn>&61l310|5{1 z2r2LIrD2+@j=&)AClMd}mchxHj4|m*WoO32oO9*ew91PP_`@Qt1iS067xnn6{NAEf z7ZwyPSGDRiCh+X`wMc?1$HXq`g9OxqgYauzt*zEfRODRM7RUcozgnPkPTgiU^K~(y zXKFAk?b+*?LqNWzk{#V`ES9C=6PL5YTqy9QnIuJjFf;GS6QK@`a6fGMjVaBPsHVac z)9oqFr)qIPF&E3DKJYLP-oSO9N*f*j=CoGOSB4FIIm=Fg@cJEq zBZ)bLZ`*o+rP~ePwo6`T@qo+kg`6$4-am_;Ax7g$Nx+so`1 zU+9xDI-%;|-xr2UI_l@7b@$Qxh&iz-JrSw66NP{LibeI;nQ0Tkj&m~dIQ`)`ELk|e zls{9m{K5u77=K{m$y zmH^9pRRjHW^#AYV%a@PdSr#@wiYsO1qxAJ(-}dbMIUHvI0a9;S{&hFYqXpWfU|WJR zpb)XHBSO7tJ2I$~ZCi9GzoUD#A}vAjQ`sl&!}J-iiyIGI@$G#R;|RzTQ*Uar9Q+3% zK=#e+sB`@>elB?>EGzZJy)(s)RuHw~qhs=Z{dq6-8AChXqOMV&?$`g&Jm@>%v*a*F z&9fU{v%3ys;znNTFdXL0{-cmq-RSS^UaNmCdtH@7fBS5wl8R4gXwP0`+RYuDBLD9} z0u+T(sC~B!08e*vN{dbK?<98sX=JpQfMtC}T3TWP0mO>deZ~mwFtJm%bv+agY^91V zRG4tQg4C?+^#X|Jm#hBzG%+NR24FE{@1R+zZC`LXb*6l*zH0Q10+pMp)@^(%bSUB) zb+&IFdSAFnGjiif6(QWCerIuI-!I}ajcs-U}x ztK*T%hXT!T_TtCJ-OU6eNwXn;OHZl?nJbM<#P@WMm4@&q4a2XcW>hF?jEo_HzBRvF z&9od*!FRthxdb1{|7LZzx1)X{gFwpVw$C2)*+3$mMjH|T!@VZ8uSC4lC%IHmy?dP_ z`CG{r9YfDFNf+yuNb^lVV~43whlyK`X;Mc-M_=MF>r9z>AzJ{1@xqs&C0VW?F0>|b z+93F>xtz>FU*^^BQyQAD!x>m4KzoRGP8ZzPO9$Iq4NOotOZE&C#VhxR5%(?=<@_?T zTY1$Yw)Ql`qwjigd?lN%ryv?@5-sPzM(Q(egr0f?~PV?Zcesov7%iz5>1^oXy0jJ{&X z@EkOSxw8!GGXKvFOb|uZr45m-cf0a$Ms1!oxoV6p)xX0k7ED|CSC#AT}VCr?-6( zCT@~bPPwX-8!m>07?rr6mtwu+547MBt~E7+jk~NSjCyV62h=6T?nZh+TtBp(p(B^j z1L>SNQOq%>9l8BU(p50U&wR%t$NqP#l@VP#E`VihdrN)wKp_)L^#lDyxrR(U8d)Bm*0nkKz zE<*9z=m!|biodt3#au@bRSS0f63%MR*Bt=_|2=q*IrBJDir)|Y$xz=#=^Z~K$&L%A zGPY0J@P;7~D-FvE&q*=vg-LE#poG52rIdlN!u%1O99QctUUbNyu6Zo?!42O`Th;5$nT8$bDRqfpE*<(x zoRG@N>Q~v=^L(M9$QcsIX+O4B!n3x4!oqy<1?I$qZhITeTr@w~DIOrGR5dLlzGkPR zat26D-C53SK;NVyBXGaQ5m0gZTccuUa4f9}#5)UhrnL<9RauQl0tSSh-n~0C zzr_{Wm!3RXZ`FUcD3b8JV=ZukjI2XjPZ~WxhvrUXDAf|5%r#}S{>?ia%2z3PIuk)o znZ)@b8wM|Tt%2KbvA?NoomkA*;Mx2>CucNn62YCU?x5`LL(HD6Zflt95M5&FXu>YB zM_liEV1qY;#`DGwAfx@Yr*-bjP|`(RE-M(uv85ThKWvFEyGwIyn&A8mRDQz|>poLO z%o?2qtOpzAY2OHwCC?Vv*XAex zOxDIBs{!IUYD%r*Vfuv+et*FsvfTnmM1ei|lEhS7bhaa`nNK(SW>vRk*~3Wyz4i!{OK<9R7eJ!Rc<6?$F%dZclVS!5O6uI78P?`R>bbrU=l8=1v3` z1Q-eJy^dYA0+WHVf8;>wt*u(rt;HM?Z7qH{3t(}ep55NW;J+PJ-^u-mD7+YR48@f= z^Q1S)T)*O{1Z*mi0*7C4i9@?%&_s*amHUvhyPiIhKvl|{wz?u~kN0Z8?EB=@^%nE1 zo>ElM7^Ug#Ei|t#NWf^>5b05Z@OlyWVGKzlneIQ=c3-a9@Vg zy;+J^TT(T3N`HMK60T6)7PYq~d9=Gf?+s5Bk;&v}&2j^$EAn3ZW+v7}Lh<1bGjgr_ zM3JFGC(uzCgch*CujCROKlL{SC97kB#tynaM(!iOIU~)DH7MIh2r){%I`S4{#aP{cX zqb&rSe%c+D%^4Ngtkx+P2ByJc;_6H7l)bLA@%^qm2*{+8sCTGYOO-xSq)5Sy-m&qP zE?Up@k@3XG5(&8OvK~d3C7=QtMq$q(C$Uds{TUBeB&7pshn zbOGz=!ur3aqu&gE27j_k9&5~D&YcNQjQz=8l{h=EdZIdj->e4AnCghc1BIQ}h6i<` zo*)!^`^gAG*tHmmzI&MA_!UR)9P1C$n}f#sg1yLXEw#=wWy)7fnTRA+LkSF0)klg! zj<*baq;B^wubFe7Rq_G`wnwHRpt4FggNED4Y0Ra0zEaCjM2Y3dy{R?p$E~@4xcxjn zdVa2aPb{8v_#FQ7U zH86~9+8#7a2i*;dzkayo?vccvm8a1c7xkZ+RKQ6#ZFiF#ds6*N?`N42D8NP)t~Q+E zv}6VnIN=6>8?7ab7DV#bqMnXawGSTT72V(Xw8HDh33MMHpC22#Ra8Gm(l>yuM#>1S zieHW-CBo4gur;g8#G|C`+2>YeFIN~M(su1G;Q498dSO)5m+WVmGqwL;Y> z>Kn;HKm|E;on2R0XcNxuXsb6UO?3^f4v=rc9v4MoU?;QZ+(sj{{9CqiLQC|_7$^Gf zhXnx&pia3&tOHPz${P<*b_3};DhycJF)aOQT;Yosli3olnBVdadHp2#gq+Ro|7Zd9 zNh)5GOVzrXftG}7%&Y4XXkYL?#KlT!pyO3$tq&bm`kvyShf)y1oSDx^iNN@wu%?Yk zI&HkKg>t!vCD6Sx+vx@mf!vTGc{|U z5H>ai*6%28))$jdXBKWhTciy}EUvJF3LX~VNx-y1R*Amy`0}of4TC-K+gQ2l4^3bE z3Jl8Gl7CQBQ-G2lg(_7k`(o5egDFtL^9G2Ro*KN5!PqBu;w`(M zeaSjNcM#~b)Z|>QKcupOf(RQkY}TN<)<4p~e-YR{z-xJS+7GTCNyA&p&E;e%Bhqxc zckDuW$4#5~S=iJi;tv+HBKtiK7vM)zk+XtHiPj;yLkX}Wr<6vG7#1I@4*_DN-WP8~ z{5x>KO(eo$`>lw>v2D@y7KlIJG=xjrfYrNj_4)mTn5Ldnj@owO)$lpD_4i@&{^3ja z4UDGs@H>h!e_Sz#4AGY@25grb{%Fp{i^l-R6E_-C_G~!b3Sc6`E4t(IUvWGIMLVyF z8>KH?F2Anvc?eY|!u`Gv(7fw@-iEwW`XPG>SeeXui*4i^c&#;a5&kh|gO6z=Sy-Jy z6?zj0_;^Y`B+*NcXSK#&sg$c6q%hae&o^ZC%T>d_@|!tG%umx{+?AT=RCU)2!!3ua zp-qFTD6l(ucdG;++L-kY*sqM>)8qLkC97RtddgCN35Vi}~l{l*4 z@)OG%EJ0;sz8vM{P_$0Yxb2`BJQ)phHj62^FzVRDm}Y7d7Nv=(HTU*Ew)&XCia*pI zsfmAnSOQ~(>i)K0|NZnT3D73dk(?0fLn<_VF$X%W56Rl{4MJ6%Qp_%*s>h-= z#+LySz$$!yTg6=T^}{AbgQiqqdCab8t!2wEhx=fNP|WMeFy_jFbU3)ApWGh?K%XC{ zFQ9vp0R2`f6PkN!z!A2hLSG^Ot!KyG2oFznpddZ{23_|rLGoYOWVT$T+R)>KJuLuN zy1Zl$6j}*ZKQ(~OiMYR19Gsaygyn*mCsyxNYSf00JkP4@@dyV^>%7ndX?l(yatLi;`Z@jm^7V8XbXseG~Ww-I0x zatPBN6Ti3q8{sA;GejuASQ@uDzaJR=d5)WPm@`bw^6bSDK@C#;+fbtZ2R-BiBm8q({|jvY@0r>Ib&b24Irrt*0iZZr54v1wS9g9T z3M`~`MjFKA+NPAhVhUlPqroqMVsrMamV6#RZlQxvD=BeZEfYB+h3HpKdM{xWvxeW$ zdtRSR9EmxgBCkbtom7l(D^yAk2fpQXwzPe4op*Wq!*bg-&Qr^_qTZg^TOwP%J_O<; z&SLcvAWj(PanZ&3k(HUDrJ{EBh~H2F+`8as)pzdi6gEWhK$1=ek~C;YBljywS^~KP zdy>qw)8)k*=!12!L-rAfB%0=Sr826ZS=iKxV30F!^3h!;hQ^nn7tqIzNMqyJ_ncHa1*aPK(;q>1o3ao-sEM4BMsv#%y{3W?| zbR*n7;cpzJkco0DK#f;MqjR`ddyH=zkGZcj^|;vkk%Dt`ZSYAeHR!vmQ}nj0NPXY6 z1zJJA$Gwk(l;(|e3|6$;TuG)p2_Lc=wx~VQnP=8HS9Re-)`ggi*?e7kyZgF(cM}ZO zVMS>e*3(YiNL?@)&IweDF42*<>d_9`5G)6hr=Bgg z@Fm^wZ+0(spUR5#8CF&qh_!UjtZuaSV>y?cL35c#mY@`~2 zchh0ymF9uQ01_|5-=JY8rqW-0p}Lfcp2}Efn(MFS1npKdtdOMz=Q>(uWMm+#j#l0B zSXcP*bMVA(1kb?s+noN|N zQs5QN%;(b7v(TvVzBp;)gm0m(_gtGDRJR&bLTx9L$%RH2U;Af<6QM;o6MDDFTKKnu zKp4f(g%Dnp*RQ$=o}820zH$lJI041^Ej=WScJHtP7P)8&upsiR zLzgqrMS_JazN=)p#s!&?*{RS-wUDD9_ow*WX3F_!7GwPim#fS_@}oy_CyIgjVTIs* zR*oq3u67n9lj;trHBsPK*i2*Rmw9ArsvEI-=cy&(>3gSNP*VWDN9{7@9AX2xDQ!p| zZk#>X^5v}wPln>Nrlf!PQ}Hl3EZ6;*rDb#Mmi`M+A@F$ygfRbVH=Zo*Dhi-j@O-uU zu)Rs2s*rRnUnsag5eykfb_Uzr*mZB2!e|;i)$(w0mAwT2^WrOacC_L5+7Tb>cS1km z>b5Jhm-Xz6s(jrocsZg~-qPb5ZwLP$U47B~TI>U9h00>VI>KRPkdP2Ns2OsF`HZ*M zd9}FZEr555hpx14a6S)6fDo&S9r{b{m-Ad%7Jzr==}fqFLhMf}Dqj16NG_8m)%hKR zA^d4iy1!xn1q}>bC7nHx9kG*R6elUE!tb?(5DKnCCZ`q|y#dRXN%3r7Et#i;os(wI zJL-hkoj9AtlamHuy($DS*lEvNDAg$t`ur7hnwtpagk?$d-)mnQo|8V*br^5taD94{ zBA02Iu#IiTHJWNnFDu+FD3k)9zcK>vX{R=m;{y6U$mmKy z-EGlV0bLf4aCGAbHAAO)zWBao)9jt_$fe=?rE50~lYaH_^}VnoKle&RkoG5_BQ)5+ zJ%Lt~AHR%y2wZ?_98O%J$l#RO97XiZ9YmUHCtM|OQ?U+2Fh5+l`L1g95uy??97Obg zZAmZCDQ_+b9P1re6`OUwWe@YYE}-cP1n{{0(ZT&St@q7~?S?r(ce{*H1JkT$rxz194zODWKKM0i}~dsq8d zH^`~f?QpSfg!weZzPzBSH2#eG0NtckUyUPI^UBhbjyDB2_bk_&bMskqW zn04}K1$A=Xlj})gxxwwjAFx7^ulweMN)A<145S@WI?4C?oc3Zi=s*ZF z493y;clC>B58q(eh~|Xx-ko>D>G#+<)tuP~Z-DzlJ&Zj_Yb(m)}Tk|Km5n1Sa+^(99 zv9*MK-&e3uu+;%>SDZeiBb3|788dRjdv#vrzjqysuu?nMG#uGnWwP;whp+C2KxN_9 z^p8I`ZkM;PhZ!7W+IW%z4#(tjggMuMkBR_z^+}Da3zSgM_3A00bCY(d!7|8+`vjL4 zuWJ(F(vYFhfxr>hp;dC9h)!4;o4s6mDdfrE$jTFotVVzhnexy9Wrm|tUsx4=N4DUCbSBYU+j=KTRd5h^0n`)a-VZN z`@=xZSv_Zu$dKAO2}1kh$X1j0J$`$=S5kjvd{@izm6@55BbQ&loojuv*L~*F z*udtp9>IJW3_GZ=SX?|Z-C@T$vi?d2oji)vBbmnvn9{R@ow2Sa-v z+T!Acm!L5t_9BpFb%yW|NKJTMQSeqrf4U-B2(=ihgJkYb+UxuzQSRV??XX@wRcw|G zx)ZBeEM+j8$$7iS?+=gdd;ho7$hPGA8nv?JX^Z-IkN32US$tfX6X-=U{>~p z;lF=zz6sZ^r@i<7ne$i92CQ3unAY4A7UNwne|dO*NaxDx*cB)9cTpF(vzO*qe+@DH zOP;7h43juW#M+rBFVBo!J~LKd_rppQu!RjsFJu9C8rEt#JA8jh&S<1$?6f((ARu$; zy-nLmyB`jQCqMrn60y^3TWXpZPOAszB}!C41X^|YoZp+b9lJLo2&Jj<@g*!7yIPNc zyw&R*$;(YR$a#qVth)O5M>}{#cJ;eU<8(lKF>T%A)jcvjk!DFt!2XY?kbY zRcnR9BgH_Q{yZW=SA?T4dP6Aqc6W9OEweM9$l+(1hm>^is;5!ycc~z7+CiyS~$xkv&&W#^xuc(#404K)%3! zgGb>)>i4Qq5*Id_drh(BjwEn76hi10{9y5plP3AN zc5RH?ezt-3`;RLG_xb+0oBHiK{=FqHjkw9ibnPy}&LZu|$>JJrC()w*{-oP%>hhxL zI(v+hO}ye?`#NJ8&OOE60_;L{PfDAz)VCovjn{Vsd1rF!Xo5>VYi!>?(oU_XFY%EG zl+TW|srf0*lbBi12ntMy*_~=5Y2iD-aP*e9K=0h&={UOXl>=xaJo^;OV)BZ`rMb({ zhg~9*2I~zs`CqN(=Xt}YzebH<0m9wJlu^7N@jYE2qXL2a&7Ne@FUFsN0=n58&k-Jr&oI@b~=6?6sGyf|AuIrJU@pH0yH;Qpg&{`J<%!~dXWA7Q(WY&d^&Nw=XIw~M47EmA{9YK1tP!l9WFH)6=bOZq*6bmXvKtc*N zQ6Tgpp-GJ*N+;4gs1QSs1dt6S|JqB-L07+=r zUrtdBCEg6r*j@%8u01l+pPVjq3j0n$P-M$DJF$x`Us5vaPnTG)rhIz>gNG)Bn_Y;7AT%j0B|c zLH4!h5)bG9OfWh&rVQ{`u4bD%+dIj9P~wARYkS9DaOQ4}bX_2F3Ie)9{6thsUkcUhax&EH~NwytHN1q-4WU9QypqR)?>LFO-5Eh z5)1D38YF%JAKfOMeA$y1cVhQ~2$od!&lJnyA#f)Fehxop_dVIgmMD?Ep6_nOiTK>? zC0+~376zuWbQ3m!@=wNnkJI8HSZ?IGLUkroHt?tV;s4`f|DQj)*d2Ql72}E0=n$8I z^=BV9G63ZMQ$=`NAh!wMu8S2826h`A3xHaWztPfm1>JNqr7XpQl4XYYuKT|I`Yhnb zhp48>jVx_Z$U;4q{mlj$(nzM0kA%Fvz8>0%ZAgk(zMaxq^s&(tPW}||E#5ud_;#Oq z3A1EG2WThRSE@D(yqR-?q6s?>A`-t|`*CST(5n?9{sDzI%@m0C6kaPTI zo%jIkgQ8c9pJzT>b%Y<4;l z`lTtPP68}f#OzMND*_HNx{31cpltw70x}l#!0R;c8vwt>Vf;~{Ky(Ph%hLsinR;TU zOxLjm%&HT4i`3^w1pfo5i|k1Yh|xDk^*8i~^vR zcX|`yEVP=k!PYYV(wJVFk$J_^8UnZINa!~ZQ-hfY6Zl^Ce&C0cysDmYYQUAFPtgW& zoT{_UUn|JMW!sxS9yLzLuh>L#iA}|mbk{bNyVg7daXT0*mFqpz+>{yG8%j=^*%&9x zKCh!aD2Goib$U6ir)TW|(UjgQy!H7x+N-=tr3tIryp{l)xi0K1#PH%apWT3PeKG8^ zy6@?<;o9cvM!?w^)0s83O{JN^;~qDTlpa~sUdp=J5F$z$f*&c7!NZ4c&H@tZ%fxjDEK0JiajUfJI?jtii6g2)^*rI16zGoul6dE^|QequT62Dug@hD z=Qt%*$;M-`t%+gH0MsJUzrolHiDc5r#3sf}Ywq$IeAM+hh8g)Kn^q@0?Ni%-w>lNS z6uvxFi>f5)z(vl(%79#?iL4s6t}1Z@1sAIcCrdj^9u52aL~#mp8@Ihd>kXW-t(jWG zNUSv72Q4sDXN8=bIo0L$5yBB0ePmH#R(tLZ@w)a^yYeROby{`1|FlEU(wrU@S2i`0(3i)+4ry7m3( z_a2Eqx)=oJp0VLHIm}5$OCHhubz*xAVBlM)B6KIagnv}VDhvLumyhE;d*7k_Fy`Y| z9s`vgph8b5G5nrvmr^qjEdiC|t++kqq|WoTP+FQF$(j{uGr;$t&;6qk6DB;Q3&x9bjT*PueL|(Q{2<|<~ zyp7#12^iZKWzAWG*Gho(_aGD+=OHulQQM0aofI9$g<5WgUvpi!7sHpIw?lgDF0&-$ zgcq-Zl8r~slbr;So?GjpRZ$xWNRJVNnU z|K7uI41OvpP+)2Q*HM|je!8qz##g5EX^E()FiazyjKj-7FPPkW_b=RxcYFA4E5FZx z^9vR5EeO?(7>DDrhZIf4A9eJ$Ee$Y~)4fxSU-Yr!s`&9PW5_HcG$K-o?-@Ypidal_ zlB^7y#%+%?J7=S2U+u=MQ#$ufIz4b5<9I8g!1T6n>XVE#s^F<>TcN(^=9gK zsCK=aFV$pT=9C07-R;7NV~AsD`*opjWeX>%&CV=_YMJ~9cmAZgv0KgI*+(J3L)@O< zk13C=#$Bd|2y@+798S`Gm}G;yY@x}?my%1i^21Be5fRD-X-Mn_$Oalspgt>!MrJ)P z1Mwa5i7KVMij0D#khRdVtF+4D&0UB_cl%hte{}zr4bF=~J8p;+cUB|!N&$9u`Tvid z0poNx5>%xAdH||$yCN;n!8EIf8lcm~$38*06)uWs^9t`IvdUg@fA6A1-hB#81XSK;O655(F`uSOd9k;yF?jQai z7a)oz5oI39u8nHY{M0qW2_cU`b;{snz>Kjm%zAHkUS8<8&qh4mVNTOdi<1OC#KQqj zHZ64QxC(TzD5XN={okr8?8yoD8GZ%&b|?I#tJKco>kGRNv+#f$S@H%Bag~vK<*9=5 zkV+u`zg$^6?`Pe9uca3XWS(>M0>Ss^!H^>M#@=JnI&LXG26OUh5S6c=7!#0h6~_8z z=BO8?DwD})>ZA9kxfLw5^Ge$h6q8*~@TYy`4h9H%8l&lb^(7t%5xthM_C@RlUIcIh!rrwA613Du$GF1}=w)H}} zhLdI-?XZC@ISo6+HOG@k%+9byI&K91J=SRas*3WBaX9I{5s%3P^p1S(>f4B#N({u?v~mAjcOPgm&wkBU^D$axylAzelwD0?MQ z?OTkoB=O^0Kw9I8;_ZLIG(CAtL;*b0$Z0_akjv%|blbMZS!;EOPw2rK7M}ps1EE8k zaH38Tpt%iHGub^zYK-TJ@w>xQ=BFCwuO;U7f$5wwQ2D_xy{SdZ*e;$uw zjXmeK8Fm)qSh)K2rj33`*QK_zl@pYp{$vS}jvmoMkN#=+5d*Nxcw`u*9m@hJLTn4Q zCR%7f4LQzy)HCly7zEwf7^3(sKT%1<1t>W=zxna!Td&B2PV5R3xg_(nM;&s$_fa4v zSC=Ib7<;Aq^8!$HrDeVP>t75j74dLen6VYr(tItloC)5uG4b=nkdJl4uGX-HD+VAV6=XrbJLt8i}hT^*)futdwq&uj4&vnvldGr z{eCnPP`Y{0F@(6`$w_t@NcqU2$#V%>OLn9H3Z26w5;O1j8#h#Y#NMD}2AYNr6+2bm zoKTwnkx0v=>`)YV3Q_*K4&@KpsJj8SL30EewKYKIO?uEO2N>7zZ0_wCqugbs!W+|p zFO!k?34o4Kh@8E@;`%9L*L64i398A#!FKqq+|q9P+-+5)ymNIZZCXDyorXD83*eK* z&#pl4@QoEV+xw8UGm-R^$zJQ zG9FZ|Vz@8I-a1`nOk6su!=a>#7c}XHfCKas*bM)h#=&x@Z;71k#*yjb%;nNM!ykS3jTas8b8K!V&!< z*E7i4!dtei-oIupGI#I|$c!kC5Lpr$;i$1mug?~nd zA5VlE*Vh4~QcudXNoqhLv+sl0-R=wmaAz)6#2tobd`K8nUsrSJUL}Bh`8#ys#TlK3kvj|i(rVb-jq41VJ2o6!4wi)S=hKZ?3_33~vSr{&YF*f$; zps%)R@L>KrW@c?0N#K~N86ZAQ{>*F;Y_45U`MEjXx$a!ZT7V)Z1A3;A39wr6rwqCTb z9`1_4)dPyCbWXc4g?z5W7$jy=y6D^{Bedtno}Qk&`^JV>}i?o-b8jNT@7CNENj+(^}8&A|`ppH686> z#kTJBsw{NZdVj>6Qe8G7G6+{XS7Nl~VSLv5H+v%2hkTV7P#XY%L&3H5XJ>pb)=si) zg2h8ZBWn6FVg079GXdNW12{*{H+fFBdfWA*+EorgLslsyMakSoRbZz||i;_U6_=c<#}$ zy`Sr3+{^H@dBDUio%H;z^2tBxP4>Rf$W@TrLL=q2Zho3>%i!g~xBsR~BF6Ke1u1cV zaLT<^WWr!IsWV;{DT7KzxkaV{0E;VVX#vC>bt^QHRLIy<9R!!Gb~zsFt$nE#UEIL1 zz+iHcbn4;ey;|$tw80tm2^4(HSMn&$5+Y-RRltO`ma*}v zLH3BIq4jXQd9TbwG6EIgEH)fkXicB2T<_LlGS2ZfS%lp>*=!Pt zc8qVma4dl9AB5$yC*}C2$r4J&;a~0$hE`n9~bYxfC&IX#)Efc!Z^S5@WQBk2^o~4NMStgan zDTx}pb#{uMtv%d|HGLKiZ?+{^ zie2FOFx`*~g23h~mjh=si%Xw;X@cUFKFwJ?_An`GQaSSBg83i)G0e+0VmO2PyNv4QRT#dixw=8EGkTvR@`4huvpD#Q=YrYFWtSU9nM6yvy;cF9QVPeShH}X@O@`WYJFPSBG z__C^?6G;Gs*gmA`RjTe8MYDrCoHll~cwLIRQjK(z)P6#vPB-XvIrjASo&eKi7o0IB zWmm80BUwg*_EMnvJb^sVKM@Kat2g(e8^mC-0M^A}x$6BI*0$#Tedl(QjNQO70=`%z z)bLW#fPu{GjiPX#leU!K?22g?r+i&)n9d`- zl$fdhla`Brk?V#t6WGcQ6bqcJ7j+*0o1MaeOL~iv;$T+!b=7WwS~uF=8c{`!=~h~a zb=1&l8I;<3+MtJ#dfXvk<8Ud>Q9f1zJf94=f$(V9ErlPr1=&!OY4BJWJN~s-Gr!&) zP=M;X!v*wsr|!A6{5l-UNbHey1;euLo@+XO5svHRv%a#n)zf|~AS$mBUDXw*IG!)2 z-+_|ce+QO;emf+!bfVOe(g)6ob1Ouj`UZMfdb74{^1*CEk7Uzsfzqb)Rg(Wb9-Zv@ zI!|I05X;dvUM0&%g@z7fOb>o!xI2%U355_fp@>iR6jSpEg8MtH)lGDn&1tHx_F5qT z)Pxb)sf~<3Y%p!Y_uBPdF2))S?Jr>KGl`~mfXy|!|&-4n_gQ-i}-1Zq~N8fSg%cNp*T615Mq$p9F{F<;r^aB{4;8bMmbqxBckUy(&-JSJdX$S~*HO`R)Y<^Y_K(;V0 zBuoPQ>tP*tk~5)0_l2{l2!t#(ZRz_L) zR>#wmPPV36YpttFn}kZvo5nS!Q*#$3IMG~1E=BK-9U#rbTPO76jUhT}tHphi`Jq9+ zK3j_tZAg__ALU!SEMTnJiA;UbdhN#Pc=K0_J4*o6RwtP;jJna&nfFHD>qI2lF&H~W zaoQp^TQt_yOk3<(h%&i~p_iTa-u7z#bn#jfH@8^ZU~Hy;-n(t-OZ! zdE`S_ZrH>lQ!~+GM8R?Az~s+bG`i^Zp}X?Q`RScmo|ZMHZvrZU=RoW!b{gQ z5f{9T%P;P9Ffblx68xao#p?`HeD&-F9pO+kO{@Za>J|%r_=x)4%YYu`k;_N7qtq!r zO68>~z*+lWvUbiI@kM@hVA6#>$Yhh+*8Y$9;gU*%Z0H+!$j4dt616k%6SGS<)f&;PS6^FP6x|NiwI-+2WQ6X2`b^&5G{OI&|( z;Cx(wp6yFGVP$0;-k~k!XGCnG-45>ZA@mKeG+`NWw>D{$anNjShOVX`RoAuH0St00n~SZ`}^v`w_z|^ zSrLBq%zfwc#0(k}>vyJ3FOU6uA`j(p~YPc|c z`rL$MZBC-3a4b^0Bo$^4RZm^JX=vT}eC zO|YZ&-X%~D3?}LNS$bs@z+1YN*po}4?U^1Ki+S#aAx55spAT9DB!$bHKNK_Vp;$?% z8MAcZFmON2`~W-<&4Kx>-|MkTMX1LhUCW&$HKGexqnEZGUNLzk0WMWB?nGQmjU)Xf z`!DzRZg8(6&;nODWo(|I{k*t)EoGW1GN^?Ch`MKoNI#`L zJ$EORy1hKCnoX`~LPYE`P7vWiWRy)XC8&i*0vRt?f0shB zUhTj>B1N#<$emDUE1e*+0QqE8WA$Xs>@cn83>UxsWB-i2?$p{70K1{dlb`EuyVlK~ zquMei{D=fb8PT$NjUFYZ>cF*P+BI}d!!hMM1G=cvkWs&rzF{e=SwFht9M@STgD?R& z>^z-PU!J@-mm)8P^8h5Sik7oduz(J1S=Bb)5!zpsamF})T;=ATO6Ci>X9@zSnoiE1 zwD>elrWSm2PkeKruGzFO;Ymli*^~9$9G1`Kngj^^l5zj1(RyG z=)V0f&2#&B^N(Q-76yd&NT*zp4)txGR+fHrs$~#7sz3aXQ@=m&>Ky_E6wPU^*yLz| z-@h6wiSCKHwqmR&J1^QF-HW&R-pv5r9)i*BXD8?#ld4$VzEf3L(z= zkX@OZtktpYj}26t=JlQ^IrvJoO>LutbwMS2nh)aJuu?;N;jWdlTLiCG1~A6lF-jz@ z!2VBQT_M$#9}!fzbLjX0w&n{d+aI8@Nub=dm+XGRd~RCO^md0zV>hRk6fAr}8W^m@ zVO`vCkhDPSMMB`Rv6r>Xi{Vw^A6mzS8g_(PdUtB&V{`8u{kvvw9y3$snF5|n@g8fk zx6c-vdQ3ch3UJ~Id+j?y0*xzf#`^OD9E380_^&f}^x@y3shwhs_G`c@FX(E;9wQ** ze6hjAvhtqNi2~+A!ehlSCb5^uui5p0FXDcEe^h3M6-Mly1ei4z ziL6o^6W>yDS#qwN-%PE~gtb`O0)VjG!yPh4)vwksb&gkih48+tS$(^sA|JcVjN&qZ z*`4le-4GxQ%;oMKL+=P#=^bC{2q}%#zx5gzuJatwxd{w_lnzm(*~=5`#)$Sj1$wV> zl72AOuA-yaB0Hqj2sf=&s(=SAz5MgUju{LPMgwr-Q7%W!s^~8u(j^}SefTd|^B-K& zoBG}U^V8$J_B_{B9)W-{l;>KnbF1S4w6*i1_9J7~3uN~C)<$c&lSWU>8@IHM5kSoB z@n?`Fi6O})^;93f?o z{K9LrAY|G7)jA3q)4YLq{SWv`cPA(c)4TKlH?Mf(J7|?2+t*%7p26Uc41hq0OZj&g`-iK(BZ#L?AwZ zY(}AJU{dcPSI?wx?*6~F0BeoiL|W2M+`%CCQNjI;g9^Zkw3?N6hn4g&kwjWY-oBy7 zT?4{zmkJkoE{qpRym=u8m;8fOYb4vrc5c283*E>OJ}0z(H2ljSFwSh&@YH`O8{{)4-;NaXCI9!nfVj zY#GefiaZP~_>;N1$6UNh?r8nq{JOThXoc#w&r1I?+E@GtARXr(9z5iHNs0G$qQ*Nw zkwosLlZYMoX+p`v<8&L6mZnX{s2Gf4k9Z=^Sgsve$$0?#vOKO|Hji4h2rb0)iryZ= z!3GG(r#DU$9(V8J_yjJ8ozI8OEt!&bW_8k-s?&uH==2Ebr;FL~a#HePees=tsZPEO zYn|pWYB3g2?Vx^ZeZaUTw#{P*HEf{2vT*OdBD0YcA>lmzgT7?3e#U2mfN3geTn#Fh z4yhl=3T{x=mKPnd;}Ag~N?8wDz+>`0Aq?Dc z5y9%~-AWQGL96zr3Vdd{lg=Gd_+5Nby=f=Y8-HE+NLsx;BiJ)br;$N?8a5>6*N+7RfFR|qs?bD zO$tQ4U8koo_n&QZLpOFWiK<)tYr2ZdBc2xrEwF=T&>_f^itNXenB?*>n&BZc6=j1} zEA<=+m9ge*MSl%ULAQ$-gwFR!5%%H!k!l$-o%Qf$x!iN1u=yVznqQ;&J5#V5xdBx8 zOv5C-EJU_;G5}IXiyy%x`85;57Y2kSYbWo|`r`tg#OOgEwD5~F#{=g;0b`Hmuru&6 zNay~UK&&R`kkgOHv|aj+H@lB0jR~hacX3(YfaT}Jc@}y2rr%$bh)zLw)b-U$C<`F` zSk-usnadHkX@z3N@$$`YOCsjiQWT#zDZUJbHo4>FDn>NQ8S1A)O%>S+3AUCJPph#B z$)Nyqy6oa36Wopvb4wjTNKY?5z_dZt5o>t-QC2+MPOIY`DsQk;@1QWTdiH* z3JsQ_;M zH>~*W>j9&bZ>Gx!4#(=Yr%AskeH?F;I{4>MK}CiJe$@BL(%w^wN`h@xGHLgXI!4sLTWcp{26>7^52c*{BLYv6?L&OG<+&by=AS7OP5&ELIh zp%FBk_bzd~j3LPU(1a6=f2xiuw5m~{mKR0m{{As&z9aIvKztZaz|g&3tfn;SVlBN= zZ^1mrtsDiv9B+@#6)Oh2WW^S@5neAG|JfcjJqu`X@YjkR{`DT^N0oZ7L`=@%V_Yh7 z?CpKYJ}?pNo@ z<|SP&scfMPBgN1pu4_&J(;34nSokOW??YqqJN@!LQ8Xy0!;vTrJ1<>m78*rX7Ov=^ zITm}nSjj9-kpPu_jb0Q*uey{Xrc$b&kv7&lPRBwISQrMm_wUX+J*w3I&2`bcRHE|N zn>fEj^QQ7xSk+w_NUyFw{<@|T;poYaPfja2Uaku%jpIY~dV$(D_NGqLLRS$%LM(@y zZ%%>8ReX=`mlpl@{ht1Nx50cCGa5;XWK>Q>t(_~M(c!FfCn{D)sc)CqHAGe~u{ojx z6W*ab61(k=pa14`PvV5bG{+*!kNt?cl_#3;_ftDzHqMY_+Z?d%U)+d_&6E-+D0_)#CXvG-T{{|Sd9$i?eUp{qCxD+#6{`aawtIS7P9D-$FARY$P!R%f=C?RoOG*CJy57~M zi&DMg)&t{>es>7zpBC9-kiV1N!@pOJUof#+)%iIutQG~#3-`{XE8YHe3STDrC|N-n z5l%bi9+09l(wBk^h>{*D1BeozWd4%qG)K9K(hKXb7)d>K+L&+5+MWl*7< zHtwcy?QY6FVW>k(UdfNz%4bRqF$J901SWQwp486O9kwowKH(R|DeO^n$37R!a#DXk z?G)QcILtUhTst!n65bZ#P5g0)_$E9gVhy7e!LDFNJYXLx5ARc^7?@gBEaAepY-%)} z1KJojiLAMR=RoLY*WC2uGO}dIIhYdb8wqi2FH1hf>xHWl^fB-a&(dVh34Gr4gYF>o za+Uc(KYg_M&2vf>0PpDD8Tv>{5ztieGdA5tcKzi5xrnTI*d4^Lo$utuJ!Yx>r{tWQ z8_#qa(K>Di)kEdKg1~>sol{AtdUm`ivK<6ph?^ZXd|2_a|ICNGa@wB+cyp9}dBl^n zBFQN-@Ed-4hmAS0}VaBwrDoGBtZWI_qY z>Rz!Ug$;{eT|CPDs@pgfwp2P{6G^u5-F>3vtHoG{<88|PKeVEhkaK)phZ+4?4u;KH zVCXgig00+n_s9Tt)PC~XAQxaaZ>(6&a8A}@qDGiL<*f1e5$`c@k?r)X4uy7`8^HD| z-%%QgajzdX)Md%n%Cn!SI&1q_K2xMEl)6tkSMGf~9NFKb!M9a(%WR4D`3xy;5XKWI zZtdZVs<}t4U41=WnO$poDWX^c!eSNQ*FCqDg$P$&oOrpk%iLD818mTAI85R6uWhjR z6i`N1OUWdFlS`4yeB;U8DShSO^EsC@FM3`ayK8x?YxZu3#=RFxmyCUG-mld&1Ru>U{6c*T2ZWzG~2k)vodHAuTTGEAwZVG z|7fI2;bpZ=dyLy+qR2XHfg}%C?Lw)87F%}D@k2@ss}J{o;tDRwK>Z}5kp62g_S++? zJ#swwbp+7A@Kz%zx4S>~4jV5bvM}yzGbOEK zh+hf=otYXbIqU{fmO7MFOhAgg9gEaBvKz9-oIJHH3~o8gjN}JO69x;a?pmSytAXUC z3WB6XZJlWWHeH=s*-c0@Zvd0K?tzYJ0>9?iNyNN67cyBPy$^rAWzfqxj4h{r=YIE` zIOM=$4&qvs)YtCL-@=yBCpX~of3;;lBaImjca($$m_ilf`#%|t#*DyaH_Pd7}iz-Uyt(+!ylY=nx29>7mSajTTDBv zdwAZ5k{A|^`Gzeo)vNn!!)6N;18Xqfwu^kK{VwCMpHHy7Of|yr`Z+Sn;ORiziM(x1 z@}}wMlcLvm1GE1VAcdRyvhNG+*Ipzu0`MaB)Mj=KfDT8fEu_idQye3g)!Yj02&nzS zUEXxyCv(d`c|k9*$NEwN4JKuLmZgCO+40}h!fwa$nLzi%9xlpGC#A*uCr9Zk#0FsY zo=gfA)9Z1|+-BX~5s6Q;&94#lkRYU?7{$2;UcWsOd9>LdiS{=)_KD3l z*Z(aY&u#Uvbd#-4x^)9+RU9~i>UY#ucej-5<@Kg<`!LDJtdjBH#*^F%tpI7|a?7qO*5)}hj=f!RBhDU?!z>tf6&t4S zt;052q3W%%l0!8^{c&v|NGuF#MThI#%IXYz{HG(8zbnHc>3rqce}pCe>3h8Eka!gk z9(J$%`ht0w$Bf1ukEf564Q8VRE3ov1KtKk+s3F zl`&L9tkWXSu`@buRUQ|g2?>QJx)lnTneLFxKNrP%Osk^57Vz03!d3Vylt%7PVL+73~{kUd>m={bq3kV-{6e zg;f)ZztYt2YQ|YW33sgrnwvvU3--XQv&+qEJ|C7@?w&gVg7>&(+X<>E*}CFL3$d`~ z8cyjl3hi?FmJY<$6Qt;|Y?rx!`t7W?{GEQ6iIpT-TI8Yv*)g5lJ@ znoJB;b4SR69JcgEB&am^mDRX`onNanj%C;1VAwPLLJB^%;|U0zz(TykI9pdzHApk9 z-MdSo-cjrK;1+z|I|QEih8ocA5ILGomOcZaeH&R$CFHiGX%mw@x3uE8IQTHCitAvC}E5 z;|4aQy6RhrZlw_Flec#L>}ghAad2fo!N^yyhmij?4|o85iJ-?Pp1l6YzRS-8`v>#1 z_W!L;&J}SjIdG*-5@P&p9)Crjm2VygDb|*rQkJgG*9q|KJ1Ctcybot6L?rBo8yg|Q z2J>uAd(+3QDn5pMD&cxc){o*wI@hs0W|f!PNQU2rOpsmBC$>j?EeU3{zBrko48Idu zQcMpo-Re(#7KA$l%lNN4e6HWsCms$<>Q4q-&I+B@NbQBtMA47#Uc;R8VvU--wYTkLE6PX|LzRKFB#O~F>w~g4Qa;b z)4X6Sv+7F89VOwW`qq_u`SIA~W6CM`Z(&xsSkHrLOwaQBlSL)`BX!F8Xc=XQi6V2% zDHgOZsCOQ_%JZNJ-Y6AR8WVx4k;V&5;lAteEmTF>U}2?f$wLu_xV|T2dX4(E(irL_UWyRU@ zfet4u6u1=@{e})*U>3;?-8v>{HC$+Q%O_ar#T;)r41YX}5=M0|6MOpc@~b{hT8A2H zP~1^7vAUL#_Vg2=jXF5&eMiy=HCWK70_0MEp7a}6S?_^QFxO2Mmtw&=t<{}PC&_tf z5A9ID^+bukD{VI~CK0TQtdN$d`Fx7pe`ST=?Yf0S|LzuIS_X@$^>Fx*?P59Xlih6iUFzb|-QkIeO(0(?N?{kUC)Ww~PWz>=qColOI!@>CD>1b)DFanqv@8Je8<(zR=c2?_W;K>9yd!jN#7oU%lCaOYiuZ-(z}fiJGgU$ji#oN0s^Y zZ&9TQpLKQ^;OF{0c$-KWi&SILCOw>4KdCeGvQU>SLNjc(?0~|empJkxtNehPj$cg; z!oEsJfTwLXXiR5E`cEZ0Pvo;9zU2E^GQE1P1~?nUOD*O!XZ@}0IfmOgNBM(B+P}*} z?hPUQD*$|YmPi8s8J4i&CxaQenS;`<4!~J5f{=J12Ad(*v0Ws2YFZ8wKbViJ-4b+- z^Mn0-VePw?$p7n;Ie^Zm85W+^*a%$u+e7pmQq_J&ZRq4?X>{7~kact{sz%99|MuD` z_XQoHT6|LSda%potEeVOfEVxw<~anZANZQ8l& z33^pv^?Df*dTUzRMGTYS&DFQ_v>ulap!Rq;+etC;~7x?!->Q`51L$l06RvTuiX&qjF!Bp0E!uI&wqQ`4Y$ z52({z={=~|QI`n@+4E%pTpWNv)#+v7Sp1fIc{o|d%J{wKr57|OxtUN=nBsM+fYW52~a2d&n9KP`0kjK(A zt$*faUfet4=k7uYSyxXn>|C+sQ=TjvpECeouOz#<2>(`VdiQ+h3kh&5bR5A1itc|P z%dpX0a}!~9+$GT* zz1X4GbSVju)m?heN6 zW!E5kpo>NIGybgp=?J=uk?No6@>Rsl{3D!S%&cv-T|Yc!iI(@;|M01HhpwRR1Xt`B?$D zXI_CK5r^?yOJDhTb0sk_^)s(UUigvz!Bp(C%__I;QBrj>ppr`t`dZ~W;^8$ zMaM;a3MEBj1?;5q3=+W03rY!B@k3!Prcccr=$`heq_hqNboF4Q`1^u{eVEigFV}xs z-U_ACbFo$xUAND5gf$P#j1?$j zr)G0z0{-E>-c8nGX92MDkJCJ=0KOpwP~pv%?ImxH@-G?inNB~s_ZNJRtrKO@{-bi( z+?upa@8cY%gT9o*6t|ceN!BSedWd2h$$ZmBfT|mb#oe=!Mcr*|OLp&!e)!R>F>a6R zt=%`57!AaQWS3n2UwV@S3I8TOyn#{z`Fq*(+@RS)s_NsRlUr6DE}$m7$R2&yYf4#o z7dwAJ&~jCOsMLa?fXB7l8>cc)4}sm{_+A2Y-HleKKAuxRG}~*!hPgij-qM4nu$CSZ zw7IjL-XcDzG|myrHbN7^2OIJzr>RNPnKbP|NNnfQadwK#&}oSBKIv7`N$vGm=>$d; zeHyodZ-lzvvLWjL`#&F>5!*t^)dju!JC@OMBuQOn=9jjR&yh3G%v}-tFO9&$Aq{!y z9<5(=P``BZsqti6M#o=Pp7Fe6X?&$H?p*z%Ctpv2+q4b=*~)WHyRHy)Jb@2wy;s3e zm4WAjH_?K=UzS?05qY&au2dnh`eJOd8%)`VE*o_=xg9rq%2=v$hmUlw!^I3)9OHAR zcr7VfWd(Z9tfeEwrcpP{WL>*Tq$aJ;H{py2ESc>~otY$ZZ{8X-vM-)yS2iJ;vHS9c zMr0uL;mc^3Zp0iNc&fE;p||@lvSg$Wq^^3Om`8`qcN4Y2>b{Ptvyxm78GbSrZOtx1~G*%6J@vafD z<6Ec{5fthqV6c{%yfJi3h2KF+Yn?6|rC;bSHrdU9XoXxR=AiM7t5|1f*t*UT5seJP zuinZS)>&ThVuq~c`FQa>Fe{25xl13pJvuRKW!KqgK+9?4*ROW!XjL=anM|Qe5s=jP zs5&*fB1PdI>GC>U@s>A!Zect2#-X27Nx=P}xHXzR2c ziWI9_SU0mXjKESxQt-FNhdb@l%m9DIjQykpdGCpOx3u!a$}Au_Vm)Aw1N2()wBaJ@ zwqK7l=H6r}^L%I*^Es@rZ(VX45-MzHX0x(6t?o6G-n~1r_O1_r>on}K8^5GutEudf z&PGmnDrg&n$zO!nO*7L@JJ~j?I`4Yh`%n0I|mr#f~=^ZEAsS!z5F3t?cWV3Rjn_Z z*^UnZPu;S4%hUcw3wC#olK}xJZ~R%iNeMHBBC%`0+9v`a67Fn;T;G;%NyO))qxsdp zvw#7i6Hj)%^J3O)ME1P|h6mMY)!3^z+YLEqKpH@?Eq7k5@+LI}5vPm*JSLXBX-zEd zWixr#cUdz2Lc5%SHY@kue@NAL6F+~c1D5)Q)`i;nh7nyE#0d7{Ngcli7E||j16h(Z z+QUwY=(Abn>=3yh%6*a|IkmMXwN{`(q2Wt6Iopg!AGN2ES@VKJ()_~KlP*Z@HfNnui z2RfU|0~5h$d*hD&pyKrfyeByzJohv<4v0-JcTeqm5%}t7;&Ac5#HFyi%yM-Qm}@<5 z6cg8`%Vk2hAHN@8=*37Dq!hOn!Y~TUI5*lqnw!8G6?yj=m6{<{f@C&DE|KO@TGX*dwz& zju;!_i*_*>3e|pH1m?J0FM(9Ggu+*^c|qrjWmZexBWp3}DIL!;h^%ousL%>->!Ld} zP1`VyJWgC**3RQ*Ws`3ox8C@^^P^Wg`qu@3Z6~_%50LPait)~+-%y~U)PB*&CtV>s z_M8>aWg~64Fk!n}Xdh&2d#2C^5*F7?6ccQAzQe0>kTHiI*M$ES6LXgg>8HF(qJOYSjGxXLXH)$ zZp-iM2m=j|Y@{u0%>=tss0?M=<8LwY+@(-I)>v#9>%?@Qc;=7pMr}9DcOX@T{w>VY z_EK|E#miX^hjtY+pG8sbYMq>7>20-Kfo|JfLR&m%+z=Bfa*AmMO3g;64<7d2Q!iJ) zAc=|)8Oxh$kQUAwDk_cCp^8|e!2gtNmFLO&Y8F~aT0XRz@dN9ba61jptjbZy_!lCs zl$Q>t8`Y|`PZm`l$?eyGQ*i@|%Dshtxppb@*(5^pz84+WjBdt8+tUbSG0Ka85i3>p z%;=3-o~OX@6R#888;Q+&*<`AGe&hUa%bB=;Fmfvq`2 zy8YWcL-Q%HfQiI5NbkIEsu@{YY4Qok7bZ4tPG?naeLu3@ufenOf#}SCWbLddU#AAd z=?5t^ISBNYXRn*k1A~FJQ9anFsZ*UC?(ILVVfsA>l z17C?DZy5_BMB3oHs9!k68}gEZ_h#~4RnehRXeKx2hBdh6g^Z5kM|r)<};vT zefo*+vXh(*QwAY{*`S%sRJMD0=XYi1LS_+NYyU3C^mFrW;wwYEef5(EusaUS zn);k*_=ws^?cZUq*sSdB{>wwQb79?cgmCKu)BgnDziT!tDR@K8dFBW)QUFT`2O3@%VUPmU!t2(1XB9y;K1__k?vs$o+(O zXNFX0{?AD-MOzr22T0#N9r=@(E5%HYD|%SSf#BXP>0_NPl$hae+t^Y4s{cAjx}`{L=1i=pB4J1XJ*MtfcWmt>PSjy zfrH-q+Srt=z3U%n?Qgob@~)ETa?fe+#l&-r+mz7PTc$0OLK*#MjxWtDz5EsGZEAN+ z&|%KnDKP=VW5^Rx@ODbJ*s11OzPI=98KwDa9m0VOc{f4c8Q0O=ztn`|WLCwna6f6X zcxMzOGn?Bd?2ePM3ZRW|d$rvOX2pNal6sppqb5AESnf@}#nI>6g=c=$%WWVHaRMld z39Cj%Y}5u*?Y?gqmp6p3`N2WJ0nMjLd-CYN*x^3{7BVmB^5g^@%*_1p;zN+-$mT|N z86u8-3dZDF(l}2Kk^K%ov*hy~Hw%MC=mbvj`-@zkgEjdmL@KqQ{mBp&fTc7U!)l(- zS{@Cg+JKIBBJe=rhV}etN>a_UTDKaFERTAI*{~9PliFV)gWxomgeS*hqEcf1npg{U zSP!o<_EZyo!T75O_keT;Lkt>^n9USf9X2cR60~UuF9-BB%o3-7&)X7NVd7jj5>kMC zEX86r2#L%>meTYttCz|ncmtGq3HCwOgr>aTZUg#48px4bFMY)S%>}N(0SHW6gQ*hC zAGeteZ)QzAB+P-XPwlhVuTIPwLkG9GASs;rJDY9i~a7($Qus2Fq~B~Df+cn2>{SMBglM%HOlJ#~8W zvUg_f#|hg+GxA#jBAaZ?S?|GILXjA{&$af;o#qe}?5!y_w9^GcNw#PR~}0OFzP~)ZaoDq}syc=I(h4U?$J3 zmzID?!K^2|hU~OW|Im)AV0t!yRHKPDh>$Y z-<4JVI0~d0{C)rLVDGuPlhWaR=7h!TX|N{S8@`Ngvs6hKdhYTiQgR_di!-f7gf}JT z(^B<`)&&rSt5aiKo;$K&-F3}abIhz~p$5GNK3jY%um1&Prb9}eq?%FEix?=|Td-|R z(N^BqhschZpmeyB8#cPN9}@xLvJWL4w&=T9Kmc|ps|4dQ5$?W+QoSFmlMV~<$#inE zu^>^IdJUqKQ61pz5mlcOD`O^3%qGcSK)&Adb1BHkTS42f$+>4*R$FIy%pc(VpHG2?v`p@2ruY$Fj08+h!FkAr zML&Ep;Xl3j%#0ZqEKI?2dqB`y$3PC%!j7n|>MABI9~ZP)2mxrn4U;YOL8*wpb56pD z5}UDO-!tnLIukPL0|3e8UEt=~N?L*y6NMJ1JP~uqiy2N9iE*2_3hRr`efX|N#Adi~ za2e^hj5XSh+3<5M(eWld5#i7sU>6+?Vzh*>0t@YMW%gP_;{>jgeGjt|&CP5ba8kJL zM!*nTd!5oBLhRkE=5g`nxl` zyDF}eV%ptHRznj&2BQb<{4K&m;uDn8@X8t+Rh)m`(*OSi@t)wlGu@3FZu+xifS91uUQU@XvK@3Mt`xHn1}gHoDrZFHYi#HC&kQS z=9gZoh0i9LNet0-PY;v)DFfZnKZXPso&(@H6&@4#zY5j=;^=qgenS~5r1U0h`2)ph zRp{upW>zyyASsHfvQz7>(VZ9e#xxa~rhBsCt6aFab&k~HQL4+ z8r7fEOR*7tsk?e)#1Cu_e=JOf4j~ifOTsqJ9@&k^eJMbe=5soG|K+iL7Cm}Zh^(Gz zhcS<|2Henlb34V`aqA7&KC{vly!Bcp*?{l?bWEX8zXs)Kmf(54L(o$E1?myYq&N|C z08OI#H@?$^a3xv_d-IC`T9NgO@WXn|Sr@7xu>iYP{R7ob)H>Wv(4d&7gZ3dK&z0&( ztFRkqLy&8b$ z1yTWWy0Dv`g4Q(OFj}Mjc8$etd^S!2r5z6ECg#hB&+Ua&F0p6}Ex}Z1Hgk!0gP1&h z_D|O-oazQi!A|He>Ma`=h0a{ar6O9(b1XB_C?nQlsLCO_ZYsCOausMCQ2umPF(#TjM^#$dyh0zN7sk zY4`8?NaI(ne`7aw9lD3Mf{U;y$ocQz>H@*ve02ze3Ha<`huq}DTcz2~Y4tHB`Fu(cMIG{Wg#+l7bwIP+miz>f?U4tX5n1g(mT;`TgUz3fhioUg?u)|k*EVWf3 zp^iA}DpkC6h9y4#X$ORIEs;lfWIJq*AFP{$nqnb(tT7Sy$;daUzwJ)vjvE4Ld;dKy zOt~bAjX7%^PnM5Gu5XbFb)AE-DLt{m z#E1l6Qu_)YKIw5t2gkEsuj}dy$bAW9a~)Dy$HiNEQYqrg8WGE41{I3W@+ZzHSaY?t zL2|dzS6^1xtL%{<^${zBq?~&uD70&?BA`wK<;xS7qp~(D(%MXlXN!S_n_=Q@0y!<~ z&N-2fR3}@&gEpF*SnPNJH^0V-D*Yr`;gUHptk)=?!#6BMV<9ZAyeoxTQn1Dc70GW0 z-&PX53F=0#nm5Wh<=Pr;FOH6Nawiv7BS#CRYajblj_*X8#r=3g1HgUOfR#^#WB)<` zeiijfKbKJ3Aso9cd+OUK30*I_Nmi=Y=n|mhxY_mEf7PDU0x*@HrorBikU4}n%+2J zl$94LG3au(TNx`H>EXX!1j3o8S`B^r;!>FQ*SKHtQsS)QL<#86mH;NXJ4Ch1><`~; ze2d%k3;caobKnlMPrUIG%L7T%ZV5&W)MXCX*Hi5A%Cb*+8A+&qg!FWoea$Kc-wE*& zo44u&))%mW9ih zKdt=$=DUD^m=)E=SUKf$w|{09XaF{DiAZIV{|-qdboJmfhl7ey+06wRmgMcsO&9Gp z=Wk?-V6z5%#KI=jb1md-CcJO$)%md|#mxM-DwRstwUekVPT4MG>lgeLNhi0Dh-sYA zIe@1@?i`Q8LvQfY7ZWM051GsfLD$la;u&ahp-zG*|S+Zq8;- z)3@Dt#&ruLs^-7P0OqNwVlMGhzc#og!8XgI3aWY;_vflp!g`&C6czi2c7Rz76W0=2 zQEg%jz=$(CO)aGwiSps`E3J*JCwBMMB37xJ4%^}M$BJ_Q^_lG*;c1}V{^ds~`ridI zFpHz+?hg8ug%qTfuv5ET(@L*O*r>)8lQ-6M(JY-8PkTHkd?U;Q8Gn+O?5HXQ=RFS5 z{N*cIf>u6%o99TzUku-PGVw$KRmQRj;~`47sS+c43eY777)EFMSzj9@-5oJ$D-=j& z5RAm^{V<-zgN5rc<8W{pj2mZ|L9CgdT zSz`wj={rTuLvW9#RN3C-=L}PI=WH(7pOzK(bul?FAG+EHZLiFYfaFN~={uAFbCS9Y19 zOR_GYzA;9pmKDk5cwn-;0AXxMKWYfgb;fU0(BH$S4B+w&CYi2j(6r@0Fj1#gwM85- zlTt<}I>{;NXxIhzn$^>(PK8)QtL0~ud1f9e6tw6{0R#ROHx z@@QL{h{PCY$21df>D9>!)$kn%|Z9_j*`{O zlAxSu+d%fb@WT)fgg61m`GyL#;7+of#?-#R@8c13eBlBoSmUtqIN`7ywq>oQ(e|KJ zOBvLzB??6LZCE3Z*$%4Zo7ul6aL#UZ=3YSi7adS58!;oXwWtiz;H0W;!Y zO5>ut?2I90x`kCQGZmL$#jCJdt4?|%))!H6SZq(LbD~Vw?cZuGl8Ra?9lrkki%Z`j>SZwVJSkOrm=f zm+Y7O-MALTo5ue*>hR|aIRM7fd+j6nZ}CP~zyu8Ua8t4(qXkSE$q7*1nvN`ezeQ`6 zFR*aKv^JL!lj(BPNECo)Yx5zbg{A&wtl1O%e_UT9ZUO5KU*Nf^3D7_zUI`jmS{-Ye z7INQ=)LMT;DJE{N+4BuT`Ix52J&a-OF3hPE;|d%z{kxg06RB20II^VW9@oQ_FY~C} zv^|j*L3*KzdyEM#2bTPR=>fJFa?J=<)T-WYs_3PG{~)spA9zXa=p8rM5HFdLOSHk~ zCL{r9A--wT8-a_tk5X)C4=t2?P&tVjkzN=E6dMoRm3p?#4|?3BIUXdyLzZW@{?l?= zp8D$yI>+$%zlE8PdlQg^MZ@y^+@@*=He7uQc$vt%I%BXaLT(naLn7Q+&)BXX9U1->w&$3_v1h)jg~!D>FflX0CokV&ey;at6P4_UWc<(cUB zNt~<46z4wtdYG>#oB^R2ogJVc_ghTx>l5Im)HpR?#*1WdzWCdiGXx`$eKEMRB>!5A zHrhqO-mlp})}KZ=QF7p%70ENcI-NicZZ3Q0{?Rm7@+>nvu35I4M#w24NTVW?u8%tR zHY8`aZM;YJx*LlA88C7;d(xlMx6_Lls*C(SXCK`6;K2ittsT&z!#_YP^Irjf=`J&m z`?rC(nE>7!)QhAa2I zQ|pF6Qs;V8`vb0GJwbjje?TAOk}b_cTJ61CS8QZ5=KIKg)3`I{g0E28E3>eC0LjqUxJnRCWS86zRdH+ z&P~~04p&f`+8s9qps$!9ZIK08{U6Qk795RF6bq`5I+Gu9UP7uIGA@ zH*Py75g{HIr&MIm@iMe_vi#LV9*qa9{V>f$3F$ge=2U875Qj@=dW)yeA590Xe)}WZ zS1*SmXT{$h-8k@URt_*rF?Q|Zy!MtNOkqAzYh`y&WKL1|3-DxT$Cf-tY|h$Gp@UA| zq&F-j%%&Ll>@YU#xjRmG{s}7?BD)klmz*@&pGKK`<(HllEAncLu<%6!`}y*`{$s~a zqx>q(+pG0k-Z$4*k2mdbY!LI>SPzg9AO3~2pmmbzVXl0SI|f#OxiB_G!C>O6?i@dy z>xS_qn-eA~IgpffulS2wrcV|>!qeHoKQLwQnx!jsZbX03YD)V@+I!4K$ren%FkMwc z#8bC5RZ?9f3BwMcdKeZ9qRP)s2FDxHUUs3e6yL3+D0R3x`))|IuHqYn(W`}F=VR|a z4nqV@r__8;v>_L8ilMzcIQCL^Wibq!73$q-o7)Y?p(f9SWEqx~R~Tjf2X67PiZ+n*prZk1<>YhlJ*5B_P-yIEav zK@_QGxmwJl7;IOQ;iz&7(buQp#A}qd&=zaQ_a!!tKQ4{AWfdmer!`mq+L_rCb2C8P zn}7v=_)8}EQmffs9ITJRM4qiI!e7uhp4V3^lhYi27NHp zsIF2ES&O}3RFn_dXgY@e>IS1g`kO0Wgc@|4L(6-}id}{^SXKGd|)Q!k>CKHtO(mVGfhyc=ZPRq@3d(gP<) z;fJ8}yz)<97@;&|;Moy{7+j5dHd(gt3f7z*U{AH^n}gXloc%s;{?apk>=4u+n}M{I zb8T7L=I<;_4>_)vk5z71i)O1owraZoJkaM_@H8s?pd$Da;9HpW%Fn)e!0c}A%t1>r zApE3NG#AC#iL{qh;)(TYD@kM=Pk$kTwsMU7@tJkj&Xhy1e$wCF!rW%l$4q#a3nlCM zr;QD0cuC8&ZMNRFY6+-#7XUlZizoYrq!>G4J&knY)V`4&YdV=~PO|sb)>fwK=0*w1 z{IcxYiiVBH^A2#ZZo!UNj_r@+ruASKga4eX3jTSnbHDeBFHY`&ItCFug~+mU`q-YVWO|Jd5Z72wIUfl@hv__#;ip+T!s?D-l`kQqfu!P;WDH|e2vDIvS0U_WIZ13}KOF_5)aeA`VwU%e8DOFE~% zO)Wts-RT+gd5m$SzhmrS*Xc^f2Yebco%D?M zBQDe52Y@O#f$I%&K}J@zK6kSbqyQR7|9&`PV+YPTqYddZ&l;Qzb}fTOtZrvU1_rls z$0d@L1H`u@6SQ}0MJ)+QYF}RDjNMBieZXktRMJSihaz5Tu0NJfRTAwX$75dzx@*Pn~yI3DGqN= zo|m7F%dbqGLc`9*EPWGHM(D-g;1_Nb=(??m_GxA(LGqCDAw0I$Fos5LECDp8%fOg$ODkt-DB3@W3*ZE%`@ z5%moCEMlDGO3u*dApg?dYMWK}OMIP6Y$jj5eTT4_AM6=1l}E&rw{88Ry|xk(8@)pL zu@2f|1w9gg+syQ4sG^e&P@Nu^GVrB-Jq*IeGop>!?nwS%3DS@H;%%ARV_n5P+8 zlzN5(XHuV`h5Mnc{1wMWhenHu3{Y8T|6TL5AFoh6)z)rvjwQKLFv3?i&A<2Lr36ji zx<_5SX}gtGxGT>E^zs(~|LOnr=Ry0=JC_9k65TYPt$xtDVH&rWc_|4oY_vz=7t&Ku z+k4L9a*aqfNPDf^O4}f_TX9IjV5u^Imf_3M{-(1&%-QSggtmxUJYl>PH}RfLa(txd zI%U1{Z3jC0y8c{>ilcI7koa*|@$?B(Svw+`DyY^QVKdaZ_fI-6#txliOP%Q9y9*=D zAK_{mqljsCsLYL<8E!Q)*Me^=OQeERNpUFR-^S7QlecqBn-bx#RjqFSu`D+G;O7emdllQDjj4z-x;SZM5Y6JAn0nTY8+|41q9ZC!_g zbptz0%&IjRD|CwL`_Za8)_Inw)E;Yn-STANdf?D*<&BZCKm?`>S}O7`i0zR}utu`t z(BM5VU;$v6xo@(`X0_LBE(T^Q*$zdQXf}Srq1a91ja1UTl9FCC;r3x$-Mmzkn3Qu< zG2kRCW!`>Xf_B)1n6clv*qDYdj%N=2fWox@1d7JP;%N#$zd)!3+Z&SG0}!WQBqxWN zZa=*ckil6EvCDlT{7?aLN%%2gt_NZxH(+`{37N}yB1-QQDeW;1=jNpxCVROkD8^Xj zoz=>L_W?{i$-2WVx{bLd&pO?Nelxqf%4y)07M;GP7U>lF{q^HOFX$S@x7meET!MsW zzdL6&*ZtjA7;7Qfwq(DX=#8xI91|4p-PWD#T~XUFmc#7p0=8&@4V-h#Q)n#QDLK(W z$1-59Ms!fVvRHHWi5C?f@f)sl|u!`0kRb5 zVag1Dun@ed&Oh6zK#PSL(9br&r}aF3ca*hSIUc|$A}?j!=J-?qw~5k{sUb_7ZP`1N zS*4&xkG@Wzv=bfd3Kf7At2?>?^n&_FiaZ@LLBp*qQ7rY%^K|k2;CN;VPzeF?$5ZD6 z70TILVm0v6GutW*05f(Pz^fHm-9CpJKv21ByiIJ|JK&Be+>tsXlo%Rdu0xkQ(-ORj z%#rD~qFhcf$ta6ndg669nT*L6BE2vMc$D@Bj;$1Jdq0Z!@sO?o&~wbzo*($(|Bg4U z@LeiaZrNs*2zhbL=GP!m@TQxBvAH?BAF9UV+UZ!RBBx*60kA{k?6nZ(0EvyN{`@A` zT_kXJP5n{EVIpFSqZ&aArmE-ApS6BWri5imhqoM=HGD~w`05jmF)Z$F@W%~S&z<$5 zJkiN=)&M{r1jVsXJf3|UgU{@7N6#(PLk>|rE@pL0hYS)I1F@Gp$@3^WT|XcWPN8*- zOluwCRQ`}0mlF2S1YJC52P z^hYpL!B{CtPEM_b?H7EACJJf41eAbztl=O+`et=%7`X{MitivGR(O7i+6a62L-xn> z7TGoN^povt9w&5iO+Voa?C^h=OYiWnlmwK#x5(pbi)elr2RTiVu%p0|QK3xKAm(9` zrxRhLHayqOoMM1iot zV}(BEr_WMa-b39ty1AwLr`4t79c8W|-m%>s{n{VeuWn}*h>ne4&Yt|?I(KdW3I4IM z>Axkeqk<>M@~&s$JEH&zwyr&FrQLg;r0L7kZ}^K}Mpy4u+&Gte-DbWY_y1Kz;R2Q#gRw)@9mC+8|=s{D2BJeK-*TOcSz_>1iBV*yGe# zZcXB*A+x6v(J+F}twD?8*<)c20ZMrvv~Vh&95yfuJ~HaJ3jDQ{$%!56e{)yggWXx_ z^A3%lm~^^Q_e1orT&E zTi#xw=O4Bs(I*}cn&eD(ONZVFZO`-`3|M0AY|+n&l;QgNO)7NStDWfTj^15Bj3PaQ za6V9B7xE0rx|3oxRJ5eZs*Tt+|Ew0tl+Jcxq&OTRgCUWP$z7m#--=!&F3kYv=%&&H z)kwa9C#1p`{|ITrc^W7#x!u;QaeJrN70_H?$+B^w5M9*hy*KpSqp@ioUM1+W%JZR! zMB=jM{1y{Oz2a(?J~X{E1+Xt$IqfMy$Ei0~F;2qZxfR z;bF$zQo}!^EtGN;Vhoki6jJ8=#*>B9Kv!E_WW(K(6ueIsUZ~73Nb69CNNXXQ*jizY z4_^RhYRz@zs-qR`#S0yl0Q%y}2Yc_(i61Npa@|7$Mb6oiIO&J9z1biWcOnIQU_|u zb8mS1DMw<>s=%}1M4~`j`07Y?<=3Rq1K9T;wRM6}2^QmtYyt=#k56%u2IQqgigI6w z?t{o(49+mW#gxIfe#QqiFd|4<%QdYZ6!Rq7ziU5)ECPe?x$arq8$zi>s|2TzNDx96 zl#VnI*)<>7`iL3Ec3~QmtHOP^g?_vJ_i^6vd00Dft1!!VWQXd3M@Mf!HIb( zc#iLb+Y;@QE!u_vF2gb*yvQJk4K0^?ueFt!xC)Lqibk!#glq>67LsR+)W>4}Lfx_z zde~e#*p;Cu95aO3$O7wdwgDC>DH#%8*J)-)>C&ruMwX>5dn_m*tbA?-vPPfLH_1T# zF_$90*}UXz3lt#U<@kFI3;gCHrerWwk>}t_)@(YMhj@}N zb6^z?fvsQTO!R{|hYq45(jrsYx6Z!)7P>uZTgUl+SW(eqSh&DnD;V|$PbOmofF8TD z4NqJ@Mr>~f|Kdz(ygH?l@S1_KOG%C3Om{K`R-zQ>-F{79f`>x4Ux(6R&n0htdz=5{ z!piA>@^^Z$imRp&&KbD!cZhaa}iqL%>2iZrR{@k%w|btAH?gcKBn&im%q0F z?u6OiZU>l$-$HM{E6_-kDt=F^B`rwxUJX%jxm_4x090GV z9pBRHd?PMC4|(e~*H5;i6_~YXH%{cu4~}lK=NtBCN6EyR!>#fCJ#aJ|rret13J&TQ zi5}nAYy6nri@3*lFDg`RyXjC)FDfM=eQW(M#_Q@#>Ra zxvZ6@Tp+f7l5FZ0Q@lB@-#z+`oX=^+X^=a|=;j3W_t4esV_UIJp++Qj4!$_T?KC3u zjf%vhU0Z%WBm~68%du^!pC_0TV1OJ)W;iOBYYLN>2M#tZT}wVg{#qETX&+pZex%{3 zRD*vOF>I>g=<1jT1|YyKzX{|#QLu(cq#Qw5E29fp{PF}lRSShHSPZhR-$dn@XU{ts zpph_goVar#hDs>u9FX$WA#H|HE9LYk_r`+R~RbfBYP~Pqw%w zZ3Qm}#y1g($WlWYU; zF;P@gpSe&-htLrOW{%xj7^D$ckzk5kj8Rhx^`me;1%vo`B|cZ8-I{TZ<#$H27!Ia$vG zoDgV)p&qzA*Cg|%#BOYh-%Qw*M4~Z{=DCl=rC3L~fq&U50pvgwb)9QoeagUv=DitX z-3F)*CaR|^DWHeZ&K9*yXF5E*Hc$f#zom3K{;RTM=s9ots_9BPEg<4jNq#_ZyxUB8 zyr%~YftIf2MkB)__IG9(Z5S+Q{a1>|QVtj^x*H*WjwrgPzw&v4+;44DlHCtH2um0s|0Uv>jGToVL_DGb+{^iZxIG&;COk9r_;a)!BaNV z+SgiqT$3PhEAg`2r$p}qk#0VatobPDjuJ_v;sQTqdnU;Kin6UxwtRbEq^_$6wY4(+ zj5YBoX0K>`L||g;F^pjq2`7`YtOW4u{H5U4 zrMz1{8V4!B_)w`v{(x)k_6_~hIV`V}4+lSug#@)x?F!~g0D-4xK`tYMEXA9+)ps}J z^5(fxg)eQ4r#V|rfGPRIAN_E@lD~H?@0hFp;FuLY0wN&=3)z3*r;+gU-QP-{AR|5o z1NyhW13j&&BG1D7!v}ZRtwpDB^}BZ1U4?OfYvr#CjV`(kdZ$PLZx zrL@xPD2;{WeC;(H=(Sglx|PpJJeu?80}Di!-CF<@Vv| zIxr&!IvF4teLg8^(5C%pm2eSmOgfz8xI_3 zkr^oy^~GMt`us3B)fJCu^jPKk&i<`x*6Q3x{e8UO|Ec%>-n%DCA@G-<2RlDwV5dpY z$UK)9jifhD&t=MMC>C-h-kg4YT^6Uj%FVAgwnsWQ)ZMa7kwLWfpZxnbA4u-bB;?}bMxo>+oJQ3H|Fvor_kNho_T#n z4^)E^vL!#e%)aa72y|V3i`wR$04YuEnHvNCRXh1%Za>?$FGC-dxdXipoAA#~j>u~B z&g0)~i|FN#J?{G!>L!Z_9(8z@_1DWc?^fMhRazgv)?dwhrw2#GhBt1QDBCB?lik&ApD7Q5iqIt)s}_z!CE(xurZkrImi*d>^Z$RKjuttGO@?P2!y`_=wUNqq$ups zKmU&lP=Ok>c+AepQ14AgOqZwL?iH!+Nn!DuNVRsDGY9O&sZa~-8%db66n#B%x_z`l zZ}anIgi60q0i-Tyh+Jepzb&OhSn{BCGd85|tXm5)^R73f5K~ zLb2I3tC#DRcolrj%pM#7S;~ z$nvk?SVNj`h-7m6-`EUKV#iW{@8frzw*WwOOwiFMvVytdad_&yx$EiPi2H3kc)5Ge z9CoQ+tPJ6CMaOuF3PKxsvOWo-4 zLLYleg_0X>-yo%Iy-F5BOKqeWL6JJWWq4CGw!PlQjSTKBJRELNjfVt_OE106nER14 zu6YA)7@9ZnFUNJ|l7Yo=HsVU<@sq+94&O6>9deA|&)R_h8jc|R$H_-Q){Ga{6sAX2 zwa$}xb(ViwB?rj%6W^=owRZw^yhF#rhmJ&SRJLJHpIamUeO`aPbhcAvYqYqqxu>At zLfMMTg6bM2DP*}P6@iawZ^0;68cQF_?qrd{{8`@!e95LCA__UwZy{bfegb62Is4r& zMYO~2v`F>_pf0iV;f*LKdy-X{x;Jk_4`JQgTtMteCsw6jH2V80k!*Y)M)HBkm=nmp zQ&tWSHve*q=cQM?{*-y2+_-=bOS$+e=vEuU4W|%cf|PT zCyW4tn<&63%cIhf+q2K`SKM5msDE}?gqe9pt6byXN2}a*79d14-R%|;RN8<2_48GH zIN_5jq)t8A$n78?)rDigbR91dl8NJEN~ZDb%$`w(mEs=A0l0avXS9vv15Q5a0(m+v zO4;5veCGV4tATe==9bPs!XQDIBf&Q(v+XblrLIpKB@t5d4^sCgTa%2J=pfm(P3myo zir8ysqgT}5WPOyyu=4uOZVVKW!qrPgcbKxev9*>2a>&F@@Tty8V2wXJR1hKVVlpPN zezPh|DW@11`p6o0!uwi~^zwPl_&Pl5lSrMNZwZ$*15?{s^Q6e!)8@RPD)$wW)0|zr;L^874_}WwVD%Srs7Bg^Uy`s zx_vG_fm9;ez7`)P3q!(lVoX%{ZMUG73Jsu1Tth$q!W?79CR;`)0IjwyJ3 z>dukYEOjw*^r)B{>#xoF5A)?9P$gx>mRJd4pu!x`$XF-Ur`CZX8EzDqJ{>P0HUt+3 zAn3CL^i^mk_*CWOImA`m6^nGi7r=M+9B8UPW`3T>AE0B~8s!};4GS?yEMkU{idHE6 zR?}IUc?`AUJ8GS7?J_c@rs_#XoT5q1)pY!mJnn1)1L%b8vo=4$0(ZGZ&|TT0-!BQk zH*E|M1Y|D32$I5%X@vB1NMYIH9}Mm8DW(DP*?0I(lo5 zi2)1$-3{0?tZvnV|1L@9a=4#MHa*N6sj}CNc$J!wYSiB@mM>DRzNLe$IOm_^|KMm8 zmGsJsK@w?lfkp#2Nq5D#<-h~ANARUN=e9;gjxUXRwE>LB_v^_qYX3zAOiTb~@VBdM zk)ONT^rdT#R#R0&w}+OUy4#R&Ja9SW!R#}~5U>wKy38M=T-fI+?Fwo7W+Qz%=^9B^ zm`ua$luOK%ON<$8tX6qc{!H4Duug_GQJEw_i+`1VO7AINq|vs`u3jCwMW zpiD&YwfPjHzN@@=t$JzL-9zhj>+|s!xzudTB}{!newn)gKhdGV44l*K`aJj0)hh+H z%9=9@weMrpD=j=bp#vsG)UDF@IJMUxI2ir9S)jk<#ArS`Xlz%g=ZgK&q0XxaaEvgMUF(AEZYY5u4Cr&qSjkK7A4QXrdfKxCVdfET+527(-& zSPt)}$&^7;@el327%UJ@JhEdo^=d#49M{_?9&f}7qSl|4jzMa;DLugC$kS>^xFBLp zWOcsQ@rtZx1GI~crWCt!s23&CVKKRs z?uEO{5Ou}2PoR#2T`DK>?ZQ=aI`N!e=EHV&<%qmjeK_R0^jHaE54&lc{mypY92@d~9X zVt;~54s;0xKM_(-x@ybn6FYUIB1`Gsw3e%a&+#_<^7(hVRiCaZGO*L7j<9^(fZT$9 zTANdiVd+D3V)f#1W21$7>3=o1z5xZbQ_z2z>;D71()zO3Etl{p8!`S8;A1EM=gV6a zH6@{CheIrf1I0}lzo|kv-06*db$h8MQe#zVCq5GtiF@KyI9vDiVsSMt&)@anDY{bp zo&Li4vC+TN^AKq7$%dnS3cTQ$LW|J0!bnQ8bz}bBIK|}?woLhCp}Nj ztTP`XXGu9yXjv2XyoH9vUOI^0?ND?*=qtBb^2HvG(?MIU2RGgte1Kl5-228;ASqj@ zoRKMY3;l*#W!5@EvM>X* zu=tT3_1ym!g|8NNAwd1A<)34#pDp(uJ`IQYzZXV}!hGR|Qa8Yv9;C%Dc=XhHIja^F z1oo>APG#9-qJw9x9nAFe-JuZ_S=Mg|2(eN?lR>9}5FaCt5~6|(E29daij{`sj7yxo zkoGI^hf_pmu_@a8!=hcMo}8r-PjS9FoDIm4eeY4>As>g#*y+{CT|o;!>4Bj!(YK0y zJ4@_m{0ONc`%;+oP*3U()W>bAoSQdj1D|Y(*fOJm?$qpzY8Crh)NfOv8p2G-bXMz2 zJXGAM;+RS#>jW9vD^dp$2Bf%)qfx{Lt{2LvB+MXjduRMc)`jJN^caq6MF5$#m6fXa zbMLw;V<|-TRnP*M1CA%xTqlP{gT_i-(j2m;Kg4>Uq)D_ze6)qROEz5V61Xo?4YilK~uqpK?{uE0s{IY@ZMg zT0mhkziJ)gvQ8o7tmhvcg6=00X8ov1tpz$}gc7KsDn8+r28+=5m0On&r%`<~k^dgc zpZ&NbJfvrZLT~Pjp!ufD9U-Zv%)XeGGlp^J9FHs)PA+-BZOL`@(Q1(^${CRUHm}0f z+LFUj{;`LRr9LiUkiQht79dP(7Tg+ne-onrNWL??Wf$;D=jkWeKCF1u4Wmsq(1zG> z<uM3$;u%Xj8!nTcv-|25)(W z4oeM!L&rCVTQ9v(XAODY8(B8kTCeq+GFAB~){NKMBX2O}sp6E2m6M%H%g8^fW>f+J ze*jU877huQ4OQ}}*gGefV~p2af);8LE7ei4vPQPW&@r=3j+sa36g8xm)^$UUmG$n( zX^Nb@Gh-pyT7F-1J5yI}|F!Ys@Bz4RCWo_F>IC)feq(N^YW)I!LvB(h6m=J>PW2dO zC|S2`5|6N7gcv|*0G%MkD`E`ZdtK!Zk znopmk&wLf@azzO3v1vOIzuN|j5YFSMBYV5G0>MQre>qvjWqE66`P~sOU+rxZ{uh* z?nstE4XdiN%k0T~@#zft5WKD^>#{Sw+6&(>lG=^Qaqa{h3d%djOSoyp)s;|45NY z`JBg{r9h=@p}!P>#_(GcR*Le{6+H?u00V^4v+D#D<7}?mn2s&pC2pqA(v+G$Zf-re z^pAanv9j&-;Gx)Z?{5`T$j=r>e1@Df>}$LQfL%a`b-W>@x@LW4Qc=C@8+zKrJ2_pZ zbv$Y7Y9=OZJtJ1Ji{=EZiec-RC}b z#fa>E@3p!=^U{MvV>L$1$7>{nEWj`k>|LVtCYW~V=4{cg?nQqw8E?v z3SRQ=kI)O$tC}^!Vy?@g5F3cjdxXR0Fn2;i`_d3%e-%e?24`zHN$bu@es;o0A`Nr) z+4Sl0j1@y)?6Zt1b|OCVa;fG=r|IBam|THY2XHmI&KZAWu0cCvr~nw=JoSCy9qu3I z2n5VCARzJ2CCuNzTwmdtNi?b|YsQPCz)+P73mH8)Q z4=IZaoz%E&j&~hq?mFm*tn2b&o8Wfj!PbOiC3E1&_dzf)@gMq4U<$g~4t{70ep!0Z zo4rWKH7_5sO=03pt{k;gs7vFgM=89#5@%@)r|hznF~UajnI#qk&^f?*-lb`u4g22S zS8CtBYyK!L*st$tkD5IZejM#Tw4nF%LK|f24O|jP3 z1z#^=5x%3x&cLqY`!(M87Dk#bRG+F1DL0Eg=d)5;BmeO$>m?s?K|CLOX=82O;n>W* z%_9rIDd{+fFZNw=hj2W>O?y3tq7&$p&g!ZWdYygJsiwqi`OEbT+Ezo~defVyzfkP| zzb)R#KK$J9At3ab6x)F0BQKJL4FXtsjqpwluDOm|8KzpGIJemQYht}Jj z4(yLYdLT^nOasJUO6f|e(BIDg_#4$6ElItSW zs9tlKtI3uUD2=2aU8BLl>0L&jDOcoB>$<2=O*#nKAP?liH%qKL)%B(6a;DK2AtJ;Z z-llaI4UL|DxQgjSW3J1^zB(*p?e4{e1hYGW;pkVZc5@%jTk8;i*uU@tdFWvKHS}YP zYs|Vs(Hk_C#1;w6gLGC>YmdZqKDH;=Qdv=)p%MRX@lg8!!|RB#{U}2bvAy=PBW!y6 z#@yQCr0d@U|MtKq&G;zuRaEcl+D}9RXlW z`eatfxbKfZlR&1N`N!*Wtk)-*wodI+=0{$>nTT;r-fyn1&l=q?7BaT6;<=q;5oZ0T z&=je!no^jFuG4YxN0nH>y~p~{A7@H4svx zq^H`bm@?(^^8m>9p+r{p0?Y`WQQG?cY3iI_Cp6J9AfAvI)x{Q3kjwf0&05uX)>y-N2+IFW< z@>5INNs*KV8___eONQ3(r;Zt+TtAqrDL5T9X}*97l1iE>RYHeR9AO-?Ja>EC*?{w5 zty#iBYo}L69S_%A&n&X!Og0wf=SG#ATnV3*(5}Z!SVsN+`c^q{&$>agVviLCpvBo4 z`?H{Pg}$pn!98wHQ$`!DB_RN9EG$@>iq$O(9!sAe-9F1Rj7uxXu?(`{H%K51&C2I~ zj64AY0@T|yEf~gC9E4|#n81M~OYqr7ySj_&H{Iq!3MwAq5}%X>jtuLJ7-+c2>tUT8 z&Ct3_DVl;NekpvK;jl~G-(Sx*k=$I>}AxQ(NB ziO_6IdLioe8?T-jm z%cb6ry~sop+E!#jSqUP*a@dF;4(p(-JgJYk0i0}ag5mQI=c6+QLgzPM7}q-chTOnD ziP-b-AGB6d7SP_0Ji3DT+Zbib-9B!i`E%_&#JNC$-4kX|}#>^w+L+78jKm$&>W z8h&!~j{RpsM}%J}>0ZfU%E6+A^OX!vdr@2<+ux&fG|4|*MM zB%zT`ayB_(r||Zmc>C89P;YRuk>9m3HEIVbYz;1p};wZthH%}*jrGU1Et(+tZ)K@S>o>_GS9Hea|1BihHGW(%L32^Q&-xa8}j@y4yu@g8aqkM~b6pd1G=+S+n z45nMco3Cy#!%V`yQtu5{>>*<7X^?@KHT-co(OX|Yg<4+GEsq5cm7B@j8w(?%GlD+8PWYp6-Qd0e z(jZgbR7z4(@Je)sIQv~Bx23W*qu!ayU1@uz!uxzihld=Cw2WUCsXkVc52$gIGU(n6 zrBr`7Zopa?Hngi%wj$c-3;Km8bWeqiCAg3rQg{sgaiCc<@2Ev{`K841usuIpV2tf2 zKK<={C;wL6m@)l}--wwl7bNMN1(mL9)!6UPv8-7U)~r7m!)Hpi^p(B@+`W){+>UHt z2M(b`1{;61u2(p{@NS~xMLqkZhFXX*(B0IQQli^nPnb62OE&od`eKICw13^7#}M+ zo=s&PE@zFOs(+Gr`)krS|EpKSDZTjYhwqZc6SKP;AUQvL4&Z}kk6Zi4kklPSb52x@ zFapBT#V<^`9Lp{5(c(vJA4IA+)$k**)>byLD+;%$WZb+TvqQvqdP;*2!S>W^RZr-c z=7W*E0~b;p7s4Ah5!-8;d45GyPcSH&S7j*d79Ck-l+ssS1!#8VimM?-!bEnye{1JKTmps&#!HEZ zr}J}m%ocKl1GC31#)RxSO&6pHEgV&ug{teOf`-y=X zBhoN-TOc7RJXr?ln^p}+%vF~Kf6u5b^z$KaVjoAy{lfOKquVv)6=IXs-@4R(gR$%= z%iSW6738blsRyXNRCwdT2NULkE?>Q5yeFCbY}xqsibhR^Tu$2JiJFVx4{yN@K1>Yb z_)`#-%n}{|w+x@u(4r)4Ki_$3Wi3A~V}I_QlA{Vzaj)-OPVOAFqfmjMU3Ws=!9}Bw zqi8KRJ2huinRJRSc)1jM@RZe5P+>z{o%9p)Ka@ojH?4YlT;E zkwA_+!EV%t5aY9=BmLge;u}62ZYbmm*Y@=DD5NRzD0t_q!$uVy2QOEA3?v{@PDTa2 zE-rXtR=-I(y?*pI4AS@%Kq(|TW@URHA98;LIXZUBVnf0 zAAcYog}v5F3+gsji3F$n>Iz&Cj{H&DoVEHA0_@f1YMjOs5Uzxkl93Ol#amjQNqwab zwD?_iSHY-JI(O-e2v1%{9nR}sf$PMD0wjOWHOrhkBmps%nH=D*W6?fzlfwMo4<^Fd zN``oQr9o#tiZEu-y1+a1$Tf=GPJIB)M`xYpU9juh=Meo?hW|LjCt$%mJ~vi6c*!g! zk}eTgT1b%tW_WMBxh!Y>VFDj%n&|pNDc!ySbl*iV@e;8F3M;}tF~lt5Cr`we9c~3d zSY+T%ozcmT0<7`w<#z<;$g{vedYTA6Kcy$fPk*y%y%N|haRDYG?burfq^hBl{lSOS zG=7aTG{$dVzEu1>f7;3GJ{bQlc0^zUiB~A6I(`t$?^y7?;c*E|eWZ{ZnzuLiSn~l8 z`DTw04ycz^xrhhA|^P$%aGom*310I}BZgtgbt zHF1#!hxCX{xj=0SE0JDXoOe|6-QI|!@=;ggFgMiWjp(d*pj?BgwK$w3B_U*)vDU8+ zGs){59c8a<4bCz`SC910FoN^}%$khR4P{*XtlCj8zXxtqSB*vDKZ(*Km9m!zj$M)Y z<%ip$k8r`3se}#YnOBxvg@>1K1|N-^szOky&K>d11O+QE(v699lFNaPi;gtP$PekE zOPeWj&++q2DPX_UkcvEzj+h_Q6OE)Za~nj2+^_Tu?~=v>z9s zm97Gu%RS2b^|kz;&?Hb!RoPk!rZeiz3uHp&JB2NKr?suV+6ME%igmpipNHL9sjKH- z6g@xdfBC7Mcz0*sd*I0WC){oOJFjZ4^){4KJx7aM#eF)|dXa9k zEQZFpG@s=|KJ99HJ_Y6A{KAli{5McIArjl#&|3f1<8cLL11&7I3e{1*XS(A0>P)vqZ6D^_KHja=t6VxRo|F=brHs;n@ruU1+3LajBHME zBa}NW%Mw~A-9+HMO1lJ)pzs6l$?bk!zcpzaO1%%S7hU2kjjlVtP1;%;m8fT%ZT7Ru z9$1p+&reKpx9AzP>XlA%sd|w!c9<6sPXQ_Tyuw>cAGg_bAC~yi0FBXZhsJZ1rumeW z(@*IwwSf3?7W+oo zbcMYlDC##4*j~#m5r$PzBm@-co73Q|T!J~q1GxRi&buHo(OUnm!~C2bQS(?KW?iAR ztVal=okjYrjne6*`5nH)8rk}4CWy{d=&?XVfZm!e1PoVzk+J-iq`zR#sJ*YVpu2O% z8EoXfgc??BB;O709MF&E^}k}OMLqqWjue?Zo0l3~@_&gH5`sT{Enn0rAxiP>;7O$H zU=A@{=QrDS^c{Pki{{0zWK3?vj?JV^=ghf?Yo*q1>B<(bDHol{>xG0V7JObep*48k zS{fDC8{^#vSf;UnSN8mghHN7UbGtl+%E(+~f305x+_pr(d~v`$yp1?i`#JT&$-Oeijb zq>Z{V!~&DoxU>eRXf#r%dXLa0xCUP4K{Y=D?EvZ^*)B9w7o%ABW*6dH-%^9d3FOb6 zfpC3WmgMJKg(4Z0Mre3Ca}1IuCX&~}1z}B-DgtCUW?K(sGZ<#UKlD+v9{DU}-DK2V znTly%6WH`RTMQhI4PB^XZz>jQu2fgd^^NQ^f}c$hV7=20czd3F4WH)8dii66EqYfM zo_Z?`5&k@21&b~Ap_L~-OI;^h{*jr}QZO7L)^aW^o9SvP zlpMftZkc~Sjgkt#8+7v?hf;}9-tykW?M$mX%jR+EP(uj4&gW9^>Ab}>9?d^5g7R*) zs3>gvWDzbvA(&g7%Pu&tNmD7L1>|w!P8VWDS=y6a5GTM-QxP`6Z}o1~hUJtfqx5Xm zP}Ba;oDJp5@c$*B8{ z-Ni#!*8kfDx1BOmp=co!3XhNU3#2c+)jt?6Qy;*J=QXDb&8{BuV8dtELcT{!gb@b0 zZ!dbY4PC#boTr z5Dgk!%~qG+_BKi4wvk4x*AM?2r2WI&ZjW=N3L5dyHO?jC{Q`wDC}&~ed5a;}H3*mmDMs9NgMaHNXC6}C-q2q%aT34_0V z;$;DI)uh%IY%19<(W^@6$l*;emo?pDH}Cj73H+&z#bEiwqyhLpU*{@dg_1|b_A(Gq&FwyeYjERRopnFDa};h7*t* zqg(S|z+16fAHRPLU8KUn0!F6fhjkUkVd^JfyZ^AAe>`&gL)>#?GhQ4gE$X|onU!oFVh(HXcS2PN*c}2~&aFUJueDY0b(h~}|?)5YJ-#8vS${o;f!o7UtjK021hK|3t z{j7W=)8PYuv4s>ncD0kkig}qD{DaChE8G;{lR5}13DGdE?8yk+@EJgI_UJ5R3b1vM|Zg_@_PYe`FjO4dOCxd#v zV@5Yd;V_)dIebqb&hG9~|9pS48n-Ba!Yll}x$*a+xP_H_`W!Yj<1Rvp$tOb6OG4JN z`%hHNNcQe$ekk^tT^zU3rKM8{rfPT1rYkZ)CdO|peXgaKnKXKa2!O$p6M!?SL@BNpDPo1o?={&O@n^g`q6wC75sCw3pEHfDO9D?DDO4A;u*-p4W&bl9#dm z>HF9OR$97YF&$>5cpfq>X^kgnw%QHWRUyOU7LA$s%tok|EKxlgc0baMp43@QE#ZQ3 zX52=Di6ELQr|UKhdgv+mc`Lz*9^|9UqhSt%XHw2u!<$mpgzE(Z?T|>78hd)RSK)Lc zGJF!i(}e)Z{_sLp$ZG%ACU{T7rv3()=v)`l(`L9{68u^D#61Hp^nKNwe4=SEvsCP@ zC&znYsj4QV9vetJ`;q51CDGpp6P*YbW4a6`Vo`u)tw8#vxji)}lr+WeQ>d-0WFc&? zPmWhxV%2i^r^yz~&b(^X?jbu;YsggTsTCbegN$tsEAZIpsgGN?`ai9ApZJ-t$CA0Z zb7H1$4G8>nA+D}$aENdyUO-o|grMh96x1~Dk#frI7d z)*IQkdJ*A&I2IIF82B1z*OWlhB;1}XEgf=_CHmU%-;k)UafErOi5`kt3Ypx$H{3IL zW5GtYat%3Ldw2d~row|Gp=%8r@4q{3Eo1BUT%b|WyK}NwNX6Bae(DAtR*nadwc+lO z16CbJj{824$qcNbFK&FY8G+VzRx{6^9*uK=n6=fV-E~0|c_XftJ1N=@nFfbXvH}UJ z%6Mk?{zIVF>s7u5&3596DFO&qcx_II*kdzxz$5n~+$a5JN8A#r&ap6nSe-frhv9WO z-%F1EB#>GF6g;d&i~f#4p5+}nc3h#odgXIe!}7~7oWcXLcz4W7#mb(ygQGs0 zN`ng@d2$?QpktuPj68GSWl!>5-)dgp;jKQZO8t~C{V6=KL6El`lyn`S>K^vi8wziX zAuw&`4ogo5!jL<=*|$Kh=eb)9zd2hjcz2q2v=>=DNE80?$Sw%PgRhMM7ym2Dw?%WD z6Z)e$;w_^7Ua`%ZwrnN5k+df!XxV3r^}V2S>Dc1#c>nx_0;TDmd6HVaMcaqys0e#UbZNBa9>SZl1gZ=X4*Ro(3^)`}yw=DL6hLTaS(LaOdpV z6dm}#Jm_Cvgmnqu>|feYq5=DOd5 zP2oR4>Io9x7ujf`XabFhrrd{B6h?hFmqfF2k52I^HuniCl^-vyF|Mu`yqMs2+!jU? zA$UiV8qZk|yShuUlD;TM zun|`XISkXs$a!e0=4>zL*vbr)Y?mG1@ut~;nCie5?`V1fD8*hurGO_wbs$} zHx!#!=g23CgvkAj`s>sWsTgtfPYGdp?u9FF5m>UK1sW7n5(h2o4cwqFoP|BYAATmn zxK1MkOrEM&)7eZ1kJc!o?9qe*N*wIO_#z1$I}tUZMNze^?&esY;<_S^1QE|Pb1a&j zl%9+Dn8NZRds{)l*ZUN06zNGUG!J12l7^jbqnIz?{QChdsfFsO?DUF|9H;~p?cF;q zXkuC}m>THTssrrVP@J_5!mNV{^*ND8eeO0*uOC%KcrbU#)a!Frw=@1Cdue)w)wdS- z(A4;iHW@>rP}}f3p#HO_;kc9b%EZbC4*uW(`@mhcYAbe)_&SEJQ((J~8F_um+PR~D zpq5~{{^RL&7kxH6J%X^V6MUs5D|0bpFiSDueyWf%@${BJ=7>B0b_~ z(}|)OSmt$NBz2DSq@=Aa^QM|T6erWyudPD@6 ztaB)(X4MTV-VwSmUj}RQ3nDc_Dh8rTN0&#g-CJ^@;oZ9s9f!gQd4Gva0N%Dc$I;Gh zQ)kQN=JbO8#)JWP#<*4G$3ZI$zJz3NcBWpm59*s&;$fdAA(B_}2EsY>@QRa2wqtuA zr+PWrsUU#nnlZoEfIDZLT0((}X8<(8l;o8fh9A%∋0nYmK)pX|)WPO|t8+>wnq% z`N&|vMt&%eGl`!VNywV25f?=*O|kqB@k!wh0lQ9G54))Bz)U#m`ZT4FlrQfF-u%0( zvZK6=3(mG4>593?ONk1%s)vsgjM2~jvubCwM#4$d^6Ua*lAiV4_! z?01H&ej8T#y)6<-IvGMgPy7V8mIVns6iD>G$`*e565?2FE6-TFEZdtQtK5844RkGsozsM z%Z!?>W}wVnr_MC58VKcuUV0a(62sBh3JD3(8(c0O^6|9FEhsoePsAGd?92nKa?9Pb z(YfmyzUI$qyB$iQX4h|)!_3psYC63(m%%8^1LnP`C`whL* z2`lx!+g`OJ%#@$k#KW@7?sq6PWaJ=m4nO_ry&2~{tWO~pep_L1!O$=d-`H(+hJ3Si zWv%V4dA+oLlo7O_Q0P9DgvoU@wSGz-92}H*F!iTmbDC~bhW8M&bl!fpCsWSatkGFw+T``BNt%E!W}VsJW!IcH6dLQY z+ixPRy0O!aSifd58+V?SIT}k1$ z*d}Zd>nn4Wigu<$A8II2?wpXd{RrjZ@tcnyjWf``Io@k{ens>?g${8d$TDO~A074rk7w#oR1usyLCnT$@GX5+x#PYJ8JK8M1lPj) zkr-_)n2N8;SuGl}Y{c8MF{iQy5@Q<-_j`l`P-=m#wagw~w?>8{3-og3LHjEvXZsfD3y0?a*?B68@epe;I)B ze`zuc`4YgBv^7eM+65tjAY0=rg#;72MMW|Zn^Pl9TN$ZZ8^4#oXz3qf=0K=N64#&w zvJFL>19t~|W2-6TXXVi;L@2BhTT`5S*M3&N=fHa4ip-h7=}kN2tsftcA1}AK8ZU)! ztZBl_VeSK&9w^_v?S|q;96sCe9-pL8sTqm>x-^2i)d3$fm{j{EB4>v4Z9PY(3b`fS zRpUAaEyAD5hHZw-@EJF+Ds&Z7eME3EKOTOl*j6(f3ITq!ytZs5`Olp8FEk39IFQ$% zD!HCjBzbpXT;Of9F*x?Y&=!E>OHol@Hd{j~bZ>y#4>*@SfY41W{6XhUg~CC$Qy8Zb zewx*p2FWv`Rgr8bsnem{6554r(L^`kN1k=_;fBjGs^*dw z7DsKnnv>S7(MOSO1Y_Sb4ZK2znJzN|)79V)ZD39-!HmvobCdU~jwlAtl%9v}&Se71 ziR&>yJoy|mZw$82SJ6d$xbfjk4NLFoQt7ajo&e#6l7;4SACJxKN2V@g{mz_Eo(_0) z-D&41$-(_IABx$hdnFPN3E+DG3|jud-AJK!W)wm5ah3c%!dul2h$+3wtqgv1<*tnN zZuNA2GQW}3x_sQmHjir5+fI8+!^J0mllCqhE4pe9;yb)g1e3^g5=mHVZqOLo>e`wQ zxR#Tq`sS)dFlb9sz???QUF4L|QRMgT4N3Yk0wx%bwL?w$rKOdGgVB%EbEIF^P`a(Y z6WleEzTAhi@Fh%Wcpk-p)}6HOBVyiCF9}vm?^a&&))mU6uwsInu&CrqJc-3UArRDL zk_fbZvAL29GCh>EmceHv)&B=}cIfF@v)fDvmVB@S+;M3zeDY0F*{x|l+2%UkSvqB! z`uKj&OO1|~0{78w4IYniJ%7s5_QDEEbJ1gVtAw<`sbG*PgRtw?_)2qnUA0}pPfwBD z&SEcyE)Q;__J8?M5tXZw&nSA<4Tkwr7e3(mRS;w^mnY7h)TRiVw#J`L_W71 zRtKY^*dYf=-$w$Ms8u(lKVL1p#TF~1UCS>vH5AFWGdQBk`bn~aI9+V;l{+<%|j5<2IcJh zkWHZC<7UmId<{nMY-()8y-AAEeO91zJcf5KE7)cD7Xr%w)$j5b6{}tUWrpo^c0{Dg z)Ex9i-mASrCW>b1KG9%W^D+tMxh=+ zZSczkG13I!j6-RUDX|iKOJE=(ZE^Xi||+-w_=2R)j)ykBy^(#AfqIJ+I@LFNNZ6D5(KZ zz^E@*c}qHUY2)p~KlT=&taEW@X{Zo6P74T4({3=IMZ2BPagq6*Y4he>m z{d6~Lsir^1mq2##WD$pY?aGU){Q^(N7C}F@?E2>{!1VHHSHZ97=@HqNY=h|V|DC;b zjud^!gqqZ^b*?p=As=IHNORsiXa|BFEd2T5x^8Xtvm#5CptI;R^$dMB+?L#qNvC&x`*)#ewd`$@Au9PNUp?EffY+QtE?96d!kD>8BTyXG=F*lxlYtSbqC z4UuVy{^maS`kcAz)%Vlfen(UhzDx$H?qCRgi4h-VE&T4KtDz0@`V~OxCy;mM=Y2XQ zvO^JQ#W^9c{e{tgnAu`K`oJ<1+h1nr`Yi7*UtA+4J|@fJlMT#WPt9LL$o@1@rO>|Q zn&=Fl?c)Bw0mq8($8MaXyP}deKudM}7r)k&_eX;dy&kZXgoNSq_rc(yl)(MxUJG78 z^WaMkL#1*+M05IT@jgg!w0E>G3oj|L#L#S84z_M0!04ck=w7GL`PEO9+I;o$tG=%R zG~QYZwJ$htWGw;$`Igl~so=%dfiKO|kg%MZ8fgD9aEGE|(Ypo>z&PXOS~(9)76L+^ z>XkgPLYbBg$5DOJ*6CHNK3A*=&LeVxOcw*IO_i4#vt+-S+7lm*o}t^i%czm}`{T+v{+v^$ z6(S>4`4Pge)>+uTeaVZGzp>I^MHOd>18@4o+MLeeS1pSL=k4lN-`ozdfc|)L8^Rsc zV$<`u#ozj3AF zNNnaOSlU*{-jRkq$LS|aJAqp4fx&3t1gNTe?SVG&6up|XeA$;+TQd#`uV8}5mW*e3 zjhihhzAJiO*_mGt+0*YTZiUp49NB1Hvt3|u;~Y0}E4OD)73}BRcTnX0ZIk|r z6Mr~s9ph5-y8grmqP*_dF>;$4M}d=luEH3^rbK8H^G`okhjqnxssOP1|D3dPz-jA? zr>=C!{5-PUBOkj{RzXJcXLOBKZHtxDAp?K@2Cmb2-9g7;)KdOuoOiKyv$Djq^q7)E z5NIGs@e1ZucEAdnSO;_w!%5y5$+dNU;kk};HR{j;xyhT4Yt%1era=a%;)?uZnxa`0 zR|Go%hMu080hR4<)dtMe6B4?9N}2?~`w7?b4~qWNfZ|~j)GsZIU+whIuj#U5<#OKW z6~d4VH?^(WTO(hsJjI!j2nHcOL6%f(WX+PqwE_*vrthReeWdbrf(XjfoKSh5c6DS( z<^27|@Dsq{q6$ol$W?A0pJ|4^7cGWS0sAf0FIl`*< z3dVEqUcCiGAuJJooIo@Q21>F&^*5fb6N@0h*By@Aex(&AVP2`QuqA;f+IBQ2S7vs1=%?5{Z-n#Y z*eugvxy;A!oUMo?XSX3MR=u~oXVLMFDLz4ZRduFIb@*!993NbnFN)P6J~5u`CZ2qi zRkcbMQ@(GzGcWnnE=%^P0tv9vvE${m%CrwZ@u%@B;6r|IW)A-bulLwGTrnp|OQ7^B z>aCQB5}FU?#Scfinps=pIoADph3>#d$<3QQ5Qe))l2?UQvZ8#P3i1w%^hTeB`)^d? z&{1ST6SOa3^=<)qj}tA{04QjwZWt%^3nFeGCR}_8ye4$9no5#7D{Swm%x3ev{-I#2 zC4q0e!Zgy4#caW=%?;BasPCEjp2scqCt}^4`#_V*yP*Ah6yKP!(E-ms+(vN7f=XXC z;co7a-nR6Z?|x-NL$?fEy%AM7G_SofRiu9|KcjJhx1HzKBo|cq>4*a)ZBRmqnO@59 zskZa+3~Z2t6$LQbRj2K&KK#G*9-@ za1a5qxwodM^l@yzS@5#sYBWl(;h3a=k)cSupxL{z;_qYdk-B4YXC36Z#EV=Wdo6D0 zi_`FiMTU~EKT(83qrRN7Ez z2r`K=^EJ#{=$ADdoB_%4%xgc{-xw$2$R~cI=*PMNibnJ~2H{733r6-8ycxJ5-a?~c zOQP-jDTVpmB8e~;b6e>2J6nxvAL|?FhF0KyTjd%qEB5$>ua9)IQwcL}5 ze-H=IgFJ0m<+ms-Mw!RQ&KgZ95AS6B%=nwlOVb27(%FQ}?k_<-)(U`})BWW`z+B0& zEw^8yB)KW87=DN3hW=D8pI`oCjjX&LFtD#eZ%ZTdBup2yoHi+BqqueE=;jFFC}zb_ z4liexTl@>PRk!v0rFv`4-=H>$*InZ{Mxev>V!X!(_3Q-sa>5w^XRhVAw<0R$qV8LA zCoWQxk0=@dGe-t?NIvHP z18{!~2790b3H;z{Dc%BbrRH+x7_>TLbsc(h=8w^X z(wy!jPiUK!4VvIS+pB$-DX_#**MD%)tBlLkH(7y*xh|4d!-VcmEh$HC1}YT+*aoJx z+RV}?_FsFT<(L8Ra*l3{)|M?QREIM%8hr1q#>2g&NJ}qd1wFFCjeIu^`@+xAK`Jcc zq!+6Wt=PaZnW|o{u$g2*!zk1-yt&)lwfqzZPg-7Qc0Mnw4dt&BG=+;G_7nnrgOc$C zkfp8}M(vm1+yne3W@ThR{5O8166X8RQq4aIUz<%bM^3NRFp!o;99qc0IF{N_ViVS%-HKNIw-%S_v1et@um zI}bs9rW<&%Dgy6q%GxE}`=rhHprK} zWP?;)PyJsQlPUL?D!ux5j+Dh!D`+I=RT1pAo5cJM%}9_1Qj)l6iYx4tSTbmrDSDmE z@wZbnuv=KkKMB;L>gJfx9+5onj=VIHP#Hj5w7HbFG}*Po+TNEC1ots54$l?IQ~#du z_7()q+~RD~VLUmxm?XyzUXAiMMr_02AU-`>33&2gAbHcVK7aE>&qXkx)$btYia-^Q ztBNEL^bc|4uKsx2S;6OSF?Mn%!P{Es&c;hKyYlR)*L$FJYUSq=uzn#5kld%IFBK`o zRGV)P6y+vjIEfk9CJ^m)#pe`K@k#Nv*g3qN?d}{EFt{~okf4out?>@JJGo_G z0sV=<3J{M2ZAtekY)R2lvpQjljR&0C#-N-tgWnAIfru8()ZBq7`n2zKy`6Ep2&32S zI1hG?wJ^W+rG;RE&LasLV3L&H8nJmeR)fG@RT|VcPW{^fx4egT3#IwzS;Z^%uy?Lf z=relf@FQk|6TJKSFlQ^~qo$ijTS4-q+6+J*<2|Vdo<3`_JKuY5btu|Pe=Vl603MO) zojzS4PbCspO+O+KX&^?`tah5&uUaWB5ny#L!=ANr|5H3^T?+&Lqj>^@7E+Zbj_rbo zOH%Her%Aqgm7kq(Et_LqmvIi4*Kyz+u4H(`&Vg|2Wzv=kQJ+RN4JVXv0g~pXeFe~h zZWVGnz6*L9B_8KmKGzofZ1z>%X4h#MpnV_SS{TF#Xs*PoV;DEa6plr^n)RP7yoD7j ztQ`2lD!ZRT2#A|qW2$yTfkyNpxOEs)CV1fsA^q!+8r%ZZVLK4xJ!gXi*l1(&S*dKw zD8BVLe0Oz;4Ri+5oaaY!ZE-GqLy$Xc=&71e zO8>jhQpuDtbNF4*!;3)umrMFqKs=9;C0N~oM!51|a?V?@ZS4IC7xQenCRgnY#M3A~ z8hyYt#G;KCvcCqrxv|zZ5tui)r)BjN%KkA<8qlYDiS`WX8*U9cl?9tn_%j>WU;c9h zCPdPje3YnTSX(D(&^)~`zxH0Wt7ewtSYLOsOpr4oaf)becn?&tfJ*}ANZDDL-N#0# zlYl&{&rMN&9fy6~`u2Jd{}Hn>SAfr*JeNcMO~rSd z`<6t4w!Z%8H?d^2gKHVn!kUJMc`2-jU3*oM0ujFedU!$HhO!*w6#{(~`B&!|eV@_| zWF649$XbIuBDD-mMUyjpzWSB~J5=mf!ghMwFDxzERJU3=Prnat0kXL3K{%$40$EDPjvKC|E(rZedJ?YJ&V;E zj}-zLv3dX9B^zA;ewf}8Tw(k_6ew)J@V7sWc&>P+)|-C!<;hWZPY{z7Pz1;iQ`BGl z0%ZvcN7dY#6RX_U` zmBn=R0B%Wb`9m)#boGx07I23MG#S$fm9~I^Y?lpjxwHms_^XmdD3J?h-$WR4?Kdif zrGr{-v^8jV(+DX&0f}t{x7r{x;MouKp8xohtyKO}EgxzZ|4oIkZ0Ue{Xermz-`~%~ zzTTNJ0?d>?W}byr6cZJV4a;-gKO&Gv42;E#C_7+E>Mbx#YQaieuL~?Rk^(6%IE5k~ z(nGugwUoVqEG$_e(omHm?M8O&r54RwwrG$Fc14j3qQBuhe+s*vHtMc=pI1 z$8e#aFcudVkC|{sXL;DeLk_taiaBqmLq*eOw)JO)U03KsXyDeT9XDqDpxcL20M+VE zY{F~LB-D0z`Or$dgNFT7L!_r~@enqTCsaHIK~A?q9|N^b^JmxoB-*QS79iHp@k4Oy zbD$7W5nzgBIoSUlwZ(?3CAs80j6B~{TPSE$o_2CL#=91~K zt_s>UQ<*(MLIVMV4FSIxO=e%8#UHqF|MmsCo0L-cf5dp>PC#Up`u?%>&VM-GHZFv9 z9RMa=4z~-#yqHd?2bR?ZB`H)5SX9YJd$?s}{J67)FDOP(xtZDm(nk~P501J%Muf+E zyZ0x8JKP?ptCt-pOrjB((_cA4iz%*Ch{6fB*sY4;RQ|^_k3XFLM6>YkLM9{-u$=7* zUM8ye>~QxPe4R6PZ3cD|6w9k-c+j;V9r*}Hc@C-gYP-RHNbFZPhaoZ}RiSmF`dcFs zZ_P9I_il)x8u*QZr>7rS=CYsO?EQBS{C|Yc+y$V(cD<5XZ|`}dOc@1e$2_#Kyb>Bx zr3%JUcnej#0a6tS%jrec17l)DBlKCt^Ot>niFJPB?U6fRQj-)flIA-%bZtFcj*y?A zSst<0LOWNmy0Qco+`Cz7Hky}p;1wQs$)CEp>}dy6YonN{A-GeRe|5#`t@KctjZps$ z!3%UM#edORyd6Z1!Wx=06#=<(I4084c)o1m-#>=dpW73iRaCz#{}W2L8@XDA6DYRk zaHSdxaiXAM`M^{&H-K9uYtXCGO*mei=ydFtY`8?-T6rC^9Z~T|(Es7<89%E-8Zvy<$V5JD$ZG8#@cDdUc_&qziIbwb72jtFHYof&@btMB)l ztKaYUc=+QU(dS;D*ZVb|ulb%Q=X#$J#h*mG&VNfreRgybJZk^k(l~|WJlZ=yIQoOC zL|LcMM6brs&MBqrx+kurq=XD@CRx|64kSS{ij_JLOt@zsI=X6<+0xdF7Wj{wtp`p8 z*0XQTJ2^per1rqoaC@pQWD>TmLyG9dDhFgdPI7L3w|^Ya0%E8;#7NYu1XPGI889P+ z&)(d(9V|*jP`GQDKzZ{aV3H>ry7Szi2Xa=3*Hj7&!%IOM!j<(rRqRN!lSyS+x2^MO zK9(fbKq4q_+w0WkbjY#Q-;4?N2VLmCs9-C3DMWaz>}OPYb2A>^V|jR85SuQ15IX&R z_;s4y7MQx>{y?i$y``3}d^}=wlCXHjZ_M1lDbIT}HHguKhYt6<0B`A}Q<$~c=lla0 z3iDMZxsA4~k^B}bsg#cOW*E3nFxBObarua(7(l(o;jtH^T?~x@lmwX@wu9IgnW($Z z{ddmIR-E0QA$$@V93v89;0=l|On8NCOReNIFsKoNw5+}E>Xd?H&t+34@W)4p#;TQr zX@HAM$2La&N|X?cDZ>CanFhjwQlUAdu4d2!Kaf05bU(H4;_kWr(p_pe@6)cyTAQQ? z=r()2Qp>6voxc1!i~yKH`CjF+bi|v~fNHln9-ADTkH6w@Nh#Lm{m+#gR?29CBSQcV4Lb6S^6tnpPZam!^JM8+Ehergjc}e>42}O zaz;7|ySD))R;gf)`wKe@c{~ICIx(oEnmzbVL51?H+*&fM$ep(KKb%YWFQC#DUU3?B3nk;Td&SpDA%|D>7_rAE`WES2)vmr;oDm()ega@=e@Nyr|<>Z#u zT$>B~_+x(K+H_O;F1>~`)cK{V!D_rP*@?(IQZis!lln1-(iw16Yj)LsI!ft29q5fV z0+3MLJy-8%Esjq9r>a!l-;J2uG#Xv2bZc3>zqmUlXtuA7@(jYRX)TUdZ%E^?#Mhte z8Ix&sLt4-<5CoSe1l@@4I_81x!W$dq)6zlE*Z6VqZN{UC&DMxyu2o45L}lYOSI}^H zCPy+?x8eAEe11L)@l#+s3jhOr$tzTMD`IyNi*AF!+zR3xY%I?V-Z^`>s1vKRU@JvO zUve}69$)&MmH$@I?rj)KdXofWCaKTU$ER#%+<}}2*=pxpb@FJgjAp!eG@Bu<=pOLes{H}Y0CW9C@L^9ytY{6F=xw-AUn^|Wx0Oup%| zL!K~vJ5II^feIu=APik4a%CFQyscp&pk=5ah+rPX*Bg;`7zY%tYkCRYMN&oHCG0ZO zMCQs{1v+0cy^lXtn`1y7607wPG}ue{Ob+f#<>PC+j<~-<*&Y57Bz*dlC1yMkDG{DU z;U^h0-L}pm=ydHf4}1JXmh3$`f&Q8FQmws_j;KXOCxb!3$62V+Oa;Y|GklPUqo9?b z3#FEm?K*>gjK8LG^BkyZ#ydOhTt=H$-V_m7kK=rTT z>1q)|pwc7!z9~_YN1rv}9U2s?F*p;gx2EOan-x>bW`ITwcu>9#@Wl4Y7Ji{Cf2QlG zTI27&5u)SZFHg|U;B5dqTMip8ZDiISC{AdbsO>TIQ0kWL_-!NwLZh2+dKV1hz|d4LO~1!@ckq z#aFMb288cOTgvTnUH0LKPh{m>s0qHli{9d3^guM{s{a0;`9UI0z!ubm8=DrCAdJpbZ^vGoA|e zOi#6Hms#uAPA?um4t*4J557G~7!iw%@5F(K@&DXj@Sb=dZ>8L*xr`adgQOl@;_E= z$OmXb`MlQ&I?GKne06sqU;x)v8o1Klw;GkDL0z)>F41)wj5V zC$@I$aO5ZfK_*zkp;`VRMk=-^=0C;%(Vndff+9b4I)Z=M!MBj_+tdAo~=q4ocVLmVX7Ip%8SriZ1Gx{E6P73UFp8M-cazinh z4%5b{biCQbDAmh6PXxVAR5u%Swgnlb6k8koS-JdM!IS*;4Tmj|4QTlzWUi12z7pz= z*G%*b`9D2WI>@o8Qlqy$uc^C)#eorQ%ltec?DRL1lMaE$-@994ED^ts2+t~pTAG); zw16Sl)ZmYRHK`9rt_Tw95PYF~oyU2dlbyaiD%f3L$fBb{kb$0dL(L>sSg0JxL9LRs zV)hYUn$qV=gbH^9ky`cO&BJWA5kGa(`Y!y)!pyzOJ(HAdW&4+I`9~BNOKP^ z?{n-bNFWps8%0W(fdpyu5#;lITXWdt^ejXq!5jui*V*OP<;g(j!3c_^0ro)dH5K$M z81rpx1j$qzy)fLx{OrdCCFAQ)`1Ko(A*;@&CZyWeKx5H+Un@P(>O4WmHS0E7%jA#k zru+UD`~@cemqJAm6wf}ZIY*1ffXXLf=G(K-*n|*=d@$E^d-yfUfX{_Agplo0%PM9; zf+6KX=+^9&*LCpqldXXuvJ1vyb4WtSHFW;rpi@3k@y>X{kaMj+(qh6-JOTU0oj^+LwsT%0iEZnQqUZO#<(!UjK$f;*=Z^+co##aw|~H$1yc}crOieY|2ADNn(6Xe$jWY#caMMBiMfo$rGiQS$VR-Z z<`$T`cfk&P5gmaG)E0E^7{mgpH2iUX6@rcBN-U91hE^Vt1KFvr3Vs!$;5T0df3 zVlr_;4L3hOe*ktvk?zp`zc~d=)IaQTu{kusSz%~5;TsfKv9W*sBXW;~SwHqg zPM(k)Z~Wd|HmD@Xg7D5;8E5$SUM!$!_Vl{r?FK=!+Org^nZu3asUwL!+DF#+bwzr5 zR%e5`@wNUa)OGpZy{804#K~01-d{QDe<{-v7CPd6Q2qnGeEl?2jZFUaLLo-0Yup1l zvecjWe4aPIx8JT%A}&pOa(I^GV4@S2cX|emO?hC0Ux7#Bwju5A;amuh=D{dUaS8>2 z%>#Ptmvm8=uOgzhY3?}aF?f6T5vXMp*ng#T&)QQa{-xuWf++*t>+_V8{8ym^X;#5gM`mQiMz#PwhN>BBEWLTejDdNL~LoFPZO~ENE;o;82}Az zGNI;NQ?YFB47L-^Q8;Wi9ff=WYunH2W0!*mS=U`Unszvsr19#ak2~70(m7ra13=3T z?<%Rd|A_eaL%Npgo~K%0jx>RcCYmscs&ZliF)Ik1UuwbuD=xd~(!MxeqAlhq*l(b5 zfv^Rs=ri@>Tc>6Y-suJJ;YZ|(;X%jIYR~w#Y9!Y$ko4=j>n;Pm2IJs=cl2VEx!Jc@ zYWxtic(#iHJ3Z<6OLTbZSTE?GJ(N#G#hZ&q+^PQ5$n2UBtezeGW4VS zzAzvm**|3HF+7dlLJ3UqH-oo5T-)z>g0y1p3A^Ze4)Iy<`E|SaFU!37=;2Y0(=pZ` zOs>Hi?T*k&)dGrL=IH?a(k7D3X2>=<{weR33dswHw;5}iy-#sZo{6A42@T9nhY283K23cs1Z zke&`*3Z$CMK+VDuwqi&4T6uqUlh*3xa^Z;n>!*6Oe@#-QRa12V zD7>$qQD?KP{`!-6-rG0MF{%u=DIi)B7de+FKA_f!85J* z-*N-kc%Q8CC~SdU3z~x;+(FQxb+bQo52jZ3e?Vi`NB;qD}ft;$e+o z^$=s`t`$S#H9B0>eTsX7oxFZndGJVb@WN6&xeP*7H`+Q;L%JhSmCrl0MMCNW20 z!pEP_GJ?mS`o6E&Juk&Fc|T~Tzx@@NyO-R(SN1!#|5i8+_{+m`iMow2ZR+z|{*h6? zI!OO};if}Cpj>=Af_@+9KS#Vn<{j5J2V`GmMa7=hwzl2Z7vpLXLcPZut7B}bY|ocU zuOK8L(3=_lh4nS@HlQvnnj_Qv=gGf1NS4ZLJO@IG zM}@_S-yM(AOBci(1Rs8!e^J4>pq?(qNIrJ?@CjQEWhkwO0A|9yHAAa*uB4uQdP(?t zhY~2z@m<}^DhqO8SMoDf+TXF*fmIs4-}H!m&3I$aAiuT!>o1UY#7CAt=hF=})Nm=M zEM?BpX_giUFT6wL$#H*?aO>bZt`Mgv(=gk8DoRfY=*c+7;pN-!5CSnj^IO3)?$K7b zUAkluFtm-hMXE!DrVYOaV@190QtV`|c&E}I&PdeV=Ek;p_SoJJ+b*H$jvlPa#^4@S zjh^^xxVd{brjIcb%wLCU2{0NBodp}RW=*Y8^(F736+A{XYdfY)9!4bK^# zeXo5OX4tUoS=vh-rAE9%9b;R%zc`q4gbc=6zYw#&r~aWm^T5H(pFR7xO1qc{)k`b7 z3PSjD#YM%5ANf7cw9{PhXG3LRE?zoM2j&l0CUe&`UUT1V z@$qk~eYZ=wAs!@=A!wQhq2e9JjrYBO26`b>svGO?-lC1;=zohuMmUA-BxdzfACCFc zJ>KIpc*R>mI|zU2vd0#n$~!zBMqA&J$LoT<2j<9x2$;TI)pH!mk0U3duDv@RjG@+0 z@Tx$MP_7GLwzVHDbOFk_p5Ht7-K+^Cu4$FS?lYqR*-5qi_>iRo73IE z>zt*P9`@3`DuFVY%Q7DRZ$QGu7#AZJG*V-s2V{kQxf`WPwz1D)kwhyL<*sx+f?zk3Hp_UTo@ za}g5;o*g#X5Pfh)CDRl(vGTyh3SfvH*p1xTiM=xj?4PKDD|E}z? z_UGGRRMh%g9T8j z^FLUp6)#Rr9^M^OyS&a%CQs;qai16UcC2*I_V`MJRKcvSbed256Axbo6*b&5(S-Dl zfq??}vnv)KzYI6Ux|cjQ{LNNtlmg%sh0{e22%Qj5V8?#Vv8p zcjy?WY7KlpJN^05gyJgm8F66-VZR)4a>M0!`@jk{oXLl%ad|N89V@*+^4|Em8@Qmg zFq>=Z4>jIGwU#Z#Z>1U;z!n2(S?gRD(}1sqW(;)Vbrtvl@1uNwYmo(}veVzIAT2*R zW%i-!_7pT>UdHnJNBjV#Z!nE-`|GRK+5LJYhLY()!W7zj@Aa?^2y!9*-Y1g|?VCUHfEj?O7GIO4A1KlKx|D0yLW za=4(L-*UDc$mV9~nK!fZD`&qw{sSyB%kA`sBcBd0$80nhqOz4n(-GG)&r?G3GBQaW z()+Zt>4Vu;T4VLuX?6Oe02S>up4;qaLLbYX1fFzov3cTzR)kzAwZ`Y{fcxsf;PJasLfe zRcp>6tk&|%=;-L3)91_n>CE~^z*eO|rmxMnsrL_iw3We(-GND+#DfMXOjUlA!V1xD z8O%^R0=4fTKSd-Y`W~lGNY7Y;;?K-12Ktxe`UoH4g>c07zH#-quL2eWSj9G{1gD&Rj1c14_4r`&4v`|wVq~2APim@b~T(v#of+| zofwLJ@Qg}n6mfba85A>2oWWCJaJ$iGf&DRZbf%&13mElInW--KqW>b*As_<`2dZuE zb6FQRaB+POTwuy{JGDo;{0K>8n-N|&rJJARvpE>>xIwdV*V?5L=qwVT?TDC=xSZH0 zDx4dvv>UJ}Sj)g8eiWRPJr0C#Ixj_!&M5W)wBMUw(6jl`KOK!S6gU`(^1Fh!RZ%BN zqmAT>rVM+Z+pkSeY*nis+om|R7o!aEaw7f>Bq^9>h}%tk@Qsc ztAi8bxpv{N64%sS)d8hEwou58sb-7fXbV=|>cbPnYiVx>Q+mZi4CYEaTD9S$Y0Haw zN^-$WBJ`3{=SI_m0hBb@wRh4meD734_pq{`HOU6`yRu-`aR&J4{KDvkzMuQ(Hhop0FJNDQua1<%ZbkqdiaJDX7p3;*X!E_+77E?$&4vtU+61QP z|fyr)Ry| z`Q!0m@J`vOI_pEnI>k$J^F!;70{u|R01%Nm6G;hMM5r1#>ASy8Y@Prd z`0g!=`}SZNjmm*(Adbr9>Jut;vp35Xu$AX&{Dl1R`l{C37zhyxM_qrM+zrh@n%Ga$9(amhgl(dd?68SHDC?Wq}*eiX8B8NI*Ka7fHjr=n*65>fOG!F2CPUr z(oq_eC7ekopMU|92o3W^xDc3Csg7KgH)h7?A4JaJsfq78f+)VZn|Ch$pIcMGOj3xE zvW?IEZCqgZ+)3t|mo>LmRlm{^GJYS@Eyxk<9xC6Z*J3f^P~=MQo610DgeZsR+AI|` z7uh8K4u2ay3!kjEeLlZ%97ICiN^M%(vxJ9uHYS;m+40w+sD(vc=*Kt|wz&O%Ih^kO zyPtyhKP1b26#xhK3V7UwyARg^v`AT7jh*C&kp^qlTzF zyj5;mfkK;?*Dd+MpWD6ueUQf);P|z!9&`P(w$>f3YN3){6A@w5=P5tA&QgjtD#LGp z;w0tpYJgP$s_cR_Y-}2Lkc?L)ul#@}PE^7hHyIKTMZ|)i@xN@=5n&Jnz8CUXb+fGz@+bx*iv>)& zCexz=_B~tWl*sN-Erxb&zsLg3yf9)2rd5hYmKM8@8Q#@JJ|J~v zjd3O_McKcV*{q#~G`wdaUiD|*x3G(5WCH`FW_B^6XYUkgpYql`@#3X&g^hB}Ct6E$ zL{D4+C_TdMobt&O69>HSPUWmCiu4iz2hUciJ{tF@NVr%5ja_Zb_b_45xoN~c@szc1B5;nS8;PWz*r_)bLV2>Hn+wmX`etiez%O`TYbN$&`EtJ|vg@I_YM5jj*~9x<~x(~6NmvYjCM=U=xRhZza7=VsDJ zD7%1>^tnB4M0X58U@6HZdvGQ zrU&`f8^E2(r!o)y+Y7KTn`*-G2knJS7$t5aOZBj1=^IH`W|7uI>Erx}_jE^wgH zd*c)FyE3nIP$)jEbS4tS!nz<9F6Ewr)Bo=%mRq~{zHkhm81)@?8kh+^hG)PoVOBSt5_o)&NrpkOWe zdZPtUJa9VfR{vf`7)*I@e*`vPKgVesE%z$T?~{HOMdujm8fjSTtfMg9R{&2VL-RE1 z19FR|>E)T0vlW4qT;f2AYLC(3)J*T*6Bm#VV14VF4x~R=c0vN!+4Fi{bYB99Q|1D$ zpO4520uA>qcsZF~Jm^{6J(qn`HKvZ1Xap-B-|ze#Bmu3} zqWGt5rm%eCrBa1MPtANUt5_2Jxh}zv8qgFl7VXx{VP2NE3)*KYe?$&>UK@)Oxn@A0 z=L-PEC}OsJWAPvO9b*r~@FbMcjpb*Rc~115q`?lr#pdQ_O1pbx68@k!NO|n#C+8wm z&F*%cr=#i}ZB2m*oz114&fz1SLHgdw)wSL{6`~l*7Q}3Yv~+u^mdrhytp2fQkF&rL z*-h4Zqt^wOr3+Q)&cg%_j$BXS51+o3p^q1EwWdf?AWk)RXYcaz@-eJ0s7OrK?3-{Z z0n_`z11xaQbrXON60GF!dFRx0(gVRhE)W-{5)w9BM*waxa^UqlL%n+6vvdGEIN?-{ z!9NKrE=pm!WIZuFe6a%9CIWg)56D0P*H*PZP+A26D3dEw3Vj-Y5#pi?#L_nQkP;_G zR1F77ezl}fntSrW4(qn^`&+9)vy$-Z4lqD>0-3jv-2RR3M67QC*P{H&M}m$kK-r~% z(|QA(mZ|}X%QJ$(^gN6q_nqUX<9e&Em8p0q+|P#W=@Z0n?xD|!{`dk#O{J)kQ(c;6L6R(1T67X<6IxQ2 zELW*Ucl4_1w6j;(boL$}80JRJ1Vgp&v7MzdH1J*GkarSFE zfc|z6P6>f9Gb9AB>!(fs`f%|2Ne_?U@-vv{yssl;eSk`VcUxZlSwa8p&Xf*YGb zN*g@U-+_l9%SI$M^lcKccwt85y7KzI@fL#Cy@@@kx&iSt+OGpcYhY~Z%mmg*CxHeS z9Qw13k`o4qvc0{1Zgs$Z0jh|#*Jd95F9SGWHK1KijvH2OJ9=-BCz^}IWEu1sJF;tyW~#wR3^dIX`%Ex=-& z4&v2)p^aU&R+nu8ZUG_&Wv`oS2M@6!!|zX68!<+$Q~B$^w#@FX-!Ul128IqD!`*96 zux+gDsQ>zYaX^6gBVj4ySsnvcWP*6ON5{D)UOu0!#F(=%cC_r?E`Vru9n`p>o$eyX z!#_&^H5PB9m^&?}pnGy_t{*M$lRxNRaQY@$!Fj)SfZ-)IyET();mLL4K%O@dtQh;G%Nx2gA!Ox$B#nFn5AB&0_MYvu&35}LrPYApu-{Zpr zGTG9@vhj7t7-)i$M#eq`Au))xKhX*PjsImHT!@i$qvIK%YO7{LmIC@kPTbYf=2dj) zJtf##O%sXeXVsP}vkw#>g=hM|L{t4qLSz%?8U?X>__Do9%-7h(iG0S16Sm8#(L{x5BjGI2z9>qQMtD13n;*G?ia*I-=8^a*|MXI@?b{2L7xyIG8X!qpi zSz?_z7I$r({1F8)EfxJt_RkJ?hRRNC#Yppla2|GkU2B~zC8!m zvaHQ!LAn$3QYQ=e(?68Wo*R6v+>PRkGZM7lF(gw|&ftwDqC5(B@%pz-O;48=s&I?< z)mU0uQcP@WvPhuJqM$=3xT&-de0gW_$=|PD#s0cb?}sAUsetWVRM-=L4ON|bBaXd^ z&SvGurJ!YAUit0C1@b23lI+Z`w$AYC#nS6Fq1irjZpzWl9X4PsO7uugZ-Xr6XXm4x zv0P%EBg>^DMbL8s&ON?~Mn3d@v>Ir1wNc~m15r@Nq&gYxU;4WC(fCwwP4AjH6D|{( z4%TN3{#ZZq7F<`&8HYAv!k60(ez1nOrM`T=xZ)KMad>*}h3$d}q}~CL}ubbb)l~Z}Sc8%uoj{g)|-Aclck?t$V}z zHVmkj(dqS%tMy<&YCnpl!xb!`M86*;irnaE4?lcUQ^kfraWb6@%{<|^CK?b?WEeJ~ z6a3_`Dzr_bELv$DQUxQ0U<_qc1cH^KtHyoa=@WN>8;4DGEryC-`@=O_NM}L^ku+H`Ov>vB(M*? z(#gp70Pg7>b^`%0Gb=Hczk?D4d&7_+FHn}%>3SPq>K(W1I|5Y_!> zcojG?xb%)}a2xlJO#OlVLEkICw8${aQ^6{hx4e>*JDsI>>Qn<+axw+fPDZA%qNIx^ zc4EUE-R*KI2F8@^!`0tCmO8uAcIs5!f;FU-bPCw-bbOcc(%~Tfk;Q(-{}O5G)p8?=CICFFb{YA+Ppp`jqsAY*^IOM z3$u5W@N7UWgw<72iVQEp$(s;gIe~p>398Xs7te~mjzZU5y@cFysIuys?rH1=aTweanV0*wtQ!lmT z)QPivTPB$0L^a#ckMie}2KR?o{**wS#sy#@HdNG#_H>yMU`-d8lC&3R*Q`x*+-CUh zRF3pot?=)54}C&vIu&9R0x$m|)IIlwIv!ov7wvwd*;O}v*G9vvE*MOhOT1mz%1B!L z28!`+r3|So@M#y!A8j82he4RQSpJ&fx1}{_?jXy2$XqX$52A3XMtJfuMAWv0-7SO0 zP2fzB0DE-u+2AVU{B#%$KDP6_pk>|rvlSX)Cq`8)IqOJ8ULY?cB{$CPY_hjA!mGF# zDLKSz<#Gy4UlGCX&ibWLNizXEVqrzhE{0peM~sg;%}_OZvb{MKL@`C67@YO&`SX*a zoB?#Fj$s0pmbH-W_dj~@GU$XD71dA*5s^{Aws<{GNUhW5t4U;?d}^x?$z{_4dh6D2 zlufa2e=vdOGf6B#bJk5%>Q^piVl(}9vuDOl!ZMcp1o)Djn6}Ig2aYGoGog`wplL<) zme3L2(8Pc>_63J1#tTSm*dDA649cNT{Zi<@+1|YczPW{;?{}I24yefxODP)(7Bf6m`4F_|`DsIQ4*U8cX({4d~DI z&L4cXVf(nY;%v};zBd2JyBpZkwI%Xttb?@ZOI0AT&YYdbu)YE+rvef%EEWdOUOCfE zm;7G=0A_^{`8yUY1Xph1jckz?oy-f*S1Z5xcvNE&L=PmZPVHoqCftzYfNlY>pl@;p)=dw9~iUtz138SyuK`YxyF+;VF-t27x%{E*_BscCsZ2*%?T_X z(Qxe*Xes9N*ug*;OdGX-(?983c!_&nWR@bppjERwg+{v}^=EX-Dn0b|MYA$(a1@~nv(I$^ZOm9{xhN-7FAM?vYlRiK zSOOB)Vxg=O@Y2Y0Ail)jWiqG?&q2$*{6qt^!S7X~TN7_VtPM^;LV>tH&65Vm!!9|H zc;BshTICIETgyP?7L~L>w#ry&TQ7AsQ+6shYU(4!if)l!T^S&k-r2D-HR82f!ZhQy z2gpj1xeD)iGy|VnbDv@WSFF#fS1}F4m+z$XSC~wK&3?1p6i>f33d}@D-4)zma89Q2 zQ{Z%w(?A>B8Wx&)WrLuIOhPZ&cxu&8M%XaGeXoj`FYsZC+V1TU7azqO@c6+1f00vk zgglqqeSpr+yakEE7G}TS+f(XxF-{1nn-{YsMM~SGuwOteC9tSBJoj`_0z}TdQE8x# zGbroy4N*hG;u04SaT5eQG{BPc#@)51qsTB=({}{--AaEwuc>b8NOSYG@JCZPsQ>Nx zk_p7R3Ldm(A2u-FmBi{b6BL$z&540(e|9+lZZOf2x_wo}`=|B}qwFR+(l}Q)FL9{T z1N9mGUza7(yc}rI&e;a3j7=4K`Rqk%B}r4ob+QvDMLy;p{e^k)`%LipEsiPH?dE;Y zGNreP+#nw8mXkYi(_Q6p{@Bi&y0R^fckvHo1pTiaM&|;WGVIxLmm=|I(DG*^j zCiAS`EYWruaCy-*Lx6^YPV&Xz_m*xlTaYVFgX0Vu)m{rT68Sw`49$gzJ#UF9sqy5J z+3IzrHpk?L(05~aJ!&Y+>}Nt?$0Ne&a?ryc<6nEC+={`QEDrLmlj|lsF;zT!SkI2m z<}bZv|6V;=^@$ExcJBv?_O(i0)PI}LJ%kwuC)A3llr66xup21)5cud%=I z>4}|KUtX>*uCs`1;bHCV^o}>D8Mc5FP$Y3sGT4`S#mA6t{sHq47$BuPeJ&6G0|=|a z?^uFkp|Ui8kWyKw-O7bY$?N7iy*i}aXZ(5R$1^6d=t{h{AC|2}m+3Bv#UzPEr-AA< zYa1XV3I=3L3Mec9Xcw8(E4|3Tt){u|oUdt{t{tajlYx_Nb8I`tX?2z6HlVOb5z4W- zk{?x094cu7N&IkB0%v@0a&AH@)MRh0kxC={%ncAdO4vBJ(N_UM2`koq6I6(1L`yHl z1v!>HWifI$!HX3>Jdo`0Mcx>R`;^Su0?dTM?2}pkTE5Y4oafQ{2ZadtLW!~Ioov+T z$s+yxsYP}su*NpUQKbM_ICVy1ZG6Ul#(9QMBJDQL2_B-Xydt#vqz-M0(@?eZodNl7 zlxX9Rz#e~B$FEDVQq$Nv9F4ingBC&VDZN}awf`{^eAok848L9dEv4PXb*^S#N592| zW)_Q@CG#U6H^RiI;U2}s9e^x-2MM#j7ABu7DMe|OFay55iL9&y7t5+|UvS-Il zk8mGI!rf~(x6aH1CI_aSz~-VyyxH98|_vTMTL;jF*oR?e6Ua& z!*-Ue=0poZ+~|T3fh<6w4MjfpcM!4#gJP}m?!6)9%J+!SJ;os61K}HX^aNWydFzz9$3b1ITF2^q}4L0oC1B&v~}qI$26lj;nIp&sZjD5bi$-yG?UE zcMq?fKJ4P#@WxK#?t-KE&}eXxT%xq8YTVHuoHw2GYGsWYl1{2Xrl z#8m95!eY($6+IASp5)MBnpnhxEiFt@Vyfv{)_H5Gh3+w_bPX?mZ)_>thC!L&$)_Ch z8%(6UQwkA`D9T#24Q)53jnuslTuvk&7nR2um!@&OK<^<7uZ|*#c;5|`E@}5baH|!A za<%JZ8M)XE)_pap5YZMmO`R1qKGm)Q)`G&GVi%7$#emHsfY>{xZtXV>nS6B=wFa}4Y`PFNS?T6?8bC(JmFoR(jpIm)<9JDG z_c(gIG44RoN&9@eVa|3+QT52hD4h^juW8}QfL?r_=y76S%+R;_f^{SRa!F4jxK%3l z*R4_)G}5z49iX|<(Yjld4qKcA<=D~3o17#o2E|_(pO193>$Ab(mdDi z-bnU!Ywd3CUpp+hKdM^HeXi5j>3{*cJS3y=0^B-JZ>&0CG1W1x2j6%t(Fx?F>av)| z@Vd+^Zs`8&n=Pk;0N`kSr7`NiMh#FI@fH$!)U`64DF%_{jl-2&l}RT#*>r+iB=Dy= z96)|*!ZC%1oGmLQC+}PKB8gv~H%@}!Pop;X5Vf`PQ|ef^NwbNyxTIf0{GQXiFv**G zUdTeyq<)liQ_DcM$+$VSsJF(xHFaEOTsl|Jk*~aZ_j|ClXY!4rd)qSm<$bK$w6!H6 zMswG}n3Xrg_xpzZtm_TADw5LRz@7@Sg!-3*nnx6#Y!5rgE`G!B9WM8#M8~m}@x6$6_=TD%XoK%?&XkDEfL|O#4`gK_b5C+i!1q0PV<9;1yIZWIb3}Yqk z)HJKqI8{S#-=O<(xpgBHH~p@?Qdo%LzaM9vI`}3?Et`KoNflfrn9N+&){O`j-AscNkHO4clzkigNV=v**v5|R& z?INAmO!0@>E*+HWWMbTXIzaf*rD)w(QfIfSD`z|_N%m2Xb7D4ADdxVE;_^x+j&S}Q zcTDS`GTSZpyH9RVNk;FfN5!&mT;rOCg=3;R%i;)5e_7v!1yZ*-oU=V)$U*AgUVyTu zD?P3G%!WCri<;_=cmc~aStPaB*UZtIzH z7^tTkrpIGzRaD7ex-CC{$>Y-yI~LRW;Tq}!Oi~Y6L+NCjOdRRqKB=m zt-%?bD4(hvoU!xXmcfy=Uc~D1USXq0nw!p@CIi;fkaO|qssqNiHvh(-c`3>!H5g22 zZRLNp{+dPE@X`URf~Z#`?NL3&hSjLJmwtw%<7U3xJ71HG3_~C1q&#=iC!I*v*b0-J z5^}ZVCQ3Xk-qqE*AOY{p+W$5}({wAza=%r%NeeuV^tLv2t}O!jAj>~3Wcgc!EY_YF zDGKCwBnQ_5_$6eWY01B}Xu->nYtuph;;%^{183H+bNDR<7+3bM9B>n)Ufj#^gxi`x z3?@onzV*2z(O09(D12Aufsf1HljqJ6Zfo{H%w60M9xqBUEn*%maa=9-UWjzO>ME&2 zZ0w8P-L`f098Zaznag`n3sySqZ7-E(y3hl$?wN}TJ@xJ7kAU^s>g&gpb_G$MWtY!E zOsn8fJ14nY<{2GM(YgMj43lStB4jgtG2TfW69x3rTlb9X-Vx%=G1WERP4AS$8Gkm8 z^jZ#3yw^S~0rjj%x7ua@MAXy0tcL*6+(PG$84kpfgl!HzD80@f5GDw2`PK-RXo7UL zo^vqtKdsz|N_HB+pD0EglT25aGZr+aIX4mci_h^pfn1k- z%t3Fq8>q-SVUiKysx4npwix=JU~6P){p(YFHg`Ut4!*s_w5L?AJ$T^_iw5DurOBx~ zaae5TyZn*G3um5++X*-ASPNL{srUF&NlkiRMnuMi2$1Y2BhL^ho)dz>9fK29)iu~p z3sXHMOz}@~)2H78UazwGR&BpyRHef3tOjzs?m5I@f;y=Z?AA_g$=!)J{%z3yA=3IN zO|~OF4J0>H&2w8-FV4v|{ftECyXw3f8r+r$d?tUhWq^x@tX-n(wpK??n~d%U_nnK` zGH#=Mo?V(OwY^bb7NQ?6F>#ww{4~Mr%U%Ne4pZ1LrG5WkC+c9{zOAg(%k4wdX!uzk zk(;_b=u5dD)}qA~D+axF^;HeCvh2(I?Dr{Mbkoex><7P5q{~2JQt-#bnWYX zhM68`IFj}dr6NMeNyUdnR38VfpTeogpx^g+L_fv#oOS@LU(g-5c*2_*Z@Js{jRtOU zOoat+1&oInq$A_J1gYb~yDquSJf?lN_M~w?xyV3g^KX0->Pmt8h#Xn$5aFRdb=k&n z*tg@71pj~oih=ExkVr+Lg^QO?VnVi5*re7)E>dZR@^a9~>bVdnW`iJ@)>^q?VlhWj zcUq)W-1#PlzIcu}f4j`o*uaFj$b&hlm664kJRg^}TaK;WH2ul=hxT!wPMBL|bXUbR& z&_??Y9zBOlP%`_5P1E7qD#`Ue&px`?<@h7hR?CgX$z3}u76~VZMGT}~*t*&z=7_^t zk8vmMyf03?W}Crt=LTfzBnEw4BHs+X>KmkcRr60D+wBN^fKOsYzSwU7CR}L1=;T-- zTgt7bvsYlZvW?flZOcb${YfHs>H9TaIXaETD+v3s)Rv7q6G&f1+% z87?W{4Kfw2;nJNSWLA>GM!2Xu#(TuAmrjZ3w8$2teAX99My4oJQx=yWKT47^mjsgV zy$UHa?+N1YcF&|HQ_jpWHKcdM4T#h+j8Uvam0Jq9#yp`ayX-DKbOacN5gdbK#AZmD zCyjBuBvBV~8}mezaEVFaT_-FD$KKx%j}Qx0J%hM(iD+tW>`kMqE`w{RmgzZni+3)S zmHRG5o~ZwqT@^YcWCOeWlykH4NS?D!Jbt}SwK4;0fzz5$OR;3YX})^{kwlF}Jl(6D zccNvVQ8_l+4|uy`MAn8&IX2x1_xbfH5w)UI$y}Pv#kf7r#oVG0!* ze%gIxcWb0nW3k~?F)Oi}Hj+`oZ41KZizwZC@7;r)H77_Wrp$v{FFD)$h{EaANU{7k zId<=fCANdg>*8tW`yO#@{8-W|-b=EgmfQ;G&sUb#{%DF3@xPoma8vGcVTSJX+sJ$3 z#AC_cP!4tsS@>Oq>}WfcfNCT2ZzB08;4pCqn%x=jgB=qNY{~Fq(UMYBjF3dLYf5ya-08k~CE>Vp*{tF5r@Da^ZtmPd=Db>G-5wWqM4Z85 z@g91SZ(e_Y9`8KaYZ=`PQC(O`c#Teg*s6@&nJm%gSar-W(DeRx>dn*X$;u7?S1`*} zV5Fkjn9q*dB&L4uDe~Ffon8+RR4pGz<{*exmWzyOIJH^-#^l3CPnNti6VU5NQM>=N zx#?$AU&fAwZ32RIz3lO*xRAnarZ^rDgD`N~1Ta567i6`53w0ntK~S{nft3xg{&4=a!y<`jykYQbQ=^6N-%L|NY>A_dEt1qxY=# zs>UD3(B$Vg)-z#q>l@C!a{7rL^hq|B^qL+Pt04n@=G#q0XhM_Z!3w=w-)5Bqjg*;8 zW5@tpRjDEQ%QJ-afdg|~1kPlrl3XnpMFbCIq@jb%`zX$&2u;%*9h`k?VR_)1FWY-! z7*v&#f_$%cctAGC*eQDfbfZ#z&M_ZPGFI~?OdkwLddP7D7%dZ1F**GVB!5^g6_{A& zr#nGsmA;c?qt`F`Y(FIZzAm^5#B)?+rtkTE;o?32g(!*88PJUjBdHuvat~{>0NO^E zu!PN^P)=*H_l(4bs3d{flGq#G+h}s2v^JVC7Z8ZU^KBSIkp4@Cn=givbo(2(_>UpG zJ|ar*0J}d};^3=ymWeM2MbehVG$0E8%g-gGfCM~3Spk)}?@sM; z2AqRfR`;V1?=OQYOAPc07zgIQt1m+Zp8QOklo6Xadr%aU`*}M7n+(e6*}6U3+Vga2 zQ0|^YL?ALGl`Mgeq9;f`^&X-KGdI#asHIX-r#=htU_aPr+V;Yx5S#NGRdK$uOO}KH z$V{EQU%kli7r1!ss1o~Px&06e|Gezs4y=7+v|IL+NqN?XYr`_FDlwkaB5$#9Sb=Kw zOez3fcBesFiTUBC5uJ9_{ksWBO7`*+WgsBBJxyz8-&b*%JJXdtBl)k0-Wz~CoB;qe z-nc$D9Fo|i$L2wyFdY?(bChCf09cioYn$gvg)K#AqC$kn?)Z;!N%wK!%RCQ2b5Rtw zsp~Nk4Hp~aDRw4q=LT}GJ75dT%-Sg(VP>imF_p4+&2X@;j_$62IX)Z2=BcTwxtVKk zSNnN*K--`#&7UyaOarmYHAgO6v$_V7~bBoZ~{Xmh;Z+q#}{8vr{(VWP4=39onEKb5J@uIHq3>lTeYnc})P*L)!U9B?t1`s1h@mDeXgyJ<=T<6^8<#o^ zY3z_2j?tZoh6@RCUzFg!$S8J5zrRApNP%hv-?jTGE5_=URg$!E-W9% zfK8^$(DOMrR);EN4U_%%>aNg>@8x8u{q9EQIhZ^Xo*F@{Ht!@6SjgJsk6r=wDKL6o zjPWyf)ZM353N!6A80X9I{yX7z-mwpj#3-?nvW9H@DSXY6pttwXk~RoNam)5-(P}B{ z;bXAGp1ZkV!Jy5}`UU;vhn{mgRbGo7(d@#tVO3F&@ZlQ0!1oa*bFd_VgGEzmQX~5C z_IlYPM}HCyut`f|WT<%N2UiGnW4i#Q^FS>xWN&A+VjdC=+Avu&%=;W?57vVlfaDBS z+NfQ)pFNotGRCwrH^UOyoK)=&C(m_v;=8&L0MmDW1;YPnRo10XhM2%hN7Iu8;tu*N zJE7?`G|ocixNw$E!q@P#eU%mQE;4zOweo^N*qSIVv$%*ko)!j61cDZAqQGP(WIMPm z1eA}SKaoOLtzP9^#}zM@ckt-I0D6y`Bu1KVR(%Z=k!KGB!0>XD2-`#E5>|Q?G8}aCo*@xChCS9~?Be zDHj;vq;w*q>NaFxCb2)|F_s>mC|Kp0pOc!aEQnccHR<6Df8@Vr9808UYfe&LakiRn zA$tP+FzlGBe6J$##^ptgft;p)TV6X0_$b#tzXz^syY`5mm20^t_b=mNtsI4|J<}?< zSRd-=+x5&t2&8e5DemXFNpux<=kZeEiYJNlSvZfwcJyTnmtBg00R(=_FNc{>o8Q|v z_W;@AK?b?ti`*u&Yyg6}W^UC|jCG!;r94Ah129N&2u>_r%zi2e!q%c`Q|2`ULKo&p zUVT=rC<0U?GmiZOkA%;zOhhzp6#1ME$zx{Z@)sBOdp(AAIT;{rJxtRTzE~u85Fr&?GjJc{FVE@5i49iY=+AsR8#GrgL=F168{CCcFq>$2$K>>pig~J;XU1c; zstdR~o)HK8Uij9RO0&= z)O-55p~q3qcR?|C6)I0uJF$tYGvL&}FOx$;C@uW@t6|QnulcLSotq`$1dLF^^pFi9 zpF}IOPsZ2GncMC5eii^vBSSJ;cWnLqXpp72^s&UH?l8m7s-*DO6{)-F8$F$pu|1{s zjqeyt40>OzoEld0w(ZUI#`TGPbkL-j#pKI3=4IyZA(riJdLqcAQ`!mW!seUrOUnKZ zqau6dJ$hgGcnQKGVa%OB{ut$nH_tk zw+;@|Xf6CJ@M9T)P^4A!ytL^DYble4X17ZmECa#&xYa)y+iY$2N;9F2;Kukv=Z z;vC7n=Qcb=Ha){=PDm|eRhMqof)g|nomKnn{@!3ok7k1%Lrz5#v~&mo~ajY(52> zb^klg*gm^J(%siG5-6#s2b~{~Du>w{_c?3Le`N-vx-@l()7YTX0ZiH=l_zOPdJb^D zgJ!Nl2ww}U%OOkcVhOD$VM(TOw}5UDe5j6W+OQU9^mRxr$@GcvOk304*J^@sQTBdL zN)$qr4IJa}rSTDqEn8@wD4n^H!$oByAf#q`6RXK&0+zA4;1bgfFzn1F^Ygm%V>Lng z`D&z_1?rMyhF6h!(L#EXi~6BSQhVvaVLJK5_}SM)1DBzH0Yg~r5{q$2-XyOwb$r%PZ3qLpM>6Ip!vo`RIoF(nc>wTG!!CA+lXQ}c_sryw%G(4%)peHwS zr>)cJX21U2WBsgx+B6A|-iM#K!K2sKZ{z(MUD4!n z9B7b?fJ3+>_DnKDKUe9H$tN>ZNHp*g>}A}Gr~Cg(P6K!#S$`pydH>G{y6z?z&cHni zhAFc_99rGzL6gE{5t}*M`K2jmZ{-Px{DOm)^MVB>qXyiFKARf+Or#aSULf*NDIB6A z_7<$v*SS5Y^!Zb?@g~hx4J925YAzK1EQtiLwT_L=9oww|+`zKhm2C<{tsUS5TE%(G z_I6;1Y5u*VgIJh4F1!5YP@1X&=A!_UBMV99ehqDJPu0$xF3Iy2TavQZtncMyUe)|J z!c`anMcf)^{*irILNKpT?bzpv`0|>s^T`^eS%goKbebY`*ur7GYr^3|K>p0pGQO>o zU)tZH`8HhhqUB8ml&Mh7N(FY(Cm`}rw1pf9X#6-NkWE8fwTJ``ystwZ2zI8pc3=$Z(7a7=`sxVKYy?RSZ@%77yp45sUb*i>D-5%VvogQxFR+I;q_#($O!e{#ho zu9B*B6>$jacXCYOxJ?O=>M(TaUsL-OTgWk0{+UnnwZ<#=a@geH)z&a>Ea;1w^`{nCdk;{e>3T5mD}{K^hEu+ zn~u@AFD{0+XV^;y`uY-vO7G~JRV|n26p|=`*UX^VcUxkEzARM8nMJt&74$O@$luPJ ziS1dz97P8Zps%VRlJe&AHh-S}_X54cfpuBa6}JGi=s&2j!u-7x;_mHgU#BNR%GJSf z85=>!wypv6{5M~4xC z-G-6O9ga|}=omD;H>iU+YXUyNTXUb~O>>La-lE0?;m9vKE@^$AMOyMi*w)(p{EW1T zDU*DvhkOC5&g|6`8T}Zf`5=4TS*vN*Pp`<)B{@mj_ghCqtz4e%RPUKy@Goa7@uS~Z zVCEL4u=$|YEpoyC&g>pCtek#!q4%0TazCVaM_n<>Vg?gGx=IVJV%X=(<4HT~&~kxe zr(lPp(@Y<7z7b|=*f}t?M19kyxmFs<&o|zr2L>JUmiK7A>(-1^#iqI&Vk9G%kb?d` zSH%U>O0QF#n~@cPO)yfE8L+;@{=}E2$?Kih_yx~;?5;CBkUUtFv)~-0z=S9sU=K&) zf)XkW^EQcta-04yvcw`!^_`=+IY$VDeST*^+Ov(V8Kb))1$tXw-LiML;O2gS*t;&# z1E8a@%Q;{A&z%w?y=fC9K_mx=BB1_lkO$0Z2{2I`YLV7 zfITtl5&MYehJQzzs{OJXE)S0CWDg+Uzwata0#fvmx0$TJ7VX9Wq#J`xCa#AxdG~V- zze*hk&{*7L$e60-r*-? z`h-2V6WK~|Vs1tbdJ{(@{X_4ndJ$C`D9$!zqmexWZ)d!Iu1y} zJb{^}q&e$qeii&tauN(R&JPz5F0~RnY~?mh>2xQ9&S=ro+?#0P4pE>h&+ryv%LCee zKj4;SEHEJaRaNj>@+mbjsRIg-nbydH!Wt2d)r#`~2|m>|5YHO7>p)UVdtBqXWa&6M zxc!iM*rGF)65v!dxTFfyUgjR?JFmaJPZ(;;{$i5<;sYH#HoKNNNI^m2E~N+y@)?c`DHNnJ%fcFyQ#tlY_8V{W#Ib201>%dE=(~kQwITyG^XiwU`hbZ%&s=Oq%ds7zlSo5|xpYq7|u= znIH#Mx3*EExwxwJ0E`k22nEjBsk^Vs)jpbpT~}FK7^@-G?QvunsLeCGtKqQCi^jUY z@Y9Ko1Tu-l6s-EEIz8F?m1^yCIfK0j-tC-MxbHVddP?2~$n^>&fg~|0c)%wdR4Mf| z0;Z)v$J)`tAcb?^lj4Leu>mFwGfr$$L)*hON&|=zX0EPq4I1Qm9Tjor5iC58Jx~u? zulFL2-SxYVBqg>=jVRnJaKk4cFn`BB|nf&zN3mzsa3b z1+m8OrrDh1V9SOze_afktlYolr>rMU!{VUJekQf#`Y#tT;59a@UQF|8vODfMq z7W_{@s2ySkX`NE?S^CV$LoPRs;0j6Ch9dnNY*33!ra)R=#ktsg6DU&583n-_TAhUg zn++9l$i*^3NS!Ch?mVT3G=)E$Ne-!BY|eU9=0%`cMh{Nwsj{TRmjct5L1`|BwsCuK zS$~n%n%TwU+q~eT141IFyJ{@^&f+WoWifubW4b_+qH!{U>ldr`+itUl zwYL?XX^L$g8nEqEmD}H z-ZxJOF!qc*mGw1_A8()g3t9N--00ZS=ZD(7tg*-B{eG|g&;PcIdvxH=McA=FaSsp? zZRCPUCys*cd$M@-BsmX;J*(>Z)W84^*A&n!{t&F-(7woWvb`nyae-krz!L`W6@5x2rLxM$Ge4=Mt2g(SN2&Zk5LbH)ovfGr)S z0-ogX_7etr!C=P_5;_YMB(0ry!7oc@G?N6nwL6Llks(Mk61Agwusvno;PlJCkVK#) zat|&!wGG|&$oBnz!HfXAk@R0F^f4{|WK4_<8`n)1Z4gxx7C!%)!&v|!jxm3{MPW{O zA8pVhGW-rGge~}*VTe*NSp8zdEw+*7HAbYR8+8H^_l!Ic`%+<2FuHWgAv9gb)Q> zq6R{`1^1cgT)oNNt*6szzS^WU=&W0z%3OR7aS@P0tTZmAKb0`kFTUYR(446E`@H3P z@PG!t+y{#vgy6)r5@Qb(h7108-p@}L0hBX6b}=y)`gw#cb$jDGTKLOj$jZ!qSlymZTHYIWXAh%eo>!l!(gT9PM zO%4BL8*}hpKk&ct-yBWH32XxhJc#hCJ{I}S1j8cO3T6fd#90I7$ zhck&m@V;g>P63;$b-r!0WIoj&xA@GbBmhFoFO<$!XmwLAK}|kg0Z~>4`y2B#ZSOOJ zXg%MS`g{NQ!{4lo1@QRU!a2)-oCi?-T<2%WCYD=OQI05|W|V{1zELVo`W9ua_ZuyT#tnA0lC0t z&izPZwIqGeL}k>A3^lmf&ng){ zaoH6iE*Om5wjWlEzIM8h`~paynigY<+A1o(*{3Hhd~S|Yje+6+$?*mb?}b^vwK+~p zqTH)j7x*i){uCs?gOd=TQ_{NvW8Q}g!JDkkTJUfOF)&rrr37t+)}tgzgjSwk)x2vM zGw?`ZKw)Cr;#J{;IOa<-zysId&W<}ldw6iSxSI=wG!Td*Q-G#vjbz3#jB2FRH`8<# zMOjJ5CMnF(dNO_8KPMf&4JZulLY6E$Qf9D2#M!M^sw`xgB@>tHRHE|g%26+;DZOHe zUhYr{)>84x7(t(VyGHib|FD+N;~!erh$+Vl<&XNB{ZH+sX&bm_r;tGIcApy_I$@jB4=ja=C9$RldsUx@!98 z&KbfL$x1ed6FjGF(I{E(EbRl7)E6vaF=_V5JP;r}vy_1`57nw^L6gb!A+zPM6MaOk zLACV~3Ui>$qo~4Hy?q8_WnQIsc=Y{mR#x!N>X=fN#aR(8p2f{D1wClblQgAXL8?~2 zq$-cV8_`%!w^Yv3+#70h;c$g03H-~PW`2xBJTupil@>I9 z(?Q3kbKGz3k=f@udp)RCfMiDnG}@^CsZD}I`Hdr_*qgaj>7LE!vTN%-Iu>-M8I~JI z(S(q%_}D08ERe#_LBa_}mu_(N7;vUJ3F9IqkS*8fT(zROI6oj_%<+X0Zc&PZrR_}D zRJ)?_^Lnd(8RALZs-F4Us1~dKj%1zhUv(e52$z8J%@v4=g{*FOdXuASLw$X};Uy07 zg_2i4u679BuKdK~RxVW^OM?8pldi;F0n>1woea4#fOZ0B8o0BWB4Lk^u-2<{%~=A&MH0Pw$qe3w)qp;0=6-#1 zdxo=`hgG2jdsIP;l^N#t|^=-AT4bUak!cU}t}%60q)$$zhnyeHdr zmF=0y{R4Uu=YlQniDBePXPjVu`1WPG6EySl7z$nBIZ|y4jRk$QxUKw=JaY>ZtD2c@ zo)PEE#}qxB%<)xOJ2<3BV-t;0RZgQ;TOJgN#m|`&6cM^0@bT5Gx|SwbMJw4v#8EI= z>yyd6nv2%K+0<<;eV<@nX_Z5=%hZGP^cSl(#j1=->6TxmJ#L?R+#1k&*|me08>xpC zJ9CHEB;xOYQ_BisgxL^T8dKql{4q4}r8VUPA6WFw_v51FUFyS+WF zRiN?m9AkJu-s0*6hpwzb9ZYD}J%tFwoWU@3rl+sw`5sRELO2fD2&mYJctgQk$cD;X zBa+Vu#?(ov_nY#sMG)CF(*Nd(0O7;;M{Ge*P1jAk$QXUW;-ceZc? z(9-Htxw-`pl%4~>=c!q5orc1lF|cd|v!Eb5;w4eoPD>-+DHf~jTYplH8FdsKn~Z?a&s@g?pH+IH440|aI+Nq}+znt9c|l+V|2h4mj@RW5hshuotl{99 z<`aJr@L+y{UV57r02a6x=?v%SQA@Ri| z1mnA77pSHWy`z16XdOm?&;9azj9rDQq)nB}1=(e3Zn2hV6Wma#Xd#jBr}MYrNw}A!rajaPNX}B9I#;`N7E;n&bBE zDY*XkL^uIOI^GJQQI^?uI@kMNPpEQ{zz$Wabu6>(VQ!tO3sd(*kbFa1Z(i_n@yCA( zOG|vUWZbXyPbOM%8dx1V$+~bdNn*c&_O45|fp9v%dp7s*pF1Vq^yqte)0fD8Sx1g_ zL8Qe=Lz%ffdf>(_irBF6D3i$PCl&ySQc1k`kn?1s%v@X)83(BPAx$7fl-SyUD3g>h zXT6@aEkYC@^TXNKrZ|QDkML*K3%AOo`2u%m!^M@st{7H zNYYG2>b}L^w{H6rX?_=X!J&IdG5%EN1lvE=UU0AA`?%LP@pcW59)T7_0a>uK2o z1s?Nw5?U^YMO0ax;z<&%jH_}q)S)0>tC%IF4?1i8T1tg#=Z9#u^?-=a;@p6`7~E9Z ztbl<-7{OqDq|fSwrVowNH)0N0u|&IePY~hAv^A zf~8Mahh6)Kkj=%2cV#zfugVqgN%scocd5kl0Gdaeu^q%L3&!nYCqgJHC>Y1PBia?C z=rQw?UfR$C8I8>Zjhnd=pj0OOF`QI#kP4j)I!dru+n$h+npa8suM5}5EWrtrSOZviq9O)%!}wDLzO zeu*z*6O(s2nW^|mKbV4J2xQbRaVomXC^Ofa&B(RutDV}9Uh)5AKQ{q!Fz#nyKy&WY zpNIY32?hE$0KA>^vqt;@^}gsUBD^<+(s9qJ4r1xq(kyVm9PdwpZ(j*>)^a00c)U#c zdcYF3T@Vui!6<+=5 zSbx_$YS)TSeRtD|(;_Fw`l5L>7mxHu)aOTjz>2-C#)?&UWJw7=Mq{c#yp}?>C8-rg z^4GlsHHoqF>Lztm;u@ezfF%-k9FciCruHM8d6Hnv=62SZ$tK7ni~x6(y{DZ_j3tGw z;Vr1}d3sJlCe=?!`O=SosEUtaTLiptSx5NjcDX1M`_}Tt@>=AN%K(Ic_SY6@x}6}k zqT!tvSo-OPbC;;s|JZQvE!cYPagcSTsc_<8^Z~hfdC5^kPie?ok_eD}N$>~zpvBuM z?TB!fGITfH$;_nkEcy-09+DgkcbXU`Suu+}-h)w@WOfv0NO2lw0*F2OP|x-;?g%pw zFyb_ewe-zgTmgVm7e!zm(2Px(zaJGBRUOAZO9m9>yOUEky6=`cgNO<%5U_?1z%vr? zK|29mu^wj`^Bcle2LpR)CMtDGiQ9u?MkG&)XZKh?!Uh=T1z$zBZ2Adf5|C4=3W)f7 z=S|E04%8pn4({9(>+|Fx-4tO4{cqO0$zra5D_{G=S8Sy3%w)-R&vyTukiA)+1o>bW z61ys&qbf^&aeQsLXMF$Qx6SS_`{63bQ-Y(JYu3STqs`tcC#gu*)tAQhWSD?5IgU|dJUhD_E!*M!w=QDxa5(85#uJ=lk3q#xM zo^8pRSzaE-s3xv>OPH`^%&j&u=;R1UP-jRXY*9XPF>W=l%Yni%^JUv6jyYMzz%7fM zc3G?s|6p*I*pcW9m6E+)VI{%^7n0lZ6T_f-m*!U*5pg-q1Ew@ckkYjlm1#+ zHNQpsdBh8@!5;$hJ=H+hB&ldYF*T|e0beR7g2$8Kx0_a^Ss8P*fl+}t?3MR6fZ&0p z{ca(NxD3!vXFr|o4Y|0){c<#s)p6jbPEo1732Q8xQ`!iByLHrA>ujbDs)eRrp^A-U zZ8ojGgQbUDEx~7(_Eq=&z%g(7Tb2csL7Rv#V*ys6ime$2O7>Cs?gLBOjvNxb1$;Bct7f^AW(Mb4|EvV4x?x1j<enXnUym=NeudUpEg-bQ4_tkP)&rEs>@?5gK!CNdY6h{;=DgmNDx7MP zPVRcYrXEY38|cvlsRv>|YOHZWCBal+ggN+HL80oh*jt&l7TGK`i-&~{Z= z@QOwhvK|E&@96(ZV6zPZrnRe~@RQo4=!Z8}tKRs)v+=oaKCr!+T*K1@$^UKnD ztg-ZVjTeDF3J-UvIK~}N7-b(HMhSk|n<{yv!7E=1ev7}$JRFOy=VvNyQ@b1U!$(Mb zaMwxj*x)W#aj*^A$H%Ch4}?WTG1EOirg$-2y6 z->rNVnDYlt@!y4Kr_ZvPS18A;1&wh$@vIq7ey4u9f(6J**Nztjdk2iRTZ~v_dW@ub z`JS;Kj?L*aOZxQc`Fs!rT{6MU$QYUXxCu5Gb`h#%HHP7S=3&wK(6>H6SN*j&RMgnqW# zS?VpzwKT~Vk~qpgF}H_z^=@vv^ifO|8m#Y#*S97>CI*#_plVW`v}^M)=>9(7b#yN)k^r3@rpFO@r0L7v~=XN+&Tigca;Mt+ovg`uneVz>VBjlGLnJ?4ah3&`$6#+XMR9VW%lSJpJ) zmwh+p-)1~M`ZSYoYp3pb{lM<_!r5Wj7H{2?N~-lF2IMDMEd9JH^}?lwNG=WNn}{P^ z+nfh&uqYJsaaY%t*Yr6)6IPnR7`n*$?O;)LU*O~HZ@vtv6)?a!s~aw=Q`PRI{rq@L zp=K`inEx3=L8DC{Ooc|0MUMTR9OZuJf~U0V1tw)Iy{1|jjznRhFEaEvw8ZX^M>yRn z2+#NydHfnDN!Y6-6&DJOt`-|rQ@7lDKsZTVX9cRc5N>>6teS6N!WI8)Gu2c+_4dZ7sc z*Bx3m7w<5pR-o@KWJPoVZRfhR^5)=Zi->y7dS+bBW)*3PDt+Ru zDqkwDF?-?ui0W~tn}Et<*_f~WjmNi$350h)pFw@;(V%7bIl@mh!a9rr{Ph<5%?B^_ za3TEP`3t;0xVP)TUW=A|z1LiQXyQqE)ceq4Y|+u90vA8luI~(uTJCUhmu{b$2JX#SvIj>4-U)Nfds^0!~ zOK~b?va0P_aLIQf2evTlap$xO98#>Yw_Pu7k4BBd&7wdCrng zH+HIBpW183M9DulBux!_4_JDjQq$}!#?oIeJDd$|I(bVd*!&IMl$HQp78bQn0K6as)AW_9=rSzPg`D69!!p7i7qZ@=V{cq&kdflmm)G(E_%yG z5>3HfM=xc%S2>%!zkM7@8)i5&z`ylz_+(V_AgKv9D`mL&lA(yr6{5>P=_!11@@;j1 zYiWYdt(=v}ES}YbpW}7oou^*9e_XdAQe@88=NJWiWR*pRc@S+^|H@Ih1$23S4y3_X zo+dABPP>mH#^fp||xV zR?2m+cN?olFgHNh zX-zOTKO&{VNRm@z=EeM^oLJ{OZ@DaHzaJJt7r@gF-TZeW|N6GQI1|+sx+NpvTNfPImF#{Qv1*{Z4Xz6=qtDty&2b?%<7%q0uq# z!f==qnkImv7cjK7mleEn>vS0l+w40fs3z0!N$?$fJ9nm)ze|C!nM-#@LAt!nFy<;^ z5Is0m@Z(ev1iM7n+pvA|_>0lI_vU8uwUbcI_TJZRuRTc?9jM1BAZw$ORXd;DCfu&Fv-kk=YPsW08F2fCguB3qST{qiQmaKNDC_;d+B+-=bU<(K zlW7Zuvf4|8_yFpNF@Rec(mTj_vqKM8_QEpa27!4tV)WSN=#wSvgZj&En1-$&xNtQy z-`AQ@s@`&zm%7se6xR~wTk&(_a_mRnclj$3%y1EDT5Ckf+1wNVxbgqzqT~ww7jXmV z%9}An28r#elYTzmshH&S&LiOVGGt~G^n;MrXAIooxU_z00Lc)sOJ+OqHvW>>p(;@6 z%+G_jw=I({Nx#~9@BTwRuG~$5*UCFu^8Jr^l%6imWrXzuI-JP+@l%C_$r4^6C4l~v zCds+?=8%Jg8{HtoxigSFiJ8~qmEHDtT8g(+J24VpL_05saa48skm=3ha81m}^8#1z zqI3;GcLd>AAK9^qfsuFz(cR--J+LD-KJDEvj=SZhC4W;>+3A!Ix&R@HYuVfT*7ZCV zfjZ|?r|ZS?a)I-7`kcsLNjkRlv>n{rlF)+Tk}2uP&`x|5N?e-z zB^~r4ze2CV(vD*cJ_jw7OPDqVM$+75hAMv@9mehMu#4tB#X5IBSw9~(7b(KE4|ed$h2NS zegw|=xTSjcG5d1?1P#Lv9W>yUK`Pz{okN-seHLGeD3%&QE7d9lhwp^(Vujjda+UQ8 z6OfZ*x+sko^NoO-XigEiJ;C=vKD%mplu*pLIH=?>tV7s9xD~DFJgxYKzWkbOp2+(c z{c`l;^W3vqv;r-iI-r$s=nY-*?A}$$GYeR`3^C+CAjd=2;#_|bmvDxM>Rs>J zmrw`aMxC<}#OMVZTJn&_i0ht?y)gJtxo1j;C2S#Jt?aFPyOKRu^+m&hLt+!?tPE)W zWp~@y{5x7F(=iL%Kf7fMSaA5*v!&t8S^imK+>%UKlKa52J~UI0IoDjhJqC7QXsb-@ zP~vFuy?Gy96Q?HF(Nib6ivz!CqA(%1Fg21sdF)tbegJEQpyQrJZKtynmMVm`hW*Z4 zMlbL@D;vxRa=#CBegNacMigA{IlUhc85V^Rp^Y|NnKNg z?kAB#6+Tt0ae?DaS(+``Qdu9m^h)EIZ3nJ{y+1lV%c`m$N$xtm>JZ8K0TPT@Mty@! zXEQ}$q8s_Hw+PI6)hj&yjQsz!BL4*0EA$nHwqNRSBZ{B1Z>^+LNp0`i45>qoj$)tU zjC$WTagf5GnogOQ0ZCH68sFi3_KQrbt62v%H~H-d@!2xml;JdCW#g;hhBKEgzK`t6 zek-nLQdk~BGED6|f|2HPV?vDDo+FJuHg34PQAvH%n#UI9az_c=Nv5feYhl{tBIB(U zj=vaFyLwNLu=HY@^HB@f<|0~!zchL>ej`^^^mg@BW@d^*_b{?+wA5dhARr=bc-6FS z)qUfhugcN9C3goQkhCGv)8{mA z3%-VIrD30rWb##Xkr6Jr?9I$>tqdnYjUdHNF4`)qt7rDlif?_sx3}f0@1**Dk;s$? znRVPYjHY@_17-W$G9iu4K%cKq#`@nqf2s-a0PUey^AnqErS+Y7zxvc5bAv6@S()50e1{La3IqxYX}im zFLX+<`w98NMME6ltF#xI{4_~wxeHv1wlZ8D-y;;07H0D~i8=W7hAD*2I*l&^L!n_KeX;34G$Uim1;cmD=Rz93-z5FKs%u?a&&_&aUi4lfiW> zwH)G#Z>RZIwe~V&x_$yvyb$}pgX%vx3|s^o(#f)C>tQ8{(@Y;UuPFsue;8QBCRrqB z4;OWHwl@_mH#3ET@d|jTXqu{Y4N3s~(cIBR+(4EA!PKb5VJKiY60rVxw##E1;7Kl3 zdz?MDR#L>4_ey@5Yx~{t}-ogG~k9!gq}iZauIDdyH&Sr8!KFQcfs7 zU5y{Ij(B9Yr8KkiHE5YQZo;=wU$MX_eU&3!x5`zjU!kI?A}wCPfGTix(6=<&slKo) z$#v`?%2i40Vv~*Hxt+GR8ApkF=KT2!b-C~rWH;Z?CN^9`^9a;Ze*rtVG_;_iT-5=+T@c=`C(pQw(3HXWFC{kkgkKLqc7aQ^4o zJGRy!H17gS=xm~4XWh>px826u3KUlHW$qk|EYipq?Ne zwUr`2x_5e%Rro5C`rGsf>CvRSG3sM4?(o!%PI*q1r}}v)SIaX}M5|I`F0eP7*}Plo z!m+2H`|<49B_DoqcC~9i6#1`1+bJ6C9b&!qvn5{30&10K0s|Y{ZhnY-P=6Qu*Tm6M z*LH_~tC}NyM13g14x`FsjLy0#{iBlR98F?~L%WL6nHkYbVCC{LC;$Hagcs*$Z_x^_mTC5 z!`}1lnzQ4|2S+c?^{;K(y>zNCn@e!aglv{Zzf;>T3iXDO>-r@T(%fdP>hYGZWjv!- zfn&S2uP0!AW!4#HTOrP6KRPrgXbca@vMmwzT;Xm67t@NVQKvzj-STzu!0-dEEtFku zk8jfW%Kw%#{1*lJkRb*5T+1!-p_IuZib8pI;%e#!qNA5}of?~<)O*U_Z_DtpG29e2 zJ1sGyrl8iSP*QLFlY@70Bjs*dZ#2fx4h06L;1qjc@cWTg>R0bWtE6HzMU_c)bjta` zas}2or)m57OFi*XP)Ga0J1ZBK&IFFK;J+(fa@R}Nq()uTLk$?LcHw}T*c!w_3nZE7 zo!MMLm3yjR^JancSO;ca?)SF6_i^!qhLXjdsXO@RDF>*eA3Kp^TPPO1{!gj|_9*}H z)cbukXJ1*lmc^42)QWMj#X|g7O)oEy?{HaYuoT5?oo1x<#S|}=S$r0gHY}A09L0z< zbUdRJYfo1$rR2nrE5OXz4JJi?=IQQN;CE+x37Z%y&(VI*nq3dW&}sjE!<@{E7Mz0^ zWABkkzpIj54*CQ;!NQ$hJ&vX&$d_}O2G*Y41dvJM30LlBut{KN+Ref>2Usga?Bzyl z%>QHSt)rq~yS8sYP(T4`5CrL#Qo1`OhVJeZ=|(!Gks4}-uAv)A0b%G45pn2d82THp z>wTW@zQ6Zd=UOb*EdGGS8G9f5*uVYz=V{Rq`zZQHg3m7kf@8hq~f`i0thgjJ>m;8yZ zm=_+k4b?L24*`b%>$LjcQ~$r;WPC;*f0hoxoNGJ-3PsyAu*vx^^R3Gs4aO)O70N}4 z%Vc_|i?XT%E9-J6D<-3!bfM^+0cPFQHm75Q`?~Fbn<9Xx?aV6sveqhXB&CRelao_! z{kN}6lugqrjx*m}WH{SpEHNVbO-0CEK0!7(NSbgcIrGemvE-dS=3LQf+X?w{?jp5V zoYfm?Mg+58&n(O3zuvb_4xPN37Uh;p=%ZFp7f{kDIT#;5!zq)vvQnDgs_6@rJvm2>=g_6kuPC??>3pW-DVGPE)vdj=K%7# ziaaLruKeCqqf@C*e5QGRD0tg3e9ZjWc0CID59_-ws@|`L>j_frcZoV^CUAakdMd9L zp003EXqV9RLcTJf_8n0SnP^|U@iDDA=6&6mH2IiHo+RduiRe@*wju zCK$@Bh{SWm%Cu3zmms2+z%E_C{fjBX`}?u(pD+#j_xTj##B2{g_no@Da2x*w>!2W^ zT?p$?yQ9L1XcocClB>9B&Sm%W^NsHP>S;fSiX(9Oe+-iUf1(H_`X5FWYrf4x8+Nz6 zyHVBDaW$Y=Qck9h4(o`V7hRLiv$x^V_##RRTeid+KqslMqx1pt7a= zN-M*X9E@^df6cQGsSZY3E()(c|KB>f>yHx0MdZN_Q3 zN<-3Z=$^|aznHS&JhUn@tgrnFNk&30RW6V>yQi&HxlXL;sEpNo?1ANiHIM{4k($qJ zEWpWz=6hOxk@y>C`0f#c#=j!?5NK;yGXfvpFx2650I4`B23O5=J7x~eHoFHvh55T@ zs@b*M<$20)9CSx`#xO&=JXl|;H4FMJz^_oclh$K27|%r-&Y_8OkOxgwkrl1~n`8FB z1_fk;k1|r~Mz&OxwwoXGD=Om`fA3WWEH3^4H}`|>l{5O@cXG{Dtd#uTjWzowgbb9G zXc`Ay=3g;pnx0m;OY;$avuFmC*7Aw33v|4bulhoGFUQYM@GoOMrzEc5M$(>lcbp$t zz!zk*Pc zSz=?4xv5E{nE8K$zfeT$Lkiv>(3vTt$vh_nx3O%;+KQe1YDHqEQ_q%C}cy>NN29*(BC$r;+H8U*ZQn!`bh>1 z`I(-@lR8h-=X1Zs76}qgTJ7O>0%Va4Jh` zzP*pvEthE&xp`>8`vN||M*@En;zx(Q$A6_US34$Gh0WW7?BMSxMkbYn$x&l3<*gg0 z%*Zae2^jr~@2q_F@f`x>nK8LL`PNI7lIaf*AUrRE60ho~{pIQB5ac=AyVfj^qI;J` zp6?$07H!9D2C1RuR1l&XjqG6k&534ig8c)5v(ET%>tT+YD-@*v7NrrIR@R+gAB+BO ztb(b_6z)5pJr)3eyVv7OVwT=!WkYdS|#`Xzs)eRCb@ z^p>JEURJuD*fu|8vk%@9BWFKcP*aIrhN{JjXX|7w(^C6kRb+rO5>9@(v-A-Hjy{f4 zFHErQI|_=m9QIzRjO!YB;NK^wyqnk0c$k|%`5!L;C+M7`x{V{b;)2TkF*9o51SZsFy7^2l#;7Y4Joy2t)9_Qr8m#3sc|4$5kIM&h*i-<74O+T*+v6htPl3BB&) zX~q3as`(I}JtXFs`#dEi-_^dpvttLOid1P4y4wcxOnKh5W?HdOpN@_Jfz7GnYdfe@MVcg+t-DO*SLS0u}*E}yWx#`CEAPI|c}S~Kk; zwHF%n>?>DqEanS8M@4>KSZ%Ds2P2CB>K?^U(Qt8{*ngf#eQ^BT>-q^XE==*>dW!a? z>@F{)XV8Uzux=OQTgA{z@fHoV;KQS zc*qwx8DPStSTI43Q5p-q@QY@&-Y(02H>eeG3(uO$W^#G&Xr?4;K{YpwY8D}4f==8b$(q5|t=~To`6Qi>>*Avgh z0wsH6e{<$cxEYx5#h+J|P=b)}5?AP==eS%s#Qx4i zaHh?8LsUDX2P};t8>~lekA-)~IS&tm%OSRYY+Wm@55VY|oO;pZcfB72sLixSGcX5) zG#FDN)q~}XM&S0@%;GvJH8B+W`ms^~IX#Fq(k-UbduBP8VyXnuV!_UGwq!GYQ?u5; zo-x?gB)FE+XLzMWWFBWp6(a->V(kJL{!uB;t>%5TuwG{2q&(~Ge+*cRrR}x$vEFD9 zolyY)Sih~}-;3)C+F!101*6NRKH0(ZJX6wl4pNFzRR#B`$Kb zL7I_HML8(5;r_c!!rtyoqehUy6jSjP?Zq&hV!Qaej?NcaQ%(WjQ+B0@q59ZxeDS0A z_7loR$>|_tg(da?{c{}5AY>Oliy=w=y;vzMa z6n$wex&QLTI-pl^U7kdV2M7e+wVM5xTfH|RyxfWPtAh+Hr@&kZGKF^M3f!a2nXheh z-z%1|rnN;rKpN;(`DbG7T5PZ8x~N~1nLAAV{=!?dBk=-h6QX43nF>*j%|`#tf9)33 zC>g@D+FOOir}I>g-m8=8PSlEyWtPwMz$R(!K9j@(}SMvdo9X$C0UGCl@#%xdWc@BoW77n`@iPG_oDi z3}^lOO4C?Cg>15}XxI;{V_G_9Qi7ODU#hFvwd1G;&ys?^N6hJBvc1>%<)CPMK*%a{ zt)ri|tH_Dy={{cGB(3q8Tb#F#DCdCEfqz7(I1LtSVs{y|8-Ssf@5-_le=ook?Eb(s zUKuCm8oSb+K8GQ8w3_U+N=J|WrXpTg&75 z_>{V>7V&=|sZyVJa)m$fopb$AS)*Odl*aDt^Ja5Z5J5FQw`Ww+d zN_X1Q1*{V`vqa{$x30PMCK)7uK8SoXYQraA&!WJB2}|RX0IO=hR|{e=uV@ss;ZLtG zk5PEZ2Qudv{p{@T1t`myht6*gGb_PT6sf~+?w{H^^5*4qC5gMzs≫ZqBR|Tfb$K zpn0W*?2rAna>mgXyG>?Z!Jyi8mfrsjDPnf^6=B*w5@LC~UD2msuD>@prq{;J!cJ|4i&8 z^tAi#4kjKvRE$X8a347dgPFWHx{g1lfFG{#AFOp;#q%uLsVpHm2-ka+wN&dTk^H&t z)yi+%tPmHa;$7G0PI+Dj^0=^c7|(Dmaoa2(D;<%61_uFAU=%Npo*H^9d+q$KNBJWM zn0(&0-eb#{wW=3`|;xystbky4et{VCHxm#5*!fWZuM>3pchxxX`>perdA=59 z=U-15`IZdralE(S3L~0!S;RRLg0^Hw-5qaLHyaFH^JFr9wK3qwdYY&Lex2M@)9U9G|JPLU5se|+AoP%mE;lnsv;JF%%2+r;+3>>04Co@ zOO~X2+^BFHv78%V$QlNIjguAYhL_w}*r)g7k-Ox5TFRR6X`)vEW!|0Io!wOr~59C+U?IwzIo=9-}&4lt$36->5A<}SO7s(-qhtp z={$VyRvn3-YdFXR0;S=X68(#%CP9O-rQ?i7nQs`L$T?H>D#|Xq_;pUfMcwi&@a~(r zT3p@7vz*ADjxEFqCzL$$_Nc=Zfv|n}i1;o40Ni}MJ@)qQK{N8*PJi+seQ&Skzl@+{ zsML@B`0)B0-!wKrR1;?}x+vX!qR=TVXlldzuA3g_5oC2FxUxXihIO2k9GzKRixkxd z((daYU1EYlQD9d2wv;nJ^b%;dyHbMz9LvJ)SE^ z6D?DcUGY97pgGGSfr1*O%ACU(xocI2AJXqSua132K@_w#@bEOib+o{DE#Cxp#0kEr z`juIUbQvc|fF~@yE6B@oRPK^h5g?Wx>(_AfFMm7wVTTP=Zt$8vGTb z!$46+Nqm!ATHy!90lWCjq^?gG8ej2WD^FD(_)Yj3;=fULWLu8-O#4w0*?>w~RegFV znDqwq#@aSju7ymtp+4jy1Gr%DuF~%?n?%v_>vtX}-A1v3+yp6rziHk5L3$*u`BTq@ zb$O8u`{FyZF6R-#W;M_i$o*c_z5V?TK>4T+&42?Zzm}Jv?f|LGworRN-kaXfnc_hG zCeEV_#jor2RX4}|-n_lv84EUVJ=nPG*p$TcQ~xf?E3SUMvhe`ljNl9~Z|Wif|DjQ? zPiSh}q_Adaoo@X|C2a&rH_4>NYA`}n~W0O4yiieeA>v^3S^t9 zdgSd1)hhgt_Qc+k3f(S;R`{}G4m}kzOT$z1=_8C(=Ir|9v${paZ}e$Owpb-1`14K{ zPAKqh1W1Y8c|nd1kgegQB(xmd{|MwmFB$8|`esF`1z&H|PzV+^xhv~qTRQS{p9Zo( zQj>19QgWTr*r-vlFPDj=52V=`l9R$hh1?TT*oOp19kXz@WUrIlkctRI;A;BG;3qy% z0x&xlpga<(is=-#teV3t6p`*!WoEgOuSoH(RzA{snCVzVa8z~5CN6;|!YiGRvW-S* zclWNV-wn~MxP*+(V}EI_Ou{X6m4C5rJ86c)U6nuI!1>er@&xN4Ck zw#X+N*LkJIQcF)ILLe4t>Dis{E|;LV{l?xnNN1#B$Cm`5hz6(om$hWovG|pd=6XcQ zU!bj~zQ?t=w^!6;GqZEJ)JV&}sz#ku-pe#MuGdsEdz-dm73oQEtCjG^P=SNX>c#LK zbpZoU_3SXW2zXtf$O5l~xH`#HZNR5$`}r7E^SwD;T#A?ABCRGPSA1WrC{9-lx+?FW zJev!J9$Y_-Hj=hc`6Bv)>FX?M5VDrklsm`TXL-7*m(`#SgQC>OG?kOh;@su@F7i(&Ws;lOpKM_H3aqvf-cjnPsrD? z42~&GzEsE#)Gs8Uv-5nHS*TADhJ_w32(OenN;m%vBkJZ%C{K@pc)TURu2z8!vxa4Nl@{?W?uY2^ zXJQIO@uXw~9{t?Fo$~ih3b$rxASpe>1R?k4RMxJ?e{QHCG;;F_9kz$HlXzPZOT;Gi z!{%mvQlQi0pyv={DD`zRHF*%sJIMC@wl-j!t-t@|u}J7YMXCAh-XGyC)UTPOJ$d_P z^Ln_KjAYScvN5gsrOEA!v@$eUCs8_14Pptt%}SKj(ra>S4(OArOdtQW1Hl_K3hHQ# zRu&m3vDNRo)3GP7He|D;+AGFCQ+)d>nI%ciNts~rJ@m=oMz)J{yx!^4@jf{@`~H3x zN{MU>K_@F2iWY- zY_S17Gj&4j?ny{E?AJbXN?L-SHAa=@o<+hN)rzqaF1Wcf(PVrwLd2W*Wk1wX7MxcJ;nvv`>peGtU$-5m1)*ps4dUns#&1 zadF=k9jG5mHWSVEHO-71^+&h7MnR3@=tCEWQs(nQdV)HEr(bu3f>yh}oVPD@x%k}g zn{H$a@jf05at56uRyOehccl6Ek9?!r4x3ko5)wPwYIF@S&G!}_PiD4CIg*n~Gi5%S zzK`foV1{_Q1Q|z5{V8sf4YMd0)iCsv>5Q*h0_QpU z|3i!FO(8)1BJC0AJkL9(G<<|)b~y>$9snNOT1xKqqilL$qQ~U<*FS7sIw}$TeU7HL z%bUJX|Me4`jT4-e8rL<}kWcUk~PSWWTt@;|5e_2TQjFyHGIX3WWK zidNx^NL0(SWS^&T#?t)B4my8x!{Hk%QigVhmpJX}v4Vpha41`7(9*V~U*V5@UfEP$ zi+;39`|7|V;H1SkHmod@_T7O(&&zTt*7p4oqHV*>MbV*IRDfTJBH9Zc?W(f-y*c>A zIqInl&$lVMs2^v1Sj83UU&1EQud#kC;HVU0U2K^NhBrC z2^2XZj1?0h1nHRxO6Bm7!Tit1#mw6AQiHSp@h4iTRq9Nj3WX30E!N!6kg}E17r%+> z>`@M(M;+Ton-!&xZdan;>Nh%*4DbmxI+7qe3sN<6Mx}x8WDUjl`h~{(pDXKp=@Wv! ze@9Tmzi{__d5TgaF&R|kxglR?u|n;&T?eBf40QmIoH@321na71{gS&Uc?yOuedqC7 z8oof1;2FP;x(YgNy6G2ZBInJaqco1yWz zM18l`?xmOkIp!5RJY-)uE307=JB{gAUZf_z7Um21O-D_he=}Q(dm9jb@BHo|maT?t{i^%QdA@F&bh1UG z_QU@R0zl$}T+ligRkZ0e)A_9H-Sw4^pFF5cg^w#>YV)PY+6Oo}bRZqJxsJsWO|%kA z`)$Kdf`kAn$cpqFU<0nZiSUWpZxM8aakx`3GwP1^=N763*(@@7ckmqAywSHIfZ~&E5k|uO16R zR8p>a!EDGR!y?nrr@^|C-0jMLjP@gqK*q(*Z>amks4?W$u@*I!}#@ll>ODN#ia3EO=@HV2E_n>-5d{wZ7w&8>7D}dbSO_tr3Ub zJ+7_yxx)z&#e%UscD&2^f>hV9Gha~O4wUj2ht5xoblL($MoZEgI?1`KE*@+ErgC9L zuKrNbIfu`|9~#$9d9O6eyAO;K*7GUMwJ)koki3O?`$pY!*(1g^A7Xwz-{KZ~Fk@E;vKZE|QtQ(!tq5?=3l|{>@}7~#g`PkjoTY1Eta(NFm0CiZ&m=YCZ2zNa?gScPk)wg=sCrjd7v0~a0bwq zIXMW&D%m-(V5h0h$X~~3lpxtllO%G;He=2DAU3}A<%y5*Y!w)cPBneBDA;ZRC;0AY z2@6m*TjQ>mX^>pn8pNIOb|PtDOLKs+gI6_3%8=5E&EVUr%?JNG#!EJn)7Wy|+|ma%aWN5CuQv{@1}Vp^{&mu173$I$$?-u0+_UZ>9cKbx_Sr|zn#FFfz3sPCu##-ntAL%ly!q^3Lp zN4%+N0ZnwN%kIcIwG$kQZj@K&SAPIp5WKIqAo-*Hay;kg0Y0{Z!T`3{7WCI@=OSZb zSL!XKSob2yT=x#eIu&g7zK)7jO3Wide~?e1?(ZuyS& z&NN^*#And%c5EX2u*HMYGJzX77;2{-zZVhwoYuJgKa)6A{6$amiW|JJ6cMdgoH^Og z&H!u{X=^{N1YL4(w8T^>Wc0C%h=TvnNz@w$ZILZdK-;!1s!k#L}B-h5`I z&`2=|l3e+Z6>#R9dzz?NWRsVQuZeo*tk6h#2m)L>cYlqk+x@!PG_AphBS6Q26dKAk zm==K0*Focu%d!A9ee!QEWd3X!e}j0I>iN>~RpPvmc1nSg{hboaUBCq+n=%a~%^0R! zq`TkM-_2Hlr}9M40H}5u2wmC`^}70<*ORQY+wry!iHJt)P$6RFL{cS9l?0H(AMFyP zD5~qG3E0%DotGCKd(3kE)RUKY=WP2FZ~k38IXON;b_RfXLL zp9ec2S~zb9H!F_)WB}lYHSve2c1u^CV|L_9IgUQXb@xDtzeJ4ElI<9hhu5%}biT3JajaYVoPm0G;{M z;B$Z&K^k#pRlwjm8Ahx|{u|X(-dT*$7+P&bpICZkF!7I)y$1a1$rhbY9FOcdFtm-X)x{&MmDHjqqHCSQ6?*sp84lGgeAz_ve&GGF zs_`xVfpo(eSUW}jV~i*^K*Or~S;>+%G5DZCfbPdjOAd>YL8tw(j24)!W}mDhQN)%) z78245Ns*T`fLIsJ+UwYzAM50cVpJ>izS@xe0Gf@H;TBJX-`bbZHD5O&#amob@xyEL zSyhM>P%QZTHq!Jy0!ZF{62Tc>i=!6si!cfas>k`8ImTH7uR38kJy-rcr1O@~WgbJJ zZMt#o`u?Pj?`dPDY^>XVi0ZbR7WpPM_z3Id zikNlYI2YY$_^!IHvtn{Hor&w}J7&nYj~MH@8`~7_-V$!shSBE6`|q3sAMX~4WZb2N zx__KTArE9f-osD4d<`O4alWR=bW8xi2a_2{jZkyr=uh7@bgw2v%~uhM>nosl-J4q%aV+S%QZG8dLtg^=^(^`$b zvw-!7(y0$6iNewues5mDeYkiC=^0BVwj|w0u+IOZ=&jchp^`H<@c5HoW(t>fN{)e4N;>Ebsm2GXmN)X;+f?216`j-OvgPr0Rx6)x zkZoNjZK~X4pi60A3gfX+15mS?hv_>y6?aujBAEc_`*CpI;425f;BxdAIAwrw9I|gh z`MPOJS-~VE%#IjTp<2_DV^r*oMbddy+;0`vEve#H-UZ5p^|t~Nkd$a1fEBa9JRa$*;+{fbw6Cz z;z9l98zPZ=QgpNH_j995KrLC&jpqUhU~CBjx+)`N`tU;hLRvgcX(s9;|EgO*`0jgq z@NG-5KjLqca^%$#Uj7K|I!9@!WbHTsdw*e^&I!R*AeJejCn_(Dpw-5YMl0km$0ATi z(|d`z7A?h$iW7=XFCkGpjgG3g)5obArjS6Jz2RvyKib!jFI?v)h6}IQwRP=dO$eeJCy|xMmnDe!g2vD78BgVl(cEkA~|RKPEl9TJ?|De2bbrs7^Zg7tJRtNZO-8G0)a)z zo>|Gvd*6c;k7`DQE|ABpG{-jI7qUFWTHU@^f#XKBAlhpSO2hpBt_v}<(bAvjAWS01 zq~ZJ0d9Q3PkY_%t`G6MHJNH*lvbky|crlcxvM~?K%<%V2Z$Ugymwor33T`x*k&VYs zo~v$dr9n~|UrM_UuK={Fw=|oB!o6l|pm1A40Yu)&s}K=hf^2M5KsW!+Va|5>Kvo1; zOLCNydMGqzJ1f0tNQ=;?FmNwNnc2O1L^=L*+fd5zYgB0=KMI9B!t}uWFcof;*i>}} z<^U4S1PBUI0s94Y1)eq=QJ*$K&uNS%>%99&`jwYZ6z~t7#Ne(J}?M;D-KqV z@WWP3@bwOud-s%o*T|ApePMmH;_Vi`?#sMro^LBnb8D>W(`3P<(`{nZpB}{JK_ait7w5}X6-SvreQGkJC8(~ z*ZkH?WiV^pNxKD^Z7>;qZJpMhYcyr+cQLkMFRBHgU+9y-+0eQ@6Q+-*X9{y$2G6n@x3+C(7_R1eOb99u z)VHt`%*V1-H)=+KpI{y#@A`nKWw;hy8DlwymF{xG{l3dUG=AH-zh~F|)yBp>vPkawqrOsWkVy0MI32s3G7FNk_TOvAxHrw< zfCj%ZVvvrY3H5L%f|p? zvPPiLSBhD3qoc#+fGh3M&M1s4xWz1Zg`lIoN| zj~A%(XTHW}vBNi2FI46i(Htn3-K z?vo_JLF4PKW2Vx=4CZH4J_Q|n6_qK@0mK2lTDa`n0K5d%-Y_M&52Q{DTd2{}U5tBW zRLIQ~7jsZZFgR}KrqXtLuzzZlJg3@Y|IhrG_HG}dERN@HmA9hDSluSlUpeYS$#j`a z0dkppw>|9Y>=s8#!!2TNE2^;Ys{)V&GgOe?_i>okDN|xdGIp1R-_hX7wS!P)FfqUO? z=3mjLQi@uPxmrul1IuqhRb|+$U=1EG^qH3U;itFpi}HGEZzUF^i=_a_d*AC< z)rXZw63FzSdW;keGlm_u5+cTNq48jn&e=E7Q~^u75jISJcav;6rG+ z@^R}lN7o3p1_JZEULBwysIV$?2ZVc*T_(F8vEA3C=s*Cq$g;!TK z{oM1YofgPvX9hbMR&b^J_ne>sxG_g7I6+#kDlPBk`L-3($X1CUXAMfa*i5`U6%ysP zz+L8^`hXw>bN<-#mtm##*rmfgkUig=^-oxp`3mG|x@fFyJsDC3dGA zH2%E`8qiT289%9u#9BM6xZsy4F=)7!j}`~3*M+(o<)~CB!`f~^LvQqG;WMt# zcVnc%qQ1X$xf!X>wKYr#jRm@oJscH!lUC=)scX*ntg6++n2&N+YJ{N1!Uu>QBd(-j z>fqJDUxYxnVIa>yo;RF)r0tef&rHd@{AF(qG=f_qW(y?2fN{*en9)#{du}kS@S0Z; zW|~K~vq7t;<{AO=5giGfXQKB#kC+K9KSHc9z{r@9)3g6wetc+k@2Gge_pT&gu|r%! zNa`HMI8vF^ynx%@FQ<1Bg^%1!@Aq^)I#@1wXx#>hbz48D)t;-3Q(glhI zR*-)2hv=E@9S1-9bL1#j_I2FDS2_$`oxOK=Mj|}FEs|ut685V4Tw!b*Hxw{gAbG?j zeywQR!G5Uxe}41-`3r2qFfRnS<`8EYA)1NSR0OGqii))a=w9&^;ykb`TW$D ziNMZ2lsl01h1kysRzA?vRvlKNz%&#&m zUtQ{cMGRFEf2fb|Lh?@2(F>RT4=qP#=FLaCS zXR#ypcyXd}VFXCZ{c;V~&)>sW$KI55z?*ab`hKhWrU~gv=hJFBjHXL;vjqk=8=8e; zyecrjH9{&|-`6Ml)?m~tcg+p19~Dd}eSaq(Wh*{;0l^YSAgbHr4f|WhXD$^_uC^~q zq50)xvu*i1ytdfAl^&vF8WmfY=#QmCHpWzRuS0cHnGDO-Y5!d(^)VsZz;6SnUQa~6 z`*xpkTudu-dLmsdEjfIFt27AL?2m#j0ZEifkSDyy46o=*rLzgRh@ z4nmk~p0e?Mt!`Ow*Zj5Qvz4$Wv|qJlb~~Sn!RMhqc7qIo4hwwX$bwd@EQx-J<`-l0 z2x8X7qZ#1CC%0JjIZbOC6iIXA-n~{6B=jtQ7mBN;`p)8BOoLv{w`qCeA z1?thFwcHOHXFfPWcEs|JqMb7&tw@>~xW$4*W%PqikOG)iLBSw#lp1H+4*oRCyC ztRDs&_B`Y?E+;1lK8 z6_bX6Q5Xsn)Yf7G*>bm^G+`PQiUs{yER1aSlMq)cfN(=W7TqwCK+T*U?E`u~iYg1j>UQ85Td^=0{u(XglWKQc#C*SRQ8264 zVAVZWqIPzlZQfI`m=7~94r)N(eWDxw@izCQLMX#5)4TZFruDe$cG|s)K+fQWH-_m- z{!sX(J)Q_~+-ye#5(5AAWToIwwoXk04_7)QzA_(U8I>{wMeyg-V{^f)TgsQxKn=}F zsn@Mxj$^3CL^rsSyjA`E??G7rQQJl~1+ZCpu9&B2y?<*MPdH@w(RANKs>j*u6XB7a zw(R80X_jT8I}Er1Cy>NN8Ep28|J|nexdI}}$2Gv(E_`0>Qt2jFN!e>uOFBX=3d^dCddE^?d>UR1*ZSmt~cdp0m z6|JmY;*Cmw2)1Ae{n71d1{pQjwOCmA;64jr7W0i#U3-RQZ+$#*v9~oAlFt(c6QDIo>xR z*~pya+c^J50P&mfo=OzM-yuwO*Gl@cl7(0s-YAp99!B1oSL^;z^B+PO^A^Yqz0uHP z{JpTKyvV-T%}z1ynvA9WM`>k_#G2#9&eRc1{IRVA8A4VbiC>SX z=>#jk_F6R87%Vd~!(M3jKkizP^mSTobyrnJ;VR9D5NJLP!x@__lwM;g%}r@?>P=H; z?)ME13<-=nM+DO&yWzK$iVO>rEV7iscfMIwT!}hCV6liTwjZ{5IAAb%;G?Qs>XqT5np=w?xkO zdI7Ph_4JWaI)X7T=7ZC zlYJcjZBJ|z9ww6*0S<7T942UN#|h-|Ed^HI5Q?|N(5x-&$QQo7J5v7PaK`hrd^bqO zw0z);@k}b-J2sVF_mw5(x_rEy%BdDs1^vZnFQ1A8Uib_ooNeCikTuP6R zOo+)Tj>X2efD=Qy{gr0_r;ARBRc&%sFr?&VNi$_P}R$d1Hn#D^hpl|o?(Eq6F0ucOqFn5@(~euXSu{xN)Q~6cam0C z&>ppP+U|>fx#H=9U2D-S**0Z`RyQyp!kzB~Ve2Z zg3@n7Q=Mb3w1&wGuLWC#Mm8Uql(z%l=vKQRNksqQxnQgts+jAT7-o0GU4)%=6aSg> zu4U08ZfA9Z^0GkKvp15%^HZ1FonOg{|8cWp1P9{mKL6RHk?1o`;@PX*^z(q~KvxVvCjiqzPk1PnNB0C8MnIn0`T9XEUR)b_Ox5T9>{{C{P=LPl;il=-7|&F zMJy^iHuSGb@Kp#+C;eF*hQi04!YD7lwfUznUO()cK01UJ1fAz>w5EO#8j-}3e{NQQ zj0T~SYYZ4sKF6To+nigZ7X!wXq=#CDs@$jUixe`}A}N~1DkJuuoc%R+7CR`|-TsQ> z)InJo5%Mo(^MNc{@Se<6*mNcP1PTVR^|wMmGk>=!CncZ?fj&W z1f7AEh*J3Yr?of>3drnzmgT1Na5pmbV{H&8bysqzpCY%gGQtFE z-KlQjP$ZtzkZr#1(f?+*livQTmEdblQnazmWtlXuiL8^<0 zE8_gXR{U0<7%yVt(~HVF+#Rehgku<_M(VGH>(R)b!!T-?y2C69NX}Zpo1Gk>fAZVE zIqtvf=N5Tr{xk50o#Ur?Sv>~QY9=@CC|Y?Z+JE3&Zy*P~i=)(ixFAMr6MJmm`U!Cu#hVRLr%wd+9i$j9xA@5Et zQ#UEu(sIvUd-KQV{DiN%v@@9Mq5?=V973rBMwRb=lrCBovY=_ucrvl7(JV9|C(V=s|Cyg2bym>WSpF2zUa z>R1tVL_xUUR9QNDCp%Gf3lT(hPDXV>6aF(8**)kJg8bPbETJ}RTv!;BM98E5%y31& zOcISxs&FlTUHPyp`3jlH8V<5Nm6v1)kULo_6|!H^IzQg90GrJ?$5!(T-I zygZ&1EP82)J6ZPp#Wze=sVp zBw#s`&lyC}!6*boRiwc$AuIzT7B=sW7n7hMq2pCn3(*}Z+wyF4NC%8aG=7cBUdT&wo~O?ReMuq-~oIe%c59R=G@ThV4Gwu~*ERV82Or3>{8X~GI&VqF`;_c1;C z;dB+IkZy>4MctAluil-mmln=tJ=Y0L>2|y=>Q9jv9dQW@Hby*J7<9oaSv@G~m0()% zKy*tqFXl_RHihn#-Dji9=yD_HZ1Z~z!BI2fCwKB>x`IFaWILo6chbe+|^QaQgw9ZH+Y*@??b_1Gl@$fpc!;#5;G!ce)V`ubXE&gO6i)o8~2uuE? zVmsV%n{DF9Dgp;}6PI#G&I&pyK1Az|X^9_g+}75n>vbeV^`ngFbQWH?Isb87D8aF< zQh{wNUK?n9>SGNqk?b$hvs%dHRK-D+%tXzE1rFD|hZW+7n-;H8i;W#U{%lv4ZqTH6 zbBVAP!5+XE_RT!rTi0yMuIZ0Q$NoPIghFav&Z-4A>6+4~jNbhqIl8j~-lp+?KkV|;n;XS4D1^T5!GUBcF zN7)tH&DHHEDS=8_s{TGlYA2VrP4R`L@2Zkp{xi1-YZj0ZW&Ja%0nkiXsc?A+#zzPw znra>J@Qn+3H_yo8F%L>FJUPYO;o$eHfZe|}VE+vufhHruo-+lon@dy~pr2*Hh$fq} z4?3L!nN@$ar-3Vf3bLn0eRqaT6i4TNC8;sW=Ru4W-poWSXkN+wR{5_N;ErY`$WC#i zjl%n>M^QvtE;s%~9E`jKGOir>dM*CW+^|h(ut#c!d*eeYHE>D$9wWlZEg5V42ncGF zfrQwF#m#1z+<+_p?9AY19Rw$Or>bYk=l-hI*9R}8hsj1+3K&p$yZJ>=Y!R*%6y6I4 zeh>h+)vDbQ?EBe1UTIiWq(uM8?^llINfW^FFi*XMbVgrdT2M6l;yd-kk;eaHmi@U< z_oH#SbRsRiX5~FKm!p3n6?-PPxO}ez3KxOC)#ng+%5X`77{i}Dq@WQ0t@QsK#s4{w z`II3_)Vh+d*r{~Wr1Zk{eE5X1>BvkwPquj{)^dd`L8C%qn6ED1ZB^H|AWRIKg6#ON z7O!==NK$p5GEe%{g&2XMWm6;59pa^uJUUrj3)buw^_>30j@0Unv1Q4MT&;=BNScNH z->l!oXFR#)d9@j1XC?$hfH7GTG=H3FRQVmQ)%o%ZG#gxM_tMy2+d-t*$sut%GH)g0 zv_&1aQjckSQ{`;)EQ3JGJxYNa?&ng2V5euF9p@agkQjUlF1tV@xqw!OC9Ok->Ma@R zFe^IvUS?O1IVSjsDB1^J74vskJ*3qWZz{>y_97%>KRf{NCuHqT9y;D26L@k2H8@=EA%Z7CqrLoALeZArSW>()R2xjtX!0h5Ia zNfQRZuMn`WR#8p+@kf`@g|sQ-ucL5VRElL>d}<5f>&zg30^ik9yG;iTH<~uO=`L{J zlbJA@A6*29*9*0ds7DpQhAljNCfw?*zVGGYZ@V}C5u_0ivtsQg zBwvQMQI{&qwm8LeI0S8d_(PEriQh7N%E)bAC@>@fsKyUiw|}|(4|7{HZ2sw7S!BC2 zKT!snA{J4J%Cf77Ub`FXv~)4JJ(MnQ2H#g+EccFFt64PLI)(*UD$Eywu3)S{??}=bnugx~svRYrhZ3Ctcyv)^8|s_cx=AdDH^o^b-CEj*Liu$xgmC z%k|wZtZ?uS*gcn5sl}BOl#t^q==$G^i!jTgKV<+{8Wb*K>XOAJ4HdKvjz)FgOMF-T z<+&<`zXnSGFeg;d8mrOaNOdWkB&Rt+y(LBWn;o~b$ndv>>RM0C?voF6YCpObq%1Zo z3iU4JuLuolzZax8VgqY&h5YHSNHD76V*+!907!O}GX3?nVZ1_Q_!3;L{B+>ts6mTK zckyC!VJ78m{_CARa_ZS|1@06PVM9KNGtwOS4+AN;#j!n2l zqOqioCcUtw#8Q@1#~q)s>V%I3pIYx;&ZEn42Do=$q&%SV&uq*qq5t<$Y#-veW-++){vA=BD zhp-8Pim*#YR7kL~Kh$`~#_5x@KN7vjx%I6jXsde3XuQB9WL)_zNb*Kco#dhyPTS2( zE#-;t#oZ}ROy1Q<0>Y_F=Mt0;mQ541+wAqtI?)t0!V}j3my?v% zT)llik+abj5F^=T;zcg-ryT0CUKxRVJ#AZ5AYSS2eNlkGMB=$Z5S~$b;U3%ey4UAV zmHKm&bB&?+Z}ybR1z86BB`g%7157P^HUJZ(He34p!_5`71iqGgOod7%EvNbJgJq%1 z!6}BzOM{znbhk=l3H@1@C-#4Q$&Pq_o71b$CD`4-?BrLQMgz{G@6bnH-o7fteX_K) zXA|>i(2-mkaO0Dbpmy_r2Kr)H){8co3DrAn3#6E$6sPSnBzcb3d{+vEL^`qJ+J2UYGw(W``bnf$( z=WGE6@YC5or9Deo=RtCjhv{%Sr){NsagE0c!gft&7Tv_woD965oJl*Nn!|P|p!d?-vt()4>+qqrrQVqFsUz zACzZ@v`Rl!8(NO0<|hP$VtSwHy(<}4AznM0D|vyh@;n~*VtP7YS^S*ZpqhVHQLay* z*p%gkk%HDlfV259GsUtAT^)8cB`bnk38CHe?^p2e^YMR%UbnCjHXN2$ax?lNmQ4Pe z7mXS}G~?tC$L6$1x_?IYz-{>+_NrIbbn2LGAM7R@d@O3VW|i-dFP_E-LDom+Z&McQZ_R_m_Xkg6rnklXXN?9Fku-g=%&Q(+d_V z@f=pIZoV?#sE8}$;V6O|?dyWYIWJ83mE0J=cxvuzHsUcIENbSH)MsksrG4ina$zXV zug3&+k?#$&lm__&1dHHQigm$e4`dnpY`=)a7M!Ta3z=iQ{hC&(dzP4n`v^0g&gjBld>~N!h z4~7I0y;rgda2mA6$TJ@m>2B7u9wIkAJz<7Fq@^zilbdq!UM62y$HG^IBW`a7C67gK z`z15x6TVjzF0?kP-xdoCvWjFxgzqmuy?BQ_;`qsJ(qoNl;#)IdX!7}kcpH-StubVh zHz#=B>i$ubyhK9QQJI08OIoE`tl7#?wCduk;ktYUW@B!LnnC&ahQ%i~G~v%*@9i=i z#1+kS?}m;U|F1eUpm+_mbMgA7**VkH_$E%F2$M*)2>L)hAV**FJqwUgpTb#jUX!I1 zkfpGf%%=##E#}kiI-i_d_uCmwN3Fa*3g~$&&qlJs@7e8+IeHv&6{+K4)ha{xR2e6D z`B>V{*=rNyAZD5pa4$M7ihVRq^&eiHnJrs7(Z32n!j3PBlZkmgqqb4fi0}?J zJBtDppXGKMUe||nnAmUH_-`P`*N?wx>A8V>fmwc-{5PfMVln*+xG!AXPQN{97l`yh ze?yZ!LXRer!&wIV129^Jmxo?0JEU6uW<>1IqzJaOnt@com1xsv3;+Yr!#;GqExd=@ z&(c$2)CONFc8{h>(^RJRT|z*=k~@&A_1IpZI7PR?`lLa=h+ho@>R@n@`S5!tHJmiW4C<}g}rANB^tI|GZW;`De|QP_YqE8mQ)v>@YLo9}{dQD#OXLXYQpO>pow zv}M%);{sY7r`^`aX|ty3-W|U#JlEE56s$Ewf~AMMF|sxl8IywfUKJ8C7g9R*5zn&{ zhET8b#74Rg=KLt0OM@{57am7MS&tW9Q#r`+eYlVW>#2*M%1e*DdaE)=rE`tSOJMvbqO+8rcJkpqKqqarldEBCeWN z;ie+VaEO&tMC~GlcdYK{BL5g z8qH-(C*zPW3iizgm`7iy%4%AeTw_GE#-$JXWNRRuyOCG0^b)_Sq^0IRL)(mc!=LX( zS-L0QQ7jJTe4>bG+Df%plZgT|;7rNZck4cvdRGylDVh}fZHyHKqtw(kS zaw8!mPWIslz_-i{+zSwI9%~ueqR@73T^ObRjkvv(XSXXF6sN=O?f-^TOWLM=Via(W zs(sIIL~&&Qa(eBmDnYMvfG--AMA4*lRDAXpqon+`Z6ZYNdtyapYs=lx>&*Y$`9<;R zi$sO2C*pLkd&WJ$`F5Hf|2xPM-kNPiX%_&Lht^!u4THoI?h|PVl!#tHDq}Xv3DfQc z!0Ii*($c z6tD+GsrQhLCB^9j{kH=|17Wj=?N`7B`$_&Q(^0ODBN>WxO`-9CG6kI}!hi}vdEZ}F z-W_b*HM;~ym$jZY9<6$WE`zKdf>DX|c|&Ft%X5m)UCNq=!B3x$sX}JJay8@5#Xk4b zZg^Ay{XjBiEOR7!l~9l!V8ab<5W(cjjUqcPKKhoYHPo;PFi?VWOr1CY%R9UF$Tt8a zdmXtUr>uWQQ3r!;V^-m5L(y9#AMNfMH_-9UAHM_QYCqm>^-bJj+~pX2OAM!3yhzN$ zqGOs(!x=HmsWo0 zqDAWRQ`ruCk9#gJSx$bx)BAFro&S{)C;hL;i#Hm)zsi36$6xI#0soI)QRGYO-RJS8 z)cJNFuQh$?Kpze_FyVB}o{a8Cx_SgjaK?dcZuO+TLkoyyeBJ`PElvzC7kPI;Fyfwc zAbwa#11_{jhF+skfG337^8Rxs^xq*hMQk>exsLFxSXd0L$^bkX}D8OLD*wkf`kaIKl~r;P>K{qZipElF{LFhu&E>k~nDhh}P_j6;y{^s+I4f)Z@9| zP}BIGhpS%Ip)NX@1^*FynB?CtYI`(GZ}j0=3ays)14Ru_@kA~HNdcG9fote*GPTja zup~T`(+{KU;-K*AW>k&Zjr^LbP%pkdGU-Y zNbw_LC&^h){si)_0q`we!Ge=OhcnPk`jVaqbH_0*!TWIU+!m<%K~kezuTXC&Nc&9G58c4Kpoc0Bu4S}>Z=!ESWo$B zTC?w3%c5?xNAR)*V=qeVym7498*V)HWZ|_o80xGsJbZEB3gTLI6o($Mg+#R*v^(Cm ziXh0izxPwW^xXt>vvB-M9}T$&?7HFp2;_aVoP01;88mNN9}{CDxcLL?Nt^Gv80}a;`~IoXr`^K#K`nh@ zxx~cBcDE?(Z*LO%gQC%d2nojI=%9flz1aPi3N4+itS;>7F{(Q3X-nkXIKedKCPN#) zlhMHNjTGx9m~+QMWPcDlx9$en+^Ba~ZGsv8y(O5=y_;pruhc?d`x*d{S-Q0IY7)8*T%xMMm$~ z&Wa&1|LjPWuAe1R@OmBm8e--?36DLpFH79@Q+s-$6u4M>Mm}0n(Yz8uZ*c%4F75Us zf_u?MCAoUU8k*`_d`rW{l}g``$O2llkGgFUU0Oh(?!?AB<)>>+P`w<4s_^*opR4{% zZx7w-cD5$p>{u5w!^R0l`N$nu{@uWuZIXFPT!C*3*Qan3FzFMWU){ z2h{oOHccLE&y)0=xt~iTlz$>`%aInH2C! z#Gz79oeZ00N@k#8-=@vdzbS_8Q(C_E*4d|=GI}sy!}n24WztWCU#XZ;h4PNut(Mr8 zQzf|oG9v0}I^feyZ+uz=8_d9YtYF$dT7gsufE~F_y!mmJv;wRU_O}5rbxpQ~g%s}$ znAXNCuD|>42AJEYmeSNd`!cCegBZdc<(I3L^x3ON5TMr}gz6nojd+E6#!3NIK2Dm! zhx}s>0}EWO8Tm0KAL1@K`kt=TYVIp#jGl8c+AlSEGWF|@sxh1DfZ0nJSG*hI7PbP<>BuzgSP_vCw zIB!NE=um~LS#B;9`%B}Ih1d)f2?{qL3|30Cl%@`=?Efgp9LmD4)lF8ocXSpMkNLp{ zX1@)U@>4eb5=@kePD5LjYpanBbZQma61f3BXk>C3^SZ+~TZwM%ZKU z!{39Cb-q_#3}4jADNKS#t-TrKTxa)0hHGPQ+Gz={(U>3Rxat38JW70|Ig!x&w2wiu z-gbpj_VHQBYXzalHiB4dGzj(}T%me|q8f&FzM4eo`S4rbpD3Co3#XXiT!zEpx-;PQ zK_G3lMSWWw(=w7a>C-pP% zC&P~{|E`<-h@_eFIX5U*j4SXx`8&QP>A&*_8<9f4LVRxNd9}_O>Q1KD`4fC= zxNhopv5)vvp}>ThyYqu$tz#Tb9feGb`SfAL0y`9|b3RG?0Hi54~%jD1Ov* zyR5N`e%PhcYcBfG2hh+$M}J+LjHW0E9dq9sj(0zOJ8>+UA;{k@ji}2H4EUBfRdvPY z^co$CWAW?BY9L*Ei+X~6@^T;<-td43#<$O9LX{+jf7Eo zmm2uQwOVT2{~VlIJ_1Mft7Iw=GJ3t{lru z2YCarn2{%7zoWfPHWSNP;-zGZDgSu}8Ufyv(~~0;UyBw9o^~S=Zho)?CYNhlR2*1_ zp6sjvZ8zp#jF)V}B(dKs+hB(jsFxPF1sV%w_HZut$A7fiD2(x~z&Xmja^2RO9WK|$ z!pS?B${8OwNHO5k$)YJG-PMrc4Fs(TrM19i-1&>}u=S0>z1Fi651Juy7aI4G>gkXp zd9u`4_`67CvYbgyeM)if&*$9MJnabEc6lhhGMV`B2@ndaOzr@sfw>$ugAMG%3KdiY zvTFRs9~NkXW2qL{5SU_h*NCgBSpV-nVBCavzf!TZOM6JTm#N5q`@RGqR-9q+91_1U zUeLn3W}*m)jFOwFz8gf_$_Z;jgVK4rQ{pD2=D$y z?Vl`C9lr10#?9$0iVSwDGb|M?B?OC3(%{A@MMDk7f+aF@D2oII>hn<$6aI|8HkVzU zjGlbmPnFW!s(KaI^NYM9cbO0-J0Otmv%(s1$AU1wTwfo<4$2OdTBJ=712vpUJyKfO zV*QadQ3J-vdxo#0(;0MGq;egsKwL%LNRrV3~lF^t#>vK1rk6_1iJ| z?b6_mDfPwbSx(da);9|TrMg{(hsvq;6&#Q#WSxP_dI9Pp@fy;6*Sq??FAR?{7VmXj zbxbBXWkewcvg{v~LiJ{En}gw|Bp=Cm9;*E9mZ#AqBR=^jZ6EPzC4XVb*;ALl|45xG zV|yB-+DndlWTbbC^x&go>}XX)gYx_cU3}Tp{o({~y>v6Se6PN~E7OJ+P#r2A( z`<5{EUcv{g>TRk&f4F#pPp;K{f52IvniU`hgYOO46f`oVyq2wh-EWp{<@023ZzR2S z+wocp*qWwD)grjGw2zfy`p@Y-v!Mg_gz$`kyV+Y9`=4AU4XDIm`KzQ3W1(94t zJ|&EuJDiEblpjF8VJ#41dxx+O*4Y}EHbECPkSdDzJWP=BG-#HdTeMGP zYQv?4>W7I-t>l;?jos(FUwBWr$xbd^Nq{BeOMYEWa^(r${>t8UQ;}Ci7N1+*TBAau z_oR#CEIy>*E4aO(tFMdp0nN>Nd`}?B*u}3C+0U(=&6f@DzPR#>=tUxx`uujQoBR!U zEu-#oK*?s8SeO%)x{0UM6Nu3V&Z^3C%5LYN*kUI8y!m~N{3g!12)eA;_KbUMJ#FD! zA|B*Dl?kWHBXB-Ks1AudR{E}w3#H{$X9wBIoEn2KNpP1+7edPO-XO10Oq>(%+$7bcoL@ z5PYEK^UsR9)pZNAp49z3WMnv2_not1Q2ubh=#ep@mj-zN*ffXD!E;p!;ZeelnU97Y z+&`4p%CdXC{HG$qD2Fm*utrN(4{68%4UQ_257J`Oiy6BfqsQ~}dX?1In1)Umht$|B zbQ^T6bIepmq(V1icy>7q9-NctB52?eo)pBK)PGlKcer)U;GK_w4u1x|!eu1QE4#uC z{@eE~Z?5k;C&lH6;%X1mIFv1%Mjr@^Ua!WnWejHZ7pb!B?!}4x>jjAGz_F`WY8z&r zyR3IZQOU?w{wBn>eNN$zMkklmV<}W*>*LZ}HuRaZeo1lgD^8*1KPes+bCQR@v8^@q zA~0i}(~Mw4qC{=-jAAIlA7xLz6kM z0I%K(Q!GkLtkkb^MoXF!+R$-Uwy*u|xPu-PQs_K7o9K+=X{Q&fPY7I}EGI>+H+=A; z9t_*ARvNeR|ASqt#B{(Zn^u(vCikm!5`}o|zMpy1Hx;he2woky6iu%2Rk~&226KbC zj+_vCu4fm+Om*~Yo5N3<_+S>{+E!qlU&rFOk*Mj1A34k|)*5VfhTPx3f~0u$9Vna} z>{__mh>)KKP^I-MRqFh&O!ewE1T|vjGlaE(nbn(vZUDNrfy38bz;D?IoXGvjXf&;` zYSSf4%EhbsL6A?fAQUBfD)cP4ZfDq{PWX(Ow!EAMe|7uotGOyrB$^#o%s>(7U09p>VPv+8oQ;cMZ^6#p$X8JId?$4 zv$vC`PuBdzH!1HxX5Noq&qavM3{bN>9G6CMQ`B~ZL?9DGG<#Q3ROZ8b9X8H6(zWyT z8Y{Lmb)Rp2qO^dNo?FeDd*lbR)`2NChZ;!dkD}SycyY4gDBmjI^a@E|;7#K9T~AiJ z2*e)AaJ}~~r`yi~UV5B}ih2EwcfjqRv4=AbQ#?*Np|s*+Du?9$$nLUAQ{2L4Y??!w z+MV#QInmqtV-VVCoaPJMGT8CYJy~H5#Lo`&u#D5q_)s~^y*TC5eQn$R6qnM5d_eP1 zRUZB}P>=-pz!w_MohVJzmOZ$)G;JTd4Q)k1uYtcDc6@;sm!MZYI=NOKxf5Jj9);u^ zwK-EY8r}G>yr3|PlRD84go#LTCq$-=R+qmdoFn1>mzv4hWQlglRUo1x5N2Qd#JQ?q zHJHxG<>oWbo|m6xqL+1M()|~CiQ;Xr;TT9w+~?eK#@VMRw0ffMJuWfn>BrI{*+71q zNrJoRUJftsg*RIA$GU;FqZ4QHA7L3Qo$JCCKd-Z1%Td=dDb%s$VNVMz@4l%v(cN61k&Zeoq{hgFG7o_y)b z-@?m;J`k_EDnZ)&RT@cRHlHf6>t}3Wrab&a#b0mXeORj<*K%ppF*j+ov`CAD6G70B z4lk%~<{T;6^FX||FLyz#7ti~WBHm{-=KzyXCS~^AshKNKH7DA|EQJ6?#G$gUAW@<0{S8TI^bnEn@po_3M7@xOkq= z4)*`z!LqovpC1!Nb?Q+Ngx_CO2H0tHs4PT!v=fdbz~|&2o*pce;V%^9s!k7=O^OUr zB=F$o@m9ZVW7uoly&bq8H*e5}>hskrD@UM@gCanJBB}|Hsg|W4U7~5bG|ZSR*v4c$ zM8l1wT9}DoirZbq{H|P59j;~KUhyZ);MHfIOi^4fq-5XlH>O zB_aDf9ub^>qM_u%l6n+mDVmmlYmSbJ7b1#xCgu@Lqk8u@=S4O6T;%CS^;Q`bZ{Mud zC;@o-&80M;Dyyq_^r|&VyUH`wp#7sP4agPf1B#CDdCgw)g%c4}uL$V$F5=)L)~(Ki zD7&{DW0&I$Rl<#oDT1OKj-Y%pZ5RvJ3XOTWXuVzg=3>-uS=6kqNZ=e?HetFI}nj%xZ*nqg<4WJsoUCf*MHnB2o@R<@F&I zf_Khik2Q+oufs?K@oV1==RJHWYDYpT** zi5^H|N<%6moSZ*5%@_Z%4H{o%InK>xH!39>JG?zCw_?jzpwu<{bqDyQ_#LSSk(hd> ziz59#lDb{=c8RB&D<}|*pLc|H(P`Au|Ji&$v+0RhqZ`am~8QxJ8#q4WVEBBSsYGxF~X`{e7AyrVx)PS!)qk|J4&?4jGp$KKAJ|?MhPGe*3g=Q~Spnp&2cAKh{V0p~ z@`X!=uEZIxm0b|x7bFoy42O&Z^`E0sh~IR-1T2{Cxe7qJ#Og?wE&$-Lyx{JeM+6@l zr35}Zq2_d*0n#PE1RVwm9M<)8eG%p`*oNs;!17z@?sKc}K?_oRaO^sSi+bZBP}t&} zej8RzL=Jz2IEZAf59fcUk^CK((`(z{_W%2V{?#RpX}9l4NZ-FFu)rynPWwVclSn}? zLx81-bw<{TV4>G7UP&+v#S6rK^CY$JA9YaYiLRj?b$Nhv26Q&nnk;D*`i*v z75|FzKw8>fL)RsDQ>c!P)wLy_GjcWs_E+3$1TpAHv^q4V!>I48)&DZ!pA}d@!j#Xt z!o@0RRSc<$^iPiJEJ#8f4q=5OHlfXsOpwb#QgpQ<`3gA3WCx}qQ$F@Byag93-G686 zvC~3(EhAJ}ffp}Y4lA~J%D`#dC4|K_M=hAp%jIyZWQ*7R_(tCXs*Ew?S}T9gD2_66 zx`8($=0()clfy?%@_u*tcWbbgcHy6m2AxHaJ;zf8&eJ(Rr!OmD4*ZP zOaBs)|7r@IKr=^}h~5V~?w0GqBaTDlg#F0Z`WMpeLh&yLrW}vdsD!Vhf_IU=$EXg| zBion8$udR^pGv9h(|%Tt$i!-2;-4VKR`EOUIF28wq@Zs7qXnxOy+*uW;E+g$N|}*F z^q8HMp?beYpO`X>B*Pfs%nqG$MglZx$cY378}hUz6_cLKAxMsQF;X(vS9L#=ZXWeF zj>xuj@fa!+ZKiSC#!H^2w6H1garVz}!MJi3>Z&6BOc0;2ib0XsuZY*=l}_J9gCi3* zGP77$Z7wxd73CYm$u(SHWdlk<-ahKGF=<0D%KBxSlZC6Z#a$@}gegsPm?mz9)@fsm zo4U7h-?x_yWDk;fmA`K&Rho~SkQMU&IxFSX| z(%!;gvpk2y2SZ}fn!QKu&r&d+|W5tbfpHdr~G)S-I=NIUH7CRF8T?wHcTseh8w zjTR4HxJ*4rRhr~(ZEO#e*UViQbIf+T-n$z+Um#dyGc(CHVj)=^! zHjjY5&EtFAn1BmE+tBhop*Kdcd;6azO(fG^>^=IiR|wLc*clAD`G5GE2UD~l7)S-Y zxY)3eRXm72hlBM7!AqFr-cLYh*SZWqLiYyT+|8VOoe7k7osW0(MBdwXd4IK0Q2vuJ&-D*NiNcmo;o^|3a{w2C50dHdgUX`G7g5gV6ok zJ|i=j)}jeb{gdZ68ik!&qr7JH(N$k|HKTilrnKxdWLbxslNw&Dy0g7Y~eA%q0blW?g&+N?#j;O z0|BgFV7>jM`0xu?rZxts9Y7DDj0DU$=x|IU)+-#xOqUqpCiSOt9#-UoSx50g4sU0g zMyda=Dd=4BzTff5Dux+P|x}Q=G=YH;%hdWUk=~atmq{-OahqYbziuK@dB*1gZH#@d2pxt~19A z4l}5lLT`c5S0hPnFOh~TOrA!xDc=rk^`X|b8T}8Kc2E)_pW)0tupfWWIBlTYNto(_ zch2yXesZQ*UaSm2jw7N17tb3%wvc=NpY?if!(ZqvWY%ogXk)%OI^Dj*{ilvD?MiUT za?_A?$kFWXsUsT>PJ_gKSO)1docu;&8~7*}(&1(tYm*{FSE%j=CEV)y zwvd~|ZR&mp_~uo48hpdjM7vuK=e8!F6!1Fsw{)-jH(mBv9}XjVVQU_vw$DB_odO*Q z4O*TzM)7&0KrJLngnd7zJMoALhgCPNeZokY^R4>X4L5Ikt`*8H$#chG!1=iau4AVE`ZyU_%G+R| zr$7w1T=LssA!hO%03Rs|pMYrZiLCz!Ia*DgOcnRvg{Xs@q-z?~l(v~%lvR-B#sIqQ zNl&M65BK{1$}!60Y8PLQIe`NN5QV0PvX2GMYi>8xY$PAL;VX={%;Pt zu2`;1ge(QT`tDbV`MZ)6TF`e5)H(iv52Lr5duZSNF+y8#={b<66v zrrrD!mi+jb`Iyi$XwBT3k>~jzC-x=4TUc4%ZAN5{B#a|HZo7E>!^e~;Tn)9g8`#_R zhs)b0>fs-uwbaASd@e>vmXn)+R+=lt5)NRNj5O7ZQOmc|wyv$VoH66Pw|uMH=Fcij zzx6Q(ci}{MVs!a7_n1KsSe-I62>JV(Qep?YKWaU`bqnq#eF)Eo0gJYazELL7wZ0+O z@FFlIVOJA+CD*;NBVj2GHu5Dg4Jvq_I7C(Y_s0fE>0h!#S1D^l#fu@GLM|4St0;AU zx?qUHH#=K_@tGV(eu3eBa0YRya}MK|`G z<)&ZOU%%TKgYxa1g=ORpbqyDIcrc9GN{*h4KFLl?ZRjAL4pT`}c;2PFM@1B!i{2*M zRzkA+hnXxBb^l zJ8MEcLE=wzBJ3cpoIx7dw&kz2AKsG*67-tpk#*0U00Oe_g5e;b$xe(fnSN#wCsI%6 z<1>9IRuSCevS?<||Axyn#*=SQ3xl)ew&({&lTi%o=-MgDrr8ZZ^gc(jn*OD^r%aq^ zX;CQVBc{RV-`D)2I%SCl7d{HGM%A$@)k-VMOGR4N{joJ6y zNiiGV?(zpe6Gamq{G-AC(e0%b6CYEYy)YZ8WYbH5P{h(Fs}ch+-YSOr*^^caK&MX; zvGBalYnH3CpZ&oPpxFd|=@T+xQmJlgSXt#Zp`QA}r5Qkg8k)#A`~BY&A%W32r*-Z7 zy&3TrS&R5hV{-PuItbHy%eIRA!WOcWhK17c2F)T5zH%r4)sE0!m}!+@#Mc|ry!O~3 zzwbknN`BUy*wV!hhQ%oMag4m+u11TX9C);R{~~V@6dxduXGo4E<6%{m6>SE>s$`GF zH_A5Om?~7}z-MoM7F7>M2LkC70ZS>m0%MKzwo31EI1_)(O+9LslwPgMq7GO$qnYrZ zJFv`3`$fqvx0Zg16bJ@@gcd@_WsNqm|Z+nX7)B-+u5VQ_(_p&a^+_=8>lK_ z<*b_L$Z@(w#*%)5W&4)1SwQfITG7ujf48--tQVZ_SSAMZgk9$?QTGvc*BP6!RYz(? zmZEX2Jt`BIN+JV<7dZqTypCE$h$67vcD$4=Mh(5PjI;PKN1?pDD|5Efse)ppq$jBRXY$F*1VRa<$L(vp|tTbxYgweOQRu|fVjWo zKnUT_4nM~@t<`*RY64e1c-3op2+S{+oz#`at(=!n_KpRAT`Xf{o{tz*OP&8AH?|Nv zP?ZBgL;Xp+zmi65d@{-!bhk?@+SwHw)){&K{C?%vvRFl(eHoJyr!X23deRoWF#2Y4 zQr0r%owq90Qi_H9BcPBXP{)dZ7h8;1ZwE9A&J|t{JCBMk_I3oAXN=?G_9$Ok5-=(% z@;zabbxynxcFf6;{QCB2PZ~lLX{E!B6OlI9&6sd>SK8$a45>R3ddzLBG=VL>y=`h+aG~_P z*@fW*Py>j$y>uO@>n!6d0_t|kHkrJmg0H(?kTm(6c(xH}^4g^Ez<1A=oCfv+SXYkX zlYic94^Db7F`T@A;91t3-|FcHr0k7SV*a06+Q&AO0ZbCf_Eq-tL>UF3*0cKSb*B#q z&7oTZ)U6nYtI3_e-l3reMb$Yc-gmI0_f)MQQI9!F+8iE!sC-6RC0(3RM|-&BB)6j?4_>6`YTjU-)bD4Sy)CppX_-*7Hxm3&Erxd9 zGc9c)D*Ut>AZT`>LelVT|NgY!9ixdyJKl*rX$e$9kAH?2x;c;CHcu{0 zxH3q=q#Kn-MhXzoB;Bry===^}^J4-|WH)DG^KuxRUgu+nQ0(RZkFU3min49nz5ziA z2}z~96;Qf6hK3nB1SBM-C5A>yKpN>9V(2bO=^DCA>6R{m@ASUz>we$+dfw;zYgmi5 zSgbjZbFX9De*4NYfzDlF4;FQ6w8KX=-pOO6LZZp zmhScrIbmx~ETK5tMT|xbiZF~bDS)7Z;%_YgFi*cNMq5@FZ)}P_#KVC`QDm?RA%i6` zmTviDabq`I5>7EBlrdE_Ux214N==C~(XPBDfS-F^A42-9VA=0lKDvBl5Kx_k}sZ7qdc&9QVgFf!odAj4RO}T zic^*R6I-G2OO=O%cp7J^ku75ym!kBZ{-f;mrPDcg)*ie~_EOGJN=fN>=&t zDTID{-m5MVY02Sek0slX(s{%?RuVR6-OT)UHdnI8 z-vK)N+3L3zEqJx#6i4eD&({w06C+KIjFca7O$H>ETZ(m{jsmuEzJ zB)CimB$;!qkn^J5azl)Pq~Bvf@dtx&w$c1&Yejn%Sxr8D6T3b1zw!3F&C>g=3t^jI z+J)*Ib$@=GL3Jf?e>pX?g)|vfnvUnvoC9fFlI68EaQ-P;jiQO3%F5VN*5>NRFj;#^ zZoDF(Ml?T#{0yLg37f5|U{p(3Pni4elK*uC4j^*!CXc~ z;JAxm2|#chW~2)GO6|Np7Y_0<32_|enE{AfqH`_1ULQ=`fm2Wx0Q=Qe&g?>|fz`bJ zX<>=R%btvU=N{M%OlDW3Zq_WCdc8{OWxI~VFZl75LEoKj`V-C0FRh}FHox)5Xdo9m z@9$N=!|d!7bc+A!bUW!;#IsXN52+vF!E_aC>$06X1Z)5!Q@N9WGi$cP(Y?2XRvdw? zQ!2T|A(Wndb`=r}D?LlWp)s>)s-8bFP| zHfso5k^2o9Eh_l+FY!Je4eY$+g*;+lFE8~6DjT#5xvd5+_> z^yC?Vb}L^aqzGB;O-DRvRi~XX;e)({{4{BKb2i=~N#si+0dO`;;#qZNM3KbYxqk3x z9Z8&9l!dQX-HW{sD=rJIY8oJd9-Ur|)7sk8gW^#x0zn_1G0)Rr%ZAif-#{t09T0J}ee%%#qPnfcy0DAgN@aJS0$vmykpW+vu2TsLXNuh{>`i^4t}fNf%Op_i;Zk z-Z=rbA1sxSKi^RY4-ba9hyaEm3Mp{Kjh(w~2x0#2enDA7xw{ zB-bS&uJKlQ<%^9cKBSrmB?-=;}J zq&W5a-(8B#6P;Itk8ckL7mhX)Z@YvgTp0=#mVn>dPZp|AJ-A{LV*PR1VQ!uf9Tp_( zJ+DiDV+vfom9UQmxS&Cc>K0{jnK_$v(Nh;M1s#Xe6i;CP&4MDW=oYJ*1uLeWK7^A7)JlO1#sa<5d)^TY=c6H z0P~~Ct{(gQ4+_dc?!Q81vI5;NV7Ef=FtFw!atit{+&8er&~kh#(fl76)eLl(7o5Qi zMQ38IS&<2hZ+E|Ir+HZpxu7c`Go*IaAjRY4?Bd9eXkkog>k*}Tj}lVTOR9m}#8)sm zHdx>4Hj9tMSQGPdTFV95L1=YXm-u7-Mr9ts@hs<@U+h)A_?-mS3~(4z<>rQ;w}t3wJNNpRiuDJDBl)^lX#M zWLZtoYF%W>_IgvN@71BRVzJyHrkZ@^!UGx+nJ?bm5b6_soGBxrf%xr=OwWD3bm~o0 zq_0}KYDzPbZiB-u^%nK=dpzkE6cCFJC_9q8kbUMN38B5eGE2CQ8$u?3^HF|v@1*UsJf85<b zDV{xRv{9)Qdqh0uy8C`#(@}b3`y`;ayj}o!sVLPfN+4zWYNqYJ zwtwIZkny+!WW^yRl8~zZSnB?|!d@o!Uq)6^Zk)_qCFC!}Oj9-QBpLe>9zP9X!16Au zAgoiz7+^JizcjmCqcHttI3u{CFh#;@0B+U4s0VJcSRFIZE;4+AU51pZL+kGV)8t`^ zmmcdv+RMK&ui{AXR-bxw%iB`hRs4v=MP}R87w0*57dGYVhM0Ggs9$<$6QwI18rwu` zu>&v}f{6wsTMS^YN<`{>U7`xf779dVKC3y+C*fH<^^(knZCnX#KIgky4NL`Mntu+l z=d48sVSuU~lrrz7wsZ`kKLsTU)uKwHt$WQ6w?l4OaUJMCmyA z>EchQnE8&)Bn^W9T;IM=H_7}_S~V-cG@^!FM)NHk(iGil+KXh)oA3K%V7+2Shb+;N zI2=Is@+SS**%Kadn{_Sn!^E9!f{Ev!Q24_*5qRRmg(4^OZA;HqL`U!3&JfK%9gDw1 zpHc`;{^0=F1E7J+ix{l-W8JOt%Saa{99ysB4sAx&jD1l9)5spKBhAf8Gdk_kzNiEn zxn48f=qD@RG`DIbFN>hY?$Acwcu$djk58r3l>RVD#7<@4OCq_tQX4kRFEN({iYGD)Y4uJl&THDaV-gdynk^B z++O7;IsVz)0HG!tY;atI^Zw5vfdgt2pl4OrXs=4{O26okG7%!LtS_3(G2O-a48h^- zZc5>np(W|IOqC;IrZ+1k`r<&fQK%7w#EJDDIpt zCUA*zp(ml6JI22;2>hf%dGFI(c4H03_(BW@Mzx2}G>@E>HO_9nR^^^i9EM9f z_mY>Win{VIVKp6jRyd>i4v69u)Sgjk-67GwLrtu}Y^wMS)O=1ri5$C{h?bAy_Hm?u z9gTqTI^8j~Da z_$Z;>&FHIo<4L+9LTfplh`VA6>K?7D{!U|>tcY-6@eakgTu()etw?qoneds$)}~(5 z`GrY`*(CZ>t_Q9;2N#5Cvhqv9QR`<+$ z#1F3D#&)!2xRlgF%&VI)@Rp@Qaw|cfuinUzAw{r)FjH3&At2aGh?M_}a2@$BhGxq(#%LSNPKDT1wOtb%^euU>VUKaZ zc`}M>Pp|wUIsJ)O=qbHVJTA}n*|f8wjs%j;G=iD<@DN(DB4LM_5EygTM}Nd11i;dGI`9eh&ris$0YrBsqtr$MKmBKS z5>+ZsV%UF+1iTICM%ny;i9+c!aB#k)f~M9;?TG|5Kvf>KH=eH0CD>D`y(f#s9sszS zz1RrL1H|NIqakuII9=GUGGU#)LWGqQigT#$s1?c+S7a^SRhGo{V`Sd_-m5``x`jUk~0MYt4QQZrt2t*KS>txktCMH#&e#pDHEQo zzOCvdAoWBR4p5qEZ29GKD_kh6yTRP9UNbXsfSUK~MM5@``VUJi<4Rob5x4CLKFZCr zlR^8sjYON0W+s|W!jH0kp>fznijH15oWV_)%FkpmB{W67#Ls?cR#w&dKMxtuNu%q@ zzM~{6+?SA#kf^bwm_6>$t2lIJa4{U3Kf;u{o+K80gj zK@l%~My1r?GD{IxMp8Vo8Ty@S1fY1kMU4JB|a1lbWey)go`aB-GWQ-Dson%fAaTZ#QWfw7(P%ND;%qNev4;PUUwMpIH4 z33 zZY7(WAM()FQ?9{~^!CN;xX`E0St7=WiG0o0O3!mRdA`4T@wXRUR}^Khq*_Hv1g*o5 zfvV*C-(*i<{|mq&e$oKQSZaby3ew%2+OE1e$oFpmrGVMx*@PFxNt<5zA=P_Yrslf14e;QE5ssiABk+^i3754OWFb+!Ws3&trWjkJBLA$5h)vArr z;NErq6aq}(bW@;y|It#chJWbAjQQf))}kmm)QkGCR;wi(DGm;3yS&h?$B#$LgR>>w zphd2;=t1$Nc}8we7Z%njkCr~bT|JUmlIQh)_Aq|I(%P8*uH%aQWaVyjMBquHBpF(~ z=4dET3cjV+{_*M3Y>44;giawHcWWo<_M3da1TwlFEq{mfy(w+ugU8sU>&s{#7nRLn z5N-F{Ovhj$+awOY3{|L-;T(rz z>2Dcqlzj0l30{$G9zhW=I^QS>Hrrqw>;|j4E;pXfGNKXAdkS#KUWv=L9L=wHh*GFvfs z56+zJC&L_LveG9Pi-GYq`2c+vuc}{xwkwu7WfZOA2{Se3;jkaSlW*(&ea+hd736|= zR)Sb&uz7_aJ~H>K!6fwREKOFLy}(IR1zY7sQ%Wvi=N7{>MHM!Ep$jmAk0p*-YEcYi zWedtJX^MlZsGDQD+KJfVy0VIEfQv9YG;~TNbqjF^MO77$zbsMk9b|5G#Yf*StErD4 z0ArvPORHSJXB|0s`+r97%A-W{C@zvhXXT^urF#+4aF9u}Dw^U}h(|X&l7PdRlq>5{ znDaV0VW)qoiG1n6hyd1d<)vZqRgj7!Zg&gLr2&?P`keYvB_@MDFfE(Pd5A|L>}hWy z;3T=%Mq_QZ67u!Jtv@H~Iq@@&q(5hQw6v=E_)}@f!tHSQ^=IL0K`x+DC;=JZ_yzsg z6IWwte)XyV$H1Sr(N7AeZ|Ih9Iw1f?s0QlLXhhO^#U6{LNF(~$D@yspDo`>5Ipho$*iyk-_T#xrWl znNqAEOHwqXcX!T4b$7#0aHepyHXBLP&{LN={qL4|GPE5KZ>(zR!gP-tW>2Qj-xlcs zX_#|Db``M5mIn^}0%#po(;tVpdK*p=Vq(Si{~3AyM-<W~s@R`!S8W7*ABUA}jaV-wD-rMN}PLw$?!0YALFj6%Hk zNzJ>2tO|-?Xce^3K~k^q$=mQ#*_HWEy(>mc2S=ZtzF0J!ArZtT{WX$b-0anpv0|M` z5)faS&VCp&%^Dr}38{D%%4eQrd6bPBsj0>@f4xZ=kY6J_N{lOV9rt6k1H;iG<>}R`t(C3X5>`0dh^Q+3Ne!B{k8km zemY7^1$1g8o0$QMSKeO;faiLt&G-JN&gE))n-k#d+#5}Y-X7Gu3mrD(E&xiNB_XV` zWH9FjwE&XAQ;h{+McWd9QaW8tqw+Xdv0s91+Rjr}36295&KI73+Fym>WmiPIJB#)5?D>O{@cvbp|^5Z|-4?w6f|I^Bqf?B51B zeE(=)RKVBOHeeLKaZtbFhIQ%{8vSrGfcaMeDM;@Bej>k-t^C_K@+S}rqu8J2A@HvM z*@^g~dJ-m^($;HGKKC*rbCI5ve|c(4PRK}m7GG6wMg=aB#3vlk;Bv*R3uAD=a2*S0 z=Eap%eNG@VV>7c|jr|&9h4C$G{`k~gulfW!F`mGB5Y6FF=gvfx6eUTyN_u}dbJyfk zZv~Ya8I;ZbCyZb>GiI%G)5jrjbN)^m(b1t!4cq;|{O5r2SuUI9EDm(B&ibe@5~PHo zMDPuD&h(eId_rduy@NQW)F&|9qy;v6N{d0KJ1Dv$iKLZ{=hKN$E2+ux=Yqa znAe+d!EESL{vJ29yyIrXc=Y)%vCRRLEJ5@Bkfw&MwQFi_s+AiNajiI%ck(itro7d2 zjE;9Yv`4ipNRqhO@$lG|C;5J1hd?Ym3`1T4Cxp+Xm?$qHD`u~dn^R;G=~=Z3A|_i0 zkh5HTHFQ~tRBP0qQ}#DBAGI!s0A|5M0Ctz^1<KGQD&qN~+q{kI-?Lp<;q^eg-(r4KSZ&5CTuc-xX3Je%SuHg-I+DY^Eb>$sbU6$ zj0{W3wJUMPODz%i|I~S}>3#u}yv9Gkyd{8{c?bZ+TNeOv%V;4)v~C7tTb}h zp##_iJKVgA{OECmdamp+X$N!hKW=u=o^8K=3nKCoAKZ#(e2Y2oCT6xVQ?9%KYO$An~Ya7Hi*N?O`c@B(oNWU z%raywPfkey4cik9yqZOuEf{j*3MN^?*Lf69y z)+Pw3!B=4TqJw)HI(0r*I9q=WS!&nA?_TBje4rmE@@nL{)2HU0PMQ{+Pv3tB5<@Qv z{X7Z%@`H)3tH-Jny$hoe>T0;%lA2b57t9z}n7x#npB}s-L>QzxrKvX26a|nZ;rZ_p zQfz@Wu;IFk*@jOw#(mz0gdq5n@Ip8C;tnvBC0>TNCzSQhB!a+xaBnR?H+!3*YHObJ zFEEq+`Dr!SbFYlOX7$_3pj*h(3z?7x)$CuCDNS-)dOzNa%>M2MmvQozdSfU*E33NH zfo7(N+3Qa}>grw*$l&)tCe3NFLtz3`hynxWs#H1lPS%skmYA(?o6yIHkb;Yu%t zf3Iu&K(rO+MiU`2-El#jM}nwyZ&7X4M1WL`O!yWGC`sd}=tO`5~?h z+hzdyhXx|Q@oh{Q?y3Ai1UXX0$o2lCYpCN;X@T9J(IJqiC(qT77@(~Jvv!T0wo@*i z6Dz{mW;NbM|Iq9KiMm)Ci5U6^!hx1-t!v-7u?VkUaLh1kN zQ9dDR-<{-iE3&=}-HZ{g>(`+!1Q!wW0f~YZTv_Vgh)jE7=fkK}*ws!S&moXyrz@5(2+{vx`S{5Q ziYAh@niIZ38tY!qvWg`>CYMw;Q3h^M6~^35K^=n#^e)8Kl>2z-F3?fRxdw@1E7>N} z=`^^7B)L;hV@vwGphIrtyc=HW<)po;KPv!5>TbeRlMumleRSbvHLica;-ORw!fq#E zRvr`O$S3Z;@C&cWYiFE47~8~o8~f5PaWV>b0eb%OTOM?Wx8`#0?1%HD)XJ6dp1Sz} zpV;L03#<^MRau{-U7YWMsQD>i>M@U}epOnosPSX3Trzv0N$p&Rxvlp^iQ_)|BJsZn zcD>I)WQ#Dd;&6t|Ag`CDPG;e8i^Pvo#*A;hN3MQ+|1%hb+_IA-TQh4cIjhzpL=R;U zh9zjcMSvu0!Ou@>Pq72ZR+}VIvN;};XxD{-9fl`8WTevDO)d};4|z2YBcyenF+2Lv zFmu~ej^%o+ONSco=-czwx*2i&+pntoHdz)`X?8?d}5d>fB4#{C$R zKllPNQW+xZyf=4z>@^fXu;83|?pzk_JwncD)x?Z9_qP@xbfM(P;xKgS#`Q{o`r#=X z&;JR8e-Cp&sDk+Z0M4+%@TJuZ{Cyc+z~@)@Z)}8zKUoMm!Z;IJOu^B z>(Ik`NVdiCgSoQ57yuGshN~kLW>L%N zgi}{9hqSt#woKs`BQV~vhojo+E zBAs%{s7U7VUjhi~=5EU*Cs0#rTX3zb2EHVd9e5IzRcUz&AV`yQYmtW<gLUaMAPCY{CdczxDmlliSlRQB^7rWXpAT2C>&#VcGlj+0@wf$f7aX0XM3kE?ds!V*WC)tU1VU8vID8tfM!^febdhYq31GFj*@2@ebv)~n ztJpjxNyjMOL#=eyi^pw%elmwA|0;VFtb~kpI_SUJfU|J)%ft>-v*j+W{kMKr5x{?2 zILdwQ)BnrAoKdQIzryMG_7^?QY}evukMr_l0z6;^-Jw( zK<7IkQ`w&QYZF^Z3W;lL!9cwMCxm{Czo{NI)i`t-b9J-NU#P!lg_Q-*+BS)&+S!zK z#yBw56;i6B#Uvcu#q)ZL@~~jBOfb%%$ZIL@rn~WUC-c1OfdtFeU`);M1RbvNK8G{D zc$7bn%vn1$?7?VV$$VJ%#=6IK$oaCG?A$eu^2+yOnPYOCy#=wL+~c1??f)i{bns01 zv)McD*AkqcX7ig~1mn#y&b}t*?IOXbdqE?lqaaJ-#t+e9e5bT7{r+_rbyFw@Id%$0 zWvpX`5SS>D>pfma;)k`MZAol=J5NsOe(tWY4g` zVH@a5J;Dg#vvLC+*uRePIshpPS~B2A>X(z5GX{NNA~F=-qr+Tdw+1s1pJznA>7d18 zg`DMVZ(Rt8go?vv3}nZ&`2;NzoEqwzid`Ws%O_7+TEg!k{TyE>IyVpwyurE3a| z;LUZBWBZn2isorG(&{tp=2>Jf_tun`>wSKSdDAE;V2=H;?aTO{W26vKe`0s|?Z>FS z%4UuD7W-I@>5Nd@jV+lub%b&Y_R`p6OEw;vu2+_H?9-syA4z<A9#T5c z7k2!Y)AF|CnWqL@HjXRpsXk+rSm>z>V!)+;%q z__Q!}5e#_bZ*Ikw0@6C#l4lG(g-{VuU^0NbDdnNuJRkEWKN{Eg;;Ce6SOCb5pP3Bh5t2x|@f6sBP3c zL4wpprvuM9?WQ5nyCLU?+P3p|r{Be{6fj$DyzA0-79dS%|Mk58uTx;a?B9Z+shE0? zq2>OjF8Bpm>hbMd&-=vu`N@wh*oy|Op!9-^1?9yU^r`7$XRkZATyyBR7kAs`cRwQ{ zbrdwN^}CX)9xt7JtQ5$L4EO@I7%Mc@q<852{fo-mhAnA_-`zi-DBofyVm?xl$;t0I zf9gjMGpo9}xQ$Z0Y|jnnnqppjNcl<&cI)zytmIMw?N5XD;IWZAH9-igCZQlKw&^2Owv|`7%1^7JkwZ}+Kt5-Y@ z6drnW7rFJrjw=gzZd z^pFej15R%Hc1ee4imLC~Fc7-A9!rgv)rQlVam(uJq0aGBS3+JJgGz@Y8BYI?cl_Va zyv)CSIjG1;kn71P_eVoh6sqrncCl-Z9GoS2L6T_VVdbvW zpBoRzN-SHI-+WWd3u83}FUN-~{}>ZEU`(A>(RWKsF=v!L=UtyaX*tH-h%$S*?^PE! zfq;V2(05qqi!QdTg!xC)CS+#p2v%_?WG>m!m+-z`??qT?qmMXe zAm)0#mri+W>L=FbBPToH4bZYiC;yRB+EM9V!)G)}nKDMUds7IyulzU60v~UcelI@X zdQ}Bx5NVDPSlAYgVuw}oYiP?Ks<=?dg_1}xX=pPVmKN`x7qTT4-{2i7z#WSZSDzo8 zY0z7CEb3++fF`_ZtXuZ+mu#uj+vnvis{1TFl!a(hobN0#;IZ(!%M9GD82Ia3R*=Es zAlQ;e%%haQDt`BO&jJUQX3|x6N{dKPA^QI-*8l4q_(`l%cgD|`zKh;YeW_Qgo}WhO zZvBq9gO*pfx`ccF-7tP4udeHvbm7>w-5kRE(jr%RO?UG1Y~LNfoqDR)57Ke=&l-W} zyyTuMoDPGv8hrzL++|6DB>kOA44oTPF)&ARoK z?2JnkDeQHNi}zRuUIaaLBrT83oO}oqje=bKOq&-ixDRZ{Dt0axk|PaKFFk1->mB=8 zmB;WnT)g(kN@ha-3md`$xeu`v!<{y<@~M#9s}8G8i3+s>f- zQr}aLH_~Jjl`L}vu&&+laWVl@dE52vhxz!T90$9>mJ5Zvh_S~*%n3^R54LeH9rms1 zYVflr>%Vyb85#S>+^5-AnHbHb7cOs|KrZn0z^Ov)&#?Pa)E86@O92^t!eR{JDiFg`V|%9wKGlVd4~wUP{a7F7j#h+k2AE=6&e*n~qw=I>o9h}zh^q8ybsxs^0i-C-AJS78 zZe{B~rE~i`D>J;Rw{#NQs=iF7P7s-NMGAhEIn}+j6I&CJE6>Eq=ESBMUOKsbSZ(5) zG=X#sXp3vX3WVj+T8o>jSn+JlWo2^a?|PG{lbMJi4;I9;8;t)gp^LA6LA_v9DvB%H zAH!jGcg^pBdZ6Y0HtX6DGzLWCswNTHGlj?hBTMac~aLd^Z<< zgmXeFhMMORG^3v7y7ew1Rom#cn&XGtp$v?(wH!lOU4%=ta@6-&Nk0LZwNd)RqXSu^(+GRDQs zI@fnfL`B6LTK#H=q+#bT&b7_T$0>?4h;-qRD4RyDEtdfYjer_{Qk$aMN}22aTNK=@ z{S^?hsSJH}txISXH!i=bysKs>8QC15&}7h%B`c+$&WE+uwGl1t`ThSSbS?CKNU~Fo z-)F%jK5S##*LA6t*X!(Z3Mn_k#$J>PpkDTqczT%K*XW7eVJp}1gQpQ&ecCG0mg#iYQitPu%WVBU-k&vI(V zVkTa9jk}G=odyz!bA{rK=vJ$vin-?zmhbKsWm_nYz}_BcmzxQ$QnMk?z`BR2sN%V3 zVC*A{ST7@L%uxI z+LY-P6+j(k0MLrz%t+J7XI7R1I7eGq>u^_&R#cP2!q;H0^iPG_FsRSg-BwYyD|jpi z++(QCHd9T9d7*J@XtsM{y>u%0&*tqv#k~J5{w%=C7}+N^hcAH_1|)oc+}@sOeip=1 zzDeih^*T|HkB(Hc41{bIev8tr!D`9Kee24rVXUN~ibQsIn0a>tq=4J*B>FE;a4Bu+ z@PY|z%3TyM4W(}d+akG%*T8qNbU*vrqG#ew_je`=I6gwirfzpnX?EXhJ}<(yYkwG{ zYFV(a8Rpelip<@1ul4xaOOmR5vt;|7MJ`%x?rX!;!k4|fR}MLKWIBd(CwK(T0fKV0 zCM98Z)-CLccNZ)7Ma<@72a2tXrcX=7FRsOYdGS7+Kb)+zhO$qosONP>tRUVMF~?0T z$aKNSCgtYfKCl_hh1d#OQmt~1G2@nex;(fpX-5XTE3*mrN-oty^h3wyuU~C+iz~2UOlI7JJl&3oNFJQKed$h7 zYhzEZxKRg>4`|sNd0|Y(voE?fh~iays4rk`?3^&SyYfarYS6Q|#U}?UTH=wCl16>r4OT z%rRpxT-^4ol9Lu1N<>8AU^nXd0*cLkZo zSOmJedc1?4D09*GFkjzOxB35lWpQ*rbxV@sFI@WXJ=FPKvxftPv zOWo3M)3T;cue{t1pBP)(1NUy8X5&W|>d8*hYzd_|O->@kqFaK)&sHt)GxT5aH$aDv zYALi5nE45aoH4;I9G|_8c+gKrYQbA%zf5^ow3?Z#^nQX)Q4^n*(5{4OXv5(|o6J+dGlY(#$iaiIU@5PJH`>&UZMTrO zOFx@!WUv46jE`Z>c>HD#Q9)J7vpwdnPMcV^1#Sz{Jth2DE5cr-fPt!bTA^-)$a^@+ zv4Wj+SCM=sA8SFCZb~dQ5k8pVJ@n4^j*7SK%>CiVW|Hp@DqEkOqx--l`?(Mxlj_{2 zWejk59*$@%L}oPF!4$|&`5jB;z|OiS)-Gvx$Xsw7pN^T75gFy%KOwO@xj|4 zDMS1qBAE>GMMhIx5w^8br0q-(x9+x-cL|w3J1K)PNR}rSY@$6a`^nDk;ahK`JaZ~X zp4TF>)P1#413wa%2|HW7w>)*OgjXjdTW(bA)I?7 zy?jDb+hu@Qul`2)uR)xvxe9)_7HZAhIauNC*XriOg$qg2l+q-2YOV3Ns!juTP3p;> z@2M$MoHnqq640hd%^R}%Q!v(3nXNPgaUPq$Wlaouk=VNifKPi(Zz((_W z#s}s6XJx?+ye`!QYgxzfa04v=li{mME^w~1X$ji~pI@K) zu3p+s+-&6DG45X8-ybc2v{r6!eGzr#jou6j8y;d8d~o#r>SihJAEpp z+&mHE$(h-p=mN_lvD;cZ!hR;-BdX}JVigJ?WoRCefPJXN9A-Kk?t!xAd&q)y=I@q= zHJJrRRjsdAiO2HG;?^3Mc36AsCc-ycTSSX(Dk8-Nz3K!_O*l(w+hqi5wfm#T3Rpdh z2Lx=ycAbv^2XCp?7wd^r$dUF(ihqxA{8QR_d`+q;MHv1ux;NnP55&~;9}brtGZDsr zWHC{>gnnn)b}^p6SvJA9=r^5TiUT8s<0K5TgozUOdls}@-vS!sPdW4zW)42u-}$HZ z4oJ~o(Y#p<;GSs4@!Xk_b}r#H>dU5t zY20|a_ojA+J;I(&yjE<>XVX=bZ6@k~m_BfhheUlR;Hz|zeyb%vB0Lc>;3;%*hiGg? z{8;8&YmM|{oU?MvvwW|6(H>V(B`P_9r4BMAgMM$gyqDX^#gRd2yr3)i6mBVj-?wJNa~j&)h_>wmlpyUaL*&hcPE`$Giwe(&%TUvF&Qi^(m8{9w@Ph zfk|qlC@i+@<;E(d^(n&n=xN>er+yb^fD#8zE2|FQgn;bBeNbntvBBrv*9qXnuRVhB zvJOM@q8Odz-q+2Ci%uekkax8Sdvnif)3~!sITI#M4R7CkI+pnB@+3-v+PCs@S}a=d zvBUg7Y9cY8N|)5y zYH?X)pSixWVm+Fga>kQB$L_ zV^*VK)P6l0j>@nz7j{!93eqWh-q1M;>LRZ-jAR*P#V-e(k=tM8NHh;U4_~36X zz-u=3%&7jF@ZH$*bg7F$B5jhqmQvL<2K-jV_NNbmWO?!T zJ|C4s>+wad9pQKVTT_MWPXT{N9SV#uyKZtH(jNbJW3eiUP7X=NCt%V8OT!CkRk2?v z28FjgubNXt4@fj-~Kc~k+uGqM`zPMV*UEt1YytVwX^Rg%*)N;^E0%{8I@F{ zV=X$LF0ZyBsTkG!>!Xj()wDnOog-?yes>H-lqD2%+eFg$G?lTJ4=C9*96W^quC zE~O$^p>vxsn~)nOV5-oXe?e5r9Nm(V%bEf?<51)mX@nhkk%jd)$_d2GI=z_Eja1!R zdA0za31=sonO5r3`|7(T7}*G4&tZp@E~>(9#5QNckhTUifIzLoXD2Ai>zd9M{`%j? zI9gm-ySQ!b=ir;;jpyZVpnnIA|Ag&!0X+UD&MBXUJ#lpZUC}XKxSI;i=HgxOzebP< zvFMu0f=4ebRjG@B_k^F+U*nU2o~r2kZJo>U=eL#fLdXNTygMZf1)nnzs`txc%eF>$ z_x2B#v0yJr>DhT*6fxG0@7%=C)J{AfiZZ>vMRM~Ha7~hFvuo(ev4%YpEUKrkbI7U~x%;^g6jN80a+_QX3UHq3 z8<=I2V7t-k*Racj8NOqw<%vr@Ntym8X!C(FuvFS)SLQAyaxA?#VUK%I1EQxqrLO=v zSZu0_A4!D`JI{bQA)uZsFP<9|&%KGdSBFe;&`GyR;L8n=A=gkjY-oT}y#A6THO7h9QE3OGyosPo+4 zgd;N6QeYco$cE3dv<3vnKc$wbLUB7JP`$(nhfpnnf* z>{G?>OY{(vV=FGeq0LcS;#HT${l`1zjge-MjLCiT|B-d&@lY<_JFdF8E!H-K8N$q< zl6{RC%Mclc#-8jV#uiy3iLoy;vKJ<1Y}t*yn`9RavZcjXMs%|kqTg$|_kMrf|Add; zGiQI!bIyCySPPy^7gSt*`sv9+{6)S?D3clA;;)%smY+JZ)xj8RA=!N3Kc^_A;pSN@ zURm)&!7sgJR7d>R*TrVMPnItgF^l5ACz@ly-*zWNK7;p27I9~d)?{_BS888|R+zYZ z4F?nagpSIy7hP^f{^hZm{^mWYZ>R&6*IbGeKO4RKQJmEh4yA3(A#AZrvVzCZ?A>9m zK77U%DH&s>rUufQB1n@9ouI)vGfx`Gl-#w#KJrtg^bO8m(V@g|R>82zdTS=R*Wd@K zwkD%)tjF!mio9#g-~PhfsCNj1<7W6p`WoiU*b76p@x7O2&fRvsa>?xPUbi$}O;UX7 z^Y|Db6br>(!=f-(Q9jX)8cawq_QR_{g<_EHp7te5OSsUC775VbH|;s9*iO`dcm*w@%`QsDO!+Iv>23b8~0z>{HoDAV++P3b2&e6-gbNQ*UJd^iJL`pK?QX`-yAsQ`w5of z4wQU=gtJF~ki$Ix|8vztq4EsjVkXT0ezWyC%k-Sc*!wR*Ja^FqESy=yHugwPH#*Tn zU7gjb^0HTH_=hgDqFlY~w}z^fRfdSJu3}AFG=az^Uw|Xyx306;Nc7_YnPPU(?9mml zq3E`03%kDW72&JTwfgh2V%we|C!#KrZVye=W|A)|-0ZwnZQ?EGeA;o~rHQbR<5ewH zOVR-QUs%-|`_s?`(7g*j zmMfi3{|0glM~CwUXbl?|&VfY;L!CWJuj7pzd-M5B$+#M+_rUu%$$r3Fnh5Dpm!Ry~ zWNLtaBanlH({@8A4Sp=$vmY1`57~D*`+hKHj~1gG8wYlc7eP??3LjAVMV#N*)E+4uGA4)r%LV#=IrMGJH7+i9TkqG z<&pJzs@uUB;;GR>%jCEVZ>;#xLWQFIj8tD0+?*x-VO;jTnhMPT)F{wBt@Ts%Z^F<* z=W8wgDrRc6(D$0IvPRHx7Z={WFLlQMfkDpJhH=e*zw=RjzX10IC*-|dWA3+ zE9v-4ik0(YWHy>r(O5n4k26`Ygj$(3=WOTj-?pHfF90Q{Tu@YJN@vlPXT*bLNR4A9 zLK(cu4qW0oL4Nin_(FBJ>2!!Ky2A`@Cn>^e+M)E^`Q%|pzoqc=5^ZtA+uZ&n7b$wD zE4;;(SYxv81vaU6HT_({_Qrf%P^<|^tap3m!YprFnWBc*+NF5%pusP{^WKmgE1Cw? zS}4|Xe~WAr1}F$QPI)o0O9mI?`+P3dFR3Rad2g=Hzj50X82oD4c|YmKUb1K%V6Njx zF<)}xD-irU2L5e>Z+RHPNlj1kLI3_B^vs^5o0M1Mu{+4pQDIiJ@D+I!zWvS9G~mD2 z3dST>o8+=0#?eBVu+W}1p4h2*SN;11UgV7!AC!&hpJr31iZWk&a1Z@;WpTwODE) zlD8nSlt(sjzN+);mDj0#RXo!d$jd)g0C5yRS+)zrqu-<7n2)v4Zq0ozrpmxRd~KXx?^!JdrT8jy+uqe@VOm zV!VOIxC}pZ8c#GcZjUmVK`K8F<{A|(AQr)M2R2K?FJ`48iXoHM)jN|ZK26LPXyZ~0 zYpd^%xM!#ALFp4tmJ}M6pDbNcff+8cBSCU86FjysyoGdwRzlpDF#a&_{m#5EOpQPU zA?}pj>_9nQwDe~D$K>{vcZEejkB>_dukkPM&W^FEDsZgu5g%TNW7edi!$lt7&mIAx zbR=3E7eHMfh;oX(g|HxfOxb<0#yKuZ`qq_zTmt@ad@Oy5{g=G9@1Cz|yFQMr^j$Q~s7j`F^apQ}UNXrwB7?;cIN`%q+E*dogyg(VT)<##R1T9IoV6%+X z?IrngP|K5(lV6`5#6Gph0txn{(G^Y%j`#~r@VyMPfZk-gk`(eEEAu$XWSa~E>5+^o zZ~O8i3F$g)3}oDelu(ljWU0~oGI`yrEf;r1_#1p2A2}eA;G!pA5zl#}U{On$H5Q~; zy{7%y(~g9c!v#RIDd$WdVeE^RG3w+^D;B4rM*(`^vq@GN2%6aqHhV|0C9IuD_4Z5I z&02}tiJ(%WEJdXGT%2Mz3UC_EdhJ(!C*v{kz`kTw9~vZGJAcGR2a|107U6JT3n zgx~%RgMeSH4le6t(-CQ8p~N}wlOmef2VFCBNH7bC?tM7f;x>1Ar&|b`fGb4NyJ3IE zB}c&Gt+B7PKX+9`4!-Y2%~VGhA)`#A%nYrmoT~lpOQl3u=-Qnb3Mp42Rs!M}NLf0e z5^zrbR!t4`iB1qHH=d+dmYw!&7O64NJd6YD9&Q-$NvEmeOG;<)3sfER>ntuC3Mfp; z)ma4=jEnbVV%!&}c*xnXz5D|OC;U8;h9F?ytXZBp7x8NW;Q-j#!&_-iq>2wFG)TRQYuig6kqH9Ym zB!3yq-obs%_#rfUx?9Pu`<9-1afF9xxpmBs@gUyEuT zCfAZ)75Esj_=1mMoXBz|U8tq&VHy@8mBuOVta`N;RZV9{ug>-kSA$hc;8D>F`g-_M zugg>UpP>BWTnEqY&C-@Ct!e1y!;wh5uJk_-zWb$n@}WPeZheY*5&xuF5#a<`2_eR| zBbvfP)B0L)0y4rg4_X4hg;E8bjmtRl4A)l0ffi-rTDvI|YSx@YlZM@zz#B`uX#SqVIABijNsw!B%_u4tAS> zkp}QLT0K8-0Q*)ZqKP)7@H&f*MQ%DKCRZ@YnwuJ$3QHr#m)26j8NDD~`A>Z- zC?Yjx>(M4D4>zo-ppu=xuZvJ=Mt!PiL62Ec%D+lD_>t~zO{Y_dus-NL+3vM+vokXY zMC4Zmwru@z+x=zeX!&gn(oJ6MMXrHhi?OY#umBX^4nB~uup0)3i(d$gSgtiC9RlPiYw(pQDztze&Fs)AWx~HI-chW` z_S&Z8=1VQf5FLE7>TZtQ8TStK{1W*6uMaazOG~(H)}x0HGABP{95XFt)ZljE!FT?p zaQEQ>Go$n!-QEiodx|vGw@D$~midf48Ra#Jg?!0B>bg(@gj}_*HJxIQ-1GJTG>a3~$tAZS1Ci}i${Qpp>E*Q{=vy^7YzZ$`_MHY$y*HdR?!f?kCX@mVu zFA@#Btxm$4zqj)ec*=FE>lU>rBdQIfJ!95U=_w1vb^;<5`qxP9)b(Qcy^3SnG@;g|4&4%Xb!bU^P*ZzqW?=LaKQ%G|qc49h4i-w&bN?^W_ z7n}(ZHeg7+Q`DC0q}nw2_j{L9n?p-Np2mU~rd49vn+IR{Hj}ll9q(6~ zHT((V&Va5N4+K?sqltw)miEw3e_+VS4<;;E>D}6gqJ!hwlMk2s>iP2-&tJ-Nx>f>{ z<{aQ9iGA~DHrEC%j3iiy|E-AD_dA>B{o!rNq_rnbc!PEYGz-ztO(R zSEP*1>Dx}uos&Hul5gzaz=HtzoWWaEezH7 zFR0%v8+J+g**@O?fTaCG3(!=sz@^K3nyNb$`W1SGm75@9i1|3>1w6h^q#NlgVM~VzHLJ&|5aX)gg zrT6fN$B?rL&?^+0ZFAArvek=aPml!GY<;H3g(EuVGn#7TT>gVx1T1nRq6aC00Bt7$ zf`M#)s2a7>lJbRC82^=y(8mfF28t_=1QXi15^s5zby56Unarh@@Lz6xj`t z-!!6#3YyLlee>D+)(6k{3Y1_ksw_F)-@X*SjP z=^!Cw{t;@JLV{e%138h2}`3pMm(wKSFc$XRRdCZL}E82HfHU?&~%8Y}eJ<~=;N zYh+Ti!iaim0i)UvazJb(M3Y^|KG5LF-M2E=Y9YGS3!*}UFun58yAlSl&3#`gIDf74 z+R#G5lIr6B)Is~Lp8qbS67T=`6q}rAC`^|m7*4wX@hNscA0(|)e;ySUbnNgBNb)1ANjTSvjKeUcmx3_ z=%K>&@qYdUSjM8yF?fz;&kmJD5@cn=qJVXh6fs#2eSm;~DDj*=wdP9H7va7hGZcH8 zwl4L}M@ZLxLj#Z*hp;xaBVm{xPwe1e=ryU~hntE?L{NH)6ycpTy-x$)gHt#zt{WSk zJu3YP-sv)HwL-p`!ZBdECFqG}ugu*(ttD(qV>VEzeKi@&g$Ku@*& zQ}=Tlj=AlK_EFL)?tw&J@8Dgbj$?^wFkQ5d0;$zDb3a|tOS)z*_JjUfC8UC5wJ!}^ z6`p6zLA(9D@#P&Tw%_!=V3EGnE6NLDhGKRa;uszG7J zIJL^is~1-&W#AiP-AW3#Pi}~o6(Y}i%9nJ&i@bz1^Qgg?_5DQETpZIMwUb|H2Hp_a z9OWE!5SyMThi~iCmHZYRNu8JwH1*ls&E{;ZU7=EHUr;{n zeyWL%nxP{;Lb6slsjEVO7DM!7fRiZ@Gio$t%cC-_nSod7zK#71d{ty7-9z!zxl;E>JLan^ThBz}M10$>c zK=KD74;MpkCZb=_gZC@k9*rwrKcyFo6c02 z;qkfeFkC{lZusuSc+er5PmO{W;2%=Dl6}_$TqoyssU?(<+3l@v)IdD4o?cRmVG?(Q z`7f<~Nn@JRj4?}W&AsYboL>$zCI^18`tXjYLVKndpJ`UXamca4IQZweFQXetFD_;) zRP6`AS-rbwKupLkVqKHr-!KedODQ$!Z!46Ix25-0ESJuUJ(Hu6}TP|5`0VX*`u9Dqh z=#2z|Cd%}s;>@#HbrVYwc{6jgiCa(Z6wO$jL(7NF(#CZ_>2EFoo%sZB`d$hOtrSjZ z(H=;SOeFUC(&tY9qQmH29kRcH0|t5i@rn3sm7x00sb}3&&3{~hxOAviGQ8;8M{T@> z0gENhzMb<3&ypJf0KVZ8sc1ex#({O@UfKI*L}>=HZ0PX$8CI^kA_e*w?aZC`E)B-b z5fAOGcD?l=bhBs~e_2-d;cR|OwEPzqB!+F=9=4%CvLo?c)bYBI{Mdp8HWt77Fpm<8 zMdLbw;eyCvr>ZTWO}Tt=&FB2C16t&0uKuy-fciQD1!Un0s+eO~%`erig$PjXE@YZ& z`e-oOdeMO5X?)Jm4bq$$W{}ZeK2RL;N?tZR`pG_qbC}&JPp>x@T(2G;@ufo&(489T zRwSD~M%Fe3Ecnvf#6HX0zKQtZSyy6p^LNdVP+9>}bJt^3%i)z;jiy)ggGkFj#jaHCQ(JN2%X!Gm8yzk37_U3> ze`_*fET?erv(b7_cA^y2({VZAF0MbR)upWMyPLA_6_|_RY1r_?hMtH_SXhkE)h#BLOI^VzH4v*uv)w>pP=k$v))b%U8=zAy+5K_fOZZ zpSx&l;7wY%<8YC6zkJ{iy(6>Ud?7I>Be1kub%y$Dnm)M;|9vc$nkf=wsk!(k`4zW71A0~4=CVPAMb69M& zN4@1*Qvm48eE7hmE&4O8(I2H9aQy~<39X8hF1z`ciDqMev@h!7kC$5h1}+yFPJ)Z^ zf#+Sg#9U7*cUj)`-;Q2{ac4J_V#g~z5#xfAFZ;pA(^A_g@2by+ef+H8ZclP4vBQv+ zYyIB%yydi{L(Z8V-3QIr;Wp>tg1U<=mxlg{mIryEm2l`!u0I$UPw5G$h_Zeq#YzaD zfoMI(0j~;tY5?N?ecwIbTd)`#K6POC8M}x!yqO!QeX?@Z7LozhrPvqEsQ~!0o%q@l zjs}a3e0#JGn6fzf$iyzUuoCI|NVwdP>*+*@mW%VNFI}j#FWTX_`pG#1v~rV=lCJDr zoN!qPP>r!lwp;4l>9eq3FTG5lJ@b1vbt_;b$BVDl3sr)5G97V1%S?eCMzpsc{GRnH z^=#i-6uf1x+<*U?DS>$W>Z8qHTux1ZQ|*zhvfBGF7CQ!rjOp@uU7k?#r1#PTnHC$~ zg9$@viW-!wZN(LUewhS5c;!By#%}rvU!nNavuD_*Lj~;+Go8AxZqV{iTDVO`9mx-l z5R&02s_6KrE#B1Kp15Gro{Gq*X@YUjS7YdP0~0Zu3vhvj-fg{bEb+V}peM?^G(emS zov5Y-$v@ES43>10HmiJXHI%-TCy zesM7Y8=wrnBw)zIIExgJxel^5UNs_p_pV|uWP94@bAJBR-k&gL8lwhUZ!W*+{qocA zOMRmKSiMEL%c=5Cpm8eHp?e9d(LB-W+5`hlz|(s8o_Bmmu?H1wyC@Be*j!E~s@r+a zHqqYa-?_EX2a4&@=TSc-)V(F$XrjTPQ@JGtS2`Kly5c(sif+$;c0e(|v{F=QPkYcmoItI?@o& zb+bgzxY|0lU~x*j)%gt({qgGUEJedola0Py)d!!AqXNK*ivUpMbZnX$jA!EBMpImN zq))J4?E1jAGQ0ptHRfaukhrQ-XJ6|K%}s!*BfFqg9#YKgooS1@H_0n_{k80VVFnAm z6F-G}0iaeeKbJlA9z>TtP4^zTfkOk;nec1(v#zJzssM9c{=)uL!u!v8nLU%el_y4* zC&VXyU+9n|I|AMq9y6E0GwDdwFDQoQ*hL&K;+=F(kr3||0h*i6OMQ2!23oCAo(=5x z#=1LicrasJQcNT^Ws`}T*yL{Mx4;%M=M1Pi2dYbQM}#Z@OPHTEG_i7xHiJ43XJaUS zC8mtVgh7X8^XxL@4_v_5914G^3w+SQ0L<|UF1%`kI-$OO)Ztg^cwan_J5%cX2~Ki61Ge`CH! z+Sd~Gu|nLjp!{A?0p+Iitj&4#!&6A6E|g{U`cN>HXXWCj3%XzDEO`8Pa4y1e8v z`sZ3+sEh$i>+v;-H&(8sf0pv^cztYNU1vjoG^>QLFkc$&-9#?QaL57_ zD)XnNK>+T=BTW!GrlEEqFiLMuYENSm$3rnyaMNtIlC(QX&gaw#U1EI@2Hl`2X4Sk3 zL2r0VQHKg-wgN{DlVUk<2ddpJ^x`Ukd^7lYuDaS=uOO880rychyognZ`meI{S)=B{cjP^e~-5>4C zNijB21KG2P$Z~ECPY{%Ji}}pvH@&5S1?ZnR@X@^mRC(J0Sc#cp?C~OPak*sTg!UDG zCA5nyz64$}#H_aX-SS2c{_gVGxe`3=Tj6KzV4k(^L?SuT(*L-SOmD7cNsQuf)SKo8 ztCc%G`UY)exO!|YJ`chG(KVCWC}~QS8;P_d@o#(ucuv8zhkq9_ z1DHA9?_jo=7;_pg=|V1AXBdl8(rg)5ksC2=klB+Q09q_Wy>cJ-!pj3os;vWFzYX{@ z^d;ziV;v^zKCS#f3X!k)PTBly?Rm)0t$0vk6z{TyD=2Ialr_VsQC+Z9gWo>hQc z(e_>qxVOfgkmq(e6~+W>^19`^q5FB=2OTJu3wb{8FGYSu12ButgF6K~J#dO)i)Kqd z`)gi?{;0Fnpbv*bL~O5H^?zyn;u*Q2SOX*a7O5zV1ZB&{{@hIjMB%8q%JDq?Z$HsG z{S4ExZ6qBSe${M^_bh@7{o7CImoWkk*d9gL z-2dN&0c)1+Ru*C8Kq)^$m#Ye z+y?yBpc|#F;2Iw;Lw+nXE{t3gB9R@o#q%}15>CwIgI+D{%H}JQp!%@uPme&cXPX~i zEQx#lf7`%vz{VwyngdO|qt9#-4J6EWcOLI2!XKgm} z1pB32PJL>+&7zGqA`&05=<3LmcvBy z%}V;dv|`g?$aF&NQfwOuSkL~<^^&e~bH-vfR@vzs@O^Yeij+Z~v1cV0 zcIo;UKrR-q+ZBt7d1F|%$FlaC&y+c98sk6g5Ra)gS>{#v7KIBMxBBs8cZNEcq?EsE zG-WSQ5DvdSIkFbQ_v)hzM*;PB;-TsN@-s(m2c^**57yl{D)yh#7Y}zw3D=vc$3yRO zI?3KEI(GlN8ae54MK~JY^i81f@1lu`vBc)bE>|IqD9~+*>rltPHP+#D%i=&g4B&GE zhl2IF0UBH|IR+Xkf3s^jnqF!rUZ6i@6lEqqh!SpHB|I9B=XNeU)hRmWJL@bX3-;_| ze>k=#vLQNM@-Fj%Xg+qRv9u~@q#IU&KA8pVDZL5u7{8TWaL6Iu_p#vmDoe{M{*0CN%Xcen2vq#1Ri+$2p~T0h1n zp>{)1X@=TF_ll#@MlgAGbaVOKK~I06yWI%%J_V?d^mTG-1TP4_-mbn+z5pW#JXFD%gWu2qqw80n;a@&Mny}WSy%McRd4y|;!?abvT+xe;8)YAJ zW@|0Ymmb}%Fa~ho7P1Pi7q)2*RdhJBuoeK%wALecSg241fyq5 z&p=Ni(cM${g2P^<+_&f5FJ}#acIwA$89pn^$!#?f^<9E)gxr)7m(NC1v?%KFHs8CI@Im_TTx0PbkPx`z5NKW&}G;i5M~6Y zr^iz4XDkf-E#+L>5{VV?lZ?Z%@6K~iNyKOT_@=`x?w7jx&FA<>YO^*9;6?)-$g=V! zmAwum?LQqz>3zUNeR{xobs)yP6Yr(rZoJdC=hh;&yP*9oy;j5@PH#@4!-lv89Ypt1 ztH*ClAN>oTtY>Ex*%}E`3$un4puhkVZJ{?0D9tGQL7Ji66Bsiv{5NuR$PbCn+bZ&fC4L7o@X#sT9AUQf~fCVFb&W%Dw!QLcrsAIYkATP7&C zZxK<0lYaa&-cGJJXVav-$8CCDOH6L1`h$gt+iKxE>68%<=Ia0*uF0<@bi7=;O*fIa zY{OPiOd_%YQfn{;*a8*pG4VqY|8i9KTW7}IiS7IR0osD_Yoy*qRHuPHxk+wn^pW$p z!t8BN>n|}bYHhHt+~L){2Ci zJF5;yZ%Ucp_it?SzN4R^?0|M@0yZ17sUyjixd0cPBc85%^kMz)R%T6t6orsind5KS z3i{kM=XQ0<%BcKxjl=$}(X+h$tp2a9vdQrQP{*6{0(<60_~UH>9iFC5N%&K3$;+G> zyd=n3H`ydfPp>fH17PwIyOV7Br!!`q+=d0oAE90KKIA~OjFzfdA~Ar}Yo;QYACs#H zU?1x2B~DyoqI6$+jOW5nu%Sq|a|1B$I7&@D^U&tW&u^e>_BzgrrTuHBpS0^UN+h*- zal;c%HiBz%92O_n9R_Nb0|r{47|2Y4Y>R$)O*XKgg%6AaQ_qoc88P_o~QKVCgkp={%^EZ|#TnWy_xKaM_x`6%uD_c&$)0M3nN@j0s7+-4d( zkz`!1VmHV>h4e!$mM|NU8<3_|wG60z-i6wGIFq_u+|2IDnA=9HXpYx77%l%^^g5KH zK?7|j0+3P#uB#!a1fEa`HNbbzHxJ-UQ1xqsq;3JN>tq137dGc+xCQtrLERl;(uJLb zy|n5;#WB?m22&d>75U>1SGfdFR*%*sg0*^wAAsC@XQcW{a$92U#4HOIn~q?*TIOF( z35_Iy0tvR;OG|yfYv}A6+bMN{rB48#@6rwLgQfG`NV*?XtpK3T_M+PRHm2nnWMCaN z`0e!5vFQ4V-Uyf;%Wq;D`i-ZZ&g~_ib$y%ENF1`{B(;Ju!%i} zhzriIw=C~wCt%ZnDC_&V^v>mI0zQvR-mTUG%C1NAOC)OEjP~{?`<<7L&5eo;Zc+HD zPj6Eyqkqv0i(UhYg3ay)O1+iCrA)BX+OWTma`min5cQGcz|`1!MGQbwAV+_r?!0)e zqqsEUn=O3zV@d#$t70WnG5{Dwd}kW#FH0Z>`lkHPrpBIIhOX=mq0X(SNxCPfbzEJP zeLX&hQvbwkR-e5rkLH)^Klf_R(dvcgV$ecr_AR5Y zwBLZUr#H$sAH-LW^%q_KNcHdiy$Kk3E@=r&y;*HG(^BH}Klv_R-0?qH!B?Qi6GL%Rj&uAu=2e>d9bqwb8u`FLU?H9&^w&9V2ij(S91IpwQ1=gtf~- znnX3racG}(aH!CWyhLum!KR1jJ1(TfCQFoLJp`m=F9qg?y)({LahfgGSwGnbWNMVZ z6_iCH*8#mC*+2R_zj97u;=Plf7Kxv|@-Wp46FOJY`aMhMF=oqI5!;5YF2e>&!UYMELqpgeoETxO;?#O{>TO>wHR zIsO9iDyxn~Z4Jh=*q_L^pDg$pF(>=_n;BkE^=_a%S(fkfL0NutoASykS^aZsZf)zw-0hsTbcvOtZKUo@*??oA<|1qZ zy)UoiUZ|j!lrgvex~*8ToBq~+3~WI`7dWyfZd<)N@)y#qNluF&JFhAeQ0jE$BG;Zo zr#U)+Gc@6e%olZsPU7Wr-IFDd@LUKIE$K5(_)^=B9YFF|zMbws_SIwU#bUvyUv8d5hMt962G!P440NSQk$~LpO2_`X1qrqeo~7EpP2joRpsNVdV`P8$wBZ0eB5fFhg zWka(squ1Wz@x}(ieuX`_Vh);PuHLF@!PV$0x5c-%u+nGK>q9z=Cqc;7PWGYYx4cIN z;vs($=&??E$_%0f0+~^kd2?D)o{q$L3V)@O&Bd}frc}f;Ojo-Nm#dCXJiM{zNoqwu zhF{qMFO(`=l2}J53#FfRE?b)@$eyTJe(-><;sq^h2>CR@dLq{AyuJ?db*j1C-oquS z?Hrv4h{cG@-l)NBtclZq)+Cm8F@BX*xl@dHL2OgcmV!E_;c>VXQ!t!Xzjm9C+30S% zF?pYwVUIbR5 zA*TDD<06d_A8}6FQeRzwyl{T0&GoFcbO96?F47BgeQ8y9&|Br%)*(KXhMR4M;pIJo zGva09enD00Cc+R09o@yg&C?6+gd-J}j{fJyw<5g0$Sv@EU{Um2nE&oD=lC(DFFC(; z7r=S{mT7Em_N)IOefHX`CE3Ma2s1zkbpYCl>-=nToQz}l=7U_{M)7Lo{2uf-7od7U zwyN?1QGxniKf>JivL>*@Shf6O?#vucf&Lc`MqI`h=Uzfz&OAh87ISaQ$18a8 zTN=8a!nm9qEnF;nQS*_GGi{R38RA=TJ}Cx~+=;v=ty`WtYc*@ak(pt2u^Wk|R`JRg zXIX2peMZa!g+Q$ICb#;?tz#ynOXi&=i$;x-@f##~_x-6hiKF5W44qXZP4E}->I`@JKnh=vZe3!oOUz??DsfX~UU=U?`9pm*>hVuASFJ8W zOp9@ah&*lOv_a7!bKSv2qIVqUMX`9TBouzQFYyNUr*VPQdJX@Oyu3@Yxv$Vj9sA>e z3hJ#sY?D;ENc*|9ew4^LRM;P1NgNE7z(OV)x+E{VW7@)iVvXj%xefT(W{0bYuC07* z0bQLiwjJEOFeTAYduP-UXIf106)Jwwooya)B2-r(VG6nsgbf^x>p-PmxV!qFrWXRp z!!2m_u$6_SE`vpXta;!9r~=JnwH^NA=^nZRlyinG)3J|zSF{ErKqK(Ydc#+|6pdL; zJAN$!Dfa{18rXoS=_>-6+q29@-;ODGp|eQX@)N5_*|b31qa#$)dl4tQfes6FZcP!e zMpIjJDoMU@*xITeol=X(cL*Fhpk)d$>c&@o}hTAIQ1(b%h}_j!8h z8AjqJ zq8{aWkqhM^pO9(;4-<6B7lEy#Oi`UC3~+V?<|c2L>@Y~+Vc3lhmmM4Od!h?bwm=Cx zWI+KiZ@kKPr^=|5z$W=F-m&tAl|dY`6vn-MwimH<+b6chrw3Sn?%tZ-nk@MAg$LC+ z8K2junOU!@;R5)3JJa&E84(}^ivOOk)0x|( zt{HN#@mEl#dnIoEy~_7@)(YK)Y}XEjN(r;^u%gG}gIJ9WPw|{~tQVczALR2sxCgz8 z#%g(+(0DNjnkld$W+!NB5f!Y!VZ2(NM8;a;se}4-q_;SW2Cx-9WZip3`)XyqX4u57 zJ`iiF)KARVV4cJKC*xth2F?e1F=1Q2ajFxyG+m2g)rrfP2rqDa^+Im?x0*=||D;cP z*$0@xnw?l%^43&vl!Q-61o{{T*=?`kTi4J~M1JoBNYu_Uy%o8{*MX947C5c4KNk5% z9O&+50gbcgFNxH9Q1zD)$EWVJOi-TA?n4N{Ob};slA}%md>gppm;r2A ze3R9BLf0|ipg4qGmqez5Sv4E<`LkSw%Bo>Ew|9@*=S3=Ss1cUSJ{aoc>Q3f-Kpu-U zf>wzF%K}WHGRi(mC;X3BX#QIFy*JyflNNd|l}3xO@l+4}Zj?XJ@SWsvXkUn@Oqzo! z(6yMs+O>%Br$=}MJ>hIw3rID{!tfay*-3}Jw^Qd->MP3a8!)faEE-FX0=!ffZa2`~Cu)7X>y~B(L;6Yz<|W1En-C z7W7U)+@?b7wW|+)qX>1Zb4ghB4L zq5huRy97<~hprx|QJ}Tp3~&e=huqgrlLgY_Mj7Dlymm`?=F_}2w^Re=?H_kn;IlIQ z`+pRv%xGJpkuR=)F%+qwesG_Krz-H8*E+O`rE<{(JTjO*RuX+zBk{GAbEl<$F(32Y zZ74qvjIsYWZ;o+Wns@?dK|! zUYQ{LaU*x<|FR&Ie4g&v0*X|av{}L8+R0Dqgyqs9OWm)s-KEp(A?H3)chXQ`CW)*= z6;xus!1jNr30pv;UVlQl?*Z_+*JZG}Q-1SFb~&I~$ZFUi(y?vlL(SQXK&%5W))Z)I zTRvi}8DgJdP<^sPll32R(qIgc}HDyDyYZ+gx^(KIRL8B@rS3Cl`9Eu2U! z^=m$J=Kp`n47=w%ZUE;zLZyV>g#OYuRo9Cr4Gx}%eqM-n!R@U=pz>k630BbVqH(MlqOX#zXgzW zN}Vjn*NQ4?scEWxLvU813p9v2@zB_W#RCEz9w@N>o~Fg)#yWfd>c{PO8j~%q>*J7W z3KPVW{`~&$KeFck%;#m2NKpH7uGQ-=Mv)|m|FoQaKtk}31fa1bxoH)j zS#vunf9t`O8M4Zv5djrcDQXv>-2Nm4xMm9zC-r06u!qa7QY1te&6o0qf5PnZEA_{z z!StWI7gJ=A8Yw4aqQ2cV5UHmI`=6_*L1v=Hfx+d&&y2f^7Fy?l4%6=a<=_9obq+#~ zqYhAEzzQDdPbx5Ig%2KyZ-W5Udu_ArY1KWI@g}!wrzgc2bwcQlct-K|kD4}sm)7(J z0Vi<;NIPlNq*-9q-a9VY;VP}7RVw{AEdt6?Ic=9~bfIJ0*c@#qf+x*DhJlya(8}HV z#o&!Faks6n6a$9=_O^h;d|tx~U2;L*!@;W>3r4h-^vYO-m)s%R1fTrtf+8YpX)ki%Xg;OSu@$lVJdzNq@VgdMl{_Mh9 zAD(DI>R{K*<#*4D3(QvlyY&A)RSLGTKdB!si1#}zQqb~zeyzWcSo7`&aImY6aRhw?6#b7T$OkhRZ&0;GKHR^WyR-m49yqo9z?~$X0%&>BZU7XuhtD9^eaqJo;b=PTPH29DX0X!C5jqv-i!Wi-y05ve&X~XlGv0o zt9;^&Q(m5})gK_Cjzk2m{&rg_-kqvwLd-6FaVE8^o+F{lWV_e4_m&=DOF4ixfPGy! z?WusVCDhh3QhSKE?NRN@Umv=|(x8#m6Mzv}^Xv0#3>VE9e^MhT>;xswp8omv7 z4?lV-SuwW~596Sz7kkpZ?ZsQe@)X-`I^4D@U+Ls8iMJ% zdgalG8utT8e4d%Pp~+1bqaT!;?i6`f$r&A249!u`bk^9Xd1H8gG)>xKNc24J7ah}L zWD_XYUO+zMNjIK$KE?P=gw47>Iz557)Sq(ppO5^&mfkP#)(bnlm%^an_mS89Z`N=W zl^#9u;-J>c6lPG)s7@^YxzLe&j|psg%f;yN+K~nWk|^D8!MmFltjHc=987dCCZtMgONUWoM}a z{U2Lb85Y&rwPhT2P>~KLq+3Gia+Dsry9K0Mq=Z2_rIA!&fT263L#4Yzx}*f81in2Y zp7VOXe{)@K+0S0jihDIMrX@fpli^Z+glK!MrB%$3lM(`9^mtKUV=;rWahmwSO#z?Q_7r_NSFT#Zo8V9@et4Qu0;4Pb1J1i>);JF5PvkYN+=Te0QOr3;zl_>MMHQm zvIl0@ZaxPh2h58s*#*SI67-QNvvh~El8i8&eP1LTqG+{D|NI6li0yLgi0mst+iKr ztgr)4jMD>$94uzUPb_nVjDTTTga3g?Se4YM(ije{z5VP9$;~!kzlgqgHC)4a&K+DP znGrhMXEGL4xNwunI@#K_MgSwHFOS54e&K@#AD0@`+&CWwq*9a+!@)Z>$a(EEg@R^| z9uJya(Xp-LZzconioB6=x;+Y+;~ftw=Eb1FMOmbougZ8esjQTdTSrZato=JGBSY7w z`MRq+07qthmA9jtLNQ*^*mL_DtpPp(ExAA@xGJN|5Fcf4`mwWb01~|Q)A75 za_ca*VrwKIj`w~8)J1TqMn+d|fOp#LkGi9+41<9q>r&bZS-NYw{4!yqQ75S4fe&I; zAjOG#jpIsuJNj_R^B2dOqVH~&%Ldg58~_ETBJH`#g z6w$S-AV)u7h6T*GD_WgrpH=8sc|Rkn~X@= zjn<4hC^OXx1#5FA4ybZj8Z#)~Aw5@>0^FKM^=$-YVU8>J$VjzkMtF8BxHEp#>g|v9 z+1J0xlHNWGt*{D6q>cQ;(6A*goc z-ZzIC8&G$sZsZ9dqh6JCv@IH>qVylOHVsOEdR_@hKdMSXt+-=8Bt z$**TMr~55}z*0lSSlBCgS8N>-iBKIt7po^hW9?C*@e&==1?RixiQMcULF5JXy0%tC zMBNEpQKW12bPA=C)tQQZ_|6Y1a%-gEfB~%>dGk%Kw>0+tC z0#41Y&*#CDrtw;h|DwqLBE|vu$A~gzH4*aQo$iI|H;Z^0K#qiLH`mjOgMkjDNl-S} z^EQ_|VSp=J8CZj%$Mu!Kfd_>0;U}~T06a9T*03M;SSn-4^f43O9hEKEeN;H*FYQ^W zj1pWvhrN?*$S;$d{gsXA;-Ll%EcKWS5(CjuZM09=G2c0qzbmi+Cst9&D)!4#2@}Vc0>yw!E{W3#Mm!+q-MqY0hW9a z%DQyJY~$ndYNQ*T!FOQwiE(xvP=jarG~?sKPXEC=ZLb)7T&-nWSTwJCb4=gy4~zc^ zUC9CZzUUsg?RD^+a6V&lfY#=1KM}%9ur~_#qLQs@3IfnoV*X%yag2Mkf9;mCSH#1D zpyB9GO8cl{XVqg*8ZMyF+djJ^vZNteRa@2_q$ z;MDN|-Zz9pzutzl_zCYn5U!ZKdpnTj@*)bU;L~k|A52&3A0MYVYgL)hwr6RmH7IDP znJbRrqj%(uY~&DM%L(=4m`80kv$Ut7uh3@37FWNuBjp`MDc1FYOH^ZM2KU3{nekXI zPLCn8f-J)oV0^TWoJh?0PKMpWK!B?o;M5*?Luq7NOK|LhL>;WbKOS&z)YLi5i+WxX z4&{hnaP%isaEH0S^eP+;4(ZxVF9FVf_jgjn+TRP;%3Q>V(|AU@E|6=S!@1I=vKzE-J_mb0VNHbob;Tp zgv-k7&qSdeM#n(Yf-bBE@toZ?Jons@XDiBPx!d+Q0aC3+YlYQ8nOsFOYXlGsWc(;ThY}#p$Y8Pd%Z!w^Dh!*3Db!A@&j+~jZ^~rPUxZx?l-7~CfKM2S zP+od`wl2p7^knrd)jB^8B&SgTCb5}%GL5%n)VsRQ&@(JQ)^vTz0LIE>G0w0qNpV0hyIGhtqt#sQ*BNXye1*^6Vqd zSFD*1NBN;^in)tvJJZ-)~h8Fl_`iWXYS=OEK4kwbINa5j2TShQ+q}c`1Fwi!$DsJV)D5{ zRqF8pv!;sdmfF6P)6d96Jl>qSdrkckvG%?h%*7jSn_3a?@ewU;8ugRcWyQ(G;|!b6 z9Cx7|oM=-vb14rQNs*7SGtEd+P1c%sMVB&RIVF%y0!s*`}5ZQ0)fl0QZc{qv1V9qFnw^9rX zYb-1z2M6fzUlYg2&IXJkbx{C4{%IcN8{YZ86P>&vk|(hJPd&%n9|i+$XvKDU@$no? zuB3g|z_M*&7~a*89vXTe;?tK;Oe*Hdo*e z{HCc@T;t1&0Ol>%1Skzbrs$PxfC4S%1re2-M^cJCK=de$&381-P7O53`&n`Abr>u* zyG^AwW#mbx%g@caVr9AL{x~pYcJO##Px?$we%tQL+S(pv&-x_sg4Ju_6kKSMAx%<{ z*U8(SEYHu1nF+k(e?t$r76m{^rbk^oJ&{l!l>#o3%@c|VjXNo_m)YM!HP8u8<{fO` z_COJ8qMSwQ$125xL01q$xfumHV78&vAYF^9&f$E$bw3LE%1m2!M_A5}G?)GB$1 za4z-lB_)%t0pO@4{yLvs4tSEoEnnR!hvY8z}x`4Ut;U+VZp;#>@pKZ#AQ1=_=k8VB%fEkl3T*=56dehyu5QNQNz9%}do@4**-PtXiwVk^i}oP}1}`~M zIIUtaoKSP|Dk?=Qc`_A!#(!rhZOsVH)aJdXIB;1DAf5auA1I>4X606lJI*?je0BVF z!l}CHYjm1?*}AmZk3GFGdcn5K)Ke zIawhU4eemUkI8W8?(N_|V8Av5oZY2Q2kNDG#!%615rd$-shT*KUlGDs#OOnttc4i9 zwF#t3M(U29p#qUs({2Y}9Al)iB78E>9khi|7nLe3_7yiPB}O4?w{uSmu~3E`@Y(%d z9B~9|fndv^g%)-QP)q%ftw;U*GxDz56$G<*kJ*=vsi{-6wgzX1UYe^PPoY~692Zug zRda_P#7>OQl3*G4I372&YMiX;1Ly4JKK-0a5~NvB}0wZO~`b8Wzys*13y<@WpK3l>JxWkogX35y@+Q7Vk0bIFRPys$}tTJo(-93HpL0Af+S zifH=!+glN*!Rqrd!S)pc7F|+*;QVQ zP2gci<0hS{GiDH_+XC_^bh=m^f0e*p0h!7Gf|8r*->t(0AHwV<-eSd&fd|A9d5$TG#RWJWlpr-4J!Dt;Hy8&)|7P+M zpMbyWK^4;T&is2!tWiREigY}Jp=@64`J`HE*76=W${3R&l zwr2nVyFJ5JX7O=)xzG2|H1=g4tL0Wd6}gRm8N7c)DG$^(a{JN{tn$P#F9xYFF zgXNSTps9DFg|nY3r3maPeOCSPX`~{>Y2sB7C6?Y{fD@dpwc4v86P?GtsQal&)%4K3 zyI)E8i5c;G#sEq_y{&P>XaAug5#VzA&%*Km$H>-~p@O9H(oyzmC_>L|S33WGAT;{3 z>VEb$5>W&DXAEl0ID|lSR)gF4E5*1TQYp#g!>D<+V5m-n+?(_;4;mdYw~eKr%=YcZ zApyEizuJO8NV{SVfFvPZB|q0q5ts4Ox)7@szSP4n5*O_s(qb-%QD(A<$7aM}l?2M8}E?+ARy5gZw?VVq{q z?-5*+sWu<$4>qIhWhf+K0_Y1R$Zf_8#Kbi$gXXX*`9S$R2)J4*p*Cp}6HzZcsU8!5 zUgrjLEz7}ICUuV6Xn$KUSyTnsf)ezGb*NXO38ul?uU%VSg*$F^sC0A0*-vF)q-$|JBK!IU3$QIP_~T z$f4$Sr0?u*f^pz4ua{$I?w$O2RZlr`<;W2%wt5K6fYMB^EcsGf&n+qxQnp1))uq_f4dtB1+4A znK1`ls@yy>arCES1EQB!Wf56VBL&?Ho;T9;X3SeG4;Lwdo^QR%#tmR}IzB@DSHn-C zF3j!K-Kn?O0fYEMV%|)u$z$7>4i+GiJhwm;?4mxIE6P7#%|u2$S}_u@u|F2M3z>|A z1_WE(ELz)2J}K`w!UE!gf~&Ook6Mts38z1K!(E4`G4DTq^^+3zY+_g8xHjts#pcJQ zgy}!9^%qt=f9z%oqSa8ySckq1R)bGSKuBP4)+HjWy zYG;7kXM$&3q+{?b+5lS{3rivKo|}q^F+F06 z<4(y;$!-t9-T6dp7u$Zgc0zFlzC-1uZ~k*B8XJQh%_cz*tRyTD=<~-59>xBQo~2k6 zW7-?b*t6C~$L#(nVr?tro>IqqyRQysUHM>20 zeP{(jn|)E8zJxv`PbNX;A)6&qk)xP&q}jmL5}66RX2`po}j-^2vSqOruvuq zY8HFPppt4x(=}MwMYI!g6V5lQ%(BQFrgWM*#OI9U`w?5m8hTp|dDeq3_`mmqVnll| zmUIydIU6}>n#0Y?N9F3xWa~@4_#0X~ioTN+@U_im0;*%+LC`g8>Im7N#_@Xzifxg! zg*Aj@Y@E>D& z@!ahsD2oR^3(wWm>wF}SK)kcj80>LPnC}g32%wv;v3x;MuL`(x&Hgjyn!FciyU#cl zjeSA!28%7{N53Z~WxV0)|9MlW5d6Aweta*RS2U;Jl9hOpg`gq`U05De5i#q zeLYdoNHbmlc-`D?*50FKjnjKy&_6hOXja-9^tzcnq33j2CZ_tEVDLUlKoCopf}e{= zd#SN4=5c=@g8Y*=48cJ2*))0-bNd+t^&$e=)nh-~CWsj|WTlRilXD20obh?G+>V-i z%BpFLlIPoT+Jmgo)?kYviKIK=|1^4IWkJZ}4Jn}dSHRz1ml=1&a`Iu3$KPvs)_d&_ zq+CG*1WM3L0vC%O3lK;gI3SSeW^D6#9#yGyIy}aytDo5!>B53c(E(A()zJJ0l3eQR zz1IK-ypl)AK&Zj08pbheG5AcX9K|~~bFTc$H)g8DkxI!$9>VfN#DR8Au)yNF`5~6sg__zJ+_k7fo0{7K|mkh}LFy$n0d0zmy*I#iZ&R z&&8n4f4_)r8FkPjJgkOF-0h4pqXDZ)HPHNJv66lg_IuX#W4K$%#(>i%8v|GkkfB4tbwC;9;>R=2B6 zDiIcN@!=W-D=HW!&qrTp&tBGuQa8_{^Z)wi|No;oSAqbp%P+LU0{;YCI#{&^;fLOE zSwC{i_}45BC@W+k$%0w1C(HWgf%h#dsD3>9OCO~m&!qkUj9quEi9@W#q7L{a!qFKh%JH)qFkHzdY(On@B4)jdN>D7+*!WpLdqT6n{!fg9K|B8D2 z{z*g(ysyoK@$P3foI=bg*!@bEb&PqUz)J@d4HCi+=ONYO8}4_Qt}HQd$0DKu$;^p| zzTS0-Hi=UA_k~8;ws$iHt1OQ;`QFO$WF(IDRyS8S@%~d+`rC#9obs9u5UK6XRXcCU zOJDhES6rX4i>Z|;TEpLydo>u+XTZtHdzC;byRM@mq=MI=Mp!FX;oA44aV2DAWK_IG zn11YPXSQ%u*tGiU7h?u{)Ba_`fY%7i0xa@I4r?})3k@+paKjexiT3VCPrIhcyk7b9 zc*6?zHiT)(wb^+u3x^%-_u6aur)qlAd^5YA2>U(onvW8J2Xj)wdUSkjF)n=JdG3RY zPR3tcc&P*S5>O@y7a<4{0W*6@NwOL5tLCbsJ^lwEz$qwuqr>D3%uCdIT!B72eMj_Y zAEkSDHy@?(*NX@_gZmwrmLD4T*@bmY3G`iz$NHC0LG=~U$>S3S-1oh44s1U1w}y{bGkh<0kl4OqRdQ`(VRO~~nz%+?*+H@Gz8F3~~Rwob%bWeqETUiU{c zI6MIuOu6FOwL#!-Z*)hmKX=MQJC)3UU>Q<;e3yKY-j`uL&SRikPSfIuFQi6p* zMf&Yr=0=$eqDdL+&Bo@3OA!GmM5a3%eM}dlUT*?_@hJkDToT+IrQZT+@n#5oC;e5IF1{E8z@yui+=jTal_>4TXXmfO znk(gyD?HlgFZu3E5w*Ot^(vVl0Kz2WEM70)A8fO~%SQi)gDO^mz78;mPaOg2(kZlTfcw$g#wpmkcd6^mIyR1tppRYDvl^OHq1zdUxHV*1@N3H z2XEbqi%hF_nlzj7QwG6L8p-M+t-Dii%y#!*7V}IRYOUZZJMbu>O0nhF3-7cV2cgDll+wHj z;-{Wu+wluTb@njHh3dyjnrqdpoE@&6In-}X?N5e1j{c0>Mw2wHS9?p3ODW zD&R|B8n$>+w9{P$Zr}RUn)hw&m&EM!1FmqacV7K}xJcg;5`Q&IC_QGtM@0Tuqlu0Q z5IcU=eQAyuGJdXiM8J;FUP6%7Kp+D|+wS-wJ_nx(*fBgMAt8Yxsn~#0nxq8Xl;_tg z*epQEuaXA}F^Busq6$;jZ8I zP2VrY%GN7U#8KC%r!DQ@;?dr>;ClW$O~UpO5Dtv3Nv!{Eh+%_*XkSK3!0zL$=Y;Ar zzQ`)ZOpxs0Za`0Gf`r^f>kkYa^Ld!`p0ani4?|&2Ugmz|#aX35OXbfEvw*MMnQ6aC zAuBoaFx_%rz0mRCDpO8w{jpy53u5PrqSl?3$5h2@vGkvQYg@l?;09(BaVh^dAiD%G zoA?)e#xveKts*KvG#LYeMAeSj-j=#Muthkc!`Z34Y|T4qqi1j9qmfMcne#y|N;+Ld z$O$Ak14E?2&j7NOP1_1IQI-w0HLfx_`xZ{GQCJEU@CFfYV$AxvJsq_;(_3pbk{v+* zq2!3zK{RD^8U6U4@h5Ir3q)+Pote%40ez@XB~kzt@ka~wb^;ix7v_zuBt0P~h(luQ z;Q3&{DX0OS@0ZCtbUwLjnnaN#@$?=a$jT?M{BC;{r{jPG6m`M0vCt$ZEgb+Q`jh-w^t4R%@Dtp0VfMs-H7 z?~p>p<+X=s=D$b}*j+YyK*3)ZO?dT3Y%icAjXov##H&=RG5UHuV6flb8#kk4gM=^x zk(g+%#x)WLEa#AKj<|Jxn_{dCHDd$gM7AX@K#m*f-z8G>dxOJD$hdenJXx!zYW51AY z{X>))sN<#h*V<2%b!Y(hE4!}gB9HnN?c)_slhi3jhQ0hHoB=PN2uGrTyJ#1q3CW&h z>&GDva$1h_{#;d4gwEHy`u-70Tydqy>0V?zym=_0i`L8HPo`A`O|IQJCj$+ouXUxcU6( zf>VfMZOwg%LBWwRpRR$*1Uu3+{EGDcN1#~qx;UFkirVExFL?vPQuR(Y^?&q4LoZfg zsJw#-QtDHtSFJZg|93_ggOkfz0F16Z90-^;2r9`#LsX?~#4|5F_|TwG2j@9ck|KAK zmQK^sMZr`S;2R;TCCf}Hk&{JMq=^Dev<0_f0ucf%@<8~_s*Tz%=F?cVnc12Kg~zn) z`X}Tahuwz$OMh9v5T#CHxoiIyMG1>Fa6x705!-8uD=ycRj`g*i=Dh*TCnGQm)ek|B z)PU0;L zWppo!E|*OIu{>)n4N=D4ux3*!t^e1Deh+k}%U8W2R94?gKx z^=b7mfdomLOgiuH1$oK!-Uuj-<916`j?cYlpxDMK^lXsym8At$ANb`wO%CAROrwrg z)G4?X4N%dL#-_cF3zSvv#96NBDrhkYm z7*Z+CUs@3!&~SP%bN|JR$H8`v-YH{}W549=QIDsP`)_#{mLxx8D`6l_^_FYBCX=tg z8;9gu&VCOT26nbJi~><5PA;V{(CZ@aLgisSASETGfqtC?EuW@PQRN@}VM2BeF-;S1B(R4-rX$tonD*p~Q{RYZZ3s_oNzb-vpE{A4rCwdCf=V z9vBqKG8#Q?0DrtE%N+U8 zdBHLM#(gL4_=>Tn!jn6}lbbLagQ(-}jEdciZ|h>r2)CVET;?h21y^zt&uW_(f$?nR zLE;f^XR_jfQj76Py|*vml^-h9`aX%k8+9#g$6&?AL92=>f|Q>Nbm3Jyi2tItW z%_QoUvR0$=+QXsfO`F_?hd{lr{t7w}e86*GJ$-+l9PQ~l8n|PZ*1fb;0c(W-Zb^rp zJIMKGTC>qlMFocJo3c=b91xI&q&xF?-;pn22X8X1Yw6vOSn6*@kwm#Z0DZ5NccN15 zSB~@18P2ar`IOhrbVbQsw@7uM_(5PMmTc&bWKZt}{ku`{;?bTX3h(QGMAv>f>#G72 zVj9u&^cXb^e$eLD8*-7f5(|SSI=bJ}1`I}6kBq~rlOPtdfh7f+3))%M$4kX`nb)-^2w1dvs6y`}ooQVe9==4~ z1wNebJs<`4PZ?_TTv}jj%_2~;MtNfFH=lglMPzg<=mV>jVNEOrg~+baiNOud%t=$s z9fqbyR|#+bS0elZ8dA^pG8q&61-}^x;|)FYweJ%p^qqP!k@si?i9~LHTo$w`CTF{t zfl6c!Dj&O>`G`L7Z8f$u!t4mJgKNf5Y_a0iK{N)+tEYk2J~fDkL6uyo(63p|k6n+Q z@ENiQCSbkOQBa%x=q$q&vX86}9Ef&*VVkgoc_T1FOBMW!l&WBR7XMu?Ej$9tS~hKu z;hjhQJBWjvq{<2wa=@yukEXv;Rzoo0>kt=3Pr2oAjWA?Z?uH;kufR@S$4iS2n)5oA z3cfhqoE^-S<-hYgPz{uiNEcxNu6_~K_Q}k-kY!01*OM?Yi`*iy(dTKQO~JZ*9}0m_ z^*!2M_Q&9!WE1!v5l&KLhdJTZAMLJp*hJVA#=ps9X@GMv-$$Aao}`}*=mvmY;)_e& zt$Z|a1s>c678%7Gp}s%aD6EMVF-jswHf4&wpq)M!RQ-6%ex+!(P={~`r!@4r71*Nr zVDrY)%7eC?rrDYYn)S20M!E2N0djk8rH-vdME9W!O9h7c}P$Ar+ zH>D8Dj@eswa795}?wF+H-24`aB3vWB;NcWq*0M2G=Eb{8T1C;BGNG;OmWzvcJ;)T^ zVJ#27FO!da%*m4ylGd+N{R)Y|r-4ApkXuG6&=}Ck4nNtfVeL%hE?^JZAJ$?On7`Wt zk@GwrXf{c-B1rjBn(3PBS)s)3nBHIj~LUeub|cRt}6!neP(Mu8SC z(LTv5keIw9R?rU_qc!1vYEEntWeeLGNp=%#5mCCpPEb&FTyDK4gj>f=W#8GDr8Z>} zGVkVfX3gLfvk(Dhvt(y;66IZZ= ztLJn(-SlrqOX+n)EqVd88j!FW9F?ZI3TW&QF905huLsJx8!pP0b+FX<#C=?#nNA@bt%o^VmKA-v-)Ms)37PAO69ETnF|k6Vmyh{lm`9 zjB3gAkqPVp)Q(K}CS;Ho*7_kUl1ar`&;buTj{CXFNu9~5o5Z!=+Ce$-{SrOGwW@1^ zXWyFJdxg*LLu+!zj#j1ZY)C15L(&fX3;f-V^xC8$)PAADRhxqF4lOv4IbaT+5aCIQ zb|N!~Dx2s~b{sAsDHO0v(26x=;5BjSzmiQF`iZJeaf&MmfFKqnBMc{d?)C?-DJI>T zJEQB@aAya}d#sNyETaMns|}jIIlS={+Rei~{nBV3KDe85@DaWzNjHTn4-@jxKiV(w ztUVDsb6>E~-A&m3QBf~wS^R~z#M+|&9?x#)qvrSiWaD?s&xC_^P5V{Re$_Mp-jSj+ zgXkLeV<)Egpzs0zxZ^bwC8w8xIDosvwh07YLQigoZypLzB3%+$v$X~L2f4V`IFc2Z zqaTVh;L0M9KK-Uw!>{m**Dy>W)-szIYW)%?R`Rtd0l);r4jGxbN9fV@l`@gnv{-L` zB6YSL^*A}^D54xuWu|GAmGnw~QA_!bYcr%eeCm z@NbC!vA>zk_s_QdJBPk*J5LRue%b0hP z>-DN>338f|#6;_AGqD~yk2R%cS^f>YJ3L>#*gVgEqKSwRl7Y-Li#(6wJkf1j2NDJR z!z@YjG|)}STvy%~&iWDi6}T@lQSlSp(xz*(bEe|ef6UxRpmoAKvEX>dEyy} zYi<4tiHc^GokQZdQMh2!H9~E946PZxqjt3)&+6oOW_`@gH?r5Ky|3MJ?dJ*w^aJX& zCc@;vGUFG3HXF127NZaA4z85$G9ln40Fcym(A3RLyqe_%{@X^>ezTPF%e0Gj3%yI$ zQ+`E(?>WYm>G5$3yGFG6?ab?R(LbzDQhqz42@BohZ!S{}1f1-6ZIG3(qAP7A?g!Wx zUjTSB5?pl~S2^V~uybX##NpHk(KThjHAkLDs<+QuBGeN%M^@V{n;pwpOk4K{|;3lwq9u!@XtuQ3HS9N8LPWk3<4bRdt+(BFMfcJ5S$D*-NK$YLky6+dMnj z!c^0?6FvNT>~2&3T^1$`LfFS!XuJO1lcLZX8b8(*Jr=WKbB_gBTz|sn;%W?CYD^X!F{@&E(|nW zN#wy`XtR&A+?1R}CU!qh8Zsv4JA7_>Af_6qOM(dAm4>dlCA5F%L8KJi3*284A zs5s$sVRHQagm~xLp{9^Qx1AX;U2NH?0Rxuq$Xn5*18q}T)T4o|istpy4aG%T?US?y zacWnF+a9z}WVH4BMRDWnPn^|$D1W)>a^1#(K(>cD6Ha@Ahm{&p`L^D1=9a1e;Ag+pbW9PMcn!Um! zHIq5xG&y7nY=nP167J@vlRz5GDm#*yD*2jM(<0?LPR->*pS$=urb$Y#3gO_Q@f4l z$wcAtT24>3&OH>V0fIV8k)L_VZz?jr#*p^ z8&=Bus3ZHAb^}iaYo54Hk}!*gLnOEH_%&LHYh9-%Squ~G)`V(VBBDjqikODTH#eM? z!w*;AQsT=psCMV|39<$Nl1n5D;(Z^TUq+}cq!O4Ot5@^T?TPG4&d3B0#as1?FW36$ z%kT{l2hAiKiitad$_K5Hg>(d5E-j+91X5-pDa(H4yMrdOf=1>Op2)7rN7cs-`>s3% z>>?M4=QS$ajhRG~;j${Fr#Vq9qDYtOZa-?8>d};7k$*ROQyQ7g@^QO~+)g>4UT~av zdfQJ(L@J5~gcB(tZz#HZ5L4hzBk|J?u(Ccio7Fk?me^i6lB79Ap*yokD3Sa zH*=Jfn7Cu-9o>T1ru12?K;tG>)=|U#OUNnZuXqE6LW-Hehc7LJXB-kccS^j`OHU*x z6Z2V@E&F>ae+|uSsKogk=q8hO@0RMikK$avs=85FfDFTWu1>gC)LOaBm>S)1P;!C* zi!@R_n;OJ~KnrnkZPCl#!k2QIf4Ugh_=vW3tlV>Jk}B!mNDl7&1^%)|#VPzWo7r)? z1dhJQ-FbX|)KgMMU4B$>eq69!;dxRqcHlIbqr0Dt!WP{x_3o1cprU8pu7zVnA&s48lIITLE0HbExc#W`B(6W2Iekar>iEA2V2Q8}>V;#swEU&&v8$`Q7(rYG` zZ{-AP)z&+h@jMT~bFY|sq2*K;?#jEKTatk|+pD&TX*;bRsbz8BOS`|JOx$v#6%qgP zqS%r)4%)4g4a628ZAgoW=^1G@{PP>86?W2_mJfmWgZ>}{s7|;S_35qoV4Lj6N9co9 z5=fJzY>CI85RzC3SG#t>D&e=EB@;XCy&sOyTVahLrh>024U4&dJT6jxbzGI7oCK|& zv1kNBYa~39dv09Mf%g)rfmX}P^W+)p^jv&6mqJ#M6RJLl6K!BC{Iu49iwv+N$*ygK z0mB;k>dRRBW(SUW%y{u=sM42fQu`A@?E>sC+Xi1Gm)?yMT-SDQ)9Z(1?zTHLRq8ep zUF@b?K;6MkwS~N|N4jK!)21V;Kl8y;>CEG&kO=J-D-+;$}_v z4drpq0v^S;EDR&DkN1y0i)Zh)w@=KR{jhC9rz^cT^7Cl7Ij>@RoOj7W??iL-#e|hs z8GA^KQ{I^8`6>Tiy8mszel7Rsc1k?8BfL{4GXX`>S=jy|+? zO+ zRJ-v={(da@*VZ2*bkOQ4FyGVAxdjEvfq;=EkLAem9uN$D{qgBLE<=Wbnh4qqNjAK2 z(Z_LMb*ptZ5$H~B74=#F%W+#qL5I<%4M2+-KCn5rYKu;Eo6HcK!x4G&%l!3tJ;gfmvgwOMCIe=Iplp`=&Lbn=kN<4sV9@md{n3N_s@yQeiZuiijjG*O}{$X zc+xG^5yy&IF=;Oq^THmuSd8;+%yTTV-S|(Hu-)6_MQXxtU4zqXPVerM~|L5(Tt7Ma_(K0A#ledTrd?=)e&6zG_(hVUt3>}JMN zB$@{*$W255a=L?2^s-d;5Z>UZZqj#OJfVL$EMx#bMw{7DdTI$7R(sMIldtIXlwtkR zi{_V&VoZ8n=!jCuJPeujM@~^ets28&``9(&DW0pv=D04s2-$b^5z!tR(vZySg8X~H zTnwO+bP&HtmcDHL{io~b8L+4EuvVQFK<=&*E>)3>x=r6{DQU^0brwna^8mU0p zHIGN~^0?-*bpzY&ca>pJvW&)q5GL!EDs%~o%mP#)?L$F4zY>Ljp90CKrimjtK!sj@ zp3aQ-(s4VM$9y@;Lp}AD z8h|70S`qQ2Q-4z+2UnJXtUKpP=S>elJ2Xo-P(}BjTE+>7W5}JJcPmyOM2j$4iL%Wp z;pih}iLqlhbz~026Qd&avqBHQY}ORLtdwLP5`%7FMG=4 z@YCXYkL&~EC&p50U4)31atMlkg+`#LI!Pxf7T(R7^V6ATUAGO+jiIY|c-}>YWlvbw zo$VL&NE{(Bd8c>6oS!9r+^Rci8dB3E-30EO@mefN0kuiI)e+!XW1VfxoIQA+WPc~J zcNJakJOTVyB@ zl#LPSqZs$dLfbCYp_^AF;mfEK61jVRlzNU1c<-^M^nwKr!y2Z(BOjMIlV8xJXjVI# z6{VtN+yF=~BvbggA9d)F2?4xF5Po0zT2;Tt$;3>4cOS~bjj@$+$lFzBakd}K#-%F9 zmR|k&c%yWn$zN#iy_s+adM`C~w1g}|CaC=e*-P@D%bsUIU`B2v0njQne5}+GB!f-{ z^1EV>FC2mdfXtywAF+Y55w=P2N3%l#P|hsb>N62Fx@g7h541aPOrr){WQ%wh2{^`1 zWbWK4|B^K(vECYnKYqn#JU7f&4K|QzBvZKG?Lnx9+yQEuxSD6}^HrZrxn#V=qV68b z?PUf2v?X}X!nj+3QI?7cJ7`lJ;phl(Ox1}}n35KDthkp-(9m(|OPIBaokutplKM&q zuP9gE`|>;5GzJm-i!Rdv>#@TDYp(rRZTm3F+RFS-W3W?J9ea26>V-;1$2=#GR;0&( z@_6%$v)}Pn=(`6%1%p-K(4c2uur6LcFzl_n#(uYIH6xL+vB^@3G2s_Re0;w9Sk6NA zgEyx8*Q;aaSE5Qfu*FHRseKp%6YPs#&EC?3prQs%fJN2h$EzaHao=ezv*yRKm;6jc zn?gwu3D%q&|K%#c8ukIz&ok-JBdBtY*(g+5=cu}lFwJbdH8lk zIA*UJY*xa6^DY_4zRZR^>W07;HXCnvjVV+~ zW{0#J6laX`yv?kBDecm-fQd0;g3e<*_r#)vr+iy_c(=;`8BzhZf6kL z0C6XCon+6mqk-=A%2Q$+VCCF~nVf^~aIn+(V8mp~58$+6kJR(?hw~<bj&u>KP z!$Mk~eoRnQgRPVT2F__*P4C6jHl&lfiGP1kDaQfWmy$UccCuCPNz;&!Q?`)c$oNZM z8`Oc|Fz4_<2T-I6=cVnz9*tAddbRtgM z*%_Zy4ZNoCqfUV%4w+OFJycX@_`TfaZsy{^s6Xdlg z;!_+c7hg6qv`Fy~OG_#5WEt8 zZN8L;D%jJ~r9x{>N(a+Q%+p3Ww?gWnV~={_nWeota?eG|hSFWVV!UiWJw1^MopE&o z@lT8+dAUC$T>QA$JcjN?)W>cNv3rgdT?w0Nk6V5G_fOnN!r+2{^PO~o5Nw_%Ew@Jc zhl9lr1#`m8sjx3dmOG=QMt%6-ESszQKQxRiAw&_2nhHnfVuhbT`%8v)l_=1Kf|4_F ztAI~VT~r6Mf`=ClVhWV#^vYs}m_Z%G7M0N;y@q5t0W)wD_8xG5i7by zKLnXonpV!Q@#X{=isSHD=Ojn8=(uFTBR9Emuky9+Fz@nZiaZK7Mr8D@5qEIcZ0Z%k zL;`2ei$rAsrn^aSC{sJM5*kON+05gixD;#*h#OUe`_oio;aPg6L}U1N&y5L#3FnTF zD@~P~J%++WXpO-7@aZtgfE%c&?9P$LtVL+kB4be!_2!2i3(s}?3>1_4V_w2A{kEnv&Y6h)ioKt~%M3p-|MLPUodsV-+-3|i1q;5R zlNw69(bYmSw`bYnrcNHxl0ZVJXhM1{wJ2^CVKChryNPy$%lHBz5r~WC{Zha&oC+zn z4lbxDXp{^6O+_M`ziPL~P3e3nDTz}y{lRALc*VjI&Su;du~P59``hKtI-<&e7YWz$ zi*YpDxA6G(KmsY5Q!q!7Nc-Ip-oSmVXZO`h9{09Tp)3r??32x(!AgEFF3v8yJofdP zmz%gN=MmC1Gn)D6=~zW#>5Jj1Bl&O5J;zF$=zr(9p73p89oKWRySTUk$dg>NFT=F6 z7W)*VQ*@bAF{ep0&ZPDH8QQEeZnjv{yP{*3HHm(ouLq+CpO$N_G zG?a#Mw7JQb%%wJ4%w#=Mxl9Jhu_m~uZ2E?_4FAt5Sl=*y~UXlkdiXjb!hJ9f! zoI>QhaPY?;tUwZcebZAQOg%XPQGm#?!#FQplJRP6!9$NupHnN1UM$2+-(vP4uFcVy z#lN`0WcY2OsR18`e>Fpi_)murmj?0Lu~Slclf` zLx;hYECN((yWd!IiAgE(NEY5&7nf9#BT$<*OkP=7%&RF(2vyL+_gsu`=GdNQf_{4~ zZ%u~ovmWmr^{p^XK^&{tVJos*l*AGV?VZv2k~jX>ae;9i(3psE!S8N^`d-o-lcLM5 zv!Dj+eMv@Ce&6f2+&Ad2VY^$U|M2m_dRiK@8(Z$M(QX~J*hsv%CLT_6S`hwq^X&zH zdlPnixK&U) zC&{}04maZcEr%;MdbEV*-+g1m#UvYHR!$Y^@U-E-i}6YJ$bW{d@G1Re01!(@ zZX5Xt9TMH+2+zc8t+J&3DYDmbTp0sksf45}zt;F>y_WnV={=~y1~&S;9Vyb}X_KeXsmU+sh?0)O*Gd`vhp!ohNI79q(6Nufha6`(zhKeBt@`V5+#HLc+fqzeF93i2amxBo+|H?T#z-Y9kPx zUgL9#R-*;@!(5Ax_b&o9_L+D5yV-Bu0UlXI3px@vH0GkRz)qg#Yo4OmP&Ow3F;j=> zKSyf{yyvcq7Sb`H(1^Ntx0~A1vi1$|B$(agN=-w?E(2uPn*{YtryM7;I%gw`o77hH zAnScH7Z<$fq!od%{m3knd!rIyY{A?5f*1AwxK>4HTLXY|8dCas6A)ABgEK+B2pB3r zAk|1GMSrktsCMbWjof7uYlw_glL}kqsoF*J)uhRn8R~dm#MG@gqE^Dm%I!wAddAYb zMl(t7Z30RpeHguqth7TTX>ug2bI{CXUrP%`L>RnY@sm}mw zo|UuVQJMZIV&5Xm^+{|onsS8rnlU-(UYTti5>;PWVSg}a;+Ws11x7D>^y}soR_O}8 znuJ2xn5G?a^9RU$xW19`tujoDK*W8)$dc2tQ9Q0;g;ybNf~8Y{9vHYhc(lo5e{(Qc zLxG~7sz~iCLe4(WQ3F)~Vq;`l#n`MvM=u0*v89btHu+bvtSe)g4Ha{ZXQSAm=AKZYRyl%~Br2gf!Ae#8fsuoqR! zQ}t(Vct(D zxlhUVhrjz0qt@JgNO-k#8=#tHI=`t~mx4PxGKTMotOY7y*MQRYsGFTws1W=KS5aIL z!jL;WdNJNwdguGIEE#{ydC90b6~og=M}w&=#P92jbUTgjMbD+AyyxCP?pDVz#Abc4 z=>Hw{cPHxd(+|!g1C?Ay2f*hoH=S~+QPzOc4*K+m>N^mAxb*9IaFM(xsnpv1Ec&Mr zM0>%9^KBnsRt2|H1W!Ehi~Ny7RScy?B#CVu!~rMQaO81UFJBNIH{Hv_y5D-8Tdc}iTt0V z+L|V%7=LFfzO+^1=Pr~kIz~*@Si_|l?aLL~gIN043V|C3eRI40z(DF3B^creSQ&Zp zcgX`@eUy}r{@gJC^+y6HqpG6^$?bFR5&nf|Vu!oNrrm_zcglPJagiVeNShLm(w1jAXe~ z`CjugBmwYkB>3V3wPS%P?V&WBkY%T$CT4@S3YUQOuxX5P4{q1!cTB|n8{kWu#p?^b39D#lBJtrmV2C1E@4$*Sdauc9W2~ve9w>2$MatjO0`YAM05Uv2wnG<+Pm-}Ra9SEBx(~Z$)vGl_Y z0b;6{mS^QAj)-(1c+VyFQsQV(V=vE@szU*vq@x7!uARG8m^k1f+LUpyXxN3!Ev)mw z$?!*vCy#HCbQIjSQEj(F3S6y)D~rgMba!}~@z?Yn?+Bdb@#}ZjS$BTW8pAXBnJY@T zS>@k}zZM2xR9?rr%OU_B?NNtPCfJ(NGBQ=(N@EF?u>23ha3gX_AF)acwMt~|T=;`n zp(8u0juj0H=et!SM;Bg@O5KK>!y0}$7PiS^&rNjnYoHUDB|MVfXVB|J^mIUkXn3C~ zr{?xi?TE>Bm3j0}>4G1n#azo?u=tL?}va*f$ z?iWNOfR7{{s`7T!zXF3Cd6?tN+*-Pt`WSf4&OL*=*LE4Yr%<2_ojGEM}OO5UO`_ks-scL|v{>X*LI}LBV0j+N-H~mG|Z;ByKxu9UC zAkZ~0Mmp|4uJ|Q5OzXobf1nk*3pBb$qOROdG8E32O37f0$eyYMlk0-`Urvz0kVMtn zpKX!mGZ-i2Cc)^?6lr$){i$jy22M8^f04L|V%oUv5-G&-O%QS8*|TDwR?-=zxa z&;9oqdw=!kmu;B!4oRw~f(||d7YX5^;e8Hn4n7E3&EW$~Ho?ifr~c76Nhso*4Q)SUl*4(c#VDz~FHW_S z%E?RgjKoy|5ppr-oPUVq6zBXUna_<6sEkd@!=Cgb(v6z;6@!2aZm2=-g>U( ztUQ? z9P|5N`-7#7FKmhTc0%RE3=PV4KUxl7PUW$eGFf}YA%d{a#Rt2Kx(wl(syyQTZ$iy{ zxb>p-azm;|A5*E#AYeJ3nC0kO+_?%YWIwYt6dX*7W$1OyKP0)O-k^F%vqF(vZTngU zF3(J|;JTFqCXR?RD3vrjg~hw(T)pn|tA7txS)*JQ&nKMj!!vKZd3HW&Ktu4g&x%`` zsf@3;mb1=mSC(Vqz!?MD%t@Fw=&g3{1(zufV8^&!N9OhL)Fk z0N*HP(UJ~mW0z`BP@s;8`kWmFT@!^;OaYuXUJ$qW4j-GK!GGTdd6uv{_OcB=t$%jh z``77DQ+BbuS)z-kH9nW^-w|I9$^bNjz7=T&F{TurN|dbgaaZ#@3p z-e3o$gC$=U^GEtrYDji9ak_`9NeK2^m>mEKqZZD!t^0RFS7NlFr@32ZJ_VlnDjuy_ zHfZ7doJ#Aeap0ayViS7aU`zWAzd^aL#kiqR(O+OsOEl!C=vSoibwVC)MwHY-S$b@K+iDX=9!9NTH>tdF znY(en>5*?z!-&r^&Iy=IBxB3&X5|b7(YWDYCs5cEajy0Zox_zzW~04(-H~cz}j=kg*pMs@722-AyqUUDOzVqRhljf zY|AHCuf4ceMW00KavbLGth__SI&Qcoz@>PnBg`E&+_Al;Ew7XDK_-<5(?9Z@2;Cvm zDYwBS8Es8A+olu}vPGaa>KblkE|hRxjE;t4M%3<@q#w)_m}|_}k)CoFzNsG(;-jU4 zhfZ<02v+d&k;*Wejoft_KGyt|tflm)znMVqxfecH!>Vc$(Ggpt-!T%ldaiFN3B`u6 zKHH`Izaw38b<<*Z`I36XGnZ0>gv4wPha-FdkG^T&h38$QiVOp5uwzM5HIwJ=YH+h> zo${)EyqFL62LL59#BGL`i3NMmax7=#qwG;NirD59%Li#cATgdLH-zuTp~4iue0}_Z z97%H`hl+}gY~n7dTn&KPN6U^I%ir+%8b*?-EpaF7%VA7Dn6=5!eU2Bt3&C59i9q!C zb_YKYwtig?W%W+c%;+V{N}Osp2PUcV=lrZCAZhM(-Wojn z;diHt;86p!F2z4f?|@#-Wb|F6d=we-b;`s2)j$}M0Q}t-k_TQOh3BLh0qkkTBTiX1fusK#k1seAPkJ73i7-q9R#eubhM@q^`dO| znSYV?RYMdR=a6g5mUHTiM$@a@(@Eb*{p*c-uJ;_c7#-TltoNb^i(s7D7IewNT~&7B zJ9|34%u=a;#@%zE@b&xqncCBKKX!^^s`~t61M;o+{%JGv)I0T1bY{s7IAd&0@fVdB zC{$heq)emck}LS=Xk^Oz=-@Mo+_KD70VD6FV@shZU zN~)p~cIy^W^&~T4i1p{dmeA2u{BRz^d(m{_3iq-1oS&Xe^WMpP#_ak5>WAbo-4~ppMW{`E`e0E_gWCBt9j@ zb?WQ{S9eP(=4RqbI%bY7FUtk8a@ZG>6Gxl>M zOKGfHKKox&#-IwCG8zQ*4sgadhk{gtPr6xHvqArZXi~cbE`~GWxim|b7q(#uCB^efS9|=uO zhZpFJ`|A3PA}6rFJ-Ol?V#fRU%B$!2- zyC1th&FQI{{ilWi%M%rFU%w~wfxijeMb5-ZL1bI6^B`jFmNO&u+?U%0;7@6YEp0N! z0`6&MXjg#~aRE-L?gJRs-I85u7i&kG$bHFohU@Brvr+mBXVo(h#8Sm{q=Aymy%!b< zPY~%PEeb2Rh&4914)ICQdad+DS7=*9gj{1{J_~yUA{~_JjPv?Jq`5Cfo5ry<%ya<% z&^|vPT`cJ={|pbl7iagLiB7$>_3`+Hj*EV+?3yjx#Z(b}Dbs57VZ*tG-}7mfbXk
  1. uP!(}gPDr;58F6w3b z@kMi5na)v^oI4>P%6g)pbdU!vp?$X%Z`Mj_e?}ApQm>X4y{?z(^t?BS;f_#H%T?E% zwbYh`UGMu|T6>xJp=B0er^s%NMCF;v7@pIfjgNFi>K+Sq?kFFSDfM&-aE~6>5J_} z98>W1^dWYok<|sQhx5~w!dF{#p2b3uEXD9pazoE}7_-$s;g%C17TwY%}V2gn4%zejAXy^;|uD z0Pi`Af|KZ!@8=zyn^gp@7I7Lr-(A;gqT1U_ndnL;6^R*ahvEI`@ge*k6G*EgQGIy_ zetZON*nGFHN^ieCXtw?s!Y%>LOhc4E@AlXLB&PmhLH$vHdhD6m^smfS?+;#m8~*BRNi-(Gx&J z9_MOQ^zZu$9qkWZj>MLdlO0v1OXZukf)^dflsh1?4#1AB33i$xS^uu*&m@HgJnIy- zUUUu~&cx^e`iRx|cLnN%=OR&2FM2)nou4OgT(M8t$!wGgS{Ft2vQ3xr#Kc17F0uWW zGmXyQb6@sHJU`w&)Nf>Mho@s8l5-oTsH zim+<;fAsvVIKmQZe1xH)+9=QJV$kNx`Yyn@)s*2iug~oI!2D*_Y0R!#3J2c`^?|d- zt4IlLgVmHb0Yv3s%_BWt%r?ul5NpY#fHahVg!$?WOOtuT?U)`q5>OFZjtf=-a%AI=ggyVsO@2B8;M!awGs%jSi!wog&(m_ZcBx!SC;ng5+;|u)sSHoGhL8aIfHrcX3kG z9$-=zEyEv5cVT;jry?rpItZ#L*(_w}P|_W*`H$4ftxe$r;&38AAkVG!a|$`{9Oe@O zG513D-xTz0L${g7+?BON@nk|xX%dxtyr1q$5z&%J?3r@7*&#xq=7tH)2B065E^Iq9 zdT1p*Y$}>Cw6o@I>WO_5NbuxzRN8RhMtLJB_J#ec(r7eVUiU|e7%wEZCi!(3HEA7s zq=SjFvCj;wj#!vcx~9)FDb#MG@=pyOuJh}UL9gN4rPwHO!}Wy61LDBV21&@)A_75h zT~{yhq4F~*1J44dirzL__whhu2ufH8I5BjlVDDRKJN}F}Xu&J5^DRk?!GD==7 z9TuMpyOpsGwlg6Z#9onXtp3^%=4nEGEcWMRlX|0PT;j6+y$w1K!CoG(On$4IcQh6H zJVj#f(|(~*azm-%nU0o$EM|`wdu>)w`~zo#GKSl_suI28cCydT-P!!wQY^1V4}D2s z5eYk|zGWSJ`yj%!6MuroHAowazceJ z_WGJ{?XScG@Y!raP=g7G8`TE;HRF_|nd0}3=pO}d^jJ~EXoF8~|L-nCX~@!e#0;23 zB%ayzO~Y4EhSM)sMj5;&35HlX051L`7=Z(-Z_LpRaOD3K)TH;y{^v^n$mre3eWQRr zg|GuN4iq=7?hJpVNv0HE* literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/worker_require.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/worker_require.png new file mode 100644 index 0000000000000000000000000000000000000000..534fe21346fe9c65f258639dff4dec9b01607071 GIT binary patch literal 11975 zcmchdWmuH$*7pTLK^jQ`>Fyf38zhE~fuWIRK)M8_y96AN?v#*D=@g|yq@^W>j`zZS z@8{Y3c)q`1xH#a7GuFA{zkU%KY6@7-UOapB=n@T4=)@xl#{3(LI!GlGeVPu|afYvA(aOZY|0(#^KNt$>xx_@0TF_4{=j``UHw`A~xWU25JQT|?-(C99Q zLjLD2;O0&)n)r2llCCj@|9ZhX>&$<=jSl&1xMX6L8HxyH%D=zx@S(&i$m9R|DlM{7 zdnn$XpZ5Qbgbw(VE7m{5`g`Gu21fBhzWN0GpU(mVbYsbZ>1{t#; zqk?PEVt$&SOyJXbs6lklJgdT*EZ|u#7Cgb*(;sDR=RaG7SKdD_CDhe7(~hMUdAM^) zj%rZP^Jrumn0R&++RZNXzH|(D)6UXifF~kC-7oi}U z|G4N$a)-ou(|F8lX^)+fBBF*h>M(Lm|I1Y#bIa%bAex#I3}JWy>?uXL8k$N^l@a?m_=y?lRXMmh}qJpZ2RVcw`FdepNc zvV5uLKEmCvT~_gBzWUU;8t1ZK;tre{cy~E<`a>Pc-f?sM&arUo{1#Y4*7qKx{4*Kh z8Rs`Emv6b_-X4-EBLc4X_5A<5qud+xO7cUXOgr!ab8L?3JX>&vhIg?yENY6~U93&X z;D~82c`P_isRtY<9W(XpZb{sG^AEGtzPX-wUf*zkcdaLJH4#@TakGLm<-2_SGaxJM z4Y&)aB>$dEFF+u?L(2QFRz1`6hTXw+_zxcHiCUIki_>py2XXv#Oa!pYx@S8SP zFjsgh!>v?~*tl+?8ndZQ;iY%}ynR($>2769Wfi+!?mbRw;_Nn6ya`J#iR8)B2)6L= z?47IKsw$4Q)9-5VEV>z5Efc1Lj{Cdgev*pqAWXBftz<2=k`E?6+d&+Z6IoixTUmj3 zO5#MEx+c}2pI&Rhj+|_RdYl@gpyz<)?xK~~46%fGQJYd;bUtNxeLtlKt(Z3P-QPLy zxWBFL(d|uj6M;dNZx1}nEaJqzH2(_5C`BdG&)#jsEa-yX9j8tK)+H7sa$cKt>j2yK zU#F!cRsJE?b`H1=j+0T~pKUvwftMyMw# zY^r0l>ML81hjzL#ILj;BuXY09QsEWX=WU1n&mj#vrDa6H-&AtBW{=!`mQ^_0v_}P& z{gV9H6e!%a-7i+c^j1|;A+Ccv6AF|SxH4|lU$I5Bqd42Wie%uwujrsMponXCnzp{2%;Ncu=D!$r8yC}~@cI=*NVw!V0=>VQDI}qW@}zbgr&M8* ze*Lcr_7duq-(B=e5QUh=3T^)O&5toCpBWQ5gaj~npr@0RiUA&t&?;QG%UIu;cObRF z^z-XIV5glC@a3Dr=JiCCp?f@dL)*S%FTKy%oDDJYEU*{4Cv#)600@30Y33ev97$) z6))~e-s^z7lPu?XiSgTYMhOEnw0i8#y^FOtPO{SXF@WW@?SFQ6UBLha=l3kx`dE%oNmFllG;XX|9cY*fkKF8yXyRrSIMsMZ`F^~$cc;2p zJE4Q_)1=G*F{+bJj%qO$dObf=T2l0>*>55;A?N)i9F|Tk?=CZjvQx$vw|*gAkPBn1 zt4UH5F0mAz!cxL}JG`?#+$V>W(=1o*<4}&)rEBD6x)25~&hnTqwzAK}|m z@7TB!+Kc!;a=L8IOCazuKofl~C+wx+{F?R6Y`?f}*O^winb#0x?~kXr{q){Kn2i^_emDJ8xd(_? z3w2f83g(`mS<`o}F*fagx-n#8t)8F8CTkfWptqoPmq_M9CT<=d9u};%ml*g}&LCw*hvx0V=gi2J zD<5(T`CksFJy;7C)MGkS^3^`umgUrEbh`}G0q(1$es z0`8qktR4kXBNEN`;UVae#Eo&XJc#a|%gO%vF~9Jbs}UPh7RLN78ZTWXnC?~ z*$>evr6H3vEq!+H%oibwVccLeD+M0XOu!AOLOsP#b8rYl=oL4MByP8*zc`G|n;JPa zS=9p$8$^Muhg-8ZH-Jw4HaZ&niHq&L=zBV3uN?{WMh~(xcV^r}xYaqypNkwK>!QHGdp-Mr)`Y zEjuoHIU`pIt;<&r<7u-fT{@9An)yDY|pw%FE#U4y@Qd^L|wUbrg%_+o~ORe*Ej`VP^qxS9voeYm|v>q{$q zvvb$rbuGySQk9;!K85XGW-H>SxfcEkL=N$%){L)YP5pb&dvRm|WapHq}(GjBLltN`b8#*;@ z47GM~A*VLQI(kX%;l3>?{8$>jeA2rR!HdBUhD*J2*g<6VpTwg-%X5$j)P3+iY3Qp>B zgTCr4&l*sImF%SOre+mcD`BUViQ!0r_{YaHM)%VPC*u-O0v%4$tExkM+$S%F8Q=~) zQlNu$n@)GiDwkz_-_gI7T}6QuWFR`&Y5%2{UYN#tQvX(IlxUPfGO4F?kS zZ|*4Ov`(Aif4HEXAW7$|NSDa%V|u?Issf1KyxE=?I1QCHiPU-^j2_i&M^+4-UU__u z7dC>RVXr9JAdU9i13gw`8NA%@)X<-;j^TZl@5)$32e44bY{FIYR;SAkCp=hJ6{xun zgwyl$nNquS`u*vM3a7}?s*4`g&;MdJP^25_i47SFU#C{WYw4e|c%1)+G>a~>S=xXR ziqIF1Oll&w*oxzMVFpp0m+6lATDbdB_sD&p&9v)mKu^Fp2C?byqX>OJ^` zH8MmqB3Uj%ld)pJhtXcEoX~yv2JI;tvV+s@%fT0xi98u+LzT;N@0%)P z`N=R+y&K2tS1y03i8y+{M8!yQYBV zZZI?#Q?e}k#>Jb8I@lc#eY1i>>06x3bJWY#>AXj>oh@55Fff;Hhl6;s(7pWT4F!xH zITQikk=w>N*I-0sI=ly4K_!?$^T~GVb9rap3@NhYm{CcnjlX}D+=GnakUaE!dFwiG zoYC{&vHHv{)Z)=QK|LWXK_r*n`eGkq7Z?|l5L+Shk1-;!v;~~3`sznQLVI{D3f`lK z+vK=(e~jeO$2fPZLawE~v!zNTnIU~TQ(51Q8zB4Q6Tv7ITg6cQn;MW=j3V5=hcq2* zKPp_pn4m!_1Dj-Ys|pNoE?=RNlzKByiV?yV-U2Fd7@$w(3t2q$X{GAHAeRDL6 zJ4<|}#c8^r&(nAKj-Qy*bsmeq+(=eraKH+9;!GHTiPPs6MJERGGULw{j2#DlI7+p&sEjP1{_)? zu~CUds!#j89202Dl`SHEO`MItf~|h6iet!-G>OHNu1k`uutw3n^jRGydzZGNGVRBRh02nuT*6#iWWieSob2CM3k_c=(%)(GO>^fdQl%+=)TgyL z@mN-rUC4|EqPn`Aw{&FMl?C{)#AF;ZLaGoMUK}Eb(CS+LSQg+Nq!2*&q0%acB)e}uH@w@OIUC95 z{pFQr8qE;*nNJo?#Cw&5-Jgc_0h#D?BfTKRuc)jc z4Nfu12z-@QUZkF*(EX@oxWH5%_gJKTF8B4f+7eGNO5IO^EasrMiXNI&Jd4JCi~4a6 zif;uL$zdLSQE$2NtZ>h=QVb>z(b!- z@2%5=LF>(AUy^KaX6gsHtyIkw0~w}c$<%LKtT$0nxND}%Cp=i{a6BA6f>o1!inAu> z#`KxqOk0Yzq$iGs9EO)TrL`MCr%6%Ct?$Mi=de_cr?3cWV;;^PDKhBwGgQwktUk&E zK|0nu%t8~bB8oGJWpI?Orct9dykOk3*4f;wCkOn9Cc6o~k5kVsJ2OwBJTaGZMtw*r ziVR-q_&oVll3hMNK&0OnYf?8b%NHiu{sQ_t^t<&Ue7%f}$(ro1x`auK47OK(rOJuZ zS;&PZ{*VNR!QNaJ9|b@1EI>ixxb~QA?4&n8EB>}K9V4%D%>TlM{W=2>57wR`rKdxb zSHz$D(j2H{s0gg8_YtNE%_@aHLd?IwF8H{sQRq^%^*OXfv$Iu`GH)W$FQ(66Sey|k zQ5vNrtLsr7!F2D~8RX^l14E4au>^&_f!|3c+u^<|nkGK?w>0-K+ylje?8>=8cAWzb z{S$mTb2Gr|ki?-)j++o%unHPG(rxjufYLvpML$8=p;yD@+MrCfD|x!$$rCH+RBlv2 zDeUbbP`O`nOixlnoY=@`3&Xz6bf3aLn`N(qFMs~I+jP`#7LXyzT2BWBXyp6T9Rb%E8o7lSL@>l zR}-GzG2G7I@hNUD4^WxEkm|%^`FU1QkyO!=zEWu3qts#&Xe3c(GRQsoOordrskSFi$61g(c@|qnsg#rJ zMqWHuA-7sRVa-_LS$6AS4R{+90qjkmg2vao#3{AcEIVgI)?@kv`O_^%5Xn9V`B9KKK7=} zpQS(de=P-FO{f`ptNvRR{YHuGm{ip~T%_icFNRSXmv{}UX)UcD124D^vord%pR}9! zBc@!_tjZcG=0wNLTyGxZGmaXz;>>8ra|WGD?UB>2uyHE{37VUi6nyyjtI1Mr%S5QO zYtCw{-cFRE6OU%haW%m8VAAhoRL`;XsITPx>2J?swc?$81-~e}RhUL;zXDjvCX@e( z!xd!bv)f@0cAYEqOrS%meWatluN?f@>T0`l`9Z~@q3hw)=bWyQflx3Q33Pqu@hMkv zniiCBKI$bt#4DY1_}8R56&11_k2h}EOLEu-h=Z+o325KVUXYn-x__A4>&_}`{HUrZ z>2_PKvjv;e$IIlBn?YIUe}P|^ZK&N%Sj~8*@T+7il)t$EQrrL1H`(hG7rY?tGnK37 zlDnSJN^AN)AfRn<{S-X%%U9pgxMB>S$ZC!lGzT(B;~yt;Pp(B}{VSU`SsG>rXC8zI zY*Va|;S&i&iF(Q|`IFW~*Wr}F-6F)B<3YyMU!z-4wwvY0t;o$PsZf*Q%f z)=6&4{!2`>l{k5?!a3nq9}KR-@f&R7`Dcdho-eG4Gs63Acfq_U%)C#+S~dA;#zqIX z;DGeWl(~7<;)wl1Ykt>L%ZU!c28gZu0`uc3_>A#KR8Sm=8<#A*h*M#MyZj_ImawNY z=(uu&A9)na2voVvJ~QBM`kyZ~;1&jL&a@@?YeBW9hXDc5*aSP+%Nvv-8ihXX$(dLX zhiy^#WCkbXCs5>;3T>w4ExPrQ+9Y$^FU)@br<>463|KHHM_SX?r6y;J&(?!kDuvK;1@HD{kmoa`}pt zqT1xd6F7PGS~L)IE^h>NVpX_c%)5Q{{Z(-I)N_>N0sS`9tOTCcDTCg8ZS5DjF*j!?v&w_Nka@9vy}d)6G&l8pv&)? zjlVfs|8(aXQtOz3X!8eb!czp(nSzy#ANmTzp_Ud7(Ss2&lh$&^wcDDaFO+meqj+@n z7$-ZxmrWBpNj^mp2wS zMVHtPXf5e@0aMoDY`-vvB8m6-0)j0|+^KV=ob_nLN*Vk+6JASiNaNYO{kcfTf>Vp5 zEo1ZonXP>tppBMm5^%j@d(aOd#;$|(8wygtn`CM2YZk1yLnAL0o0dJh^KV>VG7DNt zG{<%3FM72|d~|^EQGatHEsq@_J~lr=Fa!eXuKL@4!0z=3kP0y$HCUu^8laRD=HT1p zb+aGSKT(9Jx(b(sa~=eF@i!igDvpeN%O% zypn^a31W_&WJX#Wz*(f{eV)9&++fdPz zwYz+I(dVyB*>!w7>P)_ZZ(pi&B!Nyc&?4+H`SVJp^Fa$zF067t2juJ1Hr!4%v)Q_P zYTSHQfdL5KLv{55_jRflshh$&D?}FQ!`@ddbuoA3JqMT74jS>kFi5O6PnOU!bLqr< zBNMt}e4yJ3#C=L~Efx!`@#U2u^z4%i z{Tk0Rj19-u8SJ$uM9c3{sO#O$Cj@b7ly;G}-`%3}^ea&MzggZ-rXzh9^UTs1;d*p? zzWle7vmw4NgnXXh#KN`_X=o!VzLx92-{IcCYyP&JI-|0vJMuBXi4NoG`I0Bp`wTGy zg$_`sMA?xkkp>Vs*~xMDQWxIh1UZ!y(jtu~5JYO=q^4=lM$?Hh5I(tFJc^QJ-0h$} zCe+z|x2FLwHg)&JLUrguCC+%{@5Bbh1Qe3s@q`?o(0Q-|g$~bG>7Z(KfhV)}lh*nY z(X}R(om=6A0+wRmOhm^Bv|dfP%1z?j$j-~Jfl-(A4>t1-&Z0ng7?tJ`IMpj>gv)$D z8wLCFGXkGs`U(GF#`O4A;$)C4zah%RwGcup-q%<)0bDFtjrb+VGVKtsYabpJKOAqT z%a}Edo9Z6zo$X^9CvoQm9idH}*T1qH=tN{AJ*$Ix%1n?K`z~p;318-t6-!)t{|!c8 zBP9QBB*=m;iYFWMkxmdaRl+BxVOt$NDK2IGs-PAV;0qL#Kuwtz{yY zrP-Ar!9U`)&{o1c?mFv`sfLRKld^X4l8%6Ks4o*8Bo}lqa@guN8cm?Oy`eQ zM|^^+?j+IEK6RtnUXB-O7ta@TyfbMqv-i{Yko{ai8kzr9pj9vEBQZw5skgyIl{L5R zQcty!qY$mCv3RMQ;XirAA@1*Rzqt(Nr$7(#P!@&$83cQBY6cqEh5}xA1t%cjn!g0K0ne_l` zuE-;#>=jYe-aM6W@L?0(Qq>0%;@8R9f1G*7H09CQU>@kQs2bef_CK0%AB6P$x`b>J?AXq0X(hm_4~@Ek8lHHVY+1e-BoA zHe|qyC^u*sot6vSb<3{DKZ4ri4Oy2Tr?#%M>=Igg}& ziX;^Ww>d>`9kp9c7<~v|C5F9h{f(e8&2&+Hak2fHq0}l<8}d|>qFB@dA7tsSu=t?v zhuT*RWpVbu%mOSFfD8f{GQQKzws7+fZxS4t!+P)#!LA|k%+6A%&lf_&siCX3J>O)% zP|Tz>TPNgq%7sia8*>3Y=TymKE)Et)%`TehVjYT7*p~o>B^TP-#2xxR)$&|H#e2{_ zWFcXjbn1>`D};`}GRk_lwxizsR<<_)op*I2zM2UUKu2z$iAe&BF*vQ;~x<-R&qbPA8veR*=1yh+WTh ziJ-QJ5RXpV1lr3bMo143=#uuqHcMxDH%TFTus6V$IZsPNgq{gts*8ewgEglrZ?%%FUhS ztsqUWOnpdaNH1y1r?ORF%XKhM&j)bVcX=FSM6;{1clf1Dq@~UiB$;o$A=C%TrTVln zySZ~$d(IUaHMn0M{>?f^{hk2@5=w?$GK1Ta|PW z$%LsDMG+tDHa4O9KaNI$YH)^%TZi>kmwqL=fu-h2?=ByaYl@Ty(3eP-yw1_ZPDS5J zXPMCYo?PJab2Fc9L|F0S_{&S$c>99>M>Z#a9Tp}MFb|nWRhdR5&sY0UM&jPr&dhJS znIt_}#=oGMHW|P?4y#8${N0y-*qbyo;JGHI|HN@f-g1E~XGA}{`V zMiGU98(F!t_`2v0IzRg`{MoEEa8gRLx znVWAy*I4m1x-%*oMi(uY<5W0pVKIG!_MW|kPxj%Pc%7|ce-GG1_}ls5*aXdCjit_q z{FmS)hIYg-#uhQ9S7n`M7V1jyu{ktviQxN7bPM$34ZmNuK!REK_2Vt&qlf9p{+$j9 zBN`PX^3NU6KrbWLE0lc0>gF0sJAPA=^=IV|g1IQ8CixpFhV06(cQ~KcEnwxT&pBpU z%E~pLIY}J#Jm0tnikM-P{?VAE0^!ZD^mY3Ybr?3+-{IX;|B^*5a+@55X+8sqOh`(- zR!+0{2AY4J_n}&-eyun!mPJ96y=H(W@5N6gexj|$yH() z7q^?E{#0IR3Wb80yS*5JpT)6aXS2D9Vn7>n>LF>4HFDFf;FaH%NAfwHP*7n@Pm&4! zDomrd!p zeqC;5alS80xcL1IfD|-tftD?&a#ZJ)t_M)x==mPDdDcH5H)IagQcck+KS=Y#jhDUv zqkjN0GK!gt0t5rFNY|bf^jbvB>83!{B&00in+o?0KxglWg^8EYiv^Xo5bu_8n?q?HETZod2VdfWX6Er;TG zV`J_qou=d+0Dtj7TIHC=Tr{8WlvQ!`_ZP%C0#+5Xe>pnw7bE)yLVoDb4{dTXU<%n; zEhYgKpFK6{0~t~F0P|e+3ABR;en?}mYg6-~#n(E|7}{5SA4zRLEy!)ACIN_;0C#}q zA=C-`zIJc6m^I0BAXrGrnBgK#?H;@qCvwmzt#(@Lw3@#OursdF3rdWc+TyufK&E3b z1yBKoizV~f(^EK~ojuv+i)L+$CfaW{IQ1SN8I|dL{bpb<8zHXO`wi-{@(r`*QPRzK zf5IeQNsh$7q9XurJW(TCctF$^PT+Ye0G1+~?TAvW-*JbiDg!dDDnpkpRHL6^Y>vQk z$yFEt?f1V;SRa5XxvDIB0<#CmkWW)_fj4W?1I4y0xp|4+V*pmgBgNIUn;H z+sA(aqWFqo&W=`TYyfqd&kglI^j(H$2Zl#wY31F(yX!_l3&~rPG2h?x;3*Iw*8$@s zteG6sf533QH&g>}toaD2nNi2R#a_^hZsJlG3wep1@qqpMnbM}sk7YNCq4ZOnY~h?W z;rWICcYg@TEY|@e&XAD%Dm_IJs(J6J z1Av=egSSNEYNp5Z<4dXh4kjpxhDdANzhVh!hf#RwQzzS3Hd#}9yT5xJf~l)$wFFvp z`6|se4uE>Ax~l(o8$bZ~vlGc$`C92iZVo^YHO>PG?Dafh#&~A`)BvKtKKvq_daY0d za4S*1pC@+ciHc2Qq|VIqEVWo7g7&CD5V-H&XN7v8en00AW4j{FlO+vHBq>jre1P8k zd+^hr!>53u=7u8@(OGDd+W`3YGof2+09y|8taY%RnFI%C&$!Z z-vQ~@N|#Vp;$Jrvu0S=Q*LYy)F8D0zDx*6@G6a@Mjg?#pobIwlx?TnJZ_d{J8)b*~+EQqJKIchm1r> zL}mN-@R~1u&^M1*i^N>_kmEAy;3!88K=l z$c-5*pP@r+3sM*acjMve2G0zF21t-krm6z!nPgTQafx91c9udsO0I9OthGUvA_w9< zYVzwf!yHESB1Hbu?LCd&?KYq1tl<>|m4pIB^c+l{1VXpJoYv*@Zz7=A4+v{AXl7>6 z%vh>yEg9*;;v|(9h~F7AFTz1*<|^9ls+GmY02rDl&cA1{CdPa_Z9Jjp<+hWi_sw}d4*X^>SzZr&{Hd0^yynoVdug ziphcq@C*Wb!9nP6jFdG+p~kHgvtpzXFJC%^jqZOk z6_O;haK`7vM3qu7bL5lVewuobKTN1Ovv$qM3v%pL_8||j@V`qD3v8qJymQW z%|?;=EN@w`RYeUMi3}2IKvvF$^vE(&K(6yokMSWVlBYvfmSnoeEo3UQNb1opIp^Q6 z`{6NpC|IPq#O2mtz85C5A9(&7Qn(86?U)c+aM hgCF?+cH{li>LAh(%4n84fJ}d+B&Q}@DQz13{{WrF725y+ literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/worker_scaleup.png b/website/www/site/static/images/blog/apache-beam-flink-and-kubernetes-part3/worker_scaleup.png new file mode 100644 index 0000000000000000000000000000000000000000..c3015f28f0d6ee5144950f834a0c16e078008e5f GIT binary patch literal 7471 zcmdUU^|>GAPv&pArjI^$N)n~NC-#_si2gAzziuR9YaVrNJw|1bcaa8ARR*u zF$hC^=kvbbd*An8xWAlrUT3ep&szJe=UHdJo`_f4s>F}zA7No(5v!{y>tSJGgE99a z06ffkl2l=9JoO+~Ii36aTWzP~knF2>@4C`D zJjSzEvi2$~{lD+SFTbT<7*?1xl-a(-r3h4JC0F||lPZIa;6kR$^k04c_n{gtJ#GSI z3;6$i{xOyy04Fk?eDW&wfBRv)=>M;Jr3T;%wx4uwb4kow2G@Y!mGvUJ?r&XM22})~ z&Hlh!IZw>FDKxv?Hxi=h*j3?ydwHe;h=6|(7WdvzWR!)Ow)9q#+o;Gu(qZG0bJ=Gt z2huD*)WX)y*@Td(bIX>&Ku(qpY2f>cn53%dj5=MP?0&|3^y4%qhGVU|^!0AR2$v9as0RIE^=}YOdoH8ryj7nlAl~sT^u_SpReF^x$Y)X! zKbh43BzTLe3Zl|7;ddmeO(S=FwY(#5#C*G}nsV83?6%->ZJr|2I%0nGi*_{y zlN$rh@@hf-crU`M3vCYF7R_0Jq~FMuuUu4mJzc%OyBwLR^lUgS67qXLzm?UcaJlS0 z;!)+ZQad8pCKmO~lwFt*m+a1Td7U~YWp}2^-=k$m?pXTN=Ki88AlXgGW3xNv;_rSn876Pw35V`PTx2lCgkiEme#|It zi+z67{BBiYSCN$evUxxbk3{Y~Gm80oGVJS)!u=Jc&kSbm5v?2@|I@^g4d%<{h>JEs z4`r5RH2hMq>u;Y>dY8r3uu<&!bymb2p~c^Bx81xjiLCpZ9R(Kqy&Ttmx~*@L9q-+r zKA1p=4|#4qv1qYMUA;Z>;#qJ)i4OoYd$=rm6fHaNE~p#0A6?fo6>OxaBH)3&j>b8o8d*%Dt*%ffa zpL%>6-9C(c4oAjrsVZFSwk(V}8^ST$Ov-r2Bae*W(o=SJY&&eE^f3n?$z3kji7?Z_ z7W$Q5EDqtY{Qa@L?c%$^u%%(ts<#s>Ln-w2`uXbBxOOk%t&q#zy&$;Onf@3qoG3dCCX+lI<+*og1 zFctgl>*w&tK}T(5$g8R~%x@MZ{5{KMgA{Fi=i1<}voFf+ie{dVD{HT1CspZVd4zgE z5vHrH$oNleU~-Yx9;TyLa8zTH*11-^0%!Gz6!-O?hX;;T9Vq?*mH=Pq?*ir-J_=g0 zPuyUS`2ud)`@aJU>A2Sya~55D`i)(T0vZ~zM!f0tXaA|Ytsp`{UQ@>!6$`GbsW|_7 z-@CIMAxu3@&|}-d2EE9$3D&s?#MD-D?#_$39ogF>LgD*{O(st1d=pjv7vHyPsHLYf`rD2s^ZocUglF+wx=%cn2j#w#c zN7jAUS>~$O-wl&7{BpeY3sbjxtH1}9G;rSuA9!8ADA&L}GA_WeJMI?vHF(yYObD8A z$|Vk#@Y%#rknLFjeBYCVfs|3vri<`$smFs-n~M3uG5ffp3EoP6df7VH@hzQH5>;3p z&!FE~86T0lE*zxUKrjK_$ug2y{@7E*6L9Ra`UHc=m5lr8Uv+5t-xr_4;(TpOT|y3eSnfVvVX&m1jf}Pv*js3R@i%czy=fXb*S;-2 z;*F?iTAat*JLChl&9-tA!i1cXd>1gNH(kA+)l*{+Pa(*q74%0&Nn$Up4pN|s$vZQg zTi>6#M7`9Ghybyxt{UiQE`3L0J5hgg1~SDqYSw*{LxIdARU_)!e2{Sl{LFv&QillzOouMV=<`82Wq2^bmq zL<4rp%?96wEqT#L?9NWgqYp$SuA?csHNHorC>s$}tt|w(0!-O{F0CASj>&Cz-$5}f z0x!R{2PTk0e&Ens(L2q1gHVt+WI{noT>x;0WXzt)T&v+U7fuw<5@a70!XulC+zlFC;Sf#1TpTf+A%~KuXt=;FUkhuKy&0|i^qBO_8 zEUmJVzA_HPI@wIO@LH)0*e^L zkd86eau<+~hPWO)=>O@SgI?agwnAKKd$sCcjoQgiLl*Ys9=mI(f%o-NJ}~!rqa4wo z0x9E_a=940qcWq857KKPlzU!FMUDITJQi$zpNwgG( z%Gs&Uke8M{^4)n}Wmq&2925S2R;DGm{5_0gQN=}cP&u8tB2D2;N@KTC6!}co>ej6^ zjg#6_vt7?;O;+T649$l*b1$d6F(v-1WOg&a5|sSC z8wdpSbK90GIOM#VksXj}xGP^-%rM|tT4XqtKM=xIn-T5~4XLq(6&m;4&eoYxqFL#G z-WKx6l_LEclxX-pIWK(#*ZeVkp_w4YyVI3E%8vz|`LmZdszoH=Kdo^%F5qUPACk6B z&X~pHkw(*Pz8iW&37UbQer*zSASNKz@tzwf^fmXrqUR&bBngkwFnp#-xULQAD}B&D z%iz-PdL~y@c3S~!0g(2D`vard9|rjss_rfMqhEcBbTyL1;~MDO67*`Bk*}g0>=FJH zSn`#Zl1Z`gi!Xg)lS=@4NzyxD%A~`K>c&IbBwrMXZ3v(oO>T8{{wOg%WOXu>*~wPS|?^w8J&>FOM(RSWRhb{z$+V|YD5N39q0KLvw+_`0Q-~G~J z2{>m=aYiuPDc|7}JoycHr`u+46!<*9$V>TX&v;V>QspNMrIl+xLv7^x@mlxoMu8+x z6K15pb#fpab|UTL**R*t^IuziV)N-aM=fj318`z@$+Fx>OyO?3Ssdg-y?y!d)5!l| zv|*gD10Y%*8%kfmWtHT-Grr3Ggs%8JUzo!pW-+BN||oFO$KVs8VJG$Swfhgjv+Zo6X`g8L`Kcs zRG@wL$-cG4Z|&uE0M{Y5>hW&zsyH?K=$jxi+_j=w33fB@V20(~A)Av0-YY4Op8&#{ zx@+j-m%tyQh=uUZ^>~?OZ3uw~J523sW}tmIcfwMfW*URlrx?q39V8|bfgf z@hT_C;{MOuAuhfrZ<&iU8Gt$#7>Hlc>kQASPy8C*pOAOBZ;vKp3I6WfL84i-1Q~!_6^(`r zyksL$4V6FeYK;!$7Fcj$!BfwV8zLyw;vhV*I!*G8Ny_}2O#2K#S5P;&E|Z4I{q`I+ zoE|Y0*S|;HJpcnfJ0Rd7uNs7t=J$;BqF)<{gtC{j2$-qZ>%+N!_p$>$-Wm$`so8#u zUmQBWCei_AhFhfUqM1u6yvq~!TObIx6$xIeTQs8;p|uoIyK7eypTLQ3@_q9J!%n9Z zqR3BZnUCQSjdpkE@=oH-Shq}C-P}A(mNUJY;P^v%u+$&1Zt#)nkJpdcC_=)on0I6a z;gSdg2Fa;Td5IrjGCgrr^{0y%R`yqi)C6bp#RmkdY9BA@qSCyEuj4q_y({df`pNNv z^UBmOXZ=)S-!)IRWx6qC+XDx6pCN$X6C+c6;3mU!M^8Y>PYWzmPMYC6M>?jr8drYj zQOc;npZmXzI*!faPX2DBaIib-a0?4ANtY01|6O4HLJoadr9xGU9ux=7Rr}E@cv0f& zwDvQ=Preix33Fd%+cKrialQ9Vg&ze?2vp?-Z`t(KFpWSPZ)`v%@@>PP|EXGD3tYmU z{^pu@QE%fvQm?}bpP(4WZnY=5&Npn~0H^bpwj>iux6kz@0?V+Ygu{&(J**V#j<`NI z1*m5>{g4BYsW>66&kGDYVr1TD^ZXvXXxGbll`lFvE`uoN7Wg7iH~Vfa5A&Pn@h5F< zJ2qjZq(tS0D{lZvu(lP^^QhuAc;;kOxsNrFLVe17=fxmZUU+rGWGmxOU5X2)C9(mb zUOKtiyi4mAZ;KXi2MN{_bVD{S_(qZ!TKbE{RB!s6iG^06c``k_Vhf3pwSD8fe z1K4Z{C?C!Xd}Ji$Q4j5=heM(njC`XJH58}*n03rr5L!HVv~_n)ob{t*z-jcMQfpSg z9KsL84o(swOmA4`=5bBk5T3ldnh1~}=zaXMYSS$$(VFAWYy683A?Btct{WQ5P1Oaf zG6H-elR@?BFoL0w;XOXa$PpMNoX>i-2W}(1x+Il!*S-mPx??`j(!>pJ7d{Q=osmhC z?{27>QXu>56pENG+m_rZjxigAh$!a=W-M0GS>}7}PlTL33+h*%ZP~2@X{cFvb>SuO z(~y>}R(5LS0Ixx?98^@zg|z1#^=i+;lq`{Df0LQXJxvEJ3LYxfCpF{;o{JAaCc2r! zZ8+Cwih7bgV}8A{O1UPg@s0+j$9~;?O|>VJ$bSE&yHQw6`_`s!zO&c;mPkJpX$BSt zcgs|T{rMx)nAU1Y(25^%Q2WMcv7Lxs2O{2+U-$?1A*iw67#t`ot(^73vmEdp$Ccv+ z;Gv<(iY1732uCbsL(K0)R>k>CSAog<%saztxoCRhIThkVku#ZLqdhMW4O`5|#pnD* z>#eS{ehTdnI6#qL6t_H+7Onp`6kqHy)03>Jo@yntx|p~hZd(qrZ<2G^soClWJH*1A z()b&4?-Z@o_oCnOkpj1gSbH^@IfH@#(I)0H0AhqVBep}&lEj^6j2BM}|D)G5*!4aX z>?t<;W;Om!26787_ozgU>?H@{K8W?xhKX;s&C|EDx=%U2!&OI@ZCVeUOpQF^mhJX`KzcK;hG80MAOZUX9ez{Brl) zkhf1ar9#H@4MM3$rN=?KwttG*x*wLid?n(F`z5 zLd-bsa=bY@Eaulo&)!vfuZhpX1!RlN`^Cl_;?4Sm+~|zIv=fpj*Q4-nnH~K4N#BMu za(r+1>6rDqB|#)REXVF7)^QMF%-O8yx3-49tWeAWJ@n7H9*( zH;)ZN8FgRHIS{k0Mn9c@e2l0rZS}9%dWaZe|x|^0L5CxQepZ!7PyH@%!44 zM1lBgtk>A40L^x@T8@*~DFl*I;Ge*TwNU5Ho|&+pk0u5;*c^ecUJ1kkPr~dTs3&PY zEu;+F7(&7vxU!E#vzcvb3C^i@_+rqVPz!cpVqAdc7ZRn=$?XX18-UHZ$))a^z)l6q zV2Osm;rfu^QO2baEel<}!l4TAmzONdsd{Mn)C?ngAhLWgnr5gm%t1QH%)RW~MCrWX#${sty=s53eP7xK z5)Uv#S0Sv|VemQu%O#(j0jkvf)lI$G2*#{RXx|-URw2M|8|}u%*xL)O-9OVZCW*u` zK3;o0ttxyaRkJMGJY2B;R9m_Uk_jI5<}s*tW|pR+?-`p70 zD?S4)L zyD426Id!Zu@|s2`p9R6cP0FYAWmKr3v2rp->GP0na=7jrMg>37;hJZzuS#Tv_=p$q zI3{&;Kvdoa=Bh;s31iP!9PUvG9Ko~k`ybsvKV!m6Dr-FD<6 z=E!iNFi6^_*MdDDahXGNk%e+toIVH`KE*X^npT@Ra)M zUAv>PyrN=!?}Xrx7e(BxLC@a`Ww`iPOGb|Bz1~##3zJ;nF$hu@%FFD+0r%-R^e23L7VSeSYSaG-u)U`=Q8nkhz zl=;9PygFdbD~oUs-zuIj)t?B5SJJIgWgLdnO9@lf?HNO&r!@RUL!iGQlf3B4%_p&< zCJf1KXU+bZ5|b}=hO-(;5B1M@C(^x4*+_f7cLbn^V2TcE3gL7@yl%($*K=O+u2641 z!^e-~3F?NgG}py##VKc(lejg*agcfs@n| zTWawOc}?>=k*g0Jje<)ll5xIk7?w=YpENzHP1oc&PCyUI+Qu|Rh4oAc#M+l6p3{}M z^+Lu6waJ43FG1Q)H1!-7Fys@>EGJ=XU|7d_A&~Jb?jv6$EST#6VL}p%x;x}i#j9W| zCHUlYLysQ@z>oA5wvFmh_D|>6XKn=JH|jX}Cqjr7sYOAxH`yFdMwM$ZhwVfjP-`6G`gw^wdw_0&43 z48SMF7tRkiir}QP2e}?Bs-mw-#!2h&`BtpL-h$HFUlrviwhpnaGv&>P?jMTGHuE>| ze%d>ghmLKMg%{X_(1r#n({fInX`iT-y*QfVZ2jKso|8mU$~2pdUgGIRC;`3DA01U0 zU@eED%=jH_3!M_DLEK^2jz0UvJos4z3;&=3UbsrX?-AKv6GEi8NcMh;gQ}+Pktn@8 zMb$#(&upCn0bVSEzuYXP8;VPxrnS(q+Tpe>_!5<$jV9R2djOxzFU#MtakrS3v8GHL z;GESC5diT*g`x4Py6q!Se|Q@=Hc-~eQ6n4F4XU7cX*TDCyX{1bQP`$0_|JLwzzKGZ z%9c~)e{>C7I6vh){YQp_f>mo>iEL4KQ0HSNFMIk5>pzo4jknA+ Date: Tue, 6 Feb 2024 11:36:00 -0800 Subject: [PATCH 45/50] Fix to take StreamingDataflowWorkerOptions from external options (#30232) --------- Co-authored-by: Arun Pandian --- .../worker/DataflowBatchWorkerHarness.java | 2 +- .../worker/DataflowWorkerHarnessHelper.java | 8 ++--- .../worker/StreamingDataflowWorker.java | 14 ++++----- .../worker/WorkerPipelineOptionsFactory.java | 15 ++++------ .../DataflowWorkerHarnessHelperTest.java | 30 ++++++++++++++++++- .../WorkerPipelineOptionsFactoryTest.java | 4 +-- 6 files changed, 48 insertions(+), 25 deletions(-) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarness.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarness.java index cc79ac6dbc0d..51127c2dc2fa 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarness.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowBatchWorkerHarness.java @@ -61,7 +61,7 @@ public static void main(String[] args) throws Exception { DataflowWorkerHarnessHelper.initializeLogging(DataflowBatchWorkerHarness.class); DataflowWorkerHarnessOptions pipelineOptions = DataflowWorkerHarnessHelper.initializeGlobalStateAndPipelineOptions( - DataflowBatchWorkerHarness.class); + DataflowBatchWorkerHarness.class, DataflowWorkerHarnessOptions.class); DataflowBatchWorkerHarness batchHarness = new DataflowBatchWorkerHarness(pipelineOptions); DataflowWorkerHarnessHelper.configureLogging(pipelineOptions); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java index c6d8d727ef4d..94c894608a47 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelper.java @@ -49,11 +49,11 @@ public final class DataflowWorkerHarnessHelper { private static final String ROOT_LOGGER_NAME = ""; private static final String PIPELINE_PATH = "PIPELINE_PATH"; - public static DataflowWorkerHarnessOptions initializeGlobalStateAndPipelineOptions( - Class workerHarnessClass) throws Exception { + public static T initializeGlobalStateAndPipelineOptions( + Class workerHarnessClass, Class harnessOptionsClass) throws Exception { /* Extract pipeline options. */ - DataflowWorkerHarnessOptions pipelineOptions = - WorkerPipelineOptionsFactory.createFromSystemProperties(); + T pipelineOptions = + WorkerPipelineOptionsFactory.createFromSystemProperties(harnessOptionsClass); pipelineOptions.setAppName(workerHarnessClass.getSimpleName()); /* Configure logging with job-specific properties. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java index 14efdcc5eb02..463ab953faee 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorker.java @@ -64,7 +64,6 @@ import org.apache.beam.runners.core.metrics.MetricsLogger; import org.apache.beam.runners.dataflow.DataflowRunner; import org.apache.beam.runners.dataflow.internal.CustomSources; -import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; import org.apache.beam.runners.dataflow.util.CloudObject; import org.apache.beam.runners.dataflow.util.CloudObjects; import org.apache.beam.runners.dataflow.worker.DataflowSystemMetrics.StreamingSystemCounterNames; @@ -472,9 +471,9 @@ public static void main(String[] args) throws Exception { JvmInitializers.runOnStartup(); DataflowWorkerHarnessHelper.initializeLogging(StreamingDataflowWorker.class); - DataflowWorkerHarnessOptions options = + StreamingDataflowWorkerOptions options = DataflowWorkerHarnessHelper.initializeGlobalStateAndPipelineOptions( - StreamingDataflowWorker.class); + StreamingDataflowWorker.class, StreamingDataflowWorkerOptions.class); DataflowWorkerHarnessHelper.configureLogging(options); checkArgument( options.isStreaming(), @@ -486,8 +485,7 @@ public static void main(String[] args) throws Exception { "%s cannot be main() class with beam_fn_api enabled", StreamingDataflowWorker.class.getSimpleName()); - StreamingDataflowWorker worker = - StreamingDataflowWorker.fromDataflowWorkerHarnessOptions(options); + StreamingDataflowWorker worker = StreamingDataflowWorker.fromOptions(options); // Use the MetricsLogger container which is used by BigQueryIO to periodically log process-wide // metrics. @@ -506,14 +504,14 @@ public static void main(String[] args) throws Exception { worker.start(); } - public static StreamingDataflowWorker fromDataflowWorkerHarnessOptions( - DataflowWorkerHarnessOptions options) throws IOException { + public static StreamingDataflowWorker fromOptions(StreamingDataflowWorkerOptions options) + throws IOException { return new StreamingDataflowWorker( Collections.emptyList(), IntrinsicMapTaskExecutorFactory.defaultFactory(), new DataflowWorkUnitClient(options, LOG), - options.as(StreamingDataflowWorkerOptions.class), + options, true, new HotKeyLogger(), Instant::now, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactory.java index 0929705c8941..a3ec8933c331 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactory.java @@ -46,17 +46,16 @@ public class WorkerPipelineOptionsFactory { * @return A {@link DataflowWorkerHarnessOptions} object configured for the Dataflow worker * harness. */ - public static DataflowWorkerHarnessOptions createFromSystemProperties() throws IOException { + public static T createFromSystemProperties( + Class harnessOptionsClass) throws IOException { ObjectMapper objectMapper = new ObjectMapper(); - DataflowWorkerHarnessOptions options; + T options; if (System.getProperties().containsKey("sdk_pipeline_options")) { // TODO: remove this method of getting pipeline options, once migration is complete. String serializedOptions = System.getProperty("sdk_pipeline_options"); LOG.info("Worker harness starting with: {}", serializedOptions); options = - objectMapper - .readValue(serializedOptions, PipelineOptions.class) - .as(DataflowWorkerHarnessOptions.class); + objectMapper.readValue(serializedOptions, PipelineOptions.class).as(harnessOptionsClass); } else if (System.getProperties().containsKey("sdk_pipeline_options_file")) { String filePath = System.getProperty("sdk_pipeline_options_file"); LOG.info("Loading pipeline options from " + filePath); @@ -64,12 +63,10 @@ public static DataflowWorkerHarnessOptions createFromSystemProperties() throws I new String(Files.readAllBytes(Paths.get(filePath)), StandardCharsets.UTF_8); LOG.info("Worker harness starting with: " + serializedOptions); options = - objectMapper - .readValue(serializedOptions, PipelineOptions.class) - .as(DataflowWorkerHarnessOptions.class); + objectMapper.readValue(serializedOptions, PipelineOptions.class).as(harnessOptionsClass); } else { LOG.info("Using empty PipelineOptions, as none were provided."); - options = PipelineOptionsFactory.as(DataflowWorkerHarnessOptions.class); + options = PipelineOptionsFactory.as(harnessOptionsClass); } // These values will not be known at job submission time and must be provided. diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java index de9d9cf7d155..c46a112c4e77 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowWorkerHarnessHelperTest.java @@ -29,6 +29,7 @@ import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; import org.apache.beam.runners.dataflow.worker.logging.DataflowWorkerLoggingMDC; +import org.apache.beam.runners.dataflow.worker.options.StreamingDataflowWorkerOptions; import org.apache.beam.runners.dataflow.worker.testing.RestoreDataflowLoggingMDC; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.RestoreSystemProperties; @@ -63,7 +64,7 @@ public void testLoggingConfiguration() throws Exception { DataflowWorkerHarnessOptions generatedOptions = DataflowWorkerHarnessHelper.initializeGlobalStateAndPipelineOptions( - DataflowBatchWorkerHarnessTest.class); + DataflowBatchWorkerHarnessTest.class, DataflowWorkerHarnessOptions.class); // Assert that the returned options are correct. assertThat(generatedOptions.getJobId(), equalTo(JOB_ID)); assertThat(generatedOptions.getWorkerId(), equalTo(WORKER_ID)); @@ -88,4 +89,31 @@ public void testParseDescriptor() throws TextFormat.ParseException { public void testParseStatusApiDescriptor() throws TextFormat.ParseException { assertNull(DataflowWorkerHarnessHelper.getStatusDescriptor()); } + + @Test + public void testStreamingStreamingConfiguration() throws Exception { + StreamingDataflowWorkerOptions pipelineOptions = + PipelineOptionsFactory.as(StreamingDataflowWorkerOptions.class); + pipelineOptions.setJobId(JOB_ID); + pipelineOptions.setWorkerId(WORKER_ID); + int activeWorkRefreshPeriodMillis = 12345; + pipelineOptions.setActiveWorkRefreshPeriodMillis(activeWorkRefreshPeriodMillis); + int stuckCommitDurationMillis = 23456; + pipelineOptions.setStuckCommitDurationMillis(stuckCommitDurationMillis); + String serializedOptions = new ObjectMapper().writeValueAsString(pipelineOptions); + File file = tmpFolder.newFile(); + Files.write(Paths.get(file.getPath()), serializedOptions.getBytes(StandardCharsets.UTF_8)); + System.setProperty("sdk_pipeline_options_file", file.getPath()); + + StreamingDataflowWorkerOptions generatedOptions = + DataflowWorkerHarnessHelper.initializeGlobalStateAndPipelineOptions( + DataflowBatchWorkerHarnessTest.class, StreamingDataflowWorkerOptions.class); + // Assert that the returned options are correct. + assertThat(generatedOptions.getJobId(), equalTo(JOB_ID)); + assertThat(generatedOptions.getWorkerId(), equalTo(WORKER_ID)); + assertThat( + generatedOptions.getActiveWorkRefreshPeriodMillis(), + equalTo(activeWorkRefreshPeriodMillis)); + assertThat(generatedOptions.getStuckCommitDurationMillis(), equalTo(stuckCommitDurationMillis)); + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactoryTest.java index f8684edfa2e4..62d38d434b8f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerPipelineOptionsFactoryTest.java @@ -52,7 +52,7 @@ public void testCreationFromSystemProperties() throws Exception { @SuppressWarnings("deprecation") // testing deprecated functionality DataflowWorkerHarnessOptions options = - WorkerPipelineOptionsFactory.createFromSystemProperties(); + WorkerPipelineOptionsFactory.createFromSystemProperties(DataflowWorkerHarnessOptions.class); assertEquals("test_worker_id", options.getWorkerId()); assertEquals("test_job_id", options.getJobId()); assertEquals(999, options.getNumWorkers()); @@ -74,7 +74,7 @@ public void testCreationWithPipelineOptionsFile() throws Exception { @SuppressWarnings("deprecation") // testing deprecated functionality DataflowWorkerHarnessOptions options = - WorkerPipelineOptionsFactory.createFromSystemProperties(); + WorkerPipelineOptionsFactory.createFromSystemProperties(DataflowWorkerHarnessOptions.class); assertEquals("test_worker_id_2", options.getWorkerId()); assertEquals("test_job_id_2", options.getJobId()); assertEquals(1000, options.getNumWorkers()); From f08058ca75cf41cc535bab2e0f5cd6ef38c801ee Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 14:23:18 -0800 Subject: [PATCH 46/50] Bump cloud.google.com/go/spanner from 1.53.1 to 1.56.0 in /sdks (#30224) Bumps [cloud.google.com/go/spanner](https://github.com/googleapis/google-cloud-go) from 1.53.1 to 1.56.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/spanner/v1.53.1...spanner/v1.56.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/spanner dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 26 +++++++++++++++++--------- sdks/go.sum | 44 ++++++++++++++++++++++++++------------------ 2 files changed, 43 insertions(+), 27 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 5f84993d3ff2..71b3f5a3c994 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -28,7 +28,7 @@ require ( cloud.google.com/go/datastore v1.15.0 cloud.google.com/go/profiler v0.4.0 cloud.google.com/go/pubsub v1.33.0 - cloud.google.com/go/spanner v1.53.1 + cloud.google.com/go/spanner v1.56.0 cloud.google.com/go/storage v1.36.0 github.com/aws/aws-sdk-go-v2 v1.24.1 github.com/aws/aws-sdk-go-v2/config v1.26.2 @@ -54,13 +54,13 @@ require ( github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.13.1 - golang.org/x/net v0.19.0 - golang.org/x/oauth2 v0.15.0 + golang.org/x/net v0.20.0 + golang.org/x/oauth2 v0.16.0 golang.org/x/sync v0.6.0 golang.org/x/sys v0.16.0 golang.org/x/text v0.14.0 - google.golang.org/api v0.154.0 - google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f + google.golang.org/api v0.157.0 + google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac google.golang.org/grpc v1.60.1 google.golang.org/protobuf v1.32.0 gopkg.in/retry.v1 v1.0.3 @@ -77,18 +77,25 @@ require ( dario.cat/mergo v1.0.0 // indirect github.com/Microsoft/hcsshim v0.11.4 // indirect github.com/containerd/log v0.1.0 // indirect + github.com/davecgh/go-spew v1.1.1 // indirect github.com/frankban/quicktest v1.14.0 // indirect github.com/go-logr/logr v1.3.0 // indirect github.com/go-logr/stdr v1.2.2 // indirect github.com/go-ole/go-ole v1.2.6 // indirect + github.com/json-iterator/go v1.1.12 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/minio/highwayhash v1.0.2 // indirect + github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd // indirect + github.com/modern-go/reflect2 v1.0.2 // indirect github.com/nats-io/jwt/v2 v2.5.3 // indirect github.com/nats-io/nkeys v0.4.7 // indirect github.com/nats-io/nuid v1.0.1 // indirect + github.com/pmezard/go-difflib v1.0.0 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect github.com/shirou/gopsutil/v3 v3.23.9 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect + github.com/stretchr/objx v0.5.0 // indirect + github.com/stretchr/testify v1.8.4 // indirect github.com/tklauser/go-sysconf v0.3.12 // indirect github.com/tklauser/numcpus v0.6.1 // indirect github.com/yusufpapurcu/wmi v1.2.3 // indirect @@ -96,12 +103,13 @@ require ( go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 // indirect go.opentelemetry.io/otel v1.21.0 // indirect go.opentelemetry.io/otel/metric v1.21.0 // indirect + go.opentelemetry.io/otel/sdk v1.21.0 // indirect go.opentelemetry.io/otel/trace v1.21.0 // indirect golang.org/x/time v0.5.0 // indirect ) require ( - cloud.google.com/go v0.110.10 // indirect + cloud.google.com/go v0.112.0 // indirect cloud.google.com/go/compute v1.23.3 // indirect cloud.google.com/go/compute/metadata v0.2.3 // indirect cloud.google.com/go/iam v1.1.5 // indirect @@ -183,8 +191,8 @@ require ( golang.org/x/crypto v0.18.0 // indirect golang.org/x/mod v0.11.0 // indirect golang.org/x/tools v0.10.0 // indirect - golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 // indirect + golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 // indirect google.golang.org/appengine v1.6.8 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20231120223509-83a465c0220f // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231127180814-3a041ad873d4 // indirect + google.golang.org/genproto/googleapis/api v0.0.0-20240122161410-6c6643bf1457 // indirect + google.golang.org/genproto/googleapis/rpc v0.0.0-20240122161410-6c6643bf1457 // indirect ) diff --git a/sdks/go.sum b/sdks/go.sum index 8bf9aad16f72..eefb376c6dd3 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -8,8 +8,8 @@ cloud.google.com/go v0.46.3/go.mod h1:a6bKKbmY7er1mI7TEI4lsAkts/mkhTSZK8w33B4RAg cloud.google.com/go v0.50.0/go.mod h1:r9sluTvynVuxRIOHXQEHMFffphuXHOMZMycpNR5e6To= cloud.google.com/go v0.52.0/go.mod h1:pXajvRH/6o3+F9jDHZWQ5PbGhn+o8w9qiu/CffaVdO4= cloud.google.com/go v0.53.0/go.mod h1:fp/UouUEsRkN6ryDKNW/Upv/JBKnv6WDthjR6+vze6M= -cloud.google.com/go v0.110.10 h1:LXy9GEO+timppncPIAZoOj3l58LIU9k+kn48AN7IO3Y= -cloud.google.com/go v0.110.10/go.mod h1:v1OoFqYxiBkUrruItNM3eT4lLByNjxmJSV/xDKJNnic= +cloud.google.com/go v0.112.0 h1:tpFCD7hpHFlQ8yPwT3x+QeXqc2T6+n6T+hmABHfDUSM= +cloud.google.com/go v0.112.0/go.mod h1:3jEEVwZ/MHU4djK5t5RHuKOA/GbLddgTdVubX1qnPD4= cloud.google.com/go/bigquery v1.0.1/go.mod h1:i/xbL2UlR5RvWAURpBYZTtm/cXjCha9lbfbpx4poX+o= cloud.google.com/go/bigquery v1.3.0/go.mod h1:PjpwJnslEMmckchkHFfq+HTD2DmtT67aNFKH1/VBDHE= cloud.google.com/go/bigquery v1.4.0/go.mod h1:S8dzgnTigyfTmLBfrtrhyYhwRxG72rYxvftPBK2Dvzc= @@ -38,8 +38,8 @@ cloud.google.com/go/pubsub v1.1.0/go.mod h1:EwwdRX2sKPjnvnqCa270oGRyludottCI76h+ cloud.google.com/go/pubsub v1.2.0/go.mod h1:jhfEVHT8odbXTkndysNHCcx0awwzvfOlguIAii9o8iA= cloud.google.com/go/pubsub v1.33.0 h1:6SPCPvWav64tj0sVX/+npCBKhUi/UjJehy9op/V3p2g= cloud.google.com/go/pubsub v1.33.0/go.mod h1:f+w71I33OMyxf9VpMVcZbnG5KSUkCOUHYpFd5U1GdRc= -cloud.google.com/go/spanner v1.53.1 h1:xNmE0SXMSxNBuk7lRZ5G/S+A49X91zkSTt7Jn5Ptlvw= -cloud.google.com/go/spanner v1.53.1/go.mod h1:liG4iCeLqm5L3fFLU5whFITqP0e0orsAW1uUSrd4rws= +cloud.google.com/go/spanner v1.56.0 h1:o/Cv7/zZ1WgRXVCd5g3Nc23ZI39p/1pWFqFwvg6Wcu8= +cloud.google.com/go/spanner v1.56.0/go.mod h1:DndqtUKQAt3VLuV2Le+9Y3WTnq5cNKrnLb/Piqcj+h0= cloud.google.com/go/storage v1.0.0/go.mod h1:IhtSnM/ZTZV8YYJWCY8RULGVqBDmpoyjwiyrjsg+URw= cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0ZeosJ0Rtdos= cloud.google.com/go/storage v1.6.0/go.mod h1:N7U0C8pVQ/+NIKOBQyamJIeKQKkZ+mxpohlUTyfDhBk= @@ -259,6 +259,7 @@ github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.9/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= +github.com/google/gofuzz v1.0.0/go.mod h1:dBl0BpW6vV/+mYPU4Po3pmUjxk6FQPldtuIdl/M65Eg= github.com/google/martian v2.1.0+incompatible h1:/CP5g8u/VJHijgedC/Legn3BAbAaWPgecwXBIDzw5no= github.com/google/martian v2.1.0+incompatible/go.mod h1:9I4somxYTbIHy5NJKHRl3wXiIaQGbYVAs8BPL6v8lEs= github.com/google/martian/v3 v3.3.2 h1:IqNFLAmvJOgVlpdEBiQbDc2EwKW77amAycfTuWKdfvw= @@ -308,6 +309,7 @@ github.com/jmespath/go-jmespath/internal/testify v1.5.1/go.mod h1:L3OGu8Wl2/fWfC github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 h1:eQGUsj2LcsLzfrHY1noKDSU7h+c9/rw9pQPwbQ9g1jQ= github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6/go.mod h1:LIAXxPvcUXwOcTIj9LSNSUpE9/eMHalTWxsP/kmWxQI= github.com/json-iterator/go v1.1.12 h1:PV8peI4a0ysnczrg+LtxykD8LfKY9ML6u2jnxaEnrnM= +github.com/json-iterator/go v1.1.12/go.mod h1:e30LSqwooZae/UwlEbR2852Gd8hjQvJoHmT4TnhNGBo= github.com/jstemmer/go-junit-report v0.0.0-20190106144839-af01ea7f8024/go.mod h1:6v2b51hI/fHJwM22ozAgKL4VKDeJcHhJFhtBdhmNjmU= github.com/jstemmer/go-junit-report v0.9.1/go.mod h1:Brl9GWCQeLvo8nXZwPNNblvFj/XSXhF0NWZEnDohbsk= github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= @@ -353,8 +355,11 @@ github.com/moby/sys/sequential v0.5.0 h1:OPvI35Lzn9K04PBbCLW0g4LcFAJgHsvXsRyewg5 github.com/moby/sys/sequential v0.5.0/go.mod h1:tH2cOOs5V9MlPiXcQzRC+eEyab644PWKGRYaaV5ZZlo= github.com/moby/term v0.5.0 h1:xt8Q1nalod/v7BqbG21f8mQPqH+xAaC9C3N3wfWbVP0= github.com/moby/term v0.5.0/go.mod h1:8FzsFHVUBGZdbDsJw/ot+X+d5HLUbvklYLJ9uGfcI3Y= +github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd h1:TRLaZ9cD/w8PVh93nsPXa1VrQ6jlwL5oN8l14QlcNfg= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= +github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe h1:iruDEfMl2E6fbMZ9s0scYfZQ84/6SPL6zC8ACM2oIL0= github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/morikuni/aec v1.0.0 h1:nP9CBfwrvYnBRgY6qfDQkygYDmYwOilePFkwzv4dU8A= @@ -418,6 +423,7 @@ github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA= github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.4.0/go.mod h1:YvHI0jy2hoMjB+UWwv71VJQ9isScKT/TqJzVSSt89Yw= +github.com/stretchr/objx v0.5.0 h1:1zr/of2m5FGMsad5YfcqgdqdWrIhu+EBEJRhR1U7z/c= github.com/stretchr/objx v0.5.0/go.mod h1:Yh+to48EsGEfYuaHDzXPcE3xhTkx73EhmCGUpEOglKo= github.com/stretchr/testify v1.2.0/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= @@ -479,6 +485,8 @@ go.opentelemetry.io/otel v1.21.0 h1:hzLeKBZEL7Okw2mGzZ0cc4k/A7Fta0uoPgaJCr8fsFc= go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= go.opentelemetry.io/otel/metric v1.21.0 h1:tlYWfeo+Bocx5kLEloTjbcDwBuELRrIFxwdQ36PlJu4= go.opentelemetry.io/otel/metric v1.21.0/go.mod h1:o1p3CA8nNHW8j5yuQLdc1eeqEaPfzug24uvsyIEJRWM= +go.opentelemetry.io/otel/sdk v1.21.0 h1:FTt8qirL1EysG6sTQRZ5TokkU8d0ugCj8htOgThZXQ8= +go.opentelemetry.io/otel/sdk v1.21.0/go.mod h1:Nna6Yv7PWTdgJHVRD9hIYywQBRx7pbox6nwBnZIxl/E= go.opentelemetry.io/otel/trace v1.21.0 h1:WD9i5gzvoUPuXIXH24ZNBudiarZDKuekPqi/E8fpfLc= go.opentelemetry.io/otel/trace v1.21.0/go.mod h1:LGbsEB0f9LGjN+OZaQQ26sohbOmiMR+BaslueVtS/qQ= golang.org/x/crypto v0.0.0-20180723164146-c126467f60eb/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= @@ -550,15 +558,15 @@ golang.org/x/net v0.0.0-20201110031124-69a78807bb2b/go.mod h1:sp8m0HH+o8qH0wwXwY golang.org/x/net v0.0.0-20210226172049-e18ecbb05110/go.mod h1:m0MpNAwzfU5UDzcl9v0D8zg8gWTRqZa9RBIspLL5mdg= golang.org/x/net v0.0.0-20211112202133-69e39bad7dc2/go.mod h1:9nx3DQGgdP8bBQD5qxJ1jj9UTztislL4KSBs9R2vV5Y= golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug+ECip1KBveYUHfp+8e9klMJ9c= -golang.org/x/net v0.19.0 h1:zTwKpTd2XuCqf8huc7Fo2iSy+4RHPd10s4KzeTnVr1c= -golang.org/x/net v0.19.0/go.mod h1:CfAk/cbD4CthTvqiEl8NpboMuiuOYsAr/7NOjZJtv1U= +golang.org/x/net v0.20.0 h1:aCL9BSgETF1k+blQaYUBx9hJ9LOGP3gAVemcZlf1Kpo= +golang.org/x/net v0.20.0/go.mod h1:z8BVo6PvndSri0LbOE3hAn0apkU+1YvI6E70E9jsnvY= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20190604053449-0f29369cfe45/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20191202225959-858c2ad4c8b6/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/oauth2 v0.0.0-20200107190931-bf48bf16ab8d/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= -golang.org/x/oauth2 v0.15.0 h1:s8pnnxNVzjWyrvYdFUQq5llS1PX2zhPXmccZv99h7uQ= -golang.org/x/oauth2 v0.15.0/go.mod h1:q48ptWNTY5XWf+JNten23lcvHpLJ0ZSxF5ttTHKVCAM= +golang.org/x/oauth2 v0.16.0 h1:aDkGMBSYxElaoP81NpoUoz2oo2R2wHdZpGToUxfyQrQ= +golang.org/x/oauth2 v0.16.0/go.mod h1:hqZ+0LWXsiVoZpeld6jVt06P3adbS2Uu911W1SsJv2o= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= @@ -663,8 +671,8 @@ golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2 h1:H2TDz8ibqkAF6YGhCdN3jS9O0/s90v0rJh3X/OLHEUk= -golang.org/x/xerrors v0.0.0-20220907171357-04be3eba64a2/go.mod h1:K8+ghG5WaK9qNqU5K3HdILfMLy1f3aNYFI/wnl100a8= +golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028 h1:+cNy6SZtPcJQH3LJVLOSmiC7MMxXNOb3PU/VUEz+EhU= +golang.org/x/xerrors v0.0.0-20231012003039-104605ab7028/go.mod h1:NDW/Ps6MPRej6fsCIbMTohpP40sJ/P/vI1MoTEGwX90= gonum.org/v1/gonum v0.11.0 h1:f1IJhK4Km5tBJmaiJXtk/PkL4cdVX6J+tGiM187uT5E= google.golang.org/api v0.4.0/go.mod h1:8k5glujaEP+g9n7WNsDg8QP6cUVNI86fCNMcbazEtwE= google.golang.org/api v0.7.0/go.mod h1:WtwebWUNSVBH/HAw79HIFXZNqEvBhG+Ra+ax0hx3E3M= @@ -675,8 +683,8 @@ google.golang.org/api v0.14.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsb google.golang.org/api v0.15.0/go.mod h1:iLdEw5Ide6rF15KTC1Kkl0iskquN2gFfn9o9XIsbkAI= google.golang.org/api v0.17.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= google.golang.org/api v0.18.0/go.mod h1:BwFmGc8tA3vsd7r/7kR8DY7iEEGSU04BFxCo5jP/sfE= -google.golang.org/api v0.154.0 h1:X7QkVKZBskztmpPKWQXgjJRPA2dJYrL6r+sYPRLj050= -google.golang.org/api v0.154.0/go.mod h1:qhSMkM85hgqiokIYsrRyKxrjfBeIhgl4Z2JmeRkYylc= +google.golang.org/api v0.157.0 h1:ORAeqmbrrozeyw5NjnMxh7peHO0UzV4wWYSwZeCUb20= +google.golang.org/api v0.157.0/go.mod h1:+z4v4ufbZ1WEpld6yMGHyggs+PmAHiaLNj5ytP3N01g= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.5.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= @@ -702,12 +710,12 @@ google.golang.org/genproto v0.0.0-20200204135345-fa8e72b47b90/go.mod h1:GmwEX6Z4 google.golang.org/genproto v0.0.0-20200212174721-66ed5ce911ce/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200224152610-e50cd9704f63/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c= google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo= -google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f h1:Vn+VyHU5guc9KjB5KrjI2q0wCOWEOIh0OEsleqakHJg= -google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f/go.mod h1:nWSwAFPb+qfNJXsoeO3Io7zf4tMSfN8EA8RlDA04GhY= -google.golang.org/genproto/googleapis/api v0.0.0-20231120223509-83a465c0220f h1:2yNACc1O40tTnrsbk9Cv6oxiW8pxI/pXj0wRtdlYmgY= -google.golang.org/genproto/googleapis/api v0.0.0-20231120223509-83a465c0220f/go.mod h1:Uy9bTZJqmfrw2rIBxgGLnamc78euZULUBrLZ9XTITKI= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231127180814-3a041ad873d4 h1:DC7wcm+i+P1rN3Ff07vL+OndGg5OhNddHyTA+ocPqYE= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231127180814-3a041ad873d4/go.mod h1:eJVxU6o+4G1PSczBr85xmyvSNYAKvAYgkub40YGomFM= +google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac h1:ZL/Teoy/ZGnzyrqK/Optxxp2pmVh+fmJ97slxSRyzUg= +google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac/go.mod h1:+Rvu7ElI+aLzyDQhpHMFMMltsD6m7nqpuWDd2CwJw3k= +google.golang.org/genproto/googleapis/api v0.0.0-20240122161410-6c6643bf1457 h1:KHBtwE+eQc3+NxpjmRFlQ3pJQ2FNnhhgB9xOV8kyBuU= +google.golang.org/genproto/googleapis/api v0.0.0-20240122161410-6c6643bf1457/go.mod h1:4jWUdICTdgc3Ibxmr8nAJiiLHwQBY0UI0XZcEMaFKaA= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240122161410-6c6643bf1457 h1:6Bi3wdn5Ed9baJn7P0gOhjwA98wOr6uSPjKagPHOVsE= +google.golang.org/genproto/googleapis/rpc v0.0.0-20240122161410-6c6643bf1457/go.mod h1:PAREbraiVEVGVdTZsVWjSbbTtSyGbAgIIvni8a8CD5s= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.20.1/go.mod h1:10oTOabMzJvdu6/UiuZezV6QK5dSlG84ov/aaiqXj38= google.golang.org/grpc v1.21.1/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= From 7a46686577d5c7cc67ed4a98d09cecd10a039d15 Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Tue, 6 Feb 2024 17:39:56 -0800 Subject: [PATCH 47/50] Fix a naming conflict of the TransformService test suite (#30243) --- .github/workflows/beam_PostCommit_TransformService_Direct.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 0cb4352c2fbb..1497681ae991 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -91,7 +91,7 @@ jobs: uses: actions/upload-artifact@v4 if: failure() with: - name: Python Test Results + name: Python ${{ matrix.python_version }} Test Results path: '**/pytest*.xml' - name: Publish Python Test Results uses: EnricoMi/publish-unit-test-result-action@v2 From 4645a76ee2a4b04f93ddb91b5f0a94d6713021fc Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Wed, 7 Feb 2024 10:06:58 -0500 Subject: [PATCH 48/50] gcsio: reduce number of get requests in function calls (#30205) * Reduce the number of get requests in gcsio. * Apply formatter. * Replace get_bucket with bucket in _gcs_object --- sdks/python/apache_beam/io/gcp/gcsio.py | 51 +++++------- sdks/python/apache_beam/io/gcp/gcsio_test.py | 85 +++++++++++++++----- 2 files changed, 86 insertions(+), 50 deletions(-) diff --git a/sdks/python/apache_beam/io/gcp/gcsio.py b/sdks/python/apache_beam/io/gcp/gcsio.py index a6ba82a6e07c..b5a291428767 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio.py +++ b/sdks/python/apache_beam/io/gcp/gcsio.py @@ -175,17 +175,14 @@ def open( ValueError: Invalid open file mode. """ bucket_name, blob_name = parse_gcs_path(filename) - bucket = self.client.get_bucket(bucket_name) + bucket = self.client.bucket(bucket_name) if mode == 'r' or mode == 'rb': - blob = bucket.get_blob(blob_name) + blob = bucket.blob(blob_name) return BeamBlobReader(blob, chunk_size=read_buffer_size) elif mode == 'w' or mode == 'wb': - blob = bucket.get_blob(blob_name) - if not blob: - blob = storage.Blob(blob_name, bucket) + blob = bucket.blob(blob_name) return BeamBlobWriter(blob, mime_type) - else: raise ValueError('Invalid file open mode: %s.' % mode) @@ -199,7 +196,7 @@ def delete(self, path): """ bucket_name, blob_name = parse_gcs_path(path) try: - bucket = self.client.get_bucket(bucket_name) + bucket = self.client.bucket(bucket_name) bucket.delete_blob(blob_name) except NotFound: return @@ -228,16 +225,15 @@ def delete_batch(self, paths): with current_batch: for path in current_paths: bucket_name, blob_name = parse_gcs_path(path) - bucket = self.client.get_bucket(bucket_name) + bucket = self.client.bucket(bucket_name) bucket.delete_blob(blob_name) for i, path in enumerate(current_paths): error_code = None - for j in range(2): - resp = current_batch._responses[2 * i + j] - if resp.status_code >= 400 and resp.status_code != 404: - error_code = resp.status_code - break + resp = current_batch._responses[i] + if resp.status_code >= 400 and resp.status_code != 404: + error_code = resp.status_code + break final_results.append((path, error_code)) s += MAX_BATCH_OPERATION_SIZE @@ -258,11 +254,9 @@ def copy(self, src, dest): """ src_bucket_name, src_blob_name = parse_gcs_path(src) dest_bucket_name, dest_blob_name= parse_gcs_path(dest, object_optional=True) - src_bucket = self.get_bucket(src_bucket_name) - src_blob = src_bucket.get_blob(src_blob_name) - if not src_blob: - raise NotFound("Source %s not found", src) - dest_bucket = self.get_bucket(dest_bucket_name) + src_bucket = self.client.bucket(src_bucket_name) + src_blob = src_bucket.blob(src_blob_name) + dest_bucket = self.client.bucket(dest_bucket_name) if not dest_blob_name: dest_blob_name = None src_bucket.copy_blob(src_blob, dest_bucket, new_name=dest_blob_name) @@ -291,19 +285,18 @@ def copy_batch(self, src_dest_pairs): for pair in current_pairs: src_bucket_name, src_blob_name = parse_gcs_path(pair[0]) dest_bucket_name, dest_blob_name = parse_gcs_path(pair[1]) - src_bucket = self.client.get_bucket(src_bucket_name) - src_blob = src_bucket.get_blob(src_blob_name) - dest_bucket = self.client.get_bucket(dest_bucket_name) + src_bucket = self.client.bucket(src_bucket_name) + src_blob = src_bucket.blob(src_blob_name) + dest_bucket = self.client.bucket(dest_bucket_name) src_bucket.copy_blob(src_blob, dest_bucket, dest_blob_name) for i, pair in enumerate(current_pairs): error_code = None - for j in range(4): - resp = current_batch._responses[4 * i + j] - if resp.status_code >= 400: - error_code = resp.status_code - break + resp = current_batch._responses[i] + if resp.status_code >= 400: + error_code = resp.status_code + break final_results.append((pair[0], pair[1], error_code)) s += MAX_BATCH_OPERATION_SIZE @@ -417,12 +410,12 @@ def _gcs_object(self, path): """Returns a gcs object for the given path This method does not perform glob expansion. Hence the given path must be - for a single GCS object. + for a single GCS object. The method will make HTTP requests. Returns: GCS object. """ bucket_name, blob_name = parse_gcs_path(path) - bucket = self.client.get_bucket(bucket_name) + bucket = self.client.bucket(bucket_name) blob = bucket.get_blob(blob_name) if blob: return blob @@ -470,7 +463,7 @@ def list_files(self, path, with_metadata=False): _LOGGER.debug("Starting the file information of the input") else: _LOGGER.debug("Starting the size estimation of the input") - bucket = self.client.get_bucket(bucket_name) + bucket = self.client.bucket(bucket_name) response = self.client.list_blobs(bucket, prefix=prefix) for item in response: file_name = 'gs://%s/%s' % (item.bucket.name, item.name) diff --git a/sdks/python/apache_beam/io/gcp/gcsio_test.py b/sdks/python/apache_beam/io/gcp/gcsio_test.py index f8b580c91c95..c9a7fb72f779 100644 --- a/sdks/python/apache_beam/io/gcp/gcsio_test.py +++ b/sdks/python/apache_beam/io/gcp/gcsio_test.py @@ -43,9 +43,15 @@ class FakeGcsClient(object): def __init__(self): self.buckets = {} + def _add_bucket(self, bucket): + self.buckets[bucket.name] = bucket + return self.buckets[bucket.name] + + def bucket(self, name): + return FakeBucket(self, name) + def create_bucket(self, name): - self.buckets[name] = FakeBucket(self, name) - return self.buckets[name] + return self._add_bucket(self.bucket(name)) def get_bucket(self, name): if name in self.buckets: @@ -92,40 +98,51 @@ def __init__(self, client, name): self.name = name self.blobs = {} self.default_kms_key_name = None - self.client.buckets[name] = self - def add_blob(self, blob): - self.blobs[blob.name] = blob + def _get_canonical_bucket(self): + return self.client.get_bucket(self.name) - def create_blob(self, name): + def _create_blob(self, name): return FakeBlob(name, self) + def add_blob(self, blob): + bucket = self._get_canonical_bucket() + bucket.blobs[blob.name] = blob + return bucket.blobs[blob.name] + + def blob(self, name): + return self._create_blob(name) + def copy_blob(self, blob, dest, new_name=None): + if self.get_blob(blob.name) is None: + raise NotFound("source blob not found") if not new_name: new_name = blob.name - dest.blobs[new_name] = blob - dest.blobs[new_name].name = new_name - dest.blobs[new_name].bucket = dest - return dest.blobs[new_name] + new_blob = FakeBlob(new_name, dest) + dest.add_blob(new_blob) + return new_blob def get_blob(self, blob_name): - if blob_name in self.blobs: - return self.blobs[blob_name] + bucket = self._get_canonical_bucket() + if blob_name in bucket.blobs: + return bucket.blobs[blob_name] else: return None def lookup_blob(self, name): - if name in self.blobs: - return self.blobs[name] + bucket = self._get_canonical_bucket() + if name in bucket.blobs: + return bucket.blobs[name] else: - return self.create_blob(name) + return bucket.create_blob(name) def set_default_kms_key_name(self, name): self.default_kms_key_name = name def delete_blob(self, name): - if name in self.blobs: - del self.blobs[name] + bucket = self._get_canonical_bucket() + if name in bucket.blobs: + del bucket.blobs[name] class FakeBlob(object): @@ -151,11 +168,18 @@ def __init__( self.updated = updated self._fail_when_getting_metadata = fail_when_getting_metadata self._fail_when_reading = fail_when_reading - self.bucket.add_blob(self) def delete(self): - if self.name in self.bucket.blobs: - del self.bucket.blobs[self.name] + self.bucket.delete_blob(self.name) + + def download_as_bytes(self, **kwargs): + blob = self.bucket.get_blob(self.name) + if blob is None: + raise NotFound("blob not found") + return blob.contents + + def __eq__(self, other): + return self.bucket.get_blob(self.name) is other.bucket.get_blob(other.name) @unittest.skipIf(NotFound is None, 'GCP dependencies are not installed') @@ -224,6 +248,7 @@ def _insert_random_file( updated=updated, fail_when_getting_metadata=fail_when_getting_metadata, fail_when_reading=fail_when_reading) + bucket.add_blob(blob) return blob def setUp(self): @@ -475,7 +500,25 @@ def test_list_prefix(self): def test_downloader_fail_non_existent_object(self): file_name = 'gs://gcsio-metrics-test/dummy_mode_file' with self.assertRaises(NotFound): - self.gcs.open(file_name, 'r') + with self.gcs.open(file_name, 'r') as f: + f.read(1) + + def test_blob_delete(self): + file_name = 'gs://gcsio-test/delete_me' + file_size = 1024 + bucket_name, blob_name = gcsio.parse_gcs_path(file_name) + # Test deletion of non-existent file. + bucket = self.client.get_bucket(bucket_name) + self.gcs.delete(file_name) + + self._insert_random_file(self.client, file_name, file_size) + self.assertTrue(blob_name in bucket.blobs) + + blob = bucket.get_blob(blob_name) + self.assertIsNotNone(blob) + + blob.delete() + self.assertFalse(blob_name in bucket.blobs) if __name__ == '__main__': From f250a8432305feace7d8968739c6004d869a4eaf Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 6 Feb 2024 13:31:33 -0500 Subject: [PATCH 49/50] Adjust transform-service module names and deps with sdks/java/core --- sdks/java/transform-service/app/build.gradle | 2 +- sdks/java/transform-service/build.gradle | 2 +- sdks/java/transform-service/launcher/build.gradle | 7 ++----- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/sdks/java/transform-service/app/build.gradle b/sdks/java/transform-service/app/build.gradle index 5125899bec9b..bf7a47a462e2 100644 --- a/sdks/java/transform-service/app/build.gradle +++ b/sdks/java/transform-service/app/build.gradle @@ -20,7 +20,7 @@ apply plugin: 'org.apache.beam.module' apply plugin: 'application' applyJavaNature( - automaticModuleName: 'org.apache.beam.sdk.transform.service.app', + automaticModuleName: 'org.apache.beam.sdk.transformservice.app', exportJavadoc: false, validateShadowJar: false, shadowClosure: {}, diff --git a/sdks/java/transform-service/build.gradle b/sdks/java/transform-service/build.gradle index 1d72c1d424bb..2fee0b4f1708 100644 --- a/sdks/java/transform-service/build.gradle +++ b/sdks/java/transform-service/build.gradle @@ -19,7 +19,7 @@ apply plugin: 'org.apache.beam.module' applyJavaNature( - automaticModuleName: 'org.apache.beam.sdk.transform.service', + automaticModuleName: 'org.apache.beam.sdk.transformservice', exportJavadoc: false, validateShadowJar: false, shadowClosure: {}, diff --git a/sdks/java/transform-service/launcher/build.gradle b/sdks/java/transform-service/launcher/build.gradle index e225d8da2f1d..5ae514af1dc9 100644 --- a/sdks/java/transform-service/launcher/build.gradle +++ b/sdks/java/transform-service/launcher/build.gradle @@ -18,7 +18,7 @@ apply plugin: 'org.apache.beam.module' applyJavaNature( - automaticModuleName: 'org.apache.beam.sdk.transform.service', + automaticModuleName: 'org.apache.beam.sdk.transformservice.launcher', ) description = "Apache Beam :: SDKs :: Java :: Transform Service :: Launcher" @@ -28,9 +28,7 @@ ext.summary = """Contains code that can be used to run an transform service.""" // Exclude tests that need a runner test { systemProperty "beamUseDummyRunner", "true" - useJUnit { - excludeCategories "org.apache.beam.sdk.testing.NeedsRunner" - } + useJUnit { } } dependencies { @@ -42,7 +40,6 @@ dependencies { permitUnusedDeclared(library.java.error_prone_annotations) testImplementation library.java.junit testImplementation library.java.mockito_core - testImplementation project(path: ":sdks:java:core") } sourceSets { From 4f3963f7878534172b2ea094f069745f92f0b307 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 7 Feb 2024 13:07:28 -0500 Subject: [PATCH 50/50] Bump google.golang.org/grpc from 1.60.1 to 1.61.0 in /sdks (#30249) Bumps [google.golang.org/grpc](https://github.com/grpc/grpc-go) from 1.60.1 to 1.61.0. - [Release notes](https://github.com/grpc/grpc-go/releases) - [Commits](https://github.com/grpc/grpc-go/compare/v1.60.1...v1.61.0) --- updated-dependencies: - dependency-name: google.golang.org/grpc dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 71b3f5a3c994..8683c181a99c 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -61,7 +61,7 @@ require ( golang.org/x/text v0.14.0 google.golang.org/api v0.157.0 google.golang.org/genproto v0.0.0-20240116215550-a9fa1716bcac - google.golang.org/grpc v1.60.1 + google.golang.org/grpc v1.61.0 google.golang.org/protobuf v1.32.0 gopkg.in/retry.v1 v1.0.3 gopkg.in/yaml.v2 v2.4.0 @@ -138,7 +138,7 @@ require ( github.com/census-instrumentation/opencensus-proto v0.4.1 // indirect github.com/cespare/xxhash/v2 v2.2.0 // indirect github.com/cncf/udpa/go v0.0.0-20220112060539-c52dc94e7fbe // indirect - github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4 // indirect + github.com/cncf/xds/go v0.0.0-20231109132714-523115ebc101 // indirect github.com/containerd/containerd v1.7.11 // indirect github.com/cpuguy83/dockercfg v0.3.1 // indirect github.com/docker/distribution v2.8.2+incompatible // indirect diff --git a/sdks/go.sum b/sdks/go.sum index eefb376c6dd3..acb53d867165 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -146,8 +146,8 @@ github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGX github.com/cncf/udpa/go v0.0.0-20220112060539-c52dc94e7fbe h1:QQ3GSy+MqSHxm/d8nCtnAiZdYFd45cYZPs8vOOIYKfk= github.com/cncf/udpa/go v0.0.0-20220112060539-c52dc94e7fbe/go.mod h1:6pvJx4me5XPnfI9Z40ddWsdw2W/uZgQLFXToKeRcDiI= github.com/cncf/xds/go v0.0.0-20210922020428-25de7278fc84/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= -github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4 h1:/inchEIKaYC1Akx+H+gqO04wryn5h75LSazbRlnya1k= -github.com/cncf/xds/go v0.0.0-20230607035331-e9ce68804cb4/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= +github.com/cncf/xds/go v0.0.0-20231109132714-523115ebc101 h1:7To3pQ+pZo0i3dsWEbinPNFs5gPSBOsJtx3wTT94VBY= +github.com/cncf/xds/go v0.0.0-20231109132714-523115ebc101/go.mod h1:eXthEFrGJvWHgFFCl3hGmgk+/aYT6PnTQLykKQRLhEs= github.com/colinmarc/hdfs/v2 v2.1.1/go.mod h1:M3x+k8UKKmxtFu++uAZ0OtDU8jR3jnaZIAc6yK4Ue0c= github.com/containerd/containerd v1.7.11 h1:lfGKw3eU35sjV0aG2eYZTiwFEY1pCzxdzicHP3SZILw= github.com/containerd/containerd v1.7.11/go.mod h1:5UluHxHTX2rdvYuZ5OJTC5m/KJNs0Zs9wVoJm9zf5ZE= @@ -725,8 +725,8 @@ google.golang.org/grpc v1.26.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8 google.golang.org/grpc v1.27.0/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.27.1/go.mod h1:qbnxyOmOxrQa7FizSgH+ReBfzJrCY1pSN7KXBS8abTk= google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc= -google.golang.org/grpc v1.60.1 h1:26+wFr+cNqSGFcOXcabYC0lUVJVRa2Sb2ortSK7VrEU= -google.golang.org/grpc v1.60.1/go.mod h1:OlCHIeLYqSSsLi6i49B5QGdzaMZK9+M7LXN2FKz4eGM= +google.golang.org/grpc v1.61.0 h1:TOvOcuXn30kRao+gfcvsebNEa5iZIiLkisYEkf7R7o0= +google.golang.org/grpc v1.61.0/go.mod h1:VUbo7IFqmF1QtCAstipjG0GIoq49KvMe9+h1jFLBNJs= google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM=
  2. !2=-P}~s9$*Xv?o$0ELo%bf1nojX`PN&x5j;iVoKj;LCz36?!s9U1c&XruxFbW ze)nwKw45&eT2geWTAN(3>2m*3UabXR$NTMG#1>As-KgBhwE&xwd^-E0&gQ{hh8nH* z>7!9{nMOhP)6;cFRqL z6{8%Mmakg$F)jVt`-|S$KIa{-#|!o^s|y=ksSd{Y=`?R|qlcsm$f10*!~X&8GvJB$ ze+{E^lt^xuz(+t)~IMW^Mw2QCp8xHnH4BA%?yniRq_@x%KL_w zR7)L>VdyI+Y~~quwq4dbf0w}d0&*MH%8{TYTFJc_l(Fk$ANx@N;bPY!6gM;(d{r@D zfyjVr6Au}jdCB!v(`*HZT5V%(p+Q$&Ib(8>=Q;%#M+4gi@c)j0Ga#oQO#}==M!kx#G|~mFWwOXfZOj4 zTg{^ab0D4c(zt>8q6JwqzC0sa3utVroMejz?u{5)K~aD*`-3ybm?O02w}5!OgF$>AL-hQMf1J)}q$ZMdiqGv2_lli`IGs2}{rMKC`GN%}QiG31f*U6m? z73^xfpW=vHrmNjb^PF9ku%;Fh`QbicEI-M;19$J!Kv=y&d;EzeaTpP#Hg(VRJW4#n zBAU`~cOteiu1LiyhAxv_*U0G}*W*~!fPyXM>{bm(g*er4;*EoZ)(l{y>2-_9IpX>f zBAGc#Y5odhR42Wt#Z1@9Ev*&C z7PTSv!<%4n5Ie* z5b?9%{_$GL{8GZ~y}6t#CKKc7C4TU~Wk9Gi>av5tTFsY%k13!6cn=(s>Fq*lr`FxS zEN)O$U_riYbe_?bYE^Ytnh{V$a>ebjcfXp1sXg#9FL=0XCE-q6Y%>EPF{Wdptzh+{ zLX>%G#}V%1Bf@+veroWKYnLye8;xaV_Nm^&t&4Q0Cbr7T`UDsYY;>2dq0(v<+i@F9q$;EveDqz($6Gmr=a+AWIp&lgA!lH!oS(j^vtk*k(z;+=a*!gHEf9=|0z0rl$g!D|8E72* z9Di^|g{D`Aq!Gk*w(%S=$^JD7DeG}ve2!X8@4Bl_CRljG-UL5NvPc!@8r0iH!FaWLd&_8F`m?`N`Ey$$i( zY-6IFQrfvabUzkRW6rXO*^?rgEhAHJt(n5GMBMlr@#n6g!NZkKc#d)*>qe#5)bTLBa((}&21IerBoN+nk18lTy~ zEl&5M?J5?c$~9b`+wV15foySSdCs0%+hMjn#MZ51g3Jdq~M zMo0$HJiW@4#nua5M?1j9Dk|2s-!_GM ztzINkV=xG5ARYJr!?ggkpmI2^IYyJxV%oVNEb;2GXfc#-V z2p)t(@pW4Dv~&#{Ox5hLpT%IxFPPf@#l2E}no+i}({rBZVl-m_f+?aRmMUWEY-scC|WV6Tnn+EXWs&wg@izG=b&sW3I7c^|HR=c3`8P^Km zPE}=tRCtG~y6!&v*~p-K7V_)IQ8ame*Sw?*NU-k&*-gs-M-MM;{anH!M6GLoXkJ)@ zWH(`!U6;@Mq{3;qC!>PQAnZhEoV!iJYt3V1z@}G2qz5W*eOrU5IgC?nY!0>TxJLSj zDI#^i2XZyPijrgCpcDetQX32f!xIXhZzDwYU444owN-kd^M>TBzLqx9z$PTuPc559 zD+d2*ugdI4|9o`mVdpfyPCe%4L}gZSbaK^nM;^urCtKC4RNG@NklTnHqn7d@U*J~QlS%vr{=^h>NNfqhP+h_%W6N@gL>gm?bvP>wrk;+wppfCmD z?pW~znw_pAEk13j<*yr0pt)XkDjVNeTV&kEWg@WYxIUpi+Ji11DeV zY3@~kc|*a?jJmo%W2=TbNDM_$<}FNEdf`DZyGM4Atzn~cC#> z^*0DM^X<(C#1)CxW;sO`{uUCHMjOvSuUKIYqt=jmN^|6OUii)VNaG5|Gw&Zcbs>Ob z2x~7$cywICBmVt4yV7V`+a~{(<)3H2*2DnSU^h``@8XZEfuoymR|3-c<)5Y)8)w)g z1u0wT$I3{NYoSv~F;?o!-m9?)_i9A0-I}-c%RisC0@150$4_;E*mL$gI4UE>-xwE8kjYGALi7kGB|jK9~)gZIh^DU zgz|mgY!GR*D>|PyFF!96hx&4@4Tyt!#Cm+jP+`Dk zZks!-OLzAG*q=zdcPC~=c|D2LftFB|(qeTTpl0)L2g$xtr25IxZrq}-TD@KZ*X0*r zx|B?@IOUbc2lJ1w^t#ob{Wq-j(U#AVI!X9(hTfj*MUTI=gM*-^@PnoR!c2+vTmgEX z!z%f$JFBP6CzSKEZz<=T8blSldj2MUBx9acIX4WliCRbF=PI4K#N8XOiCF&Ljh35m z)wr%HT%385raAD*n39t)56hM604bU6Ow26!P_T_RLvj24exlkIxQJn48EDhGnUT~y z$pGTeWCkW6nvkL8+1RTEE>HUdreZjU+z6Vb-2;MuFE(wCkNg%!G$VgjB0 zPQ9z09&Cs6Id4LqAkQ{;c~*H)O^nL_1vN>6*pQ}Uhe;WgG{dx6ssiqfukTNyTOSCH z)sz{;pC)u5;5yeJ%0zNBC?<{#$ms>oeW&DamDAJx*cCmW<&{%LRcHexrFZXLgb)jf z)<%N;H|Sb;x0jM+ex>Y)wb;dU3k{s1X--uZj28^yw=P1*azz@^&M>t%s$7pcgEGAy z%GZ=Mpkh8;1$8zlBUuOHW$eBwNSCj`fuk%F4f$*Q$Kn9%-F6pFWYkFZ5IJ!_){37a zYS(3oysd!YF(^c&Sda!j4BmxnYsvSMb}g%BsKj*JV`0ti1>W2J6_%@)t{#q%b({kY zm0N@VKyBNkJ=GZKabEVv8|oy}-STAULx7Hhi@O!&sNxp;sdzKK8-9~#8i4dU^>c4t zNy!QL%kK3X7%h7XOH#M_y&0r&2If7`nk9QADg2&^ZJ-+UEIjY{{;O!VR5LJ0mSUsA zaga^J^`eak5yb9#TprN7Y6v;=O}!9$vGlENeWqkB;pE)}0-uN1%X^tvubw}~scdKl z(^=#;adR3mZ!FEb?HU^OgwH(9%x>dWtB$i`Qs2{$KN&Ppj|8TWNd?mVrl=%|!i%4U zguz@T2Meiein-7@N0i-^RZ7J4vW2?gW6>t$mbM-oo8^P^k&C+cnJd82hc~3VWaD=x z%!$4HZfNncPs}@VA8E62l_*_1A2J-;9mji*F`Yd5L7m|`wVvUqF}bRCD1i+8Tt}Oh z|1!aW;=957AV<#msL zkN0*@e(Fwk+wcK}a+xeri6VcVgo{Fm8+US8tF;&A)xubUX9q0|M)$U3Q&W;SCh_*# z>DO?shDW*BJ&!GTJF95=7YeQ2s*Nzt(Fvbn3e3c;Fj}_W`plJsl;fdCcBB_L3zF70 z*sEcsl`Z{aF4Ml%8rcj{W2g^Z214bAHIWF}frd89Hwl%o(UwJ&yB21#W)WOJ?8rvr zH^aO*g3FrQJ--RVT4EO4L~Blig8yjYZukXi)fuJ6^OWz|_5Tsf&Fhf~0Cv=~0Pl?D z4NqH9??5{6n$lrz9gn{DD%)OkEg-%a9>;a901Ymf%K7zfhjf_^A`5!?;x!4=p(&&ht5V3gmoNRcovpC=tv)F zTLUZUL6|781{+ECHWp!b&GhSe^?N*1She1ETi= zl{;4d-*I`;Q1yU^JCnhlSLDH5R6Up?SIU33BfAR*Y$VKlmrHBUJoy)lZRJGQcy%b> zxX9Y6rS;=K!P0~JF!Z64`vQu?oK~WsgB#DHWJUdcDztRbW@8w-l6`IzBxo(OxWEOE zxCML$_7w)zMAP7&jl^t0X4u{8fe+0*QbKga;YUSLwWd*oNB$5UvX|E?SF& zUkSlfGL}o8Q-%63ePKQ3I%*0|%}&w-G`Tb(ndn2OG1cKuSt_3hm|QCEUO^lw`Pbjs zAOO220|8BOPNeI<-Z@5yc8wQr`Uz1X+D-My-BDSA$Ar=O!cji@gCiJ zLoaNcj}JnR8ED{)Tj>g?&Mg?TUE@1}J&$ao*C$Gkv5}7k-n(u!Us7kf zcA8ULxSv}cw#tXVEYw)DGkkh&JRqbh(BHdLI|b>_XYg#Y*`ReqM!({F5n~$9r)FJs z(pSU~`G=Z~wy;@qOgQ-zs~xJUS-LC6si6c2f z174C|%;q524!>U1xI=nEFIA7a%}K5p1t9HS!#OFoNqu@*>cjDkp#pSp4Wwk}BLClyuu-)AQc}ZtLV`2x@Ax``mD6hM< zTni?l_`o$&yrUrc^bpGhPF@4=v-DgbKlKp9Zw5jawwwELZOsOJ%gM0sga1Hqk>QJ?&W^n#BjNrM9n{ypnoc8GoCTN>CZtukTxe%XJxm)9rzX&6C_J7lE%e9k?6fG2 zUv2{aNH!|HLVuNMk}6`pRygW^X_%Mz35>k6Q)4o|d1{}b-l%J0TU2Pa&nR&iFnyd0 zL{1YwT1R(gyD%oUqXn7a&vl58L=7jPd#+tb>%o%C7}j`qQAV70nX>jr;>0YHXj2L^ zF840<8R*LKv|!pAx`(sNpS*c%zhphLzdbdZ=vLqBOpXXNzxIRnm<`+tYE3qlgxy}U?2~?)UJ`;Q0}^kgYvgI=p#+|Ltaf;$Xg5aZ*QcYZiCwh7TtXNO3K_+ek;cK zvm9wh(X?un)hJ3&+Z}OsfYDJnzGqih#?*T_yk%w*os{LJ;N7cNhAD1s6I%-d!q@qPpO4Z3@rNzb`vtpyX5uv(Z7Y((;C;t9#kdW1fw?7)gSD%F#Sdeo`@RNS~cgrypt==c(P@F+ZVs;Aj5 z)+PvhaEtgo#AfV|KrR}qCC1Afn-t%zinNYCP3({{vHqxLie;+*HG5sy_+tGP$Gnu+ zmVbR_x51p9Fg5n1e{<0Db*LSAhUVph_#6f5@U*dK$ouNCXS`pwI1-v!&Y%`kPW7U# z+*KF~jrjmvsWiD;V?iHj^zOm7Ovpo5Yw}Lr4D9)UZV|raXsh%B;Ttbgibl}1yA$%P z1un{tU|oa{5S6hp@o$>My+zi>91(ObZfuhvy^anLEnV=pMVVS0eO*eHY<@{P>EA>; zOxBO3CWFm%+&aum6Z=X7ve+U8BQU#@iwjw49MtjO5Zi`-anf(t19zSsz{ZB0G{T1u zPxni_@(5zKY;&Z!w|jT)W~tSp+GNgy_8dfW1>TIudj2?r6p^)zd?}}E?K-pJ-LvFm zG)f*PsLbF@kYXN0|;{%Hsa^+j*9VHhgk}N zpTIr6B|Wh5Z^8H-W#s}t7vGVQ`a2gC7$fC>eWsdJ&fP};mw%n1`sBilI;QL;6Sdi} zXL#3le@!VsGDO+7`nm#3i8YukyjbZRJ8V(@TI6{eaY8HglJ<|x-38yGHv-N#m=(S) z5)tbaSJ_=e4ah&?%XxpZ7oG}eA*5N>zpN^1k+=H(;=o8~Av=g{o@|mj;-1+(uNoTA&#NFK;T)lruvl0h=ba%xH zE3zUgRBbw%k{Q5VWX(xUF*StI_Hsia)nu_L#F~_vO(!P^PcA;?Rn614#;?X^R_;4C z4A?mP5Bp+iY{%DHy6+IZe@2cCq}C-2quC2-+qftahLupabb$y<%pXzN=?eJ6Wn4l^ ziKEzIfZCBjTdKT#$hsZ^6)K-}gvNwAI&zHJ;9sxCN8^fVfY%GSJCPfg=I^;T7x_Al zsTA$FpKfMdo&_qp{j4O@AHA~ldN zQ&tBlHBkfkiXy!K6>xQnt}+E*{zUuX%Hk zmJzF+K%rHOsV=S*XY?zye+rsRV_XJh4AywZtDitH&6!W|-ed`ePM6z70+Ilcj0LG; zT^EwGAQ@iE{o6_BnaKqE7ykkOYEr zd=H1CVLAW4rzjBpO@4lc#n3vrq$Gc$*UA)H)+YEEX9{nDxO5B<$BI{ypbc1*?lRff zv?!Yld-T9xveF_k15bexPrN^;gk8cMG^>djz{~kWBLud&UzybT5gqXXhQEt_(es|Q zS+Wql#CL)wE=BEV)_dw>F`cOz7;~t-Dd8tNjrN#(`klgcQ=jE-I-{ONTMjFpKRab&&o@vi2I5z$Eld1>)g(R!!>_CT`UPqaxQBw(1NUgX1})}&|(xxn^E)0 z$TgPoGBaRr`kj4l{(F@yj59Lnnv5p-bp{&mQ|N=RJf9#?DBVyzr&WL=u7Bx0<$g`< zJPoBQ4qD*iks5a`Q>P;4v=xO$96)42bwN6*D*A1F&v(qh;#9=+y0jXxo-c}34;J(w zQr%4?KJ+<)$CF2VGkk1>M7J|RWeU$=5Iga~_gGni%&*SRd( zu><}BU9vdFwEpmhv1k7%PkHc8KC{z_`tRBNcbz`cGJL;NK~IoBY|F%6&&6cX;?C9TIYl=PAg<7}2`$FuQXzRCV`LBZ(i>`?dTwi_`{Ed0& zVB96N>IL@qSTKkUDyN>QGR!Wae?Gifyik?)-{EM2DA48^oG^IpXa(jhSzIU{1{66a zv^UKi9>cZTZmuU^D~4Y_f!o~F!LlWXf6y46hPkkGKjRyhMijO@Z{s@nMLOFzq`>*p zE5e7yxXAHtZ)pP;rFeRTz*SYif6meeD?s#K*({CRIxI+58`REeU*`THIq?70sEDF$^iyITO#%k>#J zfQq~ktB#OD$|&blM7@HC*(m+F@b|`WRy@^jVXRGa4>kVU;f4hZM7FTv~R*@R8H37At+_AbyGlkEXs~VV-7rPy5wo7T` zBJX60x?8}G+?Su4uI0QCbp*ETYh$023$^~FOI}51qFeYZE)b;^ILk(NL)kAU52S5# z6hh_eNnXxc0_*6(l|P85L@y4wn-!K)t?_f2{jo6Mn+N1=LI+c+0D}oyO5avWX6?S6 z`a{0%WWP+|`6(04tA`g#>*6CV2=4Pc<7>h#FBDHllcx{;s(wylfHsld=(3aGE8p z2#L-}w+jX}?j*zTkYhvLTcdg22b^S#C=wpW=b3$L;PZ`yWS4`mXZT8uL_N61k6WeJ zfp(WySe0Ie);4}X-W@QL5LVpz9$HP;w_E9PD-D|9zW;M{sQdDp<$KpJp0Q+#?fL|D zoM^l#YZ&QYH9+EF`n#bb{J@&K@F5qISpV7$&|^$fc6I*zdYOhS7v7Au!R#kAFVh#z zUMgWa8iy<*U#Jc)$Eo}ZeUdaBKL1JL6!vxLWM(D&#H`HLcqz-7ckmTMqC2Ez5;yUF zcf3l*E;q&A0Y4@eJZqwJNCfbP03raEgl+!6+Z#Kmo0INxW&S%?uz0LYB|GJNMklXl z;R`ezbnIlXb6tQm%1h>(C1BfzE36Z8q?ZZM-xR2IFQZ?4lAwoe(wXM*${Z#bw(^1W zR_$sk!QqB8I-Q5^jkRYsNOZN%mQ4^SeUL7=&z7~S^-L;3fUZP1Vj@k=m{>Y9`=!i6 zzo~x(CzS1K25ifO>KJ+m;tZt;l~JL~m4IjoJP`Cz2m{1Fxi2-sk?mHSGxX z$4Bm09X^(U1bTa+pq$*oCnsm0-Y277S06)IbHc7aq@@|9p$t@SkA@X~qev!DBTeiv6?Xp=l)H#7tG(h2)oucDPG$Pt?mTTU1*GnQXiuF=s1)QHX= zTlX`Rsf4!Hx+vz{>K0q7PxEGw`%MzTn*BK!_7yq)Fma~DKZsl_E{skS|P znDod}nEbERyrThnfiDQUqlXb0hustP8`+=@ek|5~9wgo5hzTD1pBDf?Gi7$_4x!Rm z=O>Q6FychL07F$D_Tjg=0;qH8H{*mP(v14%Y_-ZuDMmNUw$r*yxwlK6fkqLSt7#C9 z=`jvf5nz26l4yO3{AluSPK_iOpn2FM9`YGuBH?gHh1{19kF&ND=-02l%$LKvHuk1~ znfS>6nQY<7TNb;C^2ac{LZ)ENU!V4OG~X}H=R61jlWJHs#r<52axQ)xA#`n?*`WL; zMO6ZqbTev4fJv)Aq;*Q#RV9CCoCM{*Vu%rn7nGs*Hht<_# zsCR8U@k8-lQIEQ6tjP`*>|IF0Iu{{}WJi)7fN-Egmca;qKcszRQU zhzv=bInR^;$W4ocmjZ|uyB09U&~qBNjrrEvy zv8&jazu#6Q**;5WCb~L)kd8l3q`USL#j8=1Bbe1@k>2UHxyJQo0LL47Cju;k<^GC3M0G*PJ@Ne`KjxtEgyROO?fA{N+x=y6 zRg3`;D;`l4Px<>w*?onD-J!nJ+;49upx_xbsyaxtZ%YpC<~j*$(sw_0x{#K~-C3~m zWGVgjh6y#21FTx)8GOTQNZ5E;d24ERCO=DYm=v*2ivzdED>aTSatg2{SZuS4P9i}v zXif!msv$11N3>Rby-aF*yze_U4_J_0`dx(L-@9K8lXkxmp@x#V9+2$F;r_3b?NMvi z2bl+||EH8GUj?v(!ey59&paS={mZ~@@N_z#WRg3c3I)>Y}R&3jY}bkuJxG}ZygKG?`o($ z*`ni*{CM8#2r3Hn8)@?x^492&#q98MWMr*ELT?Z1m-fBV|_rI+S6OeAj`{DT`G^}zO$pf=*Ab!U9Sk)h^zO6u0a z0|SRR$=WD82@wz1Y_007h-T??X-xxo=Wa@PRR+E}n*CV_fzqfW`-LrihK=b=)&_wt z0bVZ@QhHN=H#ps*sR`BHe_JNL7%ch?F43&;%)>1PoQWfar6hp7Z{Hz8{`vt*k6^FTP~5 zXJ*fynd^G$8VQYA@2r>mqw+X+pQrYXWsJztrxp7}taA1$i7y+2;<)*6w$nWU2fv2X zu?t=9Z?IS2AdkiC$~^YpZjF{1oQ@yqMhE5GEye~Jrd&N9^XU|;v9(GG0F#6M`*A@G za2US3I(+L|>SdS>^roBai5HFi$+1|*h52t+;klxQk06=U_Mnxmv@2%%y$g|CAHwN_ zXWjg)bCWw>;JWf|B<1oa?`QFoS zz!3}+o&uqBkzTH@`SE=+sa~PRWhy*pD-*VT(Dj+VRNb{VSd}#~w*qiJDooDi2C=y$ zH^v-YsK+Ug1IE0;0*E_fpGUgJ;Gj~0 z;!}zuc~8$zjO+KixV^&eJX6881|$pfN*xU01ZEmmJK}qyWw689IOPW8QUr}T8sMi7 zT$Q{Hu_M;oqai+X+b7um|EldniMw}=92ipJjFJ82uRx&*>593BlX}ZG{mAxONVX6oMEL69up+uYL3XjQ)^8l>TEU#pSsM~PWMOCUn`av%Un?~B5`HQ zoUnaTac+z4`{{`@@CGh}>vi=^Diye$f0(P*9UgP{vQ(39bzPcGhfCoDj!zYN zZe`ZCs&m*97rMchIR1kyYz}ZDScRZC7p3`C*A%Rhbh+*8BtkLo0u4;XT9$lkG_G%C z41RqMeuK#?dQJxiM-BuqoBXuTE2ryUZStDq4LPd#d5l+?(j>~W1Ji-M+IUyFII;p} zP&?1d?_bqCU3WU|Ti{}3h3cera-^J4z^B3YuZ4H(Z*zzvcH74CUF6(utpCR+4juWQ zS`)ZpcmjP^Fe)&anbRn1ToR=qg8 z(Ox1V^KA+2hDZJ&`6EgzN8&K77c+Whrzy%?FlP|s@bH6t3FQ+ir3A2${V|bHW3Pa% z$7@L(!05}!O7^Fy6F-w1u_b5oVM^v5d9WB|C>4n2evJxKP!Ypa2czSL9S0;&r0eDi z^5B7{!5cBto1aL^4|V7Z;~j^KbJ5L%zBPa((VtZbQZqaUXc8=I!>fo3_V2)p51V_+ zSM7~V^7H}wU@GzscM?#!EJ3>pwfWQ8v&uGfd~&sE>lV2uaA@FHk`n{2v5s8%<9^?6 z&(v}~Pc!kGuYG!17)(}1Gjboea^!nn=XC7>=5qg$bTyj78BP-;9YXR~dWu7`R;<$O ztME@0%5&l`S{~18$`n&NZ_;yj=bo@*6w{NhuD3k6tB==Q3i|0YA22zxX5%)z=Ko;J z;OAtuXUT(=!m($4OLO`owm*Ot!j50B3%hr68y|T3p3H%~!-2f~iL+b+SwUUvehJ>Q z!?u9Fo6S`7y?EB_)A8kMJY+l42B=jWT}w+xhzO4B{x*0R z>S^=-K8GvE_zHr#+Y-GH@e#+TTbPUGmjtoeJzMpwXi~K!LRGqbpXqxuf2PQU6@Q>f zdIXzN7adPT8#z?qO36?jF7d*BX5bh}(3(p`bx+Q)mth0*ZOxX{Y{1-6{=9ZfAF~uU zJHy7ET4tkyjMilvtosi0?w}VnpO`I|5e%BUKKkU*6(J-Rd&_T#1YPfr_-zh(4oSTO z_oV|MJNd-^&nxlLE`e+bZ-*+GUynm1c41B$_&Kf9W<*9f8|kBQZ)!cS4wTZx+deXF znKXXZrI+T*qlW78@|=s}Ez4TCKEIUzp9l}V9pgF?Qey%>doglV9kQqq7xWrQOoA?s40M8tSni$6y&q0Th; zZlLj_bRm2e2(7=Pdnq#9tTe3;oWW_RmeA3;B)IZem>2+R#jd9KD#T6rDUqTcCR1K` znvJJ0f{q+M9GsrF5J@Y~6qUyKc^b9(MWzW_)I(kE>O_3T#!(6WqBF_M(+RXsnOkS0 zdV+#NoSnfJc7f&J02b0;nDAn6fBHYKS>Snv;^z~;pT%P`lr+&lOzMRaTwBjp%cPfg zX9}X{`$xP;-XFFT62ZvU8z>D(8^;}M z?%;8tqIWO?al-J?+2I#H_995blm;y|qLcmQ>nC?Z``Ewl%jtJEn;d>VaE$$9DfvZ?YTlfulCVjzsAbjb)qp#lN(6a)aAdH<#k~cKj^n%w5d}zy~F{AlM&-m(LErHANwMeU%h|2^~8Hg!_-6p zVpNuAY`AuX`c)M~;bXEpoh4=H4mjDv_>?{CHgF4Rsr?n4=0+tj#_;(7s%H78em61bmmX3PzCu7co-a>TdA4~$w0wP@jyQx)SP%37h&T3EI1(PHv` zItv~c%@_2Xb};Z^@#kQ$L2EPp;*LD@+;E**(d~gY=G&EiL+@mcG(oEkKc?@u?CS|~ z?lth6@v?%%C<$w7!@GB|Gs^nS{88nXm8${8*cfMpNbSEVC;H<5KL$63P*Uf)=qYW?!C(mg1uB2Ne;?rL3Cds}|+X-(Az7UlW9r;eC8ARK3YEuo~i@4~^4 zqC;UxCrGrfy0Yu)e7Vcb^c+2oCn^NjE7N@(BF`mGIWg?NGFD1(qdj%1XXLCpNHzC$ zne?nuvXBW{>)|qsi{EG+z56TUDeA+Q|~#VKA9pyHEoINN^8glqWs05Y37sYSbS)8@6F>(i4q=XcKvp zs4hO)qwe!<=zPFCrOaWsQUBoT@!>Mox9gf&_Jq0#(E~NV@E~!2M_69O09rQm4?4CJ z*LLoLab8Nqb2EthZz?@Xq>u)dW~E<$16XVq3q z2uquz^-oXRWCsps-Ef7hiGd@0Aq_Z#Lam&*#4b}rt`)nEln7bFSZFL$q!*m`$iDS3 z*!yU_G>~4Vc_V)%&n`g!Lb@nFxn?5!_x&=vS?YHYk-BIq$$3F6ZqA*XONaY8$Usyy>|zkm-VzyeeJb z#xyXkrK*1tg{`acidpe>HsmqqEh%CFCmV@RLYO(~IX3mB+2NNS1V8?ash|H=v9iSa zhUiM)o0P0E_S-B%KrDJ=toc8&8HhdyfbgvJ(%$$FZdroL=F(S7BFeQvu1h&$iV1h83WT_Q`6Rg;lAjgNj`qQ#c z&iU0Zs^jR+X%a`T@p4rm+uT!4(i=Uj(!Jpq*}=Pp2(A9|t*U=7+{i)odtgf6=mDLr zN#4m{7P$w08JefbbQ=B3DEzv28hRS9!VV*T*-ak2tB4CrUknyjc0a=8ZgMu?+Qclh zUkFE@JBm)Iymy`El%ui}hS{M-rWu@cVsM*z8t1u-)_Gvo*dgNqpvz+c^g4GKaPr-0 zDoZ*QN1YZ(fe7aQOYowImxvv z&Wz6}C%JbV6huAcY$zKUsXdasauf(Tt@d9I_jkui)hAalg5E8-P2z%~ml-|s!HnyA z+s*;(1_%JV0itJl{x>>@oH}_q4EBSk@zlUsgZSSO$B}CplhPo(;rlwO-+tFQ3oe+J zMXMe&QA{7@FF<;X&?8|3nL*`&>Ez+pK_tnr(jX-}Gy;kukxo?v3eIBsl+KKtPGCu^ z2@B+W%eUhyr(tf2W9({3ar{KO=bA1eHk>20?gf!cLs$M)b!3CBoj(*llZAww-j{*&H92HEGlLIH||61eu>CbcZ&I zPd6%&1Sz%pzMDKDg&@uWvEf_`dfm&w{K=1_1v3;>Z~3TsJmqb3tW7?DJ=bXv?WKnqIs6Ah}Ot-Y+%jtNyz%Lxk)b z_=b$ZQkhbOTR$6nNCpxHNdQu{jC;NzswDz_WHne$i8DZeat9zY>E{k0c~H01({ky? zF`!oTkr&E_@E+eh)79@f&9RF;(ehO2HSO^$!R>Kt>|ooitS#yl$YaIG?;_p2 zVI5~3lwsTnNR)Cd?95GR3pObT%P<@-!14)h!0CDeWT|C(ci#2la$Y-3j?sY1bSNxv z{=EZ#+v1x}@SD7|1t`K%{e(yc7B2v6)qmHRmLkJyh4W@|2S7ddsjr4Y%3J9>EfyMu zEqNeFrA3nOvML6fz={7N4XS9VifMr>)HPdexF=ywD@ihQC_SpLbfC(9_Qt38pqD}~ zP59HJ}tQY5k z>OB;kT(QAfH+~gLAAw}eQnddo)BubadciL8!M|*nns&jBtrls#u7;0OxY-8ZM1WJl z5Q3gf#QF1~`g8^QSE2TnY`qAMu=|Mz!7ynZwt**Pb2;VeZP2X~K32&C)fM9UWA+wx z@0Z?s2Awj9?~)<`o~Q#KKK*@dn3!Ksr>0Dx|m(87Yg}qg>m%V?1GYEzNRod zaXgfl_jXqYztgfs3|f(Qf>}J{<1Nd}3sg@OdQGU@`VWTF4-o|a)8!F=g5J9_s9Wq? z;||FmmeT!)tz%~h;{c=S;a_#^eC%nv{9QXe;H^pe!5`M0;O`utk8L|sMk?S+x31Lu zCUXBF<#Hs~SGjlDzDrf!zhZGnCcledW;Ja7%(C`BzPos^*~;$bs>cvCD+RKCFzLJ( z*QLc;tnFeDvCLb60^c334BS*eC&QM*=Eq8%zJIN)T8|Xavdf=+$s&!?_EFEeJE%k! zVc6IWl?VnW{!Sjf7Qq)#D9H;e5lYQ5c(%xKaK=KPO4<~6;1T7$TvM{5>V-`(5N%L( z%ZgSHc;}dti-SZj9~rSy&4Uf*s5#p(;(pXY-L+Yu_U`yMsqet=w`BTa-L;T#19qF(D@Lv_uerA2;Qh^sz0`arM3 zf29tbpX>{nZU`zcE0eKZ({=gLfv>7NpmMbMphIcYE55G)@n?n#U3OhVfXOMxSzJfq zOpc8P78l0FIJwxm*i2UcS!|2e5zEF>k_^Ptz!@(6X|mBFA`L%&Y`|lipzx^}6V4n1}Tle7dW`HuQ3 zdC|8=`ESo&#%zDVM1k$E>jbU(=dTWt}6>&4`>LAeoSQ4XmnHI`Vjl9Kv=g_Jnwa9ox51phz2-6pr?;_xCt9Q%Y+z!X?ao$%qXufJB>!*z zFeh$%!5<$PD<+sY(t^A2d0kZQ;zq-o(c-~1(SVsihCtim%H+uowFh7PlGBQQ#2q(~ ztJY%yS7@BuRMxZL7AWlc+zN+#8b-;`_Z&_gdOXR0^aI1u{Bgow6MmiLwHlmsK)x!K zsV>#uvT;&qb|@7KAL9(t0)j`$ zkg`GYZ&HT-QwB=3bX2(k=+K54n`-Tnu>>wT7udin5se zZ7VU&(qXdA4~}A_%$KXW^3;|5v~mvhqHhe0i%LUTxjHP7B+-qp*tjl<1cZ}ubWKT- zj3AK$NG(=*bdyXD($QR*t z*x`=s)2(#lR5WX~%arkPU>rJ}a4lh)C+Oo3ELw2xfuz%i7ORt1t+l2ZTEdTf!-_MD zdkf-aFPfrHK8n-L^g9|y`>Oh}zN&yuwkxu-#nrjm?C{{^kDJ_amrTt81vnT7xUx?H zXF2nQ7aj-nOjSG71sA(F7|jMX6$?h>xioXyo>C-1DS8@mxz+#FVMFq}2Vx`+@Mm3C za$gmKBLSISDLTKu9Ep7A8MEQ;|GBM7hwy4}pnI$QX+GsvwCcr;6hu?+WVyr)yTo#H zCCw3+a_7=OB}uiKR4@bZ9a=v@wqp*dk|yXZ ztzCi}K&AcMmYFB7vAl2r6#(~nAkhWw1c#2g2_S_4C#{l>)Om5+6hNlDsKFD;$fD8>To98T0p-13XbXb*akU zv~S7rdCNd7+OxFMWuUIbk2yK&09bilcQPXQaTYbGwghJUuCcadTH?cRn?(NugsI16 zhf}!zLe-INRPBA#2>XqyxC>zy3OFg4rX+BcgSwLxv~C=m?T(KpkKGIBP$8^LDFlV1 zU1BvhNH>=IJ!jFo5{Bd)97Mj|E9mJQGFKek90`nUEU!B%wP*~}cO6bPft|SS`yi2a zoiIjNa#oM)E3!^t?tGwByGl;e_GW_YlnyYOqBy_w?+D}%5Dtoo%3ei)!&BamMf}5S z$Kl50!^)MCspcRDT_=SB0kzH)^FAp_cKfP%*3~Gh?sJC?W#boCQoIz*fy_p?W8YBY zYQ?NF7RbevMx|kAM7~xYLlWRQg-#kPKLW6#gP&B3RORVdg1$7*ClSNBs2YP7)!K@- zI>ej&lpWc0&*gqnev7EmZ{&Qw2o#*(W(1wl|EpWc?*g)4%u9*u-2WnM->xDM&TUg& z?*^*c1)W4Hmq^_RwQw!r{zMkca)c;zF|u))CpK2G`JPwZ2^W@4NY8qn15xT0Hv77M z_oGY_I1IRAb58r<IUP-^TR`FPMRKS8{hppT^wgOh8}a+FDrr0R2n2r ztvFl%w0$)o{<~MF0VteP`?Od8lJn%1IF{>QHr3Zm*e2_@_jzUhwhli!LztJrEkdeP ze3Eyzmkg6DLUn`*QESLBPQ$9w0FWbaU5`7vX2{7n+mpWYq*4ZIS8nPxLhw`EKD^{~-C(!bAL!_20}fLjLN!B`pjrv4rlJ z3B_aN(lj=xDwOwe-GO>U$g+= ziQ*L}QuhDliMjk;Kc|j)$m)z-Iyq?{i6n z;fS?t@2_5MkQFh#*|hlVm+Y6#H^*UC=Yk%mb6FSB6?Inzj;;8ye?B9UqSG=Yu=_~< zaWW>PkgN!`Dn7T56Of%1KuBWRmFm(xeew1Y;ga|cwvJZmbl+Vxyj9w#547u_xhU|Z z%v2i&!e%z%HFh}>(gn5Z-afm=JDi2gJCV$ zwb{h2jhzqVu05vZX@0+W7v)=d|K>k6H0+2hOyD~Ebub`P&jQ##50hFZFGlXLv z7LSX-$R23P7WeZEORMiUoiK_&pH<`KwrI{!d*?T$76ZPeJuZ?;#gXPcxlAflrqzAs zlJs9mMV!S`7dyxkY%UbB%sD=*M!m$ASQ}X$k)jrm4dTN_UFuVK(DEL>#jYfJhTx?{ z8#`6zE0RaInHWG*WdPINW3`R^MN=iBfn3S;;galwKXQ>22i_?L02_yGXVCoYvrYzb z<1P^SVaW<%0bWjPVrP00*;WjJy8Crv*CNCqq94{jP6O>@)C*=q&siYuvyDiH96Nif zl?s^meXJ@kk(e=JI>Ti!x}bAMaBn0QNp9#DGjJm1ESiS-r_s&8?KBG|k}j-^ly_)`8#2 zmwO1%0`MPVxxep$84D}Rq?m%8&^a1o*I<+!{*13Ti>*Y)Wvq~q4bQ7*(w0pySq*Dl z|1b}4kWNd((XB(39){K6(dV4TB+wqcH=Y3`Fk}~5KaO-l$c~!}drp7q&g%)x{LN4p z(&QpCiMfy0ZMf74bN9XYVGQ`@8aJr(JaBAAGkpZn^pzQgXShWa9)dk%ah+B0fEW7a zX+F;(+dPH{8OoGGK6ZKJ(K6DY%vf|Ot{?-vH35D+l;(doxi06XEbmU_s&P|E4c-a1 z{J1mR$HtRn6%kLvoh-?(eD@~xjLU8|ThnAfbQ%Wls2 zK_E(-)@Rz^=6sE5o97zda{Oq$?W^)ZK;G0JP4jy5mkX-_B&2t@>d3!&*1_NcUtoC# zHeRU-F6gYu=ozVv@+lxIIPpC?cZTW+V^>fWAPTr5QDPa*JtM6s&@fcG#h8S3nt-@9 z31__THwqRnH+4PO);x6Py=Op!Q$K6KJGbMR(Jha)z5(i?EUKC;qy@XG#jZP)UC$e2ifaKgO2*<@ohN+@=G#EEA{ z^{0!p^M`dQCtrYEz4;SRTdMX4wFdX~IroooX>LUOPHOkfb6vdU;Wl|;%>r|GN{710 zXsVdW`TF477rL}|D5O@7Sld6@8K5KXd~{12bEh&Iel@Jt0Nc3cO2_E3Zl)~(jtjiP z0iG-PxzY45qk+3l0&?8Gf8{u223V`e`h~LPs(VUfo-2c_f3q};jJ}$z0ip8+S9^;b zo1{qxS~D{}n++!7-GEDY>#VeTt!^(lwE`>WscUI#-gW2D9-+z*RIiABi@|fS6N5S& z?+qi5-jk=iaD^GK4HNr{Z4<6M8K`t*9;N5a*+$(|QBOJb9mZw6onalMHGQ$6PzL87 zxV|sF?Ybc<`5p7YnMxsaR}QoiDNK%hO~E{WSBj%)1aYexA&bu^KM`jwg1jO)6^SbY zH+{JrZ*@NHAiWo)(3dS%R+ar%@g3BWg5>w(B_+jtM2@pn&C(0~`lWc>wZ*2&PBry4 zj;G;WFaWycE7Sd*&~v4Lf~0ZV7y2vD9@}QTmEtJ}|H`vSeiyj)DQ48z_;1A=c=wmM zOg{hiFy0b0^Wv^shHj z(XOzaF3l~iW=VcuYCiGZ7+sUC8@k6Kj>dTH>3-^qcaKg18ooQHBx zfx*S#!wjll#3|Fz1&(y;`0;SF=QTZI<$0}G&|dAPJ`{>KlhdJ1)vFNEb@!{_tl`7L zX1-Yv!G>u&4d7DmdFR?~Gn?xr(_bfsKquKtGh7Y|v6MFK-H)cOa!aqPYn*8;cWp#2 zQh1Bh2ihNDZbFK~GoLLVAOGI}P1gyw1$24`N>};JQWcaK1hVm>Lv-7=78p_oK(qG@ zc_#c%_V{Db%f|qQn0()Q;@F>blaYEyFz4%)(`bzu3{11jw4{dG*#B_b&R zgc~5|VHGHX1!%~QmBl!PFdm2Z!6$9UfP2RLzMO`)*di12DUj(H*O@tzWW}}i{PM9A z2~PZPT{2XkR>o#o9QpRQsCP)Gqw!|cVNq&jejL*A&~l|kH0U?wHF8#tMa)e%%6 z!-_yF2en|tU5T2AxvvvkDXvG_GwQ~u4M?|QB*#*%wYPAKf+YY zQm!4d)~}EdZYAM`STsgmV5CMQb7uF?o^m159YsKxUPNwqZqB5Tisd`3FE8rUvm9b+ zWQC$jd~a7D#%HmkKGnH?<6`|VkiOIsMuofPu@SEp%zP?;%c)g-Dpkhvw5j(w8L zCL=AdfV=#h1J43@%srdoKt>NF;cfOYO|8;h zSd=%ZH6j;`G6bDwdrQ%fz2nlUVorw-WJ;NJ$nD#=b^HbbXiO-L9q1fV3NH`G6Ka_v zS1=4eF6@{6Siw*!q#mS-%V{l67LW-_A!}V82GtGyJXempLmJYo&;59{&S#G+5{!Ad8&*ILb+ zPn?l(#LW~t1*j(b+4Cfc`l6NZMhKC^p$f1QnRf)UL4 zDTVT%DEd)op2R{SQxM6BV2$4q0k(~})GZZ7X6#{n`*H@#Pw7CNj%l`xpR2zqYgDz* z*>N&$faqCDhOP58IrkEXK8;<3j6ruArU{PmFtF4@&ddCK)`eC zV0si;U2$~(kiPbI`m{=MjV%wf-1=E?P&I3C0EhnJezea};UQ*Q4%h<3&HURk1N?ub z7lCa-WzVCxkUxUT%N>C73)@e3+H=%jpfPWqoY(%y**JR{!ZuO{h7$BW4N*@@wQvt> zktW}4Wa-?t#o=ajGSmz^HFn&Ez&VU0#d=4SEI0$DZ<+gJ0Ec1;|Lh5YLC*~E1=|d$ zeK^=u#x!vjRWFHsm=r5Sy-%ql#%JGGN)UNyH0sd{%;9|M+2xG03a{+O5~1I`#W6cC zArz$A&3MGpy0XHK848eMmK=H6x`VplBjquriyTTpFY%Lnc4C{hc-to_=?Tfbq`eX( zEB(HAs@gM)d|h9I=4S@VY^V8yv{$*OU#%=ZwwF5qiqZ}{DYujR@uz72&PcVN--e_u zzHnV?;Pgy=mhK|Vdv0Y!cY_=}dVNFPF4Ae;-kDgmohyKws{p@c4UY}|my3V@q4RLt z(LeAR{F_;t9^Rp3BCI4MjLk2f&dg7jcT5{N!*P{?lU?T`2|Tg9mY1UejK!e0(0&&z zuItn=BCz+!G)Jjq;Z3lrxwu%%h2!88AIN~3tJ4V6 zEIjM=s-$}|;FnCbwTRj9lX)FvL5TEl&!Kf@!<#3$^mnOCVF;NL(5Ck>GZH0vW)|oK z+X61Q{}1!qZO_e}Wy3Hu{;@+8dY5_wgFp84Pap<{9)(|Ky`P}T&8L)ShiE=|#p1e% zpD87nQR6w!S14Ydw3z1K(sj&33aj-4SqcHSRmV&bzJX?D5t? zOlNLT3@K>%X7}6goi~42as5PIx%jKJv)ESCZ@Ha4^;ZHh0>|-ZA9<}FdH#*cA9sKG zh2I55bcN6M7x%t^87o`@q=Zoq_=^Vto+uMZw$Drni+*)9?MmHx`$K`dn4EL)Fi8l1 z+pL?*cO;|dNt-i&`gPQRT414ZM%E|FkT`3FCdz4<$nzxmrPG^&D&R)Lj<#;U`H`Ng zpu~Ep!%uC~-M0hP->xrDC(Y2)u_|@JZCHG61d{jN_;KrVJjZKjsel{`uQ%n#FMkiK zg`V*%4R4W)yH+!P^2I38G3QHg);wUu#uIQ11u$Z9GUpWt4awcpG3;lduL$(LWHIci z4~*HBfr3_SRhg}{<N@nRlkLZN7Szxh;!E?)uu ztOe>4PgVC1FM2{cswQDOo9Ki9kTZI}r{h0H{ik{5&go)@{y~bqt*Na1p*iATzee)8 z07J8PsG<2bhpo1PbQz0whg$TIVAD*J$vg44B*j@6I+>e*v?24hZON%W=0|Sc8rH zh|B)NBgdEP0+`Vv-RE3Stw{0fT$wtx3gj>G7&)@DrY2Yh030$rYqH(HPD z#h#^mxuF;gux}htQ5pbQ;AZyCsocQq?cl=u>&ChKOF^3zL6nW-8lUL@j*W zZ;4T4tV~B}J$laQ>qVZ^m4V3>Q>5!D|PC z^tqC>Fm z;gKy^h5i;5?GJ+k@g>^XiyMt1z5%_9ff@DGRbnN6VzE&+z`(852=4SX3wFtW{;|`! zTE*fdDu@17nNyZ|`)cw3giBF|FXkMgTpBTWVpUaBwdlhGC|aT3RjTZQ9>XFIF}}R(8iOs(m7+9@ zF*m6NH1zxd%S)v&xpQwb`}sRu@yP{C3R_TO5_7*52M) zde|Z#{QmvcLPuSS;PuUlZcP1AAmp1k3it0}Ukp9@qA)np*i;#o zEtxScLdq0Fuw+q*qd$K7xtPExbIH`V4JPA+IZriUL406-ur=9cM*o;Jg=Z6|$5?j|>mxqomX1qIroS zAYjp^yd^Qt0z{vZKO;A$ldyVmlO*Y2yO{M|ZKB#X34-G{vG`hK)Dfl$zJ$>GcGV90+7N;-TxBEnB@pN zibP0QbIX~@9<()XHH=!d*eFrJIUD%ARZa-a$FkMSTWNairNjD)$9l?~+7`7PGd0_t zj%-aZZcSjPmWQ(L-V>5hy#bX1T^8{V44gyh(ATF_{7NGYcp|Co-K9ub3*9Dx%8_$4 zYci$TYMxN=94LTSfyC#JiMn;zCtzf7uEr@KrRse!yJ)O{5Cn^k@5 z{TjPhy_}9{w4IgMcG3OY#hm6Pr(fXvck&K^7(Zrv&i5Mpu+ASGI%sET-`BFVsw_{3 z>Tw>Dek8y;Ow&7QciMX+|KC{vZhHYlp4EUF1p#qd^CVzN85Or?2MtJlXRVXXh=LsA zRR~hUBuN6&fJNQnV{@!0Qx|?ZIkUs?aNzh2vF~;)TWYF z&--jP#9gzLPJlZ11eh*Y*D(EnIoOJ=Xe3@i7hpdOslgS5jhEv;w^x%_mLd{g3*BpG zuYo%#zo<*ZHgy?1&=K}mSv<}KP?wQh83}mwch|F;|H3uo2DA%wv7)@!s?dE`2$#ox z*=jM=*uf`St+n3swW((gW1QBS013+{II z74Bd?i%qKybFPeysEoO7m38VX_Pb9bJq8rZ5C=2hs54hR+@Ddw(TsEWr@4^T%a3Zk zg;lt>NObpgCAlr#zTl4+@0cAq6Xg95V6U|IV91f$TXN*P4#0EB}$L$T=29F5H5>0q=bS&PxbtLse#v5ge6_2WtEriN7uxgi?{;@8C3hz-wXbisfv_*Q$p>IQ&1NoHKQ4-vC$9XRUJ245@ z9Wa}I(_I0=yry_&b>VYXJA@QU>&`)!UNX?jr<)FVUbksgm4C^Kkgv8iM8ta?gInfw z8WiE1l z&R3gl1SMVpjFj!SR(0!NJpGcX#eief1hSKf3w!^490HNJ%sK>nlOX--Fri@dP^b%t zz-``F-K<_&R{e0s`|GQvySOKB)0*VDP}qvAy?Isvx1SO!Wkh6ChrnD$bga+0697TM zUO5PgCogvgO}E+|w*p;ZZQQNtB`ghvd69zwS=zKlA&4`|F1vKltIlVB1QIOu;EdJW zQ8~)z=tyBxoqR~GFIzb|x&=>EaKPhB?>-Q60F_3jUDOXG-l_dMaI12(0&^1`g3xtd_iX4S{ZErBifqa%h& z5A}X22!OzsU#~hK{1;#0;Ey}Q!?JZM1CMfeC@1_I1^F+$jN~)NqVNliFzMRyjnz2) zYB0UuR;DdS7)2o;#?UG>7#LUr%a5df% zblJl+X&`bnmbzqFMZKr3)ltm_;RWyE-9susv#^|g_~D1niygmeFHeABELi>FXa7y} zT!;h0nwX@r_-}jxT};@n+Hm7I%-4OaHLwL5V^%$G11NE$8ZgKj=#j8&w=p`2F9fx> zfPM)@&=8lpd99r?+*WbXvUnzqT6QjIJ~4GTB=$o>2Hm(sKoUhb(f6ozPRvu!nDCUVR;#SM~9|Rn@;zl-5xRg(3ueYv&R1R zHuaL<4>;I;&D38~-hWmCMhYN6tHYt{YRc(g-Rnz-&; z$iGDOFlQTd%jiWE-YCE8yn{e10{!EcM_ z&KIWN_^$m6Ldc8&z&h2=&FBAWac6HAqOcFwWq<$M9S#Zd1yo9dXOXHPycTNTQ;xxI zw0!A_a4jW)&$qT8BCcwv0?onaVhR*&=GP!=&x30py{g1e5%0nF6tn7sYmpgYb#tSq zq_482u{xJx_^+JG0B)eU_!?bxtc#gH>^kR?h|ySZJUW!af3s$FIGpp4%yqv~T%HFU zID<%JC*6sHQCdY1{gAt-fnm7sw9kp#2w-Hmpl+q?1g}J){BAx-2~287W-OJ@ z&84{ej=SeWPRVWkRd^5$P<*>DHx_O@sclh5wk#r!21J5BMH~14N*|Np4!E2V*XNew zzttZhMM;QkZwFNV-7HT^+>8*gC{WR65k+2hM@_q0Bm@_wW~ZO}ydF<1+g`j23EK`!i2wJnbbc2w znPqiMA7fEpa!>vamvcO&Tyjqp40Mu$%{~YDKEX%#hSg(1g3Ll!_ND3>5mFw_V>rFV zLTQ1KeFd11!`~hZTY?-f_% zX!NqpxiJaoRhJ|fl)u*o3+By{@TuSwE&|qE6W@6n8q7&?dBRI^S3B{DBWl?`mGp9} zqT?xok80G#OD_XGF#0+65-U}3T|G@{t7Ofgcy)&D+!&usyE<9#)4NB0zN&1xX%d*1 zx9L>1z-+u$JYn50n4XNIRH^a=hJw4LOS>;qh`$u8ln?!PJj4-+5)Gpw%Q z%8>X!%`D(w#dby-Z#|HP8K<7XYOFjU3{SDjnAoULN8@aSJKq=@V~o8k_e`LwlVQND zAKJan=(zDCz=!7DnO)=SWL(x`b>D<)xMYhqD)?lAP@(UxXlvn#d*j97&*|F5SXUEp z5e&Mc7a;>b;YZNJ?|6-OV;uwfCPZnN5vv|6p6|_j`)>B^GFu zR^VtV|NI-Fe~0gv@3*yyRm+w=|ACBy3lUHIy7_Kn^GE_P=5V#3T{}lFvuIg- z4dqIm5^uEIvF}R#R`!~{yT$pZm30xtVS=3h4_n_IPj&zPU)?IwGE2iA83&PM92(X+ z#^DfIM~du}y+@Hvj^o%zILGE>XLqZ_k#(#Pr{dt)dmQ8Z-tPNz*XMKpzW?+%^4Gas zuWLS^&x@`KST3_Bj}M|zLLd;(b*3a4jpz)%@b0%d1BO9(MB2Hfp4=E#qsF#D<12Yb zxu!Un9C;@k?&PB)4UjmfHA9J{!Dt1#B5!vC%fAxnq*;Y>a7-jO_}5so z<-A~_e!9-)R(gcuta=bSp1`_c4zi)!s*+5!Atfj;xSD|eR0zs+n*W=(_C^_a zdK+PUzW;)bCm#bmF85(Vt;#nEFgVbi@EG##iLqOC+*tE@CSys3D-M-@WY%_ z+-5K;ZAr&;(g-$}gFr)JmHc~Ix5$6=Q=qmPiMiau?e5D36%;*_Xd*m;4Y$sU}qeOdW<@uM}3;@qZdt=je<-fB$@cG}o9pG5| zdA98A1;LCY@s6-6-V8=!&z^OtaJY7Tjkwk`u^0q)AjnC^cA$vcEX)NwKFwQq3z7jo~#3R@5x>8_QYiDCo>su~&QI!b?&~lbmRST1+EkpeAr+AaX zjcT=DvlhVN1En2XWA5K4h+mXRqmYyqnecKy6Rk9~YvL%o`7z?ecZPd)wD2SR3#QQG zfse|S95#SR~mlytZ%b$CK{HQB*w~3C3j`2gJ<<9OmSW+ zBMtjU8C5dEA5OOGBIFan5(oyxv2g{_)Qv9i3jYlWa{J|>E~ZT1RnsW30r;yvUyQ5b zN|mj*GX5&CK-<=iaR3aReDg3LI(@GFYclgI4sr*V0*m$8OIMwL1;}KnWAQf*L00G@ z!848!xDKVkIMM*rd?qx`b)ity8InV5P2!aYfd}}qK zvk}Fia`S=PElRjRyj!MrA+sf8OQM zFp5?L{75gjgr(fWct9eQ9Dx=6?VBnlf#TF+|MwCHwn^ zN-;a%$wg;u1#Q}!(#N$^kKz1r@@l0bc(EO(nOE%ZB2U345P3eALP)gsZq|f+$z-8> z-lQ59&JeLs6#>YpHH^*fJeinm(w&cZ3<>D-&-T|w_LZYN)vV%lPxNS_BNcbc(d;L* zWC3E*uHi-ZoTV9MDeTqz@*Qk=JNP0#wzR#dzcG4#5Li$+)p6hcPj5lvC)ZrlPI{~!8rtM zRzk`G(A9&}C7JtY*EnXMIGP@mwcaHJ15om|s|bZJM(~&1+m+#rwM{uKNS*vuf-i$U z?13Gmx_6f~9>be=Z5*(M5q&6&9r?=hsA-5&CJr?cOl9yX;+8ZCHvIm z^$06CAx`lJ1BFbRsG@u`%7_Q*(dzHj=t?o;Zdj?Wh0iBZ1k=@~g0XHzTW|UOPtB=_ z9pVK~M8Xy+kfyr!>4H78uF6cAuZie|izUDYImNNZvuElWDn7A7N796j*LoW&urCOD4i(aUnDlm0mEkrocUEwlPPG*$GELche6a{l`Q9(~MLeLYF`ao$kL} z`gG*&g40~BeUDIVO5E?i-h7!l0beirgmEOG!gY{@knZOIrx~tu&H~@ALp;lzuy}Hd za^u>%o=AOIHT;Bb(%Js44n=!ALwigMciP*5Pl7Yj_bg}kbbL5UnMrp5y$2M>@ObLG z9$qmB6rIG*H{sJRcW6nym^Qer#9>u=$|;Az6aWw@R(|8-qV1W>V*6mqjYX^lXmvJ$ zUGeP?$~xzKTB7>e)t82ZK{E>+gEUv=1{v_~O7mAK1JVe_dvUa{;$T;fvJ*$GbGLN6f`ZnCBE zc$RHu~{ zofYT9+sQ~^5LMiV<>uj85f;)SQ~zvy=LTm}N@6X91EPi}^JT$CKKPql;l{3OnZr^E zgPD6o9p}cTvU!HP`?vBY`oqw@^A)TRo)(jw1vgXr6Ym-(v8iVLykb@K=aZArI2RYy zs3Y0k_bUC2qaNn4R4dw@c3puNt$($Ro5bwbm?w65jXzVt9%dVJcd{~8NO`AySY31Q z+{z+R^nfSBy|v=i0e!&!oCWMNSD>E_C#XB@X~A1tDR5*PlbwKJj_1!$w^gQasdd9#B4v&U|s8 zWzMK?s5ZynF~7p{#)QStY6`)tuZ;$Bz6l-um(B!SkDy4g}}w~ge-O7QYP(F zPM@Co)kMzK*I`)Nb{tLP@P2w9a0M>o>E4tlanpJ1+XpclXa6N%QW^p{?s(1&AWTJ$ z!xTH>rL!}ShR1|)zT1|0f7$2{)+`9}J2b+jJSpA-2@jR|QHblOOb-K`|L&$@kccK_ zh*1HyW1ee4e&Vb2ZvJl^*c)h23+uVR0_d+{4e)^uD6~$!cqIK#!+Df>Ciuv> zYT`Q&2pg)Cg`Bx>`D#+XDZgu|=x%H%`{Gvc>2sA)uom_x zETE;0$j+s%`6!X@BvTxeB3n*mKYDB{{Nal@&ByA;&uc2Q6soE)?J(Tc#PUDnuYxAs z{NPtZ+TK|w=x2^>%zdlCY!tg~=VYdG?&nVssVuP&8j#1+OQ|DU#K7^*fpuru0Ac_6 z$6OAxrYij3tUA6hyaeJf*dMqHj9J=ZfO*|D*=k6_KF;)mQYjzeG=;btE!2+@7S1&t z zUKw%UnfI7t255o&w|RCL9--Q_s{y0{WJjs8b3PcPyydFh0Ig&6&xBW;lzP0rbHuHnC zTPmu_^rEn!Ui?6vGU`i|R7q7IuCs?`?inDlKU>E_%z`$^^GfJJOy(Q4d+Mo1m8wtFE^ z`zzCq6S;i2JaU{PLh+9h8-woQ1f0qv(W<@zmzg2W4&rLg=Sp%_cH$lFCtvm(^=5*~ znEtZzd)-4o{wa^M|4_RA6W}Fdfx$$~MCtRcUlBjy_Is9`hNU=>mq1XT8+0f1ftbT8 zpJ6q}DEN+JJkS9GnFqj~V|$Wx{9il^)wB}zGlNFnJru27?AVk}dNRP2J1!@B-;Gb} zy17VTxoWb#_F^RcZK-Sxweu$%RMv~>@9y**}6eqVSqB$^Fn()j8nT#ny zdvmvVY4N=XOT=v+ObSszcrTB-Rf1@m1v6>8RW_GVqCs+O_2vEr4Ui#}cr|c66`$wX zurOb_;hOpUvx4o;>bh^>7H*y3{@F8xKO@rG8O>T~!ki*LC$u%E6bRfi#=?NB##b5U z>gGyar#)w~!T?D@a<$w0Ke)b%^?N@$cOq%OxN4x-!4KMfyL72duDn}O2_K>D1F|($ z*e%aJ`?4Os3SY=RX5w(_{-nFslD@VNMe18%-A~a$DAUIhyJtr~Er=U6eV7xp-e>T9)8^;!B=Z zCxiY|0t!Xxi-4EwndB3kFsP1~%lt1;ZCZwy!i@o44}5};HGX$dW4d`?_7L-XgLZXa z!DI3pnfO`FJCo00R{iq+6Mx1&!{@wkd&aq5q`r<+kHAb%6MAt{K-D;a2@o@V`DQ|3 zbf$@~xbAQoSmMVq#l&I3c&}xf+Pq7mfSkQG*RJ2&sdp%?72NLtpA`F!NgLNk zh$S`Svt5u&=^N-X6eC0+;{HK<29sZ>W^iOWTW6hM!SHdr+Am#vympj*lnqYsCZ+28 zcsQ`4>xdK_aZk8Az+2@l#TQzz{P@!=z*ilnd}>uP87{ zF4qcezKPM$5io`#%@r-%6Ef!=QiWd5$fzTo)~S4UH>%u6YoR8u>4jzNw2~y(2w8Uw z4%O~`|Nn>gdwNB%N;23p|QSLLRyY+Xuyp7Qi@3OI1% z$9nR~r5fAP`2IKF8g2$0IM{XE&(ug$-mIatUUqhT(j)iTf~w=w4p9$VWSg>`Z- zywBnPNiNcHhdCa~VUM;6aK$CuvL?NIuq+51ZAmz84}m(rNlc#f1@s6s{6`wyt3wyg zEeHe`qNR@C(Io6bIF!wO@HaA1@Wq%^_8$=xI9aHsO3PB^eS9x3Mak!hOnZSqL~BB( z*cnjqWC%m!lQtMr!STl`pQMaEfvy*M`f{(58%d$^R0F|=E6Q+2TN`9!Nu;P{46;HnU110T4b{e(9wkr-qZhZ$_)e0TdplsVR3!0UQ0fafE~GK*PN$F z7G5+ftzWEL2Qp$nQ$~6cQY>_ThqSb7HR3_G$cqb{eEF2td&0S4CUU(VAD}0s-B}Ni zqn9{(i#-=3l4APzA9WeFtnq_h(PBaF0E-LHV27-)&Ze{rKC zZW@Mtc;L8-xpYV^fB2QorJ$j4FIH)O!%bT0|KUDscNQN$k43cG^sqsvl!D zOthcwS20_R*MazUzKN=bd+^qxT8ALFzYEX#Z3p`j3-1>Z7ixy|(XpnD;^sS9%@Fmg z?)@)HdIY}DU-ak(ROd|lEDp#PxuoqFQohn3c^Mui)ERZ<5xWV2S6Q7)7Av^QAJIY^ z3$^kGdT|g5xg{F5ZxB4Lz2EW%&@%vahd()7tgP7zI|*l}f2lV-2EW)(a64;)j{I^* zSrc*Wfvmvpq&qaEU@+6!U*Y}V?fE4Jub-?3*H0dB|LI8-Kc$~gon!WIKJfd)O68)7 z$A9&q{z9*JqXJD!o%@;d+zRT7=Xu*usreTUALrUgx?>^zK8sCTh-4Ojhlv({U;g&>2V}GD)QmF2y)8 zyD7kQS4jQ`Cw?zOGJ=2AYi`ib-ChUF|0ZeHszYQJX%o1v9*F9an@*q2GT!(qLm`Ob zwa)o2zhtlMD`#!e6Fr$f?Bj5ERJL*ROXW_aa#0sRpIERPf^m%xkEWW1c_XNsBrPs# zTF36H>*QfUsIE){Q9gd)yr{(2`!Tys`o&EXr8Tcv)uE5AXz5vj(DvkB8));mii);f zoXT~xs`Vy6VYR$7-9mhiJgb0jvr1rinNZ9!49ye=H;S=g-t~@IHgHAyNU2W?`H@wj=MJKOS-IPbiv}{>KZV=ntDZq^?4O6P2*QXG~qei zT4Z?sI#UmM!_73qAQWKL6Rz?kNXcB&J9SUmoDld5Si&c?I|6>H&cq7-zfl0%Z3?AZuIGA#4Df=9ffHNTutxM}dwTKR@UQ?SqTg7T!@k(oJTt**r zXQXl6iqx1>?ytKWhB5|OMcUv@1)aWNCZ(z&_$hrbII0V&jx?4rXLw zKzPUN`z?XrMem`z2KVv^7F3)C zRF>rDy3&y8I$qxwdU?Fz`z`=cMMZt^uj-dP{fJyTMmzYh{xM8kqH8H$NRh{XnG4@Z z(DcA``;M;>bG6f`tB!b+Dv=D25j%>2gf~QfHvRj0z^0D4jiIpt)YoLR@>> zZ6DPmwC@G6tb<7$g(95!K^?xJcz0ijE~vXI05n&HwO#b?0)J-BrLJBpX8{bxByc`4 zua)fS2H4Z5=N9PS_5@b0_UoG=(*I>o1pOpbyRA7^t(p3G$4~fR)q>HS5&pu30(+ft z_Lk%I6A61LGQZvZ*CoYn0bS&>}9H3#yDhUF_uDE^#_G!!FJ}XXL z|MYoDPe_Sp?AgKv;=%5cGD&R^v}=pwA4}n6u^xOjK=&3t{610CC|6;rL0X~PsFC7H z%(Pf9RV8(F&NV0lGtvV&>#~Z`p{%5)bzH_!U6}IT55~HD2(-3rlH(ef&%`TADet2C zRB&9$TXsk#vQ{mjWqp{{SbXbCjzwc^TWWT9cp?s}6?$$eqG(X3W&>N=wnm948pEiM z|0R8%qrz>p`K`bF;KfJxdhT;2gKAC|FeI_3LxthIM&6_(B}Rcgc?LkS2#Sw2Z!0VKx?HuRCi%Y~$% z)}a*fMCC=suFk2>wV29>`o6ku_DG?#Z!+`gUsPAQC57){r>{a{B60^NsT-4}w!`9< zSR69-0c*Tj zq->t#@_SadHSSxnw)as;DIT+yJ}tJ5c#7B_DIx6!_LW?42NOLOEHPQjHw>*>*V=Ly}kJ6g6$?Ge#z=ZLRTom7wrEmYl$ZaPrG zz>0%g4DhfkOMrI5McHF|-u68-TK^f@Q~G1N6X1Wf3i=RkA*~;Q^eai0dlweyxg>iXNbDoq_79VgQMfPIT zM9cDO>Ph5Hdv!>_^q>}9=(w_=~VOfjO1 z*2qqf3@4lifSl}i6wBz>&hwBd>SS0D+bb6V_Um1bVcWRMebXHmzm6mcSiQ#r^yHOi z@AtUtGLazyF{-a9^PkL{=!NF0U1=+EfeqY02&G=1W3N))e%QoKTPIX{ETnzNr0rzp z88uS}#+2k2{na_Wt?ItRNSU1PX6+bkIE}RDdu^&V6|358Hj96*rGJ}_h_2>R%>0^Z zV*DvT$o9BgTtb7ArQ;HZop#o2^!|6Mk}3igl4NA4H=>v9!XbH8Dum+#LZ3xho{`tJ*>gT2KD708j=fynSWJl#gC!*7mf6wETI6 zkKuvI`}g4ZsvL7b);5V9+kH2RK6@t#dtLN2nHYra?1&x1dx=?W6dDP%sk%@CvdjAk z*23ogrj1`x*s(7A2+uKhK<#kgllK8oq>4bzsbaPqOQ{Z}kjt`9wbE`nK}V~dF1O|E z6yg0HeAY$-mBE{DPa5vVrL%!Dy~|&7YGLmCbz+icT#Ge3AS9fPfQZNd@uSPzb!q1K zB{#0BX?c-=>NdC@!x>ksq|9sexC~$x8O;;?O<5)0I-0Pw1^_FUA3i7i%ZMKU%^&+) z-{eI8nPmd=pM)a;CqG&r7v4|Wr%nJZ+orZ4(Yfvk$>8=2)>4yUu*Us&#n)tH?Ig7A zg$WDZyQ0jmD<9Vxq$Iz!_h%8|^=!*WjE48~`>TFBQ5m;hG(8v-#0+U!WpKNbWW(DX%30hHFqubAHv^DwABv{0#_+%#K3?pQM=EwPceq_>RsR zRzz-hdx@jlm;}lBST!^4F?Z*&?_nrR<~~*%7|z@Afu?!pUV65q%$<|bn*W|J{{ISd z56$SsaJO&?BIURpe(OR*k8@5_gk|sS{%aa=>Xols8@~JwN6$_TtdcwW4zs!2-pc20ylR&vrPvmOfTF=gUwzHaPposQbd?2c0*g!sRaF6w z%X1r0dBPQ|Fm_e+MAPC|8!kK;c(mjixh`Pm`|tss;&Ui1hlNSk zx^(~G^A%td;eP6t`0sanI1v25G;H7BW(b zYq)VomSeG*F9Xn8nAVqAPdl3<2xJnT;BX~Rm#Jk7podz`S!C%pfE<*+Fi<(t zM=`8g!wy}E$4Mr&t#N5*MK&nSe8?Iq#o0rf7WqD&0D_=DFL(3W*1cI2>$EfD z236l!_Zu4xO3!WOD2r-Yp56Cz!by@BM4Dm$Y>oe6IJg2!hm~u;Ca}PVN&2?5QzodE zG`8C2?{Q7uZ$9@5>r>}4WL0!MV5POIzO~O6{0~{n-sM*;cGuY1a!pj;*hXB}IP=b4 zSPSHGk{VfgAwm#2!>^JfrBdm5fy&18QM0pmZk1Tq!~Oy;T>F^-x23)0+T*_J(#G@Y z!&+&azs!ECgmP}ed_uj&)BeYI(F6fg%AMkG_w55aKs90-R^=?+H~S!2M5cJd_XWyGXJ)BVw8LOAqj#nkt0NPMol(&2apUr^KUcAm0|ymR zp0D5EIv>6lFg7Q@Ott*H(%Km_T%>Eof~Yq)>Y!}__p81Ex{^MwdH?6-jE8_!i!kRe z!0jOlFz)uc#IETJ}cqALKXEa0N8HWV@37{`)|+tGZE^Ujx2 z)kB4ug0l-CqUX$#*K>@k+l&ykQh+A;)AJ01yuuI?69c1Dj&*&H;S|r~et0r3GKww5 zr5#*;wmwXDyh=7>`SnTGxZTjel|S9O@R>>yIf-<#92FH!^?pn`R_SnNwAM_4kg|gR z!EAZZ`YYN_?D|KG-Ld08Glue#f(%(cxV*uF=>*IaoZrbeE{94Kqi0P$sYOIC3+nVI zI`K{x|G}ectwfe{}$YnA>q zk1k2<;gV)9Us|oVRlia<`-k0BsuPkMnPc+aZ(SN;H*tCkYfyPy|I@c+5~i)TC9oQ?u&}nX67@e=16+fO9d5X6NhEJ8^c2U-HK) zRV58LBL%B2Pw1uGrJC(`H1BE3ALh4(8WJL%)*5rUZ!MFyr%D>Ko@tQIc1F8~*n6cs)30lS$wq|d0m~m3p`E1py3uyXm0+DjJ*0t*K=h)aA?rWX0HbZayfITIi zM#bBGiey_O4N)G)OY_POGRJ3kXN+^QZ&hA6aVu|gcH_(N0np5Ti1>cxcct}rGy8w2xwWEiPj+5}rI-7E*G7H2EV{p) zME*cmLb|<{eA*S0bJIgR*`6)sV+<)8itIVp+)Z%`F#f_TuP z)$k7%K(Vp^%9lSclhBEMk0Ph^nM2lxBpTO=ZiB`)Y;n`-&Cgur7a*TO)F#dfLEay& z`QJjTjYw|3@)H~|C~awrK+!s9?(R48jx0*~XmRUd0Al=#eTxo%3iOX2W=>={bb1yj zBm#thUCm|ZFRQ9AhC(kOzV2YqI9?kSdDN~;w|0kXamSHCD-am=B?~q91h}W54IGUn zW?1cG9td{BOAPXtKVRt;7t!>_+tiVWbHIX8Z@2xOvyy;bI}2*E2@#r%U|?s~_jFlhq;Ts&7#)9(VcG?IhJo@2}S7RPdr@)SSc|~LCZaN6O9o{_r6@()@8`+{T}qpl z2b{4ow2iNiY5)v_XWhc*FgL7Jn>wN#xu~&u)S?0jo4i7?fe!A+tu5;{|40a0rVhsKj=ym%wq zz7EA_&EgwZIHZNH>lsA8ew=nGIGqEWUfV>Qc0% zcfO%Q69}+{^Yw(c7;L2h%Ni5@D2-9uQ~D}mJ)$wOF#$YPD#(XDft42Fm) z0GTb5T0F%Z&pWLl$hT)GUesX%$S9Aey4A5w-PbIMjI=hfF9zq;Sy_NWpCmq6@ za_z3?85Q7mGfz}7fMn#_MH$1&eAyDIDOUZA9eODzlSfwDYr&fV__?Yp7828uTgcmZ zIv4AYn%%+a?n@~%);a(rGV8PNzn*(;Brv-w7r34Y?LVd8E4zI?K&7pR_eD}mKW~W&zgYZj$uJX>m?y(^x#2f><|4oFNj0HgVjC0h zRQ6a686k^wy!ZHuvWeF}UrA8f9q!KcJ?mqOG|0^GofW?4qO)v4FNyQcGV2mZGD8B8 zn4wzrU7oP3DNpNubl8yzi5b;6C0q_O-%gWJYqw7Rh8cym(g^I}Ubk2`$6m4$k_Q$w z-=M(U2bsY|RVvuFwXB`CgMQ4NF&ohRj@8)Z#GA?LH>wD}jR3{_qW&g5R}#<;AIxxi zyQ087Ee4a?2kGzn=em`bJWp-?X>dmEj=?eYFF8p?f+F6#A^ZZh4b_cb;@8fre+u$yR&S)7tku=NJaH-uAH zT8#$`SOrxdmy2mr^nnATYOwz67T~`8khsb8%d;jJC66`mkBY)>9;HF8Dh)Ci<`ZZy z<_I;Rcvss9aMjeo3n^ ze=_2{i4L>QIG73J2rT#CR?_hzQHrA<#6)#0?d`gctN)S<_}gX|A0L|=!Y`?$jy6{! zWd)Y;w%)(lK0pt4TbbUAOy$=3ky=?Uh6}{@oXhR&3ENX<91rr@B{`mDxQq$m{g!u#rOw#Lao|Dl z=4)T^x^)`%4=!4^z1&?e>03hpK+(ZPNi^Nc@LjB3OeTIObekt(R#2Zkl|Uu;M|`IH zHFL`okelb9JX75yp&zQ$Rv_C}QZ74dHt)^w-4lZ%VVc=#iNp2|Na>%6sD2x`%C)6$ zkInF`Z^j+MeX43_hv;GU(e%%%`0>Oisuq1cTbX`df{hbHJ{NMSiTMT!L2}pItMl?% zeL{u-dBumZK^pMwN`p;TR(|0|OV(U3zyJ2G!;r@fh)ZiFHgV3Q|0e=!yn((!%gfmP zWbIhl*pzIerCq7X3ZX^T2jJ3x>!5klL_J=cSAfCUtre(2tj$%@L?>;J*}20$2+E zSXK=G?JR$lr#<_F{0SPzP?7@>6{cFXL9)P;_C07jxVEtXCuVGX3f}i}%TFl7<>!Y~ z&M?o$IdjSYKEU0i?2+nplbLb#%dOE2j{tH7CLom)I(K653* zRV0s5CrTUN9D}|~bx$zp%mr=b3Ons8=}T-hfc}ZyLs&@&40QNnc=1fDBYf6UE8aTq ziK}dzU>Af5qR(pVP?5*$Dmn@>sV#Bl5dJMGNXHL8)m)#1#J*+S(^3B7;>(oIn#+s& z`s#P4_&fE&XZLVMrTknhxn7F@os7}Q+gLNwyTsW(sf+N6R;L5aT3S`vBu%@g)i*~8 zd1)f?PM_-v^9#38AP}wrVQ@xyL}K!+nV#_p+BJ8`AkV?rYttBb#?8-9#n{5tvLv#%v<>0q2l9~J4kj1 z0DKB=^3~jPED=N7o6b8#yLDLZ0LXic?_z-zQR?g^`}RE5{&?wNC3~A206cj=9F^@L zj2D_FrWFOC8?9>dd+2p|+r4r7@@u0q9nvF)lM;IwOrZs(2u?3Rp!R(@OENzMnv*Lj z^Ydg}-@gOoRiHXlvyr$3?DqaD7Z3jp?vLN80WJn(#WJm>bbiL8etx+0I{G8C>R85d zz9ev_QSy)Y(+zMhL}4^STSX;5QCKwI?(ptnieaswR1-S}>f3|7b)3CA^BcmIac*<=DL)3KLkvo?eUU0{p7P%&J*4E%zkZ-Rj({o;tvwzB|BxMVB{d`7ItG#}r z*H%~eWu)N6*LV8g2zotlxpkg3=i1RL+#;YG+1x6s)KX8^=7c4Kr9$d#I_*n0qGMoe z8gKqVziKO93D!)yi-_}^XX)$x?0*eBDQ*hW<^%8}Wf1a`!3pPiqQXQ8pv6djXT} zj~+)+gz+~1HBI_@J!>2Du7cK$&%-T7Sf!eF`NUq2rH=ItayC1C?J^)c6PvBU~1 z&}{7b?nrkzKSmY)_=YM=5ODOTglJMd8GafLjP&kaSQz4uMvLK|`|)ZQ`CIKZd>Kyu z-Ycp{UT`=@v}mT~P^u(uxt)bs)XpX0y&ZfOH4zz`BLN8oY9PDrWS`9now^yt5;che&utZo^!Ae~239_A~$nouwp{ARMaZ~$M9>&MshoFyvJKFFXVlb?7 z>VpN!uqroM3C$|z{5{tzkm$7ty?A&KwAX&aVa-XPLp8K@=QVDRV`J$dq2neO+a=(O zN4*XLG@szUshe;#R?#D*K-G&rh546R{<6a@25v1-m%pXqe;JIX4UFiWuM6ss%ljXt zPr&6ZK$iAecR5T+*ipOFa}|ba>&HcRQ03@K*;WHya&Fqw5wItNK5z~(kow`I zlH`Kdma>*FiKb-N46cTmI+m%HNyN}%xGa*Mx^AdaT1A@@nl?(DaS1x|!oXX0?_A*m zXrowXzlqDAYE#2;u?@B?a_TqVDYI0$)hGrRpl_3{3pE~@ArSb)r?`FRjXCRr5!ciX zIIcP%_CrOpx$EvLxmkPMY~m)PWnpJB`}v^vKY6})%-b1?sv=hvf+VZFy7;7nfBO<( zGS&ip?@9HqhT^VG{Gsld0B= z?8?d`VN_Dvwh>k$iE^U;jr?~2A1XH(-_*_vkn!Jc`#`k5U~!}VgJJFNxr6WLhzjE? zv6Wjan4~7x&fNAqR28%_k&rd^?86yKbxq@f%5+1Ac>QSx1J8FKuD%Kg-083Y@c(s! z=B^7>6r;I7rJOQg@GgPd2NoLZ<8U9iir~7AcjN*N8_Y4F&xxEtg31y+SC9IFTXE(A8%NG&eC2j?u_i6BgfV>xKp{U^-aVf zaDk8X`fFa>7f=1~n;SWNbDIa(eSg0>_~FnUguNwmGcPo~+ePj&rf7N?Y2dgI+(L13 zXnB88lLc+h`$Ax;pE=);q}Tot0K+^0BUP>T%m8cgUEcN4ezCe>YNy>)(k|&czL>i{ zzSSh#xaf{�jcqo@-YJph7dcbx-?BJhi6Hu7g#DU7kXtpVm=wD>tfZV6q634saRW zleiG8e6W3p2Yps)s$g0>Inmc`lC2XLWOc zsFty6!&e0KH_(hui6qAi%Uxf-lB6ZOgn*XAaFa|IY{KEZA#n$qMDQM661u+lXyFe_ z`?AH;YcKxSd#)aOPmO>U{cn4{d3n3QjO0httyzD0iHMWGSjB*rZ|L-_1j618~Y37x~5>{JuYOJz6G;g1Z(r4lERW%wcD zd%s;Y{q$(8z}_`IO1f(o#3sYtP=0NK3pR8T+C}#Dqn%vLz)&ZJa@Ccoy_AEk=D-oF z1no$h(nLf*^Rw}Fsu8Yl+;e`O3sB1b0jv?NeMU#eZ;rSwIHqWQWp>~5E^K#8y?sLf zbYaQFMb7}eIt>S^K@!F#rLi0(A^l)(8=WP5RppIhfmM z&DrQ{a)g8C2`MudcL=?{sMBX-iT?>rl8=u8Yv+LPA1q#VUIfYB1p@hhV)_^9;83X7 zsxo#(bBd-3n>rE=z<+8XQZGtP+7F z@O*KKF0VhWb}0Wr{SxKXyIx^-yD;K*FRIck*CAuq>?>jD(cI>=im4>CQht9cH?x zfKW^SA?{{H>Pl4Ejqfg_-Db%?Lw3_=HH&cqlr_UgG0!8bE0i3_@BPv&V@GSI3p|+Y zvea8vA0<+q86Gnki^3K;MqeU78Zdub-IQSxPr{iI;fHKaSZOo0VbXSSRA%>t@@RIZNB z4HOa8tk1d{9(3cTm6W+a1qDLa37gF_Ml`Xxttu`$5xF3seLm z)c50XE%{dIt>06x9HsrStC8}-OhdSne?(_X-Rd<4Zgw+IqZwR|x?nOm_8PAF%C`M( zsttc_9zP}#p1itQLUb3@7*OHwHfT;V>w{+7uW;2j3k(l*-FHwl{SYHNeoI5&Vvkxw z%xm_$W8&Fs)K}B&{^um8$WUK~=O`Gw`9?j@20Uj>;W_86;eI+{uD^}(POtmUn$;DKwL6~KUIv4Qe)IYykhA5{|Ni;u%@%7 zZDrYAS#^~lBG`ch=^||r0~T5m5D7grr3fS-N|mlIDpe$q&=Um$1PDDK-GU%B5;`cT zG-)D)UcVFe*`>-wJmw(oCpX70IX&YXMV(Wt)ntAd3F?B7ME(^gcP+K~nu^iq2? zdrsfL3CpEysp80GMu$%;xp{7CGux{AofZ?_6Rpe>x@cyKY1b?~It}0CiOJj@AsDBF zT|yZ@=Beto=(|AD0X|(&f5MS9?S=B{S-I-i)LB8XSM?#!->VoDo$tVeG_51i^F6B>UH6-jXr?_t(4M3keR^hug&2 z%D1OwkqV4l&7`$E1FjDWrK#2aCKH)nqBbm4^BBaFG4UK#2#ps%w;3lvXgUQAq5hrc zOX&7x5}godY*L+VHZk8&IS_8&-WoEUv7v;1#LOE9lnB{v6}WMV|5j0p0s`S&r|haR z$I6n|BF;T99X@8Ns<*qLhprHNRZh+V0)1p;;6om>;J@6mv=#db#en_X+rei|HqcuP$N$epdAQg{@46+l{ z5{?3dFjfzn^8N@5QyA@Hg^S&*L4y~`BXZje#XLq=Vi$1|>R5%sn7VM1j_y*Q<@YM@Y8r*xi4!IbYU^Ni_%fA)@oRGM|u}2?u zmlb#Z-<0z*XOG_|i>zz4{F)CR(7A=>8(F;h(5Fjr&GziZcT36vmbjqyS5iWnY*z-v z0}IX77HF%Yp-mL3)gp=J^4<XWjgWSc8g9GtSR{O-8hV#ds^`Z-js3gU|HCcJ~Aj zYQgxF!P*C8ue$AdWudy)v5DevExp~X_=i{tmJAYa7?wIC5=+723T4qeTR!MUrxSWN zn#fIE6|eVdNt%iYUiL;v70zt>Gx3y;l2ThT>40?h#M(2+=8NrG=lUHU0EQpd-c0>n z`fQ)=eRVYe5Zunc=>I@uO|Ef39YYUMfBs0kaeF#qvtPZt=fJd?^`Eq7(D7x$YOP?y zL7UiEAV2nI-M=j;HePtC=NJy578L*#s@3G8mDlgeS6BOs(?t2QYx7H_q{ZpAO3@m~ zHF=l{eKm(m4^|DVq_wveE*2S~-?thLYcB-DDXfIOmn5I8$chY1-pa`n6+}@79gw{} zZYL4)Va=VRGD$7dlVh@imsb>`{7Z`$~)xKTkRCJn?4!-O}2QPk~O3b_6fsnIK1uhX7(~mP=Xk#%uP> zH9W*HfFzI#<5h>;PicVLFcCTrCZZ^Q{lR<2?9zjK(rWXW0R%`9*2xSXCw;Wr=i)j) zjc}+qK+rb}%uZpbHvzo65Aik=AqCP$xp)8v0(gapzc; zx4@r=+s%UncowFTa4k{(-OEv4F|*;{x=ld7r)3Zs>^vr+0{k<~MK_L(Q2%;vz81S$ zV@BM-{1CLj$>lgUe_ayWO9}kYh7>FOy>g#kbZCc)n4lVzGx3V>XNpKcW5zFz;g~W- z>6#9FzM$RrI?m?;gXTV2gxn@OsuA<*faNjdT7O?RAdoumsEE7p2P9RmzLN-9ZZ2up z^b~8|&GLe^c@OfF&M#lr3aV~#X{i`yzwKZA<8`TK&pGazrsl7a+{=gB!rIhKM87*Jw8)|7>sQt_%G*_3dk}el^M#>~CkGdL4?rsb-kV%m)*f+Fz8J@Dhw-vgEl;#rQntR))=Sz5=~7m8xcW{{9eA2fYG& zC+9Cwu~Ji}v4Nu)d=k1gCdPp0=^Xr4@&TGCCQIhLap2X*_I7KY3Guc?_+4Y<$V}mk zJ3@1*WSm-`_^evomqZzMCZr$$OgMf!TJ-3j!(sc`;d{Umj*IqM|7Jp9(zC>u+CfyJ z_K?U82%1)$j1Y=Kn@lK+#j8R0OtuFRG2U=`a)<=3D|nOa); zX~vYk_sEk62w}4~4|;EfmC8~o+0;rb;Ikd}UK~@IBV|=B-JENk4k&Q@ zX*+tflfd`aMlnzCj3sk+EOjT_S^OAFCYGE&?N>3Pmq=IDh;yRXXpp;q+vK*qFyZrB z$K6jlyhH7|One$FPOWi`KahN@9dERL%%g@-a8$g(!5PJT*W+XMy4@T|%e%VQiTxEm zo@YLGgV1UCIjt|jX%J`M+jOUI=pmls1Nr2KF>&08QlM-&a@~fn+0BrcYt6Bj3>~ZG zaa|I5E@C;qu%qW3+s(8UiFP~PEL!B=S$XgD`5!Vi2p(wX_8h2~JO+TzdocW7_3mMUWY@K}#n z(@g;9f6f#6idkqLPdQ8Bw))j`xo-k2lF_KwVM&A&1@)>wS*p6Wo&I|kps;~q6k#*2 zgoRe{YetxzZ(5Q4fFEC%Wr}rvWdz`I;o8m?38=}HOnSqhBe!1bxABj8eQWz}VCM$w zzjCt;^*abM_JPLmowbg{UVw;A)<;b_e>_DZ{Jwcy7L-(MEAS^lq$lRoV%kM@9!n>u zh+d&4?aRba;;W<42<(H9SPaI=w^mFBv=0zhO01y#wo{v0sAkGWP%}Sz z-@TW^dzNc?C@4`ou^P;mFnXlcPuQc*_8S@yOHj@tbFwH~!?;MKiKT|~t3vtad z4GZ;)2r-m*lyJ!di8-+=Ac9s(u#~G4RMn*U&riRtx1}0|$baNRD9W0`1iO~q^vk8i zxE5m&;3B{3oCV|9)Ks2jC$TdHoD5w>$ob~#g{8_24C8UjX})x9NI%Zx1L;OIqHV@fiE(yA z;`<%K9cNa)e=#(enpD!K$)e8>u9jO_G0IcyAQ43nCe~38n6xR;%^bPFG6K83ROZJr z47->?SzT^342x<-8|h(iM4*egmD@>X)yQHD29Lz;`^!2ZP)y(dmke(o%t7IMXL&i* zWp_Z|^;cXSXkBnVWIq(~npyPT6bMpH<2zbkEg=~8L2a$YE(ZVNTwMn2&%4hvVH_i1 zLHVzP)$|(4O`9b~I-~3E%9}3_Wk2?d(Y`XSPv@w;CHbk8EP5z5n*)bM%;#V!U)NWh zC^5Z0KBLckW2QRsok*v;BYi$tg!5t&y^g+_%5a+xpD!)~J*xG?>qj|w{^EIhAV!Yu z+wrZtI4*%U3bM3YWv{31PBV6K@CaQ;yy$vP2s!uanNe5Ew-Bwp)%ym7yqtAa#TX`z z^b@AY)Y2|qQ8Ey5u3L_Y8$qLhOk1cSrbyQ&A)?3Df5bIzCeZ0INJt-DKt*?|w;Kul zd7Aa^h$XOubN+%bYNhB1e>+3Bru+PQZ=>ldJr-YHx0nfSA17&H5*D1@oZ(xR#4LEm z=2TwAEYWbxcY@)JGiOMyl7hb(GuE`7p)BtkSTNJ*2;@1Y`A$pzMRfH5NMH{2zZCwF z9)S#Y5o9pYDTrIpjtmz5(g_dFD&t4FLy&v9sSS0Jun(F3X@eK2)R;M0SapZm z0ckLFz&~V)%a$KGhtw%{Y&cb-8X#>pu1?2Cv~4Wv=ExM&>B5!oj=)9L0;IZ1@$G(% zge@Q2fQcd~iW<=xT-P{~<9F54A?%&H-tM!iYYH<3ir5gH0R_HKb@)El&v^>B0*tUZ z>&llI>Zt1Zu{5YG803wpB<8D8xw}2$!>4w@po1%Byt9ZUV%?^>-(!3e_6E{ew|ozIlR8W6 zYqT@N-;qnCxb%^P3IF;>mbG%*z0mtg?4&go%)g`fJZK`tF1eeo7XJRD2j}eLC_BrG zP@Vmv7J)V}xptC%NKjQHPl6KQ3%MCHppWF7`|@a1c*Q~cwZKq2M6ZJk-KU-wuVuC{ z{xF`{V|z|ZRLOp7llU~1;SnK2YcRDLYYdY0H$pWO`lbqM3mZB)sSuNhWd1cD@%v}^ zo+1%;Z+yB3&;6MIUYR=6hGizZ>u6l}s+j2FeWX}Q-J%>`#qhlNMBd_se*ZSN9=EIO zZl4DW4g@a0i5@60-$spm_dS4HBv)Cd??iVFc6s(`|Mj~+8hG(p*gYm?{wMzmjOzPH zm_~~l5!2PC6nVFINIX;jL)}Q9%?^Uho{Uh3=9`!?uu(d z|0jVh&@prIG{-*}#u**>_fsFtvdkyp+#mBuZul^4^%NDK-j!q8zVIZSO%Xw5A_4N& z!x6^Uot8lafxiYRgi&|;>lGyxX>Be9Rl-W{XR*H4@7?I&9GyrBLotFv$;ax!gz(QJ z&l1kWgc;D~ax0IS>8Q;@G$gs+v6wT^ZnauigaxFj4j+mcSC`){E=FCZ$KCBjHq6;0 z8?5DTq-#Eywa;)r?7Q03B;3_wmenB~T|Z|p=bA1AJ)X*Sdvhvwv19ZQzZYyKgeV-h z)C-aBREZU0Es4KAH0ZuvWx)YnqFfkpf0_1oFC_mW@WaDX|I}>{qYaJacU!0410lI(|Hg+p1g-Bg0-QhLL;2v{5b>wAbP$%*< zy2j`iMgW=o#QZ#!;xr;?sH4{Kb6l6o@?x2Tyuz4Kp@G!B=5)r16*adO)GI>InumJb z_Z0KA+QbDHoUto`Z%NQ*RmJHzy=q?6Q6NuFH}IQagUw?2RKDF{{1cM$!k`Z+r`QtF zF=yoxfiw3_Yy29ppydJ8YpF3a8@X!S>UGTMH&AU{cVxGDF>7y~{%P~F5&r(qAV1UZ zedib0HvMnY;<$ICE)@h(*MD83a|)#b>uA~VB;A_yOmI;w$Z%u$W@u(*!}%7y)C3Y;{bW}c zYjxNMbeMfW4_tQtm9+n<4vD{k$f6YLH}mYU77YpI(qAo?ZW3v`6e33#7tHS2 zXe(*+oG8`29js+yc?1hKbGoB>;3*?L#V~0jdK&Lwz^6MNN`x~r%7fhnV6BReo=(qr zuV!M{9VQv~#dsz@G*q4x!WSnI-!Mu+hh~>*k4IIMkPnz?PG6w%nfZ!uG|k*mlWP3= zC6m9Q-w`O?&_sG0THPFVed`vAQ>m^1`h+?=SDTw=PixP@`Pj^z$vzCZr3lk2JAi%f z#b+>W(?bsCnUo(aXiu0D0ULk%a(u_G^2hQ<(o z$QP)QS7dP3`nuRizU!C=5nXoE#(78tL^tBkz6mjeip{tQy;fqu5NNDuNK)VZP60O- zUt+(7JmKaZsk7W~gEw^74jeTw%A?1Yct#(pSEHC)P7**Ini(9DyQ;UFM$b^8IDU+f zL9`Zi&frFPQ@PCH946V|;-oMC#!rLZ+g^2@3p5fj4-)HQc6|T+hwM)3*$@1@I}R=G z`0lk|ParR~ZOs+>(Fe@VMSSds?8~vx#TXPHi(7rth$$f@_{qVNn+f2S>XDOqq6^LM zIiGdiqPuprBbO3%cTg>qVTMgjwyjW;)t;H>A2qA+r@>IY zCp7W(mEmbYIm8d6W0?NZ%zIsZ?ON38f+Fl(Z~4)_l(ep6QPQa#Cc_p)ROTN2qO}um zKW=+rDDc9|QTFCLVWNQ@XRc;-K|9TiVVam$zdNuDEO~sCJKLx6#zNXFHJQ(`LR=VX`Z)PDZ<=_&*7e*Q350z2PMMn=G%0< z`8gPh&J4qH7)AP7YeSbZovd(@p1Fcwn8LQy%Hfp7BIvlolK?i@xyk)vbHKwTUc*;I z(9r-6J?=}&=prqK=SanI9jrh!1?hOfn_6I`q+sQf z2i@(sUrnIy3j z=UDYw@tr=!%?_QQ#Md!@-t zSiC*;GR@TqZr5I`$X75=M>jr^-mBPk(Gdg?_7s%oq#F z-Sv@2d9!`&4k3=>VFuMSPj%#?I{vnthSXvDA8Pu(u3*~S$?-&h`I=kA722T>aenQP z57^0Xz4h@qZ)dJ#yJOPKtpYq@P?Rag7$+@B(`rCR6DjM*kPSEW^xn@90<3JvO=uhF zF93yoZ5eKpy^*9VxC|N4N2Ga9G}UM>j2St$$5&>T{kjcEqmdvi)(dV5?>vzZHWB@D z^xgcAQJ(*rYuIij=uK>N7S|sqEb@xqL#}stxr|GMoGBtG%YBP8F&KIuO&VHrgep5q z!L0XR$j#08Qry@-f^(qosoPSC&0iS7R!wmx(W;?U_F{si!>k5|wytT7IcFBFp(_`j zLqN_K>JRsZrRyyqQQRxnB%Gg6Sp2+)SXGLEi38~ z4*=4s%x?bNLYW@i5fx{RfxnH0N^I;TYOG5yV@KNWt`u# zLLyomBN!B`jCDmxWp${foiJ@&lH~z6_?-0BrtX8iMfm)o=y;5;jW0ROO0rB;OiKM> z{JLqOZ1i5Mw!#3*EK@HViQJfWCZ=JM!h+I%5 zz%vMlvyqCF^j3F~qCzZ*-UH{pBU-+s6pat(lbjq=KegcB$~z751)GyTdR5G3i0Yom zrgz%wy%LKhVnprq(l@mUt!Q!oEKeg~%w~8o=NWoLx}AFWx3T05l-?7Uhk;k@RzTt} zWCw>tNtF7|;9y+>R;rwl{blE=#d79Ikh$~)#8>~^GUERu(aS(X&zI*IaC-HJ$>tJ( z%fQIR_!oYw`q}$PT{Hcqwdq7Zd4Te_Qlh=&)z^bpvz;<^CIH0 zA&LrGk>T&f|4Ge4dr5^+gMr8=G+3nl^VN_vn~D7Z81p8#xY;WjCM zxT8BV8gY*>5-m)>*^=;Ff}9Jk$PSGiBq;mXQXTr^-yN6N6V)}qBGgsr8L(T!8;g6c zm6sBqwc{_;p;IuES+kMfB90$X4ZwX^ClOx=Z%P)3sJ^IG#>vfHr^e%aXX_l8Hd`)_@92ufR5$S8ZaP=|~ z%XM9$-a|25#G}_or$lx!3KHWfg{2%%wT@j4Rnd(ykPocGxIUYXK3M~O6@j>Cv*tWK zZyM$p=@+YdC1z-7O@v*ijIgg(yuXQO?;HjjW%qNAU8I1is2?cv^#QJMigpBHWRh?g zCW_(;4UulcJ44Q4sq&T=LbcQr{v<4Ax163aULx3vn#B@?aCGN}GiBPH7-LDh`o#)p z(eh76bv_mPrWBeRNHm@#6`&u$8!nTq!T>i$+7GH;k)LhI6FONc+&tpuoJ^64k{BB^ z9WN>{$ktlz=s~sAb=ya|XA^gI3ZZ(-G9wbndV?TEvNoMVce5j-c`ut(y=p%7BTU_U z>B>fdW1dm+Kiaa3L-6{Rf2yshICw~(p7J_Wx}X!uF~RLxR~uW9%o4q}S}82%J(~%A zV2X`wK+AJW%#<3H=(1QWXL>~1QSVDRbbF8i=QwkT;Sk%h?`l}H`a~oEppX&D)F)h_m^p!VQL0#ed zO^iH(C9lr2#*;|^jdcz;sy{2Rg&u8h4c_5;c1+ko#v;(byrq#GszAZBll^A zngY(VTOgi`p){WuAPqe8|)fSy3yIg#GTwH&4`ATrBxrq?S_$5;_y)SV8RvDga8j2lY)rY@_UDF^LM7%OkE@*xisnx$dlgf#*UBj_bX({U_BDFin`z%xHDjuu2ReAX zj$5~{vV<+LN`Boo``MqE4oKw}wVlEL_ox5!SMT#8Uk2?WpEKn7$;U1x_{Ar1R}!2u zJ&QO?b5C}_K3%1kC}=qTUiQ8N0{2_XlrY5W)!Am@KUr&*l^DaduN`I}_y&IM^;T zw{%{n-O~b_B-{;Ki#5%uh`rTTbbUy8V?}3BFx#4vl2w1y?b#G0>cnhLj)bYsS?oeA z>YU^&5kV9I?J-w1FBoY{JuEUL5H)gB3>87KSs>|}X@*hin1=4`ZO3|fW_NU&0Z})0GY7>%u;4~$Is57dU?>Y684(}xkv_u^ zX%G77r|tnfLC(tRi*8^pmy~4g%9v!Euo#LSbfVv@hDbE%D%vV}^7X>H(y{1R?UXF2 zz{-lFImNEC%hs2+=GjwN83{W{vn1e|1PrbbHA85!{ts1g zZfe0o^GcOK-C*_RF?IckDR7b3;e8{^C?&>4bUPaIi`guHdc$V%zk*|@W!H^3CG3!P zE5bS>qWav3cd%<>;zZrEVqR(#0!i(dG%vk2!qT*{4_8H#OF!I=moVgU&G9oQ724vu zDlu~*(}akVy!YL$gh!~FS*)>i_nSV$*_j`P>jmPR6*q%d=JUSxE=>cvoMvko)Y9W- zKPlQ$s;H;Ev7i+}vu~|I369AppzT>sTo!4lmWh5Y#chs^X6MWfPnqqu%YT`W^yJLq z-?IS!nse5hgAzk^v8sBW@OsD|jrW-AU3`azDP~6EiLLCd|V=?*HV(f4~1fu@fi4HbbPw*4c+)*1ZqffHfAEZ1@(@c-#`xR^@GtCx^ve zz@*^m&Xnfm+N|K5VLNKDi6N?Y)&`dxrIw!XM~4u>;X+L;T<|Gpv5``_LtsFl|9PTx z86O-~0^8_=yj(YtFM#QuD5^GN1k;z>EO2?v z2aHvzg=+CAXn;2#6&Tm2d}$p^O2!>cJ>GXxp)ZP$gg2PdfEggE6C7v((x{Oyb43S_ z3-t~5#J6=&aMQ~_ z!dhVAk`-%UJO|7emM&!Z&CMjR^V?}-gI@K9^cN$O{@BT)A z=tqd`7N~(ex%-3$vBi|eKAmE;Rbgqqz_@|Hu+-xnT7`HQ@n7iv;qNq8u6+49d);rP zq>70Tv87eX8&TlfzkZ^N6soiFZ0lRML9vYZnVQWH=Tos)$~)AmAHL^e(-SB=6C#}X zqW_mCxoMtCIMO9-}aBXm;LF_-^oh- zFSiL|yUiHM_m&TMgR9bf7bdC-;X@in4)f@E%rS%NXi%7UJOwIFm5$aBgl-A~!JZgQ zh_<|Gqe}ZRvvbkY>ZqKCrDivpOv}UPU-hIL(0aS;$+{cWFhy^!3|QBZ)Xr>lVc|nM zD@j!dT;malSgJKCFZpwypV_#+%^EL!NslV-W0`7y)osyVV_59tm2!jm@wDUY0~ldj z;XN&q{ln<)PRIEVD=89unxP%9eaaNcG(hbte3j~|X^?QO1_*c12?wo>Y3}i{rkFzf zsw_WL5CxiDsxEZJUsgS|jSA(#@MEw^^hp>`7Y4ePf=-^Pfda3n0i)(+q6wE{WJB{8 z7+NSII_+f0O>9{4j4fOE9&M(5Y-jp82Ux2bWw}g6K;TPl-m66)&gXfKrW7Wv zb`=d`UEHdMPevkf-MPn!G^_3$Q&~l6L!a$1xUbG`-?ce&@$9VT|C=A~MLi>&GfcFg z*_sDf8O)7GQMA7~Ab7l$iZSB~Ey`jMCjANF_}J{Qw7HJH`CzdCNl3Jh+Oy#6Jkt#F zLv#b3OARdK?i@lTs6;UUP1gdm>yxW9yd(JpAKH0{6u=$}I zo$s>*I$eh6q*nvLJO*W1}kqNsls*id5X#Y|C77IMBQx+~9Hbxm0Eb6^_Epk8yO)T`* zNaI!gwPi3itf~8ClwX}KHQ|Gy4R2K7`Y|D3Px@EOk=NJe!BldtI)y}%S7+AytTr)< zxJ>A-cP9oRBn+Co>X@tqo}tZ*bqoJOvq$)^0vRn7j;8Lv$`TMI@K3js=|pq8u?VzZ zn@Ef&`z(mkpbQ+TCj`zE3SBkd+TvX(bkmBiqA5FS{BI9z#@f*o!G@dF zwS$)ZF6TOvgRV(!JCBJs;Dkm4q$2D8;Y$YrNFm2z2!nFJMg#C-&{0S7)$8F$nVD%y z^kETc33W)g)9(`%P$&Q>etKGb!aX~(dY9ADCUm;cx?t&(ije%rDHr@HE?n_Zg>KS^OeQb5mFYWxhh zv1Mfw_7*EGSBt*wR{5!zji&h!K$vJj7B?gQ|3Cl5>yH2@2fB<;z#KRb{q1`$OirVo zc=eWsdy6`vkp8VrNOL=9Fd`_iGP-2ZH6OJ zgu1~$Q2E#+XKQ8v_gcF+$3|L6SqOHWkoIZ-5~#PkrLJf$z@~XUL6A#3Ku6D1tk5`$ zB>;CTS@L|sC`;M!TXh$&@>S>!a;NeKD1X|1<<-Q#)Gh40dU)s1`F~?BO!p<{Pbk71 zWtB9Km%rkz93n(-Vme?Rs7ooTt%eJu;thBJSZRd3L(|-84h;%5tqJ!=nnKl?4+KTk zl!l1AZcdv~F*d$zBYdeUzFmi+JbYG*FfMZ^B>_E3zO@WL_IxdN-Jw7u#xV zFo8Z9#hT#B#GIo9l7vz8*04ZgC()hS5E%GOb30`az~`esDvj>n;q?Fg`J*BCz!m2U z(5gn`yU!8X6p$u+UG-8PY!D4aV!gtS6*GfAY|kqcy3bRyRySUa>izqG#l4 zSd5|*{Ah`>j|#?F2aAKVNFI*U0T!FyZRJlXqvJ<^dBgrn6G3)y^iD%Y=YIr3{!>oB ztd0bE9FW$_lPk@X7}39p`Jxv2M@QoKRPQsO2KQF$8lj{ZK?uv8_)wp&q$OTLq96`E z*v2Uq2tz`fnaGT^L3Q*ywbPdYAIIBR_Z@q1``vzpqG zL`~h-x+D!A#PckKp~}Nd*yUbxTp?O_j<^~YEHC>pJ9t^%^EL`^sCKBay93ErsMLiN zTTJJ5{C!1*C@kK;_L-EekL9|jXPOP&*K$+md=tHF@j_FZeQ^eZWN-Q*f&euKbwFly z@>zcN&l$!ZKThfS1-5fC{J*{9uE zaqzGV4Alt$Mxs+z(aZ%Pq%Ou*r(7iAB4No&jMW`&z-tj5>w)#<2Z{JJRH0X86}~tr zZ;E=I>|*6pp2~qyJ5nfbiZRexZTDN3J?d5^)fKqBds8tRKu$P}T&gW(uC}Y@FyLt6r(Dw9p}*doJR-2}I0DCh|tsx3wYI zsz-B&(q+IM4DwFSOy&nZxSPP6u4*4c?Tbf;4oaTv`|@^d8~^ev+4W=IhOPXMaF_{s zu|C(-PR$^lFIHOr{-&L1c6hVo@RP8Fm-`hzJXKl$d>|t`uloFPpRdmymp*z~b+GJQ z55|e^9x#=K*8BhSpSpgFE1WRe1kG{?DL8RQFyXQtB$SilAP|8pr3a;DO4!+G^p*Ch zbvr44dz(N!lkSTdS__~Rn~Kxlk*kRkonnR=)hW1%C=aJ6+#1+Ph8x0l4%a#3u_G`NBDkt>Z+(usl*3p3Eh$BrX3ob2O03kb^d=5r56(fyi7+jzX-T6<~6}*lM zb^D<{J1fos7E5RWIrpu2w%YOUHOLP=E;Sp>!)8u0*#TwVxAuHk$o0}uU4C`oQL7bL zgPBoUvoY-^d$6ii^gbV0uQ?vXbM;7F4^Zu6{>*3oRH5IgNa=WV>*@T1oOkUdIX&wB zollnG@U})t>3zhJA0K$FH9f&ABKZ2r4A9tU9<{s7oRHM_8O~i;DG7?Wpp!*73lfRX zvcqVZ1QQ>H89Ex_P?2QmxCx&O6&CM(p=Vmp`8-%VquLETrBta|V&k9&w`mq#6Sj{I zZi@8tcHNvpx<$as`7?-j9ehuE4*))ioS(CAEiJX52_Jg0O?MNHR|adiLf@2`H%V6S|7v-LCs z_9hTgVxk$(1)@hvw}{&A&?TVjFHI=}o*?6XElA#b;JV~rz& zczB)s$iQsPfv0k**)T)zOIc83Y91z$R7TAMHNuQ0i1Z9-K*TgF4ZMvg#77+G9`xgg zX`E#DxwEXGhdLX@1%wiHb%>!P|HM3eGSc_y-Um^K&PYG+?WoLD%FabHyj$Iv8LAuOxociP`-Lq~ z{=3~9g^%v-xxQdvhoj8zJ5Z52`*B3dD}McASXw=Kb3VgyEZK{c(85;Xny*u=z7y4a zYqb4tUpq;`5TjS+2b3TYY#E)3^OfN7FmrHgV=$vL+c*O(z0RDp6kr^`FAg*e{U`Lv zNU)E`t9)?do1)iGl2Ih%K4n~p)86)fPgNh}q!>u1#|UKw`9kn=6liRrvEo#<26<$e zBjN=7GM>*^a55x^1{3eSDR7B97t4_t}M!5#S6@^u`#6QNccX z2toR-{fvm^;D62-^nOBFKg8yH;qOByx9*-P%E0@|RxN@(obG*&8IZ+qYJ+b2bl)`vxOb-n?y=<7b;+{hJwi1f>3U-?~0I5}W- zT;Hn_$aA3>ftxI4AFc61upHj2qW|L~U*g_asa;{kv+N}#m#%hte16FHJCx6Oj?Nq~ z75C0FRsK77qGB&*%aQRGb4{_WS+q*czOw}xh7j$cg*^zB@pJ?0_b7oX%A2``wL>JL zNs6zKb<2@0EskzsJJFWYg1;@~ik8PJc#8krU^0R?M?7EwUR=$e?9?yMKdeSoeU6Q z?B7j!2nPa@C(g%y5Q%q&T8U35)A36Q^uH%V^WC|YV?=p)uoN(NY~Ugg)wk4$mOL#g zhKfJK1BNpp!NNx@qBw==vW|6XaX`?R9+}fnii2JQI{}3`7v34S$$PUWaTPMoq*l&b z>JST6gg7w6e!&^KIRdS(J&)F$^ywacn~ve=*R~aH$@RSIvs&x{y7%a8r<6gFmRF^v zTSPS?rhL#WPKn)TSpk>V@+V{XfBL5Y1J{RQy$kt|;S&BsM+eX3etP-S0c{umNwao; z;6TN_Hxns9#oz$-%W6! z9G-pwTT01kWS}nM2=G(4wBqA5)5y%35cB)ui+cF221)ZflbAOi-4=??{z`~Tx_4jm zFxlI#sqhu6t;s+rLSJkw-9P4QX4U$}OrYx7r_Fc4);*=FPd`g*vDHc*tyP|-sAg|m z1^Xg~7ro1LdiLsj)vSGNf6KSEMJ7s=KK!M-2tQzYNm%aBe|j)s_(u=VhQqcIL10sCL;VKA}66G0c&kM@8@v5&7PW}8yFyn zFjpyPF3sagoK`&WpJzQ#_Vg*Dq6PC$9e+{e{?HR;sVX31Bss7UYFsIJ+;^(rEdxl_ zM5bRk0C!Ji960~=*?fqupEdHs*QCQO_F5Myyfdz`70qA2QA`B1LlD)R!>@ilnWVBg z8_96k2pdng+L$nYbKr#3i%$=}IaYxTfA#UnfaTkRBe7O0S_YoU{)@c`2CVxvvf&1N&d^;aKw3l;ql z1efCUxy2;jzk_Sn{e$<-)8D!HbQO^Z17Lx7hS}h1SLbH+93d}_E%tMiFroYSn$6EsK2L z;%~kmCoT1dP@BHp6$A9^tnuT^Y=C^qOovylPGq+SuGa_db96yQCX9Q`w`pc5R({_9 zW*V7VK6HHMkIqi3jZP~l^vNAbiTmvR)@<#5{pdetDp`-xYDWBKTQRK_O?`j&(o7sV z0Y%}`$MfD4lwR&g7pTdUv+-9$eN;#7P6$fbG&`rA&9F3uiXHClGaE9iD8o|(Lrh^J z2tV9x_>`S)V9BPXsTlf;v=3>+4<&l7$W-4schfX9=6bmB^Mxd+v=(@8Q^@)9$7fBSn)q>GkwO^4U-AMiT`9}(5%N7B{vW&*d?92p>TGFR+X z*LvqOHQ{1?`=>cLH^0qnO2M~Q7zc`-6!rK^s}}okJuVFziMPrf*V5HHzJP3g)92++ zC(g>JvZ+tigkj!qjPHElm%jG9&Y1BWp;1TdIj&}7@n|Eed1L{gGW_`Mm%EGbAJ*54 zBgOic(iN*5&oDAjlNB4&E~=eVU{&cY_I~LtMSX1a3;t0BSHy2d6X`$rW0Lybk-n%u;v+npfOcT=}L_GKgcX|wARiSVxj z3*K+Ps_gvKcAXA{g@g%HlX+}#PB1O zRGBb!Juyg-@CQ#edqc^attsiyF`H^Y&Rr5itmxjy?tZ!t4tYuEK*0Au+c&;>)fD?I zOsvFt-U$asBP!}W%3k91cTQK=ZN>}9K^{tI1azaVK3-{qDO)0uv&}hYn0n_K|?f(jdL z4k}i@$ZpL53hE%lH4MazJ^EamW&>|#%-OIv7LnZZYL9SfvE}{T-&t3nq^soEpnfI( zJOOS-$9A6DGcJ9bP&m{SLpa~IK`TySZ(BdJyBqI*v+CK_;H!mhD^Pc4f<3Q4a3%-6BsxpaEMtguBNc#ivM zyuHzkK9E0H1GMyM>uT{OwG8jKk$WP41k^L9> z8)wv|p^jCoRSe?*cij7L1MI!#%h)<*VEbei z*M3WUnnUVc9qp}o&tZq@UDIpa zKS8+>x3XxW3AH0@;MI#<9{wvY?^7pdy&KJaosev&lfFOa8+pGu5r0-NyVo?dxt)CS zV>e_MYhIL5D)KgWMn~f>yfFF3?rg37UZnC5e7r}T(j94=h9G`dUH{%nQn50j$c}zwK@dbz#c>6S& z4dCS>`vASPhZ2M(EUKVs!Z#2%dx0Qs_RuES8hnCN3UM0JEqu(m^rNTa+H|Qd;930+ z$fq;cmbBucOz>9719jHgN1yawE)oLI#uCe9=Hae3? zOO<2$?LK3yiNi<%Va#&5Ak;3;-@!TW$O!x7r^$Fvegsz8bq6C2pV6|gK zci@LR^%OQre_T>`x%*b!Qs!)nbci1Q*M%gp{Y_GY8-9ap1ANl~S;^?ydGoL6a|v-R ziwPZ@jwc{9wX2{h5LaTUs+sIe+lmBSI`Q?h#dT2deLv-h+-LY(w_o~8^_~fD=7%Yt zS9>j=#=KXg;?Oh3P;Nj$=x-{v<-=k#?Qy1=UR`M>%maTHA3*sg`65ZoP&}TOfb!wY z94MhNoncN|Ofl$gZ$a9eHM}r%H3Bp~muYjJSQnN9VUgwE%~{%2?yuT5=Ltv~;#wEE z>o|AgTu=`Fs~rx@rBzardG@gBmUegALib|L(#ka}8pPL=D43EEIBHfoH!nIk&=gN} zsITDvi!dJq7|sh-SN_L4@-Oc>BaawIcuwcP`TftA?Ray+yRL0o&`&C?c!qX`hjy00 zq#2hgTe`CyP+(qupEJozfCNi*7)7W6at8zB(hGpq2_EAm?v*R`WTgnXg1z8<6s_jjis zyXR>to;g0n-q6vWI5407edD|?WB_aikL6V-tnbrM?)Jv-g1u^%;)xVK>8KruQ1b@y zJ2+*@Ws1LS@cXr6Q2wgV`(5G;SKf3zLpu1h;DBQF=aZW^p86OBdw=wwAV@xWjX{*yS#|;fmEnfoBnko9o(_bTx?rTouag1`?ks zoL7M2h(kU=qWj$#@OI~0Ziy2Z*aDE6g7vSygB##{_$MTdtKi@UZC#1*P1mmYsznCe57bFeryLjMAY`1SR-+WdB^07;_NFY zJ*)=cVe>P|(mnE`Dup6d!jj=%gCU2inZch;`npHXRo1>fATJ{Cc_Cwq3K_D?U!<2! z9M9LyH3oUjv{*2lL^2E5!?wBt7gn9u=+MxWoF?eHz$Y=-?vWVHdY|6h`4~Sw%_S!p zbDbtQ&2dDh&#lu2QfeWpp_TkFZVWG5^EU&IAe@$^H#9r?(ybgsjuJoqTQ8_QWN={9 z8S2i%ymD)k#OD`o`cZ2kmCF-cGhhd-WhcVofaZ?yPS`g!Em#@(_~OTHkEXEmBECy% zs$t7bYOw1tA>XmGc7RN78H1f)EQ2@J^HY>8O2(MIEe8MJ&`a1hDRQTS7PBL zY^7R`FWY8p`;nqjFl~yBSS8@Qli>*4pvTgPfzpI)gS#lV*&H#`lW#1yK}xM4uu5Tg zaAmn~MI!xv*v*$<+;T{}t2oSH4%dvlR^N8KA3r%0Udvr08iHV_H&~&Pm3rKOYqQr9 zCP{`ObCB((ZX0X}rYv`JE&=vDrs`Tbi+bG!8f1!x9?L`Ee%M!C^~7K7%kR1=(W>Zs zq4RjmNy?|p>yfWgGf3Pp!NWytG%z+0D6VtMsaZY@@_XcyRX2mCo5r1tw(-bu`q3Xb-}5r?@OHtUzXTQe~K6%5lK}_9DxkgZeI? zo|^Y(!T_4J^5Uyhwypp>?456Y+lDh|mfMy>T)-gH@YP0T>&cgauyRcTQa>Y8bc?`U z5Imsjx>p2TAIgt)Oa{f!sH^ag6u3|$MXswqPX~06)O(+aM{PZ;n*SkFcG|{Q7X5{+ z67q~~Ck2$~(GuQMEpn4OBge56xA}th!5pO3iUYpx$0DRoAq+{CsOo(5-JsC)%S7-8b{_>UY3HTcf-;5Ga9WzUQle_11a7u_HoxFlwI9rH>f8g`eb|FD zG%zVSUy07tOpkt^Hzf>tr_31NePyNa&0m?;@GZ(_@Y{dQq34=vErkTxPw!ju&N1;2K;RE_P43>L)L|C-`=SWUBm97 zklFe&;SHGX;mm_T#)f`;39{vr2w;E8RV8q+*h3C*i3sPlEx!zcO`z?EAcsPKKNTJ# zXo+J-M1sC%&Y+>FFg?t{HsClPpT*H);bsQJsV_m4E(YDj*gVw;x55G_vs^5nT$o?~ zxbrIRSP0?8;MK8H^vLunFCi_ptHZg|Y+F^dGKsUSllA_V&IUHD`skXFq|u&6HCC^B z1N&WT>I{JaQF%u+_;HgppB39-p=cknWJ@lN<1Z~DMUF)p2s=#p5i#X&d z-|fq)gy4g`9`omjYFs53cjLFhRl`gVq{~i!C-=Hf#3)KH%~;}I&J?!y0Dq(J%ySr~ z21KSmj(e(L+WZX`KjRldZ{g-=bm2Wz5c1D%vGdGkIz~|AD`f6IwE;`gnVzSPoHT1dHn7#Du<=)F~iR*r!>8L$-u!2XIz6rTPLxi`&lSxGSIL@^l*eNWpOn)noP z4t`(tTxbT%&6m&5@Li`>;r2cFjLrP|rtdVbeYUZl5B-=kvYM2^9qk}v@33vY1QP_~ z02Eg!1}h?oYA~Aj0_-dxo;G)+lox!rM|D*>Jr5;VU~$ zSF}odzp+g%p*~})?@@Fa+S!`sQ&C463+fR2;SaS6ZtAIOhhIJ54ZBlaI9E$-RrfA* zo~q*Hta2jSKYet}x&4G9)L%E9f3SZVGEfnF-9Fq)s%#Q(?HDeZIeJ|<2k|;6F~2x8 zQA&e=Q8LWU)M{#Cn^hVzga&3M^IaLda`)rSqOmsJ*w_S5km1!X0wEXEGe(g$@r~ym zn&7W9c*|!YEoEZ5&ia1;{$}rS44zGg+Xs;r>V|812`I(|<@m10fm}@TfEC`E12ep3 zEd4&$7`AuYJo~(`QcuWRjz|S^VqYfJEB=03?;!GZ_<0w6TIil{3$bq{QeAJ|_$?PC zXiIBviRN{!=1>c9oT8VjAf#v>;EEm?{D^oI@aoA98RyJ!b8WmVDH+Kv5Qji=PtUc^ zp1hDj^$#oX4+{|5tl%kE0kCQWm;^wQ&s^*u5vW2r2aAI|4^xq!MQBRhZX|so* z4~6bCH4Y_)WUWmH1buy3N`*PtG`q6Dp7BgF<;0c#7y*F_vrgdYJ`(81(YY6J3q-O% z=M-S+ukn;!2zQwVVMjwkA;zLb3AS}US9&40(f;1BOqC3^=0z;x^24{si;6Y(k6G~2 zUB}Hj0}osIASU!}lRqQ(G>##FVP;XpU`6};s&%w=z}IyCZs_?W3Zh--J%T2-4eZ9y!I0q5o5OUM_+5-$!V zh9pH@f4mSPJs)z(Z+anBm=gfW^~Cx-rcW5o=R82$TD48E>kf<83DjMpH^=exT_LJ9 zMKO?PScQC)tl71wY=T6|Z}Imq)vfr8y_>fk6&!OA_@F>tB3u=n%tK6ML1Fs3FSAr~ znODNO#Nd(`zyado&}57Ct)}yT1?^R>+@*Oe`#mOOMRF8v`&ok}4wVaV5H05prV0=># zC9BLEbJ3pprlrqRzG_WcP4+XE@Dc&qE3s*Mt-;eK&XQu2CO`fERKi4&AASv!5pAwk~$&uEe-Bv_jK&s zXjG^3z5_TU2$d|EbpdaRPa?zy_NQWcWB#O$+{ z=^j^-8sDe>ZwW!Hw0kC>I%b0*4|}hCeI0tA0k=~&Gy5$ImO}ze7EZ2mQrUm{!p+1U zNKaq7p>{zDUmQgZR!(jaHkjpYT1o6YQ+FM=e=H|7BYT0%a&LQvRh86Kv zc{ARI3nOWt^VoW?w=@>Eig4@!H6v6Nez|(nmCE~g$Qrf;!*zEhOndthqLIT>FfhCP z>CD%VMvuZ$)j<>Eq)s>SX_X*1WRt^aCZtM+fV1r3RFA z%VBVj;UkCvZQ!XYgKYVPMof6|B0hF#(Q|Zg2}*j6K@$=9WE+a#W_F5zTjR&~5Bq*G z1axaW!G2C_FE#HjaLJId)5nrWd-U$;2tIe2$)GfQe^}u|nPfI)T9*@>PN@=(N!K z`mc7a^>hpby@=oa){EHi$YyK+hAlX}SCzXFiX1BP1ZZ65jJ(1n0CmT|rqBzHd>2e$ zS#@3jb{-gu8uOx1=p(7FnIT+$COcm{=zgqN*?b)f7E|*M8OhI{(*0@A?TFdNi0d5m zG7gXRq44z6wI!l0v$cKGl^|B%G^%P&ok zQSqr%H$v6>_|iAIghLM{pB5X3`$BNN$n9dEV8TZT2R=%#OiOcOLy$uiO*jXBQIq}y zoFd`bUQo_**Qw|-xTP-qO^(G`f$Yc{*;474R$K3dF14IcPmWK+kR63o^EGi3lQ&+m zHzLgK@))x+KxFcGHrA_uvWG#5(u&jZgUetp zPVcvAD7=5AgT$BL{=#L#2MlGQilWZG z51OhfOOLnQR~fNlm`E%1U><^+ey?XX4b5pePAX8YOEezfFxVk*>SQa%vv^a>Fyeqa z{FRnFLIGo{MslM)IFf+M;XjvAr{Xz;_LT|Fu`4kSuB70NK|559Q2$uYF10Ucvw=nZ z`K3nkyUri8UKzyu>Ajv}%f#HyW(!b25sO{anmv^m&IJpL>R1RXTJ=-oL4mqo^KVeK zto=Svlei%ew64EqK;Bz|y`ze`k0eb%$@3GDPxPpWKxk-0LXbX%pn(IU?VM*=tM*c# zyYX*H(Zyo>a#`5(FOC}RWr6SDrd0A996_9Vin+z6p28cS0dkmLFiQm`_s(WaHC><{ z6j^%Mq`Qo{kM&M>em3RET6IFhz>{*#{w=hKOOEf&xafIZhl8mfE())_XK$?F+~u2T z_k9kL?^0{RDntQo#o)V|aV=ahOJ^=u5CNg8ix^$AVgCZtZuD9hp%ZYq zN^F@5#k>`E`KH-5*?Uo!Jncp5qL>@9mZdPe?hgD)J{GwxgveMEmy zr^?KEXcXahq$sZ!noW}s#3qOLc&^tr2<#0^CuBI;)TTuI00X4VU2NwquI41t#&vZM zh{ECqEw8~No3zM-%T2L7vQ^ce`}Y;U$4{ruy(nwROgtkQx2!q@gC%eag&Q!Wuw8}; zb^?>-+T7KNI};v*5-jvF=*9PPQuf+fQrL1e`U=2L-e4x^t}4`W(*==CiJl?F=29;( zn2ZBw1bcC;XhsS|g@5=De>v+5qONh$)V~tv`yzJvoVDZ@AZr`}V=O?agqc~TusM$} zS5q?mneJF-pvNwmO^=EvC_$aghAcA zwS!V~g~~KtX&fUr>uV(7@G8dODSPV+wjigrBG_63&1IoJLp+{V4ULXq>1?d*)e|#T9?A27A3H&^^P*}0HKCFE};FFcf)i=2G37@Fk+Oh+TYfyqHG>Zbq15(BkIQgqQeb00>b%jDcqn6#|DZNm1}h3P5qc zd+}z^7?MNzbwpc6C`T0xLit_u{Drbg^$cQnh)gjO8(`0xp+G1%HFheJs$d*=mD3v* zp{$j$&^;(MG!SjeIv)I(ZL=9KUd~b+n#|B734(XeS#Uvk>lbBgT$X)yaFpL;XU+4~2{l)EujCeU{z_0;(3;_CW&CDWXwv3~w(k0E70lf z+c>n;eC2UD*yK1bH@BvcSmE^Jb!rn}M7@QCK+)|ACJlM4tOta?U`wu>X>z~R5R*~N z`<$W$|CZapo(#J)O{P*t9!;f4*<}j9`?i3Z3kqP;D*9UBQ8u%533m!XX6w}oE^LGv zfB(G9K5d`5uEw|dP39@kB5$Y{v~mMYjlyUAWK=A)4}ieU&YQ(cf^bDpvKCh`S+*#{@`PbewPf%0rwSv8313S5Ct^@qvmHbuA;Rasw*M<}uol<%<#Rb?IAS{h!Pok~YbTrIfV+O$qsop3=z0%m zifT#5q#y^X*%GtuL_17FEDhQ`kogF%OK<`v>(j5mQrxVFuOb{3HH16MZujFVC(c5v zLwZExv%LOm+a!~~p78KAH5S-0xCZ7eCLEnBmA4HpG(Gj`DjPj)+6iyktz{7-GWIOB zCAz*ESdY;F8n9O(+OG{K%mI5q3N1|(j$b5w#UQqxBsedP^)1_Abj`!7g}G%d85TGX z!(j*^tO%ue#1-XQpN*)xA8F8FCDNA=E+8xQO z3WnoKj@f&KU~n%^DGSqAQT>wy=P@^l_P01t#QP`hhC4J2dNbMDC~Q$3Kk^H3O`gtD zEg9^~l(ee0W(>*x5+M|6K3E?o<~=d_?x+Y>KYi{_@3x=bP9bRTDxzUJ^cP3)GSa~3 zp3(;U?<4ke(OttUg7nvyD+lb2;IHq~a#m@ZR9!7q^QFaH3q_lTv$5b>ejq$yH%VZ4 z-?1pQ-W&3cs+O0lm6M|vdjs5wOdZG~Lso%ebD2-!u+y-9c1H;+YPc@Vw zi!Htfn9YZ(l?c`m)d#(Rxxzvlx@e8I;a)j!K!&pJGu0ZU=Po{jzk}uFBIpfW3$L(; za^cgQ^gzN@js0A!Iz57cx#X%y^is$s{B)1o^7QRjkg3ad_n86g^ujcg8GpMpG5GD7KMRc|G?(>SOzjPc1M0;K{nA|re)*z0LVy|!~}rZvA)g} zESwGDE_%Qy+cR|MzM38Vi3Okltcc5*63)iN!RiNG?8jg(KdO|x&TXfMY=~<;uzHs& zg#NkEWC#nA*T8$m!w_>xD zF_zPhh2O=%Z$LuiWi=>m3? zy%8}Ht9tm=*(VolG0TdQlotFH26MY5sA+O$Pn7^Rh*1c3AD(?gqQQ-y;)Se|T2jL8 zrNp-A!<5x_M84%NVq<`r=OWnm#?9s>D`3Y%6LF?V-*E0k4n6MRGxN1O)p9+{2Bx2$ z;ovu(w_TaFeIo_KG?CaY3|U_nogo7=Njg-oMPlaIgp1sT(X2Own&__*Jk`OgMTqNX zx%0}2@TA#3kW=tJ@N^~NLH6O<_k5CfVTetQx-VcC(JV~<)&aqjt!=d7k9AEw3sfW6 z+O7h1!yy~;)a~$m=&AIvXFQc5VZ~{np%ynrQ-o2g`YlI#Cfh4YDDM-g(~@}YWskP6 z5)unk)aUTHn#1pcd5MCyde@-Z+N>PFR_UWl7L=b?kBp(pzV{EfmdZU-)AOUEx&R_4 zqI6|xVlR7H=*;QK@@hXUv!MEAB^!1&n3QrMQFoWh3T7uz{OBJb$0C{Tj%zv!Mq3Zf zG%Fo9O)mbbB_Qv+pMe`mqcG9|e5@pFivMi#lnbUKv^^1{D`k_~BBxK{vE}r=ZdJ$l z812v`rRO$;yRsuhle-RK-x)LRnA{kemR*YEP_z~R0Mbk4PfbP0;j^Dsq2(Znz>Ep1 z97sqMf*reE{N>o78>yLk9MM0uJ2Z_$3k{G%T=gS72J>*zGIgZcYhmnQ7_OzGPOR_Z zd-V(M@Gv*vi(CwykM>_$fPx**_+ZEjIt&T26p5Y*QOfXJq+J@W6Bd}hi##CEdC z!%})73ezy7YZbuI;dWIJ_R!4>rHaA{_knzp0%H=#q*n~zy~u9qgA-|ywfEeAlh}dS za(Ns+W%~=Jne80=L4BNWvIeam$3@u?;^Llbrugfmr=XWEy&OkaD|L*?KaT@(UbX64 zm8!|~cP>Ldt!nC%P#R&_^(8s@0P&EewSe6PDwt z5udYVxwdLe_+X(Hr?|LHZJ@K`dMuB5vrln1=4~|O;pN75u=IX^I1}zYA70@FfRmYA zh1Z*!G+iaUry%8BSQA!=YMELHj1qBciQp2UFLgx!Q$M23F~ykNv$e=_srVF%=eNG6t93Ml2xMqOEDr<#HQamhRDE$ zf|E`ehqTutw{ndkY`x*Tnp#iSgagc9NtZQ;$#0z^2)M$C1e=GuE zGtC~mi&EYN-%rF?afiE1Rvg?rWX|oKT%z$hvRd{g09f6-I2XT7^2RCqK5L(p(dYp} zA^vD5)M})QztB5ywE`-J`yk=11a@mJqD{g6TcJmpqJ3;*sf&dytZ~mT+mY9kM#K_+ z`GM+<(I_4*g8EhklezcoCBRBKOR(5enc7;wzLH|CJvOYR<`^q!96aXZ)v9YXSZ;cf z7_j3;x+7m1NA#OR0xS}z&_iBlm369AapEp2>7i2ngH4?d#4)D+(fO=W9pi9MeGF(v zP%_hJ2=J1S`T!$WKu7H?MD1E*bcpkL9cbqF&R`XDK?v42UAw;rd2SR84Idd-o2mjq zx#H)#U2LkEB+hcs(W*S>^+J(lXDYP`{ z>Jdaj$9QQ$Vq@Od*p#eYoip#*(S5*Q`^qfca9taFEJ?jjWj@V9L1i|WmU~6{ zYyU_1V0k5ZP?7;TCO?1R45f=Z@*o!XRwmpO-&9JaEtvBg%7#{y2WD46hx< z29>c-3tdsxwi&vH4OJk-JhJPR=`Xo1uV#yHGUAN$m}>bfP_{UgHi0H_Tq__BAO?*))1T7SU2FN9$KD zD`}M@UeGo`m)Q0kx+=(wHiP8PjDa(hv#%`q!H`BKt_d;5Y?x=NwFxY23a`&MZ}rW} zvqbufpf@A(?CGm7&Dsy*Li+pj80%`+hGvr{(KZq))?T-+7u*>T!xOiuA}?@#>HH*~Uex0T<3Gyhg)U$@1MiU&?fBn zy%rPs()d6?$|aLqr~Oh6y|5!kUb1`j;ND9^Wcf?Qy9gw$UF6EV`f@8iqjXLkH8<8( zu;A6}Fd*nm_UtQ=vhAVrO-nYYfxO1v6IbtS@)2SGPS7YJxC{uziQiTNV5$N?2=lHr z(YWBG<*J}~XVYdH+U@&J2ww7WyFu8Y?MSb{mt(L-LcGcCzGYQ*Q$&jD%yU~m)+>M%TqqvpIuvo3_vb#Pn7L_{)qL0 z7zF~QeojdU)DFbe!oXI{7=HjIsR^PQ;uXinZ#^=GN?Eh&!4;wP48Z6dq*p++Q$u&s zPeV6<$<%m)Y}PL#{_Dl}>8KBrs7M9$&O2Y#yY9+41xgv=Ju+&jt*j*VyVVq%SCJVg z;_zHtSpn)a;k9H!#z>bw|A;!uzj{gc;`uxx#t4rNnztoqAzSZeF4jKHLdIESj<>Kc zlAYpOmpfxy-K^d%dQKrGVuk!33%)XZwu1lt0rl}MqY`hR_&`Kqjgr*-*U$8z;HWsL4+O9HAQgd? zj&5T^nRB)}-&4lPV%uBH zmTLQwk0^&r3e3iK(unHuoTOL^gfHNq!TtRoJ<2C##JQ64t9|aY(Z1 zJx`Enz2cNG+gYHhMUFQ_;jNH#gl=~-^JAxE7xUvm=Wkt^49Hy(YuEIY7Q5N~^0Mu& z8io^Kt3D5CXw+hNtjZ7cD}EnTjO+)&f*jmoh|;_PAxN^uXdv*qKZJtcQZWB&uUD1} zC@@hjo(!<%4Ltw(aiP^zkc5ECtDPM7R=t^wM)})+G3}1+!dKKr+R}McJ-JPms1(`P zhvz4<8i{5Gw83(Pp$8u#FCKlgwYYceR6aQQLng-@JuczwK1#WxD$vY$MnzdgDyw= zTtMBCK{0^AMu}0_!*x}mTe&9tnK(VznWzsa5y)!?MF9Rg`Cl4-d}diT_dvkIsCNRu z?hBZLTx(4#^B<`&x|S*4?>Mz82oQttuqp3VK%K-drlC7Dh~KPkX4-1vt%xbeQ|Q#! zp(T##v@OX_+D<_>*aT9qyJwuuT#E>8E7+I*u=_D4n$&^1L6Z6tbWw6`E6&C+jwoco zqzQLQHEKZhwQ>^0pTTA!JrivPQa*m(QF4J7vC5x2e7N>*{<+6; zVta-WX=f9azb+ec!heSvFny`YXrFy9#UtfTvurM@V3aVSV0=+2b=JH>N>cTfvP;z-eI?i$bu0`)Th{d zQ5S!F1QI231Z-qmSf_kZuiAJSf|yl78$A`N0=T!QAl)Q9UZ1Rta++<;#TLpHs#`Sg zLO9{4M0;$Ri%vtFM&v&sNNiJTU)m4xA=rv~unZyn8Gw^INGwCqR^xCzXD!*=hN(b0 zbIJj13V9TSN-IFs!*cMhZO1)s*-?j*l|ri*oL+P@L$6#VhvrEibN^E7gdGa~6ZEP* zv-KnXLEg5Gn-V*{$XTj!`xN7>y0~RUB8wj4Ixovc7AU>sQnWF`XFMki#}0prUGyx7 z@&D{x#P5+sr46ID-!56pq; zObfZ|pFwfK*AF`LP2r`;(Xy3g#uC&7+1Fm@$*;4bqlvkuFW#Ch>y4R_85|>`i7OQ#sl}xjMTfbHssbAhL#Smp_OqH zWle7Bi6ffYfeRCi)`V?Y4wVe3CRMYVc2%9J4ghuXFq*H?-SZ z=!T!;2~BnJCt3xGQbxE8R7#FI*=ivz&b277V}vwo%q>bpr4gV7qXt#Rh}6C};QHQz z@Z53*rJ0tVqZ%&iQuv58lyXb0FAGUlHt5T3VC@>`bC(>ur{k{_SM1umQlz0-T@bGxd@6l4NwaOT7PXV6?K9M$AAF> zf=Zu{;-fO85Q6wWB@9#Z35pb;xOAv#aq}bTBM(t^!zAZ{RIv63(WHm&Bz+CB^zqY| z65RVtzcaR5^kxhq>mr%n9SeikytY#h8$Lo9ai8rNoJB^EdcPC&#lsT_^44#b_|EmJ9pA$TN$#G zop+}@(o47Q&8ndka+qjhlc*kgJfcp!d7MfzIP0 zT(pi<0$to3Kg5M;=9_)+tsZW(zf8jaT-gV1pt$=#U*e;Z?n_he-y&@d3*+I%qAz(~ zKA+H;RgpF#Riv+ROuIict3E|?4*#}Aq#Y~in4Z>3)H50wC6Uo%8K}C?&CWJ)?wO;x zgFxL4%yE*fUXiJtcB>j$z7TL+=<%mxKNZ^_DD*@wKE1OgW!4xNr>Gk^gdx|ZXZcyG zPIG#%1zV-M?uta_)_Ambc36ZD!}T()GNie8wInmB#kmi=jTzrnC@fR4_#l#kbtOr; z>t&{P4azUv$4woBX;!bDMXH{RA(|K$zqBf^YXbW3q;O=`v?k&+UL{GPQW&BRGpXbI zkmOrRWY1Z8G7>2HO`7Bpdjkq^8#$fQ$6H#@jNC5CA913HPtSopOF5WlWLR`kNN$^K zG&OCNcS@mf7s||auS2|xbQHt4YGVBP^G`GY1(52g1w)AUY@d=;5^SB4Yn_Vp zx+YSs%9E@%rFL3``>E({G1SYf>1iYPEsr!Z<&)HnhZ#4Gbe_vhbsxu*@-H@z?3O0~ zS_`^fWxP)LQ%befm{KOsD%r~&cWjqe7%XgO`tvd%<}E+Ba~DCBc(aFeN?Peb%oDd8 z=oV+{R%iRA65A%PdwN?$3;7cen^LXVQ!-AbWVok|i;h_2J-|_GyNIMF`};T&(z<$S zwlHp(%5M?n`fl>#ZUT2JW!}~sqjzalR+A8bAf);?H~+v2KgEZG_d`KvOZ*sR_h=rm zpyr6)6-`eXA(SQ>S@SZjN2I%%LaI(9+()8Dk1&597P4h3NAHYpy{C|{SVYp8LlSo# zMQIU~P?^02ZrnmwK+5O$2WJS?! zrAxM#M%^G#-CrX;uQBnXcaE6WXIx5@a&S&>Z*dwM&9y?M>*_qXGd0)a%-VY)YeXtl zyrJmj8BzqlMi$pd5CL3)ae7yjJ?kw$5IZL*^RuKd=!0!9z~{9uqUL~a`M2M{;d4KU z;Y?NB6J!tgqhf0^^}atzzl%bWI+;i?>jh zY=eroSLowLKAMm`Qmn+3oVz@%@;0NP<;)Lk)hp;v5Z zx#ELYW)YSLnA$Ri&a|{i=7VWHTz^M)Ad3vNVGPgEI?m#+RXqpf-3x(Mj5Aaj7c)wK zl23qnUe@&7VrXMEM=y#Z55#t+k;a7_D3Zzf$~u&^-892IrLs+Af!3U)eh|JLUqggg=dNtwCxUUy`K~r6r{P?( z)T<=!O>N9!!=0Uiih8LJP;)K$v++-nZTmyfOGiknA2OE*=Ucg_8`yRKu5y1#9IxJq z=lCb)h*BeI7UpNeM= zhDvTBzSdg6lh(ojX_A1(#M?>ZkN=j$GKQ~d-rQBACw^7(oY|e@2(A*&tY(Y)3~Wr8 z5N~UnAAhN5s*F=)t}W1Xz|uba%|Y=V>zDj^WaorUM~>O+zD+f&{OZ%n4jcwjea%gX zm;UBVlQa!AQ%k{tq8Ie;s_ETR@B}6g{Ur+hERsCX+u6C*xg;5gXo`yC}63uONPp=c;1kBWVo{@{hVnY5eQ@Z zQ0I;eaEjW}?l!5NEkrZ@KKBa~pJ#~&g)DS5x#zk?HicRBP1e2mcFrPWi~&>abXT?J zd%p-f#G`4hfSw}WZ*M-`fS&AhKtS>bfSX2Ga>8F|Ee_A{iQ%SgRo~1#eS7!p3_XJH z9o6%$p63J}k%hNP;Y+d^L=tbN#fIlws4Ei`-cPI&u>96$A1+>{;iIxRFF)YgyDI4_^bK zs5e-v&v3bOd1Ue-j+D_i7faf@DiLaYsjfF|PU5+*YV3vMIf%@ql<`&nHVFnqF-q}` z{@+Oj*t|I$Iz*#QC>L}O$*IC-qPR3o+i@Z~&%wz?HhiA6)gSoj+VzIbHzf{Mj5#0k zTmf#9-b5KGIiH~2p;c0+zvQh^ltH-O$`e2jCrUSGtT88o3E0Pk_=XL%{Jul?Wa7Rf z!w^>a_mc@9>f-qvZDh$}nvT5{V^~~HoRm>|t?Q58v8QV%*{~EuO1>Wh#l^Pbwi2~# zdr++}9_netSahBZywK&$rLoSrZ-AP;$g&aVCdnI~{e2an{hkj}8q<^`D8c7z+MglDzbKOmCs}6_@!p>L1Gvs)!2URQ>rqC)mtwA zje>zV$q>5(nZCf#VwvyC;)I|d8Fd(>^s_&bGSstjNmi;z0v8}~YVGyNVXig~Z2|fV zy*3r(hFg371$hXfD;EVV0Ee zvw__3x?5^YUWD1^MNhII(!smKia0TRyHyt@h;EOf6cSu3?Iw7po@$1!9$IZYus(LN zl5~DXSl7Ar_pe6i`M5nM*z%NFYNS328}>;V9`vM9sdG=~zQu0JepF;k>F!#%$fpqs zTkc!iz~3BIwYr19jhUEY=_RAY#k3s~#0S`|sAt344+vV9te#V1VZ#UL)x%?UDmLgQ@7hb0@YOqMG)~m|5NctaGntC_xamv?qBp7fT9DlY9OWO z%-;*)?_vW8_s{+#t@$;eKg|O>12!;ae%mQn^5Oq~$?F99e}&3ELk#W3t`6Cj%}K`Z z0*^Hl&la6rv##3mfWSsc>5HwojsL=PzeLW^jSiX>W!*y}w&$23vOxlLs3+y8hhYT= z45V-@IePt-e|$%l8{l8{2Ca@Yw?GpC-6=!DJv%qxQXdo>up<2S)Tpo7GDjG|ZF}X> zkoq6Lux>5N7nKjxMc43w{6{tj(9SmJNUsWfmMBnwZ3BfQo{i*RjVlYZjlwQ;f^7zW zZ)r={sVU0@{Ykm!tk(5)c_$-Sg}cRY;D)P+k$J5WEou|h6<*%bax8fFnspnwA1o;{ z!8ze~efaOvq^=lG;6;ta6VBj37tk1A?Aq?rz`ap7TvcSd(fSz=+}N5ljIgQ_dvz~s z{k~S-F7LX1qs8viboSvE#O8ZY+B~#04bdI*eCxz)fsBkj8!dJ_ZxLXi97dgWX8CsJ zgkbCbYO{@>Q8b(e^;86StA(l&+Ac#%cI|0iMZigRH-G;6o7P!h5A-)Oay|r7OI8hC z(uK>U^{DwFf3OJ6Lkb&R=y%Fb_<-(`%%CxE#9XREv?$ZO>#`}Jjq@69d|*%zNwrIR zqote6$9Dt@7_ElqPjdU<54#`gUX4obi(mjZ1DE*ru75wy-zlHPOcyowgHVnvnF`&Z zW;OY>hjEqvBlN+;Ly{~KzcL(ucuN+6E~*u0FC^hSdS(J)kL?)rFO79{}v^pFAA<`#OIk_v)o0?<13Y%pf0$im2|R zOG^6q)Oszo^iQt)6ItCf_Xr5Hx=TKI(C+de{+!v*gXK60f>iWoKb^5Z5tN;ALFnvc$*Cf4sdaIhV!Rl8G zoggc$-XYg&BWY6O`C$DnoS=%jgVub&Wiy{Nd){V^zfr;Esyt#o-e5bs62gnE8 z07WDsh&^UiQ;;)g?zH+n&>tE^`|X>?Lw>)r|84)}w6ys*d$idEQ2~S(NSz0GT>(K6 zpDj7GYbx`{Cz~Gm zDn(us6<*y}M4J>9cepD_{%1Tb1%xA9k8Z>?`wsbo?;DhmkQ_?UrHiG#&YAiFgw8wp z9wy>DLe_cUG^Alz|83|0N|YzE@vL_U8sXJn6KK>dE5Zd)5C1P-fi5Da@nfsD(IpZ5 zkd$&{hRwtD${g{Z>!4ls2A*=zMt0q18U%An!55YTF6f=|W?N8Lz!6~{M>*DscWpZ$ z$i)o_qM0Mp8fdHRgqQDSj_7=8XX2^=wt|q_!}!ms!K-D6lqi+&F#|8K0=Nu;#6jLE&o^}4V>OKiqP9RH`zQbFOFmd? zLTeBc#PRiE0^dGb(>p%<0avr`-AJNnnfbDuM%vEp!e+OWBRw-}ldD@^1XXPNKY76i z+%W@=%#74W1i~F;Nffe#FOFNyUy6A>^h*0r4kE7{G`K~2IU}9$i=|eE1T8Z(u#5bA zqxb%_g118a&A;(+hROiu)>$Y9nvGYRE`wkgn(yI%VaM@PYx>g-a0I2?euF?LVDuHv z55!{zQjI`GSP}cDLst!j%``P!K&3w;4Q${?oydAfN6P=Q^HN+n_^#ED=XO?Wx@`xL zsicql)^+qxW&5vNDMvGYYYr5cYlHY`1eMg;xs)y{v9O*$RaXZ})odw!gA|*LA!^5r z<-2f}QI-gTMkJ`V_6+_Ix|Rt6eDnxh$mb8#{rvq;9A#KZOU?VTQ-SFY`e7Ww)p@i% zTSgfRuGEyDSL##nlKb&|SN>C@|N0xl^zAb?@4^@Kq?Y<0?@vs`jQqzzSWmOZ# zCwp0?P9^i4$Y_*^&oF>#K&$cbh2Hd}^YVsodbrA(z3&5$NnGgrV7x_5D}D!xqL({8 zu$FfNK#5R57x`la(f^X|{x?H9OW3C(?@Nszf?mEgRo=3$An%v!8-E7qT)K@GvE7k5 zg+|S@bJA=IN1HS??}o*fTHP)cKe#R(LeRy{_->SABZC8^Khf_BB0e<5t9zmLp&_CW z^a$nxvIj2ga7MV%7hJ_>X=dsxlp7J61l65Hw0ZWN0 zE@awhUuke3?(d;>6YbdvbCSFwf+%7?!jP~-fnD&IVMB!^ztNs=qXiviN+BTll9&u+ zMb8Mf-tI;3!Q#wkE;)9s?#gf%(4{6a!gL$Gg2?|ttX=rM$x$CXfiDi^ySzVlA76by z6AGpE=ZXkz^tj0Br{zfP&wMz#-uSF$WzQBn)iL< z6U-<_PCW4ZDUiy*JDNKGXPvYDtl)mIHeW%V`)j}706x*wdnPiw9N6k>iuU(ZI7-iQbwmv2+})=i`*`@qIr{>#RE>Y~LC0!NX1#TwX& zx@r&bz8`*4<{y|?5s$$quSW2#J3e`k4RUqE6Z_x#L^<%a{Q=1O0GRbDkZ;|gnj!wp zg0bU??7I9$f>3)@N5btd>-_mg#^&G=(QMirz;db|DBK2d%d%AIT^fSYL;A%}i@?$1SnDip4uU#Q4`la;j#>3``HjO~`m zG1WwH37;((h>ugSzQVk2#h61*`9NUz?u}Sq6Y__!xdrFe$ostvXs=k>4J|i1g!}-%r*AKkUB9WLAeZJB=<%mY z`j>0qU$5Zueb9kz-GC>+5i=fY@lryR`cCSsJ9}sRE*n;x{QlbictQDFV1sd7vpKo$ zYymM1mD||a`oCEjV7^1d2kYE>60*K`{}-Brd;2VU=em)31#-TQY$Q}qfRJw6WP$Q{A*6yAxquL*608DL5+DbKkL&DhuA#f)1Ej;@_y7C<#sU)jiP zz4O2JY$4Ce_|3!J73=p`D9E#^{EtZT{|Hybm$&>6DW?D9?H~<7zYA`1SGI}VyaC@l zN44$QRXG%#hPZZR`=&qnKR@-8pcY`$DmR^gHBA>G09V=+#FEQ>JexB;bA zZu+mQqur1)UXO15jdo>(tbj8G73(VRA*5D+6|4Lt=OOEh#RaDJh9IDi5^o9gLgUDt%2fDY;l^qtb{FaOhbeMC+? z{eWzJVY|CKP-lHdNt%EU{Pwp%bCGYMoQfZS}op?Lq&FY?*ozz!S_ zOkWq<|MHX|l-rnei~5Ia;eyI4KEKIcH{AalWh!|99rcxR|KVCHz#*x5oX`J<%;o~< zSUF-Tx}g^P#ZLdjjQ{^L`zq%d{y)a-h5_3>K&hI*Y6-PiQAeHMJJ9~y>T8Digy{0L z^h)k|P+DKjB0Zdv;0Jn&xX$kDo|ND2Vo~?@Y!Qrxwi4vTML4cE{}RXl_@R^5Yp$q| zX|(!6X<`c(nqKttz9)&@&S4nXjv&M z%kBLP3})GR(-d?D;9;GK-^LW=Icnm+WzTsT7c@MOx;Wh+01A-f3VSYo_u0!U1^$q) z8V|Ez0S%#C?LJ)E#sr$m8x@v^@#*+M<9uA1qd%rWBM5^hOhoMCG{9^!&X-#d`!&6M zw7`_lG6EBTuGIi}x0-kC`CF}=b&3YGA5*jg(>-cY#37O|)V{C6N`E@)v(n0pQjynN zry92Jpci&JR6RIoUIvOGbv-~MAf2SvB$G}s14gm}8o*LmXWf;bzH2glwsDs{`#$IC z=)PjI%@UY7!@tF0K5)FC&#OdgE0Rp0|fVf)vtiM&@ezdFqh93w#F$MWz<%}M_!nn6PWa6&~;*174dZ zu9cZ)&?N0#mPPW3#|?N$=jbAhCUD<+9p2lX1RJDWFa>zFS3G z_GVwHC)~Lp&qjEQ8Yn$>*5V>GxBx!Z&#$X>S^RR`pcpi38lSc!^M{m&#Lgss25p@9 zExbX$$qToy)#P;}l_abt((qEdqECaA1r#svGsyvR^lN$l@`GU~9Tm<| zQepS*mP{Lb|H{Z-=o(&f@ zs8g;3Gmf?M?H51;89fgmjePb4P#3SmXhc`RqHFsb)_m^Ny&|d2X*gx&>4lJqkD&d* zmO_k7L4#btH@SJ>pU!HX8VB!p!(eC~HZD>?|efoqb3YW2d0jX=}a zmE7UWH=$m3hi3`bv@a*g_Mwl{|9|bBcTm%5*Y}l0*@);a2o_LO1lfctOO+zJAXSP9 zN(n`!geFCSP(@L9K}3Ymn+gaCO&~z%pw!SrLzN;3p$HK$^uT*v-1j^0z4NT|Jb%7_ zJUipe&aRV5e&t-}e9t-GPl@oCk~pK=RV%3=`jmT1;87S(BSFS`9H{Z>n2s{rV7ZF41P+uPK33hW4@;DUP~k8Ix7 z-0dK9i)o!}X59i&Ti?`JAJlja0QQz1qkq2H_nu?-#&pFn;7qNJK^u$(_ys0Xp=6}oeEzsn-ITkdM3o9b~nfOeVuTbwgGy~ z;5~)!0NaWHQdu3BYe7izx&wA}rDi)|;mcVyiVt98o#TrGR$93`yWjRIv!)H;D~`Nr zQ*6=ZR?=QgKL*Gn^ib%8m|Xg-#EYI9wDl=EHIfx)A+Z z*;ZN*`z_wT*A-a!9>zV6=S#R&R?pk&#GOMbo$>>(pP4c+060!f5HYR1Fo5boEoETs zQYy#_l|?;c+2v=!DlnMAbrfdi{%~?5$2T|xX}K}b+ffk#?P)d4e6Vz-Uek{(m9bwV zL?NOSpBgZMvNBZ$L@0}o{wUT90AlmjK<*0o{z_1M=QPckjP|mVr?lvYLSmC`twVm* za>A4x&(GzvSVno4EDuq;As>B2Cu^#tw94fTFdh?g%ySQz0?69mAn)@qH1TZJF-0t5 zflak!z)io=U=F?tzya%;c&fIwI(B<&^7drlHVCi9%rzE>y~gR~12vn}nnT+Qg4@z} z=0mc*05H&fXr>Y`0o+u^fX&6sOL~Vl6MBB(pCT%`IE6C@K7vJ{$)0{NrPr_NX>l`~ zE}s{WQ-TK!-NOXlP2vLZH!JKbuzBlP5q@O@3qslz`8hbsA@&Mnx(EafyMV1eRz8E;eaI=lNK%qeZRa!G|7;?ev$uy=0%}3EYVQ7Evj=1 ze0}#fKuj0+JxJpy1=a1|`fj3xdQynnx$R4W>s@1AmkzpAb%tbk3_;vyq#p&?C}1jq zaT*Fxqz1+ComdeqACyhJ>Nsz{kH*5Y$I9ti%z}&o5Ywxi`Bwm!s;um5A+ z+jIUj`CZYT6mS9=!30F;_F3^2xJ1ph|6E@7vxsg46smIa`xk{_^4 z>1O1p`rUmSaOOyiTEqv*ddaFeb*6s-;X1-CWgC1o%@=PWlwpK z%!eFDh5JW%Qoe-hD1z9Nu61j3Ay!@~2aPrh1Ti&xm*y3l0bI z2dDb44Qlm?=k^8ykBg3q-#;gYMC5r8NQdl`+#4%LCS+hDIso_{8(GbX4vJ}$ev)u2)n`Bs|G4$A3zN^z2XiSgE= zyt*#gvEl6vY@dWYH^@dHOY8Ea#-dj(0Q8g!JTy<%G zxt9f_7g{?&n6$|w-KG5*3H>+NVo}TvGvX__mSJrMOJU}%uVFtbm*RYbi@y;$WcTjT z=kw!1-1#E7O^S5u0SCYQN=*>R;|{?*^>FdLQ(kPrr2!M+kPl43=?|{XVAdeHB&BO7 z`riO=IM$6-l)}t8oy)%Q6dkCOHQ@`=g31UHQNpfXFq06*{^3#hW0kKq4e)^Iv8@3> zVqQQE_A#{;-TgA8ySrl+1P_l$a<^Q;i7Xw3SxxzJquI&_5?ZKwU^7?3tei>JZx*Ge z1aKXLCS6o7k8Fn?Z%AyBumn(*PRwiZ=mfH>?QT)8 z-s)vCm8gf95~Hs|hTUStPGYNE<1=B<)9DZxYkX?<*`)*1x_*Ew3bp9@}+&m^X;k8>Ett#13+uz<-HspaMU`F<^5$d0!VgVMp_p&8WA5 z1}g03-A*jo3>Ze>B?Mv0?>A4ef8CkxYx-%?RaZC%?cv1VUb{otle~c0-h^f>r|EIa z@wlsaz(#bR`abSolX=Xls(yHcH8j%hB&0BNg6Fx;AQh(R6W#&O898!(HD`hJ|KAP~ z)Q0|i#?bR2%ED%NAPjM1|Db>UJ|Dppe^_wBk;SPYh^oNvj2?tBym?+kK=e zEDa*RwWUP{tr{Fbsz(%WI+@^G?YU!CL1M4kyC{{*G8RYCJjpsNBwn+u+Uz5qpt zHlfbQa0?Jii~PW=rz*9%oabFd%IU&{v-9yR?f^FSP{YSJxwCY2%VACv;A%fV5VFYq zY!EzJlL4xo8uuS#Q{GP2-3hwtZo4$Yv4Q`&oObrQ}sb(*>FCN(!byI1mG@C47do5G%3qt z6utw8Zq(wZ@MGfu{xwI1`)LIxK(6d*)h_?x%otF*vrV4!xiR1K?Vl>IgXS&jm>5{l zHUJtK-6It33lAOI-EzvT@eAbcFdHG7-3&06nC)&v(Oz=j*D5>xA_UFMjsORHj&u06 z@ZuI7X6(Ap=ud*z_mc8FlV{sl3cHg(ncx4Fi4>!xIVQ(AN@h<*T`)_H)WIM`&L|Ye zMG~(v3*G|`n`ueStc>x3=!*gyq0gBRb|Cpy0ntnwlKlRVr())e(Me_z#qsy z5OdE^r40Qr6Q20l1@a*_z0_9pm8q zfb?~vsxi})iDD8-vp(pJ!DxXZcdNE}ebP1A#=C(o@{zv`Lc{+;#}AO1eVP%YzCmTo zh@V`_sRW?9B=2dMxe^F71DY4Zvn^?pL*yB=`)HVg?v>bpmYxle6h#@CP2y}J82|A^0U zrz(hjF}#n?ao^Vw1@0Qi7m>w!P9^;$gGHDvIsY0HDzoNnW+Z&(nT~urQafH!J(>Nt z!dUjF+)v6IW%HzOTOp3Bz|^r#`cOMk25|rsKOAcf#O_Oo6?cY7&<+knO= zZfxXeNXYl)FI(z3pRaW{ZC$G&E5zp!hSn;R_{$*w$Z8`+E)Zb>lSDzh0m4F9PbN#) zDy2Qi;G#D3f}wpn!lEzdOEmq(ZBjF29$C(o7ia1ghzQkC`+xr_h-t1XD)BkR@yYC_ z=7KDWTR5EzPU6!xS?avm+%v?K?AO2;U4zY-2?)(qtvvl3ElI zx1T7{o8)as%bH2BV-|%}LOv~_=E_DD`=BQIxxn{8e)@*&2m?JziU7`a$xFm%T%K*i zGmJb)u55tuD{>?*qq<1s+udT=gvo?^L5@%BHUCy1|A!tMOcQyw?92>$vORXV=6bqt zx3MO2>(dxtX`Fn6)n=USxHI@7S6h^<#IS_@n z1XuYmd?xAbEAahMpDkkd<^I+hwDGh!fpm?mGb+3>trW;4S#e?IPZhHdCd@I26rQA3 ziaap3cX-Zc+ir4e%_21>`0W6LtD>~nbOR!h@GhrkdmH7|YlNOEyzB#>U*+8TU&@tH z%$P^m!K*#;z5BH?u@==MtRQBI|bbRmjDf zWL2MF)0M4&BplBghVhW-;ilgh4Zgx8^stT=>Db%)cp76#jP?Kxl3)H%_Oh{9DRf|} zS0dt4L)CDhqzF`rsMbb-!#TA*>@60+-zn8E_Q?io9TwwtM5eG9` zhqB&$^zQki`Wc*As*Q%7@D}yHAI>%a=c5Dy+W2VJiMA=f%Lx8-rm4)y=vKYbsyP?x zv?e*sw^1g=D9NoJu@vKCV%;Jc0gD~rg1r3cF5=9Yqz?kw4$ptl7-+?U0!_kL9fdLI zR?1W5R??a!p~qEFOhK>CRjP8|dpfeRZ&T>DjJ7syvM9Zy-`33pc*kM8c)Uqlc;}QT4t1+@^Ha zoCVo6|<PlU(NE6QNPfPdoBm1(zsPlb^!aR2gYv&*WA*_R(V@2IyG# zm%XJGfoBmlqtQJuyVmPl9Y^P&Z@n=refdHpIEJi{lp#<;rmm2PxehY>2guat}Hrga=ak$y~405wwf~! z-aIqV&^62{Lg^V$Z$ z=aX?8KpPJLq{QJ(iBpNC)y!N^wS40E7Y>9{M!QLD&-J(S!o2=Zl^2^SR;hvO)11hw z883}SB6ZA|U6;(Zgp)jIB+I*XXqa(^L33Tsi%5!;`d{p_Gr5fNuM?nN30i=>?D=lr zf|pdfr#|v&#8Wz!D_wooHdgN>i)3a7!Y4 z#8Wk?=izH0npEN!S$Y%2#PyyujRgBRr|wJ{#-F}Ut?bXM0k~7*XI|E3t^23v0Ebe+T)%@9J)sc*R}2gm!Kr-blrt)*_`>oyTQ8Dp(! z@14yFYf#jUj~c6R?#)dt&|$myw!@XO^2uLr(fPpo0b9ZB z^yr@Zh|G-EPA14eDRVTG9@l_py%a*k#HLK0Hw}`Y4uOR2Zj(pg1GcNN79QGfz=7&5 zqR%YzeV79Se*=^j3N`Gj&Y2NI8*D#1Jj(ZQQAt{xwc7SoN0nFdbBlg*%CCBKC7E&U z;+1T-Yl7hHl@CR$IGNmVyz((;a|Pg%Jta;OoOsHVV9_5DzQG!>APq0O^>Lm3J<-d* zJ<`p|yn^apS)`<2OJe8`wZAj<;&lPgIuMMDuF`D%KpTI!^g%ru07sjzo!`< z@@ywxRwt+1J#&^0&bI42!)skQoF<(`?n|5HF3{p7<_VoDVZTq5c0PzG7rtd29wJF3 z%D2~|2LvG=Wv=)kdcqgT)E0ZYiAH-~CFGWtqw}VVr%p?(g8*%E#-RwmG`Z(R%=(=L zT{oh*Pw}7PPd1w?hM9TzQ;l|6x$ACAe#Qp}((m_UfC%tncBkR8*w%x3EL?d9K*>Fa z?IGZTckMvL_+~`{1ewH{~Ny4D3TM%8*&z1DbO&4GP16PJS= zbBZ+j))#}VTcdM#5O2fBlV#@_scYb}OXUIy`-AVf<$tdS{?`OEXJ0Uy4OjA>+Q_5a zs0qJ(D8bP+DRgjL9XnoMYzodK@Qw zMn9Qw7i*7-jK5Z6O?SmS#rJgYbv2x=16u#^(!e67vrD1lc2`l*)*Ah=AEU|}MTe!t z7*NbURWh^&=)wh_WD+*FS(W4`$lTlpL0%H|B%}RvTI4U+z7iRM13_{qjSM!^YPI$* z^spixZ1LAQh64dW*4DgvQ{r=;y3j}+q?>T2kkJ{ACZBa?+ml)pM)E1C3fCC-9U~Av z6Q*IF0qFtbmq6?B#hj6+oC~eEaiPO8z~_HhRiK2Pm0)mIvJPj&o;m6xY|c6&%@$${zsKliu%!*0Xp z0v_l;SdP=V)L!C=(1zWf=e6o#(25K3hi(^zowk? zQ;N4sZt5109IxTuYaAxHeBK+579*DG=U7GA%h{WTvs-feA8lI8Xuww2Fd1Gc4EE^+ zLK2b5^HFaiN8RNR&pL6~7GXID5^+vfdG#*#Fz%Q(W<07c(@5w=W^pK0O96?QP7r!g zRQZ~FS*I3>8GlqI1pEhrJbUB!GDFHHIYgZ4rv8LYF z&)4|}b(_enY=GWm*8XFYRcel3l9vIN#s7;esQLA0{BY1uBnNJs>!jXe{%FS{7$e!` z+)~oVuk%fWK+t@MFE5tSO`8Cm9MM47IRAOS?d|{DntrkWOZu2@z(YiIGSZLP_u>k+msCXt)|NdEwR<rcrnlWKcYc1^G%1aRYT&c`>+nC0GS9+Bhm3~WlPnK9_;;O`bJy{$2KNqge>zKcV1??p@ zIZ)=u%D8cXa*%VMMx34BJ6PB@ z*gQTice~myF-LVTMUIdGND%Ueu2EVsld1_Nf3~-(B1S=r%X|8K0DYDN!)ldDrAy}0 zAF0TI^IZ7WGK5AX_gc<8Y_H1bzM9g6meKJmR$gL8GBEcQX<8bY%97yUjge(!a)MqW z8DeecUuEwiFq%OjXpFy1Bd;Y zK)lr>&~4xNb;M(GuLv&ud08y0E&3O$djD5DX4AK7m@^ijoXMZW=$p@MkgEfcd9wPN za}3ztHZPPzJ%o693G=%L_{V+i#hDwEOCWc-&IL|P(&y-}kv~$quk*6u1_{4t63KUt zLwnr?@1Pa$hU$&^!D%7p=w(d1=ysVZvD-k_z1|SVLOGDiFo zY59xklLaP-y%ahs5V1!#HD6C{y2Kxc=-scuC8PbPXs2G)^UgG+2aaO}9?S$e@eN>9 zA?1y&&PN#1E}?Z5YoO}jzs#XB@>VWZs_Xq_L9Ft;dqm=j>XADE)#lmPWgX90$F#cQ zOTYN}t$u#oNs8&oS&g2#jmua8WlcIQKd=tegEi+yG2v*c?w?tz-Rs4c2$w!Kd=t(5 zk_dae<8|7RCUAqi>h*AY`kcw8H6pv;1+6{~igS7#XKmAweavMf(Y>n3_jqU+)I4?I zSmktGzZ`sL{%cAD(w-_d+yvp77V%2P!)~BMzdB9R7`A^zP`k5~D0<}Ju=DAo0^rpU( zE53l#5xWv=eWB{U-d2i~{zxPRjta#RG82NIv6~e(LGhyuJY1?CGa6#% zFI@qE?I6n;X_#@$aIm>Qb8Zx}8^3pdJ!KPb0P=ds|B@Z7+I z4DsD7Nv&~CbfH~x7`|=hzscU|#xrA$q z-)^LwDDwlo2BSbaX`v{UmE*i{jhshaOJ}o)jr3jwCWkVItp`e6%-62uH7X3}w}S`T zp4b2@e>b~yHp>nnai#64h%JY@o)MsBMaSUn&xpk^Mzf?lyiE>Y?ge+)nhebbdZLPc zXx#zUlV*am)lbE<{|vOud!oBD_=l!6bLW_Q$z7bTW&s%k+DHM}!KcE`&+e&j?N-}f zwK{lNSUEEET()JD;RQA}sOiVvvCKA$YL!M{Ku=Y|wi)HJXX6Q!&2naz8YBbX?7wy9 z`cA_0 zS*J(IIBD6vML6~lViUF<0;cHL4FKUPYaSjn>lgP8J|!rCCXfPgW`fngg8kSrU){K~ zo&G5`{woQDcbaqZBN1(>NNXI^8w@Q(@1?p2;#0c!N}nPsTlL#7`xhk+IjeJaxbsK& zRkQy&^SR#Fsm3uEQr;T?cn|XxOdm|7s*cN~uIU}E4K?ZzVo-rn${Mg&8!z*s44aPk zB`U&#K_3AB*8ym1mfvE~s7|CJ+Nqyt|T9H&s!{qzYuJlIC zzT6RqD+=ITOY&>IBVq(i8`MjPSq5>)VXQy7w4W0I{%B_|vj<7eD%2NV@?)bf#r%3VW%abjh% zTUR(A>pes+12y8cN6)yUb;v1b&hmQRyn-=j+-~pB4-d!et~GKwc+LNnOm#5YT9@;K zqk!+!iw0O#+G0%eR%Q!beVe}wo1$L0>Vx+9KRR;%XUMa&w5Fial6+^fcWk-gFcrfk zc~|eRmj#Rjy!vda)MkGI0Wym-2|pR%02PTq0M9r}V8l)cP+5W4j;h2&c*?DVrE2Dq zhwOiT_TS>FwZtOW5Aa-UFs=){kA7~(6 za1VE~xLG$00#N7JeNG9{psiXC)dE1-Q7!noLT8nKea8o$@!F42!EQI|`|&zu8g>+l z|8Q_(W!s6drJn6Fu5fRgFaDNvp(v-uZw(Fw&5nPZjmqIm{Hw)IT2yCm#UJMiw2XE5 z`kf2I>OKUtPAjS~Hp(jxt_0+*e+pBgd*`X8rsho#-j&}M8ylNwY0g~x83}%{Ve*BA zgiZ>D(lHPtT$ONGjnruzk7D9nGJ0}j0Xq2VadH(bpP1ok4N=4 zAJ*2@sq5+M%W8_&?=>(mct1S6b@T~bRY6(#NlQyhW=rU`c`O#I0scR0DJ(4P z2tU8m$FE<%z5{FXlv6O-%EV??fduN8=IPf{3kkk1<%Ss z&vP@-5F*-|+5FfpHTd`7u}Q+%-k1pc8#hirjfnUVm;>oKy!$>44K>u(*4E@%`}ip2 z<>x0>2l@D9ab&mpB<0(RCc@1~8hC&)BR`@3;RpP?Mr$YH(Z_W1i_5In<(9pd+;V&LO@COV{VBv{q z92oUOeV2VJwU)&SXzT238XPotc6HTs=L~xczLg^45WTmxRVODeZ)?1;qM~Aad09^B z{CVA>m?bjWH41Dqq(MI^oj!e?zrDr(_3e9D+tQ-_(UIJkix)&TwBsANd{6BB)^9`L zFdPwgRQ};F2VVqy-sA(4u zf8Vu25CvPSAIkl(Nc>yy_}|a|*Dd(ZNB)1W$^YNiWOYwOKaCl!}T&PQHz$mv8|HyItGktp>jTG!!hGUxEO_m-TkrIA4OJGq_Q>pxx?YME#0zF6`0p>Ld+&PA1^&wUVt<8{Rwu=|9lUp2VaA3{P#QH$XAHW)?fG^mt{3% za51I$uS>u`_(&xR{^tt_lEVN0bQ|Kyv$K}ArTR88@|qNGlINd2dzQ+J_ivFxZ?8`_ zVE^9shM%2PB)myy(KV!Swt1%BQb_*7(`#J3DHEdH+{FI={!eaRB1`!5E7!pG=SPZz z?Zt>v`vJ)%X)T&GHT*Dzd-Jj$Th4|-g%)k*V_zOsI8B}T`}5O%yF>edys3$MNWMyUJ<@I6T-U^usB|EK-%ZyfekQ0TS>Ce_tR#XfvZn+R`Dw<*@ zK=)eXli{>@UVXB#&9^cK18Inpy}2Y46$eMrltRmP%6ISHv5Je+v9Yn)4i>PPo0~I9 zy0b=pR-0pqqfn&(W><>dZo2=$AEDBTk_S|0&YW2r^O%s(%p<1dd;R(~LP$tRCZn!p zIoFYRH@lyJlr*f**7x^}@BUm8d~f^hwLGrMg7o9_Qg~@<5zn7L7b;~DFh$0_7S2~6 z{LQ8ld&u;~eK|TX5Fud{C86avb}J}>fKjvx6b!#=|Le7U2wr4#w8BD9T3?=#)X&da z#H5U(&8ZTuPR9q2Y+`N&x>o=Ecn{mIlp%(e-~aavH8pj8Z!bA~FXge4bB!>q>CcZW zGZBVPOv34U8_P?P2bA+&$st$ne=z<1?ajygS!8wtd2#u5CkIb4>cOnh5lp&@iL88l zWIFlAEw8THcl`8k3MWMZjjG%wG9_;3cPuC8YCPXGLMzB>gA`Sa(`*Y0k=-!quu z5<&#Msh8dJL{GbupD&~fGsIPRZL%ZRK4}t-++nBsm2VP%j#qy&gj_bT-O?qt03I~7 z({;X!-tEtK+>1)F1&i;#`0fkIo*uc1*#A;3$R}8MGq4z@d8RsdOW@Mw%Y-}qu#jAe zqBa`FDMx?%vk?T|I$4z+Q|G!sX5Et-Rpu}<-55qp!_O(N=?rPAN%$-w0qb?w;_$vZ zUPyzP5Y{y%B|M~lWMt&c!l!88-7yc(nK-cm^A@7L!$Y|(XYCx_AXqVrkurxwf5JCv zS=asjWdn=GTwN-e($mwCK)KT+D*fDZNqk0)Z7cz?Yimw)BzV3@>usLPr32Tp`v`+s zKQq(v8D5KLmT}r!GefTRrppxC^fHgu_|ST+j5fUBH4w09CE45CL&Ha1W}A9ttZKpX zQ?@8+&H!2&aEJW>XnD8*o5o1WJ;z3CP1%deU&*&{#SoMT2~cnW_x4^PgOhMIcT^42K1 ziHQjsoOCkYZppia3!#L>#7*TB{xP+FhuZHv)#RC?w14^VUAlyWhzWnv_yhrrUA;T* z6B!@hR`TEng2)}MVZ+wzte=Tt#n)^9w#`N`?VOO15YiBWe^E+`5mH^Z)Q-k|c{q$& z#{0E(_p6Uu+2`;`=mH_i7}_464@pX*-PqhLgf+K`F`oM~Q72csKhKDRhxg)>j73*o zI=z%T4_tvm)m_WF!#*Kq`64{rlP70M=!I}nL>(GWjt}qU>R)~L{{5SP)00NUFPW^2 zuRWuyJfb>?B?=e0s@wQI(;sXJ? zvn(v}OG}uD)MCz~N-Lpc0aslH-=dw=RHp1@W@hF+$TC5%4NIhcw9>URRVO>g(#FQ3 zC7Om#)>m9wMn=%>_cvupV|n=?fhK+GR^3XbvG-#?tc&SCefmUZWSsouN#JVTDf4+r zw?K_g%}4v2=iT{&StH+8pPn4vym>QFldA;+U(jXtt9gQs7=%Gn1UWu(bi6y!=x#&C z$Ffr$E68J9O?SAv>hYOr`uMyv=_ASWWsVb%+Y>l03J9Ft{@d?57`g_Du3-i{|0jm#|*h!<=ax+<^s%t(L>py7<9CT~8mM znXgYT#=CVv{WNztm!_sOU#O{fa67Y4TEjd3`?tmI!Ae(-4{B-Alar>3%E}{yg^HKM zDHLf;_c!K95bi}@IhWa+c!6YGV z-I><7p-760gHXa=4q?jdd>|EnW$W<2NKNgcvsmg%7G4^%$*^i@Xi&}7E-oMU&dy0v zc@61Df*2H85>`Ro@`2)@SLc5`NTn{_0CmCaVu*IOl|^f;S-{aczC~E zq@Tchjdr+HDE?<;aJgw{65=3n+(6!S0WECY_Xg&h&#R38o84q zTDXuAkpBL|hhZ5H2+4mp2(Q_EyHmu9;l6~B zuFlSwV+dyVct;V2q3`c+>mBYa>nSSY*nfXV$S7=0GE(v&;SS3hbn-IaeHV18&QVb{ zq0cn#IYa8cG$@D{7#1ErJl4znXz`a7GDvIUYqq<&wdFRyxY&slK#v}pcP$?35_KH! z`r@(LW7q$=g+UbaHJt2x2iy|BwNUa>nMtL~<;5RauT61I&GJ_9e@Wt&4qo6s?kHzgS zjA$`*rb3Q%&*1(2eL)r$mNKU)g{lFx9rFkXJi$Vjo0Lu<&|$oK=%GQbc4zJ3iqx;S zH;B(sP&DY5*fy}=dDTpIUedDM&)AsxaHEr7Z*#ueVPy2APj8=VD@m{`*B)(`a@h;7yP8ikUN{uTh@ zNnEU29j{5jGA%4DRLvhKt~)(Rk%8Ccc&v_N4z~XAT)EP2ow2<*Pz?B>u)12>8p0py z@6gA4nY!1mA)6Z;=9>$>MQqo@3q&1;3pg@-pQTB-wtl`NFOOvp$Og7Q zDsH(s;1qoYH0J`SW36-+cp*XEudc=%uQido*G-`B{8G^3WqEmdT)M=gzt6v^W%d;$ z8GA<^Zx5Kb$D_#y^<;2us!0!$a_|Y;^})VOWrtx1RiP-wMrmnjU2ALh+v@5zXfa8A zSmm>uH|9F}=A?kPF_Uqj(s&kBtod>(C zy%2*9O0jhJa(FG`A=MMGOlxavTgto+m&@r9?Mx-os*J36?+OM?0@AM9eha*zq*Me= zsBoy*rZ`2+ao8aP3JYL_b|?>+gZY_wxv~fO1B*labw`Z{JIkHJB@db)7%X@7AC6Zi zW105K?51|x(9_fNNK2>o=Njk~j1(0Y524Q%PRba%x;}AwlGM`NED)y9AnCqj>!JD_ z@*bj{>nju7F`%rb?>~Ml`%l0|6yM8KD*XN1Ry#}W{I6mgb^D?@-A?(z##D%hg&C)CkfjgIxh*rDCb{Yu>9@K^+rK>m=<4OUkvkO$GYRP?!wxW z1AF#_cFgb@)02}ElEApl3&O$)kCq1eR%DUEs?}Qi{*y!4tB;Nl=e24?v7CCVy1 zty`ht;X+7`>q3wF9E*@80pK)+!-@*=hK6#?kWf{)`aR__Km_*XMu5a~W8)=U=LL`) zv!-zO^HgJ19yZ9}X0IGShEdv60J~oHXzhA-87xx{+@JOWF)=Y!o=2{JY0fab86-{J z(MC+SSHR)2*!;qR&BXvq%S*R!-Yl>?gv1a+FneoLjk{M4Bm+*W`(@kP+uhI894&V8 z*OffIA}gET(%SmMvkrE%-nwp@fQZN(iGIW^)QVPjdSvJ1aMi$}x3km2?_leS_KA+M zSIfMvy4-Pz79%-1`LDWwfT>e>tT(xRwwC$T8_{Ej#f}rTkWa)8o12@t{&hE|de>VS z9SKJE4*y25_?tr0Es#*w)qUCRY~A(p-qNm7iS1LZ5!jD{j~_ppkK`NI41e}hQ+qsl zch=<1n>XkZHu^w!ZfhvA>ZGf`!vo#$`2!aJ3ulokzeCa6w{Kf$e0q*$x;T(u{-Dn9 z(7hp?Vgx7V9Iv@u>F|oHu(Yz}`;GD~p}RUhwE$D1oZUi(QYVLQe{`MDMNNX65nG!@*He zQH?f>zj7_0GnEh7biEW5UD&HTJ(T-!mvbpS6VM!wl+?)Qnjwt!YV9Fk!p%$WyHjb* z+1j~95=k;Bj>8LtT&5kd2FTO+h{rouCs*HU01vMoLU%At$|GkuL)z;Zluo_UY@K{& zs6kK1?A^A0--C8zv1<&tJ4I_mY{D1P8=|jxpDco$<=jqpZ?8?P-byxr=OVc7<4>Ag zDVx-#DKpd405FVIxCb>p>a$S z&mg9w9VTki{5IM-w2QF=PLIp`{ne${aj@$@>*V+N?@i&Kqor-zSsCjz_S@Ei^?IrB z#uO4m;Si9QflSGD1%*L~vadVKBOIJ<7Hy9c7hln6c|(h~`H`h=b3L4dqAcpXqj}Tw zfH-EESGl@{!6>Uig81wV*@W#(Cl3OY5{1!WXVkTqu!o>5C-m-np{?K3|EWnO!AFo@B)ty7W1_M z@=ow^YRk#Vp%r)MXtS4C_&YpUsMaBLCA~2zm&(-cWZ{T38J`#ZpS5DJOf!5pqP~q| zM2YV?!Qz|4By8#I?2LgS6{f%ks_Y4Z>gwv;A!7Xe>QBnm(j=E4p)Fw^wG24kPMVxW z>Y>eP+MNowzEsO9_Hk$dysB$k?7pm@-OH}}vK}dT{hAdmk?vECxbBnnSUdA6jzW;q z>`!mzF!`A)yF$1P0tybZEdwG!I9avz;b;Y0Nrse1-nUwStV`Wu6D42d>JI*LzgcdH zp$%JxX|{Bq8Me?ou(Sz<4BES{7z|H}T!1WKzO({fF)NKKOU1pq-rkZ`Fovc` zxUxfi+ndCG;{e?<8)jwfdIqtFU4R7}_>8N+ias3u+WPoBw=ajFZ!`Pph@|HlE7VdR zHnyO4OCgBo`wEVfIP#iI>azQnusdk?+ojYHf?j|BMyJ`YPeqqAy#HQl_%$uzz4dW& zi$TU)M9OIrkqS7GJ_LdylrRJ8F2OQXEC-7e5@zX_yhdL%BV+j(yo?wC4^;K9SJ%`G z4HjAq#nKChHn;%m(SIr0^FsK*B49CzkxJLBg1GqDzCUg#xMfjBLL2!H`zbZ6XWRZl_@u#VBeoqC>x z>}h*$b{Uzps}94@u~IXYRSgYOopK{c=o@#IhMM>cE9(7Ek3D|X`uX4m)~q!WEBK_T zam3zvu5#H2u)$Cww+@}fIh&2=SMG=I?5>Pav9rF{xB`t;+6PAe(DpxoRP`ic_kIf( zE_XMLNSjA2e7KX6vtk2rmF~BrS7zPKz}BczeQ-UypXhO!QXE5JXA*z$yLXq${Exl0 zA(7C|Hs>^ZsKTC52s~5K{VcUY0C`0K3<V2+U7Tw60mbUi)A52qqnaFj*m zl(_gbo_zELe(@tEX0+m-=htC_vP-RySZXYVA@j5C3EJd~uvCO?({lH@619bSs*B3X zfBuF|52h%4w5V++#q#IzR(!I?O))V8nj zZ+VOI?5#_b?nish+4kjj{ag(Nlu`+v3$O`km|6APw~L8BLF+9OSUiHdsmGf(nK{F3h>$gXL6d6g7 zJ&~=JWx?XRhLxc5Ui&s#mvyngrlkEZo|NQKMzM#w$B%*IWeRIBJ&PAJ;eY6myh|l@ zx_>$7NHIg@y?JtSvcd?Aj8jaVFL?8)H9Ld~9qQUK=aS8N zS=Kx=;Phnf;KV$o2nKKMKC|`p^~Tv3;zZf-ZKEvIly|R-ArTAG2DT>xPHSGv>Esz+ zVY;66gOJMDoH5*kos*h~TjohZ!o}aD&a*$EQqm*m&Ym^huJJVBm6jg0zG4j{PnGBT z+y1f)|ATg=0U>KWeSD{J6Qbe)_8uvvMI z=Op|NN+t=YluHAki2{0BL+fd&I*@uXF)`et%Xs(pS@F-^yF6GIP`CD)hJ0`N>9c1S zN@ScS0}GDpgRmRjfiK$uihHU!G!!QVi`P-)hN2?%;kVf&FAo-mczJsl!PGCFc|RXdQYNL?&HtMcG)z@C`8vmvyTbqej&h+`d1Z1G!(wmn!gK>#N9jdXjtvX+5_uR|u z`}#`Mp(C7x{sLwuv#qfaif@MJ-oyaV7GW0Wu6nlYS2UBM^F|@L+|iTeq2jhY;m07| zJViO;-cOL)FTCo2#TBlX)HedSPvtOuSs6ygVgc52FW&McS{+Dgsw&PZ@^ zXK!z>3QN>^-P6N~)0|le=V_eHot-l8UkoI8ANp){?d-U!fMl+6nIn?&*~v-VRK$=E z``~R-h(NckEQs?kg)z2GN!e{>G#LnzJKuz>h2MO%2o+>kyabe6r;7FX%31Yl=vH@~ z1S01Qp;%*AxZ#^aoWQ`IB4kB$2qP)xJ6q(`cD1Ka0{q_OpexD7mnw*W#xkHq2nX&% ze}R!hZ@Ecbdwsudue}r!fsIzf#YM;mk^q;HUqs}m9pySOa}eqy(IIu@EuV-~g3Buv zu3y)KdZ-J?K8%3l1bPN$BjrJ8au%5GTKnC-h_@MN+{)Z@vSW%w*QrkJuM)ep^9V-IxA_4A07S&dkg>xR9*#m+JEldQ3m3+21YGyUlBA8Q)t8rKf+A+%SPbM`nLE!9AbRvC4h%R~*n+MIIA= z1Ao7GED!gyb9@FS*+*6uH|S-iay(JS2zUAxfLCBKfU0Q}9be)5J8RV?6l4f2j~U}? zJr|!?t|z11r{9Z<_V;5On2-Fn`;=JV zLmjLdX|c)$0yqTlzXDgH0(*iydqP878$R$3piriM>-q76D)4=fx9yfj%E=)el{GZ}Nt3|X9PKY;h_)d}slJ1bzt82S<@VkQ zx#i)~>&FM%xx<^g6Gd&`{Q^SGD#`*b~fvMSUsjCrM2d)?Ew83JDY&_r!(?78`-zu|R(c zt8@k%+hOO=Sd|!zdy$uKzc7XILELqo1T^q-e1?(drQ9c{TVmFBcJS_&`EW`}8T`gR z@diB-uFROcZ||5dw|txzT?Q?i1$k{!=kGp8;R>WMknA|PxNjg(D*{esk=HOHzZAA1 zOZS?muNv6OjF(k_>l8xSfVw~nqw5V76{2y!T_fP}@PIVo+a3kwG+yQI-6v}&0E}%R z^*$Xk$@v?8BL)_s@Bp(h+r9_<8Y=LN*7`B(>FGh8xb#lr3Jf;vB{REqFy6j>`xeJY z)M@fsSDF+pJPFUvkx+71D>IUlF^7eP^&41v7Z(?t!<)IeW z(Es}qI7bAteDSrL-v+=Vc}F0OdOzgK9G*ddMGZYYUJVfU-r~$NsUrj78`uu815CEi znv~dej)%$k=Z{SyIL)JblVG7`L&n6w-0f}H~ zX{kG;nrl$r8gP1OQZ++rd;+@IMIj-L0xXbUq9D0xfzd246ud-92y}fM4Y#%_kc;9l z$%Dz^EG=z>phYVyFE1&7%@!pR<2Sp!90%j#P^t-roMue>?p@O#wIYXK$KBUO$7R8= zkYL)Bh!fxkLtz}b>>h%!CUvgE)G@VOKB1H*`3SQ^NKRh<#RIhA=+RL_3}iFhMgyAO zfS)yb?`V=?l#_9$!aY4bUjs{l@Q`cxyi{f9(JvuDzGW(tXZL#IGEz_kfdrS`JL?j| z|5B%{`pxsyRKulxOCfqBhN7M>q`@?B2%}_Z$OLy925cD_5In6?tJU}4ZbnbkiLk4s zM#4i|K(G&%IWU4=6AO#a2;J!#WM!DU%6m%0RDcfin9bdi`s!%M1?&LKiQ_#Wzpw5h{0g*H&YBu8c}(<{O_k@mjLdm>MC^KcOH>`%uiZI7eJYuKNWqpo(7tOD6SU@@*l}W}EzjrD z=MSZYgo=Ka9hKfvS8sy~-3PK!j$tJwC{nEecb6pdk(F^kpB&&0;V`Up!3D;m<-WQM zF0uzY9VA}^?C4pX^^Z`jU*Cv1hXkTf@#ju;EmC1)30z&SY`#u@`lPQuC|aO7pO7O0 zz{)7zxIwtQXR`na1;!0Q_a)s_8J}|?Ves{N;XYrMV5XysghA09Y6K7g_+<~rqS0?b zGzr7otSe9hX1A*H;>Ze?fRg zCMkE4DvwoY3NlNQp~#B2>}Hp+E{KC5pgO~JFY@3!4zdP35aB>sj%Zm0xhLbTCDmE4 z)Ex@bN?G6m0<*F>jD41`j#hh70iRy9SUpWhMAQgWR{=0ime$th&@a}ux5L!3nqcxU z`rt~Hhz+hRt#Cprb_4-f^cZT=kV7%>O_wfRy2h0_4d|!%Al8G3`YHpWd&6$MkGfj0 z)MZY^On^6BkiE-)O-1j*Q&vey24oHRn+woaQBk2E!&2YvbPW2tb}`41AUBcmI!)Rf zb|4@*>`7~BaMu{ESZoq`KxT*%t%G$z++~&kAa`7G7C?Du@$$eeqIm*o?zS{Y0mX_> z`{cXzcQkpS91$q>e%8vyJifOj4iFd*VUl$F+UnWe)zu7}hJ}FiK~c838(%XdaB^}> zRTZbcJ9NDj^$j@Bb?T<#)L}F6^I7y03^KP1U z_6<-s3Shck8Z5m2l3vzhg-O5sVH|{n{&FW{7i45JJdz+l3q}j$%s$hhBCD32rvOdI zeK+q^Z}vzbf#}_4tnYpsukmr%0sfIggn<+yFZ8e&sQ5GBNCPv#G|La!!M(f32 ztegXt6cq`IP>%-8CXG<;ab}ViTH_cM6m6XsfQOdr`*fcJDfoj8MlWU7r#EpTe!>V* z2sD=Yh*0ZVkJRcUC}T(k=kK?@lJ!FJrJ((yj+wyaS?YY9x0#v$8jMNp=)&}S)XTAg&;;>+zN-vMI;)Vns8I<)BR^IIzVdLZ_)>lf*p%z5DS%k z3pbLiS*}nM#WGwaLe+o76rhMU#l4-!GYbn53|H-*-}WDuB_bj!a{dCgk%n_q4wp`3 zZAU8vr)Or?!89=v=QdpJB@EIcVZC|6DF7Vzy7G2Q%7YKpq8dQMz|~PH3N4%{EP+-X zt8@`{YJs$U2256s{?$i7zBd7`Dg>I;?3qzl5H@}&!Oq*8mnchY`_90C7zQ~ZBqW40 zl2Cq6SV+hT*z**>3hC6#NO_P11h~P!(8l*oyKv z!Q%vIz6sbQQUpp%#8~_7w=N1~Ja0P$ytixt7d2T5P5sHrXq{?`$d#Qy8RpQ3P}xP_ z^j2rK81H!i;~{X{4af~-lt;tOO+-joSkUe#CpdKom#;FAw#Ku>gA|61KyMB0HTIms zMe3pNVMqi5Qwsl#l=G*fjQGvzPdBmSGO)gYf(CdOw{@pTL=E_n+VqzHREKL~EOjruhr;?OWiUj-%N&Q2@J;))2rQFHhMPu(j zm(4^Y+e`%rgd#^e;b9~rNz*AVD%AtNg+<;2gG87Q%t*BNKd1&lO2D|BOU<{6De*WB zjflWT;C3!E#nh3XgTNTQZ;qlufXwD#R1x<-5(f;yuAWYUc*3w*2qbuRtuDA!P~e9A z+BLXE{m68E>(38-@FTTBe<0)M=eHGMfZmCRwuNuuNdaJ5a3}o;JQE3n2*FHSd>cfA zGe~2x%re6tcr8?4lG5yw*2Q{257Sml2jOB01~~kBUXIB_b*@CB<9k1Y(w;xRlsaQ( zL+6FR4`JuHZLkn82T9u?@ppAK9>5HOph{H=y;&4o(a6D{Fvb2 z;*~3e3G8*h#`wT+EN z05kXQ-VKBp8Hqb6Tk!niv&#p%Obo9<*z3tk(7a)yvVpX8ul{l-gI7`KO!mJdlRQ85uhk2 zyw@;;5wX#E@ApU03=OukjPq%kBFODo0G^Y8DOUW^UupR)Ja> zl;(*{&(x=lYrMCG&S&ti69-;(98Z;5Z>CZhGORS@yEMVq0XlLh*Cm*9OH!_-3bGqs zcm885E1w0%!bN~%sI-XX}AboE+wBQ-wd zT0XMLpKGE8rbANzzgfjPYo()lF2c9&2_BpwK(k#RRFlmUztsRcsi2`6r*!c-EnbD4 z6mS66&=Sl4-X=RQVSbml7Eml=Q(q8MJMxk^k#6zVdB=t3?G!Bz~yA6;m5 zp|Y{Dv6iURF2j?kqKB%P02^BX4e@H`UGWY`QDy)2S&MtB80t+SWC~%_35drzTxSJa z2SLTk$phHypbO|LH!rS&=v+p#A25vXjk8f+{bYM{(*kDTGcYRk4gyTYNrpboTT^=Kn?89tNRuAg?LuhoJ0HMcrV z)V#F^WLs$^-i>`2pXdpP%Y z&)qw{BO@uKAey3fn7_tnM_3^v&tZgpwpPYcuuMT4qA6Rap~5PF%q#>%SfFF#mq>bE zQnNtoG9}?5RL(HJz0a}LhTyKW0Y3Sh<(!F$|Grh>z`%GZv#@pSSZ}wrFlxXHn*k9L zr1$md7VVuaF!Kz8RjmWcq1F3=h3*s^EfF>mk>{igB9Z8tZ2$S8T)-b39GpX>o@-R4 zJ(vZrgd{W5F2gmMdIe;SaYaB#NXS337J6~m;n~U0pEuWTjiy|+*ZyGNLhCraTnaQ@ zyx&%zvi1(@rw`kyzVme=9!jS<>LY+bg|@a>Ki3+coSB;I#Tir7t_S97bgs1k7L69u zE$esiynW>|7d~n?VgjF?1;lchi{*O$r~6q&U0teGhkzMWJf94Q=oGA0Z$9qd2{_qT z{>aJUkD4@5jTKx8i@D`FJC!iyp(?@JBuyRoMSbRzZE{kTQBErR33J$6GM@Ep^$b~2 zE&V;pma6~-eyoExZ4k_k4v)_>F?DI@8R|{f`r@JrV*a2gdW&|YuhL}w^aslTdVkRP zwB*kbrD#5YH3+^8a2nd*SJBVHePE2s z{k$7x0)}nWS16jO3odZfxrJFd527xRAgv%OyaWr)9o%a^(1br|awRKXDCD~RQ2hk# z8N%S^R7(|S!=U;V2yK@@?-SkNrX~`FCwQZA7v`9l*jS#ObhO%Z!i*~EsGtyhxAgwe zXD|s!`5#rprIIsCn?mZ8J>XAN2|JvIpr9PyBrAfN_1rrb9H@W<+~die)Oz0bbHuaA z@rk!DjVXiG`-8xwL7jh%R^t1&`CtHeeoG1bMbRK^gu!_AW^m!YrY5yPshw6bUOZ44 zw5h~9TzEpQ61ZqrH*0+TI~xk8cA^E)uDEIU?*ZnBG6BjDGhXc_I+0XP0YBX^AP^7N!$6p;2i5V@b-uT66ba{ET-y2YMiW@Zi@*SyT?RGH9C@knsl+EkHBB-R>(}E8-~_GRn|uUxw6wTa6_n4s4@?lXk6%kyx{Lvx)bnN#fLok6%C(&uYNEQck(j@ zKZqrJd2m4BN|__H<|S>ubTP;DxZ}$S?Sp`umb8gafftW174xbJjmUWI2p34sOGMpX zwFjTaX1Ehob^tW$ZP$Qoi%Nw(Mq41#aJ-T9eE5(LT*Nws1JJ>SKS*A=SNT*xJ4mn; z=U&uJ81?}jJbm?w$kEXepmifxl-IoyeO}bugidI{%arc>0tVZI3cCgfY-ASZIaPEP z2O;SLG~wOK3r&l-b8lXa_7JO7=i1XOoM2M9*|%Vu?+-!XFC@MLoav>3V@2N1|_=M8n=#c;z2TM~1e_9He{rGSPJr2=jE&RppPcW2q-OK*G-j5*-(L*7y z9g(rI%^>O)nl<5dWyrCBjf0k}{|J--=*(28!@tVUjy$9x5b%XLwAY(=?%V*sVmLX= zW9Y;~xjlqVoxwn+J%0Y28jdTxh1QJX;5&RzN=ig9u({yT=l}lw`y@bZaC<_CwsQ3e z%ENIe*Z_251eIA1W+Dno%3x?+7C_UA_OGazgCSrIj2kI(DzWXBjlc$iKZpWO@~}!v zGr`l{fY{A(gL6gApu|y}KhMjP8K4P7Ntm=c2(C)7UmExC!|Ow#t`8a!l>?`aGU&Pm zra@zdFL%BIR{-SKV`zE}K;-sHZ<2ubh^P(rL8efeSGlZd3k~=37`Tps#c8 zTy{=}d0}b1h7rcQh;~Z_u-1bM{-MRDKr6bNOQV%k5INxurq{p#goY>hv8e5vs1m8r zvw~(xkhx|cv%eH~zJR)!Y+|neJ7PHHwlY!2f9Zo7cxI*c(PM|=cq-FsXG4wlEaO@d zH!uG?R`}5_GC?mPP7%mP&ws}k-T%KsjGVIvzW3^Xf4z6e$JOnZI#wH6W*JnX%n$wJAodBXDI z6h&#NA>zrEgGxdv-2x180x8|5u|vNv7o{6?~^4LhkX7(6U21> zz(qMZ7BHK_IX-RjzUe0_pnOvYf7jroUfbGI1mXSY=xB0g20I`iz|+eMO%EuGaBdI2 zV6rT-qhewrfjJ$1CjVYTE84!t0!|e6*@iA=_1Qjo{yY@&eilZy5f@ii7(o%ZL5PCc z`FWzlb^aP-a#%JHKfriz3Zn!P2;MMsf&+1>(F;V5Xh7T8lO*EnrCOgf?{XzJKAJay z8E$QJGXzvLknAbWoeKvd73|sLH>Z?*Ell52)6mP-Q5Vd20VS}F5oBv z5|Yr23>GjHPR-1Oz$JmP){OR%;O8fQ@ZbST4nWCu2IUGahmz=U?(A!OyF9pPO7wG` zGQg4rtMCSnv%wj=j+dnoiWhCjw{_%Q2~>c$Mm|YPe_q~#6FqOB3VhTnX0JWkxDEURyc`RGO$>#}={Xnnkea2G z3=#a1{p$Xij>$~w9*qL6JlL+R!Y@-sIc;tb#RB;+Rsib@Kp$>}Q(6_EF?_)}2%PsH z;Z}#lK*R6)wXXC8%wp8|r?0s=InO@7X>6R{PcA4SK@aPT4&|`TDt-jo=iv9w!kt64 zMP*;SGmc&Rhlj*UVQ?lE)-}vNS35g5Rv}px52Qy!*TDPp*2Q`iD{x8?zPxLgA^wA> zr{elSae9}Pbx}sf=ywAA+tSL44VVC6`^l=#+4Q8!1Eg^%19vv1!*I!`Z?0gf!@hlp7LKHwL3A(zsvbrI~%%j=pygao1qc!+AV{P^OfOv zo0IzHz%!5Wbs|nkLD)XPRj9-Z)ZderFC%~c)YL3ferL!>o0kE!rGkP2`h<{b4IgLX z%0aqA*~T9~?h9zcNa*CZPk$k>Dq=s`e|$<0s2<=IAjuTKj~V^|?D((O_D_Lc1iA7_ zT3Q^`0jL||`zkU_aH10Ky;1m;m{4;s=0txpgdD;GBCsKHX4Pz6hPdADS9(U_Oq{2` zKq4Grgz5svoM9kLZ!%x`opP8$Q^EG~d#W2$lB=VDYp2m4jkSf?hIzym&F= zy~S?sskh-5PDi28H_F(FFyvl1x3>#f0B9muKCj;{Z1Jl}6I7K;92^%pIPl@W0UhO( zC}q-}81JmDoyALL^Fhv03LFtQc2Itc_L>h6DOnxN>0{>_-G3K<@MDfL!3{-~0GWlu-f%v-ks+QcssP zDV{8Ed9scC>HB2+Z$r;14eC3Jd%0VlNw@hv#PQcw z#hhJ_4%M0Wz4`FwO#&#I;ozu%KqQK;En?WLrbq(!s-eNSPnU^-@gFNoOM(tQ@U>U` z^8HxcV9I%)Q;tQpCn64tEP4(U&aVU)cJx|r2&Wv8gnFJE0#(1GU2_OJGKza4K$)dI z(F)w~dfo)OK(Kt_pH0sl(r|a$%d_cO#@y8Ey>(Tq zyLZn*G=-k-fAg1po*`DkMa-UX%Ts%8ASnQEv)E&Lk%u{OZR+R5oG)8YY{@_cLf& zDEX5*|9sJ5olYk44^MbDpiSaSC+ax6v72wmm+$69Vn}*Y$>XoI2K|Vwfn%(SoV;%I zwr||N{h>fFF2M4=Jx?guXlL4DbYg?f1I9-*hT5HgK97shbmt96q!F+_8%LaNDnt&?qmvQF3!P} zazjDk%&bdg9QN(3A5*#IG{6r9%&ty7{?9usrs`3g&j2*1yglkM3cUN zf`UqLU=fOd*$^EDb;}=~hx18+VBkYHIhYp@fz}d&N7}eC*Lknb-w$OHbZfk?;*m4| z8>8Vx3jVRm4P*ZikE2-VbFJ3Dg2jRL0vQDtK^^M{N6k@GFs)+&(x3qOX(8Q*495Nz zIMWveEC@FGX(4Hpy@GHe3JojZ;alEZkN5`X8{Pi?B!rLd;6^n@IC^M$aA|{OAQrj0+qC5h#ZB$^aVD6gWWJ@oEWB07KE?6IL1>>ru=Lg=Pw6 zRKK0TDN^(ULsHK&_in z2#-M-39^PdEUqJ6h*}@bia-gvBkp}u5&bwD^|D)WD z-jd7LE5-u`ngc@g;-G+)84!I)#jTpRt-nE@ApyvR>Qs^QaGsk2L|BxG0F0Nt1J)Ki zB*&!gZ3!s{o|2R`v`beQ#j%D zQot;@djaLLAV~XQ&7rpbnBRmM>1Uo1I_xWhUJ1=m=_OIpySH$?Msj0PsQ4s+s8AeR|LgF|GY$}lRXQ`+{V3S6C^WcLwz-`1N>q`eG z>j(%4>cNQd4Rm|B=*R~@csC$sz*I#D@Mf&gUFT&jfsa$fw!dx@QW_>rj}7@IH8C!v(4fX@!*O~NPY zt(hiUp^H~_A`tbx7vzqR=~wKn z{cG~R=6HMJ5^mx0&C9++I=ejxpjB>_u-5hxsk_aC5VAf+gm77(VrkH`Cc5(bj|kgig>gPx9d@5walST(%_)t9alGEb znPm<%B)my8TAMC^?Pg7z-&G^Vgf|5l8$qL%*Rrr08O|`jm2ONf58^hcI1&DfTo0O) z6`)31&pZr(M)H1po!lQc)x_$LPiqA&UABezHa5y9<2Rj2S!(qn;n8b!feqcxBjqj5ne- zx3h5N>zN+D8Ce}qr#ij&Vz0M~j`BC={F_`7eRlP3jY@PJuWV>k*ziyAhX*(AO4fhG z|GR~!uMY2+)idh&EV9oyB{0<+=E@XV1GP z%B-&$s&qA&w5?96dCO7?Z8-B6BkaN<5engXk3#w?)d;-%$!}EDMHJt-Oz`&E+dhVZ zoi4vi3$|F!u-?rzrq7NuCzu8jMeD^wBUe*&i@3gFxw>JWqS?_aqCUML&{ zM8}#z(SO!O_^_j~T24L-sq?2a;s@Ab-!=#eNIvxYkJjJ*fMe-WFGp#6SiN2QV0|?3 z`iJS^Nc?Fbtv43)Y27)034!0X#jbmXCHK~h;8!f2Zq{-%ECbgAn7s!e=oX8oJ| z+m1S!G;6*T%F8U*gPwO927J{D)VVpu`pP5R%X!4y`^;4Xy&d6{AbcC%5cg7k50ddd zb{#3}wcJ~0TquK%++wcZ%jh(cGvZtD{JDTtT7|@g8|(<%2AO=(OH~g)lXu!(Rwp4{ zOcwdaY+r7htKfL~yH&zqyHe&YJPrP&T1Wpc%Vo{Q*ABM7?`{(nx|_~rVFyT?jVW<0 zFufwWI(a{Cq%BMG+u_A}18S!yF7!90u|08I>Rc1cTdsUwE7>a4)OJ3$7j=Ftnfbz1 z<>lTPJ-T`)Ybv#f-Qo?l82QIv&s~StrNzk&hYWwS_LNFz!xnkG%IGm7 z?=_w5|8b-CNUXlp;eF@KdAllJne>EDBGIy0r9<^&7NdtB`O21AC{Scn_;z1l&#s?G z&S_ly`yaQKl_VI|UVHYk29VU$dbV`2G#Psu_N5x`5Rj8cfS=af`~os)^!cVV!@l>$ z+Ho(^y7OOfi_>iUf*L}XX%h;M&4tFXzi|m6t0b5}?Wbj0yzo&_OHUK?+g6nyTC7iv zeB~1ubl422%D2Fz5X3sNL`7cH-_WFY!aw!}r&jew2RHbc&=#!8rB%7J!0~rUBE@sh zGS}GHIM~QiVDR)jwSx(DQeoPLMo^c*`*LDOv&H&h5$p=Tg4ttBuGb?5*9P=nIPBBc ztDonk;P%zTkt=@qQ-Q8TO^QU*9y287&r$G(#H-uaeps?#)~br*(mwso@3vUT5H4Qo zt*^~zlfS#0HcKsC(zqw0_0;t%b)d&pK9`te-lzK37E`y~PN^R~e;Hj@o^$tb7uJhnOvh` z#F_kmM4bgxRa^J=4=LT;tdMRKD;U zqBWrV6~bP4z_(qV^yzxaBBMz|7Uh>3?4jSzm#KbPEZlp!SM^2D^p|Um(Cg+fevcC^ zjZ-O0f zdN%uG;*h@@(`?1rI4}1JaMGN)t@hf&lV?T5ydUuB)WDc+O9SMd-EuPG4`29r(wlyvlhwR2 znQ2KDQN;>k9np%?ijulAT*Ljr&`;#j_f>ZP!W)d*+qC%MypM$u$2A{yp<11%~oIg3nGo#)MDs*fgHLMop!@Ns2jz2>s4s;@bg@Z~!HyA*2 zjd+m+VSgj&sul)P_&}z78Sx2u6Z{F9Z?OojB~VB(D>D&T2uv|zLCSwnO|7hXvPf2! zq}@-Ld=Ow6rt0bcDPx0-IQ?noi<%e$Tjx6wt^Dav39hX;1*f_&d0>c^xWv;Lt?W0}_Q&4OZ zT%(BUsi(eVMB%+q-JkzR>7MoHeAC*`bu-IOmXx-&b>6HA9#AxCCyVORpyc^DdwH8g zFmYI6EO5TEr>##dTwmL@;{5kJJfW@EV}q?KFShSDOmzw`H5bn7NLRS)1a3Kf^wj5k zYdJK^rM`mi^Q>L1#l>6lvB{k$P9-nCJ?tjc|CnGt=5C=q8btDSE2?_GjCJ_2)I=3j=nG~npmke(iM&(yGL!IEv!(96!bt-k5?QrjD&(<2`(bCWA1 zVT3$V=CVI$JS}zV-U|=BAS1ZV6w7p=yznFMfikC3#bo4d>ZHo`S&TIu=XBc2-=`Rv z@6y7<$)|Iz--rw~lg2U1t0_56467HALM41;-3R&S$3frV;3LRqvb})fa3x%Nke^n8 zQ3ge2Sm4}~R8R|(1j8v*F0O�?IV#++Tb(L5P6dIhq5!n?`>?JaadzyM<;6+x&El z>GtLNkG*a%DSu&85N3bHW_Nb5ljm+(Gfp?{D_%+!i%PiC$Pp5(`dHOGo%*GR2aoA> z?1}-l;h0Kw#kA|;v3Np*dvxI%3wzHiJ680w>mE_+`}D1cX_w1*Gt;gIu0=b;TB$8w zi3^G85caF)k~j5!B$hQAMqV$nJCR`I*1Cf_6cNxed!}}@)LUec-&62OmkhTap6t&X zWFC7K9GrJ3N*C?iV@i9>S=%SV+eqgk_nFKy|3lR>kqAA$_rzM`NxqBf^D+9Jrv1B_ z8~ko?`PY4mUpr-X&ugwme}1nvvcfr|9@4$Pc+91cW#L-3nw1}}C8(o5^qZ3Ac97-v zNJq#@!7SeSUO>KMIB7t3`{YUgvf|Sll0~H3nek-;T2xMWT1>BZQYTN;eKD=2^y_{Y zBrDG3(@}=18XPfh@KgT8t8Pq-<`li0t=Dy?EfB1-ykj&_iO=IwJeGfd$rWeMpk0gO zQJ${Em6>2URdJE*x4Cfn z1b-WH6o9r0urAbh=38U>t_<>bXbH81ftMc_kVRdfrw4tO-!>DZ6%cMJT)2xqIcTV> zvq6RFaC=_kTKs_uqM1pT4G@7#Tm<~Z^}H1YkdPwPctF3UN_JjN&eFSV)A9O(K;C(l z!=v3CyBK|$21h3sv5ejER{A67!@KEtQ}o#WygLW!-`1>O?|o<`$6TPf^YWYh;^I=n z^LE1NWDy?q`jRN>w>=b1vcdwUkKg^#Y0K}dk53tvB5pSJ3O+IN-Y>Tau2Aa;Xs8JR z`+!8?;Z_W&xtAaX0%{r4Dt%E%9wB}b4-l8DYFH32$AX0lBnbJSLi_|)0~&{7>lSG! z!zLsrI~uYEf<1u-WS|Iu5DWjwMNh#CTeA{F_Gh0@{5Q92I?msq%kA!ts25Hvbbr?U z*(;_&T%Cmz%B@-;ZaY0@D9PX+<+4FRAC}hCo3ElBZ^f9LZv237m8@#Xp3FyGRF*;n z6MnJgu(S)+!Z}SL*T56@gPNK0r;ChQYFf&1j51WH9grU3UY$b|yD_T;B!b1V-0h&oYk}wPJQxoj$)Ah)&&7}+54{wXU z{rax+S#tUe?n=B54Li@dV~RiH?87s@%;+xjDxiwpl~NufVCLqB^~JPAGEDuuSKjls73aj^ z4e`BEelpKl*Hg{2IeO`GgBc^m*7n1jbM`yqM<(%F9FhTClJ)#i_hE#~l_)9`aJC4-5+cdEr zt~?wj*S}8Q^_FZfj_d1viwd0c`y$P_J#Wg40|g`4#9pn(#k}@We|?;};OyKMO?ksp znzcrG1+KJSRRV&z!>7MqlPDR#dq(^Rh9h5PAbULjZO#RgmJw16I&Up`8jL?pK)s#! zElie+>Wz{4v6{$x22mfBpm00RLDZxIR*FAWubTzzotpIG}Cn0aZ}= zFDO8SLkVC0&yfR?9R_7h2Ut@_K}KxZOa`){CLn5>Bu2A4PHK5|(LnGh zh_}*BQ)b`w+OM^r!ZzY2%_C@!o|&gw->*BslmFrqm1!mz@$W?;j&id)w~VhI(!oc< z@Edq9-c7r55quU3!Z(H(pbX~#CKn`m>k0Nf1Op6~f=W#mX~;n}|C+u9xulXx0niFS zM9>bQqz90>v@=AoATfOz7W*!|wk5~{2-bVQ;BIbKOXDV6arm$v zZDtV-bku!4j*zSox&m`g1^JfIXzD+*;a%k-$-Epm9h;JOxiDUz-ldafm)O}h;G*Fx zx~nMuZja0U@Y=x6(zw8`tB_XWLe8(bw&R*ZK5gs8QO_G^?^jC``^))sUx9f^E^uGb z$EQ}98!;n5x(gBZA*N=?e)WQ`I~+FST(HZ1s&;y6%rpi?GQ`#kQ@8`FgqOg+to=+3 z7v!RCP!6a9?bemTIOmAB-t)4$L-T_jHBa?&+uAr9x2CuW&S|ID@@`uw=wZQ+!6TRU z;E{`Vj!63>9V4^uq}`_tyFfx_LH9U*+=7yw(yKley>ggv=4#XZ8OC_7;RN#v6X@>y zC46oc!AF<>NM~Q`uUh7&=eX>(!0&WQ_{bPiz1Fnbpw2g+VF5NSe!Iu}HmzFCamRzZ%IO5mPa zHv5x;XaIh$2K?WAs(+V@nr18&IThn$^+DsO-(iBx%TJ*1X_U9}CtTTSk5_Lvbo+2YvZfU*8$#?P+VQj;C3C@g>C&r&3j84zgrP0W@>TV!Z0-Bj zcS*jNb!^;kXbUHnuvBdkX0DCpeP<_mD6~sG83`0 zvNNYKE!$3(dLR8MNsZU!`^EqdC@5WkJD!@YY`?H!H#7b4$R386X?a(6<`P~S^Rn$q zf6xf+hxF!U%*p1D>!ZB>q?DmeKUTrG{@Lxv#}aZex|q4|17uN{h;6LWu5-XA7*g5* zH~6OCUHi1@Fs8sRdLio#b=b&!Qha;CIHHLTHa%E*FB~^|F7MOuNe9K1;(;quhE)r5Pw-P>z zS_oIrTgai7VJjt&iUUU0l{3rrG?KUY0Rfg~ZzNB%xn*+Pn?j%8oE3MYT8JZ#5Gxm6 zrPRw!BYT>Co6wzd^>EIM6?cB)tx4W0Tyn*KdvtPSHhfrlhku~I(1aIxjf5RuFa$iK z)u{?77AcL#{qJ%VeIMp^3@OPn7#k}5)b>4`gO!rG?#UKx=0i7Jbr@nNj z&&%cw`a2!BM<_kP@2=->jDo9wdxd8|f&!o8=XbbffzcMzMafrp&l*xGzcnvr$W)?9 zcYEn3?#s$?o`s;(1{~wbUCs-EDh<;xuIWGD?vnZLv)M1Fw%Q z59j5N7hp~@G`nxi$nC;**2%gGVn1=@x}Hk?&Ey*23p2?o@s8lZoS&s)MqaeYR+yla z$<&beqFrlHx7~$Te)RIfe}z%v;mumv;D+7&o%15S%Swt^#QXuSSOJT@MUCI>P?c*t z0vj!fi}Bv1mC$oTPFx)KX{6c|J%}c7lDPBv*>~LYcQu?uf$W6BxE+j* z$?1;{CwptA!{qL<@ zU;fdE_GN8Se6+~!Gv7lw0!I7Lop$n=u=9B<>c_z?qj{?raXO?Nzpd{M*xU#!eL3tK z-4SULTSoO`HLorI#th5H$Ae{erCITr?=+YVSckEDz6jyGkopwFMT97m{?({NWLAq) z$Cr+xq25m6w|B+r$F0m7HmMeP5>^C`PdE~WNO>$(Wu^fc^zo2Ob*ZFeg_S_GPcB*y zIpbms)^Kg8a*rC{Y2Xg$;?@nB)8?>y|BBMYy>TaZzG^E|p;&nP$e9Ul8toBLb{+Y? z8DV{9@!eaY4E59R8Xb11hsKxVRoN|S6E}B8+6Rq~G>4R|NIw$nbivara;(nM55~=R zkJ7IbDG1py|2O=gb@{KxZ4hv@x_{5DDIF}wizbvN}7* ze>%VQ1F-y<=?X8UnvLR$UM9g-2NQ`DJN+rqd`EcDawZp!pZ(#A<3@8b0Sf1r$zqeI z08@#%cYup!C-F9M)hAbGx5!5i$Al%Z{(JQ1v@tl5`V)ZVXZeUG zbyG9nlTbl>%aR_IEAedi%dsS#Em_!e4=y?4`6t)`IdLv{Zx(ksYqQ6s=tfyB_XZ)u zRkc4K)*BcS`slA>!BGCaa|BGXb|bUu-ziL0F4uqTs=rpl3R;-3zAG6(vokb}A1VtJ z$2)sI>do)IRQqhVv=i#jZSR^GhLq|Eh&|o8fU?vY>V}wIe)1)hL~2wpRLBqGF3e%6Y$nJ-JbVAtzRVIx&;&f)WYk1sn5h8|fpr-f0(_ zKSvk(G#e?Hqr7^8->!%xde_ET#g_h?;xB{UWOE{ahLh?Yl>N31%wu2o6a_%S_i)de zJ?gC+OWWjsN7wex(5;TK_Sj2G{`uFB9MY6{Q@$>7(cxm%w|>4$DB+Uw-k~?)I0KYq zq3vxcnx@p?Dz2suZ;cBR z%zckQp_m+R?Ve@(7Y(S8baf`>k1mq$?k$t*Fk6JAFZ!3+$Q=cFVYj`liIVh}n<&{c z;`nb0FEzkpyhk5_h8p|1ADS8F;8`2u z8`D(w#6je`^O#e0ufcM0J8qEx6M?cwTGWze2M#H^UzDFP3A^vbtUe{7*m4vF; zeIuzzbg|}@Y1p{}!x)Zh_${du$fjs-&|sm2pV3k=zr^tkE#}YE?duxS>PJ1e#!Tsa zP`2tAL{wTDk zOTPu*gjc_MN<1I><;vMrl8V?bEOjG+4Bd<3G!u!SBvt66rm#Q3+N!nXakF1tmctW#Scrg|^0z?C+!sDjJ`8wq1 z#p%$(yFDgzv{ZxfR%W~}#!miwt`&~oiAiDM_=nY1XZhVt=k|W9Tz7llE%9-NtVy<8x!Ag?#r~l!#x$eV*eJobJ}Xg8|2|RDQwHNgg=hF1S8RV< z$FgV0|92`&kv=_<=)N8F#MY1df6v+fq3h}}PV8%6b7eW3b=n{MiN6Y2E1H$^b}$#F zX|$bBYZa`I&SA^Q;@0`>-+D{zBcAR(p|eAxm1s|vf176F^~!VXJ-o7%*qh`F)d@dj z?PqTGkIo2Fa%OQMG3uSTZr{K+ri`pl#*AT7j`8CCaJ8FI&0j>8#-_}xCol!WVkdv_ z@Vrj%+`ak9FXiXf|9Eq|OgGfqrKO+T=m zX7OF4#W~Q&mvVp&F64_|N)rRt-_Yu%|CZA?huwr3R6}!~P?OoBZ%|jnaqg`-fV|8_ zqay*KV-%#%VnuN?;G;S%K6+HE7A_SgI+#mnQB2lTgs%uQy!so-Lgx?+6f_dk5d?wN zPd{t&KRFYxnT))qH0521;aV2O$R2*qgZGrhk~}Hny3~E^A1lX?WV3HtIeaZy&VJ0N zJF8}1G1O1iDD+Q~cyd}PVcpmM+p>rov;y$8P7d}L5;lFpkb8u2$n1@(EbA>&C}nvy zP;mN0DpyLF+6DdyztE793QJ*@TI2ibfM>li8sN!67(*B?y0`H*GZexEG}I+lYntiV zgO(FnmFH6=9Y*z*{tt`@2?cRGCO&yyx>_vzsm_n`lIAyEl8TN4M?%QNW1=vhu(k7%x4L3w z3%qA{6%*yp$MJpq_20uij26as7VY%Q?+UCrnR={h5Rm`c-(ICfpDWqoJP5-4tDnu0 z5yNa-nIbnG&KPadEe!5Kwf(emeMEh4y-LQYaS-j_8!acD(H#%!>6qy*yF9}Vg;SzY zB~*js7B#tdCO@eXiLAm#%!O^k>}m1o6~0dDUbq-sqcqX6^xPBNJ22|6%V)h{Liqrm z=edw<8=<_AjuDG=k4bAEPBxlODgL&frKVT@MNFlg%$DaI1W%rQ+);@O%+pO>TZkgs zRj=43oBJr?N<7`e^q(#T)HqLWRr^*xw)|ltd{Oh}(}#ne!Nq#$%M>U3*1WR8CG#p2Z*KZ5Msr zxDE=yD}!3>FKyTd$9gxZ`*GNDOd%Q2)Nyh{TGeEFjW4@s=6ZDByV!A&jwca+4|#_L zE`7BL>bKoSZ=1#JyKb_BqXuWn7WNk5^oE_X(c5gixA!TB$2a?Zy)OR07}-f(JtB$2 zQ?U1rGfu!ws3sL?alD5ww!3Siis`jd76*AD=j<>2WTdJSoEo><-il_592g7;U!mZ* zetlhE2-B{W7uA-{$8%+L#B1eA*xyI#++}G}?0ZAIT-Y_9B6*C5Qoc9ga($2) z(mtOf?fsmivzcE+X|~D5e{(z5sh2G5)K9NIF3em%py0K?{|s&}1p(%QTkQNm+bgjx zsTo#joT_lC5I#{zuQ*JbF?g2g=}(M5>G~JAqtEw))DFV-fZ1VQlIUa-=9H9OgN$`{ zyl^h|XqL=DywSb(s@uGN$)Pl!6XO#_Tly;%c>l%+!-bHOU&!(gTaqU4y?shOn=%T5 ze%<6S7975rlG`hBt&6mi&Dazof@by~)=%J4t()3+&*R@67|bU4D3J1dRj}wfc1JaR z#dnEur^YRumAf;X`Uah3`N zF49al!wM+%#}vj8KDU%z{7 zX23TenJWZcN%z_3SOS9d-%j8wL0$`0`O`qes}`F}Yne&X08zH9;oy8t4FL~bhf8^& z*89}6APs+op%2a^Xt@qTP5&%87#6bd#>R$pi>rfnk(jh(%sfo<%?8QD7k zp*+L855@^kad(w#o^2%sKGa}MvcF;|xLpmgV;%l7|7)E8Qd{6L!$5&$TkKh>agQjE z1)tGs*;bMI@G)$B#f$p=PRb&FF_V!65@T7g#Tp4rMs{@MxUGi_UI>VYVjH}it>cq_ zaZAN`a=1d*c#FPel%L;EAI5bvVlW?A9@yelMqXd;X0Uz2UG!wD%Tli`UH8@X%C-|w zts4&j%PIPQrzKjV9544z zHeS6?;!mG4YIJ7tBCR?bZRLmA`EtxDrR6zRS*5cQGC~~e0%=cqH)ks8Lh+t+7BpA=OHhmj)#G zurGgOquSf&WN1_x+S4d3g&0TCc95}1br?kD9raK%E}*&7(KAC*xafW5)=7AAnD21**``Sc7j}CZ5I@U-kXL~ zk)pxwwd)g27-7&;a(Fh?_t2QsNbSt>dhwHcH4rQ9JiMY;SBzWo{H68RbrJ{95yqyg zq#KwEE;J6GJfjAsxRNwS>h;Zdf{a3t^Z1(G!yJgKjrtqWJ(5Ez65*-(!iH(ct!Zf~ zmm-T@)ze{qPf7|3#nfxY$`j6>FMO(~E9^S3GoMx=AzE#FBeoWkNjlb~ls-eekJ<{D z78zqk#m6Va%}`YB^cjhtok|Kn=X3VBMtS1gQmojSBrvBSlK-mDSDH)@8r6D;?ea7> z7&Zh~-Zo-6m79C*<*on109Sfyfh=xyT?cZ%a=jzOpEr3tl?n8IaIfNQ|hO3=!Yy1`X>$& zjr}PYQ-(-}1h#aEt^6EAL&GZDc66j$C|wLTSaLg|>V+`kt`v3;-xTQ)2EGWwbOaO~ z5Y0f(3)L9-50MN4fEdF`2(-R-pvP3X&&wlNuxlj^%ZtsMm>&IBh6V;mXVwKEiv+*e zUKYG(XtL$JyfpTR0V*WVLLv#g&gXr{=|UjRWr0)dSguf&VVObak!Yo1L5fpe>h$I2z&)_sbW4^af3J#vf;JHa&>v|F$9R{vf zXcY=nDMagbjT@}=V9Z1y0-*HymaswqyKo)2D&+R6crj!IIdQj7qDtR4h=wrXAtD>{ zdQneN0pb!mM&;ny8ZQzhN&1=$T&Y};8PpGv8zu|h4c+~HdF!mTtR#JebpX;;zOzaS z&>sdHAX&=JoDyYFit<<=<3Qkt(21f80Buj_OkYnq02I_oL>WT6OTb}3eB_rB=y4E( zTphUs0hm`1x&+vMnSTd0s>~-}2)RydYn3;gl`!~Xje*~gwH9a-x`a~28 zTz3dX4Uudd&fjMS?x0RpFG5X5L0=mhq+eC1t2TQB<>vpC(L5;Ie=oMsBs#q^gb%T+^j1#AqwSsNDC1fH)3A17C8Ph?-cDd*?&oHkoIkK(X z92+oC0Y_*7aNO!JQvppp`cj8J_;|et^*}D@S8~HU&<`NSRnP+E12hV;UP1enC(vU6 z$Z$lk7Qxb?0i-cBGiS?NA3ZiM#>njJT<;&4T5`0rPAz89>xu}Jxt`yvL7lK!yG>^d zw+0Q*oj_h~=L${L`aol@wg*Kw?v<*W>4WXAsb%UbfL%}zK*Td0Ohri>vr` zQKUD`ATTAEt&J;|HLuODGfVXu{J8fzxx*qZurorf)dUQMotHt_3%pNTY0_^cQ{G zVh}cXYuKIU{`3!#wsNHM!z1lmKe<&@f*y?puhzR}PfeXllaW0%|FsN!W2pJ?2kioyOX!{bZl zPp}WlQY^1LfRt~Zc)Ex5!>2~Ylxv_}*W=RKKU$}t#|+m-e-vdW^np#DZ0luCO?5Sb z9S2+#A>vE}3X3o_=kj7+g>mXcOiWOqN4#SuFka(8qlZFbDtWJow_+(?3IN?B$_D68r619)=cAQvPkDT{OBa%_O=I zoE;+;(fJ1|Q;nLFnvF9;=H579WxX>`59Doh6w>SYa>NKgbSv+lhoBVA%yuJC?qO(c zutmqGrbYmE6LAP5c0NEJVJ?+(M^_nWsjV)1OgJ8W)`LIM8<=e+u? zt=u9ds(l|aizF88^Uqjl@_mp=^yQ&AFg%gI;=o)|0?%ZX%P5;0K-j@Mrw*$TO8tc3 zWCOoGbr3;}I*=q+QcQP1zJsTp4Lk_MEr@D^vv70kmPm$XihkJ&HYM~n{C?lhX*i{{ zDLB5UtazZ?Za_zcgmUPeglO2IK-pKtxS|-BU;FbGlomY=ybHR=z)YNmu2u!{mf}Em z0llx#p(ivJ2?K;3($U&?Tx>_GG7y06fb`4-Wfw%Ih$jw^pqLrj5ui$$k-H4DC9^pT zH)ay&fdvzPT4M{YyUOY(+nq8_;_jqPRT9b%yE9ZXCA`x80p?HLipcYCDJK&nMi3k2 z;%nTM(7p(Pl>$%+q1%QJpPn}0aRTa#l0Yg5>5fGMUE5(X-%N7P1?&@|^nqplWbXdy z(Jo+Yz-0XRA9e*$UH`}mVbvd7TW3W?{DiPtY?6a(N+a&~Mg+2hz}hH=sOZj(4oByB zeJ1|UVqv0gi;fhTZtS#?v6h?6jA7e{Js7Cit|G&U@yA_R``%z}8i9Vi0M|ieY6x9+ zcYyCTi--srV7LUw!FGi-okl|;^tidp&~-r;;1CGPwb+QaY#3B;B=q!88yX~mDD~-0 zd!KK$dS*kj2IZTqQ@IouhONi=hc%uYS08NzW1PxT4pBX2nT^1h>hnc^@av5LQF-U? zdiu0@Pt=&v-(Ba+Lo5^+8an{O>IqyU21yUpe_CtQ57gImL*XAhmxw>9B z(25OBdaC3t#^9trT7ySUP-~86U+y9WrvAfB{^-jF120^R!`Uk#aY*{TK>3OKR;IB2L zzF#4^X)?~-{-xC5mxoQ))xx;0@oJaQVr~Pk+J>xbVw8*dII*kR#NhpIP`!lV)S^Ga z^-R9-BM-Udh!F8#0-bF)Sby7SI)N=tN^-6Gv6Q$tCA1u?I-zA&RzNNbsZVnr`rKgG zww!FWXel+FwAQ8`CHVg5)|QQ26blpe`IlM(*XhZMX(E1<3+=j4v%>TE6)`DZ(G)8M z(ViXXqh$xC@XI-cFFX?D0L~{gUIp03U6{{p0Hp^)rC?na1ld4ZoEt{??LZD7V3t2k zU`OW*nokbzoa%PkEUVl6$jW)8@6p|_(vShw*FOy!O#Bh$#`WX0WOd6~N-thBx?tZs z#mknK02^sNfsl&98IK&(FhN^Co*`;C2!PBvT&LE3(iQH@BW-rU&TEJLQ@Mh(*ZbnbCv0rb>oUxQI>D)TT-yV)9{dLXA_smV0W(Q6< zTY!jts<{zYDF^!-!rn!Fg=XJ;<5d9bB>;Q`Y&o5SgJqw50rjQ~EzV&_<#Ka~wgUjQ zKlC(0Ljm10Fw)&Y$7%a{HB0vLcZ2XWVsuNUa_8{j+FSjNS6&tygcg&;K4mc5%z)4Hc!dgw`3HQ`#NAPQLHIdX?`ZTAV*FGQAK!1Bc zcjE#y3rxXz@UP26DJ2|;p3oF-e|w%8^pn)nKtyH*$J3=tmtdPU={*L}K3ekD4iE}S zz(bFq`JhubGMFkbR)|j>=siG{qz9TUAO%c5pS%=%@62h$_se*2R5t(nx55QdnO!2Q zB1oX=lV&y!#;9B*%T*J&zlyhnE>^sS|0dKsWhdY;a&H0FH3x&4d-E=KD>M>CB4-4B z4*T9NAfj-ojo*WZ$xfK2+?hZlhAtc)7GXXj0A{!|Ma%8{0fBxqaw4wp$djzyvUx2$Za1aRjaB&G29$zqRLy z4XOD`Ch|8<>*I(@#~M`vx=2qQWk|aU3Sck(jh5!pbf=g&c*1B7DyR8((uq-DeFK@Q z_YRQk)!$(&B>^1iV^EE=z!;gj#y$r603P@lX#gNS@e5|0J*X@H0b!_sbGER)em&3Y zNsI=Z4uuBS^LqU*_ueswWRF~xh~AVmYy1^dPs2(I7%RWJCE(;6c#HQNtiRCrpD6co zjjPjt_$hb+4LLL}-W+LoV)Edg*{qrZF^w<^1DgOkp+uOJ2~5VWgErjx!?SRJ50$Zr zLrMV&suUF!Gcr5jlNq)`xR=Qtb!${a1O;Uz6bgz(YDDE;i+7H%^QNRq z;~qi=9#;uFP z*<$6G9Vb87l2qr9c(FPWe>R+gWDdR!v2)qhuOlHxQfp*Jni#M6|ceyPCY%@;FEMnZavYko)be#N5zEC#%AE1RKh-Z$_ z^QK8Dp0m#DW2y_2plJB}O%l=ySNiae;k2c~}%IOfF2rS6p{Z7)Q z;&n_UcSAGZA-d3uIwvWfn|`Z2hffAE+1VMYoXwRq?b=rmhKq zJr#E`zm|PwJOVG@Uz zT(gmv7*8J4( z!IWGkW0qP-=;yXT#MrGm@W|K%E?)>lY3;yr&n`pzn@LUN4Yd>jI(?krs(nEpTQsCf zw6fB1=H@~;tF1R^p3L`BQo7gNOA59i@BE7?_>LenK?6pF8hQR6v@fZo zWUAL$XG+{uu>;s%jOa7@DfGT7n>V=#!?3L5!8+?nrEhtsR)E7ph_VdI7l~QixJpJ$ z1e~I}l+wYSXCEW+_I#>UE1qf!s+@H&P)`|%^$iudgjau!J^pOv(c3Bofx0w1b@*Jx zWkO!9N5hY>;^C?ZexXLbG|h5eO7cmB=N0Ge`%uZu_i8eMnyoP!Ae_-q+1Xp7N77lg zQEaX{MCArh$~Gz{s}m56l0C)L#}?%LhV%QixdxVrONq5#*lX-T8joW^wZK#?zmLOP zp)$Wd(862ie3D9>syw$NC_ zsZ;;2;!{u@YpMw*;e~r3b*5%rDOa3)G2K~CplH^{UYsdu z{%_~*8~T<>84|L!ur}&K7<%KK-QB&7G)nn243wK}+M4QB5Yn++?l3p3RW6AnSN^$D z6P4&Gakin`U@MCWgGRsw;#!maGD>2j#soB9db?K|VV(WisOsC^|taLqn z@HhWwU8jzJKM^x$?R{^r9PfcuLmFWSmL=4>iUtfa*5-@pC`9hyemwS$63)0u%c?|i z0qSTSP~|M9|IRcFc*+l^F~T;2wCkV4_)GoGv1fG@n#i{%C89o?zgYQ>LV-kzHDxSu zt$IZ2GG++VtVqXg&#O=rM8e&M%e`v)s)wJh-pagQ|ncI?9bX~ffQaY zm<-~$gH&S-PH|)(&Kq?>HR$oIjhp0^EPoavKBK>`ya4K^ukboL-bi_=7_BFJNhZ^J zHf|K;-@5XtcW%E%X5r2~?>)$}5Dz}+Nu?mL5W0w*Djv6WzYK-7cfH!l)hu%A9q@g7 zwa)P7|K5GySJ_51w3K;U5X`$R@aMlPLLY74x+Hqsur?>GZKmFEpiK6@HaebV!%Ol@ zVtKuxTP1`RmOdo3b{bHqQap`DJ-p=b-+C5oNjw+zGIk9LLmvwnPRA>y&B+AK)5BR+Pe;%UyuStHk(V4T<6jtsd;}Cz&+A{U0-{mnW)}p3Pbqc z>H}mhs;#rL6RMZep5Z@A(Q@yk2VFF|Hyc?-YM{m#TDfxC!oTd&sM^Bl5XyaSECs19 z!U>d_|F7W8gW%D9ws8bzq@`IA#FgQrS`Iy&E?{)r?vz4 zjRm`i6sp+cE4%-Fb(l_u>;yR`r;h|)J)f45Vono^2H$;Wj z+(K6N0>8fH{cw~?!4zOEN8ia}pfLKaA~ife2($e4`ZjXUhiaEk*`VCTByS(Fwu_N$ zk!g>8UTG1P?~SQBHZV1$epkI}$?OLQRD{gIhsLJVwFBqnwm@zQ5gu#`bvsHLZ6ln= zEKe%WOV#F+HjP4Hdjzk8CHIAjqR>lt!rVlV%tJtZr_%Zg?*?ox|JJl4x?uB$f%IXD zLizmlXg5mMW}ExJTFHS}IK{?iBSi5elJG#C@P6P>avIneLH?7EODf)jkMa@iD-+St<5gt9pLH_Dtw?Ak-4 z3+7%G5Gq*xn}KY?J@J{DV06GOnfF;akhZ?yWXC(e*%cuug)JY}z zL`oFZ+pfGv#&70sV(;cH6v)zI5Lp*-^9TL!MbJ2OEZY>vJt1pQKo?*;4euX!kPwZ} zAC%S?(w3B@J8Vtf=iL94*F@^eGb-}v zig7y=VTxr0T1?{;X$X5{1UaPsev+?E6%9cPU6_=9;y}zNoEg6q2D4utTxT118Vu(z zI^RmLvPHFVm~b*Nfnt;ESvbbm(uq5gTHziz=qO(sU8*lhoBKOHWj978f?mgOI{xqL z8yi!1RDjmTrMb73No8E)s?E9+_Z~-|`}Em367Q%d(VpP{wE11ULVHy2qdAJc?SS?)BQsfVN3vwLg_==H@{Os1(CkEHCUX+aQvWO0PK(|Zx3K0}C{iioOx_AQ$Nb&u=ev+$@3}!C}JuH{zB2#S)9>v}XLt}zE@zwkR zZ;%8^nO9UB)xTxE)^{aEjvwq8jQCp0H@?x6q@`O8Fc}m7ZQ&~^lQ4Jb%oV*R4b zcFK2^?&6D88IaX_hyX~!lHZbJydgrUBNzg$;ypHq4-16x23?#b1KH>PdoIU=CLHL` zisHRb>ynhQpuNCv9~q~)qf1U4hs664mg%oS)F}`PyNfZsXX`_z0~hRFeNt0a=_?$0 zw>_xOHe%tOWb)Q#K_0`{aphdTLh+q?Du)66@VJO_zH5z*(d~4%B8?&~#;2RADT}xN zleM)y{u~e#By&AcW63oUn~ss1cC^$4#o_UqC4%Cc7wUn|YRN@0{Wqlac2?i9W%S7o zbF~-;#bs5A`03$x-Ne2%CPDW~saVlW==}wG*KnBdV6a0tbpBm+h7!CXKl^Iqa6>Hz zV!Zo(tGg5a#vbT4&SO*cUR+xgkK5|u=%ZAJj7RZSZ}>4&OG0e3G8e0okvLfm9*JKL z{hB!5p;|FPAc%H5=-Ydrky&7dni6oF%1 zUwoAQLny|ehJeP;j(M!z9av#5`iT4wYV3ZZYromR@`{*Wlnu!0`T6^oZg89_jga*tO=(R^I_O zoNZk|)2QVTFQtjAl(W~}NzpJ}rq=ZAS|5*}OE^oc<4gcgV_2#&vI^b?Sz*eLUia`= zu_6Ki;;zN);f_iFsBp8_aO7LMMOW4nmj_pk^nERBmmmC?c;F>zL}7n}h1PBo$Y;K$ zvxV-uv(*&uZfcO4QEC!bZTh6Rapk*K|4jVkTRKa0-!z{XqHXX&BF+<5uE!Q+ORJ^< zT3lR?%k0Jl7%0#h5N9Os*%)60OI&WUx58(+G{ZI;n(m)a4ueq{}MFg_~YLB zrwzwcpTxPWx!K{L%XS$pGTy$0k){)Op2W7pZMLl;m-jb`IRwlUf>&O{#dY3L(V=7Pq>I&P%l_LexT zlDmq}?{Smj3FlJ2x<6r7yI<=-pK291T$r^eMKh)_j-t{Vv2p}=Cb`AlzGckP$9MTW zJ29~_s$udgn_J`fgF7s*`nss}xqDWWwW*R_Z`(R~&=XHz%qLBV#9SjUKfBkT#6y~3 zU~B*M=n==!4Xtjn3LSqhGzwlqRB0B$L&Y{*j{vGv%R)OYCGO;x%0B9(276j)D7jEe zm!wGbFgqFoeqx+bF-&TQ_3fubD#thI+7@vG0;cypy|dMppM+*BRdCjJdC|7r3<{Ed zHtCfsCJ?J;^5TYf+D8Mr3tQWR2|?@P(QfAd8x_kC)i+6b3(lFDdDBs2U6P(Tu3Rb| z6S}h^)H;m+ef(R^v!S6O&bsAwmawmyjW{)A|3}q(hjaOc@8b_4dy|<__Q;-vWF>oN zkIG(=kjmbo?3KMTA|pZ>A!P3t$&NxsiHv^N^ZtB(-|z7~4u8ClGyzOL&$ zuk$1t9vKSgx|~D*>FD`Y<(TJs8RY8q;Q|Y8Tr_#cME(U+)5Oj{dIe9<{-2P1YnYNaiL;wROXseGnd!e*H5<&EgvWc7#68>g zxf(Z9dCrCZ9?DnwdP$otbOFZSrC`z@H9-L<0q8Dv_Lpy4R3xc$fYc`H^9Ozr1pw*! z0vgN6@y5i+2oI@#BQk_JParK)NbB(R#uRcvO<9r3>`wY&Glq4#le8wit1&StzC(nA z$$?j>h`>XRlL)s6gTW3leKh=)?%aYCmvWA^WY(j!u%r_`rDEFrRcoZIRTp>Tbh0mdoK>;-)5*>@hM(#YpRm8!!>+Jtm%^3QB~Kp$ z)5h6!3_oGl!qq|E?pEq8y(%3R)kB%FuW8*Xyp*;|kFIRKPmi^2BD`Q%>zEfnHz_X} zg&2Xf5o(PI*52P;|A96Fsshk*dN5nRjmq(+$vof%wT(76S&#@qX-DAC$qdb3r-Zno zB20bnNbc*mIt4Ky$(U`nRa`Whc?#P1y4OhxATaUA*>^`|CRRwPe z1ZrHu>wkZ2+7m1}{#iJT(0=|Z{{w6>Pz72Ew>kWxH~GUAU{Qp&u>qHuI{XV_{ru*Q z)!o8(@80#tssQ(!7NDu1_TJ5|p!|6V!mg;LPArT38PC@^TCWchZz=bu{~EWJ`G|0z zD$aSZNiZTFXEwAX*18!u$3H#lsamass5>!^=S&cf1vg$2>-OrinWa>6O2z4g3DB)T zHIq+H0s{V>{;{URL(x2F&^pbb3iU4^ZA)ME-9b<-TA0%Zy_Zmch?qDyIMT!%DZv(p z@s9n?n*@MLPc~dX7N)t-uZ*RjoR+t7L2u;QFYIqE`rIk%y6KcXeP#->Gm#@7IJR=_ zxK+X=>{2N%#(mDcqH zWu*Wp&Z556sF^paQ44a+K|rfo+?=UJ1%qWR%M5J6xCmV>luZMVjUu->j}Pe8Z`~61 z+q(m-gV)!8_RhVId+9@1jOk7IH8G}DH!RY%#};b+b&DF<e^Vw=$ z`*eqkGeac&*AQ(^(1R_ExOuBT4XT2IqVYb*p!2i1vJwi8b~K>6%mLz9##?eIgaOTM z8&C*7+;09ky#-i8NNel0wV|ANz>V=>Po3(|-ZnP}?buq;5A#0boTL1qEroY@-UmV28a3G=~ll__@9H1I!!1@)P8*fc{ru z8vh44G)wT@bl~(2wxc+CI?bDJVNFrFVpP!wm81sg=z#Bk?rcx3DQYMq<=WL|B$s$d zisDkwnY(g);?Z5`TV^U9XNU|PPu1a1HH$h@`<#r0{H^o2A*#Cn*+BgKWp*sQ`h&Lj zvpn1{$~}%ewGwk�#G`93qpzL_%`=#D0St-V$hFcL8T)XJ_#S_yfHerBlQQvI1_I zK{T2CN%iPH05Ls)i2fd3&bb2^15(H1O-f2y{pL-u34b{3j=u+6L?B8pF)+yVLuyW& zPwBlzGgF>@0v`NklRUF>&SY&frwL`S0so?^?i#Q;6_nd2zJt;pE*nlzP>@cULF4^ZLcoV`!)=7X zdnw;4KIOA|!W0jAbmr#PHrv}@FGW`C;qov(((sZzlPX1Db*y)XneMw!35z61!@cSg;|T;D8PMzFUQ4!r!U#MX zx*J*A`g%vAJ&dRN%4Uc=S;093P-A6NQ`2Ou3J?iF6{_bNJ^e*fWSE$lQ6)DeRnU_$I5W>`YG4OU42cLu8$VIz@UYsLx3E`F`l3QzqQyF7jo-tYSD>$*7Q z?AbIftjnKL!7e@6+g^ zN9vu8vt%9+f_obn#hVlfi))_2-$e!lR?#2MUJ&0h_GsQc4#_--dj-U%G8i+7e1zY)-d3)kP~>`Z_Nc+Nyl z4#Rj1R&4ojN%{{_C?=$rx%OKcdJrP8F6R`?j`L)&&z;+a?Fe=e287lWimym4u>uWv zadQQ#2!V@H82+$Rdr-v~ry8$rPbiN=Gh%zd0_!6-Qq={SQP+LxfGcsWy^OOnjM&%5 zXwG%Bw@uIrT-(|aDxiCtxcA)fazu0M`1rkaVy}ZCLkeJy09;L3qMf4xu7}9NKk%ss zGRkWPtJIX@*4_B1D6CE)luZdhU&Na?A_2foL4CVGiaQ8B;~?e*&PE`oq*;FqZm1n_ zq!M1d(7TJ27x#iw=Yb7`#(R%l2jxEBVIgx+K4;RM`)HrA^653X3tqz=p_4k?;mdo= zOe;waPgx@Bb^{CIj&P3#NYINHuW2bDwqbVTv`(>Z-|gMq2gDmW0#7QZfa%FfO;`i= zkzi98Lv!Qi&70^0l3u)ukL>)akP)dg`*7P3FO}57+mN+1KxV?}LEOQH;H#$S>Uoo`>YB|x3c~)R`j%itVR@05J`d6B$HxpyHW3m789oA>b|R$@ z!tiBj)7=zrX~ak$fBSw2K8-+DVMd9h;BeoK>KYcPl0ifmKYRy7+>W6kGVrsmG%&6M zzAc#Qoj23Eo@!y4=rj7@T7|mogv1dSWz^@t7)bBfD-|BOLE06Te7NBhsQdkVrw~5T zuplEpw;`lpHBkcjIt(oJ;nh!OwRk&nC%qrU*3f;Do`P(3Txj%-a2bki{~K*j16FH@qn0dydbUhiyhdU}fGpub&?MkpgAqfHY*n8*Onyz&vbWc98F zUdgw&m9{^hkiUD6DfK>hsHmilg^$&dNchWC2~v!ugozpK7Jevk4<&7g0SgoD)2Y`+N8C}Qjau2Wpc`I($7GtJ<1M?3&f+ysD z!pxOId6f4mX7E9Tebc{G%idaL`WNx6Daum5$o|%U04+A<9{z+P&QOetn?<9KehliG=c!j)KD%5 z44n!KFN3%4aoMHVckK3o5({?)xhEL1__gEflJQ3>r3JzrHDr#Y7ytAVAc3+s{lo&5 zYor9bjq1{VehGf^J}wH1DjjFC&6bxqP3j?DWVJ#S*{Nj%A~I{=bACjd@OOhLXe>y+ zErTUG2`075@fqB$U>0NT?rv4-1AcCePflgPf@!+#Lh3t>Gtb)Q&s!L#W~=8DQVEkE zM0lA`d7Vb*Zua$rRz2aY3s-au5R`j0tkqiTBO((r>&q@uPmXw;Jdr6rAOyuF&bDDp zFQlVJaIb=Cw-O}iV9sn&)+l>HRaG@KG!&c}Tl#+e1r>X21Z@7!#t%GxTs1IMjd?v6 zIdtQCnZz{@YiBYwWw8!-^tEf^-MQ8>Ka&!6k&`QW2pc`R!B0UjES`Y(DQ>_+tL^(v zYjIQTu=%N%;m403tFLnkK6!#+^tE{RZwxiI11e?*FrdIcgd86S)wn|~Bq=C#9Zz7t zuz_$YJVwd(=$)!5&dOZsKYDeNl3S0N7Jg_rM=&IkzSkQ5joqYNPt}FpM6OTK_Wgne zMA>^dsM%q*&WamMHN+2Ie0%NAcz~;>U zlWTHsd#8Acu`2LTXFs4SgXtbVql8OfbF(Zdlax-`u{G2cD2GJUez1BLrTNu4O%Ogh z*u3uGz%6&Oa{~qo&ZD6e++`t6HMOHPeq37igY>IZ4}tY>qCB3R?rDP5@7j(Gq;k;ELu6%lz0rJsiF=_4Mk6Ry$zwC}@o$Mz9c8D61G=iT!cm{^BJUyX6 zol(I8d}lF&rDJ4-0;PP;JEP^t2~aOL|6RItX%|v?8_)|wHTFFA*ECOnF@-v`KwW}m zY-|jLTmthyjPq+YNY8+tbOW9p=<8u<4FyCPWw00a|Fy1bM-cJXT5D>}MI+4TyLRT+ zw9seowUD~5s!Mu=8qUOFqmky3z4tPmNgS7t#_6k)N+O`^;ZmfzspA+qe|lg4Bl+1( zOElh94)9p*d2_cPaZsN7clJjf9XIgsD$g=%@>m4Vj5-3;T1M8+!PPYmGE|f^iF)6H z9-tL6EO;g0XX~GxLD`>TpXA)`ylEJe4PYx*dv@%Ip+5APC-taJ%$UwEzG*Mdbr7+u zr6+e{zOF&kB_cJ{)XO;h;+}6tG@I+ynbnYg%7q2axiOFm3a+i^l-ofz0`)FBo&u?I zmm3*Si6PmW?)ZUtfF7qrZZZfx!BU|lInYV%0h)0ruZ2-IFQq6e43;lEN?NHM7R)0ZOM7J9B}oZ{{THB|C?IbCOWS3-t>6oGD2LPa9#ts|+>y;?k%`NsqwUpn!k3 zedCw&IeDUyoR`K+6i-x&X_uOwY5q9>!ZtnWlVM|e?!dwB!qF%6mtkzh$LB`u9+dB6s~Dg8T?(m|ivPTR(;s z^Wv+?_i^}2E*TlBd5;a}HU963DJ|+~4h8)7qdlZa>9WEfPKm{jT>iLC9la{C0qKglpivpg=bk;T`lCnWJX>!rVh zedCPkhyqL9F3v71J6qS>+Uzd@1wMdw6iKW4{L(g{c zfSib)dmDE2P`pU%7;G3w+p4=$b$H}6(6V&o;W zby$Z+ibf<0i4tnKpO<@-waBKWhS11(y<60cje-9%ci}+lfQp(R+eO((Zl=4^NCrKO z`*?M)KE1it+56s=H!bn0n^W3YJ#^$gh7coxYHil|*(aj(l+ObNR?ei#=|sZ(a1?lw zGIB9dR>TVxI!lueTzUWh=4lZoJp{j*E|Vn8)GWajdZVVS!=XaNA*8EiJ53zuKPxgU zpi4_g_seb3$<~?soT=gXh*bYZ?xcYb*T1=Op%Q1R%-gYrb9;lYt_!+cu!(jYcH|<# z+AbRA=Z`+}3a7ffaY3WMIgO#pN9MxVFTy6q|7qJ+H#Fb{s&7>uecC;y-?ZCKO|)f@ zIUF`6rS4~I%_WA zp2CGsi?R4$T01)vA>2GX!ZjS?1tSK9ZL{j3b#f$R_=4IY=L4rdvs0NW=80wX$McB< z_>XMzS+bG~`En}BW}4;e)Z!jw60R-fWU{Y$yFODf!l{O4n@Ao9HR7=0*dzCBpyJ&; zW0f4$y}UB( zCQ*(-3i@N+weDM&*E@uH5*2xt=6aBESZYc8~A$d4rx+&HA(n!5QcJn zJBOJ1Va5huU1j~4=)@Jh+D7thc$DZQD0AI&n{7fX4wuAw7so4I`6{#F?&_mr4hyS+ z^TBgl)*O>hl7EhIgi?!!lS5ahj)c9UOEDBzDs!4W`q}&T?P1OJmr-t}?4Qj~y!1bs zR}D0!@vr+=TpfFXw{{d5l1zuS8uYQWj=fa8j)1f)py?COf|ha5PDC|gKb`V0gs$5mTci>Wtp1FTbF7@%8z;IV+iq5NGv&2nP z^JfGC!h&lHIhnj`97 z_d8_UF9!MI%Y2-O9pIa%b@DIqXg!(?s$Ac@F1PqYPk5FxN7J5yXRS95M+@5W7IL2? z{XIf=lY5%PW9#@ox64Z!eqO|&4J2JZ&sBTHIW}6oje!d%=DeJ?RHZ)oX*^TQd<^B# zKRg?`RJxR5UW6&r&9zM8l9F;5OCusYob5|xY-1Sj9)lNoc{x-xhAw&DdsLq%rEBYB z0D`$&lYNq<;Y3s(?Nk)qY-D7WrC^@<`fj14&MWaob4&A?38P?A zxo1$BbQEdUM!U%vgzA2#NLLJtSk3uj^zxwkc_bY7)tgL}T~K`TspXFnJ)ctgZoT_O zUh*!y{kn{{F!6AX#D_(i%06dnECsRblY%s!#a;{&)N1fCg*tENd{Cf`=yHZ0Gyngh zB~?|J-!&khQ7m4_(2=;WYyJ}ZtL!uhHWsw?xy`Z7UD@i7e>CjDbE^7Yh9Km6wuXSv$oJ>`hMQXMl!w&-tf7P9-wI{m1-+Q;iC zl1>4ktF}7U^$FGYA8J2`d4}{~OK#uncrmpHBtDfk zm%FGV!>}8ee-%{NLJO11Oet?XerF}%!1KHnSK`d|vUImB852qAep-I|_s8FgiNC%c zC{m@tlxYjOx(=zDCSAD%wJ>|Ri5tY`q`#h$O}2hUrB|{~migwE{z{hmI0=abc2Na9 zevE~H(=ca4q0=)jQ-wbykf8CyjqsMJX@w|h=#yd*HonU!jdyP(2_ICjmzvXCOZ9dKo1Jf}R%#9i*cB#at%o{5#?vAscu_kTV05M}YhvHx0 zY>fm$IzVNnm)=5&%dSs^g>1DjFD&1&=;z}uX8rUf<#CqM>~;Om#1^L%&jYXD8YICD zvud2Vxo9>2TP%3&-Rz6S+NW;ccaAU)VmPU`_4FpBM(mjRmnC%z)c3UuF;TYnJQyYG z_kBuX36tP<_2wQ*xYc__`QFhR?OWY&yz#1igR6#b`}p_>T#6e??x87r`tC0m^;uqX z8BSe4J`c55o<^C9OYwmy_k|K4Lut2{NiD8X_^1pYnb4x%XP0SJlpTR9r6kcx z8E-QC|NMg&7nkxY8yy?;LCiavW~+a3{)9)HMj7IB4yr9tQ}GhYok+iGoV&FwC3{a26>EG`y2Ep}L3 zu5Thw#0@i@U9eWT^(y;NYj$sGDsI=Vjt`06cam5u+wUxUK`>D=s$r7mAVe_K zzTtVwY6Do?F7EpH*1H4p`n(J9vVWy6Agt6v>lfE;A%fJ81nb`r<4lxj&)Z=kvETbN zHN$XVWD8-bjHpuj*0q9CAAcvfzEpQyX@%K~5Rot+N40p?3;uSFcy&&;lF8Kj5yfqp zTytMDidj1j*K&AhcD67p)crH8YLDE{Pp4eKFWE_*wAB1j{KEF^@=BkM-jjHhA6DsP zbGFBti*k_ah+}PT*3F1^b)BT0HqoZ`%WcnR4YFpOtGv3E7bANA2ca%^K-i;N1^tDA z^V=j=-r=kNCU^dOo7wuNi`w;xO^X|>VYP7*Az`v*ig`OUt*aqofp;au9@d5#6q0-p z3$t$e-|b>=@A_x}e88#9^1~S0zF${PyKT)!)BaVtd0bSYmX0BA^<vl!wtK#9t>;B z5Y;3?Wp*QPocOJ^;Ayt-mEgYD##&L>xJBwzW_BirD>*!WVOh<2UO(*;Qjj$L*XQEU zm9F5YAWNa=`|PF9UM{9LYuM33F)-A;-;KeB`@j1qSo;AJ+^4#ijj@Iah2WP076v~K z+ZIyG-N0`z*c&R1PZY1<05o2yj7;DfpVIz0BC9u_x&CR4L}*VHS8GI-?k%V?UO6v? z8Sdy?Q{)?bvu^S#oBOKjA9@->I-FrnvwOI27ilN7w9lhwQEtMsks_x@Zd=`+e1LFU zgx|D3sKXu_GA_9-O_{C&1v~)-uA-_3o=IUbLUKp;(Q*Or088Fe=ri7aNaQV1DV*d# z(pSDz{KAQgFo!Lal}Ed|qob#nL5~8e8?MIRmIAFf=PH9HaH{Jk=G&5!(&Vyt?kOrG zG0{T%ia#Ts7q{#C5F|9M*tvgRVVz%|SetFZ02O64~;K!-heZG-F1ASp65MU3u)Yv*8MaI0G z_>AvS6I9sJsqBqMV?%w`(Rwo%j##|J023>>m4(}exLxc<4gni#OUzbBplY#D0F$=W zh|H0_7PQxQylU`}k9Vj?IHjoPVYgH|>@(JOhK%9#jHZ$^^vN@F1pWqd4s;?CDru8O z54PrOVCUks_W3$g{kvHwsC!p|%hM7C{5j}VD@01X(@qu9IaD+E=^{}S+RC>} z!Fh*a6>P@2t4{Lf*2+%xTao&kc89l^Ln=+rk%yP>cp zm^6!`ujO8+?{`3>0P%2iz9j#SiqtoMJ??a(YU;qi@h(>Vys~$`XRvWRLARX#{ofn6cAqEhJ;9qeAAcOlc9HtK0XixGNSG z51V80n{8a-oSsr4wL;OPg3^yRJzw-DJ`&+TSOLgvoGf|CB;P8E{ zY@lE?;9kHlOyiofR=s7~+3vDPn-KSAVRFBFZ^*WWPQOGe42Ew}zY(DAAO=?^a9HXR zO>azBeZUfws7{m^Fs({a;F5RUni47M&VM?_y>`>(UYHNzZtfq)HTA%9fscx=LR@{p zvJ_}Y5(xZ_4sWBp7%&V19BpB3E!^1B4;RtV(+dWbi*v|y23a^j2?dRfjh0(w1Z-+I zZ(g^wyomZWf%FD=+k*=+YD)!sDqN+&vD#HJFX6EZFU3JZBF#u~`VHWA1eDZ%pgKO3 zC=rqm>p1`7{;ipwcUnHw`2X6|}xU!`==-fEg(g(>2iwIvxV|a!Kio>5%%Uz-% zn7$htUwezySbXln9>t9#`vTl*aJUBE2nro6pmVq2Pl8%SBa!j(J+OU1BSk7SER25T zP*+zMmDsShw?7X+9;oHo!Q3@;tLxt%00b8L`t=oPJT83yP6(tDG52{U@I~L*-@lP6 zXn~3^NKUJ4E4#%8PLJ&bQsI*3-KMywc|q&^P2LXffJqxj26j#o zFTYhNJx0Fy-Mb}-j)!2SQ*GBnYW_YTU9SxRew8A^tUVr+4}nfg0T+yYw)!xID-{0Fs9m4}rFUciaU za@;5z$klg_oG+~6fZWC97ZZ8_jX_SpLKgN8;A8!)^+cdHH4NrEfP&}2EVjqPhdpqU z>~6jN$_~C&AV=dI_xdN;mLP3#aKTUvSQBBu(gC7Lr_g`iQSleQtovjJe;OyG=-#@c zDlFe2#lFc2*Z9ll9|^3FWXv*$Db;W6K4|EU+xwc96hI%m@g^Ma5;?xm{gqGZvtUvb z1$+7M_;~73PER~b4_L6lZ2jWqCh^m!PeDTkrSqXie_ADTPT*SY?EW~@5U986?O<9s$w51Y_3U{|^E-McrxnO=Sb zM+NJiGeC)S0sIEy+IhR~SS$oxICJ7fh*uHm#WSh!< zVFx)ztIYFVQRJl{bnsLV?dfcf7Etk*;3_!5JMMnIf24@_G)zQpJY;Dzn3lON8>n6J(IL z9yUS4%&R-IAbXUkPs1S!O<5$OOTZ@qG#to)Na7INX()358<2vU`Xk0V*I)`H)z+f zk%V%w$c4JYVjYc}?=_;>n-2a|H8BDxxb2}#_t0nhg;2^IE37h+1snT z?I*K^<1l1rcV%C+-Upjkq?)NhNvvQTdwsHnJ>=BXyox{|Plp#SSvIrG*6rmxnp{N# zOztJoZNBPq6I#F9x#>-S2LV6#n|5XMC=_!?96(A;*VL5I*}1ZM*g?#Ni^XgGRA?qHvp?(OXv*fexquJcat#R`0xdkTwj4;r}7bF)=YM0DTdp_Tb(= z5TAQ>+Re_}ivP~E!qpM$VNC9$T%v<{bu6^8s64e7NE>FWPWk)0)kx+wuZUUFzZ4}m zUU1m(|En0|4j;W`C_vEycvT>#nL)isF92{eL%D~LSt1H*YWU5XCD|@6E<1qU*?>x% z@8J&HM_XZ(i8~~Btl@r=m=@#HCzx`nR_#$M#!r@^kQ%S6(ShsVMJ%MM=No`%^Tpzk zia}>9EE8I5@gHs}p9IaO&%Qu6OInFBsL~wFx}*9@2!`EKT3VV;#+w^h8oZ}v2E1VL zX*==CFb)$1>?m7^y{L;L9@)iM6ypUYQa+=KHsHyS)6#~6e8YHSJOo(8I#u4^*B4c> zQPa@)nDg{33YY=xAdXR0Rb}`d%@gVerBqZ^Yda60*>ye8%(!q1VvU1HV$kc!_Spx6x7p9de zz!cr>B$Lm*5kdM}?jjowaNyd3*!`dSkNguRHa0e(SMBM~d~ltWfP*U!AkIC}nn1#> z!XF|6a8ZD~#LDWb4e*|%d^YI-aW&j}q2@KBHJ9tk_bQt|7%%~RI)xi`m?*`?bLRil zYo`Ws6pR_wLb18W89tgGu2?}n6oOj~!5#G_G{8K8^-`bp#vK_O8}o$fL50K`s|_@7 zJb_>YSJ&;={pjbc0q&vc)Sd52bh$JJKi?k3_Th7;sb#M`1LyToo#@_0yne?K_5Gf) z>x5=eOK&l0*c^exWkx@~8uaHSV3d>52 zK@ttUWE<`oTId4n8VtYRAFVM0yhg$(QT?R|>+#kvxI7o(Hje;;F8auTE(0Y94>k1! z?GDhDHLs_VWPEP>e%dUG;`?%Bb1!b>Ah#m48Qgq+A3xFIjw6vYxo8_y&zZ@l@ibL|r(M0JXFe$ZqT>1ZfJ zOZ7MnN)VQCo8kNu8P3&SR{c^Zzpx`%)~okE(`8-tqvX=cvQAwBS8Fy-PCW3YglrS! zO6?xJ0Oc)6XF=gE=esZP7~s#R1p!h19D*vP0L?3C-3fm@Xnoz`*Y^n`P@g1$6biNB zg|gPrUBj`gp|}S`;DGePIX&y_-UvZJ?s!wwy83vSm9bZyGvoyFNdMdw#nZ8*%&iEwA>2 zm%!^>9pYQwFcp$I8r+itoR~+UTo(e#E3XjzuCLH?0rIY59yv(gVIzfw4$ia?W>}@9 z8vAyMJH<*8U%|UqR?r;*u zg)NIFm0@5-j}+ay5X&U}Mz6RTwlc~I{q#wV`oe|I!K%H1v?fO`h0Na?HN-&XYY(rD z$8z{#eT@dORmmK3e|J40vw-bq>xMP#Zdhcy$LJWNK@^a_O z`eNetC@L}TosXIQc_}9S7H~I! zOg`aI0OUE~Y(O23HEwAzPtI+?&=6>zkjbp%ou1w`9H-xSs^}td@0$MOPAtN%Vd^B_ z88<$*mp*k1HDPTuV`QU*0)l}YiH?k9bQt+tmnq?!7Qn0k-Zj!(l2Ft~73IEw!Duzi zFPNS)f$iG{dz6hb>OVy&&>)+562Z*GBm{gLr$0qMY)uSlTbdlWkjTN|c>@ff2%h;^ zez({@f5jy}M~)5C9K!X|Nt~o?q1h*1wTT_cf{yk0rhea@OBT1B+=^}8=%A6oq0UB4 zxVt`biAnnYN4khHlrM(_+UGoO81(`rnsZ14B6u7w00FZ z4-~|mJErGjY-P5KMv)Yb(T-MIAAxS{<_(}o^xqL_<-dIXe1wXtB#A?9R`)aJ7#cn! z1Ancl)9QB|F*3l8@%%o{145w-$;Ostztdp8QwsPl$wQ}m{a2utLm6}-wL+R@*)R6h z^KP~MDFL?OI^$-y-21(>dp5cnJ*`R6t08#?vbTxP1SQCmwF96fI3a>LL34{wlpP@E zSLZN-eRg&R^&{AiHF{v7Lb6a}0yu}Z^&v)|NOb6GC*C9IG6lj{YKL49x|-E*-vWy%3{JSY zV1!g0%hu~py>Tv(g~a+c6Abq4EIV#3$JETwjrI1_;pt}ZFZLd7+CkG+T1^SN&f_gQ z^8!;>Y{q45Gz6NiMg?e*nt&JqV-TOs|F*XL$<8?_pbVFPKpFyBAj5;T8spcN=*SYP z_zRTMT!<3z<~?Bi3yng2C>TMXE{*wv{m{i0_yFr)KgiFLH~`^H%4>}nf=39rFVjKl z9~l=H2I~|TM}ju#w6%xuW-~I&x9;0UiybDN077{0S+S$CrZCe`owq$pPC!D?!|(>N zJn^LB&tFhJM7>EQIYzn17UrxfTQ@zg$&qb@-R0)h@0jlsQ?4fo$~*DWeeTn3IOW8E zfb8C2ihSuaXe;^zR|`UYhRR{1PRtPIgrHFXbr|&p=h{{D`Jh&-UaqaJT?G0vgj_iH z@PC1KkXnfWUps&P^uYFb4n(|go9ZX0{@&lm z%Laeu$-swc#Nz*=_c(fWSYQ~}$QG+Fi6JYxFIeTqx`=M{&BlHL>j%F8mcEt$K=hH4 zJ@xe>HnmUB^NLDVsn;dp^r#`_Fm@0b6BAPItW1P~?7#+7=g2J}pbXLsghWKTFp(H; z0}YP<5%BI;(at;{i1x5hi&L0p@B~UF^c3ryr%2J;4&M7$ez44lFXWy+v*1vS4ho@l z#v#lwk<-(ItG0W~-|yvV*B(iLDko#tV29#XMLmxa-P;%~R_IoH`bwL!jrUMc@Zd&j z2q@iyQnYlsm@a+OR@&@JF*LbdX>Dw99!)R$2V6uJY*d6hu=z`t%kob4;VwbP+YO`{W|5_hx z&d`b8i=Eed|M8;|OhdvCdO>$};R0Zfs>N;3w-Nyy0E(7v-Q9T5wjqyFg5EYBn7q>a zu-MqxG(P?J-*_2JN8v(N2B|dKC7Op}h!TG?LmR!VB<1=<4geYQ-K%A_u6|O0JQdz{ zef`@Vr&?}J;<#|ut2Z-ymmt#?RmJdkyue@cz8H|Edq)?GA9`mY57>x_iN}vWl$Y~? z;@#cS&WT91%N?kmH*_pad!9fFg-=4F3@QKbhVyuMOXz8anH6m46S+$WgL1E!?IIUwaxMG` z5Fejzn(ONIc)QXVJ4paE6N**MF1MN*+))|@AWz>z^-ADf>2{Lnl!mN9wLfG0<)kTP zKrV*d9yATv*x40d>po*k6S6@?FlV@rDe-Zjjpy=ffyNr+T;ObCeTgEUe!!_g4W$!h zz#!eG9fLys)n9dqVH4!y|K+`tiLhl*RxJvtjE;@5rS`WA5l6?wJWogn9M#);^8y+b z*5F*;EfQl49UUy8Cv2m!9U|Nl7zvpmH?iYBy^W=x#8GNUn0ko5ZhBqHxvurOwa99% zGcC^A5L-Hom>1HCgd%_R_{(R@gipO-4isgV!XOg6sA%F=u{l61AmH5aejOT$yHGIE z&s}_P$GcTHLf(L91P$xoCA8`ZnF3dt3qNrup%Xg?ebUc`8hi~!$cheO8SJu7iU9xWyE5on;YMX zWvP~ruyKquG$p29De`^0n-ooQ)Ms|-e9`D(=w5GKDb!)Wf{_&V&rmhG!yaF%PK+jB zcZzA;q*N-oaJ^{KL?c_&?#RAKFRDG{{3Ukw{Jw7>l7-I5;o#!h9PTWlfQV4CkNO~A zn0i6>$-Za4xFbIejQKUjaru~QhAI}dgM9|E%a#p#sPMn66e7k* zc(oO&N`FGz8KuM*4^r};L+%up&!3o>=y|wf1C8coSUPS#z7Uuv0RlUGJzePDg97sc zj5c+G=K}h^&wJ~FF569c`es?^b_V(2*17<#zb4+1_i9(#$2+unjV<9fEO?ESLy|3Ug&`4VBJ?hy@c3(;uG712pHL zWA+(W-?2p1d~4+1nimJvlip zZyZ)ur_Z1&J3`FgUkp=gTi)rwv7~59PKel3(gwC9EPnJN{_35{FfRQq14b1B>lqBj z7DR3#-|m4}F%xW=M2OG!sdq$1`QgKd1s9Kn?NhD}CpQE&+`;%tw@nyfBaPHR!&OIV zX_`QTp~0%C3KA5LEld+-hSs#!{rA0_4jKS@gaii*J=`{j@eykn*Kj%IQ=ia*6~n9Y z(Q8;iN0Z*ftO8axYS7p_qV=WJ=Z%TXUn3mfM&{-Ldf>zr(3JQdZ@m?E#fYJb&aRlc zc4&?`u*Tc~l*WK}YmY%v@~x?gq~JS286b59WW(xlTmsf>anXTlp;F!?AMwpRx)(oo zfnZyv7T8dCkkK^?sPjA1$+`T~wb+`|{k_$GVV7wkRKj{48V?8CU;1g?O4jYvw|)sF z3bMJV6FV4sp7Rlwm<8fQ2IU+=J7D?N{KUd#LGPtCu%lT*$*I6s5a_Y+t)rhkQ-OVi z7V(IJo*tzocpCsnkq53w^eN1)$h6#v4gn8c?6EAVzEy8bbh#< z#=ih4HS65JUc4Y^jv!FU*n$E$=sYWeC%dr!F9#HP8oIhqOi4*e>=Ywy&AWwzc&>_9 zyKpG_PTz|2cZuHJ7|UP6+tlN?P*D@U3W57AWjz*TX-3{P6vk0j?n|So3Y<^psj7`QO$`kt6O%OI&r3AuL}49sc-3mosr*unD7PT7H(#mO8G)ey0Crn&o`9wUNfD2Sc$MPj9|tY{1%d3rvq2cw2@ZF# zr+oBQ$nBnn9lR)z+~c6r-f&%ohnc5`7;NfXhDo5KpL~t zBG%m2TE)J=_$xQ;Zj0tO$}Qe-IYA=BDlgAcR8#~F#qfxT^P8I)_ZWOUYkW9|rhk5& zR}UTPt^+-K855O$E%gC0uR4|2w1R2_S3*oa<(6s?JmRY<8^|{inw$9!dK++LOt5XR z1`qd5*^z~N0WW9HSHrRCiKXwns2!nRqzKs`)NN??hr({YzQg(3LfJl}U|pUxb@`*oUdqJ^6_uz5y@IxKsVSPq+<47#wV@W0OBU z%7Z*h0JrnO zN{aKdQ{tRr>ae%&`#RSQTlavr2sQ#1cqjH(7~0h$22+cu;aOO+{rTRqOS|Z5x8y;Gha5 ziLKGi-*I5>6^3L&^)y-85jrgOy}LTpF(wfuj4;{>wbx8fz{7B;-oWw4sXpUz#fXDR z&8uVMw-$f?6ec`oXbj%Gxr_d>{ZGu7;1_6Pe!rCbbyWYBvIeUnUxbD!C++%LwC=Xw z=(Wjful35N|NR&&Gp(F_^w=(6)*kTWYbC^;3YT9U&V9c~6sCAkv;D7I>a8}>A11jY zdK!G3>YAD$1E(hsj7AC7pX*Zet@I6xKUCJEhdb6;5m6va-d;L8%U6I;pb8ez)oS`f|ov8)O=vX@1*m zw7u~sz_2z=hRrftbOlp>{9&+|D~!sR+#}ogfy56N8p1HpNb>hFvX`8iYiy+gPh-!CLejn)NNC**UgG~rI(4F-((+4m#Pu||3@bUQlv-Mfk5PEQOk$To-; zq3#72%J{2?j@;rSXNJ<90@vcLu-cjH{iD{MJ4d`(-`Lc*%YkP1Wg)1DCu%X8_M$_!pfUh1I{dRG&0gKO|*miVaj>MJD`yDYOH1w{9#i6UBwJeBCh z_fSj2{P>}(JfB`=kgUG6+P2T)d&!W@K(!5DLx?y|?X^RUruO+pf=#igFy}#Q_+Gb+t~aDzDjwDf^x3!;WF#vb z`$PvA0G*H=T&5;ugUIB>bA3lu)KY zLb36%*Z-CN$f{J@`-Q7sp5eNDX4sV|s%egOSMXU$)jH0~7=Ov*`zdeUdHr1>7W^?y z5aiYvSX1HEZ{f33H~y!#|Matwq}Sg%hyqelQVO8@%EpETE%Anilb>4oI*!xC{7_uX zvZvV|d4KcHY`K^dGxb+YGEbbOpz7?%5u%4Luqm9GfOiQnZ;jvnJy0(of~J_Vjt;p$ z7z?c*q%jtcXyHWP-@0r~r}naxMZG?8QMkxwH1p(cT@KbQSg&UdzZKj|6ToJ71= zajF}|hd)!?fH^3dJX<cxM0756}I>sYEy_vr~eB==%abZ==~qQ+I2{ zRfLU6*oKp2so02@0|#und(B4(8cV{d+NgyBmF~!uO>_sw=U6~$4c#{&3#5nPH&f)q zus2DxK2^B&5Z|$Xm-B}6mGjfLlu=m~=q81d&p*!ju5NK!n+Gi>kYsOzE{Txi=w+zj z@qiBV5vVY%w31t-Z{Yif22N;&|Gl|f@UMoN;Z-(1ZynV+1@#{?-`jhJs+t^2R<6?L zkH#PfBxwuxx*sHP|L&a7il*Ie@J%ud{*;9MiJEs>?Q`wDMz(sYD|^8J{71v|CN=ah zVQleD*?=&?1I^m832uj2?8CVz@|Q<2#AuyH$W}EuJ~tEIK}M%COMXKzH4~F0;~6xD z?T=WtZRFr_fKXE#ql@&~c$HAvs1{wh1d(fNb?4ico?A%Qb zXx1&EyJ*p!98O*;P1R0)X6uI8&~qU@_Eq4^cswyae7Cw>3yo;L152(%%1iDpJX0{C8y6Rk9B@8|EilM@*Pm_uxbXWtE zO_~D@zBE3NihNb3m@g#?u)?Qf^xdVxq`{%j4_M;9gWzwc;9MP!g6g-|~(^pGEoq zqv<=qxoq42Z$gMNA|n|o84VQ`B?-w+M#%`-kqSwPvRX!olw^hMO;$!ml0wNIrD0~2 zmE`}qp5Oa_kLNj#=Y8HczTf-4#(93$Nl8jev%`3;2xPap=$!fSC@T)spER+bzQhM@ zs^0CV*nRkjWThQn*>GpB^(ss=Jn5uy`fyxR4TtQ=&T4}3Vy0aIW=$Wh0#fAy6|3q# zXD_5FfpWJgaP2fv?wEODBCGZ;S}T6HK$oX{WuJQKnNbIm%uXALk8KC}eUS@ciGqIU zQwBehHME?98>9=}zi`LfC>C%El7KTa&*4q8HVr3+^23q>s{hP=B^M5(UM zn%ki^ufR&o!|v~}|79?6=+9sUstzvy{lIu6^C0+Z5BRVN3LCb5*?z}5cK40KAznSc zAMORWx?*$uw##HS`yR2SBR>DTD)zbXQOFa($G70!u_=-9A(3yn6)dif(|yummKu7w z6VfJN`a27Fb!t&KPPLnRlbjftU`Z({Wo%hAvONw2QD%VjAnj87zeenguO>Hf~J}AcM?y*fk3QwjY2cnzgSVsQuJTyZIpgpsd=eX0aC24_&mk zQO!gU0uc#3s>fTeg-XcT`8{e+0ihht`AI;{H|s1`J2_f_SVX- ziB7mM^Jl&v5lH!?Yqb1#H(5a-3;=eg{a!Cj2VH`iwzlep3p-1_0Q-{KF`?Tsn8#H9 z(f+(-W1)*Vp$zf$JoCk>NT(gU?>dVCGj#aP2I}$vK1pNcs{6L7ab*K55g_(__t{ z?4_KuzM@LUV z&S?ogfJgRhq>43d+r!pYJ?i<|yU%aeMXEG&T5z*f-ETNUKxn`=fR(O8(7;gi<^Y%@ z_n$qtXjU)`H0|j>E2@!pq{ub9P^u<2RtWq86A`u`m4?8%9yp^Yn=aiibl?4DhD z*(`{}prH`4G+=Sg+Sqr?<^*raH3R*NlnbiW>~_4zBWVx!~Y^`tRSTl~Ak#XU_9H02u^CLe9+!@1N}R#lRcVf{|U! z_v$TQN?LGsK>-=rGzm$>0F)VZB&O`CIWp-W1MA$5JdgCXl{i^mLAeG&>20XARx~p_ z@~35}X3+jSoSobHN4?m@Y;ZBIE9GCLue)BV%wB;E8MG9MALJ zbjb;MlT0B8i`xF%TjmP5NXWohad9S8z|c71>W9Jre}O2}2K_NcaBZ1l-tWH0oXvMfy8D@=#?H-uZ(cQ$3Qnm6skaW91xL{yJ3EMKsg zL-g6e`>>c_{i{fo9ff?Kj)D&07L0E@LK*pFN8hNRhZec_Q7%(qqLXdtlWl47Bb|rq z;&fkMVRu*9%wMutX5mzhM2p&IFaxPv+;ghN`r&|$(gv7PT=Hd`=9cWAXL)CCN0Fk6 zxo%iVD2Z8+|1+`)a#lbEkhAh^TGi3BqtKmJDr*H9suKbwN8U2zv)aBZPF!u*#(>i` zU^rvfRf_lxfVDflY#!$B2~FM!uY@$;wi83N=>@}LsnH`L`|OfSk1lN4(q`6uJ>vji zsAf59_44c@j)YUOtF|3{cgQl6B_vm!ZOlS}#_F)6n6Lj&GBz!LN5vz<$`#oO|e+M;J}q($kvXQ^CyJst5+*Fs@_$^IZI=g zPz%NaBY-VN#d}_QMVbCMc2D?9GD` z#rEA3oM?(Tje!fr&Rz9-IYekdu&R^DBN^T|i1Yc-<}ebc2-__<%Qi%8`ep2yIrE)6 z*3`@cxf=*`iO1m>ZXA=8l%y1=n_cTW(O(gQ60x$ajSIPnXT}k57v2|$(~{^PAsh#5 zc*7#nxc?$XW^Fil?JhPlj%58S3K*MB#Q&HD1<6lZ*H05t0eVb{Fbb1wDRp&qiptFV ze*BC{(HS5tk!Keaot`9bjTK+dP&ln<;=cYP-wlwx9%~a5)5k&sMtO-l=i+tdwDv87lyPq-YTCcMF~>YB1dMEag||{FSa_ZVw5Lgi898 zyp6rRI7Gui@$n+A_;-3f846wYkb`52=YE>vGgU#`jKeiNpVNo0APk5N z=n4YefwIzP9>W`r@=F5-6@V{f5c6WwN3#t3}_o~xy$ z2yR~=NH>y6(u)ZZ)HJ~p5l2OxRW)CMhnT{cM>pW9JDP*6|@gWh8Pc&HaVejuU$Og~PP zIs8w`+^dxkzYXKj(I{T~J9Q)%OgOC%B)$o6rX<2|zeOtRf&P*T91fg8s)r#>(i`6Q zvmXg8Ou~L7W`>Shz>PdJTYi2p`2|?)8a-|8WWlU5nNGJ8hMF<^_qBC=h!9m?w z;a$-ei6?X)kWARkn~Xb=y;we%hZDek{i_|Kd-p~zRVc*?VPfn?N(~WWw~RBEdNNE) zBPb!r(SrKUTUuJaUbVff@@JF<&rV21;&r$DuKV`=`+W?VpdX$6nwo zOrxY?OVNK5%z3Bw%s}OJItmX#C;Ht!!7UOJA_#%Wa@{t6Loac2w88;(+P;HoU0$w! zufxAD@gKI|n;gGWt#WbYQT*Mo^2*Q0T~9oZnJ;TfZ>amE4czU4xB*^(2=l6W(}6=2EuGBT1{l!ay~Y;5eTdk-d)Gf^B|Tn5Q=2Qbu>)(1m>Z}!f% z91&!v&b)abUenlk2MU^@^35_b;f(%DUr=czZ)FtziU5{pI84&Lpc*~{&e78ft5XGF z^-+ws<2`t7KS71&A;`%de7xtJHo)pd?;zDz4u%MxzaJB$gC9|OeFYtL84F@2I4mMU z6>`D2qGdR!w$E=Q`M;YS1-|xn8SR6fe>|USD{DAM8o8{%@eJ?P5)F7pIo~6Hl zQ`qmW*_Ed=;r0uONwR^KmK&aNWyb^>d6zplTpaO_E8skMxLJ!O-Y6t>R-?69$xj5< z9!uf$8j+h#Wd}xaE8oR!AB(3(!~sYVTdmj~lr3SHeu8ZMc8aX}|6eQGdPCq`i&p1*9HCO$Hxl=8uzrlFAF^?Nqv>C@G*BK~IE9$JAu zJW=Yt6{^QcQlkR<@`aD&-@nCkM;aJG4oNw1G7cJ$OSd;2<&tDmBnu1p`WMZF4ipriECc643RzzD$K zk&?P@Pyf|d!IBaZ)xZvjP7~UC3#1Hnyg0uRZ=Rqfg-7Xx=}0CpsQ(*~lvqLzO==BpElA5r`U^<-5iT-+%)BLw8KcydB*MQyb8oOM4-ZkE%yGozX`+AbD8I6Y zcpIR|T8DTAyUyHA#QB38^Fc;wC;Teu#s$p%UULFyV(4Rkl2(|QrDCp5BMG#M z7>8#A&6pHoTHm}ZjO${eK$xvk?ADbK6{W*ZJxgQRyNCf!9)m%9e&;4=!y9pL;}Me2 zL@qc|If7gz0aoo<=C<`XNl5Yfzd?69EUM{uPyBFB;*5ff9GH*G=StKQN*cr6S*4#{<*`*Z?su64TTQ`oeA;G3FbAe4reyIwUO zTHFV^)0cyo`r8Ch-+a(tPS}0(G4nlz{W$GLGW7H#;*UKB>)kT{BMgkzU5!xnqx*clzTm zK3J#KRA|dSmVb`6Cu8ez_8byoI}M@FmTlXJsqeIvl`keKjwS}KP(;_Hu>AKTg!^A` z#iX;5)s8Bo(^-NfFH@(9yd4qGBnA~~=_3;zCr@^5(nH(o+3glR=jy)|L1#xy$~@Td zVr%iLI;yATq}OqC*S$1+C}$?fK8u!Q02uIm#q#38CkkGCD3ge>3vN}V!-q+zn=r&} zeSLkQjo)wtt~4<jl+qslJk@o9C*}w z0iH=jwM2{#jFU*57$M%mYRkdBTxe5+&kjWO{kDB&sI%Y`JxHc-VpUkFye+%@s~Hy* z3{WugrfC0ovS!lM&-7*n@)-JpKUKt2vK;J4W`5Q1T=NW91N$T6o3je}Nu0GWJ{s2a z>i=oz=sJpW@bP!-4Z2v?cn>JRQNG| zUe5IVw>Mk_NQGW~jP7A@i9<5F9ZN`e$gEM|2K5YO|Nj2cA(OwcJFLx5_22kTk~k6j z+{EBO)T1a#i7t_d6>%B6psFO}&tE}(fclOoS1(wB#$I)dj(!yv-$IP4HQr0#21h7`5wGbZ2c= ziu*J$ov@z8-tk&5BP$J@`*%w|+5)2Xbkrz@B3C#;x6f{NNbbYYH!6=5E3^GIWaxAq z86QZnKHy{hD#1GPWgQ}P_@t_V{$mZ^hMSTc~Hr$UKaHl<=WEyRI)$3si<8ak~t$td&f3w5~(VG|& z@hP;@1?P*CGQp_Z0v+UumsgAX&OQmo;hQrf3E6ZYGXHX&dfj3^L%ImH#!C1f@eE)^ zQ%;oIUh*q3FOcm9R+G|}wU)Sx#$*_gZ;VNwE*OgqAn8?86DvRjNfaQk@H{h`+zo61 zn!2&eL1eQMKcA1prOhSS+Uitk>b_3NlN8J{#fbet?ZgY2aBP z&l_OjYh(ggopN$NK&6|O8g4(O(Y_QMweag%@be&uB@z(6`vR>CVmZ^IF$tN1=eyA6 zBl^bX(o5%hV35Ce?;bf99?PUX@1Gl}T7d*_5{q|sl9}49^YkwLDsKIhXqYk^847xO zdr9VT`C}`d_E=iup)C!71s;cJG{k(u2ci~ZxIDpw6 z=Z@?{=k6O_a(2sVlcDqc5q;@5tu+t?EQMG9Tf4vWIWq**#cdTOov2mK4q_KsPV*|s zKf1I!-@nq0#cengL~*LXkKxn)kXN`AvhR7*Di9u^nAsYumZ^mBjFT!C&a8D46mLKGv`KgtOl z7K50GA z0O~(FB!n=Wgb0>~!l|?Z#~V#0Lh)0I1Wi}QL$_~ut`vteW(-7iWf8ym(w`Zt<#V>S zWC}zySDBne_~XZK7y@_mvGTt~3AKVtL<&DRB_#zSn~RA`Vv4!vCR881k1jt@*o#1+EHwwy{ z4`mYz@6Km$ye@kE4U=mUJGP3Vp{S$EhxFsO5kn57s<3_6fsSIJ{u_3!pB4R7s=N~O zO&X;6`S~GDGiufLDZE*>8aeIg!4kZe|B<$W2}cL8#=NRHoNorqCHohiaxv3qniSJQ zuUbB~ThUzp+t;tz(VI(MhvWCW1NwT7!=Q+6x?zA0!X>KqxF7o8iK{o_AP$rRjBm^3t>l@_Lh^Xtq7 zE~H?GihtZ<@cfvRQCzJ4EeH{Azj=Vw6tA4Al%mI4tPb_o3V-7jkCa%$A0w3qe>s1! zi*5HmB1DNQ(J0f>JhP-ZyGg_;wOpBzKc~>puRPp6DoB|QY~d3p2j?{=>vrxc*rP~@ zGZ3XWaG1s6roy#DN_FH{=pXf~t*aA(UFTg)-ecv+ZBE7yXEP5My?e-3?QWlv+LV3g zjz%K)hnwPynrO-mMVb`pvfd)cDqCVO~^?}%S>SOJSG zZC;`KJDvD6eqlC7V!sY^NHY#onh zq{$bV9NYA;v9rXa&AMo~by1wvxAdyZX~_^iRlUc2P3tpSx*9he8&-knS<&JV$7;hQ zdm3{=EHqXv3k%_AdMPKZ6?ksfebiPQ97*OV=DQHiz#RVgA^g0_+T_ATzZ8t)4d|-~ z*88DDhYTFV;4X(HzD&E`?zy`Jl#Fl?kni3lF6UG^Go>?fvjoxg+X<9RtaK>wnhTvy zxm#|-AO)6J14RCax(@(Z3qSxCo*jzk1Pg}4|2T*#F{Z_UWxA!Kus}!3H_Rd-Y{=gU z@8z`F1Cp?*J~@zb@kLh(ut3G$0#4n19k2aS=YDp#+@QX;YxE;Vclgb0=~v1*i!hzv zEElSVholfWIkhE?r)tAYsnKrwfKS@6G1~9uE8MVVZsjbBW)&brsL4)>A5RpAKIl|b zT+(lMfs!1V-q6umM=>sB8{Y`@3_xLdx`;aVmT;|pe)1?56b}1k4`SP{ot&lUjs{rcD7qI=~{Ktud zGX(&8+FCA|xF}x86a=Xy-s;d$HmF=a163o<7o#wGSNBA(p`BsCiDHXzI+0N_pa4Sp zX1n5-;S>>h0OY6^>OUGg+a2aGpc6U#i%Sz5O3(Z{%KrWT%mHAc%158I*byHWN`(Y$ z=(E>nafGVZ&&|%-+-cv%;)^DV2>M8-LsWF~;kkvZ5x(Oi?^O_txE+n${zLDDjFc1C z9XjkYdcK&aCtx`R^Bvv?`9s`gXc*CVe-i~?*b4M3E-;j#nge?Rd*iL+HF9y0M}q6Q3iThL^i}3*&&MoFOXlrEI7>j z_4bEv8k*Y=zLU8_MZ^_^wM4LiFN_EMLq=ktmBBS%rBHObiWW0tK9iXbcdb`XH1CTm zd28cxZe(;|t67!Bvuq{4C9eSe4rXZafs_Xe9(?EQjV^&;k4W>* z+okTuwjQn@To0Bjw#&KxD3(~6(Re`)ZWhU#VZaT1n1$(_ot?=``U%_@kZ4X;6VzkC zAaRpm5kJ3l_s_f3i^oe%O-vBDpmt!f8&w*}A!3e&=O;XZiMA`)A0-0muHj*(vGa9? zPCrZ}y^xA}t@h3Qbjqp&9>H>_R6 zx`H)N^wHJ+tDs*#9fA*g+xq-yxWe2yN`6{ zi4o5%+#{MX3eg8_bsWTC_eW4W=6zWKRt9p9DjvrJN!|W4H?am)K}S~?5$qB(JGiIL zK_obkqKG4oKgULp@0N8MQb z)wR^Ws_Tguarmn*ZL{GOf`(bm?6 z2>H^b_C+ret-(ZDo;#;Q7~5&AdyIJDpIw3vJd{o8rfPyP0h&vD-@gxeQCvK1nnhUf zy15!La3-(OeI7k|5|l{GMv072x-#$o{?=TcNlj^rn+(PD6sD8Yf=nLELc=rPnBrUP zG@bOKpOeLoULrR6Ax`h)xvM}la`N&vBJOI->?E(1o!F)$@(tnN)1vWtk5HlLzYOHR z<5P%GuhC6)IDu(Pt10+m>V7%QUks_;-!G{2`>+znUUvDG-b9S zGgz_ur6Is8g9X;Hu~j-e1Y698%QKk&fah)+MIb(Dy+$C$X%s!@er6PzChFaT_lko$ zgH7efrUX5K-mvjzQBgJserbI`4rc#o?CN?9$J-qNjIffW^?`?gJobAy`}6uYQfIJQ z$4*4QqgBL+F;)DXbgA4`u5tZ9n!ALQL1=7kAVKRJD%+B^oL2h>K5ye|ow+k8IaDVbZ+FnlV+mud=L#$6~r}2 zOuCMI9YU%>$#a&kboHf6a}x5h7lxscfkw_B;QX*t5om8Ba6vT7wozovi6H^185Ny7 zOhP;aC?(=I;MK&0frW8?xOG$cr*4f1sVYKh;=&*6EnNrXr5eY`c>vgC$Bk1pJ6MS0 zh&Tc9;Z)#|L1S;`HDiE@Rx7ZF5i55hXyTSm=rpg%OKV-T%O>||bxPOhrDD*fQi#wv z$Fq+&cEaijxT47*2$XdYW?~lI8=J+U%ofbsn1VVW9!(ViFaa~)!K)Ym#>gY@F4cXA zjY3VA{LstH$e~pm43?GP`)K*1E-x}~0IN$GBpAF75N7AN4(q`@qIhmN0Tu<=-a6B> zQc;xDk%$PRA%SL_{Nent%qv#t9w2*p<-VJag3I;ELGKON*{U#_z*Rs{XJTi@eQHy$ zg;W7DQV)q4llX|aKNj#^dG@-*n$6V!Gy04c*xLIdiq-#3Pqf6l2v^sf09_3)1UqvkT)Xlh}9CxYvoi&}820j_n_)Dn9lM~gZSJ4C7frI%#6YKs013m#dI z1l=u&N~U%?%o5!bRZD+l$Do{itfg{#Q``lHfCZSos9s-=WQc$ZB=$G zP_e?M>aeveOLq(2z{x1!_Hi(B8!Mk;#+{(1uCCT_M@iYHO*1J2~UhiMBovqD23h{}BN85%30)*?d7p++L4KAu@DP>O?lx ze{FFqMdBaG%)9fG1Bc3!c(mB9?2>ne2b3=qaIcEWch^hNSJTk&y>sV|S#+XU1JF96 zS5vs1hQU@ZTt3sHb2CeMaKm}u6K8WEL!_?0bj~9~2)(wW{W;D3&+mS$_?PoEnXXCO#>1|32T22f%JjAC68;m^*e`Kij_T%3i zYbv1AXfr2KZKQXZb1(qp_j#3 zed^34^S}?Oc%?*(UD-WjqITVHcfLQ6_}RsAU!;aWI(Zu++;=_?4ZW7lCC;W2*Wnw; zi9vqBqJZOw=dC-S8HKqjij@LBw@qY<0~P>jkS6UxZ|8X76SwHVQw=ELv&<_bMtyE8 ziTR#qRARsnByV{TPQ(NGb&r-E#& zYYt+u@&CeoXgoiRxv9$xwXu0c+AC>{1d}_5_5{jwf6dLW9J9&;P+k$$7nP0CYp~T@ z;IP2w@w9IYViKm? z+bmO+F=lS`@k!iwW1k0>9%`3!jNzXYZ(4h;>Il30o_#S+G`)K|f{Y4}&8w^wJo2K; zpxmV2#V$yxSp6+{pbuC_(j_6xTm_W{RH2HPaRevM*QH zWk<2ViNv)c=1DtpR0`m*5l!_{8{7d4-d`Y(q_!vFJPl!DBpe2}ay{r)b}lY|Oe(Bu z$(ec>4Z$Kwu*phermoI^4g~*mE^+3&0SOf)c`WEg&z7Jjk|F^E>E;z1P*ZvOp8c5N zm^G^6e3Ow=k6kr|LCI%lxGbCS+8u1nsqbpyBUdZj&c<-XvBAFlfE9@hbhHe;QKbWu z<*sWR3_L;&!OZ+Mb2c_#?$PDmW^xm+j_|PLh}f+47-FH7e^t zAK?GQM2i5F=I(9x)Fv^R$nsPA$%3CmT7*^d3Ve5m6H#&kSs^h3S1}#fvqB5nFCvO3 z!w%g~d3fw4>OBzqr$1Nw&v>rUv9ue1_8pP#zoRrh3tE4(mZM{_`c%*=#r4urtjjA@ zkN<$L2_(wnhMpT0)*SBQ+n)Bzo>*tPbkebJvG}LZI#t831F)P-H^eFH+*fw1sLyj1 z^iIBqf^o;1TLG2zfDpn3L0+Pwq5}}C0#T?2zhhu^>J+7>rdEj}0`X>d=K8Lzf&C80 zcP+Xk%n-SQ`X*09IEK4dCDZY-sh4u3vkhajoT0*!$MJdkxde+RBOVX_#4q;!8ou&& z>y}QL?9Yvs;t^VN-?=pOkudgY$9=(%?FaX)#lbjAu5Ir zKkC0Rz9>=MuR<707-TR|mKY|8ujpg4t)6Td3`HU}A_+4o?_asj*D-lIjwN5?>Xrwo z5csi3j9lhTDcg~_VbWawz4B=w8ZQI6!0ub`$TmP0854J7Rupso2PAE3a`$38DX-bC1 zZs>N*q?SC*ADZ-S*IcJV^mgIgah}~^o^x}PpAIUq8$c^&m(>$ng-nIfw1qx>8cc#m zB;-;gr%jHZ(Z#{G{BH?KBUySK<&)K`A{D&k&^Q>5H~j;JOQXLwqA>UW53jx9>)Qgb z*wg$>fW^rES61SX$kiR1W7_=9`|tUgwXsTmQ~vdxwGb>8z8?QF@mBJ@+GjDO_*a@q zDBUyhcRYW&Y|Vox5uZd~3m)Q`g-Y7C=s<=U*!EYA+bc}$OEZPsoiZ}rOcSu( zJ5}J|kLW)VRDs!tAUWIlmql?)`$t9bBjI=j1gX|j&I=<$_*quSdL|0{gkC%2 zLEoPrrWsegv>WriBc8guD-%W{PIUvQ`up*?{PfM<{M zdNc1OH|QjePZZ-F=R0+C6Vw9=ppVb%(Mzf*DpF+i5#=P_8~uN2%0xF89v*HcG=7xG zP$6e3`#are@XoUffCjL6VseKk8F3$yK__!8RSI@p1A0bS*mHH&u^An)meK1 zJ;b3^bIFYoI~f3PvVJ2fEHfOr;f`(w&^3jHMpPSou1}x?Ara8f2%z;$%^7@b zoKc5GIWRnY>-vh7hWnDxn33U$fIk9>gal1G1h-<4_DKY_S@)wK;n{PpPZ@3q$so9m zXn*Im+mGw2JMDPF$Q}3P)+H=Pf3bDh71E}E?Ps|Xon2SC9y)6yt+`QeiWUW!ANy^SY;OU7TrOOwhMp$`WL~;SF$YZXDYG)D>Pd(=&f?TI_LnbTQaKR zY`Ff6N_BqDi%A!|fBCq&ZEHrduEUK*rnon^yeGBgH(kg-j+q$`N?Lr=qQ6Mg3(WhW zggQoh!jFx}h6sC%_D1r+In2Vt&p#BWOH|v=zXwAE3ZCVWA9i3qf z!v$1CX-%oEaaSf}wC0CVp)St|j=5M|@L}3un_n%@7Qax^+-LrOWxHx-QO7uzl-nSb*ar}GhtT(T9 zMeqnhosb4+a;Nc|8;bBhKLSGGdBz|4mMbq{Qm+3Yw_=1$xFbP>C?5VB5{@IjrNY}A z;*P|w09m=M`XBbaZv4;}ipYmgN9Bj>ejSsifp>Hy(8nh07avYACB~nk-&RnFUuBGM zI=98`*`ov*#O2<*Z+=_;=t5}P$I#pJDdnsw(RZh{++AFJKm(Jp)Tjg#P;`ga5mAnh z`#uY#a76?FrlY}s2zlQaR~B9f4>}&sP?0!@uMBl6n6_uYm@nH^-6wz_Z^<6CYA4YUrG{i z-7=#n!4ZF_f-`gfJCzG&;v5&P8C||5-^|MLc4ou}Af5y84jq<5I;}jO4amg>xLq|+ zDP&Oal;Y4=RiO=^XO4WI^?)zdiy}Zjx+@ zI?I2Lj+>E=oFOENoTLpS0C{(r|KePN6b} znt=KJfjJZ3kop}g-TTEbX}0S;;&!260zi~f*!N@M*AHRtJ1s~yh8nFJ+IW!g1`O9> zgmCV8aUB4bhlj`hf+Z}rdD^)=kuyUz(*2n8XDR+WuB*XAr6&*?QcRSD@O>bosP6{M z(}P7zuJd(B?y{gTXG=|4iEsWZZ&|Ch`1s@8Lipl7xGn%l0-p|0J!pd>fzmHl)-|FEQa1&e5Z-{Qgi3IxTg-gv+sf^P}8J`?{pZefY%~f z0}d63o_f3)2E|iG59H(K(1*?L%jAA6ZN)&HM(G2X$EwI2YBu2I&WTexEAU*&7y!uf zqWdm)L#24u$%zD}aT?oP#Zm8v8e^llcuMONT4VPec}E4d+ba##lt^G zyMNLA=l`df;!eCaUwq;f@{*_*k3v_5KaRN*!I~O|#5eZnTm7;$iO(q9b)7kk5-E;U| z9_bs3r5~fEP*_yCry{U$2MdiNLaZDG%yo4MadfWvqOCKWB)OQ(oeOrtp&q$+a8@Q1 zIiWkEql)1yj??|-V9Zv4V!x$)28xQWV9ZtXBftkS``_rP z@Qzu_&HcD*?fv6YVl)$iqU7)GnH6>p7q{5FSv9&XHzon07RRHGZz3Fa5O^6#da%Cz z?RGy-?t2eF{3sO);rl&IFTan|ijWwbR1IhwBr4@vwK>Q<+IP`rlh-b|9sc7ScH;b< z28hqOa^*@|AJ_4&aQJOTI>D#d(fXj<3B}caAK9+B&8r=91Ih%IFegq5U6(3k$_C@V zz+%%;#DGu;hf@qC1Y9;*TJtNbpx1A98buOlKfEuoL`mKY4ZCO8EeoDjxIdAK&%V)E zL)g!}>u5qt=&YciF!`dr4+ZxW>|49*+j3$5nS$dUdM{I&QaIpKPwPQJI66su?BAO0ZdU!?2K&7zJ)~yvizEM9~@jZEVP5gi4L(qf-XLLgN$))=Nbr&Vvw&MDU*~ zaNK}7=jZxqYa0fNh<_2qg`_G(rM!OomQL4gp@r&tv(|htVU5*cQV`$Br2KCi>BaI?|idcPNU9r5Ax1tinQMT8jJE2DK zJxTf{36Az&_9B!!irldiP@^KKmmn*;X>BBrl{gDP!4foX9`z@G+1uL=$pIJf%CIlE z;CEDoWG&n(we=`KEHnxJK=nzgk^5b~0?r>f+VXDDS4PXZkqlkDuN`o4;I(LF6$;d5 zPac)u^z}6&KbutJI*lGAtLVsYl07hfW_Q*uxGPDB8u*UGVA*F{8(6i#%Rr#U&dKQu z*#8Ug_DVd?P~87X<)a?Z{WYhb+yyo9sdJ?`cF7DPq@j}xNvISsIh+Q#VuPyzFgGLq4Hz@bqE#(`43WW|F5PiQ>z~mD283leVZuonVkvD;L4bfM+;<$MW%`W<3Vqqhz8?9;?;HI+z ztpQ2}A3lTuNX7Q0ZN3Kfrqt-$UTEX4E5;^YT6i7}83a-^gMc0Kj{!gy(^fQ0jwZ9y zdtg=rwmy%(ap31qB73y}GQ*H}52LX1IWahi)@7WtnNI1LWZ>Z(9qBc$ENgT6)}!cj>Kmo>9nw`f;W$aN*n7C z0duc#<4F^P)cLksTNQ5aFg<4KuVj&J_29vSO4N&bDfz7zdybwrWmho$Ar5x23_LE8 zbpKCUzY>iPnQ+a2_BePI)MEb7Vo0I4$YVqr?oSYnjo#Gsppl?c3y0oz|@MO)R zJy+*-q`WOa*^a{YO0!I)w|wUKsmkk;hr#b68gc z#MzL%RKsn(!(~8`3^{@o;!;Mzj}dSe!h~+afGwiXMZ4)Rsj#34Z4g0hA*CSSykIlO zq@ny}#NS<(LkuEf7<_gn1{Cq0@cEoinn#F1kfY*(Fd5hhnX>0WY_IuID1i z*b9jRy0}2eR1&abz|U~j0v!Z zlAcU;Ak{8NI!}$guEaix0{|S=4Yc$8lOLGU#XfHvZT6~r`Zt|^<;p>lr~r`YF6;>L zdbX)>9+Ff|)Vjc~j^N0Era;&J(@vi<3xUu(z&XmQ{l*zc4Dckm3Ag(c{wtAvkx5;s{hrVL z8})R;jG)O6ItPVeYsSn4Fp)w?BsH+Fo50T`O-44Zrt;38IHQd&kH46-Z>aOVz`hx? zMtYh))qK)@%eCf;KWO{790!H6VZbOknfiCYQ|V@sc({(AIm zZwVI!QS)EJbS8n|eulZwAUH$Z#{e;qIy&_bd!seKg;#kR0}%|it=o#dMG~Ijf+Bh` z7<(%TpfKzQ2#B;^IHib-6;)mUgvH#@Nrea=tV1cm1dB&u_8;&Ax3M%2!vlkpfM@5K zn|&|IC>5xbE3t}AOI)h}b*GJuJi2%9Ix(S1OG`r%SBvHw-4*S1MP2LA=i9m(@ViL@ z1^i!=u+$hZpB7)sOm65G7rrq=H`9!P5b$WH=v8W7(uPXTmuPs~37!!t=a%|Z09xo* zPGyG}5}$<#{4v&P4d5A~ON0pc z#=yX7IIYwtcImY{MgePprchW;jt^p(p|-Sj5Sti@AKr!InV_!_eFlOB%Ik(OI|L8b zu!P%W{&_SN8+2(&u0d~$T#XWq-2C3p##gtI&J*)i(@ekl(H*ziG5+y6SAoL-p1TO; zAeG(w@g!#Q(I|o(IWcz{|4ZlJAW+w z{i}vONTY}aQW?Bz%^EQ@hMtQTE?f|S72KkJD3ko*nX|{;mmYt+QRh^Y`;ljv9=@P*!Zbr=A$zHxU*?noy3M6K@p+`S{c^HsBBy3fQCF zb1Q!SYv6zm0=9PQToN4f*9$$q4i)@>G^Y0LqhIx`g*^89kB9G!8LD{wy?FR!z8z}Q zcQMKR86{ABFWqbW_6(~4cG(g)Nj$Volp3h`)b$fzzbd0&m?+a^jw*!ArBbG7#`Q4D0Wu5|Ub%!x#m#$jaCZu~yy=fz@tAoce!)LB|vs~}Bp zrZ6xtFs!n&QhgRa%6rh~x|`V8B=zk@G2DvTR=t3>{9nLRF#uU^&AWHO zUJ9@%xA~#CPMa~tWJh)-wYhKg#3Fn&P$yXS#Vz_o`(K@i0CAV&3b$~$m|COT)nFGl zWd9*44*N4w{!-?l(La&9f`hyCTN&2blT{Sc{2qCgNxT zz)n~asLzhz2&17;A!Y0?Ea8nzE%ZkC^NBngRy?>V)M{T~iSmO*^V;&#CHRTeCJM63 zrkPQ+5-br+7l5xp`0PG5CY{9i7ZPXyRtM_Q@5`N~6Z1)_b89fay|;WWUN?5_mMab1 zythRp;^Vi(Gp?qaGdh1qDE{EO+nma}iVwtvUiV$$vJ=m5*4xrs+VryGp!3Br*AUS; zS+Bk;+dl9H>SlcB3@E2VwGyZ7OC^EyyFQ|K>fDN>-5Y&F~` zItOOfQbPELkOQ+kfO1H}4NlQ7*hR!}vszH$tid8dud2aPw`7?gyym$82i=XPz%tWS+2pC4hY}635GAZJa(~Hyk$9rcpv?CPeNI17?e~6e$ zx{;P7QUHzP8u(!E0=pm>NhLuJsK2?7y8gI>PBhHRBBv~gheJN%fBF2ocAXz;7Vbe( zIO%c@MrA&K&WIl$sGa=kxXuB8R8~*qFRdm@P*ky>A-yiCo=B$4Fn<(~o?M4iu)W{uMAbBuyYhxBn9Y^4-o zH^oE@!n&Xtv;wQ5rlS*x!;(09|ILHm=LNGqk5ge1vub^7@x_rIFUo_|-SRO*2d9 zbk6rsPXvqe=IxV$xCkI+9G)c6TL2XD4H5@m^z_1FFCVg#kSr7vu1=c#^;4JF+ z>p*BQ1cdamlp^#M_~j4`wLR|;M;8WPs0xmILi56A$q6*U<^1_79BXd`D*!i<;|68f zU*|3$TD533Xeu2#AyQnVD7YeyKpsPFqp>ztvLEp!AO=cBLdx~)K)_Lrl%H%k0)f`& zRON7<{`pHR<|+sAa>6};pR^KB1~?Hjn6k{gb6QzB0KR}%x!~CFu|A1=nsMdo^%SKL z(F9@@#5vBQDKvhQVvBf;Z3888Uv5RKFEmOYS}T9=`|#NuI+-cM(c97MBP##k0CB+; zLiL=C!tO|`8${y3DJR-SSDR{mq1=XN%$2q4nj>f!w8rbj9Zy`zFgmr||NO_bRI{wD z$h4Z-ltsnH$Nsb88^B)m`2NWD^lrXmy1JF`%W@I3Bi)0Y`*&GOrt_kQR%IH(;s9F4m{4;h1d96;;yEzF2|*g>mn)k}Xo zx54!;Q)6xi0t@k4NK!ZqxZLFRqg(up_$`3AlT2A)H$hmeB6EsT+c*%TLED`DB`t~e znGDyG03_%ym;YT!d)A4vh>eZShdgJZ3`5y-7h4ShOBV225CsX98EFI3>inPI_Jl9M zRv~I$Vq$~l=ifigU43K_UfQnmu+3W4_+kWZ!pokv%}rGhJHBklx);-tNNC-!U0vQ>`Tdya!IgAEH6LbtU zGg0c`9rYY7!N2&(C*LNUBmf};>=9)DEqvu_&hKCvu;fJc5P_1hjcUjr##}L>XP?$7;eKHgeilKRs9t-pGLP8Y4 z5u_O^!l{B)cpFIRI?^@cTuo>jW4~ou1@(l)zsVIYUUf-T@AK*{Juj9Ud#U2vtA4M|?3t+cH1K}A zc^)HY7yPy~=0fm5Li1Z-$vk6g`(Fk!V3Z7w6G`m_4s14Wn4>ZeA_0Zc?>5kp2*(7L zhcSjtxl;d@=FDsA>y6|;*f!{6C>FKi?V&?j*F#gb8u(_HmF6PrJ4YI^&!c0TRZ=g}9rH>tru4FK$^$B}c^O{y`M%$mgtZU&kE zWbb)eD7mqFNT7qk_{0^c6$$ZA;4lN&7oT-f?#H1&Tr7j*fCc~$ss1U<%= zA=12B5NRZ?2s|Josvaex@%LKKnNy>Elso6RRIT>2N6$DIqLucamn&zpzx-{B;pjjL zN4x6ftqRu@^(8WF3_?d9?`+C^x8_U$&4Ud+AxB(Z{hZ9y89kV^S^d($qXSx{PMncR zHWkxvBuU;AJPq?zv%u4?(l*{7cFQR7%O06YdZyae9$|eF_s8Td2As6Hxw)R4Ym$75 z9kMOEj@4Z_8g`*lx)omc;W%#{hvfH)!&ny3FV*51@yAnEqzE= zw#AvfmUyR{V;nNNz;*LCXZzxtdm44F$7-2du<=t_ZfZ7bUe%;DuvYNHsPEc^rmO zQ#P$=yQEI$(n&Rbf!H*1m+fk73t!UFfg{SyJK!^KiII@3Ndnm6ZcH=P!@rfNN`L-Z zp61N&z2E#>;~%m;{#1+TtjgYA0sO+hoR%lKRTJ3p7XsQ|Ko-03XkCVum0YK*q2XEE z6oJp!IAr=Q-ux6Uc%$w&FF=Ihaw7%Z?c2BKznyfWx4_I+!5+=VzaS_aWnS6A6O3%R zFLzR4dm*W}Kp4;fx4qMT99cLg!}Uv6I@$uDbe22+aW=z(NC#X&m{J8z7KBrk+jVv! zPl(i-w@ETGDiwdsYsG~}K>0Llzoakmte3NjOG-#c6`m;BA5T3;#EGh%l>0eg-pO5u z(Pcz1EObweji3Jxf=%EQiZiBFOKY?I_jdv4u>_8vbQ@z-JYR5*Rgx`@_i0k@NuX+? zY9!}2w3t@V7E&+Xvwqll=EH19lH%^ITbo9twGQdGJz~LkXC3d3jN}0U1CC8NT+18A149L4F=gYolRq>p4sp2Iy(Ts^*H#^LJ z=<1AVetEgW?5&hT0h3gQ;hlD~E3EJ{VlsUoYW3w#r`r!%7*@P#Y?Ov&(_z;xgNVO$utYK~lhg<*biW+j@nbo*rJtX#*YR0UHex0;IEliVijlVbAhHjf4VEzC|MhxgDfZ$1 z(NlHWRC^JLsRHGZ5%mF2b|WepTp~Z-$7C6^$Z$?u>TSKn&dz8LmABao;VW%`=dvfKdrOoKk(^W}V_&)tVqOJoR>%ILy zgpd(QQYtGcLPkXiAqhp1J(IGsQW+t$vLb|(GD`NQ6v;?3Qnt#j%t}_M|L1$od9VL< zbzSd!opXA4e!uUy@6WoGvh|D{d<;8V0G05(6VUE~{;*io&hF%?Q#Ga)mv6O?&$670 zFC$8M5sEu>9K~B)krr4;=^{mXXMBm_)rtK2tnk^nxsskYUFPE7d)N8k>YBjy)J1S! zTxB8u{wk6YgeKk|b1W>`(Z5aZ;J72UC; zzm#+K$B&GX=f7~uyrDM#>@l8g+cF8Q)?jJcCwME!UqYD~Z35t?(%}cncY|2#ZiqMX zC|&TY=AoQDa}ohq_;m`9K*oGE_u@p?E!rwhPEJD;A4Dw7mGnAVQRNZl8GA$@e;L}V zSlX(&Z|{Rw07`VhVsJl3H8}O|Ms)Y6@cXvSe)Ho?sjIJJ4(e|LWlT39 zibB>y%i?Q%y^(rBz6VZgJTF`lor@Kk-kx}V?LDQa%b*L`x5gKh0-xvdf^$h(*&WT} z$8F+Z13=(GxHVVMb4+^g#jBhPmzy&Giv%*6g9no|wY1Kn6?NsL^g!p04&7Daq`N z3>mRKM@b%nK4s$IeBa4&3DkzxRCz>p;$s5lWQ{M6)Vav^d~^~vRCxvlqkueFc-};m zqTil?fThv>YpWk^RgO_<_#sDRb7OshubhB>LO znJ)jbg_DwbVn-JMxIDZCl5jwkM}+)X&5ffF8$lX40jpwt_@=#h(aq3MuAs#4#;G#b zY35@MQ+zi@h`0kSvCIB|Qh&A*#yKU}}{o$dUV^(nR zb>nQvL;G$`9&^e0_d)pRYd%%<@FrV{P-HOD?~~Ps@DBhZi8TZClmYQ#|fM^|XSX4MEr{tz#quw#2OR>hiSi z&h2GrQ3@wr4WfF@zEFuG|8=&zmY3?wupE>FGaUyi~vSO)$o3nIvap88$dOo54 zc5rngtUD9NsSU8G}wai*IK(htuJ)gsID=OE#~ntGgiPeO8(#nB0|N zq2c_>Z`EqsgQGa?vyg5LB=fCV+}*pzqPI|&ZrZxFh1$IF&6|bsW`qaUOUE7pgn3)H z`bcJb0VFmT!iMZ{k}a;&Cq^ru=NKikE3Pry^3fBCHnN?m%>fOYPL_Xjlbu7q|M=qO zdr(LIB{BMXlaea{DCkzNE>xI}_RvMThqnkNGy2|kJhmj1I5_f9;26Y?upcGmWcFkt zh@MQn%x)-z+OLEPzGpCN+zQ%n4tr9sKAt>GrcE-ThnhcrOqxV_p49T}fwz&KULA<6 zq~f{)lmS~{@ao33)7evkBc-F8bDczuR;*RuOyy~x<7?YE(rkJDzThr>JlEp>plg@A zOV_o2xK(7w8*r_z#^P0ht;zpVjFU{da31QE+0Ey7zYq-)&JUk^#d-+%|J~0=bGPFa z*WWKxb-Q8d#l%zy{$=~T8jxiuQzN&jL8KFc-KjUB_Tub>`4&a8)>0gwE<@i->Qe$z6R_0P^)QNHlDT=cyZRdK znpTjpM390NNcT#dh!*jPLV>x&`7mXix`8sl9P|xwsu9uycpvG&+TEv@fUuLlOP;vL z^DRVj4gOdh^2`_g`GD97yMajX4~rFcW4&ll2hiD+jP+v7FNsaSVOWjp%pb)syb%QG zD8q5SDEy0PUN_CBmfig+z2{MlXO^gYGgVC zp>oLG?sNY_#Sc2jt^&xBdT12eNdW-#kFO~adc79=Wn?%Hdd}708)Y8c8?O$$DcY9eHE}JGLqr@k6D}uVfd_j zW^Wjr@}1j9;&ct>Ch?zyV<)zEm7;nlG$~1SK<7!iLo#O)4G;cCLq zb4#Hh>+#;GS_#5K6A9V^b|1+?V)(dOk>F@V%(yc~nlriRHd35Wm;n`krFLD#a42j&&TpCr|y5h=Kxw(vyvS4L+sIMbulzT&fRLAq1EdBPXY?S`$)hPq% z9524+9JQ*m@k|6vL&$xgnnO9yrr3)_EfwW43Cw2|5iA#eJ;c2 z%Xua4!S3}!`LhQ~73;&}g88k{s^hx6CvKbYfW*wq++)zUl{5b3kESD~WVv112JN*t9bM{qA9kAWnrkSu3P zR;*XCO!{OXmE)1yPvl8!zBO+YA%jRHSxh6Ok7_Lc2mR)LKtPL<}-g95ke(SJy{jC5g)70E6bb6|afxe;{&DuVfFo%cW8^ECmgNt|q z#WbClKujS-(~sGGC({&czZ z3W!*IN2@J<)`cIh3PeE8`9t>%mFzaC*zhTcuekRyD8RBie^%3ZFL&6l1h(Kvaa^_+iF^nr0QGt4RJ3w^L79QK z4ZScuf4D%jxYsv#L**SJyrn}WS;si0G$rvPE;6~)caH~}8}z&A2|rc2yyl|-C2Z6E zL_0hb0%u`F=a$R;R0tL1LI4bbM4-DS5FnWyN{|h-OkLTAqS6g1?7%vvZVN6s2)$gO?4GxfV0Tm4(E zL1U3MeWq`AvT0IhN({NWg6j69_@eJK2P>~vNib6q*Cb+GMnphIE?+J)>1|{G=a;Vb zO{`{7w_;GihB=#mg~1|ZLzAgcNhj~?yLnIDGHEVdWM<2gK1&XxIc5QTWr`C3mD5W} zsHcgElBjJ^!r22QD^6JQJ-IVx_8zYrN-jvGe|dOkKKWuU>t9_U0?h;dn8ju6PB)|} zUWxo2f8*9Q+E9^s+NxhYMt|<)#n|HwCA}aPH|wgE$pY6IW;Qd(9KoGHJn*?w$z=KE@raS%&oLO zVRt3;+r`eYoe>rWYUWGv`mx?8ijQL@p4np%_V-41a{jl2jvj{+P0bqTT{LrxGkk14 zBamhac?-jweB;iJWQ|5m+LFG4xV1(%<}MA1JvtND;hbtEO&%GnbQZ`FAi^yo#Gahz z5Hm>Wx^5}tnXfXD`hMI({Enqiub4$z*U`TG=y$INZMV$F^L!g;yslg^eW}eQaH&kG znxe|7)^zdDKT8^poqnRff?9%^av2aj*f8c;YeF%3!P-Od1)|%ft;x=FQfL>)tj!lh z{St2j?nY>exg#WKIb!_+t=C#S{q)m5!Mc^|lvvr>jr0aA0>#Yz>3+56q<*f_c5skp zev)1fmPn84c>Db)les;gp4b%G%Jy1wb7?vh0gR?tZmjr_p5EE}b1t;#Cc*=gNFknG(s$DXY2Y{!H!u?sTiuL$%}Ql{>pnA6#WD z{ko7z`3YS}Y)gK6iP4Ksv4>)Fy{($J8I@#|%wK7DH!LYGnFrSsXTh~2=8bLV>6N9E zJ2)V9xkmT<{FPBjS%>95a^9xj{O%m6v)xbig7TA)X1;$GD|RL8r9S7?_pXd?go%tq zbU=k2`0iW{VZ`wGUcwRzbf?hM$dLL*(hhfa*NIaPBp>D}trlmhFp2r@d{SrP!wt(v zS?tn99#gZAx~_NKRNt*~`~Ie!(O6#TM@%BS!aX9$UX869L(C14&5_wA2mq-hz``>R~G$uPGWW|tV=?~3{+UE;-Ky=Nmv#T6&#j%|GU zN(% z-t2Z`HtVXDzSdF4O$v%!`K;Fi6jO6|?Y4>IJ6O3{m4DX_$&N6d`-xWY=3NGc@z0zJ zw6T|gLb4%61H~>#G+zrYGARQfdda^}vnE%%UW@xVRc)zyRe^89t*XYx_vtNZym-o5 zw&DBT3+`HIgk-DyemZFDxbaOf-EgceFldgaN8N8GZ8oHs157TPEa^o^&LMN;`h%_a z1uIFEVvvE5LfGYut8#sj)`!|OXSq{%==w+V4p`~D@0MBD_%b!U@N#o=(c9zCJ8~wO?(qAcE)RVto1 z_wUyMjNM8iz%bmtm3Q(S1HNr#$7VUzs0EO%K`{^7a*siayl9L?Nw72d?ysdQ7jZnDlv^^ z-x`tn5j$tN%Ub8x`JrH*>jtP<1})D@JPo+Q@G;AyI9-fk?zUUzol5)ilbd-WxPllry_MGCwZ6>)6DT=e10g!&yW z+@PvOV3jCec%)gnipBoYHJfU+YCcgPyzO+_C5=wx63d3D=XEy00(o1mE?7`}QO{ah zSy54-*rKJ7VPb?lCt^;dO#mYL#HPtE_+D1q%xs`Di(O|OR$S4lz3WX=_71A;Ogd`M z9JMFo&wA(0VyE}_aTq&KCaMfBJpA(bM>mHUGNX7gC;(Ux93QLU*9Ig`WK2LU`Cv~$ zcth#G%!Ar7@uI`qe6F;l*;5%QZ4R^)8dPI?we3Tq97C)@YS7CaUgcC|$r>p^0g!L; ziVjr-K_KAnf{<*mMI0jX%@ma9gJ}D7Ot-`_C2tE2>HeCvl8q1HCx1j4TY99_^~no`>iHZBRDBeP?eE{9yl@T006xdXM%{FF8N~pvDh8% ze4;P$i9A^j8wzOb8JsyXyM)o}So*w+ku$NKRZab}=8*r#e-{PfQ@CW^c3@EjViH8q zAv-9fPh=*kP!uxTWgi}Wc*aovs3I3zp#{&Xeb9Io!HWwYPdyMIVtJ+XN9$2)kLjXEFuayyVM?FfM;TF5 z-||Mv!_P>0qG^x&!)Tuxww~1qHowzCl38NOYoAn+1NKL$A_g?zLTnUqAb5Y~k-<&z z1t_=w$EmnT)?d%RUXk|_9wpN&bCCQfX|{OKli@uIKXgpUlcRbiUiN(5UVjriamyCLp0L(# zlPnGMqk>F6Izeka3w94rRX_~Qu1^%jBoF}+8hseb0rjvPiZBWVir#wYyQ$$lMSAN# z{F2czwVJ}|N~y{dsux)X#ZImr?aLX7c~JCILn290AiS;Pv+;L5rocX){ql!k_#;{d z=we`Y`QBv}VS!;>6kjNa%t79K|MjaTS?? z@NR1Jn4puLn%%qIRXi~R)Vn{jU!^0jJzjG8Q-d0o=Fd9@#V#?YYzy6|T2#4-V-G`-M@qlqofAWaC99&N?0)mmLG7&LUar;Bg1S?jG~2h5qev$1DD17 zJ|?~>3c`DedOWQ2%Wgc_yJ7!p5gmR2P_o+27arX>Ms96ed32#1BjY!>(mbKdhX@s@e}PRMxD! zY!VBUbt<4g;k1EK9ucyn+PrKFZz_@Z;UU(?-FV!DVmb^GhR5$(Uj1A|qiYy7z_Th( zEIzUP#oqnF?D|?`cU2}hj_0p)P7Z&$MXg2bNO^~E(XvaXbzu0JmLJ{d)B#PKBaav1 zO8#j1h&mep!8jln3M zo}C_tfcmT^;ZJ022Y^`sE3M<@PsK?wU=?w&Bdg@BjjP=Nym{Mh?X$NQ;?1 z9Zw+-``!aiK`!_0XD#YZ^n||Ln3SVo-XOq)+LeX;)_+ZgW2a1>>Wb)7H{Kg6#Hk1i z!+yZB-`kFf4IoaXns|tCcwvVUuQA#rQgo8x35lYDXSn_uEf?qtQM);0s2eQbmD@^X#_*gA1y8O28HJ5BE zN-KZck^YXzjyjjl^xK;U*!9D=+y`596VH&)J|uPh2l=%C?$c42cw{;FkA&A?Uq{}* z|M_ut2Ix;`;O+m6p~4AVSu_nuzYZMBcM!{+2~|;WR99a04y&{>;KX{|LGl4fRvVFo zcD0dHfcJNLj%#M~kE2tAl zUYpC6NuyU^Mn1?ymE_@8#6 zb!5vgKKU(hd1TA2Pe4qHw0QjA2KKF!4BRz5_4-FV`~XO!B&KUvl!N4^E7n1qR(vCj zj&%zY92X$ZWyW407&U|>@a4R8EGN!RI)DGX__*m*1~J%caUz_%li92o(iHupHEGV+a1=;Gp;drA2@mJ>oM* zZLFCQ#%FeQQ4sK)(fH(-xACidcfB4%0A7qbZMiWECKUm0u>gg?EF~vs!vVW(0lL=UCGiPU*{1sj5Y4zoKU{o{8#F*QL@4533r>F~EHyMjKlT!^cqUO%ks^#SYRRh{_+rSKqh`(E3|MpVXAk2X3R z$sKwv!1PO%Gg#+mfFnzO{!nMxMh^9O8k5ICtDN(;9T!~w81q`Ry}!DN?*-Rm^0@SN zaG(mV#?`HjHVlU_QTEq?!XqYil0hsJ7DobR|AAJ;F;PzS^HnG&NSy1s;|E^04DXSs z)k^Q4!!R%sErk+^xcy0H1UTx(cUNPDSK$a_1kXv< z9W>ZPmP^B|G220m^C-iiDCS*H6~#}UwQ=AV^M#BjKdNk~q`;%Fk7Xawn^ zU4>|4K{1`VZ~0{}ZMM@py5C2;^Ye-ZC+gI7rlEB{taV+rpOej~LF9>oD%D=2hn@j@ zIrcE-T(^>7^4H?+JGF4lg{R;~;G&oFuZ06P&6`PIflliPlDSBfB3GE|KdM=z;o?2M z%eOj$;d>7nA}qwYzT?m(qChM$LO~ggGiCRHp(Zh~z`KYWAamst2h_L3fPtti5_(t% zD+-w^s)Ps!$ls+u?Zv~5$FFkLp}a;lLsjhSo~@QfP+UIR)hHI=k+F7klF!~>E%VHN z|CKgT3wg5%a;9x+;JMW2N0hYCXpn$kGNBG5!pQ(Aq9QElm7h}Qq0b@5X4CoE(m1WF;mX}g?Epre`Nj5Nw7-$fIN=sIRSV>i2V^h5 z@s;=jnKv2=L;-TET^RTIX2TIyCZ-yAkO=YKQ_PIYl8mpU^h4s~@O_e`KI6A}UPlo{ zEH%Ws5T@qL&Jox0FOP#@E3vz%iw&8vkdfG-tl~WWNz*``l{La}I(V%FcWX22=V?tU z%I1owo1RYt8F8SjiKV)@6*nvm`_|DWGx<2}kXz2-xfh4O{xHDo7z}#21IZr=D_RCT z?6s{6_DZs{%f@q_CAfVi11Y@cKL*430dWiy#yBXtxw)lGKY}L{od7L`+`rOWW^nsj zpt}2tezo|saEv<4I7H$Ok}=~4lPFnU%;#+67fb)T?7p|LO{=P7tcuY>oHsncE2{Dt?TMOTD|hI=J5jszlejI7R_8h~hy72*rRB8E%Ox4#8R65w{x7NN>;8(2&ecZ5uOsOOAGvXOv4co===I^~2V6GA6&< zlzdzFuhBNq#s?fT;eD&DmTB!7&No^5A6($B{j!IV_PqbWdYhLw5-`35nOty(tVRox6@2P3d+G_8fF_$*JnO$ZyZgA|; zn@~Q`HuL7zJu?={*Dc?eH_+0yD{~FV@BZ#Oq_sAwX-AfGdHi#RgYwAyr=g+w-jIpR zdvjQ9!(QCSmU)0IAxWF&tdi8+e2pI;V9-r)TLOL5kD&x0bB8bK+A}^_3W`j(dtTakkP~y>sHbv_9AI)b};DrkfCJ(?V z1$ZYE!{@OLvO0zcf`o#&yv}k0EeK6x0gg7E!m|@PD69+E^8J>rZzLJM80!4_;GOIJ zONV<_x7yP1J)VlmU)M-&lPhq__ zOHL#@e=4A2#YQ~-_<{P02-_b;(Q?27|x#Z~j>Q%E?STBhs_+QWc;3Q_;Fy5b*hx_q0Fh zUlm}cIPr;2} zN*q}E^`3NK6K$Na*7yy5D3p6JA<9OqpcHOJ)CaCv|4K*Pm>2@_3K}(_>I@JN!Y*fp zGzfYYmO8w-go6jQQU*#^m(K4F4d^<&q;QGG!)f~CE8#1xV|Jha+zmtVZz~ncDXmfz}w~0Q@T98kkU|vVl3aXmJvQ50_fBLX(M^kB;B2m8W^wueuF9H zT?o|WgYXa7{TjlI8+3Ri)Q;RHsFQ=CTVo|}3l;=2>iuf^6^xCerl!Vh82>@b85UKn zD;Ss%<#~B5&1Pm` zzaMZ%!Oiw+nGmVQ7iXYBU7NJbPd(0Kontm`pCA44tT~Y?FJP0CG*xsS0f+qOak~-V zOCLEt@O_hhk*K=>1Qzw4Iu>jF?!n>Jeb76@VCb4uKQ`+@QE<#zJiJ3*ehVTRfqSqa zh${@(yW4mNGqG5}FshxX5tf33P!RHo(&Cv{za{r-KakuupvvZ9;Pp*Fw7z_k$__o( z`~Lox%e*nI9GCgB9N9jdcq7pIHSF4l>-qBw%LKfDWRNJ!N-lh6LF+&Pg1(h(wtiTy zRG@DoI4atka9Fhbh_{$L!KRz? zZW}b6Xra2GUdPh6on3-8eF(K`q?!QkRg&rgOw)Y)tW-Bv5%GGUAlf$}aqz%_6NZL2 zpv-`Bw9X>pSV;D=vlP#T>|+#4g~WEw7M2vz2SyKbFPYZ;U5R6&-0|@_eV}cx4$52V z13!n@Hs=^y7XFE`|*xA@Kyaj?fCCvImo*5*wSy4(Tco0zH#d`|5zp%Fan(XaN zcX-?cGX)%vBpW68faytsPdq&sS0G{i&6!d){6~A?oY#y@AYZk<&u_Z3x$=llb=Q}x zkVqf>Xu#l{diu|So0Z3{DHOd!@3*e>$U2_SB*&`=FOtKzAq^k);8NR$T@=C%Qk-ml zPL&O@9Ke-7cTP1LbG788uFz6!rT04Mr|Inf<;;*DbJd~gq;5v+;l|>^f{wN3CcU9+ zd1p)MpNf)|0mC}aU-ydI7C90gqduHQ4ApxqCQ<6|1T({_k-cDSzLlTqcr`6@<8uPto1hE4)~>s|g~vBl|Te<{#k}kpVX=OSY*{i%y7u&F6x=qJT3qN z3}C|!F|lAIRU&5uOsIV?P9hnBewX_8j1(^w$(q*U)*AhqT>O zEWV|g>D$^9(7f%7JymUfxt@}B$aS3{_d%`n_2vq4OF{;d2jvq7)8K+`U+|0%UGpBP zAr$CxA?ZAv*i?|s+U<`<-0**;w3P>hl zI3$`5Ow7_Vt4}T#dn6!gG%YilwEB43`u_Ys+T^AWdiUDWMu(G9rLw!0vYOLFAeA@B zt)E?=;`CiT2RvfzgoON|rHMi?9jjp>r>FM~4qgq07AN)MWd;hQ*4lb{bXXh+65l2u z5O9{qAi3*$(V5u%_YG1Be(h}&YHtU{Sf{ymm>;9ha6A`G5&0BN?F(}LR#m~7h#%d% z8M1&|khxbEx1_Y&T1atYpqB9WJo8{pUIyno%@0gZycTF33)^Q>E-H0?qsXHw*3cjR z>QiOe?hNNc6eUItS>&P%evEw$XYuy(3~fnK4Xb{voigU7ksZ7A3_;5rV(r(+6-9gH z3HC~hHHYE_@wPohoDcJiZx+V^t0TCCGvE@^N$XdO2BX#x9R*9 z!AZAynY?byRO+7D-&CWbh`N-z^;DC%j~?}i2$4b?c+fSP8&F{5hlVg7H`?T_~YE6Po-t;s3&(;C| zelhm+$vlV>MUzfdCm-WF{rX@^GWC@-ZqS-zg<|oU1KTWi_^H%f$g6b>fTjT2Er-?x z5z`NQYvS*yi=Sgw6|jk+@8j0F|D3ZpAjn8yH9Z@DqjiAjLHCC=v1)f@4IZVK^r|&> zO2194^!jr)--(%b_KnDss<+$N1OxA<#i+uXAqj_u8n=~0uo)e+B|Mx8lg3eKk z@^eUMe+qdOGn)(TX~>!%9Azp2AF_|(uNs$0%v8k~e`TIa8aa=efL)&t`;3ah&D^BN zD{9v1G^3zZyt&4qvew|v1D$l}FYInPUHpb>b;O|T^=LS?A%vi9e-Hf@4$Z{9cYh_w{b?!BOW_?$Qj zpTIn?+>+t0>*TjS;ne6}WSCtg%A2~_^ZVILlZHu!TX&8gqp!UIF#Bn>VBUe-dopA(5?|+X8FB^?GMZran!Oh)t@NmMEZ*AOP#oJYUqBRbaZ>QU1Hb!|-K76pbxCP=8n0gw)4`OA zfhYa$avxoV>vPP0o%7-p8{0H!Z@J4(_#2amj_Vs5N9K&97Ui4_F^##hH0AuQhXw8Z z>CY$Ao1Io(bRHVLZb!RgN8#s&ttcjbIj9I|BpI8U?shsxpC1wr`EK%k{Yt==dUkT_~cs0C%G zvijZN(oAYzQ)7$9+nay8@X9QS(1{S*>WQ2z?T@%i#%~K6t^W7L$oipkmfQf5^`wm4 zHK{P`=d~6*kN>PcA1`9{=&zyqgffH3trHn#V!!|0E0%KUNxK=}T-@;_8E>)L{ETz* z9{QORZ(0vl8+FmKwJGz<#z*Ha<#v4$;rnk_YTv$J9(2xdOZ%mD(a#%3X*&An{s8re zOyAVn93l47ssDzzy7+$&t<^wzF!bR{lcN64rfT*u$<8$=PBaU0tS+7pHda4Obw2*u z0bQ?mL+CVa89m8g{NMN3SX|siBP1PXDU|&o{%d;Gv)TtcYAv_ zP$G4+@{c_~h|8bxPRI4wm>7mWb|w2q-pQ+d7TZL37VP4T9sd}7DRsm@)mqYl!Qo8y zrQm|~-1lp5{tnv5-*F|^zdqVD<-LT<(IDgHRnxH%D z^smSN`oy`WQTV(6ImQF$2Sn`3x>LA>|MxtEZvOL0tSB5M%RlnD_t*S&m>dm$&Fo+Bah>YLr^4Q4x z-&4~~e3L}atjc-X^?cBSRxHlmo_9YL2RCQMvTg5m-}vC)J)lfDuxb9N^f*mOj2 z?Zl*^>Mc_{3sa`#mg*USdSMx|#Z3jZBRppQ4p32O6RddZ^+Ip<0*K(V`rKc6$( zamHBo>j~S&>q$HO1liM-Yy<_R)Y!g2B~!uUrb>km*2z$9xe(LK`B4ZT=;~pLf*h}g6 zxEiQ=$nL)tpfRr8*mypYuetVD8aqYD#@n_ERWo_zmXK+CSA(3U(k3Z(^OkNQk{$Za+CV8>-M_|0)X07|KIi39<%AO zn=Fr;?2~oMcY{Zy7H0O|fKF?_bJGFZ7sofjeDcN8Aa3}pZZECH|9-}#e;>kT0uwpp z7#e;C&;vtht}CE?0wt@M=t1bcCnsBO|OI{xOf9MqKR3X`>ClRERv^ye*< zDu7-&Jn`2xFUrtRx)$(6ekW zWgDcy-L)l)_s;%%84ZrqdN+uPh;Wqf26WxAe%ZKY@7H5NYoi=vl8qbJmrSvJ6ScRZ z&yW7(P*yl(n{?}QUKQtmS8bMTu*Z+Mmfr6(p;E>c<*~V+xi<_l#2D{%x=Ek^+|=Q% zUdK~H&61Vq{1SYLL;?&MSXxcG22LT-z4ypgeJ1sqZFl_IqD)8HDz&>HT zfdv@R*-tqS`aYJMKPxKwuzmhWh`t+>Os@ZnD9z~$yqi;~r&8B*efYH1zh|k_$DDuv z`votk1<71kXy-`u4LU1gG5lepZs$7LBYgmn1cj1uiZM)e75>CpKRB_2gzNwiBj!~J976%|zhk0<~8D}U(b{cYP@>f?V0tv$vgV)=;UMiPzlUEAO&y=0?+ zQj^&F9y>4hF`8JQ@o-;uCs$h_oUIPu#AtjP4*ei91RO{Yk~&C)IdC!w%snF>ND@z! z_|X(v&f)}H8n!R&lhbtZ0e+uew>ipEBX9JxUn@@5M{6TI92M#Fhna>5;18jKKai0b z$p$JiHkP1(#I}j-mJc4D=td!n=^ZPHg*|z}y!{ON?aSbDFsqCP7JqOEWHdLK?n_2V z5YTEyekk*5_$&%39>bK$jm4t?UD(| zNPoHv6FJ5_44f_P1h7p>QW0oAg7eCI#8DO%--C{gG>+QiE`@4gS(eO{WY5&SXLKI@ z`6^U1de_1Wj@K^|nBEG1KtE(8$_U^hb~}8ZArRP?@xS90-k6mHaT}okK-d7q*#PdJ zII|H=i5D$Fk^&b<;4}Ca7!3aLlVLTGA#3pOfBlfLjwB3&jFDNovj6bswK|(l@?^1x zZGn7^x-|XbJ2d7-<(}?1iKq{rBHc-BEwrX5TdJsiF=jzoMMd}7A8@S8QuAag452MP z)gRgFO_QV*3<)X;lKqmq75X`tdF4Vk?PVc428iDh+VzmCKv6(=eBl&$%o}Td3^#8p zs2a)AabwQ)EN9G5LUpp(c#scMM|s0Wuqu!j1@tWt+7DVRGKwz+NTbdC$cL;5!G!{r}1>PqrZAb1T*wIhzTo3Ovoc09Zhw=~( z;9Jwf*Qk4S{|YIKpmTU-BJ6+v8MQ&JtsAXi-sajpekw)^>)I|ii_41{6r|RTBZ-EJ z0(k4ZKmPickM3@&FPUnV2b zOMa~dlCu_2jYd?Vx~!6S~%i<3=knvWb7N$A=gX|^KAI_N!BR<@zLfaqc8cu zIuB!_?Y+x`48&_wd3t9I0*ZE#hypAt5*pSXzK9tM$o{nR4ux!njQl+n@>I|a--xtU6vDGwgMSsU_gbvHnnE!5dFL{2_Kd0dS8^1OYg_G1YwN$$~3qybA zQpc6JtUT(ZJY}3&QaDZZbyr#Ms;U$p!_R;JwD`H&z4dM>=wsFQEnY};l;kS=yL!C$ z&n(9Yr}dFL%l<@XWQjGN@@?ZecZe}$i{+2IzZ7~D6-F3#94!AbIQZ&$Zo=0XZJr+! zdG4*73>Kd5Oz=A7{;Htwv!MY?XK>G+sUxSGSH#)3U41y~(40fv#zNJjGo|kz>v@Q) z$k=S7Ti@||i7K9HqtA0oiV6l>E>(sGEXlym7%#qCA+^?_CKpUXct znv_)<1j;2lN&Yqa*093dc)Mw=kW+MW?TJXb+t#l4pQm4ar+dn@gW@6D`iCWX{?vHy z?k%rZ912vY3cJox%ne+x5B#>Duu%wpH=0?voAS5f?X30uQH6+l``@?kUi8{6CiW$c z1^(8OW-CcaoMG?dpvU|MO{R}F>IX=1uXiqGLk2eqZln-`&@f3f>nO4)C?5>E zq@a5~q0Gy28quqk6(QpQq%_!MSx#3r15nV{b+*HWt0Pxil!KOVQcb z&qMnw>8y8e->$w5EE!R9b#6wg)q-b-b9_@7q%Z2nF1gsYQZuqnz0kZN>328vjac6L znCo=X^M7wBB}*k9)4CEwm2#-$#pjp^N`uk3&+^q4z2oMB7lxwN&%V418Yzs#ojbYbZP>G%kCO_Q#bzc$RkzKA zE*^XKCi)rc$%mTh1&eXh^jkM1FKj>IvSvq%q!34OTHp91;Su(To;}+<#9J5TXTmPp z_SZic9gfbqHtv)1;`b^|Q`VwQj(>cb1C3>-9$q8%3K@ag?oG1h5)kLqX6 z2OpzS$~cYQl`Y?U_l;W#?AiT5_%zj)mwgTglzipnDSfNn)>{iv?>)Y1_lp>xF9>W7 zP#U2PtmcW_RW>ZLtJvf(H?zXkC;QKz89Q!LzTa6}yK3dvy*I_5hnGKBrFR>*F-mU; zqN_R>_kEyq|@qn+0DE@9vB6T-$m!a@PCXhtV^aU4J*78~%OdWVLtm#$Tf}j!qjKy~G<1 ztPya?&{%tVJHxDPj%b4Al(%72!}AR`_lmCGU1dfcSK9u0%Y4;X=ZML`mPhh**|UG! zRjOwB`$lB^$1A=0yDxfisu>z`Z}IrbOJ?_Cq!AGsjvx4RyK1Mfu;RIMJBSPil7qw9 zdPBvXrdYC@NLcR>IN@QdN0`B6INfr)i~Zt+a!A!bpp@|84AqF4+f^<*-j9n$ivx=z zxoB#-bKLihOsK~RR|xhC3E6RU?Ix|~%Bt(7H81RaH(oCGPP(ae57(HOo&bdx+xeRu$-?HMvx-%C?7W=y0-rKK#zrpx@ z{Kem0vY;1UpDFFmb-b}) zC9PvVa!b!npKHwgz5ll7#Q=X0n}w@(sJ&4%BOcb zhtg11T@|qi>$|@0Sgg;yv|vhNL(Whk<THbx`(p%$IY(anZ zPk*Y~7bg{OXj3&}@7SRRh4QJN+?EVVYDp+!OP|to{G;0-X3R}R7Yl_Kp6qfxH^y-? zHvdWe7Ej`j8bMetLLnfDhX@MQCA2jQy@_XC;>R?)Zl8MTt=$ppT$o{NRsQQu&CfNi z4onGSlB@LjPq-9XOlBsyMM^t8iipyfTJv}$?t`!X6jO!G<11(VZ?f)vRwZ@v=m_n? z2dA@_J2uV~@MSz4C~V|gT9s@1WM$rdWP^2w{$g#nQsp;=xAaqu`C1O9bLSEs)aY$z zS?)PqX;w;c9F5XGz5ixL?0iOF?eD+Rg40j-cz0*_}9Z-4*2(G$JY{y4MZ&_`;@UzSnp%Z@SLkDW4?dv)Xg24!+D z?`*xGI^yuL{OavD`Mp>l)~t8S@=u=FepVA;KDL&+JvyMxBR1iUahY=R5w+RR7d>f0zBbN2 z$`qTQ-+s$=MOfEGh?1&4zR_%Zu7GsbbKT;NOcl}f&*k0K%yIm=Z-a3k9n2OY7J*}( zOtAPjO2TeIK3w<6M4GO>1P?+0I{Sm>u;39|m;o{1B}Vp7R1&^pyFwkI^eXwc^ATeY3z z)u*nO-xh8%JCG^wc*16C%@gBV!HpV?cEh`48ZTTc`>p8GIcBC8wAY+Y-NDwd&2kOX z$)!-XX|<_G!s*LaD?Wd2x;c-G>~YLEWqxa>t_=2X|HCSHl7|z8dq~o&D`SrVCt@Pe{4h!Q+dHzi~X*1F|$tB`w#w( z>-`;b(fwRlTJ)M2x=ky+;w094ZCYKFUi2w0HOkD9W2iG!^2M>P!QdjR{ z$ws8wBe#Y`_f%rKgQE;xkFRFN4tY2ji9-vrkSaL;;f2o*TjkRSp5#2RLl@@4-R!o> zt~73QeDbEhI&cMpy;s6ix^^hBcNBjRQv${ z&^l#Ng%zx6k53xB5v_0G^P94N<0{l#^r=H3#nqGdm+{YfN0awlt0i&brybRhH`*7_z6jcJi2fzI8><-K>qX zuvu~OT*bcl_jix;Da__)hUn8ho1ylsq=ddtwKlhz4Zi=e;1+zXYeG$*()w}}XF;YT zsl=ROV$ra3ecIXCdG}Y+m8zFF*cL6)h6ON!;t78Wp*^~HT9Z_4CY zjX^N;GtXZ=UV9eS(`4SuH`t5{>uktoKFo@vYktz9bw=tzi`j$T<^ijevLA0i(&M21ZB43ez3u+HM;Hg%dJ&gkEh0z8W|Kszd^+rHe``gI!7d7~$A3+G2>4vvKs z^zNM4bl}3}mDdNOH}=>A5I>~QX+GKW)u-?M0E<9~?OUd128oxR-|i~!<`?QJe;gIe zxt3H_FAJ_g-EbR`MlfsGmT9Td_6b(n6h-_Hlh_*`N|6!{%^5-Bla3^lAsNA{N6YPx z(23Ke6G4ZG+G4fr0@kSh4nbe86^fUQ)3%%v*yval;-`2lb~P)QMzZ(rLjXva}BP{%NP$M7skOh$1neY_4eF21EHv|II#M zcmifY+W%Je$3tifcK#=C8|npqpS|!HF6^T*x!J&d3S-S=TwtWq_)J_mJYJ5we}I+4 zWLANmh1T-c`Gl*29y6$&tdt+*8bfL_%E|&n-bCc-yVQGqZLfn=ujtwpMq)>hf};h_ z4s1Ircf}zvssRBdCT;H&!%{ErEdIs$A;Mq#_t$E5GXR#Bid_Q8W1cMq0e!8>0e5*h;L9-?p%hXSJ%x+N?1|MDhr&L`Ru z^CC|P#N`otxq+mOW2cp&@gk-c#HjA~kHItZ#q{Uht_zzD3pl(M9E*FojGmmdUQsL93LmgF{v=Qf1L+h7ugK1(Th znP@9qxN##2)acW@Y@uI(IHwn~iCVpNPl9Ksi$pS@Yl?uv!|wIXEsBTl9loAulNH+N z7@R^Y?&fNG`Xh3p;<>{h6}UY0*p>2!%ik=0O-W#eupi=2>V^{s(z_SIaRb>D^N!Mh zi7zH3ECLVeRO~?{KjGN>H{1)27j$3{vDxZH zpcRDXof>?<*(iieRf6V73>o9q)*bpRr1|B|y~v``{>J7L`oq><;@9Q7`=!3uZMIO? z)qO*T0xt_`MUus!p&{%2hY`q)zuFOZd-_L#al{k0PqRb;FVi9Tt@*+rb zO76C|N28ILyI?_k@?nMi+7yK!iw`GWS-%FHtDc@1HLxT#{7)$-&o~0vT46r?qdmrV z*@~1p)FyA#oVjwk4qk^x+Y9}ACLboiN}t%e1PAuZKGc0F%|K~S7Xw%VCrnfK$Ug|8 z$)cNUIwx80U`~yP-Wh0*rm(BwB1XH3*aQ>$%o{>d z_Vn)#Y0C+^(x!% z(V{ETwt5L~TAu$lqee7zb-fz@l3`L51X>%#eRm|c*R>jXy~r`%)-iMT!oi%jNqrV6 zuAJfhAUh?z``WHCHy*&ZhB;Rv_P|BIbbg9+IQ4Ys4jT{?X+HlrNxIJKGj4AkzM`9R zIaA`jBwhJS-|&fh)}kzj*|DX`H-(28++!&Bk!7^2dh5gXOD8ato67#<$u9ZmEWiXo zNuW;>PMfHkUia^nXVOyoZ~tXF={j?NOw}xKf3QFcAi#rxIG70x+-ekZmx$N1&11Xy zE}rGE%>3u4Uhen2F1;loN&9I}l))B?ihLjTq)wT>;MuW+y!!zycOJI#(^3ZK2S`W6 z7ijq3pOVunk5!5b-xz*tgF9~*+2N)zeo99CAMOfx2bas$zC zj&hNk|a zI7r2Ejq0Z0JKY~VoH?y2xy41cg+n5xX#Re&Lr@w3qreal9*8pdB;qq+m8@d^GUm!& ztQR1(iuNuCD(*24ydRDn%{JF5R6S>Z?~~-}H{w8GDNepGEFlr4;!B#WaKVFF#@Zky z(osmFfKdcztpKpG^(+ij`^hyzQuVA|$+06pe$vl*X6d^13TlQe`EMjOOEk01jdsO7 z)mj4t{?;|Ap?yEk!>A&-{q*07dk#A$sL%^sS!hvxw=UgAiAQNz*eJ^!0Hq5bI_&eE zJ@_5ekTk{r-jJ?1Oi8pT>ZX-*dBxfy`hxohax+ABU<&5XPeG$SRVTlt2^`50 z>aE%E=(d5^)zh*?SGL^aOzYJMlm15`Ny&Q{k*H`m+$?3v*j4!i1VHfCFzdjts%ox- zt;gS66%-O0+k>Ec7KPaW~hcf!$cxk9LFXCv(ygJjVj5NSRL|F>2x9^t3f@wu)V z7~y+pc(;)a_izpwbY*)NQD67earWv7X>!`L%?|+-k`%RT*L8(07I74Z1XJ}S1LewN z+7nT|$F&J9xTTd>ro%aYap%hgD{;-gd($1)cpUn0`@AX)zWbgpE!-+?jHnzS<~f6t z?ryS+mIImfV4wen>T0Wr?)vajze}yPJi&C2BC=Cob<4JuooBn#z@_OD@(<(BeB1uu zQ~TjrBspDwy59GTu+BMmO&1_eZo2nHlgY_C9CQR2YT^keZh zZA8bc$pFRWk?C?3#)@vx+1h=3m$FJOS)X!hn-SO=PdZ-4pqE-+9b4h zChn>P7K5nDRM68HI;L1aElA*)f6j>T_-Z)jKKI9x+fl$yOC%>O@pl2}uAG(+|1SC$ z!P-z(R;E5ni64Gf<1IIKUu%nbC4xLFV-$qDcmVkU<_>OW{r7RqwQ!J8F<*RW?;j^p zo#hCCGKcG8K?~#jc~(k8Ln9E&xwg_K@e-~tcjzhM!i4Z-){fox)a&-%fk>I^VZHDi z?m#p}t$zfSrz{gjx6F8cNysUu^O!w6EMRx?e2YrL6B}kKjKzD~&3ijQx`LKeS^Z6c z@I~1p8(3bOzE1JEtjixU%DpHMRJH-5{BmjTG6plxsrr9fKH95!&Ro^-tLJbn%(uSD zz38X%rAwjD$IbX5U7{%-_(yuM`?CLz5eu9MvCwg~-dFPCoEV$3q|Z+|&>ojixlgqD z#%bCA`O?T3c}2EFw_c45-Jet zeURe96Z2;M(~kQ-Ou|o|x41L^Fidv15DFuvs-l03OYqgLpwzs2BxI+*vRN|uF$$(S_?H7b<^w$M^}venadaipeup@wt7wP zd1-&Y*TXED)K?ZMs@>zFlXCOH28r)Hlea|fhsn|Y<2c$MaVa^r@u|o;m_*h*EdBDD zJqT1c&H-3}5Mt|Y;QSO6qD#Rd3DX9LYqEGP9d-PYmtn@;1HniLv*Tvs*Z-NtoTL12 zt>Sd1*~o-({Gz9<3in3rvCuKUy{XQ7JQ@~2|1M3ISH;W6hqq87IPA=xr#++3H=MJ0 z!qs_rTcQ){gXI#T)5D(y1{vJUh{09W)ckYZb97h9F{bIex!36PwG$*CF$o*=I)ySF zlWbfQe9)cS-WifKY_Qx2Tc?oRL$;O=9=kSv-!q979NZj_j+p&7=r^my-Ys8Z&+(5O zL~GeAv%Jgw6O(x?9Me8{DJk#?ZZcE#HV@TE6WI|aGSBcc11ficenGWCafk(5Qq`o4^<1t_Ppp7so~mh43+81RHV+9WH6(401`7m< zY{VFznc1<6dq1Flc(;=KNofVQq8SrF{FV1Cv{>h~=aHs`lXJ=m* zP2Q4_!nT=)BvY0QhS&6RT6+?w3JiB^@5$w)f)AeuxI#tsf) zP>5NaIVepOl*nQ>O{^^}F75*?>gricrtE8ZKg~3J=PzH5%D?)o$zLe$u{t3)h0+abw4G<-x?rwMQLpolSn4C}8D`gb zaDXMiNkhAZf0Ms=Zigw<@6^|?H#4nRQolTSP;r9ZSx@s|`Yzie%1JDVL6+>Yis}n+ zh7wy*Ws`8=uh$M*1?t2X11$jl4F^dMU=CbNNT?>fssG}BCDG7IF_VlLaJ7h>5JF={ zBAtYc^4n`I4w4uO7Xb#!>u-7_d0Yq+#>FAV&ru$pSgn%1Ej+K9HU_s>iX@(17PtPj zfVs=2q;mzezD_OQo?dKL13p7hl`0F$jV1WVuDuR>Smw&0Jh-o_fgC}ITi{NG!G&iF;&_5f1KVT`}k+?;=IZrY8QBvD@x zKfVSCIgDbW|8=S@U)q znzsz)izDyum@O^MJnp}amRFRkCo41dn4X#(vtKGYB!)sG*h9Kt5l-X`R0m_>EoXk| zL|WAiyx%$W;%>GL5p`+$&`yQ>U*cmB(H2Dlw$bxzu?fFETFIljRY!-O!WX)u0Sqf( zVjhDQ#>?ZH?l($lRW_r;+!34N3J(N#t~;35vA`5au5?VXod5m({Ecf2Nfq5irs4So ztt9gRrt~KOp+y?l(3L|~NKs%gz_$L1$-7VzB}U>S|0Wmx^`fC5oelcmEFW30;6O)8 zM$|Xu!u^`-Lj-)|YT9C0GDOa0ZVbvB_frX&s(8=!XnGmV_mfng7lxs8$*HOFe(Lqj zNk{nDv^;=$6}oK)@}{4?_*g1+_ujqR=&2IB#6QJw-;grTrXY*e&Wzql4*wgIjXC*O z(n+xm7hVn7SX0INZpD_2%_kZ9Ef^?Us{%vVm;K$ma|b4ef3Qfw5EwQxST{8h`Uaj_ z;j!;4NroA5(jx&OPd41w-_nlc3FH-ez+{6Yrb3oW`uHR@0qXO+9ea0P-laaZ07h>2bV?9s-LEPVnOGFUeuYmGPs?oE8Lq1pg06$>d)7P(>7rHuuN zSj(Xi%!u_8X}x}tzjl&mF);Q(JDS-oVaK|v{$V34*og(Z4e3qJ9eb)aWouNX3vrb0 zkS%=02FqrN6d!LEn%9XA=b4{y6ek8iB#syMpP4uiuVMCryB1amLTsEn3PYO;(sdL& zHAx3f+JU^z>*r%yYnoGGqG+^{h>;XI*GAvbO5gaQyx$6q0V(fXVW-Xz%V%qPzdVl! zTYNgH_#qtoHLuqT)D8gtc#)vI1U{Z;5S)R2VFV0yHhxh@;_6>TGFr3q@NlO6nm5Bf zkb8n}_W>g2j{!%~nX`ix=;Wt(Hz$2t$UnnIB0lp3!xf4OOgy*u>iTS-Bf(xWw>bya z7y7H=Z8R!h*|Cn(-VODMgkssYi^|@gIjO%=@Yx?rnwpv}T)&R67E|&o#H4h`ux_|iZT3571g9D7Db*#k){k`v3~txJ0FEz4OhC#18y(B&|)T} zUJ?nn;?kD^t$Rph2CyK!%)H#=xOj44cqbJYZgsx6xbfA?U?qBKn#);oc)I^>Q8)OP6a`LR`ge8y zv#CF;H}WrnRnG}olNiX@*-~0-5WqK*TOD;aSZOU^2+tXN`t0R$Y(;kjS9Gn`EMRuK z72nSpkobbC_ly(!Eb!im`)Dw8-L;wl_+s?n9fm@z=d7#H3M<_+=rgl`5Lk_5lGlh| zR*BA9bL@b< zZLQVw!25SrX)yDv_kFo<*3smTj3ivVxO?R2-?jy;a8`Ezfn}Sasp(mpC;q7L^o@t9 z(F~x_zOj~KW4gqaEXqOt;pFs{sSPgnW6X4xTDn8M`hDLq3bC7z)Xt4-pa>@s>y#~` zjef4o`f@2DAp(XsG#Qx(H499&l#IJEV@&IrEw>~6an&Dv(J#m~p+(t_Vfk2M{@2wF zi6XZ;#$fux8rUgtPX^Wvl?b@m2lq;lSMf6r%135PMs)uyl#IIXq=sxqN}E-jl!f){ zwpxg8w6rsf*ovUe`46KDmv0oP5EP+5yvx z<*dxiGflb&R640 zC;lPCu_0g>i^Z9*KMrqMHP_Z{OZBstW50e1P_zJ#UWYA&t7v*)b+I z7RDx-b-%XiI)9y-(3?Ei#!Hpg{_?4~J)3aU>{@ezQN(1~6WhJz-PS=`e}w2lUJN#D z;uL3nC9fVWu;FY{$N?+@J&3gj)^8mN+gy%@ZUlN+%dHZ{;CTLxzlF&n`1#BfDX)nd z0kh`Smv$JiP}wq)f_P!Dpd;2(=6F|FMTx83svCavSmrZ4eu-#?&0-Dsu*4kd#6*># zypKI5rGC@RTNQ&~L9U<6*)W0oo&rO5k2DG1BJ)b@?ozsRr?^8Zo zgl6i1;|J#gsOF=~%QZ*{#0qY7nZU*m4!+8!H9a}m12-i?i6%Zk$gu!&)B{>d1q`pY zsV?OHU|Uej6lBL;{;(Wg2m6&}D=+}q7N_%J}WRYj6MoUpS7hLf*%1Z7RoSFUtS>n^?M}IQ~ zFnxPJdW&3ng~j*Dj6ITD<4fo*j&^f4?OASrX0jeacz}bm!^H`etFqAbH6B0DxiI(F zU!W#?5vO!P;)>vASNBkAZX9PCI!k*oq$(F^wZZv9{br=?5n4)K@?T7a`EbdWk=dzw zYOGVxT@6F>(Bip`Vsz`}(sGq^ax0;U25XD`WdEbN z(BIk`YrINDmO^Cvug$M1I0jVezSX-ke`Hj4(K|aomHrs#Q6E#|&9dcY=nhhU_~V2| zgtUlkq4v}UU8wmi{NyPsw9iM^NOAzT-Es^hmp0q&ud(Fg98fb8s5we3zmO5l7lB*e97VzXa2`>leux@Gs#d((-z$Pw)e$-s57o}B_>Gu zsyK?MUh~yh9?X2^plu#K8VVGF!A!>#)!ogWQ||T5i>3YaK3_gO1t1K)-^CL>M_I*# znddvXDTk5D)F>0U_Th<7{5#3|4Uw6ji?lqzRu01bJPZ>Px%e;W%kC4i@b}XvyiW4n- z?snq4d71A0kB`1BrfJUK@7JIGtxf|218jiIEN;NkZ#-_}s{O3(m+tDlTvI*qJduZC z(-r>i#8=C>#hK<`)6SzuH^L!aT3X&$QnJmi#RHcMo7h#Z&dZ@er%}VIfAy)Wu{96s zc(Vp;K0lc-vhx*PeSXP-V3IIroO^GP3qweR`VAH?hWSBzcZjPu(smaRBsf_a4lTmN z_y%j=D7b7=vbW5XX1ZcXbBErLKHUV(SSF1(JuHV=%|aHPOfHhXwSiUg!X;yf^91kR z)ct-@2HTCC*?{(O;EgI}WW%4;vsq5(uS)2)Hd?y8|K)b-hXF6FsmuWKp z=QHp!a&3yasp(2Uo`Mo}ExrD;tA%Qb6vs})s&t&8jo=gZJ23Z|1=C(`)@pC3=^_XB@AbA1CP_JqmMw5J_&WGQ#k%mfxt>UHkIL6-y(hcrf~d{!m?vxX38}_97KG7A#EOY(6`*Vl#j2{ z(M4dWfrU#5vff|512}Cv{V+Ex%1tsqnfHBtz|`QDBe1NXiEIDEbDBCu*uSb|zn)KKES0-~Ri{2Ww~g2STegWbQ;eXjX;&8$4Btm^p>H zYYXDNwiK~ja#s~e{hEIA>8;tWuBD9s5QgC9DgZtw9k^M}+D)eVzI`k0u#i);>Pybt z(%fwJa5u@BDWR?HF;%mPq5i4v0S)eCG^Z~92cJQtTx}q3rE}Ffqb9AdYj~|$2+yv? zPHTb2((?X|OD^!-=n&#Vd#P|&Pj;rOFxS(v;gEQ-Ha5tNlf|8C(@r(nzJ|{}-@f{P zH_4pH_q)}xVp3Fs!MZAZKR6H_yQYnAwRq)uqiM&uZ{zioTf?q`v~<^Pr#Lr1KQmS6 zGZagFJPsJw-MQMnik`Z;A=}(zg>}!;Yu0r#)n$BzW#0fdubhY?+u()Y^SDYod<9K& zvtva|R$4iv!M^IluiI8q_pWUJ`olI2-_cIKV?k^fgr3;4Jv|A&Al(hmunRj1t!L+y zuCKg)I%V^yUQp_TnKx&W5C0O@w4nZtPtMMwtZn=yRv)Rrorm1S?`Z?1Rw@%Q&?<@z z?-LysiPGSRA(N_^a9#-d4vRMm_Gpr1)tSQKO!h9D*#-H6<`bU`bPLWmn;-weM3rkv zRHmG|NLHRE&iN%dzwhpq-rmx#7w(xAVb#ZtFi8w=DcBCe2#R81P}@+LNWMlsc%FL@ zy0Bn0!MBN4rFcWciQTib<6^|j`pm#gx}_tfi=JGJKlO7maJAq4kEioPIh$nv`;MO< zk2D(A6=j8$iIkqbB5ZfLvYvgP7Dg52zN^4c`*#s*hp47*W&6L!1sH=40VSN4NS6#* zD0y^pt?3Zbee_P}NziVGB{6;-)gQ#xf{Ft+%*-|s||^d z>Ncr$-2d&>+3zPt()TX!FrH=7wBY>jy&U&-R1Nm`@Ob|Iqww$!?t^g+Me$3fI?SRc z!nLjL-&;1aELi@={L?yo-Fo}>a~N4w$6!t&>!poYy^Dd&7N>2s@h4+)RAhyO}hR_rPC1>s7C(ly<*Xn4hl{P#7)NIiOJ5liK4K;wh8 zg}XmrEz4C{mDTrj=L1&*n0BC}`t;x5_y_89rS)lq%`O__4}*UR9raL89>%uA{Xl9@ zwIj~pf}TLhu~j<*R;5VH7xXzv2DBS~Cp3g8jgVJ?N86G>qwHS4NWv-6Btr3!%PL*{ zz4Ds6;ae24E$Gb5TrJ!G)vaM({$n?kRv);VdFxH~rz)S~O|hnKRn7s4A*OB%C^cnS z!6OV^BNKkXoYVio$60YYzLS3@M%7BsIU%iS-SLjQ+aDY4;JUbU?}EMBoZ56D;$}}I z;~z51G>4ch;2k?&9d4tcZKbiaDDdHZDf%yr3vBY+)`l$$H`+a6A0%@){?X2O`NIv< z+r}a?rNdtDzw~wbl=`g3p_L<-SG`egAnnDRm1u@{HYof3D%-{QgORqtW=c&Y|HFE! zeA7mWn_f$ztX8q=lsYB74dT_%zx(`hCj#O_meW{wF1}qUH&q%t{%c*d!Z$5)pviE~jS zn-4j(wmMK$46ENETOs)f_N+JETeRmY*)vowUAqQL(9UF^p;pch4eu(!N_B;NnG13{ z$JaOM5I;>kOp)@|yxm2`x}W7iZC_H|opTRN0?G;)AK%@>zP;gSZ11<|iCfuGD}6S* z=TpX|`f+7RjadpE)_f+bHphou;gTH5<)kNl8zjhb2?>QG5{fhxYrply1?U;-=rD;M z`^a@wf2F)rGc%08p|^g%m8}_#1fuHBw#amB&!)jl_#0TkZF^<3`g3Wl-iMEN1)dtY z4BuckMjfeH(SadHqhAqft7=aA>&~6B{}TE6r17lE)LZS8Mf>s^+V(6oH8oxO_~=*^ zsId01E;US*3D7J@+R8071ltST#2{h`6y4;%432zSXBTa;u;@q{U6dYaleLh73t-#c zd;DB2DR1t}9WqpvKbM|h^;O$I?uPX#N~WPDGR^ynpX~3xGheTmiE@aJHSq@=dDD~D zCn0_yM01)Xn!%huIpEJ+dy&^F(C5IW50%i1y@}{(Cj$Wie1~i8s00q#y7g3?AsOTVe(-H)0iMh>{F!Nx*f-{`^c5|-$`Rkb zmm9mQ2#!JmkziL}1BcyjFwyOSy7O`YA?X9>u$AQ}nXkggfrr0m1#Zb5K)?qAdIJH@ z`h7Cw=^-s1sUH>!OBWrE){%=Tn3zp42|hk>B64iQ0~htiVNE{KKkj1ehfEDdnv@jj zH%xcW++z~p+-*Bso%6Z+k(DU7dn))P>kOVz*FybW8^Z(-d_uS0G@zvzzMS za9eH#CthL#@?t}SEpVgPNSdU=t51`l29opf$noNd7#~1y?#1u?M)KNZ5h6Af{0MF( z#D;@<_QxjDaMC60^!7c$qIzeKYUpeCe?GtOyIy0!oal=V|3&Y|>%|V_44KRPE}2#+ zRld0Jrsu2IZ=)Y=T`{mof|Ui!s!6iU=t(wrl5YY$q!G&SnQ6%Eji3CMiDN?^1fj{} zA*G$qw;k!M?5g%!ViI#|H){hc)h&44*Az zesL^C#A3zulY$$#rgkCp{#kwljFO*X$Phfr2w(CcNNou}t|;)sCQ%c`oXX7h%QbE~rtEWKj;V%0*A zo(yZ7*1NZO-4+)jQeB_#r1iA|`rLfc${Fib{Re)&U@F;kbFs>*+TZKt(O+X3cGBu}f~ras7Tn2L2J?Ik4(C}sa(yQV;Z2O{ z^HXk6WCZwi51`-Mm?7Y{QQS!L`|pD;{BL;QY@ChYIGRL94Npq&CEjz}Ha(R5`j!We zYmvHOfsI(C*M!kbcaGJtN^JPaL*u>UGZO{x%1(0Mo%y*kc+gpNZP;qRkS|?3-oAZH zh$IvRTkPV`zIcs)rWyh+BqUfl5AYf)KXnEc(q0y@S!yZU`t|Ey!zaQ*HH*55ln|w3 z<12`@3-leuHU6yz+CkE5B!$zHMk9`??&=x&vngtvHAt(h!=ufz*xljRL^51AbeJAY>c@q&^oa|lvZ`%q@Q5q z-45S8pIEMoWBfjXCH?_=U2o;tm$c1LmN`QGy!73Pb+tzijngHRe0E;`Ls0+Zecn|r zt5|p6YF7_{6~f#a-!5P{_Wd($CINm)JbMCvc~j0j#Wr*(%9VA(=?hmigLv?#?Hlv{@btDR(THJfbZVHUU#{GjYc9pN`AIlP9iJgYpbPO z%Wwa|(w6{$!{^u!C$>0c`pg1x!lFPjXOM?cXeHzK#70S&T`AE35h|6ovd=`0SY%G@ z+PQw0w$Iv{{pH5 zbR3F*c^1Y+4$u&iWE;4H1M0uISuJYzE#8)k7?f2*WQ0~oSY=gD-q%_fM}IYKK8ttSz%lF zBVJs*Y9(tNAlC7@*X|n(db`(`&?MqYT001*uj^A za2&6@oaHEUp6H(_FbysV4iUB*Mws`P=Ix>*?-4(KB_^g9M3>Z@P0p*3ru6z#`h@aF z)oeU%Cd$0TPj1+^xTFLrDcI5g+X5<#n-PF$j-nQcsauU!Nv>U63FBgtjzkj|rw3)H z%A{71wO}NwJ_n#O( zN1~Llb~90c_%`|VhVaQc)GX@4&R975c*wS(bGNxUGmOmu-~s>~O>;QFrK7FQ@^ zk$(}V<}<#DQ*OlF8hkMLAVJ4Lc1@xUM+Pn&!F3ULgcK$cmDx*p5qK%L8Xe(Y#z=w5 zr=RElh19i(Cm|Pl8#S`7zJ74$hz`82@NtB=xK!H0>gP0TNW)pY-R5kDAxV;ufOZ$j z$wkZu2L&B!EeW!NLDHV%-am+<(_7^mI7g<_0&XSWnC5=BmD!^zQu6%BjIj+T{qtt~ zPV#Q-a@R_%s*Iy&BCPsh7{@)|1X-HEf5g8Pz z2#~h2fFjI05mnzR_-^HR4H%$^xQXqV*fV(4iXdt=5ivIvL}UlOG&^WuXJA~8iaol` z=6GgnU1yf|>Jy6i_FFY(Bi|JogLv^}2#<6CDU(!~6a}6IrcGwf2p$k|^oIho&Abii zKEy2TkcX$Z5ng@y64ntft}m!$DfoyP8|h|{LZs6GAXV+qz- zTwJ?vem9vG9=}|;@G|m*s2GtWxB(vicJ+lar^vko)tZfLP|c>-MLH-|ZlAMSm<4I) z+y9^64}0gjkh=Qv-~k3q9dUuqf%59VZ}{tE+R1|pO{t6O`(O2G;G>oPpTDXLt14>Q z`t?S=x~LX)pH3*df6K+Eu&T|U@BgZaP*k#RBltOYenZf|OYiw+GW~)*gZkjEM7fJD z#La2^zkeB6HkHkca5dHyE=df0o5}1d$L!iG5zOiHd&%H=5j8(foW729JPNAYZ=@nz z#;ti1TtJ&FKIovwJu%-Y41AQbhUsqyzHmT`_0|6gs=Po>di;D`J*dny~nj3b(`_iyT0}dO#?YKN!Bx@xBgGm zSNl5B7Ffyffn#tnDJhx+$$@#G7{3_Q8+Vn&?9LGcZ*MzwN}W)ne2a-!e01#@TqQ<# z`JngHeoqtCFq;TCjNn>@q~QvOtV6iQRoLg?op76`^OAJ?_?OskF_VHZneb2E5h&X6 z>p{rS;9qq1=Ch)zzHc}0-`9P$58RH24S=95*kNn%W7)QXiL2qU@ zHm&{pS73?OU%3ST&M;kze@iV5p9x5BIS;EhbE&0hTp$Bf9A64P*Or`hs)+_TG!T+H zWWmo~ywC;}ACa4c@q%rUJj8y_y6K|Mk+W|GwQ-YbqPz0!^A&M+N8~D~()a;V895SN z)5q}E0Ld4m9aP!L$;r4$9G$qZ>x09tTwwHI=gK7!Iu3>(3nXa^)I>R46oueohn0dg zw2~%vFVSEX4uvGk2WOV4`d;^E(?9QcW*Vh}Aq6%kmbZ^3d!?2l=W98v8}NF-*ulw* z?9?#5M15hKW6rr`2}5`__t#Z#HN$RB4{M1yuWB_2U~*pd$9CkSYDVHrXrA%AuUki| zFt2)jixCDuw`cTq)2p=#woO4=9YHS-Zm_qvN1-AurjNxJKM>sWQKu1QomqFYQ{IU@ z1Ig2BqS0OMx(gperc}h`ff{3uGdCA4ughmZKwGwX^B)e5(-q=K0LMv=JeKq4FR?91 zy_##0>ACygI{nc~D6+n(&s=-j^O#lA=J6XuZTdutg!jCHu+kkNG(!R~nL7vthc z7TVN?H8L`1-?@fX_Fh<3&E|_YfVS6tvn)|);X^S{%`;80I4+ZvB)>jBC&Sy^RAF=P zO@*WUQ(XO&Y3d%tDUYAB0Z9!Pek>{0a}DoJBZfWTw}(~+;T!fJI6%WWg`DbKT}Kb_ zrUYrHw>+O>32~Wth|Kp$rIjM#9=2C1bN1NBLyX<;LYKB-f{j=H9W= zTP23I0S9YOu2mzICcpMlEe|^M%!X6MZB(vFN=a#Bk;db}OurBxA5NZ0aQeTv=yCN? zccmV8CHgt@%oeAMSFeUmPWpiwRll$4l+)kSJFYzl;X>rsYHd^R$9AYbvDue7a(R5M z2w#_L!OX99KH|F5_nSZQ^6)$xu|{wa#$x{xhNzp&!WoHIsyns|WF&qObzaqNKk`wi z&FS9=a#ym=vxs5uI}Z>JyF+u&W#6IOWLL|#JqKhJ^ua9y40h}Qd0V({eNaJ z>=^7>r;yeZwEZ_+zb=Nyp}hNdbO1T390Q(l7pmuBUsZC8nE#+!6~1$2+;H?~O8WgK z-W1%_RN#Ka_;l+v7uijl=%m!ttX>uGd7-;F>xab2rAsHqW?viGkhEZ@oOIg@=pfe7BWV1kP1A zr3KS3P4t6LH}&!17v54_SCM0Mc=#EV1iuapPD=4Anz&h~UGGlYdK7CDYFw+YAN%|3 zR;f0bIHZPgEX9Dr3^P($^m|>aK}O<(1$aje|0a8Q-kX;6p^kGR=5(5ZcUN0~!(Nf) z`!$CBMN@@F$HhCeqC0o)wD?cl(fMYyskW($xyd9ajqzgh1>MLZ`4txJ;K2)}r~5UUdwv$32jmZ^>P4GnQZ#c0kew>23aimvtAt2K3}x34MN<8oav zR$RDR)K{l@D;Ph1xW+o@zCW)rHXW73R~&oC$ulCMD%j2t4jTCK;AJ+a!TN)_b&QEW zjvD?93{cjpUySvqvNC<~$=LITgNk98aZ}GvAn6ipwAFuvFby+Jhwf!qiNw`2#=l(2 zVTT;-@lq*XQQu+_-(u_1j{OClz1}M6);Z&TYH>50jO^A}_g|c7Of!>oz$&+7$?|}8 zi$~(lj_ZY(X5;K07tL#Q>W)LU_s<7TH_~qe>Q+wONzXJHNGswQ%oiKrVeQEgC(%(zNrqP`xZPE1`HekPAx9@G-gVP-@ic^Y`YYJfh zIxc7a{8&q8#@y=KCP$&e!Kc0JqaKy*sO`oMF}1)D*9c2?f3|Ph@r=Y1 zp?R0btm^ZYsAg*>cvs*1!d|z1)5&d*Ptz73kDDt=p?$N+j6Ig6Ws! zYU~bwzlu?h%x%tCEHP;6Khx62OXw(Y%knm-t-`xPU7W5c=L8z)7523$3F;>WEel*@ zLN{AVEk$_Rw!v&o1N82Z$g`|r0|~+d1#z{A>9aZ@DPUQzNJ<#|=`R{TWR4?4l;|eO zhN5!eH~si}F?@Y=-sOO7-*JvsgHk@JJma3o(Az4x4z=uXtudS{Mf-me#{`~;)$$`9 zdEv(+;g9W&*)zu`p4isO z;eB1L?@ya}(&)Hn^`+M}|N8@Lh2O+{DDA_?|F3VZxa8n#8Tmf*u=n9i72{ZTi%Xi^b)gTxafyvYK;ZdfMDaB~kNY zFrL-_T=}L?S0C{F5+Q%^e{PX2RydboZ7@0~C~v2xH4?+5+vOTcYlX=Z`*#!GcM8WQ z|31GQJTdb4Y_dO^qYB@#a7(E~Fo{bF=Aj za$pU+)W4si;&&V_XZ1j*v-`yN#Q-IOBO(^vsbtx!$xJG`fq}Im&lP&nLozS+l}1T% zg}U2(`?PP_FeT zl(Qie(sR8G-0o%kMqj+qiPP0Eytgk>EHn_kjPt4S8nnm5Vp_U_fj7`sO$F zhAg5jz;6Z%xs`CCl0t>N1sxfp7Fz0{{?M%(Y=AdAKX(?agDY^lw-OD_puI#i2f!}H zdo}GIGDZJP)^2u(sZ}M4p#AG_?JGM43B#kNTa0-G(L&pcz>Mj(b+Iot zAx$jJhKO~C8FyxlLszORSsu`$*MU3jH<&fA7BSGZKqHkjoxncU#FL9^8>%K=+kTXmm1CbQ=1J zDkg)RS_U2-IVEBnYD?vi2TiAeQ_?xK>oBC&xE5GKp}3Xe+<9G|3rfr~$Ck7hYMG#- z%W9l@V%X(A`>Bb_0b-iE`WlKt!Os2r_lF_ek4{70xm927SC+9(m zt5jX+`{}Y+S|7;#WPW0^&Ud)|7DmAx@EM@$)Sx&Q3(9l;dVf3%3Bk9i zWjF>YLATIEV@kxvE9diN&n+mTDXt|;j2(oE-^pB8JR(Vjp{9{T7fTL3DVPBAZeHks zWv`;;=nQEJxU&X}WVh^oMvNNWhzj(oe zvluj@=P`C*MqXcE4P5IeorW3&v#dJ&g71$Fh|pil(A6u@cL%e;Y&#I z-z+Y!339bvo`ti^_h#c=EL1XjqAgA-=+&P`<%%O*71~9!{6F2=gLqG)uZ4So4EBDF z3ThXONZsw*cQ1SRdK|!M7{Islz(<|x*GftTSKD(@@SeY@*%)#Ivx$`=ZP*WpP_H>hpEYaA25xh# zpBI(QK8JU>PAMu3eptJMo@u_hxO`hMnkpE<_E4w|m!~<%#ObYAxsvrv#WDzT(l9i) z3+5C7#B{1hGPg5c3DP-Pw=ONglU{k_*e3yx;dVJL6yi7{=lw+ahKET;=g>R44F_Lv zQdN-P&GLFGBrvS4Vy2NWSseFlq0dxktwmD|k3{LzjM6HfvK`R0)ue2{4EytRX?SRk z^|~veV8mv^9Foi?joOf8d!t_o%?c(?mbWhL(8TattjyJTV(!QWUW;|=(#pk0akPz1 z0$YXhD=DNWokOxx;S@#P_fUgIcLUV}8TeV9+o|7vlg18}4RNv9wjDvPXx;(T)z#BW z&oRhqBWh8SXbohDCrrj$-tE0GS(tGOF*!1Y#<=^B&%1H6@~&Oxw?1p?Apn>(J<-EK zN_(-+tHCdrGX!j`#Vc}0m;<>y`;k4rAh4A^I%k)38*ZN-Zt zG{7=Rl9H08`TAQ*Uqg{ceGeph)%va)fhg5Ata&WbQ%Yi$pKOt-)wQ} z=UQ`V_iY_Jvb%(=ee8+HU5(!UH}vgIk3YSQ1XU={-J(tuvzl3`3PkE)+*NmIMU}J4 z88O^3-8XYvT1JM6LYrV(;;!MqPk*^%WIQ+l8^}W42UKQ z8TQZW>UJwV?UCTMD-+xFecKW7PAz)e{ubsSytfarmabdK6EHUZ2EARI(#7P>4XyJIDxjldE9i`bmtG&}B+r=y=z}>C&2g!0 ztUYHr6ilcJiH4dKcNL#ccgD0DxE7%JGs~H{usmBcqQ!|ZQ11paIwBPHp|5Y2%SadJ z-CDfCU_4gRxI}<^pX`Xrhi`r)&A$9#E)DG`@BnHMRgJEl2ZCa#axi3;1(%0$E$8Ip zyXRh2F@zl+t;$M_2&0jYb}$Mcw$wlwPXl-B4O%E_DD#YyXxoTezxCt)Q{I)vQ<=8$ zhf!&i?3Jw;MI|yxS+W~Bm6COkL{nZwgfP|E%GOvbHKjyKcI9vgAyH{s9qL$1D%BC% zXr%XlnRhj^lKi)4|y}Gh}eB1d?DJ{u9asYKp5z)};zcl=#H%MLgp{}aXFM}6# zS$Rn$PNPpibc1_6)U#3)xUW8jqiSaA5u;Y%kyBqy4#d%8vvuMSPDPCeyeb`voNu*S zCdtQ_musuLa>SN77B_JM`lS+9Xw#MeHEoIeQHFGmVn!4)I}LzAO&Wb|VgSc_U!L{J ze3_=f5A?psQ~NtOp*WfadOU5fHaUR8YwLIo$0utbad*KPKx34{KfH=W(mASX68s8J zaMb=7D4{}Uc7C%C75#PNFXSM2Zs(Q%Yf@Be2Woy4y6v1%aHjW5wi&@UG7Ym4JZ7}p zQ*eipN3;#yb`4Rq@Qk9u6?+AP(r1uUM9z<6^3*FjS=;f{6W`*B8f28Q*99dsGFcn) zLGzn~V^3!=#`whj3sk15g8@XfB)>LX3Te-I_n)JPqSqNtN!uZ6?hTQy83~dGyNsT2 zjCIw>n8*^qT3qGR-uCzf&J{Ny^u|G}B1Lm#U!0Pe-nf5Tp3U#Us7P9HtpPaHX=EuK zj^m!Q(DH=B2Z~RD&0(#b2x?Jgv7)5e(P;IH)f~Z${LF~kx(j|YVhd;O=Qp_sn;ij_ zk6AB!piVifiW$Ifb@#MxG+>fzP};s~1G;@kdeQKxc>-0ut;zSqscZ#M()8cxd*Pto z7{V$ zn7wq84ri%*vUr~h&u~SHHDRiaj}jr8ZjxV z4R!DD>K_wcetzBOIy(EA8EGg1tq%&W(xOGv$kcH)girIsY5f`x#9c>O+z~pl?5l33 zUa;Wi@r6;n>9Qp*epNS0v<}H$+FJ_lvHj!2n3EvB{Iw3Hffr-0&GyNlv&0Y&(7-@R zUeuBujBIg(bRrp<=C7yF0^oaQ)asx;MChnaesku&o?D$=#SiCh+b@tc8t?J2cR6=C z76zh=GH}~+=W=5*K_RY=-`f7W>?%{>NeBW`N!lDM_9F<%-i4lgyVZ~#B5w1nUP~1w zfE=uH;~GN0EwG7Z{}5KfaCcF36^o4SaPVY88}rNfB>2aW%V(*#F8-4hg2E( zE*k)@h~v@nY*tg)dB63}9ih#-`Fup_76gW2^osQu?rWF_+1wDFah>N>1bsZwQO$+j zjH&l2sa-rrb^22OlRsl1JHW;G(TsvN^d$zT%Fq);C;ov0PQujdw;v<6*0ETo@P$a@ zdQju|_SDR$PU?(1T&iPehB5GUjgpc8=33vpeY=q~Tv`#3r<{p<|xm zrj9dITR>`U!f5u9w^!~x!&RCJjP=Zb#*?`)tLvtk{1o>4B$jIvk z3ZbpEA=3KuV92*O@YLwW;l%W&2DRv9F+u%(LrwT}0lOXv4kBpS83S~5!Lt?zzX2z} zY90TcBcNH|^4KDkU4bH_@84^=*E1;l)ILpD6WZOyvBB_gVTM}=oaa>OmkIhHro6md zcKqwt4#bY^pc6`i$kkFz6H13;qKi~uriceOC(Jy9>q-(EUK%Sv65dCo8QV5SkuA$VmXhDqxJ>9JWB{jc`B5K*;oIgImpo_>a*MI(YA(*Oaw#*WJ_3$+eQI z)<-KxLJgk5zaWiTuQ>#pQUWA-!i^G%sP2(-xBV*V+&(coKz%gK%M)$``O8}^F>-qZ z?`nASQCUk%98OG#SNNe7MfD@hzv~T;pDNw}8Rn;C#osZ5YF9W8)eTQ;H6c8yIu}eyQIy4{);gtS4*aK-tfCDKA3N?92L7Wrt17r(>YGc@l z!QiaoRbzR&17V;rGKro(dj)nNim;85c}r5NTs5fMF(oZ%K_F(6Gu*;w0_$(qD^G#5 zE{?{4aX9m!!<0Iy2lAiqSwr{&py0IacViJP{SA2Q17t_*(5;q`-nD=ra4g;Ij7R|< zcj!>n%Q3rA*sJd=-O1FSivd~~5he;Hl~OS{qB4bRf|!Ah_EEDT8@cLV;(EhYQPOOX zOpQta(MC|Jle|imiehM=fQA3XT{C3F#cv_MscrVd^zaJH-8;f7=*)(&zpC`d|O>XsH4tp861lV5wiX9->?} zt`R17z#+)ja00;e1>hI~ne~BJ;U^vnUqWh-ea^jte<^Nay>KK<&hqFmheu|r@a8AObkESb!0kAni zOR9v8+EZb5i+OYMUQF=Edrz+hmsn+PBwP$0Olcybq^GOn2bKQYpl#NGfdg46O(HeIxy>1sx_o>!c+VsWxe}R)>Z8!L>Tq^99tZ&!8&wl4STu3p9T!M?;S6)7ji0Yu6&98r)_ zcZz`dhzde1H8E57iLsG&*4E8L1bw@MohStCd{|5n1!QFFKqL6ZDgQEdtO-!Sfw5v* z%EfmE+NiBl%f+(?zXyKB@d{?FC`7`PM(j~PayY?CL-@|rf@4jJ}LSutYNy9tm+5Lzf4oqs*!OQZ^}5loEBli4DE{OsA^9_+@dVHnVB zZWlV<2rvX3p_$PG@I;}(L?Dj4gIqvOQs1h1ZC2K0$qi|+MgV(Iweqf~7T-aL-T?~6 zR5%iKc02zR=>+*dIJ)R>ghw`Z&Vpv#k+RSc59S!#7M`nvUj|iaA|{jHVGZSz04-=B z9pA?hNFIF%UQ|v47Xq}{R{E#KN!@_jfaPd$;`&q0D@ty5l#yuOcB>UL4TPa-5?(uDctDR9C$?>auTKA!dxq(;Jc`HDWMP>~ zzZGnxgKBhGRHn9DbvkSf!6A^_t=Ty_0t`t1(I3%x1TpS328Tge zAsa4;K0GGsXou_ueZ!Nm?c1`J{I8-E9*lQr%aUybNYYFD57{@G_WHqmp0s!Hz*qv$ zTa}?BN=XZhk_rpe$+7?(fwOf@s3Yh%kgPplw_Td28@PjQ{Ua>w@^mLLE>;92`#tVv zxxoZ?uI1X7FJ1_a@ny{WDpMUnj&R-1Vi^G7fhQASkObiD!NQ37rS+@k5G~@V3U_Y(=+UiNg;2 zMH}^;arRy_y;zFnC@PQz;|LQ0f{M+1Bd6`fFyQvpBW~NcAQ1tGx4@&F#huhyR{?(r zB%P<`LzGfW&nM;dCZA4o_2Ww^DvIJM{>+wSXml!+UZT=#iVz-fwnq!0p&)z{+4*qs zK{G+&As4cX3yOr(57#+vl|fG)Gy_IHu>19Zhlg@RZvse!W|JZZvKQmlsW=q7#sllR zKRj-SARk*O> zWE22uKLTsr^Yf$TQm$)|`X5-J)a|=)$}cUksMiNO0}&8QRbBH9#4p3iG0ZKt$8ErY z8=X@{yf?WWR*rLB%DtxY^rp2Rr689jqMma|`~|Cq1hC9naQDjD)ko3b4o)y{wv9bG zpnu9S2@JekQwXTdZ0+^&kLj9YF|D~aPVJ=ES z)Ta|*Us&lr4z)pX`j|aeDPC<&8Kl^obN*&a3;uiulP3&<%P_PLiiAOinz*=rDI(a^ tOD!t^`TIXFl4t+<_kZcO|F3`eHYXubL9R!(W<7(Km4&r=v8j91zX32ySovoLw5_(9n#$;9g=cr=}@|)Q$Ro(B$V#%Zoa{LfA@R; zVGc91_pF({;(6AyBa{`TUZ4`7!ok74kdYQwg@c1f2Y%}#BLRPv4CH5k!!u`9DN(rc zVWMp~I5IdHagnzk`up=J4P_?R!bkN6{*l9XW_!Lo&4 z>1sq($#c-D#ros0R5++|<#SX{+Jz3!+b_>QyG}J+YI`5nj`$i(w+#zfw|1TSWm&Xe zUN-h_3f%71jA+nDKq=yoso?(}zKJaH$>2Hp{`>FKffn?M%|(>1T?vE{AWjC61`Z4; zVl09G{ssx}QO9HRpuB=S2V& z`YMv^0^MIzBP5pU+wW~_r?jorXYJd4YabtOmG%rz4#dI$qjS=$i5!5pm&bx%yj{2~ zrp@xH_L|?-fA=V#&SRS!%29l~;I&&e`DV9NO$Wd2D1)p#&ot4X>gc?%B zt`k@DuMfGX!P1><)P?M%6o9Qg2m#K+d|#S?@g@9+Ej3NM<;`gePNze>6?OA&vsP6< za0p|`)gSK<{Y$(L8oy|p#!I|eINV6LFQ3w}Z?zg=t}hk3{WIbwAoNNaBnzzCn%C1} zJsH?`US_WjthSPC55C1)=OONQu9x%f@9O8>bV8Ja5m(j{l**Z31|^X`9EDBB@E>Sq zcy6Rl>MWcj)zvpY-0YS_^DZ>ZSboVUzQ35Y9~C1!KYz00-Q&%4yA{XTZR3Bhz-h)DJe=7N@+P0ogP3n8^tMAv%+Swku z?$=Ih9Qr@rmIz+X)y$a03L?EJw_}ZgtDKs1>?J-QVXmK3)$?3WBvxAVK5S*|wr#uK zC{YwVpRl_4V{z!ANW#Vge50rr?E?IOJ?5gIm;8H`J6Ef*6>TSp6>ksyE}Jc&OG`no z$`u8UyR8I_5$S%rPV3c|cq2Amtk}Ful$aJCdtEyRDG8aNb$BB zX7q^uh&IU@6>0e#|0sC}m`+Kbg{$YHQoHs^uC8U`>He7)XS)WEO-&nV*5mzbOZW<% zY=$3BPk?A;I|qZ=uFfm`TQ$4_iR2m&ns$q0eby)^vmWm@QLBDim6Y|McPu_WT<%0N zmVYJjIr{!4<5a=kOYOX6Q?R1V4jj6kY|wTy?B2%d#F~`__~oSU<@~!Ux|SXFiefz~ zUC0~_p-ts?^wpc$#Rm<+vr)10n1>&%e#a@j^E>bm_o}Vjj~ZE5-LyLtsw4gnr&2Ft zZLq`*WB4}ke!ouBy8=F@*?IcnU7^xV3usp@+9k1(>a!fbZcn>(Qt7k9oTK;@t-Cu3>M5 z@4of<@1L9_0&18qHhiZU4g=8X&nDzQUjn8UyoCB3IA_hC=0g28GTavmHxZ4lx=0_` zM5DagMAX$sgl=u_eMHK;Q^-K2>?5AT&V|iu2(JW`F&Vo{v2ZAoa$&`?opB}q>xLPo?#p1*Yk zA|~227!|XCkTo57$y0 zTtYWvu~X%-m25DR>!UDX-7kvOJ}`_w#PO|=Dx4U0mSFgJsGZ;4D(SR%07hbnv9mGazMEXbK1FM!89VjIt|dwOI2%QAilELxTI$0!ztVAbeKrPZgY#moh>tw74XP}SYoE0F2&}se`!tnjvthS~?d?2^ zn5a{yzbDl)OHtdrfwi{&G4M&`drdZ(m`OYt`jJgC3yQ^de%8&z7b`|FFw&`NlWA0f z8F@J5@+Ev#Co1E_8vJ)u#{+mYCS^U3<&bl!6{a_4PEqSxVcT<5=v`bod{aOqrC=d< z54R+8(7S)>gBNaSfX8`G9~ZoG>8-_HjsD5h@%2hLW$`5t`jbydE@>qCtN5l2RRG7F zS+brLqhy{#M7S?9n%IPyro(r{lBc)|T?JC~;yWhl?kmcgm0KW{gr9f0OOFooAL^1$ z)7Rvrqo8*leKfesUD&VxB#r3{_^?jZ4an%dtH5pm9_BHv(U?l2+FYP@N(qNXu$k0I z^mNMx_dL3HKWRd+QNNO-kRedO&j_E%N+HlX9kyN$J9yIg7J3$X%l02+sDH4}n1lx8 zDgj^Ll}pHM=-g0BZW&DohL|&kos0m9`qE{}>F<~CLBXoZ=f3EBc|+9O4iw8W_c0?| z zK-D=m&o07+QKuVI6|%QXBy&jAMux7bW=UEM`1n*D+WAHdH>M=u5;BBReDOJC#z+rH zA}3)G5H>HomhE{QeBpii=<*rj0CRwn2`MMvNFv$krCBg6OdJOvPoYK2n2>~X8j$!jW#hBv%Nr7$h}K@~@)z?&ZW|WiNs1tMTf$PY z*PX-T4HJLA&x;N{KEC1EphEWz;$Ve6A^VQOFep|~q}8XcNH~tZNJ`EiSA=5|ZU`5H zIrZ#&i{(Niy;;74ze`F3dbXV%$ebq>D32NvpIg@zjLu;Hh6Oo4#7Nh0IQ=g#V7o)^ zyU_H%Ki}n&ti{8M`Sha$COp;lsw;MQ!atA-UClq{6b1Os05d>?H?u{~7ir3+kiJLv z*c*VJQXq(sp>#nFhd$CgQI01BVso83;$uz~&5@~W9-xm;uSN(!cr|ao)9t^EV~ZbN z7tak!ts0CWv;K3glezq5Usog-$bY`qikoaC$XcHYjy=(7X_#V4F>*nx~5#rE0v z;?HES@WZieh7+^;Ep@EGiFECp)4-&H)>)8JswY^v#(+;xeM|g(y|wo&BLl%PT~|qg zgX}*Xsv`>7MF9_nubw_PEMx_%N8nPmy&)F71pu~2e-49n@k@{XBZ};^;+wly!gme@ zmM=o)6|q89STkL$o@N{CklMf2vdE~N{&jlw_st3g;Po((=KEa?DKk!NQ{CU`d#>MT zP9bW9`bFw*j}3pznjW%_W`!T4>(S$Pjbxlftf*h`a;41jljX{Ww1lh`{#`SG(=;d%a#ldu9dg^m;HVj$^tf_R2G2AIy(thyQc!TY=lBFtD7zfQyWQ>3R+b?u!Y1SJ3(sk^2x>a`0x{)kF zsI!{Mx@@JmX=kMXbwm*5W=;PaDu&>#G`73DP2q|XFQBqY2jITWVl~?{ko9{)L9wTS zF)oDXy{)ei6UouZ(`~9{TAlMhi%n1`ug=niBMkg7Pq)?Zr+%Kos&nXm`NEP*i{HHc z{vy5nT`O)2UN{-)Ov(=PqWnvttEDH#w%0VjYglOIQ(ml4o~7E27xzS+Dl(4a{h5mrP#= zBkM+G9tdHcg$j_7KXG2)6yjMVLK|fr@|G0VhNZx5lIfS<_7v=&)9H)8y4hMErp5qu z9p^4z4`%Y@(hDr~9{ga=!s*tcCXH1~$j{`P;oFWIkO^ow#(diXUIaBW5H))%P% z&cOG)O(1GX9mIj$1)y6&V`9;Xx*4mIo%{sF;+Jo>vS-GzWM_xx(>SEanWSbWdeWdL zFJCWx(EjP3X0RyMygL* z(@Wz0_9IKvd*5Mvi_a>;HP+sBuz!-wvw(6Ggwws1kF5H_31>Y0(=Qd@@-^O_3^1BM zJ`sosJ#M9Dd7Flrf3Wz1374zz~xo5cFmL1?K{OKJGc(v$tS@5YARF( zrL7k;e_1Vabsxnq-!e0KeS+7^sPE9ZH zsMcFC2`?$rFP!*w0V%(;_43&-r3zMNL!2T`(v45!^UfoJyN7X7yTO`_+I!mefs}|W z6sR!HU2J2B^Nf{WBZ=miC>U9zsJ?aI_Lm@y`_CRid));1-bzu2pt=9+c#wMkQsAB3 zotoi9GU2%ZCt**ao6H0C<>r}*KILBz37`Z#gzXdCVithZYhB?W$j9&neZFDnN4Jtj z)Xi6Uu|iFL5nidhHz}*@hlm4_A}YVNcrbz*aM8pY(F6`#PKyEL=!R6;4^@zrunE!p z$VOR%r>v|?ijyiAh$a3bW<88+du|}(mg;Iyx0TxX2%fX2WGpa+p{Q1;Y1k6vUnLxq{ z`&`G10v(Gb#HiP0EnZ5IHwpO6^$VU5dV5_A@;>l=7(qp!-}U*ukV%xnHY`gM;z7nPwgqnev>a!j$BB5OF_H$QN6YzDg+ez}&t zXm)efuItCvlso2?1##SbPL#jO_ z3G&3{rO{s*Po`|}n=c_b1Vm_8NOy@3n5w`M%!n`rNj)EbMnIK<$p?F}kc|zW)2t62 zL`VF-goICBHJpsb0<5jyjerBKh_5FQ`oC7&7A{Ejt-H`R`TrTH!u3 z?#005=0Q3}FBOLLeZ2@hlR53Y@8C8g4mW$RaVH05z=dN}FPcMKKy}0FGVC3p{t#7{ zW`Kia9lXvE9%3lrDqd>sG;3QwxhbVgR}ysGUucPCs)^zZnr%q?p@|MZP^ItudumrR zi+ALy0lfg;ieOWwb9*lbuw-;u?#K<7rhA}ml=UVq8veb8VOaH&)lszpz;M&`6|25V zBIb_9evl1tcAOvgT}D*G=XjwOK5gLdOI&miqWE396$@c3Ht-D-0}L!IoYfBZoi@`W zSHn0yZxQ87>bAEvsv-bWHeF()_fiMOTPM+R{~#G7#UWI|FXPa~V!Myp`IXsT(vM*G z24q(=h{tULB~s^SKT044+^6QwYb!yASn;NU#rQ<&-HcM_v|ZQgVfh$FNQy4ndCNmE(6uCiKb0`gEAT=e!jXq&2`b=&&Cp9tmkLwSG@O7vURz;fog6C{G z^=_24_#2m+$YRK$3K9vAPDK|dC*L?w((tI@)iT-OVkH7=1GKIy{O_%a)jANX$7LC# z^S5i^{|u|Z5odAdT&1S$OHQI0CT-lFi{c<;YsO;nyTWP8g7lN@2&OShdta=q4^mTT zCh$u!!`%bUZHg$a|44=l)%SKmCi|#sWK4p^jY-+1-WyHXrz#-9m~-`G2ecfH_xyX? z<(%_~%e>(a1I_0L-_4+4j5T$E*u6Y+>z^}N^70L;4Ym^>mkuyVjfYTP*1E;N)4-?# zMen-0Y^`ZMu?K8O3O#a{Ce5&-igLcOlFtTAn3KB`(&iWvWojN?jvL-9KNMWn^I?9z zSLmWDo*a1xD8u`b3?>}C-bQHOFl)YNjm+n=1Vny_4hiZxVe&larX;I-reda^J6gLd zX{gm$O6dR0L?(>Psga20eyXyFS~6jX(D!yo5^HYiu=G3 z#mTZ&YI{%3eBw2Hih@Q*W=B`eJSH&20>Vx0g9Dl%ko16rNn6|2>GL=4NQQ}KCIZ%U zJ&83uH9ZATPfo#GBSdf*Sj0oWA$KDJ4<(VBdn@vr{1nOoxE9IO5xFl9FF(X7Hdy>k zUb66d_Tz_6^=go*mcu(LOS~T83jE$d-;UbsIIMao{?$Nqf$&G}Fl9+T*Cdej&l`F< zC$+O^Yx;85z65TumJpa?Ei>qbq1>&HJZvIWW^7f03DL4nKhNfN0xF#KR{#lovTTdK zpYqrDkADtW#d{EV;ok+FiM5$-7)Q<@$`62+mrBf$>hjiNpaLmj`+I>PWTBeo(@&c_ z%I?q&$p+Rq3{xsID>CJ&dqas&va<1LUtf%8?-bQ`Vbl2HSyn2Z4I(ecyw{Z*6PD;G zU`RDWG0`(zu10AS>XYC z5eXx{;J9>}%KoZ8Lz+;@@kbkPx2Y(qD!J`+KsJFL#NXMItD>rWoknOa9c5m9`uS(j zhqsbA4C1P03RE?EvK8M{4nA%bEX!z@74%J1EokC`w5n+wtYL^%xUE3# z--}|w9}I13`6trwlOA3#)pFu;(oE`2jfgYMjVabwpIf0sl;{`+AWC|tvS%d+8x4Xt z5dRW%JW>#sNZuXjXH31N_WjF{4O9D~eRd`aF@=CuRZy;VE^U&$^0Dcw18gkTQGIQ< zk|dN>c2XI<9)$H4Dd5TS_@K9ouLP%Euup{ z^ur)nB)K}k)c5LL(-)3eAqy6b&&7jtDN*1bc1i?09(8%4i>h+WlUNua<8<4)BIZ{N zz85;CGt%cYD!qA&mw0FAxsnlQmw4IGYqx^qI$CD%mj>iKRUrE$leZyxP9NK!26nlWa(qMwq!Gj{q+n#o(IGl8QDIIa739cfzL0aWe1A#D;%ej>(|~xVs@XiB{C8;3Uy!#i(3&M&-*H{< z)RP&G8>4V!=i1QskkvsO9Vil~hHFcy-XyV;OQHHfL}CFnaiGbiV2t1T#00l9>JjA%9<<(8vRSEXHhBK^#t znY5T(%;W+hI)njpvNJE`L4|nmSZRHpG-~{`{EZy<19I_)9u&GS7>pTO3ZH%M5Y&j$ zGEXho#Si~b(}M{{;1@o4#yq4`uuHDy7V36uQYA#UI^=}eYyyfJ5rjYQ4hqkoR!Md^ zv;pwauNOLV;Tb$D-;Ist=IJCel!98KCu3RkKM3n1&H;;A#!LgFvW~gV9n>n6WPAeE z^Lf3WCAw-GG`=N1k;LNKX&IX>4LCUT zOQDW~QEk6fdP3Bk;&Y%y;VZG|fhSE)M~U`QrlKux5t*ueTqyfu=5KaND)cWG;&P>` zxYYXA&s*kV`8HCt`9dAAZ5r2;CJWQ;W}F)@rL6^azUDFbT~rfVE|FeB&PP6}HTlrd zm{JZqmzsOUqyrr_sh++w1tW6YyS+e_Jm=z7Efr*WEfO%!&>+G3!|h>(H>|ssz6+2J z>+L4>Bi#;Qmh{p|!=-af1@C9uHZ#55IBhwJFf)wfCG+6|PNR^~Pp~`73d?-x!_&Bf zT)arAXX0bct>-;6i5+7MWwW&0EwX&wYDY!UCIMk2sH))s(5`u$bU+g~5^(t+hIto# zCIRBfbJFT+&s_Pr`|}$MptV~v&r()RPXY*SO(-isuXlz3|(A}j3<+>^$cQlc$f=a$Q>!Y6%{jNgsO;h9?nA~G_*y0Gf9 zzEfCjv5}&5`yxM67PuOvKJrqWGT3`#`%6~&E*|*!l`*#$Jw-Rg5?_IHbqd!Mj)PvX zct<8)*tio|(2=OiK+!Vv6O#$W${$%>5NPM~j%UAxWXfLme9c?a;gEo+mn=mIif@YS zmhE(5JON=A?6_s?nD77ZTawJWmAT1YJ za^Fa`m||y%Id1hxce}SvTZRQ2aoI-+&;q{ZCHlcQ$50c$1$`C;Ef_D9F6iUy-(#mE zvwUh?!Jn36W8maybT{L-X7Jab2z>EdP5vYvO92Xry7jNH)}-FH=0c-Ijg(>JGoflp zSPjHqzKM^B#p^o#q}EH1+vgB`W&91;k1#!jCY7+HZu z{YapAW9mC`Mpc3449?y^Wjd_;pm>Pbs)4FoQevIerXstyc@iEA>24FO?|n9`%=;K6 z;CU{dB}@cwMVBguyR-@i1Yc13)^dJqh^IQHyOhjVS75~^PE3Q)X^4Az!j6*3v|FczJiw?H=YB^)BJAT(SGo3r`^%4GkFd=lVt5vq@7UwWvOPK^F z&5OYTkn?C=aL&S==(Q8Zv|v8eu{hs3{ovzriR}v`koj7Di-qm{kMO zefQ>CgjkG-W(R}KI0rCaN|I9OOnEboC6!Joxv>OY8(Cx?vwm=?1OZd^&x|8rW5iTZ zd#SeDzR@vs>yD&6qFNI>!4E(GYxujjkys@dsHBQVsO-~okV>)#oYG_P0RdI63YZbk z;T8UK6xG1rTCrn}Jj2m^CS#alj>{pqQwBd>wCr2Ytv(`LlFOxT9$XxfqloeP7kSD9 z7$FKJfGuYx(TV;WCciRkPdEg*b99dg72J4~TL4F&4YW-y^3$29`*?LD^=N)M^I~75 z&0|7Dr2CVZLiQwVENm`u@hRPEqXS|FPu&i&3|WK8nb47O%`lDKhh+_WBZ$7gk?tGf zUDS;ou$#8Hn=c6dlq=9#?{|MbJ&lL)B9zID08 z_V}HmpcX?LZnpFb&s%hvyNBPUnN5ykyfoq$rjmdPquFZ;vr2K)AD_ttL5i70F3

{s^`}=ZO-*yMPg1deOG)?* z72nObQ+z0WxJyH&wW*N;`gdQ`jaKE&)~t<>^faUbowL6p20m&uwx8%_h6~CG-8>=I z^@|aBPhv8#iKFG!CoD$Y->o}fO~D{#F6|Z>$hHK9akV)cxEG;%MGMYe;Ir>b>u?gy z@gRyu>DH4ruJ(rs@vGy;rK$71oWnNE#OV@{uUN;L1*}*fOCW0YIqf?%P?z>BbQBet zeA#8=D%a<F?sGv@hmmp5mv?!qk>X(!+D+SQ%DX*zh5IW4&;sMF zEWUbZyZ3_orbaya@IsRrXyT|ISTl%~<1E)NY=u*Bvdavj@=oOs)@*4z^ATpbpsje% z>8n>btOPyJY7L&N>4;>2zXEm^j$g{*PPl`%GQW4F!GqSj|FQO7oN>Ojp!c?>ZYdg_ ziU97Y(vbc&X$U%oF9r0uuP3&zP82d8+}Y}ODj9q_{AKe$ZRYNWFnKCow}6j#=P+J_ zrPaBdlh&F{IKdlihvgi5=jVH{e3Ke3KrNDG-?A&|2Fin3Q4^D}NwY@>igyOHh*14M_}8uM}CX5j&I@#(jKJ zk5Q~}Hko~cuJ@AVdoi>?*ZDF|M8awDNq_Z8b@96EVwZ3K<2%-9$9-?o==WE-S^i;^ zNqT_djW*g1PtyvaXhJ8slH=aXXC)kz6temCOL^Omv`cH^QbbBPQj7&vMp;{iR#sMB zvFe@|q%$}MmeF`L>cPC9%V-NmLvI=jaWrieLU{U(P?%KhJ!~LMIa*$o{M@h*X0V-C zNO|E=8%G|^Vf2o~+V=adJ2|Fp|L2Fv4*G`ZeqT{O)j<1Oss0PZy!7UE(u~Iy9=csc zjGu!%n~q2t&FqnPpD)^q7gn4~zloFOjs1~~F7RQ4P(3mLd6NeBHRASnG_aIUnmzpJ zs06bZJuw1%oZf{nT<~+U=wBc(%CgS?ZUgxNxFXV%G05*UZ#;c#PL^g_GLR@)NwZdx zuVDc)!FVmR-BM~kNtvWhx_6$DE%R|@HQ(kBSXZf4jjA*c;636%jI%(2PJsEF8f+SJ zNy2L8_{FBFT%f|b0EuLz+5LEN5{ll$ak*aUK4!f=80FLaTZ*U9_OV@JN6tY3lgSQz z^2X5re#g#j%Crlx0gci!C<5uwc*zU9^gYC+t8r%(AT1!<)VlMaVZTW>`rqH-rF1vS zz5B0pqwE3uFS?8AYagEUZ)fjWPr|UsWLb0@!7i4~^yXa}PZwNCb(`U9)>ed4qqhiH z?A}5m;Pi)1aL!GopPf_1=8u^fP0P*ZlNy+R{n@lvKh9`_=J~4ggYg%u^Pc-%;x!)? zNUoXVUmfYbiy=QnHQ|Sz`)Wns%HwSx)HBAvSkc^sv8m`&i-%(9-;egT{ZDFH6HcX~ zceLeiYW<@_!LeVYPZ*{fgj;T~NJaEl%JVHEzwZK#n)>HVVR5X0AjfqSUk(;Cb3cZv zC`1>|{n1 z!iaY~4Fr(u1@Z{asT@v~AjmZmSd+y3y?T(nD z`^H?tJFwQ-zDMO=%yqGzEfHL%5|jBetkNpXdKWTm7ulGDKwd}<6QUcCjUb%?sL$nn zxA^*1#j9kOe)CXrfntlQ+Xt27jPQvEw%Ms*GH;?jIGp}-;EWrsaT31j{bTizncnjE z>4D&CA{%=mr-;eIyvGlK4V+%x*~h*G5tIOH?D69A_r-vF&ql=R5Q8y+54B7;_x*iSZ8#NNo6nu zw@1jXPVJ3lR#H|C#W^{*y{ne{k!gfqYAd_e$X2-p{b0p(u+o&j8>z}pMJj2kYZ^5U zT@RUYQ$Fs$IGbdQ!A+=TX6A3za9w`Yp@v8lONI0k(rb>DsY1H$9nNKfVVZ6rp*9sO zmCn1b%cjD=Zlv}ErG#Zc#O-#p`8sT77GM9-!W0XR|3aeTKg}o*;%+YZc!%RJl?14L zEXV!kVec;edcT#^&iyS_gPL?0y7pnZ{#n8Hocra+h|Mqk?}PqX?p(EY)fe{>9ps+H z=&dEF;gJw>?tsbXK4wGD&X_*4#LFBihjM?Pio1&ubaW1_A2qwRJLU)1%|E9>ALsNE z?A?d7&~trLxy8`K%-{Z2r`-Eum6ediSJ+|J`w!K1oDuK~*VjB}bB=VsAg~#{1Hl%s zqg4$*9*Ca}g(>LV6*`rMMdMfgiyMa=gGs<9h#O&OFMle~pODoI+S)YPbUxbxBF$@3 z-YF$o*)&o_fs^Kf8iJr9meII{i$PH0RFU;js^RY4Y#$4vmMF)SSUd7j1EIp#2kVch zYHo0yX4SoL+59S*8WmOxvBFcGhTcOAevp?t(?tLwt`H}?yeUx)RqKp9%*GZJ@wh$B z<@)SNwb37GO+5wpiw@Vj|5C#~pfzdSO)yaC@%!-~JO9gHh_#brWPrXsN|>g&+~qTs z8RFbX^w7$MxXwM_PVX(}v|)TQqda@?UvkLYPc&Ef^3RK&;&1<3IkenntZ*h#$Pxa{ z&DNUrn)Fh}f22Z1c+Y{w6ZoPaSp#@46Zu#88b{v-Y4pM|%-m*(Mo86zk(c6o(I@GC zpU6a{UsSLOD)IfwNG`~~cq6Y!52~psVlI>knk0a_mjcDb@!HVYIitpceOCyDon!T2 zkI>4cq<#3$TJhp%=ty6_!T*lp-%u*@gm!o?d597-h4qYyIPW8}nH^95sncY5B*UHK zeT?nTE7qdY?eU_Y63NAUr&f%&v9qK3G4v}UxnnwU74RveciqF4r!*kA(SU1UH?$*8 z$$(qgdh)y6oV=CPRomT}w%RIKmn5J-7FnZMWm-zD#za@Zc&Jb&Q66MLQcp5OJKk^dRtnz?Sq?W3joMJ(CRc}vphM~L3= zhMD2AvG)1xRD$eqwrGOA{SJvGrgFlwXck*oXXQiu9CBVga60i-{`VTQnKJ=ZDE7)S-^-o+-4Bz`WZ0Kf}%s_UHGyye>O3LDhp&$^L&?5cm* zR{^KP*PG~@PI$<|a9`Yyxi`6b9FAhFIFpVkv@ckFA8Ymk+ECf{{;tGg<|%5nvUoOQ zpw9RPU}#9C8f2*4QD{|Bt#EjX{mmF~C}2?n7?rE-t><(IyzW!_vzvX9m^ABh^bD<+ zaWd*f<0|0FTNZG&ChSNxI0u=~z-Uu$QG8G^>; z*20Tnf7g5auV!u8h=oYU139C=o>D(}PYRkko6%=&tao_b@aj$$_~jJ7WIvnkkoKnV z-sI)quGwvK_zlJW-Cd7*v-FaTo!4r34_>19i(Mv6^OtEWA(^#~DcjqDlw`F;s*$8Q z<8wJ@4ivxRH7!PnS$OvmP%KDo^~dA`hzIlB?$8RP0dV29a5+~loCxN@b3omG9LxCG z`(RAxj#baC=qyj;Hso{22((mxW$Fsy%Jj9fswdR^@#el-BM7asnPEMVK%}>;NsdSd zWz)rc|D&T?7yTeN0qM{jfqScECxUn?mzg#oPRB{RF9)~0Xk`rnzjphu;bz_eYG`!)>Qeq9Ha3IeMQ5dlINN5MvR-<0r+M#ZZz_6M0t5+r}$%Fv+ zh{T5lK==~<+tAA#>F66o)Yznx-8{4|46BGl`4`Gv$Q|YpI-%FV?Y{Uxj=%RjeR9A+uiXGC5Y}r+A$P8*^DJz|e87Vi)_TYRy;t^;hy~9YHvM@6SydM}e6N zjc6d#v7Py6Z!Gis9sssxy=i0qn8K^rnarm2hh7ovRVeYd(tKn}3UENRZuQHdsQ!4j z2pCRSZ2%~D=#ifT{e7Nc8U?U3R+$$2&!sZ|gJ!+fzt+)M4&?UyGv*_&*HnWh21HZ^ z)99@n25uZSnnUWBP(&HGyzSp?JAFR#2y# zD+gW8O`axe{uAKC^N2*t2}gvHBbfD@`!orG2~_OV9Srw(zGbU)bJb@eZvbhcv-NiT zJ|4C2b6w-!#84`@IPvz=cmNSi>JBafGU-9{Hk*Fw;Ay^D04?`85wM-2Z~h5{jcM^j zM!|L>ghHd3J)ZmqZ@-MO7&d1FZ(akY0T^HSyy^EyzJGP{?GLxb?J!8!DD5aCp>L3q z_p)~%rm~jlv${XE5U&Yhwrh_J62U$5nex}B!;|94XUX(W`bVl#v$1K~Gd3CpN_)Hg z%Sm2Nvt9t@#ATPfBlnZ1uZA&!F7nN-Q#8ErW7dR*yT^V)d;L{ppR-sxN!+O0h}RzU zdC;rq!|DqaePA<>WHw9ck6Jnze)9j8;vA^WtbTS8_e@lH2rLFlp(;7^po8)B*N$^p7mP5y+{eXr zgfBq9hF_eKcMcI@U~9vA|7oHb>yoyj5C00q>3f*ENw_N5W0G^RNw0DaPm_#Fq8`)r zehPiqS#&J->q~mfr_FD67&)dH`tt2L|CC}^6C4#VKLwqE^pIZI@P7fB+4+Ez*erAS zB#bJ&%;8k_)eHQ}nRJA$*PZSEu;2eNyc?%&WW6?`tYQlO*M^LORSzj~EFXlJBQ42% z<0<&^Bda16Fz^BO_Tu-ovkuHdF)!+3q!Ds`W$(seiiJL`gIW^Q8*R_SA-DeuFn8Jr z^j#{~iL*qF$$CL(=3P_rJ%X$htklG=zsVdrD;7OeJ2ORlI@2)8qfJ{AM$m zHdzqc{jX!0e2S$u78y8KWQX9R^4b)W(T@)ZJFKG}MVQ5%W#oIcBFTVE{UNkqB zZy;3EDz4GeP*u#o)lvn%z+BR29=CP?pis$o9{VVdnOFZdfJIj9>^mIXK&&iUu89MD zDr*r>s-a@AfnS4kf7x>=p(+K4f#%&MS|G(aj&qSQK-olpKf#=bG(xGRHm_BGAkqP#n}^y+EmQfePP>r%z4gMYgA$&v}F>u zf-QOmdm?60w5xdPcO)2 z6Q&n;0E1^pVmy>xc@DKOdC)hed0QLyu{=|70qZdcbpZM~rzhz&5)S{M6$!H8r67o0 zIRJoR2}EQz)R^xCk}NuO*Eq+OF)Z?x+{vSZxC#Z)8NH8dY6C;CcZc?faE=F(7^0R_wF-a zE{g-bkD8~hHhvQ~ZyufjRTB{(G=mRHSFzZl!7e%BmVH_vb=oo}2tSf-*)Ly|;Paj+ zYI+9&J#MIWA#uM)jjLuRetmyEHDfHe#uf7Lz{_9gdw#_Ai9V%bG5~9h-V`pGrkiCm zS<@hWhHB+}bm{L|nyJ;@w9=$n_J#f#6&7xflQggAPh`s1-0I(R`;SOYMY( zS-r+|jvR@$U-#&Qy9mr}^{XH6T!c<0#7pfndDQd0-Eapd?N|Zok|*vb5R~)0Fr>CX z_x+>5Ih9DN@fx_bQI$k`iJnzr;XET<}zwJ+7t z(5NU;(wNp*LAn(CE)Xvmd+?7eRD>jVCozNxh$*RUA%mdM9q!)rbn=(G!Ewi(eO-3F z4$PfkHa^5L#3*06Ef--*Vdj;<$l^`X(Pa5|@aLpe=>+G~0)a$6kE~qLyk`xD7)Lve z?_bbciaSuqpU`(&axJPus@~O-|3}nCB{%rFeJ#YSdqUG(XZP8jnc(j=8^ZM9RP?u1c=bB}b&V zqoLQ$xph9-!0M=|L3{8xUk_GP>}cw6@&SJ){qx7emvVHjQag{aXfWN3A)p&AmMVdg?^dcOzhAAL6CaZQQwNcXU80>{$j)FF!}k2_azynC-%(4x6`bwj zCM%IapPfi0^xlgqatFG~j@LFN=SOnLADeb2-7u^b1x%uTa5f zeKnG-d>m`H^M86L5-N3$oquY+N_(H*x22uVrh+<=X9d5dFpvMkKF12iau{zSmSf3b zgJ^m?X%9Ly#pgaU1WJuAs@lqwg{lLb%XZ}BS+Td|K2a(dsvABPMan?X)Y<@LywFSS z7!P~m87gcjlBm+f#FQ~NerqSvoOZ1!K%&E!f$;R37HNzBLX`>n8tcUx%TIc@?!CqH z*G-r@NY+%PM??PG-WG^^!D0B5lB|a}unt|`Zf~`SZW;RE*Wf~$!DB5(uTduH_DiGa z+JU0g(WQgJc5Tx6swl+T8WmHaHoz8KBxn8iY{Rubn%ANX;iwxs zncs6T&lpc6u5aflZ6lw_v;&U{RlTz=&HcRf(Sgl`1Fg(L`?%c52yq7 z(ip>+TAFq~2fgtz^zHkbB}Yub#BRCe=4ml0KlQ z)hG8J^v%AKD$y{o?)a}}@U$b&`(fK8plMdvD(F(-tGNPPK=s-1bzG(c(=)hd(;_bv z2$dlYud$%mGFGmM{%mw7Z6W+qR}re#cu#|^{QgM>{<1|(pQ)}}{`fB+&HNcT&GZ3F zKaOylqc0UADx8qm|B41JoH-7puU~HK!@Fq9e)4);YGKmQFrBwj_!41%8_{{`;Od_?dQctmU zhy|X>?e#%{M}eb8#VD(ZW|n+qBz{*u$0q;nk~@q3!qNQuND~liuW#=gEZim?>o@YI;^636=W?f$ge zlJ}N_%pm_wXi#dQa|=?Q;&1JnZacocq5;zSqBZobpZ3#|;eVOJR{tLF3Ob7aP;cHuXcqkq$gqBDRvGM`LIqq)V#pi|jys-5 zn-{i1?t|NJ6C3)VT?o7Pw$DCOe>}`L|4_S~EHCtqeD+!57LG4&%MQ$oQ#oC2S7^H< zb06#t+jQOZ(wszyy%~mcWC~Iw)SaS_#tihq*35lw+zC~ zcWrhsLC3b+D*kIv-yIY4ICWoOD(DTm)t~WXv;gVc$HuD87k=co2t*VU>|3&G=wHDp z^@|?32g#3mc3(gHlS1xqKCa3gf3j)RGG!l1rO0h9|C*r1a6fEO#@gz;wiBhy$8p=P zTFhEzs?8IN{Xhh7%X8%>zmpYPSfuffQ+SS)33lxV$g`*$6rSV(e<`yp4Kqs1Kw{;M z9j|YPzfDCI4f8Mp0f^fZwOp$vh=4`sGcuX$+het24E#)CVcZc$0-r%sm2JCUNNL}A za@-$}uNHM3ckn_9$qj0o$0EhYm2$r-UatAX_Ooe;RJGmHMZ;R(#e}bulvu*rL^9Fj z@~;o)_Bub4iM%_xIh)$e3tO&MU2;^a(5=r~`t3f36x{TP&9{}}+9(vk7a5GbY1!?k z8dw-#^c;s|!Rgym$MMs=Z|XuD@~$5s-u^y(-B3&Qf0i$c*hH=n3$af5uUz6J<>6u>3 zwE$_~lppr(AiRJ!xh`h+^$^Uh@rNw_4X@L*rhkH01|lFcnMwE?4Y65I%%b<~=Y{Bz zR}}!2LFg6`s=9mJdsGki`;RbgpRm=BmF!h~@J4+>h(y(PMYM1;PC=3RLY>G0nJ`fz?~| zX{vAR_3chEyK*{x_3bCNG^_BQm{pPzmhmvmr5M;tnkD-L*wXyoqzrRJ-H$QAwPvn{LnGDm!;fiO-Q zy#yEj#~VDIAi_iTYmBGCbS!zogSY0$Aw)Dc3v^>Ald$0b6*WFW_!xeVAGb@ z^}D5gu5l=`He8TJR%@ptoRE1mGy2evWhs=x(tjV#yP7AC#YbZ);I+TP2lkwYUYVS> zLLpy0w`pu#M^)-!owN8(Tz-SbQd9LqxaiH(Nb+W>5e!E;14znywp^8=aEJXd-m6d1 z=*!h&o36VHEI*A~Ph2&!(dvDMOu$EDxBELA!B*SrQ8{6_t^i+%f3&>BQedyuu{SG7 zPO>>DiQwkjBiLdQo?e8>&{^rhdjUrcyjWYK_vKfsWrsFR0IqHkH8!juO^m|J&>#9C z+sowTy;%8lup(b)fO%|)(v3eY%kii$8D|8R;P3B&Z8gv7OOelfh(Bq4BlNwcYj4jy ztwo<>PwS- z_VMG&W@`uzQ*NgcT#|-rCI@bYf~WGdbp8uft$}Dex4!uS(x}cz3q)q zJ?B*YpI*-oe*x~A^`mU^CYQ^;s!oj!z7VHxGiombHvGj#ScC1fHj9q(Qd#A&^G@nm zL2?nZ`c{rgzw=aA$*bw9{CY!1$7?4i&!DoZ*az8s2J@d7$!~|#@;@Z2M=;1kS->ZL?uz&A(1gkobdVzB|mIAT=2lQT|238x;dOF^U1Z$6)JDPf$UY@ zYTM?Iy)DG^_K>eK(02SjFUVoeMX z+;$7&&uBAp!DL6mR8KtK^v-O`(2y))6U@Q6U%X<8-1V${hUaTI2GqB zHEEY0h1?e*5Ua9}NqxcdSW=iSCJ-}rUG(8O81BsG6RZNWUf7vlsfjVKX@RM(P1cTP zJ-KDp@ep#sMhF*TgOt-&)?Ab}ZIz=CaZ~9BLv!pIv&=GE3=TJKDH^a=2hU~c>DcvK z-^~_{HDi6OF40z$?g1n>zVJ9-y)cE(qWcSVmKQ;?@_hH(iPlqvGTPKn21_qVg(B<) zvt0!!6A1m#!nf?`2HtS$iTQmeL_&ey^&`P{mCk@!3X9A2--I1L);+T^J0<3i<-D{< zv%6$~K1!G)}~A+TMQc5U1Zt~KI(f#vnVjO%$r(^SD9hoPI;WySXn*lZF*6F$7y z6yiSQ(~Cfg*tZtLyIZ!ktgMjtAwc*OkNuxDLVfoY7Vuq|{5KI+EeesJ|K)wIzOIYw zG=jfUXeH9o=jQz5p4Ec?RR5>NplCTB9b=zPHf2=g{lE3$9YeK(XC}H%)93Y>;0pjc zIn7wI9r`dmdLBmVkf-)rFEjMhoPiI^imO_GSq$~n;Dqp#&d|lSjZ9(+Vi0L5AvG(tCIIrRq;Jub4;@CBJ$>BP@e8 z^}f^r=}I=Uj3_V5$NTbY<=XvXSnJ@mOV~n;d-S$jn*qd#?s_;Rqn72wkXmUL?lQ;m zu~p)*=prU7eBW;NR@&bF(jmve|sBV4-s!*IUA_hz}_CH1(Qe7GTRItC}#++Dp zn}Yv@J`qIfRb|l5Ny+$Q-5ACBo1EXRK9Wd&NFEw+>|KX?b6oB{#07hH1xd|J70Kb= z?vTw^7Lk2@7)qC75^$?vE-D9`0wc^q>O~Hsi8`XIhtmZ3pD5;vWyLu2gE%On^ccuK zM3S@$Ik;T!_$p&L%m9fe5}r)n#zjnm$!t9_E?v~pEYY2~S9)^=kuuezWfi?fHH=C&gN6v}u6kUgWrmD6C$yoC2vE=^(FjR|V zlU=SNqJOwG>MFREX^!6n`&3xZOP-Jy>cqV{`|NGKA>er|ct1}q!#y941rYC!M%wnW z=3*3od#kgbMl6dLuf77^pv;|U070E&=4jcGZt`xs057Q!a;@L(;hMEsDxD%5XB}C{ zgah>cTD?Fc`-tE4{rlm!*YAXo55G+TDjTz?IBWxmvD3gq#C?OKGe*16T5t@+!LKiH9AW)|so#5k#V^1SfZeKL7YdnBHIxdJRs%M#e^OMTp|={Dwt74!p@p-P@-;wu1(cZnp=hPcevmVOs|T%mernpF+; zX^&=o_>kt!JA~{z{>*O1-Du1EYI6bJc@zqG^dqVwRJ9URYdW~k4Dv)HVq6+5g(8{x z%b$vI1|C#r=EK7cw?t@Eqm&QWJ5y8JI9nY% zIp{lnF5TUDBB72@4$$yi29j2YRN^dy4Qg{+zSAkUsJfMc!pg=2&qUFiE-FwFH|ISl zl+hFyvJe$`foY}}i@2T9lqFs*9fZl^i$=_(fPzPYu)CE|c06WF=v&wRd<_4^1=E+R z&hD|hsj>0pEP)B*iBnHUbRD<<2S+)pR&_v_Ab;|z=lvU<+VyO^7e-JTVE$406NZ@> z=P8^r9 zIm-2Y4+fwpv$yd=JVKaHG~{n>=t#nOMyAmw))>rELGJ`t=>)C)g_8+E=J^QX2PqWbs!pn1?^B zkvI>mq;fR?*q{}B5$v;-ga#s2kbBUKKpSMt_j<=}oZVBU{k)&_d#VATy-eg7dj9Tw z1o%*X$W^CyM`GLyv_$1xabmW#V7Cj{7GJvuz5NcvVt8$~IFdBNp-r;6zWMERP+CIg z*V*Jt#^UKL=qCI)Z$`1;8Uy(_OK)rx}MY1xH4LJ$OJdb)!i$tVISr(t&C+H&r zVS?&?bHr*kj=KX9{Fz%JzH)WzCBBGegTzi@yEF&=8}- zHgC0BqE};~+`hMZoZUv+ati)gcK$RKH3OeZ)4P*)mLI22>XlitQiO8{lkFl53eU$4 zDL8je5L;Hh3u%<6P64Z1cf&b_5uEA4aq5Wc(qE_Nz9Fcl2lR036$C`{t#p^omD_r-F(H_n#VAAbA` z{>0xUlWI9W3{v<_BT&~sksKfC-Qsc}YzaCUpgKz%-H|3TG}i&+`uIlqL6_VNAMKjZ zhGn-eSXp=fR)Rh-EzXxH6#H|!eLiasZ$|!I{qGF~Wk7jW5#C}Q9ccIWFWoO*0G?P3 z&Pea)CI!0$QY^R|1DPGRBvAeQ027hq_f_DuMKOzXm&=L@Yh9eIFQhE2x zjuB@5tyhRRs2mRN6|U<5(gi$nt2LMTKzYB+5Nv(V68EDG3+!(CrU^Pi*J2+0v8vVQC!`-K1cND(H)6lh{p+v*<6(YwuKJ<7wrup-m1QDRz`- zAk@&;y+bdBv(YZ{x~C||#^62BzwdV9Wq|(8oXd4_m%tPwAxw_qTb!hVJww*p`L22W zdrrGK8yAr@B!w{|UzY}AsQ%?#(8m@FJLWmD_P4%40MU6g+2wO3W3FfuJ3wucDmv#t zln0Zea~OOjN^R^wh3HHAkFDwghJF3XCJGjC}Eqyi;!3MTh1|)$+?_^b89KR#*UO4 zwS6bw(!5=7GxL%E{BH{Qupza)iN+Io@dw4wu>Nn?d_~^nY~NlaFR1XysU4X$6=vnk z$?IF?b!WWZNR8H)ww$5g+AfCYL|~u4f(4TXz_e5m&@b0Ki0;W<9h5sX`JB&P*-6e>ub)xDyH&-QYgf)9%dq)L zl2p4gBcP&bq-T3GiVnK&UhK;^nBhC;qhYAljO(QiBk1gR%p{0%$tHc3aJOMMb29%v z6Uh>nWWN6`dTxEc7YnnvkYNV_-)>Ks5@02h)xVzY-2v8;Y-q@~*cR+#Xqk?devl)?MKnU6S^QP$i-&$&ziZ^jYY#KmV?O20TM_5|3#1 zdyt!J0FO<+tcquz2#?NbSz6Zr&h#_JSvi@okFIdIEDV8Jo-ZVM|BXoUPwThLPMJLu ztOH1ZBtthZYanmFhbXGU>vcU=Y|)Nigp&lF?gu(9eO@kCo>cIZ_=wz{{hz3f$QJZd6T_$Z(kz(%1vX1usVn z_k*#7()v63yHRtAhVRz{_#~0g@0NGI^vyZ7R@S%(*?;2)UZMLmzC1hK;Sg`uw;v8e z1@~z@(Y=^ts_5qYC&%@N$-QGY4}|*#l;MlsmV@2akWCscW#QD{tzw|H?1KrvbR&Me zJ#H~0s<1b9~c##~xqv$Ce(*(;XYibj8CU$kQ( z+=que(yG=fMbSK_!yHssy|;|6H_erG#|v87n>_4F3NNH#h_#NSG-yNm-Y)=;a`szb zEcTO3f$(<6R&3;cX!r`>S{cyT&5-7NUKt-9xp%M??$7)_r*ddts?Agpj<>z7i#wN{ z3u9P&u!X65!B9h&k}O&M-Mi_rxISKsT;kzgu$o8*H~N~+@@q?sW`% zg>WJqXN=oq{b#7A6P^P@HNW#nn!ew#&pzK=o~l;JFKi7bh(}?|J-v$%4pYPa`E{~8 zO_4UaYYnHG7=;%PQsY#FY~E>!j6@YW>VYWUQe`D%P9$R06XN`p>aGyD*D1lji4s)< z=ZH-epxZA}4i0p(7B6{Y=P}~u6yf=g^ONF)`J7&{^X#44#8L_N(DRE^momz zf4w|*$;{_Pf`l5ShuGcmUG27k4Kht<#%Ge^-J_@+$!De3h>FHDp2Pa9I7kLaj7<;Z z5X#WmrKx?bPSYneP4#X<)uTcyadu_e6Mgh6Q?Y;L<(%?gwoL?lu5)|>y5RCM)qzpJ zLy{`5%CM-bVm-0r2N4n%KR*EB5aO!=aPdkucRo&*t=T0a2yLeax+OmHX(KBQ@)^)a zhrb;WMMo2JNjR?QupWopSQQl`7+pEQoV&76%KbnCqFA~YsvDE9cVpfSp4 zQCWtaWSdVHJH3}}eqw3JAnwJsKXvHsZDK`-=VA;QDJEBDjb84}9SJws3$l&A#(_67 z$Un0r`PqE|r;Zwk`FBQ)_Bd;zm`6%oK2zro4`CIkJ3rOpY92(Avtrp|ldeprqtu4i zX1+4}P92c_QUBW%^#z_G=DrfOq2_y&?;?O>SBIBHoyX7*eK1ll*W$#f$WLqc2d?n; zHiG6f0pH#rRSNPg^R9nJ3F;-9@fsq6zh6ut*KZ!jS5->Quks^6YBUt z%PVVa=W^VzK?esGLNj|27oX(I>`hRcDZd!EJ(=zP*nGr!LGFt0bTr}EZet`BAn?~{ z5-pvt_U-66&kE?QGCvm*XcjxWf2byA(7)z(?8(d_u_~+HNVJ}>Ygee?(8zEDJDLv= zE$d6(5h?fCzVp6Z(W`sAJTZ!I-|tC**IPXlW<{aw?A6;0$^wcN3a-HH+p~lVUeU0T zK^PQ!XLU-%Ff)hUiQSEw;P>ySUC3G45VpnmQUe>f-!g~#)cR$`K5(B9=b z^fA(2^RO?}dA{H3Q63s5r(HXcy4yApB*s#hum?r4>M#E~DkO_OyiG!LD~Rpeb4wK~ zwInpZr{7y^0{fV{I*{W-6fUTwzgNHP(a~-Q^l{Esl+V5Si~$jTInPo+@JqV}bsB-X zwDJ!)bR87XewbL*IhvLD)&I&3-w%_;Bq*8}MGbE%_QBZ`-<2`S6TVq_KKD$>MdZx8 zQ6{hkYVKJb-BDh@xctUYYS`MN6oudZIBJ#6>C@~*8;)16xB}g-r#qgsRkkK1JlC$XD zQb@5K%I?J_3Jmh^ck}f%aqU`Sh%3U$4XgGI(K{xJi(j=)yNI-`hnE|WIBa2VB`f^t zVXLl|Xn8zaKFhm2^+b1@!xDYBhtV9oH#%a5obL@qA1oO9V8k4?XV;ct7nki;IOusM zQ7Xk7Q#%_U)+Vq*%Qn5jqcW$et;LW@vnGg`6z=dYj^e{UWo|5kf8UEFSFkSL5|40u zaTk);#Lpq|f}uP27tLZKO18eUG4Y2B57*HWr0^_AHdgKRcAU`ApUmIhLF$#japjcW zFiB5qu*Wri5O?LF-J=8oP7&n1V}SS_zfb)c>T&pX41Ze=Yqilh8%+sO+%8E0pF;}v zUd*KzX6`_=S2y%BH-R; zGz@$yI^sT)*;47Z#k;^U@tgAe=d7>rNrL5W>W1Qw%&JFrZLv;pfeZGu(zqw)wXjXh zaaG)!V43iPzsUsFQvb5c2DkRiYjKsY4CS!LFRTnvscO5BoeE)T%PF%zB2HI}!2Ixr zIP#bz)z<{19>pO`0`lwEvPU83D|w4?9?I;}jcTvOhy{li#6(mjXP^T^sNcAdo*)^~ zcA$#o%-v9bMDmV;(;`6=cPzbpUl_QwQ;Lh8JK;WPO0`XFvE<0X`R7`Tg2HIxM2PQo z09sUr9lY+LAsF|U`7c^=}u_|q+u9RLZnNY0f_-6 zq`Q@FkWPuAyYsuapXdJG_x;Z=*j)SEYpwHGGp+$-lT{2l=GO~V+~5$$IxV&Q>|3Ta z+pRh;ck7R&qkgkDaObb+HYIKXU7V5k*&;oMoj|q1AJhN^MxiGmBuhDyg}?+%`v$&n zal(WR>T&79qQ4}-3+}4OkSr>#{#B!?k9*wO%Ib3p(DiK0YBOhqJM=K0Yly9d5sYs0IT%Zx__|h!*<( zf~kYjwy{p&*y@W-4^Tnk3(Vfl%kS1>U++^$)IR8@=G3G%b3*@#1pHbFJ)lIkjxuxlBg zMFX^vgC_RLm_j60fXZA5K9{xlU@c*Brrgr%Bf9k&e$|uWpVfX7l8BpuyS4q2Pha&8 z@-~gzizua#Jvmy$7ptFjo&7#Z+(AlMhe}Sqo$V*0?e!5j@+dT`*hxeROpWLw6;VfmfBinozYn2~tTNrtjk1mC%uhGCW(WQ> zf~}sF%d7TQks{5MO|st3P{YZPvkG!On@0K7=Xp*Nm0Da&tAQpAb#^C<<5-Avv*maw zXaA5{K->gGE>k!xox>E>FG2TMqU$KAIlHm;_hO0H`XzlRT)N#OdUWr@am0aFizy+v zo@%B>C|c-A9mS}+Qi1wxU6e>LI3uKSbv3o#Na)k9RU|5-k);+* ztCRwl=>m2^ytb=h&4O7G%d8#WRi~tP=cZSNRz5H{Augh)BB4y&zjqNriSJEv6zET? z1zQ3lV%{72=i$ffPEN>F;*T**4LS-Gpkjjqkb7GV$BpB}|s zetfE(JcLP*C-f?&JirtH2E$yf36q+ID-BZtsiJmp&1bn@2=lKZV}&IF$UJu-rbw{d zF2ea8IszOc2mUck$;^sd)d$Ps-J6lp#m%&ti9EH%9WWl02SRG1SyYTO0t|Sd<2EF* zA|LAM5AH4#>H+-6h{{PQ$Hm7?ETNTVnLP$>&NP;|pN55W)L-cE5hngb1Ft*7H5;^^ zjb4f~4G7|)EA_t%^r3@~F64L?F{oN*T3yV$V>`EG&nZ@4=ZYUI>dI|=wVO_xoOpup zp5fQ4-hCwHTnvZhNgS`%LX3*?ck-0$&@?ut=}+FKKPrd`5?VgDqy!OIbWWAIy8hI< z`_Mz|#%sjMpp%0o$a~urQZeRFxci3T9n=siF>NWAS@08!hj-Vuo&Wt1lDGi8$a{e2 z2B|Atg@Z+V`Hq5Q6L6S9=OGkn+cb!@10DGWW#$;h0Ag3L#o)O{GWM$hzA0xADe!=; z1*5Tj9A2-p>8SkE&O&%&YA&**ip$Z%WBMF6SOZGEb831eNjx-9L95!112y7+-ixac z0%wDTZ*pO6fO&h_DnwXBeap17>SkJ-&BaMO1;s3WR^o1 zDpXF@RwpsY{+-u3*yPcb2h}4(RVnkL1R2%=N_!&(4w}frr0x2M(Ckl|e7PEOB_If5 z0+%)NE2UT?5&YJcFj&o!0a7$T!n``2&r5A*LCmjJh~!|Qu#S~6M3m`l2@^Chl-3OU za>K$4ba^2Tl| zzB0UXPbtSaWHsP5Xzm|%HDTQQj^x#oU0ZONa|74Vt2^4=U~#X0D^j|pSU$o?L3c@; zNzEMXGApht_a;}f-mcT<52}*kkpxRV8runJQ&>xl#0gHkf0=A7w!gJRuo+rJ^?kBX zm0Q*6uacR1>Wg+*yPNq(x<@?DMcM9EQsI-pQWxNF2qz~nZ-t5@%ub?E`&2&(w0jg{ zsl9{_mlNGvbQS0mhP~CLS;$kN@HTH_2KjX3yI)ODiwV^MbA+;g-+ zkhz2=5Bc6AiZcgYf#C6QerNdCwWVYDvgos|W&X&T2<*TtPC^Z^K7->+y!VRS&(qkt zf=CDgC|S?z0J>Np8*>`_!D4^$cc5Xn_{!g}h@mWv?@ zwS4#nHD|>0XQ|xA zl!Qm3?-#3sE2`gyJ?n|Pa-GEv13M>eZ$1{Y+Ve)FD_-RGc$r6yRD)(>T;L%ePyH+? zTdgPGV-(t>Dxh@JgY+8(>sWYFi>ezPF~Vs(#_cS#u*37`&RhvKL+NSU1TVdLLcv!y zL%h>9*l0FVd1s9tJ>%#9K9@pCq!J|#Ht2s(iWB9dy=Tl*(vgKpw^4R7kf#$J>UDtD zbBk#(Oe$%Ct$xNXsz7Gmm59QhJs*CHem?|#jrCD9{APn|{imDu)-H=2;|Jvsau{Zo zuERnq+rYIS*?z_7LE>;*xvlZ&-7EAYUK@1?OXe@m#j%hO;;&Eh@ulQfs!JvC9-EAN zkO7}}F9~n$+9esqH0qTWku%l%g3?6big{H4|K?#ZZ>Us0k=aq*HxS}? zdZ*3&fYKwHxbPnZM+h>Zn1361Ahf&yu10rz@wG+6PX5TygmIh!N&{dnpCb4|eZjOH z)M(?`n4E^cF(po9FJqTBB61?+Uc@_tBWxDo&%1!5x5D42CiW!FUTECM6&) zJmO?josApb%3oO7d$~fq+kAA}F!8?i5X6aug7w~qBVXVyMWTPLzD{PBjKlwDi-?5# z%eJ#94Hn1jK#$w4xKz2;_VB$ZVLj~$ST!_99|g`UrlTq;0P6)iQDVtDQCQy^s)j}u zH*9&(M1T_%>fyV>jwn;bPvBvJ&Qq$YCqP3?u-0>fCCgmC_NRq&85L&?92ci%+C1tAd$yU1Q{Gl9XA;r{cKP4b5EOvK}oAzWkeMY4(QX(vL7jK)AKLwU^0R~Rj;{bs=Wd?3uBf1UTb-F;m zGSe#_a_sj)pI%K$2yjP6{X)7^Q`bt>6rX(e`+_NC$;C!RPHB2q+j!K=-V=>=8uJhJ zgwB7II;hgop^H+9SC|kt=AcIafab>!v~p?=3Gmsiwzjse@0wE2`E+`v#lA2}`hOL|jbV|Q* z8NP4WZ}vX=i^q~KrJk?YGXhl~8W6LITiUevKAipHu;x zXAf3Cl~uop?U>4}6+FbwKTcN|lcUluPl6>k(>9B3QNezjd+0@>RrZ{=9ye#@eqiM~ zRiH%BKe1!`XgScQ?nP6A<``eS>9lkC%I)JHqu_yI1@iSqU7SNNRt?hvs14EP@i2ky z^p5rZ?L;+p^$YorKdle8e1#ayK&!}Tw zNvp9&(D%JwrxwvN*gvu#@yFUT-e$H4NeBnp?M#rdngsAT(MW{Pfqa9kfRWkgtm=v; zv&D~qcg_QlgVzGww05kkWzpsvZzGbX4>;%%i3e2g zyRAER?Uv)$cU?p*BTW}^NbTvt=0&TQzn-cx;%pYTFURvxB(=>0_f6ygj=GK(Z>P19 zb`tO0TARg|8sG{}C~cd(JVE} zNEL#<|LHr+*TZhPp2!D5Jk;|EIQ@Hj1>`*n;g?6_K(popT$V()hrbt7PT?EJ1I&T> z^oiIGPrb)cTF=mqE~B4NDPfOc2~y!6T8}pEcX2ZB>QY*wOGYaJ6R3jD3Z;Z|Wb;6) zAD*_8pvt|Y^v8b_kGqDlZ``O=$VDcIhb!lE;M{c9&ZNPEs>l6p`|!P5SNfBtVkH-c zH>Lfj!(eKov1EQd@H;}C)6K7E1c@$|hP?ae`9%Wfsy0Mak`~xi;>*^E?!Svbe;DTv zCF2n|2|i<-qsQ&yS`rDg#pmc%LxpuVHuW$#gA(mKl%IUu-jh_Y%AC$UL` z<$YeDs8z;iuQXlMF?e=YizY=dJi1=?b<>#Ur7}kOM^?cIK}_r1oe#pUk(k)cbLx7( zh%3$-l~ZG!=eX{Or2kM7NpP}4TBwJ9CO3~*b}J~BNUrbdGrng0rp|hjjj7Q{!R()K zjOS%!bE2F1I;}m2zv&R@w5^fL`sMt=W;MWy_-5V|@S8@kcNIFwd_Aak%_H*}%ms%A zu>R+u2{5Qe1WXIG*R3z{4=P|lar1xO9QtkSx_?XJCddSaSvN+3PC{P$t42M0Ko0+x zCe}xsh4F6X!)F**sj$nwe9GmCYCDU6Fi>l%8c8kaoch_LtPZ9#;jRlAW~Q2ocKe)W}+Q#I&U8%=OWWvHsjixkI#WbPpbUxw9;vT za=G5E87o5>XWq`^sIB@6H$m(V!XyMBu$Nuob;06*{MOzW9hAi%t?3;dr69KF3-pEI z?oVM3*Si^`@4<{&O5>+qOxgIVmG3vHFd@WrPv@7L)0f0@)LL3}cT4!%`0`|Iht5}|{_^}~9<55+ z^m4)zmP9FZCX)svbc-j|SrJF5=$*f6^Bi}wI|tnCgU#$aAH+FcbEv7r$0Dl>;>SeY zD`TzP<9~zh!fo&~K@1!1<^sn?myJP{b;wGSY%O1|+==m`x}ipPAuM9Sj=y*8pQRS+HG?Rc}7#HrAWWF$Fa|`)Z&R*Fa-u z`QtTBNBWz)b;|plh~KR*j?hM^1E*Slhb=;O%@i3RtdnJ4N6ExtC`y?Aq5yUmWir&! ze|CMzcIEfUVgVo}YyDlmG`u=3+r&z={kn++fDJCn?1Gk4B}^u zD>iUx-YzAgY_Lybw^N7I)h;#)-eXvwTjR(1NDh;4!+EyWl%r6(y&0B$p9I%PUhYsw zTN_;%xNt$RbX$t=xLO12jyjJB)P+Ogf;8eOm4I7=m4LQ~9(+pv9z9f*H7RA;Rfu9A z4KGW;{A+*;K^T22z!2j5)c4EfKhLttRymR@6}zHIFol*Bo|FJuRFflodGzC#+L-gi z35Bndc)PqNBmQLl{Fr2dyV);_Go1~)`4bR|v%)jut;i`=*mGL$)zgr>HJvpW>Dwf; z4$33ln!;sR_VYO#MgSR#{_GPv8^QbyP2l#Gvy4q9mJUbiFKInF=6A)s9Q6bNzzfWF z44!6mtmTfoVeLXvC;klX*wIq4Bdk~9K*MJIBYB2$_pu`xh8okw0hMcBtlG6!i{t@Kx4k}Y#VQc3flSR0figR5;gvs=?nx)#UA&)k z13jZ(a08yDEtr>pl6zdh5``-z5+u|ZACDIBqt%DbIGb4Z5`9XLdd1hT+!UAei}eKc zDy?IWuME$o(qvT9Wd@!U#+42N^d}nt;42R*)vb6H>0@IQg`+6F%mX3Q_T-ywwwlJs zNB_~ocN6JY==;}Mi<`?ujqBp$R=CjW^O+=CFvE+s^XyMa1vwHds{6sEci%C3i;#K^ z_Jxm1#j(%e$H8GMmlsBA!|9C9FFYk{m!6fz0#;02GpLosWhheOymI@;2>XDS<6xQ_ z4^u^QHSa_iL>=vv$a8M~0utfn{j)%(ri>svRe1cXZIn&JJMMrSyU9V66ZSlyrO_x1 zef7<^Au7cD$86$#n8mXw6Gk@@V%T=6TAMh~!{6F3yB1$DQNS(X$YbS?YdnHRGhOp) zKXM`mcLm_R67q~vZ=0J?_eEs~VuNg)RkG(*zr5Mg-)8=Ey5yGMW5qN{<^#AW`4V-R z+2ZfkXRgJ)D}DNeARMEjwuGXqYZ&Ya9TjWZW znN`Q1;Wo_`VyR~T0*ey~f&Y0Zc_Jte%orhZv>T0`HmtB zVsx;$E2D+5_x25hK6jfh**<&^(!5)ry#?Xg_BC06zbs&csQz7&k4mJ8v>67<#r0YOnd`n^i{ZEjJkmG07w4!@RY4evo z;Nw*E9DSx&1`+`#y?>cE2-+bodFYEnq+*a$aoZs2FGGh~$J+GaQ z=?9;)yw>)Ol1EQCfBN$F*)y~VzVH<5%*e}vb4TF~y8un)H_~}*HnZ-6HQAHwp}JYN z3{3|<+^2mLUIn|42;VfZCbqU{JYx9V-GU_rRXzyQlz+sRGznI`;EhBPo)@DF@k%M+ z_tg3k-hd4W;b4i#GNaySlR=Fgs(WkFBEL7o%Sr!^9MjfMQ({FSS9X z@I;97a z$FF(v>1CqlZ(@KQf#q?L*W4sp)`T%G6h80FkxdXcHV25e=2RHhrZ@ZTX`d}@v(Cyo z6j%VBteGh2rw9>Gt~KH?I2kZ3(cXZSGo1)p_JAI zH7H`rB?#j_#UkV%cB6s=L5$t!em93Fz--U5vg`WJ*mmmf3R8#Mus&h%%?bf*ySS#@ zr4BfYg~lzu(~F1m%b|3cw?2P=XrAm1Sv=b~xcJPt(Mg+8zBo-k?k|5_=n<|bpOH!! zl|zw`m*V@vyT+~RCZQ;zA;QT^ajN)z z?X|+E>oq%Tyge6y1|rwweV&i;4(M~@ydwliWCxG-C08KJdmo`#Ye)Nlcqff>yJCAOG#v2niVv8a3C+4OZbmw)?Bh zVNbA;X2zbLM7%qd2m85wO>KCy;@Y%h@u6>1>wjODX zU>LTPBA{wz!0^hjBq%%R!P2%j9}B`uZ#-bJH5OZG3LGs|XQ2kcYc)sU%f$jRtu|x% zBVQ?w!c%d=8h!j4>K={HuOW6xb4R1(SC?u0`h_w zGL;ZiC$4B9I$3U^0J}Ntig#xRR@3JNF3m+hMyuhSl=W3QB^txmdnfh7)X+`}9GPB4@L7*_7j!DpwoS zlatO>r)e*FTr3f3BUXRmqy;`ceIg(G(`j~B;P{V4cjrCHx#wya?oZ*5|K&aWlMH_` z-zBOz3GT7egB+`rCn;iBsDy1D-vHa}h3v4cJ%C4aeFy?`DfHlehNjAdf2S zgS)QwDu_ldQ3!;FbTe0wk}DaWv}zYNd#(Yzu2ewkLlguMyqW4d*VB{HF$Qk`96t%@ z4NUh=b^E)KcxHj2o?mpYry>Cby$!M9V-5f&F~AMBeC>Z^<8o#AaQ+@zuK{o$E^zgQ zrL9ZO;)c`LSnn=JNrJ1=ve~6>-@^QwmKzz9%8DbQJ74hz~RjsB-K_$!!BsiYUV*;LtXBZ z&gP$89bjpV(CeNbs&eZ)|zASwhh9C?ayuaNWof5fpEHmueKEFvQp z+DNK+yj^TG9AcJm0iCPPO+HNE=feR3;+K7VO{8~hl-0RDI!)u1=wlfv`K>wbvcT)iEPvsD~?fRG> zRS0PL0jyEi-%n*fGLX`Dc8o3NspZum=|GwCgwz7vC8#wFSybQ8{E@L3RJ4fWEXaPM zhzekFx%B8fDVmJwRQ65Lx~UJQ2BmmMyB%g&(PrJdVM@ zlX~YEF`5&wUZdrc)Y^~5t_Yar)VvGnAi2IPT#-kxtWPDvs7}G=19aLuE=)Fj=e?fm zIy=KPy-R;3j{aDy+;!i~owRb_u$lFhN2;kl``esYs(QQ1+j+nu^Z$N0!3k2hqS-(F zKG&!-e6`OJv{(;_Grh<)ofXc~`Sl%dmka|@854oZLTA_G&IoJ;Ou*PoZQg?(97O>m zQ4U5xRhA2>7PO0ojp}z}z4rvRLOI7L$0HB($0j0AE#s~SP^YgTbpNcXC$da*e|1`D^wI#Tz2LKL zYhg^0xk4DMOX@$hipv*-vwhA+6}X?C$TmckOQgZ=uL6dE-zUFJNaL?uF1w?+vgTje z)JIgpTq}|4Kp!{LDo`C`hge>B{J?8;Ds~V(!NR;CG(9sNXb>{bHRCXwBZy?XCs3E|vciqfOdRU(ocdwM%bM-|T7(dZKHSPEpMw=itTgQ6*ZXtna^)DB%N_RbHzJB2?zX%&2{h!Ejq{n zWcYfcuB)!pfSX*-@DM#sC)pf$-;Iy&ka*>HUrwPVST2T60H>A#FApFI|39ZzJa4#= zIM}c?-vGd8p)tWn0~1w~YXL)a<>R6C63LcrqibcRFz;p5a7~b8Ht{FD3nZ3><$ZRH z-*Z|n%$|=8XbhleQ9MmBAQES87^`dJC+HvqGCj%wON4Ljj+ZvgD*oil<48?>L$5tG zt-sX+OsC2n`(y|7%G?UfOR;EsW8Eu#OY-r?Nr=*6dsc2J?!yvF=kZ%*{?)J!Hwe!k z0mWdGI2(cq%1*c{BrUC5$e2coUsJkyz8T)eVe4j)>Cw;Kaua9X*j1HX_B{%}(ZF@& zU6}BrBw!tS%OGucDvQ^Kl6p!LwzJeMH%kBr_SHX)2_G1i%@d_%PD@?t=WyQEwJy+_N+olRXp0` zE9~SYKGK{$;B3Fg)eH@EdAbyGTr=l;?xT_IgoDJ1BA5(yIV{2Bw_ZOf9KS4zpjhsq zIQt&^8RuhDn}CdUn(VucbpExVIm0Oh7nEhYq5H-{8rPx-l< zLA$Ywqy;B1XPt5~jko2?$`xAgn~b0xM-?WtEEQj_Of*SS#CZ-}WO?JO-z(GeTw3$p zvJ%N}CFvN%#sIWV!be52-doD}uS>hb^>|j@F{l!DQdR_B#xV~NMTk*m?HV|F5!=36 z#SnK+54(A>AUWe*tK{gjZ3~O9>&&X_nz;($$a_M5V)SyzX%?=CmRBcfGr@(WO8q*| z?ctvP-;>B7FSRgS*F!2w7WWCijB4yc;NRs3E%WMPg-BUEV5$AbOI^l5u-Ha? zb1F6#_sQ0vKL(jy_K2yrwXb4m-Rlq2?QS`Kq?fNyF-Msdxei10lg8J%%bwkR3I&S6 z^BJmr=}DcnERiKgut<+c6{MLfsgaT}L30whZ&mvHB12^a*9AR3s510Jnh^Av3NsgI zBesCMH>dN#`h9rD*tNf zuU#+foSz-?7oeu89Meh+8=f73rzh|bDW**Z+Y~@jv&#ECO|syuWR__j1KfXegTp;6 zh5LqfAAlpvt0;K|IMnfCuyEBnP#R7YSBw!$nzsp$7j%}$&tFAGqj?TBm3}wbg<__N#YKyQr8d{W;G|c}mrJKB_zbof^v3W*tkd0D;jZxg zb+M3PD}eUL+-$5US)!+WOb!R{52o6@__^J|x8|O$xwuX=(&cOl=QQfH*!*FCA#>vYdN2hup&u6;GUXFBh3U%~A_v~ewVnjld-Y0*HXRlknTw=O zQlEa%#;33R^$Hu*eFrglC#JkRXWsdGS%FoBXgwFpM0|o+(sOOim$F>%|E{}^p z^us#GEmWB4nBstNvQJMH(ceU#r#{vbU%UQ|l5M^Su5b2o`XZ&|%eHWRaeqE7`D!^y zRc#;n1tMIW`d(_)Ht-OzFQ%EWx|WhU$ftvTc&*2w_0Dwxq4y(k-Y)VZHL%ZRXCoCC z*U3bSLffhv!{@r^p9gSJe0MVu+Cq%3-Q5=64L&y)hpuPSRxD$)K+AwlPh3lJ5V}|J z!Uj%vpjFSTV@n3ve1z_?_4E_LW2%X#Gtl8I@zTtnbjiv|nbX~AkS{cUDG3ZS`9L}w zV`O_`GP0A!`m)zARKv-+Z&q1?AyN$8#x!hC`r5Dh1nW1azCFwN>d%{kJBjK=sqXW99P-ED0U7Rh5zlRGg->W)Q+J%6?!XAAXs#SKA z`5p^p7_7A1_%XBCo(sVGYPm>d7V6+j>donyyr#>YF<>RPETn3L9AW9hB` z%>tldZc7Y0Pf7$XYiwBH&*u8j%WP|sx$ON(PgK)>OaiKN9Zrbe=5X$LifG9Cj;TuI zcccAj--b*87;feBPXDjZGh4uY0w&l|Dy{tsFu$T@L-{)uuvaFk^`}-M&_yYcCsKIQ z$vSA{N277HN(|hu_h`%~{GUk?CJa+j1_q(UxJ#H%f`AY;(Wm?DI{w_TZuwpTCYBUr z4;Dh_oA`PRDsgU6@6!A({f&6A%6;MNJJD%5JRY5YJg~B?*X0c+ZZ0kLXbAw{hWD;d z`j6*}X#$zlnmZ_%)%Ifb5zD$K#B`O%{nEYRgroL)uYn|*jMPT%tXLWslwDk?ehe;O zK0l1lM7n9F=|%n+k~n`ZXg9&Z{?FG#Qcl}LguB2ac`QL?VpN)0dn8jVsh8rV@}L;@ zCpqFJK5Dy<^ejA9Z77YJ&V@FS?Dp3vk39Y;iHIkXH=nNTe>N4s3(d4Be}|aNQZ7TQ zfLD?w;SI?v#jFsP^i0nrIbC}3m+efcSPT=tO3&v#7U7HXx8}bpx1;Hp(YnSNZyYcX z*;P(dSxy;tE!;+T7=%=kk*R-66uShA(q5eVIF@EZ&S!X&&v8XEEIITbbVIOMT55M$ zXCD9uW8LbHe)*AJ!aGjnF;Qq{P2~64(w!KC&wwyLv>T(cXZhLMarM~6=I~~^@>Hd- zU|ITaT#^&9R#bT1z9FsV?bT@x$mfbFNprsC)v4Fc3(=AzEu$))@oV$(04VdXjuxEv84DJqug88Y@*kCZHNV%DH^giN z`k`TQvbLxUkNkN%;qM6d8Sa#pSK>(g(bVnzK=o{SV8osH$#yoka%)@iQ$%Y5x}YHB zWrLvmIBVHZa%@1;oP_#(@3YX7kE6kg(zD*ZxCZEa%z$I_-`ClH;JND0KCK0o<7LyM zq3vD}Z1X~GrLnHYfkS*C533B4!s-kY$zG}l_A>!k40lJ|uuhe8Tscv(#N?lQy^HJ= z^_`i}buJWX;M+1OdHhYqQnv24{&;2=<>0K;AlojOJLmC0X?;QW?;?`rc z8-rcFSg*=DoU^*ysI&*#&Nr1a^Q@sO%jtg$jim5fseQ&KJo3Tv zp%!*9h?=BglQ#$Xqm^QG3R_;#*YD2s#aG90+uXKUd3Qwl03zD>dYqm0#CX^7DG@6{ z^p7gB-T3&H9=A*`{c4Uwj(?9F+bdLTSA!jehCkXNELz62TE?0ib`@Lh(nl=PaK^{$ z5bBExGb8`r(S4Yt+0WnR7G<<@IHC>U&hOV%w5#qz(*wB>s}aFay(*oAkG^B59pWph zsd%T42^jlU0SvN;CFV}A7S;Rf{T3wmeYdBvlT#Z@NF+z}e2osIX5L*N@C_Wyuge>T z2O#^%@FCDhzWX!)O#j9nw3JZeROU!BO}V@Tywz6cz8*G?&JF2CHH+)bpm1K}e50Be z!Q9Vp6rdefD+Ln|_oI{P(e0iC(5d%q#Yz8Fc^_0Z098Xz*Q$s?h7YkDm-T^0eWz4m z5K0bhbRBMSFM9UlqE#jWl_(@m8%>fW1;8+XRZKww1+XeJV0KL!4N6w#^BzSfs$1`= zA;18=3?kClUqw{)v!}rVCVQEcnlIsn`#$CUMFLFn7Ue*8Ux%jvnO^7fGPZ6azIX(W zn@mI7sw|mYV+!yX&<%VEt$m3=VqP7hV zdJ(Esei}^-#r+#H3dpe3d@#6K5u~iGK4zJ4a^Px7U38~ad?-bl#3-**R@jxg%@i%; z=HxBRZ%gM#^Q42orzNQw_gQGM?a{?bGz+ahLyo|bavH6$8nPs5{Y1>WXUNrHw`sE> zaq-peBJdP<1#rl(Boz!}WhNe~sX!M?pK}-Qix*#ytPL%uEBbJ~5$WN@ltNnyegwIA zK+E%=^1Wc;;&q>m>qdv$!29DL&60*PSWey8MOZQ=18z(w~4IKzy@_ zM67#q4W#ggHqYio2_pv&4u?kkw@bd4k`LDOHko#WxN0RCX`CLAY)nbibW^A~Y>wv1 z=RVZ-HE~ zL|53mrClf09Y6eJJnJ1%r-27)KWz+Gjk2Bz+@m^Apa4xbvOp2?(<|asPCfD`Ta{Mv zzKUqU^|vJ(;1yY-A!U4?Z_=FLUFki8D0|q;(-$HlT{)}3Gdl-p>=?E}Oizv_P_%g`{dS``B{eRyn6qQ z{Dn`0MFs8>?eTG9Pdp0?{el*_E=lK%lIDGpa3s;efd}R1eb9Gw`Sm$Cx9=xI^-_DQ z)OBWj{l&&8hT7c2ESYOP-RwQ8`4S{c$el4C2Y=~z?GKNP^U-V}xk*N8|NRHMe~ouQ z({k}SoUZ>~{n16V@;(W{v!)Mh6&bmEbk>9|^)pZ{h;_-EK z)%`|UnzA>4l0RG|hc&cU^~-%yT?V)={UU{v9ApYw_i2%C*nXBnMKf&gGU1uoyh9()6<}a5uac7gtpl+cprry;5u@ z9V0Fflrl>fUN~kY+>Pa3PnszaXu8|96)m{Y($bt)p$~Q43H3}mRX>vIz69rF!$KeivvB(Q6hn2oxmr8k~fgnb1C5W zb+6V^6~Ly+OiC)^Uq6(0HR>bpk@Z(6S^GR8paxkuCB)Ct^?XR z*Wfs%+(8mSvfnsrtywWhy}so+-4n$Mk|AlN!v6+K(1n+r!#Pqp^Z)*vlk???}5#`g=X_ z&NWCtn#G0)@__+ENi>7iJic%|8zt_k7^?)$vfrVg)|}HHqnWqP9C{bO&(#*2I51-r zsk*H&Zgw#2QoUJcq67S`T`r1W`#;{ay!y|U^#9m6tgd*q1eEKu<-rS1GfS3?=*Fzg z$-^FcMD&~*CK$Dys&p9Sp6Plz#O4K)LGc)H>1nT@8I8-RO4Zkvdox13pyG5@VIfIP zx%|0TcSLQHoNyq)>MSc!tY~hMPdM8T(c(t=7~Bu>m}Xj&q7y=gKdEt?I<2GWH8^ZwNR_6k{*kMBSmPXy}m z3KAdVL7vkN;Y&|s_28Y%;Hww)l?1$on*w5u_0Ck;7A_gky$-7105vx8 zL$Ta=Jh?yyaIp9kz+;kpgtFH}BcvsU91?D~{*>O8ha`+|u!{d)>glw&!cizR2>z;k zv1&_l8ADGK62D%1Jtq|5u#>{l1@t3{u#hfSa^ER}X}IoSf{~kUubS?T6fX0ls;T&- z?l^7j5r)E^9BmzVv}Qsoe}JlG#=}}OmD*TF*-lvP@{$;y*$TsC?gr(c#X@oH2kOX!`Lc7EQy;gZ-0XuyFXP57JB5Gh8I*2Gp% zOr`+laB?M)%J&ENr$Kpfuol6!JqJQMuD%0*kqK)4shSNYpP_#{Z|?FvQ3{?9Sp^Gg3TD)$_Q!;4PWmZFJ}FMbK%jZ284JZsIOuN8UUqTu^2Xob^BNW zF;*IKlF+4?eG0v-XZ4}eo%GA`eF|fF+LqpWQoko3a)~tKX9GAcPF`JyZqEk~POnQ< zoHw#_X;m4Nlv-5&&x?hIK44hC0+jOy+cnLoo=Tw=JDZI*HjIoekgXtxa5KLciG{Wy zmhZs9iq}?}+-z?0j(~L(sAv!pTt;7;u?sMg%eUt5Jm$o@q zWChjnYa9)PpPp8<`UXMkCzZJzS;;B<#4D#FeDoqaKRwH4Q&D`}HFHv>Fd6TCCJKrK0o?{)g3R(vA ze`}JA9~9%|z8^k9+?T`fLtIH0=&v>yU3HP&L%i?-FC){qDnM$%^Q~SzEH?ovGcG?|`jstz%|5 z`H}L2o4Iuvy1o42Gcn8njm3_3kxjR&Wug?>(C}Qvu{pg52TRKNY7@yK$yiD(71e*= z0e$W>*CSxQmHS)$9W2~z_iNH@{5KALT_ADk;N!(huV1Dg{tQ_~-&s!6s^u`X= zKzuXGeaaLE-24j7n5&p;6Vi0Gkj&5G-{rj4Yq2e0y^t!pDB8Kz7WRI)$pds3Xmr1b zsN6{m-Hbw~;&JZ+s!?@k`|Hyz05w5=;~m%StHHb{T`%nCJvi7?atiEb=ZjL-l+0x%>Q*!$+*r;x?VU3@}c`==D0uHU9rS1hxP|piZPLoTg-Z z*YHUb#pCPp-wn4YRprZa*nenUCjWoA@;g>QML)*7t(%=K=MctGM6*kKaEWlj8ehJC^r=H7=tUIL|*x zRr^TgHQ;<~&zg4~Q*zy{@>+@IYp~lH(>RM+7x1?Ct3S^F8DF|JS*il8V8Np7buHOZ zLs>GBN8(xbj{Oy7GZq{n(oNDU;u~rJ{BcWSkJGlokCv2==nG|2Y1s;5{I1vcWa)*M zD&a0e#B*^;)sGGh*T#BM!Uj_>aX^|swBmhrZwZS@mn^1o)NzfR^ZafC&t}-;S8y%b zgYd==g2qU=A!$z#bLE6ezT!iSdyzxUIYWL!K|YXMQy*ZaVb~G#X8d$*Tyhmm>+8m# zsn?RQaBnN9Ia9?^WjFE#D9Mlpb}{4|-D>+U7xnZm)DJONxs1~;UAv6G4Woc6h3Fw& znZ)-bf@1$#f8!I`!A!Tm3E0_Ue=A2tC-|J(9iIeBhH_;B9Iv#lTqtcJk|M+lfDH^-s;_PG{)qPxiHTpPPe2McZL|BFdGQL1snn)+9eFjX6aN zb`W^>MwP5YQ{_pN9<96egS2$%&Rjc-#uu%o5!#7oJ4q`}!HtP-xxy*iMgSJzBSvub zuNF80*GoU`pvQw-aP|b9bv$xl<MdvUx;VE^F4+omdz`*k~JRN)dLAL`#K+hfYGUP(?cHl58 zr7XPH!^nNk0w@a0Dz2L!$TpFH7C94)RJ^~twV8$143rRoh^Wv2f|*q zumo4&`Ijwg(*Uz#E=OAIvt*B{5_3H68vBmF^Nj7)qstLAtv;g@K`@l$LH7rMtV4mWDwZ zq`SM$V(@Y%=h8cmwbPKSMI_6Cxr`6~67muG-!-W+;$~6)_wP00lk(3ds{F?i-;N z+!k*7OS9LyCbf0}s?J}Xsc#l}zS{4Nv0TrtyL~%k^t~fW*KKjFO%_~&#YRiXk$N19 z3c69It2PmO*U6_B+arOKg&HNXvApePtF4qH4yCSU5;85NmWgK1kZ91}=Wo9At+8lp zz{9wtL>7SIZu;_?Su{aH)@)FM-t_$nf^0?kzq$wpNKX4chk#ON6L(1$u&s;-{&#%Z z(?;WJT9)ovI=9ijPUWNUN;VOeTsuy02)1o72Z!o7fNWCA4O(mGOa`XmEN~fz&vtY%lsu_{+R^0O zYb{JJ3(&-^{0Sj4t8TL!1TC%5g%oa=G_d8HkFCazcb@!f5PZUm`s<+tb&EHqCo z#?}jPx{oQED?Dv+-?Q*W!CCDS?sn#Lb1xsAIhVV zE85fdiyXwm3jDs^o`p5?`Z9b%t;5r}KZh3@EqYxBSn>#*jp3kt`Za10s${j3)j|uG z-pu9=%mws*zU{(Dh<>pUQ3{!k@Di2ITN8dIJ>7*bmS(3zxwYP0|5Yd^5_vgnk*Tbp(%C;_f^d9Tth7!=4Thp%WO4gt}@tpTLJy`<`kbnZyy&Bm0kcc@(D#i$l&JPQw4Z)kSJWHkN@Jq7heuBa=_I&g{ zouYZtnX=b(ogw)Xb(;)llPFR#!hWCK+__{kVJDa!IuY_s@}6U1BVy799DmjpD@i=0 zoMe>sQ4;bnWi`umGd>S1K0I`y4INW|fdJ;^KKXU%ollEZ- zVMG{-q3smw)-i4k1%9_ddS5I}G>g|!_V>*rW>#)Ku%Zq{-43R2_jf)AFP+e{Z1`9q@P8Sj2ry`>tvMfN=J|I|wP1U~T@fGczZvs1 z=115TQm2*EOzR%!wisyF3o>GM*bO;QU?r&Uj@@9N$&Qr%^L16 z<}5bfp8e>rHn=CM+Ml~9p6pnS63YPbD*@SmIi|FZOOMn0BA)l#pZest2wky&z`nrUlTezIbu+)|6&drpCAw$q#_1n!)5*tni% z6ePclt2Y)bwrlcw$J2yd2>%9qF9C^)rlRDPvN9&8`776fBKEqHmq;`mf5-h^SNRSJ zP`3R0Qk_xJtlk{ueYr(`^?N^X7{efy(T^2Fv#P`eD=SS}@#e#Sl*e~e42H=0#KQMf z8qn3Desq3(Tg@;4i|~ zy_HF#nGl6bU9a~jg>QhgD`(Kj-wCgemKsk=y}#U;hGHk8s4G!5-DsJ*N6eGvfI6^B1@4z<|3?cDElY?Hb)KT& zzb;^^l=oB&DRosIp+`Xb+yu=m1H_kLrwacR&SxwrUU89mSU$<;0*~yDup}*oJ%rld z{R70xGR@SKG}B)b_LcZ{r%uMCX&HpRthGEt+b;{FN9b+rU%Krgo=aNcw%*dDy!Fvl z^zGZMJ{^8H0TAR1I%(p!L)4bc+oZuP;v|059s?lQUo6R!CzbiN95U8jB!TIA;nWb> z0RCwp`m=$dI()b&WG@BRyL-27>B9GQ#%Em+*O!tBerzNIm?goJ6tsUD1O{m_Yxb5- zpZ)4ZzwFn?c1Yg{*|(r><3B3B5)o`8F#tuSHuh#J^zIq_P`|5X@UGbqEL6VBGxY|k zm_E?x1}K>t#EHJdM4H~;5_P^=XgOJQy{tM^KBa^jVwuof_E@;iRH#+DQH=j*Bv)lm zW&q>a2mAofK#stwL~}C;f8_(J11l$a;-86ApyyyTj0j`9BlGqB&7-hAlxQI1W4GW% zu&|ith3x-{F63~?dmb?2vD;Drv0s3Zg%w-C-hP1&d-O<+`QU-c>)K zWqUJ|H$!EAu8v7uP^TiDg`S!A>%$@YeLUVwOJI%Iy!+E)sfxz@!yZ(JS+?(_)aY`o zua&wgQ3TLcKg(wdYrtbKBD4W3oCSs9lrXIjZTL<=drb{u8vo;NQ7SWQ^G=Nv;~imp zp5gxPpx%VABA`o1{arTwIrVMfKx*uI#4SoF#feHJ?o#gt#=l*s^khK=>{@=heqHHS zw9KaFgteB=9F5sT_D|Hyq9${Rm8Barte4IwRUV()bGIxfzO+f}U5nJ*ZMMY-d`GfD zLaS?z6Zi`&hiipXFBv)%ZSQxwX{pZNMCnJPWaJ*ae{CtwZNaXB5pG*?g9?lVgu zlQ$#H;=wvEAg?3OU?|s4Dh#nHWc~!uA+JHIeM+x4b4qptxN@B|O}1i!Z`N(@GYH8Y z`W!9zd}cPxy?zOsVj#}A%&ddAcNZ(2kV)%Z>sR!JbJv3uO#S46K3Pa?O)GkyImGldz_lm8*RCH>7h22ICSg3yV24P&J6wWNarM#U~wb6XH z4N$4C_qXdax-1{l5ffi2h$^RlJ*@}Lq)--5{@l8%0bthu3TJA0S5rG2=B6497CGsA zm0r%l{Nm?tLto3(qS(TTT(ETA7uy7vnR1IwrA_k&F8vCye9J09<{|rSgtmZ<4EE+a z{;GN8=H3@Q%bje@vxNa18Gb1uN!o#Uw!{|Oo%s=~qI5Qg{`XYaCuTiP41DVoZ zPm#O!zCCba@R!sBl%Ghsa9!tqhDrEg`gb0xo}JXWjeWtuZ@0kE+Nyij__}9p?sDt2 zFid+aj)^zA28jPS1M+N`^i9p3pjJs1G0+V-HQ6(Xg%9%TqZH_y_~+>P;YXn$A3zHs z4L1D#<*>_nW=ElkIJuC0(vk>z)GiyE%B zL%K)SbMJUBV=%4P7SH+@YM1~IAa?C*nenfLLjouI{x@$%%E%92C-~cvU0TwzkIzeu zm0}+A3svflDu7i)?~V;vWeS8`o(TTR+P#!MztMtpecNpw$|2pl9)uYFg`<~XX41ym zBl91%asRzF^v!riZvQW8%vbWdEMr^SAvZ}N&MtRElqY5mw#)0SF4fLe2Wr1$V%$cC!n1v!6D9f!BgbZSq%Ataa4)YB7H4=|X^72jRrf zUzrX34Rjky5qRZ6?nv>_Q|GE(;*5wkaBjunm5h$=)v%tD=06zTJzZCD-WDI3`#5GN zdg{EMMbqg+pBc$!5IIBIiP=f@OET!wB1-{V(V}m1ilD=R?QRld?^wX)QX+YkQQF45 zz)`0}zA#1T?vwLP>zq}~tk&~<=d~(D6ouC~BOei3{A{T!qA6$-HL2Ee?8$HXO!To^ zxUsk!UTx^Z_Ic znG%_MA3$oi_1y`7z3=^wU}qKmpBGp-chh*j>{gp`qCT>n;=JVL5!*K%uL=g7_P2sU z4s+2Rb`naE@#{J2fT9WDf3!UNlZ8WmE28IUtwI9(+N|Cp7%^<^qwCDk*)J-3KLnd@ zbfvBI-=xAqIBo4NjlSuYXQUyx_cV}mUvSg-@&}Nf{M1w2$}q{o>&Q#Y1EO2B>-uVu z4#DEgIu$i$KIR!v%U<-%817#>z6t_Gs(-ctX&n95msu7f38TsU&y&EOG)?MZ=T%DFt z(HVr8!RjDW&~7!&B>0=fg~Lvk9qgpR`eC5$r|7KiZU%DxXyg^T_ec*OnpEy_*JVhR zmb(jYnr$EL9hI}4<+isFlH~Njp@}KAil8}8klHlUVNy-INQ2L(t%%MP=~>19E~>EP zk#VxuuI(~Hcrwasox0P?f&|KZJ>Pvm%6gOYDfHhp-P&hWrNc7?7ABpg()+}Nq9MDS z@#8z4*X{HdM{S3PEP*4~@jv|_TN7wm{2!@k%4+Q~z{%0lr;Pm5m=v|#i0GAJwD#(i zE8FQt9HT4UQ$*vCiY+i=oN4AEJEv73Gx&?JVUOjV>|~PD}$Swu2_Osx0xt$PlXndWx*u+!2lMQXb@_qlB12gGrQgSyUE|>{mQ%4 zQaT(>V8*naY>^=gn&9->Q$VM7dzn6cqigLpZ=~h)Dv&OGiW_XQcjPD8;NHov-J*bM zJ)NCP^cSwb$E56YB&ILr z3iq`eSiMg=bXkFIcq>QQF{ij@XGc`qr&E2OV0BnSG^x)yZppn5gPj`t#0$C?ol(@T zqZ0j#!DIJT27KOdZ$Ce8`TLYW)j&b&wW}*^hp``1CnC|dH-F$ecyH@D&wWh4MB<6v zZ2fWt8^w%SeSk+Mr$ml2q+2+PV_E@s0rfE7;KwM%Q+zZ|)V z+ry!p8Q+7|^Y_sow?e>I>7_yqo-wcT|7WzYuDgq}7R{9PYWGVf50^^aM5J6#SlWB@ ztp=))@LaIo9HWW*{_RshU`Be&w?%{cBs9I5j8>f%)&hJ2td>d=vb&FZn8>eRw|l0n ztkl~T$iLM&JanQ|@V)3?v-yK*!_~C}f0(LUl3%som$mT)xDsJDz(lQjx3p=4iqi{Y z zS!#5@>e893k>XV;Pg9sm8yM!2ZD@eqA~n;~Q>dTpBneco3T_;2t@n^;4E z2>Y(lMu8_rt<37rj9D%yftCI9cY->Ako=E?f4)~-PkKR{ZR4RTK^L_FiF|4$bKGrkrtJb~{ z_?!2swWaWwFsvUQgvW0fGL89oShEi~8zh|*5_LqB89>l!wKF$HpllX*AP5KtFfc3P zTqX^s#ItSX-=B15wupFvsnssUcPVmsj^wuBB_qqSI~*IptjSh5+ZiZ@)!5Yv?+C81|$OTo%$R z%jH$W`$>9xn6$+&@<4F%d~~Ctmu9f871#iN@eG|2PuS1VW;HYsTA{lt}#u zkN)By^=+EW*ga%b7Hu$y_mMJ=)?U)Ndw>B~I2-0n?`8?&l+ckvU!C^hGkt6mw!RK& zy2JsKxMdZnQr~3=2v`-#DrQaq>wkRT4NCG31;_|Ud`SeTJU{c9$;JO*@<sosUca8JPLFD1IE~CrE47{l+(W>*^LgCC5 zeinTC{@Xqa+BV1n9w+)_J=vmxhCb9R;naRgZBV3Zk^GuE;%bS&^E@YUt7MZFPRp3M znLa2P{Gc2K>Mr)GpX()m8y<8Qd!BvZb(iL9lrblW5LjgMKI(Vwd#TC=vflF-Y`#H1+bu9s$5b z@(H@@5o26UoD?de65(9nHr$a^xJt}BSlyYX)oVa~s@gB??Me0yX?ubxvZ|QP^0To- zjNkk%7PV%yc`dm-y}u-K&@(JdVnXKE<#clG$m!hnFSzB;h2x?z+*CQPRZluhteX&rRIX&)9qoz%Q85I1O?(*#-&2ykxfc=53EJOcxy2 ziDn0(1|63?AUi#;(#@Ris-R6y#Aa06W0-wg4Sodlmg43-^Mh6I!bP@9q@{dcKO>hr z#Or=LWGh+KDrW&V{qct{cBG4kb$nWH2c4$grs{hhh?4ZFTlt**z7`)(X|M{|G94S zSvFsMM!F6rb1>hVB$0R+5p*89?Jje{Bhd}^bwQ=wI*C>QR^h=6u3kUV$1HsN@fyxGSO`in_{J?CWyVJDglKsJO)8#$79O|~8 zzbZDe#VxzQUV41=0rD7{qvv-ZKc(oh6E*|7|xYC&G!%KJitXC~Qq&4h? z(rHH~3NKR9VRaG=v+PJTfHJUkA`8!pSj!M8fOY7WC1;p8im~Vtf(4T+AUfAxe?~oF znu#^3EdF_5-sbdsh35Kilghq+Zb++`>H0~@KsnWEM{Rk#+gQv*>JJ7c!78Qb>+CBP_2F+I3BS+4zjDpy!BWnD94%3i)Vp4ew4qt+wxp_D ztx@UJ`**Ep=%-l0p|I4p}2GqEzy`*ihi()VckY$8s!`iih-yFQ&#sl{_W50BlD{lYepdzj3e zbhUCof9}@%)^(4DWg=Q%>w?k!t`slif`l_qR9?4b+?Dmqa`%2wv3zTn)}nv$p)0Ci zrCcG}pp&!O(l+Nc;eqIVJXR<>=&QsJ^;+H|*&?D-LsYr~acnmY24=Z)#;S1954ng7 zmE}aW3l_KBKY%}YoZ&I^_wm1(?=N`F{0MQlurSmDsMHuzONZw{@q5ZnI{*8k`>&5o zgwekNHIsXf7?eZdry0((s%F;wJ-Z`xZq)gGcX)2A)0GI}Xo3#nWdQMNItjSjIjY`J zg0J1Coyk;l5+@koRv+16Jdza7{i9yL5ads+)MaI_{L{o=FNad?;8n5n?TkRKKWDXb zI(kg-P3d297y*TyC47F3=J^TU@eq?&s0>A5Rs@UoWV4EGn5h`cgqJL}bNC3fNcQ&ct!E|rIH zNX2GQf&&LC!SU6Cm+>wGX@bUp%7GZdB85RZQAYzq9RA`gvgK6SfCw8D&sx!-n_=K1 zaJ44Ttic5P`ob7~SsaaZo}Ey$Rw;MhDtpXah^J)Q-w^+jX?I@g0paMWeibNOVy5Dk zf>H=`T1Wa-P8w>nnCQ1L_9m|ZX~P?nSd{&L-!M<#LAgv;;R;bWTLrhDI`A(~<`@Y9 z2RCIdyHSdAmut`oGvI8IN*IaWXS4Aqur-_vuVpVAE%3H_9pz6;EAyRVS>-mwM;j-Y z{{9;Uo&W|P(?`uAf@I_{_EXM5OPg-%<#l0!)h1o9t7M~yXCV45U__}zKqV!!X-vms zs)<0fO+KfXjXqZ{#l`lB+Z{+bl$$UTdc(1o?mz>}l$9~m8i@{Iqg^@;-o(3A`E&Yx zJ?zmWBe8%^3ivl?o6kG~&K09cnW8r|fsv&vfMgDN&Xa9(KdMpDaR)Y!F1gXO(# zmKp^c?j0}NMq4{8r`K5V)hua7W?gv|8=!Ld?0jOqS{<*Po|>eJ`)T;6}Lij z%Xm3Nutw?35$xTHZfBW+OIeN)@rpPSLRnZ%^p`mm%Uv=?b6af_tg^gz}H{ zCh6w&)6OH=fs@Af9_Oi7+rBhJR7(O2h}KQ0h2D=~m8E`Yk$<9D9>vo7tjW>qbHsio z`(c91bOk<6{e7Zv4iav|MK4-BwnK(ieO_kx23l?v{%d6)5PXFY5Jav*T~z{Y;Do;vB8y zGTJLdhug`+Q&C<&`oWqPa)VWqC!fCb+6NsbSbT4avJpD0cE_8U9@RfU&W{t8^7I?Q z29d-n;`)+dwE(35Engf2H%MHF1A@P~B{7p!r--E*y>FvbkSp;SD__FE1`C|y$7-|n zPRb`2R`BP;)o;5S!;jBH%Yp^Da-atKpm|;<%!1AR=VUEx|NFmxdg*22{a_Yr?%e}4 z{4Tqa-tc)eO25jCX9%2K&hT<%>V#ik8V&e3LwcT{;QT-{lQ%HC<%#l#iKq;Q50=7J zs6r;$rjN*Ce{`RAS!hm576#!w@;I6x8PUHb)6qx~#639gXXiH{T}W&+@~10UG$jov z=T9;YAII*^YpQ4%>sgM?>`q0yk8#WEWSX9kjZ~TQGM$_8M(S#Hh{Ez`U^$H&{g%)m zLCc}ZQEpfbMXY_7ammN?%Bjc`B?J@skgtgwp-Wl zOF>GcEK}miSc?!p0k<8~2Kyt)@WIs6?{H)+OQrrDmRE4J9bD*cHqcn3a52b>sqEBv z?x>RG%lbnSl)lEFD;2S3B_d}QS8g`cNeTyiDmpPW#bv+rHH%HFEAmr(9N6YU)BE4o z@1FodM;8~w8oZR)(DXS2?LnNG6i6^ zf%DznKz8O-@&r?qw%#6>S!ddN;GxTV97J$v!=CupMKZFt65jXo}}0^2HSP$+*Hk3$q zl`C8$#XZP;`GlfUvPZ${{Y3aE%SZ7#b{}_Ke;#fIwXyxCZly$u^YH8U{Lq1l@(QI3 zE_0FT=HIM99D(aVe6soSbgMgVEhVUm(nCcgknS|vA_(GMBT&-8#A1RbycN)`Id~!g z9n_WeHeOywu}+!)9F?n5T3qZbDYUzo?JrS4JDb5|wyooPU=rp=o7sDlT6v(SSKjBp zPO?*lZ2;hwR{{{6yHt2Qpq*7`f7HPV-pp1FU9gmT{s_<>XulOL z@y;B>%WUBO7Lf#eAY|E=!NNnQT??eetC%%Zd>2;UE43;S<)m5kT}akwvin|Sn)NQq zTFJQ{7;s$7OkMAj2nm!=$@++*t1QQkC^e0BnjeoM3>i8hyI)#NVd4t0?`A&%u)t}8 zzZg*SYnn114Cc#t;|h8B2>E_=Ku2mx3m*BR z2~jJssfxv1fzYD3dN#tR%adh^8l~RwtA%(*N3=|-OhCjWtBm(I+CyKl?azNuN1OaH z&TAY}wQKIeu?~`m|7Q+DYKZw~cNz0%nf(>u!Un{3EKx}Wx9|M%eVi$Z#(I!5$?-H> zzGpvQw3v*sE}}^EUd1@?;;=z-WI=mlFGv-;`u#QT4G$NC#^S7q70?-cnhH;$@A54c zODWEVDMyr7oE*Sx9A#r!Oc~vz0_bMzp?LLrcQkyhP3zv#2thPlu+<`?{naPjxB;J^ z_VP4>R^{%utEJKk2gxt;68Z@>`s4>oG-6|Q`M{b?ZSK2xP<7d|I`U5u{k@lO6O}NR zY=AJ!JRXNPY#!0xKfJ$9jqn}?mB*eRXXNgGhC%wIly=ZBboaGZW6S2Fj;jI_PG*Qt z$3sQQB+D!jEy`XA+Lw%VvEjk9oDI^VCwHJ7Cl917(q%F1fsl>P zT+1y{G`SQF?7})p(8X5AV})nQxzk+QFPYt<8hy4&iCgu2qY^$;u+y#f;;U`y-1zwW z>cF!qKk1@*9-5*wOP#44C>TGI+uxN<+qe;PRx)}t%W$&9Nk^D9}P zSZ{06rw~4#A}t>q^{#h3pMp=)rNy2in1-(x@25Ws`uNrH>UJQJVio`>6GTVi(aP}@ z(Dci-IhtOM={DRBjt%d4X%``bV);<@kN$UKNX3eC#9S+nwV_e~B&n{#0j!v*aG(BW z5h`|bd^nry7^$2wdy{w*>LjcA@05K^goOcyg(PRtV)B!&Ku1aocN175c`{Pa2_VP}yi-XYD z-2~jOK*Bk8z`b+mbLyQ;;8%DQV;jU8{f&f^8-{H7O@hG2yVCBixFe6a+-L|N=<(z^ zi+21V<5K-tnalkG&e@zQVTszI^E^?}v2PNY6P39(L&hb+lsj?yN-ih8yRf!oRbvph zdh_*0>I{o2-sgYnd|VXltUSN}CH&uc@_)72$Q($Sg3s|i^u3}|b%3m?GENYQa6ec4 zCrZK+@sPu5oaPjZ0Fh))7on@06=V{51CCYqh%elfO`wmINHw~QrN`68`BJp}C@%&y zA(J6l_@-A=YZR`BAJX%EA0Mtqk+LFpeN;Ar2m>Gm8QSXCO+Cs(P47=dVxCGg1eiCw z!UIFLt8m1o3w_jBiQ^d&6j{s1*XRDZT}3zPOmC{7o#Lg)c?#LMOnRaXftXISI7m)+ zgq1AA%Gv!~DBIn>g;WU6Y3J}ysPmD~H0XjRK?^5eR&TsLHSJv=VImr9_4#+@oGpoq_{il)l~d@Z5*%RC=i!c)NK5OoRg2SVodX2M-?#%7F$Mr1x;L zyrk(AoNKZ@OSv2~tBzj;yO4iH%5aSLpyE~)zY^d`jO{SY6aervVj3hcIV3jAX{DbpY%Tz84gz%J`6{{fUr^scAwAp=1 zP^skBq;r-1@o43fy703t=vkowGQFYRJ%@%Z(xKvG0=Ad4ftKx9MMX}4`wuht3NrTL<(u1!YK7j1{_X_(}jrsDFp)5{}%xdz8QVz7Q&x^kd z88NQZ$1H?56ec&I#MiDPA~sKA$`csuhc$b~ zF8StllNpFyF^57!uI#a({Z}^*KVmo3{vm9B5Owhk-<-Lx^uj-6B=i>)sUfFS1c6}<(HjcQA%>xG_q8XiPP6b z()2khn=_}?mrN*2nc)rDz%5Ai`*Xcu>j|KuxA7N7967Lvhs^ank~&Zfzsrf}M|K-E z4^1m>h$7$0^5KX|sL8~sW_RPgGZGE&U>ek=@K-Fo0A@B+${R%Y?Eo7S(>vZ5HoQ$+ z>0;DBtw}`bD%rd7A>cyb zvQp4PQD=XP;LeX~$78a@J2&beqg)aTQ%MN?(l%R(?N-92t{@j^NW=E_0B3BE{h(=l zJb;aO0FcvyYw-Tol*0fRi(L#SJ6tCxOz_{EMDF}fc7AYeWa3hdKdOCa5 z@j9Voq`fp^1{DvTb}Si9#M1(dD|UB;!qIkj(qR#Jh9?2nWLB^$PZZt_kW889xY-MU z(3ibragiTpGsrYK!u0xzHtF$)l#8L@RvU_lib!}2qbw`ub|P1IS6+CqxsUfgg3>zO zE2~q&jIaCzDlZ863V)dH3rq;i-A@mVIXCy7V3>B~N0L*soNNE{c`P#IdXJ(}?bz*;@! z6m7?A==a|~sqIt(_AiGwKuWZU*!Nl)3D;*VaWjH_ z=^?1%>iJzXZ%H-bLElV(as>-s)@St$<9VaUoqATFC;tdrSR%oW(Umch8p4NHsdAD< zG>Y4RM@#Gcb$4oip*%a=U{dxTv{ce1b9{QnjH_W4&-ZGep*dr-djsZsM>cAupH}b_ za$5OJa`&5*+5kZ#+^KxJC(Xm*)3uAXy7&EuE+#IO^>4y?4+0U0vYJ{?wgK%qn)}`)=*4LWp@wh@E$IFtpYHo1FI~@lnJ;h1|yVl-29?pj0Ldw{hjFhvKqDX=;@F7zX zdts-eyuh^sJ3ChKr+$m)Sgx0AzTxs^#P^l& z-5&WA>K=_?*$KiblOg_x}9FODf)|C?2LE?H`dQ%D&?`3gtdNl6= zq?O59I~{H3e%@JvahBHJJbY3itz}3bio#bB8n$-S&BM^4mbjb`((KF97f8_?9rD5x z>Fx>8+LcGZzp>8y#XlwY))^0W-48qE-kvVcuoU+`+Zvmcc}s1JFl7@y_3xm7buR8m zKB^{>&FSd27|pC^kY9y~l2u!L-xwS4#`0&~U3`7JGWRGsqJkUk)N<00+oEDQ0=rB$ z)?(7l+lB6-D9x?}j;8ekvs1GlhBE~?5kl;>oXOGi8H)S}z zwp>>MMWZVB!{la@lxKVDmFhNfOmNqG#Rs*SW_vyO3A*-w!Eh$Ex>k>2)-)Q;AB%b8 z_1)_QH{f{@TBQ)Afo!M05jw(&iDx3yFT|AoVuTXSr%1SFP`w|6OCk%rMw(vz9$_(0 zy(e1gL}LbGePdtCLS5z6TWo1L1fOp=0RLHzLVE*Z1-V_6PDsY14g4G5`D901Zh6bV z#@OWcE|3RF)oMR45>s3@Ku(15Ooc7%g2T-i#sX~jXv~y1FkKvei1w64q;6uH6z&L@ z@G5NQsVS0c$hDL%I%Dt^Tj4g6(8(xpzxDIQ^|aO0ldMl@4OLp4of(~ z4H^q1>e*C_H<4d-OyGao*|q+C}+3ROm*?3Rn z0@p5F!8O|Ry-U`T#_m!Q5tcWGcg=G{ZQe(Ql$oI(yP)M&2Z&m(y9nNnc5k!^_EN8v^6dhNxcl`N^t3)g#<&2RkUfC>@~3cnR;nsNk;NE zWP47nkbEnP-N4X0HOv9JfEjK!;yPDO&h?)KPaO{_H)j3yI{9KSVS$TBsal=EE0hnE zs&YYMICT8=(a3HY$q$9IkWV&p=`2}{W}gHw+lo!(f(#OP)ILJkqj}vvB?!Zr5_lx? z3Z*WKvj%pCg56!i^W39$Tbgu`;H2dWZNb4^BgFiNG|u^O&LO9&6X58ifjK|gZwQcM z)o74_29PtA!$)j22OlzjQmyi@YV}30qYx10_eFo;2`E9dswVs`@Gg2j8DEDcSKo{l ztNSWTT|WR4P0_6Mq&Y0!c}lk8+`V;mRm_0_9B&%Pk>&Nn?cuN8gA9wM+^$Pob@(5b zk$8#nMz%7=N2+Uzdy58biup;A=llUVskH29YI@Yr$NNhcaTHL{JxC~)6bXWSFOD!v3X(b=I>GoVQABeLhiy&N zp2*am>JaM%$9^n7!uabfB=F@q56OdO7p=Awk=(VVa465wyw!5Zr5)l(_O@ zLL1LcTN3kU+F9%O{sBY|GBZId9WAh>^>f5k zO9{c5tqac#$^poFcFeTA!y1U)9&24wiY)w!G)np{W_T~42$e1>u9PcfyVv*0e9$u4 z-P-n<#sCyXN*h4}rhNoAPThxa|_@na0qGFVR|?G?r=1= zD{m}Qr7btdCt2Ji??EE=p-}49PBY}qoGqWWZAEwUsR?Eds~O3hSBV#~d-g=X#pk6( z(NmuY0LAsDedX-^+fAw@-CItWBs8I=_A%Epf1t6(JEbG`u)#P;^Wg7-z(1X7WsL=0 z9eUC3M#HOoXrFfTZ^;psB@c3N(9Zg07H;igvOVl90_(b~In?2*9HfLsKeYdnfvZKD zR$-y>OS;%H5gn-gD_SOJM@GB9%_E zFbk3>F-8>|4{;n`RIuy_+fY%c)T5OVEg(z~(KoPV=af7e+Se<#EQ@ac%7paBwPQJ4 z-_-Urle(B%6a=KEE@N9^!Kvpns0gY(N9^0$D z!axJ^Sd!M9Eq%91D^yPxVMZ&E#g=YNn_I^mR@ig3if1sI>dNn(MZ;W21Te|%=u4v3 zR27h!KR+|dtC{&=2)V(Q>YLWs=-wcV6!?qFJRp}8pMpX~gqU^JEuhvU*aE?vdHDk} zC)wT|L<6LBpi_IFjcJkKBDA77z3g3y!3`%A0@kYb%o{${Hq()uEoaKsqghA`zK{15 zuF=!jT+QM@@4$M!xA#szyC`b$)$Lx)?m^ZY>PRePa9D>jrGWr728Gdc#Giey-AKIx zheQ3b*hi7_r0HM>pV#EBo2sV-jw`WL1GrwiA)D)sc?zis9dt8f)h6f2PPi9`z5?kN zi}q3}B2F;W_`SuJcY2N;sK3A9ae_vG<^!LtUttPxF3kTzU)OEWTr)j>{jYJsoAFsI zJ<|mqHWKP|7&97;CK_*e0UE2WgUXstz~(YrS+lX{fcy4%P$?iM$1>x9tFEVvOVa+# ztY=@9Zi#J66>qX{)?O@&g@A?d2Q5(F(ZZ*{}~4ZZxsnVMw5VZ*3v3HK(Y=reuUVnV_B(z5N$BSbgFcK z4k?#_JVn8@cB;MBUFkMU2*KnTD{36esx|=ImTr~Ssb2{l18DTylri=;VR~cB* zPG>K3ifYiPW05T7@<$+ql3j*p$7O~%inZik6=du$7X3;*A}6z8U#Igr2Vq66;wDbQ zHY!IQ`#~{@sVGF(`j#|V{o|!TR?5Y-`;hZXO6-t@55TIvyaJ!RL+Dl3!U# z5d9K|dOp}-FfY}6kD~P01h#e6c-_Ws6|>pnWKg)y@PzK`(Xpo{}t( z-pC}kIxn5OPF?Grj6}XWc-sh!Smg&`0%y3w)@>Uz7Pr(+grZKD5sMt{#^u#%cbfwO z_#koSaI3p3-u%kRaks5nda23q;4 zdR|KNRlTVJ{i(~aNP5PhmqLGYN%hW-tL>&DU`-{9U;95aon=&%ZMe1vkWNwQkQb0n zX@;Q`6(pp)8)=XlU}z--0qK+&knV2jlx`Thdl;JixW8}zWi41Nm?!S*I?v-6i@iBT zlq>J1nCbq_#ob&5a7l)f0Ad?Y)YA0j2ogL#9Zzg-W0*MaXL|ZyEczPc&sj+}cz}>f zI7$xeL(L`NORAV6@GX>>Ua2dTP{CWB4!|I}9k-wo7Q29SlDx*iz?yGa!qHv zWxwKLr=@Pw_jWVU$a*z?UZ=h$UP6^fDfFXT%XIDm*Q)Zu?UL7&%tR%3ud$6qb7^tA z5awqcgc|$g`1G^=*6wo(3;O?EDS0OjO$QY4Xt=)s{r=6Q6M~Efv4Tj5(8%qAy2`p= zO0nlwwzFJ(Td4J<_mau;3bR1MDr5)+Oapkw&#^ZJ3rqvdD-BAN$XidYWDt*>W|i?j z;*2SuCDwTwRves4hS5*<72!<+-{Nn znc_?>=Sxv1c%?>(0iK^ZyU^ZO48em@=`QThM;meIC0`0A|MKOS{LpV<&XvlazZWP; z^>WkV6DY1-I1f?^|>L_6`p%n1j!7?STkK1X5Wx6R?Z9k6s#F+2;2tYo~%x7 z0xJI@(q1WK2p943yNNAX9oUWM;u6w|3E2G_{zm?Hy@ltLy?PV_l<^VB?2in3>`i`K z{nUM&$ywK<@G&2F6!Y=)nslC^?Y9qA1X(v7)(#H5B%T7)T!)8YmgR29cjvKY$vp&P z{twqI?oE%1y4GjbznJ&OI}Yay8_DqtxAuGqcy%stEV8QjRPdr>ms@`Qc0w6*m(Z|jR7jU&Heq@)*m$a|&!nW6x*xLMT7LXB zMaGpFwSV1Xm)U_1gct_iRd$yoG3|>9FuMbpTfQ6@ zi`BTT`+uL&n?f{-qD1xZPnmo z-srp*$Jm;mtz9q*v*=^V$+zwPamb#2+q9UD)=8wLwgC-U@hBPudsOnosU-h?1N7Y05mn=+>0Ckd5`e4^5uHW}j3Qeo;@N|Lq!hy`7bSvJ0S zAJ)vA-SL+q)Dkcw15XVvk%a+_)@}LJ7F{7vZ@P#F&wK_Hf<8dj?LL-qo#3Keo5x2a|Ne2&q?wIS11+au|_z?0>P%_=|mM=?Hgf);g_4$%(r9DGiG)aBQ z8x@Z&^6GbSHHySS8#lW4>-IJiTy;5`dc`j=VP`P(<44KIyVrmhLRr1e!GH?2pz2ln zdP5m+7Z>9H<$}Pwy!Z5CZ~cV@@Y+l*hMZKZayqG(F-9Zt9?O!}N%kccjV0Qf);C63m z0nr!r=W^lKzKJ|Hl9C_VC2aCLST0#RD9pS9XF;WCNzd2)Aui;*ujMSWXHo!b16&cCZGm`qSgW3G*17AN#pkcUBt8~Cc_*R%A2txxu&fs7o6hs)`QDlIr;Jp>g!?#>qhkQ10tWy^t%1M) zgOjvf-ez-N2L!OC&0|Nz znZ9qZ1$j{VqFLuQ7dwY}*;i1p%_HWv`%AqKnc^Ozi4JX-X9t?a$)O7OGRF{aJ+cwv zjCb|ZhCcST?yFDT-fze4pmg1*?li{Y;E;5=u3J5;LPrlZD)O90hGW9ys^ ze7oH3K~;2$BQC?M$6G*CtE#}Ik?jXP8JR5Jbnhy1|su&7F*n)n`Kc+?Jsd7inM#0*GCFC0hNP8SM2x^b8zL% zSfK>%-$b;~_dfF_hfm+wImjMTb8XO0NObw-81^2^W)ehh{|fv((t6P*;AGqInMwQK z>T7pQmUwu+pv8^W+vmdEw$hf%F*TcbQ_PhZy{|=}_MO&N4D7{$br@CgzirZ}8h^ar z-^yc)+s?_e7HOY+-QE$TZF}kr*ypQGmksyXHEhVN`8RP3Im1(rmO@i$OO|b8c2D<*g06GQ#G&R__;YIrEaHNb@xA_$pbUkP+A~-l%xf;qy)DU zreujK>%On(a%08}T-KcGEhtmQ72P_pw*G!tFPY8TrpLW<9#8=|7ft`mgp>b*6w3HDDqS0j3B4{{j1u`GBd6lg~;;ks>)QdV@eR!6k!(@`p} zEa^UM242gB%U`a$j-s2DwdiZ8htv4P|NE3B0IvVCSTYn85w%Gx{$=Q8_GX}x)H19Q zYGR-6?@Ziyo^Yn}wXzg2bq1_rzCs#yN^d~XgWoKTkvGG+TCdjJF7oOoXvt~gmTIqs z6br2$J$_;1<)Z%5w>`?q(jRG{3(KzS!4ckW`@_E0GX@@9Y=PIiezxzt_i+zWq1%NkN}I(i}S?=e*wh-oMK zRoZ*SnvO%x@r9b-t7zl+xKTy>%KXOA0tD+;>&A;v9kk&Oq$x7{IK5~LMv?K*>P+L=YG*zT zj@sH5SR~fKSF9PJ7{{ zD@9Y@US)hczJ$O2F2ak&BY`>mdpB&{F#)Lutgd4nG{s+fWtTJ8iv3FASPNWWL9ZaYWz~DM!)5 z!DFI_uYPw|%wm_bW;WNGgDV`yU;P0eb=7TIV+p;WjXRFP67s$G^(=JQY3zL}WlrPwe7&?eqk(NwEpUR5K%DB7I(SQx^f z_7zJ+Zm8Feo69CR6Ei@_C6*tX){AD!zM8f~t&A+{<^ca@=jVeLMDY2O_itI>f(-7W zy+V8S4?16?WjSA5bf1rTG!tJR4R(HSK?QOi9E+0)sK2X=>6(HyP98v%I%?{;Y|U;>sWVUU4F zvvT=_2(J%$L*nywd*=t0bM(Ie56T1bpD=IHH7Wsh0&{x3?C=9PRt@l} zmp3Xo;lcy~7ZVq@s1BM<_(!#Q<$yMZC7$Li_J&=ux)-D*q~)C~i>AJYTFkKD1S&E{ zfVFNfD^&dGpkR_+}tiNhylz^R2!ZSJ+m&1Q6bT;S*%} zD9B^Lk@&5Yx0Mv-&JQg>25a4I8Me`*ABkE^J%Xo>iWTS5bjG56p42r_Olk1+ZCvln zIX|ZN-eC{}W=;ygG3ZQK@=l&sn1k^L?{CJB08NH5-xalB-M_*zL713MzetS0tOX9x z8`)Q1^u5Rg?dJtXb3&+vm!TYDhix8e@GWXmbEn(GIxEqztHz5`xXp_11^5DcB0NL#{B4e@N=r2NI;r(9awuO-4|YoiZiA&RZC*laO4Qv<%DWUYdE%X`7f zM}D27uNsI21?64 zFhyVhkf+oq-wCI9GI3*!JIvxP`u4J3ImpOm84afh1H$ZSV{rCpf<7DKYlyAlWZx~x z6!+(`TjO8pW>^70honSJ?Kl&_v|Do7pLu_jw{zG|@4P~5t!|dp6gA=e8Ng08SuvfA z&aMjNMu4DoZa?msGMd!1li#!p+d6SSJt8#bIdno%csNoY{AzGEFXP{OqmgH2^HiV5 zMSAKk!|=lRAyx`#!$Haz(C}fg$Zmp|&zuu5)o|${)312x178S3*sypDY1y#yW2~YD z0s#ysL3rtmB4_mX7q3n|Rn6%K&k?C+^sH}ocyKjqmo13J@7eg7R*nn&r4_j$eMpyMuV@;-w`}_J$#?P==6E!$GTB z*|D;)8yjj%rhwnmP{3hPMA&Kc>dZ{6cFNLA1B8aN-z%D?Piw9FNN%gr#%V;jC#%y+ zB)UO@-rgAk6(kL8-^WxzW4O7JA&u#Oe zSeiG3gp({KY!=wT?e?=|7MDBQh2GDE`YUh=vEg@o#V6SqqNW>Z|4x8n<$!EA-de|L z(pvAoh2(J6&i-QZ_!~y=m*g9Y?;D!9GN_Ckb(CB+mx{$NbiS`llN)Ob`VLOW@LVlz z9$GNVSE&j`R;~!oTQgGenZCJv?Y7m~8#b|~5WhL=g2Fv$1B88juxDpd_6?f}UMs8D zuIIF`CI{Agg) zE@@u^88}!tQMhgH@?#$fIGl%;6`ztum)X(#q-rh}P#RPp%K`H|m${efSwoUNuCPaz z;kz)h8!z;!bfoWPYZH(*vzj{1h|KJjba%gmL=J|uG6bAJ`c@Q{YLD6ywv!eh`|bx` z?E8ek17aAzkN@Y(**c-kDyB>Dy0Z=EgLoE%$@P(@=Z{yoDKGZsZwyGbA%3om=p_L_b-fO*FUBeq3475a;bIPs{)y$M zEv&j9U;b?YN1C^UxbQrV9|cy1$y0$$--r0fm{o+z#k)*xO>i}JU{w!7ZZ}Mnw*)~U zM{Ch~`j~Dh5vg{6)fobw;A#`-vZ;ieKkJss)4+mM%*0^@%&N(rvsVU^ey#lXG76eP za|hA}Hq^KoMi2M)zuUA8UPC8(91d9q4O0~~QSsiCAf4t(RaYGwDHcqK$>;1evws#I zIy@YBm7Z}8hAQW4P47Dsw4ZY^@y;;C07oyjB2=87khy*}h;y8~}*6d4g*Xz5nwwnAiTS5s0DE z*T#H+9gs8eXD|`$+3=UyA730>)v&l@Fg{S-ed-0MU^WN~&3r4lhmYxX`a;L#;9xrF zNe9@U+&_T#8&2YmA^%L`twEpOAGlu9Aj>T>FG6WIUIQb#h~EE>FYzg4qM4E9<73?A zvWGv_P0wpDE$&0fByQ^_jmxHBNqys)FTJ1O5so7guK_D>_ebI@v8^Pt9e`stJnwg3 z;!!yGuqoQ5VwtC*1(*2Z_(Pgx2#^FW%VkD$5x=b)xuG7n%7F~;$K4|P6EE|9qeb}6 zNg@NVH>v{@pScB;-5Noo=j#@0-)vxD6Agh88&id9ER6tFlw*i`P%SXbkho3Ss+t2u zaxTH>Ib7kva%rIDKf344b?E1r_?h;0dIuQ#C;_5cUpmY3sgCrWz0{*22D+Obgos72 zVEJlbMUw-oQO-%O>1~`AMW-3|N|RycVy%$Yzl*K4yXf6fL;cD&L+Y9S7NmYueP#Y* zz?)*NEBeN#^EnN^`_g3bbu>gj#7NJ<35Rye4cgeP9?>y$-4Gs-x<#($Fv`=&8+b(7 zG=y%5)HKUh7GtT^e`bfS-I|tv#1Je9l74zd3QWrPm<3o)aDBs(UHmA`<6QN*s%B2m z`%tUM4rQVoiUIu131&j5*A1J|6TBQ{K}X%UNFdQ8h1)0b2Zj9}c}Muc0g^@lx-E?_iIPlzk4jeZ%uSZ{uxxtPGC8vHJ~r_wxw+X ztAq;u7W;9GUVk{ECcCz%7-Ds^n4X}1#WbVCsm>k!iyfRLz*ruiH1*Pw&j~oFg+6aiH*-cW@=^(Q$wDe8)zBo zI51&}P~LRhR--*6T-M`{5P%Q@$xH&$g{*RO3?+|r8)i~!G|O9l(UvTZaGP5Q7!W=_ga zsh4H3V)W>D44>(^g~1!iEjQzqK6HiNcmGtD$tYSQaw8vc04UucbN-JOO9OT(1Ax8t z8-CE^Bly*Qi?Ct78S&@jm7|CIFn-3?B&jZ}mJdw_)v`x6p{Ffp@{~08vNwb zTmQ;@x^mi`ni+p<O_P8Os$Y}@B%m=BsvUk|lx1qx-K{yv(cnT_-G8Fa%Y))M% z@Ls&a`pgsW-{CZhl?H_EIuh<1NeIOkLCmery*DPivIBRyBQ3%5ojKo>IX8LgG_TBY zqxx>F1HS1`=iFk##>u{%`u-zr{3Xu~8Ibu9OrT4XKNRbMgS5U*R_)AaIANQ{)*-vc z3E8Qln)>vP9g5ESEr2Hjo1<9D3%{VUlitpkHB8b}c%c})G4%8u%_q`r!@c-mN2-?} zhoeHi2t@xH0Cdn7>LYDif04V9f89oNP~%4Hk6BTgORyO8v?xVYrR-*N-tPdUQSEwn z3uswd^>8RYlJsCBpiZ>VFW8N{mDH8ZW)*{t55-xro7H|LXA|R?uoTG*Q4rOY-bZv+ z8gus2y+oK#_#%3|XX{v#!%|~mcCi{L-f`Gg&X_GPzH**zC#LsJA4$K-L9xn**YVdq zn+j5+huKEiBp(U}4-+l*tGs_WOO}MT#6V5&|H`}t62mtTDc=#!5K@pN4UxjYGeozh zu1#JnCRQ||r$fCeNfLgOV|VSNU&N2)FjhR1d8airb(m2b^L{8_n+7q3zvs~!!6b`1!T&Wq-0bfj zeu%Kj5LRCK!Ylz*OlBb`c;LNjA_v1na?5U)0Appkm-h8E4#bu4wA7)$CJdDU#NkED z@T7=+7Be{r3vX&J#8+lKy< zwBxP5ykO>;g~R!U0-qVPDkD95Csqm64#tZVD6o>lbwxA*e=1?`>&?XTvPB?iAF$Hn zE2W2@O5VIDl1Vi>@k6{!dA&D^K?nXsW%yDel5XFFhrwUfI_f{@^0cB*PT9M}exfWK zJGB8UlUHbyPBzPui~hJ!|C?Qwn^FIVI3FxiCA}Kk&7aZ1$__Z{xPsLZw5YX87*%`~ zgwgA`L!!c|;A)waPI*9mdV2v?GD<2+ck#dT`S0E}+c`yyDT!1wg(1D0s61@Bd$Wv* zRZHI62y`hM(%%8c3ZTiWQ;YJxHMCz|=O^00tbViPL?L17CIzohh5jgcf<}>FfT6P= z5$>g?19mr0-!bC^XE`ojaZVB|D8UEXS*mStM=412&u}s<-33NnEQn~s6^p-dlWDR^uKHA0 z)tx+(-E=Nv%=)@?d^$T?CNK- z!|3EaA!}=XO3+OqvGoo9E(o`0K|93*u#U25dtGkoyX@2;x%&;0sERsT${9TVF%zbm z)03(-hGtXHdj(V*pWR>X^Hd}K2V4cSP5X9xs2w?RMKObJ0CjF+ow(g}@z4SuofuR? z+>_w2K|>b6j7H;}U@*+7SeCd`joYRkzUif@ODUcVyUaBxV0SzVy#o?}%O5KKxIRI@Jt`qpBGcbn8L8_xstr-|DmPH2;?c7CIpf;a=+f(4QGtb75p-n zN01;wzNmNFiLTpv^5>g{F&YoAZoNQJU3g8}6Y&(SADEi&i7Dw^2_TrpxCH;nh7$A} znv*PeWq*h(K)gyFBTM}(v+%jqi^BT)%*DpgnyUOmKoCWh#w9CtKid@G&H{fwN^kN0 z1h=eO2kEWGVX-XLa%FLn41CIPD*jCZJ&R|4VD0`exW*2^Zj3wPq&OdgwtPFrV^pv+ zLssR&PpOgRXlV<$9q4wq`YaDKQc*7;{bF25%`ntY%!e$;X|Notxak(fy!(3kZW9m7 zmOL@WfTHk{y2(@REK_-k-hp6#^Ogv&$M;v}{fw5UlQQWorEQ@O;zhso$5@!E?R1(RggGw|l~+3(jPps1n6g;p1hvWmJx|V{qaOWvyG%cK9z@ zYFVQHC@y&vL>F6PscjWazDOUWTxpc@k?ZWVpDwdosWYsHc0YQWJQLUuqURXdm9+?= z*}lY%EI!|C^0CN1T`baxkSOS1PA}-XB;2_lef2lysnPiO>$ZGKNoeMr{pwFb?ed*p z5tenWH~pUTC8z%Kzok=Q{H#1(wA@HF8^au6x|cs@lKl=ARrrfvE zpBc-&4{tq1zyzwv-0TN325j{|ginF-iOS)8jlvBg)FU3tUh3=iaeX+iay_UD^ctiT zqNFup_;E`grFnKLv5J`~}W==}R6!O@pL!x?gY&-C`zY$ig7&1elji0RC$| zX<{nJTFi9My9yvqvh)#RothoS+5C{N_gB#40vPTjOb2+dH|t$6bgcF_JL*b*p&|1~7r80=MzRKj=BVDx@N1>LaIp3JPn&dAKsm5Q2nh5dZLyhDmT#c7p)Eh% zZ4GTHCgc@YWU3W;_XG!m|VJVkT$a?h!^@T7R_tv zbqn9=_LtN5)}FwyGK;exi$P4?ML6r*cj|n^Yza>X?6U7@PJ$8zH=ebBV@Ot+oFe(o zd@w~bBm3@LW&)gwo_!jqYxh$^i=_XW*@#A2WEIr7TU_N=a;=%STb@TG4ED+L!{OqPwdo9|WS!J3$|y$gXa-(|!q^uek`aHn+pIGcHNUl<@Cb(G&l2MsRfM zDClqniYjqis?f5k1JKhy|4n@kv3=NdE++BVz3w;%fi|trC^^1cWw1*sN>1nz**A;| zeyUCmm05B{&oYh-;OoPSSB+HARU*7dCv+_3GZ+!0&OS-rbaU+k9ZfI|myd)tCY{ca zU$SdLFgGmBGIWI;^egTd8DQk%HV%AAp^@d~9aoWUsJ1IQu1Vts(otcGUkD^gmFU$L zc0+9KqRZe%ER^H}x`FRUohyM>Cs-}O0hlWrzrxcISaH|7aU0v8^U(GO9VaY$^z~za zD))XDS!QLqCc5qb-PUoR$Cs zMzzj9za#Io4;IZ#rhy*`S%!IE4JOGS@-XCi2|i^b$x79`u#1$fTTn+XwvfLcuyy?V z)aNkfo;`p_YVnq_H4E+8z{@OR$cGcEpuT$gFQT3id{5Y)+HNGSk2HV%zUz%<-|8ZI zwYnzZ=>K&@ET0q1mo+{_IiWaZ=!=|6-o9bmHj)!IG(4gW4Vn|F9@+2NmS&`GPr7Y| zq>aq^#NDlHKSz7MCWsuo$lTc}8E9FNp*3m*T4cN3Lh8tLV^Bf`?$?!k~(_)r@;U~dXdF-mlU_fs#&m|!sExx4RkP;FpFv= z@4R9dB?sUBiyWtZR%K6}aPXT>RUkyxFMp_k`zFwS_X8Rq>^p?>iHQ4+`!NlA>xwOP z`BPGtTUwutO2dADNlNbXs{8qZg!yU^IN=^r-TS9GS1rS4di(Y9HWb9PRcKC2zE5_R zg!nC=3UHl1E*vd#Js0;f$Yx0)A3mTaIezvelV>+@I5Qw}9j$BT6E=;M2l%D{;t9L} zKL>Rpodj4fq%A7z78Qm}@nfo?8U z;sFMfq&zIMYn&uu?VizfK3FC%4}{Yf6SWBxk<=^DU*fYOuGhuP#sG&^nqcBsuOw8y zj(#T9&}+}45s+Hp8rHTB&d*G`=(36@+<#3r+pW%L8B1gwlv(W%37{c7qOVQ3w3rp; ziYA>mO={Q(?%oMvSsqlPS`X$VRa##nM5t|mrL}6jEm+?TC2KCDvwI!*E8V4E8nxXK z``w}W0a7}17Rl>-wEL1eQwi^#;dg~^bJ7*^?|=0cvVa0uOIXY zVHaKWN6L_EbE0g|k6m;WGUPNy38RQ#5!`35Shhu_D7hq)r}p{Om0F|K;m*E(i+0WL5L9Qc|?O8j^V4uh8~iWX#59-?UVB4~(s-CG8pt?Sa^MKJVXzme(T68dce zFVlgWz8sutn*HEu^JMDmLtksJG4Xt2&s)U=2d_i(I_1!cxUGbS@yPO%=JxaVh8z!8 z_TI>xQ4Y_DjvzF;_uEEJ42sEjEsccjcG$~OugP`UUl%-{AYQw<-1?r~uF_%_1Pz8L zvp$X{1{KH_9abI_gHPIrUitySFRRBeZqIg5J7*hIXUDMbx!gb)w|GEC%VR?9LW+=m zir3U?tSHq2ovWCE+qJ(<*XF}OGyDj6=ni4zu26SbBi@p+1t=4+BamI~xQ1c|rS&0S zmON{kB|N>+mYqbvqJ}niKM{ZOfd-1#!yoS8>ZbJytsNLGb>j25<@qH#XyWt&^cL@= zA3K#Z8+;5sRBODe{w{D77rxqg$kF5Le0*y?6;+3lak0d)XwpOu7V zv)0qDBK$BW5Q@H#f~bDw%|2cWEHD&Z>XaN z$3yN;WXpDznkA!`TwgQnN$tpJx1*J{=%PETOk-1^{q}CX5EV3V48hx1_O3rN_2`XtmHF=s=k?eTJ+7B` zjXzt4+t)2kFQg4IcVp%%nHtn1kpHMC1x^bz?A%zH0>>QWtvX#@4JZ9|LTbHCs3dI> z9nt(*G~B-avmu1bRU13v+1~m!Fmt}BC|`xzV5y{#N0Ic)15>LN&tBslknUq zD8sTNpshGxIei%v!01icTz2N4|c=v{)3G2MQUhf zYiR0z+vBWxeh)FjYPmh__2R1!IFFb;VENG9tiqRUV-^dv1eJ?CsG{3RrU5r1;Kx%q zm~$&C*-7sRs(OMD!Nf=plqNX3are8864G-J)%`pyxPIPeZ)a%a!Y?5D)PnEZb>)L} zku=Ms76mI=T1zd(2Zn%{C(s@ifP2geC#(yRI70~2=ipAf8h4DiR~me*B`}!I^SQ(} z3?s_JExM+|&0H;c8uUtE6GMtjWv#ZXgJn*6k(y()+<3aK?JpX2JToEIWQXe?#SHrp ze_#?mn60dT85L%jXBW*TIfs9O=t-^ftp0FPAWdH_xf&L5u~x-1p6byiAGGW_T2%QS z6r~n7?fk*ZC8VOE(o_8dabdUQb(pT;lEm>oK>K!GS3;$7=QyzZ%*^UeV|>>owqG2V z*lWV7G)dGoHOCXbX*p|(h}w;v;2w}-HX_LOD?$2TNg|xDKCMWL#&_ZFR;?CtakH@~ zDyv!S>80}5DgKsj0)mA4=nf7HI%cOz!rNzxB^}#HfjSvlO<~yh8Ae4hXAd(y2`*(F z(@+IPtr|dKp*cHnMyL%HY+;}=l$#^XO}%V;{ z=cqDOvd`FS<))9R7c9u;EwMchWHR$T=0PC&GZ0qTU0jxN;?%>h!O=q*Km##yEDq?W z3LRG}^&^cwW=QJ^EyMh-9KZ(9h4sn99ih4HzKEc&uZx7|qFN2CyTQ<3j^u`dDIIFNd)smih zG%-^2@WJP}SrJS8dcJ1K*-b0cZa4in5ofC$BdPf~eRTJ{jAh(ZVKSe*>`1<{#t z5U4{ZIq!b{(4JEWL>QTuZ9s3*mBfO$B~g>%*d10L)$fj*c01z+Wv@!g2lg(;$D9q7 zfu*5O!hiu2CF@hDSW_Z9^fz%y=7a_P( z6==&bj8;7WhNSlc2*(St!7b#}D)j#s?B7_O(ckq*b6xYt_`q``nI)gCw3p+wy#xQO zT*CtW_}S5Y!iN5Nd1oavh2Bj?CF(omW@l)u+8_u^-Q&mG0ocIjEadb%`oBuzhw+pn zG1TnD*j8NeifO%R6EbHB+}NTXX2dI>B=FRmbT9 zuHHIF)Lsb6vn7v}pY5*DQFnCBEPAr5KgiImz^MZ61>()}wOUnhHB<5I$!(3i=@KIv zZ)Zc_Ao^pmS!IIk?c(T)|9$%Rm0U)}Yd@E1E(%*?)*p#a6tY#mOoNrK%?7dcNndx-j*AoWe8o^Y07ySW1!NtJb z?9;RDi;2NljT}0PqAU6SsDxh}e6%%GEk%|ncH1AMI2~|w@Kj{R)s~rqjLyDpaU^!U zp#FD((hw!sE$lr3#MzoB!i4&@BUi75?6qLA7fY^*623$NdfZrB{22tpWN6kZxkx>T zx%2a(dwa*lR1$ycAh7RFqg`|ZIN3Kw{`Z0yk_Unvs0P;?0B>b=4W!&+Q2m0fjhiN% zVToi!z6Y+>+XR*f+uAk9i*;c$eyQM~{~U73OJ^`KDOg;dlr(ImOafys8qE%;yP3?Y zotaa{ot-7>#OCwH#;3Y!K)jMX>haXW+hZU$d0VkWN7AAp)w~ZJoFx8G98q{K?K93_ zRHEPrZNg!QTo`f;ogjYn5k1eqg7&XOWO#)N0l0^uef;BWqsImsK|?FSdFX=Wr126m zNOaigQQQnZzE2;OC)DJimz~BlLvWO=W>&GUNHK^Cg{*r2@)w209$Q`!g?($g-lD$o z%Wk|_`ciPQ-^2QmT*h6sxtXx6{$ACE5+t)qMn3oH+H2m zkA-#S)n#Ld?<{F@YW{cMN*fOSepem1ycoEhrXjL)CK75^Tc@SG90bm%m?*LbnLSf> zz_pPgOk8K?D`&f$%MGn+9@Zvu%a!`Cx^TGdAW`TQcPhZ2q_e&Kwkf(b^0Z967T(Bk zrNY8(^qs>V5>5h53ASDJxpp~cP|AL<0I$6ma552np!ViVgOIDWg9} zz0(XH;h!U@#gedkpT7xC(eX9?g2oH;KcOOsGg!&yV>8kAPhT{dK{mQ z4o3qK!NQXlI`*H)eNQ66ZeGX-GnH=9Ut2;@(`b&qmkA0uW^sr`KMG;y*EA;~e+Kxk z`3Kg*(Rd@apYFX!!z2COJhAQGc$3Ca>9&vrLEzLc+CG|)rUkT{2~`H0U%O7swyqxZ{iOePm--u~^fS>+V{x|= zo{|*3hps}#L)oW7Rd34DULt456&dTe{F@45lkKnHRlJq`Gi7L57vd%+@rsyBU7a1; z30=)#SQ*OQ4Z2m!!78#`RyeYd&=^J8xbAhrXFvE(_VZ5>4C+Vo_d@1B2#L&x;d2_v@gFS-a_x8u?m)lOQ9eE39A0FBA=JBD3Jjb@>N zssNKaHGR{`x0aFW=HLW<9~Wrh3ERfpbM$3CiHqGDajT!e`NntYe(Esd1>|(O$ZLE6 zMR=h%sE}Nl+2o@GFs4UOXQ9sl$ZwOS;L0;h6*h(`OX^E{I=;YpcTNw{pG(_mZ+*Gj z(v-lDcX*8FTLP7#4`#qd-R6UK?q*zNC(C6Pe>)2-kN%tWwS8@WFR?6xDbF@bto9K_ z1V$!1j{T*lpWWEYAT8cs#VuPY0vDxC)^l6K_w$V4pZOn?V*@gEYiuo?9qb;jt#ndP z%qzWt>}21jF`+~%;S^}n39k?Nbwa&uWL!b*oU1mAa7H&mXx3dzY!!yV&V-Zb{*g{7 z($MqH`NsOfno7v|KSFxZSO7GkJe(>F2L_$B7l?jW7#@L?Yxb1ZXsV0sNyCQ@i%i=! zVpGM^Zqz>#_mBbZ3I5E^%3$Z1?Fp`hh;p=Dxf1Sac+!msU%0~74%*~5X?)yFQx$2d zRW3S3vZVO6>=5UM^3Eu)2OyqnY1oYuarU4kYkl?S$K>bh5}Vz;#C$A5Ds>S~vk)e> zlMk1!d6e#j-8bUfSx-bG%?sW{XvJ}2NbL^L!9|`vYyfM#QCY`&s%Tk082gESB0SGa zAAaa5gOwHY3<3PJFS#j%nY|SO0OB=n5Y-~eS3d5Q#tC0si13T&G6?S+oPjR z>nMq(h^{xTMwg4f@H1^~Y5V;Cbh^Sl|E2QGP57Q%M@};^KBErM9?_*E7*=^D$b_X; zAFbQu;FHoJ(ybC!+t5rF$|i86?O%UH5^QpvMNd^=BB~Qv&IaH6R3-#>NKHXo7QRtmC!a3q7w#;fzOVcB6bA z-}UEiLGgK+6^hv)l`SsUqZg6rtb}7u!k7hOo4<>8fTiOF5y(HOZqAZg*Z=m=CXFCK zxJf|Y3{jhIq*-o)_O?L1lM|b)qlK)r{ryp5x-9aXo`dc)e=B%LPG|&d<5whI zw?29UvYOqhQOKZSQVj1?zG(8>hg<%J{Ee@sqQeO<7E_$5=3E%AMfEG{^nC1f?GPb@ z`n!x{r-?>>+3)CmU2YSu(H}w9Rmt4Wy!|R!rL)V z-u?BFA;3TS$yLbyQnKu%lVyxN>$~sxS}L{O0bILVgFaTgnAt31V%TpCJrS7mwsrQE zVrW&x4Qa-ly)V!eNCU0u8^ZoH|3qHXj8*1T% zx4eJU4dmCxWJE^_^_fA9m(%+;>9Rrud;BiZF7=QRjEUUc1OYeW`l*rQ?FB|INl>bYVWh z@ibB+9xz(alE(N13i8;wUogIqvhh_E`OnLq&4bWe?xm?Ft)mZzrLSo&JAKf=eLoZ;a%SIS_zo2m?`CpX4bE42 zzRMrzM>2J0TYDz0yU2df1I%_vNyN6!#rkS*6-_NY^h7#@tLbvw-8#aOtKiy!Yqx!= z79J(MTrZR4-1?!DoY4AzXgbTdD7*G+58Wv(A)?YHT|+4dNGqKpEu8}l9nvTz&CuOQ zw{#4RG)N51&<*c(Kfm|?oo~#(_PN)&*0J<9!@ct9t(#*{XTcwkl_B8ZFnLu9uH6ZOV>f(tVGuZZU;6vGKn@} zWc8_ zQsnZJT_VN>?TU7r_b0<^quGymlt|6w)}EP=$V|SlIAVClJbCJOFMTnqAFJgs`3mg= zpb3-X=%7mlOpyrVLcu~a)9kQh$QfktmG^t%VUJvm84tcsZ(yTD3;*=t%ZY_F!=?|{{l)5Q z_$FW!u6%U~#TFl`cBJ;%M)MQi+=!Phru@H#vpMP8k`Q%2H~H?-6WKxwj0Z_ z+?N_XOZrkUjFMh)I4DrK`21upUB*yFT%c$eTXO{33(I((c@)31EbIM$UVt8&rC^V8 zWx@HpK%g&&JBvZKu#SEB#C7Jg_Jpow#(PA%i^?S*R~h2-Z*~G+x+%@!^qNd7M*Mtm zMhAi#z=U$0<}9JIqLlTf`mQFeH?^4_St?}?_kt3pQ5RC^I-;jdN5-e zwI9%A0J4f6@`KI7^CEz#j^r(k;zkLD%7+*%z~vF^V?^iUS0tI#R6@v9mhKQ)z4W$_ z6%xp?5jvs~HN#;`8Qgfg?SAA>@|jhWaR)4PnENBLGay=y{^h^+ zsu5T&Z*EOOfa;S$2HtLVi?P+pj|vs5xR{=61&BhW%jT^Y0P?D_*OB#A#9piL+l=l< z;mxzVzKmTspVA<(%J{sW(z-E@RazML{&<;J_o&+IvZ|~Mi2|*b@|%tv#-h-eOE1;0 zhQ!y*Nh#nK{kW^sayUrYn0@9QbKS=|d^elpwz|>0d_xq?xs_spQ9?`ln{7S5K%Z1f zH&f||_S%;ll<;2Br74P1O4o%8R*Cwl>w5=41lwln3?~Zqst(l}R?U=?d0OYkc>K%2 zXYmn3Vf_^szTk7aI`#LgHQFTYDFl(A*0qa3th{4X^rkv_A^?rxY$j(krkU07Qwp39 z7&#G+=pfwCIOD$PTuF=39BJ9G$=b?xMR@CNi48~3FaaqF_dmtIOwwlh^jGxD6Q{;52p} z-iS^9zWbjS{3q2*#1sZ1U3QbC9XeQa21-V9bgh#hHaZH?Uiudd=(|LZu^>@RYj z2Oj^SVB{D7RcTG%U9JGONjJq;{!qI0nQNQY<$`HHshf6Mz9cjqA|7p zpI4Z6xwOVKKF#~EmfaF2F6B%p?YmCz5nxWt3fTszGYVQ^KxZN&mU^LBmM~*P&#Ok{ zN11Dhb`IE+u;1rzzIoCK`|a(<4x(FeZ;MEqmrfdNX4sT2qwvDBbX7>fx~D0+nrMSm4ipZ1YE$1v!9c@V5!igqxUSeS zJG|n6w7&x^1QUSvmmb2sy#$yseST`KxL)Az*mMr;u`)|MakvBv+$zE?`Hl;_(byn( zEcQUzmN{vWBFKC+zyvT}`xD@|E+{yTDxn3|*1=I@s8RC+B&UTc`Y{+NR9Q=t{VmEi z!NDNYk?$5iK9`WpqOxt>Gvy@^Ge!6V&P2Xm-?_CBcj$ZGc zxl0?S3Pl?*={WeS&Juq;x^+b?yuGqAo?@lU=JCHARG>*vZ$NK0_PsGrV!)<6`23xJ zbC&#Pt=oewrroN=*haqxuZ~$_A#HrY9jn*qh2Gk;g?fL~*^YxJR&qDhK;Ge0mo4_W z-wvxPIA$eVZ{qFr5sGMjywDf`HsN)+uR7GujKJCF!t~$z#eZ!TklQN1_5@TV65PYU zhddY^d<1Eyv9|`dl9yv>LnSYFH}bPT@Q}Qd;(0;(&af9f;`_@8+z1mn43}ejue;>f$u z*YNW!Wj(Ew?(5IdTWlZ}+p*W@6F3O>F%iVL3CSfvUp!D{_)?6~rYSBS^uQXN?b1Cz z7>5hHH4zV#fA(Cq#<{zc1&w!WMZz&eE^WbW!Mbn)gVKPf@|V6b3wrtp(`F~>rubxM z%q;gKeWr=>(_`8#1C)Q+=jq`+(d_{x(Wjfuc-6Pa?zwgE!}oIlWCdjpG-k@R1O}MZ z@v)Kxl^r()Mc%^dqbn*vUd)#w36pYd63GY8OA93`L_yp}2S`3%GtHFi&xsuma8?72 z4qx3Eo$gz{#YK8DqkKhu+eCT#Rw``aOS=!{Fb*R&8lxv}gNPLTR=&VZ<;=MW-5H02 znzRZS#ic+cwO3P;zyHR`#kK@3%oz5wztjEQ#YV7m12@u$-U4CxJsmsTew{6>>zcGH z{=AVyE>j2NEmfy8r)INe?EM>wd1VF3X?%dm8>^@;{gVuS5i)6$J6oreWmtqr5;H$0 zn6DjZ;)o0G4|=MP3(`pWB|E$A+WJqRQKb{D=Bc1c4>Gau1`yOp) z{A2nZSI)x4-nbzmhAVi@rRPNNmPKOW>1=O#8nzr-lG%Rwu;|+Vm-oECutP03l>p#D z8d;>|6juiq{z*48tg?J@9=n=co|&lBXaAi3^`+?mK4qAK(WLp$Sii?tFM1qo_tnn) zC%XroRLe2BaK3%P)Exb_qd2etK8mD9HP7NlgDU=IXaDHD6}~F4k?Iihg?W^-2B3Ta z!)f=>+f-Biv3QCh?s)Ye$eA8B^F?`|&AckrD3o{r%TP9310edj#hkHx7WCVWh~3PYc!dgUyt3-D*VDT(UCwHq zIdh~|R*(iy@df(chpmf$Q`$Yh~6F3OY97WKZ|UQ1*nOW z9NaQR?YNqh0jkLHV)fim$Zr%A&@3ow_Iiu5q~Uh&{N}jB5~qsyGX82F$X?xD>{Esu z2k4)&`Q5LNPSia;-chP(_e|~g^fMHO^0*7wCWT!0-~b3aoHZQS^@4zdiXX83UG{u> zaD4|EpvFfBI(7q{mH`v2YbOfUS4pOWf6@TT`?typiO4kor}6hH*R*GG>x>ms^;1pF z+fEAv13R0tz^lK-g_(-TTRrc?diJ4}!3pvV+So87!t9Z~bjp#}aY|Ajq-cs|R^n-V zoLxXGHwSHXcS&tQ&R{@Av?7)%B%MetpUqn|YGF_H-s{5=nrQ1e&gr-J%PUezw-Vos zlf|ZQHzD@W5bBG@tW--Xp?@-oiB=&a z>>fi2thyw-^W?e8vFlU00?tE^5vag4Gx$hLtAlZ)~FnNrGVAVF>G(2WJ3IFOUX zl0ZMH@HzRewEL5fyI_9xUJVwTu8nJ~)2-QT$_t5c2IWgyR1tmr#lyHrw3+EIR&OBg zbqEQ6ZHRh0p0^#oqtrTjIwj8gRV1?O15NE0px@(7^O>!{-+$my;sNNpreEsDG`sv8 zsoymDreU#@IwjQZP)r&2zf;}lHg?01Dk{@*#*9WIpgth;zJW$bV54G2hx0BxOEy#u zAexQ_^&$>Q1C3w;s1LFdG*uBm2~Jbt0ST*=Hcf)8Ev|aFFzF-;bGkC{ys((fpxUgh z@QakzACrzSmB84p<|zH2X~zZ07SEZ#9oW;WCvNjyRq)$ z`1Cw%g#D@nfIj$H@H<;85 z8Pe?rFkzmc%s7rV$CqLW$#`b$Shm8_qY!HPraSw+iB<@P$;Q}BRqa?#PuyLxfgtv4 zG_-_trC^+qTlJ$0_o)O{{~umV0|=fvxTVU%gqzpbZl zTyQurNykA6ugex!unuHBmYlucM31((=TSc1`FwiI?E@C!t9wE|a_qTp#<=#4F++xX zf^NRY$L3(ktrjZM9MX%Qv+OOEn5o;p3AXG(!ECQ8oRYLLXB_g$ROK1pfW*=U8WQfv z3D|ICBdo;>B6Wa8J7h2zxkbUo|gC}J@ zWiD?MA#rOGXbx7Z<{_%H&Q?8GpW}!~jrbcu1OKr&axI`|7$_uW%9~ZjE=Pnx$F~}US4DnF`Wg)XM z@Bb|2K5PJ4ZhE>KHRPk)oN=<2^<416ZwIBt$~jK4r65sTfDy8usgfZ!f%nXK0POdz zPqWUVhr(!C?s|9(Ambgki*Pn)63<@eov94^8;ztXd&xJMi!=vpLttD7y;UG@v8M6x zu#&F8UbR;@akMx*=?Q+NdAP-IK?&qw-St?wCp{|dx2gu34SqGoElRwe9yuv41~h&a zrZ)R8CJJ~DOn1yZJi1YiOGFhtsFz=u`!w;rm)<;oX=Wl%Q_Fbx@{PQ)PdFErDeQ z#~W@mXE>?-E$=AFFX49eef(^!#G@S%d|-CNpz#E zwx;zHc@*uz;8r)K1#x#tsqw1-Z) z8?`ZriaMTVwH2Dm+C3lKMAt)~@5EryBUj=>XlZ!-wI2UP)*LgLKH!oIn=;>=(Ms)3 z{Y)MJF6ZnV^jH3;8O?}RC76$3QT1S|^6qf#@Fj0&T5b$3;tU7!u~9?EGRZep zpX6WIcC|{I==t<@;adm#_A&9ZkAd1)#sflivRU*-Xc9v&T)D+l9Wz|FBzJ$Bap!sX zW~%H*I0j;?{s-eZi2KgX@JeR;rK|mpO|9=DDtaSeSjSn{T~<01-8cm}YFM9%11VO_ zkiC6P+b#DVb7TLM9;<4l5iJghuTG|*S&6Q!A+x`axL7RVb{e1I|ZEKIy z-Ie@HSktZm~ak3h_0_298Q&zuS;YD-Cz6*ZX2J2km$E z>mEB2T$nsa+Wr=t)&tW&op;pRk{5bz^>3!lSM7>b+7|e2#>Fckwi5ouXqfofXhh;w z$jsl{UYxrd4MUq(vrE=>_HDD|C9?1Q%bV@rJ9M15Pk89+Y^Z{ZL8w{xx70T6UuxIV z6|d3P5PRQ+I!fK*B27gIL`?^1y@;L=b3fSw$Q;Sye;Nd(%4fiLz6g}Th$!HcloWqJ z?@D;b0AxQV1`r(Rh=@4m{IFL)HD5zP26{zJNh}oH1=-z>QB_~$zRsIRkp~f z8P!pp)R&r_I0h8Q+@p;dsD=U#&x&1&SA?{xBAjf2ZIf&ovu+lq1Yb%dPXr_MON!pw zd2sS-PPve7W!2}Ei}7dYcMHpbH`|+;B@nEfGK^0jvP-*|BgG+b^M`3y<1y)vm9M8| z>;*H4u0(Pnam`y>*C!eKtgqJN%}1Lq`ZFn=p?WSEx_2h?S9KSER`t(Sl1}K}I=V5> z?9XQSHI2!-lnk-~@Mmc7`51IXXO?03j|TU7k$z-;jfD(}sey=&FB}CBlV%Lw3_2d+ zNF;GC39aT`%o%r#y996D>@*>Y-gJ*7qCKOkC4R)Gj3TLQyVP?}CBuy3s587lxPZF; zypDIStnH&YpB(s6_ho-qWxx@=tQs~5xg zuhRM0HbZ*+U;dQe0g@NoubPbVO=Z1yek$$U>D3o4W_nu`HnSTrUT@yq1FSkuDsy7J zHuNtb+&&)z-s|AE`qRseFPJxb=fBmC zwmQc%_P*s#6s3#kok&qy z?N=ybyZL0p2>uQv&LJ9wk@jONx*7yna_(TI#s|k}BmX z)BU8~9V2=W`O^Vc(w+Mm zj_}KTU;@y9bF1D=bSo_f8S`yaufEWW9LM7^xPU&t5g-uX8fSgKpBz<_h>@-F$c2r< zvbkg0qx|9ae+Snp)V?aan3nZJI7+ja+MDN!xGEB{AX# zyq0A#j}|!0xs&OCRzC=Br4-POIbPA)NmlRFEbV(9-ngm`#S>L;f@}so;lINO@j99B z*Q1bZx;h!B`#Dppg_vlu+lDe$mbZzZs2}_rR)C% zIZr(~TMlYM-g?_N&@JD5o!>QY2c1>`k3onX^pbjp<*K_4VzZ}26b#rP(3X14PC zM;1>DM*bhI#70XF?pp}@FgaDBKJiBKydBS^I>6y&e~(NOg|ro zA%;|G({1BT(t+1ie-xxpS*ULZ0kaa@eXg|k?N)!yhO_?(Qs=_(KskSl&C6;ice=JG zhP(u171&VSIu1urqWh0C>$Wovi(4#$q|NM36*FKVsE(hNmQSR& z-?xsWl^**`>|EQb4892Vq4dYX_r`}U-5NehEALe*ZXBgib{}rUN^))T{==kRz(3^6 zTdR&nkY=jdQ{U60x?Xc(fMMd^=O2pxh4%D%lf8(oEF()jAN%LqN1>wlt6kyvV}X*K zfn4m0aFXw1wK#<=WXqFst;z%2;d>8rQyaj zP}<|4>71%je^q}q@O7~0S9^tf@V~=FtGQV*3y{Tq@YCdvWg=Gokk+rrBpLdTv>d5B zo9CN>{!ozgq;1R5shjm3nn#YCmc7F2XnJCZW5RKR54Y=mFSk)a>hN@)%BWDjt}%Rw zP`fFtNDWLg+t@1OZ{VqQ^16qn#RcB^12KbDv?Uh44JDb%cz+^>=Zfmty~@?pFqI?R z^Lfq^Cu>}>xX@8^Z=xI7Cub{ya@37V0Ip^vIkXX>k|FM!L;PY?qnT!V=95RYH*RTd z5E-I`8}ir8>>K`6S_mH2IR* z)#y33s}?g+T&LGBtF30`sptmOV3KXo)ExL}wIv}L-voVvi)H!^jL@Y8$5Z^;Ioj7~ zfZm#fC~8R^9;v4iTHQ4Tt5zdo#OytyuQRcV&3*!h-mM%(U|u`YvSu0u_@(A10?*du z>9)UTpDLKZ;@T;G4V(08pRdLYoJTEsM7oC{tzIg^>%sWW1N+04S^0gLo8#o7B4(Hm zH@k(O0E+{j6v!V0Q@;NBAZwHbJX6)K?Jp1JQ+PMknL84`VKKvCf8HWASRew<9{_NE zT~S&?22}9SV^(+Ga(J#>$;J16g4?!}*#No52Ja_JO<;VwrOW`F4D)j>$E|$J{1*6( z_(YG4(!!67dCO4^s|5~{H^JK+9peA7xY7S&j@N#PrkT!>M$;>FL2$^E{WPbnpyu0& za)IaLg%WZ51Z$3BKcO;_7~fs&NH@Gl&&Dq^{{U&j%`;e@(5A%UUZ!L0?7`HuqMkg4qRy{IADaGlhAx>Sr`Z%k z>HRC6m$u!=G$QWL3{acUlRn^l*m#Dp4W~}Ry92O;r1%q!w-06>w^q0E^YCOtck1<~ z10(!8pBtaAx;{?_>B&VN4-mMX&+%E=Pt#7{RNe+FZcHAjip=fNG|J9tMtKHBA^eQ1 zZ_am~GtsiUjk2+5aJTYj_ghg6A;$uBkc{ZPCo z%$JXW)$Ja0hPX|n>mktllYYzk8=1|Ugtz42n`~SVjWg+qDK;%xS4%=VGXuy)mZeX; zZro#D`@SM%*X;UY>VRQ4Cf8v|Ja;qX3jO@9PH8MVQl;s|-QLz2e!k4t1V3 zwKGoYg;y{tq7sZ|bE&%6^1SX<>xX{LCgGC_lcCXF(;P8LwW+G}r?dV$CG{hnxjgH? zwXx~K`0xK@H|kRxg_H4!!Cq}5aF03f-xf^!NF#t9G8~3@jV`r38so+`ZI7F-5<-6D zg&n}D>o5>cu%T$jaHS&=Qg)hPR;J55+L;s=1@UV$#|MKvxBKe2gfdTg%POPOTey@zNv!@1UR)TqlDoUMJ++Mqf zRgK#!lQSsse5iScLXdh44dUSQ0&(4TRWOVqS=N-d@H=0EQ1ttM*{YADM_!{;w(T>5`r0E_W(;QL@mSXrppIIg|f z!cr$`h6!!hE2)#ayENivNYF2QkLR@&&Rmq~uRI@QbALUO7FC(G>X&5$PbD8Q{|vj^ zTG3;7-BCk#k68POd?k92?y?j5+$&?X(*Qsr8 zbml0Om^}g^2adNI4srz@Z5rvs#K+{ragdqDu(L9J%&{r{^Tu5#{VfnK%6mTK+$Ma{ zHXLy#p5YV5=R;fo)X0Q5S`57R1X^a)d-@3Yj)@KI8?{81fn+~ty-w}>u6VrFw$su5 zI`y6lK8M1-=;zwwF-cEW?Z}$m!+*y(dPN^APJi z9N$=vv-S-Xe3CoDM=3(R_1c^vjj6pKE8&H)=N-=yN~6EW^@#cKsnkvPrV~_)TTmp+ z{}oJJ!}*)Nv9^&-FGnkv_}O(gUp=C8h5vrx`OfJxuFDD?+OGi~lq*e3vWiIJz?f!w zTuxgk|~&8#=CIP?z=4-rpP^htMW8GGMuHGLi~qB`jKd z8vkeayYtLdq1#DB7{wgDBPGr~GGb5IV(o2tBbC^SNhq3BA>xpE2_=-yd=N|N#F#{Rtnb~L<`ApZ|39zb z9z707TY?L)=(6UR!gpKh|0Sec#BdJ*A_YGZ^P>)eda>CFHV2qCuY{yyVN#9CBRU_G?`aoZu^JcQ^?CAcQLf!Rr2g487n~U0IwySzE zcP5D~m|2@EA@$ZXtXcFewtQ~f$!>k;YR}`-nJM>zSAakJQ-f$e-0)bUM7sd%Zh;iN zTMRz7YomO6bsRln{^#Gd51@5~(kfREpRoM6fjR0kjx0n`yB-0n;gqeoms;i>Mr~HF zWV!^bq&nIkMcpJXK6zcPsP-muv6h`4F6yp+$Xl$gyBG@#8tVht3>aXQ zYPx0*RZaI`NqCOykvQz!X`Vcx{F1rvzKGNr^Xm;$I;0sOL?ck>CP1x?ZzS${>fI!g zA6E}gfwmA%+-28(2!f=0*P7l?_kxzqag0_^ag!lH==pYonw*p<1l0>b<-(mmlJx~V zAA%9#$I{yGW(N@t^+h@c0GLh>_wLNfeHD?A7^x50&exE^*U4n~kvuO6p9_97iYclY zgfILOu+_pm5r&DjqoQNrzwVqnb?$`q!W=cD z0n66FfHir}dTillSKUi-KEen8G~o9A!RM~cJ&*aAZR|A$JQ*O@S(=V7iO1IO&lHqq zK9_4LU#yMm$6YrO+2pu%r5!hcD40o{YC1i#1RM35QqYSHwOx$cV!Y@w1#GZDg*8%P z$jFPY=~#R~_3yXHnc6oX2H7fX=pPSWsn^;8_*)HQ@KQ?{=8stHV}ePsD8!1k&^9Jx zO~RZq=VsEaoo7k$`h+C z;Yw$>uhx;>$}X#iCYYl=^NpdN0*)_Wr^k#YiWZX&^csW zC!hf!6`exYJU(dMHz?5uI!L!-Xq%;}gYq}qq^(H)%mZ~d=l&@CpOi2UaV$ftY;3W) zlK~`Dm^<2^Yd;5AlYos{3jpd5MxKOkU&|80l&}bL^3#Nv*{-xw3@hn@bgp}d=9tfF zN-7Gl9;Jss<~jKm)|Z{`6(ip3KO07-7^2tHn$~&9 z(+FiDXTxt*U6?7Rm8>KJb~6z)W%U&Q!EuKx1Bl-aKb&BZ$iPKlPlOWZ;~c9b#sj7T zz9UpN9ixIPZw%4?>5ZmNM8=BCM9&&aR{Y?=O3?F#&|F+^gz!X!#6dD4@;Xu@V*Bt_ zY~$}vFQ#+-!2_14!hOj^G!?&r)|8b$;h$Q}Qy8%+RUT>>110xYTZ9f5YMBK}jI;L2 zn=CLRiM5F7J>SlKwrE}Q^YeS;WQ{&`a+CP?omT4eMth=AqLPWzPk%4sclcV{Pd|wN z5Lk!mUy8Ic96EKcOc&{z&7jHA?1{~;6k(Hgt-+0WDuhTDQ2s~)sR3Moza#N5#K&Ja z!V9hQeIf8Q34+RRS8?NGcgHfx%bt)Q>0e`bM#Q$W=5U8Q8!WR&u7#M# ze>{idJw$mdCTi&w%Q#W@kw~5y3|>U44c6Pt3@!NXWmxVa3GOKO2D@n*Y~K6ZARqe} zD~Kd-<7gVb$z(L_eSGj(?2yqhqXeHA0M1c39=6}emiB+*r72FHc$k^s%%sFdJJHHTR!+uYd;%p?vYtfhkU`x+`(Z8iTdp*vnG z3tqW`Zpyl{t@Cw}UQeLPA?_<=nAU1bNu)Sdlrz+EwCGr7PR)o?ITlJUzmtpkuq5z> z*RlWn6~=azmfl5-o9-fmhDl5dJ-_hqX^p3Xj@v7Y**|N)TI7+LxdDOSMn~E7J$Ru% zYQkq&O@EMsrH>tNHdEhb9l59+CAT9&4s5^0{!WTN)ZZNQes)`cVgt;}i@?0hc>Q&r zDKExkb*j$-TA&MTrK6kb#LqM(7$ESoE(H(en{5^>wWAXKkA6h$j_k#&$)L@n~uX13@@h-MHk$kZS%PuB=L(fzVBpzihQ z!&=rxIchGIO8OA`WG*H_j@}ay`z5OoOIHvk%w)xLBbtK9^8T!cK}H8YUN3l+Oe695AF*eHzQ=$Sv3pP{$m{K zH3{KU%Y=0*eVbBNzFn#}?vSa#JFGKPpc`@+0=8Br)rtNy$kzmWVC+CyQsxX-4Ga;x z`iEJ5RUXB%^GLKec(ZU#Hpnb&o4o6^N8%$oVBDy7|Uje7E2A!)PNlgygm6QCjeG*nZ~-+gnNdPSv$ zleNtAoq@23y`W^?H9D>o)%_$hhrAZ~osG)w$MdBe)yOl0%$Ts|Cp!yy*jqYW_h@SNXUtYs}=kCQg86ZoHZlFtvuWh(P&n97;QCu0AG z+X3;aP&Vu9DRHgwf*BC=R&bU;PX#=Vem@pd_^a_rEk=8`!H8z4+fTQ5!^|&-HxO#4 z5RU_1Iu2kJEZ42=Xg z$2~O}eMr|ec=6qhhJTHNu_}(HmEoMTacvoMDIz1?uee9N;?u9!Tk5(<)|uQ;a~f}z z#*Ijbg|n^_^ysV(bYV#z#$OMT&*FehdCJ{w4~Z6N4DtF*&bpQ$m}Is6%m4Cmp3^fD zZLEzOv?w=>h4wpNm8%Y0Dm`TATW~s;%)T}10qE7eoq-<2B;QdRU&-Es$S&j?#eOL5 z0aW7wmC=_?{3<_YZ!Z@i^lY@a@DrK9A?fhK_FFc)g=`eNnq_y_J3jb0`0gOyjf8R8 zWNGL>h3n3GYNu8%x(FZ@s>dgB7Gt)SEGcy}canEj{(ap&Rh>+FYWRX|4ESfJK-M~? z8v1I6HKOyAx$q>ID?9(zyyWHSy@I9ZXT9V1Uf|a&)MUQBy{nmOYcv_?VNe%Vk4h&5 zempOzc2~T1UwiqCP&NDgd*`*WTtlqX2r47~$I=pv(Egyp<38h!$f!95{fZGMLB)op z83F9=m(?%QO`asK@}^?Isj1%@VIG*o&ASKEK|2+9N0IO!*T3cGDqN3&H;6M&YM*lo z&ioledK0XlW4qPlzdK}$v}!Xl4#}Eyhq1yCM(0E|df&>C6?4(-(_LTpMBz=@J%1x} z(2ZOzkQK9OzAr&P)}mc<>@Tj4l&!18_w$f#WJj+EYY! zswQn%A*~-|OtH~Y8M_efwz}&zw3KZ6~wR9Vx2LT0DkSqbaW0L!wJWmTc#U#0%omspia0U+Tcmtfwau`(G=# zJir>+8biPGKGt6p9>$)I%N{f_^&c%FmL)V7(`0Ib@gXZ;_CMpPvLcN1nOXgRlfkWa zT9Y){FX=5%P3OvxJ5 zUgwg(fZ1}jrdp^xqzgWSpc{Lxhm0^4PtEF&w>YTe`pO@uHW+!hRH+Uif_>LS5kJqY z#uY3-u|&$UBBv86P2$i0b$cLb7MlGUzFwne2P=hsxId_c7rl?92zgW4vg@U1*HDj# z97NB`tzpu zm#XhPP|1o1dCc$aj|c3{_8r4~vF58Y-$pI?w`X~$t>WLf?i^uF1!E1%43eM9C9R$V zC{xR;Qa&n<$O7TNtFG@-T#>OL$Pj~A-OOJ}YhBXT@w;VouNio}&m4;#R zt@pA!?^}C1d+;tAMY|7g*5dKnd}{td|6L_-a{>9R>$I=k!8r5d#Fu;>&N!dK(AlQ$ za2~d9%T`ny!&XsbuKu**ZO^<2h~K`+r)Dse zsL4Daq~0~-iQ1o4@y}IF?TWjzqt2QpQ8sc`;S4>+r%38h4D}BX`1mQJTJQ3iQF9$CGGq+Z-UC| z5i!S*A(zb~f#Mf*Zpo)nF?6@%gtCn_b)HiS*9kR`0+p(Ebb}t%e#@t11P8SjKHie% z$J$L;pWIp%7Z;JJEbO01rR42+(Y2SJ7b@o_Gb`t)K3xa>UUY$vob`Dt=%6>k0$vnW zl1edBU%62l8(>=JA%1XY#p7kArz?@)?rqAItj+%s66VQOQ~hW^dt7!l2xYmF`mZIn zMopNO?L)@KDJpOaLfkbZ?#v>(G(M_`3;mK;&a{$6)i~E}2?Nvs_OJ+Sp6h3F2@_Ymle22bd7fV(6VZEMC5||9{I7%xWjiFvgA5UQMW-Trk=K?N8$Xws}v7Ld6SJJ zvr-+V$0ASN8YxV+F%afZAgEuNhQiXBzR)p~Q(DS`-%BR`k!ClF09A&NH~C9CAK-n7zeXM20y6I*;t;yrnzZL~I;A9C6%mq$4T zI0uS2SD3%oq{q=-YHu6-TxZS~-};NZ2|hN?E7w}!VE>0_>$4>>H(r*w{Z3+K_8O*e znfcabEcug5ZFAlGwrISSaK`+t$5Ee0&2BLqW4snDvPX(8hRo}%O0o^;clvhixVBKmRF!WK_M>Z>=peVf+d> zs~fCEk`)EI`|txuiGdWbP6iixH3+S=`fXC~`@Lk0;DHkxmuEDx0OYLi&y}pD0GwtN z$VF#ESgXtG#-r@h5~(jo7mRVjzc4myTrm%HrWOJ+F;HUNIkbTI-*zWc()k{CswK6# zxTs6P*l4m0=l`(z+YWkUb?3LY!ZMeD$7g6Y>^uX~-Q&}H9-ol7O&wLE_3%!2RvqY! z8ydbMysv=p_kDehGW|J=qT z1JG^1zWqWNow00yKcw_+D@a6GjvUx$T+e|o` z{6PF`#f)tZo4cR8#OB2?aUhG!W+1`g_)t)z0Hw z0~I_k;^?qk_OmHc-+698?}a~R{rG}W+DrGUwyk&t)!Fx-?wC20==WrS45~n zTV=jOdJaiv^n2FLWo^f>F&ky^jiomIryP`aDAMn8 zQ1Al18Nx%UrzX%a;JLJ<`GqV=&c4!O7Y%nZ6vfWo0%Sy6U zxa?|xx4?rE+M=#-?kl&pl^_MZfKAR*qYcd|>oO|IJNBDA-s_@(W+qjMv&#nm_v8`8&S@k=UfCuH+{r`8mae&a2|WfS@m`-^Oq~{{$oOori;Xb)82+8 zJpu(Q<@Sa)%|Jz!D~o&*=65P!T@(HYX-&y#X72bnQL2_tA~A~ciQHZTGSh^PN*cEx z6WQq)ZmW*FPKK(MNRag+dD_vJV#9Bw=vXBKPXGzU1_J0W99b<@!XmM)v_f}1Jvl;X%-UV9`O!Ui=N~HHX%fn^6Luc>R%FPVhFEf zzu0etBChR0)9N7u8q57jpV51~o-QiF>aeH$hIm@i+o-mdnn&dHto`G|PtJf6q%Yh1 z99XM5&o>*n=ho=cYnl^h=1%__=*-Htuhp?u=?k<1SE{q_G~4 zL1t}t_LoC$@e^8UZ`FPfE+bvt`2=08`F+x}J60rX9hX5T;Iv8fisiMoCE7AGJvD$f zbd!#1cCj&^bG!M)0U~%Mo9%~6ji}olS?uV)c5XxR`1VCFRr;~?XGH!xO5`+fd?_}nEEoRMTAqOXh`9?;lY|Boz zdcTDBvd#(28SvTK)W1>QsU3{pFukWDbwY6Ppsg4z{E{)l4iHd&EPnXoS(c4&KA*<9 zU0iUXKHIaRkScydUrd{tOd7<=DMYnb7aae|Id=Zp3#f~NmTWdfj- zkQbSZuo}q}uix{>rjq5hZ>uvQQ~~RMccal~MvR82HnMAU>Rx|B@jjp}GQ|C&SgTf) z+NWzPNEQDX|M4q&rdYtJ)@a`@wC&a>Qpk~MM^VIjA;RzOlZna#p``)+w0(DW9d8cv zpwj1fN^)yANT~=4 zn6{qO^0y)tHvcx3XJTjtF)k?bqAH^0!iiZ_A9IhWe7*z;&QYJeQQ*}oq;iKAuA2Qn zn$E&4s_y;zG}1_ngn~$S>jT12BBjjGL${PPNDd*Tv;srQ(B0kLNOw0#4c*~;{9fl~rGYUMKG_Y!5#Iby_$Uh41npf5N0H5q z%r18ItWG!<>$7Wlv|@wHTW($Er>-bqCg}7BAE_pOsBXvfW(0xFenoo3X74>2X=F~x zlua!ufuYJv;FGq(le$jllBrFQAhfI+-riJpU^OLb;^o2)AH8Hl)q=i^)1aUO?X~fB zBbb9Jjjk2F$**7U3r5_fy-+U4Ft=^d*Z*;+@%aPd>Pf5=9M*eRqg^nnPMpSZX6wdK zG)S1nW#r#$!A*99EA*9p8(`yW5)ZxR&P{!g?fY*XtwXDM%cB;O{ z(!Ia~j3(E%5pGuBf1pj%^IZBk;SyuA(uzELpMH%-5n@CTG```ePU$_~1RnL_VCoe2 zqXXp``^)By;D;B%FSc;;cL@jO&sa>f9qUO)9gr zgMn5NZ@I7y5La0rf7^A8_mXd0C8)lO60@2t+DGsv%pJBd{L3l^m8`PBv#;_u^3J_e zzQJbY7cvN4(rOgXw6A@l<620^>9(rOT|#P|0WEtd_?+t-b*5_gASm_hw=U%M9pa4|T|Dv(G>|2ZV`_ai0oA+u~ z_b%-bjRq#B4y_aOL{r5cFV|WU+GnxpOWGG|abdt>P@eV9B5@@4(TN^f3e3<`n0J0u z8_>}{lK9&rN+&}G0Oj7b(aCj;XIexn#c}7w;6>=jVwv@HL$yT!q?Fq!{mee6X0Wlzo7(Yhbv&(l`NO8@Zmz|->#UPHRDNI9m^=n z_)7$3kbz~TTg|taz485N=Wt~#TU_6|<+uS_w8s5|808E>-ukTdlpN_7`@rS!4Nga3 z#flz*-B9+Sn$LslY3jsH5Tt@`aw#XGB~sPz=cL+w(QX_im|83=i?yeNbnpGM&jzpJ z+t%694PjuB@gV+pNSh}MsLfxMJT}Md-AZiTPmp9zLwhP6qum(7fNoh}y@FrfB*+EP z0E>sc=eK9kMu;a#OEPAICkIVU#D9q7F2-^1Pd9ng`64`io|5ePPmU$TBM6Wt4=y`s zts!X8(B(`z;=ds+()1>4SRTrbNYMN7IFb&U)`vNDnyW{5on}qvy{he;$to)~&%Yy4 znRHrWmkBc~UvZ%2;{SyxhwV7KsXV8YaC?RWvBFC$dNA(%0rDBJ=GzphqCpqV%om#O zc1o2G0NoyMTCAl=T!W{c^zq4^v7EMtQ>~V@<*G)MAw;%rLi=l=pEMJ-k`bAm>th*1 zjf(_k+MKX9cWLLXi8xZU3F!_6QwRLC?&~_cbGbqC?A=14ah=r|*(24Lzn(XKx>pW@ z5bnIJOxR%@SQ|IW(fgZ?VZKtS2(-z>Ts;r!GH{km$r8aii+n&2k@Gv3lK>@X7B;xV zt9n^UVK1FJlaSii%!LDI78*x;GPAoUd?klUM2NU;CH}nP!#1<=G=Xm`U7Lw?`JHHY z(pVooZP}6MqQAUvGTjzQ+m`dxa4E%4E0V-}s6+^FzZeMSlDT0xr!7f%_H*p*UQ@ZL z)u_jg`BZVm0Ef1~AyDHE$5a4^H7^DO>1`m!03^u99$`LD64eOdQO(i$CW*ia*(Hdp z4c8wQ*_&F8etR`PYxc+)JLP1r-Q4Q_Ih1v3!n9X)#-Lf?J5UmZzAkK<*TcXg8*R#f zPIatPGb1~_kTnvA@OdApQUZ32;Rn+bpYmXrN_W<5stf&u?5%T!`~{2xfEk^w&pFzz zrvUX6IUb4=7JD($90;L7+pMsBgQKpu8vFB(a~f9(%$d(v(MqE}ffg++#5kz2_v!P! zYHk&296#iJEfMFrjIZgwr4%MxJ+dEOqZH5MVV4_d0W+wwX7+On43cGK_UrJ%>lo?c zDfDOWHu`oc#7jMw3#)r+#Wd3qqaYLasY+28L#J6s8mAAoVtV5ZQ=ayw(=!O5Wb4Io z+TVSodM5*Chx)sAkzEp+tM%CXOBT`3&zh39$B$qOjHV$|7!eEi9jwp1Vc~5_%h{3* z6bXH2{XOr4=hloi5!)4ApamqdUIx9Dcd9#E+9mB)|96BS1x*plh2Y5z;TjO;5DS2R#^*g)eAc);lK<1qE>{jH#>NTl)kSe;NWHdHZl092O;!PT(!dH z10`nvVUNQ7+Zn_D+DB(vFt>VQP8|xF}}r$8eXGjG-f?|Nfv3vgn>?ME@9e zI!gRl)@jn028HoNp7Z&3c=$pA>D|Yicb42oXKW|v>~8nh;+rb59A2@tRIlbN^R{Bv zS{j1YkvKW-tCnY}8I0-m_#W-@`rkcXceQ;Z$Wd*Uz4k#hVyjykcTjo5jPM2}wmoGsJJ-Pt>Cj^?Bj^R^6c)hvlY~4Kc`Wy(J zGZ&|QWB1(kXc=x$CxJ#7#!7TW{^D1=d5e1 z`cW|xYM(_Ty@ehZf){M3K+plofe>L3d}ef zZ~UOiaj``A`-7nj4;z@vxBXG0TuwED>o(DGzY^5_3g+-Mib6Jg(#8n4s>{r zJ*(efAZACBmK;@%lG5aTYV~bKGSU2A^X59Wl*YTF1a4DL@e5HgW5luf%zn%E2kK|o6zaE-UuX~mhdcS-{cDU)tpi<)uF8ZSxSsS^TYc z4(YOhmbnqgz6(N5dWgDyQQ0?6V?NL+tYW&?2>@#ovxNui#hw*NDGnuv^_B#Q$A0Cp z|GT?8QOVR%=Y*y~M60#@j>lgy+OJzs@pb1-CI_t|Af@EYOQeTUklBU;h=Iu^K5x5+ zX+Dp-5m6t4yz=QF;QxERSx04kYynO)#)Gi#R|P)bJSB`XIox>bmU^sBsHs?50rO$P zft24eU5}$NM|Hju9{iEIO*}3TWRfOJdf$FvmIk-qaGAfc=4S& zY|GpFnB5hu4<*M8Vk`7F+z0nKzch~6W{<(J*m`am>^?}xbR{w9*>PRxqR_OCw>jr?dPQt#UO|n zi0H95HA078t&=pjDRc{sLsn->U6ZW0gFX+G*7+c~-KX#s!p?Ba1vgK?hz5P7INMOk z3VLzgUIc43J;rl_oDgn!R0NB^^Vq!s`h#pR$+_B;=oXzeNTOF7UuP{Q*8X=)=$3j3 zxaWr$-$>3#tn=~tW(Hb5(iy4mY(A&_r03GMLS)FZ*Ce;M5Pt8vOpQ~c?t~~hO z6jSUGw{Ntg;N*&})B{ahDh_)x3r+O=BSAJRrA`NHFl|YPV62RUkUa~6T`bXr;i!%X zhvwev30tQl8_5dvv*}&~n=b~KNfCZx-_{8_#Ay&O1TG6-dYDMcNSSo>F4V}pcR1Q#hDmao0wki zCQO&4u$Itn;KIhxuoZUMytkF<&L#I(1;G%~1zXyx+6kgUcVY0>!J^ zSP6cYOG;?9s-YCd$WgrW6E4D2XCEbbm&E{MK7=2cqYnjn3C)<4VPHmyaZetO;N zzNRMn`7S^a14_7Z*7Q;1-mkxKgn3n7G8L^y>(m~vkpH(1@rAA=wwsFJyMZr&DZ1u+ zL$%s^b~tWY8wPUG2PnvytA1cl%HP;*72lY-mk8zk06gzSG1!`xBCS~iy5YCul?*vr z>GfyS@71K^>W{5_hAq>A1@P@au{o;Fd=} zob3_^4-y3n*PVN(UJN$GoG~G7l0u4Pk#$Iry^<1~jLG=({Mpzq*2ksVQrI$MmH3P` zF8f6`KQ1&|u>}995G`%Cv1k23MdLXCW3q<^nN?=4Nk ztc+YFLo5Nlt0mB;35MTgO|jqE;T2k-+gA{7(w%)cy-+**^mExdiP*utZ}fifMF!m(a_)B)}zafY(C6&u#=t=VeFY$}BC3&)F5f!b+3QHJP^ zvsne~mjZ7hvHiOI3J!9y5te6|i4Ao4`~lTCg5IJd4OV#UdHyTvu%F|Gcp9c&sVM;i zIDVrfL}1tz+j8#gS%0oqKA+M+&{j}0ReWP!*A;ug3#$g8W;&S%fXwYTC; zz38VU)*{!tR26WuvOi>$;ddb*PP#HY6qgQwP{oJ$1h3pZhl7Z~HV&j&`}S(x%z&MS zquiR!5GyT5nOQnPlnBr=MUA@Q2?>pmTTS+k=tyv-a`=%0&TlYcU8QSn$HXW|+=VGE z$?&?%;X?pV(Hjt~%Nz(0FCyJEE-xs0byT(dZQ_xy;}Dy!hr*&?*X^Dw5`p1XPYHNnZ1*k&Noq_h6lhV zKi7X|j?iX_d1}ZXMHl;nPME5Zbj&CzzM8Bfo3t5{W6yriWtg54koieiEf9K>w9Qa~fEN~Pz7^0(XGy7)J zir*<+?}_4SqbkI0!@vei;#N|0-0wtYAlT7pI^*)PyX?0)_wf*hKh>zD0e_~?lg+MG z^1Z6_m@QNO`O^fG9;a!30&kOF<7x52S06cw%D&Rz(U}I@4uW4@<;FhhL|mw%5F%x5 z82J`DFhK$?BkDe3vY$2sx3Lv*8>fd}?9OolEX%pvXk8VlRF^v20yS5I%~Jw9C<5G4 z%Avano!Zx5kdcAJDe~WJ=X%YSf{?95UP{&@KpxQ~rPp*ln)Po+$QYe?O-YqoBk++X zv2oVEdYvM1u^Le2I+A)aP<)J6ZCM{)mraq^cAm3(o_Lw)`e9tA_eCa2 zprwIg1(?8HCYk*hZ>u^*SwP6}gEC<+*Jy}2qm(frg~C$j2>%PC>#R6a?QHiJ>{}R2 z4kAuTSgTRb`KFW``0D(FWHAKnV{Inmf{lt>OLqL3-_h-Hb?3dn#AkmGpXnQ3Wr_7- z%eLb_NPgZ(2@dLkkBDE5`k`Z5q8962+KN+eefDt^y7neNfw_eH3zqeo(>jyWPQX^dkk zS>98ztL^JZ=DUE`plhvA02jFr%K`fElNO`2^dYH%gt{W;drf>`s<|oc{4B|WnA!4> zb=p5?hAp#Ld=Bz)#r;rwnr$SVg%uzea~2&dOcw_I_+Zsaah)|}1>AT7JYg2!Wsf{i z+)!R~_`rkSB_{H&EvVX#6S{U;oF-rRrh9(`j!;P3|Ek397KY1T3J36Z5emRUK-+1g ze2;ae)MM0X>Hhba7`+M@go+WruDYN;eJm-eEdqXc1QTa{DpP0u)Ex`+5$R0-MyQ98 zc1M=>Y5?u9@>#UhiV!qV0L+_9RW%_wcE6b`?W>xmBacG{?%wQj_>BQ~FS(&3D681} zZvI#gr{^Lp=9RsQnO0#gU2AIi*9{lE#~|bS2#~{U-ubZj5?}bG|FkhWjf+Pcpvke3 z>R=6*qM54(=b~Y9AJER%c;ZG)64et}Ml@@UWNRGH129#nAdMAOEi2T?VWEQ!ojo{wff zpeSC3!(`c8iz_9wl!(B6qov1>?y7sHg+$WW5YOXnI*;UOdjRfEh(*CR|Ie8ur8Xw? z#Ba23VA96cI1Y1VQ@UtZmcf0%s2PWw$U9A+$n&NrU_^*9exU{jlZ&*?8ny`r+BpFl zb?IirO}yz0VLJkx6W+@mwqh!xKU*m!=o8$rRDY!)neRbSF2;NwL2WCn-GA{(-wWX+ z76P)DwtdmKkJoAntX5&o#A@^L`-aJ!^p@wg75QkR_+dqKPSdw_M>ItfEi*@5!t4 zT}){UXSqYdB%wg)hH9$(%2@0Ns^VhAs8@StYDggB9FHu(xAg!*%(w$`3;}w;1B!Q08JB8@XEl25QcBhKAQdP9* z>=A)?47V>%-JM;TG&Kh(B|`p{0OEt{+s*hF?L+1o1@UPk1fv+ss@oDdtSEYymgdm2 zF;{^0j|)KziSfoLrG+v_dy>ssS4r1$r+p2{of^c+ff7G0=@l=I95D?pqcE)&>asfS z7f!IM7+sBx&HcnyJ3z6Or6P_Ewpt_9Mu9rhx3GNnuDOkyvr5vC=}`TW6Bti_bh-$w zZ)Y9qB!TO_xm$_J8f|*A;X}p_ZbgZTNJvsh>eMxNFEO9-6!~Iq(2uG;^1TY_Vv1`f znQAN2u044Fsqs>nhkus)vegD85mL{ySto5EIG=iiGEKUQ&@~|NvPztwD3zFN@kU+G zE8sEgq2T2fNg7Bl)A&Q{edP|;k2`9}srkwut#Zu>hcJebqXF-!0d#e9#Y(b$0dzK- z$me14q*wYK7z8kB5AGu}ukkb@daNHCd7rO0&xY}W^n?4ePQ6;9?L9_Ntc>%;SgrCjq_ zc)jKji>X_g&!KDK^?vm0H;EWZ3{p_V@01Hqy&nAW7z}mk|d#mgTW6It&Tj zN`3PvAXxGPqg*5xq5YI_(rBO?D^5B-K5zoGB_lc2fM62Tk7G^{3lz|jz5K>}g!mm5 z*aBON0S!YVIazRrMa=!P$il7ZJh=)J@;A0h7?f#k%4WIf#X0+SQ#PE#eWS4?)zpjc;kLF%M7@=#% zu>(D0`QT5f2a0aXU`34TAWi}|OZyFI-|o4t)z;*2N5{@o9Jsl5vQ`zhKfl`THsW+C zbuCeK;kc;JEM~2b*WNZ9j!R*z{S`}^p<2py7ucOVQxP@z<;0or76FhGa=kSzMy7Bm zSNt~tqLmEL&gXSjSE~mR2?JZVZMdebMrz$@!<`nTw+ULh6afx%xq}m-OY{i)=8d}I ze{XGuIg#Neeo6!?Y{vLLCE-v{RdO>}GwI|8SUQK{`qfq@y0o34w4(Q?6VJ|+o;^sA zYkoQt<&A{+7cSSsDNfq-J%zlIz9!&F*TdlQzrpjjz%wL|sp5?%AHd`}VQ!Bp;>0S` z+bFehpaI)43GdEI2SF?cG05v6DG)ZCYA*A-7K-aYucEd#{Lg@*ejfEZQ7}L8?78S> zR_C8!rC{tSnT}j&~6Nee?%?f6Q(6sfa|B3N)!T$dKxyS z`W1^nJrDmHD^cfVw6LCf-|+!Z@t@>4uOUCJ&JeD$8z@e@#;rUBXxaaR`9fXyj!@pzNpUW7^FE}& zw&g8sJ7mGWR$D9Sc6=fT1)y}+eB_EJmu1moh2y@ZfGHEdCRG{ z)oq}Y=Exfy2b%ljxACb!YQeuT{d5@c#Oz(hYSmyVwpmDW%MzXuq$yfW;HTEv&8)h# zGsdeTZN22g?H%k3RA_sysIb0oRXA>~dj7EVu>E%qkVOo8?jIPUcJk_KLe%L240`qOu`;5>_yS3l!)#U|^T2wJ6~V}L?iVl%(XwemL|y#V0P zKC%WfdFYos_ltu{8W{FW>eH#ZESDXf^ay>W;UCA;CK+TXdYbTy*RV{FdvWK4qB!$0 zLGU;mCMGX8l7STkMW7M!0@@cUyt*R^a6tH!r8zpKm?BWW%P6W6Br`x{h6wiA@_UO zYo5v`>5JQ!yDR6EQh}5~+M+$Cc-RY}Mj4<5IaKXpMk+ z9e$SXkKfX3!ng6IntbCkQ?UPi|Hyh82jw3ap(9ny(K-ABD9Jx!OFi@{=n6?4=mEuoij1x!FBpjU?8& zWGmFFie>M)7*>Kz@WUHH;F5w?4JWi}($-?F*jY)MFQPXPe^ zsjI%WsHHNW@|Fy6_|iH%0)s-O^eH$U?k?e&S0;NHwG`y8JJ47Yt=L8s*4w)AcVN^P z&snbw>jzAFRU`fn-={a~^z${_+Kv$)=Xa45QTPA_Jl0sd;r$n)Ky812p`u>D2#{y^ zU7|n^9o#LW7v;g!=oV7tKIdoaEi%#PUzOlW5^v0riAIs%BE%(&`Piu;!Mu#6Z(}b%|;h-ae*xHn@J6|ey%zims zGg>j#P%)^j;?k~jp>@UYFizTs`4er_P+{ob*z{N0)$XY#2Hr)Ps;pdhNLh%=n*gmr&uO&ggF+l+(StM67U9~~kl`_qb=m&PZ7@YK z8tXdi(Cbv2rB|K$SOGk zqCtJhxB_$jcg!9%(V(t-e{kHxS*Mgj&C{e_DkTUJa2fwWOEB`MU^jI#o&Xl?yF&p! zur=WOIVLfmz3an3njFgp_f($8nB$k3op6{OQw6x9nykzEFeteFluL{eJ2}aqU$09F zh+G(!!KWqM!DL2xLdG3|7<%1&fQW|3ODeFX5XGKgc--~y5gnTCuHJQuHx#8}h-s{O z&D5-Dk_zEJA{V)0H*K(4hgZq%EaBHm03ildhcyd08VbSAGU=rE9#%btLic|P|1`(1 z3`x(n2jqVmN?P^=!c)&iB=UV8ZiSwW8Zul90iynS@3Wyw9kjGG zH-OgWblFI9pn&$zhPhWYqN%OTE%6hJpC;`c91-oq?mV!ucJm#bFF%u?gng2d;9;kx z{N)^KoMOdNh?f9 zGQ*Dh{28H3o!r^>zp}T}v%W`Y=w$)*EG!k&`D#Dc{4-e_O!@Lm3`RC;8d>G2ecQKD zUUi6w@R!DWFadL-GA#K6B@SnB|xYRGKn?$!1VI0)^k>=FtPy{0<#Mr zCucyYqA0DcD&jqRoodk}IblC=z7S!-&hoTfrsgHC64}-!?macK{~e!tq|RjpMJb)z zi1QWRzA+}dT9}wx>QnaHqGAgXux)F+>CZcj1MOn*x6BC>9eNETxX;bX0%>bn&bkKx z^@%_hOxBg+R~?Y#>$%sCdx-W<@G)~l;DdpACo54N+Aypse-vW#-&=ZMX?t0@srD1v zOZIA%OcAn&KPbV($?5%LSs~9bz>G`n-u?tp;qDiQk2LXQ?X^E)AwT`yaioFSUYe1O z)Jj%Ld+86Qs;79$jp`XDb^jW!$nSzJvnncGi}7NSMLg6*#M2~|3zY(N%D_f)Z6`}h zjxaWz7dSvD&Y9fmL4kSaYd zLMEL_7X%uy6};Hbq#tF00ihlS$}4M^H{b*NC*Q_kwWC-pD^ zJlwy6qBCbpNMf){*v}eA=+J92d0flJqlElDTu+CX6sXf9^=Y*R5)|1B+wpf*$OQCm zrL7ch6d_z;Qidh828`DqCTs?;m!N<{7xtv+=GGF=+-f&dkCD*2$TMTeLhP8&}c6Fhs?3jYg-wN|8jxV zNI7Zrfsn9j)J6~Vx3?{r5`iX)E(&vI*LGH}Ulu~7K=||4f>R0!?OF1b5cT6|lry~s z5+HfT9Xbh{n<6gbhpYp&rcIM*_2A%l62%_B2lie5&T7mIV;JmsoYiAVlQ|gTNN;)H zAfPp(&O91dT$-=l*B^D!)bU|I-akOW)P--JUa8Q`3G)`kwp19uI1H4eZVB^1%8SeM z->+P@`TkiH$Dy?QdpgLAXCToM)*E#s1VCa#YTx>O@&Q*ur(^&W=i9?7HI*rl2+{NZ zqDz3_;tse4k$8N~+cG&|ccSFE#1|fY<~#B6%@{Dz4YhlFdYx$*Qt37XjxqPWUJLo^ z7Hj@%$?K@O3Z+bICX1dTB-!t3V&E8bs-SJgbjRu{xD97E4+6tFz`?6Ag(QgR0I^T8 zQ*E64K0qAS&|tF|E)>K)JjlgE2Aga~@{yc2w_SV<9g4wLKYl<|d}gbrN;7XQzyMpZ z7U^T=eIl*)Yfud@o6mQBgAAwCv|N6c50#2x-4-;zC&TFO>FQDas0{9OmbWODl!?R> zEe2u;Og%Jf*Rz=xqNvlEy=91bZBfOkZ`j}4a9)2L`=S1;qAf)INRV@J?V%GN$@iLw zD=j>yP!%=5{PIeVfUA5sPUHQlw#mt@jEAlAR*Nk*XG`-6(OfP%ERsPou%!0-WZ=-@|C}`I1Oi`IfuIv&ZN*x^^z!18@M3qIxHStmQ*GQd#RNpRxPsQp%mFLv@D7Xa1J zHN zJeLKCKuG#?sglK7+qod{-}^IZI2bLve|N#YHTyllZQILh^wy`j8~bl>-Y8Wr_JD5& z7y1;R759m-}WqYhkV9E$oYkQ%(!SCn}zDJ$;C;_q| zd5*0BWU(lRQPMvDPg>7yXUqZnH%fla{OB}mig;rtc4l}yu&LGWvW35M zz&*3)9C1J+RDP+^k*nGa$+Ff-#~u6=r!q1z*NbRgD5{ya7J}cp=-Ab7aag|Xd~?o^ zVEkr2U~0A8jCf>B29zQhg$3=&?mw zz@*48%_hl>)J?}z(Xp9yMlf*CCFPX*CqV%`N{pBh1YW8fWV`ZL^YoV%6%5iTDMdsD6yiyI9Vr!S?A}in@Kz0$brxJqsUd(X*n)@zbG8YPWB3Zks z*bH>JGMst)8y;y@U;mPJkdEVuTzmP-PAx7Z2Rz`D@KPx7Gu4lKMHEPB`tW4DrJ9iMQ4MWNpSYIo9R<2!<0&Bw3 z#cCrWkv{mS*lis+)*j}(W!b_TdlEGN8$GsQ+QYvais2-o|GdMvF*lIk9935oY`bU^ z1ks^;I?2osfl~loc2Z5z7`OiBN)(y2Hg6qn)>*No-b$&dke5sSngMsna+ikD6lX!z zC}^t9GLe(+t@7^;fGy_soZ&&4zd#9yn^!t(iS`VsNMiMKjnH8L@|I-dxr}RpMP*tj z<+j5wcIWJ3V{MBy3z_?9PfatY^b?wG-iKBl3`#2f6K|_M)FN{{pM?t4;gua zS3Wv{ud8J93N>dpiSb@i&=rO`)8!NZGj1G+7=qNV06(+l(>3BWjxSwcjl7M#B-;4J zO1Ct!AFMOz`0M)bx*Bhmuc64R!W@9GpZF%aF<#dhD`;MBnPt;{Jq$fIQ0H$w>b2-b zxS(aoi!W)kc6I)mI#$`zHw&pv0hbGQM;%9S+yg1M2d@u5ZvXy6l-uqc{;5A9&m3F_ zMgzG@yv?zD)5XbgG4RLR?8HREP3)+$?L*DnXLPdZRj&`>|_W zx9b6B<7ezxDO+#v*NwiF4m3-YdYweKt#3Yb37VXI9hbzOG3V#gR-SN%r;ILPAd6OU zU3G8S@7j7~ha7B+XH4?8FN=fSU$+5*R8D9uv!O!hm0W1~;`kSKQS9RYPP0ijq5(D_ z=^?gUQkt$cyD3L5uXFEXmlHtSV)h;nAJo*}zF@kj7D!Xf2kcxF2%gWU7p&%DD4!fDYd}QeY7gJ; ze=vy2oqSrKLpna48-O@DPO20;+G^S1JbE0`p9;AGh63N3H?+Quxc4y#T?f;it7Kt3 zA^I#2efkyuk4FC5fe^I&A_GZNCLbg*Kg-V(CPy+7T3u9)oOW7lscxbK^Ivb=?mPo$ z2cMcYOG^Z%*4Z#pfMg80JihB;n6I4~i5Xv;{lE)gA&yB3U=5q(K%8tjZm3SJscZ`- zWe3KF1>tWvX8F-9G8%u~o--3}a~APq)ywuo#tN{|b+XcXOD`?hszJa0qvnVLl-Mi^ zffU`}N}Mci?h@?;p}c5$nhVuKP0nb-KB+g1)s|x!#}6VqDKT50I)Xf--tPV9>C!dZ zaTUYzk)wdb@LVK~cgbr|oF&`P;+N~6mOE5tk_~VCg_Lv{P+!MK)8eOo^fSJgFO5a7 z_rH4~8?|x+)0DWRD#G?qXnwN$q~qa*_i?+++0YxgyZUaBu37G_#O+-r{zKSO-yx7w zIb!}>Ce&3T*0~=S>8=bD`%v0w7%#jStD($@B)qjgKCPjVEI zl`G6xj=cm%mw@*!Bpka%_s%9LcKX)+7mxh486aRo&g*Pf+rv??OLFaab1>X1$P4M( z8b9&`Wn&J!xP3JP;H@(Mqxujy$#MUiR*-hT_+$ZNx~uBaG9+4#IIUpcwwi1~L75CN zSpCTqj*snDV-=SBasJgGSM|?NNjU;?%OGA&d!F-t{1w#!O<9StmTH4kulrJupC?VQ z4tAYwA?8@*^O)mJ=QktI!5Ztqm4TX+0WV4MI+^cRvX$Z&2d%&t?=XF)teSqZrWgkr zmY(5^dCNHc(LhH6=Dj;3Y;!U{?IuXoq68UPkR;-Ou`}cmewF*I2jOvfezYQS*5t?= zY!4t94>t@96Y78`DR3HQ8X-M}2K~EGpoxB;0+1hFwi7KtEfU9Vm{yy8KeUOnPYfYh z0Xj!A62lMtcRraBuQRj^JhJnWAGe%Z-14%sqPG1H9EB_yS`BhZrKem@~ z>C!u2_{Cc|KcKffDC@M`iVHhaI9W73U{r2ReXlPpV6*WjGb$%tCImEJ`R9o=>V?aw z6!qdyt$)N9%6NVMWnlD~F=9{sCP{OzPVwpUFm7i&uxa{Vrt3)~6~f7ew5g;ceABWb zHR?+eiM#UweB|gS;5 z6CA#%DZjTD`?`c;F4K;P!tqW=L_|*KPx<7{rm%H!ZAJJ|lbiNhuB#pyc$JRmkIsJe zUOuI2nNQmvWbvIOPjT$+Cgz*~k5Xc6HP?=MOkly`IP2Q(n7UdR@ zf)Csy{D-8t68i#)G}0eW>o%Qv&x>lrXw{lB>JJlA^t`lO-&yvdPrkYkHJqNy?TySX zY4+6kQ$HgakMR;fVRO~TGG&zqR#tlA8BYpi8FHnhRSyH3)yrxJC#O$h#_Jp(Qd}cd zJJ>Rw+BA#hh&ZnUK-bEORW)li36jxM|E~pKUg%@hMl=n6CvzYotP>CU_+46vBFaA( zo3djR8|wV_-b$cD$+U`BovO>2-&?7?-iQGwtPTZT&?^-x+7Lu);l^(*O6dGTBc{pv zxgiIS4{!O(172P1aIpBDAoVf%HTO78Fy!9&iCwpJ)!H(!M5!k|C2lno+7{Sr-Y-d< zdDiq5Fp#}B^nGwmKLo}IvB4ztsX;zo#@7=rYyCUVS^%^z3@7aR>*C0B7nTTWfrex1 zR3$RYlV|^+`7hiS9slP$pSq?ONV49V;6w0K}%%~{X2@ACU@fUM=`eheCiuN?46-|&W|V}Z1eSW`I?ok7U8 z=JdlH-HEp`sj3zeCbN|^(rSL8-F4l+;(=)c4FI*h^qA>|f zMVnB3S4qm+S9oDk`gXz7j~OYl5Vg%HAImg#noD8m8t{OEbb5KN@?e(l>mkR>v|h)7 zVl+#R$D%)R@l7ro?xZb`Bwrv->h1zvi+hOSuuHNzyU*}4I~|#iigVgx!raS=>|Q12 zdIanG+_=y^n{{cJ5lYYVwjFo(;3;F%S~~tq4s6M?+MgihAsvg*8OnE`FPhdp9itF~ z1FE@!qB^q@Rf#|aVa^>Lx-k9nX<;E{OL~*5GWC6Nne>10KSwT<_m&g@V2{H0!KJ(-3Ptt^t@kyn*@@#~=h3u33~eB(`pX}{>hc15h! z(1Fi6swYPK{ZE2mRA%9D%d0z&^FL#grG~8;HDmDx7AWP5$GndVMU|98x1O47cSkLE zC9C;erlKBanY&AF%R>W2)zdoX05nnMaK5^kU}r(1H8TY4p7P1}4D-Uk-a-eSy{#=0 zxaD=VR3Yk3XWe(>a=V*rv2s)~$m8<&;p*9d)FuB>?VDxyfY-A|0n)xn#dG1qO-C4e z6D<}vNCyUb3YTBzE~}2X6MdX^>d#uSkfZTX9)24-Ubp*}>YWZ2ZdMu+={;Y<0w;Jn zDSSMPGr7@xMs+I|lTG6t&^Nrp2&=q`NqO~pQJ$CW@VuQ5cJ9G76PKh$gez@I~Zk?kh_kDCP!FY>vOre9unlPD`ywHIop0LFmrD))*^} zzs3FDD-1XL?psZf&Ay)YOhs%yc2iCf>w&rE^(Nx%ki6;0a^$(2MCTFc+U$aKl2`?1 zb_B@&Xx1FHr}6iZ-5W}@wT&q0ttFZJDcAo1V|m#1ppTFi&Z*2=RS`Ca6fwfd4^1bM ziF4FwPFK9N&R{XOot5*x+}+txf*UFui6_x}YP#lSBcW!dL;25n{Ciw0R!U;+g~dq@ znW?Ae?fHiRzU|}cG%;q-$syyB50Aff99r9OARal#W{LLfbU79j5zlWm87W0Nq$H;t zEVP&T2un?gJ@yDx$p1erfE!=Y2EK=@15m3D?zskeQqz36+rp$7-+At7ci&Yc5?JMR z(*4XT(Hz14O_JuC+W^R2m+l43S!yZc;w7-E{fG{oI+sC6h>Pb_+ zF5ti^b<;t`y)Lsae>c0Lx16*8taywaTK@W(vjd75qPEdnB z*{Mdiqd#;E>0~}V7rqDjxJ~L)Cv-NSqp7@jXUjN2MQItc-vI-G0bl6*1HVulFAAlv zEMK~o8!>u9xo7#vGN01oe_^JH`1{ulS((~=wCvTQ{bglg>CMj7#+7>`&@bH#r#K%a z9}TBua0&lbQ(>3Yqd#jFn(O)r34MCbbsArGU<^tG?P;0Wdx{^87M3>M*E;t1{%#r1 zNvexm-=|I$6PCUkEi<5clvtJt1}$^x%Q=Lu!-wQ8qnBE~!~;oe4N6XsOsW4RwY z`4d9cZ$v0-hbX*7e)NUt&jG};)Ec1VgmRfX*Af#cr9N5Calt`}LMd<%cnHhZG|>$4 z`PJ<2+iy6N3=#;2-W1`SGj+Utjfhn*&tC$wz27EBS&XtG`X#?dls-N?Q}ecOM<)8_ zKyG`WA5ZZrs4+S!;&XO5>?1pxKaZ=o#4>Ld{^lCzVEQwc%<;DL_rvds_#T7dC$`@Y zA=*9}$gQQnR9{n=b0_xcT6Ydv4=;+gu_zi}?bWRa{rZ*uPBNU$SRUKROoSfqp*T_N z^CeM57eHHdpHAcK8DW14LtGd!Ad3=u`h*D5vBKuUIld@!Z;}@lGAH9_bMrKECFZ(J z|3AXsIxNbrYa0g~5d@SLP$Y*O6oyc`k#uMokQz!#lv28tl4fWIknR~uknW+SLkXo* zTKc=VpXYg>`+mRg`2GIk;6V4aW36+qb*{D7Vpvwh=7|_(mFnMTikc-dbI7WW)~EMQ zoD?sdB{1j4%?XszP*3R}_N&MaIF*Mh>h2NZ9@ZD%aT%q-HQD-jxgh-f_@(>T)VF1F$aXr0`6&IZtsN-Lu2c9Eqo0x8-znd-no(`8T#|=8y^AKlWm7 z>c2V*VNd6c}%`c_-&C{sxQ{`RmhS)}wI7-ajBRuIR|Am8(*qQ4PCI{?u6B6Z{v>-)(;)8D5GaF%0dg#A$W!w6Z*vz%MJc?> zGjjQ0hiS7+qP1B`dzb#e>)@M`r1|3?=k4}{(phzVN8kD|-OM~o=jzv;3L;c48Di}A z9|@o~&u0u5vqmlxzxMii*6nuQH%RM>Z)9v|)MX}Hu>?Y`q z8@_j?yJX-NF7VExeZewN&*mnvLm+*^Ru@#usQ_x#jP1W9&V_?T9jY|;YLtyQC6G9X z8<{{&F+60ks%k|}898{rzb6`|Y7$EV(RU`fF{3&k!Q`9hs9a|U5#n<=%v-`(^dx#S zNw_V><{Uh5wg~7E$;0mVzdT8aKn$95O5f&3nMOSoYP9$?>OYG46C1=~PUy&%@o>Nh zQUUUkjp0%lhVOPYYJI9ml1Z6_=ybaF8c`ov9AWAOA53R$C>`%6eiu-J%SrGpMy|IG zaCa{LN#k#)Y>Q=#>ob*q)5 zw4)hqYC6z!x26qnwB$sv!~$d#5$#1Q5s7IdH{1~9UfSp$CgqQ#a@u)%5qCIlmSV4u z8I+U5j=ao*@)k^8Wt41h3g%VnDUcMGTKRt#wgS%-@-}?cSPlYbOlG+9bhDcjEFB== zG>{We(prTJ#xfpU@Wx(I!s}{o$6~_e9cAjXzz@ZFXtOS)@=%at7|@#8H*u@*#-gFe zD3#+cswd_mTs=5h#A($!PuAHe(mf0tFZVWvt&%S&r$M#cp7q(zYkN+FjVh07WEh|+FnA8= zMt$T6kSpw0D38Wk6)U~8-IYwobKI^T;QoJ{VCb48cDni*>)9uM^&QimtkPR*u*G_| zS@}+2pYvM0H2gT$^Da|g&_VldgZNU|K8GG@Z;12{^Lx4Ke>>3_0&f%xf`6s?t}DK4 zjX&sp0J zP7Jv~*d`giLne^O7!$gr>F$O6=yLJ26t6lKPJ1pTcru6YzKrh;ZMk6k*w$~W4r+yZ zSmco1AAR^@CIY7-p}>K$cjTfy;w?`pK?Tiy3!^w+uG^Fb0n_quhS0D0hix%7^5FhN zNH}lNjbD^&E#&&dNLdp%B<^dFvv+}=NNh9?!RRVmD;FXZNmo40IYi}SxWNlpoGlCQ^BA>LJwbLzv_(`cS z3VHq#KSI`mO}e8TZ0^UxNy%M^3p(-b4nG~~;iOm53t1cIW>a$~q>UUrpk6Q`48;5@ zx*$#u6))vv4%JIyQVOsQRmP&feB38&3R~;xu{aB|1zkm+#PxiNhB4`?E`w7FaxM^7 z74Xfw6W-|;>rUgOcXSt#o4`ff?z8pRAJ&s50;Ab>57AN#3D%B1tqy*l#Ks2hw(F3lex5S9@Vo( zrF6Q z9V>EcFohG0t0XK8mHdukRGfEtDskNJ_ z?5STXfi~@NmZ{<_ic6h#ALpHOsTH$Dyy%g$t0JhGJ9qIyZC)}Bm<;hj;ZOq#glTZOBeVNp@4+9zlyWrW#B_fEwi7D5SZ`P?NE(ak=d$Ky@Mos)MKASlcELF0XxiK5`R ztE&(3af6-^%0=AtuF4LsTTT0K@8NIV>>q}1zYQy)a^0RfjBb4L^Em5FOD~D-$WC+V zS$51*_6(aKJO1u5e*Z#yB7%IBa+!{r^XKm3c`n;tv^W*Wbl?P9 z-MALC|8Vd*GpPL;%B?(d&vJCWP3-5V4x^iSgv~%!9=6<+M=nz6ym7km2St|2`je|@ zd)+x1w#LKfaE8>6m2c;a2p6*2WfBVlhCL0xy7CwXIC>9CUWv&|wB+`dHOUOj9)~Zw zr|vor2(MH++MArNc189+C$cj*qVGlnTvz@kf85%A*DY{JBcrjap^Jx`Rcn)qSGxN; z=h|$8XpS+*)6^@PlWg-(4_wi0{9jLImZ zP~W;LUZ;DC7vak?PmoQjh@uZHOLKO7Voe0vBj7mNWI^Wp@t7(+6ES19w}t4HU5tR# zHL6LAxJ8AW6%)9>{`|fGHEH&hYq)-d%#>w`@;X{P){ln4L5zx-g5!FhLEJsK+irV} z7QBNZ)@;J783BtxWJN`yOIS&Cvq|gGN%lRUVb1AZI|Yoj=_l@0ImKB#L5A2KY{k0z z>RjeX&&%rM_m8`a0A~~r`$k(nfgjcKhJs$GKPu zuhY2Fwb6pu@0Z2hPMnc+(J!ASxcuepO=mGAtW->)4Unk{*2e^;BR=2+)3CzW_gR<|$s?#>02=o>hv<`Mi~rV8j()MATbJ6;EIu(p1?9 zUu**kMRN^GoY}?T3S7kSCmbk8UIFEdu=k8Tqs@gnfUYxlYa=CMfNo35QqwQrkCg`& zb8=neIz^&ZYnD~HdjWhv#&N2~?kERl%!xaQvwkdTM0p-Ay|7S0ECk&{vP3+UiCQ{r zy)se@Rukwj*(j+&0%4fE&E&>g+Z^olo}P$bkcvL$>sHZ=pPtN;rcwk|WdGJOn48;T zeTI5#$6V80BRj{ve$N25rs;w1YaZfVC7n*Mde`2aGkKiZr(Tmq?&poKZLs}Q0?9Je zNYJjx!F@tWWYeavYdQG@`OUYHu2S|*DcvH4xqd+T7_qOxrj>u_x-~V1k(z*{AtCyH z44u4F@||($BE>I{b@o5BEL1WmuAW8{!m35u&HK?G+4oopOG8e)!5f5*W^%OmUs?u? zQTvQn-81Oo(n5bav?8SGO~rtOZ6%A$z8odm>dk3wM(2?4lT~r{X+r8g^PPpnz(Y2% zZ|PAx zqx>CnyFi zkhd1&RG_}?keB8fLn1UOm5cH|h2rtD&gZHo#T*bR<)}ND%X@{y)Ct?8SiHHtU~^Sb zo`^x^&}i7gbaSdfH+x4_@cC_Q4rI>Ei>1U0_$VN|J`V3l9{5$uF-8W`x$SVAcw^JQ z;XHL7GjV~KSd-}U1xDa+wbt~Q1>#chmdKkJZdrAc3z-z->fv*%l&4}zwIr)Rrm;AS z-#frn!&bG6H?yW7LF%wvIfGTL@W&m>%4>gANt3~Aj1@75fg1?F_v0|&dIr})ZMARA zL@}_SCJ(i3&uM=OR!g*=-S~B*cxyqd!&B}Ix^E7L+1wxZRQ(wBriTRusWv#~cbzjk z9(EqL8h7&TK^&b`qFyOjr2FtSYGd_v4PzT5KY9+|tNWJsu$z5#0i+6@jigH8M(u7dTu0>tXBwG4?X;~AeIvaaR-QT z3c)CsK;$DbJ!gfxvgZY#AiAcKQ6_G`xYONMZdjNWfe5sYNGI-1Jxu3p?+ZA()d>Fu zG?O~UOnf`*2_`BNENR+0p3=A@entTQvhN?X;v~%xphNum_jz`4MO>(-2r`o@Mg{KV zaM(XIVdg}6Ne1NlNG2~gi;Ch!NSQH5J(Hn#++?gZUu~B8JZVw|Chf*=&CvVPDdufO z5TK`&rwWkoKlM3+H|*E3fkaYhSfc5 zUsjv|pwXPLa}iL&uaTCfJ*URS3}3gCE~YSZdG@Vza=j_H03*mdGt%T!mc@^Ehfi_G z?(Gt%o@w9hC(L(GygAWA_HTfE16s%P<29)q#xLr&I5s8&ql)E1=z8MP!z{2;$RLO* zmVG=oYRK;qS{1Ak(yZqi_p>_&wt1q*q|q{&?S!KVm_jYNVlj$a=u{Hwh)-857FQe! z6^&0mf8Wm&T@2+e?e25e2g;w{1fLij+b*8~E>Hxe1yRQp?7UV3NuHs9NfCo}&sc9`iyAwK0EBOsW-S=~}(0 z)hbKhY*C{BIp=g)qL+XzD)@xq>oIS2g;Rm6O|HYY%Qy?IZ>LsX+7mJ0fSO>>9{1gw zixi6a?ysIpjkQ6(-1(c}-y_Io_>P#mO*i?d$5Nwha?eOvYQ8Sfebr!R23(y z^KcRj9*ad15M%uXl%gysf>AUYR!pc$F~-hNGMm-5)grGUNmqqL%Mo5$`S9;z{>y-* zB5UTn4FGFHG|8dy#2jeM8=qvAE{wiq_L-``C@4QHS}t=C?M;UmOcf52aARdMkl{^> z?}&m8@r%tArC*X3hpA1(zX2sbz>QKj>kL;?p3gQ&4#t2=CZ$rr%@%~+aJ&zyIF#-f zzr(2F;G05B353b|c)!Qp5X#2VgEnT#1FtxSKa-~*7Q?37<0waBV#ngF(INeDXi|&K zKgDw!mo4i(5oI1AZ|I*FDZc2{0xvUM4_6+}+~PY-8d}a9n7)D>2Qp#Gbu;0n$E;0k zYWBkHT(-tl-<`Oo1`!^@wxawArl|HEr=H22l|=Xz5}-!2>z>Kv3Fy8bkRrYz*{18@ z;{-_i;@$e!>41gzZ$s}lK{jE2)~zQW-Va5$;dnwSZh=7h8bTzEHRcTHY;s#I#h8w6 zhB2KAAB5FGJ2Zq+YgvqGt2fFxQw-o$h4~QRRQ7pFmi5Vs7Ma>VK9GG*1@|ED!uER- zW&>>8>j_6AVs6THnLZlU?QbM#Hrt%~TKBQTj=w#^f2+j8*5>v-Z#vb@Zra8*mexq( zn@MZ~QXL`XLfXxZ&0~!Nzv{sMnc)U*`AK8i+*sm(CalBwxm3$BnM3vXq~#Cs*uErf z-zO?AOefZ`B5Af&kQI-G*>Sdz=IJFpq6Q*h8VntSG#X~RUP39|Sw$9}9g+&`QV~V*Et~+Qa%;9@H(%<+ z{vo;`IPL7|DpcF_YK5XUdkNhc21V{a-kHjJcs2V5uTW$#_EE>b@Pjc3XSO*%!Pyw~ z;F+(yZ%N7n{Ax2elo}yB*d0QK4`iJnwGnuyp`X{WF|aR>B5~A{s8!#RsEqP0YH>uO z2`^~tWwIU#geVx3JC9H$FQcDx`;a2_6uQ8{j!2=eh0&k7RIC z5S!mXA~QJe`_@dpUhAE1$Dfz=Cw)qpcGVg|BE0!0bEl*GN$;QH9pFyZ%{_fp)q8>l zy#;RwE5rmr;d=$s;3x25+BzG2#c$r0yLm%-W_s{NC~boKG2#s4T?kj^l0;5@_NYWe z%XgM9Mn+%}9ox~xTk9<7T5U;$r;$AZ%kC;(L9<4u=KcH;d*gmOc&O$fmVI>zq-`cq zQ#bw>HjMJ%E%cY?cY6Jxn-6t)1=7P)<`+UNQZ#F41rj1zzA&(GL7;8>fuTK9@b1v^ zO#?v{kUp;tUO1Ngh7o#`I{srLU&_!&Cf!!I(0*&hz;@^&&7M+M#7XKSSE7RUQ~qCC zHct-vL{m5$n9z%{J&%}CiFRcNbeJeu_xt;Amjx|yYEpgIhiYh^%#taR23yE3wX|D4 zv>poe`Z`S>Wu^ocBDMN9Mpsz@>P<3Xy$?7AMdd$+69}7SbMdQIQmPU?G2!-1C_Tw4 zQu*ROUiH2hV>ECDb^6bQN_uxfxgmaDDx|Z@HoT345e`XODkpqw8{d8oBCD*X-W8uc zJP3}KE8emFK9T=v!7ytMJnWk;J&Rb+ zn6h)TQdxg6%6$U5NcG+OXabo0z{TcQ$QY&LF;19YD77IoP2=~wo|-uA4o$^Mh` z&DSCv=+%o}(=$mW z{+1Uk!*gkeCQQsSRfDx2xPBFEFAmYxHax_lbJn3q_d*LVC@=2rPRo}PjApY zk*@b!=qGz602;>q^;?Y@ltCMVBV+&-1b~H7R^^CHB@S&K6a=_`&Xnxu<`}GA*7FrY z$FT0lVQNtwfZg!@qa48{LvJHOSpWmRHCllM4V3usEOm1|Iue1G-#rQ^n>fxUJ0)=3D!`b|(A_6P)!x7)&~j z!sqWG(arYSg&=~(_yrZz4K3C{Fr%K#O*3l~dg*O|hoXSINFqq=3cd?Od25q3+AKNw zl0+c7Od6U35;Koet=P0ark#YhAsU!^tKV+aDo;c(*|ezC}IAz zm}s5$U#O7$(67EY&mDGQe;V@kMv=c%9r4`aj${FrR!~3dkwUW`)FshGSPEm_!`pHt zP|y+uBR-*8r;DYvq@=PK(<2mIZ8ndCceD44nGtp*mgxb8ao(hNF|5F+u!%RxL3R0_ zmTqK!c);uA%$v2Z@u+}EqIbkXJ~ zC0LxyU^>X+k>v_a|9V*ap3yObR{8{y!`PWxTPm#XeJ;-jY?=b}wmiCmmrcB(4 z^Gy*QJvwJ*Qvy-Fh|oreM(2^%*F*kt1#E?K!fJs55hx)a7!$w17~d)s5coW(B*n4A4+Os|WTo#}q-t1jh_>Des|BJE zx=E^}_0&*PqLgoDyAv&aNLz%Xtg2ZOjFH03qwzE@o;3n%u@c-dyN8i~MIp;ku_4@>E7rpJ*>0ekoq`s)@rC)_*vvSi z8^y37E0HBK8_1d-=Sf_(_=Y^mAf0u^*37L!DBi-w(8ZUYdFo(UUm|HQP?jJ8RUQvYJFDi1lNIMkS=9ufS0 z{7tg#AR~yi$b759z4H`4PT=tL4?O69Te*fX7hsTFXR!>?+olI%Tx9z9O)d&p938WYcdM7XL#_ zY?oiRlP{Hh3y7^Gj~ye-^&zsHCAUOhs0t-@NV<6O?x#Pq{9q+%Iw;i{Z1qXIobU|n z;^%86Io2T%$4SQWc?dojOhCwKYm!0&0@8DbLP#btARKf3x#luUoLO>64L|~n3lh)) zl*!}^1VzeNvx!58b>q}es}ntQ4DKtUTu5f_WuJLdMp*|eF+2ijJ)^i2nHws_R&nIB zI=ok-wb=&_Q`x!m93A^Pvx-t@djK1G$P%CV!c|y+9OG~LcLEM4AD4g3a2r<)N+~xU-zEs0f?aM- zY>#N<;`qPe82RW+>u=;Xr9rouY}=8pWGs-%8GWd!#sd_{j3l9{z*XrEzllc$Au4F@ z3A7V30l?(MtZTv-3|NXd(@)fJ=6TPYwKQ>C*0Vk|b8WUxq8l^&m>D|WM}<+x7I;m0 zyFtxPM<+tx+0)Vb)hE!5v5g;B8K=WO{xaEGd#F*Vi!j@um5Yk6L7Dffec8D;WxRD3 zU@LG3tB2 zspzFla|McSkoNcYtakrxSw8LqG0p>7M$Cr3c-G2EpmaBcndN~N_BOa_?*JV|w@PG? z7w@aMfmaKMrsW5_Mf=PZ6J&=3)C!%41Xdnp%gs*e@J8~M zuhK>^wDmw?cEX~7(;5fw{GiI6d5=3U54syGz7rmeo$er48S4~ZMxC}QC`v5Q4bmlj zAenq@{{nPQTW+{e)-(U!>^^Q8%*Yz+8q!c;7%y$8C{N&Q$o-J_lZ9m9jV}Kx(+!~1Gw27b8 z?vF{)O-YQg0dft`iqgD3d&i{biIJ!5YQ+NfKdfU#ZjPI!QM zpVc<-QgA_P`TA3$iQa&w32+C!ZCEmF#fbf@AhYj{iKMA|byxHKx&`2|-W3uX z4%svXpUuN9V>|GA6B)$2!g#(WuvZkTPQ)P{-TP?8?Wu|f^{J5d>B0a{pv*VbeMOYW1UhyRH7a52G4Ws&h0#f{mX9;@{f} zQH8?>HHt^m@p8?A+C82{@Jb;%2vUyUmlU3L zeFIIqOt;kI%+ULOJSGgUli+c6SH zEX5127TDg^7rnmXu_tDAI{K*0&{RPDa!go z4bv%-p0BCqyUrAQnmjY1BmKRZXN^-9^4Nl(wAfEdikh4AgM{uL5u6IGYpz988F^Dx z_9uioP$x}ccE&METsE#8$`r{YnHm-B_Q7XvoM4fRC|t6rRNjjlY?2rg%czU+kPEg- z*2cE!SFpUB##lK=Rz`7c8kJ7jsk$i$Z47b6Vh1YCf516`RWAKk<{4EP^JnF6lH$kX$>-RckbZ z5MK`Q)Gkk;CeL}`n3I0z0CN@-Y|wjHHGeWCOXo-8%S0A+d63vyL8ysP2#3%Y0w+KU zaaGv95G2_(S5hy&;K9p1L{KOnbaYFqnEa3-pMAgcs&JFxe(?*{Bu#&6pX&5)Z4nHX zF6{&ax@?Q){Z^-Lfrha4GvcIPgbfo8q_Jz-T)3;(Myu*ryw@F1qqB5HOp@@TTzB8R zDr8sOb^GwLeSVa+X1fmBMwnlheCI{yf4OoxTt8_a3!R1aJv1dZF1EX&cYye_#Bho zO3ZG!|33ER99yJ?yu!WgcYqSp`Z-(9oWTvOh$bFA6gMnyVEN+$_QXM|F`KK@Zr5*U=CC)2M? z$(N{a0bSaho?+|#1Mqog`ks+#9!N&~fZ4hPp}^kQPzz)jY+S>!^TAo#|UnJ0yOc=LY};=fVi*H1Nz;G6SHL9%b3 zautyChk~2pa7>|PHl`FRaZ<2xHlckiv({S#5dBlH2zNk-bO~tar6Y+faF4@qh3a_gRJ;8L)kBE8{#xK54$r0jW9mr$Fr}USg}affil<^yo>hsZYnlEI;Zoe)UPOXiz?q991CR0@U0Hsa7A?l}lD$%>Y! zNV1#Iv9&)gxsB(Y@2q3u%=GtN_;&>2??0rN=RuT&aqn!sLBrr5^lOi^lA}BKmQZN`1=t56WI{pyx!!s_Byx_NJ@ZdpHQtL{@|Ub z5a1Z?IXM~m7-nnb=_bKS6fS_|TCX22(D#eom|`x;(Fq~R*{_bERL{Z(qD{m9g}DFc zc5ppwCd}_a?{lYs{CK5Mk7{p0Z!c=W+mc`$O~)g(uv*@1<%68TcpGqy(r;(}^;@{? zsCBDHFDd>X$MxS2WPD}<6ysUS+nO3;KLO6b$I&D~NO&&sBE@A%mF@#6AJbBL|7gv-{nrgvFBRop63 z?}c<=toq>&_ov*(UOTHf~cDV{YjQAgDOw|qVDbC&|P^g0xq2~evnD5%P|G(R`s zUfsP}Yu42>AI)L`G|vMctwhY_uKqFk6z&U5RLrCF>;IbC|C;E_>xsVHUmKSr1qOBO zo%d6vlWYX1aK?PoW-d(J&-5qv%6$G@rk3bMC@^9?^U8$qzjC4ezTUqND#ddRvefx? zeRXn-VV*rawYP}kJX^2KvWErI?=G=w8`i0|aV#xrwW-E)um5A){yxn0mO9e_N3+hW z^#e`P&y32NQs~wM(z>Hd4dq2#dTtFRgKsL)PH8`vaj|148Fgnzamf#qIsFZ={-&S* z5h6c*K$o<6zPU`f_(@Tp#2Glk1aLH~o{eOn}m(ov@;EIX<+zAKx@9LC066OEcYN;-1;J_mIC+LTV1x~Vk z>=vwEk2t#X?GfVnSESy?)lEWcdSAiBJ~eJUntC;3Pw^I5h+>T(EeXRd+#p7oOyiVn zcdLKsgP)}=Xx)g-C8a;{1mv6TuF<5sQ&ZAM{>Ygj^yULN@?*% z6@mmfWv{9l;x$_8{^Kb4zq^km>~fAOo62ZpLkqkB=T z32K!r&5!Y*QLKpjeMM8yZYcoAvUhJy>i`%89Z3r;9HlrH`D-H6cW~?f_@2Qg4?Kk~ zh>x?&wJm|v=jrDyA~e|E!0{GVIc>!=J3)bm{FbVM?ds563$=puP`^`d^ST|)-yV6u zC>5x6Pyd(Ve_vNJ&hly^muEKpZbMk5C#T6p8j zkH0pok_k^)WQ_UM+y~1)>Mp@jIe{O~5ccd^O6I9wSAj4GD~Rj#enB*`vmyPzB;?=n zvQ7JI3R$UF}1m#fQNP51H#aOGEHY$!&j+NDr*6{-GC1N-NlIu#+2 z#i$t7t6%5^!wHh>&*RH67WEY+`|Y%vRO5e*=FzZLP$3|r;IQgV-i7}KK|M~>3 z)wYNTM9L^-8C5Oa?K$nX(pM3u&lwKMdE6Kr3;Qw@^9_V3=vmYPxbcZn=;_Hy?-sp) zVm+~{;g>A#1z^eaIDYkvMv0k)R(J6tl<~H;y#Ht~Cz}owW)UX68bccBfDHjZ;UmFn z?Q3+g0(<8Ao4uDgqYwKsy=7_Jef(0Z<6~)qbhqI(QyG zPCCEH!))9A$Qs#Qd@|UJY0`r#b4f2tHeS47=2wu)e0l(ebQ*N%TU1__E}H)$W>xNh-Bng3#j&X z3E=Uuzq|c+8Gqn+q;Rdf>9`C{%phohcQ=qzW{5`!!Yj0$K-%unsc>00C5nZAPI|$Y zRXFdNUwLub=>>L}c_D!79sL)g(G%l(1-#^^$0+Zr^**!%hvN8zgK$nV03*_Y2mAN~ zN9!Mezl{!&Xtv&_T?t1N?zd02$0&=y!c~z&6$9cbV*1Mk)Qk62>M@gM+i=x3L6OY^ zA_|}Wp72%`=@HC5=kk)V{t`amIJC`t4eS32qt_CWMe6p^UDhNS%T6l#3a!eC)aGIg z)i=#s)C$jMHqU+AtoQIB|A~W(7ocw$w4_59)GfGNPCajY{OHFh0soe2pjN0;%>ZW z&t60E3Yx*^98Fs-QIpw12DsJuD5)27e}^@;YqV!e>W)AtydS`~dp;{S=i@`R{2d%6 zvigy_H;$PgLoNb25*7_#?M)Z%h~Qxkh?LOBPkU!Hbs$POHd!QnSLV2)fzIQh;n}h@ zBk?>_{)0;+ISDv4QYubQ5h&d#%_xD?F#~h+RuC&$R&nO$#gCSU50a8Xo=@MUl~?7I z2eK3S7xl#btjG#tj9QSy!sBdX@w-;KYzL)6O4#|h_@wQn=#&+Sp?u`~BdyN^sT*5= zLklUVYvanP-%86MmJJotJU6To=39=AtGJ=e7`~Srq)jI(l&%!?R!=iz>t2*`z_SQD zd6^T_$xpSE=W1h}sZS>AYRL#HP$mVI5=&bZ)Oh}ccHDKyM#iR$Te3EA|3hL8vu`6Z zQhF7}G&4L)2Q;bL50{IRvu~?#p)a_AXvLIyI7)AMR>m9c-Ae9G?bEmvJ!AgS$V=sZ zmSmXVaYN>8iAro%di;wW{^irZ8zDpW*E(D8G$VjyVgW5`P4)os31LxioXEQfD+z#Q zHW%oC766~fSaIy_Sb5CYgz0H|KhJW{z4+~j*YhKz zlq?vac!%F9bF?Ubn8Ymzq-OD*^tMxo1y}&I9Z>s)u_-mM(Dub6?MGWHPwT&{91CoF z)-Cr3NF`nbx`g_;EqW_TwfiqoXZ0vUeDh6}ci>7nG`ObNbN`RL+0PY`?Yufk;aj?@ zOkeXiGsqcMt%UcvKX5Mc1tyO@H0Q5kSF%a-z-KR#>{65=%C+{nv$N(c>5zUV%>s|{ zEIO4ASmvITz&%lt@z)~xYaM$((`tuv6QFm z_DOc&@IB+;_F#Ij-=sGmRAs57JC9eETS&-~*pX#1Ayysm3QFR7#b%1F20NjxRW6`p zQomy2@5NF^ijC!PDqrq#Uw~=pW5q-v_5WWLnlzE7$oDAopW@p&VZ2bB zuM%V&HK4}UhM)^6h)zIxK5PD9l;kF@^yVMx)S;SWDvo9{t=)0&J=+W|$xd35XHf>&LC4F+y z{x<>pj};|(qsJ>_03ei5%2o@$o-#rdv0k9BQsd6+qcT!UfR{4G& zq@ac68JksnV82rS>D%{0UkoMbzOqF~b;&R`C{!r#SR-%CHtL;1K_E*OR%b@Z+*`3a zH)>UnLcsgnqk_4ggK$h$3Ukn0T;g3(=FRAl;|lC~#Q-<#YNg}VhvbMs^DAB_W5N?q zN>blrqLa2&o0f-O+_aKvb+bURY|O6+QMUWzs51IT`wm~+N{ARZ-2_esJh&lK;p#Xw6%%(V%y$4M!OA6(Zb7?L zd8lBn$6YU;Y#ZhRw$Y*RvYAyaEP-t?d%Rlh)aMsH>0_f z8i1GR<>qzzl2-D5R^xJ4)mspyUzD%%tq5*xeF&MdsXG0YmC9N{_F6%RV3c&?R)nbPUL%7Z*6^7AH z45!w1mrX$atMJ-s%U;{pIqYpB*LXFdDLHBbJG5#sWo-GqihzLJR3W>N^+YhGIlNb{_xV z2TpcPq_GURr%k;S+#>{OuMF^v^VGaV$8W+lSMPEE`WT(4>2119aS*=N-A!Bq85+A5 ziU{&BM0&>Kq5i7y5Kb%3d1N;7`2e54kyWdAH~~rb+H4~NU-GgN*6rdso#$%2Ffz* zeS5|bf7&m)Q}4JgoQK5&%E2#LuEnTu8NU1Bf&rd6vQBKhUw!2GtAho)(xof`Kj*B( z_WnM(_;*_9-yC%jDB#DsEKZf(m}whkRnObpQb+n~AZ}&f83+!IFRVxfQ}t&14;KPP zjyV_wq17+E;Y}U)z?r=8f?i^4$0#E2Vx=8;&4VE&IPP^G_Xq2h$ti{7l!IDj_Xrnd z$OzjlZP2XomK@S^iE7R9@wP2-;wrtGg%MyBGVwLp8fH@yx?A#2`L)w(=Et6nd&a4? zQHlHEdI;>gN+N@bbtlbQ0X}AN4!d$UX5{SdK)VIdtvUe^u@^v@r$!8r$XikCy`M72 zIt7E!WWQhPA0!heRYo<|U!#0|KQcr-OR z|2)3`58M43{#Jvp?=hUa#VSW$X-oxt`D#vm%<5VUIULX_oKYo58^txfD-;Oe_a z4G9t!-eE8js!Ry9PPw_TB2h&$AQ8%v&p$LD%43d@Do zCi~#ZnHPKr*{&UE8v=7~r*iK(D(3(N-oG7S8PBU4d*|M#dbxpDZS#Qlx z4LuhBM99~IzU03kk z`)mtLu6(iGQnVcll(JAcO>^ww^!Io}z0GHEKsSyWk}O(8=naqXqly{~zKZ={Qkehl znk7A^wUN2VTBI;wE)&P*m1jRxkyx;w?Y`e2u#4ZMPLBZJ;t#GlIAJ4QVb$KEHVCfy z^-?s|5W}L*^MA;C%cv;3_x+m=0YOkgq#K4-kS+n~j-g|Q5dq;Qr9(gwkQ!nX7&?@k z0hC61Xz7rW?(X_ueAoItZ=QFoHH(^S&))ky&g1x;c*n9&%1k1U{~SaQ++aKI13BcL zHDs_r671Yi_S!iyefEz0@T12qGvS_>ZUHgivxSTU$bful>mtKP#i2pY8&V9xU&w=5 z=mpWXB6zPEAkBS3ZKTkb;mcIt^g2#^`E1iN_io$bl8(rqnGKV)t0H9s&H8D+U34bu zFa_LXH=OOyxOc7mXIIpZ)*d z=Qpo__xWu**vTd%cR7jUin~4g)1)o>^CT%x%ny!}&#yT+x|vg1kk1Wxjn#E_A|DFZ z;)_WTkJXS6yV{(s|50)y2p0!_YLP2m0r22oDR5-b4y*F_kSGS9PQgUn7UVC0|BKi9 zUlaT-QTsh0KgWJ+!d8qdmrzL-+;ddSD6EfrV_IwL{fCq)7@nVE_C354*xk5D1b*3U z_*63?Ox}!jZ$>4&+K_nmVug6>zYAHBm0X^vhuz(D67G;65|D~6W#L0|D3f6DHgE0H zE`e5&(W_DO=!Vy$Z2+@)LCqiJBN zasj|nU&L*y^h=8`vx`NEmisqcneTI!32!b>l5E=|IVRF?CXOIeJ0^*`^T1yw|3z}M zD6#KE3~a4Ue?fTBQG#Zt^YbCExBMb|xxVb*EwNpZf{M!KuiXAsA*UqUftnlkR}b6^ zDa!!vDbI0n&dtlg-|l3_N<_E3tDoR71+t0kIlH|08n;ALC5V$eM}A{vFzf4q^S$nU zT@?=Ao5Ls;BhP1e2X^6P1_N;%#X@BcZ=P%v=U=1^hh+De^2k)1V@g^7#WIhkhaYlm z-Iv+&PI%8vW8A?cW%X~K=f2`te)Ot)fAU@Z&T?*?EN}P~kOBWz0t9{YWgQQBRB(WB z5z#}noE+wW*VoKfdE7_JPaM>f^F{MGHPe`TL1;~k!*yA>;TxTjVEvNWLSxeX+I<-0Cczm+=+*t;t+VQ0Ppr_Ff`%D68dsAA--Gvhb-Xz${c9ww}$ zm}?LDM3JSW;69X)_qHf3Z#Zs!d9kz~z5MBhe6S&PF|9B zBCySrChPdzP6$GPI}m?Dg)WSGV$77L-qX zX1rFTqbTbbX~^GQIQjIC40K}YEOBe_SIzH%MH@` zfCuI;ZBtYqc&YtM#C51{01E5X*h1}i#j7kU3w=Dkv#fdM>FUVp?d-j|Kh_kwDqGP9O^ zN6)}-T@Lx-JRA`zMg3wT_ z)6?^92OScFl^aubz^v+Sk$-^S^GBVq;FY}(XHa6bQ}1~yN3og#_*2PEIbtotG;pmz`%CWKI_Xa`c< z;n=2!YQ!QI1|)|(DJM|OYSL9_PObjbNb-9{*z0o$r7m|wXH~=Y zx97DuVEXNfgk$=1DMaue{N;Xg>!iT5*`Lmz0O*s>_+QT*$^+bgSikp8L(DN@IYE83 zO18d|$3O8F?E}#wdGXL1fa4@8#1(i?q!pp;Q7n0!@Cv+4CF1ao>AAjqPL|i+oD7S~ z{{q2*BY9>i!s?rg^+MlB*60}CFND^fmIH^rnuG*os9*Z1FFz=|>HO}uOc0--+?l?j zb|8TND_s`8ePosA@q}+IHTi^wQn$hth_MPj`tEqw0HEsCS1$KIKD}mP&)W6hV3`WY zw<_KKt73Z918+kxy!n_%eA0c)<(A<6att9&A zT?U}1vA~5Vzy8Kyq$*zODIUrE@Oa7RpqVa>57u|&3CXKpU^Z`z@n>ipM8E+1Gn9($ zC?SeptOI3jLnXo|=6JKK!6(<>Tcid9F;^6WIfV7&IPS7YMa<%Fes{N@SICX4%t=Y| zl8b=+I#EoD8Iw(8w83WMrlhKOO-+xwSr=x8&rn0fkn1;*R%E9{mi|&Mdg%wBN(8s{ zl6ULDHICy9{)4~I8>ypUm=!2P0s4#Ae8>5xjLxs88{Oe>DqUg0gS|rHQ0|NNL=KjV zw)A3v5V%gm<{1*wv-MY^c^TP*!{d-Z$0t!FnkD%-dBZ?D`j`~O) zvvxu6`vxPRMG$EuiQpMpQ9PX}1L$;z+vvd*&|lvcM)!6eRZi&sbBefRM;$D6 zajmRXRknXl4hXxkb$Zq|f08q@C z@x4?uhht4QPsoN*etGl1D>pFyJ%5Y#?)6MN{70|U8PO{W6kxMn z<(a0}&TRe9zd-bPxN%MLurIAJY0KJggX11B(<4Nr)|I;Fr>Fh*W^(xoz%BPwb}{mG zG5A;<23JKhouiL8Yt;_#OgZh-vZ?tbFew)8zvAna%9{=k=R1=Jl7FCG2FD!b@s1I* z$T51i!v}`rphESq#Y+eATknxW7&y3#N3R|4pZsW62N5mM(oU45Wy({JF+@$bcUKXJ zgTH~opPF;Vc3@oo5i3(0Xh=wDNXSGZE1F={QL}7M9c5mGX;r6jji0@(RyCaAjLw`+ zMfmq>5Q4=j1;2P2iu3>yrJqA7pVJ%QU`^pJ&Ozu1w?<+Nar6N&;ly%+oQtyT6KEih zwz6L&V-5_Q@9a+by(}exZwc54Tw;KY;A^bj^8mdXz^`5N*S&Xm#e8o}b>QNt_OR2( zH5vb(-0iu&g6|6#%ldNWpK-MY@xHAPW??X`CQ>M?tj>_nChp+Qy)9Y9a}PO)V^`>P zUoisc52X}K?QiSIPI)g@bJPC>WG06mAS~)9FbPzi0_4*B^6ve>0dLY8)$y8;WKi9%=lNTn z4&AAJ(t;#yIq++2wQbaanq)YZY|;bOApx6lUUz^6r}Jb_PzbL(`m_C*p4DuzE9ft z0`5F#ipL%{ELh*Q9*^N>J5&Oo;sMB0nFun95X*t z#3~UYfL40;>Fr?S*!?RW^Y$*oL47hGlcO$5lJ^)SZQK$sB50MydpX{J+2USPmB8+h zUi4)W#nC$?ru=VLA4us1F=td&S-t=canU?QeXh7`H5lv(l>0Yh3bVl|0wk<5=tj+U zDn$(S_HF*3!v_yLCFC=H$%L6xss8E6)2(tIliA?X0I}Ir1X$U2zIHW3gHz`#NE!Bo zXSDooP0QDI_$Bzb3m;mFZz9O|ryy`L(N2=F%s@Z)jk^75VOR5|Put3Ba}8Xb?=4?x zCF*$Yod*~ggS;X>`!i(7}bJh=zc zq0Z)Tp{{po&Ug?hnsGm-^tXf+z$mn0^}fpWP7%FzQ-Nb$E`DTM{#;)UXL7jG6OdfN zp}RZ&0j(+e!g2vOd50R@oxx;G%x7A--$aY|mi>5Vz3#EUqRJ)5GAJixe0w!uh=6t0 zCf2H6v>_b67LQeHVw5c&UlI{+u1& z%NzoCpp|rpDjQxl=wmeQ8q%Se=f|BCi$Z zb4c?`O1L|+dr6~-U;n-z*heRF^Q_IG<&AI4^V@6&!9X#q$Ze4d_(77l8XT^qGWe6^ zTK5nsq@jT+ReC7#N}zrll{aW)V#zV<1D5-_Z8_d*Lg~=Twv^wisL?Iv^o|GJFJ}Iu z3~ZlEHPG2QrL|Fm5&PDYA^ta$hi}`zFW!FTZ<5)-^n)2NEQ13e5X1XkF2zJ4b0RoE zBR2=LDKpMI_Ipk|i{k(Nd-c1n+X>KK8lQPhVb!jVHg~Tv1&@FKmdOX8s<0jS{}c{( zYRj)^M+7UdE==jL>w5j|*LNVt<{?voCXd+TMuvPKYvl#H`qV@6%#f~2fLi0s9A7>E z6!cN6GKNuO>bDt%GBpCEZH$veePp;}Q~Sacd46PZKo;IHx_PB(GxN{icplcX0TxjP zEusD4$_3C&+U-TU?dcyKf6Y(bx6WhsCWe8i_q+Flw#4q8$Wnj2dex3=x69)1eShc3r-^fxET#bbJ=L(r57P<+$7 z)R)HG@?d@JZ5(K@6@G65ltyn}9Mcx5dRzF|h zID=+*jq^y2yh-4oPM)yxGjhoAGNu!OydEZWbBMZEyGQ+Y+o@p@g8{HkEkLGXMc#T{I`%ei%5X6ybkS=8i#KY^$Vd}p3zgBK02*%ws7l-{M6d46M|aFBc9 z6q3{5;`}jmSnwQL9D+_KAbScV+Ww$*mBcC@u-s>1X`3P#N%p~ES>y7|&d~|@)kG=g zCslYg_U~IAND7=Qp1Ip&8+g4Q*D*Bdm97khcE==y8-JVh{cfc4h-$Ltl93IaC=-U@ z>YYaK>TRF*rJIdYIT>)=yEOhm7?8g)yh$fA)CSu+Z^qg&b*hHnJWVvY{tCTBzp_Tl ztN}rchvb?4l6g@8YK??SgC4xyFGC`W>B}fZl`g6L7$2%VGT z<-z=x#$-g_F|yxfTScn}Xb`7OnYn>qS)(!sh9x|oo~ zFJ6CmVQi^17nF}LIL1ZJmtOe67#^w*&xRW%3Z{Q3Z@T2gFdDEDy~>%6`wwyG^i?gw z8C9b1ZA)PJT08Vv-EQ3$B}(arE4FRAaXeu}P*TZ{--I#ko~^0@-8*ISE86xych!(3 z9Dc-cF4mQh#>YF4jgXUWmzy2!^i~_|gpzziS-nhwo_5ODB?)FR$3@Gc% zvp(}$-6cTNMGg_{q~twZ;QvlQWXAJwBI%>KpicKFaEzN=ZqJ%z-Yu?#dv49Tj3mkh zUhVuuL6`3{K>7X}?!_d9q5d)pDRAn9K9( zbmqRe(Itq8LT83aR(IbLRPm1G$)qzj8BXWG7}2~U?5a~F&(nb)j|c!qu~+oriSzpz zOWA;5VgEXwQ`0i)v69@Xvf&Ml^AXy8FMC4Q}(=E@*;0d zCX~o_(C~9|b2OBSC#MpWEfoD(BK3hgI1E&(l%OS?2s);yCPsF+J?Q3ovIw|J`NjVi zym+%OE-K47uCgzGuN*max1mHZXuo#G@WloZh5228EYyu`4>oWoc zHtGO)r~*@dTct^Z-gCaN+JjEypq(P3wz&6J@aOWpw$u*0_^hXa`%X>$jml~Bb_KhX zV?9R=2!DeGmjiDPV7F>Y0dsAjbmH$Ue(a${jN@IrCT%}fZ_+!Eu4LTg(yRd2IA}#4 zI$KEWlh|_#Ui+>uTBxR>&jU%rm0sE>XU`F7sH^6 ze_m3*TbCafDyJ@y!w|5!=)QrW;kfH6{fPr`J$$Nmds-W_)e3fTpN=0nNX1PjeqK-;3H*^|p}rt<%^7753-* zi(#n9Y3--JU$-y5%>otF3Up2a8=9fFhm5#iZ3WiL8E05dRqb;Yd?%0bH2&BrNQn^< zgPq-st`@)M$U|tuWAIb?u{C1MU4S^YOnVfu0BRZO6|nx>!9&<@ybVB*rIE>-UxTmA z0KrEpS`$b}s%uz}pMjj>IF5M{T;n?YN9{c^J31|ml<8>)K?_M1T&jjUsbgh}|GV zs}0)gt5!kKix+ffWqLln<1#wNa%Vm%Z4d}xKdaaX#>0MBfl7hi@;xXqP|U=7|C8dXN|NfjXtsm*e!o1h+cHN9g; zJ48v)@Imcaa5?*E6{dc7teO3tg{j2)i!OFj{Clb5q%#A&09c@UbI?uC zrEp%X6@|k_xFTvmK%GjRc?e$AR{e6~wHjl=_1A&o%);ad$C7~D&5|Ylqg&R)iRN3Q zNK26t!vftai{P6R4)={BgSZ3|g3NCL5g_sN13*7il>ml8d)x$fLt*NGtyQbL>(5zi z;)ft~aIs)FSG?{UFb4^+t-if)#P_b!tQoQ>?L=1^K;o?G;Y~^#Dt$M3YDvq;yX!(} zfP0HT1TbI8<3+%7`o+4OQM1@m-G4LI5{sDvGT?6Q4S6%$84jtHObyJU1J84vE4n44 zE$9^NoXJbvf<6^ZGZqQz^FE=vj)982 zou4#Aom+8;b*y6k>D;IFswfe{a>Wsvy#YCb^{Qt3Fbr4pO*Ol${Sna5PAmE8HeVjG z*D;>dq4SVI$W%baNp~tqi;YSTz=Hizddv1*5m?E!8jDl9%t%`ODBxQ}f^1WCt;Um6 z_D3*w#HDHcym7zPFeq$Q!kpG#%zFm$m}T`M0Fu z_yg?2lpmSj4FH7FS7ea^q!=#vTrY2J5uIlmvLP{M@%`5g{3%x*UsVB#Ct(6SW<_& z|0?EWkO5oyKF!@r^BKqjHfNVCQym%0#oOVgsJ~hKYD!^&`%RKcIUJuJkIbvon0biO zwE7q-`q^}Q%sZ01vL_tbj#Hm<(1wbxTzJdhi>VY7-f9;)E71aAn@#7K#9-lVmLJo3 zk@$r`s&)!OH$VJy`sHu$l%hWXLVE(0YCevJW||JqvWouI`FwfrB(Aun4TxXq7HabYnyEc1+|6bC z8Amb`ITPDD~^nF#wqZNUk9&K<@;a zK;zr`JZa-Pd#x8*NUBknKR82-0d)_s@wE!y9_)NW{RL?x%i>hJ{ZZ!U7_W-p*gOu% zbNM-2V>sXp7|sr_Sa9=taWr~>nlZ(qX7Il&jP+oH^hi1;iK`d5M1oU_53jqHASbmZ z+9-{CM~Cr*7ud^@?u7#!G+euK%aF21g7#_^@#;>Ka7m(C26P%=6-j!&QOtUS>nM8B zjO^yp3FJPy-rrgV<(3#5G7A1+w?3ykb%R`S*W&)OyFNd-uWNSN1=(mBG2HJ$@G zd`nFU=SF`@j9$E18%1j;8IkV_oLygNUyKN^$Y#G>l;(o?gxcWJeQ$r-6#Qn`e%NV_ zj6LJe@yUdBZ@p+xn&V(`VJ0sw;X{yC8(>!A+RCg@+X!HXNQO2H6M*lFZ$4}MN2y*a z<3{4vY**Wp?z^-^MQh0S0zLyL(QXWmS&X+NIlLBZunbAVGTOs6OLS2@H=yNB9`Z~?X7-cv> zFo@~Ji{Zz9qhSqXY^}qV;M5>{g&}o_*{83x2ZfVdvLZf}NK+x`s%SfpSWUGcJzmXZ z`#|xgnag42v>4d=qt^EH&+Z7M5FS>&YEdTXg#j`4?-DMtq_Hq6bM-%V%^k?nCPJCd zr?S8vSX(~Eh)HxEKd?VD5B#TQ|GAk1nCES9DJ68H`G-hH9axO5pt0;P&w;>HAcWtk z3qp*I7Rr4PZmfr^7CCOUOw~zX{6=Ce0N%6vWP@(HXmSai<55MroMcxA%$3jo;6a}p zSndh_(`3DY6?ES;uHwTbsE8*AP*0+dSS zWc^@f{!uQ_UB$bvzgd;kzM1Ua)~8p@>?IAhqTWmD>P4Cvt2G_NHZ;psnF62tPAOuT z*S{KRlP`S?gomr2^|KNx>}QWM53sd@mAW5l#HA{Kw|!UVy~FF;aG+urbWaYxO7{+t z_Wq;fJ`e?~Hj`qu4M9*U3O1eFxz{AuNHNwL)Xcu8uLXviL3;Urf&znjaerboxBB#0 zlaHg1PtF6HsmS5>I&`|NM9{sUi(cy#NNC@PXy0h3^da%QwP#np$ong*q^+Z#^?Wr= z4$J>yDRmdC*Y7Sz#i7W2p9CGbTlR$6LLDUg67ljIE2;B^4@vrwK$Yd;KvN@~2Qa(q zO2w}i00hQMgEVE?h?UdPM4^ha{q}z?O&q^E2sD)tLs5Wu#-;MKBGf3xk+=-?yFV=` zo2xR^g*rG|#@Qy7L8?88c+yP74+JLKmL3Z$s<7$UB(|7pN;v?dT6mLKs!ZjeXNkv| zofJZ#0uXlAQ8=;^^|546f%=rFDotzKFA@Y0Rec{oJG-~dEpryN4oo7g`HKE;sWUl) z>9pNsP;C$2KV=|EQRRt9fScF|@P}E}-KHL0tM3LMuxvC9JQI-au&v*?(= zN8Szw-?H6h?l!U~Ic?5ZV+QzsraMItP?C#%F#HfK1Bkvj!KR+Y9gt$L)a09*dDWV! zH(SPOkxUgUEUv!GVpG{B^UL~!A-v(E&6+F=;Ir5zO5n{IkM^_OXZqcpBYptvs5K=C zJ$QPd#*Zt=a65iIWb=xK}s`c8;SHnQ{9XV|H*+b zO#j;Pe8AUW{ojfqrp})){`<95AqKt<=ePs#15R<(Q5@DGsSzQzU9gEaFJ+Y;SK?tw z1#uibp7F7vsz;i3!M4jIG`;2i++?x@gSG(1PxU-Xwen1O9J!GekRhWefo%k%(%;(* zcXd;o_`977lw86&1nQM^`s?&;Hv2Nmq3xd&Bo-(8E;J>lgi91fcmM~Ji@xP&b-GSx z@(f|ZkuNHW5vMqe-i2YInF_ z;DF)8Lbq)De#V#2oG%xT%g2_R??1}E#+%qT)2C8ABB~<)d)1DL$$P}6(p?~Rs*MUC z8Y&!tgOiS&v?W&HuTpB1w70la8T}zSGcW92;~U54sx`UNYpbT;{@_@<4Y~e<4jldr zaJo0#|3z{2tfX37sf0N%Y)XOQAN)3;6yI0}Rdg@Qi~M(fll9>tZ(u>%GmnKo9IZra;xfR6 z7%;3rSc5(abccU~W%^Ra{BOG2SI#oiz^K0Fe_GUK zmpA3N6KZ1DMrXsOgED97o?GLk&I1ao*}=x%X0L-9&sSd}UJj}fC%jy&JU|Un`=>0w zRk42a*kdmB6xPeD$VpU0RpUHGay`W|5_{eD4DY(5u+uFTbjY(s$4VMj3>8-Vh7s zvJE?{ZmH+dWq;CkvbFjcJ8bfuzDTqU*wvRwc?;TilAq0{-2ks| zCvaJdnrY8Fho2nh2j&P!zn_M`8U0CG(Eql1`qBD(B2%Iw zQwia$kK1uK59h39{ah1?`}i1}RaU{ixB4LrQ*?k`zzRVMZ#1u)L5Z9_^S~@eG4%p2 zfzh`yxL+h*Um{Qa!S7N8h;2W3Z-hiIhmi<;+8n9AdVm@#r;h))cZaZO^_6Y=NUlSK z@A6eGk;5%PnL5E|K2ZE(^OlPd&N058crC|nudD9{%$4G^4C+U-K`|@!-4l1WiUTrl zi==@~9peMcN^?~lz4=Ge%ngd8$^2$Gd{7@;-6N?%x{n8F5q z4`jfhnxcGAl)rPt&L&}5l8pEaf306>^*C=vZp8e51!KS=aC!9X1|o;50;XGorvj5K z)#pPO6nW}`ua}sHUHaeUZx^vvKa|NvTDW%(!n9egc>C5R0r7POa4BiLvUwNRxE#KG z=L>x>vOLQBqeTA@Mqv$@t5l~N-9(pq>E_ew?rMs1>cjy>Sz-Uj9&#gBI04}j$mrSL)*JTKE9&hjFS`R7 zl49!V`PXXL!1cG&!=>IyjfzZIUzJ}CMs4(?B*TG3_?4C(Lb@8)uQJ?t`&scZsI%gG zcz}M+_fJD;3Migl@+ldoeby|)bPc=Y3zSsU%CM;X)cEY+dR=3L$Aw4^0gU@*kv#a5Rop{BY(;JR{0&82oSL(X4AUtr zUTA`GuAbF2lFHr~UYy33LjaLMz{?kC??841j(Q5{U3p!b4+SNz_uE!-qJubDhf!{s zRO$`c3_jz;sD9hPBz;fYwm^G8;1{@ZF7sVWPjjeikN8Tf`fFeUjWQB^k7 zlqf1Jii)e}zH9#U2-q;SQ3GkDY5UhK@Hvf{BqM5J68rR6&-1-2OzxSqVt=u>neA-y za(gsk&wXpVPJrEc-xe@myh+-O2Ju0XvKkJ#MgrY=z|2uj~;D9 zZUUxv25~$YbjGh_v+v64cg=dJE&P9Yt+YS-kW{OKj3zM81P&qgwrGUU3+**I72^nz z`QXEhWeeddhX8F6i3*>BdnxPbckivVTY8qQ1}-CQT$Q7K-Y^@8)k2vuUN`}tJ$BQ+ z)`e-UkVebHmxfpT0er`1ZID>GcB9+21O%HU3ghly>`HW zaAlAJ;wK0uIdJ?9ubyWep#;H%t7CxSEms@2kSR9WoC3;p+FH=TyNs*1Fk;~|(MG|L z_lnWb?#h~v9zDCzn44|JC-uPvOcrVW;k#Gobcwr*=+`nZ;L0%eF78L{3!xK23I$_! zV{_4HyH~M_#eknb;PWFPTQ%wH=AxL4FQxg|_EX^OGGQD|47&vtI*Q|X)Ej7X1qr9) z&@cv0Sfc*a%%sRV7A!o_Y7!ey^mYNfYRrUyVZ*M+H3Fu@V%m7=$=8i0mjnDPhoA0y zH8_;7L~V3|NVeBRa1vJeT}61@IQA9l^eV`lCi~21aw`FZ)KgQ~8aq9Q^Br&i zLoSk2QiB^#M;x2!hHYp>Z12s1`cuE9|8Yq>@oa%O7Cwmq~e07?>Ugrm#P7S`|7?8U>DG%oF^|98Yp0XjHI=<_j zDHZP;go!FJstcZOjBq_F<=7j(T=3~2opNU`9g4TDc|Vw+xXn)Sf?LZ65Rq$a4m;`9 zqGY-|Ez0XxF4Y`g#|M)H%QdI_Z5F!&@1+_bAmZmgs|Na1K5AlgJrcWa-2)=aVbmf# zeF1HIzA#85Fi;qC@&@RqBwgp6l_v^U=dXkdP)$b;&fhsCf`qmj)KY~+Nt$z)m+*g* zCh;_1vL+1~)-spJ2cBKRs303B^*x>=I;r16^S!ZkI11ZfQ4W48J(lOU9naa~gKm%e zY8@l1IB{B=6BJ}^17X!d6^6&-ezYa_Vevr2q040UbI)xA_<=IEu(vZq5(jcJgrDBD zNx_P$Ek6ZFfhf&+pA` zh~#tGvSQ;DI6hY|qu%qgSo!;#bCT+xRZ8{zW1zzzqd^SavCBUMEmIkEk{jeA)WLW$ z)1g^}8nTs=-_6@UWyEXRIX@sq=jew5ISC4=^2-Q;+oZpLs5rY-{I?SV42}s+b4$Mr z-F?H9seB`xn0WMt5AX_nr#$4*GAxoYDAZ1-gQfp3lthV5*;}N@i2=CA3Cdi9T<8FC zbV!Q=adiZ>8Fxbds|VfTARJB-jg*&vL<@(3iL;sx&_APjRMo0PMX92o1$Rut)Vi1GDT7`OS>GeuI$f;U_O# z;s)dRg4f6+!2f}PPj{1}zEB)HY1cR?yCKnaukkE101!e@z*w}DNW}u$YvQ{6zd`@aW=EJ zqarv4j#4Q$MAFDsik-A_>8Pag>J{OFl(5I!p)cNI*|a!o8O6U1Ed#gU=;FwOUWd0x z;@lENt}P@PE4D>Hvl*IVm1xOUAidNHO@tp{@wz1+Q|=o}9;sZ8bKEUAVN}*n}9at9l%YN-q*v`x_)*=YF+@0UU`zLr+HC%Zdvx1X&2iKF9eRp{0my3 z$w${Vokdf%W=k=o|NY?ZXM{CY&0p_r!9OHOZSz6;=|3TP!rzh>&MlNK8rbG!$TW|1 z_6nn($pbuZ79`i3E;#ey@fx5@-Jo`=cJn0;@#MXb##mNAtHZ{u&M|n|GnGkPSc6s2 zWjI;5%gh(6lp7h~Nic6qo~$n&@Kiy^(Xd!`w%`%L$QsI z1J%bF?D-sIPyAX3^ZU3k-p5$(PG6+G54RgLeGexwTU>q88??9b(bmRTV}jOT!mg}s z8m9-c3Q+7P`etQ#J|!R);p~B1h2n-`D3lj-p*O55zK|4>VYo_TuA;Mi1aV64jCEOc ziN1RB{uwXD_spy3FHehkq1Ez*{JJ>0no(j5Lo`HBQqwgsAPP=GT0H%Qxlswp;X(po z>hlDDy!3@L%4@rmKk~{Sir{lNFZAgo>jE*SJTe+}Z1&qgiT_r8D3(?@M?&X2-rV!i zw!vR^X`Pc4XB`y$sed@?&{>={{J3^*I=i6KUvlIDi9sF+lM?M5+)G<|fqE6@`Dno= z+L~$K8MncHpW2#unJ@EP>0axOS}G#pXD)zz@P^? zJ^$wg5b>uYnV*D}`OMp%*v(-6F5NvN+#T}R1(k&ww$|blVc(t91moo`Cdo7KU3HWA z;Dy02p`v*)gaB-JvWP+iS?XPhJ-S;sDj5;{;z{9NcfbFjngP@EZ zRHdPxIX(nf^$aQ8S)D^-0Xr$G?zt}fYSJUdZl)cIQ+oI*Gco8OU}N3`0Rv?n_sMng zL#-+$JN7YV8vnZVzn6Qez)y>(_U?RMUgaUfdj*@NYhkgBGLj`I%%*KA*~G)>vbtB5 zG3>_#APww)53&e$*s>p3@e6&xFVS@Tn}yrKaUNci==(c8Fc;T0KP#IN4WD{Mh_R=A z!na4Ph{8Ufw30|YI_C23m^XGyCTsQ2dcgMsZxmru%w^jDrFWyMB%*RaM)dCO_dbm)y|JRwMmR ziHv<51YKcqc3pdsI|$j6zxGxJP2Uw54Gt4Cd}2z^UONfeqaRMe%3y+jP-Y&Rf2@P| zXp7XapU961j&%r*6-i~1v7@F`cUrl&(pilzOJBxP2ehua`!0-xk)&ume49;sjH zax@~wk8$1@=%J-g3o-jZr&vk&U}7iNEX!=cl4RY}^_nk}=@$Wgj5>ab>3JU7wLi4oiiZu z&|^rB&`w&m)uG9@VYtCB0PNf^FS3){N2kYw3<4yIR=_;Y#Ho|{%mHvbRP>4Th66rw z$-gIYS5LDHc{gCm^x>AgAt+<}n+7u*H`9=dyV%H=f(?%pH^^Lx^`lQUYeKwK;vOo3 z74~Vhlc&*y^U(rxi4OTKb%e4u*WO{#nrec_F8E?#u*^R363TTe`gUAGd-!3nC`Cww zj&sUq!lI~K(jxnnS4d}b@&I_yfsdoiR!hx_LFP7n=WS-p!#^J*c%4(I>s4fy(Dpyk zs-{^jY=y7@Kxk&av@>!RMR(@%AhX^{Qz1)z!?4 zCJ}VAuMyS@G{0<1S8fJ8LT9Iyfe%*|E5?X(uqSc&a1q9rq=C$bR43O8C@`Ora_xve zl}(1cBa{`^jSbOZ-A%B*^P~0u7u8>`oF)-6u597w%A8;FkXh77?>oLjg?xjz=ZD{@ zTNaP&XMgYZar=)#-H-$9n~wt&oBL`NpVn1iMe>-ehU&x{_0%q|4g1PC*L=oD+D*1> z3LHV9Rvl~P9dvJF$e;}E%pJUeTpV;@>(-Y~!xwzIztx#a`L0QROqs z?5p)yta?%-q$Ra)zBxiIoKOT^>m zLwc)f)4ySp@)h}VTfp8}Ha`CpxNS`9fUf*eym|A(btCpmFcmJrYui&I2j{AT7IA8L zrBhYLiSv}fRTBPl8FxSV1elCJnwbCQ>cI0^l%(3qpa0oK zv~`sT3xN`aS#!QL?J`4%x!My)>Kd#}HdCeZq&fm6W5a9E-{WuQG=mmlnX{2Pv~7~# zh>=?hffzC`3dS9Mut=u7%q4}XO~%9$#u$^)4B)4^eKzwCZro9HxsEBv~$ z<+Hphc|hxXocE@|wt?-uu)>KD2uTO?r$1`2yp@6JVUaF`YAt1odt6r`-Rtkzjyy{0 zoiLmU^!`%rcVI_c;AfsEFY!QT5oPHRzarXJFki*;#%9hBDSjA|Ge0~t$#V{O+9PhY zvk)`*3`{5x`cb)}A#+-V)0?@F!)4mn!X$>pOVTPTOHVX*%XRzJJY@SCy(9F6>sv3$ zixRAcQ>p@flCeB%J`zR3l+uN|t^!T-xcO+gZi~vSFJl<9T-3*G&2M8VCy!*#&TI3i z>~$I)8nU`k2I8l~+6uAn%ZF;zceoDFm}w8`A%nqUs6peInah++&yxT4+)_f4@0U&9 z7vi#4+BO5O+v7B-2U&H{DTQ%YlSQf|aC;b8s7kKN6tzmh!!pAe?*R*U$nEo`w-r>l zd}If&s7Q(&_ldhy8Scdfcv`C(_ay0Mba%b*%R3>y)~Nf9seZ>TqI`*7w6?U z8mL&sgUR8&B-t{|ik=Xh#iwt?v3b%_X4pW>}W zBlPpPO5DB89L7(7aTSvLbJAQ6)%@NA1>j7gP`*`Gb=UR-gwGceVzNgAYDDjFj`b0) zq`(pE8s2AhzWhh6tx%s6IdD!DjM=eY_volib(Jj>{7DL*ZR*kEZ!wn-F2oqA4$-~h zpvazK>4xI}#&0oobtc&lGv+AD?{b~i{wOwXdoWHXu^vq}E~+59K^`oMEy`d4?YYRd z<|Enw+)UnDzNs{_zFj{TKKA{D$h;8aP7iD!NK(}n)U`(iODotIbL|YKAlJP{dFGG0 zG1Ex0@6mgxExH{~cXZ1tYGe=$u+#ZU(Ra)}&9+SkQZ#zVTP&?R~1#|b)W zD%Uiva0gPVH|76dT=D@XMfGG`s-%QbzJFjuUXuJf4)?A9TD;4~gBsyA0mrm(B)H-? zL{=DKm{zPG)j}%^UdoXZ3>eP8{C8`A-oBpq+nfit1h3JKeB6bfzG}o3e(++Fc{JW~ z->C9!D8d=LB+w%922LN81yS|QjM_$snZYC+$PYx`$lO4HFf%TrzkJaX5Hd zpvh>eE2kU8p{GP3`$%`#tN)7YX;nI$6mTW)?~DkK&uJy!EmpyAcg&7@HrrENAJgSE zCS$0t8lfg++!{e|GssfRj%EaYIXVfJ2-fy36Y4RPnPtUM$KQ1=W1;YVVyT19*|A@W znHLaSXz{mE)@)mos(FI*eu3h+DtZYfAWzeSLny@@pS{QMIVxStl7XOIBW0E10(v@v9gvXTI*(AzfXL&+QVNkE`y7N?9!bBOfHRBhPJ|A z!PK&qEA+|)j)U;VHklG_NfMH?ss2xI5pGlDaD@7XdxPRw_tjkmnZpb7m2?RQb*%?Ddl!LH-pT{(0 z)}G;=P9=U|l=5Bp2GbG_`o~}2UG`{QaPZr+5C`j+Pe`(HtJNvClc$yAMbaAlS-S@^ zzLB|bb`LRn8%6=XtyIL-`|UBBK%Ehv;uyD|&3iyy=|V!ja9b)k()bFdF;8q^8%j29 zzOSZ~YunaCU9El*6|hbfC=A^*wS0xruYm&(2#S?K;F>SN4YlZ?9vK=$g$mSn)w~dy z^CTO7kWH{ppr`zU`v1{%mSIt~Z`U7M7(zi>N=3S*VE_Xp2c$eZKRFz31B3xz4qIYZFc|HNbiC>!~%io-9`%)S2WOqyC|l zsk$;fQkDkufc`tBlVdh>z~>uW2N0g?G&`>#=k)VDjw0z*>iO@kk>8_|AfrTBo@A+5 zky;Bn>dT1A8m#&}$9+QB+K{~k8JpH~U?inv8>86Q3#^B`k6@pb6;mPpCqf97n_s%WPzwx zyJh_4Q2s}UPm>Q_N&pJzieq6xi{I$$iHqdYfjVtGKgjad0^>oEy?5~H%!eV ze~Z0UfH4K5IQT#RQ6U^uqH`qM=(I>TFXuplFohk`b9bZtNR1&{UcY z!{Ir+czGyehsk9d!6ABdzv~O*BR+qMB%ZL} zC%R1mPjxH2=JC$M>&g@VdCE8}`TePDXA2K)tTQb{sMYsbOHbvHpjjMW+EK@$>J9nji59{X9@!E8>H_X_e3^PpX~GSz*(_MMWu&aVR;-r{I4 z?ur-gqlO4u2eakC(K8X!s5BK`|4`w)YD0i4$)5SP1L#9O6!rfsQg7wI*}t93rbDcs z+;`o%5M5mxuij~u`I8TCi7q>`>9OY8S+_#f%%AcJ+zH&wO8LL?BnDjX>f@5Tu@*MT zBQf3P!ZGf4HSg<$$wdC~re}P_Wz}RR)8htrfLW%FeNU!n1m8@b0KreP*+Bch=nj_K zhn};x=PqZzZ6l^_(!WG<52ua{jw<1G*ACo4o@jw^=}A|0^X2?O+iVe%KKG00omLQ= zhdi>l%3HmR{c>{6z}*9 zn6S0#T%}X%D=r2z=$e_H*oKzt5^BW!T*F-H;L+=%oa@U;hC94x}2iV0-#MIb=fKq_mO7PtZ39l-N`X?2qh zy)!HZL8E@*!Mo#n-oKcKrl) zD+LC=q7*$_y{#ZykJ0!)_{I1Tq4+mg^C84hFgH!1H9Xg%`T5VnHSpbOXCGqFhHJta z(MUk+7N|`uFC?f81&x{5P6}>N@d;nPXoNeuQk8}+0mHqRD*j6O#-jI79%hT&*7q)#;Gv$G;c}mVmS6p$V3JLXi-akab`Bn(Hxv)0F++%|lm118 zheL;cf<;LElr5s25x%yUci2L{AV6s_qw@kHdSzbvft&0l4-o_>Qv-6yIfMZ_^K8t1 ztcGt~B}(q^xo{vyQ*3kE#mQ#b9*p}*5lmo!dCezU;W8{xl9EBX3(D6wkMNMP1;6|9|~2+V|b^Ejh5j1Yy7&x&5R=W4WbPT+{{ph0J_v2Q4?;Rems`gnMr;8<7Iy zeMg%UN1%NM2DYN}EM6lN?KN_jfZWHez!dV47c&AOFnbpYt#gg!V)TYz6#H0v zjN0-UOsmZ((QE#>hKL^=VmoHuv??7w!Lf=pE)hHGG7p%xxi9A^#p@f9z$Es8=r0e! zhwm>3PZ@jI(N7*DoL{I)OHRqs&V5-<9Qu*alEIuF!IC9x&F~_rk3k5>>6^~3eJcqC z=&*%oXYdG^&RV@T*N8iWo1O+#JBn3hu&)z4iUhXd637;y{OzTM#9sTM%4Cj_Cp(9- zs58T${0zF#6B>-Y?X^gn7qdLyS;u0m3C$Z+le2ZQH;M^seH{)glQQ~bgl=}}XIK2N z7vH3mG}^g%cxbWZ$Vl(F&4>P@J5w`$GE0dQ>RC@ln@;>s2BNdd6Xf(GBgl&IO2}ig6@R|3QJbC)9AAn3%Och z(Dp?UI(kNY#qj-l^SoqT5-?w-W}GAL$O)32!#qT1tN3?q5b)?d<#o0kfz4>rPq9Yz z2u-|C=Y_&LU{VwCo5&<9$lCJ_EwZs0<=4v+E@NhY^kLx|zzHE&nmEsHaSskc6dqzRtd& zXdFlxkTt>M?$4;Mo>$%c6GaA_I$Qyo_<=|sV?t`*Uk}TMnF1tk-GzB*3U=>+67wK> zO4yDeFe&sK*P8@~10Tc;nC83qWu97!2ZYj*-Ln&!w^W~!NUU#&^CpdBy(Dmo-cmSQ z(LI5eZ)eX}>^}FgvqIn0t<)(Y1n)zYI{l^J)x!Xh>{qie97eTfN6)9&BmCv*fT%&t z@r-F%^QfWqWZ{{tLy7nLLi)AE9)*dt9mwh;NlCy4b`rZ6!cbo__40rkOHYJZm3%og zbNo*>q1U@Jz-d)mZ^zbWi< z2~Y5Lwh)n_o>MDFzFG$439*V%+G_^gmf6ioRru%xQ3go>m3X_522zjLKxrN9lYM_(SEgRUURwOWi`RIv@i(uEOs8RgBt zO7>i$!dzlI=nhRLeL4cM2P6(T zN>I`dk*-B+wJS0gyAlqv23}K2=?Zy1ejHAq@s#h(e77&w6h}Ki%?`YI^zl^spyl|> zI6blBef`wt3f$%0^kD|>WUQJ(!h1g`$ep!?1BV)&Ldp98Jt0$|H*lGv6tq6*TD_b4 z=4rq`FP;xQeSbD!z2D2PsNfZrIeZR&^JN4$GOey#lSdDb7~iJiKw0z`S>&6t#tb<> zc&WuLLO!~h8j&@%%hOf%UCfXLha`{EDvMVm?sjJ*EI9!6)g%)?2)BtPZ^iDqAXdH8~# zb$pCU$KC;^@oQ;G9AxAw;|yp)2X7Jz$|3*btNEGn(%R!bM-TO$2@7hXS!-EUs`x-8wzM4R$RQaA{j!IzAu z%K#x`VKxESXy}^e>yqC3AIqcYUT)h>0v%G#vEz>+!gkV*`rb(4{*!lkiMMP@_Zb|1 zn%vzEU$*1wkNxg9+|c!WxS!f&P4Qxwp*3j70=OVXe)WKS8yc+`>I-qC;?iJtZBsL7o6aM03@sdMW((THyB{&2u3My1E*P)bY==svj@b+1 zhMvPggM`vfkGgJ;))h-xOalz3EXD>A@oF#micCvBNSDK{IBDpVS@L|WFGsWEA|yfI zUGsn)^jG17SYr=j`I86dOt9k)D<(L70=0aas&|T=CIUuyqum`sPXpPiX%>C1VmHu4 zT=Mh9Z{){Tjvju6m(cPbT)(XeP=eJQMX!VjijSe+FOnn^tJ&Xlq-e%ob{wRr8-Q{) zU<9gp^0HRPGsM>h)st?YVXBap?b1E_t74e|=v-RGO0Txtdt=N1p30&o7hDzzrp_ zHJ)d0GJWi=m6dNvb1T=4%e4AUXMH4F5UIj&d$a$1)EhsH?K#_0%X`eWT@MG%qI+?v zhhMjIJVU%0(uT9*yR9cp!N};KB*ZB)k`MF;{d2=kUY&A8}Io?|-tE!gls^B5*P)e?rq%aLb zJI^l#XN)IdvP!~vGM5y100CAjl#h^IS5o?fC*+6{WC4HPk-|mqIxVlODYq{|@Sos9 zxLzO&xryH*`Xmu5UQ>Lutkce3bIV;Uvq+@gT)jjk6+gguMVcPyxFFiT8kI#11wc37 z0e5-Z;<~Tsqw0g7=2c{q_c(4QqdQ}HV!g)sqoM#FNa?pdS}K09^p^z)m+fD)vQ^h2 zZ!s_ZX9C4hB|DSprhl;_C&suc#RAc>#gE`;_>OsI_6?D4^sX~qyO<9(?ayYTqCWa% zPn){h4yS!JVI(C6kM9%SEIq<`fZ;*r_o|KF`p>5|Ai*a^ME4PA=fU0Hapm$A zRdO9SR(qGs*B(YZm;X(3oSG2UNsrhz1#qSA2v?u zx!&)&bOkx^#1;RpmZE9vf?~0ICAfu6hAzSLi(>v3a6|n4C1*ki`;fNEj~Gcem5;nt z>Pg@HK7uLt|IZ5mEMR}c7KD*Yae`iSPWFp0&@DtIxameWY>D4oos6~Hp3jzD?EM?l z081bIbw+ZG;f#!T(o*Ox_`A^^XC8zD4BJ$b_IdX)b#tK8&i8!r;c(s6pQ5%i*scE# zVk66spLEh14VrfWjy(Q`wG?!OfIpC60Uo(c;>biFIUYLw`EMC$mRkd0p~>|6!0MEI zy)ydk11i-_-6)TfXsW*SpEj8EDL9ukKH0;n7Z#>~3iypIxyH3LH|^O95inZ_TW$p8 zx!LFOdxGopp6 zZrttUyGnO4qx;fE0XevwZlV;|grb*rOUO@Gx`JAzZ-|!OrM)q9V7d}$CAhA3-7-7J zl-R^3C${354#(lQ6|)w{+Tq-LR`|zhEdaq(ZHA13haRZ8K#4@zY6xJ%U5V;l_@0(> z6{}|J^6ea8%IGrl_dNQ=0e!xfi;2HmH^VDLuMj7rdnJw#J7B;9F&zdyySK;XvpE4f z4BA6trVX`b%2dRx&*r=X=`%UQkVWd4EZ-2$kHJ&dpo&jVN7a%k8bgz64y}TT4dff+U zSJ}@JX3ucx<635v!zdgJa1Pe6m}$SKbm>bH-1{Ql)|>jCvI%$*E4vh*jRPKPI>lJJ z5yppPjDW_EGF&G2xcMJN#!KXfw%A(|h&lerfa@v{1=NBrKad@idEL4<7irh^e zf~gHys3lcc|BHZbs@i54+_ZuoJ8{@7R6*!iK!QZ>r}Z)Y=;Zb8(9cpL&Ig0%A6mSw z`7QW&*ygis=zgIxUqBsxF3<37MC&G| zSAHZOH6x_285%g%QXSii0qyuP-`dW=*PdCzx(?KaXGV=toHu7lB#17c6D8py3GSCy zg`CpbT|Ep!J}UR4SV0;pB5wu;{w9~?z`TGNzM>KH@T%8$D{G+5QOwQb3#LI!p7k~S>$%>NxHRL)GF;A`Xt6htxG*nm)?q1CWp`Roh_DCEc#AA= z`LHdEfhT{}#L-XK+nl!APb+PJ(iy+!QFq=f_de*T*L+oKCpSdbf8*qkLi%*v<*N46 zU;&Vo^2eZRny5qhiRo}etxWd-UUA97oz(3j)(6_&&Fy9eAH3a=SSGV?P@aFUQgaNn zB<$Ajlr}!a8@~%SwzEY&qFnd?7oBzSjfY@+RirxUv2<#d56A5Gpk_z@m7&3#TcQLA zc<`tb>r*8!r_V|d325<)Z$?w!Y3zlA zgzN(bl2Z3zV6-3Z=`bu=A*}4ZV)PWkT`?~9{P@6l>S%5GNh)!W@8)4XugR8$`xy4U z^hJL8IreW&l>R?6iM+5if839p1irQjBLnH-W#^s z$L+o500(Y)s^t^rBw|u*Z>-7mHD6wh9mzt?((HJ4!YXW~u_7VfWu&B6#~EMNdIPs2 zGxg>9n>vE^P_XF%O=SI;(_vc(q58j_+d5^@<1z{gqbYJmKm*5F7*FmQF0_ie! z9_xwE-vc!o?SOHLAq8kv$U@ZVV>(jaSp~d8K5$E+A+9>H`7axRzr=M*-bgVK_Mf&Q zYt@W>j(y0GzPxZ8Ho*g5c}nzKLcK9)tyg9{w(6>}5--c@A&tBb>jSI}@^EDMX+a%% z=*q^!LNxxi&{rZzZ=JSobV~v$E|FpbT)Pa8Xs%sPH7|gkWw?F+J4MQd<3@3FkYip0 z4HQyqI9||A^8a-rHTvmN5HQ-h@;pGxINrm_ntG_8OUsOb(MGb(ZvVn#b$)D zQwqYDv>xe?wu<$>Y1PjX$BvkU`eg-k?f2v0i4}5t9yUyWZ4nM8FMYQZPrTi|lDe+c z@{&tL-H(n@pZd~htz-CWMAGA^HP|SJ$iXiPOTUstf<;$I#;hc-X5q#-9$RsC$r%UF z{bJQTgH|V}$@fMEu;IZzjDyncqmHlF24^&ig!xGYw)?Vvzjq&`>^0sXw-2WM=z`9I z1&~uhf+2yv6yus1rw`)xPTis;oyh|AW`g(v-d!S+w~7w9T1PR~R1R{ssDOqPdV|x9 zx+l?CPL3KisKNy#V2<@}y__CFz}3X>>;6stl_GHkd3D6X<30S*IE@ zDjhCq_li4-v&?^uXE}%EniH1EE*1@%RItsW*MMBCu8)PC&52}4*r-u-aLdDioR+X7 zkapqYX%0JnwwiUn4Q88OGHEnM#NzKFvAh)KhkfO8(Mj8`XyiQakaBY@cPldIz4FCe zVIT&?UNKqt_JTgSS~nWv$uE?!`4v`zsC=>zJk8v3J2rc@lt4JD&TK12@&}b2dVpr>UtE+p!mI&GM?0KOJPlmbmoAmx65r* zLk@M%hOA^Rjy^4Kou9}!un{%M6<47QOT-3MpY-s5?dv7Zop#*I@^iTZ51J22Za4oAIO$B?-nQSyj>XR7r-;t6a)w-Boi3C4jpPY;s3xr82sr7ec$mLm6GcFyB zO6t&rj|Hn;z1r2m(GG&pV}o|9Rr1c8rw|$V43sG*CS$F9z@4wjeOKg>JTy9XlGok;RpxyL6slqrvX<(AQco1(+KwYKrA*R1?+dK?Cvq*Po zctbq>z3!E?lV1xr&;33^J=s>*Z>b?HyO;`!b^pE`xkRS#M~NBgm$)!q43Q+=?U=*E zbN!7hA@qdEIfJH%1GTUrqG*7BQ5=*5^n1%}K=!kY3fNu^O>3;hn_!?#gRD6Weydt$m~PpzSl& zyEpWxYywc-dQ-TPKZF^t?y^z%!)JV3A}4fL!EAFPPKHxh5!lx0x-hQN9eWnj!2uZ& z>1W2U1^xulE|K(Me>Sj`1~g+jU4S~!w0Khat*1%zU^ZUF_&f8XW8`@c?zQ3HT0TaKfg2ylq|06q-kt;Xz(7u0o{w?g}-seghPNOh~P7Rp7bxL z56ECDmJXLjkkD!pnUA%5)#_)7R`2yUIJ7a(d&H~KSerm3^xQ-lyl=6%VvT7^zen9L z;-DN)V6j?HhC*s>bSuzK6}m&Ti^~YI=|C;OHcfM=g(EhOhX^KP@OYCWQWR68ecz=2 zM?dESBSVmyu=m3E6GMH=FQSH8=$>zJ0(ff6@bdG-t3Be&#%-^3-MHHSiy$d4 z60aRRahg^{a)4njaY>LiBx?4^aXsc#x49Zo1MRUR2%Tnah>VL3NUK?zuIM}qrXQ%2 zV}Q8ZXH}LN8u0D4a6BstomlkShxWowABo;*OQGq5(pc=Ma{b7B-L&t-zyUuX8P2s5FoL_VXEmpiuT9M9fF{6bA8B2; zMV$^rXdWpYn6tB*+uAOgxvYtyG;lqwd{9IBwBGMdbUEr5I0$)O+kjnS4$v5N zul;2{3J_YDhZMw;AV_6pA*Y$jb^m8Lx|qVcjlvv5H1Vw`KxQQB;hAjQ$fQL*66Kb; z9+EacRTp<+>iLRHHLyu@nKbq~P2#3xT(a#kYvrh<-R#Mrupaz7z=8iVC=ViQ@a`LI zQ(hML29r4Zh{u2XC~_=sUv-1AH-1!6{>9&4-mOr(j3yB07A-{jA^hu1W}L?SBp3SW zQ4%PLT0a%?RR5#3$EJLphb}obMX5jIXb9;YG!@?CalA>=v;J|Ou zq1UB7*QgEN&o>(it(S$zn*yYfd?YYJR}lobxczzh575*fnD_$1*X`ADR0Pd-=4);4 z>8JxL&N@GAoo)Evj~B)$2=MnFQ}#tAA29f}kqOOua<0bujzh28NLW~?Ar)sFZGFdO z!wl^|RPB1K9@(|>$v@<=Yh{@?fypJ~)-xn6H_<8PkvdK*V1t8NXlnnR0z3Qy&#Tr+ z=wKkjBR85`*pt+)eH;*F`K$8pK9-C)Ilxr(Nw6Fr)_}v~Y&K0YC**t*WBb9Nq-G?~ z6ddfm6n1Lk1AJqMX!(Gf7T0)FLC{c%Ao$NRBra2Ct$AvRyU1^sM^_gNZq7A*hG@@` z!~0RvhLZ8h;w6*mHJWpA+Db-ga?{4&W2s)Et zk%0?een%0>y`)_x0e-e$4~niI=Sg4of??Uo$rGVs3DH(n)(HDtvM`lxzq9d z3EC!fgbQl2nfXTapus$Z+BZ}bhz^nkVcJ=Kkg_LjkJ&0O9d|o7%f`NnW+vFagH4!h zNUNs{vNZsQ^s*)F#{Jk6@&2C6Jizod@1)k-H9kVZ9kdni8|Y2PoPP6MPF#DER!3_} z;UI9QeWix31`;h=JAvr(s#nl6uv zC;B|pb3lm0m0mC_IJqYpl+988zK>?O-Ux9JBFzgj3+s#jLOfN`3LTlCdfy?N5|sU= z`#iUi;P`oNdFVvek?Ak3mHwC%ga_SxXQ^7b36d+&26uVjk@v}Yu^d=?DQ%dQ8dtR za|go^m`Gt1dhA$x*hN+=@3IAaHiLXU-8P=8r%^Tmc9_VVAnX|Z{>vp&+J*2T@r9mBGA!%ttfEE(a%?sbahO<{6Y5?+wxR zC!5ipw=ATIFyHF8)B;Zar$wfN8ea%-Dj~QhAoKl(4=?b1R_j*<^dfxL(>cH&K(?BF z`!(XXV!X}0D$7%4Ow+-!Q|s|T0M+qhz4522x64spS9(aKo4_O-DG+qgzkZPwDq}d} zL;GF)k^8E6TLlrz1a_llF;LnOFhMK>&I9!sd~1@!Of#VI&tWL+5#WfJIzkwUx*3K- z?_gh3R*CAqjY|m(OGQzEe2!O{C7%O&19ZE-?JtheoCEs7bqs7hxO8arMLOm@iK+Nd zv;{8w+Xgiofx6L^gVoU#PD4c|7Xs+jkviMf98CCCp`YqYbe^dbO{#!6$$33UAj;3Q zg4#KD20bh4CcX|fgp>)k%8Frr2Otwe%rA(rfKNeGkF?M{>_E5)6*t&YzLdzJ`ZbM5 zwg5BDVCoKEm-mOxq@LJjhn`qgO*F;&@n*AK`(7bQ^Z{QVE+gp`JoI8_^7*}xMwRkk zfY0b6nt74?dVSijxNdTXS;~kV3mCrB{schJzKtKNT}GTXyo52&<}_job$>BsoWx#Ai}kkSQ>~L6r1Js-^F9O z=cbm9!t-!ZHmKXOWO19&S^m%RtI*{a#thGY(dMQZJ4rpDQR{0OQ}1Q;=}QVHmGi9~ zfZgg59Ef$yMQ;~1TB|AN4fT#1=!rbH4!{>D8HV@mZw>H(6Aau~UTh{4oVHR#R@i1v z^o4#Falj_9tz^w9xU|b7W#I5HW){6~3d=v-mw2E0DuaGW`BfZF5-sE>>|yY?8;|TC zTShZ@p-U<_tk7uT6LW%{s^TMr2P7F(uIOUR={e`8u z=lb{MckpI69rFNEfOy8IE_(H5#;fKp?X4R@MO+28*8dr-5dvKWYvt%Ru=sW9(cwhf z;bi7is3@SiI8FiB12~$q=GC1)W}W>9;NC%0C~fl=GkMKb_%N4-A5e`Dq^Z_l1L6Z1 z?UZT!g)lNc4>sZJ5-%`lvPfB3^bd!^FP&-&+2 z_3ihS+Pdi}slDrA{{K{efAR=AYHVy`9m|XA0PR%+n(J`1@_%{F0tG0$t$5G*Ob)X@ zc(d$<_lbJ=0GshCYuOOxk(KC@Nl|VJbuNuxg^Q9c&e8%U$P78JAV=eEwl;Q*B&4+_ zmk}ghqc4$vPE|P$h+rMQOEFUqZ>&ytxTv}|pSmrD2nZ9g!;UtQ@&w-!?RbKRxW%VS zZ-qedgHl#|i{$>H0=>ybm$9-{nfh{v05m5+_@?-@LYV$E@MFne??vU-Jc;zE`JC?+ zr_67CQ#PxjVXDF%!A+G}d`myDU61{Aw(Dr_?iDDcF;Du)6COO`Dp*JTCap{T#cdK` zmiKO=I!{RQypOC?<<$><3bp!k0lST9insMr>)4mjhuCYHk-HtzN#m@LUj&q? z*75=QFD9+W`xhIq1Iaj{aP+sqN#fgpwbvze3egK-fd&`C0Q4vrL}lW8u@@#DQS`C) z+51D&)X3)Mvvb{9fUNg?DKGI3iN`DJzlFv+$lB`kM$l^i&j5@0wgo&c02w*9^(X_niPPv2Uf6NMRx& zPS0Mx-g!cAt9#$j|A*vdzOm_+_PS27DgXWA0OV+MW}QHY1sQCrrHQCb@P@;oqfhFg zUmq56Vi8>9C15QQG%WBJTJD!r04>oQf}Kj)oY>ngtTnVwl>u=eTO z)#v5J)5k=+&LJ$<8QjylaV6*WR>gVzNaofSb5E8;&%BMWuk@bB);E9y>SS`M!38K! z#-ZWWN4MY>m2ebw*)!q@Bq0neqS*}zr2w=G-om4g##;H|w~&ntmrmL9@I$XfrssZmx1h^mx2a!Z9MUsu$n;G7Ul;Xfdd_fV zUu88xPJcrqfOxMQFHkYYm&Vrp=k~@-0@$InQv{;Jblt83fu&bO;ce|H6`V)zn7Z&W&RmNe;-ezx6P8X|Zq^dz$%e|8!gS+; zp53=^6vRq`W$2{ra+q<5AQms-XhPJ7-v^uc0@N;=g-YYUO29#M^!ey_W( zV^a$6`Yy~q=R2VTki5>Fu8EqGE^_6@43vOtWUPQKa~0Uce*JCGe!RzSt7FIwKGZ@S zNh?SOD<*wPJZ|}@2tRFKTu*lpSFVDkd|EFv94a+MkqHpLm(Yvd8MBmoKsO~h3#Nt0 z#>L>Wvm8WDP?eF*7N7R!3p9YjXo0;U z0Gn)kp!8@2%fucMbaM<7XAgBt6P38#PSVn_+c)q(>}+3yUgt}C*-xn+6E-{yyVRNt zS-dvHY^F|9IPg{Y(%AfZ&+1r={M}Kf1=DwSpK=dVZ@W51vc&JXU!P8%GvF@T-EDN} zBdxRJ<~po6I`Kf8OYpv@!mb_VzqQcHD5)_P_tu$3&sKHDM($vS0caujrDj#|%|hvI zP)XPyl){!{i!h;lo~tKA{VJ|H;PmIY$LZ#joNR$>G~-;Esp34T+T4?0F8oYJL7&HL z1Bc>QDn~8g(8K#d8eV}r8$a=*U!MivM0Ty&C5WrGZ@YA@yyeM^F#6?XHud{SS3|5H zie-f(0C3BFBZy!JweMRq-<}W1{#EjBfptTt4n1Hr!!L{odA(I`wg7wP_P3_g?_OPw zeV!pCTKhV?_&({k8w06CKN!CL+jwu(7&&6gC9Qp)*01roc{b@vG}0J#Gf;f#&1Vp= za+y`NZhcVWpJYKAa7C$)qs>A#Wm=OS6@JR$e-unsAnoM&Kk_(){f;x_vp7>REU@RDw+gV6~Qr-KkJr<^B^{Z`9`XF5zaE0Qj(rj1pfqQOB%2F}0a#!x(XpmDH3I1tOK#?i!0 zK*Ks}U(L(+$$SpcBduI@eByNE`BN`wPswA>KBUPo8$WZ{%QU214?+=~phgUaY{5U$ z%mp`$3Iu9oa;)`IDgc2Q$j4M3+h=E$pzlM~ZL%D5QvCjw?RbV`A;pg}k{_BfG_4Jd zj$&7ClGI1m3u-c0$fr*nO4#_DF#m#MSLePbFzJM{4DbAr4jKcfbg%Db-lF~kK*C#| zUA*z!@k-h;0(eJoG%*!=JrQQ8eUJmB(6@|QAk zQwxHpurytVl%KErRSuP&47_$S0&;4@0pLx090qwG+%sJF;$TVa!@>mcy-u7X%ID_R zCrt(vMIw2Bip85xcRQV{H$okS{I?&DmoFsPuElsH3~=zYbhp*llG1eXSZC{Nsw4BL z1hsVI8j7_3rF}!;lu%NWp;+ z*AC>cXB{6xTq?3+AAoivN!aw*zzM3zV3#}G#bHMG%HJ0Y)7@ol>c%uKXjzz19zv&I7uD*FhO7T5^WB*|Vb(V9rOa|0~Joc<-b{^e{x?(5x03KSa%R=u9=Z(s%ChgpzYPHDyeju6K_t^^ z0uAUF#|Ffuto?vVQ^S6SP1AYbOu zc`2k?w^)ngODUrO8D7NF=rTM;VDO_zaHt$$+Vq=NVi4J|R5n7T=E`cR#3$#Bp_SJu z4D9r&q|(bPP5TW4kM}2k+)hm#KeUY~Fq-%zXef`zDR~=A_G3G^#KVNpFaI-neCiS| z-hWn*Da7wwS--3pYVE|E=bGi&)OcIN&K!6xzD)+2zrvPi3IWOr{OW5}BJDdk7E#~b z)P_qvXInK9XnbS?*Q&^J&L{$D8k#qD7`+$0T?L0k{cEeM$a!n2|LLv1?3sJCaW8oO3zXComc0amo=NqBQia9#&lTO zf$vgns~=!>Ke_fnHh(*l#DoVvm=Z|NkFkx9ARHYEg+9i+8ERz+;J=TRT}i*C=9k!O zNcnJ=;nE@$Z^FkBfZ%_=Jink8vD9*)O&%(lXu>ZSp`2x3$<@K2V;Z-T%Bs#J(nk&uRJ{2e!&(Y za*#QXGYlG$IF$BtPGBhRRqtJYiNM_PRHOM`vbN2{6Y+>a-387$uf>364!H~ES^(JF zz5<0Xr(7-_nIp)=$bcimI8i5~{`s3KI7S~fw$mZypjML%({k-#jR|>;@He(2$2e%<$WQ3j5k9G6Ot&&Ki-YcqRTql4P;ldVbXEM|k zKgp+3hI$nIb{y$O3{;*bFE~!Z4LwI&WqE3C8u|7IeIH0yObdKK;*+k(S5W_Fv#@Ww z{_eya=Xcgv7m+Z%Y<(ud<|{%PU7yguGi9ofonCoh*eohB=|^Sk1Ptv2=lfbJc7Ho{$O~6a2G0Q)8N+&0@Ri+dP2ZN#4rJ|E67v+vN8xXQLI8F*dYv=+ zPMauSf9Rzu`EKXsyRFXqjga#7;?(C2aKjsVxo&!iI0#GQCCcS4Pg3(gF)!&Q+Qhh^ zo*)~an;jnHn5*+Jm> z-{qdhv+WT5U^n8%a&J9jj|tuNiA_<|Ju1y@=kdbz*n2oyhAm})!h-1~pgoaFwFioQ z!4f;%9?UPn5Cb>+Y2)x=1EpU^ADMjQlLut?E3O_o;jlf)qyvfKr~2#sHZ;>T493y; zH6;&%K8_*CE;g$=p@Iczg)-JEwHLfsmJ4CmHGPbi$Q54AY-ZM*9dAy2vq|^8llGMF`Cgl_LjWu9W!Jmq zwO)Y;K)Mr?0YVn)gu|bCO6ee%)5u6kn#>}43flE`mbgY{$jMI%(PjYv$Y>708%s(L zNBaSe^C&hT^Hh$3T;iP3Jm%baJox4Np4FTAgFxCLAQK7Hz0g+8@WU~Wmn)Q*!7=>x zAC8?l8Lt4lOJQ%dBH`d7(D(Q4C1@3Jw@?BxXj)BOeZ1-~Vl$j`KAd}- z?(JBi?g;qw9|}$xEZ(~N>%jf_c46CbhW0fG8Ht?(Ut7SWN?t_ zxrxP0z9G;IJ9?lSkrlz4jWZ4o3M!Cg&ZKd*I^hPjQulAu~WQPt1DBR9dXfFmU+` zGoNZOvtWBd#Ra)+$%bQgz8r3X@CqWpLamDszF%=xtfp9vOhR_IJWU?7Y=u&wlSH`# z2W40ouk+VM`t;?IQ?gZMDf=;qgpxW+BLjA5BOH!2fi}Yd8Fjd1@K*E_0KNhT;MnLS zgA9(v037mJOPpG@W*LNFUl7!S$IGk9_Ch~<;4Q;_`ZoXwl*0j@KfldW^6+Eh$!Pho z-y+c7j9wPWXbip$_xGDLZV$=~^sPN?B}#AqdzkWkfwaqL+%3G^`@EeyN8&>`FYRdg2FEs{6&iC3Hm`l<4?nc z1K*hxqirZ23#9&BLEE zt^5y|02r_6N1+#!Pov7N4QHz|(|VY?D&_KwdNtLhQhGk#tn3P)z5)*TpEPGqfeGLh3C!|53ktI|D_=*< z_%3}u-7$V0t`c}KXEndJyE-U~Inv#hUMu7RE%D0r477wD=}ASMz9)rw2X9Bmfw@IF zYOl~+qm9btld55MTlFiy&7SPij+{d14pNWg*)4>321@V3)-r7e7LqSszo;l)NJ_po zaQxtkn65W9Xau7@(6SST;IV3?I2fDC?4?Ii(M5TrrmHJl$<0}0d#a0Y40Y8D_ejD>b2kpstKd_^KWM} z-q-90ZkIH+7fUa)cwV+t(jq3eAE<==ih6W^;0P!z_a{wBZ&?6pEj&jKZO~tmwrJKG7;cI2X{*AcU*7;r zKs)3aS=JcWLD8vCNm0${#*dguzB~GMX>uHoq@vRW#a|C3D^+F|W&x(MPRPY{JVrH( z1~$4TO8Du93uI%1uw&)a8pNM`*YyR87Kr8&D`Vsb&i6LyEmG9ufacx*J{xM`Pj>=` zG*X92(@d|$uJCfmCaY{i#RM1{E|)!MrrEzlhku_XXOahkBc1-;{1!U>`k zi+{VUUQilvQdoS0h4k?i7RSox@k(ymo?RF}I_#l&-Qz}3)>?@)L(iYMFN>pRcm;qE z4T~Tsz1@PgFJQBSi2ZseHQuHY_@L-^I`Wt2gIdS=FAN6Q`PH?vX8ODEQ0C5sMByE8 z>x|b7c8@KHN@A_OL^iYN2P?T<7q;ehaPB(hNv3inOT$+pHZ~rV=uIFLzQoh z7Ykr17#XbeQ|`0jf^R$)bC>wC8&5Nw+w(Bq$^dlac|YJ&dl~GpYhXL0x!0rh7kt1J zePXL5+2fed^^}=1>-AUW3!5`yPd_C3+bjG16Vo@vwtGmRv~oHmYoq%mD6Dme5@uu% z6dzWZK2Vdwf_w~gCKl2Sb$DmHE@LHM#U2vFU2HadtjDDuD!41S)14WBU$z90v*JP; zE0qLr*}AW1EA~cep-DxF9rV9;AxN z>_o9JE*t?ywP;P7ou6XfZu)l9Hbza^@{8{lZv${`fcfkRr9zd}m$LOw{9%W{YX?2h zH|n8*NL`I7ASczRrNwI_!{S9xu;X{txYURmnt;D?*hm+@>?D?l^Y6zNd|Qm}m&L#6 z7wj3Z+9;4*VX;zrWO3UP#-w@7qTZ%fXp?BE>ul1Kc998{T=` z;!{{LB2&&bI`kf1(|&-nB@TUbKvrua2?NGL4|XBZv&U*iVZ;8*M;3cris{B?buK_s z#kUF5xlN;$W*W~~_p`U_waC2$#_AarJ<7$a^W|mI{saBpGvLvOa#H?gR%C;@@*`z{ zWp3G&F{$nHXH)jvqyA%*IJA1k+%YcLX7Ra>lUU&iBy3mR1a#7*-R*w_$sc~-BRM!DFSsxH!x#DN6jg1&7m zkO+u7*{~IJ_z&IcGd8%MYT4h+_G&t=Pdsb_h>`;NNiyafO|}Q+Vz(ajM3z^u zp^@+)RT7v1?2k$RjIA1Di`Yi4f?yaY7sz^A;pD@&RsKU#vWW%$Jw{(|>+!ORfqoNr-k%g%m|7SH{lAT?}MGs=2JKQ>ikuA6WQ-C$?au;|bYjniT49R^oIh}ey z1`JMRmCI;0G#qO#wk>))>34!;9>x)ANzIo4(ef`#>sbT$sx6v6Fzj!AlRzfUS9^+a z@4v7bHV2WT%_p^u=5}M+9`Zb7wj}231d-f-Ni=RNUBdd3v`zqJzH_lts>mUpe)pNS zExh+!ktiTZ2zZ-dlmkZ3D2Yc8^6?>-=FueWpEJogV^NCxF=PpY6AL*|Q!P)*wX@?~ z@fJ7FaqL(8@L`Sx*=K2POuF^Lq2UFJDtG(W8cpT$<=! zqy100eYeWJGyB>lobnv@TVO1KW3J0z`guh5XR8ccj>y~V)HF#81?#kNi?`Q{->=Zt z4$DW-FhQy>EWXxogBIY$P)h78E61`+;N>iK$LzLY@;t@XI6!ae^44EQuGyNp92;(e zEJuHzP$l~~BN&&LFbYz;$`W>X3VKP#w?$k`UI_gGZFmDdSFAn>Bp?n`y3228XG8Me zV`?pasFLJ>C1iM$IY^P;Jj`<-g*$VJ{~{iqPB%5)rnnR78bvlS`-ka^=+YC9$L4n! z#0mAN#3*RR-EFoy2Xb*4gw6lmY%a&|{7Z&Vs5@DB#n*T@BniW|1xDZ3>8&X*{)UmzGw5NjLJR@Ahoy4yIebb2?@a5P5$bmV(7TkLfv@( zBAYj-K-J&A!GnKtV?Zh9AE*1F^-i#QDKKAK@H!gAGF23eRRg(Z(z)($3Gg9mfFR#- z>g{SzSubB<=yXnzsixk*of3S%SfjU|P;0o}WZ+NSpT<3RY*cxphY(gZ!1khQrf(zJ zs2@F|x7s;OoZofRO;!7f$VTwfzoA7nycZ&hvJ6>kNN2qePS|eW(Chk=_muI%O6-}3 zT)EPgI=eDKkTc4(jPAn7tL98{_$fRJn05*O3vOsqcWz^~7Z6bjKu*Vlt=$x7^=!W< zv_9EYiD-ID8TaDBUe58nBAc&5I*ZX(#%d6Wr zEPK|+hXf}6{_B7*&yiH*c6dbTIVr3htj0~WLmc1$Zv&AJe0rzfuu=TxxVtawvbw@0 z8P(vGTr~(qJ%bH=)pgUjI3567mDWYl%|!r-G#UcO$E^&EMv&inysD%>?Wc?DKW)&q z7-5n7yyq=Ec%9;BJ_>6bF$9WsX23(a(aj_|Zqp?Dn=kQ=h=}kS5};n5yxxlw7{To1 z;1f034)xwRf(;`|->HIW$!FyX-WN4OPZ*gymU639DlVV@bou18qVP=d*V9^z;rKAE zgNCoQO(xej9nB0W>~RW97Rgdj;KmHkIXBhzl#FuVF}H{Z1g`CWYUWdBP|X&r<2ty&u3f zzLG|W&uy^Z83A*1-1lo97g}LQ4Dt;lHrYm#Y+|3g})?#l)-V-N&9TTa@ zb8P}DWYWNlgEef2(Xd1=#C<~;kXNO(yEaCVzn}AHbcp=d5iL8IT^g3k#8X(`2!gU_ zH-_GR`an$;-|t6r`5&c&*#NH{xTlV?FrJ-Md-j@bB{hGh4ggtaIgyHhJ}LNR`dgVn z-sq*sp9;{hN?4%?BDnllHeQ-9(tw_@Gdb#mHHjY^wzeO=u05%#lg0{BbPg%Z7){e! zz6Z;=mWi)*=#7YEs;w{i4oF{AoByZ@EGV9QHFfHvV349{*%*7foMP`@r6ocg>O9@TEV41SYif!ck_oo461eP zHF4CIExkJCi!Zi>xQrU8pG)f7Mu0EUn*dPi*@Q8DtWBfWKPkTG4T~ZV!pwOpg0D+Y zmH-o3+2BCjT+{NMqKNsOdEP5;O8Unhz+eZ|P8#=f&)sHC;+Xhe`uX+2M?yA#2^JuA z&f@C_dIYj0>`9_Yq_$fzIA?yw;VgY4XXrbEr0uaFJ=G`8y8}!B1*%Z+Vzj}aKNmFa zG4h=im2>d%ZuM29B;)@OR^ao$@77N?LUUm?j(Ad_M{W!{2;?F^wg6uPu`!ynYF?=| z<$3p4tWpT=_g%&d>*X&f&|DlIpFq#2?rQUI2`Pz3cwGdd66JncslYkw-KX(Jruc-@$!1^8HXz(z z7K*MaH8=tQsheM0K6@0F_{`V;yNG6LQ703^%%QLG-UH^`*g-2OLnngz5r4u!1*Bu< z4^wP>AKx-t4SvcHs!R+dko9%V3Ucs`S?VYAU30=Xd~f>D zN>KoM_722yX5em@EPH)Nz@D9kE&n+>;O@V*G`)n(h7 zwxA{$J%macGp_KYGt&gqW^CDmT_zF_nLecJaTnGo@m{`s;&-k6hi^GCM5WuR_uh%yYr*o-1@Exyt6fSD1pu$99P-D3qmx#xxk!WIY&FF6(u==(OtjOYP!k}n8_7i9(iR|~+`6)?Xa^G-Chy-X4$p$34Rrr&^*pw`3s|MXBz z-|*2RqImI7LELtOk4ffv0%|I>Q$rm$)+%;uMrmpxp!8IBuMSN)g<&lYcs(PD*)BkIfxo%iKilfKcZBu9o;Jj z@ZSk>7QwTS0)&!IPuORf_fH9H1R3>lq2<%kjk!)j&r8;hT6sJvto`~;nJC_H5Y+21 zh{N!xb6vh4CnWyj8_vHJw3+W{C~wn6t}}L(LAuz8vO@YNn99~w1BVj*$8LtFCjRmW z1D@%LDDIZJ zGHT&4(-QosJm{TJ)&jHTxz0@oNOm38?^%xasc4oz;=R0kQ`6@T1bbkKtw>N8oi5JA za!&P1-7TNT$@0IXRo%))gzQ?K&51{x_aYsG1hVcch%m}F2+nMPLU~q50NvxWs8Y#E zn>CM9>g+f|7~{s>>Zh0sajlQz=-(>4#Ej+)fQJR!D?1#XpxX}yC4tNvLQUnSkuy-f z(2CmNnQ#Bj&TdH3yodLcMciINget+qaFdjpf8cQdriq~tAxwgP76ANG$(1RVfz(>I zLF8cjTx}Td7WayG+jsnhKV6MBw$*m;<#&r zZT?wUEc(r(j#i#yn*_`#+IJ~#O{e0t>x+VzvES8sh+(2Z6p?E_JSG@<8>;)bmTk ze)HHuTe=3pV(lrcv}i^zS6C=BAf=*AY=VN2g;9uETIyKxG&QaX^6+S9bLkBg;q;&Y zC9+1H_@TYHD5#y!BJa|HP*3$WGF>;vA}!%7l+lBkU`$CE(9T53s{H`Nx$=K)9|R#| z53L772%GJF0UgE6<~a3%9+2}Pu)S71HG;E3ro8k;a9?56VR}qxvuiG# z2#>lt&{F8Pnf-oB1iw?Zr$;xg&J8%Khs)m%LVVb=fZ(NM0U3M6!w$LB17-la#AGfZsDN|H6ly1P7%Z^ub~bYGv+ZFj)z+GK&4PuQmb zNnH7H1xHL`e;?Iv+3E*Rc!t?3fOGouX13GF)cZF#5geWeam@e_zuR^lJ>P#X1eC_Y zVcn+o^FUw3;IaT`N9L)M@-#83Y>nH-Z?CdGiMWcqD8lTR=X!Mpcr5CF)gZj!AozSC z@`&^#y`nWGs3lQoM%m8yf)e;U=N4aID*0qc1}|5(i3^vNWswkA0g|VZsi9!bnJ!Fi z{KTk%BvVbN4Nm(gXLiZ-xInz%kMG@i*C+x5fiXmU-j1X^-ssy96&?j7S-!PW3XXVY z1u>=jD^K(GWAo$ufT{TH=y6EM2xKB?l}wAeI>pW8f1CDe?1mp7K|9egqzAxOj0$_i zR=DSqvD64VJs_pE%a7%=NOE7)v{~#3Xty3G8u`^!@DC*JueweYm^36jdRhUvPKz@` zSUAU9!#_AouykU>!RYAezr0ny-T#R-@cnK#kKd;%)m2mQ#xx#Qj75u$aAWI+M?#NLmF|lwD{3ddP^o35jH*0M}!ajg1;oPCf7$T)6h^=&3G-v z5jMkZ?Onh$%4cJ;79oXolW&+WEhzFW71Rhm>GF~L_^!hyYlKtY*Chpk2UvRiq}+{A ziazM*L-!k(AO?j4>Z01EDBA-X#p>0ka^ws}?>en@?9u#SL-N1HuFu%%eYzRt#JXj? ztRfGBoPDCGjN+H#;#mYTNberRl*7a{&6aiEAj6&FnB^86iX07N66U+7Y)fr3;I(s= zup1w4pW8QcF@vP;^FFFAui@n1gu9CJ3qY4-RG&V;P4iI!YSY335AWE3?cP4K#p81N z3f4+5rrU8)5-hGIn;^#JC{f^{?_M(|6R`df&%?UNWOO8=zVvAERge2urXV-jwFXhh z&6$BeRt(BxLIl zh2}e_M>3NF6NB9NpcU}52-tmoUs(LaAcPr{D8o$RWFV(%)T`Hb9?x)=Y#H@I_(-bDcNjcT^;^_sUX-*m?4R>H25+doQn)R~|+iHNUzt zdgnVqeSNSyhshw3nQab{03>wRqqPshE^F7-^8@caQR;WdWs#8K``?{Dl_ut4bfB{y zbLei>U~vTh?_P1jh$2~{;_IxJJ|YxdU200hxj@+XrRCMr8LI*9#fW`FXF!#661rF{ z$#EvqOc+=jj0H*+GTQHgchl%^>{A=N4zY#nPj~q^c)4dVMO! z$5hFXTkSt zrwi(291>-gt0_D=-f~pxf3*WB&Vflh22${(cQ%MaM3YzFIe}=j>jb|&rQ$tC8c!gZ zj7yN8R2QFCe{B(;R&?p}2X;2*aA(IYX;M_jwE>7|ue#U`))@~SI|2;#&RstzSjP2>YY%5AKK>9LQ1a_3arF|YWZH9G@Oyw*uoWxH`s(Q=ycgNj+&)FJ;vHN#L+M`e<}&+$TFVL*n@QS_>{xI^ z5uBW`iEi@18;b2Q-vqNNTfGgNK-=mCH&6eVzL+osls zzKf%OBH7fEYfe#_$kof zdQ#y9Jay@@Xk&;O)wv(l3$r?@-4!~@euoheI_-IMR5+%07Ku&~knBlI&7(@iNK9c@ zQw>~Vw?U6-(71QqV908sx$e1iKT7sj( zoU3uYL*>s&PK)i>5n8rJ;mlV>^0HttFUtzju>mXjc}k#RUg=?=^t?Wae7lk>+GAvZ zaU5=yU9CsfW0`HeY!gv9XieO6C z*B&B9M35V6&q*uK0A0x(i;6h%-h{+$Pcg~COf!|#{q+!kP@+b47mXq9L zs6Hr%%q^VTE+qkAi_7@#)7=23D3>6p%XBA(MRcii1=-+Kl(ev@jV;je*^%S}y2gG9 z)Zkv*Asgc@+m2HLzJ@|6;Qo_MQ#ZX|p2?f}4w8~`Jzc3NG8}`u4A1N*$&x)CcbnL? zBOt0?PhCzla@phc*K4?|_25kW-@&Vd-ZemoHHR^|5e%ZX%uDPDwtUqv6Pyx~7h*}4zn3J9>KgQG(CUOI1_+=0pvOTDX@ zS=S(qg!Rz)e`?n6?%VOM7}{j~WV~C1_Tl(CAV_2E!~jch+ztXk+>0W_Hm%RYdDI?X zwx|%0#`|oryXjN)V2pPDOy>!hc(jml#>rq%a}ZL13W4pOLV<+~ylcDYo3PTSO|(lw zZTUEFzJ{g6lp_A?^fGvKQ4l7@tYAZvVq4r3>2LP1jpurfFGzR~8LQf3eJFOsIBdTq zN7<+e@Y92SjP=)MT*36zuC)ersb3h(a)JYkNt;#o^Mie0tTb z0&l?6Xg%iZuW!5FZD;|S%ZWocU2-Orz)^1AwjhU>a z0QMKRp*)da-q1{c@ZBO&cW5N;dgXUUjMA^Cq_k~P13&xfMvOx8$41Ap+9V~ws+}tG^tje0*) zekpF6zw-dG#3gw!0Hd>&w%3mfc~M`z``c(W;m~cEx=HbmZPB@QqTSD^jnj8g2ck8R zdv<$GJ}tEsrcD)H74exo1G0w^W|&4u{>?Q^)>=959bCNzh>f&DwMY z^vQGrOXTW+9+o)XVYWi6mqx_aJm_qfdH7tPYzjNoJDMn%{m^}c90-!gwD7Bme?VF} zbt58GSerQ7=QDc!$CfmI+1*-IP`3NA#fSPterK5o^3^$>De$WjWRRKNM(;Z2qI<(| z;;-+MmGBoE@9YM$C)$tFZ7a-1YU#$FF-zf!L(CmV5Vn@)MYQ->2$Z|Ma+P@{MN#`xJXLSg{Qy89+$wkHMM^M_vDQ8?r3FbLIX1Q zt=>o3+4yAgRLZr2k%Jeka)7Oa*DC2WS@W?pqZ!`(TqURJQ7!}Y&+w}Eol@JVgDuH~ zkPih(gvks{WztKgJ3mEt)=*>a9~E2%AggRf2E*wjSz!zB=K@B1LbU{51`((m5nS0h zh%7z%m-N!A=11_d@1kLqJphO?DCdGy!^re4k8PcP+)IFht@H_*pGD^+lka*WI8913Hg-qaF|E^7s=_z6bHt#@mrXr|L8HQvn+@lAo(N3^x)dA0`A=`$pE z!J$3IV53&PE#!?pRcQobf1po13_r`3Xw%~Ze8;-lWBuoZir#pItx?K7F z`2A}dC&mQ93=cQqCcI^Z%W(<*E5xa}!5@7dD=@w)cPd_Do9S-bix=+g9r%sDBPSOo?5029fHy9JlKh>}nBIVqN;rD@% zuVotXTMWTz@-#4un)us3!NbMZ^U2aweTO;sKmzN&F`r&FAyKo9lWv zBBWQq&vLe{gN^84OblA!;bDYeg&0)_DZGfR*f|2w!_JWGHBBoT;L4cs`k9Oj~7S1pdatMz==Fu=u5{?VJ$(UIhCeNs|* zHO))gA3{ zb;0D6nQUtzt0rxcj|Tl@9-}q3L*k=l+-HB31OiD&!f)6XKcD?an;(JBm@wXAU_uCM zdu`BE_++l+o)kEB4i#8a8``|!M=_-?x)X-#{gsjqtzYZo|3xtCsFT=_XFj6Co2H^{ zSyAw(2Bci49Jwu6N}x&R_n6rjB&nxNGecbTse4?zL#w+2FYdf^#>ASoF=?m*gCaWP z5{rNRmZ1EVYGA!8JmIRu?`wM=xcsV+>R8c<_G?oMb%2L&PTKftS$m$I?oUVYj^=JnKw`w6l?+_y*^J$0BHpZzYA z#wb(EgK)sm`t$=+Rv7;RJLre&_->w*)GXLb4Pe%w;xPAO5THJi$DfECMLzkHnh^c~ z7Snp94QpHjD*U{e6B3r2Wg;(yH<6r`0u2NpU1B9wh5B68e`Exq$HlF0`<}g9l0B(E z)2py{fbhvZWqB%2R^85F*A7cSYv;4sGmcsp7vrd>j7y(%J7Z^y+NRv zN?UgC3mYY3JQC|nphA58(z`W^u(hn=iEtE|V@syDV%v9K z4mA5ufRnzu(R3Kch~4@+%SJTCu=<#|^l+tHda1yOOMu{)smz#l5zu5iDKC!m5M~L0 zz6rMDXHR9p(q3?^h^O4$1;#F9Xg9DX>&WY&3Z}fd*87VH|N&gf|{t z{gO?z8Yxm!4N9iiecyMJdGhsZRf6uH@}|=k?e#by*!V^Eq>1<|<>_br1g0&&k|&Jc zu=8$>3;9)?#^MZu{k!?-c-#{EU|qgBQ}-oH*_b;|5ltR3!ASb7r!DG?YNbUJ7p-+< zEEAW4c-qspUlv?mnH?((md;t9Bv}WkLx6ttK z>7o1ijJU0g1*epnn(cLbMbymJXdm~eRN3wi#e?IYZ*a7OPd*@zV~hT$DIutX9=p?g zGi0R5+)dL&=i;9yi@W8p0&xD?YO%NS)5ZcX+i*o123kR|rm`@w$O4SJ~%h;*Nd5(W>K}wn~}_@Mn}o;{1@c$ z?G#f>4itOOCw|$ahcffPZB-d1Lgc{Dy+d+skF%cE z4JuB|e%8ra5FY@EAbYq)kHT(t^bpTh zNS#<8XDwcsaAoMjEd=freIfsIURnz_*8o3=%2G zI&5NAk+0w49S)nIEd%R97V04=_ zoK947O5SqKR@w@C`yprf3H#lFs)?}aqrtHMznwoP`;&-(k#X_jg0|Uog7L zor!LZ^}S~S#EzQ|6W<-{w(~E zLcW@{B7RyDuct;6fe}M^QK>VJZM3~**6+mokt)6P!j^>+M?2eJ0?%x(x#(b#o>yBWzbia!=>V_|r-gVzw#uy|Ao7+;UknQI;EYCw! zw0h&>>xr->?p?!tEX}L>IJoiUcy}DvbZ`aplst0w!9n^Yj2J}9*fNw1U2=Frb>M8ev@d_3c~-P-6thC zMXElH{$jq-`L&OkdD-s2KYapU&bG|c{@q1U0eW$oKo1< zq834T&7KfrNDM%a-Iny$`nkImpgA^YTj7P_UxME-Z z+Wga?R&emvSOcMeF^V0v4y@lcJ^#+j>8mP+efPL50>jA`;ML*txHa4`0D^wbv&bY; z?=woc91IPHdd@(EI6mpWcod%UO_-+@Vx~p7_M`5V_iutm>Z`xtt*RwTzM{RfxE)R~ z7tHMfBnd-hPH4i8vK8&Oh0ZsHgAl8$s&3ptnpc-(Ctq(54b2t8yHz%+1fm@TvwnOW z$eAQa0+tG$B^2HY{+#0jYeofA;+~p^?L~Sv_rThm3WTBCoE)7VfJso5w=d`1)%b|q zn^?cycXQA4l^`=ogH<2{u45yvl|e{&x}&ohK{yr%NvF=YlA|&fz^8chye6;tQTI@V z8IJAQ$TLjZfsP<9$6_hkiPQ6&ww|H@$qCrz?ja+2!VjW0unn9V#NPMcWjym?cS@;Buo#>Fi#>8{_pjD=jG2Gy{q z>5;3t$Y8FVVz)%an-+FEw0){A&7{WVKxMCevo)F~;#LcfZ!&;xp*pw#{&4XE-n=zO zHS!!7u|}jl{rrNQmpm&t1P6Ln{U&$996=py-Fl-s$l&+bpYxP~@cNNm=Z;;hTSmFgEq7~up|uZew7FQv@m_D zB#}AB(ESVLpV7|q>HD#&|3`n<{oM)P#_j=fS^G)*S0(nbHSiPSY%OikU6H`hKB-V9?0-KQIjg&@lR}Jx5 zs)b{xG}cE;9MI)@O{PFsQDv_zrtXAR0cUsoLMl+qM4}^8uTL3k=rMQ^m+2;2>c|U3 z7}yEIi@_Ki3yV6^kl#I!JBVp6s^^Te75+Dny=BFg_@)T>Y18`#&tET!cN2Q7Y{YTr zaw2~x;ZID#m6**pb{>3hNgmcsQ>+6a_L0YJH(tSKSrDQTM?7;D+o5W?jci_CMj&06Q4>| zGEco#wbB%TyRKy&+C}o~LTIAfPy-I7O{ZwwPD!fudvo7CzS%JE5a{0@P%P=(KmKX% z8V2>kTX&e_eOc@NYcM1gvRTzw>^CB6mg zG5Ps_0JaLzuXjf_q+H{se+nwJzHmx0P);G-VqrpEM7I8Q>yrNbjiW6wxAwuaOw4v| z2JVAgw@0eC-=QS+LzsYEadmfP)tduS*wk7VPet{Zt&{DZj8A zG%*Kq2>({coC^S7bDYeB=}#d6GlqstA}@SlZ?0h`Sk(s(u?t{S-|(Wx(SOe29P!9` z9&&8Z>ByMi&+UFm#>|6K0}DM}~? zaXh=7Arqb#r~08=Nbt(k+s4r|GH_;y*FJ!KolCLsZ3*Wo+FJ@C=|4g!!MmAL89V#@ zuUzwfYJGW2!N-o!zHnOC65H_lo06(1sWOG$SAKq>Xd2blYOiux=f56eKc6&e$<$N3~?}?TR`IFBf0o0(LJ)sjW6V5b@EaTGX4!>UOAPx5}koc6+ ziT~A;2PCw}9<0=69%O6@*2rQ@d6oK|QEgO3{FU=&90e+RNWpm0=VW`0V7&4fL8O`e z)w59^lc2$qKP^YqBSriZKifSKcVrZ78vce|{=1;uK0>{I@8vX9rviNaNE*bORMO&omc7NJ{WdN{Qb-Sv@Qp&(MGcI8P55Z82H~@IUt{NnMKT(wjqX+0NEdExM~8If`5_ zRGY%co|!GW?^)5H+ST<^TT=X*dp|H~!6s%RFWqd5%(4mQ9$AL&MldafIP#6#-}j`L z5Q^1Dq^s)Ii=he^?hKXMaKym0=ebyP(zK(F6IWMvIeSaEwYb1)umK*}>EAW%1tB{; ziZi6(ILnX3;30gU5-gNzHo(X6?#H?*4>wf)LD0F(7g_k&djwj)xqBoZJMI`4{;2%7vvEbi&pX zgAo(WK}ydsT#zExkB_xvcWO`R9GK^)feB$IY~@ls)J8f5zjt2igg9qS8$ zX(})Z2xz4UBWEVyQcDaqsC?aKYJ-eGsiQ%apMO7|%#js*s3TkSfZ^!NaXegjG_QxU zisQe}1#5WChws|au1;pY79>>;lS0(4IPCIUy}X<46VWy#Ii5=6KGP|h)X%WNgEt*; zDq}kt%;2@o^Gkhc-R-u_Bu`IjI2+xulD)SYjcS)qFa5G4|A0V~qn-hEtp)-64o<=f zYL+c#=tFHtW|OM#7`F??V9n~mQMj{j$xLQOZ1G#;f}wkhDu!u@VQP%Z%9%%^{Zw3Y z#hc6rVARa{L7YXXV-+gt$pwmu8rR{#C}>2$5hU4>6MGTkB*6<&WmnJrC|0MvZLf2y z>J#TD!8YFH21ame<$}>zw#Z`K6xa_jX8;cE>w3aNMkhc8e4;hzpA#1S1rIt}eiyfu zDoES=W!K6^zM>|sSwngt%JH4BW6U?`eAS|AL{gt4%BU3zw}(U7h}@w3qu!O7gk(-G zLU}ZyV-u|-!CYGe?XeI)m^q|A(=$WD!RAXY>nt58*ZW*AX8W%Fl)Zdi_-Rg}glB`x zXyvbk?`<^U53Z$zi#LFu{6?HnV*Q}ls5a~$dSD`8 zb(c=78uQ{lE>!xtWU3z=8IWiP+;g4Ri#+ojgqSLL__2WKu=d>1;!Oo>Ci>#;m_m(! zg!9~8TqQ>SAl8pdu9YX?W~!A((tu>yp4FgC`!JhE;NE8zSPdet6#u(V=jG_t z0>fP`dnV`iO@kf?hg1kf^7Al^$cKl|=GO)YNR8UW}48dM~u8~7h|XH1uU%HaS1rvhpW zipS@}a^%nwt-4G!VwSK=dRIp+F4^vap4>#?U{9c`V1!N?zYD|=ht|JY$TCf(eLY7n zWa6b5Sp=Y2{otu+2s&0lW-8a>Cd=7SFXUfmxBz4qQ6`xMcstI}Y)gLxSl0Qk3#Ws7 zzAvi55WbYRN&04Egi2hFHE;o2FA^ZrlYD*cvz z{5za_#M_Y?zqH3dGu&#wuKc^du)YQSpTB0RQ%>vFw$8FN_fqd;dCwIWf2Ar22c^`w zf84c(+*?0io?+igzmzbovK1Jn7UB=oRCv3n@b-YbDZBs4M~X=OsH=hHVQAW*9~hA~ zsE+OR^L&$Og%7h&_G1!>LGxhT6?xitB8yUO*{1!s)k=Rs2bnthqiGKNL2w(HzrP4$ zq3Im)HHp-k-@q0R*O+e(ELrWlSNc>7cFTkb?aLG)k`Gn1ADrrXj)k?$5Mg66H?EB< z{?t0^X`a!ubOzdK(oaC>9M)J@1(cSo+y~P($p8Ck?1QNOTh*+eGfNbN#KtFyJp}SZ z8MI_c3+vK*y&+{$z^-Ngjzm*zpaTOweO5T%6{suI)p0#bX?yQ382*)GPx1ed^_5XkhTYmT zbhmU$E8U&aAf-q%bW2Dxv~-8k-H3E|Ba+h10MgwJ-{V=|`{O-lJ%3rOnKk#b@4c^m zMFsv{y{+?{?>Yz^bFk^B@e?43cy-TLn`XLo-|MKP^QkCchnyc*dt&kkjh$_9gFXni zGcWV`k|v3j$P?z=Ru57JDY3Wnq_sM%b#R#Up%jxt*;@~@v2Ne_yWz>(z^9MYA|Av5 zp3F}Eyg4P`ermuIvfX!~LKUCaUcTcx&S8F%lWu6KW32nUgf;fVh$4cZI=>VzWhmM% zOPG%%V(Fcb>ZIUJJq#Zi6}3t5EM@W(2oaQdGgG1tPnr58dC85eVb_s&6@RqU*emvM zn7EPLaK+WF(AI7K!zC4pmm(ji2oo5Rd#b(I0@-7Vk8;O~4?2#b`@AAn+mL2)nXdS31_2X^9Pl%z)-l7Og7Ul#xU4i@tZk*^qvah?SvIT@c$pD- zedI0E(uqZ77e#H+1`G}onm=E^yiT^M&=f);_@8U{-_`q{Ck~kf0;l%VdzCmq39m#_ zrr+Sn!|^@*>?z1cBQ7#9E{#8DR3ekXDs)p3_^sb^`rS^p+HIjTq)2kBOsLhglf3%| z!m0TYIXVFEk9UB%L&pEeK+OMWVfCj-beZqiiSsCzp-Q=7<{k|@V{vtMkicrD>|>;) z{|556uS>_ND57k3G{-Jvza;q2Y{l(|LGFpyGkJwZ-C^OH>%%e^kkPVlYNsdS5Lj2h zM6IRj4-AcotYY$Buzr-d7$EkoY&AnuYjLpvQQ6P@JfUW~9w#Ig@o-d}t4=if3SY(_ z3sO99AGc7AnLJdtU20(c*Ep%)x64El_GizPt`TpKWyB+L?D0e4qR8iY_uYFj`QI(* zB#?^R(yamA-OdEu;x@n?@nJY!$g%%KEP7}m(D9fX?xpbD>9HcN_VXS1;Au3SKh$~a zwXX{C>f|BS^aE|-?RA_+eZecviwAjEY;^pXIS#J&gRP;*Oamg z4waTJ-oa1Uue7|?)GLdz9;m0Ki+{l# z85^tftGg+%%)-9yd#HH1bnUtotB(s*g_Z|nwf4a2q^O}t(ev8x^JC@&eX$y(^{jJ^ zWfG+tRWH@3c~1(8+U{4^-@7{%QpIEnka=ujD9quZoJ-)W`g}act~6Q<65Dep3sn0s zA_iwIjZTR2f1de5!m7gWav()CDNX8M{Z{J%Z;r^O5I)dxax#^K$xh80te4|`;)tph zEC3}Fuxv%zpr_HKuLJ&!A*y08NZ+4Yu*sdfIykn|Dkk#wQvaoHhKU(_(~9jepfc4W z2GWBxxWxlLYlyx@yX6H^G-jJk&kVEe#-!=ey!3ELuly2aRC zc_w#iuaefl(T5kaUuwWKgr3O|e_4AvojtfHzhE_NBaRV&zD=^_;|tzP7qQQ&JJ~a= zNwl?pxj$fM`cR@KvU19}E%6dL7UpYI1(8HxIMbC)Eb2u0c)%Om-MHQxD%VBY(BgW~ z^H1*n6jdI7A#@X>qiQj``FXjCb0j4mRVsJfRIyL)^AKnYNYBF}=731#iLWD?$hAP% zkf9r;J^Iv{A)dEDvZKV-?{AxRvA>{RZmT_EtSQrUr>K?VMlDEQS;rQse{NBvX2crb zCW$taObggK%y0+R?kUH`22BD7>AFz09f&QYTDxFoX;KxB?&7z3pwh?ly}K*E1rCJ& zfZVFjjX?ojZ=Eik zC4p`N($fIsAs$#F$=AWsQHTIC0i8d}$&Ts9Nd+V&WK?o{Vpr8?G=~IE6>wDrEd#b5 zPBx5FBX70J{qtjl-u%J!Sv>|uvfjCua_Vp0C2RI^wr zozK2z-hW99wcxL9thg;S_~09XzRWBfmlO(*^<)`q+v6d@Ou+ggd`3VYSmW{)hB6=x z!av>3kL{`akVa!HTi*B;Q*#k8H~V-xbL~TWEs0kf;4fa2%z&comglW++iEdND)6|5 zAw-WQ3O8GE$0u{{(D_6-W-|---M2yD8M;f$-s76g7wfyT_keS23yk)cf=8u|46%Vu z{5Rnz^f)$K4X@)@sZ=OX-B%{qTVkw1K%SHF6{CNZ6)L{6bw6hn>a|y9T1=kw zWY{yoml)pvB3f*B)$siM;2<&JhzBV`RInJ&!cSz^1-Jdwt)`|dA< z5B<%?@hv;H;vy+~{I*R+$Xh^ZWJd-MQ%T8dJ@6sYldbr|M}Cog`wAp+r^|AwS|{;= zYPTqT<3NYWoAeqpKvf_jg!!w{J09S$Fc#MIiXlu=%YPMbHJjkbc2s6fk1R$?0@34_ zBpl|b2+Q+zyJ9+&3e2(ACbG$O~ zSEIWvG^USbNgh@y-5$R1h!URr>T9ymq>}#zb0A3~ zd|r%o$q?=+MC5x<5p2`(bf_PVXQ#*~EL*8QFU$9c)@rFrmZvYg{(Z;|QAeW8H`Eh{ zz(2|SA>(4dl}kfMB$U6*x47DRTrXJWdmDOgy%GTI_e@Xm#e3_$u9~NLd{p!rgRUD@ zV&K-;#ROPZ<@o$IV8sY;9uB>eqOVJAu~U5G5&Z2}+eom4A5m`Ym1OO2Kf{0i`f&U! zN-#}!tJ^1CeCq1W!-@YHtwZobh-Fl8RGn$8D4z-CKaa1|QU0|qA|oj7-5k({n_u;g zWeUqhjRM7GaL(RqT?Q_nXAfzVo2BIiV&EmesL3OX_d``kwd(Uc&*WNMszS&afmS7IjJzd#V=F?DO#l}l0J3j$27VCXg>8?Llp-eDnZte5%~3Xei)1HbeIpyAtaET0>N)@TzbBZxA-Faq97OH& zL|@5dGxQX2a-TA#;wjj211!P(ExkzG58jaM>tD(DM38($B`CbH#f05Ti|fDPqECX& zDA%MvLZ*8S9$SQCxW{_lj8tdTzm;H_PLl345}wS$G4^fm&fxf&{oPv|!cf+i8YqXk z@bd$3*j^XS10nPX7juW;k63S!9;h5~$UojS^$|cix<5uAAIw#Ta+weJQ-du3HjNQs zIj1~%bP*qUnFU;(_6}Y^fYi6s>m>)h(0(4sr%MuqxvPDIT}BI95Sq9jbPvnQs2TEI zj1GUs^2|JJozmqe>)o0r`&iJD5MRW-$vvN$#hGk}{%K?OdX!f&Fv&o14T@a6|9LOl zAuH=>{ViF4R234ARPmW!1MAc!fDkR${K}QMiR`Fb=ZBvNYXUD#JB`#jpHS1Q$IJ1Q0Vl~L! zQWTM0ROn5lcqa6G2v^PH7fq;{g0b~|r>Af8Sgp&LPqt<311mj7dr?L}YH@)t9Qjwp z#Ly`R?SV~i8)+SC27@JbTlAK~-6k!$8^*E!cXiV48iad1`F?6JH|TtqUx^JW1d9uR z*3rclQius2dtjA)O)xOq{#!%=C#c=z{311Zj5*S1l`nP?@@7s?{NRV+F=PL2e)xDD6|nfrcO^k`~!u#i|J zziry*&Hb3{8$ZU2wer?!^#zj{!StA%(7^9ubIVb&ep*OZME*cg!TNpI4UOTK!h`${ zrPWb{93VR^kLepVFFsV*W9f~RM!Ovp@ZInW1@O)R@9k#y@7>?wxoi@a;C6^yj}Q|X zqoOO;oQGLCJWgR@Pds+ofalMn6i(EMDqM9zSBENad_cArk#r=4mHO~AF~7_I0Qt}C z-8z<;-?E6J$v;%kHm} zl#>x#=g7&gM;(v)jjoQ6>CdcFo+ohAUDJmU*2KnvuN*p7SvG8Rbag`rMxz~0A*^pv z)n4Z)PJOb~ObVzj8>#OdUdKLPZd(ep``^`QPussXyXR)bg(ao&y#zn-%t(6S-oq3;&K+=sQ`2vPy1@yh3Rx@{PcKd ztvB|qSing~b@8p@q5(eR`>*dag$nhrQKcrM@k&CTEZSe_c`7|!MCO245&QtPifu2l z>=Sz3xt6U6JzEij#dfNU{v)J-ck$nqlL9j4|t)>O!ZGV2c9(OL(BQg&nfqqnjOXuK!6AB1mQTNU88s>vEnC@ zRWz|&XroF7R=&68ZUi$5G6peDF>T+tV+OFoF)v-&)b(TjmTAq1dHdeC+kQ78cO5ag z*H&{-PIE!qOdo0J<&Grq%UFg0`EiFQM5M1P0Dc{)ps{}L0TSXbk6#c%oYmT=MsIqn zepH5aL8o5qAxQ@raoYNPOPd}|;&hOY7a39yE)Kew9oh-0aJ7Wy?%Qpq`)?;?j?430+;QGkwKOIBin)EEqN@6ey&L* zI>!u|DxFJfX7nZ7Id<+$>QBw|`EN}_;;)$hXIlGZ6VT<0W@!A}Zlly~iJq4BeG@Fl z43h6=Vur{&D4F^(qsW+Uyy%`P43|j>RyAu=d8`mt8MeCFD`rxgi5ZAni%qjN8WD1U z4=Pyl&z510rt!jKWfk{IP3*?BWcnEDY2Tht#g_mG39@qMj!gxDHkAGB zu9=PJ+5o?}2Q_->9oPqTy4Ty$o-u|5EAiwv=v9^P=*Le6ubxKCdpzj}sODlHjdb-A zzz;OG<`?IU;Hh0>X%6e_!v*FJyTzAyIm&RjiwZMz9 z@^tg@IKzkIc?8;)dIaTfx4@ss?!j8zPp~!z%sHThthxw{nJyY=>C(D>R5)L+Q3`qM zGC%QkIu4z}-ez~bvdqL`?@ypt$^l?Xt+~Xsrv zTwwJ!U&CZE4tuDDCzg?a!no7I*2;SP8A(UDYaN4f3Pg0r;Mw>-5vio-u1+G=(Mqr!l$1RUHC{w z%AnuSL{SALj7ORr>26CR%^>6kpe+}Zbl^tWL(8};z~dhLrZrC}uw~tKz0gR!d^PDT z(Ei(4ivhi8WfS8od*>rw!?6)fnfE3fj05sG%--3hspJ=m7D|O-`K__oN{I;idKLre z=jb-P*!ttiXa+)1i^1zut}>aaIu&Fl&3Y{T)i%D71+~|lIL8+aP;wT7N_f}L&&S$)sH{4x`A`Bmn1$^?Adi&Y+rf-#LxwwPHU=~liW;p@Jp|_wddHXxO+*}bnnzs zex>oL3>%On#Hk6WvOcIyfjjkKDwLundA|3gC(yfLwp*w{WNW4~{mJY@a_2lr`2@W5 zu&rdHn0>wVRhp2|`5|_xShvyI`9Vai&E-S}m55`V20IdHr$ZsJhZ5TCreT_}JKKnd zH|k@ZF+NYZ9=;Hd_=>JWr}yw;xDvfr z&p;QQYco@JmLebq*IJz6teAvFpxL`3cJC{2Grg&?-Ouaw4`^OGCsjbnqfY60$`%*L zTHSF-;rqhYRs}&N*MQ(pix)Cv_?a8e$EpSZ`T<4W|e#=d3O`+ zz^MKSt4zfp&C3@dp9vIrdj+sGnw|B)CYTMunu+Nf!aU9o@JXWf$4TwwGQLymf1P{Y zyYQx^e)yBXfTsugBK34}Ki?#VsKp=IKG@=fLM&&|SAZow5s=@~!4gG|;LzfBga(X3 zoB&otLpp|`wsGRnQH%2Xu$qwLYC&C&;#+5@EMk}t)6PMW->xxkXl`6+`HtT>nrN`GN~%~ka4)AOT`uuHj?Kx7nh zG83wFmI21jM|MPp+W}!(P^3ipLI^e$me2iVfK)J=`EjfRH|V6mRs7+wmNo}r>2L5_ zPxY&qu!VR&SrV?|iLn%{DLVjii?$1odbfJp(lYk;g4{I#-Qy_wlcB#MQ8tB2Fk~5+ z9TAbKppF}hn~3QYw;7B6&5a1m5o96ifh|Vv-vR@&Q;g8@ofRKTX4}cU!mO?`c zEdLRf<%1K4Rrc#|1hXr`l~QXzkh?Zw6YX1__2PF)0$KO$fTaL0tTUR7zehayCUmQl z3gu@R719s)!;?$(&&VBexXRKwoxrbfs(0pGb$k=ZIaTj*HeHv}Cogjcop;*KTYr49 z&D1g4Z=vbUQCzaE@mIxDiqeiE%|SDd15qSsemS=75+PTGle2I5)7IBx1dU4>sxu^g zD(<)x(w``oWYKyL<;Mbj_-@Jqjl(CbntipuMD+P^@l}q7tYm**5n_@fh!xZerf(bX zhfW5nl7Z!lL^D?z2yq1t33$LB$ylfyRyA6e!7?FVA+|6hI~ zb2|R#lP5x`8kYaEZt!H*!9SN@vk&1KUjDLwi}!cIfCVVat;33~S1w?+w0mrV=|2Cs zq*-B%Q+NGaJVK#!6RiLBULIHm@XW_hgvH>A`SI>-Ij@F(JK9L!) zwp3r_fvR6!4venzrH7sG7|2J6-BB)S{GV@Ef`|JEu&sT}n|{QaGkY=*1HvX$i5 z%ily*NNn$qo=eND0Bj=<1e>`EOv_AHt-hG<3lwlIV>XlvFV}UP%=liEU+}=Auvf~l zaBven$X4h$SApZR1X573*-(PIN=2SbBtu1ZWN;UWyh0RL-5D^3gMvPj!==wv){h01 zXZb$e_jd*Cd21Fa2e=(A%G_+c1P#288I)X6Wxez6dm}~FlV~_379^j>SjWDrpq-buUEZKSZ=6Ww*gosO3d$I z7*nmW2dw2!dbC=oMC5V#M=mqe94Jru?SMI@x9pR^?O&l6yOCZxs;;X?-zlnzQc3e$ zV^>*&I`w>68Ee#0S=-obrSi9AyElL2sARljNx4q9S?@6@$>Y`2<(I~Q;L%?WbA-RO zrQrnr?EEOGuY)m<7LveQvJc^;YQ6dBx%F8Mu9Dc3g2P@sfY-sfvCGq!4taCrY)WdI=zbc%3>?!Vd$`MD1Hl?KEGV0wwQ;MScJ0ngh zP*FY(jMG^JCn1fvi)17YxXfI3`6Ad^@6XeL6&qY#uC}@_r_!>JFDz9wo8hqS()lt4 zoJ{%Q5EA2v| z^QpMui12CoYFW1GbTo>`7=@DN0V$Qg0pgo^ADz4NRExiWiBcZ*_~JD(FJ!cU|IB}x zwYx*td5&4pa`^iw%(lhqig0(W0M71K5FSO!NB)IQTt49&+(1I?#?!oer)Wbre5?qV zU6mYZ%_kAJfRC(3!wb8eh>!3GPGJYIPyq%lY!noT?ZhQH@zZV8?haFqPnWJ0a2PXY zvBc#j)&tSm!a@h@JbyLihIDPr8To$x1{T+UyjYvau_2j1-E{YdP4d6(jGE^Van;!v zJ%k$zm-H=P`C1k-o#qJE^Po7O