From 27a9f92d92f338fc57727afa90485d6bd01c77c6 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 5 Oct 2023 07:04:14 -0400 Subject: [PATCH 01/85] Do not rewrite project version in BeamModulePlugin --- .../groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 7 ------- 1 file changed, 7 deletions(-) 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 c7a62237086e..3a332d8d4d08 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -488,13 +488,6 @@ class BeamModulePlugin implements Plugin { project.ext.mavenGroupId = 'org.apache.beam' - // Automatically use the official release version if we are performing a release - // otherwise append '-SNAPSHOT' - project.version = '2.52.0' - if (!isRelease(project)) { - project.version += '-SNAPSHOT' - } - // Default to dash-separated directories for artifact base name, // which will also be the default artifactId for maven publications project.apply plugin: 'base' From 0304cae8cc8f3d69a7c192beec071ff1d8474538 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Thu, 5 Oct 2023 14:40:16 -0400 Subject: [PATCH 02/85] add bigtable reviewers (#28834) --- .github/REVIEWERS.yml | 4 ++++ .github/autolabeler.yml | 1 + 2 files changed, 5 insertions(+) diff --git a/.github/REVIEWERS.yml b/.github/REVIEWERS.yml index f472568f3479..1c260fbea5eb 100644 --- a/.github/REVIEWERS.yml +++ b/.github/REVIEWERS.yml @@ -52,6 +52,10 @@ labels: - name: spanner reviewers: - nielm + - name: bigtable + reviewers: + - igorbernstein2 + - mutianf exclusionList: [] - name: Build reviewers: diff --git a/.github/autolabeler.yml b/.github/autolabeler.yml index d1cc8296d303..57c8f65c6ac1 100644 --- a/.github/autolabeler.yml +++ b/.github/autolabeler.yml @@ -69,6 +69,7 @@ io: ["sdks/go/pkg/beam/io/**/*", "sdks/java/io/**/*", "sdks/python/apache_beam/ "redis": ["sdks/java/io/redis/**/*"] "solr": ["sdks/java/io/solr/**/*"] "spanner": ["sdks/go/pkg/beam/io/spannerio/**/*", "sdks/python/apache_beam/io/gcp/spanner.py", "sdks/python/apache_beam/io/gcp/experimental/spannerio.py", "sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/spanner/**/*"] +"bigtable": ["sdks/go/pkg/beam/io/bigtableio/**/*", "sdks/go/pkg/beam/io/xlang/bigtableio/**/*", "sdks/python/apache_beam/io/gcp/bigtableio.py", "sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/**/*"] "synthetic": ["sdks/java/io/synthetic/**/*"] "tests": ["sdks/java/io/file-based-io-tests/**/*"] "thrift": ["sdks/java/io/thrift/**/*"] From c2d7d5fbbc6a0d89131a624665116758c194e3ea Mon Sep 17 00:00:00 2001 From: Joar Wandborg Date: Thu, 5 Oct 2023 21:44:09 +0200 Subject: [PATCH 03/85] BigtableIO: Force minimum value of `desiredBundleSizeBytes` to be 1 (#28798) --- CHANGES.md | 34 +++++++++++++++++++ .../beam/sdk/io/gcp/bigtable/BigtableIO.java | 6 +++- .../sdk/io/gcp/bigtable/BigtableIOTest.java | 33 ++++++++++++++++++ 3 files changed, 72 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 650b33c12407..0c2c2e3f79f4 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -53,6 +53,40 @@ * ([#X](https://github.com/apache/beam/issues/X)). --> +# [2.52.0] - Unreleased + +## Highlights + +* New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). +* New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). + +## I/Os + +* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## New Features / Improvements + +* X feature added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). + +## Breaking Changes + +* X behavior was changed ([#X](https://github.com/apache/beam/issues/X)). + +## Deprecations + +* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). + +## Bugfixes + +* Fixed "Desired bundle size 0 bytes must be greater than 0" in Java SDK's BigtableIO.BigtableSource when you have more cores than bytes to read (Java) [#28793](https://github.com/apache/beam/issues/28793). + +## Security Fixes +* Fixed (CVE-YYYY-NNNN)[https://www.cve.org/CVERecord?id=CVE-YYYY-NNNN] (Java/Python/Go) ([#X](https://github.com/apache/beam/issues/X)). + +## Known Issues + +* ([#X](https://github.com/apache/beam/issues/X)). + # [2.51.0] - Unreleased ## Highlights diff --git a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java index 92a0af205482..ef868e8bf7c9 100644 --- a/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java +++ b/sdks/java/io/google-cloud-platform/src/main/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIO.java @@ -1326,7 +1326,11 @@ public List split(long desiredBundleSizeBytes, PipelineOptions o long maximumNumberOfSplits = 4000; long sizeEstimate = getEstimatedSizeBytes(options); desiredBundleSizeBytes = - Math.max(sizeEstimate / maximumNumberOfSplits, desiredBundleSizeBytes); + Math.max( + sizeEstimate / maximumNumberOfSplits, + // BoundedReadEvaluatorFactory may provide us with a desiredBundleSizeBytes of 0 + // https://github.com/apache/beam/issues/28793 + Math.max(1, desiredBundleSizeBytes)); // Delegate to testable helper. List splits = diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index bb70eb78984c..971b91d89b74 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -766,6 +766,39 @@ public void testReadingWithSplits() throws Exception { assertSourcesEqualReferenceSource(source, splits, null /* options */); } + /** + * Regression test for [Bug]: BigtableSource + * "Desired bundle size 0 bytes must be greater than 0" #28793 . + */ + @Test + public void testSplittingWithDesiredBundleSizeZero() throws Exception { + final String table = "TEST-SPLIT-DESIRED-BUNDLE-SIZE-ZERO-TABLE"; + final int numRows = 10; + final int numSamples = 10; + final long bytesPerRow = 1L; + + // Set up test table data and sample row keys for size estimation and splitting. + makeTableData(table, numRows); + service.setupSampleRowKeys(table, numSamples, bytesPerRow); + + // Generate source and split it. + BigtableSource source = + new BigtableSource( + factory, + configId, + config, + BigtableReadOptions.builder() + .setTableId(StaticValueProvider.of(table)) + .setKeyRanges(ALL_KEY_RANGE) + .build(), + null /*size*/); + List splits = source.split(0, null /* options */); + + // Test num splits and split equality. + assertThat(splits, hasSize(numSamples)); + assertSourcesEqualReferenceSource(source, splits, null /* options */); + } + @Test public void testReadingWithSplitFailed() throws Exception { FailureBigtableService failureService = From b8eb9df44ab30559de2ccf23c8313604e003178e Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 6 Oct 2023 00:08:15 +0400 Subject: [PATCH 04/85] Add Publish Docker Snapshots workflow (#28794) * Add Publish Docker Snapshots workflow * Fix timeout --- .../beam_Publish_Docker_Snapshots.yml | 85 +++++++++++++++++++ 1 file changed, 85 insertions(+) create mode 100644 .github/workflows/beam_Publish_Docker_Snapshots.yml diff --git a/.github/workflows/beam_Publish_Docker_Snapshots.yml b/.github/workflows/beam_Publish_Docker_Snapshots.yml new file mode 100644 index 000000000000..41262a58b1e7 --- /dev/null +++ b/.github/workflows/beam_Publish_Docker_Snapshots.yml @@ -0,0 +1,85 @@ +# 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. + +name: Publish Docker Snapshots + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 13 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_Publish_Docker_Snapshots: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Publish Docker Snapshots' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_Publish_Docker_Snapshots"] + job_phrase: ["Publish Docker Snapshots"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: run Publish Docker Snapshots script for Spark + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :runners:spark:3:job-server:container:dockerPush + arguments: | + -Pdocker-repository-root=gcr.io/apache-beam-testing/beam_portability \ + -Pdocker-tag-list=latest + - name: run Publish Docker Snapshots script for Flink + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :runners:flink:1.15:job-server-container:dockerPush + arguments: | + -Pdocker-repository-root=gcr.io/apache-beam-testing/beam_portability \ + -Pdocker-tag-list=latest \ No newline at end of file From 13e4da3c2d75cdf13d738728177d370505f925d4 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 5 Oct 2023 13:47:19 -0700 Subject: [PATCH 05/85] Bump google.golang.org/api from 0.143.0 to 0.144.0 in /sdks (#28839) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.143.0 to 0.144.0. - [Release notes](https://github.com/googleapis/google-api-go-client/releases) - [Changelog](https://github.com/googleapis/google-api-go-client/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-api-go-client/compare/v0.143.0...v0.144.0) --- updated-dependencies: - dependency-name: google.golang.org/api 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 d817ae549857..aa147a8dc682 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -57,7 +57,7 @@ require ( golang.org/x/sync v0.3.0 golang.org/x/sys v0.12.0 golang.org/x/text v0.13.0 - google.golang.org/api v0.143.0 + google.golang.org/api v0.144.0 google.golang.org/genproto v0.0.0-20230913181813-007df8e322eb google.golang.org/grpc v1.58.2 google.golang.org/protobuf v1.31.0 diff --git a/sdks/go.sum b/sdks/go.sum index 9f43e9a53abc..12c43b056f84 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -674,8 +674,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.143.0 h1:o8cekTkqhywkbZT6p1UHJPZ9+9uuCAJs/KYomxZB8fA= -google.golang.org/api v0.143.0/go.mod h1:FoX9DO9hT7DLNn97OuoZAGSDuNAXdJRuGK98rSUgurk= +google.golang.org/api v0.144.0 h1:01xgplvIwdMpnrlenPHMgRAAgAH9N5Zv21Qu6XwJxSU= +google.golang.org/api v0.144.0/go.mod h1:OARJqIfoYjXJj4C1AiBSXYZt03qsoz8FQYU6fBEfrHM= 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= From 6722ff197e80c8c797c911a53c0a518b91e0f729 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 5 Oct 2023 15:31:18 -0700 Subject: [PATCH 06/85] Schema-aware Avro reading and writing. --- sdks/python/apache_beam/io/avroio.py | 271 +++++++++++++++++-- sdks/python/apache_beam/io/avroio_test.py | 32 ++- sdks/python/apache_beam/pvalue.py | 3 + sdks/python/apache_beam/typehints/schemas.py | 4 + 4 files changed, 282 insertions(+), 28 deletions(-) diff --git a/sdks/python/apache_beam/io/avroio.py b/sdks/python/apache_beam/io/avroio.py index d86f59e3a411..9225acf346e4 100644 --- a/sdks/python/apache_beam/io/avroio.py +++ b/sdks/python/apache_beam/io/avroio.py @@ -45,7 +45,13 @@ # pytype: skip-file import os from functools import partial +from typing import Any +from typing import Callable +from typing import Dict +from typing import List +from typing import Union +import fastavro from fastavro.read import block_reader from fastavro.write import Writer @@ -54,8 +60,11 @@ from apache_beam.io import filebasedsource from apache_beam.io import iobase from apache_beam.io.filesystem import CompressionTypes +from apache_beam.io.filesystems import FileSystems from apache_beam.io.iobase import Read +from apache_beam.portability.api import schema_pb2 from apache_beam.transforms import PTransform +from apache_beam.typehints import schemas __all__ = [ 'ReadFromAvro', @@ -73,7 +82,8 @@ def __init__( file_pattern=None, min_bundle_size=0, validate=True, - use_fastavro=True): + use_fastavro=True, + as_rows=False): """Initializes :class:`ReadFromAvro`. Uses source :class:`~apache_beam.io._AvroSource` to read a set of Avro @@ -140,13 +150,26 @@ def __init__( creation time. use_fastavro (bool): This flag is left for API backwards compatibility and no longer has an effect. Do not use. + as_rows (bool): Whether to return a schema'd PCollection of Beam rows. """ super().__init__() - self._source = _create_avro_source( + self._source = _FastAvroSource( file_pattern, min_bundle_size, validate=validate) + if as_rows: + path = FileSystems.match([file_pattern], [1])[0].metadata_list[0].path + with FileSystems.open(path) as fin: + avro_schema = fastavro.reader(fin).writer_schema + beam_schema = avro_schema_to_beam_schema(avro_schema) + self._post_process = avro_dict_to_beam_row(avro_schema, beam_schema) + else: + self._post_process = None def expand(self, pvalue): - return pvalue.pipeline | Read(self._source) + records = pvalue.pipeline | Read(self._source) + if self._post_process: + return records | beam.Map(self._post_process) + else: + return records def display_data(self): return {'source_dd': self._source} @@ -184,8 +207,7 @@ def __init__( name and the value being the actual data. If False, it only returns the data. """ - source_from_file = partial( - _create_avro_source, min_bundle_size=min_bundle_size) + source_from_file = partial(_FastAvroSource, min_bundle_size=min_bundle_size) self._read_all_files = filebasedsource.ReadAllFiles( True, CompressionTypes.AUTO, @@ -280,15 +302,6 @@ def advance_file_past_next_sync_marker(f, sync_marker): data = f.read(buf_size) -def _create_avro_source(file_pattern=None, min_bundle_size=0, validate=False): - return \ - _FastAvroSource( - file_pattern=file_pattern, - min_bundle_size=min_bundle_size, - validate=validate - ) - - class _FastAvroSource(filebasedsource.FileBasedSource): """A source for reading Avro files using the `fastavro` library. @@ -338,12 +351,15 @@ def split_points_unclaimed(stop_position): yield record +_create_avro_source = _FastAvroSource + + class WriteToAvro(beam.transforms.PTransform): """A ``PTransform`` for writing avro files.""" def __init__( self, file_path_prefix, - schema, + schema=None, codec='deflate', file_name_suffix='', num_shards=0, @@ -382,9 +398,10 @@ def __init__( Returns: A WriteToAvro transform usable for writing. """ - self._sink = _create_avro_sink( + self._schema = schema + self._sink_provider = lambda avro_schema: _create_avro_sink( file_path_prefix, - schema, + avro_schema, codec, file_name_suffix, num_shards, @@ -392,7 +409,21 @@ def __init__( mime_type) def expand(self, pcoll): - return pcoll | beam.io.iobase.Write(self._sink) + if self._schema: + avro_schema = self._schema + records = pcoll + else: + try: + beam_schema = schemas.schema_from_element_type(pcoll.element_type) + except TypeError as exn: + raise ValueError( + "An explicit schema is required to write non-schema'd PCollections." + ) from exn + avro_schema = beam_schema_to_avro_schema(beam_schema) + records = pcoll | beam.Map( + beam_row_to_avro_dict(avro_schema, beam_schema)) + self._sink = self._sink_provider(avro_schema) + return records | beam.io.iobase.Write(self._sink) def display_data(self): return {'sink_dd': self._sink} @@ -406,7 +437,7 @@ def _create_avro_sink( num_shards, shard_name_template, mime_type): - if "class \'avro.schema" in str(type(schema)): + if "class 'avro.schema" in str(type(schema)): raise ValueError( 'You are using Avro IO with fastavro (default with Beam on ' 'Python 3), but supplying a schema parsed by avro-python3. ' @@ -483,3 +514,205 @@ def write_record(self, writer, value): def close(self, writer): writer.flush() self.file_handle.close() + + +AVRO_PRIMITIVES_TO_BEAM_PRIMITIVES = { + 'boolean': schema_pb2.BOOLEAN, + 'int': schema_pb2.INT32, + 'long': schema_pb2.INT64, + 'float': schema_pb2.FLOAT, + 'double': schema_pb2.DOUBLE, + 'bytes': schema_pb2.BYTES, + 'string': schema_pb2.STRING, +} + +BEAM_PRIMITIVES_TO_AVRO_PRIMITIVES = { + v: k + for k, v in AVRO_PRIMITIVES_TO_BEAM_PRIMITIVES.items() +} + +_AvroSchemaType = Union[str, List, Dict] + + +def avro_type_to_beam_type(avro_type: _AvroSchemaType) -> schema_pb2.FieldType: + if isinstance(avro_type, str): + return avro_type_to_beam_type({'type': avro_type}) + elif isinstance(avro_type, list): + # Union type + return schemas.typing_to_runner_api(Any) + type_name = avro_type['type'] + if type_name in AVRO_PRIMITIVES_TO_BEAM_PRIMITIVES: + return schema_pb2.FieldType( + atomic_type=AVRO_PRIMITIVES_TO_BEAM_PRIMITIVES[type_name]) + elif type_name in ('fixed', 'enum'): + return schema_pb2.FieldType(atomic_type=schema_pb2.STRING) + elif type_name == 'array': + return schema_pb2.FieldType( + array_type=schema_pb2.ArrayType( + element_type=avro_type_to_beam_type(avro_type['items']))) + elif type_name == 'map': + return schema_pb2.FieldType( + map_type=schema_pb2.MapType( + key_type=schema_pb2.FieldType(atomic_type=schema_pb2.STRING), + value_type=avro_type_to_beam_type(avro_type['values']))) + elif type_name == 'record': + return schema_pb2.FieldType( + row_type=schema_pb2.RowType( + schema=schema_pb2.Schema( + fields=[ + schemas.schema_field( + f['name'], avro_type_to_beam_type(f['type'])) + for f in avro_type['fields'] + ]))) + else: + raise ValueError(f'Unable to convert {avro_type} to a Beam schema.') + + +def avro_schema_to_beam_schema( + avro_schema: _AvroSchemaType) -> schema_pb2.Schema: + beam_type = avro_type_to_beam_type(avro_schema) + if isinstance(avro_schema, dict) and avro_schema['type'] == 'record': + return beam_type.row_type.schema + else: + return schema_pb2.Schema(fields=[schemas.schema_field('record', beam_type)]) + + +def avro_dict_to_beam_row( + avro_schema: _AvroSchemaType, + beam_schema: schema_pb2.Schema) -> Callable[[Any], Any]: + if isinstance(avro_schema, str): + return avro_dict_to_beam_row({'type': avro_schema}) + if avro_schema['type'] == 'record': + to_row = avro_value_to_beam_value( + schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=beam_schema))) + else: + + def to_row(record): + return beam.Row(record=record) + + return beam.typehints.with_output_types( + schemas.named_tuple_from_schema(beam_schema))( + to_row) + + +def avro_value_to_beam_value( + beam_type: schema_pb2.FieldType) -> Callable[[Any], Any]: + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + return lambda value: value + elif type_info == "array_type": + element_converter = avro_value_to_beam_value( + beam_type.array_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "iterable_type": + element_converter = avro_value_to_beam_value( + beam_type.iterable_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "map_type": + if beam_type.map_type.key_type.atomic_type != schema_pb2.STRING: + raise TypeError( + f'Only strings allowd as map keys when converting from AVRO, ' + f'found {beam_type}') + value_converter = avro_value_to_beam_value(beam_type.map_type.value_type) + return lambda value: {k: value_converter(v) for (k, v) in value.items()} + elif type_info == "row_type": + converters = { + field.name: avro_value_to_beam_value(field.type) + for field in beam_type.row_type.schema.fields + } + return lambda value: beam.Row( + ** + {name: convert(value[name]) + for (name, convert) in converters.items()}) + elif type_info == "logical_type": + return lambda value: value + else: + raise ValueError(f"Unrecognized type_info: {type_info!r}") + + +def beam_schema_to_avro_schema( + beam_schema: schema_pb2.Schema) -> _AvroSchemaType: + return beam_type_to_avro_type( + schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=beam_schema))) + + +def beam_type_to_avro_type(beam_type: schema_pb2.FieldType) -> _AvroSchemaType: + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + return {'type': BEAM_PRIMITIVES_TO_AVRO_PRIMITIVES[beam_type.atomic_type]} + elif type_info == "array_type": + return { + 'type': 'array', + 'items': beam_type_to_avro_type(beam_type.array_type.element_type) + } + elif type_info == "iterable_type": + return { + 'type': 'array', + 'items': beam_type_to_avro_type(beam_type.iterable_type.element_type) + } + elif type_info == "map_type": + if beam_type.map_type.key_type.atomic_type != schema_pb2.STRING: + raise TypeError( + f'Only strings allowd as map keys when converting to AVRO, ' + f'found {beam_type}') + return { + 'type': 'map', + 'values': beam_type_to_avro_type(beam_type.map_type.element_type) + } + elif type_info == "row_type": + return { + 'type': 'record', + 'name': beam_type.row_type.schema.id, + 'fields': [{ + 'name': field.name, 'type': beam_type_to_avro_type(field.type) + } for field in beam_type.row_type.schema.fields], + } + else: + raise ValueError(f"Unconvertale type: {beam_type}") + + +def beam_row_to_avro_dict( + avro_schema: _AvroSchemaType, beam_schema: schema_pb2.Schema): + if isinstance(avro_schema, str): + return beam_row_to_avro_dict({'type': avro_schema}, beam_schema) + if avro_schema['type'] == 'record': + return beam_value_to_avro_value( + schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=beam_schema))) + else: + convert = beam_value_to_avro_value(beam_schema) + return lambda row: convert(row[0]) + + +def beam_value_to_avro_value( + beam_type: schema_pb2.FieldType) -> Callable[[Any], Any]: + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + return lambda value: value + elif type_info == "array_type": + element_converter = avro_value_to_beam_value( + beam_type.array_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "iterable_type": + element_converter = avro_value_to_beam_value( + beam_type.iterable_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "map_type": + if beam_type.map_type.key_type.atomic_type != schema_pb2.STRING: + raise TypeError( + f'Only strings allowd as map keys when converting from AVRO, ' + f'found {beam_type}') + value_converter = avro_value_to_beam_value(beam_type.map_type.value_type) + return lambda value: {k: value_converter(v) for (k, v) in value.items()} + elif type_info == "row_type": + converters = { + field.name: avro_value_to_beam_value(field.type) + for field in beam_type.row_type.schema.fields + } + return lambda value: { + name: convert(getattr(value, name)) + for (name, convert) in converters.items() + } + elif type_info == "logical_type": + return lambda value: value + else: + raise ValueError(f"Unrecognized type_info: {type_info!r}") diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index ba35cf5846c0..68f5226e64ce 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -35,8 +35,8 @@ from apache_beam.io import filebasedsource from apache_beam.io import iobase from apache_beam.io import source_test_utils -from apache_beam.io.avroio import _create_avro_sink # For testing -from apache_beam.io.avroio import _create_avro_source # For testing +from apache_beam.io.avroio import _FastAvroSource # For testing +from apache_beam.io.avroio import _FastAvroSource # For testing from apache_beam.io.filesystems import FileSystems from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that @@ -125,7 +125,7 @@ def _write_pattern(self, num_files, return_filenames=False): def _run_avro_test( self, pattern, desired_bundle_size, perform_splitting, expected_result): - source = _create_avro_source(pattern) + source = _FastAvroSource(pattern) if perform_splitting: assert desired_bundle_size @@ -146,6 +146,20 @@ def _run_avro_test( read_records = source_test_utils.read_from_source(source, None, None) self.assertCountEqual(expected_result, read_records) + def test_schema_read_write(self): + with tempfile.TemporaryDirectory() as tmp_dirname: + path = os.path.join(tmp_dirname, 'tmp_filename') + rows = [beam.Row(a=1, b=['x', 'y']), beam.Row(a=2, b=['t', 'u'])] + stable_repr = lambda row: json.dumps(row._asdict()) + with TestPipeline() as p: + _ = p | Create(rows) | avroio.WriteToAvro(path) | beam.Map(print) + with TestPipeline() as p: + readback = ( + p + | avroio.ReadFromAvro(path + '*', as_rows=True) + | beam.Map(stable_repr)) + assert_that(readback, equal_to([stable_repr(r) for r in rows])) + def test_read_without_splitting(self): file_name = self._write_data() expected_result = self.RECORDS @@ -159,7 +173,7 @@ def test_read_with_splitting(self): def test_source_display_data(self): file_name = 'some_avro_source' source = \ - _create_avro_source( + _FastAvroSource( file_name, validate=False, ) @@ -220,12 +234,12 @@ def test_write_display_data(self): def test_read_reentrant_without_splitting(self): file_name = self._write_data() - source = _create_avro_source(file_name) + source = _FastAvroSource(file_name) source_test_utils.assert_reentrant_reads_succeed((source, None, None)) def test_read_reantrant_with_splitting(self): file_name = self._write_data() - source = _create_avro_source(file_name) + source = _FastAvroSource(file_name) splits = [split for split in source.split(desired_bundle_size=100000)] assert len(splits) == 1 source_test_utils.assert_reentrant_reads_succeed( @@ -246,7 +260,7 @@ def test_split_points(self): sync_interval = 16000 file_name = self._write_data(count=num_records, sync_interval=sync_interval) - source = _create_avro_source(file_name) + source = _FastAvroSource(file_name) splits = [split for split in source.split(desired_bundle_size=float('inf'))] assert len(splits) == 1 @@ -306,7 +320,7 @@ def test_read_with_splitting_pattern(self): def test_dynamic_work_rebalancing_exhaustive(self): def compare_split_points(file_name): - source = _create_avro_source(file_name) + source = _FastAvroSource(file_name) splits = [ split for split in source.split(desired_bundle_size=float('inf')) ] @@ -334,7 +348,7 @@ def test_corrupted_file(self): f.write(corrupted_data) corrupted_file_name = f.name - source = _create_avro_source(corrupted_file_name) + source = _FastAvroSource(corrupted_file_name) with self.assertRaisesRegex(ValueError, r'expected sync marker'): source_test_utils.read_from_source(source, None, None) diff --git a/sdks/python/apache_beam/pvalue.py b/sdks/python/apache_beam/pvalue.py index 2e86c9eb51c7..90882651d0b2 100644 --- a/sdks/python/apache_beam/pvalue.py +++ b/sdks/python/apache_beam/pvalue.py @@ -673,6 +673,9 @@ def __init__(self, **kwargs): def as_dict(self): return dict(self.__dict__) + # For compatibility with named tuples. + _asdict = as_dict + def __iter__(self): for _, value in self.__dict__.items(): yield value diff --git a/sdks/python/apache_beam/typehints/schemas.py b/sdks/python/apache_beam/typehints/schemas.py index 229a8af20bb6..ea836430e8e2 100644 --- a/sdks/python/apache_beam/typehints/schemas.py +++ b/sdks/python/apache_beam/typehints/schemas.py @@ -93,6 +93,7 @@ from apache_beam.typehints.native_type_compatibility import _match_is_exactly_mapping from apache_beam.typehints.native_type_compatibility import _match_is_optional from apache_beam.typehints.native_type_compatibility import _safe_issubclass +from apache_beam.typehints.native_type_compatibility import convert_to_typing_type from apache_beam.typehints.native_type_compatibility import extract_optional_type from apache_beam.typehints.native_type_compatibility import match_is_named_tuple from apache_beam.typehints.schema_registry import SCHEMA_REGISTRY @@ -284,6 +285,9 @@ def typing_to_runner_api(self, type_: type) -> schema_pb2.FieldType: if row_type_constraint is not None: return self.typing_to_runner_api(row_type_constraint) + if isinstance(type_, typehints.TypeConstraint): + type_ = convert_to_typing_type(type_) + # All concrete types (other than NamedTuple sub-classes) should map to # a supported primitive type. if type_ in PRIMITIVE_TO_ATOMIC_TYPE: From 714d3a9aa047261359a3f599fe1881d12c6d830e Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 5 Oct 2023 15:38:19 -0700 Subject: [PATCH 07/85] Add yaml providers for avro. --- sdks/python/apache_beam/yaml/standard_io.yaml | 11 +++++++++++ sdks/python/apache_beam/yaml/yaml_provider.py | 6 +++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 9ad4f53ba1f6..1d6aa5548a82 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -67,6 +67,8 @@ 'WriteToCsv': 'WriteToCsv' 'ReadFromJson': 'ReadFromJson' 'WriteToJson': 'WriteToJson' + 'ReadFromAvro': 'ReadFromAvro' + 'WriteToAvro': 'WriteToAvro' config: mappings: 'ReadFromCsv': @@ -77,6 +79,13 @@ path: 'path' 'WriteToJson': path: 'path' + 'ReadFromAvro': + path: 'file_pattern' + 'WriteToAvro': + path: 'file_path_prefix' + defaults: + 'ReadFromAvro': + as_rows: True underlying_provider: type: python transforms: @@ -84,3 +93,5 @@ 'WriteToCsv': 'apache_beam.io.WriteToCsv' 'ReadFromJson': 'apache_beam.io.ReadFromJson' 'WriteToJson': 'apache_beam.io.WriteToJson' + 'ReadFromAvro': 'apache_beam.io.ReadFromAvro' + 'WriteToAvro': 'apache_beam.io.WriteToAvro' diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 3a88f6074b37..20d4b096916d 100644 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -684,7 +684,7 @@ def __exit__(self, *args): @ExternalProvider.register_provider_type('renaming') class RenamingProvider(Provider): - def __init__(self, transforms, mappings, underlying_provider): + def __init__(self, transforms, mappings, underlying_provider, defaults=None): if isinstance(underlying_provider, dict): underlying_provider = ExternalProvider.provider_from_spec( underlying_provider) @@ -694,6 +694,7 @@ def __init__(self, transforms, mappings, underlying_provider): if transform not in mappings: raise ValueError(f'Missing transform {transform} in mappings.') self._mappings = mappings + self._defaults = defaults or {} def available(self) -> bool: return self._underlying_provider.available() @@ -731,6 +732,9 @@ def create_transform( mappings.get(key, key): value for key, value in args.items() } + for key, value in self._defaults.get(typ, {}).items(): + if key not in remapped_args: + remapped_args[key] = value return self._underlying_provider.create_transform( self._transforms[typ], remapped_args, yaml_create_transform) From 07e26fd014d8c8a811abf12d354945b7865a75cb Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 5 Oct 2023 17:33:34 -0700 Subject: [PATCH 08/85] test,lint --- sdks/python/apache_beam/io/avroio_test.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/avroio_test.py b/sdks/python/apache_beam/io/avroio_test.py index 68f5226e64ce..c54ac40711b1 100644 --- a/sdks/python/apache_beam/io/avroio_test.py +++ b/sdks/python/apache_beam/io/avroio_test.py @@ -36,7 +36,7 @@ from apache_beam.io import iobase from apache_beam.io import source_test_utils from apache_beam.io.avroio import _FastAvroSource # For testing -from apache_beam.io.avroio import _FastAvroSource # For testing +from apache_beam.io.avroio import _create_avro_sink # For testing from apache_beam.io.filesystems import FileSystems from apache_beam.testing.test_pipeline import TestPipeline from apache_beam.testing.util import assert_that @@ -221,6 +221,7 @@ def test_sink_display_data(self): def test_write_display_data(self): file_name = 'some_avro_sink' write = avroio.WriteToAvro(file_name, self.SCHEMA) + write.expand(beam.PCollection(beam.Pipeline())) dd = DisplayData.create_from(write) expected_items = [ DisplayDataItemMatcher('schema', str(self.SCHEMA)), From 3405b31d0400a8bf22cd96f50f0a7121b11f8276 Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Fri, 6 Oct 2023 08:26:13 +0600 Subject: [PATCH 09/85] Github Workflow Replacement for Jenkins Jobs, beam_LoadTests_Python_Combine_Flink_* (#28846) * beam_LoadTests_Python_Combine_Flink * fixed commented info --- ...m_LoadTests_Python_Combine_Flink_Batch.yml | 140 ++++++++++++++++++ ...adTests_Python_Combine_Flink_Streaming.yml | 126 ++++++++++++++++ ...ombine_Flink_Batch_2GB_10_byte_records.txt | 27 ++++ ...ython_Combine_Flink_Batch_2GB_Fanout_4.txt | 28 ++++ ...ython_Combine_Flink_Batch_2GB_Fanout_8.txt | 28 ++++ ...ne_Flink_Streaming_2GB_10_byte_records.txt | 29 ++++ ...n_Combine_Flink_Streaming_2GB_Fanout_4.txt | 30 ++++ ...n_Combine_Flink_Streaming_2GB_Fanout_8.txt | 30 ++++ 8 files changed, 438 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_10_byte_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt create mode 100644 .github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml new file mode 100644 index 000000000000..165b23703562 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml @@ -0,0 +1,140 @@ +# 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. + +name: LoadTests Python Combine Flink Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 6 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-py-cmb-flink-batch-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-py-cmb-flink-batch-${{ github.run_id }} + +jobs: + beam_LoadTests_Python_Combine_Flink_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python Combine Flink Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_Combine_Flink_Batch"] + job_phrase: ["Run Load Tests Python Combine Flink Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt + - name: Start Flink with parallelism 5 + env: + FLINK_NUM_WORKERS: 5 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB 10 byte records + env: + CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PpythonVersion=3.8 \ + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Flink_Batch_test_arguments_1 }} --job_name=load-tests-python-flink-batch-combine-1-${{env.NOW_UTC}}' \ + - name: Restart Flink with parallelism 16 + env: + FLINK_NUM_WORKERS: 16 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh restart + - name: run Load test 2GB Fanout 4 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PpythonVersion=3.8 \ + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Flink_Batch_test_arguments_2 }} --job_name=load-tests-python-flink-batch-combine-4-${{env.NOW_UTC}}' \ + - name: run Load test 2GB Fanout 8 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PpythonVersion=3.8 \ + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Flink_Batch_test_arguments_3 }} --job_name=load-tests-python-flink-batch-combine-5-${{env.NOW_UTC}}' \ + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml new file mode 100644 index 000000000000..1dae586de21a --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml @@ -0,0 +1,126 @@ +# 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. + +name: LoadTests Python Combine Flink Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 7 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-py-cmb-flink-streaming-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-py-cmb-flink-streaming-${{ github.run_id }} + +jobs: + beam_LoadTests_Python_Combine_Flink_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python Combine Flink Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_Combine_Flink_Streaming"] + job_phrase: ["Run Load Tests Python Combine Flink Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt + - name: Start Flink with parallelism 16 + env: + FLINK_NUM_WORKERS: 16 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run Load test 2GB Fanout 4 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PpythonVersion=3.8 \ + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Flink_Streaming_test_arguments_1 }} --job_name=load-tests-python-flink-streaming-combine-4-${{env.NOW_UTC}}' \ + - name: run Load test 2GB Fanout 8 + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PpythonVersion=3.8 \ + -PloadTest.mainClass=apache_beam.testing.load_tests.combine_test \ + -Prunner=PortableRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_Combine_Flink_Streaming_test_arguments_2 }} --job_name=load-tests-python-flink-streaming-combine-5-${{env.NOW_UTC}}' \ + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete + + # // TODO(https://github.com/apache/beam/issues/20402). Skipping some cases because they are too slow: + # load-tests-python-flink-streaming-combine-1' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt new file mode 100644 index 000000000000..8295d1c8aa86 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_10_byte_records.txt @@ -0,0 +1,27 @@ +# 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. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_combine_1 +--influx_measurement=python_batch_combine_1 +--input_options=''{\\"num_records\\":200000000,\\"key_size\\":1,\\"value_size\\":9,\\"algorithm\\":\\"lcg\\"}'' +--parallelism=5 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--top_count=20 +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt new file mode 100644 index 000000000000..82f8bcc7c0ae --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_4.txt @@ -0,0 +1,28 @@ +# 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. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_combine_4 +--influx_measurement=python_batch_combine_4 +--input_options=''{\\"num_records\\":5000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--parallelism=16 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--fanout=4 +--top_count=20 +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt new file mode 100644 index 000000000000..45425b6bf153 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Batch_2GB_Fanout_8.txt @@ -0,0 +1,28 @@ +# 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. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_combine_5 +--influx_measurement=python_batch_combine_5 +--input_options=''{\\"num_records\\":2500000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--parallelism=16 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--fanout=8 +--top_count=20 +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_10_byte_records.txt b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_10_byte_records.txt new file mode 100644 index 000000000000..12ffc1790e46 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_10_byte_records.txt @@ -0,0 +1,29 @@ +# 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. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_streaming_combine_1 +--influx_measurement=python_streaming_combine_1 +--input_options=''{\\"num_records\\":200000000,\\"key_size\\":1,\\"value_size\\":9,\\"algorithm\\":\\"lcg\\"}'' +--parallelism=5 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--top_count=20 +--streaming +--use_stateful_load_generator +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt new file mode 100644 index 000000000000..c7d5552a03bd --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_4.txt @@ -0,0 +1,30 @@ +# 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. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_streaming_combine_4 +--influx_measurement=python_streaming_combine_4 +--input_options=''{\\"num_records\\":5000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--parallelism=16 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--fanout=4 +--top_count=20 +--streaming +--use_stateful_load_generator +--runner=PortableRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt new file mode 100644 index 000000000000..bffdeab2cb11 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_Combine_Flink_Streaming_2GB_Fanout_8.txt @@ -0,0 +1,30 @@ +# 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. + +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_streaming_combine_5 +--influx_measurement=python_streaming_combine_5 +--input_options=''{\\"num_records\\":2500000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--parallelism=16 +--job_endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_python3.8_sdk:latest +--fanout=8 +--top_count=20 +--streaming +--use_stateful_load_generator +--runner=PortableRunner \ No newline at end of file From 03741c06e0850abf56a3437b036264625784b8ac Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Thu, 5 Oct 2023 22:12:14 -0700 Subject: [PATCH 10/85] Bump golang.org/x/net from 0.15.0 to 0.16.0 in /sdks (#28858) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.15.0 to 0.16.0. - [Commits](https://github.com/golang/net/compare/v0.15.0...v0.16.0) --- updated-dependencies: - dependency-name: golang.org/x/net 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 | 6 +++--- sdks/go.sum | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index aa147a8dc682..fb883c93af19 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -52,10 +52,10 @@ 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.12.1 - golang.org/x/net v0.15.0 + golang.org/x/net v0.16.0 golang.org/x/oauth2 v0.12.0 golang.org/x/sync v0.3.0 - golang.org/x/sys v0.12.0 + golang.org/x/sys v0.13.0 golang.org/x/text v0.13.0 google.golang.org/api v0.144.0 google.golang.org/genproto v0.0.0-20230913181813-007df8e322eb @@ -165,7 +165,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.13.0 // indirect + golang.org/x/crypto v0.14.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 12c43b056f84..1fcb968e118f 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -485,8 +485,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.13.0 h1:mvySKfSWJ+UKUii46M40LOvyWfN0s2U+46/jDd0e6Ck= -golang.org/x/crypto v0.13.0/go.mod h1:y6Z2r+Rw4iayiXXAIxJIDAJ1zMW4yaTpebo8fPOliYc= +golang.org/x/crypto v0.14.0 h1:wBqGXzWJW6m1XrIKlAH0Hs1JJ7+9KBwnIO8v66Q9cHc= +golang.org/x/crypto v0.14.0/go.mod h1:MVFd36DqK4CsrnJYDkBA3VC4m2GkXAM0PvzMCn4JQf4= 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= @@ -545,8 +545,8 @@ golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v 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.15.0 h1:ugBLEUaxABaB5AJqW9enI0ACdci2RUd4eP51NTBvuJ8= -golang.org/x/net v0.15.0/go.mod h1:idbUs1IY1+zTqbi8yxTbhexhEEk5ur9LInksu6HrEpk= +golang.org/x/net v0.16.0 h1:7eBu7KsSvFDtSXUIDbh3aqlK4DPsZ1rByC8PFfBThos= +golang.org/x/net v0.16.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= 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= @@ -604,8 +604,8 @@ golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.10.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.12.0 h1:CM0HF96J0hcLAwsHPJZjfdNzs0gftsLfgKt57wWHJ0o= -golang.org/x/sys v0.12.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= +golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= 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 b90e40308e223f9a1a37a9547ae4f745ef495ecb Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 6 Oct 2023 15:01:40 +0400 Subject: [PATCH 11/85] Add Cleanup Dataproc Resources workflow (#28814) * Add Cleanup Dataproc Resources workflow * Refactoring --- .../beam_CleanUpDataprocResources.yml | 61 +++++++++++++++++++ 1 file changed, 61 insertions(+) create mode 100644 .github/workflows/beam_CleanUpDataprocResources.yml diff --git a/.github/workflows/beam_CleanUpDataprocResources.yml b/.github/workflows/beam_CleanUpDataprocResources.yml new file mode 100644 index 000000000000..b6081e1891e6 --- /dev/null +++ b/.github/workflows/beam_CleanUpDataprocResources.yml @@ -0,0 +1,61 @@ +# 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. + +name: Cleanup Dataproc Resources + +on: + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_CleanUpDataprocResources: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: "beam_CleanUpDataprocResources" + steps: + - uses: actions/checkout@v3 + - name: Delete leaked resources for all the jobs that generates flink clusters + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./cleanup.sh -xe \ No newline at end of file From 6e90adb753eca968e36cee3a7fb6bf98db5c0080 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 6 Oct 2023 15:02:25 +0400 Subject: [PATCH 12/85] Add rotate credentials workflows (#28815) --- .../beam_IODatastoresCredentialsRotation.yml | 78 +++++++++++++++++++ .../beam_MetricsCredentialsRotation.yml | 78 +++++++++++++++++++ 2 files changed, 156 insertions(+) create mode 100644 .github/workflows/beam_IODatastoresCredentialsRotation.yml create mode 100644 .github/workflows/beam_MetricsCredentialsRotation.yml diff --git a/.github/workflows/beam_IODatastoresCredentialsRotation.yml b/.github/workflows/beam_IODatastoresCredentialsRotation.yml new file mode 100644 index 000000000000..36e6b238cdfc --- /dev/null +++ b/.github/workflows/beam_IODatastoresCredentialsRotation.yml @@ -0,0 +1,78 @@ +# 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. + +name: Rotate IO-Datastores Cluster Credentials + +on: + schedule: + - cron: '0 2 1 * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_IODatastoresCredentialsRotation: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} + strategy: + matrix: + job_name: ["beam_IODatastoresCredentialsRotation"] + job_phrase: ["N/A"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} + - name: Starting credential rotation + run: | + gcloud container clusters update io-datastores --start-credential-rotation --zone=us-central1-a --quiet + - name: Rebuilding the nodes + run: | + gcloud container clusters upgrade io-datastores --node-pool=pool-1 --zone=us-central1-a --quiet + - name: Completing the rotation + run: | + gcloud container clusters update io-datastores --complete-credential-rotation --zone=us-central1-a --quiet +# TODO: Send email to dev@beam.apache.org if something went wrong during credentials rotation \ No newline at end of file diff --git a/.github/workflows/beam_MetricsCredentialsRotation.yml b/.github/workflows/beam_MetricsCredentialsRotation.yml new file mode 100644 index 000000000000..9bd795f0c2a4 --- /dev/null +++ b/.github/workflows/beam_MetricsCredentialsRotation.yml @@ -0,0 +1,78 @@ +# 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. + +name: Rotate Metrics Cluster Credentials + +on: + schedule: + - cron: '0 2 1 * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_MetricsCredentialsRotation: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} + strategy: + matrix: + job_name: ["beam_MetricsCredentialsRotation"] + job_phrase: ["N/A"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} + - name: Starting credential rotation + run: | + gcloud container clusters update metrics --start-credential-rotation --zone=us-central1-a --quiet + - name: Rebuilding the nodes + run: | + gcloud container clusters upgrade metrics --node-pool=default-pool --zone=us-central1-a --quiet + - name: Completing the rotation + run: | + gcloud container clusters update metrics --complete-credential-rotation --zone=us-central1-a --quiet +# TODO: Send email to dev@beam.apache.org if something went wrong during credentials rotation \ No newline at end of file From ad159eaabb6315ea3dacd14839a60e1e6f45cde8 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 6 Oct 2023 15:10:35 +0400 Subject: [PATCH 13/85] Add Publish Beam SDK Snapshots workflow (#28795) * Add Publish Beam SDK Snapshots workflow * Fix timeout * Do not use comment phrase trigger * Refactoring --- .../beam_Publish_Beam_SDK_Snapshots.yml | 88 +++++++++++++++++++ 1 file changed, 88 insertions(+) create mode 100644 .github/workflows/beam_Publish_Beam_SDK_Snapshots.yml diff --git a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml new file mode 100644 index 000000000000..7aa5f6d4ceee --- /dev/null +++ b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml @@ -0,0 +1,88 @@ +# 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. + +name: Publish Beam SDK Snapshots + +on: + schedule: + - cron: '0 */4 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_Publish_Beam_SDK_Snapshots: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.container_task }}) + strategy: + fail-fast: false + matrix: + job_name: ["beam_Publish_Beam_SDK_Snapshots"] + job_phrase: ["N/A"] + container_task: ["go:container", "java:container:java8", "java:container:java11", "java:container:java17", "python:container:py38", "python:container:py39", "python:container:py310", "python:container:py311"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.container_task }}) + - name: Setup Java environment + if: ${{ startsWith(matrix.container_task, 'java') }} + uses: ./.github/actions/setup-environment-action + with: + java-version: 11 + - name: Setup Python environment + if: ${{ startsWith(matrix.container_task, 'python') }} + uses: ./.github/actions/setup-environment-action + with: + python-version: '3.8' + - name: run Publish Beam SDK Snapshots script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:${{ matrix.container_task }}:dockerTagPush + arguments: | + -Pjava11Home=$JAVA_HOME_11_X64 \ + -Pdocker-repository-root=gcr.io/apache-beam-testing/beam-sdk \ + -Pdocker-tag-list=${{ github.sha }},latest \ No newline at end of file From 7462069f8ced2491c3e5c90d2364c96ee002171b Mon Sep 17 00:00:00 2001 From: magicgoody <131876064+magicgoody@users.noreply.github.com> Date: Fri, 6 Oct 2023 19:17:53 +0600 Subject: [PATCH 14/85] Github Workflow Replacement for Jenkins Jobs, beam_LoadTests_Python_GBK_Dataflow_* (#28827) * beam_LoadTests_Python_GBK_Dataflow * fix permissions * removed unused arg * steps name fix * comment fixed, num_workers fixed --- ...am_LoadTests_Python_GBK_Dataflow_Batch.yml | 134 ++++++++++++++++++ ...oadTests_Python_GBK_Dataflow_Streaming.yml | 100 +++++++++++++ ...GBK_Dataflow_Batch_2GB_of_100B_records.txt | 28 ++++ ...BK_Dataflow_Batch_2GB_of_100kB_records.txt | 28 ++++ ..._GBK_Dataflow_Batch_2GB_of_10B_records.txt | 28 ++++ ...4_times_with_2GB_10-byte_records_total.txt | 28 ++++ ...8_times_with_2GB_10-byte_records_total.txt | 28 ++++ ...Dataflow_Streaming_2GB_of_100B_records.txt | 30 ++++ ...ataflow_Streaming_2GB_of_100kB_records.txt | 30 ++++ ..._Dataflow_Streaming_2GB_of_10B_records.txt | 30 ++++ ...4_times_with_2GB_10-byte_records_total.txt | 30 ++++ ...8_times_with_2GB_10-byte_records_total.txt | 30 ++++ 12 files changed, 524 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt create mode 100644 .github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml new file mode 100644 index 000000000000..fa3459992d0e --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml @@ -0,0 +1,134 @@ +# 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. + +name: LoadTests Python GBK Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 2 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_GBK_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python GBK Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_GBK_Dataflow_Batch"] + job_phrase: ["Run Load Tests Python GBK Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run 2GB of 10B records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Dataflow_Batch_test_arguments_1 }} --job_name=load-tests-python-dataflow-batch-gbk-1-${{env.NOW_UTC}}' \ + - name: run 2GB of 100B records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Dataflow_Batch_test_arguments_2 }} --job_name=load-tests-python-dataflow-batch-gbk-2-${{env.NOW_UTC}}' \ + - name: run 2GB of 100kB records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Dataflow_Batch_test_arguments_3 }} --job_name=load-tests-python-dataflow-batch-gbk-3-${{env.NOW_UTC}}' \ + - name: run fanout 4 times with 2GB 10-byte records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Dataflow_Batch_test_arguments_4 }} --job_name=load-tests-python-dataflow-batch-gbk-4-${{env.NOW_UTC}}' \ + - name: run fanout 8 times with 2GB 10-byte records total test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Dataflow_Batch_test_arguments_5 }} --job_name=load-tests-python-dataflow-batch-gbk-5-${{env.NOW_UTC}}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml new file mode 100644 index 000000000000..da0cf8eefbb8 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml @@ -0,0 +1,100 @@ +# 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. + +name: LoadTests Python GBK Dataflow Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '10 4 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_GBK_Dataflow_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python GBK Dataflow Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_GBK_Dataflow_Streaming"] + job_phrase: ["Run Load Tests Python GBK Dataflow Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt + - name: get current time + run: echo "NOW_UTC=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_ENV + # The env variable is created and populated in the test-arguments-action as "_test_arguments_" + - name: run 2GB of 100kB records test + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_GBK_Dataflow_Streaming_test_arguments_1 }} --job_name=load-tests-python-dataflow-streaming-gbk-3-${{env.NOW_UTC}}' \ + + # // TODO(https://github.com/apache/beam/issues/20403). Skipping some cases because they are too slow: + # load-tests-python-dataflow-streaming-gbk-1 + # load-tests-python-dataflow-streaming-gbk-2 + # load-tests-python-dataflow-streaming-gbk-4 + # load-tests-python-dataflow-streaming-gbk-5 \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt new file mode 100644 index 000000000000..ad05bf1e85d3 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100B_records.txt @@ -0,0 +1,28 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_gbk_2 +--influx_measurement=python_batch_gbk_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..8d3358a12f98 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_100kB_records.txt @@ -0,0 +1,28 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_gbk_3 +--influx_measurement=python_batch_gbk_3 +--input_options=''{\\"num_records\\":20000,\\"key_size\\":10000,\\"value_size\\":90000,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt new file mode 100644 index 000000000000..885c5ca61954 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_2GB_of_10B_records.txt @@ -0,0 +1,28 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_gbk_1 +--influx_measurement=python_batch_gbk_1 +--input_options=''{\\"num_records\\":200000000,\\"key_size\\":1,\\"value_size\\":9,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..1663e646f542 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,28 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_gbk_4 +--influx_measurement=python_batch_gbk_4 +--input_options=''{\\"num_records\\":5000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=4 +--num_workers=16 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..4a1768c9d17d --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Batch_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,28 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_gbk_5 +--influx_measurement=python_batch_gbk_5 +--input_options=''{\\"num_records\\":2500000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=8 +--num_workers=16 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt new file mode 100644 index 000000000000..057f71d5627c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100B_records.txt @@ -0,0 +1,30 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_gbk_2 +--influx_measurement=python_streaming_gbk_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt new file mode 100644 index 000000000000..57c1be11d592 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_100kB_records.txt @@ -0,0 +1,30 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_gbk_3 +--influx_measurement=python_streaming_gbk_3 +--input_options=''{\\"num_records\\":20000,\\"key_size\\":10000,\\"value_size\\":90000,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt new file mode 100644 index 000000000000..64d224a4663f --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_2GB_of_10B_records.txt @@ -0,0 +1,30 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_gbk_1 +--influx_measurement=python_streaming_gbk_1 +--input_options=''{\\"num_records\\":200000000,\\"key_size\\":1,\\"value_size\\":9,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..8e38713cc66d --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_4_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,30 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_gbk_4 +--influx_measurement=python_streaming_gbk_4 +--input_options=''{\\"num_records\\":5000000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=4 +--num_workers=16 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt new file mode 100644 index 000000000000..35508480662c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_GBK_Dataflow_Streaming_fanout_8_times_with_2GB_10-byte_records_total.txt @@ -0,0 +1,30 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_gbk_5 +--influx_measurement=python_streaming_gbk_5 +--input_options=''{\\"num_records\\":2500000,\\"key_size\\":10,\\"value_size\\":90,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--fanout=8 +--num_workers=16 +--autoscaling_algorithm=NONE +--streaming +--experiments=use_runner_v2 +--runner=DataflowRunner \ No newline at end of file From 2be0605aeb79261739706e5d9befd1ff0bc48657 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 6 Oct 2023 09:19:06 -0400 Subject: [PATCH 15/85] Bump github.com/proullon/ramsql from 0.1.2 to 0.1.3 in /sdks (#28857) Bumps [github.com/proullon/ramsql](https://github.com/proullon/ramsql) from 0.1.2 to 0.1.3. - [Release notes](https://github.com/proullon/ramsql/releases) - [Commits](https://github.com/proullon/ramsql/compare/v0.1.2...v0.1.3) --- updated-dependencies: - dependency-name: github.com/proullon/ramsql 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 | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index fb883c93af19..d1c98259f1e9 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -45,7 +45,7 @@ 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/proullon/ramsql v0.1.2 + github.com/proullon/ramsql v0.1.3 github.com/spf13/cobra v1.7.0 github.com/testcontainers/testcontainers-go v0.24.1 github.com/tetratelabs/wazero v1.5.0 diff --git a/sdks/go.sum b/sdks/go.sum index 1fcb968e118f..1c39718964d0 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -386,8 +386,8 @@ github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZN github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c h1:ncq/mPwQF4JjgDlrVEn3C11VoGHZN7m8qihwgMEtzYw= github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c/go.mod h1:OmDBASR4679mdNQnz2pUhc2G8CO2JrUAVFDRBDP/hJE= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/proullon/ramsql v0.1.2 h1:PTtsy2iml/CW3Lsopyr86dlIs7JyYEmfLrfYvQVXD2U= -github.com/proullon/ramsql v0.1.2/go.mod h1:CFGqeQHQpdRfWqYmWD3yXqPTEaHkF4zgXy1C6qDWc9E= +github.com/proullon/ramsql v0.1.3 h1:/LRcXJf4lEmhdb4tYcci473I2VynjcZSzh2hsjJ8rSk= +github.com/proullon/ramsql v0.1.3/go.mod h1:CFGqeQHQpdRfWqYmWD3yXqPTEaHkF4zgXy1C6qDWc9E= 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= From b87629928c92b3df4ba88fb42184d5ed505dbe2e Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Fri, 6 Oct 2023 11:01:34 -0400 Subject: [PATCH 16/85] Enable typevar-name-mismatch check (#28849) * Enable typevar-name-mismatch check * Trailing whitespace --- sdks/python/.pylintrc | 1 - sdks/python/apache_beam/typehints/decorators_test.py | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/.pylintrc b/sdks/python/.pylintrc index 250932e79812..e649e07e5ef6 100644 --- a/sdks/python/.pylintrc +++ b/sdks/python/.pylintrc @@ -140,7 +140,6 @@ disable = stop-iteration-return, super-init-not-called, superfluous-parens, - typevar-name-mismatch, #TODO(https://github.com/apache/beam/issues/28241) Enable and fix warnings try-except-raise, undefined-variable, unexpected-keyword-arg, diff --git a/sdks/python/apache_beam/typehints/decorators_test.py b/sdks/python/apache_beam/typehints/decorators_test.py index 239c9bd57078..3baf9fa8322f 100644 --- a/sdks/python/apache_beam/typehints/decorators_test.py +++ b/sdks/python/apache_beam/typehints/decorators_test.py @@ -38,6 +38,7 @@ T = TypeVariable('T') # Name is 'T' so it converts to a beam type with the same name. # mypy requires that the name of the variable match, so we must ignore this. +# pylint: disable=typevar-name-mismatch T_typing = typing.TypeVar('T') # type: ignore From b9398e0e3e9e9b448fb082cb7ed5893af2d202f0 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Fri, 6 Oct 2023 21:35:25 +0400 Subject: [PATCH 17/85] Add Java JMH workflow (#28860) * Add Java JMH workflow * Refactoring --- .github/workflows/beam_Java_JMH.yml | 68 +++++++++++++++++++++++++++++ 1 file changed, 68 insertions(+) create mode 100644 .github/workflows/beam_Java_JMH.yml diff --git a/.github/workflows/beam_Java_JMH.yml b/.github/workflows/beam_Java_JMH.yml new file mode 100644 index 000000000000..07beb1dadba1 --- /dev/null +++ b/.github/workflows/beam_Java_JMH.yml @@ -0,0 +1,68 @@ +# 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. + +name: Java JMH + +on: + schedule: + - cron: '0 0 * * 0' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + INFLUXDB_HOST: http://10.128.0.96:8086 + INFLUXDB_DATABASE: beam_test_metrics + +jobs: + beam_Java_JMH: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 900 + name: "beam_Java_JMH" + steps: + - uses: actions/checkout@v3 + - name: run the Java JMH micro-benchmark harness suite + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:harness:jmh:jmh + - name: run the Java JMH micro-benchmark core suite + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:core:jmh:jmh \ No newline at end of file From 57821c191d322f9f21c01a34c55e0c40eda44f1e Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 4 Oct 2023 09:53:21 -0400 Subject: [PATCH 18/85] Add python wheel staging to GitHub Actions build_release_candidate --- .github/workflows/build_release_candidate.yml | 76 ++++++++++++++++++- .../download_github_actions_artifacts.py | 15 ++-- 2 files changed, 81 insertions(+), 10 deletions(-) diff --git a/.github/workflows/build_release_candidate.yml b/.github/workflows/build_release_candidate.yml index ded089667364..3956551431c6 100644 --- a/.github/workflows/build_release_candidate.yml +++ b/.github/workflows/build_release_candidate.yml @@ -20,7 +20,7 @@ on: description: Your Apache password. Required if you want to stage artifacts into https://dist.apache.org/repos/dist/dev/beam/ required: false BEAM_SITE_TOKEN: - description: Github Personal Access Token with repo permission if you want to create the beam-site docs PR. See https://docs.github.com/en/authentication/keeping-your-account-and-data-secure/managing-your-personal-access-tokens. + description: Github Personal Access Token with apache/beam-site repo permission if you want to create the beam-site docs PR. See https://docs.github.com/en/authentication/keeping-your-account-and-data-secure/managing-your-personal-access-tokens. default: '' PUBLISH_JAVA_ARTIFACTS: description: Whether to publish java artifacts to https://repository.apache.org/#stagingRepositories (yes/no) @@ -34,6 +34,10 @@ on: description: Whether to stage SDK docker images to docker hub Apache organization required: true default: 'no' + STAGE_PYTHON_ARTIFACTS: + description: Whether to stage the python artifacts into https://dist.apache.org/repos/dist/dev/beam/ + required: true + default: 'no' CREATE_BEAM_SITE_PR: description: Whether to create the documentation update PR against apache/beam-site. required: true @@ -147,7 +151,74 @@ jobs: svn status svn commit -m "Staging Java artifacts for Apache Beam ${{ github.event.inputs.RELEASE }} RC${{ github.event.inputs.RC }}" --non-interactive --username ${{ github.event.inputs.APACHE_ID }} --password ${{ github.event.inputs.APACHE_PASSWORD }} - + stage_python_artifacts: + if: ${{github.event.inputs.STAGE_PYTHON_ARTIFACTS == 'yes'}} + runs-on: ubuntu-latest + steps: + - name: Validate and mask apache id/password + run: | + echo "::add-mask::${{ github.event.inputs.APACHE_PASSWORD }}" + if [ "${{ github.event.inputs.APACHE_ID }}" == "" ] + then + echo "Must provide an apache id to stage artifacts to https://dist.apache.org/repos/dist/dev/beam/" + fi + if [ "${{ github.event.inputs.APACHE_PASSWORD }}" == "" ] + then + echo "Must provide an apache password to stage artifacts to https://dist.apache.org/repos/dist/dev/beam/" + fi + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Import GPG key + id: import_gpg + uses: crazy-max/ghaction-import-gpg@111c56156bcc6918c056dbef52164cfa583dc549 + with: + gpg_private_key: ${{ secrets.GPG_PRIVATE_KEY }} + - name: stage python artifacts + env: + RC_TAG: "v${{ github.event.inputs.RELEASE }}-RC${{ github.event.inputs.RC }}" + GIT_REPO_BASE_URL: https://github.com/apache/beam + RELEASE_DIR: "beam/${{ github.event.inputs.RELEASE }}" + SCRIPT_DIR: release/src/main/scripts + GITHUB_TOKEN: ${{ secrets.GITHUB_TOKEN }} + run: | + svn co https://dist.apache.org/repos/dist/dev/beam + mkdir -p "${SVN_ARTIFACTS_DIR}" + + RELEASE_COMMIT=$(git rev-list -n 1 "tags/${RC_TAG}") + + python "${SCRIPT_DIR}/download_github_actions_artifacts.py" \ + --github-token-var GITHUB_TOKEN \ + --repo-url "${GIT_REPO_BASE_URL}" \ + --rc-tag "${RC_TAG}" \ + --release-commit "${RELEASE_COMMIT}" \ + --artifacts_dir "${RELEASE_DIR}/python" + + cd "${RELEASE_DIR}"/python + + echo "------Checking Hash Value for apache-beam-${RELEASE}.zip-----" + sha512sum -c "apache-beam-${RELEASE}.zip.sha512" + + echo "------Signing Source Release apache-beam-${RELEASE}.zip------" + gpg --local-user "${{steps.import_gpg.outputs.name}}" --armor --detach-sig "apache-beam-${RELEASE}.zip" + + for artifact in *.whl; do + echo "----------Checking Hash Value for ${artifact} wheel-----------" + sha512sum -c "${artifact}.sha512" + done + + for artifact in *.whl; do + echo "------------------Signing ${artifact} wheel-------------------" + gpg --local-user "${{steps.import_gpg.outputs.name}}" --armor --batch --yes --detach-sig "${artifact}" + done + + cd .. + svn add --force python + svn status + svn commit --no-auth-cache -m "Staging Python artifacts for Apache Beam ${RELEASE} RC${RC_NUM}" + + stage_docker: if: ${{github.event.inputs.STAGE_DOCKER_ARTIFACTS == 'yes'}} # Note: if this ever changes to self-hosted, remove the "Remove default github maven configuration" step @@ -192,7 +263,6 @@ jobs: - name: Push docker images run: ./gradlew :pushAllDockerImages -PisRelease -Pdocker-pull-licenses -Pprune-images -Pdocker-tag=${{ github.event.inputs.RELEASE }}rc${{ github.event.inputs.RC }} -Pjava11Home=${{steps.export-java11.outputs.JAVA11_HOME}} --no-daemon --no-parallel - beam_site_pr: if: ${{github.event.inputs.CREATE_BEAM_SITE_PR == 'yes'}} # Note: if this ever changes to self-hosted, remove the "Remove default github maven configuration" step diff --git a/release/src/main/scripts/download_github_actions_artifacts.py b/release/src/main/scripts/download_github_actions_artifacts.py index 181fd0c8b92b..5fbeb51a10cd 100644 --- a/release/src/main/scripts/download_github_actions_artifacts.py +++ b/release/src/main/scripts/download_github_actions_artifacts.py @@ -44,7 +44,7 @@ def parse_arguments(): description= "Script for downloading GitHub Actions artifacts from 'Build python wheels' workflow." ) - parser.add_argument("--github-user", required=True) + parser.add_argument("--github-token-var", required=False, default='GITHUB_TOKEN') parser.add_argument("--repo-url", required=True) parser.add_argument("--rc-tag", required=True) parser.add_argument("--release-commit", required=True) @@ -52,7 +52,7 @@ def parse_arguments(): parser.add_argument("--rc_number", required=False, default="") args = parser.parse_args() - github_token = ask_for_github_token() + github_token = get_github_token(args.github_token_var) print("You passed following arguments:") pprint.pprint({**vars(args), **{"github_token": github_token}}) @@ -61,7 +61,6 @@ def parse_arguments(): print("You said NO. Quitting ...") sys.exit(1) - user_github_id = args.github_user repo_url = args.repo_url rc_tag = args.rc_tag release_commit = args.release_commit @@ -69,11 +68,14 @@ def parse_arguments(): else os.path.abspath(args.artifacts_dir) rc_number = args.rc_number - return github_token, user_github_id, repo_url, rc_tag, release_commit, artifacts_dir, rc_number + return github_token, repo_url, rc_tag, release_commit, artifacts_dir, rc_number -def ask_for_github_token(): - """Ask for github token and print basic information about it.""" +def get_github_token(github_token_var): + """Get GitHub token from env or ask for it and print basic information about it.""" + if github_token_var in os.environ: + return os.environ[github_token_var] + url = "https://docs.github.com/en/github/authenticating-to-github/creating-a-personal-access-token" message = ( f"You need to have a github access token with public_repo scope. " @@ -321,7 +323,6 @@ def extract_single_artifact(file_path, output_dir): ) ( github_token, - user_github_id, repo_url, rc_tag, release_commit, From 14950aab7947c6a5819ea666da7aa298782726a0 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Wed, 4 Oct 2023 18:31:46 -0400 Subject: [PATCH 19/85] Update release guide somewhat according to 2.51.0 release experience --- contributor-docs/release-guide.md | 1092 +++++++++++++++-------------- 1 file changed, 574 insertions(+), 518 deletions(-) diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index f066047f6df2..189bc61e15e6 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -16,198 +16,206 @@ limitations under the License. ## Introduction -The Apache Beam project periodically declares and publishes releases. A -release is one or more packages of the project artifact(s) that are approved -for general public distribution and use. They may come with various degrees of -caveat regarding their perceived quality and potential for change, such as -“alpha”, “beta”, “incubating”, “stable”, etc. - -The Beam community treats releases with great importance. They are a public +The Beam community treats releases with great importance. They are a public face of the project and most users interact with the project only through the releases. Releases are signed off by the entire Beam community in a public vote. Each release is executed by a *Release Manager*, who is selected among the Beam committers. This document describes the process that the Release Manager -follows to perform a release. Any changes to this process should be discussed -and adopted on the [dev@ mailing list](/get-started/support/). +follows to perform a release. -Please remember that publishing software has legal consequences. This guide -complements the foundation-wide [Product Release -Policy](https://www.apache.org/dev/release.html) and [Release Distribution -Policy](https://www.apache.org/dev/release-distribution). +Please remember that publishing software has legal consequences. This guide +complements the foundation-wide guides: -### Overview + - [Product Release Policy](https://www.apache.org/dev/release.html) + - [Release Distribution + Policy](https://www.apache.org/dev/release-distribution). -Release step flow chart +### What is in a Beam release -The release process consists of several steps: +A Beam release consists of the following: -1. Decide to release -2. Prepare for the release -3. Build a release candidate -4. Verify & vote on the release candidate -5. If necessary, fix any issues and go back to "Build a release candidate" -6. Finalize the release -7. Promote the release + - ASF source zips archived on + [dist.apache.org](https://dist.apache.org/release/beam) (later archived to + [archive.apache.org](https://archive.apache.org/dist/beam) + - Java jars and poms published to [Maven + Central](https://mvnrepository.com/artifact/org.apache.beam) + - Python wheels published to [pypi](https://pypi.org/project/apache-beam/) + - Go artifacts published to + [pkg.go.dev](https://pkg.go.dev/github.com/apache/beam) + - Docker images published to + [dockerhub](https://hub.docker.com/search?q=apache%2Fbeam&type=image) + - A tag on GitHub indicating the commit from which the release was built ------------- +In addition, each release is accompanied by: -## 1. Decide to release + - A blog post announcing the release and describing the changes + - An update to the webpage to indicate the latest version -Deciding to release and selecting a Release Manager is the first step of the release process. -This is a consensus-based decision of the entire community. +### Phases of the release process -Anybody can propose a release on the dev@ mailing list, giving a solid argument and nominating a committer as the Release Manager (including themselves). -There’s no formal process, no vote requirements, and no timing requirements. Any objections should be resolved by consensus before starting the release. +The release process consists of several phases: -In general, the community prefers to have a rotating set of 3-5 Release Managers. -Keeping a small core set of managers allows enough people to build expertise in this area and improve processes over time, without Release Managers needing to re-learn the processes for each release. -That said, if you are a committer interested in serving the community in this way, please reach out to the community on the dev@ mailing list. - -### Checklist to proceed to the next step +1. Prepare for release +2. Stabilize the release branch / burn down release-blocking issues +3. Build a release candidate +4. Validate and approve the release candidate +5. Finalize the release +6. Promote the release +7. Post-release tasks -- [ ] Community agrees to release -- [ ] Community selects a committer as Release Manager +------------ -------- +## Prepare for release (~1 week before release cut) -## 2. Prepare for the release (~1 week before branch cut) +The following steps take place before the release branch is cut. -Before your first release, you should perform one-time configuration steps. -This will set up your security keys for signing the release and access to various release repositories. +### Decide to release -To prepare for each release, you should audit the project status in the GitHub issue tracker, and do necessary bookkeeping. -Finally, create a release branch from which individual release candidates will be built. +Deciding to release and selecting a Release Manager is the first step of the +release process. This is a consensus-based decision of the entire community. +Anybody can propose a release on the `dev@` list. There is no formal process, +no vote requirements, and no timing requirements. A committer must be +identified to be the Release Manager. In practice, most often a committer both +proposes to release and volunteers themselves as Release Manager. -__NOTE__: If you are using [GitHub two-factor authentication](https://help.github.com/articles/securing-your-account-with-two-factor-authentication-2fa/) and haven't configure HTTPS access, -please follow [the guide](https://help.github.com/articles/creating-a-personal-access-token-for-the-command-line/) to configure command line access. +------- ### Create a new milestone in GitHub for the next release -When contributors resolve an issue in GitHub, they are tagging it with a release that will contain their changes. -With the release currently underway, new issues should be resolved against a subsequent future release. -Therefore, you should create a release item for this subsequent release, as follows: +When contributors resolve an issue in GitHub, they are tagging it with a +release that will contain their changes. With the release currently underway, +new issues should be resolved against a subsequent future release. Therefore, +you should create a release item for this subsequent release, as follows: -1. In GitHub, navigate to [`Issues > Milestones > New Milestone`](https://github.com/apache/beam/milestones). -2. Add a new release. Choose the next minor version number after the version currently underway, select the next release due date (generally 6 weeks from today’s date) as the `Start Date`, and choose `Create Milestone`. -3. At the end of the release, go to the same page and mark the recently released version as closed. +In GitHub, navigate to [`Issues > Milestones > New +Milestone`](https://github.com/apache/beam/milestones) and add a new +release for the next minor version after the version you are preparing +to release. -### Accounts +---- -Please have these credentials ready at hand, you will likely need to enter them multiple times: +### Prepare accounts, keys, etc -* Apache ID and Password; -* GitHub ID, Password, and Personal Access Token. -* PyPi account and apitoken -* DockerHub ID and Password with beam maintainer access -* GPG pass phrase & 16-digit key ID -* Access to Beam's Apache Nexus repository -* Account to access to apache-beam-testing Google Cloud Platform project. The account must have permissions to start Cloud Build triggers. Required for Playground environment update. (E-mail at dev@ mailing list to request access) +Before your first release, you need to make sure you have all the necessary +accounts, keys, and access for publishing the release. The release process also +requires a variety of API tokens, which you can generate now or later when they +are needed. -If you don't have a given credential, follow the 'one-time' instructions below. +These are the credentials you will need: ----- - -### One-time setup instructions + - Apache ID and Password + - GitHub ID, Password, and Personal Access Token + - PyPi account with beam maintainer access and API Token + - GPG pass phrase & 16-digit key ID + - Access to Beam's Apache Nexus repository + - Account to access to apache-beam-testing Google Cloud Platform project. The + account must have permissions to start Cloud Build triggers. Required for + Playground environment update. (E-mail at dev@ mailing list to request + access) #### Apache ID and Password -This is your Apache committer user name and password. You selected these when you became an Apache Beam Committer. +This is your Apache committer user name and password. You selected these when +you became an Apache Beam Committer. #### Github ID, Password, and Personal Access Token -For some scripts, you need a Personal Access Token with `repo` and `workflow` permissions. -They can be generated from this page: https://github.com/settings/tokens. -See https://docs.github.com/en/authentication/keeping-your-account-and-data-secure/managing-your-personal-access-tokens for details. - -#### Register to PyPI - -Release manager needs to have an account with PyPI. -If you need one, [register at PyPI](https://pypi.python.org/account/register/). -You also need to be a maintainer (or an owner) of the [apache-beam](https://pypi.python.org/pypi/apache-beam) package in order to push a new release. -Ask on the mailing list for assistance. - -Generate a [PyPI APIToken](https://pypi.org/help/#apitoken) for use during the release. - -#### Login to DockerHub -If you are a member of the [`beam` DockerHub team](https://hub.docker.com/orgs/apache/teams/beam), run the following command manually. -It will ask you to input your DockerHub ID and password if authorization info cannot be found from ~/.docker/config.json file. - -``` -docker login docker.io -``` - -After successful login, authorization info will be stored at ~/.docker/config.json file. -For example, -``` -"https://index.docker.io/v1/": { - "auth": "xxxxxx" -} -``` - -If you are not already a member of the `beam` team, please email `dev@` mailing list for help with any DockerHub related tasks. We are not able -to add more members to the DockerHub team because [the ASF has a limited number of seats available](https://infra.apache.org/docker-hub-policy.html). + - [ ] If you are using [GitHub two-factor + authentication](https://help.github.com/articles/securing-your-account-with-two-factor-authentication-2fa/) + and haven't configure HTTPS access, please follow [the + guide](https://help.github.com/articles/creating-a-personal-access-token-for-the-command-line/) + to configure command line access. + - [ ] Generate a Personal Access Token with `repo` and `workflow` permissions. + They can be generated from this page: https://github.com/settings/tokens. + See + https://docs.github.com/en/authentication/keeping-your-account-and-data-secure/managing-your-personal-access-tokens + for details. + +#### PyPI account and API token + + - [ ] [Create an account with PyPI](https://pypi.python.org/account/register/) + if you don't have one already. + - [ ] Become a maintainer (or an owner) of the + [apache-beam](https://pypi.python.org/pypi/apache-beam) package. + - [ ] Generate a [PyPI APIToken](https://pypi.org/help/#apitoken) for use + during the release. #### GPG Key -You need to have a GPG key to sign the release artifacts. -Please be aware of the ASF-wide [release signing guidelines](https://www.apache.org/dev/release-signing.html). -If you don’t have a GPG key associated with your Apache account, please create one according to the guidelines. - -There are 2 ways to configure your GPG key for release, either using release automation script(which is recommended), or running all commands manually. +You need to have a GPG key to sign the release artifacts. Please be aware of +the ASF-wide [release signing +guidelines](https://www.apache.org/dev/release-signing.html). If you don’t +have a GPG key associated with your Apache account, you must now create one +according to the guidelines. -##### Use preparation_before_release.sh to setup GPG -* **Script:** [preparation_before_release.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/preparation_before_release.sh) +Run the following helper script, or you can open it and run the commands +individually (helpful if it doesn't work as intended or if you already are +partially set up) -* **Usage** - ``` - ./release/preparation_before_release.sh - ``` -* **Tasks included** - 1. Help you create a new GPG key if you want. - 2. Configure ```git user.signingkey``` with chosen pubkey. - 3. Add chosen pubkey into [dev KEYS](https://dist.apache.org/repos/dist/dev/beam/KEYS) and [release KEYS](https://dist.apache.org/repos/dist/release/beam/KEYS) + ./release/src/main/scripts/preparation_before_release.sh - **NOTES**: Only PMC can write into [release repo](https://dist.apache.org/repos/dist/release/beam/). - 4. Start GPG agents. +> **__NOTE__**: +> When generating the key, please make sure you choose the key type as +> __RSA and RSA (default)__ and key size as __4096 bit__. -__NOTE__: When generating the key, please make sure you choose the key type as __RSA and RSA (default)__ and key size as __4096 bit__. +Now you should have: -* To run the commands manually, refer to the contents of `preparation_before_release.sh`. + - [ ] A GPG key meeting ASF guidelines + - [ ] The key added to + [dev KEYS](https://dist.apache.org/repos/dist/dev/beam/KEYS) and [release KEYS](https://dist.apache.org/repos/dist/release/beam/KEYS) + **NOTE**: Only PMC can write into [release repo](https://dist.apache.org/repos/dist/release/beam/). + - [ ] The `user.signingkey` set in your `.gitconfig` + - [ ] `gpg-agent` with the key loaded ##### Key ID -* You may need your Key ID for future steps. Determine your Apache GPG Key and Key ID as follows: +You may need your Key ID for future steps. Determine your Apache GPG Key and +Key ID as follows: - gpg --list-sigs --keyid-format LONG + gpg --list-sigs --keyid-format LONG - This will list your GPG keys. One of these should reflect your Apache account, for example: +This will list your GPG keys. One of these should reflect your Apache account, +for example: - -------------------------------------------------- - pub rsa4096/845E6689845E6689 2016-02-23 - uid Nomen Nescio - sub rsa4096/BA4D50BEBA4D50BE 2016-02-23 + -------------------------------------------------- + pub rsa4096/845E6689845E6689 2016-02-23 + uid Nomen Nescio + sub rsa4096/BA4D50BEBA4D50BE 2016-02-23 - Here, the key ID is the 16-digit hex string in the `pub` line: `845E6689845E6689`. +Here, the key ID is the 16-digit hex string in the `pub` line: `845E6689845E6689`. ##### Submit your GPG public key into Ubuntu OpenPGP Key Server -In order to make yourself have right permission to stage java artifacts in Apache Nexus staging repository, -please submit your GPG public key into the [Ubuntu OpenPGP Key Server](https://keyserver.ubuntu.com/). -You will need to use an ascii-armored version of your key. -This can be obtained by running `gpg --export --armor` and copying the whole block -(including `----- PGP PUBLIC KEY BLOCK-----`). +In order to make yourself have right permission to stage java artifacts in +Apache Nexus staging repository, please submit your GPG public key into the +[Ubuntu OpenPGP Key Server](https://keyserver.ubuntu.com/). + +You will need to use an ascii-armored version of your key. This can be +obtained by running: + + gpg --export --armor + +Copying the whole block including `-----START PGP PUBLIC KEY BLOCK-----` and +`-----END PGP PUBLIC KEY BLOCK-----` #### Access to Apache Nexus repository -Configure access to the [Apache Nexus repository](https://repository.apache.org/), which enables final deployment of releases to the Maven Central Repository. +Configure access to the [Apache Nexus +repository](https://repository.apache.org/), which enables final deployment of +releases to the Maven Central Repository. -1. You log in with your Apache account. -2. Confirm you have appropriate access by finding `org.apache.beam` under `Staging Profiles`. +1. Log in with your Apache account. +2. Confirm you have appropriate access by finding `org.apache.beam` under + `Staging Profiles`. 3. Navigate to your `Profile` (top right dropdown menu of the page). -4. Choose `User Token` from the dropdown, then click `Access User Token`. Copy a snippet of the Maven XML configuration block. -5. Insert this snippet twice into your global Maven `settings.xml` file, typically `${HOME}/.m2/settings.xml`. The end result should look like this, where `TOKEN_NAME` and `TOKEN_PASSWORD` are your secret tokens: +4. Choose `User Token` from the dropdown, then click `Access User Token`. Copy + a snippet of the Maven XML configuration block. +5. Insert this snippet + twice into your global Maven `settings.xml` file, typically + `${HOME}/.m2/settings.xml`. The end result should look like this, where + `TOKEN_NAME` and `TOKEN_PASSWORD` are your secret tokens: @@ -225,400 +233,407 @@ Configure access to the [Apache Nexus repository](https://repository.apache.org/ -********** +---- + +### Dependency checks -### Handle Per Release tasks +Each language has routine dependency maintenance that you should check now. #### Update base image dependencies for Python container images -The Python base container images have static pinned `requirements.txt` that are -designed to be compatible with our dependency constraints but also not cause -runtime installs to occur, which slow things down immensely. -These need to be updated at least once per release cycle to avoid -out of date dependencies. +The Python base container images have pinned `requirements.txt` that are +compatible with our dependency constraints, and design to avoid run-time +installs, since run-time installs cause large delays at start-up time. Ideally, +we this should happen regularly when dependencies update, but it is important +to ensure that they are fully up to date for each release. Follow the instructions at https://s.apache.org/beam-python-requirements-generate -Ideally, do the update at least a week before the release cut, so that any issues -related to the update have time to surface. - #### Update Go version used for container builds -Go makes security patch releases of their tooling. Ideally, we upgrade as soon -as possible, but it is also good to ensure we are up to date for each release. - -This potentially affects container bootloader security, and at the least can cause -false positives when an default-configuration scanner is pointed at our containers. +Go makes security patch releases of their tooling. This potentially affects +container bootloader security, and at the least can cause false positives when +an default-configuration scanner is pointed at our containers. Ideally, we +upgrade as soon as possible, but it is also good to ensure we are up to date +for each release. - - [ ] See if https://go.dev/doc/devel/release has a newer release. Update throughout - Beam. See example at https://github.com/apache/beam/pull/27900/files +See if https://go.dev/doc/devel/release has a newer release. Update throughout +Beam. See example at https://github.com/apache/beam/pull/27900/files #### Update the Java BOM -Tracked in Github issue https://github.com/apache/beam/issues/28379 - -Ideally, do the update at least a week before the release cut, so that any issues -related to the update have time to surface. +Google releases a BOM that pins compatible versions of their Java libraries. +Ideally, this update happens as far in advance of the release as possible, such +as just after a release. But if that was not done, consider doing it before +cutting the release branch. -#### Investigate performance regressions +To do so, follow instructions at +https://github.com/apache/beam/blob/master/contributor-docs/java-dependency-upgrades.md. -Check the Beam load tests for possible performance regressions. -Measurements are available on [metrics.beam.apache.org](http://metrics.beam.apache.org). +### Cut the release branch -All Runners which publish data should be checked for the following, in both *batch* and *streaming* mode: - -- [ParDo](http://metrics.beam.apache.org/d/MOi-kf3Zk/pardo-load-tests) and [GBK](http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-test): Runtime, latency, checkpoint duration -- [Nexmark](http://metrics.beam.apache.org/d/ahudA_zGz/nexmark): Query runtime for all queries -- [IO](http://metrics.beam.apache.org/d/bnlHKP3Wz/java-io-it-tests-dataflow): Runtime +> **Note** +> Wait until the proposed branch cut day! -If regressions are found, the release branch can still be created, but the regressions should be investigated and fixed as part of the release process. -The role of the release manager is to file GitHub issues for each regression with the milestone set to the to-be-released version. -The release manager oversees these just like any other issue marked with the milestone of the release. +We cut the release branch on time and do not block/delay branch cut for incoming +fixes. This is because bugs are always being introduced as part of normal +development. We cut the branch to prevent new bugs being introduced and _then_ +we fix and cherrypick any truly release-blocking problems. -The mailing list should be informed to allow fixing the regressions in the course of the release. Issues should be filed and tagged with the milestone. +In order to run this workflow, you will need to provide a Apache ID and Jenkins +API token. Your Jenkins API token can be generated by visiting +https://ci-beam.apache.org, signing in with your Apache credentials, then going +to `https://ci-beam.apache.org/user//configure` and clicking +`Add new token` in the API token section. -#### Triage release-blocking issues in GitHub +- [ ] Run + [cut_release_branch](https://github.com/apache/beam/actions/workflows/cut_release_branch.yml) + (click `run workflow`) -There could be outstanding release-blocking issues, which should be triaged before proceeding to build a release candidate. -We track them by assigning the blocked release to the issue's milestone before the issue is resolved. +The final state of the repository after release branch is cut should match this +diagram: -The release manager should triage what does and does not block a release. -The list of release-blocking issues is available at the [milestone status page](https://github.com/apache/beam/milestones). -Triage each unresolved issue with one of the following resolutions: - -* An issue should not block the release if the problem exists in the current released version or is a bug in new functionality that does not exist in the current released version. -* An issue should be a blocker if the problem is a regression between the currently released version and the release in progress and has no easy workaround. - -For all GitHub issues: +Increment minor version on master branch and set Dataflow container version on release branch -* If the issue has been resolved and the GitHub issue was not updated, resolve it accordingly. +This should be accomplished by the +[cut_release_branch](https://github.com/apache/beam/actions/workflows/cut_release_branch.yml) +workflow. This workflow will also update +[mass_comment.py](https://github.com/apache/beam/blob/master/release/src/main/scripts/mass_comment.py) +to contain all of the active Jenkins jobs. -For issues with type "Bug" or labeled "flaky": +The following must be manually done or confirmed: -* If the issue is a known continuously failing test, it is not acceptable to defer this until the next release. - Please work with the Beam community to resolve the issue. -* If the issue is a known flaky test, make an attempt to delegate a fix. - However, if the issue may take too long to fix (to the discretion of the release manager): - * Delegate manual testing of the flaky issue to ensure no release blocking issues. - * Update the milestone to the version of the next release. - Please consider discussing this with stakeholders and the dev@ mailing list, as appropriate. +- [ ] The `master` branch has the SNAPSHOT/dev version incremented. +- [ ] The release branch has the SNAPSHOT/dev version to be released. +- [ ] The Dataflow container image should be modified to the version to be released. +- [ ] Due to a bug/limitation in the workflow, you must navigate to the pull + request found in the logs and comment `Run Gradle Publish`. +- [ ] After publish, close the PR. +- [ ] Manually update `CHANGES.md` on `master` by adding a new section for the + next release + ([example](https://github.com/apache/beam/commit/96ab1fb3fe07acf7f7dc9d8c829ae36890d1535c)). -For all other GitHub issues: +#### Inform the mailing list -* If the issue has not been resolved and it is acceptable to defer this until the next release, update the milestone to the new version you just created. - Please consider discussing this with stakeholders and the dev@ mailing list, as appropriate. -* If the issue has not been resolved and it is not acceptable to release until it is fixed, the release cannot proceed. - Instead, work with the Beam community to resolve the issue. +The dev@ mailing list should be informed about the release branch being cut. +Alongside with this note, a list of pending issues and to-be-triaged issues +should be included. Afterwards, this list can be refined and updated by the +release manager and the Beam community. -If there is a bug found in the RC creation process/tools, those issues should be considered high priority and fixed in 7 days. -### Checklist to proceed to the next step +### Checklist to proceed to the next phase +- [ ] Community agrees to release +- [ ] Community selects a committer (you) as Release Manager - [ ] Next release has a milestone in github. - [ ] You have your various account credentials prepared. -- [ ] Per Release tasks for the current release have been handled. -- [ ] Open issues/PRs against the current release have been notified. -- [ ] Performance Regressions have been investigated and had issues filed. -- [ ] It is the proposed branch cut day. +- [ ] You checked the dependency maintenance for each language. +- [ ] The release branch is created. +- [ ] The `master` branch is moved along to the next release. +- [ ] You have informed `dev@beam.apache.org` that you have cut the branch and + are proceeding to stabilization -********** - -## 3. Build a release candidate - -Building a release candidate involves creating a release branch, running validation tests against the branch, filing issues, cherry picking fixes, -making a release candidate tag, and building all artifacts from that tag. - -### Create a release branch in apache/beam repository - -As a final step in preparation for the release, you should create the release branch, and update version information on the original branch. -This should happen once per release. If additional release candidates are required, they are built from later versions of this branch. - -The final state of the repository should match this diagram: - -Increment minor version on master branch and set Dataflow container version on release branch - -The key points to know: - -- The `master` branch has the SNAPSHOT/dev version incremented. -- The release branch has the SNAPSHOT/dev version to be released. -- The Dataflow container image should be modified to the version to be released. - -This will all be accomplished by the [cut_release_branch](https://github.com/apache/beam/actions/workflows/cut_release_branch.yml) -workflow. This workflow will also update [mass_comment.py](https://github.com/apache/beam/blob/master/release/src/main/scripts/mass_comment.py) -to contain all of the active Jenkins jobs. - -After updating the master branch, the workflow will also start a build of -[the nightly snapshot](https://ci-beam.apache.org/job/beam_Release_NightlySnapshot/) against master branch. -Some processes, including our archetype tests, rely on having a live SNAPSHOT of the current version from the `master` branch. -Once the release branch is cut, these SNAPSHOT versions are no longer found, so builds will be broken until a new snapshot is available. -The workflow starts the nightly snapshot by creating an empty PR against apache:master (which will be linked to in the logs). - -#### Use cut_release_branch workflow to cut a release branch - -* **Action:** [cut_release_branch](https://github.com/apache/beam/actions/workflows/cut_release_branch.yml) (click `run workflow`) +------- -In order to run this workflow, you will need to provide a Apache ID and Jenkins API token. -Your Jenkins API token can be generated by visiting https://ci-beam.apache.org, signing in with your Apache credentials, -then going to `https://ci-beam.apache.org/user//configure` and clicking `Add new token` in the API token section. +## Stabilize the release branch -* Tasks you need to do manually to __verify the SNAPSHOT build__ - 1. Check whether the Jenkins job gets triggered. If not, please comment ```Run Gradle Publish``` into the generated PR. - 2. After verifying build succeeded, you need to close PR manually. - 3. Manually update `CHANGES.md` on `master` by adding a new section for the next release ([example](https://github.com/apache/beam/commit/96ab1fb3fe07acf7f7dc9d8c829ae36890d1535c)). +Once the release branch is cut, your job is to make sure tests pass, fix bugs, +confirm performance, defer feature requests, etc, until the branch is ready for +the work of building a release candidate. ### Verify release branch -After the release branch is cut you need to make sure it builds and has no significant issues that would block the creation of the release candidate. -There are 2 ways to perform this verification, either running automation script(recommended), or running all commands manually. - -> Dataflow tests will fail if the Dataflow worker container is not created and published by this time. -> Should be done by Google, in response to the creation of the release branch, and docker images are hosted. -> This should not block creation of the first release candidate, but should block approval of the release. - -#### Run automation script (verify_release_build.sh) -* **Script:** [verify_release_build.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/verify_release_build.sh) +After the release branch is cut, make sure it builds and has no significant +issues that would block the creation of the release candidate. + +> **NOTE** +> Dataflow tests will fail if the Dataflow worker container is not created and +> published by this time. Should be done by Google, in response to the +> creation of the release branch, and docker images are hosted. This should +> not block creation of the first release candidate, but should block approval +> of the release. + +- **Script:** + [verify_release_build.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/verify_release_build.sh) + +- **Usage** + 1. Create a personal access token from your Github account. + See instruction [here](https://help.github.com/en/articles/creating-a-personal-access-token-for-the-command-line). + It'll be used by the script for accessing Github API. + You need to enable `repo` and `workflow` permissions for this token. + 2. Update required configurations listed in `RELEASE_BUILD_CONFIGS` in [script.config](https://github.com/apache/beam/blob/master/release/src/main/scripts/script.config) + 3. Then run + ``` + (cd release/src/main/scripts && ./verify_release_build.sh) + ``` + 4. Trigger all Jenkins PostCommit jobs from the PR created by the previous step. + You can run [mass_comment.py](https://github.com/apache/beam/blob/master/release/src/main/scripts/mass_comment.py) to do that. + Or manually add one trigger phrase per PR comment. + See [jenkins_jobs.txt](https://github.com/apache/beam/blob/master/release/src/main/scripts/jenkins_jobs.txt) + for a full list of phrases. + +- **Tasks included in the script** + - Installs `hub` with your agreement and setup local git repo; + - Create a test PR against release branch; + +There are some projects that don't produce the artifacts, e.g. +`beam-test-tools`, you may be able to ignore failures there. + +To triage the failures and narrow things down you may want to look at +`settings.gradle.kts` and run the build only for the projects you're interested +at the moment, e.g. `./gradlew :runners:java-fn-execution`. + +The `verify_release_build.sh` script may include failing or flaky tests. For +each of the failing tests create a GitHub Issue with the following properties: -* **Usage** - 1. Create a personal access token from your Github account. - See instruction [here](https://help.github.com/en/articles/creating-a-personal-access-token-for-the-command-line). - It'll be used by the script for accessing Github API. - You need to enable `repo` and `workflow` permissions for this token. - 2. Update required configurations listed in `RELEASE_BUILD_CONFIGS` in [script.config](https://github.com/apache/beam/blob/master/release/src/main/scripts/script.config) - 3. Then run - ``` - (cd release/src/main/scripts && ./verify_release_build.sh) - ``` - 4. Trigger all Jenkins PostCommit jobs from the PR created by the previous step. - You can run [mass_comment.py](https://github.com/apache/beam/blob/master/release/src/main/scripts/mass_comment.py) to do that. - Or manually add one trigger phrase per PR comment. - See [jenkins_jobs.txt](https://github.com/apache/beam/blob/master/release/src/main/scripts/jenkins_jobs.txt) - for a full list of phrases. +* **Issue Type:** Bug -* **Tasks included in the script** - 5. Installs `hub` with your agreement and setup local git repo; - 6. Create a test PR against release branch; +* **Summary:** Name of failing gradle task and name of failing test (where applicable) in form of :MyGradleProject:SomeGradleTask NameOfFailedTest: Short description of failure -#### Verify the build succeeds +* **Priority:** P1 -* Tasks you need to do manually to __verify the build succeed__: - 1. Check the build result. - 2. If build failed, scan log will contain all failures. - 3. You should stabilize the release branch until release build succeeded. +* **Component:** "test-failures" -There are some projects that don't produce the artifacts, e.g. `beam-test-tools`, you may be able to ignore failures there. +* **Milestone:** Release number of verified release branch -To triage the failures and narrow things down you may want to look at `settings.gradle.kts` and run the build only for the projects you're interested at the moment, e.g. `./gradlew :runners:java-fn-execution`. +* **Description:** Description of failure -#### (Alternative) Run release build locally -You will need to have Python interpreters for all supported Python minor -versions to run Python tests. See Python installation tips in [Developer Wiki](https://cwiki.apache.org/confluence/display/BEAM/Python+Tips#PythonTips-InstallingPythoninterpreters). +### Investigate performance regressions -* **Run gradle release build** +Check the Beam load tests for possible performance regressions. Measurements +are available on [metrics.beam.apache.org](http://metrics.beam.apache.org). - 1. Clean current workspace +All Runners which publish data should be checked for the following, in both +*batch* and *streaming* mode: - ``` - git clean -fdx - ./gradlew clean - ``` +- [ParDo](http://metrics.beam.apache.org/d/MOi-kf3Zk/pardo-load-tests) and + [GBK](http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-test): Runtime, + latency, checkpoint duration +- [Nexmark](http://metrics.beam.apache.org/d/ahudA_zGz/nexmark): Query runtime + for all queries +- [IO](http://metrics.beam.apache.org/d/bnlHKP3Wz/java-io-it-tests-dataflow): Runtime - 2. Unlock the secret key - ``` - gpg --output ~/doc.sig --sign ~/.bashrc - ``` +If regressions are found, the release branch can still be created, but the +regressions should be investigated and fixed as part of the release process. +The role of the release manager is to file GitHub issues for each regression +with the milestone set to the to-be-released version. The release manager +oversees these just like any other issue marked with the milestone of the +release. - 3. Run build command - ``` - ./gradlew build -PisRelease --no-parallel --scan --stacktrace --continue - ``` +The mailing list should be informed to allow fixing the regressions in the +course of the release. Issues should be filed and tagged with the milestone. - To speed things up locally you might want to omit `--no-parallel`. You can also omit `--continue` - if you want build fails after the first error instead of continuing, it may be easier and faster - to find environment issues this way without having to wait until the full build completes. +### Triage release-blocking issues in GitHub -#### Create release-blocking issues in GitHub +There could be outstanding release-blocking issues, which should be triaged +before proceeding to build a release candidate. We track them by assigning the +blocked release to the issue's milestone before the issue is resolved. -The verify_release_build.sh script may include failing or flaky tests. -For each of the failing tests create a GitHub Issue with the following properties: +The release manager should triage what does and does not block a release. The +list of release-blocking issues is available at the [milestone status +page](https://github.com/apache/beam/milestones). Triage each unresolved issue +with one of the following resolutions: -* **Issue Type:** Bug + - An issue should not block the release if the problem exists in the current + released version or is a bug in new functionality that does not exist in the + current released version. + - An issue should be a blocker if the problem is a regression between the + currently released version and the release in progress and has no easy + workaround. -* **Summary:** Name of failing gradle task and name of failing test (where applicable) in form of :MyGradleProject:SomeGradleTask NameOfFailedTest: Short description of failure +For all GitHub issues: -* **Priority:** P1 + - If the issue has been resolved and the GitHub issue was not updated, + resolve it accordingly. -* **Component:** "test-failures" +For issues with type "Bug" or labeled "flaky": -* **Milestone:** Release number of verified release branch + - If the issue is a known continuously failing test, it is not acceptable to + defer this until the next release. Please work with the Beam community to + resolve the issue. + - If the issue is a known flaky test, make an attempt to delegate a fix. + However, if the issue may take too long to fix (to the discretion of the + release manager): + - Delegate manual testing of the flaky issue to ensure no release blocking issues. + - Update the milestone to the version of the next release. + Please consider discussing this with stakeholders and the dev@ mailing + list, as appropriate. -* **Description:** Description of failure +For all other GitHub issues: -#### Inform the mailing list + - If the issue has not been resolved and it is acceptable to defer this until the next release, update the milestone to the new version you just created. + Please consider discussing this with stakeholders and the dev@ mailing list, as appropriate. + - If the issue has not been resolved and it is not acceptable to release until it is fixed, the release cannot proceed. + Instead, work with the Beam community to resolve the issue. -The dev@ mailing list should be informed about the release branch being cut. -Alongside with this note, a list of pending issues and to-be-triaged issues should be included. -Afterwards, this list can be refined and updated by the release manager and the Beam community. +If there is a bug found in the RC creation process/tools, those issues should +be considered high priority and fixed in 7 days. ### Review cherry-picks -The release manager is empowered to triage issues, and accept or reject cherry-picks to the release branch. -Cherry picks are necessary if there are outstanding issues at time of the release branch cut, or issues were found in verification. +The release manager is empowered to triage issues, and accept or reject +cherry-picks to the release branch. Cherry picks are necessary if there are +outstanding issues at time of the release branch cut, or issues were found in +verification. + +Check if there are outstanding cherry-picks into the release branch, [e.g. for +`2.14.0`](https://github.com/apache/beam/pulls?utf8=%E2%9C%93&q=is%3Apr+base%3Arelease-2.14.0). +Make sure they have blocker Issues attached and are OK to get into the release +by checking with community if needed. + +You are encouraged to ask the following questions to be answered on each +cherry-pick PR and you can choose to reject cherry-pick requests if these +questions are not satisfactorily answered: + + - Is this a regression from a previous release? (If no, fix could go to a + newer version.) + - Is this a new feature or related to a new feature? (If yes, fix could go to + a new version.) + - Would this impact production workloads for users? (E.g. if this is a direct + runner only fix it may not need to be a cherry pick.) + - What percentage of users would be impacted by this issue if it is not fixed? + (E.g. If this is predicted to be a small number it may not need to be a + cherry pick.) + - Would it be possible for the impacted users to skip this version? (If users + could skip this version, fix could go to a newer version.) + +It is important to accept major/blocking fixes to isolated issues to make a +higher quality release. However, beyond that each cherry pick will increase +the time required for the release and add more last minute code to the release +branch. Neither late releases nor not fully tested code will provide positive +user value. + +> **Tip**: Another tool in your toolbox is the known issues section of the +> release blog. Consider adding known issues there for minor issues instead of +> accepting cherry picks to the release branch. + +## Build a release candidate + +From the release branch, building a candidate involves selecting a commit, +tagging that commit, and building the various artifacts against that commit. +You can also run verifications against the RC commit (verification will also +occur during voting phase). -Check if there are outstanding cherry-picks into the release branch, [e.g. for `2.14.0`](https://github.com/apache/beam/pulls?utf8=%E2%9C%93&q=is%3Apr+base%3Arelease-2.14.0). -Make sure they have blocker Issues attached and are OK to get into the release by checking with community if needed. +#### Checklist before proceeding -You are encouraged to ask the following questions to be answered on each cherry-pick PR and you can choose to reject cherry-pick requests if these questions are not satisfactorily answered: +- [ ] There are no release blocking GitHub issues. +- [ ] There are no open pull requests to release branch. +- [ ] Release Manager’s GPG key is published to `dist.apache.org`. +- [ ] Release Manager’s GPG key is configured in `git` configuration. +- [ ] Set `SIGNING_KEY` to the public key of the Manager's GPG key. +- [ ] Release Manager has `org.apache.beam` listed under `Staging Profiles` in Nexus. +- [ ] Release Manager’s Nexus User Token is configured in `settings.xml`. +- [ ] Set `JAVA_HOME` to JDK 8 (Example: `export JAVA_HOME=/example/path/to/java/jdk8`). +- [ ] Have Java 11 installed. -* Is this a regression from a previous release? (If no, fix could go to a newer version.) -* Is this a new feature or related to a new feature? (If yes, fix could go to a new version.) -* Would this impact production workloads for users? (E.g. if this is a direct runner only fix it may not need to be a cherry pick.) -* What percentage of users would be impacted by this issue if it is not fixed? (E.g. If this is predicted to be a small number it may not need to be a cherry pick.) -* Would it be possible for the impacted users to skip this version? (If users could skip this version, fix could go to a newer version.) +### Tag a chosen commit for the RC -It is important to accept major/blocking fixes to isolated issues to make a higher quality release. -However, beyond that each cherry pick will increase the time required for the release and add more last minute code to the release branch. -Neither late releases nor not fully tested code will provide positive user value. +Release candidates are built from single tagged commits off the release branch. +When you have identified a good commit on the release branch, run +[choose_rc_commit.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/choose_rc_commit.sh) +to set it up correctly. -__Tip__: Another tool in your toolbox is the known issues section of the release blog. -Consider adding known issues there for minor issues instead of accepting cherry picks to the release branch. + ./release/src/main/scripts/choose_rc_commit.sh \ + --release "${RELEASE_VERSION}" \ + --rc "${RC_NUM}" \ + --commit "${COMMIT_REF}" \ + --clone \ + --push-tag -### Build release artifacts +You can do a dry run by omitting the `--push-tag` flag. Then it will only clone +the repo, adjust the version, and add the tag locally. If it looks good, run it +again with `--push-tag`. If you already have a clone that includes the +`${COMMIT_REF}` then you can omit `--clone`. This is perfectly safe since the +script does not depend on the current working tree. -Once the branch is verified, it's time to build +See the source of the script for more details, or to run commands manually in +case of a problem. -#### Checklist before proceeding +The final state of the repository after an RC commit is chosen should match +this diagram: -- [ ] Release Manager’s GPG key is published to `dist.apache.org`; -- [ ] Release Manager’s GPG key is configured in `git` configuration; -- [ ] Set `SIGNING_KEY` to the public key of the Manager's GPG key; -- [ ] Release Manager has `org.apache.beam` listed under `Staging Profiles` in Nexus; -- [ ] Release Manager’s Nexus User Token is configured in `settings.xml`; -- [ ] GitHub issue release item for the subsequent release has been created; -- [ ] All test failures from branch verification have associated GitHub issues; -- [ ] There are no release blocking GitHub issues; -- [ ] Release branch has been created; -- [ ] There are no open pull requests to release branch; -- [ ] Originating branch has the version information updated to the new version; -- [ ] Nightly snapshot is in progress (do revisit it continually); -- [ ] Set `JAVA_HOME` to JDK 8 (Example: `export JAVA_HOME=/example/path/to/java/jdk8`). -- [ ] Have Java 11 installed. +Set version to non-SNAPSHOT, non-dev, on tagged RC commit -The core of the release process is the build-vote-fix cycle. -Each cycle produces one release candidate. -The Release Manager repeats this cycle until the community approves one release candidate, which is then finalized. +The following should be confirmed: -For this step, we recommend you using automation script to create a RC, but you still can perform all steps manually if you want. +- [ ] The release branch is unchanged. +- [ ] There is a commit not on the release branch with the version adjusted. +- [ ] The RC tag points to that commit. -#### Tag a chosen commit for the RC +### Run build_release_candidate GitHub Action to create a release candidate -Release candidates are built from single commits off the release branch. -Before building, the version must be set to a non-SNAPSHOT, non-dev version. -The final state of the repository should match this diagram: +Note: This step is partially automated (in progress), so part of the RC +creation is done by GitHub Actions and the rest is done by a script. You don't +need to wait for the action to complete to start running the script. -Set version to non-SNAPSHOT, non-dev, on tagged RC commit +**Action** [build_release_candidate](https://github.com/apache/beam/actions/workflows/build_release_candidate.yml) (click `run workflow`) -- The release branch is unchanged. -- There is a commit not on the release branch with the version adjusted. -- The RC tag points to that commit. +**The action will:** -* **Script:** [choose_rc_commit.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/choose_rc_commit.sh) +1. Clone the repo at the selected RC tag. +2. Run gradle publish to push java artifacts into Maven staging repo. +3. Stage SDK docker images to [docker hub Apache + organization](https://hub.docker.com/search?q=apache%2Fbeam&type=image). +4. Build javadoc, pydoc, typedocs for a PR to update beam-site. + - **NOTE**: Do not merge this PR until after an RC has been approved (see + "Finalize the Release"). -* **Usage** +### Verify source distributions - ./release/src/main/scripts/choose_rc_commit.sh \ - --release "${RELEASE_VERSION}" \ - --rc "${RC_NUM}" \ - --commit "${COMMIT_REF}" \ - --clone \ - --push-tag + - [ ] Verify that the source zip of the whole project is present in [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam). + - [ ] Verify that the Python binaries are present in [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam). -You can do a dry run by omitting the `--push-tag` flag. Then it will only clone the repo, -adjust the version, and add the tag locally. If it looks good, run it again with `--push-tag`. -If you already have a clone that includes the `${COMMIT_REF}` then you can omit `--clone`. This -is perfectly safe since the script does not depend on the current working tree. +### Verify docker images -See the source of the script for more details, or to run commands manually in case of a problem. +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}` -#### Run build_release_candidate GitHub Action to create a release candidate +Verify that third party licenses are included in Docker. You can do this with a simple script: -Note: This step is partially automated (in progress), so part of the rc creation is done by GitHub Actions and the rest is done by a script. -You don't need to wait for the action to complete to start running the script. + for pyver in 3.8 3.9 3.10 3.11; do + docker run --rm --entrypoint sh \ + apache/beam_python${pyver}_sdk:2.51.0rc1 \ + -c 'ls -al /opt/apache/beam/third_party_licenses/ | wc -l' + done -* **Action** [build_release_candidate](https://github.com/apache/beam/actions/workflows/build_release_candidate.yml) (click `run workflow`) + for javaver in 8 11 17; do + docker run --rm --entrypoint sh \ + apache/beam_java${pyver}_sdk:2.51.0rc1 \ + -c 'ls -al /opt/apache/beam/third_party_licenses/ | wc -l' + done -* **The script will:** - 1. Clone the repo at the selected RC tag. - 2. Run gradle publish to push java artifacts into Maven staging repo. - 3. Stage SDK docker images to [docker hub Apache organization](https://hub.docker.com/search?q=apache%2Fbeam&type=image). - 4. Build javadoc, pydoc, typedocs for a PR to update beam-site. - * **NOTE**: Do not merge this PR until after an RC has been approved (see "Finalize the Release"). +And you may choose to log in to the containers and inspect: -##### Tasks you need to do manually + docker run --rm -it --entrypoint=/bin/bash \ + apache/beam_java${ver}_sdk:${RELEASE_VERSION}rc${RC_NUM} + ls -al /opt/apache/beam/third_party_licenses/ + +### Publish Java staging artifacts (manual) -Publish staging artifacts 1. Log in to the [Apache Nexus](https://repository.apache.org/#stagingRepositories) website. 2. Navigate to Build Promotion -> Staging Repositories (in the left sidebar). 3. Select repository `orgapachebeam-NNNN`. 4. Click the Close button. 5. When prompted for a description, enter “Apache Beam, version X, release candidate Y”. - 6. Review all staged artifacts on `https://repository.apache.org/content/repositories/orgapachebeam-NNNN/`. + 6. Review all staged artifacts on `https://repository.apache.org/content/repositories/orgapachebeam-NNNN/`. They should contain all relevant parts for each module, including `pom.xml`, jar, test jar, javadoc, etc. Artifact names should follow [the existing format](https://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.beam%22) in which artifact name mirrors directory structure, e.g., `beam-sdks-java-io-kafka`. Carefully review any new artifacts. Some additional validation should be done during the rc validation step. -#### Run build_release_candidate.sh to create a release candidate -* **Script:** [build_release_candidate.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/build_release_candidate.sh) +### Upload `rc` artifacts to PyPI -* **Usage** +This step uploads artifacts such as `apache-beam-${RELEASE_VERSION}rc${RC_NUM}` +to PyPI, so the RC artifacts can be depended upon directly by consumers, for +ease of RC verification. - ./release/src/main/scripts/build_release_candidate.sh --release "${RELEASE_VERSION}" --rc "${RC_NUM}" --github-user "${GITHUB_USER}" --java11-home "${JAVA11_HOME}" --signing-key "${SIGNING_KEY}" - -* **The script will:** - 1. Clone the repo at the selected RC tag. - 2. Stage source release into dist.apache.org dev [repo](https://dist.apache.org/repos/dist/dev/beam/). -Skip this step if you already did it with the build_release_candidate GitHub Actions workflow. - 3. Stage, sign and hash python source distribution and wheels into dist.apache.org dev repo python dir - 4. Stage SDK docker images to [docker hub Apache organization](https://hub.docker.com/search?q=apache%2Fbeam&type=image). -Skip this step if you already did it with the build_release_candidate GitHub Actions workflow. -Note: if you are not a member of the [`beam` DockerHub team](https://hub.docker.com/orgs/apache/teams/beam) you will need -help with this step. Please email `dev@` mailing list and ask a member of the `beam` DockerHub team for help. - 5. Create a PR to update beam-site, changes includes: - * Copy python doc into beam-site - * Copy java doc into beam-site - * **NOTE**: Do not merge this PR until after an RC has been approved (see "Finalize the Release"). -Skip this step if you already did it with the build_release_candidate GitHub Actions workflow. - -##### Tasks you need to do manually - -Verify the script worked. - - 1. Verify that the source and Python binaries are present in [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam). - 2. Verify Docker images are published. How to find images: - 1. Visit [https://hub.docker.com/u/apache](https://hub.docker.com/search?q=apache%2Fbeam&type=image) - 2. Visit each repository and navigate to *tags* tab. - 3. Verify images are pushed with tags: ${RELEASE_VERSION}_rc{RC_NUM} - 3. Verify that third party licenses are included in Docker containers by logging in to the images. - - For Python SDK images, there should be around 80 ~ 100 dependencies. - Please note that dependencies for the SDKs with different Python versions vary. - Need to verify all Python images by replacing `${ver}` with each supported Python version `X.Y`. - ``` - docker run --rm -it --entrypoint=/bin/bash apache/beam_python${ver}_sdk:${RELEASE_VERSION}rc${RC_NUM} - ls -al /opt/apache/beam/third_party_licenses/ | wc -l - ``` - - For Java SDK images, there should be around 200 dependencies. - ``` - docker run --rm -it --entrypoint=/bin/bash apache/beam_java${ver}_sdk:${RELEASE_VERSION}rc${RC_NUM} - ls -al /opt/apache/beam/third_party_licenses/ | wc -l - ``` - -#### Upload release candidate to PyPi - -* **Script:** [deploy_release_candidate_pypi.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/deploy_release_candidate_pypi.sh) +**Script:** [deploy_release_candidate_pypi.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/deploy_release_candidate_pypi.sh) -* **Usage** +**Usage** ./release/src/main/scripts/deploy_release_candidate_pypi.sh \ --release "${RELEASE_VERSION}" \ @@ -626,62 +641,79 @@ Verify the script worked. --user "${GITHUB_USER}" \ --deploy -* **The script will:** +**The script will:** -1. Download python binary artifacts -2. Deploy release candidate to PyPI +Download previously build python binary artifacts Deploy release candidate +to PyPI with an `rc` suffix. __Attention:__ Verify that: -* The File names version include ``rc-#`` suffix -* [Download Files](https://pypi.org/project/apache-beam/#files) have: - * All wheels uploaded as artifacts - * Release source's zip published - * Signatures and hashes do not need to be uploaded +- [ ] The File names version include ``rc-#`` suffix +- [ ] [Download Files](https://pypi.org/project/apache-beam/#files) have: + - [ ] All wheels uploaded as artifacts + - [ ] Release source's zip published + - [ ] Signatures and hashes do not need to be uploaded -You can do a dry run by omitting the `--deploy` flag. Then it will only download the release candidate binaries. If it looks good, rerun it with `--deploy`. +You can do a dry run by omitting the `--deploy` flag. Then it will only +download the release candidate binaries. If it looks good, rerun it with +`--deploy`. -See the source of the script for more details or to run commands manually in case of a problem. +See the source of the script for more details or to run commands manually in +case of a problem. ### Propose pull requests for website updates -Beam publishes API reference manuals for each release on the website. -For Java and Python SDKs, that’s Javadoc and PyDoc, respectively. -The final step of building the candidate is to propose website pull requests that update these manuals. +Beam publishes API reference manuals for each release on the website. For Java +and Python SDKs, that’s Javadoc and PyDoc, respectively. The final step of +building the candidate is to propose website pull requests that update these +manuals. -Merge the pull requests only after finalizing the release. -To avoid invalid redirects for the 'current' version, merge these PRs in the order listed. -Once the PR is merged, the new contents will get picked up automatically and served to the Beam website, usually within an hour. -A committer can manually trigger the [beam_PostCommit_Website_Publish](https://ci-beam.apache.org/job/beam_PostCommit_Website_Publish/) task in Jenkins to avoid waiting. +Merge the pull requests only after finalizing the release. To avoid invalid +redirects for the 'current' version, merge these PRs in the order listed. Once +the PR is merged, the new contents will get picked up automatically and served +to the Beam website, usually within an hour. A committer can manually trigger +the +[beam_PostCommit_Website_Publish](https://ci-beam.apache.org/job/beam_PostCommit_Website_Publish/) +task in Jenkins to avoid waiting. **PR 1: apache/beam-site** -This pull request is against the `apache/beam-site` repo, on the `release-docs` branch ([example](https://github.com/apache/beam-site/pull/603)). -It is created by the `build_release_candidate` workflow (see above). +This pull request is against the `apache/beam-site` repo, on the `release-docs` +branch ([example](https://github.com/apache/beam-site/pull/603)). It is +created by the `build_release_candidate` workflow (see above). **PR 2: apache/beam** -This pull request is against the `apache/beam` repo, on the `master` branch ([example](https://github.com/apache/beam/pull/17378)). +This pull request is against the `apache/beam` repo, on the `master` branch +([example](https://github.com/apache/beam/pull/17378)). -* Update `CHANGES.md` to update release date and remove template. -* Update release version in `website/www/site/config.toml`. -* Add new release in `website/www/site/content/en/get-started/downloads.md`. - * Download links will not work until the release is finalized. -* Update links to prior releases to point to https://archive.apache.org (see +- Update `CHANGES.md` to update release date and remove template. +- Update release version in `website/www/site/config.toml`. +- Add new release in `website/www/site/content/en/get-started/downloads.md`. + - Download links will not work until the release is finalized. +- Update links to prior releases to point to https://archive.apache.org (see example PR). -* Create the Blog post: +- Create the Blog post: #### Blog post -Use the template below to write a blog post for the release. -See [beam-2.31.0.md](https://github.com/apache/beam/commit/a32a75ed0657c122c6625aee1ace27994e7df195#diff-1e2b83a4f61dce8014a1989869b6d31eb3f80cb0d6dade42fb8df5d9407b4748) as an example. -- Copy the changes for the current release from `CHANGES.md` to the blog post and edit as necessary. -- Be sure to add yourself to [authors.yml](https://github.com/apache/beam/blob/master/website/www/site/data/authors.yml) if necessary. +Use the template below to write a blog post for the release. See +[beam-2.31.0.md](https://github.com/apache/beam/commit/a32a75ed0657c122c6625aee1ace27994e7df195#diff-1e2b83a4f61dce8014a1989869b6d31eb3f80cb0d6dade42fb8df5d9407b4748) +as an example. + +- Copy the changes for the current release from `CHANGES.md` to the blog post + and edit as necessary. +- Be sure to add yourself to + [authors.yml](https://github.com/apache/beam/blob/master/website/www/site/data/authors.yml) + if necessary. -__Tip__: Use git log to find contributors to the releases. (e.g: `git fetch origin --tags; git log --pretty='%aN' ^v2.10.0 v2.11.0-RC1 | sort | uniq`). -Make sure to clean it up, as there may be duplicate or incorrect user names. +> **TIP** +> Use git log to find contributors to the releases. (e.g: `git fetch +> origin --tags; git log --pretty='%aN' ^v2.10.0 v2.11.0-RC1 | sort | uniq`). +> Make sure to clean it up, as there may be duplicate or incorrect user names. -__NOTE__: Make sure to include any breaking changes, even to `@Experimental` features, -all major features and bug fixes, and all known issues. +> **NOTE** +> Make sure to include any breaking changes, even to `@Experimental` +> features, all major features and bug fixes, and all known issues. **Template:** @@ -755,34 +787,46 @@ all major features and bug fixes, and all known issues. ${CONTRIBUTORS} -### Checklist to proceed to the next step +### Checklist to proceed to the next phase -- [ ] Maven artifacts deployed to the staging repository of [repository.apache.org](https://repository.apache.org/content/repositories/) -. Source distribution deployed to the dev repository of [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam/) -- [ ] Website pull request proposed to list the [release](/get-started/downloads/), publish the [Java API reference manual](https://beam.apache.org/releases/javadoc/), and publish the [Python API reference manual](https://beam.apache.org/releases/pydoc/). -- [ ] Docker images are published to [DockerHub](https://hub.docker.com/search?q=apache%2Fbeam&type=image) with tags: {RELEASE_VERSION}_rc{RC_NUM}. +- [ ] Maven artifacts deployed to the staging repository of + [repository.apache.org](https://repository.apache.org/content/repositories/) +- [ ] Source distribution deployed to the dev repository of + [dist.apache.org](https://dist.apache.org/repos/dist/dev/beam/) +- [ ] Website pull request proposed to list the + [release](/get-started/downloads/), publish the [Java API reference + manual](https://beam.apache.org/releases/javadoc/), and publish the [Python + API reference manual](https://beam.apache.org/releases/pydoc/). +- [ ] Docker images are published to + [DockerHub](https://hub.docker.com/search?q=apache%2Fbeam&type=image) with + tags: `{RELEASE_VERSION}_rc{RC_NUM}`. You can (optionally) also do additional verification by: -- [ ] Check that Python zip file contains the `README.md`, `NOTICE`, and `LICENSE` files. -- [ ] Check hashes (e.g. `md5sum -c *.md5` and `sha1sum -c *.sha1`. Note that signature/checksum files of Java artifacts may not contain filenames. Hence you might need to compare checksums/signatures manually or modify the files by appending the filenames.) -- [ ] Check signatures (e.g. `gpg --verify apache-beam-1.2.3-python.zip.asc apache-beam-1.2.3-python.zip`) + +- [ ] Check that Python zip file contains the `README.md`, `NOTICE`, and + `LICENSE` files. +- [ ] Check hashes (e.g. `md5sum -c *.md5` and `sha1sum -c *.sha1`. Note that + signature/checksum files of Java artifacts may not contain filenames. Hence + you might need to compare checksums/signatures manually or modify the files by + appending the filenames.) +- [ ] Check signatures (e.g. `gpg --verify apache-beam-1.2.3-python.zip.asc + apache-beam-1.2.3-python.zip`) - [ ] `grep` for legal headers in each file. -- [ ] Run all jenkins suites and include links to passing tests in the voting email. -- [ ] Pull docker images to make sure they are pullable. -``` -docker pull {image_name} -docker pull apache/beam_python3.7_sdk:2.39.0rc1 -``` +- [ ] Run all jenkins suites and include links to passing tests in the voting + email. +- [ ] Pull docker images to make sure they are pullable. (e.g. `docker pull apache/beam_python3.7_sdk:2.39.0rc1` ********** -## 4. Vote and validate release candidate +## Vote and validate the release candidate -Once you have built and individually reviewed the release candidate, please share it for the community-wide review. -Please review foundation-wide [voting guidelines](https://www.apache.org/foundation/voting.html) for more information. +Once you have built and individually reviewed the release candidate, please +share it for the community-wide review. Please review foundation-wide [voting +guidelines](https://www.apache.org/foundation/voting.html) for more +information. -Start the review-and-vote thread on the dev@ mailing list. -Here’s an email template; please adjust as you see fit. +Start the review-and-vote thread on the dev@ mailing list. Here’s an email +template; please adjust as you see fit. From: Release Manager To: dev@beam.apache.org @@ -832,28 +876,43 @@ Here’s an email template; please adjust as you see fit. [11] https://hub.docker.com/search?q=apache%2Fbeam&type=image [12] https://github.com/apache/beam/pull/... -If there are any issues found in the release candidate, reply on the vote thread to cancel the vote. -There’s no need to wait 72 hours. -Proceed to the `Fix issues` step below and address the problem. -However, some issues don’t require cancellation. -For example, if an issue is found in the website pull request, just correct it on the spot and the vote can continue as-is. +If there are any issues found in the release candidate, reply on the vote +thread to cancel the vote. There’s no need to wait 72 hours. Go back to +["Stabilize the Release Branch"](#stabilize-the-release-branch) and address the problem. However, some issues +don’t require cancellation. For example, if an issue is found in the website +pull request, just correct it on the spot and the vote can continue as-is. ### Run validation tests -The community is responsible for performing validation, but as release manager you are expected to contribute as well. -Before accepting an RC, as a community we try to exercise most (if not all) of the tests listed in this -[spreadsheet](https://s.apache.org/beam-release-validation), and those are good validations for you to try out as release manager. -The goal of these tests is to validate that we're able to run basic pipelines from a variety of environments (not just our CI environment). -Since there are many tests, we recommend you running some validations using an automation script. -In case of script failure, you can still run all of them manually. +The community is responsible for performing validation, but as release manager +you are expected to contribute as well. + +Before accepting an RC, as a community we try to exercise most (if not all) of +the tests listed in this +[spreadsheet](https://s.apache.org/beam-release-validation), and those are good +validations for you to try out as release manager. The goal of these tests is +to validate that we're able to run basic pipelines from a variety of +environments (not just our CI environment). + +Since there are many tests, we recommend you running some validations using an +automation script. In case of script failure, you can still run all of them +manually. You may need to have Python interpreters for all supported Python minor -versions to run all of the tests. See Python installation tips in [Developer Wiki](https://cwiki.apache.org/confluence/display/BEAM/Python+Tips#PythonTips-InstallingPythoninterpreters). +versions to run all of the tests. See Python installation tips in [Developer +Wiki](https://cwiki.apache.org/confluence/display/BEAM/Python+Tips#PythonTips-InstallingPythoninterpreters). + +> **Note** +> The community's validation means more than just running the tests +> that we have already run. It includes users trying out the RC on their own +> downstream tests. It also includes double checking that our human-language +> instructions actually still correspond to the automation that we have built. #### Run validations using run_rc_validation.sh -* **Script:** [run_rc_validation.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/run_rc_validation.sh) -* **Usage** +**Script:** [run_rc_validation.sh](https://github.com/apache/beam/blob/master/release/src/main/scripts/run_rc_validation.sh) + +**Usage** 1. First update required configurations listed in `RC_VALIDATE_CONFIGS` in [script.config](https://github.com/apache/beam/blob/master/release/src/main/scripts/script.config) 2. Then run @@ -864,7 +923,7 @@ versions to run all of the tests. See Python installation tips in [Developer Wik **Note:** running the validations requires the ability to do the following in your GCP account: start pipelines, write to BigQuery, and create a cluster of machines for running containers (for x-lang validation). -* **Tasks included** +**Tasks included** 1. Create a PR to trigger Python validation job, including * Python quickstart in batch and streaming mode with direct runner and Dataflow runner. * Python Mobile Games(UserScore, HourlyTeamScore) with direct runner and Dataflow runner. @@ -880,22 +939,25 @@ write to BigQuery, and create a cluster of machines for running containers (for * Start a new terminal to run Python multi-language Java kafka validation with Dataflow Runner. * Start a new terminal to run Python multi-language Java sql validation with Dataflow Runner. -* **Tasks you need to do manually** - 1. Check whether validations succeed by following console output instructions. - 1. Terminate streaming jobs and java injector. - 1. Run Java quickstart (wordcount) and mobile game examples with the staged artifacts. The easiest way to do this is by running the tests on Jenkins. -Other manual validation will follow, but this will at least validate that the staged artifacts can be used. +* **Tasks you need to do manually**. + +- [ ] Check whether validations succeed by following console output instructions. +- [ ] Terminate streaming jobs and java injector. +- [ ] Run Java quickstart (wordcount) and mobile game examples with the staged artifacts. The easiest way to do this is by running the tests on Jenkins. + +- Other manual validation will follow, but this will at least validate that the staged artifacts can be used. * Log in to Jenkins. * Go to https://ci-beam.apache.org/job/beam_PostRelease_NightlySnapshot/. * Click "Build with Parameters". * Set `snapshot_version` to `2.xx.0`, and set `snapshot_url` to point to the staged artifacts in Maven central (https://repository.apache.org/content/repositories/orgapachebeam-NNNN/). * Click "Build". - 1. Sign up [spreadsheet](https://s.apache.org/beam-release-validation). - 1. Vote in the release thread. +- [ ] Sign up [spreadsheet](https://s.apache.org/beam-release-validation). +- [ ] Vote in the release thread. #### Run validations manually -_Note_: -Prepourl and -Pver can be found in the RC vote email sent by Release Manager. +> **Note** +> `-Prepourl` and `-Pver` can be found in the RC vote email sent by Release Manager. * **Java Quickstart Validation** @@ -1103,25 +1165,17 @@ _Note_: -Prepourl and -Pver can be found in the RC vote email sent by Release Ma * bq head -n 10 ${USER}_test.game_stats_sessions -### Fix issues - -Any issues identified during the community review and vote should be fixed in this step. -Additionally, any GitHub issues created from the initial branch verification should be fixed. - -Code changes should be proposed as standard pull requests to the `master` branch and reviewed using the normal contributing process. -Then, relevant changes should be cherry-picked into the release branch proposed as pull requests against the release branch, again reviewed and merged using the normal contributing process. - -Once all issues have been resolved as in the `Verify release branch` step, you should go back and build a new release candidate with these changes. - ### Finalize the vote -Reply on the vote thread to close the voting once following conditions are met for the current release candidate. -* At least 72 hours has passed since the voting email. -* No release blocking issues have been identified. -* Voting thread has at least three approving PMC votes. +Reply on the vote thread to close the voting once following conditions are met +for the current release candidate. -Then, tally the votes in a separate email thread. -Here’s an email template; please adjust as you see fit. +- [ ] At least 72 hours has passed since the voting email. +- [ ] No release blocking issues have been identified. +- [ ] Voting thread has at least three approving PMC votes. + +Then, tally the votes in a separate email thread. Here’s an email template; +please adjust as you see fit. From: Release Manager To: dev@beam.apache.org @@ -1146,9 +1200,9 @@ Here’s an email template; please adjust as you see fit. - [ ] Community votes to release the proposed candidate, with at least three approving PMC votes. -********** +---- -## 5. Finalize the release +## Finalize the release Once the release candidate has been reviewed and approved by the community, the release should be finalized. This involves the final deployment of the release candidate to the release repositories, merging of the website changes, etc. @@ -1267,7 +1321,7 @@ Use [reporter.apache.org](https://reporter.apache.org/addrelease.html?beam) to s ********** -## 6. Promote the release +## Promote the release Once the release has been finalized, the last step of the process is to promote the release within the project and beyond. @@ -1298,7 +1352,9 @@ Also, update [the Wikipedia article on Apache Beam](https://en.wikipedia.org/wik ********** -## Post Release Tasks +## Post-Release Tasks + +At the end of the release, go to the GitHub milestones page and mark the recently released version as closed. ### Update Beam Playground From fc93a36d869ad491cdf065fdb2b96dee222b4a69 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 5 Oct 2023 10:46:04 -0700 Subject: [PATCH 20/85] Make parquetio schema aware. --- sdks/python/apache_beam/io/parquetio.py | 99 ++++++++++++++----- sdks/python/apache_beam/io/parquetio_test.py | 16 +++ sdks/python/apache_beam/pvalue.py | 3 + .../typehints/arrow_type_compatibility.py | 6 +- 4 files changed, 97 insertions(+), 27 deletions(-) diff --git a/sdks/python/apache_beam/io/parquetio.py b/sdks/python/apache_beam/io/parquetio.py index 734dfa0bfe8e..4696e5ae7927 100644 --- a/sdks/python/apache_beam/io/parquetio.py +++ b/sdks/python/apache_beam/io/parquetio.py @@ -31,27 +31,34 @@ # pytype: skip-file from functools import partial +from typing import Iterator from packaging import version from apache_beam.io import filebasedsink from apache_beam.io import filebasedsource from apache_beam.io.filesystem import CompressionTypes +from apache_beam.io.filesystems import FileSystems from apache_beam.io.iobase import RangeTracker from apache_beam.io.iobase import Read from apache_beam.io.iobase import Write +from apache_beam.portability.api import schema_pb2 from apache_beam.transforms import DoFn from apache_beam.transforms import ParDo from apache_beam.transforms import PTransform from apache_beam.transforms import window +from apache_beam.typehints import schemas try: import pyarrow as pa import pyarrow.parquet as pq + # pylint: disable=ungrouped-imports + from apache_beam.typehints import arrow_type_compatibility except ImportError: pa = None pq = None ARROW_MAJOR_VERSION = None + arrow_type_compatibility = None else: base_pa_version = version.parse(pa.__version__).base_version ARROW_MAJOR_VERSION, _, _ = map(int, base_pa_version.split('.')) @@ -146,6 +153,24 @@ def _flush_buffer(self): self._record_batches_byte_size = self._record_batches_byte_size + size +class _ArrowTableToBeamRows(DoFn): + def __init__(self, beam_type): + self._beam_type = beam_type + + @DoFn.yields_batches + def process(self, element) -> Iterator[pa.Table]: + yield element + + def infer_output_type(self, input_type): + return self._beam_type + + +class _BeamRowsToArrowTable(DoFn): + @DoFn.yields_elements + def process_batch(self, element: pa.Table) -> Iterator[pa.Table]: + yield element + + class ReadFromParquetBatched(PTransform): """A :class:`~apache_beam.transforms.ptransform.PTransform` for reading Parquet files as a `PCollection` of `pyarrow.Table`. This `PTransform` is @@ -191,7 +216,7 @@ def __init__( """ super().__init__() - self._source = _create_parquet_source( + self._source = _ParquetSource( file_pattern, min_bundle_size, validate=validate, @@ -210,7 +235,12 @@ class ReadFromParquet(PTransform): Parquet files as a `PCollection` of dictionaries. This `PTransform` is currently experimental. No backward-compatibility guarantees.""" def __init__( - self, file_pattern=None, min_bundle_size=0, validate=True, columns=None): + self, + file_pattern=None, + min_bundle_size=0, + validate=True, + columns=None, + as_rows=False): """Initializes :class:`ReadFromParquet`. Uses source ``_ParquetSource`` to read a set of Parquet files defined by @@ -255,17 +285,38 @@ def __init__( columns (List[str]): list of columns that will be read from files. A column name may be a prefix of a nested field, e.g. 'a' will select 'a.b', 'a.c', and 'a.d.e' + as_rows (bool): whether to output a schema'd PCollection of Beam rows + rather than Python dictionaries. """ super().__init__() - self._source = _create_parquet_source( + self._source = _ParquetSource( file_pattern, min_bundle_size, validate=validate, columns=columns, ) + if as_rows: + if columns is None: + filter_schema = lambda schema: schema + else: + top_level_columns = set(c.split('.')[0] for c in columns) + filter_schema = lambda schema: schema_pb2.Schema( + fields=[f for f in schema.fields if f.name in top_level_columns]) + path = FileSystems.match([file_pattern], [1])[0].metadata_list[0].path + with FileSystems.open(path) as fin: + self._schema = filter_schema( + arrow_type_compatibility.beam_schema_from_arrow_schema( + pq.read_schema(fin))) + else: + self._schema = None def expand(self, pvalue): - return pvalue | Read(self._source) | ParDo(_ArrowTableToRowDictionaries()) + arrow_batches = pvalue | Read(self._source) + if self._schema is None: + return arrow_batches | ParDo(_ArrowTableToRowDictionaries()) + else: + return arrow_batches | ParDo( + _ArrowTableToBeamRows(schemas.named_tuple_from_schema(self._schema))) def display_data(self): return {'source_dd': self._source} @@ -305,9 +356,7 @@ def __init__( """ super().__init__() source_from_file = partial( - _create_parquet_source, - min_bundle_size=min_bundle_size, - columns=columns) + _ParquetSource, min_bundle_size=min_bundle_size, columns=columns) self._read_all_files = filebasedsource.ReadAllFiles( True, CompressionTypes.UNCOMPRESSED, @@ -333,17 +382,6 @@ def expand(self, pvalue): _ArrowTableToRowDictionaries(), with_filename=self._with_filename) -def _create_parquet_source( - file_pattern=None, min_bundle_size=0, validate=False, columns=None): - return \ - _ParquetSource( - file_pattern=file_pattern, - min_bundle_size=min_bundle_size, - validate=validate, - columns=columns, - ) - - class _ParquetUtils(object): @staticmethod def find_first_row_group_index(pf, start_offset): @@ -370,7 +408,8 @@ def get_number_of_row_groups(pf): class _ParquetSource(filebasedsource.FileBasedSource): """A source for reading Parquet files. """ - def __init__(self, file_pattern, min_bundle_size, validate, columns): + def __init__( + self, file_pattern, min_bundle_size=0, validate=False, columns=None): super().__init__( file_pattern=file_pattern, min_bundle_size=min_bundle_size, @@ -421,6 +460,9 @@ def split_points_unclaimed(stop_position): yield table +_create_parquet_source = _ParquetSource + + class WriteToParquet(PTransform): """A ``PTransform`` for writing parquet files. @@ -430,7 +472,7 @@ class WriteToParquet(PTransform): def __init__( self, file_path_prefix, - schema, + schema=None, row_group_buffer_size=64 * 1024 * 1024, record_batch_size=1000, codec='none', @@ -534,10 +576,19 @@ def __init__( ) def expand(self, pcoll): - return pcoll | ParDo( - _RowDictionariesToArrowTable( - self._schema, self._row_group_buffer_size, - self._record_batch_size)) | Write(self._sink) + if self._schema is None: + try: + beam_schema = schemas.schema_from_element_type(pcoll.element_type) + except TypeError as exn: + raise ValueError( + "A schema is required to write non-schema'd data.") from exn + self._sink._schema = ( + arrow_type_compatibility.arrow_schema_from_beam_schema(beam_schema)) + convert_fn = _BeamRowsToArrowTable() + else: + convert_fn = _RowDictionariesToArrowTable( + self._schema, self._row_group_buffer_size, self._record_batch_size) + return pcoll | ParDo(convert_fn) | Write(self._sink) def display_data(self): return { diff --git a/sdks/python/apache_beam/io/parquetio_test.py b/sdks/python/apache_beam/io/parquetio_test.py index df018a3a776f..1cd5f1208cc2 100644 --- a/sdks/python/apache_beam/io/parquetio_test.py +++ b/sdks/python/apache_beam/io/parquetio_test.py @@ -30,6 +30,7 @@ from parameterized import param from parameterized import parameterized +import apache_beam as beam from apache_beam import Create from apache_beam import Map from apache_beam.io import filebasedsource @@ -400,6 +401,21 @@ def test_sink_transform_compliant_nested_type(self): assert_that( readback, equal_to([json.dumps(r) for r in self.RECORDS_NESTED])) + def test_schema_read_write(self): + with TemporaryDirectory() as tmp_dirname: + path = os.path.join(tmp_dirname, 'tmp_filename') + rows = [beam.Row(a=1, b='x'), beam.Row(a=2, b='y')] + stable_repr = lambda row: json.dumps(row._asdict()) + with TestPipeline() as p: + _ = p | Create(rows) | WriteToParquet(path) | beam.Map(print) + with TestPipeline() as p: + # json used for stable sortability + readback = ( + p + | ReadFromParquet(path + '*', as_rows=True) + | Map(stable_repr)) + assert_that(readback, equal_to([stable_repr(r) for r in rows])) + def test_batched_read(self): with TemporaryDirectory() as tmp_dirname: path = os.path.join(tmp_dirname + "tmp_filename") diff --git a/sdks/python/apache_beam/pvalue.py b/sdks/python/apache_beam/pvalue.py index 2e86c9eb51c7..90882651d0b2 100644 --- a/sdks/python/apache_beam/pvalue.py +++ b/sdks/python/apache_beam/pvalue.py @@ -673,6 +673,9 @@ def __init__(self, **kwargs): def as_dict(self): return dict(self.__dict__) + # For compatibility with named tuples. + _asdict = as_dict + def __iter__(self): for _, value in self.__dict__.items(): yield value diff --git a/sdks/python/apache_beam/typehints/arrow_type_compatibility.py b/sdks/python/apache_beam/typehints/arrow_type_compatibility.py index c8e425f0e96a..34a37a886bab 100644 --- a/sdks/python/apache_beam/typehints/arrow_type_compatibility.py +++ b/sdks/python/apache_beam/typehints/arrow_type_compatibility.py @@ -311,9 +311,9 @@ def from_typehints(element_type, element_type = RowTypeConstraint.from_user_type(element_type) if element_type is None: raise TypeError( - "Element type must be compatible with Beam Schemas (" - "https://beam.apache.org/documentation/programming-guide/#schemas) " - "for batch type pa.Table.") + f"Element type {element_type} must be compatible with Beam Schemas " + "(https://beam.apache.org/documentation/programming-guide/#schemas)" + " for batch type pa.Table.") return PyarrowBatchConverter(element_type) From e94aeb71fffa9e9f94fe9a2242ff12fa7e7ab560 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Thu, 5 Oct 2023 10:56:25 -0700 Subject: [PATCH 21/85] Add parquet yaml providers. --- sdks/python/apache_beam/yaml/standard_io.yaml | 11 +++++++++++ sdks/python/apache_beam/yaml/yaml_provider.py | 6 +++++- 2 files changed, 16 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 9ad4f53ba1f6..af738a589cda 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -67,6 +67,8 @@ 'WriteToCsv': 'WriteToCsv' 'ReadFromJson': 'ReadFromJson' 'WriteToJson': 'WriteToJson' + 'ReadFromParquet': 'ReadFromParquet' + 'WriteToParquet': 'WriteToParquet' config: mappings: 'ReadFromCsv': @@ -77,6 +79,13 @@ path: 'path' 'WriteToJson': path: 'path' + 'ReadFromParquet': + path: 'file_pattern' + 'WriteToParquet': + path: 'file_path_prefix' + defaults: + 'ReadFromParquet': + as_rows: True underlying_provider: type: python transforms: @@ -84,3 +93,5 @@ 'WriteToCsv': 'apache_beam.io.WriteToCsv' 'ReadFromJson': 'apache_beam.io.ReadFromJson' 'WriteToJson': 'apache_beam.io.WriteToJson' + 'ReadFromParquet': 'apache_beam.io.ReadFromParquet' + 'WriteToParquet': 'apache_beam.io.WriteToParquet' diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 3a88f6074b37..20d4b096916d 100644 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -684,7 +684,7 @@ def __exit__(self, *args): @ExternalProvider.register_provider_type('renaming') class RenamingProvider(Provider): - def __init__(self, transforms, mappings, underlying_provider): + def __init__(self, transforms, mappings, underlying_provider, defaults=None): if isinstance(underlying_provider, dict): underlying_provider = ExternalProvider.provider_from_spec( underlying_provider) @@ -694,6 +694,7 @@ def __init__(self, transforms, mappings, underlying_provider): if transform not in mappings: raise ValueError(f'Missing transform {transform} in mappings.') self._mappings = mappings + self._defaults = defaults or {} def available(self) -> bool: return self._underlying_provider.available() @@ -731,6 +732,9 @@ def create_transform( mappings.get(key, key): value for key, value in args.items() } + for key, value in self._defaults.get(typ, {}).items(): + if key not in remapped_args: + remapped_args[key] = value return self._underlying_provider.create_transform( self._transforms[typ], remapped_args, yaml_create_transform) From 9602af91dcd05cc28db655589abfc42af226dc90 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 6 Oct 2023 12:00:32 -0700 Subject: [PATCH 22/85] Update error message to account for more detail. --- .../apache_beam/typehints/arrow_type_compatibility_test.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/typehints/arrow_type_compatibility_test.py b/sdks/python/apache_beam/typehints/arrow_type_compatibility_test.py index e708b151d905..1e9ab3f27bd9 100644 --- a/sdks/python/apache_beam/typehints/arrow_type_compatibility_test.py +++ b/sdks/python/apache_beam/typehints/arrow_type_compatibility_test.py @@ -206,7 +206,7 @@ class ArrowBatchConverterErrorsTest(unittest.TestCase): ( pa.Table, Any, - r'Element type must be compatible with Beam Schemas', + r'Element type .* must be compatible with Beam Schemas', ), ]) def test_construction_errors( From 84b9336cffce7af17aa21437ae0b878984f918bc Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Fri, 6 Oct 2023 15:24:43 -0400 Subject: [PATCH 23/85] Merge pull request #28665: Add GitHub Action workflows to mass_commit --- .github/workflows/beam_PostCommit_Python.yml | 2 +- .../src/main/scripts/github_actions_jobs.txt | 33 ++++++++++++++----- release/src/main/scripts/mass_comment.py | 3 +- 3 files changed, 28 insertions(+), 10 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 6f4bc5e2ef0b..3fb650606b40 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -64,7 +64,7 @@ jobs: if: | github.event_name == 'workflow_dispatch' || github.event_name == 'schedule' || - github.event.comment.body == 'Run Python PostCommit' + startswith(github.event.comment.body, 'Run Python PostCommit 3.') steps: - uses: actions/checkout@v4 - name: Setup repository diff --git a/release/src/main/scripts/github_actions_jobs.txt b/release/src/main/scripts/github_actions_jobs.txt index 3ad16e555ef6..6f200454afe0 100644 --- a/release/src/main/scripts/github_actions_jobs.txt +++ b/release/src/main/scripts/github_actions_jobs.txt @@ -1,8 +1,25 @@ -Run Go PostCommit Dataflow ARM -Run Java_Examples_Dataflow_ARM PostCommit 8 -Run Java_Examples_Dataflow_ARM PostCommit 11 -Run Java_Examples_Dataflow_ARM PostCommit 17 -Run Python ValidatesContainer Dataflow ARM 3.8 -Run Python ValidatesContainer Dataflow ARM 3.9 -Run Python ValidatesContainer Dataflow ARM 3.10 -Run Python ValidatesContainer Dataflow ARM 3.11 \ No newline at end of file +Run PythonDocs PreCommit,beam_PreCommit_PythonDocs +Run PythonLint PreCommit,beam_PreCommit_PythonLint +Run RAT PreCommit,beam_PreCommit_RAT +Run Spotless PreCommit,beam_PreCommit_Spotless +Run Website PreCommit,beam_PreCommit_Website +Run Website_Stage_GCS PreCommit,beam_PreCommit_Website_Stage_GCS +Run Whitespace PreCommit,beam_PreCommit_Whitespace +Run PythonFormatter PreCommit,beam_PreCommit_PythonFormatter +Run Kotlin_Examples PreCommit,beam_PreCommit_Kotlin_Examples +Run Go PreCommit,beam_PreCommit_Go +Run GoPortable PreCommit,beam_PreCommit_GoPortable +Run GoPrism PreCommit,beam_PreCommit_GoPrism +Run Typescript PreCommit,beam_PreCommit_Typescript +Run CommunityMetrics PreCommit,beam_PreCommit_CommunityMetrics +Run Java_Flink_Versions PreCommit,beam_PreCommit_Java_Flink_Versions +Run It_Framework PreCommit,beam_PreCommit_ItFramework +Run Java examples on Dataflow,beam_PostCommit_Java_Examples_Dataflow +Run Go PostCommit Dataflow ARM,beam_PostCommit_Go_Dataflow_ARM +Run Java_Examples_Dataflow_ARM PostCommit 8,beam_PostCommit_Java_Examples_Dataflow_ARM +Run Java_Examples_Dataflow_ARM PostCommit 11,beam_PostCommit_Java_Examples_Dataflow_ARM +Run Java_Examples_Dataflow_ARM PostCommit 17,beam_PostCommit_Java_Examples_Dataflow_ARM +Run Python ValidatesContainer Dataflow ARM 3.8,beam_Python_ValidatesContainer_Dataflow_ARM +Run Python ValidatesContainer Dataflow ARM 3.9,beam_Python_ValidatesContainer_Dataflow_ARM +Run Python ValidatesContainer Dataflow ARM 3.10,beam_Python_ValidatesContainer_Dataflow_ARM +Run Python ValidatesContainer Dataflow ARM 3.11,beam_Python_ValidatesContainer_Dataflow_ARM diff --git a/release/src/main/scripts/mass_comment.py b/release/src/main/scripts/mass_comment.py index 3fa60fcb3e18..6d020f2af8e3 100644 --- a/release/src/main/scripts/mass_comment.py +++ b/release/src/main/scripts/mass_comment.py @@ -135,7 +135,8 @@ def getGithubActionsTriggerCommands(dirname): comments = [line.strip() for line in file if len(line.strip()) > 0] for i in range(len(comments)): - gha_trigger_commands.append(comments[i]) + parts = comments[i].split(',') + gha_trigger_commands.append((parts[0], parts[1])) return gha_trigger_commands From d26a78297f279135a35c8ab003d120ff482b1818 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Fri, 6 Oct 2023 12:35:52 -0700 Subject: [PATCH 24/85] [YAML] Add JSON parsing to PubSub IO. (#28754) --- .../apache_beam/transforms/ptransform.py | 16 ++ sdks/python/apache_beam/yaml/json_utils.py | 183 ++++++++++++++++++ sdks/python/apache_beam/yaml/yaml_io.py | 16 +- sdks/python/apache_beam/yaml/yaml_io_test.py | 123 ++++++++++++ 4 files changed, 336 insertions(+), 2 deletions(-) create mode 100644 sdks/python/apache_beam/yaml/json_utils.py diff --git a/sdks/python/apache_beam/transforms/ptransform.py b/sdks/python/apache_beam/transforms/ptransform.py index 28614c6561c7..fcff86d4c50c 100644 --- a/sdks/python/apache_beam/transforms/ptransform.py +++ b/sdks/python/apache_beam/transforms/ptransform.py @@ -1101,6 +1101,22 @@ def __ror__(self, pvalueish, _unused=None): def expand(self, pvalue): raise RuntimeError("Should never be expanded directly.") + def __getattr__(self, attr): + transform_attr = getattr(self.transform, attr) + if callable(transform_attr): + + @wraps(transform_attr) + def wrapper(*args, **kwargs): + result = transform_attr(*args, **kwargs) + if isinstance(result, PTransform): + return _NamedPTransform(result, self.label) + else: + return result + + return wrapper + else: + return transform_attr + # Defined here to avoid circular import issues for Beam library transforms. def annotate_yaml(constructor): diff --git a/sdks/python/apache_beam/yaml/json_utils.py b/sdks/python/apache_beam/yaml/json_utils.py new file mode 100644 index 000000000000..e2cb03dc96a0 --- /dev/null +++ b/sdks/python/apache_beam/yaml/json_utils.py @@ -0,0 +1,183 @@ +# +# 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. +# + +"""Utilities for converting between JSON and Beam Schema'd data. + +For internal use, no backward compatibility guarantees. +""" + +import json +from typing import Any +from typing import Callable +from typing import Dict + +import apache_beam as beam +from apache_beam.portability.api import schema_pb2 +from apache_beam.typehints import schemas + +JSON_ATOMIC_TYPES_TO_BEAM = { + 'boolean': schema_pb2.BOOLEAN, + 'integer': schema_pb2.INT64, + 'number': schema_pb2.DOUBLE, + 'string': schema_pb2.STRING, +} + + +def json_schema_to_beam_schema( + json_schema: Dict[str, Any]) -> schema_pb2.Schema: + """Returns a Beam schema equivalent for the given Json schema.""" + def maybe_nullable(beam_type, nullable): + if nullable: + beam_type.nullable = True + return beam_type + + json_type = json_schema.get('type', None) + if json_type != 'object': + raise ValueError('Expected object type, got {json_type}.') + if 'properties' not in json_schema: + # Technically this is a valid (vacuous) schema, but as it's not generally + # meaningful, throw an informative error instead. + # (We could add a flag to allow this degenerate case.) + raise ValueError('Missing properties for {json_schema}.') + required = set(json_schema.get('required', [])) + return schema_pb2.Schema( + fields=[ + schemas.schema_field( + name, + maybe_nullable(json_type_to_beam_type(t), name not in required)) + for (name, t) in json_schema['properties'].items() + ]) + + +def json_type_to_beam_type(json_type: Dict[str, Any]) -> schema_pb2.FieldType: + """Returns a Beam schema type for the given Json (schema) type.""" + if not isinstance(json_type, dict) or 'type' not in json_type: + raise ValueError(f'Malformed type {json_type}.') + type_name = json_type['type'] + if type_name in JSON_ATOMIC_TYPES_TO_BEAM: + return schema_pb2.FieldType( + atomic_type=JSON_ATOMIC_TYPES_TO_BEAM[type_name]) + elif type_name == 'array': + return schema_pb2.FieldType( + array_type=schema_pb2.ArrayType( + element_type=json_type_to_beam_type(json_type['items']))) + elif type_name == 'object': + if 'properties' in json_type: + return schema_pb2.FieldType( + row_type=schema_pb2.RowType( + schema=json_schema_to_beam_schema(json_type))) + elif 'additionalProperties' in json_type: + return schema_pb2.FieldType( + map_type=schema_pb2.MapType( + key_type=schema_pb2.FieldType(atomic_type=schema_pb2.STRING), + value_type=json_type_to_beam_type( + json_type['additionalProperties']))) + else: + raise ValueError( + f'Object type must have either properties or additionalProperties, ' + f'got {json_type}.') + else: + raise ValueError(f'Unable to convert {json_type} to a Beam schema.') + + +def json_to_row(beam_type: schema_pb2.FieldType) -> Callable[[Any], Any]: + """Returns a callable converting Json objects to Beam rows of the given type. + + The input to the returned callable is expected to conform to the Json schema + corresponding to this Beam type. + """ + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + return lambda value: value + elif type_info == "array_type": + element_converter = json_to_row(beam_type.array_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "iterable_type": + element_converter = json_to_row(beam_type.iterable_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "map_type": + if beam_type.map_type.key_type.atomic_type != schema_pb2.STRING: + raise TypeError( + f'Only strings allowd as map keys when converting from JSON, ' + f'found {beam_type}') + value_converter = json_to_row(beam_type.map_type.value_type) + return lambda value: {k: value_converter(v) for (k, v) in value.items()} + elif type_info == "row_type": + converters = { + field.name: json_to_row(field.type) + for field in beam_type.row_type.schema.fields + } + return lambda value: beam.Row( + ** + {name: convert(value[name]) + for (name, convert) in converters.items()}) + elif type_info == "logical_type": + return lambda value: value + else: + raise ValueError(f"Unrecognized type_info: {type_info!r}") + + +def json_parser(beam_schema: schema_pb2.Schema) -> Callable[[bytes], beam.Row]: + """Returns a callable converting Json strings to Beam rows of the given type. + + The input to the returned callable is expected to conform to the Json schema + corresponding to this Beam type. + """ + to_row = json_to_row( + schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=beam_schema))) + return lambda s: to_row(json.loads(s)) + + +def row_to_json(beam_type: schema_pb2.FieldType) -> Callable[[Any], Any]: + """Returns a callable converting rows of the given type to Json objects.""" + type_info = beam_type.WhichOneof("type_info") + if type_info == "atomic_type": + return lambda value: value + elif type_info == "array_type": + element_converter = row_to_json(beam_type.array_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "iterable_type": + element_converter = row_to_json(beam_type.iterable_type.element_type) + return lambda value: [element_converter(e) for e in value] + elif type_info == "map_type": + if beam_type.map_type.key_type.atomic_type != schema_pb2.STRING: + raise TypeError( + f'Only strings allowd as map keys when converting to JSON, ' + f'found {beam_type}') + value_converter = row_to_json(beam_type.map_type.value_type) + return lambda value: {k: value_converter(v) for (k, v) in value.items()} + elif type_info == "row_type": + converters = { + field.name: row_to_json(field.type) + for field in beam_type.row_type.schema.fields + } + return lambda row: { + name: convert(getattr(row, name)) + for (name, convert) in converters.items() + } + elif type_info == "logical_type": + return lambda value: value + else: + raise ValueError(f"Unrecognized type_info: {type_info!r}") + + +def json_formater( + beam_schema: schema_pb2.Schema) -> Callable[[beam.Row], bytes]: + """Returns a callable converting rows of the given schema to Json strings.""" + convert = row_to_json( + schema_pb2.FieldType(row_type=schema_pb2.RowType(schema=beam_schema))) + return lambda row: json.dumps(convert(row), sort_keys=True).encode('utf-8') diff --git a/sdks/python/apache_beam/yaml/yaml_io.py b/sdks/python/apache_beam/yaml/yaml_io.py index 4a1d12490057..3321644ded57 100644 --- a/sdks/python/apache_beam/yaml/yaml_io.py +++ b/sdks/python/apache_beam/yaml/yaml_io.py @@ -25,10 +25,12 @@ import os from typing import Any +from typing import Callable from typing import Iterable from typing import List from typing import Mapping from typing import Optional +from typing import Tuple import yaml @@ -39,6 +41,7 @@ from apache_beam.io.gcp.bigquery import BigQueryDisposition from apache_beam.portability.api import schema_pb2 from apache_beam.typehints import schemas +from apache_beam.yaml import json_utils from apache_beam.yaml import yaml_mapping from apache_beam.yaml import yaml_provider @@ -131,18 +134,25 @@ def raise_exception(failed_row_with_error): return WriteToBigQueryHandlingErrors() -def _create_parser(format, schema): +def _create_parser( + format, + schema: Any) -> Tuple[schema_pb2.Schema, Callable[[bytes], beam.Row]]: if format == 'raw': if schema: raise ValueError('raw format does not take a schema') return ( schema_pb2.Schema(fields=[schemas.schema_field('payload', bytes)]), lambda payload: beam.Row(payload=payload)) + elif format == 'json': + beam_schema = json_utils.json_schema_to_beam_schema(schema) + return beam_schema, json_utils.json_parser(beam_schema) else: raise ValueError(f'Unknown format: {format}') -def _create_formatter(format, schema, beam_schema): +def _create_formatter( + format, schema: Any, + beam_schema: schema_pb2.Schema) -> Callable[[beam.Row], bytes]: if format == 'raw': if schema: raise ValueError('raw format does not take a schema') @@ -150,6 +160,8 @@ def _create_formatter(format, schema, beam_schema): if len(field_names) != 1: raise ValueError(f'Expecting exactly one field, found {field_names}') return lambda row: getattr(row, field_names[0]) + elif format == 'json': + return json_utils.json_formater(beam_schema) else: raise ValueError(f'Unknown format: {format}') diff --git a/sdks/python/apache_beam/yaml/yaml_io_test.py b/sdks/python/apache_beam/yaml/yaml_io_test.py index ab6298661c15..72675da278b0 100644 --- a/sdks/python/apache_beam/yaml/yaml_io_test.py +++ b/sdks/python/apache_beam/yaml/yaml_io_test.py @@ -167,6 +167,101 @@ def test_read_with_id_attribute(self): result, equal_to([beam.Row(payload=b'msg1'), beam.Row(payload=b'msg2')])) + def test_read_json(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch('apache_beam.io.ReadFromPubSub', + FakeReadFromPubSub( + topic='my_topic', + messages=[PubsubMessage( + b'{"generator": {"x": 0, "y": 0}, "rank": 1}', + {'weierstrass': 'y^2+y=x^3-x', 'label': '37a'}) + ])): + result = p | YamlTransform( + ''' + type: ReadFromPubSub + config: + topic: my_topic + format: json + schema: + type: object + properties: + generator: + type: object + properties: + x: {type: integer} + y: {type: integer} + rank: {type: integer} + attributes: [label] + attributes_map: other + ''') + assert_that( + result, + equal_to([ + beam.Row( + generator=beam.Row(x=0, y=0), + rank=1, + label='37a', + other={ + 'label': '37a', 'weierstrass': 'y^2+y=x^3-x' + }) + ])) + + def test_read_json_with_error_handling(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch( + 'apache_beam.io.ReadFromPubSub', + FakeReadFromPubSub(topic='my_topic', + messages=[PubsubMessage('{"some_int": 123}', + attributes={}), + PubsubMessage('unparsable', + attributes={})])): + result = p | YamlTransform( + ''' + type: ReadFromPubSub + config: + topic: my_topic + format: json + schema: + type: object + properties: + some_int: {type: integer} + error_handling: + output: errors + ''') + assert_that( + result['good'], + equal_to([beam.Row(some_int=123)]), + label='CheckGood') + assert_that( + result['errors'] | beam.Map(lambda error: error.element), + equal_to(['unparsable']), + label='CheckErrors') + + def test_read_json_without_error_handling(self): + with self.assertRaises(Exception): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch( + 'apache_beam.io.ReadFromPubSub', + FakeReadFromPubSub(topic='my_topic', + messages=[PubsubMessage('{"some_int": 123}', + attributes={}), + PubsubMessage('unparsable', + attributes={})])): + _ = p | YamlTransform( + ''' + type: ReadFromPubSub + config: + topic: my_topic + format: json + schema: + type: object + properties: + some_int: {type: integer} + ''') + def test_simple_write(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: @@ -251,6 +346,34 @@ def test_write_with_id_attribute(self): id_attribute: some_attr ''')) + def test_write_json(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch('apache_beam.io.WriteToPubSub', + FakeWriteToPubSub( + topic='my_topic', + messages=[PubsubMessage( + b'{"generator": {"x": 0, "y": 0}, "rank": 1}', + {'weierstrass': 'y^2+y=x^3-x', 'label': '37a'}) + ])): + _ = ( + p | beam.Create([ + beam.Row( + label='37a', + generator=beam.Row(x=0, y=0), + rank=1, + other={'weierstrass': 'y^2+y=x^3-x'}) + ]) | YamlTransform( + ''' + type: WriteToPubSub + input: input + config: + topic: my_topic + format: json + attributes: [label] + attributes_map: other + ''')) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) From e665c1c73bdf35ddbe4b2fa79a7229549a7ba279 Mon Sep 17 00:00:00 2001 From: Svetak Sundhar Date: Fri, 6 Oct 2023 20:42:33 +0000 Subject: [PATCH 25/85] [Fix] Broken Healthcare Notebook Link (#28876) * Create HealthcareUtils file with shared resources * revert * Add correct notebook link. --- examples/notebooks/healthcare/beam_nlp.ipynb | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/notebooks/healthcare/beam_nlp.ipynb b/examples/notebooks/healthcare/beam_nlp.ipynb index 4ba4a5e0a739..c2061bc4d75f 100644 --- a/examples/notebooks/healthcare/beam_nlp.ipynb +++ b/examples/notebooks/healthcare/beam_nlp.ipynb @@ -22,7 +22,7 @@ "colab_type": "text" }, "source": [ - "\"Open" + "\"Open" ] }, { From ee7a2aea975e94fabff9a44508e4f4419ef11d2b Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Fri, 6 Oct 2023 17:10:36 -0400 Subject: [PATCH 26/85] Fix perf alert tool call (#28872) --- sdks/python/apache_beam/testing/analyzers/perf_analysis.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py index c86ecb2c4e20..0074625d3b47 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py @@ -160,6 +160,7 @@ def run_change_point_analysis( def run( + *, big_query_metrics_fetcher: MetricsFetcher = BigQueryMetricsFetcher(), config_file_path: Optional[str] = None, ) -> None: @@ -209,4 +210,4 @@ def run( if unknown_args: logging.warning('Discarding unknown arguments : %s ' % unknown_args) - run(known_args.config_file_path) + run(config_file_path=known_args.config_file_path) From fba098b69dbe18c84c007018f727f59aa077ac0b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Fri, 6 Oct 2023 17:40:05 -0400 Subject: [PATCH 27/85] Bump postcss (#28866) Bumps [postcss](https://github.com/postcss/postcss) from 8.3.9 to 8.4.31. - [Release notes](https://github.com/postcss/postcss/releases) - [Changelog](https://github.com/postcss/postcss/blob/main/CHANGELOG.md) - [Commits](https://github.com/postcss/postcss/compare/8.3.9...8.4.31) --- updated-dependencies: - dependency-name: postcss dependency-type: indirect ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- .../yarn.lock | 33 +++++++++++-------- 1 file changed, 19 insertions(+), 14 deletions(-) diff --git a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/yarn.lock b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/yarn.lock index fd09b6c9eb00..f72e23ebd403 100644 --- a/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/yarn.lock +++ b/sdks/python/apache_beam/runners/interactive/extensions/apache-beam-jupyterlab-sidepanel/yarn.lock @@ -6767,10 +6767,10 @@ mv@2.1.1: ncp "~2.0.0" rimraf "~2.4.0" -nanoid@^3.1.28: - version "3.3.2" - resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.3.2.tgz#c89622fafb4381cd221421c69ec58547a1eec557" - integrity sha512-CuHBogktKwpm5g2sRgv83jEy2ijFzBwMoYA60orPDR7ynsLijJDqgsi4RDGj3OJpy3Ieb+LYwiRmIOGyytgITA== +nanoid@^3.3.6: + version "3.3.6" + resolved "https://registry.yarnpkg.com/nanoid/-/nanoid-3.3.6.tgz#443380c856d6e9f9824267d960b4236ad583ea4c" + integrity sha512-BGcqMMJuToF7i1rt+2PWSNVnWIkGCU78jBG3RxO/bZlnZPK2Cmi2QaffxGO/2RvWi9sL+FAiRiXMgsyxQ1DIDA== nanomatch@^1.2.9: version "1.2.13" @@ -7279,6 +7279,11 @@ picocolors@^0.2.1: resolved "https://registry.yarnpkg.com/picocolors/-/picocolors-0.2.1.tgz#570670f793646851d1ba135996962abad587859f" integrity sha512-cMlDqaLEqfSaW8Z7N5Jw+lyIW869EzT73/F5lhtY9cLGoVxSXznfgfXMO0Z5K0o0Q2TkTXq+0KFsdnSe3jDViA== +picocolors@^1.0.0: + version "1.0.0" + resolved "https://registry.yarnpkg.com/picocolors/-/picocolors-1.0.0.tgz#cb5bdc74ff3f51892236eaf79d68bc44564ab81c" + integrity sha512-1fygroTLlHu66zi26VoTDv8yRgm0Fccecssto+MhsZ0D/DGW2sm8E8AjW7NU5VVTRt5GxbeZ5qBuJr+HyLYkjQ== + picomatch@^2.0.4, picomatch@^2.0.5: version "2.2.2" resolved "https://registry.yarnpkg.com/picomatch/-/picomatch-2.2.2.tgz#21f333e9b6b8eaff02468f5146ea406d345f4dad" @@ -7392,13 +7397,13 @@ postcss-value-parser@^4.1.0: integrity sha512-97DXOFbQJhk71ne5/Mt6cOu6yxsSfM0QGQyl0L25Gca4yGWEGJaig7l7gbCX623VqTBNGLRLaVUCnNkcedlRSQ== postcss@^8.0.2, postcss@^8.2.15: - version "8.3.9" - resolved "https://registry.yarnpkg.com/postcss/-/postcss-8.3.9.tgz#98754caa06c4ee9eb59cc48bd073bb6bd3437c31" - integrity sha512-f/ZFyAKh9Dnqytx5X62jgjhhzttjZS7hMsohcI7HEI5tjELX/HxCy3EFhsRxyzGvrzFF+82XPvCS8T9TFleVJw== + version "8.4.31" + resolved "https://registry.yarnpkg.com/postcss/-/postcss-8.4.31.tgz#92b451050a9f914da6755af352bdc0192508656d" + integrity sha512-PS08Iboia9mts/2ygV3eLpY5ghnUcfLV/EXTOW1E2qYxJKGGBUtNjN76FYHnMs36RmARn41bC0AZmn+rR0OVpQ== dependencies: - nanoid "^3.1.28" - picocolors "^0.2.1" - source-map-js "^0.6.2" + nanoid "^3.3.6" + picocolors "^1.0.0" + source-map-js "^1.0.2" prelude-ls@^1.2.1: version "1.2.1" @@ -8351,10 +8356,10 @@ source-list-map@^2.0.0: resolved "https://registry.yarnpkg.com/source-list-map/-/source-list-map-2.0.1.tgz#3993bd873bfc48479cca9ea3a547835c7c154b34" integrity sha512-qnQ7gVMxGNxsiL4lEuJwe/To8UnK7fAnmbGEEH8RpLouuKbeEm0lhbQVFIrNSuB+G7tVrAlVsZgETT5nljf+Iw== -source-map-js@^0.6.2: - version "0.6.2" - resolved "https://registry.yarnpkg.com/source-map-js/-/source-map-js-0.6.2.tgz#0bb5de631b41cfbda6cfba8bd05a80efdfd2385e" - integrity sha512-/3GptzWzu0+0MBQFrDKzw/DvvMTUORvgY6k6jd/VS6iCR4RDTKWH6v6WPwQoUO8667uQEf9Oe38DxAYWY5F/Ug== +source-map-js@^1.0.2: + version "1.0.2" + resolved "https://registry.yarnpkg.com/source-map-js/-/source-map-js-1.0.2.tgz#adbc361d9c62df380125e7f161f71c826f1e490c" + integrity sha512-R0XvVJ9WusLiqTCEiGCmICCMplcCkIwwR11mOSD9CR5u+IXYdiseeEuXCVAjS54zqwkLcPNnmU4OeJ6tUrWhDw== source-map-resolve@^0.5.0: version "0.5.3" From cb802661f2aa34cd574f7677a64a1b9e72fd94ee Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Fri, 6 Oct 2023 17:47:42 -0400 Subject: [PATCH 28/85] [Python]Update state cache size to 100 MB (#28877) Co-authored-by: tvalentyn --- sdks/python/apache_beam/runners/worker/sdk_worker_main.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py index d3442fcb5987..1e44a998ba05 100644 --- a/sdks/python/apache_beam/runners/worker/sdk_worker_main.py +++ b/sdks/python/apache_beam/runners/worker/sdk_worker_main.py @@ -247,7 +247,7 @@ def _get_state_cache_size(experiments): Returns: an int indicating the maximum number of megabytes to cache. - Default is 0 MB + Default is 100 MB """ for experiment in experiments: @@ -256,7 +256,7 @@ def _get_state_cache_size(experiments): return int( re.match(r'state_cache_size=(?P.*)', experiment).group('state_cache_size')) << 20 - return 0 + return 100 << 20 def _get_data_buffer_time_limit_ms(experiments): From 7531501ff27f53bcb4fcd6942f34dbd45665805d Mon Sep 17 00:00:00 2001 From: jonathan-lemos Date: Fri, 6 Oct 2023 19:21:19 -0400 Subject: [PATCH 29/85] Fix interactive tests for Beam Notebooks release (#28880) --- .../29c9237ddf4f3d5988a503069b4d3c47.png | Bin 65527 -> 64875 bytes .../7a35f487b2a5f3a9b9852a8659eeb4bd.png | Bin 728485 -> 687768 bytes sdks/python/setup.py | 4 +++- 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/29c9237ddf4f3d5988a503069b4d3c47.png b/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/29c9237ddf4f3d5988a503069b4d3c47.png index c748ea1a2d0a4f1f60273f7d4ff35809829f2c6f..382063f75092d42897328868efaefb88d744dd07 100644 GIT binary patch literal 64875 zcmeHwcT`k)x^3CE(niK^6>JpR5d}p+K|w$$TU+T81qBfSp$v@(5|mWrQfXz{0165! z$p9jfRB|klRs;l;D6z;nLs1k|c>CBh_syL(>#aBQ-d*pm^^Sko_Mz&8-|q{1f8YKN zza7%oU9#}|g%k>93GL^f3@Mba|3sn4{pp*paAcF=>ZcTnK!x_xk4N0Y2b(=|L!5=v zV*}yk8bg&E%+BuF{k@`LT*5lN?@aeEe)sLkU+%sCX>HCo#rf9D_ts~urKNsea{qht zn?X}U&&vDw!pt5<{y~mT&+n#&6+IoZy%&ZD@<#@I*=rG#T4(7sRBdE1R2^s3Qy!t5 z*?M`E>Y3>DKq>Vk*MZvjqludY#UV--nx}sMONgSmew2=nU?|19V))Y&-J0jd8I2A( z(%POM?yJU5TF-KHc%8w8;#BY21XC;RZzz=Ivf8wuA{i!syi}R(T4#}*J6@uvl=Ta~ zX&vd&jjxs3K8?`}74{i0pPOe4M~V1720FZ71Ua=y7G9rE`uoiE#fnq;qkiH#(_=+4 zGM>GU&ebI;#vfj^#eKBZGS71_m-wVez)e2pI*Az;a!a7>e?RC0teU(-$N~H?xbEZ7pqt1dc^rCR7uCzzz3?olCGoX6r!#y1zPds#x ztj*^aCmYl4+3p=TlPq#Vil4D|^?TV3)Fx`3{rK=zHCj&+nzNVbOC;Agj?1}kf8i7i#bIU~Yyt&woI2AjK^ zj2l{$upAF@-8i$KgQ5?HsTn^%!fW*tzc%gQn8&I0nCMPqS+(SP6kc7TF!`Ha$mXo} zYYN#eeU(`1AN}UXwAcj6;-YKR>^`>^_~p3tIgJ*GvCkJS!P-V_KX(*QNq29%a$nUt z-Jw~Nvpx|Y&ci9nH3t%KPl5ru=)q18Cwy!oRJQEi){b!J^1hFc_S7X=$hfy&;@1gq z!!w+LI$UqhQjK4*4U1N8mT%XZchUY}YJT=lOIGg~?F=?hx*~B|abq@3a0iE^)j z#Ecp4NR%w??X?x=sb_DC*7{aP(|F7cwDC{6GD9g9Ix7^8@~t|AlT`)-83%)8(`;*C zSy9)$F~++g%>~H}{+jsN$-aqt>nOpH zBQvKnNUz0*+brdfxmMHyJF9(eJn+b0dS)U@v=e*w@^le#OHcOQ>~G4NP|6upv8{gQ zw^UP}^FEaBn}xlpqceS)vt8@hq71gLSNDDEGiT1UVDaNlYsI~LUtFvq;!lKbHdXoM z{jKZD`R+55eQ}QIUs2Rv#3UwPO{DD$k`>HyD~gL(;S&*z7LV3TVs6-qz1LR`Pu<`rqirRWU+MT=g zv%~Pr2E~jf!+dWZ_UJ@;xQ4;bv7Zngt?@-{oSt%CDqM*cbR_AxNAR)NJ@xxQG7F0}t{;RCecHCe)H>p>%$CuxDW-|1LuGXFUI`1Awks0(NIWKV9}ZHI zGyd0MT*Je|LqkIYoAcZvXKJPq&MiRJfc3zX5Y=Liu18|ag+br^e*1%MHik!!)=o7# zMn%c6cJao$oSmKP*g?U;MtHD4>*V ztNEexW7k7*&JV6T!1E9gJ>YgoK=+qSu^;~ECnn=PdPz(=6Da`8m<6{N@hX^zMq_7& zWGLzL;y5*B54LOdKYL=xD*GVXeW$vzy(Zp7gm7S<=gGl7IXOG~FJ2WSTa2^iPq$x( zVYA>>*wWm=SEbTElb`tFn@+#Gp)rt<^8L26w`7Z-9e#3ET*h~*-g*Raj>t_`p%%oZ zSATmE9&}SmefaVsrDKTN7K~=si8A%rrRTQUH>ARBtj06uyj3`j^x07qNbR~~)L$Kk z%S1x6|N2=SKA@R5b}A*I3~7oI0KO>UZtcMiE?0PvG}ZUvSl8U z$ljelyr98scnFN+R{>8+7a|YrU9W!eJq(Bgf)>)kD&^&|a9;RZo-*?O8KwYvIcbFE`+1a_1KU%2m zHMMi%gKDs>O-18-<)*nDuEUC zV~x$31DEG5T4<3}_w-;Gliwl3>!BGS+I)Jv*S<0RV6z?0IoVegs~_<|W-{2UOZfY5SeEc(cZfu6RT{!(ihL5y(jJJI zT7xh4Aj&TCJ!)oVwk%LB)vn&@-HlbSBfX(qY_QgF=4lu+9mx%zb_R*7tK@DY!gK|m z@a=Vp*nl<)WkBJCkLw)0>$vvFg6}qjdyf~xR?im&tT9PA_R_BY72l^@M&_QXb@`hs zi@11xeoj($|8u_dGu;34l|i!g2w@?cO%1KxwRT8m6_z-k8f z6mj?Nb|e`3s>Nb!Tt4DJbG%71LXM82qW$ZOb6|3HgkNRN+8`Kzl4*LP8;JafkjMR? zT9IHJU;FEV@7@KckDiF@IvNx^Beksj?29pP{pK&387&l_H3ZN%^R|``HLk#0v-~V>SDq9dSnW0V5*HjUgRS?) zgsxn@7y(TZVV#HxIDkj-#fkJRsuB@x2rEHih|JEy@-1Gy<3%)kGlk#1JU3s{%q$mK z0LVk|s#K|z+IZ5>@zjS<&4?Lnbpvi-|8(N(5P1OS)YR1B=A780efB5(hRVX!f%iRy zWyB^3;&0fW_;7FQ0DJXL_tEa91h~6t!l_O)=fYrGnk)qSwT4owZmPFrj!p4huWcNl z&vgLycIS^3J<&~fWcvzO#Q;v+0x_8;cTvtzdLy2KqZd2>qWx}LAJ+&?SAC7HJN%ox zfq3(DJ0eN}S}BO4i;zl>BwK2yxt&Guj=SmH{lS!}T{+Va7y+0Ck7e|JT$|Pf=wp#? zSHFQO^}4nwIeyMXcKYQy6Nwk8?#X#mo4Y&+rS(}6MhUpsXOtbh3>+;^rA5E&F^ z2!<1g6M^*%cZWGDxjzTIdvr9>EN&vYP5s~&3Wen{FCkg!gv}V(6jQ*d6PHp;08Fut zaIH96<{XBMy2J7M z@)!Av_R^FS@prP*D{)?nTEAWL9QzYtI$2^A$`AG5hR7lhI0W~YhpzCKz!7@A*q(M( zO$x;>ZZi#pbob>s6v{UX=Ho}QlAv{m>goRAd{Wh!7dpDj!bT8C!NZonSvDWE@Zrko z#Ph!rU(k31^7rf4b19TReY==K+4VCXkg~Sm|LVbKlb25pHRMXrP49(;nb;<0(vbD_ zLX`+0Ss#GAU@eG|SUh52u4&cq%0@;>InAUg8gW*M*!%G=U;@QP8ijJA)MS6b7E$c> zk%OioIZBS>vw9MWN+|~Y_<#i%KttNOQ1~uxc>?+v5PBNdTHhW#!q?^qjebR;C?YwoES@7mc zD1Ga*J+XcdGCp&AhqWy|IGlqzc8KN(IA$3R7X4@3+@>n1qHyMSxZbhf#0vYj543oC z(WWW}gx00Mztwo_GUe>xFdS7rsA@Y|rIDqvq`ywoBSD|aYkk9zV`UT+0@Gz6fTx|S z>sH=VQd(*bG=mFVx59>e-gMV}i?{G?$;cT|`0d;2LXo#%sA53Id3yVqiu=LAVL3(8 zWtKi;JQKQS(wIR+>apxDF6UDou*J;AC3Fn@6qxD?q2Ofg}UBj zZ$o)S3)O@(PmJS_Djs|BgPfeT-w;BV1#qN_s;b`ULlnw%BCm|z$D^xSNB9i$5J7qK zdLm}u`*GJ?m>yB*y|@Mv_Wa03MMXtN?fdudgUK2$f(i1<#0vdoeD4GYr&I-KO=FjnGKWSrQLtFy{2}x8KvdQ@Q9c6m{a{qahi|l)Beo%CG8TV&Bz1a$JYU;p8 z4}1M`P~2eI`p$VPU%h&@$s{Q}EG*3I-VuP3{dYGJ50@$jS6~OQ`OB`Z^cry;P*hj5 ztv=uN+}KNv-JvWHhRYVKvwd@l#Y~XK5i)THSe1CQwC8Vsrce&ONWEDh-jbboJj)3= z5h&)f@9fY#IMVV#b(XA+Gl=*`Il0(VZ?COd)!5`EM!8s)R<`COJnP!9I&&#Lc*yDB z@jLlMPfov;c43GWE^DO?*SFhP3HvsyRppypS-7Ia^H*6~5I7T2AEKl$50S-x{#B2@ zwtVsFcRF4!+zzvhfzLztI*N2ulYZ6R^u9Ot-RfFSZ=PAYP_V2y($mv!l|F8v@$!w` zGGlM9D~o3d-vt}=*FO>nM-DvjE^}4ClYB_{Y+ZK9oAXkVqNYWDNzzXJ8}r0_KerA2 z7MzsZ|6qJYiKwmj$$sXyF!z1YiWb?at-6!Kjy_Cp9hAve4yADxnk z*?ncU?t+ng-no3!6>8pJJ9_ zdzTe+aNsCiaPADq-vaUU-)LvNImv>C^fGp(11p_gKK*=B4mC4--L(ZTPQ5LGJ04~G z&hiy&z|9Hbb~Hi0K1=PDH}9I_FT}07E&p~gLJvv^irPb>$U{mfaHXAd-tUs|*x=Qf zvIfIQl%Uerutv`rS5+=#9_ah*gjjPi(O_-Ct^4)7rSvqn!NswaLoyk~csxsw--utc5iKgW*T(f8n7GLtbr8#}2JcYwHi`=8n* z9Q1;I0ZgM#j-IA|h%jfwM(AsC?wrnHDExMKVcvQ3AlcJVYZz>SxS+>hroSf9t9wVZC#dcq7p8~#!a5|m&)oxE_xraOOJ;Dzjh{{%(; z>AC*#;GM(0>yW6N>zcXw{Qh%r=R}j_VVbtVj{KsRF6@#u+6kKPyfaZH7&^lDnll(!DYaB^~C%&`YO# z=sQ>=8aZnADhNL&7S0y?_H~bG2WEOJ~le_-kbZ@3w%Xnu5tc$oAjx7O1D z5?BuJjd|*I0M%`}G9z_QT^AUnEznXwdw$|(T_UwOwNED1F@xS`K!o3bve}U6qA2(A zBAK2j&ywQx+YWxx9{>E_W5SnTWfL31`2mKvz46A0dujqdt*e2VosSm{)HcU;&fj!x zqfzXVyrHQ%0_8Z9WQQ+yXP)tRej$j)Y$(gpt?6a6*XqVR*`AF}0H)4AvaWkYo^M)E z#FhVo%HmDlXUN8EKAzee7#o#!uIgz9AoFbbp=RPher3Uw)+WB08HSM%dI29qa?U`}>nFST;ZL$I}zI|;1lkFn7RUzW-6;4^WwPd*s znZ;y=#-X(<@pvRE%kf+mf28Cofo^`F7%giJG*gl z#--9^$;obWM^OwrN^1Dk1X2Utsqf;oyPMnuf|D`SFDI&m-x|Gc2Bfl0qt7Y`)gCgE zzcVrq&>D;ObRB5%ZB%@#vmd~%AUO4NMp>(ZSY&06iLD-2TCO_T#Bq8}#)?_Q#a;W? z>Pl`pny`fzk@5U!+^d$5YxPvo@i#{Ho z0Aqz(dbrMr$+T#-mmA(sd`Po&@9VOuqPmaf=jeR_7H-RZm+mR?9nw;yzx*>TsC0&2 z*>^Euy@ui1Df<+rZ4hfoW~y`S+>-M|YVU{F?#WlS{%|bUz4%BPC)i(0X&y>w@^|gkMeOaP8bCbP0WE{I3(Gxk^oUL_N zNaUI!`OfGbI7x3M?_!vLOFE!qbysCez$|EJiE*RUnEskDwKfjb}n^)t6GxA}=vJ^Q*$lv$I>pd9P94;4H; zC~Dm@9Xb$>}v>+DgMD~db!C|Typin@QzkZTdO z7K*np7!hmSp3zyBV*!Tex9|ZSS1Ou?GbhS2HZ=K+2m-0&tpTq6+s+I0vjX?w6j1pYzbDs7VZCI={P-j2>=&k@R1Khg!$} z3L?!uMVgI?YsFvh2!!~)=No{kf2#`e=mzRi0G;L`=wG414ugpK*&Njk(~40#=T4%R!@vd8=UjY?7EP+7O4v-q$gkoc;)( z?kLYlckezt8sNa6=M@@I6?=vLPt4KT6KN|x`PpEzWkrD3u%pRRVU-PYu&M82#)bpA z{A)jUDcqrUi#Pv`e-SyURd+KzKi4%UfZOESbIC&UZm&EZV;=RPSKxQ(%J5$2F=8Er zeF@Ghl`Z*$_dc!HT!D;Ix{&IBQpk@Al8wu$3OHArq?oF-UmvxPP}O`}kSLz{lw$^4 zf?ocWLqNWao-m)U-3~%?2CO4cHSBN@hpK{p`>3##ex0B_7W6 z6ow$$C!_g{l1&U1+i$lHS+A+d)j0l$4G{FOv~Lm5r@{kVg_(2e(8dGlmq6e)(1JoD zLs;wcj(Y~lf-WiwPU5Okho`ej*ED2=O$nn>aB~K8or?oa zGE-CE$iO1_Dtoh+?rHFhDlv-HfjKhPZ$JElf`Te7$Te$VJR`c?@_k9krY_nScICb$ zmn*zuKoty%B5#F6-g!I1L80iu-TPT5+TQRL5$->flsq?#*>4j_z_uH2{MzoC&{^08 zl*Hn~jGi9U3yQX>+OS@$ZeL|p6}RDB1w+oDoUOvk<4!fCcz9-CH%1kQwvk^o1#CP$ zJX_Dqn;idb+-F+IaJfI4miwP|nA_?d*K$H~;ul9V#TqbXP1h6=^}SMLN!0ZEgh` zVf}USbv=Ws20Qb0Uk0MC(5Ya!AX2q5j|JgB|8m-pZ|``jA13_#p2eKv^;l@SX4`&z{cS|ti74-xJ|hEa z$|kMZbaRKzg~XzmABsNVKEzR<5tq!QH>yd5GaM$gr{w$Yw{I0*05<`byWiK>C+1>D zKYIELSRXxcEY*DsKuSYUZKRajnrkTej51e^%yvYt5b{B9v@A zfY5m-3kH)j!it(4?ku^xsaeCWSHn7r*R)8v*`?GzW(&~iq3(tBAat0<^~GE=!5~n_ zZ%iI~{Pwy?*xAdP%y|goTThk72Scu%Uvo7#x7n?EyXa_6x;?6>v(4NTz4P)|htfmt zTdEU`om&E_4liCBzP+|yRUBb$-o~Ah4QckOV-XCKgyZ+KHSA|5pBq1~PRub{AWqz1 z9&WIj8W|oMTF2!tVow*Di|g=vPp=dCmK4ho9Q4h3EJppNosw=17mB=en-Ett&VMv! z%5bV>40QT#R2e8S+fn?;ZyUOVE-c@{Kxmn7YvX6}p%;75%63Uj94}TO?3EiDWEVcDK zuxOv3qLL)$);Ib4Y4K!MSJ?tbk)Zg~tk5AUDBQLxDQoJgbrF@(Hm_$WzcESVK9Q&} z-y-7nZIAXb?Y|4!XHp{1dsnoSJs9!@#=h=z2c+VwVT)J4*u)DO=)R^RRekFAw60$* zADQ+WBGKmy1J%8_nlC$_ zZp;>w@pTz)^01Eb*k>w^5NMuK_RUo}p-wiUq`$P&`Qn2{Kf!rC0U#_d@T^d1R~#^f zJ)S#Sed2w2IZ*NE?*gex>rPJ`-MS)&DViN{E}xx1DD1^@SZNV_gmdlt#T5ZeCCI5h zunJvJ!;g)siM0=_^-G#I|D&vO`OowASX#Dp80_TD?h%heA)6+C9z zG|X``JwxQCx7S?S(pAfK7;SI%cDxHv`gC|<{`p1}kEetd12;41&NX-Qyf~X*KEjiQ zrPrHyDrw9UBVy=GIG*5N4srkGF?5sR7QRn)FLiRqN!HSc92rJ?Z){P>&3F2Ph))a_;*Up{-m zmllv}gN}qdqdg3G!qidk?cZ@ts{f-|nE+p}b}dJistxbjitoUmrhaF5uAcQr#dI#x z!S?4S6>zt*G7E?_f}cb^Ku(p5s!sW9LAgy=Qh!HM5kz0EZF@tb9j11+Wezt7D7D2c ztW4U|)1`0}>Lyc>p5~g5FBcOv=tN7DM^H^?O;?uoaJ8+YR(cr#BoofV_F*zUuN4b@ zb3SbH^E&a(mbslA^_JYR?ngO;uO==DR#D?}y_R(e-Xyerr2$wl{8htv7KeN7<^={oS> zWjz%lo8q9|`Q~w-(ei2S+gbs7BhM^N%25*lgT(Aa7_&11Ek`jBTj5 z&}x=b49eDf;$NjCOL}v{9P=VrRw&erq*ev|?TZ@zkI-Q$;y-F+>)uQvvR$t^&1&kHr>kZ8qrl*u8IGo3)-}((a0r2qpO=m;8w;L6ooA$IQcpDHWO> zqlHTqr?!R{!UcIimxd`>a@wJPB619GA?v&pmbL8PX_Ed|(t{P!i-8;qb(^aC9`51V zq_JtS;ZyZJqRm%cE?m=huft2NC6R}~Q{uT2iRJ-5E>4}9wqURuCw}elZ*dVIXzNo` zDww|0kCtyxcn#y7fgH2cjmS$Yk(VGR%n@Dk7rKwrd@3|%Z~2L5i&~b^f^K-X7aQKU zuUe^O=7M0IW?z-P*cy4;{-?DC5Lh@%C|Y{&Dq0#xIF!Z<=U8q_l-ao7l_b_!w?&v* zU&~OhN;pz@`ZAEg@fD?L&ccD%ud4L5%A=^PU;M8fj8OC8J1ZYooh|`LfAxU)@ESBd zpx{7nl3U;}gIb3c13=0Wg;5*uguGK9YnVH;7%69RLIEKtFA_J(N5gOy-R8RXqN{VM0ZkiRA^4C;OI&TxvCj z#CMz@_ZpG;#Nlo~S1o^~SZ$PG#EWGv)YFxdjy>9~uV0Q;ggh`qNu5JP@o&FW#CrWu zQpleb&Vna=ukPS7IHuh=!X6G{O)5+suB95CI!&}*1mZfmV^h@CKGpWVHvVgSHvT||)EQXSP=pT!oCGrEG}LC5|&WD+4Qj{q5EV0zD(B+QpFfi z<@zEbSlJmzLI?%H%XZspd1Yl~G?dd`y7&sA+-QNY3F61QgIb#{hzqD?i<-i85bB)Tv&iLH4E~;QYLPHl!rr1c5?Q z#ztoKCVCQYtf%4PR}yqLDFuA_BrW1f!m4Xm))CqNO7-pw%ZPv28%8zPQOgfKpgLL+ z6v;fSGI#3Dd}?Toxst}^lX|zq!pw;tmEHQ|{*NRVTz!?{KGqJg@iPcl2~}Dtoc3bk zkNp`!4-wr)c)ZpzZ(5)(37LMHk%4v^AxzU^Ut#UOLN7Zs$sWI;$uxv zA9|H4X54971bb$DeE8$rv<&6^>BkT^KxtVd7v-(ugq+@B|454v7@>9Skk9Kz=Shx+ z=JbPQ>-HUomO6~%m<8zx1XvEijDr!HX@CrM$DPZdGRrNnfJSfl^UDspa6D9&hI*sA z*VG_}JtXwDEdK%GG$V#f>e1LIx{QbJX z4^Z?uGvgwZ(O*Y%7yU*BrV(^OgkJiURViN)8+8&(-!>w?M z`XHQZ-jQZ#+`MayP1e(#pgZfX`?P_*`x8Uh6-pn*7XEE&0~+YCFgp+23cF`?Ugce|j@21#8Vs$NUi zXkrRO1De=_wqBEr7KkFDb;b=8uMc%WsN{)CE2SYP#5_i4rJgcGk2qJNZlJWqs1W+0 z_TS_%MAU*|fbaBopgd(+KRxm_MgAZpfQ0(<`btHBT;f~_7}J5q?{n+cID_J__1fA3 zAP1Se-8 z_4V~_2{ji6e>(YM7Kn32tDQgK7SP9Qm@{{NbVy%CxW)tmP2#ba?U*7WLIB2G%;VNi z1HapSdi)mGyyZC@O$#0u9SG`#fi*zDiMCQ^!Ro451JPhg1vGx?csNW9sEQ=|qoDj- zVRHQCM7fr@^v>yreoJSzi)l-Hj}|yrXsgqL{G~m6p}v-HRb1X2v|cZrkc5QAMS;fW z78E{~u*D^z--aB?1rhPa_uHQn@@em8+zo`vcB7TpSv1C)T5qFoDL5Ud?baU`O9XI~ z1&KM4a8Yt@F>8Xw?^1@)(;x_Ytduo7oHN0lKqHhGy@?HX+P{oT-1t%m4Y5mm{Fg$g z{1(0im73ts`C62R(O|HN*yD-rGH5ff13oj8gnm{D0(9tz@fg_QywNv{LM=V}pzvj4 zN~SzQ3&-Kk+z~caq&kL4rH`=&QK((QM$(8A;6sLu&Q4D7mPXhRR>%>GUmk0_u!1SX zVn)FSRG3$Bd<2@@0|Gu=W2uZzE}i8IiS8H%wK7q{q4VZXsEKA42xd4C%TA1!#x71S zJcQJh2d9MBh4C%if@FzDK^F3adXnzh7sF%vBS#Q(DY&0I*&ee)8N^}2q!HO6Zq38w z4;0#RB30g*VhFC0tgzC1t}f0ZHWKqBSdFx^pH2zBSP>S~F-R7T`%J7a#PU@TMdBP8 z2!sFxrSIS448RJA?r3pHGdlW4Q;^i%_9}_Y466Y+r(0$R~rXGaS zsw&Y)*i%EQqj!t*9OY^VN72v(I#P!!zjwq6JHS&md9BC9gA)tEW{DQag=1C##!JWr zp#Wd)^nNXEl#$;Np!HPf%>F}ke2*266qyEzJ|tk4dD-%_Q7Tp?aQ52^Z$NkIi0U~+ z@eXjYPwaINpkY{nuw>z;tHr@`+}=;?(q8rcCKq`R0{pE-tBsJ|2#X<9&fO2r5y1w) z3Np70eQ$I!pD@4FIEA<+<2qdatsuVWDg`^Y$>x<^)v=G2n2vU zGkim}{1{R=rlSZ2eVJhU@O;A_nb?>ZJeHDZ@baI9;R$t=u-PU$6AfFMT*Nkv_dzNi z>DeDA)kI8#5ZN5AuMPOo^Wi=b4G8lGX3|}K;diVg?9~f#8`a%dnBiL}UmG!tLF8Bd z^cZe-0JaPM$Sl~>hw-52M{Ho#NOd|G>eyfq$&aN9;$Th#OeH%sw)_37#fGaLKf+)= zF&Y=EGe?eA3F!{z&wyV1QC@`*zTomaz>8)+ZvuF$2ELMf!4-!ZB-hB3=n zX%XNcf0)546}bl!BV@R_NcZ;N5EsuP6_jH{VZ*@(c$lZH1>F@ti5AWYU!;}&r4dkXqRQ~0Y!Vz{R9tlH6pTHNUiKOPk zslkv@1^zd)xTh1^kb;w*$#4O`UqmW~uVDpz649czAx% zbvk42yg?DB<&uxYRrY>*BID6{8=#GttIG4@T8eIcLpfAZD{2S=i;Z%rqzy!+m_cA# zEgqL#=ZKln{;IcdR5}0$)-Ro~Y_qf(oGwtK`(9URC=n5W-Y4fP-!{9DD%>;xw;qAZ z%1?Xs5UCRr4CtXGTIOkre!5QQotX&S(#dX0hzjQD%!n@cth~NQT-&AO&PGhj{XSn# zp#meD4A!iLr>AGX%vl7#C~OOC8>o8<%ZH0MBh8`@jqt6IL{q`AtcEXt+DKnxJ*RXQ zF^d?O14O|T+qKJU!9Q=@00Kpg=!D=Vrr_A;lJg9;Y^t7?5qUWxq8W}XOG9+oW>XoB zaF=0IbtXlqhSuEunImR{3w;;uO)YSbi&EcFYqVK8^%C+bYClTHGq*5HG1rpvMai2h zEX46UAt6&33`nAjKB=n+zM3~8f>G@RV3bfao17U)W_u1K;=Tu>BMCu6 zzyrfRjL9yjxUtx@pyNm@1C9q0e>#hsB&IHyLO!Nlp-#5xV1EU>e1%}nV~YjzMXEa& z33r6vzWoUfP9!+2`ZR2pFf$B352jY>hly5-D&Db1l*^*7?*$q0J*w)9z>>}o62&9Cd(w^;!Ug#?Da;fdMM=%CaOO0|x9QnlD!y}?^ zAd~+D-Ww-=z3S;fOt&5uULvMFnh|>dGInC3EJ(JmI<5*h^$>g*gT_87-RD|zd6-Qi z#xJXPN_h{q9FIF%Eb7@jt+@O`E^xA@mX;QWoD^D>2 z#Y#Xc+{4=1 zI@Y%_q9Mqv#1gAn&CB^D28d-F4i*Vt62ea~|3su~%Hf_1t9;qXZ8_7rfPvq1UM zJWBd8z}PZuG^S{jM08tY!~Pb;0vsFy^J(f7SsVC_h7kinA#iSV0g{o|;*!IQsn$-Q zX1sxHB455;Ve$*MFg8}*p=lc4c^rccGYT7hFJZY6kp-`)M!sXxzGa`K27p8Hy$RN@ zn`l9x(!R(5a1-Z#x^+(kBMIOM`mrF{5l|}R0m6eUU^SiZZ}Y}de*DHh>3=bhjQ-7+^5xs}*Cq@me6@{tf2O zGR6vh1&izl!RN}*!2)Oi*`tZm=L@zTIPll95HiGjPU{!^>lURVayp;AmW(ZX%f$%UFMV%J!i87?NC8#PP*a72D!rJTLOW^uMRfZUhh6O=MVwpDs zNJos>17;*J=k)z~#yIV;CJHr?`zvK>(s;=L@zw?ue>P4r7o%K8)QC_GdN*jaRr_#Q<=l(&&{0da%&UnG!3B3LtAwK!_ zf0%Y87ld38#D#xVluw!sX*Q(UAoG)sKso~H2&5zYZ%)UO;ereoWVj&11sN{>{*^mq zR3xJ!85PN>NJd36Dw35bG8_Ki&W6*>!MaNj%u_D@z#LehCVw6OT8|dQ>u-^DPAouQ zMkF&LnGwm1NM=MbBa#`B%!p)0Br_tJ5y^~5W<)Y0k{OZAh-5}2 zGa{K0$&5&5L^30i8IjD0WJV-2BAF4%j7Vnm|A-k~{G2=&a(@ctM`7)Ec86BZAvOQx z5Cl2yLC(LB13~2I6FCb;PQ3vdkdT0c1SBLNApr>qNJv0J0umCCkbrzu0{K!1@+}$U zYePt8L^30i8IjD0WJV-2BAF4%j7VlgG9!{1k<5r>MkF&LnGwm1NM=MbBa#`B%!p)0 zBr_tJ5y^~5W<)Zh|K7~#hpB;Gh}D!`w^*F7rS9w{b2pj0$=prmZZdb1xtq-0|GjfJ z;xzeY3G&Snuf-Fstr3tb$LH;@b^49^7OsD60DJO@dxI>0yX*TBuXJs3W-ukltQ8u5~YwR zg+wX;Jx~hSSSK6nWMiFdtRo|lTD$*0w02)z%sc$tw;^yX!A@y=^?yp*ed6-}05s*; A{{R30 literal 65527 zcmeIbX;f6_wlz%RQ4<@8u|Y*3DvF9wDhkpk8YKk+Dhkr8h=_Co0@7BZ5feqw0zsrx z5fMQ^kiIDql^z7?d!%m*fublVs@}OV_q*?R#<}nJ=8W%-`)B8j;Zdk&@8?--t~u8; zSJj_Kwbhp|S-(U;Kwvrbm!FRd2>jza0RhSHzWEwI*>QZsQvm^f9`)y+PFx8aZng9b zv}hXZRFatd_2J%oThA_BxaI2dpN{VQV|9JPwr8jGUZyk%ygd7p&FOck4<9xp#Xml4 zbj&K@n@?FwzWI7d!sK^fyQ!7sxh202`Soeir45X}1x^-@l?sim?XvvLM31e~6bj`l zfexB_z*emi_kX0V%4X(y95$#N6E76=urcr-qnUbhdXmMy5)de97|mw3#T%q(ZCdN6 zGS#3pxNqjEoRGlltUNw%Hd*Yq@|Jz|#sL!4%w(|z0s;+}zg;9C@WW)}>HE9R97r_H z6;v8dlCy2P-2b8Trc}cHJr){+>T0}cb3MH}%SbQ1nF23Xt$y}smi6uR`|1qEWzQec z)6#IvEbOjN41XE_fgM_{#<=*m#(pSudpE>|6@jnik=C$ z+DEf^cXKt~)}%SjwmH?>6@PtUnyZ+q>NegjrsBTg((A7li(NTg8KHt#ACjP&dVYR4 z$8J~S-%oFG)Wix4sd4XQao#MYyt1xODDwK5#-DBX-~Z_k=lVk5bxxC2>M_yL(RMBa z)mtPb`wE2AT3$=Gq}#W*d(Xb;D6wbE)g_uSHqmy=%e(e}cqAEi=?!m|No#Q^%pH5P zG?pdqJ-ty%%9=aM>ZVt>P^82iQ*^J0QbrMDm)7J=)!FC`t=s^JSr! zzsl4p_Gs%3A#omJcq?^}Q*YT)N=}k?xMG3Na*t1cNzKb#xGYkQp%p5pprFw1 zIiWS^DybcID&4LXyBuw&{b28Q++=I+Ve42k1~2QDX0S}SlB-?5eD6?wqS?!{?`XV< zNRe@y*{P9cEY~$5)%}ch-m@d=_*;fO;$wD7n(NBUWw*sdxDO7eRC@C`eL6}buauZM zvaG7OSV>3hYq)cr0W)uQv|PSj`Rv<2*livZDqX(q%4RA(E+K31p1vuS{=}0r-tNV2 zm2dZyy)xE;WHdi0n0NNwP19WWeOA@oe@aGN884P}ZL*5dl*v5z`O`a*@pE7O`i6L5 zdYk*uZ;QqDU+Q>`wBT|lD)|UbnT(FBVPRp(V$0<=_Ru91Ml-r(@}@sIbiH>Tsw1K| zZ?alb{r38M;o&A@-m_Rms$}@9i3+7ix84xv!5XT`_j8D@6S2pmX?z~;>7%yN$X?`D zOG`^tlseXj)@+*|uI%PWmslz+!~I++o_%@X_^!20udF6!-)rUSCE(ig-<6e>E%#jB z8mFJQM%wU5gt9{IvRK3Wc-pNs((bImlSLF6ldQ|#MRzU39Nr&O7p;Bx)3pSXY%%AW zW7x@3M#07#8<0;oG)VCn#Bq>^A3?He*5Q2*G7wAfaQ~`XQqZ5)%e^Arl617v%|O7 zn-}_(+xu`0*DAT#bO!(EiSWwI%*2M+^E7}OxV7*ZB?)RW*1w1U3=pCjw9|m;uzugg zKye`v$1$JfB61gN`m=@-MC5E55I_MD5p4h*je9$6yC)V2)S0)MG)fz#0YHR9%optqj~dws>|;2=qb5pU|`^{I&*=l=GE{cKb=A>!AMgome+(X(N*Zr z7~4 z>KzH{AT^xE<8bP=pYXm$cphnuKlSo#{%nkQRstgOw&Im*iR+wzw5UNC_Ohk98oD1n;WWd*8gSb*Tz21J#Owx5grr7Z+zDFo{G1sGoRt zv~wttkvEz(RH4uphC<|qJ7TxGwx85D+2=VuCP6I;rtq=$=PWIS#FR5|KiKDBV@K-Y z!|z}Jwm^2zo>8ELCM9>W(K1p?Jpy6)~X$n}P#GX>Q= zryqX+hAdpVhDcIuHzyAJAgXwIx;sE;3BL#T8Lb(-#@Hb1vek}_Q+QE;zin?>$jYrp zGWs9i+MySpfl90uq3i}=rgcvqKq3&pH+J(nf&Y?{F<4y@?>pS{ztGPeytt9Wr%Wzh zw)Vp-tB+TQfFjbkrtUezBz z>IR>!OE6KAk{bGOP{=agX7bHQ8ZUquHru`$&(`%o=_$1N$IG<9;^t z=4{#bc4vBV*iv9ANNRi3{~7#>dZB zgk_e?HLv28f<4XRe#a|&qz#hKz4u)gTR)1NcmDWP3sq6c{Jn3Gv=Oi=*`gF1I(K*J z+T((1+%bJqw=H~WF0x-iRyGB9D&l25d`L@8ZeHUvQ?v#G} zIAuA~S;48-S5!&PIen|vgUNx9F~%SnvvX?&k{XGuVtCI1?O%x4=j>*M9Ky?U-1@?U zn;eyKQ|HScL|@!UrDGRt0a_npbadzvva(JUVG6;f?gLfo;;pf|F~n0je0{^M{M-d@ z!0OdIP888}E(M#oWX?@B>oULHnjd7}P;<^ErCcs1+@&eXUo{K4?=sSKGSSp4|LVfT z+^MGS;sDg^x;He=+m$+=o!5nD*v$4Qb$`&DxpSaKJpAoSbwi50HWB08vx7o_ip3`Y7;~`@4Lt zK0alQG_wce5>2fOYkV4>F>Q;=%9Me^Hv~lu0kydC@^B?~LiXr$0CuOnzzHJR-`$Lj z+rNZUa{Ws=hyzQcYsx5$ycei9^WnGaMvrGn~!y;qoBsLXXuLH@i)Dx(S zG$B?Btop$|>!;pLsCuBLa{ST_hOaK{1_GJ5G+G=qjRm~*EoA)OiBhgj?~YHYSP%G@ z$sQhmFJfxu0-_lvu`JMwunXQHCrYvDHncOWzD5m^f z(;p*wli)v3Rr9wX-cfq%O+9S$+cj)Ey$7O$larGPl&tw=7%6RVwEV%oU}>YE;NW0d z^d@d4r}9vxx~TIBUdV5$doO%A4bHhxKf-%vfJ;laX&eXHGh$M(JijfZy+paR7LECi zxV4RUTfx7UtDSHtE-LD8Oo28~sW+r>nNFhwDvxHy(uCITdf3sbtvK+cChgq;Fj8t? zMYu`NGJ$ua1XJjJU@hj}|Hx~qQIh6>A`&alq)S+}<+u$Ne40@E&Jjo%YrW6O(eYw^ zLO7owsCwUri8ojTn>ac-F=htiYDS}&3_j0C#ICKwNTur;laJr<)anIL)A&o@2Q4$m5m^{IPdByZa^`83o{L**?hN)AH#jr`K?-dKlF3||2 zMmi~?M31&+$Hc@iA%gHM*FlXpP8AB4U;@I%mwrNp@Q1{MLcz^Y7A#u!i;j-j(6Q?3 zYNEa)yOCy!EH4fGrbx}=r6~$F4*}}AX;Y}*E3MA)2j>1NX=Bp7XdB(o#R3l>0^}s9 z$H$ah9!VHGmG3Lw>gdx6!KDgXTk3w3AGHR8(<0y-pSRYRq1z-f`_~Bwd^0aYCDsZE z{Cej!1onZe(6Zmye&1doAn@A9y821+U$Qn??04y_vdbX8&1=4nE?!5d*Y0Idvfq7= zAO85uEU^`$x-M`8*FC28 zX!7cr7U5f}vu8gGuM7@s+MH1TiBF*<`hHv}ARy-zy=s{npFS~Qc}qXBcy>195QFD& zazMF1%hAzs4aGDEh)2+$?(Pe?r)|bo|2Sw$_v8*dK6J||K*C^p=sAOymT|=_Ch^qE zPku^F)#*D0uFd9&e&K#sx?f5@{{Gaf3sp#Ox}>x^MBC%QE4u<;u?qpC z%e|xmDKsa3o>u7Sbla@C`1+ZmELXsEW2G8DXLR*af)e1u!_OcnSu>HeN8BDY-d%}e zKP^|~0UZ;gk=**0%n4jozz7cx2(vT82xD>Gbku1tya${IwnTgm|$Ip-}AW&$dYXe{E7Ta-79WH zejF)#k@lR|^2gnEJ+fln{XAK1r7gL}nfj(8xpgX1C!Ys~goMzlobaymRDAsf0fE!f zRz@|#D*zxBq4E?-Nl6uzT+q4IQhGaw{=}~XrqpOYXQIxXJ9i?{R7G0)1vmzX!?qy@ z{8Bgl-AG5G-iszgR9e$TsnENCz+a5&f+XsMKIu_;jrhm%Uf0@Mp`NN}zC=;db zK3b)T44z@#W~BDhO}15!4_T-b&`w|hz$KB-d(k$UbQD$n`6OA(lcapPs2qHT8`}`MLGfR$1)-7V-G) zo|`#^ODTDm*$-!B^2WF-59ImWTc4356xo!Evo4G8y}4xV=KXeQx=_w+vL>a?$zSEY z*A!=;S3;bgbV@<~h$iiO#MOZ((uS{|-4XBW>2X$_nZDMXo;`TeH1=McyP1;fXiH3_ zXHJ89EJCC+V1whSK0i^x(T}swYcvw25_l-ahnLZ^t`t7QoqrZXmyzeX`sEg@x_EKg z1EqAY>@<$lh@j%7l6u|XohF8FuDt-^H*oTvi8a45#gST^uGTMZrnP;JimfR<_Vcc? z`}?G*^chxqrJ2=HhBsXz<^0^K$L24FZiFB~tv)>T=4liV^&v#TNpHIjqSVN5)73Nc zTJ4nM&ehQy_;M$Y85tujqn7O4fnKnz(Mq3u(tA5n~cR27dy5BcA!M%&Cm&5;{Vqai~qRIpJtJE~DFO5A<;+syJj=d*+>{BLB* zU~Syu<$1K=^U^+ZtO2^%vm)wfR$tpzB_?k=i4in{EKk}}v~64rO^wnXeUphH20uI^ z!gR@k)vBPrCNIb8%}PDz1Amp5PiJaXqJ0XhZu1Sb0Mi6tHYIm_(7M?smsLvf-8MmN z=h9=7>wLhPa%L4;HN5GMMV)Kz_D$`NiH`w{NdHo0Uu$oaXu7@Zd%lMK`h)F>)FzPL0LMaD9chRRU7#zPM(;cd*Uy>4E_1GVC^5ZOYnDL1-=N2<-S znE>F;noZJ>z+)a^S4sw;RP?Q-^JdfK2kYDOuBdOvl6Jc8AAK7DRwr&P?LFP5^iCDA zd*}G&U4aiaT|+tCW|4q$@E1M#|A3=^#;Cy0UH)o8tYx7=U`TwQ*l?(m4mCG1OeQq9m?d2S*yYtyi$$yaak3J+UjCHVs-r5@3GGP%PgxUR`J6P0V2Yl#xn zw%}`l*ZhP{6}z6q#Hg{$0*p>3Mlg=!!eiS#b>HZ|pwW+QzEK~4Y%|TG*>>pO{B87X z+q9NI6{IUS6q2t?b=EU6NLu5{bCynZ#<8fY0o=Kd4j#{qWseFS;V_!1fe`X7H03eHy+Z!N*jz!Rh(rDBW|4h1M<;VNtHWJ| z+-E{M*{vQrU5A&Kx)7<5V|#QL(Ytyrojx}bz8k-D4Xv9EMETKiG-rgiT^X`JD4s=Q zcr^!h&L>#I!_yffe%WnawBZ6Fl@B)TRt@)uZf)$hXgRtA(GslzZrw z3P_GwPSh%%$}%vz8?LJ7{Z$@>dk=13SkG>Xv-qt2$pc0F2eTFVW95?rl96cV{G;+UnwIV#_|}ae6ZnBoniFRkP+$O(%WLzX6jg(*Jo^;L|tP9@WhME8T%$>H;>$ z{vFo+--B~UtMc!JhpUVw{Ipoy@ybMyfvBL$+5XM`JvgF583^xAQyNuFGAT59kb0q+ zcjtzW>T$g<5D``6!R^Ts!NJk3QcCs{^sKuikL2{V%|atmP(F^YwUgH8_cZCK%?&@7 zqFA>)s}k;#Y;j{hL?2zv?D9`vO3@#$(5pY-opa^$C&6&PoKZ^s29G1bH6_u(CJN?t z@%pA4jtxg)ck;%lO=6z?x9eF7ybSZ>Qo?hY^}YrvrZkHaFNvD=WR-)@Bh=BZLNDV> z(Z=qXzLS&m4co)IOTUd=ZA`8Fg%a?kWKIS>2^=0CMnTS-MgJvFeyZ-PInB#YmxpwR zL-tvgdAPQ?JFO!EDl~Wbav!liv*~x!-=Y`Fu=tTtp)O|nSax-S5qm&hUz$s$5oc#$KyFOcdWe?kngE?0RWFxRv3-Wn!wY866i1(1&+No+ zCUteT$IU-e|Lvj5A@@r~1qa2vFSit$$AJeSw!Et*vPNl_iWUeh4LUHuUH`mm=c%NzL5A_{=q$4@ z@kqF$w6wG>#D`R8e%(|lWobf27F&s#7s?oC)$5j?BU06ImBaFb6X&zVlla~X>@vN; zeY7B(!LY>LhW|{E+@KFKUmZee7b_1Hz4DqjXP0Gx$0+Tyov95>yrInPkJMIv^aV%M ztLk^{L@R!5PRQ50>jgBNEu+;%UCg0@)8|6@GkD;TX+aX29v-8j+`)eN;{h9LIe#qm zh>%3yJ?yrbJXE>G(wyRsoYF8{QBUYsqdocf^Y2oIl4%^*5bRp>%PBE#^TfFt@b`Gj zX@7K*?-3Q}HSOgVRJ4CBxBq25v;CQX6<5{XX>E0v?U`0`p7eu)%$5Z6m}IekK-W5n ztXPJ9wc&9iHSGeIyS6Vl*jpZIR;P$N|4t+Sct%^r5D6!)sRyG60w=RR*lEz-TW zvCD7!>`G9|q(<i{yx^bM zZLT9`3EThTBbRrzIP|r8jMP2`iQb}iWQAQGYfA6Ml;Y$aK?@bEl+E@%OZcDF#rbng zbfGu!jtm^p{X#V_2xX>FSuyA|Crap&Q0V?*8M|MN(1C#a zYyA;M{Es*H7aUDTTgiE_yFDHR!~O7#wwCM8&e*yrQ=DH7)|E31vRy_deJ8_R$2*@3 z2pqjC_bD`jSU{~6Msy9`PW*9O8p(U7a=ydz&yAUX87cq#!*%FKZW~%BC#{7uS%TY~ z*1a1P5TIKp%T?z<&buRgIv5Rf{NLY*^_zk|5yEqBpsq`J?aSxvc3dy6vu-epyjcJ8 zqfEKsW#!?bU{mkZ`1b^}{Q2-*_NlDP1~;xXyDCjga_=doS_gOk@~E@&c1BOyZ~CU* zR;6pCdAxxPsf1CJ7lyqXMU?`Jk%LKFK#Fh*P?ystk_GN?+#su_f#|Uw9Yx+BEz?i$<-X(#!7ePKiJKwZO0CZHtTv>ozQ_QKN$MUX;`eKx?!s~|O^o_00;Bd+he z6La7y0dulhB7ANEK;-!!(p1)vJ%1#f59L-$nVUG|JXFIu%l%zl%;Pb7Si!}MUzmEH zskELk*2jreF8d+}{hW63*-lxS*W<}TA`;XMwY9Z7+4ogEDq7isJZOAAk8R&{iR z6Ye<^8v~{(Ji84mMj2PtQIX}|EfkBCxA*Qkd|PIS)wfx2=)|x&P?>}HJDxJ%VGMkF zJCEpHHaHR#=!d_VBS+t(>3XV8*2eI`UaRAOHAA9sTK1pK5WF4l7LkR2qpf-2+SM~N zllC5Av@@X#nw*v6CqPvJ5htnss||3N_b{K$Y|7~g&{5IVlLbTvZ3f#Z3ztt+ue{YH zmc+lacu^(T`p5!iNzu0UA^I(iGa0GyA5ip-TRl1x8Bb$kf_YKcmNn1%Pp_To)?Gem zPn*#4!uiY8#m2LtwrOZTN+T!t&9LOt5)7P*7O+QLPiODJLJ6epJ+!#M!(46ukncx4 z;kZ1zIo?$ESSN0rra=Z2<*915-P zY6qV)C++O$5{t!>XzY|q4f>PD(QIks=^@#INJ94{C@(_BnXa&0`&=&;T~xd5Q92`8 zzcdKES*<+WDA@`6FZ9<^!e zGgnQHX2JdaPvU@EKi#gl0(o!q^-a}dm+aUn*{uBxdw!_@Xtf(iN2ECC({+`~Qhs1< z^(3pt3TIYY^mAIKgpW&gmU|0L&$I@j!ym1NCb!ME)oWHa*sbDRZn3<^q>W|Myco{q z4`RaI$3tQw4Zn{fT4`fS+<_Wd-e>>BI>Qa%^GG1G|5uHW+qPCi{ciqIMS`(+OMP08 z7(T7J_3w4`zDV5v9-t@6FET!%^rd8cHt%zTE4sla_iVndmuNcoHbCe0TF$3o1*f#u zw6@5sh!0(1c7)c&9)4*x%{1Gy>m|jcGsa8eqR<53=;%o z)7fFy_f|2+PPMCJ@7?0HSWks1B>Jv57>0ll*nTMN%&Af+%Z76|L^U3-xfvJ{p*nL< z*t+i7T}E=NOS5)}V(MQS^zc6layS#j{(o4^q-0txApr6H?e#tFW##2_(*X=iG`)`a z5S97uVNZ?fB+1C8*a5%A!wmakt(4A1ZO)bcDSgJCo#+!0q@_Y`?^^UgKBIkAsXX`U z`|$G%T2;pqB+<+Ax;EPO$RX-6bGd*3*Hk2H%K(~U5#ixpmu6kIC~DoUi#Ko@VafY> zca;~4i)Qh?L*HhimNfX@PCkxWWmTh@=(yqv z3*bn(S9<~l+pRgn-d*O{b=pV3+R+)gd0qRf^zGBwkuRr7|IAP3kC^`D4=up*{Qs%( zw&x=H8U%Gga}*xTe}}rD*R1a1G8N@7G~3*4?ro5C8qEt)uM>SlH-B?fzu8U~U=3Sk zS9f_jZ>sd_Lg$P1syJz=9B?@wa)f9D{PEb#w%ws#Mci}Rp?xT^75ll}OlQR7V>!%l zeUZ(%t9*ECFd!_ga9rvCw_+G;b?Oq}c()OYzeOU9k7oN&La_@t}ZyiNMAZu8r9 zKCfQA%E`&`^#!|HvU-PvmDLvIT6Z*==5Fbz?Z}`<+{ORITEztfrn|sP?*s+8ym2V> z^ZXpmK(b#@ujY4|gAv52|qbPBq_JhrwqJq*(1h*buExL@Y zGWMYl0^MaoOKGp&WO;gfn-kLu4vqaWMj^SWG?U0S;v6@lQk9QlLugk*)qc^g!D$z& z9v|@Ler(AMM)5#OEfX*l{Pe zS5S9=mHVnMKwHo1ka59SmwyVJC=Z4|?59QlbZtfgP7m#~2^%hmPn= z9L^e&!7)yleB!Hl;iJAH|0sA3EOBM`?=i*;d_$;o*oI<$E;Ovax^QXYfKzKWlM%_- z<2;DXcr}vNoEG@k6YF>Fz1VOXEvIF9{Cwn*oL$RUB%eF_Su#TT`NWD4P`V0F&I9@8 zxnJkl*_=8hg7aQ36h(t|3C8 zgP$mxA~~OOZGMO@3g(ptNsD{(7bG$`>4X$cA4&TUDLi%h((?IQqqtV1e5+%^e0w}^ z(|kiU?c9HzjsG%5=YJUZ62^_>h%+CnVN0F2Kg+GQLM9LtifF9Ft0VdUHiZJQA@pdJ zzgksZ*O_vo^P*nH^*Ay-wZwe2jFQsqL%|!P^=V#T8ibi>%<&fAXi=K}O}29XNO7UK zRnz6dTQUPIGzv7&Bp+C|p8Fc7{-rLcOby?!TCoh*=dY?7uMQ4^ypGFdXH85Qxn5|Q zt}2Y!7T#Gm6gto@I6I^(xxw|wx(eKY(H~&*z47Ak*fl<7sq!;jGabVPN2ZHox-`6# zSVk%5wIYY?l;>&!2rFqaacY~k;m|)0+KoP+_oqcNl0gsPFyxG+AKId%-x1PTSs5zN z#BtX40QJT5qA&jgESfl6@ltWV$Sawo@norV#NxeSwdZ-=dpu-j)^FHfk>zEX_;xG|dl z0U&L$V@8*rFju-Sdo+2%$}*z$@ySrj2(N@bjtcAsuuMH)19J0@;O*(T8XMLKmrpdFDD%8atPQ^%3m?wv$R^_~M zH-XAFdv(EqR~~HU#(H;*qQ>g(0vIPi61@~c3`>Z>(=f7|>AlaQ#M#Roo#?JVCHYxr zIXCSraY_>KBKn_nwu#Hs7oB}G&!_WgUVXaFOG|XnT!*~dAQ>aqPo2U6n#P$uV5()S zbGO9StjZ($G~2umMH1%?X{Y|J^w2{XCqvc?H2fTm1-ZwxtmJq72!RTr$L{Exm zbrVV+55dzh8&D#(6o2jrkQLc7|9*1va#gD~=Mzn#!@E{({&HqT_&99r!p>|vjNALQ z{PyX;`4>J%%;(#cTPFu=hmr~et>NE;-6bL8)6MO=PoH>C;<)9!g$2AomhX5CaK?JV zL4WJ>uh14a|I&xw9$vg#pOdx|rzd0ii&K)5BkL#RRMFE$e(|nTZ+`z4#z8j(5$sgXRfmLcZ%{~v zBT~ehur2n$mPW%A1N_CWI{lR{);#;^+9H!Ho!84l<@uj(QS5R)zlTKven8uriJQJ_ zcb|nH8%B5&wmMwn1FhMvb2F@4J--S3dVCDdAEMFAk%!|AOEC+l_b`0q?;N>sIZiU$ z`S8x|cpvOS`@mE!*yS?Z@Y{mL@KlV$3o6rgSMKPbT09KtiJ)|xTiA&#c&;d@zA(T= zPpX%|lnUE8;qT`2=5#eNXflL0yc4!i=fU3CpRaYogSg{;lnRW3#JC9d5WZa9qE=Y7 zr<4Ryp6v-53oyfW@61s3m>Ooxjo7=kxmm*RA0W{PyEq)}<1J2=Ch+iKb>7_)w$C0( z!_V11YIf*;AZWZXY7ha3ZfxOP=M8bfar$YA&jFqWf`$pc`BHg_z|o691F!e%ET8al zK@c#wF$O>0AXW;g!Jp2zu8VhruNB7FS%2~88C<{9ryv^+V&bAgiTR}MSD6AWEh{T4 zO3v6DF^}P6j}Bb_q}aP!-?%D9ClW=q*OhCRDM3Xy0MB}UH4dvKy8GWM)dH0onnh&6V9RoAZz1oy)XiQDGP`_0C&xudYNaSC&Kf{b9<^PcX!QBw|k z?jw_I7aWptjOeC_gM)$!=Sl*%TZTL9p)csCI@3RIc723d0>UIef|tM`2hVq%EMB7x z_EhJ{VvG^NMAr8tI(qfE4e}07?RK|`-X7+d0h2p~0Q!hjF$G&rts|`QzP`SMjo4Y1 z#k??ES4v^Dz)KE4a5`Mq=<&C{xxO?&VxTQ|_Ky{tiBTma(3t`s+m5aZc!`iJ3nw}r zUK5;w!xTp1tkEvsh&@fI%U4!VdF(X;z16wi1bG|F!q^Htrn9yBW<-#{j_iTS78-_@ zsPr(z0pWD*uZq$Q%|7hA)}jB=4I=Yzz%z*YSJy;eCF+Uy!b_-5emKZ(r}44Ken^{K z+Dd_km*GKm2<9ik0ZW)~^^9y;aHXv&%}2|x8V`Jt*7W?8pWe=Kxi-Rej6G4qtH47y z?WO(;r>h|SLKQUpYGV38Txfw>Ihw-ZWUE7!SyWE!JhQ z#tM~*z)`b2F8W4T(VzET7i~)Y<;B$mSa{cbJZ;LT30N?Iy5qZ1Z%7rMn=l}xM zMu>AR3F>=4QIDQ``UKz+Fp6m;P+$}fGgz5L>@qqZ!)J@zRGaR&hBZn_p=9+xhO1MG z{{wOhQp6obSy)b;VC$t}Y#^2e=eZqbA%x95KZAz%5p#Tm|C|MYM$9uS`K(Id9majk zU=gQF5T1BqWB}J-is?BR@VnH-y(Sk&b}+9zVu0wk;am@w-^x@y;IQx@Jb7>%anpf$ z=!mi1nMvo0=~&JjR1)GX@xY)gPgfwu+$>uh24G$*;hOzw!J;Ffaw8NP2YkqDX3`BQ zK`T`LKTa6&^&Co2=RB&Zk|XdBTSmB@4_9f7N-)wMrBG0qsng$=`>oqzhm$EW3nPSA5Mh$_3*EC6HS2B#x-cB3dTp%TT& zS7IItn2`hXET*|eKRha6UZHak`X^uthvq?wA~|;xp_l>N5|qvwV;td?VE5}r5^!}- zfp@T6yTAwzOm(h3sqZyWE)TE0#rgB+uP<2*=YT$bN2s`tE*Yb=omBc&VVfrv2cFa| zB}~|D9!xI`+sI$|&;gL0p1|&**z~Gb;T>d9zli7KuK(EVcYn9}VHXz{yewESUK!Q) zZbU=`On&IKpeBf646qp(utu4sUNGB8oQNPzJtYO3VRZ3zMxsSN}4GZ7_rfuz6vWE_0;; z{`_`~Nr2P!w`Q|}+`3pcyt|TnXDAaXzy(wjlVr#e7fgPJSVn{>IuDAwxiGq}d*e>< z!p9S*X|B?%Gzp8tQ(!h;>k1a6ezsLg;D@>&@qhuWujvF!IzB8 zQ003gg)MmuA`FnAIBdYFAd1F5zVS~u7sKucx<4WYPHR&4t(4Smfu9>=5CJoZ^_V3g zO!dy!Fhs!>4nzruKMg8&{*=YMQiWZ{=w^e{@I-a?953R&lY$-~G84dyW*vx}B=z|B z{wL>s`xpm9{`Mw5Nr>R|sqsUy8c{U38q~Q)VH}0G9bCP^`=Frrho2VXEMpj;!$BTn zx}X;FuRYyTP+*%(9G?Mj!4e&VcT5R`T677jre($V1fbgCeTbPVxn`RQBQpB;-LxD5EOQJyCU6`Pok!CT{Hpi&j6bR?kNbo0lB zbM1K#ky9x3GO5zu#ONvP^H?>+OxTRo;F%?=wBzi<+3&z1cU<{O>+|SssdK#LFe;{|%KeVH zo+ltqz(c8w3@j)bR9E{p9hdya@m7}7lM3mbCRuy=5(h8(rm9Npfaw{Ch>-T`zN@|@ zfpRM82bO;-ub1lSRhcEOc5*tOKOW533v)h^ZoMIvIA$XT;g-r|JnT^2EAVh{d4$R= zigFpl0EvS7D8&*yuYsaF&dh{wM))_%;lhW%0_O3=#E0`(7KDazH=yce3>)J-#i66iyc-0 z3dE{#I0VN7+X{DWQ*f4r6#^4KCH$o*8&v_4I(e)*BLYrvt7}RnaPA261hGR46XVNE zN(KR}*y-cb6wCAH6IhOJFxkZhcg$i81UMy!tc2V$MkRvkZb3EKsuQ8?@aZip!t@W~ z__?F^S^pffJ?0Ka{wTndn4HBe_cF#K6r9fU_-*4S(cQ-I(?Z>Itty8^wy=I%1cmw> zt8mc*Wf0qCG;~c6s2+Itt^-D6i7Bv!^+eLQvok=}aaRzVn5WS;$hbsI<`HuUAp8@P zA9eU${!0b8f`nedDC=N57Wo-SlA(B01b=i4u|k)~L@bXdt}myF;51Ybh|-4{&gOJ` zHPk#bJ6@!UZ>Q2>ph<1(d;vyh=+Rp5jE)L1c!kb(BY^!l^Q^d2+3q3lcy3TH#-xT( z<%spVl!(Tz3knt_UKk@8o6cpp;f$LY71THLV*S2Sds_ngqU>KYX(^Se*_dtN#K9~8 z7x+#`p<)n8RmeMJ0OrV4sc5qWfx|>$1Ris-*nY4XNSa|VaKkj45O^Jfq$P3E3bVvj zXQL;W7X=>d#{3S;tP!-8))m>s452L;0mg%g5PJ_b3$Q`^IVRFOYc#bqcFE3Z3rlz) z)#!Z2?g3YrZN0&X+GzYaF8?0J;9Q7i{Lo%qm~ULbt}`)CdgCqfy0CS(BJ1Qv}Ziu}o_`ABtI&3H-#6{rh^TBy2)bo3z-F#=V1J<@HV0f$loHFe(UL zWeQb@^$*=^l{{Md4w)K5-`q~r>ujn7-0J*x6h7jX8(wyy6J2;)gbmiFk4dN2kHR9O zLJ{KI63P$JKH!4~no+i(elt7jDJOtFn>TksB~Y$Zaz~`GbtBG60=Vab7E}#uFGg{>AhDo&DRzIZ`E=)?V2HeZ+ntRE@skM*HF_Z>hjNLji1+^D z=mbpHsLSG7YypU|Lf@KbLglT&L@ag_%e+RF_S5`e`!vFX`0Rzjc4TZkvINR!k>G}D z+?~IOd;&-v46tnLCE74g5)(Wyl1G3XO@HnJEv~51e(@&KdoqJc$3&YKN)EaWp}Wq! z0s5+fz;n}=FHr=8Zin=R(1NaTLRSF;j9%ENTP-dpfutB=(uq`6p#^+$KeSj!Q!^W@ ziDA)Q$d?-0C8!Ol>7LE#D8+$OuUoecEyh?jA5YxJ3IRUv#atWas4?fy!OqpsK289i z!9wC>ORLowbpxAn+@7KbOz7}NowkhOftB0@0p7`qog>6Hn%qE9VhzqrzP1R%jeI@IIQ&)yGiiSowbe??>t%Uy{|2hmGIjREC#00}?dzUkU>7DRw~uqa7O+S_*BI)R_*f8hgdn4GFcgK^>6&;0+0?~P5dg%TI_!ACI}MWC z>cvKy(}|BV;NderTt-_IK(~OI@y40s5M5AQ2vSMRiKBx-G}_Qb;Iz~31qI1L17c>> z8EAm_gD#&#v7BJ^4YE0+6hDIK)by3f7$i%u1||@B7d|{ZHRZ^+gFa9|j;s5w1)0NK z^ghsc3?C{DJP9DMiZ(Z7h7mfoPk(F%-QA14gsg12RInThgqV}X7@jhIhy|;OUJMvN zIjA^rJA9+9?*8WEe}Ge5Un<;drcb0THiS+1(y9_4VmvQQ8G;Z5-q?DjW*(85*Z_Bk zs)@vPr?CG-;X0J>b8w2P8V>*dc3y|Od zfY9CBhblZdIZ1pl2|ja#)w2i(3dA%u>I_j{K?I0CIMKM;d*LWbDh(lmb_K>s3_!8~ z;B^=_pH`pRyd0nIfpgCGhIV%}ypG#fthtO?Qrlq(l1KJ_ndx(Ez z{%K&k@_SGm+7^hW0y8<=74zG(SxUs>p2$GKYYz;K1%>5L$vThMQF4UYRe3k<$ zdkeOX05{wb(e}G7YNabg^l-z#(p9kQXd5wLxdUdWfG8juq_`sCUkNmvC9W3Zi&Amw zG>B^6FS~56%CY=2ZEz$3?Szl&h{eZgJchOdZeF1axED zGaZ?)ad;uQ_U5t7$4orJCYHCj0`9HXhvBhZOB^c6$zT1Of8mJXXSO+UPpzya02Y1#a*56u3iN9l{b{@EW3o@B=M8PPZB@LL?9&_*}EXs1X&HqYWRPu zhSRj+`rkmC1b+R2HuSCBwp+h$A$<1VW7ST7i(!GJ-_iU!^#%{vc?N$3;kZFSlOUSp z!^j08BZSNa5*$dhBAQj1>8*&PXoVOwe(8!5A za`uoM#v~~LNeRfoWO4+XoJl8#^hsqzDkD-Ek;;fvMx-(#l@Y0oNM%GSBT^ZW%7|1( zq%tCv5vhzwWkf0?QW=rTh*UTe*qoj*aG?GDdgA0kl&j_evuIP6Zw@%>&Or8J$ diff --git a/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/7a35f487b2a5f3a9b9852a8659eeb4bd.png b/sdks/python/apache_beam/runners/interactive/testing/integration/goldens/Linux/7a35f487b2a5f3a9b9852a8659eeb4bd.png index b21d3b606a37a14eb51318e077ebd82bd55dc754..f3bf660dba0f5e51d311efcb66cf2757f6549cd9 100644 GIT binary patch literal 687768 zcmdSB2UJyAvo%U<3n(ZU5DY+rAd*p#927(Z1SBIlNKTS7M-&l7pe0IHqLOolg9MQz z0s@kA&N-d%YNP#yd+&eWKkn`E#(0h%4Vg0)wCkO}#PDL9Z6EQ!v4Q*72@h{NoSac17XEbO{kzL4!uru~NFcewtw zAbHj9Mp~379*1=q@p9=(m0NN?_#AktG4K+f-FQ6?dmxpZnj{sBI~c+1Ryb#+r;8?V z;w2PBw!XJUaqD@mHJqXr%+#sa#vQCgGR##UEC<|+YtTY_V|Ujkdb4%3Dx8;UgL%z6 zA1x1-T@7W%JV%C=k~UCcJJOS-H3E-Z?RBtEspHhGs+qOj#y3(q9c@Xk)J2QJ7?cc% z?T#L$tB0WKbeK4Ym_x^3x!CM)&M}Iv9ju^fuN&Th^&=vq zsr1~l$BW}#vtW+Trswy*L`)_Jk8}3S8D00el(rZ?)>q?t z{e|XTU@lUUl6x!i38G##_7kil(pFdZ0@|Oeb3X1i>xi30lJ{*R^qMgW^Nw~;oqurgZ3R5PeU-LvhPH;+Zu52cLgG{5jxjCB0Jfv&xyG4=1 z%-1Tg9UAvpI=VY6!&M4#f?HpOR@gKPKS;bd1s8*9IeCe9u-IBlT>NR*9P-OZwU^6M zzcFtN7&*$WXx~GaCmP+pnA|nzOHLD<#-o6qON|+Jod^WO+?ne#o%kFm?l0=Ke_7PC zR66u3->|DuuIZj<^#yK|&6SZ6)W!aI5s!Ewmu2kE2$`cJW=0IpyjpiXrj1(A@k>+F zX>o8UEcnZQk-F|5tp|$7QDbSlOSHnS-zpy?y&?;`7vcJCZ8^Mz! zuV@Xkx-B@3CvbW7#)?-TKkOu?yU1(B@3K7D5J0Ktu~ej0Y~?oUL-hIeg;3O{>}cl{ z_%aG{u+Ph4-oi{wO!(P%mf2+{Fjmn7g|sfGhOd2Li;!2z z6|?>68BtEvc7G>XEMapNI(TRR_D-r?3{$+T;X+T=*{k+d{_C@3h%P|b)43CVX|GgMXOv+m!-Y_to-eZ}K;(Bl3&70y!$^`7rZ37?UR zM@-e6!7p{vO7v!F?SSW9ggp(D@SGf+fiBwHD@(KJmD?*Zev$1TM#mydO-#<0Ri~$? zu%*^7^?pz^8moN{+XFTBoU&5a7B)1vYV}D0h&mr8k4R)Dc&;~vg`LB@`oe?nd(1s2 zPw?9A^F4mnrfRDPw$N_LhIH->4rbkjlwOr4BckjM0EY-T1Kp5QmS`pvm62o#iO&vJuz zqSW)WgAdC^^F;C4#`jqT)wdtLCR=`%Nj~#ofywUb80@yQ;`c(n!_=d&3wq#6iU*cH z#j91hJJH>_d-rag-$|_O4Q_5H!8p9uw#`tv9Bejlf-NmAvhwoV?p0AyQDBw=PV*`< zGR=~`^Es=FeFfm@b~vTwg?VzVEZF?tp4R$FVPYBU>fXv=fxiiV%Wm_+av_HB$Y7a3xcf5Wvu%n{`wvP8qoQuWx52uN#DBlBWP*haZ^WHPoZvr>l`%#&xN*8r|;GzXR^KmeZCke7 zToJplbp-?jL_D@h7j5eOPs6@a99=$kQjw5@vmCw`cv--Hx-|;mioCS+bt+Pi?L{{T zY!IQ(@yjY1+)Y~3!|(Tg|EO$aXxLwDT>;AtgZ&!C&6F!%Vl&i}ts`VVc@IAX0ULa- z)9UD_NF_@ZmG5t_8-Z<`@Wv=BDOF?ETe>sUb2ReC>3L!5l9aN|^E7B&M}4BWO$Fn= z!tz#VX9tV!f0@}}*R3qwW!j&S^j!T!1b3=lZ)W8*Jdl=_*35G5OqNc%8+b;XO{csN zJn*r*rbh^#U|wB5oCjZDU;F_Yzu{gLEQWK1XK`K`)|uSv>FVlgz~9x_eaLn`py5pB zYZB^2(lAp&y85JG(f)kn77A81fJc<#{(uWb*6Ppl4LK^WwIfcxeFn_9&ts*k_ucJh z%*v^b%Yy=|W?<9(MV200^HXR99eq?z(`|NTK!cOJa`mdor>DojJ844T5(SgfvaQ-W zQ}QqoKuiYMr)(q#b~03f?Yt}?48+5oN$7@6H(-nQ+M4eM`||=9KuRF$xtog;e~Rb2dgV&em!$)~n1~2xW+TQY zE#ViSQmu^Q60m!z>7Cw8+4>p+5Ax$4}K1udj9M7$B+b&do!NOJH<;-3L$E3 zZ4E9#1s?@>ZrUEt?)#Yk^08uDo{Lg&0k3_>(5COzULL5lI}jTZlaii=CtquZM_K63 zpzNeP<@*}~yH$Q~AVV(5&%$B>2nqnvKA_o{o0hx=wZ3HzGy4j(dNoTWBiZTm$WuHX ztBt;C8FcaHJc0~T<6)8z*a)uof~i`=l)Tav5oXgbf6{@a23eil3omg8VE=qF;ep-c zmpAt)kNiF*b~f*7r!nAGf@^4cF$Qy3G}xgp9Vt*r9td70_V?5$znpXwex0I_2thvh zyQDtygZ$jDJmGgjoZIKjiL;(?NqQU)XgB{tM^;!n&!*P=8L`~AE}NbCA*MvLAiv7E zVhxfq#;aGsUz2ENy*1igpIFsH4GDTq`US=a(**Y}3~g>q58Qn%?6~`Q?4}4!aJPl3 zYPUs^dzCTfEOONpilCZzC-Te#x}z7)ia#=OIr?lM3A3vl z)RlrUYVxzvyJjjTwl>-I;uL;r&2@W4jLT&4O1!v);}I-rUS3|xii4|HbIgpAGGuen z?V<^S)_$ll+)*N_Au;#%;SmODL67CZ16oghToZnmR?vO(NAAGN;O2%&aO1?9D=)Uv zc$eCRM#MdHlaoQ@iUhG@mHSQnQ1nfU-_H|XdleEFmvv3Xhq2Wg7NjPH#7%$KD*?MJ z9VyQ=g?E{J%Zf^ni1ILNJ&InzIqh|l`H~9=^wd2k#M%zzcc6^6g=~-6M9!|^FYV0O zR^V(0>%{$o)w4HUg~k0J7R>B=R$Jzj?e7eszk2Oh)76)ZcqPQg)A_y?-LQU9*HNuA zpfgaiyzgXr;B39*-1s_bB@Fj`*Yt{6ktSXzPM%VOO21@4ha+-R;n@CVWS0dbk!+n3 zbn%}Xq`UgvgZZ6$JI%$wREM-6T!ouk@r^z!`x*HVhLaUr5p%Xx)+Ivn2?7Hj44y&o zBB3KZHJj6IaiG(&6MmeNn2sTk!FP9WhF#o$K#H_LBhi=qiY;xP>56Ski4}XO;SodR zUoG~~P#I6J*O1@7*tUlY&8Lo4F0!b^-&IpnOY%FZ)Ya)r4jxpTI7`i=(SOY6W`Tpf zFL{wg&*sQhFk5r;R}O6Trh8TI!dAPg&dg|+QjYFQU=BOFYm zKdXpd?(5e3_&{IZbaDTS-5Dfw_73d{Z)Zo}*Abnf0mv>+j9+ut)GWupclh{G+<%uv zRR*0i-<^Ie{+jPSIXMb(m4lHmLa~4bL*AHf)Qzb7=L4n;_2}X1k9DU`lH&FVe2msQ ze-vZYRaMoqHQ9ar0|T>kDvHSymD5g%`?sSpGKyi=1E?jj$KRe@B0M-auT<_tA?}&G zc_n7ZrbH-$LyOLLw&P05Zq@x5To2^e`%Em3OVMn;8v7lugk#!FN>IBU_H=i8Ihyu* zBN>=QMMb~G(ffWg<2e=qPeDMLwoaoA%!UT>+U{hU5sddzs1`wB{o>}47`m;of+ zjA@(rEUJRhoo3f4Tz)~jvvitvXW8s+;} zm2n+XIMEkM)z{#$-cg0~aou;HN>O9ix)^vDUnV1K)Or-3u3Qn(^r+l?^&S2aUk^Ih ztss?Trpq(ExNn6?@+E~Pb( zviWB3lk>f{gUNa1iP96FX*snz(_A>KbaT4q+_z_|J?DtzVtAM_nWd%tc*#sBL7i}t z3tI@$bWUR23j3Q*#STa1H&mx8zF{cBJNJiJ^9Rp-%`Cw65V{$WNc1jKWwL-l=R=Lv zbc>3WRc7x(U_d~CBOSc;fuEQZC1n3tFIl#fA%p>(_1$uW0#vz1DvcN&xyn@BCK9*a z-T4&^Pa{t1{}d_z+c%|Q7re>-%^)N^oMP?_T;y@on@ly!WJ~_`M>izPeNTMV=ekpl z3{2aEDg^oYH-t`^Q>6K9?eRire9b4F@*NuC&0^Jr?Bfy-Ke4!Z^elR-@7NCH$>J{& zqA*SA_f(NFtDGr6)BE=NMqF?NN_Zc45Y1;<>|N&j1`Vt$+7J9|-oh{-aHq`FQyQ~PhU^(7oQFMMIw`Hn)d>*$m>@_ocu(HAFwwbq< z)*id}Xs`1IXTzf#6nfG!Ijqa*m0`HlHmBR;#Zu)o1yr%Sag@4};ssK_L-O-U%l+dL zAvi(%?^E_p3;m3_$x^|~xaV{nneXaQ)OKasU|7c@Hs`y$&|IzIaCd@mspCF$LPA=^ zr}a`~^rbyLDO6P*95%!?)eOKe#QoFP)_fR`A6HdXEz)$T(>@MgdEgdCP6bh!T)gw1 z52Ij~8WJ-l;0rT5yDoI6Gmr)OAk48|m>$9qUa6!JmLLPKJb9JoTl22>k}vtI%-zUT)uZ<^{ln>;pWl%oCS6-o^(CLXaKVT1;ln=7EMEARYz{o+( zBc5X0UgwF_gkJ3G#3tVeF^Umxd}~Uv_W6J>IoL}a`teqD`;CHpNPay=U>iTiK6wzP zgn2%I32%)SahXc*z7Q1!$?zpFyw@|^O1g95Dk|EgC*dlT>610&yo`i|+VqFGkK?;! zycz{^dE;C9@Z&$B%l~%I{MT=mzkdDtUhfkwX&ISUj>zasTkp{XGj-mKP3v8_qJtRm z6O{fvWH$c>*vVq8|K&@ocJ6EyfFj4m-L*f@t3=CyD|Kvpa*1B|aQ4UMWV9zHo0OGP zu`WYo3#zK&#sZ`_DU_QmZSU`S4+Z}+1qRrTFL3)UV~^#dgr$kse(W+y=qWuQBBOQY zkL^mWga5U=`*q(GA-DBM$(Fa?GB7eq{_<{)FJHcV zlRYLJ+H~&F5fWgI{;SKe={t*{${>|y{?DX{B)a^qq|NHjR+1~Z)9heXYe62 zH*;}OT{o`MZsKay?U6L~9K7(>3XAGEz0%<-N{3yh(siAi!V;Yl-AG`fc7 zz`A*oOGWJmD&7m%jwRCd9D69KrW=Rx8ya)<_+03(^3!WF{_dxoCT}}f=+;?(ymtpJ zPi)6=)2U~%j3&5IpUXF;5*;V(HlKDfCbd#@?z2xX3EkItp`D(Af?s!c3*&n6Y06n| z1YaIR-G8vslcg-Bc3LO8G=LiF53(Y)ruQ(MwL6&21a+3Gnqo}S@+n3XI66$iy z)I!dl4UMcTPd+JPzr$U~hA`DLGhHBqOGas4Tga#_E>7DajReKPB@K0A><8 zQQcyOJFxFtci6&- z(68n}AEj+2Z=OLN;d_n`uC(lWYjg8OT_c`Jib;NNvJS1ER3^SIcc)}zcH+N(=b@_~N?ImktIQvvC@_&HbNJ;m`wJMJ-)3$P@eR03h9tTi z-;Zhk+(X6^`tbMv4;`KWS8_^<*#0iMVr#K0$py`X->@=KQ=@7=q1IFvEIRzra)*Pt zD&BQ{j#%xd?T0i@Fi#sKw7azA3Juci;6>JR*;;Jx$MsaYbqN#Un{`ZU=0{sXe)PZF|sU+iGZ@a@KC;p%!0 z6+?HEuCq;Hmx8@e%Lg0nj4SMm9U={SO|qN{ukZa@32}Y1j>NqPxrL1ShP{>O`*)FJ z&f|Ck_m&8DZ|u~}`h#DQlAMav-K%@omCmM&}Y zY14Dukb8FJr(ZbSnJbWG@SVEA;ye4TdWa~1l8V$135k~p(MO5MPL>+vk|!Fy*rsK7 z z2a2UCe<6=CHtE37i%#=D4A#T=szej&#xyXv5QCCa;S9{e4O6|zc1kny)J)ae<-Icn z14|_5tN09mWS#!_3+Tkx{R#a4ULYTz)qdZj8^c_y`%<*t*X~4$UbEvBzkBy{Mov|p zcir94CjQyamuOI%r#Who@%Vm!F6VxVIh7OHKvQxbQ@s zLAle?nQ{x6x!sp%ZRJrJ9x^3=x#^UXr0MC^{04?C=i6*6jLq(mu*fmCi{h047~0Z` zvi(E|6!F<%3tqb&>tfs3vhgHF6n0A;KG|^~8-d$CptV%<@`R(5FF*IPYs$OByEgQiBr&8@Hvct+>m7kNpO82cUemZ4EI9)?57Yww_Dnb*KIc+xA~bmr>DP-)tz|7+3KT;R&t=UV4$9 zIND!qu2UM4Di`gG{ux$Eb6G_Yi~OJh!3aR@SA!Zx!&8S&{%@x`b>Oh-Kfol0;Ks9a z7r%BbY`x(?Kv|6#7Xz~W{|0adaPh+NyGnku>~R#$>9N-%ZPIjxw$Gk6o!!>Ayn0+dZZGj*ft9#_DXc)PW>R z;BAc;663u(U9GqS#+Arg=2f4&zcmjz&OT6WpBp|8<8u_pekNrMFu4x565A_9yI9 zfguAgG(Vd+;Zu>Geu7Trrti&T1=|~(&C-XP1hKWZX#KT8H32p| zBpqhhJzcg3Ylh%NQSu+d>fET>8QlAhiqCjha5pW8x)@(>v^1j!K$WTX!9#APnw_A zz>mL!k8osChzGAr9Qw1I39jApn&bNQ>yW#d-iU&lCFFL0WA3I34$2tPRhQs$wE9TK zbgLHmeDtfFM*w%FURMf%O}Pxmo&9v^kDG6)M~CaWPj%;1W!eDEKSMWLp!{2O@J#CyBL6c5-!++)&qa)70p&NeGT-MgBb9vk2nb(rS;$+QIM1q~@o091 z2|Nd%w%F*>!-5gLd>*zUY~7orMqhG>E{mUij-GEz;NJawDHXl7owy_V&(!}iVa82$ z;_=qXcjn(jM@K}3Q%wJy?hPeA6>uoN zyEX3;RO$62OF}~N)yH$s0@S?A-G`q_!m;#6)q?3Em%vJ+`rHR9{}!J<9+7YjyUI1H zM2_%>kKfBwcXA`9E)!4#KmL}Tb>YH=LwXSk2mYcLg=rLkUR22Gl{(D43Gg+4G9b6R z{qIThe>m^^zj-r-)h=*4Yk;y0Q$y>~T>kb9!%a)o$@MElJHJ@05ZD-=yST(70Djjk zpaMl)S-~!X6Z@_vgi_aCfc~GJzsw;G<3dIw=(+lBEG^+OZGOnp`H(>Y{1D)3YJS(* zKD;mm?$BUl#-Gk`v1t^R0!=L9;YcQVlk(B<)8{}B&vh3_C6gRFz)TVn(vKg_^*r~s zL|g(%6>n0iG?^x#1m@!_=yix;@DJYTXz1^b)}?fGMBL^(1)oSM*~OA2Cb3Hpj!?~s z!$+RHT0C;(+0e2^@p$OzIvehbf$v#AgoU%|6eCAiSAKFnc{;cBHP)JBB3qmeRt?@~ zq=rRg?f#guA98D4`dJD2VN~W4m8a+kk!*yn*l4*$_ZbPz2j(mfF~dZBq?gIpexksf zLZ7h=+iDT7=~fY!p$nZBG85oVMB&d59?L<8H1(|pCO6^u&PFPyy+g!f!-7t^sU={oAki4zQQ4t~4g0st&b|}7dllqXs+%e6rh5xa`WU71&hE|(Y zC_-xRpqb|IY}dfyle#p9# z{QCdE^d$9>EBxb-8p!G|t*x;lE}zoFtC>iRaKvp+u7OqFs`Yl z`xd?%XZ;vryP`TV@sDJboT|UXav;~>8M7^J2eMM<3ZsJ@p=nNS;n8x-I(&+E8}jc-nIo3@{-vcBW&9E7)l%Z+I9b0QosCKJ96-b*xVezRma!fU-t0kG3S+`&o( za5Z?1G7G0!64u)9=3QR8BMtydhHKr(f~rw)4jYlP%kbB(KOP#d5G%abRx*NK9pvUC zyTIT(3tOI(({{UZuVhG2C8OIfrFJ8TS`0U(K2mvOtU>F=Fnp$`NLE*OSg2rTP}ET4 zAF;umNz&|bNdbp|Cv&3AMU zXI@5BHl)o)uXMJ`w)#Ny%9R57rT&srRm^wJQppG-ZzTxXeo`OtgbEqm68X5XlA+q) z{Yb=10T=8WV`ZFFtRmoNVR0iR8-xzg*@xw%(~ z9;VI&yF`9J@!>1vDgHOH>qE%VXNlLiO{%+gnk@&M=G`~4>iDpg0;F&@@~D5a#C>zQ zH{ZJPHs4UbU03^?^h25f;l)=fg-pLz9s^Tx+>fzU$%x?Je=p^{+)oW$=_*jF!*SfU z^{WHJ@z){p)F05eK5qP~^NXfHpuci!9dAX|4u$%NL+=+U+6%0{e;V(7rYrfLuIlh< z(50mk@vfeX%pwOt;3*pMwz24t-}p#Bs%A`kkj#X;A;_KpBQtBT3nk;1Pc>3_c^s@* z6!p_RvxREiir$5jUe@@>yQ>B->gfa>yNQhKO2fUz1kCz|-wn=`UFmk5%w4g~FK}G! zOAZ0IY(L!_c-uUOfruN28PyZLCZA6$n{(VYB^AKp=2Y(YM-`ZwqPYuNWtp#Yo%=AE z63G-V>R^J4%@@-;H$NW?I84rCuB$Aro$dYsHDq)+uB@fRv3w28~MdB zx~7~Ug#r+#-5j*@QvXI%x@yUwaPkO+m$;4^7vKa@Os}4h^P=Wj6I02My~8M5266}S z_bPC1i2cqTQ-bTPeAoqEgy$XJiH;?OM@VVO>9QZ! zwW-TLBcyVz)8Ub?ipk%##~2$L8>Pw*uRax)oNke9i9G8A`Ro5>cApa9h-~p&eP3Ey z4xj`W%6N8NbW&u(!ScXo@nR}Y-Mos>6z`ty&L#yzjTRKOO`(;1}4%DgsbJ0QIb zqPkA+n8TrT$Z$P5QEO3z)O9MJQ&pvqJT30K(^k!QXgg3aztth!UX444*2dRKiMi=m_448we})B_5LA6faO5H=^E?cGST$KRNNd z@Imom-PZ^H^&Ku>yz}!r`p96OpLw&ub!V~LkLIw{jDo1w;F`nle)eNfntO>$|5lD} z)gf4`nUcfO;$84#mDF>QG%WvD!j*@`9*{R3YEwga8-J-H*9ai0B)S-_b0 z(DgBWAA}q$BWV0`#fnXT;m08xq-GrcyDA_11C7FyFBL(dCsg;}jJh9L{Sj0o?NC_? zGn<~?oUh#)z0jLOE9Q-Xf=o-OU{|YnE8s8 zN&B*85WidnLJGC==^TE7rS3d_DY1-(&Th-rjW8g=W^AMJQ>Fvckh=f=Wf# zt#0*`5Xh}oJ3pqgNOwNRC4`4>AsPxOHb)9Rq283kkR(#07@cBMu~eLqU7lW33y;po zz@R<8%^7V275G{uHeMTTd?005&~xloRW~o{=L2O;v}N}34&9p<<(?6qY<{E+x)(da zI8HPspL{&ZG@d!sI>MfkBXS{BK@z=9P|hgO+9xW-Z>?ws~b;F&%wcg?;NXpF=F4s zZc{RVg3DH2kqTPuNNUt^8FCt}@u{(ZDpzVAvjEyt3O4D9=6@eh%&l#9 zUI983>wo%6xyyzmC6EGQV>^3`ZyeD4}SD!kWN=j zx_cq~Auq?~H9U+KmJKTP6-Hb7?xh@~^Bq{yg<(Ck&bn^~^@It3{<+#I%V|AaDX_|6 z-d{KerP@bF+@Os1f2seGPg5qg15pkf=|j0T#kFw|5wKTc-8QD7x_Z~|l8q~3g%DRj zQzYva_pdr0_?;8S7#4zG?WOI(j!#Ues&1L)VqWzrzIh(LG>ljUC^^bOQv>I+)Li;C zVa&?ac-%ovk|b{RnQ9RP5=6fMRljU)+YSzh8Y!b|4kSQDQZ#Ol07yaszvG-9VxDnO zQ4iXS%F4>23YTCS5da<4ZrgpPBd!xC@ktPPT%q{+a}YHmB_ph!)vos3gDr4@|Awf; zbSq31lu$2>h-+#_qxQ<37UJEeqh8Rso7MUr2PxY+CtEI{Y!j1Ia@hZ8kxJ^#4kI_d@MW zQR0Vks`o8g-(GJFrlk~rO(W7T_Qw)z&e;Jwn-vS5KnemhF& zIRBj)FVilv6N{=*Qd8?mmJW@TPgBc)>cD08lEL+F5@g5e5E0h=_m3{aZl)cHk{}o2 zifMxhFarSu$fkgziIty$U-q|`DlB_B3i<}Y`k>%{Y6OSEZ6q8SpcMCn;gt;9x4^x9 z$zeWw=Nw^ys!A!D;cV@_guHwZc_P$q;co9ib;1wXYg4>X3`Q=Vw0Ce{Sul2a3daeldyjIM!pzm^K`Y#caxPJ|&=k)$V=*)!^~&ofRJ;B zRs^?#!XLn-LSEwy(yJ4s%q${-q zRb%--I42ol{4m0rcVK&I0DZ7rfk+&m%y=x9F$oBifes@P)EtBK#p{OkAg=%sVj89;!2BXz zy>-dZv~vX-yFCB_5Kdh|Ja7=nXWp3%CG_I{Agwv?<{j6$TM_vGDheZ~2ks!3>-u%Y zh$#?6!eXZ+C$s5RYN&R@&Jw|6HssSb(0FXlp)?LOaGZHpRaLof+&E!{Zl%DVktsNa zx4nnsrrfiod~5Z1PN+Pr>fqUkQLu>G=p0VIrj}!~O1E{}^81H8&>ie;%%E3mNpy2c zz|m8C$VY5 zbD+flp^R<$Tq?LsF!>=+Chob|fhHxUtq0T&H6gHMAblx)yP<)|azGRTGOu232RTr3 z06>D^SRw0~vL|%nh+cBUy7TC_krCU+b^~ROAg%!)rcq!bhdZ9#z$a&AwJ66ow7J$u zd*E3Oku`Ejz3^Z3VYp}N(jmy&=b*5b;6I%!M1FLotLJd|e);kReAauEG&o+EK9nqS z41=s7#8+Tznwpw)bNINq9iVss^jtF1(#&!Xadiu=t*tfNE!u0G?|C5NkXX^NvdU(3 zpPyYW+oaZM9>(ef{-p~$v!mO*1Y)np;MGMFY#{_7VF+9$1SQR@2SCJ7PJSYy-J})1 zBPpp08xAg0rfFl7O`f<~^0}<64C+New6zCl6NK0xZ9PXp(T(Wn79N?J2B6|)m^e7( z6~hcu8SCrog}1&dBg*O8=OkXDdVtzqoT7GrIYZWlUz%wc;uth~{Jw;92$4d=+=53d zvK<*lC-0`77Ka4EXsS6J7$@-M*Xm?^$px!)!G*3gHZ}%d7Px^A0lo>+E_ z_V@Ez!n78()a|KusPm5tUyFl(|13?I|SDef6-gp2foX-W&h!H^*jwWzESUtn@ z?PfeD=$1i*d*m5XvF=NL@*>X%hu3>x$8>;u&qm&a`~x1Q``zGB6SnN8Gs)9;iTH zWq{3|aPEO$!z}F7oA#4m02s!LdI>TzHt2JKx^M=x*p~ow=CU5$WySTPaW|djzk@Ok zqGmQ=H1MAgoR4RI1ABy>Kq!6>a4VQ#GkdQ*nB>(~#^;a6@j^=vvZBcY8E4YqMdN@``WjGL1?M+z(@}Juk%kMQHo`vcG5b1$`nL%rLc0qv> zNaoccZrbx-W>wF&0?$1vb1DUMzLe^JBA1(_%}2Dh010)g+>hz5LMA0EK+|OV{evP+ zuqkM#U@i?nvMcqP3h|M$dur8S^N=bWK#43GwhL&j&5HUFyMBF%j|2P}h>`;+IblJU zTR73ND0_#Rl$BYfZEp~vseV*4p6g5jW$nub!@1$AgQQ?l&GFAL=dT`e|JXiBSO?3J zCKs~}0PotXPH;*f>cu4Lu>lf28rtMI5y+RVS=7Ab45G&ViM33v5_ymSmvvM)&aH3t z>PbmSQHt)o(aoXde>}QA)smE$7!neKEDqQNs7b|%6#;h6&LW8!85P$bSFT*)I$I7y zzZhuJ6vEi6hpHkUhkPstQm>TcW9Sp#>i$3^3J^@+-Y|ZJw^@_h0Ga9M#zs9**X6Up zW;KQ_0-97b{w2<5945uOz@$wOG_|9xvMF@oZd5F+yfJN7yhLYek$wyCRm&)=5pyo5PiuiwNyO7 zF;&P1d#HMEI)>A|dVRVrM$c_3{20mk@7X#B^walqKx_(Vh*HP%jddOrmBCIV%DSh; zU!it$7}8(a%Zv~&p2#dKnQ&_F<0+K(PLhH*z5sMWirzdKTqs;>v<6H6`CNF@(yKHK zA;K<=ic`F;xcp{-qV-sN2=YQR_vb6&?Cyv6wbR~rqWVI5>`=w`-w-v4*Iohmyt4av zIQzpdCa3h2M3m9t@g>Wt?79cj|N8P_(KXAtf4R?NYQGy&hn0~m{-vd*AnNf?eh&X< zfzyqhDZiI0K6OZU4VG{~3gD&&EsFe5y%3F#5}kS-75 z6S5udcHy&gEaaLgR>s4m1P`LZ$*CZl{u~T6A+%9}{vbfD@Ox}$qRP7Fpv4HE@k0(f zq7=P?8gCL5Kd1x~5GM%j9hwj6f+!pY+>P8@uCbR)@wt6g2`HQ!bArc9qBqC9 zcSx?)1zUX@aiF{7X1#C&^Zh)Jqd+R_lzRw6kyYP1fT#75Ht41S?gci6Ddc~!lOdN{ z*xDbksbHt0d%qe&-wXTD9O?~27-eA;(9i~w;>ee0nIbek|4AnSixV9Z0%uoZMb-nn zfbzd|gIvD@;xE`G(qaPK1RAzFG(t#l9Y6js0%=|Wv>XHlQ%HRw2JVAWw|aLx01`HE z6+`X|S&-(iO_3!hvXGq|f*lS~0l_e~Bh_MW^v++f@tvOYe7QG zfJ36m6V-EcL7OdRH+}~$EKCDjlO=9jXS3tPy~rEY!{5#sb)S&?E^C4)cHoYl-CKG`>KgAz(fWGzbQ^7Zg)K z8eax|gI;N;g=Mjs&7$lph9KYoit=F$7TMN-C-Ef*Fzv*D8R=Yf1$u)3D~R1+$nkMH zA-uWKE=EEJ^%nr#USZBP;qRtH#QUVr@4oqgd>lIQ;M$Mh@lCu2!DO)~YXE@IK#^sM z{ge!yZ~F|K9FCE2R+3XtYypFC(5nqZ96%7J`)?lrt58#`KhTPodNzHE8`u?qW(Y3< zXYQbP+}#B$-KEup?iUah5n37)`+)W-%&-uHL?K7O+0f(1bRfw(!FKxg@bL!B&*gZk zu=Gc>gO*-B;yO548Hosk8#j=31dt6tTN$G831JH^D(HM76L_}^SnJHpO!~-kWvOf} zfI*jga}v}t+VGMlFM-iCDAELjragOBdGZ95d$qQ;L9Sp9kOPO=fVM8o(w*R8FaT7L zrWvU*Hv0z+mOLPe5)sjfn1cBqgxouYGijVz{bXENm3wcsV_5r~J34W`ILpX~nuq?UYlk~}~N2+y*4Y;UI(4IQ)eMLItJAg;X zT#wV8y3GF#9Pk75-E&5ZKk}d;7zDEL&fMIbIPtSjf5ygwKoB}606+X2vKR-j1UcpN zEOGL^fW`r1P?mk9$D4f{9OHxD0g%Qc$Qu%o;ZnOK)TTdM3n1VMXaoV&Hk@K$(CN>$ z(5|0`k4xf~Dd*@#^PiJ@S`aCN<y)-T*?OCx=QPG!9Z%5WZX9uOSm zoDDQehCxk4}q6m5evE97cJ+m>lwT}t@{P}bLk~5v}kd68|*tEK9oMb(GREvjB zMlcTGAZ)BHA!rXalYoj355MWUsvpyq+F_QM2*Ts@0VzN-iTfLBqVJ`6GM!7Rkt)8n zxl{Q?Z2G4oi60QeBc{t5+xgXiTpViiflP*=kn?5Y3yE=_#yLql{>!)$SLT$|R5-dZ zA1t-I1ihNTEkZ7K0KDdu-fhJ&Aa~%<;ns^t=wTu)E#SHP7`jC`J8wh&X3?9ygt`}D zEU5?`gg7J>d2I%FAx9PiPaTQt)T{u!+1jC&1hnq@1(oEwHv$c^q~+5bfK!90%x2$nrB)(h z3jF;LoMk~e(bCiel0Z$I6mZp8YwJ7>o{Md)IhBT7R2Z_U6-lel0`Y&DqbmwfYu*b!5Tf;WSXmrID{!p|K$F32dLSFZ z1DF6H-@TAYyccU|q;nZ7*-4#B5)0Vul$|+xXz<;;M6bb-FmrOkzH*063qc6+zk+eV z#sSPZMS6o7^8ivJnL$|`U&!4g^X~L+U_mc*&^#0c+H#|zq2XhCIgrx$m8Xppd*Fbo zbdj7gn$MO4A+~^k*qx5Cg@P@J46hRE{y2f{cvn`}WX1!PBVI5L2!;z=%aCh8)`95R z2fzizDGEguLuE2qT1WU| z`H^d7FFuLMs3Hyu4rt(fswAwt8k{W1ZC9YVPFWZ&3*-#<<>amxvG^xSTLBib6P#1g-!e6)Y+cq&P^naB1gU zuQ4(*3JFzw2_*8k0?^4BGJV{HqVJ2w3CKh|N0$L01>T(i?=Ecm^)kc>V1o<#7VXP` zc!8@F!&Si!e(zs^Tk7N)HXt#MmR8KL-d}%!Y#WJaX+Y-Mf~n)qeuG;b!_^PL+yJ@a z0VkKR)*_%&sLg^V1wfNv;e-uofrI;v4w@PriwK*JdypJ}`!bwo^x~5~Sn$26f-KoP z66yp>YcqMN$=@=kunZA{cA*eNI~u1*ewTj z77SyKu-SqPYe3$F1^xWa(ObvPHqrP& zFhCm1K|q1iw+nF4E*X1|DDv3FiwL(5p%?KWsWobZJbVNyTyIDbc{xF|Ks63I=Kn`o z+rJkZIQ{1HDbicm(P?rnWC@CJhGiS}xf_V_9w zx8X$smSRfpSRn%}Mt1N^+5yS}`*pRv%Z4p9wm-IA4@sM9(Lsrl58!XX2N?QmHgG--!EH56lXNlh{}v{Un;F7^Rk}$WJ6V|FXd!kHf_s z$1W?pPPbD(&+Hyf``gw>G~x zAJ6+N+xHbWp_ZmaBT_0E8S&CsNy8_zGx7G|YDP$W+=gY8)#a(%O+IuxGr^3{1j-lQ zajBdociFnOvo;b-RP5`J7Cs2st`!#*&_)-(#_0lXfs;nD$D_CST{N4_d7{BX>NKC2 zdeJha34KQ4bQ#e20Jw2DnA=Lv$cBaxipdKslxDNdMoq&xNt>x!nmTST2lC3T#r@4B zzWwPoM>BuJai{FrC56RPLmVu{!6<$UUJIhFbkxg%rXnapIv<+N_KKzd!=-4fGlNIV z%6+xWKcF(KSg+@fOQrlhC>8PADUrI%=e&p=`=0SEILO+~y_3AX$;;-!_ zj2K{pk-i&rq=GWdLf8h6t7I@D!*wVl0A&E&Ejkh#AWellh6yL|2WZyJfs8tA=Lm;$ z;U+CW6HGz)ger((IM>_Tz+X>v2i8qgxao_Es zz?<`6Q|dBCbk< z$P&^>OCu{GEwOZWcZaAfh=fQ>OD-%d9a19FAkEStARP(_(%iGY_cwFr{&8pSj5_bU z1Lynwp7Z4Ad7jUC9Z_L5vb$N2#o91l8p^0$Hk|#9zae4od)j0A`0Vp99hHlO<{3A; zAI;5<;Qz7HUhJKhUcRK}|FXSw1LZ^p{}XuB+ff_1Wqi~*l;+RuxaIR}LnUWS?9a*y zA0@Gx&5T+ZOjX9I%*#yk^btbxG+C{aRGRR$l;`9O^98q{U1pH!Nv!qa$*k_uMJs=&2k0agI=V~lQPM{~gLVmq*=GynhwPrzmLz+^T9 zD%6X&a{%E1NC+{lm=9nD1jyz~0A?x(Q%e9+HHaDrL& zmNOr?tm=Uoj=7mekByZ(&h~-W*i`13iN?PT(?DtTPNtvaj43G2j!OBXAgC+U5eZPXjawH_@&QR{wIWV z?vt@wdLni7uP?dd2D+1JUnJkJR5|bcef&+d=pw*>IrG^G|HpHSfs5G&NOdemVV>5$r#nS>`gXtv_%_$l$`Qd6!}8CT#9yCm=PH>oRJ~S2iMGway=UAMlT(j6t+Uk$VwHnNU{1k;i zzcfZhG7S_uy&c@Bknm@(`Sk$tRt$y=v-8e-d@jARQRta*!Z@M9m@eVI{Bx}&_@0~+ z6fRv|_btg&a5YspvH#kc>d?5--g%i$mqgZW(d#dwYo-x{F>7)q2}F#z@eu0rGqZ420XDUEvGBc0m6F+k}RP z$ZSsnJ8+#MW^E9dG-z)C045IP22VN7R0pmER57FyEkRyT^LWGU?n({U572JYgrmrX zg0Or*;Rcs+>42WF5QgVgca?ia(M&xCYeNgSk zOcm69xKGoZ`X%1Cx1sr2&xQu&DhF85>XIe-mnFcPS)VfJh|CFP&0hlqM{)HZJv%y^ zlev(;*}i}v#b2?A9c8c|209aROHlj6^HcQ!F8dOIIkVo8H(d)R-I>yNrq8!d)a#Ty z!JlRHMp#9Sv|-%ya!S5#Czg5i1Sf4H=!2D?0UH_?=_?hNsSj6bBwqICv)IjUkX;(0 z^o@QWw}_TFo&lc;5Bu`KE+ukmb!7(%2ZUok?;H|sAcDRHrX!)+9>u3%Zr_ay}B{`+oT1CGPR++aKGFY4^I( zD(u&T4lU^_f|`2YfjKG*|MyV1`9&v(P9Wyd1@(pPUyxBzA*2$l@%)Np6+2ay#>DR|GK)Ltk zf2#)9OgaF;tDJOrZ5{}d2GvvGmMnCM595I`nR2KBz;xH;2(TN#g5YN?&fC6jO;TX` z?r8Wl?$K({6Tk+IgkI!yDBdL}C;uGSdix15Gha6kXaPsKsOTx^kb#JKfF>dwQDrRS z17SBtc6R%(ubMCYh`+J zf9!8J`qH>o3wt4yvJ{78}Hqe01YQ2d97 zfetIw@4X`&O+mAHR=`UQ3c&@3&Ba2yp@+=mD1h#u(I77c_Cw`oF982QJ#THz2G#}) zeFzj=ba_6W0}Mk22cY{M1M&6I_U3^f3?PoLnJ&C5hJ55#?+`wqN(tq1H)>}_LIU)I z`V(1^1*V01K7AhkHBhs*p4D3%s@2xmcZFI%6W<2yrD-2F`C)@^7mZH2?bYHGaQ*sb zwFu}G^qzbueG*cQu|zi(m5_O3tbLpW z-ZkbC*w=BvRmnqSr@1YTyDn1#AJ=E-z56L9vk$+A7o)BmB4-i z3^OR7)7i#`s@|cUGA9J8GJpJd+)i4X6rL;Umcm3L7W2N{gbd!u_j{i69`%Z`5ocGo z&fmO;=M39j*0s%(l}4hw%`AWqE=+M_!qpGh5FhtJ)u>*Ti7An>Y;w4e_-b@>5rT&h z##>y{;dwKINW3TaKD=@FMUvIQ#ja>9Fhv~_F5l`eKYOk#YJ1otl)-SN=S9I|?uL^^ zq1g&(wnEi{x}f??&93ITJne5!dZs>az#+yecDqIE_KddKOOB$=#2n?@M?Iqj!4L`f z(8Q-G3vPdnXe#O|@yk*pc0=qG-1{PGxiVi~Ho_v3ZCkPMlgRS@`lcgE9JUp1i~*rP zNXU=xAA*<{*B+uYE)8N@Fj)lnt%ENMLAe72zUR+Agg`;lg$;k=wDZkYRx!uGtTV%> zd6)31dDPFp0P+VtZ3?C1pkInnLgFQ8Lwfx-ky9Ulo3)@$2!@9NzzJZc)6z~*YMRV? z_ICKYt?gKGa<@fK&>RNTZ>nuaIxa&z>SUgs4h>pnTX`UWYBUmE&gLUF+o?D+9~?Ch z4sWf#&8#k+Vvv?`(KX0r1*6+jdX~o@E&I(?)3B^I_8CK8MyUy;DhUa3R zXTQ)#S&DK#GSp;Ioujf^-Y@m`i60EFN7jQA%iz0_Gj+&rB_Qt{`S8ePWlx)5tvo;X z{b9Os1F>V#dE4+}hFSRgW>o|>3Cwwuod7lsDhhS8 zJK!w*#WDbF07^cfVsff21gLOLJOp|c=IMgQ2SHd1=#)=D1!!?$7FQhs9gA9^@d2Pa z0NVN!Eeob9FXFaS`c^=60=nvhz^w)#&`od7SqBvSz982!H#ZmcSo@7D6o3{M&?CWS zMIN4=Cgh51Uj(e#pn|4TX7ak*0+j!kKva$E1O(AX!hJPMzB?r8E!ot5{9HKQ>V7pQg>^2{f-V$Z$C! zaDG;xvWwHZV3LCe-4h|%AJ}-h-HbXvnCke8gbGGiT2{*<`pQckbdG%fPDih%1ummP zTI`Cb^$j6B`$?qYo3#%|8*3#cw-hB))N7TKFKOtxh#U=T>+}2?*MD@+yiB(vgngIY z{s`mD=Q!_k7|p9HdAXMBc@i#pW-puSx#Z+{-tpG|0he(makYWdeD>ozzJWbxd2Q0V z<(jCa`({ybNn3$yD~Un0y}QnxJ9`2}6HBd*Li^A z2ChrQBz6y*ib%UbW{#UVyiyJ@0EHct16*zrmJU)9yTTJ)Gr%a7HJi=AzMiS)sGzu`;IhY|pe zqpuyz_glIV=X_!EOWXd?nPM=vLCgAursdM^Y_#V?wG~OELbmGLl+?pCYVpr?mpf}M zoHMk?t-mhxj*eTrl+hQbhc%Bm#e!UOZx9HSQkfeZwYTKWKPdm`5hybHCsV2LIa!=j zf1J3nL7Dsa_)L9)VqQy!amt66u7V?zjW1{PFUJGFWgD8K9BtJkoxXoabPo7j+r?Ss zptC&pT3NeT^aMMmq0Kx#MpYG_?Q|@@-IA)8czox|U0A}zm2Kg=UjT&JP(D&IW(XL^ z_uRVR@?5xo+vu|HD`5&$1VASN;B0Xf&PDt--YQV-3Asg%yJ6$H${=#Dxa*cb2qb@k zoc961|L#bh4T5ovGV+iLxo5^)FBMNW!)5MVxLtmGgi!QUn1P(^4X*HHs$QIDMPEgE ze!D*D&0#&L0ZLo*7g2Uee{*tv)nxjbtvr?-_yTj$V>TF5*fHfEwI`ij?kGk$kNW(c zN1S#Nc`m6o{ojhodEMU3VlX^4~wuO&l-QbqB%k2Qse5I2LXXLdXrO z6Stunzojh9iL#F5CCG%sJviHUS@(*l$<7;A({n>2#?Yn-Anpt@Uw8iX{@2}gy>JkB z+V;Nm|Cy+j8*(#8lTYp5+a?#gy}_AGiO6k#xq@5w;1 zua)({zO5&U@_;4PlJ9H_{h9Q9`0{l8!MNai4^AoN0B7%NpZx-A1&Z>>-sgnY@*>UZ z9s%2(Nd>gp1x33fdW&`d{ot|Ifro+CK3w&SDZ`+;7_36T^b!+q`&KzNC5Dy<`_P4~Y zA{r|e4_J*d*?Pk}Q9gm_YzPd=#%ff`h*VW0jaD@u>{dyVEgfI;)N4}78MjlC4UF%$ z_C9eSE>+oE6OgSOvf-svdK5RAJ7gz7b^9)uO*Zg?5Ec9&(M=SWh{mpb@2WsqPYzcG zYBgB6qq$1A4!dYe+1TM~NHkkG?hi{W+u4&dMP&9(7?QCtJloF_+~gvAk18s62qOO+ zl#he-*})|W{T721MD$YydXZ}Hb`#rt6wMHZ)jpVR7Ro=k`iRRUjQi77w+7r7FMHIb zb47Jp|1Qkf64MUF#wBBNSA8n>#p(nnx3v(iS2};8(QI%sH$qTY|A2&bm_vD`N28Jz1nu&741sCD@Ed77-C$sN0wN zLPkTwUNH5UI*_xc$@JWx-u(VNYZN`cP}V@BVr`cwII<((r`c~=55R*lG3+LL*~_pF z0(m$DfeuX)|tXNL=ZP*!aX<*#J7j@>NPfq ze@UNxwXSMLpMyPh?Gq?LJhfil7QvjL&0+HqG=Khe5VuZ-_22@ zGz}J>Cwp5NVhvk|Xjn?l3o?@E0rs@Rk@Y#Twi~^Aus4}_g(ZK8Gq4wyR zLDc}OKhnr2kPFpILv9|&b27(<&ZFE{UZsNF*g(Hs+DO8d&#A>wD2?q*}s&cB(Eo5y(Mn zW0NBZf?>mXU?!s{;_ItD6$AoQx4}G0ogh=^f{5Rv)!Jus25*_i$<~K^K5iWf&C*8q zE%bo)p8YC43>bBgO_hn1DH?Ib;J)E5K0&;91;?u8vX9%|-upDSOa|u$UzX8I@ZklJ}?iaVh48q2+x)%6$x`H z<`jrTX=#yQRICR9s_;OA3TTNCSe^Dby0gj2yW`!+m`34TC8Zl0o1Br?6HZleA7#fF zWmF`vaoZ|bt#o{|N-s*QXvpU3jWskH9Zt_4&e3a>IlgpIFGl;elQh;&QwD*2G7#gw zhGbyY)@E-(JZZ3N(&NG2)0AIabOO5W03l-W)}~B$ax9ltzQ4b^g&Kj^ z%E`x8QGv(tt>1o>GvvTa9VOe;?hsaO!)b+SXXKB4p*gt_yUpcXGP~*FYrs`Ioik)( zflwon(IO41eU^;Py$NGCN^P}wUmI~Z+j5eL2~Z<*){M6p+$8b~vVNYtO$^=ua`jpd zxo%>UQ?%%SmAGGrFB{hN(QJC+gZksXSVmh$<@g~A^O}!J;0(~od`t;j%7zx z+VshGTR>wOH4&cvBq+Il&n{nL@G92se)}}LPPM(u6`~KgN;?;b<(-4Kc16@HHFXas z(rPN#SLy|_?X^cJZxiJ3kTie^xUOg*Q!+86z+YHTKX|KRdGWFfEVJD|xZOWrZo-oS zLUwaIrVhp@(Hmp%VLNHoj?-tfUDoi6@&%CM{HoT1G#S76mf`B2j{4G^F<* znAb?lV4sKCX=xkRrsNKV4c|V9sfsP}OM}u24iriz>~z(nL_KQgO?Dh$-%lQd1Mn?S zrIvj<#s4fVdt_siNw!@&=@>`6ag&bw6O9^4yJ@X;x)A$AiA|-xy(#!)EuC$oNMQUV zjdrD8LWW~xl>Me%hw;QCek#K#AMi(GlK{n(=#*A zros-7%a2N@a+wChhi$(1ZOT*0)2N4hpy9Hv5mzN_AxtO99Eda55QwER7a{eWQ0J5= zmp$N}2V)mc@m^n94(VNYL0KxIDo00Yto5|kych|yhMrsHu$Jh-4F&}BtvnxpcFy`i z#*~%7A+D$ER~0K^!2Cc6=7z+YYD-UoLN#+kMX^L}rva1JyFQnVsyDdffWQ0ts%!uA ztV-(aOTpHcf2uaubfX0{SEctVDU^BC$iSTm=me93xPo#6_&gkz|HxBQ=HPq*v+>UJ zvmtS9<@eR6w_qgE<1Ymb)vzp4UEvXevicl6JeFU`!Rk^bPkn#vpEPPUbbWVEe_ngx z$wtB#y3a=$IYg%D5ryx*85)|Kl9#q`JS0I|j%ak>kw_-6n2U1K~>=@Fl5DvS{dJYZ$w zW5FZO1A)a`sO#9J1oy2ujrIOO=9$7tFpYHR2N)30gKJfK!72~Q&5IG((a$yI%FpCu zkt><{gEov&+)s)rH(dx}4qR`ii`eWaqm}op*(SHUMO{^F`15n82;$_ec~ibyPU?dw zhz>#&#*sO|ENW<1Gwb!+qW|?$cqesz6&Rne6?l|BXTq&RHU6$HJKch7QG@_J&20U% zo;bZ<+~-?{wR@c*agw4MDT4G-aElGMPKA4p+a40yk$9kq(YhxyYQH}9?L}>h_knDa z)@njvIQ=``ZNV5M6wT(}SaW}MM%Y)NPd3l%b|&=SdG=7fz7k_GP^o(-^AC;qW=~hl zENvE&n-c<4@_XhH-)%wE=mi$@~(G^o9GUW~F73zntfk(|^kw{|xrHKafs&)x}oT-U8JMfeV#p+8+ zhi{;wtz}owEH2kw<6jH{#Jez+TkzOFBI(`tEK7`^ksI5Dm!bXM=jjDqaQ^j30nw%3 zEbSySJD7!dJtPeRNi;osMdxaDL%J8LH6t;(+M&t)^!2w>;3CY^+X|3u1i?~8gx!x* zf=qy8DD7K+-m6}*$2*W#y}S}?W(>$Wh5=tj%z{^)n68#`jiI@_uY-!(*vOk(aY#U3 zolk|x5n;5wZ+^M^5q%SJkk-?s*!R7=ddg6xl!Kga^Qju5p?7)L_qJc~%2z@JxTH*v z%&2eTbIsuM_upxvNWNK=u!IlnXjjDm?%_~_yLEW!TK0pW;|9akpRF7gM+7=Hd$bzT15{pxTD`OZ#X;XV>}la#{+o8&y1NQxxv6g?rp;`aIX- z*+({ZebQ)a=@mS6N(&S+MizE`b2!w#@E3t_?7n4>WRjEy(-hQkO#i!V@DO{ikuLGj zs5J~p-`oGkK?YG}6(yPYff5{gvI+6EZ++_?RvC&1mO6-{S#$R;BV!7$jpjF?X&(FW z54?!g<@mbr_=0fKvS(djMR9?`y6WqVgh1djH+Ls$x4kbt^S|C_!pQP&cn(IIWD^jE=k+j{@ef!{g{)5lj>+&JR6c=~mk&|JM#!?XmF2(|Qe;R@A~3<_oonFK~FUXac!O5eHmt#jK$ zn749vT{xrIz_~(?^*DM3i9+%#ue5SJK!d-{xJxD>s2T<2i;~(E})~9bAm$ zCg3tN`NK^|)94h(wwbKu-~je-_ZHp z@TNzox-qvDyZq22cL$HELWRgeWl_XpQcDESfD%AA|Yc5UEF z*SNU?_P(!-^22ugWbT><2W=p*V9GHXCi}}F51LZJ@bjaz`9J>Pw(B|~>yjvUCXk{< z$eKJA`b?wZ?^na#i}nkYF}?|FBazwt%k9I9$uKVrpG~+_i|YPVs*BaBP3d-cUoD+_ zyJPXrCX;-)V9F9_xs{@(LqkXYiYF5$|Ksj+I7A*bVgp99%h?kiT;9>GX-F$f?x$hL z@hBPa1C^zjQAuJD?~O}X-S=V82~|x59L>?| zRa^OSgwj@EN=VVe!=oxsOTc|vHbqDAT|;8k>SMtNW3&XRt*%@e;R5$?!Yc!e_fHaC zEdyHbn!dO=>Wh7b<<$dULt@m0W<4w4_y^7XhZ@9;Sw(l9ZRd3kPQWo1vv5UEuY2}h z#FYcE2aJ)5M2+P>`4}Y&F1XaFmDMaFI61*8R@%!K?XHHrXe==aAg*O+&^JZ_ln9OH z{RIrT0shr%aFqno9FBNgURxr|&;7ZH_HAoc?@+si4njAkD&;I$%AALi3S0HLDO`Ij z&uYTatDCE?k+yd6F1Q}nVsPU-fw_Z+Qhd8vp&@6DzM>)KXn04ov6B3((F;Z2>xa@- z*UUnb`?A`Dq_*xZ1=MXvtYO2YNF`}t{oY-1!|3VNgf!xup*4{VJFx&+V^OL5)TBO| zz#U*08G&0^^Jsn;_dX3s21q$K_rWI158HYI$~+OlNA>cC?hz*|KlAhEs~HLJ?hq5e z{2RkL>}QNhTbXRDWh+^W8^Fjq>DAX)V5*`(LGp}Z^65E&#bXfcn-iQdq>R~<1)#A%E_K#@Sn9(fpChv#u*Xz z+=24qtFP+rESVmhnyw!*k%0 zVz#+Z775_?Dp9^pH^GtNtas&z^Q+_5koxPHTM_{s2jj_&lF_1W?#gOvD;o$Vbi8|Xwk7HHBy;f zRo?qeTWRrz+el_T7r) zfxv#^p}d)PQz3zLA&#^bsEDtnF{2*^$>cU7@E;c}c`(?o{kq2Raq>yUn(H}eG(8Eb z6H~58TRLuAOHG`~uag~#FuyHOM76|)6DW}vRTJ+CLi3lz^h4snt8F*rzCXfQq7&kBd zBO52t0Qo=pxi{JRAv3$cDN~yitr`Qa5gfA`$+vzUD|<2@KyWD!w*0WWVtmv6RE)M% z`(A9j#rt>fco3y*lpZ0g+&(IKq8CCd2R^Nm{tE$<5&-(N|3BByiEkDr3SmVr?y}Nm zj<2{S?y>4cfiMExdNFY#(5Jkq;ODq7IXvgC+wV?kRI2@cQ1$6biA4agp;(d*@{)T46&PdH~5p*ZYuW>F-3^lUsQfDDU90YD}lu2Yl zAeOtf9snWV!97}@K^~qXz2+iJ!cV1K%*qCq0A*ff^nGmh%a|hH3y~#}fQ|%vZ{L}_ z!jiF>0Bh7@mrse+271kc1oAYpRu!&#HJyOG3*~Qs$2&U$T z0-8;8K_X*C$Y*%TO?ZZ@cKrT3vk)j8FqvTTzjhk|uc`o$kq;x3QQ;k{36=-Ezbrm5 z%A>FyGBR?>-xF;VJ<>PSI5%!#wAlDPr9Ayflg545Gy~wLf=@ zasK(A;$qYuBL_bVRD3qL77Ska0|*<`tjj*;m@S%_>+y#*V@et2vj=@O!6g~3oYu>A zyl!s$ZjM~2$JzTg_;H69_vfqv2yH|Q4?bMQoS!9J{U;@o9=>O+FBOZ!14s~R5$T4X zur{3Fq7@VVR3(rP(2zgCmL=BN8p_8?e;BtoJ-##j#hf^Gc6nF&rpSof=uhT0J-CEd=k$eD&WVespJ`uFEp{tMHNt)w1a~nhNwS!m`R(8k9|XgyRHvX zsHn$zYYNCcHo?QO$K%N}qeQ8#3%@_pPdCvaKU7g+hUnIBDOZ8zS+3(;x&3G_Z92;h zU)YqzdqKJ6NPu_%O0E!a_1tLyZdB{Q*HZV;TL9<)z;gkRBU^-B_Yg^MvEXl$N!>s^ z;V;fo4&$^UuZe0F{hL{qu2?IYt`e=tdWSgw2w65 z3%TR#ks`s97Vpti*0x5f#>`UYf(7V7EZWX%bVOE;k`E*CR-YvtTsCMYK#)FWHjH>D zII1!TfWZ#kaO&Zl0(ADo*e_37v1I?fv_32M&uI$Cl1+n;U?->!{@q#iKL_=j(j#Wu zlpBb)Y)-dDxZ^zI%7#iTE?Zzkez5}eGB&mF?4Ix!@%!50ACMmy0ai#QZzB*ZtKoop z^MX|7py&O0Cx*Fya;wI}zNR>yJ4Pf8XzQ5G=)c=W3`lSXW-Vy z^}kHb`=5%)YhQ&C5)S;RgTJAC`W3V{-#*2ovI0O)>&;3ctEBd|A+t7<-1K%1Q91%b z2Et^2Pfx??y#$HYPW$M2$?W=udbXklB`aAg?D2Cpgtdii8sL8tng5%%e;UPXiBRs{ z%FP2tYJEDbYZ*e5~#gmrK_dGeQm~;?JmB)OJWp*-Mc<@o3>g~i|a&c&< zD@~`X3KKa+8pCSsW&nRXZ7{w*riFe5J7rU1fk4ES8pXBh9-Y;D+a_=byr2o_($=#N zQEyGARcar5u@iM{LT?qqqR8c=mKahHS>BNmpYq&0c;9_mf+6ao?88H)2mhWxyomeS zH%^sX(eHl2BCEMh4{d0ss<_Q~$d@qH z3HLr-g+6Vl(GKC)=gU3}7 zM^W6txl`4B{XruPf$fTy>ITQXiPot89HV1jT|GsS#i$+2zJ9j{(oVsETxlx6RFCPI zN0!Ul_a*#P?|=4fk>KK^4GI42BXm0#Cr+5&?@t(*xV#T0_sPfN<ifkDpB4_t+13=tbox^epl_ z^L&KEDG{E}6F~j)Ro)@vH*#g(g34E__T2`dflaF8ee)Y42-|eoN`=kZy#k$NB_kAbpL&~*Db;l-_7G= zD-w>H5>r-|UCe%7J(ZA5x1qT{`^a{@(dst+mKFu>6gqw3;2E7a;1KAWtkqIaMpEB5>EuW?xzPP-vv4g_G4D|jxH2J#ioBic)pu>ZIlcp^S-!0+s+?>fheDM@Q z*zx5pRg8Ddrhq*ZR@?aF->byAgI95S2}L#~Znr(!wIwmxNUS5iu`3>nMBnvcpDok% zwr%;hY0nl`?D}2JGvp{!Yp7e)70J;vd?~#$cQ!_YAr*Q*AJ@7z%x!r7&uf%@{`!Lh z56NA^^y3>PaoBf7i9Ry%Y>hkgBwW?0kByP+j*zTdM7XV6q9FqU(+Ota(`S;)ivM$T(_kVK>b+nSkt0$=ulR?8XTebV$g+XOgXJ-~$5}?^ zLkw>Jw1gFy>&oX+B(c>-SDx00{)kh`O=c6u`?}q7`GO2Dj82)pG#TH1_UAXMy~*qw z%l~#=qw6|>bqse$$KMxxHy>kQNR+no>h!6T#_-z;yr&SJyxruyspU$Kym!wY-v8_6 z_3uHy>nKAjpHrCAn2Z!m?}wVHD_fhTuyPe^M$k^ZBY-hVMe}xS@T|^=HsD8b*4$V&XkNchD`a>fS)!^?x7#J`yPA zqMCwjrRkLqx+-|FOskdO?Z%TA=%7y{Z7Gh}w~vOY^w;@Cwq>IKQzle0_zn4m8!yXd zVq}-P8SlkLTbm_5_Itrh%-Ci>YJ(!yqMDn`E7R5g6ZYS!^Z@&oMw0RCF$SX^U1)2{ z&#w^Ch~H7)Pvrd+Lf1xnFm>?5VL;zlugS-0$o#)uUS9=nq5|J^qRB@KKL)Eun!v^` z?!0#RJ(%KdX1P|AmoMhxoA8t)hoAI+pM4*EwhEafq_MIHaZh_mS0DwEQ;3+NvhU*y z$hZOzqqB-1TXLyOH_+K^^tlUbTr(BM{o=lo3s4AGx;HGpV|zC_{-Cs<NP*Nblj0BNxKt}wL8xMy z^&cO~1qi*AegAYIlWDi<({yeVfsWHKpW_!gKh|WcxS*bCA&V zq;$i=BP+!eg(5v{a6m8atXSdidLa^9d*02D*%&LFZl{pC<9}u71 z$j~OO*Du0~=aq*0ngSZ`aNwz~^VmSH08Z!jp{dx~Df%MEnyK5=?-c)gHU|=S?sBY# zB9&Z8NsszMb<_1siZ0q%1O{%Yjxe6gR$rHf*fU0v_cIM-W~IyKjOhLD`cD8taRNY0 zVXhT{UpBOFZ9}o5@4wzra#um|VAO&Dc{dx#&0?kEgV_7A_GAs5-5|LF~1A8_g( zPa#-+Wy4ZmHJgF-$;&hQWfSyM+xVAudQ#7jWu0$~D$j>j?UTk`?}fTntx5fNti?Dx zrwoPj|6#&w$*X3{+-;F2LIsE|Eomy+RtpC%T#=H7L|hvy@WrHz7oPSy)(=Z(sZtpJ z`SI_SU2)Px3dwpHY6QivZu{b5>gCSfRGm6EF`Q8L)vk|R{Ht6jcSmG}EXK=}v#u;9 z&3IS;b{nK)BHisj?PbKtl_ec$vwK=51F623oAB_3b2Wo(VOnkg@BKI>chmT*%%0^} z%#bXW@;?RcPleesbVMO*?#!%?&(o=o&Xo9bzz#J(pB^*=|?K zKlvNpcRqR0mpoI^q2pLAp5>_xg8|25ROxCfE>cN@InWWmN?{6z!Xb5B{r_YqjSdJL ziWCVokK_?-!?SnglaiHkFo`;;3}2b~nehA;LbIaym63tZE4-)D0kT)4krMCV)c(9y zpQoi24QCt&^_~Ylq_0K)=SSpmLF)z#SzIlTq7<;l>~AU*t&h=8WX6lOnpwX%?9_W+ z+4c1mRF?72xtI7_&9?;Njj{7;Crw32%SUKrxjy1Q_tgemqc0V({IFNh6#!(DReYoH z$Hz9AU>+n~bt_cNQYh(FSV=&$^)U^0(0Vo4Jm+o;3vCGyzj8_&aqS+#RA=nnnddH* zJ5TJZwMOn31pjB&RIY$ML%kR-cGvy0#}^RqN}=w0q`pK^dQiV?Fl8Nsz#}1#KAI>Pc5D9AR+SsL^`I~$bP;6O+#M8{ z5IIhU9%dG+&+Dyu6YL?Sc=1!u9dj%6^z)+BWo>haxMdY>69g6Jbwgf3O(P+p_zZ7g zvSCo^odA5ZH!Sc-HzMKdWk$gV6asd+Y77U+F|+S19@WwRq)#{mh+l>q1O|acz!y(n zHW4Zgx(sdqIFhP<%Mt9x;MJgq2pQ=tHzaw&6*aHSyw{?Ir6i1tvN%BSnE z&5xK@rUR`KIj+x_p8DcTOTh5&D?woKW=L+>{|*)16@XfTyK{sW@F+}vy)5InnXr+4y{KDDw8d$-P#E-#MjKgr`Uw{d^mh zBHFg~pPy?5`kI`R%l{^IGVGVryY;7*g$p5z33n4#5=gIx-y~G=2%#GOS19{m;AFI!!=Rql8-#8Xyehv)D?Ct3i{>hhG{U zVZYQm$~sxhC-%cS3p!n(KCW$gQ8t!!a|LfB>)dgHJL)4ah#D&Pj#XrqYAd$MmX?Y1 zMknYhU$#fj4D&;i!f$=b8|ry(Tg|JEm-Fk)Q2w_H6BEPrDgql@{I?YL-{@drFTz-} zN&F@D0>_=cr1&;8@hMd_%Nle%2WRVy`m~D^*q~*Wy=h5Up2Sad2g2Fd8m@$UE%Dk z31hMfn5r0cR>y>4J?gy3u@jYBp4Wx6^(a{dWIw1YLGtnMzv1qnZOu^4aj9u~qD-$) zxJwzi#3KJjDE!UWP72dV^3z$Irqd2rp58(2M*{EItt>cnD(dE}cXfI!C{bNy@oQp?$=%IaTO!w@_>4ftykd?-C0oir4VPxlI{3kHs%FlpAW_{_>5 z`Axh-4#$+7+fDFIk@!wRbA~!xKJ(Bzt{i{jxS`7&i=YQ zV=5n;^YvUIugMycRU~AG+{NFIgF)%>@t#6weMzAmVRsj=OEHIwU=EYLPL(R_R(wg! zf9c*KQXtQxiR@{*@Nf>Qua!*CGZR@-pk}J(8JqC#o2MbeGbz4)*q^yi4u6&t9}0ov z(ZQjwj)l{YGl3Fc=?F8SA;{G--?nOOWK8t4s*~f%A_7(&?2Af-_1}jotE`FEx(RuR zjFn1Ovb<8ZIF&XJ#*RP%$_)p$k>o@gozvfcZGW9bU-9O0dq@Pgxng3evX@-E$CkRb zpi(`e#eoj%%w|-&_ZS@_zc>18uxzZ~YNSG`(jn3EzR4}T2>xx9tz`PP0Hu zvXWD`T$zOeC?I#g8Cbwo;xiI0gLo$?h(x}I;)S}-ge~b&jK$_2j-rHEmmL5(RC2yd zViJ5Ye7d`eB5Hr3;Gt{@iDGbw$?yKU#u{N!NV1xGrCksec2td@Mv zXn^Sg3$33-a)L~W57UTzojY98c`D56z0R=QUJ%p(Yp{;P%x@T z(yn^$>D1ru62E=0znwm`bKxPfWcvtMaC=z1x~#qV>+6RsZ|+x(DdbsKRYP^NvQ@BIpBgsBPjt$Lim}_94#7r{=}nde6^y-a@2o z$t>zhgg$;pHxW!u&DmE|Mfq_;bSOgQ1SxQ*HvyhHdzqzX@89nJ|?4C?@b)%0At)m1A~tCcxbO)YB-6 zO+r0ZQP-9;`UxjZ+Z0+MuQ=Rnc)oe%xR1`JwB!qZ3%gz^_lmf{>(}l)86*v->)6eP zRQ)Agws3WKSl*}t=f$>3RbK@f{nICQeu_?BNx$#Mo=o5Zjh*1t|I~;I7b8XPE4ap} z$W;zOR8IIGgwN`Go#uDvqrcCKmj>LUz`W8*(Z<6|LosHu7O~}^U4wKskD^p&Y)(wu z0xm}%bPII27`Ym$pmNr$j7dZ->l&JaEJf;ccjL7|B6lRAr-;deUdQ{o#MdGhZ-a;a zYCJbM?jaeq=uV1Wm zXxDhpv4l$@4ntVjGWu?1W$37D>B$$ixp;^|>P;u~4FgRB&lx&AcpgQT-)WABlZGT) zEgMPtZ*I?G(swP`tda}Zy6ZHVO=7mCimNS|az{zEQ>T|?qpDL{mKA%+2S!qGSu+3~ z@lg(Ti$eQu(UsjonJRy5qQ4WI9SW3rB2f2}`&35J--30_i|_7FysBjFvyy#x(~c+o zqwEtKPrImmawK!3Uv#yW0#9K!Tj1}+^lLj?;A#J!Hf(4#{L^j1Z^?UT)rLthc=eJabHEv|-O~vcC#`dqknY}{!f9a|re1Uv;|HUl) z-f*Z_L$IzUkD*Lr{Ul*G9pg!sU3`j+BuDIwjr9F_h$P{VD?g;0p&e7YdIk-`jZHi= zQ_631vRS0t^WvHPd{>{2O=bUp7i2@J^>eU!7J(SoV|t}n3iAMUV|)N^qB23Kv2KFF zDzGvYtv^@kJR2hoJmsvb)!lK_^}ne_O4~+-sT*s&SIN_7t#j+Tl2loVP`JHOJYA%! z=B%oD|8`qCaoYcbnU# z_)li(R!-4O6Zor0_vC{oY)@}*nToe3TM~sG%DFQOnm-a2>UAA_<=swN*VZ`WyX=$J z+~14)JPt1u;McO6S4z|KN=~v=sycNCC;O@1Hh7~h_{8|Buh-#1;$$>_Ge9+?zo?Qm zitERxispB>&>eP`5e+B5aEz%_OYVSMgRy1c zgDZu-dYxJa;SRvFwi?@piK(Rs?0aL%kvoc18LR7uorD)VuJ%EV+^ zo_R+HeNo8HTnUY)V3Z-^Q+`z|Qb*8L)-3-@=%@9g{4#7JurL?rf$nO^2_MzC36WM)wzjhGkStc6*)*~IIma^jYctUPy>vP6jtdSxg(~R) zX0a95k1elX&GzjOIZX;RuPkE#M2lXF8iwJ~0;_|`$eU~w5@(#W8!AB0XJ}$N0EZKS_rx$lw_FJS@`=|b$8w(;YasiP-!~NBH!z^N^%)RP_uhq zYzcw*SBZOlrJhi8wCCveN^Y`u`;**~QI>3=HxY`O_xzgkJYw)g@#xgTC}S~795VC< za(!I$ISMcNTA{midSd)1hwJLIHiOK(M7eM-U^xQg4|~?@(3_7nKuE&Dwpiewfxyj5 zS2*#}B%~l8E+P0OP;I??dXEIP`_sQxEzq&#p#-&K?|(IrWorW(q%Xa-06=4&=MaW@ z2|)`bNR#zo*Hb{gGlf!osEjNFWr6vBvvvLfhleQlkzC_JG^#K=6>B~11Q=>)IQ@-p`R$Xm$Gv_rBnYh%`a7fNM`sr$N|X#dm|=S>jX+A8 zuNKJ{2MTDoggRW58V*Ni7OqT98_YHe4TUlR z4{JzXghsbD@B1wriIo14Ai7_tWnsAdH~c#@>{HNpv~_tLRo8>eu^Yp~pZfFWUo?qq zWtR}qFfLS1EsJq&Q%*6WSF2-6%llNbI)KHO;$raK!0z=Sq`M_kIA>a#)zbcSCRVv$Ka@3MU2sn{A{iI1J$5DY`w2CEZ(~Q#Zv4u_nhndhn7gh0s*_GtW4D( zey2l*4Q$kJYBsk20-dwI@c#0sqWMU@;dkr&uLrO9_L7Y=mx>e?B-KC=IMzDGI!kiE z)&@$OVRtsuvobpko~6Y)Grv?#N)BS*Is51EZ^@J*B>@ga5zx7pFMvSS374uWgrwf< z8Qq3i{_c2XjPlXG3RNhpbWzBdn3y`9Ih`7}Q%u}?%O9Az?!O>CJ?UtP&E^BgtbZwz zbe0?cIes)0c(B;Cjq#E#%QP~nl36xYifV1&hJ(Jprwz+4;FoT!jc?f8JGE5pV=Ntk zxzUCI#t^VwLZw7xtHfh-XTMp8{t}T+VI3AN=Cn21wyiyr!%*>c%O=HkuZ#b9aG1fR z7xIEva6no=d%db|G$GoD5|+za_uqM{F;QkpepcA+;dey)^K19fxKNh)tBKLr1&1qs zh$RCkZ6mjqWwFk9&$C@hOBKd={#@oecQ6_4dF0ORVRY;Oev=YxN~##WcOoHJ_(r8A?mO z-z?;wY}=)qU9RD;c^*TI&o0U(g$H_4-@r%$);d;A0@8C4zTAsk?L}VF1_EJ_4_OCy zM}Lmtv?iMZP!rs}34t~=T6o}je zu(p~iKY{s*M#!ynNo2&0KH8JhD|kuU$(v4x0cwKE6Ld4_rdOsXIiJ)?PE0*;(5||? z*#E~~JRR4UA_v3>M+v9W~b}D23fn0;1WFR$YZN(e^lD7CB`PRTwNS2?hJQT z8)<)c0>}@^xADM*)}yl<6X@w=C#aG@#jn$_F&ZO60WO^mqMnsWTXgYZ9gW|Duk1R! zVMf+yA%5)=^sb42F$e>0z~N4SzS_h}A`73&&DU|hKKgonZDaGd0v!lIV|=K!RifH> z7I(7s{iSO+q%52K6qv0=+o7-sE}uL9sMfn2Q}yt10S$JoumU!|#FZM8Ivp7XbEy!rjFf@#~}vFQj_@6`5tr--Gf~Q}KojmQMIy8+n7mFC|}? z{E)@-(Ak@NS8ccBI>V3DfJ10N3Mf)L;tg5jD^}c0YRV$g+`4~VPOL;TQLD$qA)zoHht{*%m@ z6k>BQvv8|?a(FdUPA=jphPw&YTyN}_^uWFfzqmvEc(&(QO>wdAekRSr$D{Oph9qQE zrwThG0F>MILVUijSMwfxx}?OtNoh9jf^T=f0KnN4@R;fEeb5|2re4&EdToR0)EIO@ zQ8n_@U9e&z%eh~O1>1)*_F)qNT*cM6p@OyZ~}u`Bdq_T6*io%#VOch zXM8_P%Pw11O&ND%^KpY(f*r+_*A6!wK$Yp_S}^73dLth!8VkWbaRBrJ>1}}Wr0O05 zVL9hle#if)=a^*iR7oMyS%}+uDgn<%yNkReiboIlGkx}H_T9jl@b&IpXn3hJ<#(6q z^{Z6jmcAz!iokXw%#Ou9{VL2S5JUb_|HSp@bMXdUG|4eTb*mxfsN8SzrC0ICWXs*0 z6lxoY^1_-c%-h&5=TG!_O<+LNnTdHHNNGj!GY!H7+2gzR1BxT`R<8bXc_e%a^55_I4>Ji`~9}%{=+S;{4Uy74v0uBRN!kBgJ&Nr&34p z?0xs0R0q@BWU~Kj+#gH^o_ta8Mhgi)`w*FR)Y?zr%rQqAKX!CoP+i*4bLE@!EwNGQ+1q(@ITzkZuDA6FGes0=2G%-j5v?nj1huG^| z_dS7ObchEIwj1|hMsl)k*WHg0uTL)U&b#N)fMI+JTjtPzU8y>MYW&#gGG zKE{cL9)dL9w>%zpE2)0lt`rC*+kkD#F1NEZkiVW9iXZ2dS&1_!Uh5OsEtwAwh@${G ziUNqKOqm2+r@@}}0K@YiW_2<2d?0bd_g~M;zC03D8LL?m;4fsx@U(;Z-ux&S&DM)~ z;zu(&9(4U-=Op4e?H!+O=iS7t`4d-zL^!xGApzkmnqW9q;AH}3=#Q3!7gdTZ4D%PV zvS$OGTibe#T{Owq++{k<(s7{v;j+rYr_mN`FVyn($tg-xMGpC^jY|mB3{A@$HyA<5 zh)tYqF3v3g<5y~VvY)g0juayuqXCg5W*1QKJVu_j`^N}t+4 z{aSaFM|g%lnB6_3`dsawf?qum{^3v~PqgE!t_W}H2YlWj*vf$o1k+vbuHn@$ukPKBk2I)0Gb4VmLg#U6I8}P zUJlCblSNsvpZ(7J^c6n{HdfpH_rd0t&v8!x6|w=mt~&36O-#&S+U=A7H(K`0LcUU_ zr`y?3X#?(U^0wr5T-W!L5`r44uh_%wkswX|hlES2Gxx&|@(%vIaG2c_*US6SRCLUy2AHny-euUfq}l1BVcW^7!Ua|1;z67k(VC6tGfn2REUhi$8OO2s>{Uv(4 zLc~y%+hO}KK0=9fFI%Ayjbb)!7i2VTSW?_t&HvNdvCOJa@n4og8dhV3u*$^Z<*WTWI~>}@Ah=87RPsayD1G5 zuZ_K*Z?na}*9=Rzxd^%FMHBSffby}N^05BZB*dy*qGF?~Rd5V=KK6f1eo&+gl5IktRCOm|6!UIHhY+B9c zf%NVX)m>4WUaoKZ*5njtIxw-n|ngSn0%-o~Z0wsv29f}v}Afz+|-n0&U$P{L0?%tv!*ui^U*buQ*)XONBYP>()4Shb+ z(44>VcAQ-WD>5rDPYdnx2~O+#I-Uq9^|o&UGyiV9ejhHEq334R!TH0;{bqE!N|5%w zxuXDxce1LM>OtRr46_-Ecr>$Q8o4@Nwdmpt3=xY8ElP%a%?YPFLPGLlR6N6E0%82| zr^MACl`BcC+Er!8*EM000Gq2w20)oqLj=OySgB74&*gxo0C1F6D4qK149-e(@ks6` zApKl0mVF=WV7~iP7#gDy=^@itD9hBwNK6S2guULUb$fC>_p0jFGX@xStaWuGN4)Pc z@2@`vUuzZP;YoI<1^MO)tf7QVa{ez02Lq@k-X0Rh)H?gOC7u14M1LGkZOepeJe30 zfX6+IQPw0a(qmb)I#v)!k|-= z@f5bK|1d+g=2jpzLn@&;@^_3XJP+3A~0jx{Y4 zjBsj-9GR6;YO5nx@71l7tf$+lV_#b3%N(4(%BEjiTv?ScFD{})ywBzRIn4?gd%hkb zUdYzrDK{Sb)au(W199>CsFxiFpENbF@5u%+Xq%sTJ5#pH2;;UEi?%^o+~}qVuph94 z=?>K#o6cTx9p^jg7wHY)akXm*M-CNtfxq-YQvhT?xr29%FXh*)?nVk;6O~tpSFfygdhGh zsV|d%QwjM3$`4tz%ET|UtHM{od5Syz`ZQRWtq6EVU~Mw6IN`w2Q#9kk$fvy?U#Ou` z+u(z4IB|)X9=h@S1klCUu2>t@zp^V+?R}Sn`a~c)+Ez;@F+Py^dP`1&rFc|{`j4#) z7Eo4VSTquzA|!pz9wiN7`Hc;(p){IpAs8$w%!iZ)$6(+^k%jv9DpFbsF4}(-8534h z@}@^ey#Gi~&0EdsBRZJtK~YWV+Cr^D*vnlu5DF_f29(2LOxDL!%{~?c{j;cf+xL~kBW>`n+>oH=otk-0RjVoQC-=JMH(j49YCu43!WTf?D|t0# zElAwS@j0o9HP;?9k|`^HU+D&>n;(K7i2k6mMVrF#JlSBd&fbqOuwHz$e@sDyB@Gx zJo>CRv>@TOYcV6J_s5F_BjnpZmKE-!Vp%cclSitqx35QQncj*A0KE4_w@=w;y3|3i zVCo!+600v8!oeTW`*;$iZNl$Dh*FO*U(6jA^PEtT9uKM1iK*%J&&YqL&48ASDbJz~ z9T;n;m}PaQlK02_V|8tV;x@GLj)WTc>*~kJ9jYvJCUc4Ld2fFSuB;jXZ!*|kZz7b8 zSU*lkl<9;5)T*%)|1dx|anPo`&-?GPjA;Sr!+!~ymMiOjPfJ;%QgfeT%vs|{83mjs zeA5U%Uo3XLx+{OfDM0jLthQN~VlD;K6Tz(qr7qrNg)jmDp~@szvKn$0zjNo`;n_CN z*S*!z?>akrF&$(7>zP8mY})Q_jxL=(tM0?!j}|1S%V^Xu&+hj%SrDB=?0 z78D!`)+OK7b9?d1PPqSyT1cO@$ zqdw-5OqX%FhC?5i7cab#gkfmZ6WRw4s_=>PRQ&V9Ml1VOMI}3uY+6(xBnYNy6=t}Tl<3g08U6e$2A6LV%q?vEv9}d(TS)ZY@?x!E-aStJ& z;8^|p*Qhhe0GexnRpLgs9B)4`cC?JV!Tvb5Xo$#w(}!CFbOq?yZBei1%L_|yUIH}V zh47jx&ol4*=izUy=MJ!vY+(%e^)kj_5+Tj#E*{b^_H zRsA9S>ZS<(otrcerw2ks$9X7p@9i-+PMi|5%v@!IvkoTuvcHwcc7I^Lm;}8%u&q2D zcxt#l-gxl14z1=eFEd|NJ(DiX$4quolIfqfPU(P&^l1NWSOq@0z6Y3R1)w%J)HUWw zD{yh1qwvpo$Ep><6Tt#^zaaZH*2inwIcj95X7`1upq*$zxQgwFEmlfPIg2R!7!{|* zPRqh{1N@OdWJ?t^lv^uy;doZ1umFD{rqR_=GPj_xpFGitEG`a?p`#FI6^Bt{z8@q2 zt0O^*zo$ywysugj|AxYLPLl1P0qgFX8#TX%+%?NooIj2&31EB13MxMgmXtj(G3LXu z+ERkbNkTXwv0{dcooI7<)G8o=_i-UY#k?(J8`R}=fL1l8VO@8x+fW1il^{t$^5|D2 z8#3T9q;5yoHyP|2AH+p~wsv6KziGHoIk0xri3Z!aJ{4)WOV`mC?PRF2 zd-qS~Uv2K~p1|UfHKy0Jqo=|2tCl7h7GVlRdJK&F^&TX@J+j6MdKYqOaq3G__s&|1gbJu%GgBq)!mY) zwg|{sXmM+(u1E|xjT&$Jui=LY)>gAoGJ5EWUzIx9yAk zn+vUPNL<^MZb6&p#*C>Qb_o_bI3*@m7@2`Dz&zgomJ0HFpf|l1UL?}%ax?#~un=lz z^M~wBFgk$;1KVELJzRf^{&b5f=mk`>ah=Y+&q}#7^Gw12PyBc0=A&fZwb$gg_iXs~ zJC7oys+%86BXjFZlwcU~ zo2`1;(U3olF`kgLZEoH#)ISfWqADkWiU|eXo1R}eX|GQM4j*EQrwKOCZ>q2_c=jSw zGZK$uRT}6C+_g2eLRTF1O>Z>1@wP-*zp$5e8mZ4tFm?MagBRj*I+=O{0Rp7H?VB~Q zrJENA&UN&sj$&f>U!|7=y{Vg9f50iD#nI6t*q6bjf72{%X2qgpXDPhaY>C1C)tpWh z2H={1)O)_eHnL+wlqIi1VQfEkXr_Y4B zS6^IrxZy-iA$!!V@n3)hpc_s2C{INVxmVa3U!uySM7y;6Op*#H z#}&`L8}F#ZqnyiyJE^KZa@l~@F+ffDn<`+z;PI^o0hP0k$sf~W!cp0P z%cGNO^`rlC(9qtsxd z-iAzJb1NS#EP+xEEdvzZ+Ipx^J6t}m;#F}GAdi6s1EKKy&c?gc66Pgf>hp;mB+iqp zN}Gd}_=%!_&puyWGIHZkYgwu+%-36#iUh?+%OcqqU225#7k-YraI;}qYkWJAcTWJi1%olx#5Lxj%Exn8ycmN>-AIegBGr^h z24$EG5j=a>TUfxg+wek|rnyJIuwch#)~m+w1Ro{AW!dgpOI_5(^Bs47r+qWWDT%tu zyBvDU^F#@QUqu_dpwDW`RT{d`41;HCv|$68`yR3=Gvb*MitJ7 zzGqw(W+Dfwfq&;bRvztUE4;c|h_W(1;=HYMK^%~h!>(y4IK@dPe|yOt9zY%P*Poe8 z)6Do27lXvlja$(x+v5{E15#Q9Xs}`R35)Fsxf+#4d59!r&m#^f4~E)yV7o4ZeNDW; z8DIFXT7o`-=d=M}|!0x3n)a!ej*N@~BG!)^^LMYd{2w*%+@$hk_u!>^o|pSMl8 z{QGNrNIwdv1h+0XOxbL#-y5CLQGKFI1ge6$V@y>ia0O=%TAHD9_O_JY*t~mu3SBDQ zTD4G5a&oO^>AVIlk!@D{_EL802RQn^Ijb#x{{GKjXTaA$)UgIox&HJMKEmOzIyZo4 zD^|?%u)Jc&*D)jR)YL2HH}ZYxsMj{w5Lc7#LUj&$flNGCK&v- zB#c|QXmbh-_Ek>b^92L%woi{wQX%dO9#B%X9Rp;CT+g$>6g>VlyX>X&#RTw!^paSo(KMhxDN+As!Yau`O-chc#d<-Z^|G5r$bqbmjA=6Oblc;6ho$XwYz4`-`}1 zp5zXzH9~OOF@e;s7DN@z1v45XXyyq0J+!O#q}3)I)hG?lzZ`V@NZ{A(Vy|#`vK^-iZO|(Pwk9Thh<5nqVS}>xdFjLO}t@L<8A=nPjjG z_!K`s)BUOdK%PAfG}{)c<(!DO0O2Umu5rBq+O_w?#-*BspRH%F8$Vk!{ri5MB`(b~ zyVS4n-;Dm1lMIo8PN!1rs8`^pb8~Xg_p!@y28h6g=|g}uD|-yuzYgsH7Jy6lP}lL( zl}-I5ENjs{SZlDI;nt(C-1NOMcYJ%dgC&-DJl*mw`}0p)IIBFZgkAO2jzWFSP`Pu) zilT_pYV#EqMhs*-xH%?U;NO^jX=z#eeLpo+@VVHeY&!k9%K&j$sE3-2xTm}>t4;qD ztHFXAIi9!l{PVbBm_aFQ-7>Ek{FJ#2-_CvZ&Cv3jl6VNu`^q@Rq9j_7#F)`z){dtf zt>;r|cs`Z=fI;IkV@8`fV~H#&h55hESXXzkV&EKl(7e->0`_~_pt8{EzE!-O^94ufE!w(diX`E}cmjUVvO zjvJ$IAegT=BV92NaiXK9wD{hYgv~FE?kVtuy1SI-_^q2_4)JI-$8tQDpf#mn^HtV!qFBta>Z(g zAqF@&3;*-i%amb}8;xkxYCU!1$;VSFpz1nG8)Xj^n*L&v(4in+UBUgr@Ru1Xx?iW1 zK%Zy(YF?+HF6_4O@6w!{auP8hLU}qT;eH41_JSsDPVNGqnnuKyQ5K*_^R@i`+)Ab3 zV-ddSz9m+;pS@|E%)@4er)JkQzHi%m<6Tuf9c%?{S)7`F2Z%)pK2o(&DkMHS-M0E! z>+Y*`El<(Jb3!mEaG`YHJKk7)nezw}h5IU~pOQ{qpYZC~}*DCEKVl zK`9#jN6-Iea87we3|(KyvUltkM9cIfQq zWu1bOJ<(xDn)h!jskY|t+h8jU1>NH!9RdGhy5E0?j4_VIufoAKx&nd9iS~a%R!DoW z+ke6RWVcAb9&L^|h&6tgC^rTbO#E55m=-5NCM= zgMomk-Yf%EYpj7T{o8E4{39vbuE`o9#>QLEp5L!*781n>2#FCsJF5#b z0Xi%}T;_U3Ov8k$?6I$k`aNFU=@9Cqrfk>2DXE``xSun6qAK&+@~#>^4wI1nl0oFl z>z5$VzutV)_=w3zYK2MVWHGrKui=0|T?ttr4P1t*g2eDiXzf3r`~OZG>3S){pBaAUQ7wE~#$F*4SS2_{MzLxU;`1%;xyOfX$8Bh1#lt9H+5 zuEb@_ATx>5Oqkz18v;2tKkZ9i4~U`r9gdEyuEwm7Xy1NwE0ZE+-!0xSAzjCIC3}oT zEdsioLZ+jzSPGl+PkbKF4sBb=U|mKPyj{ou-F2EtoRlI$pW>OJ~Aku5LTITPyF zLDlQ=gv2jD-cIpjr1TLsBb0mmlTOos>jT(sIp@W!$9lSrnG0}{3))r}GFZ&V9`0x9 z$>sQ0yvh;s5DWF*DIP&`3g(IpQ87xYw@PS%Kh28QyAkB-DIxR*&N=5(1N*j}(s?Yg z1euw93B0Fucp^XKX;rNGeD}bb&CE3cVrPCI)sjsF65lX}h_;|9tE*i|Uti${cT}9c z-3PtIOLV286K+$o`Wvlg4e*Jn$-g&*jEa9V8A8N#3R|3}y#j{n2enSf9 z3Qk@<*Fz#*Bd=RY%5JC#wfdUcfX1Qvx=Nk1!wWsM@aa z0bXgzH-BS&5UMv^OA?&bnb!d2G4dS0SL7Ai4szPrjBXi2Z&PRJ?Vj?}Km9$;2`!mJ z3Kx!0br%GH0g)+@O-h16JwAGJUi|s4cph&b52J3b+dS&?P`E(R=OJjO7NpEP6L@EsQ3KO))lVuxzg}pg4xH?KT9(~Zh*l=gOSJPC(q(0tR7zJUFJ2O zrxz{o2`3;Jdt8n0qgH3w?(%4WZ8a7twqK#tO~A1B_!TxOqCnvgmV7|5&Wrt%(ST-T zVA_^uI`kG!JU>ug|990n+f6&|Rhx}L&968IMlDnwxrM6h`eh~Y)vkNDwW$I2Rvs9g z18okzg4+_eD`0jEF$FQc6^)_iy$8O5X}|C0v^d*1{pH96uW9^X$VD3cC?ejA`5xnkdmP4`&Wl>M6;L2?EheJu#LjmG|dx|zYFH0#Et;Y=KEg#ZTR|~{GQ^j)2U?QY#Pwn?#{0A z-I0Ubh>(Co2GRn=yZyE&-b@R7J1kGcr&qsuk$Im-<*y3G!2mzi4$XksZ7C6HH0@7ma}gbD`c}=-r=#VI=pImyYO4tWmDy^$^V` zv*93X%wK>1d7yMlXFjJ%=E~WviEPHK^UvXhaW7kHyYia<`btri5dOa8wr!9duzf%9 z^KO|m4?v(L4E|tUQ7dRHl z{+@~Q?H95h7i^?IlT!hk5D5=)Pk4pff5IlOQ? znYu!YFNq6FHWd0aa`$zNibll4b|s8N&KuIDtw)|13J6wN+)-{obFhWRxXPtjQMHT4 zn9Ku8d8kDd!(xk#g$B`7Q!^z%XPIs!HRunj575HZ}((4bV<+pklcT$^!H_`$9`{WfuBxhY$et zjf}JB$pxI`ITA%r{N6t~6mS-w4h(9+?v1S0Tq(mC`LVFK!uWq~i)tipBGf@SD zE^B}bM@>vkRIXF6*15@Q0X;9;3V88t<|a@{g;-BLRoqc>#w&lhH#k^D5`s}L*zwiu zZ)r`|)zxrdtX=smE#At19{#~HF&+*e>$#-v{wTjU`wU9zp!E@@uft65Fg6p;qgFnp zM@zz!Krbhy`{Gaezd+}p6mi}vwPm_#&@kU-sE0f* z$`;sIqaB~VDelhz^4{0<`ld9pGDuc6C`P#LfBrU%Huu3**GfAC1Gxf{#1u4^=2KJk zTEj8Hagj-(fcE{@Q{=%RnVRgQp7t4b<3%H1*X4rTe7n#$hAVVfLyj=lX*cWR#s_3$ z*{c_)01eT=GH)u)uDUN^jn+*i({T;j+q|JLLt#ls<11&2S>t}2wR`!N%GQeup*;_> z{?wFKobmwOckT%A?O6ZyX8@rHmkH#32=|*t$+hZ`Et19M#`~!B#F@0_*_$eB>Qo;Y zD)-9e^9Dn*6R4kup2OD$U#vd#zvm73SUfY=`=~bds*nKR;F{>|u@Lv!wRol=2{X6h zTy?QPk)I7+O&eJUq*tkZ3BLhGSOf!tYY6Z)u9sPEs2)5hXuy6&(J955nB1@R&w;MJ z>>tghq$2*@X5q&Q1}{39+wf2yaoP4pWs3nFFz{;b7gcZGofVJT93ka6%i|?so#g%>%fPv~FIsuEIN{gifV>ybo*?+xy(4MiYJz+IHLZZ# zX&ZbF%3gb>a`RR;VkliEZ%%41d*VxEz+UH1?MALF-#=6O!C=F%56Xwfo$fg~4!nm{QrV zFkG5;n+gQpj@{k2QoX+We(DW9?@ep$4Lh*;+;|K0iOKv6%=3w-PW*mZ|E4GygACMc zI2(`Q`Ih?*f&(vMiG$r{ARcNc4F2$U{7w5wJ#Ty2x4f^dAw^%-m?JrB@~WIH=W1C; z2URh?|LmRbOJ zTMNG@b_LWtK7GTqky`SNI#cR?g|BagvP zS0sb<^=IWGcs^M|s=??$#lW2E1RF!FhKrdRZot;d} zzpEqW_b8*jN144;`E0=E@Y&a&^#;$JS`w@xnOQyLp zsMH!jynEq6akJgoPL0RuGhm${{nI>854#wYyKU(;1jbcn=gj_3qSjnO!~nw zy>r3g4&J(uQ{^Bd=GJJm#c)~uU{k=$lsvQno1sSzv-G6@to-T1-Fo@X)T6GfV{#N% z@NBxqEtiXArD(H;7H!EJQ23Xhh1-)0eDlL`$&cj#vz{6p_DP_KVnMwXMBJA*_HOpS zYK={#yjNONx}3q;LEIg5G%`|}pAcF4O9`@`5)Ve-EagSms_vL!f8Q^3J8Hqo!M9{4 z`?mcB8L?ME-;v5`t}g4x@DCSqjo|6;=7E?;cnVsbRTjj6k!kGYNoQw&Vir(E@~~tu zxQa&`JK?AQYB-o&1|%aX2A=vBHOzJL)>z{)0XvLLjU1pzR<3{jQi0<^rKb>Z|K!V^ z*A!js(SBtLhe|QRhgL8v@{vLgJRS^yT-+2!^Be=faSctboA+TlM*Kf59F@F}7imgZJtY5F!R$Vo;2ywz+|M z$D4Jjgkb*D;GdVdFs>Y=oBuaIV4-N;ch~33gidDD2=d9h{?bt{e(7Gu{IwObyu7`< zo4(iH&|)H^0uARq4dDZQZ>WL7+{~jcO$!U0X`#s^Y@A zTs3UWk__Hc;6WM^SPBi$r~MCg>i_zc#wk&62x`I&O`rB_x`nO8!cOYHg;(i1CDzDf75!wlf42dkZI{T46PQ6U0Lf})k4FIvS9k&M+9c3rR!CiAFq z3pY48TGFt)vLJUP{FUEc0SwbtU|x*3zQ&6wp#f9Gyum$pRT6E_HQNDA+u&)mHWtNH+@WSe5xDnNv?Fi~JgNIagTbx(PklOJb@5X1w;GZ-d>pWr&zZRMYb zg}uKEHjND`=y!hg#QuYLrRy*q1Q`Vn$p-clwK7}@K^Y^7kR(E94~$KKcAW$`Lf!+y zdW4v?7@P7j)o-4yx}Mt+Ec{>_P!J&?eFu|rrQ>JHW#z0YU@=r-9j;pKm}P?D3Jhtx zL?Ijt1`H2^Y^1KfzAQHKubrBf5mf$Jq^h!afg6T-s;fNU(A$Qw#sX|A97IvxJb`9+ z#3#hUHw+OK8%2fZyrjW@yrd;$%tSwrS7k}uM})Zp$wQz}GiS^lf&vUB2GZ+w(;Nlh z#P@JG(pGM1YOKKCot`|Cz?uYS$j#_OggFz-80BwI{dRpu%sq%4M|7gZy#?+!5YnUi7ZkWuLDypcYi~3>z@FK{%KQ{3+lp--es%+f+Ko#(dDx8GYSFnTVO`Z ze-)I`+HCb}KWt#z|N7#kZ~e~-e+eM^f^q*dD&eQMa&JXwyc1-Rh z)vW&tI*O}@*V{qJQ-};_16}H{bht`d_M!>n0HH!`7?HlP6|4Xg3-kj9RaG>C)mdd(p zMJM_8l7?Ofh1;4zL_N$f(}?Yx&Xe5pz`>^ zs4ln(-79b%Y2&dLt3BU(w^Gx@6gnv`w;q3N&!l&j(c^K~vCsy?BTj-y7U?wY%wXI) zmk}T4FyNUhD@Jrc2l86HoduraZFvbJisBt$+`KdG3WnK^>=$Fqx|nnGqEE(6hE$9HVp;XhZECY^N~(^pkL@`P*rD$rnArARYbWIr z`HS_Q*QxL4R9p*tU@E+L52l-4FzSpCH#j-9SZ6g|CCz;E5@rftRy~`GIg4}id1bpz z%BI`OJcxb$`ZL>k;%|EApHlJLowp)v>wQXR_+DD`iQ-=*blr2FBpDq_FF%% zgGe$1^Cfz8W?d&ge9;fXU@J74+ubuSoe$5g(jU?54#+vcb2r#1^pYKV@hT=l;u6u) zK~eFH^5aUg&W}EO`TNf1r0KCkM2EC(!YD9KfV|RH9V$1Tu`Mhte#xUbjsx=oT{!7d zgkN=sN`Qif#*D>YrQf=RJAM>+=nDb`9*7^S3cQk)yGDk55xw8P*^&~=Q*MwUhNtUU zz8L|eljdK*d`G>Wc{$Lc#eM{{n|kx+4P*&z1NE$%Bm;YjD!-~fo@c~O#su(7tADo< zX{Ng_mc{LVk&5)3hP}TgE&98w?rkpBP40jdud~MKQ_<1W-O(*eQs-=We zi$HSOnQ7Bva$rx{zYpuDnd^t_XG_}aQyQMXP5b0IPDTO`^rv^}0y=AsBg+m>0ypkV zH*PH)^V{|^pMIY{zR)yp(KJ7Dz>3wy|M=1gRwH;!Uw|A`tis zU7nV(_u`RC0V6wb?&b;mXrMITn~hIE1|CxpCI}bEs~)Gzs-P1mTv+yb9KkVg1bs;o zw9brQCQKl&KB5){a2Kdk)u3)M!U^HhI4}iy1B8PUX0k64s8y+}+3brb-6Z@4^EvZ6 zGmZp;3kf1HU(Fex@C4y0uAfv#?dTJx;;s%{Wc#ef>unY(a83JHH&OfbhL+C2KwtF} ziLN`qWqae9)Msy>d;9RU(w)4q+%AtTfQZ-PKXUis@WzV#aoj@h7`RoiSk)O00`{);|Z5&n}3JV+DB? zgF0_97DdfK4!a0lPB=LF28b?3DQFYvAEZkIZVkc?TrE0XsH3muf;Lb$LH=vofra}Q z*WVMqWSvzk%~c*;e{bKvKI%lf$jg6e4C%AID+Ytsh&iq+ zUiG1m9rgoVku}FZjYSEen00}E0x)7u1OlEI-__U6nhS8kq-U_7SfL1)vS%0B_`IMM zg0u!X=<_}<7LAz;I*$w5!$2;tzo(HD>5@1B$e-P2Nm=QZOQ*(*){A+2Gxh&Gqm}tR zLcuz|Lu^ND@5G@J|MlWdCmlcnZeU~-7;C~|0Z$a)GL=2yI!x;mp6hoz!9uA!DBO5k zfJ$o_+-P~p2MD_O@0iX$I?fZKryv3aZJ`KRezLNU?}fL?f^tUoq=&T`&$-5@TTBS@ zXKRl$(I>!XQzKx`=_)3>@iGK`Ar)A&et~Sj4JpR-xMbZ2%ojO`HQLU@_evr-IwPj} zH)?}2)0cEPrv?!5GdHo>T^#6FAk|0khcbuAz2%(wn8>url6@;WtZ{GkL>T!Zz45C* z>soeo`D`#3HPmTO?KAn^*X(x{fVT2M06J)~6Ev>ey(ewv@}%$S&*gcdwHRV5hUELEW#t-~ z6e1iQO7)hkBxnsDdBGM-8xLB0o0#qG=do3&HL-t?+^Vd8&gfw`tAP%eBW407E@){v zI}0sBcVG6ni0N2vy+121?|jdXA?Ee#lVTvAiOm8`l9}K@NZ;wF*{Z7hh?8CSl^bp5 zU%rP5*?3wEA4%~XQ!+ff1rV~H;NjwW57>lLU4s`7P1Hh4AH7y>LXBGUzQ+pPh#=$4pY&ak24OnJr$ew%uiLby5`rgI7_HoTLF-<)dEdqhb_lkOaAb_P z28SA-iJ}1maUM4SSwvkJqKruo3HS*@cF-w1;Q*d2i=9HBN)cKp2rRAMS5*|)KO`51 z=nF%Ih@eA(AooD0paTkA845cQv=EV1cpU+UjH{DpF8*%=dZ6O6vk$7Wbf*|w`)_Dfs#=erU zwx(3;Ss8gB>JMaGe26cu)|>09FUp*|a{dRDKyVnQ)&bnV0e}2ch$olVsyO+& z7F~v-Srdi6A^YDN*SE>FbY~s*ds(esOoZgcahwQy;lZ&&oB>H8+nL zpPsT$#^AtTvF_%)zsHgQ$FGl2?OUr`mkE5`s`o%>4Z-*HavL#+pK=DEYO`!@hvz(i z2Sm@H@qMVn8HN~aGU})!2&Zv151cLZ?l2Jg-UspsUEXLiQuf)6a9ZPEu(xPj?wbNI zsOI40CrfCG?@+306|GTN@A?}U3+_CIn)Ijq_fgwztAW6K99cj{7dDf4{Ep=~q!WuR z;weIoZ*Mm-o8!(=+A?vJ3R>E>G|gi3)9LWU;gYx+p$l`uZZ`IkaIk;1i@Kb6y}X!u zA{?~icQxvCeuDBC3b}5vQ37E;4H+<$Pj2f`?@FVi`uZkB!vdgO(K#M`D-t0XLp8k< zpNN^qT0otI2K?fs+5wQEUDf?ir3kaX| zMavA4%agJayHAsGxJ&Y2Lq3V~1G#W#SUCHHwMf&M-_^=l;6S#)u^eCs-aZCG*^a#E zi1@F zkmXh3XbLzlaZrV8*C;}YrIqdni@m30vR@-@a#Funeedf-x$E3hI&)2dGUAPH_b5zH ze}ZTl_}l+yOj8AiZ?w8eeiL0cd!gB(mbfQw-S>7F}jT_N2OB>ji>ZM$Zx~G5@A2vJH#^Ibx+=W1V|dGg;n%TSshcD!cDE0)s-= z33%DHOoA>Q=8cuEpa9)Yp_$gH*N!#%naxT?J{82C9a%X z{97sSE>&d>gzpY(eqhpoFxbBm?RH&XV*=d!QviF$78)YnbR4N|r-r^#@my91G{q#0 zjt6OOo2hI&plTi%U+K7C`Al!IMiWU-TN;5CK<}tM_zlk0_-xR1_%MKEpu+!MVg-A^ zY(n%9ORUt~o|{#FVEIe{zp^hWt9QEqvBIF<ma*Pq3CdZT(eBmY95 zMW4_^U=H!Kk86lvEQ96;Ohak$GggsMVudG)h18HOq71H3Z`duDQG{5(ho1lv8PqT* z8v>TxTD|4G9tPUgLX4?_Z&+x`<**XlbOLM}+g61Xl_zwE0pku|qnN${yHn8tRI8q( zA%*22Ioq_44mI;$VrwwF`m1|1pn;yZKwaP{#Cnz2NYXB8z^8~d*2-bg5#tfgV$ClQ z^!TS`e1qsCW9GXh1D1X?lUAm=wnU#nw@vd#ng*>6>_v=qwlXQ&#)XRcX790D-?gs! zl5D)uNoo6HX?M=u%`d}(UhdNV(`|Ahd(q{Su{GC$n{iWd`~k+<=HiUT6;Isl%gjzGC|nL5uQsmlyhpv~sL+U1O&O~T(VBwxxz z+A0rEPaQtOBx+5+X(8w40TFrY8IzI*--^a&(TdVrB_|p0xf>bi0=ecM|4LXho#%9) zXAxyrAhkMIhk{C|$wy`Gs#_Wk_MgEZa{ufwB7MU`Lus^RMbc~?fUW^n?(iD;JH!$S zE3>w#GBJ{C%$e$TGz30GdNHeTx|LmSmEur`L6V30yIkewzbYNsKhVmjfLZXA(IVJL8n?>(Jnu4r~Ab3e~z z%OH_%M6iZk?MG79)`Ne(+e;VyDgrwCg<|xF2!<%8r0WfB2R1aHJ+#tK^+d~$rmJpD zQ>E8Ew^!leqtbD`iVY$rCQ{bW2;V2sjq`fmi|_?pRPk7msD2a^AeMO^SXy3gR3y6e zm!|6;HxIu`|9kE`m+p;V0KdX&{&SsUx4%Q!3Nq$sY%8Lb%LUvXK$KZa)wb$MY@QN3 zMp`SWl`fsu;J*UI>zknYV>fgi2Jw`Z3=tI^nn~zbbM8pKV&_eO?Do9Wbx2!G2px z@)Jf-j!=$~@${5WBQS;Dvt=WBJ4T43ecMx@1QAd<)G%sq6q)o6(|^W^01VGxs}*!q)qSH{yX9%q0b;?V_PmD`D5>NTgPPMu#6Erj-vALQr3kK&JW=^F zhB+ihDp~O_{bdRr$|dm@P_fAyy59y+bX(>XRng?~M%i>&=_xzFWZ?B3ETJGXHA>E4 zV@wEkcsPod-~X4EaoqBnvMMb3)S@;wSf9xMW;qsU8b;J~n53W86ty^t zP6~gy$)kZ*wvGV9MQnU`Igs^B9@;iuVeZ;4I(f${H0fO_=Yi_mV{YrRBlab6T?`3_ zYwmS2nR_;Tm9hbbr-E!LrFz45BIERZCrajxMi8o}MNQcz{;ok9&sEx*i%Z&xvvrLf z_D)9dfpusND_!oe;ey>=0FzBDtW=koKNFkm+YLnmCP-N=Adc_!IO6))?yOwB+z>wt z0b}JsUs&GQp0>JG%uGZg7(Rz7F&J9b@kRskd?s&CldwL!tf2sqfa3dlMZb#ssq5cB zZdtb$yGo#@ysQ{^6k-zHCV1yd>C{f*s+*sm_Gv3mD7N!Pi!i@yUgC=$dbjwvS1%&~ zqI$|`ZgW<1&3ZQ*o6Qb5d_5^Ci~I|H=Srvsl$!$~kU`^@zu-TTJbP63TdT0|dW8vC&(Gh59uODL zDEnsS&)w1Q6Z~eKwF$w#W^Hc)npHRJ7k=FXurMG6Iu>Bs#|RXX!k76?paG09Ce`luQD%IsI^P8ad6;@x7|8_k*KR3eJG>d``bo9aYp$hhd;TWAZ}nR+&Q^fJURJzzCot_Kl;2L6|$c(WdKYKp?hE( z5(Z+E4iaL4i{G_ItQm^WG&q;kG)?R1JDB=o=kEiCM?(lOJO$1Eb)%s?G(ME}2)~h^ z$#LqENfkeO-lz5z>rIyUl2XcjzfDPIo+e_A_?gS8p;tN;lF4uX;^+Fy(AbI~jAqJy zIa_j(RDA&5?Od-HzYJx`tAm5hWJ{Enn?Q;dq8?t+m zocmijLQLRy-ET^!NS}n`=7PDmpEVI#lWyBBBBC~qz%a5TqP*dmO4;8uxBC_`M?{-V0y-9R+-$tpRHL~w zd%MjLAxiyN#~hJa<5=2x!z9k^NYYphkdcX4@ZSp;Y7ZSvs~jTpwOj0YG^wjTF3?#u z{S$&U0}=|exjoyO`a7U^Dh`efqN+~!ZcmVeqzhN(!79i9)kO9Y%n((tF$({TZ+fpk zd+`OptB_i{3?n#tcl&ESFa7x>Y6927d^8CKx3BqA!vpECc15b7&r`gORYH^7wE9xZ znRAa?o9S*gXC}ycUOlHHU(js0PXiE}$pMN0K+{KFp>%&li+#g0eie@0f(BSf$og4c z3^HGmWx}PqD55@d*d2$2Rwdw3yqZ#$SnnLRe$W!9hOw9%Vs zyOK>wldrr;Ca#x%J%>wC^N+5c<)du)M?9ZuhXO?>5m>3!E6*u( z8Ay*55f7HL<0EeF-@f>VET6wUcK2JCB1T#*c4wyIV=lWkuI?CiD$ikHoTsrc&1m z(f^|jD9#MB+7FZf1T>m3!wnc26vhk0^4ZI%)XPzU^gWFaucOF>KH|jEt#Qf#a0#|m z4?_=>`4fDBAya+bO++z3Y+t3t-50h+G29}Ep2^-jvAV=h%{MKQf^yJ4WDB^iz^0Xh z`6#n>a!Rjx%6;4c+6O?U?D^UNPlBnvE4465byaYH9UQ^uVK?VE{9BRqr9aF@o%=)} za0Ljp6B!fOxq@O6wDx zB?kaL*>>+EBm2oxglMU3o`t5^!=zS~5=D!A5vU$ValFzUzI|NBGaFbMnj)9_ z!5y+T7+Vk60`&eLT`sle?ZU4z$abSxf2yZBS~V_a%k&X;poEdYSBu^95uh7c+HvNj zrV3N8bg~onYP?)5ys#yeL=`Q7D>wgz0(S}Qor*O_C5_5ApWkw%RIP%RbixXcSfjxd z>`J5@qwFb^55Qac3<_W+7xvg~%@1+8l5o+`1MWWo?f}FKwxp*ZiUwH3IytG??2c+4c|0O7qdN8%6Yz|FRXJI~*iDJGw%DxnaoQ*(kfe!F2nd(7U2>vTjh!eoYFk?yjw6-Id zrvQ_4t&@gaMt^5k3&TwoWTh!uZ-R#(8p}Qc+=V`TsE`{sG1NpSLqdL`Jtov)SqT( z^tkF#Q}w^^$NcFL*YHUsPu2d&@2&SS6!?O=GQY>^vv*Fp0bdh$Eq3)N zj7&7po`q<%#9eeRo-;NVgIL;qf+>qG_O7HpLsN8 zE7u8^1q=(YSCexEU^TvpQyfu2PMts?)Y1eVn+(AMXosSaix|}I458B)8*F@Dn_YF< zfBVn2b_uSH87Jk`bo!BeCkpAm;zbCCxF6Q8-*2d)IRAIC1}9+82-#P7)CJnB`URmQ zj_?%pIsPc<@+o0E=p1r2XFcdtC6#^AX?(g%IN877df=T^4@7s@O?aO*XVzyNo+FDV zE#r=oXa5C z$q51CcgD!1n>0Z}$WM1#g(gXQY1~i0Sz2n($zcA)p9)KH~m;0Co13#+S|#2XnG1ZLudk%K28lAqNydQifb>>T(VzEq&o=${ z$sl6llUXz9G;-c9FX@EGC2@&r?;asKLg1#7|JkVMws24tuVmbjhnpz8>sDoXwDv)n zhzOLz(%Uk5JBjC8TLdDajrZ$hNtlC~UlNt1X`9o(3A>Tc;**}cE~r69b-T>>Nq1bH zslUCKD=*KkLe8(kafjDc{~a5-_6n;&81G-MP^jKbkUg&qd7aK>^%1F;;0fOn>(ZU2 zL@mW1l3rHc&ZH9stjB_$_I5KyqlHvmM4+$xO_xhQ!~z!ejSkZ$jBx8c!xusLMzmql ztxg@ggLW%fJTe=@PR%s_yY$m>Hob8CP5sFi^o~ zf4_g>;m%)yzTv2iS5RiXm8@WU3-k^|^VyYKHB=aC7D3iZ(@DcgOhs<`RlLN4(a_lb zR^pxGKZ7<}wtXrH-9j-QkFIh)1*3jS1!q>c%2(y*|AtND@4ip8d)LfnrkH_IxYG%L zr^3Yn14GFWiBRC5B))?_%Mk&|StAma3`ip>- z*D3sJ+6XNqF>U-J#5Uf0c9u1sMy~7n;?0O-{K-gVw&lyuW}teXsop5)AE-0yLqIvI zJMn=EIXs)7;{sPw{aHvuZ_R1JN-Q5C_sR4XxLmLt>`LlMRya9sr8O6BG4|P3C;uZ^ zdSdCL@Q>&UK~DV_!B5ozLsg|!?;WZ$4lkk&+!qVV(ajLzQBcjd4K)n(zKt<B}d$wEpoU+`fkd{R=Y5UH?gYp2cIIAljwn7C3xjP>qk}?rm!|pOX z{4^aP+Idp2!K~!nhIhG(OO&K633(1W*=UiSR{iC885sB#Nb9j3=K-y4;!7wq==}TY zWb8en$+~7itc|9C(Gd3AfImIxpkDy*I)MR!_OazCuccbTdAoUbZIv+kILxUw1>gm7 zBPZ|5mcstmMR4@>JYCdAt<_u&_z?l63dbH*;k#{g|j;;WxvX-yzEbpC_B2 zIq}kaa7|RiSH-lUFUovW^}a`ak^ z9!l<(j{kX~pXghp=e2hXy{l3v@09Y|M>-l8D-7t~=65^E5IyJz2=&gTdtZX*|;J+cQUOc}*wzeoU|IchHTL zcb`q<6Y1XtT7n^E^EQ)OjDIARsK#vT^=!iA&EId+g(s>!=u&=cES(E`pdd%e+j|=n zEjyN$LKjdyWJ?5!=c>d0bj1COy6QmThm6k$Aoz1DrU0C999t-}?^sdOVs^vj@Z$Y8 z!Dz!S;@ivjMgCh7KAirxH+&e=0blI&mEKORzbRcrdk$1fvS*=lK@rB20+I!~`FUyPtm>2wYn!TbDf} zpO=DR}x;+}v1kBk@5RT=)VfVr9IN6zfPY@{4u4=J!puyVJcdaWBYxQD$ zg30grN1Xowk{S4IJ|K&+LlItE&Q+Mw)hCnP?PO$AS^E1%%WrOMz7slaXxesSU;5+t zt0Mh~+MC+Um`5IM{2%%P0{be7)3}Fvzgo@0^P--v(^Cl}3N3YtZY4@_O26SA5${{h zMa^#`?&%mGG^y3=PWRd<>Gs!@nER^dO*$WDgXKb1JE~|Tw`s?1+_4)|r)m|VbH@q- z#o=E6GK6|21(p4DtB3am?k#ihYyUPFIEvwoQ%Q4a%8VJ3v%9CH3aHbV=x+BvcC94br_qDS12flK|Y+8Si z=Wgt?S+u3!cn;^avJp_Es}-Fns$XE*;L^|_5;)jPX?_Q-GlF$3?+jJ73^^k82P0g5 z!iM!6rhfOwB&bl_k>uUH5O)!D^y{|=H{6cbu4e)*oZf6(N1g04;|EC~m;V)9t@<^! zc{G*-q(mj#0r;9mKlj#&z-6(Z(;pg)L3t}n7eY6&I;X+EBf^_vdS}PKK+I8DJb0@0}sLs5Ot7)^! ztU+FLICOb{jMfGLAAi8b5;mO1Lq?gamocZ4<+4?Aa;j$chf?e2M9r>Y^}XM)@_)T- ze0AISmg&!@#QVLATY}p(yjI1t9>gdF+k`yMHJtEv@En2{B4hJtgE|7~1LhW*b-$kmziA?S^SG`4Y zS@B7uydA5jGHhPl=(_J@Lr)QU@HG(klyJ6>-$UcZj6GkB^Nda3q=4W&(8qk_erGXn zC%qOHG_I1fFL#YER?S~$2ZQGcgKg*o2ed3|ojw4|0Kw!Uue#9h3boLLc}M{EunWh6 zYY_17g*h|KMynEcHLcWDj_1{ecYc7%(Ron#nb87>_8 zP$?V<1G^fy{M$#bw!$2rfzf&G-NtnjH%~VE4_^$c)9+YGZ*}Mvh!aMSdfe=MP+Yg+ z)X+Os@FHqQW}wj&L8d@_c~8vxvcgMUU=~89c0Q&xHr-iAl+Mg zlU{%%==^KYUtNe^_Idl$`GX!3vDKe<&3FEhhLrUJ;gA*&gI!T$e{E$2^i;z$=`DqJ z-t(wPcdN4~BbG#%3bZ9G-at7%*ns;0fcxsv7OWFwx=%H0-MONt^SpkWWKxN08=Wm z$mP}FJsEi2ZPwENWU5=f61JwP9F-~Z$C(kzlM`(6=d;4icU2u|#$G3vF+Or)Xxp;hr%)vnQ80LZ*<4hn2g;Wx@o<|`UoRMOsar^2L65v_ety)M|V zXpf_%!>YXi#B=}^Vxn&E!~2(r!~0El!cFPCUI3B8oV;D1xPWf$C|W1-MV65iUd_tI zXL2*Y%|&dt+8Pm-q|p84J%UK#9a~@3&J+BUO3|ZH`(DmZB#j!0(2PTAzHqxCXO6wB zEee{ihrVgAJN~A7>cfWRY`Cd;#9K*iNkxBr_-NJI^JycDzwEzQy)l+h69xB?W1OI@ zP(;?wv4&E&YZ^tb>#TY`p8_`sl<*wBL*03oCCr8YIAfo8^jV3;R8my}Cf$^-lhKC6 zbF>IAw5oNV|`D`QN#FRJ(S>c=Age!3;FWnOYgSBahzn>f*vpi?b@r!%T=8|PjmOG#N`R>h@C!ru~opcHwVN>{{^TqR; zvS&TdyQ5aQWE0k2Uk<&`Y`GARQBhjm(n({f?g~Yt1<*HfY9df)AZv-4n z?^>bYmdVXfa3il?^b6U*)56#N%rQM8{K7x?GNcN%bG}d<kq%A!Z-&41_WP#Sp<+MrZS$wY03M$F zt;T6`Hn9Kh=7*yu96u!zZrICLnMANula}%Iy72b0(Ub|Y(gC{1oph$+TU|inxuD?8 zExw!~to5A#;bgqSUe&ul$%=7&jQaG&NVOe?h-2365*4}|IfAplx{vebzW3zsWYU*p zPi|x~a@sxWO{MnQ!SC>J=TyFxv`HP>(wp3S;U4SX<@J~$XLqJk#f}fA*LTzSe@LYvFn7W@I(O_-bUmv? z`{84BwSBbUD}njf<8seZlYOA=HH3V6>l`8;2_A`eARFR`E%YC)j&fEnKROk8X*>Yd zTFRw7V{PChT*Dpvlw`;(BY)=t&R&y7!_xV-YV9u*C_Tep~{ihgt& zz4bjg1zm+lUq~xGUoC!OZ6{nbqA$vkSlcr_;oAa%G@yfj2^tMs9!T`*?Jx4kv!5tm zLJm4RGu;m}gWjBjuG#~~9x%C?irgPdU1b>kk7k9lb$uQAHl6ZAGMoZg1z(l!)|TvJ zo6vW*{cIC}kEqmd$v|trpR09~)nyi}YG7b}a%$t%juc8Wl3%9Xv#E}EpD1tX-h;+l zC}yd0R^CXLy$!wN=~()Lx4i0aP+dJPyz#8Qn)XKNSa2`^f>5VrhM*Il@XBXvHytk5xQ|9i6>>BSZhA|~T@O3{Sl0$GWq=7ZofLPra`SHK02(ZwRlpU?%t z{-|ld7|x&p1G@*hK6&I8_7;N@LOQb3y8e#K1`3!Ahxw)>Ktb-C1LZ_DW0yx}Gx_~$ zOJgy=vu(@0rXmT5-e2bsieHsm)I+u}&W#5GjyGe34Wq$I4WEj+sy_H)*3MMnP(9d} z#9Uqphuevf+X5dr{k_PO<=1VPkk_W->^LXLQBIh1a`MGu3z*8jHkLuFtr3}-9x(Q4 z`wc$L>e$M41`qRoWNXA1 zaMe_PsTya-FOa-t@+%P)x`-9HfKFgujE`TfPy`Y5n=mr%<-<=dc8&3?PX1?J=kg%_ z?EA+-R|{uM{vABoflDsY2kWR^2fx!@=73Xj^i@9ve)SRtCE(WsPkB(M+nk&QCEl&3 zC_-1z`Q_z(Ch35s_V$&P8PvJz>*a^akAoHhg68KYt<}{URx|hij4@wviB#cCKAVd;bgV08!SJED|Q%fpKHdc9TQ6>Fm_tUj`GprDuD8g0H@q&KS>AZmV2<|2(@l+iiHWV_J>1^z= z#S`0b$P76c*WRo3bF}6B9X?sj(wQ!cFnE>-yU-hk)HR(?UYOf8_H#m7R^z#JWooMf z&Df3NdSQS1{=x%#NjaOAC1OWXko~B@y605UWkM?dJ%FlmwR^8RRa1cSGh96U#Af&aI@OIzxrpxFz84{1^Qrb z-`QaOM;w6>Wa8>OjS0a-SrLH%xN&M$CIdW-XR!PgX+T>|z~xP(cgIs%?2o7}3Ut8P zgg;)4EgFT-4%i>FG0o?V43oLMt~@Z9O8Y`V&95Qn!G)<)P5}d61APLxK4Zek#v5H4 z!a+Lfgr^B>r;#aswPFQv!%sk|Zz=re`53^kUs{ya7xB;oIhAE-%Otn>y5<7`$KuCm z@iT+<9VN+6W*E+9IS@;wLDwn3tonisr` ztr9l;A0wY;#MVBhBOc`R1mAw|r7m(mT7u7@{SJ-O6VP_)qEt2F>^JblcH71|OCTrr z+q=1Y){v69LdzfQRXKZ->kE$bBc~xb{|S8fD($`UpE2%`S(P=ZKl?I6z#DUd0?T{% z8e&+Aqq9T?o|7(GJI1eHohg$H$*lEx(b{#MSpWVuDw1Czoolik`NrAD z+s6J2RhM=nC|%q`9&SFBtmO3i=KolDWwjWVtzc*a8%@+52BSHBR(NJuUylp${3~%win0|7Sx4TF`mh zK`t}kuu~{XF`ls%JIS!lmR7bm)q*X1TOQAmS@}=QDmXBnR@DV-kO3(#cliU!H#d}X zy?mi^I^{UpP0r!fLaqsAf?$imOPf!b&=vb(+)*U4k1uxe;uRcL{GG6rA{hD>AG-*NRYNV|EO~xUAp?Mu;zwHZ?cfTG9sJ7(CL+wb3zWk@u&f(cd zrf1|4xpB8oWw$-y!o+NIq3sEh-GA}IClo?K; zz&E$Hmlx~%$pP>Ww!Nnz6^9G z0~u{V57X{_NbUGHjDh{qpM1XMy7}p?tu9&I&++yZ+J@a-+Bjxu*OhLd>^gqqrB2}v z#Czi`aCriSWA^V`IX)hH$`YE?pLzKo$yJ?<#6_A|VE@KKI{GyB+A#;>X@h=_Wu3Qr zHBf;Le$_Obm@{7u?q78xO{4b%7FsTu13N^sV>Pi?5WoH8fH4RN)Ua!4jJu9Esj@GQ z0risgrVGBBL2n@5G`SHXWMpu_r3&PhuLmHPlUv`a{!;Ld^3HrB&8yR{t*AlmXx-73j=A&3KO8+sl?bNPI18#cACihA3h_ z5FPo-;OJaelTjP1rT4CS6RBb+#{sGHd{F?OM{;h(MD9*FiV8Y+M&(B*#k<7`cxas2 z=*0$jJP8G}@94aR>!t>+x(A@&>NYyxx+j+kD;hw!^#EX-jryN%y}_(jyuWr*=NDeC zYY*H_fJW>Bb6TKZA|JP2#b-tf`y@c>=s^g1ev4NZwwfjR7q)2~Hd-sGlZW-kqv?H4 z0FvScs9}>FxuZW*Jz`z86|?qmJ?aybeiNp*$^W13@2wY>Gh_|Uz2IeHkNf*!KP$IX z>)ddd5GP%-8UT8L0*b9bYFN!XQ`KAU4;zX2g-nq=?#ClbrUS>Cieo2#bmD$ucL@I2 zmu=!nP?gS_+?g&8{Pu3{j0O?G+ zIt;jEoL+TiiS>K;@fCYGr+agU^SlJPo7d5d%1Au)zT&$dejI;?dN2!(C&qn?EjT-r+0D)ozh4kQncwmmh~u03Tf9@vDr0bKJBN3VEX5#B+injOf|G;qeM@l&WCq^+e5{*goTnec_4050I4e}+g_wMFdCw*WRx|I%d{yjO z87~Q^hCQ`$wKqQ$wtJd1?8s`<6IpSH&Cd2+A;_M6Tix^hvHIn?M~JYT$e@Zc=TU>V z<;5{ND`+dts_)~pTNRmt;B2JF?7(|H3d|?by%Oq>ipH_P2XmvzR?6eXOoJsRLV1tI zq)aM;zC@VU#_o<<* z{OU})E~C5cY+QPFGV=Cf+=e<+y&JVVC1x`ExR5X3X#uFMt9pzEMUT6rH7UnL`Bj2l zL7q7O@P~ILj+RSJT6%W!6$ae356+-Q(-D*BJldTP)%JEwT(4y4_ zFW1sgkrW&BEBQL!{;#SoQ$KlE&5=Vd%uIW|{JT7U?K$){I^NMwo|||FRee@93c3MU z0ZKNhrOvPMk!$}Vt()YR$EYPr$A8bACnLMMhn^Vo`@DKISLKEW%ucjaJYaX|tP#s1 zfu5VA6^+7et_gOPh6(X1?~)2OC0Mom80e@#mEYrq`y-+6TnchAaD=aCypf8#Mc0Y52JB;G^QlGjthI-evEr z8{8d`543C(#i>a=!eg)kHWw7k||-w8!YtWzx;hO(G1$)ibBIq`8QUAMU{)Y-Ki8xhd)r@)GZs zTt2AEeq z7sJgT7RA*1)xyJ~);Ymi17C>|j_iB}{hgr(A4hd<_-f3&^6f^r`9Ppv#r?_YsZe=&~BOcbeo(4*?x&Olbp)7{rD_y4$&BJ|9$dMEh$9qlE6Wp#j8 zz3s&rG?b-yVMX{P0ydFTD>NX!F-yMPFjBT?@GBx0KKIr7Z^~JRM@vn``hc^(m7rGw zYg=x6c;j$(L% zZ)&!m{pxHg7fm2G)VfIT^CmE9%3zA`S;$yn?2Q^Yw96iWe)Jm zXtQ78RUUu_KQ1!?_6jy$M+dX(lETA(<}7QF12Pjo?dlU%EX;z0GssG03pNtx%-iM* z<#m%({$y5cEZKhPz8VHi|7TaH!cDpuWk3oPLi;0MYmX-_xqr1_1_GqEzs3nn| zQyF7#6ZW*48Y-wvQj8Lh7+<30mMH_c6#)cK_0#(zXMytu;sIk?f9OALcVI{6^3#(n z3WI6vaT4ntq@1}Y#Af1dpW+{8Owvth-@c6yeb@lA<>tvY*zU9^aezzqI01ccA<*f@ z%Lo?G0P+_z{#{H@MKHP8&CgnT+(4u3&Q8jN8E^t-Wv?$*!l&+w-)`j2HF`IDbngzse`9v0HziQDR<3(n&ER`W z_IY&0uY5i@Kl5xYW*Rv47dwzfHWlb)l1KRI8d?ETEp=Y5e?2P`em$c4$0nvp<#1^O z_u-EG{kY^6gH>HizL&?Y72;&t#JS02eff!HF}0b2M@Mia+e^j+2Ix^aQ%t^q z6-dn@z+ZqqX7C4qkovm&*m7(6i>48EB*uY32SBn2R;K|b?o_$_*xqXOpGj4+;wnEM zGr0gYTW2V^>nLDfbHKb`mi^o_rKBZR-$MEf-PcE|8KD1Y~i! z-qRyF8r=dcCQ9xDz?U&ooLYtehi+X$IG~VvUxBsd|MB#dVNtzbv_lUfGKhpC!l0CN zcMD2LcXv0^C5@!CG)PIz&_g$Zz>jW*P`bNe;2!?>KKJ2+-;g=;p8c-9_S$Qy&-DA| zKvo3ZCr&RkgBv?C2wM1;7l}Mob|a3HgIG4#UAx!hGabyl3{q;}fy4r3nF}>k80`~+ z0es+Ki5^_#cT&x_Y^eLn?gg@a+DjD(f1!rk^CJK-wMmgSdpb}e>1uo%NfeqOA30T^ z`2wq0&GArH1OF^HLeVYN!s$Fcm|Qy(SSGP%RTVUff?~2u|EbE-DHHHB6twcu%;6Nu zU<&YVt>?>)Rxyy}zW?s}HfhwFb(i^kU&i;`vTl0uo$y%Hjd^2Y7a)v~_jwj}gtuAz z#&y4gng}kqDtH<=HB>;HPj1?YV2@db+wkc!$k}_%4Gsg@4-n)m3DE`a(%XashKT~mpV=cNLf~i@e5>oAu zMep>jXhFe3B2)Tpygw*lj%|F5){_ZBEMbhw^Mi$C59Mq@l~Q)dDpCXPn!_PigVmE0 z!dJi*dD`DQ=8&>R!z|Yp9N@`97b@?=?_#{K#CJ1n=nYMpZsii5qr8ZsxQvv3`M1sc zD#3EtQ`5okRxin~W#Zw~)wWnb81!+XzhGL>O1)dB05FuO+qDMbTrig1?c~AI zp0yXQXV#d|-r1V0FwMpQlVyy3RL#HSQ+et8mH+Z71s}d-ng_Adtnqjx*J8h9uAPp5 z{zv^R+{2z#O86iEM`TGqAHT};_{`8b&B>3e^3o=OZneS?zE9R03ig8Z-HeO}qF zRT&5BaW)(;2*qEF1(^gfBhf)tur$EZOEPAEG&S`G~l@H^5Lf9{t}0>ff96d z;CMSNdb1p0hYlQbR!e}jM~c&@xu&DBuU%Jus`wSrqACVvuYQdr2gae%gMzSffZi+e4E}rAAj#E^L_bxu(<(pqadnF12v(@Z@AV>%! zf>aA0TER|<`l$#Px$NmJv78wxvX+f<$D1q~w)EFdb$v zc~G9`Cf8)Tv2DkJ2^Ig_DSf`Tw^j_92eI`gf^Yx7nSeo*V|(bwQIOCS|7rhvzD!CH>eI=7cJvoqFd7-j_S}7X*&jVuZ$RAi zE>0uSSBbu)*PnvPt}$h|VK|Ug>xa+KyLYLF8wG*b-cCFRncJjZ;$95y8}f+VU@@6w zg2;b11@>4)x&T5@;~gG;cb%JS+Wv&sZ6x5!#;=pEEwFF=`)LRW{ogL#PDReIhv|}a zLqTs@Ygot&^F~zgQ?;+60eeybp7(d`$61bU3v$(8n}jyw3|1Pt+JSnl0I(=k6erbM z-vn%(a|VtTgT2@RR-}AR*7p6c<*wAt%#9#_-|aNjT{$V`U-nE#cP3%xE{Jx4Z{1N8 zxlH|GWy(BE$*&)>>_^TSjSB^ok9Al+XaW{(Ki0{m4kxJ;Ph0-;m@zAmV<@RH=GTk) zbSgouB6~i52F%e@ObrgoYY$X3svOP;#Q*z{bBwl1xC)ja$WJjH@r+f3^X|EOF#DdX z-4en8%Rr~!KjxRpylhb3z&QjdIR5h4H271NPbpo<+F~n>H(jA!hGcuXYTgBa9>F!K zHjY^>#hdkK2DMKJUFh>8})&!!|HhXwHEtpE7Szm`S4 zvrf1!Sf8I;w~CS;f3*Ilirm4|(?lE`zpnkJI02Vtoy=5n;3WAMr`@IsV`1|1 zCrNC(>*gFAsCd0WSLQzFh3P<`G{tz%Y&kdxO$naXZ%)vg2HC>g4@D$oOF8nac~#XI zOA0j9O7emRofD$##R8}G{=~sAYOx7;-fITjAN$|gC5g<%jnHoAn!Q|=f5I?jRbjwZ zY8>%h3gEprBpy07P!u&pmygWt&0u(av;o2hQV66rq8s*=#$tCtVi3Q*Iz#PqArO>#@7b>#4Wp~qtn$_Om& z3rJh(Cw0q96C^M29$aQndM$9lAi&Wi-){vXq<7w!2pi$gsbC^BPjqa=046G)m}V5c+ybAd+lLT=4T z_+ak6g|CuI37ogeU+@*Jx%Laj;D0!UE^PDU-Z&Bu9}<|0e%Azh(hTMQEQod<=C=%D zmnnHJA>+KQ;$Il3@r1gdPK0@Fgyc+;F)N@l;Np}vLsMum#J0UjUzDOai=NiMa)|L| z+bL|=a^#PL_jRf#E-=hpzt)E4p>I_lpfHCcZr*676@Ps7) z?-S~diTR=wxzFP1TZ6gkm0$x?NL*LM{)h%5G9cDj!p9DaPs`KmhC>zdw6QF!W!W%Xs6oMrprI+x-_- zN)oTMVy;Dg;vmN~;Fxr%W;+x_sV6Iuu+#RTnraiczG3Aal#dI7`Gd1w(@%;vv9d#Bqc$F%Q9{r2)I!k*mca zh2PFNPA9}T;$o&q&%|$Q&opVgYaoiAVwHL|nwZ>r1nLffDxVzp@+=<<_jw4{yHy&b z;Z?cKc_=c%3J-*khF%uJBd&;5$&DCQ*_zD4Ry&Ncq;JvtUyX_IlOUk4)PuwHZgA1N z*=u!Z=!*pQfk+9CBEEnUWpVtDP%bEaA-M2|uaPw~)s<^;x`%pj)Fu>6@s#d0vSz=_ zxe#LHBgdhpceHA_-ixIb_X1>88df@!9SP`ajrID05ropB~An9T6A3v2vSUOx1Zx}&4dcqTPN3_0MW?Qk#A;?Ogfz}&S8mLO|6=_Y@)+7 zt{BUQs<#N-aUF(!ZILZf!21#%%=%Q00jsuYuIs-9<=^}go^uG%mie@^OgW71Pm73G z@C*U0{MGAS)pM~aPVxd^b!Xu<(AYAko+ikMES+cZ%3{Fk+P;`y&~s8@vM1<)sSCc# zMWeJ|`I&5|g%=a}O~sGcdrje1xNmnGMX*T|e17v+_Dh2J2EW*@^OW23_p#Z-6Ji_i95EQMLXHqBpSE z6n9TAVTD_urU(9`Wgr&qG9xqoF8?H(tA@WVypMoD_1c1G8Bz*Jj!G;&U4d zeXTMQ{eT>lzgREUu>^LA-L`qFUg1^gJ|qj|r!<@RaQDRt!!p#pVxC@fr6Ke)N<8gn zPFQYfu%SI9PiR|mS4JYVf$2xJnR%xRfUbF8fW&O6($qAKB%3s~)M_wHU0#@Xe>G-$-x)hiU%2R0tiD;h<-cbyqHrE$Aj{sI7F|Ja7G zOhnawuk3kh%WQp@t-ugQozp<-9!-&>wg|2zhG6aJ`YjWXJudFBnC4x%ueNW-EGNjX z^AY?rb3=yEaYL@@r4BvIQoK1wnl zMWQbJtR+Ssli=x+SuJICPNg+M@HYvQ)oU-HJ#r$w z@o%D!*Mwmko%2(+S*$L$h;>3{L+=Yn4+Kd@=(+Aej2@D!UqyRp%kxZ43R)9$cV(FU zTaN65QD&gHq1IWZD^1;V4=w+;bZoVOxb<1`rAMClEGs?H=|{+I)+*0Trz;G z-4=dE9AyCn7f9HR2Laaz+$1IIr7qkkTDIt5Qa$_CQ9`n{L8S}-z7$vU3W-0m>-Qe80m zSC*Jz(ATj~QEePjd-5^~cfF4H)4EIdaJv@e3E(@yVSD?uH>FuA{NM0?@y=LmiAr`9 zuLLsd*{i<^4hbJKi!`NHkwnt;+7Zvu|H#o^&q|Z^O564)7p%7BR|Kvlq%v)ift+v- z6S&aPYS|nF5FTC%t2zW}e|pst%|4XF&_RFB{&O1!Rx9tP9xm|Ow|opLmY3n~FM&Fy z2Q=`yES80nvP=Jj-@)n(WSdpR^{qo$zu{mS$IY7Rt@MYIg<@(r(M(aOFZ{)(FR6*4 z-rrcISRL2F17H-tDhT8r-cCUkGWyoD(?fE%x}zg$M~Q^94J8`h!~`q>AdhdC7@H_S zAZtan<#Y3xid$)`InA6szKe>xaaPal%ggqgT|{HnBbo9YAX8E%#Nv(}R@}dW#K54l zZ>vVTk=kU1M%^3~tF+3mtO}>B+}Hp{NiZN=~VUx_5Dw$OO z)4FL7C%a(q+rh37H9Ok@S2&q5eWA|KpcN7$m?g?}`552#`tJAJpUCZpi;6%Uzi z4+3X*jKhy>EZCq9(>T$ptQan!m{$Gw3MfX#5mX;M&R8M=14YQZVMg}Zu4c=42=Sn{ z;(Km*tUyqNQ8e_Geuy9<-y0Xk73Ec(2@_f3fB3UwL&u9kv;bPrKO13Z=H_?N~& z7hub;_r$CCK!^a~_*Z6Q?OE06UO8ati*!|{5X4>#<1zFiau_6@RYAY@K+!+Z_i{3R zTJ zD{P${+GCSMx}rK9!_U)lwX0efrO8qfm#UV$puLEJiQW{s*O>9^vtsND*ILpqf0}(_ zSS$~u9XXE_|EIP;zcgSTD2`?jvGna}QP<~j5K8d~R5k@;$Dcr-BC4XC2&>s11>bhsU_gGRBKw4) z5t78n&?z?sZsq~-W=~aw{k8aLDffZ>A#6T76z;2-6*V*RY$oKdx4fYB^JqV$IC$=j zvV;VlU-^NZQj5(IT;oCJtSZuO2gr%eaODR?H#x;SI5*m=jY+uEj_k z#!g~OYtT)S$7uq9*W({&yV1I)kJ#CF@Sj93JIt6WcK;@O^H%j-W3WO3G&3PC8WZ`G z-l!L^bZw}DJlhj`+*HPLx2GHJP8tNjbs$k53MTBrG?n*V&Vbh~vjrTAw%p=DK4^yg zOiINa)Lb#pPRWdQ3_&AYzJPp4u_eqnNcp`95d#H=GG)w?5dq0*`HpOupa%G^I)z|u zrQh`kaR^wl*aR87SYDf?q9Q{}w)!HPXYUme7_BOPU8gNHd75PY;H5W&+6GC*_Pp<@jKW%!D?YD2HE^@bgg7MV}VX9z+MPxora{Xia=r^WAj=8?`L?6omB5O;KhaB#x*K9e#IH9X zR=I>P#eNfj{oC{_ZQ}g`AGmQgvK`do@Dr!X$x_Vw8@@|CDXH~_DgwmKaf&wAAC_?|47=(Y(U-x3jH< z!-`;=rJyb_oBMxfZ`pl+eCeJLJAx4CyXJE0ogPvteT4%(xfGm!iMaj}IlVq5csg)a zLry2!cDL7P^_FEKffKs^dw47q=Vd4s5EIB#<%_hxt^?#gIJ6h-{u>7!E*QJ7GRmTR z8Q$m%-d7b`R#R`%B&$W+j@5*Ib8N;mM%F}x5{9g@c>78MWGaI6N*riUIKlJ&qbGZ# z)X%UX^Lgq>BJa|3m3tN($&NToUL^Jjd4uBAr*bPXxoz!uPJmly1hE$Acp}^VD4Q1E4ezcw%W+{Ff`*3 zW(l^7h~OMWJkq%cu7#t{i~A-byEWwxHPNZ^F8;>v!=}X~L4+Y>Kp_f^__){MPOdO} zQmX2oZ*z;OZ#sq!5@TSlz1zAj&e0N4UJHHU5Rd{5dZ&iKlrV zzbk$kqGs;PzR2`it{Y3PIgx(Cka}`bWl+lzUBd;a&XOxmrSfFmfK6>#8g@fNERdEe z=PwlsIl2FTBazO=N-~{M`|ZMt0xeXO`YGgDU@1}%TJRxE0ieemb|%n3XRBEgBWEOW z9wV08vQa8>+`Zn`B(#C6^FL;@L%~ZICC2rCs1cN&svh+OEndbeOT94x(2(zZ4FZ&E zn}FeJ%ny1j&sS_b!?GhFbUVVndPy7=D9oPVDZmdlKzwD(b%=VAXq~UK-nv3m_Zb|M3^sDV9X!_C6ogar+ro z^~%jr1}_yoxh;1$10UB!A1)`0MZE6Q=)Vn=L+tEs-^R@Ll=6v)hwzcmHkeYU4$ezpnr1Bb9yaGa04TQ)^X7w4o~pYh%WIzb`?=Hl4-8VA=}g6Sp`$qsF@1#Zbw?7Li=;XtI2R%F@up*pX_?78iB3)`tP$Kd=L8lO6Tr8_i2Nt+`q%X$OXakPS73*gCc;srlUh1-M>(tNY@l>FTfO=F(Nh z(`wU=E3{%1;qY&%W?aXHpMC;5#B*X?9Fe`v!Xmv}EFdTIye6o+1#?vaFTWHka32v= z8oZyo-5iL|9s-GV4IU$ZTga5)vZ}QV6O0fOtd00s8TuE00dGc@F%QrLtk3Bz@$*DX zo3KxhcR?fv=?sRl&WstPej^CKdL!i|tr)CcpZ*+`WoFQJQU>!=dg3sOBkSm1H(Mk& z*)Q(hiUW3&3JrV82<7U!AQ3_N?Cp=1*vhZ_q+T-)_cnj|-$9~yJV*|0B8|`8{E&5Q z8!pY4cq$Sn3FcJ#E54?)K9l_O;L_XOiA#k&zxA~|j5_OO7bZ_vM@)P!RciC!Z*BW> zwpm0b_?xrOCa8;}T?Vwo3(1hQrKu7SLizVSS-85{`>Le`>YF&_ox8ni#w3S4c0C(za6^tN@{jVVaU7 ziJJvpnvf=|Out4}MSM?)gR0vw(+qJiU@~v#^i}OP>~|nlaSg{x7ngsv!S(wf3zG;7 zwUQr$#n}Cp-raOQ-LB5*DKz=f$^!!u&PD6+*}<8|SLtyJnR(`!4L-0am~Gjplj`eJ zOk-72pq6mVF>JS77-%@o@qWCp!L{7OA3hGBZ*y$qq2-~@hb2mLXAp(}Jb&$Y{=*B+ z1jf$7+yuIEvhg!dM{DM7f|`)=YNi2>;o5OQ==*5Co(ubWi*Q&jNrTAbXo(iWaqOvU zp?lYtlHJ@EwX{i{VD%0c5xr6xz^_mB&m(EeqqVEEH=N@OwUeGUz8;}) zh@f(=+}G8JV1$k02$;4wAc>I!WPewG&=)4Jc@3OKKM!Jx(kM-;@;88+U9Y0HYFhu$ z&36Ckob3b~&VD40mR>nKUBy+gE#Xp+lqF_oH`jbR6386%qy( z=Spwll*Ln#VF^`IiH?+ryQfQ50RpfZzsp+{_Gvl(v*Bj_QG(b-#6e#XBgbRm^4JN3 z4R1RyQs!H=Wf@sKHOy4{g$A27FHiz_{pp(#vCQf6s@A-Pcra|nTe|pK@h7k5Rsev%8 z@C&9T7)p}V(&hTs@mN|Y$r3Y5s)o@Xr9M3IkIlU#xbds}Rh|22!~m12Y18(vz?Gom zO+j`*=0(^!TLnm8AtCHtqyp0riK9D|T5jF6m@PMTXp#iDSE|1jThLPIpA9kHj|xRV z;FlBx0?h^|_kS-E+TTn#JnP7(s0u$i;;@hDJ)pH-VL)gU^C3IQ7a|&kaoeS;L#px!s==^ z^O#yBFy2yMoHv`oZy6&58*`prI@M)c%skAqD*+cRgSrHD7*!W(7iWu^)kj0J4m2RX zt2RmY32f3q2Cs<^Xv3G3FY&MvFAX_RKJs+rn=aB1o_=CQm4^8>wwXEO^Tyt!;jk0WxGcX@wIxfrkvta=@yESFIu0e=mxVSTy-k34Gh<2@$%*NgM~!ff2- z8!WJ`(|W7deiYOZN&epq-LC(+$;ZA^eZIiEbUgOguw6Dp3N}QtVq?1Ky@O|-8MnkQ z;;>t=wI_MLI@*SW9e>(7rC;cb$IFpYL#35jI^&jw`rm~^1Oa+ai zh5PWoE`E8B-)>?$AvrGr2^u{vE*lcK@I}@!@2=QAF(K@8{aSX{w2R5pwZYp#B0<)e zw>qQ0g@&wS#l{Bb*=q;d&%|fLuZWR6lES*OzZF4FS%tE)G>QhNL~+9RJ?;1PqBn;D zE6blhe|AttKB8AFU~YwoJM>bmn>L&M!tAqP-R)f z?|+ZFPfxg@zU^oA^z8K?SRuNYKvbX2)l~3)qeh+k(Bk*I(F#S);HGd~j!Q&67!#uV z&I&tb&RYo=?u;HX#5Sv2Hmy&(=p-(%N|HMeJcBX~S31vQeD>Scu))faw_1dry+3^2(-43m6B!xctbf$K7Ra+LvPpQZ^KS(|0bsw#Tm$md>3No zQB+6FfTKc<0!qoPhp-T(!^U4b;)#FIR+^gYzM zeXIXOf{)Pi)mKxJ%%j#H)tPuT3C4KZpUqqzry8Fhxg7^%$W=Bee!fh-#_4+w%JdGW zTdJ?w>s+XTa@BU9`ws3@7QMXbU+Ol=$*yGzLn|dTja-5ME}+p19nQN5LF19VjJ9|@ z_BlJm?oQQ2v2HeF9ZLh7^CP~URet<5dPFZ!BgnxuAAv67r zHafc2n0k5D^o_qjUlF;v*Kg&DunGo!gE?$cjJSNVRc!TZn98)H3NdiW#ECB2F>Wbd z^cQx|);fG9Tl${`2q?|H3MRms8#biNyU_h`L0qBZ`d{yflj#EU{5e~f&(1^z^0Fg4 z*Y9oCByelC>enAsfiO^o#_>Pa+?$z3Y!2;*mpLhFgUMSU`4blk@>{taOtTZ@!VZrr z#;~fyA6@pno(-{rFDES*duw%V!@2t={oB+bU6JJSq3721Z4iX5zo)9<0Tz7uqMYP7 z9j@tIRf6&@W9u~deVz`VqV`9g2H8#;G{mNF?%^+JlPlgjOgS6ag)=@1&}aRDZEZ4w z8RsI{H~PpTl(gV4yh3_jav*yQ(|8KsqAan-!B5d=Lz;HTaDc#f##K|dOdLs6@?9^T zS;G;Z_D_YEIMP*D!?nq)hV-X$$z+hxFs*BHj2j{koWl@l4l}+Bx=q;KPs!0&w2BRK1K~Usztx zhkJ(!pXclIqKng*iC2(lU!*Wcx!YZ85;s`Qkvz}>9@WMC?^5M&^!n(tb|fP8)hv|8US3 zutN8+s)#BMXxGPoETqoByVq-7n*cq!Yn094Xc}Kgtg23Y7Um^zP_%DFLQ7<9>gc!Q z@#TCV4n(gEyKH~MBmrK`uW3=bv_F7_s!&ADIa1yp$u8l5{<|5fXx|-rxa(x~ zI7)siG04amCv>w-aKCQ2x7-d$#ug8+hXbx9MO#q#JiwpK|M(}5RArS&>oP88MAD4M zu~PB|{=0c?LSZ!`EabZN$<)7%U*>3|C0=;B#=iNFT#9ovxVQ^5Y}XaNxwU;tWZ~D4 z<_^*M(0metB&=-JXauO#InS&hd)+VZ-AUk%3<&Ad#?Zb%a!>!`5?@6%X zO=6zg+?Ter?d{?>w_hMv`2o*Oh^6xC+K!*l0TsrB3^>-ulHfVS;;+eP+?xe2lwVnv zM-MXP1BO?KRilviv7*F0v6B`Dj{9l$@(sf_DW&}oR9JuucqZ-UCGXJHZG89o4h1UOm zns%>K`mY7t?`_^C$`Me9uoQ*H#0)=N$~~YMAC^)xYK_Qx2h)HSO5vZ+Qs6cgsqW%C z*$<6;THJyEY)EJmlQ;4<7KCGj37B3V^RP1Osln9k`kp6jDx=Zt;hCd%PchAu%45a| zC-(-<@8wy~EZ*UfvwjVBG09BIkX?J-lgyJij#Pz-qF$K zJ_{4NHm4x?YgxEzQ~lIqKO?hD(>B)CoOSa;J4L-jO@|ZK*lbbsOi_&v9a+Q1)6U4! zK{l67fe7w@yT9FBq{|rt*l=jqw{ea~IlnG8fW7%)*Y04v(e~{kZ)^XZuKJ||~>02`1P?R_cYwb)}ahAcEY{ZfJb zHg6*Fz>(?-X>7#@fyjPFu8yc72}UUHGcgFJGL7VN;A5peh)n?l9{R|TWM|{j#uzF* zAL3#W-K_Qi_Y);#g*V_Oa=NcksG|yI{*9abvqDz2)Xb*=-W>4IhPu6U9&Wq5eOOn# zpN&h`mI*U%v0JncYjIN4UlgZ+rs@?c(;hnhYhAD5?eA}d$v?G;-Fud1V4pz z-0J$U3RkzZoTwY#5(;t*XC$k?~FbIT&JPQ7)|JmPP^m+@C zc(qfxe(19^iRHyHCb~Qb8!}CU@IJ z)p2np06#JFw6;tP$*I=HG*1!dwu{kC)KY((!D-F1=M1U$RW;UbO$}$A&H2ZNMMd9- zOFu;!AR@LgCjg0Da@if?2C5ATPDmy#>5`a zL*@tSV)ebI6k(?EU!OV#f9=5#-xiM;6a7(Q_8Zy#!X9-Oz!33x+C$|dRY>gSY%DA= zHt{_M*2(ue<7AosCvEunv;1}=?u_tdciU6(Qzckky+%%{qofXKjW<7n_kL|~8y{^aTi10DBM+xzXua^^sHx&1Be_`Pw3E2+Ql0hzwcB3_J&Y1^Rm3pPbM1|W5Tb{-Xlv}pt`B^D2JBQh!U!lbr^6drVgXHYZu7XLKz z7*0%t$HQ5mOfyfbfAvf5Y<)or;u7pvZF)_g*lJrP_d|V<09@W{wY5-$sUX9uLgiw5 zU*0tVUIR(`bio?Wvtnix-3=WAJIk69adhkafSO_B1E=8X+Lys26$i6@#Y?UPTo-#? zi&pkZ4DOzBlSk8fg(`wpSZ&NNuNkYY%%iiU;>X~V>xeG2;8L~%E)+a4e`;c86^s-&~y&2++>2R22VahjOe^j<3}$rf4&GKEtV^NLasYOSnNCLuOmq-x1< zc%23}7*%B$p#gJmH=nZeseUE0^wQVHI+->Nb4LN+5sV%^8n1DBemW73ap)vm9{!IT zA>PW4Psha%!24ATeN)+SD^FuVfWpXhWuOTem6zdQ=^~pcs~aU?>ztx8istC&Nie3{ z$D=tcm<2)Kfi?94sLLrGhD5pCj@q_Ss>v3uI^-S`AW#RwzS&$bdzBBquT^UX3i4H2 zQ+5vjjU7SN^{1@Kcp`^yGQFWpKL<8RTb5m4C-`fLYWdW6q$Xm$_$BsNjtyP2w~QtO z1sWbI4pb0h%;F%U(v^0TpU0)&HthZm@sRfYtd zhIJ*ApY@CWdh#Dv@4i#-lP+HDnO5?0He zzAJ2*eaiXANDmYG)OXocU*c1losukr&yoG_gdEyL{ zH-O-&W{Bo0O4`uy6>-zYe`&b-!1#`HWG^Wbk%Lkn!=}lRPb)@c7{;a z`tmxl&o$Q9QVf<#dKM^G@DKYD?I(6`O;QQWQNoM8pl5Krrw}o`x69Pw>2(-=(CK0_cdDxJ%g$I!!qn>d=N0#u358-c??MuS0C5cXP=r(nY7rV^Dmt~~DjqqJQ z1ugtR8JkcNT*$?DVlA#bF7uYP^1nI^b-^<{(bMl|;pSc`p>9EvA;W^hU&ntOXGUeZ7J3}qqM=7XYr)T$R&+JG zt1{p~za}_dPHDp%Cy&6DK(=I5J`M&$ZD)*ukO?v$jA(r{aVtU&UHi~K^QsxX=Sx1r z_|v4}0(`;^yd>j=0ycPYb_orSdD4lnelI#4~J5tz@ z%C{Cu`DnwLr`|afiY<5LL=&sA(kaWO7t2Q+d9Vs7>@e2W2>3uBIek8pHIHv$`{wI7s_mZ2zAr2iWVCNL=!|7F zp7|Ee#7bitq}rc0TjWALg9#mdHQNISw5ry1+(SLx1Q%v98!auC1268{rsan!AkiLj zmuJA-1c}&3=sQj&t7YQ0&$-$1oYi*d6-aDEazfKy=*4=Jj{kzA(}J~Ut8uH?aoKU5 zOats)0-Cv(t%i7+kn`91Pc+HZt-(U9M=jQ{P_BtbpxC zu=29_Udi;(Wm>l=c&fJj)n(ImWd?p4+3FjS??!6Qs3OCxCmE{A8|5=vM>>g97Wq15 z*vTEo2VdI=LmPD7!I zr20xGjKKjIvCC<`nokiWo=p`(jPzXo`e&36RC{slVWb3IqNb?Fb-mK?g&Eei+%A$^%(}v$HM{ zs`ZOu6F=a*mb?FIMQa>yo*|1|&)kn*_qk?TkD`O#S-Y1a1rygSDO{E$%2K8ZlNgz6 zH&m1v8r0-v3wa(;Y>_x*P;w~V+YAUfH1%(!{bzxTFMkhUGZ&NZae%7Io!vh( zK*pG;nM? zQQq5ZEHx@@CA?k^z4UKtgKPw?YP)p=f7Jy8Y!cS&G50gOij{D6(JzMB_3sHKk~Kp4 zW^GmmL6uIio~PrVd{J8K7NUGnZWLsHuSR16XopZ?n=`1&f7kfS6}SG2swqh! zILRFZ!mMW=S1Tu<{|Q{^!BndL_@};$VMF7?$kiS7m*K5$NhHn^I^2+AFl(H{a3+%= zT`k>}zcLU!qV)~nIGS&p`C?~?Yq%@>SJSUz11ICQW#(Opz(u&_$lbc^SRn{hvA-A4 zx3_56OrXtr!>RP*cl&jGz@?kv=~VmeZ9pYH^Z%5$08MZpmdw`T`C#y#pz0y4BP_XpIU)IXKb0@~8^ovj=RFtgjj=WzoP`*&mB zuk9g!(Tgk1N@_9l9qXrG{lY^s+F}91J8M{=%E$gSzJ2xt#TfYj(JcC4NW!Ep8k-}K4t|hCQa)Ns6B~(N(rd@A z6oO$A-+|aI*w+-&C9IokkH9zf)disMvex)qqkw9BjX7G{5$)*C2aO8+5r;j51OyFZ zo=Gd4+9>^}2NTf5vn)+c0phU`uC&=q>xKa{#pB5JnU!2I%8eIKHWw{a zvPS4me~`X>Uc%(6VHT|-jyEjZDLeej2&HDCUM@GrI^XPi;fYM-bZhL%u1N;Os^3M-&d#pQ=~fPet30#fQn zfa#hgC!p;Ia8U&GnULtw-B`Rb#W9~;v)3eRqH%l=V7A3ks4Wpyz{HVKT>GC^c2c8v zTP#$17V5qQbSB}kq<}Z@E%6LFinFpcOw;;ms)$HoI-TraYPVi$2FEK*@OyEkMz#kt zwYMBEi?~e8^+?#w`3qlKV&%VF#meWmhyrdSFF z+WR0@b>077anF?SDGb3UB|{7j`1ftVKn+8da+M~J5f&z?*}PR~iYADxHXIG9O8OQu zmUIIATa|SDgCn<5N#{Kl$cgP3w}el_N|3nk2~*k5EOFm6`{I@FrTbjM#p?n@a8>-f zv4KeERx_EY?MvqNSxd$C|3lPQhBf`ZZzF7te9P%Z;1+zV5ir^SsW>{kp(iF+6u{y;gUW<wVRwFQ1egS<_i-_7ct&wbc2ooQ&qW%V_Rgz1$#@W~16 z%(QtyzN!n3QO;ndDPL%496m_K zu^I-_47Q=R=iLgl*`gEKPis}0de@`>bz)nD1SIAcXXy;4Msi3nD7g1g1>H5JsvFcqpx^*&L zqRHCH#7W1`bP`E|N%D*J)P?%t{IFD8Q_`vkLwr2nZ)Q*0K^k{T8!0us$m%e3m&yh& zNQqbnX3mV)=kZ97m5(T`fi7MsD=oiY5x+zfug;;-Mx8eoz`+RK0{mdH$h=?Utt`F`_C5Q}|R*_4c!(E2$a5BX7?=voJ2GnUwRo%9{IpzhS?>t94TPg*+D@%2x33 z3ggCe>^tj5{((Y6FNup?lgkbR=K|c~GzsT;8`XtGg5Q5&x)6T$F9pqpX2XJPP@+%4 z)+mS)*k+|SevK4Lsjm+Xucd7^OTAQ?c8Z#yZFaRbvH5F>P5>$hxL+j=9RhMB*deT4 zNk4y&O(Y64s3FEzY6ul5Xu>z;<$C6*Bp`e9uQb0#r7ZPLX$q6F8fzbj0*@HD@xp5f z>*Ca(uJUs}2xFLbi)f<%B(`tF-BlY=3mcZQh+YmOc_U(Z{V04nY?Nl`@8;CsUz zjH_0+>!99hk5js;xBm=wc%`?!!lX#o|K=G;+i7054C>3uZQ0#bHvtu=xGFDp$b5Y| zocmAih4X}LdXcD6{N3^DhxU65@*k7ddS!XqMOqFb5AE~fF#FbxoTXq z>v{MsC5u%e@iE(`4ODQfFT-Vj!QhKh(1uvUudsRN(952V-D=|=Kt0sre=?YG|1BGy z)l?r)!9ER~*dIIA6w`{HB)M(3E=iAkk$Q`miHn(J-Pq$%ze8znr_a?kOp>(SX;oxL z4C0eLZd^oAnaIm)TeVD4-uk-Q9H+eF! zvo5TjpqTGDRbGjFTtM18b|ausqEwT^3Lyxei|aX+5ZCi$AG;+IWr504h`v(sV_AdT zEWhlk-MP6(ibu+h^yG%4K2IK1PrE#-V1_H}aCmS5Yy0(=9XNRTeX^0NM~ z`dF2cE?w&#HG#@0!g+e09485$mdz!Rz`Ygn8pQ>XAfvR_hxVOO>mjc+^Opgf=b;B~ z!$RDW8>mmvVB@3)ni$@3<#*4~CSnvQ8##?BXSom?lgnOz!#>v98{*+2}Oc zhZ;p~)(?Q@%4gVVu|!*PN?*A2N>)s&*EK0-K!LD1*UUlW5KCBID$ zXjeU-)KnbUZBlWbjJbpNR;+k3M z0KXk7k-K5k%swI@JP2n+?1Tscl8M&7Rv+1)!ucx$%EUpd={>9U~f}jJp%#R)h z;b`w`;(FiTzaK(NzVE>s6C8Vw%8d#2rnn5Fp_^4z6RzrfUB;6|3RKJl9R+qRyR3<3 zhRujQZ->v(j`UlNc4W3@$r$A+-f@(E0XulQhSCy*e|ENLp| zb63WLSs$KlRIIAa!&_9iFR1+kgYkXRD%UDJ5*$Jo!2aD+V@fzVjtEQPbc z(bC7!qUQ}VC8r*%RY743SvdNshQh@=crx?t78Rej?~sI1+ zr-0a>b7bWi?R!Fgr5rIA+vU%`T73{4W*9yjJzE+*rKINaX@_8TG$7X}6%^#M@(i3H zJW%rK(NT=u|C9<7|0xv)F%nano}pE*-I_MEzrJV9jlQ(bO1JLxudhk)Ox?Ej6cQ+X zOuv^_W54qEk57@od0&rbZk?;qYiAS)mNg^85XZk~d7ms0hJ4Fa_hF~i%>M}{O%VTW z^3dovyk^t)vY#`u-Gz@lDFE1K6|asceGAayH!S_|6(?0+?%v{b*$cg^$tDQF?omQUX3 zxv!<_mfN!Ctf_jQdS2@OW9W0)UbI=R-Xx>Xp2vF>+Iys;L-cYa$>^||tE2djX2gqN(=b>=*$!1cbJ{L0hw#YaqvrVoGHIxS@8x{r%U;OLk(?#sQ( z%UED=$Q|hVi`{2`Ty-uk_EK=uyYNV}`?oM^L8G=*>!;OzGOUwe^{(NLm5tYH8*3Jn z$e|cFK*DYTS5Ms-Z&mS}K*;q^j6z#ZuCd0P8`$8LXo}|O4r{?jL(IFZpjQQbY^C#0 z4V*gPp1394l(A3kUpMT`K7Xjg{CrX*z60I@OKIahFI*&?dSbTLP~*85{l>IudD-{d zLrY?yvfor=S#Kn{^W0%Mpj@lQ!qv)X+SHT5n}P0>(i}El78M- zU9AnSp;nOJ9Q0;N&f4@-n%%9Gxrf_G!*lOP%dutaKxa0R0KPOSuVNp=vnN#Re^7Gp zD;LHUxiYc36pWrJSyaC_a(1v`X>;FYpDEVAcP^drSrNkLvJq2{W`B)f%h}j6JWMa1^`yX`H`G-!j;aD|f1wF7v80kAf}!lQF_G-RVOh`z~6 zd(9BfK(51Q5+A*crc~Och$Qf|N@Z}!MKL^S!^hwAh@GpC7oj3#Z%>=!)9h$r0Z0xr z`P_UnE!yX^4!mPt5=#3zVu4ROd7pk#l=sx~{C>Q2C~|bp^RX{*jo$w=8mL-re5rzT z^g%z&ANxH7lYN2t57NfN1B@J;b*LWrG~Se2Q9L|3%f9*&8nmhntmdnL|8+|GpbsB4 ziadDNu!Kx_z?-gBDpwU7NQ|YS6BHOQ{bcOYUeoo_&P1x$+{{Z;vS+4G0- zECx9OlP8@39vTB51#2entuEaZbd(eWAXT;8WzW-CeOdvRAHS?l?_U2vV~R1G54%C( z#TaBKy6^(H0rGLn8T~@9>t6qedi_&b@ec>xzw712&YP>(xR&M{_5XNvL0|?SY4B;& zzgzrU%z%CxW87)VfV329-)lP~O_sDRv7{6vd1tO77Ujf^&TxVjsS zg>%Oi4OL9=iyIABX3=9xM0hAXd?LITffd{(j#YQ2O^M$7b~i7FY}U0d`YsqAo|DF9 z(u-8~xUTy6>L&{^yODL^#`aQ1<7f8>CKgh8Q`?hg@$tU)B^0~|y$kkWdivn@98Yld ztUq@`6R$_%Utn*|>~ZMdY=slWKidg$e;rpYg%opv8|=}4PMtgq#{w_K0=fnAEU7;l zHZ(YA>OUP7GH-qZYId3Od@|%~0W`M|`1uaWF$C3c6!z6IP_i?HU|*S}*#YaG%>r1z z6@}JD_M^CDU+ynO*}sJbxZyEKmbQ5Qy^I*`b<~>58IM<>%6$y4tN}76%of9TTOuuh z6XD}EE^pGqfu>QhlYvVozbeL>ls4$}4fvPIo4q>>fNCTtt+Vwn$K?R|V}*&>YC~P; zwkHq3a#dW3{46=6MGZ>l^0lA#?OE8;4?f1{{%dNe&?LNzHy(7^@;&>ypyNtrkAbf4 z_rC`Jjnkx*ox437Vj=dUZ}i#Y>_#bIjOI{SxO@u;NwN#L>9i)Y12&NpW{)_nNAzva;&+ zpTl?XteD=Q?x2Np&RNl3fLa*i?r@PNDSIL07R3Fdzi@S&8%4=4P+sj6SzaDfxR=Ew zzU`x>lPO^bSLP~5%hIta2PEJu*7Nkl{pL~^0X*j|D-r&Nx4ayO3|%<+R&mACaN+^> zB>%p7S7y2|2?JRVKS~K1rDw*l(=>_uP{ZILnMbZ|k&%DOY1(KskLMpK8S#*YZyx(R zF=jU|(J#OV!r%FE0K68FyrM_=bZm^-XTaEB5K2avSq*$9qhZ(&O`MEwf6QUPUBNH3 zn=_37*A7qlOSNVeyWwy;zdwe$8EpG|8m36&bD$tI`sMukNo9iSDYW)F9Iem7^}$9) zM7ReU=jhhw}$4&~2Y59R@x+Zi~Fx z9+#E$p)(MN6vMQ^#?-Z}IRB94y3J2IxS46!Z%Jb&`jxi`6vcwU$TW?Cfq}NsT&v51 zT40l{7OMtk*pot5{>VC%@m!8Q$oFgWJxc|m;Hh(Z9QvKL9ob8w9KV#+R_)0@R*q9H z1uGwAah2rhYwc^}mSZc+%Z8Q4*Lxc*Lj)|f|3Z;}H*rNywPJwe2GR8zN3y8O#nf26 zedo>0S;(hQZ9kZ1Yx$t-SY$OFAsjBO%a_V!V&FRl(HfikIG?rhd=a@gN;P`Z@yiS+ z&PQ+tP4aImz&xiY)c3L&n)LISBuI$7vVKR}oB6f-DG^*FO&aGLp~cIw`zEed8m^G+ zd0@0nE8@j)cpD!-?fc4jpq=m?=p86iqh4i4M1Yd0SRccs4!2u-K}pj7`QS=UuQRM3~Wj5#tqEC+evI?@+>t3&BI*&7PL!TTrwM3HlJQ{B^lw zW9xt^(bYE4ho#ZL49QG1dF`ugwGt~G^=pc;1nfg~w#2*dApLJ2es6hpNit|jzx<$J zr6kCrWS>y!>-_>;L_8hgU>SIu@tUL+z0uvdaed_Wa^2^6VfI10&Zqa+=gl$fmun}% zLDw-fbe6$o>|_7Ut1=njT8r%#sfmMNi(I$q)flCe;}3;iuAS~Xb+^xtEGwNVU{Iqm zPT&&RD&eNZU}x)0{ml1SVqX%NuxhEN@#1r`zN0A`0qV)u>xF(BO+~}V9GW63e%$bB z;U&7o)|WX}O=0O`8izaZ!zBwHRc7Cn%f>2*ILY7r3hd*YwN*9A3FT73JPjk-)3`kscaoL$ z%f||*=M#dp6#S!%VH8plBM=FJwCKen_MwuocLB!nHY0udL;6E1N-Ee4b`{8cL=J19 z&}LH3x)Q_)@=Qgl!6>qAP-o2fdeVT>7fc1y081_0&-{bu>48!h-}cuT3KEw8x9Uwo zzB`^bzdUm@WcH7M_%5E6_rlP`98}qWZD9Ci4V@oJK;Q1j`Jd)^t=p{e82g%TuJOkd z0y>(UP~PHtH>{w5VP1W6uD#0>n503a8sk+j2YArO+Q~t|Xv}7&oHn73WC&;gEHyw6 z?aw))ZkKqa;C%d(cb8aF1Isu|ES{@Db{P6XA$_c=#aGA-V^%lr>_p~j_5)F|&4+$m ztbyUDY3EXla3av)OD*LTsIL|{Z8Zh_UNdaRGl~}$KEDd*bvg+rQsVlZ2MTYPZaywi zEh&{S;2zVhtXz#5jP#eztoD;zJn#+q-t?MzDNfKUsw-B!H$#`Hg<9E2LVI@NL*6p9 zMovwP8}7~Dcl!VfgQybmMOUA5zwpkY&WMUD}jhSV^ zqltWOAfYYJy~ugFBBjt}3UK{3VY>H>7*yxgv&2Ten_(5o(&?(7>QpBaE397knX+v? zl2x?s6s#V|!tWid#|>_ zv@=8u2i&3ROrh`pgM!`EWU&#s=GF!ZJG#2AApg1lxl;UBSzfL}RApQT$m^Ku3GsS% zD^D2iY4H75$wx$=dHV4!>(?9(6mM}Nzl&QRr14Qr+l1xvwvgpm|*-L z^w<8?Iz}5#I3}hdvLLhWgF#n?X?KI}b(6M$exH6}yUv9viWrLjE}g)gl>i5n;>Rw) zgOtfM;xo;Cr$!}16oO{v7(&)+-o(V5$u=M@JWjcgxu>!5@$uPi-50gW-IDA{_;@|M z^Uppa(RG~aF@@V+Z8QRgUz#Uks2 zU`k{?f>0Ga5|gGj(40q`EvMepSgB5-yr$?Ga|4&xT5q?#Rl@H7NpT74l+AsuvetdlW**LKIi#97Ieq z%<89IU5OtBy)rX!tgoynw+EM5TtVz74a=QRzqoJpnhZQzad(~&zGl*6f@xtmO$>Gl z3{wfhSnj!X+o(@p6s(*&?lsrFUf84*>#2v4Z zl9vf}ct*rYwqIP#{!d^&v2xv_V;s%RP0ljqYAF|5yZl zcucz-Bv6H7Y^gU(>(^<_>c@(+rC`<PXEgT+TV3Cm7za z)c6>-ELY=Th+$(Ch#NIrFDNj2ASE7MDReMrNvzx(PJb{UwjZw}oxd3IHK&2y*B+$y zi_L63WlF9&Zo_U?%ZkBBb>`Ad@h6Xmj3y7P?oMw$e`~HNMmGX z!#{I{LPg*aIh(}N^vr0+a}yv9vHkg(Q=JH&G1uVnZC%_9ygpuld<6C1xi{WlYcjGj zIlB92ll!%A$41_1>|*474m1cuqrG@Bnxasws{`&jpNW@$IYZsuvHIH*zq)_T|5gQWgExOrzzl3zJp_T zjzYgn6>@u=m!E+hX_aj(^vXS+Jlkori_OK(TOx5k{CtFaP%ofmo@R;4P=<+fZSg$cQ~HBO$S%lQ5W(n9z7W%kNjX z+jwC_wfbzFeEB7Ly8`;iAZc+loGHP4lB1U%3%1eVOdaJm(8E}A3?T*VDh?aJ)r7%lR}F9X1;0X*Bm#z>V7HAk3+fO4=u9-dXhy#GgB9=WQ$SEQvJVfH&M zsedCg3sc#z%+guvHW%y=TDB&9nptw=5Aeos<;HW$DBTy=1Z#myd}NO&D0N!N8@K#vl~zG2t%F&imFZ1+55!Ic+9W zrDNx`h6P;bI3?|uoCp6H!$b{j3(GKW1J<*zSxv-m%(YeGi8@xXJjVQGi)JmOcimNFon2*sx$HO!^X)xa1sQ^RWufcrcqrlPTlDy8NBq`Ob4Q8nMlHXC2~|KIuTHAkfH9Z=Ax!7j zNMXRy9v`^+=xcTVQSg^#hQ?~*m6g6~9h9h~P*XcLN8G5IcXQk!TZbqtFtWDTUDE1Kpg~fuc$|t|_;ofg@mTC2hkQ?_3Q!4B_ z2y$6?U1PN^{5m20XHOY`>5MdDA?~NUx&ym^xso1Ld3zbZhHIvIGA3|h&*gVv_Y>2`<-^f0ZCXZPnY*c61U5-FI9Z&ge%94y zd&~!4(i{t(kAi0-F*%i`IWYO3G5*M0(jCczuJ{59K;UOaw zB>NBlUtgnZJlk856JCJZpod5%`8tLBSD@GauL6pVvp>`4^0WIrDr{JSa)Ya*l{fFgQyOh` z^!3-KL$IxfN`1hb*{GkO3KA))A3wg*pJV;ePQ=EQiDW>32wpF9_cd&aq>mzm9kx$b zK@76(c<0qeW&&R~GRPG!D*7{;T5R}-NXzvI`N4mYBfF8Lz2dUxZ3(%VlA}_33vGI4 z%td_317vM_v$8liHT8b;lN)RMdngahNFnD-$dF{}es8{lpHX_3b=gK^mYh^VWI*k| zyzppGHuFW-zqle(SgNUBEHMRIbj3dYk-a!Vf#u8f^i>f+UnIULk7PS`KrXqQ7Bc9B zH}oWU$$kp(bu-A~(ruFWP%I(C6Bk!R{nL4&^F91c~93~fzfpC@jKg8=`?wy&Y* zvu=kKI@OZNO;7Rz?i`*Rz9bqzS%v+$%ZyqjXGhn~{9!1_G7Ng3vwAsZAlj>6WZG0% ztUq=7XWb?na1sRX+>|1CZApQ#HTOf9>J-T81+z=k`F3*ppQ>+VgV?kE(-vAp94SqL zI8~8=gle|&pfRoa2AqS0IsaPXm&+d~%&F}e(YhKQ~^`RyK@=N-~u)b|w9N1Z;n zv!IhJy4w+8*{Iorj}7pS63}z8s-LeUb^y)$MCi%QF3en9e_W5*NjjvJhV}+qju4a`;;g43{jW#x}G1|P^&Z^7UafVd-?c+sZ( zNVAc!09j_)qSN={OOt7dF%|Vki^97LUjYOY9I~JeZa<`w!CeO{kj1|0sH0<30TDo> zjRg%$5l^QfjyY$ytS2(A^fhhb4*RxwFITa|5lgvzsg!AtyG*rB4z~*_s4RI6Tb{|T zoSXBI{OR?P0ud4SzM%fqy|J)U>2my}-WIqMwKd$F7K0Ug2A{u)c)ApL*Hqlp zFXlV8+!L*yci7&1+|<0R9jEl{K51;wMmlL&;_!7h9n2masq_X!2}QxTqY7AHp^Z9> z0hzkJvYWvke5($<{nLjIxbo`e>U|gIS#3ROscGoA=UH_!=}{!40Zo{>X`sr6f~ca@ zyj&Ozg62bsm>a!YM#koG=gnL9lv*jeVr1xYNQX%Ie9Pt>mjno|MAL$#0@%HvVmun7 z*3kCILl0ovq0tIioHXThs$yn)w2^E!m5@dOThgu6t-uM_?^!u1=VgcYEBDghf^DFD zRXP>t+yd&}xz7F~)KwBb!t_uxD z<%5P#IfP$jyH-10RB*UuVS9LzawlZ8<665?ab8Kdm`!OiWuiKfEV)3XXb`2EhcNqO zCa=@St0WFEscW`RbMiu3a6iALo}5P?E6Sr@sGv(1p>3InC@@N4D-Y~#yz=1*7Y%}a zxj=l~X$WvHs*i9}8us#g@2Iq0O_3DGS%LiMr1hZO0KUYOH!=?d2Ks%s+>`rCBg+~0 zl%5VvpmnUC0M3!549lI7m%O-`+Fx{Ks2A)bDKiHjB-)@0+rGkvV)4c^)IQsNz=rQx zYmF~~qQas;UY#m>JPi^|vy19v+$rjzr6KiDc9b=^r{gedD0}Y07pq`>5LVvGhFz8xWh@_uSy9cdVoNIsxni(G)Co{mp zMQLFayd)f0TIn0H z!P$z%vQN>BoNgdki9UuemAyCiiEAc^*ZDpLu8O`w9Ct_(pSgUKULJ;1ZGP0)^reEa zGybyhVSHo2CVbDgQP1GPvgzvN!x0I|BL8$sHq^;3#h9wj+u1hGGj9$^g~b5s@EYZl zv}^RvM>5S?MJyo0F0^8;ZlKw9ku-{lpf>}gLZqtFmr<=~FTnqo50coUJoN;%{tDl2 z83g5p$NMSZ9eYz_09k%N2|MKK$im&u(fVMOAH)05IMEd@V5y<4W%g2I= z8h1~ioz)?1FfXX!4Uq5JnG}|<4k7tYPAjQY;R8_$5^+}ODyIh+S{kU?zwc%26zYbh zm14BCzd5P!ui3VsmHR(wrMI2*2E^tro)-oqKW@xfNlOtdxoih;%myLO`v44pHMxnA zQA{AcOp@wG3w)Mo|6TCB{vs)VN04{sDD(+TYx|frM#g5mfu*Egiz_w4lKNi!p}m(S z=wVl2rfwylN};~)tV&J4PwpgR`1QE^4x(G)lVn&_Xpoqb+3gNSi!ty(;pbn?JvwQ& zCV%M}PqAf)^u%Hq0PV955JT8b`z+|xxQ3O*3Qa@MR>+sU`Fka^h9 z4B41ZHEEl1bc@fCPlypK`eCm_G+6}kI@9s?w%-{IHd?$n^q4$fvhTQ^?QyeC8oO4T zJNFQFkvgw5OmLT?-{IpWXp5x4=dITVO_cz-cX5C%+1&NRj34n5@@F6 zxe~i9reub_>Q8dM`&N{jK$Cj5`<^#C<7olknq}-=0Me^l zn|KTdN>~NxWpIKqg~974vq%2E(7qS?@-cs^bpZT5WvaoUOFBFvY4MxAp$gzMk!?v5 z_9t=0EoCI?5xrtI`6H2ole_ZZdX%=sjPGFWjOvbX$Rx?o+v;gYqPq!k!BnmDn)eUn z*a)1=7reu8ZA1mx)t}sYP{Q`er~?kD{F`%5wA5eIMq*}v$=Q-ux~gX%&1CUnSS z*46Ex%(qb#*bKFl>F#=)W+J{aeGB|nkja#@5F+12sD=;DWr360lP%M~EI5wbw zRYI4hEwQ?m*E+1`p5>Q$irsS?XPrjTXCBPE=N~GRPuHfs%L*w0OIu>L!f9MV)n~1B zM?`_#0waPUi#lN+z$Qjp7RoO_U?a*u zUAJ-)Y@e_J!{+~SabFb*R&8cnVynCR?MBd=<5d6(7&YxZcN21)^D`=DFS*dLL{PMM z?@i`GjMeG}$%lpqe}_q0LS|F-mZpu|~7+7#Q$Zv3NwJaC|qdo3coVffrwA;Cp% z)K1y`_(u~on%76=i+Z8fo^PE48M(6n+{Ix|LVs3bVosvSPq8@Izy7<#O!$3yhlC@V zu=FeW*LpVwa|C+aX8MtacumwE5Cg|rNZ<4;Y zhI)5wEME_j)pG=%0zmSuMQyC_UuoNj%7Ra8?$+Ow-$p!@l{Q~Hh^Br=ukp=mXv8jj zm%FB68tmw@G0)6_`Z|7cFfJ*{Zy>2E9}n8hk!$n&CQN?&kaHk|nPhUVIHNeTgr?%l zOzCwyb#}ww$LxD#9qU&e3J=12{Q9g2%s8uy9UP)+idz1yTs6qOzB}SHMTp_N{n~0% z-FcIx&(DVf?&%h$)3yF*>^lElR;&m-j$YrjG@Tlut>1S!|H=})|C?h|R-g1`|WR=(ze;h1fE;2mnSU1Hd#^$81XVGrnzShJIiHmGtuFyNEahTleORhHXUaD)v~^_Z`USnG&*Y=M znor6sT{k;@0UZS#QwOpNy3!i^FuI(e&xE*ma}*vPBnHSaW8%HP|AISzT+Dl*IUO7z zp5|Rt;)m45oHgA_B<7DTIcIfFJ_KBJ!EE88H4S&Ar9ZDO%Z?Es!&aE>fFq$~?(>5P zL4*l=TA68@U8*}pen7o$x7kYplJ7057A%3T?vMS9jIe&Ur706%(cs75#I|kW&q$#} z#R5qbA_y5^6p>GXQ3#`7@umnz`R{0c1_ogt1b0q%|4oa_5EVA&ysq9@r``1?!aidV z&562Za&y-XjQ?s*6p0dfht73JhnttZU6BE^q2Q~!)@g_g!%FLJ!>0R^JXdw)R!|kC zLxCkb^6icTi^VH^tg2Vz+rMDXVJZOhvia}-pCoOiQ#UcPJ$_H%E#->sd~CI!jR$yndmfEqf><)N8az~W~oybY?U%U)% zI#zhycfKLkwYGeDNW$zV8dSV^=G9#R@Nqbojq?Ispa(scUk&{9vt+aE@luZnIspYC zn-sT}p#yUa4GCFvs}wb^?I9n(q9&RAW&sw35^=Uhv*%3#%kU1-z)E#(0sSGG0s*0k zmC8^~nURiMRpo@HJ4Vrl^F!s~9KGRbblw7g(@xAcW#W$sUNix?vdwY{H`(u_hAGy? z;N485l;=w;wA1$=?Pp;uKSH_r@dkClfP4ZLB-JOQyTZ`#LVHhNh>*>A;4GlV_kp^9 zEi2u1W}WVco|IY%}Iy{PCvB|PU9%uRCXOF$8FDCnssCvXCsOGzT$ z1m3Mx3XuL89v!!^PTCKF^|7`HebN&Duh@?`x$TJvJZ;lQ{|cN5IkO$FRq?OBNA)*G5ZigUcgqfUYPjO=Y+h=MAMG z$7hCdp=}$u2}n71+j0VOBGk}fzrO=dsZ8#zc1KftuR_DRY^Du-y^vc;LUM}p&YNdv z5B}A67%nv*w@$kQQ-~%1FI3tT$l2lbr%41r9vF7xLKOk{!XP@;CCR{al#$)u4<{pd zwRkKF`R`#-bPx^o?fYsF0Q?~Cm9<9f1$ymyGJ7eMyLZtY(CTL%w7ZzpotZoQnG z2!#qz{N}DZ`9_pYrKEKV&1Mbt5Y||X;;`-`rMJro5GIBG_s7`?NTIuJf{hW;1A97P zGVK|ChJ?%#dg+;low;U}n!(nD4N{6F`6DmGzw>Uv3zJeS&0DXOjAyQwA_p5y%q~sC zMJ?NZBx48w0|T!6#CLfgi&!CsHGDHo9Ucy3ScPzrrBqije@Q_VOf#z&hNSnrVtdKE z1kR$^vh3mQO~Rc)c-6`9*arGp&&+9M!=L5E%}5tQxB>CR?6mc1{h6ys(v}%z6F#E- zzcuG9Wb!^(Taa#FKJ*UYLEaCAESs_KoIfC98_R<9Jg({fX+EX++W(lvt1Z~GA0O`x zoQ(3`e-ZsQI=x&S^EJnlR;6;PBQFbWm#NE-sQ-I0&5D3Vm#C#J_|qs4ld#odB-J;E zNo9Ivyq?fTPIHz?psWz!HfB;LwqJWk{ck8@%wh`0N*pszNs5ByMl#S`6;sR#tL`@6 zosc^(&z;vjydK4G1z1Y(?|OZIM%~5|;`6&Of?Q5V96AgsM^5~m{Bx~XqHpqgC+pzg z`%v?-{O!*M=?$OT;X%NCw217#rToK6(udLR1L#t81O@r;_LR>wg`+9D;iB}L4}mgO z<92FkwfrV(G%*&8<2t;21Y14l(D2yqkVms3cptF60>iq;*v&i?G@QW@F`p%|xL4Qc z`bX6qPJ|_iQnsL|}-;?jD~L_j=U#OJ*#vuJkQ>CnSTdm}#)&z#+>jSkPj zGoFW!_x^D%FLuS6ft!^VM4nS21@N5|gAcH_VDtA?o^t=xU`|-Q8 zOOYKMTL_5;pa0eXW4Oz7M#|YK#D02*>DJG4`s%jL9fq?4X0^Q8X~OaEwfBUylui^Mq=ZNg5=2W~;AzyinYEu2@?GnXXmF{%YWU zh)DuP8nw2e4=jx*OwV{Z91wz9|dS`5K$3Q z(C}x)&%uD)!ODbvq}6FJ3&Ax$`lvC+A{uj0L$CYg+HLx}dhG{lt!c({L8lpVgY-Oc z_!r#6tWwxugMjTwg(&!6bzMEWsd{UZW{@)t+|Vw7$dEes@ELyUkyvU&bOWz792wta z-&XwKYK33)Uj&%X=u1Mj@{(;_YTxo$qiGDJwHxFM)Iua3Oy_B< zjkh8ZoGN_T&R_db3jjzw`A6rr!F#nNfggCQ1sotCj92Jl%tT}`aAD8-BD}* zk%@RL9U~CH#M4oCs2>exU@lHRCGgV1WaW$(j{J8Q%bv1=)hVlWG>A24eU&luz3J~n z;rdNtt~wP5K^41`knF2I$X?J)YZ8>GBxSnKg@WL*TN)WRmGvI})0Opc>F&g1CpX7KCr5wz{J4U4LiHKsxXg!7NaU14 zxUTE%PNvrGNy5LaFE;VQ*0w<8s{Su>wTls)>{!iLnvya2ix*kTJD27srzfINs?iVh zzL9j0jjQ12TY}rc?-6IB*!n_&@`AF1Rdp;nYkjLVN()7CFsxdQBMjrYCs_4`m#bV$ zR25Sq%5n|~H;qh+5-q8R8l;ZigW?y=per?q77WTOFn0UxZpL=rjpbdO-XuwSA(cWI zprpkg(JN;$Yd2xJQ!??m`0wBJem7WsJZEHl>766ll{RFZOnc;GnV&8H^T)qI>xJ&$ zM;3qffH-bE%!N>^#tVG=<;g{kM<%CT0QO9<(r=HHMSSgu&oxSk4`x~XuE)P~*I?{4 zK=?$ho`JZ#02MYIQ@yV$RubZUU(+b?wRu0$lEPG50q#qfGkG_Muc#V9g%}qBTpNc@ zZ5AEhCAAS$*Pkt`Bn^`}*Q6xmGG64Iou(`+-yYxE{>8+nK$oOoFRV;NOu@D*H!)?G zz0q!;pjIGsGvv1O4axjqY28`ers4vVk6l88FS=_U$Q&>(5Zsh;G({N;+#wh$iQbjeU}@Oswd^_-ojZf@w8 zV)zofS57KrGWoa1$uPU2i()8T7 zkVah{h7ZWTr$oAT^8ST0na?|0QXp9TDGK%@4n&U%M$$msCN1D={QmxY$ji%+g|Dw$ z1zCRGiS3(tn1ljhI1BkwtOXrS9pu1NVH!&+PUk4mUW}k4L+XQGdSX?1L}IiZe~BY@ z3<;E>s!Z^yRA6t%8(I70ugs3I$o6H!_Ud%NEPZ$ht5pI87{_ww`VxNB3~x+0>vo?A zqVL7~a`pd9ajKFbUzq`k%udo@_1OQfrBl;~>2g>uU+J*|Pqn+B1P+56!*!|?KN^l^ zRCcYzQ}lriIQJIx45Yt?4s)uL^g~FLCjpw;!|R1Q05uwD&(1kE?fT8mwSMjh>`hPs z#n?@R24OEl61)x15L$fLaxFeK{rNJ%)XQ;QF7*QaR67$4QEzJp-DRNCNPQ)JiKT&1 zdTPzfcwI}!6oVhpBlS78S;74O%S;8HmX}_EWZ3# zJY&1Uhe{xD(|%g?KjpInMB7AcF*E+>3*Tr6?%6obk??Nl%EYu9WW}?gsG1a5PgG1-+l=$?T_s`oWy$Vp;*?; z5?#xJ=XvEi8LgqR%PD31YnKqiDl&14@$-cs%OAXyz(Y}IFQD3WrTXxm5;4_1lqj4U z*a_AkX87ZoyE9rkSafzc&rWhA(tU88Z zo^%Ucq9zunvBFvQ1{QX*)LjzNxT>D%bI3P}suy{F=N)@2Mtw)dEir~5LR_CrRYSziL8A zXX<9ov?4Onig}WEKk#|+3-Mr#4c>9qDHorscF`y7n;eGe9o~|a|fp1W#o-( z?$&|if3)EuEFLL6>*#bU&KVmGkh4X5GN(GEnDLKcO7upk*;jvX{r2bw!}&E&msF~l ztel#g1oV0VG0^CH~1D z`d-t(>I6w`S5l8SQa^;APq*0|Fd`R(ZWjRX(N)fn*_1_hLOMywYZ4_NwwOuWZR^kE# zf6B&ctN=U%W#veUE>!Y?=53UvEq|&6OiHN2?UVanGWXaHjmms&p%s*(_wZ$(HNXbj znDlwEq~@s1rW}~IbwVKvLlSgQ)=5HSA9=>%FYSu89ZklUf$M;H!%1YoS zICHG6KDlFDDs`$>;bK8tjhS)ma7I>!+S-YrW8W= z`l@3D$2N25f_`$MMcGig{9|HlA3%6uX`fWynNkM~C(xmcu=`L(!$u&$zbvQO!8PXc zDCQ|dm$On$rp@vnDojJw!BMZZT*!1`d;6^Uq%9}4`wAMyPN7^x$%U!D2oQN~J4`vR z4{NI2b_?P)kj%JTc@qkO6dNy6%t&{ZgO)#xIIW?YTh^zkBZ_)r`?QxXKnN`w11U28 zY`}y)UPW(;tzSjSw(E1abe)zg+YoN}<}gE$aUmQ?sHrXfA8T(N*L3*)jiRD}g0z4N zNOyP3=omv0bm^2PT#Z4fhD$#j3*m#eJ9!}^jF)fg`*gK5~len?y)`{b@-C;z7l|4QF~nNpDvtq z^=pp&ax8o!9hR}NpZH7!17dxIgE^jE?KzEx-f-A^#wC-w=X>YNhARy1%Uv_!tNHA!M>R+ zLd^uOkoc-x|3d;{>=$cjNpun)V9elrs-In4yUWC9Q|#Q#7nK`odzavEK>f>)0_S(+ zL)G6}Smi%f{?YrH>xS4+JgU6O2_b~PFsj$^>Ko^eF&ibsQ*DjV4{r8LVP8)8!x>Qu zOt2UWe9{nZ{BCwm2+_T6cu5+WcE6-ewaS{jFyg5zKjTAImeSQZ_F=~<)5}svbH6goK+6+V9@PKEB3nY|Ur(S0b7U4pk$2p!tPaXGnK^Ee`6ouCD%^ zoFGirmYlwlzx!@H(c@pjcTMJH;IEX7I})WqU+A%?wZ78l$=CZM>+Ty_uvmIrLkqu4 zd?Y?#u4@38pLY_5o0|)II-fds7~u)lS@y4(A@)Jw&vw8Xi)H^)cFJ)NVA)_0w`-4I zjl9fCt{Fx=ZVL)mD>hI3c8`sal>E%uNyqANDmO47dDXANY9sF#{blLl2hvFEur$sR z_2Op4+cM=awy-iwbrk@(DHaJDp`5o5+94_phK!9k#v_VEye4X^5Jav3!r*s1%H6m! zxS+w(Z$rv;{q<|wKDWEsN3qxoD5si{jy*S2-+(mak?oL4+$*KSS5?C-P<9Eq-`HUP z$w!r9KHdNYtG@2-9PcFFo|v)ufwe_j^HEQXZ18z8Rx?xbK7;3ZYGF2PwBRcchZeO9 zn&79XzK=|Co`+IL8__>7P6ChTC%BJ5zvUnO7-P|+ObX>O8$xq^|l==dt6rMtiG`7a^?FuQM8* z$ML(UV?gW8u{+I|tvi0v8;RC`L_x**&m0692o=JNfdjtcg8NOG<#>{e%gaf5qmMczH}5UponfbHn-?`CTW&h!z8?>xP57h&q`+o&KY3AT z5>R;aIb_VM=R~?O>ORg)2B67#PO||4)5}2dj}|-sa@*h}-;K&%^}PQE9rKmvkNcV} z=q6clb>~s8j+X}H#fRKZ5BQ;sSa$N9l(lgZi&_#hr&1G8FZaB4Fi}G^4 z8jAjc6eI8J=%*o7@uBLIilE9eWT+ax626=KJ&I{duVA{6U~lX0!@#N0)Ve*C`gPkN zHA+e$a~?`Aeb^tD1q5o5wi{h4$ytYO#xhh|*`m%7JnN=Dw^7L!?y;-QV#`;BuC%A% z3z@q(P9uOuuHQH*83!E2Pe@}Gl4G_q@3+uN?I_5rU9=~<-+$%js_5!OT=pyb!}qV> zf%5butopk83cC7gaU2CHfdXq8|7d9Zs2}K1D}JB~HRn*E#~u3WFnHQ%ERG*Z!q{A4 zsIN2|IXya}5xah&^H*imRs_X|&)mv9d7PX{OQv$*RCoP7UOZM|rOaW;SWs@y;ir zY<2^Y9>+Nck#i;?S=`_X&T;ajrA4H6gQYE`E=|?tPpkc#L;M^EMtcK;XWwV2G`_eL z)XpfXsU->54)>OX-r#g_g1%i%UYbARYx`q3?<9kb?U3uwNah4mm%%5 zU?miYFqjr6I!k%k%@5))P?xqhfD8$gR>4Vf)9pVXW@xh$MMZxVLe9||UL|rMRU@Y| zpf@D=R@PhYUM3nT`BG4ROM(KEzq%ZZqumq^--_TbjdFo>9fL`dlOcFl)nFsVAW&L8 z68?TXr-*gR*s;U=>Z3rKaMyNI{^6q{c1J0h(lq1>-4mQQC+9ZTUv+#_WgjZsRmdw= z*G#s0I>nr?3Xe`R&hFzf?oI=NDGRsyAe8C0Qv&3@B5^k?9NaCu%&~Xe{KkZnzJ*mAS4nbENRgB!`au2otjV*|-~S z5!C)}G~fXCZjs()A)vBSz8>I>$vA5_HG);wlNuc3ycBq)?g`Niqng&~*8Mvwymunb z{Ix=TDg>!nhO}i8-9Zxk$WO
z81hlS~`&%1B{n7{rfAlvvk7{VE+{F&Ty zT>*~lWLK~ZN|7%H~T!qODW<;o*6m*PlO=aB4dy5p?`>Yp@@B5hzFJl!~k|t~P=9=va*K zf8^}VzRRQ53)d6yb5G3D`YKEQKq&T`7kOTVR=^qO`gKN4uoV1bP8Wa#gg`2bYPVUp z!>qGQslPvj+u?c(Q zyDP+XJ0Torwu?VbBU5#yn=q2_uZpv{&|PaC&{Tzyo4?V-Md)XLEt(Z*S(jP1SGcSD7Px5MW5N=AWi1A1m8WD5DBUQh z*AqmrkIsGK#2gL;9mC<$0iZI*V%;$xya%=QA7pg&24+sTOq^*)y+AA}z0SUGG>a5H z%}UE7P$?CNINwgOk0d2VUbemU)e;|G-M32l0Ju~;i)xB9v(U44*;%m1<#TtO3_Q)s zwUo?KO?9c<6u!6dfB8m(M0DeCl~_9nJHyF`>=*RK#8C-!+7B5NdmK1oEi0Z$JpRmJ zAV$NWFG$D{OJQC{l#pg1vOqx76Uu9aRgXS_p52-{>5v8{p0cs1d;Q-x`cf&RljTX`H0K_^LW zH3#V&i2fjCa;=X1!HMz@a(hzka&uXo2=JlD$NT5_KL~(p5H1}u6TbS6BWl2CJDkBT zPIUX#hBhT0s;3j~I26^SN9Hw&zuJB+iPkk@GuA0~YJxn-`!yo8%oUu!nvuVA#G$w9 zv7Sbm_zbO_046);owa=`?uh$Ib`vVoZ!yTl!xj|Bz#C)2#lBnk9K$RLlt9~5Oc$zqijH6a-nN!}<7e!;U zTw6R5(bQLz{(=dnc6QS-+LyCQCk^v{=t|6GMSwc(V;r0WYCxP7yoJJ*@oTaa1P$(NeY6^~tqbR^J8=+xC z$qhjyK#D?}bke!}1z$4f*M&oT@4{9?qMbMkjW;Xh6h^_ zD6(s%xPwzoXxQo?TTwhO#RwPq)Bdfpr@J$;2Vv~+tsc=Q$P~`-H1(ZH5qBk?L;oh_W z%kn+uLoW-#mJCidoX;-1mCy@5a?Akt{C^-qy5W64XZ1)RpS~QmJ`)V3Hf11ss+^*3 zWVn^^)N#i?tT{ZVfnO71!1>xaGd};<_tXO>PUo29Via+WPY(}!77Ac1JgNKlbA=ADtL2A#EZ-k>LIvFmGC0_z{`vzzK%bp;E)-Vf-v{h~i>T_o=dyrjxieN(Ej{V_6OvH!&uW z&zx?I#lH%y8ykt+#(*#&GF@_^qixI)Dov__FH&Z1cA=MKGwt<>1ZfKpU$%$)8dssH z!W$m(PwA+8CMKl5jR-@E3hAz?Q8MBB=;PgEDjT#$!Y@8bY^klq6_Y{e{I!kMbc*cN zAR@(n^N0hC9t;&#OTRiKCh9#|vlAKUA8^50PGle)2W)^!V1OMYCgUckv zF#w12K%S!Y)U89i{Z!!L)YeOg;p>*QC$CMfKR-LJZ}azB73sLlWL6twZ7m13v?=*` z=2z-KK4Bd5SfIuSeaPP6XP!(tsxJyX`9Q^M0_8MN;be-Gv#|jklbgnJ!d)AsSjyhY z%oloUxj*ckk7{V(;vdB<-%vAY{`HlXv2&Tdah@1-SgaN;kX9O{P^uZUeHQuw*P}V< zsK1X|QpkzSU|IB+GyVsopl6xpiypHF!5(KZ!R~`o^E|4|f0iX~=6TA?dmsM;MAa~| z68;*ZA&g@jQWGi)&!d?XTn$@)+#s`)ELo%jPI~8Ml5La&R$XO?jGUk!7WOkv)fW?x zb<|MP*ac`*ZuL$2MV;OKyTX*A;7lv&EBAmb+bH5^M&Oe{7N(Qz0L_ zXasVBr2MmPA?>=_dc^ovhs+I32%*jAr3mPp>HzLgT^o|94q>34rBl$|#3LjN{lFoZ zX=Fc=X$dZl3*{Hog*{|q4S~LwHUxE})?YxWQ>=}SIL56!>T6lS>W0&VhTjdi!z|T+ zbi*Hcin!-b9~1Pjej^_u-R~%8QuegGyow-Tn1n!})R?mztkm(Bo`|Omz$|uGP^0GL z9V6bKLZMJ--~rT}W^zh)HfyeF|tMV1XvhIf>u1=0&S!oG91y-5{Ov<h;oE`Q99oKh_jQs@0MD4+Z)OPa$5I#`FAnMm);m@4jUvPc_Kw7$UldC7@ru%=ju*M=~I7<1R;Pddv`(UKkTT1LzHx%&_`I ziESc@^z>j}r+m6c_cx>lE#TU8G6!&?d87NUZn}Eph_S4*c_BIwwKJ>U z^224BoN6hX?#4}mR?(p>ajN#iV3&Z>>>(Fc*SjFq z7x~CP{>DwF5GL)EGOg^KB*T(Wo!#aT7iudt-B@NlkB4geoZsQa2})WDZz{v?26#x& zl91iCH%N3aU<_ilEQkhTFQRP!NPmi6q29bbelpIvJ$R;VXsug*GftTrUB(B0ncl+^ z_mY}PAy2QQJhg7_=TrN+AMW5udoS%9p|%>w1r$jpCd-P5P9BdY}NTH zk+M-38`-$smXIyd7R6IqtRq4cv-?b^$nFaZ+kn4KI;2mK(!4Jqm7hLj#P(Pj{{ug= z#F(~n=}B+BS(|%}xC7vpX7Ohu49V_{yClbp|Nfv8eh-z3wlS|Q9nwAf^ zkPY{bWuUph8#q1^!2l^dBUsy>;IwxyFx7ME*|WSAO&)tC!qkG0+)@)26H;GOP~x6H%RKG@Kb!vcs5YWZ8PV| zlo+0vD3O@xxa2p4vc(O@rO!UN&kBF#pbs&0X~+|BZZ^=fW6=wrLz84MQBfiX^h2OX z4_g|IDe>}y+7RAH&RzGP+F1PVc|fjNl$r&Rwc*Rx7SkQ~8dk*rPq3KAmiZV2wgo}%?jN@ z!y)p)NqtixqP&S5Ab?bmuq_F)=N9i4r@5$x&H-_rb5Xhw>8EmSV+4s+%v)=e2HeE+= z13w&2&o}mr**q86EkTjVN!0K+V8o<-P}dV1A&01AYBg3(5Y-}k5GfEV%$fdu(0QnEKWf5+NkBenk`Ww_2be`TLDT%N)M)@l6wnvlzxH7d zKmI?IDDTgpk(5fYHcg;63K7SCQQtgOMgYGsja2^}^vC^Yp9b!+q>z=B>MnfM6)525 z)n=bY7;uQ#4+~O2Xv1n;^GHAx5Ei&9oJ38%nOA_jvW;ic2`yv>mF1&eb=qYihO4LO z_E!Hj0)dkq*Vew*2azFcuLP=4 z(lm>)kG|shA8w zmT1F5XIR#lp~^XsZ(^J`Jw;RJWw3~AJgtmX;sn-Rh^`ubylOMbnxADk;VkWV<`nvI5 zLGDg7fK;$?8;OwdCqsrDsTOe$!N2MR7vn15Q^)Vh<#hf*Bw5G0u9V;2YyZ|OS!{mj zsnQZb6G@(^%~uYAf1Y}9X&AU{f=z!@O=FB1$lFImZHI@LKdh0rTKqBz)*oAYyRK5K)-)QX21{v8OWzuyCjP zt)}Caz<6%vsl}1x@=NHibhp{n^S7odpt3*g4-%z(zC)T zS>-e9tv~xO*_ij6$sGqG)y4iWE8|<`59%itBVL&m2ta;#8R#jqL5932{UJ^EG@P`v zVV$9mkSXeM_Qs+14@dG|ki`8E?inYPNP6XN#+)7VT*^JO|0Lxg(R0@p*eOdmi_@&B z<6OS@n*NBp${#@H4;%qLDzB(~b|z`2`6_l??xApQuI_M;k-K8UU+KPx+j3~KI_Nxu z((1ga;YTqgE19e1v824>+|n2Q;-o7<7w2APF#gw5L}pP{t2WLnBwh7h@<~I-t+VxA zU}yhzu<&ScNM#+b{hG@E=dJ#y4+I}RzFU`sbYCt;PXi2;zb~G;(tWlWkqrgXvad|A*n+6*Cf4eM#}b2Wf=?W0=$`-P`d9{`4~ZyX zz3?{)LxqF1LD6uhWtXspCvwYw*9-$LS(nXwKz1_#&WIq^OAWmJ-DI-Nz)GaVxZJo- zQSCG6Q4^vlt;8YCab;>1ZjB;DE@4)YeJ-ApvDh#O-o_Vec?0av*Ru5-3VYA6n7qRM#8G$5m;Dse{1b zV{W=h34`S7e(HSM+r^C)gCSZE({=tt35aTr^G};Us%9sPoTc3}H)P}G>akhNZR6o~ zggpQJpdJ4n9gy%1{J8y~vD19VO175Z%nCuH5Mpb^Z5ewo%i~g^#9o(U zs*-QZUH6uqjVZn6)t6WASbf>WsTesUtC)R%80%luyy6BlRRVJLqR23~fRa%+kXQ=k zCS`OzZoAJ0}XEB)Qa%oDiGbQr(=U z*a|YUME#I$;C~1ExEvjYho0LzX}O%?C7b}CT}fHLwa3ttDHZlL>e2yXk^|It@HgsJjHD3O5>0T_c8T@di3t{PtY#FF zPy#y0G0v^A8K>B1F?0de-&}d=Wj21+O*SH7_j7+2UrMUmNwnWHG+Kcoq+d%Fn7Rxf zaLFP=05_Ua3P3uMlcE}zt6HKA0gHl5XsV&Ap~cjp)@kb83~XfXoX&_b_}FmH=qSKX z@j}tFXN(94j{*wc=2}|n@>9Re5A!3SPa&86$)pHEpu!qyWYw!f5^~0rm-tZKBJ)>a zPOoOaPEaTEsMQr&tCg^D5EyIdlrt6^ZrhSsQS$-e2P8j`EaV1_Y_6P8p&b9^SfWmg z`?9}QQ`pV2I1k64`-#e1gW?`QmFIuj#X}GL1=`CFz&!qMJAL%eMETeLSKh25$(<1R z?_oH~96-X?f7}1Rll^br%m1;%H8{;jV>;T4d_w?Pj$_q(>RvvA1$RX0c(J@NP#*^Z5=QovCr1 z8&h0B#q|<)X=aTpz-2-Y=we?Mm$=i3Rbr7cQ_*fIn@CDJr@FE;-jf z?LY@RCHO1{ubybj$!<52!-7YYY8n={v2?9$_#dMzf{*V&>uH;Eg;jS(z-=^T@-xjVxV#=lM); zz8k+^q{n=*#bUYFXf~4oFUyhehB~d zSZeQT^!h-!yh58<5A-EDE8yyr&r+QA<@-;>>4MJ4?JEfO7PXqW=e%_se($lQ=47qh zvU?E3%ekk75rY4w=1dl8gev^_!uNat#$$XzljE7b=d<7y z)5ecnOy|DYU8TzM8yvCWDk(wBZSor@O%vxC5@gcSQYJ#A_Ub=7Z@6A9LJG6I+U-qT zTaPUb?fNiyw31_$tT^Aq)s1`j-L@m!9!sCr5HI`fT^2s*n0#F+ZD_93h7xOEtVSGd zpJ%DTvhuGuU-MPEhv(xmI zO^3tDS;mKnZ5_dfD+e8CbJ>+x2=vwp=DrRrF$y}TL)8GK`Q>((_Vqv7OR#_rg_xO3 zM+?~5CEGzOX4`x%5Hl)8j?hZBZaOSvKFeg$%C2JpZN?L08oD}x2~GHD8SOEvxd#th z)QFb5tR2uRWP*3|Cs1A{2%+ZPk{2y}*VRTYnP5#mJA2;Jw+kA!u1%AA*kD2iiU(aE z&3v#&D3k%k+HOR}ZYs|UI0BUsG*)bP>c8Doh}mn}S1ZhN`$-IIJ*?4Q^4mGvKl}9J zVmc3YJ=GBN?_{i+8OPauSl3(rDL(lZIvo!8L2obPdf{IwHIiSz4W=v81f7-LETnKc z@i}dX?twNBSG9t7);8!`S3~!&m!H-4xL8Q-C)ALFh)R>%uu$)MC5@Tz0H*Z3%f=%uiS@*dEa|@ieN6Cr<0{7$^I$%0>8m#-t$)h6 zD&m>#8`{tvzY`8%H%qTQ)}7})GJVVgRx+o}*`=54V}i6~4%FeVVAqnToAjLFqBiRp zriL4XHW9Md@0_oW&OUY2{5^?D=DFJ4!Gqa-QkmnL!<^UoPll(y9+BF;W@CW=8Stzx z%`=oJ9|j9;xU_XG7I2l4fXauXo|yu9=ABiCrYcEmpny?x3Z+>abS1;TUcKFZgkpq8 z%dV&tdrB}^k0hNe1)ddjOV+Ia+}qy0ty`uaxl}pmILR9yneYVkkjnkK0}Yg1TaT7w zO9cSODWKhRwxLW6*lnxanbDy*duVk_X70N_5Xj30H>$T@IhWdNb{gnzI%%+XGVFAs3Ll4GGC>^5Uw#oQCw*4$Sm}G(K9AQFmRl5p7Yb#7KMW!e?oq_2M4FaZe4iMx zK}oO7FQ$81;6Af0LkaE}@@BRSf9x(td-|>M5a0UTdaeJQ-+%8)5An_5Ej4O^%3T9G zX4?tUQ8ow*OBXb-uY7wwKiv>5y`R(K(!N={x_(eKq}6gDVt-X;zBc2nrISFkuO-F} z2NDJ7Hx%{o|@~} zY!)fk@fp{o^efc@Fam8W;PQ5 zj*m_`AT2|oj~PHXotyf9;CC2muVwMt4=3a>*Qn2InyIqf_m!bW#%4aJF|g6L{IXWd zPp{A3_RKbT`t^G6Kt7!|RL)mkPmYCTKR@cxb97+CY>ajQ>$2rjOo!a<-cDyvN5^f2 zMZiWK)l$IDB<$vFUm*qq&8g2xFJu3-C1KX@{FmVtAK2lk??T2kxuzPMga71ngs`BO z+)k#?XFCiHh2bPtyRjWhKK(w?X_7&A_%=djA;(fuQkG$gD6@AX z#FzUz87J7bqwQt-;KM4WcDWR?ZC0(|>nKCcQU`r7S7}k;#-7+W88Vzz%`G;U9%_2@ zc+FRm(j?w(^P=t0otcn6nGFYOpddn_Pko&xaJ`mH?`4<5#c>gK~|8201xLiMEgqb&bj*EZ3}-uURY75{r|*-H4mQ`d<=oaU9Z)Ioh;jW(vVwC0n* zd54%q&B^8VoNVZUmClb`_9xo-46@f3pRy*Wt8*1~5>PhNmE-$o6b#l&qVsa-37GUW zz!k(NhdrH|-{um{5+ zF+!ZqMxCpt58%J@?S;@1n^M@Io1A?XJp}4%b5V@G{W=Lqv_&rk`sJx#Pobkb&c}o* zCD92o?ZK}Qd3%>^^m1pdTs|{rj`y6ZvC!p~t&&Hih77ri~~`r+M!l>nDv zn`V8m#m#IVd3z&x(s?j)c8Chr-eqO)U=%`$cj&hapJjfPcIB8gvANO27d8nO=5b-!qf;F%Bu-#nPwT0f8 zV~g9)uaQ9S@6>{2ZE^b!qfO2b)%wWMCx;&bLzlll`9L>Rf=W?Q%>gb)*ge@Lb0WxnAEUYTvG| zq`r2l10?+Qaw#8SJ1}SF_WsuJUT!S!`|WngvBlYZrVltiZwd7J4fehoTuP;Hkh{FF zpbGYVeFj%6G;Oza}H~s_KYmA0!!ELb;bK8GUs9|Qi^qTJG4-7lnj3`F9FoNEq z?zNvSJ>%=Z+{CNg{AN>4!h_`l3$sygB+h1Stq@BavXW6=FR{*McRKeDb1v0!J9xqa zF6DCEejuKtE$6E;8D(zP1}sntN~CX0TYGz-xy3;ff2w-68^&fa2IjWUf}i7DZ}BTA z$ik>2AX6bmyt(5zK&F6w@D3imcV|}++r4;%=JkrU_Cb^Ne>TI(S`S_f1nsaGa|dVj z!4nWg@n)Zeg}!-{+c@N-<)IAvbhW+i+8mEA=y>b4oE#T3%NA(&c@Zs*P@(?^#7eIy zFYs&dd)G(%-qPO1Va{PaUspw7B(a_N9=j-HJmzM_9);P7y~8?*nO@$v$)NS0AXX6s z0QErQzmo5+GQQYu$J_JxwE`%O3fU;{`UaV^nPshxi)jOC+3pYjwKsHJYEPj!z1|Zs z!Anzf3mbaH)yJhSO&w*q4seBkxgr3Vj(&pQNqLa#YESp>?0?_dBnN;(|7q`o|HE7Q z@AW3t|GY`rFz)eMW4X=gKUV>aDiA4o=+>Hv7Z`jrNI#sG{SwC1_)5Q+g&#F&1L<)_RDe$i-1+7WDAJi>wqh5ZU5tC?5_F0!_E0& z-mBZ8WJm&>nV((K|DNKdec;uRifiln;6X))UyKZzmwWbl=UR{RcBR1(7KFXWQBXic z%t;@?)yUc)592Q_hCkOB~KuZ8hur)0C9ZUlFeqaPR zxwx={)8~@E)A3weZlf2uoLUwirvlu?zmhyQDE+BR!>_KNxn>G_nQ$hqXS7rVtni`w zH{3(cP%WF)pdCc?`FSZ{$Bf(a$fJAcC9exomD}A&4-&xMZ2yedF|-IidJxh4chGU& zxs3?a@3n|gr?!*DUVK8y1$Uqd*PZHyf%Q)}`woX%>A!vhb=$#-Em-poMqBpkZ~%LA z(z}u62kgMk9jjV~nDx2p>SNjInBLX8I>>&K7KBk1wY|sk7X+2eN~2j%YSAVAJ=AsrIkXD)^&w)eji-&2nLE$U7# zWyOq}eMkQ&Jwml@js+24AQ${j*fi6G4|xZ57iq*TBj@5qU|l3SqE&?{w$A3jo>!&c_~=HrJdNBejB%yQy&V>d|W1W#t;gJ7^2Q zF8Vz8>~DCK$IJi+N^g*Y(95?Qz>K#?1)XQstQl;1&PGk&p5w7)wfc^3G~6CNfPY;T zzuf~(pIK>(Z>{fCPI_5g(^7H2uD+;$?ZPcB(*Lico`2H6^FHfa=zlJrnw{x3vSx^N zIm!*8%7&f$+1=hGCTGdr?w*XaZC3A7+74%(v{?lo{^-CS&6o1VFPp<^b|g14 zf(Ii|G9Cko9a+MD+iB~FJKk;!1agu`eTA5v?ENO-N^3aE~-fj0suaV zyI-;P+?$@y9Ac)Ym*1_@w2nY-r_JfA=y3nvlJ35Wn0M4c-o0;s)6&!G)0#evNWUSM zC;B&@K>KF#4~_Mob_Fn`{P)%W%A5S-K#?gxDomUDn-?qKr2k#esO0s+$BD#M1@hf- z-jBT>0^@Y1;DFEWu`&s)Cq;KN&J@-1+*7Dn|gnjpr`0%4Q+3dU1R-6i5@C-W0IEk-u zGj6WAIqQQ+g?Z580W2A{A-=Ijxn`GfJ5=pCi!+2Lo*PCESZ{C=SmU{mYuk~(-#?bm zgA)`?I(S+JO+1uCxmEBIUj{R?On-gNRFXTABCu*_mr>B~ZfG}s2D?~HTNlYV(U%OG zlIY;(;-ddu6k&bqlT%(-H{GX<@3u+=^Sw<>9vb92UkErjdqndl#2hbpmA+!$8NKMU zWHa4eBX+Ygw^u(8Ma!JCFWouc$UQ+h+uL(Un>V?yp*vtd(J|LuS9iFo1ApqM1fG0z zmAU>oj`#c`N}(f2d>gy4d%z?k>AV{zZJGTOEKz=S&F)R#%Pw&c`sOR2MmFBXz7)3Q zV5z0D1;Yyey3@>t`JJ-k+Fa0@n>*15_|JQ6GuR0E+755RZHD$?hYXQNFwAYG+|s*S ziH*z{lB?5rZMxQjrDs@QM{YWlO^w~nP3?BccKP}+{kI}b){_Mf6b940!=mPYT`q?8 z?KzviIY+~qS4Z+q<<+o5>;q6`%<#CjOybrqXX|fR&>^OxqQX4nw4wV7ul*qSS`LF& zX(12Aw%tx?Ye|&*s))Vks%Sk=EtKW;!!lMVnvS}b`?Kakvm!nGyN?BW1f z>1|r27o-prYsm`Fy2y)vBydoBGBR6H5g&XL&xcvCnJh_cn`)qKsctSgyx9UcV$KEO=Qi805}izo)lDqpgU^IYo4b6z3}{JgoM8nFq7xnp;v)5 zq8G-rEE|g?gvT~p2RN|cD;o5&2l_L8;NfLqHl6!SgN7S0e_tuCwub&&ZY}<_44z#| z$rN>aJ1M;qEq_j_pDt(!hEf z$rKGj^0w?Z-#(VwuUh^1AsN<`7aU!bSCfZ8)E@#Qqb$t>eZq=Bzib(m3e(!9T|&pyhxbKlZ_ql};hX0@E$8Jo{y0E}T65Z_1Ez#_hTc3m^EBbE8}yUDDe zE9NHMvM+7(D6IWIP{$|B6`7_aGtr)~P?%wHY~C`cD)#EMaWmK@VEn zF(bn{u>3CM&dKoHZ#%Dq44+=1&O1P2k$v;hWFm)|Coo9dR%e`kUkAr*hKIq6Fj?TN z4~bbdQ>xyrY%C5@@tBI|xcr%un`$$Hl5+Vcq_4wVI{*V$+-+=SS>jJaN5}qDy6aQl z%LfIzTUmT*zXKZ;=xjy2yds|P?$#Yn8k%DUiVG=&CnPqY)QOC>@@!P;ekHohBs~x4xuDBOu^&LROdAB_<&AVz^Hy=*!-V|7+=04 zDkgS|9>6a6p@XKeE9j8H&+`839oo->e_ z7n#xd$VsArz51U111cPT>Q!l|`04!TW$zuHkUn6qJIxJ=co-8SagNu49j-iS0*r?2 zp<7HthwM2pM%v(X$-T8?b8hM2O+I2BhV(*MGk5ZKHfmRC7RkcDoadeLd|(3gSxQ3jof;2p`)z-N+6KB>_rt zCZ)UlOx$F@3ZU{2c6uy%@p_T(f(eUmII}p!7;K(&l7gfTs)t-*L6hy*dlqGRPvIT6 zXI(t?8S{=WK49+d+XEu5pWdDyH>lnC*-cq=+)0J+l~m`lsmG+foLnBF-Wz*M+d~*!vtdDGVkIbYu%QZ?NM##=77d4ph63!wgo#68*gM7 z8p--BwH^c?1Qg!+?KM3^CSB(zb{xcsOIZs zj%gdO|I=&dvpyZ#a@VIt2f;0e9}OW8G-rMX<(2e1_^)YTdDF+piMnE5<^@yE)4R4p z*UOCAvK!A$V1It7QJK5j!|tGkJDVi_#t2h?lwB?(3N}y~@H3PSqWe9>*JhN&JMW(k z)P4u1et*2SG<0jMV7yS9Uh~6HGLI5|+Zl9O#1)u+yk-2Y>eYIH+7_xb`m_wz>I(sE z>DY(#YyWKN08gUlEg$)Bj>6a${7^wMRMF{qJLG^({ZG?$u2*SzgK-l6T^E3^i7I{8 z?lU815qJ{xPeGW*Tp99p9A=cUFH7zok_Vsaf%;1lxp;5ZXojPrmM^AqV&tw4Jt{Gk z9lj4b^x&`T?D_zU<)rKSz@=vKv7za03t%>F=iu$PVORQk_@!^Y-3Ggj`2926yU{T- zHVumwftcB**TdLJ*NO->GWvGxl51sq;AHq3U^%q>j^na({cZzTvROHq3yiz**_Pnjg$m;;(mZ*9V7K!Kqghk1Dm(q}N8~T_rAl^+3Sa)2lG-(5;U)uxnHXoh#sb)i&(a z_f2Cim#HX~Zcm)WRCGMgQ853*oX>LC+t~9ZAFN}Asgx|#&hB$=$-ndGMay7frYkeQ ztWF!6H}9QG$zmo>()VbUagcyQ{?c{crQ;S4ND4j+x`M5*k!%c z*C;u9soPx-t0P}8@Se7o%x!jz3>j`0cFBJ=WmQ|B!UpU5T-x;dI0)POe^K||VNGrA zz9=pmiV7}if`F)qAWfu*lxR>8P*8d=(!2CdP$?=>lqS-YBGRRV4iS*9Af3=69Sprg zAUSVXYwvx(-R?f;e9wKJJO44nIcMg0$6J5zZ;UyeBt}eS%gXfp$&T}dL{y-`Jjrc& zzGP^tkFX$yAEKhlnk}zxuRi+jlSg+x!hO6a<}?yjP9RECBam-Xt2VIqJuP&@SuAd~ zVOP?iw3aoNK8U-o+j=cQ_ctGil^mWHi4Q)=pJ94fPN zDxWsAG0DjVS%oTZQ=^5g`7kq;ldaK5s`EA&xpVxKTW(y9%)0`-umP`@0~CMN?hUV1 z%TE{&4f7{g`@$<`A6$rq4Xe6(qV8P|ax;kROgHe&ny&OBdErAgiiUfF;%};RFn|-Z)`~`aRz$YjH2}}!{$VSzwIFPh}ULYg%8Klo&Y5`ch-$*rM>B- z*X!a5Z<4Qw&le1S2|7u^==xVBXxz--CWd=ee+prNi3k<{o#nOQGcziC-oLHv*ISsI z$qzo5fq?=u#!VMunnt!}wGA$BVHbauf7rA{lITg2{A4;`h&uo#F77HtYR4d+(I~5} zqc!wzTHx|L6TNEX5Z8F|bF2JP^RkU>p1-83*}jdx)GfxH4e|9DO|!2iw$u=Oe>lt; zonWRAs5I2*GvQ7jq-f!Ldo)rr^KH8lB52l7)ShROgj5r|HgDc^Du%i zF}~Wivu&&-F^GF#SE^F9;GRbjdFVve%|HIG&qE@7JqP^dA_!itof8?Hduv6v>DQqS zM@`mG``}3UgPS%0$y_W1-)Hv6O?Y1@1w~pu=09p*gy6c)BW}F7Dk58=*H47Y5S`A&<9!{KY#i$u)<)TSV$b#hZ1CAc%$FC zZ77GD(6%YH^{ejm37Y-u{QhJt^B>(cNc8!2fN;_=z#!hE4Vu28_jH*@!Piy&evkY9 z?%H0rif?E4kTbxXXgBA`n^)l44+aSd2@fRQ6E(Q^H`mV5@Cg?aoth)#PGcL zacV`e`L*rA1z6IISzqQ%%Ei=Q9F}0&Xo1)E#W1Pj&FZ7y2ONp|28ds8WZ4EkOKTRnH&^CNDa)Tm>a-^X z(P&q?H_Jqk1@Ssnm0OJv(9$CR+9BiO9~7@8(`MIpw-6*#tzL&lUlewQE=J$iG|!yB zYUUZdl$cwh-|!L1Vvmu(p<_l-%y_iqUng_dn)>K+Mz1y;a)V zK5JjKhw%Ry%wOfbQ8vvP(VUdzDlZr-K^R&!@SdHl2})?6HdfDGy1^QD(cf`<_m?QJ z>)|l`&h_JdyVv}7kAZ;?^wd||8LBrL;;IrQH{PY-yhzWf5j$J3FXf2gBLkfIB2IVp zs{hZ*oZ_3y6G;*qVW+^UffaM zzeXyA9*B<~ZCu#ssj@5I?#)|Rj=CK<>_jwF2U(Wj6}v>m)nL(6*fHidBp z_}33Y$D}uXJX;HN`-UTvPw4SW^jp@1QhtoOs=@A0&!=59m|Gzads}7ZW_oVs^ikox zH@0gN=;uCLGQqxG>+>ap3qM401D}UoP(AdlG{?s~0fA(!vKeffbPQ;WU&U44xb5Zi z0#VT;H#8*t&D&W1uvW*-Z#p(~*M{CRdnTsHH-!b3&ie~l&M-dik7#MmEM~Wh6L`{a zc89(3WIW67sfgCu77>vmTeo-_R8fpzk!{TtjraaWht~S!3(Z#_yuNms*`eM(*6eZ| zuSAuv+_v`<#cqarDPIn!Lx?F1Kn2mgc05vD-5XD&wTqcD$u;D++wG zr8{;~%ti%%(f)C$svY!Gi{FVveq>uo6)X+DX&2I2;<{8-5YNwjWyBrN>;JRJQiTQy z?`zoo^*k*33iM42v1gQs%?50qjO;?@8`*9+pY~12d!{`WD+vqn*uo=hJ=t2<5yY(s zffJ{Zi>RdoNQ<`Ykbda1!Qpq!Jo5614AtZ8&e$l8!{7kkDgNqe%7#k;TaRy)Te<)q|Tc1?kH{=qNAb#_vUB$R!mk_`+Y@Hdj z$eFKun-vqas~WkPm<4K4_r%RcX3`ftpZG=pjoks(K3$i)US5rCcBL6PIeYK3a<>LF zHW_cc%|h)BqEOek@g>yaaCN(0vCVb}sxqzk)6_`C!oAC9>OIL_D!&K4-Q7i7R97`} zd0cLQ$c}}*JaDErDclH~tVGp{arr5lN7%EMKTU=c1)|~CL#%`Pljzums$TKvUH{fQ zzslK3jSnitzcl5**9oLYdym}srq>>K3t`1L#vb#9x=#H}Q{zc#iHSU0c0K3MH@~qw z@36IT_N?Ah5{{B6mnAc8a{WSK8XzYK_?sA)Vxz zsVNM3mKRa?=SqKFdTw@hr*@awrj_)L&OR^vd^_N z98y+OQ|oI>yUTUgV&+WA0SXbFpFu_kVKu(q=1>JV)c)et$y#qBIkNPKh|ldrNup50 z>yK!o1ZqQi43?;OC-Vk*$uzlz)!){6F(S32j#Joa#W-}@B0-WtYORD6Ct?lAo-C`6 zO-3#AEqL)0I6;QF@{5cWqC=|CPgu8d9!)0>m5;j)^aYYUKCU!0jK+kLB$r&(z-`+ReFxTnVj9x*f1^S2U8H}kxxdQy#>v#Y*q zy_$x;Z*yl_Yg@(A7|_X>*w=9BNm!5c!^`fK-R#u$GB~uJM5ce64b{RBd|%=_HsIH?d zb#(ix;EhEXw{LEJsLY(ID#%N#HJ-oKrG)UB+YICG za9Fd3*>%`xo@*!1uE?kM3ij4XNAj}=^WCtAXRmhnb%1mCSHIUn1*ZFUj`SniJ*x4! zZ(s$C_h7Pj9U2$PTidGWd;3HW8Ij-SUYn%zJ+{!h{?(gMo6kE`)tTkT+1IesR0!CX zzQ9@g&7=Zf#cpRX|G5+lAF6cpQ~AJvJ)3LOjuIM;j){p1algZSJ32MauGDUWVG$DG z$neCJbg$|l8s%>EriIJ!l0=@l%%de#n1uEtt$c`lukUp;8~uk-SX*fF`o9x1mKklI zCnZ%8LPL{x2seC7%(|GX&S=l@-kMRq<0<><#jE3#l!yZdZojAHI&j-FuT%W-;SA+8 z?t2mR+MexZgw@6|scR;0Z@nE3uQX^J$5x3uO;?mD2qqj^~EOSJ^5;b-F6I zxnfee`6BTxhkS3&nO{F28+4%YDEooRb%IYtN=k}>x3Gl7E^+T@#pYZ@v#G5%MtG%g zRLu65goMOEv2W{!^I&L#{hphr^jQSv72O_2>&sG>+?hYo101cN#@ z!L56hwBsFn((;hEoQs2%DJr8(KNaA-guE!>!*xOL}c4NxBi!7dJWVBb!ZdZB$mc zvcbeE0x!M7@>qRuuKt_H^%rHsg8Tzr@dNdt^=9(9B$H3`XkyL<$6-uGjB?SvH}Z5> z+u}&`@aD9~$;M`Fr0Q)YThbYa z8$wtlms|OFuSVLhs%MQ#2if4tS`#O$A{AotUdIVo?sPXFrurly=djW}E1an@i1u0C z5x#1QEPm4QmX<1e@Jqfzdgy|k-3GTvq2nIsb=$mkd%8Cj#Nn!EAFu`_r%k(cZ5MIp z+1?MgeefRx5~*0=p2k7 zH6Bh-)JTy*e+_()u9)afXiIoxT>VA={Sg0b`ZG%YF3cal?0s#@ zY48s>;`a2!V!H3{qQlE`e0?avt_^LH`suP6+;G-%-b3Qd$0X^xT5oUf;>CWg;`?W1 zHP+aw;UFd6dK($}&4-bq@|o@mVIWbSmOocHgIMXBqnW>bk3$%C_M(uxb6YhT`2vq3 z=@q|DN(pZHToep*=md)FKI)B6vm7X~_S##-^kok&G5u)V8~w^UIF3i*7;nxMR#g;Z zwQFl@b&BmaTd9v`AXR;x7d?B4iS-JlHYBY-`$EX1A#6<-(8L5IYSz3@-=zr z2WOAtd+jrIIKR3MtXZ@F+lVG{dS*?NE6BrCOeX?!23zbQ0 zZ1E*izJ(ti);wEYUWS{T+ikuj%MkU*l7ZBYa3g?|$qNgsw~20@#O!*`Y$S*WP|Xzw)TElm?mJB_<{|*`CynL2fUf;S6C~)-o}fNakoQcU_qHtenPp zB2X#tl<3(9u%2<5=hC-<&+1t|KHk}j$#<5=CR?I`8nM%Kn$%^5#(g*FPu^xOE}}Ve zrqCQm6e~2lvA|OKN|)4pe-P~a?!+sUTkP6aa|DW5R^dHqRi?Y)8WvB-=zKGw#_)?I zJ#xI?lcF+|@bf;2?tB~Y-IO|?}f7NXkCurjYlkI%^GaZt>zXMetLi}k=b2Tk=2Ol zMC1FElEkk1TXbZG6DmG59 zsL;uZ<~{e&b84bS$;=UqCrY(SpR^!2i@GR9ee9%dPeDB?5cR&x(dN#bJ2p1i>-kt4 z`ec)V)`MSzP5AYzStuwbq~zq}=BKK5#7>wVZy_D?s1J5iyP8|dwEHTKZ{aF+sQY?w zVAoat?6=u4q zGvyPRaT8ys&}GN>3rK>oJS3N53Qv~ChbyQVPd4NZ-;Cr-aQw3)gt-Jo-2T#GXw!Y; zi++{!g0V%urj5$6?|JP%Bq=7YoHL@JIDhozix<0FDWd_$>96ROPlVT`up)fl@Ae=_ zmBiVf9ccy)D*NhyTs@Wxz^ z8s%vwredGL#CDkzurJWu%;I8YSBi1E2$J&o$WmQc&_J<->;};-WjMmeDtWStgH@O+z&JM7BkCyij_luKt3sUZ?yKXv~+ED zSWQ%tQ|o643ExUQ&RhJ{g0+SN~i`Vcv7TD&P_kVoNrde#H6Dq@v#xv)FO7;L}k4IfB$cd2!!o` ziQutELR9!e;j%Pj89BrQ%9D_A9T!$nQ31YwJ-^1V$vcQ*im>e|2a!1IzP&@TV>!- zfuxK=GU~5?k(YAQa){C3sqm}F4xggyb5RREw3n5`?N7tUt&n_s!wOHo6i(;cQ{WRX zjHGo(yS%A8O@7QDJkiqxly*+BidsZ^y1ALB*UE(4Xl;OLfyNLU9r9aaqYe+f+or}z zHrT%=QOZaA^TUtdBj1FF8v+O;At9*JP-xzMoc7B37|f1$$hU8Ip4Wso-#&KiSaS@2 z67r+_Bx{j&j(#4jus?m8T~3abDdh6?g!ASXfCUq7?coIng5*n_?MkN)PP_N!LhS4k ze7^gNw)uPF4&3?vWtGdVl*f*7>6Sh*+vt{-mUg+R2j5UpT~+-YuDt!*OU1~@=;h0o zu{;+gBq|?%M8=l2%YggP)z!V+`r_5AmhZotns4k=mHaM|B@aQ0Y!o19lVYcjV?z1TW{Y7MTprE)!mK_DfJ@PPMApdyb+s+OEef;`lG&Q%{ z5v1Od#lcYSTX}tp*u6c3+oDFM>PP9I-s1ta?4}aFyK6B9zJ-!Lo5P9nX;qsU7cN`? zdz11|==yZBHHzPOS}>g92^qCCX^P-%5A?aY@qKpnRQ71=au@L0JPSpcSj0C7*;-WR z)EX8!+s{dPOYrfH>?GAc&PSc3FXVWXg(hy!Y0a!m;@>*oB*Z+e^t%W{IPYWLPQ11d zoFQzjt{^Y(sCx2d!ia^CudqwJ%y=X}s`~ap1em73GLtrIeJ~7e(@HhCc-yw0`2K+)gknhugjTJX|qN zc#qB$%X}MGvLNBZBRT?=|94pL04#9IugQ>0S~BRfY^W zegldPf0*sXE<%bA!PjPZ;M#1xjR>xSaGw2IzHNp2k*d+E&7+)`F5OIp*9+G_)~dD{ zZag?rNRp#%65j%fs$ZWRad&%Tfv~VJOjukeq}rUtD5oj(mr<1I`p#|LPE%(0x9I;e zlGch!yy-&5)xJ2LRgdA*3keN%=H@L7`|1Z1*FLU?l5pL(q#nD(sd+m)&srMSu6LrHa)T%3>jUBXM8|HrYu)usH{hZnUS=+4r}YY1T$wd^$c_4R4GVnRW+ z_u5Q{pP%2aUxow?dM2iuZVN*YV~5L}8hb4SV?W)GRZhNhjFXcSg(9hCsbeu;cpUJ( zIVB^WSGl;FI|9hhVfjd#C;Ie7nW0jL@v6-cz|wS)$Xej41)gLUu^lMt&CwB0{p7W| z-~)*{VRNxWlSd}y+O~pm!)vY5C!?uy5mw!w^HC&X7=oyIIl^aaH0V^ubz9jmRu*Ba zZsHyxn3ij0EX%YtS}*@`fJ{n0iae9I8o2TdFuqgYW6PYT;b*!Bx|8ox1t71%2t;w- zUF^bo6|TJO>_+1aA;=f{Ri4-ZwDx>wnnGh38^Uhr2K$}CV%sDM4?z@hH~uVRCwP0> z<9D0iag6%~pbs0}xn`oMc+V_bpESh%2ra zu^$=AH*Wm;^=qP}7ps`#w}DcNj^v4;%Yi4(3Odg`9P9!K@M{%ZntuOtswNP2DXc(g)vm}*Z-T6h(7iskokh1=fF zy12{lBHKX`>lByjUXZZ&iIu}vE#2@+pkg+Jy{x?>+1;;(78}a z|87=@M#GzT4<0?`?BI|um4HC`wkJxOe}8wHfdLfglJ)q;{NVJDxV~~1yqMF}7J&fH zV+W>?26^bnk-(rJ90p}T`R3g_=;PE=G4N?_&OU$sysoYe-;=dzg$@i1JkGu{*=lah z_3(oq)+Q7JS<`RW)Q?5X(FCR}Ny^8{)?;VQ5_H@mt1{IeeEU_8RTT)m*z)_u9d@nTLubW#P)G^z{STkBE^$jD(eEPfr*4&u7xZer`r+=Oj})Ne%`j?P3b5# zta%0sr^7Ej_{FHsygkvj|4WVQhw)(k>U+y|4z8}bk5Xiuor`FamDA*#+l}5ycsPRG z>M^67^F;gMa=J5Tu6u7SIZn15V~6~w*lzeeRK>B>6cnl98?@E|$TzWVIZiJNlkfjR zUt&N;a;sU@zoH`xuP3tyQk$bE7zIGvCJGH~O87RHmro<_$jTzzm(Jh3d2@U21K8&3 z&N^Nu}C;tUeg|B7##%1hn+_^CK7%v4Um{%oJ_iSG+K3&K`R|`c3_DI&SB& zZ!ZN+TZYP<(&4m|OhTnj)9}_G(@-{q{^y768EB|2&a$xV_3GH4CDGB)JQ;5|If(7& zCqC)?T+`Zmb$l;|OQoT%&KQiZF_Z-=(FgB^v!+Q&97M`5pt0<`WnWJR=a)l4}etI#ZiLDU_?Gp!D;I zw|%t^dU^2R!JO685J=1=<@x<@Jo)t%`U#$EGuNDFej-S_PHpjGfR6C$6rZ4`W)re_ z2pk_yr9_^HX3+8FdIp1Ry^7xQXcs<*j^w)mG&&`A38MD;B;p>te8_{aF+bTFBP%1L z%}Ghg4&Dr5(=XuWwt1L}9*F#wpfzN-UgnrqSXg+`m-UvqvXuAM6I``VmFId(5&}bm zyd2S!XFy5`lX@@aWQIgO>$W<4oEG-Q2nvefR%;pf^YrPDfCAzQ`m!|muUwe{;|>W9 zrlO=2U_1e$0Kee7HuK2P@YmHos~mFXO~`n2ur=xi6j>A=Qo~C_2G3-*;G4dT%>EeE1W*2d zhrtXd#oVLz+tb~ArhmR-j$<`&Qwa0f{oK((p(S_cnIG>NSbJR)5~EH zkC~sF`%ZWIjd+t9cMk+jUowD|lb@gOSSAsYA;qB(y|%Kl1MLDE-XLnWT;;S^E`bIx z3sm$xr6$d9Z@T=xAtqKUSmiH-K-Q4(fOP4A=R!2PXIS>-aW=VOP znb~^2OC)A%q{^$KqXW``BHLU@Qy}2wOZsGRwS2|jC8q$@J_J6v8lLFs)2CMG9neK` z6zk7Vndg?{cXV{znMnzI_UswFdK^^Qb8Bfdn1P?qpn9nI7U;>wA`Vw#Z-DQD)aK~n z!vLXiT}PLfmY!e|vKXoK0A~w#MeHus!Bar!CLY&|;?ukNT%o+Y{1l^rA(#&ohrW9| zfV;qhU1JIXtyJ*3i}yGUqXy-6U7h-2@DYVTARxN{{|^}n6*~ld-C`RoZQ=C;6t^5R z6!R60|M-%Zw@4yxBA(X>JpT3~`EIa2Kzlv-oS=Y!mnd^h%`*7K@^bm8 z{28UZ-C;K1p?sHq-DU@d1(*Z_kO!Cmv^%ko=WWb%T#WGflm&T`zP^5(sQn0d z&#WwKP} zP2=9WB^b1ABaQOC4kM6uMBcmG3F2;wr+?BNub*fJTT9$6&@)vAuPp95_b&O`JXSJg5n=I5&*dcp+b%3Z9n1*XUs-@bjL-4hoRTVKRMqPWcz^2u}kcklB*!K*qs zIYn?c+lYV*DTO2lyVGz${5+)MenaQy_wT5us|yLDDEL8$xKaEDz5oDSRnPvSFC`Tq z<+G#76Afa>)hhf1Mg1OO69ua*x;{@s?3NBXCHdqRg@$@PUU6zE!m$Ag1M6O%#@=vU z#jK-{uE_VE;a(W2!p|Tn?Kbzp?ZIR9oK(R0)=9zp)9?{Ix#)1@poZSPII1x4E9kxq3HZWpJD$aR8Zvd`u_zX z_-~7y{|;LC-?-2O^VLv5^AwTRNKk98xPGYRaXv%^9iH7wFstG7a}11a@nK?KQX9MV zH!5tG*Bt?t$B~}giefqtmDRVfStE3x$v#I>)N$^8TLDOQb)V7{X^Gc8!Uv()HD=ga=NY1Xsg~* zcj8+8YC2SC(UFlw&e#@)nAf49#ddv$0-%E?YA_~>TbXmG85f58^hx418*5`}r9spY z1z%8AHB0l!96z*+b2ZUZ%)BEzo~{qQYf1KYvnx?9e=w_MX_>bT+23^Y8qD)fyKdfo zjD5F7qd-z|x%+Xx&0uw*Z6hBD#O`M))?d!x?j4!S1~zcz51frkwX(2v3f4xy$V4k* z%04uR{KDCcA5(K>tH>8_o|}$NE-p?kKIv{s?BB{(QK_2uvwALH(vjrj zMANOx&j{THw_hYo00Mdl%oq%8v-}n zCo_e?Q)!oY2GESf7*PL+CBRPC;Fa_eTNpMncFbyO?ZJg@Cg7wbHt4B8$ zBAXK#B+M5?fsZgeJiO!F8EJ{L#ai+HS{~=b@n~L&XbwEZ8tH1=91svh#eST&$sV9S z!`nJ3A^X)w!QIn33LlXHHUVTwYh4`-6WDc;9KHSItumJa5jRYbYYr1L>3sSeLCJnu zX?LZi-YK|x50*{serx{qOBVU!L0rjkNXPBg`X&*@wj=LqPL<5}XQ*fFpBzw#Av(ir zxj_Z(hpUZsK4`SFqV#W~CTq3Wzf}>aVHj#@$}IgAc(|{4u`g_jQSP0lpQlt&JaOX4 za~kBwhnh*+?IvGD1}?$D!O={-T7L~(0mXT1Q^M)M0pTI*z9;Ry54N_WsvRePf2_kq z@jR+@NC`y6@99yFjxO-|mwMo%X_6JDL*>&76mQL+aCP6<96yTMav-z~<3(*=2M3pk z$jHhL($Dl)CqO>7zMRpO9zX+#^mFQ?{q#Sr(A2C_JQ~F|Oz0{eH%Q_%v`@nWxSL{G z8RrZ~MV&BPj$_R@rG(wRNaRP0{tv{NM@if6UR3NCE~fa?R7+L!_l&lvSyv~rL?%D_ z?Wo-Q;N6d@r57(42D?7aP@Ex0gh>1cteEZSU)7Qr-B*Y`gt!y={YWy{DQG|-! z(;2^0|E|5NQev+~L*5u~XP1klq^1r-w8a-Ga&3-5C6?smI+<7g03G8{*;sNKEjD z?0U~7`VdsI(Y`-g8bX+~!$dS03h9<6zj5^{utSMrls|Lw2#RzZF^7wAwWZ7I?Vfe^ zaXq6C*4mJsD`Y^ye%;}Bn1Dn zdftt>Hxp`}J6)GUFpR5Z(44G@8*FM1;zVdwY{T z3W2~7$)zJh`Kb7QEQ^k)u;tLbG-a-+mcWz0?FTED>x<6&FRiT=jSOpLsCvSFK085V zV@u@_J!jNZsv1UJ=?ZEAP&TxY_W|=JyE0(SRYHLYJ#lwW&zt2maQ=kvp}@Yd`Fx=VQX_ zzkRz=9yl$;6q2IQ!~yf0q5%JKi*Fxug~q`D@sIs|VHz9b{mWeibXnRZc-3^@amC@0 zkt*{fmX^1fCLM20$)QNQP!Ye8;5s)n{}~qhQ}{_BV;BAeeHa1o5j#QE&%XyzD-((_L?vjrp59}5zd+^Jba^sV! zsi~sc#l&A#ZbyB0KcY>y%%S<{QQ$N<#lwF$r6W~O&M-w(in#QgJ?TVIvA3W}e@bH* z_1tjOWK`wCze(dwTVyI~VFbDZnHnXbIcc|_@ANj<@bRRHlZGC^mu#Btz0jGHwoXa7 zN)8Guq0WCutExuQ)DE_`_KwZZo91gmJFx(r&2ab59ZGiZ$|_sl>S`(aV2;a|d)xNO z!yd9XpWe>vqhdF4jLo^5@sN`J#1ZM@==J1jom~?0oH|(>!rdyzOw~*UwRWhmciS=m z_@!v_={$CVJ4&0&l_i~{Y<>-XTVwY7;l;(H(APpeHQcSSS9y(%g&(D5G)4eo@ zLr_ru<3o3McN1$Z5hO zuWL?4_m#Y7)v(X-P6!ic`tP31#2ed$QC#TvB=JGwzeqsT?hNwX{ZHT-=7!zQUl1&C zUw#aG|L7BnZ3|K&i|mKYa!i^F=qg+nPlmm?@+f%=6lAXZqmQfVO8Iz$Emf?GqY3{srIv$JW>XzJ|9fEj^L#^W&MCo}Qioc!VNcp=r&tD#hAe zF->_Nj;Ej?{j&_nR7k*I_D;5bMKvT}n`{zIz0U-dn_GIUaW-8eqx*6zOB5XDc=nv_ zN9^Vg5U4f}eMGZK(EHBM6L_rCDHf@*rN8Xz!rs>>ge26TXV6Y&Z8S)45i947m`k(0 z3F0qb4tO}39b*sl5$SyT*IeA$dHQji`9U0}LzQwF|FI5hproxWXfVV$)?A?zP_P+ zWVBa&T=pdKIPn#F-9G27areAvF1yUq+;x-H+=mUe`gsFlT31h|c+mU$;UJ@J*&I z)YJtzO`~)WUq)uHo3m(=%e-)W+I0>txpeEnot*c=nL0`ISGBusL^v;76=DSB=z~pn z{hOn%s?Bv}r5|jC^C?!IIfn{F@{Wrd56}*FwQA(j@@ju;ZM`0sCEptOAda+)AE+J{ zUl=MI|0)Ql`&Itp$bGx^tEO()Yh!(W_$nb&u5a%s_4E(YM&W2uG`CjmQ&aMNJ?_XH ze6E;Em?Fcs~ zC6dN#=RSBSFz`N6m4q{I<$~B>ym)<1c#6t^E$Pz(CWA_6?7+m0Bw4Q(C0Q{qM{-^Eyf4TiXdeBbl6CXbs)yaD6=NUyD35g3KKK8z2 zlFHH|ABD9^gtQdCQ_I#aG>%!|R3t9^OO*fAhwpO7S_ru=Zig!75pe0!4=#6s`~g-q$Xal|^W+H)(r%V) zo?CB@{v*VfM4GV*-oa0bq@9+4LPbSIEq(ccr|eSf>|cz!h>M%d%NQdhWkhrLC!cAk zzHMi@)BgH=}N3Q5a!o2WX3P~-xD`Bi4b>NxODllh<$JOj_SvU&hwqK zb6BS%&ufVFWC42IfykYs3#M;#md60g4{+&fe zZm&!_IiGB|6k?K%AwR?~1exUZPq6>g7^)NW!IO0dlDa7=y?3|dMCc-Op?(&|ixkcG zPZFOWZ65q#wz|Pm;koI<|3Sa5ULi(2Qv!bayy&Cq?oOSO`O+uL$|6y3a{;k%n(|p4 zd}B->TyG+n=I#Q;G>&aI-#SKdv5KHLDIaxCnH{bSZ>_vzNa?#~QIpI8z^9nkg7*ep zWT3d)q7_=M5R(XR2U^~KH^D!TtsoJeBXaf75+7W7gSyG`o{&GA@YW<>6ILdxcVKD< zhbv62Z(CaQ7B$0erGv#D3OVX&?v}$159~K*WXR!@qCj2dV4a&zv0epnbK<2ziwRb{ zJ?q8Gmq5{pZjkFr^U9O&u@biHb;Q0>3*{qMl1(QCX&G51PUx4q&;G1=E+2g*arki6 zvx?0HDs})>(9#tm)*8Hv-5>XM?HO|PFdZE(Wbcw!wbpI~yJ5kuck4h2P8gA#fB5q% z{1P4VKEX;E5>tN~7_JBKP|Hs;!;TVqvN7Z#RgQEbYQI>n_Tu|Ui2_8bLM+hNewYQ* z)Y@D#N7V^G zWX%|`L&8I#K4;Vap&C8eJ$oH`RM4h(urY$`zT)}zzCJ8duAfknMk1k2*c#cK`$|7i zmyc~3qtxgl*8JlKT5en>y**xXt4_(u?8g&lat!-wy*RnB7H{k_Kh*p`oy8GRppM_FK8*2*#28c`+r_qzlB`;!i9^>LUz%WX~=*Tg4<$K@!LyxW$r$B zFF4<=3Rz*48d{lZ3A_%?ulxc}sOf$qjyTz-@W}vo&foihKc#W?C-Y z{(-UIzu#8fmD}?E&d`@jl3dS{GmG$ODLQ-X<;zxjl(i2RkI;R}Yy1}K9T)DN)7b~O z;!@cdkx1zPng`6HD{E`tiJq4ZA4S}BT-y?LGKb|6`jpEPvt9P|Jl*>VqPht%mjAxA zH5GMA8Y)4V6o;+QQW35Um7oomKdf|;qOa%=L3ViQl6F#HV8EpR1<-##-OGDlX_%M* zFY;RROm|!#SZA(U{_!lptPp!DD(zvlF4lgkt@uH3`IF_x(3PsJs(Q;w?k;<8MbllPpvRaZI*hTPQD|0eimCey*k})USa%RQm@>5GRx9n6NNDt z6vtG5asetZT<{|!=Oz#fk6-0Jo{ zan)#~`<-vUl!B&L*C9Y}ZzK3LnX9U*TB5lw`fq(9d8Npp9({gT<2QhAOBBbrk)Apj z%MN1p#}U)~Q?A`>lSQ-2br{lfpxc+b;RPomo(jhDv?|CEQl|}kC50@SzR-s7>IfO+ zJHJ_9UXUcH=juBx5)VS^n`rL_oL^4tqkY{=uUu1mTcC6#Z$DUlwbS3>mV*IMtepDC zyzu`xp^U7lsUZVZ-`m^U{|2faPPN9glhfA@i+^_d3Sr^9*=p{v7Y)0D4DwKsNBfgVf0 zHP-s_?})mN@5#~4PzFOPNNMVOrk&&ZclI7R%a>33S_8K?-F_PYy3qs{uSX&@;K-e4 zbfvGMB>Npz=e44<*IjNV0LPl)XegT;2l(+P$&KUk~^U6BhcEi;wXtRV<2^D1( z6%S&!($LEbRRrXV#bTTJpSeE<)LO+|ED)r@yVia7&Z_%2%_W0$@7`&$e@i_!K2X^s zPx*-R!vgKbz9IxzgmCHLJ%(tDU`87)?R^N~c>BQ!<;O$NA_5O^HkB9d40Dhj8trBOy z+G`6Xfw~C<^})v>%)&s`egIoVlP!b)YQooWgn<svheK;Bb4!Iq!VOK1b{dnfIO`Fzh>K!L#LEt0Q|ydvkGH6U|-E4j-=^ zEKzD_9VVl9i)%do&`wDbHXm7zz*q|@&JPrym2&=~QV{#7Vq;FP%yWLcQRz4X_nJPY zdU0(bmgR0Wed+!tj|K99e?0d|Wo8V&y&tMgqF@wc$%e=L|22u=R?SVcBK)w{C@6my zW0tY3y!gjNvM|X+`*mV?5ED&aJwwiIt zrm(qd02M=9qNku$UBhBq0;z6eZ4kPid)k{?ekzQ9sz^1|3(Tt#t!Qc z;x-b1s96MEp6ncQ_oB&Kf9G5v04`CyutuQ^*#Jbf4&z3R4Bgpbw;fF#K#U^Zz3Z>k zQo*YPAUn_s=|Dm`d*X;J-M-eG;%9(l21!j^>LTpzn5hV#&3D~v$U+f?@%ULm>}qs> z{|$D3`LtqQm28VgRW&qpbn`F%AZI5-k%PAcWUHCLQ0u_R?NNqV1o5{#jy&+g=c_0%lf0-aae^kFJ)hy8L`| zTGZo$WqpSi}cipN(bLdkHZ5Te-`BJ~bk==>adu$8-IA zwN+=a*>E5*;;cy{dQ;QjJ0h@`{NIUt^?t2M+VIK4&?M_(3jo(1fik=2tW+=)`7W7) zo>7+|Ok_R?^7GS%iYf6yTB=u6>MJ$t-_GR>gNq~Oc0Nm6%V%JDWVv%jv%v$Xaho$I z6$-1#`0jcB=9$xhY$)R7as+4aj!#5RPpa+UR0!Z&VBL5vCv`p@8*47@q8Bt{Q#pg; zuJc+&U*DwdrykN!Oua|r@rSx|RXY$mS#mn)OtrTNpN+X#nq=hmN^d&I2%rW8kWmM! zou;8c8^|pnS5?2FOs@#UOF3=q;6&+p-^C2~m^2hjUR6rGIdAZBH~VbYK(Xcgj3b=s zH}orQaV_3Q&^O&2{-oGg?sW6qFSgyti-i6O2<-p&c<+lB?}+Pj#VqCB*8N2)eXLj2 zbL8aZ3#3gm1mk`Ty3PF_-mLQTJjnozgx`WtkPmMn*7B6cJ5m~5`SuOjG}01dBfz*r zy>;o@HHTk~Y=F?NwPSK~bgMR|a&(-`+l!QZxLzkgtK>Pt6>ulmGcyAcgv?f(9lKV@ z0sG5#-N9iKVVR zE089pMnl#+I<$@JrOtNS`L5;F`&TJGdta`Aoa;iTR42G3Mx&TMGSG!t6jnX!Fj>|- z&VQXG05r^3py@^CWFLIXE=tkxvwn_MhBC+W_r0m!dGTY%UQUxrzgYb!7qLedkVXUE zy?Y0wGRt@&z4FaW)f{zLm>iR%8TB~!WSj_TyAM69lHO$EnoA}hH6?V$zD-rmi00G& zS+VG4zqw#-o1vfc8_+H@lo8i(m74+O@^ExcU7)w1>sk{)=t-pc3!I$ku+vmu`PYAd zpT~fHwgKDs!(GX3GUSVG9G~FX2gb5t+~XQM;BD3O^f1<&(2^V}mUd`{X6SMDPd>sK zK7%627oIM73C~@V;lPmIuZALVP!8uS-zE9_5dM}qg94|31gtLJj*&itIoF4Fl4ZY> z;5Pupb*p-=XN9b20MaPtH~`kcG|6uQ)kmo#H>e+=xuAr7)m`9+}-a$ zjVr}WoE?-bPpeROof#;LfR#*;8f+v~P%&@|8SOH#44_@^aga4{zsV)gR?b!dmh zL;q~_ZtdW6pi+H#;s8DF%%03av}_)*KMJI^GLA#~Bru!RU|*E2i8c#Xw*}e;knY!# z4qvzcs8JjCCC>5{~5F=I?XJdhPWvhJEse8X zf#}U16j&Ok4)PX!NZC;&Hx4?;uV3!H{wmFtYmr2*c}J=U0IJy!q~_BkKdL)|eBrT* z0oJvXn;XVuTJdqR>qxv7@`X`SNRbn{c?+(D+-WCnb6&f)9P%k7M~UwE-DyP@Fk^cG z#sW++=?mF20dra>7#JV(4#8&j**aCpJ{2NX=$5v9LROjP&8O`D5plZ3?vmdIA3J-S&UK->|E;cx_gZ$Svyvo~M`51cSuEQuY?G>>%I=wydB?_}x!Wkux zHk!|bQKIL6Vh5BVL&9E9A1ou29M3DsO;bt(uboMRQn$dS9(r)b#>Q^*SaB#)kd*dp z_7#zvY}d_?o;}&nd%fOs9C*%>ywwRG!Ixf&4n-e!~OmJhlA-iebP&iAHO(VBg>HW4bg{jWxBn@ zr?mm+yS?0a&W3ks>YpyXzykorYMBl>p`=H6?YMA>0T*GSW$Y?mpL(pxvZ3r3?pPzaneoS#& zjj00tB3&_CKu~Z^3;pC_qG`fZVwf-+CXCio=uc+=$aiD1+^&@SlfIJ*OOAZ8?gZau zxlFGIB=^vX3n)zFsWp6&+0-;G=R1o+e{Wql?sdOA3`)s-b`{7Y+ku8pzccpKMl~WL z6&RwKH7a!XYWAST)Es!{M7Imn@|M@Lbm3!f>YhRq>48R*9er)`EA-sVLyNnD{|K&6 z$zN#0-~a!!_`iO>x?<4&C5S-uNDQT+qPcY)yq6j-`ANyB30wWAAQAtejCRVWo|hMeEk7tBD9 zr#_5%9}>b0RCfID$B9*f7ii{y|NYUTtN;1mgCeQqV$TDsi`HtqfAqy_(e0!C5Z8px ze@RuC+uY0pCsD+|B;nv8t3IzZI_G*?pnJiQWalJ4jfqBLZShJ+fu#XVuEPU${aQiC zUur#;|7Lif7KFRwO(uofG5mveZ1F_9MLU|~-qN7~pEkiNyywxgn9oE00&AecWFP~k zmqVB#iv8I$Jc>D^;9p@f;;zO=77kmDm1Vdc7RF% z|K!S*E5yv|#uxbwpBO2gfl)B(*xVOw*jq~}Q)Z+T2?DNRi`!;Ba1E=0%mGv=+DB8!eBj6v3E3pI_li`?mF->fOtq9K^IH~L^VF39Ky&o2_5QjE?2Sj z^mq6ikO4k}&KCS&GSEHnQ|uOdSOzcF$15Yk!$Hcp9ynHGUaP47a~p$c%qy4Pm^(nH z2PO~4lKwOB3qz}QKqFA;zep7gC1zA+^*FXKS52gPVA#*FSta`N<=J*83#4)hY=Tt+)GTc1IQ1_z;=Q93e7O%DmU(f`OmII{sZ9d zc4g#u%x{AbfT5TYP$>pY%e6fAqVRQreW}uKgu(q0ribCq@q|x{fKrKk4|FqMUtf#S zl6eEC(b?HqUwoL#y)}Bwz#1os?fWQK1oEhn##RciwfT7}pI5J5fxPevunvF?T&iyb z(}F3$H0VH>G{Sgwn|?{q#RbqbI%8Ob9abXOPex5p#< zXhVbs1`;A+4vyMt9nCA;m;n50ZyEjwjG*z)&%9xwChA-UDn?*J9;W_DNl8bbc%&2! zQNBmQq#Akp^9Rsu{BAukbsK}5p^}ImG2~OBNZ_%VX+DRBGOU;?&1jg>Pz zeE0{L1ay;6fiDQ<%m^SjWx^R?hH&=;Nc*)BwucWNu2+UNO8u^NVxgnU2GVjxH1NoN z*Sm2Ke&9ph3#Vr^tMpv@`32Yz!$2(s`onmIMJZRtR+CpT{px?`RjU54c$Jd#i#X87 z()vI^)d5xrJPb6M?_{HPZ)K3*`S0hw@XuUyNrqvP)C|f++Qd?}>9) zf8wm%&w;_p$_g@yZteE;;xF$6I{xs!97k;_3ZEmJ$1YnvAel zEIG-5U-%QYC$OS05oBEEhd@q6lf2AF`)j@h3cJJ5baoE4OpWiFSTQm1KKG|$=0y->MHu62Z+kCBt z^I-1DC7{4UN2$_sVr6<-w$^bS*l(x%<5mZoQ!E-aW%x6|9Rsu!`0~dPz04MG7nrIoGJMJhg@8az;FiFRw-@gY!EFu^%Aios4o&h zEd{=!9C(W0_-LjE2>YMQ!D7Sru1(h00^3G4e;?eZ`A}Y*!~Wr7vX>`(AMi8Lmu!wq z35vi$04=l}ULkrbnGUY(E)QJAz$FDH<@P8g46zN}5JF?SOG`@-9Dpl3?#SnhZ}s(P z0odWeBB`is;GpfpF7?HS33YSu^FV*~yf|}0{D9yC8HmM1p%nArIUxZ-_<}lkLrV|_%q24*^?|n?!wjVn#hx2!ruAZrwMAcn#cTC?WS7 zZ%`Z_9i@BRX$2<)0UcIc0YW=m#p7gOftJX%UWjA!mt*&%Rmm?g9_H)QQN!H=agsyu zg=n1sen&A*L^(r{Yhw5bVms}Ph_#yi0iS6S;IUch@p)qj;1IlUl7 zbDZG4K*SLM2MHMm`iuvlamf#Zb_+p!;WWUQhT`8Aggs_vW;Ui8ra**c zF5Y>6O$B}!e1!!Qyf}!puV(q8L4O7ijZxUU@m61#H#fC_pbD%#gYCqx&Ut?fC?p=( zwH?k%Ztv`v{rubr{LJv=KcDu0f|0=cGytX^w3)%vshr%&-v)0977s~pjKiYB|3kVi z>VHGm)fVX}+0Yby@nYn-*Vd%LH|X#FzI5Mnef>1#msd7bG&}b?x`y)o$RJ40$wQOOWnqtXx=3)t#Mlz!fk)8J$xOKwV3U8nq*}sa!}2I)c~c3oWSOm7Hov( z=@1VW*UL!rTL3^G~@F8JRG#sFlyTi3!5vSt$X$t_^LI) zON1>6$_ydU1_VL}yigJrP4yz}MqG?H@Z`Y%1?hwz#Hjq%`|M48SY2GF=yju&%Lo!l zJvuv>vJgxVbQ_wQ5^he&?XsffP%DH!rbwy_Fi+w2lgs=?Fm)1>h=5fsNcRvYP!49F)yNrlMo4*3qyamI{T^wHL6y+0Fe+Q@y=1kk-etHMwTK2cBpQlUg*8cb-Zs zmgtL>t3Dtg(5=mnA2pYL^{U8&QUziiERe9#-)V?$E}%{<0}8eBp~>y zGgnS3Ko}b$Q#EhB54lVg1RcnaBVnW-l!jo7tJXSN0EZqZz=o5)^HCcqVY;3lQ#YEl z2OwPlA})Mp`}G)Bs?_al*-8&f}B$x2Z_LS4kTs*x($dIR9cMh!+TLv zDw8Ime4Z)X<0IPm)vI)c%U=bQTPAZyp>68k7g9}TH>zrg%tH=Wk zpwQiC8uA{<<234B9e@O`^~E2U^&sQ16cs4rzz9kpZVO#^$2~sDJ@^ELAh4nh|Ndzl zA|#hi7ODqp0?`#n2u8=+p?rjtJ{y&7*pm>qQMXGn|NGltSPCJx{X9qoz`sJm3zxP4 z&UwBQAsPtLgoLyNG88`3Ran~|N$s{`+l|USs0`6T4@!sBl#~Dx79qd{?2W|G=uBv- z$GxP}x+`u%Js!4}+JCpV$)1?XQ(Nr;3G9sk)y{!V+v%)2^&$ouj{ajX|KHJ=>X@&2 zXc#;!V$^cPeDPvY@js}`#>co5R{ibb)Ja>%+5bQ^QFCs8_uUiJ{5Eb5|ISA9F}X{VmI_}y!&Ac z+^;LT$@j0YBwl?Yv%YgUT51Fu0JmJHMkS(achJp~?U4U9n)P3(Iom(B+W!ZIxc?m% zpT^A2ZVRbwNWqm+2Jh4^K=-ywfL#R$5fmY4zB`m?AY>r`gzB+^K~qQ*&Di7mC!V&% zNET0!T=1MhL?fh@d>l;%kLH!M zJj#uF$?G+f8}F2>T5M}oWLN0FaJ*0h86~c1x$97#3LJn0 zDpAK!kLLwLDFU8Ro|b62AZd$gKmjT_@RApQoA5v1&r!n%bx-@l!0_eLyJusB{@#9Abk?8$Wr2Pz%7VsYFsO(A4 z6ALY^DH1i5MhPGhAO!((6afKfd1@ZWyC9@Blh@b#bBqva#%K8gr zn_s?sfzas$wOS7>7U=tYq!j%mnNiqqUIQO{8AyB+8~YQ~mdRe{kfk&NC=Pq|075P3 zCEPuIJEzF2^ZD%T42Ou;8sG|Wu;)++LXHb+H7v~0Te8GR)}QT=6wM_%O}3-OWKy8M zg&f1{@UIR8d5Arb*n(fkcujld>Qx8^25MBz5&+r+!9!qUZyy{q0Kx{U7%1~cM>m!M zF#v6|1a^Y}q1kQ?zimqJ82Hc@(bad353zQ~y;C;e_ zfcT-4hK7bOK8OH;8GiM;@PwLS%4IQ;lTUI>!h5=+6gZ7w5Ri~}%-bp{4TGB-Z+S#d zZwjRw;Fa{Kv{xXB1zJ%Np-^=Jo^S&qLM$MOq2%kUt8S31LHbW^b(_VVAR#d^--{&e ze4XaN)T_i7|8!7VA>_fT(h1s5hs8zy4L(Ee$f!|cAF%J^a}DTiHZwn9{opZJ2Z^(( zKtfFgkSh=vvXFX ziewz9Nuc6i)pe~$SDp~96S*u@;ejE1MK`kB7T;(82?aT?Gjpb7k9`$*QNH+)I><;$-Uo;@GuiVPYV31}f)xg+x?MmCaM0pd zkldio7{X7zR_N~mk*FLl@BXfFk0CMbddrB zJ0Z)+lh_z5&wcv#id=>y+FJ5ub9(0*I7uz}?HnDM)hgB?uLK1%UwlZ}LARsgYZ({^ zKnK$CT=|f$KxC0-JBDS2yA0Z6{pfJe18MWiz_7!|X&ds=LA-~^vu9N>@d1=f1Ac!ue-tH5S1&}u-74eS^~xnFuB z_~<4`vSH!kMt07F#o*!K5F)b%SKhq26eSK%oC@0k8xbTj3!-d;Am9h!8elt+W2B8K z)xW&vgmUUwX0N$VCjJ)NPFVklqx_eC3oU;6PJ=uc%g-FM}!!$SeUk1IiE53C8Q|;D*3z6={jU6u|tU=n&V| z)`m2&zOGJPLj#Q&tcqvLdqT|#Iy;jfP6Q9oe|S#G(q?jX0{RGWVYGk{=Epc$ummqa z>F(iTM6^df;!y6D+TGu;5bB1^4g?5)Nx%#$>MZm48QOzGK?2J~{@?-A6Q;|K4`Vu^ zq5^$3psML|k}1$DY%2*02*Bu*58BUw(Xaf<9&*gq^G5wMV>$XL*aZM$jyCz@HC&vZ z0Ky1GZ*yxa5S&OzNDw~%kp}X`Hwp60VLudm^*lg$bjEjTG5egI1o>BqUxC*pFX94> z*S;X4&yLgwAB+`%5AceT4#L|8pl|f4djS-3S49Q9Ve(5lNr2aUqzGQw8W8fm@q|bP zoUOO7!&okqV9<)NxKS4=ZUEX`u$>^229PC5zy%!L?Ntft2oTmeSRL*)=St?W(S?`; zJ)gEyHMfa*pSQ&5#sYnmxS2p(*FTp5Jonn#6L=~MyH{V&_Qzp=t0aSzg`0~D76M56 zAY+%;8uI)1Z&aPWM_9!i#bXe2%2xy-;C4{I%5WW`Tx-&2&j)cns373pLPJ850O{UV zLf5K#>nkhtF`b~62?MYBEL6e)hbctp1$FCRwuh?KZ86aa_9Zy18vErE*g>$SP>jcQ z8|EvnLN3lp24$ymiGDk$h_C{SCxK>1pjSz=mshgH*}qjic?ql9>~&#^gTXiK@$^1^t-e&8Q_57@?MNoPJ0&s$Mh36-mL^#T!nZ2K$ zFwmlZ8GrY|0<0K@GN($7WS~?*`&USbN2T{;+VvqD01uM^rN_BjXL#j)O()jX#EKX) zgT|)~pCRoAFrZYLl;34rAFvX1xoqnK#R7qLl-Ac8!|Y`LpoO zdD!)QuDi4xV-KMABMTM*xCBr#KyOzYuER_}z@spc0QIF}*6U}Opw|Xb8vu4VX9At8 zmG$c2vz+#8SFN{$C?-4mHu!jPadCE1Txd+tDi?i5tDne2F4Fdq(y;UhGLnYV?asi| zLKv+T(F5I{^-A4<=$Ny|X@4Q!QBqQN8*+G+Yeh!1@>$^8Gl!ku##m(p{8=?0ytp}h zVX5gjl&5HQM5u?@4@Ku8m;vH9R7eXZ0tr_=ltq9efEvVHCt5j*YZzoeOTiyQR%2sp zn**56sJW2K_T16VVt%GGwBrD$Tpul^VP{7Zk)hGjmQ%UU*TH)=vEaYf6%r5Fl2A?M z_Zc%P=8b?lr}3G1Hh6tl?s#6i4DobqTwExy7OWc13^gDPfs{#BRyNy_dGA?{YWWJ_ zzxgMrMR@g^!lu7y#|J)teh9-+ki38lnmTjLFx=+oC@*NTPE4Eut^rqNP%X={r~@6p zVhF$x%3a+cLzz&^@ygaB!>trV%dDbbwdR{1n8)8|D1iS(YxlA0AfaFdGv1q70_%hP z#S7{{NMg&RF>UPZB>KWX${=$zk7V}2^Zhb?;#Dpe#nH;A0W1Ab;)Jhb^k1Kl9q|L2 zpj_r7VW)}@hZtN`j9Opr5ThqUeSnzsxkZ%VeYSqDn}DMh)Bq zc&2eW$dSX3jnPrn0I9Z;#x0}$5hJx%I#9bna!}D*f$mB}k$Yl}bIG>ygQ%h;Zw-A! z;###$-u8@U^MXTLcsOdQ(x{`N!LvIgY3EZRw`N@ZQLm(+TY?U zU#v>}YT)OgPs{ovd4j~tQtPXbmVwg~#p}ntI473x&ZnQ8nBU?8GZu8(P=~y6t+X19 z0xd8GkVaYW!rxDT=mYW$vd}u9`KMZDTnwtRqDnsIjZckQLd^V~)cNoKG93wIc~Ey& z+ndKt(IOyG8IhG%CF6x@DqqqPLSE=hp|x6(7V?kBw>x1&=&ybi7Y+j))_20RID|;8TK`|cmZz1N@0E1xMC=iM? zv)@%)kk71@^44VlkX{a<8pPEAcVJ{?{R^U}+Dhz7x}UXF5`=8lzBJ2y@<`}yX5cte z$ElAuenqBPXmmECP?}amq>^rXxxsUA{CqRhsw8cvj!x6;)O0<)*dzDImB%0>`5vBb z8Q;J+>2nAAh0gHrDE&JZ;YszXhrwFsqS5vt2@io$b#ihmZfrE)Qj}j_Z+5h#VWTm5 zEIxC+x=&yO(lVvoL39gfQ3Tx3(Vs?dh~;B;sYP()Uu4`M3>iRH#iaT?vmtXR zX_!>YlmW?5jPlR>{*?b`My_ym7Uu-RlU{vcwi;(>28{7C1c*=k;hSIDD715E3k|Uy?j>tseioLTXn)3Tn8)<4AZ@B=I~j;*Q;i-K)<}M?EZ`OcAhG z<3F0-W^uX^Emn6tp1Pd4aWnhSS;WZWufupaoBy!T{o&mW?F&)4vB&DiqhqoA^jIXG z&Ya~F6Noz$AzqaGZ5V2`{h9COmRGhGZ|RTTd+7@0oLZ@&R2&>;Qw3QiR4Y&r^FehB zj|O-rB@fR?#T=m9t1^02ruR`oW-N9)c(BlK5_U~W#P|P%x>p5Cd}D-d5Q~WQj&b%ja8g& zloH9k*`MSYa$f(2d}d+B?B+D@TVLUrtYY=G)~y6TrKQBl!?51OqK?cqvvr?Eymxb^ z^7E>jTZTZu%{wez{NbOrBTfCA1+afHuQjY5-+rr9vN`8nxpF6>(_kpmuyj7&!t44i zp4Y{_1xbXV>J1c$?$Z>~!lnmDEO$=UT=kal?s*<~ zEXUq=X-qwfDitq0ycd&P36f|vW1Sv9nWr1P6*y;?tvR&#ZL^E7hYAL4qtl!xn#`or zb7`z_$2ogSM{BPOpAY2)<7n`AjFGz8>;2d_v2@@kCM$cPkz(7njS%DRO{*9Got=na z5E_iUNxR#QI&e+=UMVeB^*7bxj@0s}vG|1BW;8Gr1pWwJhiy%hRZ7wzXn+nQ;Ga$JuH*D?&@(dTfjBEP!~m** zUuYp~(0#Tjv;+(hs(DCUh>>fcJPe-zbx|mdRUjLLzV>Moo^YB*kiv@>KKS2TTU&=_ zR#(kH6G&ILWTEw=8{1Ep7YRaJ;%$jzzGoIpb}|;X`1U5oqtwh-_|l*Iwn+4DmeiLW z^(LQe#uF>&b+i;jxXD|xE&QDE;0{&OI-X9nqUKy?@lhUk`Hr{gPU(Xw_I|C(Kt7LI zN36CtrtIWYN3zZTB*UPth=jikn^tw}hj(Y3Yp;t!9hjV^{a(q%MX&zWMgz(EajjiM zc&7-rxklm0r%kI-%`unV^USG)TYq~icX#{}$zozl7WzYt$W^q)_7zHH>vORY#KBn& z>$!%=_eD!-P8s)uB{;k8qMUmE@P#%J_;-7Kc!2)a9F@s~t|$B&PPa#y5>sXUe%|=5 zt2oaPUB`C8KcJpd2_lKJ-ec89aRgz0hZ7mC+FsD)Z)og1zKPEq?=sAk# zKa$INKXOcuGQ`(5sMZDe+hgcTu6Te20V65;@9DJZ~Z z9HYPsfB^Trd1e>p&6~jGJlZ}pVxC3&2Y{Ioa7M;&2n}#@5b_JmfIJ<@kOIvIMxQLj zd{t%T5$HTX^-WnGfXe<2BQ}r|Li)kv1IHPO0}cs^iaI#oReA`JXz^W1>ImeQwlVI# znxY+Ls1pSF_>LjG{5?1zLDwS%YLy^%2vrMO;~P?AKx7D!(9HeO-E9HB#>9SpZmtYK zFKD#`>We>gRZ9_OF~*FB&Xw2Ryfyq>s`q*Gr68MH7nJ~xN-{NBIC1jSXoI0YT{i|M zLiKpiMv*W0(B&(91DCWwNs3$gg5BgPNC0JMO|;tROeKvDWg`g@FSLv4Z;z28UUVbp z^Xp9_SMnz^qjs1!R@Y1_VhM4@C^nOC=Sp1osOG|(r(cwFciAOR9ZidD*l*Zb7Z}1z zYL4%Y*1fKtFH_Dpu(D}YKakTk@^#P$U&@TApNH# zvZX%eW?~pBaa= zrCdGtDNM#eX0HmPEO>Kp_jG zbF06y_0%$;=?1F3#Z=c;ML_0%-Lb?$T5qu!GWNR6%tdfY!k}vmBWPvNNP%(>`5qc= zP>q8e@MJ|=e7xG2hKsW^5rZN+6`+w$DC%Y{_Mlo@U$^kEhTd9MRu+JS_kw#bif>?G zY>bw+8gq)@i71$Ji!Q{$`1H0XGu-MwD-xanm*#es5~kz@_leoWF^;I3%A?fJ zyAo3p2}fM=QfKXvm`)^7d&?6vHZ~tH%rJ^d60yz>&X%%xJpBErf~W(_(>2pi-9m)J z#+(kU74CFz@_95xMJPT0o|@cyHR!{zQHi!$Hrtk~RdlY=`(Zw9DH}&}>(#?OEH|lVcZriDx9Q#a4tt?e z(Eq9(g{1TOd6%P>6SPN2#SCm2uwpiyFSV|5i=q{ZFUwEdqyUjx8n2E3Ch7#B%W)ndWryqZ`p zkoemNUv+keWf;5W2ov$0Ahq)(qb@8YcT5GdxN2hY{79V2D&{!j!gZwH?28lLZrpg0 z^OG$rqT9%}ml?NCjBoNGd@4)lw1S62CR)at6sUZDt|V`IEgtm4)%3zUh@@QHCts!# zA-l-cGu^i?PI1EFY70%VP&;tRc&5UJK)w%aTXgfND9zOhRx~0&I#*|N42bpU^iECk zzUG^vWe~u@L-r0?)b!pW)Z66}=niiE>CPTXVrO36@vzM15=91*vb9yfr zG{u(XUeG3c6$P<TSB^)*D27A^xe)7ukrDp9iW&CCQ>}A`TZd%Go9Fd(+H-0mCgiB## z=F3)Yob^^_+W2BX(C>sHdNS_KTTPFgdMlO)5*fIQmm+j;6_&^1zHAk0kf=W2l9C#I z6XPLV9sWQl2AW&Y<3_i@RiEUa>}AR4KtTj0ad&fbb1xJS&4Ca<;1n8tvRq@%PtfIr zDMhu+0cf796b=ItIik-V{D6O`=G$y*sMcu)M{h}!atwo#^q1E`&yV4zG-SbP+L#w z=cJcC6FKybqZw9qW&8O)e34$5qS{x~!$v66BGRf5_71Y>5uOQ$18dY*L%GT@o zwhWzG3$AHGuB38qXU)SOC&IWh8C8C#eescxa`lB_z4(#Z4dT)T|9CvTE2pIk zE!MoE9&OHdyxZwH0MT7J*GCaek70Xm@hw^ILp0$gQ+jFFlL`#GaqAa zlUvShr>m688xR(yV-UM#J^)0cM3yhJTnIn!(`p5%j*6nTNmy!x%D}ngCt| zCT`KEfdB_$Kr(k z2+eA9b5|iH1{Fq_AwBG5X}EU%I!wqN!+Z>jrTw=zmthhMJ<|!W18_OC(q4Up5Rxw2 z1;GPNJ&BE_^&y4HCO9|931|V@-`>Cgmj^_1*hJ+`cq;%?Eq%&T&1HtMmt;-xhMF&{s$ZEnVuoQ-|C7~G^9d>(bhKnQE82L9gH zz{nXJgv2aucAsgSkM?w|e{KxJ{32f>l+G`rNv`rA;jL<#Kw){+F!A*0_iXRU>4lQ# zRX)X7-mIQDGac`RQz6f0-$zI4(IKV}2lI19s@?6M=lAd!_iQ#Kc?lT3n&^M|J6Xh7 zgR{fxLP?ljT3CN_xxSSGcST(`$3GywPRLlva&zlO#IQO$SvTCssO@p#&x|KHMty4j zsVe=cZqDn;Zw+>PH!^DViN&jSJ11mvKN0Bt1Gu1=fy1KQVeUkb8_qFGhj}SZHnzT` zdoMB0PIr2t<#h`!+=*z@#NA!pFl~E~9A^dI`liU=KlxLEy{q^UM4L-UO(nnrqg>oh zCRz&{C;M0y$(1~uoXp=%_W3?E0|~8X8g=1niMKx18jU0B1&r85HsTit&IgEM>|R#T z!|VstPB6h`QON{@1OOiZbwQ|kUj6c=yGrFx<1C{-N%7Wn^!cVW8FNVqf8F2l+Z&ZM+FLae!mJ<&zL${Y=FYJ{;xwuetZ=w* zg57(X#B?%lWLJb%1OjIo`hedsxf{lgCZJgW7Pmio|I(ZyhuW=ba!0G~^s6DvkC|p~ zu`n$ya8yZii;`@+#veZGhK>96WS&BlBs|(=m1Xh3abrp-@ouMcFkriF6Lo<>Bl>+b zXlNH(sTY~dC^d^M(MApKbSO6szPq5H0FIOqtKs%~lM2P<3hd5ymgrFR4{v*}yC(?l z?1)CV)P;XzAYELp)G59=d1Gwl((i)pPpmAkH2mqDOmZHCW8GbW*#2 zx!W}rAK7MrO;fI(k23w$PC=XSOPnSOnW|V*L#vVarOF z)%*nt+DrQQZ!cFTjG9-(GPx!)i{%ck+Sfcz{CW2mMk$?iJQ2CYdkId{2uZ&BQEkq< zlEkTv=B#lyMM@iu+2XofQ%AMSVqvf8e2lw^5r;2Q+MLJTF(@9vC02lHR}l9Vr&FSv zBcq85Y>EB+q=kkolD%GhV{UCOtuG4%he|~XUboHLJ_2^0^Ey6hSZACRb;M*==!aoH zQEH!;E&ea>>JjK~sY-;y1IO=T1hzAF8R`_rb`uAVnzO|l5bJnoTigcyI%JzZap|Zo zD;9vZF<8~=)G;`5VeRv_t9rhYFdy^8v|5$vcLHO9H1koN#*Rzv!jA01fueDOzUdwG zgj$^=CT^Rry%?WHIG0#_+5mO?o=ACLN@#!tc4(jIi@2tgJz1OM5$6bLqvH25BVIto zGt?o-um??JAIYo9Y!c7n_&ymYwq)nmuEN&L&gb-z!%P#1ao?$ z0ndoj-1=kbyB-0Oawf?)UYBtG1D{0rWY%Duy{laKWEyuYE=C}g8ka?}vOd!A*9s$b zDT0k|OU0*uRBR)SnAx;6H$uYG>Hv9k^Q-1I5*w0J^K7uchZ46*!z7j*_`D~>Vq?Q1 zvj+WM&_5#BMciI6x?R(ztRb1utSwpW&eeT09BNKtu$iHIxLY489LsBSI|c?xGY3q9 zYeswy4|$ZV@QjqW%Ui_q746Dkb>@_>VayR=6>s^)9>)I3l*LPqO_`auQI&fKyl9T) z-S7I2_72};(>UV0Gah?$F{)OXb#3%)yxcyCq@PLXRa=-%|6>f3jBVMNxX!_#uu|F{ z85@=*CHd38wJntZ7Z2I(lzdlr_U6`y==))W=6On*?9|4-_}E4?<+-J7?OBB4dheKp zgr3b#FU+juING9eB5iq{MB&zWG2dP;WYb6)iv1j1bHOunPAOskj#0!cNaANJRjfUR z1Xp5e*c~v_XU2r&{&!~x49J;@@r)`T9o0(^*|aa_Z&};*gLv0Nn4rC&Wc{-TqjhB< z(ynTOdJ!ZbV&;yoTfHMsawjwJ1IfaDd--PpAUA0$C6dofC!o5VBI{Ojx175EHI_G< zSp3}Da)FJFVs;eLN~eZ;U15?h|xXL5GS&w`;+>M z2hK+LCmXrVeC4Vw@J3)t<0dd!xF+u>#@cU+?Iu^=EdBtyLNq428Qg=ky#DVR`?8|${Ro4avUhN3W5K;(9INx7cs#)uNZqHgIgR_V-wRbqYL7aubCX!R;&+xWw;2+DnbOewdoQ-a@Lq^Z+s%Jb$E6Su zSixw=Y6Tm_h@T*Kd4ps1rr5OXaV(F8aU|E{k~iu)TidK1`sgWK?HjP68NtYRn`r$0 z{Pi)KhGi48vbKN8>kDitQwK3wwqm@bxF~FeEqCS(%YRy>a2N!}+(W}sUN!_`ds=QF zF%w?jb9YVQioX@bEoY~SCm?Xq!_%2vzT5JF&2>2t>IVtDs_9$s<4h+uP@lv;@d z!~J59%J~HclxL&pi7-+TTx`#Bhe0|jX1zO)`?0`H4mCz*+uO^_3v&xAD;r4<4B#p8 zN&h&Dc=zE)JbW}0PVU!yy0c{9^!W*Wd2WjegmCqQ!6ZxR>mHGlme?%ZapH6l?UGv` zGM27+yozLIS0o`Rln_TfZV#qTt-tE6{zWz?A(0a#qCvi-y_uIz=$RGk5;#rPhBY;= zdr}`{C~<96jVSOWK3+=n*6RM(K-V5ZlSi3+V{Z;1Imp|X*48{1csnn-M8!Ceg7id*4Do}6b-m*vB%4uis1iKnZO*oT|$BJq8jdgj~1}_;R^Q_W~{65+z(qu+biuLFNc!0!z zza47*GO_3^pF_3AK5mNGJ-?u|>maW6#a=NNL@+$)qaEorl;x_^ev45>+Qz9f&)IIB z_CI4sB=w5fva9((G7#t6o`-j9V8VmUZ9W7u5X5n*OL=jwdOu;RGER7cUQ4m{Je%!( zYi5O43O|$-ekpCNJ)X0TLoyR>r?pN8y#KrS^6TSOb%uyZ{ENSF%zOo5r4iMyo~7cy z`v!MeLiYA@NF+W9g&mEu*QN=|xlAmRWMQ}eon!{QE_qWKZYL#>twVn8xg(Kd0Ci%)j)Qs1xKE z2IlEg@oN}7D0fNok^I`|T0~72Phfk(#`N_#e6pyqm$Pfr^D0GVguaFT>avqlD-Itf9r7Fi`caZMM7f-sl%*UtUhE33O{eYU21*RPfgX*vfPx? zEsNzB3uzNezXT4LN&ah>s_I&os@iD!JZ6no3u)oJ*_y87wsDdLVQw`Y34=ZJfo?Ba zJN4NSkoSxlbH#Np1lj+|f!?OA?V0fY`A%yAqZq?)bBw1mOUKr%0eh`r~ zJ$vl7wts9bW_T@vdd813ZZ$^X+UIs)G1(xziHn<*NuWQfixi zRkcqSpjLC=ngElLlo*%IRjBW~T@SCE{G=*Zfw>jR4ev7g`z}e}zJ~kfit}ctK)k?c z-#s_#+@F2SE8Me>Kg<*j;0mj^+WhUV`%URncWNpp^jo3!s$f?hsz-TF5c&Rf0Pk$D z^Ct;>_bO@r<$)WzlKsl+8PoIW&XrdA@kG@HlXYxZlwEi6zCUFwYu<}e(efL3dwaWw z*FxV)pt?b9t^ah4zGac@--%6s1TijOts*^oyea(q1>S*!=Kc9|pD@}WvYo`GF>Di> z!`DJ@4tjJvAL_B zFtfd;>e&Lr@iju}*gpc*Uy0S#c$LjC5%TAQi7P!K=i0ShwsZz6!yU9HyY`b^bc8|o z%!1i66uAm##$!@t3QGrhp&L%B(-l3o0|IuW^XGJnR=fjjb3Uv1Ac;tLexx~ELmH`yP0^<=H^q( zp{gcl_EMrBmCe>N6yD99jDd@Z@EJie9JhD7>LsrqkW#g{o*chvbh8M`Gv(;es*_Sv zV%XEy5*o{Q!Y~aNJ)x}VpGLI4NGJbTlNx9IM0>w&UnQR;W80Mkq@A{SW&-#o26o4IMp7guj>sZeUKDE!l3ZSMZ1`4}ixz_zrp`ULL$8wy z>UXoRt4bWp*$phr4AM$e?^%?wk?L_X|LnL=aQ?%CL=%hBCt(L;xsk`K=niVGAde9F z&=7r>->{y+YYsh_fk5$=`H3F6gzhx3Ss`1DWY_&$wkDZ`F!tN|*Q zTk%D{e186PQ0K(6Uk~g=GB02KPD=RVQ6njiFY6F(s+QgTMRmm(l-Xwe*Hl`)NZHi# zJ`oe9@{%Fd#RR7wH-t|&z175BKb&6Y#oUcHvf8qr$RU>?g!cb*GkK;FBqjVP?+-~t zuwYLyb$n+r`lqkTnlMLjG@;HemawuUTJSOBOd~B+D5C5 zBE!7$p5g#9vrNIHV74*RY=!K(Mn=rVgi0AGT&~Iuwz&=7BXRQx_ro%m1lM}$TRYm; zW#UNo_C2cy6jDEA&#w{&;(fcOhs4Za>6u-x`g!)jt|wJbvbFx8La~N>L5UbI(NE~6 zWC%~w{$xx}vdFz!rz{e5pl-1TH(&37>Y)_+qo!B?*b(}+X;S?e{z##_UNiI#EOOFYH?#Y4(& zze|%7O{l$hq=UE3^+K)7oX@sj;-N(uFKt$ETh(9sQ}#vh9__uBgjgqTLYR zMSeD~8I^nSOU%aB_FLbs4X|qCFcDn$m)6gB8lZX?s6xRbs4Nwro2!OPLsC8jN#CV6 z2N-#*Z+8A9lnxPOck>KK{C>dXrqkh8m_OK}oIa8E<8tfeL&*;=YAjr+7QCfziKviK!C44CdT#^IU^Wh57^}cxnxuJnzF%*MAnavR2 zV!bG-ToB!tcWjN$1kGZf9y9FBY(EnFzG!6alJ4Y}!(Y)to3AzHGrjJA+qUJAo=5f` zEqBj0P;lUDFTEK>o0E4Q{ph`4-XEe|{iOCL-BuG&^d~E9-+7Mvg1sxLg=?E*R8N(h zTtWvOG=3?-o{D;Yi?WLjr|Q*rTHL0pk6c1^L>}(JT0-&^?YpHbJEA4RLqog!rQrIwK485_xWQoa1WHa#EAFOew3ixJZwN-+ak?u-S@sf#h=^#?=C>z*!7 zr5(2Ed-rM^>IpMsdWjv%hcgj2F^bI2Pti~5Y=rh4-gzq}{I9*cvW6XAd){2s(2VJ+ z{Y=qz!tZXODKI7H!s7FkxSgTv#MNTi)V|42w4Y+~&Xt#|e5{o+O2@X+NZFAhw&olc z7;!0-+8d1M)AIZnN^PJaw$qcz-|&Vlycy_y%l)K=-!4@4CdBNbG{juiLIh3+Vt7%_cRjd_y6`Tje% zFa~;c|FEL(EBCchV{ZP9`2XxGDAXl_3pEhfB5{7 zi;tT{lzZ5Oj8fW2So_#4!- z9m&cf`W4?@4Z;7OpiAyM)zG1|__yKY-kvY#(44~2`E^`Hxe$B|>GZN>~I{3{i#(gsVbmEJe}LrGFx7H>^B5 z5V~opjQ@kB>kfqa|Nj+nQdz|rk&!))%#aUpMk1GyO_w;Ay^**OeSi1&z2EQGd_Et~=kxJ=J_EmUzG;4VaQpm4@+}E`Xk6y0 zNqUCJc6vF|tW@2exDP`imm@%7FBB#@-21K7@qH5h!HJR*iwm*OKOk?&5|R@B7KsQk zw8csDbvu8G)zdb4c(=f8jrWF}G71^95Dz^qN$_ZsEols)W1<+TbrUF=xX^ z7Ayjk7Rwjzz0xgkH$|EuZ-5yO;KvS8G69$AE`Da&c+wh^&^KI!Z;Y(>GZ(-3KhXc= zTQ6A)tkfzS~kt`hnFV{F*hl;9@O#KK*#==bLj-$_tpmDr&1? zmGHs93iLl|!uHVd^mt4gy=!~H(i-vezjyDNJQH~z`&qi?sNaHq`-F>!(FGwY)_=ZH z&hyd(KkwaFSB2x`E_W85iVp{2K#t#ZM$Q+f2SrXOWRwwWDDM920656-A@f8#LRwN; z(Xzp%ljo;!c-H-Y^-dk2|DH)+Blg&sU(K558Q=5ZYcW2!!GnBxHfU2aFi8L6M#%RI z0Km<_|3;mV{l1(^h~~Y((rwr9YBu52cf%S8-k{kATyhbR{`rl|i{>{y^ z>6(HQXj$*i-)PXX{1FeNBba-VcuEEqK9@PU;V?Cy2CSo*wHUHio#;;oBI$6KRoO=p z#3MUmMSU`Sl~O-6A}ZGN`PoG7UjkR_!<4}=SsmMZovnLASo0{GZYkg(xKlwA_ahHknHc!lk?h^Sd}p_I*J!vTi1s|}yBcx-i;~{tZ!o;rkElY; zm8Bu1G8-~r(cb*6P%c|LR*Uy6H+rL{uvL}?+pi2rPPSqjGhtdu_l*z|`Z93Z3-j+%mbecdHeGN;(*gGM z=_+;8%`pG%E`zL+CM`EI8+MWthzD;?*ZZUe=KgDb!2+GW{!B4fk2w-trUNoQ0 zu}*Oj6zG{6m0%FMj1bfsH{jPhnP8%P9ABULK6tSdE#car`YvueI#yl&zY!Pb!|t;* zxe;`eKfV7!+~)XGImmPAtJzWn!#1&|wuC-(FUSYN;9krRU7HY7cFSuGijYNOs}k&P zyIqY^xuNyNzcoemX-)W@waKi8l$`)!03|eznOR+PVr2gQ`KpN5mG)Tqy}1kat{a0Y zJf7>z;erw8?@Zm0U`mxui0Ehp(*-W-2|=8N{*qBe@=zDplwe}hyiV|WE&c1FQU6-` zo#{SxO3b)`e_Hj_w?dp6y1H6X?MmlL<)L$t$Zg*eP1S;3hb@FwbBq3o@6Df;f}@Oe zuj+M)ML4_0hL5vB*eMfMI#7!Vb8Zim(smaYdW;&tne-m3iFeOSleGAq$?u{~5PShAl0}*z$ujMo3$%8;o_5VR6!&6Q3R_E-*(;0F7 z@l`UvDWFPP8(to39FBfUdxd0t^!)a&LvkhS$ZPu#QmZ{@U8mQrJaz`57ViFw8w_uCUq;6myZ??SOO&l-#u4td$ezv_u%&@b;&kms^-E*9OxMXtdStog+ zVBF^Gycq~C&#K*=a_w|p6)#k1m3_29-ji2fiBX>yEEsed3<~sG1quF$4$PpuV2sma zeY9R{^5-p=nMA0oK;)^`Py%}2me$>NK(Fqrk?Tg$Rv zsM2E13s-d}R&M7qDb9a9WQU{e>l(d^o#!$t2(Gx8!eYFfh|BGJG=#+0uUgWn|E|)i z)Ty>;=Vpj&8@K?@MdL{|nzyrV@B`W|`#MOcSrm(A(Gq=L`|EFp5782d!y7Cb432=6 zF!3lQA3i5av6(sdoeEh3n6E;+pZi|aH0CBCvyA$e(!7ZSsjnu)E}s_VJ~Mvp^fMg9 z>L~|9nwOkgr0n6m#;8Clu6lbI8!(G0dvDSSx)JftS@lz<9WH>VTohY^m-F8)+KMuY zdshYhNN6E~glXB(S}h}D#@R8vr`6)mkoEt)Rm#6`^mqBDm={|M&FYXG*>4)bq|LUu1*vM-Ax7vz=RRBeKM_1r~%;&h&X~ zyPV>|Gdzr^*P{(w0Ri2**XMd41SYZ=XS9}5`wfO0J@`_?5nHAbZF9m&IzI@|$FEb%kH_JuI z)kse?^)K`P@Wm5g(KlYm=Az?Y#F(>IK4{(3iJMbzpM+Q~d=bh^jmfS zyRLnJYJ1Wqjy!lN`y}brS%gSS^7l^z-y)jVPi5N~qszehlUqfjQ)Uje7P$D01v(wm z7Ja`!9AbZ0<~aDtu2f$C*VC}M;6)9WvWEcZD>G%!zf#Tl?t^6&i7g$b1xyqJdykM! zTHXe(27CU0hse_km>xHJOJ3F0 zEPvL@52mXzynU*Sg7MHtB?#;Ihc1zbXMHsP2Yz&Oeypw{<^>) zQQtI|)o@cy3nbY2@FxPgc=u`JSjM8w=BLxqtTw)Uan|PZ=e}=Br3Spet$ujT6zed{ z@Ng1R?`z$Ox3fw+*rc;`wcmXG@87(DY_$0-`et|^_U3-$Xkw0yQ%!-87HIo#Q@k#S zt(#l_CqjEMCU$XA=n99r=;<@{QKxJ32DZ@p!Z%@;aEssf8CAcmfhj$K+#v|~A2RY% zQh<`TtAm$g?{xlMJXLBh2Iy~I^o$q9%4e61tq<|!74sScW-ctqO*+`cLjf(zc-{lM zHa$(!wXjsZM48K`|L1SsIW=Do>&uroEz?r~>b25(HYy7EIwEVZz`wznI|CcKha52|Qg+XlW z>?toHlzz)4w3$w1MilX)u_GFidzNX&x-S{~w|{+v-a;nW@bn{rCjgcrOPlb>gm~)c zys0K7;R{?<-R8D2QDTl+%s#l8o-Gn^1!c@C2|6jbc+DYPCBp3 zkj3ztyG1Dbv5MHt0Havp-zev6!J$K(xmo7`pPOuo6@fH)t^koEHQ`&%0y$*$yjZEmj& z;s3vE&xTG<&y5IX-YBlDIl{iT=wpx1`*4+v4#y|>ZBhfOg{CD!HXfQic5&uIij{&Kcmlm@wJP6y?|1`NON#rN+E&PH_PhQPx%Fyg|rm9j4XaAgrlp zpSxmdsZvR)LJ97bkDCA(d!;<%-zqDfE*2a@0K>oKi$ruQyjF1h7LGr!(6K3+nDd5f z+FacEVz4y_H{?cyPoG|xQF!$OvJs-LWtPh~>&`&x`Dgn=GQp|;h5|aBib({__!%-t z);4v6Ag~B*n72>px`R)4b-fSy7InP8wkTDW{rm(h?>PnL9dyIeo)t%HqQxl~!F$y6F!DmQQor2)BI@AP zk?=e3rO|oYrFYXhwLq!!)F^)8FNLRLdQp8Ecf{y|*2qfixAgN=0+tiXp48)`wsi-M zn*Pk2jb)jQ*gkIl4|GHFa@(GHr7}mU^tGU)*K?rEs zZ>b=F$0u(4z<>s7rCE9Vqv?gaZW2!7h$1Tvd;wBIp`7^9zs{vwG&iEX&sY8BtCxC$ z?7E7dtso=rKaC`oCo! zXyl{rOfyPcy|o-k-zs`3+_UkjRAtV0qpv^Tu#ZkW8})HzAS6LLMRPLl{4hNoW<57> z*Qxx#Zc6?K;-AX!Ar-zrwHS8jiMxn=>q6Ww_<6k;H$XyAa9aQn2HYXaQyGIJ+>7Yv zwHIE>W+g$QId7mJ8Gzss>PHN@XiqKLlq+}NdKSI07DFgm*8Q-$L^dq=3wGr@Jv)D| z(E@^@mqiOg5O`w=ezk$iB@rGX(TNvQE|?i`V5O1^TqTmlu*vG$C zD^Ws!B`>;wS4FQoHs`$138OoE>AW#K6DsT=%D@RkxCmfM2H;B!{~sWviUJ;~1FGdK z8dg2E6GyZ6!V;3Dviw{rAi;>T9WqL{c!LDT$?~VZ%VT>U#v&Xc+sb(r4I<`6#$~_z ze)bYg-|HYjh6ISa69504EYwTD3j^Nvjfl+))6~>q3F?W?M(?2{qd{|$omjDJworva zOMgPAPdKv{_yOx9?)%r6S%WzqvR&t_5vUaSr0LLp52G0NU7?5c=JiA7=7iQou^X}R zIejG0)z6YqX6oHsz_$72sz>%qL#Ekiy%QuTB}8{Q;Wa# zx?bz2G$9{Mjl}ZyE%?h+;nr}ARr1{eB*K4(fEv?(-<6$@L?p?<5SH8pIGB~egS$D& z1=uyRU<6puu_MyKu{rY7%fDJbuRFZ_`|^{?mCtWdks?JBIDZNDbL;-6TLgr=1A8rz z7<{34kwDRUedkSqqMxRX9l!EDin@Pv>O~g&laz4{3PuuBO9M^Zv6*HtZ!1UYyFnv; z!vd#sH=*G6BsL;~IpO-1F^ua${Nt>t_}2rXZV?Es8)DNLux;DR{AgL?aCS$hoJ6#~ zrS&o5%*ikzfUxq+@}MAA z*2@8TgnYKVezxf6r27Uk+k?P?_S7C93sr6jh^QbT=hx%M#`{p#wRJgB zWBQ7|X7jjh(sgQsPX~&st`)+?YsiLaILTcRhyJd;An^#&ntDA#wgt=$A_y$n^Hw zulSO}Up(M?Hbg@Q?=sk-8?+0kuht-s@)^r1Wa_~&eIym?q0fL!lrBhx8XvkpyBYiC zHzynzEx#cM>hdA)=20SZm~3_u#DMZzA^lNfVu@tY^o{EBZ(@{l{HMmlU)!~@qbRoe z_Gq=zjbiEYM{>D^7w#>W$8$0L`htW$Pml9cEVB}z#)TN3Y-&Vh+D(P1pv!diOwUUQ z*Gexudwig}MXYDh?>QGQKM57o`{0$cMyCE;B^wJ<3Om=ICCmy=-HqsyTzq1wl)f~r zQ!1(Jq~c5DrN#zwDc3U)BJLM>V|58{iwae_+%|(5JK90*h!LZKCp=2ru`?*X?+Bbo zV!ddQq{K!x>;tJkjxg=;aFc%8#b!naw6`sq;=17`rwQBf!SCE)H)CSp5!`y`n~_^B!m|J^qOMkpQ2JBE)-lc&9PnN#ml@&UTd*ZqFZe zJGx&qdOUsI+-kABI68ZYNjTi`Hh8{8F92m(`f)n4n{f}Pn=vNG=uebzPOj<={RaNA zE-Ld>s?P;R#SfFV$$xC>#$2KCuPM9TbO31k7pdLR#@t)VFUtH=#78d6v5SwZPcXWh zhp)~X>)qRZYO9jtmNs;dF6#d2vyO}{Q)y@fhXsQZ^{3F$cDI6IaZ&Kw|FLV-KJj|vKe2pKm!kNBdzA&Zz8!$+cD)ZyxNdX)sPQa z!Y#SVDH7D8!+8b!J@7i#L$QNMJhsUjc2#J*PkYw%`}s33Vs_r%yLU!eFBvN%vfgJg z-B9``KSubuMM|!8gRJgdbOYE(0%M$E?)|e3!B#yp);ai>;8s=niXmwB939fl%*nE< zsb;Ni86(qGZed}I?X;R1(8~9a!Xc5BiiH}1tU57X^_5lr2SAQfhV|zp27dCh*VS)e zx4McN+(Tv;$8eYlmtx_XLZlQ{(W6CRv?!2TWQA};A{48mmUB}VwUs=IpLxbA;Q*WN z%o#Er_4hB@7l@knA1)8ky8d6gr*}uuukU{D=uG7obSUT8Oce2>4Bh`jI z-(EMdcI9RvGfcnyeJ&D5(kdF9u&F=t(6%(s>65q6T;7S+W?R zeEJr>lVzcmD{g#Y8k4e4PQjXXCSgZ@a}`S|N#m+LLSAskCc*Zr8W{kbe0*v_bJy>X zz;Iu6-nq5=w;Pck|Ma2K2g&busLiL#?2M~4C%MW^eUGqmdMCk&#|t5D>TyN%R~N42 z`4FuHosylx*8-6q)QzK~(pPQmls=*k)3SEvjADmC*LJcrCNV>m|KW!csEO)DxWTY> z@}L6_Pnro*$V{?U&7qH@iF!SNZ)ILhn@Ys>J{783&dO>sy?+`+~E52gqb9XysYB+6~wcI2*@`O6F|9c+_F5Y*tgivOiku)Qs!RMHztWseoqL;&>4 zGiNex0yuS)sFL?N$%&u-z~xtR9MzQRwvtosMgTnLuumn&OtHLN&dhx=8a5fLg^T<& z$k6uM-uH5eEVQ#$ln~UY%B4Oswl}u2f+^%O*t63Ca^7*`IB{%IIhR!q7_O71z$Jkg zxf~WF1IH#6VtHaaC%y)6KK7+*h6Z?KOyf;GOs;)S+~7@4mX^RJUo$qKs?RTPoEem*?=;d#B$!a9lE3TbnDR3#Sn}WU{!+;j&1rwUiOuXzEV}Szp4`&ay&D# zA6MB(nlFaDw_9qC&gQxpBa$9oy0SuDac(oQyaG!PkbzIUF0|rYY;3Hq7{~nidOKio zrhoETd1VK`17HC9&;S7V?!83zXf+8DIKEmlcyz0+v^>f_7*1Y)gb|s_OCLy1N;G+A zEPA73xKOeZrauD82Em_YTFce%82pe-M?()1e)$iq6EG|vf4Rb>OwKNWhfem z4=m1-QP3dEUsm(xMe&NPQ2x`541**MZX{@X_;kQ|UARHkUjA{?qI zJkld@7TvB|M=6IPB@iC(?3-!>mkVDD8j>f29|2c9*^hMwe1EWrN)GJ-b%6)VG-O&lG>+GhU+9ZPI>g=zsF=!!Bv{`=42 zz3PK4m#=BFW3|UdO)O@ZOTb|iulR9%*xG}~8_&*2(`!n8Vmk3yp?l6t-HtW z-?h25#1>Ce625A?j#o-8uTt~;$T0VKO|_uY&9~-lkoq~zv%oqq09#b`g+%g8-a;kz z=uZ6@DsS>wTmXUxifd%?H|37_iY#>j0XIq9D@W0=2aWYOe92Y=A&RGuRouuq9^z z(scE@%^nuzxNEL(ixyyM(wtsC6N^EYgExPI=eBu^R^AqntNk7FS~&&VpU2$)oyx7y z?}>0jEU(XJK9b|~tBX5mPz`Sh{}E^*@tWOEtH@PtL{snc4}saIUTX`7hm-S}*+bk* zE5?dd8#n+Y+obJO547rE!Hz5%Wi$7v^MWZdIGNA$=ao7EDYe4{e1B49wpe!hv1EYK z;Wo|qczhe#g!i>VE{2q#o1_;4Rk<4gWmqoxd$pZlqmbFUrX_x%E^sU6Yg+7a5Ikia| zG2x`Y@)ukG^CaXBC>V$d(F8qQ@94!-V=_Da0`b+1ac&9BAaf�bmX49IlrV0(73h2uTKj6&A3Sr<4=MaLRyUT zh5$};Mj44q5L>+13SKS_LJ86VRBVcV5VeJOJ$En~~74A63I zuLOUg?o=~$*|?O zvi@dyId!6uciS|r!ip8mY1a7=j@dY#WBZn7q6Kb2vl_VYyoOu^n2g}H${$ufpZoar z*Kmeo7m5A-k^osh}j~SGo|E};ZqL1gF0$Wk7rYgH%t;(Z=vqIn3tFB z)lK|C3OxKWbQBEv{a@ZMp*Y!2aLd5;73BE-nI z)JR9|j-^XufKzj1?du<^*(hp|5*uRD!PY1`p!|~AK-39H(cl5G&}t?`^gxw|-tR3u60xHU4Uv^`k_7zO7a-4X!P&;e6@nLh%?Yc6`Fj zMIZgVq6-Mo1W4*}ds`~@()%j?iaXrb{>;5o*3pdM)CC3QJWsB8`mMde;S1l^PQn_b ztK7s+5B_T-c8RQti;g`Yepqeed!^r*t!(h*PXjKgc=6fK(A}vY_R9fO(=FPGfj1lw z0?t1}SQf7JdUbR|35oN`6M+LF5y1ZG#&JAEVArOom=(W4@%1h`&&zCg1W&huXIu)D|XBoru zR0}%A#U#e~6bY2z01dv_qCVwxL#WlDRnb5!Cz#SRV{@fF<0ijdf}DFSo^-GL$ET*9 z{Rb#23p8AFtxl$@voJdexIW6y<{bfb#=`-KQ2_+j@V5WawXG@lIe7WdmaWd9)gQ-y)tU3$OCospi+I-_! z@**C5fI70adpY0Un;3Ug3jL3L$ZK~-1#5x-cyapU3HO0gr~uxZTtt%wl*#rExHAjy zds>(5w|vxnHayA9WD`nhw;pYp-{G@F+BFeiyv`|s zpN&yNkA_0G?2UvYwMBrIh5}$8qioUF0Iy+w1TYFTj1#azmLY?J@892ke!LSKt1(7C zo^GQLoy=&@-%eH{H#ojudT#5CstZ8J<4CX~{Oh&;?z4C7Oet^3f6|_LIV<%dam(F( z=Lo3|TAXYG(G;P3lSqr%lVd&`J1aED%-dfraJ6`f;Su#jrpX)@JR2J{Hnb54zXO7| z1~QBM*m1{&tT%kOKYY3sd7wWFi5^@e#W> zB;Sl`xC?x%=Ji%&ztL4xYnu1R^}qW9VB`&;E3sw!az>p6?b$G4`cHqhllcPSpoNmKQM!WQ zDB26W3YpRb+cmDo)YLG^-rfjKNqVQuZgK^vsy;o4=MHk}R(*pZ+PcXq~E;cA8nP56e;otCK|gfE53ejzh{yjmc* z04Q5opCamtpS+92aSx5eFnaM^1Jab}F2et~>rlL=G7`(R=X`7ba{v@YH1OY6YG|b= zV#{}d3&8D~;U*!}?X{yNa2M-gMk3`TESMSI6*GiFF2QWg)nFezkbeKN&& zlyfrZ75DS0EuyMKvO6?sRyA~u;{=(*r!tH4^9a6IUy)l#I6wu&LLI+8>F%LCkuw|+ zuBt+P3?1f1i;ImjW9vKRe?-Bg!ywU#k$c5TJBCN0Pp zj_GFTgeeUOA>`Yp{k%5lA5V)<-mJ>nva+Hni*OxTuh3DKr&_*R(Fc14(IoCa zee9&(aPDaNW-!$^jV;HL)P7_e3Ats|eBV6rX}Q>RW|Ck~u%9UdG$|ee@(A5D=HR$} zTYqGQ#W4(sk%`{&L55NR!N%lqXPu0KyCRWDJl=Jm4J`(gzI;*kj@d};fZ3 z<mucIwMrDo?}dU&=i5VqpCxc2Qyjj1bvX~I{U zH^SsfyOM|GNmuB;W8GDuwJdeLfm~jIm7jqp{zmM)$?J_ZG30avqN|u4(tfrnlpATB zXV71maO(ul89$S>h2N88j7S~M-gvjAEul|ottPBKSn$NG2IfjK$hs_!_&M`|{ z0R*s*K`VmuXpt{5a*HGG{iU(T`5Fg4ds8Jn?NHKy?e4(vCBOmS*_w)<`L<7gn5glo z%Yxa-CAfzht}SNU$kQuj*wxEt$eOD8&9cK1a6sJJm>6XGvF^kpJh$r3Qz}$=cVm_D zXq9nbOmX;k{zw{P=7mU7Iph%{#_zo+xaPhT7hYZ%p&!Ic${=z9CEN40kU0kOfn*zZ^@RZ8l4dE@axMkw18TFtt#Mv}X& z__{11P)-c{Y-2s9->>xG^}BBej}eHY*VJ=bE&(0@0?BDu`8L8iJDnsa9j0u=9ZMK7 zn9WJwh<)|<@ILvqPh)YngK6lt6ZOYTpmIs>+KV!GE(dxop)!~1>}0HO#t+F9?)30% zNo;YwMkFm=XW$jAS4L&}+Th(D|Hb0k8&fsUD*JM@tGOx%5G^GDUr(6<*s7rPZM%+k z%331Im(nN}Gc3jv>p4O;*Y0KB1&xA zy@ExpJTI#(#=PO%+6?b4qW?HF{JCV&s?b2psdz_}wVp+HB9Q0pegvcK)$&4-puXM< zCkt6Y&Q77dT$F?$q!ho5%+t_0ezNK;G-rj7*t>C9Wj@r6n6b-c>{j0%kD(q%sj}B6CUg3*b)9B#jM0^9)tOK z^1A;8^DSFvP@6xvYKRnKV|Hg&vB-e;hQrT7u8Vw@F_hg}pOra}Z!30GVuRiLuC#QM z`<>@lbWENd%ygdowg2;0ko~ns_IYZ&kDa(G&k(ouaEg7vPHkdgp_8qIRC00r4)30j zn~tMAPZW@+{l5Htp=k6|xi6%Q&I>eHNl=dR8yTFE#^I;u>oko9CY|lv&!Y#E@5XNB z{Hx|*cS`CbcV>%5J(vHT<@98+vq^D?lYfduj>Nr~A?yA%E{9gv7+Ltc=fHo*Yp?4h zcWFgj3@MhZh$oHQ+IV_5LC&}u1_n~H_s35b=ZfqjLsyM|_we1;IBzt%v>tWXGFx27 z=lP`gX1ffVkf>8qW&h{Cm2Vav@x;mMn1&ArO@jfU5`5$F;Ux)pk`)418BcJ5$NF># zA}$sn;kcv(=BGF_f~i5!X6=0f%42baVgi}p;vHk_A^z#-78j0fZ|jnLyl&{YpRnRCQhu zr_@`RS&xqHaK}daA6&Yc7~Ig}0=p+?Z@yqjiKH~1i-56zM^p@3S-OrNequYH)OTmm zD#z=n`lBCJ9sG0!9a9gpLSifXpc3#(pC??GMJ-XYNhBxBdu)@*#TXwEnF>{@jm{uu zoxbF|K0~FI#(~X410(E_A07ucS>vq^d@#bMK$h`~G$1BIOfNK0{S#B}Zh7zV^9VNx zncEp(Mg(gt@S~d;z}uN~^dD#ItX0QCaIs!8!%ps)c)DpA(4)rdcTy*3k4$KwsDq2| z!<@FJ!y71H4To-&+gpvGCrJ@u?fU)4&L{BN?X&Zex9`4JhNSi!Y&DF}G#s@ACRpIB zRz`dH(B6X=&)YR9`MtrrKBoCIW(q#|jvcaxNp-C~F3%cl>`#1PZP%R2+RYi;x;v|J z@UgvgCB!ot(h~MS`r!xZXS)-j-n&qX!SH3w_%`am$;>(Nj#F-uoIzH(DJTnjBmAcY zh%z|%@!Cc3`;<^+jgvm|%4|;GJK0H0btwke3v73g`n#=_$K{%0q0gKDHt-o2jva|h zJhKx;Ix~zyGTBWomD<_uTAUn`Y(y@$afDXK-SnGvKk0*ps%07yXplX3?`oKt*he39>V^sKFuzpvV#czio7|GFJ7<707kFh|u)QLGDQxmb^)BwJ$dnG9)x zw&o;ix4CP*uPTj<|LLYNKs2XSv1d`7Fptoqh(CG*E?&vqrGMxCBZuvPg0_ZE>_Yas zTo1g`oXu>94ALKXT!~^{H`?A2-f03^{w*Olm*y|c1733=Uf?{tar`%)v{g;L$!*{{ z0!joVKciQh?LLC!apXT z^}Ev1SKq~8vP}UeamGZAktQ9;oA?{)9b-=? z7{L;qo$q%*KS{{*U1Oa*Y8>NZ*Z7q$v^^=RsN6M4D9-5pgN8}cGPQ~7vojsr%s=v= zR-+E=JFJY#_qWnrqixKwk21FQ?Uk%k+P-W^I($m%hr*?z8~XvLoR)4N9vvTk45zMk zuImmI?A=W#-4mQ^$#3-Ws22V8beIv$-xBU$v{%7bdh(a@{fS`UaXBikUbhbQJ;k2N zL`TW)*m&k^&;WfK3**)4`(v{)lI&l{nbGTwe!e`$q>X%YF8(QX*n0d9QN@A;<2)#Tt}Zjnq6Mf!OoG3h!}?D$ zqai?y)WDqZT_t>FiPi(0W#8$?^NK83u=H`CLXSiih^5i=_E()ue!&<+kMn6b4wQf25MuYA6qny>`gANxFjV5ttFrHx18gTfbr6r@aR^%OWM@@Qd90I zm$M(^_C-z8Ta4yq(zzAOAQgB9fk><9q|b7GpE9PTAMjOT0GnS~!z{B4a|vJPD11kv z2#z%grb@WEx(Pd}W84BGavLh&s?w~Jt>HGew-;8;c?_8|ZxQ++S|-=~l`oHM91b*v zhtEa+I`4N?I@T4=1asio8s8;i#Ox#6m8_ z#;dzj-3Zyq47#(%2_)l?eGS!zc`mL^2g5T2d+rCg;=?oDr9th~`o_T2rd7bGXU z&Cf|tzASFc4lh5>@O@lxL+PTcCnTtLxVF6Va4oR@&V2kiI%;vhnOQ6}N95oj#>Z!K z^lbP};}EyUGW(caCWrU?I5{o!8_{Tr=>A{VSyc6msQTvc*z$Y(KlA>mocfN9AD*v# z7l5>e2^$dqdk*fCO+MSWusine%KHPTOw|D*NeK^_fJn06G?e0v3gFQ-{c@QWAk+U# zpx-O#%>FZ8HfEdmB<-`2h^a`EnlUKX<|j$*GSfZRhCZ^@qFvwpy{iyt7PD8yjrTH? z>)M;?Ki-Xn%=z;T8LzMhnVaI}wwFlknd!t^ev0A&nO{Hc_3OK2#_b=qC+ZJz`HA

`PW8BU+~@lJ{iij zF&iUK+OPj*tXP3!I%qS=6h5nEVRSI{TTm!@7|4%Gl_Y+2%T)kO)zVXyq-*XpJgO7^ zT*6qY%b$*xjs3gtJnLKTD0j$+iu2pXHlU>ukm?7D|8vaJwB5kLRaw&@vasT@@WVy< z2hfIRByUgBA;MRR;lapbPC)l_20iGJqtsM27i`Yi)}aHNDpHHj2r075m2b)ke7 z`-Dch;sVMg8N<}!Mtfr)6`3xF(K$zP?cZ|7DGaD1b=8t!bUC*GseajMDsSEZ?BDqM zm@moQBzn2eoTgYIsH*p2|I=ri%hnj5Hj^s6ddB0ei|O&XV0h1y_3*e9k_D5kGN1E0 zxvg)8^i{I!N1*4i7?EC9cv7g*>-ogY7O_Qv89DY?pgLek00R3}Xhg`Jq#xIWEP z-q6QzYvsAmm06q*Sp}KwGZVz7qy?T$ib<6PVmvF;3i|dAn1Tg{-a68>pRE^B%n8I# zZII1^dD)8>Nem0ieyro&T7mLYK?PVU!DIk(%_ zpk5U$p_>zj>-|~D*olY7!}e~^pEu=w_9cBtGw9@v>{skslw?XGz{ClnSfK1Kz+fSB zpQcIu#cWvVm?w%q7^cNgmi9i~UzLB`syvh8(Z9vSJZ&76hvnXJyWKx>1$NCV@~ohw za_l)`_`B!3g5A4}K)U=`he%>}p|cB3fL4_>u2?mz!%gg(o#Fdrf+#!@CwsIgNv0^x7x?V@7=50v*2W8%SaTB-! zVj9i#eObc8RbZ`@2S&05u{vLCKX|e`+He7lJaW&{Xz6hmB|UhBhGJDiDqk>XCwa3# zh#VXVixwZ~cLez(Y8$*E9nk_73j!7!qE3#?QJP;GNA~BLQtU;sx461cO!P-B+Tzpo zs&zounPs7(81cbZ;!d)JfAFvDKLB&zu}yZD`$=iEzz1v>FLF)NoaE8o)Y2kq@lT4P z2cVT9m=arEwZM?j>Lz|bNqUD`bHnqnqH+EoUX=YuE04C*i}D|21hP)pz za^vk`hd^3rhz|u>CY!a@WHwhq)*kyvCd101ypfLGIvwD=BpASgH*U*E;AE+>vIH00 zXV{<^yX|+d53jb@pLu!dQl-RHjV68Zx^ao7+EKo^m_o@|1=_@lyQ^y*s70(i5TpXn zESrnchHkV0m_-esttc~59wxJ_Ul6$(^ycuR!j%f|*2f zbc-XSnJ?VQ6xOe+c%LhgRL(&|2ZV?@;Zq%p>|p*y!}ow5zc=7X8Wm2sjIaGHXePGZAEW95!{<+8nP1&H63~amq694lq}rpZBHt{{@Tv#^MjWx zP^-}iQR;Tif1k{Y)@JB|_)#Fh81du<)}z5zeW_+(k!3sV;i}&1eGtDgKdn0QXN0V9 z#OifsT+QG9OCgK8zI&Y;op~X0Ur@%sF!mNXnwDl;URG6(XfW0W#M%Ug{9buK^6WK> zy9fi^lva}&nyubDlQR5&Bwcqr)!+Z8#3gY>Tq8os9@#s$keiX4Ywwx8$%sDOq+BC= z%ecw7WM?N_GOi+fX5Gw-Yp?rz_x(LQJp56Qd(Zov*ZXx|KSA4B}3{GoV2q2)dW2|D@F9(_v(p{ zwT3ON&VmYq))p{heLZ@Y_8}V$a2V!xM|y!IH6;6JEVG)!@L*?Qh&6u1KEL-ZtWDl2 zO5!`@M#RTiOB0hzHKxJv5Y8c!C#?683%nKLcaY#cT&hF3+zivAvjYM@^D>*Txe}D` z7{mN?goBbfiswDu1zVo>GIs1SY*5~_ZdDyjq4x7h=bICIfaznVWp?8OAJo}w{L^%0 zaHLB}-~$eX3HY)1T~`yto4Tp=r8==ta-bWs#Qz?TA2yR+U7dO>Q+n|Bpfaqvxk}mq zZdzzH6@UI)z~L;iVD_f()=2p}K#!Tf_RkaFDXU&5H0u*KhjW%57I`NZ#qMK!{5+_8 zbd+X4X@d%A-%4nie*J|7YRG?yIa?mTM~}kH2r2@7J_)x@<($(X+%W3vJc&|PUw_aY z{{F1T?=Hn=Q^xp;g!emn;Oys>ydbV-{&8*8mC(EAPuT`*wJ z7t~iNbDRz?Q=LUmoO*r`cdY@;{?!d#soU8Fs4nk>F7IMrf1BMiF+U^mPul_Qobgl$ ze+3MuWw)Xy3ge(}^!5rRE~)W^-9etee=nBGCOe9p{`<};Rk+{5`h0xBN!rX?_3UfN zgu+VTB+5tBB_OmiNz14E_i>HCK2OYM*`I7u10Vqn=fr4!B@XfB(s8lcDjTLxvHR;Q zIQsVX8EtjmTl&4<6W}oZZeYPb(RES7_53=PpU>horkf{Qsm_2@u1fm!E`fE4=Y7)M zxQcRFu}X9{S05`-_{Zevr#iq5@ng54&XOD{tt!pcbp62gdnp0iFX@EIZ>nGVkMqV5 zk99g9#bcWU8M9q9eFV@_0`^bW^zQ*MT_(h2Eu>vUmE<77iKt8NnX`|V;Fbbu9or^7 z7`9x-)IWsya*NJ0sh;o-n?6i#+4tEaa((FbZ}OwoUnQJJ!Tb6j2)Q~8RDDm6bVK~U zP;ScQB>=T^Mc00}YBM9dz-*amvHv8vjX}(BY@9q^dGCfmLu&?XG3x=U)3X=;`ly&{ z!@b|{8`_fnEnP(%5QbL?vo+`7uAtg4JFQKe@fv5b8lRIth_O?1r-Kvk+GKWZjqU9X z7dtm;^gdNlO1_4ohVE1uUdyiYm&R2qW3J<8YW+M-rai>UN+;XtMh0g2mlt=;TRucL z>Y~7l+4Y%!=v7Tle;u(`@ zy7X)7hueHgSUEw?)lk*o2bWuEX9$yVME+2fb>>)rT&by!C|bFJtp|wN{-Nu#xMKH*qf5Jd|wJgFm%Y#My%?QRN7CjR{PwdVhA4LqpCy?MV zTWd#vsl);z7r8DI?%v?Q-rZUln;E(B#52=zQSM&Zc;p@1vhm|e5|g`<7w)7l@2<6% zskFnpY$8sIfDlG4J99n%}>M zp*FL_T=C=npb!so;myhC4>Y*mG2P1^((F|`OnH^W>DRJH5XQdr1>3N;-M}!?aLUuS zWAj+MlM|!X`&qmK4i;&6lRX5xBCc=O*jO`euK%B$!au=?tJ3RdKAR3}`y~grRA}u` z)cKoJN4@B*bSlkKv=;v&`8)P(B0s3y7b_oFUXTZ|LA?VpgiBc8AqvR>4dkVw1MJTg z_T)d&zgC2bTK_sY?CtJLO(|@|jq7^!u|{ThR}WA2BFcH1Dta`G#?~1rX;F!HsTc3M z$QS|lr(gs=M5(2&sq|D)v3-qR29RFMJ{cI0uT0iA#j6v36_Ol1wmSpP6o57|TU1VC zV^&N|18OuGjso&8ttDljx$C;D#rjMX|GkjGS6H4k_xG#bEZ8mm83)#&?m2!bL_4Pl z@U80LCaDK~3I=^l=nwF`<>}V|KaPSNz5kO+SZiYRsSf^CX0*7R#_rFDz|XX-f{~as zmkOuph7S~_kL4&?>+;Qx;fHndo0>V#eH{yHi~iFpizRHb9C04@ESU?)Z?Q+m8=V9M z;DRt10~xV{7v^_=#9WUAw)lSq_EmdN%~(p^vA=FEJ4<-P2T-?WZjK%2!fN*M4ppq% zlX}h&0=f#T5He$-#q;gFn*+x2IGS2pKtVTP^Lg?g@l~!%@$U4?Q@z+AI7~n~(}wZG z5uH5B!6xb~HG#XV&*ue3<)p$J>&&4!sNrPu#}G28JpXN}wZSm8`|?GLWQM)}D^7OZ z>?2dNd;W-9@m!>;BM&xl?Q3dhAzY-gH{J2up$DFzH3VYCe5Jex_lBFk^N!BQKL~(f zcy0Y*h`y9wuTZ$qO>JtM<6}wl_?8+|0HxF-$uT74PK2t&=sM+y+CAoHj*tlc3qpO+ zcyuS8KWC*E`V8AE78d3lByd4&4>LfZkLpx)lPGYJa$2j3u|gYzQh$~w#j_v5CtELU zXnt|ljNtZ5!FYjfDhDg{4&AA}Y+6vP8 z{JRd(=3N2ZX=4gdyUfcw6bx$PDxi&FcIDQI3qe<;xNr2`T>N~nUAlz0{vRh9IOa{tDvU8gMea7!Zx6IkJp!_wPu zHp$=qb*aEdeD=FQ%GM?kEs7-0)l>TVB=)%Vq3!+lmgyW1jm)qm9Lpi6^U@ylkjfJmNdT*>Ay1aT`Qq-*R2ntf2M!tIZIP0D0cj7%5ODW_1NOqr(kH&F`H3(+^Cpm(bn#D2UMzNX4$I0O>HWTE>~+>TQA-j3`6__UgcN~ znWnAyNc~StBaR%1uUFkHj1hUTKcuuh&Y)$Xk{6a=F`B2Ovv9OpbjG*jJKEw)>DQ-T zjJ!cqzkF9=*{#K>EL0R338G?X-xYX(6%g2HD)Ga_D%i@D9`$C9_dvrQcrCS^kCH0G zjvb`gm>f(-s}uToyQsCG?wG36ia;5N|Kx#Q?;;y?ZmCsOc(tBBrLu6GidT{cXbzBwCD`|}E%Ft=J7;EtVtIxR_RxlYxF+o41lhKRs` zItq6$J;N@-P22ZFWIfKG!{oU+xU~iH@<=bjE(>{Y_rWkiKt+QAJ6}XD=UR@aaW5y% zIg4z#m>Uio7as_O-Ub*lZKCYzFh$7d5og@xs;pvjNKyfRU#S70>To(Rr^3s z^Hel^?@A1l{DwhT1t;!LO~iJKxwxl`B~o! zFojKs=+|?bJ5V;h=<3KUHLs8l;RL6S&+M_3I(GUdoPd;PFZ8Tz%L+A_zZT z7}j3CVe0B$?v3NM$JF3PVY0Q}hpuL;)bzQOv?KrdGj|`6zR@*>{=5!B0DgWm?U&g` z@?#`h;RjWZwFwq)-6c6w^Om;Sb875-a+uE;Q}_1A@Zw(0B&~|yL7sBEb7o8HrYu2i zRl8=cqo=1_6l#y6Ww#!jGNbpZH+?vtH)+!|;p+@_#A1i)yR38Ir4{I^u^ea$XM8ph zYkaouX{f(W72nf-=|=dK1O{dA>FLrp>XUTDFny6zZ;XgYn0dU|mM*x7$s z<7#<%j{L8-lfndSpHodJ3j96YR-2j`pzba`W5xJ?$m+=bddSdzVAmANs;{D>nr{%Ox3ll-Ptto zb$Ju}R;&hw=2gC4vnu_8O;Qc1-erOJRU0^2zz*&y`2d8!nCinhV5s}qM*F2$YHC~f z+OXL7FN{%JvPx5Cc(=+U^~$4^UnxiRlTF`)*wj0SXD^<+STtq?+zVR9=)+U|F*&}c zUEPIBdt;5Jro`%aq(nvO;Du&k`|0T6A?}OtdnGRUBOOklYtQY3`DJ<&_TeaA<``=@ z<`!*8TzhVr9_EKDM@1Z?L);^bC-#kAlNY zSXk;+qwqL2NZ{Ko2xVl?a z<)Go%lZcya?y8G=ZJ&%h^sZ7OC3un_3*dQZkoC+Zc6xp-b%tX|2|Gc%ePEP4 zQh&k)W6^XT!xGG$=HTf|tUSHY5lHd%oHGvBXdzaQ%>^O942Mf*xcPDp4b2DJ0X$Th zPr>KM@v<_PG~67~FV%EcS5;62jrgNqrN5i%Yedq5>`^kG0Jq?(*Y5nMt?oU zKNiBGpNzD58Pw{3wmL&huyYBKCGGtyo4Ve+_~jFZi}ST)_q`Od25vyze~Rn!&y743 z^yM^1{OscsYB){q*QtTJVfUx*ASdwQu|SM=kz8*6>JmJYiRTLCIh2|7dPpq~)dNU% zHPwd){a0zvj^j>G$4BkcSJXvQ>ML6wV1NqWr+kB@{FnQ;JO-2Aj5>9GmxCDt6*tx# zG^|7t3tK}OmP#28Dy87FHS`ns2H$i~Zyi#J&AG!`wVnt4Tf|d6VtF2$pV; zk2R*?3u8CNQS$l~57Y=i$8pSp>&09s7Po3**-!y0q)jUT-%PyD#kUKeGG-`XrO<%_ z?Dvh~zoq=mh)H%mnLW4S>DDy79ZuePFHlewbS=Rg-1pS` z)7mg*SZ65q^eu#TV?MN=wBX%dT3P^^HX7$AaypCTNW<$0GX%y6&)L7)o`60-aw**x z$LatG16Xn%`id^{d(;7?ShM~Zu|rwwWN{6Mop&M)L(8!>3Y4Es)KyeCw_|!!p7CCe zW;*Ad>wur~0qkoJB;xV{kr6I-<%%VHSo4+EI! z49%g8v&Z{c|Gi6yVnCxfTd8K$aB>!Qk7`S^l>?UM@Q_0NWb>x|Bp|G4f90B49=iXl z-FP$H0&!J<=sEu%T;sE`$%z`cmhMvN{i?tXmGd30o?lk+*l9$Hk;96)ccSYK4giD$ z0(eG-2VS-94J7}}fx$rwy^$2dGa5Svytn>lntH=LDfq$Uk7`Q`BU5&aPR-=s0Shy; zGZRPv9jDMCbo z`-8s{WK9X1x@^u1t*Q`#ZU&a@(fyU#%3?W|LBrGCE%ZUk{&a(7`xH=ydFk&YHso`# zwM>qZD7;HEdE_zJz5^Tu?W02?msQi5I43Q!G|@pHf+^2?)wt(;Ci6c{1i62q`L+pN zXEzlsR(^>C12jVNJ6{}CRr&}+AG1P1F=Rs`Ng8kAeFFY-h2!*-yc^lS`o_B(a7C?# zP0bd}C;N$e`lak9_*$7yy`d8FepgxUM)RZXKjnU%GrjMMdLoppOKt~nX_b`wW1CogS6cta=dVWg zX_VFt!^YGl?9D+8dS+jopR@BiSh$F6m@=!YgKSW67#{Bl%qQ;(dZ3&;mI4R5s%-Ab zzXLwqqTE2i5eQepuOAI}0d`UyeBE@iQ}xKSxz$60Ep7DI)0;u|7>tYj56gi$>POgX zoy-1tDzM36=#plh@Zga8e4D?wifOi$YF|^M?dMrrZ}p;@n=48m z6&2Mu)(=%3Svni3(fzYht= zbH7a{dF{(bu@mdvwjF0+L3^tCzzXtg8`VpL6C;kg`s2 z1p^<<2s_a9I}+%_)O?w36Meiic%;fFnwb2vKaHeY6QtEg$|^_KRg~7so2h;Z>1Bg03SI=Ex1 zDFVk@m|RixU@qv@@&ohA#cXNp6NmNFpVA@jGjh*c!vxU6Wix*ZJ?yeH?!a2*0Q>Zj zxnXFvx0}Tb`35O|XKWhX`8w=O<$iJ(%6D&xOSrO9=5j0C!pJ{@A=Uq+DWba4F=u>g zpxzKjRpw3pmFn;YTeVFS+C7oA{!^v(zv_EuzQiQ0cDyEMfQf(7OwS1+)K!@}-xpfG z4TUI&M<30}dtx4FWEIwbJq%NbJovkl&1r`MWi$%xj8vCGTrErr9Yff~|H;rj<`8S! z#YujY`Y2^=?j|A{Mgj^H;vAcB4GH--{om$7?!^W_XVt+9INb%Qm(_i#-g7Rj(p{5n z-s(xZ4*+fS5gB6v_4`-j6Qp5DFj%Cn)0+;L!a5m*JDmqlX4rdF1~#|vw#ee%kjkrh zQBU19xj%#V4`Xp#>^Z6@G4~543X{1}PG*9{xje&hB{H(( z?f(_c%#7wHiRFoYOmn=&MO;U}7KLPY%aI)lm~g`Qn9kzU0WQCI`lnL{k9^^)SaB>Yb!;%V)63($R(bZs(DXf&v&SZ} z$f)A{@)T{wAsmMUuOGEdO zZT1JpM@8>GCi{p2%=fmSFcbq8!ib{2Cln>mT4O*C%)4wPiR%_qn!_N5XSboI0SMfIZ`#O=3!ZtzA0Jj zb6oqcPB(*hMI4M0xc4}^SQ?R6gLU4TrM1Y-i}Pi}$UC+8f`*jFGq=lv9NY(azm4oY zo@dLG{Uq~AFy^9VLTE3y?(jtrSMh7#K6#x)eyzkeC$2z-GlH8L#5cHD2&NpI9qol0 zjQbqEQ_+}{e~YTq{M_zOfiuArNjOTCG`E({(i9|cBl)`-90oPM$mxUkeB(bgBmIreR_eVGN!Pw|wmcm;E254KVA#9p!tW)2@08j}k&$&4IQLB7t zmH_`pG6(+)?>dn@sMc^ZTT0lMkI52!Y`25gD0OpLwsw1iD$#Yi&O~gNp{mYvnF*jO zbap-LK*kk!d#15tHFe2mNIq}8kAvIywZ0${ymf6@$uEeVbqnE+@s%9(5$8^Sh6M95 zg~U5nh%bEr2vUbmAyNP#k|!bxJQz9M+7gJHte(izmZ`5NH8qn)U(_0n!g0xFp#u`! zD6eb^Ee^Ev(lD9}gCPm)(rq;gzHbd9;k@>XN=vKD2m3QQIS$+a-c7N4ud>fy`_?Uh zI?d|p9{w!M-*e~JV59Rlap`et_E=+Acg%;%zl?j||3d6yHCn`3+P~Br0s==tuUgg_ z;C*zOHML&3QCrA`yNmjMfAj9*TWi{;E1O#P0<8iX0{Bb-r3Z9f_Nx;+@xP6p$1h?eYS<^G(vH{c=}%wGS9AYh76?^t0A?D?9Z ze`WHJrYIMHFbwkj_8ROn$Bsuy?EOkh_dfikN3UCdJXQB{MmCt2?yKgd^eYI2JNwr| zFNXV}TWo6_nKi+P>89$j0bs%bBNXxoBFRU)=P?G=0=Sw4(&{*=hm|u1%#-~5ds?b$ zW55;{r#8K~hD(a4J89)`Y@U`f9%u%XRhK4bZuv-Sj7S$c^Trv`M?C}R5vX5M??!4H zn@)H2!uC{k*jBr&l~O?yMDQw_93`HgEYTrApDdgQ`J+2FzG{u}j%>R}kW>z1n_ zq+FWatT6nHz%_7a4nUxLJ!aF-I(n{Cg^QvwF2t;%BZ4Z&c3@3)d1Ct5I{Q$VIl?YsYOdL@IdF-3mv0bT8rZZ(k`vtLlB@!%-uRSURI{6m@aQI-uc6RX>Jy1*^?f_oKc9r^fYPSkeW3?$_@%U zSF0mBO7iPhIeYY()MK&V`}&H3ax>`rqZ>TsDXDKfkIA;!%VlPpO;OMfqh?spQz1l(Oq0OXU!@5;Z-5!CMrq~0$A&03(b zY{mTA5n8P9a(aW43a(FE6cA@cjAq#1Ia=ua^07a>CI2WjNF`n~Fj%MWUjVa%bDe2jH z$;pH`O^BU)`2)u$-t%wuUL++OR#lOcD+58uPd&z0V*7$+6*i<+0n1Tf-mhcSRd>lNA=iV^yr4f zA_}g-pTQj24dTY~JJgo)jd3bO1G>=#sX;(*OUo&zsz-bL8z5;fz??0iBI5N1xj24N{QZSGw8^<+A&^4!gTr3d(9fo=Q2zB?C0h>8SQ^7yma=Ac|aF5M}{ zrY%Sd>|nmb4<3*&mQw%A#=e+RgP{Jv28lk19^Eo1eiIwFAZ_=x&V*FoxLw-#Z z*k|T?BxEU}<}Q;~qLk_-wS_S{X2VU%Yx3Eq5=>LsV%b#7HOF^o^$!&vEtMe{saNbvIQnp&z+LeA*?I#PLw~2Fn}WC26t40` z^X%>pWhsFIIqxOq8}{jTG7vuT79~HxsWVZvB!FSm_mKL10PBH?VlXl=cZ|2mZ|y$hqO|_v1-|{aA!ZO4wFJx4yOZy9FhhE33*17_1HUZ>eY_cv^9? zF!skOY|7)>$OuCO(CrOBEj)M|zvLk%_OxWMrdyIwTtY(ma=$FTWz{@a~A^Ql{JGc>p8ZJ$Rl{5(g zINZcfz&r}vlO4u zCW-G;`p^5^1C|&cv^t_-xa{-ssulno669xHg$E)DWs6pU|Aj6Fq)RFsBaJ4_!-9s zh!+0sSlVR?Y=XhiE_S+Xf3!OTah?2eid$H8-C<7yi^A5&Sz3PX9;vuVu`cBbTkuR0&h zOQi>@$W00G+r9yEm|w=D2g{df&WP&xu=iQFyN){y>56K4!920@D(g+9JElB9vF)yN z;tAqGJ8J3C_$)vW8h0dG1`%pPl~UI~Uj(<`V2mm@OT3~k81_ksHhn>S+2-z|TwB-$ zp?-RtnOS8~=@w6Sx;0ARmRL5I9SZ!eAPDtNQ^mb}ru}FhR+8;oEu&eIHY69jDDGan ziuaIWE6ctQ`qPcpi};cgSl?f&dV@>Bi*gC@v0NH6dm0npIrCxP*f+ro#=7_!L z5d)Mdf>#^Yt`8+W130TR>Hfvc;@Zza$%#u^hgn1Qh0zv`zpYU}uMUaQPOj8R#Z3%D zm2gGyF-Q;OPqyfQdVkDr=>6b7Uto&pO@Q{q<+)$lq>y3++!c=@-C9J_&|G-WkCdEi z-%0Oejp%Yxqzix8g=D*DU!_E8*=c|2crB5>FniP^+I+TMeZ8Na*)sJ@|5I_{2}f#5 zX1DAP8OQ7%MkOt_l~e_zil*JJqxeKA7_WarQnuOXq(UEI}|`2P;#wOw=WDQxZ)#-n?3 zQD?1kI-P!l&wX#a9#hc1iDt@=+5VupyRTxNM~-}5GycV1vqe+YeI*9)TZ5lsE*%J$hqc9|>GouPI;GNYOAv?EKw5oXhF1YBCy_-WgbBur_+56i)<>G+ONBMu$5P&+7diCA4Uztu%;A=>4RKh&L zo<1yUvn6G--KjGQE&U9RQ^`@j>|XB5|1wRzbQd=k&aJux-x#2NwBOhs7`S<4d8Yj3 zS!8PK>Ju%R9F6mZ@ujyaT+*qFcTJAA-*7+XFQB`N1poK0ZX0is}F%nVLyyB z@9zIzTYal4kQ+`|j>%JX{&m1auOpdJv?xh>5VC3SD-Nm={*&#l9q9FPJA8a+@-;Zu zB*pYT5`1r>TKk(dB&v#%&stoAZkn!M*h{PJ7$JbM!`6+dB~Iw6x)pQ#DgQnGd4sI( zj>VaIsIksFr#nymI7&bF(5a+zs6q~dXVoU?#H<2+QXX&vvwCj|prz!X^vcNlM<4s- z{6b$SF08Yfuz6L~7$2CHHYVH-mNPejAb;J_s&EJb9Go~dE(TO3{I1GhBneYC;xuO` zVB8*_37@aN0+MK&>K`{1yeA(oPcN^B+qATDa45sFRx8|qY91s; zF21@v4c+>@hQ7RcXomGQG#)%+@Nd+LM%j_LC!X4>C`@8tt47IbI1X?iw^zCurDY-__5o z@PE{e7O#umAV-QkWzZI#U2FcYcFY9JO%bVf)!}+lF6o zx;5xN_Ll93r&dYXW&c2U)LSxw?&PPs<2l~aRCSPdCV-WmGLn7UWpoKbtE!~7`&&88 zZx7=#I8@AL4y%)gQ9pWeT{sjcmBXrMx74)~?|Kge)&EC`geN*sV_$lgoYChgG&wo# zrnaKZtxq`L3_WPF*_kBNg!SgiO{6Q&ZUl5n=H!kmilaWbGmcM|qh&h9QS1@_Uff4< z07UJs_`pzV^)L1X@luMmG)GI0CX<3-ky552=8rO6kUJC$JM9Y@$yBc#RmpzRJSj=} ztyR>?sTw2>9D%cJq1^jLsXk-AvB@{5aET>}C04v{B9k1ToO$;OIN$b`Fy*ZQ@v0fOUnzMA(@{pMCEv}q3aJI+v~-@Hw~%S(Zxd{5p8!#VRBG;& z4Zf!_Zxf5k$ah-a6XMmPMcK<AaR@82J79j*ai&i~ZN;-!JtDA!%MAT+C3 z?^Om7!h`zoMx2$?@2jcAX2nN9P8s>fmI;%saw@{G3?}F#^C#x-ARdahtX5<;eV$~1iE2GoGwAH! z?~FNOWwhzy3NMvt__^hGdJz^+MAi7&&0j~`r``j9p9qNK^weOm(`Kg8RdG@_laq8? zncFU!wLD$U*Qk+xl6Y4OD$sot`t;xdy)lOKS!?^{#`EzVT5Rm*bsjWB?Of0xgYKisMlVL4{rrL3Y~wJWTNS;^gTMkz|iMf4nthdx5@Y zUq#HnPFIu>gkN@88<#A3A>pXCPX5GE8G|-B*bx~ zy`d~{kAKC76Inz=TjrsXOqiyzXqykZQEiKK%ZR{&07sIFCY9g zbx!!Cq$B_RgB|1ar+Rk=81ThBH>7_4QyKtnS`|0W0nKxPuVm1PYKwWXE6WcmIHn`K z(&OqncC&_I-dOpVC6x#eBlNTTG##3`17f3e9>ot7`{7yw_Sj*mcSHO7oD$r})<4N3 z^|^WgUpw*S-_3<9iF3-Kthb1!PVkvnK4#Ox{0FCAf(m?>*8tIVz9G_l&ZdUc8p58U ziww+R`N@`Bj#IJuwToBozSa_uz#hB7qgBM7Mm-Ixcs@x9gv%D%6Fn$iV55q};$0xG8;u=F^g|3GBlSF9A6#ke%(Mp+sJuS1X6z zgPq&3FUc-;OPw(Jp|_kdbqef{z0pX{lbbO%kSOog2Y_;^k70?1 zufy`o1Tf~-M09N^y7o%u(`zy>YWDs}DDfc{DhoR%zs{nz5cWs6|RMo zz36lTqMVtB8pdMX;<$(WZy?7(nL^yQeR-aoi}>s*q~Dbr46nh^qPmSU|8b$XTEj$d z733Fu+TIB@>Z;Vh2k5u@Z^j~oGa~_#3-C#h>HjjMAiQtEy4>Vp9epfd=}}r)`hZu( zGITR!HW6SAkn0vjW7Ebm1Luh@C4hF?3_(M^6X9G?K$Hz6ms&CElk3_>2Ye9L{~HA@ z*71wQkjS`mo7)Y`no{Zl)PU6e>Zzj$`1jpfLt>V72h~;j z;P1Icg3NptVu6Ru2E9}VrZ}SZxf}36=2Qa+4Liw+3Qa-(pC%pkjYUmbQe<89o9lAI zOt|ZiJu+=+GY3>J)@WZ!G@tx6=*+UjINPh$`Xw)vKacqJVVl!cOa~&<;NRw8nI*|1 zt5`Y4pN}@ZK!EFTQt~ny&JbE_Sua<{Z-Y5p;>OH}7Y~q-Oh$>|&YM~lMuJTD`46IG zd5}Kx^fv23V;M=-i`RfBvtySO(84-?Lq6>P5zRaM`dI$Ckn`{8C6hNk8s=Kr#Cdlp zKbjNSwvp;~U%W0+g9d7^_}>x}_ddV>kD&Vza9gv9Pff-<=Hm}OWC4a(m6Nd+QN=3G zow~#~Vd#-dDgR_(cnqWls#+R`%Ga1Lenk%Z&TfM8OE1G_KLo z#C5)2IrMNg3T(>k&*5TLYOv3N&tSCwpE)D|{z&Hvcdhx5h>>$!gh)-24N{8i_fU}C z^MDF628HkkGyqdD*fL>Ak0OjZ%T%in$?)S^imf*=4&sku__ih(K*oxCZH#6UOdS8~ z`1h9MfH2jKvp--4VEiD!(&U}MI<=m8^?YiD?pCK&SjZX)$Awt__zhzdK)xfS^{+S& ziJ6~{NOmJ!Prd~X&~#!?)2LL4!$vc{!5#r+RC}mD3Ct|`hTuEE5pCF1h_Pk$+_<9D zC^>kuy81Pxg~*#!pE4@{LbYTbTa%Th20JeP*qrI@&^Bm(#7~ULm}{Lb%@4G-Y1tsx z`VCqrQo=tcv_wi$rd}1)mo`VmZGd5t_G)T9-Vc2%(eXxnI`4+vBYP7{?popJjT7QB zQNRujGBVNhc|(5`Jl8Z1QNIf~aTy3Q+PLfb=K4#Iam;Jev{pY0!BL;aP>9Sbo=dOh znLs`o_)j;HKda4$gb=08e0+3ZpNxCbpg+gPXoH)?>p3?4PTXjjMy)9CK{-6>aGpd< zwg5<;g~MH3+#py=aln37fstzCqhbHW=`=5mbR9GVKbBa%C#(OhS)THEY4YrBvmm;_>1L8l zkCW>JNy;40HQ4Kxr9mzU> z;Axso%sV^D3*OnV3Icvl8uLg3xD&(pUU3f!;4+Y*2iyt;2Ov|trL)DC`@e1NE&c6# zwtZ(sp$lRu%0*2EcsEe9rN7VKp^F0xU`-l@T@?laI=XY)O$g+U17jx{8AuJK-?%*2 zxS8$nNX=dI_< zi-yZSG18b&+nRLv!8q50ATetvsV1M0?>m zkN4Dj$$Q;~@f%H4+>d`l>xF!88PBWjp@pm}qJVeoKVnW?^NZHA~w9IyqtfJckspqIo0l5Px|mrWMs^BVMj;dM10s~v9?E|jX7`4cWXD9@gw5W^HL>ppL>f`z2`UQA>e6Ix;-VM z!NJWRX=O%(obZi4^v*Y^ z+AmW`=TBAJ(T^li8@VPn;xg=j((Go6ppQ@Ss#dj}(6Bw=X)Yf13L<7pW}8(0;C;%1 zkParv!?w<$p`ofB5kvzTnRT1F9sa7P=nPZ;@O<&rvksT~dlU5z(__Z1U!7Revj|-h z&mHMcU(_m!kbP^v2vqg`wS@_&8Fe8eD@|F?%Y!nREy*ftUD^lWpmS;Bm@j;lFEce>TM^PuH*(`cD6+=%MJ< zgKh{P4{Y%#QrSUdA^+CY8K7*z+E1x`JNa{7k}$&2YO3iW@{5C=YVmbx1(ir&*Z$C* zi-UknuCVK()i%|t6MWw zLOYSfJj|?f{5pa)$_54a85G85Qbrijl#_`AuD$#acSiAYMr7CZCnZ>4hhhb_@rlp9 z{;!~)mr&O(A$IvRZ>S+-RCepZ=x0R^QNe@hbeh zxnfD>NdD-yV%UB>Y56FMeqWh1$$8-RHh|-Zs*RM>PI}ue8Ct(|Kki~T@4THiyaTuh zdH>~Ru0U^Xqp>?Ndk*a{^9=4t{nJz^2~|Z2=6)<(rtpU_LAB0k2`)ciKz(- zPKJd?KL%C&cV%_&a9s_=U7hpMbk?Mji=VltwJwdeUf9LNuiSE`fr{t%`$b_psDMMT zYUOn0d4rW4sdAoF&vp4Du>I(4{;=~6bnQ|2MF)NOa;3$1$RDEBUT)s;c&B2^1yxwG z#34P`g=+XgKK)FSGVm)tNh?0?uk`A^ z{(j+K{kGkEy#s(;+3+Ey+CPeIgtVbg_tL_9uqsZFHx)~^p`Xkr&MuDwNqdjD&X+a0 z#8>1z1?iP`0*~gyNuz`Y8mr5_G^_KOX4&NuTD$&?Ta_g-PI@2PrXi3u0oko7UDNhizhy2LD{(HUQ1?KDmEgU1`^;uS?--mdzYr2){ z6XaPUz=pe{uQP_jIzMp6<*Bd;rXu@UzhEXCztrn@S)hv~@X?3?VkmUF`|H zsrq*3lwAM}wUA94(j!dlhQi!{%($CR$Qg0RZ)S0SUzsuKh>2z-j?ud1a=r}65utqd z_t?Hg>WLjul_Y$MpXNK6%)4v|KmAYj{6h77E1+0fQkhtz3V})kojLc z2u$>Yre=x#eRunz>y(GWuh*JaGq7gLweB}~v?Do0Mti$NLxQ;kl0~N5Qtx*G6m;T= zAo6|H*J%rb_xH7|cKiylp^tced(VcJ$UQIFt{gw2QX8$VSbGda2jru$xSLZOJ+Y1z zqwhM!Q~D@w@EQp-&!0(>R$GC!)c-GXoR0hV+G~pR3m8K1V*@Hz3|_9ku)Pu`1WjFu z8igK6aIFU<{9oDu5F%qVhDjb=w|Gei={5*a*x)HccZ1!nva#v*E zt8M(&z@L!|O0;)7^uu?#J$L*RmHsbSclh_f%BU**8GLMFS+s(QhDwUgAyv?Wjoylu z2PsTNb3MKL?XkhZGni91Gv&-=_ZHoeOhzEMs{T{cQbMt<`+e zsth>kII50PHn+;CrHMGZ_|fao@4|LlII=h5gI^zP&;rMfH53Pu7Yn(HtgQ4kwa2Ho z2Zn1~2bw;O{^|zj8|G`TKMxKqZnkSr>-Rk7rg?J{dRC)NOX-eLuWaCFWE3|asGbS6 zX)&03E_nO%_z~WRow+{eIfh9%25BP)iizpBDU`MzNUi`*RJ{H*u$5a8aMMx)ZfMgy z<-;o&2M(bQomWHLAcihT$bXVD?u3?WdfVkXf#LO`TYwDAlk1t{GPiy8&D9~blf@_ThWrNnt>fOC!dp(lup|}P zf6BXL5hY8@Fm%Rk#au;|Fio(5X7JbDs?v>BU@Ev}K@rW8HZ|}Kz#eAx0q@SCt%*!Y zoMeAMRzJ`7{aE!`%U=;fVMNF7he&XKSHH=qM2JtCY|VYuIz}F;;mBaig{Iu(f^jqE zh(Lwn!mqrKKNUgNyc;Bu)E%k#w3s(*Rj2L;r}7@7i^e!{O0U|%mODtYJM)#>a~B1n6%PO{?-g{X&2R0b%O~;8IVQhvM7Ut; z!yS8kUF$9y&2Mv4Xom8DBTPGkr32@E15)}sI*NA};g!=*9|RCJJlZ_n55T}7dx!!7 zI?z7?`*SEs@a}aB+j(c&XbXRINbTEPE!}%3u2vs;u2iA-r$l)!cN*60sRoG;4kD-J zl?6q9{CgHOeV8capbdQjKu7LD?F%`-)kC9oL}Yhd-z?%xFAmq%LWELSkv~7Dv3$#5 zBo|x>ZA^Xyy6a8*9}-OaUAP~au;;jb72_Tw7l$Z&X{LM=C>m$Q{Ik0kUvXr&({u?{ z(UI>F!*aJo2Kb&5$!3HzAZ%pIUoX4plyt@(ve; zI&K7Y&qQWHcuaQe-D|KKNTk1J%N*V0eXdhxy|p`xI3TZvkC zA+2<{!h1=qXD#Oq@D+-D!eoWhR=VJ<|Fk`N6BUmDag)8K;yiA~aDD_l?BI96Jsgm8 z+po~jauZTRW^9AJ7N5P``(j&SFN@nYix#hUD}xUa+6HsyVmB0Dl(;l*1KHk{R0q5d zf()>!XAm><(9$X&6mef;iZWK>UKcP0Nn2+YA-pA$a(EinUA>9n9KWVCRPsUu`9(y^ zZ1A??uNP8Otc>tE@ef(`{B`_u&BgVBf&a(WR|Q1%ec=v>q`(jY3P?z|igXJqLk~!I zNOw0s7&=5+Qes3zx?5U67`nTL8oFWN9{%?}-WOhY;IPj=d#(MgFZ^Bpm1!9)c_Qrj z+wSjf>OA~<2)EzST@Py#xAWD$1WvOl9GGf6?@xNmnbH>-s-VN4ym=T=*io*`R3NG1 zUB_Qo3|Jt{7`}@5HBw2W@;<&pOK;9hNJ#MRm7e&YXf;T5cNFq*YYyjATPUO_foUJE zV}#F>JUnGxO9cZ<#jtmDxgW>rQBjMQ`(8vJhi=oe+1CL9rOQHbYeJ>F8k~=pE`%s` zclr7gV0gI&N!Qo&2ABZ+1`THKZnfONv_zEdxE{=qTvlu23gmJnKcK7Z506a-nDC2Z z33SXD?0MjW7TvGEU!B#FJ!h08dO`6$(Ierz<~isIj0)GmnOJC+_r(I?j$2%jb*cH*Qa zDwW`;nVw#gZx;Yyey;VuXW)-fkjAB*dR)4@dF*p>cUNR~bc<)~4dM6(dmWj)tc(A> z))Rs*KL>Y!$@c31di>sjwVCAzwg{%p*i&-ilEIENzZ^sOda{vC0HQdNXnucu{MF(B zK=Vd}EKGVBnL{;XIBT-%WZy-AU{oYAz_dMOL+h7sZx2ySEQ*eL?x4{-rcWTJ0PwqD z&PK*e4P-fZ9czf@RPkZgW30j~6!96tcf*nF_FcSZ_$#9o_C*-OozSdXsD* zYD4u>ruJ+KfbZ7AEHSay8nH+)gT9+S#oXH%SS;1^^3|!Pwzg%r6wp1D(c z`nW=9(sFWu}fK03zRV`;AJL_`>KbkDejGWnF=TKGli6tIR57!a6O)_WqybW zg~-Fy(HaOXURDo>L8?~01%iYLU+t_=rVNIK^HKCM7Ic?cW+3_M;u z1!9~pu8+1GCi~R6W2&Ky@{i4+?A-8C-C}M-`(dyYES&WbG)g>m^W=P=7QP_5JC@j8 z)c3k)YUzLP2m3Sdez5s6d6Ji5?`w$%#yrX_KG441It zW;*AnpV}pyODha6@_W*_2+JOavHBZ2gns$(iTM_>FQ=JWr+H{&}!QHB4$25s| z{dCB>PEa-Eqr0zjS!$Ttb5^_EFdTZ>q~$+#fP8_X+0Pj1TCVYZ!lpv^=tP*>gpA{% ztK$Xwi&EcC&aYRHw?;+5;&mX<^Fqsw<88Yuw`MQ=>Xd?{cnc}clCyI65Y6Wx@u%` zZnvxQ2CDwdhiDn6k_o;~c9%FMH7^Dusj?q=ZE^?zQ!+)|+F*U|9O^O+Pfd=dfl#??9PqB5!*W{=gs?1(X8*;^cLhdm`ss-Z{xRq_N6!J`tR>jw*cP?OY{%1 zRIBXC~I2TfY&o2|Qf*E7j zYiOp}y$nar@Z;1^(RsTR7)}_6t_`N(q~PyvZ`NC;Yd>+{P{?^3sZ63`rYfIw;YE{; z&@q{$o#9xMTV@ipj@;0IgOf<7DlnJ&H|GCT1^?k*r$xY%KK8!9YBu`odo_P^hZa=)#25$Lv@-}Gt9r3vw$_^B%o&;r! zzqMRCc@qe!i^$&)dSbqY#X2^9QQ+|SQ%fKXV)&2j&n{+Ke#Rrm>2*e6x??oJk4V8%@k!Gr(JgF)?dy7GcdR~p$UFB>v55LXSEsA(() zzMX^4vintDzq4qg|Mi7i7p<392=Vj3@7hj!4|tOYUP@E~Ye@L_VRt&Dxvp*=JMb8Z z@hXYLz9jMZM1EY(x~BRAh|zVu=N;4TB2i&X?g_F0ICNkavDNFp=Rc7qQ@y1h4CtCL zoW;$3`BMv}rJIDyywsT*e?RS^0%amnPp$|)+l5FA4m8S-0-s+BVI>!2jBfy2=lQD_ zaOGZG|IT;rasC~TBuNUv{nS)IekNmQ-Dzl|up6KGw9 zi8yo2R|9rU4#a$E(vC+b-=;~@`HgkLvCxWqLN}fQ=?hjJniD1y=MJ-Yb|5mbR!KkH zCwoJr_CRf_LJZ+TJ8<%IclJL=Bqr;jwFJ^@@v|k3_~8uVVT}&a%aXUpW%B`33#iX0 zZMRK18R7xA=)lXH1N2p(WL^k_{pj2w2l${X9DsA$0h*?|frU>hMT>g+)H2FKm#@3t zRIySK?}yHh`uNO8Ao}1S-Tq4Y8uA+YnsV{mkP%T?iqBgy4ff9WW@QNGU=tQi*%6l1qjQVh`bl(9XGV4Zu%( zmY7Y&G7tefWT2oa|AD$i;Gsc5E;ZMvCG_OdRgIB1;5R>Vt%x~s^zwtmVXLIj={ni- ztw)~oS8;T{7uo0=-WxludL*CsHcsHF!To9E7v3QR8f_^-3<77~ql|735a_kSxxx1h zQ2&vEw(SzhOQNY00b~tuC69R^DrG45Xmd=Des^ycZsWJQ7TgpqA5Z%S6#aGJDQ!HY)w^O22l$ zEQ_xKdPrYzuS+Mb0;a&C-R}*2SC7dU6U81Um}IlF(}O@{vi33DQnd1R@44jc-rJiB zlk0W46wyvg_i-vw608MHPrLjs3CNvGpA;F=X6veOa{4{C%cR3QK*|GUCgn0!FqdVC zi+Gfc|IO(uF1#8Wq^;dZE~4$5;(}M_ys5UzxQe^_gTL_OS5dP0rSRxAa3er~TBxRU z-}`F<`g8XPXK3MAsVrd7HTyo;HP9eU6b<+xj9I4}$JAgid2Q|9-xcND9kkuI_HBQQ z&bE1bal1^1W{~-N5wc>mbkWpyQWSWJ3;J_{hIfay-47cLWeeRAv|aD!`?dg3NB;jB z8GzUbJq5Jt=OQL`69;G7^%x)qk1VQO$u4{}WT5J)WNL`Ri`fnR+)U0S*KU_mEA?nnmp9?y zU;_3})3i?-)gFH2M!o%LJZdH{xfw|wqmT;_wAdI5NOd{kuml;7+yV_LYfun?q{r5S zpS{PD2AH2PV_7gGE}1MB#9&VX)$o|0{<$}w){|gU9<9?5QzA3jt-%{#72Cm3Md4kT zi7U9Kvz+$5LC`fwV6FNoYmr9LvJ6Kld1?YDa}ot42VJ!aA*_?J)V2ElEh!R=#ehlD z4K!S@@^ELcL}BY(*TZ@0dDS3|$OH%v8#34hye^(VUgZ@#UY<&!r0fl&Mi6%LmA(@^kj(5+l41J>S^Rg+ z>J!a6OVF!#;<}Iwmlo|Oh1ZCTku58#rMeR4)=XL=&kyHgW$-|+{Y#0S2ptfEFu$*S zYc=}$f}HiZW+6FJ+|-4bG{x*45XdR$8da49DPXj=Kwp)=nX=8=hYeZ{KVjBFiVz#A z#v*U^mfzHY@RC&NC!XuOG4eD&mR3|NeZFpkk&)tqO)5wG0TdoT*fiRLVuPoV=c-mY zd;F%PwY`Y*^dT^*r(s5>n+Clx!onR&CVGUZwi5Crz>4oPc{W)k*E!_(cBnWPMvuv7 zIfl2C!M=Vj--5I3KVEA`fXUYwWO8Y(hFBcArY82M1s(P{-gxx7V}O=!HxGndwzOK0 z{=VC7LJGEAzeVOf2K^cT_Yasfw4K9w<5ZLveV0xo4+ABDiXCG7&M+OQGbsQV>-=cv zym3&vY{s$fIx9QE_w}xMh{wERXp6__0gFizv*IzqtcJns8X-Z8mh}{ZRUk3>ZKja> z*hibJ2sdFnIzwRtpk3bE%rzjyPxfm_;M6>pyLs=}{XJ=?y3v_;-}Aa<62t^5VB7xT z%&Y8UJuY7v5viEHz34IdM$R75w$X8MHF|OXO)@iMh~sB+TvO3{6szAbu2x*_eBL9P z$NH*lNc`krt;48|4#Ku~;sceR%a)=N-^`45AeVOfQPX+h$bM7vZPkb9KW7e?LDIIg?s-lU4nrQ)tg~vI zQMwfgJGXIj`67fsr&lw+O+T1tH;fxzQb+PWp!f=;#j5d$l*pPeD+2_}L-NR&glKu7IMRO^>{&yo%d3l{8ka!eRXZ>MsrslUNY$ zdndC;Vd6LEfom(_aT5Mp7G-`Dhba$*=dQO5{cEj0@Tx)u6C}3urK5bzhB$E6K>!45 z7>G4FyAUgskT^05K)01~@rnZJ=bK)?0&GI8{HV&H9RpxL=Nb(k|6Ag4KpY!P0g z&C924cxp8_J2o+Qc|K$%OE+b}&|IWYhfU+KvGO+tu893_DN=8kb)7F@Z#J`{H2gqtjWwGH=8Xgj*P8 z%k4Ivs|>q&s+*!euJG<>c2?+X^+Y@&C3| zz`nTTBFZh`vN!OUaOrw8)o+Y5@KCgsJ1!3%dbM!FV-(s4P_=FV(igC9tSj5nCnL!( ziE{#coQLgLLlj-u;g|@;g*w%)QJCeRG}38Jqy3!I@J_0FSAy$YW?B0@mYlC^fk{6~ zD|?6Qg>GDv^&}#!7$35&6i2}W|4+Fyrkl$y6!&G}a=jeRJhuJD&a36?eL^z60-f5- zg7BM>5QVIeLCrUPJ~0IFq)8j|Vv%CjEQS7{6jfZXF+d9O9` zk6YJtCfzl$H6Y)5tVvl0H#{th5^Jy=2lh<`ZCJQIb1jTV&uGDGQlO9UK&c>5_XNIu z)kj?myHCo=JyFR%GlZGDue^+7KIqIw=3RZZEfq|o+;Pjy+B&CL<1*)ZYFdnQ`Gl}~ z{J9l<(kj#%l!ob4Gy)lwk1DiOQA!yg+emQKYi}I&qKDn@Q{L|`-F4^OpA<3!jk_3?_v1@4W_V0bbVxpJ&jCj)h{D8I#_x$_iOTWNNrk>!Mj&{~V8 zqzzLcU5=4ZHF;fPK_Y|!uKIET?t}s`2x&zy<20vmz(hu~yWi#zUfbpdr*~td|04le zs$Nbtm+HCu{-Op}Y7|@uRq!m)rM|q}U*!>qiV^oerSg!zdZZ%qmQqn=QBd5lYD>1k zZ6@6BvNJI79)00>EEJp39xid38+&c#e>GqfaO8Th7+?50hTBq;DrPqZh++clE2^h$ zGV!SitZ%H$_PsnSQs1{Q%`cwhS$<@OCwyC$kuNRip-@at)>rl096EG^b&8wl%#b+kVwk8AzlUkQrms;qC*jGcGhS7c=&saNthr$_RbjE&NPeyzPQlr~+rSw~bpu3CgQBm0oH5DUZ% zfKR}aW!OEErsi z-I8@x(P#P=`8a#m0d+`aCZF_V)+CT&Z-`KnNYIx6&GoNVkG8;;H8e>k%$^<)C4mSD z#-tq>|0etMPG7&)lusvGHE*4gBqSAzbF$_QjCW~Ft|40^pD+Gv2wlqpzzI@@^|JO3!!j-856IwD^Y4ZrY4 z+an$xs`MN1P(J`;-IMo)HRIMa2i)0IT#^$)-i=LMNi-AlxR?=OAiO#lFN}%-HpKg> z0KxAx31Me0Acx}R3W!jDupNYhixjmvs;i5hR=GL0wuq+xrsxmVCu$$$qP^?S^ioT%I^LWF;_hb?K6-6tQH1lgvo{!ETNKn6}oNuoKb^ZW^ctv z7NqM(QQ3RvU*2!GD<6C;Y$NT47VSC*p6uYku|Oqk+bl&Y>|H;%MW=L7Ccs#B7tj&J zpa4NewX&B-wfOyX$JY?fs$LO-5U$VAps(%(VYm!YMfNkLW;zt`@gFI2KSSAJsIP+QHX`LZrw5L5%?|$>KZ5!N(XN{7xmv4?66R+w}Id zyF7F{)>(T*DU1|v*d3;KXDHRBti`Qh)=3+w!8TO|X6rNA_IeGZ56C-!-I`{>tz7Lvan4|)xsgzV%4i$k8nvAOs7!3nsvTog^P6a?On z!^8VAOo9S%qXI=^Y(v1>QMZ{2HzIqVhx;`BrSJoAPUmcQ!edB^-T)2izkpUg9$VWw~3u{arSUYuns}^BIi; zg8MWhdf_v9nAqU7l_oW`UD-XQQNDWIk5zlf6#MJou}T>UkWifj{FEsE4V@Xp28|<&)=4IbLr7d_eH(eL zWz*nV)_IXK#jAFQYDO_z=6Z5{d={O57@&YxQdmF;!(8nL&tYY%?5M=T*@fN=)Rpu! zBCX!}La+=fGDLpOLKmH3cry#B!3J&B&gI-)e&}ppPI*)(esyUkdH3c(Tykdn zR+5+dpX6Z!!eb6)+Hl)H)mkbbuxMW=dD=r3y2azXgT@UD&rhCYFfH#-#P6C5`Hn`5L|gBNrkXe6 z9MgY}pM*A}nvD6qQP3jk=e&foIH{I&!st_I@vEr(%r{$v6km9_>2$VvFUI)P1uvLU zC+rMke7oEiDPMkb(HPL2V!srhhZdfkjYX>n=nklf{k@EBY4a9Ug~X~Tx1tRWYH4q5 zjjkC%-z6mWeeSkr5QZVwXeM!VD9}KRvqXsQ|8>Y|er{{U7iP9 zf2nOrrKz%0IAZ6EXRF=aJ`II~*InDODP!al-G+%X3g=?iQ2W~fqX>ucR=cT>>?LU* zuXT1cJ{X&%a`n2pT>Wv*iWMKGf8wj=Rz2gLHzG1aeDiEg(BbV0>XC`xfPLK~(0=pH zA|h~;E?jWGWg~^GHx9K}cRwCU=6il1>1i1Omialgt?{ddvia1)QyeJ_0u4=%u69r_ z-2%}={&n-$!&9x-cD=#i0Hoeh3NHnTjPLo<@nMLOs9>0X-6B57IA#z)G2=X2sL5QG z<3#U|*698R8IQiq&QBKc1k&QnI}|$k1fm{l52JFT8K*0bUe$-jlR}Y!?X7ceDG*7e zexWKG(BJqC0v6i8_qk^8u5*ZGf)95+#T`9OZh3RA=P_Z<}T;8b|O zPA@#V{Z_rlaXy7|E)jVn@ApG=eNoTh>T(I4MGg!{i&9&UKOy(uvHK689sMucaoSFK zbLet1{YDgwh(I3`uw?4f7kY4WrQ7aRa&Beo780O_jY7gSe{k+E{X5%CV|abLTEX6h`V0s$eQ0*s&+V7)BzaSf0eJYb#!PTwjcZ!IWkU zcmPqEV3Ek;2A|A2JjtMAW3EYn^0mWBk?%_jAo)+~oG#3BRAp#ZA7@+WQIAe?4^}2s z%s3o;)x7viZ^xw5rByYH%>V-fdC`nm3{PGKcb3vx8YW5H`~q}}oa~{hGFI%>{I-;o z(Q8}t1%Y;c!QkV+Q*>Mu3JrdD^Zo@%kX=E)-8y#o3-3eQoXfxAOXu$-_ZmwlMdV2@ zn*Eqy@vY`3k>FJ_x~m&OzqvZZ-u%pg$NdgM!fAvytU=|#??Pp?DyMkh#nC*P7WwU6 zuk)URd))_sq(}8Y(lhMQ=G@ENdmnIGO4N@UX2PU&NtWiW`gf)&&mx zjB3{Q`@Y}ye(?*o;F7~5fWMH|lwz5Vob>CjYTo#g{OfI7Jp;zO=Y?vlFMw!I6~V^B zYaKVG5lpOjQ<+E$y;?bL2pgO~ijkpS)s)))m$HXW9wh*V z?!tpb`D|d7N!go&NiuLq||__hi*vSefW-t&ICO7fh|Tk&vm= z3S+Job(JE{MXE>(p^@Q=0mEe2bKO{&5*aNr3EC@T;6?wS3`&(Z&mjGsI61Rb_bIHc zM_X|QO^Ah7WQyL4fJ57hmP0wph5P(_q@&SIk05Hks`muFVi}@4*)=|%ZxncV#T(E! z(y2mx0E8RKSnZD&T~AN1EqR%H<%9Jh+c*-#*IO%vys>O-@Hb5!{tqW2fGV{%S62&3 zdLH;A+D;>o({buW6WLy;Uy)+>k8yIm>}-1DV*Oo#zRdceFN1WJ5@bmqSh=ngP4=y2 zg@!tq!CK1cVN}q$V?al7U3$~w_-*JDZzF-$>(y_mW z(@O9E8a(pWUzq#v)#MKH6nEavuWXaeU0)espXM#Sa8o`PWxZ*+;CLnZV*g7shE!OP4dm5sOe-%>yD!ZppLlLGFZ`!fS@gjvCz5APLB1AT_~B*+n&h^| zRK?XpvoFMG_FAE4QmX9D1)gazTXz!aK(N(I$e;~jkS4^s2#cCX(_);$uP6#rv7iQ~n$x-!tt`(W)0{~00 zFxuItdhVb{9vB_ez4@oAyR;m+I}@<%Vq-=XFW3rdmbG(Yll$DnrK%KoG7%&UNPC4pxMYgB!c7UwGlnyqsJ~MO zY&c{A0XxA9>%{k0WFQ$3Jyu3tq)B~p$B#HAZp4ZBQt=)eDiA$nJWtTmV&dgwAx5F| zMf~``!pe|u@uq1v_y1JXM+~+wzbmXgZrkdA6$t!sTX1&7Xh>E&8E1*1$3*N9SPf#= zdn|^y*lC#0@K9Q8bFH)@`SSZ5^FAoLO{C$ndt=f}LCSV;EJ==GcxUu9tyqSN3?e>t z71uH*={vVyEV&%T4~mwr%{0%^v6EQb7yrSJfPVeKs2~`4uaNQsknW|1O(Z^0)MXy3 zAGB`C-~4<#ocaTW4>tpx#`yz>sao$@RDgqsRt;F1l11`N=hocze66?byk@4bgKa(L zB!-D^>Xg;goy9al42mXEIA#^*0ORp z);*!9BDtw*IMZMyzgbn*V*WsEGSeU|No!sqh^;*Zbs4~&6F6~tnellC!AUuJ-3S7a zw~IzM%;C4&ht97AT<(5(L7HhH-+?uSZ)g8+nPiVdzorDKg-@4lVE5sbrJZanN!gx( zB)V$X*SJDo;OZlA83h5rv|X%_5`}22x^w>5vnKh!>wkT!vmRlv)ITRT67n|Ys5dUF z!q&A$Yp->ss))~^E-IPSk0I>ObZx&@Xza#K1m2tF1RVXNrF{J4ou{U!`{-f&I=fK6 z%@$O2{4%loVf&Z2DE+*0Id0nCJqO0^hSXf$73?6ynjPcGdagLYe8fc zizBa_%fxq8-OeYiKyvr{Bbw9_eu6h16DfLKXSezibXW-L;#QlyMza=q)pxiZucq?Y zE`pC{U*d9mHi||57wKAsg6`d~TK!ck6}A2;*aE zf~@goHMVRT@FhkI+p@G2>K0KPL5A-}&qGB%_^}11(*7S5H?5rM_Bg zaqR%ZDS(PanEnOgI=jNgbXmm`bq9M(^0R~s+GX9jG2Tf6O5w5hJu(3sFYK|*=3JWp z%sDRsS7oQ&x5R!h`XB%r zz5+`XZ{@x`=4eb1J%~A)f(^YWuI`z7a(^=Nu3|TmD^bS>HRlKzlG#2E8nw+haV=?p zmc)2cITn74hsrn9M85!*S&$WA*y~BzE)d_S4AT1ka?i_%EB`PjjQi45O~$DYFwn~k z<+)tveP6=+PtyB$%O7FNn6k>qLTVU7g6dSiz=#yGRo#k2J-zFKOtOyJoSmWL%3a$_*K z0n9W9o>b~3E$XKHog`XQgXW=^r0p^RI||XU?@;AW;c6_QNl>DuDiRs4ITu%D2d-f# zbIb3#(J!lPke58!XDzEmTmRg@!`;5Kd}h(~26Wu6CFfe2B2_J|&AJ~fWx01670cK8MsR(fUbj6mN#KmewzyHMQ0D?e( z9!u1(xBDl>rM2eAJhqt#!biHcKls%-lA}s|auTq=Cm!*v8QFUfd*TqF14!k>HN#`EDaGSeIW|r2S4{|P9R7Gv4V$cWk?=1mtM-c%s>;FqvfxK27;r(bBm>lZLF z@~uUR7(Op_mc)4O)HB9UxF((1tu-CZmA1uAh?TGyLoN5!!l32CHKih^YHZ<-Y@mc_ zptO-Fw8)|<`=GHb^LLB)=e>E9{223j@Fez6`8%1(ETmdFI&aMFcp<<>(NOz0>b7yi~1Y=4RiPrn)!BiwNxIH`# z|Mp70og%LXV_wxX)`VJGl_Z;H25xqBlBi@$*3z{{59p+1Wm?Q-o;-==KY@hnV1m+A zbX9n>GXH-1(aTaMrbBnyA#L_2J(y5PrJ;2;RIi8}DYnz)ycJrFG-e{e-QTi0o|8CGT=+OrjAaYsUI2YNGdZqcjBg( z0j_!xvApW_KT_q;{Nyq}Yu@ud_<|t`S~k@JS1PGL?FO(rHFFKV_bD;3jmKlD(f`;K z-b8@4T6CR!Y*IbhNB@jV_BXdIV|SIHq0k48J}QN^YX7{tOQ%O_SPV&uQL>*OQPhp? z7X?hmjPa4^lVrNnJJXh_4oV_%99 zDgN&sI30!vGV%v|w;v6Ze4s`dwguMq&O9#s1UPSM1c&gOfC|P(btSQLLqEYes-Fo7{Rl7hNk+fs2kRGD&qg%vxOBU+xF%QV7f}*v zj?wQuXD^}ji48{*J&o`zOuz!UHDD@1*h}0;X9A)$w{BNl7^z(bO~S7)-B$U7B$(M+!>01^HkW`82EVW;j^dx6ZtJZx6OUF zu~Tc0%PdRb(pQk6lR~0M57T!V(&-;Vl4P@_Z=&o+N+XT)>coPml;@$Ac&rrfQyp%HMaaaG zZFnGbdN$`X6ADE{(l8c(WU1-CzNqC74X*L_{qN$q`RzfheKrF7w(#liyl*DYTHc{m zLzeab#OE`8h*V(DO(Dgs{I^H*FroeSfHaAPSUELL3C*TC%C_(+bg`s(O+A0&%E2SL zD=Z3^XzLo$&DQ5&hsCEl#*01G*#EVE$9fEw1?$@iTu!{tlkU->gSoc!s4MgqHXp}b zkHazEm5qJTOc;{k-DegfLrfN-Dhm>|VgR6j% z;w3}89-ir09;o-}SYc|m_V(Kw6aK$m0*!zU5U^h>`sicrnOo`C>;45ick}8rl zh9|l}clt~rRC^+xe^~dex(9#EId;GZBPb}8$+uuaa+H>@M>d_KqCfV=C#`@9cH0lZ zk@oUr_q_YUE<@Nv_+dvnQbYS|T=R7!r3lza`i+sGa;P#Md91oEmB z^)fDMAeGCsFjXS()U_=sNN_c2PCDVeX6VK>ahoycc(6wMVG6VoCtcPqjV=6BI`RUx zWhZZ{Cn568yiLDN$RMwzaZ7BCURR6Fo!tI1>PpSf)k$0DngZY&4z^VNn8x^K?8d~? zFK>}7PIr|U&~aitSD>jfa40)Ci=wv^^)L}Ln)Q}$;s;U=iq88JQHzj3Fwv;e7O7VX zsfe9iNki?YwtquFP@gOKh4tC2bz`l^XC0r$iMtbi-=Aukcw(SG?aKFFh17Bt+6shu zRrIi*R(jsMtARw-lg5`h83_6(_hY=btC9hmRag4^O@Nt@eYU~Y1=ou)dJM~NHP@4UdrPty}(@ijvIEAe!=_>L{iPwl*GHosbb&}0t<}@ zjZorJ~0L+HwgM_tx$)b5`( zAq@!bRROdRhXnMUmNlfEp77nblB0~b^Kr4N3pE3i-H*GHB*@poA%%a0FCGRlW)8UoDBNiZfi)C8aDdrV3|b=lbq;u#ZDw@36mT;8@#6ZzsAZ9)#K zf)V1YA}1RQM9?Wb28?GoxudSA-;i;IiPF!RUi=86xOuHt{*&$xuLyie>psg_IdK zTR-2O6w-E$e{-3iCp-)avLR~SHazy{t=M8V^68oGl|1UZKl>2qbYb9$30gcelROp- zyzvB<4A9^LLISflxXfnRa#C*I^GY(HQx?a3pr$dvHHwTOwr(rkgXtMd>X||lI{PoH zfB5&MncC54uwoynqYPh0 zl4O?PbB2Vy6X^hfCba4wO=QA=nBL`!+r+V+@h%!!`EjO6XE zJq}T(koWO(9KcZ8+2DV!hj~JVRQ##eB`}NUTke`K8?hlA?P>9OBp2!OWyUv1Pp9G= zI5G|;IxRM_?;c?<&rn@Ii}e^Y;%ml92>rBrwo8!`@6^f)4K-N|xC=L?(tpl7|Iza; zL(-wz)E^;Hj!jf%AgO7BL`@3!m(1foKG^36lhppSP#ZYr`S|PWH#@prUjO{B7F>f+ z!fM-wQj~6r5??#jJFM_%8K8F@KRgydimmA+bjy^M+KnVze}_f+I$IX)YSK)(fHSbJqfCXpZ-ahIeBeHF%TESX<+2Hw=V3-;f$SLcM|B)EtsSMu`x6ilQFAsO^N5EY>g8{O7rEKOs ztSCHp`%o1_$)OMiUHL%fZQ(8vN#nFb$Qf;Dcjjr}7YQvsd3hZ{j!<7=mQ>jg*k^9lV))~a`qN}AFc*Z0R!S{fHu@qQMrbrGo z4+!TTs{=5^jqDfK?DFXv{W%u^ihZ=i>zynv2M|YD#QKsx%3Yk&>qj*bpuOUku?X6e zH6cZ|vCrsLq1{;pP19p3m}^f#K>%y=i}I6Ie zuCHi*Az-LHolOk3i97ps)L-1gTT14x97hWJ)A~kO%?t$kq49SkFyM4MJT-$GSBD&v zz8Z5k`q^s?iSxfzKcoZpNRG(LYO+pD|BDuBB_6w)Oxm{q_G8Ec9p>38zs}a%lxvit zgTeC{Y-sX@!J+JwS|1huqZ0OBI-7^I9tKD1 zT|8wi^KpuLCDZAaHkiTSG1lI9Zc2-TWRV)7-{CpWbND6Df^)JZ1>@^UQ9>=RJ`G#k zmY$AbgPUzQ_j=GQ9UVIaFyihQ2Cz6Crx=tcjYqNXjC|j0U}n{i#EJd;<(6M5p9uoE zjC;(ySS&6mh-fCR)yGX!6&*p0rTyv(ETYX5`#1!u!TPf>CzG>G9_6 ziRPjfXjzeDWT+xw*XrW1P-W|BNewAPe_-E<*?c}$Bzihaxa!HJbsQjp5br}*8X}%7dG7ee^|Q8s3^a$I|$;?LwAF8Bhm;0 z!hi@v3PYE4cS<)u8l*%RLb^MoL%K^Eq@^3)hyQv%@nzOx-TT~o?mp-2v-fpr^Xuak zp_78X?Y4#~PkUP)%JUqUCqE)#Waowo^nLblD*yxZ z-p1>(9hu(q%0vi6o6sq(;~(}Uv~B@?l$f=beAn>_&AClZzGHt2a9=Yh#Iq|ArkHQf zKxm>WWvoYQDGL(0S`79&ocb#9 zd~aI+W|L?mAfJY8~+F-F)R`;xc+ z{YdV0SlHitlNBcB;vB4v3h{3>CN7~mNKMO06o1@In4DAv`4?bQE!De?evA3?|96hZ zQEGrB%xu9fq;=@><6=vk344e?M3`Ctq4Br##%mr;kK0#a9?eqk;U2Z>aM$|nbXPk-Th5@-0)Dl89|;o8j2eLL8>Ui?V^cZXD98Kwq#6#!6id}n8jaC zT9pY`M*}KH0aJ|KZL7rJwk9dLG7DGq9N6k~93UZKC9K8W93n~s*uAZ-hm<&et1KRH zRzwg1Udhg6F`?WYOZPi-Z7Ys9{RDq4xFx`vv&$ZV@4|{qg_IEg`OQ+^9kFj8ac*(4 z62E@qUNQf$Zo~&&&zl^F0k2A%I5BU@o(j6Z^q}-le*;W3)Z41cC7+~Dr@@5lXu|fzmV!|+IOSvuA>36&`D(_H|Ep%1NHqt_Vvjd z%5eaPpgtG>HkI6x$qkv-(v+<-{H*yIlMFI=E^BYz`>lK5s*4;o);jwd;dMi1mO8~Q z#JXIuu$H$L^X*$NPc61&Ucbp1nMdG-ehQHyHXF&XSTsbEuY z&e*EvSTxm}h_(DWWHu-KBX=tel|RXX2ZqkvDNE(-j3Kg=oLB_+PwGMw!;o)sRtQy& zB>28BDpn2b9a;}+Rr8YF=bxkQOv8C(4t>YfG*V&OQ!cGEDLD7dR$I*qHx9GCK$a6c z6I*ll$r&|UthHw15aIA6NqgU1U4cb#N6MKu3o3(4K^4IAX7$va#IJPRqluWaSDwqX z2$&5=Bu#)M{b?LoSOhOKw>tMwcWX;wTRq>2P|*XSP7^2}twd#t)-{@B;zs0bi&pCO0j{Gb%|J4kLWZeDR+O=5GnQocKv zn%agL1YM@&+p>+KObvhL?6JB(`ddA#|Mp*jt`B2gQKT-Qc zK@hxG#`~TANRn*kpMPzY>*y2KL4JI>Vk=<1LtnWres7U%mQloa7&F|K)*at zpb?<&Q=VSf)}rTY-$)oTqOC~+0(lVMk2kmIKYh9F=^7QiMAWZl-)ZzUF8v9DL+)q% zo-S&sdVrHEYx;arH9B1-q8pc}nyufgM)j#U<8gSKUB<=ZAFwH@Eq69%1DzU0c16cz zoeH09;pPc(O}zK~lxj=?ye~%MQT$}2-p|eFS)!~ZPlmS7o)az zR1;*%dJ8gAS;`7G18)^PvI+W#rgzVqySHw)&5@tJp8$%ag7W+E}4=jDhw=8`X zr!@7A%MKZoIKgga$fo^^)r8I{ef)sJD9WbKQEc=|p>g%*Vo+=g-c!}@vGziBuX1Gd zWMuXIhfiFWxAGKhZ6kAf3PmKw7PnhB22>Jh=m&DXDP=;5v;TB|k!7 z{?4M5&|jGrlSU0%`sKm=lc|0YK^m<2#j$m%`l`XE0YByS@Ytj_a>e#r?j3*qd)tC+ z|E;VSpeRD+KdpO#K(2=Qzy4i-#@v2Ej&eHBdu^;Z6Ix;*5geXeVW#Aod%QvFx}lJA z+uWaQ{5eDRPL8HYifgBsLQMftzdhmGJ<)4liNHDjl>K&fV>cdknVMSbkW zc6;U0{&^L{S~x~N)wf1Y`FiEXKcWe+sCW$>&+XrnRm!NqzXoe#MKvc zg?V@xXADH2J$pvmXS9)vk~HA^4D#Z>j%7)vcBv(qeg!%hl6=G4&8-(PV?eb^3kOIB z=u^;sA*L1!IAc*kUx*4|axC5S*K)&b*U3&69Th*BCcmfjM5^5FD}2DNRqCT_uC6+p z@TbLBgd(PhxY$Ca)~h$@L17;XlofM@#dLvdBtA$1O8#-{@e`zD)6M z*v|D2f`)~Ko&XPQtimsoi+2_z+?a%JJ1sw}Po0YwckvW!r(oqgJxW$A3<~b7xCzjx z23j(@sY}rnoTIeDr_Ha+lWnRs{W5RjjA+~c9EexU!4J-FOY=A3cG8u)j3_CUatS~w zZ7uBr1)))IpHutd-9P?_1#Jn7&$j`#q^l!FoHWk82$nq--NSvs~joeVM3@&3^ZhJ5T) z&5yGMwJ+!tkWe1@MR0p%B9qyLDSYPhNp6XHpX(D{bBe$pS%e8%Mm--h1qL=k(@2H8#sM;XP|A1Qzk8pb9R+ zDbopy;{MbVJ+*p60WjdcF~RdvFi7%Z7AiCn!GMfMQ zhYiN}p&n#~!*cdF8coZR&832~^m->+^c>ZS@@Z(kHPpxGOLT}XcP5&-p8ifNmUydY zhR-I+-m9FdT_s%wS4sJot^zbG3!&VIs5m*8ABx7)ZBiY7=4ToSu<^+mp)^ zWi6_ri#EtFh6)m4QvKDz!J1@wumFXue_?C5D3}nbFq_3w@4mB$#?>s(t!eyPFKA=r z(d_Rzb0QsmxUE>t)Pam4Sr4y#UB2fm5a9e1p+D&#`We62jgaTR#`5BYpA!P=W8AXO zV|@KRqQfYQu0Ump?8|qVK|0O}!SI8^$>S{Nr7mAvbBFtwg^@J4jj$$0Au_F~*q3MS zZjbg^>yH1wGAr=K8Ssrtg;vr(Aaf)o0-9(+iBfX3q{fF8 zSKSj10X&i!69B?M#Zt5pdm~11fv<nx=^sKjj;9NvaGVC^guDwX{agbV}}r5=EcG;DkEou^eMCgxxGa z_g_vA5gm~%_?Nm#iLPScTK-O${L7AzTE?$nI%$g8OR?Fr*O9@RiqraHDy_LB8Ez)u zYqDY|V@FH99U5H<$FVnKo!*vujb$X&^jGj0%c(V7$qkd;x%)f-JOuK z0;gv+=KDwMtY%i!PJr=8=+1HY%9506K7J}b?)$j|gS`1AUX6sbcF zB|eGaW6IHJxZM!HSl61Wqv&h3+nWK(eO+LE&%+xnbBChY^(Y&;;XxM&5PyR~r-|~u z@5{KEdmD#eMmA{N!lT*1NI$@;m?7l3nh%4tAoJ0UVDqyDZ2~z?k3pjtB+#rLBN`nh zPkz>pd)tmj{q*2UmGONW)WPka;+;{E%j@Yl$@`PWt#bByCATIA5>p5EiHq~BMYw&# zwD*M5pg=3LNkcN3A_!Dj0|wD-;A=LF9zU&7Kg6y)h>g1F9A4H^3<21-_t{cr69^Wi zn|1nzT5RB)E#b1`Xu?6DaP?Mw`QG%ZlN0ls3ZA(|ow^k}JZz0Wg%Lhbx+@?CG#`5T zdTp}o8TmUw2EZsUctL1SgEFA_pDIY@Z08@!@RybKtwaDb>94DM1jUYv05M2?Tge z1JoV9HZK*5wl)UF%=}CopNgmO*f29>(7T_t_ix)9&+FXY0liA|#kxONckTO4M{dhc zbr)DG&*=TRGqf4vA*v%5f%Oupjs>XXU;PX|ZQZy#Dm-X7*Za_n@u4{~x?*TT(ZpIo z0AN6e_*93aEYteiH`_YszAYt8frH~@N<1p?72SQ8Ujmd1h>U9tpiR^)9>9%1ZLlPCyUQahIxMV+VTa}hGTE7pXgK~_z!QWC}bEOsf?TnlN4DPz!#yb<} zMXGzrP9ayZ%s z=Zwl^W7J*lbqIe{YP~!3emrl~k**$pT%``pe)|3U>0W=VRdC~P=*o#JsDnP%*I0Ms zc@A>?8B_>i#)W`7$#)h6r28EyXPt_JPZy7UmrJ*HhyEOrM{np0KA5ETiVia@1_b&0R||ShNO{sjOml2h;q4jvvTDzKn`NWA_+?NvI%PH3KZzb&wdJ8FMVvdqFWm^3S zhNo#g+wB?J2cgJBOzbh|vWGQo-Q3zVbsw!?foQ4}9rFce_dTN_QhNaHsFs8{VMkt=XBF2%Zb7ogi9q7@pF#rKPacd(C#Ii zWDHV;Mgk+sk1{ktVFpIVp+;E#YQ#2$dW6y(){8O6&1G*j`rAErHtQ8@t6 zcN=B+)XqQrmfBFR<^y+`aR2u?|2j!zSW+6yfMOWMtpZ~nNlMfnEH04)tO$N&+1<4P zUK%hEh*-*gq)N>s?KdxDw`EE#P4`r1-QlgTIM|kDn+rV%`zn{K+=X7dSpzN3dW2Ws-bgWI!m2*JXxhbiGR5O)DzwrJ4{a}VrMphbk-g28%9Vt`xXlV#k7f+dC+D=9B zwa1J!F&n6j9aqPs<*jB_m-m*H262(DC~x^5FEozv3sSP-w$g_E3DWty)3Zc09mz5j zq1r?_iF~~iYG#Uso0capq=M$Y>T{*wBu4Um4W+dUGU{Qd8G57ur(U# zZWo(bw#@*$MZU8Z%ab1>qSKV3OJ6WJXU^?3UGMvo;&W?8lqg(mo1L&mpKE>zK%Je( zG0TRn6w+yM7>+{k;EPZLm9+&EN-A>a2WcfPB=6izZru)^{AWxTQ>5>gMwRcAe>v-|}_FSrlK<%)NXF z#3+Iz<(fdDH46ZN@7RYNmMMGg-HyL`Uq}acoW;A0P=Y~VBpC?3G)oVO)r$Ub0#3S? zPNHWrp&4m)CPR1XOy3_?H08e@ix!vL`Ml_ zm?Z*8K&%$)ZwqFT z-CU1imma&;Rc|(+^*)KcY;z2~N-e(bu}}eYg4Kz5h^hsl%5$;RsY7r0%m0JE{u~8~ z5O&zNuWw-UvwihKdTel9tUQb5kexr)6_w49h;O6;XzRXV50~b!Y<4Z%p0(vjxgxw@ zc!|b(0_&V=F)CaSAI9S8_ktkedqtop4M``5( zEDIsvxcz9n6)%&$haGd#wI#TVVBdg4ixNG}Mz7KgkZ7a8z+p_HFz2UV!6 zV~nq8=yFmEx`)=^#wI5ln@@+CS|u#&+@wYi6hCQdUj4r5G@tg_zbu_}D*kzVoOO2h z5b?`r+9;FHQ$-U$=i|>QDXxeg%%6-_$BVIcq4)oIKx^tuq%@l!V)UQJ*@voZ-wPO2 zppGDHU-u+Y4qsCJ2(pKgFb2H~de^>_q6QGR630JjI_>F8(FH-INsI$XcR1a!XJ5Bp z{yRG^9bMLxaIP)5zF$yfBFBg=8@~F%t$@dYAa%{w4hZXmqu{zQ8_vfdzm2Ns6I;0m z{2}4mCudrwB@W+2V*Im3O%IXUR_#1VdX;t)adNUVRgQ^l@>7~!WXJO`_|IU*hMKZ06kYTvO_n(%)8*IA|+a_OhPRfl*qy%rkhmDX7Z_@tMsks5|1`{?Q+yu zaP|SQTmTZIj3;fxFOn?gTc(J|z=Vr46~`)k@u|NVI0;HwOj6L&Bpv|S(8&I0NT96k zavcu)x1_ymze z3t&fGPfO0Ng}lR;1@<)kP~G}Lcl+hn9S?kj_A(owmLP9OZ_V^M9wUd7y%n&ch{Ekn zh3q*L>=E__?ya3(R}OAcPVfFyZRl0KznKTjs_q9g>J!fCk|J|H0QTGwl?tSislo)R z-nF0O%q6f5+sLoK>Px^y&QDqDe|yByMwDe30;y0VNl~o5_f`!64qs_2bYav8;upeq z`>;^BiRUj~0i%G(QOa~N8y7cNontxW-=r|tA~g=*fg>lyxyB|G|EuFmvPxnLqwVZx z;Va)hxX(?28hG?8cfHrt;ss>nDK-W8s`M*o zfp0zehpI{zqp?UzzMiFa!335_QNw7uQ-xX}C|4gTZt6tz9@oP~YItyLGH!-~L#Yq~80WL}G8zS-Q>p8Q6wC+kd zSZ)Y42MEOcn#u{>`5O^rgt9ggVPzBsX0-iPO34u|VL$B`ZS@`{=TO3h8`Qy4xz7KW z`@Ug6x>!}@Q1~z;J1R)Y{czt4X^fI zFaBY4-?#JvatDe&3kFyI*5@8QUPPLLPejm+b#tMGCJlK0BiB~TgQ(sCKv#U z`M!0B8?GAjh|j9qjPFrF?x%-~8QCXOF6z6uddZ!i#$HDNtUKErpp06oV&5k+Mn<(- zr5L32oF*jjLROIy+=y}N`+HCv)4J(yiuXS1yAYi0gVW!Uh2NG_dt2eaU}^`%0nFxD zzDMvna3M&aWz1$q&RRuwN89(H=#!t}Zn$eX`GqI%BC!*O4~k)^eE`qerkNSZt${?5 z>wmXa@Q#@6yqcM`evp5SLiTO(jXeh^H7g_(;=NJ%A^%i4ATQIod)pq3u zS0tLxgKK4*XCuADl1$AC3P6Mr_{X=Om-i!T2|*yr^moDQ!#ESb@?8y{m+##xlq1rh z(-H}h4N{Sv$LReeDtNb4@U{!Lo_{@AkGn1*@dJ7^92HqMdb+p^Y4b|r zdOXHy?6Q%&;`k^YhS##>*Mi>XIC2(E%0h7G;uxI`h#nt=r-LM+rple(X4p*xqjl@U ztXHO9qO^SSfx9P=ozQe7+Q+`6FoTT;Lbz9bc}L&q^T@L0s%LfM2C?;eqjXHR{{8Yx z0{$;w)O^YiiFU;0QvnGmXr^f5%&>}t>62^jnc->UbhHwe;)YgGYdV);2)c~z1`2TH z$W5MiM0Ah0b4=NZgl5m2%Q(c2M)PzbaXfNuTeTFi;&Yx5Ib6%|agsY(3SPW^$IUH; zu|yvRQR11uIq+BZ$moW2L>Gjqu21J82IJ^3m|fh~jHS0&V@V#aRegK91_vbpQ&o}=}luwq;MPMj-G4sdH2oc6;af-ecS2pwQ#?eJS z0lBeoBm84^Y^J`?m+IEJT^WmS=AV-@O?dI4 zV8*K7D$t@(b@QTG(MdB-gm@38VF)eIEYFQrkwq&F1_n2b<}>WB1Ud_eLshs2;uW_S z4%m-3NFkD)Mfep}$avxNF=iwHDsKH3JHq+2tldSA{B$n9awyMoQ7zzw`c5GSp$w3j z!>?D3bMzep&D ztQjqR52gy*Yg|1MwYW?USz8|QmLAo!iOA~^4Oe8@LQU7AzA+`@E)wIsWQSxD&^m*x z6<89*HPy<#=hxknIUqCFFFK$~>*V9K|0#+`3f(yBD2@N4Na@JJz3}T)SOi)x!Bs*| zG_SS=Al(2(pJ&R0@P0Z|WUbtgIU^2)z?7DL=v=&GMGU!~&Zh^jHSt8dk{=E<%Y$jF z)wXY3I8k_Iky&K$$V;pEcP~7LuoMAm+11V!vn-H?(u7p35KEHf0!Ut=H25GzuZds}Svb8P)>~hmtWrgMhbqiJDC^N`U=||Ne1zZF&uiclb=y@5|xur+*S&`VV7p zhnnsOb1d>(Rdlg9=W2Jp-IBgcr{&t4kp+4y^994V5kDH1`Jl(6J$Mg)oW-3iedCO` zHO2bOA0lfhGfe>X(IFFQ8_4#6uwer#E7B=eUyMT+0tRlP#monx~B2;GAFSl>84r+ zrW*8eo2IHnq4hRoe`r0}CBhUwArsxXuEW+9Wpb|BI|)_w=u$`J^#+ zj~2Y|G(?hnfth8JL9Ds$dadwmZnH9Fbm^*PJXckdD;XWc4I;^XAD3axE;^&-{sqYP zy;=!fs@Ukr6)LjeoKoUX7j>Mm;|89r!1g~4(4VR*K60_Pr}t-h5) zLjf`;ASvVjzT&B6Wt<0$<=UKc%3?r3(%fdLgjx>#9oWz$R)=1eIB2*JspSz_X>ETr z39eh|X+~`w|FiK0VKOeEI>_gNDB3Qe>>iI`xq;_UJzW z*>q88&4hLl*Vlpti)-nPdqsz-+zewVt}H|n#Lo@0DoUE-RplkJpg$b0*G%zBCvSC) z&zGktXq98Lnc^%_BV|@f?aRQ!u=E{p6a7NNrMx}P-YIoF`8n>_xU5w!B6;<-CFLPV z`ak>MqEKXb;Y%uR70$sZLXv1l8r!CWQD3tt0T?n3(ww}x)le0?RdxkZ;2V1|ovnQ| zaj~8xt5C*sJ<4B0+sp$qilAzFU%bir8)Oi?l?apO?u5LssZoZqvH%wWjU^AJeL<|w z#)$YG{HNUNKTcK0hj63T-=g>DrMg`o4W~4vT;38ioE&M+V84rMiPs^ysEkBHA)$F8 ziBvcc&`#DHH@Eoj`HWZDW;_G0_h2FaD0f7Y{%)R1B2H>7Ty(}~1 zX6JnB2R}aK6|;; zPa9uj`hainRE-o^OAPO$U*Z+`q`lW}<<|U@@FFxb% zwR6i34(j`|v_Zg*Qn1~At*=F_zp5q^pZ|j6@}-`XaHqgeBkKt(f%66EJKsmMwgk#M z2mdl*yZ;ylbXHChp4Q*2JT_B5wu(^{0*_IfF&9}$>uU3nbfdU-p-|AFZ7p=46%+J* z^?Gt^+);Vyr_rvbk|D!CD0cT)R%ojMsC$e<2vHe83{N$D2c8aCRYs^e2%*%`jAbdS z+|Q2Uerhd##<+@2%vqkA_ism$_kJc>`kk~Mr+>@1Pqy>Dpuz&6kX3sXALfjneWOUo zYMB0I>U$cE=5e&4CR7w$*}~0h{2uK%FCBn~d<^S!|7sXP8$f)E_#B9YgaS5*$b7*y z<=x5U{f91y_3JAZ4jx&^L~mhWf<))B*HaRXnCEFF&cjjY zQ(xa>;j%4`w>OYm-KA$5R$RXLHAnmVRi4yCg#MBQMuM$60OSKKL8+0Xt%X9AuT$+y z8eIpMSpMa>uC!f9qGY^q8-(-73;sf}6YJ)_CPvCTs7^jGc8>s6Qy1%@%en=`!N1_Ecywxj7 zuH5GdRcWea2v_dZ#ilq+qCDaYI%Mhq(MDzXdzrp!@)xpZGw_G-LVbD%`=f|kUKm5> zD9({4mP0`ey(RfGt)X0t=jrVizu1S`@0#13z9Uz&4+%3L#4}hOyO)V*{EXht)L6o( z90vxg?gbTpf- zNUMJZ(QIKjZFatqI-HF%g?Yy5aF&+g( z-f3ao3!!Orl1P4!7Q+n;K)*uvFB2c|S3dbVB=E3H?Ps#TpAakdcuReq6Z@FMk!S+I zDZqz(<`2s&gh=lWUIG`t<~J24RVh|4FR!V7M!FKa^y{Q(Qq&h-CbzYY zCxMINa%Sy7bGpDEdsr-?Y0R<~2~^F;hfJOgTqIv}0LUH_1O^=6RsN>d_T2L}*JaGX zB5}=puND4^FR9XtDc1Ildz)gT2B*5B9lf)HEzx$-)VoFe%3_-D84Zl(yt-r(yy)Ws zhU8LTst0_If21 zM#Ii`>~6CiGu{|aj_D;`PX-=y_Sn3scKdj%j)}Wq`wZi8gTPgIlp=b?N-1c4i$`XM zBrb9``MjAD8^ll=7n{rvb9s7B8`r+SPG1s4vKE1<-+)<>K z;eg6Wu`3~*43ATW9nHgIv(L#cL)E}t{__=L6c+ir-N{*C<_2nRjKv=+R;Ys>U|+?5 zdY>kekM0t1Qnw!m*k($F5Uv*C&g7PCPzQwzEl`98~i-xDL~IDOZUAT9i3y z=qyy#l>EqWC^-7z;>e>>E6vxmnS^Ox7st8XfwU zn(Z1`+71lUOZgQw3Y81`Qu^kWaZk9ma+7vVMGqGIP0j$`I^%0z7FbS_O`%&gkb&WA z$sIgGM6`qrXr z0zo^u_UZsL@C^m_&W3Z|nNws|Mf-aHzL;rIztWNw0zN1eEXSmZX>_D$A? zFk3BQ`_Z5>Yfj=JY6X2NP5I`n%WTp3p8JLDCuxK-^UWgxy=rg(zhMT3jcPVPQYm@( z1%wm(^I4k|41`yH3|h$4`i3DHTW&N5o5wN@3fL(2)cB%kM`@WyGn;NRWMv#o8>DQS zM|UiM$(IIrY*fvUz6Bd@5@d|$UfhO5J@TqH@EyM7UBWu&_B@$}jA#=D>Wt!_lN)E? zJcSggv5RxNWe8d$sw?NTUuiG=8J!&^dB=U4>SW}+%@!-J?1U~Nu4bCI2f?wJ(BA9L zoFmDtZB=U314{z%#u>OqN-Gj;iLPyJwzf4#DT8rpu{L$QMFbTVe}NsF`8Q92YL!Zo zV_GjrIdx1acABw^PcJ!Nw$$dEQAK}IltR(8qvhe*&2|%0zww?Yw8@ky)xP(rB$lek zDO=xf``iXws^3T4vqs!^R*rkEi7~WU#14(ZUVQHqU*-v+y3uFJs1%lIKoX%*{2;dS zgzUER)RJ*=s2||CRxx!y7>Ke*{0S%rzad;37YYEwu`9 zot;}tq4pVBTh|(3Tl)LsCX%NYOp6F#FOf|gE2LbEke?kO*&w){THe5SizRxE{MO!f zTe=~`33^aB%osyZPST-<(O@ZrDBJ zk6TUOLp`i9;2x|z-vh-5MbwpxeDn+U(d6Q4l`Pw!QdbIWuP=9GJtho=zmxi6Qkbw) zdV$;h3OG?S61aNfd<#L8?R9#R;9x*3+xqowEqBUG^>WWbci12*NmORfTcG~xJ5#A1uNeB7Mm)|akJf7gJJbe<%PdS|5 zzj!K zmaCkcoTSDv)FLxamp7wcM|O{X?X3k}0CZk3x5f^9;|-?m)j}u>Yh*Un3wt&2i>v&+ zTcW&dxTKF#p2y0MxWGq*ds$T{B;kgNycKHw_xl%*Sq7nH@F~1faG$vM;rpJdnH`&T zpz=YAnBNrJ$n936QI3W|#`5NQC*LckZkLQ**xFa9z4_`0QQ)^7n2CH$DH-;SL7ubw`zs3|a@YZTx47~@orOO1TFEYVRae-h4=;=ER_L%7cD5E3LY1Iv?Dh zr!xx0#qV6Y)wM)By^sLC`5B4}?qr;D{c}AYdnpZ5KK-Zv{{#6rbbbGt7F#yMA2e>qGB>y~WaO zPA|w-K{SGD5;~u_WTU(_{sm9aY@zbx$+z+8* zbiltiZ2IGGXz#*l)9AKugpyFD$q!l*@l<1Wv2=21tHg`sAE-8eTGJtr?Kf7M=AbJ6 zv@FEyRw68>QL>g^xa=XuoSP?gD$nCE3x55@DQA8WEW>K*3kV{n8f}k#KTIK}k zT!Y50=Taf-UF}z|p6=hQG$nl1PBvX;ExWC3CZxq|V(@*s)-eo>rnYs0NJS}+pJYpT z^+}s{eDNY{FAdYOt3H0 z$%0bR{HHk+8FhQCQ;yrQM&Rl`?G`@I<^d?A`k6?A!O|btwyJib310+^B}C7}qucHH z=losLvJ#E`=jVIC_Ow{k{^xT1xM9Nz-ty7u2(h&_SulFP@X8mN_WE$e3>g(j!>ty9 z2`n~u{UMd)Q; zu?WJV-Lph3oh_+v&mrLPXq3*T9hYS;v@E|st^XTERhooN$oe7T zqNqt9?eewu+zZ+c&Px|dopNPcUYfsY6^#8GOJyB&_OX#26Q9ov;lT`rEIEP6(X^3yl_r!=F za$ugnUvhwvJ(dd}YU7exs`DxXtEWI9GyGshFVp`0``VJ^#5)kVN&&80ZJQ~+{au0$ z)z+JKQ5~w*df&fUm0f-Nlam_>@56u42F2{#hw_CQ*C;V-yfX{|#HB)Zv%EzyUjcri zt0+sw4O?kO7he#bh5WAWHE@>0hw#5EiY%zUzpiMcQtQC}%EcbCZ*v)vCrIq0rQW?F zLxH?d##zE0ZLQb#RVsD<&{$y~h=dIK{`-E#fm)}<)4JWFK@#Olb}~HZ6oU%5^sy#i}{|U5@MF`cGIg^lK!~D zD#dLkp7#xa3Bs!_8m+nAaaPd^hXCM%L@$`h!~8rAxpIR!q`(M*pfOJQsyc(Hp&r=k zP3r28;~kIWV(-@3Kj3s&u?~y1KLjrbBL*nr#?IzblZbeZUawV3xx9NIvL!H%nkI1* z+QHu7S8xJJE`^U}8l|LU;V(m)JR`n>&0R>j=%PVeLP@{W}>*qWyofWF&MlhPuS*p5fr)%)F{ zGm7IwR-65I9VD88=u8xrg6iZC2lu(zVktcyWIMYRp>ldPY*OVz~C@D(Xx*~KGE66)X3p!i?)(f^7o zcwxcK@gy-72V4rsOFwayWmvfWfPZl+R{O;3uX>LQS9o7kt4C0B}LWP}X()BIH? zvM=AfepBw<`?!o@jCS>~O?Va}yRG=$1f^E6vOqP0Xr_7<6tF z*1h~7+P*E+7C7+OiWp|$c<=rL#_8ZYD)*q)?9?$ za}eB-iklALi4LoV$&2F$;D@npUUZ_Q)b2V z)h>NI;A4x!DN4YFNQ$C&ucuQBqAAFNPeJ3Hb*7Q*;)IJqijgz?=p*qC#1oWgkc}5R zfo`rInpv2rq&C#IH;td`5->)kiC+g)m^rHfaZOrOocGlJE;KY?7TSa!ZU4X47UwUW z+&F4xMrvnA`IAQlQamj(!Y~hJSrbqUvxNXuGk!Z|?crS~s_mvgctpJX{VxhZf{$-p z6QVsTv?-4tckD&kh}vn^7cMuEq2(FJg4QYAw5j>W!?}0N`JAOnNo=ZH>(V*=df$-@ zU+xi$?%{>4A_?OXY*ckAB}N)&EUT8670k~o9V-@iHO+tC3l9J9<{!~FJ@SCDoVV31{uh}G1K{0`o@q_?t%PYl2Id!9nb@+rCQM{^V)i` z9TSk=vN*D#w$(_GhS#{1>q@s603ksp$ssTfDSH3;YlA5f#ZRTBHx7U0d~TI(erytg zdtYsxo*M!1ous(hv9u(KsPX(~x>C60tQ@-uvwDdPr+QhQa+mJUspDdQG${gAS#TFa zM|WRR7h=5YV!;_9pqmqQ^z=EH?V_3gx@^DssDTHzt3eUlG|bx5Ht}_a65$CgfSJO~ zpaVH(cYqE(3PAVzPa!-|IcZ%bbXk{eSx8#~&M~ZBOd{FR&NTnr5?}+;V{B1CIo*DW zs~Pg?-zjoZBciY}u~wsV8(LpUTK6{Bl`~YU>lI!UdW_~pgt4SciE^eh8=tLBjaDgv zf=v6W(-l#}v&VK6xaU_`EJcT`+>JEtGUP{&C%K^=U(}-fQbefWE;8;rl19c^4xwOA z$dtE6pLY<^V2_c|zZbE}j*#47HbWGQ3`$N^^#G*??^ZqS{%NDvDPC)q1UAwRBEunX z1fFR=#&D-jS*92aYMy+q8uK;zT>7MGt>EIdd zXw)HgW(SG5v@w7~qd=QT0EGfvt>!k%XU*LosmhAXbQ#LBh3~b3iDg?RoIRZLg#|Pz zM0kaD!M3$WDiA4PZc?LcYETu#jL_KD?3!+9{Oy2LiMoZP3L`hInS!z=u`wpipr-2T z1q1RZ5E>MoU#W(GhXuY4K9KaF$ZHr-v~d9BwI0+6Y|wAlgj`+`u`p+F>P) zEyYYBn7+rr*4x`H`<86ImH@OGty^qxa`!Y;{&ZS?|8u7)+U#uouvPE)&C_Y^G;~Y% z*n3gTZRO3}GtfRH2abuYWYYI28mcV>e(->tDGwx*-8#Qjc4`@{lMbMux@L>SL3o<3 zUivr=j-_wqk>;Lc(NW5tFXAZ0`?px2&o)uZ_rEW(cbe~7?f{bpWj08R1tBqZBh7o7 zK#R9SUZ1BTx-{_WHyV;QU5=(C3}b>;XKvOsmHym};?On? zTlqU(e!Y5%*l$`nJ$zbUKcoi6!+!*M{|#&(AXyCL(E)Y`|9`s^Of%QyS6&kjebYIj zU0*q21LPga3*2O^UglDM?uB!!dpNTh4y`mmR|h(yKNQ;h z!IEJBOW-ExwLa;~GE2VVRWR(M2|_qls&QVnC&9=}o6=h=a9w7+ zGHY$kGeNKZ1^rYbe|h_~)JyW6D^XK`jHEPLpchn{-S&WmBn9roU_zz?(Mp5A7PR{V z73hk<3WY)Ao{2A&g4)cngpu@;m3k=PB_mbz(Fm9@tSCy26$CPlQf5tZM39$-tQi&5 zPwjHLj-jLEOyuyAk+`hs8f5trr~1Y*V);)jkQe~{%yVKp!zn)u|AG|^kmTG-7;PR1 zg=B_8T%GkJVPFx=;U|G20mhqAM)2VlR5a=5(}PboR!TxC$AFpZsoO}_Q~HJG{RBq1 z*NwvZig(`=GU$3eg{sN=?t}Q{$=kNI3jys4fZS7khtw5_BMt^#cQc7s?tiE&p?n9y z#e~MhVzH~*EfifwOrCsMB9sp@DVz}e^h0#WBsokp$%FM&aXY*p7H0}&pxfV(paJU+ z3guv%VjE&_VXtu-V;g4JhgSIuQMq9JM}&z3pLGlDI799`ql~2sSSzg%d^Dp8!sJ$p zBezxtW%6i8wR&m=kOr{Ch44Y(U3jBrkfRVT__}laq0|-p} zwzCP~(ou0ENK$OnKOeQc-liM=D)@q~_Z{6QHY2uAbcSq3e^>muGT?qY1Km7GEvR5S zC*u>^C$v;;PiQ$ADF{=Zr3|4%A4SK4&nR?k>Kg4aIu2QBZ{w^9#7=aRs|7VChBbqz z?i!T`uD^;DtkdMFx|^A~8abYl>z;1GN-h`J>(`R3z1gQtI2r+v7OSbB`*VdLm^NU- zS!}-UXZgfF{ai?MU*(9b`)IiMb0FDuLXQHPsnt8@sA2x5i%sKkL9UPJs*woCRW!1{?ib>4x`)1(e1~qPv;W z0zJzwgQb#J)T4qK1`NVG(8KZC8%@)F&_Q}XvL)j>;(FrVQ>G}!+NSWt+J^C_xHAS4 z3vv}fh2Cs9g^lFQZ+<8=5I$4R760$CFQ6*YNN~=}qP{LI0RE7LX?ym~BB7ec?{oGk`%B`$WJehBV zSoqAOFxr-)_cc^Q>Hku@QSFWvh$$1=Vp)^ysn2r6#KWpGC)I2p^e)gS|ZIp|G(y69wM7(0`oeWD)?DM0izmOsf zlTAt84zo43Rd{w~Q5>ww=}Pe(8kri5a0U&gsSI2uTt?8@{8KzhVm9o@#2CbI;>Xw~ zoXwn>RI6xfMtkxLTRhTCG`6gi8gp{%vilmFGguD&Cx4xzlTQWB5 z6y$Nx&u%?$c5X5fjHloH2Ia_k@`W&FP^I2X;8cgoit2OUyVibx0pAFvl?kcNR`ua1 zb@^2>MuT)ER%MP?d9U*95E(qfrZ_(J2lBHLPc<<9gv4mz;1#J7Z>4@QWx#|OyaR$4 zUV~(o-|j%5IKY)5yWWaEeG38D%T))?$E}A5a&o{9T_njA03ikd2#`aLjS*#a9?_@z z&$J^%ZsPmLs1febIw_;7=iT7!=rK6j?7-{x{j!{T3FqwZ#FHk! zzrELl@i1=SU}ICAX5+>EjhP8AR%6F5`?1aq?Xs+sla|gt68*yOmPoFTcr|0;Q|Q~Q z023qiXY2WtY=$b0fSLWiew)a<)l96BeK%@!uNl1#Mg;M+lIY_>fJ{JHNPC=ZuX+nX zknZ4H&^IN80-`N?uRJOh^Ee(maxUR<4fc4M@YdO~KNT+=$0BT!Hkf!U!^=O-jk2ln zzS1D$2pMYpkoQSKgInVi2(j%S=jGFYMzMXRWq{}hfqgKcK))FW?d3lFuI!gVr}(H? zLm}Y+z`peV;eP=uL(ev(qC%g8YTeeI!SMIE+vcka>}Ao+F0^0W*AnbBYkoLT^T$J0ZMw&UT18l z5D-|utke7*zxsG&_K#=J_KW^ug1jCWX_>~tKWkp&v>YsXU$E%bxcP?t7Gq{LJIb>oW4f9Brb*O#fK;M zW?E9BvH%8x;2j+P6qjD5Cj0BUwYxlw>M|gETN)66L7;c1&b4P*^S8-xlm+r~Rrhjq z+~c}pZ|eiFo1bMT-XbOozB`CbTh20=50p;T^~n_v{TNTI(?NF=3gMj%Hj3)hc^Ofg z5(7e;xvg)xHP2=N=4BNRVrCa^)$GtGOOq9%2i42wx)zxsJ(^nAgnJnCo;O!v#~20o z_1s9Eb5H9*c->Zt$H;d5_5AJ@_kSLV=KWrRW8$C&J1Rp3`lAeHU4`GDmvi~4a1PEt zm8Z35=Ublhtq~Nt!q)XE{knHlx~$QhQT$aJeYQWG2NRW+=4XK9u18(H0>%VNNQ3hA&tk`4&qg_Z+VudE3tPH?|Fox=ek*}DY73uzMEMWwOQ+m zB(`oMX}Ha^cD>e|FW#mv1!~lQ6~A7k-CX0Deqlh`?4~1KQR$pe!|>F`rYck~x5X;GCveG<F?pElK--w%apwpW##x8 zTX%d{%lzM7Mkv?j+PLI1FcN@>wCGXl-MVqXSy2upiTRF=%RO5|C7DBg)B2UvOMV=; zLV?Cy80IuKABM8t6C&bwNlFb*09Q~rX<*Vz(SjD{ zx7#IOj*DJe{>T;CK@)E=Wi1Zv6qa^IgErTaopO1^ND=WcUEOkq!Q=h($97Llh-48MrDR?0ZgbmzUs9fJSw(^OuiFKGBK)S-8Lc0z&RrdsVtB!Fid0_D}Tqvw)JW|y=^ViaDQRo z*$#3DE(Mn|s|*;VpGmR#p`6h04^s=oL3!^Zck%FK;lP0mFdrsConEf?9p@yiW@_#F zjH*@-b)`vB_a%!0-e@9-;95;&VFdHM$+E1!O z)~@0+?Au#SK6!9%a(0$xw>AMzBBt@q!^vhGUQC;u=<8q(iaaD$K9b_wLAe#e0GQ}& zJzz5=mPjSzs_gn&Q#p=z@*HO;Z{t2yCdmGg{iDhyQa3}+D%4?ui~das=XrF~^~Ijk zz3``YTPH8^+oKlXU7z)*ke~gw`XP&mZGzWs2%PYxksnXlKFlgS-Ym2KCdHa=d@ZHzT0dwC~^46cOB zTKuQvIe#)-BaAt(<7iGOKCp#qJfElCaOf|fCWwER!yve(7N)rNYHz9#;K!JUO_E}5 z>v6Yd#N#O+x7wXec4+ij&FvVs?|k}nFZLA0+izHs$YIUYk@YwZd6f(U@8tNrWk(H5 z@e9<1d6*q9a%JWjk8REvj9a@_L^=wal?-WW{h7Z8IeRt>()sSfJkq0{Yn&Lm@N3;k zSucl+YQ(x$npkq4`E|MFs8=#Ao&LFWvD_niTa*vs?F9%Bz#xO_fqOd7iqc5ZsfkU| z{{ABIsMqe_j-hSllM{Hx?pr-jCu*5#v#?!SWo-$S05wOpAyFlRVpq>CrlB+GE z0v;CEuk9p}NXIF*kJpc*tQc?S-!tHt@=wc)rTW{Ltk=d%&TV{F;5^nB=nGor#z~Ym z5#JUKFQX1il7?aJbRpgZtZBg9Nf3^Ws+=kMBB35fpQDar|t-B=} zD2*=73#m>~6}o3;StVx*j%)0c0gosyxTXP$#_d4u_5IE1Je;-uYL&~oW7;cCixY^i z-EX*W3TxP<)U2u7@aMWX*uOdZjs3uXc_8|nvF3e@364G;8+Kd2$^85!-oFeO#E%)j zv40F;ETpNw6ashv$63FHk?0?IAQ(9UAc?cH9%=nhkxq@8Shanl2U3_atoq`=G<2!M z1<%ScH7*t5(a4X%z+6m@4&HP|4|cCx>$Y-v0YHIw{R|XYDU21CGBd?;r;|Q=cioL9!+V2{(ET|9|$VwU! zcH`X^jechiv|jVwL+GG|<{kxONLS@4gAbP5I$GT1;o{ozbC6AEjq1XACiU$hEdB7S zt|zt6!2|uZqV8CG{~DEo4YIJX@*3@FIFh(Isky4*X?GIE10ZaSR-m2m`d7f7vbJ@T zd2=M>>c*I$8xn;D$LF?0qm_;Cp^Oy?ZHezNTX-5vD;GWnV<9vV?ox?capfF-{gDs^ znz#k9oxz4D{DreU?nR%i3K5ZBY1$pw(a&DHS9)DT*B1&Clf;O`&P>-&l)I*WD3Y1- zRK=S6$W))L7b6e>~V4 zHZRmOw8~sGWZ|{MC4PArcGFaMK7$W(Sth7m44l6~w3C1rjnjFAgm`$w>&Y7;fu;CW zAE1zW4{5AJ&(cnA+$t->lhwZu6flR#C4c`q(+dOB%aS@*E!*~!BJw34Q{}xdZR>~z zxl`EUmhnTsa*yh#k&DfJHxzM*eAx+PpdG&q8To@K<%jJrp>oh*q`xz85Jjc&Uoc*o zjU^l!Mu(1jw0FLVOhrZU4lJ)3Tifzz#qIZ{gvPu$c9L1WTu45_{YmqI$2d;m8_wJ} zB#&!ymO9lsU9@k~22iRPzoNTlQF8W5<{3VBA1Rsp7p0c1*nddl4t*TX9`B#mmSHb! zCPpP7XI%Fvz6@{$BDDRk^p(g%4yVaai_Z`^dD8f8I4nC^I|qF6;n4tbNY$cS9rSx; zoY+eo`e!v8n)et)G;9~yH#AfpP8PUYz2THL$?~h+uWPXB_`ZB=O^UeKoXqI!YagNy zv_2Ck0@%)`4-85!_~z@#yg^Yr{+lJ`JSPT`)WI58N-vxs?$ln}Te~U;hv5d3u`2r? zw&pG&*F!GxK%@;abCyrjS!tn}SaZ0;Gsafjs>r%SiWn zW%9!(l>LbzYw2NcAn-h8p`X&p?5p}LkqnmnZo%t49s9o;xmH85d8kYqe;ei>>5Pd_ z{<_z--7ftwRwi2-;p^URvRad?l7BTSGILnaH1{b6D_bSl>by5F!^5BgJ%Gb?2DD0k z9WEf|aJ6Kr-_xaYLr;+6dVd_MTU+$`rPuO!gWdKtKpLjP>z@lB(1nC#Vn#MT^-be6eV%|*=$pBdfBs(go0$(%vC&_tu)t~>1JUqJ0oqR5x+Xk{WBlH79pA*8l zJ&ozm^pQA>ezscxQ)G!A(SL$;KXJL}j2$El+dahgAA*}1H4n0U6l?JOn@@9M`~?3s zzVJ?N&w%sDH$Zk&ux@m}~7STXSUs`I_X-DJc4%>4~D^|rVBz2`YRyZ&~%es$;O zW_s>05;-gYj5X#4^?-TRlQfh35E{Shydxw&xs{}CM6&jmWUXNta$A9J?DGhzINm+i zdT+zOM||hlulRU1nxE9n#ZvhBC|@)qZu(iBOrw%D4jV2E>1F6xU!g{6A{5H7Y}v?@ zfCd*FV(8N+7qWL%Vu2%))E+6%ATR!+Aa6~7V55oix(RFx*KQX?MDgrZ%b}L~dJntI zRCAW0I6@S**g{Y2v|5ucx;n6hlg)vX^k3pUPDF<^+8-kJ(IK?tAXUUW3olMf@dW=I zwlEM0Jniz)Fa3GMFOj)c_KQ{l8V!miz5IE}qyw`1FJwM}B8xJAr*AlQZ-yl$4$qtc ztJI6+qN2xH7tC`<>9<^awacyTSM3G0_v64y1gQCXiHX^eX0L<7b6^O@TxQ$6%lfgT z&v^Yt!^DeEtbwz-NX1wEyB~nsRD02aaZlj~GxT#9ft3Ta^a5wil91crOk!%F74|gANuX%^{=wMW6k;o1636B+HfXT!ZXNp+j&JkvPn*b{t_@_c z_t5=7&Xq|gdyTTnfthbhF^5xNjews4j?q6y^`g%0^!}4v=N-(HU(IdL3~r7vXfXrM zg!GZBbU)IfjoJkB|FJOoYlyKtcRDu{-i7^}{xBVu71Ci;nrp~A^V|gNvpr06FuTOS zI`{pvW03D8#`*x4*ZH}}z4Kk-ex-}+aGTsO!TKV=w4cFOD-u{*Wzoy4z%PGkosO{Hypi) z&pf;$d(&BN-^o6fVHk0jE71!1c*X}wO&Fi|NUki(hE*HkVNA9miB=+P5dk0c-#<%}FF)6&o z!)X7^ii(1QcRwa=iVl>Gfk4P2d?G6WXb_8>nn3yp`=+wYq_LWz0gbZ~Eg)lGbMaDm z`Qpaic=mcNu4CboDwi%7jE3=GPMxaK)QnzQrVJ>1m5;FoGk?$H3d^oshTx?Ky-{T= z_l&))cRt1*7=9g+X5(7qzV+SZE_cI0C3*Hn|C(D0+!V3xmfu3Mp5KG}C`)`xcHn|} z&wwy8R~(l>?vs0Zba(WIL)v8tmi~%s=(ccu1N$YO^fM#rxYCqv{`#Lz$r0)u8)ERC zdb07Re96j|q^aB7GpJNg3*`X*A-CJ^%mns8p5^t1Gi9<3*DVlFW;C)?R8?!1l5N+p{BxU!>5FbXDD)<0p+D1KvS^ZkZQHXm3<^v^mN)H-Xu156fpur`%1Yn{ z{)m_!NUAmtNpU56q!FdqkoK*6xCKj@x&Cx$)e{cP{N-quTgFlsg4UV?w`my(@B1P7 z+_ISA9w1fas^8mfVZ(@=TFmumB<1oYXn{!g9AA^1+GXJ`?B*`e`;yrEZvMWvM)#OY zce}b@z~h$P`y#}eA$bgxc9*4A99AOoISF zoMe=)AV8_a1c#`e{JT8JUUevoK3|$4;r_k48OPmC*_*uGn0@%!DP(9{+Pn@m!+$Ow8b6Z>vF9B93B3?e(%x!*|EHbLse1P;b>&pyhjt|9Z?GU zfa}87SMtd-XM*YJ^tUksfLfp;+VW11-VX%I-+l)DMl+~EQ0?~24{}PV$O`x%VC&63 z$xx|d(cKQ>8aS8Z^qb`t&{F`~0w3-@w?`eep`phjS${ezTr*WOsD)xZFSTk*pE8=U z`76NDK~JGkO5GE~nJGOB`K(^*jf3ZD2#-vdS6#GUsyQIR#%GCd0!89DIx*rQ-r9QR z=3~%1Q*f`SGqqTiJ9sSzr-Z#&JGL?z58PGQs(Qi|JDc9I*|~AoD?5RMD7MNvy^wy| z@$a!A3Qymx1I@S7fMd#ay=blKJ}HnLp6P22xcWlS{b>4K^>6(5R>l$Qzxjpo6^_DV z=SF&0s{|i`dR^tBDKCDG0?W+WqJ*J~#22!q+v+RB{K84VuF~a+zPVk?nSN!SpQ48q zpT`$Q%`b)_a{QHiqL4QrkWuJ%bvn`cAPa|UYzL}OAw}cqy$!#ogZPf0bOM7qS+J-w zZv%weCYJZT`K>oq%wzIoH2*{1$5HIOfDgW#T7Nh<= z=1~JdW&hQ*h7!qBjgX6-$+`HfrBzhv=+vy#x_A%-Mq(fM{6`Q$p>5OL2$Ui3g@I^- zgs<2@qR0wXEc1{~mX*2sDV2-{bL8H)ZugqcDPKB7V=p+DL${H&$mo>Ehfj17*HY2p zY}1RA!11}-(lX`5M3QQbKcn zExKViBE^;0%YybxS&iZZV5i22RBbl}{T@;<=Qw&j5W!%Vk*k7XH}JCIrCuYL1~UwK ztV3a%9trXNN>srEVzG?S;>d*!k~ol1ij=n0G~|_?j!Eju4v0rPi)eIz&HcKWGV3|E zV-!n5Z#1`CE-w*{rf~8tJe%^x(=WM6pU6{WMY2?=%=qF6ZGA;!iEzJa#?ko`{u~Dt zJFf!UfR)Hu`aD^vD6!(N+_I%;__yhAJ0p{JhQj8?0{xCsee*+hjdl5kS}t5GVV!!BMDpmw;gKH*-NqmytNA1? zO2GsN!EiJuSdD8t2?!}0>EiiG zsoj{Mf*il&m}0!{&R0s2OQu;=G`1{RWixZkU}Ffq34++LStIU9nmR5N!5MloUJF2< z>ey5KXe}e>Xnp_Xa+hmIIIK#Jaj;5{jKXT?T6Qr{>3DuqS!!~x&YTJ%BKd?BX=5E7 zLV4}$X;ZY|(@B5E&4NYi$BF>-zg_W9nRuIR+zag|oha;D#1B>UlMRnB-u&o)bd)nm zj+U}<;Jj1gmAu!-;Su#kKD0ZGIXQo-QiVC5zBATi9C|3+IukuHv4zwM{akX0pR-ow zwIpI^s~bC;4)*i}l3?1!nZ#f1Zwm(iJprTCl$B)1qr~m5slTM8b*VnIA!Kw~zrP5< z$$1GD#p8lgF@-4X`+c(uR?~Iy2~=1=J?PqqwW+vIB)@s39k*c#gI!qt1{JPkHQib9 zmXFB~n*%ml*{DBnHOF}6ls6)ljh+d{Hk8IB9*u-C2a`+3FscvNwR1R)+=r>ElaZ#2 zh}}o(hK9)(u2WPqbd~8e5>XS0PT6VYl|^0pSWWw8iM#@V#znSzzu@xyBFWRapJx72 zk$yQc*&Pu|&w>4UM%pC>|Hv*{Wn9XsI zzb4|3?mn?nsp&F}HYpzOQH^zb{Hg7;k>QG)_rEwR^G6QUs)w$OX^Us3(@RyY=M;M} z7aMBf!(tHaBRTZ=!?(3!c4yO@hcP zuMb znd)5vb47I(PbluYJYU3d#B$_m&h(kCHPEv*ds6eA%vH zZa;GBIqn`qWME6dWXl!y;@vv3Tf{dJS?z_SgG1J53g-8b^>{_RKCuk}@2*n~SH0g# znJ$uiov5t3tpyma)t@X&7_uem3S)J zcwIJ=gsEM6+UM@*+RaAdmJE%OQ--grlLf-(-pFVJ417j#D$MyBwKd{cJ?W@1OQK_2 zRJM&ovt=q9N(^r(3=fgny`C$Zmr|@CEO^<{9*g@*z9lNBqHSp@=-=#}y- znHymxoYXrbmFRvbuC9*pTHKt>2a!Cs z30zj0!T6IhNCmgX$$mgD&)Px~7TYxgKAm7JxRkSy%UkJ71~kW_LjqAp=E%;G*WZ^^ z@=H~?s&2lySHt!aY{9hP#kUol0*zqQvaF@k>E@_MN#tokZvn4OdRhO}uA#n8e%=-X zzx$^qWpes5Ilm1fGcK8ym%;Cn*|STj+yaZWfv(XtGmS-8`#bSkiPJS%9oIEswR&kn zS-nNE5u-99p!N;<^)d@1@=S9sk8P0sv5zI2|o;Sq*DA-Ib(B1u#RMDKF^=vd^d} z6I9d`SwNCuWg7{A`PDuDr76Frmc}w0Y)&u5vdr1u>I`9qXkR5vZT#(cQKIupVm9J3 zfYS{bAq08@cD3V4$%|}!OWa`GAv1wGSbk2Aq-|~g^Hx`d6fvg3ny|0pTRe>Q#;3xa z@N}H=uUyUTKf=1UNAV~O%2CS?#%9j8XbM{)@??2v&}pv(()5rRHubGs56)eVubzb@ zI8YF+1Ep#%rPF&|tS)}BenS^FYY+bIJO*i8|P%kvq}@wo>XiTOr3SI*zS3eyWR z=jB{q&VhGP-7vNM*T<($35M>^xb>hc%$V*J;q1?nHB&Xr0q+ovS(_Y844)y~546#4 zUxd<0s3^0Db%|bxshmL4NeF-Ngu@7`RWf;gldmdmhsX_X8FBhPny)%i;*%#NeRsL$ zR$3azPmaM6>fk=Cl+lN^COWSrukmr|I@H+}{+)+O2Ku|2R|lz?WjHpC9j$_968EaN zicr=YBS(}Vt;fxdTn}mL$)j=E^EzZnN&$`(rzuDgeUx$|b`URtk@OL8h#PH^sYesN z;6kes#RjNwm-`aumkk%k5@H?~5YPht{C(Ynq;*^3ze2fQ;-{2~!KiDwO*d+31_ zt?L8cSvlSr12@Im%J?K*c@6#K47mwpc=`p#h zP-DB`!VhMB%%|PltMKn{8@iBa@d{IMWvW;I46KX4p@->SGU00GSs3wX==9cT$7i}n z*4elZ!BA#%o3NhXBzu#K){}BH$qDt*!}wBn9b@ghqg1Z$ZxBqXQ?WpIB2`cEvo%od z>rVbDl6by*T8sV;+lU_DYfuA&G{V5+CY;O5 zD<=hq7T99%=F`JGc9yL@TDC1qAMmBMBZ>g_#IoI@RDwpbU)(=p1>D&CGRV8M%*x&q zxdRjt#nw)KG|6A_vMi{Tewjm(-_d*RX_QtHLNFXU5`^p-%_->{nzAmxxt$dgwdOum z>*>kn!r8rFBE$vx-Gcw@E-3x2#oA3d{E;m#j%D_HS`${fRbgYoJNdAXNY~Od!?Zxp z6sosAPaP=Mu-|2Qe0#np<*`z{^np#iS3n}#slQeFS$n+b8?=`9^WM^>g+{v+7ZU~A z<@mIvL>##V);H_t_frD*CEmES7G+a1GCk}$IrA5P)fz6?b?(NTZw$qeTQtC7&V8U< zr0`5GVud|%@yn~OqdJ+0uRxe37-m>QT%(Ywdwp;GMV&0&J~hJQnw0PxE!31tGnfRS zQD>pgVNsf9iHQ3%vd5V$pOARuuj-s}Z{RkOsSy+QUiHy9SgkyY#yR{CdMCtEZ$Fy(wC!LjyNBofXo{m)8TH zcYD_<=Mml~pJuK0^Q-hrZYU=$1 zR{scrR1$`-8Z*!D7{*W3kVitGel$&xbW3BB3|@`x#GYR~QzWR9(HIj59xe2M&8)x| ztwoAcAj0aU|4(9hULZ+5iWhIWY&3TkuvvIx##-XUuA>>P|7V4W8Hy9nAK72qD=ew& zhC^#3h~v?&Juh)oaM`K$*1;?wF&xI|dQ6?xo^F=Fl-=3(^*eFn@TU1{APGjyfL7Jz z-SysWWZ3PH^-b)#4C8&v{dEb*=QXf2fsvN3%g|%p1Av_V9-#2-Uv_qgix<9qpj*>F zO!~g5u=V+atOUN!vJl%5m;gq2kf+E4CoU()RQAR`)D(GM*$caC#tjlQAW0gm6+h2S zf)yti-Dj1|5`!E0xaxG8mGxc1fhf^#Cb-3SlyJ9^Mvv8)Z%>#g^^)B<> z%7PqKK~Jv02`Lt){z8dT(Zci0A`u+3!(vJ#(xE!0ocgi*E{mMMC`(auzfh?C#X+j~ z%_!G>5r8d)d7Lb_$I*a6AU`cIN{&LJyr@X1m5lb`Lwhi%(}6%?-;tO<>>>@}f5mF2 zd_aJ{;LUU&x2b1p!a?ZxAvp+%ra|r)-H!n46qYRS%QC5Oy-*CT`7T#%f{qn$#J2O5 zW!RQs$&$@BY(^e1FB^U^~acXRFL%KIXM(w3y28x#kK>OK*k?ZnjpVqwgw6 z5OA>H!QhM*Ex7(J_%#*^XE4nuqr!m z@-n-$Oc~KSk`NgSBg;QK5pX?7`r7WJLl?tup#1hx*eRF!k`A-3wzTGL+lPv~k4v77 zBzeV7ITgjd?M$@>fCXn+s5^e-FCuk9l2nAv;lqr{1?SKu%OECo)e@^NeW%(91_ zUTLX2)&3rvPjmb?{kraz>o8UK^r8V68cR6uSgCd#=E3Z@Qmpn@F!Zf)cq6Vp1Z!*mqpZ@ z5-0zEs$0_p(5DCVKYVaOOBQ*g)=o#hDwVejj~~XADwLIFy~3N0>8q&P*|F|Sk^QH6 zhI3YdJu=28P;>n}$7lSq>$$xeEtKTI^V+zUOa1J`<_wi8(8~H7$w->i<@Q+d`FmSV z0s|nR^x((Bdv8jQi}gl&@AKaF>w5H?V`{YCD!cRS>@Ho;d48gw-0jxh+nRHCV-x$}}-M(v?wU!v=a;=#8u+UTW>R=@mi@An91s);wCy|}0T;Ao~|08x`#I6ZPXC%g1MNuzIM`9}^zabDpCEERJF;M__-%vT}W8<#3#KElzv3SP-!7dD18ce$Y$_@_7+Xm^wJ$5G^kWcNc zLhV_FS%O2$%O1&g| zQ!~zU%iS|=({P(439$@spVzT@IzMQ7g*oCgTC@ecr+M3hSl|ixIy&QQqhCzXFfe@} zE7ddSkL6Z5HR|uXJxh1uYopKloon=?R>~hf<}u5DKrw+4YslL(da#v4g7B$_dlF-! zRSsgg1@cHr%mEp@rMFK@(bhKqW=hgV$-N5U9(2to(21PW^yh(ph-LyPF*tew_V`a| zf0I;Eh=dX%X9moWWEzvu>|o@n!} z;_H6EU-O3`eW)7e+^uozm+8Hj3hGdDU4WX0lULTHh7)iNNY&AT%JHjcr6AC6i~i@0 z+!!Vgc+fhqN#biI^k{AAb4c?xl2~R;d2Ncn1w_A)+Uz(qfXUrvGadAn4QQ5HdKQ0n zXF&B)pA!CPO~{m|a<~7{`tw?mju)0d`v@^F{vtO~w4BbvZHI&B z?!fT#W&hOC!6yw}l*`9AISI#_k2(j3`k$B0id)qw^m z@42-wL6}V^r#m+l-sjn2UN>PC^`0FyUSG`2Hd`8wr`HEk?uV$or~j?2j5i#aWAiBh z71y%`*v=anst1%(f+?Qw7XoUia8v@6`t1gGVBz@yaP$fhC~ z@P~mUd!q<8+MS4uMQ%VZ5vj6T#N+l^2oL-g%EC1-{f&NnsxRqxBS|z_lFqeJunW=Q zQy(GFMg%s^w;(;|h*6!O&;IS}X4g1=FIZ`3S$0nXAY9V%qDP(B$sSAR%PvHq1=nTf zy^C_dQiKXQIqJnA9Z^xg%zWQwUVrN>apMli7Yoj_yw4==rvt@ZfI0u1_k3NL#Ni6| z{dV93$TYUw_;6oFRlRM43EY4KlM?hxYB{M-4f46uS$j>*C6`kf-J8|R67RFW zYPV>qcjXeN!TX-`z>BExIL-l7XiG~GJZl$4h}-um_qFSHYxn2V?%XcG>q(v7d>3J) zUBM6u#pl>mXJ_^6^oO_cW3agWp-_ObG>fF@HP<;!m{Y1%#fh(Z;08b!Pd zc4y|j&t{3=Kh~VuRrbC=-=92tPbGFa5$JW^B5|y|KQ*QFk_ZhHEF*Jk&5DwaSb^f{ z)jky8;D_QXV;^bh?(Qwm!iefbDNQ<|Nlw0BCh8`AYr4Mw2su<&;!!TgS>w=+!cmTV z%yA_9!{T$2CyFqO?O#r&(ED-2vH7h2?}v`6 z>h&j6jb_Np=u~DLg{fINeDX45lH_RYV0LLT@l!yOR)_mbL*8_2b+GfoO}=F8EcaNI zbMpTq?$3a-y{bi;4C+Qi<-CsN4Jqr>CwYM=h46+%2So*y)$XEW(bE$pDXY8hGm|HV zbQHwDHPq;1=KP;7kH^Kk66YoR;+|{07z6G@Id_v4wL3;!c`DsCqN)!U*|qkcZGYG& zRXg(wU|+1%r7CafP}W0JW6JbfM~X|Zy}==dBk!r~p9nGw&5vf95T!Mn4o6B}0r(IR z>i|2)jv6|hpRRu@ z@@#4c{-;gk2Em0Zm)m4If#+hsh}_L6^PPt$B3&cz?tK2<7x6yqiTgu+a(^&gYotX+ zkxdcV-(P)ZFPo92PRRPYjMe{Y1dJ(}^3^R|@YlK*Pl!9*f85m&OuL-Fe|`Fle!5Tg zrNxiSGAXcGZ(;Qqux;%Kl)}LJxt8m>`aD3}!VLqHBS3x?k_*NT#!~W(J-i-3;hnbG zwO5<~+UR8MY?XarIB`o4RyB9lDc_X8zo6IDP4U<{9KUZQ@H`>ldiezk4hajvp%uSz z_dXcVJCny<@7O1KoX8P8o~ zbi5mVNY>Lo-FK4AI|Yu$Yy@OOF3&zLpW#9w2uwU5a8pjjP8k+Hptk)6kp!2LYN!q* zE5-MZ8J!O~ERP(nN$OXO?zNd!k24zK*oINMpJpplPo52l?785#_ZyKD{7*N^TkF`E zSosw3V{_^Ye7}VX@5KLDhj9P|S~9^v1Z{|lTTq1xsyC8B+iVcFe-umvZgdEtNIV+W zGEz@Kg{nfg&9z6~fLZNH&nf!%<4;3BYUy-nv(Ph`IvGV|x+W)>Q&|pj!+sfn(QGNA z-*RLm8BMrDpt7WE6v_l3j7A%_oHeCD{c(@GT?lRI@f`ht-kl5gF`Ldk?uzUiJ4IV1 z4y0TJD$Hr%PKPXChSd+E7=g-uHr}vheO|lKCQ(#-lP#!IW8t34eVs;pe}PZA6(Vu7 zR(5(?;C?`4ebXc`fdneN6=5tk-mO=Kz{u$t6TT;CEZ?3a;7u0z+agErQ`eYQ#m>*K zX4a-|2U%`mRITxQ{yeE#WVaS*7-_wZS_N7*cU_OsW3}(>h@MHs?=jMaq>Fw(`ME`Z z`V>>B4UD!^sgXTx$kSVoLCb(C!kL-R^CriFCB2iku6&3-ukc>T{TMq}Z92$hf;2tD z#{u|OrWam7g^@-!{rjTHH{|7Pe|{nwh3AsoMo`)SkQBAEVzmfHGd$-lk|EKGD+Qd@0V&-CbLEmg1bLZs~< zIayn!Yem6OFe8xGl`+q)dXx&BwEdc{C9!Cjo$6-vrFq~YFNg-rB*aDFt^llG>=u5R z2pYfJ1^xYk@oV`n%YK?jNn6g>WLsOV9Kig4bBSjC+0gg27}D#rr@+Y>AE&N#nd_S5eL zBz-iPi>fcqhnzzhAE(1it#k zG#p@BD1WO8{$5aqdC&-?f}&%lu0uL#fh=agxD`5hFEMbezQ}C{L#uv$N^KBCu{CgU z7n{}bRY?O(l*QW@g?4SpI}OZ9++F}qw|dBEATP-ef{0~ycmhpczttEgL6Eb}7-)qK zQ!IKaq%aReu(aYMV-o6lv2=xlRc!_-BTF~ZHpx6Kh9oLpbVxIyRNz2_s$UdOY{}-+ zbN`OIs;9yl&1!c_W>&&Qb32BL1+l21GJD!HEZWZ1$_54;o^p2fac3mqlI>n~{`z&R zZP7Lzl%`z`hSz$o?%A3o(wk+HFB)pUVROo^v&Zp^FsU3q!cHZi0jKwqH3-j?e*O+P z0$f5cut2m1$VyIQs^MqL7|J&2Z-aPF#I~-f5Wj>TNoh%GfpL6%uLLt-oNiTnG;$^F;l=nZPPcf6p17;-V~T^f>F`)NiWkJS#AfC zcJR}Vwifm+C{HE{vDg3sdWF`R3BeFNI;Q)lz0^juKAxSH63b6`^z$ETR3TApP1@&2 zhtiZC!VPp^CrFY^gg?Hki5DPuSfto>G$X-5MUB~g_+-U{YPID#40#Hnj3W(b%&PD4 zm_H6q?@h^QqFCaS>VYVNXzXCpHWb6p{9ek1Q+FMzqy2cKcLv<00O&Uiof4_m89Ui3C zE&z=V6URU5O?4jwvmFJ$xb!wXEZcHQiv**TUnBmdn)I){my9OzT=%ECFtY?{z9^uf z0pqlV`ex_w`ENA#a$j(=NB_{dRM6kkl(xYfekMghgGGx8LT{x#q^YG_@ddvLsxu43 z%sukp5bj^gI$XMMb?Mtkekhp5Phu?ZOlkio7s{;(SWmQmJeg5+epeWUR-~NokvqYR z((NP&m1IWS-WY*FiC>wcAbHh;wrnsM9Y#Z6=<&|0xhxc~ag(U1kaZxpwT`=cSSn}H zgzJRon|Z~tNTW?#cS zvLz17mF>KGgd1Q%29ahZ2WAs|3O&$yc{)QB^^BRsd7En~{hmY!A#juqF(9a>vSUUA zL3rX{c8Ss{ivCUo%u>`biu)oM5O`ZA^mLe{gHt@Zf| zG|bdAdyRFqtN?WR>-mpXM-qhH2QwhqxR>P=qnVd6661A?504YP!|amk!r~S;AOBF8 zU7}Gq%f{Wlq8;GI8tsPcMkaIJ8CY+rYn^3`7&=f#urunnk-F^!XU`cMIR`fO|5wlK4^_3 zphYTd4hWqOH@76m00ynCmMRP6J+$emQvlyja$_XVGgH?Q93dgaAV?%32%_aJ(dA<7WF_uk*z?|I($Ti>_V z^T)f^hqdmRW!-b1IcJ|;uD#E-=^>7Xll^kkRVRbfvdos8PS!(~D!Gc!$=l{q?-f)= ztLJh3z3{p0q~+2~C*}$jX$L)Sn>Oj7a@puZnR+vY7aS9yE5j1ZTnAQD)>Z6Z>&P7l z?yS$b`8KlvtT$;p=LIGH9ozwzXh*EAtp@hn(2?Atr;do0kd)0^i%R>rZC$h4=~W`%MF z_;j^n-0!XbeA6sP0wrrJqWfKgU1>%@nYAwql=i&D^Uh}q*?;Qq?}fF0ZC`KDeI3z; zmm*bi0?z3xlmRnOqk^@HDTR_Opj2JN$UKV zOe_HG%qOo2$#9I&_xb*SS&9fL?^vWb%wH%&%tnoXJQ3xd!kWXT%2Vj)Uj(~)IKlZPavLxaQUfD4?AUT`ymZ!#7@btOgn1GhFfTj9a<-3Z02X1;0%*0f|~F z-N+F1&Z(tJ*)Ll@_ExpvybNA@EC7o}xR$ZjMVa^Wv8KS*D|OoVzsH)err|~}ovU7- zw88wmD=`BgP`u#HI^uitdc2Ekz1RhT|3Wn<3-kFuM~A7eoc|yE1D6?qoX4bp{?FAz z-1iLTUG6{QcT(Vcptv0#*a|d$i+K_&sH1oYJIGL3Tu@Tze#%B(4;)6i_l5QjQ1J2? z!H!@baE)8<|Gp~s;~|>@(Dpd>^%l#G^wh^^_ZZh*R_jMyIPvhjA92%|Qz@Y z)wv6_Qu8Qql2)wCUosL%LVkHpb=r${(dJfAgB8r+tR{&$)~R`~tf#;5vln;uvMqQO zQM@2DQz3zq>5LUKxI_4v>y%$i1cVY^CD}0etMhPyY3WgxVG_A0*HJ6 zcL$5g|3p;E22}iZ{Tx>4c7Ey)JJ)bJ)BWdSozz4j*Ce<V}=;P`cp$RBZ^cj$l=8q3~VTR3$SU*<)PP(NT!7XqWAhHY;IG)%$Ga~^S5tajCHF}EpDf+-$_^;sKiua!wSg)A3_Z_6e-L?1u+Pu5SP z`A9<1dcDN$>~!TTU;Nx)bsQWJ=yf9hHf86%ELzjBbO-U}A;z@ByasS*E?}yMHI8QJ$JWUUntHcxdVUFbI zJ-B)V@E%tPKtJ&(5L8b=A%`$Onhg{pamxFY0zAe66;_VK(Ico43GJ+p)r+SNd%U(*E-k5(;bUv&1Qu+&RK%J3O1v@AIQ$F@%|) zXseXW4hlTE8s>Yatc(LPy78Mm$e6vcy$YNY3)l$KO9(je@LOpixHSwJc0la}DQf>Q zNU;EI@>k)DKK@FP(HBhd4I4&f%sj_)IXA-r2eLik+J$W5su9OsK{U>C=H z#{tDeph?t=2))wHsK^o7KeL}HoUaP+g#K7IZD&fUME_Ub`1Hba{wh6WCSlpPY(uy}IC3aM5c(s-S*pmg8vmEu>Dv%Z=w}TQ7XAwn ziyAjt5DTG!gpr$~^du2&gY4JH6rtWIYC7IQ`l0<7yC5 z_SE)*wPJItb8v>5{ zyB_&9VAc)N)5V}3x4WX=`Gy%_L<%uEtlRMk(q7lPSxhlH!QMEiez}DvRDlA+?lN~K zAdRy~p|P*ljkGRSXW%Cx32yONwd&w{_C@X8Z=p3Jpq-t-t}jwsOpWsg;ikRVmnouO zi8M+oet4eJjTJv+I`Qsz{ex>HDn>l8_PG%zY-YPF?RBCLJN-m1wHFd9RxK)hbA9e{ zHS1wuz-&$2dAjB3*E=HChQT5~Gk26GfLGGlTpm-b! zsdYs7RP{3_Us%-l0u_#l>Gx+_n)9xXEQ)E)mHYy0_#yU>-uyn|mg_5MNO=^vUr3O? z6C>t-B4xrVmcj+fG;+h4y}k>Okkmnr7ltP#(|bAA^J3Y#BdMQSk^FJrns8{R_&~ya zS3+!ATghrFu$3=j7;!aOuZ_; z@ryv|Op#c8$gaXag25)Bb4-F;Od1{uuzLeeHkz+DfUL@F>)^n^K-0EciO*g*tRX0) zlB`a>)Co^KNp@xIhc=(8OKyCl3Raoog2V6!(L&`msx>sjAhysksQ*K$s{?G*W0aolO= zyV>V+ba0Tq@SoDt)%Bdcm;w0Rr(#!}L(3g_I(vYs$hBfh(FJ#%mxR8dno7P!Z5DF4 z5pz{SBrglhh(czDe@IH?!8Mf%9KH_RzaV%LjQ_^NY?LX3Q$?kHt4);9Iw;W_CBDApgg+gMU$Ps3uwAj8-JPB4@+Ao#NR2L+(XK&wrlKRln;#=4DuO*__ zt2U&3kAVhan<@x)qrZk1A09a3=r4`k){+KP)$RcC&@eeI&O_CxXF^X^THnyNqKrnN zVuY{W64G-GN#+qhHUBSvRbol8a;i2zsx_nSWmUg+Tk?PL4GlNj5*c(e?6UT#d&)K)Q93Mv zINo2iU?KEFL?oQO1zB=zjE_9QlH&}X=*X{}{m%dTG=8OuqrE>Uc=tm$9bK;J!cq6y zhB#%1I)<8C$x&9Cd7*3Opde?aH0Q_~(7%u}Cn@=LI1-)0lghNPE>UDIo%> zMLs8tVC6_$!sRr4|lkTI7O-P za-T9$<^&x+F??+oX2|vJZMj$-VQ{kIx!%~1(LblYJ(qwpL|ejDzQNRM9ID_@rbHq0wH>uhFbprV%VlOAhj zivhEmr|=wg+Y0t_)z`HxJz9Q2pB*+rc8S4D4`*IDa-q~W#A91GijC?;?M{2$7=*1y zi!c>*LLlMx& zInUKltjw{ZE%Nt`UGelarP;Ce=wh*?rA!K)7t!pTuN}*CIXG&Wj(x$Eo*LE_-r*(e zoWE!&a)^nBQu?YZ>jqXGAA$io;sIpb2>sSiHfNUz-}&##RI>S>{o)@)n|u8x<~u)! z&dihvaEW_ea>tHr9Zv{Z60&Hc^{QRZ6k~^IA2P$7XmA6ovDU`F`;A!#zYdY;%UXCh zs+5egv9rhTj{HIvrb1A>ST+~R^C>psc<8>6ZF|sRR#PFI4acz>H@n+RIH~wcFnxPl zZiS|0owB<`jo;rbII9ABWDOT5(qwKj`cEJgn?p4LGqcRxg2$ObaBH<)-KJ|E2l3o; zNKMM7%G8R?M6)D2M-CxC(8+AryZ(iMo=|ygK^y@wyuON(Iq^NyJKJ9ZP))hU!=6R! z^p?!=jj>{=gD^d_R|V`7kje2Rm8bou-1NXD-dPr1Eorz-jcQ+_f;6!#C9|?t$_cJx zmfHN*U!TVlbzcy+2mX0)*e}sdANiCQ6Q2$-fIvoLec(A%;P{zwzD#wk*>#b=tSql> zFn>kS)L-yk*_cVVOCFMNLg8DJK%UGi{4FWIywdSKc-p)h59dq7R!41U za916JSnr{s`vNvS@%-Zj)Vk}(4`0# ziPO=n3XXdR==Vd_>RR?+ohrMF#%x@Slt0PYIuQj_<~kcsxEGWLuyqBV-{ywt=!WBt>q=jRL|XTU zJuW?o3Wvr(Fs8DpNmhODzS+Ra^x$% zpQu{Sw6Q@e5Oy%Ud5SPlE1e+Sgokj0MU|#1bYLibp zalLuOHOxTbpm&^lnA#kI^TH$^(LzB*MRosv?1~1p;`H@hf{%RsNhEBzX!m+4mY4RG zDR6bBZtZPQTD&%TU&Qv+RR!fI=Q9eo%pL5Rf}~J($DmPOD#k*Do&7C5Epc{YDD^d2n8>O>L|O$@RzeitytK-Rf}F*XgUmlX+T9 z1gg{C7O3PTRbvX-xUXUPnx*rHIxq&2eKr)P&T4Pi-F}CI7gU2-V&f-a5VgbBxv8nC zdFcX&ul5U&={@b=IFH}SnVt!H$FhjBOe8Fi8aN>-X&v&0@^Sp!C|SM`nbgt8D;Dg} zImBd~JDHA$Vu{9nG%_a>BU7OB@c$6ua`(or)$prExW&n;ve z0?)brh2wo} zMFa{_Gs1awu-u;_=IB4JbVm~$A2p#_YZo@KNF}SgrD9trgl#3g12B~$Z*^#Ts|DS_g-65YblDL=Q3R4F$e7}4E8*{?3N?`snK^LorlKV2UOFCL=y zM631L`91SlO+;;EaY-;e=VJMf3bj^tK=+z6ABUH`DH%t0yq=<>vQt4m=D%&nW?Knm zhG}>fL_^(hxV5xqTSa1=pFVyjix*tTsxx`y0;v-G42(q1^r`TEl+3#*8Hx z{QF4~=?sJj^bH>em*}AayaF2E!KuY-KbD^>^e}b+rmR#du5N#_jbKGXmrRH8p%pPE z%inU}<$iXRKr6-%J}uO;sf>25`xl)lakA#ych7Pl$FR{(RiOzH%29BHpBB|YZ8Ti% zdpCn9{)iy z1vnHfKoCmL)09AE0+(BG7dKGpYGkCXkjoZ$_N!Ta>1SuZUvJ#V>QUik01F@~JWA4^ zVx;}qj+Uv)HdVfVl-#nOHzFhPhB+cYppyr{7@o!}3a*M=ml;5Go%PrPf@TJb!j%-G zlq&AAV7X5|C4{_0lI@R#o?Ab~c~i@;<(b)OH~L@^10AK0%n`H+`s_x7qGyfl;cIa^ z`X&Nlml9ul>x!CW-}pJ@&zSY>m=e>mDhyj4%4Tb&^cGyE zSR43T7)A?i{g66J9!d8u@nsB-HaFSj$`T7DGG&|FcNYOU@v-n{^6xWR^3zpdp1e}L z%Mg{OL=yIqSrS2;5eKv?t$En=q+1pq+?@$|xU5l-lciq6F8JhcrdYn_UGG_#0$dYZ z$;F?qnRXGu))uRJS5Gy=S(52^(QK)A@}U?!+O^5wp9ajptzD*BKuP_mSd2F3vmeXz zE;^oT^NP+tx*6)lzbN!C`M!RqMn2toD4UFpGCf;l4Vn{h-K&b zqbs2cYYNV@%NKqAw`+c3A$|}j2B(jdTUb~iy7v(&-vh2Jfo7i^s5W|cBvWuqJMUe! zffc~<0{p)j9~AQ$j?wD1GOCTq2#;7kytA8xj1acL1i=`l2Cgv9@ZaxIdzJap_y)41lw52+v4J0MoXrh8je(CXb<%t`X zwFAtEb%hD6k#hS&8*M@F%#~KTgRPYaHgJ7OEn>T;ash^CC|a8#QWj6KO#dysS<28> z5NT9v78&j)$>duII+J2F9%)^Hnt+kOO zY`q=ktX~w_=qlC$*CX5JC5=+@XI^lCOu4p3jf(K_+EG($7JZPy&Vns5pVv`n`}YcA zqzL-iU3|dXN`BY0)DtRZXA{AhO-?3~+@Fkd1T!;=ed8%ENf)EmCpwgUD8JSRqz9xd?Rl4O*&{czdo z)vlj+^EWU$W5{YbfB-m*pr3lUDc+?9tON%~hNK5XLQ0C^ire@Axrf z^{fHJ24_u*kVsQ8tp8vf5*Fy9qds7$BFS=88lSlBudIz*SIW`-Eu z2y(H|^Nl|5%I~%p>wY`o1C(X?4*|i{0;lRUN$p))Fa))kx*KD^7o{IK9tR z8qmER4hgONeHUNv0XR}S`#|7K=kQ*!4N=kZeFMYc)AfjJo60XY4?!*ZWn>CeH`j3+ z3ky#b)R%hV(^QZg{EJ;&L+QMQ7+`#+{Iq=*AFg@C6g+Ij*++83>6rLe1Je?kGtQz6L%7Nry>v7OAc~r<+AYy~oUfDn$ z>|xH&HZHG@vgSJBx$oGN&f>sm5qhEAuKL=jO(ai>7QJbm5{l+Qxv1`n&yaPv z8`LyyZJcRMp=?SotUU-o3bbq*J1O$T@bgJqZT!NYYV zik7D9WC!hrjE=^jQG&)Wqk|r#BIYGZJBcLu=ZjZnHEd5tbtQhS?qmyCnPtDUt9IP``L1LhZxIs|lNOGTaZtuA~AnV_L zzK&B%dIxXK5u1xFb7N4TK8XU{ ze8RbBqq7;ftpHwxpZEun=Y$;%ZJ25X`vM)i8P z4uDa~5Q8v#Y^N36&(A!(x%8c$Mp9Xm$sF)UpH>{W^V?m=lW5Gw4T@Gd-gB!rbhph^ zqTMPhStN?qq{O!@>Y)K9q>hox0f||v@B$V#M;*|Qm18->$rvas6s$WOT5!d^ip!mN z`PBDP$;g%Fv<{PA!f0wIuu6n z_S;lKM@oJnVM=2L%0Gelyu+ZngKc6{r6g@8Gl4Z;asOcGF4Q&a&N-mt@4>{UVKo12 z`RM>t(`x+3_S-RFYW2*$m-Lr=b-&z9(cOirNzpaF6BfmVJ;{$qB zhaV}uJrU};_V&34GlXkdx)IiER81JJk$Kc%&CBZZ(; z$#`jq!Sk<`T0ISk#(;%D3xA51O-fpwX88` zHu+;yU9tAnz3%lR7H*TnhHKi$es;+;hw2T;tn0g5p~)GtBEhem$ro4^!OxyF9!-vD zo9EzZ%A;cQl~r1OrG$t0#_9j?2*CoVQD2D>6s% z;c7G~S@lb#*y<5}(2y^pniy1rsyW2_#uD_>k$?JBZri!m(7{7p(OcHpJM|5XnR{0v z<4ZZNj=a*^2r|TC?WdrTCHGo$gnkXSY-2V+1u?Y+ms*i|E1Qpj1BPc5JZOS9SQ#-| z$RJN6Os{~_Ueh8A>ZDsm5$%uTU95e^jcC2@1i`Y6HpCU?9uds|2E5NA_N>$6C1QlD z-k@ObW7Wnz_4m=#)I41s5}!Y~T3g%r)NTH(Pg9v#a+XXh>*bw>RQIyg8>Hl!SQnH zwpjQOS09}H{XTEj5wFS5U}ExDOwN497%0>8F@I*_L28SmfOVzK#2~BU z-9o!H{1jg7a@=t07vcK1<+a0`D*AQ379LZZ05~yXff!+e++KnA#jc@Bgjor!K}P8D z!U*q3%bFOq62Wy5@3hH$osCKx=Acfq_bI~e*%DZ*KWlChjYgJ#HV(bfDyX{g;kJHF zn@WB7;q@K(P(B%w=f~souHYrX(2fVoau4JY^8#72#nxzhD{XTJm*ydC1F zy(}4F$ocTG_;(uMd;nC!(4a9|Fv?G>lMvn9Z&p2FZ|$x1LxojhO9A3j)*G|#tcx8G zMn1({?cgG%Fk2OMjMnMg6@?=B@ts9u^cebBsI9tnBt2RWErPmX^2iyw=m9so&aEW) z9^z-KAjj7tCDM$th2DZADNYWF?||MAf4b9=y5gwyy;$#ElZ8FQ-1rAJ+GEZ zV^acUhRJt>BB5*rtO{(kKul@i5ITa^Rsbsy0&%|r8tzITZ-g4Q#aIMQ=8q?m*Z?lp zAL~B+%8r~PAzR$O<;ODFa@i37rwcpLDkVC;rQ#qy8v(Gx|Yi4}l>H5*HM zM_;aIxi@ltW{r!}f9Dgp^4y5l)m2{;98Uqx?(VYM->ay(4G~O~n${JgK2B0?jy*%P zFFNNu8>y64uMbuv%z_(&5wF}O0mJ9sEla4dB;JszW33E;wb|7Gc@=$Q6m%hb=)%g? zANneAl49PGL?ro&?dyg)?h8se;}_R()|frz#`gj#5p!$}hIqmJ7m06P2Unv4hd%7l z{3xsL1%asLE*Eiuda)Ojp8)R%6!-+^_8sUA-fa;X=o=7Q0zcH0w6_<|CqR`Z2=pFE zm~U>;-@c?)XEeFJW`?!gUjA=BzvS^Ty@pI7r6T zVF#>q_R?tS?1^8KM)r`j$cN87z0Hi`(GE%s!mbey!|?)rqIr5lo2E_<$mKVW52;Mp zBwe>Cq4UQOeyeh5z}cSDNP@kC?E}za`Zx~o)mcO>R;t7 zEy4e{NUaiQ0_vQ(dIn`)BrIL|G+%Eav+2=QN>QP`Z?qez-R4k((u!f>q$Ff( z!&fGat-H%TY;0|cMh#9XgaSJl1yyoC@yGm}*(=Se8JETv4-mcaUVx81d(gpQsq#pt z0OPhHW;a#{Q2o%MQlDS1h}Ad0Q&u73q8zKeXosSYmj%lXqbZn~sh~eT9%vU7aP=+^ z(u~nd zIXu$UVY4;c5}riS^(6Vp?_Px`UuFC!8k}2QqAHljN;VLZ)s{v6t0^U2&#@QXk9fPJ z#!psWT?mb(3GHh2W=AQ;IzmI#Cpj0IYu_OU(-ZdF1RfZWrMjZ!f@EVe8dRTa zZG^2Kez~Bp-hJo3BeWVX-mCAnnqXck$2*xWwKpE0e^5K?0cuHmtE#H1qR68s@g#ex zd&P)Hzv(kY+nke?+hkTYyRhBldUG&*=L)_a{W*e zIsrN4NMzD;Ar~lxtY*IEQj*Ej^1jK7C8Q|o2zX8KTx4-^ui0P1n-d@_$6j|6NP0^T zpXBaoD9OiNY-XQ0Ko*uCnX$+lKH~^JUFoxqS9IIn5t0p4QhV6*M!Rozh3v(heH&v= zSgHMZ==YoJ*cC}v2$R%Lv1hDBSGZBHVfD{_+I|M8FzpOEMt6Vkvh9c4BKU|{t}mBX zzlp~NDFq+VpB9zCT81a?o0ec}?*7Y=xuJMqR5uvQXSQ|_ihChilK7BHdw;GgT^m)| zYoRK!*Ttuo0GREl@Y>}w1U$|)`&EcNZ-C!0urv&ncH>#{C9tSRzAoLi=}QFCGT<_2 zk-;Fw6Je!F#VP3Cd0eyqRo2KWzvSuJxeGs75lm20NrvP46o8tD=+R?B1w+&4Kh;Be zFd-BHSI4H`L+*gKiBd$AQJ9&1{xTJ$?c}^a5Rzx9$Nvh(WBFwol!4Z7j#11lgf?pX z8WdS3^=R2H`nv#($G|?anJPfFS{)u8IAW_7n47QT1atsUYjpjzMi^Ix#A989H_HE8n8dUuAyBnf&9^YVJDUh;B~>B?cJ%FWU2 z@;WUWzklZ>HN|JIZ2}hME8{!7apz@gYt|evM|GapJ5+2+0f)!<;Uzb*?VTu*Wav@y zJ4T6rEi-(NHcxUWemG=X3`V8-yJTbp{N#Q7%qE=;d{;GV`o=65*=`>u_4gDso; zOPuzr5;X5UcbG1``gu<%ghFx3d$knlD)FjTQ7g8DBUQ#{{d7lcULMdrgQ=$nn*|HC zblA9C;Ww;vDpo&&lFM92P_&K9@UMCMCis6SHY(7}R{5SnOv1|&99HWs-oieNSu?%(&@QXdUzef6aTb|mL(7cucc zAgRwQpW?;b;!=z8@!rLIO^Yc!UOwW&HU8djL}iEG!?|c za>VV!6`j?;1Lkp?sIk@fxYVGDnno=z6={A@M{`oWv$?U7VXlDaJ-R;TB|qMVHOc1Y z{Mum9=Id`?erEyPtv>IjdUNmI-UFHDz7(f=EV=aL?3VkD=jjFcEcVzbqswLFRY6!8 zV0S2>PYfLS@j`;H58F*TpWf++Kmexx$D51dS>M_XRMaf5VR4(`+MhM2;4ksscE4{* z1SzQ*;S10tmGipFmVeRSVodmIlVg)_O0N9&OQcmviTUXECI}oeP8U|D$4g#Y>l^w= z88uH_$uM0mBeG|ACY$!My3+R-biB1Ki3p?5(0c?t*GK24{Ktut4= zP5#Z7o~$BlY(-|)gXCvPwW%tI5$nC9So5>fjArv|5k=I9c-lbRB5Uk~eD3+++XbF- z0NX~_9i8}}>8)&&i!&bY9QzppN--jYO~LyF9f^J9U*Q{~$X*`hi(+nYm-!z#nR0TQ#0WB?l{)Kq`S4mgekfloxyZU8y(@hIzgDNt zE`&HBgGZxD*2$`dVtzU))|^|j+az8v#!{8hMIwzpl3fIVBrn&F=&S8UmRg@XZfq8q zitN~q9?f0LhB-E$FUnkSU*A`DZbkwssoF|gcP-4Jk-&B9=rr|fd-HVdQzhe{r{~Dr zW#8$?yle`DCbespyr37?1+z7JQ0QcW8+RJSQrEM<=!#&?wJzXB|5PT;Xm;;Y`i`By zhI)a`?0AbYf8LZ`x}HV3YAzU~(<~6M8$Fkvy-nwkj;R{K1t!|8fA}t7)*Y_CxNY3r z*n9%VueZuy1YAuNr;Xpg#0`@02sll+J3AdDC1nWD*{1>DV*zUy*PZmLKJ7_p_8FuA z?Hu_3Cq-T~+T^oSBJItQoof)%6L9Ifh>hf4_>dhSzK^v(8Y`cTB}PUjivWk;$guYp zOi%pwh5${%AY3n#q9q|S?Kn~Nl{l}-Alo6Exq0^Fq}w!&dDDku3T#-_25Y|Fozr*h z?2@~2UDam^=7=O@@x8nZp-9o;-8>l$hoVJKd#@h%d0O3H$B*zA))oR1u1{** zzc|yNE?v%Ff4^{En!n5P+H7g+SpXTxDU?;g22=G4RRx@BKPSNZ?rZMdxq#?WrI8&) z&vGtYG+LfTm_J%SNL0VkvjC(3=OK-pV^mNdSF|FiMWkiSI zBV10=2Cw*>zDIPs5wMn<5(2T^Gqv0&PnxHX96rbz1TCM;YU`YcI9yM!7C_MsfHN5& z?VCB?va&8_pLPX~)URUTH(RXaMo9l{=uA()p8usMvM$=^AHP|B)c5-<7o6GdhFx{3Bz5_T5Q>cv+aM~5>hgS47dpAv79Q{Rn z*}C*q?N-li#NY=02w_nGD$7E7db2}in%DY84C{_k%bnI1>B`%b5`5P}ESpXb7b47o zJKeG;wIHy%)eea}(+f340H@^);P1Dc{Kj{UXbXzUbwtX;|g3WT#sf2D} z_k$@Q2o!(ZotFCB7}8T-QLzlLpo20lrWsxMUlIt*C+sZBYt^>E>74}i758NsudZ?^ z<`?%&xo31wSD2d>;G>x)v53wHa$>$UI`)cxk@2=Vyplf4t{pvE?C-oP6)^yZLG@Il zMaOWIw5OOjSE+aZ!_b|^`xLY_{u|%zynzgb@ok7VqlOyr`}DF^ zDDT}fPvhpPtz%*BLW8A=hepCF{jy;ghFNd_(_Yyyy@Q7(Gp7j@PeetR&yWOpIjpLF zM@I!EP5x*59bZjh1FKGB0Je7_eLdOmrn|dah&}Q^*g?16`6`(Pn8?I>Pp=errkR<# zp^@L-TJOa=O;dda8{>Q`~YmGVYFSmT>2TbYW~-E{ADv(v z6H8nCj_6s9Q~`&#M6K85OQ{hO_S03Xc0cCMolgKfVNFDA$)K2d%d+&vKX_eV7|wD)jl3Oz5()AUm`6B3~8yU zeS4R9o0`~^5}<-NN9_d7QBmPX!t88pGpoF1pI2VKFwUfSvga+U{^5h7v-1>LnxOY? zUr#BwOhCY35b?^j+Im*v-s{gPLb%p<{Q@??G)$#Caht!rt&^}~YL1rF7Xze}I!D{T-E z4fYFVywG4G7GN6pr?o_-sFjj#N4NX6TxTvb=^ulZV+Oe7{Z7^N^wJkgV66*M&-nvYkNy;q z_BGe!{$(#EZC$jrDUAMnYX+-fIl!e+bygI+RZF zXi;2y_w$-p|MyVRyviV;TDmqy*gfmbkn?Nf8jYFrV^QNjv3DDuY(Hr_Y|r-eeH7jW z{4?(Mg8&r_zFy(C(`|M*8Hb%E`YFakL~Qyz%O^IiXRAF1^GF>FHEob10=wOo~ z6*6PhC;C&f{kl@7`(N)|g*FR2k93lbRS3r`ihS%Dt*EF7IaoG|_~yO6*4lV++`l?U zCMA%0<1$F@XL&JI(A`}YjeHkyv^i@z6L5k(Jak2COMcuRh36`oLpJwwmBJtNsVXP{ zP-rVl>(j0XVj_~BJgF3^Z(*EIfubR=y|y!Xa%Lj3d)6bxS6%helRvFD3@0jmy}`j0 z%v+aEE1^3ZZs*@_ykfSlx_dBS6Ra{@gz6T`Yzt}3H<-3c$fZ$v4}c(GApuP@4H*wiz>k&+rG?EEX- zgsshHtT0Vk=j+$6J}29HKdrvXWuMKh#wR^+#Ev~N(Q{K2ank$ycd=Dq(?V~^dCG2 zq<#E;v1HnSh~o?`UPiyAP*=5UmUBJxB94#cOm`Q!>%@lTYBqKq8+qm(ZRQhIf zM9QbhVZ69~QUv8KHHoMIWhQbhzk<Tdr2pnZXKhiXR)eSVomlq~Ag3QFiC|>eF)~ z7M`XiKuV`7SVCiuSy@P#L=Ccs2wCVwZ5Z7Iz!^5$1*57m7pchQPUU1)!{+P0=#`r^ zJMX!+GoD^#UhbDK@w?9*CJK+T{VptK(-$839d~!B%ABOG>;X(g7L2i6p0&QtDi3C~xrUNXgB$V>69x!*iR86&W{{Uw%jHum)Gt9imOQ!{2*aK#!sZTg%S-Gu1Z> z515r84{N--Y%#@t=Z6Z&5w758$Kx=qLhTWo%Cn#P)pM0eLUP|9*@~t(=>4RSb^LR( zHZEg&W+$Ce>m^7WHFfp)VO2AKwQs4jZ^;Arpy8v%eQUMJR!Y9e={oC$o#rd|`M;$E z%>KU^d_bVH{6uDxu*k37&l(`XiO;+WS6Vy5A4)ZAcx$4yxAvF9Cp^rBfn4^&4buF~ z;X$(3E4d(}lEFk0qI+ZJ7io@*`K7TO5m80&ORdW-ju?dP{`^!s-nyBMZhO?h@hdbG zaris|IGCU|yC=P*28dQZ@d}3uEkb~+*Ma4=uIDMiPIJBE;G=phB)BHs=}Io+?hzmp zr{J&~={ec3lnkfbyAHJrk@O$$)Mxk2H+?V zDDxTD@~4*%zTNPepQRM5(_n&lD{aP&s+`?4M?;>5lH1SJnEZ(il6tBhH|e5%dd)YO z&iXY)J_K!4+qUK0pVa)vOz_^0VLYc;$!AvQQ}3*b7G6N;S0FnLjcDug<>s5?-am8g z2L@+p^}cJv1LU)?%HtcFnX-Q=Tz;Yl*9o0l^-JOL3qF;%`bAY#l9qwNmY==^rg~`P z28xD#Td!U;AR;2E^FC7ST$FTdUh;Zh=eiclp=wrmc#S`fx?|K_S}-nnkSu}AV? zfm}X>f|AnQ`)FIpvGs=*cY^tBdUjx7OP?x`teV||@09JT=c=SVPs-i9z@3$n*?cn? zunzcdb57pb6$Xt&FJInxA!Zo=#yzN6`mR%GaCUj-5^#C#gIO18pass6rHi<9u9hy4 z;=htx;3Xb-k}_zKK0Ew7NWV!}yh(?+dj)dCx%Cf&(rG%MFF#_a=Fw|taoJ@-xCb10 zm29UmV2Yi40~+f*X09+R>1=$ky1#WFvGrvSYiF%q*7>e|m|7+!@~;OuXD1ST34OFy zX@OB3TmZJYwA+5%%3k0nRh~pi4mNQ;!WejJgYiu2CsnfJzHdCtXV_~;si;{mAANOa z_2ZOiR*e1CiPLyvvuPJ!^W~7y5&&($j# z_*-V3&8lEOSw%`p+V6j#Tx$P!%eCLdaX_CwM_pvvvyXGr;wT-Shq8%%dH3#fSOTH~ zSixsr^PbH!dn^YQXCyMDb+SLVMPy< zY)8h>Me}-y^RczxlbSytsOR|?vejd|kI=nk_29{a=mH0CCOmn`1DqDr7RAZ?0JClL z0K#PJ1TAoDlkYp#nm(x2Kf6$UXUB1~peUPNZCg?v96VA|F%ta4>H7I+LMK{^V{2xg zA98V>+fdRG5Iqrz$6Z=UA2%SqPj_=9!41x5QfATDR~O>_C~UJUGoO&&VDKFVqS}kd z#u$x+srBcWUkycHRg?Il5Ta&1zf;2a5q}oHLG`YV;QEhmqfwx* z>%?In6`75>nY7b2@*1YyMZrwxW%zPkQpgDEq)xQ*)@TH?a)Mp$TNYQT7K?^eMDA3_ z`f|8vy2_W2Oqk{=L?@oGRm>H}pa0@QfVf~nuO?wY9yyCMK|S65e)rB@nFSb63el4? zlb^5;A-MML`}gnEz{7Knx9MJ<*5zwAVJEqp)jIoOY#uvTjrro2=Id8y9La(5jKH03 z7OmXA{))}XLT6%Pf;ihv85WdVa`jl<$1ZOFgm&@)=Y1`l?ZgUicURO$SU*bUoWy(&~NJbfE`$T zUpjEN(6hIJxHW67n5KcE^fuP6_LAqv2l6^Pu7>f)NWCb8^hJ1B;JbG#jP%dN&KT;S zl5nF2-BF6Tjl^xN#W8X(lh~O|vE3xsPP$nMpNl)_9z08IolNg#*A+l1LuCO1v9?s_ zxJo&^y@Xz{F>ha2;V<)?*^N00E{7M^A@%l@@YO!3zHN;pK&4P?z+PgkF5>!he4`2@ ztHf%K8G7o+I+oG{aa-Z@Cp-V5SvR=uAgJ zRaAI01mQUzQ;@k-%JfJIUm6)=B(lhkFK@irx8g+V=rYb?@Emo?3T*hI`J-@c zI+)SHDh6%6y1cBYtXde?n5WPja7N-!+z`dgEXgUulFo@GOudd}b^6jyza9UfMmD56 zgvpU?p953p>gyY9vzP(Pk)KbVCs3oVxt7&8tio}2ETY_0H@s%?)x1>m9sTr>@#vec zJ;)!*r>0?M4dN2xGElDTC~VXAVHa^$QALGFZTqot_%uuHogFHVRNXs)%)%t^g{S1$hQCT%E_SJ5K`WF zNJTG2sYuf@d~DD?Mua5acBnb|_vc4maY#^*G1O7ZNkj zTAxBvOr$(rS~`ALw@L>Z8WN!w;%FYFnyXmTLKdUHhaQ=jJ}uXr@DI%#YuHb2zI3=Qtr{T5L!z>O^=Be*d7guL$ud z{eJk1IC)o20>as}>~~}Qy^u4@3v*wKi@%j3i|i+CMarK6$kbpy!*dUVwLYg7BC9;bw{W)o^Ul2jDt!+1<~_r5*l73D5n zL{~SrCpU`Ql%^acdIeRawbI%=#2tkP%fIb?^G&1f)QcDMl>zJs_hH`;T2`|*Ii|mQ zK2*imXzty+yHLci;7*EJYwYcIHb#%N#`LHBhaPfsX4RTFIeP}Z!1cKFxIn_(C(EQk zdbKB5uOOR6cAAKq>;J~i=6`u)CYJ8F9W1qv^j1&Q`aw6TmK2ASP(4YfWR0}s1WM2P zqEe(#k6me zhuVIZuc|B!YBDrXPJ8-U=!oKp5qklrYX=~fTj*>wcYj^GnpZqm+x$k-(Q>O*6BeIr zyk6Z}>WlARe}z2SG{cZ|M)K((hCq?=oE~2osZQ4+%>Qm={~Vg#J!5qKJf}*;$Ht>< z8s{So1QxGH9t#eV3WWa4GQzH4NCKtDE4I6DOlXc&hrWAzd16MJHu)?kD+mJl?(4P< z(u*yEDvWGv!_B8ftmfV213&O=)$kL#&N4)60BT(jV;~T&N|m-ccS%?{j)Em_nKl-7 zHou91)9IW^Z!AT$<84O_D+AQiRP!HI12 zyt>H3S7f_9iN=!!x>6pcZ&H5A+3@JR7@2IvIFXLXibX5?kDee4OnY-cB<;SnLuJWx zc;L&IFIb|BQ)iGjoLs5NZM@~QuR821%aMoL^o^K?IKv9U!fA>dZW_c8!A7R?1+rIOO zdLl-`W@_i0|wE6U0oCp%=~JZ7f^Q#6taje9o~!}t(gQ%5*%wUp~1Ib*FN+YXN6 z)0-#y9_JdYqCMjD*&4rj;4I&VOPlm1#GnRTHh-5gD!UqOc4lqy=oMu?b6y)g3~dsW z*_EuFbj4O{vp6}$u=IJUJ}e0*;b3k|D1>f_Eyei3`%-i+565Sh&S>SkHq7=4ASd9s z8D}t~8HNdZ`R?lUM^2oOuvj^cL@M&N=X6>!^3;<3mBv;CTlgD}?CsZo^jt0oGSp`U zPWdZ_)#rOkKa5u$k>LiI3@y|ApzPYM6E|*h+rJusVrw5C5RMs~zjUeGn(!httJj+4 z^_!z~);iw~g45A&ouQq^%5vE$Ul)*iF0{P&*Ny}GX@BOWayu<$hbiAp=vH~sf6{P# z|0OO@{I0Tc-%m3;T!5acS~HdSC1?RuxbT#SX)VP^$_Xr23-#`m6P8z2T>T?* zEt{$2H@<~BPpvOol-Q+!x{S5hh5A~35tP35dMMvTKD`3_O(cXMtI_t%%>3+>?hq`BWoQJKBTsjL zO;e$|9;z9kd`L5aQ$vG8f>@R4mB|S~ludW${KoQBT*Zg>spV4rlY?pJZQ6PL0#)vQ zp-!fT3;v)|Ry+3x|ClC{bC?EGFiy1iqP@Mn`x3DvG2__a;9y(2F6Tuvpqb!QiY~IO zlGO_JEu4$RU`A$mPX`ADR}uuod#wTaa%Ys;(7;d!Yc6gvYTxi(Ibre6x@eCrzZVl2 zS&t1y`a|?zWE@oeCtEX9;drDI$#^we9rl#-tM#>6=jSzVPn?Z3oWDI79geq@ak^(S zA$wj-Ozhmb;zcoOX^L{e#Ir+tiIxlxTrLa0ex~G^ttd&t zn!k}Nz7NJLtLwfwEA0kl0^J1V8HpSVywGO32hsg%4J4n#P+LO{6Kb5ADH=I1;(Tb7 z73(ZWBP~%8zVv5Rb8|nhH4wlB!A!q?(07odtV>kXR5&6nvJGq9DgfiCdEYs0JkW`= z>}jh^HpL4rrrzCDf9P5{`eW_HK`XF)9Yz8v%_e%`IBs^-mf zr_`TlcAqm2`uil$=<{-8KfhTh4LsVD*E~R>13EYfWnbN{t_&!sF>{}E7g#@ zvF6=RS9;d<-3LCeoaY$WfaSsH;=6LlZ$M8*h#6^zp3my6E6H-grZ^Y==edq0*VlfG zMDWu+(0##BYGsGp5`5jf$nG;;KlHPHXQ89lh3%buj#Zq;FG@-t7G zSp4Tl^%}?poHzYIsLiI+xcJ31Rc3z^0rH@ShX*80r|iZEQU+^bNRQr3C2-9P*eL_@ zHtLaaNQh-as??Bf4%+XIenW?vVTv@c$}_7@mfhK&X&1l8*JwP`8VQ*HMAm`o) zU$jKPNfG->Hxi3;tCmsc!N@vpugAI19)f++&au1>W|Q4B`)UpMJ)9Q(Dqq<9PRT__ z<}?DgR$oT`m7?(?LBX2&2E~ETXf%3*Kc#h&=0+=cFCo5{)IJm0&vA4MkM3WddgqFl z8M@}d5oS1#*j!U{b#?76$U%k+Idm+_!OzIJ_UtCLLdb)Uz=2W2umJ|pL^@)=eN-wa#gcXkKyI{cOcKz zw{C&=f}n2H2)azK&A(?pkJnM1H9{7a4m|#8TJ`WnC4@fYk5ZEiwX@i=WtTA$dR{8&}Ft~y}!Tz17>s&Db(#@ zx}}z1U<3Ggv*A)Gsx0V{T0%SbjPcWMT(_!aU$~3fBWiNb!nS7Ihz{qdMDTlSR0Um1 zpYe#P6(Qml$O|gPlqY{mdlaL zI(5x%WopL`Y9nghcJSi=qSg3ZZjf)g zY)-7)>zPGXR0xWVGc+y~A>(!tPy74uMN2B3l z0$u5#BynWC^qqA1F~HTlG^nu@G0_dKekjDx<0#TaDe);J-Yofk%xpy}U2^29+*I;p zVH$3ge{7I{VXalN?0osfG3o1t!Y#)~!z%e_Z9EOPHnCFr%WG8gh$_a3#azAv;xv$i zx8kFgc;I#Gn0PL;d3SYTq{cX$Ji&#$U0LSL)}E?`e_Be;oT#=GiL5hjFpR9@yw%t6 zl$FX(!ri<+4Dmwe(kRrOXHSc&JeP=;eSYLL9o8Qf@&-q2 zTcTe2=>$$(6d#_7XkT_8tfB@ojzc%FH$j66=LZHZLB=SLzNyus@#)h`0u9oh)8EV6RSWDDIBq?8^0F=z$bF1zLV0yoDjY&# zLf4{FF7C*)iYlf3bW`l@V17bRFxnb?;TVq%2ft7yFDCG7K+)Wx&v0wN`khGs2uy4v zfifGZE{hd?=`1(q<)474F`4xixZqtKTvt!EOntgZ-QfvDI45LzkhzZCI-~;}Xjk)> zy#wQ|77w7b&4(<48e{YJ)s-JF!WxD(8tR{sn4Va&#}NoI2}UbZYBcEWP*c2JP_SB* zvdN8?g!X&8X~Z4i+mD~55h5fCwJ(s2KGS`rqhSj!=K@<03}`H)d1Cs(!+JZ-9mYq@ zs)h;%Tw6n~f?Q7B{UrIwe)L7DtZ8e>r*=X7GD98sK7~wanCDK3Un&nrI-GAsW7WNx z&Wm#>DAx=XjJPjQIrj^l@}jRB@_Cr2kQuc~I16LFrg-=6Pwg~x<&`T}py#NefmDxE zQc(D0)M-#<@$z>}x~iI5uk~v0Wc7&c`o>0|izayP48B9@QZr5&SE275Qjr>(p!TR# zD$uN)JJ(+?c?*^YLJ{=yk!ohGP{mLl7tI1&TrZ?Z3KrAT^?@xX9};Bk>3VizXTUcp zT1=0Pi-U0R!uj*Fg((nl0v6(fUw7ZTj*gC&mKI!-x~{ISt*vdo?Ff7h`n0U94E7b# zMe}lY-o1a-p!0%(mO-Pt{Xd6}{N2j*ue*@^p~VvJDtG_>bQp5hV{=|!IvD1SR77qJ zx!OK!(_5&uSfPK#jMzsCV53|CGoVRA7Zrt;JP4^7>B?L`tMwaQ#V|wSKl`wHO5Lij zuorFJ@Fi6@-yo%}uE%l+d=e!={U589!P`#;Iac}ko@_iws_NT}`{B;D> z(Z@FzWyPHM@I!+>K7JGlYyPbvo{VFDgsMFo%INQV*H?lUb)|#w7F9m{B z{q9)8SqAwXn!Yq%Oa9H@DofdK3aP|YM-PYz}-sLk5 zA(gEujW+4s0xt&dDg^QAz7+`n@a-EMf)dyGxWIm`DmVeJ%6tGp)8xNzwkC&^l*kHV zFLGVd;gLaC8fPhljT zUE+h(Q&{4^f0v;w%`qa@zF&KkV?S|5D^wjC1P2#JQUI+Z#FOgPo^n(c@;`wHs%=f1 zI$8nHm9_-}qXoLk_%QvO$33@YAIn?#?43p9b#!H8v@_G}S36Da^l&mPv}XjlYD-tk zf`J&<{fK7w~R&+4w7yn-0-2ju1R~$R;emFSc1FN*_0t<3Fpy)*NH@QeY zJqXqc^;^GJ8CLj5ZY;Nx!HMb0wQBuVN(s8?g6HPA1yr`(? zN8W@WXJV^4?FUtWw4qy=*7;a9VXsF(1HQ38&CRna^USehQ&UqG*TxU3z#Qe}IluY! zL?rTnx;pmw>C+?cRypyMfdC&kwXZEtP06g7WKUhP*WfVxHAqutkccr=~lz zAS3l)m~~$n4aN8$O`)p$Ii?-SZD~5>m6`5LqZ*AgZOAa&?PFwHs-=!b+ME(HWnyMl zp9{GyFVFw@=U3O|Nt61hDpUJ>-R1-pezhMT^f4IioXG-*sRKnnaf2;MYS9u75SI7x z(Ml!10|0;a>4H33?1jOfuQx-u)$b`OQD$)@A4YZ=yV2VEXi2yDA1La&N$r>GIL8@S z&Ix;Cw6Jwg9@re^Xo(!_-U4{vbu`@MqJryWQbM7s$x6}sXk=rYJgF-u)Bxb3HIzW$%t5T}GHWZh_h3M-~-2&kvTcp3EGu%?GNK&P*e`9%KyZ zKJHStwFNwMLj%8^1+mD(ZFw>Tjh+qPq^uIzQJeF|brFZph|LXG1+)5Db?2tZ2eK(e z2u(C6G!%J|rToDcs&O|pfF~*SIb8E)x+i}T0DYO5Bps}E42tR!BjW*O*U=ToO~=Q_ z=hzTs>}()0lBJ!N-_z}VkVPNwDr{E+3k+^j)X;mJ6KNKZZp}ZG+*YR7>m^Cfr=*8+ zipaCL$e5_ThfjTAlZGU4oO~d>e#TR1*8Br4?IyF_qNY{h+M9{ufH8CV9Aa&P6@>X# z3KzH}BO?>$c7U!J&l$^^6( ziS7JgIbg8Dg)KB}Z5KcWr2Qowryu7%e>zqlJysW~nPc9XVOR-=B}GIjToC-i;Y)FiwP z0YLs1QgW%H);(**yBKSM!*H@CX<%TW)R%E<(2u`R@6>}~GXU~}&u&r|A?&_3S6o~S zTgnMC=&)GfED=IxK_vmZi za*WbKPP09NCiU6ot&=!X(J@)ti`sBO=BpaheRy1xHh>Et^p2KvTG92%by*yR5Tm!y z#jfI2!2V;L&s=Oj+%W_<8iOS8c_qTLkJ_F5p%e-9ti1C!YgI*CAOyg@8*uV^tZMu{ zS$ZUrIMa>owiKy%r=vUPH|_fvGA5D}J%XWY)}QwuR{#wTyQ&RF{bHyh;H)H@g#CD@ zNz7S+XLm;%`B|}ijE4?EmmxGnZmuta5I0CkB+B_RfV6LSG5zqQY(O0*i)pa<)h=G0 zT+6o9&Q&6c-)n6o3_013A+3j^fD`d^WO+$R2~FY;Fmp$V+x^hWf-NX4X*YTwkS;4E zBKwIGKeF4Zg1KN^eMxFuV+2rZ&j-uieaOnN@wClM`b8unRwzxZ*2(~qB&*cyL^#Y+CS;O5n1j<|~qo`Ha(yx8& zx;V$Q(t+RO!=v-vuvC+vi)QyJ@A7WjBH$plq&-~R9e)XslckAfE){ekth`S8g_?yN zi}o^<{JuT=_ThBXy}Z0Y=wY*1_T)Vb9oe;y;XYcb(>#YyJNqvEJQTW~$-G;4oUoVR z@TwW9Hl3HMr#Ou_#VeA!u^89^;7J8C#maggAi6IqIXgRxq&UBBhO46DrTO?~Lu+a> zqiPInrwO;-9R&y~5^LT2`HI>Z2Kf|?G|jAx&xlVx#Gg3=1V+)~E_<46ar{K{N?}O|fOAkTw%E`i^$w3!|VcsVaVmx72tEs7lXdFCzm{}Hd zOz2h3`sSl7ED4RKQiM#{zs}UnbtH=70Ab3Ty@z46y1G5U>cgjB*zsKdBAhvtbc=4de2UA32hN+M>cHFc*rjDO&$wKhf05Pg;*si4@@-y8F7)%1Wq#5_VP7 zdbBni$_4F66zCr6pKo%A&AC!srXIKz!LTn(1nC}Q+G*QdH{TYbu)o?sqQtI4g(%CQ z;_=|p(h{uNGKsjgJ{H3wD`eJm5lRMiqQo=8=ATWfA}ei9``<}_`*!eVe5<*rP2VPL z0b^^Wd-qh69(5jCzM_+xN+4|Y;oStaSEf6`Z&zxdCqVoacKM+JOdw6OOf7?EU!H15 z7P-5;J9T2=$#B!#?6A`%dl+2_0xHPRw} zJrP4u-Ir17NvfB4cz7!PT|K*h&%cxtp+HS|5X4^U#UOOVn*I6u>m?j%85;{sx^@Jb z@|4VEHHfaGd*WAql#Q+E`9KNS8DaAlQ`msAZ@)uxP7JHzJKC^85Mg9ehIW*VWjFE+ zR*=)i;;q1gY}1m`Qkr1}mq5S3K1m^%QzCj6AQj!&W@>?K(gNv0_JVkQniXk?K@bfx zT9VZ*8qm_9Iu=1E3!X6T$(v8t!?B?@Uo<=eZA&}Q07~hSNv+QW4zmh88z}L<6PE$E zje(f}m&9qh?c%HT0x}MBI{OGx@`b~1a``p5Ia`*78Fwp zC>Br&F5?IR0o62(25?5^`hUs3h4<7`A78H-zYktRC-1fxh+V!s9v`M}D|Y6Bd;pJn zatpZCt~FC>m1SjbQSy(SztkCLb*4V5m|=Lh_kf^3lR??rBM?e;gZ%^%=k|L0_U%cS zJ<#=^DG>lrbdZP!4~7Vj1`Z0%)`B+ukPi+=XO4xY-w9r+;4~d98e}s1gZ#o5NvHI3 z1DC}*QIiI=UWP$AJp9sZFPVW{+Xyx$#Na1y(AoHN;&sXMfYo_-QP+KrB zvtUEiQ#4%0kqOPbP|Jgf!xU{Kch972V)}l6x|Z~1 z?DT238me^f9eH_T)#LqN-xj=RvI8X#K7#Y&G?_#o+BtdaD9eCUD6COt zy9to0+M|D3&6mOLfX8vf>GghN16w%WctO!{MJ0gAVEFeZR^)sZaxIi{kmFt`k2s3D zDT!eR|F}-DOyb}oIeouq9x)&4rv_fwx=wH~9 z;4fOYBz{U5DP=_H!bWf-;logpdyXGlB4b3u9u*#*qE4pg6H|`&Au3@ zFvIH5D6_J8C{xQD@E+{)jwjc^3#znx5UUg^0=w?sC1}uqO84zsw}yfAe9y~bxZ(RZ zpct+mrV|Oje~^VQ3e5I+W878EbSZG9ge}^f`PgG_Z&4`#)gj?m+b~ElV%sl^Aim#USvZuU`ktURHL?H*O6f2#<4F z`@s2Yzh%#eeg5)g__%CYxg-Od0{6-r8;x{rfV52YYOap4Gf*~5zk%3wTjqwOwDlDZ zgGvVmOByf7MNZBW&hc=g8;)?JO}l#6X=n3_;ReD*WZn}TE>dic^3B(EIY~RqdQrnF z0l%SrZ_>Zch}Fq{@!EF^@vLckhT?M9vc9|7qY7L|wdNjmS(GA94bo zIcu7s@^pVR({2V;OjZjc$Ki2C*|(J&%gfgzJ!(hc1+?%`1}b^{00`A` zKZT2ntJ~6PJ)9;Kv$u-&`Pk#!h}d@MpGIk~aEX!nIvco@(Wf?R?FbYqv`A~FgJ(LD z)@aBzY}>1D(WY16UT&P_vC%oU$ilNA1PC*`fk9C)-Oq#o<5o~;TGhmy(&lVj=j_ln zC>k0~czzQaq~5;Q##vK-4)7CutnOQ;GMyI^DidVYh`n{kaPMDX#gW?r+G1qd#_*Ph zr$PkAqXRR|gj7dSCJmhe>~?u_a3F!BL^J?vwiesj+f#Kpz1AkDdZ8dDo_^Wu0T&USv=_Ubci&uv;*2r%Z!M3hXKVVT+!=TM`LJ_S&6YT@{ zVN!NJ=atZ=Kf=Ux&g(%+YvA&f8%}!n?lm{IvT}emanW-CW2mLB?x>N}j3Nu~muqI# zuSkH*qJD*L^S;!Hjz<1{dv`yZfNLerFf?@5%F+YTtk)m2obWKHRJ;oaD9Zox@gvjg z?H_zM$aM0amydpt?WHRd6_NR28&_h1mJjnFst@)ey99oV_{FC@_7e`qx z)(Ogo-@pGTT3(TqeS7Ed>1Tb}7ZrBw*umYph1?l(FC?DUhd#%bbW%5XY&fa8m)#w> zmm}_TrnGX`*y3`n;5ZUKuannpmnvi2IlZo(^4Cad&DzZ6Jsl}zeOxQQz(f`DnaZYY zbB_rVYFr07IF-VStjLmRE!of+26?bRt$U z{#Z=|V3>2XS7Cstwb>QWOfl&e>qo3l`ityOimw1hL0SU|i? zO--o?nkl(q&Fssb1yHwa&zVU;tU$u>=(lew45#v(e@Qi@DjchDG9}hq0=wrIt`Dcx zVsEu!hO$iRK>)y`1(4$@>B)*cswVd%;W&vezke@e^62n{_61568VyCFwfGr4JiCV9 z0oV{)CL=z7?!oJ{V?}KK02=bGqgD53UbP>x=-MgJ}dxy`-$N=^S?%dCI z3Bq$%oX_b)dx1mk@^^cQXrNM~r2zk1qy9hm>1y$7dAjMKj~zU}nw0+Fb(2{bhaR~{ zbVgUyGcC)Lp^>G%0P}RrTAJuV+GV3oo2;G-LgIWG&n6ZCqqv%Hf=*V=Ul1j)HaJ`5 ze*EssKK+oc+~_)LU@)yam(hcoZ?2v)OBs!H+dfT5oS9p+$yVc z)`qxVb#oWmZS;yt4Pb?l1@7y3r`+CURY(D@gb10v*aG~m_|p2A6mJ9LSA3xus(uEoV`W&J=`xbyL(2%=&D{0E5HdQ-E;*!+ZymshuK z)>-l0K2QSXxb>2Nx}yl?L7o}~g{pP!wmJZqc`?1(v4fTO=FL}`JY1Zd%+G1dQH;Of zTl@su?F}@8)uxS!t+MpKrJpcHz0(yZwvcbzx&OMaOJQpAMFnyGN9PWx&@Q_%A1wFe zkGTU7B-BJV4%!0oxU0v_7@w)6TaGpQ7@CsIa~ftkxpOD*kX+BmVC9{k-w(x!_I6QZ zjcW2m2~N(qOClo5X&Qj^{7+BvfFrXIqPXXC5AM_qT#yF{R&1;JV*bxmZuQhB*G?8u zW^df70WD_n2%|9<0>47H8gRzC$5`l;8)D&4Xj=D@gdSkh7~N6NiK4{%Xz8pc?`u>Q z0MB66lU@F`y*){93h)P{PMppB>-@i0jr4ng}$ees=Y|vCuuk|SpE5(^K5v6t0O6tJ>%{V|W=w`uT<+w{OUJBfh(zYwxK;dojnM>_XRb5awI$N1YfD09-v*QJ^;~zR2=J zEA8MIUY{<+XyCaR8)`5K0!J1fsnZV&mPn{gfb?_?aVhiY75?n^+tv<8KDcSDx!k*L!sfAXHUOxcYH{N@^#1BAjM7BSLBbd&c2q+LxNNZLR^cdcrUM z8YnKdn@O;;lT^!#Xe2&?hJHg%Rv*~tY4X713ms>fei~Pygi^>1A@O@=v{UURR zEl&^V2-CbrlcD!`-Lp@fR{_fIu{qJAfcbn?Q-p9()t0&m>G_!mVF7LP_H+k|tAV@w zYwX`wMC~et1rv9o##WM7XF9%l>^T1YqL9!n`WH|H`w$9k$5YwbHodnQ;Qo0m!`F%~ z^T~KFS(;AH=6V=X+8-wK(@4#jo7t9nnt7|w;mHQHAiwyPeBJy^2!bj@(`=Sq zvPHRe)1+yg%QxlXOz&#l@9RTZLVd)7*1Q1%F13z!T ztbEP;H0|gnk`=9Nf{_V~?`}rAzW7oz5+R9t8p0i-fq5Lt9uN4*X%^?nP7c&rNf|h+ z3|k}ZF`CNhe>zQ@>SNNzh@gHp8o1JI?B(l=LE^X(_x8YzXOD=Vc`D|(SvG%+OQb0O zhgooi>{~C$x_VeSy1EwVWB6gq@`GlFsDWSm8rEdhTJ|k8GVy3*P{+gu`k3 zsr8@bgK8&>-fk^6(uw=x%dSJuEU)V<^kUn-Pg4$^(U8gI0t*NSmhXr ziP~^N7?0j3qOcJ^y5g?>o@kNd^ z+E5^Q@9(dJ^&C?VQ*c^WD zHBjA}LMmFlsj3=elIfmrtC1o+Ppo>q3NbqL;f+OU2|^@Hl!6n>tb4NKyLaiy%Dz5W zB=XmTb6kViOF2S<0^y%4XG1i?2I>$FL%B##id98z=bM<59XX0)I(%~zV@TZscbBH>%Esz&0+jM@TGQ6ck=uXmQs_aGsW2bsK)=Lsepw=6BEr|ej!5p(!oUr7Mn zJyNRwJ7Mu%7J{GF^}xs~A*E0wEqu{E0sUPF8PmDs5gYsIKQf4Fni844DGt&6 zQ2DVW+EVtYrt_s7fEWFJ|G&5XKVSHXlbgG`Tlm~Q(A%^Y7|_d?m7YKUiM7c4_2e4x z{J=YOTfb-sJP>Fag7Ck}GyXePx>x_nN2B#5lNe0**D4767n8nDL!R|4sfb9Lzj^qS zC>WY$-mp(bF2%=G)Pa$1uv~GE?)HS%np#^-fWVTQ8wkT{SMsmFsLocAp==KLx-y)a zdkhU8Hu<$}*gon&Wz65}`MAKpn8{&&Fv#fYDN%E0NO!GkJ`j=V$)5^VK@fjhJG|Lg z{v0Zf8Yp>PZX;yXwDxD9$Gbpw2VD)R8adT*c($tc*O?hFYTD%85J5`X zK>RZ}Y$lwnIYq>(+&n>pFdWy+KYYjrPpL;%2B<^|C=({--VBi0xD|GkxEV9HxFO4`r-i>N*4LjYuS5LK5rk@9~ z$07b0l@_#04s9=04+jKD+7d?MHNWDf5GVyLpXI@DMky2QP{r0*`KT30g*y zHXkEw0u~kS-#;o_8+AQLr=3ssZH50S{X%<}p;2B4*!!<6gea&bYo9yRn$R5$(5Y@+R8{$;yM;;WH;gF}hYxVy?^U4g9OU?9jkV^{x(HWo1~v&e^++qY3bJzrW*(#7sI~b7P?+hux!?bxjiYwSftU!W z=;IurK`=wsdo%g&t9$XM)|>n=_ai2U46R!;%fFS%%TT8MT_&cIqF@U;TH;5C6a3t@mfBF>X7# zdGaGG6;|*K*wg*1Zv9eBVM?B91gTTpzy?ZmI)%AaQ z@6y5>KA9q|8j*0F#}dX z7sozxYc(|wrfq;4dwHH~tav*K+E`$85P=|UVz?v|>Z24Cv-qM!<>{|)jyzwMT*$ACTA)(B3ug{oCT^E=d(O>;p=`_w*o#`jSmJCKe3_2lv)Vs3X%K0z;% z?9GJqz)RZUT8*dR^jmNNq=O-81&LL2^9ZhMw<^ny2mRwog3)nthCnw9rG?f+HG-KR znT1zt(h+y?Yi3W1Zqi`v9y`H1_bE$@4RD_2uAQ6G!RQV4$<_o2E95T3tyn!*mx zaXH&orsN4E59sGe-3d<6@kw;Yp@5OE=tPa$o0#8QT?>OTsCp~ZSnvL!if7r*lO!TW zCH(DMJh@0K!`wdxLxK+66@zVE@sOi)W~E#EY1!}Vz#7)PN?z?-R)tg2_`)_JvXt{- zAd;8ISTz+EfYkss+z33h(Ii4#V~@na!>6w=j5o3eux-pTq-?uTd$1ntK5c2*APtbu z8kHAzGHq&^7~_vZ88*7dLqR%^;7}v0gMcwS=m5o}*^{v07gt z^gOO2I>b%dY9S{W-tFYEVD>3-d(>W_lBW84@{DI9S%c*RHsRFQs24ifvV+jI5edha zLJa3P(tPn{vEEFSeNvwq zJBswWg|CL~=U|)clyd~_p-X`L3N*GK;b%CzS$4xFgr+@?^I| zyKQ<`8U78g{@+m@LIB$8OWz^>_QQvzzx9&Y>o;%kdZmZ=zWKGCl%!i@q%mN?2>+cM z{&zmXWzAP`b}wP;SN+X5F=mYgy^}dMqfScq29F)pXOm6`mI5W6J_M``bhBphK#Bz% z#nE(WHVG#xSO#Dl9qL^rOUop}7s27?D-NRV>{zAXNbeu`jjl1Idutqd7gRUY@Y*?> z=?Ya|?}7tk#Kaw{Z((aF5Lv1WG9&)U#VAw-ege7&_9JIb<`&^8pM2Sutbtp< zqp2CJdX6WaEERktUJ)8S00OP@#I(6UJ+dv$i-|_ekU3bvtHVA+p$8%cz-GQVQ~&ht z{rgf5li4?I#>pLDp+`7}v{_uReEQ(`?X2K6AZiL47=Ym4lOlXR*8avz)QSb{NVJqo z^;c1#8L)wBp?0nr!Mv`(9m7l$F+N-RLu2(wCRt13ymgrJS%{>%(k#NDxyq`qAnk^^ zTKZ(qI(TS^xeTf}pUX;&o$Om|AJG54;yc|&JNUaBig#8W*_Vil+m4L)8J~;ri~Bi# zD0{{$CIra7g)kWszD85I*nKkS#js`fAT*k~F#dv>`w^{}a)xE41z}gwVy%Y*raExc z=9@;Kqf$!T^hOcO_NMTF0PUkUk5{60zU>{wMPns^9fJrhVEK%i^*v@}TfDs&_1jHt zWp!n&|w6EG@-&Re$%1a!pR9 z<@xJa^UgtVRbxC?$_vG_4t#$GxBvRPW?kqdx))cKS@^U$l+kgP`^YH?sIR0)OETXu z(%RgZvT0AheUM}CVNcGu468`8CXZeel6 zP`U3c*aJAv1gh1K_}Xb6;PWiVA_3#?GqZ4LQv^uqx?14*gl6%zaE@I2?o>?^vFLh8 zvH73fs*beK=q(%`<+Y*Rq+5^H;!<1(hV1ZUdySL}gyY~NePEZApgFU^WPxzpTmvSO zNMWnenx&cl20OeNM>6Y3;?%|gfgF^jfD83?k`#UCTLufLp+4@T2c1yKG( zpImd6Q4c;Gt#SG!6YzaOsv)V_q%FOBZca`bQVpXhXWOh`2I3?m@Z-qAP0Y3NW zKj~O#6Jck!3!G&fRudMNaGA6ZIprXM_)}^y?_yPvU7JM}ck=9eeLGIR=uwewXAg$v zE1WxxJ#rH04{}M)7-Wn*UJRX}3uXHvoX6;gqy&{1tvqM5JGkx{Hc4v8sn!%}Py@5u zbN>R|wWk2o3;PQY)!oE{Om21~@NgLRl`f=(haBqR-ise?jhw4dl{Y;V`3e^{g|p3)le z<@50~$1i$8=j}UaO53~51fF6TKmX~x%gB_fJGQpf zUJ7bZT$ISR;qnr5q82uq*5>W$hPvc4{Wb6(j2ptSj;#yRe~+hUuMaS23tK)QxaK6U zYb%LxYrIVbmYVRaF|nw6=+T1poor9QE|Eq)nztv1y9@x7=>SLaqjX(h(GaoNHVXRb zJ4+#G0SU^mtSZ?|=RD0FVm^|22~&p#B3q$1H;H47`WNI2zo=|~f9~U75yGoupWyPB zC8H~|r7#>wAImMAZM?jKX$Q!e`wYAqtE^kg%0Wy*0i|4|mADkO0p5%fU>I zqwK^&_3m0;0Etq^DQ8)OGhV#F$xxT@j54|z_AmYsv0#c;Hk^oO|4e45X=i`AX8nOc zK)!0DDZ=Ag;=A|ymR?`$9BXcBf*NKLBy1`3V@QYqp96M&j~$Z#%Z30PJOUw%2PKDQ ze@|p}4!bb@f4q*uwtwxPJT4BQrFw2ZVt6&Mil9PIcFBV12_VFJMBn)R#RZPCg^-ls z;9%H^H;;d*qP6tjnhNFJ+{|P2wFTW0~YLJKyjC>3S{l&#q0%V$N5~va*U(hkPQNUo*ujO=(zZV~@&C z_hpIN;drI-;O|Ve;~E569~J;oMp;P-^nUSH2aY5rKDmMFkC_nEvB8Y;;kkCxU;6k5 z_j2j{>vP+?T zj4XS}v*EG?V_X_MR;*_AoR1l%lI4CVhr9RYXgb+H1vg&>aErlrKOb^U zrHMD)1zASkL+@jI92fn$q8H+hDNDrvGy;TZ-Bu}SAL6I-t&)LTvBPApGYH4pFYQF< z#Q@ywyGnxu%CJ0M%DpZkM@4FV$)=v%`gQMi>FQe0l9Xc8VH7@m?@|d7jc{oyV?=DM zp~-B{t^-+I(at4IyRo0tlw~G92di*P^*MxzPo0XK#c%!6=R5=q5GSQ931$mVB?(7O~R%XDhQXxp_;t@wP5`Jav|6%RD zgQD8Lbx{mkFc1U<6akSeAP6WKw81<4THZxdb;<{@|2YqZyW|85LDScw0eZ96OjA~l@z$b+P);Bntr|`k)V>08; z_#^IPt{RQq)N=b@Q&K7$bq0VqSOo9i0zSOJD1iSq1k9a>ieOi{V^- zRSdicg1%@U54}MmVTrJs^20y)twQsZ>ic7I#>f+%3mCUE?>X(ay!66*{dNkK9_s6p z#eSaAt=a!^d|b+r;?i)Brt*chb2ZyU^5t^;i=wWsq$KoAsL!*f+X2wB%DS~xz%Pt7 zVXHVKn1CI}N$s|3q;+&MW{>vDn$nTVe2+8T22)iN^f#O%kd`;_)4dYv-s`S*1S66szWI95!; zK9j(G5wU9f2;mEjgeO~qL^qNQA`8D8F@=q+yTeKbSE?4*=I3ouO-@&KMs;shSN~a) z6Z$!0;!$d&S2n)ZUnlEZwc9tm{B>a@DXFUU*SS#XtsM;wxrNREmI03p<^W0P28xJF z*l!PGF+rTVb3U4Se>E#(e~kf0gW^e7zJT_2-OTh_u5_rC*F0lcyk&y%{*fHOy7?RJ z9?B}_m~^CiJ$ixU(sMe14&_+B6aqS@JEEN-Cp%WG)x&Gic_>ulV3Qa!D$C%j@cpkBt}pxov2VtL~- zX7YmT_lb&MgXIP2k%7XXtKB6OY>;bQ!f!`9*y63^Q0*#gys!xg*Hi)j-u-nA>{a{g z$`qPsO3hn7cBaaVYovF*I)6VjHqIcC)F`JxSk3Yav(e7jpWc(}y{F4ytY7kM6^dPG z-_i;PP5u?@flC`wB1AzeINli^t+y)`Jxb_3rB<%h+W>;<$(Q=i3syFfS{XVk6lx?uIj+NN)L_!%jL+<}Euq<)onoGmxk0T)kzQwB-#G{Tl^Q%APOw^zB7 z_I@XbkxNI8QF~V7s>z8ZzuTQ^h-!+8yRV0RohKAYKC5!=G6z#>s&rD5l%`Y2!^qfm+Pkyw-;0!6ogF`azOOt_3JNIr^7bY}q<*oy zY{JqW{dPdTzqC?MHOgb~Y^Sp^(RgRERgCCSZ~uZuOH;FYrCC|(PtD>Q-#cM$e&mj| zQJFZuWYdk=^PphQ@+JAqlr!bkI10(ne>+LqbFas`W*qD&Gdb=?0YWo#=q9XyGo7m{+E5LX|ZwdwPtG#0mEneh037M#$GYs7h8r| zFmjI(v5SjgHW(TjZYTZ_Y-#xkZ%P8wDkm!y&J!c-@pNJphNA1U-kPGAB)!_B&3(bw zzL7r%882B!l`-cjQ2Xr_cwZs(P*2X4n#B%zDwf5M*wtk(yl*Ka%j?9Sq!%Qnm6!X| zti56dp_qI@zUYf{5My8&_h~e8sv(R8&2{Hh?fuM)z4-U8rqId3lSMeL;@>P5vQrqA zzW=Dy+;g=5YS~%Z|LKc}FcpqKW(0YWk)6^$jH?{D`uDn4$p4D!->VKzj=upfvj5jb z`M|$A_f(MKmF-a9efIawrw+1isi{>HM)3^~3zOwGP3u2dBfn>4V?l3pLd~A~N1>ZE zN85Jq5-XsT3<}~%Y47h`-fB1foqX%BsScuvxs8-v`GwWF|B>It`yHnX>?*o?w1e|6 z#!w0gSr+lf69b5=O|SkY>60miP!fh27FH?#pTAFgPa5;9ATd-j)2SPELcfJ({}vA@ zMJ7Xs>Z0N^C4WsrZPPbKx}z_yDpfPTleP$b=i2;=QsD1f3Z>XTBbSQ|RlfZ1TMIos zlOxwf(RS{?MnUzZUg&p6oe+QJM%S$X7SGv4uVuZ(8^{we_WK6%Bsii_dO!j72IoO7 zv(fREJMnA3vDZ@7_Lz12lLn}6G92rg$zbUqqT{yyKB?rtlkBa;zx8$g=4&D zlLJD4i9SSKO3=QmhOb<2tEZB(AFK3eWOg5cnx5f+2$7gyPyYwt#oSo7-!;w3CR-*Ipy{`LqFe1u#KHDaH?{Jx>sY5=??)Ho;F=faQzv{ z>ptd@k+*Oln4O$F(ZJ+{Xk`gpWS8TXHx#8oJZry(bb!Bh?b;F7z&Jv;@(W81mGvzCX;S#me^UJK({9HIZ&>5iw&n5;eYpvvV!W5+vp45ZHrls( zPcw3Ueje(LealAOn8Mbe*4?RcwIL@DtESU?h_@hV&wDaaEuFZYM5fc};~Y zhe)qA!EQp62;;s>rTr(EpuBH}GLEZF0vDe|+Q%_#P)kgO!D^SLW@t}*LoRG~BfH|@ zYf|I8ZBV)htC1lG%Ls++dQd>A#83bNI_lD`4j;Q?edF3Z1^AId^h~#``Bdc#^L17## z@w4VxH=o}x$dP|cB5q_bZR#wPc%zyi>AhWg{TCFj;eNlOu7<7jzS)9X&IFMc?@8#Z zAZ!WZscqGscu>zud5+k*~fT(9lHDVL<1wPGCCBgv>e{T z`)pU#MHl-M712D_H~zRAZ9{z}OIXUWK&U5LEGjB$fr%e=A7A>~28Bq8Pa4iKk9&+i zuBRL)5x1cvY#y{(fZTp^!76Oq9gsrrCbj@W_fc07t6w`yPQCIh^Ib>R2<9Dr+K-6)&O`u6C{mlXjjR#T5C@|` z!;;aU^2%poK`T@cZq26GEjh92394|=#;>EOFv*U0dTfx%1yLZ@ofI4~3t*<_fAAPI zgfZM04n265kF}abn_2EpVf%~Rw&x` zud2C$Ot3W6@)kKrMYo_q5)iHQPO$pf>JJ9eoQCh^O6;S~6u!K#f8w3n%6I;}ld7Ke z-d}C>B@BX%nA(MJ>xCeb8S+bcLP@P!$IXLvC$##K4gArV>_cfUrb8i?4(f`DBhKxdev6o4N9e)^-6xzFzzWf~vxsxR@Vflmmd?+) zHES7Dji*B?tk#IAHy4c*R>^O34~Z7Mkny59DmM0Pk_{B~w|^LDP+J4$p>X6En28lA z<>y~4n4=yS3d&2tE*7=GQ>0KTje_1{(~zMk{l0l0<~m-|^)ed={W}@>Wyqu{a@zWh z&Qc6UEy99PX<|+aU+MhoYU90o5Z)Vu`6p+2W;%}EmYMlz4xKbM;b}h}7AAmkh&FQb z*zDDX@^6tRe-uzn)=6k9#)j*|_Ull`>vo0!9Vz5iztbUR5Gs-B1@v9OO6||<*2K!9 zJ$20eiUqOCA&mjrt%{K?4^~WbcBaF+ay*A;qsEUofD|kWnhB zh3Dm1!7L40MjO#XZ3!&t5auA?_rR{^V2f{qg#>a`48_4PmMysLc;+ol{9C^;Xaq3g zflEMeg?sgS!ZuWc!l!cUzeL0qWrV?Vtb5hfrG-+FOn}S>2dKpTI>+I60gMFnkUIVn zj_gL>7aBtNg+Ym~*M9_a+ylpqYT{1vE&@?uShf@EvDy7>shPUx=5I7NL%XN9 zrX(|P46jfnK>#J)HHXyFU+}hTi&P#`HFxNex70)q^ZW6zUxgyOzK>EG?Zp{cTVg!$`MzgKC&T{Rf8AVu!v51 ze@D`0*r5zg`(Ss6|1+3)htojUq3>N?UC`O4Zn6k`W`-MaR zPr}%a6SGsr5{PNV60$x-0v^hpNfEcMlut()NXoo+9vd0CO~ETDh#z=hY(#RG<`)(%9OJf&?F|S=mAHHLDe!@2>5mqtgph?gbm%1ZKDzk1PNf{pFcE zng}a@Bqeq7DjS^k2X-lAGBJXN_N~GRYJeJR`k3^?vl1xt_K*-3fmduf@VwRFaPxg3cie z0TtQf?R=vpxu(vn?!7Y|gvJ9Xpicwsb5M>;9De-VZd-@$-H#B7nU5fX!W1t-5t^IU zqK3}tr9%d3LF z#os9lnwnIYetQ{h?8zJ5f^_8BKG=e6gE6oC>S`YVm{rGf9(#>KP*aomd;x4ed$@}I zJ(O@vB!^AvOya+DDtgV>ZP1cg;P^iW(q4isNkV3IA9Rh+9sQP(x)=FaGOYsRYYCusXCHErI?zp#D75jy$*Ia9}Dt@c*5K%86==%El_rE+ieZt@B z*T=#M*{pkk`}lj3tIy)E_O%xHJreZt=BzzzFI*v$b`5&qz z;N#3gWwzdH$m)H4JON#UP8wkI!J7oIgJ_<{$yH{z0W;HA;b#>+R=$zbz5ULc5nnX% zR#9`&>S^ZcWMgz>WaZv`4zou)Uw&l~2rK=VHxPhdva_+9eFpA>mJI6MqUysSzfK^v z06vVl$Uwf|C|m+Xp|2dF|HyeZ6vbT-GLMIeGhdr(#^%+4%3$X7hUWo#7|3kbA!+m^ z6}QJrX!mbIMktHV&V;`}WEY$%a`kftAJf5CY?t~jsEjIj(|iC0=iq06!~l;>uo71S z;CnctGYDyiH2`vd!K)P+Hs{%<5bdCpa5$xths^?-$iU7)&^EM#a0lcW>}YkOEH~!% z=TvTw>v00!yQ8f})1bP1_PUd_)SxfTF0{5m@Q<$HlOHoC zv9M9{ye~47D#>12vPpucPB7M*PQiWvJu&?`&e8VKqmN=viP_mm2ZAHi(vJb3x|Isw zVEaQN)Q}9Ijgjd2Q4mhVQ-EW~#sLd6OZ`fa+6KJhrA&*MT!9A#ru9+z%X3%+?T&y! z#pbogh*(1#87BHMG5_sl1k)h88QO2-S7|-zrr<8y(6+=b7BlKLTm`cXtqAt6^&1v< zIv&Vr`nIu91%MLV0EeX+vjksheG;$CiWa4K2_^eAgf+3fC$6OiV$&M2qN)*TIr-Jt z0{E4I-_0P))#IwZj0-LnJ;E*fpeAzh9(4R1IR*K_>c*+(%y<^#N?% z-mW2`;Vg#5ufP8}NxxAfr}fPyBVop43@{A=rW86rEC7nf!bT&k=k7!M0-u#vtkQ(V zVgQ?KU{M8BwLsZ2fy#8`IfJxOS_bdus(FK<&+ar6mE&>;iEb0O2>yyK)R@8W*Lt)y z{Oc!R5taY|T88>NZA{KvwdeiR)!793*+U@qkF&Z$4Q>F@Oh|kE=;{XuLOqPlV}0`d zH1rp6F>;4SJc+m}n48G1kn&F0jL;CAFb>n-T^@QyQDU*69$I9q-sM4bCrDX7wABKy zncWZ8O5svC0uwsqsU7U>78s8kX=}3w{*edAd=&2!O|!PvA_$#0ut|f5&2Q8Ag$&*i zTX@nK^|DnUL(a@e?aU9bvvb&mJU;2AYFlK(DqMO34(468aR4pgn+y*liuFO&n2AzMR7-~jOkTuY-1fjNBnyTN z^OSk~wUcV)|3Efi$iCB1_2A!- zKyA|tKeJkgtrE4rpVjL%zHF7>$E*1N&0Z#Bl-Mh#l9yv;Sco8J(*Ku5QW#vJ@_~ol z`!80B=e}>+>3ES({~@g|JP0tW{Ws+CJp9Ol&?(TL$jybG2K{E*et5I)?~~ZcX1oYf=CCKtY-XX6cfjP=h$mUqT~b<_R*c*`-*K@XPaD7O#mttIzfe>Z#uScqs2X|Lyg^R2Ui7?|;MB{eKy}|NDRGD7O&y04n9I)Coqz zsxOw6K(t}3Aad7iu?F-(&F_~d5x)-1ryqx@#{>EUnc-&%nbnE1*Ou&p1=~K zzs%IO3w)0EF1RA#H~@~eQ+C}p(RJcr^n8VW7m8((jg5;#2sFgIM+gd$cIinEV?pW? z2d{cLOoHQF@Hb);*Nzu4*X%+6k)15{yPfq9+67L5dG}f>)irYM1t^i^F%wg#R4)W@ zPg*s?Q@V27YwHm}bAShkW2;I5!AC?p0yoANm|%#4yaRjt{l(&TtYHq+_+|KV;Vxx4WBi6tUEhA)4pPsc5s8M83Krh2?f<}V+lQuE$ zp*g1GN*q0#k834rU;}}3LICbB2@E253aq3ZWx>!Tocfef4T0%gZK(ar8k}F)g#LfM z&c~6KgQGV;0uY6D(0m5v^$fruzpzxC>x96+Bs1kVlHW3bpguW`Z{U=q*WBj>VS%PY2jyqa26 zvANk>f1J(d<7en(haIC{FzshZ`nOhfeU$?;5h9=9I7>pYbMu|A2 z4J6QpaNq)X5QsXKq#B$}2W%qrK@Gtg0KFl7gb1WkK_~BZZ}wcHimkkgvG~k)_bzY3w|n5EkDi|W z-{f*Ue0;+NTR`Wo!?w{@6a4!7C4zY@EiDl=2;y;l1a;kmoAI^gj^XUQyRST&(8T5U zTf0KL9+OAdLd4b}axBEmG)Ms2kiiI;9K_TipY4`l6~cgcO!zZMj&}7XB3vgpXIH8= zwCaUC&nem1%z+js0snTz0JDgU&9)5eC$ykKx>%W%JUY#00oXz3%rtNs5D5`e4nAKB zru?kix-=f_kHmDOc3$NgG|J^hdLUJuGC#`1brucuR5f_F5VJ*4;Z1Vpo;g`i4=)3W zCV2~1!ZFZO&((gnH{a(P04`J$Y)j*Zu zBc}+91Wg`nhp)|mu%3QF&XSpv!|umH`xd-!>`E6PA}OE! z9dHMpLjNq#MkAS9!1Q^8D{?0G?TIpX1k^t|Irk@_qZLnly|)#vzLGWSKVjI9-s$1W zoE1=OKmH2m!C?6Kn?Hl`4v_1BQ>d7#va{!5(gg$m0JaCpGs8e~^t~1qdP0IXs1cDy zS<|%G31%{}*#fvv=3phM(d4|mD{-yRe9RSAg#?U?!!Z5P2{4-nfaaj#fD__@lB%9* zMsj48mVVIkgasf@am&r28QI$lvS-27*U{BQus6(7aN}g$atA>lMo5DG) z5XKt;n?gYm5fKPF0tJPiPltW=h9)I@{&U%@xa#nuHwR#sNAv5UDD*J)gCOn42;rQ+1c^2Dd-?OmM0Kl(?PI;ki`J)>jA*2si+bNWJn||=}wV6K&Ia_ zBf)J5mI~y(#>lz#np4&0{p4+mbQkEOhro3XRe&ef6S}V5LwI9dyx$$)ZK>XXPk`ho1lfRJB?LB zBeST;NWc1twY9a76bn6BaBUI42EoD*v>Np$6<%X>Jc|ahdkp3Y1~12!CRcjG&G7= z-#1GGf7jaDdXomGb0Vkk!7ovD)Ek@Fph5&X7F6Y6zU2`geZmHOE-_Wzd)84!Ij#Z+B^5a!Gm$9aY?YD9V)h6^Xkg-CZe3v;&$;X& zEcxqv=v9X@3aB^JJd^+R542Z4Dd;c|_%$o|?_@?SU=dchnLCMJwF?>)Bm4_UAr4$( zcvjHWVGLzG#mi4%alpA6?eWRDj^?(HQkN@gCJ|-u^Pv)KN0Wx)&bPU$6SuYFKBXwf zx9(^!dyK(2pfl?1A&Ei`RY3J#kR>ABj<{W39zD<;5@~{`t?fKu7Jvp_i-Zf5q|OX= zBTOVXYy1xiegkKw2izrCXFxK@7ny;+1XtIk(bV@Q%`)H!II-aP%ey;hrm$PXU6cWV z%VSo(!Z#J29s=2fyuu0QhA|Kl!z*IY=T`F!;!7bBK8%C4a17#9F(=%gbpR`w)!`e9D49%|&>7DxTGTzA1aR9S zDkFCnhmb-n06f5TD2ryj3~?DpN5?DPil^TyAm|J6Y@h*d5x>*db};xuo&A?sLV=B0 z2fC_L)B!x~ph85hLH%DowR%PoV*VOvh&4nsTX=ZvZdY$ty}*_TbgpF0vN+rqpu4xh z1<*sITNIqunqxPGfVx8BTmJcxNPrIE6l3l3Aa@49k`xRK(5=3>MQSuS;n8a8k^+@E zVqKsm%8*vg2Rx@J+t=H`Lkh`&h3*ht0_9!>&Ygnh2<$(~`o|Zz&j;t}70&=G$Ry+G zqKro4y~e%3{YNYu_@rH`+^#x4y1t@!N9B>&-W`GH*_Wgr|lN*5#0P)Gn z%8~%SUMv@_8UaE0O@uuHfXwGY@fLQG!(6LWvNoF7gaov*vv+!ydse8 zBFWF<3~7#{RM6-H#2=i{y67KMd7{eP>{dYFF;o-4+w~#4{DX1Qfl1+!7s=1($FhG5 z?)DvVt?IpOh3|qM&`7J(n>3Zs2OIhfTzSYLlzF$)Gf!hw4r5c21+w2naz@<+q6A%iE2hJfYBQcPzz~IO|F|t|;?O*?KgI-H` zAN-rmVDujki|oI-7z*-Zdt@Z_?1_Dolb(nu_scsF9HDA`3ryte83i&jgIB$Cu0N=f zrE;_=e^8lCJs6G@bfXy66RE6!B|Ga-*Fo%I9cCcf+?Ai}@FooIZYQ0Kb^fLVaoQ|+ zO{&&%N7!)SKKnju`-e=Z$Y^gUVcn#rBFSI6cq3S^EM9!3UHO!X3X{ZU%xQ?sqw_Zx!v85(?wO^&PW1bz!96<@*8cA4tE~ zzJ+}Lv{oWlChPrHm(bf0PbG=4jX*R^Py=t1EHMi3cMJ3jL@TLU=RUO~arN8K)fu?z zBgVq%8vsg1fI9fBk>mr$SsF0h5uX#YGWRy37Mk;oJ=St`>Ai6y>L#9MKcv$8S?`YM7;_a`EfQ^hbjr@7w_5un39- zNw|P%c%2Lqwbf_fbnuPZ=P{_K_Bb&-#ORnEIh8r?ZG8Ia(dREXF6>*A^dUfPPrFT5 zAcuc^cX|YmzwTCX=(v5Vq2>MQG;M3&ZPEcj@QBt%6|r}uF_F|bXjF)rB;~`y08&1WHIicbvL*vPnqhbDDxlk#F@N< zI^(UeyNND*+u%%QW&^QQK{c&PExl3;d@w*IdmfDw^?sA$$ZJ3}AK}s$K_wxM=pph8 zkAV_7D|Lm@G;{XbRtJtm#k;pUyg1(eRM;_*hE_B91x2>AHU8;HhI-3*_JPaFovTGM zyYsg4??XnM4dq&zV_DcZzO)=UeGhj>%6o_*M+=XB`z?@Nq5Zf#@fGfp6wB?S;WQ6@ z_L6k0Z)0SAONrU3_mf15C+1VDu$;_ZR>tM|TPf)K2{NfZ#N-)nLHg@%zHiEBlQwlP zVCLgvN*VBR=%sHVOHXZruRn1k*5k+*QOVqhD;OfR%tLXQ4YH8sWF~_|kR%C6`Itun z%s{#?%+E`1dH*+2Wb^F+e*Kc!K)yV2n2amnlmTCyYwiVp3v_=+8WksF)gB=3v()eB zN08V7Tmx;Usd_ zeZ1frB%8Q`?}vnCz+YPcsvimRPiXSp|MK{PHAEL?2a16BVz^G#rsxz5F4wD!YR~DF zIj_LFJ!+SR!Ct3Px%C-*sQ$_*UUokx7)vHou@PLgEM?yF62JV6K#kYmuy5SzvZLOc zsB58MlNuq`!ylLJ_bWZ7w-a|pzcYozxJ(wxd0P3PG5+>l_{^Lu8VU5`T=&bJH-j>w zE0sQJBzk!*eRFPnlXCajPexM9q)D69!ETJ)vpqs|HO4~Svp#{eYkXiV zHL+}qJ1=O_b1GBBNvE;ZwtDh{^&nxCHNtByNRbEvJCeRcV)33I?K zoqI{NnZhE?lM5Muq!6#w5y^c*Xz-+iMQekv%y{?M=69bi7kUH4HTe1 zb;~1hYb44BUcYsMYab*NumYc<_>SK70z^U;N`y`o0tP4jk#Ir{jX$VV|gF6-qIV(uQND^tEn~ehk>nj0rk(Rr_ zTVsK|K)^P9f@7i|7#q7o!7CuJG@y0=euG52h42xNQiH4aHu^5&^pdXSkCb~VHAXFr zqbDVkZR; z)~9L=i?(h2!U!SNQ%T$;f0{0Pu2Aod{V#T7o&!#fGK*rRF*8ZB?w0r?!zHyZB~q=c zOSigxjMn=uJCU2@-f2%<8#hTep!Y$qcbl+QZSM^GRy49XH;xcOrPc!0(vC`R`oCW_A#N0hqVEkq%BILiLR}ZS5Qtx2TO-Z2 zJcI2FWJVlKM9VZJD6sXra;ka)^|cLq9#CE^O%W_|8RUCGd;|RSo4Nr_Mf(sg#DlDG zC%DkESpYy&=X#IOL7=b=#Vw%kIOL98PKQwDi>ge4A2j`?3s^Book9#PxSeoIP*LT1 zUI5^MbF9G=YXLm_fZ~R95YXhCo12jMHjHGE5D@OJF*QWe7LsQC!a$hI3JX_}nDVlF zO$8*qxJVv=j&cM_JiBCnf#8czUu*B^ZKt(pPMwlT<@QOxrN_tTzg+5vRkK(;r3fB zuIDj&@8gTRgnGkTU`N~<4EG9&BXuWGuo0HUvkOP7C(F|V+L+`iVnkAldrs(K!K^2`UcTAkAf$fGkGDkj7;9JvtApD zT6%6nLd~SF(znaK+9#+_)PJ&(ofgY|xNFf^CCkPkjRUH${Wv97@_Q~Jc3L~!DCD9j z&wjo}(n6n4?-P}Yw4NcgnwRtQ0^%x85b{0F_Ig#$NH-@Z=Ol8k^<8fu=Vg=dU;DZJ zfWSe{&4}#Eqsq$3X~C^mEEb(S=!1g+AUrQc#T(yK^yN^^Un$vyc&v8Ty@`;5-w5J9 zDOg!BabTDMsYM;DJD*-r>0|8mXpTZ6ifXzR1d5)mis(HLR9B-hqdt9|$RxWLD%6Wk)6%ZW0x7a6F&OFD=1P?#LCt z*m-w}`foJ-u97%fR0>=Aic2rnux;(6#EqMcDzo|hY1$I_H!=c4h(6wd-A{CMtvJ~D)8KtxO?8UX9R4(Q=rRI@azS0i;W`D$?+@{0eEqx>yg@Bv3X|G`FR2=_UoQD1T57{=AnB z&jL0p>#cp*&}r1Aa7eKm{DS(~(sItc*`lb!;J{jLc!#09w6qimX>csCbD!gUfSB+~ zbO$~MNu4uq*xxQz*F1xwF#lqHYXY5Va}axAE-tUg06QYn@YIdN#s5eaZt775* z$m{n<(xIlD$c0MM>`^L~DerI2JDh_kWwG7(*ds5yp{y$l7QS+(B==SWpIRSMDszL5 zzuX>nDb$zng>;CoNQL6bL7PaIK+E0x8!J8Xbr<~~`q3;8%0xymDpDlJ+_&tRs-ur} z;Edq^fsN0*)r0a23uGm!=EiKvS?{~8_0_4SJJp;YlyulR#@th8#dGG;_|}D4r3reu z_sv1)ORLcmajj|*XWsq5N~S&5Seaxcef|8|s(y+2eHs0o^Hc$1GHyA-7OqB_6%7py zKs!2YXoa3L55QoqYo4KdtC0K$>^K$@FCD+i>*<|ivw$=Sv608G@C4}hkA?6jLQn$? zD>Y9-Wo$)7MTMvg4GnRq4hgX{d)(~=YU$8sq)njdih9BoI`&L>BkT$tis-U+rMrRH?xd$zxz3N)bFi8l`o$i- zB;_7=0;S^o^uWlZady7C&`6Df_ig3@uf+~y(O=0P2>Sy`(HBQY4jk2bIj%-D2@>5K zqxMsL=jhxo8g1Y0X>j7pd_7*N&r~zIM{0CrkM5)KIAG#yDxuJS(1EI|iac$bL3wIS z??lqboC}sP3RKCZ@TS-2wvb%JJMjap2aW%gp1?xNMOewGD0<1ObdabM^x1}er=nqu zDHfY3Y^ioB{235*k{E5RR9i@JI0I~yIjK&}LeF4Mf1wR(> zW|MiDoTAxzxAA#Les zH+;XWleHe$R8G&9Hrac2X@R>shl2Y$KPGn}u|DD=E$aIG(e6m6&`hNVbLHJb^cIii zu|*2pOe3utEX?m-M>5D|;s=sD3T{vgx@pyQr1b`uFL807AUTN?rhS%9`P^v3ZYsmDZ7+zNj z)b(753cw8K$BiCUymet~eXB8_M*b?gD`e)6uN0r`7YTg$GF2!0TlO~fEBkcr0_J^V zBhJG4BaK;ddv%*RkzSN78@$ffw>_28*q)g{{^*ZM560HpOx0k;QZe zyL=DAi@HS3Ya7EdKH{X!3iWnb)2=HR{1|Z+m2^xn!Babf6E9?|?~yr0ojJMnGFEJT|2F@%hq{#>_-JbWKKqkXmv~xU zcH66%xUqtZqFbzzFP%KLeceyN1KkoB#dxC_qA$rcNjQkhLVusUv_%2@tKdMNc+DNYX z8{|wTs&FiWTdvIF!Eu2K$-#vDX*>m6cD5|!HzT=d5Hf3G90u}JuzZU4Z?yno8-QUH zIdj4YtTDt=k-{2C)PuBG5V?Ne?TT@MhENL&l%f+S)kkq$Y41B9iEV;^Mos)SntNrL zFtuwS=joX-KNr2c-aTGjA}a$2!yLCv-f>`+L638j-efi$dhf=2`*Tm$8(V4bk!f(E zO?{c?!gOs&&{yPiTDCfL@FD8i;#QWw3ys@@4k3* z($;QGqQTxwvbNiMiw{q>`i3-oONUjD5akEd7OUzr%jE1NCo|{#Gk`#<-<+)%L_6c< zqMYQ?H^k%ymW#al1w&l+!n<#!4pj;A`%b0nm>G?E1!(#=M@pXUo0jsaXk=a1^PX|V z+lwSmVV&KoR-5zB4=>G4{_#EVt!S;mzjr3?j|Dwh6$wq;%?Oagm-xldsrft|=;1cj zJILQ&ppMsO&!RQouM#w9leC(7PNMn5bF@v0Re1bcj)9hMZ@4+GChea3qEGuF3-q4W zdKE6XW-B3>>(R?^+Pb>#y*j$BNpdyW*|tr0AgY=K*+5m`_ADvUi5?=mA&Bl=Ypvc+ z*!p+s*PxQuD(;-6{*p(`tD=u ztd-7$3FHg(xwvGf`2mMZ6&Jz{cQV-CSjn8{NFq4TP1M}L2t`6lPz5;Vkm&TdsZE6! z4S@|Rx&6mT@*WyvG(9zm8O+bg-wggIlycIWIdP*zfy6GlJ3y7 zug~H;lM6MJMgFH#b)79EeMwqnW_nw}0pwD#FEQMCk$=vv9?@Wm?Yh1Tqcg!m8S3IXN*(%T??}F~)4UBy>(W23^ zG$w$wu3mU#uI&T0w|=VD$!o6@p&FaBQxajS~LIrq82yzqpAC6hH#ke(B!X^3Ob)L?3V*Gn<6f~3*?xO zrR_S>O|=#!crx*l?d#8T*z5&t^1BF*z)M<4%cg-p7tNlZ4$)Mrs%@EC!lVtx{wn+Y zGKhD|VR@%#GZW|hlfvB3xk21INQCRL*0Z|S<)^wm>n%&Fni-?ds7oeg&#`wap(Up0)kh$MY zLGbR8<@Bd_3>xe-3j0c;e|ii`yBf7MTaDB>x1CG&+bW2=Nb@4Ivw%MIs6>CO1gdR9 zBF;7<|BbDR_BA6Bmr)@O)%pIvJtI#Y=Qt&Fcv|FKnSeQ_1Q$E>sE<}4TFTh80$yTL zvpu7_efLv*tn(EbwgRI&F-{WqwJYo&rF@PE`&p--dFeE>&P4QwH>n);LdQ7&Cd3`` zhlQYMcSX0C@pl*6S4i-d#u+Vypp;`(vt92i)q%6DJy?Lowl^=_<=Cm5q z^OSk`+*~*w8#XYydRJUiPmz3gzB4wk>-aS#4-}6;c*hX@o~-^X?XiKEyVdPHrMFu! zxAO(qKi~FsIBC#3Wxeh*_NV@$#_ZjX9=t*hs5cgmYzHXfYdN2qVs|UE8ZRY3e$Xqb z7p{0ZClzi~EA-Ou5XLmmlKgXmes@Rp=fn(kHgj@`?v|LI&E1Q^nv4{`PI^C#qs;Yb zqkYK6B68{4*|$%wC9nChyn6)Q8ORR7O^7FtAF=lk$6+ibFiR!Y--DRHILz=djoxc( zojT5>lzYNxBJ$XEO8b7BTPgPWW>slsS_vYv4= z$ntz_cWJP(wv=7-v$@K+HqCx^?~_gR*X#Cc+hYge6E4B%A5+=T=~w6B$fNQvel#!R zkzoWL8>zO z15YUFXUMBIl^N@Idooi4m_`bvY!49p(udnI3{e(KYJ9*p zC@Zk%UQ=wgt$v|A{PKA?HO#kQP*JVhqn{=Bd&+#vFEtHU+b~FFUNTR_m!wzTZgW%} zY;9K4{KHvzgXV8KZY>Qm&F{>I=Z0K;jN%4+=s#ZH+<&MnYAB?3WY|?JW^ss%>(0P`nBHLaKFJrg`@eI!aKkE&PRo>Kx;h@M}A=Po7|^?V@V~OptTi)u1vj z|M)&P4xqT^!;4y5gT4a%jHZqru6L2#`hM)4u z|4?AP>jlA>)7qaux>r`Ji}SfD7+h|6U^*19C1Ku;`BLw}LC^Q^Vm(E^{oL7c+Bs%O zsUlCJY0&TL%b0rvwTpft4g0kYEfzId?U)jCV|}7YNVqnBN==z*=pF$OWV04YT{gXi$M={Aaf_hO@U#XmX&+e zr9MX26L{Ukk@2En1g~T51|1EQ?hK~P{eyYedGg3hk>%k$#-mRglGS+wiXNz52u6@k zhc9y9dI?<6KT(sSnDStUo~J^dW}hhhXd0f~II!TV$^NEp#c0m>3%*Qt5I}#7QPSV< z#XzFYcx;ZtsoNC;r-Mb~pMU{6qOZro9r@(u4R&PDZqR{^Uv2krU%-btJrkX>31dU;VLCGBTXUS=SFO9X#l%H6NHnRvQ73G<-c zLz_p&S7jQv_qNVfUuuc{VjGz9Gmt~o-x<(EWtdjs^J9tl;lzbwvfO_5ZF+}?+2Xw% z&J~v%18cmOiJj_m=gM#wDVo0nJhM`*fU~(-ltRN$-cozZC07 ze2&>ZQbX{&c}77esy>#5%HP#UxBj(0?T0$-qfEc2Gvqh9@Hl>MYiu5$UKG<#TEWPg zRhPxJhm5Zs87IQ%Dfw34F!m`bKFnja5D-joUXPj_c*%P-NUOoq_i&rU=HWXr-BCaD zjkxb|20egCO=~dH8{yLtWR^lKr{ji3XE;iY?s%~*zyzc$tg_E+;?n9)vhIBrkO<*% zk$9*a#bKmLsUzlM)GaDQ+r>w#uU;-$t#u_hu#+YuayYRjZ`4WcvRz!S#r-Htbsd4m zwjuh-DrN!GOnsGJ`PXKQuTQ^YOyw(i@5UzWe#%=`FWm z=Oapo)x4Zj3aq}Tgg@q|Pq=CvXK2)(=VUdj0%d z+J`7do))d8!+(rcW0Ko6-!{$L+m$5!!&4fYeg&#vFD<@eS-rk_# z=6C9@RPW|$(KBM^%q+^x(zppf^5FO@c4xLh>*i#>&`Tgzcv3AdlWTzw_7zho9c zf1bzEEHRQ?MUDKHw8TFtoOWJF9*tZ?7Vb%#b5u;J zSDet}HgpSHZdQTLbVVW$MMTiJ3%txM`-oEQ-A|Fi?fr*eOv#8@uYF<|Z(Og-mBmG# zh-f-TJtR!TrWy4ADRV8q9P=nsYoMb+=bWbe-5`|2xYfyov{OYK0irS%ekrM!)wTxi zpEKcQjJ+86%k5QE$@s?&ruU*wC$steT^;IpZhlZSI(tTsS(o?q#>Ec|{XZWH51Q9~ z;uLCZ4M}Z^^~=O}33KKfi6@axT82-CpdmjG9e``*1dYKyVn;VP=F} zb&KIrxuiZlI!(DeWK~w!+M?gH&Xb;U@kB)Cm3(cwtLQ&G=;7m9qG>?_)J|fmJ!f!q zd};Htl-P6ka&v5X&Itch3(}+OIj-aP1fE}9(wF7%+*Y0+2Nbfq{g z_UIoyZ*}~f=l7y)6h-SPr3Gip6K-Dg`zP{fq(OC38+p*>*ucN3U3

VEogM8O5Z% z|E1NBM`w68c>E^x;c|lJ%s*V6?hD+rmwz`Y{(00gI_;4-EtjF{%-1qvgJhF_;==9= z4L$DqFV_?){ZZkCI`ac3ZqmNJxI^hMxE=km)d6EUoM*t;!B2CO<{wVpoK}ZbOv$IL z`L8Ejy+26WM4D#K_uE}kYH~|(q&=piS)abT(sNbo5{eObs8p7~wK=FZTx8h4X>&uI zK|zzAGgap7EnJH6r}hgOL07c7@6Ay)pR1n(jb?YJ7&VCgCL0IO%0i}s@8Yt|6eR!2RzjO|CiNSbw)ZP8Hcm?9@*nK zIs2@PJK^kADnbaMoI`fT*{*EP_!c62JF6rsE(xh<{XhEs9}kZnJs!?|-kAmd8q%0=(-L(PffE(>;g6u#L6rs!FxDgMgu zaVzH2bL`mGH)rYMrqMe8h)-gmIPnny+ZSUN@?z8$^) z_|WIcIOYmD!@3ccLfm#tkXi2(8EHi3PKue-aA(HH${7a=XY#vHWc9jzoW&0F;L=xS)G zeNJSB|8jnta%*v0T}6aIMZ@^NdcBx;K&1L09XZT9f!OR0YgIWC3V)W0@dvs{HNuih zO}O@&v!3a&38OeLsQE#=nssqH!~wbGZf!pGa&Xit%P$ME*;eZLign57Iq+UxtjK@p zw5;KU^vql89ZEM-Pjubx5}~HA+7y%+;`nzH=<+r8kAzZ6B~44AaA8(4KEe1&n{j`{ z&!YVw?<#zf*>BI5rbu@B`}|kpLQFE7dZJw77&!7KH-ox2_L#J-OpwZvAKD>Arh<@(t)xpS~Rts&qwNnPGMJD-{xqHf5vDNTS?wYc=+N z94>buKlrot?)ljX)Oe%13&#eMuoN$T3z6wShA|2P_nwK4tN0sLd;Fwcj}HZ+TVPoC=qL3J{(%Ck}a;H+!P)X5cUa><{#E4lhX%*M#WYx2OYl+Jz%CF}p zij!a?CpF&TlN-4l5+_a0Vy8uBVGNN_m*2{&y)?rR&wFm9-=O%UI0ut!s_XH=Nb7hc zLx-e@DNcMuo8?Z21@G}aqw0^RJ)0n0IoD~7!5PxxbP3q09}BYa#pBjw8OeHClhqJx zV=0Bi&BBmpweUeSl!%Eex*Ya4V~FKffxmi@gelwh%CVNp7qLB$`?^mIhbGfZ1c{HU zCH>Y5te20YimjG>Eczf!u+3D~)<5BLbtCRB!(s0pnQ{ud*ecoDnw6fWHdsZbf()Jr zPfemt8LLT~YTf1#pwpb(haUk>dBN%bGkjEl;ZyI_+S@!_sYFjU(&;6YzPaUF4 zv4+oxp-0VxYgIlv$zY_0A2r(x0afja98z^WMsJcVcxn+fA_WSAWz_|N>P6cpx|2RX z88%lc&v~YLnP&r~_|n3Hz8F0pGb|A8!F6Z`)J`I_h4tqIjZDbl-6ZouOW`CzR$4)Z z(u$kiQk2dn#lCwlTFZ@B}WAv$Ez|ipJUSZKux6r?$1eZu->a-x0md zu7fmUSUX1m;t>MTwon#cEylzYx4aPGV5j5ru7PQ$23}e<1h`IsP5T({8@2TGcu`M) z7R6y`%0?J-K?=OkQYmB?@}1~B!-@Q;&*tu0ARTd}b9Z|tuoMQ5gAFyx1>@Cw;Fg=u z#5U*J=+m8^CRTl)VGP#;Lv!fjIIpodFA zLMWdlH*M_ONs%0bjl*_l)|`>jfy2`v5mB(9S>x3o1z;48k!i?KbiV&vp_F}gSZbz7 z{FCFMI9R~e0%kWd`W?rf$?E;HJ@?`gkbFHzKRI`#QOxSCDo~hV;Lhy0ZjW|aLM(e}v5Z7S z$rPJE8lfETcy11}>6v)X)5e0kTh$82>3XiKvI?6%if={|N>J<@DQr`de?&P=Im-2V zUPLm`R7$?kYIsC{wwy7afh<_p*rj;p&)fD}W|^1t$rzWcT6n)h68M7(;ZsG#>UY$& zQ{7ARF3L7ievma2sJ5pb@F>s0p#u55wztnIpDrDH$!89G@vc%1c6leVq)bh{0JmhQ zpDAJ1oa@<=>#1LxS3qdd6`%N}lb(?3KOnJQOKU3ALC`{ z6~h!=kLxk_KC90EXUdN5eLjCBl$J#mVaxK?vTUt4N?D*ka6&b5FcS;NFiESkmN?{5TO=0Hv93$$oT!x-IDtvS&M04bi>^NKE)W{870)zX= zbY|M92X97iTc=+Dp|bVM;4KYVapDTgo{`IKyo2HX6PWzfIst}KBFtTjXFvOwCB?U_NK|O z48v^Y?o;KPbLyQvS&=>E)xIcmE?c+=n;wV(R))7)n)|p*l!fPRAN!~I14E-a8(9@a zes#aE`|e{#rps>{{mgl4EqH}tR+Bg;?#$Tr#iE(~61x2C3JG5amEICdKO4-4XN7=M z?~ojDzUDoqdV}mZvU@mWb60)ywyVrCr!x9(KnYUtle+r^3-84B?cZB?-iz$0QmzrK zshYo!M9y{rkAVIK%E}SLc6{*GspPYBEF@Gr$r_llJJkOgEPiDF-?)xqE*LRAAObFO zIHk@e2cDe7pYmsk6FcO%dc;_GOSx9j5T}>N3aq6G)Y+CZPtAhQRy;9$>-M`mk()j) zTStMTe)Em~gGYB10!k#J!4CcI1$T>Nwi`WMJl#JCjjR0xnu+*Ad6&GptgOey*0aqN zD)xW2Qtsrd-OD|%_`-$t9~ttM;ml3W9xuF4b)Q5Fm+z!hh|LkKG|fQ=+?n4<^Dn=v zV86!kgK#je8cw;?AXOXrsC&P@bXfI+IdaupC-EWKvw0G`p*!YS@tv8MmA^?R(s(zL zMP&v@o@$s2S=_9*(_g4c{(Iarsr9y-5BOzW9{O*Y1aMmJ(lO7RU0JJB4B$$-##eHX z$`C2$yc%v2(Lg^=@>1)MJucC}J3arQZq1QO2U3Gz==id!FAL(HTdH#9z@)2tZ5Ik~3S*m+Z-U8yZMZZu|pgVq`eo!2L6|S__*F?pmYv`BE$Lb1{;Ne~EFZEMn{ zFQYRSr4`=3Y`j8l`RzLT7z!%oH&7JfOa~wUX$gkwTxEzlPG|-Q>J4|gDy?8YyX};j zjmd3&z2GGiGx^$we9?X(s7zeK9dU8T2$(tehY}yptX+Azy3oOkOHJb`^i4`x6Zx%_ zZ+oe6P=>gLzwd#l$xQ-C#OVuRW|D3O&uD`G`z@sl28yrLG_j=sOz{Mor~F{v{QaG5 z>~hiay2?Q*|C>^d>8~b=Y&+fC44{#M#!-VIjMS`2VDUQ9E9v;rr{KjX9)h@l5lipw z$)e+VWsPvDY)s4MbQH?&!*VFo|5}y)&-S1Zo*wV5htCr;URm1M+Hn{nec=Qgh?Ood zJo9&W{r(BQ;oa9|ISwgK#)o}tz40b++tL?>ceTf!Dta=O^4G$lyz`3wGku0g=VBU@ zQp}>7!u}6eAPYzX77|>d$^AG{2q*an$lIBy9UbBj9CS!Lhb^*Kpj+I16e5vhM2B!s zH(m~%uUti6Lls@W=}+F-3=e0kksKl!p$QiU&w_=s9WLs_C}0p~2&ad+gK2H;^zx)B zdGf7#ii*H>FT}Ssfh_m_%c11?q_8n^Wi;YG7VNBPd4>DQNN$@WjGqV#mrDqA&roeV zksj<@T7ZZ98P1eFw}cP1Q5yZbWLOQ5Ag+?e9rnIugpKcTSGfQ9aJXWi!nG$tMkQcO zST2NxRS&dXAfG+_Grazr3dg3=gb=I1^*^BC!6?WP&c9dtfnZGXP}R!7X)(bBv=5)(`Un0k>=u$eG~@1RX{$U`bIim-*zTnG7;@l zWPGG)Qd(srjGx+!gTfD9sQ36j5L7+ew6OJp^HM%{Q7!@g)*_h}+l_oIaeoj&Pyp4e zjuSktJPTeHe9wXxU?Gpu^3Oa2e)9)uO-yY34&Rf!dA=H8dlu8ze|+X2ySH=&$zM8Y z%f#q8DjK`4z9)0vFUkdLPDJ_+mp4{bRU7b+hWHFHI?hCg4hNdvt**;&(j*5vlS6om zr;94@?y6~;&f1v1`q%&bf4dy&lF1uL@|o) zfB1DU(~jOY7Y!y=KD$ja7SuyIh~bWY#u#s)Zvgfd-&P^4w2WC;cNUJLvH|XHQ$2nE z*gxs0!T^O=GJN^3rSAWnlYuQOx??2p{m+H@Kh!-7 zXPCLEk8&e+_5;6barQ<{e2*Yje>r?T%2S%!^U6`kc zi3TlunE9P0!*v~*;9e#(iLgwfL(k2hu4{gJ?yL{f%a{zot`0-7?D8+tprwMQc_O9% zx~KfluQiq=r)b7BK`(L4%YFQLn{J$o>q$%g-MggE_7OaqrIKce;x#`oKx6Kho|js* z%<4$n*RcWaF=PCujU{Gzj-Sqec(+IKcU2)_j+&A@B>uHAJ6@cKguDKf2I~14tM<$o zfZla04f^092K_0Mlu`pd+`C2G9PleDtyfyS%MWT+>VJWLXS3$lL6=FtO8D-JT9dWf^sbo_KizW~GNRDpz* z3R)X-2q_n&pfslJFlx0A{aqy?eizi_vKHapNRp4qFfT%X^$!Mqa( z;ASS%9O$mi#B^T91o4ps*#BY}jg{_KmXl#FFqx7)6f%nmd&RT0T-Wx8fQ0*4l}XY; zmF(n*A26Di^E7ABnil#EM$a-kLn#Zi`fj3^KT|&unG5b!|JMiS*#tn3TRyS11+iC1 zINPILJY8RkNkv!+af%w)U{JQX0!)eGSn>b=?3ZpKW;#%#00Qx z!>XRIFZdd)oo(eIoE zNuE4cq?LT6;*`=?V_>FZ{(abZ&c-lfZ@S7A#f>)a)EfKsY(Y0o5f1gUfo?QD%}!z{ zjsMq0=%XkQD}>SN@|VKm6{EuFZrxjr_*G#0>6sCSE=OvLfye1>f}#lEQ3aiVH;ehB zcZ&{U6~_+~aJgzLVlP~tBv8Zn3IS+VX>|c2g59ilshv@ZmEC}K>@5A6ybmNARXd5S zHIkc~tekUMF9gKAx40pV#GxrmOAqh4OC<)p|F#D+Xr=a*rL__QIcJVaM%u#bUj}&1 z9zVJ=%XuA4r^GpLoMnnRm>v%K*Yr%*@`jIF_g&ixDqGABvhE2B> z7%&wL$P@T92VxE`s2K1B2dG6%Yk)Owb)ZHHB|37K1?x*ares()(45UQE1eXxC2vY8 zECnZ(##J`VG13HK;(ZL(Qw8`B43_0YB{_q(xO3L|0b=V z364$v>u%$ljVa&oNYhAEgXh^iXD|at{xYeAr))UNh!Cd!=_Ej$f*Avz@Cnpdm(`Cy zRIQet0_qFc8a#|?foBNu0`^Z@(d9X}OPROtlLp8o2OX^$kyKAi;xTP3D6U-05w!xb zuTHD3$yMxGkORi`^36klFuE+nQS(%BT%osM`eg{9kV>F?+j=JsI!vm`9`Lj_8mxS5 zFAh)n$1eL_0*uU;Ql3O*;y|)dfz71ghT-0Ig6A!CY5_yL4qQWoj*a%OgOzxuk|)SS$VqGcR|&C!fxMd zakUjm!SQ>9YXENrVFXUoZPv3F;8dO!Tkoz3kp`24Rkvr4A%ue9#<@Bg7l#oP*kEW9 zKkeY{huLY(G^<2_4NdX=9>!;PML3=1*3JwDlN==#<`qRqw)y-Ic4pK5*Zqa{pE22s zQSy)(mf~h_s^aBa0RnIG-;vNin|;A!x&9`0we4vB0uu7$R#1(hrPn3GBjkB7F?E z9A_j!j3>P&q~y=uPGAg5DmmNy>eWJJJ24qCF&U{4H1N=Wc$E{qDQ?zbK}b{6zmAPD zmu0MjB(g;}d(p^zoFBmE-Z+{{$`j#NU8++haz40Am0#A72L5!R28ha}9>7u#rfJg# zfl|K5SC?52+_?4MHCZQM|fl0Ma8#Y*snDMY4FqyK1u}S}9`CqgEAej>zk|XQM zXLb*BJ{Ll5fYTCoq8*6HhLNk=c>l))K z3G(*?LyEEl;f(bxU+`Yt)NgWd`t#e0g)ZC7 zMdgLd?(Nuh4nE5X(FE7)0xq*0C^FQ=ni(;x^m-DlpQVVHwiRZx5;S@?1Qh!>p1s{j zV<|A2A^Ec=(^Lz3!0Kk~FSA}S5cVB3T5gOhFui09HoWoTEL{v5SGy^QDF}J>oes%i z5d1r>2L{#!C@mYbX_lcal9Bx?7m0TocwU(R;39kPR6!GUM}cpZ^#J18dW-izM?+D{ zJIYG8b6fd~J&BjvFC6jis;?Z;POxg(SER$*ZJImSz2e7)c+2MwBwy&R{U2MYz#NTG64DKhjyTqZ;r&GL~{eWN+6#)?5vz8DzO96SNWoTsfbCL6R@oW`!e= zwx-s=e-sh(Hip*1eUpOjEzMA8Epb=2i`C)-d%zODI$J_xF%14rD<)q}qqyxzZK~z? z<(AjDS(~-SptDAhmR^(*&DPHX`qA&8GRoES+9wer!drK#Z^oaFhSHgqHUdW-CJ|Sj zlp^Rus@n0zmURcDNl=I$RV7KR$-5aztm!5xf~JS_1$wl)FCj`}1(;291uQBOeiY}H z>m^@|J?`q-Mn%6z?ken_k2TJ{E{2&co4r_N?DLXKxco(QTOU{1>~~FLz)skpQI`O6 zl*7=@R;j!u16`xdE4`@+HO&Z8<-TCdA_V+nh{SEyf8XcZ)_kV;<$?3~Xnw1YV+E$O zXyd$|xWBM<>-I@H&yQ#wv7;z%%dMZfI+0*YI0`Ha-#HbYrTIWQ$MSUF+LVLijDpP& zMsW(8PTQEyWCg(hl@>dTL`rX%ZIVInR23`3Ll*rUz@(Lb_@f9UYB{a%E?O+WW#JwB zjra0e6;H=lqOoPgzGGWfqphmPYAgUgBG(Hv#{j#UTx6svRQtQ%RZGnZYofc(Oe+Wy z1R^oUIDR(3pTD64^v}8+hK2$-NkewMW`n9KJ8|QtVPBEtI54N757q(#V}XSunuK%f zuz}Cbgei!ovDW3dmxNKz)Q~ep^{rpOm~4C1ZrspFkZGke;yR9c#Y&6hPsjisk`fO_ zg307Km}#l_NS-_XVc#YJ<2f=sAi`FrgvfyUW3NM~eHWvhrUL@T6x+~wPV$QdIYXp# znAq!F>#4V3ze0laK#_g@f-s?9Dgnu0#G9qupQnoYd)Xl8nI69~o@*H25={?2b zgHY6NXZ+E-oFVex!8fNe9U7t_YEgq0#Ny~_`p(8obN7^wB-88*A*`<|cZyLz{f4y{?q;vQD7EulufN` zS0+>a+bU+l{rxxNA0~lQ41mf*2N=L2cnWDhI*I~rqsY%L96?3Se7iRk9I5UpZ0&V- z8h*2`YiA?m0;_T@A80K;#KK|iDRE153&3Sl{FX?VhOSG^l?4@B_Y_NO80TyR_#PPu zmY6^}lH-5gEd(r%_t}X<-#mK@Y|SkmiZ@9q-%NqL6+1y)d|sO=POq=mt$-AyKymVV zuO2P!uQJDG%8EuY4<>lWrE*I><{lZrav*pP7-&2tbFd>`6elNgb1Sj6K4RIYEcSZt z>g{ks#9ju0S z63IoQDs6cb?o5PKuKD~JFgNex)m)q7&6=>+*=Bjni|dZulGk#^az)xa8_DuQ>l|lrhE|3Y|K)GcBZM9!rsZf z!gXb@>9pKY^$~c!o$r&wX6Y~dUqA;Sfqs#Aupns7av*$JP320P3z?Fi&-3A+5r>oS zOM)1A_nQZ9{_MdlR`|4tHVxTm=YGrd`L9KaJYO`DAMh?rMa|<&9*TCLeXweKCLPfz zxpiptt~}jr8dce05a2*|c6792d(4%{&6q$h1-#!suuM<2LlV(;r$wT0Zp523Hd@)S zEx(HCnb^^JzTI_F<=OPI((av4wP*>9VUQrlew?fjpW=(e8JqX&h>dM8A9W0zo<8T9 zmkKT{Ul*ZSJ&t;4b>F)wVLa%ws@m!;Nr|^hhiMg}%w=^IUKKiCNXPn`r6f~`Guez# zUy;2sY>PN%srDOh^4q=Bvp>TR`}3TpOtbhF@8EU9SW9?_VkurERV?#k|A9woKOp^} zl3f8joYYf_dMssAe#lQ=tlTNfxs%+S{oJYVF!!ah#(>1d^QrXl^t0hWW3$}MF~65I z59uswTD0tlIe23@+Z|Zl?kT4VG)HcWlulU!8k%MLwA2|b2l)D+GEAQbuM|)=9$k6= z_t|VEV5J{{OFK{#2yuDgez~tm1Y!%yJR!dtn(40x73m*Y zs5)5M*a^X;@uxUB1MAQNG?F7*-T3~RH1J%icC5ikYa(Rjp1a_z`^fhz+&`kZ)_Ky+&J8ZGj4u8s7;+X3?VU?Zb#rZ1mz`MdE;s`a0&QX#EF~a=!0bM!iHq z=<>n6V)X#a|D@mGzj~By+N6UR0QRA&0Evruw zvlMf_9WA=g)l6Oj;7nHm|0joB!o`@_ZP4sUIm$Ev>BuKx>TTrxbmCo~+V?1#LJO~^ zy^2NeJc^@ZPHWrq7xk&@=toyvp38E}ve8D8p?m2W;RO+>L(oC*TAoR{2%pU3LH3l= z3x88yNNN;Hr`mpTTpCphUmU2o_b$wDP$nWwYrTN>9250cd$hXH214j2eL+=^Om;sZ z#k>N_Cn1)6S;S8&;sua20;;GNgtY=|U!}QRtljN2tdE(C+ZK();JxQRTsBm$9ywpj z@hDnvRe2ShAyHFHxS{n-cm!E;CwTAzC}fGa|vLgBGV~9 z|GWrzx5Kf$(&#+FBso%xH35R0!Bpv7dDA2#Asm#23h-cf^qSjF!Fxkxdp`1D)`8j(HcZIC|L2S}R_ z?YgN1sdSKMXO?DL^UKQ1;cdIH#s$kKx*J)WKRmv&h)9N$&-s1y%1+f zpscxP$Z&@P3nI@6F2%eu#$bM~=WM@LRZ#wYJQWA`_wXIp4Au3vG&8SveP zx)NWrlLBC``5y4l#!}P)@35Uey`zLkyP>Eg)wGAFq*tA(w(hp>FBSzlMWoOdNsd%x zJu)&j6co<0`{h%oIT7ltQrK$6br<1(XS3#8#g@7Be)~Jwyz|w6RYd{Vv~~ZkVrGl# zc1kHeyK0lLW@$n`mI~yS0CJo-58j6rDhqi8f59v^zSkymTfHq4P5LuoNC`AFNB!NDInN{+ZCW#9kV04>qrQ zV1Q7u=xV1-@cMe8=bAtUM0h0Q1HD&Xggj<`UNnd3?c+N{n9lYgMZefgbNDE{d_X;% z{oK#bXUw9=LERsYF+{aD>H*qUDp@qJNo8#z&2C$vcFOBY8WJpfzz~>5IPP!j+z+r@~(LM{Ac2vbSMwDFdJx<=_4@q zBuA#doi@2LI3iNNMVgG~`tTI!hiQ^1cUzR8801ph-A(NI(=!fmvVT^mZ74zcSu;b-&HPs6f}Eq>jzr&biO`RBhMc^@W-@ z4&d91Hg22U&gyRnn6XL01ODtudIk3&7t!LtiE?tC({~@UqmbqCAAdMCU;T}NjTWIb z7M*+Mf{)fLZD#c1lOE*BI%H>#laA9)p69eFwr!b@u*B7uDmbV}E;sp2G_TBm<6ZB9 z@&OH9k%yGKbY9u#g(8&8$6kBu_|G6(^21kUk(`5w$647I z-|aLm&XN+JJ1%FVrzw6Koe9KMO^EkFZF)tb(x;Q>p6R?Ibo4IXD# zl(BXA|(8gm3rQxg#B0^<}VNW@pNi4!zAPKFyI^_0XpVDZonJt zu$a1RGf1xgu$LB4Q@^0gVqJi|ma`W7e9#O9LqBSCvJ)^pjQRSm0&m5QZt@@6NC~?W zsZDmv+xsmOVZcbOQhdu039n?Ysql4F0KTnTG{qfQliGz zT|4V2Fs-|-3OfPb40XhPyx)-SxqY-(J-a(FAhWz=ry9`n7nAOno*x*1dIgzkX?KTY zn4q9+d-oeUx^hRQBH>*F^J1Zysa*=qstK5+dl z47lLZb5ss-ZL!yh+k1w)wjujVJzA`$$5Z#cNPezo;;^_6N*J>OGB&8*XQGw`49>|t z^?d&~XuGx+>FXktH+Ib?u2&<8xnWYhk)_S*Y{?99b(BQdw=yvT}UBk985mwWj^;-8Voqa5(gBuBGK_YqNAyp(N`QnXlm^~t<2E(F8*Toy)@UolwUQ^bN7LAKdoJ0VG4^Usv6joeVZpD zi`nVPEr=$mLe%n$&rRRHDdFk8w|At*uZWUYnt?riyGx$$IXHe;Ta-C+W*q(dbl?*X zgE;A0|H-kaw+|}F3>z5EO^2q~W+ewXLw$Hl9Onnm%0cm(Q#%_cCu{w_LBQ^#yi%rO6wJT<$EE6N1M-uYnQg|0M$T4U z`EW4Zb7}4c5%apn&);P@LUUMkDN>oqGxa<)cl`OU3gIt>xidf?q@c6HLI?wJHiCo3 z#Ry)?nZLYp+apZfytrP!86WFY69rMq#_}BMV}O;IpNqzg*EQ}j zdp_+64GlJ;Ssrgzd=9!-N>S34#e^EmTLOK_Stv_ArzEM|Ru>gF=8oPi_X6}@>jZhg7s{(;4=uK? zm$;WqI8RKV3(-sd*fuA}%=C2*4M8CK;&TKzROe+z+!&l>s5ldwu%q!aJ#@<{Qtr#9 zbLZr{4c1tE_VJ&fk@2m@W8(5BlF#L%s&6!e9)E(Z;jMxSXQD30ri0|WpM46b+K#y) zw2$=u9X{^J)A6IKZPbn*Ece0(Qzx=bpKJ<1cfQPK0V-9`ZLf@CLb3J-OIfokT{0de zp{X7Ih}^6jA@Jcr1T_9@1rPnUX0D&V_m9$+O`eG+`fI}(6Gw%u^GcX8Y!nm z2jt^x5#6o%b|sI%k;X*BUwn%^+hSUJ^)WozfVUe5=p4?}2Q9rzOef|d)}2;0KSf@Q z0yV&FK}K)Cd2!Q9Iva^yGHe*sDAzMAp8+=3jm;x$)C6CirLppAQ)aaip0Q-oV8CHt zqO`ihe{s0g`Q0XjBnvhlad9F`GrVh;o0{sb1=Q9b8G#7aH`~XtIs)g1uj38(OK%vb z0g{?451D+VmVpah3EXCQyKl0dKwJLd%_YCisqYF!fePul8T3@1v7Xx;gt1nh`TCzC z+)^@PJMIJJt*vPFpMMvB?yHV8cFKNriQLY-zPZaQVxo}8WeTFxi7`xX6^=t#7}0zF z_b_em?$w?FO<8B2{GaWuYaw!OJd%qd&Y21E)DOz1&|_cCw|9rb57jh?KmMrBX=xQ! z)O6O=_MMI!*Vnjgf9eKq%C)Iqb;bUUUJOD{r{0k4a(gQ-V1xUzX5!b}7!p}#C$*Sr z3Q!j+W72n815}C#HEo~M994OSHCu)h_X_){8C*Tnu_3nLo8@Q@8wz$APP1wKb0#&3L>2 z%63_JUH43tQ#Vzwt(6HUZ6~~ls=gv}``u1a<><}2x?^q7bFHgJ*!mWqTQm08f#zig zZ>?IuaR(32rDJ2CYN-Cj;tmsDOAEiSW67EZ>1xOgHA+%M7GhqJqEh9kAWsJGg$tn6 zMfY)=<6tV`}nYSKg{hCoysWy(-Xi>b?#lEl=Fg>duD#BBHvf2Ep-fApXxfJk6vO711KYxSM<#Ddzr!wA8(}`8)|V zgjOR)6@m8Ny@p9_gbn((XIsCFqsZ31b1l7x^ak?CM zZvB6S%~J(8eU|pVagygP&g(W=`8c0I>X!?3?^`x&1m_*bJ(8Lazt9%~$p#UYKmOz{ zOUtlg6aeU{>H2|PFqOG!D|o{R z-&Z8w=>Ik=o{B9_6Ry8{^k7oVzA(xd*^P=)s=k zSr;C+4Xa&szh766sO_+g9etAriN${uR6k18-;#CH0fT9^9wzqeM;2LEHzcFvTQ1I3 z@P{(Xgx$22EFR05s4Oc{Ru^mUh;EyP(3C%cACBK4-Ci=k0-}u7UR8#Kb-tbX{UahY)(a``q;RPU@iLhT3)%1{ z!nC#?FX%nYmFq&za=ktMgE>_$ZqHO@$C3K~EQ}*cae>Bi17sv)`)Rx_Jb#`*%fz-D_WK%2oSoN+9 zV$w$-p;szx4l6{dIu>0@uy(x3lsyKDO#`JFJW z_whAh%}4{~dIf5lrw6@F#4lP|9xrv&YR>%DB{eF4(Hb?Gqh~%dHUB$c1XEH&jUzTM zuq-o$m6>--ZtG{!_8rMohtNdxQQS$HY)xs%72t%$>BiTvzq#mxO$c`)QqZV(*QcFL zqJH;4bd!*ukh|)sR;0*%Jb0A%!>Oud2Ao~x=9jPMGDX4cR0As0w_@~PnEAX{E25~U zkMy$DD-CUE<`x*I`PkH)>y6nUIX~28=+Gtw!Lh;_Dn+Bn0IAf9D{U9q zFjN4sZEB!vQwj7J3%koiR2iN2Q-mj27OZYi-+Jgi;n}loD;{cX{WxK*c^!X_%1}g`*S^$d?fwbzAzEocXft(XW5KcD6I})7N|#Dr+MAtkL(^x5 z8WjPMe%%??{rdeNgzzJ1VKA8AfV8NS;vP}Ji3Tr6P=x9He_`Qv9}uk*019(mUmh5?8;2~{L~pd(g& z%&BkbO$BRT@n+G#szmu5k?#u_*K=n_+B$>abC)Gd#ik+6wiTqe8ce+^fiO?FajIfj zoifa#!&T1A&&*FvrQ>GlAa`WEqaS$7@N@Se^R-(Zw@(RrplW=3^arju6=SJ5*{{2q ze&6_WH5kooIq+2OF_tDDf82=t(?32Q?G^un$WHHmG8z0Gv9H741gkkWOyv&tjQMgq zj`6~>t?g^T$svt?;`6f=;S_?(x;&Sp-6f_dbprGWF=w3^Hf}66oh@o4og#uigVCUP@|M z>)0WVq#0`Ma=Gg09rR1qSWubA;@4T}kB!VOq{@k3f8@F>VTD42sr8h%J0*k(c*jB0 zmGQ%pX(vVq=oQf{?j=*-^vWlvn_jDS^?dYHCb@$LaJJ`_sE3z@sLts@K^3LmOW{~u zS<__7N!sS-1XtO^GNh|X{nyrzT|l`IccHB3sv?iE@0<{5s{Y#V!{6=F;m*5}??y+3 ztY;q2`3|!7QpKpc?tLFG_t6;X9Pp-kwr$%NH1M6L$o{$I>JfM0xal~1(zAI-G5=8( zPF$CfPwu8ZcN;$aNV5#@0XT;Nh#1B-N%LW$D{gZILKEP-QS)9LVOX14U-#Pd&YtWv z)w8Ujki}gng-0m(_w}psFT)?mf`&K0{K(k)4vTsaGpE+0-N2x9&E-HO0^SR zneTIvo`sDYH}ug-GHSbug|Vl#Nsf>L7lG*QO6t5(&~*lU+7E9~t8yWg?nn9E&H#tn zoSwCx?j7{M*~9P&BieC5%!rQ%RprEND%?}*3O-KHlnb@= zyuAKj)g7s^P!q_bpJ-xCdTpYM7+N5Hr56v$s;qfaEB`=KC7!x*aVXmgo~sMgL90w8 zIyryP1s!S*9(4^3J>-j?X_Mu2S#g@D6jCBAE<7ATy?aO{XfB@h+e+J^8ZuEUS7}i7 z_e6(K*o9*)Ne<=wh+FbDe-SK?+A}!G@YE>eO`?1!a zsMLgY?{ie(!P@$Hvj=|?RNp~vDl+yZ-&uF4kH9x1SzYyv*mz@aEB^Dw?LYr*0T<=! ziFaq&k3$M?>ccfb)?x}&FI52p1*%phJC{aZn^WPD0@`xvh2J^JJ^DSfeJ40%&(D5< zs-9(AFukb@cKC_@k=8ScYG~83(0T0sou764)&m2%CbuLVjlc?9B%+EM@+3R(UX@h@ zb+H~O!Ll%*wV7L{p1HF-75PaO`qo}Lhj&5<|EyytsBtaLBl-ih`vqtk-FzIQ%c#V= zY0Al%tY#0^1MGP=#{Uruq;6SY!bXj(k(rcnB$J4|E_%akoop=hvH6f5w?CXw3wfkf~_1c_;b5j zQBLi6t9rTYGq2G*4rmQw4h4dwyXUqk@X;*-TxrjzK|MH zxgs}UxqJDwY`XnK@y~^UfswM?LKE!u<_|l=(tq?omU-*uaA=c2oUBFv&0>^RII(St`?YDqNl7AD@Al!>0ntCc?$ND>>7rhF z;Xv$n;c6W}t%{>Q`B;Ef&u^z29V0i)k?`r_VSo$~?=leJRWJ<1r9GH4KLMMzX(mVh zSFg5uX96U&K?-Q!2WkA0eMzh1J>8{x&@uI4Q4#U8Yyl$cD#>`x4KEufVe0&inqDBa z#(S5l*5XA@amYg&>8#n4IW7Nc{QIttEvUN)m5R-TOplX-1&8cN#?)8o+w7zV=N^oB zZvc)r@$r8TyEnC7*eF;%MZ+xq0~g;`zrr%3IFh30Q;u$?Ff?#@cvD7FQz_+Pr0Yhm zY!0DJ59ARyEje^lVs$Ti^)7E)ZvhPC8>~6Igyvd3)uZ16U;8pb08aZ;id|sHF*Vgo zD&Z`=Rg&fi$+R(puy?e5ZEmmra%4tDLQfrOK-DQwETzfs%_AhBXAe0bcn7o-Iuyh3DJWKX7Tx}4Sv5n9}Kv9RL5zn1mY;`Lv}H;z%o zB%n`IeWbs+7EsOp;p?@bWP;JOg>Ny81eenLKb5w*iW&r8#=5_Iwr~mpz2NA;uv(}W zS0lBoUb|iESrl$cUjFR0U$H30#dlh-BBVe8kix0=x^zQEAN3l0y2pTRzk2uIwwDRo zm4>5r?LNyx13y4oNt^Dd?hqjhR|On@C0N^VL@x{S%4_5C=yD)&IZzR7%q!kU=*0q` zd=aALpf-NocsBDB2uSPrNp-wMBF$Kms!zQ52?tAX?6US1p>cbjGtnHK$8A0YL3XoC z8~a}~vf(zn5S(Z?<<(N$sgmkyae|;?x%m5X@zqv;uFu^Dy~<6+3E%kXOt!gF<&qL5 z*m^$x&_#|#&zz0wg8&`5Zt)>4P1Qq*zi~QBF!xlPcItW!d&Jbxy#SE$%2#iUXT!V) z2!tR2EeVfQf9-Hb&Ct+0)4B^|38gJ8w_YanN`vmL%M4NuhC*@}lYe?@{Qxmy0>j8c z8ze8|vW9gmibn+%$GC?)62prO?2(RG^`@ryPfx()o&7Go@uCD~V4K`Xy9(r(N@-R5 zw5jRq;v+*yX-Stus~rn)O4*BX<4B7F=JhJQZ$F^u6^=ma#KY@TKR*>DOF}FH$w7{Q zE?@CA{dKgXL?nwGPXC%6$7E6eZ5`d{pe&AxGNS16$b8`8cxv(?)=-(Y%V#Y(LHL&f zX`;{jT2fM(PH0>pwa8D;M)ct8&uJRHUo>k@{k#7E&D^%$@b!x9%jjanB%?eJ)uEc$ z#lIKfODhN36jelzT_$?93o1uN2OU9Y=BHo8erK=IfCRb4!@KF%)QZ1XNMjfomr$h{6d>f()wN1~l> zm*eTAaj%0~nSGzY$V?<)RN2KUp(8(tfASt2ntd!YK=dZFw9O zX9mIb*f}cWgMP8oCz-|}l?p0r1=g@-)&k~v0LZ5W+Ok7%4;jh4=~|~G zZEs&MXEnFSX0_d>?%k(QY zBRW)Nf(dLx&Z-$Zc^0CNj^b+_50$MLv3Cn!=+Ixe`)28ZYq(B@BV4Y8P=934hRbkRK2m&0;#g9xCR$I@YqQ*C&@-EV*!O8Iv3NgfDE;v1bF%>sp(>ep z*cz$KR+yGVLermjWsb0cGWA4iL@$87THs7z@i3zs>TJfwotxuEN9>;r5qgT5d=G%n zf_UOTJi7HKL?jS`!BT|6yY8TWglcvs7blGxlimS}HmzV zpflTYhYvx8nBK|M(eX}={6h7T?*Y<9pkZeKFI`CZHVDrdN)q@Hm2#(bpv(&S02$Gz zC1$-7CaUNOf*A7~S7^v9s@V_kV*3miVw>5bYW)lwBgo^CNlaC=HOrQ^4z1^H^;1f5 zf~sT8PvyZMkD~8$G>M=h1c8BqCQts*A=K&wlw*Z?Z8dOSPCxg`7>|~E>8p$H=pzbS z*y-*ik8Wrxz1K}NnKQF)Hrx@N-%EBjjL?XDGIgI2018*A9Q*{y9MF=TYbSPTx2(E+ zR+9nKB7KH%z94qc%$DlgCO&&%s&oFi%?Nrq-6~gRi0f3M(_Ekg z&oupamXbC&g}BA;yKVuFK&OR?GPiN?MTl%BSzdQMg%BklR{xlji6BJ=%96~8{UgjO z10~0{eH9crT5NA`1A%bWlT0t`>K9SnLf&MCf$S~OeK&K{`zh50*-SFm=QOw5&BxV| z@GZ-Rj)@XGdw^rn*YSIG+=`S@znThTe01{gq|iU|qkQant10hwES%LH37;q9STu z>qnm{{?a~36Xtmpm3&;(*|&ntt{}r(pvNYIr|f+p7OAVc8!WE^pT8IBPf7MRzYgF%uJ_C& z^K2-;sj&`KjOzVf8TXsQHSY0u(ss##fgw}NMA8j9B77o8<~T=2Fl*(5dMJxx9eV^f zSKd!_wTb@-5)7xkr|A_WO3kTqx8c;G8W+I3ko$@NjN?5%Ojx zV06@9FlyYZJ5T@yHImD1v z`YwB2=;FchsLmUy6F375B6Bz}d@$RZvL>`7Jz<=*unAT(B1`PVtcYvKR6CX$5hmnI zBTRbd7B9GOpfgBakO;KlBI&l)pG9)ET@L~I+u+`c!854mT>Mv3{I@s#wh;Of_+xZdJ!)NowX*zPF=9}*$?g!T=L)_1?BePiQ zZsU8;m1&7Wd-GC$lSK%xdXz8{;6BKaTa7mWSJ9sUWch9v+yEaW#TX1-53fzhT>vWG z@ug4TwRqs{?cnI;vqrz)_RD`-?q-`)9jJ2?w$(4{fg1;i@uKh$Tf!WobBCW)TY#uV z0ti8^ruL;(Kkcg&(>KmmliIA*;qq5h2Wsp*gfb<-bgEUF<69yUTV@{Q>m8a~89Rhn z2n(Rc-skeB1Rud@JWQOC`(6~NQ=!9X#_~t~jcyCmW56{Mhf3~_ne;Xy>u6Imbj~FY zZ6J1G<=%tW@8p_@8EJzZTE^0ps-0QDu)z3O#xQ!HL;OrFw>eG>7`7x#Zp?z91^L5u zdM_?cRZB`UkmXpXf`zYG0s`y^dtraSxK0Tz)XvHct*`4N@{)4!HtyZ$kO&CD%`Nbr z-p4L?-+x_*?-PUxiQmORzGT!78rCUFSp7VhbzbBy*zd%%@?FN@Yl^FdV1%xNI7=SI z6vy*MIh2L(43lB8a?9(wvryPXryKp`LX`EU}B zpi&7o92L?;PftK*Jj$ZQL^KfW^nh6X_po=~z#Jo{)xK6^mRhbH63AtnLX#W6Bqsq# zX>n9~28me2zwQ0HEIcvWI5Sg4)FPXCvg}?t=gBg#B-mG@7QZ`fSL{&x7_hY0eF>Kd zf2EbD?|=SSIj*zTp%#Wf7UVsfN})O-?Bdr6V;Ul~(0qZbdULV^;00-RM%FWRw~smF zb8LC{oXc$`X*%Rm9bkE=N4M%=a`m@NgCNK9C^+^|zwlQyV3+2=ScO zYcT>h$rK6Q6iMsia6sq&}<^Gi)IjsKPvUW+hM`oDf)NVNuB6Af88Eb z2lURZsiXSv@X*u&ArY0DDY5d}zcs#oB14ga9JbGkhqqN)p*H>+jy6Xkh@Z__QpxC; zF9$B4zvAMmcU$7u($_s;38Pi9I0di-ZF^}tf1A1y)}k^_xFQ^Vx5DFsxJXUalifQ1aDf4sUwtlpbc0Jthd| zcRCR@QK^#n2|2gYFN%wTa5h!`b-HOi()?$>_iGZx1njOT2w;e>D>aR6fqeH!pLa*s zr_lP*GQImNTmSE!!&!&3kBDNdx+b`ixTV*%x$g=fs*VSY11%uq@m>*+U6yZuI2Yc; z$eG3?FkcT!zorBo{##yIlyN@;4N%zmC>A&c634d|ZA&dACnD}Yl)@ZX6KRR#8|P_g zCuWG9CjkxidhaDSfX1la^~3{RtS%`5KLul;#+sZ1%!;ZUs~&n_mb_Yd%FYyJx~+5H z8bR$+NeBTY$lvy4JeXLOwdZ5E$hz2Ek3ci1jkxiHj z=$OgH!rLIk2#WXU1Q|B>l<&%PZR5gTBL`c25}1Ekq}moN9@_|_HzOLRrsukEY+8UxD0c66 zJaZsQ=WPNyy9`G&+*$MTFAEB%JFnAfk;kUnCI)DCU+Zha1iuP%fjA32M#0S2gH zh4_))-`{LwzCwNcJbCaCNI#)ZQ>9MqE!DlylJ_(MyN?a>Zx4+UJg%5)EL2C{7h#g^ zRYrab%8-SP8$!&qqt%Fh5rYw!Vc|J7yT9q%uYH;}&j*aVn!e8qA1_5-%e+W23C12x zND~u4VBOGS4Yre=>sCh(-};6z-^wv<<~OONh;V?Dqf$g+*O`p4VCl>*y}?Rfxt?obf%8cs`hRN$1zo15^NnQ}BIuRQ zWI>J+UtYBkx=`V00g=-`KYxlnFU@480h(`hPNIF~+ylYW#oNj2OhX;+VfaR2t0+udgG z(~WFoYPhS=dH-0E+2D!JjWg#b%mYE-ZI>kYyI5Qjv?=BsS{R?J$t~@<1u@TH(sD|_ojVd3o&AqK-86y); zFjPSyCMIMZ2Pc%mRg83Q1Udl*caU`O)_Jkn>tQ5qsLIPGIZhre6&b3aqRtn3p&{9O zMCuK$&guOXKYUz&U9g1&9px-~d^-=wdZ?uYk}u*G01A>O>VL#(LVjsLRp6SIg7mbc zNeSNY0Y}hx8aT8Vv#6~OE*|(!Sw7E=??0l0tZ5`4QkDZFhFAhrfi9F9$L;Cu)Gi1U zg0|}@2j$^f>-taX=+|--svQf@KsPqV`0myuxYi}Dd!X~GBx)SWz0*jb02lgu+l_yy zs}A{y8XG~``8Z<1h4l4!&}6| z?Ax>R)^~H`c^pCH>nWp-L<&%}AJq7s+A_?OW4)@%ihu8leXSk;jsV6AC7Ip1 zjr|a!Re~@hJqr4W7E>#xcT68$qYq%kc$P%+mQWLcgrk?eOeD?u9x-7!06ntZYL&_d z_BUc)d@N+jJ7vEEQxdVc+0Ln!N` zt9N5G_fCO=VENFO%%C}IXozVZxG(s45?iLJ-jA zJ;fo1`_ruc{aQFxgkTONl|RXw-|=ccGV9@ys6{55M@zzNvF-k`g>l|99joVCDMr+q zsUSof@eKN9+`D+wDOw&i9VGx9wBcM)bLW*TxD%YcW3n}Z%6DjHjF!v;&P=Ulr^O`Uak zDPL33G~+$(q6B>$I3i5?bIU5HGCn=>%i^f{V^_hsF&c(BXz|fD_4WHc_Ju)By*scK zfT(baLv8+P>5P%7tl1ns2t%1Vm^dGMU)NA;odp@in90Q~9CEI6vD;uiB*}dQB8o|x za=f6*ZNZ_+MWN?0KwngC35B`~jI;@y!Ec%Jrarv2p-@{_qBTT4 zyeco>tKJWw{VXVQ`e3RE2t8u73@!y=7>(NL@Vpcqn&PPV2x@H#JA3+9H+l(dQebI< z>_FZ3N+ckpV?bHXhsEP)J`fm!!5AVT)$Zu!miUIVJjO7v3AUz@;a{WQhSYLe({b{i zdRGh3ZJb9+9vt>AIr<4l5M4&2Pz7|lxz^a3hTDfX0T}21@`sd>txx65Mg&TYH123M zP9lW`WioM48-F_i=;L5R`;fPU#L<$uu<`*DHNv8S%jwWA1*PLW+4EOhu#&+O_6`D1 zGHhj!nPswrr|OMvfrZid^C0?iZu#%mLrcZP6~RM7JOi z_H(*=764nj_x%nf+r7^~22et?=7Ii`kzw-|DErh)+zr;s=%Cr_e9`gJ=&v5s*!BCz z_9*EvzRrX)t!P>xPpw@wb{||?vUn~|w||qF;4{QX4Nmm2)s?1O zW{LcG6fG|o!@+SF&C^F;YNRG6pl76==+gA{#$2$m^4mKqzFma@KXmBG#JlEBT z!bRyRk3Zc1MX7<>&csD2UrlTQF!JoYY(n76<}a-*DhPk5Hob>16Fj~L^Isk;sdX4I z1%u_|DU`o16GZFY0uT{gudR>kOFawWqTy3hQ(1m(osgwd-!=y%R1EUgm}))u*J7)A zMb6RD9<7WNn3W)kZRtfp-te;8Vy#&ec}o}s0e1Sy4=b5O^dh*vd06@CQs26jTS@3B zK4wujW&hI}0W~HCG`#Z@Vq>{jDjRL@I}wnx#=}EJtX(+^U+Z(c{G3!JS^yv%iIVkyO|*0|G6>e#3Wby~CA z3%+#4KcbXU%pUH1siW@Oj!g+_>5QHH1j+e;G@uprhfrW?k%$&!)31jL)$Ml*U=1w z;`w__M#Z_{^5S(0`UnH>fgmpmG_?$%9aeVJ(4k^w<;H>;_h~X#_9I+>ZARQ@)$9GU zv+!T}SM5|`K*P0K4;-Z zs3^`e@Lg;m2=dEjXuBL(`yl`&GV2nCdV&l1kt~FTICGHbvw<(n7s%HGx@M?-?*o8U zd;%d0Zr#56h1K-WB0|j=p1O-oh!j|;NLmR8w2n|HgXKmZ01_NVo&$v)>hK1>Rp#hO zmRhrwh*fKc!lsQXe%sm7t}n$SiE))9LeQl}#G1#i_QwIxoIy zM-Korc0`yVm3yYHjVPH!z~^H;4rP9|1`-yN!C;La+xs#Y2yFR$i~`z_A*%@BSUYbg zKksrOmqjXK;koaezP9=I5ppwPfQRt?)$4Ldx3szH9x2TRBru$7KTl3&QKMdXm`Kin z+Jd3?bQDMHYVUw<5s-HRc{hyvXUzIUj=8P(lmSO(Ua_wq{AH*dI?uwvB^)2a2><1KeC=Tsqj3pz25k$Aqc7^9`HTvwOXIvsWTWwEPdI5q$1#iNvEfXu4 zu{9YY&+&}VT4+`ZY@|%rm6B3I_tOAJ(BS#0q4w?_)^srx5+AMg5IBx7rufrLEzaOj z((!2eLMXoGRGh%N7l0q~(0;iwMF(aITf}LdkGgj1A=q)44+BT7q-_`oGJM$`+I0L` zc6(ncx%MqRwQMq~>*#_JOiE`har~*rt={F4^`|GOAfs=MG4~WW*6Z2%LH^r>rCxYK zFmSd8>TXF_F4qrFx~@t+NUigOo9j^%z<$rVOz$&tVku1fp6drXi2S*=aI#Ig`*=@w zjOdx)$0&Md7dH}U5ZK{)sba}KeFzwa;mgK|c!*<{nFkLl7MBX=Z%Yru0EG3PDCUDh z5`p#nT@!B3xbT_*pEL5bQ>OQM@hXsh0v${ zpX3@@Ig2Aw9FDuP7UN0jR>gP(V52SJ3Y6P>3FI+dC@wXe=x;yq+f6KlY550(Q8QG8 z2NMP-gFa?iI8!4-e1QMav#m_W7owK44psWZJwdSADQX z18MVsiovajJcpo;r%3DabY>L4YYTuDy)4b*K{ack`zzj2nT`(r6a!c-ak6%~| z&hsQ}9vwufJX1+rJ*Uq-+zNqVAoB5$uIwOb+Aat6Jh>Pfp>3cn4R1}p{9&e@SV?l6 z^r>?DD-Z)oW!fxrK=?aZGyi{Z=7MBRGu<4r3cJx?>iRN;B1 z`}P8?{i(c}wi};tQ{(Z4eKoTz>wO{siF(HYO5B^^b9D9%Wnlm`OM%=xdU=9bdtut$ zMEE!L)IBJun?J^63S7nB5iJwg>_qE-2iNYB{^8pP2$8h-?)@IcRI04t$Ghr zm@s_O(c6;y?PL^76iX)BY<;ywTv=9bDt9x%F;RQN#;st7T>C;zF;od{z6L333*WHp zOA@mc&>J8<+tyB0`nKbbcgz2d$+#1AM8_!rZ+ddcS{RMLSDS}E=w)#|R9jV=f7(== znJzJ9@Z*K&C3hcg(vy}k7%wg5?)s)ssM5)c6Zd!}toPV@AoTWDct@eF1Bhpg}{%mA#*ICPCo@6P2p>2QHIp%;YDy}l+@*j?mYNEce)!akcZYI_A;_rX-M5~x zfpsnmcj>{#_Sz@H8z*tUJIj4W>zgwYV}}?e-}U2FDsH&v{{%r=@^3`|=m>p=305PG z;BmOzC*8h_3A5OPQfx@>muXX%Uk-h6)@UNeobhGx1& z$t%KDd}z7l&WHHm$+z{Sc*dcSsS}#U1;Z}{AU=7kCxaQ6<*Tcoi7tY2RiLV|$qTmIx`F(7MV#8CAyqH}F79wd#42_*M7HyD=hAJd>-vL4l@A zoGg`|g-s`JMm$NQS4G6p`@N-c-mp^9ewN?TMR130s)<|-(xaGAYQw`Oh6<6!X)r->nrj^yg!5`G`p^*Y)08LI!!ldicGIJHaTnk?k>W^ zvtg|_;1_2rIiAe^q5AZ82RFebp%K;Fptpf^Q&O@8sxR(U#gUAgKgK;kbULi}r$`nH zMNe5OJS2Wd3}DTpI%UA*7(NgHxUC%3v3wFw;W1A~N_|kD_9-Jcn9Zvpfq8&1$hvMb zfKNUiwZJ>D=e54SybuO&VA|vCR4EW-Ab1u1ik?(cm^Xr}mD4zWoR(uSQsDD-o{m_& z30)^ssGKyPHqV~I#;CWCIgp52JgJ~Se6pD3iIC1b0lHBC&d#_T_#bsxr2f>HCS-&G z|HL65PPn{teLacHuPA!x$u4x1!jMZiPi>-k_YSCPwbjH zFnL~H9L~(Q2Je~+_Kv=*f3cVf##LI{|9)}lW8m2A@aL)IQ{&1)-J(WY!DeXgQ(=d) z1Hle9M~R5i-Va_5u-4TdkU-pr1jI8jI`rwk?R-tW1*gBkS>uW@IxxIk{^8a2JaV7cnfxyi}&#J zB-9q@eSnY)XU4AB$B)=V@d34JLXW}oOEmgob#%1cXqr9Y6ITMhx&>>5seV=Kn>#RT zEJuX2`n}PLj8Dbrli&6g{In&j+CXO{liI^q;OTPwXkeuH`}50#wqI|7DxB-;OLJU= zFnarB&~)a5lf%(1d}IH3PI{L0!RHNSfB&|gHr;D8t#swPYld9)$<%V0<3v!w`cL!O zL$qgm%50nbeEIZfNi}0D3|2enBy6Ec6-m#sdS5YH6N|}8}Ec=HFcB>nXF4FgF8dZ+W3N+O1`5%2K=f%f-phm1& zH^i_iDf4q{@~&94279-Wegn*OIC4yw7uiVTxcezwVB?+Z?L$V;lr7eHy zUhZ(AXF6;VOk8(;i~U>JEzW|RsmxYUEgmt<4eIs3yb}1*oqv{XYw|Hc-tthOv1RoJ zD?!4xCNq3}W88fQ>FW=hM6X9Dr9pQ-VNXkxaHH?efxTfI8WUGdS+0rBT1Uq|m_yf= zNfahFIQHDZ?}QR|t^_~os2A!!3-T83FXL7%=TD3_fcq6R7SB z0Mn+o!A%LE(mXFQ)E)PG&|M5$;?$9SZERUOHQ{4T?`$5g=_T(ZG2?-x!0a6*Rto=> zWpBt`B|-hcn3f`n#h?CaoGPyh-cfX|{N}21bjHPh;OBLRm44H!hVT#a7ieT=wNnpT zpG2<;KW3xOSrq5`t|O~W^Ee*qWLwb_8Hf!rHz7wSK0;gNR>mS7VC8{IHnyM4oXMG} z)Tt>DVeOcPeaaGkdb35nR%U8FQ)U{{b4Z2)1P{z3$7p(y1<+Y>6(}TF+unog(z?Jo z(+T7sPrNhv_w|7`z~6)#V%2r-iwrsOv{#auRj7mVw6o)ZS$xsP7)?e|hmH(Orx>8x z@C3J_7m<;F7;xutyD^rjz>cN(b8HT1#=XZbmN|n|RI+i^cVDpIRNKBd3TW|g1heNY z3UOMy?T5={J}eHV^4l*MnHUS5_lKKagfFKbAi}0@v!VVnKfD7A$^Eygl~m<5S-Jqg zfgi|yDt+a@MA&(n3kAxBKw9+de`%2e+;~c##{LOnQzbo6*k?I(J{1T@O&Mq82|w$8 zjvDF3YQTH#1V!F;8DjOx$XkYp@SjC~{{0t)Zi*AIR1#qKR~lm)4Xd#C=p_j4}$XwHik^Ys&MT$sXT#=_n+X)v@S z)9U&daFsSempF?Z(WSFMTRTk4taLK}(kaNOQ?P}=md&x;^)A5&wKd_GZ4`$c2Y`{g z-P!FBwm#plmIizspZ^-wG9^uO9Ey`FtyDF5?bAje#LJJ7fSB!Xya>h?Yig?lSdfPjL?tyX6!dW zLH8_kxw>UQS+UIZ7E9)~W-p5>a#i@5Z{zDdk0GHQ7s@YQ!)%hLVqS9ueYKP`66uxO zZ)#A^GO~Oa`#Ot%#kvGA*?ySOL-j|sjco{m6ZWQ^OA=T=Eiaf8Wt#RQ-<=16tWq;J#r-(amgz${+G4k7nagl7*n&e!5lrPZzBWqRJk*+dX z90-A7?>N1|X`~)sggMdv#9qEz(jG{HjHR44|T+p?ol;dC7c zmO*4w3sxi_yi{f z+FH^W|F-6L!UCFp^!lso^(n=Z`iDLNEO-fttye*^$?>#h&-5P zIb;O0o(TEQ_0X0QW51}?m+%mY4+fqETjU?JmTxqZyunLO>Nf*^VQ103Q7i2uE|mmo z%SM5*HSXm~;KiYi>hn>V$Mu|Libb~6n6Py%&IF5v$WCB%#i}FUNEkw_^@f){M*?&z z=ScfqGAx4vIPJa-KN;Wup6--{3ZzK*;f>B~E<#bfn`+zTRE7 zR6r*8-^|UP5l`1)Z(c&8K-GVq(DnC|i68ACslZ_iR59m4;d8NCao3(k@14LE(puTt^+G$fxNtV7S$>E)q+_XbxSAl&C<~It0t!Wa-QK5i;PtD#lO2BsAq4K}pTld&+1C)JE)xB!TiL1w z3Osd?c#2S?&{X8iiMZ)wApkJAtzue7Jh|h4$EYO9NWqKck^G10>|%WaSbK>=UuB?2 z`w=967fUO-^#4YN=)VJflwMIC6jgu79a^^y9bfLQZ?TiSE;NQr)j2FteMN%Bg(Hl9F8?APBf z2TxafpW5krp*0GWPkkqDzBM?GF&_X}`l>aaeNO9nordGH-vPG`p0%@GU+5cLY%Gno zqJ>tTF0&Sw@a(zB03AZxkcKAnISDj$Ddo~5CMN2>;1&%_6v}HZ--{%NioDa3@~ZLi zM^O)pnuFEeRZ&D&!=qNNs_vg(d-eo5Z%cGde+)vH!aB>%EZ1|7vqjN;?M4GGN$1qP**4>jBhDa z#h6pno>t9uKzC}wQXm59@KXov3+H`Xj8lJRI1SVmJH;1DCuxX2eP&=>c_d>AX)WmS ztmI76@?4G`LU9W;G7r#Pqi zaoYXB2HTMaLd1p!&9AH6ucPfHswqwb{sE9J;l{nEX#=?lz5P$?$|s8%$BQ*gx1t_r zP8s&MP3h=KpwE7LeL)|P(&uY&HD)9`dwCxk`~A&WI2T$tU$whTTR3}qz0-6(iQ7RI z_?<9Z?M(SBCpDgwHtYmRo;`0mKWAw4rWkpe^)%D`uz*eS>iNR;lf6qTyYohdi|=ut z(GRa*SY2{8ohcby4;UQJ^ZIR*U3a@>i63p*NDQ1OLYBJxmi8o$YE&hGpX-0B7y7Vv zbX9lR?(G}AH3Lxc16KzKISpq$f8kBT_Z!%vB2`%hde+K2VetA%Z(v9sV1mC-qf$~r zA06S36W$YxqWFxKKU%I+YGg`H=#@#ah1J`xmn*|6CSWdLO?(3QZ?XSLJ^H zHqsc*udgBv+!vDkj(9JYhI|isubeM(4bG3Q|72bDU0)^`>>o*K;aS=UE)SCf8Mao7CO%UM?`*CzG*Q+@|pSIc%6f3@oOto+tKUk_0GjU-)8 zkzKFth!31k;>>@YH=P76TsJUOz=KS7sBX5hm=V)YENhG;^AgR6R2AKxLhicM?Ro-J zaUVL0-=Fx^E`chX|L6J?GknsA`0dv4x{JdpMzJUAyFk0`ec2mwzR6S?PZHTb-%}D^ zS=?_6n+HkZd*$^I#B{h~og8>y>3Y|oX|J#8jNxKh5)b74OKiS;MCY8h>4>4}P-#BN zWHVIaBCGL~M)FTf(}6L_pV$4ojvdzYC-VAl=g_0Q%W<9P1K;_9aq0ygG-G> z9He#OsR%9!EwUhoGyBwA4P1cFF1I)(Jw|Gf+0Q}HS1C65B zpk~8igX*skw*6Hvn(OY_*Vn&4`z{p>`JC-t{kgWH1!z`{JGxEVL4NBD;y0#Ro^=-f z+$Pmz?&vo-$fb@P!Pn_qSz>xj6Jt_D%s+HotL`R=c~kjofL@StOq$AO^&z07X@L`c z`qU-fkVCeFd11$N{(#+~aN^W-*}O*lA;B-rL>6IEC)#Hs>LSh{?k6Xs@m6(&K5oul zbf9L*vo1-`j$@Uoxy=d?D9Y1Vcc^wv9Sh$%?mRj+-N|e^vROEoWNTbZYy5SYWOwo1 z?o_4Afx&lUz-xlDuU6`7J}nd$FIMads{dX>8l1(AtgEaHYiw{XRWy#gcvuBLV8MX%KB#(djuUiAEg4jlC7Z-a~1+?*&E zfY_NcyLmmnu7C;8rYS)j;gib#k5>30X#}u+8I(;#{@Sxoo6m^vd!VJjE^|G6$t&WM z-zxCh23nFmeV^WA)psjHrQ|q!QRh$vm$Y$`muUGIO1)PyTn^SfxF8TNn#E!pCzxmS8`I-caXMjeF6nlO0cL)>Mgf41h5&eH z=HBZ<%UyqX#|y64?|{0=&O5A)KN!~f##nUPjX+uX+^Bl)xf+rKm(V*tK8_FN;t=2n z#(OmV81~_iOnve+u{*vQDyp(M?A_=?B;Aepm0W z&$1WX|Fj4-UE6t|mtL=xGGe=i8s^JO*6nKGu`l1O^ ztsG(1;p;#^iNDa3(2r$%Bu*vSJl;YHYM56`0e=R0)Vc^U9?6HT_!D@ zRob04yUm&dNO|HLub>O1TqLRY%5>%UDq$D#*L2Y!1B*CSJYI|d8e@;4zT`3*Rc z1%O2Nb0rKMN*LbT`9AlKIX0$utig7cj)Y0MEle}*-I>3D8z$bW-@(=-6l#ohC>Zxw zbILqEFVczRra{K+X{7BbVrnzB3N_cZqjD)J723YGOh3^F1EIo4_4n-AK}NHd3K{4} zT{WZ6%%@!vkgHw3hYVNK@2}b#Y{PCM^e!NkJG>#eeIUeucF8xAnN>RNOj;?!L z7uD*o&M!n~IliB|Vlsm} zOqk@zHh#;&a-Xg@PORKEybISPH}Z%RTNS8+Q6@zq)C0r{a%O%j0m2ktD=n;E#9*UB zTdMJ?)n4iL_o`MGx+xb{wq8_A&f?rWxPy-u^L_cIur`SXuqG~_?Vt0eW->_0hCJ^- zsIMD5|2{sF0LlKM_n5NW)Uv5+FP|a7^#Zc$ct1N=4yKS>0tax!uTw~%CGE*toT@-e zpm2hSl{H#^Bs$n8NpQ=g^95~JZgp8LeU<3rL7Od3AUClh11zlel^U|>h)}lsULALe((yEhu|l)bxt zbdO+f*UA?7s{g5$W!y%9$y?D-df#b|*Mw{a*rA5jb2=VRuepln_;B26tr(n+-h)}# z9y`ej=>+ZC!g;|k5Igu05m{p{)dV$P8YKItY?e68U-E zqyR-FK0u7UgD-!-hI6-+I}1#?Qq9i(D_mRLt6TEW@MsZ^*|#O*^>{7QA-0&prnKsnBZ?a_Mysi|D2P^g1CPFjaLHX+#B z=Ar+&32O)*s0Pi~K&HAo_Ub$%C8)~G`j_Lf1L07S%FBk^*_KodvwImfRYRnq+l0yI z*}0Y=e~_myPF?a%0d1kidsP+MKbNyMP~B=;gqoFFsRtRIT?k&GW@|#_$S9IG`o90# zRoKIO1c2Jzx@qAnN>WfpgYA3Zq)M_-wL(voC}3Qkx~3nczXF0f zk*!ArK*%HX-)pjM{Dzq=sJrd{S-H6SF(cD3{b65NMg)vh*U;5>dFX_TMG~X)5G$Zt z_N8brH~nuXj2o&_u5WtX0!35 zHt*QE*6I~th}Vy60ejazp}aY9qKFqhZ_D(G$lBB<9-%nlTSeRn7Wc4l`8^kWH&Wo@ zkfmD+9c=&N_eXlK8UM>O^}@;HS`!dxv3CCOP_IDaP8$lUiRmWJyx;NS&-hOOkcJ?D zz3X>N&z~0-Y}yO=ytK#h(-Hha3jsRcNL;=~L@Qsu#@j9{hJuybi8;v#aX&3zwXXQwZ*3tpn z)1muKZkh`S4BY9BPmc2+X|r9%%xQ%pk~I&nxsoZP{_q?^5MWU}swE|jKWBv>%lh&@ z3F=IySQ1PNH8oq%E1)mYBqNf}fSZ;6I1+8#n*{XuMDINj%n?}EDL&3$L=||kItl>Y zwEbT)18qcXrmR4dhMHg=7reITcYS{H-DmRorVPW`@Xg8H1+w9*|7^%Ow0 zz2MvzN}ygjs)2OKG}&b24ME<;;cAr3BBjB2?a55a;#Di2t35qLr1ahj1>)2p6Jk#Q zNdrJst+r$xP#zNjd&BbJtSS-Q%P31+e2th^k`#zBFy0X9uvpISOh#ueeznoa_c6iI zon{nuV+mS-ow^7GKq!WiAhn_YN77k_MfpB$cxePA1Z9yHDd~_7L8%3a1u1C+1f)9` zX-VmBmiW;n-3`)6cP-tu93II@7ar{-?fhOr)+g<{TUL*~ zwyKuy6d>05ukm0VM4O4uBb^*xphdvQnBD&iT6HKx>qPmdwnzq0rVh8WvJMdEX$$|a1TY+1s;yeeY-2L;prY^lnhjC3HN#p7x(4WdYRpXcW#b64NF zdMA+&l>ug$uNva`eYr00+s^JRoZbvZl$K>^UnvpoTA;ZSk&28#-m!&GxtvCS$0qV2 zr1Rx6`D?~nz>lw_D8?!1=4gLffQaK{aB*dfgGw@=c)3_)Pwn+0I2`4PI>9TiWO?*# z>EW-HpbdV;Sbo4%+6&)j5-q(Vk;6l3_=jaj8D-Qn(_LU9dl?wT(>HQ^dUPO!v>@#N zcxl?a3%95R7#j8}=Ldu>r95;u3tz28gp(06^x1tc1S%y>E8i&w12r^5vM;-%xI_%K zS?u%w)Bk10in;Wdlhe|?962?~@HMdRfqU@<;>}oQ)s_g|jc1tvZUxEbIEZTL*Q+LJ zl7CDWv`v&kE1K z`D`^V29XZ5pE}%%YsRGk-l)3AvQaRN520aL`TD27%>VXe%*?1Q zc=10gP?-Mxh`qd;gQG#NVIC|gj&onHRZ>HSH_;6rM&eS2g-vk}(jy?2Ik9{_+oDuf zoP;0zdr8rtAzqR3Ps3OL{B9YyOy5=Uybn8SiF|4!iT(xRjHtna?tdNTeyO0TtyGC# zId}I-;?7b9={Hob>&>cmX+MD^b}_mL%+<2sA$b2usrmhTHuJs1;-wJ#r)}pco2BRL zTI&7+h63>SQUyeTy|^v3t}KcTb_>H*63NdmZB6m~u_ZU$Os2zq%ZV_*^e|97mAnP& zjDDGrU>?PS1`W0aqV+dB>GCPh338x<+_#9PcG<`mi`2P(5>$+C$tTU0R6c|%zMXB) zQ1hXe@>!}O1~m%g;gB4v*f&^)JqomSD{}2QqNZITNB%@F7GEUa30@Up#IgR_s-@8F!$k@+Fvt(-lHi}i%-mLUEk zBRZf`_BVJXJp|?bVzXa1O%R)X?{L=JD|dhNWUKsn)Yx(~Jgk5U3hW%~FaoR=>}8kg z(}CD%CGSBgEM9-Fxxb^WSnFV+p`Zi712E!B0*22SC-aiE_vB-f-`q?qMMPg7rew^W z7z5x0E#b}d8+mgrV@0)ri()%Z<#flhx-JLhw?g4xgkGE|cYRC>Q%ir>-mYk76p&~z zP)=P@@X7bZGrCvta?0@$DTbuzv`k;j3;(7HvwucJe8iT5GkikN`42%gUR43*;t0f0 z0-RoOKi&f~=jG8zVjl4P)(!^)d%YXp;R1aaiHOacjDEW zK(PugSPI_uS3w})`zs7to^(@tfq}I;aK!0bHviv7`O|_l=*;x#gB?CnQKUDlct8*O zLKO1Lo>J`{#%8pB$YHS&-BbPX^68%XLEztrKDhb9- z|6bDOI@+6f!s3D41UyZyC*wMvL;S6nAk;dycJ>Fr+pM)xUJgww`UgNq>Mfl9K$C#h z8BvFeee&euioo#sdDl5${RIQVn-~c|{9Tlr;>X-8S(Ub|Ystbz2Puo}dfhifPc@@Y&*00p^H(Ht!hgXFmBMQ#ED`?2k<;W680n zF5-Zwa33Q+rwNEgD^UP*;PMFBl2l(`B63CEx*rONZ)^JZDg4cw6^oWxz?IeAc(~~_ zUksx8MEG$9GjoBnkITDz(zJcyx2d@nLerKGr`v(e|5b;qb?&GyO9e6@N>W~Mi6EQg z1HuM(& zb4fB~;kSnoMk-H(sZqtTSqwGteeDTjnb~OlkFpnJ8iAH9GkXAu#I#AJx;_x()odqy_DyfC1P{0!|VcTFeE(mWX`;A@Y9VQ)Ah)m5cG6cRU>Cxi>I> znBzl)k{7iH%BANn#tiN+1YK{w%CA6^bt&)7mc;E!U9d68wor{FPH)cNOl<_TH!+%vWc_!=!dV?*Ky0c5J3T@qAsSZ&o3IBpI ztL!QdFga0fac`-(AC*ag>+tCzi6wu`BaGPs#>g(=|+oXK)Gjn4; zM&&e7r(J&UP4TAFal~^@K#$Hlx_h7&ay;LDf932a6T!%Fe{gjBGxBbXx|E&c{xAbZ z0ip@GK5D(l5Wh5o%~{o2T=WPpok>w&&oA#F9}Wavo@8PJpjv8CwN00A;oD8M;wTGm z@Drih)M5yce3it+-HzMOqq33RM?dnZz4u-XTXeT>((c*f9jY*}rO<)0Lu6n;PtDn{ zGkn(XV%A5fWqr19wa0bXBn#8Ba|tSb2$#}1R$yXxWB){JlJz#BQuz&5h_T9ddRh>O z_!DIqS!U@Qd`1Q0Ak`qw1S9E0nY7VoMcz)O&~((U_P-_`UfC|Z#tPcUngcv1+X^8G` z%oo2>(tC)40s5zjsr5sMuGMB#D-w3kEy;x4aZ=lQD&Km|_&9Qe2D+os{HlvwB)Grq zsIL-7R*AQIAS?l8Z2sx7TF;Gz_Dj6DT9e>7bP4oBQNnE4KC4Rrwwxx*>&eo_kpR=u z{?QPYthhJ!m9WVBFf zQ(LW6Do$I?jw#1~VT9J71ju@(m9X<4Q|n z*R?Thkn+0(4lCMNy%zV?TJQVmQxV~tHZ<{z)<<&VzmJIC$A>}-G@W8~vX#zUFL zz6DF9=km1Y;?=|PvXhR>-9f(CjS8rZ;I1#y>w@2VB8K`gG~*T_j@(zx(AC#Xqu>yE zBERA;w!#ETZv7|UX=Ll-jXYN$ancxej!(EDx6HAIewza~YY`jp6b{0DM|%=Oy*yma zb0%N-bm$DAp|RhhegNSu6yF9gbPPBax|NH0$qrLUM%UwiBBXN^SF%%w4C!qelu$`hRyjFvwbOqp=Zdajia+>;wNY@?8DY4H1xV)=hX-X=Gc zqO;T19w}Z?%2*D?g~9b?LwWo*6kKfo@N52Bzj)iIMuPUkn~QV~hs$_;OyW1y zFujxS=Qu--FGTkn<#$eTM!p{}`Gy<^%L{hg8m;Do?CIz^y_U_3va^3>kCys6yTcaW ztL1L%Uub_bnhfDrbZH8~)p;(WTq?EcKFe_%r;SY4&(b52^5ERh)w1N=eme`w{K-}F zcJG@T2O(lPQvJ#GwV z@?}^u9119potX_ePI#S6ub&6ytR9cQDyfNmA^CwB4HO1bv*GI%FPYi{`HUIZUX=cM z=#zg~e%w=f7;n8nBGSEg>+=DAt@uqB@$!k6xGmB)=qc}c66{3_5`c$sTCpZU(VCU@ zf`4jVj{(;T0F$Jg;O!SR-Uu5v&DY5ir(1Lj$F87Y%wIo`yIfJrZ%uxCz&j0)8*8}MEe)2KMC#1f&E*ex_e_!XON4a6 zH0RjqICoy~V-yVP59oAtRVbZVbTUBn-I*wBIX&cXJ8~WDeb`reh|IXvn!fP9C3paX zR?XA|1bHYt0 zeU>^-O?oc5oKRorr{MWnFyLS`Xj0Su4k&gI3G8~lib;9DF&2`c4Bf=S>Aa%2N<>`F z*RC$2%YX*3Akf*lrB$+@tj@ z!le~UQa~@~u5VpQr@83qvQa;-5L|jRA3d~FKZKOs-?(^>CEOj@+%)jYO{vEdP}MuX zVgLfXa0Vb0cyw`wlFA$)_!LE}u6K6}7Kfw*UO+LlhW%(nLnC5bp`lkXai6`&J z(fWVZ8o7Z{69liIZND$8T-g>~>uYZY+~-C@4Y{bk0Oa|t@2eJ=i6ZQ-1M7-moc4pI z8g}h|qhskw=z1%QJSZRaNcOb6?Gr_6ri8~1tE-IK0w%S|b6l*{%=TjWnv1s)-D|cR z1`EfjNi?qHZ*81JL2V^ic=~snN4H%vk9SEU-doh($3Z&NgJ*b8+3xbg12tHDX=CY~*rX(RnMiBp9q-P?Hhh6qWKQIeE?+B>PIYv9=jf}2&PSw4d3G&R{nVB8!BOzSY6ponxLVlCEutuH5jJ`+11sB+wK8! z9e^W$GgHpz9Z~<2Unh>roqyDX%M$`($!#f zxET**5KM!P-_FjU+9FJfVUv;Kwzr@bGq$6Bk*tJlUcOztYJI$~txub}UwIOZ6`N}w zcFP`iaj9Y7WSjxF=rFKE$OX{0_}op5t!gG32y4|(4|k@1YA3j9HC__NEqsDsGMJFG{>O@?^9Eter`gR*J7rfB7b1&&!lXAR+R=nIIVfogv&<#LDB~l+kLo*ZQ2KPHP8cIJ{o5r>KFgQpwTrP+9_xdqNE5-ZPcn;t@1t$g$k90x~sN&~n zj#}+O!b1YRL`rTphUF&~jb5eG`cx41xX~@eLSaOB8^LK^%GxzO-KUjb6UB&VPiE{XLhil4Sd$Y@*rq$GbrVv zw(;=W&Vk-i6P9y|Gt zIDiX8TkxAmx<-02sXz~RAdFk`FmnVLrnimi(Slix>XZE$0d|$#uezt=R|4I2wy`-7 zu-ij~7@O`p4ezIy4ZM`n#95>b{?Wn~U;JW$MUD~c-4DlVHph}h9`2ml5fk3&M@|#g zzb0RU+TucRy)ljPfAEI)l>ITvM_u1f1Ra7<{ne%DB8y65R# zXi?|zZh>m>qBdm(V^-qs1eXPP%317$^eW1y z59YWqeJ*$pNTv7(6`kCV(o^ttYH>jHKpPHtGizrlCy6SGEyuMZ*T!hb)AmZ!ssHF{ z3OfDQEb=Z|CH>H8!Z~qVXgz=n4dB1MmYg92#!(r}-i7xOc#X`@Z`lo3y(M>AUxh5_ zzqU&KL7*NerL$n}_S-rw8Y1`7<>v;DBnXdnBw)-D{0~)p;g(ljD?M`jha0B|X+C3v zAL-ZTZbOkix2(g6m$+XwWVK|U$xSvFUPlXZ&62cI9F&{UHh8yYmV-w#SLG6>_-)MuU>qu!=%tS14hQ)X zp2aupA+h968l{>uI4aomaU1sKVn3)#N5r{}W82Y>+}a4uX2+S_N*5nD#s_AX3iL^m zVs{U9geqS?Bd!0f#Ozc#aqGJyhcZg*bj3CT@ctT#o`jN*<|XxnC35kN=nHTqvW{ZdL z-`t_ zy&R^*&;gKQ1jTY2M$$1=0ps^l+c=gq+ z0l1a*PJAh43bR?JFwG^ft?2flUO{Q z^g$2j=OgOX6Xy?jV6oF)nCtb-iH+<2N#uP$wK(!uBy3j0``?qX7^pmf1V&iC9SJFO z`mfeAyXJVHCqN}C@9uDz1-2JEWf!#>jo==UQo+scyFldR9UZQ5Z}Tb!S=zZ|EsM?X z>21G!mT;#h-%+It1;&&&Lh?~TgldwXlzCqYMO1lU9dw@r@OsqG!zvbask%FT2R+<@ zPqxvh-9DG7bcd8tY`uXr3!c2xzjZ1_SQVGt9|n2fdOQqcJ%gWoKTmm_=71px?tW9F zuLng2c0Q?5)%<+BLt6=6n;K+R>_cMiiwe0#&bXky@pC%ZITnzisq$384vJcP>=t?Q z6}@j%GZfbA#!|1dIUY~lbKDAbo0#Uv$(N7$mz1#Gj)x*W_mv)9N%hfz!f=OvwdXab!uq(DSOqn0-u1foj?e5fEsFHVH z7*dhmJhZ^Q1lhh&k@{V<-xP+&O%&y{f7c+Wo!axVw!46JxE zSw2A%->4fl%ea#VwRKI#BCtYn5hx(i;f$2zq-GMpWGi-ppiW9nM=lXO^zAR)>EF#} zSb5cgd?EokQq7{~=pl=mJ#H|=>y*Py#dCw;ab>V|w+L`li*vu{IYYqiWD<^O2f6(T z9RSqo(XE>^>@$8ftL6Vz)u>|A&Mgji^bGWe~m`08@m^-3Pb$+Nt@2ZtJ}twN*2NfdMaemlT7m*|f>~P54-aNZm^u7Kfp= zHA}-)Y1CNc-6!KWH#^QpXC=eP2U|_sECBfCOx|%l<3~Uwxpdz5wMPOKv~pZwj2KQu z0qF?19FC6VBv7#i)Y(2PtlM}9OKwN?W(atm&rG=t3hE2ET>h$UMI%0K>$GXGeV_p7 z3%uJG)l)XG>BiO@@|KfRQE*9nx6~yz?X6-t>z;F^YIL!%RI-^FU%aB@E5(Zhj`OLr zTaU$#3v7D+wbH5ob~LgtwhI!&J1B=a1@{SrNgMpma@yBy>%?Ak--{9NB)j$mk<3KkQ(YyyV0H2PvFbCSd8y>JyRimm$s||bkFo$ zrxTj4#*N3{{v?b*kh--z=G8XAmk+G%YS&?aAs%MZL}{$ReBW~gTS>}KcRwYj&H!|g z-o_;dBr&b;*4lHDlJ}CA-Wfndjs#xL=KR=!Q_q}b!P)v_N5B&3M7}FELYSsH<{tnIzZz7wwEDGv zBc*v;!~_H(_fxnI<}J%a%YO1PL-2>pK*Od~ltarXwaENV#_R5qV%aXQ<^@lsTyuNo z=VKd_ytEs$Bcfjl)pDRa zpU%R+m{Z*T5vOj^DO^Iyga&*EGGObZ zBz6|Mu|GCd5FQ|f#t}ksw2t2lMeA%p%uFf_V2{B}6h*B1?SyMR6G@o@YAy*9qBZN=bod41KgmL;Z z)5pFT04%M7_-BxG4LiR;L~J54mp_qDCitXDziZEY{Bw*1gwrq}W{Wu?VBDg&mrP_! zc(NT^-$5^k3lr3dp%kjNOFq;Wt<#eRCYltzdNey~>3lml+_943dRDSf?{WHf_=jG= zmk+2~FQP%@KiceF70N5*Fwx>GxL%DJh>@o-eb5B^p`@_B{otbvqK${H$9_+$D!zEs<1^@6w!=mI+}M8(Lg(J#i#w&`O+qUP_l4V?-A*3 zUpy~7O^yd{HBQhvluJi&u(NvJT2LWFCMS zoQnkStR=A9`q$;aXKwf8%jXHy*TmSXEofLa-gEpdn^x5KEBbf4jj#$e_ST&e{Xj17 z`Q_Vz$j5&&;@9~wol>>s>#15)kbp7lq4(%iuJik6G8F$lrmq%tJ12k)P~T#!-nUi}pigL|*ea| z=B06%djs22zXhs=<^)9Q%_#M2Xosrj_nj(vjkRdcieiK)BpD$Wi~mkIW^cFm@I_t- zO9;Nxy%MR7HFFbVdZomgal6pHmE=6GVB^5^0wB<;;SBsFb!{Qlpex&#qHEExZ~kwu zCxTpcpJ zLYZDQ`**Xc4ZJG4;ym8W0gaLB7r94KqrRA^o$KFF9r2kJ=P148OR~qkDpE0fyVV~q zoZ_-|m4mp`Cp;eHHn7|I+OwDQB~w_B=aG-V1k}27OF_-O_mvCGrOLZn4B4;LvSmVG zGf>EC**p6Z(~@yhifdX|Y$?^N#clPQgZh78hj3}%Qqe~cf07^x43fF>ykdE157sZO zqtzMgKnH<*l!mEqj{_LRolclIF#+CFULB`)*$E#zI_M1v#_HokrAzV2OaA>wiu>a- zaU^Lf?$0tFFH#8Wc?vTJ!vI}O$(SxY>Nyskxi}?jUfS8>){e7{d1hA+VUi=<0XE@LS7zg*S0hkzT9Q5tYOWkizPPE}x$?zCPx)i2wE-kn|g%JszA!R)sz`rb!u;*ioZ~{t;TKB&U zLd(pWDxQa`83K+Q4{20SR`cN}?q5v9{UI?T5@xu$;aNr>*C0jo1)ES>f;lAH^=+w` znA5T!okx#vhzv#f)^`+8c`f&Y^znDuW=2w_G)uN~+cGeiKDU0J!K?>V=+qNBF{DLG zEa2=qGIQRe_)ptN2fUX?wW z_>Fxt3Fjwj~2n$41IhbY1PXzYRXz@S4D3Sa5w2zO+_o%AHUdW>dV03J@<9 zIG;Cr}5)^skE;q$$)O%=>Yo$MXDh-e~IT?|51nLQG#`M0c42ET8TNz zYCIn319^Pqu#Ud*rc#|&vPw`XP*E8a8{B#3Ujg;8y&h*_Z2jPy8IKu$7{cY?oDh&X zoTGuvB`xYDQqpWw@=s;Q&7#Qa?yE0!w32!h{)PfyC_VO;5{}p+m$yJ~c8}P@y~obc zo{fwEt2MYnNLn`0iYyUmtyz)=(8-D5LH zPl_5$Xjh21Z6uE=OL054^3bcevCI3E)wOS6!fx&qnmgi9>OYACV9p#Xh?VmX1ESbk%F)o#|~>tEtwXNAV0|Fk!VPhgFq+2#47xG*8N;_!8%SH_^tq$(fxjTWDh^u zXji%SVxhS#qx6tX7m?!SQ9aJOUfo-1GNmv38+ZC8C+h`PFXpTct$P?x$~K@Im@LWx z%F6-BSV4S;I!)B8+Ud{Ng=`se|KIKr#I8})Emn^#)CKywUY_(vY;O<5g)V-XB@90g z@wY3jnbJlB0Tf<3Nm=^f$gq9y?(3{+1*)VFy@j``&Yiaw6G6KZRcR)0we;LtJg~J` zzzDkhrUdQWf`|YkhP~WM-<3j9Bl3aQ)F15a<;Fw~HRumUA9Et-r@S*j` zV=3)1&Waa0ssP*tpJR$;yGztFrcfopu7NI&dm9o3s{J! zTmedBRmo+3yfDJU%c*vA97v_Ku0((?=KNw>C)2?zwNH(C?wf`?UvRar-Y4fMpEZCMKf zCow>uCPNFZW@dgOQb+@0eeF=J8x~O5uOj#gnJ5&P-f-Xrl19UfR%fbM5K}F;6cWKS zsbRa3k;}EUBA6(8qMd5mmL=i!PWl$`i}B2o*tw%{d>%59qgV4k1rVuc-VWvVrAh2t zj&k)@XQ6pz+dl>0g9)`xNVvP)-TXQ~%8m?NyIf9{{sxwquxOD{G?} zpz@!KijRaACy?i0BuGUOR7wn=%`4qeT1%hmaD+Yev;V*-`5&g`?PtGI$maTziGQ9j zy!w>_duE*a%)4S9=3Mx1^+(BO18`ojS~fntbk=h5!!GK>jDwU(c}>6h%fiq>pZe?; zwRHqFQyO-&aPQ-In!A?;Z#2#hOnT}J6rL6O1Faw{5bxZF{yKGtd^cGB%#r>eeTUt< z`=OV*_^8D*y%}Dip|~_Xgk_!#stY8)?O`fWav_!O9B@^}0D(cKF`ZRI*P@o$s|5;UFeF3WRHo4_shtb%a#p1AD#Q_Woo`NYr5yj_?ep- zV70tsewzScW(zveXvekUMX-+8qlHa{$kWDZj=USZc!Ime3Z(CaQ@^H!mD{B-HVKyS z4uW@#{&b{}YM;pJP!}$4AbS!wtzE>1UghpM4Chw%|Bt4S_LS29;qG}W<($>(DfZR8 zDwnX9J>eOVBL!kJCWwl19`^6!NcFIn=>xLcx@vx(iM_g^lBxg&6AZ6jgMPyU(GFtm_)w`qd^7jzYA}%I%0{aWoNHRAHmRxq6&Z!2`N+d#c`4Jw2MH3!y^?(iCcqV#!(i?R~UQEvZIF&Q0t4}WJ zA4J}}SKR~FQqSK6TdGU(+#2>D!$}E0rLSY?DU&gaUyek$ez9>{q|NzUnMf}`yFeO0 zN80Guo;hJ7PhU1KoYe2z|Fz`c2caYO1pxSNqSLP#DJ#wKpTq=Pxw6YQda{3)Br*~gYq`_JfD@nn4W*qiK~Y?&62{9EG>Has1@!QD zU&I;w2}e|Pwo?={!>2HeVfLpnB~4c$6=-u{^T}RzvD>b@XFb;qyR3$gg*Jx*)hl`S zyz77$*`o33+kb(8Bgg!b)H3WtHP=~l9LHZVeTH^FZ6Xk1gW_i>&^u>1A_-+wpR3aC z=Swt_|3w@Ght^CAXr5VUNaOYdE2Lky)pzF%Nw@&SigujB+@(us7IxNXXa{GQOm^m> zZiTMJpls<}73DTf>y~*5tv`Pqp|pi)!d6C_e(C`u`F~~}Y}m9E?7F({&l^nv4ztPP z(`vnWnRnf9kILJVhC61b?lSAXT%rQAVE|)&dn?7qp$7ZL)Krr`^)4-OqaJ}v@0lUW^T0&AM)n?$@p4>c)IQbtWwe%dul8V{s z?CN-Ue3hfGH=t-XbbHT?lS ztG+5y1uYB(Qi#w-xFkql#rA+e1KVaPh=@Pzxl%p2dDlY!ZG)|<$4jcO&Ie}4GDTji zv5Rle_!3H@FIK81as?s+#%_Fni_antR=IivSa9l>USj&{PE&IH>-F8;U9a++LcCIP zo@milO`0A=LsK{wlQs2Vks%#Xz^Kv9QJq!{@JRD+NOjrr(BdFfu+CH^Z-499IZDt8 z;k1XbQQ)eJzk|B%`jNiM(A}xIHE+lZH~Wf$x@|D+B0BAmPeo2GsMEarOn&5#dc>ci z+kuI)*Ym{;)*q$GRr;DV9DyUjEM>I-W_4s{{MRVofeS%;>nDZ?h}}~-3Iu4;z`nzM z0e`3@6NA6OzH)YT|5@s$;0RBW&D%6ekPoK>Kv8uRiF_f++zmKrX__WYbeQRLNm6lp zT;tz~{qS<#bNMOCQOdu@*+!UE^RM=O4;amUIDpVCeb!4Ih|}}^zyWzlmfK$+hpb8q zP@iK#*WXJZW6QnUxT`=JL0dxw<*EZMC1lr{T#vvAPWdIW6a@q>ablmj%lv1C+{I$1 zQVbd`VL7*?<7VKFUq{EUO-@{G9&u7MqWR$8<$c}Oci6uj@@&&+F6n@cG^~Wh00gq7 z4^Th5-x}+7sI$l;C!V$rBy)@z4$y^6z68!ko*|FeeejxjUj#Aqva*t>DpEMcjH*|v~5-@T@}gCB6BkvfMciM zB&z&jEz>EhB<-{Y#c#?zYaUjbF@!(U`3p=Vz$>{>aS zbi~L^+ZnE;jbA&mxYlg2I|(^~d0fynO4_{BRCNImlU}rs)BU&^dOvi5I&G>vYYfP) z=iUMh=08g68>I80RcV`^qP3|t<680FD@Y~pQpdZbxGE6Hd@`S%%8I~UD%~cNiwE%1 zLoX#GNB{K>9d8w&Sc>P5bv{0C_4K0qi7gq1-JYXzv1yDYZiLNJc zi$-vEwUpV&qhjV2} zsLHe78j~#8ZQTzBMWkR%xl=#!^Bfp*H8OVX+Op;52sZ8VU@5sTo&X4Yj+0Qanld+H&MIqbb*nEAyCA>4mkSp_ zWPnGkE5wl#MDCX$H5Mq%AWZN$HB$%3*CkZvU}D$O^~w808c(-(vwyJ|g{lx7(=ytR z24N`7gW+7Fy161)*h2pJBW1bKl5YY}sf@`QO=_n0@W_oI!0Wv@T5`Z~Tf^eZE=DLX zlakiWzAhY7%jr*puOogeDSpmCWgY9%ts(NGEkNlnhe(yqMmJ~XT(*#8XF8r+Bu${^ zFlYN@mO&yKok@r5bS9j4&|lLy5h5xcmg7#+{G<56HOIY44we}-gR{Cupp66juH^(!DftOl_qlTSZW~FEJLpUOi2` zV-U{4T%%>^DaO(=x|>G1P)_K}E&bZd@b~*Fk%izb&tT>^ay z-gA#44ZAjJdNG)>7Pskq~k1>6s;4gMJSO<%BL?^_$}+hPpyn9N4my& zB1wEZCb)OEWw9b1-v>)o$Jxvz{Ah`GT!AL7=X%#8@AbYXH4lGDd*6l)>;-hwGiO~o z3uL*uCo=)Uy6U2JRWDc-V%5IadaONH9y=lSUNE|zTMUee2dt?7_Ma1Vb{sE{v3ZF5QF^xKnv8Y^dyu}n;6maqgReCLsYyA8 z?P>8Tg#_;QqB?^{(uHeF6P^|z^gfk&_~!9L1j(WI>cdGWrhX~()@WVWMEijfg(TGA zBr%;7t^4l@aI;%^Quo15!J9c3)dhToazs5byk{5k8RQe{7dOI@zK{5|H0qs2i|vDw zm%xaa51mjEb@U430@3WyyM@9R)#RH?3l6x^qdrnX3W;^1P~If%rXCK7S9FkZxElJL zq_w$t_1a-GfQ^G`B)@BhsalrA#Y1pt!DeLgaq21Wt%5)=6g)W!_}*RI$NeK9vuZ$K z-Oqj-g;cV}L9E;=DAB4XwK(=zo$rBPP=OB%mX&3%%v@5&OXJceXV)^h^dPWWvN zDT9vP!o=;Z{}2V(Bvai+pWI|Y>sZe9MWJ4K{|*R5$oxaN*ld~|3Y2etM0;h;iZ+Z> zz2ygx=zuwa8Q^u=;eW85Fw;9o5bIKx!A(fBNp^uHdW|ILRw&2kGK1Q#?pF2v+7h7l zv=ops(We#*B5+nd8BjJ-54#E9X&K%iVX=1y9A-X3izMz&4b0van3Jg zE_(afMgcHu-XhXFeFK{|t1e2~((9J1vz!>EeUjkrqs{EsMVIbU9^(%%J}t2+_R4IY zZ9+-PN@V@8#&cPgG^f);$q|9qVbj2%#mnb&Na6&v>x))m1%2xdp*Hu zU8i}KuheJbQMFeRf^i;q3ksS?{xQqX2JS%GQ)cSi7vkAcTRCIHQxj#yS&L@m&v^v0tw5mL)ch zIi6NF&W4GY!_5(B5J7=pnY&~qIOCS^3A~p9rfv)O1jgb}KC*j|-f9Ww$3{%|=uTgJ z9O1Zy{{npI13VSvs4o`lbHpT9lHIp+0F7;u7)F^h9Aq8~DQybA`%RBIt18!bWs@Bw%QgdCexu%M| zz82`9p}7G-Kk2=BiUUEcBU{3Q@_x>=wW0AB-FEy)Lq*MMWxIx;XSNQXTH$ zxM^^a!MK@-O^jf|JTS3+o!Hq;A=Yu6*V>$0hC@%KWaxajUW1l5EDd*-8cmiNd$MS+V}2$-Kjw{u`T?< z`8{K|M(H8~b{wX~H3u%-IKaRIN>WZ7i$<0U78U^%Q1#^EtX|plpjOM}pRQUt`8cPE zVbn`Ab}5cdpVc3=%l*>{@pPdIuaGrxi(k~Pb{hiKtz)?f@-%JOtxguXYs%=WJZbIA z?C-Gv-`Cu3GOKe=+*pVSGsxgo77xb;i-zv&21Rw{V7{uFZfBeh5NLGoa*YWU6CXJHNTuF0#PbuzDyMtEMa?BmN%hD3*w>GW917Vn-_V&i?+%T?x7ip!VNQ)TUvVO z69-8jjUYNB;mzf%3F@`PUx(e5*fPxX@8=66tLHLhdEyDwO2gE(EwgB4%y~F&x&v+2 zI#-j!l`1^CZ3t>N_T3jZWJkeVznJO@f&^zT3^B3uSxDvjSLdGdYI4>F-6g~ zF#G2kgl!l*L+%`_-X95(VzwKVW#D0E_t~gpu0Wd%&O=eqJ{SzGqJr=xC$g?=OYlIN ze?yw#$5GmDlMcOJh`%%`Lm^6RHSFgxQ#4Q8+^W_7kEyc`is}#7_!7d>i-2@DNSD$q z2nZ-0;!*-q($Xm`-BO~2#3H$bbceKbBOxWt(jE8kyK`smKg=-047=xi+%^NJ_j{oJK#VZ;AE32NK?$0cj)h72bj`uZ=Z>G5)Og_yY zwtjt71|m5x5s&S3iW~u4ZGjx!NuHV%q3sHmYljinXQ}K=+hv&B1OAFQjjj+R!UMNK-D+ShL3r# zCMz&v0t@(^T3H=u6#q#-;bA$+Sn(5j2QMgyk${uQ`ot;Y2TgVSl(@N~ZT%S`4(7O) z0d6n{T|`Hy(ha$PJ>_?J|DO+AmwHz9KAb*g)RX`q-_TZCV*%kmFzUxE{z&_ec8iA* zKIBCLS{vo7%5)*NNLU1QX}2uU?`dL)wAFk@VqC;23=t%O_EqyHXkxPa5rH6&=Pi6a zHc@e%mGb>bn3!?bye|$c{?faplMSl^7<7{Qp?M-uTW#~~ZO5BQHKac?3E73cqkEa( zUEEzS(OnxLL6f--4Oo6QB;$WQ?Y}yG|MRrv;ofR~p!lRT&KMcL77tl_J3+T&7;d^E zZbs3vHNw4ExnZefD`Y9Apuc#Ou5OrR^vv1CvStZowb0&OyIDfdR`(=@o^CAF3 z_i3%RcV|XY6U0%2q6G%th;a%I&Acz=$45w-yf)-@?x6b|GSg6QKZDp4%zf36^n?3y z-YRu&qd}4+=w&!V#|Yo`dS8Kaf%s*=CVL(TpM)y#tvlB^DYTEyG~a%;-)^+os+>&4G2>O+3$MFt79=gq|mC)?TBdv+)4HAK60ST^v?3Q`|- zEarT>O9B>W3=-8skRI&C5%_@B(q-8Its2I^?K>3j~ zpCs6T-%Qk;Ia;9hE|b#!Gor*TN8)i9bmHtg}l+bh$`tNRh3LM&r9SRRz5byV>9{%x$x%*<&6 zPT5M3nKVXd@&Y1fVBCG1B?&M*Z@3i6fh)?9Dgx@GHC;V>MXmhJ!=W@|D`YB5-496peC_7t8J5r*5mK!3c=l%)1qJtJJ&Qn+}&UJ_@1j z_xi86lb`I~r#Cs(8H&SVU!RW83Y#eZG;U7=xN7!QqyI3w={pOaU=d9gmr)h^um7H@ zL#5CP8b!*sRAr$40Q5K3p@+?E4uBti+mlY4;BAZhQwVcSmib)p@X<@~(~3#%!jfy9 zYHCjo>o;O~GweL(x)`*f4*h#))yzP$_4X*Yh`dy~<%_#?>&K!*f}|vs-ocD1Tvpd1 zOg%|*+gq~tNWpn2M99=p_qd!5kb9~Q3x6a3Gk6xMp~YdwyB9vKrJ8`7M3k4bo%C5P zX5NGv3txWH58?F7M0{%ZV*@4$^WdNmB{=^Ry&4T2t>5<+Zy}&EXt%y_94y(_csq1f z;=BOVs<<+vpp3|>CW${-@&NC}0WwiM8Kz`+?}VDjoz#l>+!C+)=z7=13c2tS8zXH) z?7qp4GfGlseEWnVOZ`}AK~4rRJmIU#46rxd`1LzIA$N zqQ<~(ol|@WeqckcytPgtpF%TN>*XjWR3>}a2%FrGdUbT@7rR?_t*<*02dtz&p5guY z=4$ixs6@jT&*OLr%mxI^TSv+Z+@;d~X5}{%V_Q|d$vWTsNnp0gs(_c8TEQRnYLRa1 zPFKcRH?~8?w47n=pM7bq4KrWut-Ml>VbskH#v3l>AF5lU+Fher zp!i<*Sj__crTuNEqtfY7zgQ-udlR+!I;qz)JZ82!VDcfQQFS?v#ZSRLRHAdAaQ|h)iTCg6Q zBiY_~^Vx?VlMVvyNqS;);G~MAh>q;n$2wxx7sj)cSoodpj>di2gl=^M&l*S3G30)E zU<@B~Ya>aEX_Ld+Bj=|jh3 zu2#)Y%f3)bP=v9nh{JwccJ(~@%ATrUDZFe`MA2x;flOX|$yAB|4xf?cj7->&+E2Ib z0|+EZ87}ETbN-Xth&;Ei%`cBCz!q<*7+{IIi2x;QSKrr^Iow^eeL^x`J!h?Mr!&^UuBYI$h&vfKVl)waAHP_e90?WbB{ip z(4H_i?$oi@h8F3EgQao8EJ-TQnZZ#ot)oMT`-P^( z)8>}|np#7V69Fe(sP}j3DBkC;N01+hGdMw2XaPXWW+-P-QhXbcN_AaL?QTj&F+G8@ z7t?#nA4!UGANNw_F^Qqc5tSrHnr{+Ag=tO)RuMU-nh;gaG*e2|kU7u%%27jHR+j`9 zUyn!l;gIkM>MrBI`neZTY>LJ*fQwxuKL5tpt{-{I-Fvh3cRy0CD2D}Ee|;6Dr!r3x z#tjtlRjSsJ<>FjFbO?CrWU^2&HD6p-{QzM_{1+?(inpUdBQpRUg?YFK{6u29@bg7|S0cUiaY%@R(AKHf(8+r%a0zh-SRx4mtiNLob9H6jQ^u(_D;#>T4Z14^5#8-47PD|DFqGj1>-MS!AYE@4fvmcU6ScwKFZh9uE#q2LK)z)`*m`n;V8z!_>`FL-q%m&)Wxm zMaqEZP@ayrYkmhrE@C@=HyCQ9@?))3hp5PoawmpVU7Zk z^f$;#-&ZI`v{GZsCB+@+pHc8lePPO}PSH9{GO6x~c=qKPCgaD%*G^!jX^(C?Te$gj z1ew5#WGQD$^`;t+n(Zn~#-QzU-HyYyV($T^x2o&}t7>+vc^dEKWq`0VF;g2U4Ei&k zvL;6t4f(`H;|L-VzZWUQpvA4tXkb@_|HS6pwj2FqO&GNCc{gz&qcy|C*uf+6X|fk`{ol!X9gcaR=zED`1|lrLx}fv??w{i= zV4!eL)WKhON)9ZNg~tE3kg6YrgDV$4S_cL3uHvT+uTz&(o@?VfS@Uy@(}iLQ9w*kc z$Mll-m+C{bJ}!4U0V8~sun+FsRRj!h%*@teNyAfQ-LhXx>&fq!6 zKmUqPMHB=I2_jhfBPxr-ikpdp531`dW+9egK<2YBSpubr>Cssz^k!b0HpLl${kJKZ|&A7!~*0EnbbXpx#QSCs&|fPG|_VpE^c_X5qWNK)czBg zgSb;mS%2~MWq0-bWgjJ*XU@Fm#}5Iw{ejCX-=8lzx(wCkNRP$Gr;dqw@>Y%2dPV8k z4L)A9u z|0sa78&IzvwFLj|p0ZNY{OEr!b`@~uKMO+x#dj-j4YX8Vskfkw?c`}RB9AjY?t@U* zCdX$1E{*aJbeQW0g@=n>gRUWcj@o?2)0o$YxQCI@@Jh{B?5|HFk{}#!BE5GlCSyi zvvW|r2;e}%0xkqj&}pB z4v461C?0Rif}=%)gx-)Es<^IB5zL&H?}#H^fe>4VkQUCHx8u-h8hO)E>)-sP(en~8 z0ICXD4PXtpT~713d>3)rG;6 zFSB3`8HI!8(qn1Id9uk!b2PFjdc4=O*|4 zdEqGf#?uAimM^qPSjoig*|I$EhXI0vn8Fz`!JHVPUnH9?LygjJIbv6>JrFmOK0jiU zawFvhf*^r$JT=mijl$D=thX(_u3fX@xsN~$ATaT8+`xLmGez(dXbpBSKGU1Eqi!^e zQw%ZPjOY1SHb&X6m#Q zJ1}0A2F&0bfr;DyXCc`lP}9(*J2lJEhs1KXD*Ocu*MHRwE47Z$ z&1jk@uBY11(%Vu9GX<=9V3v6pGsKXzDRP~4g&-0tk-2AXQ$`W?>fRf*mh18QyyRU(YwL z2k5hWYGg{{@=l&uN@9R$)TCCJ!V7jlgI(`WO0`u4hOW4S@8-!kDTP_a%V=E9*{^@X zzzJkquh6lU4m%leSDm)kAA}vq?aXehbl3Jrt)s8}Fw}-B-LdhpZeyD9qQM;;D6&Qq zvHGjYS=p0~v3mAYy?^>HDcr0fpP`G!uK`?4(dh~Gk8`puZoFXBn! z1WWgog~a$~T$7))JS$Vvx7oShB;au-(Y%jh2%fdX?1o(6<-Jy^LNfD5jCnFy_)i3D6*A({LHsY)r37Mn zk4E!9f7RO7(6=kCqH>rIgFJM4DVUdlRMmR7G*((8ip@AW-ymzlKBZca5tDqEtuYY( zxVfY@1x+3#wf%g8@r$V%U=IL|=I;O}77K2mDS(K>aiA2$(P*H4YbT#RdI%JdRV2p0 zvlM*{3^JGRc$=+V`-|Y(W$yf4c4z!+rsI~;>|Dp*aAKc+u%I8SGeBtnM3VS1;PSm} zNK|J)X$sd!fOhFQt_M+&47<5^EvH6L9uyH6QlJNoSZsRK{}nS}!{O-Fw0jRjNxl?JKiCJWGYOSk*v z9^9!^ai5$keggP7$NEuw!(u)g4b=F=Ku4w|=3%yU9eo>@Q*6WzhiUWKki7Xd9o>Q| zaf>EK*;XAI*hBfLskR}HsbsU@>vs_QG# z(Jr)l%RC1C?J%{>UYZ-bQclGRELS&*hMa;2M1Qztj0u`kLZ8`Km69xMXL?fb11QuW z;RU-GRiX;p!El4Lzb`&w?5NzL`++5&0`&!W1a(@`3paVtg=C{G(vqNe0bf`gF`x zY|^TNO1?0t4a8jarhj(nsi?Y#H&5xCR-di|1_j7xJ84UT=|chJv$avfmx~OQZUVSC zyb@LWV@8?~AG+1_W$qRuMVmfcxs3N;n^89%sjp-93H<<8)nS5*@Ny#tqjq5u&=QS| z57y{n6iE$*%Yy3!L$m^1%npU{xNlh=O&5OnD8Ijm0qUUU6_nlFm#B*!2jb4=lLvi? zi}Yz4>9i3|UHx)H`|{QAkMng`YzJI!d}sU%MpW8{rMFMY!)#^=jMRYH^9Z!z0F_&_ zpVOQ)wqItFfT|5(A_1S*&+@Xia9o3Z{cU4jH5&C&qEUz5#!l!b4%4Sq=I{cgh)z+& zOQh&m97X=Y#GCo0-~GOlnYfHp4&vCqI!XBGDSh%5C8xI1_dEa;(?H>xmU9R{lp-YN zdrcYak3L;}t6~%R`4@tpF>l!$er6f{9!tmLJ>64$29XWIVU0yyCVSTl8h!P?`~u9q zHell@fFEZ;?j!viW|&)oe;A7xe~lo|gxO@;cCz+9*Ei%UJqh(Dlm&+e7bsywRtu1p z`h?MeKupT~4j22}+w1xGuqM>%K-tiff7C(be8kWU;D@wbg z(1S37lq5H@`@ed>pN3I2Rs{<*9Wgsbi*|tZEWFOPF4t#SuA#*e%RbuK(@q(oJHO%n z3jA{Mvc1*BhUax3o~+DDzTusjl@m=}H!=8dh84^-+~#d~DmNUpbYmIZdei1c&*OKg zFL%DpgbTzUQ?~zA{x<9Czl&{8`&`4+1UkAm$?ew5&=fCR#4+Wbx!ioE*H$Umu|_!f ze8mNU;(?iub_)~Bclwep^|aN3Me#Wq=g+O0Oy4$l$O@bl%0u$$x=0))vl{XDmA!! z{*>0gPO*d@{h?=vk06jJEICq|tFh+g0($k(UtKd@5IGQ9hyL(qMof@g>xwD1W|q!* zQtbUDW%rkf%DLYHo!gDDietHD{)NZ4&B6({(eeXU+t zVs`Jm5@v6$K4EAIK15KpObbRxtq2@& zhzW#v>rNLpn4XuS)BtwC#GrXb@`l8q&>m1GU}}C@IOx_M4uRgxNF3A{+1*ls!_~vp zgBrwK0^F7()`whDOETj_B1e{cC5XZkPT26YT1K`{Jttk4p9~vl38iO|# zs{}%{GJ=NBuDmpropt%&=Ox)_N>oD{4RP(-3;fFBNcrQK{bK_bNnny>Mj;D`9*Zp|9kN0l2AKk1tf)(}_9hK_FR=0W93u%NKUGco%RH2g;CD0FmL@3l3h;e2lM zc^w4i_gqXEJ)Dz`_YjB4Fp=O9 zB9bgbG^pa~%b{$y`|pIuL&>C(u3cqdifGZNlpKs>EN^V9#rSImfV<;mU;R^ud3yOB zvsTf*$aF~fNdJMWhfM`GA)WvLEdvG^awpeq%c<|`zSjvKGT*-bW~(r10@Vt05HP8)p+$&d2=8w%X+kGm<-iOumPBYWxN$3E&Tygjl5+*lnC2LT)V z#M(Dgza^}$oAH7}atDI%7zTc(Z$Rx|`-=jjs65GQlv%eYS|bNJqaX?JbFsPm0Gkgg z)?K=b@3&ksYXlKVV{;!kKA{Dw5)3S~MZkLVa;R}Bsvw5o1bZj2n59K#pRej`zYKDr zIWY23YHhM&3o!?8y(j<<88Br3uWbCiMV_Y0@;Sc8Q=a6}(2A9^A?exQ-X>lLBBU!o zfhr^$6f1A zmwAEEJey(Y=Ae;Kb|y!1JauG=FF$)c2(_qMhTcqG{>U_zzTN<83uNYR_Y{0YtwOY1etwmrM#PLgwW6KB4w4)l z!UpHSt?(A!XTE&BDg%5F$jW|3p7b_N<;n?Lei`=_6YPoKkdWX`fmC>@q@W#xWg|{Cz-zC`iqgT zdlm~#t??RJ&lsDR(ZkWOKxwL(5$#%^&j4kpV9HNwvFt|Ha0cf>^5sY6SJsnKfXkC~scH}ioQz4kH**%_!GOyx@P{1Udy6ZxfOZH$N}@)C z56q}i0TNslF4k&NE#>UI$6DZaJWXZR+D)IFnAgV&DrtFn3%jsd|IKPtFDMt#i@LNs z{Ino}$2a`qyQ~J0YF)+NF8o&j6WNt9oJygN;KIB845Yk_pq(!eDnyinR-gs9Qti49 zqfSK{aqk?#v66gGaTqJPgc;8P6#E^RasI@s9!=4fv-+umzz08B{zICjY=Ce_WS0uH z2A;DWB3Tc1kiM>kgP44OHr3L8pp}yUbYV=t9w(fUAl&3k5yOCkQi2eDB+!i%{h?7* zlJPL%Htb@?U17zT3E`9Yhe`1!1U$UuW#YoEvuoQHd6hzwK8#dYRkVAqbb^m zl=LyPC_xzEpg@SMYoSiul%gsk@A2HGkIxuBnk=R{wnT+CRZMvpE}|c~s>PK2_-)qd zgAv!8+W*81cJTEg@-8Gwd-x0VgO;=3P6})VGgjX{$J>Cg8AP?&J%PiaS~24NQ%eM? zN+gfY(?0YRIWNZ$^sDQ&tPOnCn-rAjqUoGIlg)_@b+pfN{BJPZPxNqT&IPD+|800% z^r#{jK-c>9`_~nV>p7$269We0)-aN|P`R!!YFnE3_Ml%J9M^}|;ePlGZ4hGcyYFnP z{O%bUbQ%BIIe*O%Hh4?OxP~V9IEmV965$?N<@Zi39S}=3h2#5-F^2n5#3}mEDgh3; zykO?m-P9hH*UCT-Sq@z`)ZoD`R6GH{VcRCuC**H`OLLoibPEZ)#rA zasdy1U+v7jteO^mn4WK%JaYeohs}?kqsLljom>3|BD_Y(=(P}kuc{}b;HiI5 ziYwMKNy8&(fF~x@APV4H^RY^#z;CtQAGv}+o2dF_lIY7R$NM2I&bxrtL%ZAdzt;eI zlVU^W&|Id$p83IPY5c+B+xJ1MU z($)x%ku>Rc%z!gGXh5mRtg4tM)yTuD=WHxT(0?WDC(*D4{p>eC3&*o%O5pcTEw3G8 z>gz9mWOM^-&MqTX7TptrT+=RZWEY0Im}~#268g)!93W>m&&2Fx-*=?2z%@j0^H=M3fyPSouI`M>?xw_phB|GHL*fdB6t4?dYGGC) zygwPf^G&fF)Y~Cnx*ae%J-< zr_jF>^L13e2y2A$crY2m=K+v#v8jxcWZ2k%m=X>v5{0ggO*TwG`eGct?rRR(vO#)+SZkN2D)hcOg%r+r+X_Ga92wIBO)*Gk6XDupiu3xcCVddDYZy>j`4%+dvv1l?vM1+k1t@=oGGf6qI(f zDNl0EZks!8WL7XSu!`Md+sNfV%SvupbicLgwkpMjGVS1UvUy16sC&M>4-xe>h zzMXskbh&9f#kTfYTBKoSUtZo#%=bk#TPCw>6=u*)G2t4M_%BRNz+=*xV%Gz8Pp?m_0KV-9e0Ae)e+Aqs-LGXmk3CMQZMwW}Ij8+2qV@3p z6CTD0Fc=j;dg*^n{oc1ltqyG9jV^A6~S(p z%SW^MmD-c9UPx(W6#^62+ouXtc?Yg8h&p@$cP4Wn^KcdiSt_aQ*oX)tUQ@s==I$cyWGR!E7rlrBLyL+Xg z$p~i<*48}e@2fyt?a`ZSb4Xeeui9>2X_l>W^QQv^-Yf)8xqGVt_n!y$!|WH&!A+s3 z1>z5-^MI=HkN9-eit*rvYdMGppL_7jT9@y_nsNQL^DcWzEni102x2*;@WLrBJ!xaX%J* zEs@z;2te9yBDQ)rd|}A@%er)`V3U0)z3@6ykd1sS^go;c?L(2-Js83S=R%C#YF~db z&&-s`sC@SW+ijW|j6%l92v<11|M;Pz>(0rsd&^=#68Q<)*UzBUH}B#>pQO>;uCCHFt_vNUEys|VAR5T{r=JG zxJ_gj@uDtez+`5adq_<6bO0QpM@8lhM<<;7x!`NcT>IyR-`tGRH)%mS(GC&g@a1v$ zxacqZ`~Plc*HeIm_dy22Uf~CTXqM|2tqsynidK#5iIa`~Tzlm*z$?gOUnD!136hBP z%T*!y&gJ>6FREVOeBJ1wt7*;>!zJ16$uv2(&cGyCB5HQ?^P2VW2wThx?!hQqYY;1P zdI%!k{24~_8%T7zE;x&i?VNT8+cK9Zm!_46c#ERT860)=jcw-X&rW_Z7|6ez4B|iV zbrm^>^lk*>T)wr5}1TbF`rIDdfveH1yXRbt18G6AA&^M2ZL|+ zWL1QcJ3UM0&ylxjdrvWV>`@!lLRgS!^d1Re@#Cx8xQS-Z&L2gw6OxY^UWvOlm!wjf zSJfO(R71f`(KnyP+?_sNE#s6m9p1V%Dp8x&e^zlTnKjemG#wv#D5_@s_fMX~^=`&K zPIx+f%o+hE`28!ED(&#?*0iil{EkL?e}C#fQPw!iIJN(3GI+>Q5NO>&EiunPT5Hw# zqGQ)~*iaS>%r(B&S3H}}!CB(-iB%W#t?6|m?g&#^w#)C1tCjfirx~hjI!;-$V9I|n zN`D{GdNxatWOL#o^Q6l1M zz-r&sNQ}JeT`NI$M&TkAyp4G%p`zKO>3$U8iMT+@JjQQqV^<;2eiDQcl_qrD;XOB7 zX@Wp^)I#BP5BolS3_G*Tk;4Y`0=c8*U8l#G!~}ougOk|5uQCVQNF)Qb?OU$_t#bgA z9#^WjBh`As3tB$+yO&tHl$SX|X59iaT1)=B%*9z}O99(fSiu&mNMy1 zn_jJppnKNWUcWyoId6P$x3yhNUVt*9sFB+oo14UEV6~cIu?<*uq2x>)j}*V775>sTlcWnpn8v)~cd+=UU8L^F58!C|m;!lg%)M^xhRQolO|saX}kMr_Pf7vL;l;H0jM4( z?=E*~mEK1G{U0cak4_i*JNTvQttDwc31)TZMa&hxtUrDVoO$s8Bcef?BQZY1x)PmO z+Ef+RN@GH@532SUp>~>a-OOfg`mhE4OiBKIuWtd2-L}>55*~CRSLMhP`Vo^ZsBFvccb@1(Kp|Z15OHA_kGt$@w?I1tC_K*qiNq$nVH|89)i+B z;1FBj~$-pc#YF<0m;zx5};}@J3Y(G*8ZU$aN{U*Xf^#*;z2%CJVnL-!6-wu5pBXS##9SO z1uGC+=^PL7TEuF&`iY?CTKe9}h&#R_hf~v66Y-HR;*youvi9C3Iz6?TjjlWe@-LGQ zUJ5u{V3JHUjnI{_i%>52TQ21_luo_=zrz_YC0xeo6prK!w`km`yU*F!fUi@?PTjw^p13DB)yVkX7 z|64KofOT$Os}~PX@7UqO?%V}lKFN9*w$Vx>I&-niN(f92ad|Bdcr1u%(q8_W{Qa9~0U__* z^nGP^#lyErMw0O3TOD+N83?G`&K|71qMuvNvY#32SZ(7a%rh$yE79DWAHR^H0=f9PU2&TCV=N*5i zaRB?!3EHQLMR`_`3pX8oG?36hqImVYx&A{fu~)n_v^YQ=laLPPc1TvV^I-G0XMoP9+T}Gv2?deA0Y=S;B2#~bRIv= zo!6o;D=_VS`=MX%YcJm|ol&v*Z* zy=>+*(&S>rk)W(F?qriWF8#UliQhA>2-WLk$HtZ7 z+u95aRpunct-hFa%u@TEWIsAkez=R1cI>W?vy8Jhr%$GL!buSseH&S6j z&_}{=650aF^}MQ#Tgo1r7+AG|Kn2g19h)y{X;WazUqOLJnz`F^R7ybPY3K5y&B;qj z&Mt!`0`kX$WR#-wxmfovXGxeD$ld(&BVfKwtHI}EB@LWaEWISo@2sO@arYP(Y;rZClg4khpdZgoJAH*ZBO3g0=4IEB5~!<~@K~BVt;VHVVN| zOjYy4-|~s;M$R_kT+^y%q`Ma(1*ro>T88fFqgp3xDjVZJ@jk?JeMe3HC6Egw7S}0j z_V&Ip1qKBbk~td6M}O)r^mgL_pM1wHgXd-EddOb%6J8Q12Z90auKL-Hvb#=;OGz#q zhc|?ABbvKKaudm7UpNWy8DzmY9Tnu!3dzBzTT0z$GW;zbPm%VTts$o_z||~)L;=Rx zbxCA2vhi+5C8vqn}ZZ_Kzk-aD8PC{8X&t^C&>w zUb1~IU(YG^6_-?dtFFph8_`IWS$%~{JU^Ga%07~C=-s8PiM1>9F&yQ&5Y-*|Clm3VI0-qtKIR<05YleM@GxN|lp4On0AO9! zx5!r3mwPZP2E@d=*~|;99YL@e1!l_zF?9PyvOqVcV}A4dId!j;Bbcq97#r|liQ#l^ zOkx~9W>N(-CF&1x*WX@bCHx3)>G zLk~c#VA3u*`);QiweH5yeTb%YNK(uN(h*my7kH+1e^;L+#Xken;8Siec5{1;T1uKl zzw}E*ikbq#nS@mZLRy@rP*k&T)m`LaLZm#mruF!rhrsaHsXm3f(~5{%{bHeh?Y}o5*2K@n#NIX^D0}J+ z7_Y@kIDGb|WSfbCg}2d(3pF&3X70JaZ`_MoSbS&VRQPMbBaxa|frB@$XR^@6twHM> z63#4VK-4tSxi-{{*wZrnbf8hK<-UNeNcE$kc|2pW(A|Tx&&B3pssmkF0GH#r1~;lW ziD-%o+iqQ8$^HVVTG$+70lWllV|v3b13e}r?v$M6+B6k{#0E%OR z5dq&v;>D@CLTXtc*7S@W=jbxm)N^|o@)w*Ph5TbyGW>$gf(eSjKeueg zZw+d-qXQlHw9}iNE0P| zKYPzQQ4^iM^kT{pKo06$%6{CqZoc%lYMIX1!C@0=bd?Z=L>8*sX(w;%AcToid(5eb z+4Ie4G_qtdQ8{>W0!YuHY!#c|ZgkcYDR}NRV$a5RvK-F;VZG#KS*nl87Yk!J;VAUn zzq7RrsIDp|-RJGHj4q;!2@ZVc>jA?6zL3w?QW3L-0lPb~=sy$VHYc)Vo#S_K&kMZ&wqfjCf7?Jx*- zA~+GEf1aXrF1kHFdG#?_{A9>Pw{EIjRetk28x=>*oV;%_Td;)Gq~W5qSn@8+ zv7OYaC+*JOS;BCA@cRzepoxL^Vhn)`X(FS@7!7U9KjVLWIW_q6bxm-TGz1NvU~+a_ zNiLRCKqpSH8#@9V=ejuzxL{Tf^C~BvXwzMBA63nm@A55;^Bg9BZBzO#h2LO}d&F2-#jNr8ViDs$UlSbfDTb25azFXOUDB7Ikz zb&-31f2cw9GPll1A&_rc}Dn#HUX?#!sEuk)O1u;JS^?i z;x_?&qHVK6Vm*MIOq@b(R!y8Dzl((x7E6z<$r;|1+Q#z|t^tdSh_WH*@$qegn3it} zNt@cRtBQL~4OYGe=KC)HIqEWzO06vFFZZo~ z>u}8|bfM9%oMsNT6$U+f^gZpTdwa_G2i8IT%96D85S^~~e80>9#F1*I#N&&Y5qg}U z(Gf>X|3No&9t`spevU`}&%t8nZe^+hem5S&8MjfFO~CAef9JyWRA1-4CsEGyC8g)^ z{nb)Z{L;;-i5tzvoYJx6y}Har=lvRe^X>iJ8S2Aj-9tkQ1PGM&<(x^4S^~9&sG>(Er!KeLRDD7Oe0D-MX||aE_uhRKzbv)6mAWigN0rUgtL*G#rqTGr=JW z`ZvHNAY&*PO8S_+W6`r-T9-1TBw8(ry7S%x%({vEtu@0BHmld-(pTwhRpE>gy#2>u zgJ@otb1YnX%hrbB@LD%J2EEgHD^tk}8(s=D@YxSlfrfLzeavC|ckT>u ze179Tk(uvcZ}Y08M=^Q$ltGUY>i^cqtG(o9P0Wc0hK1m3Dx_L2oL_|dfvQF51nV!x zU+ojrcb_!kG*aJTYsYD1J&vPLr_#SpKw-DgU-|!)$i^VXZ`oFJ_kSGt9q! z{OV%KWBF(utx@8c_P5Q|TqYb{=BO$Dv0(IN&8YVM@!Gcaa%FqS$<}$g?`-SyrYqAQ z;~C$+nHa!sj#|&p-|n4#TRQ2czxs5))-Bw9y9;do3--@sX;L;2H@*DctDVVOqUkM6VyLH5j8fAQF9`YF0MTfKQIr@eF@e^;$htis^sY zdaJ0ox~6Ni2?V#`9vW}lU6MwEyF0<%-8EPU?h-t>yF;+x?iSqL1N{4W$9Hiq&KP~s zm%Debs#P_s=9~y?A-5h=ELXzYgvKSCeC7~*KJ&EAv`-;Ds06k+iXf0I+if^DZ_GI_ z%q{fWQw%YG%y~16D-#=JFzZUfZ%`^&@A?!H&3q zIqWz#GeRb{EGn-8uR`+y2YPhuGF=4>h?xckTq8p{_+5`kS=%XV(Uk)Ds_e9yem*bc z{@Yo`h;_7lv81UyRruQ6naAs9G-picis@VR^IrrvOHV@3#num>n{WOXS>j6n+TJc+ z3VB~KHlEReX6p!;#b<5YF}+!xXRj#;F0DQN4@39>;7J{Ta8(*>7EGo=(U^Yfj$OP4 zfrc#oI4)*7;wegKvr}k92+_KdIA$}t8^jIb1c9O_!|kJUG8++4qmRE^#t0`XBvTXS z>NF>wSr0Olt%FvK;@9Cv%6~z9Dy1W5Kx6J%%h`XQvEOt5n1v>IKPCEI7Ak>^5o-5DYsrC6EiKooCYHc9i5%ni{Es zJ&R0p)hfj*O)KSc4J#GVH!rZa3=c$ijkb(8MGuX)jP@u`jENL^dhPnrXdO-isIX%n z63;Z(<k>pI@VoVY<2U}7 z@ixa}dz<=Snx*?M%{uno!a3V&rI^x^W1)M~lBNaI(u9S9 z>;tIobFhXWPIx-oL~dnbDy|R^GHL*4q6LU89_r4Bx&)AVfSBZAiPE9vN{04*UKz?r4vp;D zQZnRZg{Oc+%I#ow(mmzxzJ-wLBSGob{*xRF+UgxI#0ZX8cCb-4VNY(NCZ-csq~Iux zt(}aw9IO93AZc7Mg$43G)DddO)Q0`_Jy{zGwMISVDA&q@3J&wDxNkNWTK6;@rgw-U z6nHLUCt8DWx=4Uxr~7n&IoSitz#s?MKphnOpXn+88PvbD0(Co_UYh>sx@##G?h4Ka zwmO&L|M%LIArM#hSRMh(vWx~VDHaY$u~aTX%U4}w(~wzd>%suCJuj>LdZj<8eZgC@ zYA_XPTkr6EG$B`^qw=U>{;qUIaHRJRIFqq|hLjp=MU;__l78u<34T+&ieE1-C{WET zKAr#B5?`M#RrfxgOd2Z<-6WlE0M}d_Qc<8pbYL`D6y17*P%-P^#`mx6FNqN1Imgm))&?11I7wl#GNj_16AG=hw! zJwzcn!VY@|S6t(=Q(afete#VwC_gYxyUeKGu%77k%^|zW)Tln>m*P87N%CII4yCeR2{yT?ocd}0U zWhy~I=w-gCoe<>k`Y&`iB4X>?J*@w~^$`UDuan2~2cC`VFaCzt*-Rs?_rQ3y6#<}d z6EM^N3bL*omR1<87sQX%L z0x1PijC$My<+o5>zhpdX4}L06mwZQ-Y<06A@4Ru>2f--=U#Pl66s6f{bhKNM+br%g zXvY05wpxFjJ=d1Z0aQ>Bbcn~VZ{cN8;h+DURX@S?b=QT=+rg=UfRDw**;-1@{GkgB zsLjl*(Q5U-A$Ro3%b=4cwk6Y`?HZN!ic4#-8H>T8$;fFBIRdP_N)bYutTW$`*!VX~ zB+Txya{<9jnZg#p`AfFKB}&ElWvCdO6jyd^YG04XMSmt)v9odPecUj=mXwq?M^ess zo06i$|4E7>K2Q=X?nXTQkKhbof6d)sxk8Qk4j^;DOF-gK;~)8m#5p5faSmCTzsDpE z40LFBaC5r>3U;jOHv-qET6 z{YIMni`9?LU)HJ~lI>qpbol?bZLr$m?>^=gF{%BZ-6S-!aNe*JWt^qVIm)mlp~^)- z*Ul-pkvOOul;E7?Ja^Q^D^SvJ*Ksl)eV}TUi;ViYrGPFM2%GbgS5&|bD-kEiIh%3x zxL?cdj6}e{(W0VYWJHIPp(7cvcHII$-1Mga-)UiO%1}L~^^$b` zoD+?X5P68*?PJ=fg?C)~7;bW^#JGSC=2_YtJjAqX?dzJ|aqr{!iTI>tk9@{`Tlq?o zXQOx&k~IQ&Qz|b>J-Sc^<)?Oz_g=~JXywx=bFngH?&w~2v|BR{E&b83rp}4xIGfgq zEYdWahx%AL?(F6ufK7UbG`|5)YMGDr@^?Is3hA1L-AS%~gKDLHf7Enp``tIPJ=i!V zTu^}FsPo?xJMJg|7jIp}g+FHIF2BzMhEYw={mK}EB30P}*4n=7a`6IS`DUBuy-70G zYB-7?iO^ONo&h!@2RJmifObWxLpxnt0&cTp<=VU<4fUPBf+A0{jwf2{=XHMW#b53z zjH8E?;|e{k--9-j>E&BS;W5|{6B8F5^(Khdw>pXMA!x^nmKBAGTGPhE;#rxXEQPW? zHud`V$LBX71Cx53a#~H6bAy0b-p^}jgALF0yOt1lKKTGvGW~G5H(p-0hW~Acc>R~Y zqR`?Q8Fu>n??OY}oIwJ7T8B02Ii{hjO9D$ggP~+5SO31IbNJG8-hNWx4m!NF>Nxq5 zd)y+RUg>9Ckmwm-?PuRyYN*ZdbLNjm@*_a(t(nPhBkV_Bhs{zMjIu+RKmN(BoRwZV zc5)?cDiU+v>SF=+h`YTuDd}+GKhExRg1HLyVP%MliFdyWos6f&WWK`_&bZ&l&C+!! zE~WGyx$T%`2vvHVrv=upYa18!?pX3n(JU((=B363Ac8H3+5BkgUgC8+zXbRcJzzA_ z_q}4HP(~~v4o9Zu{&t`_M49Mrz1eyr4-p-Qac{kv_TMo=8AA86Q&xCuck&pva#Rdd z44PdlLcY^gT>m73N&PmjNa)M`!*OU*ns>w40RB#TY09Oka2vuQ`D zM$}6tV|ops(Y8+~x)~T#Xn>eHE|~Q${#!Fl`43|lAuvvy#|La#@V&2XTNYuXvc47$ zPV26|uj%DVRa_Ix)FUVe<>mNBTV~pAb&g^A9qV$LdJP698e*9QJ)H}&5^7y;Dc z*{^MHPs;hYQnE8i588~>wj@#&Wq*waWMzMhPV5Ngs0sL~3r}qd*lW9c^C~214KQaj zWc#){AN}6^`v|O-*~W>}L^0ZP^<00EX}eR!Ac>zm@app*YY*ng{SL^g^=o@w)N4N4 z)LzNP>2|Gj>-7*YX&O}ft>=WI{NlVrRKf@(c*lW8&bc@KeHHk>&ulY3vMxSJ&a1Ny zurk*|Gjq=MG>6BTP4Lw<%O*zGNSJh~8Fo&biW;O)YOH_fkX=g9cWxW*%OA;>Ct^kK zuG!u9w(R_Cx%rpoa`*0hz+cJOsPEr7Jk8Alr(=CMgI3{Q(=ks?|n{KDq1wC@9?qswj0fPSbgn61c= z9Wr_e5TJ~++;Wai%gRe-WcUrQ$%kivz zyulVP9`ow(ws{?R7MefCJ)JkagKqbH(G{VJi5i-Bqts`i4#k5_%aq#_;negDD;|lb zU76qeszbo7X)+u?lst2HeY<8KDUB8y>-e9&P_n5l$A5?(f1zqI&jI?wQm9KK)5K`; zDcMMBYneNR-bXli5VNZY_CB5pC}m%xU60bG^g|-0NMvsoDJer_{lbWP^31h)Ey_N^ z{zSFD25!GY@N|dCgP&M-uv?q=RWn}iMuR}=5_UAO8x`3$Bgaw8IGo_NE->gdsCEBX z?hC~utyt{91sI)xcQOIxVmD|;=b?N@h-3|aHiq}Jaq%}Zt>oKjV4>?+zBuuW`fF!d z4u6W@md7`w{=&|MqF-eWxuH9j%=jyhkLmXhar}koJwAi5@Z$vbO^c5+3^}i=`@h+k zLh$~sXB6lc@fl}cn@qMjR61>TIu^1*XRqA^4Glf>t~}R0Yn@+3weHK+L)EXyIx*vE zr3>;7Zj-Z1$J*DOs{>Xp%@Bfn&fm7Y{jZx|H=Z}15sV0Qg&*=oUa|dOYJSK-)-U=E z{so`^dzCIk1wCYKyu7?UjDKwZ*YKn33b*S;=gpiL-Vy-xh5p+J$gSs&jtbo!2X3^l zDxWX^+iFaTB6NE<0v~+W>*H^yKAO#9#$pn6T}u}* zKX7nLIFLjRSiOQ^;6N62fF~NDZ%sf+%}S%AYax6q)-~sXk550tF&BX(61l~Vr$>5c za_pYcHWnp*UN5guy)Ko^Y^i0@@i_Gkprjx04Y{%wrpC*Rov8ER`Q$3B7Vbsp(lO&~ zYyXiuPqhGKdDz2E=J{ZL2KI=9%(#78owZ;NCuZKF8&MkPycl*VtuVw?=|o< zu9mK^ZlGRvY-&Vw4WX7Oh(<>bmHTbk`z`XD({$vW!Mc(4+|ab>!^{$@{miI5x=0cb zh+70$2}RxSfpSkuTd>t6HxS6x!lcOe$Hly$HAEmaM}iLCmDqN!ONR%%DIs zW-h+YsZE&Q_;;5)1o7Pjb>Mk(uqez#A&;2kD=_~!QS?;C#;YlbtZyZ+o~COhKI!y_fC()okl zg6Bk=f6x6(mFKfai%<2Oo&Q1KN6$BJ|F<55GX#*&eb?i-|8vJ%4*@LLrcpZ~0ULXJ zo7n%`e+4+}e|Z3}f(zd%77srqVJYeFDAdi7&;FJdTgQom7#z(C%$;uEEx3{nUJ4N* z)Z@)qg_nIm2b^kuHGDv(3YB+SBWmtYb;A}h|FE6uJRVst=`CSx3>avX0%Yi)1yby6 z3yk%Fvgw6x+*Ip$2_Oo6QNu`9bTj-9S7UTBh-1q1@A zia|)7sD#G6`|r02@k3XaL(z?+m9Xf#z1Mbn`Fm|p(ZvhsR?QvByhX4VeKeYfw?PSlE0qI~5OF27NM#anX zUFDk)FausA{oh(p;`IB3tlxsp+@60;iZlM`Q`{7tJ9@x~r78|y_emGF`?LEPrnr0j zaYS;VDqP-vQ*#Z>JA_K%h$x)hd5&;Y==jakUA#+2*7qu z{$YuRui~Gk6Nd@0l7R)Tkv|-b&DB>+3a$zy+iR7Q*cFXCes^e}5J8_8^Jkq&794?3 z@MPH!IPbeFx3ZU24ooWLW2oIQ#YBTg{E#5aaz)Qc|DT>d><6w&{7=zMoQ8_=48n$M z`KQhwNBN7zJIhTglU8r=gFv4^us_0bKMu@I?+{BKk~5J8QUwjc4h4pT!eM?pG=~5% z2r_{sCr>1y-mf0Zb#G>!SCi?iKHdard!XVcoY4MR+sWcNs8Xx}&?HDnP2SE9InMo- zdKxkdROx*IWnQnxs;+DAuDAInzr%}(F|+H{mTUiq_506oT_?8ozJvEZ3#4kRqW}WQ zt@m=i>R?wNEPm|f1DYTI8m*^$l%s=(pL}g*uoJrDBtug|paRuf_z^@N4fWqJdANFc z^-XYT^~)dLz!gat2!N(iEd)lzybE9(zGggaYEYLL8T)Qu&!$Z{2DG`xE^cg>q?~$R zw&Zw>Hi`-sMwqjSGX>+u3`|ZNhd7uT9c%*u{(2s8kPo9!kPwY!>A*2EFWr!owKz6= zuYnDJR?j^tmw$`gZ+WZ$0wNnvP-uziroe?0(leUPBo{32HNwu9)bOMBl`oEkA&#(P zc_u1LH4S)m;G3r8>!|p+9o>F-}6q**&TE_E1hhIK@9p>3tH~??Dv9 z9?1-KCC>vl{9Uju)8~BNeSU7|d~FvhKTa?%p00w{pTC^H-IxQcKL4xEErXAK2cIEx zD{os1nL_SoS-=znz3}`Y`&@eg=|61Egk}6Yd5F9AE+H{@JDjPfqg$n~i~K}|$xEXB z+7GJUS;t^<)O$I#jRyx#7215$6F%#gxM0j?oX)ENZ;wAq9MTF*AqtrmHG)a;kc!4U zlkfB73?^k7ndve=NLFfkaUDxs~^aP!;H>e(CGm$(p$b z$=n=hQ>SRc&2_jV_`4hzSHYyzFEF?p0oc7-m;{(p}6h#ev5jXi7X zDe5}KYgaFt`9oLIfSRd#^2GB);9A;}fV-P}z4Kmj!Y}46gt!w6TT-C zagIXFaP->~BeW$pBIoI!4rWx{+QdvY-}~KM(&FnoE!{5YtEki+<#%2DyGF(^lg*i* zUv^u`@(&1r0kNjzw4`aLBIL8iTBlm8EJA)G5z{W*E(@rP=qSsiukBWR=AZDe^rH9x zg{a91kRXRLZ09o3XgSjyG{$h5u626AeX0nQCgyA35nKP3+f}xKZ&E9ibHT z#=!>VLio@Mjf?UG(et`@O6mD6S+?QFfxxJ z)CfsBmoQLS8n^UUx>dS1Ri1`AvGIVGO3~ph8YdQY`g$9Hh;HnXp(`Vk?Krg*NmL$k z>gSe9H|MeApkticDeIB8$Y!i?O`N6l^95~%U39BiuPq^aQR2J( z(iiMr?I;yiHP(i3rD7Qx@qo!=<;tJP!)6__eg|zzN&Q#53^5{?JH|2mwdVH)*+p_0 z%k%T&fE&t^1{w$yHe=;K?tl@^=DOHs$UfkrcV5}YdQO^{vfawC;*6p*6)8V8yHz_K zjqoY-P=%|Hwfxd`N~kehwo;9f2EwiMr(3%37q`SV_mbtz1^&p~ql}_(!cMI!k18xP zKL^=%9z|Ygh7V*X76G+ImA=h{O{aBIr)JdB8rO!d>Y7yo&RAuRQFOsYZUIe*sNrbv zV6H)1t^x}kyN{|!N-jzhkIS~)(t~PO<7_~eXJa`Cu?9P$yb@cS^eOw>oci-;C}Id^Ar3V3yBsE5OZ&=dw$9>?U&pky0)oC``Y>?d z5Us;Zzfr_h3pr=DEP0GgU+~}QpUJ6Imu_lp-^5C(H}cul_c~wO34fQ4E&3UY%zs?H zYfDDW_uhI+GR4+vEuwJ#+kUGZbBe8Q+^dtu5u0n<*`&H)Z!!6|62?6JoyWP;jhYjo>0wJ2T4 z4&8WLz6q9uH{KY#isy=dh_132F((xvDlM1e0Pjx1p~h^xy1{*CU`N?AO=jAvYP*FokfJYYV#H0$$|^a#99#8m#{ zqxB9xWD>28wre(C_wbIaS!9X3GKa%6Ak^n_H^si|ni*=5vUY!@&ucz@QcUK5lf1Fv z&~INU{4m)kH?wFxAvDgvJ)2v~2gs_!b+l38zLr^P)8X_IC}ePdXH!QckY=N5t^Oii zl1CS2wAff-N#r86?BOsjBw``6*QuZV(mG`;|CH9wvKxC7K^H`yUP>HhG^>%SBhP-k z{U@CVk0VBNy(gvdeU#3>J5Fa`r2`UiL~c$j?!yo(TZv$G+}ydTxz<0aMBH5M!p z7DzL#ctETeqH!Kpsftjg=B<@UYjUd0C>w!F$eX)zg_XIajd9_g^J~O8LMfI5ScXT* z$p9<;YUxbp<4e!KVq$?-{4)as$MgA_)6NY1#zmj;y!QrfB5OY5Rtnywm6h}bKJcIb zrIH~GZZI6!B)uq)E>}?!QR!$YfpwsO6;n+O2^@+X@EyijFyUH9xo2O7-GF(dsIF72 zFcJ?F8d;Q~zQ`Iq-0&0)KQ|q90!hQpVY(GINH8vsg(l7=zWsu|3!%^?B9ESd(CM0-p)vrTQi( z&y_D9caoVdXG*|Tjk%@CmIpPde4eG+YC`N^Xi&ov_4$)B?@ePT%GAUAjS0RWkg~$* z>%-~!9hHq2vOF!(jIS!gWfBf&!!*{0n0uKVH_81F3z$ELeTie1QkI5FZc&g`tXH$v$B9mGhnx+d4fRzk4ewpa zWYih(crpmI$0TaB3JVe#g~XN;7Fntfjv>ui%^vw`&7|ROaK3-jJMpWb%J{=yx$R*m zd?7UcPYov6t`s2oKHL}EXk}0)>|gIEzxuw;V>9`*f^Vn`(#h{>LX z!kZ$alO;waz#VA(!P`OHG@kUaKZ+x&bDXErC1E(1^guTNDK|uUO|kbwUJ9#Y&jt49 zO{xU38D|+hrIFeJ1y)g5Qn3{o{-q%f1HNIFtQ+<-^Dqa$(rmiV$+_u8xms#gUzrCw z{1HPInE&0a#(&GYd*e^T8ytT2+iVW}cmZYRj*ctElq(GDc{h;y#^-F$wzAGj>{l?* z*e!O-_mM$;+sa2q=&5u4s>Qa2?jszv{P^0BkRq!3&wl)XD%cdg@IlLd@Rrp0q;;_& zWt_Bmm*ygOl~ldEXLL#<6N+(OskZp&x>u=CW$h487i4%{EB4Jwi+L$&+*Bp=eI|mk z1LQqa6PYmt(P!bFgw~uDPaz~RZTm(a29`P;Q?NvRVn-B;Oe>>=MAHO`MRWv{=CD|g z8$pJ}*iE6Qn~~Cu^)w=nLwf%ja^#T)q~pGPoy&>;kbGhqE;*cPQ?_)%TAv?man2IM zIAy?Vi3@<&QY7pAxN{{h!@eWiKIJ@<;cQu5zR5HpX zAZ`m@ypSg#@vlVKVa}X*X3=K~PM8Kaa(7iwA6Q_t#v>YAeY;ua_a2aKC=E%Qn$JnZ z0uSrbZY+WluW~ zedcL84D}iDziO#eU3T;(!x0c@Nz@$*0EZ}cn_vP;iSsx1c_BCH=L}nfLXG|5zh-~U zm0PV1mr6?wH~`Y8bXgQIeNFc;ruV`6Wnt1&%u?2s@ABifTgONr=TT3cBx&#y>AvXj zlpeKO9;d4tyi~t9pl8e#ZpoD1ezb*jbmFgaV?Rz&F9&_2DkVpxU97!tS(&dC^Lewd zxy`Bn11wDlwnchbk^0vw61Gxx3<6AO_<@?2No|gG?$Svt$$!)UqZt81x_|6A@cj3x z)6$!BAE1BY&52NpZMeJInYzlEBzId3YI9N6kV}@K$?C)^P>J&@bz@p@@9es~u}i^0 z=joT#uPLE#h-WQ)5OI3rt}9PJtE|SykU97k^)Sm-8Y3Awq9V*7wl1-z)2XI3Ld@O@ zNawGm{&x#%wD5jUJFTrwGI(Bc4C2DHi<1g>8)N3Wl83zV*BjZKrB*ksUb5jknfp$H zIVjU5>S7^g)qw6oVS_HZ%jt8q=HNT8DDPMtM_|g9l+rd9xmEo}QT8D1J-@lOU9u5AhZkSLoBtiU1Le2-SrU`S zxzu7R?p7+1I7ub`mKoca%LhOOpuR#$6j8ADCh#CF>GN{t8_kSIUUv-3^FH*S`hwzzQbKWF2>w$!Esy@wgd-tXXG z+4gX&3J5d|fmDY7vXx43KI79;`YZE^SN;YA0 z&YYfCMD8)6$uJ#BK4fBg;z7ZbP?p4lV?-n{28>Biz;G$?C`O0CP@wu#vLT$?A?w00 zqYjZG7hY9tLh;W)U!b5<9dufMs!Y*R&kG!1W0fGFgyK@yB2E46DDAzL3A$ZqcEl>YmLr6PoK4O?ZL>fx20>nG6r?6fmJCbxY)QTc-p11&N*K zGLXJ40Mdu~JXoOg=lY+G@J6;%MuMMpd>gg=j_S~vYzY?HVS@Fohed;CqT?aFlPSW; zj_$%R#|H6`A#6l!A_?2Zb^BfAzv;$x!a-z8!lS=dD&Mv|g_(UpjMUQROHOIT_zOBK z{GyG7$Q7moR11!oy5qLiBV1fhSy_SySu<7^NRSWlgcHt$b;0Bpz9F%R?1rN#?|!iT zsBq)aDO5!c3?a}&gIEH#)MNNdLLV1V&^_4lz+Ovt@KBU>dK4+d)8q8KoU9=L}}d^ z8hMx5Q9f2?+S^$1s*tD4pJZ?Ko79#f!Kji_W`hkC{JZP)?BEFoDoFteF@ZX0O@BV` z`{D#N>{Lsim$mB{k|SE5USWYDyYL#(PO??|@U4oVgAC6+AvlDU0!}4}jfQ~1I@%+- zSAo6B)Zs-jG!X_HiU<)L)|367%}28e9sL(M#?>`K>Y4=KwkdTYk z)=X0wO_@4}t6CEdk2{J26DaTJ*vr-EeKR4Jj}OA;y5U%IR9vG4IIH&_%=r!}28($eR6_z6dCMot=cg}gP$+cn2d=DJGV zBdz_-R(j91)hsFV5iytC(4g{SP>miOeDiO7UZSMrueq8uU4N?du#B$~bD@n?tBDl- zhe${82Z{PcDb)k_Ar(D6gNcc9%>lutf`(;8wh6f^%`nxu(ux!)fZDurB`8bwG}p1u z-E?MXMMy6Z9YX_WmU%on908n=Gp@=<5zLz z6E0?UyQpWEqA{TnfVXj`@A`XEQd0K~M~2>qe<&=S548qsUNdNjxdP{{`^hV3)v|T@ zb#?o9X|j8pq8JLlx4T+T{-$NvDgRl91$`pK$x&BJteSBUao_F^8@~_E9O0iXFPX7F zbt1LbdeGc%la<0h$2Ny}>cdn`zp9Q0q%#HPQotECy6S~74Nn+Zb zcTIJ{N39#{M^pm1E8OMlqS6J4*$Lu;V^<7U!}h5ugxTe8P-$=hX7nzeNz zz66>Ajp`Ivq|f*x|E!c+xFua=9wt}x{16$FXUuL2n~2(#dVl8wv)3-$&3Kn$DJeK4 zFUg{Ol^$@ivn}|@bZR(i zYPh9k;+-;D8U@!d#Rz$;m2J?G^P*rK;v2~JkD2`fw>A0ujQG%@W&U;-;{QdO4Xm5!k~@*E4K*;6RQ_~4e~rxKS+M{%n|iVi#3ssDg0yhm|6 zFTU&Ppy$4Mh>SGEW7FO5Xa8PoHVo{*EYY{Q>QchSlCRND=)@j{oZmxi^cK*paMBdC zCn)_uFd&eW*k@IBILlS{GW ziSpbj_BJD&uKb52*DabU-YLNv1efI{Vk(hvsGbU5F#DNke)MD!aQKHllSxeLyhZe4 zxrxfA1YyFI(W$Ze7LhQT}!qm;OdztUvgTvCZL@EA=2r+M?7`9lF73GqfL7Y?eUQJ&?6gf0cf%nT!q5kCiG(MLXYR4oq#o>R@z=#!b(y>og z+FH81RWtb3I!M-btK!DK)Zk=ltm1sV{(RDP8Y5ufa=7cqXNCp?3YdmWQzHdnIg)gh z2(7FR3;9354;w-eX^^R%jekM%#Mje3k| z*CxS3i28Pv_0n)hvc}zNJ{R&XZXyn3j=QygZ7H|bbgoEb&#+1%MQQhWX59eLL3p5j z+=rOiP7!vNADzaTH2aop!&OcMH&A=e<9sEGG$nGY2XmS!aPgN7$#u%<{zfSEhyT1nBiF~k>m`VaxFi+afJKk0|at@0hlIPV!CP%8;y&enk+bX@sSy-Y& z`nm+~Pj1UN0SJja_9Gf90x)}|*sr}y+{nIaH+-;T1MbouY^-@nnJW!nW8JXfwAhN5vxbl5^&L)ajWBgJ>s;1#8u-VWhCR)9(P*D2&672cqP+>vb1w2>z%X_F5Lbz5`59n zYN_tJ1S{tah>I`y{NI?s-p|)lPR>=NzpXMDv-P%!c;Bw>c6?*$Fwr3MVU}Z}(%?G` z$j~GZ51y5TVBfDB)})9ubNuo1$A$UAfDZ$ybcvH+r`QaVa1CF=mSw;ph?F? zdTtQE;)Q+I>h$Tqq|NpjRr+!U9Q+r1LgY{YajHGKcUhi=*xH^#!?$cS;6MTKK>;wR z-^dO5zU{w@zb@Q%gnT>w>#6qp6B0$gHbICcSH{K9=S!K80x_uM`jZ|*X-rj|^qsE66 zzH5?^<^z5;bf~?L^$seZvhpv8C?j9ZX2BUYPvb)eJ?EoGou8@OR)_TSes+qA8sKo$ z5-a&N1VSANb(NbLizUn;$-dyII1)%rFPr#56_>cGb=LU0BFA(mD3XEk=c2GICxrE^ zE)K?cmbGP})`i~}I1YAoW9sHPo=@6QIjnv@yf{G0Q@CIB)d-oKE*v>JJ|fo!B7p{z zHDD{&y?qb8is|qvkYL^d1{{K%MHguhKRox!V}W(Mww)`B?P~;48bDHS+%?S%VXAO= zjGrj@w@1UHbr20V)yo>I>`=v!W@$BRbyO}12c`-cGUqR-DB z%wHn@BlL*K`#tJ=u_)s_&0BAJbs^{UmWBTTmQ+vzqdY7tMWLe9{{Ty@CAhft7W~*m z<%iE#2{tnsOS03qLBjE^NvUQ(E8IoTyA*v+-9*H!CQNYX5Q7rr`La0t&X&x%n(E8x zMQ7--BJ$qTMP6M~Vsu>{S^vFEO}qk%3Eqcd1hhlo2Kkisxf>J~yvAKbX4sLWk)v>yV+PS$C3+8cv>iA< zz;N~SXq20aNc~q{wOF`rk)9deqr8TeJVFE}K7h}xJ0jWTANh|}D*RFsRi79RNWxTy z4y3|l=-~DWp72r&ww)cGsNV&=Z_n$ z(dOA!qdZNPweF$2D-wEN*0+ew(etA}+?9!QWz2)2nDULHMR*)6TH}#Oi6hU-(&Bn0 z%*|uJUSqhnE^-<4DIou1+^3^n6Hj$dcQksqwQ!T(|(__ygQKcts~@c>CaTyxkZ63!d1^ zn0(OzBRSGXM>cx(?=zqDX%eh0of5L*`w-YcS=0t1 zksx)Rdyl=7q=hdW<*D-zq9|G4!+yFz3NC_fG#)DpTXeZIuCMG`x}4MrNM9Hd74pZ` z0nUahC4=%FJq)ND{@s6%0fQYl+j^D9K%8hK(f?g@+|*W2e;frVkPknequ;qb=8in$ zGpopS^#_U}Glwraqhm7Qf*nb(NZH-m@1qlmZBCMiht0xZOQzfJhJ>=HvQg$#nzZq4 zz1G|ED8_;39NbQ|5jr2ju|Xh1J>bkqkKcxlB-T^T@EPDJ*uhKp-nQ&t`(R$Zi$PTL z`L;tUl6F(4RZkXi)Xg6k^t{BV&UL~+dX0&MIiWPQYnJgWgU-r`Oo$s4@Z32OO!D{a z$e9E!J2B9N5Zn&Umj;vmX+qNjnkjBpAg<%BZ6@3ce5+VRuzFefCaOO`zeZn?*_~eD z{K0CGV3~txl7|K;7%fUd;&9^G;>LEg(<73${USZ}lq0X;Q3)vJsKbhW=eyCxSC=*R zN)vo=wLmVWVYB*PdYb=;C$5!e!cKSk1cUL1=Y$aG=YLk%cUK=*wxK(Wr09ED3f;fw zRvgQ+)0n^anfj9VPLV!zNk8k8TLIyKti3(zv}LMd14+S7w0WJK4vLhElSp5?>JqUh z;y_r@VzSvcW86A3t^fX1GX8 zWDhuO0p2ex924r#o#{vN{kx=j)Wpo~h?x2>)#dW8?;lA_+?i1zVzDF5q^FTnJuf5} znOE}&#s&0LL~MO=sd{WG-)*2brLX$r04@lx*LyCeOfFl4MWaoE-H{^|#l{t}^i#uz zaGbHH8EGqen{QXDA9%}#Lgn_|pi_9midy;2r|2KEQz)Xb6h;A6%rnHBPmF1fNCW!x z!z4{7FMB-l%74gHocRTAG|M(CH+NA9SS67ATP2uA3VGH7t^#t_r#%Ix{`d# zZ$!Wo(IIOSd>ZB>yY|uVoQhGFrV_h+!H%9R<(v$TCPcoJFp@Q&LJJYAEe{rgW!6=| zj0`zP88BS5^pqTi9nkyYpm6~jcDftQ#To2vwap0A%cXA2n{Up`aBR}zl^g$Ko)z;0 zmWS{m2D4qD7y)2F<e1`CoylHBJY~e+!QGB?-MAE(d6^oc$l#)qmE7>OMcK0DmLa@&nPCq*$H`l*%H31h z(tgKPm-$ z3lt!3#{c5Q*>{gasghX=OA11@-QRY`P~eb{Q>8H)zy=m8Y_|B_-JA#=y86ClsM`ut ztn09_wye<-Z$eVeskvz%{rja(Z$xmy zS$E<(^&=wqmFG18B_wsUEif_O`K;xwR(k{G$I{rZm8j+I2%=*(8m?nNfvU%=mYcdn}IG7Ore=l%k?-w5)MAm1+Uoj6bUAElXhJ)KbMEg;kifj~CRVHEka zpcf}!GPri#l#T#ddHb^4t_?j{yP^b6OaIsP{d3&GU5i z;tMc!9ddAUK`nidMSDcdz}(c@3V+B{F41 zHez*e@sbvIrggdv_sY9nwKem*&SfBkV{ zKjQ;5Sxvz9Nx^JFI8M1@A(?@gcmt5DOUZfL{~dvi2`$d{nfAojU3G>XbYyP5xe`Tx zJ*500%pH3Ql;WZY|0ONKoX5{g@vM>%xP2IG7#3w170Om=2sHu5oo;uyAr^a5xOilv zvI8^UJ~YvGk;l0AHg%t4kB`C-lQr>MA_c%P;T^do*QmsUhgRo8T8m00bdeOOwZ+f1 zpY(v4r3{m;<*}8*o=^^kiG&Gr!G4tL!0~~sd22c#>7?B3bo&#L!HAk)CQ4QXr+?DI zfXRuvSMxQk`2O)BUk4djIHSw4e+9l~`X*nd}OGSBO)TR46EuE(7HSXu-# z@Vus*LBq^opb#H%A!YSe4D(aFA{kZ$`DP_;Dnm1+xMH%lVKk03U7=>pye>oGMf}HI z{=MhUbA+IZZLN`J7u_?@z>H&;mG#{g3jBlcqY&j#<>f)n--X>R0{*YYz7<&_Z)^jR z1^pi9FBsfe9lLGuMJ0bB7 z<9jSHGo10~&3cEfe98CW(&13ZA%ndX#Kmp&*c~oT)@|It`S4`S>}5NR_RV719w`&FPoH-AdViwZN{`MTT3G=&EqL7MMAI+ z&@}3oSJ){1%5$PooLWkZ?A4q@U1OCKB{Zmr=L!?#p4FThx^z zC`+XaO^Zg*c&%nbHU9e>37b|P@!eZ=-9+1!Mv76iefy*Oz#^WG{Wnbdo$3{*Gjs!= zD+Fqn$}^s2o{aXe)|HDaJ~Xh$+oBfXR~hO^m=8p!?z& z1^w}YKt3F`1+RJ)2pOn-a(}v}=wiV_qIw=`(`Uc@edG57iI(2%#;4QCbo*}OQQufa zWeZZMs+QP0_a^doH;eN2v?7x2d-ZWb*z=`ffjlJJ?{AG?BM=5tF#m-CjJiPVo%GngDE3ZS5`w&> z1m6+mT1dB-2zkD5aC^E5K0G3=KRQ0niyJacdGlItfBfiwHGFR0>DsjB1}RSmdffZ2 z)@ALG$>tj4=Or=_NCE*7D2536=Tx=VS8SsnW45_nJxJ5muQ!ZC22|-(h{|Jaty*(` z!E5Qr!$;Q?=@gN2x%lq!T1)Gld#=JVUuU1latylqZY1RiCY<3bO1mTx@v8w-JEGMG zMwDnaHLiZvUsg(URH{reTm^}`h~sI79s5|~h+tXHwsBS_ITiRR_@EEH1SQB*u6h_; ziglfm@0)3Pw|$xsxW?A*pd+PP&Yrx2c4v+FH`?KBuiJX=^rcQ0Mqfz$|3aM2COZF3 zzh2RoAKwAxQ_rDLSgr$`(tu1P08paxiRKd^r5LIfIZhzvb^{U(%w?8YOB4%d*rjtF zw~Fi7X*T*wt~8+s_gEi|^MI(oZ^fJmx*!y+p?8RV6)r*-~R1&3X^w<_g1rGo1gidfS2`qn7k5fag9m7Fw?cqmMS?pev}kdMQrwF>oZaVr=X~?d zoSAdZ%=hQJr$5q7W@q2JFIj6{ThHe49_|+v{^drL`cTq2IPo$xCh2SFQUOj`)@kTx z$Cwi=S(sVBHYAxEdq&nNiabN=1erZ2zucc2EjpOVJ8bLpy9NjMrAqXE^$u0`RMMaT z9SoA^ZyfjBgX*7q0<}smfIQP-UHfZVN$QR5*2Q93JdOv1d-EHZIzv7jcbsR4EDR?j zxuZ{TL70+FQMR!Y@iUupI9MC=KF69*Wgb3NCYt#5T$Hlz@Nt)UV$9$2n9*VS{W5EDa(gTa-&`-goU9Z(Dv59QHhu=5?2^a~E7ij?|X$qF8WwF&{pPuZ@9;Tp@z=DnN9E^~xaxSm#qmMQ*016Jbj^dn)2)R*H>?b_ihFTQPmwltUUBHM6njhnu-s= z&!H;3bh3d>)H&s{f?v-b=QB=nBy6OJ?j1DG47~zb=?DRXwWewHix;h@L8LMUWKQE5 zmE0RCxehBi3digAun3u8-R3eHxqjw}+HWi7_>*h0r|1UJagw$aCK8AEBgF(nsE zthwq$`Bp}Zv3dn;iV}SCmf^|gz8#Mt5d@S~=pT_XRVS=47rZ{mMghG=qWA-ba{QMP;IY~GOp7eESPZO{GEx@*~i!a z8Vc#-{tL7rMBzz^njz`n1&NFEPVQF-5$>AhYQ zirG|j(*+~QSD(oDhv2A9mip+>XPEI%HZv1I;zhe0&l6}wx7Pfa4=&df44whU%`bl+ z1-2XAv08N&7-qSoTQBy_(xBHv3;TYd$JYOG(e4_T{muV;_$wk-k ze6rPk))NM5x;Kx)9zr|aGMq$oN@G|{TK+zK=?Xafk7q=%$b_i#!)7-|0;{KQ9qebl zjH!J?YK~$}SaKgzuoWh%cXuc*LAkiI*uoIAdFtKN5`4Wey6mM9E@mp-Fgi^JT@ce_ zb~YX2G7|e$hOoSe$#|fdys8xba2e{^(6Xs6RoKkWU2eZs8MyXIW94%>W zWtIGCy%@q)Y<2;(OsrBh$1vxKmT9A^c#@k~<;|cAu;3;YzDVrD@L`|r)1&f%GaFJl z5wDXkH&&;J$*PS3Cr7CASxb8=#A_=43>yBqfkF!$ErmMJN$;%GZm2&F`GPQ0>8#>M zGoR@4KyWfbtoNKWw8Rqz{VM1$mHe?;eyEb zu7R#EodTvdbkir6xO-`$xMfOB_bu~j^EG-9lNX(P^4O-;nM;S>E;9);XI#cpUVVb| z{|V6%Z^bhdGK7vqz;Y)jNWui%H~orgn%y29f#Kp|2)k`eOwu-DUsY91_+b}QD^rDC zRMqQ$a~d3;uOo zBYaTU9?pP=T~K;Sci}BUOPrU7jhsX&Q}3c$Yu0 z0g?Y6#{kiVD}APfpX))OeE=~X7CVlUK#c%yltG#2?ma}@v_`dhrWp1u<$gk0eGvU;TCQKACWS*8}`ZRXP(ov7j zsi=`jQVlZ1|M2i2P#C&oeylH&*Rw89g~_y9q|~C8vPv<>v!y+Hp7mv{PCeI=!3vqK z`234H2hPe-Ac1nk2k!MIyNiQEC06~8pmwn2rt3i~1|gEdfB~7FSjq`~_A3qf9hT@9 z&J942Ki;fUSSu=>d2X)ah!0!OTq3|){k8XU}& zT$|2|1|=Yq!mxYBzadV}pRYb-^l0IwZ*##;uDd)=-q0dr&KX{-G1g2LRxwn2YzhBT zlJ6_e?b-8-?NlCS$L3v3d3I^?2o1!}#*E8{?GxBG%?Ue&vP%ouKdkDs`fPfQ4?R}qa8CPfhw%xlyFIqZ0u>PS2aR*jCw8xJkN zKm6!glj@UVGqqi%LKF}}J_Aej8ZS0{L+oj4jm02*KI{9QY?u?bI{Ie(Ay>`7&msU0 zE3$I+(JZhalhm1t6H<)KZ98{3lb?nQu$7#CFsh@fCcfrZfP&{Y&nwqrx{h@2b3&-(CZ|2#D#qx+koR4V|}=-2PW8;=vCUJkAKlsp~33H8Nz@F|hfE!=uN z!1>-yRNQ@(-FOO6ByoE*Y-{pLnr!iGQOxvarN_Fmbs=;22p@8ode^$=cs^sf9+k)80&G zV8jta2M&Ge^0{*kA2i&Oe$D8hp~c)3MI6N{X*VIHvBjHqEA?I=6L z$KBXkJlg`-7>f*9ZaJs@TB=9BGr3TPe&8oB?PHqBt-oo%mH)QvSin;cnzN{)%+S8C9S$A@3^j+E*Ci zv@J>Zy}>Fo-~qExgp72D9{xq!Se`(8MS3ukLuxUtvWoI^vW{NK=+Y#gXei!R{8{YVGj!u;Uqxuxcgj* zsqiGoRFUjOIL>QaOcACSlE?H<{M;y*>6$h?a$<@(?+6sW2MTf(Zivj3jgFKckpLbm zUtK0I@pqT&9;AtV2nZsf$#5Q$JceI#{PkG8#I5h=%q;N4MgKq9zxVbfzr&nP7Azw(U7oVs0Sx# zMV#;fp(HqD=p;Pn0%2SGtGr71{zn2%oVb-itVkgg&%;l(oJ_Dv^HR5OfTZaxAc+?g7Sv!)9unCAoFG8OEw%DeH@S zET>qj{)*8(X!)v?^8^~of-T@BqlMGS+xf;Nv;W;NCc1&fb;|Pns|$HhVeL~1{L0pe z@u}IVMF)}HUM){&l+E5u#-IiK{)Ab=q{ZrIPl`!J$u`h=@y|hTPETqM+RpDhf`e^> z*a1oG%4i2|xlcxh2PoKu(Bjqf;y~?Ry1UJJ{^BT-7 z7nCjib|xi;aF#AQj)gl@@nPJupk+wGYnKu%QxQk>vwqJ#MF{est2 zTsN``STVR?BP$$}xHIKFpgr<0yn;_@tGA;d*h4qZUy-%1ao?j#jIiH?y75v^dL>zi zxvx`EwK8;pLX}Yh=F(C&J{@MEU-M@Dd8wJPLyw(6@{!*lr*`8NCA_=$PjGdLyWp|9 zF+vcKYPcrF9RBvkJ|Ws}K2?2~5dg+`)Xmlv7NY)o{){h7N=mww!`bq@+FVhax5+4f zI_&{Gi~z?y4oLR5^%Z6r0f7IKTnfU}#QZbQ@1Q8UAcp?Rv`Vs5l zcx7>2=3 z!|7MlhB?bhlK%`lhT=y}i|r`Y`H0AT+Sge)zF>0yTevzwS`&c)@h4w|toBv=7mzlT zo3_ox;ShFWB$34#K>a}jmoPsJ;=r-L!!+^yXs_WVz|tg#V5*_{n$!J^KW1>egUYpb zzsNl5X|dv&2=F!WDs-EUD=KTeecj%^9JOQ=^fxa&15q`V9gyjTF1aPT>q#+fyBz}f zUdG-_r2)&C!ZkVmHzRq)q8J#7roql-7`XpI$f{e7Giexl1?@vR)OkSkbLyolveTbK zFP)ooH2?*-AJs8YifR2%8!5>bop33Ll}kt>$&;WzswduQJt0*`s`q^6bAVA$vOcWG z5!p`C{-IORW|z@SlYYNkw6}m$pKk0of3I@DFQF;v%j#uPd<=}IkME2^mMzo#jSH0< zRkgyEu#yR|HUk94S;9#X<*~ zf%5ut#oeGc9mf4a23ZiXvwSF9yx7klFcA9XAJ0TKxAQS>sNHg5iV695N1p|+DFgx` z*#Z<@19Ao5%JZ=J!zh0Eabz8vecw~315Gdt*jsi(Un*kK1*wR~aWlcBEvs0^xB;dT za6c)fk<%a3RbNtbfW5ayN{97El{w zo&2I7VxsJ~QErn-eczSIs%kSSE^wR7%edg5OW5lq33b4A@v=^>mC@raHVjOKiR?OT zBSJou$-_9Q7GA9{A;kTf`EL3e&f{_1(J>~QSe8!HNz@4OTL?E><5JIU8O$5_MGShZ zf%wN^8P@3tbI-z2sK#tskr8H2=i0~>9cT`rNi|jq@ihusGx+)h_L&M>ziz>(Rk{+H zd`Wp8^yBBZyKniMUy3!L<&)z&lW}A}p1q7@vSMg+Kh&caL-)(?Z-nJ#13%Ky@m!?? z<2jC*u}pVfiDhB#N7BEZ)b2cGT#9Z%h5R0n+B_0-qrfr`u25$S=)^3dMica;k>h-R zbjmYPOPQC)+`G>J5RZs*(_9c=P`(X}UIjk|_Cp8eLmTFR1{ z0WtWCoo#aKQ-ltmJVihw2}B|+6W3bMeiqT145;M{-P0xfH2Qz=9qC~4KPzxJ+fGjJ zTV6)O;zZXs|EV!hq5vqE=08wKctyz}SV@n=ygw=;+b#7GFVk~uMj@+`!_NeQpZdU9 z`m-XfM)gkm8a*q-iZV(2O(smQAGS+FsTLygx0fR{|6MnbNphLlIdLC^Uq(ExHNoKr z7xBAl;qxX&*s0OO0IA;`=f%L=q-?dy=H>G=WfEG?Ge1&KPtWag3ad+a~z zr80%mW-}&)ig>`>dQTf*3Y$|&(y%|p<&)6OiIMB#?3)%xM4pQR2iC5(=6f?{Zfe!%iRtlp!@0g}wR`yN=51=Km~S({L>0 z0n7RCCo$aIOuFS?<8F~;D_^193%^(xKVzFwnZdQAq!2uL=xkp8k#GBhI1Bm2NfQ}E zaMd(qFQN8Sv<=hKpw2{=u)JQvD*G`98yAw&GICFisy?^KDp7+&GVN%5nebJRRFB_o zO!f#waAbVzW-aEu2}@z%Z#6uc_tStJsysvWd3HeYHgfaNeC!svn%2w`K z{1uv2Uiqb;hA#{4(rcv@Rg42`UK5QezrH~$G3H4OyfuTLy7KY(DOzpgq7bnL0?_Mb z;{^#Beo)lN-`ejN#_oGLwzJ}-10V4$f-VPwZ;!=w+wK}KYYtwOUsNuaR<OviGZYJU{B;HFRGJ>zC>v0~!JR*TZ5D_G%&D-mbu$p~OclL_j%lj^#Wae6 z9L~>rl>ZhHOJu5xK1saT5=tJ7X9*hk zW^!?Kb|>Pk@I9q|>?cY_ck%vr&Wq`HOMka%cI0S+-Ip0tjDz?1`T0wma*WTLOXDQA zR)r0RVr1zhgg1N3ot>q00st@dfg%Xp!D0)3n~p>~wKIhv#*s$S8KPOSwYvL?F?Prh zN5b`KHcRD)cosGVZJ@f>Y;1{PRbT8g*wT|7U5;wfXj%PgM|VFP8``R6<_o}~f3OCP zD!=Uo_mFFtVp|MjPSrM*yvrQt`MC^d`Sgy2GjL_F{_gbL)}Pbpf8x&!=oC?Q@CRuP_BPHv{4G&DvRZ)qj6Y7lz#10a} z6Os1iN;8$46*F*E1UueDL0_$9Bga3O&#_G*PmsKOEf=pt|>Syk|^2F(0o4J(n$znXUQ?2@*ef6Pls zb!42Hw=K4$Y*j5=_Gw-B;?lcw84e;+(WuXP*}`9W(U)aQOx5vOu`t_zea1030D2z( zKW+Z%?X`5BFY>QoIsyWmMh%;IKKEbnZ^m)}>@x-k3Os+Ndmronvi}qkn5aAkxIg*Q zf=)TcGKEy@;b&IpL0;>$A5tCPm}F=pBLwRDQjJOnGiTo_g0nOjNO0u78uh)JQ4q_y z=6sCnsEONMRw2-L> zG3BmF#(kD$c)}R>=5ESWsP?*|*%J^k?B`l~zhMmf#o2r?(ds&sIivRGjs>y_uNiW- zcU}mZG9D7T)dn5nVt}lHUhFok#ER}WtbEpe0-*DL)FEOjM>%5g-2>Ns2XJ1|VgOu9 z7>GP3JZZ~vH*JOfbb`F_Qw~W94FIcj35U}O07mCr;t%wE*WEA*ML;MBmU7NXCDL0Mn6?JlCaGp|js#SO+= z)fX({{l+V%C!^a|FD&DP@1`eQ-Z_ig{?i6ILq+$}1o;*_-~i^*bGSJMeKe3;LQ_Ph z<_!1_W>w%5JPdmn^2M2L!p$<_u&P|JT*uzNC_6?+8%A$#Jaz0`?!vRh6ZXmss4FE6 zWN&54n!=ES>kNpY+}U4np#!BBcaA7s!hvIBC82($eK$ic7rw#JcR8-ewG1#Q7v^=V{joN^{*x|m8OQb#do zH5UKL%lt5A9p!0gy(6!4&h=$-hXQuhp|_f5?cZF}Xy;Fm|D?q+A3g7l4RV0iC;&H?P_!}hO?Q7%DExTi$t(aA*x0njB6W8x$#oCVBE!KebN! z92Tk-7wUdJw=823(bRTOdsRh6M2$jWCz~#B7^LRR>8U;gxHGcFvh}53gozK`rvOu0 z;2+uKOYyHEGqP=_1SC1zAXiGkALl>)j3q<0%!Sev-Hh?lAo~7XLI)|WotP4@zJjWx zFc*AC7ywiN5kktPWx6w852QB@gkW#%JPCR7o@wm7_U!9(S-oiKuL(tIT8OZs;j-e= zvFDHQ4xVY)*<$vhOKiLXia!=1=ZE4nC6k|A#;kt&OkfW8Y8KFJeuDk3%6&`LZ5%M` zASW+C*b*PRFQtS=*)U{*BBR#niJ#s0A27ZkK@_sTRw!cJQjCOAX6fRBXhW);Sy5OX z%7x-pvH-|z5*pT(r>;MQ^H0%9k>z_S5+jTJbl> zofGbrL4}09sujzL32zo{#H<-~j0k%A=eh9rRP8SLsZpGT6ZiGPLZYvypE>}v)QRgS zGO|ip!NQTjlTGPbPszp|KyrcN3lP!)SqgVmbi8uHz@$|fLyzk)n z`@!SN4z}^T0PdP4t@GLH#R<2Lai)$gjNeNyfR5NY95P$*Q%8)tqWFidZhhcIc8aRP z_X1E>&J!k>y2He{&b!7Ke*0>TDg3A&{?-ek%x+zFxaA>Rf2`FmHg#!#eb6+ukk}6e z+7Zm_N8(oCPS|XueG=jO4+B4S0;}z(F#);*AbLJL80|y6D5`oh+<<$rndX@UxQjv#j$Xu*7vw?iONzB zAp4xCIVR2WvG6UZH&6|=Yc!@<^ws#=+ixShY+340*rW9I0rqM=vkE`6O5Ha3u5TAO zT)gt^6~L}#iTXXk+4ErK%ZDwUqzsd7QL2zLlx$CV8%cXW600@Ot#5JW*EJgGxcTY8IXqK zT(jGX1ZoOs8)SetTr{~07m>ILdgD4tU-*B?kDba}qb&6NS^i8;2)5s=ejes!mPmj2 zw2TF5EyJW-HfIyyu`ASCJzMK+4O`piFaM0((&pZCMU9R8s#_%3(PK!M8D__*ssu;?wBbLbm1Yf9e^7>H zS!SP(L_!G+pZ4244h6eOk|!rF)gaBY=2ZkRCfc?39X?1?=*_|g)nIPGJ*f4H{yz}jRl!uMb#t9vF)mGCc zPfA)=NGQi59E?)zei2h^_kfr@kVS@PxfI)yLzBkNyFA6&!o~2nCyxaClcnDV6ks%n zJ(UHS_G$wx9jG19Zz;!|IoZIs7ZaK26!?XFOx0QEbXZGV%We zf;>2RV(FBYju@HIB1q^(r|?;))UJ~9uagK`Q^p-W{}w>dsi$QxBb}j5ldbNkR#sDL znO?5dKh8?k=%ZPZznOvphYv9Owz-#0w2yCWUTw#dv8i~~E zT49!EQq{3^dqb(g7ZdwUHalBeg_%$F2a|X0(D2ZZ_dvhTP=8%8Z4WlJ5QS04D@NdL zYe&m2z^E{F6oF`&`ISAK4b{qilp+uzvPKC`m!^S*yOiMB6Sw6HLz<}uayxB#~ zS{RiN3NxRpL)iaf#5$3>)X{$R^HI{{ItOi8JDwGvHD8h5Q9lW@ki##cL@5Minid&~ z)7o@(n@|ZReBM~+ht>T*i0(9u5d@&LU`~32K$z$1vikO|%Oy)8iB2}Eb zJk>4y?fHZH^O=U@EH#&J<%KPzo+c6Wt2+>)3!45;J9sE3k>Rk0(E$lg_{)WFzm#Yx z_yo#byY*gcL?Tqz8jZYXfK*puiA?lk+W|LO_l~#xDyZFGsfSg-`V6aDu3Cw(Ht!x) z(c;CeUiOUR;KOya5!18>Em!F|{aly~JlY9#mjo!;X$E~a^AS{YAGq_P6i)RT9Bhg$ z2Y($U_jIy=>=2OVbdJTyc8 zHsv^_{l?mt4Tw7C4$tE(^oMUeYr7`(|8!mQ^frOuvd#35u{`Yaj=x{;plIxhuS%AQ zlB~Vzl?s)MVanVW?;xGJ<>y3}5Ldne#x4Udqe_)j{+Ifs^oJ4VX$X7rusn8(p~2nv z@~olwfTUT)%KGu8)1+bJ!+QR;^z@^_7*#pBwsI8wXAd|}TH{|R<=^|YY}-Ivt5OhN z1)=S(_Us;ySynADrqfg$O50gA7CdOY3`9I?#EGnDSVD~ta-4g-Hfd`OM0JPSN!FBl zH4jC=CwAS8ath>_KnQ0Lq9UWoJb(0_#4f#VIe6SmI^cS*zJAHwyV3DM$FI#AOS?+; zi_y^MUw?0qCF(A-TcwBDlM}Oq;bYg=$|abLCvq=TM%vlSc|2U@-MX5`fo*t90Td6T zAGB(u#;K=2eii3w7#D`As>G0sJl|r;KkiG_%=+bMVPNcK1+&rxK|--6ch!qOTjy5@Q5}d6q-@$*` z5-b?@Qme@Ywqhy9nVT@7ND6aJ^=0aC&8^z#Hq9_`O?7>`0fbsH3G>sVsCm0j zn~Y1tcRZUXF((LLzm%uP4?|EixquLFzF!0O7r4M|8w-q07Ulvft@r^zLL)pUm26sK z!6n~hGKjTPg?89D1s}!YU{QidjpR8$byL;YKGr8=fH@C5KX1#inMKy&2tP zo>UYkQte5~WF!x3EhT*90wrYPpczQzU4beAM(&(CuU9p(wHxeJSP_YA*a#v*MHU&D zg(7C5v_?e!E9go(=WC_PvT8QA@;g>!_V45GRsG}{% z3J$S7=3(D9{9=_BGUI)G#VJtGGk)G$+^ii%y@5|{&ZZu4DLVeIdf&M#X zDcY#tCrGd|Au&vIK+Qv<8=%Wjb{=v3hiQ_VOT{}?@11|;-lF=-)SMm?>F3IgN8I!P7dDqM$* zs65s>$?jxw)I@``uC$6w-q%Xo8-F&qYc9e*)y6y$AL5#=Wg?j|qJ`Y1yjcN$j)`ib{z(p{D(Baf(61W&+22;M zAY15PTXtFa5PUiWF;BU=Io3wzamOH^M`cW-!qV37k3F9_jrVO;%-f&Zud z|GB8AsJ{>BP5!6JvH(QI|7s^#4}oLwUwfMa(BJ;6{U38tfz^0qxTQ7MGxXnxP(=Mq zl5H%-=&%!T+-`8!HBV$W@A!a^EUS2Je+m4(19NI|*2qv-ci#X5?7%O`pVW-~mBv#6 zFmC^~{|~*M)~n^KxacLsBQ;(Vxs<#2&A_)vEd`|EfF|1p5COvvPz(0q2P><`3rhPJ z3)cVs?>f2Mm4J6YXG%@^BN@10Tp4^Uao_MAMWJY%MWvf{9<6OFcOL)5^!-)%PUXdl ziY`5%KSI-bjn3I`XbU6;ZvR)p_~#HsMN`>43dA}A*y6_aMbJId>ejt2xjWwF3?!jApe;eA~T0VvBKA&|7KR9q3^=pTeF2zd(R_?9u zpFEQIyB>6=!J*?eKt|ejj%I(?cESZ6I}^T$cyo5Izr6OY{p?Qy-oga1dW@%B{|*ON zt(}RInut2~J0D%%05kXIZ(P0q-da!lo5T5pI8X1|iOIN``(Iz{7yPCkP|J)2d3kw@ zxu2`#+@HT0x*T|2C;q3v!y+=Oi@F)_+jYd1$fyZw&(U#IG;95lQ`Ds~QrIPIXOgR=dxMpbL&mkC>IytKBcF zN$)O|tG~X{&__*)KidvF>=kuEmXw=TIblIIXH885cYoTV35gdNxo-UL65{S6Jr+HZ z4z9T*&d=Yi2g|T51eQ8`PhUO*t?Z`^+;1OGnfQyRDCka%?>7#AL?SEnV@wzsL`6%H zj&%!e8%CS~J6^h3?n9w%grth_C_D^uWr7cyWH!}O#t2^&o1U;8-wK)p{XKImb=biZ+tpK?Fsy9zU9_b+ zokbtF#MUKuwL6KlB0I#-yyo~x+pn*bJlbpeU9PE54g=oZ9wL^A>LmUY)G({e75p_T zmzyoE3)^fG+ zX0)2Po+AMK#K6TG2Evuvx2aVNO0go#inmxHC)p+*I6b3Tf(&834`Ce z4kgcJy4_ii31*$%NMy@mK53Hxr1@0EZjDsNhFVwqvH`}k-O>0~ZVsUO^4k*ui&U|0J6c_A`zD$WB6J5?3zVRA#DANy-2G#&0W?s)y zpbDmb^ib}Ei zds;bKm_Sam)UwHYu6zHZFTU0j!tHGu8i`asGspP}_Xo#uiy2ijByZ|3|CUdQ2kxM{ zR-AnnPZM!~!pEnm%119mWyj!HTDDeA@k*qxFSf^^%e^kwcdHqGIa~&RCxDMu|2;pB zYt@Wef(yIAA@hQkD3Wp1IIFeK-q9sp#AC?7C#AST5);KQSmC7v%!yQ&*AWs)aut1+ zZ$En`8iYs7=eK<#Xc|C9>p4Br+e?@P1GJ4?&h|4ZN$b)CENyhF&vS0uFlqnouobm8 zb6}NjzV_=+3#P%RJ>~Vr{GKuR5SG1NyUJqZqF>5FR$wgmWGT60@?RjwX3a~+x|aWL z{T*uWMRZ%5gvZ%>414k(YU!iY-=_~SWW#u#;d-+i)mW-F?u402ToyGyyC6^*!@=WV zhvBoq8NnPBwA!g2_LuRYjmlTMUxVjwE^Aw>U@f>;ZME8D;-p#DLlGO%=BrH{^qd37-v12MC81sPXK1MWc3) zYAsLVom{YMQ9WwpfUuB9lV<-rJ)W-vY(I?`Z;l%3?~iBuhi_ZWcP|bO>X@1Ccf!<* z(dmq<9>StT4vxG%-P-tS%N-w^)t)gtYF(?APH(%F@iff)wbVuD2THW?Dm(tXnZ2> zb|WD;`+O$&naO?I397Oc@2b7r`(iQtUF+FMz&r8l&D&>`-pYBXybPrMBQjy*2dUJT(O^>jApEVAtC)c=2|IZEVxPWyBPz3i1GxMBbuS#ai zTAiqKQh3R$rIin>mZDK#7vi)XP_x|+cYS(cjgv-{M+vngB-hxH* z!F1jPIW{0@%SU$ciKoqONB>Ekmn>%M=ic3(J!)BQzHy#CDlQ=mW$ zK6d3HJK#v>de+UkZKUK}2k2HOc-(b1zgw%GnZ2SLJ9k41SuV?KD_%})F~#J zCl3}(IgRb3qVTI{)(u;aZX}F*e->!n`8Q z4t%5pSb`sDt0wA1SC20x7CfGPX4D6lT3d0IrZLj0XD88C?g?qhTfGjw#O_)k1IOy3{nxfryD`2*HJ3oHC^NMhP~gJyTq z!<-{qXs656OG5nkrXL@Y@s7X#`l{_)(7C`f%0NBSwR3|)`&pA~>V~Pdo=+d$2$`sk zqj~)~^=p8GIhO+MmvV;)sM+rY^fVqRu_|ew@V?_JE;Fh3Ov)t^n-X66aqE56xfq`1 zDr-7-(dM&1A~EA%bU?Fc2?uQT52t{a(XwLW+Dr98)854)o!`r-(UH3^DH!(=3;zN* z*n6Oa>;5+KeZ0f{Ef=19C!Wwr>VJ%E;PnH#d}TJx66kXtj@&DpQYR^gz@0It+&HT4dB!J0?3u4j*VA5WN?7<9>?=L&dIXEl@Ox4ri_zW`l^lPKbTZZxkzn&0=4-o0;QLxrWoj+HIG zi)2M&x0NY76zD|2W$Flf(!>B9su*jug<>1E3Dg%W#sRl>?MsWU{fxRsP+iktG*@iE zK27k>;yOy#BvUlNf2EJux{MjajQ*J);FDl{IJ!7G9$xg@iP>>!3fee0(A2B$cL~r$ z-rUWW`pkK}Q?&Oh4afhJ83zNr={}tQc)Gb_)ibyd7J`^lBVRzbKznMuslsvYY<^}8 z2j_grkrehTAZJJpbMSt1e6vR-wd6<|27qOwqQ}}+$1<{(O<4*tN{Ck;1y(tgD3fKuMGDy!ETyj z+4vBlSdT-l;XWIl&^@k(Ro&1<6`<&V9;Ye{)ZHPJENQ` zbcuKV=A_3qNBn%a{q{O31U#F11Ii6=1TyPtX?}_GY|J%}5ha2*{OiT6ODjXI$=(?tU-{Lye&ZXH` zH%S6hw}wvSMsF9vMr_e#!az^juKsPqf7Q=`nNP4*B3S~aJe1uX;OYBbQ?+~SMl!cw zpJg;U)om(~yl?%z^W=EAef4PJ{Azh%&?b&du;SiWTL(%H6A=}8YOF262v>Y#`4qS% z%YWY!4?1?|dH;s%W{2htDMHH_pa;7xvoaFVG8A%%4X#yV(n2S&)lxY1=kGbuYMDgX z;ii6Dq5F&rDR+-JkU0dhrI7#hdch3$UpPgb+UxXrIF;W?d3$?>+gg-!{gTE+t(y4N zfzt$Re&cA(kH8t%IpW;#dsD#Z>-8oEOY=Gcp|3T?llrWT z`F~#|jpxn3QD1W#Tq6Z=F=nlq!Iu+j6%R4G0Wllk54mE$-3c+iNq*o|pg>Bzaa{tp zA8N_kUMDF z*$-U{zujfVleqFd?{ChzyZI}nCc0AgY`+~~m}Aib#20<>oc86}rE-Ou6o zlg3^CSC`A-*{-8Mv%b!Au=Ie=G)tay6}1UxkHb6FV9;9-((C z`)zn35q-$MnxR{3nf&~mwOh)O>8k&Wy7!D~f@>EAQICRvU;*hMiqfPgy(=h1dKVB7 zQIW1d=n#~qBGLq;1VnmAs?-Qbm)=VVy+eQiAqh!lN56CD&YJsY?#!QIt#`d9*?m9z z>1FYvvOzbfv4!wMHCUP%9}+A*3DXt*Gnij0DsHcC`U+lD?w2Qw4C^XnIO4wW8|uhy zvTGaH$687eh;<1oSGga^$4IX>SeqCb9l4L(EMLhTm>nD2bi9$>u)KnV6WXBy0)o+EF^DL=dn&A zAm>S5Oa0?TWjO`mT)w+eN0X@KM5`G`6c^|2Vz3@^^`*7-dW^lL?vf4uWl6vN=(y<* zJfCW(j}+Aze~XWtb5MXBb?IlJi`q5UL)bfHj#k5w9-AET#>UHvw*XskH&gzpH=)Wj z@wWNFz>O%IPb6#<9O;SHUbTG9t-RJF(&W7a`nyfT=GA%PbkZTX{3?m63dt0Nu71AJ zM!cz26R|yo;h_(y(aW_So8ws0NTkQi_)YW&?uYyBaJ0yJK_CQ7E4^#O`qfW+Aa>Qg zFUigylfiJz44xwUAcK<2dn+cK zPH8_6d4e+yXZM^)uyD4Vn5%Ovsl`og;$${1^W*pra8nJ(C+Pg=rt=q|2QlGwb#=as zL*wThIgfYzwy=K;bv-F*xf)2b8ocS|b(xzTrKNQ_yWfhd4vWNe6z0CAUoGC9SK}Ow z-+J~5aoAxDo5KUAL0mki{@CFI&T7{cT&js9q2zNk+U4T8<7x(lUqUrgIBU{LM~?{n zXOD0_@N$Gv5)SMx3IR25lK$+?^)B9Y>G`av>wS^Ejl#3M@85Mq5*O@odXA* z*YgfyEAp93=}jXXj-y#@sWL?1$Ioz|K8{io7AIkzbY~R3 z>gPxbE7?To3Kj@d1(ss_WbqA}!qU*8CH`qtS*wYKYsI0YJKmZI=Qw=x0M^w9DqFPJ zg5*|fzFfoCF6;vA7*Q=|K&8dk%cK{=}HtFx8Ku*x)D+rYgyRMUef>LX>s~# zbHW8Rs$y&yrt1FSnRGd?dJNLf3(w){k8NYZOt_bF%O~9T#&z+wN3y}&5-fnlu(+&b zGjy}_U0`4D+lHhY9d&7)H?~Ez2I(D(k)xN~W-qi*AQPJCL>cI~0+;l2)7^#w=oO8q z!~nxC&a!+e_(1^i_8~4sr!#D7qI_~|vSxNKe7PsF5#Kir;0GC_#$mY@Gc>+m#pVNF zCbTjaPYQ0)d(8+-leQsJj}t<>r344>DW3f_=H@dve(oW?KSlxLyVW1%1aJME9=Jc) zl?#(xOD=`C$AcQr8z1CQMC%I?Cf?csu#{lZqhN^XL96+u7bGOrD|@*ARCP@aTh8QY zynyvKpf5&}BZW(+3e=6nrKBKA0bBj_Oq`!s#9k6ld=NcMf6|q5Y_yfjp=Q^6eJDG_)sGQVpN?%&Myi(?jdU&wxJQVdPAP6;?dl;STy z7fPXK=nE*v;O~^+eY@bdO9|V?^{5|q`JWvqi2G|dK<1l@@du{^jDO|=0KE^*Zoc+E z-l*mJ0VD`szfxeRt5jIh6xgqTX~Lw{Q(Da-YLMkl>kCa|!WMPMJJYUa{jyQfu+tQh zuQD4nhcM8x9>DL?DIGlRDc{`K?2|#gYuQgqaXuPWp_9R8E%F;n+o`cb4sNBmm_LSe zuiHI2+!Q1p{30+fqDXU;qobn))Ity!>G-`0^r%9GUU2+g4O+NvEeuIGqOU=QT`YIn zd>0|S4)EM|6^4GbAHNZu&i<7wcl611T={T$)IZM&U`_ziQ3@OH_4wE$+CXTQTgZUS zDvUH;2yJClJz@H9f{TfK5M$rC*G|fRN9YBG#=8k zb5~At`YGc745#$&xQ>+?)Nd!G%;D1@=xSv#Ul5#Mp*JFG@?D97&OUwi$tqN^bXCW)$fx_w&i2n@ptrqLub` zt{tymV!DdN?i8*AqbLKI#4^$ljV6+_1a2YUV#Wo5m9be zbR?}>irU5{sgaP))2N@raBy%dMU!wWA~NRH!ElsP-O+|%f{>Yf;6he}aJ?JuR*GcF zlQ$z7AdbZ@ToIe7CSeI*+ZSAQsW}58i}qiT7Zcx;h&mxyS_NQ30}C$PvrbM)eyPo2 zmT#$=4v6S^+7`fPpob#*nK+dWegz;2L`;tbNHRj9fH{s-{oQZ0{?MFZZ zwl0MXNWhz%QfjA3gzWLw`HZb%OuZmf!lmW+s)<-TOu1JN>M%SP^?1cjdDJ?q?=6 zO(&OLVQbfI9QZo?5=ccuiQDU9pq4B9hmo<-bdPQps2}b;5~Jjeg0)7;nhaMeK!2k2 zwtLVGP&jT#zXO)HiBrUrj<=<(I+)Y_nzKQ9p`J{r?_4TcMA!)w$0?~ty_i1AP&+w+ zNs)pt;#vvx9IpqOKYl-XIuHM1$@R7#)H<%z|EkouOce+1iRW7#l5pbVuCfB&0ea!N}BLxbm^QVkLg# zk>}m^BLu-OG&eNqoFuHu45F=lJB?Y&=2_`>VL?HCtj0rqa|`p5pG)uBBCg`naxq^8 zih~Tg&x(90wKtbn98&8(MRDyZn1(`gI^wtFEcyo!}G#jogKVGS_NuXDi$br?lF(gWctw(g+`lJL|Z8&Ibx?USn!baDzn z2l07P9zSwLrAkD5*mGjN-U|^}vFgJWa7=9ULi91$9~B7d$SkJXf~*E!@Y)7Z*N=~}ohsGdi4qA|Yu7)CYfpRPIl>#rM@;$GJEM$S2HilT zGQWEtzx8G&COi;>uV)7rMwJ0@-xff*6LuI>byYR(e3Nj)$++fKktss`G~uYbrv~|E zRgKH{7hR|)!nTw^ccLMjLYOwxy%yqO3Ehkw$4nCHOE%d9W;8tcJa$48(|k|#DYqI6 zSFthb;&D4-nS&xCO(*0TyfCO`bS-z0B*oKL&6Nc=e+=qelBWLkA4!&NdyoV0%miMA zuhRDWwz?$uYkrrM8Di^!;cAsB)g3oX@~tfu3V)sZ>JV#nijU^234|GBgawDJ$Js$x z0C5%VAkFE_@dxsUQ&%C|#-C{u-gzk_+oYM>36pX3L+x>LbvfCUWNpMYQ*JrMz= z-25AAcn*SovLKrZ_ES2;2<*E{2Ml+NkraE!4)ik`#WI3>Dq zKauR<4*u??KayLkeQ=Y!5U>=GYFD#3wc zQozi@7yFdH-f{1Ht0c1&r>n@Zzfac10P#v&)#aBSbrh(&ZajYC()|T+)&2&C?!%YV zxW)%3&2}D<+hxaU-AeI&ZCNuTahk%(Te2i*UL;BhRQ>fqA-vM|Bd$RGM&m)zFHkq_ zZ)gui(B)Vp^}D&vQn;`eh*D!qv$?=b1Zk__H9eeUeb6dBh8*N7czbK|aP3b;e~P&C z@XofJtlVe0d8P_UBo1G{gWWGIg@}-*k)&EscTGE8#;&F@@L;H?l;LWLlao9#%W%a> z#%t5rOMWxe+7lF^FnQWMj&zcAyp_$sEpB)8k}9JVPR!v62Z1&epD6ac8;BNwBr`9P zQB~PIOHgtvrt#DG#OU%Fm~A4~t|+#iOC45Putm>L=ekho9=49h_T48kv+fb>Qe*G@Yj z`ZYa?eL7M|Zw2L7JwlR73wgHbK1ZkycE}P)SrzlX1M1#Z%3wU{lM6eV#RTPoyC#87mk>F_C5+!wxD3X<5ruB!iRP=p<178Pz2p) zZVIv?zB?8M2e7g z(`jyxZVld(qdK-DAZe2GYBgeJY~!?FG6+@RU0=+f%3vGkqXQGN#Xk{F?0QZe6?zel z$1pEi+P!`VtEy?zP&j#Yy6g^XQ_v}H@wK!#(*v zEiFSSxu7TKm0z`l>``T`59dfoN%enKbWTJ;UBQ2))j2PRwI%OU!JzXu`xQtFZ#p;$ zAozTza*+TK>C@-nrQE-E@KIzv&UTUS76r3Tev4AAhD&;Rk+rdsi104-| zZBb|9O*luIA2C!{k+8Lv&h_D9a_#ypP&;4+B6ZS~UyEHfwc5tUj+4;s_I7CsAiP>h z3CCnPj+w@&QDqngzF7d0vB%oH5~(YS-pSDskG73Mf`WYYsOrMax^cuVzOPTGr<{l% z>@dvL+7W)f5OUtOFZ})cpIr=FWs8A76{z?M@pCyBZ%`0zVmllKn)b{ z|0N{A3r|uqXL!BEmixby7s<@uuP_jXxJ%kXczG~DTHE&O?f=VQtr2s*QA1=XW)^ST zcKRKaK^D#RN0v|#P8PTS69o!8X8%V1 zyl}6eO@?-F7mc>qMYo)bOi>7A-+lfaP6~P(teIH)B-4|?kJO^4p%D|Ek0@s#CT8~t zm=s&p2eu;oj#yJfZz(Ckb;AQqs#IYgjSHc7U%iitavkxYv9PfxzJd!|o8>*uUn>W@vA=T*`%_ya-X>StBvkX4~$nK7SdgyxET zWW9IWCkLXylM_ZWVyJef%PETb_Df+uPefNcSSp%y6D7$@_&Tf>#o|S5X}cK_xWBi@ za=bnuZa;KD*bT^3ZpwSy>Zjx{?Ku;&mmbyRy$9+GWqO`8nhoXjS`;*zj}8wH|EfGg zN$weCPn|yyMv!gFnadgP!_Z?`Z9|kpEZ0Of={9`3oSXg)rnfEGT=m@)J z8`o9&y18MutSrV>kxPo$Q_357iG>AA4CnIk^}TI*FnmJ^o{=6&OLrdYcCz|I*k^o4 zp>VvUVrPtJgoa=~xuyFuwju&*tQyJTwb&l{0)G2of1g#}>A}vE+GI(42KRHCO5T!( zeoUnZIpSxm$Nq?YU?8#lgoX{}a%YTy$7GUzUgYEyejz4FPf^cszGsDWfZkE?qvw(_ zFMFi{C3UQPXl`z9(~L(n+LdD0Q@K{9T*iieeiCUnIL=gyV!Um>KbWaTNyjPhxbpdH z_0H@b)8wjG91IN4pHAWFIT^t3zoERnx}+0F_xy6k8`rPjb`H(P+6ucLtS&0C$xJz< zT%^>rFGc>;FMHJmkU>hW>r_KFxcHcu7%QuBTODh-jcx6t()PNgpr9bxQVCW?H?8LH zHzNsj?Hi$i#ZG8mv98HBxIh%Af`C!c2=NSMuP(1HY%JTa^sCtKyIP3{PpnH>M++6i275Mr z)nC2(@$6x*u4V$?8*DeDBd_Ld&H0MePT}=~q>5Y}SYIlCTto$W?MHePi^O%`r9$ph zSnA6_y!IYRcXMH3VQuc|-K78Q8BHWmn%%Xlo|Sz3lHa+HT}Fpp#;EM+DYOSK@9M8# zzdAcRzs+6K5VrqL6i<;tnXq6ecJZPY-3(LoRB7AF_tV*V1Wyb^r~U*8y*(hB=vmZ zj~jeLF>y1Dr%{yD5p^>gqU{)%ENnSdR#p~0pvPoqr{aF|efF+Qxg8Xak+bwW>F?^Y zUtn4U&pbuRnpj{gz{YBlmX_9=BKaW@xA`$1AduMQ;xcil76yfcyDe{^3dg>(YxO2e zvPge}>m#K#?K#-lXJ&?ycxM~^Q1x6dYaFMip5J8@Hp+Y6bs4FZzK30lIra7}k|0_S z>ABjE3|n8yAC5@8jkoPZDJm+CO?}~cYOR!{(I-qa-qS+yuj3-wfPOA|X={s}nVFg6 z?hcpyG`7)?8$gYc2rk>S-vwl*9TIVxsXt3$aE`i_CgT13_t=q>4?_FvA0!1o?*>x_ z><6N}Qem**U=`n{y|LJ~YRJZH!KfpKOZs?~%kfK@>-|4iTLFt4kH?R_b3bp3Sl8Zt zHp0wgxBB)Bj@SL@SRwuBSF?(11453EFYi^%_118@nCEwsnqacE4^xf`O-ds_efpCk z)&I`NJ$;>=uSsOk)B-FSua=_gOVPge!J^!;N!A?+Cv>3y)Ke;~7#k&$8GYm25R zQ=`@JwV}Hem-W6gkCd3w5h5);(UJB^t|2$qZmQCrH&DUn9w`v&JoN-JuAPHFGX zyvo7P>f}C zLCDaTkc?dI#d6=4cWN=UPT~E`f_aE_djYj*T3W@rYv1ktibaS;B*W)c=jNhkQBBT` z2O7V10Nj3TJSn`3-4qeAsxvBiHk#l(J)WnJIFW+foTKbN2Sr_8$PJB{s&-kP-{Ev$ zK*P%tz&@!iT)1!ot@fD|r>Ks)K`U|_W+AK5q}a4Via0(#&e4c#pG+X8F@$@!a>*aK zzJC20yZ@qav`Eh7Pac{{n3nbKAAft+Om81c6d^FZ26=Fe%o1Zf?(3&Ce3vtOZ^4u1_xNzPwzX2sH*LSRK$~?S(>sr#HOM zC>X&Y?36}SR`Rai^-H|SBy2TP<=5Yy8)eXrqj9vanRx#vNz9H&x}yH>-MgRkXM!G- z)xw`n0-gpfD<_vO?K5<8FtR@UkecbPq(Ms?hoZQ+N4M0l8ZFlbq*w6v?V`5L-V|{c z4}cuNa}M6$00`RkFcAJI{?Q#u(ceE9fAn!}&o<(h1sZ)<`z+DJf~F-PT(|9)uF#L1 z#y5*E52p@YX6s%AZmg}XZ7l(a{*&JNy;Lo!eRR|xoBNe!rp#qDCvtPFln_9eLgs2E z^VuJ|!p-e%tv(|N&u5I8X!ietevJMyWPn^|R0ShzQLUt)yp&kJ2mj zbaXmk@UbE@a21mNBWw55X?Kr~(y$2n?Ws!1=fex?DT1bb2We@U9iP~wzd`k04^+C^ zNji+&GiYk?iMYyt*K>yHj1pE_RCG7RUW~QDq+fml(Ey31Uh3y+JPD@{_=74Z=G=T2 z0Jbj!w!hroo5Cbwm8{^E795;!*g?1QAt7+rX%{2>);}P?wAf5;z3}{?;llCi!>eTQ zrlX@HkQ@Mxyj{zhLK&oRx6Ri~^|!~;+K(exU7EBTA8tw_kVw3t7Kvy$bY)^n6o$$A z%f1e}@L95&=j}Yy>ZdpEkD0yYaNr#mt;RY?QM%#Kpl!z#WDED_x7%`eiTRJ)V*o*u zyu@-K>$8sd1i03%VMIMfk+jeQtM|;+-kd73kFyD;WP#A-XsCqbzD%)m$zAJWPVR=%!0QOEJBs zeNY-Nbn^Tbr61Nb+ct5>YiHJZrmVSxpmdk?Sn&pqvL8RM!gkWA`AtTtFNHMRVmV!- z%60z3B9qDa388a!KIj#|Ui^sy41=OMOlb$ZxC=2Ie{_~3j*p4LPnpcR8dFhI`vy2H z^xJ$BrN~bZBAx}25)?s^;1k7Ds7cXdcf_V@L&ZZ?Rn^~5L!VL@aO(J;qxhBfKvh-P z5;y&&OALxzub206?@&OG7MgM?`h$#nKc4rVyY}!qc?ey_4LNB(X=#hi$LN->Ggyx!am4Y|4Bm}akCdR60O`AH3onnrU= zx~Lq>QQ?=okS>ycHslp%vg`r8(epg-`0v@V{gKG`Y_lKaWfc9+g_rxV)WzP_2jYhM z54V*+NeX_R=*79YxkR%OWNwS_tj3aBSGf;Il;6#LTfaeeIxtDfPs}6SOWWPW-!7Ou z&yq=n3qKEkw|2x_8NW$PF}{^<`}yF*mJHkHFcge*?%i6X}eC&_|t4UOO{;7z~gy=;;#|YF*Dz$VUO^s(T-o z=jUT?X^)ic4-Dwpwo;TCG60d-pvVK*iZm1ZPNezx-n%nX@SI-#8|(FN%&f=sT#B^R z)OVHWXlNXbcw(efqQ4Akhss~Z&TggUY}sF~-OzHWg9!epVT|6mN_GB%?vjM+aEL6T z?!~VFW=_g{{g~fTTxh>zzPs0AP(pKQ$6T>m_LkXBAD-NCO5wkIcaP)=c3^R|JKy^2 z*FCf?6iV9N-91~2*7&+{K3Hyw{?!{(xpq79YjPE*TYx< zg4DFMf4<(ay(~Q3->(}RuJ+8)%!yOpdv0fkGk{Yww(Q`-Bf!fI?OdFo`waFL{pJrz}Fs*L+kLbM-dXSp+`by8+ejWO*KE9=JgHms6uXK@id z*BqQ>xzd{=rG3k=FI6V)by5_U68J|2pzW81#k%14%K}nc!EPviT_;}^1w{nkUzzjv zLY5u*ZrIP)12x#y+G=U=;K6UMO(G7pw0`{|HDqf7@O@@7^VM)KPU&4&aHg^zPTTA2 z++9c{wC$rrjSC{*wB}`Ga%r+1dcSTDkJ@ZeZk?)fH7k1>C>VZ%L1uPDrHd=|#ol!9 zNzk^9e16>bU*G@k?Dvhkcip;TCZ|&{G~3c91p%cj7We^tin_yH^B5Lm=|t)wyM-O(lqg;e4z%(^s*=6aMV;# z!g**9=H>UIGkVs6+Z9nK<}nd;_r;^p0%Pap?kdw#U$R6#*WB71jMvydI7n68#*ed( znAIHhn^zS;c7}9@0Af!d)j9m)ERHyRZ1adbPGby!wnsKfZGAl+k5%$rS|ySU|6C!Q zoD?S;#E#VTe>-ojWk2&XFj-tUFzV#HN?mPs#HUaGC(_qm>Uhyh35GX#@0AA!2X{4( zoE*0MVK`5mCZsj`)yiXF=fD0%)(xU+YHDbW)3Gf8FrPeO=iq2R~?e2)iK zvFeiLVyDE&Z##%7LCo$*&TA)$3t1Z=rPjA4_W&&chz4Y63B&IAoLicsIA}Yy*eF&(79X z#N?z5AKw~QJq)FF?H2isd)r7j8e7zRX7aeA!oiO7%+RB^&F(Jur`z?cY{wD_m{(cX zX&8l{_NqrPiAbpdY&BJ}dT-J#yU+-f-T7d+5p zF}&7f+hL5>=dio#{uwnKUMZpSO>2U@j~hTwg#RqT@cHwsK~SnBh+}PHK2ksPTq3{4 zqoGW-z+fegb#bgmw%209LH|Q-qKC}VcWr}rq*%!W(43nX5{a6 zIy4bHkJ>L-l-u=hEH29J72dh|h#^uU_J`b+_aDd(y_dT>I;6B&69M?Pr+Sl`yuxVH zbac+8^0Kh7_@HJ3bai!2$HfP)fO!J^{l)D1n9;L#Z{NNJysN5Zp18F1Z_wYO`u3jc z-Pd-f2mc$Jvj2aCKL7U-@Be;JfU<+K-t45h%!L`o+M)V($^g zGf-c%(kB6aJf1rffPoEilTkdqLFc7&8W-3c&@VGpCW3K?kviaO3VgH7KMF&B$z(== zeCq!<^7H@tM?OZzun$+_Z$)m58aDbSi}5-vV337++}XONx3oQ$yW^6OK5B;>`EDw3wnE9&f8qcDk=D6Dpc?t9G2b0`*l zxB{6AYQm~&r)-Sy%SfvaMJYWR8q6%vn~Xc~tktO10>fT20lEI=TStxES05SV>QGY+V33aoS~fRp)ZPQ#f5jS!^Q`+I>Xm$+WV~}*()-N*;mJs`1#U~s zxRhvQA=(qC#vymh?MMG~opGk>&B-CqDd=bRM$5(F3>DlwrA(X8?-zaSFS@gCs6F$s zpI1Ihsl@W(;8P%hXMF|aE6LuPgsgtv7|}0zQs=?H(P&whr+Oh?xL7;o`|M@rygv$khY+Z?S}kEQpLu49P$WJSx~6hTYb z(9m@}>JWo8uQT#H%6o9W%B8r_^rrps>+PBRTKB4Fp&yyX*<95P-kvu9sTV8jF;M=( zVNf~N_dqY^I&PVXLq1P4aip%QrS-}WHu51n&N^zDTg`p5C=My0W7iuksPhpkNus1r zsHp)}<(9MAqg2Q zsjikKsXMcggoaR+@JhR=S%1S=d4rMn2y2U#4h+lQJ^_>^P3C9q&C5 z&Tul@p3J0Svf8r5>}U+}rt?`;+gX2nKaizS_xgS?B@q4{It`07Uwe)J{veV@=I_*l zMihDYlY@f`OrvBcuU(S$O2`^5w1tj#sLjs)(9%t}imi6`bNG%WzCkr)67YwMQCIsT z-dnYV($obK@;|IpTmz#7C92tYFDyK)a^>omr{`){Y_w7U_om=i`Mj$_1l(vn>I=8Z zZB&8Y{hb|FdFc%|jo7~QaTL6#Ul%~qN5=GInd=K(i)+Kln z{mLJG?7pm@Nl${t(Ef6FqHZLcpy93Usce;S4_u}1kx|9Vsh|3#4d2gQSlTWg7|81> zCgVmCkwZ?K_wNVt4SmttjJNYlikIdtLfVF;pLbo?)vc}XAQsfu6R~Q@z`_e`MwNMw zz6>~)+V`>s5Nq956S5>6D__O%G-A?p(mwf)M|0zjEWk58+=LM7#!8IAn$F(hQ$Ir~ z;OE(!%CD;n5K`EMo9pO)$U$CvRFo`dxmz~wASOM1tk^K&wt1a6{}I`!dQIalkZf6D z+~*H=ihB~PeB7U)wNmqv#L_u-xN0)hj5FSSW(V69N$gJ;DJ_k?nJGL=KIz{=U^C$V z6cI8Q(QMru)K+!#~gB=9&T>tbsqVH_sPv4Ed2kOSpToj=Km_T#l#JsGWY_R zJ6|84hf0lJsI<(?#iQ-%%-p5kl$zJ`E$4X1vf$;=mgp~l9YprQydH9Q#CE1HdnP%mjMdLyD$GRGcYi07WVb^&Cbs5?ChB6fVc_@9@bS+o|~qoEe!6srU! zI*ylUbSH`+fm$f?Gx_{@&Z#lSM)(-T#>EZ$p&IQx$U%kIVx(K5p9otuJkC5aQ#orjxy zTuMs$bN%Lb0+01Vsk?@U*ABn{_fq+2MgB}%SkyRdns7*e(@b326x48W0gWar)T&NP zNxkR=G}kv-OZY7H1LOC87%({S7hCS3ri#v@WiiT&whA!5z_w8}{)Ia@gN^)5sK31L!j1XGZypOvQom{7iBCST~{j z_W{OEk+dJa1(VM3$Btx%&-$pHrMb0i4tfSLJ+yg`$W$}0)qX zYIT7<4@v@JkF2lJS?WABdYH7$V^E2jB1@U!oJO6GA%|?9!OgR0?U9PkvqgaC?PE|0 zJ~vgZE$7ueZu+99z|j=Za+=DK^Q8XlUeY4cK=qosprU3Is^LRlvSl2s;(MlJX zCu;16t$>Nva1Oux%o>?rR9`nZl%bXL1g<9)qzFF!xa4n4Y6nGN+*$LblQCx*@Y+(w8jDiL&Z$lqD&*beO06iietJn|!XGrA& zK3YY^0&DBpSpX+kG#l)e41lz$s(7hV;)x;4UFN@2ABik8g;~@X!JCwo0Tp*!m6X-q zulF^EY;^W&PuB=qNM;U$75?hW-gRd#u`-TirKTU>LF9iIQtwW;uJ;lY)st}Cz(M}I z)zS@GFVx1U>GDPal&ArLI>I5EkPY>r*~aeJwl#4uRG9^TwdgT6-&{h$Hw#2q3MLNP ze|nA15im75+M24;ba7G51se?tc$B0PtE@bd^#MFPB~sQ904Y;pP`$1vakM;JS^1L= z8E@dfoTgFn{y7D5!)D^}u8acDJy=y3MM|_&Dxq*$54{H=gZ7}=({O{QHq5d{;Jvh- zix;%>U|`d$Dge+Lc7au4YKfrRExq>Pad zKPfA(<@I4fcf4V1n0f8c8eerT#1XJJiyekN%OH(CMKq6rjt$iX zUft2J`=XoA(PSGT$jkfhpmefQ-I*1MkC}vDA!MhNx8&t3Uq} z7MqzlNaiA*q77Q2vqaz{y(!6eTG!Y;pY1#WAdqhu*$3vJ@Oz30oyS2Uxk_Vh0H7Gu z-+GXt3_v_nwX3J+-;wxuD)HoNT!AT-R@g0XIe9SPFR>S0e`u8bF_47#*TnfKAjAgm z={%L?gMLLLa!Z@F_iKi6p~qVEzcoy6;i**OZc)>X9c`J?iPevau7nmDI zTxH=PqsZ!Rt9~14s1=-$)2|2`D2OGAKYyPfJQ^?f?<9-M$6Iy@7{H5K)`1!N4mW_L zCSmt^P=odD&klElDCx!n(yIVS+$P? zg^4BB<0Y1=tDt9)c6m=rnfH$Z6XP^_)m8ON9~tx~%K@6sbO*owx;1HSLq$F03eR^m z&2-PYW?Anrc{eni4Mv99>JB1x>sYbx$zwzv3+%?(^;t_r#ZQmOXyx`+sptth3SgUR zCot;${gqL{(pS@x|3*EUu(p|KvV?3fb6juBH&mOhVf(9FsP5>a4u-?&I1IL@K(99r z<6|Y3K&vlH31)Np1Y->2kYixUuY*MP#Vm{z7w*2^MmYYv0A@Ks+Uc6Y=Nd1%pL={} z;58LyyxnO}RyuemfPSsv0Idvi8{|+jy6n!85py3b?#uJg3||Oj~urt_ZE5N8lqou zb8pOA*0`{+-$?pd--AT{JBFL!7)}UrMWS1?#ln^&B`}`QaK}Rj38%c68$dl$U<8Qp zIr`7Z!yRgdxdrMmd2wmbFu%lnX(kvWv zXL&|1g{egWj4@WEAABy>V=?UAyGlr5DvXh{ghfo;HSLx*P_z_0F|+Gi+qJQ5U_J!7 z&cFf0@m4vR0r9}!EK%6LpL4mt)E_GzPzUrowjBaM?IF28X3-t5shRi)kW7aWbtxA> zN}wkh87fs!_{@w4wp_quq%C5vR9NFLcf3pMC)*Nj6xuv&2zX~1KWXcxyv@Ru{?hYT zjEa-P=;9e5>KWz|RrYaTs~AK|b`Y6Y<=%Xssvg}Kb@Lkk!vFEp2w(>))my_J_MXi8 z6&PvUio1S&Bsa?UFcw%;fjmjCG*SxRNQYWI3%>G+TLWB|cbEcW7!P|aj{+pgh>}u9 znolz!FW0Q%k?(%Z(F1F}(dPR@aW7-}Q5VWEaeAR!d4Q=NmvrWUAYBzVO&y_DpTdQ99Q$C_IjKK(0bk9_x>yNfSVAY)$j}9 zmOk!M!$w~W4Hpy_-^j0}ePI1(g{^$Kq+i9J@rNqW&Hzde(=hD%9Lp)^;o}uw2&wcpVuCtc3ce8s_B>QY6>s?)QG> z1{F-!NBGI&biCkZE4^`+`bgEMg_XH)Qb;4z?8dO#=2*V+S&h3%fU3@VtMKYN%qx$A zG-toj$CZ-l!b zC1oFc^aQwLNmQ21E<1eyVAm1~S!_4FaqOImtqfl?yy27zhXZ@&-}oMid$B;?w7M5_ zJ=TNR@F-DI_9M^whH;gv>)z;9A&;XX9>OtT;sN4{gj>r*GeRo;kdN7##g}jLrz(k1QZDzq{=@c{{a9naNEQRC5=sLIMTYPbz4;b55AlD5 z0xa&F#>0?rdXhG(%+{ zEe43=;&$;BUZa|le`J+RB7jb$Q+<2J9#9oKyl0YDfnmoQr~?b=0q7|&Z&Y1G#DZ`5 z`;vg#mJ0ri0Fq;5~MM3EWv#}hAXq+j3j+F=ItvNq7NBg&90 zSYM>U3({&Q^J%jRnLPs&Js-Rd^ik(7eEQ4aOxr?3n-P}IQ+mv?L*E~6OvGIIADv=- zk47?7X7*+5*_BF$b4M$3|E)X@Fa+Yx1x036uVMye3_`2}_4`tNwkAEeMf5-1;*ab4$FFF3nlbYnu4kr5$#m|P15_PMiQ85A*V`ls*rOx+ zqSZO%HjdxGWLm8|uJB)12GXIYvo90a&h7H?8?>>iQDe34<~4nQVRLMwwrniW&-}L^ zwwE1h%sdi~Id}|Y_uHV%mf`vkVE+nDSpFT|pU+XE>Oi5*A^ZD&*zdGLzC(`Ss!zHN*Q?l1pim7FV3VEUK+WL>iKAeep`-M_q9 z=n9wk2Ont&rmzMd35PGYw4xIe0l)Y=TQgbSY_UTS_y}c!_5WI#Bt-LQWMl|fJ$cdN z@Q*3Q9YWAK|1(p%MWhM_+5*7O0>zzE(xAT6Y!s-!>?!p49$6)E?_E~N4ceDn{@>D} zWTDP{1hCn_ZeU!LN3S>_g=GHA_jz7lW(XV51GubAltbn^6I15!L|PC*A7k4_9 z$}&+v%`?>ilWye$gdJ5mPpvA4d|!z3OlN!yXtvt|q>yM}*QkR)4hsP1t^#=%vNXWC=#*ya4 za?TkcpmDglIVO}j{1|b#2_QCGyHLpD6{z|J`W{-sb9!8o%cl3S0OLSfW!?k6o6j)hKmIE<;xyQ}}ltQDJ zsAaHX=6TVRqu^)sJ;Y&jsM=XISL3Bcb)Jb(yhy2nAK3-=uiQm42m~9MCr?elISf4* z&S8Pwngn{(9PQDp57kckbsm4ky?Y;{1;E1>x3Bap4!bbSo1SMZh{dvS@M&Oi1!iIw zxJvHWR{B<@!Xp0(?AP0@Oh#jcEJpIO~ z3}1WS2j5;^KJfvRLoopG6GvP6pV?I@$$a{wFRV1dJWNah_~L38fJ}Xt8G)_h+qF!w zn~(1Wh0zWQmaZK+6VOpPI`MIFgGpeDXyxoWcjmN9m%ry5O1Pw!KOdAr=2^;Ro@uk2 zynBHam@I%1>C~A#vTUbrm485`BF@Bgfulgo&L}^2s>XD*0JkkQ`7|6@Bg$U@CdabA zK=lI90q81alVdHfO=cgpJWY@)4J29mVSuh1vi4fWb=I=NPEdMc?Qv|@uY0}8*)z2@ zUfVdxY=f+~UQeQ+sc@z(ARpHt>f0)@Y2!_VX^R@pU5rwUVxz2=NvTzVN$*#ldQbbo z>q6GTP^fWflRs*92gpD@Nn*Ko$QK+f@FrxQU&?j%NxkQMk(o-V^>~@hRHMacVd3LD zkz67oB7!e|`WoA|=c!*BEh#=jDeji#GBa?r%|U(j_Hi4N^<#cAVZ8R?25mpH_^Scn ztkl@hz|Q`?VC2IUV4s4YdR$Wjgkj^ZOwF}u%DAq9T0!EsF2rC>aU@!~h&xpQJ_@K5P! zF%o>+jJ&7&=XLv;=#qffL{k9%Gp7QsZI(>ed8k9VyNk>!6bT2zY*H@I|9l->yZt6f z`ptr=bpQlJ9Q~vZ2F)lm-WBp_ke!T{y5f|8&6>@XCKh+J&35&wByb$aB)7R{{7qDS zsQg7wi8WmD_1wM8;NSfS1Yt2siMUZ%3PNW-^?^qd^d*X0Bi1NKD@nk(0Hl0Cu3bC7 zu(%ipykooKc%jGh;i`6jIaxIuV8LZGuYC0ec-%c}`^W;jnQ@Mmxy)g-fQCs}T1JKi zMi%|~Swin0Zj6$Beu28(#Uq;OHBgtoJ4@3CfYY?p>fdOO5~gTcGgQJ0jPl)pQ%^Tp zX}H|xzuQb{QRAZEcT~imza#xu^~qbs6gCE&HI6|(n0_TyN*^0Ah>%epcy2wio?EZ} zs-&|X=hum*%6XMM_;49|vZn^lDY7q?84M)vw$uqF5$rt3Lhs*Z4SVfuz}<{I$>9CV z>nB)rTLe=LvIU?eis?jOnpWUzhF&>r1cn)3y9QdRCmEn3 zr$PMC7a@ykVW7a^Sjwj5Qal0PbkjAiQnF-W!63)TH)MXkHN4h+l?bv+;4vsykAy^E z(OU$e3t$R14)P1**Aq}^5w&fn1r`PnBf!SNbKd`G($1pVNyZ;Y_V+=P=+oKlH)w%> z26?2l%qWn5!_gxlOTX3)xSs77Sv5}mTa>-Q;-xSwdX@6**?^P14pmBxyCCMxvf^bg zxW`DkzgK;0lml4w-t&Rv1rb+;UjF%N14bg>ee6w5;FL1+ZzO#K#HBFco@fOR445dh z0V6%Tq5A1Dm=FyAZzP7BU?e`n9Ia4l#;d|lUU@|GT)lKjKTj{C4zfuEH}&tDyJPA; zixyeKi2zE$5Tdf`L+-C%Gt0OwfeA~j8Xf-K9((>KmCE*1mDtNaKRvh2GlO{|Ku`v_ z8DPrZ0LmUISu(*icyHtpGCz=^Qt!FV3izGD`>_$gE&&__SNqeUWFlJuT3riOf2hW6 zUhRZk8_a}~nb~Vye4?VFpjxhL6Lw{UtY`as7S{j;k_Pw(ceoi&$Kl`+y%aC$Jas?# z9AUSW9t2D@-_jUF;=gZ#j@srnN z&R;xnKX;5V?u8m4)i>7HN_`kjH{YNo_YMp$P>Nvk(2F{Lt-Tcgy|Xjdv^%dpT5bWj z(9IVZQ%CmqAd76}YZf_BXkPBqVg@f$N>cJiMz^!^<_c}e^&`+Sh&nD}m+m{lUG)8U zziGTO;DY28YgiWvwC3hpe#?tmt@+vQNb&+~Ow}}3*K&Ux-TC`(^7A*S#8x%_Tq^_a z0FZt9Kuco#)4&@`xeEYNrv2>DIb6%vcmJPjc}Go=lX2bduC7kx{l4?uH~nxY>y`O& z&;e*S*QqtfHEK5?ldIpHI|*{dDlZ4e{c=B6r`*Q0?4ZnuX?)ufVR1wcA7E3febMoV#Oj17 zl0T5joAXZgzsKAVd=cvae*$X-=v0Z9pz`+p09(LjvP;CcDPDY#`vwqufSWEa4TkUh z`|lfX&XOksxX0!s0>9wqMZ{kS7Ca;k$h7fU@sTccOqe0n{MY`b%%UJ>W*c zPIYBkM7wOP{o$nJB&l$kAJqM zD7S_R*|_ba4~9v{K3Q2=aewv){$L>X~xYQxxAJ`KVei9BIFkw1NV`4eq2nYxup;jWV-*Z_CSCo9`cbisABxh>6l+lG$ z#ot7wP03-k1E4}O`i~4^kky1;6R(Uu^*RPa7n9EUrnv3M+qZ8o6yWcV{=P2YdVKlR z9SO`nKa~&6T0Yg@xa$ zp8FrSup{Xi738PSo-+@6&ixxlqBzA&&GP z#l^a)cENzxTMRnjYg+=|AmY;&ootl0UTz*NbdR&MS7evh2|jMzohP#UFt_7Z>q*%s z{YA%5oaoEnP!V%n)K5?WQM8_UvYc4(WINm4pPR|Um^bx2HB`rLCf-97?R%E1xHj2p zysa=kzw35%s@AnSj*+6>B95QZOpkgeDHrK!JMt%4YJ^-JO~Gvq9kQ;I}l zaJ;^p8~exYcQFE9SV=aYHRVisdLgzGmp5EIJ{B1_ULUSEuKf`CoH`ntmXR_1e(M+J zm0nyULEDbj!y+|$A!DBm^-gD0Zu|w~p;&g{xN$ zmdyDKKMYhgrYE2n(N~7LlFXa^HAj^a>gu9qNYX`m7{_no0?ZR8;m)x(T^_p2SysU! zs(eOQMG_RG+cTu?AE$qI%5F`X_qQfF3Kg*3b3C)?>y9!P#!kN@5fpB;~*7qVP@c)6K8)a`*k z3iijfXYd&Z7&XE%`%S}bNTlQUHlif9rTb@O$cD=jAF(#28X`8I8)f~wI4g{+XR!c=Hk36pUJY$e(7UR zp1^6hpMAd8*sNY(>=~=3$|5H&BlTK5R8S=7*SyKxL_)fWLVh0bUsfYSKaY22M|0)- zMQAs)u$W@B|J~`%Le4OU3dOm&V3twYn*PAF{cnT$`@qyyh0Lw5%*-}S=1bQlsE%~1 zh6t#!SbW zxz^Y2ThGPP@!o7p#q;Uchz30%2$69I*JBH&x?Kcq&(5dNh0!paF*WVXXIzOpDG_UL zMKADyg1R9IIDFqo)Z^82ZR8N+8!IKKM47brjm3uwF|!&g?mhbP!?gO}iEr;j;y z;T-8s4#?LJ*Qe=v%7!<6)|Ar<7misXDP*N0oA>bZ>MF{Ps0UUAq9nk1@6+RL2a~1a z^?NRMW=5nLH-1eGwor#zLRQp>U}{0Fo-n_Qf!JG^Za_{hc=u8`X-YuJ#F~l zy1pPb-j(oChWZ#s*}63cUdNBg_4tLmSKJIrUOuay9QYWgG*`-)oNpwmVAF6~;DSNP z9%|MPlld+)BR}8A-kmPfSkqrWJTW_ajic=QB~KdTZ(G);o{D>;Eg87kE_F#0yyM2-y>ACSYOLv5UKn5{+67GFp3NlUH;h)mz#Cp z{VGheOF93bA&{W*wzQ0Oer@3QiPx+n_VLr!ZWkh@qj?QKFf5El*N8Pw_Yy-V@raM^ zu%U@3r?FXL? z;;E?TF?#NoyLs0J*4=JQxNucSxTC_S{Z@2l1(Uh^ffSJwF=>@AbYta~FZC{~zb)Ue zRG!Qoq!rGv|D8(OY&km}k9ZJw@xgzc?8jTC8~in$oO%V=zkU3g6?-r9YF>tgzLZL? z;Jgx-+6#LPLoqQf;d=lZNd8l{ZiA(Cam4R$*Z1loeD@do0PF)k*ln*ri%)UJ;!3G= zT|#7iba>X||3Ox|C<9w_TdVK}V?bY|{sr3_#>;cunKCpUCB+tZP}SBPKKslG6A z=vK71Y?$MStbRDCZ5cEzpMRBo4!`T_rXp<7^IFc2Kk1`f^Af2h&yw`AW zA)VNIYrd<}r+2vyFrbGy>!QQU%cR`*0Zyn}aY`ye0+Kqj!_mu=Vruz`u9Q|wUkVDN zqCF+TMFAPhO1S59tiDOy-WAub@`jDo=@rfcFOHDY|)Ze8~5M(3qhl!;HDPINbW9K6-KgQ}^> z;}%VOS`s}u2;tdA@iZlcTi?6CNuzcXd4XInEZPG#oMOzP5H}RVM5Lw z)==RXo37f$>GA*>YW7$=3z(_h2gB~|Evs`5;MIKrt8UWv$=L~T7~m(~UAi^-WYhDbTNe zNuixbC&!j-XVFmYGGH!WGqr#d8gWyC4t_6bcEF-H{oT7~)ymoF*5lWU=niB}p^0&# zYkR!d#*Lx36BA`$NpKK+ueU7x0dbbSAUmi3{jmKXDE=NH*-ifzw5k4d)|%tQ5?M?UuCCAkBI3}-+N_RB@B#jPe8C-W{@=)eyQ)bZx&IyR-2YF%BvbM{_Z_?N8F{&(=)f~- z6uWlO^61@|^+R@w&!ny45qXhZjMdC`{a#q8J=T(3bE6}}lrYbnu9xJK)N{2s_sGi1B70UI`Pni&co5Uw!oqT*szZg$m!zE5>ToaZ>5dq%4OmkOCn)8CUC}syjOa%pso!iOOY~W*61jwxm??0mlju7|ROF?07X*pIK z;a;mO|B)YP2@a02rD2TTczXuKr_(k^ujQ|L9vaRx8HhO?;nvx;YZo~K)RklB<^7r) zq46-1YBh3rd?)1e62i&Qne>0>Z=DvOCe@d{hWk8NgW_3dF1wi6(n>{5jljR}QY`cC zx8~qXpYqMlM&N;RocvYTwyV2!EA%Ywg_`4(lJI)2?(U^;hkpI4jg&eVaRM9U+ru4< zn!ENOuZ%Q&w#{g_i`95A69A_Nm{Crd>3^NcU~3EFpp4DL+a%QtpYZTXLWb#R18b;2 z1et9ieK>bH)~Mm8KU=J#3UOwNvvM;Mv zQ!=wns;Ipw^Ph-9R=YY;En90{5-7bmy{h0H;+vO`SEGH<)$mqed6)&YO2q^KQPmw} zlG7d)kx2FP<6xl}H?ye;&JDd^=vGSBlFEGHXM|D50`h+T{H$2jtCOW*>b~^CQrV)d zJjo`tu@)UBg2ev(SnadjGo<=pzLA!59_>$ZQ zL|t-*APR>S*!jlTk=iT$<-LSUGr_q0(ArAUISFcxvT(v^Gb+Rhci-1VS@k`yeSE+0=}tkb>+R{l12#kA_qzGR zq4`oqK}$(>b#ZqX=tpM6bvCKd<~UV;e~uW1I6of!x-&woD6y&Xn{>vq-bR(H)-ay^ z--}o=AW-x@pXl(&dUb8j(MXTF+bkaXKi(Y^z#5|CZyHTH>XJjwKR3Pdb3M*`XWLGr zfg5{|e3nr--GyTo_?aawT8sj#y9{gMuBC`XDX|bX^dmnm+k-Y z&l~`tD2N1bXbM@!C(1_^j+3bO)>@aY<5?s^v8d0y)f+{!nH7=KpS5w%0B z=?{wWH`kV>zDTkQSl<|LldAhD!*;I)H9*c2mmTH}*6i1&-Pc#-h%cz?N0Z{NQ8p%W z>JQSWzY7cm>XXq0n7ZxT;UCKc;}&TK?~#}m7R4?01n|24s$=H6e_CH3BO+|4?rhgI zFc@9p1rTsklG5A=3Y~)BRp|pyhyq|^OvGW%hFE`!!#9FmckTd~Jb{ojH`?FNtgjFF zb8STbNHA{&jL9QT$tMM$UGjs^EBq>G%WCwNnEQKY+-(+khljZoOTERT?F`|te%r#S z!XzP)yOs_}S5ti1*DQid<7dq*M`gR2p;E@iNGa|_vUk`t`vr)Y6!FGbfCt9gbV~s% z4(7!NC|$~i&W>l`db?k}3g(fx=4n)W0&jRbn%BX4CVjh9sL<$`vP{s2M_b@XT8~?j z^kx$OAzyM@{y&&gz}Gx!STvcQqNX|Bo2K!&%eHXnG*@*u0iJcQI8bHEDFC9SZ4!NrBy_n#?Ta83M=$!5KVNH-HUxdMfb+Fw;Y&UJNrL4YOXWqSVU|l zI2ajC+WfR&gjiS-lp{El8$Nm!6ugwxR@RAh+jGl@fsXGwB~6>6s_&iLtbGiKVPo&! zqB--YNY6FlN?ABN-1hlW{+ixS7aJET1_o>e7Fh$7$|COWKNNYGb9Sy7;88DHO-njT z3Yw<(SIYgQ{Mn6x{3OG#d$#Ac_(lSZ8TA6i@x|#fnzD3`yYDi(>P8xkI)PuV8Lnd< z$ulG)LzzXlZXNk~3CQX30veSUrKe54Xrf?6CU zjUTgT?JX@@e%Z~~2Q|*xTL(^ExSM7)Ja0lT`gUx=6;E(c-tIBg`oUAEYI738vf7?R zGCCt~0;vDcv2&}_?&4l`eP!15hnb?JBK96VbQmZ_YC5AoH@|UNY=w{)&z1LcsJ1+W zUzybM!E%__;^SeibpPXvr9N@m638UTiJlWIrz{UvILbegzkw5f6O?kXfXPx)KG#0W zN_u0BMSrl*TPK2!_h)~FphHuS2w<7zflA+>v?Uj+=hGNBBcTRN=%uGGy~#(@BGb^W z!V<6LK}OEI8cE?dUl(;`W|>_<)j|h@!S{eeNy(%|qqN&oF=2LTkWTon%bM;coTG^4 zk_0Wz$$?6h{4NT+`)#F+qs;{DX%-gkzg`}TiMb8x*o(t_r!F18cmmr@x zy3>rBE$K2V3&tH6*q5ClC4$>mwoP_n^Dd9X`Tac1v^M*iN#>2lLo&{`h;#kq zp1pLZPZTItFYt9_Pa4&qQp+*;cFCu`iUgUAvLiw9@ zkBGg&!UMIu`)R_~J#)=-J&)(dQ%0_^?hJd`Md9$dMVZ?gs9tH+qUx$n(J_p0&P^Zb z?M9kopsSc$)PK0vq^jMW7pqhg_F!c7G9Z7;b8TrAo8YkLGWV_!k#z0Mb|iWhA2l6< zYxMgWUlRbbfWBq5>T)qy=|8kIp;iWz>%C0!)h}H5q$jgRlKYZ0)WnIiT(64gbgjhP;&!Ou zCT0siv6a2&`d*RK_*61anvrtE+12?FBmknWe@XiBuqt?Xp?ecA{b;aAofhyLi)f>! zRC8Jf`M7{H_ax`*X+cAbQ{)aJ+Hj`7Lu8#5rCqv-|9TW%tPUV`eQ#;(&F|%!V^T{q z;QmT-C!RQaQ#Czuby`vnBomtl32J{(h{BxnWZ*LsHK$SI>3L@9!*;eqzl(M56)(>L zLo%Ag(#xg=>};%p{hR`)`u!+A(cZac0W#$I+W;QzjJjV)2(DlRWri&yi#r zHzAw&wOfr0z%2>c5Y3jFxb~E6Rux*_wNPHsc-PTMu&$+I0KaE$x)iSvbfYbU%gEl~ zQ_0I)af--hfNh?T0nYM7XMG@{y!Mt!nQ4HP3*%lIwso8kr@VZiR~%8!iWM)MihPRPp!!djC;|V7;xg zV|H=6;tbP1E&?rlNP3}IX&QgdqgHH)A!3zppL*@h;18e)cIu6fpXxgzWWp22TKzdg z@)m!`3{%qJ+hAtlp2)&nmF^y*K{9PKaQ%EvHd%J>F5i&$iZk{<_QMB4YCkf7GvX8u zLNy_Um;+W@tC*z$NCQZ_DW2~94Y7@5e5xPOsg(zPS9A==+mR>&FTS7vz9r6#nSN|* zP`h8V@VVq6ebtpiu`eJzJearki!LfV8|!g&e1IBH=3`ib1r^-)N!9ISG#G0M?7Ya& z@A~XC#|_!!$>P8{qO0cjzhJ)7@4re(5r28JCXDUoMK8MH@7u;(5Aw+sy3Nek&!1e+ zu>-xS&~4bAkyq;(piT~xQg!Hnx8cmG#s(DtMmozirgS8kXcs0z2Q;0ezO4GkA|UOYR}v{P0Vz^nhH zrQQEUMAV%JHy>mRlnK~rtm>?zuE#1!TMU+W6If)2eqU3|JTO;cKc{ZJtt$KU*(u$? zgKVnM|VM(xx8v21s=;E zTKh^%e*Tjuvz<0XS5x$5{c%YXFYuW&dBa5b?AhHES30wT&&!m0BmLbNbCA!hmD`_{ z)W>67uyNjH`opnfmcwA!{ahlI6Z>PB{Wz9%s`EFRA|&Vpfwy*dk5zyLMCAyUo1l&( zB*F(NNv;gL4V3J~0^Cbcg|Xv*AbH(Uvn4~0iZK?&qG$jvn-ibq@dIkLGl&P^|Ejk! zJE>4M$;sujdC(xebt`amvG)sD9x9?bbqqWZ2nSY?xWalZ=rOo9ih9Ac(<57wPT+jlU&W$GuMm^d8IEt0>%(Tm`YmR@QHK0vW1e}K!ixNqmX?jBu)K|%J{jId z(8c7>8qr&t=H$>I?CdO;TimI(FexE2zYv(8Ur@jV-bkom6qxvxM@qZrMjL9&Ueob_ zUs59$=;;ZN;QK4)4}XT38ug01J{v0X2yFYN!@1|o8DJZxUEdoiWM&yHTZ-83=j%_( z|KKJNv`u1iP&I?;+W6og=&AeY`$l3ZsQV@ozdyAamT!eq$mK>|-VQUToYFTOBjSJ^4;2PI zFY|t4WNid|jP(1pE*nr%z>lm=T-~MRXnaJSc=f$Yyug%;mA?jO?HXm2G>g;~cca1` znO~tGkeMAP9~Ibk(ZwZ7hTXkINa?_jFp+83<5zY#Eyt92-Hy6GQgTTQ?blWNPnt(J|y9X*f4$0upF%uK_gU4TEL*cKmD7X`*A zzVT(sOZM5%n|2PH=`sFX+pM$NK}&ZqhLy^gX-6?>xOfUKs7c$oENX@W}veD|-($Q}5zjdT-`_aD29xWRet(BZ>#OMp+2g#p4l4*zSRH{S9d zkoyro{LG|XRA8(9sABHS3(>`k+jfrlh^fz~#N@k})2p9#2rBp&SCM)1&Ub%R(8`LOHG_QD~Kgu>v_bxALBSsA1#3H4jmVb4E?T#eGCdQQg*`m@-(jJ7)k2{#x75qO% z3&>{W#2fze4)aB(K^m4xI(CY$LG6g09W{rP>qOxYHL-rPFR z`^C-iU}t@c3wp*m8N>V37K%M@WjLyTxjNd!Jefr-4B{zMrDePG)0J{&PNqbm#qU3K zGW=?a*YgT7Qisw1&ezY*iZ{CmbsYdt)QpUtae01+O6{b$#*V5hYC_Z^q{FcKC6d4yCQi znCtiNm*z&r98?KKsU{)U172>V*K-ZeHDmC#vV9eQisOQWS;?Aq)xD920-|8UC}l=8xAB z;2*5sl0qckEn0FCquFx8)Aha9HOLqC7kkEQzzcOBO>2%%K12Ex>1Wa@C?FJfY1<}7 zzenIOCDgT4Qa_heg^kw|nIZNeV18FyEh!%ig*0tpM7P>MjAJ(7M&q62exp2*9)I}EotiL3HPbH?dlkMvK^a~;kZu!pf4Jy2EmN9tG|JsIVohb* zqh0pWD+>tRw<&h!W?PT_EcRp_?CUYjZ~XRH&_#Va#UEb4;UWSpc#!^VU=s5gtFREq zv+Na6g{?hdHE#5+jfS8iV?@8*x=d1ouH#pV{{*Ae`O+Wxeuz26n zk4}s4;pFyLxP?ld)p!Id$NMXwrM}rB^~uQre06puGRXZF){KVo6Vau8M}am|RCZs{D<RRtY{1@c5Dmbg3*ZEFoD6Ryqr`u|{+S<86B4bO zG25tAMpt6JHsd)z4k`T|hiJx@?~UtM(*sT)v)xVG26(y9;g^5+?Fde#}JDM z@-r9LP1=G&RQ~HeHg@HfY`pIbhK^9WNtUJ5aQ#twW;Yxr58i?g_WG2#gw&fJ4R}ed*yKIk@Wy4IrpxNa@H)aAu2&s??0C!F`BUO`2c8x`$>N$2nS=_6KNTRU{IM!@!_ObDB{I5KuG4*huz^_yB0S8n>UDf&HqYCSIR zwx?`=`Hp9xaw9m`hL~ce6M%`4`;*@( zW**=gOiaWZ0ti#$MSaMx@6}71BQe#XVEU}D*6%~Y#hV_)64`J?R|+z9I$?y0T)Sv=aG8rl& z=iR+b%brJc=Nnp*A#gz&WIs{RcI1pY2=;Kem6Ap#^T+2V$j8I;d3hF6Q$n0VT&j+T zO4{{U;THVi)9qOlkbeuy32YD^cU81 zMw-&`@$wgP%bf2twbQ`7Ol zB25ZIPH(W9xp(C96+Yb;6x4N8VnYcTprL-0>D43WQdzHRB@$_7=lznNdt8S1o~R8NO}^lsg$**;JO)TS*fUyc;7NuXFo$C zy-=!=zhV+}Y$*=itOx*;cW$Jtb;&gl042qwrYtLX+oxWSG%K~IqjZ6w`idp4Ps>?` z>pLtoq{XG1HZM+tynpJpJm-ZYYpW^CvyeQ&0^d%FCAUN`Olw6fjJZujG&nP;G|>UK zHq)HdwChvS{Z=pyL2OSmM(8$D(X8~~fl(7O>Np`Y)!C5wqX_gv`A{eF)D7hCP&bzj zgkm7mJ?jIMyM7)uH7KTl^3q2+rC?t1onI0?Y35cYG0iLUtdGV1YW)(Uz@BCj=eD>1 zH?eXFkF0T&`1R}?5M#hEvHc2+mTA6;r#l_q@zK^S2PJ4(?W0FpHMKpXf1K0Q6b|v+ zzo#XGxPfp$^#$y2ffR1t-x@Fj7mfTlTrUxQe89Hx5sSSi0A?ieT;BdnC7_EgFg&Oi zM{Nb=Ju+cvOUviL-1u1LBiulc(F>nOZZ%F#h+cqrBkuaAia>~(u-&>$!v$zYnh`bG zNN#y9VNSgxy)!Gu;1&u5M|d!L!Mx}GB{P3D zY={|WCAotd-amysn#>=aH&IARq@bRy;?Hss%WUQl69bM;&RO4+ zWNRKNTV}QIDRZ{wODpzl^=G=mh znDzptKD0~D5fZNA68I7;0R`lj-dl>2?!?HK(6}AH;?~ljZ>d^=;C2h`2Yi^6GxK7a zEW7(}BU&jFYya8eS=syOf59jSNK)+Htqh1Yius73)qs~@=yEWv2@@Fzgrul?rnxRy zOrtE>0LG*B^L2kVD}b?wJbwN9h1zaaAQu7MU9S6`o(4i#MeHsSNdJ2fLxoHvs2WHbk2-JV60Z-U1_ z9}2`gp3^QFYfWJ}aiTyt&7k(fBz7?tSfw0>3f$d6bS#2cu@{3#o3I(C^>(x@xM0JP zQm?U0&e=}&TtRvb=SLgVw2)Ok!TRsNJpdnUtgj?3@#F>uHsf=G(_CYbE{f!dkpPNZ zl`9M2R0RjC(4Few%sl8$&OP5AlUIb*XngQOwA{yTw;pQdNC03(Nd+7yJ9hYW6 zf_Tsr--=tO-)}vzHbO&ioL;S70%m<>cKD>>?;?+La;bN-txdp-1CipbZ@-xR>?gT! zQD7$s&P)9gj82xg6Zli7bfmX#ku<5i5)V=L=w^b86FgQjPXh!MK<&ZAk&B4fYF)#M zSH+}C=cs*7v zdmii!{ibn06yM2XFAxF96qNIdjv~pyf!lY{cEmoWTTss3~!F9xQqV6S`+N!5Q{VL_`FDrBTqDSk9jPjG^mx zS|CRcj|Q1_o?9K;_DF1bC<270{P{NHr#q?6`=~qXaU|PSgW!^mX9RQslLOA-0|Ntt z*~?*!`A!oZnb_AXjy=2@DPfM*Tx!@cbx}L_9s3IM610c$=;i>eB^6*#NdFa$6b*ycw{RV#}PzdhPobHin-=TbaLLR|dR*&8_(F!?K>f|0o? z7&uoqwG}x7ZET)*1WfbD)61lhtGJxtVHki*(Cg3XH>}Le}@u*%yY^0s9Hk z6uQ-KGqHrwss}{A1bKQ1q`SYdfTY15YlV^SwjWKPpk{k3 z=Hfg*)(Ta?fvtZLVG^5?2x6cHk!>s(f~{Y4ue~@#9#k-LyjmD9%9|tG*%r)aOg`QC zj1{E-n!?DV@ojaH$3aFh=ljDZ_|a=iFv!qc;Y~DI7qG0XY*lHV%<)85;She)c@hb$ zz#OvTn8$e_hj1!ksF?-h1b)G=yrAKF5N3|UYRLc8P2hHH##(OW+FP-(L<3tj2*TgQ zQdRz-cF9W*QKH8DWpUEVjb**Y?yE+>iG#TYQN{B1MAUTfbrb z1)Voh!KT0|w;x0rTG|Vaiw|AUoa|^s>4BX)cW!`ugJWx@=P4BvjtKi(34lY;51yvc%HxCvR_0OWgH;iEUm4b-}mFR z)W;FM%j{t_7%Hwpnm>9u@xuN%ZF?D7xZNdISYtNVoj7i{(OCT%coCD^^_ z5Q3|tbJ)z>oXlbXhQYq*+~zPaFfiaec`ImT1fKzCf*{5~3X{#v#beJgjrQ#j&Nga_ zKa!gT>=h1a-sNG+y|;FZd6HWcK=>Oc(Mzy*Ap%iVT-@K1tSz>&YJtK2kc5Z^9oDcE zgm4JVPfdYCS&4C=ViXm2m>(NDqK*>=mV_)MxtfY)L_R7yv*s#vWCG80;wTie8fm~7 zE5<3B6uRxjQLYYSc;c`EM)2?I;emlBi1(vI)ZaUHfG93<`m_OdGu#%jwb|Ppro7=a z$v7C{Eak#Lc-(b{s|GcAmK$%Tnp908<)S0?>cq9MBcZz?(DcG;)1p8h96OH@;+D=c zWh^+!MD1q{;+s|Tp}D#4^2{|TWb#PFXfEAoyY0H#?PNr=DY~ms~ z_jK`ahdEUmV3q>!>E&Qg`C~0HO9#Z?%dD%aO27v&{5}h6#V@E~0oq3z<1+5ILM zEsufBijzs*@)j2$UGTHg?De!dgd`we?08`9AF+ubZMD2eY;l<|;iZ1kwk^J2g52-w z<2M5=uZNB|)oy#Fnrfiw%qlG0(cIh&-B8D@Crbrv#7cgmz54ej+j(?;e1?q%v@|$6 zUM0;4CJLPG8+l%T-M@N(@CyPM-fjk7#{K6Gl{UjXV6N8IUs|u^$dfx;my|Nv=X}Nv~KuE`={D z%&HwQ+3HRvbv`mEq1iEggk0qQtgtB$_x}Y-)>(BTi_FMl!BYSPIO4CQeok=LeyU9Z zBkiN~Ee_xR;L`uwyM_GkBc63~3-;5|Wv*(}hfuB6NL=Ndbvox{q zS=qNdA+|WwqbU}b@Q>odk=bGXt_ig)$9|=9g^{@dp^Z>sT|-&$J0-|R)$EVpY5a1E zc%x~_bF$OnxEN>N_{wPxi&KOA_I*G6YFjDTubF2*oYoq(4h3@60gU8=4a&i{-_|H#7|B`!YHj!fS@t^IZi@ z^bBFQu+_5JY<;1`YN8tD^@4sl1?yRkx+Fyt0;^Jt`1tZk377S6?W*!V8@UQ-vH2^a z^=lI36BJu16i&%3eUw>|b$#LAma%cs(wx+*A6D{8tZ7_EV{C2bzUw1h zX`b03A=x3S3yGo3^cQT0f?6@ijHN#&2M1{m`T2=qAh$@0%9iCm>G6GTva$I0j%&hv zx}}8-Y8Dp+f{8lRXc>5pl+DOwXKUL5(PK0r^*$z&{$Q^Bl-JSm)@+2wP6Ms;W9?iq zQoU}FETZ42sM941cXXO|d|j%VJpxZ;(tT2b%c9QD^w_au!otG8)m_VU1f7;EvnLmJ zQ&I5;-B3zYKPqf#TX}yv@w8&NPsnGT)}#R&HPTG*O7ALSh&5hQ??{=Dj2po-UVvH# zPU@G=WUSJ(T_c0$$?xj0FezOgz!|q>=nYyYi#S>k+AdCMR_1Aj5l>2Z52e8s|E9;h z@{ikwtVrGnbzh&(izTb!>o+U4_i+uQZ;jg9(9)N5W@FuJAshF&h8xgWrJ7*S@%eFN zTytXQSG)R+>0USxtTAR0ZfMQAraR0Hz`IqPi;(r=w->tklFQEX9lpN4L%P<(9n(dx zo|dk6GA^(&RV_PP+uGW`DsWwxTp!GjaMyBMMuM7BjrD^`FRPDht z==eoE%%yxPMs{3fgI;8Ilz5FR(^IRn>sjf(y?g!phK3BBjYMC67e3e4Tk3;#d_&G* z;0-yzzpwa(QS;NCt_XQMA04&g^-iHC)0VG}{gYx`_xc70O~ZzIdvE95t%di&AY|=W z)8!PztxG{g#pNsJGL4p-EG<$aaxbyV zndAbZ!mB$G_I#x78D`8qe;0LeEEK`u3w3{)n_OL*0mO^+bG;Kn{=!V5A~v$}?dKD~ z$&S5~I7`evZSVG`J!GgQd1h+rx#&Z(LCJoNq8!D*`h^YjV-*8EeJ=W=*HL&@j#yR* zM~DqyZ(Wh<>Y`!>eZ9kF>fZs?%6ft&HF})+`yn$h*zNzEI;d(V)=;Fc+m0iaG%-aue6Ys`2fRbGo8nirORLpzFm>iykT)dU zLa;9e*M7FwIcO(rHS2Q;YH9tVmup5E7#Fgsl$-3xJjpt`5niV;Ffb5+iNM+U<{(lW zqXPqnONkggjg#>T^!v)t_hF9_ zpBo{<9`hQGq=I;l&Sf41xh3tjlA?qUj!R#NhgVal9%tCxy?eJ(>0t7SlyrK*=EUZy z7LS8!*_O+iO?#x!6%R?V*LPxdq@?nbR1O|KeAwv{6A$UlC+$)>$+mPSiN8o{y6y z)X@y)xX0o9)Y9}4$h&;#poL4AScY{%F!7`8N|U4fC7%asVvIa>t0T{EX;%dvH#al8 zrK5wku^tDC`Qz1?%T}8g&{a5g-l*|e{BP{xgmx=#0&^Lf`7Ad|56Twjez*W@X1x4T z89_(Pypci@6e85J$SN)gE+!An%$D>^tNFX_CWDv;QrEod-?UHs+oj!1VYmJD%0+n?j??iFTKs+LQSKoWUDk9zF&QADVea zR7YnF2W|wx3h&%E#7I&*>9*IT;q&7VRY9~u(!3%HWLTN|%vYOQ((mV;J!#8Jq<;2$ zk)I<^xKbwFP5%l5grgRjA3ozd8&+i>?#jKC4HwN_njn7$%FJ_{|9Z)jd)!pNj5TA! z4b}wKz(`(t>Tc4O!+hNRACxQBR*SYd*yR0;)fN%e2thwaMqyiDB=9#;NAyy=SwxN+ zPv)(i6!-G)8wfaa_UvgS(sa1@4mI<4@rzpQ{cxxTjdXc=dErKi@5GY-M)~e+PW(ut zdMK}1!Ea`1xp09Hv;?e6@*7$CwUfF$5c$7>*T>|k&_Z04lgw+fezqy|xrO68?nbti zlBfQUo3aJLvo)~Xagmq<2%|Ty`A9zqFXvUgzqqlnacI|BBBlX3X1F~ZZcJo5_utU3 zC)#!p-Lt}lOuvutjWXCe43_(oTtu?S?R_cA!Uor-etqRS1y{GCwK!8YL8Z{mG5Giu zdy+Jg58-obZ|Q|>I$=Ncy~h$%(rrh*)Zu!c>xLhbtryy?CCjRw(-Pbrpo%Vn_~&rY zMe~iQc2gZeDQ97#z1WkwJx{xIkl@JrIz}x-QE}J8OU*)wFi{K0{r~#s*FS_CtIK8= zzf}(3W=67P-nMrf#)_(YGL&B~te0hd1!rTues$%W^LzvClik@|!=oOAJ}8?D7guqH z&Sj9d|L2oHGQWHGuAkk-WiWxi%Wib96q)btGPw>55>KGb$w}Nu(Qb}RI+f~}n;D5d zm@Y!3Y-m%!Qbd$`6Ee-Msuy@h8xomg^t?uhA8X$94-O9g`t?ulGa;MF>#?fYib(P$ z93#ulf4)8aw(#L}T|aK96sPDBCLbYgMJm$SHy{75rzaqZKj}{LlO@A0VY?G2PB4h= z^98?>l@%$mS|0uKH`7bZ9oa73jg9{4Gjj$vZKf7$pe1MIR2{|-hb&u zwtTht#(~+C);gC+OpInNPZOk6*9D9^FcP5*_mjG_}(IVpB<^Mb-l(L4-mC5 z>o{7qq0A?U%WEk878@2&gneEQB1*u@J-T#-g62#|nsoG#$LnZWO3J-Qr(1y;p_puU z(U|YBLz{VYNyvC(I#!!&>@%TOfKJ5}Gv#Qaya$V`SZ$A_eGVQGjOY%#+=sT00RHc#>6Mhd4@{hW=c4k>y zgZ7yERxr&NEe1~!2!v`Q^0;$&KzIZ#<`yQpeZqc^ul=m8t%bEv)>i&9+uVoufqIqyAwDT5x_8v{TmwR}v!7Y*drZXIv7kt5W3)Bb@O25Ov=spgbwL(`Xy_;6 z&{fmuUje4g83{ysIkpP21XcbZE)Wmv7`sB%QoHX;le!AY0GnY zA?IX>8@;6#1J&(|%_!+j_LqfbOGob;I7llSEkX1u>-?1WqyQ-XTRgLOGE%qFMB z#~bu*8EaE(T|2^Zz4X6+rM1=~N7Wo2`DgFSbhvJ1Wgu7NzU?Ts*hjWHj~ZV7DgOO0 zJvIL~1Y~X?9mOlSY08f3zIgfazy4qR|JqCE6||nn_mBKFIJf}0p~dB86_-XFa|T^G znbFbFb4sBd7lx1EE38FeMIh~~_*vr;sUDqhaLk03qVU%RVlUY_G97(cuz8Qj{TOC+ zu?0W(`OB9tpFj82Nh>9Hwe9=iM!hL^?Q8zVy@qzK`;Z5ku_E|1zv}WZW571@jn8d{ zO$8l!?>8>P{3B)*g@3Mi`lZBmmCaU?7t1w8X@aVKpZ(D~I^>$}o4Q#al@!oo(EPmK zduX#M-=jmtzAI;G)6>exePN-x_jKhflK{Ch@0N?@-<@cQ$0TUu^^w@D3g*L01#C*c z$;bcL&|fPRH!(N2@I?jXi|zDpWE&fhbVTkS^{p=Ob}PA%1w4Sj}u*L6uTR59u(YqdH4SP9*4F^UB9)zFYYwwzpID#|9vg%^j1nj z?5^j}ANg)!`%C`n{{4GpSTE)M_5L(dWY_p?P&eab=t8cIp3S7WZbj$l(hqGOp$s7* zr>>-$?93}^-=LpIPL;be*bn(8joe%-f;1oWgID2e5$yW?3ug~*)BfQ@7RlM7@`6U) z_LuGVGGxJ~i)oOStiO`BYJ)PyOP=Z}Cznr$FWBxlECUgN-$D~#d!-4>b=i}7?S8gQ zg7QAXGm_2&sD1G+_EXXFND)n)Ng+wdWNxh1Y}|*eAF3mA!qY{p(QnObuhERB)QA?& z#vL_Dt(|Ko5I7F+%>RTE@+6DcQd#)M>L8QUhtsj7GO@K`x#)1`Wv7KsemJ6<%N~NO zql67}wFQ^;L6`Gw1`+s3rJ;N3xG-6SW{ZKcx5#{pf}-*@*2emJQk^tcrz5GV6FM5p z4beOD`_h`#S5ObhFmagjq&u+Vr~5Cm_~k^_Xqf#7>zAU_KAFk_d-B)DO>J5<-OYMS zAkN??&y_P(z}I}wqKr8(K5Ha)o9dILa`q%)QYB9y@}5vSGi(ZTm@uR*=?^4`G7a8T zuvCL==A66EEpq9azBIGv38kMrMBC(Xf5^^OWo?9~?I?C`%ZJn4d$d1!{~7*( zIV5_$_?q4g8`gi^eral@d&9X#WT9cMYs=gP@BVh@4~A{Q#)$$t?1G1GieK6wU}8C& zt;o;I9(4v!xs~Jik*s4oYDg0(dnY;evtRj=di)Y0S<6-4XmxSgd;%3i=b`uR_F)pH z1pDEr2ya27Ks9Rt)TNO6uVKs{PjBCMq302dg5xhrg6&vxslmX;`sBv-<{~MUP)p>P zN$pPat%H!L9FSxQw*BqFGE?F6VaR1;!G%0mlV2*%l+xRDblP-_z!P2Qb`eY28opS> zl9)47F6XjbKW8;QJB)JPnq^eO)}6=B;!DdRXFrF3N=a487kw0N3yC|Q3%{R-+H@^_ z+eVes>n&uU7uxw|!*F?u%X!*!89AVAq)JXiujK4g2f$e5_dln@s?6;K;iyE@< z+u9r%zH*VB2et1~GjsyJ^bZ%6=wgq+;9F85uVoT8gDzDJL%-3*JU>ts&vn+SafB~qE$E0B)wIyd$}oN z9ZQlu3fXURtpT}ZX;uX`e=OfjG&dLUj_|V|47EWTKeb*~NLTUqZ%?jjt_^*$mDVc$ z?woGt6*UyN$*Y7~>E5}|oJIu-uvuV0`ph6l&I$W;b9c`w@(%T=5CS-IaxvY(@avReJ zbK;`z^ww`6g*&-|Quzv&*=M}o?OkO@urz*N+Q}M)JALJ;zM!oUrvo?bo=vA!;63v) z*h6k~B}N9vd4MOp^+EP{pL-rDbn@o6I#kHGGW!mcn(Zn4in=nD$m~z>>{+8kr#<9y zaB(Fyt410A&eW>&?U1{uL?Phm-t)E)D8(NXO3lKsQE^$(ja3poIuFZ&hoGKGpq`{_^ejQ>t)&`^&4|Pu}i2vyw+JKDgua z>aujj#`2b3g;QiJO8w>dxx%~mEC|q$ZUv%cj3pV^;ti5MYR9(|G zi~$e?!3YQf3QCrdAVH!?RML3luZq~)C>-E6Ox;o&61F;qkx&sp!Tzpoa74i=pHC21rQ z!6?f8F>d85(SwtfeJR9BL6D*hWosxSBYqkRj!|fCXcdDQvW%_d$Je6{mLdOfn0b`1 zg49=FcZ^Yo4$-G@1Q0y){QSJo)LW@O?Pu3;>afczQ<%t~{Ualu8${9WDD@m)+3rH7 zP{BXM;=8r5)DSJv#({8v4BOUT2lfcO4KjdY;5|X)1D@$^RnadQT2-S1B3didI2}=Z z6@-C)*ax&N@n{nXWAor<#8)35+Pb(14CZ`?J3+vRFrhwFwvglt z++FqVpP+&mUZvE3g6L^0xRI3*z$S6r&bf-&`gOTa5OwbUhuD*z92$U4&<%p=j0 zKSK6@&x-9pgb)Fc0xo5@a|lyeydSM=QwC<30Jq!$0U9Jr$0Ms=JLwtO>G*hNd58h0 za6J|oSwDESbghe613@g25B3{CHkoie8y(`vF3US* zO7an$I)$tS2iLzG-Qsh>Xj~`R39WpAqNb z_GH%@zx_4&TB`zjpAPL zrN{XGYT!;|?uzV)r%bow$Xk_|@xTBVs;Q-3H$ey^6QSp5_`lPC`45NvxrQN@iC4*)kuj^gJ zMV5hj2o4e4f$eo!Dm1$bkqBmKxIct@Z!px|t+r?xX<7{tgZ%Em%NdW&U$Uu~u7Pc( zd&_)K7ze`$XUUZgK!7>%HqoZm^gM@Mujieysl)6uO=PN|B*bSu~gx#7uQ zxQo#n{}z^IR?$;otOgIQ*vn}|Iz)WU5sq@LG>e^q)l|KSiW3Z%Y5{MtkNAGLEsLtPNd!m2Iu3lu0(6oTSryB z+W=5dB?I#kovy;_v21Dh5tJXG%P1ML$dILXohqA6c&EL5Z@I7=F6+{xD*J0#+2hYe zD0j_l28pp<2|}(5&|a7G$`|FtBUplDVx|e-SGciztuT&WVWQh}4i_~8Kk4gel$Vzy zftVxbn~NQxEdivn@((1{yX@nT2XArbbJ(cN?+YaGNDV-dW`k}2wseP6N9j!U_SMu zS(T-BA4Cv$z{entQHtT2hUeM`oZ~cv`2xW}F_sTt86dG)UcgGAc!j^w<6FS`9rddr zGXN0Q!H%N`&j62F0o=n|-8&~m$Q0Q0`w_(4db=Wcv`?I`8sD39>(PhY*#vhP)lEot z#sI}M(z|ndh9^ry68K5+?%-2^aI_K`~C0{zBIXGPv0q4?KW^D;TLfA8bBR0YMDdD$J+kR7!4!`LDteR{Bw52 z%EH3J5CB+Z(XAe_>L_F)#)_9=($)hxvP;+M&Q1!B!W*mh+HB7aY9cd{6txRRDheZr z*dS~Q%fC)V9_F{I04Qz8f3K$gXPbrZAOMBFmk)>t}5r$d*F1Q@P02{t~$AY1Qc*8+V z8VylyoMg6Z0>Dt%Hk1Ye5LsoGllc+2@hVuiu-7|}VmDt~2MjF|PXP?`K!Sqn0gPM# zvn2+C7oZqmqwFCGw+(V@uOF0Mkh%W)RZBVnt97q8$7>6gA)Db-5C9=RIK<}NVOM^N zfqGO^Kcb1Ut0QR}C6-cNygK&Yz-=Ca!&v>fdsdM9aaV43ah3dj4y#8J*Ue>hS?+r|%@uz%&kT}3* zy*_DSZ0>FJ?*#!9@w~D5!RG7JKFDOkmdmVv1eGFQaqh|}g{d?C_Y+MgJgNQZnEw?g z2wsMt!lTH9DF4Y%2#ls)ijhg3yZ=uLk6in&oYYy7|KoT6^Xe5Mv;zaoWwV$7^VTzs z(zsetMt+~R_6frK!FRJGNmRwyp$)**kmGrXxvRi)0O+thP^J+0?>7>>jAdm0?@Xv% zJcCf8@FSu4-?!8-XAX7$%21|Q?U ze;N?H83Paj@`2+i0PJ9_QX4=<9IUsdur6#6Vnf|a8q7_&^Z=of>4<`OpJacmxWQG3~Sy^QlxL3HMd-7p%7;>UX( zjtmFztgu9IfROJU%hOGt__w|x>t{)L@rRH~FBe=cB>Up4$7(Ff5zOn>y%$M> zfoB1*;|S%UjO=oYBa=h%76mps^JbGUQRk^}Sx6uj6w>rvMz(CR`@^tZKX^i>u^og9 z1*&Wig#aNdu$+UapNZfRa{$x=;Bzl%yXy`kQ@vBUYU*`AK@Rc19bVf7FuG%?U_g0? z0@ZtMz}|E>EfYo{X%Fc6K)FvcuF3iuD z0a*d#mj#+d2G^-@rz;=IFynRt)csEH;d~H)f(zxNy}d;Mj{fF724y4|_BKFQrC7fC zAj5+{kU6l29RbjS=NXexb%c!9Yf*Ph$EjBXQmUelHs$i|s%5;NQc`>q&K6~RHrLP( zDzJ$ZWpw2?x)~{3`<)um+!P479y$pg-UigeF?&FSreYAr1rcw;K9Gh^N?FRIKrkR= zC_sW?sLpiset<(|fHAU%350WBurMMSTY1Pe0L8TB4PAvN@)39a?ma11P{O*n-2UkD6Z#8;u|p?B1gNEaKs2gh=^~cDQg@) z5)aF-^UnUh@eQ~VuxvMxSz5@(WN58FN0zTv?JMBAz?bX+WpKy<<^ueT%h975Wx!X4 zk?5*bTbNUg#2f1mbl}dchP_}ift#y75*B5VPW<2a;&a}Ozt8}3-B@&@$y>OGdzGbas~3u;&2{*@e$opy!=h*gLpWf;_&lUE@1dC z<6xVQ4UpLi{~kWGVAmzj0?=Eke+%+OF1=C}Hq+s4FmJ9_gyI4cI46>{ek~Eq38A!O zX8MDiDt>ZM8`Gu@Z+0*t8|D-dxvMZKEtz<2DxjX;}_9*D5PLlmPIfPlf_AbYWe)+T>r z>pkrOs`$84g9MIJO&YQp+ReT~f7Y=!kLZXsg zHWLel11UEC4=I^;NgVTYb7y=Q1CxWzl74@O@9Z4z)zq~HMx*Gc5RP}yY0YuFfiM<; zllozm=7HW;$CxDP#9TS-Zkmt4We7xP>O1o}_xd3R-P+t_#ksk;{SmAJ?27T?8Z*j5 zhST?sq61_!NVY`9`FIV191yu72{i)oN%kTv423kPuOgPkrZGFGg-d*P{~Ax2Ra8Zy zR%Oh;QB~kzu-xm>6zpUQJ(@wW;;e>8VWoldOw=f1(u#&p3x+<#U5Nj4I0d)#0w_Rb zq3K8@#=!*dwP;oGKnL@lw_vE@iwQr#0M0LBQiWjnAoQ5P^8y%jg{fgtjD3I&OlA$8 zObHVc4xL0k54rH(j6k)}uA-$2gb5DZ4>)iq_ExiRE!{th&3jA-x_<})g8oy0TjwG7 zd1wm_gXLhgtT==9O97u+0{IOy*>&mqr}B6~PEL zZ+*1NhhYZzfB(n{$P&lQit=*Ej9@_FfOBzq?k5R96;B0Jguh5|0e43S&feb`p>=^J zp7ny&#{kI~JlLDb0sA;`C@3-*awJvm7s6{aCz5bTABAVJug(_P%peYYqi{V>GYWCw zqJJL0(JfouM{*IcKv8CKE9xI`H|x-Av0rt(RM4@#p$maA2KovhS=)yBJl?N9b_Z&L zmw^$Br|??yMItpY8!*8u2-gQhN}0nfxVtKZ?nAOj+7cP+kS-9?8Q(CpmSCm%NGT#~I%pBtoG7 zB8HVtkOL2v*Kh^OE{T-CM)m4YWc7BC3dl!@04MCowOfVIwxVZzpxpNyvV0ANnA9@h zgG6`7eT}J)AgE*~)+ap|Ag5cAL9<@E!fkK>gj;Dg?de*GKEPxs;v$Wy5fldD|4Jj$ z%z4x)MKK*#Wj$;R^5Ckt)xa1b4)Uf+VF?O1KtdBr97&RN<*itbq9ZSID&{-m9 z$APhUfSkwgzA?l`dpe|X0EyPD4Ql;cV6JZPJ8<( z8*o`oxJ|mjl%zYmRY;@X#xP7b6{#Prw_N}w-AVOK zFXh&+Kt%^0)TQnfi7cJ{gRJTPawOyeofw*6H)*>1R~?Wo9b@!y_o1xxxxv088X6XsistP~vb z5$3U}k#695AeX%|NcxK1Rn?Kx+{G@*8@+&(`w*9w!n+bDi4NcWK5+T-{=_t9E0mhAcUYV1P1zS8bUwBglgmZQu; z<+eiyU!u<9;hN!rySBkFp3C;xiRpYa($ESZB&>0~#1Jq;+0JKuk zZa4s2^%JomkoEA*%)XpL04K!s+oeg8R9nDpryv9$cXe>k)!@^~toUYSul2uS*l&Xz zU9GkTUQ+_x;ynA^U#5;X_pyhje<;L?j0Um$dQBHx8v0igD;M#@2Q#@Rw*TVB(x>H1 zZK?~D+~>Y`W1?kUL&}C* za=J^q^QN9^)J@66>l|hF;wTVX0d&us0DjSK1R)#`3zWtH92eaYC;qUqvV!EK2=n9A zOQhfP5$cdg`5WAbz|1TGwI;Y_1_Vox)9TK@yX-Ze8IU84-7(Qx)H6l3y6)9u-=Ey+ ziLc7tDAy%#4m2%n>ETbM6J0JiUiD@mZ?2PN8BuystJsm~)YF#{%0Xr(lv7#x^KbGb z#c!*CHZ9tS%dO^7ZG9k;NlT2%YpSzWze@Bl3 zc#r30j^;iuQfgUPXn2}dju zmu1g*p7md;9=&HaTp5+hfpaqe-BhFPW4HNVF5F^kZSU@4S0j86Tnr|ZZ@W*W5ujP{ydj3T? zQD7K)`4rigC z4e{e4^(zP4Qw&&}7UgqCiDk3Z6$esLLHE)xud6$GF9z$H?1x!9?{I{%p%_;@*Ox0@ zbY%Pa-Dn$|!&Y`@gD1Tz_NZ2m>dL|@*Za_W6Z?r?E@Qm(LkB@F*!8J>Ti2u=zJaxk zY9ae=HHEYoma*!IS*#SVp_jwF)@|1(!#TeTM(*simmj3Jg4V>=3h*xx1FyjhR^&KU`0_$9=V6 z@`Po)T~Xy|4@LI9=J?}BhdG%q7P2X_kKOE3<}Kr2v<<|c({tS46WutfF=SI3?TZNu z9w3@1Ujcjz2{z|5gku0aw-0$E6c3>+|A6-x2W!~8M<$|yT>0p_9u?w@nH!GRf87&aex1w z63-jm6A(KNc-GQ&7<$%O{P1;FB>H&2Eq=As$OYF^hg%7sJ;q40l`eh0#{IFV-u*0W zfcN6&O79-wg<+fg1zv7}uPZmGZSZR{gbY<#5d%8b*MuCln79|>ir=Hz^+39lH75Tp zg6C&L_2GPJjI5H#93M`7w$J_5St<|5j-{fGZy9}BeBJoDm4}T>FKR4g<3RWSd;Q(S zpJPtXwesv`aVf*RzSs+5ZUuq&P03PPGfiFp;e(a3bnFmtN5#Y03dP+6~F z1wQ@(wg2Y;uocpvyZbj-d%`S@|KH)UBWFl`peFeiZ2^)P*bQPr6!;h5$@=htm{=sY z1uymj%0R5RVnFslS;L9Qz5rxJJx#NU`rH8dr63|WbAfby^R+yXJd{id8a#^+48nu@ z2uaQ((ct0`ot6SF0uze}cfy1hz5+p_$X0|v8aTolxR?qMXIOeR1m(gahRzdm#j=SwZ2_df1ur^cKY!RJ;%l90&WcwPxG~ZWFk`d7#{oYEC2?M)0Gl zc446yl>uV+j+?~5UZL8H+pchajzoPHMXCsyy@nkM4Q(F{6}3~s-E zWMo%ZH0}z%aX+~;uzy>7Wh-9g;S`mem5BYwVstzCWK+-ZQNq|<#SXS&Qc(L@$Zfa9 z7r#2NJNWR#(Y4H!Voim%-)31)YIobWfRB`=hI3JWTC2k_{Q4Fvx$uI)g?k)-RvN4pIdJ8v z*zJ@tz2xNFKCR@-3=Ja>e=OySV^d4#l{gY&T1maV_}IBDALlzgl$DWS zS+?F^s^`n`d10Qxa6w@6hwb4=Dx2c1C^Vc9eB5U1Ri9VAmjg*RXgA#<#*TMbmCvUl z1wM|`_+RL>p4_L4nklu7Zp{%l*+nIO%FssTs3`av0G}LqnFC3jV>&hsdcZqxvz2GGz3Jd((>U4NBl6HYyo zKw9uXz6>6G6_IuzYCMGIgS_+M6r4;Ff7J@bcQZ)ksx>nXiI zsNin$x1zN&;grB4muG=nKRe&wFp4AFyn=(bZ7vj(>mx7vw^B#`b_7@su1uaa>AO}v zBNV2$|D|NcbD-dr*=;8+vgdvnjrXM-nhbh@&K#ZNCY6qUUm~v#V}1=|tE-<#voVrO z*xlO9u;tRvuNym_T=Y{F+O_+_rJ1-tTY_A*JhMn^wR|_4D$!%!SNfrTY+2?o!xU+E z`t8MSt!v_6FD<8bkDnTe-Lxlulq|u9+r%?Gth;%vst_~TKc~O4%dbHv%^{XvHa;B$ z9=UR1;3q@!^w|&+1~mnYKQ2Ib`cB7GM#txr-l6Ae>@483Lks|8(g`3MNv@O@c%GON;+KdxFnP<`;1 zTBG{D^FUss8TSIn&mtzwwWAe?&1G3aL^>rYJwTX3?lX>L_25)@AkZVVm@neX5RQ?O zKcYoIbQ{pf`o#{!XE})vQ#FIvjgOH>^Vpu|6owgx$ zYJirbBEj)0XohBIuev2os<1U&ImT9qqZP6SHEuq!Fz&d3>pAa&72rMgO=z$T)302J zpr~HFCwa->+w2?K%=Nr4V|$HdVfx~`my5~N*rxrMro=isO|IQkVB=LZ+O>O=5~*Wy zKZ|0eat<&zQN~v%Ekck6~vZw$M9WcO+Pk8Wrk zNRufK`iWGA<%NY7tc&`uToXm&R(pKu*Lg~A;3n$HTdB4{ck!q@bCNE}_XVPw1+wY6 z7?myL32a5cTavw)mL(v+*9k|^V*&Jd9WENKf>c%^4XB=%$A!TA=Rp32=y4J0H&XZd zOusMU;dwiTb5}a(%3q=SP3#2C-QRZgNEt?~^fcIsg1aRIs5G+vVGbH3lw0!VCp{amisZork|qH7Gvo zj9YQVvMP(|OTuw856LOxFYAN``x}cr@bN$Wb+xsdUju^joFWc6pR~=2p zqWsO6Twr~{#7o*65%Cm|tRjFfAlS0C#R%Ar0$5%wy#+NO&*5SOSO6}KnfY%2ySuxd z={J}I$0gFw95O;7LVZuIJfRk&Yo)rHoB(@ly&}1Y;9?ry7OE|z2>|lG9(n#jD4IZ> z8Ci5v=hzH;R4;L9=YKB-!vL*FXK%0cf+Jt|hf{#fvm*b*9|P9VJ#qq!bBM?cAzReL zdb<8B?Wpgcjrj5?hR1nn0HDar;O9|zI!%j-2&7>E3Qi(k*vn zJM2UNy31wf=MhQzfTKVdmlUZ3dUotKQFr3jZ_v*KHvB>I+4nMBR~?f2ukxz0Z6#g~ zD%LTP!u9;P*ndrH4!?ZB($u1*V9)ofu;}heVW2Ii*UoD$aYct@W`iZp67r0N2(1@4 zahHy_3>l)%j%AJVdtbR{S~sMSCXF*K>TBzC#Hpi&HmXM$Lh|mFXZ|6$TWpHy9C43I zMkc=f3TD)wSSbJyi6YfbE9 zmwTVkfyemOo$j+19;$3b=4lnRIwq0m$ge7wrTW!Z9%D^| zM59d+FYEH#UKX|}fQ<@~Uamz?);j719t| zR{W8m1*Rqskg8MPW0Md>-MllhsNdsve*!7EgMt7B(Hlso8K7(sRW@?r{=F7pvXvnD zL&ObG2?jFtvw}7_g7|yy#nu3%Dkp{p>7@D7&1Dr2S%44#c#nrbaBM+_cQhv}j>Z&6 zpe-OuSIop(f_}K^u1jJ-PRC8e=Yz;=33U0tptS@H3pVgU3i}3weh)#bWKxf6hXOjt zc`>ouAdK(;zW*l0<>OA!TH!(ZYGB`Rc{2);h5-$YSSh5EAPI{HT%9TI?weMSU{A;W zIM)ai=5w$eB4{0vf+kf^gRRFT?N-?RQ|C6_Ci`_Pf2QXEMC_(r`5d`W7^V}eTL zXhSKZbkVKbKzDN+r+nliCe*ib(AQY)c$8=}_cvbj%hLGi(IRZ(#Rn4%VO9AH5hdD| z?|Ox`_C`(mu@1A`M;rCW`$re6RTP}|+h;PA69(?A)QOMnw#8TPZg(B)?e2yY^;yT; zTe9M^4;JGY3~h%_eDw0!tmim876+(Ts)`|2iD_lj(lP7Kr(OKLR+q6A6NR);ANxAV zkNVET;=-ge!D*K8QB(1%4O9HohjSZ|^{Gc|THH~;?Mkf0X51zecaAyEObWeU(spXr zwH7q)wfB2y>xt>B{aB6fL+dDo(c0v%l#-Y%4^|&FW_DN~wFih6U=PD|7Vlz;Aar%C ze5~%ilCjEba;=MEgNpNG)u3GKV_HA7_gn>q)NId4oL0WEo$HU)ztb8Nht4~^4#)0o zB%`|AS*~@x4T{d`$59$$p1&7l4Htf5XR5UlbJzMDe!h9vCp;0mJ{B<$bL==9&AXpR zVr2dSREU|WhCxuzoP=_WC!}IXc@yCUL3%y_USt6~-3|4QU$P9MAQJ~N9}jd=JO~be zrI5fXL_9q@*weKUI>|tv^%E382%jw`zK!r%knU~64n@j>6R(i!FUZm+!Y6g;UC@sEdL6 zR6$Wy!5XqMAmX6R0p)jy)EPmT_fM$;1#?wQ0y^H$y;ntHu=o&U z8cz@rA3^{NP6opUWqIOHLAj0)AaJk9;(@XiqDLTU#37PrR#7KwBevTmt!N-S*>$8uY~{tj--Kb^SM%RF$BAH; zuwu`JgJzWBCLU_;vgrTa6vcY&zw*7CQ}n7$;PEuazKsX)(PFDVX6sR!3Jvz_o8sbe zW!7!XH#^OkJtw$c>>Y=Wbsn!)?;lK2c5NI;9UtK}RWYu>XGa!*t{5p2BK-hNSU5i{ z5s|nXut#0pOBsjeIV3DsACGhf;Xeh?4?!e;&)q038Ud;bS8D%*Z8&LS>`4%p$5_sB zR&(#;v@s-}vnVGVKt4nE6gQmsUhIX*A6z27U)DNzuNr&Ey=ahk%u`YUMUCul2Qv}m z;@H{hF-u?Cd8v-7lHT_-oU8Mb>%`0V$GlCq%XNbXjgK1~yh>-@^l$D>e7a6bHRj&% z{je&5ZI;wO3egk%r^V-7STi|L?jyh-L;7Zz8plCx1N}cir+_|&P7z5s!D51utiF}I z`T$Y?7d1G@k|22&(pd)bOT`Z;W7skcA;gq&jV$HK61$CA1>x@|l#Wz~SS+TT&(C zH*%9k=I=4fvlR_dwQr4ntUmZ_NWL>f8MN}PiMkpOr3TE;juSIy4R`Ahr9Uh{D1;!@ zM-VuwZE$xQwF7elVFT&V2-zz`G&RtSI9_@lg%$$H1ricGv^A|kjn5Ng&B&XP9#2m% z%S8)5BWu|U?^m3lJ3rWnGWxrjXqrXbZogD z*fW`I_$?RzD%50qk?9Fd;dz}1W>2g_n%&=h;8;;dYKF`EDZXaSp5i*DtCM`e5`4Zu z9d!U+R!hXIpYmG#FN3ejkf!|&;xTr&o}p^HD2IQ%B)jxaAeb3_gTo1Gu~Ge_iOP** zkFIk8a8nk~n=c9^=66&57_eibhz92PXzgc(p|!EYfvn5HMk%DWS0%T?C9s75TB4x~ z{8CrWe_EkIu{!s^>NxRDvO=JQr7fU0T0_)qHDe9lHr!AxB@ygQIZZeB>s06#t1$}T zv1Au=P#63p-8v(G@;`t5zt@I-M>5O7C8z&4CXtTjEsEfAZz0 z6aU%81C*e#692XpfVWtjto`q&zLK2&?~^6n`u~Nj|Jxhi3@%+J*g8|s8klUBLuhW2 z%o?Z`!TgB?7rDA$wOWzA2>*9am~~H}I_*Vy+L#)u`_ihHr0wbajOA=61+OQwDqJvl z?3%Z!fK68ZG0pvbUH3VE*BjyGnHXTb@w>=b~=slk+74a2;c#(KL$Wn&i_C$WvY_-g z=G(L)U^-imdUWv%B|*~pn*WN93I6xzWehm+GWyoVT4mdeD$RJBvw|EFY0PKtcWkhF z*Y2De+o?yb9Ws9Y7Q>p%=`R>sj|w(v*Og0VO*WhqdcGG~Q)4T{D|w5#i6pj%?`AiB zy3LFD!xYR_+sy1m`G~o4P>0iT;94B!Aoy9H)YpG^S2?HVi;J+jKD6bgd~V2@?kCJXP%SH0rF$PxgOUFh$4Uw}>Vty+$m&r^c)1wc zhH1Rqogi|Z*M;-Eq}Q1)@O-yVZn@A9eF7cI0z{bga9Ovo-%wYd%^1rO{JSw+p^erU{VM%VnTw z)^lE6$R_&N+eUuM2PU7%Rp~)Um1S3>o_kqoSQGT~xJZq8K8adPJn5bdQSkE+tkPr}R<> z`$YCt_9FB30Tv@_e@v{uF}m_$YyNH)I`Dn0rl)C$F}1&O?3J{Wt@-9ce2KSQ#pPIh zPv?W?y=Lu1E%ff7tzU@UDx%C!Q_l8gWDJv}mYE-BC7U<7PIDON{`U5lCGj(UdPy;# z$s;U`?|;*LeuFphGzk}TyvqC&4xZaQK`q>6RXXG1H|6Voh$^zA1xQx0-w{#n!&*Eo zKz)_FKHg!j<6N~V^7+cQXbU2J9ei3!l0d#3+sby*Pae%Zaadx!XIkEECStblro6RG zelzg-lQ1&9FVp|opq%Ano#DD>O7yxUH5xCeEwrZFqCEb4nvbe%Nn$*PnfZaET7w+c zX1v;E>+yxJS8sWl-5UJs>BT8t&~dE+EvZqky1|R9`MQy}NyXDw)>3+n$5tqm^)iuZ zhtxtO>le|_=M{-&g$9bRo#wM1bdF=TkJYrXpB0SOxc_x?1!)a1`tTC5Gx_JAC#0vT znHAK|`Mgz&PFqopZCq08Womaxy3)RR_dHwEyyc#(x;2wnfsI1&+y?Rb5aY&)vcjKr ziylcGp3)a%a*uxMYA_K?wBk-riNDqMNxy~aBAX#SmWmzIQ+g;jLgAlHn~E7KzU=R9 zI;eU{SF=f$MLyE>=2A z9Ank2YZXK1gVijREfaoTT9xjjJo@~{@7exQ zBN!N!nilZz5=>2D-Tkpf&rZ%VoMn-CaOA7bd^%wkhyKJ#YwpA6yXuaKL(lC?>d>%( zxY5=$l&qrQa`6prE2bV`KOdb3r794!sM;$Wxc=OTMxkSp-CzQX`%ALsYLj^yu?-h$9 zGJ4D08$r5zX2Ffl|9r>nZgQy}nLxg^!h<~Pdr4mBFi(6t*D$JZZ7j_>rT zk}`$k>7;1+2(JW8k)Np_760oYOye(KJNf*&dLM~M0G{QoCxid>;rDtIk#*`{eJt=CtqZ^%3u}*iLIg{~>Vs{BA$C(4Zj!yy`wJ2~#e2Qx9 z6FS*Tw`I3SxdXTpg;%NmTxV+q4)p-@)qs z30K0dyLQuVz)-)eklOU83b(Dzw^9hnwN`katG4>UDwc>vTBSMl*6KNUY->UQ1NG+` za=ygY&iY##Q-2<8u8KS7^Ri{F5fi}QMRvp4PmSKK;pK~vy~VxnZ+-^yTP6m-Sm#&P z&iXr%WY7~?p0qgaZ$l^e8FN+9B31_PeMs{8in+frxJor~;^AVe;dAvq*4^3{8uFn1 zmhR&>v7U}k+7-}aEOGDOv)5aAdz`JyqkKUhb$4APEIkRIy%=#UId$)Q+FzF8TTNt` zn-}OWY%|*>XDOr@KE2j!wi>1Co#~^XL?m|lDgIrF_tGxkUq7?7Y4vH1MG+TZ(9Ne$ktW9x3Wt@yj5-Oc)mZjb`~!@>n$3?TcltvI{r;PW zUC zNF3j1rWIiiX=%IIg-N_IE_shf9o>c^8!nc=ZugKFeHKyYbrt51j0Sy7#aBT zktyRu4X=KGJxV=|scxes^z|lLa2=|Uvm?LdaQ6Us6E z6m6q6aPx{tl#G&@p?Nni-?Jh;6jvCo1-pM5#v6K9hVXTwd!_y zOJY+1fWS|bA5xp;&-ojtzWGJ?TrDyqJJ$c}9}Dz-!e^Q)uP>2uoFJ$%N8DwU??ZYL zKb0#KH+Vnk?Y<7iMx5>ux_-u|FhnQfiXuBUI3>)7aXj0G=U3U_$8)jDL$}NI4CL7^ z`&a%|;$$w>%fZBA{BB-2O;i$E%-LOR-CbyvT_1b*-OOFP&6Kz^zkYJTzPFts(2MmL z^I81*@+*Ld-wDH5qhg+45b0nFI3b{yVrC&9VfXPaUTvs~D^=4{M4haNfH{V;h*>`3 z!EE+5)k}V#Fw|z0ulzrKOl>xn^*4STja!sSzd~AU4ZPR=+G5TM6;_o7uKLc&fylOz z&N+}S${_t{nS5{Ne#aGCht{DsMLL|W^>69*-%J}XPna#EdlJd~+nH-ZAx-bZ8*$3n z@sqMfkxUP#yN3Dp88z8S>YGrkWNCVjg@c#Q$Hy*Q+^ODxHCSM{P+wnx6iY*WSlmV|P#ZoIvGw z5IH%iq~=kA zo^ftDW8Bq7N3r+4g}32;@aD@;L_AONLh6bSqDfjJD}JKEqbgXPkqR1;=}rXi&M8i| zj0%3F=JyHUDfL2msIWd<6vl!wK4Q8s_#zsU%D>5Mn0Qs1PBq%0Mnfa*X2Gnz zaqs$RVy5YWn2(jej0$TkybUxKuxET;K04s5x=mK!v>{TD;tM$$GuI0zIz~wi<$(de z)1!>MCfR&}^TJyeM0iVbh6ptj{!5m};&lsyDv{4(IY!4vR-Tm3|GO#r9kY^danwbc zE@!LDJ}2mjIxm$@Fv^Z$Vzbdg=ly$JewSG5cDI;5?%p&H@*wfM?_KxKssh#>G{9fF z~47_E7VpYgar5~}u`uHRlEbn=1fqrY-{eWWxD zgw-0DYmc&%#-^-^Qi!fDJ!_MQLN{N3+pIetYU^*jP_C(@je1m%;%uGLyC&>j*6fRJ zK}E2#vxqW%0(G(sJtdh@p&A$co3_KZ@Hia~bS3Hwur~Ug>QIkNLP|-b9`P_GW>9Kv zW1^=&#Fmd3sl)v6pHl7?N|e`BBJ{1b@J1d*M)hajqsY;>(a(A#d7?-)(q-&yP7&0c z0bfjjdu7E~As(WTT(eTMAu_h(G4^Gq4f9=<+W&mn)w!&~cMD%neo#vam&=RYyV!p} z%Hbzp(IBc+Ap%2vYv(0Xnvu37^^m&11{*FiP5rwVvMu<(>DFlq!*^R=b-VQWDsWaP z`9F+j+}+&1=AHa30k|o%FlaBbbY= zaT-x%t@&0JC1e?nhHD(7u{jqSNsCo+f>dPF^97yedlc!Piko|e|}T5gX2T0DYe_xx~JOFn5cUXhBu>!_}_l? zxfR(-!de5|N)|+LYVRor5_6vj^3oI^g!mwod`omS3QlP}JAJlm5E&8+u784c&5O~WM!W1D0@p$7>NjLUTn zzMxsINW%Xd%N++_LNsB5>~fz-`pfTWKP4$;RON)dhwc(fGU9wPEhuWvyws&>!L(S- zbk!SEzewm{`<04`Nrm_TIr=mesWT-+b5tmks&6k!@m=8wj?;30v&_#jQrEn`srH4{ zy6p3dxtHYo^`N~e+)FME}M8?$3 z=P^bVLlSKo^ZAO^F#9D^J)xIZxW}tkY&o9%Rqr<9rauk*I{!d5JhtL|&+oQkAV{q5eOAY?Q6xbElcW#%i}j=4 zsTrM+=xY$XzGi`!Q(Dkv%z-~6o5%OO-nvoy+_JtxplF8MMA01nHo0zH);T;KqLB8S zY5wP|>3brBTN!7X0OFYwGEz%aM9_dZCG!2x_$z-(=}0F8mC}0D*o8u28!5Vl!6gX9 z(X^bOFTbtB&zvV$~l{%T6DRk0<*}~eB?6{M;7nm_5`KVw=(YNGRUikFhr+nq-qj>q< z*DoJq?z*dC+#Jo4da=2vLirOb)W22#oe~jQ;hl(`dKH5G<^=HMxoY6 zvcH~@mFnU2BD8n0vQwGhTHA`x_T+hAEgvk7XTHOBTEAbYDE8LdDVXpqg43tWO&+t@ zmB|!8gvh2Zo61_6{ZN~<$1mhYxKEAp+01;2XROCL^21BuKrlovm68 zFuv4y{-fX;U|;l9?|nL2$r|!stwq7UN#pz8))jvDA;4S-BD^pgn@Zy^*>4!Z9EXpS z_k}Ri`2gmJmWlt=MgPl6{s{>YlaX$23O%u0XX+SPORUe;v!3ywxp)062fzCyfjNcX z8&)zY#qO`}zS)cFE{w|`Y}^xUhi^Ms&CRW)d=1Lx0eX@5jMQ+x+kTAQF|pno_y$r~ z_|7-wKcdN`=lZG%su4s#}_Vjf@Z?Q+?`h{4~}dgI87W;R}hl z@>Mfp792!OY8ocicrG@z`#>3#ZJSicm9Om<)c&fm=`|z2V=L_mf}eg~RPJty`QkBF!a2O5XY);Bn$)L; zCGBmW4nKMQXC8&fpKpu3y36`?-wD8O5HjSHWJ$P*OMa~U+pa&mlowNpBvnEIN}|a0 z`zKbcLe-K<(=YjdxA0c+EXpM$Afw(E=_LHvu3k=0-4Jdu^-g14uI z)tE0lJU6&P4s_}icOkR9kA!Q2tc>{k`{z!MIf=INSi@p0P+=V?mbR&9mM5#9vgy3^ z`PE4Um}EIHqYSCL#9!zhs8UPYQ~M`qMW4=IM0@eNLN+m*YLzhC^$|t(+4h9fTLN^|147Ty_6pOU0706#d`b42~1h-rI@B1?&(1@liKgN3z_Yb`~?S3}!%ukE)8 zfy`~r3+kpmJZ+z6-o&*%I!y3oe4Q*Pe?hUEO}{qZhw;q3Fz%0>F*UlO*USciZ9K?; ztMn$Z_|*ut<%g!Px>qr%`1+yKez%uZxD(Ew7Q9GcE?gApdPzFJ>%4$ligNf16xj&S zEj*rMp(bhu4WIASp{8n>2V+jy0Mwy=4i|K2qNbdZGAbcZR?GI8-BYK0BNg#S@Wy+e zr9#Ta@JM1x5`sd`CBC=e(wEXMA=I6^Oq7^WY^t$Jgzs{>bKc`k&kiU3K+^~@(uC-0 zLK46rzEBY2JArBG=4%CF`z!0cEfb(4pT`_t+C~W~y}3ZG*Pn7})?+T*$BH1yZ=yBP zf%fb5l2bu6JV zW|Rrpm$6hrD55F*z9dUg_N0XJmX_b0KHtZ$|Eh-?i1-38S znY=ZejyL&43u(w`ephLE@elpdA=>EOOgEKuR~S0@ofLRiNG-*S7n>M(@}ox1mHWc1 z9gTeTE{db3)e;-C9{NLjZ2y^c%tx7^T4g(zs^WYMpsKAx6QvL)A#Ok|B3cBWLQf zB@G0c`CR4^dEL>>raUU>aT!2b0Ubg2ss!jAVEBzv;p$hFllF7YrIH1}6d6Hb52mz- z88IVVBr2Z7!+-oP0SD3yXpE$&7~M+n&Z;%c7O)6lB!nC(@mF=3a->2bjMM3+)3gD)f4~e2!}mOcgo9pCK%V05ybh&i8R9+sf>Q zmVg=2CFcQbYQxbm7J71G?b&Dce<#c<&?lb4a;b$zx{rkjn}t;UH03QO_-DylMy7z0 zosHoquXLp4Eb_rsvlmiz3}gG$3<~(tg;vWc1>S3o4~)+P0?j_2&mZi_PjEP)Csnwd zS+HWeI%6_kx*~HmOGz6yYVCKEx})6HcUBzOjaeuO$@lJqpq7BukK#D{Wkuuc2=1#q zeQoOHi6?=dHhk^Mv2awTk&z85oGw>H2nPi~IUXEJEQ6JMx}6`%m<9zWn6dk)=0%H* z=uVgjV^@!F@`7q;$c}x%1TzH2_9#t{=zcc@W?Ur@dfHDkLzWjn#6UnE@lt6W{(+~! zb9<%K3-~==q+qYqaI4)vRW)rb_n+f1*1J!ddqw`S7Q2ZKb$9)p2}$=Aiax6`~#d8pWQ^ElC` zV48EDdg`^bwB=tLKPW8@LVniGH6>7RB~svaumcVN2i#WG4Di-LmrY0MXCus{s_j2f z6QZrHj51lMq(HnOPXY54?RPY=Kx+~BCq~5$I|ld^5|iX%F$GPsvSpz1Y(CDBBpSZH z$7MC}Ke|uO0x0$mXegxfZip$S37XifYOKZ77A{(>;a?)M$}}?d*&XtaMt}!T_!Xa^ zT3%z096MvF3i!m!ct_c{jRE(o$_bNznhggC*WCI?7|&S{#pYSSY8BF1$meGf8D;$u z?`&%gNr@Hu^Lb!gi)p^Sz8V~SjQ@=FikTvfszUCZSIdeLzRjzThu7Lkl(j+^M}R5l zih!^6NLQSG)yve!fD`+Vdb8%Zx^y^sw3c(NaB1=L3=^c8$rgcgwUsl<7@yvIy$`qq zHiOv+olJ=#6*h_jFN>MnU~qT$|37S#A2$hg%b8Yz9v3eT?7MyHUItYgHo0M5&(t&; zi1n#j{#9WLB7RBTa%f+BPq8AEC;<@oLRswX#sDwB%PF8N_0RfQ4|uCq6Ae&+8Q)0?2dtR1 z?IGVDD;lfzC7CF_@%!6ZEMy2sq+i5ZAEh9mYL**TthF81W|%5WTcgiH{tCd?paF?g zWwXlNGnKsruB-Rc17E*nSvtUtmndykX?yV^N2<7H&h~Nf_75_H|ETVj*mQA52vo7G z+#Ixh_ynA_oQ>!^V>uNrJcNwBwjgcO^O^wFn`)&J8dUa8*84?j^Ep?LZG)k%D+ITv z_55)cO1M$>{2Ax(p#Zy?P?~gq#4dq5Rz=NH6-q(s^qpa&G%Gx*?yDkTL(SEvR*Qp1 z-h)AAetguWm6(KnLNZTwv3S)^=e34a-DDM2^8!p*W*Im)va5}(yT^18EWhB;@*@d; zXh$PJ3EP6T41JF+rYgAWrppq;N`*7B@(44u@xGDO z{sFxxBcyv=|4_H20mdq+8%n#hui&n(LD8Fym_m*}03QJkVv==*IBHEauuFL@ z@f7`PWI&hU2#%c)kB4SiEu_&T`6#Fzu**_Zm6CCs#~f!5B#y-=htL>Lfgvk@Ba3OZVC zJh^>Nd%gnk)^IWotHDe=b>QxJb<6<9zu9Wc0zfu3Kf}i_*0|FNcF;hK#i@@A{^%)v zY3?w7Pd>VMqoYDl*eqt5%WHY&nvHNZ1YMRQ;X^cmK8C zh&()8uN@8PmIaaZfK}eVxJu(tVn`y_F;Ojf*jtxT98v;?RorV7T9q}$!*hnz)hl_A z_+_(u_O+*wTMZbJ0^j&J0AMWOgZ)lo`cU&wRgkca_T+T(YmJC9;D^%pa8%u$8o!b6T4=p29jV5(BDr69p$Y7vj}?% z!tA)^sOxKc<2IG(%2~-b)0v(qU|{4}rh(}4i1*;Clwi{j#GwX6MoH)Vzy`jeX#Kq$0Zj(1XaX7D-Uv7yz2$N}AxKFjZi^*Ecw5d3 zh9UXF07c%q!=NXgsZVAZQCCvx$UKGt*%;LT7+8UN$Vt&UoV(MO3q7_^EtbU8EYY5{ z`v|JeeiGr5AGyZ}IM0Prug%uLwFgZ$YMU#0jzG&+r$VG3zcypNsQu1xwUG_o>SKs5n12aXt%N}oLgAsS>`p1X4HrE@~y5S(Uu z?TU8+IUCY)(5WN{I^EY$UT*LmT;}qI5?`eX(JS+$Jy3s;;$#iXOtW0N@aoobFa;-Pb5Go zIJS5HwVYT0B^3%FT{lyIpD5(*=Rxio8{q&Nu>v%Z%`5|mm}8X^y@I-`TVqcOI)?*nT3 zB9>7P3Vcrq=4Vg|Mm3^&miiK~Hf)#aNjnC+t*FADeUhdFjvloIjT{1)5C2 z&3SibWHXJ#U%qdw0Zhc;Zc%1X6j?#^BXvGF2kkjDD;5YSc9>|`Zok5oerzk;9M*Y9 zY7+5` z;E%GKE47q$!d1867I<00=I0wXr4M8co~VNq#(}*t$dKp2?4#aKN1d?<;RULbg;FXs zq=r}mpOt`-tUPEi?^BWw&>XDoD{@Z?NQ(4{7r*C%K4IWX6!2e}3KxeZXl#LBd*eM# zTPbCIKKUyX)jJsZ1D8+L6M`U^6S!yUlt1%X5jc|%KpkMOTdA-8aIriaJWbC@9?b4K zutoqBTCnPSGxZwyuxf5Pa%{F`c#g!+kOqukM1m5peyAY~E64f@usPIf&4Iosqx^$U z9qg~g+8a-T)G!zrF%Hgx)`xLJJYRL&>RFA$SW@8yRYN<4feTGLAg46LOpmZ>QcV>K zOcl%?Lu7$fTm!E4Yb?mhKuDTjy}@Dd#1T}S7;^|L_NOv}GP=Ir2MS8T2>;tFZx44a zpRvFjP7yF9J?jLJCnA!7pY5y=Wz2jNVeTV578bu$y$fGD)J#icmH2Sx1gKu(?8Vge zT~w5ywD0z{e?mqSc=bEJ+DxE3q|SzQg7G~X;0b&<&t2`M{YBdWFfAV-$($8q8>dDR zZ;BVp+HNCGo0~ufDG?dq)%fUI05)W>aupBceUE_1u?oX?J5`L?+D7pI)?$4M{FPcC z>x(D?%;LLuD;?}=>e;ftFSecBO`wNlLCNn&OcB8oGsw!GGAzc49oo44^Jce1B6h@7(Qu15Mj6E06WYc415oWH zWW)4|AcGz!Hu38*wNmH}W2|K$A)s_@?;wyJc6y2*9Cc1>>a^w@3K3mwg0`#u87?D^#Xlp*l2`%T1q0hIw zm;BGAPrVrhYCphfpa5++a6*oc6HgDoH2{u*{e>)z@}HgLkbt86plSliIF5Mu~+UEEO2q{mm)JBzG^kGluO2PXcV4lfy1 z2mSYq?z01Dh8V>CcBPo(Y?)D$r(v>brE^q;N-}Md7gC};)L9YtKs=t__#nMP`vpQF zZD%j)ekZ6#jEiZ&GbC>#Emx)=PP3T{jOR8oq2|Xer}U?gAr$caLG9jHu5iRHpBZCs z%ln$pA(km))YPRRdJ_z`rQ;xY<2l=s++1{&tcV6Bwo6Hk0iTNlN9RG2l#`|hA;`6p z21LAZOZx!m&@LSc_kGh=h9)mw2l2rqi#lRmwX(74+TzzhaN-R{~ zcnZlz0dxLZSyT1vsR*-aShgwTB*XtcFKgt(sjwVDWsMwzCUNGtz7P<;ev$#t8MWd2 zPz~4sKR`22fLUpesaj5jgafJ0xb)qmDooe)`Buf>&9aRw5I;MxC9I+8x9=nV1J%1O zYEskDm=^3IUvD-%fv5xy6J9`EK(` zU=LiORZM!z$Ar3#d^0J4yk^aLgtV8ce~eLO>I(r8Zop(rKPwxgbE0^rj>Z6&?hmMZA0wLz zf-gu``Mg8?835<#cNM&{3Q7vem=8ZuDQo?GH}viXqT+AcUrtM1@XETPUA0w;CkIu5 z24<*JfL2}|oF{53Vg1a{eYVAn^jr#H@+{7s(8+;8M^-={Cu&9;UuNqbtFM^rDSTEc)U^cT6F$@0?>`h>!_zyIQL8oO$ z|0(0HRN2g$Iek&8EV(9?7m75QBD5`Gr?;h<jk zx3#`Bf=gCtlDy!Q8JvcZUX>CklN*&D$OfC(7VMAzTVcNB10j&nQCWbh73DMNhgGhP z+n)d*=idLB<~t zWnC5~YbNn}t9tM|jjFldAPLYHrwx=+hr2VYFtwhhSMX)&pDgM1+Uw={OjInV8yF$Y zJe*L;0((ZnN^aZ|YZC6)imm2&G^`ka-SJy{>cin#Ri8bWX)iFd*t-#balirAUjFsi zw_XDR^j+XbGk?)#^*)`Pya~D>FacNlsG1B4BE3u&wYx~AjKxs>kFCGF12)+x6e4_* z#RktSmngn-qNo#7J}3_<0n@=@nyaj(p+DVW*y+IO2=~5__@yu^vK~ybir=JuQiSY( zjpl1oP|6Gy@v;1|04)OmZeT_-JYF=0>megp!FI8L#iTrUS>{F|-vA`}KmV}OQM|1+ z-Ni=bL9v{r0&$e_+9Fj9o#hk|$p_xPZShFI|01xeZAcHItbbZxYCzWpgRZy@N*%Ug6-7PKXhZEM@wj7zeYfP z$}NcIo(b2$QU$>#aAJN=mEVrZi7DOy^h-yjZN_sJ0!I;M4xlx%u0&wQ@~POe!!e5Df8 z)h9@0x4s@@4)3RvFjb>z9(edfgb2{0jQfKp(A%o(533{`ByV@r>N8K~v-+fp&1=h8 zgKvareic%W5xgo_@qRJ19sMZ-c_fh_f*uaH|_+zs+uN8{m-uLj(nf(4iLbOVgt{efzqjM1E~x%J;gM^ zl>w;hcize}4jk6P@ekG!{m?Y{5>gV44z>8h=2HET($T5b9$AE=ATI{iW^93L!M2fdo z(*%G7gkIb_BDxA=ZEDL)e8;NinQtcmBxw*uM({Se`DjI%)AS&f1?cvHn3;PC!kKW$ z2uWZx;4)iXw!D%DI(yoj?_&h3Fd;G{z7Ii$Y7Y)9cgbfhvn$M?(8wRRWPXgWj-3d7 zaaKDMY<17BJIP<_sOvhc%4-vIMhX;fr=Y=ifx83UUXnCQb%I9XKHA4?aT~yCH>(=F zQk*Asc7X+VfX)t?T2&1w?S-8Q04dgws{&o!2xW8I;0yBodtcNman!0uiPqApXG!SI z5Mgnurzg=G!6)9mJ*4xWGxi~Xcry|@w+#oV0n#i?{f&{4X9**&_t34 zMuY8fR~Znx2Giq&k=Ao#^=7m6d#cWA0)+>}g%cPHB(`HJYeOtTnP33;{fZ=VG&>0Q zcBWTMB;7gKDB}^A<E63#H z4^V}Y$xwk(4i!U{#{-2aT7ZV8ODY}d59dJFy2#F6kWcIHbxW6=VdbLsf*?S&g_C79zc@3G zuEU-IKi8|T?gip{#16m?6a;wpXHnDVT6s)tUga<=N};&@82?Fx{8YIU$@R~3Gid~i zGC}2;B8$4L^qCEi6A;T35oQ$rUX>Z%aF&kyL|oLyvF0}YA1$!F5@%gRv%oJyn=KrX z9IOUkoy{3c(RO-GqMePZc7dfLE8$$t-J@P5t=Fk4g#4P@C71z!ybwFk>T= zI$6ktinGViK~3Q@#AnO!Dh^4Ij6p~YOEyp17Po4J zCocvafGR3An?gb;ivPfUT`vp@EUo+{^7x@tt>BC0TA=(N+x>x$#~b>2>t-$bv_=S z$1w797d)1wW^;ZFDm%Pw!VK@;CW{>m(RYBPJc!y329VY@+c1#8ApEI6tWl~NAx(${ zMbEFtH!CyK*PfQ}1CurC4`3vxiu~aIuj^tCgXqr>E-3K6(PFUq6E-xppS?ls%DJ+{ zvFS{)VvMQppVk7OT`?(3xr}7W>kony=YL(NqKmIm3*`Aewa8-uDzU3T{c$e*^!*l2 zqt%z}FA>WJj0}KAgD65Vj8>PHV`E{)U0)Yk`TC^a4XarirSGPuB2ve5AEX-GZmC_w=_gY<^lDs9M=A&~| zKlH#|TYgN{N0L-kzmxFc`_;pwuVLEwAQJzLJ77F0php;i9ZPVuaz;+{ z1}RZLdZ2CKMH7)SHTh})_6l9vIrt+j&U`*g$SvLKFI6P(?WFFjlg3iY zUsGiS@MohCK(i}nhd##4wZ#EPM?-aVtd`nRcIwHe05A6qD17$b20fXLV)k^+pD$X! zByjWoq4u=ksS&8hV5wP*xs6Bj-5dH-%^Eo^NVJXI>+3?DQS=2gSBk6R^;=5{M`1~m z7wm46SDu9k?~e{VpZt?mmJS`EXQOw8YUwBp0eNaGXbHCD5%7|${4e;S?H`FUO)oZD zng);4Zds>|@qo zW$ACFE@!w5-DOE6^6fM>d0HNFN7T$j6fwmQ$f#a6{G6Vs4XxnGOv<9&$Y6X}tnC`@ zX88W+fA!_8dDri-d5rZ?#Br-yXGcI+hWjj`EV=^p^-{yw2z7}ii8a$#eF$zIn|Ved=tL9I70&oWTKwHgUA^6 z*Y}h<9pE~5DmF5OHO8E}(Y1mPmFeW8H-hJb8vL00Lf-o5UCKQY7k+i=f=SsKZ*A!3 z2yl^G=&yU#l_!&YjEf%eob^)?V5OD}wmXyxh|lTyZB%3`wqj41ZeDvF_{>RxDpHG} z{>c_lHSVZb4Zp&EX?+jw2I!iLU6Y2!ia5tdG^aJ2`)m4vyXXd419e|OpQD1E??ts( z(12IYOFcC4jfPw4(cQIUUj3rjTxpn!C}Td$Ye8=duVT+*BL5JNLbr+y;9m9-+H84_ z-xZtt-`Iw?;M7;wF%GOc2bZMsMRJ5~BnK3>w7h5)85MTu&GnoC?NM z`GJnwilCiQ7kW)NY+|lx21EB4mCE_#B3+ga}Ot$7c&Z70? z@(P0%;Jipx-#o!Ay&AF$7}$fujN7V0brF6uEVkfQkK?Z9n}pr7yTt-sA`GAM=Lt91l5@|tMjT5SvopsN# z407rax;&Xw!QY0rJa^vSRPmHO^!k(CIVGxcc-st))gJ9Nu|Qoxi{g@rx3J zxsZleN+)A+!4}L+8Kz?qu{w@Ony0535<@>kyL*jgXgp4StKWGd?lU&>vvzsCMti}k-qovD*Jro`Pp=xHEoUC!;D4O7 zq{&Zsuu;4+iL1#&dGJz>-Rn}Azvsr4pif=;pIg#fI@f)WM%|_l)??^MJIP|C34L`c z=sgKA)XLxkTYeFkJ6^Qte9OgWKLWk}bj2Wb$8!taze#`OFn_?UnC79*ho7NM6Q4N0 z>UV0W$-7Be#(7Vr-SoDHqFNYpR4%XT^M~o9f~&06zL#-(Jl~`2@mNeQB@rq@L6pfh zf%>E?(LXU?_c*0fU0#iN&>>RQ<7c2H@M&@I(<_CRlW`()A=!Q;eym=Mqw&;XemR(!;}1+glQnxu%`x51ewnr-bHIyyR@*FF8!)~Qy+$WE+W~zG?u=cP`$l?>mN0CAYs|Cj za*&dEkaS{H#`I&nYnmt>QJ&|Rli(&(4lyOWHJYr9DmV7K7)beNgV3?+SFmrI%+$D4 z2mwTCc-0M4Y#X)hulnmA9;KG*z5Xqr_m;#VaZxe8vHGA; z)~n6b&RAbb@JLZ5UKWO`Jn?&xMjgDm>FE&r=m@=D(S z0*5(e>&dVDtTUB^YX7wfpN|()-F=gy_C;NQRV`1&oSdMr?1CB-SSil8``%F4@7jA~ zszBOXrP+=7SXg7LQ0Q{M>F*D)b64+}86$!x^|9dQoAXgNZqC@cKM$`;+gEzpT|U*M z@9EWjPEisT&F&YXvi^)5PZ1a%k-6nAgW_iPj`NgefAacz3>t+`5eX_VmvgjJt8#z8 z$j1oDvI0j|5R}f?tmKT2Mp^`$TZWP=KKVseY57-WC)D312|R8z!zT}aB>!PjoSo9G z8~!r#5MgrBrh`G?Oo;H+7?tteqvfc(h3f}FLF48xOP)M=axb4ml;eMHK>kwp%dp`| zy84B@eH;fg?^H>rp@-zW5lZmE2q{zl`LefP{3c4JjoeV%??pSN{Bk z*kTixq>*j$g>`IRMdgF3BHg>e1UOKrcb#f$96F`Mf7{a424pw@vsQkKzmOw&l>bw) zf>e6B=3+~-Z;gL|Kc@V0PXFCv`m;%uB8B$eAqV+et2IkmA;Q(2w9#tO5)aOf)ONI# z{jF3@8R4zuHQIUNvh1QucTTAwB7@<0%RRGW=kx;u7ZqP$^oMCPG@%PGxY!wzi<|K&|U-L&nc zC-6qx_6CBlLX#K_)`^z87DqO%@tCqtj;laC?)cKUx&vj<)EBL5tGoog|#?Qoku3=w7bzs zUrh^c2X(i`z7H{b@%o{*_;-rw9%8qSb00-|ysjQA0`ds}CiA74f60D|-~4%#(+H$U ztKKD#wSU^J3(O7Pu~3}d>_%i|x!_28^Hl;IbOJ@i8;{>D-|*=D(EZRBVLHrxeitJnKzQ-XF^wF%5_p`_H z>N^Yb#5j9fX>M$C+~x^0t)38Jb*U>X`^{Ntxj-e#ABuY|Ig?b(&y9W=g~{mp}{&mCqRZ3>`hi$zGzC`qOs+Fwo()ZjowAnso?($36vaW8Yh#J2adbXX4|xr! z1b(pny@zxKlawSy*v+&NQLM#1>+|poBHu=auf$7C zS$?j`P1s!GeHrgi+KAwlclvem)dP=Kh`D>%O8uL(&9t^0=cp}UKy3m3Xl!UumUW=e zt0w(TrlC1;A_v&BiCKAiXnwz{=nAUH=pg&l~eH4V>Exv+FoH|FW4(<|oI% zoX*<2TS-s$h7=a_vitpD@b2=P0Sfg1e3Q$vAEY&zjZ)HS38pSd+ui$PKf`<0P<=CT z=d`BH1-M*Zfnr{9rKhJ4@u8-{A0}s_`DLKpMg?!lD{-JtLo(ht{E;&|w`3D9lyi(n zjRl4X-m%&f1>I7`V9tA64H;J|d76Zvv(m<<+>nwJ zfZRa}fP3^n_t0woT?W+rHqAh(nyqTTb#%d@n=G&Y9R0OtxWv(Q)@r1;@%X+tDkX41 zcuM*9(fE8QFK0V%G>^rU_u3Vbw9df5JH4t$3TWiL_}4tP0RuG^0F699g?&n};;yX+ z0!uK@8v3y;qAJ71xEMpKxafCi=9ne@GTXaSaMZ#SpF*4Gx75)%G!uG5`7s!N!|pqq zC@#bE-dMh9w2~~9g|$Xx$wxQqFCLmIzM>W{k*w4sYlnfaWMBW7$%f6!>oFB#?Y~@g z?JL=Tf*GkNK7H-;pjW6>68v1(pEk+4sluw^{L|ZK(3-}@&ZK-}l$?#3tigNM>01F# zERaq?TT$5K>r%-iy~;boacueE(p%w6 zF5Q#NzUSI<DcOk4)ileu{s$`p}!+mll{GAXiRVdY7fyGQcs@slh^wm zNe{A2oJ9MmCdQRU&1Y)ld~YswRpc<}Ri1c2_tdsYFNjbm*6=!GSpnA1k$2}cxkbgc z^;=HaL+fC|#tp&@PPw*V0=l<3w=qR^U zi?{pUHlM%I(mfOEZFnwTgX?HS6{q1m~s%?(4U?tAagb`Zwc5H5~ODr>O!USB- z=MhuGJ5Fz2epXwPx7Zhu*+;?hIXm4f9fM zif?-E0TklLPfgx?Db_OO&Dox@XUh58UP`<69^A;enR{hb_zppi!>plCMUxI4<%suH ze9~P4gSWm+7V~V;<@#%j*u;!j9}%Cscybshblm^0`nD=Ruaul7&LuE&qXjqSC6-NC z{WSB&`PBT6;fsJZZ*fo+Wo#GMF0wgS@Q!HT)Ktw6s4~Td8K4Tds!gE)s`lT4cwaY} z*-CFY`)8chpYJ-t!MD7|$r0SkzO*++bZ=!lRxK-V{JQ$aT8@vN2H?23QR64gf9e?D z?z;W}Ya$!mVjNxbj5C(slsIlD(|So+UqT$*QXYusL-WtxDN7F_W9`#&qS z-+oK!K0PJ_0v3<$NNsE)1y3yrSwcXLma$Ur*?*`V`0*1PlpnJAQ;a9n;FM(D#Qt-9 z3Am`hSi(ntW1DH_AS2&=l4xI4ZdhT7R@t8VJ7X?y$7MSCDk67-?*6)QqnhZ(74NZ% zE-RS^!9|}t^JmSo33CxGtvN;Fr`NtfvQ2nIerik=4c>OnH*(i#n^LU}IC?icS<$tY zL99F@^j=}rw`$hxxhy|gGSW;N#OleniqS9 z#t5jfq91YM2RTT?8a;h`IEaUjFr{j3gT)#DbSDGrx6JayeKQTz>NVaT);#gckkG2O0CX7htvk~R6wkIFf5i@s z0cz3Gskqi3SO1AQ#y2m(a=M!AnI0zcu(I^Q1k}ZaKVJvB>H(M%gr%yTRPCQkjKoe# zX~k|%4m&rrHP9~H`8~z@);4(%`{%R7xM?%8;lXPe_wallACi3Ny9TnOnQgMutM~n4 zg}*FZ9&p_%_Ksw~v^V0R36`?gg3VZErBjY5%q+NUNlF#SpaK@J-L7QgPu<*7)IgjoS#t-JM zqg!L`TKIl9$?~fd8b|{p$ToO3LScBgcz~Il(ZxL9hJ|iiej+(s15WH>n50vH$+|i^ z7!1DUNj^l>bc_&N{CA(KfJ^W`Y29jIxP@0U0B+P3J+dO6mI4kM=o6Q>5cFyp=CbrQ z3_i&9aemE9k_%WN&wsi5+&(=_6MDMkp)TEWSa(ysekraZqPI;x!~-A(kK{5WS{5k| zNz`@+@<(g&R3etWww^=#@BXPT7x)BIx>5J^LEYWIFlSSAC2j7+iL3)efBtYs<8haA zr9Q!d*wA4Xu~)5`riX=pm#b0f1^xcsqru}6yIY0>M&SwJ?sw1V7i751n*R~5!sMtD zcx;m<=k)(Pmj0Z?C=un`J21Y;NB2K$NLpMe)ef5NGUnsZY05P-csE@yW=BWxmwo(Q zS1WO;cLp}xF=NZ#^4{THhYOGYT4j+N^Y6l}!~T+tH=NHEDE0HN@ zW`3~OmayGXXz(z_0DoI5>zc~pkwy;Df7X8<2!EKUY$7xExe@wm^fRAQlbeP7^Bal5 z4y;t=`SpdfY)U`X`tKF{2aocf4IyjslLd7m2nN2fOrA6{WnB03e>l~UbIS(baFQpj z^PI`vn|88}9*7jLnre~Sx-X|99fC(s&$@k-N2zRSKB*!5XQ~%$zDJb4TFI&!N&Gt zZ=vn?6Mg56)wHp>Od_`43#LNSTnhCvt7sKh0TpGReed z!AH@{MaDy8;a=rVV$RRgNhRNmbBUK^dE5TfH~*=e5+i>e>J7fBy2Dv4*3)f+s^^a$ z+3z=y6Y7*AYC?CSW1dPL{dU_Rl_dwjjBNp@WSVMGQ2#t2hmpD8v3u&dlhkzE=ahSw zRD6_tU?}N`bu35E|HTouIUC}2AUc}{u{4|X6g=dR@ z+;GRHwG?zL^51rMcL#a0>HT3@jm{+$hv^X;~+mK6=}swrJ%*S`m- zc_m`iDV*oBCpZ6z;*E&#@ZkPv#U5Q2vsO&GW2lNZzPNIA=rhnm$*mg>z94@jKS<+7 zLboJJ?9Bhjea)@2lh;);Q%b`JI%Qk;^Ay<^+}t(nwd+yG<%ayR4>I~XGu_Nb3(2}n zb*FKr+(KTG-8Gk6N-Dn_tTe}Pb5+k}UrQe<)6LkmIrwX$%4O){v+^TJ_{f zo_1Xuwc7hpb|Ll|x#3bvF-&I0ds^RD zb5Awy=VKSo4UhL}`2v<=#^f};MFZo}H#F1+L6FEx!nkehTHY~T25DU-&EPM`vpuT$ zQN{{v(HxIMU!(ZQO9M#puzqUr!`Abw-nW8&^zM4t?%WEx<5@v`XlQDS zzNr7cvP3>FHTq_uFA}wsq`*`dr995$bR%>?*=4$4fxJi^y0n$XF*?ts_6kC90M z1uuWA`+@#Fe2N`KXY5J~4t^rLVUOQNS#itDn%Lv05@gmG*1$J~%$77*t+Pm3#=)Iw zRJn-{fvJmP+f*j6dBfs%OpA0bp%uN~`x@%cI2L=VGu#o8&9yW9?z%-9WRgTV(gjRw zgN_zcgZiH#n`CP~IDhs=NtXI$Rsvz4!=T62~E>4OQtiz@GRf zJsu#!K__=wGgti`^Beo-U}yQetm_UVUbuyYFB_3Uxl1z<;pJv@xQgY6YsR=GJWpt1 zTx+mZ7_U(U|9`{vlCV{@F0pGq94>Wnl)^bQ1hiX%peAgd2Yl!`i?xsFd1j3FuB#~5 zoztfqrDysT{2``1X$JVW6a*;KG&U*;l-jGo#&w_=Z(>5y#^ryR2Tnn z)p7X21N2OicqZp~L~icEPLA|THC6Rnz+oV$neZqD2OsGFhL{$Zt^e6S_I8m@@&c{= zZRT9h%}<{MRl!BWXRdgAT(7G4_ode<=id`YC0`0d7f{4Z&=ds0KXIsvIq}+xSyn2b$~mZs)`&KgMy=fyoJIbfTe$eoeEBi?K}b_xBYlkFUa0;b2-xMe)C^ z6I5v{2h!jfdR5|&id{252)X2AuYqi73i;)HK4@PL^*euGPFP23$1~?Cs9XhL@p9U$ zk&k%gZRP_l|G4#5+w={d3aTLC-6j2$(GB?Ngce#uiT;$CiMi1!n?|>-!LB{!y>Jh& zJwxPblkjUU2t!ABUSAHO zR?m8~K2$Kw)hp(q<=A)yJPmk2r%0vkK!p1w^coAtwy&9eJbu-R!4t{gyJEoy)WLb1mg2G3Kq) z<9)UJLieu^)!!R#5T&+ZP?aQ;hBojAA-s3sCb{>Y`YT$VFLbF>*jD-ggAYcm^Ir8& z&p$zwUNPu>J6_XH9l7`8-~i_-M!CI`gvAKe;pcX>&Rz8zc#xe@v$cABen*xcdV_-! zT+dgN9IkB72enlMBRfv{GCBH6sEXWHVISutYdIq zaLmj%eD(Y3J4}B22N}>Iz)4*p1g!8?h?^WN=KihPvvL2a-LrU`EKXAF&w0IgJ+VaL zsnyW7I1bJ;>KAh|Iaz63PrSj{-iz9?ALN&gf6)oa=N z=W~}5QIRIW>%LldkaMa{r4HPxSJ@uFUw-aCs%HDrt6v88U17r}K_e%^{i@yls)cXZ zf{;1XH_*D3P0e%%m>F++_Wua9bZvJqs^b!EsAd}iC7??KkGn-+rshuhQaqwg2;AJO zNQ8N6aj_`N_}w?VyCNl8EP8*`p15K&;_kWmZjz2|@0v-Gs&ExQM^1GIOYU2k(oJE; zlJu~Q-(>cG6YKCNp$bB6Ttd%y{Ya}?54a1nb!Aj@_QaL?kF&?ZR9fujD#bw4kc+z{|4aRmQY3OL7xJSLkdX_tsHbfqK;_eGnH@MGlTk!acTXE||DE~vpv{WbM zf#|n?87Zt902dH)X_fQrq26o*kkKH=UguiBV zp7-nh*$ry#@eT4CZ#%u}n3!*WjM(PKgTIVOWjWX7QRj_$7@`@>y9ckgM(?L z_589A67P0zsG3M|dBHjwT^0C6`n-B{i<#bAjC%PE-V}wCHs48q{R@OgV0?xP2@o2U zNE>8%5#uoSPF6_0qkfo1MB>L>4R|fcOC$wUD(!E`3?(77Yvf8lr;N-|9!si`UqJ_Hw)fs;uV581ToopvCWngD^C6S&?YDoWi)bEuKEo@X^zaM)|m zNz=urL1r%0Du_3b>$+*Ljzj~eCS>?}Km*D+Vc;n`w{Cu7qUVN4VYhBgGh{pAkQ)Yc z3mNP_zBhaIUTw)n(B^5xikI`iGR{0XM|M zOl|ige?W3yih>qyT^=gRs`-`_?RqoVySS@7&xP(i@32^!&{o6er@F>xPcSM?zLZWM z{O*eTf!nwNY_UW%aIyK`&`a;>m&@viCyxud<+tQZvjUwt^SgvL`lI!K@`%8%%&|@U zP_i?-ASv&MzWrK4CIXJqw}ferT`@ny?Vh4X@jOgwev#+qKzCAX@U~7tqvU=uWsvc` zz_($Kb6X>xJ-A#J!>XTTGk4d?jb8sm2lR=S?w_Td{pr8qa#5{8KVpFPN|I`*zONkF zU8f3df_w+Mg^GP+U4FRY*3}MOJvowX=B_86no#k(e_G5Y`D|nPpicj4&8KBP_ud<2 zuV60@5aE%LP4t6NxN&5*$qmXN{zzrVsmphm%j1^&Y@I`E?jQi1?Jq;o;C3*lY%sS+ zBgdX~^(utT*q`qF&f1oq%!^^ue6|0_)tiSy`S$PQW0}b^WZ#ov>^no0WY1u%GqPtH zLX|2PjMfTkgS+Yfuon%kaX5Uh_BB{RD^gPe!cO1Vt-haH0$$ig#UH5gpmh*bO z&J(xIa_zA{pgghCyHubJ8h{Pr8LwT>Y*p=QJgu35XQ*JVAfgU)J6>E%(ZW}kcZj=9 zzZC;bgCy{0m=>mNQ#lnk;`#!otvP7u68SD~6N zy8gS5D#R2tKlZBKVzyjLfJ^MlAkm5qnUA}O5rY0I@TrHR905W4DQ7~_GI)1qy{8PZ zh-A(vSnoP4-#+B^^rpj_y+7idSr8(3crncm`ZFu5LTuTSXl}&6GG_eou(_FBVx7;s z1yxUgx4E+iatHl&&87ceri0OuK_+OZ5Rd|_L-19Tfh7G17II~_jZ8q`H;K{XK&dl$ zT{d=Ku@TF}clcJ2Spt0MvCD*yG91Y0GZf|jFn z#G#CFAAQfxTf8TmRH0aQURhAiPZ!#Z?Yw$E(FLsN-ox{!Fqamr)na3@mW#|=893)S zUz7o!Tj8f3vq^Ft)l5t_MWND-s9DCx-19dMS?SS)=TI8!lG~FfPvj|49Mv9f8y)6$ z&)~D^XS)8>bW)btr(TiFziYy>x_!?3o8k$7HT1cx%X`bI-tN~migajxxeI`wEz|wQ zV?cfY+TpBnl~T(0*alhCdfaE&^+sx94Bo3y$2I^JP@ryTA<>9NOltqcwmU61zeiHT z+}91hr{UE(a<oEwU=(#goBQdFNx7+MGma_GOQK-YOg$mrGA~1a4>BA@!0ur2r!A?0-S>5SaTw zmFHBj0^QGO)S>B_m!~iuGq|u3htz`-*WmF9q~`e)7eO>3fo9D%&&cTq{3Z z9P7$(T942vETVaV{MAWUmnd(U)3V?NuisofZ+2w!(X_n*ZB-llCOMR)_7m29D~(z= zjQd?fo?`xa2yaZ~ohZ&uW{>U{HNXH|;@d44qH#x|3Vb7IZ{bqJ_ z`5@tz$(Wt!e-XRkw?!UR$zo%v)Jy9LA*Mn-vbuS#-7!&kFAe@Fsrj$ja@|LvUt86J zXnBIMPw8Jx+>#qbY+iBT7BL&oxIf!{^XYq`|&5J&`nkFysrgRa0#-QFr;=SWjG3iN=#pyhjdm^@saFdp+ z?6zPj=R0~#MqtXa#hiZ-U?llh2n|v~ zgi~<}%vz8B{Kz#XYMXEIi@CZvb}m>tH2px6$;-6gOk1tph(A+LvRX!E++c3MpD0!l z+2UV`_S9sITNibUeJvP3FT4+|dvh{TBIh3A!&?Tp>j9&_$ei-B1E^6-aS~-+L2X}% z^wp0I7>qR6c{ga1a>5&H^)==PE4=o!Jo?tOC)vNwOG(VRv%OV~D-v_PID>mWk1M*}&CG_c2VSDt0X zsXu+3KlU6N<~dFGzh@a>8Vlrz9k%jRkgI?QUl@H?ZE)=x`!lY#F?nj3?abGVU#S3@ z5R-{R47)(_{G$}I?|EU>Y+HdGc%S2MZ73qt=hu7E(OrRH#N{xJg{@ncrplv zR9Kkciyn8M*`Ut7Ut!3FQN8%c&)Hq4L!3?tB~l8U3yl;~QqXnR1109l$~cctuf(lqdxQD*7XbcO za!o&YO@g-Ff2udt?b$%Rzj;C)+<{}Ogon=GuGC3AE-IA0dAuDjcK>pd)dl!S`r7#t z0R{5f%TG)JwloVe4X~C>&^sc$OeSym(KV^FFLVL)4D?1oJsb_}CzTs8T^UKYBX(To zJ8NrdZjy4-!IX1A9+<9c)azXC>o({LLj_N`@X<@42@Y74%NqFIx`k@oa}j2?s)dJn z-qO2ooM{9BqkwTrgi6ebO8&7&Mr`&Ms!38X$GGC~**EiSi6ysKMn;+XLhxP11@b85 zy_mK81dy+rN+vx_MMmB}wtBZy7upI1oCYAmWES{kk`~m_1U2?Y0cI{!Du5`0Ak&5h z9pLZWq*?wPy3eFr6(aWCx~LQS8djHkV?Hpg0_Q?L6e@VdTwezJcnzi7lM~HC zG5(Wkz)gGT)ydp>Cf>Il5dp%!jw_elg6L9LG$@(VY-Tb>1C+Y`$^KJhiTOR)I}}L6 zltZTBcGWZrI%1nOoza@=)>3pWK(Zk}lx5>4=vJ#*WU={*pTHg6u#Pug zZaR{n0odlR!R{RwYx98VG8bb7estNq@V$!yR0@mg1bGMEWQ{HHVPddcJ=<30#pZJ% zT~%qSE|d^gXOdSu6=}>Gcp_Bdj`20WRfV_|-s_>X6{}aecKu+C4ZnlsXY~FFJh=^} zs2L}^Ii)vtBY z1m(6`QbZ^!qDu*f&CzO2bF}}hL?WgLK9I@pUWsB5Y0hV2byu*q2USJH$9xNncq^%) zYd>U$_N&JTbez+cHOPa-B>SN@b^zM!->0y?{KpYjtZ}vJbzO~o7`(<6m2N@^(X8^_ zuNg^y5~{6%le-hXRtJaJ`;m}`nP5dE|M#H2qJ&NhbJ{HDG{e*}whd1q!vtRr!UzMj z#qpp&5>I=BdR<1hN#KgX$Itoct*k_MU^s{si6f6W2$DoGtgZ}TlA6_@Habi3s?L=BYx=b~rP>gb!7c6D`a4*kAI{)Om z{P>g6{mdUXE9S|c3V>xazxz-~1c_di6MfW?ot0jcz_`A!F=+&+<}cM^o~-OXdS&L$SB;O~jGK04DhR*Ex?QffV~of7@Gv;=&d( z6I*nT^Sex+30`Tln7x%MbT^*|rU182TUko8kcYqRHNGj|LnS0270Y;1KVcJyJL!mGk@2M+qDsgq)jUST$p(;Zp%Rm7C;D*k)s*zc-t zFIxwg(ND`t)4W@wL8aq_4OqQgpg!Lkh&Q^Od0$4yTy$Wr3TU`jkqj!f5^T;pGaAK! zN3iq8N~-ct%C->qRE=q#guWo!nXCvO)-CJVpK*R4GqSf!V_yoS$@VzYws#_=c5a*6 z9FnDcA!xhN#*ml$)wu8U9xRlI?XW6i&K=^psDHCzydJc_di!LCBPZFN(d@xB1!t9PFgXp(>~KoZKz2>0fh<_Z4ewzKIzA?Eakniz|$sc39h>(Rw> zCU_c4O2(8;b@i|%6>4mbs;C3nz-G0lW};2)e^T){hg8`+86x3Bu2DCK#$v7Vgcy9A z+Arp+8yzGCh1rQQoXUnJPb<|~Ez~6p`Gl1}k$uLM^L)nRXb6+eOYQbiVf5vwg;NgW ztuX6B*JaF5+no$5Q7l~somKMWiOOlE!4~Zcj22x9u`)S}`}?~8P-o^=j4-?xgdlBj zu581#`XS#0YgxH{N4L5^JTf77(^Ruu>RA9!7ilxD6wew3tx#qe`LgS!hCn9Rh88nH^?O~ z`<1hI)@&YkNKSDX613L3@uutLSl5+I{g%MBlJslWT?}V43II7spnY|bn9=^}n^nP9 z^HOrAWG;Lex$Q5%+_Z&6{aP_^{yj_NUAcYt)_SWUwM)eLZRmgNdVcO1%3XS^;`t_# z4rYrN_1eLX;Huu;j_{;ngHPEE!6S7{Kx_Ax2EGLeT@hsUl{L4Ig++5zJGxr8-Q+4^ zArJR?VVG$UGR)c334`Qef%p{=J8k?@m5la_CnjNt9Y1mXcDPR(16`?mGApfyG_fv0 z9K=LTxu?#uQYrD%K8CQ1U-6YwR2N2N+861b`^=B5+?4d)%pnivo(X@z5`+*TN|v(8 zSzo(Z-?*Q_N-NU&n3x{%TEx8A=kI}K#-0rRy|*Wk6DzeSd=KFW^!dfu%h*VjLvi4fyNg`IeEgfp^Grn69<8t#$WaGKgui}Dm~4!JxhDB!f{jMA)#%~Wwm z@1&L`MRUKR=Yg2A5%s)BZ}53qt%+;#4Enk64K(Jijw9)$@7J!=P!Fjv7ku6{&^rGD zl!dTVm~R@@ywK*H?-sr>u-t>2+&6@Y`XBq5{@#_rX1>{p$*|zk06a(k!nR3&Th+A< zG$J29X4OC@uwy7^JTy^e6QhC`#}82=hk8I?{BUnKCoSAkMdf5DszQ#+TI@{wYZb(7 zWJvV-)rm7fNo9g%ZdMs0sRH?< zB~+k^$2od?zMLI5xgjO`=io@N4oS=5qXRVXqDBCB-GK@dyx-9z;fwX=VAWRVQj?)o zV514lx-h0WH`#N+s+?J8oVA7UHD-+%^GZnnX{zkg;Y0X(px#&y)iCbCj*5PslahDF@=reGa2g|f_!g9X6>y@da?H3= zMu1>aQuyDn2EZT#rCdCJ^((`#zWLj(FiwGDpS3^T0a*Si#Kz-SI%C{#IK5wrr|-#l zMWa8uk6$8_yQ{_(WP`zNMZj^x=)2c)jq!VCfrKQ> zajk^|)B_{=kmI%uPs9!Zapv=?VFh)|y4+LgM>TA6uNi_fm-i%YTFT%-i>0NwDQX}S z*NKld=*@%gcV-S#gf!@;UtaE9lK+Ild^*}}PCH4@hcji;zDi#&6HJ|F`EqSVYu$i~ z{{_Lkm|661F4{hR;YysksJ3uL3LrUGlmFk6WU`%0*Lom+zovyz#a;Z&C1u-UzIXC_ zZ7z>JSF}w&w}EQwdJFD}8x}c3R%$s37jIf$m&YGc%)iI?G-V1bSIUE!FJ|fw%x`hUbDh1IvAFD`u-%gYF9${0d!tS>5(ep82G-YeTbL7#}*RC8afwcUFQ2 z+_x8IZU*SbHvnv~0L3r5_dyVKfjh<+%>JRoyvu(SlCp z8|L=$QprKm$=OTS2-F{MUVW;z>ri5=d!}k${C~bLSTQa_OXJrjCH12xa#Ee3D}#?K zY&P-qwX`04=t#y}rzvA0VOS-7&GRyHv(fj}A&EzZW-G0SlE$xZ4%ltJ*m)T>eJJjo z`#y&qj=ybEn?}ScoC*tSvEbrqe1EsQTQ=ZVhK<=;4)9++1}-7vW`o+Y-M`^GZaTlG z#q7!Gj%M1)+o~&f#_PMHbTesS$_xdmu888$NK$CKGlaYmmz(1!ZMsXtPo3kxF~Wp&%Cp`gZ$8zv#sRXT8Tx; z)d)LFf$LfI)V-<6ogs(uhmt5CtVKbV+!wKWHqcm!3$l8{2=5pY1zJ$krQA>4g35d) z&fgbQ5dO9CmNNSWtqD#B|HEi)e>gg9FUT1ekofr*q>zI%^AR>`Msxh!pOlIO%xrSNHWQ7y! z0*&>&QS>#?+}=fWXLQ-U2z2M`LuQ;W7}}C7Rx;T_U+b{l*>dWbcG{3W$i2F{bN1y! zWl#d#NLLm%@$TUPW}wri%D4CLJL| z_ev_MHB6BJSon(Pm(F@szwWR9$3{-ZJ!@ibR{FXGyUJ%nKsI}EcJU3q7W~Zu7ZV@ zFR|Fn7%d2vdN!kwXseGco|jU3EqScCG+(~Zz5NdyUmGk52pd3+FW@+vq@7&XJTV2v z?JdC`QzoN325rj4J^?qJ?0;F%`-~1a@7F&-^*NpIbYR2N$~T}&S|nMPf7 zEB~CfeE2i{>MR5NX7XcO&ASV{5(3knXepFL(wA-5qst4l@dd?_MZkFdZe_yhx3*~ckogiPm{`*#K8O>Sl~F?wAerF>;D5jQ-tvS)8(JlkvRe-;Jn zl0%|f$TwcZA)G4LJik`m@{L_AB{mdf)9`py5S@SF4GBbHqd!_Bg);s^!~&C`9<_C* zyEVi$8N7Mv<5>;D;m1=1sUnwD8!_oGi54?34fD`4`Be?osNu<4Lz%VvcYM4<(D`U_ zP2p7}*#gbeX7$I)d97m+IGwsm>%xW1t5z;Ni<%&={u4LoSFZg~cIm16*XlpDM&~4Z z>QzzejmvDhmC3Q?=C_uK^FGu z#?+O|SrS>gTxa_fq;DyI?b5okY1?;iANt3x@2b#IS~4@vK|m7qo&=N%Zo-x*?UEy+BW;8?GI87Oju;^ixl3{f3$ZdQ9Acn zoWW?!vpBd&sio4LKKTK@N#gzsGNxzhfld8)?jNUTzI&9B@%BmNC>vh! zyb#Mn2nQ`~ir)LcH2(ePXbn|Ph~+TC!m&A(Lwrr-q>9ABxTRNCnZ@o{I8584t)S86TB75IB zKa2HG-pYT@sFfNdj8P=qXVd%UP}Qip_Kj2MP|>(Nvg|j)G16(zlX=cf=VpjQt?d*q z><`WT(S&dOysh!J7em@Jp2$lro4@;{&noh@gCZ75rpAQ;3mHuj`+3xYN%~89b;PFd z&Ml6qkDW@ZN=-&wl0$+`$jsa0eP`Xn_FUC*vu!0-OPkN67@CA9`<<>>j+~a&b6hDm zdr{mGp@>-u)!K0-bkyPcUO)lJ=NR zU@1RS{wUTVo?;~Q6Z?Ic*No~`rj8|~0ljP_51)o0)WJ%kqUVeSh7q0%w zZ+I|q*Y{c5!yfj)m(6osLC!x4z&`tw5%~S0)N6-fcUt5RnfMp<&(Ek)6BpgGVnb~I~@(bM19s}2|ISv8hUJ< zcJElg9unTH2Bvz699E4_?3EBKFqzjNldE1qjd-Y-hN+`&?V_OteaH{E_rsP;)fSFU zeQpy=<5)y8;Ek2p%a)$zv_m37F^)f>&sDz?eTCv_3wU_0P@}GqrRSNj!8&!2X=kaKJES`xqI$=!sm}N-*OA+6nN93e^M{! z6U)E+)};)0ry$eR5qGAKLOM?-gIyKh(zy^R_1hcDypmkpH>g8C9Ff6;Iyz5)-l{icUwQ?(tP#ksq@QiOU~n~MbV(tF4g=$rkF2Gu(H8n zRp$l*B@#4OXY0qi7G!5)lE#6IpZc(JS*T^1UvB4(WaX(v-Xu>!TdDk zM&Ahhuqh)Paf>`k$s}_S=1p6kx8gj*S#WHXko|Gf#!@2ns@^XF0db-}SPwok%Kk%7N2?P&xO!x6K0MQ3?}_jWRd zzm?Olv&SO-wr+30tdJkeu{)Es$;eLraXY|#x zi36(2{n__Bgj}P$lwKFuk%hU~ESAoUU_!>@Zw7t>Jh6u@#jl7yG}RiRqQ%Yf`VV^< zj_iE&*4C4Mxn&z*xLen^gukYz24@OVlTkp*$jD=pghu6@fbLae8x@7EOp@||x$q?pziSV#ZfCvf zUWQHIkUxMwDC|2xXxgQk#Z!OZ7vYHKe%7CTd7Sy`jRaq-CBqc?MeTT6jy|e%pYMtN zfBI78_HGOxyqhA2{n7HiEiCJzdn)?;e&3>J&aRxf*y2-(l!>ViB9N*C1{pkR2V`9RHY0ZpgTwEIWNb|$fU%AFMo-TH31AOLjBcSnKF5?P*ZKT;} zMlk(xKJL8s`#zY_!$pSHkNS5`X0UtCAgzwH11*^m*(f%(ZJ;=4GS z+7;Ie)V>@LzQ?4eRi5XG{jy_cubv%=L^rF+>usJmJF;Z>4?YZhoF67s;Candm_l2K zfNWR49MK5pb9fo^>Vqnj+3?t6xL6Ne&pG&beC0sJNF<|2;LJyAAwy;(|EV=$gZ1P` zyS72M!+mz6Oe|)yHR5=VTGFEFtFX5pmNABjl2Paf()PzNRDaa>{#HW_OqNr#35*RV zgG`YFvykpQe{}x^yPFK*cr|~ZCG+f+YQXjQRU+$ldh($5DLyKok6FOnjl!od{Y6=& zUxIMRVRzYFl>R+_Z-g6_$%qe>*9XdURxiX^#46RyV4W5HGRfhOKKmyMiGKZTozqrL zMwh8OwE9#+>3^z)p5N0u%VF`drpNR}w#9G}|E)(`7&zJI=cA$!9a4YwITjW@cI zK_DaX_l5w}K1$%h%j@5H1}PHKAo;<&9A&0vfah54X&UGlpRXc=5IIO|9nIv_-=IDv zA?W4-IAgI)i_EHuA1qDql)&b&k!%hKuReB#gF6?UFgIc+$^V4|q_`FwvIWkz94gR2 zAVl)escBs#$?Kv>zlOKmSFvofEE!-cfkDEz5~bk%tbN_2IE|JRr)4=0BtGL<61rRp z~9xN&iMj*Re!lA_6D z6p(Pe{!1k0T^5|l1ZY%r3kTk^nz$9j`vU!mji&?(ew?_H^z?1g4P1Y#BG}%l!E(7J zRgE@0GF3#Ph@Mvi^(KIgiSiA0+{st}MUwxX5QqE3ewX7o6{!X2+h#3y5!ZPnNf#Wy zfrX>GuNvv-gN86#G723*G6>f9W#%qad^uPgmgHeV;NmMP-Ypi5$vT4N&)gu!%!{SM{{|rS0)3rR#lpS{ zjLH}T%`CX@1lw{j{J;066w>tWrgEtm5OcKus$(pMaMhsGWUm%0jfQzz6>U=q4`ZD6+;_hV z%>R8qbKlz}&_p71zKylOGV_jT5nuu2<4%au|7mlhfhwxR_`Ll8Euh?r%)H*B!HXPB zK7y05cp!JynD&q@A@kJ(B#`;GH^$+ncOTP%mr8n=?T&4>GjR9t4))_RFx!{M_49_q zpMaaEC-kQFJ<|Br(8o*!A2$y0{QuU0TdvS(W1!izxn9p_5^FTnlJi@OZZtq`O6Z5C*STMFWJ1Z!79u$<2jJW4-m%&(Q@zk|$j z-s`+hz&qHB|K&Hq;=ZdqD1lnpKO6wlPj@_I>ODpNUC2yKWl;*+f02w&64<`xK2)`L zc7EU#d>m+JmQY@uA5hG1(0`9 zqK{!x!FHp-AZqynBqoe(=YkSxwR?2IE|dpbN%EGisSHm7G`lqLEQL`y7cB?ga-a$c z35O4^OYVM+u?T8;%L8j~nuKFb$~`opw=)S<~)itm7AX}v>AN{bAFR|O$IE~vX` zng{lAxceFVZVx@gErJy5=J%@AxH;yP3;`lYF&yqY5YXiz?Ly1a6;LZ|g2kADa|hf> zYGu&4+zb1RuRqAuFwxhCfwILAz50s=2}An{ zRQ^O2^5byo*8y@hx)^W~kl-0*dO@RoT@nDx&#(G(U~x}9xi16m%=yR%dP^T@S%fJM z3UGEkbmzbP+rW(pGVgHaU}a+GX5F?Ti5H1mZ(EW$eMlE%FO{iM<_;7QNfNRH@-_V0 zOpr28@>q|Tv?>P$H@ED(fZ93WHe=E5?^6Ox>t)UMZkSV=#TU;1-JIG87@1t+yTp>M zj`d%vF|>(}&}AE0AQs3+OF0=CD62dD-J+z-V#@=U?wr$txpSASC>9u8=V)nG?Kyv- zcwX3ES4wPo7=HGXIG8f^jf+`Mdl7)E*3e%$iXH})FCgPf21|wc#IBgv9mpL9bpRX) zs16ZDHrip3A-N?Bv)n_2d^k2Pq7hk5OgtL^(G{SC&m$imN&v&EM63W)3#g(NQar6$ zbSH$4?ZEWXc>8k%>fp4CLF$Lg9FSOyItQ78l4&X~0^-(%cSn~$9SNdKzCuk^qR6gG zLLfQGs#QVXWLHhfzsA_VmN%+NBza@7WE81`IUFqsK({?pU5JiT6PV^k{o%ws8tH`! zB25fkV#$!4>RU3td@zKZ&l(v^9mH?H2z_7(ZcI_c5jp9vQJ_XUyjIP2uu)<#7TF(G zC5CY_#zB8JlJRws#;uARK48J&e&)Jj>9&5c9D$BDBZ{o45AG0%ZsZXYvn&OW&|T-9F@`Yv$>o~jk@B$j;c<@wK; zsR7|~V#)DyomLj@hFX!;>CczvlKP(~a*)gkTp&zBpGo?Oz!_%_tnFM>7*yN9HSMDd zg<3LcIquS$__-y@3XKxhf1+elj%>{M5UMn?fT=w;{+Am432AbGRY2KJqqGxNt@=qG z^%8i>XwZ2~)ElkamRq}TNg#w(^8x2ML^)g%vt)vH%tHQ&kWNv+IE|7O34x>-bKIvC zW|rkVeh~PH++RB0;h!7Br?eTlWTi?!nzIA(=q@MNk6>Gk?%p&}fl)oz=Wu#;1{+hPp+3>RHZqF6wZ2V8yexc~{_f_-w(_ z;B0WR8mb?;%0KB}y~L%~_+`=nCInB`z5=epF%7btG#3%Y{?^iW@F2aL2nb4&8@mW# z5mgjC0cZ|Ld|RU@whyCthy4X{&NIE?-?%|Aw3OdHbJpPeOa`NM+gCF=KL*EGzYFIf z?oy5Q!jS`}%fz8_FFMDDZ>8ieZ0CG{bXM9s1EFj^I zR2)PKRVl@)Um&oka8quLI5Y~QwTkKyli0;{f(Ug-21#rz63WG7Lj#IBa$pZ!% zk`W$We6dJ`kAqMK=ogi8w+9lz?p*JvTUH8rZriay3o&lm<&7=6t4_NV{WcJZn+=>@ z|G!@hA9pB9@SExS+`&6$<7I1P!lU+(($KDj>>CaiT6oU!jAOBo1_3Ei{8Pf*FaWRHS4@+g~W5{(|q<- z^dc%ndED5^oHsH;K${Xs$GvUYsUm8hTLPLs&y5(#|B9lSNH(df2jH>CMpg`4Ppf0q zSJ~myCI^Bgm_XblDOWUBn$2|I`zHE z*==SO^I@se32e>rg9)HjY>YVOK=O*o3`7%ixy~n%h$N?1=67cJo{9zkZ-Af@g6l(> z^Po3LVtrIbVnMOg`-NV*A(^x7uandmN*D37tgjsa7Exk;M3?-!BrJB~iM+`S|d zKu&#Ev;ioR01IAUjpz24j_!4sMg^fU1?H@z1o__^3k<%UC6lSll^m;)rdy^v=Z~Ak zDg;39U&*>kyXme1QS&FDTs|w~{B74i(9ZOCUy1mMbqnF6P^$pLzzj7S*wUbibDjr+ zsVm6q1p-qtKvK(&YS0_-@GF*m2ZEQ4b~c7cZ4{7`V)yu%2@T{B zd`2S-o_WCndOn5KkjI9_PgR7dH~fNb8U<9b{Avg+&oWkjsrrLG)%}s7k=Ko=_mmwq z4d;xl-p(ZwQBLZoUwe=EAo#WnGKwq9wO_nU6$8nN-gG+ZAAadq} z@Q525?BZni@8B}}K*nwJl`fI3?<-VRA&%%C>!6gf(*c0fG~F-vM}QE6Wu=s?Bd*tazLA&XEc_HZg6njMHwUstm7^z zq5j}CZ|5#QT#~r4W$k6l9;6I*9dF?h4~zALW^t-ptiiNQ(pwY^Q~JJIt}cU2twndL zS8XzKyCWthrp5CbhKNBEfGYg(aB;4qQqTu8^i@HqGpA#*#h0_%YnB!_)0{2cW0b^ayKg}TN7V~pdcql5i@ zu&>IY3>uUdYGy7IbLs^D-@ z(0Hes_4_%V=iecG@`8ZK=>si}Q_`!&^I4sEN=?LN!g0x7adXDiObsf_ETFW)Gw1Cw z+F0$YB1s2>RJl}bXc+tS0QMIi`ZFJB-6w{NkaXY-P|j>0J)Y%^3pI;(wsf?VeaqeB zYPIDs(Z5MISykaiEa{ty81<>`HeRAaW!!^7bTb0$&X+Y*9#7~9?eio=*`|I3Q? z>L98ykp(Qcjk=&580hQNsH0vNe4ZM$Tk_Yz;k<1#U$oS1Pkoo%@}d&b85C2yPu;jQ zB?T}K{i>z`c9Fk~p;h~SXcpxvcMll?QS5Ly>6;Q{h;8^T$CO3y{R*4P?fIJXTo}-W zObkAq+v+GF4s@^wZs8I+kCAKx9KZYODIk1|MiM=d223HZDt9M|Qz0;}gfchOjU+wU zBH;KMb-cS%Bi3U!@Ne}`e;MF#7k^bMJpTqHrHC}MXA@T`&MKDj-$E^t?QuiYOb<6l zuDH16sOkbSN9V1UOdO83v3h_UmdGX(0q&){e-ZAca@F821(BJ{c=Q(?5?#{s6v#at zWl5ObX)h6RpGcQ7Y3}AI9U>!z!b|ZnQCciwRK6UpUn2ohpNv`BI8gjB2-J>+(6LDM zr#;&;Pxu18c(hGy#t&Ri@+;g_tc-E9-aLCH*%nK55QvuJE50Aoi9c&a?TYxvLx>#X z_++DNV2Xg3lK+Fz`Q&~G1woqshoZ>R9(InxRsYk|O8U;j|72YlJKB{w}fMlE{kIm(e zrwlPa#Z_)}eC?ojHpa0Ca^X0guwUj|JaJD&p}M(`=pm^Nc*`tg&YI*yFtIL5JTx&c zV|xy&rj*K=pzi=`qLu}`T+{CR#l(^-V#T)9m|Ml3Zf?b-#Em(BzYmR`nMQM8Uu>mc3^F`OOkXHZPzCt)} zESK1EOM3OSd|ANsUDDQAOZimSnP{b#`9VZ?7Ya|KfMD-*Mn_r{SQ3z17mZnRgn~`# z+)oV3X>KJliT_CgfG9Ddi;jco7oBk&LVV&vVTs7;mqOXKAYM!?&|0gt*CUHs4>}c= zEIVQVVLPOsq9-#0bVl!qjItn&3AoH))~Z9_DPAi`xJNWt93oOy3>3gLY>ea(iP%q= zX_Pb{n(4Y4=#$3YR>8(=Mk!VcM+^v66Peg0clUHjdCLOAzN+|mj(66s=(g6UBb>sI zn5&cI0IAf$6;?Y^fRrsX1WK;|>#{ymY zLuut+gqg2`on2)bG9d4qn=*!t*M>JVgT>1kTfBV{3g5gX2SPl}vlJO>T|3~b4o3Po z&I7mL>nQhXqPu05#8plwl-RI$Uf*SVbASs}m(9+zXq({1Wc(6oP?!sF+o~CmCm71J zP0f0MBU^B`kT}^|S^-vtgY$s<%zr3o~TalqCuhd1`TJm znCMbVIPV9}3wyz?LD8L%EC`4(QJBh$w>IuT!?$S2z_VQjB|?uX)CJO5v?u-bIij>e zX{E;2OFcz&)$#!RloCgi)L>uNq;~+ALP6kNY^=lr^v^3F_JBZKhJ=pKa+AYE*lY~V z=z4+H>QRowXfE4RiF|UHDiazf*^xJ}Zg6k{-&Cc7g!?cFiyV+=6N!L*NLEvX<@GRi zT(5F&g{hH;j3HW^v?iUXIEpAc@+!73mTQzS>UG?b9Uv*z&w2BVHsWxr2rV1Y3w;1N zoDF>2y*&$h2>fIof@ivL;VMpnSDPcYSVU~9l3Y!JG+TRAJqI=e4<$ykoPn&85{#A{ zHs>z^v}Z9{tS6Bx3MAN8?FZ8`rFq`@1RXc`4xpRPOTWIdwg&#&V?nbhZgix8WTk3= zK}L-l)%&1>RwU31OsFka!irJ=%6;=Tsi0S1pVDgi(kunVLV2M+W~H&8r$x z!&~nrR(pU<01B?TXEE(iNpZYo%l1X!_+150hB-=SfUmWTxc7Au{emggS&)a`Oc9wo z55&<}Lo3jHxdRVj_z= z?paqdoXu_n`e-unkmCx*!mRFRk6a@<*h$M2GY1H=_Cg=Xx>&Y&S|e}Kqk2)UkRcE6 z`P5^$3-NmOw$-g$x0HAk;qonp(^Xd@dRti`Gn60>b3N_Y?r4fX59 zT+oA4{KtMIny}r`Q+{Dg#$e0ZK1no@DM5>$G4syg^n1WVPN4~=IOpnV$>MY?7N-D@ znsb`=kJ_(_)e!236N!_4&=X?Gnf)O|avTN3xSC)PoP|t~YGz?!k+}Z6^U0G`Sj9D7 zZ4)qTt7g}}}Ti)G<%XPnDmWM)~^q_}LB$-wxh;s4g+4gG=NQR%A-3Kv|f3Z+U;ZK@w!~wA_2~ z44!9n7NTIebR$Dz6gxRj^5tN0dMSbImo6%j3RnlRErWB6o%#{uc-`q@@UCKzngmy5 zun3DOPT|6Zxr5FT)rDnFF%+hhRx$I$&zR=V(@K%V^F>Q=?4O)lrsp#}*~hKH=J$q= zHWKK?Gy#&y$-jAqrGCSM1Kfo^PP4r|sJ&fzW~a!Fiw-yCpYPs0d9^Swh!%j5SSJu= zPI4lQ2m%G`xG&1r-!qtI8M`tQd4JO^g-t&HwhiJ7sqcywSws;arJ3Rc6G#L4dXl3$ z-}xa!Ix~}=94O>4JkEE2-^bVYKqN}6pW5$3G#k?|k*Qf9pplvi(3m!31@_qG0+uOp z?R#C|Ade1B0LRhJ+TNv;e8urP)ERzTFV-)g^}aGW4gjy0P2K&!CH_!N0}VF?V2Sh1 zDSeA^VlUO^w7e`SA?~?IG72^RL{3+`B7mqW0&MOCM)%y^zgNa%A;5OLMN+!ENvv2E z9Q?D>-CMnqP_bETJR}1@5i!GME>ZGIR=L;yAg>UIsEuVH?Tc=9qXP4BKU2&4p7Am7 z=2oy!JmHHj3Pi-S`OPVu1A*bEM#r2WC)N)X3;%2sg70hdmv z@Foqwh-cXyAEN;e!nfS)yb)PO!ooM`}ulvXLqE8eu{wEk1|)FOW*75 zF$YdT%3pPYc5=X}(ZFIoGn@K#84}Kwd!cXuKHu7!$WL@I?%+aM4&Hz*V!#1 z<2;2Fx#&&xG9)Q_)6&wC5=dq6_p2)_6AB(l3}Yy7G7uv83!(}Bz-4}Dgby}&E)=DP zx0LbdO}b`Wr9CczhNGB*IZzlg(QU@>DY-5~C_11e)ozh%;S&KICS=Co|DT}(^usdi z+pnOYpqbuo85z;^-GcK4Cs~dVAQ>@P<*q78^8sIx;p9jVND=lPwdgv|J5 zyilKZRCq^{W8Jcj?}s-V$Ty1Ii&9!dxV>|Qgu6?VYO?LBMqDF6IZ&OZldfq+@zm=K zt;{UZCw@GGD9oU@&D^p>LEyQRC7Oi*FK2qll(_E4B4a^L+qvoz`R}C2HW$%bVS47P;fS-ckg6QQS+w5r3YJK( zvpbomY#M3_+Z^>A^zK8T)fFz-&Ur3K87jP#uSrbbOuI-85i_@Wefb1^z_76_Qq`;D z9jWe=YA{Wu*R+X>L*x$v&|H>qBvU4PT@}-AIM>u$K;zya3eL#7peNrqJ8mvy5v{O{ z!!qze@2Y!+AMHb03><&h0R#-O_MX{nch7W;bRMiD&l#wJq(Oi;iZi$cvYlcj`=M@m zML#*bDpz=!2mpxvei&VURj~f9Wl<@`^HOewa>`APlfNwNZZ&Q^XQe_@( zmEI41CqjpcIn$7;K1~<3c46?%&D?<+sj^>u029#um)u6^-=(mBFmVLba-W~y&ZHrS zjgzb$U2g!8H@BTrP#JHMNYD`b0bfYDM*k@}_@O;V1@K|JZZUroz&9yF4os|O)P|jCgO8<|w zzmAHk4gZJHK_o=N0Hg#JkS;-x25FFH=n#-@VdxShC8QCgdngG32N*<3x}_Nb=@@G0 zI5*Gt_r_Uo{BhRV>sdZ79f!U5UDqeB>tX_&NWoWg12a(WJrExYdy8yCBO3<_1zD_` zSzdUc<7uK4%L#eC?mdp*)k+h$Zk|NVZ+BiraZnceD9HKgpXPf6U%;e#eG`s=fP|_} znm*1ND@zFip}B=S$<~w_2Zx(2(i2e7gb7j7z_I&V0&I9e%TW^S5UlK=z1Im%vealp zZsmkroDJjO-8T{2j~_o=SdzFwFKLO7fmlZb*YBeZK;vaTst8%BvEIL4V&+JO?k4f4 z%%t0q7CADhxI_?(xRdq!|D-iomUw0ElCo0Z#!~)IDA~5){E9|JsTU9zIJa<%N%`DT zaqkMp0Mbcei2lCv)=rVwolHGw*;D0nNpMz|?B?2=ec`8~Rd5L!%6gk>`8)qwnVyt@ zQkFfK)%yVi;tuQ+S_D@a<(s%<#5$n3v>Oy^bMN6@Au4uB?%7>e> z&y1rgc-+AdgwDlB)JweDPQ-v&}QUf1&mow|II}d*4ET_ zt8(~;(+`}Ro(arxLLn_?-*-k)`kWemmRU3Sg6v??&!z}OP=Q06y|=Dko z8ZNe=@aBWtfp+V46iRKLo zA0MLSv-qXCfu>I?`|6|a3#hWDRF0a8ipou}2HJ~5$Jfq6PFz0T7@4*J8cJeZ7pP)T zhEj9A{je(_u zj>R|21x(J*VDBuJz97v)5f(ClZTbw!r4W@HVm&s*XEZ_I=w1Sc?@WLH6h`d^m(VP* zAu@pkrUee#7gpclgG#EuV=a11fs2Ey)glPs#BIqqY%*(bRuK}OtQ8(Ueh87^mjpAF zbH^vGkbw%<97Nx_n+S*WWDA=#gk6gM3!4Zu0z|eI)Qy~LuZSNoV~z2dkpP!dWG);I z{sW(?2m^u!p*ny5|Aecc0WH{G+dv3A#uvRJ#k6e-0n_d^_#q)AQf9x#tc*25M|y#z9G^wy>ZKs0m?wDq`|GUOu7@Qr%uW`Y&#QbGtJkbr=Ixs-G?$} zIDEt&um4z7Kw>G4!d$+d;B#}ki?oY*WbwaSJcPk;&LKl9qgbdF2!t!xpS%WMJY9T> zC!(HTpNv_3KE04`<7Os&8&v`?fGhGTUJ@ib*wr}*Vo^0oKcuzxf&A7;l!-2+DTcH5 z5?Dx$@I?#J+T_+$*oB&|A)W#W1!h##RS2&Ktk5#R|05DxR)5=YPhip$Fh7)4UlFU* zur&iYn=U-JIL&gVZUd;M1^{U$Gb+3=C+|-HjS0MTu&Rlp)k!;P02zRjkUsubBSNeqLelxxL?Cf;egL};W+3d_RGO`RBq!HF|Dkz{J zAK%2ib?i{}+-i2YJayDDa2qH`#3=ROhEOV9*cA+7DTcwjhDdRuU*jw1mE6T8lFjwU zHMW-=<>PtDd*e%p3<|`8>yMlfy6MFU;l0z-hKATS=qDF!HiAHF{F%|jv-|zv1gYjz z)=3O!SBm0E`zbMZ_u^l0q2!Vw7O=1l-@0to=#y|wTbE-dCFrDI56z(tr6lO+sU2>| zg%8WNZ`jM{5feGe1}q;C$AA@70%u)OAp_op>{~aQ6J3Li;OOEV zSNMt7(}&?L-Y$x@6`RA>W88$sP9J#4$0FTXFs%IvM}S|DP2Y?5ioo6b@q&XNxA>@? z#69zOxRrdKjZz`yrUZV-n>Ox_4+?-{iNZOH#eKrK1END#!-zo1i`yRHtgK~qj9z*b zYWgIE-`VX&jt!M+ZjY6Wu!_qD^!c0g!h63CM5Z&p#fKDhoc@8Rc|8#C_9C_r>i|)p zUa}UHDCBh_`#ATTo?zet4FyMo&C~S;daw{g0Fgl|lQYV$mkcwi406{^mRWUmQP|72 ze1P{GCoXUUAtZCRn%y9I5kU)M`Sx~z&qBM(i439g{#`!U7cmwhOVL)h(F1hnU6P%Qq7SU=1NWF_2`!8A2Bwa^&^ z96l3Mk<2G@5G?RLCj%W?Ve4=?A+8m2w=FgpyaoqoQOEvjgV7cZj=g*89$xA7?} zCQ4J@Z$B(KoB&!C^rmvw*%RU$3jqi=MxJ~IGOg#d+@ma3%=(j0LlD8i1LpWd3q7cj z(S&sq4G}a_n$M^nW^|)kZsk4$RzdGa>ZqV4W)TX@*K42lQ9nmb9!jRCJq7{p5QtXQ4X+aQ%CE2{o(W3k6O{A{KPjN{Z#v|a zn0S0B&+ZEcQd8My00Y)->n1O}s@p`s;Ym^Cfcjiph`?E*_=Ot?|CNVxwgg^3U-W;W z;n#!&YT__bnPr|}Q?)x*;Amv6kw_u&Yz}dwBLYl8{Qr<2MVcu-$5k#Kff9W`&cTe7StA(s$?|m!?B!%Ja zJ+8%SR?scT^O1yxuP>O)NIIV`&{$pSF=3nY+@!Bx+B?cSZz&KpqB@J1Ix_)Qk4O50U(Kz?>sgStS9RVzz_ZP$3+x(>iOO?sfM!ya#gE9mU&(P2#Nw zO&P)-8WFSbg>$5Z9umE}ybMgUZ9f^9p@y|j02$sG`Rw5IPV21VOR90@(HPl08JYXa zpQN7=(78!B-}7Y>u>^2KO;)7Jm}Y~A+uA)yazV1}-z_p205p8cX+Pfr!E`O2;lTQa z43=1tf58wjkPCkSx})g|(%~3jh=3(EjA8FIF)|l4MqI+9;&1>lgMQ4C^fPtkQTJ^P zFXUVAtOLOzk!Ik=3#?wsnrsS_?8|^}B1|a&Is}8)6r%Q<@a)nJ5Jvjk%2K0lnJ=Kk ze?LJWPaZO49omCbaDfortwH7wJAluUSh*W23?6qmQ5}S3G%Py$ z3Ud1Ybs8qTxO)q{J=M8}Dp`$gefvY5=e-2|Q1U|d($E|r=`1*>zAv%`;3V-?l7}So zqKRD^F*vE+xx5~S@@1aoH)E-Z4z1%qoL2&Z8yu)yOQcLjH$f9enuDt5SCGlM81;d>urF&lZi#DR>Ai3FSVm&I0DPjT^d<%ER{o}A=%%)H+ z>zX^D`EW(GUtb1|3uoo3Zp&*Gf+n{Wa9i%rC9lZ- zv(&u-K`s0#!hYS0HY0IYQW^k{IuP>S8A{}2vDMS&vyF%gwvd62SF=+|-&d-XPN39P zD14zS`8I;6rrh@-!5D73{*heiOZE|u%#H5m;NNn0gYyk?9*juz|XxK zhQ}~wA}QQe%uvjK@}tp{pYDx|b}%PS#{2xV!d9i1qft0**@q<{A~X9fDTv#C)*m4b zjtSl*rnlMFrhv~__OwSK{U*u{bYk#sB^r%r-UHG&6!Y*^OdfM1^W%ItTt=pfozo1B z#bv$bMQbjWMJ)LNFH62ZE4*OGUBVD+J^nCXWG_~xxJld0_B}TR%byPtl7K1VEB`_c zdNSQ?cB!GXj8Iq?cy2N*SBmWCi>;7B$iuI+EO2;31>8oc5-BkQEV}T>3hHFCWfpA~ ztzD6<4gTHNL@wg%3QHR)^zQ@VPmd_ReZ14Z*k?}5y1Qs+CQDC129O*^Hz25@tB`ZZ zE#!$Eyszu_i>HHtFnd=C>;}M@!Zy!tQjaa6KQxQWW0DU%J_!#pW|bb~-7Xt0hTC{E zzcWad!OY$GR8)&6B@&*Qk>zzfBldMp{D>g>Ongz6A@@6qr>gi?1z|RS%Jmi)jdS*NqOI@t2$t$V~T(`Q#PkJLOW|C#dsO{l{ zz-r5ZXV~Q;qS~;HkW^dX?kp~qBqY1XD={A4*vIAmyqye+UtI_{YqnG#wcdEG=J{cS zhD9oH~}qK7FIHgB1xl>6zrES76x||5O|Z`rDS!-UwkO6IYI<_LG+Rvlz*0 zUF9F7JDJ_#s42t0VXGFL+=z#q@KW*V3l@(Z`B^aDudm6V|sHVDH9Atc~z$Cpe!z4z%&G1iSGw|Z_K zm(YSTO8Q|QkNuqyPDkzs<@pKJ1k@+*f;BX0PeJ?u-+!LyGnzMCoJ*>{)%tOSsG>9+ zR(K~Dr}#yDpvoqr!eY{xXHOnD)k9u4J)`coT=_ovX7p`Y!#~#%LM0dRrN!GHL9jk` zSCmj~i9s}WAunEf@r?!YFQURm7Oe2x-0(sfH=|F|e9&mP?ba_=5N0w_o@WJ!kYhB! z;OwsYCbFk;eB^|&OZ{c^i{g!#rG^R+Hm&{;;fZJO1v2jUc|!s1%jgw}PvQ?Zdw8B| zOJ1m9GiJthWmMP1ukP^O3m+6*!47{XSg}}bmS)NPPGcGHEqu|IB3=7WhVkhmId!|e z@XaJ{0g+CU5tq<~kqT0+H7oJl)HUFXSddMC)%z zu8n$wo-fz+msHa`wn6#BuNND#-;Jh`DVQ-%>=K9SmMI0t-RvQ10zr~~cW*4_15DhX zmSQ}q0Z9!RV+LUeb2I~k`SRS|)(aIV)DMJ2ZTGP6P@yHenL(8rSy-0~vwpS$5fXji zDOrXFej~ErB>|gh?726`Y*M-hi6Jt|qV_R1SM*7Hv|QNQpC-Hdh!F+USD(N`4FL zg0-J6!2jT>_48Acy76-;V9}05OaO%!oT-&;kr-al?OCg~Sl04R9w^6^dMd7IA-TO( z`LeB-tc*&_I1_Fe$x*t`pwI%kL@X9geTib=@gB_%!+KKC@alv18FKJ|g{N~Zzu(17 z(5m=3-?%MMsp~SbW6~XFs)_@j;SC;mm9M>ZtOZ!5-^=-{Zc4~a;QyDsT=sszoQqVT zVkl8nu>{dZDGLEqCVbp*iF!Av9|)=NPI%p3x@7H70nUojdlQ7x#9%}fy>5ku5>Y!& z*Ots`l2Qpc8Y1y-nOiIt(eH1G*vp*vYse{!3(hp%S(IF7@i;MD3(=HNn}EWQ`H_k?57Qlut6 zSPCem=_xc}+WN^*EiFgw?trG(4s)8AL7){(D0?$8%`Ql0@_w0nf zy&EQe4n;S&*Gn7nYs4^q!aKQTG|c7o!6cp`Vo*1mN1iV*T!U5}eT!@4&1noUNk11D zl{#V0gZt_M2vQ2F5)`}nM#AB&6{RC;j6l`fd&?yBGAg6jCQbH$#`B2bMrdC!&w2og zW1c*mW9P&?iAl|ks%Xl`=1`Kf**HW>k4L3ygM`c4z69ck-~d1a;qAc|Ax|@(C(Sb? z(kHFZ4sE?tjAJMi8Q~Mfe_+-f6ExLZWmIquI>!at1DY^qF<`z3wi_Zy+OHVUh$RsFp^b@!)(!S#0ZUSy!ZJi;< z4I2X)>dtattF|wNo6P_bA!ca}0od@*9^(g1ligx6cgK_@*le(kpQXL$Q;);j_~g9g z)l`T55SL}oZlS6E>0G5Yxsnhv{P^i@^ADb)w`Fu$*di70zwX8;e(sRZF~(7dFOGRd zH$~b`?R!*1p0Bbf@F{_1FHvsYzXS|G`jeXhAmvu8l~|1d$%HEG)P=phJg;I_I|%BN zXf(BaBUJ?ryoT4_P8IDbnQ>0M9ln3WwYM9%YO^e8aGyg9-=C8<#gX}yo=&@~2Rz~D z>eKL-6V^4^BSEi4vEe!EJ3_SP(zS{gsbp-bcYNb`+WUCs<&nrwwx z%(#F$?p~2hAStEKv*MM@QvZoFJplo8WgLiB;5Zgz!1@HeTS$Fd6jyObCL5>dyeaT) z1Tj%gV)}z93s%mepTZ46dnzY^fG-wRyD@`dWPD2{F+X|d!=aPyQ}`|**>s4(@;QTW zE>N@WjivOPY-adF?%c3HQmZL9{P)%?;=8YJ4IaFkdU(uo2d8}WgcV8>!L;X_MgXJ2 zA0_nkvoIRfGMZrF1`IotEaya@z5jmyo6m}U1xLZF?8GV`t=Nsn7R3+ zlTGGj0mk_b4@Sr5?RPpNqSiV{Uf+Tz-1~H!P-?Z;%)wHW{CWE`yp6DIPJFR!X3n?F zOLib@S&jIAnBA)3|HGoD63UGve-h*A&B`!;Xr!Y}+HoKvv-dK3LL+-OFsp9lyMqk$ zrHndE^bbyJO|^#SuaC?jcb-O4Fr0`Tgdm9GwVFPhI3p!zO9J%xV`e0LJx5YPxEMmQ zT#glg$!F$M7fnmVGq}`2?nYXE1NA%UV4ZZFpe6$;H@9B$`eD|Fe@cfOTEy!AJGNy%1FO&@rCSV+v5)&Eb_aH7CHlQ6Y}2z0>S zNMmXd>Oce<9j&q;HR9#y&$^WjVry$YOHl2jZOG=sbCq+XYVhBs+*?oe!z&+lDRz?i z-+F%damWj~>LpSuU#A>_abeRdxg=aH5j2Src8Q&ln0>ypAu{+l(QQeKZ8;m_`YmkH z#AH_~F<;l1U2N`)94=q~$H8~uCQ`8)q|gT|4Bz4xfZtexESxT0S{(Sk|2IxKF)0M9 z@_@`+RuN^IW~rh=+`rK#@CHdoYL{MV zNu)*c@AA5mf1Txhs=}=Ibl~_C9=n2BtBv>VEVI^$gJEUk_>HkQzV0sAV)TZyQY>^h z7i9NTratX;BcsT&h7OWk44#WAf)NnoF@M9}#?Beu_Bkk&Od$4*(zAT|t>nWOS^F1o z*2=HI1R%(~@%5xNd=S_(i>Yp`a60Dy%XlZ?sOxI+IjCAYt?+3u9whFg>G8$bm%*>( zD!Py1M+G3e49TJG13a$ALmavf8&jU$67pM}*k_E0I8AHsaWUI#XlH&F&Pidi^5Hg$ zo%1U`?OJd3V5+Jpt%zl|n}(T=)X0RKNN=q0`f3gSgZlfb$6jxrY<*yy(;y`LNEE5` zYBVi=O69aHbg`^ZgaZE=ma$V#0~PkNM&N3inP*8!zIK(4BM(o9Az z$l!mvch~ZS^Y+{Bt9#4}#4xU*AF!bXiI+5Hn4FSK>`JtRX8eV2jl`nB zF&1BCVu_B^G5thW#*+fGaeMWwYUh?vj!vSk#IJLrAkp0%wLP*2589~p8f@t+BF;ay z<2?uRzz1L}EUEs6W#S1@=m;S=6p)|z=2c?({K6qpK=bn7UElY?n?hz8>)BktmXh*q z@E?+3GpFGTu|?9~o-b10pI`hd8O3QhL8e(1(TZ(OAkIN<^^zezWIpf-#b?FL(JqxW z%B4qJ%O8o_@1B|ws?mstKgRzhb^7EP!JWtn{QDt2RUb*lwZwbhgY8rTsz}M*-day8 z;SJqlF;_c|4kKd3<@(P9l{<9W)F=F`{h=T7uh$E4bPCXsMfJ1a-HB&VdcYwsd3M{I zJx_t4>|HCKo@Ir>AJW`kWXFr^1dIqm(h%(-n z@Zaa~Dwaq{J-*Vxy-DUyH3sJ_xL81db#nzK-e4JUNhnh2r4MUC7sh?DSU;Pttvdod z1$l=DN~QeIUjr<6b3I|aIjO<*`2onvPVHvHj-yzxmn!KYjWC^%4I6^fk82+h@UEN~}(+;BKdKq1e20%8=rLX}a0~M7ml( z)a>U^3cI0Vg@PSA@Iu-g<{OX-qlN2^I1=B#`_CcV1ai(mAJqSz_5bbVfQ%3p{xYM> z?#4>SW;TslyXgZ}&rF>TkWmf>rZ1%~aG^aQODehXjc5?p3VPHG@Q#^Q%g6-|HUD zhoch`fSs=MoB5+~9IfoZ_M(6iB;_ZiYkzl};de^AIy*`=X`QrU24?MV}v71|0OpitB3Ex+GX$#?Z|c?`2vr z#FCO-o{e76HpfooHG<%SvtKG6dGfb^ob9S~kqKs$LOxA5|J~)w;CcF1O+ZVl)qVhM zQ=qsOH~FW5fg@WA$EDX!eNl!_NiK6#RYAbDUZZ2PlHB}r$I)uo%u$$!9~!fIy#E6z z08^y`1!~eUD3{k@K@7CWf zm#b5p%nw(m^(c>)X0u@eUZdo;)O&hzkerr|RB;sZO!}X?IT#higE@dv4 z#?D_}G*8q!PL1VHA&Uqf|0!2Ry`YsDS)E40dN&8f&L+6-8cvA*`^LFyA_!M$K0A!M zfSleh5E^OtQ|MhGRbKNL0@k8}w4_!)^+fajSlg$o6B+e!;*3U;<_m=W_swvQujiOY z2vgcW+wusn5xqxWy$3p?s;rpi=pFxnM=|l-`T#4{=9{bZqKthE(o;uBTrD%)ru8sX zeD9)Q>wgXgQU1pDme6e>UR z;eB&(S_IlF=1*-T^-`KIp^@A13lkINI$7vbQ~-I4z&63x3YIBCtgp~ZBfc-4U%_y=-OP-8s5VfB{d^gSOob$%8u+N^Jd_0$p zLIAbh ztYG2@)K@35fQ8}nrK6g)!K`D^>?F92n z{EnCGI1v{cD|oeL!1q+XKD|rrD3;q5hZ;DUJ!tYkdHz^hkTVpp+-lvP4%q8<>r8df zGxDMJ>i90Of$(42J_e$KdD*xmo{5mzyICWXfIWr$;igo@>^)2-%-O@8ZF z8#`f8=r(Njr$JcO`9$U2X&Ud;hW!ns znaJzpi%U87i#36hzJgCzXHMNHFjhZi=%ei%;H&yI?@uMSiAA?(l_<@>Z;bV#kS`s@ z)BjWouYHbb^h0<@Q}K54mzy-sZ~*e?wREIAAT5?;0D_-i`khB zSmxM_Tg_f?C9k?kzX(DMtrbQD%} zSF^MA{NYI76tEv#ZMc$YBzW8vVJEvti_F+N9SFLGI=(hQG`QAhUUL}v%7>}g=~U9V z?c0#;-olcwlz094Il;j0cL#mp8^n3DPA12a6d=+=oL{Bz$hzL?b0#*Ade2#|f8}H< zc}3KwvP!bR)VlDsDgWH?`yxt82v(qk277?_MnBiXdK=fZgH?>=Y99OXDX4|QYo}^F(U5Q+T}|}n!38#n`e$$oTLL5 zQQPdlxjcfgz|IWp$8`=9u!K%R>3D|0<9B+TzXz}1oe8H4IJi9y^Pob<8HEUg*S zY!zM$>s@t@^N9&5kBCzl_jub9csU`CQbS_?|4wIiY8Ap&?sCRok-4M}n)7 zo2O}KL{vLR>c1!k{%yRp_Vj!$AP-;lK3ed(Ho9o3;iMg|o$}8@9(RHT&*F2gsem!; z%5rgB&6K+qIi4|j)OYGUJ*GMMBh{PwmGVUI3)eI=>T*Mq*Ecve)COe6l^fOAwpR!cc*Ion4V`aR8 zLo8&~tLOryHhAElCcb)EZz8^3=Gt2w242L~(gm9OrK4@Cg7) zbA=HcooF*;%RY+Hx8Od(qixoFiz;m?y{(z1@w$HMWAW_q1u#@iGjU*O?dLcC&WXIc zPQK>Q2c}Z%2uZrtrcKZO`i6vEPZ{6d0!5@r51Pt2G_*l;4z)`eH zi$r$`qG$p-f7FW|YMr+=`tw|tH{h*|!)r>Nk1uf>-B)-!_a7h&faasTh|}BhlBF6KJ#J?}eK&OIz%}RkucC8kLflrG#Nu zXP;E*Wn^$pg+Xuu?P`ntORW@tcx0yu7K3zrzOzb2R#}+HIxVra*I$#C%aQ%;KT%c5 zcy(9XMRe9wxPVhxM}sYC8?~Q}WXd>Q#&g<#$)W%K>L7(~KJ#oWxt}`V()5USd;4Dv zR~p|==y@k0dK8`i*gzoQ;1rME5m}@(n5X3R`{(F@{{_;3DW>`Avx-qb+ChKIr$@wu z4>*(1KaL#cH^9bra%DH@G}pAZ>V@Lz1cg7Is{Z)C$=zw}?>#d0$fVpxe=3%5+*MEe z-*!qS=Va9!x778;(@+za=jE|`v^8BrsM438PS8ukES|? z4^#Gq_bSZszH%QN#1_5DEVk7L$FJMQY?2du?>@`5z+tJX_*bN#L54R5FEi2i!RtlZ zGHb2)d~E&4?G(b_VCj`DC+}KT@U&*xKvBY!`ygLw>+pc{()3T%;~BGp@WGb-P?m@2 z)5W%W65AQ1Qe_@^GY(T8Fx`6Nlvt{B5YH+HT%JzDHnLor$s?^k)^h#H-IES(_tq^S=|8R~rwz>Sk)uZu6j{3GHN#>oGWAESC^3 zL6oc6Q^kI92~rqh(l-EfNm7i@7wbCObhQ_EvB*GRWar;AnIQf9bUSs628>QmMI8p~ z1u4M3V^Q8ym|4v*W5Xn(8(0?D7Z#W! z`wD)o;HN6Igt4Mi@=w28W_L&Cu@E=y&2{`Fz9abPP z)U(H9r5I}dibYWV5FV9McC!$lYrur<;x6&&_ArkJbH()F9aMv$j?4=I!@FG!? zDf8lQes?_zW-@BoU(vWRGAMo;?sTv{VFKHB-kp9~Iy$KWDPB+mNgI$SqU2YY)r%h} zRnNn2u`U`pyO>P@HT5yMP87=N?_OkXe!W)|&jgG&2Vf=R{~+dw`Rc%}PTlfpSbLIH zhO2*Gr}DEnd62;{L5w%rqlx>ti^5>H%dB&9vE`B2>0~|#PK@I7Ipy41b~<^$8^|+# z{gBRQW2%Vm+G$-JuTJb_MR-T7LD`@)b=6BV^Xo;ckJF=nXzK~2)G9qmUj3M;s`KX}`+Z{*6z1syPKf>Z1CJec3ZG+abF}~4?dkFQ7-S}T7umS;%Juh; z>AcQrF<+!p_a(RwYWE}b>+ZD{G6oK=YVUe7H|%yZlJZ5M1| zL2U8=BMb87UeWlUg`}Wc?6zP!M*1!O!Six-!fWlVVKA{EC+EopBX@gLCxxBI?;O6i z$)V_oge*rFiCzS3(<@U)OUrFWncjn(bm&L`<;%h#ls>)lR|2%BuK1%+_+)q=oBjvp(PByA7&D`8)(~R4*E1h8($uoT%xi9e6QB!P=2E3XN^uUvD_I^5@onai{!}mf$g~MANwph zCK2v{ov-Bv{G&Y=7mur&zuU{)1yP52)b=EjBcLlL0|%$;i2CQTgvYkK4v0x|LXvt@ z3Z|uAN$|DNds7YuGP3nqN_f_Jw4BcO3X`WQTJJj*u;fUYVXHJ#Cl}M;yI>L! zP2<=5sj^vF=TdXeFJQEjD5?sVh&tIvr(i4R%w`m6o_>^}SRq*pxhb&#d7`Y%X>xSZ z_HZrgE6;NUjacJmt{-M6BnK;`JF{O7pI%?p{L5ne)HN$sah?5;Sg^JYd5$(Y)8wWQNKp^%29*0DE!sN@@Y)Y2jvx+`cl~J`|E818`iD`HYkxLwsw5i7o?}?}X`wF)anxZ@g?@%t1SilB&aO z33F}*;@{YtAU3XGet67@FMuyVH!OFi%NdeKj}1+c|&m_o|;{@|(XAi>gcOw_eU4X6;?0sK>DWB$PiW z<76j0a#oFI9jAq4hko|tvI;rh%u2f1+wwE48vbwhHfVD!@ssU8l?D@6*>i7JJ~V#> z>0LM1|2swW-$(vWEZzT3acOyje=Z-3*E#of=lM@#O5U9-Hvz0{*8eq@Umb5rX#f12 zZ;$Cw;?my3&awT<`>(Hg7SuA*z82lDH`AsAGRhYDtzH=X=)YLQj=!aFf)6h15C^zUQ-|Q-h#jCC4tB1ujLvpuoIs|B@~C68pM8zr(S7PV_x`jn zCY8w_H9rrUz!HgS+ABQ6Y@}2p6X~a&S}rNgk!bN3Hot;S52*O;J^Nhz#LsqvADMW6 ze@$E-Z%oiRi8qSMy1Hr%b)@n9`(YxAroe9c2m}y*?|y}1s6_@#4Jzl0hXuwfQQ3)k z6$}wAA^qw69nB8DsH}@g`U7N^QHx}od5kcp3sZspJ&(T$bY2Y;adPhH;e;K(tvH1+ zy7}ebVt%Y>h~|&WWFBe($NXh7HtnxIJ6@~C&ZcI;FlAwXo{Lp-bQI!yUsjD@ zS^fSpD02K7r_SsAXuIX&;-D>7C7EG8?DRm+C}2)qj_zRcA@E}ywq1B7(D|KQg*^|S zL3s=s>6rLjwmM96cUlgf?X>!!{j3-BHU6adOGi}=il4h#wGIf(`^WMeZS-;h61C)3 ztYd8AUcm__B`e?@v$|^Rsi2;yW8%NwKVc$@4*Imyd_GV!D7d;iU}PP9heX8r`s7Xl z4akZ=GYI#Y@x^pJ60UP7dgMF3S?P#EiaIit@XL$rUSZ?~XFI~3lNf=hTrODWq-mIk zar1L`H|UWML*pgL`w%;G77Az$BNP?j5~z%#W}rREc`S5sFlg50LScUQ`nV-{rgogK z`~k5Y$*j+y5xrbAs5Hamz-`{g9bGd(+f&cA>{P?fWw|mrD*o@y`A3Db-sx?hmr_?d zJvmWaN^tC^BSc&!k`DujF_T*71(u?W%out=SV@mJg_#}{DmVFL@VIYoo7q zzSd?2w4N9*K(pSMQgRk&kS1|NWLeDbs6tzv{y0raaACeW{UgeAYVms0tBrTGHdW)A zLXV&s{(gw*P)%G6uXCK4uC?LS2Pt{#NzGz~?^&0Ws%)1d4V;h>s6&vm^e5kD^myLy zJpBtVN3U5FU<^g;2H|I-@pD(?y4{by@+6-G6-_@`{{S|u%Pt)8!?h{96&({IidV(A zVhd;rTaI_G&IQD`b_d*eZGwr~vJd1ndQg^n#(jNNMFGFR9NkGcO!+=@Y#Vwpmn21L zlB|W7nx5h@kMqb#c(Lb^iGR{BLQpI#A1qrjrPJ@E$6q*mr>3XgTZa+HieviVP_}?n zarANtm$CCOH&tA`?d6Weyoc91Lj+?CfzL__ljww>^DmIb**B7sRLtIj6S2K=_-q_k zfu!mBr=$~d&pS4=84!~b3loN#JIUzg?{iM>Q81Xp2g>gemvQ?9 zmtK=6%J1mY9J5#TqL5=j{d@Gq;nJP8gZj*q+|x{* zVdaXq%Ewby-HRPAY^%QF=RNmzMpF>0UUl2~c{Ubj9AlLxi$-Y_-wJMCJ>3`r6G_&4 zeX$hFzfG)QPO)`<<##;$()N&Il<%}BpD0EXyz(EY5jA3k0KNaUM3)s{f#7b>!G>v) zde)N{`zC)N>_yt%vX#|WzHfG6!3kHwOZgvokrz>QhK-gO^*JHvZ}r4Wh0+m#4Pv*) z3pTPI0rLYGn4(Po&cvdu+~qI4E&A1-cQ~>U{wp_jLWz#PEO(aWJq~BuDmE}2dFsPEV zEP(R#JCAwO^49%m$dxYh6|%S(1ZW^U1j8p}UIrXrpRTUbc~)KxXvV5?%8#n&pK*jc zr6=HK^_lE?puE`ja8$obj$m6w!O+9z7s1CEI__`PF>u29v?K2s_po-x2lC@@)mZ}4 zm;IHb1Q{+ZwvU?Az@d`FG>#>9_|s3vM7Y|f#u6Kk_WGCWV6tD>pqQZ7GO(=e82vDt zs!>&$@8!4podBYV(iN%r7{Q~4a{?NT&%enSAnrf?TdZVs2 z0UKzs)&+}qt>XTKu?K3-qj`TsOX23l4vo%jX$M23LKTpH@hSV0+cv~#03N@ zRWgL?=Xhdx_-zkU@z5=xs+B_gK+A4!!SxlO7oN&CYAr87f?! zOP$J#?(w$4`FH~qn*wkDhS9wdua&aVYhj&vm_R374BPDGipbbR*Q2iLqaUZhTXQw8 z_In8%(alB>e0Xe)fqi~OPl~hU-DJ!4)eIb3TD|2w6>$Bcll&sC zX|)_oI-B6gq+}+Xz_RD3N&IgwD+9Cv0DkyK)dCAi+&Po zugysxV#gWB?X>YnM4|^#Po+>kHyKp*F^xrb$C*)8hXi*Hf&) z($H&hG&&D{QCh2n(d-XnkQBV&4!DW}{P41liT7Tyd5nUS4?wnuyHdx1_uM-Q%Mw2y z*+lhSY-9+SSVf6CA5YRFj!rx~%bJ=ZY08g>L_Ky;i-BCojQW5BaxRhcvA>#w+;0w+ zNah2U#!K*O?P2yPtf(r0vo46i?s%s%W@ zZE%>;p5qqF^moUHBCe@Q8ijn(K>Ws5?k^?KPQFzmt1Di*sU6OjBRJCg`Rj{y6e?9rM89L zJHe{u#h`M1Na+g5fS_?xZd~Xb26{CF!cgU~Nsc;B=l_&MG2PV(VKJIWCDYL@%k0gK>h-$0t4Gz^m}ek>lNq^(SU6VotOc`+TA)n) z_*zBb`!%>(y=3S@KW-pF7--)D%2G(FCoZW9Ef3!zA^*4Pq}$@ylMqWJh9NYXXgn6# z2a}zhmGx}*(t~Z|hno@F>z}%2rY}2HG!VdJx!!nF(baemm4%l%dz)x+;fm>80Q$G~YR{QP~L4V$k5*8ewyN@9NaF98=2GxHh(Q zE}N%auK)e+zNR)GV!h5qyzkuK`8v;Io3-pG82ygd`s0e~=9;&)vHy>KWk6{8WaJYq z%~17t`lvWZZ;fZBT*ut?70N^)7U6zo#jEL{_d&Hsam5Em{%$_B{1$M%-{4NAN`yD+ zGQ9}lalL|P=siAeF*|;8zH@ff=#)FWRy$$5;VWGa_tJnAKjmHr<&B?+o*mKMOk#qm z@&6BP?;X|T_O*-Rb}K3fx-B48MMb45y(@@_2#AO%Rg5(0T?hdyh*GQ|U8#x+(xpZP z0)q5TNFpFLgh&k}B)RiNfBW7$Zu!nR-#GchfsnjeWtL~o`K-Aj#8a0(YoI);oUg~_ z5%XU;3)9c?4^{(7^7^u@*#k!`tC%xrB|@+z zRSYfMo;mO0aSmVnZi1Yr`OY@arW_B7u?KE_bAuz%r1zGkxy}jBuDl%A(_JZcp4LhP z!#6k4zUMuvnC>-UTWxMGO>Q^0p8bKLK?}QJR$bhSUPU;vav|h%2_=-ZB2H*tcWc_z zZ(U86VPc+OAS%wiO4WbHzq`8^NbT{*{8!A(OI0#Za32`Tl^M2G%CfWqGi_bO)DsO2 zvSOd)6z#|-z1rdUeKp}yi7R1d^^vsZ;&-4QcH}=N9LGUWy8J6WJb=<(tViV(N$5+b zwfB-_%b#mx7dTJ`nsSBOXMg2dmoEyaOl5t*m&`J?ezn!sL{A{^pWPvCHJut78#_dUHB>kQC)W};2q@e* zpnsv<@^6m*?5V%P-sK6d%x0$wYdQVcFrGZ$Tv~`53=%}SUS)^Gy0x;0^l*LC)Nxtd z#sz9Uf1+Ffu~N_XS0-|1gq1Q(1M$2FzpY=^vgt&+nvnIy9dXdM1J!VNg7dW$W zm&etQ6*&GqnO!sBM{cBTIiT^TK;C=2B{{Gxu*mA&B`MtHFD=OxZtD@j_$>0a{A_El zjmDsSvkqRgz`m*Hekj}UNO<8n?)A)h*Q}bEQHo@&1!hNUD zt<8Wf$t;~83bzhu-h{h+m%gav(f<#gE!a8L{edxMATv=OGFiu0tg-NMuIjG?Gp`yu z&Unpjwmvpzm<*uV67b0VXDrZS55O7yFW zf%Y8Lhu#M#Pp*?fr-a|Ki&59;qz7J689yh|eOiNd=EF7o`p42{e6o?%D?OwTHtPF? z31^WLYmmz{^zycfYE7pTY)f}h#!8$mlsGXUKt&w|@0m0_bqVi2#_KVVF)KppnP+B* zDKu3(dQTp#q0^|0(KtDWka-Hoicj-{cW+8hUh7DS{@bv-Yfb@jR*rfhno`uhC#`pG zBM!nU9gwKPjZ;le&Rn0o@ZBH3o?MD%yGc4yF%_6L3U9YwxAij?Ge#?Bx_kbq9iK?R z^s(W$ZrG6eZw0Gu%lbo}pJ-32oJPNVu*|c?RyTk75TiCWF%qp(u0MjK43rSk2_@>U z>SVC1XPF=G->+DPY+-z4+2~R$c6qA9Aj}X=85Kh>2G3@iUt@R#euntKpMf-$qRSnjv*)36LD$JWABH@a<7`9KCO%&( z3YqA4jDoq$%Tz8m*l3It@C$e3Qr=U?&%In|C|@KyhA^99?8_N%1yo#?Xt1>~aK>?5 zizb=vF67gs8)rqVvh3|8zPwc7Pg!@8jU;Wogu=(C$DV@`u&>Kid`dAZWOwbvG(NA*G%;pL3A zD2+`r)|Ed~<7`5T&09qz`G-?)v_d&4f}zL2x(rgXQ`DCUv@9+5*REVW-@t{T%|)#5 zoyHZz1qQpNY`)T334!%)oWeJMbj8^cik#oNqxF<&{A2*TGBe;QMuUXtBINk`a0v@;x1kN#iD0b-`8z1j17!O2l%o|-RXS^DK zvP%eMAip3b*8anlXO&kEbDTJ`h6gTMhS~9-mNbt^J3Q&e&2jz6ce}&cckbMQgzm~G z*4-^!0htw2=&Y;bEo2XVLbYjJN>xlexWwp@HabX{|IJ?D6Hcl*&Qvaxe3aa>WeZ54 zL2n9%hdO}z`6jW%gLIPI>(M7WoIqCdnGHBRvA1ge9OLQ-`X&wcRE2{R?r`S~L=EXPS~9`?8& zButeCm2t9>fD!Wj@Fi;0YHCUaj>VvcDv|u_-{)bkMPgREBWDQZJ&^Dr7#aZ;W$4dv6-5YR&541JN6Mui%B(w~@+$nbI=4>f(xiBnI`dSy zU_rfP6Mdnu)nktjDqIIoo5cLIkF!~AuTMRWyY#_oRJrBVIwk|m;8eGZB;KYZxK1oa zqeZPit}*t5C4dupOMYp0!-|+XfK5(BL=9{WOMgg1HP3 zzY&p&z@}J-<^)Bzj(e=`JaZYT@bh~vFue=^BF4wFTWZ_wC)+2>MYdcY6Xk&`=oVz3 z*X!~nwem9-OG%xC#~LM2pKB_yEY*DA4qY1@`sYR?^KphiyJFcCEmN_4@JQp{qWl1& z!b~*&_&i?Mw7Xsuz5L_HVP9+W>sOv;oRR;^-x<|6T6O-gP{gTV5}#>|2eC5KtZ98t zb+g&t|I1DbH?^eiU<~hn($FJ0&ChwO2&&-C1%)3UAKQIg#}_WxP>DwM`#-})<$aod zTG(3kqnfjxblhQq(W&KH9Xb!@Mc)Nw3JHOp+4Y>z_8wAbjO2Ce>Bn#u#jgF(b|PFf zxzeRKx7o5pePuyTJk|G_)$6q8M5WeMI-aOUqxiUD8KMbS`~3xYUU_HGN3)7e?C%FS zjA7-xeEZS9&=h{N2DgHN)kb&vz`RDxF~y*+1_|j3Xhz}GrAqy3mxrZa2uk=APE2JD z@p6R#MT(an_4I&-qVv~?>?9UzffoanZY;(&IOOc$k}!`ORe6S;97r!1ZStrCoeGz zxtSTXn&0%V8OuEE!q>uoo*mL+T_4nO-y=tl#=E=wc#==9r`xJsTehTDT8@6mmCR&& z%Lx(VuPlJb)M*^3iD@aJq;bRm)L%N|pLFho?&k4(dr)e3d~f4Rsj}R6Xo*-A9$$f_X(~^DXNzUAZo^-3QP3Fb zFkDos(8OX;vYiJ~*$h#F)iRC5S#skWwb%_kToXg|C-a{*^cmh<&pSbixzL&Mw(1D1 zEpNs2L$T-yNq=VN$K5HNZL)@~v^o|I+;+h9pc@3=zCJ5+yS56*pYOhLTsCjP);+97 zO-_Os57C{5D)f`R&0CyMCmRXQ|0NhiLjfgXna_AvoK4wOW7A3%G@&4^j3=97Z8S*p z*RQbV-tA#aS`QTI@6f2py#Tmt^nr_YebsE(6W|%XdQqNB#gal0mzW$tLf15qA@?K_ za(xcqv0+>%(lD5RtDvG2Y>_3EU+u=vMaxom%<8vnW3|zrMIvW)k22#DhkmtuUkx}E zEo~4ZEf6(u(n&b#G}1V;@#I%Wy)q^pPt9hN`a@ZCQzIi~f?NOk4Qc_zkosiX=2eT{ z#eM7mT$GYks_#pNfee;DF=Bl0ds|gNj2iuorWvW_!s-%(nw=czKiC*8tX9chDl&U_ ziR8n+J3r2b&PZ$)rhf3Ij*Sfw5K4HG535wJ`tbp0NCXBD&}@iI?rD#`y1m|AMB`{@ zjwB`OILljZ2QhvR>$frbU|pf8rIiY|nfeH2>xA=6x?R}4hvt5L=~`)UjK-|&BK>6i zOAcsbwC1$BHX*pOBlC~@hDL`RPZi8X*IeG9&+D2; zW~_@QQe~=NPWH7tsj3Jdmn5o&Rm&RSo1T7^>{XwkZn4v%ZD}j7g?Y+oDG{k5bM}*H%lvC~)Z2BB_{1FGwgBXk3b+Sr zWum4vM!kS>MaKvg={ERj@`Dly8}id^l}N2l_GwXZ|*wSebO^PO`pIq7t>9Mq(`A;TkUd0=KLwuklB&Z_|ZFKe3IS zepWSSE_4qy$yAGIh0JAyQ(fAxu{&sB`g34i5dd+~j|Ui|d#?hnXxIySLHuO5Ln@E- zF<<`~&P%Xzt})TW2+f{!JXCzc@5G*^o+7W2J`qSyJlrL5w50f72F*s5xrQ0r;A3(E z#a6w{g=cxu(}ZN;ATJJpL?6RBV3cT4y(EYz>l2hnBrhu>-xD~Q9CQM@{U}35!T-;` zM62fXLg7?pK9Xyvu5)L}VuU!)^+=7`3#Ec~Xrh)qn_zK?d}Eqz7Ky?)S}Eu2SqsoYPYZVd=hvv|c{ z?BF%c6TN=X6%ChJuHmsg(9#wPWmP0&8=DlnS|dw0ybU&lNum%7s-hVE|;or*#VaXzC?z!=NB$Y5L$w*BnU^G0$A9HX-HIb zy8+iHt=zCZmEJx8I2FD`D27mAZ%_uk4VHR7STMw|2Mhr-|xK8NkG1-VG{&cOc1`!_qD~hiu6^kgdmhPt<_(Jb&puiUMJ=| zjnqtf7i;xk{Z<~yRBO@(+3)6-a0e9Cw6H&tOSSrn0-Ls5^QbL-($p6%hS~Dj2D|-i z7&626@i!G{F8?SDjbrs@=#O-^@+ZZ`L9(FQb(B*yWU7q#`F4G=TfoOpOmkaI#kG~Q zA#yS@wDv7LB{?@- z^Cio9S7~g8SC=q$bq<{;Rsk4p6#5e3ORC%nbeLxln@Mv6RqQq=Er6k_rF?i({O-*I z7G`NBvdo7s|eG6rtXMv^^N${g|b)S@y5;PACU0X#{ zL$2$7ke81;hU<$KUU|eP!(4vmoRhOOLqPq~ca*VMXf3dp7%25XPrVgCjtgN^TKfyf zKnWAHe%{-`==_=*D!mFeX{M4qwjeKjz5! zT0eMMmN@%tLrv#lcVEJY?U;3NN${U3V=LLM{se7>ZhbDfWmbdVy$020K2c*WPlsv0 z_EmUM=*Eamq>3p2`gF)pP%YZ$R9wyJ=t^jFHr7m|_+AdLr8u@E*;_7sK8bl({q}s# z@6ClPapx;R*p}(!eq(i&cG(4CwYj`1hwruev`PlCDX&^_w8e`z*Q!jF;C60*ZvXrk zb9t_*Gb{Ah0Hf2kYN=e}G&i<=UcGwu`4LlIwf8k&I7P2t*^QD+sJ^YgVS`#;x%Wki zYB6pwSOYaZnQb$8Mb(8K6m9-A+Do?LC>G1A% z=5fb*uiK}bFjLjx3hw!1Z@1S9oB(2;S1nO#kUdVG=Dyi^WHK3>E)XI$4xS7q(vfu;IjK7#_!TetpW3 z0kee$F!ABA{7?I}uGySA zHS9_0R({XHU)$rL@q=A3e#x2QzeEeFMaZM={}A}|2}8Pixf1VQHU9I9&itIW&g-_jywRx0=A;$E0!=G$vG_=2 zK16^;svqqI61OVE|73)e;dSD70t#86c)0VZ=Ura4>h*@l#46a~&hfb>nN+m^X#$f{O7frV zu!mkgW~2hJM@C7?`HhHAico9yHj<&A%<; zR0m-j@>RDw-c(a#gjBrS({%Z)Jinae6ux-*vNC#U8VuK13VLRbvgcKcfg;!b0zr?; z*&1H01q=x;c+5?7FL$2%`3+ScP3d~u87>-fb22mT>{H*_I=<}87Kg$` z6DKuCNS5;@%$k66G#Xid@*G-SnHM)oy4hu-*Pb&Ht!8BMJYt9V5kgmKZ-HB% zb)AWofgo(i+&&6=L-ro^9!&o5z&@R@Brn_^xM#+lHCr!=G(l=j(P-6TuR>5U-)s}g zT>NsEGk{8DzAtQ{k0p9IA^ilU{SCQAHG^v(Y+oJwe0zOr@Z7f!yzg*?c*s(}qhqpP ztR4R_{XQ2Tw8DP}_ZtiocS~DczIgE>E# zL3!opGg9!pp@sfPu|sQ0^761RY-HCPlgNBz5R)DEJytsmWRl_Q0Ks?-Ue3#4fIqh= zns_LYt33549rn<(UO2!NykMjrkX+AqK0jVtej(FqRq;gkV>SQj%Yd1^MsUr^YQ6L7 z7U;HdwM8+ zQXF>~Dk;!lu`+nY8`lR1bKn_3Yh=Z@OqxTDHgJB-olyQF?*sVj>yZjhZMX8AyWTt* z4@BT)Q%!2n)LTLKPp@H7!MLR>z&~U{1#Pj{mOWkA8awwnMw9OJ{pGP_RlfvTyRWZK z-o?d;YN2r4A;3E%CN2-F(*}KN!g+x!@$2*Y;Zu)zwN^c0E7J91q?=gt^Yh`nqPaO! zOptPbMlzQ_UWJF*t6Tu7P6!LTY@DpRcyEUhzgEaJv(BlEM9Srs zPaVZwrV^{n^K6*4JB%xrhabTVAZP(u@`3nrx8&vXu=-GH-@vt7$gj%;>K-nKcTT|| zRtC;$h!j{Q_3HyD0dj+V;PTt-6ndKuuatFpk#VYKp<}B<=j6$dFM{S-a2SEvB!m5~rs`h3GWLlNA|?p7oFPdYnGdoa;lmF>LBPc8 z@PfU|M=)%ISjm3Kdl&cGAYyO~2a%@&0O&IZ?!@zgmUx{Nz4qa7BXK$j_mBB0xpf6k zzw^7ar_}*O$TCVEz=pD6o2~u(?ai&c?kmHQ2v51`l>M%%BpW5@6E+0 z?JJ~b)(uw&v%y3~=7U|UhOO5?FD1!2!V!*M+fd42nOyRy`Ck2}jRjXApaR(^Ku$5! zAZ}^8ALgC{w#%(Mr~CbtLJ-HNQzig3iPad!t##{@PQ*!DL-1bfh!U?>&MbNjj469k zD5U0KK8vt|AkP61WMSwFfpg!)w@bm)4jgUTM!v9-!xppHT`%PGJ-=Q=Eoi|>ph`$7 z_GL-%NR*P18l^}?BPhwc@d(7V@7?gh;t?aIagtX$VBQ71NgEq3*Nx9WpbpB-+^RGP z44+jsYUMmww@|PGPc?BgCP2`SPt)%T$UFTgp8?1#08uGe>bIuZ^lAX^BnWCKFfb5a z*|kO3C4P*0#JYD6DwjN`P-L~_GXX=VC3;k=fB?oyS;C?M~1b z(b5p;)}~)d!(bz$`Y9~XxYe&*GKhu*40?sT;bGooV+k%#$E?8iy%}2GMs82ngXK@g zi3$0(o$NAzwDUxI%uMBc3nKQQMo`utU!ENS`$U52L8{i)t8N#kkmu)5ULyze6OP+A zHY<@VJKPxM5PAMYz&qFjDcr@0_K~Ls(t$s2 zb8g|{GSzr*k`8xz!gxxZ+VAv$_kWb-8c88yZ$*eyrKD zD##DaR1@LSjUfp2c{G==S`^yVY-DBF7s1*?6#Y&wMkFT(wxN_UQ{gd*?#aFGtU4>xe~h?34rLjcJm=0V__ToR)f3=&J*pNj;Vy@SXi}cV9JGOq z-Vz0mUYx2+odNqE0_wHgtRaxb{n)w-lE{E1%AScE+Y~w&!fyFlBdTE=G?@twZ(dSA zg1hF^&!+~5&b@pb7l+ucmL!$=INOlGv3SSjjlk4kz}#SytjhZ<0;nJeN#GmNojslq zgYPmwmmm+V-2K3LcN%LYYhvh@9#v-1*212j*Vzvq7Z)EdXp&A;5A-%MF^Lh;fFcNB znIYwtuD7NJp?jMj-5h;kt{N~C3~@p%iY;YXa&hNqHDd+`y&qm~3pgeiSu5;F1TM zK!I^BeiKiX#eo6Af^Qt^GBuWBKG6x|_7ZXgoF-GXZ~;WRK1P&O=HnRR0V@SjZjD%s z`croia3wnqo$E`SJ@fN*xJYAD6B^_T1eFEY>KBNe%KUdZns;i2AN(4)XyY*Q(DCE| z=qn$zFyTch?uEnOphY4eqzm9;zQ1hGHcs`P27=o+*q8+Q$(ai3k~$R@T2#gR`h8(2 zd7e=CBmFYg<F!Z-9|W0R@-;_0|M~%Y&l_!dh5o39Ur@#SfL5Ih zW&}YxAi1tz?`=ktLSV_KqQWjC&6U;VhIHPqaQxU|to~%i+d7~=S{Sai^4)Ou{a5~U z?F$h+%0F(Y`b~XL)xts{s%Q+s1t2{{5a&jAMzdhY9b~<>WU9aK&mVTqNjEp3}*OL&QpYsqr^*eNU6YHc64J(*& zGhrMVr{W)s`YIu@H(Y&ZP7gnzvHat1<=Jb8PSDn%FGn2f*6kr7Xn6ubgy~x58h*<7 zH{F=vbzZ)Js-uPhmUQ6hoy7u-9 z+G`21|Nkfc+mq>kDZcu5VE?_xm|1_mif4(ZdU#o58&kF)eL-cw$$%xys`i>F6y|X8 zkFh_Egp1CHu4cY!Ov%rJVmZQK7-N6D>fvATi;4w)NWoHaxll*`{0g7-TrYcf)_^z2 z#LC(WJrA`@Xbd%!bNFiCr*8=~ooe!m&p5TOyg4^P{Juu<0x7gr$b05}KglxI9=}BM z8Do+9$61BG&cm^>_LRQ-8X6N!ZJ%PuYve)Qm(2d=)hJ1-=}c6|yZ(MZAMSq@@lbnBSzDo+EVxU*BS>uiuGYpk^BaM;wiWIfQDWB<26;#5N z%ZC+XV${wGg~4_UWl^@QLSk|mJ4!#^aT3K0%9FEAqvTb}Pchy+;zBu5#^6*7^{ov6Ow^HNh zU1@a&Vb(!q$bCTXEwQXoa~n%+NKL1)`lVR&k;x;iljZJLdc;#hE@5p5UIBxbmK0K@zBcg=GFDGkoTn@nnOhe}CW0vn>7Bouw`|=S2UEp65f$|=h#xMY3rw4 z^woElw7##*Ia%KgzJ3)Eg3{4QaxYa}vX2Ah!Y+&R`Es*8}dzG{V( z1uh)M^;J-fOC{e;H^1uN^f0Z>(9CS*Z0^U8vj0S!Ias%-f(Yx0Vq|W~YNLKL&&}T& z&(6@6e!<2bdf1M~-S+3oo{kqK^nFXF2DvOh@p11>{o-8hCX~GDzJu<&+evGmd z@svIqN8(3`a|^sV+;~h+)Q^@NAd5RtcCMF-UYd}G782sT)$G@sfeHsMKrMi^I;SA^ zwz5*4IJ;(byKOQ#xy^p`kK25LcX>&?)%^UK#SCFHW1wUWkO2;BVQp=3eUQ@RM?n>c zSfCrc=>w@VyYHiy7|Jyjv!ow^A{A0`r>jIG^WoUYi>f_Q0*Y3J{uKIF!HF>l4){@9 z?;hEDVC{NJ$lV)Q!R7LRLK^)&k?}n?%*pk%9ECdS)kwdIwrYQh1*tV&g6MbR(HK&h z2Mon0Wij~1q{c{CZEcIb@HfG^1g**Ky?*KK6YF^mSC^l)Hqv%N*nKs!N_lY<43=uN z7|}9Xd_SC9jX1Uh6yCs;(jWDy8&hO&wnd10;H#65onD_B!X7ck^(m2B&ITjoQVfax zv$J&%3NHA=EID5ak~^d^*OzI|ogJ|S5@~jkRV?sxBq3xPS8c&JYc5;7^}EQ8bsymqNi@m%;UA=!vB3Zy4v% zVB~t0c2<=VyvE6$#EfuRQmJc&Vd7ma<}VD6e^I28)z}3Ts@TyZyMgTeEt@H7uMci6>?!K)uDs=_3Dg;9g+DideI1%U=Ql``8RO*swBH- zE`*oZFyErW08UdJF|s6_rG*|B7T3kF*{xj!3111t6BoB_FxmOhPA0#%?^+}-%p~-5 z`swuT=#!`jg%W-gC;D1mt-!|v{@9~V7c*?|lX6%sq8gPk{e6buhfJFf0kMhLT7F>s7SfBynLY3NFy-aOK{6Rh-qBtpu?>Dizc%@(q?R$0@@0R#EDxh2)rGc-r0d$-xMSh_u* zt+MMx+qpyYg#>?%k%i6yLZom&Uq~_}Q34jqwI8&Ta~Z$Z)7&eXdyp#ztGi=OMJuTu zx;xT2w+dY6-$Ov|tuhWL2zc8ytu4`fW(b9A3AcUex(h^yPRJe(UPmFF81#|9$5->f<>7&!*q!6gW=k z@BN*^=ioSa`+xq8r}jYjUsk&AqBtacq-clMzUfwOn!LRHbF|>x^}>E{kH032&UH%u z`?ouPdQtv&zxuzJOgr9s|E4?GL9bp^j8+-2UxsST*`wRTfi|RB_0irrR@&0P{a_rV z^n_~UOCSRPT>~p1jF~+R`(x|S90I)bgD&rnt-eeT32eK~C}kXR_|}nW=~i)9-SO_$ zp3vYPovM$fznAvnaSB%$+5OWji&IpS)rSH>Wt!WdDPIkwcRVa$SEFfPaR--~W#gE+ zNykui_Mq%0XeiMbYc#>AOR=4mZ#ozCV^Vn4E`?Za~yr{|My)tJg>%aMz5aeA~`+gA8psd#n?xDQq@!z zXOOLIFoO~R^~CWU32A~$OzP%+Aa(c(0{_5ywO~)hEQ<;;{rf{*t1f_qN%xNapa?j2~_kk*bkgf>|WvZA+X_qqiNSyG?D<8fP!XPrJdBr^m5NGe! zmb_AMYSS^_hHWhJM?0PL!#A9qTt`fgL+199+2RkKm}P*uoIL!VuzRUNbNV?CBiGy( zM;pmm4{qC%xOe_y=HmKRj;Itm<6% zhR0vDw3a^?_gKxpFt@E(lnLM#Q2sSx%Pgp0`aOYL)h8MHt-2;VGnw~;Hr9I?sg1n% z>$Rq}>F~It^#jH5tgh|hqCvGo;T;RR6OHMta8YU{L*zetI1o{OjN5e6_Ko{mH-A09 z>B+a32emqDtUTKfa#dS`bxR)hvFXfF>~zTykiBeHmk@wd+|P>Exq|I*6{nF9fRUt zK4dY_kmz3K)-zoXJ5sPT;36D4eT*?(?1)dL|Lio5QCX_xOTmQS zUzpAR(+B*=w66_`XCke6Rz zne&RMFFR^6`k@AgX1_xV9MD+HecU=a1RV5KNL4@BE(TYZq`gtH#jcDcA{9cUs6z8- zWA6QP>h9v^ty{N>osQh5-S+aPM|Cv+u{6n)C>xHoqm}?ef`0tkBfy|CC#Tl`@pl(_ zy(=+gS?{?2Xxn*>i}^z~(tkYPiA9btZIr#pu|RlT+*Or2YehJ^;SX8ryv9AdH;2#i zhwX3CohhF5rg$5#|C2L$bM!h_wNm!GUH?AzxrE5uhYIWetj&qVTbkSi9(C@jdPuNi zk(71SV$Ue?cfB^};KNDg+Vj%4?6c~HLK&JDQZ;TR{SJ7yQVpnxl5UP{rO!*`RUU^- z6yzmK(AY(I5DCnR8-6Btm$Y^H0ufC`YHAf2(+aYLfPjDlRLHV+|y2E9{w2wj;oV0=^xLdzsZQ zB{97}S#dB<%5n(`-5M~;IGdo!1ee_YD$DNe6K8~je|-T04(i%iW}J;Ga1x261KRB6 z5$H2V{s7hWAV|H+yFfv;OP(_IE)Pi=q9=e@Vhx4Ta(gqVAHA@@!e?{y^|s+8pO(p3 zzolApc_k&TeK%ZO&@&RMC%65bkmc9AwrD5nydjUwBcgtJQ+r3p92CHbFxKYgX2P}U z5BWF&$0V=Q_up*T|8T#qi7JbZWizQze#Jnx{N{weZtpffTb`O{#+qdy#=JtxA1!+h zmFsxFF*G#PegFvWn-8+vknnBu5(;9#!$rJmw_CcGjY9EdrQb1i_8OGINXYiuH$G2g zua3jvt>ewB^QmmfibF=KlNqGdF{M@iQi6ZQWLBbbN5mnd^1VR6yLTDhQ`sb4;aWBt z3o9`QInG0d2@8-#75BfI=vGoT6eb8&z_=3QPW@+xlA)Ha{Q!|Z;2OG7wa2T|qKaS8 zmyCkzxMV6lA%PE75kARoZ2c9gqtsnNL8@e|4A-`%*n{4)U~y2T#V=;(5{z zH-cFnFih;<+Gcalj$hY=Gkd%>)#FMJ2p?LYG;=x4=zX?j3CSzzN}+>aQ%*UTA0(C` zR<8JQhl>cBLTp{^v7~}1j1T>C$*Wo#*J7`LkPLwJsn=pXanM|7)9JPHX?Cxm_;;I` zN9~at`!3hIO2e|H?!5%VGhFuwERNv5Q_^veSR@XX`A9YWqYdz>6l#|#P;nZS7l$1LNq{oQc(=K26{Q4k1BGYn{#GajFf*PR{iAIKg(0Z>q45=MDg&1-+dZg% z8*8rogl(<*b|~?Cl8O&(E_N#Coz2am4J?=T)ch`Ut=ZZg*p+6F7AV9;zsKh1&%&cR z{FqSjP(xS-Aj*ZUniZXB1cTiwkaHhYZYoGfNFYKW4+IA|UX`i=MZ~B}thL1uiDJ})-xIk-9hB778GO9ts8@ZT6yrHjdWkQV^OhRU?y>kMT_DdnpaJw_x zm=9oYvm#*zfz8wN+09h5`W7ldSJ1^(jC)-qjSOlnKAo31>g(}QTImhTq z7BEz35sO^=X8sgmpvtjb`&_cBS?^-JoTI415@>)p_RZ+@gHy&x7f>+>*?xxPM_kE{&W>A z#BS|hnRpx>hz9dvVPU4m8h+pzc4%`8DgU|+n7~bNt)hQI)E-KD30d7;U2XVKCfOE* zkpwIn$$K~987z7))*bX8`6X~&7ok84pZA1izz5GaBNaIhK(tWS(tg0Jx%?_C%b{kg z(kiG^(>n~*$wY!CUW0|fKzRce3BKOa7;5NS75iRwIsEi40~PXzxv0#18W9i_6ooo4 zl)PTt5mPnn8kOE+ZL4G62g{F4fB_PPu#&p(Jj}}EX%2(+0h0%u9Q1H)M65j%_IwaZ zwnxTR1F#GN&|qf!P%&VaydoZ_MT74<9m($n8%cn@17>X#)XxJ4z3P)azi#&a{_TxP zy3@6Zw%6JZMw{YVQ?)`@=0;s}w(RqVjRdjkqhYLjmGG{GKyBE(Z|r+KGJC}Ev0-uU zZA0iM4vB7AW+(ewT+=L@?|46P1vJCl+OPW$KLe35jaEEJo3YC1 zbEAs0u<#sh49`ym+q8QZL7DRnkKB8cmzN!ZtV>y0;C!>vIMC=^KYjs`)xCcKB%%-@ z&|#TNE;nuut8-1378YkQt}o0kZP%8xsk{k14td9KPY;&ocJ?j7 zZMeW2UV(mRN(u@<==wHadC(xX64VN@a6sr$Ex2+$5Mqscy(CIe71yY>rAdSH%`Y4H zK8oL-$jx&?GXWrf0zwl8B#TLdhXr=%i)%oIlrUF?tlz{2BlX!@;L>eEpbs49$Z@u4 zuLDiLzirZF1c$@HpIm{Igz)N&!AsQf*>?5xMDLM1ZQ}rS#k7ylTY%>ERn}EZ$`0~( z2oDXMIR>iK^lB};ulLsTD8G8Ns8ez6Q{lyl(`2mAG+0(d9f2Mse(vQ7p8-vI?FT^q z^M(Zc-HlrZu0Jy@4?y|W$bwf*fiD*VI5fuxCw)(Qy#bTz2sLJ?ssBSxL3niOQHp8L zJixTQ0}>bTGK*kk9FNDJJ5<>3kVpXI2*1C?4Vc;GeV zkx&SrVk^j|T3i#5j9dU$4t{%m7gg`}ifbPNe1HLkphwVyqHWh89iR#yp#j({(F$zX z7rtE0hV5Bh%nL=L41`sva@`FWvOw>tg@B-0P2FegI}c85yPhbfh+Gavw-13Ez`GCz z7YruG(R?Gqsbj*jf_)c358dZ0A^2Mwb`B8l@5)5?UV0An1#eIQBqD$xFPNJv0Z$N6 zLovblAR?_whVzKq1)L@g;v>tg1axK+YpIk_VvZ>L8~%d+X^7qw&`YlMUcf0FFQ0+w z^Z&ih{35Vdn^S?D1MBNc5v!410_O~V=wnHVKOh63urPo)W~9aR-B$~SkxV9wqNno_ z+yDkVX|qP~vZonbtVtuLGN&E2A3$hj7p2fMk!G=}=Df+yT>o!$NP|QZrk4gRmOl#b`ut$gY!t`Di{Gvh-Z+AoEpuHe&x%k z_Tz)t3qafZZ3tTu3*i=mb;}$6L8i!Yc!L7HY9j5?!>|Lpm}G?F46M3U0H2V!0-_4g z;ci*k&c=kUj?l?@p+F7gv#x<)nhoq+$V|L1pUMG+q18t0aLz~U9iXtw<-U^wF7NXZ z#Gay&7{m+q7UApAVL{RkTRz(KQlO5_9^lp*@L>5|BjR7jA z^>*%E2BXnEsSxD>0%w4legq<3agPNV!gnX7vp_k9*MiSYd8-zQiRQ_vbngu{Q1WP1 z9*LALq5mePFdyd9>?Y{GFHFk*WK?eL%6zz$=DqIA+Q&I^I5&m4JKk?P`a3^-{+nmb zm`%usOF!&>|GitsoG$C@Rc=)=*!j8U&u6yu-=Q{7-q`SOl4kUwUe2}VTYu^U=QUpO z>Xfek_Zt@MCj9@+*qpBmLzn%hu+hE#Z=KS+e+~|wH`_Z>hW<|)%M2kS!;}^VIeA5_vY0bho=^4YtPSTC`~RM%0um@FDnD@MsiA(x{}~0x;m!X=-s?a4 z-2Yu!x;jWmPC~3^I`e@;!+)VCt4bH zL@>yRyh>O8`U-}x%a6bl0?mO|94vi}^d=_tNC(Q1ktT&%B@r4izfq=NkJxG?r$7ddFZ}*p0|MmyxtQeci zXP-!UYVm1boXVfUWtV)uoqKhGM>_j(BM`;GLz@Lvz@Y#LBU3;LA#3q(jkAf^<`Ab|NMA*_Qtogd)|k+AXsU<`n_9tf;ih;K*0qwcbH zof#}c3>{*xAY5<-unq`#2}$MwV&^>R2}vFbM9*NuaNpzQ06uj<9!U${?u$Z=Kzb|$ zXyg#7-Vk0rne*bTavKU4dBy&P8Dhkc7{0VpAAvXmj&JD_YenRiTmjLD}>AS{dW=5Wx zf~2Cc=5Amc!@C)PFDiy=!bO1r4QZrjF)ScNrvMps?C{(632mK%&o3s6rX zw6B9;DOJHGcH37P6PS=mz)deME|#hM^_P?iu$awYFF}itoOY{ZvC&G_?HWQ;S65f0 zhz30mG92KeASOP3y#Y4IAQTCA0Z1nUR=Ejs1LEL6Jkbj0^kJy`mA1zRfWYd1-+r*= zp5pIK*^n}D@Xb#YAp+6D8Pg&eez=wnV(=Eg&;XYGo*63YuXf8fLz3#GSvOb{NTC-^ zB&8W_JoyX!tUG8=Le{tS@st8^>5B65=YT^FcwU!p8JxACpkOZql&~?Oa4sR%{s>fd zSqv@U>VlOVfY7hEd_E_~c>pB#&fY%I+0eyzwtm5KIWZ4a+aP$i06>5g0AQhb1>_G` zov2^ikppo}d%~XhFNfJJ!-md){sYDVIxUiHMkvD z@L^p0U{-|yBNrDm?Z3 zS9&hl(lx&hjG2Y6nXZR`@?ceKyPl{9Tq=)L2Y{}<0#?MRUF_txV9;sE*5A!?hPf+P z=S-%P6f;}yrS*oGsLnR**8cI<`r$*3p5&W1xP&G&bs78gvz#xS4=0x+rUq0P3&uB- z&vfU(C*Pl^h_!k+JGaY61(PId!SO)WKmz8AhP+ef<9#|nl+9Y1ALrWFqQwrjzyr{) zj}n9?CGe+9U`G*A23YL^02}}cX!9it=Yuvca>?Q;Hvy7WFC&>oC-?R)XPh=R8VVi1zNyOEIr9Z1QO`2pLRi;Df0S|t z1bFS37o3ZLlvCUm1CFX&IXO^r0&e!_ulKnyBqeSg78wdy5W(NuCM6|h$8Y1?t}Bmy z?(wn9aD8}1T)Q6zNy>pp@JU|&g)qb6m;-Eqv%C8xvtpzS2-FXN@E7-;64b*U7FORe zuou?yEhDf!v25F4#+0;+bNR8Uh+UbQaN@bU?W=qP;n!PDX-@ypY9rkjic1x@WfxMKvWQrNSD6-bFYC6by&Hp#A{%j{$#x zLsv$jTkh=TMp)^`xoeOR1R~)Tibp7Z(V9%Y`1ST~ZEgYiGrj)@XKx)_@8YA7x5|5?Hd&APLe#mnv+rb6`}VSDWUjGAb&{ ztqdNjt{#Ho?+@oo_n3AHzDma-_HddL{f^WIkaz6T>Yzr3ImmvHs=HwS>zJ7_gymQm zz?o^=L^{WSh_IPoregDDFCzt%!P%0P(v(Wbv^Aox)6->0{vWh4`T?`vWy~G)HK3Hvy~Q%*t#bbTlE#3nYk5sL3u4{Vxt7$Ffo~`X5s~G`RQKV4^3i9_nrGRcytulWzChTRgYh+Gs)OT~lmy|$}_Zxy| zQ24>l+)3N!2{0!Sxyx{d^6c8(ry)n6Gne||!h(r}N#&NC?tmOKQA7A{i+qB8e=33Yb+rnZ9w!)%=dNL>1aU!%Sf{MDl%e+X?WK z4M2bcT!|vSqYHVh9*TULT%hXEK=&eC075#CxqGJa){O@};RPe7ApCqli-HL~5PF$V z9b&G3S|HCQ&J~xeBElFnK0(NKdzbV;d^tO~$!_eJImtD9(&B#>6~3>&rp6cw$F@z_ zR%y^Hh9bJL4(o$R5J-f-_7&JOcJ;Uapziwt=mhDGg28eT6)hvLYPP@9LUxJyt2HPA z23Y{pTa&PsmbX(uJG+2VUQf?GlDDp?xENZNAT1!JGbk@@Uuo>Nwvp>WD0QeQ{h_LT zdELvpct7R8%Z;?G!9P^TUWPG`U+NoReGr--Aya`s@&gVbKc=k!hW|t%1UQ0t3Gj+5 zD>c5Ybf;=2s!LbFkWjlc_RI;z;%`r;`Cz>RhMv!C28*D9AJJO=)1|z zsxmP_mx%yQA)QS?S|xkS%Qyve5Mf~G-4&BUArTE?MHRG8K~EYI;5QYR=m$AI1>(Ri zJ9u;JNwnmSa{i|S+-D8&s>xo?!bktD^qs9{!I<>h3J3J`>tkAEK^DCbY|VzjZ!jOdw1!^poE(;zu?ES7rPQM zm2TsqQmsOrTuNru%bQD$gm+dmDldOSG(;fH=>V0s?z+QS*h9!7h`4*-$1bHJ5eC9h z^VVn6rlJdA)e%wb-!hBwNVb(HFsHD;Mk!(*kYfE$ol%Q$VGZ-qNbWpzFd^C5Qo4EP z^hczChZx&gVd2MrV}sn!O$WE0G;@3HHc zf5TSa_i{SYn2&9<+hbJ5|JKxo{k=9aF3P>PI?!h$LmT&!MVhUqPd5^sxzX<5>oO)$ z#}(R{A$IDSJ~L*2{Py;D;PbkS;B4u3G(DXS?I0F4qUA565?01DuMo4G6@JOnbT~D0 zBrY>vcLrGYI<$OU#z8G-IDuv>R1dr^@H6!uZN&V*0wLumqIcTs-!vHk5C}@T{CB8| z;o}(M{Scp>0k8tdGY5XT4-UK^1jdJLmeww=Rv(|uzg~|qDSrO!;m_S48fJ?yp1nsp ztLf+0=_+&l^4pF{T1x8weqDI~(U`8Nb5A|xO8+|}+*dau&EZj zl>Fr&K`=27Kq*tyIX9P6E*vq50Ek1R>6;+6BkDDTX~>jLo=E%rHqaWW&=dHYS^)7y zr2RTj`tv=*`-dVGvKWwQHlRBFwhLm0IMiT}nhWa>uHZ32Yq#VFupXC`aLtiMBk}nkYd{%4ACM}i+6N94 z7kns(T0IKgf2X=d!q@N5El-B2hIrVRHy;pMzgo_di`4tM@%Zf zK1)EVxL0m~M~2l7&3=tg;KgdJz{SuC*^U6sGWNB1auJxq3Fj{QuGiFHT%I(6j8?H0 zp}S1hx_HwI^sPhSxk0LZ1VKyv&546k2i%tVa(#a|JQ(FOl;q7FzQ;Ur!lK#sS5*LpTu;s$l z9FM+0bSOw=;5*&Pe(5K@GjVp;r2gV2L&mnryaqFchc4Gb@iM^}T|=*BX-w+=b|0o& zMAyLxa;#uag@u7Gj<% zsn^_*ziWRJRyGKm@NP8Sje!<8eF(CD5+n7qipOucZKszTPVFEZXJ>G~ltCp34u;ad z44lVt2vGyOiWW+Kz;vQumw-tCkASo}jf-PZ`rI*?4%Q=hmC?mCh+hk-2x|uz#}ia~ zL1E!c=ey?wt+1(Kl*=3pn^-DHcccNe7aBenS@&}p06#1F?WUsnwA(N{Fd_4@hNlYw!&U;CPs^Uo#M zLTMQ;{t6urOA}uVdrn^G4Qa4L@18sV1Isd@QSyfG1$C5qpPKjSMuC)HeYW}UTQO!+ z+eE6Bv2SX=j*l&UM8EUhs1!6hhs~D7-?V?);(Wm2pSj=DHd*1?muj>w?Ab4ob^P{X zubUhF_OwuhBg$aq589!2K-g}eVYk)4E$QJ1iD|Qg?C>3K9CDn>|5`! zY8gu0tq`|~EG=+4|ESogQ(+S5?d}!Y*}4C+bzke|s0o&on#2s}X5kStK~&PuM5_v) ze_v{^KEEROPc4!Pee$xq2Vc$Xcwzr?A^jaU^l&I??Yzep6|erz_&1^NUil+RR{*sgh?Uy8W>Qp zqaYjYB(SHM6 z79ITG7F`N6y=;MfgK!N6@Ht5A3D!P9BtHkWa9B`k!0u$LAAAd7U6Bh189dkSr{Fq# zS{x)%V3F*ZI^jgyUBf9apH8Cs;`|mV$e<`WhoYG~WE{j%x~?OHKzk{DnzQcv5gZj@ z3X)*zgdBDOROvL73<%AqR2Mi!wP4M*Y`ia7Iseh;Cc|=NiZ8?6-xI+P^6JF~+P0^a zqB9%D0IFP-`zvU>OCz;AX;n!f_aevtysw_V;WzQab)PEI$4gp9rD8Bk6$eM&eop9Ce|zE%>wdk}X)g3q>%-ivcZnFD9ZI)t1uKsw4rOgz-rEVRxIeMYkHrf%D_b>L z%CZl~)@j*-%h{f(hw~53ceU&GLrL8e{Qmgt=HQ)bGdQ=MvD%wH_;_T>ytQKXJhV~XVA`(`i2)`-^4v~LAfq*tE6+*gxy$uW!HLZVu zKY@EOxMTG%Y;dDDat8`rKBKzV_x9Bfyxuz8_sDWok~M zZnyemrx-@bg$Jg#BpoacZPH{U`+luSZZl|ffn`XC~4n%0iPqk!)g^@W#JV#=dCS0WOAq? zq2=|5Xc3`B$sr6ZD>?t2{v8Xo4O64?bzHKu^p@Yf48E;XB)PeE*Y&N+KDbP?9s2Me zHE@hcEW}-SOdH9ChgsG~(%w&>*xyggfVE0ek(l?L@lJ-IuZk0~qoJe`mUv9e12*1-rPss>EwmkCnLT@rP4sgXNA6R2l9xwMlQTwAc`Q%iK{0v~J zT)5r?o7SR-phz&{aa!grD^cDwh`03Ss83s@Zj~1SVxVwBl#li9P;>8Qn`__ zv4p>WMY+psw6BFhL&tiz@Zb%aFs6&7ji}Ub5qkM6qnGKAQ@=3q!m-hmHCV${n^?lY`eubbI+{gi#*HO)g!F+A3D{*k-B|~XSw<@C&#>Y zbmx2#o{ict4l$knD$8EB6wKEF;)HKtiOlhMsHv%G&+3Gt^?BLKBZCJLR6_J~@Fsxr zX2F~@>brQ(^VdJpRuAEMp7*jveDG^oUW@TNUhV0qn6WM2W=>|7!_#xS(bBm?$J7~{ z7OxR?M`?4qbqxhy_h)@G3D+u=G?MnO`jEGwVfMc7jC?l{qknzM+E&gnI{8(yVVp{@V-{}S%PO_07pezZ(GiiE+Hp+KGItDgw8x@JS14<$XFM<$N#m>oi7 zg}A#41!T*5-RT?W=9iVzLNUNz1J4IMDGDE-?1b)t_2O+%%&vnz1>E&_i%1yk6)1hS zqJijJ_+zl^wDu4Pv`FFy-%6LX2b%}R28D&0yFI1-2^VB9d;-4cT^nftbp2u5w^YlB zTkrh-fTz{%@cLT{IUZwM2hqpxNgq!O;Sd)IW%`~-RP8AK)hCs+7qnGYyL)o{vfy$Z zt*mPhzv0Y6n%DMA_v%GLrhk=YzF9Fe`Zk&m@IJg~lzLq2#e~JiMXsnr5hwmrnzwqj z%F986(rMkoYf0uI`#PPX?|9low%*jHbmjDm6Jbi7*WY5L{Hhn)BS$6ze`#Gc`Xlwk zfBXB^@TT1w%dcZ5?~E$9gQU2GQtEn#g$bgYGZfe}G*(N^iOm-$JQ7|*5gQtMLxf9< zFmDC8Vx*(Nsm0mo%nQ|lB*yFDYn_;w0BIZ))J+hCvh&Bap#v34GCQ!sM9;sb75VOt z7?5w_Vcv@*mqWbJ^VTm03A>Pia>k&1BO3I2z0+Fm_oug!X-(iS09WTHHQ`?iK*l41 z95ILs-_bTR^It74G=(`^fG(h<&%0-kxrEp&Yd*p7)H^m;_-vWH7G*%3|BAFIfCUiY z20%bKUD5Er;#?O``WPTDV(0|6qW50gbjR368q%Er_i+I2F2&{L82}l8RecVHTE%j& z#qCtosC7kg@h|YX-vL1x@n<7C3K#?-mq^1@Nu>Sdcy9$EY4skOu=#Hxc55(%d;owV z4N4(miRNUjigLi~cm}8%Z~-vcVxTZ?0znAEj`Lh6($)j2E+PR>t>g_7;(D#imX<$| z{+sXx{kLl4DU(8Z+?l@)apTsg1nkG3b{hZOwom((PLq=99{tR09KC7T;^G}?w=Qby z+UM$cbZCEbD@%GOEZ1)4Y3=^Q68h7w6sME*k&g3|;R}<4W3y1w{tBVzi*fRHIHr3O z86Pve#!nZejrUeF-qktu2bSV7Ilef_DgT-I{t#y}*6-hoA7eT{r-~;1Uses%dTi>g zKA?_2JDvQxE@EPWrSS*!W~nsphF~lwYsxe8uU>OR-7x z^Mp}{u8(UBW-t6iPlo6tD?I1jU!rK=G@j0o6mCi%Bse7xdl}mMpVi_JQ$PCNav?o@ zNPDub!7H$Np;7EW+ooMkI(JjUN7%C?$s@gredXl_EhMZL7HyL zppDtV|7nWJbC3=E5nBvs&G1N}N96kN$lPVfaEQac$%;Xy>kYWd5sS#__PX&1cvcga zJi!V+02Tgi+mRx0-=``?t2%?aMnrvk0U2uqW`fn#RbZ7S5hi-CkROEzXP_{84BQDg zC@L+5@@o<#+f$G@{D8cG>gf;I1Wv$x6zl!$@AwDEjv%u{{Jm{Ezvcd{9}I6EbW4cc z73PFO1IAV3DogH1OVT5aD2?Vn#eESd{_u9>e_2 z`&oB`ihBX4k7jE#;0 zUn7rTe6V@ALZGfU=`Fr8?!1tn@-fqUbs=kbQm62A>R`Hg#(Ss0!^)xQ7-vaC;?K9G z-L$2|9$b=Adbi!?>J6{WVy|k;F0FK(Tkmp5(uMq`XFsE*JkD-fUO3O|DWPkYQXhV6 zT=G|o6gKu;pg%eN{deiYXCzPHYx(Wf3pJ^yU%BhSgm|~K%zG~X?Bd|JG{ePS?X;4{ zsGGp+&p=N2PmIVnpD8VK*f;msy59M>{m*wt{e1qAdcL6@optdMmh`Xnb_LdH%F7!+uAX~lYjkeZm)p5C%<9?UKpNwatABE{|{VPeRYuNZ5 zmFl7u(re?mQrBwUS?pF`p213GfxfPaRME6B6)gvM# z7+aT{94E7A3|?QK87i6U2sI;a^;x=^>|?Y+WRq1WeKBS@_ILD=NQ$@oF&9)6_S3-( z$E*Invv-fDBrYcZQB~6#O*Ed$n#sP(ofE2i>en=1(_Q;eYWt5VDZd;Z**~;N%UD41 z3CRST!FS9(P9Pn?zQB<=t{R=eh=tU^aMv;qJ^r?X8Vll1>r}-v&80!?o=EpS@5YrMTiV8(?}OC zr)q=OgKwI_sNNfeZ-=&nII^4dVbUM`oj(W-h~2(*%N>~U0j;8QQUD2zhF0Jld8P66&B8{*`xPEp1Day{$tP)TMfrQ-wItg_9iY3;2;O~t4IW|I6%KEqnS?k)5TZR2wd zC!(BDBE$90^c+gBEbt^FF6Pz;>>n#CHh^;vxur!3sLurmeC>>3(?`(W4(C{4L~c|7 z796D43q-~T((3$Z!2devKMkL}U*-s1W{}?F;;S%BMScH?u0~^P5}G{i#@L1h zobwR7p)tFfvH~toYk>7gdRG7NYLiPg_tpPtaE4F({~3I-hZuGiXa09sM`%!0$$zeC z`WpmlG56u^#Tf#OCfpF@1!>5ydzk;@mj$OP)_={r|Cj6kfBlDEjUpH|p_{m}q>0|# zG7Qj}z|S7RoJK0wP_foj;V_B(cU}~JXD)eVE}h8!xA}=QEO7H=PZ-4l12;J5k`^DT zR_<{a-u~v#m?+KWbXSrr)SOeHJ!@e!d6goMGa{F>q-2O&c35ukZP3S^&uIlWJauh| z2+mYYZ=XMEFVq$K_@-j5q=c-bWVOliTA2A8=<7(hr6%ws%~CjBNNre9jVYvsI;6m+ z)k`>FRc=_&r0WJ2#xl|WnvMT23Uh1+hKyoD_-tXlqFLm36q_iwOnCRMPnR=iV7ogE zp7KSznhX@Sq%vmnfRWD^O(Or@C_IQU+i6~G=lDioAlk(;=xr>c#$bLfp4fWMa&-On z31`G-iP92Va>bGo1ugP0x$slEV;vai5>gVush!KIt*3{lr?-29Vf)?=L;hcE8@X4^ zIVCxya(eO;Bx=LUcw4=Tyr_-<7rrZEsJ#bxaW8nq6Udrl?> z_K{5J=jU~?iI+Vw#tUL185h^RGp}MDt==_%!+=>RwCxr2=@72e2Y#auePdQx*zQpL zSz`(15w>QlQp?iB!;gz)M6tw)Y)A6k@Ktp|566DFKm3=HG@eaOpofE$b`AS_n4D8^ z+6!7c(j$%XyV5F@IWk`_waMcZ=?`Bu$>U7R`bp}}G1r+-n8(XF`WZvQb5Bdjg>*^pbTW{AxSBnZ zHBOdO!li?v`u zZ(y~&&ShQd%X~Q=^zdweo(>1zxTE z@%#<(Mc9V{xe)U-Uv>X{Z||V0j*}Hn;*&K*veO%g*7n0(;_GVc_Qb^;U}a)b6rp&% zSBP_NBof0G$Z0;uVHr=4-gvx{x_++7L!of%8oI23=Rxy(?VhACcHLD34r8j>3((ty~|} zm3KG1GY4+;=^k=^fAYL|m262ulhDeGR^>+bw%xg^f(^@^4-RrPKbEkECKx{iwA6L> zPU>e*qa)a_n4i6>&s87}h`)!gd^1)*n3J7<4)q7~LjEoKt%hp&G)Y>CZBs>h!S9ny z^Ea}jG+_fu91$Vr3V(Yw_1N$*H-vc^FpH4622w;2W0RCWo2KZ=lRi^-ojtjvf?X25 zDjL1i7(L%mp>)+bBIqU=R-l<+TT$?Vm)EUK%Xy*gcJ1f>xKwtB59h@2*BTRVnb>~l zwDv7t>HOCD*oFzVk%dU-EoJdmAruQrz?oJ5S{U^N1wnK!2~Hp-mh81vEnU*|&WbRK zvmeOwp@0U8fEh+9bu-(b4E(qLJYttHcNjXE+g<56k_kM2$z08+HXeI5pXf8TKi8|A zx#VfXT`m6i`@QkVni@^=FX2A~rv-P9@QvLgMBcVi;cs5zj%6H7{Cux6tpHn=u{5#M zvgQ_&Ufi(Y1bNZjNGr-V3X+umfWblACv>-iKhcYwd{1LbGNy3;5o^q7&?c_7^XT)p zIN6a3U*ZY=;g#4fpIjPP`j^XZh7#ddyQcslCeM%9`EjuMwF2YzS4MZAnLK>=KJYSA zi1GzR8WL5ypnoH~3*X2nr}jW2cIHpt)gCFz21+g>xybQ?*A_B%5!EB=ffRDHo{o0( zY|^pXS@PDwRiEbI?6#qgIWjdkz#m$$LVdR_OWv686&E|T{aTkA6M5qeEDGz~`}mEK zmv4SLqgzJa1-gs}>3vUh=$~^NDc2fSru^B}?zThUtaa6jXZOsxbY+Fk6E#v%ANaEe zymqKU%@wE#UC8qJEQtf2tB6wPHq`Mv^jH*&kL8yf7;_)vM-8~nOTPU14OeZLs6&%n zF+AL^v8q|daA=TlaITHCsbWp8gbV3>b0bBT^DSpYceqtOMsbh=lP;H!!r9e)xgsWu z=7^TUAbK+GrMdtqiolUJUNeQHcbHU4T!))K81SLBDv=Q{gcNxbDvcsps|y2)=G?<1 zrUlcPw!ug0-MHSTX(q#t$nN?_*Vy0PeZ&4v_00zl<`MU(1m>crI;sB1ydYnK|2)nJ zvlyY^KB6%nbYyjciRZ+wBv%K+SNfmI0?Hq_-|h@@;pC6SN@uFSRkqh}^h+q=FxgEU zrC}gmRa7e`EJ<@({u(F`Lm=~WIrACDvt(=oEg{r#a_zhFt91yfp$FcuXRyg$OVc6` z&G|kle@554UyUq!B%I~J`S*s5=%5AtExMQYXvL!C^)yW;>v}F27bQYIY;)AB#R}wy zCT@{_;0V0xph$PdJf_8aSUGevwlq6FA&}%!>iL2Lqr@Fz%3OO(CJs(2;pcavhhT9H z-76+6u)&di(;pTgXI2=ypqc;uXGw{zVeE53y+5+seK`>#>6t37HlY^zjR?)emYLaJ<*G-6?or%s zuZTtJAXN+wz_EA~s#32MTe!Gz>9ae1UzNk;bCZ{Jgl(JHvdY(SChJVbTCm_18H&Zsyhku9IT?^uadA(YWte=Xs01_F zd7||{&5bjFo0`JVYo%|#Ao=9>v^BB8(bFQWquhqR+t@S|iORVyMXwy~@+lY2@r@&| zV%+8O)?3h6kl&J&PS|!cSz?8~Ki|2GYP_+d_t0IfWUfR<4%_O1W;i(k_azJ&f0$_$ zKoP8ZU4?+@=JotknjgU@cZun%)kF2=^2c}xX`JBf^*o+By zT9LV&5xN@_5Vtqi(h9sYD{V?D*65z++rT)pl9DgBO#eZm{Z9cS38bR*riKKCQ0|S5 zv^1@7chb1X-Lc1ui*z}pUq&C&cYSc^tD~Q+Q{ft~n->*+c}9QJ2& z<5|2t*9GmyJ>lqH?Rv+3vAL-?SYU1S5maIumgAcA(17b&*Z>D@|4UOOiMC^WifF$p zsS>7t^M&BLXSpv0T~~BzaZ{Dy?AUV_sYgYDSg-tu6|crEVp;T>SQcm7h_ea*Ym;@z zThXW}r43sB^~2VV*yT{$Cu(MC*Nx%1Wc1?p#XwmtXPM-BV( z|F|Su7$hggspz6dlJHLzHVAIVaL+V3WDSEgVa@WXi~0e=ibXn-kBUyw(s#W&Dc0&F zvk>z()=M(HEkR86pkLz|zqWXKyo?!az$XUsg16(n?BKLpjZIAdR# zx(^E)h;`D_-tQRf!Iu3+mdMl2e93bN&b^upf53S(oiD2rk=7AQugrS~33b(HoGO7G zu?_ca$wQs{>U@~xr=rVc_@R@t7gL08VOc^^OOc1eC+j%6xxj|&UK`5p?RAIPo1oli zzKFN6^g$0iy`B2{(Pl_Vi5B^X`b$wWP~W7%*~}hIj(>H+rRftV|3w?N6Aw3@sIypN zFGHwvIqS^>^+9dF&-EB2mzUKR+K8U^R$D&GC|k>1T8W-tZGtz4sbF(;+*8Lf8LR$v zUpRgL%$_tvEAeK3en|-_nJ4Q%?!jenE|(xvze4_+Gk(0p22=mmJo#Q>_4OJgI~a)d zRRuBTzsPs93k``0LPsR>}B5K2WdOd|a*03L7 z2Uiehv`#I^;^o+AkuD@;#tp*TS4MHME-!DAM=CvbMHl{K?sDeo`@U%CBGy;Oi$0v! zXL0;Y8vjjsi7O=3H*L}7zQwfD;LS(1pV9&gSKex{a(4yEKB<*KZY7v;L7)AeQ+QuB zm)x@4KjsH4qVrwrZX!M$R#TvY0KyUp>q=UID~V`4LD=u3HWdr~Q0NVi!}{wA2j#!dTh41F z%XEH_<>pTeXe!Vm_sWD>DRNO1qEUepH#<#3e+myMGGVy3Q z3@Ry!V9w>dt;SMOl209Go>p$5Z*hqIF;^9d8DfmrWQ15I<>S%WwPP-%SZ$k)6-sG= zP4Bm%Db-=JchP0Uu-(12A+bcGb$kIM|Mmtu#&q~p6+h}SdVuyYb7`WkS24Co;mul8 zlVhhNZq!<)$OnFgd;I&^9vp(8fDHW1CG91lf!zi&T91c# zL4&)JW9~&(0!Ne(Ql<&9Gk{f+5sKr??iA>eX59JS$HtcK^HZoH7bFzY z_mU@sV$*}zL-Ma`1^__hV!*@E49$1Nw+@OYYRC%$3Srr(bt^GF z{)N#GkAQ+0>vECh(^{D@Vvbv`!Pwf^(emp*=}H1Ri@#+q;Rb4m2TllkXL6`wH_I`?Ip~xRBVU-p0ss|k&x#roK1+LRQOT}+A&JP za)ScPeAI>^Bf0mxo7@m+=SO9X1liD}-Vfu(9VEChZ7}BEI3S2j!NG#ff3q_nKhryWvk^i`$|VYhSM8RNimKmPs{W zlDdCa`cOaIh_B|kXP$P+l(P7TmZ^7vV?A7$E->DV>k=N$;6s(yXS+tds(;i8&EJrO zzK5)6*R#me2fLhd-yX@vyF`?81j>Gsf5VvA3X zuT@3{gQx?lketF_c6A{ivv@d&W8dk(9-?Dk)sD&da*{DY;xe6xzRWvaj0XXne6oK| z0!zGHg0V9n)y{hQu|LuN!n(Rr(A7Vq@b-Gf8_r=zxM6Fk68H3SJxEs1cc70-b#V<0F^t_*Pu}(Q3kr;U2&8FBEudCpx`Cyq zCsv!r9=iZfCJ6%`dB=P8S(hmLlySJd!s3srk-P0Rt3w`rxS90}B za^E87$6r-03r>uopQ5pWp_9hz2vDPY0l<`L$d(BuG%z2wd}R1FGsW3tKyV$fY!Ar2 zs8_CvT7qhMYd0g9)qU0TWg!%CvLKI}#SZ7|A>msTFRxf%Z*9#&kx=^5QD%zMN|_qlHnqktr=FW%tEpKqG)yxxbZImu6f(>fFKyezbuPj5aOFPyeo3>xyFyx# z$tv@i$r%6rt`Cjdm`U7Qk}3Ie#&&vRf*HM>TDRq=j@1@Toh)Xgpzafa7#R_!Lmo6f zzI01fq#}LQ)AY{w?n7U6y+dSkwZVgEt|IMgS4y$vYQ{cOVhq}h@5)fwOX?JrY5QoCLg)wgnTc zw46@uyFO#?Jbk%;vHPlk>ST`7cj;aS5F({;EEk#Ey&Ezw`gG6r#?t{;hsj&0X+br$ zVS&`wHZ%*uuXB}$-8OL4WbbiiW5E&ut6t>R{bx2Ym~ktZ5qX8;PW-1jhh8UkjiC94 zslQ`d69>gghSaUUDTwQN%E>*k+hQ-=XE4%q-8QXc z4JXO`-9WNKC4iy}C{MZQpA*I!&JMD4ecd1<6?d&z~^nBL35$U{rOM;@ckJbj3VC(m19@HjjBn&L_2Tgb92O|95R#yyNft}i~U$oy1sRXVy9hG6L4ZF93c^n$= z@i32zN=lF&N@iAGkjp8X?R?dG0n`wgO$b$7GhA2=SyOLQI zI$Y5UCwKIWd6(LddJ5=KT>?cSFKxWH!#rbJki6WbY~RQBEl!F-Yipc(fWW1IDxG_& zxU!57VFtTX4f!ivH60-}5!AGMKmW*L=^MeY%@evetR9vDld#L1=wXjszk-8xL^{s< zX*jWTR+NuIVJcZK$)Gtos=P>))PX%3(O@H0<6%_kD3&QZMP;KozQP-(lKbD+9^fn z7Ga#bpP(y)5CF)J7@TCi@7eg2;1@xr(mMIh`6f=o8d@%s#N>{-5r}hJgvAj*av1xg&AiN`WVLvjF0e--z z2%z(O<>h!4LeJ9Q`Uqf8x=Zy{Sr?SC1qG0YQq>zjD`v509raaO#l%Q@W#8Ny%lmA9GKPyfAo^Ol5c4-0-qws7f@I*i8k3;^2%L1w6vAkdIc1?;z8ykevn!DPM$+frO^At;6bAYHpU>^Q0AxHG&4NesF}tu z|B=y(Nrj|Pmzf*$O0597YY#;`;<(%Fh#(mJUEklV?#sn z)n&$fR$sQ2DMOi#+LoY;7JAPQoS${rB3>ay*W|)|ysDOd#OA;z51c(Q<+Fmay03P;PTscN*}ks|&|g0G z>YDCG#$}8>VODC%fxL*)oh)TuQn%(oclm@tplnbb{0cehv->hI&R%8`79dlOn12@Q zGP=;ED0;L2DlGx{(GhBaoJK&ha?#KGJb1mD`_ta43LNqDt> zR#csbf({A%(I*Xy!Erw5dB#63n{*i24|AGvX%Ee>tu_sKPy~(039r3j-**)#){-Yz z{8g4+#M%_tO-~z(f*SKbYl5Y2f(JU!-AgL03oeuv0tqh|`(=J8(Shq-S}3;ga;Cu*m4nsR-Ly!Su&X+;~z->wJC+kaRypRVw@2WJLcn8odnCU6wBDQ zgNXOO-={Ck-ob%LjP=IYuU!VgFr|BtJ*P2TJjmbDe z0whr%?6-WR-vjpNUH$#HNnKJJ9T&Bm-q(7B5zrRMbFG{AMo6}Twe8UyFilXSmz2<8 z1)49X6^>d=J&xCdb97Aw+Xpmj$lA^U5r=X{u+-#pvXLWXv%yt1a_yGGfqW+?p9P7(=pO zjkn6QVe2XHC;CXs_pLSkX=2BgPxj`3BRzzi3^{Ve5vO)%V|c~ZWKIQPh3crT6eMhg zD^&OHRp#&*!B=q)ngkiX@9}s&-jnS-%}R+H@e7e5uf z9U&K|iluRQz5#Iz8v3sboFX^Ovc>%Zo9R95mdWEVSHaDfpZeAxxWOMdv3ORu(e*CaM zRW)ZYZ(QoiAyDQURey)U+G`Ll(E!DWvyxRjQ9gBVD+9bol4-h^7Y}m?BkhlMn&q|3 zrK5`UF;q!OFAG&f?y9)B3c= zc+Ps6-l3jw_l)M4I}YQa$cBG0jfXZ@=EY-b7_P2Zt5~B6lda7#85?z%9}_YR3yMQ^ zeA0*L(hciH%#{9b$F2-8leIL!Z5pN$GtJ){3sH`|0gE2OVxG z;K&upddo2}j+tP^lEe140OZr`1~@RH}jjUTBylt6lv(YbpR+>%6-_@MKa zZb8-$xc&L?7W(jUEvZYNXPn1PFb57g@Fw)cY6L}A3>6|f1Y;1v62!6ts$Nyv^m^c> z@N)pzTjwAFAh6Ej7gVzDZkB=VmtOz@dRLjhvzU-(-)hU(tg~GiL6)-RYf=M*@@iwd z+Dtzx;M^ROAUF4{qPmQ-=s0fk1iVjeN_z43z+v(Xxc7u1P@Kto`w1Wv98%Q|LJRd+?#eUi zs;CSLHs;WNOg8t2NWTQTwEVqvU;?IR5Y9{yUc80Dh)ugY{c02NcJt%93PQZskdfRJ zigE&-K}_H%7_Ko^Cv)ZmCa}sN>*{quH379?$JZq#Vdly2O1W-J4Q|?>_^|)M^(Em`sZwa0 z;Ognocm3XCUFDDuW=tzsZBp++$21wwv~lx11VYJ+7!M~lBsgxlm|D@?SV;(2 zmD^doeYTFsuYxXH(hzN?&p#`QU<>zbWkfi7FyFi`hdy+l-j@`3KnPN0!HU;)qR5am z_RsuT%ipA?dmC8bWaJ^YobN}N_=!||Olauxg2sCun0s4WQzIx+j}TFP5z)!5o^N;1!<^EnnZOuU6pw{b^Z+i4dm*~;o26~ttS8^760j1W`&B7dtU0>~ zL}CHpk;!7uq2%NZ+yR4(D-`rZ+JpeX0Bw4aVwqdVD}CmF?Xn5Se-FWh7+e%{qDR*q zM(o>nPmL?yy?YlL6el^L#wxPQ4HbR1M#pHv93vao(JTZ?3dJ1a!PM6p(J+>M5z`r- zkp-;#gYFKQMIDF9+-t0p&n06dQ6NQ<$1CLA!;{Cv&?VxbSiq-|&xvI7uwXw1Q1X23-aKC7`M(oIq&pS>I*bf{69>+bzIZ$7e0)X#3Todl#S6TB`Gj! zpaV8K1*Ahsky1JZ#u%Vp=b+ zKy2z^Am1_Gbt&_5RL_4I#J3dc$Y2`em>&%55@Zv|0p~b|MA^-{LI~{JsLNcPq*PSM z`M%X>sBgWMXY+cG7D(2>5+SH|9LTv|wgjs-R8GLtEw_#~!{1F!w@9VW()@_diF}qD z#s$0!P@^20lWxBLMZXPTnvR_aDK@XScV|&7rsZG_b6M%LRdM30X9uCnDVmM38{uRk z%hwi6*(s>bwkxB@fK7iOkj}?CZ)WL%L?uF4RGl8I0IT(pusolY7$@7nm;eBi6M89F zoFI?{(Up4e&@PMrv604?_GbcsQPF2!aAlSb0NWJN{B%jT88j&m#9sl8;%%0MKwr-Z z@|mg>gT&$25~Mg-fN^g!Pda823OV$@M_-*$gzG`;-ernFsCryLxacWiLCqY>K|&q5 zzyvUpPRzfr<)KUZOUHZ}N_zA`&*O3W6I&Onvf==>0abdo<&x zlg+xk#{<R;c&1 zfFivxMg#Zy_WZyBM~9u2fQcoSzxxf&k0WJVxfa53z12=oOZSuxvdL-`e!H)fd1-wj zFA|WaW%(cR$V@35G|~L2>qRMd?X&gU`hmI^^}hp{?HoIW84daUs1&NovaZyU1nI;> zbtFol!#|#X{|2yy^S8>7*$K}$)>gIyT^{ghu>R(>uyi^03-1sBXdgYe@s!!y!(n_c zL-}6@YEFQI-YWGn5a^u>7>VYQvEm%t3 z!4V1bs~b{@Xaf~8+o;{QXaGI~5O*01wZuYaNK4hg{ANOWviBXKt)Of)1SZQxs=Tqv zM+;W@k%N|DU!o04#o{W*R3}*Qn#4H%{Ni+!l4Xx)eSO7F1mKqD#}4^nAa+i6#sG82 zD=Y(!waP&ggpdW0ftnP({n#bbe#Za}@_A^i#Bg^bs2|04-)euMVS~VKp801;bEXji z>6#GIIlm=@C|F;3>G@j3B};pK@`p&D@JM|u|I2cFVWSkF!?9Gr42C``+0e)KZ58fX z#^n5;pT3GVnsC{@QdUcHkRfYOAP4uO4yA_Pf-^*dhnIoNy~S7J1_KldO01_*)x68;l8=Z)!i#4WVii!DdV!L!`@6pn&mW(!pa+;K1$yw}T-Q44HYmn#_g#!P zT>ngt4PZ2c57)!`dK~)xF~EA&D5!OF2j6IA1ZBT`*9y{ahkS#lcE59zF3?wDzZ8Qr zkrq!)#r;QzMK3$0Vt9fb_&4x2yYk#YoYljUym>}wF*6pHRh|Dr+9nGNWDdd{gMC5y z6j!LcFHNnZJEkp0I8%jS1loD858!z$%R@j2^rw3N~bgW_WeAci(w4k$W5AIv>qT?vbH<8lAEo zwPg!S%je4+6qD95&b1z0JZv>$!*z^0EVQw*kT=lD4N`>1|Tnu?} zk@2%W8htF9wz>DsadeRU&)XjXtJ9TMu^*i~ki&97Pq7I^z80#1p%z-7oKj|rbu)EV z7*-B-G+qHOac+}oq#(AHRm}7*U&q?>s>r75xpU9h z9bf#qw*D4ugs^b|A5A-?y8EJYW%@R#IDqpW3*;mB7BI#MrhR({;P2~#X~Qk}ugldt zKxGWlU3nd-yLc@83w`lp@5)gLiZc0F=URd3L^IVtuYpHU_E`c2OXCjEj`c9P>%6=a zLKXvUvPI2T?XwklfWuWUL376-8(>-U8C{?C8m?O3)FFnFohmR4OO%~-Kq``FC&+Q0 zjLH6US{>3pA86YFP_>)_UKGZ>u2{jF3b;GRi3H-MHzk4WV zACMyzdDLgEsermGFZYqu(!st@|1lLgB9+;f-MnA2A)dx#{~wkG?`#&bT-FBay1Lnf zQyp-R+RrKb&CWqH4A~w&LF19VmgU2a@I(A^7=D^(|RDe`V zf&1k_Yv0$g40(Nt*fYSxnRx?qg12s&X31=;0$nz_2;2x#-Ip-C6jI?1Rusm(?!SL+ zO@>Za`lgCll1tb17l4oY#@^{5Mh*HJsQS%>C(Vu9A~mM5V(YU`Y(I^-Vqnoch_--sHL}4TMh~!G0Fe z>BK-26f7SV22h9wOv0Q4k^f4X8ehq|Gi~?2%sT770wxo(~0ifN*Uj_%#c%zoF2~L(5U0(ZmDpjt@k^rNH4QAI)c_LOsf| zz6qTJ51yijG0l(-7e+J6RXGq+vZ2aEBdi)2OgkvIk9j91QTHwq7$6Ai7F0MiVWVg5 z@F?zC4Ik}9aJb049$TO+czBWrWCV;u=h}s$@!gkZOz|7t9sayYb@gf0TWepH#zEdR z`+$nVLXdnxoecYbVisGl5W)`52zl|f3toY*BBb&n_!Y+ld&y>J7Sxc^4w@V*)iMi5 zRMIM7NhZf`mX?-tGok#Q?qKuh8DMV#j}m`k*Cy%m+!quv3+G5;wf6{UBN~kgV6f?6 zsev9ZVCjHF>*jqj?Rw2Ni3I_{*cr?yIwq5I^i=W<9R}5C~);81q$EW1wj5`SMBT97XLF;Iq!iH1JSg za@Xkn73B$>c9(+gGwH?AC&1-RK27npF>FzId#rKs%T#r3{Yp=dH;)6c;@uJmKL1|e zA#w$e43ae~z7Zs6rutM4f(M_YaF0b0Hm~pY2(5KO~qy1db_Q z?mLgq(J>A?0*4tZ*5hyHZvL%GtlR7ATRT|f7i%Ah}_xrzt(!*fUvH;^Zb*S71UNF`fbZwn z=+~Ue$S_cl1{=A_Z}K-W7%eZj=Rd5OhAT%CAo68TDvpPq3zD7m=NnW5@)fGM@~rGl0SVW3eA68w zbN0Dz7Q+Wz|5`=FWM}C`=_uj*e*!El&rvFvI1a1F)}7R|AY^(9uEHEcC@9v!DVNnj z#15uaBpB1#i`vLg>->d7_dm$p1Zl|=#^r>OPL}z0^Mr3on1krjFYk4B-kN|FF6sy( zsEEigr283g0jx$T1Eahxu(GK-Abe6A73hl5NKtiX-rBvUDOdMJ=(JL8-__KXcWJDe%I%{`;VB4%>g@OflJ3g)g`bhJ98EAbu7d z`#=x3ktP$83|aC4-hPU zIEL!GD`GB3foC1TPM6e|z~>81egaaY?#9anv59)FQVc%8F-pYhsnf_9>Rh}zaj>jW zQZJEK;X`g2s}O@0TD;FVOL1sVzR;pcK6UneO-_x8P{&R){e=A*GQ3rWLi^IzC+>Bw zfG^FY5jgo|2dKQs-{2_i@-Y<< zGtSWI7GM9IhKSRybOjK)4^U^&kd1bNgFxT+)!en{>sio@c)Ki_Nhy+(+;wNxKXjYc z43K~D>CRI*UL;0+dDOI{1{$Zz9LQIFOHslkOXwx)l2N82Rh8yyHzkOg%-#Sa zUA=|O`m&MqDXH9~j;SE;D{Tg^`v3Wm7*f2HJ^xPQEuD`qRUtYK)4W;W;cQa1Sy^QG z0xSI*-nY^10%KsUtN$Ec(kKKr2b-CYpX&Eu;Rf=G{gO9Efbe#mEejb3RIHeE4*DB-LtWPs_MIeKfbqLG6r07%xwm; zU1~y{?(UGq2d-r=AQt8v699wUWDi(W=*|pXHn+rBa=^s|Mh-NX!5(Ana8T!6p47J3 z?LJyDvuvl zZ!?B@rM~J<4chtq!XYy&(fs|mvu(0Yg6@Ww-25GK`8BMqi51CGfv%z^KW=I{!ST*z zah!c$511d+7r1Pe%x^o@u7R*+aghbWfmxx?-lUh{*fZ(jI$5bqJjW{Fyy~r#RKsqS zF4k7BPj61RQ#7O})_-H&f2aJs`TPE%*LjP+*a4DE)w!R3*>fV7M-S=WJc-O#mR$d1 z{M7?%?C)ez&`kZ9O&=R!&S`Zvq}hwE$jKYk=dqqzvcz`DXS%pKrVYiJ8}VR0C$9+d z-Mh(5$+FM+SEJ<@3@Awq1z{2e3F|ys)^8DMU0o7&G#$i~ce=v<#-p6{LQnk%`|hTl zQxKK%2v_qD0DT3y7flai`gM4V-nI|Pi2fxCjIR4j)~Bd&yw7)LN5bWL!oDUTDcyYu zNBZ-j?{x}f>Weqy_5J(aj0q~~{2psk)H!eT0CSshF)6dSB+m5pP;Ns=L$VH(m*(T* zw3CBqrpVeNb2Lg}GQ6SQg(V?vg%J(~Sk&P07EZd$>7SMU1>RFX=avREf*JAU`7`Bm zjh;6fC6XGq@5k)14a&JG2x~TuC>t+AGf5%~^6!)jGrU=~p0dXuYf|sY+d8(=0_t%)FpSQ7w$jn1+C#4SPPj^syT1|duB0G|FM0p+z3ah zNv>t=2Im`bm#@0QlDrEJjoG7KR3~1)csmmHW}12~w|aV7QAf0+LTtY_N;s?hn>ROg zEN)ZdZhN$^czPNrskP@o+CK$tzfsY}s|%y;I}eKY9;bJc51FlvT>J6OgbY}#2~3IR zEJ`nxwfkrIZjIn00>-CR`R6Boj-;lL-(DfE`5CUmkr?^9TJx&fExx33cEEMrH%G{H zK8LH3G9y-s#J80Q;}~wo`I+mSw$y1BuOg-bplW3ar{9NaAH~?&=m4nrr3_Gn$b-(G z!Vxz&cS+N*N@ui}{_S?4CjeND)EJ^ksNw1~1VyLzL~2GBK^~(h-aqNHH#FFAhQk7*w|Ll{%)qQ)^U$-^9pj_B>#hS4BI#)ccNRiK3>m)4Hq|e!Kumkv|dYDbe=8nugVSJS2yvOVS)l9BhT?~i-{q{ z@hXQJE=xB7K@!bXIx+i{(B6j640UHSlEEy>s)2$z{pR`F-JL4} zSE`!R^^jp#qZ*r+O?_ftmXADmEr%6BludN)3imj^_svGP!M({UNE{hCtChU2? zcV8fwJT0iMgI<=aYB5rEDisat=Gzne?KDg>@iSr`fiDa`ql}@ZvM*B19`~4m+Y~L~ z0~o)-^HBRys_kM?S%Z$sJ%``I_Pa=ZUf#QX)wk$#TcpRH{8K9esztH4lv zv}3AR_`*AxEZqZ}YCretU{88m(P0@bHu_BJC%Ho0aZ!)vqJFkN;EotxM0=60hzKoB zralpQnr0^@ZDZn%laZ5jn5^uVkQupFhw!uC6x=7x(i*bTNmQ<&N3 zzr?pBR4TpPeqq@JnMgD0>G9x{@$`ktH4q#?x>-@ak#L$<5{+8({niqSW*o+q6G|nG zU9Ok1AY3q`nTZ8#P)kOCMp}k-Q)cIM3*u!{sGWGve5-}mHP6&5B9+8iAqxxX0H%xo z@o2zIP3ToUD(&UD*P_>;8wnI2m)GDv5suh?7V22<0`GpH)pBiG_%JYI+~n3~%^fr; zSlR-%3x#-8^Skym)-)Bo{`0Z!JDze1jfh?530WCwxmnR$hzb$HYQR`Iw!t#O__%_4 z_#d|QWoHWQ|ER(B2RdHFr$yp!Ke}3>V_$j;FZNteG9h_xX83|?P+MwI$7P*ujW_q2 zC2pl;4@zu)mr`eEgDWPv1X#tKE~b7^eezTh{xUA-lYWz5Y}K(fme?k&*FkdwxmUoq zMI857*YYm-tYXp=f)krdLa4PqqL07(VY|mk(MI@ToS3FA=M6qngeRfdFK3SbC0Dl9 z8h`4_8oR1seRK0Y({fuEm-Q76%`R;N0h+8decGEcEL!DD;ft+OqtozY_;C|)OzZHW z*m$j6dy>Y83rr6SRQ5X&9sUOc3w|Abyubl?k{tkfK!V`YAJL@+zo`?~%`5v>A6hX; zZZwCMWU8;I@V)jdgL|sp&-JHV@WY&nVEMXVf0j4>wrElJMf3*2;YzB(cXh6}W+C~O zmh12C>IEG3lbuC~N+&6AQ*Lmxy9fUwO76HK9v=^a{3kJl9w)Ng$BqiNMp6p)H&}Q6NV%twos?HO8s1G_J$a?149|964FLaBBgd8+}Q#ZPmvmsH57OFKOsY zVo#6SC?a>Y+UtPrT2(52mE3!yjz#zOXV=e47~HRw){W(>Xs)#FEzmLQPrLlu67R^c zC2wXSn$L_-ViXBlUwgh2Jp-(Pi)piG~hqEi;TD1%=LYT~g#*6+Jr^iVE$N zgrOAzoSjh8$c|=$kr!=d9pj+bYn$M)CmFyakOSpv_;r@HOAe0njFb#b%&U|<&r1_? zPrclFZ97uk#|ZI@BcoS5#hcGaFHO>~3SRg){z1k*uT5?zcIcbdxk1qtHx);>MEo43 zuRz-8e0Q89WG;p<$5{9yRgAIZ43yH33m9RQm&V=PMPBJF$Pkl0*oo@#W#lLer@y{H5(sFOG9u{(ch6m4@QuJEBe4r$ z_KOh7jY2=D200+D@(}etNPko)|DopzZsMSJE0*V2xXi?}RUWQZ!3_np5T|2-#hsk7 zEigHi878Nlzi8)+%ij@`;e5qO;Oi&+UAO%mSR(l~?g0Qugfo!&li*!{ayk`is`J`}+DiI`S@Y|4 zX&KYdTMd{;#cacOBK@)OxT<(o*OY6;TA@zdf9e1 zPI+w=QX^_%0Xmc&Qy?bH$s@-4W+WdNGsla)u`}sg6Bigcj-EIW2iVg8&X!2=AWK8y zV>k5Oja+X9YlwcDlxc|MHD;=F`rvt%b#y3aKfMQ2hg9Lp-DJK!dn85SMz2$iWIoDq zo+Qx*_=PcGCwJrcnTp@LOJBc`{dOk-M_^POq1xL@325Xreo7mSB^pz{3Dx|R2Cvfd zXanJ;oJ68LQSXc2mz7G^ntFc@tULx|oxlRLJpaRl4N>B#Q%kNmBkiyF2UP2Sz#P>( zo_TzJj~I-k@dhhToZuPMYLuldA$%juVEAhdjV_N&wuNG{&YLC8miT4H8zfT3x}ats zuOPVur2UshJA+=8V@)2N%Wucs-wkub-BnN)TW@6CB6%_x8LP1970r#lQLuf&cT`p_ zUmq`6cIHRI1u6UqqDL4mNK8Is^+B@7LbBVr|5L z!&sb5p2?NX_{n5i6PNoe8pg&wk8XV8{2C`#k&<+Bf2Y^6y{s-gwbW)``8N8``6 zZ6$_T)!hWTh0Soo9-8MVk8Y*>e2}Bb=Kr;NSAOLcHCSjMa6E6mdU|FdbAvwIbKUwx zXIcO-Tx*)^>@fv?DFe46KsEP{UNtA-R2A#e8v2r1hUNKoF1@%bel`1FliaLrQ}7Oe zfVzEqrApETdM|}(uRdxK+*&wRH?IN`>;E~F-?yP7ad7-Bsx+{DtT|byqofXRMm~2i<9c~@=q0$w~E&XtYsbpLKwEDbPr^jpWA0**-@K`0kIc}$E9?99{&EWa?`|f_^d-=I*9=&UF71l}zcG(PhcD z(ih&s=pnu{kMUF7T59X+GDFlRVVtH~2$lOXk2Iw>&IEHk0$q1$2sVhDGhkRJGwt() z&oYh(h~gL@aCaS04E|-uZo1%S&|+2zl=l-XCa~TL!9uJAqSwnjynUKQs4-_ObkRxi zNJpJARZ zpGO{@rSZM58V|kglndSa1tl{&lyGWDqxZ#PX|r9cN;dGHcKAVWHUV0-H;%TCPOSu_ z(DR1IQ5_TT=R)l#=Bz+*BF0EINfzo>=d#X2ZQ{fC^4DEN@tTmlzolNfh+O8XBCaroeO@+2@b+nwA4CVtgf45Ui zhPTXvDY|~iBTXo}G#&YCHVVjD%4A1jAMU_1XerjkAv*WYGaRzD)}2`w``_IMDZ_Do z;MRS7f~p=|4@Z0-9F2?Rh&wG9Mwff53Z%{y+bVn;^&Yty#@!a{S9cc2PP=gJ%~q&L zwRuoAIXo3CTLjmJ1h+C@e3 zUxJHH@sHZ(j}eGCeJnIB$+VV+=XSNK;&>x7sNvy8#Yb}A=sJrA@gr`tD7_2JR9}@C zlc-U29k1QTwe_#fbtU2UDwKA%ZuWr>s5-L>On5p(cgM%MEF5Cz3 zi>*Qcwbq~4eo-~%OkNFdsRk>mgnZzvgrRWpHapw1)w+b?!O6F8!PM^{Yj&gCv;7KoH*Q03An=hiEv)q8nI zW3jJJ5}c&btfC`Irn1#$w1}= z(nR2}irV(;21wzWBk48P5J0i+-j{OBW4Hp<2b($l9D>?Ir!em9>r>I#fw=5^@1-H5 zrex`w1P|wYeH*D%+s0oxa^LmK?VJ(fZ*1S2sG5m7);eV)hK85G4Yu+|;gxCIYK>V)Rc>xD?@TNf$*QAjL=Xaq9WyO!e>!zX(P) z|DkZCEe9IzUkEnGUQr7sLSe%XQ6RjfAkzHr+NU`!OCx0r60rJqG;W<>tRSj_J*%rH4Cn*635Z$2IHkWx#YHa#e2>4omyF@|#oR{ccaA z154_;@gZNR{=3^cKq<-N{&iRxnoGTPzv`6MpNF3+)rIF zjr$H3s>;SkK&tIHIZq?XiGD0~Q=3B`qB@lw3?~i5xrOrev>9rS{2(u1YW9 z=2ROi0ji&kvsqV$44Nb(zq82hoV_U=*dks~b|p+biCWk7qIxZN+I9P0!;V_J7>iBB zR(d{^a_)#Du-~=*&2!Nb+@=MQ-#jR)Ad0@QE?OfgmY4_T-$qm#kFQl&b?ss zP|n_UTt7K4-6wS@#G)dw@bLhyAT~E1%$y^fPu(o8P=)yKhSr&*GLQ1+UlbZI=_c_@ zKSc>7q0{1B(=n-oSQT^q+v~%Uw$!74ID2X-mXuG@W$4={9mNtidm4wb(XG7?HaW`0 z(j2TW;>#$Mtr(-Y(@-^ei?J0>(R6*PdbqosD8m1k=n6QO;#2pA7RQ4Y*KDD)P<^;U zHY5diK6@`IbGHi@2ViaS1oh&Pyg;#BHy8E7lnK3}KiX<~m^$`i zc6Qe7k?pVHea_Jh!c{cFsX-`mSt~os&*EM%g2D9Z5DnUsre7iali8>yUJG?&OMC3I zJmYf3mUgOwS;?k7^0ph9q5Bt3ha<$JcyT;@Q(Wg`QKJ}W;*a@sL}4I56r$lFo^|yN z_^p3wC1kNv^&(>MdL5?m2_8zX>l3zZIzX)e~WILSfFjx z@}!<1cjn9(idGe*pV;NGypzzoJ$C8r!)DfA*&k9H^i%WK?H!WIPaU<>^vf9=jEw>w1Gv*rAZTXjY*oeiJWg6Os?qveLH-6Gohh!*%g zJ15mUG*1o0!JBA2zk}l}eEhi$`}#n=_3sd$TG#1JI)m5R-+{Tt~o2_(-z~_FrisBGA zGk-{&$BeenpE%#4{neUrN0k9Yfzv^n1+pS-YUh}_9ypQOACP0g01ZzP6kPPid(F*UGKs{53O=uSeDUMNw z_;V*i;e}o1BQvSNvP2*q=^q>OpB&AordboV0$V;LZ-etNVV-RR0*=L7Mrj&!a5+)(7cx4J6%za?@{2VM)ZO zcLvSIyq}pGXu=WvG4TIHu%k~V_gZzFo<9C_T1gLQD=1&x;hLrkgmS&ezGLd-f3HHQ zKqu3-##%KJ1U6uV+idSl)L9A0UaqdjhA3k1w@7PIZ2A6OWQ~pmdc{07I7|B0uXvYT zf<)-2W(T4vuW>zKY4(wL4>QtGlKiQgqXCvE=EJY}DIKWd|ADO{(lEC8;ON&6KAmH# zqWgU4G+Zu);Gmg0vK9+Ovw02?KlvtTCc~(u1k9Ko0qh|CcOv#-Hy&87uDv>rJb)ZE6o)7N2=6X8g`3hB}_Fl@K6{_bAIeg9`fi zGmxr%h^|fQt+9V9Lf;&OpP}lzrxHs}-ec{OB7Or7Lf%9(Mnp%88W5W-8@;l?eG;aJ z&o+k2{CBTtd-O}C-izSq=xBfchmXxqt3ur3yP_U~t_(BANk|p?d>1t$w4%BSC1{B` z$GAGP_E_jd<@sx2i;GWN(>^NYK#5H}1s}L>ZG$Xt|zj?=#+lzPge2mqQshSY_-cpf8bOckH(*5gU zO}`aV4Lk|%&BS$NAgV3SM!UO%Cj->3$4wG*z>U z@f#$+*>(BiOAqi2{`lsx?rWWVdqTccmX_|OnY(X|8ffA!Ly=D0tXaL1ICo2{EHK~V zk?oP(FzmCN**J* zvl;V-Bs?`ejqs&y)fszyFqh8fgD~CfDY@Qbm?^X`8Zbt4lT%ntS`I9&2Lpb(9Wpb; zt#?et#Q4@L^l;AvetfGwuOa4#Ga(e#bBtxYuvH#rfM@$Q=EpQTeCb4=O=;cMcPR%8 zI>zS38Iv*(rJA(aIwQ!_RLGhmeRk?tzaBFU0$)3>4c4m~^`u5CAO zYiq+hJ2O*x%F=ly9pNznjhfF@wLoWArcl^3GpH~&PXg;Nm0<+|d9|RPX+=VxMA&i_sE;IM%=C#oZYX@8T77K&0VvR{_Ce}7haSK|#j ziSb+}06l%m0SNQTICJ{*Hx7-5kxZ{y^A!rcLsX^z12x@(`_N^=su*$l&4RDqYlH!l z5!>^)?32CRIoe&~D(Pkfm<3l9DY(H)ojAV#^P%70QSNOA$i)lUN7AsQ{@`1?Fzus%fMX^n9bTzieH>yv^231DqAr+*SxwB1KO)kX?5QXtofnZ`o$vz1xKnn2ad> zpSfOvM(`R73#4>47$JUWNIh^8&Ou-O)G4*M)=-jDxG>1_yqK>-4AG9S4VdgSTj3e1 z`1m=Co#TJtI8QeZL9F&g@$T%BDc+Z5M1XeG{^Od5YCz&uacu;@#&}$;F2P2QG+zAo zDE?TaH5o8Rb4_S1EKJ`gLBa@} zTXLC1qM_1~JIRVHhtluF^8Ojsg6IzZ#e^J zui||sMT`%x8F9_6f2$e#Ve%d>b5De%2s~2XuFGcn#Vh*T%OW5C0}XfiWPljz1XG2k?l8h#_T zZw_=lEyrg%PPula5k>TdXZ)iQqdpzzDHrDQdf~&sB(0uC{Rf;CVk@_h-NQQPEg8da z1c6ZyKy~>7$YI^d<+1p-3)&`hPEXA`c9(+5Kfgv*G0~v7JDDp>9Ea3pxoH1!vyvJN zLC}hos2$zPrH|}!TwA|i!Me1y>?Vy&z8>bNicHAUfE1~ur-_swC1@54rUa{94#`Bg*~ z)5QiF>|25CuN?bTvrn{8Uj-$@|K(5QYb}4lIK6ZluiIfb))l^d+4bXnk03tE&ggY| zHhOuttF5+1)esuB9P8MeLhm+$NS6v5fcVFt#XKPZqEjwk)3utNG^z!4atcamF7<=* z9}cOswTGC(*=0Irv~Qy2(UuY%wq||Awbj!IwK{NHdp`)ZFlY}1BZ6Ec#yTRyntNQr z1=YFT`i(?6ioLOX=oyaUXB!%&r9u0OiCiL$P~q1JMh%IVYPcCp=j77%J#YVi*wtIV z_0H4ND%E|eUIuh>-7xE(yqiOHgFgsO`_i>IYjM!t0?Qqueo6=&$*Z#Y&ble4FW&yF z_m!2&K(#fAq1n-zaq9;oT*Y4z4(nO2w_=DkELxD7Um;SxTBA?9RIw>96E-+yh&&y( z9a1xssgkfZ!`#BwZ!K;`Pd^oCpy^9c50nuOq3o4kJILWTM$s9Ei99tQNX}znTkid> ziMjAZ&)xF(+f2|EN<}U(rD1MV7SJSWi?gX0LXYGE9MnyT^+`hw1ViCOmdw?F3%9$o zAGzBG)v`sx^ycgL4>}K0Zt*IcNBq=s`4aNwrC;P9`gEieDR~ymJA=^Eh%wUIDh&5; zwkE?Ny5J1xCAKlG^+g8VRzMMY&=yiiqrc`rpVa-8Epej;WPhu!b;jkU@(;Oty2hSl zA)ZG(4`btq1Il`t|50}-2<*e}=mgwdy*-PC``QZZc5(WYgbxgo;+PfR& zwaMl2tCd%x0o7fj1?Zf){rc-$^M+%jEs2_;iB zTKpK4Tbf?ph(8C7lTF3=bE&JjHsL6RVWG-p7=bv6G<5}sO1 zOW_4ePnS;=4%t$9Z^M#Mha>!FC=Nr(Pg4%E{t|35PW`r1-*W3`U`fYyRb(o)Zglh? zunU|P0@K#n3$kH6zT@tzLMV_%p0r-bTt!wa^%9;5rm1ophr<>FWW_Y`heb9D4XgkI zQ-u`&ovtW^Gw2fB_-%~II&aQ5`!$0Soxz{K_<>Xjfc&YMks5H^%Qm0JviHb9rZMAt zp4ItRo;sF)CCpJ~Q3f1hQF72w*+bQF!4IY+XXL}j_B0fmrR1cT{%i2S-JP8dlYuHc z(=hy)lIyeQ7~$lmFwUbpg6(pm+4fIK!=B)e48K74+1{aw7tgQFZ#jLNzILk3+S?)6 z;X_ybJ98n;IT}%guiZA^nZkK}_Ngz^s0J6Se;MgVLu*K43{{=`Uq^6o>_*xX1MeI3 zQ{DV$=oeLZBRAg=Sw8J|7npTKiT?=wK$$HNpMofQ0fteK^Izl@KIMp$HV-s!Hi~YV zHYgh-Ljp}^OG)Pu>J;gHB8`+y<*H%yAPTk6s~r65EV(E5*OIyF*!%WK$Y_*AA_i}d7P}#ERihG@>~B6 zq<*#?^9CXX>*5MBRMZvs#mVYm-sKhcdQSRQ1W25p0X7Kn0E))KD7~%fNK0oaD@t#E z8=@GE^oP9y_@_vs>^a-c;mp=c(Uika{b5NV3n&m2w&JyoOZB^v$F9!nAq zor?6x)FY$|WxL;R#{SHA={IW|5MkB_I9-Lhn=a8OYAk-;yInAOHrPhX-}Kvm3%xr( zRqETBKTf2>bvE=f%OV5Nod$SmzbScRijbC z^3-vb=f;}bO)wN1`Ka);uk`lxh)3AbtL^a5MH?DA_=%0~rcxVzc~<1`)wwT3wcIT5sTa~`@B7W1xOnaMO{exS zPV;%=wzAGwE#-Oc19j{I0Y4fqGszFfz%S5CvkqG1*VaO4EL}usDMFa(U=ntcLJ9sZZ(^f71hNTcQL9UF`79CUT1 z%d&?be6&`%_#k(~;!kSOYkvK`+-rOE>H?J|w0pB<|~V^>}0!wGWstrO%rHnFsMw zRZoOSjU-Yn|KRiZsvta&?a9wG4nJR-FURj*HrnQqEog-Z8)6>!pP@3{+~~{RHOP_o zEh7HZiAK+T(>6K8MOwFRu!?uTPUTCY;dd#k$e9e96}v#d*A~L>V6I#@f^d&Z4pcJ8 zSHUj~n9Y679?iZrC%h8H9Yw2Cc5^|!*}RZWZ|+|C&t>>k)#pMG%VVAtVn5JrxlFF^ z3IX;yDW<$4QQv67Vm(^$$Bx$bJDp3;MF0<)g2+EdpQn$>T0!5Wi;hc)$rN61A&Dw1hyF|U(i6*hz z#Rn<|5CPskqjIO?(e{Ro^7SS@N2G{Eh~H)|%)YGa zrTO(E;;tVy>Wfv0vD8K3)D?wHJM1O0;fQ25Fcf!5|~qQl1z0SGM=-g18-=xa+{?-~2I}(E;;q zgOLr&#hRbLFptCM`iq!~6wLXgk+c`rzM))=ajv(@BRqvUN~0%NA3gb=xaDfE z!o}Nt%WL_Kx#?^5@BFd|0XnxPrzfDQo13{GF-D`-uA-|gePXW>mN=qN_I%I#Nk8L> z3RjpVyAY(cxebi{qe65J%1H@u%dPl1|Kh=G)wxr~LtwLno6kMn>S|kcT#X449etrb z*0>J6Vgl(>P0%+z57by%By>V2{~>`9QFnoA-fL_<2%AlqupX&VS?)BIY*PDS`tV|F z4wFt_MeQ71=a+(y?Qhkzio_!St5JhP_mU#mbes&MkK!xlLdEwyA4ku2c3(+1?!{6} zvcsKgMJ$gkvsDbydO-!ouHQlHW!Kv{2ld?FUBZ!_kFgpfPUrMh=&~K1LPzzCQeT6q z_sAEB#fyXRw45+^<}R@6qr<<%xr`Q-8hil8c(b>6pPeXvM0m2M*X^6SWBqL!cX|u= z_*FyE#2`II>($o_gWHECR!p|(GaPKiXj^UFJX`$ECr;+VTF=C}Pn1et*bSA1j&VyV zwm?+(?efRS?_I)~jVjYS+CJL3u(a&xz|cxJ^~yY5HhcI^%O`aWnFE4@(uQZhqC;Cwf%K+WMzY?i46i!_CH8jGAo+nc8X z5~m5aBL~FoT_L5tTu($@>TCIR>my#b8!@IGxrNF?>hFB6np<r>Qs((QIS6}>OKt6CyHmwF(UQ5J38TnT zsmVw0)Sje7j6S}1vM?*%h-JGl~xqVoe{ts_KlN)v2$Agd3**#w1tudtQr_1`wOG!lOWLCOY;Xc8goZ7( zIrT^In(#Ka9!}tFduRC()vQd4+Q2ie?Ld*h?l<+CCb-wSWA{~q*svrix|t)xOStMk z!>qJl{@4d%=rQ+mN#Nf3Wic}2wjgG3QN{oblz>We@Hm4FSU>d0}{Rlzp7l z1QBm5qITd$8lsC=t3XLtCK4)cCB&9m^(9n3Sh=Tj>2qpL%vFW$ik4H0{h7tOsW`PG zcd@bG{|{Af9Tw%jUy$c zqJpHtR>}ZH_gUlqedk<<%gaA>nD>2Z#l6;Z-ch&3;a^CcSJot z2$#E{p^Ho^&tjdo5OO!Pr-{Rmu(cqVtTA@zK8w} zjs`J_Ot)9d*M@>tZ)7Kv$1nQ{{t+E+VB$x0n^{_}9zGLQHa~Y*Ubt4D&O}a)(5C_~ zdG5>ON;%W$t69sD8slOg)~wcyW0uvjRrCsf5X$OuxD8*=+fJ+90Cc)r+R>=fr1s9ifySNdG}=qz_Ue$h z!EKJ2PlEm@@%Qbe%I&SJs^f13788}5q>mpSjDQcs%oZ3%4A?U;B;U%75#C&i4?iBq z{bcCUa+Iaj%ezYZr!CSo)f`K{WYx<)I}vzA_e|^i2PV0fxfHy4Wtc>6F=cYTs>f?B zzh0$NaKQ=&>Z5$?-Ci`LsRR4_WwUR;1%>BBiX%0AmGx61$q<%Nk(EUwQfcRW@MAPJ zH)jLbg=GW{0H`I_Js0H&jHzQ1{JERd`VTl|-R@U#%SSVD?lJ9PWi(3vYt@=w%G0|G zdlCZV)Ya~59vs@EuOiKA_uUvmG6q844b*VH=4c`6zYv>R3CWAjGnnu>CC+&J#gC6O zkx}_(YuNirksp-quRh`!HDT4^-N894%{5Q2m*>jsE#NO~GQ&3-$Jo_|I-~5tS(R%ypLy3=> zp;Pg@^_@end&*gWy&ySWp|yc z_tqMAsUlYS-3_ze;A4v6l4ygYE6p0+wqJnHH}v5^|B zrrmDduO1(R)8B;62J#PHe=u-Qk_r^2LM3T?9+s^sRi4v;03hH+Mrh^pd~wlt7ecg3 z$!aeNJ8R0WX^kys@S_}41vC36>2xtO7swD@7}B2KbM#wFpb=B>a2YHmzJGFKes0dE z*8J^*l_xre9nC7ddQYT#*u#DwcHkH#N9fK<(PFnP$|^5R)~SDc`^(jf7K=0p&xq8puEfy6OIX}7n?aRB8xawoibT-ny z`g8>$sLwLI&(H|us2)kocDBy{2}(lLPhNX3g0UC*Bajhb1-4(T`?b) z1hb^sDCQ=CkbkOY2IWz*2=zvikJGlrCVO5QDQkPHUmJE;oCCok+|9P}v@rr{*@yUo zC$dmgx+xO3M*fNGu%e)Dmv&I?tN1=KQ&#`hO#E6Ued>r+-)8XDc7}o1v`HLCZ>8(< z0vOYWVh!EBUdJUbzdqEk_~MO;|Mu-0qp)l0U3=|S!xP;@-HiT= zN2uqH?45VSrbC>lDFT&5G7W3Q#kCvwm6INZ2l!!E{?%{zIDA(YRsHDB4J)f`PTUB( zR<3N)G&l5$$W06*4xujgcl^D*XFQUFZne|!_=x;DXLVX@FrVetX|~0@3CewyvxB!} zlf|BPWL}SV_$(E~=RFhH_0&8ow>r-^VEl(vvS-Qu=sGxh#(S-!x7z(rPhr}b3E#(K zuuD7zs&Udk$_pOn7I3_rB>nV8NQ9{=F^aYA&e|NQ`sf6>j5?$W6=8b4A=la0t1JAACfsTjl!!?- z(mCI~7E_|c`&3^{(ZfYBCMwIxSS*O<+-)|dUo$zaZ&WJ_^WQjP*HnDUHUz8U!quct zpYgfXENK>1;@2-%DpJOq{V^(_T>WZN@<~b`4*tLWj~{bP@CB=f7VBL2kv(8m=)FKO@Cx>Ys&ZP{@K)huijlP_09P5WL1BIiH}kJ z>miXUisCxm;ypa$b4$uiWY$#RXIF(+{O-|Kd?g|`!>qdR_$&)*YgcBaHvfUH%;o$5 zJ`ZzZQhgz=^yj0HN6QQb%J0b$lY@U3_g>uTu{~n--qU~Gelc?og8>rdMurGqCmwDk zT{aK%us3bAo4hJ?{a;xZ^IvZd!=2B>Nqe@A%yk4l?bdoNllwHV7kk=ZFJNi*w%cWi zsar2slz()ee6JsT-)-U1TA*yw+k#V2bZzi=BNa`Q_2zV?sHawYo|SL%jSqWQ zl-3PWC9fQzJFc})XJy8OPclVZ{1#w!A){3LOmd)w#$tlTt7$&Q%84bCR+?wbA_DYEEx?KG(&J23X zlqlA_Z*ywuvnL|2-U(#XtTL-;Pvu~`pq}`A}r%;cqQxOViQAkcm9B8c3_O~ zt^HNHkl3i?pRf0GKZ>%I^PVer56fOI)wUSWYlM{DxXWkMG^~ErfxTQiKcBc!6>;s| zkWl;D&!n4IN7bwPPc-!8uHMrna?b84+obF^?A{H}OK~VbGSz<+%sfy?p64)hY)QVqWwFVuB*V>m^1 zcKHgz?d`wsSA917;L0X9mo_I5jAv+LnIXi-kCM=6dlm?mpkJ<^jZ?-9yM=RPJl0 zti}cUlMM{^6*G$;QG)#ilwvd}CVb3wyr;aZgnj;@GA){p?`E;U8_r0C4;n5Ho_kMekSo{ZltyHZnq<|OblB( zw;*4+Vn^kHjC7BH%>7;`GbuV=X){&+QhX1C-%OxtrDN(;*jTo!$#usvKAFf0HSLnq zHRpIzmuofx&o+Eajbz&p)6Zo3q5Is>ee;D1J%d|4_LSRtS2(MUYn!^H+2a8QEGfJs z16NBBFYM7~iSHuBT_@65eQIwuu}?xXJ$ZlbyZTz)@0@?wuj&&!TV%_NH=ei1u4LOw zO)D>YYB<|D^F5p`)b~kOw0?e~N#|`_UN*Da4(>Gd0z0V#E_%TtX1_A8rc&ryiBeNeGW|VzBFQ%7 zn}-C8)v~|Sd?vIkep2r>pJF1Tm_R4;osqd)E88$dKFCcT)_3E5cbL!XH(~qhweQ{o zv&H&&>FH}<9Ge4%nvNd^#7T#-UHIPgCV)Ltmdt=HN~%Y-eByLkHgWBd{<&qxdnk;X ztXIG}k$)wcJ!-$FFGL_!J4>pm=>L6mW-#iu>f@4Rk>q4%x3zvtF6OKMI1y#awL>e! zZANE4U<<>u(nqCQ92IlQhsVn<^Sj0e-m_6QubvBr#)3eN>bISoI04;PGXk*#oEJ9D z4$XsynDeDfGLB_2-Z&KRjpjr+?`kETrxt=6iEyZ?xPMV8_v2@n_l#*{56Q$Elzw%*GX0(tOZN))s3Fw z*zw#G!U6&psp&BXT#cNPJDl2|6x%WBH|mM^1!zhnkG`^UoEUjlrQPOIBy?7(0j zGoa4u>iyM@e;gan&VGoCE05yW!-V=pJ-&7r-s9$%eJQ8dw!U$(Bg-+u^5s`{Qv!j|4r80(}aT?@8PkEj=YD z>SM=Ryez7MkSe#{3U9L9hO5>VVA5_J4>wRN&v&JDR4be)t$X(#6ns)n=l8S}nJ2uk z&m_~Tc@bdy*bxRG)*2Zhen#?jwdLb;gJfpE==I7JY^$l1ZzMmz)3f+(wex78x^lfX z*tyQ?`~9_7Xj&zGVi@K;73^yL!P*`HmoTXI9sQ7yWOSs;aF_R zBGVepul@N?eJ9->KJ&yo348OKe^Ymh!EZ0sZMbmt)h}6jB)ufr?JExiYE#)}tg=`S zBt?GZ4%wd@b7GbRX}pX=8? zGihjcNxvCbF;|uL{gB{jn02~NN!wVpb8yvUMo6$sBjj;)QIVITsF@vx-4nSI7VP6< z-fbcEDA$l0C3s7Shw_`U*v&+hl}h~-w)_f<1kRV2SP);eNx|o6B#^}|GBomeh`guI z75%j~e)?S6@_y;y?~%AqO%0{`Vi*2&43zStx9t zJ9#?}_3!$Pd5dl@vfO*#BB!1!NbI&+El4;>Jv{}Ta0rvhQO#Ipmt#F+ z=d^>mdWJ(%+hRZA?Ejz+!j?TtJLXybg%cX)qNVreLR@}yIc!opm86~FSacHX>X@xR z(S5EdO{EW=E{AnNM6zI^{UZF5!Ec+=L1$Jn3Q(v>SNRgl1-^|`U^vHx8<4YO1h$^w zTYD5It+EhU9T;!X`xg3`lAUt<7~oSP4wfGkHLvWSL|8@w$RtQk9m1GvIL->!)8Yiq zt}=^z)xMKLd`u$61F+a!0`e37I-Gn6^=nmk)Z<7`I!SsG3r@cLijR8(T=b~F>eJQw z!da^qfoLEZZ3q(+i<&xkflle;(CS9_(T?8#{)>q$#y){}5ct>C^DVY;Oa1T`fPGZ8 zXZ2Tj7op6e5F-bh78a;rsv}Df91ZX#`To z^UFbHdbtX8Fda3BzfoFgeW)Q*GIW;(-!dGDRDs_U`p+UbNoV$BwWtPw4((mmPvvyd zaM(%1kxd}$2l}QnvFGHUXIxD|R0wOrNP?ZxtIYpA&(55=&pX%wFI{J;MRj=V&F%m5 zw-66+4Fwfg@aE^or`lSaq1pP4ZP{=-SMd?zm06S{i;mpE35 z8U%)tO~IXtA9Z&KPWAoS|L=Pz2VvZ1JHB!Pt&&sJ*-v3YOo8ygzgwZ%7hf;_RME^} z`5#Z1*Sv#)3@&`4r{Lch_kXh&mdKm=H8lXX#3f@w4O(4ZD?no z$+elj#Qxu*Dgt7_VhU=0LUUfu#eEo$)o7Hv(>xsz%8+J+inI^mBU*>zw;c2nfxN1# zDUZW9F4dEF=Pn&Wv{%neM-ElFHAWVL2Kto)H|9Jea>U2+nd|n*rr%5lqcWG$;`>x?guW-=~=l%$Wq+0llK zC)j*_u~<4yOrCq~JLCUOF}xI0lfZHOP*cE(KzAGEn#c!K@c9~3soc(SA4cl_B^eK6 z1M>fD3&FeN1^zw_1xm+PaF-PgY+lynoQ>=-Q7zWKx@Y0N2bUxqhCTtdSg_HU89iXj+d-- zT;0*A`UBJd1DU&xeU{s@9p(8^{aB=%P>hNc)aY_0#{Cpm zqhUK5_pvL@4yf)}P1hwyKE!E;b!gJZyjR_f)Bu~sK(bk?ed=+_cZy^uW^-E8`*6$B zxl)%!!6RhfnGmX`nr=Hd>wK<#^%}|6YQSD&y1*p6QkXx! zdgBbO-7~tA|NM0Xv)YB98)_;q1RO5TSQJSE8RY-8O zV>J@^ZVp#;UsLrWRh zB~`d&5o9>8Y6zKH_&pc^mHWr@(gVi`-ga#BYpKUDC^uD#za>8AKh7T*eag~V;`0dJ zWu8R@94gfjC|)^2nni`r>|-hB@k=&YvFHPrW05SRa%jNIR-xFP2z(KqMW?GdtZX|> z>KIcv>))ruX>(I|RyE?Uv|(e>s-DOH-+3xaT*hyS&{T`Ol)s(8em;nrq6eI+*oj-_ z#3@&2z1#%$X3;6qEh5|~8NkLi6PMM%<|688RD!<3}IuVzO|5H zok){glEo4qkJfa~nn-!nW44b3GpULf{d^9N_wm4-f(G#=sIl%+Y=Wy3gowf<2P`+^ zo}fm>K6$vRha_$Eo-OS~Ff=qZy0n{O27wtGLkvf+FJY+FUi`Ow1||I!Al#`V^e>R6 zZLNo$p;Hbtu|5#vBYA#S-QVZu_pq2$h@*TqHv69^X^|um^g4OP<6=syr2ORUz7)$j27p8SxK%bgfu8FJX&+-Dwxh2f{FWCL&%Aw<% zX4CR(nyg39UX*_hIP|=GAOseOflNywbCF~3G3dIjiPwaAIk3eF@i|{mKxt`1q~mSj zd2&=A%fm4ze|sY>OO@Gr@8^Hirx20AvkxSiO75w{anNMyh zPmSQTrvxlb<`ooNdUwbXHT3W-i>Ine2%rj%A#-Lk5K{iwg)iburkT- zE|x&>B;K%}qGXTO9GTb*9nrA5Hzyn0nDYg8Q9 zQ0mX3J8Fis?o9Q8&9mJ;@#(vH?d8?zPu!XP?w`by^AVgKQaGXCJtvZ&7Zug9_t8E z#Sq-v(GJLKT~bjdc+>Doq!3jc2;PnRM$@F?H~EN_=%can;#mcI$4wuNtebEi_UX*$n&<#vv~Aki7cTFxOy|7ZhU&up3PSd1=Ej|pl6K*wpLs! z7HR(b7y`>3D0!Oam?99Z7!L;+TkRlY?7u0KUZo9EfzwMO*s?4)h*5uepjnb}+7=kf zsO?{}i&CGz!^=tp#`5ww<5rX^aVHIsR_7v2A3uPu7I;w#a1u$us@-TxUQeL#^}?u0 ze>2(bGlU>9cX=?A=F@mUo6FunP3SWLQe-cv**nQSGWtLkm`*5&G(bxjaRIsbgI2hC z$A%J-LE}Da-7g>KUDV+ZoE97W0Rz3hg0~!0h+h9dHN9$+MF|0Cg)FbjNBWuC$19<{$o|O|%|!;)iR`U#S34UxG{R?V zJ3EAK89T`Exb#92Vg3NV`&bJyhV)n+i-ASsBDmtV{i|amEL@kkugak)kj0U)0Gd!c zOz$H8*ofx#$I!wHc6?S!^DoNtUkF#OlOZK&hB2gx+=-ljyOca0ems*cUIPu9cVT7H zu|$exP)D#BV2nO$@a80$uQ=XcKLh^=Ul#fHCe}abkZK{oZ+NtTAWD3Qk0nEh!RG#g zElamXViFT6C`|Yle~`31Kzd!LQG5 zAg74x(UxjX?yLq%c+%hkBLu1vo7BK6#*;CI0l{Ol`C`QS+$8oW>$TNovlh8`p4V(?U|Hw+(n=^;!*By8){#<3G6%BNo%Epa*i{>Y0hr z4W~b9%W+CGAE?qe08K_nsG9H+-8ZjS>eEXny4kUr89aBBRXqSb;g%MXm#LQ^?HhVb zEINSdAr=*bYpW6obeag+hYH9|*4=1>y$&SMyQ<`IO5*yxnzQ>kCV)`i6qK?)LIM@f zC87%%B`ZLl*LpALtG-JNDpE#)?8N?Cf>3eLKcS`D%Y!yR5YY-ZbJ;9+cQm=cpFLTJ zc)mMh%!7M4Q&3^8rEvrjgTL#QLA+sQAr_=oKH&zEvDxl+ngyCH<1V8M&%Jo>4pAV2 zg#}%yI|+@@1|AxffLRs3Z=5@HOcr6y35OfhmsD$dkB=P<^25vxD*KUi>3&4;#O*|s zUmnM6HR=H?@HZx3K1eaP;?eQ2JY%$Vduy_>=)L$Fi6I(LQhURR3x4m$8m@FzDJ0NB z^}q%BEalch+6wFz;$0v657y_vL5e_tN~>k}ZGyr$qf?!~qXw zB#o`2OU9Y;@v`~xU$!alf;ZoV7|CjypQ;aYbKFR3)htMJ{dF^5e_z%1rfWc3q#h5# zg?E=yfM=#%!kKPV)*BEni=oOC0Edgzd7ST^16_nG+JK>dK*io*|-Js`RH z>Jip7kdH;E>ljd@-p=Rv{-a;|26Ov>}rM#g@UIXdrz0~beqKV4{ilw;V@e? zwzmspf@#eNO2dO}cStw4IEEkiKh79(DeAu3m?r0rFqbq;#a(GbQ))nUVI$0`bp}*F z2;NYDyka5~?96UGYG)8&7vJ}iO9Q=Ae(>RTl?V!s(He~roa79*UYDYlSDN_PsHTqN z_`SMj6~W_ph`n0a$YQw6kp1^7E1&ImOZgs~7f7qBZ*ET0!ywCxX+<)7)yMc(G?92a z5AgPk(qD%uXu($Wa7eF36_6oZDo41ed1h`Cl|5HLF>m}v9@QJB3qc2S6p_!ZFB&p2 zNC&&OVZ%V|C`#oXHJko!j8NUnJ(Wf-5HI4@cDtIj1a(g;K+wZaM5b zs?M1$>yDgQEF-0+S_e$sp(M|hMCNK*){0rG7A7JoRbrjB%RbTlhw)bZ^R+CcKy($3 z!vGm`ws4pG7-F{SD^e?2p<8K9%Fp4hh&?RY&GD#HVs+DFW0 z#}Mi=o%bmR#SKQ3%b)8tErnB;CAk%q$PLwEaNQ7XS1Y^J_uk~C=3vV@Qe)PnSwTF{EU?uk0z-0tNaJR1Ms9??i4;Esa`LI{#*!xm#tt2T?T$0!* zGDM^PCl!?FcSftr>eu?1ebyd<&aP^wB1-Ge`yGfoo6Gjazm>-Mu+2TibY$w81r?o{QGLx((+;dgjn%&_#?3Cz-ytpgcr-4VuOFv6@r}gYUZt7zkR>G$#xK-a z6N`pKsj+1@TEEbEEb(H;m#OcH7O)FwMOJ%9~pwlExFabP&KlbwbXxskJSm>E6>zU$n$vpRJB8FFb#`b#nkeu}OO z-S=aY)~v`XPQa}4voOelyk?QmFen0zp!s+lDnPS&aoCm{_UU2asGUpNiks|fV*>8J z3)p8Rg7+w=f6@YB)(;W+9ektJK&HC(*oBaIcRP)xebU|3)&1ziN3U9er#ZBnQvbQh zF+@EMK&hhfFxoWNg$~fYmo2>GPYkgRE3-@?_Xn?6VF;0n-RheUz)*>|Ki8`~rh0Nz z-ef$Go_w*zae`Ua3G`+JOr}xQ8U4`hv+^n}yW#F##=g}-fk$(6`GeY{Q~=2$G1vKc z@Hi7W6a|knS~Ik)d?E#ljOo4RG|}=lTrFJ`poPI1QG+~pPnwnve_DJ2VX(;%3z{BX z32}6oP?MSt$Fz`av4o>WTCJ3P6IqlNauJ6Dsl)jPcS^jA9NXJMBN%dSQAd^$lo+Cn zP6u^TIdcJ5uzlWW)7>H3yEFjTPSUx=Rwg7r{T$3=BU~ktjN+dR;cwPy44SoU{%1%- zvj(e8kC=Qn%0%N6=3z1&`mq-mZ2EP%z41^kSNsu&Zsma-fDIq1*%VzwcuuKv$2aya zhR5Sa-k)WR!;g$kc3rAxHCA>(_M87%y&cQ|i305vRdl(5=uyte^saFYa_JVaQ0pk8 zv-w<=Fe$ni{Kq7wsc25{8NHt4v1rG|F%ShHco)6xMS`2ZENiX5WKgV{)fywRT#^vCagTAOfP^slqvqM0U?nDC2Y$hnw;D>@+P!e{Kg z+rI)0nW#As6(_Q{%!$+DIx)w`PUdzGGH;^Wkap1 zHKiDA6&9PkU#4uY511=Lr5wubqYS!#GUOov=6X))m3CNUikm(7k6 zUf=8Ok{yxin@slP%fz!(QIS^bb9+ga+eqAsvDL`&l0Dby0Iq5IcFUJ)plcZSa+i!^ zPOp(h(Cj*7pm6RNjv_{sZvZdcm@93U>aDr8yCBnQ2?&#Ik(8Sl3txO+i$1~oO^3f7 z+cA{)DHz!bM78!yQ&$&2zIWWCRhzNy$J%}~x(~D)Czu&04EG|QhBA!!AR@(VT2zYh z7!rCDVC@3owS!@;B1nplpVx1CjX;zF(l5VqSr0Vhry;DwV|IMtNktw~`Ev3mb$#Yq zKni>mXcI%}rb(ebD@xBQ!sEOYcM?R)BuEiB3aF!fQOeta1r+jVkgB4H-pS zFF&YLDyNMBX-ofo3ERS$0aseGjs4vaC4Kc0Fe|3uJOh~4pv|Xd?6vz_fs`9a3L+W3?*ukyxg=^A|+pQnZqz|jmZLJSoU0< zXQdqVY}A89$&Ipm{}Ok=&q&&Zw*@Jvza1HhSzcB*V>5fjs{L284v&e&1KwS>(b=32 z7YLLgDiJ0U%sA@aXQuV zTH;>(K;ak&15bIH+Gjk}lr%zod~axMuskg$o6s)4Te|Fe$|*28lMBz#zrIElhvOdp z-Yubr1PN{k`AX=W>R&`c18-wW$;APkFLYWPh5Eqik(Lb#Q7wMcZ78h`^=N*|Kcck} zXxO;5I=vM)cMhQfEBGa1@uB~TlRctY-p2zUC7lWl1^!*4tR!*V?MQ?sX(R-4Jv+W* zMH$p|h+N4BoLMhi)_>;axx?_`F73iqn*2hcbYcC24k$%cd__`BTwoxnRuNmeD!7}s zp3CA7c1$cvHtLAHLMGGE*hI4@g&m)FUXe3$#Dhe8pfYd5it;IY6AvtD$--u&ISYFP zWAFPr%(=Eo6wh0Xxp4u&fCD~*r;P{kvVr=$SHQa1B-MDDOdn_)!IjB_-4$Tg3%huI zLk3k%Ab5;q@B*idCbHzdK}srHA5I<>QymS>^R%+(d$yYdIYasG5kTIng*^DpFd5>$ zOR&=hlYEE2XQ;9OoF=7`7U1#c9)2LPAP-ff9##)QD7$2oWDNjN4tI=Z<^4ZSR;r#Y zRoPILS>!K&6TnqRAPluggzpiIhEP7Tdhv?ttT zH51>wd4ofpe*93_wjO7aZT1SXDh=_~c_5y%NKKKua_IRdceq0Jp7P~qKVU(f<|UOI zP*A0?PRu%K-cr@-?@(y0SUd(yHM4VbJNE%zP_-Ftk!YS8IvKI~NfIE!$=kUFSp#P@ zJN47vOl-YUyKs?4l$G^OZ1b?0#&S|x)kw=7EHbq1Sx6i{5eqVKNuGSL^E#UoHgyo@ zw?iuyI>HRIvd2hg0=C93O(U1^P8joh5uR%OrZs>8D}rCPc1(b3+Y%ot z+?29T2T@Tc3Kd9}m4c{!@_yWMr-4Sr?kTS$8{gL*pFxy6wrnF_rn8Z06TYb{^k>u2 zs4>Y#14FxBAv$`6l0y<{QU4?|7`%F+AtfrDzCie%Rlb>l*dJzVa=YmqS46TFGL3!N z>lqFC@DG5%93S47y?C6nFUl>S3qEqba*(?`{8T-uihH zpCrlSqYsgNg4-FOY&7mKPyIp*dYO^*sfPBit~Q-NyJxUL0!!9YoY(k zIP>~liabnA=*s(tP*aB40M&>f!7Y(&C^5tz#D^x@xjhda*`>;pCWz=wP_mL#WEEcQNk&qlo}(WsDmo1$l$@1aurbK z1HnYMme;8(ZW6>*CE!T~1Z(&K$k%cu*f)=pirwvixl&qz&-R>6_NNEIOo-q+14TEEviH#kD>)lQGpr)^t{H?EaanO2!y2kF%;`7S7H7 zY>4ot(fD;LSt>g8XXi)|S^8En6e|FcLO0NPk)vFRhavB9CLey0rQ;6N(xnbG&~L8x zRwGHdk6u81rIxC5~aDu}ca$%a;T4m%TLRDb&~tz6U@P1h3xdurS5uE_4EI+=n=0 zFb?2XNL+O@Gdpd9Wk~}k4jjo58E9OuX69gOEPAs~j|*OnLH5A$nElOv_hz9~sb$PU z9Z4@lYx11aKk&-40IoE)(9lX~m;I}OB9v#{2UQ0IY~u4GdqjnR&h1}^;Zo%jJ0e;b zUF}T5r$AL+DHiT}_evZ680yFlkw0G(LEN!>Sx%3ZOYBugDu*oigY1Bo*T4m6QVaHZ zmYj?tn5InSMRr`oS!{AB<5$OV>x5rn#+uD{8s{?Be`6uD40N+lY&Sk(2C||W9A@{q zg;#d|O)DLB^LusT`yI2*Pp)qzPLtJRr3`#eX zD(zAy7(zC?E-4BtEkOL6^tz*et9~y>pm{0YE)#*+hly6vaV&p^B8PUuakVxHGvu=i zy8)b9y;~xKqIA6TSC)QWg3gtS5T7qm5|cu-FZL-xoCQMH*4LcP4|`=mB7q=kei_m* zu*K?iOUDNY-f?(sZO0OjK;<>uDO;YD12toF27ZXMU$d92c2FPNz z!!D_n!`=3MRpmVN;EoPG;_gy9vMg%=&>YQan*QRn{=+d?Fe`|z9UO|sJfL=ES4{`8 zEMmX?K3RVvBmbMwu?MchQmYXftRF7B|6$wkigHEp`1}>4o(Kx;RNdeX?DSd`F8rAa ztw8Hdt4In4o+~eU+e?7gX=`&+wlMX&dyg6b!W z-K$8l73xWq5D&%)^X1kXBxZw)xCLc<2#JjRL zt+o$WR4IAPL?j44MSR(#aD?1s?S>yn5}Eo|_t`gx8A&|=9sTgCd0C4CMFfs`Fjnpm zVY;=d=`j+#1JtzCw)f(jeMwmdQ7&iM{k)dMLz+p7fQ>R?mFFSWlEmaK$@AULVwaWD z;C>g(DwiIc*Ys>Mo%cyPX!6kLMU?VA;1m0ob3!psoQJxksn@6o_`jyxjOFr)tm|z1 z-@?cQ3uk-D;=5|V*k9i4YmnWA0JtpBZ8#|XQ5SiB(6}3lmNrh`J$<_T1fShuRPnjWk*T;&5`{}-jU1^lRz;_Cou`&wB z92KLiSRt4Xr0MVva_Rk6-1=HV56z`~B}#ExezsSA{F32z0d$)jC#U`xoV4w2YS8A- z2auQqaZhy^l+juC*OT&6^0YP3UPWw_ar@Cd+pc7Sk!8BvP&CHGqx;aBN@=ck=M;-W zN>KOWK}d&$3N7Th(~ux`r8MuN5(cM0I0^q9bv9+Xq?{Q~J1c|v5H*dDziy%_%Qp5@ zgjM|`FZX3b{tA7=%fX)Xyf4qlBR(3+Mnw&?8Mm5iK5Lg!aCdiaGdy&_8p@U1fe@fHe-^AbF$=}}}uAm=i>9RR2Ek)^JX1EEV z0HDrN!BK1;c-V4h)bP1%gZ63|j~;m-CQ4i*s4Kb~hoW2jv1`IGSuHXRc2RKRmzZwx zBL;#ry*yN^O{UxPbc))na;|?uC-;Ar0$YPQ&yo-n;8lq&as< zh*at3223EhVIZ(+=Hj5uI~+=CYBQ{3CA4JO52LHvs(uOurI*tM7A({j8#@6G!2QSF^pi-X(_Vp$23T3TYjMP4}|H3*hGBNf_ z9Bwa+$M(vZiLa{j=}=@h4PDxzP|sbqnNeC(hjumUd+}~XLJX>bRRDUDN}_jJ9PUei z^UiAmy?Vsw_((F#iZr4PMINU<(M;7ZNupJT2Y{~Rbn?Z=ztYbg{mYT6XrAz)jO6r)vzS1DrO@&GW9{r z#k)t!y~Ij;s*);~K2le36X5a87-Tb^w$As^ZG#%I1?)Ckal7&;Gx5EpNMP>8 zID#mp6R`-oVbCUd?(2aBKn1qIzF?G{<`WZ{G0<>!RZWSGGlB|Q&2(vx;@1kIf{G6H zgtC)1`0YCUUw34WvOI4KxJWVYv8+H+??=SY5I5~@qI@b54 z=5Gq|alpceycY(rPwd?8LU#Vy3@9;FL18lY0Tn4^At4+yUpHrskd@7)jh8@B^OzXK z>SwQ5eexcq3hFwzS;>uo7HQob1XbiKka&D!e)}V0 zTK@4Pg`Vw&j@yE58q4Fc-bI0oHifL8e~lU4%gRz5T%L2PmyMMyGhGiAig1n@uy@>KQ=p(6^eS%&?+@s>!tuA*vr%C$x zkj}ID=UEu83xz~Gave{Eo5=Q)A@4$NW07I{QOhOX&US;@TOwgM(FY)0JGFXDIWMk& ztYSY0ddjt_`q2c@WvVn|sXB-|++UZx1^ExhpQRM>FMaEvp>RVq^XrT9^*s04`T6vd zXL}wzc+lFk!5MF1WhLh}k^!PLW!cHnhzz+bWek#hxWOLoknqhtvB;;xR?(?D4S*}GT zUtUnX0W34LQoxx}!$~c7@f{o`1rl<_unY=PvGsTFt{Gq<32t8)7A{x$_N5|uhq4og zQqGzQaZ2oMF)ouHw#o&E+M-I7jX1j#aj(Gu6Tf83m@awH!yALQ)V%OQkNhKfk{9ny z{u3Qe=M@yPM+2LK9B+)Z-O)^LT~R7D)7$3C})}l?3y^wUq^#z%W2J2IUHT{r_{VjJwKUhQFD){vEx_b09YpAG=Glys{ zKdAgRX!ll>4b)1NXIUQ!M`o=aQTC%PtV0KViiWJX{TAq>HG9;A?bi1Xu}dP==3GVd z^yk=Zb5HXEMCVT^`+&uY?*Ty{xspoy$nucGVlKkS*9f_S5=Ip9*;)^!q&B;Lvysx@w< zDJ!pjy1wqljM<)&MU)=u&Ox2GosgM4U^i*mA1o+&QH7HduBbauM>XgD{}WrL#jm@o0?V_srv>5ylnWuh&att^iR6q)GToj-3ucIbcd@g&g96A zIZAns80e|e@LuC{rG|oszX1E3(Gnv?aVxVQJp)7fA1oxYEoSU zky{P*-Y+^KP`2M~3sk1<*g63{qoxK~hdJ?hN)7DFu`~Yo0i`gV-9@c za@Uuwrg-gpHqiIr_JCnSC$Z^do-B5lZ%>$C2%c4<4ESXb2lXRMILuVP-2RK)*=5c^yc`=Ohqj+b(SD(L!mBA%}P>>HFe?RKnu zEOs+UP~|JjwXM#rlk=`~VTKk$*&hsP3sLuySFdntf!FusVLhjNk>Gvxw+z^RhrcMy z_A@L+o7P3VUKj)qfVVZUO%NfF)#*kx_50rc*>l7}9aS#!Md%^(ZcKR6xuw&{3T~$I zuB1*~d!oJHk<@fZOi*rajyrzgoT3BdxGitMZ!ko(bQ(*%RasU2$AwAJaq4SwsMRIlF+pU$0C#Y;vd+2O-*QdG9LvC zPhRgM%b@XjjyaR`$$<0=QcRca6pw+tyLfBv5q9UWD)bdElKcXPk=p=k{8v}5l+mj;c;^jmABXI`CD;s6g zGA7f3Tnqc*^C8N9SDfg%l^#D^S)MH0_|&x#7Wa})RmQ+-Q;Xjt@{;atwwY5b!_;{; zBJW=ixsKxp1WVk%%ifEyK95Kuw#iJl#iDn$3G1~Pf17D#p}rKR=I<5GG8Al&b zO+c}WxgxByrhBXFu)9f@zPXqkii2g^7U!LlvN=Sg&c$h7o|F{Sq^fO;zJI4rtiJmZ zIckH@+d4>jwxPXN*6fa@%en-Aah~t z(jBj7Q{<4&CF>I5_zUEQUDi;iv+xj-mmvpDui$0Vwc}& z=wEyr`feLA;x~L0?+I%d7+9U=fo|S9Bg0nz;mW7Q6Z}5S8V4^0MG=;Tb4#HIPG6GA zO=!CLA>S>h1bV77er3Bv(F~AO%=CukC z)KINX`~1BlC&9|D(fy9_lWu5zkqUP99lIdb_UM}r<<@{dYxR9k@R0kXNp3Dq+3)EG zUmRID7uL78^!!f7_d7KXKOPZe0Tz|R>OOhcVy-#V@V>(@goD|u+ovSs-nHBV1gL!L zgGbn|--MCY(#1CsYxuBuLTmR8gNI#=GOU@7z&K zi+(z87C-$KI0lsn<@mO>PQ}IvJB_=n>rP^IU_tp$i}of$&F2#|YgY9yPrHd9Zu&Lw zxWf$9Q6}D3vKgm<=$0|Rd-TDay70}(BcR?52r|H0N^EYP6uoOba`LOhjk95rGo9=q z>^FzM3|xxJ$jah3^fhR$HQqstOC>67o&V;geuCE_#Xi{XK-K%K371+>%KVgQwtgkr z9iVr4&5gtDMQ4(xhcr4arD+%ZjM+{IlH!a&TOo+sV_s=QbF$Vut*1p&`u2yb}iH_Ma5o`CpU z*G$f)1dLmgKC5;2Xq-*k5&)w%;2P-nADciO(G$>*KSQAUPkamF<|TK~QaTp52Y{5u z(dGTB&HBl zdi*!O>tbIOFbSaDY~OhNJ`t>c;t<%pK^eXVJEZ`rwOu=|-OUX$xOel1V;?xm z$wZh~h_LV;|0SV!cfU;7QGZk}O8j${l#cdgsoagomwfwd;>{}{ z3TvD?^x7HcaQ@GRs?UuFZABo+qaY*CDCB?);H>0#@)#&`zHnPN)P+x2_W57@k{g?> zoZWaN?9|KBb(8SzFAb}ACQ1BKZgFowoVW7#EbC>A3*gd#_WjGs-&Vf>b3CVXK;eR{ zz7mueuSrj^(3ntS%gd7P3@rZUMJXMF!RzzMuf8pKSm|eOliu)X#~#LB4=-P?yYu4? z_}l|mxy1JkbHN@Q6_1{Y=uWU^Uin10%Ebj6L12E)v0>lNnXsct@KhnJ`-V#4iPLT~ z0I0}2EiiuR314%r=*Ae)6y92c)f||%GZi4`mVD_NzxBij`*aCA!PB|2`h&8|I zDEj>dgaFR#SOt?0hAsuUAMB2ZurLJ(egmf^kU`GUNg5=c3VO}VQbr19vcEs$=a6_A z2=lW77x(ejf`a#nQ>_ZkX{qy#P(Bs(`}!l1R{)@2cN(;6Spon624ZiSC~eKQgB*)$ z2S3DV$eTxA`)hqz{LbOFsdsJSt#TVs8;$42W|RN;TJd` zlili$!SG1R+hg1#?l_xx^R3Vek#XD*4Z1{DzGWMvyp^O*@g zSw&iS4V+>YzoL_qShmo3f(n9FauLXHD| zVPrk|%%`BCuW`o|J0RQ<_X+NRa#?^)4X_&eX;mN0PFT$XRHD4A z1qh`L`f9SW%zy@m3UC~bovIA->GwDRs=I|o>=mwWw;+1dwDA;mSMGmX|nAgP?q$dN7>lCpk;3WsL-6%xglRJ~2`lD;T?*?ei*R-&RZ~ z(&A3HI{UJM8hdeF90?w0bNKeeJ{ICJko5^-ihty&{3yqc<^BL`;r6AEMiEak9#aHhk-7JWQv?5Uhx)W!5H$awndn=40kr(hyh4X1+w{kB0tECBIiNkoc_j3QR2BXyC6;XNDVYoy?V6n{Oasz_7v0kRQg|xT)f{S+lOoK3W~lm zyE?ktFL?=NxZE{=cJByNjX00MV^Icg@E`e4@j2XYXKv=~iA>5qR-^=HGUYf>y}=%0l;|%r`IQC?>c0t_?PG%3CQF zz0aUBi?#urkp@CRu$9rmvS-Qf%tHua%iD3GbFzsQ&CF4nhwi86t=ww8c*$47EK=?( z&F<;1PaEMgymwT1?hUF-#jrx_Pw)$u)QpB2-ni-WUcd>J{6=aCBk+=W!DpL2*w8}3L3PgJ^QUBCYrY-NjAfL@ zBYo(fs^T1L7R9L8mpBwT&@uF_#E&lgp188hf3$pS&A2wJYQ`N zIN{;;MN{NZlnm%CF&RC~kd)K_=>rq%wX;&qu3u!%8E~-!$Zzc;`3_#<=o_h1cfQm7 z6$7tKOdhB{Mgb4e|Wcll4PR^8o4-WzTDUmhuzn)V9x`TILG5?DLx5Vb%947B)-lobV|qq z>^f%VwxYLEF;k8~HjxU_w;exf3p7xM`IE0uhH7rR&(+2(1%k8eX%xn+qILJssaL4jq(1%AqdAwP%tywuO$F4cY2-QQw=Y&G@+<#HV)(@ zG*|_+SI}*fhtGi?6#OVs~i zwLUH-1%x;o=xt>)>}=6$>k{lE7{yo6FSk*a(B2sc$Q(GV1raL4iA9HSu zJtJc-4|+Xr?g2-pYW$Ct=2PVK->fu&#!hSAyE-}@AB5GHmX>_RZXpEj=w+el0|ule zC7KoQasAIJBH3TtN_Q#&E9*F3-naA9ye}X-9TC;l(*GMh!L9%Lr1(X`U;VZUk!yEa z?;WK8dVEZ}W)7jGLd9LURH^fecgL)E#&l}M8^4sl!@iZ?kM_UCqbOFXCb*sd?v$v0 z-*Li=7BAbgXWbIR+F#zYs4Kt8mVQ<7SpQ9afCercY`J=WudgU?WW+?LFPKB1HMi)| z{Pm1FN^=`bArc%l^~T8UyN8}HCAjK7ip&hUW0y}*lKgya8lZ6iT5DUUoeAp!6PFLz zON^X{&z*u1?}u@lJPQram^-GOXyTUWDr|6lX4b>oz$imWHiuGRar@QVT)5qA<}I~@ z58`VR9&TN7Aa=L0a$~QfW5MdH-Qdcx`8>c5J@tQ(-aD zX1m)_;YvIxsVPrDI;grE2JU|V41;}p0*sOiv+qc`9o}Y=kQH`Xh3RwF!|Hf*+XTgY zFC%2Z)}^du#EFp z-ZOoV{pzc6=k9qALl4ddW-j;cPZvq?t_4S=t#}<0&jcg<)F$d@)LkyUdZa}BB;3UE z;8S^MHK8Rb7b}j=Pal1+1)W0sTU7)oZwBihy*E>FyM{&a^fhDQbcLFp*kZY*OJREV z5YGX^#&d;B>Zb%N-)a!cqsJmoCMf6A>f^p8-uHF=fr+DDm4E4>eq!4pWzpxlNDR2A zPU;^Gd#CoKjpo#r23D!+@o<3f0sLd8 z#ibz$Ycq;YFuV9=kpC3gS6R8CEaI8D9JDN5-lj`XX`r>$ubHpL=uOsQVk4hP4dNg^ zPT~G*lQW2J=2L=c7+- zXV+x0omL3*olLt4465XOEg+YGt92is+rMmtpa1(AP$X4cmT}Cg*#XTY&|H>ZlC=Nw zz=3_{2)G`d0ep}B>k44+{QD&aREYom^1t|~{X7UclC6?@($~fQk0zW~wS4cxH5M<+7J&3a``&eTF6r zbcqhW+O^TmoM;wy!DMeSrmFmw=*!>6-V!-Mqy7K+CeyP&uVd3BRd(*RKxPx?RK2^z zt$4lNK|d8(rS)%Tr1(`lGNU=hEA7Wsd|R9iBh`Fn?wSt=dqzUGQ4HUo;#d{%LKtcq zO)H8v#eRI%mFzzJeR{4vSy*yst-6p@F_!^%sYnjZxUxHy2@|SXN!&aKs+%3Jx)Oa? zY%=>?G{4C(Gd;#uw*4q`8Qiq*Qt|7eYo&gE#>}BG1=vPBj?=O60#)m)`LGWd2_>Jf zt+7RL8~%)l8=SS9R#ca9H45J2X<3;!zCKDc!&Qy3J&7x%I@Y;rl(fX*xb!cZdTyTO zd9Swfvs8|^Y;HHn%cdS=;}uulsrk%DbU}8Sp&QIB;$4$vh za)>{5r(Hw7)BN>4Xu^po$8lj6H-t!qQs{{UQlKA1+xf#-cn=C)C-}h zTLJ1(D!%7Htr?X8Wd!1G4X6*7n7)_cv}t!a1+%ay3_ zqaQDum=e)hdMTUT&RE(ic|Q1rV_tR70j;#&T#ASTD;|Y>Z=>w!s*sEt^Tn&fAgf13 zst6<6{khIkh+Md9&5Pc*coxY@FTN{JUHT;av#jS_fF_HJf>Uv0V{lR-dikw%Cm=h}R>#VLl~>X_ zt6}8KV@xR*I=F%_#T)zQ3V1WxOH2E?A02Q)a1?miJMo0P6pup=`COWXfMNu+{2Lm z>qI`zale6p+u=%UZE9nw2q@~6f16+OzFW{%MW=G zoV|w$ZBcm4e6aS;ZU{eow@BS^E@DT}KBIJ&NEj58*v3dghFvH~`H>e%lyRS(aXtyX z<@pd-HPaUfvv2L!3-824o{ZV%ZTY=iT@cIsdHmVpd{_AVa%+E|cGrxq{ z(B=)5zMD}H+A>*QkUlpQUoVAN$8#1`cJ5{;%lb7uea5#@C#^A4Nj#Fo2AP|Y7H?D9 z*wgN}dmS2QYSoabt02CXjcuy=;OL;V^tO*fZR)c-o`JU&LY*%pRS`GzXtr2Yit=n; zkM?!NnFd>i>FE0@RgF_R+tw;vw5<)js)8uStgT3*!eo?in*_cR*p*j0*j1r+h3ZlC z?G&s~E%mN19gZI4D{Dw^O1rOtqOTF+g=%5FzJ{BNSPUa05LjIe1jVqKc%iLA1$1gZ zagU*6wrB6;p*!!+3!+x53o$h~dI(pK37=El{5rRV-%^hnjV0NA4SFZk%G4D_Tw_5} z8=A!O(Pe%qzzjBkyO>#Fm*;b@8;(DrUr8OI{l0L$?QBuq19Gt~jZTQdqs)&&%7$%7 zX{hZjtPlndoX$M_YuI31bzvCly`@#`07YTpipWywJ$z-c?d2_L=k>3rpP752VOUtt zYYiFab<2&~%K(NCHS1I9lkr8{y|FRo1yM=`!zde1%Sg0-E zQBpTFfo|o^H2Eu9@%nGf+M*z9UZVT>ZAbaY){4A^xoiEMr9{vfW!0X1qZS{aDUb8R zW%)*+Nur7RH6ij%=+Q@(LYh5=kSJ4Fy^cT4cl3v29vKU^-zYSS!zTc!PJHE;>hIqg zA%$`iYqqAc1*>u+Cq{&g#4rRWLO$^ho3O zXV?ss%1s!MkW@)O+;rh7OWSV)*t3=M2WrO3@d0)0z{&$&hmXPgPUDd~i*OwKER~m`-Bn3WYr|Am|Ft!Em#Q~| z$RhYBll_yuv2#oE+)$$vHXcO$C4nmnvZYXzPV�$LiakSV(ASI2TUzaSv$Kcq1wV zw=1vZGf4(^TEV)Z3&IIfS^ae65^c>%uh3Dj= zYt3eR-6*yY;$J>8!ngdh>42kaiM@_<`0IlaCT}l0LwY?ZPzp32xjxRfza*80<7MDT ze(yU63F8|io4mA`-fLo?lWo70QI)YT54NDf$BtVrjZaHcbp-kaY5tr3*6Q_N8XjE7 z^KAQ1P(dMIOMjDAZkbb$&9VW#YL6i0J`a`GM9r~=mX2erbOp8__FX9{^X zR#pc-^^uCA?P%%rfu+;W;adL$H+nTZi=nM=*ckFY&}fr@Y_{n^rexTd7UX(`tj=eA zi!-T#B9Wj}sCgai z(LOV9Dq~Gx%)US`5r&%&FtkJv6NZb=EaF_LDcYjKr$~P~=4EwEfsuyV?WBvXQ_FJO`~M~j}~%sYe=#9__t6_#jQ={!pfKJo_cvd*BuD#xT!wMxD&Nt zaNKRE$M!uPj}r8m>w)`8_PQ3TM_Rz9J4`Vx!AUNv^PkU3La0L(g(UuxzmG*z;ln>1 z^YObnCYZuZU1bW9Ac350(Rya&TiQ`RHk)slnj~k_JJAq-F+*I7xZ4myLaa8nQRyp2 zH#^FN>?^5Co7>V4`~2nyco1Et_NAp9YFj0L*|A2jOuccW_0rd{a1M)_SnC~PZrAcd zxsOXgG$G|dUr9&V$gAS~GJQ$2uX^~t&YkR0HF92$M#*3mUo&0R($d0~g5l zz3m)`jxNBi3b0+5eK~v~QdzV1D;Q-0|0!VGe>lE1x5XJ)T*;MpO7_?;|5a;-CQqK( zC9hya!D5U$OaMK=!6Jx!P)Vq8p|{slEL^waSi}?ow()ZnM0qMs&}4U(h3nCxIwsCR z-+Em%<>UQ1n7!n1Ez7XKVTG!UmtvZwX^gJG+q}#Ezq!nZ|K(`k5uo?}Em$C5VBo5L z*oaFECCA3i8(n+zF|72bxLiXX`G48kHM4IsQnzfCCrF}tX>ye_X}g;Q)UY!rg-_~d zcoKjm0Xv5iXc^pN9I!EX0ER+=>lP4P+W+~#O8Ig#8YelCDvT!G{d5nX!(@toWI2y zs?Mxt%AG|U&zH`3U@Zlpt;f1*Hb-8BOMV;_1affCfhIoFEv1YrO4DDhg(~JdKPs>6 z7CO2rF*4eKvmUzIBtTwgWkh3rH^_$aM%E3O%k+d7$_Ie%r2t5K1H2Hx=?S43!~yxy zs^K`(jZ3Dk#kE$Ugu*IB(h5w9?ZVg`7pNY_cC+nEwEp`qGBW+#`i0}+-+74e3E^-K z{+PB|&|NHNWUYyIEB|Q9-P2?lbs@tYkAe{*u>RBz!^Euoa{AruWZ%Irf5O*f(j8rD zRtj*gbn<*!oKV%&FppWy)B`bY&{_=}_v@Y^p8z0 zfmXl$NMpU6>iXwt$nHdKV{q^%`JLHpA=Iw>sPVjI#Y(e)p*b?*Qk=;Ta`gh_xxPrE zqV?Ts*g&gb)g{?iw@R3P{{wfv)|cdk5HZx)c#B7p$;!561%$8md%5X+KwOcFF){># zvMl?xLQE7RpImL^mtX@rhb)As?`kx>X}F5qXItIo?(G3~2X5T~gj}#S!7ki>9U|xt z_8&ubt&3e=@Ri9RI9lkdG#IG{ms<1SBo{k>4KJ4}f}u(%z)Z$nC^NT@9oXGkf%QQR zX)Dc{sNJ;?W!K7&p`a(mkSK98p3J&2Z%(f3d0I$B zdNJiBso3{Ah_#C*u$(#h{|!NVv&kGou`FM}$1O#~u2E}KO9SSJe*8~8sFSs1M z_4IelR|j%8GX=WFp6wM_W@QCSqNd4M!3vU~ zxjX`|JoB?&>-|uiB*YCo12Usj>1$2&F(XSbWBLz7l1>KEB4R2Gq2`)6S2-VL;k4E; z5KNPGpMqk|#=0K~1zRx=(DalyglM~4jAKf+W>iAT`i*LW`)v_6;w8lt@f%ij3sqEZvk=9EL%s-ewU%bf=@lTz=XlvIzYo=_-?UM}`cw zh(D)^aH8fcK55pXH`e|(zS(@M)$DWm)5`b^m3PN|@uli~LUEs1ws&mP)HWw**fBS0 zE?~F=X|R7dRX8Q4ijgxtjX{|vZAL2idO1aP<{L8T1%7An8!4`p&7J8 z0EUN8_`~t;lv;tIx#4146n)qh+(UGhy%2JH@@#f$(assuai2DjdmuL_S>(su#%2vI z@+}xe+7e1@GX{oc_LoHb=2yz;(8X_t7M{*x@#X#+8k_6dl5+j>d(^P((1l4Bq0eei z6Bfa-gZ9(vYYp>h$ejjT7@itn#mfoV{yiO~hFDDxG51@8P#~+np@Zg8g>K~2hGzZ) z1Nr<=(W^VxFIvbIMxK?n9dz>lEd<}31a~Ux{g`;Xfd#$dje}4(4Q@W)-i(d{X7;yJ zx$2o2V)%5;4df~;F6W3ci(R*M>I-p z%SWd#)GkB-wvLFBM6J@{dP2}=VEr|oU~{qdw)E-O)Vb)A?2q1i$3%|_huYED<&szD z-|9LP7w~D(X?89f!FgwQCG0Lvm2@z zJZTL4M=hZa9E%_#ls|gW&lzz=5Imx3J%wQ@Hjxn+gL&MaATFXr-jv?fP6-{fupNVl zo{21IOVyIY+rAE^Q%0rN58z1wflCgA5uczbW=mv*tpBckGKs3OImV#j--+crCB72n zA3RvG`6*mi+4_l&PT9-Aux9UNBn5KI0uxJlWh&U-`s<79%meD%t2-|{D5WOCq-t-w z>H5b+COx&;k`SQ^2B8qWJ6TQg##wjCM1r`I8@j<24E9;eBveF+ve#3yHlK!C_kB>- z+bYHYSVunTT6D+Dn+bFH%aFBp!9K5P+nL>CVHkG<@{ zVq|Iv1NNMPsQX1x-IfXg4t?ytJSVTnHEL(rH;~}WYD-0*J~R4UA*hf-HkNxe+Hr0Q9t2)utsOV2aYppzv@hSMQ;d)CV$+7g# zI4mVQmN{7sSry&Vo~kNZI|#;u#5t^7D%Z!LX_xgB2BYT7m;3J^$cuf3$nFfO<%P?z zpMSz7>C+YNVXSyKCB<4#;UlKMUTRYX5|LH9XbgOCLvD$}b1_)ZCl+*FcRGflMo%il zSlGutH2_L4SE`mjg?I8of%@or-6wWkyV=@dV!TX=`u|*R(3#7k;Jg0S(87OhoGMf` zx$Ee&njX=JUhlO~cE$WTCl02OqRb*e4E zSTfm#y9i1xqvtJPJh0)#0e0)&WBI_o(dop#e3x(dE@*JnrD0s(d^Fd zFDu6S*EunmIjV?>xgfp`wcB&WsL@GUVc%WxXb6a#xOUJeT!;KXw%4iI)Wc+(;th%F zR+haM20eTGfaLyS>1yyf*Q`BdR5{^R%1C*c4uK4nt%cUyyhNjWU(5oZ?wcp=Q?U1y z|G7as4TwyIHB&|0dTP$AFT$NMG!Du*(hvvp!RE`;)~K$t^xY;BW(-QuVI7`3UTMJe z-C+#zpf}FcS(RRH$*J}n?^O0`^+jM8FI3JFy|k1|DrV4|gs8Ck6SpUXz^aH979D3S zzKzzaT%HOS^qTs_8+~&?~0V zs?^a9Qw#!q7o3$9LTEgSq`JFBz%0H>A}`{~qfN&32l3DAF_(2!)_hy8!;H)5wbpvZ_YpkeSJp4?5d`GBrIh9ByaSYk&`Tu4Cx!utH zO0gIfH<6K7)hlW6t%L!ro%29pr+1rM>xPa~ZYYF*oOJeqlMm^d}Z9*`#T~;pS9(3_z zIs2f|$HMMkkN>TvNxwb!AcnRWbvuv+*4rsII&&Yz{`fI-Zal%aiQjj-mQ&@c#gKDk z!NvayD+S?Re0je_N=IPvD4OF`8kO{`&r}F`Ujt-SSURB>HjWr8$up-65mqrm*@Rso zOcaD9l&wQ26bit`Xv5j5KD1iBw$khs*aMB?R~)q>7KIc0cDBbPp8dS2Jd)?cPveDfLtcCrz zVW!!KsWSGZr?2IfXjA$+vM=6NMr=-oSX&Cz<2WI1_&HOT1Q$3+G@xU#@f|7B^I`KA zXu@kPywA!2@C{eE%-z=($nrzmJHK491JD_#S~1C;`N!&UXE)|pw8d3t#;aX$Hk4YY zCSUq+HiNcGYr9@O`s_m4g^UF0EKZzLIHPtw&Ssc0>YGcV+NL5#C8|l}5qO$Q?aG#+ z*<+(Lx52!GIA;iCYlWFZaXF6fc#r}e+7jegSWPds9$)xn%X{F*!GlNcnZU=OnBDbU z5asTDuV=`?h@;FLDhexS038shm`Tz}2?m`ixdn!Ww*z^Ux854iDech(1b>kHun2j7 zwK2w#Dzvr64As(5gos}u0#MMEqf9EB2S{?h{YZkXr5n~+UUjRR*)|y}Xel1u@_xws zCEy+%2-#byr;H0{C=*dWAgnok$4T}@HN;;?ve0{B5+%=O4f1FjfFd+NiB3zX2G%^O zWm7yX02{8&Ok`&gC`g)@x}|KC<-41Z+3Ny2T4JOe=iXm=cSQ2pl^Z4fz3N#y2{&!G zfUez{#lCE7lH*oh0-8iBS~;^PRw(nc)gf0>$!$Uexho<2GT|?oh0VxoE6?~&XpTXa znHPN+7=L(D7q0cOeLD~cXY*mi*^n0v785*}{$(#;96+iKgyj=>Zdla`F|%+IYwrnx zX>0ANWAKlfN?T|)z&%Te8mNM&`R`4>U+fd5k+HxdSh|0U9U$l>$w5D=AyvARH)5tG zF}uO+aw9L7Vp>>MicKqNlD*l5gerk?C%{1jeXxKFAa=koeV~}rSXOAZh=|M^<-$kPaYUQ)w{#xG%25vofnbN~tWX zY^&W%utGue4GwWB# zlmi6L&6)m@P8JoGD!Ai~92PxL<_?QCkxInBa>S({Z4!QzeEIdAJ?kiyp8)yt?npU! zi}UqY+kKo481qc1vzB*rf%gbBN=)J`k_gc?bq$Q@q|Sdc_snMMR{xiwe)r2QKsTyK zLB`0G>Y*v^=V~W!m51!hjDqXYv-!)au-;UV&oVLbeo9p~6+QA_jFpxB89e(JAiZ?| zi4&-5&B=~(pmdPX6uMQCb7}7nN8&FE-4aQLvdI1WKVJmEw15wysfeW_u>v#yompK4 z`q2GoQ2~!_ed$-753BXTA8dHlktll~NZsv^mVT1vzz+Hi?^PcYAVKt06rbN>z)4lK zx6_e3Sey8MEEc0O7rQ^?h-pCloo_0oPY?2KjGyVdnmLcu!z4LcOkby&YrPdn$R^!I zQ&pTtyhN0CE4V>9u>)uWi6pAnw1N_|B5G3;L1U;grUr45X6^&B`GHhKlSiW~brShXBHzuMAi2!d;9XEOh5FnhuK6^Jw`Sp(IxfGE1as^ zzvagf)I86w3mGfrV~yk*PRQ=t&~aais>p~7&N+0UmI6`ODUm*7iTKs0rnN; zE&UBct@)25NrA%J;^Zi4T}PX8jm^K&dSf0qfj05#WnmT}kpBwlCESdabhYWaRI4&K z(8H+l3A=O#vh#O3NJspY&m+>Vb@QHk$LaqG?Lvu(GVrpW=+cVKYM z8|a9Hen+RLJ4Z?ETg$ENg{lW#AXTWDR>&H6_&)&z?!D z-5|ni;goT^dUDu8mZ=K?m)-S^hi35kMe$@kY{-U^orWX=GZ4-YFCSX2ax8E{g8OgI zj3O@*?d(+ZtjAp%uPiG=mp{N>88)udY2iQk$e%)GAfUt;x2QGI81EqJPYc^C^Z zt*YUyde<7_IS~`vl~QmdHQT1a>7n%;&TX_G+N10DzepYBS=I+o_9GcNIWRI4S3r0n z4rQ_nDPZM|GkX!{*(pS$vj8;3Q?$ac(1OpkWKx+_!>8E;C!v0?F>^&w*RmBo=bX^w zvh_sn%X`KZ_st7;h;gC*ROh|Be9j9B99%Xx&-(g`!6ocRqaYl(m)tG`UtTjq7~6q> z{*b15W-;S2Y!%(IVuAG`>f1HT^v^RdB{>>7b!;e)}4J*bXETTH36 zt;a^}-`n*akr%d=I3MM|6KiPO zkyuMB!|iuyNR-Zenoon3)$B~%X6CUgw>lM@oHM+os=boBdiE|?gE^yY)YVz1Be~Mk z7v7m%c}njguvWV?5Quv9X!6Xim`PAJy6#1oGDX{lV04}5A2F_lTSqc^37NwVQgaJ2 zXVsjdFK{w&gQm_oTY=@nuN=*>g*8hfPIA?Yt5%4~cj_U!sVctTPKUVE&Ym%Kt!x0R zVQB0-dw67}sEuuBZ$r|DzwDQLYE8`w{FS<)(PVBRdJR-x!BA27Oiz>A);6~WP~+j7 zIusb(c$a*+_*K_j1PCDCg$A=3S4B{iy%vt+{i&$bd9I##OQfcT7`wp81E7KiE+td2 z$;3DWc0sx`>a&q=n+E%nYi)T!gcw92;9#{-><%poQa0pdfGgW+73GJ$C-}1P@0BOH z)*?3{lHw6HGi^d|H7;hS z)^vpC11o@&Dw}?33CR+h(gb~0C#{m49=faDs#Im{%>Vrh8E0_dT;+FbJpCEgbpT}k z+&1e{xwxw%Hy#>F_xToi zl7W3H`B%G?SlQ$0{|t{@1Crnrq>6-{|W+;tE>yagyMP#11+npK^cm&z*} zaj=*F>R`70l9=MY^3ZSG-r0TWUbQuj`9oZRwjc3@-X0|!*Xf&j!q}iCiY5zY z>yC4P!45#y1RT8JU%gCeyJW&x^&{qf=vWxHXckToXlwtgOIC>k-Gu5#Y9ONY;$vod zdR&oh<%y*w3V^;KY;SIC^tCUy`O_~ilkB(qePGbUHCm^<2AqSSZ>>#ehhj}|`ni^6 z8tFr#arQc=|3IVZ?ZutL&w;)vI4Ju%x0{xSJ&(u7E57r<#tSmKP~c#Fw^xpT9pbgP z=2hgusr~e~Sh0sif35Rs|re0q$d&(`9Mvx%LuV(eJyKK5Kd)|6EX~ zbfv0NH>JBkhz3r91VzU|h$I>y8Fv}$ZH4>$6|nFgXcX2cd1tcAFAU@wzROod_76+H zYOers-~GN$rTLnTyNr!Fz5;RBseCibL^u`JCaZ<}o2~2uEHY!EO%h}BDkyfoGljY)nkG?P*XoON@PyY(XK>b5k=u*^|B?q5QVs3DGL&xR1@VPBCsRo~J84 zu`;%u$vMBkQ}qYoynXri#P@X9zGv}rB_w>QWY*4&!`;86>beC(1pJP2Awg=-Q*1sqEHm?mYSpLyUlnE&^2Ntl z8V~Hx+wzM8{#c|{;VfM?sbd>W@*4-9J!*&gkt*c%{lwHeZAt&#g!woNheh^5OUZ%8 z#~^<29`aY-A%NWnt6nKfU)Rzn?*0IoQEkMyc)HQqs9Ty=a?CU6|Ih z@TVI)kgq(juqdE$*8CMQ-$?I_q7qE<)?IHs`HT!n4 z!g04g@w1|yEt;7I5HPtp-ZBL|U&n%~7a&-%_t(-#AO{p}Bc>u^pYba>FO`_OU@|k% zBRq`=p|}UZhOZ8z|6`ihx;6mH<)eJ`&A;9ILrICdTfa!wSR|2Kl2rk|#g8k~vDw^< z)`PUoZ#d?cHY(0Xs7(%8%i+9+&N38~Q_31IKiu9RHSweVT7|TR9mp5bTqC!gRq_h5 z@5?g8j%4P4CPkaJthPRRS_dr7uvRO+yU-Hz?L{C9r|!k)fUR}-U~sBU&ZT%6osA?0 z6+$OhlQ5S4l{T}Ms$JEGYGy{OXK&C>}`CMF^UEq zkoL~!*K8qZ_hso=dXBMYKTK%FL<%wB3_2}n;qtUoKBV$AM`8OrKbxtAACv`fRx_*h z`<=m+!G`>+L6~pV{rfdLJ3AjuZ4)eGTec5p|ESWJ3!9YF`(GQ&8qg z{voxljY!qWnctk9DzuUogRIpckVzA8B>*kQgK z`Q@ZvxPlyh6I(l4OTOEGZ9%XghL6?KC{x`^X1$m#^(Gb6McTg?2{f=r#z#@O=cQ1sqX zuIkn}#CpP`S>N`F zN$j_vo<`B~o0GA|!i5h3+9hay9|b(aD2UjU+0^kAtdVtuSJD$r3B*lGXNC7=|K*Z0 z1?bB})?3V8=l^vjsNRu?r|!(B8Jaqmmhe3W;(|;+e<)G`X&zkWoQnH7U93&r+b}&< zG6cUt8S*U9I zKS{KVcZfi_vyJP&6ZPd$S5FRgQ8@vos;mFL?dcgR2{r0g5A}t|K7Cfjplc0aQr)KX zJ*e@PC_bq#mho~NU<@8maRD*+dC&Pl?BKw4FV(vPHqT4Ir~dYl8&mHYC;&#G`az(M(LZxSMR zhRN^He{SaG>cne{hX_1cc0Kt=6c3%DcI!JY9V+W67 z)LNUe{ttf3lMj>i^5>-IN)hAiZzdXeECj$)(AWgUAb>hWgo5%F4F%c7SCcB;7r(`? z({4p-#$0&&b6nLGIPM;Cd3@3)kb4pGb4^tIHXV4RbSOLd-`uk(Ls>84M!i6c)Hmxzi z!FNWyP8_NP_XBBkQ)Q|wG^Lm#`ehlnt!G31xGJt@~ zoXcH*abuz2z{X)vjIB~D@DYJW*c~i~D-xt_fuMhrJLq-?NY(kHe&nTrNZc+O&Hiw^{p)|%<@A5ga{GVMETu65 zn1oCL{S^h&{J_B}m)GE6iTEJ^E`Bu?DRp|cU4UMU&OryRFzxQ zE{w+`DiQ{Wl1HVHlG05H(%nc4NOvQx2#5&iM%oQXhk(+Z0@ASw>5|yAbaUs%`<;95 z7~dV^-v1wWe2z0b$h-G?SIjlzna^Bnv#W-T$1@p&Hl^4pnGpb#MnzuGW4Q1(xlQRR z_u%Npfu-$@9zP;MmwC$FypogQ7EWOrjZjOLUi;`z-cliKbVZ2fLVaoA6%6m!MBsXpF0lwqwtNKf#F z@%a3wT2(}p+G`5~QF~b@!g5Y`H#%7X?pSK&XS|P>{6bFd$adpbrt7G^4{O5Oa^!jJ ziOz8Ubu6dpYN@@D&}qGq$QwD{F}Enx?&(xu!W*-ucbduKC{fWW_iacMjua2O7nF>9 zu2Gw!^5n-j{YW6y zL6?0qby)|GwpBsLO$n+Oy>PKvX;W-Ai4Y?|26$^Jr@r3wC?_u41c%l~64b}M`}Dq@ ze{>*Kmg|}xTiAZy)mLv!y5cz<=a$bMA9?`{P}%>cp1pFld_?eI9j$F!(Nvs6Oy2H&fEk2+u{8N)XT%`T z`pu84(gnwli@)j=0RH)+VY_6)VX1$=l04uu_21RhmkgX-3iNMp9;j)IIjuNH5!}uv zCJ`7uQdH>)N3J)L>O*j78W;JS=<)m+d$HS54ux3xX{0b=g_e^??3-cfBJjir;SWqj{yki`G72 zKium|%pE5O*|y*l7M*%aTvcL?BIn!5F-Mzxg8UO|c=>;|*SCk38CBAL>tr&9r{|XT z;;Al5Nx4}sYio!7{4^K<$90}%Q@Wq{8@L^q&s&#_4Kyd%slpAU)S^m-CkIQOhaIk! zt=*^b9xu>xymU$Aww1}|dg$#z{Z;IBYi=9~LZ?TT(xVb<(qK}AHs4)gE!!GpDH-kC zJwHM8#%ymrr0LxYmIPP7f82iiYXsVsE0vj??#*<8v#f1hHKc7@Q+rxG;r%^`o~nPS zr+&n?>U2%w)MHnGm$hn3xte;tw*KoBxxntIj3}4m+!rFjJ-44<31kEr_bU&_TxUv; zMYsAjCUVOcrBTCLa8omu?ChSgJ8`;KvDW0ZJ66hPQxnEkc5=|hGiDX8boy}rO@qR+ zP-4Q`A@A5$9z@N}C&#K+|8C$J3e9ur7C!vsK5Sny>N%JcO!yG|x$@GXFu{$9GRLJb zz8(A*E#U4J{%0Oal;sZSz!!BFOl@G_>5eY{PhQGt`9|v#YVTj2$)SkneKDlVB@@>i zO}Yg3-XBM5Sp>OG9Dm)D(sXE{B@|p(fS?^NMZ4Mq=h@8Oy6~8^JBdNM0&ATVH9dvE zX_K$<_L7zl%6tlL!hU51;8JLtAFpBrJVLQ|$It<+^AV1ChRE<;gz3LxGC2uBCE!F9%%_I{AyIdyMkcD=I*d5wU&o98DH_SN1Q;McxVc3WTFH$ ztsJwC*@#o;BQDFE9kKoA2uzejN%dt-QT+yd*y;sehxZNL$t=}wceMVwtD9alkmNjI zemc%t>U*;1Cmv=KdNakb6E4@-S$i}!9Sor!;RreQ<%Mu32J!1V$2;V@Ue1<-Ewg5+ z5etca(npL5>48*9R>fOfp9Q@ZCjA)kG5(}x6)Ov$92ZUwGZm;2Gx&G!b|r6iCZ8ut zA<;<5DM?H4VI5s8nJ}C1!u$$14`wmT&cTZcqtuROcJ2`?obcXS$`ch0r|apFKnosr zUzh2g3ItjTAN5)Jg(UFD|GAr$mmV&k^!bVTw??fBt*hPbFZVlK4P2BoQD+=4x@##d9#wS zR=pTfT^5rOQBeYd%`!H*dku*Z8aCCxwg;sK<2a+~<&q2<@a_@S9Ep_6*Bp!Qj!6ie zNLIN=jL$FSp-0O)JTh`pLwM6NRWe=I_PHQ_`g7d@@z$c)?zoH$ixdS!wA-fc-jJBR%g3JC`CD3U-tQCbWr~urJ%3h!HMW+1wAce8K zZ2RdY6KVS1j9%FtJombilS3`^h8J4sF|c+yB?}d6W*yvdd5ll~(kte^6uY}V=IS)% z-!WME>-vL^fl%xlaE9T2>#&_6Q>X9OwpJ@dMRR57KRu4p5s_ij-8tCaF5K!ZoC!@m zg_9Szx}R{_P*|5URvo)c)u+*wFR>o%Xb+o^0|z^aJ>}qFPTRWpb_j)(QByZlcK0=j zZYqy%&@ThpXy0?9qNhjnA-Az}7p32;?*mapZi=Hf6v?vXz0!s@&79u8!1_FuA4(P{LR9Mt@#HrWg98A zOMmqi7LsHA2}J_9hWJLjgs9WWzIjRV>nlM!ceH3%OlZWEhR;gPGH|tCd*5(^>jO_V z6Hb?_cE`XxY5V4lXTCazMd)}+W zExo@#WL?Tge3MbDQ1|M>1+j0wq8Pci#OA#p#T9 zk0=yeTy{GP; z?Yf$xqoYj|yaptx;0rXj{m+GxQc#SR+8g=!och>8NcbXIJVaIu+p%87Ta(1P6h^F ztp;<)gd`rMC?uQ!Zo^KFr3*@p^TH0 z6ChK0I%RjwH|IJcnA9Giux?^d6aM*6jS~|>JFoBovr5HEx?6^aLL?E*5C)BKmLAN- z%*pu&F3rhFp;Fm`N!RvgqogCGg@xg-OciDi7ZMf=1eSq9^_L9ty=aB2<*YO%BIO3X z)AZ|5%HRs-C%7I0m+*Q|j?3^!vf_HBui!@5hc`-xy zyug0STVng4thsvq`Nlgw=?Y8Vn)}KM+_w~`>Ip|Kz&DY;^}dHOyb9s;mYo#F9a*82 z{rRaD{SF86yK95>{;`i#_*^oSX$3all{+aNY{^fs!Iw^g>wVpHD$??52BP#^C-if) zv*pOu@>mD;C8qiqJEA`i6%=`tAe?&>g+o7~;S045iR_cCJ|~$9f}p806z>38M62jQ z_l=XK+H)bdK5G{Aef^9}s5(}jb{@oa4?nj7r-Gc9>zVJyEV|e_mg5QZ!r8LK>0qHb!bx%{Khxkze+JyMA!`aUw@yHe`y-#rJ_+w;-_d?7w+uN?hSEKB%(TkaL^|2- z&rhU3h@?tq`uL2M8X&;k^({$e-zy)hQ2Um2u&q+#jc`+B(TXPA`e%RBvJtNXmF<5; z?Q5dCscz$bdLkC7|N7z#A_Pz!Xs%`vA;}j^QlPmP8(R(4rR+XYv>D#r&1FD-Q1i}_UilLMKRVe<(=7TqlA zh(B+}gEzl*oL4Q&&(%eTQjQPZq%rQw<+kiA{kGWGlW&}cT{E`}czy~GU1Uj^VTEf> zbDOC+QB5&NM_lH}au>CkrdYu-Co=e47wIzu&L^8e*8c2t&;3F}|K3#i;Jy`)*KrY& zevq?kp_g`h$rN!FCY?T+-L33NplU@im5+{0Bm=__CoD9 zYmJm{RSxZMX6fr+xMG1(PL(xBgtS#{N7_sbSmrU>>J?d;Bfu^7HJ!dMrN?|%B!YSA zz!@&0lZ}y0=SWwf?}*Rgwp2}hi(!BcQoX64L>U_P6=n|P4eqQkJ?(ZPOTK$gt)iHa zvbQOONvrQ{910DZ$VzlDo5jAv@kVf7$pxvGFSQCKV;*FBd!dAE&)%2*nceNPQu5ZA zYb=*kkwr}CG*xf({H?%~y3O`I@BT0(-L}c&xgsy;?sF zIT6G2UC^kuUtbh7g$4aBL?nE1`ySc7rJeN8X99fCq4EkQF z-p7UJ;Y=DlXG088y2U>ZB@wXYU!HG`8+0NLGt|sddNMUN{Ot)(9EJ;Cvig=*Dt@Jv z#Ovsr`)(QwO>9RRTTNfCZmht_Stod#@cm4obgb%kfPB1a=clw}w=^DWr^S!ORvc%e z)P>2O;v zq7~%k;r(_R?d)z}VY`(%4ws>w2#I(-iieBP-;7$Ly8g^U1HNB6!bMgl>Os{cPL39% zsXT}^2CmS=u?ec5%&b0;*38dPp+`UBPk_1K>S>QjbA*Sbw0M3JctnG*FT4L1yYM%x zFG;ymPo~On<&YEB&D?!rurApawX|bhwS~k`j8A`SR?S43w6mX$>R9m>e{U>zmag{g z2hp8VVzIoS^U|JR+ogtfwF&~3a34MmzKL*^C+CSiTM!cl{CN8NvK20w;soeNl`_Al zTcZ?YqGJDjXu55&zn>{h0uSXI0uGK{(RZM|QGEZp%`ckCcqAY1`eP{c(jAPghmxXi zq2b6joYBkh&<@8hTs8O0(xCIC@7cIlU)N6>)Ylh!@m*wpDE7@zIxbQTD^RYmH=;Y@ zNNyy9nwei$rj@a1Cj9dfXs6-mci=0(2$nc_NtT@f(g3`)xYLTU}CX)DO+E@j*NAcZzl$_pE^Lu zNDiv677dar`!rfPo#&ImtTX|+$wR;J@6{!|HhPM4K)sEPM zg?mz<#%-%o?`t|OMdA{vA8auxiyrO_QF|^&>v_n93D8M>{(SITLQKqj$I5vj(f`U? zNmN5%e@%^aMDJ{y#kfwUO18W6$^`YUZ(CG)_9Ci@jnjS{YWF za-SA_qmI~-^icDA?1dc@bf@v9|J3)({s9)p?uR`M>GI|yJi3(`1`Vp;l1w`LCVZB* z2Umy$og{0H)7qkFK`S3`h0_$7L%ym%rR8|X;n$l;7W8Zz8ILR_!t5uz^+Yaxe0Kyh z(rGhou}>+S?f^r7e2{&URW)6Hhn|ouMJ{d&tq{wruvVG8I>znt<-Fc@gJgI>JuK4I z!1_rkil?{mNXnP&<`?_bin3^`4zjdL2ErAx%vG$%ENV*ZXK3`a%cX9pWhQ~=U=@vU z>08Q!J~~gmqLV|Wd^3u;9~18`jePYAuz*pH=4A0k%F)rvGK+z}_jh)hrmgDg&V&k9 zQ(?Nc{vXB{8x=OVe@iN-$?n9(a4yYt{7rqdesuJaE+qf?w~6I(U1YIsREk*eDne{I zLkVJUsaOGR4vyA1-qqZ>4(F9v9vfN10MXA_`rMJWF9t{Q^@5p{bYwfD--wm#*w$u2 zp1Q-A(X!dKJna%1%yi>kTUVwEa`j*!p09)kC9t%|e;RSuKPgG)+dXI^tQOyoKB($P zEY!`6n5e|_BR>jO6%zZ#+n(%7-|;g{=c8(>VfzFkl#~QUk)}U4r{;7Znq#LTDT&MG(<{IJlprgfj#!f2 zFWri;BZrG9uOFSu)l&<=&;1D}m-|Y9WxC$n$^P9ttj4j~o0DG$dH3E5Z*npTNosU- z=|u^#Mi%AP_rgnqmWM8KF&mSVy5%X>Bg;ZQ8$nt{${n#tK+IY{Y1&q2x~u?P$)wTg zjpG#aU~yPYB^{K8RQU^)8}y68Y|Er(!MdGWYu6o?26TC;dx|XoU7_pRwrF+90=tBH7)xPdP{Hxw44^b74i2M}o_i|{p+7l7JB0>1Zlr9?^{L*3uIJ$+#DK1_~BNGZ8Ig^)QKURa0drAwSGE@z^7^&!)xdgTJj^Bu4&8P z`W3EKu1d5&!7X6FfM7JLG zswq(zG!tUuoHMm-Ace%flU~7r(a?0LOeKOTh7fD_GW+5)iiq8^TKmIY`*oO)o zv8&Fb1%p4!RE*o0*t^dxq${|+{M?onyEa&lJ#cX9jVa9NX%f3ze7Q5N`L;y^M`e4C zwpK|fRzcXY;rY!=p|ohF&89@4V{Mf$%AJGnVQWuT#lY0c#aRs87n6tsW16O*5zntv z1n%Rc9x-kh7?fO{p1hja&M6kKYHr|&@ce6pm4mLkAKA;rnIHm7q93XB^ehg6mjLg> z+HX}9-_m>n$u;PmMzo-A2syI~-5bQoXQ1x;vx^M;Fck6k>BRNDGgrrEn@_P(`tNU= zk;aJg$Xq=ETA4Ij8L+ZX;hcqPFXxSv{O25JUKb0PnURfIi|)tTC0QaD&`SJ@&r>$$ zHRd}H)}NrENCsz=#J_GvEhBNXY_o_4}wtv59kEL@uB6873$&V+Q)z1a4S9F zH?C3B!cPvbgG?BQ(Y(CZZ+`$4i=q8>9PqhEDfdBD7H#VF0EQH6F4b7I)Y={}6@vHfD+&rQ&I!Y1 z8F`bX_p1SnAe=?JuQ(?F|Fbkm91WoiL=vtOkftg_QusFy|*_csts$S;M`r%2oI^zomv^n&GpTNJ)QZo`PS-rnjh5Br3 zjJ(m09PKQ1IgMPiX2Mj3KT*j=*c4^u&P6^oJl4u5M`_8?-6=slHhqyS*jNWUA6?&t zyYy_v`HfqWveb%f6EF<9aK{xHS9g+FtWW}WRo*)hTPagNCq8IzKwA8j2&0)8}@dRB$YMDs+_`bE8YzwtNU8nG` zqfe2|6bpu+68zUNp;-AD*+8OfLoskxmeeB{49+5|c?iSb`r?4XrV{8Y%Era-M)9Sc ze^gibB1=*DHmAjs`fQ8Q*(Ws;+4ru-$xV2ruT`T@nv>@XK z-~$`I`~$l0iaOLl)Z>W4ek=->Z>Nb z-dP5OINMs=c{#&zUiPANRtAID)(7d2imD$$TkWo%p%QMCcz$7k2yvR2op@UROuQ?$ zH5>Idhy%s=fyYL*1qqH$UKO~RYAJ8dV1RK@wdh!Gbe-Sgkji;naI;o^8X&qG_782H zm(s|%KXN1!Qgp9RNym>m(LtZtKv6jh3zKn6j}$X%K7cJ+Y+dA1fI%ZS z<_hUzAa$XEA09C~3J|f{Y^7WONEx@9wcUgPT%cH;ogda}Nja{IV5Kk#)9!$JS<-R#mxFnS+qv_D3G;2b3}gOKep~`5V7_ zz6ZH4v7iJ|Q#mDir;~mLytd<9l%OZf+_XdJ*Qu9 zvW;}gk}lr>FvnqjU<-3Faq6^@GVU&usbV=<`w}U5vS*}mhSmu1wB^g1`g)l6BsBuV z9txjWz`X=1XbE?odnX;K8pi`dsFuz2@TaE7#J4Bpyo+Fztzwi@mBD6>X|ZUH9)4wm zLjIhRafJ7gTjYJsPb648ix_uwCQRsrL91|Kx-mEb4gvA8ov4J97cG149;!YW%Foi# z%HF1fd7W`S*`Dj%{fXwm-a3;j6_2qfSnzsrvZV? znh!2qU{qr|^g*ZCKb`cyLdpY`jzIUDZH;jJnFw9lBL#WM)hR>XlL%Ex7O&cy4o)ke z5rIleBSS+TtDnyhfU)|zP8Vi?P(OXlOj> zHm; zr|dy!5Jn|%&_E;pC&eN3<~XtCD~dzq&L3OK+--MwY61{p-tV z1^h_33<=o8yys5pNof)*FZf+_!!f zm?~rfHfD-lWL#q0i6S_k0tUgRTfWsopTLD`G%z$Yg!zc%G?lltzz!8x>6RzB0w`CY zbOJvOElgnrd5VrWUj1UL7+#V)@DUXTd_g8Iu0qAnBG3i6Dk}$bKNLLSkfp)AwV{G$ zCkSS7!kYa@Jo|th2npiOfgJ5P!f9forguXXu55vV;Ir~0Om2DhGzODy{%V1`nP|N# z_n6M<`z+eUqfnVS)ew08{CT;vem}#4ps+rFU%V)^JL;I`_pnET+RF(Dsj%r{&X#%z zu%<`6Hlshn>AT`=tM?C<^5QzDb-IAH1cxTcqYxzjvDoQ`0d@19hbtvYN=on^9?I_+ z@prebQBesx{E|FdYyMmQbtS-KpWo~f@&O97G7J$Bj5h9wb*}Y2mjGNc5R^YXQ-vAb zKr0AA9wXD|^d!x%^v7hKpGvCqb|>H3A0fR(D-R)%A_En_xNpy#%;g=}Tr$jq}$i8 zU%ihHjN9PMMg<`PA|l?{PL`*UMbGN~5)!J((Jl!H3~c`XT~a#o{!pFYML7xBe|8eY zxOomINBgH#jZ~DBULa+><8JqWa42}E3J_!zbtY0{iSD5mn)RW(49O-UAefH4O>q*Em4*BR~m!uNdp< zT7!uN%~BD(SsDSk#(u~ne%`zk7!Yt$A_imgt)El_+jcl%dFyb0KZKHRpn2p~%H-r^ z1hWQ@&FIgGiPM_hrtjY=1l;?!GvN!P;6!GB67U%4TFaftOh}l>P)gSG{4J*PVN@ED zp|H*s&Py&`6~s&HQw_Oc*aXNNb^z@$oTqvkGiz(WgkA&E>V(eS5~G5lYWSz?_>1}Y zl@=r|pWUQykSM&wCIq^L)gticumD~Hhlz2!tv@UHFp37H{X|6Mq>7e~F5m_&?5QJC z=+Tj5Bn?b`l%Bi!-ujfuW@!N&m%~a!^W6k)dFGLV)$`b0<)(y?t#;+-b%Z%Szfvrh zxy@+l6y#Q+@~J0TA})019M+O#sbw!LEWjeef;d?7!5jgmSt+5as=CaGEO%bo!5X~K z84j49r?8VjY&0W*xakAZ(BX6!de19jP&x>JId7NK8=5kQ**#KyVg~JE4|Lj$QPfZ= z1h5t(13@!2hq<42m8$FvCc=aSuTgpbE}K`M?a*%4tgtUUr6|J&!*0X@9{#U0w0Y5Y zkY+hhJs{=atoCvWRmw+h(>j8Lx)rOWIzhb~*95~lS4Y2=v{|_F>HjeM#N@w&w znm6^>-~J4c-&wcNCewTGY}*A^YVz zG3K%M%?L86%<4aFca}2|&H#8?@vIDQE6#x16v7L6rPe5o){pr^&kGpux4+D*x;To_$cM<9$9aWGYH4tUI`UBgWRbqJ8X9srrjhZeCW~7PDd%Wm!W9Z# za>U=c)!vxz>@PGgx4-u-59`u@#X+CrSDS0sszSuCC@C$D?&q@Y&NR#PR(l2QEjbjJ zv?9eU2N+&B93PkHxd}VY@moBj{(Z2tIJB9|5`2@<%Uxji)A1a5+p5U{$Ag1uAxTM8 zF6)+qTkV_xCUcr*nJFDMl{xk&`6FCBYaI`_(ZfZ<`LhPQF!7Ug4ETgRo~2A018H(n z)CNoHIXanMhadUNXpGx4GF3ENkxu$PIWIOp^5mQk0Q8ugXJOsIxPz?Zo3RXF5{xQT zM_bQZDBi>B)iy#dQW8E;f1${HQ_@eT^dG6EfktboNLdu0P;a!L<%-tINV;uxPtc7p zL|>PXW!+!f7)K55H~ppG@JPM;enTUpH~i}ia`gw_TqEON1W?|)^1*Y!n0}9{rM^{8 zg_;cvfL`ith1Qk{c5jNjDu#>I+9Dw`1<20LUP`N4r$4)ylJWP5E&PrrE|jWBXlW_0 zKNl76qu2P>{FmfmlWs*{RmI{!|ISKlf<6QsbmB-lIpsh?2tR107Y6lj9Ga8{6AATY zs*v$Ny>MkQHEI>{b*jc@Ot~lNvP;2QO;n0_$aKx_Y!t-R<*rd3X+6obm`Pz+|G187 zx)thlc}O6p%4U*rJFPz(-RX$iZs&xQT$uxi0M*U~Fy{}+VUPesQhO#QCI-5%M=+Or z>+;$*IchY$i+!lX69*V)s*L57h!m5S>dTiq7r$tyhf;D|e4Oj-&r;L+Cej=x(+fbj z&uOWByqidu5Vfl#%;ie@m>)#9K+ znWK#ITjDGYw$-51k8e-MD?K&$H%Fp2q8RTdo_p4oCbiMkN0B)50{1Wun{rs z!-b)9fPTQKroA+W)aiM;;VFBonP*cP9fh^A~y4QEVJ215tRQfMAB;m5hYII|tus+r5Yu4^#;vFvqNpuIoa z=7g$5`=bM%LF;b>CiPGj1(SHO{L{Iz3kK*!huP zXc5ty((}z|ry@(O!tw%S?R`OMUW^vdmWg7LiLBr$w#g|pciy&0lHhZdfBCXCTFKiR zKnv^P3`>oUguFXXxV*}b`HV#-EZ5ee^a0fJKIU@!YSJ#bZa~T=-RpbqH^taePj8CS zOw(_+^B#%MTU1ltKz6++IqPW;$BW)H$Q_ist4&Se5ov7BSxk&`+yhPi@Tv@wBGFo# zMW1SSZL^COGZV(`n@I0L^iZ`{>6!7<2yS@1tG$^!Tt@m(_hymGV)N z>-dM1(X#%`@PS;lW8i#}1L{La@y5e2==#R);QAC03J9paSdY+{4|9RGZI1zDx%yCE zCCzL>QEK>Wo%eTZr>&|SqaRmMh}3-J{?C5pt`O}l4)=ev^<}miO8V=vh;LS(tdjQ8=DV3@Y;zv-spY#l1W|m20fte zUMFS8E63X!hit=L3Q=s3fRQS9mH~CKJ5iEw-F44y$jYKmQb8z(R9`jbfyMZ!y)6v7 z_Ff`;^asXp5S_)o%Yx6^ZyC3xNk^!ry`z)w1MpF`#e1;SesNSYjtA4m)}Qf#Nh8!l zeP{V#nR8_*`Q%tFIUtrxDdquW!Ts?t$Gd77?*j-S*8<7X+1AvR5jY{I>0QrS&C;YW z0oijQci~=gTcE2(-0=fQH^B$crjLx_=vpgN%~UG>c%Z+L zf`_tI?wQV({W;k?EBzHExJ1Q5+2RC1&&f9~axVfLDp^A4nLE~dG(MvMPU?w>^p&ig zCk&x{q-@l@*d=s>W4YRkNllv1by&fC@v+}U%Pq69^4Up)8>w>S`}^i|vh=I%%#=xg2ce0m$XO8A_%}q+)o{K|x4Kr4(-%s!~JgowU zye2S5JDrRhKBu1NqJ(XisL_7}X}CUF+Mq^lHZ4j|hxys#?S&&5^zePMiCm8UT054c zGydxK|D6{H-}%3C029Hl8dZ{gr=G_{S)Z&^nPD~bsp3joZ=K(f-tx{$IE~POAebk? zIAhH;&eEv&SDlwk5Z`rKrHoR?k>NE+q1a+;ZpXA6&AXf7Bc7@?A6uolWt;zJFSvek zRXtBnx5Q?QStB<^CaTlvrX={^w8KS~yEUiBU1!LN#xps;iYo*Ja&bH?)^%PfvM~=J z0Cj@Z#v%R8URu%J-HinY^ZKn8aR6Vgi1#uL3kv1hH+rkWaN)Pg0uSKZKm%Mc1Y)-Wt_1Gs~dC^V2?aulI=I zC+N!d?rv8Ea}?&EH~%(Ya+8bv)j;ghOad@u6QV*>b_eT(WBX9j#E2XP z75?Af$&KenZisc;vF#qd1?neYERV#H75~2M5aD$I6iqJwx-npR;^27V`PX?h>b-tR zE43Q%AMaRY=HlW4YA6P1WR!oqO2s@#rNNd3u3sa@-EF+qwaO^=miWxoPY^c1;<3BL zd=0e**bz#=2sO;i7$o>B?)*CykQns%w_9=LP-$IcIS8I<05(22n-t=CRcGz{*xGzS z|N6-^zyyA+ttBvPu%d}JNt52B6}$#o*$nKl$oC`1h$(>fIzYhS1iGm1SHB*@cpkb)&zCy~nwYjNX>Gm7&g0;~yBLwAYBa#NznC~yyi@r2! zU!E%U0#YekZfkqwvH&;0ao{Pp8EoSQ3jyoDC$y!%J4TwypOL;1$mJg{{D z(Skp_qaMJ&1>*4S%K>gcQii3zD1#go0pQ~bF~ik0U7olVs1n7;!*j5w8I`cJvwQk9 z`b^Ms9~;<_LPB77GkAXR1D#xKl)m@S_6s}eGg+U26(=QORxcc~+O>ke)+#sU-x4#v z6BPm60#*~-6)}H2lBDG1wV`N06$QNxZngLwP9{RmZ<3Dze>Dm4q}_J*@A7OkGLh}7 zG&HZz7LxCw46*`P$M+#^0CRFWganKPK+D_jUi(z+pVrsccP0q=03;6RV4%N$qYWUv zMp3M~vZkOY?f60RGu3m3A+9%4$hIgL3Lv8H+0iU&lnx7uJ(XCDSI*XIPF z-{a%sj=8#;s`dISk;aW48pDO=+3Goq)BG^cxdt)|d{=eYn3$O0%mhFp0Quo_-&*iG zi6ml0_1VLfCG+c z7|he3Uw48P%fUvYFY+VV216-_OS-CQE!9a_vJ4x+xqN#QTuP`HJz=xfI<$JS{JNpY z67O=1-M+(cbTm&OA!g71@KTrnHtAZD3#_LA{4~AWG>kqcNA&Wa_U6-SYT$Mi7C5%f zm>o*%d{vZgQOwky^V$qalkJ&pgXo`x88VE(C0gviu}g9aaIEHuj|lj5+9eify2F~9 z6JVfQ49E(2EVj@$B5Cgl!k)m2&SC5*0BbA2%CD1z1(6psa8nJZ11}PIbzJ6s)m6iV zN`AQVLp_}+U@3>w12J*7JrgpR>0!Mo_N~?L-^)a?8bcK_Ea6TXz)=8>^{!h&Q{%xx zD)dTjrGlqAj~ugPNa1Vx-oS}DSmiS zcJdX{t?6JgUitkEC|@F`S}n&BgadQ+PTScFtw#o|uqceFk(BcE@_5ebSdnGaa3Rzm zGAVzzi*JkChU`0^x;YxCmf%!Q@mJU8e*vWQ2+%7e6eZ_@*vH7j?wJ&AXXpO{{5YC!~CUY8e@_Du#V2#iU=&Zw+GWCSY=l zsx_7yC~XD?kbw2WZ2|q9xOZPGC0@W8gB08sJwA++pK&g>MuS(^OA8FPnP!C$>V$O3 z4`Cbg)E`;m@=c0P+v0UK^BFAFjoZzzM1NmD#amQXza$B6Cjq7T*~_;ioT#~KVABQ? z9&BrHTW6W0u{nqL^n9LAK_CuYAc4qZ8<|pQ9>ExXJkg0FAmcq@wH~hYIZkz)Up_rK z895ns!t^EipCLe>TEO1B@!&NakCuK03e-{_q%?(27C0tqBtn3}#H2vQqW49_UpD7 zxRsX%fWqpW2S%+{i2?s|xt%aLao;b8Q00*7v7_id0CB7>vfTbjTO^hfYR)e#`r4#J zS@Ml~j>2B3gogoXK%cg3@K^0raHI(RuOXVn5nr?e`L_IaNUY)f$y_U~#kR541*P zDb_i~(v;Z=6Ob=<-(k_oaNm|qw>OGRy?kS9K>O_*#dP__5sg&oN8W7A>I*1W$X^2w z55SH+I(ler^qT)2Dth@0cW3ZiXQfLXk|-m`Dl>su?_FimrM@|rhAzcA?8$*i$>1)- zUS90$92;wmaG$5l(a$sK*c$hcH^(N>LmSQ2)=0>*nACt7#{<}1@jr|VlG*B+YRpAu zpD$d2UU32|Mg|N_dbJm%)(>`^-L>BXD`OxBZCis`R09^uaB=@sgZu;|Ok8pz`{?X} zGq=r`EliOO%HbC~s!7Xb!n`%8 z53c@9QyBufRoei2U2s z`O=dtm;RwCLzBMT~Y@&qk^E)M0Hv+|0k%0aqI%BSbwfH zb`Ej;4MrbsFQp530ObS2z)E}#vRA4_StcD1C>vYkQOacwKtW9Kwm?W5RvzX;pccoI zi6sxgi@P>4^>!zfc*Kem)xgU|E9ux^lwq)Q9h-B^kM*mvz|bVUhJRIJl5=&798!yI zwu3bP0sYqLSK9pPplY-g7P}mrC{}V!)Efe*TDFVX~_QggwRWAF*rtG|3 zp=o?iPx5enBbHz{>EUS;F!hrI^yHu(C#nHexrMYD>;2V|=E$G#by(}WG^T|SWnzp<3-K%6x!y=M8R^xa`Y5tTJf+v zeTKLQ(F#Yk+AW(2^|^MkhuwUqmyQ{LSk(~=d$&I;CKRdq`7@K^W6-^N0wA?UBRpzZ zDSj8R#36`&y7OJak6ZpchB}MaMi7>h zU)#K=Zt%25PqSd-;_w~-3y=X0l+0Q%1DOdRz zng#5xiQG4r(2tPMSG2?=vR^o&AAR~_@FZ0VwAfjGr22ax-V;^qrbZ#s97|!*h)HHM z=@0cw6%_>GlnW=qd5`JKEIMXb;^;c9pz19!EfC|~5%$Mp^l(wojR2C-xaU@V4d}o9 zO}RkWhuAjHUu5ThA4t^%)l?!+Kow#rSb_C`@Q7@~CjAG0f3E-PoSmi0L0Mm=mYo(t z4g;zrVgC3a_3cd<5G|Ak41CclKjTeKa(ZU4QwT0IPcVjmF;f~V{R||odC3&r3aF`A zaxv8e25P0{-y2z8MgPGQSD{-9HA3-)V&WbE6V#Dxg%o` zy%b7n7njtILL`=~T(VHH1a_SHLhqS>R z`!qQq`NGCF?_J1DpN5QlZm~6x;j+~GYhNz0o|6d3ooREJ3f2LN$Qox=;QD z_66)$vU6$YOMirrOQQ~yEwXALJt>8 zMFN~#XrBxRs&%Bftn^iufi*rkg2NfW_D8bN0*g_?>*Ypmz1?;e4RA%auDS214t$A4 zs39!|Al(6ZH34_9NOCcTKC&e?T2)%i8sppF-(Lk@T%F(cN(m6*fbDm%6Oq!10$B$j z5sbIz3veW>JfLvpEcF3}WZO$wigz$C(PD&nx zZ~hKO6b#vD87-h86|9daZn+I!I<0=OpAS^u;X)+n2pW`N4lyIJo28=$8iUCMb)Y_L zWds=d0OngoLz>!s-x$EYylNz5YQVeksvXoT5SmnS(=h;9V^pC#_O~7X^8RYoe%i6g z+PHnQ!o<8QX|~uEul>}6-NB|1ND?#z%JddngHi*J9HiBK^$}2lX=R*d0wB|Nnh!aL z%`SQlyDv_dGnj&Pnwb-y*A|h;u9}eu7JxNJ*l(*>hU}ptR8kCnxaW#Wov)d=KLc4L z@bXl#>8($Xki@smYncCe{9=;{SZW<(6_8hp zr*Fg-kk=4ua6+Ayqf<)Rc7xzjJG{$4x`AXNrwPs8*I-gw^IBiGo1g(dK*F+_8p?6N zHG6;c!bN)87NMl1U-L?6dDy)hvnfKy(_+nER)PfN(EZuvZc4(!_7W*aa2j!z0yVsZ z+NV(AHtkRdxzy1QP!lwf5?;KXe+vtjzt~i|81zPQC?BdvZ_qO#w5*50)rUlsk!6F= zm}zEvm{69jzFe%+;Po`O#*+(P{aF<}0RhszD-oK+!SvD|b~w({FfvIa!3%!*m#Y`wsp)Hvk1W}f&!7_U_^E|da~Lqr zr4IItq@cLkaXw4wp7q2@Mqba2D}E`P^Wl42!$q_hpVMHEO^MLLu-n8bNAf3A6^hVEA9 z)3n^-#7E*14NNymM?$F$!^5W}%!^P>of%H2pniM+wZ1SNwJ%GXqZsb^_z+NoRki;4 zEa7G}j~sBT98Fj>lh&$L6yK*QM5PdrI~|up5$%7^yg+s!-*|fq4cuaKUSOk2K`9Gp zUg7gntsffBMdcG-QHi`ChIqr==xrhCe%XZ#Y4zHl)uA4?W4oN zu{fbmKxT$f_V$OHd3*4{jv%DsIb)^1n361zc3p`s*X1Ik#bBuVBeq%voo z=Ts;{DJnveX|c@nSQI-#GL<>X!ZOA(EsOVDYd_!b@B2G`f4uK|yzg^7$3807z3%&S ze}?Niuk$>wf_9=hsS)aouZbvD>=C~R0)oeq1wIZGrm_NAz`6T7McNyp!9C`bI-r9gT-=(L~x^NOC>JKsP1Oh@>+DCRE#>V zKx(oY(^;gF=ljOPnqm#GTN_v7%&2*Umkb1G@pmee->Vg&Ty0SzC|YRbIJ0;(G_I>FxD{y4d&>tywQatu8I%mya^uiP;V4CQ6ANN8 zP$+bB%-{ckQdQ6{ZAc{}8)Pm1#*D)3N>XmQyAEHnfB#&YGO00DgI~<*kRpxR<$Lk& zw<`D~AQjpYO=qxHk3tv_6nEmrq!JBINWnaswZ9I;q zh@@#8e^p!`I9EP(ccPo2ds$d-bSfa;&R9{>;e!*)ybHAe!FK%FBu8Gg_H@NK4>VwheD2-wR8Ix-7s>UIDF(%>`BM%`YF;GqPOZzjS>fW8e;~T=;x(m zg&I3A+mW9^y954Lg=RihzlG**Zj2(Ol*5$iB(yS$u09=vlFho zdTath#IL$vvpuKU8)NDsls-L;ZhS(Q&QM)qWUM3um6mCR;ZM=SB$DAFkWn`r0r2;BmSDXMvgf2y>zmycMXF#wjBy*U>raFmRBf zPBaIxZDK`W{Ar@QIsBmISBxN@T?s+pMS^B<)#KL`GTV-~UlWx{7qNjr29~I#OtQ;~ z5Wo%}DZ~)|R(vSjBtO;{&vO7@tAxZ84O88R_zqa>pb&512ze z8Tu8Uq|2&TAIEysIfW{GNeJXc^n&E3;lWnu#_7yhe8TIIE!%oUPASVz+YE|tNCrl~ zl+@&uJPa{Zr~>%Mh~Gg(_^=V#LjUL8SS@~QDc^1~awqOVNB?*erFDOrFT3GAe5&y7 zENIB>;Q5GUg_`Mpc|76WczL$H>4m+=7z4}oZ;h*Svxxb`wHTSSJ512W$d|fw7OMxc zdX8>wTHU$rjZIIU8PpEQjO|wzFTQpaemsUKf1Xu1|RGZ5$ zUfc9sRMAk9K1tXMC2P(Vn3r0WF8dyGY-Y*L$36lFMWh zKoS?u2N)uLWm%m~;E3536eHh@2pz+#CjYb5x`$Ba&cIG8>Sn?mlnJT{Xr(2)h}niV z6$X92nH#rDF9UPES^(4c00v z9NjOeXHZHwzqyTQB^l)SOc*jnG1rV_!@BWmRxxJ+=869%#y$59l`4hvA|t!4i%PR* z8iu4eOs@S)JmSD&2f!_tp;uyLr3kuPRJYx1+zUm=RvuVZ{X1Ty7CiF`?TEe>5QE-w zD4gPfdt|>sAb*}|MFOrT%Xg9=MO^LTU4xC4nY8E*&!Y?bf!HWaN-dIz53_W*&hptmb@1x zjxw(fcWEm}fmTG9@C7M3jisa~X3AJDSacwWU)dfvZ0q-3Czy`DfiWjv3d0Q=gv z*x@|%Om#byW7ETA2Sx|+5aoZqVCffzlzMp)GYl~ieFfEQ)XLNX!cMi0dvQ_mAAMW6 z4}fw9z11@Ctbd&LK}#-X2?9{QX$bOHx}QbA+zaCe>Xj9TRNx>2$1A+dJfc2G3rNCB zC!??olCJ zpv-k4ufa)l;gKlk$E{lFS9?kT>ExQhA>zs(Yw4;yPJ}72S;4Kg3+)>BXxz&tobI<4 zC_Xq&@BD;2VF5$8_!&6r@D{?n>VQ;j5%*b>BR9h8QdH#ywCo0A+rXaE(n^!H#xPMHbx(Kjee$}Vx9!I)%%;eQJ_!Yi$Kg`UU`jP~>8YL3^sMb{2!9XuH+NL}- zt^EF9O@9B!!^{tNN|lQm-$khVDIC5AT?NIG)u`zFdLF??XBpC{0w**?%5lt9(3#@_~3OVFj{GVmIb9?33UK6UeZB<)M&BnhWUlk4$60l zEuUYa&;nU|Ybs|k5APXzVS7wh0}o!d)lAXhEt|jzlOXVmJ_ZN3B-Vc^vX3~VtdW=E z@bl`>=ZEa1PB9G0U~Rrvu@6<*a~{04w#>{>&bIWIckjNM`{S`2%$qyust4QN+YE(- zI(>iA#NJ6;3fIW$e=Y4EWmvA4s4YU9=^#nFKjR5x(5L2nkzWoe0~gyP0YNKX@LYQ1 zt7bWYYXmLOWwFt>rCSWK{qWPFw`D8?8+BUvT9Mt@x0T5X>cB%j-jQ!dIb~o&p_rR**%nl;`}z)$ z^XdRo`S-dED4;jwUM#0Aiqp)*Z`0&MTPw60=KB~93aWUPTTl@wkc_aJmo*-0ba6I8$1GI`$E?ge-^6Da*c{E&3-UE=cc zayXxg&tT0KhWO~!-Ku_37_d#v%%mBV7T`AXK`U0L-%=rN=`-X#DCBc^{w3&j$<9vH zj7~}E4Z>%j@A<*BzHi5l9UthTf5!6;E=&)?;5tck;Sgi4?@ACoS%WLwyLd$4o zC;Ho9s(&s5f{-?pVi=a)|Lb>u)jyX*(I`p%9s7UXf;0gNC);mXew2$9j~LVnEZs0m zd`)w6Gvr8cV#DctV@H5OzSrV!+%DM2jku|QnMeQmcZZ~1UV9l-nP63Ru1|Q|bCZ24 z##8oZ&YTIas;*|Jo4~rL=2=8F8G0l`NOoGzTvxNR%i=wRUfy@K--t?w65Fh$LGh|yyYj-;Qf!nvXR>J5+)5u$J4m8x)FKF3-=mMT>gxtXZo?)N_W~ z@SDd}U$7|JgG)ltb0G(Bna-r$wjV|Q)n%{gfsc@stXaDrY@&qIet*z)a^RmZo8AkC zBc3Ig1zqB=rc3#wI= zpzg8UC;co0jZ{p}$Zx9=5a2CLZeFIZV(2Mx;p5DXq{c*{ee?ml>3di`iJ8HH*u6>q zz@O{aL*}OTwad2Bpu)?&#lBw{@WdX^!%C2pGNgreS$8(b-sln+x;A{5H+KKNeey$} zN0yhNhvGzdoPJ9&-(v7ZylDfHhS1%028oAN-;jnH5}ICF3{kir=`=*8UOPp%bGoA> ztzqmoZ_t)8epi%hzgchDeya}I1@9Si>U^4^OdwkT-2W^if_AxnHlN@>l%$e#>o_Zy zp!d}4k;9FJrXdbPZ$S&~7vLHVwhds`EARYtM*73txKV_NQLEFYWXG|(s2A+0;_Bw^ zJ(Wb|KXJ3hY`iP_%8zXpEm2o6FRnef$-XU6e%Is2bFyl?a6e%Thn@ci!XJG2{}p8v za-)744oa&`pg;^1V}rwK<8OJ2;;^jILObr3+KTBqHp)nX4{f&7%CbwVtFWsuj1x67 zjpR_o(jHC&E>6?}4hf&xGSgSGL>5jBV$`KbxU}6C*G#!C`Lyf7JP)!^NvAJXnJC@1 z5~Bf8+RlFud;InVd&8^LfGtQ8^a?c6w3dEuhpE|C9qxlDY+dq5^w4yMAz_HlX5w%!i6G%4~-XMzjKo}+66)781DkYeeY~(9f-Xe ziU?IkS`J#Q`sCZ%NYPFYr5vjbG1JAt>4hzy1PU(e|YL5wpi)zn> zQC+BiUJHJckiO+tUW4M_{mFbvLuKal)pjaTw7Zx`QR#W7?F9w$Zxgn|Qx|Ve^g_#^ znU?|}k3tTM`tniu8eFG3E9GpI{Qe*qbY0t5=t>HOE28H-Oo7T$p7BS>P)FKgFX}jZITAR3i1LFlfqFs zG=IV&-ij2G_uNQ}RsU&4z5Gi;`p_6@6r=+mc600Q-6h;Q(?srZjgTt5Ee<@#;n%jk ztCgc|>P~r3EtR?3?~!s2qMkaYhD0w(xh*dh45Ox4Ix-$bFTwE}NK&g0UC}7G4JRO< z;|I<{6@o%uMhwbA10Ux-ou52`u}h3(>cy>i?+FT%^EP`rauU^&wWxF8wS+@=02r5f z_cpJxXs`PfD|+LdBa0->^cNRG%d95D7PcaNQnYM6yy9u(qf4Dj^>qDydu~t?^|Ou) z;QBtx24T|9mndWb1hXmj9giBSZ($kA)_FX(A4~@qasE!9Zy_&T2O}OUZSkw#?4XJb zjQU^*$hO^aGWVEip6sf$&7Z!$iJCLNcF? z&7>0LueJgrjl?OH?(i$!ssl~ZE0*fQ(?2~W4%QM?69hk5etE}nvVzNERgtqE?{e)hIC>NY zB+YHW&fUDLPT4I_uctJTG8V8|GekvGVXYj`t)b~#B0Ur&y5Op3I&mL zC+4;Sb_pMggAs+`r_S;&*?$sz@1xGeOI}|5)9AH{?k-^+wQf&`<)VYSeZIDp-&3DP zoXUmF4Gy7Nlf8J3Uv(z_fqVz5E321}3nBvZXgtCkX@UX;n~04fQaND%^S#tbThB-n zgG!%DgGXG}C=9O_JDMrwh~Iut>e_rjDQBpuuy3zaJF4z0j)OIGwx1$FKhpgauKTMF z#zY@K=!8S@+r2N4RX zk*e=a)dPe-4vLCNgJn+z=e*W(sc6~nIs^ZBg|MQkFZpRGL(oX9a(TSe7pvaR`h_(z z^*QNVc?3V{W$k;PGI5n%u%6@8SFjFqzNgRkh5b8RC!6cyRx`9=<3$_d@wso)omJQ% zT=cQC}~~Plu_>12hi(2Ga7B%n*NfPHWv+yCR|Vn zK0eAEoKmK~xAHA7#a;`)Ext%i)5$I}*dd&rru~bQ9I^+Fgg0)`3!JbojgjK34{tc6 z`2Ky}Q_`Bj+B5?xzykF3=bMpP@wJ_(S9s-9xW$rFgy3%1%cI`CiJr?-_jrLVJpNlq zkM~pTVrAw;Vc$;AePESC$pAh`z>jDJRn?SU&w09DIE z8BlPls?z<@hAujMQ#h=9?ECE;L4SXK?PDS%Z;77%*xf8WsEb1N1?MaxE2Q}j5=$%QU67WXrv`dQp|o3is@|! zdHJ0!L-wZH^q@2kz8NfpU2Qp~B0t>Ix_JX$7fXYzTR3>@bGVOg%M9Mdw!>+X;Neik40bWQvB+lP(x8mz2fUu3AMsir4HdzhHKNs8yrXw*F8XJZGNJHavj^V7n7j&CKD zwqN35IN7`Xs>?WG&DuPh7_bB6R#J%iu_UeB={1p}U zEnMhx4Z#zDX?omyqpl z$AiB@c5hz|J){gt!JPXYoufzRxB(=_Na1{v4*-#j?Z~s?X(1&!no~#Qw=;Czom!$( z|LN|+AD+Usd+4QG#89-MV#lDM@V8TTs2XzlE@y05eY(h%Z^?g5t=t3d-PFiaj@?V= z6jRlDD2tySpIMwKr7DbQeDzlTBf_iwP<{c2{yiM}mK|Czj{zQ`Hgpx{`;^m0%cijiUml~2j*D?1wPo~_ z?}1G5smf@W+?c?}n}Agthg z#>G2~>A=DX7GqMA7?Kc<#I`hU(G}kwl**Jv&hJ)P<9`+*Xpv9Mc>;6nRzj@0Z znB|ta{`fK&CU6JqkyRPK=`x>Bxsy19BUkL!PWst=n)FR@js z(cQsS?J@pGyaGx)(=M^4x!I>Vnvuu87j0Pn9gG8au| zC?A+^Se{-JLNuvI^pSKq`3%F$dl%Jbj%)_Dxk=qE z=Sds6K-06`KLiz{CG@v&=!leEN6omcFw=Qyw!F!qFzm}jk5=;=tw=%Gg6sRPTH`ba zrB!b0D8hgnC>LT|$LoxcEr+B6fDDAj-Oh~zZD~Ylcf?rf^S!xO;tBvGAoeM-dyjZ$ z;5@laoA}4NbV=uQ(}why$GN9CChyU)yMFcC?Q&MfwoNricu>bn+XK6Awj2TlpGQF> z>-vG9OG%dzF4Cxc1N=#Vg+aR7uJX;cw9Fe2 zgH~C#=V(H{wSZ!->A?ji*|V_+t@T1UoIB3^L<}E%ikwFXzaa9;6=q zr&*+ii`DtsQfK-FguJHxAukExC~;{}^>oRA$#eKYShxp^5FAWU%k3$pRI>ta=P#A= zxn^6oU$G_0V#ODd?m&AqEzOB_kderq{5?cB?`ce!Xpuo153 zIc5@^c*3Hfh8)3`-QBIj&@F{=adL22O8zBX{Vys}VcABIbRK=da2pccTmASotVdpk zGO2_^*4DBWH@i0sg?9YoG~3_!n`~&6ogBl5W8vg~n{;dgT6ysgS#jJnL`?twjUgU9 zwj+SbNZQ}k0=M}P7xd(Cs}cq_0!oE9qWd6Nz()YoO$27%!NsTS|M%Yy=jC0)jVesP z34VVdnmhbX7-aZU^s5>x(i>%Y?e+iRbC~(!GdLebhvFalukQ%>GrQo=hv%y?guQ|P zxpf-!RsWxFUFXDQe1Z4Bx1Pr*%#wgjZOOZeJF_}`7yyo3MdZlFa>&)20l4iYnv6du;p$9}j3G*Z*Bh_X^9ED+3~ZxANlX=xb|gb0xMt19 zE5)ADu3cRe+$UV8RtA1b-q3 z_)^|qwA05QrE%F9#QfdG+js9?glE^!&o6UPV}m|1drzF(z(c_QQK5kNVSO3k)=JcO zg+eNO5;nGpb7S9cN^b3Lh0#tA3IbH=>;JYBanPlIcOc{((krL5LAmz@nyOsuM;!&` zA7^xTbvuMxG4vX?G$%wYM7{havjP8lk$%~)$ttU@>MHBjuU}z+sPG#qmZb~N(CJ{{ z$tlO7BbtPZmfzqdiWUa7e)^th0rf8=dgRDx?Al?x8|f#LT-A1?&#uI^_xX}XH4*q* zbqx3vxhN;A_&QHruUx-nNOr|77#bwu7}R|_^Xi@{7CQaj{q{;s&gzsAzxt7aJ-6!K{`{m*UEI=> zpEjLiGR+#zs3~S#1MANHcgpECZRsPe9cv^a4xA5y2N+sqQu=0?*Q{A%Q0gMK3zCa> z#$}EH`vorpAEJ2>^%9k%2k`Y7C{9Xbf~txOJy+R%?3>crq0jRy#~}L+MK^*Kuaz?I zO#^jooKi_G$uTX02pd%_H$}pE?S4Q&loI`H5#=(1xB*Om;7agX?i52Z2j>VP z9qdbsynD|ehVbCiz)0?>#vcEK>2nAY#Yge@_iX z+p=)Wc019mkTBjVq zEXub`bS6WYQWG+Br(hrGQnaai@J;^x3n(AFec_wGS!8%vwdGU)KA5L%`iuU*{?UHi zOSD@Jb9Igh+R(xJWZ)xZG|gj9mo9f{NqG3+%l(b0_xcAA+#O+DLC_M5*%&s2ipA#` zDWsMPp?mptvR;pz#2AJCX!qZL1bixhO>lT9ai#!q?f&L;C@|<@^Q%+1JOq}OZL>>c z%idRO6M8zcZ-VQHSYgr-)%Gd7o|41|e^Gri4bpaRchv2SI~1b!_9_yi)rQAs&=|(e zKH_lvwu5xxkv1NG%C5Um^m9yFhf&9;r^rhIl)^7+gF})td3Gcj$s?Npy5DGxcP6d< zu?jkqGda)dISW^7Z#ie&{K9Ko}qI*ig zz=9A1Xz4vPFywq?0oGlI2#_ z*muw}&6-Brv_T%(v)`^Tl>z1JgH2Kgk zFz=IP6UkV=rS;9a?aKHDp&}(?aEHFZWZM8d(HPg((!yn$ve_}xoHcwxK$Lp_9zNC1 zBUOG|8go1gX-RLsvZHWSg5W|k1 z-cuxkojL~-DfkrCMQNe~kq^zh1B1-=s6q4*cc#NHOy;@q1R-giI9Vt=v@;2}6ovF| zNrosCd+Yc4q`f&053sxU@^ARNjD0!YfWmU7r6s282U_s-n7a{*tY{|w! zRe1a=dsQ_sMGwl?nI@|)@m~$(rN6kK4;k-CL!IL7LpWGHc>)H>U5gfttk*DJ(oC~l z^%2q?m`?8X7(0p!qLxxRX*qF?O)zDJ26@wn4sO@8*h3LDCs1x-4pYM{X5L>X;~A#- z6w+htPctO-!fha&fNUSI#j^Qqf`TXMywCw2$|4C29@uI<;c3ekFl)h@4JCcu9l$04 z+90^Herxmm-+IpJj`tFwG~iDV(uev}PQnJQFY%DbCP2)YxHfrvGrSqa%pU`9f&sJw zt7dviPpLB9qS&z3+26Qs2k0R?g`h;uF?0QHQw}ywXMu9WkyLQ6^-U#1y0Qep9r|M( zXdO@sQs)*w>_KywR-vnoABij97HUGK~+;wWlQVu zC|Hi^wChMg7zY*5WG{$-bi4Hv7liba!VlsgN4ujU`?|;Po%U_NEe>}nBow6ufGXE+ zaQH!I%rZ3!F*i43cf(|W&|^A4-St<(ZE*Fz;;Pp@&Rl2%`aAeyeLip+ zr~-AfaWB@TxHXt_5$-c+nFXtU>pj-{xy}cfcL+mA6e&ch+6+9G?*85X@|7k#bi4)D zdbtNuP@UP)-s!Go-r*A|EkRJVb^4f!fdezyp@V-JV^Te$O02mp5Ew3f~Hlt z1o)&loKJ+MJ6lbJ} zw4+_RZuj=xX_%7bngJ&U+xNAvFE+=vG>grl=;wi>x2FWESO(wN)A6*i8;KKM?dvu_ z6Yfc1UB96|S{q*Azz?Xzk;96=-)~s@u>l0aDu48j5?-Mz2Z|k0#ikQhkyF~;3@hQK z3y6tnB9$7IKA}N>*6xwk0ixC=jx&4Y@WZX!-r$%fXcDvhL{?^EwN@-J{+4w_olQ z8Co`lBc_j{2s3tvS{ASe`>$^p^2@!|KyM8AH4!#)PR}-A{|7DCP^SDhzNliq3K!Ov z-&#nT81!x$ue~B#4nhkKTPu+YJ2&2ws^oTLpH!(c=7IxkJ<2FCL!5i-itO^2Y7vn`8O(#c^h!d0&BbQo;vP{~7RGlT?z0*steW z)G)*TveyHv2A{jKO{V(qt?^!q8s#NJpY7FnPd+zEJ-ZI-HiOcileUr#mFLdm??AEa z2hpj_Sic~}QWK61=nqNnnNbvYu1S=claM8f2;q3uJGR^6*tA%maFrN29R3!)0OIdP z3b$=n!L*(^Z~&)1wk5teMi-6l(*v8(e;1t{H?aI8GLn4LU;n`a#5k7CRYIYdX=)>1 zVXKC6p5!K8IEf;E*?9=r4Zo^VIHdXE2RV-3E$H8(mh^A=7z;l106N=PbpYZX*}LE? zW|?AZ^q)^0XI74e3LC~otk>HhUIuY4QfQk-G%u07wxxN1JX#Bs-Y@-=_PKD?MC{PyIzHT z`OX>$Kz(T+ziX6hxfLM;n)(5;x3g3Tjl*N=bxCLlG3T15dV9jy?~ip1BEIna9o>72GYFPq zeS5`>uV0RnMM7uQUS#eHPV3+WFn5Dp0u@?I@^fxA?=oUE6T9qo+8DC4EgXj(O*`)U z9X+TtI$aCSNU+QNCq7U$x9lk~cXd1G;{y(c|NX<#-HcV_FfYJtYzz)Qd2%U;g%fVI zZxUiE66&8rm#{SdykTh8`@O2ieEWFUri}~@Q&Bfq>cZBq(lvFH$HFm9|4>aZiv%rOB?Q3yobcEO(F!4C#|ek)$F) z7q%2>-Ssyfy1Z^Z1QxqY{U8Z}hf(Fq=m!HBjtc6+2xzJjMMOI_2fKvbZ$bV>OUYtAEDd}oKhtx znLm@ioY}ndq}3Fh9hOg$pGfO`&jJ6RZ9@qTz4@)abV0t454Z*&jjtOfT-da^C06Fj zZm%UL`1Xvp#T+|%jmJgvtGCGC03pjRE&Cgf>uILE^WZglCRGTmUiZ1z%CtffQMxy` z37zqX`eyVgWEZipxf-1TRjWTL_6u9K0mBI8o|`hJEhckFxGeu;mw4769b_W~|9wN3 zr2T)i3t9JmF4@`#M?@cL!*q3q5kOVVjIo6$KW>Ow9jiv9!}>5;ba|!pO#0=I*npYH(tj4m_G} zlo=()DShZsimtkXjhJ0$<^p1dht5bL0C&S*-!+8UDjoo5SD3?iRozf2eH#oR_ z${E9ZaDbl~Tk~z*4nDs<{OT8^TuwT?>+^XU$9fPG%D`v*wgerHhnqTu%t z7$wzgq4$oVk7tGF=IZ8z-!tf=aiQv+!3Xi0Z*#+cL(_NtOkl|R!5eb>6~Y*E_mxQO z?tm))5MIAMXwj9bE|o5d&NIi&=vM^I#@&GqLQp!#zq@f7v#6$)|1@q z89(#ez5YX{opE3T8mAEjwLTwBc(1(U2;JKH9Uq=*ZX6zO;TkEAc{a<>jiD(F{pTmA zw$me^?*{$=(Of5BF{gM$1G%`Q>ETc}m!fD1G1Es5B$uBr)3>GB8B=c@$8p#*afDgI z=Av4`ZPPdJh)@WwN=u^-0AG%q0cX<_l~4`85-0w0zo!fAMt++-r4wd1e^6(?d-wA> zKmCU^_WS;)uBYsgK%=|0HSgu1l?`MGIR5sE&?jReAj=}6+a`H}2L0L}cCHWf;(RN5 zQaug{p=u3bhK+rVF<+s+uD_^^T7qwye036s5C3{e~MlV14Zxe$& z<>;|49v;d>DZGZ2qY1|Hl(AGq;va*7J}`=_im9dUsg>!Qcsoy3;Pe8; z2fuoqsd|MMaz|=;v;-Zpc>55i7+;)p>*j|Ko8gm%e@cHz&;;-jjwm?C1d-KT?d4Z+ zxqNnw_NUfyAVj zpJUrs(iWZ`CKu}YGYvSCt3I6?WD_B1G5-CgZX||3Lb@0UF8ZIL9E(;|_z477K&TKf zu(3t-FMyP!o1;@732xB&b2^9+*#veLX1a{0Pgdj#)}t(k?o?>l74*pK-R6Zd#xbnsvACElafU$0f#D7M>HVnVm35~l|_MH^*6W8wvV4i*ue zPPOtfTQJT3CdLL0mwje_o1hI8waLiW8ZRD{rbv@~Yb&6Y;PF`Z-lhy2E8_j4>ot-Y zou_<#w#%-4`2@|})#Z=7B5yUkFs>}V8HfP@%?lJVZ6n=qXdvzxDHYnZ_rITuvue%y znv|bmgmDG~?NYS(gt0-14hR#zD=VH#IeGP;Q{F+{Tr(** zKqdV)6EVhiqaJSswCM&9n5VZCV2nvCmTKw`Ri*Vx*mi}CBXfQUNL{RE8~QTF$sgu@|fw%ZY%wv{5K z`~AndQ=Q3Ua+PyYu#N!nNxxutyb!$H6UAhHP@{Uw93b}Z^c7aWfJHWC2V(!4wPSvP z%*|gDmqycgla!;rJv`(ZOUD-%XAtTr&h=5{%(j4RDPd!d)=Ukl>X=Mm;u9JE6g2I* z-}NPq3NWp>6fw44R+9j(4*ZL+#7s!66xq*(J$-B1z-uvYLv4tchbtJ3QZaQm{9ub& zoJ_y}?f>Chfq-pg##>fA8{Su0#;9FaF25NzhY0;ytBUiD#YMC}V(d&lYgQ|^?i}jo zi4Eg?8W)EyOCKJ>skyY`+}s3_ONMSuuK9J~qkUfid{iLxg=<7Z|8=Akn|Hwr|BpYO zNlY5}ZRe14*}w8V7iDGHi*sXhH!t9qeSGW(S1Hd4+S=zntTnvYQv&f2(BBV#L3Go1 z(!1nRpPo_^RR}FSuyrS3ncH>`<_u5B-{W7xJ zQ7WbNzcIH~S1S}fM)~0uZd;M<8@)O%LG_6Lk59&fdad#jn)Xhr$12UH&up38F(|CI zb+4eJ`|pe4CKhc+VDxjdc5GfNZLVLS9Xb8j_d(eV&_UIc1HHGe=w_j|0gZmF%v=XK zs!1{GMChq7=n?JAT-gGCeu|-tA8*gM0y_LXmmHy;XVzrU@d~yvnH2=v3f06*V1f+R zd@nRAOWDLWp(h)Ae>z&M?jL<*vuC<943dIP*927!ME_w+eGgjKJ$i~q?>P4hW3?Eh5<4Dx9!Hs zi3jTx;a=XRBp-O>ga%O0+q%@v#crzM>zT8(Yk9-^3WzA-{HR7JUF_jFH2*wLS2VSi z6$2RmkXGb|-YbU&rZ>X)d5=Wr2B!uR@3F6jc{n6@m6EB~?XtG78nSPVhsW`o`{)r` z^Ju;|`bY%oB*#tB3jqeqg~yIq>!A{aR;UVwl^Fus+a!FR9EcKE-X$_RUP$>7$RX2N zar&Uk^3B?ba;EmRnX#VK0$A6_%LCtZ97^%^Rdl5pELPs=c=sXAOa<-t01G$2Vf|+N z_3I(!vCpT+;+nY6pS}}*(6Np*3c6fjCOQ_BdrWtCm6tBlD71M5;Dt7GA=%_$En*4z z$;NWO`Q5%_I#8toSH@w{%5VMsM%)N^8h(EBp}xO=VqPxI;uLBp+Uwi&6=d3%G}R7O zW*FoesdkyfJv&?665c@^hk?;)6aCp!bE|8rshXQDREG-C1XsZ=BSK<)X=$F8>yk} zuK2wiR~?5__=oOnI%&o7`rtm6)L)-Ap8NX;m3oiXGBxGU?U`9F5muWok}oBz@ztI( zcjrEDYd%Ryl5>u^YD1g|JJV~D{(#5bkE@e$Qibvpw{#pE2!u)ROhfm?jM5Ar*pnVl zZj{O__?|Rw-5es3VO>ZpcC-o>GF&wX>4sXH+`gE>5cQSK9lm z^3&YvRA9-%faF5|(o`U%F{3@D>sd1@sipYCh0t^r-;#_Hf_P_&%lE`s&kRGKVr%0T z343&EZV=BWXJ~42FJA0=={M;`|Mm-E8MZ#T?bl{PY=> zS}wl5@npgh#fs_}EnSQ*L6m;!;nx^&tIYDq7e7-HM&nje16YW-Q7yybq{|ki>u>+4 zB6ti69I;+~anllZPh;7577q%cO*_-T#lmyCo}<)t zUQ43;i_6MSqt*O^PQrMrE~)H_p<6s5#&AE=>F0BWN{5tReIr=X7MnQpd0UJF^Lytj z+NXXH^=++bQ-Q=bZQB_4p>37@%QO1PF>^J2VveTd+&F7unS)i2vp!!;E0_=Y*Uylj zsq2+{7Dno4I4}GlRxbY%DnP$1VuQGq_fH-)%G9qH>4ODe{)G@rs)0y(>r!Wj9{0lO zG~d@@;r!2jVl|5u+IY>O)MvY8_t(*mOy{ZhM6)=98@j-; zZg1i49R)RVbp_`Ax2;Po$gl6|wIt>15!*$4n)MtT+8^&H#?jK9>xGE(<&2Lia9eY_ z=%f+ymqv-JR9~Y$O6iNap5pmlElu+K4PD0Sdu+Z*E)BbUYuwZAX-gjDwk76kk*BKl zeQ8!!G0u7k7lfuq4f0aN9Db@rZkM67wwI6#v~&|GpHvD~7HYN9+Dc{`+igno33|~s zol8xtUgYsO`wR5vT;k4Vn2vd%uArV`g~8lCS$?JA^{OXK(N ztkp})=+yHZZ-;IB(%r1hJM$_QK0kbl%X%TJnnxTbQr_RU@ZHXb;=FUK zs`cu^Xd2vqC?7@I&+iiYWfL@a%+`=HuFv;sYUzepPdygJcmZW$(AG=sTWMCCmYBmW zT32O$t&YLFDu3s9Yn*?7%5T@RHgX>Uyc=&*6n{k~zxk6SdavbA*HGTA#ebE<{$O%? zdr(+W&ns=(szYWTLbCP>s&MvSv`qr7omL$lWasNbvrNGvjU#o>dBuM9t-^cNr+g=Y zU{pEU=lf7;By!!jTQdjKmk6PQ>>gjU4Ky2K^mENRR+o$@A-1ha(d~5t9fM&Vi=+B_1;1?iO5fij|9TGB&rUNdr>!l?9sTc> zS=TT*l}W6xULto?vq~J>Xi#6x+Ob3~Y)ab6tyS^qX4Xik>F2+hnX)e11^6X;F8pwj zs-B8x)$FS<)Rw8M@)xtxDlI?DbSL9B|NE(hLG)DNss5t*ISP8M{H`DU4)tyQ?;AyD z59gY;RrxoHGQB%y%>Vvs2AA%TknEl>p_AOKyVWXJy-Te_YJ^t0Zf6F!Gp;o+s!{B5 zOIO0%1FV8r)`C5fc{U}fJp~sA4=Lj`wT`QGVq!Ar-g7VBb+YsNy%JLFg@E|9B2p}e zc)ErN)agT=>+YZWfm1-tLe10W zY0~FtZPz{_cBbpP*|XPQU}N{rp9zdta+~|^F&yv05+Lhs`QfkkU*s>xUJKs2aXpic zmf%75g076Arc9~Hv1Q}oZHKO;|BAFCT~8rA;aMlodelMRF}@wk^M|}8grg8F4a?$ zHD)=#hka+44GSh5qn-923uYH$eGV(sF$3@Kk28rcthp5?=9!C}HLIbL8u}WO0 zeeKe?ZI^GoQ_>#W^u-yg+2JG-6O&~9*L%j)ajV8xC6g6bxB`Zhqcn(G<3%?EZ+rE5 zlXL1vBeW%N4LXHR4_e{Zi1|_U*(9jM8lWseWcA6)D9d>o81GI$w{NRX`uYamUfyh! z0!2!Y0R3Zk{Wp`}1Y7obGbZ6VeC$_b8f`9ZdQi%poul2dZQZWc7e_hsX`0%^9@osx z!ow-!EzLgjr5@8CS-kx5Qonq?7f+~pL|>*OEmDr`94<5p+F0c`$#ND2d(RW=MKYfo zO}sPsVqX)MY2Zo}Yd7~`VS2s&)SvfxAoQE zyt9=g+cCSB=`_hB_d5IWB-76+rIQnR#^zt|Z=`%x;`EgHdc$bd!@&N$%jn7PUW2rQ)t1lyNB@fZgQI5Y_j4`oB zdJnMU&_GC0@-w_~cR(0)4%70aj{k! zWvFt%F)ocSHbrV;_?5f)yEo^>%&&Tns#BiEIrUyqkt+|C&cS}BrI$M6QdjEu%~;GN zX!S3w;!&YSQH{33N(0wk-KTu6RiY;bF?E_1q^Q4VD_5xmf@9-IYs_z9>G^9h?xrD} zUGy=O?_FZXrnpDTD%0S(Vdut~FL#6r)1BKb`X}dK+#D^Nv8;KdXH(qRD`wkgU)!c@ z)&|6OS--NJiD~wM2RwL<>G!@L$E0=bY5V+Zq;J_&X4Nak(`IeykhRCk;!>e+ z+F+>qW~Qt$4}CNCok9d`dg^h67M02SPlh!T6gsx6w8prK=gzmNWF~}VpzU{?X7uhm z60XDX&K(BMWFlj5NY?A*HDi1{5I-PF%CF(zl-U+z1{YY{@{-xF95N*@*XR+KxK|V4 zhg}n5Wy;k1L7uhk!k)Gs*A70}=GntcryYB3ATGP)J$29do3Zofqny%PV}=EcQmryl z6}T*Z3*W`6RPo74Qcl}1&k^mH*lAO$=Sy4q-b7z;=B-^@0tg6BmVU5Wp+TQx0}JQF zE5{s7t+US17HXZpoY|t1(_tv)>0;ohTzzw@P`D(8<>YJAhToG?OoHUzGoqhk4ah`o zJg=^q^YBHd@EF$&1+G-N<#@y;KNPXsh<|%6T>Msc=4Ak%aW=Mxa>^*#XmaeM%bGjF z!<>TNIimzIn`<%N@SC6K6laO~6(lgaL?(~rMrZmHI8J6&P>7OaS$e3-tELbL+pnas zX{XzIr;VIqx+C1r`N~-5_O~oOpT&~#0+y)2#;oh;eWamp^W8s|c+}R=xkNx=V);vf@N|>+IWGL#}1~~o84-I)8f2cdVESZ%(~dhEu5kK5}{14_RjA0O;CxW zX9mBXl}wZ*Dd-j;GVq+x4dl+)$#h42askte)9I>#gJQ!gi}7U}Fi9+@`Z=TuLS;)v zR9B)~uB6=zP2IqqzCw%UGSQ{@uBgb&UR930CXoJx{?0n4Z0`P8Bq7QrPp(#6kuB!A z@aE=tsV$~kOS1N4FYfk49PM^Pw00%c;Hf~laISD!7K?NiUSfWS4gKEh-D;onYPKQ2 z*vQw)pWHe-GA_%?Y4!UjHkT_>#NhU`nlZjU$}yu?0@mY{G+I)N@yyU;Vv3xy!Gh9E z(@eMkmRM#3=S{Kr#CxcG~gj& z#!j#-{xw>)nfxPAsHHb%uia_QHg}Skbd5g8shpZnqRd=MEti$YGZXnVP3k1tNL@{i zX=gn2jBt~vpv$ia%J-)B2yKsEEeX4!mz1{5_88Z8#$>jweseyb6iey9=iFsY@nbz9 zqpUKYpV@{0%(}3xR<8Ho%f#M|Csa&_NRnQ8^sniVaRs-#Z78}UySDvDpoC0a7hpCq zFE`4lrbOR0+LD-zauX&Pr?ZBECj_A1>DPEZYk2QRgtp~OgVnDHa=X6OqN}_2c228_ zZiH8ua~b6Y(Lyrt(G|O`<2`m$%=oUpiR6bSUqgiV9ShT9rk7h1aD=pZ%G4NHBw{;^ zlFVOOjO#Mt>g?Me2yt`Ax?N7WdMOaUjYe6GHcDAl(nibug_P_mZS?4NEZcGoliTr0 z3Ul=h(kpem9o*Y^*^Fl&`}j!JW~aWuA* zg{jPcZesvpsZN{zCJMSN?``Qc@yxr&uWp$fGwRuwmVGmt>sR?=Tc#zx8Qa;yU^G{v zLeXGor*&bcWm?J7XcB77C1o!ES9|CB)zp=S@zdJsazq*1sVFGUnzlMqi`qdzg8?tK z)}@Fj2m%4Dtwfa#e1DBt{D1nqY0A^#VdJ$R$C%07S%<8wEY8{I?ESvK=h^SxS(do(DpSy)sp{4!ytKF02c|o{&N-CriL|jW zo6DZ+bV_ZvJnY!8V#nP_3Bw8?A>8$(9S-)QP z?0x*ROL$LP+UQ3aHf3Qw!d08gS^V*H&qY;!yN4yQjOg>(;XZgHl~R<_DQRs> z@ZAG*1F?!%=^bX0rz^Ak)%}ml+HPr8QD{bv_cFK2ky>-W?IVEx-c3hp{J6qtOE5#llD7^F_li>OfNbr%k_vo<*MojKe@lTwzTGU(`% z$mdtgZMZ(sLQrP1T~K6Z3#<$DT3i|9+-|5#sWfy=NWaWo;_0|>ab1L8b|e#YIk}74 zC6WunSD&p5#M$v9<6gmxR7X{QooAR+wx?In2{*VP5528i<%4bPg}KW9Of)Xp!hKz* zMA^g$mv6|D;ocOcD>D4K?^4g`>M^gM>^g4;NgBA!GM;9B#y&Fm;Y|puzWc1kxY`#f z7^vz1^SdoHa5YC?r&Er?Twv>!PNuS(FmJWK0-lJE5ja0%WKxv#)k<1C?w`qKqq8$% zYn{%@+*$K}SatdI<6vgeI;NJ#dT=drVJ0i5$fY4t{#mo^w<}nBh8TcYKc^EowaRLc zjXPVJfBaxbT`2i5wfsODGn<{6vy>WKNegI`nnveAH>~{7I0&yDqrtiFgq)SR=-#w; zSVqY%FRgt209R*oR>YuB`>l(K08k*l=oaRC?_Nk~mz5gh&6|gneTnSR23^Cbo507Z z^&M*RZ`t~5mt*EM+%k<c zR;L$g*5U4d&CPc9;Zz)mA_o zsWzze&|0k7X~VN678wQIKiiZ|^;FD{CDGozq)k3C28aF@UI^;S4woW*J4Y9!Y%dUx zC3uRz^@tJRl0g)ZAcfa8af>wT7ler>0g)T1ykgc+Ucj4I+T4X241**~ZnB+E;)pY+ z7-H=(t#tTeL%6bVl`8qVJAdzrQpb7WW=Xxm;}-OUu0vqmQnDi?GiN9;Yr7B^W2>r0 zd`YqALwFswce0|W?0liMiQwW$jk$FZG>JLwmzdXS2+OCN)t*81LSv?P7BuMjAuU4l zqchcd74S@3Kf&d~UzpPiAZWy#EK63vNcFAF)p=k8%h-CL&3#h=M<0mG_gf>K{bmiT z+@-_Y=+)6D4k79-v8;Z?%agdmYw(@kg8*hFi8jWTh9*zwUjz=QLze8GN#500OSEV3My%$Gr_EZbc2XIh!B1n>Zx zHB40l>j_TZL{kJ3$y68c3$IL@GKEjwl&Ws3vw4Rdm3NZy&(*7$F*IP>DcL3T^@YS@K#n#M?3hu?HPp|&>ZDf@2bB7$G!r-%5pZHO@Kja+wu zxkJ6xdsV8g&X2%x>VkWBo_^pi1IED=du(eHc6L~}U%H4Lhb<9jr*ZE_>I4|>=>6GY z%*YKlLR;woBv_@!|LU*6mas3=0v+X|VAyLML;d`pFs)|a*aZ7D%wFhD-6EMaTsTz5 zH;%)+-FSLDkq=K2lf1fb1QdBt+m5rtvU`4o{pxyU+ZVd*QQ7>A{)h0B1FZt<*77uG zezRrSmgszBLp&IDMnpG;OmQy1CM5k#{O^wZ>gU=I#=BC{)#4Y|G?`F?F}1t5emd2- zi2dtQ-9ujme7KeSW#Sn)mD}jjI&5;v?h1f*h0@l&+S2jkensAHV~C>;N&_#Q6yjv( zUucRQd1!nGgO$rMCVD`IrFQsYUxLy3o}I4kON3`DHpmSM5%aN)F(OcRgI1Ybg%~yS z^)~Z90%+z{e+55txWXAnV{-cX;6(dDwz?3WJCqE1K2jhsZdjPKspBnbaB=N7TkHF4 z2D(%gx!bXlgO3~|D?u9k;`e|AbUkvnQTu$Q9p zYNNK+pB}Fep-2Dg7#HiYk8^pVpsPukix6|{PhtP4?!WXkS!hScf zy?EBBZ;hLh@4eJ#)9T=5r}6-|+VvcPUDa(SQq^Jt!nWs*!<5i4-OK{#`THI#RzL-= zU+^$>@W6XzxQN3k<1u9=T_=;!Nj=%}sZQ)7wXLA|-TS?n^g4tT$YhS4R2C46gp`|*A{$a#L<*`%y&0*p`#-5IoR>^#ccJ2u+Vz7( z_m0K2ebk-HxHOMn@zJLzakycW@du46=3_uy<9o<{9 literal 728485 zcmce;1yt7Swl(hFZUq$)gOC&zQ6v-u1qsDKT9IxEX;8XRK@1d>6e*PwDQRgH5Rj7Y zlJ4&M&llb2-h0OV?sx9}?)W?QIC|iHfAKu)S!=Gj)?81nONs2>NxgH+mMyzQuU(ee zvgI$bEnAq$i2uY-SY&9Uw*0YWi|FNxvbMpa?KX}#>ovS>iVZut^5luf2!3q z&U|4gEv?863!CCg;Tuj=t zxHH7@kuBy*OVfQPW#7MG{__%kIbA1cuO3T;?~0x^>nI=FH|b3y;7XB9~D`7HCjDXGECJfUUOSy6Bg}|fH`B~P;K<1 z-8B6}L%deQb>t`I9apD{@7}%ZO0;KvAk4))DLGO+pdsd_NQ|SSZ&8s4CE*J3%4r(T zOS;RmLmm`t-h2vul0C&#c z(~X-IG@R1RI`al9Lmc}26wG6jPuz?$Zu;2$_2Diqy-JBzL8FGao#ZU6%Bk0m`EVQ5 z6zZSxq~;zD_Y$<9zI<8zPTzOG<+%}|xkv?dJ0?mYb;5ga^$HWGG#m2-)uv}tdM+-rhn|12CQ@AZF_yGG zUZwGLRh7^N|Cuwh4e=^9wa;nJ@)*_~FL1u>vc9BO=Fb*p+?nsswM%l1kL?`rv`8s_Ods%0zkGm@n2zDx@Vsbic^$ zYuB#n*F=;Oek2j-z3z~|GM?>sQel3q)dou@U_ZM!9Ge&+o+x&5dahApVZ40>PZs7h z-~4Oej=MGGI%JS(jW#8J_ww>N`?dEs!i9x}{n=E~ zwThm~M2bD9I(_)Wi4$BWcWv2HlYy5$$aNbl?9vdY1e2k|l6Dt7dPH(ii!;zX`^=#| zFLyEWS&yD`T%3f3FW{_*`W!nV;@OsCm-0pP+Ih9Hmb6aCnF`p@!f^e`Lx(O<@5{wb z$IG8LX-UQX@F0dbCKMjJ*tm-qFFxE!&gZh8Lw1;*eSEw<8zv-4LuFY1Va~BnY~_1s zXs7{AdwV-q<{kkYckD~Bkc;|v5*g)lF6)lC*45|Kew8#gWn_4anH;Q2A1rUvlaoi*4k@P@%Dub%02S6{q3et504WJa zNo(uMkGGdR%e$`ag3A&urfKFreUd5%lm2h^;7Tc%H4!R2Q%2u^e zlJI=Kq3W=n#mR0sNF#WpSEscPA2~vIbG$hvTEvSUZbZK!u6bo5-({jhAZ9IzzC`$O zQ&ZEE{meY3ZJ)3KaI=p+J*~@oW7P7VU)9#vAB00rahSU=v7Djt;%c}-9nA6Sb2l=a zQhN9Vy!1eM;E~+n;o;ZwCI}?NZ)=BE*Vby&vokY?EcqBO<4N=?UNgC!v*?$Y8?G-4 z;9#Jkp&=)?!g2fj`EyrS*T6}ImR2bV3Cq+bk%ETw=zE#HoA+BoSYkO1h&bl`ss%br_;(<#wB+WrF`dH<@h|yp_<-O zpOv|hg=W35AlIvdDdtl($700$qw`I$j8Feub3pW;fykd(HiQGPwOtJ4(sM_g`}tEQK{az` zpejtjVg8B~$EVE9ZQHg*(k0y)X<+jym3=Gxu)M19KJ^n zghO!GQUsR;qnnpk)#hGOTVc)QV&$EtOPLuN84;WuzC8SM%RLwc%YmH+$8X>Na~oWH z5F-4Wi(7}tI^ef;Ik7fQ{oUP+Zi}t5EmW`2-5%S3Bbat2> zte%>h%FD|We(bO?&Zd-nneV&Z>f+Q$V`8FOw*2x*LVQDf{#ICcZQC`qyoY4?KkC`m zCW?xRh|IoXeK0YLf&G7kUZtm`Y|OG633FaPVG`SJx@XUxg_#-gjf%dqfPC}5r#-6b z=H}A~4UtYZIIiAI;`Dr$gV+PS8@aG-yz(3(83*URi__m|%#2&pEAA4zz(&T$$6+6~ zQGwS28GQ^c65seBd%{nQoPj}Ob;AyS931p+pSz!uvO6%)?(*eAgp`O|=@unMQoAe9e85kB^TlBXYT=TB#y201!(k|FKH_% z0_<{pt*S~yEWv+iimWiU_31K>v&Kz7vTY_1h>aQ(>SJyepN?!c_I&=FTzIrS+XjGf zZDoPsymef5HUSgWc0A)Y+0z>l5n(wm6WY>8Q)1*R{CMw?t8Q*?*l$EL15r^?>^Dpt zH&Ac6^+}riz=7Gdh0dyw^9%7A9Wuucz*y8i-FfTf zmygDT@!N5ZNm9K&V{q7Q5RM)(tIYRA3d|dUCuiy#&9Lp5FG2IN)c4s`=&`!kQaeO}HjDbvxji}nE4CCf~)YN=B zC1f#ENk-yPeLX$lwL_!U+1e3@C&Om+e^>YG-QYL4xVVJAy?CoiFO+u^zx z)uVJ9jMHuG1iVZMK>QvdQ@hx6ZK{|7SkrlZbtzsYgOuvjg&0}uu@*VvYPjyYx;mgU zRUC`TknQwrymE<`2g=l4AqAFdEC{%KLhRSw?-xDYNzT++lN-4LIt;8xMHjK|K&~ow|~h?NmYJ(B|+e>e5^p(K--%sBgV+GiS`Vkf)mBh zHcmxYE*aG7>DcYAVl+9uFn-0&mz_Z-%TY$A#^+|#pe2LU^82gtS-K}>ho<{m2RKx( z>7`p|3!FY(t;4CS(2jR4kv3;PMJwcI`%|_iCZ$wKa3IUd@T@`G#|!uWJSe4z?#TR$nm!JVOz%D>-S}$lZ;7oRMHA01E-!})*yJgM{ZU=LCs;DBb_j9b z+sCKkwuKbM4y=EGH>V!O%%N|VqnEwwZ*R=lY7KcOQrs_hsK`eS1XsG0)ifjFBd$wJUY+tm5KYK~eP<~E`;d*U79?cVlIL<#-C=wj3*?>EfG4^&gZz-RX{POxQ!u z^YwO0WQath{M^u-=j<}zGs z_U4Fz&bN^41vj#XJGY5RZ{50UC!^`3M?$=j)%p^Wl2&ms>WOEmNW9|D5#B%|S>P0v zWO^ql^Wg6E4FlfkAKtse!yc0De5gu%cf@*e)a*vj^;A?4WBhyO1w#M^PG&V275Z7)hEhWW6{X9^Eg77x4jQ5b- z3b5+7*)0~yI+h=lvD=`ES0_{W{(U(Xg?+-l$LUAA$JyN;KY3y|)tuK5_hj}>f9zY~ z$Dn9}%KP^-OK--Cu89F=xA!X1^Y!ACQtNh?BW38;dR~#evzxii&dx6U`16BpaJ|Re zK2cepnAzpGo$;Le;gZ7I;-iCe_POe28LheouOy<7z+Dvrr_+_GyYgm$T28=FcCKVR zlK)#-YuQv&Q4tft-t7P2?HTGmA#lgS!iWCov7^V1G3{jJ7Z3;wUWd;WCe~nJ5b>B@ zbk@=7ryTC?W_SBRGh1_0&F}-+*V>*bueyqgiW}s3^A~dFgDj#Pj}{jfQ}=*t$;eR9 z{PIBfv76iFVx7~clZ1&G7|hHVCMM6@&1MN1f6&+jPQ(ZdE)4;WLI!UG+bxqou-zmgT+p(hNnFv@|#J5)u;DR_2v5oO>#QGWe{9 z)ZaR07UXZ3@&wV;B{sglK_BPHAar_zKRzh`Uvsf#vP(YSe;+XXOubbaKD7+O%hNZPm0r%W23w z%5!drz*P}c<_!t?JOhZ~?71$S~xULmk zjF(4-EJ|s)Pgws9b-etDErHJDCkpIg3BD%c`YrKFTYi4m3NPF1eE-(8h+?gz!f)TA zqN2X{nF|RCojrSYs3r+t_&ihEsOC*e&;K}Zs)$uD(fPEty%fdFVPCPxM~@yg8#mzM z&)rk;$$ONSHOYh@-@Er`r}LVwslrQckSRr4cde}CSeifK1GYCTMG35P*8w$o{Pn*O z!F|h%i6=cFU^X4<-b>e?uJq z127P{lSC*{tuf0fT4CgjerINDrCPp@jpUGG=JG&4TdVAc`C%H97PHnoF3OXH-Jjm7 z>weR?EwwMG+2YY2I@uVlA~;6rb)65{*(yC(cU!#qR2-+0Kjop+nImLw+b8Dd8Fr4> z`KJyt&nQXi0Y@fWYjR1Qnchdj1-3jXF$JWHu z9dfNwq9l(=QJqYCwyfEP+@$*<2Zr>)M=$~1uR@*B@cvW~WAS}ef7-~njao>MpM z$+qO>Wnv22<)CW281i^!A&A>#RNtWHQ?~|Xfq5QbSr2WL96HsOXCXqo_ke6+wal60 z*NE>9RJI$ljfiG^EqF+F#bZ0K@>6o4QQ|%MJ_UaG&Xy&S?e{N6*Y03!Yj2}vQ;u(_ z4)u|dk#QrVA0PLmfQ5;iCp^;VXI(kMVx9B5s5yCOflN4pj`>A!FsmE*)slwf&|Y{; zw`*cj)~QFCn063crNyUzJktRVb&1u9#}8Y(ZF)k4lf%LDHi`ui^EN>g*{cSU$bA7EAdXz2I- z5h(+&WAETJlZSca=0k>+bbT}NwxZ!#OEt8zoORW@;z4?l|D#<>RPzn7<+G+faq20i z%}>MLy?f{CK~6K(@uM*)-=7Y9_USGS9)OeM=6yyzonxoEiZ%`Qg`KKA!TtXOIR6jK z_&4PK{mn>Z_!VQ+F9;;C`R}7rB5~{{>NAhUBEN4Qt1YX0vW^lq>CV&7_}?v`Py9)! zU2KW|U4A)C!0!K-U$Wf1|HUw^GL$P(;vq|!i?Uz)HeJQi#!hESnXJ{CKA}F-O>1|4 zQ-0?uLdjJxI);tOx595r?mwtW)G1tHC@d--u>W-;UQQi!r#o-6w6tuhn4-*B)weF* znm#c(GuNJV*C|1P;AMLE9}K&tUqAeOeQjCCh3TMCZB+f|Ki15DRdpCj7q2_L{rs8s z{SmHdsb3ep*hx=#i~l-L(REa+7Y|4T2SChdY`qc!Vn3Ll$?ZLYpofQey23*KS9UdP z*?si%&Z~u@jF;0s$o)P-VJ`wDqGqE9m*uPS*RvKCd+ux45N#tbZIO{>kz3wxIoh)J z^>QXEoS;=Em!}h@Lu>+gRjlq-*t+EV#=h$WF}$rf9g}>&HeKGi)k<+d^_-VWFiu zPf*44*SQ_>;?XybxQnxC7fV0;ii6?3g&trYVPPu1J^A5mzpfoe#CfhRM}KLp}ZR!~u!w75yJV)l2(#8#(6W(y;-D z_ls>>)%weenlH-wik?zf@dI2HO9#ivr-|_>%*~2DB_fKB@2%1?_(P^4REAS#35Wkh&>i7Eh4L@&K1hdxp!nO=sPQB}3~*UuUK;i8afG zPz}oUQ;AYz%9)CU;P`?;hviiH>h#Fgd^+XXf(7#L0cGP4YR<0BaLm)4`{Y>a<;Nml z(k_ON-djD?^ZBFpz_`hIYxcFf7Zai%#zi0z1N+nNVXTy#J4JZkbOJQY8GjdC?9X=uP|-iBS$NMf!H1|zR1f)eqFi<6^=C&k>FEF>dY?!#*|8reQrL^q%XTl?ummNCQ z8!@|eYx2%bYYkj5Ubux{YQAlxB?|I)e4TV__6$| zVowSUdJ~NIf8r*8AKjQ|TY)EE&CQJ-OU%8IWT;=;&qlfasZcAa7eOapr$0cW(X1{G zu5>qbJ|RD}i2W6BuHQaa^vFj@@b1sL!3B}OC{J21mp7Z`2N@+Py6-;dyR}H*cH1K# z6dX=lm#-ALx=M!#&eWexVWTK>C~)un8*$)`hH`>^ed|)M#FjF6PW;G=uj8CvJDoAn z)|_tO}Htl(!QR z>1eCVWZA*6J}tsw`Yg_oa{t!nH*bOFr)FnY9hd7G;yX$jUe`26Z)7t|hs@)LTHj`o z8~?j6bcc&Ffc&Qwx@-Qm&Q!2NRJxxM>(l=2dqjI(?|HqIelZGABXiXnO z0P9FiX86_c(!ga}7P+|V`ReU4Va^G)Lt=iccBe;c9$fLrwi*lm?OQ~?X;WEd)$z{c zb!Iw9YiVh{{Q6KO&C%({=k?~KpwMor?IG-QtrPvmotY{hA|yxr;cvw?@z7h6tGM>q zR|;0~4H9^q3)W?tB!uvPZ}r({{Pq80=6~Y|Uz>@Je+D7P*Vfi7vfV|_P)lxt*DYNx z|L+jKiVSY&(#U*>a;jmFQ@u`}cVmZs`+Fr1u#I>X+fso`nu;IwMFG`DPcqH)JLex_ zVd2Y&CA_D)bKq{0#*MtGnHHzUdsZ9>(A4bGY6B&RawX^jr6 z7;T8Wi2}YgrSB~!2I-~Q__NvmtV*1}9P_lR#5VWcLFpj7HA~*uq-4J)iC4;CBp$>u!mfd-TF{_N_CpF2B?i_aUKw9AaCAO0JO91Om7oQmxFsY9Qq>N-0$ z-pcy2ED_@7f&1HbZKq%zv{im{;g5$hH?j^;DiESgay3`m(587h`&`|V(>MOL0hzfw1_xp;$|fod%rVAEfFrA9iILV{`lj!+m(iGS@NkB?d&tEcjPAe zQ2NnZjQnI>f*jXxi}83?d}`1#Mme5`NUZd_*i&{d*dFC|J31 z!)3!M{9e9GtbbzuS>tQgo7cYJ{e^0KP(_qX%(%!Ekj}B17d{834D?rM+}UvMRnVv` z30w}1(*b?6ZR?*>VZ1BtZ4(onUaRvX9RElO7fA}VDcIDP=Q29!gzTFLb|qdqN%$eF z@+wYAFz=M!Z^u1de%AD)N>nP@VbmyTp4##BefQrheEG_izF#;?{G{vWJ2U<9-A6<= zoO&2U%9eJyl=e-dN#?f2E$JAY$&Y*C47&As9maE1+4qpL`~M@=)TVmy;Hhf<`hr9F zRWC;~#V_~n-HfhU{iLH5rY>NyZg}P5#VZMF*@8NYkN0#iD4jag)f-d0%SE*&QJNzm zh;W#6!m$2r*_2ZiHWpH$d~1`pgu8_w+RqPm%#39m*m=c2z`L8u-C^PIZKs*Bsr}56 zV)hHe)vk8rQirkyW_}f6mWckxF#ewkHYd;{6V%FmNaUfL zTkod9FIHV&URfEi-tDD-=Jl^*{CVSzI02UYw-UvFN?zbk2}^}KgYq0Tw?pn6a4vv+ zvzk8_TC7IuZ7%pxrwyNVHu*-GVxF`Kf`o7mE?>Q}n~I z-qF4v65$hj%sj6sBHZn&W`{mJq$Y;`zu%8>Xa(k(cYDnrvK|v&5_M8*}|&Dm6|bW*lyeZOu0`W4!!u zx(}6fg+!&ZmXp~#A0B4jx!Ng4&24;2U(<#0a^zETE$trpI8_0y#UHc@znneixq_~! z`kjp5{n&c+W8~95e9`>Ga4;~}iZ>?0F*DiIA8SKH;_k3_FR$wFyOL4uq#(H-TfgJU z6HkiR`onJ5S-9xgLel=+y6Z#z@S`0&yySo@&Hs(wK|>2DTl&?p7vX)#>wqTV{coG8 zNykR0hqRy!Wtkd$&hC3zS?qO9P10Yv9>c-TPbTmYjR_k9x-EKoRUsP&^;IK0#>1^E zbj4|s7Bw#}ZcS9p+!3VwCA0*uw-Y~Glxcc z5ah;l?ejwDB4OJk(Y(vWVaUs77M{BhNZ^Ix=8T2&%{NNRB;(}=ejkPsS4qzL9fbxlRW6>d+K zTw!jNUcGzt(J#hxPhhEz*gzku;vTmGYSN8c^JW&G_&8XbDa6VusXS$AKM0O_-Rd-} zNtu6q>juB|>gPO-;Z0I`NfG|WEZ|OD}&1M>DKYD0UB(& zKBr0x9^0%jHxD#_J5|2s?0oIXzIf%lP|13oj!|9{ed#1Cfy6+F4QFMU}$xXxC8I@E(if? zf{``PM$7ex_>g)?`WU-XC&mizP?V#hrvq! zujRuhz@$@=xEF~BY-HKENEjQx6}}G)?i^jLBBuMztlWIN>#Z;Ein3jn<}OM|pi`R9 zO20E&q3RzjQb+6h^oZFW2423ta8ahGM32<6`1@+BB#za-xxYpr;vIQr$~sBumW*3` zU+gQ|$fDvI>N#;W{BF~Wi@)8bv<}?n?Uc1-y$KaFYm@8wrq{2(JgaWDZa>p#y^+kU z_tYWIJ0#nAXvuov>h2LrHtM>pdYbqSTc_emwSTx?GAdN-+D#Q$$UQ02l-YEWjT62b zFtOC=vrWSvDFArfL>{ll8VF1%o#xii(SE>}E_NWek!O?UCJv(3@>C)BH#=-|(Xb;CScAgW*_45t<}{C> zARZAG$D?5Tn9AfGd1q3P#ghs9vZ)-C3s?q5%V}S$_(znXD!8*SE|A?$FZ1qGc%**y zA)jX2wL^RkBlcfGzK&Ttlqv|*vZ2d~JFwzL^^JcQYyRB2^&we?S?ArtHoc?`ydfu zeY)*X^0x~0L`CjYwPN;sIzI2My1cPE)jyf$sCn3Kz7>m2rz2grK9})}D0%X1YXJ(x zc|kKyi8l6eip$!i-iahMr-!wgA_*|8vtn&M$ZTMCtk!5wx(tn4y!wNa`Rwj}zj#+c z#!U*#m@`n=sBxk*V*%{&4n|`hl~z(UvAliL?HL-nzmgKM4|VUkVZG>Hx#pK@T>G~7 zBqRFuLi#ot*%vx@+!O4*T+{&=gTz1;t zqtSESj&;V|0SSj^1J%ge}k+F|7TWhUpmynhg1WK>VlzZFo%p_VtQ zpEu4aHxpaY zZ}%_mBd23?LlJ&5JF|15IeQ|2P9-JBmt<>!wx1)HL*=+Y{&;7~$+FGWI8^#cs4Ah| z;VJn^jVqPA8&Q(PRx&TOc2lc6ufGd(*8+E|He> zQvIsD9CI!EKmZdZRsZ=h`tR4m#QJ-Bd|J5)F}NgL2TflYMudOA5UKu8Yt)hDV}$$P zi}MW*&aoJ%L@^^+BQ+z#CRRS4P!DykZhmyJ3ZJlzL2qFcdK1yFUbS?kGtX%S9eQTD zbMloOd@H}$GY&s|y{k*p!IYSkw9=1@ruiwmDB-)!Kb{=hwFv#Axwwa77nQSx9TU62)f0s!R8hW{I%Z zc)^|i@<4R>=dLf0Kqq6^k~&1h;7CS6aV*yo`W5J7_A^WAEF}kV8>QP${bUWkrKRZw%^&ziOo@7^r{*+?ZTDc&Jd{Dj*%r{&qn z?gEum{nJ*%vSwxz@87>i|2Ls46)`3;XIgBYZ3PXz97@SL@)c~Xtj1s6cA!OeNO~T! zD0JtW;|5h{W`2ntz7-drJn*zweHlIMRnVu|&+4PVRuOb|4-JnoL|=dW@ke0T5_*zN zJ93tARU#rHJ?BjH4u_)i9-@wu@^OT|RB6GbLiAW;*{V|2E8{fs*U=ITNs!%iWRmCw28GX&EaB| zlY2LAkaD}?gL2w1dhPwwm~_2UlRBIh+Nrq>&r$E+zaJ8)v-Y!t=*aGM6cuAEL|?I( z@A1j4QquJsASSt;vh;|-xhcae%yJ7K<#}!-^LSnP53-{1@_P0B zLW^Cl5GbG_UZB{a@1LY|f@98a)f3UCZSD%;2)E%ZR6Q#3@-b_xOEX)ylf1{1t}V}@ zmDzEqhKaEd>pt8NKk{k-!VP)|5uu_O8iL4yhTBjlNs(5~8`_w~7Nf+64G^?(U=$}aZFs#gpytL9D4cRzdtUr zCoH;QBLZB4<5Ur)a{PSKJd`~we&3;M5R;B9uFNun*fI?|t~8@YQyf0@koWxj=|+ZT z5$xC+O!LyEOX$B1E$!>bv|wgoVKL#?htLOmfL#ey9VcuYR8??Y&u|6Zi^|4UhrM8ZBIxVQdQqYT@hfBbq z;8JRmYuFWt@8ZhEQT&#JKO#hZpr}j3afEN_g=o-o#z-u38uokZ)-AL;cHmrOWGp~) z@{kPTJxNG4;1U5Q)**ZOAj^V<#gRngyX9C5w+~ks^qgqBf|8~&QKK{0VF5}adLif4 zm=VM+^H8x!7MX}^5)#kIe!}Nye|we*!7Zd?u>0d~AC=>w^IC*32^!n^Y7quHK}SX6 zTc@btZqoIt=!+#-Dax>1(8ccCyZ1D7RnRbA8-w$Hg~oOG>Q0A=T(i8Ter&4D3dExQ z6+r{GD>%G3n>uHxACjTV4vq5EP)vNhRY1@`bh>P;ZBv-!wVvHY&T__PybbO1LJPFX zET=wYWu=?8FGED^1d-cK=4(RFaUuuk_q41!RY87wB$8;0jP(L1C0aJ zLEqs_j=Q}&clRgOFf5!NIS-Tr1&{V%6>OpXgP4QUytf4B>pUsBxM5?$8srTAuyyzf zM1h(g9q>MeX)Vyzz!V_|k->E{6j`y^Bs+JqKoM<8#`n*HNJTmdk4Wh-LCCIrOLcJg z@C9rDr7$$LwT+E7&_=-u;W7#8y)bPiI`ry!cz8%0o0*@thSE*mrC;xrxA)Kfe*g3< zvGv1q%gh#yup>C8IwVez&cV4oqdPy3pTWo6v9VEztvAnXYi(6Q5@Vjp7a~Rc()zQz zC55n|v=P(|kPk8lI@;nqjnv03bT~|`L8Rb$J)&_0;=r-t;SbgN%IU^-EAu9Uo=EN5QW0eD?!02KSwC6D?u7s ziH9J5d;R#ct36*_iB@KZj?5)@K!6ZuyabsPLFEbk)p`5b_gBMXWMjOwy`+fm z_I-y}&_LWvO?mF=xjx@&hRPJdePN%_SqVe>*6f*B0Ia%i-#)^o!?D7FCqN60pv#eL zU6m9*fk&^_j}TZNklLlR59K};tS>He)}&>&PA&)s2XO{aXCRV~GhNg48oCaD98W0WYZ^yzUjl&PJVne%d)^mOQ}{7@+ZhXC zF((xgMyaM-;SUBBMfGZ8gp5FQ=R+_A)ZpolH{QjVtgU3wY@Ciy)+MO5Rv&&Ix&*~h zWHj8rRm6cIZD}zv`{B46im<>y=&A0&3_6`x#!7vd(E$w16qZ;;3s26{uo3X$#YY@Z zASxW~os6JCN_7mOD#nLU>V*;+Iuqqo{TtCzA%Hbd(n66X!k0vbkO_b0=JuC?JJc%B zf=y}{@5j!>RqGSvjt2;T&S-w#5?gN%rSq(j$K%KEufG+Bifz|{<3Hd!rh1C=yU69$ zb-XsVaTR#@jvZHjdmWx@4)*0a!$XRAmj_sVej?{4ybKJ^YNDgPs%ioXTj9spKzI~D zM;>l&D};5koT;nVufGs}d&kO(To~F&-StUJ<&F7WHTq|uuky|+!f_`&2tiuIrRO>O zA@x;iETbFh_Ta?~THD*&B3#?xx`Mfl{L%It$c0mb2XI6(jzEYw0^%YphxP^8*P^1n z(kti?kD|NaljhWe-SOe)=cgYb^A8Rl?e9MpP*%$`JZGQX9y+h;W+{B(yIR#nzKj<* z(dSAJ`As!lc}KF{wcZeVvu-GyUZo2_4r)Kyq{klYHDNbBeRHU4wP24Bt5R|=q6E&} zSbu*50zCp<#MmR`KIS!Wx-Y(vjGc=@XRm<u=mKR;&pa0svU5pLx(!Z?FOKlMJ&T+r*SWlb5AB-m1iHQ?4Y-eY< z6j=O_hBc^|<&slUX5j2&wE#+Iq3A3MfY3hl(uE73@BOhgq8O_3Lx;MOJkrm5l8(&T zGZrEkjZtwApL<3_Ms_{ZybpG@2^iCi_(H~$)~^W@yMCE$+N=Kj>Wres0y6}h;n1`* zO9eOOyX0FQ$Oc}NR+ksdA8JV(!OE;7(xy5Iz%q)o4hlMc3NOMP1ugu<sikNSf~s*h8Kf>J+BO=&DPe|-RT7`Lh5LI5#e%XM9xIIpFbibRv#i zdsWdtIH(`9aATQMq8@@!CZ;~jg#hHtl!*ZihoICr0q54c1F825?K;D`o7uMfqGF^# zzb9dmy9;|GAZFoHt~^a5N9 z770R)soo)va)&Q7Tb1fF2e)Acz(=#xGGctNZM|g zAg95NpxpLnudu`c#y`a=cQ`$J^hm_mcoaJZU4Lz&P&~k(*bEv$fTR8BX`X+>F-EQi zWiM=4t=Nkp?9?sHCWy&6F$kqT+yQ0n0yYC_s!i{+OP6+y?1eWO} zbL1^*;o7*j4^JCBMNNkz070)3Ac9yVCy2JmjZDZM8ZYk^mW@-q5h;VDis1iZ&Rb#1 zL3ejIg~t{9*RNkwlan*isaslENwzYqAU@V?%~f~#S-4~9!y4ivn{p~K%xzBgNJDPc#${>P68lU z`qim!$jVo6IB(v(iHM&Aiaa`@i^ePmfE=!t+pL2hl58ZeUbMWN1@+Q;nUFnKkIdn- zsvk8k;_N_%e2|)YxtI1ekIC{-bQpHl5xEONM*KmRAdS;ju(p>$CL#n+1mU;%)*Hd1 zdi^z4wFJD*8Mu>@ouB}Qk=_23hu($8%&aUP{U4fu0JsM5te1y}3MMH)L0w05HBv?n z!zn=az!6EeFX*>@nnHHzf*WIwj0&qiXbF6uFr0_@Mx0_2pjlhCP2#VaAJ%OfV#j>q zG0z1t2S?h%$|@I%FI(GBJ%S2V1H9Ir7&pVArpLp}tCY2zL-r2)t5g%Pl8Iyx*$-yu zbW{^2C15-ZJ7OrNGr-5JAiINWs0AuR79K3<#K+1y+T$hUU#f#GS{zNOk_r=~qo!ur zE$|5m@)@aRC8HO84^!BNKDLellY8FX1_p@+Cm*R@@*G7Sh$#*qM2q60B7ue{Mcr`k zSZoYE$sS8v#k^MU+&-~Rmq0s^tJF*R>55fvYL1g|*)lm=Xp(=(`txdLz+-#qD5 zkmE8IBGuqA>!78lPq=w98$dN$;Kf>8e^uDT&!07r zC5w1e+ztXR!js?yBJ)JCU$8ymUAwrbp92>WMkI`-HG*S+2EOLaMOS2V;&5?yd+>6Y zhXVjIP!VK<0UtmNp7a9CO5)!~?so${5hxf~H8MPULB~aKO6bu6V;AirLXaKzsvwmf zY)aBLWpl#x4vu|P({FN|1Iv)uNI(#c7beW~n4APk{Dv__q6o!Xu8GhK9b518Zrtj+ zDK)RM>^n@jlm*DnqgA$>wd$pMm#)nw$VmwlnE$WD=Vr-2$C#Kz>G5Y=>R!O^`~QJW z9sk{+R!J!Bbm?-=-PyT-;sX38q$Ie${(OX|n4qCg{% z3Y9Niaj>&XGi`6Ft>qe^3~~BRTmNJs@Vr}s|JM`*|B?8XiP}i6xD5yxyqV6y9iKb* z2}v+^7r{dfY%g-;SEWV#BeMqaib+4QRctWN@ILHLu_qmpPG41aN@2oyB9Jrqk=5P1 zcB8p5F)@TeT9^uyWj!`*ybgR}kBLo)x}o9e0axkUeqh3s2S?r@KqAmlgg@d1j;1rl z0sz>I(clPxC>A2{QucGhmACVNQ81gL;r)#s-klCh5FCOwp%rrGM<)*Ien5m6NMS#E z)9Mw!-83F8Hz$Xa*Fvu|GVUZp;9--;UV_WV3rqq^B^^OBB9}=O^22G+NY#It8qZBF ziS*~Z)o_ID?Mc0DO6-_OfmhqRcdvDJJKzyc3Gj7guY(CsD~@y|LlsU>?(%SKRjU5_ z1qkd@;+JSbI(3Fg3JRa>hbLXa=mEe)u&q~!Z$w;siD45u&Zn#RoR;xAJ^lS~)cHFY zQN%`yisf1E^?JEECl)1cvz4t5sdPXbUGZz5Kz9`M_LeLJ;bo71Y!}vw{+p^)vKYr z7FUUHWwa95_@70SM0_fJAtn`_gRA`qNw=X96gL-jSpjVwv9%Nz%yikvtGF$G?b@k8 z31lVnQ&WN9Ur_*o1KvkTseqwD1aetu4gQc2J2G0~vK%J6zJP|PLbkl7g>~RT=7HW)&n$y$^$-JgK@&v*PyG945x%~W0uw{!u*yskRkIkGi7ML-~n00n`wgZ~U|NQe$ zPg>r62M-?HyY~jzbz~K5rMi)8>=mH`*~ph_D00AbqV90ji$3J$$>$=Y)+?B|1Kz_h zrI`>U;#JNLotgwN^kfh+JwuIi-uC{+5ynDnGLE+iXP~2#6P9-fvxU}!5sSqASTN)! zv2g_HJRg86#_2E?!mA^ZKq$A+9Ghm2t*54;QHhaZ3Y0kJ_8yD_o-1M_lVJB?)Y&LJ zQse!^`oQFYPdE#Hs;6fL2A`dk1>Q_Hv5^BaOv0-n#VQ9H1u8z#oPWY`YhF^G) zpQPBkHwlReU=U_Lfs&5Y*yr*lMO3}DVfA^XLg+0L*J^!!03mpm^?^0>C7{Wcbd&R7 zN73Rb8l=P?zzk8qM5ha;c_I1bh9pA5ETtK+i%_Zn@Q+5XttH<>;g`2g3knJ%JgO#X z5yt<8&^cqq719J?dca$o%dOyX!Hbd96}noC_{#LUU6m*97Le3D%ZhD_kL&Z447fLe@IqyjpC2P5D4u#VRR2b?g?2ICijKm*uRZl@_@ z;2EeFgbq%LT@QqQG#}ezYP5cJfFY2tU=j%$S~*etM{x;GQgGf7tWx9=Q52F0W7X=8 zM@k?8^Aq!fuB3AI^y$+eWZ|rZAA`M$Y!?yL0B{GPtc)plsYDlRaX5YJ7!oc&*@urJ zUenXlcXSY#U-gvRl8%n6+C=9B`1vbC1xAL3l)zzGX36FiPItMJ5);42R)SQa<}qHZ zbDl)r<~$Og;YJ3gRuQ}?eCtCp+>6hmzZ}RD1JWXy#Oe&w+D(NV=8dt8sOoSnCFSTK z?N3z;B_w=5eq4{4xqw(_J>J%wWo0xJiCxd1{(chg_CmcS%PRH@^xOy7RTpDK?!nJv z*~YM!r>RI_KXBxgUj?6*Z0ggTKl)Y_<>CAXWNrG5^eiA)R08m>-A)qlLXh)llTP%=pBC7u1kxD$vpl)4gC zGT09tdIR_eVqo76MzmNA zqi%$n8#W4)B$h^t83v8L8`HU6!e_i7!a$miDO?~p@%n_k+iUHt^{gcLbnq1Rqa_~T zyD`OsD_1?S(J)QMLq4`*^_R&W%h9ONy^&Y*xGo93_nQ`PJ~c z8&$?Un4>1oSC2ieK)r`h@?6gE{ShwW1^aZu9bx)7zLG0UmCV!q?xx7@z_>)3FuM^3 zf);7rVDV#cS-op%Ig7Z5>iLmeR5d2rvq5|w#n6h3aiH|=d+39$=O1?(?!MOL>TBXc+u(jh_Wi5bfAsM@>;eJyb;L}~!6d%Yo7{)wfnA(GQw>HTJw zf>R{gR1kinJcV-uQY6G_Db)39_)Ai`os6^gPJFImHCb>jSU7|=HB4^=Fml0|C9%k4 z)Ie)m)0D^X6G9c}PS*v4WsJYVLy2h&5_nH{_fS5o4~QTj1U&8RZEY=qxq#PDAwvST zijsb0NA8kjXv+^_;+O$%ZIv8^XwLvdyLuc!05UOYB+2I3u{UoHWw+;n4u)44>&RWn zUFZ;BDZu{Kf%)|)n#N=_+{Oz0A4i_y1{>=47;u94dR!?vqE0JG5gw!^LG3(&>RIGP zi_rXKw?~hP!PU{#236c1s-N9>AF;*`6cPiIN1U=M#Of_N{y@h_MVj9)2G$uYw>9?nxCx$`JLG@+htmhCu<< z)ub7rD-0PJ=+pgs_jdR9_m`~N=XU6yL8W*A>;brsM*TBzP1e?R&1Gd}MMb(b5uye( z;qTsY{))ZyZ4D3dtCzl|thr<((^gdtwhZNqU0hjFiEqFpG!f#vz|`fde?>$^VO(Sq z;L*;9?+}eq;3A~$WMAv+>k9~fF1@`TjLpHXJTnfQkSWwezyH;a{Qr-$w+^a$?ZQB9 z5mZ2s7TlDSbSj}+LD`}RNQZQHhl0vRl$MYNY3bN>gLHQ{BHfL|UEA~Bxp(d#ckbLX z&Ya_!{i`?DyVkRw^(;3rHnt4dRS-=+ciAKfYUTxBqOR^K;wgL-)Y-X;vV4EWza!^# zEEd^r58ABxswHp_2)H2;42ZF(zdu%M5FmQMCh!|C0bT@plA$+UFG0}N`Eq4x33OS5 z;7rFVz8dmESs`e@s-^UTy9#=|5bTfWa)TuSgOzee0qpW3KweNPgp0s(RY2%cY&Y+o zH0rNv5t;O^K0 z`1&t_3MMtYq$8wV|Dn@XBBai26EQI9FnB4$rr-S-Y5xKS3NLsD4ir>Lm=~R4W1qD# zko#npOKyWT1G543X^??Q)K~|%g~C@Ndk|}&DC{7g0CF}!O=Tj6OLe)yEFK;nVoTgX zh$k@5dDpmcpPuL7V)Xm1kwRCNQ;Dh!ZoALZ@BmUINT7i79K76%(D}|h+~5}QMj&T0 z0H-i0hIX)Di5&oMl0`9#ii(O&1RF|#$gw;ei?rfi&+j`|oI>>GuO6BJvjP4hxatZp z5ArPX5Z%9u5nd#F2!cphJ`XrGL#F&8s9i{({`qBLtTEcPSO$>}6yHC3^dX^sa8Pk+ z9^%8AZ*DBwhzIMgK(b?RuVW);GiZ_lflg0fX3Z{-?_C5^ zZJy1^MmE5z52^7%3>F+cfQdmg`VAOkK-a-_q@<#P3<;#W0H86c(3mVVQ(eqii?pbO zb%=`@h7}gk!RH6H7@%&<0re3f0Bl0iDQE-~0HLlFh9IbY`6?j+0amvVtQ4FUkJ&^2 z9vwjfJhI1f8je^jzz_3hwHOC+fh*8Jz?wqRssW4=uz$L-Qg4Zg;PGSiIAH<;*$c+t zhv0;Oi0d=8@mz<((>o7rhp?H8lfS@f zB1i1s(s?p(0N;;I$iv=^R@W!w@P&AKdG|>4{SlUt{~_bsz5n=YO_5tqL+P=&hZE8N zm)a*N@x(83bFnafQPv!g*`Ij3YSG&zwg409D^&Ot2LbWW1IXh;+)t$n8=CZ)FU2mn z8dR#*Jg~%1K>{_Vj zHVt=D*XN&+7CU(ZzHU_Nd@9wbK(s}&M)ui4fB^nvK#YaYDc>(4suR}I?u~K1$qC!dKfmVF<0Pugw)xeaGB&;C z-3lnHIt}wgCxyQ?>alPxWS@rTSXk9K%(wIjIW#hWh*8sTuVTB{KSQ^^1qbH?XHQ`A z9SBWfef)tFe*vU?J=L!rJuYOyYDaZ2gOW+-WGZMu3 z5=-BK92z8XgMzN|of1(mP~RLoJ)F7dH~7?+zc5PSBD7DpMwtR5m7FxYA$!CKRYi~Qd z=lYuTrduxa4l;-7~+{Y{-q<7_eSCnIJ|Fq0$$QWSv1HLW zd$;ogBHKP@Yt)}H{jh#Ms#Cmvnbf@EL8*(lB5A!%Cp}mFn}T2qk6RNezFU`?i0;dMZBr zlgzB%n&cRQeoLy?KSFy4qA0+o%(~+}q!@{i;04^I;9Y>E5y%(Q0N@TRBUnZ{nBl~) z*n}CFV{m4w029JWlEZ13vswS|}kS$RJ zL%TDt20aX0MdZFcYzBEc?!W15Zsl4*MgG~3%7&5 zY0L4~WC$B7f!NYB2=b#D$~@pnZ~bmII2P8>_wy|VXaER;@aYZ^HbIaL=^YT{t>E=d zc!ABhfJ<2nAru%X@M=>JF0)HU;KUH+rJ{Kb)@lUYvty(tdn*uduC-0{L&?n#P`vHwtr2V zg`zzO-WR!29LTdGo=O?lPDQ^BKC5(z%U1DVU&3tckSwmw$7lU@?LJoPtGc?C#<8;7 zUE^Uy%C)^9{lwwD6UXAmb<%b>!3t#@F2Q^qQ{&pttyeLda!oWO>%}As^G6bycNdLp z%PS}6t3sD^Q~y-2PG(jG+qV?c&tBbrH9(VxdDqz<%}6ci$_x-??#SE4!s(;&MER=I zZB)^X`K(4|vptJJ4?ZUghZ5?@rJECnn=_cy&(FkaZ%+;%&+eJ{k-t*1EImP=7?D~X z%SyS&m7xiIKd)2o%+VmfsLbKLVO}gV9PM8@z8>%p^leJ3Zl-w$%N*VQ7SlqE`sLf( z5IOJ8xXyIi&+Jjw>3(}QDOA?a;;Zsu+_P0baj`=N%O?DWhg#ok{^FI1oH*2*MTF-# ze3NdCdmMW_v#U}yWEaawqrR?GTBx1Sm-D1YDgcX;LLdPyKfQ=N6EX4&79a*CVeBpd z)&Zc7wtFkeh^G)_0`R-Mx+(%}P5^Mqii_bF0G` zb`ALj)*FXLc`>YKZ0;5^)zDo4W8BZu*aQ4!C}E``B>c9`8WWrPCJn3~@GV>c6NfMX zFkEt2k~+P|0DdCB09}a+K(dn(Abfx~flY4$I~B|(_|bt0*{zURtAsEKh(%a34&)X9 zY=S?xk?(f8pF!{*3*8Zyhv=8IoUe;6)6yzsi;;AnE`O#$3{eN?yXf)8FR;DfQyD<^6Jkd2;xGnlp4k2tAk+<1ziwG9 zH|8-e4&vsxs7YQ#WRE0R-$1*H|L2~P<9FSz*+4HX$vBG94Uf78WV&YWi*2eRLZZjXjHHT2`2oLSxC z(w}Hvb|4CUH#9XbP0@IdazDy(N1#t1v;X&a-)3;Ol1jnmZ`&YC=+5fj!*3=Iw?c4- zQ|niexGuJ4Dl>gE_8}TDG}}kDW`m-R={+28D^BOS6!&8*&vr-JVvI6m8Z0_~jv&9R zzU4hTs!);DNig=f8SSl`y*d}PIm`sRAWP3U#VnV6*L9rn#Von7v7KX%sn2*~XsSBV z-u-R`r}nZr0t7lu^V->(9Ie4V_pyzH9CzPv*^0u8nK*gT`<}&vi}dK%mx%=?uDKK& ze0+9d-jm16@PuaVpo3~x`jx|%7BjG{03bHWy@jn2Jgccn0^;TpnwmcVG#_bzBq0DN zI0$Sws~J>ZLkMb(3o2nxK~RPK0)EfxSjjec z!Qg!ZERIbm!0$Y;7$%r`6~OG^uSv|nenJE_gI&gEci7$*2(!-@*{Sqc%KA#sp&@TM z3)K-n2A4sn?hes&%I83Fb8#^cEP`m$c5O@|Ei97;*8DB%ha z`tAWx%3<=oB909BE|4w)S`MNP9A<2x1^_*{2y7(*Bw!#oj25zC*ZV zxcI<5FyoI@iy`IM3EieVg43hNwfsi$e^FLkH!W}!&KrT#EU;>bMwIkUB0mSR&# z{``IS%={$DjW!T<#YumzWj<}}WVeG3PK$v=B@e|%BP8;Va&&euYRe4hoFqmTVw7B+ z{J!F-AfwH-EP6EOcp2^G&ncCyR5s6)+Oy`CA5P4vl-h4~@7gE4`ZL3be(ii22f_9g ze@j)9Nxd|mV5XVFaPK`I;OsGDbU-<_79KDz0Pyx-pBu2JlF+(fdS>Pp85z?ZPS+K6 z5EMs#0l)|uY5R8kN09{`A(a)2$n_0TL@!O}7Z+o1rEG~^d3 zKg$0&9S((3hfi+>f7(B9CN0QPU_ZA>Ajei!C?W5nnTV0=?pOqytvmbr(WQ$dl!Qtg!g*jXPv|EX< zL~YMl`HZ!QC?+b!-?R14uW1@~!8>=d1ys#aue9RbG{xB%z(G*clW8PZ2P{`jI6ZE7 zix-IyUpo?;U6@O|mldL0H0tB5Rn*FNMtwEVWDHHf@6KM~uqcNn*soecP~96^;YXku zxFl{*F`@%ZL_7%i-&^Y|E--BUCgoMFitxXq_#K)9k(GaY{_@S)`7tkM;gHmsW)vYy+#8Cb8+F?AA*r~XRa*?xK5 zLG;e+3JGO5Ulr(s(!c5^Jl9elKEIeuf}~&8C_KMb@^?9!6jXJ7Q|z~`WVfoTc}SV_ z9FNjGaz0iXay(Ygf>HQ%QYAxr7|V`9P1gGB2Sx->OKE5uRs9$~=?5xW|MG69K$1qb z8yiwsB2$5t*b6{pY`X+Amyz*@6&kVA<})wY?iJbXNtWR;&7E&#bj`yAsS^dK}B2_qMjI;Q@xc{;Okn2#{5tvp$p)u+{$# zrN7X4LXVf?&O9#^p(z7~OseLw@;!)h;ohVOQ!T}q4l*;O`2 z2cIRq?^i!Lz+VsltH{wfHnO`XVCN-D(N7W?!DJ^L__dLoExC=DS$<)&E>GivP+?}$ z&%5sg8McjW280A^%`(zQ$^L$2iJ4Vq|w)I)(r z1_t`@@)~4d2R(Yj(&e~#`o+}T!LF-0IZd35dX#7P41EZlenLgkIe~@Y<1H`;%$8A98ZmP55HwA_CpR>i*(UWgRM%eH`Eissm%t zyVlS1Mp{~0V!2<&{9yKEWMmfK4!&g!0$-2~5?&i;kXB?R#(jo_dKKu7<#fDZY2TV_ zc0$GN#Crr}4ZlWUPQfHX>c{f=^DiJAP_By^E_GP1goq2w8Z)t1Z_3$8x1h^j25>At z?|%t^4ikp3ul#BOEQ2mRZ{P3sOJ&KhJ7449feg`hYwuWKI9;?~C7;g$m;?j=hWTjbdP$ zQ|vRvsQKOqjnVKc*?qLXaH__K8(Kk5BU77VX;h`%BUU0sy>NQ)^EUIvgJRL6$D-s` zhp)7QciJ~Bt%#8Hr)M=aVSVU((yxW(-=Lym(+6AoXqYpjh6heHC=hk-p8g&4p2Q%x zE%<;U{C?qVs7_*|JnAT-jX%Bp6h(-?7%P#9v*U8io87Npq+}*a?H}*xHIa9=^M3QJ z>ik+(OCs;#w6WtSOl{ApvG8-KQ;2cqpJfOLb*sRrLbb6RH?P_C=bjGR=Tk}DUW^sh z-6D;vvS+g&Pl=G_OZs;G73uoY^zaXt&0GkR8T*wtXBUN6P2f^kHO^2=FIgqP9frmisys@{p2JCEnvEv-bxj_~R2z1ys zNAMR^QZG*|t0HW|6fJOikI)TiSLWa_2ue6w?&Ku~r0>NDk z7S(+E6gb$X(9i~W&QkX{qaHeB0Sf^@g4@`Pa53c0p!^g7gI0j8plpi~ZE3%~Ue0L1 zZ@c=#YjAe0VVOq#_;gp%xGY`0aW&F~e`{pZMN4O|gZanqca0_0Oy7Tw6V>osFELS>csyG!* z4nX0=;ZLKTctVJJF5grSn=U&pd$nae#^c&c>LBWR-+7qqWpmeu)5!-KFK;}|$Q)8? ziV?>1VuM8>RH7CW{!rjH)9Xj2K1TqQC%P=V0;EO0$Nm#IyFdhk6biO=CB&jYiNzN^ zsLpkP=7xWkmm~2xzU{NW+yaRKe6cz(Syff6WvkR1dn5|lqpfQ5v>{P`-Dr2;Gp zz@#=I^@44n1=HViTP6kQnh;1pB!DHnLm~mo!UPnpH~b7(^>-Y+Aib{)RW*>4!d67N zogQORCmK;q6sxY^*H^kxM~`mW?WzqM83Rg0nJ#g2@}z&N>lH#&JkjQvxCoxP z+j9EVQcsHvs$tv1t`h<2)3Lc;MZ8cK)6b_W>rR=C)WvqCN3|MnPfDC!m#v;?tiHR4 zNIQw(V#n>rH*1J3aZ5T_xq07}qB~V>Hn8$Bf7xoTkS+6Mez8N`O_RpVBNKlD+UB1R z$*ofBSqt*cemOStrOGUHl`u7atsbT`_8{JK-1km+pG?ENlD*o+Iiz2Nhak2d-jcl0 zO1w>R)F0bEImu(uJGcKJlxAp+IrPP;c2hck3yQ~#6y+4ho=Ec;*@_uLZSs%Q6GrQz5Wa4tz z#4UQ>tY|WF040}P=(aCyotWrGeky8rmcI6wX5G~keKD^Y9QAjQ0)jR0{5$4s5GuLA z8jcoOoW(BapEH0u?g?;W_D0@1Ln7oQ3Wai^x&6D8%VnObigeGV$|Enuhr8)L=2WR* zrZ$Vm-PKyA9IZZTBdZ%#x#&`6;Gc2goLTPpZnZCDXf5`*@NYUvkDbaUcXORQqb_$< zR#4!Xl7(Ms_{nafRzaOv#G*_Ur4gbr-+cT57uV*&W>jTS2c*Tyq2m-K{*7}@iClIr zPm}60@kV1e6NT7mh5r1NqZBjZ{gA&ggL_zV(@^nGq@C91wi}N-BuUcW|4G9ekNAG~ zet9srU4i=G)^0^4{ud42@+Ne1`4cdjJW*wsS_uEH~i+KY$O4)eN%r9cc`YJ$Ke8#a1klt{%XGz0h{k=b>8c9ArswqATXUe>q|C6L+EE-zR8OtnjK4TR~St1Da4;O%Ni z_Zy$OUwFhwNVX9On2UYnI52W{oP(g}1er&72LEVAv-1+U8B!h#MomIDCABAs&Je?xbZ)+h!D6Kx8C&0dfa;8F1eQ;R3zwcmuT}qvB zKK;^DFAcIbg@k5f?$JzsEzB_RK!FS}yuu%ugSYr`+_-4etX7{Ci4)#tju~cCK=3sG z<_=e54{G(0YWi(mvZI6uVkSnB7L1ACA5&j3XIF8f5y1X=t9PEpqd`1b0V@5i-1TrWAE`D&S8Q`LoXLYZ%IxODQ<=&rR$7{I-Zp(;C{m6O|Aqr zqY1bRQX;)W%NT|YrTCiyZ|vz-^owi3G-R|eCs^45PR)_y1?xlzUHu{SHA*|4 zi(h8{E;}>J!1rJlX}*ygjNTK5d=VK}uyY&1Hk@x*rrxAm8!s@D$#0W};A45UxJ2t9 zGg@FAsjx~iwJekKVR%xN`X?0ewI^d&AXcS>;~GpGf4H3819y#Ve?P^#MX7G9y`O?Q z?*h5FhRt1t5xR1dcf$#4&5Od@_=AR{0+$RGor+T{#@d(}6DG8!29gr#%MKk&OH0!E z*o_M!FgKuRrIVI(iXrX=itAo7{h04N$N9=tEic-7O-U}L8xrZ=xFY$b9)D z`!)IN=yC9U+kf>ZSe~Dp@lZc`uUk2R+rbZY0_9%SAI>;ojbT*e>{5^3L^Jb0KIQDc zMQv6^ZuF`zX-Re)`N}dU{60UXKE4-@1{gj8MeiDIVicw$EGph+*LJY9BvoB4m?cTA zV_XI#La33N)ox*2@)q5TG!uSb%H9vg1hBm5Qw_9BrjPE|EgW4fQID4652|O|Dg8ob zW1ZIa#ALsx_%`mweFY`e>SwcyYw_K`?LWf!oxxXDHJTSK%gNnvu0XKM#QZEmKO41p zN>0N_PGnd6-Lr#5P^U=czH_>`1#cZAQ$7VLheP^=abPNF0CiESRS5H{D28~MOTFr+`Gur#jA$)w|-`%Olm;c&)dc!$hf=MF;{yE1LO|uL~@_+Z+`uX+?TCgU#8RX zmDx1PRz13vv*4ThEtelN92}f>gXPD4lAkv=%RvW7u$(V&rJSFd?P1|`p)qJ5EZ>M=~5$%6b^#Ozu`4(3oTM85HQD#V&170^c(h2 zzDy2&^W7eHP8yEY8^!^B{SWVRPSCvkkjvNv+cPvnS z$}f}I-UcAq6iHDD&^ayb4Jrf` zr@*4tXI+5GQfMK@4VOj8HR)NGfbmr|v-j@TFC0qJYPph7uUi<_rtd9o!AYOA#K!N< z;2r2gfE+TjkL0|U!g%`rS!_Te+iwFL=LOaLfM@5$%A^|Fp)ubhKLqF9&fL*e1@C_7 zf*oY{y~%M@>|3fsye%Pqp7E)5xDCAv@naoOyt68O_2`?=L7G!s?yicnM z2#W|Tn2pP1v@y0VA2;SpT9BjFQFF&FC1w9%E0eCSmS@QfuOKb1`Dj8LGs5QVz8lBB zyY6D8fq#%>Lkmf&w!a9D=h#%r28GJM9ZCU52NX#4oDcJ<}teiGQ4cM@@TLYRNgT>iTY z{4~-Kq+!9F`y4xqxMQLkyENYd%~BvH*`ntR4p5Sc?hQ^ZX5p6PH$T0OOhm>omZ$&w zNNT9_l(j28c=&CA^4rmZqYdhNv}!=>Zx#<^y4CJYwkhQVACvl-mPfG}5hZQn3^p=O zyy<_sGa|J6&mZnS%)9h0Lz>TEwSvOb-g^St3CgWWsGwhFe!EPJ7uY)IqYIB21~Xrd zj7@VR+r7&`J3>aN>fc-?3M z(GLC$#!PHu#1i#MSK@1`qF zEuWntv!TBrAofoP(puU~e?m_nj1B)58@`& zt6HdxUC(k(sL?IZU9}4`WyGwx`t+MS1bI7hIFLFB2;lp(MTvo(zWDKcNvC#U7?EAx zf$oz7_d2&Vz2=j^J8ExO1QZp0%OX`n6xGrzX?4R(=fc}Hxt}#et~Dy(va=6To{p10 za>hxb4f+Tln_-f4t6{y!$d(yQlqp8vN;5~vP9_&Du;N>37p-WSCT#uO-ADp5aBuW6 zG8WIjew8kDedhMChb-2xXcT=S&Y|}EE|5))Ni)kL3!k4)stSF+f2El}y5`nN*TMe# z@&1Pm2Tw80x&=zB{E_OVO@V4zN9svErN-X3`xM@blTT${->@%LE0ARjEje&;q6g7w zZZJ+QJ>RYa3!M?&Z}_T&h#{zz58=^I;VU~EZ`Gca3u5@w;?OR5UPA1>{AE$6a#Qi>L@Czo-RvQFc#OE9L;1L*EPOiz8A zUao1*oSj0E)VHYS=SClHI9o;aj{v+9H6qz2x_(XH9H!s62sY(HZDbQVXVjxz7hXnB z>iJDuyWHFChJ;9LZ$JuiZdJ6eO3Fu}Hpfn#QUE$A^5 zc*l+kusD;0>k`A31y%(TeDQ-;6e9b28Pk6Tv%trqGG#9xG2uBT**r9etkL}{*>`3m z#m06@RY8 zgy&gPxf8%t4VjX-GtX({V_EyA;;o=ufmH_oB6Z{3<-DlJ7GM1Rxa#SLsF$R^GAkq~ z>l&#LAb*CSd{r{7t-h5Rp^@xSlT&R}grU~QUhLchhTIfw$@h9c)tMKmn(-&pX9$qr zNKL)wrY3c=tpDJ0Qy)u7SvqPNE32cfM!e|v!|^6JVpl3JQciXkf5yzJ*=nCo4w$?4 zV6X4S&R)N0?b)r#S%0sEX6ZNc6*(`BA&j+D&ePP_kopbrR9IUIJPrQy({X(K=bPvb zO>Vs*t0rR%#eK$T4GwQ##feuAZ6DOYrwP_D2OXh$?QZg2lJs361^V|uH_hM~%H$W1 zuelWjxu|gw9y{iHut(dVNY?rz^z3{o`Kp{KwTjYAasV&#K3?O4hY-Vfppv?bq1f?L zE}x8n6d#$-m7w~laIGu+kuNQ{6Df9*6Nz6)#lJMLe2^;jvwXZiD+@e}m=e}fQP%J` z0mIr}FP>Kd+vl;`fJW_YY=Lf+)}yuOlBlm`cG6?cnyMhmiD3KF);6qPltOU>UR>28|Ebs;l8wMJA(8$cJNtbHPK$E3!^C9NA-V-iG{X7A zE4Yd!#ALc1WtwpM=u#Vy1@uB~f!dpV2LA4tpFBy4D(%Wn72WyZ3qkx=R@y|2YtTE? zPGS<@5n-I8Vg?KQ`(hb`k{Le+$|aPp3(&H4Ut5#{MG9Ny-zjfceh-&lwdh{HVRdcK zlNlNX-(#THU(SO)b^2KI(JP+hCEwhyEuPLL9lG6=tHWKHZ)toU3fs*%J;9If&8CR*-GGqiq@GNH*-5m6hV&=r1GOAO`HFY?owidFsx$ z6o){3^@EDwyDz}9*+2(xNEaniT>qBa(4=INhKQ8lI2H?6OASbs=bjI}m#3O(U(sxy zJG{DQF#f9$4_s_EW@l9|(dg#iYguWZM6Hu#Q(rUvUR2Lm6mNnAWZAejU*o;xF(1x` zdn2a&+;CbrTKQ-_>HxpVKJw+l%hY}C!1bppwu`Fldl|>POZDcRaOLERf24^D`KHJ!ZwBjPEO>-bJG;M`4%J>nWzz88Z06IeQM? z;>IWN7KJ1#Y69bV8tB7jf=>M_#PEaIT5n$HEUO$>m>k!6Ym=VFmWpyYf8mk}!H>hI z1%`&_Y2rO!UU-c0S`3e?Z+nOQkxY0YIc_>q%Q@onp4J+{IDI<(HlIIIgj^WAFJ zl2ifJbZBRvpnIkbKG))??^QY+e+#~aWO5ow@@b^PrE{-^NnSN3oZ>zEZDMMm_c0wqc2;HL`*BQMEx zUKv%3tZgnNgaSwVSdOX2(U|pm)0wz`Wa6SCFy;iKjX*Y`Rr9V_-q`zwEsbN|e&OW^fJ6{cr2Ig}EI z|5SGCd1sD&=MP*kTfA??l~?xDC%TwL3dk|srNOi^sMVd_(*S<$l{?25rA-J+N=a7n z(<)+YI&{W#)QppSayuT(b2bnYNu)N0b&5=G&pONg9lCus!=3CeL>Rkt<8-rLN+zTX zgc4o!^&%9n!lSCaFI~98c_W@}`y^&1Nrf8oaN!bjLbMGk)fn?@eeGbdv+K!)&sIwq zH@i%Re23YOMdO>O?2I(zn-(TBwuE3glEcSSe`Scpx7#;nPzNKIW&M-)tYbd4d16>39$f^zv7<5uFi8)m(QKQ7$#_tVG(fWwsmk>YUlwmio{4_ef{b9_VL&Z;WO<|-x@10 zczpZTIS)DA+lWJY8GD>~iNW_R{s zE&?5eF*4frqDKg2IYhb;b4YDI)J&3XAc%V-qlG%*3>!(pzWGM&hgxRMnY4;Z3S9{% z-Z{#Z*>u*B@8a_E!3;tF=6LT;VG^ zDYe7gFC(4O)Crnk3z44#`(8CanYY0jifRt47&!!;Uvb7a%liuAa5S+UbICH`;z z)AQx^N#y3Ttvi%;F?qH(C7J7)ya)GZ{NG;Fc<^jyD(AEL)u6GTS)~ zC|J%X4-&i^=P&Y^T-=A&Cm^>&DJ7q1B&3$4RyS$QC$DHdwBm7u-gr0X&T-eazj91Y zhw^ehSA(I>7_SW(0TM?@CnNaz%d#K;T4iWl2mAjOBP*2oE!wJ@^G_6O$Wtfw5cbv{ z)ye6P>kQupv$hYfm@wuO1_@UCj39@~d4D1CxrfPrYO!x4aZ`hdWxpfIOqdZqT($m3 zR6o3ya{* z3Q@fG6i4(MoD2#~s*-ZJ>erOt$hM@V<)sCnu4NJ+(E*<4-5dRjqwb>@eKQ77CT98p zhpRq($d^@m|C+P@_aVI@c_`^Bz0vL{n~^YC)N)k7Cs1?pzz00+Q{%mZd48l&v5cI={X3tet1l&j(g#MyBrt0*nI-_c7Ahe`ViKV>7cnJ z$h3TNZ90@p#)*6gbcA1N;&8CJq}x2BO7$#}!qaKH?p%mKH%NBMsy_dhWT}?>h&6nv zXudt(3{C2q<-!?k*r({cAWu2FeDqu&UTM2w11&g3L>kG8=ub%7EWZ0U2@ujH&DBQ!}7 z(Mm$UMn+oQ|EkkK76+v^;PTv(xRy|-6K1otZCquMDSlH=UZ!Q%<_9xEl8m81CFfnA zK7FJj<9%BN90Ut=r$=Fs11ZP|Z zRCvu}h3!Tbw6mX0I#+*J$tS0Ay#B|Be`abVyy5P)u=Jad*Rm!$E5H6+o82*tbs5`^ zR}d+&=PxdIt`j$t8AiT&SV7zBoyN`6qCrOIBQpu7E%pPXul^y>oKc#3sKXiFPVJmz zCm%fVYVi8@FTWb<1dUs6voC5|=hL#sC{J5P^(=lSg1X=}L+1j#(X38rC5B|mqhTwZIUey(e|R%@_fAMrs4d_hLPqp|0o^@D9#6WLm7DV>ee zm*1*2eDfbh$QejT{@&)g{zh@}RhSy>6Cm>!jZaypqU=Y0I@rr!Oq@EXG*9 zWhuE_UPkT9^AvxLLX8+>G{tY{I{pSG;+;4 zN(fya5HiaM(B9b-xu?52_`a>m`OVS!TC(mG8k5bE$o}^BcPVng;>>kYt#l$!nUX|3 zss3}N?=ZFBghfZaJCvb77G={A5ueu6%~V#D>EKLMp=M{pQ(;xrutZEcem*f zQbaw*^Lo&9%27pCVa4%oG0M%zgI|YAxLUJIuHg_kdFl+VUC@^+R~<_J4Xz*s)*O8*`Mhu=bzMs$lgrV**JwV9 zKRNhu0Y%LTi4Knab6#owXcD_`kjOG%|f`9aDWTSa7*31MjM2 zW`t@?L6F8z<| zMN5vazqqCuh{Wtu81UWI)fWl)I%P%1p8s6rqv3ys<_0!~a9tR+C)%}cn|fyMtVAsC zyhya$gz%tt0aq@|{`mH0l99VtPbxOEO*%Fsc}sbBwl_`e9=v9eK! zb)-NuH`CK;_>PiY+L34#j^q8Cq$=g=_$qc%Od0<3V=@VEGlAG4mFn35Yo$LA$k{fOZ!JkIWV1cZyz&&Q%XdzY zkW=wu@1H}_{qkuw9yQ}gWN6s{4&U-)k$ujZa%#_3Gww-4j*ch!*dKog0=FydypNS^ zcAU4tWo;IYdd|q*FEK7*0ggslyDj47-2Pmj|2x<2U4}QH)S^o07pEt#sCxNsE#K-2 zDnsAU}pnUs>Ed)hu^(U;n2AQC@MBB621d9~CC zJ@Jn2dAu?{mQr($gIG!H9HvHSKFWV&*V4(4G%rU%@fL({k-J?nls6GloLX%b{l5yt z`3EzKR58@V*o!oHVMQu?nb^IXE1#-NO(aVmAiK0#tMiLY2l;co-r)O4wtC!)c4hzn z9&s0(3N{a`x&abAbY8R{v=?i)WhOx8TslT2yP@U&0U@=0Cal%R@MH!ZjQC=hfP*Fb~``z7UJ*yl8g zbehnEkoFjN?$4nSj)(5b_)Bl-sJ#{c{NnM}EomR+qlw41@I5DW9ait2;Cy+%Elw(% z!Iz)1Uu4#<+v*GANQB8!d(^b3Sv$3>hFf%WwmVBPG+UFypG-l4irPesrfm7phmRh`3-0LrsXiK*JJ-wzn4lQ&NHanv1iNl6xjUH^(=- zuJCDu4`D;!{HYq9aM41nvPy-m>wpd<5g=5vs{t#(1K=<$Wo)<1%pR&DH+a*?{(jS;6mR9-Zeyqi z+Xy5ooL4#pOhN}{wpnX>s1U?N3+7khx(`O+(?qdDFp*2+IT5mLZ8V|9mewL z#`|{tTiLj6Mv#GS* z^HI^D$d9nMTL?(syE;H-@;2Y@^D`4XG*`B=iH6wQGW+kH0+HLR8p$Ie48r(mrD2uAtlQsd~PDwGCNu|(ypeG`TV}i zhg8n&Hc~gO4kqZRP+2K5E)U<(M@I5ZWaS=`yrliJz3Cmb%lXep6M%~+oPv|&SKQEp zyz>nrprL4 zdZvk?5+z*PMf(EJHm-Sz5dVUdru@HmN`cdEssvPnI{N>pTBT1^T1rrgjE#xAxLcR0 znmV+cRDNseVArg^q=Qrk5@$f zpHO=;{%Fz3S5C;3=t>(L&Af}g^6$KIya~~y#-jp#I@HY3NMQGf(G#EE1SCv957R?; z?QBK6qnfD=_vZ| zem-0cVg3ZjCKVIkZ=w#Oz31t^K2trffjXAGLZPLX!L49q;M3b+Y*vsvLTlpF^NKRP z{g*tB&$T49OdFFHu^f_&Ze^M-jJx*QX7LXtOEjh3*mFb7t4KSA>`e)zh+o{^KU*2| z1ojIl72=Z4NMd#!B@&NUHosV#nD=cvgx>`^Cee7Mhb`V5o^1u;gg`IQei`Nj@MiyG^QaTa`|u z=^f2?Vn4>DE|u$7B2OmV;}=5vUw=_+K`o^2tN=(4xzLh0Ev9UG7`ylPik_ejk zH&V~;#RjK?jl|)SS+Z@8P9%4jx$p-Wn0QWH|D_sponK;ah+3vR0+39+PohktVg}Yd z=J4DGol8*FXyxPY|8D+{iiPA|&|xJr&ZQ!Dt(gT%snldugQ#1bTZV?VWLprY&v<7~ zG(HI*+S3Da#NI^$a7#Z~6pg3$JdxMG?KX3!h{nd|XN4HHfwx3ac;8f?bWR_Op5>yJ zja1Avf)4M%^bd|OlH%8EP_Cmtr{frkqfKd#W3s(Z5>3kj4@Km;20hxQC*bp0kz!8R z?ly-Rd=FB^h6vz!HrRkc=(DOz0#1cJWfHTf8g9X>v;F z9hwArHv?f36~^}r1u6jRLpA&Ax@GF*{YLH?{I{Wh*PZifBuRCWM}C=x6UM@K+PHA3 z80sxrY6*jYy(Da5YP2tav?c}{d=mVmG$E62$B$e-Alak~ z|1`^2{df*;P`n)ym`Hcgrc~xu*Pd_9jz1pce&+lDD~I zQgENceUxX;ghQ;bqQcl%h%K&CeE=SxFx!*y%wiVrc2(dn1vTljF?ABfR$ad-QBYjk z*6S~qRm|jp>V3nzYa3*Fwel`cn}-8IvsLX>iGO?$Hj^?M-UL!=X-}mIn#`Hpg^aDf z@YDBmYxd_Gm_~NSEE^Jc8`pokQgE=QV!e`=coZG&5;jV1%%JzgqcjDTma)QlP>;L0yKiP82%NI-{#KyZZQb=A<#Uv18NBP#?npCYTDO6m zPQX7Dfe?=*9#c|ndnpd3-|K?BgVrs~ zyxQ|^O7EDy5GJccr^_Knn5#Rl@)8X(ggcu5JhW)17UY%~rB zo3NwOImgCx2Mqk^#&~j1^oJL-&7?}6`cHCwG`DM@Dm&tg=ezG?Ka_IVg?&Q z(aCwWY^2jJlP}DIU5kweXy9JwoI6MNO2c@um1%b_cw}yj$L@E-P`!>d;Tuu_({qyg zDK!sX5GPgq!i7#to?T%fDXO74;ZNhGFDs#pAgqiTwX}GCp9X8zb`da^!F#o`1U7W@7l6Ve67JS#Luqcx45dJ!jn>Cm>=20 zkcYp6>^hyX!nGIY=Nkt-8nv`1g_N65#p%vZ`}DjB`wIcoJJJE)Hk#4UQJ~k_rEhtv z;hsU2JMG?ZCA#DTcuV`BE$59qHy2Kma4o~9ZS#XyWpU-wy7%&y2Iih)*;~=wx04mR zzevHz;EDMCtBX_Ztt>AwR-h9LeF@pkbYktYe*IaUk-UM4MS`z$}*T}00H79 zJq{fT+{Bv^{_3yR-qI6EEO*+Y8m#)32`69Hwr3CL9qfxlbpU&=6X(U5be69cf+Hh-tEM&Gx%6*nM~|)p4Y1w1QcCjw{#X=?K9ynho}f(lv@LZzaX>a6OwF{Izay8Y7!xhG63^eH6P?%rY^f}w~Ec+43=(G0EMH@Y{Y{$y-+Ko9P zvApehUA{7uBcrWQwnaV23!jfQg||DeYk;IerW%KbbwR-GRsRq=wyI|g+>#&U^%Rbd zbvciY4h=3{(RRn@<;TYTo@bXRzCl)cY>EIIG1hz^a=BY=DqHRAbI_w@q459f|4MpE z9^?OJHj^_o)cKk9ToQ1~(+w&PTV-5VJnku*6kG~-qbiHHSkrZeN$_w9i@*H&Z9O^4 z+OKx}<$<+FTfv?p4Idk<;Uc#*f8dqwvbtU~wmdG@Zf%0IQ>$!wjw^extT&r$9+oM( z;-1YC>sFmCR4~(jky6up24mM!0%3hm3Kib_r5%q4cT=6?X&Hrmh1%Y#wD(j~v^k^v zO-1oW!|ZFjcyi}Oa}aY1&bMR;#Q8F92ap#{-oFH#=y{fou7r|ab2HjsUuDeF9Fho{kijJ&mswzHu z`Ds}^5WpjHe0^yDQYtc75c$RU<&>I!!f+C4K2GVn;PzNZfjm>1_nUE5DeV z{4AlQkT*KAbG0GOuV$xnYIn^QZ(71P+>2qZNg4TES@RT}DVNb<-%%i3*c|}~MFMhp zZvOMXdOe_XPQOY~k_?q9-rhAIQgMl-#ILO-i+y@>{frBOGF*F^Sp2@p;+wBb#qp(Q zO1i-d#uu0Xs;teU)NlQ?Xl2El z3jALTLj_G_hq6~JtbS>z{SlN|y%i>fh_y_<}G4>{Ku@tqY`NwVM3vp3~2tu+H<{OayM;_vIglC z6EOI}3}99)kum~Ty<4XCI{R8wdV$7#NQSne)6e6|e4QW$CuDI&U!w8Ss>lA!)22l) zy`4M|w*iLWE@@!m0}e6p#b;w9JY#{GmEQNSD2uI{%je-X&kM#F0v{QQXhVoFl&zUYXGA)i>NO^>K4{tcx+d)pay7;zINBc3x{+8eWD!b0EcY>DsA$QTRe_IECX@Oxi@&_xE0Cxntxz5&PArW@wzEk;GBri; z_yHFuj-M`UHd9IMx|g>$rvC~53m|4+1dtmz?jy&hge2^KMM-p(dH+~>Zx9v`>aq zY|MLre>zOs3&I)hWJ|acY*YJ=A^z)GPM1w1w3<7lxU(mn;8g%JU1zeL-rE*3IH0p z{Y7pZRAmM-+7#Baudd~;`q{bM#~tbyuu;tF6RmiCt2sj7tox;RHN9_^=N|aZ$b*FOSN{chxi;Gdt1<>sKW+o_;Y##yq*VKDSeudP9mx<#(nJeD=ZAy5H zU5e#KuzUzZGL^~E|48xgYEbauLo2m>p%-)03j#>ctX2~VHFD{EG!4?-`~YD9m| zvJbG)0Y#VYFSzJNF|)zPJ_kaRzCI0Vx%We$jNjPLT*OZR4iWrm(?OHCbPS1XL^he< zqdoHtO$&Ng`1DobcaFUFi<_70Zt{(0Rhi@8!ylM3=HcYJylnabd|K-AGY$zJLqTlG zTUq$G%dF};3`s9{gwRA%c79|r@3!Ca(UHrFx3h<`cd^IWJ4*1i=cxy^>Oew*N?^Nz zE1!WweN1Nl;aO48A-(90VUbtpRLob2aFi1MPKl|j^V|34;%Y_L!mbH;WLN`=JWvV^ z72i(Jb5eCRY?-I2b5+A4wcZYbl@Y+Z#mGuCKqvMxMW0&aTUP!)Bo~?M+~PaGvXL{b zl}CSnW0$yQuAnmF=a?=-sdx{K1cWHqz<68!Wfu3Z>Jmpy#rb$+2DXWvPwyap8R*8Zpq!u+0SEk4OtK2VreLYO)h`5FAWbz>DMTxQQs z9#jQT_XN9AtQ4LTN?8t1eQEP5e&!JROz;9LI1xL#@8OBFP zi=kUYbV3664ut6>iIzzRK--rBYu@&1#fV?PGZ49 zB;Fm*4|yJ+d(Um%S9e_SZU1Cje^WCKH8wRlPNf#Du8vEze?fG)y*Xo(`I>HQnkhF@ zq{*QzKLMEG*{*;hGq|xo$zHp=oi$@8a?&AG5eV&eA`3hi!hHNWUP$_xd1WBsMDo+q zfOyOu7)a3VXBR+{##9Fs?C}Sow$l8@+zw_ombB-x zznR1fZu#%Jq@T}3nM8*L6FZ6Jc(S;7wJBO}&HX6tH0>z(=&J9DhM`GYB3lFpgp7CV zx(Zkp8=uZ}AZId*q@<)Zy8`F;%$`PA1UZ43_Eq+(EhY@KGbLoHiEL)}@8OxePV3Ar zqvISIe4*kREUHJ2z;Wcy0@R372-(;q$x9^@=QrE%uzO6_#HTpnGtfsopU}+^b-p`e zaC-YJB~UzrJ?+75sV@F&jCuU*)(`Oy5XD@0<4(Gyt@GFlSWadQ6Zv$J`;`J7ivgmSD1%B>C$>(0m=+kd>Cf(#QIR|)6FTcIS>`IJ^2h9RzvNnZM3nskS(>|D_RFGZ zU48Sk=(Z3yW>kfqY#e94L|hLQi#d>aj(3G+MD(*tU-AJxf?qwA3q5CUu9JE{4q_R| zrUQ<8i}bsr_C@@Q*q6rF1ItQ;()jb~G#T<|A|O>y{Yz%9fQ6$pELUBOuR6c;b15)9 z`zfsfNWe8U-pe63;syqn2rXvjuE@&M+8bVDxirmp=!XcGkMpX|ya}rRGBkFW*SycV zPkV|qT@orwSAUL|@S>7>zH`nJ?-t3UJE<^8){D57fp?`WktUCa<_TM!?VKq@!7gg9pY$`Qy+c&c z`->X4P@Zp>nVJmUjwn*@mE8=CD59yP;v&%(;QX!s%2)|*6SHDnD3`7^r=?ZDX6lK~ z(+DHhLkpJu4N}bcx;5`f-p{~D}&?dLlbhr0V%jtRw2CyL>=1Wg&wn> z#3r1N-cUoq6_B9b!Q8-e+HAtTcCPEgKiDe1L;@UwqhzIDwpinVmTb90;pLf--#IP} zRzwojZpcb*l$UvZWqo34k9xx$@PD)eZJ$a0ipTi#m058WYdr>Q>;;YA+tp3;%M=j7 zcyOZ9Rj}wnYt_(>ki7N=l^k(}~uOB%0Qhc0}Ex)tyXQE2f~#YsMwfSfwu#9&+i;Hoy^y!%6FayV5YV{QH6IVFpR} zpT?V*sx$~OyOXjRT5s>6fAggqAi-TjSK)0nRWtwFmAnMcw zs;*q*KV2y=|LRZu9{_W-osLdk$(L2!;NqwU7})xt`gbHn<& zaT;JKKdlX ze=PTv>Lwl|b%~uiV?%&f#5NAtyC!$HmmXGCXQ%otz1E1h)1}fhbwUJ2KCgA20)rKt ztG?s?ke>`(HS%wmFq)95(SkOO%s`Gb^?i_yN0A@e?`R7ctyVsI_}+`gn8dBEjs8Z; zaYy0E|B#~$Za{2(lz8)StKBIIUOjpaDE+(OPb^D5$NkHfX%xP{13jq>IJtmyhztOe z_#>*ne-rztM#+F-KJ}}Fh&DYF>1WEPsNO<}_Ky#vv%_U+vafxK0sY4|l%ZGI4}EKxEX9#21>md>_3*8310 zSu7TUpGVK@P%GeKAuL?XoziC`wl0Dd+hW;F-|y3e3#m;1Sn{M@_9;$c0twjT^4gv+ zpW77l*!2ytUR>1gUtD?V9OxW$hRXnf3ewbmxrw3PW5`dy2DM3a##c6KVz9mC^SlRI6tkO=)T6X*TWcMGD-jIV%10)N!IpCjIhBu`%^Y zh{4M}_P`(BQLYrJG?xxi^e|6ZmF3FH9Ds1JhqBCdJF)RX5s|80PssxrHds3p`=+z%ABDHD5O5s?U3^A{}c~Co#NvLo755$d7v(OR_$GE zvNmb^DCTxF9`z)%-TgPWl`8|6 z3MjsOeNDjlt5nP7%b54SfS2{3WQxbcZ2lyA*}1#AD6%j0>EX=L`P6~N`mKPc=;!2^ z+{_nLm+N)MF2yyg@JpzeVbU~9?C_%ufFZ2=gDEBb%SnAXT)tR;Jx>y2e%;jARHuN@ zff2|rW}Vk`8L6|Equ{xgRzsU=8mM$d)R(0+zOx}SWo|-R@u*ZmdhQ*x2y5BH2wJzV zWaP|`e&wnk+aH9?4kl#~cID2e?KqsJR9m4KLAfIG{<;lTEJ|a3b-tN;dJ5w$U)m8_ z_}|$2Rr)6P#9?b>e4Of#{CgSbqKzHd`)H10VTbeS&)KSwdfuWu_| z`DY>uoi%>XW9B`kmAho?^TJBDS^Q%te}#9ek_gF-!YI(@r|`-<{ED*OploxiG2sIO zjViqh(2H4AsMwdCZkqoKirF(;(}Kg?s7e)9>usTP_YI?ay2DtipiDn8Cfe6+R(vEP zGY$?gG->~l;bDbS#|&x{*xLn;>W2!9)&}elChg|6 zKzZIIIW|Q4pIcYCy{$mnOxycOhps|lO-md1N8REi0q{Ie!iX;is592}SM`pRt|7R= z?G2(!S3F7{pRc9=EBgD-h{vQz+w{PX*Uj*a-t|%LM$5FRPT1p=Sox8fJt(>ojHUB< z9D*YbA2Wa$P)@i`@H{G=5a!uSE1@Ef{+t1Bi6`j7e)0>O?ROjxlpW;CmR6Wtf-Z2$ z1+CqJF1W2tD${&xPC7fK*Vk(VQ8L}<(j_R4AYV6D>I-6teVv>0+97?63~bl8+`CMD zVtL%4q_9vwdvB;4FQJm=7jm1@92JF*IRNk`U=VN_<6T*c;q!u}{BQ5~hBmhi>AOL? z(Ja1LE05cA{kLGDQQHf(r2ewK2eLkA8sXRabHAGwoT~z|r6$dQ(}2ZL>QEJFJ0#vEop_qDad-Su1zZ3ZQYf)>@1?_WR8i>9sY^M5RXd3fWfchhCb zEYRpQ0o+7HgH_N;1CN=P<*$!Kq^ubhVAV4r7UoC$Ef!~V^n975g+1r>wE%+Xrn5rY zze|nh@cF(qN5hk%l~%-nV}Akr0F79^HNepIn$G%kuV;+kl|&fDv9)=^);WxY z@qb_`#HWg-$zo8iZ_?`%?|t$+f4N*pVy2~DkJG0=&F!u59w{gvj%qR1<7Pf`90s>L z5yB90yRC4=9D0F78E%+Gs*tYkV@^&>l?73I;;&-hl8dvQ)6<%K)-|Ead3?>tXC`3U zSn=TSuy2lOip$-Yyu`l6?TeZ2_{t1CVv$X1YFjEiRc06@nKeGtj((T$GdzAbQjjln zoK4sXP!f^fbRn-y>@8S_#}URy^^cl*^hF7@XYi!z_nF44s|h%=MU_c-B;c$0^H(?3 zd%L}H+q=o5PE6x0ouB^cY(kEV@Q_3m$N8C);?^V$K z=PMi>F|`CFdlO)6|D~1(>+0TY{4rj)?sIvIvvx7%Op}Eda>JWqR#xLN$@XWwGt5O1w;4jQPOm#E$@YY_h=7LP5 z*I}>bttgEr#?r=CR~0I2{NQS9QiVfJtyf=VruQx3Dh>*2EpxVCuWq7-{AjyfMMXhn z`EUVzV09LrEz7olb!Y3a(h`!q z!_S;i;16xJ81JsFf577C?QTJv(Y@pPt!kkSMfH5_qN`Au7tF`-o<50cpTN0!>Kh|F z>fB6z0%exozXa^FzbeT^H;TaQqa=WVIv_t&+wicT#=yV=Zdby!$DVprq!wkYaX@wY5V&{J7=T%=YH27xBS_+!v{8*#?d{+`C^3Z z55eiQ{zI7brJ)^k!AywDf9eMC-x8A3AuGRNTD9yx*o&IFU42HIx{$HdY?&|7z}0b2P0?TV9cdVZebMx1c)(ZO0H+inUe zt1Je4yVg8Qu{@r<)#T9EwRLi8l_`3JVDHl1eY;*Fx;dlxLe4Zk;a62uG4XKH1Vh6JBYKsZ&wG zn5WtjkD-30-X-Bg85p>!j5moW3<84k?E9eQ_CM_xZgt}xEzsX#dX7_=5Pp~W&I#OZ=n#;NDpOoPjX^@ zX|VNol58Y&UgkfquIdG#o8QK3S*$pvoZJx4kc4k249OjxNR$TDH8_t$rJ(~qvThccYD_^s; z6{ne(2L{^HB*9rSy<#|zn>6f)%%T+ zYR0b;!}6n>x_cczUks;9KpFRPf%78jvBlZo`sK{M31W~?m$%AMd2e^FM6;0`tD+fs zL1N5C;_HCL;pAf?=|rQlqMx78&wf`Brxa2sgD^FV=e7|Q&UATF&|gh=`HK`q8$+BN zMXGhqb$TD}DTFl{ny~jMX+oIqkH5Sq9`&nvtF!q4@=h&=KXoLH^M#Fg4W>vPl-;CZ zukTA!#Jr0c0e;9x_#Eq3PMH@U8T%XdT&>?vZ|R&yeAxjnnni4;p&Y{G2$_57zqm3yAlmM z0|nuC0qm>tXO={}g-P`7KEair2Yd#xww`mO=H{43OZ=~R%#DN7(*E+ztWevJ%L&Iq zz(A-pE%7t=ccmr+j_zW}`yLovT%L>Rl1j6~)Yuy(usAV}R0epQ@z(W)p6i!EWPr(^ znIpWaz45}Pv#Q+=|D4Q_8S*_?jI5kR8X5jocT7D!v~!VzZ@tVSsg^;Pg&a^Y$GC)d zJ;rd{|2tOF|64ox#osd1xxLZ~Q{{V;cmXNm?TAMqSBthHWEvAYCAIu`T9waw3bv=) z6OURrOqCam&l`rIC7+^w1}s`e73SsJKkwpL3 zXU@`+FmyYkfaQ(r_8}!RJnAa6GViDn@1s3xxJuQK)e}6*gwZg*0w0%R2%-Tf2ICj{ zru>mEB<}ZC`Y|0~Vfatj@5rL{MY+oT_jiOo#tvxa@YBJ`2?09{ZG#odq}(tDRDxww zpvRM=w=LSzwLl8v;LF!*)u9_V3w|;wt18-Wepq;E)-k=?sA$!?-)6}%(r+o$Kzsd zq5C_f`x_&z48dx1`D&F9+Q%f9`}qj>XhYuyCPX8@mh?IY90L9TFXa+w^F{O; zTlGzC#Z@&Y1w6}jz~LyKmpwjk;6O>vSlB({;R{MEa3bDdE?G^JXm2je2-Sw%PCZKJ zJg)&duaS8TsGB(0ACD^(2o#rG{(~i_0zzhm0{tTdEKZO}cfZ>;favk?^C?*)rO+n~ z0`t*-U5K19%Ft_X<+Dnf2W_u@-+I)wH9<+6bUPYNQ4ZNSS_X5?M1E-3*1t5%>nLeO zzJ^-({I!Pv=huMLuNHW}6hkv44KHI<+mc^wSW)h{6cy70C>;iuoo3Ct5*tQ%q$P<&p~q5kO|NUb)q4OHogj8&h=eSevaIQ=EWcNYsjI6} zkl6f3AdBSQO8JH@GOkn*<$mV7#v~yP{G=cMbM(t3=fSV~y%_xDTR%B|LSsq131)k;KC|7w%-lrgK3QNfBiw-%3VY_`r5Lf-+*GF$AUZ$HPebty~%*t&_ zv|sAzvdqm}+W@l=(YV71Boy75_ZJ28|$-ybiQ3-TNX2rmJrU9-}36QTQDk?2hK3~wZ zj!RwQvkM0_R7)*|N6ve9`$j&omDd`t6eegHn%*i9%fs^0Gl7iWs=;yJhe3XOPFvEa zwLv?Xr#ZEpMqvwI_=U^dujw>+E2xn{-Q5w-cHyX#)`s4%!1;~L+&7|;UGkJH(iH40J zo##dUOpoKI+c}dl#McN_{Xl?i{&ex4Q~6s$r~^ug+5hkYK}J9;T`2{LUhxQGAbP^N zzz@C9Qy z%&=NtkNm>k-xH6@aKS3it#U#dL#)=m+hG=`nm^cMXpLuTwfL)bLwnAA^7VPt`v|^eVjP@0pune=WD`Ob2X;S6#iE ziie^dcRbKnRKi($$f%^@bvtyy<=$!_^5Q=SGn>fajWk@UOt14REI>+e1xHtXSZo`; zvfG!Y&=zRVE1A`Qu0Ius4iK;*^^lcwa91wg*hupVr=;;9RbTStR#2BsspGi;M0m0=1zf2DOCR$-|=%?$69$&h?A}zHc7yn> zu~3&qa%yGg{Lr`thz`A&!A>PlJ^Y$IqyZIAmjW<;{rqKZ$;p-M8&AZvD$@#;c6+3F z7W`bIh<+$D=3@V9`o&h+D!3! z)F8g*jedX|sjYU^1ZNqF?JCa!!_fG|IwLqM7u!wcrmgJLS`=;L`jKE^u{O=?coTVe zyYKWTGabd8ZMq5;>=d%I%KuBncln^bdv~bpTFM&($LGj{a%{op{sWoL1AQ)=eg0_c zi^I^fGz=xWT<8YIQDr5UUSZDRW|r9OhLJ(|`|qYNgt8>U>1xZl`1aU&e`4Lp3r5BT z{xDM=qpS#)MHKv$Q_ljBmfUZ1h04hjlM&5sIrPN6TR;i>G$q$fj`}?Thju=YYAe ze(HwnIYj|eu)W>4x>03&A_-PCW6HGhS=?J8D|g#XyLSB;+LkZoIjp{G#TtUgz!zk_ zKLkFffCNJTXnrnLY3}F)3pDk+kR!3szc9X#&kM`aq)+Rda=i~WF@YO+D;yiSQ6GTY z^LwC)i{7WV@fQQxL#zw9p4I+#D38ux+u0>j#{7;@CU9_JcgyKd5HVC#(`0o8SNmsf zjF`6aP4MGjRUpB0@oDeO=v8RjzEMqbj(wpK94L&}ffS-wfen{{N-ii?u(RF12}1wP zck&oDb$mRREmlgM*U2wY2@xYOet_`?m9zStLt$C4Ikiq>FHpAc7(W)P&T)0&S;-8v z>Y*p*7J+J5=1z;&t(w(UX>5-gaeIqJ@@9A{A3YP43!QEGC=TEvEm+QDR&zZcJvjUM?nX{qZRnr)Ll2_V2yf_nCi$>wfsP~3W18G@z&x@m@>&)MA zwO|3-y|~dzOG~u3QVy9m{c6g(GJb}fVi#MZ-ON+t4e|jBTI9c^ElId|^R{=RgH4O)%rlc)L>AP-x1q|CnM(-JuP+sDAwoRBZF44f>*NFD(R69mU_Yj1U9z0#GYn+(&inWyZX{I9S;8p%g{6`ygo zxJNm~z=E%-YJ6l21>$|`#GnvsycDHnm1@YPfI_tac~ig!HZ<5Sc}_HhzaX~H+&$jE zr~5nzGC3Qp^+cpYNrdoKKOkNe5UTu_7*&!{{DMQAiK;kf!cG#(;jbG(ukXB``;E{BFAEa7S7h zYOH10nKk;m*OkIeumy({Q-|_`cmY*dck{syGj%z?1|>N-D(5i>O_Vn{Od?T1L@cQV z=9aEKJ!q49O?6eZ((S0McZeNG2ho@OTPSfDP_$E+>*fFnI7Er^OGKf3>5QPKtVfT?XIRz{)e6jvUx1aZA3hIskf@}wT!()B+u5?KvNUlMM816gW$vR zxE>Ti-xElGb8Y1+AV{>smBKri!f@a(TNrCz{y2My5PjLX(Ox+rZwhX2XJ|%@7Xr_C zjky~vU+%_ypHb;AelGMG&idcx`Kg!PP7y#8*H%-ADqq{KsxZ+DT!F$q70!cU5v(2H zhQym~EJ{ngv!$g-bCux5%Ko;9wg`AWqU%=DteX|^j`2NqHcmcsxpq!k-9tkM(D*e# zDx&1|!WXHCCIRwx=nH{9rltDlnhg5SyVP@yMg*x!=yVV7@l0{m6;1CI_zi)!~e!=ZKw|UBOj+t5?w$ePMdHB4T94T9v$2DtGJUABe{>EJ!s}O$*bV6j@PscLQ>2QLEs=+?8Ihb1%7k?Skzi1k#(hSSZ6FtjOJFu4-qi-synu+S+dJps?!#83?ojWYGiB zYJ(;ODx3$MqizW>IWyQYQx$a1IQ{f@SN(vAtz!t&2=+jGG*hbG!jL&DwoB_Mw5YMw z?C6_(L{fvnIVc6G!uzgRb!pz0eY3*rU$FJcJ6+H!8Z3a;3LHd zB#+7tl7Ybny1|AL2h^MYB~(rEviWXvoc+3nq{Y2QES@|Z8FHsDi<{&7+ST2ES)+za z+j{mWwMS6wwly{YCOEOBrmpChlvd;3$585lvTKKm3FFfPgstVdlI!KaC|YYg|E5{;nIiJ0NtB!;Q**h>rvfUz`RjG zXckP-5i~Yc?h3{CEI5krvq4p3F1%5gTLzp$m@BdA4L=Rl|84!UwC~s^gl*S^6Y{NN z#(jc`7D8JE^j@g!%l54bL7W-Jb7uvlc%z00IcjqF^~JFt%l`=eoGIkcHcUNlFSQlp z-!4NpIML%h(U-vr|FxJ?c}o){l5W7T^U49+INj$2Zv^(#(oS4O1zNjLyQi{aYYd)6 zPNWeUwgds^(T|q<6E;+2o+#AF>4Fu}1U|3T&yOc7yW0w66(k_ZgGJTC!4Ol#g4>%S zy7DdD{R?EJJ-P|YXD_Nxho`5Hznlvu>5Bo|1TG2=x!b!I{+e+*^^os)j25YFe-k|*1LBi;9u$5w-w(1KDL7!f=YCY`g_h<*4yFogb9_GccBO)*e6mj4pOVHz)|jo|dr26FLtGPpBQucqt&ADJ0Pd9=haUkM z@YtS1&f}XQZ5gdw#$apgsa40nV4IGp457%G3a{1SxZNIq_Wkg$6h_D0-6-N1uuC-^ zw@d4;hOkf~)|(~*fF*f;GT}CmhevNLF5m0(0vzXAL31N*Fptz~b97BE(3x8@H1nZ2 zEx(`>YN}i|Elq$r^iy2LwE^z1jkREZ~o-EQ@8b$+O5Qb7mIjf^S#JG1BNg5 zeKJ>!U?=s~jXpo0?Cw|fOUx-)!xwU-WjOpX~?!o7GG?)LWS^KW9`Ut3xaHm*#duOK?I;cDDdsr~j$0ii#f{SU4FF z$ZG;B%+PO*{~ghPSFVkQL;mx!En^$v_gs8q`<>0nyXMNe!|jUs0;g|AwXxSww$4K- z=7ERHK>oJ8xp%*cohoL1qE+=6o=x$=w zN!4EXSYVP}4>J5(svSyjTPO!a$LMZMT2&B)=;1k5wf@8+w2D=m^yAulH6dY-O+vKcU&h zq4hu_s`XfzY_N)Fvo&Dkmmw<>TCflEo<@#7kv-`t6?uCn=KJq0#qQlfi&nPhM5%UO z0gv@_M;){>sb>xrz|gg(lgLsI;k@=XfpaFKkE)C{oGcNV2cm6w2RyA`de`j}x?Jiu zcAX9Js3f$fvFE@qfe=@i8jyno^dtZLf1UqV?dpwMg@6~A@-F2;V1?xd&H*2UuB6Mk zf->1sVdlYwGDqfRB__^e{Lx`%X1~mgjZVbijx4NX)viZNX=V4A@L0NwXZx=;9>W<` zJ~|I|F|k44(T)r!Wgo6Gl?efp{~a>>!2|T1&p40EI@*bIz7b-jU5ac1353v)4Wl1^ zF>1f>v8|{KC_mdJo&>5(MpcXUd{0@lj5-qTShb9fVe{?`V>-!iszHQt5*T?YY?*&V&kK)sgy$KR(da)^_6v3br|zYOIOL>N-?I0#%zf4dj|(xp;szGWO0)QGAUVT$<(H0hUV zAF@y1mpLiLN_N(gKu@DkXSKBtuYrn3OVhjmqfec-`)C4#^gwdLKX=ox{V~$%|CaE< zbn5jz~ znslo$Otg5V$0GUzrnW#%TrK1$F2T}=pfcBe@8;WB!#Ve<=u7IH5bX9{KyM!o=~I0L zD|K29fW!@c9C-={^iRsiH2BePn8aSaOHO|O^IqY{MmBNL#))%Amv!g=I~)eg$o<2-Vm&C_w%9>NCu zyK?RNdY`MJM=y{83(R=}83boxHdbW&BpF(!XrCay9L->9+hI8@!C}1~FVx$nktF7d zCA1RV#K?Vt2s2~JC2Y6__@nW&D>#fT3KlC+Gxe@3sgRHRT}q4W9Xx!JMZ3FhKu|+l zlN`9GHE(Gs>OZiwxfBGhKPk$8iSPL?sflAEYge>{?Wlc*#2e=V(R@vCrhK+;To)_z zIlf|P&a#}e1E`T1iE3XC`xIwHCjvR*G9Q^8uc;<`9R2C#n@A0f_p##Y{qwP|);Y2N$v%ySK8ddR zj$<6&DD!M8(@yZs)I!Er3lXFM;UCU0P9dPWCkF;vY0l)9fQW5wfi~mB%v`48iOHRS zlZ@$-ONgbTQ_5G3rwXM%JtlYQ@p9=!@17LUeYuky2;s=APy?ts!NvdDg`{B5R5Xk* z^(hP|Z`a!8F-%>WpN+_!W8xFmB^H+08kZoc`wy^P5bSFPfa?h~-77KXC8^K;)~fRm z>|H}n&xyWadb+mDJ`(2MRuVHbHo~R9RjMS=1+;PwjGV^gA8{w?SZnAck6ma-UkTiX zntW-|{0JxV()}@!qlqZC$_)v?f>sJg0+{pM^9j5C? zyE-tk%8dys+~2ASU2U3js*rb2WJ)!sdV34#l`t6Yb%0DR1|fyF`d4ARA6eo5*d9#5 ztzJj`wuS6Io=yd%x zkD`UWX}`~ zuHTpagk$F*T2!t-{=_J=hiQBrHCmDKKV$8!H+aPv_XxZ48FT>DS%bBJoAy8Z+-P%t zFb~wcYn8#gO2EEG59}AyV3udz8GxDth-8nW*Pu@4av^cs$v3k}dwh<(L9SoWXR0kF$CHxOzxM@+ZIpC zwsxh#rPAvQ{xQ;HK5>d?$lI4!0p|_j9L*=Z`A+eX9hVEekbsA@*Y*M$CZj`wS&Y^G z>Hekt1GJLNqT8Xj=8oEA6vV9rRpMVp_SM_j4_Asn*|1sB7C+?Eku|I(jRv-i)$YnpYccqpDliViu- zoiX~HgSvw}%BCL+O&ApD&BnaW6Pu>tGf8t4dHg#~nABh!MWiL1(q1ht9qcY0^s}5v zqwqQSBgEBTWz(xMdN`3b=wc3W!~sXs!k26jn@hwM+0#q-VI?v7AoU`25pkjm-?{7{ zHo`A`0=ro*9;0warf10qYs8%#;u`jR7qt>2hB}TQ>|@V)mri(>h}W0!{KVp%p!Zz} zf+z9M|NGxVT4FUzTL3Q0zGH~C*1gVcdn(4T(dBACj_}L3lmxUNZ5BqNLJO#dgoKD2 zwZE|{ln;{F0#sG`>7qX@nS3+#qpqhHC_Kx>|LY@+^c4TNp{r|J4a`3CDfn3|H9^m4 zOSqa3o1E1BVwVl}*JFP{E+1KEIC{d(t(lRvkQa80p|~_eTox00P*=r>4K;k>w8`aP z4E!XO(7SX+G1893XJhZeuFhF5@22vZoVOw_S=Rkd#o!kl7x=~cuhX|-Q?s&oTY?<` z>YUOO4zI4RWG`ok2WrF_*$W(Dk9c%g$enr6_LHze+$CPDAs7d`zLDNVJwIW~6z?cYxSm5m!~1Y>;>Og)(v_jC#C9qnZ=ciPtO9#wg8YRhenp&j z%U-;!8u{n*@7;vHgIMxMZ(h|n^UpOYhLS%E$OfFqLExPyMKTfe=mY=lzjKq2HBN_# zZA_?J4~}u3l8xlzl^Lv={sR7(T$dq+Qe#5=G{#P}YjVA0#!Tan-@`EL@c#ZofYsO| zOOQrg;t~^R_PYf!k<&I`QzXDKUnA#FUUIdx%L%(INKB;uY2^pB2KffV@HK{ZxV8 zrK1IumSfQ6sVedp>SCIR^$sA!5ms2FefGM<5aY%0W!dvi{q`-|IGj8sVw3;s%=Cg% zmSDVe-u`bbKK#r43?Y3d>mqX|a`pG)?snyUnEUX3h}KRswf=l|M)~CQs`}Ey+kE) zKYTt5%ImT>Khdb!!}%)0zx+S2QtfTn-^F%Geq%)XSOm*gT@$$T7k+A1B;hI!->)#* zb@oX?pgV_HEh{meN3uU3`nlC?7Gjw+Rw81ngO+@kGKceeHv$L?%fi~GU`^hmycIv{ zXV;{E~gNF&mmp9C3*$%a_^M0OhD*W%^&e zHYm=5Fs9@FpNCXz$BzM%id($5%>*l%n`5Dg7@f9xH@Q80!1D1?suI2zd-f-v-`;X# zV}k|%m00ifCCigMuVx5?x&Kfan9xnp0d~ukJ?KW9A_(U>m%t|5E?qAF+n?+oCs;zo zlFU~Pl6?zci-thG;-z9|VP6yUfPra?gl`JR^N*8mMtzNOWO=RZ7?&d%CE(IX4<&HQ zJ3xPzmA(HGdIjBYvvWV#O0Q*ctiR846a!&tsF8|OWKW*?{Satxe){;sUF0vsxg}!# zu<$E3DA3-$=cr97a5yXKy)YAzE%q+@h_JGB*@wk%RU4j#unrwo!gI&yWQ^|m>~1Mf zS9BbA_ASxF02OpVXxL2j3hLph{D`@wpl2=PQ}R29sZf7Db65#xyeXW4GVq+IygvKy zru)3xo>65ZGt!yOqfp5)Z)$g(73uNtrn%%t-Ot0G2Xe!+vA^tIK@u;wP1^5U0je5? zCDs086-Lc_?u*+v#M-&?n!-isL-`*^Z9=yHwESX*$!wX!L_MxvHce%9=K!e~mHZ&}34q#C zO2)ADNX|}7$#lYZT!zT=>5uU*gG%$v#%eh~uGXTCu}haf3;*@Oq;%i{P(&;-G{;`l zpt9|1Yh>?q>1;Dc4{&2&iK#h+DOW-g<{YsGP{dh?j}Z<~Cc^O&!7)yhIcRkowYim? zN;ot{Bnj+uy{I6bV-S0&^Ua(q=cS_{ zMn)WhfH-gNZ`T8&7hxcJaZ*>ZhlCBMNO*Qid@`_n^&!*qzNn=X3D8NME!KD^3l&1I z`-RBciCe@oUBrbezJ7^FCL6RlHu>*>>=Cc-63NhjByQ3e3h;(U zjqr_1Vh?d4r>xAwdVL5>>?B^1$>4LCrGs71{xn|wWFZO>@x*;vVh%|1vNMY7z1RgiRXSs}If_moCKG7h9@lJD+#2bt5&WTyg?7FI=Ao>}@;N zPkINeoY^v41=ynI=yQmxp{D0o?L2o47|%$9a+gk|k8&<1a9KMN5#8qC8y(ok$bzJ$ z^EsIl1>$}joJiatdwcq9Iq9;;^lC;8aWT%}QHBoO5L&wOVv+u&MFRB%5J;PT%4_o` zRhnnq7}fMaLI)GIjdTg-X5Nq^{~6_1r}Ioym3+v4sai6-SMQr_9R2-DeT;lQGn+?K zEnb`n)y2k*G{RQ74wA}{n@QwLqHNg!oeMJrNKW1P2g(JZ7;#3*7 zBW)cm7uE7>PGI)q2mZG&I{F}n#ubtgOD)YpKxb_=qbZq+<{b{%lSBl@mavDwE5Ofi zw_%k=9j98d=RQk@lEA^183G`o+E?UkKPb1U8v!=oU;s1fjaN3q-yt`470xKfw)*d> z{Aacm`|@)~__Z8waV4(8I+GlQpGx{vHzlV4jW>rUZykpC*+=kDJ7Ggx3Cq z3U_ut7+78XdU6qF;5>>8y({6&`$2kSL|=)}j-Zz=K{&Lp=?_8+S5_ph6j0IMw8s_D)^*YoQ@ zhNTbOOPTlOAX7RcGy>-NGRf~I=aNGg|B2p%d)wtd^HwykAFVWh)hLIrrrHS`AdT+9 zitW)7(o8O&?_3MV9ko?0>bcbO)f@2h@|l!RzU6TWu!2{t`0)(Z2s^r9Z7vtIl{k%o zs;=2h;023GF>NAZaTMaG$6W&YIF)p}1)YDx+p4-b#pDoinfT~Ux}LsLjaPrq~` zrC0nf2q1Tv4gC8#WytG4tM9awqrnyv(|{V0#Z>5XW^!|lo%;K$l>8zl@T>{LyX^+~ z0)6M(a?84tZ$5r0lLu520Z1kKkhP@Vz!nfvxL9A;O|+^Bd!LfJ(Oo4^&P+pT>10Cw zmCX!DC86(tENQ&wC{Pr|BQ3kLQ)Z#P>l(F~Jw8xIi>-hF?l|+ch%3C`v>lw?s%_dD zOp*Tgbv`FW_fLp4)1E{51^(pAUq{|=qj%f)?BzKvC!$upYu#9n3Q*YgwcD?t5iU*_L4kXi!6Hq8@^;FFbN5n`y! zg&;$k7R?h|>ALjJq@&(UR8xY{xn1B2gDd|XrQ6v~=5U~Wl06o~Z*o;+;JnuvipK8E z4;{V*dKLT^#)sC-6NdPCjHCfqcz>gEYAXtS0^VUV$*c@oU8A2XpEVR~Z14#W zHRt*hA?6Wf={ofiFezGk;Cd>3NxmZYLU&}#UyEE{J4ADX&Gwb`E0OI{)q6$J7bG9H z-@C_@8-L_)pYQOEQGdR%zgVkGI$`Cd&o_kh{J1;KQj$rmcT+S{X8%+uomE3EP%n zX{_2tenqbNmrpG&(n#_Dl4tx{z)2wPEcj$v&AfvU7){~HU!UXtq^r*A+vi_XdK1hZ z0+kQ$w2CGP?Yu$!*$=ti3v{GckytY7ze;=sCC>}4OVIf`EF>x6_08zyuyqrsFdS>c zas2jg#a4Rr650VMx3p~OpvtlX7TYmQy_z7BCS$c5$QLA`21T1GN>N(lxCsZJ;h}ls z@+l6FN6>Iz|AOKFt4^XdJfR~%YX=ORtYJO+sY&p zR8a8ky@m-rJ)fwin=*SbNFd zRipEFK4=~8sblSA@HNocUj?G?*!2lv)mE*)!E&u4tVk+U+T%YvvUaJ@I!o!h#}!^@ z4_ux0cZ88!b_c04)a+D2l;F9 z5if6(KB8f_Ou&FZPh8uh=D{!!>v!_I*Y!ocg95yZeU@`us~7`b&$)9*e^=H}QZ6i8 z^R3uURrX?gm0j#E__HQ&S>;WI%jALu_{AG&4>F_3b*`thwAFjp|C@RyuI0$(AF}sLq{7I(9BooxfLKG4%#^CPAsx1Ri~0e>ov+b_gQmLuvS1lXh z0z;2oP?e%BieGG=@`}a@RDTNr*GhUz>F}X(Etm^@VW*ijRFD1<{p!?qO@u(_?|sS` z5Bm!3Z2shps2);uPCXCH+U{k*Snfpi`c7TML06ZNU;8~#L%G^}RNcsRqj?7wGV)GN zZMzA|lQOZ%{bGhN|H%VshxA^nS_NU%@ja_K91Bdfl?nN)#@TGH!BoBVlp38KnX1Os z{R#GwolcQQLM!BYb9>`NoC7Tzl|W|-DfL_lT0AOX;aqyLi?3GubsB41JwDX+PVaAb zqdD(^b@_xHLsQw74tS=z^0Rnb^ArV5H@4;k56R+JFC#(0kh{@bB23+G4)qIKT0Uy3 zyIC2tu5+Z({h~Pqno68cVl7qG-udJAk7wR?ia~ESk*>0%1v{I;njy~Zv$?W|98@Y8D%hG@uZdNg4f+iNp8PT2+ z2^&=A#>*oauQ7ecCfHDb)Gq23m!h`9|3u3xaGLCAnKaJb-1En)E)B$XBfpdb2DXeO=Ss3KwGrFMOefX*e1h=_2UFjpL5m+V=T#N>RBaY0?Fv|QqLz9$H6+9MAPvn78Tj+vwk z+jMYNsBfuV#-^EyL!WIS=7Pa6a;82HCPx?VTOcUYP1}+BX~Tm<2UjoWNNBv~H8+`Y zg_yUY+8~!6KXvd65J?F5%cyKdD%?VIRwz%-px=C(|g zR2c>EuxFU0&EVCfX1=UE?gdExoHob!x%vwdFZkLmEhcWY#k?}eM?kG_4?To9maf=$4CKPXy=w$)fko+ccHwnm#_sRe+f9uTW>5aCrcd;=6bDpN6v);2zVf`p0~jqfY1pDS5+7-jIMVw4J-VeU8C+Oh@S)2l0z2!W(w9e<9o&TYQ#_G)-!My5&~g_Fd`X{5M&zU0U)uU z8bS?og7FqIkuyzm`$A?|N9q}l}mkDu-X5Uc!k@%)a;3r@P zK$A30wXxOn3HPd9i@CgpDCfA}G=j`q^d-954cMf;$QZt9N=qsbAKkOCuE8QlxnJlS zcmmaGQ;xM<%4>TD8~yk~jcf(``!@P%g}@Ot0w`sj6HueU!;%&K6h~kNW^>V|VTx@= zto=oUu}5ZteM%roI6bu`CT0s@R1P_m8sK;EngeiID$D!wn{H`OxuWi-ujeN73Co$kqLvRVRc_IMUZeqTUW7t`Q=rtG zAzUs}u2iv9uGE?##r${DB0>Mr?7emU)BjKx%AoJ5LF=-9my$~&EuKrpK0(AdJOBqT zUOt5Hc;yGQ?4AY?`&{7{iE#jQ*lZBAfN9(0M(r^&O}Af>JQyVI6WegzEN64xo(tad zm!|lPN50*c~-WjU=q0GN{n0U zi)GgkmX^eknjSyF2I5j;lHrdabR`({^3 zt?75wAc3BuyD97We$RqMMR*X}jIW{x0>LLLe=-ZBtut*y;#R{{4ww$8C;@%lG**j87MvCy4%aM{ zEH)(I`;`Qt~pwaP`9i=$Gh_~6SCHDu?9#oSb@P#$H7+7spY zNa}3!PkKBf`p;f<`F0g*-vXJLx!g6oy{T@lWwAPx_e1`-#YFf-YQc|h4>tCC3hl|A zqg6v=G54~h;8JKo(D%=ar+)NHDBC3h_VN=8%K}X}u*4&-j$)rZ0ayY4_mgdo{~?<2 zmo}D3=D3!)Df{@nz3yfXv7ZI8dJuORwz|D^l_JEU2H)vM5O=iH2z{o+VS{|R5)zP} zq5ehIlD|KeaJ@&EYM8vgqMYZvSGPmJ}~3 zfn1KJWb1~5kMpZoZs65>Z?QukSoK5AKP*e!HB#ccks6}hpPZoS(cc_zq{Q|ga6&7R zl0FK_31;5CsS!^8%7DhR_eDreR(ipW z5F7HBR%Wlj1rQCSY6veZFjS`5BAJU9re9RB6>x9qtFK9CHCOKq)8`^fba)e=_xUoD zO|t@sQw7wLi>U3!25}8>soD*FzB|c~J;Ocz3knCx#EHOjjAt^}eyj1O#hA$RRhT0~B zgF*YT{Dd)D{4Vk9WaaZaK=@$Tm32v5>>aVJSrAfRyHeV@Rk}COKXRyvY8KrI_Y>Uu z36f9tWd-rydv%RJd0oO3u3+rTNh7XKzx6#h)g=?9(5aCqO3rBFSERo5E^B7k*5|Q{ z-qSAf$H+#T@b}7K*kKY!mQxecY0Awk&-N;2(sFgE`wB;3$JNMd*(3B6#;?@r4)}#2 zMexA)9*)(kRQ?V!t(|FU)ZY3LjQ7Q(Qfqg!+k!>h3v-{61nH&z9I9kyL0)mRgeJCk zDuF8$s&AN0&|S}zy$v=hG4v`7U=41DB%8lW4FrGhx%ZWIpX&KqbJS}@rFtvLqOxkY zFe}KA4r=5}`-ZLa-N=cMnhXWV_xQvWN_8EiUOtI!!F6DA;9G#NP9>$vxe3}?Y(&(N zeE=W+W|YB9d|2y?Yx7-6gH?-v1afZABT8@=eRj}BM8~Bhi|^l+OB$K|WK-NpYNdf4 zkQEvH%ZkD;LgA8-x`>65DlxNz7dGuYe|x*%ivNAu4ZjoamtZ<}d48DkZfoq4FMZl1 zaIGs3e68?>EOnSBh1zuY-snbinIWWHV&cl8UcL09&>_!iajzUwkktNG_Ti82J-1)f zFRs1#m;h&>UA2mrZ)cAZ%yxgm?@aGqh$>56^4vrdzZ|)lKc`I6c9*@aQiCKdukvfN zOyx0CGz$fel}^;#QLc9%8-v3AQ>=G8tL}1?%c1T*xgJcWRR5(Qu^@}xzHbvcD8y!R zKbgdBJ?_UK;cy0JaxCY2PEE=RGN;LKPf(s2{9U@ZpMIhRRsf)xlTOyq!s~tOq?5^^UFW-4r$(6otJ13(DZ=JV zGr+7s>^CCt(=^h!ojgo7DM)2_=<0hMaUwhDQfz$^i0W05G?2PEb$*6yh6-j!erD%v z{VlQSZkI04QeYB#*(l8puN^NH$k#rD&PFk*csMr-&|P^<{`IZ-?AN_=G)sI`>ph$k zrhyicq^ZGwD|PL36TbbPW9OAS$0wM{yt|<>1CP^U?e*LOrBI%z!`yqZ2Ck2zGs|}0 z88;|@H-4q~j%UdQ-@u>mj=C9sWFef5WUge85VGXCxl%p@O_Y2x;jN8)-WCT zAEWsZ*#R5f2wWpa9P?!l;z$d1A&aoBHOU2lp0j`Olqyxo3Y0=ea(D8rCzn&@AN)zl zG*t_p9bzLuZ;=`LK(%|sXP9|ib26$~*SiwwPP z`Mj27=?dh(kz&61mgMEUT33COS4rdHs=*k##$43kUA$(Dgh5m0wP%z<_c0nAuORxb z2l@1uUF?($RY~URT|Wo24cDBoOFxg!*4t)w@tU<4{rMnOvuv<2ST#7lEY%7xEGb5% z8tvF1vE4%Z@-W`|I><#3IC*Ut$Bxs#c`t$DZNKSdA#vXjb-|q$&mKzx8n5*vu3o}{ z3d|Ez!ac-gD%MoS=k$`;)403XiWO1|Oz>Pj?)|h@F1~ z^4F)zWK^@fJb+Y!vhM6Q_wLDhQ*Cd!ZHuEV*|0f74^Ohkp?epp5?5Xl=ngqwvW4ag zq+pMlSgP&Nm{X~!R@Sos(rFKQ{j^3f+bf1*#k9KV)`fbL#g%V^mtRAhvaAFwH}&{! zP>%|ez_-XjX&}&>r=ROSD}?A&=}T+5Bz@JoH&^S@v&@FpPbzHZG;r}s3GWn4#jNgV zyH;fsn2%_V0H`cl^94m=@HHc)rv=woQ$ZE=&lR65A6%u^)+P_A6z*h%W5mVw&ypBq zPqFY7SK=WaK@=lmcQ&>BQdH4H9S5FEr%$lwx28`3nAVxtYD$a$Hh6~0b5q( zqi2Vegkn>Ih-pC2a~tAT1Y!qAn07@RhazzQ978r1b-tTJ6j?eO?$O-*sD@w8j|*Cg zyE@Cc>crv~h?m@*uj@^SHI)}zm1o7KKG!Q@@BAS?#sPl6BA~k zlv+7kfLKzC=B;Ehi3)I^9lSvwGRQlu+2jWXMwgd7{HoP3pCRpO=6}$k-?&FtM*6&> zgXHs*=rx-pCM+$BZ0q}yN#m$7lazH>^!@t{SbGrY zX!~rP8$Osr*vx4Ip!Ekw;`Nf9|Dbi(DZ}l9J{H8TtQzO*xcSP}E3GG#eSY!i3;K>GIhWu_6P=332W_6;GkddN3=KCQl8=2?JR=7tclv1&8);6?Q}lRKrQFze^YPjSd!-%1o+Z>X?x!+2TOO=Q9glI+179;Dij zlDmfArtoLHx2+Y?;sWX#ki6Y1_g_Fcqy3+^=;16x#@pW9R;yr^n3p=T*j5=hjK<6% z-mQGpOc5F(Q+!|3E~QnnXuRZyQc;;cV4^pk*f$3RDa8lY+I(z&NsYPWY@MV=NMS-0 z>$AK=j3rY9vyX(|7!=O}(2BEe`n)Aw)N2RL!iffHU}`bx3$gy}O`2>2EYsGpPM7%I zNAwnwf1Vlfmrg(jyEE&*{MiJWLnp`DPqrSXh`iHK=W2^VhXAf^R~0y-<#JrfLMa=Y znvKoEb}}gM!?3CJb@#Pz};W{*3(I?R1_YUwC(|bd-8c zc@2X&Mgp@7m6~RGxJ{ky>=j2FvJTxV-Ykgl)@n)u1sg>|(Q0N1*VzSPn=?Fz!oIj_ zIHB%_L2(!3+JfiLKGN9Jq_uZ++_L59Z2tB0rGVo>|!zrXsmR zr(W+6lmovP5>xN6tD_ulpIzmfcB)GztrVnc}tp+ye|h+ZmuN-&2C%#&}p<&zE@%L3(j(@D#7aKYGQu@bd@LH3J}Z z7!V8EXNcO&***GVughLY)nyicqqya3aYVVTZY&VPkm=JE96Xi18$$j#@^%;Gk*`eH za$)m_YxJ|+oP|k$V7EvLG(E;e;pAM2Qd$n*NmDgpfK@gFN}#8 zzdKLl#F#3O_`+PjI%&NJHs&d=wa?#(skyN=fs9!_NZ3HP?qj48y+_;?YpNa5?-GjS zGI?mji@VO;eD;mL`nW)7#-#2x%J+8F4~nD3BS<1WY%GT8ey?Y=Du=BW$Wj6Tg%1Oe zde1Rq+U`=1P-U=4;^*&TXlTdW(&UX?PLs9CSwqt9IH|)ZdT+2HO>QwJUAq+xBbDg# z=p8rm$P*1E>S>VkuNFphWeYiK&G#V|A*I%7DfXs=qP&H4mOQq8y){Y3J|Dvpl-_*0VSgXcM1{SfWA7&9 z3L5iG4x~88)<#eMeRUj~JHwK>M;oWGxw5#dhxiqJhi(02L+mlE++wt9$9-={;{^%n z4U9cq4QJb_>mN-I&~uG3HO&akNH^fZtDf!t2Q!KP+r?44HMOp>wT1P6X6TYFF=|LL zCQi@fq}gZ|DIid4ZMdzX%Cg&^82ePXc%c+z8y7*gNwRbViigD73*qf3 zbr=e=?&9k$hxbb1RS(zQ`Oa=s*jlPMjUO_ho*Sok6w}*TYu-uk_(6-_2=WhT?NpyPSZzEb?yU<14L;V&g*JI>) zW+9IPxhK2C4Y~Vjt%Vt;0^|#0R(5PRFpn5s$#FA%qneKM^0&PiI%)Q3G9D=Z!=f@O z*3MtMuH>Z$bU4ul7S8M~J3+oF*k_u*Uptum+v1yDBW3-9qyU_BgIhc8!$R2_-A;K$ z_74eLSY3Fh2ZVX=I0x{jxSO`&a%rff=N>^qaB|ickk*v*$2ja`^4M}6ds$nB@KIN$ zPmT+V)Pb$nlt{oY@>G zKqyU5zcf8s9vDklYbiWTzWlWiO?ka(--sS@(p;S5ztOKI)LT?kymWYut>S2})c_7> z5^y+oUn`~m+4TQ?`D2DYTMD;o@Y^}OJG`RT3bdRe^(tO=H@&FYOKq02yVf00V?tCc z7}Io_ryPCmU-=L8i0ZaH_=};C0SHuJINQn4g_b%feTkNNTo^d~IBwBr|3BF)S!_F& z5bCqw!vbQx)r|xA$AJs7mzW~!7#dlRzhOu8XFzZrW>Gr5%kKUNc^1NdVS&cUUj2v4 z%1FNr%arnV^71?XZ>77^e|a)h7DK!|l0CW=CWlJ3eb9U6iCWHz-QMsctk7N!AI{Vw z$H*QAo^_A`x>ZsqKZ;|So);u}NMlu90|KhGXO|0^Bo9~rYsEfz%|$DHI=gf}Oym>q zn=4hR1w7|1;Y|{|*})%*flj+aTFMIO_qqX!@dv}3b)eg?=nN~2d3CWlOSm2~6Wr^7 zhYMPn%`0{*9ycr~8L_`leofEg8bXgQiygfBaNtg}#3F0h0Bg*2$R2=ts%1&mOxZgR zbepI`$#vo?%UK+0rX%dr`ad@n1Re1>Tx6S;oSj@x?rKvoe%{&nhHnhY_>FcdPDD4h z@@A-+a*j*+^yP8$B)vh_2`7!9cOw2XTQHan8NZhiD8hsOPt){Bb?Gpc-w)R(bBxn~ zJZ!>bKaMbk1fBe-&Zm_0F^Rm-ca!aTz}Cdv;z0#J{$nLi@*sdIoOY#};P)RVJM6}| z2F=~e2RM+hzw?{pIm9s2t36zxb+tAeKqs$*WX`MOu7OZ{hNxX-)aqB6le50r;SC@w zqy^BNRCtdrd~zDmeHF{haO$hayeYLBNi?65m|IKxY~Ag_gId-m zFRzj3ZTp!X#Vn~%ubtawFZxU`zZ}qmA3#KdBGQ>`sZi9IAc%DgaKd|5*4b6rjZ#UfR!=EoMT!)bhwlgs$^aC5|Oi8nwyyL?vr5aL?<#E zvn{;}Ys31i^&YXIi(12g)e7V6KixK6X;dx(REi~skkMHZ%AYE$wA%XT@*;hBxP zw&46=lDP8cW98vs*yi~Yp23DO;d-Y4cT7co1nq(sHfX^+pl@$M_tNA-Gma)QzGb97 zml7{c)%~-E4isNhYM-^VYm@dA#HzUJQ0~0~B{yeqoj@gM4)D)*^gPIa-}(n`m*MRQ zFx*7_V3F*qpYbE3BQuz};?$5+*5pF@oOkcjic76O>y|II_GS~2fM!_|m#X4WvkbN$=un6_jWV+Q2t=z&Sjg)7ku{d3o zZYXO-^0Ojyy^W_sIu9)MGfqBtpZ3Fv$IvZy{Wb~1E*H=ZFwBY}WxEO9|GA}72Nm#* zBA+o4R-_Rw*@W|bm}g~ zWhkeAP~EP-4ZZE&ScUD7s2qyrjDe!s|Ir+0~- z0yFg~ntUV+S@{K}>T_R9-RrMW;aYEE4EvP~keM7}U2p-$h1z*~^B+4M|HnfT_0!Mx z=~~?Eyr3KwLVshL`(Z}{dciY(kQuU#vhiQK9<$KnXlS^s`F_0W0WP)clr_r9f{w~o z*WJc8@4xT$r>X(I{-%?sYTmjQlA4+|+Ri$rf1)uQl7sfy_f&k!jJ~!Os!~A4#AE#3 zo9>jOe(z*4+Km4A^3S70vRb6;<3gskSgyzyxVmLDs8^|N)8IS*NT7|7?AtFj?nE} z|LOV|l$%ukAD+HCD5}5hdqH9;>23sM$)!PhC8R__=~_CaTR>nzQgCT0g9QQUk`@q< zTq)@mknUXKIsEQ<-x-;4`~y#X&sCqeB;{oAL6Z$Z=(jh(^3j`PO1BWq`rKqE2SRlV z&%kT*XN-ZuRkWj#8Y$}s6F{7@JzISiX`h>!W$|S6_^#;j18f?m8|?Sg6wbpUNaY=j zPQ7%3?Owsde)Ii30>+JxaSHA2&gO#8@@|J&=lyO02yLTR=d~4(`+w`}ACHLbNE85s zP^79EZD9MNf6(eP%yG zAI=FNO~cUjZ9})BS^jMybkIA^1N(YiQz1Y|^Lmmbun>Pvud`i)&iC^m{Z@iV^ioEd z>y#d*L}o$C*}#H@9O5o7VDs9wy!_3&?6y0?<817Ag%&|RoF0+H*a{v5j5YX4qJ-{u zc3v-SDACzWaX+nC(&Qs%d?=@h;sk}hezo(Z#I;r3nb@>9BgT&EV6abn8e3PR$Hyr5 zcK7zuHLV&L40^ftV1X>-&vq5uH-8>nbTLbw{%Shg!R+JT)Qvi|oF=$6E@QfE~Q6?ghTQzv+`5go!S~~4FqX$+ACN<-)3` zLqUm9StY+3-J@LVfHe|)Vvk(X>Z{17dlB{qIwVn$bqzB&=emUU=bbX)5|^JmP#G^m z{g-DqPSTn)dCP_i92H$aM^6uMu;&e>{&Noe#QI z0{sBWfFUBk*0m)yRY}2PIse5^Z||TklHBRD>xs6FLE44KJT2$KYWL;Of4AeaHZFq% zWhYGLZ!TzC1JNbBZR;=3-1eJ+c5?y-%#k>`92;ldSR93z9rI-7AN8OnfA*4dfc0RI zmh#Fi60b9TfmFsTDKMWHm`Ob%QrRkW)G9)E&R;Il1|^0LeZ{EmxfBiue2>433B>k@!_>!{R$Qy z4bzw5hPzEO9pI5EgA3{$X^U@W@9B-VsBDIuN~Th1KzljTjzzdqXDUiZXzG-{)5xw~ zr(%IV%9?p=)@69QuH%{5c!%%5NMYH;Mb|#+kZuAV>5Spx#(1+IZAAF>oe7&Zg)W7=N20d;=;XoY4K2N(s4&$gIser*1f_`Eii$E z!`Q@V>PMZ=TJfC`Ee9g@@+dic(i06La}{X^`Rexf-2;^=q% ztiZ_CZza+VbvNyHx9;_{qTKn@;a8SYFp1Za_dQ#)^ChQ_mg)|Q5<|E5kI-_Lm*|Fz z_$45x2d~BGn0eZNGR(I@mZ|_>oB4a-DfBL87aG;g@39yuc;yGz}33JvQh`s)~_Y08gd6-GEeV zkrJ>QkrT+%{pe#no$~q*>K{BHV`<-x6miI5-ksD@>?#i5}^S1CD6Jh3YE)G#0MFW z?rTXD`w>8*x}Y(edn+_0^%nlaP;~I`A7y^v zd!80hD~CEHlXo3VDIJZuwAzAW^1znsjK|cc7?{VhuMh}tF%W6Dz zON{aeb<6;C!Zyh}!+5Hx@A)U{b}lBg_5!z>ms+<8TP%@1&3>Mf7R*)ISRf*@aM#f{ zT7;Qh(#*f}Gw&Nbw&~Elc;s90jWnWDsX4n!Db3!B$i(?^G#T)&-mqJK&e^2K(;gMu zkp+fflpj+VqByD`F@Q2yx+MSNiE18eCsE;ZqKLLni}C zR7two|4PNOM^rnX#cE>JS;<`X=c)*vWa;F}EsJ>Wy$c4JYKBUGOxdgr-!`Z&Tokv*r#WieCdAFU$xmnrZj)y1>7E2cvYqUBv_yA6L4We)NB5LcIeK_P27vp zEz35_Xy~p6bQhc63d7qu^u%3hf`@=2k zf8_gHVai;7%7~zPYo@_t2WK771hS9tM;Np;m#65)d!(%{UY_l0qN0qL{xY^8GVJF# zCqG~RqXk1IA_Qrz66fe_MGXJ&)p+(F$)b5(rNEr!f5eOx?l6Fx|+_O zwSE~b2w7QAiB3J9vl5aUMhM|7@NdE;mR*lHQDj>ZG<;4)-_3UxGE^mNebI2Ue{u3u z2wm4$Nsl0F8zFrB>%#rWxDG(||3`dVq3IshDEHxZIrXPng`}I*VqnnRFe-K&^5(!= zeO#&}So62=P4Nh_z^EO1#_`{#B*3)Ko_^g#_F5O7h7C$j9(rFpDjMPw;^e#1Khw@e zY^*fD7VyKck2_`eB36&tG;}C?^jqe%W z9iy~QKzRp9Vc-mT%o6CIrLSlTT!b?>1AL2@6RtR#Z@2IMP$k|z>#j>_G8TPI;-;{rH6cYBxu7cc>C|HF(7bR=$*qs(rAJaKT8t4#p+3)c9(f5p^)RZPpY!%E%P-t^Nvj zL}k759U+uzOkX2keQfn!>P%e;q-fBX#E*KR!PVlu<51Exwm$V=$=?;FPCQdpLbTd| zUpRY7o#y;VEwE6baaMkOu!`ThC~4hKM+v8#OWpFyU$9YA&VjATN*%(EFpwq(roBwu zI^Vtt0CJ;m2k`}$RMuWV^hLy_#68z1<)C`5Aut2ghC#S+tP$iLWb8Pp*t+WMsVPXJ z_DHg9(X=Cedq<9=tINZ1AR^bessq)@xjVZ^?pf8Mjo--&d#)$-yg_hh_GswmH2Ads z@ackBsDjdYEIuJu16ARr^Ug;!ot{bX0UZeSH}JRi;gzDRT1ugWu$GglJc`D2-!#AF z8z9UzWHh%Ho+<@liABRr-t9~_EmVEqpQPdx%KD|j9l$-UhAE|-}@4#DMqFz|w!N8wl-p7<*6)>wu z{)&7<#?ko`K`~mvKjX9AxW&Xn?MpjrmMglF;WPRKVX9M|DrJ^XZQZ$l40Tac&$gBB zWrH!nf%!eGm%#5%V>SBft};0w=vbWn=+j&~8;gObxtN`H(7=*1@M?+CXT_0+kE=0| zj(@z`Uie01>QRLl=jTke>fAl#&^Pg`kI4i{KD_<&bVxe2t9y18tMj8|{0H?=TRKvT zG!N-d+mXXhX(v46P*$!6U3tE2t=`|B^KG!OijmfL!hv-$Lk|}Uc8qz74EvImrY-2j zbZY0Kf&=#KQDesCeuBMdl1_MTVYZtqduh5NSOE5oq@L3Qg?JGKr{aCtDc1K+ueuZ$ zZM45V*E&-XU+j2J)kG(woJp5T{Ah3TsLZEqu}pcx7%8!mpPsM&0p*C~BGA#xbX&T4 z41Bb%ngAZln8$Wsr{3V{iN4G*)ON;g4Igb08ayQj%3dguo32f$+%PP#mCzV>3)+wK zqb)4|-prP&tC+(gLFzN){aJ~kD~(J4TAC?8ig_f z9Fl4?M=Bvwfbdmvy+qXM4~gzL09NBb1LJ0JW^BGt<6*O_R{Wd%<+EF!db=vuWY3)Z znQuKVC4o<@MxOjU->3)Rr(1Bz4H?4uHn}?}nwchkPc|0tapQymH*= zenw~Daeo;YdJafF77TKGd%8vflARfUT<9mI=b%BY^E z>sKPJiJcDAgIqtS-6NNd?6=%<$|!Kk>x%bd`|t!9k8eDew$6rbhcjyz0uRP`5xU^_ zC=G3?yrm6MY?!~h$Csp{ai`l~iMM|#^KNP!KjfX&p|9VwUf%MwbiiVP@{IYe-;#14 z%3LJ>h2h_Ph?ShYP3|k6K`~P)*K_1rX!!`Fi1R)yCM-sUc)@)QaQ|2y8|oOtBlT=< z^ZeX)Y2U-5Fl= zL3s{!^6&B0uyhGquf#|Q#Doe_?IUe`4zj-pvh^7K!(aM%72n}c2S%*BU#(_oS%D?%O=C_S9- zJy9;vjmy@U=wLFC^DeJLa&7o~UwsTa9we&f>>E|ICD$R~?$)GLlctrWX6u@QqCdx= z-e-)}2=qXt@Bv$?#y0O>gVbl?{X>=J_6>}xaAcQOg*x$(9i*s=r|@wx#V?977e$Nd z-8^9yb!o&`>Mp0|JUkqD@5kLJK4ORDOOtpSv0Wj|S^BjCZ(!SHxPt~)tL#?g9Ko7ojIyL@r zH14fY8p^R7sZ~+x`lk*H9ERwLr)`MZD7<FOAE2hH>u3$fOnUFEXb`abr>UoD%C}1GuiwbFa2nG|R|l&HGlNyHDKecCu-R3f`v*R^3HTp4B9z>V#| zy?LX9)FpPSL0GT~CkDoHsHqm##}o>u7Z@VbNjY(+h%nVbXxY*u5|VI?xwb0uUn*bKB>?^2IRVZw@P*%z=cIl*Hj6lvk~C#b zhK#>(qe@d-lM67A9kv9c-~D4jQGHuRh{=x*upN(leM z7p%Qi-}=05*IEO8EQa~bKWt^=x_qZNF^KB4ght~n>$~ju(f6v%CI<0MnoG51ixAR1 zJ;7_SxvI5dS9R|bDVMV}j7=_V>P#%HO~+FjY9OGD&CgXW-J7 z48a|Rd3`3H(D(5Buc{rBhxNKe-c55vp*!AaI!>E zeOaW$29w7Mvml1w>#v744moe8N@^LfmeH(d^SxvaRG$-JDN9ZNlFvw72?us>PaZO* zlIW9``Tr={ZE_24X{c&^BqMqcL$r;U zL@(#S^RcHR@JBZsD4JT+9)TCv*L%|&V>_A>4>5Oh0L9Z3Xh}Kbbi;~I zdGI!V=y^}ba!-hj6x|i(c3;y1BYtbM)dPqK>wz?xcEy*vcIm#d1GqrJnU6g_?V!6>*v6EjQ2SqCJW+ZujbELr` zkBDD<%wu%ZK27WR|`NmgJhVzIwWoQ z>-T>5%n}8E3R9f=hzIZoQ*}jHd*<^)uT}n>T|M=z3fL%_oKsvlIOR*(yhK?5ri#}w zd6#MyH>vMjLzkN74_p^61+3}vfhwd91tgjNoGr;LxX|jwPGzT!wM?AeQOJ(CUXXR# zb)R17DNsqye?%j-v_8KAdT|K#fw%ZSZ}*e5>L~8siK1; zui1AVw4?&$2P!qkl)neQ>WmAV8W1iJkit}%XG-8uPZ<5yDLB&n_?;sFr#6HVqdN4-op=HYXwos5nc6^ zSBCDQN8={w&Fe{Tw}cu!SL%#QPc1@M(|bfEfR=Gb2ME7u&QWu42GPp7a{9PzT=@R` z6`kO#V=sjWbf}nWmK4xcQ)U4uwoXv$TvsM$Q-;-LTZN3{yowr+n7sSf&l;D&@|+eE z-V=lrKT2ywED;&=O;kBxgjj}Kcou6J!st~4Vn23W@R@_n3GM9fR0XKlX1WX9*Z+Ld zx$}{gQ@!JYZiXfCeF|^l(ML-0N@6dT)no_3PboJ13Dy@Htg7t|i`n)?R1+hb$`A<3 z3gD1Lb+$=08H@aSik@F&d>tE?He5)RSjmjA56ippBx-&kwn0tFc-ImW0ZGQS;i(i| z!P2Q)TNhIH7lu6?l)FdmA+JKku0_BNC1w66*XX(}N z`)`-nv^r+Kh>WbJBspf_?tI+{M;zJ23xEZQl^|S(x%UIZ1w5)k=T{0~U9B4b5b8Yf>5jU30;H0a64dc zm*U8v9C8>kWZ^n!n7>@e3E6DVyteR}$RaAx6U**rMWfO*OV#XlH4Ja~71Fw60704X z8qq(keI89OU^~M4#kp<;`=<0w~|Q($x)U_k!d``|GA7wT76gHD-%LKBUeN4E%j zmuo{a2=olJZ@Mp*=#@bL>CS6&4`I?E`1C_x*;Yd&*WL)3nX^q@tUymkP3%NQYyt5e z1|ELhJzyMZBDj*R&fa4IESfDEM(ML7QsTWb9)l^tTd#B{3j zbt_>JCuR!@=X-EE!tU`&Msa96CBU3KLTCzh>#tG69J1c62ckBX-$@RU6LqEw^+cQ6 zbT)f^p4dD2!rA9GI{0KA!qxDLGS~elV$ooLy|qhy$dE#GhNW58E z4tFkRaC|zC`)sev>7|e$S1kUx_LFt3U?BUU$JK||ctop0^s$TCL zV2e9AmiI^>RhS1>J(>RZv0_Js5h43D1LY}U)RG* zY1a}61lL-(4-u!OLCD^yA*;xC%p-AEFn#ciyW>Rw2SV6`&qck_mvA3=zE-fOMO@*~ z?}0zwu*1lPFtwvYyz3l^l)c>yP-t(mY!|pX9wY>Xpl zq0Vr|Vl5_V`4OQyJ&V8&Kh>J)rs+v~h7r`a#kMuXk88(vYTq`?X8F~J`&B4fgQx>{ zfOde?OgqSUAjs9kYydzk8tM>RZe?E1j&mn^c^suYinv9-R^huZa7Xnn!o(f%+g=DX za5>)7Rp&Yq!v)oRxnYab(h~4oEtwr?(;tqO#S^qOwfj7KCE2$zaUpxgCI&;SH3>YllON6~5WxFe;R4vfUF*N+o+ zM(gtTa;1AfcJh^ay|t}OT~8mayj+hv84n?f4?fHW8VN(t|l*XVuZK3u}@i0y=(9I*Tc}pJZTRgV}z9 zO#hq*jOJcnabIe(vXUd9*anv)`p*^J-#_HRwdFna>JhWuMu3^SgJ=dNi-FlYb9yx-Je3Xs5 z3)tWY;PqFxz;#WJ6_!#GtxgRrpf@i7;O2%{K2G!fJ{9(_5z`6} z03TYFLt&y|;tO9>x z;bK(lE=tM~Ljygb>L6~fs17!G2V+rv0M}7C8t}UI5Tt> zp7AdhMd#W-h4|OZan%VNnu_?k?$H7tK@{9rf6!~qGzf4q8wzq9qDx*@r8#}|hB%`7 z_R9IlR0E}vMgbu8b@Q(2N*VYTlSA3CGA<@tx6v~3Rf;cP<3sxFQ%C*Sb8UJW9 z+j&S5{N(iIVvh!Q(|sOR|2^EH{573d1`a0ZHi)yV-@QoHh4uMG&zFMBh3V=&n{W%C!w|f$97kthymm2v)Cw zBJ-r`?*4jly?4=Lp$|!~4>7$%rtn%iKDtWBeE=j=-F-(IkgmwZ<*uOypwVW{owukg ztJng878!?FoaRktF078erHI5JWuICE76}ZV>~*ErGac0F4OJjGWMUxyE$ll$b{_HFP)%&lc{kdT}UR z3N>w}+?Z|yrm6)A|HTK1TDAe*w1hnfH3$S|e|n~`Q0r@>m)qgyEzv^h|L{6!stbOlS zwyle739)XU(r=y7>3q6(G?n zcT7eWTjVH3rH;x->ixdf_dr)8ZQWVPo7aEUJ4YxZ?-X+VbA%V|g-4lLU@-F%%#NU= zf44TC?N@uj_5K=h-)$DROkX-KnCV0PHI37rGJ2e0G zF46^%RAq34$BqHZQuaybV%MLkwPj1-#$)Mw$w1(x+>rs=onR`^3Hc2_KmqCoWSG2= z@kqOxFC!x0yTuD>uC#Ku?Niql&Fhs3)wF;pE!J*KXEc755=;l~eh2435NACRWJ9bJ zeJg$wTgB7&xzvBZ>f8CJbP}&zGoNmVC;izdfA2EqD^e2^;dp0!_McRY5*`p5r&*v2 zR3vCWlzlpi;$aba1+M{|e;Et z_OtNrZ{Qz3=2IDUgEzW*F;4~CUKJlUjE>Ob8u8` z_L{GTPEHD5h=3n=Pi{^#fIdDiU)h4h-W9+yceanKrkz@JbLS#g`bpWMMIsMF?RLIv z$A>SO3Ru$7uZj+QPeo@o`Gp3QB@@35$R-*>y&DR!+|+6iqcmVsaNW2-^*joS|O6T zZ{P^PP${tUh3$OhpV4nQYeh*~8epub-|YDgNzy&j>rFoW73LeUilT!GHGcxC#(^VA zbWghM^2r0eh1)Nh3mDgn-j(!Ox9xe8+ta;$fB!u-S;X^4yPWTeiWsW@^t)oi_tyy^x3l`j{aVr9vj*qQRp|_U-}tMUXTO`=`l@B2Dt- zksm8Uj-_v~k(FZVm*C)n@oP-&&{Oo(s#&c)b`;rTe}ndxt<=8!SNzmfdDS|#7*<1b zi(S%cu_bkpwQbzL7e{U97UeA-GopDHV{~eTz-7<Kj09vKkS8; z84JALmh+nDGc}y!N7p?Kwp!h9m=C0i{Nhx)5-_hxh7ho#h>uY}ChYvF7!Bd(t|Z{l zMOOBpA816&zI&rvf)~8di;_f-0n6eg`rjO^U$(#1pDBFQ;x09?;-NNSK zM4^1J!?ib6k5R^cd_u$A2`&}oL=CYPaTZ5-Pph~VaQZ=51wXxs(Fl54Z(XbiB!{*L zgt9--t|iBFsv_Pjx4LHmnb?_hfRYOSJa_xc3G=LpR_*zte|*wQT!PHJ0D;a-=&sxw z`c$VxELIs)tgCD5F>P@DfSRSo{Zt$iX2OOdygzSL#~yKl5sO0Liy4g;$GTP(tj5rdg`;ELa(ZR>7^(8m!`eCk+~3S9Q;%NF9Lo=k zgmbWk|IAObPvK8;Z&o5j2tU*P3;$6~#eT(s!)~ox#GKoL_Up$h%`kX{@Jpaze;Z%= zbE*h>V^ebWqI!n714>sA5r(w7df9Q;J7yK$UFFL;E75HvNmG`qG%7zOARAA2)#wk& zTOj5&0es8Kh9*EORQ+$l?Q|(Du+?jA>Q8TCP7vJnw>r>%h^E;g<6`Oys-mA)$wfRX zw9)HPq|7OvuVh1var(BGNwKZYN^svc84$cjV6F3OS1-`v9YuD`tQGE-Q_cV8SB;Dw-iRKU%zwrs8mlTZnSlYig-qnId57*SO7-QK?RQmM?C?M6f0!nw8T16 zEr6m*%tk$RCx*oR18tf**)c`Ihh>U!sG|b7!*S&WH|N{M^5?Qjq|KR2UK) zCD?wy9WR`U9h+V08L&#&N_SP)4psoWPtOXK_;tay4(P!r*<|Iwc+WGs)oCJSV;&we z+?|@iJ7Q4(U^)X|M4h!El{GhaXqOrEeOJ*Uhvw*wySw*JjS|UBjYgV*pJsK4Pj{(S z16Sg-%4n%!uA7u;tm&Z=4N3L)yvR8*Ss9nIs5$*H)_izUw{$=K3P5HN%@u9uZYFWW zd7w;o8Y8~)(zVl@xnFR4EI(j?lQx-kf%zU_VJh<4*uj4iMzF3j3WY4eD%90dH6ZV8 zGgHFjIAhWubyM~`r@>SBtL;pVa^vS|xx*J)PG;~$1UKF_MX%?jr$WI;c2ujh)_JTms z6i)PGB37MrBbIXia<|L;3wE;JsSv;Yz*5WXbjPb9*R)RS;*T?$*6EPasg||QUN=g6 z2ccyE zu(eV5h%o7%71@JA2Mh`A44vW|EDo}B#evIRV>-2&E$n7w0ASCUy^dgue@mwn%?A5Y z7X`uwFJ6##{L?{={R4KJP?g~WmOOP8@S&0*hl zO+DK-_fI0o)#)zPc0oh%)@HGFIJAaHzw&v-?2%k5JAxW*#P6@tpS?Y~tM&Paed>C@ zyC)miCS=ljGD9kV{;xB!Whp3h@pkJBe}z5^$Xcg&5t_-RKUeS%qj@jeBf$POd_ypW z!M5I?-O@d{Q0ip`QJrAfBMC5jp#yCJVcELr_rmSKIF&Ys%{TCPLxt<(bk$I-QJrS4 zfUCh_iyz0M7KDbkhxbx-7p~B)%#?>c!2F>NF!2ca_j@VtQu5Y!yK@{UFjbG!I9-`c zXL2Fq3`*l1QP_l=YjBgNSfJqk93gRyr%N4zp)-}1%+e}ER$!{%>+pUENVq0W-=nX- z`RE#p9QG1DqyA*-RmXGBczJS@gp-xTR%5P0p;n!5x9W#6qo>xPqz)U)n3&}VL9iiKG3+5HWy8!|-kRn$svrehPed_t|0xfLy8S1`$1*Wx?x~um3tc;N5&g;KflHSSX}Vj zB%=5JNnN`9(Mh^8(F4zO@5=#6L(=PmF26rpwHCMIx38DV2ko3gLSB`q-N9Zt8?FS? zQe^Q0R^Q*G6q)!E1zL~jduccAzLd3`ocu_;nqV0o6N7@3A{{T*Y?|B_PT8>2+>E*O z=(00#C|@sW;&lr~sjwBAPKY1(!24rP6k0CRZ)_smWM7|m-QF;^oygrD%1DN4-L>ZT zb_zzFhoS?Kjde$QvXBm(~E ztr8>aOicmNcu~GzT|FNA@0q%KYy=4Q!j}**D;$R%#EvI1p&?Ick*}f%!a^892!~U9T$i1A?0xP*#mRM5T6P?V7H(8l&(A7LEW$Q5`{X#3#l2I8?#4PGU zE_=ddplQu^WZ~B<2d4tOF0UrrE_e5M)B8)D*MqjS#TgpuDgkQ={=2iQxBq&=$NIfO zwdeNiM+ov6K5>>N#PazU%Z4~q5ysAsQn^g33-VI4tBOJ5ag&2RJlB&TU!FXR5ay&? z=#ai1JKDrZX2$OF7(A%YOHyQRxJRY^7%SByn5Hh)XTLtVXMsP$Ua*1n0)H?lWCJOq za8jgGZ@j~LQQuDX%mNJL{PTlAXd z$-RV}YIEwx@ntmt&_g$9F-Va!Z2)u%I<=9x#jkjs0C(Jm)_N-{0_FW!ET8~Zie}I7 z`}TIdfV5twQ;ae&3?BCzc|A>vLchN5+9LMf)ch9F7OYN*2l`>Ma9EJ6Mw;N`51hG~ zEFjw)gaMrxQB;tDwQ$VLP{-d2hpxI%RMEfw$upHx)lo-|olb`wga6e*0NQm>*DOjiNepm<=n9A((92UCd zvo{#zuw|##`uApOjXhqgr$Qs@rKFwjQ;Yr>hZnrH$k&^Hdy8EOP!!!h-+_*|z?JWg z0Yn0eCgY_hJR8^p`~LiIiK=80UDGXK*NLgy#Xm+#?4+zHSrVY*-=}HE925rbR5dk?u{wUBm-sWLY3E zyjN&mwEiu*d)b2NuH-|x&n7IY{*_|a|H=#m8f%?1^WQ0gHEStavcj3X(=XLOKB&)9 zaW!5r%4`ejE^V0b=TLpS^K~M1>n!=|!tO2bVD2dWPrY9{x{C0R0?cuv5?NPjXY=8) zmOM8q#P9l9NV%A6DowNT@Il$ujO&D_lSC;+lhssfq|e?uVn~{lzkak^=T64~&s-BLCB=(yEHw4fH0_JufXPW>D;ZYI zPQk4I@Cy8~E32g__hIm{V8(AQeK%t6Uy$ingLTCX;>MCG6BE^QnzeF5=AQ7z1TCz9$_vXvEv9VxNH}VpQdY`9FpWk=bUUh$H2dd z;~UWcGNcjeMeis<8)qRc6ulm#hR$e=*oli~x{Cz0qWuL`FN z$*X6pfn6N;k@@wq(fE~slt@meMW0MbIq79Z!Rh$=US4giWNX_iwJ&Qln&h-^XIkQk zu)G->fj2D?aCbMoY&ABtYc12k33FPpfb3=*jt|0pGymf{Hbs44Zank)q^o7Mp;50U zVQWS#RJil=N1SGQ=S@Wty;+mZLZ#vdQO;Oc6{gj)7TVGzAoa8XszXLSIq(iJw?FYT1wI54Bznh?CST#H?L4jwC zehL*OQZK!)w&w2sM?l-umD#a;FZvn9*$U?qpT*NYs%K;wt9r4+Jm z&m}7yZ>+vhE>SUYy_DML9i&hOs7(Q%MTaG=q=2rBS0?VJd0i*yReM7>H=TiKpUu|> z2}SqP#j>$t`?pJhRH8-on{(enSb}@hLUx3PLM}Vx_KaWuYimBpOD8@3nR(@6;Q4IQs-4LwIH$)4wRpC^Y9D5(%Yb*nWto$!MT6L0`X*|S zSHt93wx?>Bh4=w3k0WVS{OTji?~rTRz?HnK#i(*%p#TDp}0=-KM+d4CA7IxgtjfA!-XKUJFr_6r(cWie}#Nf}Ps0Z)0@DfX@k z@XMO;$NCC)m5=uM%Euq0AvJb3SDXyNxhqXUrz)4;)yL{$?X}1Lkpo88r9sb+>3XK zSJOcYr!BM#Qx0Qot68mZZ(#0-y2U&Uy=-s`?w3-%8>^Y!&8GyJX6EHuEaB7l@vwBz z$NE6bmX1~LEZ;62m@_kfJ>C%#bo4orh)`t#3Zdm^;e7T(FdcY4OtoJv{QBcU0{lFD z_iAiiZ(=fvUl5its(SEQ6Lugxn4z;3ZL*lZ%8@>ky(t)_^Yi55-{vO+*+nnA-G;wI4Hpe<=he4Cr`rvox2z!>Z$tj} z+|IRKZwN5nId1yi4Pk-~OwV#7TS7iTwK~U@O7fzBex#l3qCiyds~og`@1*HIO{u1+ zxTHaYaTUpo$j(>VLZ!6(qo#M;z(HhPb#7^PL`P?X5~qK%!&^`~k`^VGk)M?v9O-Pr z{cMX(Qsdt$ZM3!oo$bt-2&Ug27+fXqVdnSBUX~=YqyX1oW9s~)!sptLeJql(7R`~7 zk{yn{*xH>7U_?2n(VX!NlzOgqpaW$DxhHXn74511`mWRVq)BETul|Ad*lsh;*vEXW zLn8?}Z&;V`_1+yl{7BUKE zHSu3^%pa<+T)msmwXv|*uIM))m5^V&7kbtXkmTB~ZWONDz9efxa4++Cfvqks2G_Mg z?6AqkXG_aHZTn4zx4NyJN?>6aZhx@*!4Iv5n*%-c6O{gCsf@HG6k6JZYN@?IPx38EK2_z3hVhhfM3jGxONiKFGI`PY|PMUuoJw(b;i_ud$YQlns>F4 zcTwPWbtx4adb%F6G)~I&O`lg%L?1{zGab~}QNzUM5TB)PE{kb?+S|u6jisGu_XrT(s?LQXh*$1*EetFqgEvLlHqX;$o$gQ(3=T; zw@UTxy(};MFDIN^-Rj+4Imcljt{L7B-LKGii|;3bQjv7@=_^x04g+XH)6 z%(Uk2w+Uw;CS9H7T}?_p_I z`H!}zqRzc?_)&zOr`=wrh(^8v30~pbt)3Y_;~$o^-@0?1JPIx5&Atwf&sNFgIxJy> zvGCm6&#sugMC+u*<8!x&Eyy@%dviwcjj_P(HS4nzUt9X>6@f6=+GO*6FLXUI7^8Nx z1=P@crmy!67S3JMC12eG$-3_^vR>^5x%mzNQKLSN?;AA!GcLO_l}dnoYn^r!{CioP z@SovuJk>4Y6p7-K@GYXRTB9&iC6LVUM@mvuvn!>$TAb^B@sd}Lm1Ez_>yGA&%rH(z zN2Ar07p8y~X+}g~v_dQX1HuXmN8;It&M0(lIgTJ12MdSHFn1$mX9V`Cs?c=GD0_OG zCpB(Gf&5Dx1Q&f`hR=a;Oay^ZQm(&!Q+~$tfvCRA>EPjzVLvTe+KqYugrJ|m79qv>XX66$RV%p$+B?&$I_Px9)^+XE6%f0(oV}CU z+@V&JL;7YHYTe){1yIX1Yp)jpLo6b_n~oB@xVRAbJfA*M*}B%1@{71G437YV>X${i zxy@itM{y#KPk(n9n%Tin7^Wn(XRYw8;j|TLaM-L_Ph&>*Jrh^0O zA~r#|7dhSnLc0*s4Z>2<|TmLIsasMCb+W=FE$) z8q|GV=r|D}?)=J7;QFD!HbYF)3#2p)tYGoXiX;IDK<~{enq&1AOiEe5rrP}%1!$!L#N4L`8i7u zNUP-#ebnA+5maN5;eD_G&oFzP%Kjg{h|^r^r(d>ol0edq6w{G_^Fi7>;VNI-lyse* z7KaUr;+MZ?GWf=(CAD3$MD0Rqtc{2;IBV&nw&G4t6=a!iiIbAxP8O0zId2z29Ql4o z592w`A49>a{q;a!mViQoAF-rs+#Foh{QdE3nWi6c4vsF&xz3}kk?(mHFWwek{%)B~ z=?FsCFL7Dwd@lT&xA{I}agTcYjpX)W@{vi}--rJRz1~!{bEs)lJxK(w_Xz!B4SjP5 zHQ~*$R0o30He_Rr({^%vC*s3(nN<9GFzo=f6KH^f#n9w#QSvwKqAv_~n9^5)T7Xf5 z?7$+`hXz4q5QIsXxVM3CxUrs3PlqE^l+ng7d51vHV|bX|={^XeI8jrIq<=FBz*g2T zop-BD^46p}72*ya^S_%6NkHnr-VO*0jnpl7H|G@H@&P5^4FtCz?*mdASZv~JopGVv zc!T=NhxD@%MSC2fX)-?LZ5ICl$A(^8kHZPH+LfX6DgJMqvqHhGX_~@*zH5h!vl$dQ zsVTsFWlJw7RdQS;b;aX|kvK`su2=A=YV@FV?4W9dpAv9YHEWOwb!t=Fy`OFgXqMlV z{s{|wUD-5q#DCtl^Cz?sGTdU>WD$d>2rg=tG~gbm9zMgIz8x%=cA2d;BT6|4P+KZ& zz+e08QEHcdDef?E9ZJ#Bec}=<8%q(i=d2xqhlpi_U%Y^~srLt_PIbbOLs$o_J)%qy z0;D;bL$B|JJ4Nnc?=P7a{3T|RW(d|;Tkxudv&XtJ$NwI{djJm$ zIouB(N*LyZlI=sh#zX^*#Yc=sd`HrkaV;d^=fn_>ub95P2B)bydxjG9g&{n^G4(c% zPTp>07WH4@Lfx!|v==`#B8v2R8wU#r3aJk(i}+zxdor=U^7_x->+4bAtI^BZVgVdd z-XECQCbR6*&1vLei^ZFaGA;PT9p2tqX zpDq#yjPhJmS{9`BC>(GD*g~eeZ< zCY;oNY`4mvCKQYO_Fi{7&A3yEFn$1!V7f9Dkh!Y0C-|xraVw*xlQq9CKLwUMH3`L3byrhKX-LxvNit3d)I){xOKGg~!q+AlNM8J@io5&LoR(x7=6 zuq$(1s&dxm?GM}UKS8fl>g*G)sR@Hcz#?0$8!{9j+M-Jj0k))&P!Y`%+6ZL8?P7$J zon1GNTdeYMS7{6(>sty)`Y&k3=ihs9kQTub0uozv}SSobk;f zvWxNjObtyCM8FQZDl&B^(zCD~svcR~y5F|Qtv?iI)qQS0-AS_jpHUZC-q-TBFL#x5 z(4@cZV`}3;Cdi3L8$slk>uLgjVxuus{B3sEV&mnl$|PCdyB`R{l6QO`!IA5AfdAJE{Y{*RKQi40bDNEGE4ua)qHS zn}C^-8>i$47i zQqeD<&aY`xl6gO)zMw2&+c4FZtB(&s-|Z?tKaaGsv9w~bA={^KKu+4vuj=v$P~EQF zcOCfkYg_L8fywv<-n;W_^?8qDpwm-e(X0^FjP0@#zgcPXGtLdoSPN zD=?h9!M?8w7k_bDiSvQ6$pQ0%+4Kf=Q9K^hkdWe z_(`;>?o{-~o9sff!mJK(1f)So!06RVtz(tBPj{NhBCW!wO1LCnhzW?%y>l6*es!r*tfMMgk*f<*2800RT)R9!mAF$(K?)@Dijn z2sj_o63<11a^9g5q7Xoo;?vwV11;fqp_pVilm&wi$gc?mE4%}TfS&_b0N(+md_nDA z_gF!~pQ(Dpp_-4Q<6wD*sYL)2lZyt_>c$vwRJ=(*u7{}y6A9k?(AF=iOCW?08#3Jt$=pAyfq!H=k5QU$O5mvOTjX&)GY zCqrDGCS}Ms_PD(z|C!*%1_S_0U{a?}9qZ_~GvL0{R9-(#r*rp$ldbA7uxBef8I z1%}-mC;aV`2;P|gw;p_>y+Kz5B#`2bS9f}sRxiZyK0lN@ey*HE%Xo2KB*jCh%m>)eLjS*Us~UA3j7DAdmQf77|dyF)zbguI*bj?q+B z#Z*+ta7(yv&nfCe<2d^)wjNEyX@v6ye*DIaaj7~$ff)i=T7iUx8kHqLjUX|oxKJag z5n3Br*DQiLSio8X5mp+zMhEU*!tnnl2?5TRd>V9VKk>xEPJ)&|F)rz^6citT2V%pd z)SFeMv|ChZw55O-jduT4fO5IV17&f@Hn)2QI0hVH6$8#9r=a*iKL1G$*7+p{UpnpQ zQvA->T-z3OqV9-DxuRjO>&ZSe~Kh2MB?gEvL%%Y!< z9~_7Sls<49c4KkuN3JgO36c`KAumDlM@>khM@YMukFbtib%R^GhcD`sUDf&u!K@Gq z^bn6X$V{JTEInZyce|M+8Gb|3=5%datRvi^TpL2xNpaC?_FX zQlPi<(X>tamV25KpY|!=&>8!9flLf-v}feIWH{a(yw8++F*I1DWcZrSuvm$6dKL0( z_)nc)v7CI{KSjo(H^~tbIZVaok03TIHacBq$10bq&@l8Ua$?L_P_0dkYSSu@HZv%aRwy!VlRG$xNMKjZ zh|EA6ONzPMA&2{l2sG8xsp>2G1WPbQk}_UChiU<=gpAZyz5+9{o>Rc@|}q-24~;Gz{7w zqi#0u^XAu|q{{@rYNRbo^J~Te`*_paKMNh6JF8Dxo=ESs>}2n(RZs&BhKpF2*XxH< z7Ct{@IGgs;Jmmro%Pme$G3r#Vt96(RfUfxN-aUNc?=15)+TYFGy8U`mPy=TD$*Ftp zvfqQ)+jUR`2`#a}YaQWU)vS|W`*tl4UycS@iUz~M{b8(TRU`!2w-PH66-C?s%T9k{9ajPf72R-Is}0q>Pf( zl}XB0A9J@BxK+U1L!(vIDhf9l^x&$jtg!%b0>Y7mNC%IkzkpJ#B!GLOx3qgSQ08(1)+#ggmc2(Ia&~P?hrDVQ7P{3 zL%x<|f@FQd5Fs25L4}Yqjb5H{?XyzKtuMG48x8AB98Ce5f0O8B-0^Q?P;WR<=gI6F z34A;}Dnh^%UOt|vyM=E)WH8x~@46q}v0m<9;g?qd+=UXWeh(%Q0b)-jM-Y^vbPz+h zaX1o{_#DBL>PCx8N35z+sR>yDZfDT$R%J2ex%8RF^a3J7imQs*j7B2kro?*99D8EM zA>q*PW_`611{IoGGchUWgREr+HHOp{aRJ_B{jBkY;ABq@b_9$tuP^?e>&?UubD%93 z%^EL^K~3TQv1tf%poC%2SrUgIN&^@zxRe0gb~zqaRxl0qYQDNV>s@5;{I|)D8H)KX ztbMoZ>GmwHeaJ|M`L##%5P1R*YZaV~hzU9f+yh}nGa9B=MQIqvTxj>}>^>=GGaX;# zRkn}@HzzT0)Z##ql_AS0w(XSZ9Hq28j0BrAETm^`i(C1co|oW^Vg&kJ3LXZmtTh>0 zh>3{!?{4J211BROD5kToPvp)@wdJmjUrSU2b<3JW`OkQdV8F9sPs|f3lvZm)k!oWX zLQ35~fuGAAk+KB`6JaKn(>@=nY6)(x*3Vd@gkV>(x~lYnx%)(6g5>Wh1|q#%tw-|Q z+XA{*Epk-Ul0luJg!jk?*1qUSHpnt8GQ|GOEh~4O=XX=GmB!+|DvKMR^_spI$Bnz_CkEVeX>L@ zJ~4AI%-2qkg90_-B<21XMx!?>zGCr2jAe%WeHvjaA5Pei3Mp<2V99XfdOgGQoH^lX zk?{W4=ZDy>aSzr}Tj@>ByZm}Q z^PIfhii6C8FB~CN{Ub5@qdOL#9fQ_YSjo=Jf{?|^;Pv}vP)bFse1t%nat)CSupVo_k0SAhWzVal0+0yxB;KPt&e511(rRW>KG82eO zA{Qy0MT6BkUNAFFnHhTOO?hKd;6C2PZ2=>1A>UR4CrGaqb&dE*BF=05{Uj?Agbi%! zvK1e9ZglY^?0F_oU623}0(707n~;eMIhkcrok81I7qUsoTj+7C)jIFi=6O|h_3T@<1^m?TfM-hgAB}}yQ`E_Q`~|RXN*YP#{zD^y4=Tyk+7Bn4{rqj@7<746dD2Q zZc+6Wc~L>~VKU9V)(=1K!%Y;N^6V!o>ecf4fGUKtk2O-MmhNCVTb-8CKCni8rbSgz zRd}c{3?~YhnLacKd;lcGAzW$<1LsLdsaDy=cB#cWVQ#HI8ylkpkBSP=2-XXST0;fW z*WKN33_lqBh@=vXQFs=``YJOVse|LjJk$$(0C)+(2$PQc%2RuXUTsw?q>5QN1b&ZG z{wwPDKJzP8Mjw(O3{!)(rpOClyT~k+k22)=?IwUfJ(8n9G74G~{UCdCL*aV68`u{A z;#F9(Po6jwm%j;bi7*>{U#exG}s@Rw0j-BM-P=HM;2yjI8JNNy`xST3Ur9JW{#* zMj_+oil|hc^=LD_6vF7Q2Sgp+?_ix`A00)k<4gSADlg(f)$PRnkcw_$dxmQiFYLU9@z+_h%9g+Jnv(obt}nS!@Q6 z{+4`R0*wEf@Gda^^4Y*=n@h^Y*hku$bM8<}iY>b`x@$7mqn!PP{-}zpx5FjWiXI)6 zgz-NPY90o7gvA~_4YzOLH^2S6mi_SDU%!;H$r^&XbVj@AolhzO@!Tp8(bTprh2m#jjk zoJ{_BA8>np_$a!+L}nIXSrR0`n8I`$%~os1hNwqi#RjIM-u%juw8{luxI12u{E3Y< zR7JHO=3nGVz};ZJD4qkC#Z@SW(t1=M(VjJGxj?$(RqM;2#nAc4BvFkWJjUZYfFBzH zaQ46}K$9a}=yR9FO=xrB%Dlb}lv-#u!Oe%yD{3ORlWYYSL=l%EHq+p};kEZwGtYel zK6nekh+3L>FtK-oJ@)2?E}T!kfSKt}aLPSXGk2K{($&_lLE33^a^TcrBcG3x5{dLQ zBc#N1*f^?VnObGrZj(kyu}N(ic_@e=$=rlPTc>lQ=loe6{PZ0MrW?C=gwAab^WiOZ zeg%}Q75e+#@;;VH8DkDbEuTEN-E)uGPk9v1*IPgRvefZssd8RBIrM(;$P8Zkh28+J z*3IUMZsEJ6#6gNJ8AtW5Lyy*%`>aywI*SnuR0DY~faL|+V1W@~(F+D+8L zd{XKlDnnbrpsG8!^)(sX@agUYC<=5>PRzJtXKb+T2&As3i}6T2|F-VHQKsKhYUO-A z7he!B#uxGP0aCMCxm_Dy+Xk_5u6e#z?nrUjxB6S*eEty~Pg$Jud2Ju_Wf2n}pWL*j z4+w^Kfqe#+3IBsT_l$m^z!-R?S&Vqx@1^43IHYSC`BhT-Wrpg1)r!1W`=@zy5WpYk zWGqS^K8X8=q6|4p=hC%y7?Z(1hfsPNpQnr#F!gNoQf%ixs_dE`YfjsHdiEy&mOL!a zu=F>@#%fyJ-#Pw9e)7R6VE4MIuXV(Ok6m za!)>xr^G3q$1(pj^zV!bQ(9rQWsVG0~rCfjo^rtD6^itN+NGK$}>6=`?(;&}ae>l&7ppxiL+3#*vZGd$dt1z17 zWoJ`-G79{;t`2p#n^~);!|<%UUkYU&Wjif(2ziBlpGXGMPI^esGM1nlRLXcK6i~s0 zkPF<+=I+B}|66}6Qx(&v-UGMYuUh_28?KcSdT9>a5)lPHvE}<@gp#|6hy@*8wNsXA z_>jY1`i_bjTujiKC~m0`P1|bk*xD_cBdiRyMLW>V(%Pc!h@Zf@w2yd;YY&?O-&wkdA=w*jjgn&%fsLJ`LYp#a7$uW;|G8_e~urf=q< z&nWo8u>|t*mBtVLqixXukwG~_IYX@^JSv=^SFsnwi%MLFdq62TnknGt6ZCjDg9|_v*Lj z$J>hP;PdudIW|C3>j#S^9+B?GoGndEWy)Zn>F;nd*Rv8Xc5)aLGturnB{c0d?ZG!Z zRJh;2vRx);ZJglp)97`&rJ_&|&A~neJPZwku|f=HKX7ROb+xW>mdWG}kb{unWFGD8 zDOc;mp=MS6V$W!EXJT%(Soo*qTZmuqXLofJc(_&4Ih!2Gy1lgRZMAe#B#0`s8z8JmwrG3D+a=A^^F5nd$%H$-kKjCcJyErG4n1_VLuo}$OoRIH z1ufp`O7W^tkc!_G)^2_5&t`|3P-&qKd}JO9l8?0-m~vjK%c)rLYtNN}jekUCgw}=& z$)~d@f^i@uiV{HxEGI=(p1~w|b5ochGcl(t(GY#;XN9lSA}`>Be)a|AaMiAc?rNtu zr%Q={GXrIxORZIRIFBBM$4EEHR9Cy5sFl|(ihc{7bvR%8_UF&Z0OQi{S!~6vGZ3s9 zi2s}57_^9%?Y2Ss<1FFa)0zVfjMG9+!+bWe{pPqK zi$LS1xvYZN?+ZuYdBgH!Phnx;At|pF5dVwVR##IjeK;T*%gw=b+ZPmu)=_mcAWhC2 z)Zt<96cgXN+-l4XDcGwhjmeYRk%_-89bCL{KCd)&$dd3?SlaMV>vKQTYUNF_PpdL6 z`uXd;IN{lXS*zJFbWlb%4}(g3&>R+ylXP`nv8NW1%eBJabsr;VPN-54o?>#futB z?}UD@jdXOs`dd00e5p>KY-x7-oKWuNzyM9w15YQml+m5EuJjgl<6XCWjF37cpRJuV zDlam3fKMrB>($W{8yRRwlI5oBY*H4q5)SQE9ho1Hyolj$bJg}6t#o`7Lq&I3;rrji znOa=A>TTU4$;nS=Isuxc_12u$#7wDwSW-7kd{h0zMF+6>)bFT5%{UdecPdyq zkqX%Fiu~}m&7PmX*1J@h9aR$&Ab&Y{6-5Q=j_>D7X^B5*rY+tus^5Y3{a{g^4`L$I z(*U$Brab%hi!R6uQR{awq(X2Kwu%7Vm5&j);lO{?%?^c<(+X``2#4c)U2Sc{-q}Q* zSB;)_>pNc zcu2IjAxEjBWq>imWVZTzJ3Vi-cZt~AYh7gHSWb3h;vtqcD_o6oMt{nPN=Cb!j{Q`@ zq5`!Sv1{bc_%r6uWs|kU!*e*M`Hg1H#hqX$(W-g0)1N=vIihfWa|wRl;9s?j+5ttA z5)*ZL<6#=fMlAprN>jMrrP8@BSpaBsDU5|?A#Ho{MXOM{ z>Fpf{5}ns%xo7x=3&?yVzq6NIE+#<;S2cB1XN_CIy> z2xVrMInkFpF_ARIaT$D!xyp({ktx&Ar#ZI$)L1m{_$v1%%0g9F{bQ3vmLvJR;sBqx zQz&J*d$Jd*liGL7&*ZelT1|l_Dhi=mJ3=Fq#WeOh`>hMsM({e%qI@_k+s>vBudOHH zmy@=uW+9H?>>$~&yGAoW!_7wa_?=@3IK%CIwx_wORC5}Z_lIuXu#+120Z7ZS-;arCNCL#` zUW~;XCDtCr&sHs3qG=wOBnA`u()`)5js$aktfqf}Tx@?xd_LrhXn7cLIWX((u9IDz zp$dX%)Y$7RiDc``1NG}I`yEVMo zZrXXfOA+^SzREm8`0yx8PK_E^3xq|?$WLfO0BVU9o*OxT+eR4j$=w0;_>q`evTu;& zQ{!(!3{~{;oaJih5zk=Qv}UiD#ymSgMj){S zg-C$!4zy1U(^zyg2Q=~LFfw+voce?#+qY+{8R+SyFV%NCj{oJl%D*ze`wVf)R2rwT z#-&&7HJUh&W{TzRL~#}?Lr~3CZ8n3+`aJzKp=)6Zn#soqSte*X30wD~^EpfAKhwob zmjx}Jl)88Z(GNb3+pDiw`xzcPTG_#!vNYhiEHt!q7R#jtF8n*m9RdN8E~T}F>Ps?o zzjHBYO4f{T{WQYVOI$at%&(W@b9p|%N3}nL@hA0#X_?BZ0zY-y#zbzn6?J=oIO#a5 z>|5hED4%+8$>PidW2KtYB@Zp?mIUzJJ6^2TyyuG2`$1f9tnsBCpE&8|R(DNBb!q%+ zGLTA*JAMr(v;JCFq4CdpwCahF@Q!FW8!SWLH&C|PSw}<>q+4&OUd{T7bgItxk!OaT zLs;9Utt#QSwy}v`ZmLRRnjnwUGA;|BdGp7vO$P(7vRxV4-ZsUpte8{(JHCE-b9&X) zyg6kievK~9zFH$~uQuQZR1(ANE}?31|K`0{yv<`zhEjQ@b`WsXHpj~FA>C_sln{<@ z*n>f3-a9?n>jxO7=q{prI_F$o$r!J!?mnuG&Wx}kl18G0A(<}C@r@Pp7E`5C+rLz( z$vI%oIh#$cAc?sCeuw=CnYFr;zXLSihDz)w;$u4thb66^KSP$jKmNr(D#gB&*coD6 zDh)4{{I%m_m8{PlDG76!E^(z>9I{vu_cniBgxvpj$1k}LX~aldRsB-WyoocLM*FR6 zXPTpgB}y1aX$=!%1ElS_ZDSo`g^M!MetMl3CK%-eEK|gF-cdY$+ewhAOehd2n4Miw z6%5uJX|as856?R|YK@VMPnImvu1pyA0k+Eizs;&Qe(hki<#1t@Gfk1>xm;=s?7-4C6m!K3|5af}peUg7_h1oL?uzrkAY@#chN*{liV@GVP`CIRQe_?tEp%VEG9OTP@cYmsv7iY3HH}N0$ZUXoga)_#v2Y06{uVzNcdRJE|(gzlPIo);{&%l*}r;8SG-mAwtMgpZ*WBrUxQ0&Pa zgQtJ$tM}$#JeTwiHd^6HUX)4u=lBUfE8Ha$dnmxFyt6MIq0n_X@YnofdzVG^mAqNy zfZ3Ovmv?C(s3hxClqZIA^$#^u?9-T+71UZ;YrZr{_=wiZ&nF>uR1H==OARdg0D2NA zcALLr9&D%S9=@VQ_=qgk;kJ{6f;9fiphtt_OB*KpR^&pw?QV|!Hcfgm?KeyMf#jvR z*=S7b(d@lyR{U?b+f@h{yL?9vz#;YTy*HDPZG1PpKZkE;YVUATYy~W%g--u=Q}`e3 z06wcM@UCDth8yzzT)k;KIrpCAD#3t-;Yf!6;_|GUvRoCO(;kUgR+Io z^TlAM_;PGpveS+}w{|YQetDwk>kZ7DS_Qpms!7eAdoZ{=lVSN5$sTK=2zF`VS#$71 zo_J?+ig{G(Ijg-*ea?|PQ;3*m{ZycdL%)U3h00%JrnYg?9tYW2D+tpS0U)Im)??(B zSE!Ti+3h+tUHg;pd))`@cyA~MqIMoya|GM)&_O)} zziXP{sb#p(C<|bA^lXY`%8>ayxn|Gk3?1Gd1t5*IlUH1W%W(gIdRrcvRrDZ9Ewe5? zQJ{NTTLW&)Zkrd|%SoaK8cPA?`E?@sQr8(Cmy;pto;k*8r!fqFExteT{wN}sIyP>u z$u*7(=SwqY2rJG3aiS5u?g17zgu7#6mv%Kl&gbz)%C@FF|~!D-hih>wdz04;49&*CwUvTU{2 zoGJg#)nF?RuRECo?tFOiR<1?I41`Kdgikj#pT7I)%&tuRYC56TGHq(vHq(z9(wp%2 zF@S&C#o6Bt$0dd!9Ck?CWJO;V)kD184!iCVLpWUij{c_Dl0KO21% z#pxQ|{cI%W*OY{@;h zdApQvC8m`u85P>WWDH1h-GO}}6}fcPZhg`qA+h0~jwX<`2y%K7%3o{`e$9;B>Gf5rU?#X_ zA-9T;(?zh$K-_!&H9=&obUt!e{ur(8Y&S0Z?Yn)Gnvk35;l*Rq;6&j+gS=<=sMjlw z{!Z%P$!X|iM3%&f3Njv>Oz9Rn{$s79yNs@?c*JN#V6P;sHH9 zR{NEe_bNJYwG55e0c*DIB9iWl%2_eenIYR}pj~94W2VNU>|w2-O|oKt{*0#e+X_)B zd#QNaBir!wqxpMXQ~^sVL&{`^+kqs5Y?)jJ``T$oHxv79ekqrz_sj#=W^Q2X5P&qe z_L_t85^d*y-yN8PGrLUSpt%-Cr9wmkP!S+>rRES5Nuw)y$)~ijvqt>+@{}<}nqqkx z^CVfi4)mzWg;gDk!u)j4+%JLI-GX81W&J6$3(1FZ5l?enX)k*%s z5^LYc5NB%r4PZL4*t$4&1WmzUYVhn4bJxc>9p>9Tfn7Pew_kg09VA0E8%R?Ee zZBZNP0?V6G-zO39D`b11$g;&|<8zMD{X+5~QO(z}nNM@2GI(I1t0WEb?i9tt;pG!$ zJ4TdR|HsOgS+$V`*5(Ch5m*Meoz|L|uvC$SHfpeJS%~fl>vaGnWSVqZ?2t%t%cI8o z2{XOH`|9nd)O)&E#Gnw`o!!Aew~|vYWl&!@H&X+?Wog6B_u=lh@_-=l0J2BviED#I z0s_&O^sC+1tM6NDh6#3mH@DO)&{{^~R5@j;gNKevM!(FM8eq;C#j)6a{%Y~JhYQFp zA$?{6=do6CWH_29rz6JUjlW_?zvmufBkkYBEv^ISqkoTBguw=l zK$Mxh!^S?$QjrmKa^Xq^!=cl(vkMlhs=BOU-_^(uHb9}|038ZJ`Z|D$BksHiB@!oM z4xel{Er*xZB2>io%mY(BaX@@Sqf#?Y8?Zfp+Q7g+R3^1P|e^s(wq; zieTlAZskJopF)@_H)LBvTAcO&SnF`Z6BGa0{riQsW^jkBjV~LOUHp${CacF(E|-sw z(+adz1aDdDiU?eTes6pY$zBmp5hrBgNBV%=S_=Uswxy1>1NTN;Z%t`AH6=_pSrwX! z{#)5p5}_;t)b8K{imZwwJ91ZBc#u?3Z!_XNyvY#jBJBCq18Ozos28{(Aq5TO~;{zXBu;r<&uv@K5obr>{iCE+XSDm){jBPcC(} znvQ%z|C#K6b~W^&%Xcb@iv>0IZ;IMv0ReawzALKkoxmL(VkPiIwO7s}xNee4n|Dt* zXDz-yhNL5`exOF%xs_EP_wj7UNjq!h@vjzxuqW$@YvLVRuNMHH|YGb6~QU z^6}%WU?3cRK@n*t6a#Ox+cWAgIxULp=zzDh2AT?qK~2%b&I8X5A>{n~>{6sfoaIYR zeyOLq2|SqH)IooD=S^*<$R<^xZuC0Z+NM53AK9|tN7qPgEp83xrq%B6&+3_y)n%By zm2^1)^>+NrImt8U0*v#(bc~E_sGIYnf6V~ue{+@4m6gpG^vn9AlMBEWu|#IQBDsfm zR#m&aKy+opky~-hZQnntA1ln!HzSlKC7{(@_CSP;bmu9+EnKnG;6fc5XBk5|d|7?@ zd|B@s!g-+Eb#GI2fy5jo>VrZ)G$Z}(CTuim&rVQ`K4h@5(;!(tZ+#|Cm#SV#NHYO> z%mv(B!Vrh3xWn9D$Qe%BazFtAaI4&Ss+_iU?{93QckCX;lHtCJl-P<4U=fTFE%(N! zh>Azs{pBGXVO?SjZQ?+IeB8K1L_UbU$$Dmdmco6#xbTeoZBd9C8Q|0e25~23*EMK_i`D?ARp)f+${;);A7{h5z ziJT^LK*fi9zd#fhg10)>Q%JQ|dF%M*>b z()&BC@v)K`Lwaf*RVG~%6ykOAhAA*4_go|LYHpoY7kc(D6uE)7DiZ)-d&&*%A@&UPZ zm7_G3=^1yj1&LqoFIgt>a%CDj9mKIv;rY{38H;i_7eXFQW`()>v)QC-3fMJ%TYV0wxo==PeU8%N}&^EP%} zWulYeqfpu(@rpNbbP^;k`c#3AuJ5$rNRDW?ypfO;4rfz_YU;4T}`P97eoO(Xm zF;s{U*1d$#yD1qMYmQypEI!1_&!2y2{}J}`_tcfFtZbTX(A7<~+~vaf)cFEoE0FDU zll=Fb$%A7*^bEo&`|AuQ?*fACv0sU2xWD#gc9-l-2zxd znT3c9tr7mE&p}E1o}uH~7dImT_$qciHiB+Y$tbrhfjA4h^rfB+^UEQn3@iL-sPbA; z<|*0|UgKhe(+7A?FOF~QDt>6E z6||uF(gmA^L#^hdo)bb!CxYgQvH}*FV}<3LZLN7pfI@!euQSdC<7nX$gS}n__ZB{- z;WTOH-NNtgEoSsg2y!l~YdO%eEd#Wq?sp!a%#Pb!Hn zdAH;59Ph4m(;thD7o@$-fk#h=m5zSXwQjbn#6Dp-H~Ljas%hFhDik{&d@=p;-+r;_ zJPg=$J_FTWO)ku@0-JmW%5oV8cY%H!`zK?zaFxboudzH_{HunGHy*Oz+4?PY(^MO+ z4SbBs{R-HS_VC8evBNw#0a;qn@efe>tj|nF66>Sr_PPv})Hh8oM>PFttARPEVXx`1 zxM;k}$}jNM1bozgcXgaBb@-RQARTIKwsp?U-O8U2rpk**<~;C50>|Qa z5daM@Q(#R*S9#%stzeXCvW_=bV?F+7Vj*Spr`xKyckb4!)61D5CLIo+iLMl-PFYpp zd?@`8M_n*A*M>idd{ALpl1SCOn>#E0A2=L?k08@~JWju@+qe?9Bt1XJVCus#FnVFi z$YsA^9huwjyX_DOES2QboWEG!y^W^X@_rLpP)`_;dgk55!`o*A(LA3Ht0k5ye^9d9 z?+G;)dPzq_@#>}3b2qjn=;F7mf1acMg^3PHzX5|A^cZq>-OevT0oil*g=vx68IrAhyx;On5HaJ~?hPSsbE zq>6uO$lSG3u9eCgR7%%aH3=M=Gx2~m;?IF3*c(v72R3EDRr+`yy<{%cMu@+Z4HP4Ui1swD_OmYN zyHGS+(=~>+Sl$a)#?R5Dk9U3=zegEg^Hdx2>YK#_DzqY8abw&xSnT-<$cNZ&Wp)%eSLnV<`uPrlxyGzPIe-#HjBVyrQ9MxzLZx@q%gU>v3S-qp7zhTcD>YY- zW*xT5UY=9=$B6@D^t{7H{OM;|(0*9{*5q9TAwg-rVwCaS2O(bJ%u+p{(E9RxIG(o$ z$A>F9YfvW7QX94kJA7hKx{u#sgUWpqrvXFq>@l;Ema_&>uFi+EIZH6L4By5Y$-&rs zRhcCG1DDBPKr6(?+c_`i;Wzaz4w`+CAwp;=lA#6GgTi(w^#4S}Qp6S%00}GlavTH;JU0xJ9Nj!J%A` zwE$*WcaOwcDGj%`dSEy+1qg8lVXe_ z@7z<4oWy+%Nj)&J_xX6p7d&l|f1d-r&Puc0*!cJB`E054dF#tc6?uF>7acGIv_#zz zH3vrlb)usp9t=uHm$iRHfw(o?^U-8uM8J(qRA6));}=82P7Xmp>pZ|!XAqycgo_2SV zSV8)H%-hj^-j{xB3=HVp`~M5?A5nYR&cmSehT=4UjPkAs*r$OZ#ia92_S~qFz@Y#c z?Tc}AKP~Z=nS&iEsqis-g+WgX1G9Aq0R%nOf=_Iw21wRio3p!g5tQ~A2}S!+@7K0;f#3;)gXq**LdC5du9jfaJ!%#Xf zXh8reU~nAyl(?)HEm-pIy`={9@QX3lxg|rO z{F<0QU}Tk-AMKLB>H15#>U?>!ueDXMpJL;eV}%ZX z`i*Evn_V95y}S;WIKrM_vs}OIcxbxN0Xv?~|L2Ew!H8>l9q@W+2J7;zcZ?=Y&~le_Zud{l^nRHYYDTBL>EK|>RyNKv{7q4(Y)Dgi_SD7_;sNSEHb zfJm242na|Cq4%D%c%N_Y^X;?GpFPg6lQDp>U}UX1*If6!%XM9MB20lR2bnxGcQaJ{ zOB&~go)S&lCv10NDv*_DTrsy$Y}H}fk5qifKTYYWKB|5>0U-yLNuMhZuuMGDOD&`4 z>`lJ7{MBq1CpHel2%!ykv$It7Yr^<*nrr|+Kd6OtGy8fUU6Z`2 zZ+D!wz~g_j!>8=kP|hOnVY!k0xOTbj@x@J8FBG`OGmQKCVU~%Q6UOh*^y=c1b=qYF z#+=Y((%;{YUJeilW4St6+&1yJ!Rn7sy7q87v^sNWF9jt zLZA=;)}E6?VMwk#R2D$fwDOPE`++f9givdP*cEZkKit5b`nM=npFQ-QJ$p4tz?%Cx zMtbF08%&d}#hAdJmaDzKYUGuR1E;08-0LBxXh^bv;Y-TiAM+Qny$q%Xb9={z)ql^; z(yA_Wnp;bO*}XJ@zV)U*Jz63 z4%=(l)oF;9HtPrLCRaNKg-JRX@^drKK>(W?SY66+E`8I0>Q#9Vn+7E-yYmY{bw1s= z=g9A%$H{Eu{8eF4K@|mB73N25@iy=~7OLeiTxqJCRFoT;zZl1igFkncE+z6ZU(l;; z^)&BOD~#j!4RhQCeDGozIga4qF9V?gIW!Wi zQr-sI9Zvinnq?9fqta2%Pf5ngj;04zsblO+!iu;?p?ftDPNF7|{@melJEA5V(N{%p zj8}bISQK1;qrCTn=$Pt#%cKqtGAdS&GOsQY^W~1JmirP#@=XW`T-pGLF<|CgLuQ86 zUjcZ`TylWN)pu*C{)Z@vis7N=nf897h|3#P;_xGq$tTr7M z(a_iFKB0aNX1-O(6!MS&`ka+LM&WKd71`slxvIjrhke6= zw{0K_BsnWWyMXjd##~1Zc{k7r5A;fv6^W*-GedB7p)^b;RV^ipJ9fu5l4Zi}AO~od zjrqgH;EdJtXN>!zY!II~6qC#pj+Cs_pwSXH!6~+ zhfRL<^t#OoHD(gOrS+@U`jeRr%NI{aomyAt{41x`BxP_rfQQ6Le?5`b^tzKajednH zlp$s>3%hkZ!OI8Uo^)Nd#Qf{SCX-&HzMUSB4B3u$kWQg}sH1@w>na~kfGb(R@zUaz z^mmzpu~>1^SaDrPdH^UyOD}Toms;%l*_+CZU{qVux}=bL;k0fgDBUx}brhmh0TxR4 z>enqxko%&4vvQ-q=r}}P@2txocbKxO9O<`7WKs8Sm@Zn5KT$4VXtVxeAwTUb+%{8C zKwx@$Wcsfc@I=2i1L{t$H!C7*S-1;J4(pK z2hB4S5wmJD^=%Zd@S5M*MkTx+E#3)O`;5RN{$rlmiJh9-Cag33D)5EjH9!K>X=+Hj6e8M4 z=O;=;)b|zo=r^hed|Pu7q7idmONgPX@q5Xii=xx-r1i916Nngim~e*%X)7I0mBGcD zOBs2^Y)KDO6H^<7P*0i^8%t7U8rYV?fn%NjIa;r z$pm>CjwCzDN}fUq)w^G+{1AZ4OIxls08wk)SCH|R<~ZBX{4&CJ+ylS_ON_b$l*CmF z5PgU~@h^9rN6=c6qt+RRCjSch1-7T#7cBFui=PJ{wF4Xi1p3=MPXwVH3RJlo@qm2rb- z*`i1#m?Hn(POp+THgAzhl6ka3qiWLdGS7f#?r;06y5rc`Z-*N(LEe72mrgG!v{Br+ zQT=1;+=3xlSHU<>*PYQn$e9zIwkz{uA;eKj<^X45cf(-dsZ=;@J zfD{;~72)~C2$tw5xx)Pd)$k5fsA0CUw%m}%Z*o4@_eJ)DLpe|2|_wyY+<41rY{Q0}w@pGDd#o(JagDb8S|;Dv`%hI5Mdwt$HI^xT@QX6eq}gYMvqDYO^Fl>w{;A!5cdmH- zB+>s4Ratwgzg<2k1M2J><6EYNkw)i80h39fQjsEJFFiz+T|ZxdAfNP1 zX(?N3Dd?B=3A`M@~krk*~B6fkhM6<4z-AyfT^7YD@Y1_PeAFCHtc8e8#gvJ_sXKg}=&@?lRmWZ4UL@eQSofM(sBdN=mfCM}mjx)<3D{}0p} zFrY%5;}+d>ux@roTjrC>M$y7{Qb)yTrQwTtM}A+oTsG0DlRP&G66xu`Z~ghW+EGei z_Az0B(HT!xNC!`%_4I6u3^A1&_>oVIEMIq$E#hT1&Rtazkfgc~L%8A=Zv&PN=xb4E zpl!F=8-ozwxDWd_c?ZZK&#UbZJOe!g86(L?aVDyPG@lM4Eut#b<{)6BZ|Z~jdBFxGI# zua|l#Im8O8CRtLVs!K?30^9S)7H6H9l^_>2aJGj=nYs$XI`V1C4Ok&Vilfdz`y&ny ze!5R7<9CHMoxOpz9u{MZVgu*VKkkZ*Ixhv{FD25(Iqa-N-3?D#Qd!iXM@h*-hrAD^ z3YfgRb7?J9VTuDgq!YPV?W5#H>!GYQ}b1^fH@tKvY#Zv zhU6^5@x1`50VTmV`nWyVYupStBf(MUK}3f4WtK=6otG3?JSmsyb)oI1N#`(N_OyRw ztx_S$&wB8~E4wExblxv|0Ja8omGyQ>(u_#2BeM-k?{u#}@OUmg92p zjV%)8(9Jb6YHMY`dt;N3nLEW!V$wrz7 z@YKXrp1ct@aHZNRW*$2RYHy%BSr^YSEv!w>_7F*f<0eDK5ZAxlWPI zalTe)TK*(%*XQjBdQ`@s4fO-V=hf482q2$DK>SPihCJ-~GjR2O7=}58e^f(dW}nsN zEAD3sRs7Fkx~@7bvH8kVRX$iL5RklJ6;&}P?iMQUuwLy{p4U^L6v074IFf2+3L%4%ySOgy; zKHrxn4P=cGzq{@UGygS}i$xV!GB$G;9n(}by^zwb`i!!TPzc|9EMXAal|swA8?Gv7 z_{lzqQPGvLC#^??ahEMY+JNB3NGC`MNzwBZBhe}7sdKQ7F5Y8+6=kLtcCTG#l5jBp#>z zk(yMnw2M)`p2Lgx^*&WOWUKWB`g(<)ktPxs| z06hqR16YTo`r<;tsY2P~%Bc9BQ{j;hP8RHLy*^eg+|w47H;oFV=&UKR{w=*zA6HR* z`%yO6%ta(WY@!BA-uN=e_Q^sT@$`oiQk@Q-7ef|tn`>Yi*bf-WMoDNN(482`{$&X* zWK+4ANt)dYbJ+ugZ#}xZe>`ZF1Wxz6u){Z~P421-&6(d}A~qLv{I`A{BzKi@&HhnR z?%{~&qG{f_4JdV^PB*O6FUQN^l_Hzhb!jXGoZUN$^EueQntqjcQ}f6F0!Lv4DUm2j z5Uq|X-H!k=+(w1k0IT(Obk1jr4p1Ios0F&EMqh48>SzLOwC`)Xu-5(Q{gD1@(a=zH zPBr`eby#cQH4_Jj-in zGbO2ey#$(8)r-6_Q}gq+>)+#BFUBpm1I~>)xi0oiua5q5%9Q^*THF5(c|!BN=O1Z5 zT9{5J6J5^QLV=ww8QS>NM^js+7gqI9`402FNvr1D+?#KqUTi!Z36`E-`*!SUPkvEj zX6CXeL*rdin9?x>z?Spg6o=6!UNLV1qC9yxKWRDieL}~+pg6VhlSbjzD_%&L@7AaJP1NNul{6d2%K?gu0sw?19$G)?B%(~$go!|8`$+;kEjqF;-so+^s3|6q7Y z13Tx~r^1>36ns8W%MXw{bf1{IEO={(U)4o*AeH|!;w?;wOhk3HKAM3b<-N8!UH?O~ z87Q6JB(=u$YH$oRp?wdGVf&It75O{;#?5hIJA-mFY<<&8LsDCa%_FssD-DLCGz`=t zYQMaYYv#ygEAthlCq+^wYTjt{pnQ9(spm;{koC}J#96vti%XXi1M%sFCj25e)v0B9 ztVd?o)>@7M7SJx%IuT<9*!X`$2IANVR+xbLx4B@v>+|lB4t>NOU3w>KImzr7+g$X8G`wj=E0%LWHZx|@s)=D&UeY0|QOgNg- zTvFmb|B9BA>J zZ27yu@(1$PwB%NzoeC{6_-m$QCZr=pi6{N&Z)7LoC>?I$;(4Pgjif_&j9qnw?M*WI z2cg5S(oJcptWx|WmyXE(>J?jHH!^@kpkT<2W%_5HjaGA=OATuO+@^{JJPsb|^UA-w zP5XoBR+}5F!lI&HSEsAn^X~2M2WyZSA4W~TH&iNhk9+8!NzX%+qIxAOt-J?mH{9F} ztY;_)#bSF&NFevuPCj@(B=e$7@*Pg@=y113;$wU5G7Q^6t{o5-6*vti?|*!+*lSdei&+4N?!lsr?T7#*VQ{`bOkC9`Uwo zGoXX^@GqeHt&?u63VI~u`Bb1A@mWo@(fk4RhjUN;UPx<_nH3??DEI3T2U03Q9kzD@ zT1kTT4@dpoJU60#W-1gqinstneV5$E`({`&@?CrN)YE*4DO(f|^iD>(4)GC1mp&|4 zD5r$}xu?}VRg`JlJ1@2GoJxGDhAp!jIL-t`u3$8J|0WDdSI$#eCf95p@}+Olfd3Y& zuJ?KR;F5^-yO&I`Z*2fV`)#{#l@7&qrOO14X*DBqd0x(w!`agpZ#bcCYjoVfwgPcJ zAL#%AKpFLv+QoL#6fb15WPy|cwpd%$k;ObY4fM~}Ii+^7tR3Q|6`ChDHiWOzxh=)% zu%i^xjT)seqaQs_nr2 zd0HM0-ZvPVET07K?%%r7P4OI))W|`;W&cM}Ssa}?VRWOTbUBqTwzsfr3`u<<&#m{K zq%+J2+n@-uQz*+PKX?Hp9|CtVxf?f0c-{rZtJHH7#pkNTeJmssp=0J%bu|rZ_ETwS zB@){>1V;u|q???5QGNwcZEE<~u~K{Ru!o1c(!%y^XI#U2vc=?M$D_RT*Gzfka;RHu zCBP?tf7b@OX>$|l`ed00$H@p_xYl!YUdAqHz5Ga7Ebu#3>@~lNckarVienpXKJjhX z@)NBxJyAzBJCSd_-|-L*gk8-NDPJ#(SDhBu#%RdT`)BdNkDZajQdt~;JN@qVZEBg| zBZvoY;_p4tA*sU8q9jZcep{TX-|g+w>13?gnZS!P-Lf*6?@Z0ZV7J?^*MtQPI;uFT z_rEG9Sy@yRUmt89ww)T3l`*NUm|}cxleu+dZ$XImp=Z$qN-sz0FzhUm%+EcAfc@!=LE11pz+boeQ( z%Z=3DJV8xpS8*X@B%RVucoNI6==IeLyXzGgH(Nd;wO#vbz8TbuIi8g=;0ZcP(9zbQ7#3PFl_(sjX@yquP=ZUDTzGWRtmN1>Fwz_QzZfVfU2bgWa$by z`26Nd6mAmuXUFLig2mT9-e7<=nSB%0CIM7DOg+mWP{|zbGo8Nkv@WI;7`Z$eZ?LU4 z8NXDqH8(rbBhdH6S)V2T1<2~-z@3RvY=6ZAY-iM4nZPlxq#8W#X_A#&TwX!vUc}(R zr{jt4?FZ8Dh zc#2-O2PzC}AO$9C8uli>yr}5m8KVA8*q+oT-Z}Rcai<%lk#HoD&p2IP*Q3&yV77K1 z_*8vzQ}@>jw$a)4eRuL>QLnh!uk+7Rk0O!}pz}ECiFw;F}kj7aOa}%z+m=LqSIc`57O`%}HYJ z{zE`&qJBXPDoSf-#HJ*8E@cQCjugF!r%S*4X&htyAlSr~J%h z1NW$_MONDxkx)_87)d7sQ#fw4e~~~6TZ%h4n0cW$+;W3AadH()TIFQ*h`rq3r_frO zr;H3}*zrzN;YIpsE@#GRChpg=>XYTGO(&qpQ+01v@W!S!tzTjm6|(8& z+XRL3ckj%+8kC3{)5c%SvSS?N1RoFNeh~nxnBONP$@*ppG*$NM_ZoeXkWVTn*zDul zlJDI@ibGX6J;C|EN=L+fn_`J2he+Su*trUvVbzFfy*PBXmOWLOb|7T+*z}ytbh&Vi zgJv56w!j|#4`Gk1^bmTR{jN#$qY#kz$LX5sM6yM3zIYpQ^eg^?+b?hUdDFAoD&v0H z1p8K3#jcF*%N5M1S}rQS_yGC-gdORs3*IG9@ZX9`PWUoN8yv9~led!1{`!f~cK-KS zK2t_sdru*w+^eIh%(#DKDbB;0YCTYvBr`CwdB=!S+j#(6jCMMoZ=Bn$VQ^~NU2Sdp z+eq_f_(BwL}h59lZQ6E^IMCGaPmMK>MI!<##x6x!-j9GKiQ|*F*8r|Hw|At zbD&3l2{2^Ys7g3qE7Cji_I|K(c^~WG_2>I5Y5rY>*YY%y#IWl zwKTT4EO5!_8ika3k;^IE8VM|Qm(3wir)v+BcQ1QlzrPN_LquIv;MAf9h8mRhaPirX z$9!l^>FxIv=Ig?V=0${Q;}N5QCVsIi&w)focXGfpqd*V!(fH?VtB*(&DaDpNh!{JB ztx{fVnmQc{Z2fvw041{3pf*GboaWHUu-&M2<=DAHol! zO?v32TwiQX%31!$Cw&|xe({U@>(Mm`2q>Mlbu5Y>nMDi)i1Lkhj1)Y5a{Gq-&Gw(= z`g()X=B_Kbj~hj1X8R?Qp{(&5bfF8HNTNJ&uf&IUHvmt1TzEho-F(Re7*tbVasT@g zUsfJX3Vd`kxkcPsWx&=FbG>X^-=baSPI(SqYWxpc;vRQ%E&QYrwIxMW#prh?gDu@z$q@reA)kw3#0_ox`hlpX?$o;1m?3Tl>xOx>wLO>3#UrPbJv65JyEZd1y zQk~#niw<<ROjA|Ah1?{ytLM3r(28wEy(9(BtcOc z#B@2_IrmmZVwiuYPBOl*x1{6$Maz-!dTm&b(aBs%P{Q4VjY~tJdY#pc0#G7keH>qN zm&wq(O!WJdeoM#NnxI4DGz=gpBk6?5N554fELi#!x<`m-=b3$BkB&&=qX7~i>*%Iv z8lOdpLN=xZP+pSiRZ_xx3pb&f@uQj^T4*_7!;k!)F=zqY znG*eE!|H}%w0#nX(`JYbQYg0~Lb%BO_a;sJpvyYxmoddfmPY{P0wExpmfWKJ{5g~_wE-4T^xJ-;k8)O)XyggpA(PPcC=S>e6=I!x3wCvk7=6l6;VqyZTou_=o zeb<$pY|#EafCmB$mI-H?1HX~e(NW`cnv+`WhswN?4)Ii6Rl3Ij_@hIaoa?y-<;Nd7 z+F7@chyv!`ez@bg7v{<1D_D_$u3t{TKD>3^g0gB5UROkVvv+7@8^YA}Uv*a!T#xxhinJ7l1bWaM< z0#^G}`gsu%;n8ugQRtgfcyKmlxHi$yMi5k3%~Ps!>KBi|UC!@g6e*Z=Rm)V}_kiMV zn;J6(4sIjK|}|pZ`OM6fol$Pt%>; zWqL8Bpp=M<5I!wsrj4X*^SMyrMoiaWAJ{vHj>KtAZE_ z6a?_E6Yl(D-nj5r0nqb&vYh)y56bwl{bm-4PIf(5CSG9gc4dxGwL^WgMZv2z2$Oe~ zBJnqYKk{JhQ+-(4zOHGuRPz_Hj?Qqlq(xT~71y>%QDDnKCE29JPnJkm`g!C}ZivtQ zEPiO%UBP(b1sJWGB&!j3SY*zBSD%=#c=W%hPuzx3A11#$uSQZOcaE$C-j>5}si@nqyVzgp3o{8&(=Zmczzt z9EzRhbTLg?HICe5@*V_)Bp=D`k~B1(t+1cQ7TK}vebvT;&-0Q?M{kS%tmIY~0}W8P zA!4zS>pebQM3)6bxY%QGy$M_U4H^hn2xv5UWor6?Q_sTdt&h5^Uy|}XxnF>Tw{HV1 z+(b-1TUMCUa>M_qrWvb-liPL2taa1fVMPk`RWSC9cru~b-Wz-mvtZMHG@`O-W2O8l zZe3-X>Q%JfCD+2Ih>*tniIl}!i~2m4tvC zo~xTx>WN1vs76hsKbo^cJ!h_))qt21T(1931tzTO5PSG{z_PiZozMO2=%8f(Se)fL zAOT54?KS$1Fp^hlT<+Vd)bOwPj{S8bvB_E|u=5O;_^R3W&&mwnO6`FNm}ijB$my62 zMMzE2(~WY}PjE{4+m_5cO-nmbV%*RvtP`xC@vy_SwQ4Za5T}mWPOSa-!|1y`1B4^a zVC*MKQm--Oaeu!XccU%^gH8{GM{OpP1?ZC3_8+B#$iF7+qP=PmHoY> zK2?ANK~E^8W`*vrcqLh)ZD2oQ?mx2Vq3PU^pNRE~;I0^mPzn}NpAPME5lb_@}_?#bvPK_aC_W3J4o6vebud= zJ00zQx|~>xY6pysRdMrocYphqKi)y|EH0+wZ@pBr#G_IJ?uL-x?fgjwl>>cvNT2+v z+s>ite)Rp$a5w$pPyuK39^oeeR<5PB)2fkpA-}!X-iC)(DtwMJ36=&ZTl$T*`eYg6 zytp_)k`G)Y*M+!48yl$%+Fd23i;;z|`UjldXai#;IW#$$Tc7E6e*Wav`ZG+R68_z0 z1Im#BWtmjMmy2Oj1+xOQdb2gXGP*zLexzz~KLSF1ClOlc z%TJ?z2X!k|QpD5F=EqpFCAm(1Y54EZjh_{zO(W^!{*Awha~nFO?JOkRjcc`t4L|}| zNiDvj-x3)EBW}~B39lH7#8=GPI-e`tAd~N^k2BARh%!?Hbs+^tb3?n-f|IEQ)CX#` zo&@XIHa4O+*F_r|q|S)QT5_)d^&Vldcvl@d#OLi}-F$7==BKVSmNyfGY}s{GLsf4* z+PzJtuqgKWIk&{neh-9$R;Poca5~9QS15Vqb;gKp8q5s#a?u~LN^*<&1F+Q;2!Y-7Za(_4&i^VRQOWOM0tDV!~PiR<>7bV@!J6Sntly>;meGHg-?I zi5I=QZ{4JMG|s4~!nb=NY1?0j)I3ZonHnocc|7!)OkQV=uCGCenbInfY#T=ENG7tr zoBx9J>pDHG*z-GsWFSaR@B5aE4F>Oilck&p>o|MDb>Is@);YjK%S zw0#}$P5Ce&yI&!t!HhoC)^Jcd6Ht_DYx*!o_2;2QN~rDD+81I#jd9#hw@|T5_od5U zMEZRmV=@D-Vue3SH;DLhKdU}x%cCT_>;ZHqMJD9l0&u*zw@NB7+N>{qA7hwu?jp@1 zDTCQKmQIDi76!^n+dq?;45SikNJ`3|FOc7D66mg3ni`C1`TEEB&Vu@bPPaQWck70; zKE(Ns+1FP^_At6bcknvMjQ5bw2@uW#HbtKar*kVO>dY`yp8fyCet_+(AP1;ZA1~ zMF(V@tw!7p(zoOx_QI0^`=)QRLK_nr&mLzMu%m1bfHGYGP$uJFdg2qHGq_fKxfVPF z@r5o_>u33px|~uF*F{p;7Z8lDTl%99Z66o)Gqw(e(DILz@5!?PGOUO&P{A21$kOV2 zn9xp=ZU`J$Pw5ve7%-1eN+@7xaH4InT?@^^(o2kGhsaQ_C+sv#<~wwsy2iEc6zN|Pwka+p2L`AqcwFC**N&vrGE*kqKIfr6MwUpA^s zEjF2>+F`);+ol_u5(U#o0p~DVM`bZ^$Z}15lBNf|NUm|Ufpu+R_2Or|3^T5@VJ**| zNEYtYJpr0N06d16^NTDNg^k@GvC`z>^FN1GlU0)BMIr=K?LkF-P$_-L zsz=EnOE#~2qrW|`6>!9ooXk7OC3E=!x06x-;adWtdL>!*PM=Od&L*AFQ$vx|~fH>z~BLZ@unMCIfQ8gk?;?`iO9a>?B1VA`aDf}lSt zO!~YSjDT*xY-WzS3RKcOUC*@GYM$Fk|0YL8W;Dlq_^Hr)iz*ud1)2S(eFOOjOoWxv zCp*4TlfwFbm*k9*ent_g%GZg@76PFG)4O* zf4#N(PYiSY%UdP#|KA<_k>b5pX3 zYJBAz8GiGq7VHB>>mt6(-m2)RXp-ioJZmyX%-D?5f9}=0o7(7qLdjZo6}Nq`({Gv^ z-21w#$(pvNt*icwh1$Y5`e?kohQPfTqWP0-J=vu(6TJVS5KAKqC#J=6Nv3wbiA8gk zOb@?sx9qj=1m53rX%dcKigrF}<;0)m&pTesjHB@%^qR39{BTv(=@g*p0YOr@_Id*T z4gQk2#p`NSc6YNZXqN?kD$m#h%0fF`ZL--nZmfemd5Sb=umYe(D=2; zbT*n3NaD&4lypL*SWh3wUCtHFU$s+bCJK1{5a1jP#9Lps>d7u(ho(=tm3-th{FbWW znu{lERGQP#t6Q<+QKe_&U1jiNWpExSYdZs2MCc8sxrc2LDqv7S1 z`DCmIweu!^hnYb}j4ZefY~XV0y!Y6h=x3n7#mzF=!*FX8_+cg?HLpLWQEo2*e>}z? zu$mbBA|aZ@@f;hnTGEKm@DTMrB7U)dGvWB?L_7%0a_AhS! z^X1{A5~nAJ7c|Y%u>so-Y@GEt=nH1MZ*8rd&L(` zCl`nglRFQdf3qJ0oXI{j2tT#s1*C4LJ>i#^MRR9YldY>cQe<3t7v0u!Q?)kON%&b~ zZNQ(=M3aDpS?hZP9@(Y=+qgr+!@l{(i&VMBU*C%}16R30Su{<1>OC?mu9|^^5iM7v z+|~h}DX+_3cPobt;un?!Z$?;4UWJ%m{EZz9nmaD7JFA?(>J25@Zam^|MQ@6e&3c|w zC}+OgjWYXOiQl$Fs#8%jyY}C~ZMRBA4h2l(nYuH) z6FvVJQMY?$Ps4A}5{RQl4A=0co)Ho<$T`n9`0iFb;63HMMNkEmx=ENfw*Tg6X7BIc zYIw8V){x(>Gy*$9o1;LLu6;P z%x2Z6vub>DJy!i5*^@2% zkGHS>CTb@mH*+-tJ9I=#0$sNF2aa(={#e}Ibd1_4N=9~78Whl@F3>G0=|6b75}mo1 zhOb3G3$1aY*RKdDTQ#h5Q0$0RA_vL`UwN~++>E_Z%0$$>*tHEM(BUE3M4=^O*naL} z@Nu*(HPFyzde5z3YD!(z_lH)`#-VYlSgC21g+WUW6|2X_9ZtaFM0dEX=0v5Y(6XTG zoFd5=do^YJU3MuEheyC?zCsL#dXh1FZrlkqdy9;gS@;{tFu`Zo;NWvE ziU1ySqbND-Se9etJF~u+JbygLKLm_iGZO4~SS<{X`sNNwBlu7DRPyC)qcnRTNYcx# zrDP1Q(97AUWc;`TwF$6TyJ~bgzvR2~!E0-De6kgt)g4-!uT6)Z-t9S?iq%LxLMC3F zboBAH*sPXazDR&CbYu>BWNW)FRxneF=pASlC+eMWTv;Gh+8J@OIh*t>=b4&BBbG! z*vAevVMF1jhp8|0n0&`{GIkjepG&uG0H&M(&BL9*J}-I(-o7Il{?tC=_Rfd2Gd0V1 zG=tU$#Ed!f7-w1z>4r{Q1Fu#AcSG0lTFx&&&2{B^$?dV29{jbIfzL3FJwBgpym~e3 zepzm{R_4>^pLsM(v+c{Mgmp54bh{_tJ3LrzJwWD5?=R@AovCpyre)&8kD2}VY**oH zZhc-(t-B4|@MZT(mVik;Wo&063`Mh7wc2tr)|2VkGaz=cje^5BO8EU(m4{>&s(Q9h z8;;JhS6Tr7?;QtreJgUwrVn*5f~A3FNTT(OAMUd^no++xn4yPXa^{B*q|T3wOq+lc zTh0gekFmy4vGg+yo~s5vF?CLMlT1!BbIGgh5Nz z^T+g^kAou;9gJJ4S#uebl=Y5Nh_=+N1*POE0H*5OXJckQ<7io*&ep32+s&JirXnWt zAa>W;6Hm7QkFBuvErzL@-GzUMW0xLRB5q-tZyp-vY0tSPUasKqzC(DYY4=(OKDj!J z(?L9aM0~!CmaSWpxWE}O_&V2{R-M7eo4!>NEe*EOWpRaZpOTFi!U1!2K6v^oHeu$yjT^8hUo0zmhtA6UiB#I=JPj39K1)bHj;(e7URR)_RA_6rx40l$5lSP0LnL zfa&C+jQ5d@=47Np>DRc1{Td^nk4DT!!%cQwRXVwY%im8ycRHVft36IXIU0$nKj5T* zde8dl;QyEhu7U`mwHS}iLwtiF*mBxSGB&xOo%=PD>yQaPVo6`scXG3q4=DFRZAN02 zYcT=x__jxg;kG}EM$Q*Qi+SJQ-FO%L>jQ|^@0@5TsJ*ZBqKk~7Rnkc^zE|4k7*7rb zW(lyh4fQPR{gtLW^Y(bJ$rAi>nw-xN6AK(&G%@SHswTxR7vMSn=eOzQj!|B3xXAI` z>5_ARmg=sJMqKZ5GODZ@&zHegd$npxHs^0md%HzX8H5d$6%gGRFRF=6=_^V-I_X4< zwr*GR*B!%aT?VW0I}cKcnzr~mHdpW6ZqBT(7g1vd*ana4@M?%$amod3R78G~bTFh8HkLZ(Vlz`LE_mrD#$07So@ITaUWBvpe6XlioNhQXizq~U)Gy4)=6yE;R^=*u&= z*4%b~bFki%?kNgg8{H1P1n0|PH)vY^Vf{N^BsE&75fvS<&7En4|I<7ju&{ympU$xV zD0k9LBZIXavXyof8wFN+D}G^8?$n7=u_xW_tZ)8ejz944=JM~qV(_b;^R2o=z(c-| z`uoB33JVvz^dHjmo^%v@!c1FSoM@R%Hy79KZhAH|Z*sO|K<#XdkCrnm_1{~{nEe&{UjY}zEfS)v z%8W~BkILo3UZ(o_+K;uJ2Xg0EXxRfg=%4lLh>_r3^EoG8g~8{`*0FooHDJ*_pAW|9loe04of?(YZ%jnO|G$+hh7uNjcQt z?|cUH$?CnH_n!~{^+nG_mk}6-$9VM(!S`~Awi;`-`6NpHXLmW9SME8>t|!4#$1?U? zcc#(K&8NS)@nc89fXKjo2s;2rCa_7pKFZ|m=g1K&u{z6ddeLVJUldx^P*-o+yjM2! zJ;($W7Tto!z)vvFX!LVH?Yy-9-vAd9!Ohoffvs3xP4j=VG}#$#9JmM5@W13X4OqoH z8MT({7=T=T}}9sxXMYX5#uCAVtiGgrx*$f+|psWJUcb5 zTM0#f>gomJ%Q;>#jDAU6qyPDvgP6qDljSnIjne~8*`sRbfWEc0pY!-V&iUggx!u3f zfoJYaOzEJk@ZIvE%>8h*b3Hg-RSu7DwwQFsz_(>+0&({C%yn{6IeC40O^4~*GRsw? z6i|kVGJh;y23ItHHdx!b`=GA|?$-ODlJORS%W&|&_Yj@7`-9VQu4>`&w*O?{@horN z?Bn&$({-nl}FZG~aqw?R)_YrRgXIE_{@Wyxdr1 z;WMdOENp$_j&;VLv;`httu@k!GE1&5vCP#4VJ7)6t`z0YZ1arIR^_fHrkE8+8t2al zF_4za;7sFy-m+-Pl{&KK)y*YX4u~~U05N>r>fYXGdU+O|>AuB7bOTWI2c$Iqb1zq$ z|0u?*o-VQk9H$iFSL(LPdooWDoav_r*0MbnJcySZsqz5zE4|l>0=j+q2MDGwZw7<1 z6kjA!pW}98*dkY^yl>u^7UG`)UWlxoSU(&V)F;{Mot&o zG;-$!EP>nE&Vk2aeFVd@d@yq~S*_Yo z5#%^i2Uz2?Z!mhaKzC8Jq~E~JL-ALxo$z6H?xH`3S~g* z1~F)IxDB-#O1hDKo8aF^bf&+exJM4#4}7U1h8=5X4;GG__ft(}wo_>?-%bX`u=ySv zy%~~#&xgsiSYQQzO%css)|hxhNcJg+UfpH=d(I5~31JEPS+UVgW+dk9hh{Jh{N0-$ zbQH1b?d7Qg&OD4$E-v-L=q`l;SMYgS$m*TV4T68uXv0deu~Z3`Qc;!eeJarshbq>x zhoQJKwBwNfcD+4jMr6M6gr+Cc#|`$v)~>L&41TmE=67^x=(w6-STb|6u-=MSbh$d+ zZ?U2!gxXm6*vOH(?M#bYpWz*${sB;&BdAg?7eC#|mtI(sY5cWRL;H_?v{gT&3(4h z32(T^#AwFI)B-?S8kNIeO?o%&iX>RV*WFDUu&sFCt7+@Ylk@r0vDKD?-+C=pK0a%k z&OV15`6si9Ee$@szsZ_#CEG!LxJLi8jNRhZYi6&t^lopN&j7CV@^Ihups8}ZVd=1 zVn`kM;l23Ai{GXq=isj*vWL6TLA$d<11&>9Y#-N(KP>ydhr&|aRER|^uHYpzb_!GfIEMqheQ zc2!IsET1|B&-6uiL?H?vHt*=@EO5yF(u;7g*%E(ebe?J#TV_w#$J5T0wqf>$)thHO84Z&v2_@z^2SiHG zvb%RS8YJ;v`udx{d0d+;^hi&#jZBTWT~Wp;4w zzK(LS&eQE_9@7?j87j@u&vyE?thvLVc6iJI)syRT<7n{`$&D^bjQg%mKqR;4d?n^E zw-gOz3_;37eE?rmQNAvQl`#?9aex){s3UbMJV#1u&r9M7OnaT`X#4S!9Idg>hbdb% z!vmOLs*3&8*qXyQ8=KMZKkdFmR^!9-UiW2PfX$)QGwU$zPp4rX?%uzz)+ep2M^ z;fy3J_og&+4z^9ilsWFLJ^&Wlm9*D@anwOg$BzGV!P=7Yp9w8J(lCxxldGANEwQwq z;IZ_D`1b8U)uvHeyz_z9_|8tf1g-9R&WKGVu0fLE8m2q3a%7!E91m)IpA#{YI1hTX9Uk|6PCY;@C9^4QUAmILip_DYZ1HwYv*;K3`j1=2ql4!R1bZs&=JmpT*2_n1 z9QMPsZp(Yk%IZ}RcC@BhB<7-~B{uUFwsE_9`>~u}+wZV?#1YI;iB6W9-d@#w)ggiD z?qPI?`7Q-4Ea~bX{gL;q6^0l}SxutO^IC=Mpup-03Mku8u=>$*J z97Y(w4V#v^W5*z2qUN_6#qAAfd*b}r;_xl9D9=e3^lo8-Zk6?_rgtZ;1c}bOlZoVp zGwC37KgJMgkmO-IRiMTqPqZNIcVO`oQjsJO2xFY)eI?dw6g+nv(B(^%(a+|(^>qHB zq93jyJj9|p0(9}+7P*HVrRc4(79K($6N=>S-KmN`TumU__C3ZBh99oG~F!lq3;i=Ft;A;0Z*r z0eX8AzLJ}z7P+ERjW=K+DD1T7tZpy1oSmk-n}=HSSSuz~%$X^m+`@FnbmO+l9Z2pitEB4L9TFb3 zvnIlwxk8%WqcdIzx%m9_Z+VsPf3+4@I#~^9SBeg-X^6sH)wL%oD~}!{?~C|O?l!Er zc-rqF1?-29jfWT`acSS@F+G2%UXn4MeqPk|UmYNU+O>mjidAilzz5OZm zl%C%Ft8x4MOq@>GC*j%a<+6pnRk{H*trVxv4Hc5s^Dmif<8bpGv0UD!U75vk)r3^s zvNu(hdLD<`x4+Gf-4_bGalAH~KkW<+NAGtqjHt~zJKy6cG)t_L zl5%vCO}kjzw6v>ix%a)yEBckz+#PbCBUo5iZr$Qd;_T4-dVk1%7D)Y;&;EXXrah@~ zRNtP_vH2i3_6tkR@04pZUZ8uWug@ITWNEo+IbM;Z;~Rvrde)83iPH&;$wm4dKXFEL zuAM*}s~j90j2*}$jptNmM4SEm=Y&?NRYKQD)VF%M*d{|hgG*ufpde~a^?csa5NVd1 zvf=xCU9#kMOEVkHk3R@m_G)m1i-^#ekL8Ya-SG9M3XByPDJQ-0B;qS7D&(|^$j_bL z$1f%o)IM`(sI5gB4G%wg@?>~;m`k%*!fWZ{W@Sus3dQ`>2oBZSJ~=6^$`=6aw!3tYJ-t1?QIg-~0LmT@zPJM^$@M0r9;m>RiBE zDlIL2*M9GLo&;*9DNJ2KA_Qp1Wgv3D$)_=GRP;_uK+wd0oeU{Z2JbeNtbKqZfEhoB^D}9QrXq zz?DR-rfp9Ub;S=3Zd?+qq$Uelv#$$b!2*#zSExE>)tl^3_3N-F?RZd7W<7qN;+mv0 z%f453wmqlK!qUiOAn8#xK%VpJAj%qK3^;9OlH?} zkzo!I=xl#@Wq2PW90WyBi`LPnko>5&Fd4 zS=Jui24gPGcu|)UyLWEk9O`yXS*mH))hrZyhxSL-ksv^7Djd8GWA zLv(C31>!iKU|?WCy4ESY@o9W~yq%l+Dd3F#so&f>l{yu+KZjEOFeh!`e|bHHidCVXN_PYO5kvngX2I z7400iz$byRF*D8K{|L5lwrJ=SrO89tLFAp+)wc9#7+ZiUvDrPlCHloXyy7u-O z_h4NEslY2j`YQ2yC>l{Rme3o=-#uc!co{>-5dUaY*uBM@%0D*R_t{VZyflp}r* zY$;cZRBEA~r(5orZ&2A^INzSPsJ2`gNLrONlWev-UOJYho3zkb<|1dTCLj^^o!OQ#O*RKNCfC!4n+ZJ6FZIcgu zYeu5<)s2nkkmNR4h33lU7w29?=fsUI50E57#CdtaM4pgu>`f-I%Ei9W8X-rBi8EOw zm-zWLrl+Ud&&H9ii8wCjj>QKvp^4ZiciEOv35f(%7fBDSqgI4kxn~tghxPjP>&(m` zMS1ng?4{}YQXdH|l2A=zz$le^d3H1e$4j_wuC2yu7B8nBK1qq!=1#cl{4A~lx6BqJ zmAHm`+WMf&*Z==PE{j$2l{ zN`IC8DkYQ1t;m)HaZk6IiMP;2rx7bFtMHbdj*i`Xu)?S?C`f)YZh2wJ6TglS6BFYv zc@(&d#gteM@9pHo+Lw#*-dbN>9@w|ELLWT`H#fK0W>MgK1;xbh?;-~h|2kCi_+Tt( z+0DW`(UKatoRpL#>F!QRCt$lAA%`T76?0{jbi2aDM4H;)BDn818y1j}c%rpOvgwpf zp2c0;7Q7`PFEtpBK;K2H>;j1~rMI+y>R}Womrl+QM)`rGXl-*}zJBNo;A2ofFrQu4~ zD7WWUmJ1quQLO*MP-Q0Jp>Gq34S{$twj3Vyh}mrT+blRB^xmP={z6Jg%_sUwX={)A zY|V>}V&N)2zP=50b*(XXtbTm`=@W2yv8R-jP1l2t8%u1TaxK)Wex19fn9wo4Ow>)R zB24WUss)Cfw|MrhH}6i^>S%@Q+~?EuF9hFQp7le>fU}T|$onE}va^{IutjUD{oORf zryO=E*r_@Jen_+ErS95jx%D7YE7geK!6Bezb$GrhR)IT`t;e-R zEk`{^!+E}a^=6B}mXsQ~;@A5>yA0S`6GTULw@hx_m^K$Q-&vLX0b_ckzIR*b)zplc zGMk|F^StMY$m)Y_hJ*S0{%ZXUvM*m*-$8YKGLogS(I^y&jDGR%WMJj)3YYg4F~_}a z_Av8-Y_(+RE*a&t*r=$FOrqwCAJF`TC7}lO?a}uN<+f76ubtngXpJuYG1@vR5NVdu z9PYlmFr21GS`&9W#~@+5)6+erE|frn)(}p9Vo>JOQ- z*y80l9S)CO+*AD@KYmiocdYHWrvWnRHfqk+tds{}^0=iR!j&g!fq{np(@1lsCyYev zVt%`K_Eb*)6ArfAg^`fdAg58RzYh*KYasoe&BmvZ?)PtYHdYB+v}8I8lO>Eby(v<-#8d6WuD|T1V>y<`*&(B@EqRdu9ZYiNwjor@$^ z6L$hFc;Y1k1N{jHg3=t}?O!T8`AYk41fSX>#?a7RNX*F*H^&=BehAkSLkW3KMVSj) zh`VnN&Tq|MFkn;3P`km(YFs~K7c9o8p*3=uU6+|v#BEC89CqV6&V?e-P(4OMI?{qP%)}w@)B%`4CvNB zI51^mmM?unpd$jf{_6@9)rFzdHs_ zPPh>Po8cNc&cFV8XLspt+Fde3>VZ66^WGw##Kc7KMO_tRIk5v0ud4$B0&>)9Rmr;+ zch@#HHWrJ%wfvU-%%Po}PW8=b?lbqi52O0CY%GcL)_K8_WsF}@Jy9U$U)J|us|-4w=LQwO>OJ| zuzEc!t2Mp2%wKP2X8rD+-9tn~L{QN1bU9L!y@8eQFB=}9IbTLa%^c8QzD$2HhBGpw z;AUio*@yWh>6GuCol=ZzZCCN4%F+sYk}P{!M%G!WT$*_r1z%-%$6>TI`+k%oiZc}@ zCMuemrc+Hdr)*W0+Jv`EPfbs&Di?0!&V3dxDk}P%!FpSucW?&dz#lc!jKf(xuTVLE zee41-e0xSwS{mMeqg2PwpFS;A`khyR?+hh{a*4Ok`&8mJxu(KaE7e4$z7s8KH_i0W z#zsl0TtN&HofSH#X8R=Hy?dvM1E9#s%lmTJ`wnSE;`xvKkPZ`UL|`wY2rr*d5IL!I zGuw5;hV1ECnd4*=k&0u*{ToTX%-|ZQrzzLQbaLLuRzLJ|t-=$>Mv#7SBPGc#f1gA= zO3CaKV{-L)O_C?ynAu1=QD{Pr(5-pxwLi5#X~62ZFzWPkCA#^h_$bzMhsff6*X6*_ zHS&YryMLg`n?u8#mp}aF?#zDq!@@%Hz@9S4w8|PjS{ZyKSY;$nyh3EbF~h%uKxVq$ zlCGTha|8$)C)2o)kmn^;4g>L?j*gD?@?V5JyYB)<7Ge-tzIw&O9k(X%{rmSXcdhyd zFT6`K(Es@HF?uX)JHEkB4qK>`c;=r=X1(kcTac6cmdCy(k#>Qia6gx$alv!XWFzXyziNFQ$?B4ZxB#mG_C*MHT^jaagZzKx|PxX=yO#rgC@f;T8iB6 zpzcn?KR&%FK91Tq-dywf^VjdSR^6Ou=nt#Q`|ViNrerv^-=g<&-r-=&9tw5;jj`C;+==_W_?ufI$O9T()A@cSD*hW~L~ z<}E@%pf_LIpO!~2#r3V@K>CXoREf2M$I;*2zZmTp!2-jYK}&dKXim26I4zG(6ZR$K z@5I)O)yf(B{grYG`Gk8{X&7~Xrr=u_m94UK1T{cnYc?@G9rk*~ZoFz}=eR2wpFAtW ztB@wca8A`{b4q;vv>L^3Z6mu_{xn`EHDdMLW?vR(1eb`tZC+?%*AGcw^AcE7s{6^N z zo_3gO{Am%{F6mD6=vyjA`Xx2UTMm}F{w{btz#S*t7H+ZLc>(EnDNUY7qsXv5n~rjA z{8}#>+jwW1QGDlFjkKO5`pu6Yd?~V=*qG?(;gKP6H~b!3DFZyNy;;+_+cFFzrF{bf zv`CcCVO-8#Hmv# zLc+p=sa)h=6Oo@~!s5fe6ET15Gw8%cEkp@yzXbemYPkY%JpH1~ucRKYwp+3WGWRv;6LhbaZjVI@L|B<+Q&UCM>p`=n z)h{xJ)fcRC6-)e`Y}dEnnmn%8Q`Us^bE?Mm;I$2nVosmE*wo>*9~9Q2S~^X=(R}{q zMSo=IW*Vmux6uBH*0weus<#gw)YsQHN4)uQ-{=Mz89D?gsB~)-5x+mJo^4rD>E`W~=MbYYI~`TcjrCrA z6s5oG>h|{cIc-n}Ggbx$T;AVHOWn87$K;Q_eaoqq6BHO&Tw03SSR&1HqFoRBe@gf) z`)a+IEh%Uw{FqMM%uPr@m+sCN$QyN|jwTAc$>!obc~VEbdKX*$>sM{qiziZ2CQX~v z0Dp9uBvvGC)4i%Ikbcfqnhau&GsC!8XxG*pj|ghx6XZ;XC06GdMK;$blgi4@cAXOd(p=4S#IVT9F}a+a@TlG&(%9+^>G;(nrhT&4~gKep710Lr~kA;Z7<&R zHXTh*Odv&0Ce=TB@mWKRY8yMkCLrM0Iq@n~Q4!orl;m#S*L@xRA=)}tk<&m z^N{Ul-F4U*9XuU2iXH8BM39l;`6S>W9?ogA^Q1lL64T`I=)PM~{W-38%YZ)E20wG{ zq-a1<$E#ZgdANm#^uqfl{hVIbS*W>E0)Jis)fE19t$-x$ix+CwK7qE0?0P`shlig& z!$`$fYjxO+1O^7ODO`)($H6hOd(ZpNoi%zI>9*h{HI{_Cq@xLphexEZuP<@~OCIL! z{Wqdpg5~u6pm3)nFTE!tFE5`tk|2rde|eM4OXqKppcLfPsXry`XQdu*jE?$K1-{R9 z6h%X1!P@a9m}}DYzKIC!Tb|V8?W~|bJ}gJL>=WvqaKb&H|;r*GSl%( z36%mA=g-seTkEHo3&QtO;vGzFZTp7C;;v|$=!@YJBJ3GJY zidW!Py5wBdS=;QH-Uc&C;ueL!W0YHUb9xnlcg zU2MzjahmA2?-&>;?uN5_5}JKj2%kBZx7+9i%;s~&v()mq)QY5SzwvP2n~#hBsJET> zR3E(!oGuWzLk$fsE$^N9tcHr{P=WNKUSmt)G2Ba}#pUHJXA$UB(~>J>=vzU&ENo)T z`u8!C#vNbclRXcEbZSphaxBlxytI)t>i#>}H(0mIBPH|tcspMnv5fEl&V7*SC}u)vs@A;f*(y3l_vj>xv1w;sQGMnqNdbcc#-6tgQvn zho%}D8Z{oSTZ4nQQ1$bd~o zCXqfx#l_3Lxkd(71fZA^-sY8?gM-4-IN5WAmEG~J;t$&m zAgU_v_er{#(a_Sm5PzlINqTY_7G8b+^B+`Olr!3V+E|ElDh^vc&_t)f411&Txo2ay5sB|%qZEuz2p^@HXjBdIAcvDoSm=`nnr~L zs-8QN0x>mBt*O~PSXL&ZoC++g;l#u%F3r-Bl4cjZ_ZtKe@87n@mG8%HGHVLNL1GPJz;Bn$}UTbJ$2wbj+t zT?d*uAWt9Kny2SFL)K0AcGC9*>KqL+DuzhP$d+q+hV~+4=KrI#YGLbV#;x3^!>`=) zHBZlnYPMNqtjraB#>{MsdWvkMcD1oi$_Hw0?TQucn$a}|tpTgS?X~L2`VYhh0W>^W zx;f@2Xic0b78aKqnr4+*1NDma&7M88oX5C7f^cN)BWbuhPRP?muN}D0v#bVQ6}0Ti zHcK+*3J7;x61H7miSAilUAf5f9ol-#lhoPY-(IKR!~LJlGES&u`UcAs!MDdaT|LTW zrl!=gHMzo?YEGWAsDQmlW?N`UB&aQQ{()&j=GQaoSy-gKAN4%Oe+NhF|A^T;O30|Y zu1T4eOGm z&N%v=yGw>Ok51KB z#hEioW9QDEQxV9-KRnoYOH2$|{uUznt5=is?0deqE`K;3 zmim!VaJ02mAY9>&5MMz_NpnnEQnZNYT=7-m-PPe&7O~h7sZx9)VYfZ*Y4~k~clxYa zp>Z4isq>6#C*B)0DD-A&9jf1}x?XT7$}i|OhUOSraflQ;ab`x+$2RSAY`IO z&8AIkwt!2cP*+raM}nlqpQ!Yng}}?11$B^erq^OBs zx8>@fH|E;33~t;2-Y#n( zcX@F!{I;TY&*NkF-I&3XGBbBK3=A|3sFQEtmerZ`^k!wUEC2VP$-H9lv!MB&mZ`=9 zjOA)cl!VvR%+H5XUf5AZoaxKwodwicsdCZjRY$fMSLp(F&iGm^TP6RW_fLt(f=+De zq3tL4LfXX$M*hI}ukl>L^L7wPEeIXcGoW)=%? zNCiVJL4!Ph4oeN;yf5{kiT|UryP~8*Gf%av6%JFqi&0Tg5mv)lnrCP^91rS3A@>5x z>@0m}M_rv^=MUl;pKl6i!0Hc5_BY$2gvtRIH2a|c{+k?8uwur!ENpp^hKRlU4cI~o zM6O8<$=QuXj)RVkal$3 zt~<@Y+-p6aZPsu@N}knKKupYIv~00>G)Bm>(|NgnjD06n+WXE4vhMeQG06-Ou|lhU z!NHPt_IYShB;(kr2u}6-_BE11REfus2$%m%?;=4Dk9J%g#{5pv-k5s)X>qkHNhzP3=G4`I1O423 zq0A>FJ@%qJUf$9w$zBoyx->-S-}Aw0hek;5ac4x`M<{g@E=p=E}&l`{6 z*5NB${wrjQ7qIS|Y$SJN%j|}mTSapv@ONEZT^Mc}YHC5_*4}oV;n}uWpInY}*DI&U zk2=NY8I<3X6&1UhDWavMq?WHPu)P4!;Mh2k-2;Y24Ba1h6bZiYl7+e+J&#@Yr%UH~@o&1$K;^pPF zsKxQ-+CN8}k0AoL)Hl4@ksB2SlM(eWWp-)Kt1(EwN*AUoxBVyN`F7JKpKg9g;o~Al zI9?2WnnI@`{AMNK#-B%`Q|T0S$I~n6y%o^n^&c6`t>U!E?Ou{XnIBV7b1SW{PhR?| z_ih&}nW+)~|4dA{l$DkHRvR0o+o+FE(AFm(QBGv$cA#&5W7*QsAO*R} z?c3S9KL-U|jLfo)Y`#j5ION!y&34I5JK!8yy7Et(x#v>-s=Ke$DiEkpD%4ldFK;j; zS*W|=phe(EuO_&L!^Hh7#5#viCm9u#6y!HTz+fg-FwoFMgoM1bw8V!oXliL$n5B4n zBMl7=wY9Z#wIu}!!x#)^bX1#F_Ry;X#TK~PzdDdq37D; z4o#ZLPoM6B;uJ5hv;*bo)2Bh4=(;(xAIXQ}U{iDzQzaC}&GkO1cz>lN9`7a-QqjJx6QyXV~t|P88`6qq6B&x6a@J);$eX-ft z>QY~942Qu`fgm+!Z-#b8{O0^nVI)0!l53%SCOwB*I={=BQsC#emI|qhFq82I&&!b# z@wHthXsYS=_mFd4sAW5QZLsoHlAEygD4nqL@Q{PAqmvUK`qTIC>!CMJ6+xq@l`FLd9*Nvz-KI@g4roK&ld(y(s-+xu|t9nvB?&d|AV)JkS z-p7u;rO#jy6Z_TJ`20o|F_%A$Cn`ERf(M(thzt=w;@a;wf1u%F36&PmdKVvW9ma}2 zjQ|Rh=qa_bvNhXekQ4e8o-;U;o@6`UR5=mL;Ub+Ig6V*e`m%&^Sd4O^(eswbt@c6rvLC)Vxvvkr@pKx42evw4| zMxb?w$9G~l$c7lD;9h>@C8t{SjkvVS#q+s=Lm3~JpwCD&$>UxfE(Q?+yT34vd^?1?4#OEIP1<#qrCVIh4)8w;=I&eK7Kc1C)_EAuk%>K9x68TRmEPX=uLT6sy zbG*uxC$c3)iOc_yMSbgO`g{KKfy zQj=s->;?yFa67A7aHgevxkAEGV1*I>AFW)*{IDev35gGVi8~`U;T)=&I+Z(q7jB4& zi-%Ks;`h3K|DN4h9f{T(oo)yQO_Z^@c}y~4wA_K4n;Sfj>((5DsB^Y9cL#phYaGh` zoNC$Wi=eY6doyEt8UY^&+fLlSbSccNC)H}Gz-V)(>FwLMgae%W)?DC8&XF=Z1NctE zg5<*6jFMiXWp>l{v(0g0uJ&`S(G~-_ZAzC?!>;ffw?a@yABF@3v<&7O@RtK|Pd4G( z=uPFmyTyPclo^ZL}AE>jS;vUQ>&M5Z!`S+I(!w^B)19 zrw83rY%%bZ<&$QyIS`{~8AKerlVvcBevN11#og7a-Ede5*e>`1$U%U&dehr^5ebE4 zc-^!H%sN_Vq5ukS6)s>qFR{sID<-{%B=MII1q6!lRYpce__|!R98HL?hq&e4nWjsK zIwIjPOD!jJX)6y+w70W!-Cc{@T&wXs)Yj2az9!{3T_5=V+*Pj;FOVY`x5a>_ZgwnH zE*7ZZqkW>nuZlq59yHMn^}zsSLn0M4eZb`m#WE=><1h_wZA%6Sw#4 z^@#`A@#?YBQS;vPt}JyP7nfZXYj!9{bb&&q(gmwq>AdzcinU6 z_fL9#j-dMXB#<79K{LUc@PNIMiF<4I@c6iNDz>3*a3{qM^X&jPDG*gvRVUA0d1+)+ z45lL&D@1qY$~G2*>FCH)N_h%s9g;S6i2>N7V0if8=etwQm6VKPw$K3QY{iQuB_$Z4 z_rAV|dwVS7cWf*yva+%cp&h-JtGyY@X>ze@S!!IW#^mSE=c;D)AAKYr7lU^IM^WLl z+|LM9^LTY7+z18D?YoW(Aymk61O+YkN`KDC{?@!nTTHP@M?8E$j*$Oy7vX(H(7YEK z?n4E23JBg_z{3|9wLFdFj*Dqe`u6Spd4{Ehg)*>fF0GQw+}hB6KcN<#F$BDDNY%Nk zLap)QmC$l%HQ*})OpPb7q@!pQ33QrrktYoinn=G0F0J$Q0t@{hm#=imgL@7^JUoBx z?h#0OT@EEz07f;^q0Bm!&cy`TG-wDK8Xlf& z3TvpXeMo-c%9TB!BEf`M&!2_C^;E088QTnq!2fFC$B#<;3~mdZiNO1Uu{O7UeMSNQ z!@GyR=jG=7p%ex~ia^-K#0r?{o0tq7T}3X&a=hwA@%^z1r#Tw`F63c6PAxTpk; z>pfb2U3$DdPILsUveL!I#AN={P1zFDF62F`1K5l-hWlJ2k|-{#w7WLe@#S72Kv=kq z*Y55l+yJ2W;-?;hRI{b8ZaO$@0mMSo6`FK>pyo^!QG59kl&}+D@gG&QH8$YsA<4mt zzda66%T$q}yCUIvFoVS=%>7sodnPBxqf?RGd^WSlcCkBUChdL|(A(UR>bcr4`leP^ z@QXccv)BjZie}Ka)$iWb*7;u&wi(Cc_Z>1+mR44> z)6#a|jhq92J3XQ}WYiKdhDM=)Sp4h)47-GB=T{LJ%Goa*W*S4G;Fx(+Ax@+ZoHsof zT86U8;P+Pv5}vS0Dc#%K-sI%uS0ufvVq>)+RyPDOEYHuUEzMLutTE8n2iV!4r2Yv6dQl@sp(~-gWg~9CD6x#*RP^*D)G5eHOH&~qJ$j^}smW?BEFkdF z&yNbJlC5C}295MP+{Hp{tbvFE$vI$A7;fM5^X+VHrNL}$MnGDaN#lbAOc`-N*U{WM zF*lbgM)oNNcBFc4tpY}^>cI9H{T2*hR!2u?tgeNVi4YMQ+C}MPZDLXi^KoEsToZe3 z3(i?cNQiK_YXd%hse~G2C~4p_PEj%n+__^mUbSCpkm%{@dAPF*jBpXSLEx!>2hf6| z4%HDa4(JZBZ}eR|eRu-{mHNVke2@K2#iS>zqvguvAe;fZc57i_0s8!&hzJY`a2+j% z1u(4P=i>mHb98)$(2LEg**cXa;CNs}RXZ(5*w@)ZD1t?>zzeRfuJRua97{)BLjL{t z-$4DqD_v7ZTibjf7pjD~nFLEi!^@0}@ntu~#I)%U94hHl*RHM1&N@N(eEZj_rKKer z9^KK}+S;)(g2{QFEA;f>I9TKJVK?_92|3tzYRx58V=81aE0-V1vn%1AgK;9#=VGar zmKNK%LWoe4Beyp%KmSz`M;M?Fp>Hd#MoQR~Q$ceNSH`oc!~CcEU?!gMr{T(Xz>Td= zO(}uy9?a*K3(P7nRbUQnjS*}C|3ZEToIDI&fYvpyYTutfe_pH9nhVs~_Fo=@Pkc|w z#CZ9#p|v%)_a68X@X(*#@7au1;GntyJ;P3FQTTG#4&sttjfW!5*K3f!4uX57WOR~L%uGq3E@2OsxWar0d=ty64%$Rt}iq+SV%J$Z$1@s-+AVAn+I|b zc<{~GH6Rmx&QMo4E_Q>}LEPf>hBSNS-6={h7v6SG3_uGkA|{#Ygd;Yes4olQ!}Po z0xDTU@4;7sf02(95j1MPJbNzy8cu9rLf!Jc=I-VOO;C!a@umX1+~1sa-3fo-9K4CAMN9F z^ua)$9upPS#L|+vzWzljD%4zAN*zkZbpW;~4;ukqxJYzfzkv9h!TKPO~A z(+DsLjCOe!gF-XkPqxk_PV*5bWr!83PubQJU0TL=4tY90ip541cr`oBNa_1L0Y`um+{o5Ncod9Z`| zNh%!RLsno1(AL)0e|rl?a%RB2yZ7$x9~{tdX$lBpdaGvc|%4)5dc?*xXb(7#0fj@ z36=>ub=7!QoRp}GvvWM3Q8PS0wLD#}@TUw6479g3LgwhM1KhreK>QpT5#8uL5&f(m zMh76_TOAYnhK4F7mP4m6v9fqY*pu&qD*`Wau#3&Jd}eRmR0qvqW@f&4@uCVp^;^Am zbXYvZ+HWF!A>MIuV_+T^OD#uAeS>#(fhF(E|*uN6OV5z_UNC1*WFjeo&t;jsih1Y~6Yr=#uU+9|~H4 ze_LQS>gmrP^j)E$NgZ`%^S-5;`FYcDBl8l{&DE8C-W=i`kQG|EY=BUw1shI5PRkM5 z!X)XHea&+l(t#$hl31a@u;gjlZz)HKIGMnYzP`SuCNl^>FdwQCoGWN~A**l$gdWy3 z3@bu=n*x9^G@Joj)vJ1a6rAD4ODzW0;9{^IdCJoxe9_a>)6if77TtV#1KMIkDF~4& zA7DM?0MB#v`?Fu%xpSw9k|v~O_QsQscU{)B-&={S8z_~H^1%ZsVFAws(F~j;{IkN; zh;wkjG=X13L#~iRIy*Zf{lr{1vtJY%Ln;DDA9j&FK|!NkWEyJE`U=LNq~}2|glrIm z9YOfDu|qIHev|gkPg7+2nM0+)WAH&GA94}Syfguaxo3Td0PKh0V6?|RpUI;~{fMSB zJr>Ztz)@(+sc(=9G>_Js^M{CZ260i(fVPnw-13DNGvu|9KZW-hUPy+utjX@P14a4M z#SwnJF8e=tm;az){`aY9L7fw1_hcb%;E!%X%J-rmIhEo;&<`j*gYN3A*|BKC<=#NU=iR^6~JFR$R3|v|iBJ z{r*gAI`1Y_ibuU=+QTzxI2U_UJ=ubNDZ{kxiiyGWE+D!RpBrlWK=T^k^0D*oq5P{| zMFn(2ZKu_wI|s^IUo7Ss?KR%1++7R)@Igpe*abDwRqSpfDM`X|)6vq>f=gp!VuGKS zF4aDO9wRgS(n@~zG*Y{g?&3ubTIAm+e~*!D z;lhZ^+QHf{!1|D`0D%XU&**D><(hD%nQPG(u8da)yB*v{!o0r+gT9nqi-V*{o@3Lj z&oU>~N;PECmU5kpTVpO!Q>S>`f@an&z| z9(%)iaTM{J3x-IY)qi_&W;Dkn!38^>RUz(-H*67FjcZoFt(=SWZP&@uLGQ1!j#fBG zkt^0{zkn(k((fd0`7|OIv)BF!#v1U6nW5655Udmx&pv~Ze*a{s(e@dAeQH{sQ&NvP zi9PnS7Lh?hd>Je{YIi0&5W8bcW##>r>sr+y-x0RjZ|K0Fes@1c`kCj<1v^CDDB4)aY?K#fWaX^cAj$n%X204`*K>j9&)z65RX+a8rAyN@ z+KbZ-3JMA0JnH>2vY9slmu>Xy^RwON;_{*DF=;vb%^!al_Uyxmjn-&1uPUB%fb!%z1e@MQeZ?={E?6z00Gc^AZx^pZVLR zzD$muf2gL$35Q6(Up{9J_YTe=K3rzM?Oldl8_$v|{pSP)IfAHLxc0jxG91Vw&Ef)! zHhN61JRO>xcS6*4gUdX@G*{eaes?;S;$he^i>$BAZPri zwMxx5a2DUGkbtQ5^z_P;!8lZQ_Ja&Li@-SzE_ef)S{|>3oqNOQI&ZeeORQf#; zC?TtTslJ8fWgdu71-~KMl7>pAx1$}vMlT^%a#c34)Upn<2MRaq>z^R~(ic!^x_`>T z&pw8o%z^Y1g(BIQ+y;qsOKUI`AyKfARaw?_A&G3ty|Jg?w}1~l5o>5e1aPncZWn`dr|cI z!ty{JXJ7XL$!5OXKH4w%n^Ax<$S0o*($inLHNfY!vM~AZ6sD3i(LPq`zO#yUIy{;_ zXCjvnQ&xKX#3lv8!^0!o1?s;v$ji4Eq6}qmI2@Glp|*WViWYJ=Kw-0U(8Xkn%aNpr zMcicV2h}VW>`)};XQEDQ|FT-%?GVg^#OHf}xo$6{k1l#mPTfR2Qc>w$ESy(SDB=~S zMb_2-u6846l5eaGZ2`y4&IF^0l1ZEf>9{sn<5tW87^b=Cs~?Hj3tbs+s1VE* zMUH^6x4ujI6cS>KuVY6%3S8>@lr7C1dL&ohNVf8Sq_3|J>~vU+|Is#vKY|ZtgL$Dl zb9E~vU>uI;72vS%zB^WGjY$++ zHM=NY=`x0zm>6=HhxtY(;i9|#VQI|FHwN(rS)xD$#ocwq*EIz(440Spx5lz5fTR`VlS)~-e_gs{ASsEWaM)@GMM)($JVD5r zIy;}w&eDEvxOTS+8vN$$W(%sTsObA(F(HAzFDnjhYX)Kkcp~OQ--b%Ku7;=4#Ys5X zgA~fnZj%!XUkzQ_6(A(X8;Qm{QEsd|n%L_DgUxZxs^O+xeQ$ff>`kinviR4z@ z+z*IMYyZL^X}$f?OnGyzX=}Y}dcbDfL+f32+*t0aR+f5}+x%MfJKx}$4(xYpT&AYA zpbu4&)Y+>duS3M-Yi1Ly+;_3U{QPm^7EcbHt!h>c=Ze_lM2ZIb`c9uepP`wu2~+9% z+RAwE-0AA;gB`V$G6aC1d8PwxyI&v5!gwxWl}o$I5xSjT*u2c62`M+QsSFkk5_HHY zqLu*!y*Z0(p_FNkhHM087Q# z{Q>X)>&31Ag~aziy1vEz15nF!JD!G%HJ)byJ|!l1QV*E(7;eNN6k)plh07b74Ewt15CsHyhHUwn{V8G+Sr zKh3^7xy_}M*&A26%PJpjY-qSJr^$cmgRK5;yr;uF5517#^XTpro3R>xWBelngQVf& zQ9d-Jhh*1%L!jiwvSo`zBBzaW%*)Gz= zy`=N!0;2`LF5tt)3mvp7)IXgA*2eTt^}_HJ`&pTT^~q@yCMMwl`b?UEm_G3Yv2zrZ z9TUGkZEhY+5cVA)@?=JU9WjaTwBnz_Exh@1!{^63vGx?gJI6{wH6Si0hiaW*qdl+oHp&mqJind%DUVBSM}v)byIZE#8StcLq^PCa?G8Fi!0~&J_BF*pH7^YM(_>- zfjcs!Q|0)g&wg>!2-NWh2M2W88cOGPcHrMUkG4wj7qo4n{)x!eydzN3cu%RuRttoW zsqIllR#rNxju}}4xq8*k2CEoOzQh6>9CpOy*Gxu#f4@;&vo=>z|WYx=y2pvQ?#AT?bIY0UQ2@i)#u%p#xEs`}gUhwtrNY?<*RnRHQ z2>tLGf%8MSm8h@qNtqx2OUeHSyno~P(@gcqs?s07gr8(zX<#lhe5`p?^oJt-QDyYM zc*`1_pvUwP+A&3hEpsoo`4lk4mNgtpo=e0<#h#Z)+(Q zCHKssN{k#Iu&Wi=ZYTkw7P2Nv6^0*u!M(S)f8WFt&}W$hSr?YoGgM+upSY>lH1P_i zzW*bj`^TytCVO~KF62+s%+0yGSVx+hy~cM>-jL#IZKGTrN!s{h8P}L+Z=8h*nhSk} z?)B=Q3Rvanaf@ThJ*t1cM7p_h2^6b_NQR71Cyqa(l)V0EdG-I7nj+_HPo{+H!EGQW z$rGie_tqzCL008M1y3HV#2^CMfU(fy;a{Oa>ZMhc@8B=vD878=_e=)ANQn(Bbjr<1E){@LaKVC_x7sb1T* z@%C=-Xrwt*QlZEYq7q6}5=x|u4P>Uw<65%A%tXkz%reizVp;1u zfA)T#@B7~Wb9~3~zVF}BQ8KJ`|L)-$&g(kQ8;$j_`ot>FeJZqu4@UP7+Ma)C5c7Hb z>BJ3VC2e#{j#kOKyG)nAHhbPT$}}pe>Dl%BucTn$N;zcOCO#s-x~B0PzRxE7yx`sq$-tmD^d za~mCPkd|(mT%+s;Sf>(%-@h}o>;9fK@U>vOB%ep1@|KT~SuTNqyNL%Su2JijfFSf*W1qN>6xQyFv zmfbn9tA|G^n2$zKZVC3|vg5gPse?;rqWHr96U6y1d{cP~JNxT?PG1I!Xnz~NwB<@@ z!H6-R3qnnFDTY}##=Z3Z{}a=D2`BsLQ5VK1-pm2XeW0p26nL7~TLNnfKHr#)y@z}= zzAjNOBkCe3qQbiQiOv6ikkMw*tL5*hK5I59S!pBa&(2ce^RXZHU)h_ z$lj2sCuj5J9e6}pR@Co~A6_rIIt1?56PWxv9AU*1S%|Sc@bs)trkZrVKPdE4N;7K> zhvdZK1GWdVqe`<&CdnE1$JdwSsKfJlvh9Fu<-Sd8UuEnAC9Y?Jw|~hHFN@yvR1|$! zEqV4rw;u;;DDwr}G*qI_whf*{S7$(J07K}c@?iG7eYQnw80~NKe%g#*2Hh28+J<-L zRc*-Bria>uPMXgP=Nwa|&+hZMpW_Td0jgrXTM0|h&Az|t+WCQ2#}C*@D(B8IULHxh z9l3%-yVkL>x!K&*)LGWKvr|{Uaf05X%!A^a2U5}U6c+qy@k({BZG0-*&_5moV~p0> z{piiQ_+W;JSe~p|`|6NszS(r{a{zmYv3~@o_f_h#11E`VCEWfpIf461RaEUvm*IiT zL2JD{!>d<)3#0eS^|*35Z+Dp+B635XOcw$AK0?Gx%W^|SUXsL)N?!a>|eZ8G#jn}Q7p zbB1V3^taaD|L1BKS?64<&oSt+9q8{jQg!zVm31BIh&+9fUrj>% zkRIjsMGZrZl%1T14WQ>syvDHyJP@>Kd=&6aHDuMwl_fz^){&l+>ET+zJ-?rPWV=oN zX2V2XC(Z8Pf43etqqot_iIG^(HQWUeS`}lu9KKZ|Y zlXqcmJdz0ZZSCxS{1eg}!Xp1G*e6Wt2eRm1B2xIDghg`f+mO+6Nm*-BF4R6r;8TPX zQw~hpcPV}-Fsa$^*c6@iI9bL;`iNHi$R(f-G<1J(#KmKT{us^c#%`JKT8MZ29Lh ztrW+hrq)EINOLKxnLQzS*S-o&x*Xk4^jO_7 z&@VmtyjSS5MsmnSdgNIfULKmC0gVto-w>w&Q_aH0IW|0Hf?rw|7xZbp zXF>xS>Fmvi-aZSysM55Ja~mtq*-aubX19e62(ZB~JG=MMk~6yp>x9OC$g`v=J+3d! z115)$9w@aRaXS;cZdkl9Uju1&{rB%Q&h3e2UOwRb5E1XCrGp@Z)MdrY=z?pXYJA=| zwUA>jLth3o%WQSDafIcL3j|zBP^Y!fR+sW9$v;GN(_0##+`89vX_)@+RurZ%_t+4}JaAk`J&Du~680MTP)~_!3LeeA+$k)oVBeiay-=v4`-WyY!+4 zK3NhYx8eJzVeh}-H2-~b>hnMU_`@W3MU>o9D;-|?&d_m0zzO}FjOe7Kq@_e-8J*=H z>iT~_nc-Ew?L}J7%y;TP+Zi}B+;lEnZ2#x8>VmSeq}sqF*O?J!_i!k`k4m0olI!dc z#5eD=7c9(BZjIY?-j{YTkusnEQ7-b&ky*E)@?h11LZGY~WjsRCzuPzbAkn z!EBfvYRj|^*Oz~IcC+9yGNbdSmtW}z_Vw#8}a#TX%=r*@=D65uQs-{8RNSW@y}1ztzUt*lXLrcoq>zdc5K%mqS4GjDcz7V z!O>F+>_pyhXDaB?5sW7+Os&abXacZuOJLwyQi9ZLDmU!*xD_GWj0*?9IYDM zE}szTY!%jb8Ei5V7q6WXQq<9rE;54U@=&@6OfKhU-J^I-fdn0k>89z)EeCSR3#W3o z5--rbPtL6)>$>7&RvtQ?F82C`ze=d=;8de+GqW+|oHo1sCXq`TtQ~`cuddiHS*eSw z?xJ_Y@t)@z^7OoSd{I}&9)Q8Fu6b*OhoskSKbG&vyj#IlAAmsV_5$F>`L)~W6LsdA zj~AA+;Rjh*7S5%t#~T-4Qh)Io)SFT8=g8}J$aYuV#p;6}X^?&G;%u^>7}H+?5%B6Q zLbq*`)^l(;56;{(|8*5^~XJ)Yg4n*a&2eZz<6_U7%g@7Bi1V1e`33* zT6jKKYz{~zK+BozEvb#ReIOiq^lWsEod_@O$QqO`%xm(evxl=&3`#Eji zbw7Q@FgrOk&!InLy&_@$j+C;gxl|sG3tplY?-E}e`x|dN!ty^x4F2)RB_RC2{ZG+y zaCCGu5iN&>h5i2*Eyw@<`SDf}*V!HC<6fGFQaPcfuoqpik!ADA0JUhVgMK78yaViJ zKOm)@M~c)H){>e3&))f^z~6`5aKCUz*I;uV^#f;Je6oH{dGo>~thkZbaKt?pq@zk; z+M*9P@3SYP|Ee{ay7&=gJ{+t&Br+?H?82PgaO1bkQ`6pqmq7BDnM^q%7CiHI{Yi1k z(y5eIIwES(mR46ONriqGy4iEmR8au+njJKLiOvkTu5;U@a}_61l|bY~6vAys^2J0U z8qwHR>ChH(WrNq39JeDNpn7jwIws=#Sv*$<2|F$nJYI6w>N(d}1Wmc4$-I*zy!to;gff(d#K%59 zRUw^)g7B8`+j_1uVU$~Or?a8O+1}1hW{RL{f@)rZZfjkp>SB)|v)h7X=bpGhCTI=?|L-m^#FPh$8Sg!f(anOO$ z>O#xxWWAkEP6sY$fIq>;H4z*0h={Y(9w6Ke(G~W13#Mtx&Q=e0Ugb?}Yu12{K8ra& z!<5>-&rfB$+zdJdfbw%lU9&MUF@g6EP|U4Q{GP2Q!lTfy&b$2g-;rda1((>o1{B`4Js@fpbnd?DBld&RKTz6VsE@Zrzs^6r$pzFQ>ji2^Sb1 zawhz!!M%Hao8pp^LQ9s8V&gg3P6P~b8sndel;Z~mG(YI>kpU!W|{)w(&Q7<)c}}m(WC>incw5384LDj+cec*7wI}$crmuhtUe>P z&o35Lc<$;^wz`$)#8u*+^Tul?iJ8yj|D6H=v9f6ORI${~{!v@IQ64G_u@@V5Zsb$3 zM0ElzOKFE;j)}ex|MxY536p$wCTgnq{W7p~B*_A$2_cvJdgMZ7h zC@SsZS9{>$(Ouyed*L$`L0hy-J>1V0p(`^3(_E;RSn1_W{hDz-wTYJSf8g&+cqZ}# zkh6(Rm3!tIaT5q~+b19J&;O_n@K5Ywhz84-$-nhiegA%83C7wb?F&H!vs_QIskyoN z5(d7E`O$YT@qYgu#PmPx-*fU@PeKEvWf@tEP0^4J*1363Y5(J=Pr)KKh1_06LXf}E zs{7X$zpCgk8h5g9yIh_6p}6C-S%XcT263g;@vLWv6f1&e!&L@1W^b3Z8LA(ahCHc} zq2~U}X%^9c16=V&{eYC~lev#vzkDTD47uzt>8sU98_WEKqdzWrXA2^>dNm(?@7_t0 zd5c}?mzCa}jKGm-xa-)PseK~)SthN?Mw2l?$-0_5uRZkkwucZztdz-pdNs}w); z39Z^}2Of;lzr3sDgKM->&8L5^GE~q$!+T%;AgR+IjtC9dm+R*KmErq4BETgj7|b?# zzv7Hv@{h8lU7~aIwBfcqa`hbsvY$tQxIrM1wkU4@;%(y`GB^6$Dr^tvKee~}mM24C zXj!p7ehG#%^H#)P$X+Yo|BaM={8V(2PeoKh;!KN`wRK8#G|*MA_#RqWkUVzw!*b!J zH`GU-|K`mbVC>XP<-RpZe9pBu%j3^~qyc&G%i!YRlr*n#svEF}kM?=`RMxKhSB~vg z4|cIDt1jjep}}5}%)1=}nOyMNnJp3y$u*BLO*syhP2EaVce{x%<3*1*TT=UHHyD*# zw`SdFzf%Y+tq~b(vy^4r{V{Tna;sk*E5k`&Me6(!_weDFhNIH~jvLW>wqe7n&4Pz+ z(_f6}k);k*WEyoHh4==Khj->Q0?SpKic`>9A7@5HSVKG*20RaH&BfK~&!w*a^hHtn z?a0ofZidXeCv~zdGD-;o;MC8REG(`)_qT%X6tU+tmw5)!5wpwnjONvcPoB_+x^AHS z5SgIl>7AL0KLb{vJ4B0_e&5Xu)15|ugsSArqt0)Fm@Jaog1W@CiJuOV8e3W}wA&CJw{j30Iv@QxouM!zLbg+!=3M=ucs zgM0DcC6shY$?E@$IR7_tipFm~+=uj**Mvj0^Y03x!!G2xz^$3+#;#q;|CcfD5md|n z&9wRV3CX|j@cL&=XOD;ihEBPhnQQu3G9V!@)1vtr(g&!EZWLHZ?yEAe8~)x5Fw8zX z7ch~}iNagqt8A-Bqr>DMW#NI<6*Oeh(%jeC@)N##>570Beq$Lx7ejlG& z==W6jX8HT8OrwrrAiEY2)MQvSULR%LOQGg(mOth^KZKCUQaiKUzlpKu@Zm31=HuYt z{^IXl0$PstNf*hqQVM7~nyZIe3L2`atjAv84K^UIDfZQiKK9_+T(#W2zy4~<7)29d zw~aHTvOtkPxKTj46D`m34~vuo9ZjfNNL<;D9ouq18~uWvsDghj|Jfh`4tDnUMuRgS z@2!A~y^`K{R5~a@E8BrjYY76e1_+H9K_|;996SFWbzmhA8ZtL*=E#JK&5UKU;Y6X^ z$>qy=6udf%G;q&J8M?a0{jz5lX1V_c;0wzW|A8J^*Py;~_RN`E2Ye0ct{WS3aczZU zSl{K>$G-;782l?t&=eXUx8d=~%9{D#=)8p*K=?Cf_FagfzdyK#Jle8f{2#}^>Gylk zM_}(>95k~PExVK`CLvKCPGz8}m`lpiwB~)pvxsZG zKY}hZ3*C>}4k<)Y~18RsK-* zoG4aP6fR%>8GmRHgs`pKH(bAVO+Q67^zI@Pd|c+rHOM~q3)}u1sDd#phWV$ycP{Ih zG_EQ5WbhFC^`EovYjd?&B5viHGrUW#zD9A?Rx2WS@b&v|)II-iT=5?SBJjYjUB_z= z?OH@H>EB#GLl$JJmsJ|ZL@X^Wm#Q4g+U5S|O$7eS??FX_a@$WQlzKM6-n2fgqDWaw zEFy9A(bK2s%QH>NU3=Dlzd_11ze9tP#XUB4kpLi9ZxyjWpnkopmn@;lLp*pzvAXVc zNHq=CTW^=Ndj}CDKvA5Boqj>dML9?ycx4gqLCanPl(k`nhxE z?I)Idumjm!!N#?I+ort%XT>Q?U{K!K4%U+v7DMn3uCoK*(_OXtCVIA%hwAMjGp4yD zjZ|-YK?@QI0+JG2q3o9q4p}O%B+P2wcAy3u;ImWK=EtWNdUc_Uy3mR%#3AkHyPwj4 zZbJaz8u6DqsL`1j0}Tnbi@ixtL=NotQb1SH>!d~I0|n5B?<>5?Z;eulMB^FyhYU&A z@cYLl1Ld#!mUVd$1hM!nw&Ll3hYnAY2DvtPMcr6%z%!F*1DWidh=l!ORUHSa^=T&9 z{O6MYIgQBJjg4cYqo=$3FXCzZg+%lVHV|B$7cYBD#|9YY^q&{NuUN||1&Mx}oP5Ws zkpdUHvTjbi{hjT+_dh`V?q}46&sd7Uefs&p*o`ZJb#y%cCmG1qd1q4Mtplp8_W8H3 zyZnr+ZdQd_TUoV@`m_Zb5DAIz$*5zBPM_W%#KG-v$$a2v8x5`Y>A_~5-Omge9e<(^ z5FM3=ueJP&QQ_PyqMxXe5O{LMTHQ1|X9WdAJ}n~F=-$4cd!Hzxh-<@C)Z^q?xy%Ov zte|vY0P=g6`2rg{F92}@ZPds-eE8?LuSK-E`Aq_sMT2qiH;7>A{yw7HPNR$=C&Xu0tRv9AA{8?)Y1@@aU zF|V@d8#i;*6_^K|dwakOM%PRuMG~+W`l8TfNT6Gzb>=!*CN~Y21j8pFKCx6TADfuS zMg5iijqa7$a=8t}kPP_=9H={pb?w^Fqc@2Bf#6+*ltDfz{_2TuPxkSSm*`Lg4;--g zUeoi!d-bh zqEUO6ENB|n@BfuA&d(lQ1Ev@^h*Q>-#Ydas#U4BDqVq&H*(9PIW?eu1%hPo%a>^Gv1-fDK{WzQAn@?7eob-({}1e_vng#LS%KkVZfNLF zzi!=@59(Sm)`s+U62cE4cC!5WCwh1HvQ@k|*ad3w=Lc-r85zgG&G~S5Ip~t+Mn(bi zm1x?agQo#!dlc$&qym<8+b@IpeCN02VDQ@G$G|Jw#=()3o6D$nhy#5x3`stSQq#>3 z5)%`@;_c1NUkMB1ECgSzi`T@Vcp~U*1^}Q`R*ovyqvLv!HtgNWVUG_$>n4q2}xqaNPjB^w;%)QV;H)^|foy zJp0U6@}L(H9~-hcIXR*6!aqLy3AZkIZRD?id*O3koqflr`*5qJ4j_=V6}W1FJ`N@^ z!J41w_F}(|)2EqmNpqUkU_j2n4tAYYb#??_FgZCHWQ0R|P4PJh&(l*=;5NTXNHB+x z7D27ko(50xrG%+EXi})v7m}H6VD}F-rcd!f9~r{Ne?NCI3*r zz-72Gy$`HJAfjkiQSEVouueO0@54Jmft|zxqT)nNo(0K=!eqD-T(*W3xUNrV2Q*qB zjM38_t{}>Sh*JkLnR1XY&es-s`uI0)g02S3V{@wkTLPptN1)i49-%8~drN%x+J`R8 z&*KbdVF;Kd%i5Rw6S^g=+RNz-tsEP=uMk8O|@Ek#YORYU~FH(X2h6= z`}^b0?<>q}f$_m`3qZUiW&s3M+RVhn1gGfLI{^W8y)YP-O{m&egJ>itN5OF{`uY_D zp}I6(M@L6<^FeT8_l?RrjeRI8Qsi4;((1*6(b(mN)d#694+Ls#U!2$BQAm(GK~KSe zyx21^kc$ZeZPuGxPM>wWRpY9qWoTzT6qS-d>_!hD_=)DNxtY217pXLyz{I4ajSu8- zOAZdUTV6M;ig9H~Bh(kbLk9z&3F7QY&o$_}xtwcfinoQ?CnI0STEmu<+VkqCuy=y# zrxciv4eqgWBV;2outQ)}AVZ>W@20}%=ZqP2ae>)g#0G;)f!Ww7e?U8fV2^|NB?pca z*smyTfCC_5v4tI;ZnY#h3V8#-pJh%i(9y&*Ve)Zz?0}`BQvvjbaItWpHfg@<=LS2^Zd^H=d<_;k~Y}LY@8Csm6e3d z1(`wXgVE58*axj<3qrIGuuNM=BdC+>A)&{W#2rO_) zS{ybi^@oJSNL#@I)*kN*KX=JJa-%#L&Ibep;?YLql?PctoqVU%xVV18dOhKYOi0Kx z`M%rUz)chWaKrZP$FS_sqySUY74w31UkpxC{=C(F&z?OpGBQ0<42jtLb}QFF?hWSw zCkAGPIYOvQ48e?INa3Q_{(|TLL@4qL3P4)VwrT~z*8k*9=$(vp7Cpw+*t%am3NI}G zkYJz6*%^eAtDX}IC?GL{0Q^qLj}N<^yf{sG6p)G&w3E*Ti^VsCn2q_9e~7)eS@6<1 z`@|3UfYbp^k%C7n%{heY#A{uC|BxuCFIrgr-vvUGbQ+xYWqX&2@ zLWCquZX8<;t19C<1AByzVa&Fh8y%;>FQuZnInS`CNO9 z|6JvtB%m)40Mi9tbY6UyqyVw7Xuut%F!f%6@l)aq? zvYHLQBjgnjn2UJ4Co@U#hRXi?)`N;ZIJnUMaXAopYc-a=9@TYqiw>L7ui?dC9X0y% zz2SfwFO)9#E(byO6MEi(m~W6erKrdYKu&f`#Iaqk%k@lOAIoCjxik4_@#_P7N|fH9 z^#}-B0~oZh$X|w0(J19oBK65H(!;FPG=44)4qjUBc1#Ilu88okfSJ`-yfytORpxZ@92ONInCLQzZ!9Fz;JShto z7ne~PUJ?uPJmlALbUpL=*+UWnC(x_8qN1WUtL=xQuwMz%FT&DblyJo<+L@*0<%yUB zIf_1v+KNk;F5%!`!liZJQ`6GU+SEAIwRM1S#`6Uat(ksGGw*g%Vj^a`3#0Aroq+*C z?P71&GY@uD9F%6p(3i){F~~@QN=si^&?vGA1Q8cUif|@r*|7mtrXY1o!?_`DB ze#Z`>pfWhAD0D3OXf0Oa7$Kl?gMWs+O>9G<_7!NeV0*Yu{ra9_Ag5^#iLam_tw%S` zfWdqlPXcBkpe!B&p2Nd)Nm(865S5Y=!ES=Jg)K%_afL?##BFYvJNWM}{gR3^Gcy=H zu!Yj>A@DoNZd|qMWC2ZRxIj;|z$FvARQ@3fr-(pF(vDYxf8S^@Gz9*3CwaMF%?HFq zm6aoS@OZU;ya$q-@NQARz`#rJ`e4gxoY*alYIv2}k@^;}|^&J@JuU~V+4T=z4 zFzP<<_2|+3$gXF=@ain%G_;Y%Kpe)y)3XhJ2I?QEbWnE|o+$`FChMzz?=eXoUK_a$ z0xRbD|FYps!Q%eu~3AnjERJ9Y*C*l~nK8eO~g9ojMoY@nYri6|LS z5+D%zyKA4}G;$qB!{~m^S(aQocF4HT+hGqQMp1hq`Vo9az+DiTKw1)e{O<3iGzo~R z6p$8kv$JQjl_WTb#A13{50i^8;NYPqHPKpxiU4*SEMV-hL$%1!mOfK5*?{MOk$|Yo z*!Z|oA>#`aNs@=;cz8OnWlNh;oQC=a1SUe|$P5Ib&>%VFFm##ci=)UzV%5zXl98xY zeEZgiEC{Mi=b!#KOPZ`p(qe@^RmyUfbr2?^!Ms55CI=%l%RX_PP7Z#%6vFW6!$YM%f6if#K=||gT6MaxeJI2(FhjtBC-LRdt$1K#FA-U!ylDqOAPqi; z4vCwSa*#1DwS9wTz5oGqh#|(_e{{T)s?=oWyOIYf?a;4ZP6&Y@w*+sA3+aH%xxoW? z^`(=7Fhrzc2zlC4Z=A-)Lg;~X4kCs7xsxuZcH&hvs9~WYA-Fs5@fM-@rm!Dxk*Xu3 zbc9p7@ogQHn6MCVxU}YHvq);ae0{q~wC9B~5GgBECbMi}lS)Pk$wD47HT9Y?&VjnxL$JbBx8z#Dx3#)gnH4 z9@5ya0|mDiP>jP#^xt^lhf<7fk8|z(%rML-e!LC2>mkb~iK1bYQaEE%QVd|=F+PAF zJ;-!KMQ_0i-@JLV^q?qUTx@k2=sLgzIZ|BT=j7-CGeAaQ+VVbCR~Yv%Qs5vEPJ`~VM23HL`0;{KDmZ(t-(|5@Wf+P7NCz! z3{A?jN=xV2ag>GLT(M#W+)s15u}^$7oCm@nh<=zry5s)+A`IVQq@@u@7MK;{fGQ|G zA>jkKjBSgYo;bz0@5*N2BTiGmWE44Lh0>+NP0yd8&H+WHy8OS8OCrHG#=+!pS!qTf zaE6G8k)bQCNPVlD=U_?9^ra;A3JyYL7a0dao|E$OXp;{K3DM3liFnyP2FwO9QsBM| z@|m->P{)DB2&NG7N51TxFfT+9hI4Q>8sLptZH)Is=L&SMLT?D-fvK>i$m5W`Ksv68 zS2++YYYNN+tT%E1$daOHng|10_2jg6fr~vxLYle)xf%755ETpHVRG6f$lW)L7Bep3 zdo^`+!*vNk@wnOzTap+G4nb+E5|p{{nm{|AczK}=3{AdMfaxH_1f!~rp~4mjhQCJ+ zhxt@}C0X&|!&RIXEI#2|aG1#%QHeXZjZi66YAFSd}rGUEjvUn z$DiVA#Q{jRnVAh^Ug^OfeE&3>uGKv%Elo%QVkYson4~0@aHp6dt0}V8WhKR(3tz}W zS7bz42@w|hNn>Hym*eYj1HftR{UQX^yeoG8&m|~ynViR-HUB42x>N)%wqpG+eerMZ z$@wL%-etvBhd=o~{m-vs!3z>C7(s{-f+=QXG8t9LaszA6=j;@W$2ZjD{(UIf96%Td zp;Y*AB(1PvhKzXeb^r9}h42!!ga1hz`@gMn{Re3udm2V2R-)HPETXIu0ErB*K8QaS^gq zi7nEszKT~`KtW-8;s=>E(aO*;gw<0%Y1w#ci(3RV?h)Y-xgkOVK^-`cR3t=@(n0{S zm(+>i6&CG#yk-Kjg@i%3&z@`X=JpN_swb~Cs>#cL!YM(#u^Gx*CMK6-&ILo?jF*-a ze-2t&z(|2Moi*wkXjAUjl2uov$)9^EvA?K6I6!0^zXxl)kr@Z@pa+p=KuTslkVG(r4T{=Na;Ntq^1vZhTtDwai0eoa0taZ zOfY5f#$JzGj*dihgZ#P2CMUNo{!&sC0{^M_;h?~1eG|y|0dLwN>vmXHR-bh}H#aWg z=)eMy7Yc|>2-6Ppq%T$gT&JX@WWV03vK3EJd&Yd@2&p1R@{wcAVJMjH8pqAl!XRHE z2#1x2N;|?M1Js@!AIFqEiV-aJlC*Bq#j*ixAz;ah3PRlpE+Eb=86AU=7Q4@mSpm$? zq+Oq535`grG_PR(3h8BW__P1lFMnY@Z5oBz+^mzchX&UO>M=GKvPW1cmup8Y$WaORDC_=Xk9^3^miX4*;0nKBy>sX>}WQ5p1R8#{HY#=aS z9JvmEFp96;y7ewL))-;xeRf!K=$74reMdG!Y%Qce1Q_9X6-9?mtS9`dSa2e;^7!hI zTEoz-O(iA$2paG+z^-SI3qdOi5xSpYFv2o08}L<#a0=lzHQZH0FtzA3U@J%<-C^P3 z+OgvZAKugU7=E`k-x+S(&=7h{{E)bMvY>KP1ub%rw%T#{)@!6u?4Q1VjX{Wo$Q1b( zy0!7#fZv}f>`u+hv_uY%ZrK4ZB5s6BM6?bK7+5$Y(+K%Mne7@L*2Ah$Nejcx_Pit! z;IC1z@}*j12u$!&oU(3CH!UR?0Z?Z^ypH@7H};`@1COPe!!-w4EC3jfyu5DP+Csk& z{Ts+7AgG9C!3NjMwZjc)XA6xX9gBPgH4jBaMM8<}lgCQLluIE743va*+p&#V!p5lH zu(7i4)630J*u7pJ%3wVx8q!J%$V>(XgY7$!8CYUS$Z0}v0XKXH2VcPmaSJX)qPK0{ z48v{)l^;k>AkCkJgceOv4%ZhUt+%G)%(cPQV1f~T$?6U>!8SXCeN^{#eJx z2C=G)^mJPwB${R1M{J_-4Ww6tHPMf;{|3n7jpqVgB6KoLZet_E*+FwF750iMjI~(K z&(H6rpq{LI1dubNV#4$h3$ELF?Ctq|s1vm0+MCy{!~N0^fq7v0q0oOD&?RMFp% zEfpf(F@N$x0tU$MnP_#?DX_O;dF^27u$i#fT0)bE4gjgxK~fDlA2dc01Z-fs z20$EarZr<1V?kfN;JExl99%&P_bxwY7nefk$(#65K-r5hTQC9v!!Xf=VC~^s?Lhxf zKjYl6Aq~+Fl2X(Iu$rBK(v(yCAo_^`5J3v`g&>)VB)oKnUoC7*+jt+MV4OFY%=dYD z2$)y#U<-qi|JPrC5f2M!7H9=<4?U>u;&NsH#bi_&0BS>g2ar!}LP9ejU7!j?ri*k$ zJx=4uVf<@}*@U08c(85<1oAd<6T*z>_l1sK*mZ4o`n^igD`T0xWV^~|@q5<5V)>of zi*14kaRdqI^V68ylomgK6qx`R?dJ;EUyC!Yqa%%-5ZnVrvE4yskXz>SxMgpzLf^sb zjQPVm+1S{e&EF`Iia=Z7+vm;u&bPxnV^`u#05OCt9A0nDs#Ryx?$x4z4H;b+3P@*` ztcffCn%2JB(#S|N`fLC|KB5Sd*VR4>I2nlok4Jj?B>X(Ye?8cdc%d}Jzz)U?1ej~o z01iOMkRy_^m?!8l6CevUo&)4MVNa_^v{`Du8DannH zvP9}5{}6LXa%8K{9*zE6;aTkHQEe?P9Jbj0!>88AX+ai}aJ#V_-Po*9!D)p0u@Mwp zOicmUE+8kt^dM*kp02~nuQ`>Lru-|5<@_xKkDltk|UsT zrj`BKojc0_b|U#Blq7}-)y0ukX@nB@<)6(DeUtmO=6RjIIpcKS_PAAR5V2Gu_mYor z!cm2*@pF5kd1E8m?O{A8vIIn+tB4jA!$9bgiHVAy&w|2e6zNk7>d*)(dOJFZ>$0)W zSb4InTIUDq1bM!2lj+Zp@<78DwTGVL1;9vfVtXQHkf0NK*%cKkv@7cUEjg1&UJ=#B zrKC^^NZ-11`)>2XYGi)-w>tis$lNH8Vt^j5r7%pxFqDZ9|2^(A-?m z&6<0j!u(<) zQT_qABWGTJz$g-UnS#O>M`;OM{t(u^OHy(MvxrP^=TSQo#{$&Rb71Pfu0MwTag;QN5r~zx`r&T?CCH#rZDP!b-Vg&Ah6i&%DvWcB>s_HjDoYfRK0knyI5V4c z{5rsc>GAOkVR@%?bn+1T!hz1t%t%7t&xnCyIEG7v9*}MaUI1_&i+?e@;RqfdWv7jN z=gzbsnS3?>91-7%w03i$IquLK^q+mhsFfhUE$#ZzUj%?$R+fRBK>i`pn5bU0<3Jlw zCd4KOi%mH|0KzHmz2zJ{9)NEUci=-q05FynFff8{B1$2a4%*yf3hAQi zuf!1WIXZ+8%E(6`(Lf()5FidDBp&R*`SI*p1#0T|a1i+u3hJpv%wIzSny0rh<_AT1 zJP?t-dk0T)Z(@qxf9dYIdVr)jRQ&@3P<&TuEot8G_}(HwDC z{k}K95Fa=8#Wn|H(KbL=1?m7#J2KC-()y5 z-(S69x=e&Z-%F1Y&fF-Ij=bk(Ok-o?5!xBh8NF>6dql{z`s~8ZJa+xjL8SRwWK2`F zI$VjmIbz{ed>&D?{eBKsqpL;ot3{vaSmi`(C&kqE$I31CSGl%LvdhPYljfml zyjH5^wZArkuWm~okMokR3g{DMl$p$wl{_{%Es5FQY)o9YX3>oOQ82ndb0f(ba65gX zEH+bANQ<6Q`5pAhD&|Z!#l1>q76q-&=gPvgO2@)<#LPSjuC5E;|sy4n(_slWrRyWim!t3{-X&3a>eQ;-l9J;`d!#UGdN@MOLgY z9Jy)5rx`QW%q;ZNq;mb--@{wnWH_=Z;j%t>?WQh>sZisJjs^W&A8urhI_H zWYf!gnpTCRF{;Rg1N!gw{O(bWoRm6WpN)E!@Xz(!q`8jlf?-Q!fU6x8ZgtK#H559a zI*;&*_Gz^U{ksn-<7dThlRcHvqxs#>m4QIp)L${yL)kfGvdHTD%62K1otGC&C@oh; zusC)FLyrOC+gA6OmX<8mCG0r|9ScS=AczU`uA1}k!KCI4n2JR z-Nda&&>#m#B`+^;>(FcBEIIdu&fXVs`z?9phw{S3VQcc(J#M$U1bzEj6{*D{1A5V+ z2Hi;XN*TsQ=Y`3v{TMJAt zHpk}-^Q<4Dl3lwVONBWO8j$Hfo70mrnES)YxPNC4*pm4REIx}9y_S@tGp$$=W7619 zr$f0TnQE^lcgng>-pKL4JxMWmUnu21H0<2VWyP_5cD;N9y{_N?IeD?^-nT-vcYo|S zarJyWy@0G7zQCLcTRTY&yLUcm*l5(Eg%)b))c4oU*}V?Umf6v@{jgHTcYPL~2j#uUj@`_xZ`*d=`n#bA6gsZjB41=^ll5H>i_JRv8;eO`*H5PAwojvX z+;H_&J`sKJ;0A@rxzH3+sUs=Xv2PhYOZa3ui)XtU<&398ci60BlJ&PjyXl`nZ8dZX z_nQ2^PwvcH^X(La4WsT?ZMu3xsa)?zoL#9IMohV|BaCq7KF;XIa9*cr{%}sGQN@Ke zx6%);!WQ)2SAMSPlA%V)Mi_Vl;e7TYQo7^H-h!g=DQ+GY!b_9_kDZqBL#D%EUFaX(=&rZkjCT^zyo zqhC}#(o-&8l|K-laO%`4Z6B&_l?;~XF>w;VD{gHCVnbM#w#5<0JsySh9+7Z|CR0B( zZVrwsfMi5SpZwXTZizVRRNU zgfMu0UxzbRDmqC25qoC{_15RLTlJkfFvKvJSW*`k^+l!hH1fMs&7^tK+F|Ft@t7Al|uss<$pi3TK9;wz+EIs*{JQ-#{8gD=8HdmQK>Cf_6Bl&f8g8`l9N`A;bzhJC; zX_Xn3JQOrJCmgc9aKdj#tL4nnE;!BBme%qxg*Mb$ zRX9+8(O~g&P~pN>M;UDe*4q*Fl+Fcd7EbCMRc6p&e38W=@V-VkePj=rG_9uQTvySD zp6}7c(!~=J-ainwK3l>kW189J`zSrJ*7V}h_43A{ zO}D>#ehMPxyUS9SzdL-BZ4D!wWIAP8-!!jHy&kWaH|jkr`@YkFl-7G=wpxQkr?h>a zf9BZW^GK>d(0#^8Fl8ZpBte#YR!(dpxM5H9P0}S=FIxj?CYrvQn$eldEYY9Jt#M># zI`_@;`HWb|$T@X4B!%8_&a`ZntfckFhPTZJsy||%$B~)+YioY{#_+i>Ty0~X2F&>e zL=6^e%2nf6Oa*+t|VRa#z=15f9pC zva%Ox;Y^wjncl@^o&SAyF(ElOetM5y1Iz^D=MA4$1VYAP&N2QInJMnWXjx^#0Z;KG zHsUZ2!I-pw$}H!ZD#EcoEjR!DfgHUfd~wJa(GHU|n3PQ#xEOx4`#6k}A}ciSzubBZ zWMq#A-)Dit3haMaJADR-{qxpjg%tDGX}Ko!o^96By^zwdUf!C#(2RP>cpdJLg5|my zr!n8bMt*CW-eU5kYwnR?vZ%@=7j^C{nK?_A8#YyQ#nJjH;9%x{#_D~Cr(jC{#Aj}X zsQj;!-1F5zg;UW|uq?#%3}R)-iw$HLCyKPKA0MsI;J_9g{_N=3*l(gw5K8SBbu^*P z;W%ylD6G#+tk3rT$sBw|)p;jIp*J(ej_S`wXNNA9k{LZ) zt(xtlLO(EX)eAC4`h#Yor5zSOv*|DV2-@Y5c5@)uJ(e*#lJ}JPSgwRVQ21%QrpZ!mz1N6WZ4|>f6hE2o7Ra( zRpr{zznW`qxc)i0bv(S1V{4DT6M~FU?Pf;ltar2FEUKv{DDR=Zp(E7 zAi&4PxE~+cWEcC@46?0u$eW-VjJtMFHg@_d9yX4x^EEB8pHIj6Z6K^#;AV_H z*jn&Qk(Gz%%W6dXZn8e?M@RD~tqP{=k=*M-cw#sky{O}9Wx^*}QG&z%5m_A2qPR2~ z1!ZQTur2h}34j3-(r=5=Lbq4DC8MLk^kr|E{*0Jj14D8_khZ^WOISuZ!t+Sn5Devnx--5}?hlfAS3J+`T z*2SfzXJ4mmFt6@WjrNJ1&fU*M+&Wex-J;p49GyF!w9%!Qd-u27?Fv=x$sbQkW+s`X zrzcsHv-*58-#RZASvS@ca`H^&Y-QHgS1oD|o;GlA=ik;PVlXk17%b1~KF&LOtZOo* z&}hNhfYfx~S=yWZEQiC#%3U6^P92*5)pd_cO-c*XN%_+49LAXI@0UqM_pTk z9T}#H(~;6liJ9WI0jZR{_MzIN?DC@popJcikhxszbOeQbisy#n>SqljZd4}M_-*Pn#a^ev zImOH|$`9r0p}Qv}&Pp^pkL_{k8z>nt?lfF4FZT9gJnzZoi&btV-8Xv&YS+t`v}awq z`eNcr;!mGH($(B&y3>bLjHO2DVUOG`a#?wVn!V6CG43ZMipxJ&^4QI>*fC^ZdY_84 zzvlRkXul;5A68}mgEa#EO_0Nn57@#kA7kp^=m`0d?To~FUOo0d&c@-J=bl;~XSD=D zh(`jfIuLmzhzn|LQpWeDe(0M=$XK)u>)MmzR3sP%i=P5Dkb;68JQ>DSJM|iLOn>9~ z(a->Cz(a#=HT_fQA|%`cs;Q?h{NOyRkFo;!nnF!)7w-yASH9#%XxGx zOWaj5yXJR!6U7)?mE^vTaTyW0;ZG-L`Z;ITI=c5cvg|y-VjpLC$Sk>vyS6oCL_=r) z+uQpk#AZLvDp&j-KBz7v&uYJUs6N|u1s7|Cxa1}TD^Dtao_De^AUyxbqljnes>3FW z583JELc`574eO}JvX+c7DVDbB`>x*vcs7bXoIAKh(ysS_N41lXw(8w$9pt#6*Nt*p z#3mjtMjdr|!!4=xTwQG4GxaYnomuRux*ony=)qx|U05SJd?>3_9Nz6Gw#l2e{@orA z{Z6o!j2Ui^al>BGmwS0_TT_;M@x2F6&RjUlCJQ&Ijbv|HYwK1tmg6G>*YTY z2smiEUWbiDZ}+&`>>%hH;<05y7wW$wL;$drGFLQ++bCNxqlgm5VXsD?h*ypsOW8ce zw_{@1pD?uKxd&t?BGSiqMM71HBqHfIRUsBfR;RF_y(Rl&L&RvTefg1*%X#czc9Auv zkQWDuu;4!XRs*+j7FelE-7ox04B6$qPu;KyPr|)UUHp&kAN%fKGwMF-YA|0DR5%$r z>W0_{9-#oqk8s|Q7}(Knoi_r#Ghr%r!Za8 zSR*tSG!AS*N3yt)-#Tj5$LRg)XQ{It3RSN;7C0uo{C@G?jh2a{eytn+mc51{k(H+z zUHzO@av*@Mk;jj2M0Jn#?L4`)!1+e=S?_ACR(jhF3&Vu_6?=!3?7bcmR%rv0h>aESr z&{?tANw%i+%w8-nckLESzRPYlUzYU2{qE8}SiU?}{j^Pi>N&1!yCeBsymbt?H%P8? z4-7rUYG1(Ec#7J~e<-0^I@Ry>!9mr))O&KTCl|kGSW2(?oMXx1*}=2Elil;$`LhHp zB)ia9B|A5MPvrMcqITQ+OkUZM%^)JN!qQFaJ@?T<@RZ^!o2fH|bH-9+zX|YPMhPs`PWF{i?{#ALUdy z9xyrzteqBa6uD0yB_vz94f87*ui3xVNAorJI=3A@8W4uj^U(W7EB(+|?_hoGP*xnmO zTcSALT&d4%$Yvknd?pgqYgq|52t@wthHr%&ropzb9L zicxpk$`kBQAl`#EP0!e?PqdP>HHCxnxEtEqRQHFTW0hyEnp91E_3&Hkr;tGwZnt-u z`KtS61{2fU5RGXi)Ll<~kFKP>N)DrKtxXm{rK4-U95C*;JG$oNS3U+$W7e?Ov`gpV z|3%n)M>W-aVWTR=(4-2X8+s5hR6&Xosi8?gI!dpCbOb~cLp2m3^rDnR=^g1(1Jab< zL=*(+iZm64JK_C(_pf``mxT+~GUv>hIeT__p1o&#(uunN1{mo5y$1&4uwnTntbjI# zVT0G%+{D0u9=RH?xDDmvQiCYO?*h-_S4zc0a2#2besvYdi32p_Ux40bJ^Tv3B@c$a z@SWB259^8=uvj0xd%4RhK$^v``zHtYF^Q8{4M?6QZ%+Cxb;!scrCNY5J$<_PH70s# z+B@6C;)ZnYDu6F~5Li%*6O`{uN{U6$+BXStS4ZsvumcQ`{_*$wM+efq-nqXAop}bD zt3+5gD*|6-@hZMfvjN*!==Z^B_d~~oi-&PX%p1PSt$_od`ajG*{7$I9U3S8x+hi1R zGhn*^Oi;)o{m=QE3Hwu;mf!e2njm?9mad+Ern>dxLM%Kkcv()dIh)AOeb08-KV%XK zi8B?p``h$M2I=DSl5y#0d2ei4#rx@7TInq4vhO-?j@{X7o3}o^O+|`IIH)M^glt9p z{dRbvpf{|{sD4sjIc|$GmZw^7|*tAlFN?-cx;aVbaHSC-;4x zy1WnV9$K3hHa*(~BlYS#!#f1B9e~X6XghwPV6#dTct=9t0KEk$Tkzs8*L%Eg3hp97 zMdx`Qdx~^+QtG8`p@%(hYmlFmq{!RI+<+Mz=FoBJZu-Z>GKQ!=1=~~@VhlDK8 z%#U*<00$sdf%GcBJ_8)+?cag7wf#Ii09Nj5<{*K_Mh43JyeB)r`R=_AYRNzO42Y9J zDa+mW~-*{|874^EwN?q^|%8wQ@g@#(W@62ZjIn?|WDns2KuN3zA3#lx-qrc>;tUOlb3qi5b=8>^dPoOaeyqvi*HRi?B5K$>-d$TitkBZJOH8^ z51e0Yd-;^{wjQ`%*)jjMGNL&?%(*%Ipf`T;Y~ofbkQ;X=#k8N zN?7CfPhm|<(qU~YYt5o$JumC|#>^|F7lOWCkWg1X_~hN|{EEvCSBJL0vrJD$|Aye9 z4*BQ(W8~d&tfTAMVSmd>tFfG>z#}!$)eW!Q`l7Af>$3;VQAD1(yGP%y%saa@9rDk*Kfacy>0$H zn46il$?fTrS6@D`;C*9LsTvs4KNJ7>n<<4=9vymISw<5h+Z;oV z?%O^h`B!95k8t0FZYn--d@w0)^uDQZNtV|Qz}v@%N`ad?CpPz0N{;mIX78JCT{-!- zNl{V$@nPsm$Q6#}9Sip^>1%gS-o1N4nedl56MVKa>M%TfX%obMFpSdAzRlete?PyL zEsy(`Q&7CW+Rs~DuDk=x+xY~O7o#LEmK4L1u%4rZlVeWLh9|y{fYj*TAqUs;M6yRXOiqF33PS1*a0HZ ziq?Z!gV1#&gP?w+Pk>Kr`KJ6`HDSRTU{MPX0lCup2f#3EI(KJ53)Fx6Q`dn@@aWW7 zo8F+o@417$3INXXruAZyMVEf@dvL<^Zs>A~DF`KZGv1R@u=VZ-kEyd+`vJks1@z^V zb)fYI1HJOzE%$AG%m)xLj*_BUQiOkZH*@MQNmBroKbPr6rMp&IGyDgjLw^c*|E&sM z33_+;c=;{~5GRENp!($*LleNxK+ED2Ak8}f88QO}G+->5-=$&4-@=BSYQRpbxzh0Y z9B~>1=+!0^od9y3&XnB;E7W%QDFhIV32)bLS2O{p48$l2JdaiZFBldC+>(C)b85If zd_3r^vKz_inck=k^1kVn5LNQ;0Y^;YT>uSp7rk4jB=>jq@$Bl|M@JUP>o3CI zDQfN!IL#}8|{ z{_ZZQ$Yx39?h-6d{)%6Bv})T7mqqi;7Crd$=kITWt>e#Ayuk}$`hLd8D;*;=!<$b^ z>X*+O%)I(5fTDZ!S$%S)X6|m#e7MXS)kt|rThE8KRZ<>w&SM-xM{|K+~YUhKE>h}udqz_s1zQ`w(se71l%z`uFzFblsG z-`t;YJ59smq!yJdHn1UA3$FEa-$CRh1qAT7XpBs zCjC<){T;uoc0~`NLpr|}d~7WKJKG-oi4gzq_&UHwyQ=}{5_)pL4S z7Wj@$O!$9YgwDj*02@gEE&DgF$CKhr@A>~)Tm1jt=Kn%(Q=FD9fU6PW9#x!YQWfyG zk_4X3#|nSR9>DbYU*W2#v`7wu(eN<}^EvUws}HITI%NM4Nu^@*=401{z`~32n7clR zJ+(*(vt1vTDJpVL&1;NbnKjY;v?j`6G(|^d-!ffs@eD$?3-Ota1+eoBg#00~zq+F}jzV4ii_&}xDKs^MHVTfh`Gdd&x<%Z3ffov9} zfKpM2ZP6mzqwvL+VP* zO`fZv@7JC_JQj+d*M(HfL$BqcT^HOvtLUc-7`RjS&`1VM{Z1jdolFn;W+iNHY4>b?b zutdSA74crG{m`l>6r?43TODaqY$A9Ld;%AK9 zxn;P?Nq5xS(ji%k(c~R+tFDJ}RHQfh^*@v68}zo1V;XxIsSl5sCSChjog9n~#i6xX zb}togbMIZCjpD~YsP*~Lf*&|TzhqM=z^=pbq$a=1d}7OwRR+9B`>HI`fnkr6*}$$G zhvVhHiD@TA3cIV;-fuYgAu$O_ihMJzr#P&Zjqy%;pla+&+!w+dkIAG~8_wc0DcwPF zFL-7{u+MaI81l`F1nZo9zuPF8(e{dJ&$DS)Pj6*jN%CGf|LH@+V!kB0*3`eZl{)~I zJnX>Q>`J;;o*m3rCk>Y*s0}4ot;&z}5Bqjwd88GkCD?6u)q@8PYvBEfi0}6gX$i;Y zV<3V>1okF|FA|Ku=01NJHqOg& zcQZg-EFv*+kS3jNwZ~#M0pI=6J7dY_<`gZrJ*_p}>BVQj#iM)}8*iqot$*6SczKCI zC5THC%ccttjnz8C!9L~NLEbNPsaQqr2e=Sy(^u@So?bb?3zoZ93l;>8Ddyee5W8Oe zEC_bk)#EwNmrV5xHBtD9wzuSq89w<|%77R4KOJLpmbA(`ERl7`wsNFd;7UpX=lsKA66dT zo&@3J-ER5|bH;jq>0yOJsgWi$HEJ?MC=4r^n}0hw&Iny(o?4n38}%IB&k6ma#lgC5 zRTs{1F6Fd!cDgB;H=2>s*5B_VC&#m>*n6deT^XaulCM*eVgm?Ty(yVE1|+-($~@X6 z-wV-GBcI^IC!$esa_8(`th#o#m&C|D)Ae2p3>f%}IpRSCu>Wdk|H27rxa0(mMBak*|0>D0{jQJBUNZ@%?AC(+$A97j+ip_a5GF zSl!}`J&YIFcdf{hBmxznHzp-MCgVT$!CKQmF&L)j3olc?Id(yNu3N6k(FTh%#xUJm z@0N0zobZ9Su4kkOx&IoZ8MTe<@y2x)kyF3_*xFK^@9A1R1>5FXg7omZO1Vq{+0wr9Q)ya;PUsvV#V6ZOGtX;B z+9kx~dp``x(BWcd9&UnfS8F|^0htj(Qv3523Q-Ef@QKEwG(nAP|GX_jj~=0vgnoI` z({kWbJ<|yp${Q?4Sbu?#zvsxB)y4?h@Y`3#NlJF z6D_Nv%Yx4{^RJ~dBn6Jfdnm>)1+Z?`DF@%@)Sj7^AyfzgWGm+Hk6&!bb4*xSlN6X%gQ|o?49J~#?#q4JY^zHX^vbPqh?sn7ajWyqp^yU^$9w5k-_T3GZn3TP@ z3Neu79ev&rso_7R;#*MXANTKbE|K@m$OseLVgos49NK7%yVp}O^r`PGN2sG>l@pDe zCK-PrOA^)YAI&9;nmIYg`)ryJu;Sp;zC#Htve4VLi9mXRBaKD0UN^0_top(^p?KHE z)Ma^nw=U|CT~tzl)2Fh zXChtKH2S^82T=Yjn{k)-ay!-P70NA+n5lXnt3x>K?b{)}i*7fmW;k4ZA;?O5i}Ee`oQS`7bD!yj9{lrxb&)>^=4k34wp(f{Jzdi7Vy|u?$}uvY0BUSmPtauX=N0{SbJTq$JIltMWvJsn=FP z{lwq$r|YfnM3VmKP7xQGWIH{RP?JNo!4JlODpQjL!aw9x+IAr+qy9cD*deSoM4nwj zu#TYBz>>cYnqIGF=K2ig%K7@IH5MhXUMQm{;%s_lg6B5txtx{s?fl4fh>W^2C|RVR z-c*s(Y7@Z~?B@8b()H>$rG?z)c5AD@QY5%$DaSc73e(wMdvB)B^>^_*fxRoW>v!rW zWn+LL=wZ|}L>DC&8?=b*APc|xSu!DM#|H0NyL)VA*+z{F=dIWug+ot2I{AIVy#_&v8t15@?=sHXmfO}?R>TT|Xzsw+pNtz!Jxm{a*? zt%0RpjfA$M8UJn?tB^K4gBK2SRbK3gL{tCg!^iW3y}>AgKUXoc%4-XpUv@qPJ&Sg; zuI>+Da%FL$7S50Q%cgHUr%P+yI-i;1235ZU?auawSv*l)8hqJ=*3yC>YKd=MigU|R zSWJ)Fj3RjW>ZP?gr^jD}=b^Rzr^zdz_Lf({+w2YtOm@+JZha+J&O(dBbcHnD8pks%*yS*U7MY|>nE2_ZWQiMmvya5 zZ!vTF!-XQV#5;W&B{EwKbJ?%#C^3mFd|VZ%qStyB72Hx}Bq@)l0n3;dVY@oD-ePR( z|03v#&BG>f$xhrY6!6ohiGEpCBX06&C#<@d?cd5)IG({ST>1$UF{z1-9?N-o*3fzT zgE&iW)^kVe_^(h%*&eL9zqbkb1qwT% z|JwtMz`OBUL@eg=vbk&HJ2;NPdWloRBm~)z0%9|Y$X(Y=1MaqrRBzgW@qC8&_1RJ| zwK18Yx)^_#9%-Z&B_nT{5I&P3TnCm=`AijVNbk^P5Q%!&LX7pF{L$qjPF-+`Hq-W{ zdnTG1Uds@37RIBYRn%a-+uJYT*oZ)o3h>YLQUd2k{>eAZuB8qw$s*YYEbp(azVd%{ z{R>uDY*xXcd@EN90d4gNn5S0?o4jm~im|nH3Vh6{rS#de%pIZwJ&jtNW}0{bltBW zOF`yX8Xk-iDoI2?Ng&?-3N`aEQ->v;S$oTmUzxNhA=&qwQ2uM{X3ail6G*)q9y>xv zi(?24UUIxsyd$sh)0qn1DI(BAJ@4#@N0R3W3sbgMJ;_Ip{&={#gougIwJ=@h z$RiM5aZmYMy%GO*&)VK|Q?Pu$&|ahv=qpSErW1 zfcN;0!Wk(d%V^3J-PP#+oD)| zY?Y)%@x$*|C!2US;OrJIkCS5?m;rP^UT`!_STDZa0tx>yMsH#iI zo?>RdDtQ>xi0;uC>0AngrWgBJ)^Ko4^x#)$koL#Di`QcZ+>4A8)sWP2Za+J6GEiM+ zW9n8k8wMOE4jy;%>YO)su2sl5Fbt%_@dypGv1mN#K3L~mcYvKdC`@5sNt22$;3lg( zsz0iW|{n4A)YR$-qFR9__6jJ ztPpBS1@IYcl}2Sbx6=1wR0EdDa9^z`7i&CiCI}S^COyj|E=kF`k9sIL{;Tb?8&|lY zuy+)T$Ox(u*+!NoSKw}gPy0vkn+A5FtHC%%l#a2caXvmxi^99RfdQ>%=sbtI8^w>t1@InVi#onF2T@(32&5%WX=tDx1g91KcG6WVb!gbm3S>HwFXCE(-W{_P; z5;BIUyt}0WDMAK>GGP@ob>3vdCD`rGLf)cu?un0h>bw{@7U8+C%AlLN9G%lm6Fwp} zY~~Yr+C=!wV^Wm5t4|H%Pc_ccE`ud2eCp|iW@UQ1G})UIY%NmpukK|AR65$^@aSoi zcX9~EYx}}mbETHlw6!6UGRnI7=~c_)@7te>h)rVK9+t?3m`C!F9&PX{Q1_qUFuf-Z z{8}7si@q&f!q8Xudh0x{rl88!Cu!JQU%k9IS_`Z!Gxrc4;jLSC*+A^(_u;voYnLbU zJM!od>KC{19SQ1C*m!qPzg3soU3m8u;v9Ae})JGuFuw2jU@kRnY@ zed)HzuG(NCTJvh2C=%d>+Oq*xbK1+*jn4NZr~GVndZUJZGjDNFzuAwvxnw)WmQ4e< zn8Bcta9Oc|?LNhgkR0i&ZpsnDxsj5S(N%=Y{DMJV^t0EolAe^0>>Kx_Mc{_rzzG-q zACTz_0GPq%N-LerT)T5ag5owe$G8Rp#)|fc7iZG4{eQ`|n6WKKp1e{%ZZ}&!4_q&8 zrYbej2u3OY`ElA_{Rf?DP=3@VW-MOJTNhw~P}@@fmD*u0IXdp<`(wD|yHn|IWDzLsB8{Q11-ze@WUP+R3Zf;7Y0)gzV=h=i{T`WXR9O#!1ue=U z9x~*_5P=>_((V)(d3blM)~`sVCjmHXAFTTQ?p@W!;B1+AJ+jg+2?``I^~48JoT3QX zIIt^&gzbbHvvq9RyFbv)`C|`0!hHGaM2NRLnXljbbtk>sr~amz*mIOcYL9y^N`6v= z@BHoCHdv8KA2@%ST$fftL4n7Q(Lzk3cKEaue2Jt++RrKa6|3TJ7p#~P@gS^*r6UGDj%u`a-?DH}r3Z*xfx&;~; zB$+>Sk*5Dx;Ys_Mm7Hk%=Lo@X=&l=Z*=s%)kJj5SwIs4_pix2?rSN~3@jw}BvkT_) zh{&nOob%RgMvEyuP{}T7M?rih64GHYOFS5t z{`(ri>VJ30q{(Zwh9wC*a0&geIMXSPX^O9@{$8qjDzZ<&@P7<~FMxvw6&ZVzB!l*D zf>s7l=0!7y;&Z@-wnGx{U3exaQpCdSQmjvpxVch^Q1mD#p4*yy|w?9;d+l@oN4!z_PpGz*&&mg;{&KlqLK zu9n@WFo6)k;LuV|V|1@z<@h>B2n=eE*?I23gq7?_x1bzqGJCPOTY=sD1_Al46!5la8uN5481F|C?Iy zuVN+2E||>I5if-%R zsAA?P@e&9K{GUiSMw@Vzk;&KCZutCXUm*>p(iasWv@#m>EAW}>W=<4Lngr}j>+i7 z7oO`?0Px}RU9XO1Epq6DpsCVOT(@mrVxY4)Bjl+O8l*0VwYTsSV0bmnw?5U#XdQN6 zRtnYS-nW4UmdS0agKKhfQ~M%Cd#b%fa^y=%G!ZOl2s2WqSTS`bS9;2Cq<+vZ#D=VA zIR2B>NTG8x2{}HmXp<3@)~^3<$b1mhzl{plQU#2_96bc8>RGS zl(5*3&?`Tl%2Ox*(z-B@l!?ov{NG-pv%2{>=#@Ev>0ELF@68I_Ej}l0Xry0*mbmmk zs~8I%2`=Jn6Zmmj`^r~Pi)&6UJrGw+dQ zKOI3ydiP^uPdR6^L!cIgxQi8r^nV{_$LEVMt66BW!qsWk%r&|6TDvYe6{OgNgoU0| zem@#&Kg_?yvxR09K9_IgmPsBdKoTSyebUx`H6t`q*SJr{r3)7TP}^bRD-oG^Q7t+cgDH))F+S*Xc1;W^{Ib!*DR%~DE!PU9xM8Q-&03^tz_yDs&9?D~+8`?(~dwb%rz#j4}or^LHt z^yZ|h;We-HE*I7zBb}@>{j2mZc8oq#n`cV_7= z7D0t`t5+Z5w~!_IDQ7-2kPNzXjVJp4aOJ%jLqp*H3nXb>x}J|jJ*|91;a1|Ca5|Dy*H}EiLv{ox7aLV3bunVYMMi|i>)RG_j7W&VxLobz zefQ8JoG~iTB=VNeh5aFvH|!z1;Bvl63D+A_A2m%AFnrI5T-JJ1uFq@PF{x3~JkE@r zTIPw=xXb=-9A*V*$iu(HV-Y+C4H5U-dV&`Hvgz`D@5(tOIJs4-)(RE4)#G`}zUXW9 zo)E5_iKJNpNFio5-m%Ux(ZkM~pXC#&jog|4gdYKeCCNK-vZuBGw58GV6?;|l*qYf88Q&{* zoAf1lst#H5^gyWnkkrsaVjltYEu$!GuAhH?RT{2!3dp5PHw2T>bkypZ_vOVV$dVpu z$UA#DAUm?1bc4r@PfM;=GUyI;ZYdagGTe06OY$onPuKK(T%Dq#9+h+K48^0L@haQsQqbtDSUs>gMvAo!Y*q6?wKG)H_ z220IjzxF@n#!ea=K{OtVF4{1k8pKDJUkP}YsxVZwIoz@`YdmJ7beb0tPhq9~IKqB5 z)u=DCkww-hX*Ttp;mY&VOUe+*Z4$!=F#i8^eB{Q}1sjn)p z0k%wM2b3QfZEq3EEovu*HIpW-)#Y@XX8sooEf)6&Y>5 z{Q4kfq%))Bo~6vKuR`;mh3%!VDtQv&luq!3?$>9}-)1;V-&+_xV|R5O6Gb&CT|RGp zX=?`QtXN1vx-r)k@X&ni&{tBfwE>P;nUyH-dCN`KoF537r>W)fcwNIQKxO`y|IteW zUYh*9DfQMI5hq{?B>j6Lm=Hbd*)~|&Vg*(Y9mSGh7YU00JqF`A0NeAjA~SSiO5=RW zGlp*6ybEqziI76q8}>z)0wA~ySM$HHsQR2Tt&R6;Q~5@4L1I<&+}|DO?^E0EzlhlG0WcmPCya*v)Qc_`h zmF^CGVSCcEZu;lOl~P^gSVrRb_)vXmYw?)X$V9W}zTKNsJmEjudOCir%Ja5eGa3Rg zN36QL>YHR9f?5r2LE_N8)d9~tWr1G-ElnCj9uA-*oo9}7E5)Bga$<|)URx!U(Dg(l zq247o@5?@=Nt}8Odj8e+QguvtmRiAwFbI z=J~Y36vq(x(nN!mvNr0aWd*uMx0$~paOGCuox3yYyRNI2q^E5^M_PM6tR!LFZ?jEr zgZgqXvHm;jT2hi|k!&RztyTN*h~2{QtUVS&h|jT@aV*=anmjltw~pBH2^}Hb!K?|$ zhajDODQDA|IRxSzUHx-y<#YOVD>dpoM+?e0NhCBY12;G2c)(iwNv_~c*E>8+oOI#; zGB3(EtS_5ZZkJoA2#0J?7ri!1Mrt*F{KGDE#2O7^Uq8>?^*usLaEQdz9kZ{gBzgHP z0G#z~s0dzzsF&z-wQQWbmzTr+AOS_BSQ-$Lwn4ov&uG+jyPY;MRaD{j%6!VNw8xW; zG^D;8&?+%c7?Q-ls>|6o#ap!c{QaaEj^?Jw`)W5MMxh7EBiIJ6%;_)SW-7x8+``Ke z0R^NBy{f@7=?h>O=BuE{$v*xO=K0DRhZ=jPr_#)}4Dp`OP6jnk;qebU)19vkz zGcQ{G`11kEQ4ztvY*Wa>TEb`RvvZD22+EwVkUEiCr%0`Dpt#lt%$lcAnJdnX1JClm zJDp~Coh6!$$rQRq#(+K|wv71kajnVPKh&P-FQW%UzSxR&M*n6^;A4~)D7^M*K;1sh z4&~SDxi5TcVBFd=(8Jcy$welO@VhkSy#sa7KqSQ}pXPOrw7v8EBsK~^>fTNJY~`(r zS&X|>F1Lsl7n=y*K^Yed^|N`0#|QS3jfm9gmWaE*`Boa=GM`&Bw`&rjzh=yUg^#?} zkS*RoIEs=ctxpTyg;b$yCc#yR6nx-W_$2)u-9C+SjTu^U*xPbYs5;#vuv@xF!?m^@TcSvRJ&x10!mo5>>rm{ zy$8F}HDe(BJUx6UwZU5@UNcqEAV*SL+~|#-T}G3KTfM4YvPERB&mL|_kFGJ5;A&g| zX3?Zim6u8RpEVE`0h;=ENK5Bg_P+Fo2-_9A#}kPq^!R9tPrE9XezEYg<%1Gcr)CVmF=XicISqOcGd8DxZifPC35xI?_-%{+~dMd;*k4xJ|1Q6OpZs z)kJ$?81gd9%`-jOn>?U$6JlMJe)E$u0|>D(KXK`$rih2Ow zIRfscE64yU&#K?5-5>z*2>5wCZv-MG`K2Mk_uqNn&p%=yQt3g*kt`HM~ob(B8>u`NWJ&&5ANs{omqN0BPN&C<77X$i&8bC`byEM$vdV?rBvrY8hRGU2+&*8>}|)225br?$gSN zl0<}f?Cmlh4xz3xZZ%k1#n286=68G()Popud0_?AE%X!tfr^w zvqwtkk~vEk8TV`BvyJOqmU~>mpnt@x>uVa2{NnL{`(EhNf8Yz7ze=qI|I$L`< z<+G;$^I#UhZ|r^+RKRluUcrD4p&i<8j>1$s1^H?`rNmCw6XYb49pn&FbK33DdR9&h{u%C}andV^kU z5K{kzZp-WUU%X$sN}ph?L3E3A8r@?erZDhnKX4ba9gpo^ZEzMYO|-L;0v99&uV(!utrn#4Q#AC>qWd5P6WUg(pTpMi+PeEnW*WBxTij#u;MeMEDp;`?O zTAzsU6isQk?SDbM{;hhkC@A*45jtG>oFD|o!Uq_pY5#RYE0exwi`9;F@o{vHH6D#t zSst(M&6&%MSxOSaFYWz%Y2|8MJcNofbt^S}M_i`0!)a`tnPOY*yuP_TV{&jP*2Y*+(x>nRKiY`7owI;GOo zNLwr!aGc#cci#VAjS<0R@3cQ7o`0UK;h9=)=3X`*t{4oNNY%4pKt8xL;~r0Y!IP6C zSS5pC9goaTngoWm9J?nG7c*v$%$5`*{%QZ@1-^^nQj^e_>Mu z!asc*Z>}AA*U8IqD2l1r;F@eK&yB95!YPKl>*f_LELt7UKa0f6BggN%J$ibXN1NyN zvt`}p6nBH=pQ%e6ss5cQUs3QS`Pik(J0;^AJ(swsx<+<(tn2=?$sa8CS#DCQ#4zYW zQU%T;`>>qaRwJ@cZaJ9G7ZfPZ+zXM*dLz~=Q^qWz7UXKvbFof%e%l|B7s<|GvgJ7i^h8pyzJeu3C+=_2& z6NlbU$$yF-LSSbCt8fBT?!jNoM&_z~{k$5`_XX$@)D4+>ti+z^*RO$8cM)4%HA!Bs z;+1L*73cd_3$31Ixr|ICenE^vcp6W2rff}?v#$7VMNEJCq)5-dkPja`M_zlsttt$@ zNGK>YmU8W(T%VAwfSEDbGES93{)%p%;de@qaqp=&{Z|iL}Wl&y}6pQ?y)bo+Ele?pvHiyvnW9Ci`F@` zeEt6YK5ar!8*db&<>us&KD@i}tCfai3F9V3K-Ut46T4((vhjM#cSctt%|maxX*!Mq z0AMWSlwSFUC@Mzmgm@3mR!yy1Xqao8Z#`#P3Ecc(ZD~l<-8H6My6k?Oug7D!TDBcLA)_Tw{~YIf*o4C z7l1e2Tb3*BTJwuVHyIlMBtk5V6jG_e1?9hGtM5~nm@bhutDRhslY_?7)v;PBs>0!F zVrpE1amCwLi?rmEo@l7EBpVa`#xSFTvfp2U=jG|OtuZf}iXswuxOo@ul<~Zt(f|qe zy$<>Hu!F3j9SjAU*d#%BQwJY>Ptn=@0)95B*v+M^X*0;Sm60}X{9wC+mUcpuxE|+J zJFSFhz5H%rRM8b?t?ot}26*MmMayDEK#rrADrcZ`VV~PZMm*|f zUrup7V-BmD^H-u!)r+0?-kjJdu(q}|DSseJv`$@?C~9wSjLvs>P}>$dwc38mzh%1g zYCb}jwb{Bi4q0rmTaf-+RZ8xY+1vUpoZfg99qP2W>Xe1U>!UJ2Y0)olN3Mz3ohRI1 zK4}JhaQGA>dF!}OdpOBSuNy|M%ou@V>S4xtT&)QjsNU}>?^w?6^X_V31QlWD`+2G0^1B>b`@0{;G%4rzU-h;5OD#MXXJLI~>&-Rc zzO>wboR@cArzdB~2@Vmp5_@4e8}(W2;NN;*U%gkAkc0Iga zYW(lGqyNBqsbFcyP5j5d^d063yDyLk!_J70jUf&#(`Ikw-nxdGYBtZ!x^+T($)k+!X&&^eZGD2X|DeoUepoxZ_hgQ0XQr64&KBc+OcOcpAu zK2GvZAB-JUBoLD|q){gK$z{*9q*+6gM}qZ`x?^HAS9Tv-SIi;*OsOJWAdg@~%omCM z`Fq${)E>HW?gcokh5@7Zo|m^GF0CN+d~x$(is;3=5Lnv#h88$Ng56*dlh&V$jzxFT z^QBhalgmLhb3`No>zy75wh_7z#L++He?}e19vxOc|7Z6wQMNd?#?iP2!3l3Ly}unQ zQ<}P$d;4|cC$rX$+CL0@E_vl(%cH3mD&n!5P|kVtqHWLd^Pz9W1g__~4<%&d&~THA zgdeTi3W^^m<+wDnLXPCzEfk3k9iBdyLGjpf?3=)o>GoTWF5Q<+|2j!C`2jX>VVR`Nm8JDl<-TDljU9nABj%w(lcVsd?}{TJulA&X*p~ z*-lU^(bM%f>dA`i>*$E2s&d2dloTl=IC3TMKmd-%@x_KCN^f5#>sc)58I4D~R!oTv zFQ}#o;BnMl!O+R%?qsP+f7_WRT#T)Q#~chZ?0ye8|FF>Odw&bVDw`_asvyjGIf{OF zRCCkH2^KM+i!JC&_7SmWoeV+*+v&#A6#b0IJfl@oSF9kC42Yxq1*nrX>e@D*e{L@? zl<5~;QgV?9Xl6~;&G&kU(KH-=O>R#%oGI?mbIomby#Cwwm&hOT7y6fHMz0sK;w@`Oe89C!MuyJ`Ax3Gg+pAZM zSb6RbX_ky&FVGr?CtiFuBwMnZH5uqAUGI_g>`npz{01)m4WiJYxj)2W%n@^8b1&dH z=17l`0)Y7HLJ9^Pw|O3vnlBAyI?XzX4oC<2s*v5}(#$n2pE>{?LgZEIrD->?Iql4U zdD+Tlk~?3I?8O>xHtNsHaAW#fC?hXLyTv{5>Yi)!eONlZ_cVPs*$y#H8;+pr?51C| zYlNrtJB~r^tzVtGE4l42Oe<^EY|XY7q9Ox&du+F_vfQn&fpLq&8zBE0EDp{eXyre@ zoh2P{@LSQUz>u3KxYjJV^bY5Ru1I;wsr$L~;lhT~ucsoTBHu{HhXn8WT~iE;x|~`Z zLO*_C&$PJi5@6(nLoCv9i1>#xQYH&kAp-Hz@$>fPlXh3B0h%jJLUS3AYKGHdX&g_F zv`>Z`lqB-KR$ko#83Sb}!cbDjQ_fH!ia{zdyuKO>@lDO)CD97TpPHq zku5A1gHK9!j`DpMgcg;B!^j1>J>qRwQys;NA}*XgV-rNeqGCm8-Mjwnd{9vcxCD>? z;YC<(w=i9Z&$EmKI7Rse&dnGkN+V@;;X>pvEKo3_pgs?bM*c0_HxOSbng6RU6xTMy$>!rgv-RUe+Ifpf6S z*01qKCeHVrzxOZA{n98f5#td_3osT9;oLV&%Aodx zr6G31-Bk_>;Z=@ngrbeea(8B-Mb13ThyRD8s|;)M@4{0MhtxpSNk}6jN4LsAgi$h( z?n!sIAPf)~DV-7nk#0%J(cK_QcSs9}zR&-?_`Zv$&hMPNPB@nFvda=+ZXYY@&{e~g zzsB0s^@ZtBP1~}7AH(WpXKJIPr|j+NT^)WUAaVU$UTBk9TAHGn9j511Bb-c{Gb){M zKxOgNNE(M3omP5E2xG%hA@0G%9|gJMbvgiXaqg&N_w4F5!ouncV0STZa^OsMxBT%{ zvuwup0p4slK6qjCZl6x2ss3lDPaO?4Muo?`@{XRY`H08Wu=NSGJ}&6$_m3$nvE z>v3Q8(i~Q&rF}C|nWz~7l?k}=gxiDP)h@%>ELIEaANXA2cAyqtMP zvb#u4v>VSDM-&wvBgWf821N(OsTRqH+omO1rxaMv*2VSF8gf~;eq57OC?nXE zgtJGmeNj)EVrHOS!kKD4iEIgd@)84)Ry7-Lh()M4Zx`g73^lF{_CLg|V&ozhBsiv` z2eyDjcu4aY#~aQjo7zkx*WQbPo|;!!V)-O6?9{kFwS@kHBup@;8N4+iLa3|xOD0`6u6pnRm=bWn|=FX@khBjHcy~-))#uuC^@|; z-d+E~5E>^hjHz70p5#w$JMHnkEf7H~f1XB?q>H+JJ8~RJ@xH$B+Y)V^p$=lfQappH z59X%KC1v&fP8fIV7a*0L5Ym|jDCYm$T#opY2x~dH$mMUER&Ew_^CT9Npy-}adks(e zo;v$Qk2B>gi({b~)p73(Sghv5!jsvWSW_lUEJ|)7VckSjcD~Vi=3a{t|E~F1$+3Lm z^^h1)UP*JsK79N@KQSmM!6tzdpQ*H8sNoS@ke?7wv(20H7@t=8PFkli>k+%ojHxhX zRw?wjS9nrWtyYdruVpUBHbvR&`M!gA$xsW37~cJ0q6n90d+bRzF!JcsjW=_EL1|S# zm*ceArMA)W?bx``sq zuzhNjL-MXi0ctTcRb&WTCZ1+-A0a$6CN$tI@m*6Jl8_d~G{WF0BHvl@knkM@qfQ;h zbYGBsJlxFD&M!H(R|q(^3Q@p5+pAe)rHPaDAuFr=O7zUE&X|(L^;$z=cG-txCU&|KG0o;i0l#c=T-6=gC&g<6fBmbW# zY1nS=@Qw03fKdYrlMXn4pW<2&u@{?uGq@XNa`A&d-%G^y{+IO=>s>F3DWLuM_#Z`| zqPL1*{;9^f>f-kyW{)D2Lb-uW{sSskBdtnhp=$iHZc^}H#(C+?wEauLpGYBlEJR{2 zNG|DB_vpda-=Fb+7YWgr*Tu0WBj%9{1Mg z6-Q)LzF@=K7svd+$=~&;bttdDk}lY?T$19AiR7qQkbJ3M7z?1Lm|=ez5YXI=pV&lQ zy>4l(*ZKaYl$0Ys+FH*SOBS3~a4>t2YT3%0dBCK&2!y7RTH2@)cJ4qz($+gFf}|65^rFc_b~-bEv4tMl z@A|p#9Bn7Ohaifd_Sc5I{_lt;6$&`JZb9{PFe98`-BM&~9VuIl)_PjJyRBQfWcDFw z9OI z&r}(RuDg}PucmE^MOW>vGiIdls%ajjB5K}5B0OlTZ1r+o`B^Z6PonXx0*(lkCsW~i zJ6>crY0NLhhw0Je4JopU6*`+oSTB-g z+5j8!u=9-0tN+Lv&0KEqW9Daor?cNH>S;>(-?L>8h~# zEduq{vN^@8pJP#~ty(yJAMre^;IR&-e59V~=t z_S0l7w`ihj`C`UZaey4=)8he4%MklV3uR-q<6^tsc*dgi+|B1OgQB4$xbs?rJ%7} z8&grrsvZ~oxD7f0AUnN5wY)R&+G=;X42<<_th|2p0aCI(Syh^KpyKa8qFm@}`E@5# z5(_vvRq1<>IF#*lhX3^3KZOsfy>ukR++2>jmj(`t)1}8iZZ*sG#SJBFlG8QSV}%`; z`&nG1G5AC*eSgqG5`K{3YVzw(#SfEUmZ%;NtVm0WAb;m|C}pmusLFyz2HCCz-?IRF z?XZuEBj(>9;lIXK-xDt`93YeZ%V6A+%tnm(Q2l#$qc|;hCqEo+;QAY(0(*uQpi{w1o$l}*)r^ZPbG zfY9m9_eT|t5m>!_fqcLkuBoA%fuTg9K5&qRUdL^JN=z4|cWo$*hHxgzhlL*SJY*mX zN{cnsq zeCArYeH9p9FbUv1v6YF^_n-2qQ^I8Y-Q1c`dY%^=)pO$H{R}iue%98HJ%3Fm%N`w=60A*Eq?-*f76n&-%0I*V@qE z9Xa^VgeBT%`N?|fGRs;gx&E5Yx87lkLq)KCU{koqW^U^xnoIp3+CF}|Ex~35M7}ZR zu3gj%4r5|pX@a6EWs1zVqIWlpj_dPF)7{E4n0Yut5;VDGxaAQXFbE!cKc7}dloAL2 z_*6RbwcTs`8?7FXvCMk;Wsyu2d%jq6ybO#L^C8c~E!H`Dh!l-|V$LROJL=aUB{2H$ zMQN)j=`Qlgf@U>TJ;;8*iN0{OZf8=%*jd>(of6koMY>Su+o&de*zBn~Q$Ly}mv|C* zqJVtneGllLg(4wQko~?mA~bfUq{Znh?fJi{@*nqmD8Ni&A1CId>3|7667$9i14EOhJ~0plQo?;!_dBXsJ}tEqTS3XO05K?Np9 z2>>N5grr|Udb*_+iw=#}e?k~vlC=g%GW_7WDTt88@K3v~SXi|Ndz9vLi7rnna~;P= zKWoZ+APNdhq_7 z`cdiK*m?&*b>Gp=knjCdsr;=&=Sxh&F$SJ;ZFpmtS!n4BpiS;!kKWc_J(ifS7uxB( z7sbwxwku#ZVbXz4N}&fSLL<9nW`-;UURB<9tJN!?>h#f=PB(?JRYwyDs4S?MqhR&r zPZyKwmC+ITT_LAZB3h@O2c^*uZa7vllqNovp-XfV7i*ZZJ)#mMyFXF!k-Ayg zIXMprqtJV92(T#@Hbpw&VcAcvHtyGH7l`0m}u2a}mwSVgRH^{9#t1&WAusfAIs(1W# z)o#02zHwLF5}m0WcuHAmU=3>l1WL9;(?!N>U>bM#jm)#()nR3y|H_t{y4JMs{6J-= ztJT;tVBpEQZNT3xk78tc>uwND!QS*FBJ|>fgDObPK>Q6pE}=2ut0kGLhTK-WHv#QK zhr_X}Y5gV!K<>s{KthUCeBu>ft(=GL&m->Gvbrq`5lM?`7kd)9s^>R&^4SdQmYGs; z(6|jGhV8abKot9MYo%vN#PVQGExKM{I_Nx-{@G zr`^#~_g9m;cQ-pbV^D`YsG&6_zki`)M^9LB=!H=`u__xnO`F> zhvZM%jHbTn{~p>FVZr1pv#jeh?#CXyVa)_a>DJYEbpiTcua_&dma)#f0A#2_s+fz3 zMd~sfh%&VF>=;>RyYm|cNK9ipcjGHR7WnVh7zT{XhbpU}5fRo0U(B#7QS+e=X$38p zMe{p68V>C8=aF$YV+7yG%w-V_f?%hox}`g?z)=os$z5x#}Cwn&N(~tBx4A23GPt zxL0i7^vPOn@^5s^a|2$-5eu!_TE{VWZOJ-Bg4S7(_2cMFd)uFOk{W}|oNTrtVVvuR zG4|K$H~}DID}xn46xmf}s9&D>cmu6QPL8jS&xk?&fpo`V!HOte zmKPL<&-%Ufb-o@pn;?OrGhEqlsC)O7!S+HoE!Ro_%Stk=zc{sg>ZZa!{jHG8u!u-8 zgD#m#I?bVAqQOm~wn{YJ#oaJl0joafd03p-Z|dl2WA2EY*=E`{cStjG`SdNHGD*eu z((j=>+bvLAB>>wz)U(~cQ*Km#aacg$>$Rj-&(=LF;65O9ru_cAEZrfXB)}4Q#`xb` z3Jo-Z&(=v_Jzh|r(-KiUJ$O^&N$*0;97g^d=%&klMC( zFm8bz!}#``|9RYDpaG$z?c{o@;;B(8i}S&7KW{5L$%0b!fAb3H?@Yo z{WuiPs2n`pC4+-~%nMFHHC>oP5cK;Zz@-YQ8pdMsL%bY5GRsf4yh_&t8b8P2NomhOlH+sTNW6>=P3eWi)SQ#wu$$`{oH z)W`KblCVtWFET0@NyowIOiQvVUVlN;xFR^54Ld7&W`<7BpGjvM2RJQ2P@OCnG0_JQ zX5DXpFB6^s_+9B`5$Pgad0<$jUInVo7R@CipBTu7IHe zzM|fhm-18@&K?9;G<*0mN2*T3OGVIb!Tr1Dv*h`j*3r4Ya^F6pz8J-L?r}aCP%$jr zl&;p!>)EWz;$Ob2nAeAoG!>CF>a{e%vjWVE;D3*i&6M#A+M^nZi4se*hLW}ey1_*c ztE7*}sV`oKBL3Z>+qVhEg%BU)Vciq#kAo^r zp-HPYA<;$kA6Vu}LeaA8HdB!+YkmGCJSZ3}fuS3UX2QgyxX2?pAd1{#_hVKOtk}Cp zb_0E|?mSR6>te~$ib+uA&Q``{t`vAD3J+4k-=7vcKe&Ueuum=9t@E^-t+^bRr7e^z z6`8dYA#qSzCAAtH_`OL9_Vkq-+UJw;VD_4 ztN|=w4x2fod?ez+B%OUmA#8JIQH!DuB6i+dZ$&65_UY0QwkDvQauSY%S)=vQSL9cx z&Xhf4{901BX%2I${q7=VwvxMp2T#BGX*%N9BQ}Lwj^b_UPwqK0zfnfnjjux!0K;J%dKB0CPTxov+^ef68jLR8H;+5)$;SD|mc%bKKjTx8DB_ zZ;?%9LsZ)L*>1pzuCUtHADE=F{J1Lve)7HQd2%nvC*G+&Rids8+ZRlQAnP29f+Z4Z3qsC*akB;cm9BS(`%~Hb7t<1-{ni{iE+y?~-CRodxl1f%i z?epD6G*lO+Z7Ox@SQpT zH17k6OmOUUA+(Dx5rqupN1H|H|JFE&b!E~K&@g$8Dw@tJ1OT>@-L%^E?fHH(4TrY7 z9(M?QSH;3pmjr(Zs(v*=2<1V64Jnh#F#(?WB$hB-ZogQJY_Ts}yjdgLS^}W47yE*Z zwky`8_+`COBMoFk8s-%g6;50NJz~1KbdjV2l3(I{bad$kaElS&FHPi#evD6nn59|E zW}B_^!CunMNCgO)Dppet{XK_uQypAK*_az=2$3^@l7~v2zel(D2E6QpvU`O##OsO0 zaIF&{SIQ1}N&_*4X%D_6e&U+4>N!yVsb*1XWR;Upx4x($Qp@m6oC;ypOnoqCZ0&Ok zU77N(*?4zLk&ayvpxv=>n>V~t-XA*|Bsih>RYXZA_V$NcPG6>zKbD<3ji+xK2xh5e zjee{omL1l%^ok((LC>^^<4$Hq-Pu=$I{*hJ{olRDwD#DnTiG!dX?sCG&mE+3g_!Z? zqX_Q{tW#eD+CeJb=wDytS>$GfZFKIhcZV%0cfUW~LHtT_el`Us1wbIVLW~W~zJ8di z28H5EEsBna82t(b{ls40AMNeTjg7^!ynk&#V}E|wo{DH}Ir*g;DxbG=-ECtJ2Nx^S z(P^CBmTy#@cnoxGkLIW6JkkqPzMpF$LuHD#qp;C`iuAbf>QZA}tIqAA&bDc)={6eJ z)#UiQ@5U~aY&OP%1f2Q$VqU@v~;}OMZH5M174JRgO@4rB9=Q@0VLrXBOYBFpBZuXV>vG zJ4gTM^?6eYYQc{~6ICs%Q&0O_?Ig^%uDV{uIqN!hHJxi-TxiooS;TjhUV6m%(tdp{ zlb~rk_Df-^vKBF{SQxW84;b&9;{R*CRJXjxdO@_bOMv>bQ&&skg#u?HJnL7g192U1 zP0TMMDcpaFK8mKk5o3c$r*+}{pGyXWiUMbgMp#%o4Ngw5rnXSE-c9I&R-O9Q#QxhK z95{%j!LNL&hopO2i?%3x4Y@R*7F5w*+0<2M1>Q?&H$3GQkgf80Rm^rKW3hfkH#9Cw zN*`Io$e~s2n>K{ReShI4z)hFk70A!;X4&rohUr3*@+y4tvq>=+2-X#kZZF(&1cG%C zZi@v8I}#5N!xYKLxz z3NzII4S}e0*q)hq)hy4Z44fUr1ka%il-=s7blWd&b`LiB$s%5WP3?Bm z&@`|f_hb8wd7k{z>t8(UzfF#p4s(|YGF*#B6L`A7fEs6qX3Sl#@ajyRN)h%^%tEmv z1l2z-tTdx+DC(ApLg?i!Jv}H?O;A#RH9)XOA1YIhFp=9yfhlix7tMzB;NyHlg9(mV zn^XPk1X1$pzt(idg9<2ngbjOC8!oec6mGiMviJDbOl$vWUG3?Vr3g{)g%&0Z>hb$g zQ==4!`wHlm= z)Pmu@)Z&+%ssWrJ4*Oa51L_Mcd(NZ~OxmvUHV_NR8S!-~qf!Sc1DPPl167nYy1TVUP4t!O`E5?L%WY@JdW$Xwx)FVUrV zb@j4UTlImIU%5K(h40W&YR}h^2%Aag5>0?suQi|gviaKQmv1ig1TW7}su^Jlb!Orh z$Jb_3j5Us%qg0Qmz+-fULzN5BHf@b*I^ibsN}h(ELSL-s5%|HMsYm z;!38|WILzi1=HN&C=#Z$GFlp3=rebK@8nb!d&I1W061Ns9Z53Rugg@F3~64RiUasU z1H5tymf5H*G*zBJ90W+373E|!r}Tuh>_l4NqzKIGpxjkzC~1$gpta}3Ro#S0y)}(@ zbDRQ9Sc??UXY8E$P+uW_p^Z09Eu06D=6vf*aH)vedfP?Qk)+#9)m)DHid?!SNY6X0aScpO^rMnsAirBWS|U*daYzn*I!1$pUM=0`mnQPFuIr(Q~8uu~*0RVj682&f$cg zucPkqkXx??`&dufXfASKSH-YsK5Q^+s_flB6jDyc05+%IwhBS#Tl?&e>w<^HlgCe5 zP*CA6P3?tOx9^wO0-~}W4~Zrb05*9!lg&Y5$}Q!zq}|zh;bSBW&STxMYP=(`0i@vM zD^q2kz&f6?PgWgAFUY`zRia$QcinIaRE9tmWOPg8_yCzwM%Jv zNX?M3eA6Sx%je2VL+vrRT>TH9A!*K2`<1GE<%7Ol#eGOg#rVL=jCE`!x60jg{RLSd zH$j^3Lx;)X!F-k>TaPz?v$f}OVN8kC(V$tHg?Cm2&&?YrHiOSH85Cy8B)!n5gy?2} zTbd+u-Fik^xXw)Q*gW`v>wVQNhST=iME7sh7)W57D~Y;L^Qk21LrD7vu^LTh_Vd*^ z!>=Oul!T#)x8#$Rhm|cAGrTyBQ^G&yPcf9F9{4TifaOylTO}>^?X3PS5|&sQhvbqt~2!|qS(ej3CR6Geh`R;;NF9YjIJifyr4_(FB9**{vMsq-p`-VLRuP> zvO16K@8w1V0HX+gsN%NW)cr@(Z;t3;0>mN6p}S9h*D-o27GM(k zzVB^rKFaE=l4^~Z?oNk)?_u-eT%)09g>+7Ds=6yPN$cK7pJ4}1@y zKnMMrA|l0aN$qN^$i^a5Z-IW1EX-mwGi-U)>(h6NBOiv2klaTQw|d=scMJVxsxOrO z>F3p#Asq_Uehcka{+=Sz6u!JU)RaN+AaMI~;$NF?jW1q3IW()BigM347;Yv{|Co7> zzS2zsRXbQ*JRrSLX(*GdExszNKj29=I#$%(e>?K*TUEvEP&M~JSzY{|a6IDdu6O)P zgxN3~W|W8&cu2Hn1Vp<&z15l*3%N4_QlfhH_AV@hAjDRq>MXRq49JxO%P46IZ+j#?bb3NLkw@DW#%}pCnO5g#Jcc3a$o+|outY0(Be#Lkz3*s*nD;G8c~ zPEz`r_YNXfreFTxu({Ovx_v&YsP;OJ02Efp^NSetVg?)AI5kXnH#?IE(!s6v@oT`D zzhG#4 zcDfSd55-Q`0|u+O2YlX>K}&YuH#%zCv<2?vKjB+4EAiRxG zg4fAkVGH*lN~b__;7^yv_EF*ZJ3iI%cWKVJ>2GVqBgu6bry;B*2umHhe$Y$E+C{$r zY@wt7EJD;$Y|Br=^a84~=Q-g1J)=u4fQTJ}*h5lmc3 z_eayXzf}3S9bHn8BHOKIvp2k0Cfuz1;$o9=we8MXYd|-C5KKvU6nZZZ0ZK@6{_aq3 z$@QRK&pycX9N}0|lfkC=?riL~m@$owRTP7IKfDb2($g-Vl&Se*cq8~Ciahvz@q36H zhtKc7V+#~-{vDx*x3|yxIs3BGY^prj zAvsrKPkz=9)2k$_(e5~ACJp0%6nfPskoHCLKWWh6e z;`S*@fx1V+o-&T5Srof;Kh8ew#Q4zLd!sP08?6X=uI(;p&oyrFKO=Rr`B0zEUYQG6#1z_2s~cKgu`} zvR6!HlH)6JX%^F@iH`nJoB&x-Gy$QdMnW|86r&(=U{nNgck8A^nFg;g9?!s$d{L@f z=nllVLEh6gvpduvC$tI$cSlbW-_%aG7&TWwdx#fw;&j1Gm`>@J$pI6t&ZR51_7QiI zA#NtdI8@_`Ys2_*d^ZLJ?CSgvl#Jdi1lzZITC!U*qU}+7nUq zMi0M4i(R~nGxZ)lsD!J(8TqOEIREu}-&+SJ>z9CGcTa#~1?O3T1vVZI2rn574W4a2lDK)tH&V{|XV%4_*F%)2_Sa;- zp-~0*z&sQQtZ?a7xsg!CM2XJfphJ&I*h) z-GQgxeRn4J_Fs*a)S>1(dz9j?h%-G2dT4u8eCMBg21OI${#I}AP2s1lSYOfWeEC=; z`(u!pZCIdPJZnTU^Z;5*Pf}Ea~DDuMG`NUi1G+9Axr#tuAh3 zvFv(fliiBksIX-h+Q)Cvw=Z_6%b7vmCW@ zIDaP56TY#TotwakOV9!{e_&*b-o2{*Z{6@uE#AYU=Hqqs9wlj7L^ogqkZbIs>?C1FVlRQ|u9>VZ^N>?X(HN~wp)#tc!?rx=s+)hz$J7b(g*+ zZ_(0JM=pAonrp31`bUS`SEtYsHOs}ik2p|10!FqpaF75!Uhz^S(m8AX=r!?1e#3wb>LmP;T_KoMzpl^dT=V$EZs@ET^gk| z&jE@T(^dUAJY`vB6v*!)z2j|!6L6kcJdDY$jQo~vUYfeC#>bc4UoAD#JtfS%Ace-F z^qf-7`x;V5B!rr!OvIkstwcvl>lE!}r}(+Yb8~eku7FJmiQR99$(xE%J;1|`mIp*5 zW=%?*-hGx(k$^zZ0_Yk|dBrv{9eXyE1GgP*8Jtw|Uul3EPC0F0d+sk}2)y}K|JCig%?kq7BNdt?tt|SK zMi}BQLLa6tF{>NA0+kfmK1A8#2(4re-1|-T$4XFby>h{y{yBMVjcRiD@>u`<37b1A ztV+D(Rt#A(=w1HtGzh|`gMUttW7L7H-gAvLG!f zdY$Uw%dQwNwK-!cVu7VZn0rqdq6D-@LH6a|~ z`}ZfS9J_X<4o!?$Zud{@t9(Rj*AuQ|pqeFF*Es*=t#!X{YdEg7RKOr{#h|-r>pUL* zUBM*|I6Jx0fzHLgN9&PSeW`BiJKDuvQX)#4OqX1$IsbG?&;D%j<)KWO%xKLB#ZOc7 z6Ep-MxP@2yyUbwPK9!`mBhvGk>3zCcYsw0^LGR9$6roM(LbhR$-)%ExX$Ipb@YjzL zXunDz=%1b%*bj!@CPE91LaO9Ek{%%F<8junV3lL%B#;-FPrKtBoO_GcS4ZL6GQZ9E zNQPH|4h1MA3KRJYN|Qw8X~rZ>lb^D05zOybb8(GK(bku1P`mppR!!Q_Par{{2%KV1 z_%s+=GpXm`hK7RLKzWIf+@HHVZa%`ini+8W!8hl&zT1`9X?bJu=W^B``Q7(4uX|4- zgWG{=>cqg-e(;i;zSlL>DTz&SMX{(O_Q$)5I*T^Q%jnHGxSRCMgq~vi(0VEvwOu_#?<`Ni$Xqezu6;IEvoMO}J9}+ zw}NZ;bu8cD{ZCvTD2a}M$UtcC!;yFN;=G_u4Kl_K2ztw_m&BQdM2xCcy-P|LuKx!$Asv+u=8~h9-Pt=i-WzLf$;h|PsK%C8 zz6?_&zQsy7rboe{HspDzGLj;u^F{fPFsu<#f@P&+c^rp}I!@EtMZw^{pjCiuug0%ac%2c!WrEk8QH3^z{ z<1a-t6&9dYq!3gOv2E={<`XRNt`GgcxUaTEE2Asy^(jMVto2)D(yEseEK)KeC}}K6 zUL)zBpW;2leGfl5IC*DkKp*}p3ob)JO#L>zac&uc_JBWOB}_xaewqy0wQX6AR@=}` zg5xm;u0EEeptgJf>X5gZ)6Z%_snc$~V&xW7R2G@a(yHV>tYg9ib6EHZ-m#&(`$}Dh z`;EN(!52*ml+WKglX_LnGd9^)wH7`t?~)8J9_QSmfhI*7IWl4r;u{>C?e^Z2-63uH zF?f6|xM{(#>!Bwz%ag>;YxLZ^FRujxw`2Y#+_q*1hnA48P{OExYNCESg&?i-%9=YW zzGg`IzxBpVSzbEXFW<8@$8+GSgl>Tbhs(7Gigla}N6fM9N)oBS2;pS#4f4Qds^fi2 z;+Vx&17BU$LC>E-Wl!^3QRwMo{XC|N=+{UJv+4c$z;<8~Cjd&fy=3Ckh|P4~KmV#i z7cfNPO`}VsG9MUOq3T#eBpuRV_+^%NDVw8r(!n>9!v^|9zWLY}bUIGG2c5By%tBqN zM}DHK%-`&^EJHtvl)l$<>?qWW^#0bVWHVZWMs60I`QnGLYz~A5(M<*DKP`ts7l-CZ7)~2S|)m zvG#Slf5)CO6XZ`S_ScF)J);?}M^`#DHg~c)A3jxDmsS^|31xxgMzS~Ip&wb5tQ*Jo zvmT#rCMRXno+c9o8_^X`=77z1xcJPr_I{EeQ-j`xp$*nTiU7h+ynB|qOEqG+s{1Nu)mN3ySWoRC8Ua3&D%Xr4 zKPXmHdC1GVCh;v+>hIuqQFSTz$O-q4xgmu|6Ua?*wXy&wTiqmg!Rr6&M-4>2$iB2c zfEbOX5uut$Qwn)pm_ivZr~9zLdDijT=y!Y|0R4eTnWdZqRkO}FMjm2>7+a)(5zz`= zGi|ZxqQ-gPI*du+)8|&O&Ln2Sp%jfhwO`|i`ADFjR&yCUFuAq7N3Gm>31*}w+(4?j zy_F;xQ!jqZ1}|d26;UbszTJ zcv#lySYfe0gauq;Hs8Pt9^DRIzNcR3?*hO8At8lVH#i}caUpZ%HRYtJqh=S#WRIwR9gwZ8Kp3G z5ziK@*jgHc)oVV$uG`u-h!9zoCZKlfOQwUAaQS-WZa0^{GH%O%1PnbfAJ4zzjF?S_M*^yg*Z$6N9q(U7f+}3N_ z)`l6k$-DDrgh9?0mylqS*-gh(1Hf$7Ww}#rmvs;%4>ud;lTv>Uxi{~Tbi}adCof8- z$+PA*&x^yNy;M%4V~xr7=wNze=jSDF?UL_+#JltC1+sYktpDMgewl#Rr#COG-CI>t z`RA>Mb+fu{7I}gm3F4h{PIlAn06Uv|_5WXJ(vhe&Z9)Gz2jW#EEw=J`Kxo-;-p1N` z@`-vS@|8zJ@Ds)9*>yWCB~l4?cJ(*|Z)a85BqCpFP@n9C{&Rdtu2Ij#PJ8&@OD_)_ za>0Zrq{f@cE?C25qE`uJV5HL=Fb&?rmeuHhvF(iEYVysat1tinlLM-+^P*CujG@!k zQd4I|swF+{H@xGmU982Tk*99@fyAxOJIGXS^B^UJ3-&IU1v^ZJPF$w1nMP(8=4&BI za%uzsBoTt>byrG|w>_A7#QX-hx6?3`+e;!2SpJ|`GOEZu2|S%s3s~)n#J;Wk(V=zy zyYJ~91LwY~t%;{58BX(bqT7?OJ*{WQe2v!Nyi)%ve?DY?%>`{=PkbBu%UvEwr-Ybb zq>{gTR5T8^wEjM8ipl1LmvxaeHCz!qd}>!a^)ZOcy;htGfYTVRsabf8VyUPvF-}cRwcN#p9?@4+xAs^liK!KJ14#pZ0T*M!MNi=w`I< zpRd1yw}AnZVyNJ!v&-QTTe6FFU)z?`HhEa*IM<%r{8GCSQmF_!q2rW=>=sSx1X0S* zy0h$*sC!JDaQa@f+8_QES1Dqmp8FjEA$PAfHlF?aGxm+;rhSU89ol&rd9uAs z8)Aa#yZl1|t4-)P&?1P0tTj^KUm5*%9Q36V^7?}^C#=ijcIYE5=y(P(4x5l3o}Gw7 zxZtQ)ZFI6Pj<4HH!z;*|&9VIK18hwlHGlnVPX-(T^*$A0^lnMq<)|Iu5ME0}!fPpO zgW#f>HTkBua=KC@Q!Rj3UvPO1iRI52$e4RT1l%pMivNnL&_M0if(N0?YhjFlhYV`! z_BQ}9X$pNrg%`JcvH^|C7fZe2a`mzsp@~^1`YbNf!K}yCsYBbApKHJY3ycS$E3|w~ z6ndR7wCNk2TtZYL^9r{_Zf@HYy(DL&lA1t%7b}IA5fG4$zM#R?5P@ERYDDG(2aE|3 zmAmRlI_2-e(P->6s@FiVJ7hAw_XS*@i^&V0GCIGdlBrUPTT+H~r=aFdYVgWg@ymq- zvcs30_S+{Y21^HBzzJQ%$U`EE|`Dc*07TnG>_ zrazo~CkPNF(`Y24A=)!cef*a}aqU2LYsqpFD~NJ4b=*ae?VSTrXEz~+F$j`l`s{Wr zf;b0zYiWI&kfXh9)yHTVF*X~?Kdm-ljwb?yMAI-EpO-{X9NVUD60rApBhA#dGzv%$ zc6e@VZ=M*xX{0sN>&}nEKwPF~76BDB>F3STo6SHHX@k+(u0Y2Zophmt8*rT@b9>Bx zb4#I4*`0;adjyzR6Y8(3r3RcrJ&f+aUkAW5g#BZKpgj&)cinG+T7-N6TCS8aiH;3U)Kp1hjexCKDbNXp0kPCSu%U@ zus^nh-&)X{dG#-R&ZDK!6E}u5bpbK0alg5G*k}3z@;V}Tl;7b@VE04yxPtO@KNLW+@N<`2F=+Fp$dfGhrnPY40 ze%Ql3kfK@NTb6H1-nsEbQ#Z-H#S8#!=RGWRsNS&zd?EuOX^U!5D4y(KBM{B~+^4x}_^~AM z1rEN%U~)JS-y&Z${`}K?%5_MLT<|T>3rl4q4|AWClx1JDgBkEZn2_qIec0Q?hpQlgxF^ZV2Drqovfx8w+!P{s5#w7y=Y0&*bG zs13cou6J~07RoH;G`^s;-6Zblblij(2v=arhrBsxyyhyN0;0=|u!dM;>{l)Am>Ru8 z!bhNdY~MgVl?)*zrgii8ds@PfzP_2s-FoBksv#(@mgaQxW6g+|cKR0wKr#k_mI}w( zd|h!PJ9z_pIjmEkUDwo*K(m^xDaZL>QIbO2(!$r3#C{}3C(Aj@Q18CY*Zbj+8Is%2 zKPLnWE9H?(ns)JTW*GnQQ?cd2=;%g0Kf7$m>(NgS0H9Gq!$*giobZ1K83(G_ zyZmMO0-bTCb=4lC_YqrUXI}jBFjwh?4xq3t-v0j?{sA61niACwFk1Fl>Yz+78c4*& zk4n7sABgFS3Vn%!%(XPr8{W0F^io*+Gqw;o2X#dIno-p*h?QKY7>f<}b=l5PSamOB zv$Yq%QwEuSpKzwQYG-rxSKtWj&HQfMPlfR@Kn_K{+b$uKRUjsp66H(^YN=m7lD3tK z=YPTVk(||1eAQ1Q83!!0GzEd=pfm&`Rg;|!${&^QiGT3j9F}ege-+>ig~(v!)2?l6 zJ6pfWuiOPKO%?@+`R}Y^E%kPn^POh3(_Pz*^``L|-&9LT;bj_5R&pQi8tYZ|UlarQ zfz=I0WI(YaLF5WIlSnL&S%M#PCE|boZZQ?R&D5H25R1w<5mQZijP@QDnaKg1n}e74 z-$MvNG@pz`D6WzDYu;InWeYZz*6ak?kmHpU%G>1KP58(mpE)Qz@b$fd_pM?&K@c-# z4)?jn(}IS3WUA4AhVKqj2gFb|)9k-W{2zC|K(77;C*%aU`N9Od{mkEM4rgW` zG@p)0<@0_j^W7|d4M{D0&V^e+fN?d?zAZ2y;M~lIs2vF*aHrM3{e#8w&edy>(aDgb z+xPqX=EJiR$tPs+R0t>#mDZmZvvSnpHL}m=KMZA5rdXs!e2#g>i#z4mM3GlN4i)|c z4A{uq&%p)cjThgDMW4=iF=nM7L>L%C5c$|hl~VW%V4oQXfuJMCPSYx6Jw$XXM!QCj z37-W4^PwP#fv0Le*ZJ&@+i;gN-#|+2_||f?zX|qnop#gjEPRToMNop2Vp4907$Vt- z#dC4|V9_yGsJA+RgZ?>cBG~EZ#&^kPG*j}OHy2#cVrU6 zpS5W=p&t)~XrU2w$Vz^b`z^VAP@}!gcVgef#JmfdJiq9`zNPemm{;>57T#pDa*L2K%Fh?{)Fbr?TGuOD?m9s5ayv`waZ9o^ zZMFN3Z8zgx>LdJKnO&G?@};OV)R))gAZh@{cYP>qi$?pm9U5`9PR^ zj^6X5j*E?VfZr{u6*#`P*i`m}=>sG$!N5}VL%acu2ebCaFSPz~9SUwSgmJ{VWfe{c zX0uN^3LP}BHG#)a)}k0-HN*dC`tEo(-!JSaC3cJ!r3kfmsXbHGuTUctwMW$|irR$M zsI64cmY}4lTD7S?QoFWVvG?99ypP}ezMqee|HPB~e(rOhb6wZDPT=*+FO?WVp?IXNK6uBJ-~n=2_9P6kY;6Uhl81jP*a zJ$8@*L$S~}o9koXu5u2>V;r32HvK@1A1(H3_fhuQbRF#Pt*V5yl|DFf=1F>;P3t&Q zjCAY{LzT4K6Z#Lz`U{XxmjJVDjuno5Z`ZI7W(q74ExOa>R zVZN2mDci}kZatUxXxS=K-WiNl&r^(BU&xc47ffI1M+#@iarzdaU|<`pnrvu7mT882v&{(Zp>+ zKtayt);B2e)nxTq#>URS=ZEqzky=Mq;&>AMQMpVhch|za;eO+ zOd0x5x6%RHBEg?DLKIc5>9UgwZnf=9DTxK$UcUe=N0)hGu96#^B^JzUhkxZ|>`<7L z2!kVq_}C!?@KUG$S8g;)6gq!k(H4k=Osp=sIfGL8?c}wI*RWSD^;-ldlsol<>CmhW z>b+qrDc;n#mit5?FeKa@)Ow45#LCE&#~1`!Maf(Z^OUVE0Jt^zo;#g^jb`FX2V-u< zCYvvP{7!S7YvbjqD8Eevi_IHnRl7(U7TT!apyyBjED|s#&wgQQLZk{Nm*oz&|C82w z*f+-3Bg@{=c0{0>be^^{_RD`!jT6&_2-h8S$k;lu3(1`GUDDVlgzjIGsrsC3OC`hg z5U&sq*F%f+cny!ktv*45dIc8UW@WHbKr+g;~77{foOv47PO;d#w65&o3$^r8@EPd4bPf6ho}wT>|}E?f6M?`QPgTw|W2+an(VN1pc2 zpS9|Sy7ymu1m?SHf}!w_8xxIzH`*S{AP3_~sv6d0*y{kaV+@nILH|LTV2?6v^_{<^ zet6jay^{_i>-RjzbCH3}tluoTnf{bLJ<#j7#hZ8dbXXu2b7Lbt(Bw?S$m31B>74i< zHcbav3+8Fk zG&e^df9FDJ>4l4}QzyUb5A=I^Xg*d?0y|9{W8=W`No!^47-jz*1&mJ-%iS8<8DH^J z?As62S$R^nXK}n>c4*RIlV`pd@q*LzV-YSr1O(2Ia-q%kqM7lG)s00}SoR?%&-nO< z?apk&{&Q9Xr|j6zpP74ljwjO}#9gE zz{Q27hloJFngA$`sVwv&|sYn80g^J z+ae1Az(=)li!Y;82Q`Oi^4D?DpLP2%WN|+#`v5yteyo z6teTDdS?gm7y*rt2j*+-%oFp)v1<<;(!@hc4tneys&2EZhLOd0wFbq(vI9tnq+Ob+W%LJGbqbFJ+S!G$D*-- z{PJDB?{xV~g&7e|R2~T_RCT^WRvovQG$_abX&GI=Tv5Dmva#{LFMBz|dyo8v$ZBIG z3`S>MOl?lB+-1m!CYhG#HtMS?X%^$Af;KGZJUB8y9Z3L0%D$j;)qst5X_Q_Rp}V|zP+fAEZ{F?l$Q!74_9jb>h&%x2|Y zLH0~=xumafjyQ>vY5v3~w$fdMsSUhkgersbw`1hfru3p862R5m)u$fqi7%k9(h zL|)+_64SnN5u#Nr_s1h^xZ>z#16UQziS;Dx=c3(k!X-*byngc0R?LQpJldEq|4{Tp zt7qhj^;k32698tND@o}nArCcBb7-xZhb?Jh@_n|AoL&`GRS$2S)TXTnYP(h#-Jrdp zOfY)p?Ehp(bM0`f1S*%7zc@SVgM#xe=VEMgC&KSfRrppzkNy8MKd#xBAfQ` z@oE^z1_ZUN>7+|Pd^R<*WjpdBVfy)_)hLO(f-9zP`o#PGJ<{&25JRj+g-zQ?_iyfWKdk5$n+Fa2J-u>7Qam zJtNpaq4%HCX$lv!`8oZa5ap))6Z7I38ZS3?)RM`f7VGZtLx)i$`MZ4aqVJth-VTZG z3E!D#LnuLKPubjykgx~lod=<%X>J|veP?f`?YozIAj+`&V?sWxDEIDL;Ri8dY97h$ z5O-*8eKVbQ|C21k{8P?Y`*NmaGOf~8HK^zPv=#o|cFU%^z{(Q(;>tk-9*so#Zbsm; zO)E9!bJMs1m}3$)XmNG7Qzn3yvNHY#QTyI?sr}=XU95V z`67mzvKk&g#+`f`U#m>(Tl)UnLwa z<)IGCsNWFlO<9nQ9mywjEP?R`3bpdOV1JU%%#3k1@G(&!_4}ktJlOtyR6@f^fhU`pFei z$bSBri0zQ-1X-5960`w~YNP>H8JT!m|88tMN1(u}bT(PT9O@pBk(2))Vql}pJfrYu zN63TzGC(}oKP0i{I+rJTOc6taxUV!)GpLZGzx<67VR5v5{z%@GdUb~FdavgZOLHws zVgC=OoB~mI?cD!jo&^7Z=q=`cWwQ?i#EhwKRxw$oALX%l0`nK4BGMK{bW|b@=p9c? z>?m9S6PjrHpD;?2x!b534s&Pv^0 zyg6NFRt&W1IeQez@FhzfdYi%epkA@a8PGho`_8y{UthQeZNpExekGNeb~0#4*3VJy z)R{jnH&;>Ii&Vm`L5)UZ|J=kt54i1U9Hbf2QaC~5LLu7pAcn`#?ADPVEu0ini7d2S zU=RJ{rmhc5QZt{GrFlQ5vww8Y9#e6vpa4HIvQ;1B9HTqLL zX9eb8L^$_my`g>YtE4fxFj2qclyRUPBPhn8btxg8HTovYki5w<`U+!z^^ti6Mv&^P z8#vzb3<+T*jEG^Fb5Hj(k6dN{iC7my+ZVdj-;$Q%1=F-J$^nYZyV%RjrkW^NJnhdU zgUJ$nCm`?mRhfKqcvcLU%XOZZgGj~HLY#!?9Lpe6<7?Gjj2f+B-I=CZq*yW%E1l?_oVhi`=KqU9wOJ zRS3LBUdsUt1@JW@d91{}u%;@u|2-DB<&7Bq4Zf*Z(xKFmFnl92G{KPVC>leK(GP6Y z5sp1W-gb5v;CH$YrcvTe>9%=i!8mgZDu`-%RS)?e=OVcA>i>R*e1G9^I^vFX9oO19 z^h(r{H=E&$WvEq_DJyl$Lzq|CoU96%L$OR z-ywhZk(a{Mk|MYXP4>Tq2FQ~Qzuimav9G_LKOScC(wkhPnVt`C1|4%YiV2?z|Eg??)G)UfN_O1;l};7J8H zh+hIre9gGRq4 zDm`RM>T)u#J;uXoB-Tmp*gJ7K^MtB18WfClvJ0ABnSrhH86}wkwrZ2Cb@aSS>6j3A zT4+%*lOorpg$vL(;^W%CsXS z>!Ykz1(W7|5GsLut&c$fgSkC`g1sMXY9aOP+pZNaE99sqEKB$2?-&(`T+3( z&ho2&ojJC}wX)2+0{_opC_~6egMv5^q@1NQwf4?vRHy|AQ%7_t_{2fU{8|s`jCt}4 zYr=3LV(2$2Dp;Gqb$xYJe6cUph@U$N znyn8yW(e|`GtpMSw@C#~a=IcXAG4i?p9CIzWL);`oONVeh+JEp7HU=DgRVcY1>p=E z7J6NSMyi6w#uTpFvRfywH$|>?r+4u4D{=DpE!^2V5#O1!3Io+CL(ma+ z=c3E|_t_S8(8@_b>u2%YqLrZlE@m}y9xWC*xB2lC{A?9v$~_=vsd!I_<0`f*ms2Lh1sMi9Ogm;RtL z{(4n=Y~x7j;y~$qOlg0*ac?82BnU?Uw@>V7JW#sE_Fo?MOEGgc9?3Nxdt5Ksnd5(q zJ!0n6Ry-OHx;o47z2>yIUfMb9syXv$JQPr>8q&t!nS+(@7I`<2fOeZ3uQzrsnqA8_ zD3U$5fa1y1Oa**j323M?yFX&Z01?)%Q(qEU#0TDz($vg0516PTv)u zefO?EP^jbfZrtf+^Ib^^__uR4<4OQhJo+3D*ewPfJXZ2ropnCJo&2RnUN@SUnK4}r zXIq^1lrB8I-lD!_(YpD-)Nvtf%VU304BFO8U4Psn zbb`Cy-JH_=@?BiqE<3NpyZ_zX*2&C)a=tz*5TZ|dkM$>K=W|QZ`f}p0xi#@h#h!BNq4MhC1j_vy0 z{rg{Rrw=#V0S_?FP|KJQ__Np@NsjKj-)+%bhnX%&una0(sdP*N+X9<&M7FrGz#VEH z%F6nEwB%DQdVt%un%U=d6F_zPE$HOS(}oX>R@)U$X@`p{2U!IH{GGO0l96O_b*JigrM~}q6G}v{iJlocDy6!v$w_BD~5G* zKTZ^22tm4@K4DagTe$3Vy_{>u1^yEXK(AmsquKJBn~>F&(^(-^_ho%1m#QpiVMfUn zq>`2;Uf~-g-z8g!73iMZ8|lc8!8U<)hC`s6)zX5&h>*DLmawjHq&$U2hi1hjIoiTj zkcMMuVzhReB2*O9uwoeU=C>)DY|L@AG7iBP40)E}5p^4IDY8RM-YYQKYI@g7h1t?t z5vn_(bX|Kp31P>&aq|GWx0%hUp2aqL&}@Eg#_Dlj%FF77<2vaT_}4)H6KdBNrfDK6 zW2=R3rTO^^8TifI^3hgv-gx9(+2qmBn+!7p$S~9As)c_J*QST_jXJosA5KVGjz0ja zLlWSR>3^69#>n=CBPUKC!}3;QRp-DfrEwTaMoAKzbrKXW%iiwK?hYq6{bI?+%D3iHOL!JyOkzi z$6T&lA9fc9Z85W8@$DAp|90?2Z0pByTC$>9C+~(#X$@N@ua%NXRge-W)41>& zcYURFymsXHRhRCDF%kk^8$G+&;*k{Ng zsY|wF1xvx`-(O2 z@->S%9}0iF{(G^XH=fef2RF)EHn!o5Ua1uZ1$$ieiTJmA2dz*eml}PpM-`(yLoV8Q zt}YXvvi?ha9Jnl=;k!k6^13ri&ONJR!El(w1wYXHWYkL^XbC#WsxR8$rtKEF-x z9c$&g3uG`nYUJIYM%tH<1?gbEEsj=C2+tm;#Q}y|;qZ&=RrpRY7!NF{bIib3++k%p zD*z;qQr-wWwOUex+1r-XN_ME=4kxR0zj3!A6L?WW0uC&t_m&8214HaKw-v!$uM~nk z^cOf}A z(rEP1Mz6_!!rXJ{ohES?pv+)UO%1(m?C^I$VSACl_=jxK#lDi4E#PP8R%%3|?1A>G zi#CoH+);0?bnK~m05`)4paklGPTSt)X5JrieUH+?K+2wJE}{eX$G`RU{OiKw){-=R z(SYBsUf8y{YR65j>E7rIB7h5YOMYG2yrv`UjraXbWvn=lIJ4+aav3er+V05;w)ky# zr)Pf7pN7fhMS|Z*#E~r6n7tnu9NZfxrr6k@f|dDZ{`*~KG^NKYVKz3I*n0Qd->jGJ z@?HDOhc$hl*OIL(!&7f6Y@7f6qZl_xIQ-G+DQniFC=!X-JKYr}u^&Wk)4<|BWYaos zyML#WiQUVqF7>#~Vy*8xg7~HW$uWWJ6IPU-< zUOGZ2SD&kwRJ?XFLza;~%H#@yLEO#?iRo<4bdK3Px~u<+FkC|>&O$KOfd7IYHTk7C zB&;PF@FE^|;=)4RSg7asTM)ko@4a!+FuW*!KwI{1GVZJC0bHz()Uzy~Bn- zngsfuae>|aDR+JTW1j$To7bP7=#Q=VHmsbb*&)0Dg7zU_4Wb1d0jQ+Z+Nk-D&aZ$- zm5`ti+p+2;uS5Stm;T6)G=+#GamhMU922tzat8jNnH_L>xd^B?+0*Q%y|Yw0PGdcL$_+olm2iB3x;<(x ziLl@6n!ZCw0}D;eoE?5TwWn}3+%Vt98u7PM(H^4Oz{bnvU(dz&B-caunH=I_;bNuY zb}WSztb{If#NFZ7zu)!$q^ImPq2*_em4?|yF=Sz)S_fYK!D7bd_aBfh!0HnZn7}hL zQjIqcK?>XhQg;E;1bm@eeyvqyw0%E_Tk$}Cr?Pr=^YE`1MO#v`ql<^2>>KUs6-c2Z zO$W%u)d~rQf_N!506FS+oxI*2`G#^Ic`bv3pN)m_{+u3pY)_8V7G{g`N$wqmgjv%; zDXPv?*@_aU7Q#QgaK}A40rvf%{xv7q}<6L+vR=$s@tQkn>PIL5Xd+52dr18 z^=C2kGKI}NDBGyWs9`9(u85s`{7A>Bt(Wv+smPSKzvm7#bk9pfB-m<^*oq$2TC!TJ zf?i1z0F#FPx2xG@Xi)PUX1ac=JI;&pmipqYsWoF*@QfpLz*Ckc(?G~T;C85TG6up6 zZf1WKqnC|W$yh=P6~oHj-lGqoyU0yUX~3nt>Pl|_DPC4G%&OO z#GR_^Xky0Q*P_O~`ri~C(a1(g6ruF!#ED#(Y@y*+jr<0?!tOn7KbFz>5D5Aq)o>t8sEZY3}<+Fl!I7lGDCPx4= zLh&YVrPiuO{?S9fr}fCm@$sz+W1FX1L>)IX=6)>k?zfa^@GZmGi4IJ`SrpZ_t;Apg z(WZm6Chrbx)Wxwrz{h^?2UZMt$n#Qy5YNXUH+?}*JM^$e`5T7odxw!e=R~6CchR}9 zh%{@i36H-_IFc`)uKuX%{4l>(U*`%TYt1v5{7||hZIv(atNKmOG+Nhq`iGMuL^(pY zMqW|U%MMcu__tf+$AnpxFYtUb<-bM9LYDnkK?CKmnMy+eb0~#^;em3T>f_J-h^<%#Mg2*f|*$_7C*~HX@X%U5j zVXM;bHtsqe1=f9M{W)?Bz;afMTZexEUerjCK054Xq|CTyy5NHG*9W$yQ;rJv*onX` zq5#VG`AgnZ;Dt}&dTiVPcZB+M!c;lhiN?jQPzHL_68DwmMja1jHuKS;HPnN`hRU=k80kGBJ#l}a!@K`{#f z>INV5iZ6P{zrvW0g0L@4j9bem*^RFP(KVI!j|&0t_`^i?m(wiNif!Up} zmdZB&<__z=3BR9}!I5nLf$IUE5%}|l+9t#0gOT zB%Ur*r@QYj{fRqwIuwe7_hK*n2eTx0k527F_lnHuP#BbtuWiZihpX7nvxw=}Y$V`j zQGI*Ks90l(LBSY_=vZT&h`be9xvvV=K!AjMEnGp%e|D>Y6>uaCI}A~;cABqx^51U9 zxZNr&@3KZ=jDe2bzf#A^=T=+LP$>8KLFR&7Y^mkCvUH8oM8%lU7=#<_>Uurv6e>pb z{_jZ$9P`yDG2D1#;Js1(FE!9#ppH5m!r+9~%sV4B{ zV$+{LP@IUhrpvB6KPD7GOEk0kU&T;te_wgjZ4#ScL**}S9l$(W2TGUK5#kW`PzddN zQk4d6!g#H?l;iE9otfv8;v-FrH13asSb$w{@x6J+fwB+;<{yJLsg&N^mf@Y~{#Bn? zEc|OUu}6=+jgncMUGw9@AY!B-UUr1LP{OK|>>DlrSSA3h7d^J9EW8E%Cc#V39ZO0U z78x1o?_Yn049Xfy-#NaUv?%57LI{i>FV<%jNZ}z18?bg8_Hav>J3j{mVO@8u>6iNi)ms0DBr71+|I*LlA_od&SAU*WKWgld4N#H=>1<-EKu{C~Iz!0gOn#0~X=zUB7VkJop2)=orOW+t zjbD-X7M!v+xjNt5e^Chh)Cogz|CF-C+%lu)c&l(B<@#YxSO7M~%QEtXLqGmo zYnljo11-;%w0Uot`IjRvcRLDIFb7V=*BlzNu509$_I)EZsk;|$`+!0qw7T{E4>`$} zLebZW+y3RcCu01`WUo#_anVQY1JmXc8%u79xlFj4IRY>Z7cW`8=-X~gCSy(et)EHu z_2xxmEr{kPL{QZT%XwAZ zY$KM!5$`ca|ERF(Pjyt((dW2_za-d&svrhhQR3T=oe3Zytl%Q}_ZWrgXsGacnhNsq&^kcE%q2|-{L#$^UhRM&i%zs5&?6Z3YOC`b~4GG;jE+k`~ z*p4qYHo$b?m;{8Fu%Q7-LE%e>nyT_a3-dJ*A*XN<$g!GO>QX0|p|53^@K(5#xfh3U z01G=2L;5`sh@I$6Sm^F_f|^^-i`SA-05h4b%ggXM6gtyqA1i(=6?B!Gak;A0-`@v^ zuE#EKG@kZa?ENuhh{B}c|5;oHU1!7xtgJldtNQNLe{NTe2&(uH-mp;Eyd|8* z6XFT_KJEs=M96-u9rqO9yaZ%}iwJkVut8He_wAI#8>bD#zLm~92zOiac=I@P7}^GHYyShdUar$_ zLgwk#ndp>((Nr=~a4Q|MY+BDK@Z8tD;{iz(ux?Z|&6B?eEH8wzBWY-{4LYj4Evbv9 zWLed~0KxnuNR$U(Xfn`27zVT+SsC3%A>=pHW_4g-U_g8n3!Jw-?lB7)Z*oICnXVD< zX*bu^lykk!CzbC?)0h#`8!dXH&73BAekcf}Ohn*WzM#BrzplJ)>3tej!1ttx+?1i3z-!uYxFOr= zX}*)@0ArNfqIAuU2Q)P)%|n=o87V|W2eSuE_Q7-jq$v!g;t~e*?`~i>UQa$x-Xl}2 z2TR7FiIPT}F`eBeo6@N_C^+cssyOf>rNQ54p?T8fVlLyd@#>QLdUkH?w~LR4zvs~j z@F3vC5{7$`tHm)yzPzT^I8qS@xSpBc4ktkhCU1rRCRdm9~;7LS>w5xN;X5&%sI!Y)uSlvO1cy??E?FH<4YU-TS>c z`8&!PwXB}<^FG5sB}OxrE%8P^MOx7lj?7vNyV#)fccO!O5{^|Rw4VF)ap~2!Bkd}g zy%BfClS$h#f5I#g54k?<-4aja>?}zH`X)#@ggN4%MLELkMcl%v4Y6d>&rS(Ylx(6E%9mq_!;qiu;U{B& zBzaSENV*$AmAB95CPvP?M{`Vc1QoJJ+?`1UQ_t@6hAMUxJudDSmkr#10Qf)bey#`QwrNu~vL0S2?FZs2YJba@I;~%QFh4q^i^q>-T9@et$HUs` zjPtpm^p$^KXO53WqZCi_wej&5=f#j_@A?ygBJ8IspwWT#=K661`jTIYM9R4p(|McV z&Et6#dTZrfa`CB{nng{r{2U#)BUh5XF==5AVD4U&BY6*$Nzg+?XT1DgFTo~)w1wU# zinKYDW0Q6)`*3yetq?E$M~hIG3Y{N!;UlmbT9iB%6Z$*xF_cUCG#{66riN+d$1pz`MA8=@K02}A zuP6LT$(cnu2m#d*n11Xx>sh@R(uTK=4P3S9$Gg!T{~f?j&I*+9*&|C!rIDQ$r)-UD z(ppZQOL7g{PI32rw-12HEKUuplnl6od@P&)p?HQG7~crcw+o-@a*~uTr*Uq8uD|hg zLFuw`=d8i1JHlgVNa>6%==hys{XG!CGb??4u$O)^b%9sf&jtkpyB5fX-&GoBHe`te zg7ULD)WwG(zp#(SOYon+u0GGSd_KNh8=ZRX?OTeA^OfBE0}4m^u!BVCRd0zTEy-9Q zv8K{VM6{{oo|aAI7HN&`O;(>I!C!T4e~yovFi-MePoQf)!YyFV&?>v|w|YA8Zqrvn zqZ!e1W%Z*qTED+g7+7-+hZ&v*oTW#bA24hj{ML~q<;@HuZW4X77b$(c746zg&u~vS zi)K&V)=TAE{)+l6jeC00TUsA;Fs`dU)@I#=#Q{ZRVMCA&EsLD^KoM=Tx0!CtLX zu9qGj@ z5QBVd=Avk$Tv8v%8`J1fY_g*{nI%2e3akn_#DvF7LZ}Mt;JV z3~#k30Evp#ki$-=03LmuIsg|)tq?F)e^P(V+_<*?oE)gGzeWR4i-}IvnA!b9gNrSL z#vwWf?J_Fn@*$+ac10OdD%u9Ur{Sk$G4rd-)NhjOU?RG)`i&v@oml2qY*(u0vWDBm z>3)r|5lfNGFgeD!)L0@|5FAytzN++^PoKCRlWS$OF^qd z^=uJmhRRW;UJcip!=uJkdDGOsTiMShdXxTg(1oCU&R%wl_%{tTkzgFN^aXBXGEIf1 z_nFpXN=M8aQLhB*T?9J)KqoOT)PETySN(eZsP`g99Ai$4A}Be2I?r|OvzNgT2r)ko zBY*^_hAc@e7<;Em53MOxlq1~iN!Q6>v4~Q0!QpQuzdQpeHhK?I4pLH*y|!io_o{M| zS``EF=F}h^!FnTW&(ou>{e7;d%%*eb0s)bkGE-l%_@K7|^A`9?Hhj9pbp>a?eBk2e zTmlfMjg8X2o>KkwY?hF(7f{N`-N|?b+qql@b+P5^_onuCgzs^x@ zI@=3TfPAdJ(3c6$o$Y7F0!bDp*h$4r;>Ue^P1aGpi+YFa~T^}n-=7`iMZ#TbvLYw_8-@V{yvVn!p zT3Ab~W9>iyPgmFnODuX-L#*sj7|VnX>CXODXbqZ$7jwUm>+XMh&Zt6^rM+vvs(pvN z%E5=3gRxnNg3)Vdnn6vM7Rz?_7Z=o86@(3&)XtFJ`=oRk;(C?n9Wdk_;9Y8Dng%$; zTrf0DZoCD7+PlNWHYY__PlTrix4cBA6yK`%oVB|KVVQB~$H1G*=DVC9k01JQ^En4f z2SbhL*B8@;e(!-*8@u6lO;la~?%0KdjjqF-d?n4(SEeyJW%BmkWhxmMME-_M>YT)8 zz}S>kpF}7bql>D*M*7~Xc&&`IJc$lR&XGKQGV&_T-OQw@3%7}Wp!aCh`e`}FzQE@Z zQ#m486dl!_>|B{5){)1+Mso`+Hh7DXbkIvpJA3G9|0woH#lR1?GhMM-+v6CqzPgZV z)PsYqkp3qSb2?U)HB_!G2#tE8_?{K%#;j{1Sj;5QaU}^sCYOznNLzD6e!h zYw>d||5D;k3fVw~JPUMDqj5ek=(6s5L*)9D`fGN!#s0R4oQ1)W-BqAGqhL?3RSa9e zkyD&f&>;{_he;aFOh_>W-QCzYc@j|dN!ABpnTe2MWN!w32dCtD**4#DxjJ5*v|pBS zDGi!!AQ7ow`Rk#rbac9+bkYR)&8;Y$u>RhvlxkRV#$EntGEE>)v<}sNu4p;k-5IM9O33 z74p=-(Fyl1Bwp!UhHnW^t#H}YTHLUx<2~cCa}yUc@QXaiO#qC0_rW4?{j?NFEDDa! z4dwT*9`1V4Flp(aKm+*k zaB$K+%$CK#gzh`6r%MI#9YN z)o|o&arMXWgB1~IH{eFmSx zfw$sb|Mem8If44T0eh3UJx|=eGsfMw_XBcwr1(rqX-_X`4;yrr`7A;CWc<*0eSG`1K3Gv=_7z@Rh1z>V;YFxkakXIWQqam|{Xhorl4cH$QxG z(bbaC>ix&Z_Wz>QTGqEX^{BO`OB*F;BKX60jYW#h?cPV57i9c=Dc;}H%Vwr2=P+Xh z=jS@Ae8p!f2XJO@jN%{V`B+V}$;r!lj`@s(qbfM+%>xN5;c4Hy)xCaAUib8DCZLY3 z6TKN1;FJt^U8Vqo%5X4jmUh$X;l1ilVY|;iEl@v;d1NYUQ26tVGw3WOXzveayy=RY zaWPTQ;nSe<=dOkQ0n6gWfwO2KnIzjS>ADun>6hUZIx8s*48{WDdtCuDiZu`Yf0X^T za}bGk-Si!ZPBMTCaEA@U?ei1VQK=3hT46_KBMAe>WsItj7Umk)5V7#3*Pi(!X6xTI zd~4IHUe)d)T4?)dD*={`zmj>dGcnzBS_q!fD>wdk8SQ2>c};%wez+8Y0Omo zTnr|)kG&y>MsL)>+%_?~aP5<1Rug%zx4!3Rs5R3!+-I?kKT5uvzJc0`XlNDp#olqM zk~*Hs7MG&<#;(fnD4=?P!ii-s$@{y1H`c(g|81>DU?+xlV(LG+<|6c;u*HXLF9qxj zy6M`_XUJ+=`B3m-Oh+p7F<@>$0osaD7}-`R9hMzj@4@ovMG1IvF7m%llKkH9)6))5 zRH&(Tw<`>=%Rc7sXU@N8PtowM|2>c+7WM)Mv==bd3H#%#IBAaF5>l%3Ij=#j)ogdc zr>57k%;;|+YTF;&4_A8eCbrqs8)_iT~)fN}=!msubu5?i@hVt z+v|}~`Ku;3iZ5o!Z`e&&A z)JFWEzp#8|q5P9LGO(M(Re-Pw@x_bD$?jjI!AdkY7S>+?QYHlmL0~mKy$GMF3OYSA z-cXTWwG9#93M<2-a#ys|jMAA)UVh9qx%>~z5yP$qwuzWMqrU~7JM((I!W_#~STShK z$64jG6FVro(36VRM>tVJUsL+kMRP7J`9ULY;cyclCVBgOlV65-O=ZN_m4z%V6x5$7 zMl&@wud|!UaiJ2)%HPeC!9Wy(&(y+7^K>Xf>{q&tY<(5~E2}ZT)=Xn{K0&`&OWVNK zr-^%dyc9H5;O6>7OQqZr7F{Z8>AJctz@QVvkMcDW__ms}p=p(rQcbw0$M-nX*wARxKTlDz1M9yXfAWDQa$On@ zDQ?p7FE`9*6420GI_MzeWuSF(SBpA1t|(1m%q-E^xpxYnBp}~HtCJw_MLQU6-jj9^ z(Y+^Ci?EELQG@UoqF4>co+J1$D0&23hJx(@!^0Idr|c?ARvW-Itjp9^K1Ho&Sp7vq zVG!%J6<+(#D;B?_xWqg1N|5D&p?xnSU^u{V+`nQSMr`!s}(-G32=OTG2&_XWFBF zNhLW){R@8~M#7|K+bU1yXW{nRGIvDehV2KK)gGj%c zz?^9_cp1S0+^LY#=l1J0!w!2}Ew5S)by+>(fH3^zjZowMKa1MY*e6axIhrpdab};l z3gydu)&U5zW_v!s)zgU;bZb**UN&w+2kT~5b??rJArC8yf=`ggwUK^dwMftw!toJ9 zRUf5+qTBwiM->Q6brn_l;!Veb4ScfUfxU1=f_}rUmT4)_en}adgRyF2%vY{?c z4*|y7Oa1I15nTd79`a#8iYtK$d#Dbuni+3PGKs**D4klP2?4VK2tQ2Byt=&4#xN6=B z+&Brqd9hrc|1}P}%(!wp?{NS%T|NF_as6GSQf3~|=E(g|ty1XOae%2l`1g5*Rqo`{ zddfUbE`R(RFqA%!sq1#dd#e%8y}LOUuu)=BIeX$?ent0=^qiq93?$ zV#C}NaW7;BKFifU#9wyx1Y`JcQMLO3gPy-4-LOJ}tT1Zue~a@!Zd`tD^MSO@KE!aA zJyg*xaDMdne0oLx2Q{i=pM9rY*9>F%k@sdcF@I`${@ry(--2(X+SV-A^msJ?u*#af zT}Kw%G}$4sDxq}|?dD%w8>B*fJk?@fOL1~NDkd`5C41D{Um;2t#f$KyZ3(f6W(3PJ^2fc zK7IPcT|;7*f8vZi#+?@C(Dm0YSy$`Y{}C zT=ITPCBx(H*JQ3ie&cBd`bqmiU0znIq&p`q{tvUmJ>#qWnUs_vz$YZoMe(nG^Eypr z%*YC5`+OY*pgz&@=@VUi_Us8+i2uMEpS{9GrhC!U)wP?OZ=SMc#+#I~i2|Y`a#{Dx@Yea;B|4W9)bE$1= zX<_<0Z^+*aBM1HxmrP$@O4eHZ@7RZNiF*D>f**_~KX~ecyIcPJC#WM-H!d-P3urLcG>QXs?&n8%Hb%^&t47)K57%@-rQ+E zozqTCy4xl4^GDxdhN5fNrbkdA7#;Z}-R^rcE(`!mLhLeF#x}CcyXB~$ax&}YuVhIf z`?cReN|gU@LCZ>-{KnB*)_pqZBh$)F^8sZQ$@}>V{-Bk(g^$l$4V6)~0 z+;TAU@|(<}u`Pc&`}MVzq;*M06+&NIx{3^h(-OK@mt$|wQGpa08nA%%u5{wLWa=I`UVd6`hr=akgPF$a1F zP`8XA@X$03u3d=E52d|Y}vS>TxV0YrKd~ee7j}Yky`Ss9Ko?rXLE$EZH%mKPF??1 z;hV7-o%(0pc#Tk=BOmmUsBwf8^?B_2x(#Y3VRFWA9thdF*OLMhO9OfA#0+4@FbH)u zDF&z!LQZ8qsBJh3cr{mzsuSMB1m=27FKVY<^HB_y85A^7*+kp2Nc}YgkNeU+Kpz^R z&eyb6z&pkvS~}Z`^8GRl&^|f=qyvyc1d%OChW*g!{ys1?`2BlS%VCitCWu<}$iVzl zr6wmP$f4yrbI5l)PTQ-nXQ83xEUh_M^$93BIW<+n*UQ5@-yr`DB`5%xSR*<|FVge%tnijW6SaQ#H>#m!xxrB3LpSG0h#z0EOX1A6YG-OW&*WMS3ds3NX) zb){vastPy#n*_QH3J>XKl5-^R zYXcWy*L5FefRZMzK{tdE?$@n`eaZD2Hcw|_PizlJ${MkNoScnK;%0(k}TC#;Ve44 z#!R$#9qWpKKwpW|ZFn^&q3TfA8s2BLl3wf8qG+3v;{C-M^y>A}`SLq(wu=8v&Vn*( zc+b7@F4D30_@+Ir&A7(*+wu)x*2Tr>>Lci@p>p-QqxQATd)BTaw}#UezqNa+JB=qX zu$eo<(8ZhY?qb*LarZabEp$K>!J7V()~n}Sv@JfSeTk63eLxTa>~9HxyUQ(8{dM2* zhQp4B?dHrxMMcnr(R0U%Hsa3Yr$gsmY>|TKI&{@qjmk4hu4@g0aKeadb|oefy~{rY z`T9>eUTU}4ab0EIAp5t8DKP^ppJenWpTmz=v3_Air`=A=LG6;Y;P zW56M1?+v!=$DL?~fV@tZz#nuI9liTMw4H+L(c1I>ZL;RSnX+c<(<>AdC1+>8?G)O9 zDR0(}TLf;1k;%Ve;xe`wmPg@saLkL(b;m=Zu+)}o2NNq@`zZQ^v-Zz2tz1>`R}&CS z+KtNi7+7Jdt6qOx^Vd|n4}mu6f<&#f2BQ4ou6Wq7R(Xa20!0qIUSku^k}IEiDat4N zJgzX#0#P@A`~f06Xpu=(Bv?5M;^~6de?Ml@Fqd?my0i6vT)fV9e>q~V%XuMiw-Kvu zR4bbWoB7=3|L?nK^FfM=XmJDV{;R?_g12C_?{b?rod>s4EYaPBF}3HW^^~iw-<_9C zVlLM_fuW8a>CH{|_Q;Trh{D*z;_(Byj@5h`Ktft+Bj&QUb8wlvpDL~H>NU> zHQf#%_yIKA!OiIF^C;SjAK{{DZ6ECgqdxdmihEbr)TW;C_7wP;zAT(qOM6|Xp?$pO zm|0)5{p=2?8J~=vAZCv0R`5WWsotxhcmIiHbv_l0;Eg>@%M$ zK^Q;bUf)rdi~HIt<3!rUzM522AULT;l2`CANQkK>Zh^OgQO_w`rnb}VZv`=^IKoiT z071Yj(aL#LU?fC&AfYnnIXUQE;nWy=0<@DJ(PjL6(py!qA@&tE*qnNP>8?z$P$9Pd zQ*QiGJJx|rG*ecVaVp(UyNpy>fFSjY4l5W+j+GPq?=34OLE}iPt(WqGt2`kqWYu(? z9*bcZ8;&I?4tsra4J};vNjU68aKLsoxr~%59K>X=d^?jhZ7{`E0DCnl@Zo@Q8wu27 zeFnDQjJ})fmCqZapveM(c4untbLj4dfu`joV(E@4Y5qRz?k>ptY;Vc`Uo;(-DLDp6 zx|>DfOyz#szNT+tTO&$Rpiv(Z0wfg41#*jd-b^2Ky$oloqHGsZip3K&vU5Hk@4mmJ z(@uO&#h4>U?Z2Hv=Xor;=y~%m#;@i&Rj@HeRqwavDBv&;;JZ>-GTmK73x3^kQNE_m zHZ|(oK*_B|w}sZA|Nlj;OzCF$58sPL8x>WUDL#>sg)VZiF!Tif+PU!KRxfPwx~Uem z*-<)o>6v5C+~*Q+5b^!ZCn>12w}G1kwcayX6?Nb%+~tU(EmmGMT+%rFag-n`AT})a zS;lpnaZIJt`Vm6SEK*^73TnNed}Z*BBx9!6`B2q@_X+fy@w|MQ8en;C(IWmYeXnZh z#QWH3US7Zpf<@f+^5n(RdBkb^VvVB63UhY_E|5_H=~Vs~+>9cVANjk|HQf}s;7DzJ zj}s1PK}SB|uHVl56yy^+T&XbG^ywK-=R-~jg~#^QG7)#}@%eVUyczzEfKS?F$=hTDoJkJjmDiy{zcm|@aIGaY=xY$a3rv`NF?wXL67|9A zxC}YDRx*o!QY@13*)6+;%Ow?~}Y_g#HyL5))En zIP|>-mXudq_v=yLI!DJBo8an2r@llICsA}a4Pz`SqzcHXro)K)eXhHkIJ!%e=~;U; z>JoiXe(rZdC)rQ&j(q8Uxg&lQGnCMOy@T$`xaCk10u44h1kNBKsP$MTlPMYTJ#xu+ z>9$E3b?%R+#b%tg8BTejweKW9jhXzTb@nm@vu5ibkdx3{l6yD6+?E;12pHjeh|{;b`YY*iD@l zDhu{}@Z_p4zN>H{mEZ-K*scAS&TFOk2qOO9CAM1oub#bvr@gufW~gl<{YWiOEHb(! z*<30Hibkb`t+_rG=A<0BjQ(nON?Wm0s8{l^tO8s(AO3=|jzCsn_MAh@O;wR#+WnEO z#OI#KbIFr~TcV7X%KQ8%zfTvP1xDvr+D-eX9VMYXy;9_U1A@MN!C661Xfcxgc)V$>K4A~JQZkzn;|7xn~ znMvh2PqaxgqY^0B5;ir<)V_V=x_7aayXrCF*-~`ZR<62=z+@7HEaJa={EcSmRD6i- zV?^juT}>_9i@Ve7(?I$ke{h>#fdjWaHug(Ir~ylj<{o=J{+IkOjb>>4e`WhAgrneq zJ8OQWo(ocTvs`ou6Qn7vU^=7Cm`!I_{z2Lnk|lS{+whTQYjj&2X~Cjog)+-_En2E&Tirq3RoIEvsjUW@lIbM zXAAV7gSej*tZi)Wo%S~W(69BY(xr!!n=S#N!Di+_y=6M&J_jW2Ec*hInvk0tSObWW`i-OQ)lt*uU^ztKLey}LR?yir5(4&*cln$NHP zy4j)9&A&?1rN;6+DF_mHu%MCiap-oupL&Qd1(WIG%Fz&r!vnv2@7Niz@~}S~0kF=~ zoh4Ovd-Dzg09B+)0Z_#Z24p{l{&sZ~hZ3EB#+K#b(b6Y&gP41aajEhSi;^S%N9B}9VIT?sudFA zrSl7B7s`nBY?&_&9Cg{M`ijohzA9u8yrJDpcj;4KOeShJf^*O^Q6+6`;Q)cD&*s&lK4a&|g^Bail$Dl@9xF{Z^&Yc|$hd!}lpL&7>F zs_2A-OJkXSYy3tvF1Bl?IMSJP%n(Ap7UON>H=yJ%sSpcS=wfPVBrL=LS#>;}Oxdw5 z4CE11Qr{jXK*p~lHV`WAUceN<4ik>kOR171hM0zxt$CiBtrRs#FNtVLs_Ha&XcwQn zUZdf>GRT#;#)M#Dv^|DO`_!zA_PKMJf_dP^Ja4X;)*?&xPRiK<?PdRK7 zi9v3E5dgCnIRB@9{Ur(pB;we95;nAC})T8@{EgS~h>hd>HWp9`-a zN5ahzxXkhDQ|<~mog)p~k#nz4=eU0VXVZG{RKktht@=>@;B#jOLoxdLRD5ES=1q&& z)@VY=SAbN7(78zZuV0g8JsNxPA>#s)#5e4D?Pm(J~p2C)tGB! zbS%Nq6?L-8Rd^;*Gj9|Y2Uw0m`AT%5$|7F`lu@xa`|^i(HZkxM zr_g{_=4q4G1IlBQQXYYeH(bn=rLvE%om#t0xW5zmDOCbX8Xf_;r{Lg)1OfEm{-4Z2 z!#W3%%gY=~nF_uMCy{~vOZlwZlkxdtk%gLi)ZPANuqh=`&5f2FR3SfjSRPOZ0Z0cu z@Q(;qSF)6thxJOk!k$8t4_6*1uq}Cy%jz8yTJtv#ZOOBSu-NT-jZz{v$65mgN@b>_ zIg~;Y@=NPS+BDRvo<1KFoBM!H-5ZnUI+@JOlq7p{bZ}S!3fOB*=gDnYG!&_s%s@xj zbtusyWS6hL(n$yZQjOql`_sw)<)&Oa9aq66_`kytaq1$aSs+~nkIGM|>{@=FgCXoy zHm7UQU&*t<{N8*XvVVI2&TMZd*)saF45-gS)a4n11!k@Ir6Toor_6Aud>$R_fIH+uhw=bIg z3rPn;sHEFfY5)U)`3;2yzf@EpPf^u~*SI6@i}CJSKJw<$Nw2Z@YcfkWZ)0L(`8IdF zG>}&K9-d7EJRuOk)Eq%U{}v)bE^MXflgzqc`hoj5>UAe4PsNxLvAok*|aw6EwB)0~wz zF#ZxdgHH_{_m##R@2h15MeqTOI#bNdbC2^;dR>K`^N(kOCU3(;IFXe?YC?y1`rqX6 zhi6N(b$pwvj#jV!6SQ)jLdFWZ%5;W7;25C3p%T`7hLAsudR9!8iF$ZNKb0qWcMX^-(Gb57c#q*$o!_|s{g;R&rJasu=1*S3{J)1a`r)^mrf$nS%JzR|V1qm-e z`Dcn>R)3O$5>C=5owGuEw(DkZK4vI$VAI|1RQ z34%rj7stV15fkG*q|WMw5OYdI11KQ{K@T!P#)s64ue@DA{@JvRr% z8BkQkpBwFQ5Qo81(>6ND&C5*%4CFmzdKKlI^i9)vr19e1XG|E3e7vf>tGJSf<+}?+ zX~~)x$%*ByRiFIWOh&kIoO~8Gl$=t!l51E>lt*MV=0;*;e`0?OvO=B;6W~Xy1G3)H z92=aXc+GEkoa@h({15_SWS?4f74Vaut`chqre%dy@q1fDbK=E-83fSJLLe^*yDk)2 zy}X-t5#F=G{P!nb?&8WiiQ$n?un{9rSrEB7UeHRx%7qu`8{|W)6xbBOJ(XqY)kJ}skarwVvo-wA=bh|^7;6G?G!GI7d=n%oSv^Dkj$z``<35;DJTMZ7BF z>p{-&6MT9nZ5wwawwY$}@!2mT(@3*G8Er%PJ&k@KilU4CbV{=6Kt3ER%IX8??b`l@ zO;IF|uAs4}Yg5H=oE_1R;7{TWC*63qTr~{^P3(Purvd8%dgmPif~@pv)Fc-iNnvqb z(O?5UOQ%8PQy>byamAzuNaSGaq5!p86-ACn8(+U=vp;ziGCkP-QadA<^XK0ZFFjlE z3N}l-Y!TVoZVxgcSgLV@mgoh%mXlfNCESjC#FMFyYz^*IJg)Uv7)U3pi+)!&Q{D4x z)cpACH^?J+bZGaGDMXJC*`I69^l)}^9?ni%j66?IgPliNg-?DHfB$)b*NO-}@eKc= z0cK9W10NGUw=^GXxe{6=^gEddEfZloxf#}p#3JmCbP1s-1?qe%ibd=k*OkU=C#mFT ztiguiWZmdq2|nqPR{lAakt`KQ9g7G;So;ED=z$(pa_pAx>RxG*zkL;GS(caFK z|GWx?U2I#2(S)&8g*E)L*C{6I=teK8O`B54IVaK7kSlf{z|hng6k0=|DEbKLKhiu3 zBklr=W@?BSeO54MuIe$cXhLi#+I{pWMIi)UB;Jw-{QRGMr*dASu?MIar-SCldewdO!-rD4}YAn^7` z8*)U99+f#@yrcdeoEz`e{EoE^aAyH@kpKw==owgJTi6BORdeQBonC=wv6dYp5*o`T0pN~3OMGZ zspaV~Mov;O%~AmAt(M2-)L>%f@BPGt6ubg{-y{?B1iJW>#;zwwsxNcCJOf$$LgFF7jxkR5h7{HlsB zXcUn7748A>hwEDek5eM&*<>P4k)6M^=+e!%+0U#Y>}yoo{vu$}ajphpX1J@m z`EnIM!cxZ?YlDM`1=wqzxsn1#K>-dT!29!gl|DJOziZ^Zpmq~YSiYz(>5ymR?ZOhT zg+S@CaS8>>%5ug-4SnOjF#|Lmvz_{T_g!j#qTLUriaW=_LArmOUd}dg+o% z1ul?F(~S~sQm)>k3sv)z9Ns)DP6rgX>Z^5+m+m*OzpGrVv3zJ0kdqMTpDeWnSN>t%h7`ug+_e60QxdmzT zTPgw!O+D5oi>|)AKuBb{ZkXt1QgRHlEy|h=6I&>mH1G%dG86rwio5VJa^_Xx^@!%b zXRI7T(iAjLcj-$Mt=q!2f3!KLyH)(mDo?lY6i~!5I5PZ$4@`pPg(~##*v*qoLPY>} zEq)TOWg~3ZCN>vJZE{TyP-5e;k?IWJXn!Rz`%uOl-b2J@wk4`&SJNFmiy44qM2l_2 zf@y6FX_N1d=IEavwiR*iL@5)Vx*9aDoQlBdUwX9XWr^x?i45Y=%%0u8l#7!xb}ANL zGVF(-ma}e@@5;0#sV*B)5;sl8n;+Jgsr{~(u8W=PPL}-kv}5^iGtt*~Syx;Br{YVe z=Wd5TfCktd=bon7q(FV^HHU;@p3OyS zxR4i6u+&(j^PUUR11jS8%T>FVL68S2#c=8SO)QeDs{u9Eg#KrLd8j@o+SMTb zfO0C;-+pC`dm2a&7$yXU3tEOsvBkOTP6C47&|z*cRrk(V)El}Rg**W-Xv-!Bh?%Y+ zN@79w_Q=t&{%T#_k;bX*XJW%NjC-~eE{fUxN%hm{g$XG)2$UQhK8YDN88=S-N2TKp z1S$*8b|8lvdzhUykxbw^8Z_1NsHYv@*3z-0m)YZ=2GMps+a7i@ZDriBI?9eyPfOh) z6`cO)(cyvtc}o5CMq6dzW+`!VG0MMY@MtgpRDl0W0~zk#FDCSf!o2ROI&ZNc#itN@W}#Qo zYNQ$Y0~dc>tJT&bmUk>Ba2`QT0=8g>34^_Mns|nEKyK-TRI0fch=iGhb5n>QYPpqdED1 z$iXXJ$2#mE4{Q@N6}J0tDaKa=o3VAv8yJeJ@#@m%Q7% z$+a?FZ~uMA(h_$+t=+<4=5g0amY0_+4gA*SvToWvH*Y+=2p5-%1|7hEY0~h)JVHvg z{7YFwvxwbBAe~|*(EqACb)_u7i?KHEimH1Ah(M~vTndN#v960ulU#sGlCyr7#>oxr zXkNg%T?6R>6;@P>`Sw>6syRH9_(s3~YFHSs{!8ji5@pL|=9}nIYmFuxyF23Ev((!X z71FEKf<7XKh9NF_5j?4EimJDoRmr5eOY>sao!#yS>tHBwI!ja}jJSM-Qt#J~dTnim zhf+*L2WbM|vSqkcEXps{QYub9yv9lTI)23l3en-F$B6Kq=Rf}4cEc70b+7R%%$eh( zyoj5|qa>?mm5;=BTu;wZN4=)hW4nCBvD#Mc`3F^<<7yIvw=j7qd$`wKJ{#`l2&lxzMMh_kQzQ0bwmc?!3u2~wf?nE!IUA`<~x zE&QT?`%(khgY(-VS6?7kyDubZu1|@2Yl^Eh)^t>VpQTB$K(yf0tK*d=6=udK|8{ZJa>|uTp5n8NBj$2NMY&^juO&Lq0#o0 ztvfeyS1Kxt8*bSIl0MhxHp+2uD> z^X{&v5q?H|nfq?nJs!4+|8XSCeWK(4W{I2+uR7;bQCoHFNq~>SmMW?CA64l8ZN6+*mMu*kU$5;|= zNF|)sh1`0D9nRav6lTzJ`XuPj^|{`JP1V?z!M%CI0tRUO_r$IDhleqJ5O@#GYwDj= z3t_j)v#{0wVEgh6N!~kj8UFiUXXneH=5dTX?b&M*I%(eByFn!$$3;K04h;agniRwg z?z8fC=8aumdf@}^s* zkX9mIzBI`*pyV<$xGE<-L0f6ja*{0?EPR&v7vubu)wga6N7TBqx@W&_8k5#{J!WE} z3c0bdmCp)l9m~2@Y%Ff7 ztT_pjdRBkcMthH>{C)m|LE|qZvD@O{I{u^?{?hIydHWHGN2c^-nVgg72mfE zlq+)Z|2L}4%AlLb!hgGfmj$HB!Zx#Kel_sRn2IGaZmviQ)C?2r0$(*RVyU!mk)UdSxeQ^yY`Rz) zD#>!pb4o;|M61hY4^)*6l9LpDxf9zl*vrdkJTyW28K+zb-kW`Sp`J|G6pJdZ|4Oy~ zg3PAD+X9=s=xbss(MLzu;asoQ4}vgx$A!W#QxK;xy4tS*wWUJGW9rkb9Rs@t)S<3W zyQ^^1N22HMwvLUfrvBEnzd{kOc|h>*Fa2lSmQ3dU^^-K-{qkLzXu?d#^G)wt{yxvl z2(c-3gd4YX6^5R*`3Vboi&)q-R6>ijxj?P?G@lyI&Hrp@!MgRp*T3!ByCvDspg~ww zV&c6&3a|Tw<03!nhK24XhVHtz<>nsO=f1D`B%C(S z_4(s(-0TC-90?~9u=Zo|4|u*-3OcBx{p&rald)yc9{f0_3x$C!wmP_MnBs&?-w zMU&(|FZ$vkqNywPfEhw5yRU`Co=*Ya8F^6^1fYf<_y2M- zzObs1r2=^*TA(K9NOK&8MQha9)_Y7u^#cSW#167scE3)gG281-KZGtA0<)X#Qq|fW zeH3!Bgze|vvNdNNcUNPr8|%--PdZHl0V~^|Z)Z*;l4cc}{O&)xoA#-UT%882v~o?6 zWR?&#kcw&qJsRp1$UR%ckUpi?qpEN^6;N5zi6xtJQJT)W-eVe-3N1ZrAUEyuGe~edU&L$2wrK(+WFBvJsLJNrc^;1e zPA)?dcSHSUA{xU9o&ngL+e07UtngtFIgP#=8~}7zz)2SUb*^@|+qr*~#xm3hyUNC~ z?S9maoXdRuW_>1$v-x6c;yBC5ujtLLB@pTObS^9yo+xjfL&!iQ`zob0-Mq(z^O8ow zf5mmmL5{6xBg8nm*J8GZ+&t$;@IaR8D^sCpXxEPJB8uPH!G3xh3yw`F?a{%uHt-z( zr8X~$HB`~MipZ48O`K91NO_0|W?|k&XZ)gFT5hi&+Mp|`maLn+U6(H)>9$V&%RVN* zXXUu!icM}3&+A z*;(C!&)rq}k_6$>RlD3FcF3K9&xWu zeEyhk?7z~{;(wU0r<#g*J*MXS`nOgSbcWzvHzODp+bM=FtIZ%v3MrcSS(Erhbfz>3 zpQ5jVpF}w4YbY+WM@1H6?t4y8&bnB61=j_V9s2_qn7LsWSPYdBUwykkO!0p|Fh}!C?{87#?*fag zAnwAXZ|L;GmIht%=ZonZjv$C0*UwAGff7IGBHywxl~H~b^#lv-WJXeHN=(Fv9v5=v z5LnB|)k?(zzTJscD*t<4P3~+$VWzDL{AgiqX!%Qp^jh?(_QDy{P(euNmKpH0$}&{| zlFYNW-k~*VELYFw1oIrG{Un!(Vjn_6V#C#(iQj|GS62xfHy(kUdm7j`)n73r2*Igt zpnV7wxhXJ_yI`Um^S|iO@iU|t-chlp>A`-eTTglpjA+|22$#-`X37rmHN@G65Q|h#z zQN{K^Nc>ojr+?7s0Wb*yEdtK}ZeRuBzQ>a5@jt1^?GeC7%(4EDvkLN}uOh*S${Y?T z@}$6(Jb+j%L0#`PYcMGyJl!_GQWJj#XkXghvBF_;8`l?-@`~%|w3N@GcK&!ok1- zNredrIqnW-X*d8!69%l|tI+B(d_7UUgdMK4Rym3eq-6LOz!R)k;ZMM#>J}@Fjcr%0 zPY6EtLCg@WPT%&E`FC~_SI_W@*j5j|;l!)enSdE$=(001=YB?R#>z1bnlp^VCtNpn*^**xY{Nc<)Q(yvSA7uH%y;gNHhkR#ZlCL$N1P^6p+|+A~ z5jgha^X4mMMGRHq(CO-Io#3$d>4?q8QEvERWy_RCbde=>A6an$7E3$D<`4HiQp{X^ zasJW$EAG};_M{*aZQCG_xjO!j9wpIAPL_AUv;hN2xRLI7Y|Q|7kZmfD_*+W>Qw&$MDRO_t17dnM^u9d- zd-<$8mVVV_>NK#FG&oAI0J!Qh8<7&pwhO4Ey8Qz|R8p2c`Xt0D@n!6Wp4~ZBIg_H{ zswbEP;OqRGT}P?S6EJYUWn)4@w$23k=>Zq9FJe!aZ+H%gK!E*l0c#QP^}Klq(eh1l z8YG=^|G?ZINgn`T9-G+WOUPFu)K~DfbyCTHHwSF7EYlgmk2U0X1(2IzJb+7Hyq4J7 zPRSR@*BWg6Kb=yY5u*G_<4&cFoKz|DW?1#-Hv45@g}f=u5^hs~kbrpHAG!KMZiV>rw$? z$<(CmZ>Zwyx-`$nQ}r(oHBCURdzBx90U2#R9bAoHpR>ua{7wh?d5kBgK3JvZy2Z`< zu&zou_NC`jwyqN+B1FgcwR#7hgT74tpWjtJ%gU}t|9ID1%(r}0<#%)A#WKVfhCA#P z;Yb7jb4)XIp6~J-5}$r}59P?cwB)(cDu-oZL#`aX`Zhzwk9RAU4l|bKtj6zmO72GO z8&e-Txd&jWS;4ayp(SGyiU9=!ieB?j6DXuvnl%CAzmLs(tbZYDtF`=}Z->&n?T#r! zv}T7W;-Iuz=DG|UuVj}~hC_LT9qwNh388KiRz3^YojPW;ruh8`2XC{xvt~nao1Dtn znfMPL$y-n3&%X%vP&^Z(a!&lrG{L#p-$l+EKgqm732Jp@GieP|pE@2KFYw>!pV!|p zrxd}pzcTPCpYXQvwC5%-=TFeBp7?KLp_Lq``1*6v(O~#?{;U7MB49Q6t3c8monN7B z!CxT@D-hA|PaoOomEZ$$flEZ~ztKt_oHPIF`LfO^wf?uZvMqV5dIX~b0E&zKb0wc4 z9JMblwU0p;3C6cvlIO?wXJ4<;JEzy8{;U96&=C3X5x83!E;7KuBFzfoWGf>f?NwdE zQoFcuI-X7is#A0YxAZQbUfllrqSLiR-Nb-6v$g7 zGQUvjM)SQ53wviUh1c$!{n;P69vLxHiGiKG!9TYp;wqMYKet4$o#g1leOVF3>2qE) za95Gl@W!$g>wePlX3F?3!T9>G%>Adlk_Q#?`Ty>7?)TV_z(iMz0`DQjyE0NpO^5dh zv1HE|xPLUO{EDFYMw28&>m~-35N_qbjO-#%;2GwWwKjn7avFG&>cacpQk1 z<57y6H>K*akoQ>TCZP9_MJA=#Vl_QWc9*a?H@@e1l){3*XAf0WQ$0_E*-BGPf0fI8 zRTSyj!P*5T+4acOmefk~Y~0fBCY_};ujIW9aA~7}siQAe=RhEa0I(St1KJC+aMM+I zYrutQH{Vzd+JMUbot``3I$e`&tS_J6LFqCd`$(QWmyVC6Skn$U^pePpo|P(n-!r$p zaU67o2R;R}0jMv4%G(1! zT5ZeR_b5gA4`};OC>yr}b%YZzys%yJ{LU7y={@1;Cxeh8S4pBoOWsZAoRYW%OLs;% zWK>w<>rhuduhLRy<*b@C30eSZ`Jv&|IaJaEwHxG^Db&;x(K6h86pqy9O=M#@S*c=a z2iL)&pm2qks;c|E&hH?m@a}$5^5bvovmh+qW{%A&mOs%Zp|Ly~UB!HW*znD;$^4H5 zH3ME|oq0LV+mZG^j8$D7j{kHX6aRb+I~oyruPVV7$)k?T`Nt{?Ln@5^2cavq-m7Pp zS;=-1ny&3UEzicZ7F#G;_B6q5OmiO|OTr<e_{NK^^=?;qCFzXh4>=-xV#V_%dUh_+kwCZc`7`|er`;Jp{V~_3d zrG|rbG1LoKK;NM-3*>F!?JLBO13UA51?8qDig-cYR+tu-Slz`FltDoU`(^I*VORvc zv%wR7GR0jp9zdkR%|yCZq3Fkr0I`+VXP&eQL4=2ed#*+jJ4o{ zPGx^nNFIWvAG2G_^8R~=U99NDrr?;A(zDBMT3d@- z2i^eC=Rqbiz`Xhm_tbSeBVgAm6#PnLEZui#w&-P+;B0tg)?Y3d_T_kg}2bB z?NHD>EX%N|bXF+W=IJP<0b`_tBB289nGqv$< zLTXH>u^C>1(mCPAPIl#;wHP(}>vs)!1za`W!sg}`BZ$dX^0sLk$AcDzRNNbV%n;&l zCY?-C@pzA+S}uvQomE{Bb@=P?eD5z_K|J=13c7xN$<4D{r#QZexvfjm%UAUr?nJw1 z4{kr3|3kW$4pT@yf~bRTnS=k9`{XaB(iMs?3T+U7TNz+u0L%RKFLc`W$iDn^JB>E< zIg)pOvyl=#<(2(Z(PumJmvWQ$X0{9!3!S5#)(#K`H{M&P!rmk%Yo53fd)Tq}Ngjgx zD~jG(EL62RqLtuL7?5p1$`YaCbKK!wRl3DOIi|udAi3G&e+-?ENKUb-prrlzq0xwu|sS`NwEu^M0B0!&F5 zbu*ElC~}tcbt1uj8dV-U6Ow_$-P7lXiKDGORXNfj^c>1wQV0U2Mq)QJ%>ftm4z1}!9Sj~HfMdGHe`$1o zNkSoOCeL-MJ$w1zO42RCZ#x>eg2RQ!TD;1?)zSr++-qkGeh~NP`4c4on;YDiIQ7MY8sQuI6izeLM!2|Kjvll`mz*7qj!+*bm$Y)*HZliyQRGX+3v`|`l*^33q9cN*o(ILiu5Mx4EMfT{o%D=WP{{M%ps|t&%4Yz|J3_S?ajYu~N(hMEaQqo9EcXxv* zDN+(cNSD&xsdRUD*D!E4|2cQ(h8t#{y=U+5?e(q&^QscM^l0X!Ak61eh$fB3l@eJw zbY8p)eK=FFaU^H3>X0mR6Y*OWK%sHuyv+D34z&@m>uY;UQ48z>|AbAbaYIdF3{h56 zK-vRmGJ~P&p}^5Ry09EG9Ew5;O(QlegHPg?Z51woWzP zpLQQPaST?ydVe+_O$e&{-6^KGhA&z3I-EfCm?d?)2uW?g0ws&8>o_g-CfXUlm#+2b zh}E|bL|sFIaqAtupQUGOcmy0?Gtbx6iY)c}U~;L@e@&sPA0)QNrZI|N&~k-@HB^Nitb!tHkXZ>?(Hbt}KR z;csQmoap*-CQx5AJRbskmE+KTiiEqOp?5se`_Tea6&`Ho@@0mO%`Ywo*xk5JIo%*m zUp0GVbL-UfDg%uqGOx8oRu&Zy0NIW`_L|4{Egkl%`B(sx4>1a_LyFbw%bP2--Jy7o zIr3af4N&SB^Uu4y=M>fif06PVhP|0)QZLwnP?_S=p^0NMHi=)8wA>slm zqWVw4pZZXg$7`w=T!r}<;i2Z*?^MrN>{PCSU(Q@6`$}1I%FHwFvJ5MXMQ%a<{MB|0 zPtSLwm~CjJevDP)OOybW4HRB#ki^bH^JcX!IP;afeDZe2!wN|XYT3ur=2T98%I6H2 z;^WR~BBd$}ZTzatgigZSHt$-=3ozx}%)ZuCvyPAu>49lTQqp&A4$;00j3xqyv$^N# zaI$zAuXu#d8%Zj_U;~a}5CMxV|6rez-gr3kxVkZ$gI<)|d9YE;?s?57-%j0N#I3+W9 zHmMR#F9w2NOrO%~UWvVSPng05{grp50iM+3?*09vj_z{jrtMm&n?0KnpdtVH$_$WO z84$97Y?N(>;W;n98k}C+m}F;DA?1JNQW3``6=5@+s033CC3VPj`UWs#Y{yi-`ln`D z2^aYd^c$xP{!5#!jjh;@zQ)d*P!jh`2`JW#DG#KuyCp98@h(sk2c-@~csPqLY3rdjxs3^|f->K|*~lf$m>o+w`E~_mT_g-p*0{q3ZCyT+nKp#pI0wUkqtl5fpE`r6YF~dZx zGe62TFeoHAcT0E2%G(hO80>gZX0yAIhX5KYP}7)+D4SHd%qX9>Y9!^x+lnM?gU|Z) zbA}%eN_re=oqE;}{necCOdMXt+cV5zCy8N3@5I#0906vyiR=8r^n=mZ7fcJ zm76}Hk;JRw4|SN5*nRvomQmTKKsA(~B;Bm@`DcooGDpI{xl9uiXWK{I0cP>4tH39V zkER{xqB`eX-pFZIeVW}m>R@4 zZis2I6|=ezKl6*~84{Nn?v}InW@s)bX!pF^){`B1zAjH*ZwDi$333SZP*)u;up{>H z>vz`+FMukz)?A4nS${V=pw@MW8U}kaCRA;xP1C3`IVM&;MKM;Jr?qi-8D%oFfrf+c z@@Y(ex1Gi2(M!78L_Kq%@PN8RRD-Q8$K(?1Mxur89J1u5iVf+&^}7(-kQgqm(Qu)| zPr(M6SXF?%JN7vDIKPg+cW6z2Ul)Qp!Sb(=Fd51bfZFgp92G~Q;_sd8OWlQTNKl9g zsTu=qs4f90!9g{@Ml2%)*|^#cxSJBXC$g*-2}nKa^n%G7Mp=4XVWS&XXWKvGa{VL< z4r3CQ=(G{iR!0E$YwA(1BD3iQ0MCvA5ag4XP$7dk#ZZEtsf>p@&@187;CKfDW|Lg|{OLj9&}<-$f#okN z#zqrDsIotvO}mrfc+G<`&1V7W44K-Ineb5aE)zWWF}m*(u3AvyTeDVP)jS%K zjOWk9Qdd$)Srm?21&b5kNtI8qdS#1xp+L1`gG{>%H|oP1e|slb`)j+4R1F9F-^hr% z?6rh%j0i{q7g)4CzzKGR_>5+;zWE`S_JWL)JUxeOBq=Sa!bR8!j2V~;S`2Mx|52>l z7z_b^zhe%ubo-?Yv$2a}r`CM4g#Mhc^u?J)t6b^B)+O3WmzGByLbQ#^{(@vwc zCSzGR00vx#!tpAnT>zYYtemtz3>~I(k@0<1y2uR*2qGFuPTMT>Xtrpj@(7hCNEC|5 z7RE(u2SsEXp2_2fQaAkW41oNS`WGo*wAKhA_nNQp4&H(JpKX!lJ_iNpB8Cs}v(O~% z4=fmAekZ}AXT$yovTJ`w;MvJ!0QM5R()b`*IR_p@x&#XieH*<715hx>e$0ZXS|{zg zmaLU(U~107U%2sCxcK<7~ z_indSy_YMg_p|n-aHiIi$H#8#+IyJc+4i-&8E{$mXKC@DC{ThbEiA#Z-RLaFj$aw{ z=C(NDj4p^EGU>?}PTDWst(*-Ej3Vw^Z!ZnuNxpgsfX~k8JOVJSNTvsk z24KrcHjSWXwXjh*Iuugh{;5*8$4^XnkAGS{9%%Y4KhMM&$Rtw>IkXS(4dFbasR3TFkPM!FC1auOb*8&y=--8;{FghFf9>E_WYaSXTl(-fQa|F^D+@XD_&^i%TE^* z*`d&|qODe{wO`2#SU;S^Jpl7G%;md34_n3@T|h-p_=G%n@R?3yoqCP&GOKRA%(PS( zL?Z_N5VW>?r&y{@I*-j15j6IRixK{v+S}ecOAJ&Q+-a?y${rr-FoXor%w3Tz(s0v9 z2^7YIqfIZh<|049?eC|~&ISaR;q~^O@Wz(4;ub>C$(6hRi2b~q9AE>TpRs|f4up;& z5_)+$w;Ne9YDN>k4vgT;1F|krMaO?<)z<$NjpMi9P=q;^N(K&w=Q!80gsSOT+X`MD z&;whzGb~=PU6XZ<#d5FpFXGO%%orV${GNaSAt079OKAc}QTDl&HpBXKRla!9r|h#M zQ!Pa1+f-3;Q)_tbkGQhG|6^TS)v8bf>@GYBY*D0}Xn;F8Kitp4rpvCTH3;VejO|H~ zg~AD&PqaPPzU```e5(}1+SIMr;0mst;kaeps-F;>$FD4)cF|*CyiH0S+bDWXK8=P> z!>eEVw${_JR^O=zDNN3r@GAOxhT{{Je=DVk_xO6@$#?i)<;4r5dOhagu^rPMVvvzZ zj$8y{dkpC5q`a4_!odlFh->G!&(19%hS%vu1rA;=+Cn6&H+w$D3G-4+aspg~T^teA zl7HC*@hxm7XSp41={qT%_0-ae71K-qy&&=Z(@Pmf-31vzU79z4Z4v#=w!9%Oz3QRVed z1j3sO8!>_$8YVnUjvi8>8Ad{HsnCJ$ZU`~zOZ&CY>gElN6Bd!S`}Ha0qeVlB3?pwl zf3?c7Fo!@w%wJ!hH;$>fxsL^0sd2$Alh zui5!Z8kPUdU(Y{}OdL#>Z4ct#Zj)ZYB~}itNy35;{3kpnXa+$4e@Yn>-}tpo+k3&{ z4OOgwxb+7Yx+7>e$)Q-kiDs@G%cfr^SERNkqVNPurPv-FEU-jFKcwbqlx8bpr3UVZ*6)+H)Fsb%L5!3tFXbS-{zB*Pd;R zA8^-AFYg2^;Rn{mX*{-RNdH~sNjg&kM8irXk2{=Bh@y8`%62t{<1OVb6mwE}wjrn_ z;>B8GzfJ23M53~k3^a;1af|E}Y}*aDM)GB=^uiGl{+CmowwV$CjC;wj@d>?1{EbW5 zfYbjJF6e(N#JlnfP1}TRT9tF}Che*A*H&k_V?wo0IwA{&c9XVqyqIQmj+sO+f*Ot| zA`CC@gTfCFwii24;Lk_Py_8~6T$gcshYV&U``zQhmw3Aq0<~NgV+Hh{qUuz8wIJ5i z>WQBFnAHUw(f?BocTmmd_2=BoMKK6rvqQs7`qe=1p8CApk5*L(YUVN9baH0{hiOrW znj2|bnNd(sJk^wM`IZJlv7j(s5;U72j3;r30As9+iD)ia{HHzFs>xBo z5Xc3F-MLG%pCN3a@Ze4|-Ev|{icw!@pC=h3yQY#N;VI*hc-kZ3Op5C6{gOpbsee?? z;W8bN>~?j2MS{%PL*sgEGvqt6+iPuLJgxk}N<;Vs_4|fsL`@bj9{D|8H2)k=62(&r zup85>_u~du*SD^{X_plmN_k%OlV)m6MuUH!7J@qeyNK#qB4X{chJT)?yrpY>bB&NC zvJt2Pec>ZFyQA`C{*ev2pn9qm3VqfQtJQ<_gtHilr20}9!u4R~2+BdV_10tt2OL1u zkTk1?aC!6N3=yw@SVY76woe;!%o29|Bb9&pNPp+bndVUWKU=JOa)bc2P)N|O$zkLy zTtCHJ$dJSh3-lJ#la`J0=apMBHbH1F!7f^ZiZqqorwXcBwN9O{n z-AAA(SRq{4MPn$SvLnpL^xIe-i9FGtc1=h7Wwyo-UJGsiDhD60=zaKfOAlW4W}sTw z_^xFr__+Rf^5}OpyG`DEJ6C?Y_up!R-#HiXEE?mUUM%mgPhf;mzP3dW+vL=X%<|qv z1462+u`j%63DO$xG&A`>{t(c~EI|#`e>J>p=|UT1VpJ0eS9LAL!c!C=8U0LuXKpFh zX{{ox$~+_;+Rw;}I!_){`AwBxLihqHh^5=1$RLfXt5}Y?=KW-&zHsJ!M%=^wW4iy| z9i5@qL8&ObrBy$F^}UR_&eePW^^YRk&BwRlRCoWnVX)(erQ>qVV*mBDhq?B$X&|bC zfr`%cMsH6^V^2Ems!NRKWN5AUtD9+tq|U8{uWyy6_AFR7XQB0~OU3y$Fh&d^KxCAftBs~q#58C=VDL#{b^tSnlvHqk!u(U3FO2dL&a*mYCKoA zztG})%@o_T`%i`(`Y_xI$6W?eSzmtghi#yT2om$>1X3e2o$IG^OXhhGKSpO*iLj8f zns*+nKqdhR#$! z=Ee^|UqLZ=`LBM355xCrd6MQdU@9}d67%ot%Jv(Mip+(9ir2DZ)9VL!QDP;7i3MOF zFgDzq^rG;x$4U)mM6~~dvOkb-->W~KS;%jW{qZ?pzUdTwII&Gef2u3~9oUQQTSj)A zWY_bAnmB YkCjPy%@jiiCzkaFGe(G`0Hp-K}y8hDBbgDxa^; zWWb^mNDSu+)6TPjAKvG3t|-iSbgaKU>s6}4Y866=Dv|$5X4_4pAL%A&)ccX>epkk; z*)JP{nC%r6XDUq`3~y$9@{QGt0mhG8$8!d*Q?cOJn?Ef<9(kTW6kM~L9qOCYqlDHE zDpK4mIiX0Gzx+-WZMOoqPI)G0Bh|2%rH@Ar9SS9wLzJV7(9$A)UirA!QAXeO$}j5zvC)yms-5Z|zJ&@EuWp z-UhLEJ>E4xBA#bGdTu(QgHBfC?)yJ3-QU<=wF1M+hFseln`^I-3jgem*inDs&lRVg z!Z^?fJZ7&|{NAIiU%U?oW7TutVM}+-aAv>2gU4D-XrCDV zWDvf6D$7*#mqrj0c?tIMW|D-BlNmtpe~2xSdysz=-nW(5T4eL_sU!5pGKbzv+EE=Rll| zJ|2u7m}fs#JkeYLmKkG-*clO=_F!GUaD#ieV@mc`8w3K=&?{AIpBM2FUzrU=&770Z z4X*jUZAvkZB$2nIXXFh&D;k>!jx`0A+J`?doIV)leE}A$mSOB~+OWc;aJV<*R zpNstc{MPTj*YDqA1-?H++}-9Jhq+aHo2Z_dbK;Hv2!f=9smr@J0C4`4~R+5$z8W zh|=dIaQWuauyOGXkd-X5^&D`< zTeeSQMICUMx2%MwyIlLcE_e8Mo1=~Kt#BLM_&l1e z#*#y^pE5XCdoXd?F@t4kcxSfI%n@(8v*}Qw_-f&ozhT&fpf<}#DgeYFE+Adx%ljKyD}=O^6x>6qv%5>~r>P znlNEN6&<*+-dX6K)*9Qh9Wj<3Pv@3(I^j^uHtNic9#vH!(ME8dMH^^flUi8E!uL0Kp+!1Ew9bvsmH|mN8YKt= zF{sCLHoU8bwQubip5~ zKQypd#4y#1WoCEG4&$laVqkB*DPm#W^c<*e%*XmcO|D{Jw&1fJDMM+U&To)zNmvZ$ z8oYH=k&mtg8;ddiQcNrtX)R=D%#3U-@_zAp=2RBp;d=C*2UcIm`!W>P=a}AUFij@e z!WT=8n!f?BK0N9zXE!{!dcl5_8PE=zZV!~8%=*=8VYlV6lF3X29j$&8d%DG@Qzp5+ zlh?>huriKLJ8o^jW;&@)gySG;gDk8TG3Rb`_VmFa1?Z!BQ|>0l0NtPX4j!qect56% z`0Z!gt90i$iDi{v<~7ME|7gfR_|j`*Q89q_ zZo<7#84u*M)p(!h?YkC=sVw1Ck~aNfZbn2nsOiBgW;l+w_b({m%8emitZ# zi;^>hI6FcB4;I^23}3<@p8Lo^X114@>(E+Y@Ki*Z#x5K4cVU*R_Foz|H@J3*gC$1Y zIX&kWf_rV$iRKiD<`swl-!-SeWHD4Bc+?zB;}kj0s-ZM#p)_r-^uc0FKGs~D$rF!W z=7{!EK(T_T9ukN@qPH&D-1v8&{X#&?8<7;aq82G=FA1Ur3)b9PA7Un1Uhc_}(GEtCRIC03G@^wW%uxFGe zk7`Eaw`g7KPdgs4J@tY*&t?Nw!3O#B?*R=T2s8yCCGZ{1Qdc)CSIIa2cM4r|t2X9~ zR{z$^s-ByEM|b_D1r^YtKO-kqD_=pS=lj9*c4W$8l-FFD&)g=oD!3o7H6aReCN%20 zSV+7%oGy)nj(7D9U*PliYV8QUlNcM@FHW|dg_?l?*SDaZq?PO-Pv~5wlWWKZaxJ9>SaLEmJqt8AxRfbm|g+l z7YPI}i8FMx*B$(W4|Joai0GA9( z*Gp*l<+d6peH}qKaS#+7gOmtC!;hjE&3?X?V&#$}aJe!OseU<&Ut6T6LQTBPlu`i1 zLRERIjAR?L(wY^g=D9O@0uO1U>=#99Y;_uXa@jb1AFOy4De*(d4*+51*LbgK%F5`7}cSDwo1o$+n9Pk(t#@$TRmR zCCl3&4^djsK*2~T0KFY&Hb7kheocUv;$gP;p=sb#y@39g*TFy^%3UXIFyaYM_# zQV=P~h?s#$_fVnpqDrAmz;f16HwL&*Bq&pUqRI~z*yYlrkQhkD`#Av(8eJ5+R3BZP zSVsP~V%Wo})2?=r!!Kfrtn`KXl%=jaXMK>-`uojcT<@(}i`I3))<6FWAEylwuzg${ z)L^RH%B&kv|C4aybq++l=%a+_4I*=BA}3$!F|q) z&Ay9tU>c3Wml4q!%ti79qxajH@V$VZmGFA zz1)veY%Mf*=!_o@q7anat9wJ{aPw!vb``in*Zo?rRt(VIW3i0`(ciCAt zrH`kAhVVCr&BqGBP*LDBfNnT~hB7wzJsMciCt3F*CdIuy{~8>Wv0Z*tm^mIZW_3ma zxN1m%y%VpAiC)W6USi*$m;1v908m;i1_DIY5I{^+9}c2Lp#g;K2wspH+gAbON4|o= z*djIG3qyJGprbzKQ2k&CaFW&P`R{TQ#~67mvDe+k49Tvv&=7B)39Xo`ZO$=r-no&P z2B7NNJ|*C&*VLSXl$hd5hf1dR(Qg0sZ)NU?}lJ;q!ro>NIy|Hoq=ZvA05KpjWjnAz$O-XV%pc>-?T~J-+rlF(>3>Jp^gx7r!!9g!l7c2bf5xLUljOhn??g!}9 zOY|{r35qE{Y((=(VbUR@0cF_Ru}1%!_k6++ZwwJ1Ofxrjl|>yM5swd|4;>s2XXTJJ zpG_^FTX#gWy{I)^8_>+W3SKm>IC6qOu^e~MvZ-PyNrv&LUOBQH0^Wgx45x~J%X286 zW-!e2#|f*sPDizI=HB!+u}O$Nlx{kDf!YbthfPaLAWh}rCyR(OEJW%4MfOJFL(EC@ z*ZnHG-1h+$kg!mjTZUYcZ=D81cS2MfvjL}xm69hh)gk9G=ONV^KNBTTV%}mffv((WqyR9(K$aq_fd(exiX0whr*gIsw4ZF5O-|!d-fQ&-2(lrC>V*WDxiU3V^82;z z72Q&NgMuVvPL$5?HdxI~`ewgIN=Ar}1O%#)!QOz{x?fS}tq9<`VwT|xDE)&yPm!Kr ztILyJ{B=Af^k=oco0Is1Ci$9H^Tp{@kAH2;Kf}dOW@ek4S_YoS-bKulwp`>%N>g9IefCW=881HDo#ouX^fa zTQa}o_E!9Awc#}}m1j3VOy`+zMxM8Avv}U!B%0t3>x&(L$<_|l4kZpH_SN*i86p?} zO#lrECRCAl${>-qBJm=Glmot!K5Hmn$c+CSgsIngNyjtE`t-i5X~?+TZum&~*F)WG za2pW!5I@OSK0O2l0HWguPs{rc+mCm`K!{0p<#-xcrjQ*n$QI?rtI5*5N(r+Wi}p<+d!e}ZO1;g4|TvCF)Riu%^aK;1`!KH`Z{NXOfCHaIuqx) z^DU8)Z)$=j1hNJR|7zoTbzNb@00jeuqC6N$JB-)@K}m)q^Af?kM_(%ROEXm#4q(#Q?Y{X-Xc+Hf9VUV@dp{8z=G> z^%;oZ?;J6W>j+Jc(*)}aaWPJSqy?oYm?CJzlmARk_QOV@^{jnju7Gf-?@n;|bA0KaDp7m2&}O`;J=NMUCi-iKvHB@pId-XJ9LoLROwKq9}+V1rGEFyoSSo&j&+GL2?5Bx~5Br)wX zK;M;465>Xk9})89V~CPe4n|%rQ27%45YOI~c@shb#o3YkT<=|xarhk6)`=@zS64G} zFs~m7JXSJ`Yua#@l8GF_fv2aM}79%$u$AJ@c>(g^?3*xY~#1=E*vd4}M~UY0bMg`&*cJo+Q3VZX-___X0SwPA+#{NvRlL#+KgIkou&UJsYp zXwajF<-i_dnh??q^mb0!K;n~jsoB=gcPTE9XPi35r=%eBMuxS^+!#Voz_8aEDTumD zX+=uiz*sXCA0u7kh9F#zl!}AhaP7bBe!O1=xb-g)^N3!j34ANTxcL^j zsleTIVH=GT(SjU zfIH`x=2hBcp@AFOWRCxz?J1j5@BMwx#y8p@Day*TKSiu*cS+#&(o#me`etRFi0Y7* zQY45HsH%$Ce(QiJy(&w-K!1`p}pTyqc zuA0ar^r@8T2hp!WOMW#KJP;75W{uW{rC(}ENno3z97R?qJBp?jYWA8{ss5iNY;FOr z)!IJavcHP!KRS+3IDH?Nq9h&ncnG%>eu!?l84NqE4PHKP8ASv69OU}*zIk_BEs9VV zJ!z+U3V~Hn0>q*H?N;mk2gCaEL4a^mfcEI2(Pl~*YEzmR4@?6z=R0APbyfN79Kp4A z4NDe!U!oyb>x7~xr~xs>F~wH_`u=rl4*OC9(;Af;&`shXHPv@r?Zn1DTEiA9@=UNi zNcy>lr(8K}#bC;8QnXHEQd#MtvIFNAfPEzs%aa==>o5p8*nGTva`seRNcp*G*SX$w zcZgBUasDk<4>74L_t%#VX;NbyPA;AW9hjkAC-YO*mycN zzo{cV>>BeXKqHEO>9RBo0bWgyXnM>NIR&y|k9CpGi^e^Bcv*Naj-B__cor#K2(X>U zUD1a|VZ$f-4?Ooqq|Q<}svCfKV)Q~7i{eGwAc_YwF0*&d^Rf#4SmX6RGbCBIZYusp z6O&z}j8F(HL$$6VBv1P_G3ZxMyzr(GTWvtRU>FFC_en<`AO2c(8iZeamn)<>O5$mc{)#c7CDGQLdJ-xLkcJ8%y?zODfE15d_@jb9oh~jtEs=URa$y6(J;|0Wq z)~3ISLCR^S7`cb>n|th(E>^;lS!97`bG`hzMv-5~DRZ})t*F67WFL^_aGiLK=h;rN zJvENn0KAWTjVhf#EK#k2r%wyMl0<*mAvI2NWw1FH^#zug&^}XCARY0gq|n zzW_Q*YS6|UXR%d9{Z56;tnGu1wajeLG+Nw|jIMf@0vDbNrHz$)u?$ruj zcG^i#d6)34FeQY0Gk81e?HDtcm%Jhw-$qu)oYWBho>np;FHX>q>1}zq=Po0k+Re3p zJ+D>nLR9wSN?_-A_2nBbV#c@TzmNW9YmdNxx<8QbyBe3GGRK}Be6~IMuElyA+Cia( z%Z4&KE>2kPrpfWDc~FvFq9q-hkYL}c|M$baL@s?CjovPYXexGcB2V+ihz2RcU~R$| zEv(!zV~$avBlJX@Z7~M0+pfpBX^0qdQn7=m1 zTY#tSA6OwK;QUUr)0o2<*kG9ra$Y))tbKWPhPfL(YIgW`-xbH(YMxeyU1tQfc`0=I z-g>zB&HZX;yZ||9ts(axiTmG3BTw3MSBh5i2LE4EnWJML+V8@fYf4FSlVlTR&pcjc zZ(H$izO=ydO^MXQso4*8-wBHY9`9lo0D`3GAr?xy)?8=qN7gZi-sWJmlj}Pj{gK8D zx|ZS!yl?rS-wow0LtYfzT!%`dn&#WTs-5j10d(mMZ-21zlh+8^5LG`&B^apfCVl4Q z(3a^JJeDNReR;f%JpLxDFATFyaFRK_j8|Ecl!1C7#l-RnJt_zc`4^3)*OOBO_ClXK z`*Hc&8W$di=!eE~W`9>%q-m_+{hZiT?fe``ya**~V1>@d-PLtqncC@0ozkUnW|d1APrJFNSG<#LdFrSMJ##-97~p zee3I(e?izNWE-8&>tdqt@rdaE=!`(@+ zhkGwCt9KcO7KcbY?oKrh3A`zoHbqmf`@1W{MCVUP&9c-K9O4 zxaG2b=jnlhDO_RnEIq`fuM{+8U0#=DI$Fx=FNZx1KhTDrtRK}>apY5anzO<|0*AXG z@)8NBDNaf8qqwY7D)%f&1x(Y;?RS724;C+UMfN$CE>n_4)i|lpqvC%3=*A+>{`GOy zKA*@=wV&v5O*rQHr3|Gr;;RbLH{UGjCOUZxwlMA%0uDsG1Hv=2q*=uOu*`VIA+)=g zR$oD&eBz70xJC&}2-f9)a2?mSC#Z(2e37013|W=J;4W0gKu=!4H<`P3I~9;!1LJjs z@DmuL(V*Z3op_mi(aPQA^zG7{c~k~n-aX;pG+h;zi$L@Ah!xGfY43e~g0V9# zbIM{T<&SUqj)d-24`x%MyXA)o#C%~;< zJ0&m2L~o=bXB@C}uKG#OlEyQ>KS{|Aw%3ek|FNzTG@FfvmCiL~wW-Bk0+j!YT82|l zOPZsW=!tz^ix_80T=;p4g2hpm+7Idtk0s#4tkEBScQuZ>l--L@HRgRNOKOT%4%w)D z5!nYvOj@uwXRWKT9xZW90jJaQ$r(vn*%A{C+AMb0K1{k+&hVTU%MIjBnJWX!%Z;et z>n1SC_g;j*fN}zR{=MyqMnT^ssN2DG0IdNCY?U_#wWfJPBrji-7%z`L@@_ECDBi-m ziW@m#EX;8-wmUinE%;SdaZde~!!Mn(WIB}9z}je%_seWnavzup4ZeuO^%P#Tm-09&ChkCb_5Ye&#-(7H98P-MM@Cy(f^U zmVTZd`D)Xu%%cRlzANG+% z<$aig{hLXWRzDNDdIE$SWrMr2GpX)c44F}S8Jge)i|Ppz1z%r8016gj;4(veYoOYa zM}`ekPcBOO-N_H&~g*1UA!9<(m{q&GQ)N`*NTpz2HUr-s)@X zbmGv)o}p~uSLwf6``LAt5xYj)@r=<>{Z0YtaoQr8wa*Hnxk*JITj;n})@u z&VW_Y+*t10JLdfo_+a#=9m7{eB8=By+^WUB9ic)WsjEzr{n4&cBFQOkU6ShP{p0yf ze#VmwXts*z`OAk1BBY&w_pP|M1IO3p_ew%Lj4=0?PiPCy0mckBjy%Mm@g?)m<+^c( z9rWAx${Fxs*wv*>i%G-PW7FkBmGgOPaL;%9hh%#}*S(D=Hh}kI7v7H#xc4)0kDdMx zqYtM>{2`Es{_w|@J5P?31HE^^rW-m?1AZYaeM98*eM4({E%fq!&8eXx^A9{da@yW} z`RXNU5V0||`7b7P)D?NTIB8-1{q+HX@S9b|51ir-=YlsqAWd#LI}j)d()v4Q-j`1a zSv2BIXn3uKYwrplj?Fm$qJN(r*{xZIy)jneHn#hVM+cLM9 z^SFgp)wOx+ci%oV@5BnN-AtScbEIc7wcOe+^^YsnzP*x_jX2+9`To_n!Ha5f85JYK zk0Y}ct2SPabchnu+&oBr*ipYJizhpDMZU_|o#t4-uK{o`DKicRRhgP-F0^(b0L6-Q z;73+zRnHOIruiZvBU zS`Pn>S<#1uUb`p~-O{W61coOVLY+Y|5~yXDtaY`(e`^~A3U2?Y!lnmhdz5P=lKh=-mY=lHsOc-r={=x7T8<-W;vZVo|JfemBG41y6 z4n1*vlx)BSRjlvBDM~jpc_Dyqvl%ZB0H*o}OPVRL{2kU|L{)6R?kC#p<#{t#ygT+` zYBz#)axK8lN)W6bL^m3Z5xUkt-rmo%Q?VZ-6Qe25g5N7`#Q^r+-pA-LIs*8##Cs8M zNV1d#IV#)DzLhe4BlYD6V%JZIfa(sa_{QYYS|U}j((cOB{=?F(`^ul>c{0RgtepHG zuiH(nhsD;%xW}t)e?rjD#0#I}NWRuvfB~`fbazzfJeTS&{QFAzV?%4zc=yvx&{2ep zQw$uZ>nut3b|H9-wY6r((@K@2tROh#`0(Qk8bxw|CB#%c=~_gi&c7%17i4-i_zy|Q zN~HP`@^}rwT1t6CN-|_HOgd&D=(_Nicrb6up`V{0SD3V#Sllr0^+sxM6dZ9?yj`Vb zsHCOkA;jyoyq{L?%z4Fh>C<|*V|poi4LiCx2yDJ^v!tmbea)gUbM6GP?v}5uk7LN0 zHy|u!I3I~-HBc2oT3J!*Dt(qkh|yi)MpfEMCR0EybZ?t=cyn?`82ntcY02N;RPjwm zujpxiEBp~Yf{U2JZCyVEB(QyUQ+!TS{Q8OF9`=TI$Is8T{4b~d7N7XF7BPDc@|OMb zcgsjjaZV>oJCn9cCU*_>4-F0a%gZ9gnJ1UbXvINDA!St0RVy`OC`BlR+=8YsW(vHo z%%g<}JXhZLF1AS>A`#_e7RV)E#4M#z#H-3lj7)g(^{S-k>9DhZP%y5o>NQI_b>RQ} z$BSqbm&0%r6ktS>2m375u-^f7r#j_rLR}? z4u@%CPc^&Z56Ad%ZNjS8k~h75D{TTjeYFIWBju^yf$D*`owmp--%@O_DWpEU_IvQ@ z$5w>kf8(B%$NMbb5$|Q5tAOzWCB&Tn?Ip4SK$BttkJvT7Rmpnm+Ij(>jogU6Gilf1AwQy;MQ}|$7wGc1b~3$ zi+S)knw&P@{xWry0{m2B!HWx_@hzXg0e8vn_Ne$AiAIY8i{-VBIWd ztDF+!tn77Hqu8rA*ztTgMSmPFXez?U|1{phKG0Q9V&0IplV}B|Y5z&OY8@hkW*6Vh z`^(znqOJhjBMf~y&Ikv#;r*t{h!;a(WMk4bOchGa_R>3Bxs$UF!C>K;$spI{;_C3A z=XIO2bN!M^ouT0mBm_*GCD5{dlevBC$NTPsAG|RqoKi|0jAOmlr9@Inpp~<%%x=gF z&ZR!_S;p)XF0$yW3Ndq>@{TPP&A2v9D6y}pLli7h#85*yu@)GaQ8veG?#(wAJ#B+p ze181~nrnuFB97y9j`8KM8EBb9HhfcP%+y-xLpvv}*;9&7gJxTQVV9 zc?-E=ea?)*C@(RZ_=XXIrvX8jajAjl{(Nru^2RWQjB$;W`#fDhZ|71drdmYBRNtUk z#NSL5ELMuX&vLzh6kpbh*@k#H?k;X$bqdP8N1j`<`1N106D z-&qS6H>R+Y0!l&O@!%{Vr`+E!ed+*Wej+>gTl#yy;j-XF9+JOKXzK+eaH zfFiVl;VIFz>+I9uq-&T-F85JQK9?~Fk}2%1DPL5JYNR~Zyzu!V3QPlyIEEV&tI6BS zk2Om$b=PShzb@COXldxuZ1DMxVR}_{j)4N9-B97fK&Y#?M~2~ht!L?5Q75L__ZM(% zq-c72w~gyz6v-U>gI5z!yxGfAb|UlPE3CeQazt&XETtiZFLHK&Ovg!>VvUYBzgHYX z{iOKJh`lr~b3Z*JCs2loKC^MK#)%1K^t}B{&F5iZrPlfO?uTqba|RXGlN37`F9?Mu z2-5x?asU8p)^81F(pg1r95$Pe-3u&`g2KoAEDCDk1>T=&+BiJCjL1%mf~?h*RnIqI zj-;;WEW8`_9))EPc{b!rdbI_o+rJ1^BChjOxp*xyT_hCfa1qw}Hh0w#n8&>(?)D`} zq&Qn0^*tcV5^@JRdMV12eyg^osDFGPwtbJfwG4r#o*_J?bvU!rzD(=$oNbkWnlhvvphffz!;A#2DJe||CuC%MVF@@yS*-F~O_>&ui z*Cqmzx$8=zI7>VBBW_yawHoX|1-dNz4O~oeJ;Wh@cW5b_98Hbw>B7mbA!YVb@_)uY3o@U3vYBv>%Zkjl4 zBDdC5l{HspafcJv!eRn;A9&s&lG>5Mn(KO#Z5zArNB@Q7MU6uJa zmGySyXJcL%$96`_S$orqPKnRaD4-MJ1JyOSfwupIO$*VjI-i>~H!~mezu9=ja@qp1 zC|nu=KZ0vjQ;Fw5Cm@5=@WN*u4UH##HJ_;V2oE8C4-_q0$*#q{d1LTX1LwkhwCv-5 z(%(V=YOGGaneGp#6E>Z0d`b!Y3U~7TY4%5?xru#iEd=Do_mwG*108@C;X1ZjE(MBm;flUU5)c z>tA%G74}04`jsVUj46zTouE)DRAX%C=YZUL;>!niQ5H-Wj|J$}>$LE|Py{MU2b-{s4`d($Fc;4QP%x8~eNSdoB~+rNozJb#O8+oC^Y=#~=z&v6?h0ke z9-x^Vw_~%b`D)ahI!KA$!J+0S&vc0Pnbm+5@%srm9vi?$sUhGXLYCch0U^z#j(rm_ zyK~3fbBsG9GCZWYIuFqnBW5i)H8&Sm0?$PsFr@WB8kSs+TWRNvBM) zyXxt0Y6mD!Lti3yFRY~e+U`6B1RMKZe!6$xS6%t0fPEG2f09?hj=D~m{eHyyGgL9f zWh)#o22t5HZF`4fJ-9D7@7@B8OoxX5)k)`8${v*ON`GNJAEfC{`0G;H<3TUAJ($GP zkpE&nu3(gIsirz$)gqhjI@etezQ%IC^LR z0!6wG$GQZ-*{1}kds|zPnlInuk7Ucj{o~5yP(O2vbt?X5#c1G zplkvHqqf2sGWOOH=mK{kTV(!OnzA^5aHEy3Us8 z?VRf4k6V;iy8gW0TPeCE>1)N&3*@N3_gND96qTKR9z1pa)Lq2V@?lZ}8!0GghVe&7 zX?A*tND!zu^5~mq566esOQ#((tG7Ca_Snt3&cSOz;#L=6$k2~(L;qFaEx+$Z*wWKt zZol$ss!NRULQyj(g|00!HQzC2tuS7kz3`vbVOU?e_pFWu@nQ&UoOpJ|Epo|12KrVK zwlC;@x0TxB#AD3);ShN?A;C<|*-TU}|3=kl?6PQCy+;ui#Z1sT6VcmT*wIwL zkyMVI|910-yVJ_&g2>!z&K1{!&iwUiLFMr1-}~*XoVKD%<<&!4Sl_~BW$s4m;=8PA z2Qr4cOF!)_s$33?n3KMR)k34ci-YAWW~w!~SKUpZyY6x=Pm zHvDVw;6Xrpzw{k>9o`?db?&zu3h&lh(JToVAMT#rd)rB8i`j3f!#C?RZ2VheY&)Vy zl}7Xm8CUcrys!yp-276WDfdd`=XZ&qj#O@e*V@HA%Aygf@A`oH)mWS5BdHJVD((ub zcXq4-G?tn7R>21R^gV}fET1hsH@3qJyM8*@6SC}DK+3GtjRBID0sKR)rkoj$9eRYs z1jR*_R^u{E!6Agvggs!YtV|6nEBkzIIlz`^?9Vv8!iy|L6i0+jxIO|yc%@8Gnqs`# ze6CG@sn-M_E|UkOcSaT$Gj@2^_W8Y^zqqy511@bgvEHg4TYgQs-1qOx>R7*(*hGW0 z^PG2+^wG_A=hHb7gkkPvxV;h3oyeE?i7Kia)0PI9Rx+ zI(z)$@HnE0Xe!C^iVQsQsYr|I=0)MzEXE!~0HK~Ol}7=lykl($d;N5k70O$zA{cUx zO#RY@+J|PX=C4h=W=4(T96mfBYW>t=NTu4^ zEoAvppNnd!qc)Di$x_-FQUs#|&IDI%K_7A@hni+w!d@XX*kXHkpD2$M7m14X{%T{s z<5-w?r4Cw2*Bez!D}3@O^t(i~uNYsaRpFk1{d?-o5A=o}MS2=#6i5EL?jlyXiw3j# zw?IR=es6ixg$zaZI_i|e;$Nk)lfjJfpJkNm+xjc)#*V>S6X{DVW8eW$KBzklj5FYc6G{7BIonz_F3or&~n4#Ir zf61sq0Q*XfN{FVIH~VHyM`%Cp_DD77?|fy=+0EG6OH~;?e?o%rV}#B+_ljKSqbeee zX7=0UM8DQp7^&g()XG&xN0LJP`t(BVjEai*%CoM{3hnZ+aFWHVY=9zaS2y{7{9Jm( zVRzycf;UgDZl9b>@XQ}?W@Hv@Q}lP+n@2YYw0wD6XV%a-Pv2tuNwC_u?7m!>p|IfX zKOOq)#r)_R{&XA(J>2NfStA*uPlP+%_8x%5qPQ(*sW~`k$quJ{5dNU zL-SEzUWYQOHyw&;_U|b}2TMS`Iv@SLRn#)Km!qsV-}GhR2yCnWxnOU4_lqcs#%;9= zi@d)4TeL6sakV+5g&iDhxw@72!~T^vAMC@94+1OxEl$c^0Nv&`#qbdkh~2wz8wHGz zV@nR0$wYjJ(@EMbRNan?t<=+x7_vB9(hXQYh%sD6UTTBM;ReD=(0N`^TQU*47LVoB z{A7CmXg;kRkH(Re9ZWJ|XkutI>9FMj*s7Fe@O7e^s%TxAi`mn>`T;g-6OV^KE3EMv zVcnaj8Zj4ZHFS2~dN1!LmmOenc-)x(&#OoNrh4%1qkyrnqV3GY}GDf}8}&T2Eu zNfetNmNRh>PCu+3b8($+_sM2EDXTs6Yuo*kkkkYH4!)N>x3VJS^sQflxnOUndze8T zHrf!UwPb<&8#jObr^+}*blDj_e`Gpc{0r-@R6`rp6VP8tfx_-y`te@+_J?bJ`lY;> z0*CFmzJK|69nrGp{rSXJ3=vW?z7rhWH;|A~-f=$UPF^xR{1{H&?6a(2_wmt1jH{6G zO9i*>_~-Wl2O#Bv@Fx^{fq8ebKvbBOLpf8hblS1dj}KvbRo(BIq97&wFmI5p(u)@h zer2w0xUbl9SG-^L6~Z53fNb8(GEfsv5qD%`Crg$lq@<#xD+`&ijs^HhzZ>2Hmure59(BW>S!D*Ps0IBk)1%aiN@Ndkn6U zJ~Q-l{KkWE>v(F}&_5rB5c%OZ&pHR~o|220U%LL;yhRWZj}N{dOMV@oDo>yrE^lQW z5Ght+;uDk=j14K_ghAjm$vl2vwW0X(zdV-uQP2G58r$qGcVw7kVw#MzTy2^(Tx4xrT z?L{feY5Jn!oP6_uqui$unG6HLH>IxF;jHSME0G{y!@&nty=mksL^}5ZRj4u9oe>*DsaA#Y=tOQD^#-zm%o>41K@Z(Av7xe%zt( z=MS(?tZPk{yREHJQwz^l%q6|9um8n7c==NRA+dOe?dO1f3It>TuLs=S-u5U4vWNZ) z6R4QVFh9Dhw(*ra8d}2N2Q9-AB6&!&f}0~)J;+}yFg?xnbh9tY8%jpdzTz;Q+S!~@7Eo)eHRcq zqyKjI&uy;2L=?3L^vA<$)3@c&Uy1xmZm);nZJYB%7hN4xjxnn8Rz+n6MQmEXJU|!v zgDM&ftTH}LrquyCzq)G>)kbuM_v5HXCnm0(kF8Ks9}mMqf4^Dmwy5`+j(f1E(+Wg! zOLkj=jpD;LyqlGtfN;j`F(N`4)<*`=A$kt^=>4e&zNXWbVF)9gB!4`YGWo<$m1uz> z2XE{%@2u;*n);~eAs;fdUDV4sf03p0Q;^GTIzY2rXI=N`htM)SEJr8J@wHtJH)7Th zt|x!r+RC%IiQ9sL`0{sM$YU$lg~D?F$QfE|tJAM5E!eXhg8P5CnW>b0IP;&GMw5gE$wn@oNMo_(P5 zdd@K=h4>Yy<_cvw8N(IX$@Ddlb674i$P|xQ6$+mI`H5V9kj6{~dmCAiL}o;-wcEsj zX+~sKUo>_HIN$7^1}W=h@5IhRuP`RDuAj=(XL%8Jc6KV5wv7?Kp@AXUXoEyD9zMRE zy&Y8u4Cta|Q3ESVe`we|ueQnyBZkmhVnWx5%g-5}rL>b=@xIR`QgNzMxe^UPejLhzEYEkvUdpTl03j zYH~q?y*TLehr{_$D>!DG)s~6K(tts)O#c4%xu9o`lcdBA59qO_D2GO#aP~pl{cC12 z3=P~iIugoE(wlAjp=0aJUlOWAx&1dWAY_&2anaYil^RrShsveA^GU_Qm$%&)YV%J= z-L32Xi7(yoEol6#%tiA zJyBmr=LDYKA+^1|!Sx^&RjgoEcB^m0m8bFZv~t(SFVgDH;#v7sXhs=bZl4M}N`h;= zCt_NykhQ!Fo+qqUUtjD6cZAhhKklwB{ba%V^R;LW(rl{Bvc0C}Xhcj)E^KpPt@Cer zw|t|0+^;+Bol1Xyy)kf2(v! z6a??;n^8oR3s(vOqITZ%21nPLs%o=(Z}J@%Lzf<4pcGbvacL`j&J34F@&Y>Gf1Wvf za%7P30s=w;0(Ir5hMIyR^K)sKwiv$1tS|Ex1ymq@?BFiY7$Kg7LJm;j%wBPVlXw)6*~S;N52`FasrjBRJI~s-6ElD z^SuqbNlx$^N(MFw!8Iubapj8!+=#64!K_AS1AR6ja%_LcfB+Q%{38B*l^lF)MPXi; zp8mTPF%aVUsRD^r5dC=kKE99SpT1tnws+}i`_s!H_$+q{3lqbVa#5{oTP>%8;-RCJ z*E`zhF*1y3o5TdEdqKS}65MynbROJOU-Yy5QE0I-xw7;7FAvHRG*Z9Kw2{u z5)sU&fvubpec}`J`uQ3j6rfevl}OmzJFTtv%?Z-*+bsj z?6)PS)e50^y+clY5_$o52909F#pdU{AQJo7Ua~?c7+5j$$L~~;_4w-y>&u^y{&YG% zF5=cWh>{K-qT|kY9ZzU??B2RffUz{6%u1s4M322UBPZ!l?7s2L0u|nB`Youtmw`L&n70 zuC;1eo%o}>z68QbrXo?x9t?J7G?NKcH$|Dc##s$db{Z7A(xXLlfYMI!9RLd}zV}6d zi`NJW(uVmG({UF2z@`PvqAxk4DV9X#vHP{)*V}ZsmxQYYgajXm=lcD)0afZ zNiDJrbU*^8SvXEVpFh79BMA@Ks z?8M5ol7v*t0}aF9?QOdmPuq4|&q?NzumYY0H<7^Zmg_FlK-b20Dm2v|V#`S#-e5;I zrI`nF`I9f|&`;CgFY-(>enNS3sbPAC2wF1uq9kIr>Et&jQpz&Z5ZkArI3va22Id5o z$`a|&{owau6_s<*%P2UT$6rfo2XLDtas?Q}LV8=e~BRF`)UzJ`H(@p%k))kFMr}M_u+Om^0 zHcje+>0taSvvb%N@Ii7K7!4X%C1GtzN{>Jy5y-yq&f3agMI#1=YMgb3W2J38x#gmi zsS&)pJkZ~g!nse;AlDJRR?wHrg6t$4q!C?UUAeQR&{po|Z1A`Do*l?r!9`Ou^poH8 z4XKT%Tl}HM*C(V(k)QALTh|_{ElMHvEbd!B^=_5(2R44C43HXi((xdTIH$PPXQdEG z;s)NLhf8`gQ8b$Nu|P8(Z2EGUN;ZXpgPubY`F>|+-vMq6Jmvg}b)68NS3?jUMG;NE z+ZpYH6|TZrBSYKDUCjQhk;BOQf5!ePYfmrs=2i6hsD2n#2RaO=6Kr!Zr&dpdW{xcUwwS? z`^nkYQ@W!4^}6G7ME+y@PAV1G{!M8IfjNd942BOmGkBT#;p84NsMj2Y@` zU9rr+@b_8UoQZ59b`YMqm0AH{8N`juYQ`L~PvUfAnYDAmM7c+reFmlO>lh8j45!px z{L0U-Wj)i&l8MIS>3Fr<4wF}$KE9M{-5!=Y6g`;xQO;7UN^U9V@gWwGpB;Kl$_*lj zpzO^P^COT}y^i3l4kmE}8=7VlHQ?f%nGLLLC=dpqe%b?JeHS1B){`GL0aZ|Sn>`K7 zYx|qat${Dcx0T0+|GNjDRaQU0ML&4FNKdi*J&BxcAIm8a*3B7&T&{?zzpF&wFZJEx zWo_7%qs40vj>^_QMb_a#_)!F-==BHB7E7tG=$}04+}^Cl*OHG#zaNMHS_rBHKwb<0 z^3Jt0&lIJuk`I&cAJV@X&?7U?4{q^^KSw#bV#_J2zR%o9=HmDh)9Ul_EBKz*_s7#5 zOy?!xi*qL_CRM<}Tu@dw9*p1}OBGX&O_nq@8jL89#~v)skNOft1^fT|^Jz*93C-8l z_Wa(_vYz+vpDzQG)oSVA!E*Nx~jWcssgcon*6^tR#s1QkRs~G zyW#Pp`0_&j%)5Id*NPtqYRK%Q8bJN}Qq{IS?N2J{$IpUE)om@zSq+}YL)jg`5yyf)D6l%`^+Zg-P>ni z`qj}o+<$j1eE9S4X1B%t64eLo|9(bF5>EsZ5-^ec7Zpx#(VHgpKn~k#UGcaox2Jib zHHtsfIpppxAq;V^cBIR95$V8z&;=o^S79#cx|V&Z#`x_lphQR!ejLLNpB>BRu4(`w z(m7$Oz1knc;J@$NMb5`flDI!imtP*dd%n#$gZ4bJDIeaCz*WyQ0-?r7C$b}V8 z@^A;pRWo#1bcyrvwS?4Z%K3kT`u*WY=`JRQgrA>3A|;IhR*Q{;n|COd+U#bm?v+?u zN_-10%&$NTUZ~2m5Q7c2$;ROz*J)} zN$|PFhnbnnxPzgovXQqMn&q94WwAmm)8_U_jozcxfAc>b146XCy+?BShAyt`J{Br4 z3R{G3{z;hTs;gda1wx|ie<6|Wqh13}2+x=nveLjx4G&L96#3BvS9!Th1ea^-Y@Y`DUC1qyrwr(uNhmL#8ZUTal#3~C^s-a z`Z*{DiVLzDG(p-p%&Ya}+L@xa_=h4=lXd^77-c!kA9}mr2$xstOuDhY#oADL50ypM zoZ5ZLO~}LZN$OhM#LKLcX#{Alu_AwEi!J(F&eFrP|Ap0imdaWlYdVFVP9*URX5ErP z*SCCKxLVereD<)K>J_$CmYSR$wYjzBg4Nb$+#L@M+AVlb$NPA_Bd8Hhxt5Zc_~!Tn z@aD{h*FVVDA~0W`eTctBaKQReAMDE6%YSrLNCQK2UqW|-x(2H8ZQ6%O%(fb&kKQx= zLz9-om9LgDz3JI||1zYvZ4wkN0UBmW>dRtU7T>}jcBUTB+;Pynj$oyJ>C?k>EmIin z5+xH0_@Q+aR|yV*f5x2jgYB{9fsuA3ma150@|D^vLaDkQgx-D3;dS*qjX@No7nAoE zYFLTt?Oe?0Trj}yoL{c40k><)W;cVF&wkW(W2#L)re-g6|K^dv=e1IC zWTq8{1>UUh=-jgQ-}zEqTQjWq-<@}r1F^wy`9Gz*T}DL}-J^$(RYR>I5J(60Ooxl( zt?NLc&$3zmxz&~QJ>Wsps(}3t>lp|5Ygd&t9@8{`e9D@0-^(Sd z&kQtwZuRHX#AfAb3YfeST?4NimVm{VLp}oTG-lM2>U|`tqS=&8yt#HJX4L#cmF{>! z;~cjI5yULKsten1I)N{bSEDkNc+p%-CMi{ncT?0~U5h{emH}jhu$d}s=rcSxfZ^{JF zecl#9Alo&$NOYZ;7RYr`g1wDaVS8fMxG%X_vy@ir)%0{Nv!~PVz}_ZJ@d# zI{I)?ZGA)JJfsh06IuFeWtN8*yOnda@cQe_d%afU~t6|BU99z&IWv)aSU?u+#+p9p4(+{o$=76R4*b%GRVFsM!`!`r8 zjIZT+gFA2^57UXe)cjxE!MSoiH+-v3YrCvQ_0{!Zo__EdC+iC%^ZH2>`|5Wd7MyoF zIi5A=$p!4D>yNA69fQX>3md2gUzbx=HqqZAN~f*0xMSbf(8R7=6X~KsRY49A5d^rW zKsgdRB#CGgJ`8C8{cUCMJQ&#=7H&Yn!J8q##ntcea8{CPo{|dmaR&5S8LzINTCPT@ zcQ?N8d;fjZ)ay4?6SR4PPaNr=s@!J@5?l+8T?XT~X{1^zz1EIb!uz&TH8Z#~y{63R)(YdYT#R$Dwv zx0H(?gGbpr#Z(_M4nicwber@R9kq&MfkI4AP^nKw*_RvPF{~CZ#(TCL!2qYwmPS6X z#6^d2qoyQ9$fKrR984Lp+)@yE&3X@%Gr^Q3GRJ@VSV;8s3$_J=S6Ue)vBhoOD^{EH zJ{oiVGF|C2`qRhr2QtgG6lj~m5`MIH2-< zzIiA1b3%Lc4mdi}4r7u<&d{LwUl^c%hIdIXJCx<%4x~xB?zSuOl!!yWkR20xWOF|@ zHqO~KJHe&xL=1k%1&fAEPO7pv%=ctC$PMbYI_2cePtyqcRrB4>vW%0D8GMaf=PNom zWW6Xs-@}r#B51UHiEO{d^MIHhVxWWKm}%ukLgWu6s{b7pwfR?tLXXKmZiBJ{ZZp#+ zripq>q>^v;Q8e%2E?{?-c* z&?_F;`4Mb^W}PzRk?%h)piwe#B1RaXnoV4+A`FDZA>~1lQu(e{z2d6;k7o#h-9jKm z`r^G}XXCwP7>A-NLP^4u7Jyz1@bn#Vuj04k+(JVORY%LbAXyk5|Ci|UozkZD@YVa6 z`*n*Vq{;$-;caE5Ir*lE9W6M!Byd8T1q4=|kn?`pi`ooauyZ8Qa8yRqaIyiz+61}= zNN!NCAgSfz*aO4hs#?Qk?)|fzo{LG>Ey__MR!sx&Vn|#r^OPh!gWl2riJUe(k1uC< zU$*|c(c5Uqsmx9*^Rx0YM%l*uzwUQyXEspJn*fM9bsLZHbnZ&lfmtpc`CWU0pOBiJW(@ z?VMyMFAD$VeKicY*`*oF*pgdKlFG>%8`eGja-&L4p?zUdCr4^a=f(6B4JX!_mDHn0 zB!!MfP{XizHJlF8QQ*t(z}6_c?*K-Xx0h=0e)w>?cL|X>t0g0x31=rX{Y}~gYcbk3 z^3}|Y5|?RF_W4_;sF=>JuE%Y9o4=tZY5?r56MEkJz|flH3o#x)8~qR5_UW2+%Q$?+$}e>KpKa6Lm2&;?sL zxd?g9Je~zpvy6i3bDLGvfV_-(a>_@~?>~K+&M?8#3U}}^Z z@2L8?Tio{YU7ub3X5X`R%8E`-ZSZWD{$%mt5;iy+m0KlvCH4-Jo%!ug_Z0 z?<;e!_pYcTV1Y|df1jWO(|*+?YSoo~!MOpISvZ81S)a&Iisn3&p$?nC<|9u#QDKXOJc-tD=FL1k*xQs5wxmyyYJqqEtP7wu@rVC+G4znp*4Nn&VtmTar4 zPD+brXei%+e*fPjBQ@-?-g&~_)9~DH;fyiCRd!sji&t0WVIN*R`PX`ukwT9QI!f@? zc_Q(n^TjR)D(sK2X%2UTTF6`^~(-I2$R7ofqMx8Oz|J9@A&JI8K(8m}qKr<0p(a1zF?8I*&4B@MPBz z+YFa%q8xm3OXH(kj1A8wU5Wzpg9kYUnUdc}h8M4!){cLNbanv-V>++?$&V>3jmZMU zU=bdGsBQz_!Bu#(`(%dVl3>T#bSfS`~T620?_&{ z7|`YL^~Ac5IwogdcVx=uM7a78oi{9>WP;_#-{*<0erO0vaz%q>=!TFu zq7Y8wKP1Q8(6r48?^p;h6oXT!t%Q+*Pade9Ys+NQwHjKsX(RMQ7i?MFd!><>3IlW1 zN>5N{+n}*oU&GPYG%l-9S0@-Oiv2|FV}Zi@((MZkRU*}VQY;@~Bo7Fl z#RsDAw}EkM+t?okD`r}V=H4p4wt;~&66y!l3KYHNFIv{gf&3^CF-z2zBsC-usc$Nr z;897n*4fgO`)hP^%XCd+Y&JdVQHqPtJl-m3(D9>AK?F5A@vqpU zw`MhjhC!DNIP*nCO@kr$^i@ zT1Mjd(YVy!4{`-Ax+11IC^Z(y*-@ndm8uNr>tI&{~L+6j!8 z8rOk{6l}HPUN9cNG5I)E^=50$+{{qMn`PMBphEtM|LAGSZB{9tfyy+l10Ev+g80V( z3w5#~1Dq)Okk_;+80x0pN>wAe3|i+VQ+(k#>JjsNYhZKDxV1^KM&wD{i%51*|H{$s z0C%rL?IscyXb(!CVHn|Q(?&h%piFvN{Gh#s>hgtFnM8ocf2qYCfR}wt7OClT;237D zU0=j$vZyD``>C@>7i>T>oSSYnqb`^|zg!8n;Kue^P zQ<5ZjDsCH@XH|nI>SaA?Pj+b0(^AOAl20P6>_`C3G$Qj+wtJJWB%CM5$HuA$-~IUU z!+tpV+f<2?$UD+Kx$42y_!Gh`){nolS{?QDr5I`r4rx_#ZR30NDFSLu@q>b?f_v-D z(gkGLY|GjPt~h$VIXI0d@{n-|wHvfxC&g2-lrQ;OC~KPI44(<5$4vo?Kl?8j{BX z%QIyZO!ocrT{I}&8i8f-N*c7B8a)Vh`T}J8)c%vr0)4>i2tIye*-YG8Oiq+TU7DHw z{#leK^sSX%HoZ&B%UTFbUNtXpR8*}>d(>`wQhL9{rhdPzwrCvp28v&=`xD9xst=wd zFL8;Ed^hBhEd5vjUWxw=gM;nYzJhEtaM!q?3c)OV`YoqfAl0Wv7Jm~be-zD5E^lqN zi*D8*e5^4qsUA62>I;WFq|hozN|MTG;&w)qt3ASMk(ELpedlLlsrcK0=k6^4?cJG` zbg%cR_x1>lk`Q>!>xCM0TNHtss_F;%YV^FR*54T!1hz`V2Yfz?P>``_xdDqn82Bfp z6E__E{O|wKxaaA!Q?mW(1rgDct?JL&%sp6F@Q8aV4U45@1D7dHk)@=)*fdwy=}%ym z7Xe-b1|lEslbxum=7 znK_#j>@r)Hk=bHI?%Y~zdVlaElTAjr{WFA5>6@(<`5+imJ_iD1iVFI@jl59vX*Dd~ zJ^8AYuONK?L7_^bSA|aVD)MqctESH; z?_p}VjFP;kl1(PvxFij9ydMUloTRIh0tWEVI}0>p_b|@QGD1s`B2EMK8iBlxH&cJ# zU-O4F&b>69uWQ_^>-K8Bco)Fd?zg{om^NI<&(AL$zMm+T)3QGm^lR1lP(}*~U7UMn z{kj=LYQ4$RQyvtC1Kz^J^^T6UfkKu?Jm>=^V!?4U{{=r~eh(?}ggc^>u=LGu&gKy| z5OOjlPmhAs)cBWHJnRSP8YP~7(>}0)@Yvu|3`&{zp(yX;YY$CGUd;!GiO1Mc^BLkc z9LXxV>xs%7$7iOKkJenr(iUB){UxPe zdib6QYKqQ&_<>xQyoeM}O$jl9Vi`ap$8#qxFUj{W^UgB5Z-JQ&APsxVJ>;|W8>Z=L zHb<+gd8`i%4smvyo134SQULJNXaa@XgVsULg!WP7gJw1F$|tryy-P_*toGOY)#~Sb z$;0bRiIuR`vL|s?{yg=P@MD>dkQ30)6Etjlj1lv<-ceWc@jbnrh*n7kPo8$nZnOrU z6a&M3e1G)EvPB;M4XpnhwvY#8#|4XD8VHor2R+=dPF`uANzL9s{ewg_Ws+z}R>?j? zv4Ri0@~D}QZ|<6N=-~NKv)i+0Gnt9?2CQ)up%Mk8TI3TyEVQ(=W}SY#SuuN6-z+-1 ztJPTXOLCy8*o3k~CYQCe_Rr%#dO|)$T^0&@UyH2Y|B{oDsS0W=e-PdVT%HUJOyQ^B zeqUYwqUiElWNVp*NXQa==`%>N4$UfeA^74)4<|{M##B0@3GGAq$6gt1FZz@GMt&sc z8kjN6aPZyYwr|UT`k?IH)(>7enSOS`ZPD3(3MeKmIh*QLbft)5w3CW5O&t7Gnef_F zQD>q^lj);Kju{83}Y`ESN=-h0P-3{N^}_OzFSxaTyp@NCpDR4M?vf={2l z==ytoi@ao#$v=3v%KMz<0T2%#wWg4BCX~BoJdJOpd?W7hjd_KYG1*3|xNt)Y(nn%n z$dji&Sf_&0S-c81Ysf1{vp*E~OI2T=|(KQJ_@7g99J9R7YM0IQr9TwkB z$wX8gN`7hxT1OR)SrNRP4QOsFaNw|MWOFxQKO5H)~GuHe_%m?X-1PR)?6xcB7 z+g}*Ud1DhsbpHy?p#w|xe5Jo`8@itwfrf*8eYtfQ-bgFTEI5esax;tP>#7TAVO}wy z+61y&$o#1C&2s5}4$ee}EqYtVNN|RRT|HBJ8rGYudY==OJF(06ApERBs7_Q^&_|Xv z?GDg^!%6(fF6zUU^uuI(3Kc88!6w#JLp3mdSlkd8t@V!;;3}*g{Ge7sqN5ZP zuh-8`(IAOpVggaWW_2(|rF8M|jXzUNt1lQvB56ZLT;u@sUmNPfE_kBHgxaql#D0z- zxUdTE4>MgsRReiruP186=e@_j3*x0#^m{_1-=_;g(- z^mwiPn0~Ma0Pgcu_O~kfImUc!^?>9%ojYYQKOsY|InTvDdJ$BR`R(a{b z4<}z_!#!5ITz;s!R#hD?hCPV=127H^ZW$22uTvWtI*+SPbqUx}xtLg|{rZ}xaPkXc z^pj!|`fnH(z}gWOn2ml9F9)8Hm;CznDeTveO$|!(W{)6%D>-KYlxvlFzV;K~5~p2U zvL{!b=_iMV6iRPwh@C?NAwY)A z-3Iq5<01m8@nqZ}_BI>Lz1UpRRn9>*gY7Tsi}e;os+paeF(GLKx&sb2M^5E>Q6=gP z9Lv4+AdYkSIo=X}lAY2N=Gvnrnel=mDSa`Wa%u;jBm+Opru zy)4>Jre>z;%avfuBQ`OJs<27f9?SU0(-yGtPzSdx;b-@_e>YVSs85>$Z=%eEfyUI2 z^fbF3sU;f+W~b9DRH$_q(~)79COn*?O3Z~+568jDufJZpVhm}9a&a!D63LIu$A3`d zmP;Xx7%pJiVDMAwf_mVVqHQ~GDs!A-RXw;#Vn=VUyzQbk7+d2PIl9D)OKgHMhnm8 z$epSzXHTE|Tzd;XEy+u?^}GRLc9P4x0Dk;v*xHc#dAn%pBSMZcu7Ny}sQ!@wK5>D4 zi!I&)iDU`zLAz%O>rigFjIu-~GkeAI#kBHA<3eN@ z>fR-nV!9(3I%5n|F}szkTruJy4&UzR0nbVz^0nMNY8YM0=tg1F1@pTT6F+k3IS^j0 zOf_IS&Ly@Xl(Hy~rOVt>F4i+KQ39()gC>&@)|Aj<=Uq;uhgVF*+grvdFH-P)nel&9 z(nSM^>r*&uGYaoxni(9QjhRYJ*I%V21JzhjcA!oIwpSRf?O}gLK+civXh6#vP`&Q< zi+8Z}Sx6#!lnBjiSu`(;g7T8AXnC{`K_?7A2~fj={&8{JX9Qg z^7cE1(J5;F~nG(!iNy|R+5huxiL>Wql0g)b6`Zn$fJu0>K zkY-gbUK?y*xCf5a)|Jdgc34H?3V`O-BUlm6&4+lDl^^5EY@XzS>rcJ^QA*oJzrX?$ z^N1M)TzwoU0LqgPa>y-6q3RL0A!Ma|e44_7a}K5kEZTK`u<6`#Qs){0=*UuK)l_9Y z2Xq;1zZcV1WZcf^0*um_8Eh%v9z(mOF-%e)zF^Dv2X7NR4DNURtV13xuwa8=s<~HC zO83lj1iU%nWUerw2sBf&XWnRnpOZESOMvKhUFhK((zWujkYhGwfTaEGCQczfoH|F+ z2C4=Bj!`hDm{z!Torxa}w|cPY4` zUrX)7R1~4TmP?@>X16tYa{M}8Q{2P=99v!C`V+YB3`>C#Ei}#$7>JPy6cUmz`Ezys zrIrIpP|*)=6dr?U)g)7OZ2lrASx#Kl*UPim&9bmEo%Tb{e{A25?90Wy?a^Z-27n10 zBk$md^H5_#z?y@NOl=q+X*DT~e9Rj~jirXo8g&udgg%5^yq!l$C>Gf*I41gsaJyl z{FgfKsD1`qS6a4xkf>J-M@^|c%ydD{s*|5k5TmD%K0tAlKv@}}E2Ces0_s-npa1^) zb{gF2rm)@sBuHB27!0U!S!-6(P%J_hGDH^XXo`PH7m_iT%Oz82AJo;qyv8Nt8Lp@| zf9j>}0)uGM#l^U1I!vlToOqeHJZjQPK}4#0(;<9WIM+!BoJlq`k2NX(eF|Fuo5H4S z@Q0V1t(@qu@&G5!K|jnwNBbrZF}{B<8#AP(1#*5jC9DZI< z&6^gDI~EQ8nqmNwD`eAA4g=q9!Ple$0`lVTJ8^7Ch(~ak&fH)+8}T$`o9k)V#dZO2Q#9 zn40OJp>0dTaVZ8U_@Um?!w=iirlQKkLEWMD(pU5_9ib>t8F?v83Nav&Tk|n=!q#tdu@ElLbH?q7c_Yhvj-t(+?eRC zG+j_XH4OJppF)`!O&xU}*xqYbEKWCpgK2Mv=rG2s`gF!MO14-pNYjHZyh<=k$ZkAVg@uq(I(Fd(eyzIsQ9~ z^@QR0H{;DWA%*hMwaRZR5zrEs$W4#kr3GG9&9zS^ZygT?A_TaGDRPy=LjTjxLrSX8 z_$x@u7@qWX+P%|l$x|9Eiu^fBl^aJmr+iXs2f*m)Vm!9?Q1~?^W{9pk?F+5gR@S`~ zYX--O2f70h%)Q%$Tqv4N6SO^l;cJp)@&BXk&7+}y!@qHDC`2K$CPgJ%_PvyZkYryb z3EB63j6#&P>{}+g$iA=hkq~0CP7KCA_Q7C`G3L2_KHu~F{&~*tJm>t*?|qz*S?;;t z_kCa2^}1fy>+)<;d&bQD!t`_km*WRtcJ0Tp7jTbWdY`j?I#@IK`W|~sQ^D={yy2Q& z&PVOn(n~X&6a5q;ktGK~k++IZiRW0$09)GWE(x*>XN1|kqSR#2VKrV4O`C4=4aR?Z zQtgy#_qv3+{q5Co8+Jn(6G`o)BOliD@hqJw>Yt=={FvaRjz)-$c2Q!$5x1T&k6Ga)3j8LNJu0KJa zp}27ENqk0jr{<=bQ+203Ob(rT^4>qNd5>5zP}r#3`fqNfBog>IXRFBg1FQn zihdb=;_d0_q~&Aj@0LAV*pd3_%?B+&uQT_EY=1q+Kpo2_t31hUk75Q{c!Nu1Wxd^V zkKjm)u%8caxQ!SEJoSQqYVkh>v0BLK08k%Yrg)ab&7I^IW2prmF|u0JqU`>bGqUgI z&%K}^&fV?HqPR+ZaLAB$ULE{qk(W#L)g{b{-E`$b+wo91^%kD~y3oO;%V#Z_{AjB9 z!hZX|AB%r$@W(lm#ns9J)+K7KF{q}&@XPc*;1?#bYLD*9!$s1sr(RFByPcIU8@_c{ z*e%TNHgnwF%%z$^cOQ}IE`ctSoYy|@gt$&U@)mMQ4bT_0OXoGZ#RS5BNNVF1JI_lZ2V3s@fx3cE%&Nj!1v%Z^MLf(0+?}*}T)XXa1`jms%lhm+AwNF)2MGa`C!8E8-3* zL@Cd^G!`9a^;E2YwqGy$l>X_?ElKM3q40K6qT{(RrS#n|nrKt$4(~DZu|yYi=a7*_Tj_TY&wI~R2rQ#F6GRQlhr*9o_Vsvh+`|Jq=VrQ9;|%(m zIjvHGY^+Ro-r2JXj`jt=uE#)Gb~LA9>4+1LCK;3!ySp;tP^aV-n$cRa${2neVJjf(Fs z`+ohk?#o_#I&__>LOoB-;FAGY=06Ygqzr})80^!_j!C+DQre@TWL_~?6;X zTne$is<-f>^t85h_;s$&{-%{?A37c@lzx{8_nn{@zw*vHh-W3T5aa^$MhvtnJ!{(1ID2p_HJDq+&bWUjY7zoyN`Am znO#`auEyBr4n~LccfC+EIGrPd!406sZ=`7mc!?C?{TFOM8R^?tqp3-YqeoxT&C0MMiap_F) zE&H6jgENu0>`wv0^9wz|E@t?3NA<&xRP3!-zSHXXyPe{R|6F>PE0=s(7M!-sx}1K! z_q9&5GWjK2o5WiC$1ci-+x&W;ESwqpEHPZ%E~-7`ey40$4ZY_zrmd*N<@_MnDE4!! zrNOQc41S&Z#bEgdm>P?LUZNV}3KJ^=qR$fc%#Ggm$F?L=BILet)}^zNY;Up({j)C9 zM-M)^`ZV&vGk<=;EdalAA$Pd?wi_PovdXamzvx!v>{ULU!n=>2*f`}pxO{W4`}4gk zNw?#*k|vAS4Qvnv2Z_PFMe#o%7h=J6A;ayEL?&= zs^7+cv*G!;XRV@LS)+RCsaTisAgJ4E`tHEeeikI)_`m^`eZHG1mhW-aW!Z^;^dDz| z0Q482=2%PJ6MvlBap#ZZ=$2_0JohCdjV6=d16q+mNl?-5s!?hF{BE=PbCZwj_)Fn8 zqjkYjyTlEc^xFDTChcLW#z3$|l}}awu9l}_=Qfz6ZR)Op^mfA|;QuD6`Id3qdiL{j4ysrK7D{Kn6}W?xe_s zt*hurQ>mDpLG=|hFO-V*s`lf=bdHr}%+_uuvZ^)fhmjN;CpQT=r_ z!g?w6&wcF;+$qzmrk9Eyc0mvOdM{+Ex|V8C-Aa?Jn#Su=96TrQW%$GrYE9 zsYJA8v#K36#RWa=i-Taioe@a3R&FRqN~+>fa^N(SD5pLAgP9SyRL{oBi7x15?K3nM z^Mth{%6&fCw%o=`c7w#uS<}d zQz}7g*sITnZ4^od?+0CBMJ1|q(xP4!W{gp;X+1|rHI2SPS6|n7mjoASzz-JQj#Q;> zRr2(J4fHgG^7k0rxcf06MJ#!??|F5CrmpcNq69iCuhMZb?*g zGH*f6TbT^3?X^&rCO^wexl~RtsXo89)ma#&Hb_~?A(SKb%7Vciup*mUs8i=|N0vm$ zXSy$bE-F6k?-i}m;X-*U$w_VWI)?KD*kED{Kk>4(b1^hR+F|G^Y7a6-ZouJ`*yqFY z^7^P5h_e|<_E%#jd{^hT8!)MRb43}rUrv*fG764+kw_f$<e&tKh!}($)+<(h9>=OptO6=X_L{Y>RKL+wN!Tve;mv7q0fceku&q9PPpy2J|c;(oOj}X3O@$GCg9M=T)ou1KM zsCD?=zN8$o^*l1HK;CPO6`DTQup%KbU1e;2r|jYIZhi`+)n{)fy;Tp~=~cC{ZEJz* zo2kNQ-MLVep7k@_aQ=vkyPGmI*@9Vh3YiM^&T6zlgwd2Kk+y|t^{n;j5+ngq{)qNc zvpG8d_O(RDMDqM+I>3B(%vOXU4 zhN4nGji0e#jMX#n9xh)Smf(OZ!sz+D=Kg-pOB|V*Pw0*-ka4)2tinfijq7@v*o_~r1cq0aNFUJgFracW$Z}w>6Jc$P|Ec%=~9aO zs9h9~pF?jTq>M9&6zZ83d|i}uL-oU}6*$7oyN|3pDYvw;b6QF}(5ClqUd`{uD8os@ z0$%G%iwWNFVYbiipn|883#t!yH^_TiQ-|paLW+)0@@Oeg-fP2qT#6qna=|}FM?AXQ zAQ_$^g(v50++NA<4fZ_z*%lh!hVM+j?Rrx4c(uRnS%w5;`A zK}nIi!+6cO{_tF(NP1!$k;Fec5XI^NCZwAl}7_Ur61ALnJG-AOtY7kMY($>X<33+KcB^vrk5 z`)p=;gUVj6zF$t}eJU5It{pzQXZ6QBXqllt<;s;Zii=Hu{i@W=o%Dhj0j*a7k27>E zN~IOL8r<>H{^Q-3RM!w@d`%{HbIB^Lcmp@}@Z9j62BajNBrUfN8#s91&YAZtiLm63ne zaBe@lhUxWfh>qcgtRk30c_PT zW?5c%G&)ifeQadDM+%2nX2^N(^@@gYmz(UrpbGmmQ>uhO3bIGHzYlr2<=GsY*ED!HT-*qaE(6&BcKO<_(`F?C)aj{rM~E}V1I|8@JCsPE=@eR#%C+01sD zyB(%Zg{Bi?^g>$qRBx#WuHh_#zZZ+$e@2vd*XT)#?Cn#wmo@_dgRk#|@21`&k4gq* zDgGS37cO8Y3ES>xe7I4RSM;*y$UlPdsI(+K=hiYtbT|F$)vimNyiju+qNH|Q)BHPM zTb<)WJR{nBbI|c0gO5Gmk}hmF^gXrhyGpPTOTV4&lae%|s0ilETS!D%FaC6QESDyQ z0Bgy)<=93p#w;3rX{$o55fA(F>!(PcB&6HE*DP!p*=3EkML+f&+n!$x)O$Vt`ks+_ zw#}N4uvtCczf}lv;MF~b;goWQ1@KYl>g|$3;)0Z2D*|pvDa~)SlL0(FpXt6$Iv%&K zG?mLzDcJ($fOmfYZ==WzcMO(Mqt|f@EHuKPG31znnq+m&&77vB&i>8V3|!yg%heH$ zC@9j@66=z(XYlTM zkqJIw@}K%5qE}%(Z@DvMoG9cJyG_LGY+<4Is(a5$w|7@)G11}VFg-6yI~8bInvKc8m3go;u)Jno7Ud%`G1n6d?Bj13OlC4&kAzvE zBsgVENMux}+Iyb}PcRb+`orTp9Q0ww2SK429B#A_@lwqA``EKnGWOFR^P(!Xh$12F z;pRGL=gFFLs<(u5w|O&UZWQM6g*|^hT<>xyoU1qNNtp-PZ}kNb;QkS2j1F2l=9I{& zrB${urse~Dsx5mfm`i_Dq^_o%`vKzol?!;!^#MQCVi$i_cWm?v9maBc0Zy_ zioM@i!Mn63UAQYNz?2_;$M?naRLF}@fy7UG?HuTbffV_mLMg?jJAK{k&6-iA;<3s` z=EK-XM{pDdk_iw4P4=e(QE1kxkmgL}njyIRm0#So!MyDis^i+|JIctjoJRH^2Vc8P zm06c7lS|Acq($2FhHY`J;@vMLEr)Pb%-yljmYFXl;m0Vkqc%ds5UM&V&K-R+)8#v2UmNHw={|}t6q+eSMb_gx+A2d z6uu;kW?5Ji4j^b-p!A1^nh8xTe2H1(Z-B#~;JfybNmEA~f}cVGU8I)SV@1IGVPZZ1 z@Ri5v*IAjnlx6aaes0gJI#a^tbXNLcgP4jPx(nVIoMDlrM|scz$=LTwu?#ZxT@ToT zt6+6_WR@CJ%<9%&EcV8j)#zyUAMKJ+kEizU!+1Lmw4x-uJBn3b8F#BG&ouH$=ot&Ea{Fohh{*Tmtyk+9J zcUTvr5)`MnIWug=%S9NcaXs1^#h&l^mcqc@=&~2dkvyEe)|tEM_i~m~LQ5<174v1+ zHeBVi@zbib)<+9;)tI!sAM?KS_2b<9F0C>zWWv*N%XJZ5u(quYT})u>!}=`itc>lU z0~&aX=RpZCY^oabwcZl8`BC9<%klPVI+w+-9bT^HH`Bj!TFl$uto|uQZDrufYOMP9 z*@THRU=fISnRGL%kp6t*eM=7BICNheY6x2xr%UDwJ05Rs)hO{<@$`yBI77o!yP-#T^@g(B-p!VfG@0C%}%y<7rN^kL( zZqPIQd0V2+9ZF|0vo1 zH`@Pm`1+hY(BWp7Ao)MhZjQE+0Da6CsOqa%#*%L&)H)q>vTDg?I=enHGE z4?r5_#z)I5$Z^8FB1bcD#C~E@GP`!cMQ#O*531F9qPD!y&$dn020hjf7lv?es{%-Z z4$yqTFdqG`E6w0-!uphq%G*!$(J3DvjrR;kM$z#$tR+9BVLWT(clc2qVY+SeIuyh@ zGpqF)mrq7tyxnOBe=Y5xXg?7>Gkm4g)OTfKejr1AWAa#iUO*c-ibt*3x#^V0l8-@3 z0`P3XbFRI*p%HJBKX*B|=f(6$p=0wBZ8lq0V8G^YHk9*0Zc)+Jo}5J8QCp9JAqPBF z(qlI!*QMoV!xC$dNE>;R_yWy^XhVD}L%x4hrXp(70vwoxi$nRb(%o=%0BlMW_> zfk#r~G#KFZ0$+HWdK?!A<$yn!+-Npjqgf6k&m(EG1>so@&^?Z@KON`yn z1^n+ZU_RXZuQB=U9yc4u2F|y>r&A`jXg6P+?@~;Kr$jNmdc(oDyJ=RA@mu@CETCOd zzyv0d%Haf+c>Ce1JoL4R$w~-e+OmErFt=xutgZLLL-P(WMf`aDO*Ph1L6x9(&HgSN!v@do{RORHF#x67walGRj98bH_)`D@K}`jFJsk8qu= zvY+3*jSSivXhBWG5PI_kNn?`iF;U8XzNE4K&d$y!sJ>oJ{o$cLi1W~Qf1-8N=&DB} z5SYKnt4dqzGAz=-1r)@z(Lt1>7^YY3k3TP_4HvFsno%1*R)D*1W0Sx5J~3|Og*G39 zMnJ9rZT7Gy^)JvI^&M;3G|AKvd|m5rq!RK2JI|yl zH*QjUD{uB&>dDLBXYvX-*<#mEcG2p)6j;LyIg-xXLd9qIL&wT@;2+;OcbYc)r5*4& zz>TY_)}e+JKMr^-p!$Z=7965$V=`Umt+nNT?b`VezEt6BVQpru#ZZWpXx;-&*O5GH zRMsb9K$rh-1^#lA&6Jshl8$__tgKhehYI1H4nlW-Ls}g=%ec{&nZygAZGmj;ypI!l zsKut__v09oWQ(2c2wx24Px~1X2I}oo&!ZH_tMvTEi)A<2ha1<$Z#O!VE`ob6BPFcV zwQ0PweLdMGRJ7I}?b06TT58(1K4;k)Y)YDk1RvYiD_mShn^g+TM!%PQu@aMhwuA%N z%>RZ)8*s2)iPQ}u9#rNI3n&F*xMf-;7yZ?eA%UdOQz?+E=Cfl8I1^HbFXL2D#x?e= zz}b9^%(1l7pSMI@+ukIds6{Z|d3@Z2aoXE)ui0-le|+U?ny#`3C6_RD75>RJvdWRzUqaJ8~NIE_1a4VBOI&^e!a4>!R_|d+y)!Yrc@6~NGr@Ytj z`C8%h&B->YPiRhGe=5d49Y7%G<>fu0@hF=y1DGI(bdi;;5gis0ynaymWTVw!ZOq+$ zd$09;(L)#_c=P+!svGL^Oiccg+K`dPsP>m1yslv{w1KLMuU5^^9N59zuC>TmD#4EY z9@{RZcEekWJlVqj>^ldy9IsCDt6xNK473H2RJOJ{Sk$!P`ym)?#=VUf%|q{kMANn= z@j-|C2x7RNk-7iq0JlOcufm6ic55HOyoG7AMb58*ZC)EPnDph_2On9acV zp`pv~WSlVj5q%p0uO{`?&Ygn<*A{>M>Nc`GSda0SQC;pDL@`_q*!VSwqEveKm3Z3m zDg?Zkc@H~7tt{=s)hA{kau>6ZBv;+rt66G@9uv;c($p+#T3w~#^=ILGVQhS`KCHVj z-YTRdfuB|94P0m^83hxpoy^Urcr3L2TBqAuMPaj@DQ(6(<2}rR>9udReJhPM<_jx^@~+*6nIzjG{rt&IwU%%hehK_wch(zJTZau<#DK--j}5s?@pz$PmcQ)) z+B0{KZqfzCYGj$eVw&{+cbeRMH!s4@WPdYovxAl$B`0||CU^O|8r4=b`RRs?`Wci8NTxE-uXOC-l}-}*_C^4))0 zk6ygvN!i10x}v@_t8K>^VL?3Eio`ii)Qoq;DUUU2bB6sHx(N^38JzoZ1hq4tWRRIw z^w{0IBv->BKYx4Or}2=mLhc!|(N32y$i$ckQ?)y^+N^lnOu^)zscsTL##p z>q_JA);glBjwFx*Hw8r@`<*nn*5=(lR_f694UZW1fc{)vMY-kT_Rq@KfTJnF_K4x( zSWm)yx03sCX?-xMW1=L2K5(v5T~kwQ2ao6RpJ{UprzcrWiHi28?)9Telt~No4i~bo z+OEXw@&?ZJ-0Vxn_;h~})y*hgJ2bQDE)_Tj#D547`<_DGX7{L@#94lu&ZeB?qg{9aO1_ryf>> zH+Y|z;fkpC8w@!^BJhV3b&A#<{*x zNNFZvA9T5<1Q1z6vtSP2Z}fpXXso+kZhXABX`!HOxhJ1vr&+=YDd(7FILo;JQdfUUhn7en?L6k zkG$7%ZDTbgOAd>T$tY|)7((NxW!+}(=f#!@46L$T{ND@8W6*7S)Kq)AT&~(Sv9UK< z#EiHnqe2@Z=Go;H_UDgdL}5X`updP!><`)%={Q)-`kD@iV=aX|VTT>eHg{>Ko69ZV zNf=l37+H4L)qcjhhjukAw6nd)sT^XHYTKW|$a8PE(A%+C6dSKaqQ_%Eq$Ow^(zg%uP^pi*jjEz0Gcud?ms`S{#tBjC|o zK|x7$LP_xW^a)bD)+N=nw3siImB8)&e8YY?oIwT?kRjYP*5E!ri;5}6Su5t2l(mHy z*jIPP4;Ja6HkG)1Xu2A_GkgSfxE=x^dSexcAwt0LBXc;U9PU9Eew7=R-lw&YI!M<| z2}(=;-0nU(`Y@uYx3@|+i}nh89bs_ZXOQ0dUK)EiI4HM9F=VC|L!o-uR5~~*cH3$g z`@~T6+{y9j0Rem}rDlPPNud&e4MiVL!vlR3e*+YM_LU$^y|GgS;BL8d`67b|Y^UU$B!C{NK7E*j7U9L4BR~RAkklGdMnalxNa&=g*;SaKm3H-f?;%v z<>Cdb%+n2>k~=Cf6eA~=OkeC=G!8Y)oRuLSSSH+MdoP2=(!FyUar~IpDjJ@4Fj-{- zFE?vmLXSe?f|tgB@c`pD(ny!1@qFxKf=176Ac(`j6V22s?|-jaFL&qt)vIYG2up(? zId}KUb{ej3zOd7;zHVBRgLUl8i9A=s#nbu$B%DLbcWMREIxI>4!yl&;;PYedJzaqL zYsCS*9{Ex^>r*7>)s4j!Qv{Yd(9qJ(sNqJ$?c`u z1JlK0>F2~1wIj{~F)rmA{f^@(0$Jq9{Yh~ZR2@LESRT{sGKXRae`r)PJJeJ5@M z-@s>WdVWYbh{VWlZ2$hKr6ueqMZTq=&@R*R2@H1pw(oLiT&Z-4Qdca~o!`im8!NLk zo23o5hK6HmUz%%N(EC_8tZ60L3{`IC3y!OSzki#QimGev=fjaH&?dL-?skMv5rK{` zG1Fu9hVY%c^k+}U+~cHybHsH0ZD`0`Uk4}d2D+T`Asgk@w49UG8o=1J>JT-`F`X;t zD5dy5^aqhazj?U0SJzq%?9VSz5^a!k=loq;4ks|dxIqK$=dX8Nl{6e2CYzszeC+@#eCFB_Qj1}^(M!h5otcE zhXj*t$hxRFE@kp!TFUN<+2Rk1D8fUo-RJZ!;>^L!Fp3s&fiIJXv3mzN9B}bKo&3e& zLhYX3-T>d@kb{G>ap)=BkR@v`-S7iBu7NmZlPb^deGH7aw`a5@LS9esRWUYpKWCCG z#G`kg9u%%=j| z@96K}h*Tl{KClx=X5_!JN>-&EeCiERkH4YEB4snNi^esrq2R$p+dUMW%E8OyxwsIS z(p~`eNawwzaWHpMd&**AQ7aUcdaWwgJ$EdnW z=K^)dw)G{>Ga%SoMV=eHaa&GmZ~1e_qOOW*%kn{`_3hi=B82FJVY|zp6Kf>HnEE^x z_tq`TQ3i%rmfko%K<(w^s77?iyo_sn?=+?tleBo2}xz!2{is+TK1-o2?Xh9R0+}vK)yc_^x{mc35{o560t_ zf{*>TDCkLrFO_m~5+c%qYa{aS-@mskXX9INlVy0^@$L+zRbkwuaH5{R zVZ923WFQ@hh~-iqsmHvPS1C%@XIUKI*4^WE8@FLS)C&th-0_h8qKM>G^jh8B4QM?_ zB;Nk^t?F({a-yxRblYMbnhkdFk$J631hp^e+2S(f~FlbL9JTWmD+PID|heIk@Ln~_cUwMpIK50h9G_oIgH`SabM!`yb4XRVANwM2q;WDW}}8@IBbnB2|$ zULbrFi6zE<5ji}_WmMupf&ykkV4bhZeEcp%!zgm+uCW*;NROqux{4=gxgEUnecGEh z1dba6Q-Ju({-(t~5i6<9w!{`DS7$H!G{9=ev-4(Waq;G4+n0iszX!5ei%hEBaol@E zC>vpQ53I&AihU_+jRakZ&-roZddtCF76yvnYY(OA)9hPn-A=zo+rEg3Whb@=;y|}u z6%?vkGFA#nlPGZ{Y#N%~FkAl}I4=dq5Vlgg6N6Wpcvi{9wARqQ zKUN^_fP8}E2qbLladF4&`eZG-IH?cUhOe_$kA0Kwv+(Ql+>e}v#G*1w+2Hp98LB}X zeQF*4M#1YBY5J;~eKr$q{HG?Q6zW~^Br;E&AQSD=K&zcIj7mc_Ce_Wj*5C;`eI4A? zrQXS_!;I@_&ehS(_3wn~0R96%%ye*BYq3XlmuV4^r(=DXkxeHEruxW-fl4;_#413c zKZME=a`X8s>t7sLMSUag)E3Y^X~k$7cojcXunIz~Vun3^26q%vfj zxy08wny1{Se(8PG=mBJP{eud}2xQ=Vt+1IiV#oA#+v|UgeRm9848OS2I6f|r&}fI# z+H9qq${qAO{E?n0Jv)vS&`tLnWj&K2Y3~6{Y^?vmgQq$5!tXjZQ{}KpD9bcbyMn8k z^{cXmxnlHY10CP)uhec(yI+a9sgxn*sO&z(64kl$`=GMCZQD@RI&kOPwy5?O`B%S2 ztL_%sy>64VPFCu2sh()^#=%e<^^-vua{Uwx3!=Boy`4Uu^1we|OK8k}8huRBjb<** z{DOkm>1G!YT1=C8{^GIl_CX_DBLN2Q4kU2b_e4(C*yB;tsdo3$LNe%E)^~var4wLf zD$BU<1t~Sx1Gc8nC~oJbYb)s(7@ZeHn8D6Jp>e;nq-J-7 z+u7Jm1tKk6j7*%&CyO{;TWWL>VmyD&-59ZFT^%+S;TUK>?7$y26bgbR1!~L&Hb+VE zS3M#ATR;`Pb*UJbBPp}H56ihhL6h9!VYyMMA7#py$8UKyJuxi-q|+y=0wGD|KF+9? z^eEEAc?!=dDh8@DMCHalmu%LjRAb%u>W!0`>yckfkfP}YJyl%msSVbox`<5#0i|vB zb)xzEKs2^!24wLbQ8{ zv~!m-IkU&U@^}W&$JI6*jh3(%4!kHKYAl%Q@{~1i9+?+sA}yvt5lf>5b_x z1bNlve73CS?PP^EbMkzt?=)$ME`dpvMU7jrf|4U*0SUG{4q{X}JVt}ozHO$JcSQy5 z#!)K2{pY_3XGmuS{-$~xaGEXTYn5OY2h#n=2R@@X4)@abmwo}Kqi>&R=)dYHU!$>5 z8hIo6DmvmKFOgJ$0YJp{zURnkz=CjTAW9n;ti4fJsdpL$E;sRNeere>xd`Cm zQgz+1XQlzR4`9lwZ%DjmhZh^7t=OwOhXiv%B>}SU`(W3K%C2+8YapGZ38>b9QT#rB z?iCs#@zVJ282HI3t@cjJz#cku#v7DSAJZrDlJ-P$0T(JK1EokQz1FNRC_oohwol(*970NJGk5lS9-RZ>xBCVorl-o>+$rBqiRymPTaFW) zZl+~0c!m{(GcDe`ZWbs7;OizUY}%Um991V2K7~%ak&I;z%6%!8sT}AlEXy@uC#cIw z`N{w4+RJ-sJe-8+0{da`u!ZePMPyvlcCta=Nu10P_KXk}*bnocXeW~{Urw;h+>7j( zZD}CfL|J&gpPFsJWs57+Q`~VIGCG;}$h#KwSCu`m`_4v?F+uU$-yoDekRw7W{X0r4 zgSF0c3Xfi%p;8APEl5jQRH(}=M8euXUQ)H6_)hGGe3zdW2~k;Ow>B^cv>dUW>T;$I z&9)>MZvxB5BkxBQ{_xzTvmM3Zr~W4oGhgYhOB2CY{QT8(58{(Yqs=zEwL;0lfcHe{ zN5yc+oKF%SnFXgO8_WM>u^z)=XGpu8ew&Mm+auKy1)$(^8eNx~dWVxTLj3oE|DvXE z@n#V^Y(Z7)B_Gxg4>Me&e!)&r&7Rec2h)6V@r}9`Vgv46{r8u**Oqt9F{|QM|NaK# z2XJ5i%YB`YNdF%Yw*8g{WxUNA8ztKRw>~)7+^mJjlf(WwX_^Zo_Zy&uj(^V}8436a!oHl0OkX zUdYRX1&MNtA5G|{+%3f+&Yyf{Lrtx|uGpZ>AuVc%)SCqWC&9Y*n1#gBo?3liqWZHH zcH8Q2@XQRDGQYJL6d)Ut0n7qaxxI01^Ptf(=L}w6{I5XX--Roy=}&}- zM^QEq5sKe-$d}+FWgxf2sJxSUqmPy^+$Q-h`PGgmvTb*cF1=vNA4&f9?Heb|k}y1h z^Th9DWgMMWU9QfDfJ$(3M!DhoblYOrn!3v2fidXPb*0S?3E7yayB$2yW`Uk+CX66? zPcl|T`Qz2oZt~oYsM1uz*t##qq9&Nq{zUyO)tiGKiDC^?fFj**n}4eq^nO9tyEqKC zjPG+RXlZHzQD~858wKxmH=>PQ>UY>dMK&;42DY};-R`KEhC0wf=I1tZdrFhf3e-Jr z)3%xz=ZucyQIG~9Xvnp{)~~u0cV>#q4YjJ)JsR1=#WS?&JED2KrwSm!#fh1~%OC$< zw^JxCZRW0_VL>@rtZzMs$}u+y(}A4Y!z(M_v@@iYeb!%})Ne~dtyXs@vxAz2L^%Jt z&LmAW`D~)GLyi#_D@{w~8XLu>GX~$#mJR2*=dLz0{EEC6xLJZY+R4D+y4m@|PCvhP z0M@F*R{J)yAd8y)XqkQ|1KtDw*h1zg1J{O|F44tVD$!hAmRXMMD`7N7?ahds3u#UVv9#dEzrStu^_Iit`c+0{H{68xT1vUG6t2`aCAjhtYA^Dk z%p22D{a;Dt5>|8gZrx1t!*TVG=g+&;H(g2!H03)Rui0QYT zYO(46%-@{_eU&r#W_`3IP)Z0lQCo7<1{ey>T0_EuixV65>iY77)ZREM^ABJ>f*mj( zqQ={%fQyNa-VKeo$aaOLAiuz?ttesKDV?ydK*0C{b?x9%l6~tSsAac#{Tj^3##B8* z4R4D$uAANZoC~DEK|jyT%$lI?0ieH)Zg!=p!=aCdy(}#slq3raZ5BCajDf9^cJw0( zbm)6m@U2H5@_S%_xX|%_UhwINl=DvBN9hvpwJ~Q&ZNVEq`NIUX)8w6gNo}C(IlUGa zC2Wyta)Fx+sJMwnPt*=;$(8`;3WMsafR_cN4Z8RB!5P0S*O6NQSW>EY8Mh4j z@u#3ac;0_uVF7x;9rx))bF1LQlbjfNd;maD1)qM&QjcpX3r4i%x;E9?pksZf0x28y zQ}^8N2PfNL{Jb~g9LKyj$MCf>OHv(ugcr^f~E&qT+$2a5j;WbV0Q{_tN}FUSJh;RE=7w35jgBL!v0pJ)edk8 z$tfmNn-3N|#G)`z$LY0l*>#_=KXzKmD$w0qs=r=U-u1(M5jLYVK>tXUw<)_$Bx+bR zdtF5G(trb7 zq`{+L6z#k-C+w4)kxFYpfSKE2+Co8g%w3Jr%@h1}$^O2N&wf8$Y3=mh$l3vqc6Ruw zw{&``|4~q8Ci4pm+sFfxPt;FQHB{W+xB*@WT-Wk$6mhF7MmP?)$$6b!?TjF41Ox9vzjkRtHuHn z{x2S8iB|^|>O28d*5BHa)Y}oEI`hF)EXf@Wa zp!Q~?TXN;Y=MwxMk3u^ZuxyINkpqmh(*%X!k{(;eJ2o;hveSL;+&S#?kqM=pHm*`1 z^u^WtD&ij$Qd-YYU8fx)uY!aM1GjkZ{NClA;NT*F{A00@;Gp?hxsv2ymwV|vP`{^P zrxg|Q^KE>mJ%iy|qC}Uy;`8(H0Q;`*kYMy%ru2JzogbOAj=}zih(OYkydD0uu1MHg zs5rd#vP}3Q>%Ox|;#o)4BN;JJDI)PB4hgB)jS{`28U>7RluaKNpi#igcPkP9g0Omh zf^3)XY#~DUQjckxP4R9IE@?Zuepwyp-ZqtCypd;Ev_(NHFCq>BKrYJDw8K|h`JgK( zkS;b7F*}_7!MMMHMtF>(lIj?|^9DV=vZ06LLlL||Ira%%tZdS$x=W28j$MII)YfpFA(bk4hXj#KiS>UGV9g(sc=iT6Yj@L*0K?-1#ucKV+gYE~fw+0d~{t*_QY;DbpJ0uqxTz(vXbxy@guX(=0vM2<=uxzeujp@ zq|)IPzvXwN_=Nbwy;iTETaGFgRV;ipzL?`H{TY{3?@0_3HqjAG6NlY(7R^4GAneBH zi;WI~)m3Fzp`-kaN}n<_N6K;eDGcltHfx4qf8c?~tAn{_pkW|;IOw_J3Uo0+Bf2}z zKy5rBA>lIfo1>WPlqwxxqG+w_wQ8sMdIU_d+0j*Ad+*v)A%9Hza2a{Wt_A2 zcA{c9WgLJuQ&m1ZOZiMd5r0fpF5UZ_Z%KVBU_V-PzGqlU@E6IUnB5%lCu|pLg)?#n z5N*x?0|#)j{lvtD(Go=I|DH3fM>7@y%ou-JjMrNbHbTH$>3BRJI#K$s5t9)uuP7?A zK-aYwii_9PjEqbL8@45Izu#XcmRO*r7j@C~j?IAt9#Hu9NDoL_6gz}Hi@yu{iR}Pj z{&0prj)7ZwCs$WF#`q`S?(+0TJ{#m#aC!dmZ`eUFP?A@}FoFT&L z<&BII9r*o((ZcP9FcCWbE|dlmXd|tQXjlemU^fTDj$TX|OjpU*aYM{=5B?%V^z zM6954sJK7q1>)L5+~-A*nw~Sh+ah7%tvC>_PSs-`(Jn469asChmS-ser+Gg6VZ>%5 z#j0@F9jKfcn5&F@evBRbq8EDfIx7jkX_i0dZ}i;Mq1y1nq^&}$%Yq{Lwc+ywS8(FL z0Tp*ebWO3BV!fE`v{8F6BRYGKy3qYb4NuD;a!+@6gT{!VQab;IH{t>Lc_-MOcGap}e*+TuG<{!>3!D<1FnBK3oy<`>T)+WGqbU1xE zyxO)NeMwcVv(vU(t|70Y=vISR?>KNXS>^e!u(F~z03q8}JL4{SH+Sa88VJYI3cJ#@ z1>(9n74PQ`F8c&tzwEXLe4#^hmo@~tuTgQ6QS}Y*SY6w;!=+2j_`(kN8yDiW0)iW% zHc~2B+l1T7$|5qNqE9aw*(@4CbYWH#C7!l>o1+qd+B;@)Jg7?hUwcraiDChD4hD6j zhYafpCUn@r`rZ}nmku6%^CO7O6TR)L*Hz+_(~>ebWS%C<`tRhGt+jW@-{AAI^tpD5 z>d}>5D_KX7V+3CcKARVm`}_Kky}g@TlrOTm`pweLy&aEfo&WVsDWq>Ca6>?>veQR> zEsUNz6oq|A>#xXDJ-Y$}ORG`kP@e53b-6q=!#Y5eN30)m1D6BWJuF?G2m zjp4gJ-@o7&&GLKkWg8Sk5ZZVBI}4FW>iND9jS(CC=8sX zh@u{zmsD@wya@sv8yg$i>(&jaYdgaRXFzQ|0$tk(Nq4M=sF9^>|Da4dM>V0)u^3Iy8r=>pw-)reKaNao{ zNiZ{V%R|9nD(40B@@_yTa|0j6{+QoW#;Ur}@qO z>|x-P2?CA8_`?|=0@Ap<#}Cg0a}gtOZWrg$-4qp75tKn~Z2zj_QhaS-!1){ST^>>D zHH+|@;cM!p#(Sg1^JYGHw|0l*8No>$L0#S$ z%%@gEOTpz#m*XFu`Vi>1x%kig;Rq057Z;`r;P zwE;A)aqVLCcI=nt@VFLOhv*tGnX26OyIdBxHWa+Q1CPtO^|n56bp7wCyU16w-p7h- zAw2NW(doO=4-c}1l6;)?YO;K>z)$c+k{L@)n#<i2CO>o-21-Z&=K;*MP;=AOF>cfi&) zUEI1&EoU}li_I(zUiWz1D0|ap&e{>Xr zb`7imc)+)B6m972-eFe2>fkJYjysD0)%}O(Kr3eeAKa?)=|{?7?xoNF9T{0@3koFf z4wth+_WTqDBx(+U@)d{%PJ#v&Irq8OrW4=ND)fh+1K8YT+aRu+G+K-+u^tMO26^@;T z3(j?j(efGSUtOX{{4eU>0xHV2-5W>UDgp{B0-}H*iZs%aI*Lk5H$#fFbb~=CNK2QJ z(hZUWA}!rHQqtWy-!*La8|Q!i@3+o*&pK;;thM)mGxI$6bKlo}_3w92K;I6dTXal} z2I1;S9C9IG{ez!Jfv->Pv#1`Z+_M|9T57D0GKpL9+z>mmL zd(K@ls%T@;s^uo3unpR!*lwYpyYqMizw@gXL-mp2e4jhN&9VwoIS0hX?g6c*m*5?o zs?+C>J8!Y$qz3j_cmYiL->@ z4v|pn-Tg_zU12*^*I7u&|MOl-jsuC0EOQOFp|!0o^D?g7Yb(5K)Zq#_+TW(oH?enk zp#||z6II!WnMPmB^Ys$ac{p81p-`D=TJs}%IgGZ^s4&(^CjQRe;Zb-v9~mDv5w5)e zrB{u`k_e>MRaMEaU2|Fzq4$r0#Skl~s;bJ%%hQR#aSbY=^NDxh zIPp!#`NS{xZ#+{}RHTC=j8v7Cm6iIo@7=o>z`$wJ?uC^l`1$|gC^(jW?EAlG5`O!~ zO!NN+7eVwuF*<9uXT6EbxHpV-;H&m}!@Sw@(Da-a<#jdXXT(PV*wn7COJ!Tq+p`)D zBL{>+cr1|O+m9`)-zc$O*-Tg-%$;akd9pcZyllPLiE=7_UF`-zn=pBNq`>UdT0p4{ z1>0IFI8(1*aaxS~oZ2I-a9o*TtyI2x^rdR|jUV{T@bk`=%B8@1zevNEJSmKI)n4E- zT_vxewEtC%|S;j;sS*)l20iX2Rt zRx@uKJ=mw3sFzl)a6BjJwP{qpxFxVV`S9kG^XXsN#$^PBXH%!?1g|(Bv<~63QR99F zCU@v5w~x3P>aZF@N^aA^JJB+)*H3q|VTbmUpwxPn)2UVdZO5}py{=?o)gqJL zN|$Sq$z~<`-K;2+kF@kZWH#sR8eO(WTG=FIWfAF$E&ddOGV1DicFRW{30`u_7At`;WC_vnR==|wB`DX*y*64ODw2z^dMf(6CC;9 zJQl5=9M*4>O?Zog)(Z@cC3iMqkNELiPEk=RJYjAqyLf~$VScYQ#%&4h_Qfj9uiigS zLQD*Z>%SsYRhv2z>T9lt3@xfu?0Hoy`BuX8pT-w@j4Sd`ernXoyu+7A6V6dXKp$PO z;VgXz(Uo_W?X=juK8}GTv522;bKIK1^tR6!Va>$q#yr2f_Do7FdGN=NFAsJ^{U#V5 z)k6C(n^$+LE`RUovItSa34K()6u+N{kfN=eVt;bOLpA9dMj1L-ytU$ z^PBVIH^&otG0bb#U-*{CY}j&>OX-n7+Nb48wW0kW%4!M!WH=S`kqkjE_ain2qQl=c zoK|R7AAJ4R>#aXNzE6dwxuuv6?HKe{O>$i>h9gjiZ5 zW*H_^CgH=&$56=!8DCqv9+}2YKyhB?^x;}_ou*(}m~czOQ~sAfUpmM~T=d3!B>ZQU z=w&Z7J2+3SNoxh3e3#R`%}~BA$4CgkYPGh zcr;U!l$Lh1a>yq`JXC0Wx%JDRivDFaU1V|$Z(yNOPjQ9Q_B+onZ?+Z%15~uXq+_jw z=0U`rGZ(0YO2LALonN>X_~rgaO=;E>KCFX)^QiEe`PcpS&k5e~&xotTAxZVE>a~k2 z=Fne-0|^aPW5GEBvt zTenst&Pf-TjnMp%q2RaG8SEa|-=Zi`mgqyXjeA~ue8ZiF@(KdtD2-01_H2}k+UN#Q zlcmZTnC_V7SKklr8^hjqQI!^Zj?}Wd2?W{J)R#z`vr*FMaI* z!|NLJv)%WdcYOc**A#3qX14X+15z>foaUbY2+y}F>L0mh{ENxtd$@=9E1J&fmuuVg z1KS5=j<4uH?>Gu=^}pROowxdWacSEH+2?|_xz8^i=Uc;XwJl+RY%6GcCnp2Db35uE z85HJp&k-a%_HA0a1mx4RUHN5L+~19;5C(-b>)B2!5iK$DLz4_i zvDt{x@K*~Dyek(d_&6QbCiR;`UF1(C%VD1dFdw7$n)~|fHfB0h)Pd3kXCIZT4+>Zr8xG@BY$-W7$9&Gz2tE}cf8;v zmt0d+Qya=uD=sqZs_TxIXQ=nR25BNwVPTX?{_|ODKHdGz`97!3IfeY^EpV01#UaQb z4I$paM(X||Q&Z5ADwndiwt1e~sI@?65yq7mm{keo9-{ z2&XQm^~`mpNT;lyk`8C1=HugY5VyA8SQ;toQkO~ZVbLfpfQk%E%!`4%DY%R8qCb`J z*5>AU0)p$@Cdw5sseRd)A#>DKlrStIt7dsPuSGlxg(A{{u|Q;^nWrZTv)>whFSuMh zsKghaI!mLhH(iO<%?&eHXuQ2Jus+w5K6{V|awQ2zJk5fY9x1g+SIivD(XIjF%?jsT zcKodL-FB)ZA!F1A({jOgnCQj^|N0&f@baPWEbD8B-Q>>tq* zT|hv4>vdW;ohdtyLVqe7F07UowlTmToJ)HZd_#^X*#@GJJbuqoAPRV1F6awQfi!5zb~Y{++cF>bO_q6B1B6 z<36FxYRM5-f3-w`XqCd$nGfV8$)!lau`QfgItMpkndSxy=T|4{QzRntU^*|;N#Rmo zfXb0O*&>CeQL(;^Ax+c|Jz+CuYgRbIf{FN$P;kC@ zW@2LED8vMEKhhM!m?9aa2vc>H<9P-=m2{j?ic}0NNStbB>r&?{|9A4Z9WXm5PMuNz z`tq|{u^G~Nwf1NATf3#t!mIxnW#kRsP}2e=^yYY!X+UQgiD>JS)p2PznZ3yPJXIpeYy>5k-MX>$4-Ik9wXq;`SGfR;n-wj za4O;JuWp!1#q>zj{1g!9{BW!2XzGcz+d z`#Bq#JJC6)x0{(Y4w;&oUcPkcco8G+>Ut85gQQ+zRAhoh)zj0nx8H%m2nYzkbB<=K zPYl>fNPI{Y$?mJKtxaTa@hmfWYQ>=&s1!S;7EZlz}`@}LtJc*rsOeG_ooz0mWT)n2*}Z@esl5WW0S!G zQCw|pZHw(CjSw*|wj>cgpt_*P*nIeF$5*ILg>spXmcy1J zWl?9_QZ{S<^jhL}xyxMR_J&S`Q{ML`7x(fhdGg$Da0^Ghv4anhcY>UA_L9k+OT+9O-&89<>XBgN@3Rw z#dNr3%#m+6`&%L!LBj_J%a7dPoCOhYV$so9Sgy&wbEDD>*GLh({QQJiQyf4IkE)7t zJp_$eQBeVx-hIsvvj*oq1vcu{eGZ4)u5E5+!UREf!8dHB$Q;cQ_!K5b$K0INeP=2R z8Od$xc(AkTdH$NCg98|?tQQ^eb*>~}i>~uozmS*z=;PxcsX0~ta_rmT z{x*;KsL27Nu-Wiem=rJ~P0Fuwn~RC!m2-8Xc+4;0;Tge{sk6lK+nH@SKY3*rSm}CL z3f2{bSaQ^ejSPW6sB37*Axq>_WkH$U9chN!IIT^-UmYF+DS8@6Lnj^Uyf!JiyEf%P zG}Ro2vRxeXBG6*ngCq;s9NkiPl+(5uKYtmheoTrl8OKAut~W&SFnG|!GBPr!G4&lC z9gU6jf;H!UjF>)ct9$tHp>(W(q56-ZA*UbVcV$yKozVU#aPIqoYT^Kk0H7f-jiWn1 z;-LiMx;j|z0+R4p4E4hJXrKPrW-G;T^R{{eAU0q!-T(|(g zKr~PWo(`x`S!9XnP!VYFI8fbSGC0XwV0$huEJSh|>O`x7u7cGeVUTaIm(_%kfXR6D z=uzVIW*s~&EC(?{z;WYgdJnmDq@aKRXu7-3J1cls7&gE_gRZWynr?MJcNxrXM0~xx z<3Y+ccmKcuxN%0aok>zsQt+4;@bRIETM?vK4uzv-c>Hs{Hn542RCd4~Ha+Orv**d> z(-oQDKeC)@r%siR96u)xG4aE%rKJ!j!!snSX4-;lggV>;Z)&L2b`c8WZp)(;pwXg2 zq0bc_81W9$Z6B_3B!l?@Ed+B3&+oVvmV0~o_4RAlqMB5ipcrt9>RJhrPWz_^qobp+ zGqKjp#3Tc9V~s(#U{8iSlP&Q(I6AVZ7QCo_a~c2wz7n??k(CF)jT{pb<7<^JpPFYe z@wlt`x-<0C*VMpj#Kv-L!uGo5cCLmL=}hfD1Cn$9{{7YQS}@$aFj?UF@qvpAazF*K zq?5}Q=9d?hYff=-oUog#o?c>lx^WDj&EEEM+%Hw|aMWP&;V?B>HR-wx2Fh9|c_o%8fkkr;dSv60BpcZvAa! zbH9fZV2;571I@s}oorvEXW81?3i=ojsGPDFcP0-wW=ceZZzuDgqMrn-!^Xzu&bZ)Us*Ot4mX>0PIKruuruKN z4()7$O$%m)^o5gdajiTzi~Eb=uXhuoPvA7XgnmG*Hw_M$W{IV~oUXch^y+vFQM^P1 zd!1Kp6eaG}OP6%6r>82WTTM0TtImV5`u5}S^?0q1ap557;;T+nf{q(A;Ks-DTF?-R z*3&(T9Kp7i=$#T47N(+#1Fg&i>-Eh8AEpcqh7Tlu($WFM#)AmtTuMSBnoau~3^Z2A zrBtWZ8@B&QrHdM_4CpR=QkFkXp9b^}1k5erG;iL%MOXGbgAGC>;(f5z%vz@b!@NaL zPruoE5(Kh8M;k0`bAkVb10KV2Gsw)|coNh+g-^oN4L)6VH8Lzdqf5(zI=l=Za>-u$f1Cf08YGtvGEW%4dC8^3N!*u%^ty2 zt4VvQak#|F0CbGy*v7W8`Ir!J$xVJ4a9HHwK zAyT<-3BrzeH@Vz@n!N@*qVG>Ey|ni>X6uZEE+t~;u4*O$m*9;RHc9p~;rk~8c>P=O zC!&S@ON-=~pXQayAIBm8a~I-eZTRQ=d*E7y{TcHOXy6Lr4`csz=X(H2K*-nMVcybl zev8eg3f}p9)Q5XaNAQE@fAh^e7Pc3ho~OK!^VXhBL%MUZ!i1YFicMw8sbFj+21T3^ z-~4sm(&P&tzwZ<>9keTVop{WEe~rL1Je;ghHAb`+`vl9>^CxROn(vab)@xRhhv_^0 zuzl#|KbU9gC7Wt#VNv3^=on^JK|`Y%sm2SAEd&=VEK(_5`uc(J<@*eIci76ElZ4#v zDyHxp*ihe|^Sy0s8(9IqnDxTWIY!3x;QFJT`5$qcs0`A$fE?|vUv?}R9WUTuv=101 zhLAHg^_EoFC5S7@z$G|C-5Trv{(TP~wdNDUqf)Y{v6Pw<#(h2J{fUnZsZfsRlO^LW z#sz4Z@YwRJ^N2y)Lw>tOnLHB@@x3`&o#!dyL*zf9jeYpO}~phvdL3|3`{{c-BjQthT!(5v-J5E9jEe zRJGU^%kS1XX~=q0liO&g5b;^L(sJ|F1K~us&CfMY1_lR7SQR-1XvXU;gwbCe#Bnc@ z@5Q^+n@b-!q&+T7GgT`V*zVi)(@Bu z6U(8wX~2s5QLF%EohYIIFyb8@HMKcuz{igt!^6My&HDzwfB&8#q=^%okkq^B@O=4( zXeMS|zCbn!qtrah6N6d}Xr`sAJe-*chgJhT>gPw?nMYxpuq<=krm%r))gBZ#htkKy zGIN^uPVQHtXRS*^O$X18To}zWEt4+EV^qq1@8h4XkiR!P!12K+Qv+o&zZD}`W+w z=LKi7*;7#eAY7lHDXXZ6(q3I*#%4W6)soM8e){PiV8BpvfTXMRkNxsynf4>2?)KOa z%5}nXb*?fF3AD`>&>r2qytJh&wVo{LP6U}=bhcFc;?MnU3<=^7%@P{-JF1B zd{RumeEISfJP(Q6`_ge(J=35ZK}p6CQk*S6Dzsiib@>{jdrn@Stg>=Bngk4iFOrv! z5085E(Gz@X9XJS-D7`O3qvCm{VdKh57~-9Uu%Dmrd%>gWM&=<~lYyd#w}Wlw`n3LIPd&#wlbn_w3e#eAUl^LAX{B8k`4`5<2R{W~u8u)2l^5jfi85ibVfRT4ar zQK1EexfijHGBdJt$40(8srfZ{Cg@75zCoL73<_V`3q%tpqnPEtd?KP@VjAYXx;O4Y z6#r6tnb=7h;+YR0;O6v$6xXjuM@LVtNgrVi`Oo03M$~MHeXb6ac!1}|EtVJo)%sr3 z6`P(&n=JHK`Xjq7DrD&K?7F1%7sZIT8QI$(>VPtS{u&(YgO(0oNZca{4ZtiDcEL&IJ2%X!B zp~l8sC4Vz~Qr4?njtlRFrAI3%u2WFR6^c!zDKOwu9~{n^pi7MFehm0i3SS^7AD^4s zgX>RttBwehZt|D_;g9XgdVBj&wOyjDAvEE55DAvr4CGq%b?DYRZ7j{lGWJzJ^tycM zvJ%r_gVX(rR(gV)i~0Ha)LBB8uFo+jWK6lk zT7gN2N#9ZhvKup4T`fd2o|Nt*o?_0I_ytK5$ji(7MLrFhB;bAKVBRv zx`-e{TYgeBb|;cewFVlKwpd}*6>EMs*S32cZj6cr?}*~>A00&TWTY$P*MYp0Z7nb> zw;S&U)xt_$L}aK}CYsJGZexy|YSUibeB!^Wn3iL^>3ylzr$VED^8EzA?F)0m)OUV< zsWKIVo;D|>r2NcTlb_u&F*TLkzWibP`qv6rkXQb%-NW$Q2ntjeAqM;?mL$($V`Y_} zn|t>RmhJh5lgpY$kmZ>Ybu)AG>>i^`2;4I=o?V zbce-*S*=()?(V)<^{J*1W~Z_7BpKbhfuW(E!*9a#*fZU5gf0h#hlk5Q2d|;7Mb%|& znMbFeNVoZ2hc4t?db3qR#*#o=Cuuf@A-S@YwtemQ)rO1vttj2A|@uawzs#p8<2%KH#k<5fhq~4 zBIzR8!s3DlQDgnN?^xrzu(-%km{EMMpC6Z6XJTA}8LolAg~U5k*53w&N5XXDNKj`? zv_URaUbQ{_{;yvBqxv$vn(9wn?;bS*)_<4bLS7?uMucy}&7y#RgpWDbD3_AzIF7N6%H(Wrk!uqsl zMqq5!%DtOd$M(kj>hIb5XUY5z&h&rb%}@b$cJ?yk`=x)DV&EP-3s1X^Y;5~h2%+Y< zz=D;*DpdO|tRwU9Y}r{n0=^itr2~&z@%=wCG~xzMC#f`M+*>dA2~oY0!|n8^u%BJy z471-|>uJB35IN2jt_SfJ<~Aevu->5Q$| zSJzOtHAhD-{<(1P3zpe9jW%rBC`Z^)K?PifonP#-DF-yLnrG#z=)SBAC80mlmz#n@ zv44u|A1+St7EbuTR6`+Kx(eoJ(eM5JNf9&WJz*zlmRqaf%8aH&L-%>I(zf}7;cn_bMH$g4?u4g?yX023@;$f3>L5llx;V+PgTAb)h=i+JSo~!2yBf z>vQkAE;_&G_E?d{`OtZRK;?Ez754WVK6gE_3-RwP!0$low`|;U+gebu?R#aty0z81 zgca76fxt3oM6^4P0T&9&aM=4E;A;e7U;p*2{L3l3C2e&sMyeR(e8g!B^iFfkP=!c- zvId{sN9^jV<1vI7HRfG8OL(}t){|cDuwDQVHMOu-ONT+KW7*pm-8XUSSQhUy?cZ=g zFE@qKyKD|_0*+ApR#27(pTPqQdWqf5VqxjnSa!oVcAwwyn4k7NbU?_;%3@qy@+yCC z#K&6gFF2!$&6YPL_m?zKR}edjX*=6!!XsWsrukEEHM7xfKcB^o3LegNUqVc_Ejuh_ zukUYB-8haXBj|jqZUi@Sn=4Pu{M0g9U1^|j-Zs0fw0Ysu`|rT5tbE{B4e4S8P2=8Q z%I|G#lC0tDgIx*=3g2MW%&l*=;#ZpX^<1^-MQMX>8^q~M$HF80y;UUOk~zatZGuY(vj=Px!?7edy;DnDxm#G^xIlDiK1!W$%X;8!`QYsS*9N8z|L;bN;w( zSy&Nzb)9vwD#sjCEx5nMs>zmZof@R_EZ`p0L@#2~f#18=c0}J1(LjcEKmogA0TW3b zw?vaDJ=>d`9_(`J+Qr7&CI9 zT;(#q+E4})_8E6Fp6cl6$VRwqk5*iWDXkoFU06B6!_)VHxwff^Ve>RZTfdmqA)yw? z)6rpOmWAxe7S?;_b}gz+ALlLNbHnd_A$pTdFNyP1je%bk_DM3rvL0a3KsZ9u|GcOE zoLKRsM2UEC*ndj%RM%eO0LEFCD_3iaqpD?c@wubckud5gVbEfvG$7p|hKKOKP5}VV~ z!Ikuro)@mlkERTRuZn7kW)6AkxIGt$=0MlkoM;RiE_7N(PP6jz-FxnGoYat_fr4=A z4?kbu%%M*mU#XrHz8LN9XW#J}la3V{$=5aZ48y9;CUxJ(a-S_X&G}t8jJ~t!iep|s zjP>$d8Yp1~7O;-HHrJd54G-D?F!^JJ26V}*HZgKLGN{j-WQ~>_r3n$8Vpn+{x&e=8 zf{CSQm6A#h1^x5K>9eSc$>CjmJSV4MsO;gqA`Bb-jLQ(hz`}C06X!-LY+*S#DH@^L z#V?xF`9@PSSu|78X=@l+8N!gP%p^dg(-msU7{=T|Nd7Xvmn5vPQ?nyG*6M(h>3LPE!$7OFQ0O;TvZ>?KkaB(_B( z;AYb5c+twccY;pabhNaQs*t6R{rHiXge11nqhiQiR1VapNzLG^Vkl*d_rJdks*QJ* z6YI#`cni)V%}u2LfyKhAFov`6kF)5&r?gNVX_@>ssy&@0IAps(pj=X}KES@L{_|kh z2d3@%892ua=Vt}%`BaHnSuSJwH~||+D7^%J^LT4^mK9l2Dmfntp{$Br+Mv{=8f$L3b8TxB0v)OGOc=(X8Xp&#P9L#Dy zN;y)xF+0M6&mcb5F;PEu@#5GMnj>CICWrl6s}!p#%gcyI^$@3n`i%rQ2(%@`a(8u&(4bu#xm6&r;=)t|R@~ znr}_$v3)5G+D8wyl=DpVAAN8B!FWrO_1iaKeUBL$ap_sRb<^|0K#6hN@UD+6Q@&P5 z(8|iY-`{eWoK;^sLKAYEjdo^;Cgi)tv~|Cc&9zjx|N0;nb;u_|6DXgiT;0+lncj07 z`4cn@nE3uYhpsMzHhrZ(6+uuIKY?S#Q5Zd{$hfO%Xox=_U#zxcf-aKVkSXx#hhUJD z^>n#rPNnQVjKEl!`Cu`#Y~oYyE4VQ1Nlx%NLD(F3*Yr_BXnk8QY{cuZyB3n4-?b7b zte8GEt(0B;Mg#>U0bM7ficFwogN*Z&8#f8o4o|Wfp7m79rXgk7bD3QETvWu3HCRw= zy@>b`QTFLY2TgMr)6zi2{8~u*G^@Hv^#0mZddWJ;heHebt-r3|uf1dXYXR?yGelKw z2RgC|9Y!8Rt~N&7vmA*=;oOJ&KUkMB4)&9ZcZkj76C|1WVib>S&QLdVcmKY3 zK8lt#Prk%k*pK|kG{sLv1sk;o)ck2mDOT6?fnwWmt*RQJ;8f~)_77NicxZkM{Hihb5-!HD>Viv6wS(R}kB@1zo-6pM*rFKRGJe0C@AxY5hm zw)1OEz|e?baaoyRCy2!Duq({6yus;?#jY-nb_<;@3(}VX>u$?9vjrjeS;7U@z^4L0 z_0bSBRG^%tUZQ8Ik{d6eeg%fQ^K90|VyOO;dP$x*03ihJjz%@lLUaHos2d!o0S#j8 z^8|ygp=V4QxHZf1 zUc#PFn-7&-t{E@w?1Xqdb`@_G@Vk(?3|cb*52jq2G)k}U4E-38eV+(17UYoc-nG!e zxb~zga51YVJ{`+o#zNlZQwFKtZ>?V`c#4#@sUw2S+$2@K#mixw?Z#2t@X4ZH@ESP zY-e|(Xu@cPWBCY6aO2kUz}Uj1o>5~IkMuL>>VYh`ChlrrH&&H7TXx!bRSypG;fsmN zK?G=5B>o?5I3D$-@yqT++*D5v;R{uCbc+w#R?nS#@Su3ewW|a%H3!oeBWPDCmelm7 zX>{m=idC#2A;pS|F!dv6#2X@={nnAqN+bEf6xoF5%z!qFY1zcaF>!vJOKdZnju2Fn zj*UT1vTze2aD zW;ge8g?-U(EA44l#Es>%pPOXJ4s6%;h$ri(DP*n*a9VFh79KmYHw2H?eTKw{S#P_) zd-txvT}PYk%X)hDWMa^KXc!Od%6;}lv=ljo((xOa)Dduwt_j^KZMo()mq_yIRTG!6J(%Q z0dhh13*X~hP*nX1YDy*QMNVy%;$f%4ST&+Ik4GFw^IkPm%H{oZ+e3r4D7RV5pHVy= z4(pS%!>JFa{m|uySyO}t|DR|a*ju>`NB0(|SY!EP7{i9ZgSGm_{MBkZRxF7@ro5l< z?b*Ga)EI3M?4;aN9ehpQoOt!P%~dZKBQS&IJ*k?0TLmxwBMJq$F`EBFpkT?vQ@oHh zMg{KCpRvO$x6k6(7z!ueNr-V4_kGd_UDZ=51|cxJQh&A4oMu~x?%*Zdj61!R^<1cVa? zEfI7OFjy}$xMNd8Ie`htCnAk{dBnzt4r|>xfx;KaS6q4+LfT^Y=cnJJCR-Pmlu~Sa z`+o2D3j`;Od#AR4@JxlByYxO0;XtLNq?D-`bvh*JQ^$*5^WovmA*l*S>&pnp`#CL- z9=`R$u$}QdJbR&(m!DOo;FBXWikPuAk^?0^?dRgS4Qgncema6RuFNTmiIocbP_&@g zq@o3SHiudvAHb_pp&BB=GCQKjKG13!6&({T!3?p~ALt6>gHr99><^999(2^wR07Vt zx_iWO;y%A5IZ9;VIya`D>JmJZkXT4p3T$na681YB03Ym(Z{$&>wZmHADV4_zAtW1X zme$ssQbvV?s8%GlRBFNQ?^Vy=SL%i_r+y?Gv^QiX$f%UwlVj*p<959<$PpX(MBnhO zTKUy-jr#i=4-_fAERYj9XrLAg99v3j#Y6Q)O;aPlOs=<+9_~4Z?diyv$3ttt&U;jW z;fQ5zUM*Le0tLItWC^O(KoaYhRi&`_vWSvrD&oqdKlhCtDZ`@{3uD;bv+4GfRI6Ox z9~r9g$^QWo(`Vjapfwu(M)R~RQZ~JVoWN74!imdhCxqx*ih2puXdv2D%747clzObn z3X&|ms(b4mxL@sSdda7(kl5 zsGK-#e@X%S@b|Z{vD3Y$ka@NN0?Yx;@E2PE17d`knc9Ya# zT8e_-dUp6Ohgfc2o|yhpOqrcp(d>%teuS|9RMS*%)Ix#!GxES-hUU3r=0RtK}!c0s|e>#dd7{u=?(tjbz{nrhCr)wJJ;#@GxXC>7N zT+0p@8YfCDOZ25gpQp;Fm8_FOO=m?xFVQJWodRWuE;FzzQIFM$fK6z`8M$sCMXm-- z9x*~zTORnNx)bBx#PkARFxyD39d;i^tm`c#WBA9JHfM*vEX}}j{kG9VQWi~1Ys()$ z(nd`u{dL>oP`o-jgr_UYSMUxu*ZC6-i7vJX{m5~A z4p1o?DO%*;KTlTRP&3F7;*wwK3U~P#9Q;@d3a6>6NqL6se0;wkFX2bUqX^ck(Ibf% z@Pkl>Se>XVFd9(Dl}XAof4LG<1DJI}0=wlALglh7o9171PR>a%wxYGzY$ zm^n}TQLO!HF~%c+$Oc_vvxrd&Y^1$J9oCG^5xuhBA$%KKbCU+jAa{T6yBe}aMu6;t zj=RQVe$R2?Q8wj>>Sa zhA8K`MhA_DaqpKq4`WYNLS_Pd{#U?|m>-|(5g^?TKE9+mCvNfUGx-@dv`UA9y(*irdFS*ac4QL|7wAFhRJOFHkUN?gwwaMdMMXfS)0-x;?h|%WI!@5Z9y2Q6o3@}Pirew&C=8YGvM<28S@Fh$6Ga@-O zs1!n5FW>7f8#@+q&5J|Tm)_GNE6ZzK$(&zcVoy2QUB&EYDG~B0^}+94=7SrvoK+5*v%N{^ zEg@$qYw8~9i&;3@5y zw+~n_nC2}!55!a22=$(_+Il+Z%`3N3tnox{&Wvs@Zb4A9jLxq~zkgyPQJuxEZHnab z<*4@xzdZ?EGTKX`^P$WT%xr>HrT)J7lfT|C?nPIcerR{7td356NmW37%|y&1xZfgp zq(wP7Y*LVg+~2RR@xIb*AnO57dH+K27MbVghi4v(h!h&4Wmv+X_mm}u-*sp<3PN>L z2rrAxP8ZJaEW*J>BkCHcqqL)K7gRo&?ddtALdWl4?!x?Y8F%u^{grkVNA)_K+foSv4ghzCg*_b-&2e#1uWk ze<%29QfKpYTlj58#XII~WYAj7^L*o}fk8@lpReP3vNY-hJkESyR#Oz6^e+-TxbWl+ zNSWGiEtFJit+2?qcLxchQdAS(+g}YVu}ONksw3h-bn&90JmAlO9Q9M*JWqy(Bvwn~ z90hZpeCy@qMKs@>creu!~hY45f?tdoT5|EK#$-AV`g zP1zQ!tLdSv78bUxljO(F?`?2aHBYZ8Wh;VmuLXsC8lPlmfB9^>Vzy!D*UvGTroG(; zL9ED8XjP-+)q|YIsbQ-rymCt$=$^?!+p-Js%-XKR95*kFYC-42Y^``0^!mFr!J$ri zeX$jAZcUlfa;2eMucBi4F?#+fp{Rs}#GE*EGB9hF9yA2%cHF(2mBq3--!qS1=C|_a zIV3U^TAtZmZ@;%r3RZaMm!$QmfCHc1T>EU)g*qkvVvq)adC4-lEv&qT{w=T)kG;3v zoUXXAj1GPsVsQQ+nmTN?8rWyu?$7<1xPk&z9T_(NjT6W58xr~7j{|>yGw8|V$ItD~ zKP5FcU;eWvAdUi}>DDFkBS5SC*;rj zKHb=*@~n4gX#_I_owdanr%eV~I4EycF$3>KdyZkM<&}Ato)@!gA<%5}RY%?V0D5jA zG|ki-z(d>4@6L2gn4}(uX~d$?*f1yl1Nqf{*IPlD;OeAY)`F*G6d}gKy1C8<`oCIN zWWm#vjfeu5d|mO`TBt_Z_7MopYJwNkKA?oE@{c($Ub~}j9XCCJ@8}_J$7i`P=)U&9 zD)?<+z@aG{Rq@`@YGdVunpzY=Humlf4xP9J5VaMnWmr9xFX~USs{h;lW`c;$x@V)0|hQBl!Vs z4?b2x!CM~IZh*#hd4>v#>n27e+uLZwJIM3=$Yg}$ZcY<3s$#3Z%hCLQ7R$~$qor|W zzGyvwirez2%LEhj6@n+fI-q(S1)YORZU4X^w_*3o*)@gNN?B?0X%+i~(YzM2xYS`O z7REH|cS4#Z)rxH9*B1GwrT(m=;s0s{tpp|ntPfr0+^bxl-s;V+h~g zg2BFOQyKJEpxB@I0^Xg?T<1uNY!46}Z7+6-(*)}2x+QiQv_cHwXoGs%k<~Zvc74BV z&o@djQ+dzBjY2m;odsEBKL&2E>n6GU7j?ng+(i}%*&6E6`Q=7%b;yKm`JRMCdPuV? zq5nG~g#Bczpm{a)VB!IR$8%THW!r`C@A0NwMx~hGu+80xncVZn*ru`I@`^|A$Lahi z9k&ijf)n6Jci#^kdPh?LfBrOfjrj6zAG~E_mrJ?0nxQd89=fw9f)CB6* zo1Z@IdyQjopwSSrh)s|5i&R8&?(5<+sx1_?z*M<0w%>)a*TOZrg9Ha;>raaB9e; zTIzVX%uPJft=12+o$KR8q}tkr2Xiekf(a_Yb;)%<`RyV(%@4P}{aK-2gwob!*9dSu zix8i8_R{xsANvAArmdm$*d~CY3D7_4io!N&;^@6Ds6X}^c1-^JQRP2>GxYnwz`ZhK zxH!&AU=SIBCj_?ed({)~|2*06U`vYqtX?9>9iZ0gzkk;|%!-MQhQq2RfBp>EENc}Y zaD**$-!BiqOGZ6uSKv64dAmaZ#sJ8~dA)^41vUTU;c;v{dFSrkyPBGso(n&xrb3wQ zxY@59xA6pYwZFf8>Uf+1T|{K;M8BJRYAKGZrJS%8?;nTAlbnBe5!dIBgU=t_;% zX--{M!p#MSH9$ox0YVu-wr4L~-^t#05eB%_%I0PTptgD!u+4@8;-D)5PR^!v|LvLc zx22+ICMO~4rX(V22RtTl$q=)s^R90l)({F4)63E(6p;`Ub2@Ik;O8&vCUx3b83RD` z_A(k1$4zOCMqfc+665vtS6iaG#HcnHApN=r%>hfDI^Fh>9ova++2#?)bf=fEfc z$Xf>oha@)?^a@79f7@s`BtV3k!&tFkKEP15wJi&~uvitDjgUjWN&B07u|Y{$8Nen7 zySg|Y+CnUyR|OLTz)$VSg8Y2=A`&pifSZn_9wIz{K8Dk<6HZlQU+G>tRk2Os3DEV{ z@Vg1lyK8dE5{>;qosIT6KuS{v6~+j1h?xLjch7pJ9j<8LB5Fzf9bj+(!h?&@>dL*p zbc;bLYZyRH0E5&w(s}`KV_7$UDjSu*4-~NTAhX6REprbV)>V%yq zVD10}`U5`V1rW0GC)iFe6y|6j2I))sDL+>(_z}$jV8;gX^snHFJT|ogoE6|sEo^LD z{B4pDxW11QI$i`7LsvS0>bZ6upe6n~U~>V7dU$XEc;BS1*$;~V{#u5l1)$@Nd(wJ9 zTmb)wPi?!1zs4L?7>zq6`4F$wTB_+o)-W~z86{0>Cr&7VGf0yy2??k)(K z3FPpTH2{8@4=_1^uFZ6%w84xxySTtw?`_dtEFQ#Cxlsfmef0kYIulqIYWh!}Vc4+hn@Ge^qoim}kfLL=A&6tH~&NS>djCA0876R zfa72ymq*Hq0YF;_e1gLz>wtT6Cqe^p5EiVrj5Y^OMYtoN{Nng*(f}k15KhUCTaCkY zH8nLr)S&^mUr=Nq2G5~NuzU~;#xz6Q&`{+7z)`8}_A6RXp43m&`x`XW!Gi(z8!Qfo zp7GUhD`-LJ&VuX0aykJ38*PsNm`bzU-``(0Rlstx!OOhq4ggsl7BT-QC z0)QM)RFv5*llNg!a`2!)yYW1_+;%ZCJ3Cwb2s4p6W$6HO1!#5G{l#KfG;XtDdw`R& zA}fgz!ysaF-KhYW1*pDBZ^l6QU0rvgWr@?UHKf+N02f&J(CY&H0j&4RfWmI+i93Qr z?e{pL>8-IQE+u6e^1W!yH8^Mzt_*lkh7rTDg*(9Q z!K(gsrL_$L?6Bf@4ROMBILjcU+3_EBtVsKmI&^E`+rsEO^hHQ0GrdjV_@P7`(K`T6-h5Y!=J z;Bo+n15|{&`ybSg)=pfjDvvjgH>mL)r z#8BL`)C28`J?j_DHlK|VcrBEIj(V!l$6}cbA%Xg(%a=WfKrPVA#JkUGzV`L?4GRlf zeS`wrU6;!R8@Kj+_yz-w0#6e0 zM6n<@_f|}ln5MHMwdbwYRdp}O#y!sHoA2%~I0~Z0S(w8dt z1eZDx3nBn5+Vqir;@7Wac?MuK82I$yE%Y;%m6h%9?kZ$!j@au~gFgsr3kVSE&Vv@g zYB%~Ux>!D&CRkQjIM^WY1J~)CwVrN^lddRgZ*K?aJSlj;fPe>K2F|FxD!L5khO3yf z&piq0hvds8ZW{yb3kaIzwy6ZHY#h)VEEYpZBtrwE044U)#zxR$+@0uKb8|ZEJcxP$ z#f`O*eMq;(*;)GFM$i@>M6h>eQ35f;m~H&LygWEAnZwqH5Zp&#(gD~s497g$cvy1H zpW6ND)2A=3ti(snU>}q>XaK+)NDqLmMZH8eSEm-}S418^hN(f8*|yZwOf1%)I7iLI z%#0otfh5t|3C^oSJS-u5U5%roLLy!{U%&DK*9LGFl*^>bL&*1m340O*j*Jg7j|XX^u@Yiz{qjWe&r{K zIAFW5{5K$^lZWl#Qz<%vK?Nt&JQ8|nX5rM1RlMZD;^JaM(HN;PXGIuyYph_@$B*(L zMIHzX;J9*gbBmF(o2c^zgPhVY0}l-(BN9qZKotT+4x>w4F$iGs09v|Y`o7>fR=5XS_3w(0_gv9DD&}Y-e2T7 zycX6x1z4w^4S>n$Z60ee@K88zL4mj+YI1Ht)(M^LYT~gT-Zm z{lCzEcS2zSq8`+t@@|_X*lRk;NNey~CS3%fh6t5KHmz5wA}L)FmD6C^L0t@$*_ne{ z;+Rv-JOGaX*d4IiAE?VDGofpXmX@|fh!$jb5aot?oB+_wV^tyueB8i?g1`lOB3RLA zAZCIc3%iRH@sSmI1l$y$`ov?+y5v9Ie+>~CIl-9;|AjcWqg}M{#C~_26x3?Xx%G|l}rxRa|US^3Kx){~(IpSb;Vb~ZxT?MN8T(%#74fVdWTd?YNQL>WS` z8<&ac2Qe)$mi-^n-aDS_J$xT;qOC}xNXZ_N>^QxTWGh7WCMzU+RaA(SnU%^)NOpEq zW*G_D*?aH(yWXA8@AudD@%^pyIFECr_jtXYujli=ulu_1`?@70yfc<>jQx~rPPP;k z6WjV8aS?Zd_5bkUgLc6^7^m+Y0>6kM!wnY7>50Bse#5&X5PkD2k*u(I2-ylKUdc*H zeR5fRpO?4Xk@OTTU=>yR;GTQhq$N-mQW92eMAS9HbFj3WK$h2Gy4%Xy8WnO+7uG+o z?>^a_KeBtUI=Hf@#}mM;TDMfs%8E0*)^lw%X+SozAW!1jH7H7CL)ZZh@;%_d<9qcl z3jQxi6>KFjTqM^18NK)xB25Ta6=O>$r*}e3ADTaa2@n(m27=dUL1?q!@CbScmR$uo z*WVB-X8Ex4;*L{VP><@6<~vS|i=CWSu+<6w{!7%P0z2)}{-dJGZdJ*7h96nz7ExbB{ic}E!-qteTo)|N&7J=IIeD6@MKmnxK}1VBI@!6o z3f=VUaAXzSKtJD}o(NE!bhNR#4ZR=KS!4a4xcM#9J>Uqr|MAL+idWkG`p zqBT%=k&k7*i~xdYq5=2-q%+_-@GM60yN=7#D8HH$VZ{ESBH;B!n^kepfpGzf1a`!! z*2FnhP8QGC_1tV;(PQ}=&?B;D^8Sre4>SvJdKv%62|=BF7`xgX2-Hk(+{epvWD2}A8F`?n#xpLR+8-*KcQsH&Xvw=c;I)m>;kT$f0-jAN1-u3GZ_2Szo zL>IOn>gx74c;P&std!nvrD3H1(iNrx`VaI$RK;R)Qt zQ875ci0c3ZvG-HpU?LtcEG)tdc|#J!P7ijILezzxf};P=pFjQmO%Rl8I`afO17bq& zAV-y$68q-On_wn+8Xg0Q7iX_}lY{X34M7*UGl&eK5FTNmKexd3lx8eyq5XK63L7Vk zB;aB=hV84 zRWot22To*er{FkgYG^D?v^D)go)-!OlUdvtT z9~c;WL7xC{j2pP4WebrXAgEYKID!zu)`RwvX1vr3P-t7NL1IgvZQ#(cYZkCyu6U7P z<3f3K^zg<)9(&sbrucIG-{(tjJrz6C|&>dNS)obu(k^jZ#+yLKk?(A7$ge;F% za0!q`s#)7KVcR!LY^E2I#{Ngd49+goqZJ};@Ey=UtXk7HrTd@!o|DY~-+IsgZH90l zxxQt55xfdoHjnvAaD}@%J0tE!o&F@OIY_4M*-HAR{HW~$x4RZCsmXvb2rlF}@qlrz zHl8#58?Tg1-iLxxkY@Be2Ht3v^|MDhz!NMA(0ra%M@#1tusL>SEzWoGH7lWtYffk) zTLjky)T#{oV_1t=3goD;`7ajaKHa|IP+%p9YzI`AhWJZxPVsi=Ta8}cU+a93>?w;aNzXedKGz4eQ3?Tm7>3S zTU{HigUV{fF7{Am1rAasd!w2Hk^4rN@jTCy`~IpwB36W?q(zQ~on{KCM-ZXB?{gK~ zui3u_w~0UDNX9jddXu9{7nB$Li066{6gX-FXfEwPcu>yR0%mhzcDDL?`Q7{f(JQ$) zLogz1Q-#y8lsSmY{ky((h|5;Jfbbod!maW2Qlfq>2gd|frcJwXnB~lw*AV3eV}~RC zu(Z?@>SSoN6mC6q-~hx?{m9GpE}Rmy`SS(p+xHUJoSt3sj1w2w?p1nDc*CNZn>uUl ziItlr%;5>?7C9jg3h3T!ZVw=d@Eanu;56evkr!bp;Q|52WZhR0Q)9PP>Rmug(2oO- zi@#@UD_#5&8wUIoW=3U#A!32f8eC~l{%=GY!j4loS*Ix}RZ85P8NMR;09&q+q;MKA z3#NsLeM>!ioObr(C)lH=+kAKPi-_7~4IBBZPKUDt1M6Yu;B;=nEQ6jA77@Wi<@&~p zc2uOjiFng>2_7TW%gYO0X*`CKK!lesU#1uLC<0_A>N!HC(PNBHaio!8myu6us955ARz5V?_EOh;)oy5}s zGog{z;*>ggcNGh}y)myCqcA)%(V1a*4Xz0MGHYnfdD$>TS>Gxv>yf>Hv#g-lgj`mH z9Ms>|ckSKp{rE9h-EUvMoa5s90Kv}W@UQeEa&sA{i3dz|>kIW`utxZ<5H9UC-Er9j zpiw}(QDU~`HA;N+!(-W>3Vap*Z0NPL1r9`Fe3|`B@+&M96cKdyy6x>hC}ILW)i0oeh+h8N3nc zVVmNKB(#nrchpp1Z_%_j5PJpZ4dZwgKw+lp6tBS=iVe^%`no`i68n#blN$UV_-t*a zV8Hatf|v~WoB_IpsLh9u7vKb~8wH=}fT@=QRWAV8;D@QUMQVt?(^P|edrO`7^cjY) zc!~coX_Wy4B@s@X61qD8`X6PB4j>5)*O7F(7C_Tn>bVUnyBO;JOiU(#L?E$fE}F;z zG=S@TH57*9j7yRYWiz(2(uQa%Tm)?gMz#IOs8fMJol$iVo=_3oGtLfqc~_Sf}lFOrvVPriXuEhE~Y}iDrl%)rvEa&I3HeMuDfYM-1|<6X263 zoe`L0oER`j8rC4gmi6%?#xyD#uL5J5<+L9=FGI$FD8SVK66P%kX3;hlWf#<6aQnpxhIeghKG> z^mJ?CQcfX%nFLGkgP7eyLW~WT9LzfmwPD5AdZ-T~+ii$#-0+tX2 zC7^LE@nomM5L3IJF5+ZkGY1_FB7v1P8QuYFK=8q*M+3KEGs9lOFgOceHG!NZagpJg zKN{CkIXW&Qe1Tb=0>lBO&DZ@6IVfvXi$IiB)HxHiuH}FXRS=tyIcyI6H7w4`)Vtj0I4eP+Ik( z;J@zL+oSfTSJE!`I$J326TbSQPH5`Ql(3AhU;VgPVICe3jv(-jK-yMF0%JXYK2vw7 z;L|5+dWcb_z}P}CD+qGiu&pkjo3OJhJ}^~GQB;Y-Z{it)3TnB{hwo}Ha?T14R-V(% zuh9JkegXau4uwFLpfox#uuzxAibR6~(s6wK@o34y)#I>bWJqtn88~u}>3Sc42R&rC1O->HV6a7yM{7Ww`s5HTQ+O>9qoGz; zMa2-d6!I&h$X(j5!a)KJ(R#jy(eE1{cZI#c8bh}0Wv-dEbr0xooH#@+Fo09xx*j}B zQ=6S`Dx?}@`V;v^!cxu5-LY!Cntwu3U>>zm=8JC$qR}irIb0Q4{GlHZt7T{t)ESM=N zk8|bS-CH0+5(h8Q1lZmg^Hd4i<_HC43j2bbE{Nv1Kp6)ix0R27b!VXv22Lw+3DV*H zgVp#i$YV4aBGpvl9>oK-ythbU0|9{I2GPr^U9bQQ0otSymh#{>rPLHw1&6c3v=z*b zO6sjyz$Ao~Aiop8<8*)(g0QIzj54&{;h6W5kVF=aO^l4(HZv1K`gM@|NL`*w(kit z62s0ukQXOv>?6M4LrThg{yfo`pE+{|B35s%W6KL)xNtp9b$oFR)|8Q6^%ndZ91fWM z{S3r`T15I{K7-j>b}MV3zkeP{DlBu+N(6 zybYp%XpY)?ot!>F%qG_=DxS}9i1>rwxi3K1lD+9IR^}k_kVaHD|on^yu5A8*h%eloy(61 z89KB@_xRBxgbl5btASGWJ#{Iv-jR{tz?k>%?*J&pY_K8Y{yfhx zZ*p))@X)YQHVzJT-|L@j3&2D+5laM+5+bHluu}p0p-7$UJa59y&5gGY4e?vFa3B$i z*tOrd3!6P%Jqu+fAgq$+{Num|1=3ssoCvSQJ6#t>^er*Z!U?6Ch;=Tm&R;=}z;UvD z5kl6mv%xY6K;a-l1P-6T?3+nMbBiv;qmBP!hsQv4I5@Zoi7Q;s7#3}HeZ8`xLIG0T zNQVQhUlKDXrXdh`i^F25u%+Q~3a1~YhP#}HyA-zH2&mKFD!u5!&)ah5g%omuh`msk z6trPEH@RH?hvM6V3S2(Wqb9KN1lOL3DHP(2!`6eM%_<}}I_qKWDJIwwn;-c0z#l{% zxDgibPY#nB_QjTXC)NuO!*$%F($0 zbTR()|E*a2ziC8Mb=mzTs@U$8qUG)DDZDw618*JVJjD_st^|h|V>kwpqd*P*;Uh=t zGcR`oc>_a(kolf@xvx|MlTl!_dCmG|66U~UqYetrNdh|<*7Sw62*Cx`(-vb^BC=<3 zim?GwC47&;tq`A)YRZrJ)9$Yn9%;=-Hf?N;Ct26O&&X``i97JKv%K>_)YI?Q+PCl2 zF0-FmX*LWWUE~%$BGdh@d7S10J?}=<#&QQ!cVZK*u)L^{i}at4tcvuawZ#T;ytggt zsK@TZVI!aVTjB(!)&5PObETU46#Uk9Z7u4D-&()FA1L!^i*9gG-S1S7L-DlzZtX-yKgcob4pXO6Jm2$qb zk7P|!6X4RY^Q=^-*Qt-GVjlFEX7qk8Yg`)@&_?HGP#GgG?v_PP;P_3qF&AL&w!DA3 z_|YSghcOeBB8k7)qg%JWj3;+mOwWh!Blzg|goIYdm+sNpp0VmX7xL`0-i4xW;kFV< z+NYus_P(=SMt56V;tcEWsFr?;O<3RbKMm~E-CcrXqF&U)Bp)ep^(rD;uBXdSEtf%T z;Ot;P7B~DX3Iv9LT9GhCMkBM3%cuw^`~;^4<}fDXHmxy16ooDzD`=qt-(^2YesN19 zwzeYKu#E)Q>y)Ep+R(bV#5p;&*}K?Dw%xIiljP-ET9d}Sk*}53fn2%fU6HNT&itIwU~9*aaYgsXIWHT!MQ-#Hk1i z4?KkYg*4r6F5MDQ)S3Yt0V%nGm_Q}`?>kmjlK@SqtK``H9Gel&t$Ub;ud1>#2|x$+ zHDm`4P!KwcoSQm2*5U0Ezk!=T34-~C#@QD{A|2S8I3NkI8ZoroQ$RO+RLfx(gU^Do zvCVQ1m@+;GP6B`b?{gn46o4TCiq}Gn(;-#2O6&=Z5_PIB$-}ldt0yB z;n-|_-g5c;>-LoOfme+m=Q8iOPRB@>%mnz9YFeMvcG5}NWf$pQXjh)~n~dRWHM1Hl z2&eZ*$@+=FhyZW$^kZ@BQ?WYcH=c-Iq+R72&*htm^_*)hpDvV3_vGBz<{r4I)jpU+ zAsam1*P3NfDn8RO|2v$kX?&aRlQLOjYo4)@!9wmIiij2M717k+j`d%f*T&z!s;Ks1 zF%C>Nm2qglk+9tDtZB`xJ7;hC^W&p%^TiI`_w#kChR<|AZOV|nX+F6>uft(mePT_V zj_OX{Wcy~TaIt<}*lTVQI*vB8YXK$eo;tRdZYZQ~%6)Jl zf4MMHF#O4Vuc=O0n^39N#Tr5Ot@00Q;~v@RExg++|I%%GD7w&4=Dtq4K&_-ZzW7Wx zE|A0S&Pws{%D>NamwY-d(`*H~Hz) zoZg?1^#Sga!5|dO>cZzv0(XNa5tuAS>KLI_qMf`E9Z~Dgb(u)Q0vKVFxWm{YktKJd z;S>_4NEHLkfQ4!X9f4znP=bO0a_KFI5_EL{vsV~`6EHQT1dn3`7mP350#k+avW03q zq~xVD+Hh21^I0hOE6WngIaVp?Z^O{ueBYpQX%O885!~9 z1A8Os&CKAflMVuK9V>xJ0vXmlVi=l$DyNg4Bw#Zah%6XH6<{!tQun}Ffx%F+yvV@N zi#GtZCWfJ)fQ8Zzmti`>yd@XocK2pl(=TqVZ-OVILK$y}9GZ%1av{-(BZ?Rt9 zALpO-s4~3O zCz99aWnOwd65X!tGFT~>U5@>hx{FqmmzHH(#7=5iwng`Ja;1o=KM^$ohLh5IPW=M&B}bFtvXp+dau41PU|w?d`eGAx*NyLx6~h3 zOdPkZL;VWt=#}KjG8|nt)pw`R&~CB~2sv~5XXQpSPZr*IPx_$cTgrLsMrKy+j)05L zOh2Z#_YIEqlH@48{rF6*;Ah3KY|`URO!mq@kZ=EzoImgBSnu(avn4|%&j5< z6Q*j#&|D}l*8AD?THs0e?^biu&hVQ@dj!^p%*yFa?4P`wAl4(ob(h+yVj7pcn$g7u z03j3vfM^a9P{oj}jt&g9z`Q&VamckJ4-6!UqQk|J#xK-tFk_ccO%75>!2FjKu|2C~ zYt6#c)RaL2@d2@DexmIn_zIBP@TA!=iv%(N35488Grq?kE6yJ{&c^sc3OWwYfw&HC z-=@#!5Njb8eFDBBR`ol<&E@?PCEdVu{_n$@=NkeSrj}bWTh-QB8v-ZzY)$U+^Ar&S-+ke}P z+51+zQFi@w%yReTuH1WFdky!;}hoT{Eq0 z6PK}!jE|XNny497mL~X^zcSwSfxfCLaPw$YpW&u3|ERRnM#Hf?71c8=gSuC^nU_*; z#rYSK8xR8hOs%)(qc$df_`JSm&F{aQt|}Z(-Zy+_u)f=$OmJm6yQGBca2Rj$*Pn#3 zK0Q|N+czpx6r2P5_MUn~zIXojr4KK*f8{uKGPp4x$p2M`(xTqm7W$@}^{esL`|{vZ z#195=_KiM&K`*9hyeU2wc)k4r5Dcr`J;iQ=-YH|6O^zH}O2P<*p%HJs@WKb}L8ioS z9K|*J8GhAIQ}6yF()zQ=h3y{AxKfUsh5CsUDo2MD2G%PEX6)Y2MG6iuWb<%~=Y778 z$!-i^JH%f{EXSRsr5#7a_m*e%9An^oY!A}hOa;#>joyx0Kb6Xz?d>r zI}#)3lUwRyoa%0`kuyqHO{ooKH;u-(fJRWL_enG7%$|`2yfnLEXF?d zQA*B6a)QXf5otv!#vtJQJQQBw^nq&xJ#%u?Ozgb+cE`GLIqeY3EVaXn`4*>I1947b!iv65|-&5+l8w1^0%-doEvhB{7Qq~~4V zei6@x=2J&RTC2X^PSaryFYz2+50s~(AFsNNsS|lmb8P2&=Wj81D;_=3A1`s%&F|A5 z@n=*7El=`<=dvkbHOA5BLuJ^)Z3^cZ*CwrmJ2a<6kI(f7efD8VkW5lkF%!4Z^C}Du zCGRuNW@Cmz_V%^SqcZE2gmk}fjctol#lQN}OwXS@xW<@pfjVAw+jFP1xiPzUL&`BF zP1%#&-cCFz!56ypE{ChgnzZIh_twrIzv$(m-)m(R^R^KJ ztgG>5C>VIB!iGMD0b-{&IwLwY2xK`LX~@waX7t?N8knr6c!&x@p670lGv{vMsSz8^ zEB&guk3{`61iykcm&I7W$OP4tfFsqLm}mx*hT)J?2y@$!7{tgeq>=BgmU;|>mHhf` zb6nT)zfn?kg~fUy-#|sqE{TuvwRLQ7bG!AAf%Av5S(<7%hx$)G7{j5OP+79x$;o2a zOB7xypzIa7W~3-k=6s=BT_)qlK&3`{D?|ZbzBM@<{H?UOKj_#g4lDWK*)a+B{IjdO za;aLjHF;-EzV7a4*?TTWfr4&Z_#4S37prm3ID40_Z0_|F_CL78YlJ6PR(C(;Q>9K) zxR+aVpLyO*lz7MI&ArK&__sG61*n-lA`kQ1KBAYqEz%mO`0HKHno^9zOB1uj$65;0 zP7Q-!Z>l-QIJBri^?`H1{17emq^oBBS_U-WOXcYFP+ z;krCLt|V*0@M7%dHgfVS_I@8<+~tIolbCvyb$r!tJB1^>XxoyJhyO>wfL^udKNr`p zHMO^EB&(G41Yx1BdP{-CLrFP`d|Te%WOsI*YucNjwCcf}DfDmHX$x>$D0ufO)1&b) z4>3?8dm*}qhB+;DYt=RQ`SU+xV{G2&9R%Auhoa{oZnL5+BT(`{W_a|dr~}gh{1<@| zM4&>Ml6aJtp+XB;f`o*Gj6p=s;9-8;uV2lm$=p5J*LQZS^#sc!%1YsH4-{T?1n72P zqjYqzQ~K>D-D*zw8Zu6tiDp`NDn{W1HMJl<8O7nz;+|rw4N!KWMV5FVP8kg!iH*1Y zdjd|evU|FAXtq}{oqx$>#gW<PLxqJt|kW#1gqEECN zW3vyQWu**no2qCU8TeW#B7*Xhf#c@ zi4(}{+(DlX5@Z1a^!}hWVBX<~07mX2-9)n`Ks5)p5D22WP<&8SeW|0Pt3~K+oA$%E zA6759Ov0e|8_lFta#L(M|GJq_be`_&BdWV~ngAKI;>i+Y`olp6UKI!bSZ>yH56HeL zQ#33OxK`KwQ2Wd^v2eD`+^`W~Y}eASz9v}euey(++gacoX;tR`9Dm@{0iH9*6&793cduO>8j+bC zzdl$z|3cFvgH7X`U{Kk_RNCZeb4QwHL0>J5T+^_e(z6q;enEcjnpSYAetI$U7@GW&-PX=fNb%E_^hw6RU9af?7l` zMFoC=PB1`BT*+it;WAQP6?8M8pe!%|c$SgaH(}MD9f3h~xM|g&7Z*il zc&Z-mMhSBd{t9q{SnY?mBSb|0-UNweZ*suz%jl*#QaV)@|_g-gU37iD>HBE+AN z6XCP`-GHVHVn3uz=vwa+deoq=fP9A)BzD1}C5GUGhyYE2j$n#h))vfcY-~^%gp@7B zq1e5tQ50>vkxl*S`n{Y|A5>AF?74JCHGgzH@rv}2IInnkK(MXq@_zYad^KC7CY>&;Grds_5uR4>O5P5KehiT= z@h{9v_-@b^s*TO6D0=u5-hE;(`{wDziJJ!5NnV9dbG{6{j?*aSVq*39)%a+7!eHRG z@rk4K_FHY7{f6*9Ct`c}yV9#(Wd6BTD6B!X@bN&=x*}Uucx@tqr?tniOUUlr*%j%9 z-^OZ`bDj!g=3%wk?_W1H>^;FvQtGnBt3TbABri#)?znQr?-1L-Po=#v4$&@id}n*_ zNXez^)~~32r#PMNv2a!)I)+{;v^tPVHi-ZuX69~v&p)zY>fxoq|8DPMSY%1@5gye5PCJ05GwT|UNMsZZ4w zS3JxjE%I|tkB{^SnO80(iF=;RaQSresP6JBJ2^WFFNL;{*%Q|GVgEd4M-e5;D)-B* zJDa0>gND~g%E)MdIDUNE{4`RXa~}26DsKpn;DgZiW7hZ87x1FlJO{007h{Oi4f8*J za(f&oRpWBZ*9cv`NZ}JJYv3B;=`xWl&)52Jodvtb_|(@0M8!JM&G#GJ9%#TE z>z*x?>VTHu7yEs0;&%or0#W8Lx4hhXn-A$N{>y5x4NFs9A}D79Fh{?uHvS%M<1F4F zb`a2kejI5{?*SW_kBb1gi5F{S%ry_fYb8mlixSkTUbyZ7r@o`AK@g3#&cdjIfa`Fdi zdCo&Om!Gv)X=IY*QHsfyz_};uZj?Yh2tt)7zX0^ zEHj?9TPx#VGqh_(+F8v#OShlyB)z2JZn6E)#l0peA&hfkLap0yteuuZRp(M}bOGIz zF3%BDE1|%z&xgL;J@j}um`440`cR9$TUf}#wu&bb_gP-U^`y%FHvcfSVWtX53=gzs z zD&>-fy5pa+e6JMun~TRiMKgCm#6eo(ii%-TiNmUb(UVzEy6CpgE77V0jH6DJz7l=y z4wjwyX^}J6B6Jmbg(t^Pm$tW>U(=}KB^K`a`bux@YE8BM^LZzz5X$~HJ9muo6-g$| zth8!Ae3nprJM2J!YtG2r(yT|Z)B$>~z!dv*lW(Th`;-!_$?mHD96MP0p|tVC9fqHT zXCWU;iA;9S8XKnL7GqYspnq9=sOs0JFpi>(5vS*p#>dG z2qHg$Z@UHX#RK}o1x0NyZ(KXEbvksZwkRt|HGMqCRAxL#w`lV{X;Cc0q)egkEd?!m z&cXxIW=lhw(lndibjLH1^Ygp+-zbWi{F(^sYecE6=M4YL3gT40SF#sz#8Z=Z)87F^ zo-XDhU27`O*pjn(Elz>NAZg{=#aRZ@wf?WlLBu-gmskIv$U9b}nHhrl#5d5h(xpIT z>UkMT<#me<@+Us*s-ph?Fesxst8mAF4T%?u<=eEsmEJ}Nh(<6aMY`L`1m9;(U)eAD zvS|ltgvU3YKR+78SHKt`CY^Tk{AjZVew*I7pTS$@tnw|Xw`R~XxZ|q1nf{yW2*4+{Qg?LAzqvkhgmTAE-_Q3?p@xZefy7=ZNG4eY zr3cAD5TQ*!%K(To1DZ~ zC+_&MHu_<=U+zcg<~{Qp82XtN}ezB^YdJkqzyjP7fK+`Ww197xRP|j zr5I|1BZo+eyI;2+@Velk-*ECENpS;W1;SY#$XlbD=hv}hiL;?H@k$Iwk0q-hUBmRE z%IL1_qhCr46C8(f!}#yiEFOX@|JeBq)KlaEgEg>$3pNo}M z;k>W%$-WR_*@z0l`;WOQs94d^R9DxmHOf4);)Cx|_r_|oh%qs-@-p=fj{lyKa<_cT z$5Z`U$s$y`fH6lQq2;{UJ##Cj$#y;RW4;Y+!3TwlnQnwqZ)x1IV$D&Zwxo2SeUw)C z)O|{eMu0i_$#+yv1rrr~h&m|))_6ClMSha- zEe=s#?YJ!ToNV{g+EzMgq2UW3f zH_lx6LlM_He#pgL?zY52(*^q#i_KbU`+L!B!V0nRM;|Mm;f#l1NOQ)%42IsZmwWP4 zyhQV|{@v=V+dEgS9pg=Ig!O+uNp^6>2-`bGHA(gTJsD%A5%#Wro53cZmEog(=ce34 zn3{f*G}p4oGP@Q!g{n$j;?I@78-X3Ke5=Zc*8nXbRg!y1+|V28r4br0uTt#1ASyQS zh?j$xRg`VjPS$Qm8ZRpkhf;duJ3|U9QM)$EnUA)PFErKJDE%h5@Zj7^a0S0{*3wi{ zpoyC?yEY=x+<&4mZ>!}i6OKhzVfMj9Sta)1Ut=p0H&wKh3N*kGPvPj2V2>SmIc$d?Lo4hh_1*+neVyBP}Ex0A*VqlaS4y_7HBzhd2RKwit>dyishW5%552ex_vuGRV9!*?%9F&1^=bBs>n za)eRQl+0-SUb1zd3wSz=g0DSBnPwD zvI--$e|%sPx*SbHdaCZ=<6V6B&;0XK_sl<@UT4V27hW?L?5#`&%HU)M#nCXNO!7_l9ZyzBW%;`T@`XT@K&<{dNp|Iyo#9xhokTk))ILe14( zQ#$>JbKr=5Z%mfbr8sF@&-#ls++6W#8%3l<(Xjh@^m|BVXlA=xZ(i#BZJ*~bxYZCM zzLfKbc}35dg1{o+$T@em0trcS!s{TSu+7L_$dx=z)py5Pf zokkb_;7@~J-cQXo18aTf(f{)cPT~s+ZpW*To-4Y1aNW@Pg6MQ_QQ|}UA2tS|3I$d@ zQ?imr{}XJEK2LV=&xTook$R}elff=4ds8(2Cmnp){K)V#*`xbK{DT#*XcX0gs?s?%y~+Q(jUr2O>d(&a z69)%c5=#mpNS1VPZf6VCGqWYHyX}24svS=_N@ZJ4%;^kAKl$$&9Nt4>^QuPLc4acg z##A@J@6U#1(jmb_@?CS?`~{uuc`dwS+#WM^&PRwUn>(y&u}p4!naNL+#ZIl?FlI_n zCb^;^$lz}x68w;L+~$5(*@m8}KuuOrm)JC*C%nPr&IdcAk2LeS2r|U^g#>V&-@jFK zZ6R^4JL>(ux+ZDEBTvb9{m;cq3>1;>@bD*)*$c&3PSpHx7YVI$eP=Ir$=~En*;tvA zi)o4=^&Y`+;>4^S+(%L`+@HTeHTY+Ws=!sRW~``kj(?`E+Uv#g112LX>I;ug6UB~q zIPE)g;l7%@jCufbuK2#-nvk4}OSpirk4les%3K@lZBgi~5h4kR|_?XcdB8@#EU zqq4b=R$9r7{kRHu5oH-EduZM>d73-{^Jr^Fk&jixSKdECdgc#!F2pC>gw!nW&$B`V(?P<^t17hL9s!R#ktDV}*uQ0KcTV-G)9{#kq>T z!_mAltPFv=;@&fQM=OL3<>PWPE1m>%Qr>ENO|IgpHF-pe&h#VC-)S-2B*gC@%QSa5 zJl6H^WiHD&4?Q|LGHHpBBs0@HALa|EOIBnRJ?w**);|%tU8i52re|;z7@aE+LTZ6_Av&+ zLpvQdO&MjCf*|>(hJv}Dv?n?-NIcN}IMri( z_S#d2er+{KA1^U&5%c1C>#Zacm+vrKy5V)g(=5kX>RMde>CI`6(2HSyf&qOI?}95+ z<_+w1Qn{bsk(}S7f2wG?zvfw@ON#Zdzx7_%wK; zcX6IiW6U>HD5K;vReOAlzRbgnJ4zj8UoDUPQi!F0CNykI*&2{aV-{`KeRjl^K`Wo22AbhuZ-18NGnPj)NRkt%891m%P2BQ_Q<$Pu90b!!3VKdG^Q(?m`nf*yeg7SQo^SCnCX))x z+GFQiTUTWQ9e9JjF$UOv$xJ9O zXS^h*lN*YC`Oig}a~1P^KQdG8ZH2a7rxV0oPw3xU(3&phee;v$jQ3IBy`x6&f=_Ep zX|~I(EU*MYlb&Qn_&;-bG7!sls}hEC=W=^ zc{|u;^K*yzCI*>gtXo`kRjW{^FSp(G4_Vo-L&B#L7Bg=|>dBqlkR}Q@er7G>+f}r` zPAFa8h=(mFQngHM+r;_hW~-2Z*cJBMa%5*;G9Im~3VzUAwl{Xe}XXq#%4t4Oks5}Tb8PKhs)TMRAk6e^hx~*zjVI+%jG`nMU zj<7TuFw(wXy8p@U{dMba`3t}3o%Y>pm2|Hp;N(>@{kq#C@?u5)(d@?O@M4aiyPXMcI|J8O(}utQ{t-A?E9oHA;04|f9DImTl* za`|H2@0~BmkaQIyu(0$eCiPG;45vogcp?6E5 z?N9v~sp!{VdOgfIt&GuvPd6&b_aCyh;x`*E%ExcUh5l+^W%f%G>X(i*3Rhcg-)Q`o zVbAzNSMjW_+l^;T8WvUu2@mwjg+N-9Qh4Y;$W{7|fk}ys&f8~y^o?%a*E>a5doViF z$Cg22X)6JD%qXs*Idju+`C1cdr|xA)x1UzK6~!)XqLT9t8ya}fl!jlKVy7;Bk-o4bC{XY(S%rq7 zmHou3?blCwqS6L(&-PP4w7=3Z)R8v#v@jq<#$ucz3?4rrm$wW{GskU@rJsDm@cK#N z<2}aGl(uDYDeQr}_7O}9bv+&Ox-WC2WWIY}F`|COj$OKVnQZUuWZ0*V{|KLyK6}cx zq*K1-gjHPB&?oVLwj*uL0iDxLFINtvRJxrdKAg|WMCW%!TK)9lMH}bI6vocBMWHU6RgF*sJ6_*&{si&d+^@NeDZ9FO^qAbk-&?6{#VZ$y!hPb+if>i z*S*m3qu_mlQq9rNnil3TjWg6Ul8|%w8?qYqv zBV$?oo)72WOFx4BRTu48CmT5cN- zWI6n+@4t_sD6~=3d4aOuEv|UEqP^0B(WJ-!z%Rjc4kXxTNx>GG~TEOFJLk|8KzXp8@QJi%E$$-`#v(Q!^A%{9N&f zq5pNN^XidIF%IMO8{X+Z`h5y}e;wZM?3o@JmXk`GQd-E&d}hG?-MzS&gq0gtK3*#m z8ZPpDFG^#0%I?M0eJ2R1-y8E=yqNbCit~!>HjP(^E^oIP6k@XnslD7AyZP_iN&9^xB4E#%0clmg*k6wt9B#ZM9 zr>8$p#Y?5{uLN7k>||I6{ViM{3P1mI+1!6D(4FkR$i`+rY(hk^SH`QbGQkC-LFGA#Nb$J{1%EB*V=&PS5twxefT)O7@>DE=-T-QT0~DkRtZl{k-h zzNqv(hpda0t#ibWJRSy}(d#KDcAmnoRO}a3-pr5M)D)<*%Qm^aX+y)K0g-?5G@BnKb2K zv4V?zXxfip^{Cva9(Dx!058XH0 z^`zD~ikL{)$Cu?1I*`r@yP)eicBnT3R}tt>zvbGIIohn6Jy{%l{<)6 zsS~BgxmI^*MfnR^f;7_5eS>tqzf)_kJg*547z<2!|DJYKq41=y!7;m6-^@hnvvin? zi-J7dLemzojO`yY@UPPY!bdZSj7X7fw>`6!d0^e*Eo2BdZ}!|k^&py<>;TgleU zH{SLFl`P{cYZfuG9)Z*-*YWDpqRgzhx!yT=Ot#HqO zH1Wy;+YhCX6)r#y^!^Gy_F;-2w04T8=5SWVXD_%#CC1nfQ3; z8QI|gelNlirn`J;DK#3htmpTvONtlB{8Lu&C&s?+QE*!Gp5d6E)c?uQDD3@vFe3C~ zrySgbsr&i-<~feXZ1lF5`_svdy}uK(#2l@7&YLC#@l+MCImBo9Ki6Wm9b`^@q5K88S<3maVY&~Td&ZyJC#D*o zv*?OmuwnGK{pv)+j1?09^mB9A(|hlikUjKnZ7*Z4Y_=7T4s~a<$gncoJNNMz%F*#% zzb?scxNCoRJ=@Dzo?kjGZ$su76P@oaki;BODwGi(j&I!Zwrxr% zqAfxklUUhSxcqqxIn;IiTTakvdtS(SV%RYvX<_xlSLg9DHfoFW2ye`^wVw5;S4Qg- z9@=}bMWqWcdv2+>(Mz++veL%IdVXVmB1`V~$>)<^2rpqDL&&k=B3_M?oQsYMx5{7M zpt*V@v!sejB&*V`ZA4SVJZZXUzB{&Pq)jMc-~X^^2+eJVqI1jEPl&79x41;ug}J-L zq<_Ec)_Hip`}Sv=CTc4MCo1wdzN0rzRZg;76mPE`k!Frryni5+U(SsoJ&q{UbfG@JUWcK+DixlMeMDhas1bhgMulOg4xoZqx7oV=#+(PCdVE&nb+{x|U@m2DT8&>5MU&sjy9!olFS&qLA99uHw zdpgwe92z`VKT51PI^kgEUxzjrBD zDi>vsE(&F4T{|};J7n*Wy|-%=SBZNmJMN`$$;u`>>&ocLy7oolW?quLen+40&maBM zIrp6N8qe3``FK8_&*uU(o+L9^V>tPpQB^+} z{0G8h*DFl^_~MV^VGO>G7FM14rcbl<^p2K#qX*g-1c8APa_#*U>MQUZZeM&pEr+4+qLAv$VGki8!5VBOoV$6Zo2hD%Bhz!(_wd_1iKGoS zCmMOxbUF=Y?pae?=+$dP%qQhYIncEjc{`hQI&>HB7xZ4b9ulmoOUiLJFxGgWv4E1# z=yt1k^h|UW|AGtuY-@A>srXd)uS-|k67Ks4mbJ7RYsGf-d}|o4v+PLz@wi_n;{4FR z0OdblDT6Vx-+xe#P2(AuTV%Jc6JO8idMghVix%5|VpZS0u^1$CvKDXhTX{7uv``Z+ z?T+P1pl$y+=wUJ0@rKB{a`JISH^X^UTzlp*2@~-yP>t^m#p08W^WT3le10;rp|~eM zF7NNrSXi)6@3IDU(%3l4@@CueIJa;u} zJ6r^V?7hfo$e{SAe}i&hl&Q3Y+0ft*hB^!HcJ05gv&XPcs8rPj7Q4SeFGBWp|C5hl znEyroeFWBVzDYZtcz#I#&+@Zrau9Ax0b2|U{P1Q(XMlCAG>1P=9k8hSi7bl{!x)2$yJ|g1$Quo6EyD~^(x1? zF%04Jl2hj|vp)!YnN>7Ydt9z|o1TFR!P)QF#%$-DD6r$j&~f$O;OYQSKIm3D82U(I z39=**-&EUHzs)gXe*d;I;5+Rj=9SAIfBpZLZpcO&ztr#L8WchPNhdMSdv%hS+nKg` z7-oz*sNP(rXZt_o@$WZ9aLJ-{-o&094;`o?nYMY+goRgAKJ`wr=y<}vCBpch7}e&A zvN{^?4_tG3En4=`f0qY)D^sFNs29)c8(s*~6Y;!Mrh~rg3{6@-Wr5zFkalTn5{0hf3k?tHuj0!2XiA?IUA~*MiwxFomD1d~1;Lqx&oAHIRo4E7> z@1_R>sc>=Km^skh8sW=00HmnpWvZkx3gjWz5N zB>~Ae3`+jkEP4@^Zqsfu;>x?K>DDrpwTjvMaQ=kN+H=roKyX=LO}mv~kvnR?f_e0o zG5)$8lIr8TZ$bhCVWCk#mqNJm`PD(xNEpS!>du4z&-SRXmvIHZBOa-#;_n@Ld9&5pKK1W zDFjs9GynK}QHRa$^QK=HggN^~z6R$c4|M@?$~n-I41kaNt> zpx`i+TDy)5xx7xtnC{>LhB@8_DUwf{uq4}q#r|&pGg;F|lD~WYR!t|s+8@(Q%#Rxr zTISitPnek%PM3U~O5n&xbiw-lC#49oWXq+vnQac*^XtQJ{<|K{Gl;FpPm#*aVb&V? z1R`AKj69KXO|q>vMnI=g-1T5h!p7vEzK#d`$qttecZ<6iGZTGhc2GIr+G{ z`gnHVOyOVM<`}DoAY%g<1pUm%2vvQ>TYuyrH|c+SoCjN882<+tux1-1Elw{rhBa;H z5A3Ungf^&+e;q@u@PSjC^Y;tzUEl~ziW#UGrq=Z3w9ba_0b;c$KE&K}3-~qR^7)vw zgVfl{zKazkc`%S`R-@jrAj5g?{HO}-`N?OxwuKckTi%N6s~(H2`QI*h22AWK2*A#$ zA^VdVS6@b>z1EgRvy-_f4n`;iSuO-V#{69GKfq$-J11IO#Fu@>8IKV@Tea`IhYbTy zy$}=ztlKgPq$s1DWkxZTx_S(}beRxd;~R4k=IP8Vjsw}U)oim`J5s!0}dV-r(8E0M)p~qi3|NWWtlEZ z`WLsIQv<+#kk}Iq71?1vd$QFpXES%QuGB672*>O0irHxJIqKD6tj*%b()O{JCwp?l z70BNIwu{=iJr@JRWPIhvoYhQOgskQErv_F}DMp9P_XyRIcM(4x^Jq37WH(KnF_@&7~ z7_Zg)g<|RPt1ptL57-GrDDsl(mJJ_3J6&L7B{O(-WMKW{ysJ5M~61sbL0G|HuRY1UedGf)W zZ-1<6N*C@AUM=4q2J|Z1AHFn7h!T!opw;Fjf@tso zJ#b(j;{JgRY3Ea`!&$GC{kmb~M5hhG`@8l$oZwH`>fx3YDcU~oG{ z74(^U=TN$X^Ed7!>V|RdzBs>%zDfQBJ1rZ-eP~q#vG~F3k;Oet@E$joQwnp z>I#z`j_H|}GC<8$I|JyD8^dRn-Iu3|+CkOoOHZU1RcJ3HE8$ajQQ^<4{Loeyp2d-?+L0`RrwpGMd7f(cgE0fZ~7G1Rr6Mn63{ zsaQ)`jj|o)Tz(>r9qh9Tx{Vxy#*$4dy^!FF8mLQ_Rsy$z<0U`{GkBTCyv z`4by*4M9jMEJClJ&-SJPjEQ^gH?0=)JOcEY&xmbahc(b=(tX$?v&OtB$)HUX9rn*w zN}Z#LRBnU~QYEz#a4cor?MnRdW@X}^B@!o}))d#IpbRDnz5EW?N>o5)1}H13?g?cSHkdC1@euoA0V>TCV3W*k)6kq^w@i)tgftfzXlbZ3Q^VhqP1IiE(FK{-G%Iq&*N`M(a<73g1 zM*Q?ZDOgd}pE1-*xiM;yRBhZ!IWcOq2^89id;^^pQs$P+ezJw&Cv|}R_7nlOfFKFI zjdwQfbGg^2pK6d}@8@+g1Ls|dd$7EcujQxj`k#-8&N}m8ygNcWEmOi zjbVN7;QS4*_c~jz>aR|ktfK?)GQ}7%yg3O8$XeE$-rq_>OhiNm;+?qpZIgX^DX>WW zw)bGJTU?y+l2vXBh~y!;J~LB-Mxs$_4d^5C1n7rO4UoAd6~oSkMwLY?MX4GF?wd{) z2{?UMV02Lac+-UvYMhB{)Haccg!aXA);y{EsPs+7vm{L)py9DddK_P_IGs2E9vNie zNk(|-y^62FSv6+rSq|!S1{!==($V>;$&q`IptCuH zhkOX|#j6RY(iA|l?6{q)Ruom6lMypuq(G3!o;8Ep2s$n%dgsb?2WsDKMiNPQH)K_> zlOGkn_Oh+PEaUXa2B*$0&D|r)q-RClDhqDqgka8NH%F`V%T;q>4?%#eCW?|4c7xkh z{w~8n2I@nu3})ht4kn_SOsA@zSgs?RL0-Jy7ZTi4Cg9!s<$1plsNim2rFEa4 zC8{})_vm(fW5tXe?JNm7bqXRuBf&B>ziR>fW#h4!k*RJ&3aecj?y+XzoyjJcaX>8D&m;@?s#8sbsI=R5)4h50X19%r^ zw(Qji3q>3sy;Qg6-bYqVLV%bV9RvZ!naVhi-2PCh7X>9w&dzT5Ej7*@@bo0{gt^Ao zJ;06BloW-C2)xQ&_Huh|=R2OgOF3LQ+ax1(rwaDu^mK(CqwAGYd0X2Gs+;S(ol zUA)AQx!r1|^f=6!+JQY}-X(M=tm%seAKdj~l<#K&%0<6G*VW?nbzxSCr!G#Y2l?Lk zrpf9}0f_kbk#tqULPwOW-n3@oK}`*f$l)7;YU*4GZc>w?G>bWW}ihtAuTaaWc%Jy>$ zl6~VZF|68mEG;@|v?FA3>}4Q>blg)r+MLm3!|3a^TeZyWHzujQ10m$ypW^i1$GDVV zA>l$284NkG5OBK`sR3tcZf)UoL?Jy<=Q7rE95vL&?h47p5Bpxo_5Kc9sGv0-tCWvhHXwPK2k^D6tlxitlhy z82djRy|M67|Kki-VAy^Ws)#NV4}1I|)N7?s~k_zl#Vuy<^2|mg&sDN7Luo zGBY*FtaH^eIp5hYw*-EjVWLGBq$D}XG{)ChBfk=$T$b778@4!UNDTPb{GjOuvAmG# zLe+vq+}5WI+NAU421i=6=ZTVs%Wyb2-6W&_Btw4Une6UN*-j73nXy6Y8rlgsTC{E?DZTx zWLld_a;+M@g6~>AQyQsZONg8Q#1_A!)Y?#|x3-pMh_$!F!~F&rFIE!FS6Ll!6lwnY z$qcr(0viG2KkbLss~czVYhMZ590C_E96vs{LK)yl-yPy}e{w#tnS4W1+pr;I1*LE% z_=lHNQ#q_x$|^t2J#(ibn$z2%pswncFwgP|>2YG_jaUAL zu2`gK+0GxlSTuOW%-KC}By*~xW|0uRvw?E`GTO_JEn9sJ>_$&T0BC#%!@LM)t{FW) zv{YA4wd=M}6SIj&Q&q2AJwMH14Df}jjNDe*g~JR78G2)UiFLUpDtuw3_>;dxS;v&b z;bpG}6U5oGpL27ogTGJho8ZJpaEMyX^`ZBFMl2vwBT`Jx{W5U&NRAjZm zBGpHKWU6sNcJEc@RMO0ex~MxI?o_~dG33dV&6SQ`CP#rBfoEFPK7?g^v?wlobanut- zT(7?ZYq9Ja;F@G)+!B1cSS@rn#FL&TAfwS*mmGepP|P@+=8 z_cT0Ebqv0*L5JKbo=&8N{V>xm=n4vba+5W8@O$&jzzrHQ)UjhV%w>f-!%)IxIC=3S zA<^6AJfcummrQbpGZA4*@TKXZ4fzU8m2gJ+$W(U;Jb_RQ58g!(&}}Z zUhqY1P7!OacEND3IdkrlA>dsdZvaEd9u0|uv(DqMTDh=CsgJ9&M^N2DZw7f`7)ni+DEc=#cIwd|sN@zU0Yl_h_1gx?!DdtcYeX4%gx zbpc}IVh1mtfO6lkY1;z`oC6i zkNGvL&wQ$~Q`T&|t!QLmp1^eGMC*Y7w3yXY3nvSduwafeHV!Txm2OOvT}#-Q=A4~n zipTj6Gm*mA0DXqLK+%tl(_fk9jT$w>_7d?6|ES+4%sB}f?fcpJm){TJ+KqQ8&(XQB z5`UT~cVwvAoTKx(>l>Xfa&dVImg2kxa?(QKQ%u9+J|g2YGe^%w_G?S7;J#9B#q8;< zhM4Wr(0IzIMVq+Q04T=`Z@odykIhhBU#Ib$WnWv`}$X_tDdD9*Vtg*t{&693qUq(O`P!F zo!DogK|v8e&YnY?477HUuc|k%srK~*F+$!s#-Xday85=a&o4VXnI|e$)Ejt^b#U~~ zvqT+g;T*uw+rH~`+Bp3&>17)R85pS7y8<{>VZbPClbQnwgf8WE_g5HM+wx-76Ck|g zdN0T^KgmXW`n|97Jt~9kxa7*TO868IC=iT2^$PNpJFP`~7>~|%2_FGL_Y^D_BEDOm z_6=XJS*av;4LeP&C76+J91S0t4`bw|nFuw<-k3Ljp{NI_2kQM(>#_Qiy><`^yK9pT zz`oWzXZscH{Ny>_ca%O~$Zy!I;a3G>7GoH(2I5pT?m5@z5Z|UzcYnBW_(c9Y2WckM z2GCT7K%RAZdhl~Bv{9mHTJg)WTx<@o(1R7SNu`XfYt?{Jc@D4JUs^gnwJp59rW_Kl zt~c9Il@Ph^M+Y-48;iBxHIo1TYCZ!{8+`U@3J1r2lzSt*j94E5#2PM9xSlTx|H|@T zOKE5t^;G(h!&esoRx8XjKCWli-iHLB8>cVKPhQxqN2Keg7)e~U_3)@k2^4wmjRyN1 zzVaZ#Nz?ZFoPSp_Cx^hEEPDFzKgd2`_+!(eN_}+!qf`eTWge#7n=%0P`1M~rh}RWa zbboaAe#=Q|!ybxb50HBvKMomd`7nq2Eage*xDcrDrV4$>IT7QtS4${z?rodfz~QVp zIYNiOQMH3QNSks13-!(|wzPECrBvARs@_dCy$qh^Wg$KXZft$4^zq8nN$Gk8fnliK zo8qR|*ABg%iL|MjlD9RMj3U4H{ZVqsE&vMRKVOdG7-V0j^*+k; z@9(eX>3Lb#RQZBPET&|)4}2<&v#!EI*&F&Nr^MaeSq_eHy)iqI`=;0U85ISYy|yxI zZ9R{5`~X7n3vHm?T<1{Ir8mX&NY|uCWJ+9-Y0bmq z*hQ|Mdyyvy+OnM&8EDkob9!2b`v5hoV)R5;CjvpW=2_x$#Fk+MG*!c#47ra+(txBD zv%Z-J(ts0P?@s{TBA-40CO$4IiuCosDUXdzMzq~1RjtjE$;HE!Jxjo7f$y6ybZ+2e zS% zNH}yON~?i_H?BGFDj~aHXR=x+FNWnV7BjC*eNP7rswcb7@O|RZLIfCobmHl;Q-Hnt zq+#>Hh`(=y@547OejjVXn`i*e{OcYN{IsfRC|Vi3a6Qmj-XDc9Bc*X6<_zr$7C!yn zBhS#S6Q_rKb+j_HREM3ddoAtmPCn+Gv(HQW3ZD`;z*ANdfv&qVE`olI1Ja^}vz+nN zrdp2#Yb2|1)N?A#3q@v(2EJZ=vY4oPD^?w?i-1xiD#vlN$l0`UF#LghUS+xVIz*)D zCS*Kcvg@(d>wPax-R}8d#}UF#9-kyi;YYrRS^NBB;pjb`!b-hI*xlG?RiAkkk53>7 zb;#b0U)5I~ur&c*wVRr@CBpX&lLMd?QMQ72k`(}Su-tYPNZbQ7M@P4dLsmaeHmvfv zsfJ8M8b_V{+)WdWJ6>GfhwJQKgCORLUCJAex@>|5n9?AQ5E-I%`~Jj{ zZPOm{^=bJjV0-63gnHmE@fwZ-_V2Bj+*lua8(}k^4iNYS7tXWVAQXs;iCAK@wfjuX zla1rbXmG~-OT!0N^)XG4X2qngw!NM+6q}wpae+2?&TCDjeK(U}o2KCqE^Fe9R0G*+ zwH~=wQOL7N%uY`h-@6XvPg)vEWSVz~+mf3b8Gx#tCjZ(b{x9e6TaoyJQdMgq@1y&x zb2{aeSM1ot?m;yXF%$)W`2%G9Y$r#GK-|I$Fx$wQ$#m5zGuFVO{H|mhC1;j5MyaZL z2(T)R70%2RDTrlD3M0Tm! z5&6~__2}`=UOsD`H~e!=`QI6pS2;|Eqw`KphJEjhcBqONMV=!&MHwT#0h*X1>GeW( zSl?RICUDX?D&EH;lkeXC8;NO3LXd*FfM|#OY5sHV3n&Rg{(@mrX==YhBqA=x7Z1%% z2S+BH`uZu|eDaWHHnPUZbxIs(7ST`BPFYq{vO4cnT~QZ!zmK7DCc2=t`{>l>LDr|! z@)Fn5>R=CR5}(Eb4@$W+8medfpli-A*;Y-O(UkG zW92Ad(t%u(7sl$T4poiLyHf6=J{nn0hcUFVP|aMvzM;Gs+8Gt^Et%#1BqANxOs+`ts>PsgHw5$N2%8%c3V~Tko6Bi2hT>1zi0QNrfkO* zwQU|U*`YyM1?uQL@4$HBl1i-qqP0JjeVAtug=C>8mgd8`a|%^60S2_MRrjPt7_-5#j6GKcI{S= zr&Xrvn~^Zv|FN$ofr!rBShj@jOiIeSrKR72K$`2v*=#qm;p>yil99F3JsNsFw8O!% z#ZkiP+#gD^TtVvL`+(ivh;7Hc-Q8{oBI)uKTy%_i@M^+Yn=EIGET>8`7zsXH^xSLR zKlBRp{DhKmwz-`y!qKId_}sJrBiQedCgc|~6ZDyp9!aYj^^r|?1h;E{&TMOFytyN( z%C0B^C9y1bCxtkcJ*L0Q1OSKszqKnruS8h}ej)8t@D_sfU^43HPA=_4==Y#TUL&Ew zE`UW!`NclH0zuN={aE#|D6+bty0vdtDo0O|KTb}2CI9_XeP%fFV0>*+q(D3&Hy{fe ze81sX$h0xG{$Vl%@is<1kyeAV1j@#E2SGd3^HV@Q2O4kAu29#RDqPv+|DZL}T%N7q zo9VP=o^PMu=?iwZt*BBbxtN8bnNQA^x_PkfAnGbl(P8iA#`)Y`v24B6g|@kHJvN!* zgl#1oB>x@n{wjg^1*p!DT&8D@aU(|4xi;NFQ~d#Fw|u1f!^g!rdJE!~eECxggUyw4 ze8qaHSbfQ1XLn5CZYOg*HA}bL_rw7BR(o`|a9QZ&x>eXQ>(4KbsE+1m?k3aqK5k6u zRA*E3PTlzv3besVe6DF)n|ya$z2Px`cRGs!zoGqve(^X<0$Dq5rjjiW5IYtX0q*AN z@pc_eG*EUSwxeyX1v~()xQ7s7VKkQ=&Urf@N}o==Skwi%ooMxQDLl-oNzn5=>(%s)I|n=VxeWqAhSd zLIF`zlZ*zzKZ52u4O&JiMUWe+`Lij7#P}H-@n`lYv`a6RsT;`+14Dq!KXClB5lf!? zY4s9qe0eFS-?NLSva4pav8W3CM?3d7kn~P8);l?nWY)TuU0%%jlTX@KCTO=2j9hY_ zZ1$24-!lkbx0^O^x44ZHb%+5YM6bp?UoZcn)2xE)Z16Uh8YM*-&bBT$3A5Y+D-e!3 zAYzzrGuOw=w+ANQ8K!jElT2sYzVQrM zk)(Shl9LQ-`y8|;FHX<{DG2kNjU`s_HNtVG$1I*$&LiOQRupAR2@M>-YYpbjW(fulobe3c#I5K=iwK4!upd zI`zTFt?!N0A#@;zq5x*^o$TNoK5zDySm3wL(!Osi==s*+?X5o?mlKlM90mKKv56CD z2dM04ICsl{(nX|Bk?KSNsrx+LiyMfXGozph6pUQ4RRZ;YI2sQj@6AtJhUV=uMg{$b z@=_d43s|ks(r5SiO?Wfq-(RXduhE10pH|0h+f7!u5-%(Tw>Uc#rXzDO@6 z{Yc%w{Enz9a%)1+hE!5RII6sNRLPH4G_r}|B6v8OW(f-rzj)2lgfzPb zw@AYI$r#_FLY!(ELKUe3t44g|zZH}FyX}pZbL|h#=>Vdkq5G@FtvEPUTk<=eH-F@; zO`00M?7st-E+`yqt=dex@S3IuH1;tHpV$`oVu$OIqDr9Xxu!r3?TqJ0(T`6-@2Cf# z3D^n*i(C`RF_NAcRi%Z+<`k`xBEWd_iJMJwel2b+JwOZ_(A1wxphrH`X8e*#aVXD_;EgKD!6g)MmFR-L7( zHqw|Nh5dn-09iXd7nB~^_3rUcDBipW8wk{kDu`RP_gHq~9imPj70gqgHf>A}-4r{W zCnwcAbjAy@mQeZ6cZVr%*x{Ca9wyteeJ6sP2euQ!<4YuRACa2&PN6UVgZ#>vEX*hI z7-`$Ei7LRS&0I@6l{KW@?7jOrY+>a_#bF1%Wb3B=xRjJx>{GVu3KZm0f}x&UPs4!9 zJ>|wxGy0!{lHN$~#xT(Vn~pUknRTVg$`7?y{SOFRpSgx>{rt>UrnrZ@?A*N z&W)#Mf^DJ%pqe5@r70lD;oMk{NkPq6Dzdvq(aP+h8uc}0FAdIPB@vwN{pJe8&;$1f znXquGmy)$btgfX;#IFze9gx(xAUa-eI$@#eh`oa1G=e!$41`Tj!}=4P1}h#vEUqqG z^8I8xV%H<{gU#}4;C+uO|wtCi9cXs%R#$LONE7YTOD7>pEMMAyLwQQ4b&;c5MX^H4R=LUos{yk43XM_?*_lN$6((bK4~ zUly9mRm}R9jQTh1<^6Py=CY5Jo@gLGc(XYiOh@lz5A1yQ&Ei7jv2m~+@6OF_M@0!2 z`^=%eiM9wThy{=Okg4U#FC6NOD|S+EB>(A?d#R}ru=nG$tv%4||J5%%&9%GP-88DP z-BeTkSE(!zv)ymTzbHbH&EpHy!sb-OnE=`%Il3)uT*P>0;Mi zshP(#W`P3ot2vut$1A5-LzZ+&#sdJn1~9~jwYIAr)kEe}!@?~JO^t(N-Jm1|3Rk4) zq(Xh~YuX&70dBh~=`Phr1YHJpaM~hdM}Kc&j}g6{IW-)hDIl1K&Q=h6(whVIsH@Us zoYbV|hw)MTYRcC7n|-oKScm62Pl$gX99QlfR-XPMv)A4FB|Tp*-h{`A=QpajIlsbM zQ-!}~>7Y>R1K8q8Z#?gJo2R=nZ@|aFx&SWN1PGqZd!HI4Pr~Fk;H3$zn3r^HDtaO7 zkHQx_asht>|EeL``uMxOry9Ru@AH66Ia(Nq$WI+GFB5OjpKHqP8?Y>&=$}z4i;M@U zo+?2uX{N?2mL&onfEU|VKrwu4 zXu!xgfjRa+|h-=S^aD*zzC6&@X);NOkMkRO$Ct8R!@Dn+A3z z+u@RXNG>5RtJD*o4HdoDTL7E__im3uH;4^8WlS^k$c#BO9B+S%`&W}--@MnC(Nz^+ zUri^YW|8+BoZpIEblx(6+nMw&A`? zNY^en8919zL(^-h(^ZMvR)s0e5}9XOWUb5|%dY>veCZKgH<@VC6i|QUrA&C$p>&cd zynA?}2UP7T)Jwn9oKCGbb_guBRFU|c+>R^6Mq=hFZcp+G;t3+bU_6Reb@I#Lmo=J1 zlQ|udI4h7DF{gGO-n*4tl6jiN%yGED-V`IsmF}g%%=q)L+kUxykP+=)rzp19;)08+ z{EQ`FpH}rmdMi{)#@{>H5vFZF?r42I40!*Jg6&XRS}cN&F5creSLoBP566$`&U%3M zUdtlvjd1JCsq7x}Y;L0U`LFoS+S~OMAG4cn|#WVtW`Bfs)F&NJ#?~Y zRcSwqonMz~{!?FD+o{yq=>XuOaT;rjzF$)}9jy_#zZjN|VL^!2;^Cy+AU`s()zbXL zw7@rbH`!GEg6jUK?`_-4k$2ua9^_~5}o+9E=Pu}#*{XPwLMZwz$k@n<7zL2MXOR6)CCeux;Vi=dlm#@TD(2UE?M=B+=^;QrDPE}1_nC=*0JCRg(PBwJ-?3JTvw*e|pk z25cUTov7;>`eO}qM+RSZ;Z5ySOz%YOWi|(v4TK$T(akeKMqH>Me5B+j8lat16O;xr zAj)>sJ+A&Uv}+%TmY-)m2Hj#Y?Q466xu5l#N~0Y{oU-YUbumG3-fNRVuO$=wb>N_PVzYqSf3KuIY>Nj>F7v9A;0WEO0us_{i$Kc7{M%UHWx`snI8el0}iIc~1 zm2zL13MSg8r)9h0)Ze5%QKhRPz6*5;5$evPkFnJfOIy}=?2*2B-Zw(ACx9>{HBHgE zMo2C|i4z8*87s=_mNrP~nD@Hh7k^cxX)o8izE1IdDlht{T|bldGu(Iw%N1U%KasPF zZSm%Y`_1f#u%L)4YptuU77={HmeA4lU?e{xe5Q+G~IiulT zyPZ1)N)OLcIvKc;fcU@}Q7&RW4MvM-@L^^sKFk__J-O+==4BChGYM+;(Dv1jB*zr*oIOtG1}ya$QICD1J0~jeGR5;406QE>cGZ(>k*#5v;YooB~BcmB=2d;dJoAjaZ$m(nI%+Qhd6C3 zYdx5h?1ip!-m(P<#3i(kj@YUM!tTQTMx*tUt0ayuGi(SOl8$wAYY+BMhT*-{M&eyJT*VRG z{xdTvls+nS1w9Lnq!yAhxfZ(rWzT6hjxm0CW8**f=^%@QNiF`p{)JaSusf9Vd{QsF zayveet{r}NMm*>p*7Z>kpG5LTmvClRQZAq?HjX4o;7I4hcSbadP)3k6WA!zH44E^V z`4zZ{h9D!cax{Ay?V9JDXM`FubDnW3E_E;H88&a$jRf~QzQ&hW-YGT~O^8<=nKj65 zk;EN=sZ+ZvfKZJNsB8aseoM_O@m@;zX0-zL$fOjh2CU1kuuVMW>nDarl>QPS#|j%X zmTdsnR}n8ak#U~__)i5ng{|!M@&vmq9`B5rGK&+n?F7M#Y-?u$u1B88L0zpM*$zff z+U~*0NnN0)b_Bu0y0W&g_$2%+Glxl*#I#5;{JNX%q#PUxr0SL>hy8gg`0J#D-GzpD zu|3@eIJM&wc}%TNZBE(;hW|R<9}Eihv|CVox-)LFcN|MnyR_*q(H;|h`_U%hvg5m< zPl}(`sM3%ku*hNY`8A~=F@o#52<{yi7Q+)?1)N75R7+t$CJk#6Q64PVj4P18%`YKr zqmoYU;X6|^W-M5sfE2$&HKCz`U{dTD;i4nH#2XGK-oL}Upl4{9py+|AAuye+ox~#E z8r^rPWyV~l&HNf_TMk89n*VXiaw_qvn51^bj_zya!x1`sAW>LgHp%%U)a=O(~q zMLxWU^lnK;jT~-uWA^Zrdze-GcaJF&a$L}w%bA&6Z)c8m3@!6YVZb3oh!mNzk7RGJUy zRGf^eKjbC(PmfFpH1k>Bx6Ceo0r+&KDn!~l-~HC^Bz1~^?_^mK$ zK$VfJhyPCC5^1(0-Oc@^3o~`r8r5O0o%{Sdr^wxIasq8j*#L_w)4b7ro#CL=*UwoG znC)OO9)V%NUGQdCbb(5R1Pie#jHJ7_*HX0F`+MOpQ~2ge8)vkIJVxippWfBIc=Dcu z&T(IK1z14)pySpn0ATkQ5DqJUBIdfHL;a<*$#+a~n4OgBsE=}NPfwI21i5Kd^w355 zM%wH@hF^BeI+l3kO%vJi&wr}AVp&{kL|1-i>UupogNNTFJT`sDYbJGtJQKKDE(+t) zh>pqOYxqL2GVT8z^|icSzPV1+IeCj1P1>6d-@F1?GUxSPU_TnBVhbX80W8k_ueQnP z?C8QrsNE{x=joN{N_N+qNZcqq<+l_a=d6P7jZp+ zKu}O-qHm|rRT+t0@W|*7Y)CUN_16dng-4LbrlulfnLM;{D!KhMTAP4ue2y=EX(_eL z;vlyxnnpAqf;c&u^>9CXE+d$s>WzOcJD~Ga=y)yd5tH4Bi-W5xS=pPKyFZvqnWGEC zvr#j-+DxqBTLHJLyZ~1UC1a)8l2={hgpacTz}$WDyvBjcNSy=i zG}Y;7Jje4_B*X4!h)NS4SaqcwAmHsI57hv2Nfl6Z{*6lUz}Aw$4JmEAEV+wfh`Hw} zolEvftqa`gA{tcK7`Z{km-sT}bs<`=a1uR0u~zy}Ij$FEvt9fXKXb$)_L3aN4(Pic zyZ=%jV+Pm^@rD1|zaTY7ZxW{%O(mrxX;ovublnqboM!`$*Ao}Mu&P*UT!v`%+UmIh z9`JMMmzugW2h;eKBB5g3$(&9W^rt%4(7DC@-QK0xUy1GcMQl5Dl$~lVDb^sM2&r3X zdvy$|RXYgRv9m8@&9A6XIRR{YOqzVR%+j>QCZ~=@Kb(Cho|2Nb!Yz&cM;{mJHVN`l zkS)0j7XR9azd?CxOnCX3jJ>&n%#Cqiju?Lp8R*3NM~cRC_m0!753(>~bYu_LGy{fB zNS4DRs1lB|SEg+*h6{xCpz)h)O=lEH@0(aIpi)eBHW_ez>lzZjAIgsn8Py!@;=LR_ z^O9wB#%jBT*)BdHq@}Q2TV)IZnA88;W%A4u4a(g@dh#t3cN2~1NTz^4xzu`=JrqY) zK!?6|`AJhdx^dzogCW!?%!h9=s)6nToZe;WaV1avwe(@JOnQY>c3oyA?(zD|;)|=no0L!ibs|Lr#q_J`&=QwmgQO^Z z^b0kVVF(|gGB5@zps1a`p4{)9?0dWMV}DUR|5inMhDZ&MQQ(n?D>_otXP2cdh(I?A zP1SMJc_2qmA9XF>B$g^d+FVbUAwD5O+#Qbg;eKiQtj1!~yzR$*ALHNpD~-0>Hz!LKB?ygg z4tb)u1-+a> zzx8Zo!y_XS1P~54a0ma(NuRr4D}v+zj(?Y+2JW>I+6~4$(V_ zTHG$ac14w2S2cR`+cKWLT`4j*(#}|a^;k!3p{2_pA5*+ApQ>+PNdgl999FWHxM#Z8 zBljF8W(fQ>LiFl{5vO9i{BNq!`tDmXU&-}UcxWwLjA^rzea}E~qe_V3DJxWQq zy&Zp?vOBTCgxuUIy4Zs%DWO@llK=jRTt)@#%8%Bs0r7L_zIb%Bctt%=N!m}_CeNMg zPn!>{>SE?bx%255LL_OHXcK%&(Dsou#Z?XLw}N>4jZ^jZm7aH>)Y%$_MIIxC;6p_e z6_uyn&Yc%+7~ig0zF0NK-JD+N^MQ{XTaC|)7e@3ROTdU(kVhIpSBI~_2#h6<(re2^y*V%t_LKmy(?c98g+_eX$RzA62OxYi^*g%>$ehGB* z?f(NtCvS!TpuO!_*?>u4Y~Y*aNV1QqZt$0?bc4D}4%XswRMT>Qt3~*AjYNu&cyPbB+}FG%P;1Ax68R4Ihzh)O7SIiXG;?`p>KzUT$q;#NnT-iyJ0aeK!QVaaLIq>t%Mm z45y|-m$uE$mv(mVw$5@xpKIVZIeikbO+J4>2<5E0`@?_74}T4X-7^)(zEl;O-ZeLV zdAHfqgFiZ&&%=*7IXD~w67IVuoW-*PdtC0 zo=$E~O=P>*ZHAvcr7X4H-oa56hO8Uvd)sh!Tw|1Vv~4Jr!QV&*4^L@I3)0(=2O3Vv zkZ?)!>FKpbfj z%mY9BY-xHjBh-W#-y5BJw2^q$D&fGrh8-ZpbJ?BPdg|Zcjp6__FAeBZLEarNNhCe^ zh%dd-68b*o*}be_@Y)#FC4}iG_rJmK*k`dAbB@blsm9#7qWqlv1z~V>=5IF<-cVWY z$Aw58fK$OexvJWw*<@RH;q6DCmOxRhfAP)_Njn!7uG*{h6{cE?{=x}t;s@Lt8~xTs z(n4o8O+B<6&35+g%0s{Crl$deE*X@bc>xqKrliNKdKIoV;=2UEG!>s8>z~$EB4qvA zns(9W?BY?cr%aUfCx#{_UG-D;yE)RT4@riMt*U;_y8u-s(sJU`~~ZxtL@Sftf9y1ltG!eQPeQIi3hLY}_i|>@- zdbR0J$W=;l>s}Gwwh0P=im#4$Ezl;gm%mq=9Lw!=TfPQ(SZjj10o7Tu16q_H8q@-+ z%-z_D2V$OcaX8!B1M$ZK{46e1Eq!WpOCL!_IOM#17oSJgz_j2-O_VLWy zlOH>ur&eSGgz7JTznN_;mX#8S|t#SIy6>Sq4}M+=--LD2)zfZA_coiw;<$&P0GJpaxlIX6Ie=ph8H5lGq9p84`e&hf^cTQDj z?8_u)Um)RA&ARLPsXz!ya?fWXSwrJ$?iGo}0swH7bxRi=%&x;M0y4}49XL0e=)TLpq1qL4g^&fYm4qh({%Ts6mF1*c4(3|pij@M17d7LDl z_<EO=C?wQ3Jn>8s+47Q#ixa1!l6#iS0CNhwu5n$y${*SEGuXKCsIch7@B%rmr`!dk zU2o2BzXPMZS-#ksx(+a4jsdx12fEV@x~!StJ(XYZP#R}HenO9PHCV#F>~Mm!SB3tZ zf&Z<4ZTx8^Z|IVAQIlR1hYXKOtfKg!b`)i#&DNul(5&-Db*up1# zvOd{oRhesGhA|IHd2^cHdk%2~UJIlAEAEb5w9C9r7P!hP?9qv=o(Ung( zbmOWmQ(LaCuMqzG-<`%sR(XK|J>;tq0ob66ThQxfHUh-QCaO;FQxtQONt9W#Phy@JpEDAx>Tn$eu_X!>Qx>z&+%oF8EjjR7Tfr6f zUaa>9>@#Z9Tx^UiV?F|Wbh(~a!dL^&$i#Jm6ThOz&T7pj!T^;el86s-Jq#h9*X zZg=XstG6a7%tZE^1e^mJnbmfg9cs4J#b&Fr=IGfu|Bk`7(zW(*46kBB*Yg#C984vH z+cr$?%W=ybGcw*Sr*sLsCS>E{vFsAWse-#-HG4;rtJzB55ImjCNZKm(l^OTh-M!E|lKDa>h-sM=;jxkITzh8YM4{tLSZ!o&(UrRPu1fP9kIRsb9;! zqjOTFO@hj_*L@#unMaB^ioD=j9)W;isG3sda*iPV7l{i0wgv1ExOmaE7h*u=p%tnF zIH5k(WN)e6{Gcyad#9~F>H=zTuo~+3v%6Uf7Z(@O;#6-lnksXHmkOj>s#2R=^#3oK z*&d(`j?EQ-!qj3cMYR9$Xrd?8H|1|FfL`09p$(mkf1_Q^w95vxEu~L&MZbSaUpH8} z;hDO&4EG#6Kfn2+{Zh710v_{;H|w95P{2Q6T27D+Y89S0XR#OT#)qV+$;DZ32OyYM zpBk=|#%*#~-^MU^LyMf!p&V?#j8=0E=OiB8G`OTkL&8G`k1^lngQGor zI0Og&E56$qy{keMRqg(6yyhrk7Frxt(%UNX2h4xR}BPv1v(+5sbhtbSeY z@%d?a%KaP4I51f(9IXF44skmjDz!N8LSy_(kNR3F5*KMB?jGzm_Myt;Q%jg@kdO%! zMpW^q$6IX17Wu9DT;`5P4Qvb4RT&!7G6|;qigJ(H7w`6}2uOOMj{4G}^)z_h##wCs zQ+r+p^Sh#O+qCtdA6@p}Ga7m z&dakP5kQ1|&eeHw6lzB^X?a`w;MzTve2lv2<}g7I4D7V2Oka7OvJ0t1D@K+G0#HSU z3Yw>?$*aJd*ZgG4DvDB+^8=_Jr+SXzjtkvgtWE{YLLIB*Bks24 zi79$oWBxQwjx2}Tt`(g>!T}iD_n+u)ec#;#+Ss?fcc0$ZSC2MG^w}Hqy8V1!e&1{a z2H+$-yJuIlJ%5m}NyHV&XePJzpJ?Bil^Q}Bja0>e6|TzaRVA*$JM6-k8xCTp1y||$9er@Gza4{ zX3bGMSIO#9L>0OFMMI93@b+z^)#xbzzPt4G<7wBMVbuPeAt_^*_qN4DmFxj%e4Qsi z{oDT;`aD*$$ejm3%ly17<@>BD%59e0Bmjn8tgx?M16>BGfA64it6#X4;(%2fQ8sIR zTGYU-+pjhyFi-%s^^%H*1V7#t1R~{(FIwGOn@VAt)x-&4idj72x?}WqK&_nr! zn|TX*QAyqMhopS6+~g3MSfyVh=*y*|Z>zy}sVP4J9B$y`n>P@ue*3IX6{CxoKibtz z9^Ftyuzu6!QFiyE?fUL%_)fU^x=PZfeIO z&g0()GnRXICBG$ZSwsjQiHFK9wm%1|Jsh4(rLbE8=SrT(PS=SXL?T_WkRIdc%6E3B z9&)*s@e_?)9S>nT@K2Q4Ydb%Hw!4^PP(V`$akJ?)z>M*w?J7k9c_lx(I<@; zz1uLIx2&ux|CqTa=Uq10wCp}cj5oh@Su&A82!P_>LquZzve-J7eqKf9H%UKY3^hQ< zrhnHp$;N3`mUE2O&B=Z`EoRV{zoU z9%oWs)r7PH=I>rmHX6yd+che#WhEE;=~@13sLdlC~=bIl&WT# ze&Le4WD)A}@LDT53q0Rl3bxXyN|5|*hR^^bh!Nj1YdQQv!>L-w0*bxSzksRKYEq<~ zFcpxlPtBcfhxYjY*}KDqb0j7R`lM?jSD48=_O_xK@5Fb{iprX&060OV%1e8z#DSHL z>6fG`^H1Fmg01{wA#X*K>tYW7Dq89QE6@Bc`p>48^(P^)zd4h0GqZfbjE!O>jw74h zUTqX_O_fx5zww#hJWb~fKD_KYSR&}Qa)yJ!RshY6%(%2+_02@TdT6S(3L~;9VZz#Y z7Xaal?Xh-H+N|LKyj%9*jZbMDOD0zFqEj!&B~{4>s1?CxoijcmxBS7iV+c5~T0%Nr zpqK{;vox#C^RD~%Ynn_{5GLF$b+R47zONfyXf_v$WaS&U90qXKwevMjA=W#;DMOXr zXr9IkJyyE8*q@9TpEjzK;9UyX9H|Df&+&s*@vA; z$79CZIg+~5>&Diprn5g1$jj2@9#9}l8hg;$yf{DF~86cc^Sd99{* zr35QGhv8UE9$5+n2VF4(hzlj%Pp)@(gPf>4iX2}EeCLm}AiQxg?p(~2rQ;l_m;NsZ zq2G={y>=l`QJH^sW2t?+wLJQLeh%t+L}4XZ%o;2B7|zOp<9K?Q5fJZPZ;(?54EFDh zE%Wk(0O_wX*gek+Wk^PtdN6U3XOPxJ2++m8RRX39ir11*i;fc!5lIku%5cZ;0|Q$zcjaKT1pR%2gU zoU`NfT$P_ddGW6?{BXf_k@2G9%bf_{b;hpu@_fH6?iR;dTCUX`*S}GS5(6OQV*_^e zhJD(B8`FyEU}pC7w!_Ft17j{^xM)%y)CM1i_IX-lDjPgTk804EoPAO2svC7oc?Th6pISo`z9#;SKqX1bo*tVn%z?EaK%5FfQk1N@_|F1 z9O4)s63;RQBzn@?HNop&>EQk2cbQ%1sKGAFTUSh=*EP{TonzwI*dUlJp1B*%qn&P$ zFB{LgZYbw>L-q~FtTX08PioxnrQX3O3Y}#g24nQ(@r?Os@d2@cfA%ab`>zJvv(QbB z5?aF})^#2qx-boXwd-++&!rMtn^)dHP2j1_8W6Z*g#i65%3|bt8OXUBCEc;G;YfF8 zX87lox1M+v6iy_bQ|h)z9ke}WsUvhf=budOBRGxX3Hyn_c^U&vv zbT>AhtNbPC5cFr`-J`?k1OUoMl&2K#zu!~{&Mo*jExDbeFSR_Me{-hJFc{z9LWdSv zvmco@I0`+$-%f$=GrUlnWwr6=3&`bAXJkVZFIvt7d3(BDFV$vhGzGLhqXF3W-WFZ_ ziHx+(J&%LHGx)s%b4wpQ@mkZvho#!S52~ci67AHr5h_b{cn5IL!8z8$iRuLM7r)E} zZ;coaU?!HkE*iAclQ6=Up?9O%=X;D&>tNWRg?BHGyB5HJcc082WIUGeRO(wYh7TQ< zwl%a8!Jreii`RqDe$J31Jd7tF2Xnk2JV}c2TVnC+)EF)PyW!=~vxbkzYO}yUFf~k?csL(}nMLgjsMc2~oJ{wTFm4KuHlMHF#(x`9 zA7Xl=K!Zd6fXZkg_G#Dw5h}9<)x#f-%i!W426=!qiAN&)o)u9iobRG4Mh&mN)lPTH z*P}{H+vZ+)x9)XzxprQb18~Q0LzS@aewsKOrX`u{Q4@8|zp- z{7rt2{6E$iwJd^U9K-_=P+M+e_lJUFJopMqF;DCX(^~@q-I_X=_qDZTQzjtHrd3OB zF8{p%y+cTV-*-#PJX4Ze->gSVE;-Q4hU_fQ?FPtj1f#|z+0|5GMI3n_WXai`=SR{; zucM+|qYsL`IicAQ96}C&9u5!0Z&FYGl&I=oVUdcwth}#C>_;Ax0h&xSQ{ND4UmedA z&f59`gmIRa)MD=-@(mft@;cpA&i0M)E!1=nTHY2Rs{XjUA9X=4XA=@!;9RQ&S<+V;%~_Q5R(kDQ8laIXkv!;lv5` zUTEW2BM(VP_z!R5UZa6%H0y=NdaJbz5lo5rwegi}eu@2@(1u0UT-KBH1WBt=8SSSL zk9!}*5Ej^Dd%6#g#fA0Zte>hzfS8fb%50X-r$!m5q7=#So{H4pTDNfbC&@>bB>gTL z=bufi%70B@4T7)sDfWM+K6P`ZYr|?Zvz!|`#p;WoFiBJ9O3)`S`#i}0up~M^mi8Dy zDIPE}GFA^0tF`8i0lSm1n0oj<%CI&adIo+wfisX#a^28Wh&wv+!Nx@(KkVI`L(9d) zTYK^EZNkDyO)m#gi#DjmqImumo}Kb{YLT}h>eS&I7jOB!P$|?T&L~S0 z|A;ThFJNkLN_SbQUidNcIQ3X!^vZb3@t4JC@rPnt?ANy;UHP@vyvhJ17K9~243BX% zR^p62RGa!?Yy3?T8^Rzx3(;$q-W;mkNA=Z_jP~>?MCK5{UO4=)S5zib*JJ3foGxW& zNAWY5Z+KnlQl4G!7Ds`x6#4}Ax?FdTGA^W>f6pj4?eMg9ENLX>?r41Xn)twAAMrFS zLX1UyZ=oLe@l~!c1QV#+1KSekjXh0z(+QW7CR)ThqX%cP_Wx_iFC2 zUJhN4?!EJ!w=6b=@TeB(6BBo5;(wz*&cZ0Uw%-2B$;@79plB4#j_eRoP2oxSsex5n zPjn2ScB^F_{>T9vmF<2|>PS7zI?S|VGwVrotg=X9F@)RO|0gjB;V9R&*c_J*&|9XG ztNzqVHb4bePUkwcXh$JfRC`3!(X2(OM6Dt8Q1d$)t#dV^-k9T}?*o7T{#xxc!hvpl4wbUqG|G1*yp{ zhgLn$Xg%;{7e52rd;o?=K|0x90RGlYca>4n{Ev8^%)Y(y_yum*3q8VaX_Bjp_39wv zb~zTw3=iF&+GM@MGlv~GmTo@PZX4v)uo!*>Vg9bfUY}~W_XD+Ek-DBl;6NR_$xjB2E1 zg^$x%?H_=Wzf3sL7m0LUEJUWNsZH$lLHb(Tfp1Jkx-Yo2Xf*=l{-u`lmx^B%?B&%1 zZ||HaMT znp&v-@ZDA=8NjzzYR?DiL7Zmt?>iQAO$eEySd613XtiiE8-K+Wkr;b zkWhTI&VdoYdRZypdI9y~=h238cw_#`_VC%+$9Q^JeEONJb{^h7g{jGUPgR-uf-a3Y zK?Q11Y{4NYi)wf_2{JKftOj}_l;YuL{*-zgYT_~-c?^qSdiA7NfH*OM$lzN$of_wu zHMwngoXBX8fKvfAG7tNpc;}tszJR3Pf2INhoNErJ0n(+^+uqmq0d6+`hF01QIGH}J z#19@m9g7%8s?3u?^4M@Q`_fqm*U!0 zmNX8BOlf+0LO?8kf{cR5s9M4L>sDv`Uf&Y7kb*4Zb+aaT%nYt2kr2UJ`>ilVK;@~u zz7|wTI?c|AV3`h3lV{XteCl}Z{IcT#NUH^5u*+euGRm>sNOO%#U`=4AI<^f&$IkfF zQTBxKnts;n;o+m1XbJQEC<|L_&QXI@!5i}8pQ*)$U*21XgAuP^raPM84SHCsCy(iY z1e6V1cC+-D`c=u0JkLHnfd=6gs@Y_`?c=^b`@X>w3}&I zt-TsHt()Zg?B8|Q?kop@2?@2d5vFM3IpmXpVpTQK z%w(F6fSNr1I@5kM`FM^{AqHIMJm0|EQ!7w=w;1?z={j-bKN%F!>+_U+nI?y#iE3U_ znS-y&q>5-U`dbxs6VJYJigEmBp)+E5Vwz~u0!7oodZ&HD3|4z?W#YkRqRX7#)>P=4x=kFgb;ZsNN-@uR+dh9BwAfV6vGAdsMdC~0n zK`wmSJ9s`J93sX$cuWYAOjVs1g3+ZW{5;GC7Tq?_Zr;C%fxm>~blsYQ69|?%7uIPu z%^o8d_H0&;??sIBI>}}L8+<2TSugg~r6p)F90H54v;Poj>eA@<^}#!so77H&Thw9JZw4MP4}#=mzI4d`vVrlx2eLcs$MRu1MT8S)nLeaE?5 zF^$2vWMRP4veZ?7(Lvsgm#W)%u$_Q@Qx_asIe?of22!gb@Zvi_dpc2czYC*L9nDSx zL<8W`A0YFkZ);=|lf=`cK0zocVmd|yHY^@uFme`=Wwj2O`arym1KVlsha*xUq;R0n z4xT9~H^82ru(*GsU9JLbJZD*>#O+iER>PlIy#Ycp%hklvtv9oJ<%txOM=XiYmD=nk z0~Ckc7T|>ff26ussK5xh{YkDjHUG0DjTLVmLK{yl=bH5^GcU>e0#tuq3Fat%)lsm` zvh*|8;fQZD$AZcB)@ng|s!e$Qs?du|coL1Q-@FucGozPx1lAK9Vg;w^nvkZi(yYpK@mW7BuHU#dD1!e7sJIzM9aB5;ov#(w~R7K8XO!(9H(mwxcq z`~O|d51VLQXIaZ$0cZB^bT5xS(t^U$<|zmgGXHQmLkKPCG{}ULry3we{AxZ=FK;!S zhg-pjF-8WfcVG2$rPr2WM~ULVL~h!JMfUy?shhX? z>z0__9M5Xzy1HcmF@Zg3Wg(g<<$F6PA0o%^OiM(QF9kOtl6mX(wRVI_sjsiv)*vck z_ZlRa(e7o&V-UggDC%_cgDsjTHJQ5*J4zq7&|V6-z!@z`@jK%2Y$d;#kiAceU+HR$ zH=5aEfkq;A0-Tde%<4kA&I?IH@j)x%A*bhF-wNz2;QXpTr$`5)eoK5EO(}G@^;>Xj zPhgm6!h_pUao;t|e%9%4R@ZPG4(k?(v2IO#US)7jNm!(wH|Ag%4ngTaD_;1_YxRrJ zx4-a^ME(%boE98q3miQ2s9V{JRNW5qyt@v)=vqF1Q{#V|y&tlCy?i z%JLOGY5Sd~ZwDq~Kjh*)f5`cH@F@T72tV;2u%b==oJjt1?>uO7dpT&_H)I??&+llW zdpGn}Jl}5+Wqy7`G#<3>+lAqE4O|OcIzQ{ebmdE-y1On>+MU<37N=L@%eRZm=iLP?Rg|0BUw})(o1@5JYl>`?0yjeDEG5*0lRH{I2iPxc|CqlLBzU4Y_~}~Kxdi~ z{9eC&&1V@$O*BT()IpXU1cH5{^jaL**dWpS)UikR{i8R-R#O7hptwEQ>jZDQiV4Y2 zNx+1IrBM74<$4Y-e&y21?R|AjbNwB2TM>GF9(wtn@un=>rERTlkn!?#`GN!XPkQ-q zk7qPwhaGb~Mvn?z55=fbMt0rRB{k1ibp)Pafj*_mpS<5s__)CydSlC-HewOF#UHYv z(s>R5y;e~^dynzn{y9jxyYLOYvkh%EpudWczuL99O3Uv$&oi7Vf1)jWd%=HqG~TuU z#`P}Cx6?Mj=14r`I795J*y7QcGLdWGwlo$9b21)^0q#X~p6!oD?owN5qc<*0gr3we z%G%y20M`lCR=-{z24P0StNA^wjpo%n`r|CBqabZAavAeB(k$s|>?K7M1%|jvnVE^! zYtFW;`OAB4m+A~L@Fi+`A0q!=Z%uSxKo(*bse4h@b=!B{b!f7DHs$L=y?nM;7~1-oZ})DQf$!OB^sl{rQdmul5h9k zJ~LeK#aws(j-Wj)uVGwQCXzn|2CSaT-yPqd#i>k0NfjYIa4EhYwl5rH@+5>6JdE5pGkr6~C*~M8*&1Tc<9uCx;XE|Ni}37VlqtaX7k(;h$&{)Synn zyln6LqbQm2uAD}kQ6b(@y7K>gT=1WxcO^>n)(dJ91PHM}%b4H$5`iy1>4Cy*DgeCq zVur2$nnz?esXLwSz|z4#)c8_=Bu56fEylj1U8_-#5cG~!x7buRuxR^B#Ra^P?RK5? zb_igpTE6{yju~ndzx$PhY2RO4x@-1z{`Fa8ckVtUz*Yd?n&vqzKY9PUgUW7FJV41% z7FlOT`+dIympH*JWRC|~`=T7~M}$*}{-BD!%X)YeV43Ao)odFyTGjt*SBSK41;Nt$ zUySKU-!CWpjWC4sMII^WVl)3H(oFh1C-7Rw8JJ2C37k!f3#pO5=spfYi(od+<=po= zGE94H=e3|6Cy`s{rx}jlcNZH%_w#PompeHC^XH5UxF%=fQFncRj&0vG3s!R`J1*+^ z4MnpwCiqNa?W3@uG-B6oMR#NE^Y(T<6O$_Ac}{ENJ|pJhYHt)d7y+l7=WJcFE3BO~ zDL0;f`uCU*Y4_^qH(Vd6waBX8w2_%J%6DIwz`kkTb*9TkV3%G*vrz+gf};4vvnJ2M>TX_7PvOWO@($6lCO`>*wn%cS#}A^li9rVQ z3d3T+PoMqo$ios&a+ESx4RA}FekF|Gn+p0 zXI-I}J0FWsd+n~{d$;n}MDlm}7<3XbX@bmok8syb1!f$78MC{5lCT``9;^K_!S#AD z^a_PJt+Tk=@@)@vaQKsdx3)jE*YA*3QY8v#H!ZeRJpD8$W&edwaeXHxvwE1G+Ezbw z6hsD(vQJl8a7H1V`%Z7yT;2d7DJNxAbo99ukA${_E#b2Qepd@QFiatR&5&!}q3JfV zr*@kUH5g-0j%Iz*)U%fhDYT)qrC!rhTfFwSwy`O>H`n_MCLw)~Q`K$S^>$r|U z(i^~B{+sVyrA_^1!L5m|R@2!;KXB&d#O3I)%nlO8ZbJmA0WeEQB8fue6bW0SVx*x2vc6Ytp(Kf&1gKBm*Nyh~h zq;{TG>|>Vp?+yYx_kt@>kDOBi54yEw558ho)R*yxUXIp$SxwiUO)YVj0ID<2tmtM$ z8JE3Hf)jptsHwkfj_JyW5>YWap7@M^Q~}=&`Xr`Ph_{*Id+>qz zQQt-H<(A+B6car3Y&|qaFyPj4xp86@NWJDM?{$SWo@6E9i;lPNk^*?M{epGZ8-xsy z{bn6D)Y#68^0;<++B;9oui&?qKuH`Sq4jPw!F4g91(6wFA`8E zz{fF2d^kg6wCx1VM-21_#@4t%&(v16Y_-(VKbeiyPtqKVzJ-j|9Y_92SF_fxY92i2 zF7bbg-y=ZW3rbT1Ui9owq;l6}Y!3%T!0z=N>UUJO{MmL_*F+Hz`_0GSpNq_^ET2LR z>W1c>#|lxUiI1f$zi__yBl(r-ec-^UprdVh7( zZT|0<1In?9*0Xh2QKx|?P+yGpB2P8`P*+~nks@W6fv6CV}Y0UM(MW!7JZ2lHIJO13anRwN@HhJ9zzRyf~s8ozD?I_pU?g8?L#P)uqJy3BOjG7Rot~>ldNK`17hAd&0|Rc^K3Z+?bF1A1cw*<@UU z^I$W*8pYUr6ijklq7*I=%!J3(1Eo$hseIux>2Mt#)ofVN>sZIKd>*{K@53@e5);pA z9sB`Y=!I{0pZ@0pdD^@w-SkuGvzV`sjMGo&WVYf&mNpcfAA+f?ZK*1j&TcQUTMA9x zHP6$>GYDG0H&==5PVi`<8PHfj^B9CpG5;{OpVy>iNm3@LXurl8mixOfb`=Nw+eMrT zuI&D}_2GSWuhF|0aN(YUnxmhIpQAl!j(PTqPiHHe3M5i?uFU?!CWGS%E_|&dHZGda zt&;Y%nuoz1+n!IK(hvR@?u$CkY48aYN}SKJ)jJSf+_#CUMOdOw)go{Fs(cCz77eRh zp4zQVI`OOUt>eFWG+q+{CTn-34(ZKj_O}D)`d9jU zg&RBEP$yc%>BVR0>GO$iNo?fT?16en70cQs+)(A@P-Y7;xSEu+EYbE+>Tp)^r{aMPU6G;dY! zLQg9sTrb>XxD{bBd9G8pGXCwf)MgIB0czK}Qs4$#mTe>@QAImafxag&Bmv6ceJ`)a zkq{wXlvF1#$MjY+i)<1w){)9y_e~F!+7{*5uMk5V?i@|&_}Zzo$va85)6s6U`BdhK ztnj;TnI}uf9*R0yxlNkrf8j*Kc2vUY55QlW0+^U^2s8Vdd1`cKq(p+A6xWBo{;SYx zy7&(JjbLSJ+?rnZx%2ooB%IR<+$~}qPk>uJTZ;t;QPw)6y}wPoUl`jh=C$hXBLG4Q zz!ES5f3j(i?=nloMQF0qpYW6#5wIsBDOAFerw{zOkVD`8yWI`^Hj!H(wnv=MdinDz z{}Ufg>iXr{kyboC`sC=cuyyv)%c|$_SV!+M{0Cq`E=pw;Wfz`wA1XprY(LrfvG|Pw z<&k7Tm}gV8EDm@LXS9UG94yo$vs;_8bnbMq zfY%_9<^0&tkRPAn!EMg|BdEG?)@$SZ2_w$p>OyQRSU4oB#L-x>wcc}dN_uq7EV3tN z{gPHz1Oysd3peFE2(Hlq+?fVBS8?p0_KC(aicYTHOpi$R-6&ao3Zn|upR*g zh2b;a{8A>>0g8SBh1htzTVW8P2$`oti-8r-IZn0dh_xB0MAA;?hC3s{R%oHoriDc8 zsf&lE^J=mGq20v-O!=*{3?MIn5Gp(<#Lnm6Wx%b3)n-#FUV@+?amC9;NiiSK!Y>s@ z3v2$?=*IicueSLZb9T2{e|)Rvw$lgim>B1H^e2meI#o4B^`JU;(oEW+=!tpNAHO_L zv5A{E_zW{MhxXH-x-R0nW;S4Ub2gv;?_(Z$`fuWwtbHydBLI;EGH=Cw;ipc?c>qdn zzvA%mhu{dQ8`P4p=D4M?wZV52)c_qL9Wl5RTO%VR9P{+q?`8fPlAs4bhBq7Bm z{S?O*2)vuBsUkpdYzimHC?OrYyhvl{VB)pMrTa3!OS2soAd+V1`9!OXQU7sIPHVL? zfsd+i$qYEy+uJ5X0mKyE_Z4Y5E{eT8*V^GRSY;BNM0s>4m0EmXGaf4ip4#C7OoT4u ziQRsCT^9eu08iEUr-arNC(SqnUNWY za$ZM57n^rGU%9t0X6<;Dv@L#c@ihl?^-WG&lUiY02@g%iM%KaI+jXGh>n#`EpPlic zu1m`)k73m;D$&95El$+jW7EBn0ilIU`;i1irl!so_~34)7oYV#C+p>35Jfq(I?R#; zy)yYXV3PzWmD#x1J|kRHni2WKhi(OkqMl2iooLa=m1=XK)8MhJl&UTQh!< zI*`JM>)triW+fWaySMC}4ftM~_!vbB_2Bh5&dXl_T@JHpPr4SPO19y=%gf&5V-=ED z>bQ1PHX_M@QAkV8mSm%WYw{ zUCdGg48(Bn;RqgZlxHhqlndp;23A9UftZV`V8L)eUS^K3hkSViUYtrVR!$JpR@oIU zZF^A+;I3z%Zi5Q#X%ejrxD~6&Q`KF{HgFg?|K`HG2cXQ2WmZ7bq;dPcs$K!ant{hTtqJ%n=_44!PyOs2QGX7f)PW)!HT1A5?voUANuYb z5450I=Bt=UA#Gbzwokx<15oMaj$#Aa3_YE!Z{O+5rjsmpUZ=WK>C&iiej-=nWa}SD zf--}dl%On5#&7J`M%HAlti)e%Njo-~m26gesx<2hZv2E?xU8U7t@P9;cl%SGe@`jR zf^u>`P4-X&I-%gQ)MQ$j`2waeFzG`)RvgsB;k8P7lQ`+-$pK}e)grd|n$8-aA~+|i z0eBiw5IkO22gySCl7S<^q@(1&ChM;60lW;S#xu`@dJQI@#DI`}r?uPJif{fBE^yl@ z5>PqT>P7Z5H#)`q2sm>!CyLNFh(cU0)u!SU#_sPIdA*qkfHw8izJI(ZvFY-61R4WT zN=#|l{d&yZX(;kN%bN5*!lqa5L&vdE2ol0_ml?2OZ-Hu?RfBTSen!Owe6m9Ok*bwGdOv??j<6M6ib6nRR7=24oE7su*y!;9`iY5di$n3a#DJ`w2 zy{v@DR35lN+tv~QL)!V4C8=d(3}8f z9Kom|0gtD8pCy_EeP0t*#^bF@-O+ob9PN)grJoiZB2%)Q+ApJz~K)w zWUb|Jvf-=uUr|C*zV=@twtt$D=)K#TCiG?r`rdGxN0G<*05^38~KS}zxT#)x7A z`HGv_y!US11Pr85Go7F5^lf@|bM|bRRV zVazqpPF?L_QRg$OZVsbewMqzWX66jZHzr#@$zIZ!eDQXfmL1CZug8kj&piCmb6(a` zhV*cbA$^#T0GHI6mLwG+nT>JMPY%^N)rEvrtz;i+%}Ap)Bn6mk0GZirwQkn3BH=&J z>1WQJik(f+H{1_^ix~q zRQcp#!{0%A>ok{zBfcJ*q0e`NRwVcYNf0I+8OKMG;*XrFZCLK}0%pLQS1Y!{1r0eK z)MA5}a17zKF5Hg1_cmNEo97$rsWEOrxQ;s>)(bU7(655LB)td>iOU-h@e!kWmXwYH zmUqB!LZm{jbOq?$row0wSFewEUv=V48UiJ>^CFt8j;ZsCYS4lJsncr8o#7Y}>M{0- za&gZIurc=9Ez-}crcxag!B3P~IB|`-GL85(8^Vs_R_5!6r`u` zga^$}IjCNI_zKZ(M8{1z;BiTYJh0(QX`|v9rJ*iMhzIxx^zccs(0;cIDU`O@ z=lvCUOeFLK1QN`Qpbg^u`k_eRB0Q`+VVypv;oen9MgtoKh_NN90R1g=aQU2Gq3`)* z(zJ<&(u1J~Qe)ky+_1wacWP*HbF*$!QX*lM=CG?@6L>oM2wiiioy!bRD#I7Z{*V83C37kq6O$8X4G0CU-B#tZrj)Y$;xYgqk~^? zl?OKhQ$9#QEg_B0DH()zPD9`)(F73$ETa;|If9BGEH&I

NKK0oL^#_u)(^4sit6 zXXJ|FI!+2S>J3;BtoT?E++AXyi39mYkB{5=2uF+WApC1RFE*^UWI38OF?1_BWM$?z zARlynjQ!a6v75<3%!#_i#Z4&YH0eL};--bxdn@|v-io@^{_7*~(0a$6)1V=hz{243 znzFu@`;X2ivZU3FyZk6j$NJi8%9#;3yZN$fdAMFH*=zMniuEl|+)t5j^&#oXsmk_LHpZF7 zuwt@D#2`z{S6O+^*L^1dK0sr+UUrnwX}*PLy8gi+=NFgmdO}5=naqr~wYnLcmL!Dg8Z ziRR6=607WLb-KYLS?&YpWtP^jK8vU3?kAYryL|@fpdqHes7_3;YtXg{S&zy?hMP5u zi`mu>oANii%QxNm7yR2Y41hna$OSC*Hl-NQX(u32a9X2cElReDef~jk_;bOBKQAUU zg!rJq8R1|F7Td#Lsl9@~dgXW!!A78%j>|)2#MfN8?XWnJ#?q(9RZ7`CBmVuDJBJnZ zIxq)qBzEzk4%iuriw|OQ0_=7jaW5;78;*)-_6iWxfdvWbc+p~-(eE7P)s;-hzt@M} zmWJYhf^M%nH@~}v4z3Ky0)4B$rI_tfxx+)$e&E_lxSE4P&)a{811F|klbXFDH0(U1M(J)@0$GPV2;&~R0s9^yDG6=dAFy;yjd=m*SCW18 zB%_b}M4d+k=wJQIWpWKfaal=^STWR@j!wMq?Vm_J((qaj*e%nVNo)`pC4bCe%L$y_ zhm{CF*z+9bJVZAP1fm_PeTU%c{nPg#ij5?We3VJ7y96QGcf{MZK{ZHpiLf-9Y5blL zQkb2k$2Dd*bXiv$TDj=VZY(}ZKJa{)ob^?Ttr4qO9H)BZGpP2LR5dJ+#{VJds-vQO zpYMVqEh!-)ozh)`gv8PdN_T(gT2eq7DWw~UB}BSgx>c|usW;mCWhSdt;$B07_quUqAK4$)R#;iVRcEmeAeCps@v)CfMqf_<0>&{{Mvu6` zBp!(mJN|d6a_#qO--p_P&DhyOe)mA2)>~!y-fQ{8uin+BNh*Q{^USk7G`2(i@k0)R~g_h#>)|mBZ~B6!g~6Z zP>G7E)TA?^s7)(m5vVv2vI=oxX@o$)9Kxh=iY)Zsu|R(k;~>tAQ8*ewk_j}=OrHUa zlV@hn$Qfx~LSG82EThIqao}LJ8DRtl!bwr1en^(BtoVRgdOr?Yi{hjsi^xMux%NQ+ zG=36TjnFHEC<$*!@vx-gD}gzPwXFP-z9|UHhm0nuVX4uWama^q$uP>k2z*W#SYctj zz=awwCT>QGIUub=FDrbk_|@nXnX!)_h1+ud&-attd}#;XRL*?J+cOXU3)1M2kRO#N zfV;+upmOzGk^c+;Ise5T^WVw|JM_6g!o^Q_ncn)J58=@SKfoRDw|A~dMsI+@Z?no9 z(Obdn`)&{4m9Q7=bc{@lez(AYAjanq4J7uE7?#J}a+0Li{wU}726SBK(Rz25eUl8n z2&5YauqIahc7LJT{pZQFuYUiyP;s+Tt+9*|EG!hkc5VGEA1^8Qf~%9v;B-_8xldC)q{lXPOx1|M^q48?B^J-ABg%k-H!?%-6&pojV36gqRKO(sF4d@eaL9LTh!y~ zketH%B~D5DA2uI-rTt60F73j1OXq;plx>%oIy0?q>Msm<(hL&uR_gmk%{ryxO}{eIx39 z{Y%on?QrmS=7pYfIXP5Mp*%jAMtLW98EA`tLn0mZ+zfpV@vNw@XwT74q#62?U12V2%y6vf+U0MO;5i5Ay2GOFMN_G`JV;)BLSe%p!)>xC z(_Fomw`_AAYNCkLXuVJBIeHZCTRYfgQ-kzm%5V)3PM3#a%cZMyyr_Hr@+q){kG7Yj zSo}wX{X(OM z^CC=@!Y0zl47sUDbYcZ!9n7!& zwe>}52nyhD7vISiVcqoCkp_dH6Qi>#o#Jw+xn$#Rb2{_eU5w8NIu5wFh%#i-dBeQW zuZmnCuOU^QW(?6#N`j7EQttsu(6o^lqMZlUuud;v(A6JYTf`#pox*bR2T99o{TARu zWa%&S@=SaymPIG*#$=u86c|8K;NpL)WEoV6r6g#eS5*WIrS$K|4^5sE1jl?0dv9gS zG0VB;XKwsli`#g%&a4oAr%v{;Q22Ngv~pu$Ka77oM#K-zsWIB@X)0yS%yuPDdDFS)RdUMl4s?XaIgim`A;>>9?DwytSKfV5& zp;PvsW3NuacuJ zvvu8V-N*-F+oWWoy4I=!5KMs7arCg6eedsoH=z12G;HNGrd-_M^9B4-Z}wxj$`x= z!f$-mAYeN1?x7wM(>*_e#M-&bK8SGdIEd?q@1E8_au!+M5uBZ%0#m5>_;47 zmKs1aQguWBelhZGrfs{nqrk*(Me=0Gue0O;dl1bh(E$+~0f1^9^)7~g&&xx4gJS^& zq;K*ESiH9Sj7sdj6I)f{X%K4FJq%iK3SgU1fw@VS@ETB>560iUv&MV_e#l;|8*qZ!&CB$X+C-nz5d&MvH=5c z*;09~#5JqLZX|CnL9R!3S|{xKOo~k8{w{1@kMGl`+=(cI@SL^%>VAZJwwDR&~om$ zf5&$Dx&Ry)=4Ia8fV(-2OnZ z@`$@~&;HKy)f2{puHVexo%R9*6eQpo>kVv#EZy#V=w45GhOa~2N?PnZOP-Pww>D#Y zgh?=Nuu>4Qoqzv3O0s!x_%sDHQLe;~`?&1b>35*g#o`06N76 z{Ykcvcj&;B?B!l%)5nDpA-cQJ`enYoPfPm3a?>S_fBwVV@T399K)sE``7fke`AK02 zwmc~uMiPM|z-CPPIZXwQotfEeZSS+AtdsHPUfEwHRNV{zwaFAW91M!G} zUi{L7jFjqmULImplgwk1ZBD5giu zBDNaa_H$fci`xkVy_VxEKtvpN=E!cyH1jTPBw#a6S?Xd!Z?p%6wrnJdv#kLRs9vTdSw500+K5c)rtEQu33}Y$+;+Pa8sdpkysvqe zo3IXcVj)ao0pUxbvJ}X)uDr$_WZ-lpbX61fi~kPc8C>Vu9d;MxZSAf6?2|j0jBT#aMIF?s$fco08zIQm+0#KU)iy0>8Pzts$lQ3?~Gxm`*dkL=?580g)k z$)dt!^xOGYYYs;VQl;6 zYw^`#R3bMAcICngj+S<%RdfO}DLsAZya@V?-DxJ<1*66dOQ~T{=g2EA3z`5-P8tMmOUv7bto{q0^XW?U4)7ZpuB<%`pq6OB8f`p z;#+TddF5jG3hs3NscrxtkCh0*$fxmZ2rU^Rg^`XttXuqRokQ5rwl72>Dh9S8WI%qN zO~(|~K!$WhA1ZNT^EM6aQ5y(%gsh_01pGt=W{|_NcOBtW31TxV*F2vU(k?g5n-)X~ zdO7Qw#-rj+-jfTQ2Xs2YMx6*o@C0HCV#kesk)~A`BW~-lZ>A#E@xNThP)$m^|S&89;!K z$g?mcg0;PC_>sfIKS)U#V`{B#ZIsZ^DVEqhTbVg5c=yDG-K~GUm(W;R-d-y;H(I>| z$>Yq}a3g=dhRmy`_lda5k?1msAaBBZHIHqDEq%x_^C3v@lu#Xq(Q>|Ttg>ZWx=EOS zZxbsaU}qMqyv+gc{nU_Y?9x%+jB{Sr`~=9?ePe}oH87TY%~ERYx<<&V?Zpbm(?QJl z-~BH=G9k|St<`{1q4-p(gs8_s$W6q+=%P}6<+!w-763*NZ~Y$78Dy49p&Ovtu{x>H zDtgW^No|{k!yde2R_CV>C^QG5HXo}3wYVy z4<`dbt|qoLc6|qX*3au?Au6i1a1jN*8CxKkwwo>}*X>?|e!KtqU^btQP!;3Xk(fT5 z>8eStvgK&^(y3y$RjG^jn@``@@_1FhC}K%Q=4q__EHvsQypl966J2%W4Z_XpR$VZU zrQOoilS7U!W`@~#_CCvR{Y0htLV4wuDj(win4IkkbSw@nThq~R6AEl*#GekXqU^3x zvtRyur0JJ+W2B};q?xz-Ok%r>)F784L`5U3fbE@^q;a!e;4j;9lh*n_RP5STVG*^k95`>e3udIuk%J`Nv^kdJ*U<~dek_)&=nQ>%w-(-@q9~`_}7`u9DWg8bOJspI!ae1U0A){il0M`(|!M1Uf9bo zU4s6RlKm@>Q%2%f|KjuX>h!mh^rCx#QtdX&@it)J%gAd-G5c|5tmPu}$a}9+ZfI(+ zdHKwcWsCF?;d*h~`S-YgF1lu+)U=ZCx>1b{s#{v?%L`1{ReAQ*!tu+oG&iuyIM_!d9V^)nRjb54omLsC3rKA7f@%l`d` zaP}_hZZ1rLX_hl>a1YBC`9{Xogmo@|*sI^h{?=s~SFT^%2hq$X=n+?)z2+r;EmNrn z46O$)w^foL?#>qiICjS7#=!(q$8IzU~eIPnPa3o3qeC4L5ZQk7GvMVEU2`MZCfR;arak32Ll7hJKLvtWM= zz}M}r3b)ruvPom4DKIk$j0mZ;!KY9xPEZru2x532#RF!6@r_I{|Il}-jhPv>pM=5|C)*#y+X3xAhAh=PdJZ4SICr=aRo< zpP#U+kmYVd_JYRzxEe8QlQthy?S2>2#o1Qz*qg2!-CLI_I`+jkEq@Dqaqc=McEVGi z0oxrglLJ64v4$ThmTC0$OJ1Z>J30Q3zJ*K6KC7Wxt=qNN8Fma^T-T|ubfUPc#tkHa zYCz>FPCbF`1j?i$BV8jJWBe*A7ZgkMEvtDnApF-P+)7ex@%W}XpI1F!l~=TK@sViO zZH4|)Xqb=vEC!?VbfY{cW-} za^Ej+Kk@l|X3$U(OLc#L^!_1MCrvUqKS4c-6)gs8$u`MHQ$`FW8f zJAd6Lni=0(lxoi1+P^^qXi=V-L~1iuvJLS>*vuuxS0jIEEhf2sOVWcL#BC+nOTvt4 zZStMpAyh)weE+56Zuprp7APozs}o>Rt~57kD*QN%Id6te-nTcE`W{^^n{IJH2QL~Q zZ=KqK{aV}@W0c9nz)t(c?#%f2N;q8t`YZhGuAk+0@NAMqY7t%>rE89oi=AfPW}`SxJu#@I`nwDEJ!25h8;w2v-8bv@rnO@AH+4V(9hglL z2U)E!L2@WlSJ$J9lrl>cSGZ&^gr-M#nD+ySa?X#> zbVQbo9-;iC0B)DIbB|sx8IkF)1;At7=-^xh+U1vIfR^2F8v^8RG1=Yh#}g{T1wC3;3@4BNO>%}9<4lV8Ev}Ul!`*h zg5_C~7f-M2>;+f+?tq!2b8&+GLb=}(;Jg16Cr?Oc*aH$9KM)w5JQG|&1$kaaKKiX7 z%Q0OVpAN)=!{Zzk5=F}T=qttLis!~{yCOih3i}^hzgI^PtMC@vDAo62U@q?r$q2$K zg3-MQ$7c-9#v2Toh*|I7uU~3v=K7D~c}HT#+0=3X8@Yt5jAZRxLPist40WsRUjP`~6POGN5It=4 z7&q|4KUnfN9>5!>yjpDDIk{>%{q*z-%6>ZLAfU2Qbhz1GaG(-f{S6>efDZBDW3%U@ z|8-0U9)9QD%5m@0%c6}3Xdm9?0(CbXgN?|Nj?ljpk$)*-y4jl${o_rHj*dmC)f*#^ z)rWM}1w(F^RWLM@cPV4{uphV8X2IV7v=i7V)q|dh`egVJ=R%2n`J2o$`=|{~~W))qQd`B&H8WF3{SI{n(#_q-1c`=i1=ju#@$Xqow_+H}x%@ z{Oa9w-E`3L*UinnezDc0si@zu+nVBW{^wRfU1x@ORD!#tU8g9}S>nn#ssDi;Z}wzOdxXQ`%fWv8TH{&%jCV-oK-}jBW}8XBewsdLzt3aYI(PfHI0d3{W1p!t;B=~Qb>Sm( zrqQkCb!LEpxDZy=awKG{T@4U&T@%*5lg67db~i)2TC?6@{TWF0e2DK>4rlB2Vl^=B zdeHa7rEmavhxV3FpYH`+3_z1r_W#~@=kyk<5E^pn27pYX*i9Ekm-Jq?m>&EYNkov& z+lk`p*|f`Th$}Y$1Uzh_WbKclWG>h*4O^b0NrqFm6<66?`K4f8`scmPOZb&=hP`>m z`V;WsfKD6_@p6`3Z*dZ89~Nm1rgDVt-1Fl>6wv6^_U_L-SuA>Y-8>|*DtlNjI@>-D zTkyXvNSKI%Za2HiQLMQo^k^E?=n{v{3Q*2Kff$T7YVRu-hMu%6u~0VZ;!ny5GUv46y8Xg+@Av!eao-JvT!A~#Zqfzu=VG}98AEw(b$qR>7bUz#0Wh$45=I$eW7%^hJAyq}9?ytzW2_~GpkC8D+ zeKCuw*|xt#L}eflUW&ovN{{}feZ5gx3(T^S)v%5#`D79isU3reS`rd-)7ZKnM-}eh zXv-Bw@35Fxlz#@poldj4d)2O{-;1_uSuR1*qoKq%$u?YcxbyqTo8ax!A(JKUD{R7M z@;)1>A)P!o#rybuwg4eyb~Bh>zvP!ANBN&BC1O}y=;mN;+nh6kH%S3#QOL|pziTXo z%vHGG{Vmfg5e;jl4JFGY5}?LVwmV>nSr%zIGf69OMy1yF<_^~kbPly4mg5}F*556W zxay&K2AtH(;|dj9bUp9r9L;@&~N z@rY|kO!6R=bts=lix=u`EZptp4mTsNd!D(=0T&xsk1}fVU+Iy=0P21FnIRuRKcvGu z&*B4{>AadT915aX>J2y5y;t#4mfQ>macudmbND(=Iw)Pf<&UNkSt>e$d8vQhH32eO zjq;P=;e{IbqH`HrqtDPfYbT!+(F6j?^w<1`d|&zWrnSH8;q)(QyW3u)!#~wY84Q{+ zdJbuzzg#kO{3U#qxJ$a?)3v``w^7!zo921q^24Ou?Wq4@12Jb=h(@FDf^30&v(f|) zkSVU4H*((lcGD+WeEdwCaE^oB-XeZ5uzWA^#%J>+wM!?hk}saLZIL4}V~{8#tZ z&|1~aN6N!(8tB9Y$boWsKiV}*eSXIyMzA1O(5jQIZ8neQ14e3hi-qTmGO4_QHGQk) zypPMFb>1X+T?p|JUS0xld$bD@_I{)WEx6QD4C%dtyIU5`Hr~nNiSZ#j*HYv0>8Lje z6$6Oz6R-!haxPhe@g;g4>=qxSJO*XC>-`S2~FB5-6ywhnV6eqF z%N2nT>FU{|fG$FR)@&{MNGD|-j7W(1Sl4O9c78XYE?j@Z3g~)P-6q8NJODGASiES% z`%`E;#pV8&`PG^+?Lm&LeH_rA*$Vc)D)#ys8Rp!4Q<{CH^TO;@N5Of8G_H;*FPmnF z0>h^?*u{Y5g!0B`EYME^GfXoUQcWjDp{an_rn@q;KFPC+y_N~Jo$Kr#%U~7-} z4Po(30}^=PLD3q{8T3nPh7GF+0TAj9O=j+R=t~bvobbN?NcHgY62P|#{*1ZoqeLB? z$8DKIA7+826i42luI_!%;*qIE&P9~&d7tYC&`1^C4dcP6XzP4$iPPiIVUPe`KaouFObaNZMbvymhI0q+T&eM1D8jt&#pn;a~ zopy)^6c7ht<(GH=4hA5dxS(bGWZsH1NeoQJnB9gLQSYw5e@7zU(T#|=ab4;2Mi36} z?7np_`KB?R_MbG7FoPZ#lcwd*S2g{?T`n&-ONU>ihI?lvz&&Y0Aq`~s+chckjJoQF zlj(zqZ6%3T>&vI+I+&m{9P`03eoTKy9iV2}q`81Oy*`%$)s-{LtZo^96MZm&K!n9Y zebm820fk|Ey^v)FuUDMYJot9(U+hdXqKIk&o2^PW4&u_b0%l&)nj;zi$E_5}=I|Z` zgs<(4#f}Nn@LN5Q+$g3?`@P36T;dV7@kD zHPz>S{^O%vpG#nz|B&FJX#Hi$CWNQ)*wl7So^FU|usGrLbY*Rbuird?EasqeJeZtO zc6&WQk|ohO2}}4D)7UV-_c<6`EENt&I0#V3qLql%z^pX+tN)=TCQ}FEOGQz4;!A-D zf>&W5ywylPWI8Cj+)B6?uaZaw z2J5&~9smgx1ZI)+dQpAzdBaNLv5(YSJB`sMqaw=S)D4>^7}t(yh@WG2J_<7@pIHC* zl0wApti$T7`^D%fd%eKD+>SR6s{Xf|8>%C3N2(MxWks?f)Mhp4oyCWp*sgtjixboH zvo|Qx^-GZ?aidC{hm33+r;x%SpjP6NopqBpu{GXG>R;c&R22bM{As0lI&D!0Jdgi_ z?Go~{r9@urS^OkZ0r+cvV$VfBAe}pj(Bv|q(-kMDBvw2j{@>K%BkeP0h z3Bcb5y!126#NQeVt$a3e1hc%_PZ^h zC$M=}0Q7js^NnM3vuzolI`Rt$bb!qe@NSG!q!ES@7>ayF0Z}*NN0q={)3oC}2a-6M z*TRsFVri!q4-wmXaRS7pa!)H0MaY0WV)ipl{|a~0)$$Ld*@Ghd2ZSZ3`^V+E8E@DE z(lDt&tTD=CcdKYs>07GdSx94iLs9QLIyh(Ou1Du(v4dk-b|`s!)NY7mp?Os0Zt9JF zM_jB)L_c9#0n3KlF|iPHJ{EzP!TN>Lrps-MHZBf_@;lgP-zMKONjtOGNgE;yM+=bV zN|_8-zafRGOq_H%5p~Z0u5^R)Z)?NJ5+E6;e8b)Vcepr-?+mzmFIQ?Z?i zs+=D%Xc&WqV)nhS94YQn7cOsJ)N!GLd|~5A2|3mG+%v}KkU6PHfp1*h{S}mxb(Ay3 z*AJJzPFAprcgN7{7&Bv>HT^hc*#Ta}a*oQQdHqUcJkAC&^UI%vK^I>YTWHP6k`TnM zjrDdE$X&gw-#ZLJ*}m)4Li0Lvu20q8t0%3u-#TBa-H386Y21*QkNO7%w15PCgZyVF zsnsKqnU0b;G`-#Qu%Avd{mbl<7x#TN6A#!U8bp~llG}(l`^56^CIqJGF=j_)QSC1G zp#`nCw>!UpdZ^(!U(UZ*R&S`IDnBYswpduhXAZo!zH10Qa*yO+ zop+vu6tOp``6^G-nZ(bQ#h(S%Bbx3qMUM>J97VD5>I!1&t=5HNnvs#+@}%e})_tvJ z(Yq8a ztwtRmXSi%7-deQ`RaM5oSS!`zB&qzjADw-h`gplx77I8EBnh^j6aq#l7&?o4>VVYu z7NASmzXukVtP=dBByo=uZ`Ony{1*t4vi6Rlx~`SF=^d=g-TDDj`K5BXO_eGM#|uL>h0jW7QO)`O**T}=Eo2_-)$cMdnnEpnvOM~nE)u0-fBx)b zT<99~rjzGEz%Z9?vZ_+ou{$6@O2P)Q71EqTsllD%vB~|mVHjy-*g^NKlqdoS87SV6a1r*EN3CZ+u#K?vp4zmIbTY zjc+2~(j~4#l+H;ko1^Igeho4J1y|&+&U8uL(*5$6^0|l80F&>nP8IxKzl2HUxW|f` zs-&__l4ST0CBU8X758koM|~7c6;0lG8h|z$F$wW$qbi)glD`8Wfd_m2HJ9*YJ zUpKgekiJ`~-H@Vi8xjB+Ur%F!K48HbeAmijRmH^@Y}|NcLpC8CK3OP|b_7Fp2lr}+ zmyW_t0^{VMpTVZwdY(Bt65~n}`PrArwOg+=AQl7+?{f+J3CrGaVUU49V62uXu5PearJ85m-4VJI4_O+9{Ml4|8%mf z!3s!;43CUA;UVb3B-JlYvqy{${Ia-xP85#+ycyL)YZVJd$f3gg$GT4D4~QnXU%-%r ztA-aU7DbijNQp3<&e`o9a52(K34FZ1JMq1;p!&s0Ng{DwI8ffQqC?F4MihGgqg?E^ zKl!0mw-f~OTfLJwCwklh#5w5xtLvHgPm(qh0E?NHChp@!_Uf0X{#8P$z5Qg96iglb zchsv4P7-)e+2!L$L2v<%4EA~ zfT&@1p-2tqX@%;c3|Xn}KixG!8Ao?z7KFj5LvJoJkRhwRQr|J0D=17cuHZ-lc;Kr) z9Jr{HRo@zo9l7O3H_5n)ixQ|rJx{SGu}kR(m^HvGAe<=WuGuuaHj|laWG!Tw?}ZI! zGwM~AZZQFqdNwFLz+30??59wPnep8)n-S~gn&ElR&9=r_v>}<{C>I*sIAWrW^i1MW z+<=%J=S_Mt6*BDHYUQjdd~FV%O+`sgE`tpU&~t2UTsl-zR(71@X=`Z(45|#?J@t`c z%n^kP7dL)%7H*ljA(9_ypDo0+){5LVA?jPdz?Vc_q~Q`uDEy?g*(nktQwH+!e61}X z35Gthy*m#2GF4jt*<*UKGuH>6JFS7;w|$4;Q>cD`&1>kBV4&>KuMJFRut!%;98(*I z-#L5KQPmT%Y;IA7Npgw?Q|nhzRy_{{+pX`75-MKrP}VLP)w+{E`-qRq%%6UtE(Jo99s@s^J0hyh&DXj#QSUgRkSGUT_jwEQ=ZW$-UO-kE@s`^!-U;sj#FHS z#e7jk#MQt;flL1dKrx|c-I0-SV4Aadix!OARrP-5Zh7I-|L&08%25jo<7bw`1@J=+ zVu;jN_yZWKE1%VxbzNyU@XZKKSu?69T=>NCf&xrLyPb)^R3kTcXKh~? zV{bxRBo4I4YMZb}NP^YACR52`PzRTp2w7Q@L1>sdujLC@B6CNF|82#%_pMv{l3xA& zf&F3)&H;J7<0f*`sYu{hsRjbo%-aL-Q5tj!pe@(EUFXZ%orZgtDzO1`a;y)4i7n!f zyFon}@!^{#A4PY$4&C)M^X>`D`MZPe`0V$W0F}&QZVwNSAS%LSZu;`H|BKmO$tYp+ za<;zE5;Q&{!WnI$dYyF zMZWWWM4uxd+Yn{F$@~iJIGhMND-9#jt#0K~CQO1^oux3q*%Vn|s{Hex`h5GY|B9P} zKp0ZOrn5m!+JNnlzttKL1HA)*%81)*}h0 zpE8s5|1whoJuf1p(eK^XJ^>eX6mEP&na|xc(s9|juSC|ds40ZualTJ19aC>ikWr*n zrG1Hr6d_#O*+KTAl6|3SrSu+z^9~5H^izg0!ZM51XC>YdP6Z10+KmQ4c*l)Oj zI9&IF5e3OEndr8#b<6pz-8zxdr~)m+I(IaBQfDTIF=MNtU3p7!B|F^F{;?3{XX)K11~cw*SX0e%R|u9U z?{4u&dVw_^!N|vl*{(<5gUUvg_lmp&!rDQW_625MUnLUW;re`hKA!tZhpEYROhDzW zy^4C;Nfg+*=au;w!Gvdl){&-JPQE6@1rtJ68&~7sc=1Ce2><1Lg#mpr2`ZP6oT|(` zOapI>KL$EsC~GFtZz6ypphPFQH{-wzuYb{lK%rG+cMAO3K(~?}eXZcYNN;gW3;z-H z*1inU*e2>LU0c)abl9qs{PreUqk4=DsOZPfT})$ZsY8_RbY{DGG$8Uh=8P6{fxCX~ zs+@v;pTP<&!W-F{hC&n3-U(BaHJRqEe2Oe16A1;Fohe2g=-LV_Xh&gddJjiIa6=_k z9x%D`hD-!QK|8)oH`id&?rY8wGBp|vn*0QChJ+)CKn5*7=bu2Zbp+bf$~ ztMODP0cKw{{C%)+6vzBl+1V}ljD^vsaIyk>G{56R3Zt_ocEO|JSkLZ2rwF3S;7Q5k zn%!;vSv;M%fp;qWhJ=;FYf@XwmB>6p^33{QYnts3I|n(R|K*WeDAYB<#P4k0SlJlS zODWL6%n2dH{BbqO9Cr}W=NPRC3ciwr6*4jx=9j5}!a{Q5y|{$uHk=`3MM}k!xac3WI(GePq7FyFEG$BBqRa<32{5@LPZ|tUStE-+jvFmu zw?PY^iDRxj3zhjZw2gVp^B=!ZomuPqJ&H8JRyeI+RSxIP#q*z#ljeK43P-H8t ze^euBUwldojteyr%7TlGdWw^|WFo#QhL}UL4*D+WOT$d!@>v3Cm9?DcjT+I7n4{O& z1vwu?8_e$NwA$9&s2&HhAIc>FF`UCyvftK>`1Myu-Aj(AghTU`a2Cyd+TPX?B{{Q< z9^E(T;`GrcZ2_hsH$JR-hh+2o+gfkBWZ)mM-@g`&JCp#_muEz(W6;+&ai>unwh^Ni zQUzZlOj5Vo29xw`n^7l)AGtaC0Cd2IUd6QI`d7Okgayq!8^+rjny#xdqtsMvECsGT zsJ24OZ<-A;fF<;s3z4=T7v}-AX?k!R-&zLmXg;A7m_w*1^gFGOh;8N`ed+J2cI{<7 zq4xTq-FV|+SC=FUotLf}X~$=;Jw5xJ(u>a5!gKSI zcXQbTrM{qjJMy8DLb;}U1|efrwsZu%ACeKO7s71>s#bUG>E8!`Q+YrJl2-^-SkMw) zpKr9{V|ucBPp&Pld6&!$TECw3tjOzlM^#+Xr@IB{AyBIZRVMhBb80O=Ykcw(N5WBukNuGqbHDrLdd;Pv_%Ha_{ zd!WMJ2!p4$@(~0Eoh^G|0I_06a5ao0-NCmk^0~DhfK3j_XFL0)`aj@H7lA-nPOxA| z>i5)yY%$@xW8bq?U&??j{>fwe;}R;T2lz@Me3ZAv1@B?pgJY?o6jFSD6pJjkKu7^S z`RgY^sG3)Gq);Kv9CJgn#rI}zZdFqLN4|pVFEkKYvrqqM zdZYjv4Twxv7-+twFb0#f102Ycj-I!{;bB3iSuGmkUw&v~_7EO-HlkI41GTh=MWUv1c4Y{fov)7^-niYBK8Sq+~4~%XE*y4wQwHcHdS1Z z8OiS^Z>NW)BZOjlCdgm^{ZM47cbH1p`XZPi{`v(J_ke9G5P6=%0L;;hYK=`bV~OzD zoA0t5gR*_faYu=jo6JoCE1@)K($gg{gVWr~r@u~n!pcqg4kM5b;Lu*EG?YoSrkMB5#tp zt(FE)?ygk3y(!NGw)NmYa8lztSVG($ZI5=fc)EYvc-kJYPC6r1Puo=b0}5!JFAfri z7I(psaQUx1`B((4RTdn*8w9kxSY>%=@3JDI4m|8689P`dh^AFhZdP_^iB0Hw7phb7CF9r;aW z@(9kGp~^2SE1xLaJr4eSJ)LgZ<+$lD-0!2x6us{mfHUg4s=t9b5Aw2x=;v9R{KW{w zuWGNlsGH}=X8Y0VRP_F$|2)z;O`c;qS7cI0>6lrOWt$I{!BPI$836g9zz~vfADCS4 z)wuzUgOS7U;}io@i~~?dJ!|l-3|7tuZg&55GZfg;Mv-j&rs(I2*qSe+Xy_8uR1|fE z*V#mp#&tc4abMj_0XVdB_iLs(BXKlPVPGL&0T&VE_ZEQ>Ha=~@_uZSM{iZTe!~P{v z&BoS*@;OG(Pw6>d@ZMo-Sh&!47Ir_W(h`GYF zOeol_7ci&l2()_1UsN-DhF*1snU7pm^dJo)-3UlG5&{AO!q5%Uozf*C9TE~FAl=>F zA>G|Qf`q_(`Tf^=*WwFb?%aFMdCpV&*;~TCp~I4JC13UrY^sY|QU}Y= z?DMs~zX2T0c0;d8E@3FOg3-X(kXk4}lmqDmzubdI;unhb-%R-aX(+r&RiXzN2)iNF zVQ0;o>6ORdDuDV+IcgL)zFb);;iE%~UL9a}Xio`rV9;w4{ej6F%z*X_-QN}7qV zV#N#LfBv}|7SE$lRl_ju%*H0w$&E~t$<>jRXyuir4oDo=ukE9T{tAjy#b9L5HyWeT z)s!chj<3&jCp7;c4O}+7xE0u8YdL-$Vj-s5Nj9jGrD}TrRu(*pIFj%az9*$3*e4L~ zRT>0Zf<#>MrRM+OTu*&8h)8R1O43mqb2&#riO?a;UjJFP3vZP|H^oGSAqeU zze%4cbp5NF<#BymB33ThX9#b`iHfgf+kh;VeRtK;Dv% z+W-nmNAq^Jw06fRiL>IDLL61KaQ-)TvI3jC%*8g(e%ACXRV>h&>Z^1S{>!6e`pg%( z94E3YaK~^Z)uOTTu)}&nmiqcWk*o`|OiFt7lTU@?n*`jdVEB58$1hRzik%In7@ho0CRFrQRO1Ls$A5$WSLck%{oKLn00)b5R znN=J~MpPfGMC zUv6-+#%jd%feKr~s$rw`B8d(;1b_L4o;bR8Gt&TuV>Mky_gYGd8`&_4V>C$nq3~Bi zfCE##0i&h_Qni#PlN5v|Y+UuP?i|${qXCHRH#47y@7#Ose&~9Yr=+|NwqLUrH}oH% zY@8dJ>y)eg9Nb0gjScdg&k*GQjLeKK8d(Q>G0uQ9CUT0w1EO00 z#C`n5cVnN}OzD+edPw+>&n0T8qgc2gP`Kd}hC0!h&s_m7NO?Fa!sWy)+`Jw?u?=ycU=Vp7%_@(Cy)7YDqnx^vaVb`W30- zs;+lD(Alq4Y)?kT%E$bxx2EMZ$d&F|6{*Wi`7{Dr6#eA;iGRP^{3m}nE#}86!2$^2 z94AiP_2Q5**t~uG;FI>p@?Rk^i)C`#gCDieA8nC-ar7^M)f$xqir>W%Ny@b7x)nO$ zQAG*zEA+irH}~hNbu~>J?ctC46hW1Tj?c_QD`=5PmDew^Ahd)olk|?Fov0QT|0hhG z{cjyc&^U%7vmK3bI8;)l1Fv=KRo6#3qz=^gT2jF%2FD9=#TZhGied4XLbHpyY`!D4 z>n;AteAwCQEL&~EQl*&C5G+ zgA{0JT(=iw!~CAT=n|;Ki*L=szV!Dfu(EBW8lcC^mm0Kx|MpCtqk{|A37%l|y{Y(M zzX4U$eQf3UuR6wa$5wL#6vJWZwQYEYF_fD`LQIso0Hg zo{&$q3r`-TAqB2po~W-73Mf~Guh%P4C$|B;LS~s4>5m19#4`eiR!xWqD(Jrv6EqkChwnG{!SbJEdP{+^oN8V&abo{ z^W=L}yjY98ikSEToM>mx0uicA3TiK<<;xLYE1rP>ia7}uVLN`fQHa)fZO!?6;H` zCtgwtT`q+O7xE}sq|5ite!&hD#~P{}vb7!Pe2?71Vw6ysFzA8~h3*DY`M>^LUT$o< zxGte0-{f@v0CRq5`Kc^(`U?rX=tTaTBh`S|!{s$6`Oq8IjcGD&^BhDP2MAOtGGa8? z17B>{B?OxK=!_o~=|FzAlYU4v5?H^Tg-?rIZClDT$Q)Vl#ld39=llHkyJ}8-2T?rH zmSpJia4nfTSE;b=o9N9gAvd8jVI!38v#^PGm7(<{$?L`J(MG;dW9cZW_h=T*)exU?#9FLT>%wK z_+{Osujtpen;R0-^qPx5-XnfwAfW^jQZZA%8nck0!5i%pS`RZYX?D(uQA{93eCp3& z7r3d0A;EE#a_!NW%MYz)5JzJ5rRGOt;d);odud92S#XDq`TzMV+|O)cM?R=sPA z52_A_oOSyyt^6O`qbFE1kBx$;0G>FrpAYl@fkdYd`G4$oH$(-orGUXP6 z{jOc=#r@sXi!Z0wfY8MaCo=CDKpGeScGCJ(9}s@350Hht69=thl-Wo| ze(q!2{t7TCIR0!h(Ex$gR_U{Rybn~!UgV(G;*1a_1#wcK4=!FvzSiCt`WVynTqhoV z1kgugqHjC^ux-6!NMTgX0Y)ETXZ%svU15vp*s1%-&PtCgU~1l2zOAXNghcfp*c&W( zjt;4`k>&p3#)37+iLr}Tu7(ax!W<04?CLXPP*alk#Hd&a6C5~q(387*&w4CgRWvc7?kcvI2e1(#+OD0x<^&^o9`*lYe^~1Oa<`YW zNa;Q{%kD-D81^j3lVOj(Cc=jm|F{rB>2L4mVqI*tG5LEe*_1pCu#3q3;OYn>082BLkuEalVkvAl~f#~uG}cQ@H>Hx?m^O_h{p}x8ryzcE0pn45cLft+xUp4MXO(7*a3E=4X3xasc?i6D|!# z$9TA2?n?BL?)tRh*LztlI3J^pZklk>vws1*2Rn!ZBbSI`mrRv7bLk%$_<3+HBm- z!y13YM7ScNN9w?`IlS&$+;C(nv0EzD_^;_HT15J93#fU6J`GlGi{MIFX#L#R@|b;W zY$>BaB$r0MO4yQ5NN=wpi!3*tXV!HFQaylgkOa;WwXuzg1pxu3$EWFfG#rz@s)Pd? zf7h7r>B2VirlGUDo0kOzw?dJMacB+$kqO|#RFpOuGQFMA@T(bsh}ZyA^98|job=~h zkDT9jL6&I%(lG&Q99ePrw&z^JRM}x~CqfM-;FT>npXK{m3pM`FhsbmSBBAREfwIGq7nd(KzbDx78bBX zDX%2|jf(9vh&OZ~+euT?b2Ge6)as|3>(8cqub)MY=2zZsqKJKuJN1_Qp9jYxV_u^k znV3rHlIYVi%e&g%%>reix@YML@bN`km;XogO$EdaZdtSugjjxxe1cJlM@HZ+n-btz7bz?lQ6t zRc2~^Za270KjmMr2;$HhBa#E){bBr54omChj);EKq8b!?WslFiR$w}I`&TRL14dI3 zhD`kWeb^=-Dv9MmQ$d9;Nndn!H$+pW>&`}qs!xfm498V!AjF7bLRB0*#{QuYsTWH7 zYOJ!jWiIuLiNn#}4;_c}4P&<&+>nlMyC2Cl43O!W=ugsYGA$Uh6Q&Yy4<;5)E=Cu7 z?~%q>x!pD3Cag7hp6e4kM_rho(i}R*^y-Y;=LKm;@zuc+)wfkHnBo>Cc^dL%r^x^6 zoy5Z!%_z_rG|YT}Om7(-O^Sq+g~*j4xy)Gjph+{CvW}=tY`_!_0YM;K-$iQu6rfu~2(Od^9c=N?p)Z6{Q>FsVhLM^t6a{XXVec_Zf?l(|W+9NsQ znu||gq4MvDmzX^a*CkxGg10fT%#D)}1!5Fg_ABX&^Id->t9;}U7NzE=r7m~Py^laa z2y5YA1%&S7L9E7~j9|0-Y63B!UNSD98YU6IC}G`JFCh;JfMmXXk;fmCJ6b*Sh&p^V z{>8CG-D$57M=YdWYE6S&9ybsCHs7Tenu~OO@uySs_3qEK5|{Vv8BSO2O6*!mrX{0j zSQ=vGAX7-b9xO@gW- zOd?q~1_c!6o)!T+#Ei0<38vcMejzTUE3f72-*8l;k=kz`lI!Zdci2~@F)DFDz<^CL zgn>$(hm?(k9@6|V^T0v5C^E_O#J?x??jUMFBaTikv!Fy5$3vG2C5ZY!HOt5bOM16T z;ss50(VnQ6bItXQGeHMNE}@K@u*kPP@V)_0hDnS~=-!N!pFLdrIYIXJ;9*I>xW)V% zV;Ol0X_@aH5+dGfbH`2{F4L1Kr_pz|BZePkbR(^BKB6%gr@^FD-{;@Jk|?9%3i+G7 zN?V~lmg#=m=ASeOgpO+yub2oP02UG-dskN_(&ca(>h3?o0Nb#IJ!Esc%zCJ@SRoS| zTAYcQ5|80Y%{Gy|yRf}I9UkKOXOD~sY`7ZuN z0=1sku*XERh}Ic@N^%*fLJFLA5^?&}J8B-Xo)Z9(0^AOj)a@A==p&M12AF&OTlIAkb-8aU#3WBur>7 z-NRkvK+56eZp^BoH@X-EFCtYIIU;E0a|Y%L2-IuA*u5L0B)SmER?a=SC)2wduO6nR zFomT8&}T%~2(+D#+_4Esi7rLwu*9ooejHtb+7OHp?$XGaWK#vbyYOaHIx(u=;%Ad# z>m7tzDt1dVePkYTWCN7ouf3T8Dvtabdyq9hASs(ktRAN9xXU8>wng1T}# zaQp;w7Q=Zb8A@Midgz?-j%j<$YoQ6jCIlqpBh8)uCDd8*>+Fk}a07|$!)m7GZFDbmYz_k;cW*7p1v!zOQM|1vXJ)))wr4nCiJIJ z^5oNiVUqx6thy?xkAF=X<`4)81d@{ySC5x~h;OLynV1si;Z-U4D8!fo#>rUmWSliW zc!*i!T+A3zK^BDo4~c}UKt>lgz2`(-6^U_zs_MJgQucE?Wv7zrwGG7aqGn?hdl11V zwg!G{dW#ngL5VHja1WA&54fkkXaC}qd6!-DLEg+G1h@D7(ulhk7fIXxgLx#9HZ!~i z$T=9NQ)|-iWGqc1EnT#qpAayQU^;YcvWPUT{qYnB7fLqgXkG%`>3v6IdP9{ghdG8{ zEI$eJL6h@6K70lc zN$`4zvIEp0FG+O>X0~hF^2Q>!W@4m%B9gR4gt+_LZ8vj+q{wgr3r7fuXPILV*vHCR z-Y1vO_kkkKiuHOJ4#kA~kyBq{%7jtDs7@@j5GnZArJ5H7k{&eG>Q^>3Cr!w4-L z7`oX2R=Kys*-}h&)jpKKgNpy>4Y5(+5u`+9W4WcAz(8r0*B_~M$PRa1cc-=r28{CM z9svWwlEOHxUv@ zc|DE~6r}_a3aPP6?abJOEjeG$==6w;55=IK(sexQ&d6n6C=y!O_s2cuTLRw~>*!y9 ze2mchcW_{H@@1_f>)$@H{@=WOq3tB|<}K5*lq~%{<<+OXc@6B%{aVAyJ4<^87H%Wt zkOxMAj@k@mbD&$NB)nhscCgjXuH)(4juj20oJd z=_4iAJaV`Fvn598hIe9*Fp)tF^&L_k1!P3Y5m@f6I#8_pI84s!Q%O-Mi1UxUt!4ch zP$GgtBx*R8J~4h2dj9M?Z!x5nHllHWtMwKOhc)l zO3{74)0b=i=adIJ%0SFt`xJxTuf1KduMmJH`0U`-Kg5mA2vCPH^wRM`S!V{t4J}1F zFq=EY_6VRqp*gnVZ;CXj{2=0r6*~49HSL%bm{6u{(GLs>P+G8|$!c)s(1}NdTuizT ztgh|*GNQ_yjEs`9>0hM}<-grW;yyVahx4keySmf6mD7D6%BMGAdwK&g4p($cP~rFw zJc8rSJUvV@JC5QIKaZ4&-&auv0pWo)xyzNCMCq5BtF<#oG;2W1<%r>RxFm-Mi@F)~ zIW)WIRMU))9nVV>SA2TF+*g2Xm8WY7C+OFH!$4YwXL4u-liz ztFkX}?UIL`A^nM_lcy8y<63WU8FT8<1)OLV7aPZEJuWo$=FaFr3py6|V2pf(WEsEu z*^mg0tB2ML>VSXViwdaUnJtTMZol{%``SLh*>5f-vN?o={^+nj zj<7%WX#zVva*_-L2tS{0UD3?VXaei5;0W3D?cRzRG(QMc5RgQM+;T0Qe0A#189!6SSqS0dc*=W?0y}Tn5iOp1+Z%&V*)$H?M3m1!dk`Tudp+>iQt`Ar86i$rQZ%BB_f~K;4 z2Z|z-Wn5n*7Qw5&Qw8<@nx>D6K4Kx1$3Zacf8opGUpick!I;hRAtNkxE4>}!#+msw z{l$HC)fb`TwsaABdgD*|fD2w7oVlYD>84>jvdf zXuz5&UE&ZU@N4hCjGMGCSf>_0m41F7LXmBHI#XuyxB2>yvz+_2$I0dC^J;Hb#9>*} zRTt$yvBySp;5x3E2Z9GmFm*_Me*kkxEq$7D-Hy3GHk21Hz`mHreY19`4-*+>({z>= z7_AF}NVaX@oRY>B+A7vur2K6tO6EP79CY_^UsX}L)R2=7h6Id9hCZnEh6uj>V&RJm zCFT5s#oEN@LzX@m?mQZW4P4wacuzLH)3oAlXx};yb(XKPrHxNlsF=@aB)A@C<$p*~ zThSpa35F-;1k>s56@GaCI#$hyCYc6bdw-hJCTl9ejI#iE-C&Ib zM3HSj{&(T6a!~d3xYu9^2JlOVrpRbTtDRJ5!e!H+u^ZrVnksU?(XfbdB!E%X145aM z3qt&+D5LLK31L4o;_kwgxG{1r>k-CB

IW;hK9*aas_2)E@ZGi79pG7XGn%x zWfn!s1y`>J{LhyGi^m6q*99;Zy*I6b<~u8efex8AuK8@9DC^|YvDUJ#c9bu>tQAD3 zeQ22iO?`vYWxvvVPhR>N)2uQ@4OC`3hJHk@z<^OH^(U`+2p)GWmEno*~?C}eh{?|FB5n`DWFBhLW1wwjx6lQ2|kh^hutLW_do zH#6g!!MPOo@m&A0#D^H1lVT;<+m~1<*50&@Bd%CDX+OplZ|9aHw_h-pM1`h|iX z$6CW1ZSum>zflDQ7;q}6xk@W!8j5#`< z>F+@oBuSbkgU7mx@ysAJ68>^&LC4v_uSq(ZYz%lL6|VBFO@7A|=rZ^oi}*3xrdyb049j3yBO%!% z=W~5{Ij4p#D_pk4arzHAWZVJLnM@d{(-%rv$oGO*JN4U@RXOl+yid8zh!hw-R#c;S znctydvpm6^J3|`TPdB&%Z5;k*Fm}1_xYbb8vVMzrrpe;LIT8?sw5+bbN0VZZ1N0aM zKCijjqCvsw0o<atw@||7Qy-CcSHL%|| zKqb=V=ZR8CJEHF^azlTjwM(nNLt zMxA|aop@~(Jl()0#i6I@Vda)n7IMQ`PS|3okPHVg-FYri!1T~GFP`Fo;EdDkDh z`O0J3?s4>R<*e_^gSp+qwEj(-DB|~{dExxw>h;gq^;nV16~OA$fNOWb8^H8}gUQL}cT0tqPoUcl^0Ibh#j$JpInTSo&{Mfq>IqAA+v>N4wg zeN~j!OiQcabfE@;UFLNxR)Pk)o+o}BUR-%l0*O&$VXhBU_pCmQx&cY{sqZ=>Yk!s* zI2;R#fD7fNlnCS0U`)8u=u3<%DIiBwXV5tSGXb-;WRWTz=b>GLrtk@CtHnO%z*46N z9pX`M$4JiJU~THv4v*1XGlS}ZT-Y4OxH&a}pg%s=P0z8ohirukGlzMX6~I^nrpAW{ zaJ-h=2fa`v{2|9)+4t$no_ajftq+=SHsq=YM>|| z(ua3D#ZoXVkwXwy^Tuvve@cE6GiDrThf+K&{%t)KKK1R}f8O?I`_Y0Y?Gf?i-(JY& z|6)nNm_Hgsc2PkSYKr>YEQv`#0-pv6cPFDYfXtYB3!;rs(8RKBGju*3?KpR=nEHzU z0qAMeOr$lCHO>*jn5gMA7l_OO&Hz8o&d4C-j&W1unwG=`3%CmZV^D38Jz1Ya`m!W4 zrxS-MXEH)AP5~b2Lp!s3`1?g>RR_0hnZVsOpx`wF=<*EdKQ_*aT<+wj`~F)#^*9~7 zKN?e!=c`P9@-iua{g=>T1r*gDvl4tgW=XRh;j8LL!_tN31Z>6-J-3Cf{{5pno4myB zW@QV|u^n-axir_unq&xBn=heZ0q+Ot(d%H0A*3%QAzct$BW}sev^Dyn!H~owLBEgz z8<5VJSEai@4_EfNjnRATZM#K?dhKuno%{3s)Kt4KyG?d?Lv^+qeZXaOYgd9Kd3tav zVS&%59YdXxaG!&;mX<9H`xSVJFMIGiyk9lb&UKa-Tc+RKcAEL5BubyN+Sty;s9wE` z_*;_`D?@Ic#B~a^#Sx6FE;_R}EK3uAf!-{Ht~LyNiF2K3vz_fzbKzd0)%4?Ytu=0a z=HXroyt4$KmM&RX+ivPN%ewO3w%m$foc@}g_e z4rWQ(^|<1-NFtlV|B$hI7nt1k2jKoT^)&nGW$$iLKuWga^>{B>Yo}xP99VHSt05Gu zW#GOs+yBblTJu5aQF`%yAq1NU=t%a1$(d$^c4`G#Q@5m#8WZ09E=Y9Ed{@(rMO(Z< z9P;ntWO6StaFLyIuC+uUz)SSqaR5Xhvy05k&(bY|EU#bEIod2EGC$`9`J6yX-%`Ng zi+XtYkH$Ln#Sd&RlJ6j_erlpv3ayl$wxb@CwjQdt(2<`vMsYi;!m(^%m8+$=F4s^B z(BaqQ6_3R-p}$pSO<&IENczeOZNL0x6?N+E5Im{3_VrzNdt7^=Eb2a6*ZQFZfHEQw zm`$!HT5h{QBFScEj)t}TP0ZScqAWkd58!d(>YYnj4=)xRgK(Ut#V$)0^Kg0nSokHx z?^KW|g9g2pBBAN{=Hd0cR`*T${lQ@LkZfHt23%yAHN8}DY%Te|qA283DvwphOzhQ> ze=xmv`x@bLws6hlsD!DTgMl|y0m^+iQIZ&#!*3D8(6H~iU>B=y^X}mIdBO&hPtKG- zFGzv|?{*do8>VC5u{704p=;Iw%)`F$y|g|R@X1^cty{ewUG&{P>aqQce5`qKZFea1It0Y8hZh$3Pc&oKYtT)?Y^g@N(1&moov z((bwRi{7S$eIC1hd4M-dpyo$&i;(k3`)E<9+cBUj3ME-gID96t=LwMa47+fo(GoEK z;E=TB9?B{1R4$4#>8q58vjA61YjKdYi`^tk})>(+W+g8$otUPDIzR|jbT zh}QX2iRVyve{1`qCeDzd`_5#}WqCFb{Lgr@R=p||Y#HCR$W{RQfdP;OIW%kZ-g+2I zu&*CF8yeE?TTSoM-|1h85;c-l=Rnl2z{IdG94x0r?_r|Ny9cqlUBOl{2du@!#*x!0 zn{T5xxk7hHo?KydCRTB&hn|%lX4#Gj( zcJiyjks_?T1nq;643l!edXQKD!jd7XLrfk?yXmv>GZIVD9(V|KD<4kV&j7J9ShrJR zmc2oIg#ENAYL%nXZFT%Aj`;Q@sUGelAOS{4Hy(D-|`788Z z;)z5hU^sW!Sa0B5{$#w>f1MdELNZ(bjB!zW;@O_kHHQp3QmBPUA4s!S&|_TG2G$CX10CtEUj@WlUIxaNF$`rHQI zk^Mu}QBtOMiOL22^&B>NSi4fAjs33Jrc{+fPJ?^%e3~UN_*Z`|k3to|(F4drnC7Ub zGHAXYT>n8i<;`c$Ip0`UO)ftK(bHf|PI0;%Z}&GlG{|)^sCY|CSJ-@=L@y3v>=mK& zD%xJ&o@}-9(68Y?YCz81RI#$7ekwyH^*sw=ub z4L4O>iBfVh$~jTf1tfSfPH@sM2Ov{pMPjU*XiSb!RJRV5*RB zcw{OYYT6DKJ%-7eL&+b9yq^C`#>R#O-KM{v14`@yT^BA;UQYM9KR8;%s3_EQCdFEq zxXXGk8!hh=WxrSTr8$KjsD&QJX9&qjFOXHprDk0_sFh#AEh%)h{|@Am!c#a;Y+H3{W<+`B{&UJ8WRY^G|kpW5B3ss1>IgK z9u-F={d?ANZA6lpi9_8DHAXImF zAUz4VogC<#==P?XmKL*OHaV)_3shw49*R5X8$Z5$HU za6x%<)os-s`GNP#$H+)h5ROR3UJ5!kiH{MXFxQVI&OPJprSg`tB@wvYI$m)jKm!K7 z2MudR`h_?OE*ZukMR%^!&Fx_HK)pyR-_P8heODi9E4J?Utt6HK%Kj+X0r={H8}?e{ zZ0dc@gS=(D>ctuna3d8k*(n~BR2+;e6SQH^A;xA9nCIw0{hc_cQ~>e8oLWB3a`GLo z(`5d4mYOY3x-t$(NR1!1f%!M0!5*Wxe!-MNWD)+(5PP}s9SBzmk8nbsaoCk zEAqs%D7~%s)5;K_R#PQ4t)7x+8c0l%w?-aDE0YKU2zDR=VFsNXuc5e#P#PSEq2ZGH z#8^j{VzTIKpzOs~!IQB()Zs)n<{%@64H^&M2*kYX%Ge5%j8@}$zb}!ASpr9rD17PQab2910S6_~znlk> zfEI8erh-$xNlM01wiYTa>P&8KJl2Ft9fVQy%z>k1qf_a{7I=%pdbc^gFY1+5o*cK_ zOLXPnEz&miaFt=915NL?qvwQU>|Ww@XG57k<#BNdo{9cx9$($~KExMvSbnaWEjGZG z?962Nd=2d9nZ|B*14|$7+xL!Rd1d+fW91!-HKJwu&hHMb{J(`+uF36Q+wZ5l`^+x^2c59On%(FBpBBZpueIH0E>q~1a45J zz!F%vK@3zZ-6!t~dG#n>GQWsULTmiOC4t_^;KrEF2R1B(u_S(7Zj)z>;yMmiQjgJb z%9#1w^hVYXGhO|cA}MO);}U@P@Sj<*Fvc3!4urEe*0lkYrs80LM~^xT1hn;0lzfnI zgP;b)hA|1DU*XwC4je2o1YZM#H^gwM?8DzPrsKn%%T-B=fq))pOsl`%b{0$v&%rMl zH7WAR3I0hPmJppV(eA9U&S0aYi@LMrg5)>305t$2O=n^|!>JaO_CjmJeQ-BK6(oUW zUCOP;6h+*1t_rBULsf+{){sgMk0lKyy-M);fo`4(Ii8W=leA;>yvvRgUeE(;yT}<@$>CnHK zBSAaFdrsXAd7eq>?)&#$U9`JkG*R4)(TwHK3|C-y;_F3ye0~N?xW~(F%i-TjNqcH8 zSe%_WhS{4T(Ubyud)L$m%w={MyMbO0v-&@yY4lL}kCA8ih9yId&h$tD5)cBVQqldH zup^$_=AJ6oq*|xUEe{s=K|~570gVsW9T!M$uzbb1a88Ye_E?P|CX8qDGv|n4x-87qonkwu z3!%hjM7{6{2Pw3AgS_ntsc`{_yX zT(_pM%U$Q-YMSU~|L`!2#2(`DA^7oB^6^!o_dK)SW!mb^ecSzaw`U+gnLuC2d#hh? zYzV z9xm~^7LzN4EV<#ttZkfi1WN1i%XEwWxm6T$Pw|_~H+lV$gL&_&hpQD66%*gp4Dcut z$|FqyR4*iG6me_;6ucC16hv%;LQ+B!J_kW_jDj5ur6-9d1b8u%?-%R1zu0Z2W22Q7 zvE0{?4z_NXw!IY+LIJr?+qL|ofB35mxBOM(u?%Rcn%jBc=jzm9GyyFQ_NN=R+hrFY zU>K2O%!9-!biJ{Z5t28JE8_tyBPC>*0`o#3 zV*W_q77UQ7Wbh>BBHY*Ff*9y0Bk?4_*+F=Yjv4p(a|3)ps0(l(fRQc;wuS88Ae3ag zV-?jB^SQ=8W-_xy5@^e_(jAq;ML!C(TVO|q$XGK{_kz(AB@M7R(4g3n{$Ozli=yz9 zTyiwBn8_1CaWMS5Z;{t;7q^J?W@th}fH6L8{MLTXuwaB+!n^9hg#PN`n!%dED5BnX zxp2^<90i8nYw8G|=$!xC+t(lQ`WSqvz(I>$!H_k$W6U8>F)ACb0#5C06Zm_w6$|XGvG1OCJ8%)&v>Iti-OnP(ucgUa@-X zfDM8e9W<>M=%lEa8u~u zPg+iYXZ_X|@%od`ZwprW zHPU25uQ=tA>w66k1jG#zh%%x-$V+8dSjB}>r0Dg1 z+5P=0DhLZ(n20nXf=+(lg7R?WDd*2ocsxnFM9HoJxQYsc2&yhe)$z$P)MS5lz-UwL zwqb0<)PFXJL{ZTE5{eRvJO%XyV2u38(zQ14q*c)q4Z-r#(&#-4a88$A7TzH!R!J8w}DZO4TsP^@-rj5_GLzi9d zdVe;Idr^PBY4`w%&Qj5~5!;4DEC*7A9sx6fhatZ}w=-K9)- zA6@?%pWS_3x!IuW&hjaQ(6wl{ziRd`=`yg1Fe~~U(X$0Ry#BM{-N&Ok8S#-B&*Ak< zWkSJhC2VFfkx7>kT+Lm@ z@5W$5PUnab_BNKSznD?FdAXt?VXhYJ#ym4`2JjOhqOcF)gIO?eP2#1m{-}|jSF(3a@>`f;gt(fxrH=kYk#!9?Ib?irV-gENBi#mU~Xm7D>$>byE zYqGYcK`8EgTCwGQaq{=qn>yY-m(rw2M^M;7^@eq6;Rl7Ihy{oNE44i zva}zHHbu%d&OS6YE3QTh{XrRSzK)0QXN67TJ6MIj)x<>jsP;u3MRCcpE%wjm8#br9 zDTCZEr3_U}EwoG>m_o#6aDfB9X-O!sy+tb4&Y%AIUjG;(#Fe@)F*-`xQQQu9%67^o z&$wIuVE$|Ua845fNr3VS`?JIgO_@asCRSdXiFU56o6P=QWh`Gp~`9wjc)zE@9;iiQ;Iu=Wp3!&aEUtg1)Y3@CzXgP6Xv#+k` z^4jVw540|IZw%nVip$iZ*CS;NC7~jtiq3DIPHl`)8$cIOzq3jjvMTXB$#p& z6=BC`4#9P{{pW3(FbC8GHNtrxawaU@PCLf?BHnshu?wySs^cG`_~#tuB6Pm@cJ^jT$YbIHi!N)I4ibGhk%_D7w_{%Mdi#} z-gpLhPFIyHgS}Y10F5#UO`^7Rkq>&#Q!pqX;Q62XtXOq2{}tm~E9A~AMPA@AUlqIX z7`6D@r5h7!fb12)Oqu+f5nt=zQK#P&e~nJz`s|;wisPRQ!4s(A$ZAe9o}+QO z$!49v6Lh9T0QJb+io}^24L)1CFBhSu%-UE z3Wyo>j?3q%H=ZnX<>)J>eXj1Wil?}fl?0)g5ZB>5LWNT6iqiO=!5n^WN(+d5okYO9 z>H^+da-2K8o!|^vk9%I;izu1#W}VC2;bMQapppR$ktR(50aWDJ?7EI3>h;SlM9)^2 z$v9q0jDlO9tZ!d=*2BbjLCm&JrQ~rQ%km9W=5s$S=H8pT{Lta}3~q1Ei`>D-{;F*5 z==D>~Gbl^&Tb5wbpkg08I)b8d)VQOPShsfjHbE_fFl3;`%D5^X4jxq)cdIch3`BB+ zp!AGfFH=+Xes75!+ywFR>Sj)Bl2M(vKiM}Pn>bu|>7WU(_KiE1cPD-7m zfN)5HL`g+(Met1%T*{sT8Wv*1um~LPz*tEc+V3FXf^+v z;8f?-jd&c6iKlKTsByv|VZ7zq=vxB=2g(53t$P>!RQosJ%-0Ul&j^;9wlb5O?=*sg z(P^%av6Auq)>`Zy0jX2h13FewVn2@G{xzW7)enXEt?NtN>`vV+?!?@ml8VA2(-^o4Y`cqfgTIAqXsHER36y)}%#88^)w-eR{5G+e*ruhV9+_IKS0Z5x13l>r z^n=E+gyFQ35J_-ScgQTrIABcDa$MPkesW1XfjVnoE6p+JJ%0B3jO@bp1n?edREr9o zf1VV@U=?1>gFe7V{yN@Y^!2x8xcOa4-i@G_1mBL&SmyAJ{FK;>_32?D!2MwF;pA|w zG2A;{VS8FgG}O>MBm7H$YL5cSm2Og{`Z6tRh;*xIr_afFG`a2-F_Pvl?}F8(@B0~~ z&0-?TUjXr2KNm-q&G=84s6=Q9N*_s9=RX`S)V@(~(oR7)mhz zi&8mpaSeUpQ;BXXfB*3f)B3@qHvfm?$4fx&wDo0x)4GOwZp8oYDnkEmL;rR||0dME z@qm-Y)8)~=`R)ZhJw4~$35|8Fh)Vs+O^e^0K6^DKIi-&MB{0IS;Q=(QpCTu};mkx- zt~rlJTAUKng%CL|An8n(~3o}*7W`k+b@)2XC~n^kMPQ=&Dw-Z-3# zTy+ODnye0jK&)ts5>a~`OmH6*K_$CFP7?QSJLv_5K>`~58}o~=IH~-x?|2(hkQzJk z90th0F1%Ghs6{6GD;H(Tm}sg4N+GC9=JHV3YNI=oSjM{_JwLYU+onBE6jU7D53(m_ zD>z`VzoVbIVPfP>?nn0MPi2b8s6?Jr$2+d7*j)DX+{2@U@efUvEMULmJqsM^tf>_# z2$&)XHZzQQ?5$k0?J+O{ATrqO7}0w?J&u$DCT&17K1id~y!sO@hva_AeLOEucbl@j zkBsz~*K|a_uLSF+cG8*Pio#?Dwdf;0g#nU85+_a(y{5T`Kh$T8^hi8jv~=~nr(jpE z*$d?_x_47)jH!6cBz_S`aDVoAyF1m~>~h?>z8)if+0wE;<^KZ2B`$t(kMbM_`0dy2 zG>_$O4Pt_Vg2FsbnVf?_*&X<&o~&lhtJLAAb6#567P+Mu`dC9(9yrGg1j@h^Q@60H zt*&>eR$k9#h|&LXk9`NDH{h5cpvAxsiua4cOIUKI`Jy-#h(307BV3iKprGv(i% z=pZ2|6D zx{Pt23Ao-BuQZrBFjM+j=zkv#LI)uZF527MBmEzeseN4@FAnDX)~5V6c2pW%eeWv) zCE)Dvo!Ar=o2}7Uz0y%Yb=|3`g!r5|4^#9|9*%9CS^NMVMJ`t0mDs#5MKz-Dgg)aw=B0YiS=ZaRmx#ZXc7b$*V0>KxcH%D$7gr4z--i~> z=S{2B5UI>OaCHd_YUq2&&79kX3x&&{Z(m0PO-|YxePwc;;S93+7F8hFe;7iz$&r~r zaJ8c#)R&}4G&L#9#Sj$0sfuD6pf7^KF#2A1%k1~XhkoK5`W-)20vQ^Q&Bf92FQ1BF zX>VYfUlffF3RI;ktFZ)5lGfvS`8M1sEqZN=&Ip zJ4WP8`)sB7;TN(XK~caYgCZn=@B^#Yo8{ZYg~M$E1AF2+F>QZ>a-0)zC{*&h>1T5) zV@0NZMBH}Q&dnNu{$<$XUD@2@MhDHe6sUepTjoN8x!KmuJR7hYnO-`rAQk$M|9tiN z*z;T{%?k;9#T%;Y#@N$z!U={0i6Vh|q)?q*TLt(|g-RtpaxDI?br`31{}F<{c!iqI zBos!#dN!}6yZJzaBGG$(|D}yy$JV4fRoY7S)U>DIrxRA|g5e{01kmr3=@tuEYgnG% z5-NM7&CjtxAhZC8+Lxf#GP{-`Dj#wdJ$yrZX`W8jVki2Mx=oG{>U~YkogFfi!!Xe5 zRN=hq`?PqIakO|=28<~3#*7I(=k#07ZuUxjCv8#t59g!zxAva9Lt9fC|3(v**JPN; zav;#>Eg04`k8-aEh+YLX+3iG85IW_G)X?qa4>kU*`WBHoHpd*c{W?8(I#YAp`(l>0 zB8WC*@dfwSewoyr+rR< z`n>;gK2T?&xW8bV9ZdD272AR=V=GA>Gccai#p?jr`?`i zH&jmgrPd-BQwaIHxsENMf~ESWa{ds~3#mq@HSXAPlIQOix z^Qy(~?;3&l-(&s8a}F_IA)t!+)(qyGgJG}`?#Ue&YR~mJ!Dx*b^&HxecCPt4XY=g$ zv?0?wp-kj36Z+M0C1(d=-@hISn-u=<(s)n=&+>xYC-~h_T&}3<=MnGTAN}Zi>+17{ z&nT4Efq9<0x0S0TWIKJ1+HQJdpYpq$*%-qGaugmBwSa8P6Xwp6!R8D_r?)MPnn?+O zDV=rIr+l-KxOwmL_eZ}#eSVu0M20wL#I=JKY=xefW`|$aJiJhY{&YpCV$KdtbeE5S zIzbAfx~*6z3~Vdk8p~vxs=TuucQHssm!aj<(aw)FT&|=&&S=L+c(Zmc{v4_vr7K&? z9Q5Ic%s*q_S(JYQk_17zLm)kPcULJQ(^6F-TF;<>@R`=Vlj{CW;*d}jePA*g{S6sc z<0DJ5KT7MUuHZ$MYw|YomyWS&@zSo?J1D|W)Jl9AD?pS zSk>4cE&AW?Gx_graK7&TXF%O{`R;pr@vwILIIL+~$-ej8c{4uuFbi4Eg$oP0y*T)) zW>tbV9qYnrzMDx1lL4QIa)O_9r|8E2?aWT-uS>D-yp3_E&geHMZ~O{2=KpWH{W+Lc zCY*-}w{aooVfYRC?*WO1F-c};n>UNTTq2?ad$yS-YVr&BchcY^hH2Mh$%q3=r_yu- zmBkIvZyyylw4WFj#hW$XTcI75CdQ;MEz!qxo*3Yx7)qrtqj;7~k%Uu@!$;V85Yd6z zarMzax!GHqzRfL}=k|Zx4qgG_77*!h2m_cf1l%2B00+bVoX;Crm;eA#KM-CMT2RxlEd*aZo4f~o zdU)S3`lXV6&C_hK4X?1 z;78N*`|%XKUk1~C-`S7UlNK3G;o=WfMQw$-3VZNK6CiU7qjoT!!5CNShe$Y9OxDNg z>E4yULLqs5c%MG6U^9OCn!`K#u;WvaIWd>2X4YZPX1vPPW1Mh6Y2+K6W7=_F{gQ@@-Ff>joWqd zay8p+mX`^hlqiXiz*BRaEvuonMeRvwc8trXZW2-}n$ccE>?;HY_3sdl6dw(eC>_67)WfCWrqLc_UoiC3dy8u&v<=&bp; zoWIk5{&jRK;^wvDna^x1kpE#T+vcRL*jF)P7&GY+ePI;_ecqoQ_({R=h?T>YzoeqB z*p(|fN3d@$PLR!X#@Ig;I2?nuWv)b&&lRd}5+-h1`}tA~M)}&zOYwIp{tuO6>NG7= zKBzl5)w-Pvt|FSn*~{ zdNn@t=#W{1Bk%|ceuJciy`u0C{@$fQK5!Lov6xm+;>M(O=B$&78Nl}y$_V8o{@zjK zjy_6<=RUfb1I|#XQ~%3?9&ZY0pb43FZ0J+<1`GChlsCFx=z35prmP7jJ}Dv@ zfqQ=7HAIw)b9H>m;8R#-j|>rXZYkS&FhoL9Ct02Ecb@7ThmXO61~?U2J$@_)s!5Vi zYO%1q#c##jA{gnG$KE}lgo^21Y?6=%ZWUXjyL@BNrYxxB9gQ&d@zXS$k!5s21ktub zMcxRWwr91kAe>|Jv8ZkRw_X84R_7Sra=bcfgRQ#dpHu%jSgCn^^T_XmGtCX!Wev?o zUPbmSe(8dT>z184IofNg85|ZItzpfC5Xm4w1z;G~R6apbv<|fwXUEz)_IwXFW-Sp zqCp11TXXeu3EZ-fj9c+Ic)pB}4qJ7%GSEfLPMm~Sxr)c68~#=kw_7(bhl~f^!|Va2 zETPbuxt9RLww;T(u8Ig64FT*A$MyX9S40}ym-w}_zH-k|IPuanxov?8h09t%2JHe+|5q3js6Ms2zpV?i-%c%nPxDW zGHIudK2HzGa~Ym(b>jWpgvvM8fdXD&u1r(wp(B$wYOeAGp^$L{#28N*F0f#Vb>lwgHFC(b42SH z^9h=)AK&SX8ccAq`{x?Teh(^8Ync7D5}&Xo3Xc!f%{6$;nm>U#dKlBnH)QE*?HoRroB6v&Ui#y4|3> z>akho>@!dKC*fNAP#IjhXUfWQ#XIP8P4%>$Adw@=6Sa+@hurc1xpq5w-o@9C z9C)Q3Reg?3h26dXD~e9ByDg+xMvFTIdoCh_50~upz@03u7L4ul8IP<3iRVW&qmJv+!r7v{?lRt^u ztR_y(Blq=^COj>Xzr?Ml@aF_DB+GSPO<D!sdz&Eji{m#9xywTrPC```R8ZNr>Fr zXJy>%`YSG857p6s-owEea$_1k)9KGlSl^~kH9sKgLEO@)mgJ9mrbl$Kof@EYY-;~A=rBc|_qXVjL{e?h4VyI47m%LS znI&yD;X@%WQ&bhT@SsSvy8T7Z4KLkWCA<+p664f7d;SF03gHR@!%{}{J{Wrx%VAa- zzpBo(e;TEH$o&yKUnxzTna>k(-L50*Df+4SQdlUS2bdC7&oM3D;?w8$tBvl2^Q~C+ z3!#K!DvZ-Ak1jTVzFTfQSF4o1DE?(F>v|^ZQD5R@&E9t)OSJy6)ms1Z zraNXiT#ZNHB+?y-~b+|NqU3oqp@s$YY`lK$GFoN%EQ&H2%Q^Ibr2yuC0 zt%x;x6~7vJNwua=#;sD+B%GyhqmHHv446+8y+k`ePT5u5yiq=n5yKy!hBiB2kmZ=s zXNeQ4WGo*<4_0Iz>T!YAr7TLKCi^nnQ`C7=5!L|S3(nk?r}pU>Yb10;)|AY?daaPC zB9_%77{T7uVyNr6^ z{Fv@L-*(%KMq=Jt)^v_cF*M9>ww@XmXgsV&uOAm; z`S1L_$h!Mez|`HNe-p2d1yWQ*`Y&CI7Zq@JRj?O5CY5#kifCyNXhPRKl5ymDou$Zm z>c;Nsj5o})6gWX^Agl{=VOeBVMxO60n*S#W{A9P z)yO%CgeVm^s5VA(MEjiPyBV*L7hh^>oWh^mS)4F7js)f3AX;&4u3I&8B){4K1Ki9E zAs2I_C&(r4C`(b|`X`gS1j|#j#AG&SUEcitY-eErYC*eI9o2NT zzx8{s)L!55dZXnbKH2x0O`nR4`XNgHF>y=cNKW9_5l|=74(AyqIIMGWS+exP%}6wc zZ_CNiq(Zozm=TQGtjYh8?diDKXOw!=T31aDl>HGT>H_Qtupn#ZG*@7D9Hs1M!%yxo zT2bykZ{?;;sX|KXk`DD;N(A8WU~J~B8fnQ zr?MwIQUr-+zZ^ba~5&GM*;I$8gL^gN>!I*?t)$B$hgT zZpIRlhKLNLe-(Ip*03{}ER2H%1ZKzE$!5;qK3aF#XEe+2uoH;=Jzi=COdp<^|HB`>2GkZe*$oRq!|PI!oK+^T zZX9`ENQDzRu`sYDD?rAo)H(hb*C`Xyg$RXkg&faRr$%wZpE1_OOR-rZ`4Ky>i%aOB zK){VuQ-+}+wrpYg&jN2ODi}kzhh=ZU|EQ`&h#jhRr+Fecb0i-(F!_&>rU+>;ZhIRwmUn_cl?8eZnoKxxth)3tErYoo zIVK2zOOjOF8wg*gnyeiD7E5NZOjuVdO=+|)y6!9xzn3fXU(>(8&JsH#Q_*kQSY9@O zifVcp`PkUl*){hx0nZD&Z2!1Sv>{0K7dXVuUpM=qBe$VK3uKtR;7&FampmSiq4JVn z6t=6b`P};}WKlcK@0GzDbu6>a1_`Y!P53i3<-;Z2@vG0YBOLRSSt&5mFYl2R7`;ET(k*R#pOM`xJmj}_u-l2-I`q6Dxn0W;7n|6j{#&g8Ij^g%JJ~Y7X z4JA=yv#n5s8q;1Ck9&RWmir~397GkHL|DY0Xpadovd8r{w)V8+KnQ23^*N?e4(u%f zx*f$F*#;tk*}5?^ARL9lzHTOXg2HDtOD8A0(_Y+#W*M%q_x1F-j z_%zKMg#u+Q|6^eMW8nD7kNM5LKkNdx`U;3`6KRC3RFLZG!6m*&ASIW9`$48Oew1P@JDo0Z@jXHAt%E6!! z1XD)60zsq?;a@R?N3l7kLv|G5FRG1e*rZ^a+2QfCoVMOd3t*~tT$A`bxAZij)ncZe zGvmew@n}afwCOo4gmNCuIGr7|A(;}j?XtyHxqjfqmvOukX}QGW^xt{Mn%Mt$Q1Np4 zcqCTMDq9Z=F{u+A6PN-x1VWQY(61nRRm&b`F4Gtt0=(Qbp`Bke&@yEhc#`4K%b*$n9*l zSBw`6{SlmrdRSDBRuj@ysZEaKXOgqhE+9fFnn%6O0T_H)(7cLny7bhS8QUfqWF;Pm z7&8MkHtHY_Pxip?G>=zXw;)|D{T10k$G9WqA(dxS|U$Nh2q=MluwnSyl#uwSv># zZ85+Ndzq7yFGh}Z(H&oIrFo`fA*A!FRO10{JLmBabQ`~1ATdym>SGqCdBo@}^r@n< zB3pEU$>L%H^&1RbEbmt=Bu^yRh1!07Y7R>(>t1;!pUl$hAYSm_Q_6iO%q(n$-fsjR z898s}!`FUK^a)Pgr(5cT;k^KQC6j&skjbVx=J8ePBhj7`&bclEFN>SIKy7w4>JH5U zk5k{ydjphymwa1mA%^D@3BD)jgdB{Okx);N}$* zsE++JCwjCg_vDv>af-%)3# z{HwTI-P2n29ok_2W|3zQcekVgGC_E^vy!(r=7oJ3bG=9uTDh-_$5~+R8 z0K<0omPY+)RO&?=vtvng$j+;vk@`33?+C@j1w=f~lhdb>`gf~OGY9JBhuF=A!hyOY z_U_B*><^6sCvgDd1CGhE0#O1|Gd*&ZPjsqicF0z`3D~2D1;W) zGKP$4r#wHXNg4yDj}JVu6pR|gre~Eu>VJ61ZI&pduD5|b@Ucai*kcGp&vnB|U@Yw8 z!}7)n`=acKOD5i>aKyNF5&&;^p5PddCNv@D4)Yao8m}4r6pjHQq}&%T2>?*$w}R6L z!W~s!L}GVnj>o)uFF}woyWDaB!YmIs5R0~Ev152A9@D=7jU1&{gRGa_ginyIJt!_{ zj#B@G%{y^n#$gMIYUb-cjJS7b9p5&Ya(NbpGVfae`$qlZFd?6GNxr<%rv*=SXvy38 zzFnNoLN-FSxQiA72uRj3dzthdy5508I0>3ll5JsyP{vRdwWs+9e+HnlJ>XpG^s0fz zc!Wb6tXK=7AdPp6tOqRP)`U)zN-cjO>?R?$9^QDkT4eucZ!Za)fR4qb)t!rQ{Z@Ix9HGEQyr@k zl}WE)$oC1N+RWU9(Rz`Y(Z>v2r}7L#j59Oo{F(gXv|50m9X;Olv_Q1XBwwSYr+;Bk z9!a-EFmUVlO(V1FZvF|dwnH+A}N?ZcDUTRW1QRJDGV zg*GbzdUMDsZ&9wa46LYpcPKQ14E!jDBA`#1$-o6ZKIv1y>~OC0t;nYiO|Fd0IDw~b z#A<^DQ`V4C5-HBVnjFOB$dGGOy(Slv%{yytEVw25?x_k|2-pb8CAEbQ?NnL}!)$kv}(#w-4|qed-694~niNx<0zIOd*Luof+D%Y6Q5Tmgwi7NyUzaQ;&<{4b{< z4`4WapgE&=1-gclbt6T&4ldvyA=BA0O5E~su80{#CXEyZp%iqj8UuIC0&o*-=`tKP zve@#Jg2Q;WWZ6VIez71YWfkJgKA6bjtpy}^xz?J;8kjnOo-MYh6$MvlAGL;{}mWabK40o%-c+(2b#h?QdzO-il z>F`Vj47lmUFI^dORpFebdHYO-F5Md4iT!FE9$6?mj?thZb`2g?y3zV2dkm>qwOx{$ zuMSgI2S|QdcxH`=c6`5U=(W@?`&N+oH_pr)-YS_Ua%ZTJch>u(Aw_>{v(4yd6ubx61dk7mkJoFDdyg)UKdT>#YFpOG0GqPLl-TuW z|8;+#)A6hwjVWL%%pZ|q2rTN5t9zD17OJQ^9=@S-l87GmN|v@g1!7h;X|1n=kJgSs zv^wUDg{x!15*iQa29y#p$)>CguOr(j9c?Em4+z8gH#J@Akf)$Vz_r|*4UZXCn7JKcOQglaq( zLcTy*q%!oWG;MGB-2n$eCVL;>q6NR9qlbs1$CbUs^C|mJR)AE?f4TF7C|TF7Vs@cn z>1;9PZa{A7rog|#cCM4O`8Y4RX;tI)V!saPWZWEo2lmgKj$vRYN_Osh`xwauDoKb} zlW$52K%+;p1Y788VaR}0r9{o~cyZzHSk}!jaMgC%lZBFbRH~;#UaJC#F=$R`g6B_@ zVwqx%7IP?V2$TrXNZCMN#Ha4_MivX8)~BB9Rl><$0p0N*ca#}zuiUIc>cJN~?vSkm z6`wmfl^EbndZK+#oU`s(hUd)rqQ75h`CljN-S=w&Efsy=iA0HqAR6!OY`LUnk=J7P z0s#Fk(r>dMDSCgV=@#|4Kcv!h&0?IB!+DdM^)q3BK;p(iML+ug_fHgUjp=+*-*IKmUFLrwiCV>i-qXZl&h_=Cw>!wPW@; zHPiOu%=BJ&h;ayj@2%JTAdPh7{%^veT5hOzBGB?x$QXjWIyi=F zp+klNBvPhzNvWEJs^*gz`>(;E5U@~nB^!fi^k{dwJr_FKei+`?7j)WAh&OoD^^jcH zAGko48}=OfO54XP^vSxe3zeDo_ftBq6M&dPV<8x@s&i6{JFO@Coou1(9=~WkT)fcI ze~fl-x}|w|XfnGlKtR&$%yl4$)b);DS?1jktQ&kp& z1RmrRUx$O!ekj*Vlc`@K+>%oTx+9(oKTNtvAav#d7!5GdlBAVyFfcX{SO?Z>paUQj@;T2-ZWAi4&2Bzz zjT-nE-SKAKZ7((7Qn@eqULM^oTBCs+(;E-*?7cc@nvzCM^_GkL_BOJvw&gMblj?mP zpf3MRQ~oX<_$Im2j0&2lygEJuqOihliyWk;RWoDs@BlpA$Nai5z$#p>$fjIrboL4G zcn?TG6tO*LUo09pR+ELz21f)LQ4a+}|L^H`h31;(9VvSr zF~Hfp13Ep|p`1QDeF952>uL7B_cV`v?!Z?8UJC;i!QXGwT;lFgyyCAz_f72|Ad#U+ zVjmZXDu7A_#7=tgPOgC%P2*4dycXe&iLY|V(~neDj_BY*fðOQFi2ld|wW5xI{iS9P;eOq6V$ zcwobHcvDy`np6O-3G#R87KuB=AAlxCZHL>$Ilt1#diBM8@gfQ3ck#$aLL}N= zZhzgxix2$Wbu2v|0FU`eM~`h=H-Y|uw*BFL$rlfZJF;*nR4V-VJs~<82?FU|q_qS> zynvqZf?ahtQ}@zwy+q?dDr)-#2RKLX7V)Bo9pd#WHR9FnQ2pXhAK=0emWKTGUE&2v zp0bCIxGu;RCvbPRR?=f)pmjPNF*iARUyZ2J=k>-%qj{wUz79!wzx(^m^8HBEhOTsx z?P0q0-o3i$&|ARdta78jRu@UT{T>NvEiFg{pn)FQkF+7NkbSQduWpcUDf}r#DFxSh z-}f}-jLPy)4XOh&Kkvbe4tN@~4bDu+=Ve^ak&h4-J-_JT-spAiEuC<2r3P-zyA_&; z<*Am7pW|C(HTM1&z@%XS=oo-N0y_Tp{}@Udna80LCqbkSojA&3Lf#u}dKdj#jW))& zHo#2PRg{EF5-eaXS^1oTUx!>2iw$q&Wm6hLp^f(y5G*#f^1RNAqUc3ZDscb=xba;@ z<`b-8FJK>@y~^c+TjtcSi77I}LWD~91VzH`i(Y;l)wjKD_mw+qAdu0{WMldJ^1uTV z1i{tCGQ^13vK)p5rA;L>y$YwzM|x1_2Lkd==XJta2Y6~a&bI=dR36sA*e6v+G(aZ| zwJn-SLH5D2J0}IFaKb4Ly)E`RvB#gYk5H1v-Pctt&TCdK$6r3nc{hcLV1RF=yGGORf za1OyuKKZ9V`j7m3+J)c*gn+pBq0l8e)lB<=R*fZLtD1jWgx6)QCsJ(G`Sy3NH zNDISYy9m^XdOU|m=ccV5bqma@K&;uc1E7;4eQt{}_E zD$f3$9vIycS`j^~`acHg1!_{R)^RXw+o>?$8xG$Q99E;b@o_7Xqm-pA6xh4xYXupH z^opof_&-0OaD8djnRmS3ghed9+o9WmAz8M)_E}}wYi>o2B@Y$=euIK-#OhTqG@Uz@ zEx(b~^ik;9bE+uK-_`#1QdH$U9}^cU_1OpsaT8VSGt<>ZH-vgDdgZ)8o!uXZDAeK3 zd@+7c9CFyn?{BDQGDGffwILK!iDc`_+f~Wb2E+0(M5tc|PCidBQfmw2opmTZ)!B3u z6F#N}CPzT;sN@fHj4uDJ`F1KuHT>{D7rke&#Iad+Q>?>>gqOh7L8puucWE8pmjM6r zO{0+;8YloB-PI#O%{kvw7prDpu5J5I*X22Y;xa%2X7|EF&fo_ZMy2t?Lfan>5hs)h zjxNzUQexeee3%d-9~;^mD=Pjx@a27``^i|z+xcPg@Ykdn*otz8{?&L50FY9{O`n>C+XoNk{m zD*=?Myc=BrVTK`{4Cvq-#pW+9J9_E?>)P!D1xwQ8Idd;1Wk^wvP&hY&qBS(0w|wZ> zxqzBymhbQy->Ozsz+AH>tRzKk)hh*~8~pq9A#iaN6P!KHE>|L?XlI^?T65sUzAXJi zUCwHx@q8b%wf7?U`+aP`nM<<;@(IXDv;7c^?;h8l7_Kqw>6WN`a&yoZp;F*TuRJEY zw7%C46mCQVZ!&}HqX{z95Dc5Iwa-dmQa8OcdgpS91QHs;w>;lKxVrb_bAct`GEUP0 zCZ(n!ePbr&3Wq`_HFzUU^u&QtEpvE~&!>jzvN#gHA z3YHuRmTo*OFS6B#2b1U9d=djz7{WnZg43btZ*q8Y&JJ71WN?iLjA-HwI0`gR;#_vm zRPWmu6h39>z(rdAQ65J>ZYGmRfkDfj)G!8I=@QqnN+tkjCXoTWwL>#}lFrX&+{0_Kf zV~>o22v2lmw#_&cc9R;UZG7LwPd=;z-|gg^UfL+Alsxe7laLglKbPEi|E+?{elzA2 zRPlS30{iIm93p+96YLx{wZ{0QGW~N$Nc?g7d0Mh@z$mLw_g*d2xPJ8JQPdi+32pOV3GtX27tVG~R zg_GsIk^fj~Qm@SmIJS&Ir1dj!!DKi#SYji}W%vbG`ePq5?)$xg%bohk>VtB@xJ>~+ z0{Cq8+w3hn*MB*0)YDgxUlGaiMv~-I^>yCR`YKHXTV@w_5Ga7thk`P7O$#aFExRp= z7J67;P*a7bmU=(Fy4~1#DKD&nqBvlVo>f~%^gZM*2{FMP+L$otHQAp}RVGhw2E_iR zW+M(~%XWac2^7&|f=U$A;e~v#-V`iP9ySEQ_^LRf8O_`HO2_rvl+*JJ)WG(CHWIy` z2u!0s$6vU_Gh_(HJa2FX02zC$m6q;x#Z1vott5@-uEu= z+S)Xs1*1u_d|lr>44y&n8A}aijusI`d1OMX88=DlDZ`{$mdYP~65UVB|j+S~A7cYVm?*$gCw{ zc!99?WhJ}PxM*KU#;1R)D9=H}J%V~+3={2n~15FhD=LHQ~o(usAo|m zn@b*N_0lOnlhHDXLE(?yA2?Z)MvXHa5DZ5BG%@x!zT-@QIy;@L;TM70q+=3T(D^RSs z1R9`tptz)NF6h^n0ChQm5w>NMS;Lm1wq^azr zVUS3}Q2Q!JKHrl&6X}!+b4neT0!(m{+by&ddQvdGmAo;A(@lNY`u*Bh__m~%e5h^>u ze*PgJbQ2KHEc+j>^6g~b{Vj#k7LqdW>PW-^zEoUNpJwDd2&#}l2Y#MWptjb}Asoi! zVgSXO`hbX@wBq~y1Vf)?B`3_;Btf6U)wMYnM(W%`2Lq}=4n;T9U6|^Z>Iw7}D={AD zbWveqN(Md&Wr^A3){d?+B~g23YF7hL-&3Ushhs|i{|d>G;Enpq&z%2}!~|BuX_FW> zESE-`sQ;ICJ*o4hp7}1oZN?qjK(`%>ACUC45eP|_d`TyDe){O$PkFXfVWU4loFh~z zrsuLw9wswEAiA86Oxw>0Y~R4h{P=w&FL_Ll2!m zP}3pvY7NL|@;-ja%E#n$GYNY z|LDB)u;=}t&oo&56<3ZtQ|RO1H(KNwp_pI@lVlQ=9PTfmIJIV33cIlWgA_CS=Ov(F zkE$8+bHZICWMa@l@YHJQ=g-_0MQW-)7rKRvqcS$7H+obwLnm(ou3+-~rMSPXl}2<3 zAmH&>a69hX6|WKvZvUTH)Surun>t1`XaYoH5g8Z8norT#d5MI3(9nK*gzOkkE=R4? zM$20gCO+eG!p*@|3ulr7y|aqqf)Woz<3knMfSF8?l4ttd?;O9*y=lYp`?-MvbHIhN z2L~&xfU@akAw=Nb;=GZg`9A8aq%C>J(f&7)_wVHfJu?8Rs1+b-r2Sv1uZl!nUESqz zO3PZ9|I*DPwAM>(iHmz6h&sP`74w1^km@AS{erdsoo_2YJ$Zz_IY+F{Nbs|8tc>5T zju%)%-7Jqv2)_Z-THD+EnGUKD&j3CaeG3%&a0OukWZR#z(%jBY;ohA9oDI;D(!mMz zs6Do;q+(+n~yF31)cH{0)1_6C$wnHy2sQ7%nMHKqCYlmW1RMut$6xGzGGw zXBh31Mj$qYQ43&r`oqQ(oJ^n`nmjFP;UqS;=`^KG{LZ*>{t?>kYS{hNpvXZoM0`hg z=(aSanHTf+A+z}=#v$i+745z`{u}SsLR!_uLX~VBoubS;bXg5_nNS)m!71rvnr?gx z2VC<5I-i4ig^S+7yL-%u`>a67jA5Gp0*6scLf^d0qV3bGHLna2n(c>spCSH{M(v@J zS?^|XJ~lp1s-@|1G_)_}WTCXvjYL>NY*UB$qhfi|BQGm8yzPKQjTAs)M4<7iz9wT6Gfk1;ikx zP_mXvA+gX^bK!5jvuWBz8NMZ4WGurye2)m#hr!q~=9O54#m0FwsevcE{T=AmA$e*MTn9}YUVN0<8!W#E-ufM z*M!OI=Fo4weed&!ZyP^remAgBekTo0szbqXwH@eN9YdV)8cG`dTUxqa?$s``s|%an zPH{EgoU7iLG|AmHDNd3^cXG)2za}Sd_FTTzE+#_D>b`1y#R_|&Pz9A03H~x}!DiO&91#qC6O>utNGrHJ6LxduYkpmrg}7RH;HMhfLyZ#~(1BBISzDkJRfLszk#s zbg6OCO+8ie$BUZ$){7n<2Sb2?daAfqCKLV#gWcyq@}pLr>s9a6(W2#iv(qYlJLkQ> ze+<7s+Q6MS8L=9(@4_y^x!x*e4d@Ih`0zr3Iml>~EAf|gHxeRQVYL>O&cHMTEs??# zsxPWeI0OwF89?!22=zT)ZW#cVqafII(VT_k1bKgDZ$y{)E!R7~yt730Arn50A2Z!bLXh z@sc4ve4+Ru|JI<^{;~oXF2g)^y#F)~!h)i{)XrjZw*ntIw6@d6Dk4kPdpRxE`2gg za`TM2rh!ab%~}*AKZ%Pcbe%S6fEn|zI}0cr3k{8fg(Gkos5Ms4SMs_S+8{eRKc^7) z^&BtKJgHzK@PHC26SD|Fr=K|?6s%#vJvP-VJ~sZQ;qmf1AxS4riSU=M!^~49CvPn_ zSN5~lyj*N-dgo^egC#kM4>kFKv*_zvhY*#{Nc=3-KtIUwaVJlyfeHGc|JKtua*x^`%>5<;(Sli|8QY z92^p!FM+8~zZmOjHn0dZw9)>?+(q#erB(a_Q{(EoPpmnQl&vqffGe|7w5!-)EyECt zQi7-zO;d%T--K9hZ>7+kgKZqWZ`>arfxVpP2>dwh`g)|4zCIb=-a@ko&g>7XVVWV6 zCtpNDO>~mgdOqVi)c<1Dug-%JbFK1AaDlGsn843$L=6O!=_ZDWKK%RC__iyums6{l zj)$CwG#D!=z{4y?j{7W+Y)oHp_HQptsAqs-<~KpE}hUR;SV#IR7L*Ixp$hEO-v|`MIjb z!st7?^b!f+1oa)2NKs#$cra;#H;T+MZe%K088mN-MT-?Q9sU z?@FwZcQf`yKoxABn}lDD4#OQ?TTaeHrdnJ6xep-py&zOGJg;YE_| z<0$K(@zc|e4j}*GH(Vm{5jpl@RSl_|OqTI#zg#sR->XS;PffC}OD{&2&vlnKobDWR zzguDkM-`zhrdZM&QLe?q9}eNdLWbFYLO(Yom1Z5Ede<|wiOI)}DMl|VAu5AjGn76? z`$U5-2BLZ#YQjsZ`BlO;yZ?Rcs2RSKa;P>HZWlLx#+h59CZ^NTpS5kpB48xg_xG#9 zxhGCflNvqxt*z_mr56BEaK=Z1$%$9A8cO=-?~oaorH!pwrcor7tR62X&4HF=G#H5) zl4RV4zct@FsYB;9f*K>0=sWa>L|1g=N|QWokY!laVWFBV796O0rNV>;AE?2+mxn4s ztLr%7ana~jWs^+)_|blbmbOP>V}Vkkci{jMi~KDPr;9%X)hbigvAY>$R!?Z%OmUU| z`X!ktfmXw<`U4$39o}$!ZkMq8zE(CkT22{y%j;7{g(+iRJ-u;vY@4`Io>HoFR$3-j zrr!*efBzWmgN`98j9MT#e)m-_m)-NK#GX8E1TP)kxLzLvtb(w3!MJ9nNRvaED%q@t zIikF=zc1wM#NB85DFGNMRTy=23c0-GMvxHZN>+TuaOG~I4K)!g)Y#b0KRkkade?jV zwAx|YppwUCzWW3^rx@oNR}zng+j-I)$H!TZam?6@`!Lu>pFcbaeYo%;V}6U3DK2*_7v^I;h%OiQ|ICDy`McbK;vH9=J*-Qjb$+y>zM?~ zjboQOZDOmMlm<9Or9I!T;ZTo$6?d0C2h`j~wFftSIT3ee3uc4{^V!)^y;sJN~yw&4;`${D@^u_Jnom}*#6f=B( zy0P`o%NJ-_xJ$N=!h2+MV=$dr-jtk;^{hc+%#P0f(s4#nEt3GM=3GrP!tvz7YimD* z73zUge*C%yNn>92FtJX180`eDF65^Lc%}g?1P=YzVSOWJcJTG zVpl-`qn$H@B7+!?j;4~2#vmr)7V-3xr4w0kT;)D9Gf|c~@rJuO)HTQu7qhjv1y`DO zy&iI(>L{UE5hg=tW|7bo>%9q>VaSbUKLZxc_19n4W^1E*6h7mb*0;$U=+~Ga^}#8t zez$_iFDYw&6qp?Gdh%V?;6@C-A^Ex_LXRGs;nfIFw`3O&aJF{oUQ2`@yu&-t`$bQgK~z8o2kP4pG`0@efmR3Yv{zbN;RX^&suOXBMe=;o!X3$^_$lakSy~$@8**TxV|3EMo@P2EUJ&kEClbr-0XOF1W$S1aw>22zlCg zcn_$&+j-ARJ$;Ym(BdxBpyl*Jf^@8$h=SsL>^LMcnN`2Wb1Nt0V_BID?^5u+$Nqxt z$ckH`#!Q9eU1v1uv7i?knqbOf5a{bVsGb%s?`o46iw!@9&!F2Y<%mN3g!7pGnGSXMEXBqgkve-qQOAl5X_r z_qcMACvxiCsM@(yt%7U2iZoMD`k2jc`^m2_9-lFPe4G`(tX9o{3cBB3gvC8dJDd5anjHjlL(6Dg5iQoBhBgPczEl7Gfbx0QMcy;lJ zGgO(EA2(~fmKvj8@TC}xS_{96Qh-%QfV?xhzov%%JoIgX?BU~=G3hPP2~G`k3ACna z+Pd0Av-7;w>?IY{;Wp%Dbp+Xp+v7JuhitZb=UH#+)s~y( zTBCsBE!V{;a-(eDdMKYQ!EZG*v~W6(K$E#Tr$h$Tn8?V!_8)*Mdn*^!-5$DhnuH~5 z6{o}$s|w~Zhh9GWp@bf4c_g~CLG$7?DeOj`Mdq>^!%_Gx(KnUcaVW;G^rUW5Wv#?V zfy=s0`eld55cDuBF#I{A0sRG$+n|^S4yCpLM=*6TUst6$*Bi+5Kn)Gm(S5NW8I1_XJISI2D=w%VhPEMBTQ#89>uoNXmuIkku}d0`#C=L>5!TtK z>+FPyH>J3*Uaw+XLMZ2INkLjCilVV9QR{TTm11EYe}^cOtI540+gM&zcWm?_bD2%> zek-IV#bRNg^W@keugt*0)b*=zm^363RDCZ|#PHs2)VE&P6;yYRCGcHK$zx}a$u1V{CPc|6ZX zbEH%c!KIdeT9Y6u8j;w8w22;>RDDHS_MC(5;xvKaR(4+}nIRc&B{eUtCqFK}70ZF( zHdDT){iWp`#&84~V6l(+j)r!afJ>ZZWkMWkMvQ`Jam~#y&eEhf(?{Zdqe}Yw^Jn!( zQtR)rRVrQ=SlMOVOkFNst#W$ofZyZP)Z4CD<}~bi!zt=eomxuhqVN)gA0oB9To}o? zr(O`K_yLn6u@d6}FqBy^qh{EYO3xAtn-nULVn1}15EBVx#OCZAh5!ZEF%ID6$+Gr! zTQsTQN{JE|XAJjPK;gX9<6q-UsZ7DWZdn)!ib5Wx{faElU2}0MCnlA1Ao5r)rZ`Gi)Exag&{YxD(i-@ ziWAg+!o5P5&ir|Ne6th^ADd6O>i&vn%V&|4RQn{JDo%foP2DS=Cg?Z9{2AH@DQeDl zYUT%)AA0H5+eGig2(cq^>QfTF+7O@zg4?ud1BpYEoY{ZBCtqfWJzQ^pEVjli z)K?WkLQEWLqN}pCv2TP-ZJQO?3vImLc#+m&2UcVeA_|0s$2VA^mXWWKG`tqfbvM_& zYtH$OnV3Ej!E!{jYMvd%ccn}jp}(uOTpGsHb)_;eC8Cm8Vy3wwL#_;>yPg z8qm$e3^gGMg%rkpBXCBk;Qn8!Zd{ii4B0KcGN|jRyD4r|>+8^{3dz zG_CS+^GmPME!*eMndBl-R?S)`cDdgoK9l`Q42RYZof5sXKCma$U|e ziu6Y`_O-q_Y^;tR!J5a@si)AHO(~bIfYVJ@uZN#wTFZTLMB3I#mC;BUqoFa13V{^x zkG7u$Q7z429}y!x@Wh(!5T9XvO5wUE*P^)fJGx9SAp05S4l1c0;eFimJhE*4Dc79^n)#h03xT5 zu!5OM)z^gCnAqCa*TwqtYhP|#%iJjrff;HV*tS?AuI|`2{n4G2aXofJ?-yjuhaGeh zwQs5Wyj=l4V;sUSvtjscGAWUMt!>V;@;t8Dj2qR_b6D`I+ zbo+$mL|NzxltN26`)KX5aPC}y+WGb*>k4Rz)-PQ-(+bwNDURYFzs#)9DPA#&@s_9* zCmhP!=EQ5zyBr;^tb-9V9ZNezCqqsyu%~!J>c+;NhB+DypFdwu6cu2C61+T_G?-id zNZ@3uW@{+samQ$X=QX@Wf9UeKm|}ez8Al)PIxyhd1-RIKLJ{Cq&4ITyLOr)hGiVVq zTDaN*&(JyjEfIZeD1ymBH{NZ0CdkHeb9aRutr;f}(cCT5=o= zEYDS7{uRR0Pnc$m!s=ibbp(`ElvHOuBV8G3VPhoiPicy6Ho%oEhC>h~#vEA5KsRBq z1$VNA7w~UbJXsA!QQucG2@QWuuv1YbDw$XzrqGG86x7O@kmrLU(i-R5)n5^ofu3vD z`E+Y;=Ohz$FD$n^(bgeon>tK~FA+#)BQv?0j>D-ERRDUyNJCAX@dGM{ldK&~izQIl z7Oj0Lui4@pYr}2Nq?Mz|sBt;@2<;t#kdpV0A*UoT(%Ra3vfOB>ySsw$P*sbyuV{Qb zE)F+ZuKh%`f#@e*s7Vr5p+>JJ)4CS(pRB5EYSefLx@avXYF_I#i!D#>!* zz}&zv*c8SGJi)>wv~u+P{UpsuJZbST}pC)-|9;X5Q;jpKTEl zlOwAf*gxKDRkuuGJe8zMPC3ShL$twrYFf)sl~vu0WrI8{@~)#x58x_URdEz@0s?w9 zi^oCjGBPqctb}O;axxfPbTcjQf$`7KW+Y&SgPGx`^7x|99ym&)(TjC^Q#JAH!$Ot8 z0t}Wq>O8E42)Sz63Z2M(4gDAkTC!ruAt}Z|PQqV}^aAXI9rm-6xJ$2Y0|4c*_;Oi? z9cn_0|9t2KA2C(}(beswQIG4C-B`C(0`nHoB1T7|3W0tVwSmi?Dm3v8b@Jov_@ydE zqYBNw1pbe}5)!C@Pcqqz*G7>*0j1>~J z>4k#YS5ISI?<#=IBq*OV05(A~+lmo9i*X5nFVP(YkdX=4XXC`IyX|#R?_*zhm~rpR z%#57E(xNK~#}hl{1+|zq{pQw8T2kL}czr5>6s)Y{OH!xwuFE!9VQR0bS?J&HhwC{)0?#ORxb|^UUUPkPbW{p2Y$yM1$Qj za_CUrr>jC^d3%LA#r83Yo?9_BJI@vjwh|rnn+4HxQMwUDBT^e^q7$R-$o;LeI%uXj zUH?tlpAf8)gk=qGM36~`gap7ixW-AVfQx?9k-cj)S6vWttNe2MfuwtT?x!0umB22? zYa}FFgSkX2jS3T@@kq1%;)lDd?CGF>L9yaBK%7Sn}c+#|q&fL!#Qk=0! z3gVx$V-=T+5UssEy3p`$EFNPf(XJzU+{XcGcN8s$6vLU}#?`t?^nshD&Pd5@UaOj_ z8gqWJ`2>W?uq71qw0M|myFQI1)ItammD1ydOweY;AR>~?RY%8Q2@W#w^#%UD3TxuZ zyOTL~F%)Sb5i-o*4ea=f6%Jr@AqtKrcEZWelXO;fZF4%3O!x|BS-p!psBBQI?apN7 zOSE9EoE2Jj*hFyrm1Bb=Laz=&^nK!LHbBPPU!+*OoIO405!%mX6j4<&(V)bk?t>Ij zV?fdJ6?z~Le0X#qMMV?*>uaHa`SzK2Gneo%aav~=Eh6U?C^qrM^TMZzvNc;T$l5Dd ztu-M6M2M8){x0R^J(|mH9c`j|NHzcjI@d>Wh)cjE!l*ETjtyCpZ(evy!(cG)2Cieg z6G7p(!oocC;LNzVD?ye1$Vi|h?8sT$BP6V`w@Z6#U3$Tv`vDCJWz%N1a1K#2oFW=w zrwXFdDJ2XRA2sggCFSNB#qV}Z<6G!cYH^VFvFZ)ygLOt~1F#%xAB#D&17b#dAD>YZ zL)XHDI|a91iY|r4oI6CREGy`t_8l{v@S%Y8OU?gScC)P&tYH|#xktE7;el3x!9Z&6T*T;+9kA++6WK0eV4;L=dex^V zGI`cVrLgGK>*r`B$n{z2CoL@4a5x}RYQqAuPQHz0nhEP2=T%QQuPLr zE>9=lPg1YTIT^6{gN?`a1`t}23pRr_Lve(5SEBw=COVDXxH zR33dS8B;7eOKiOC8w;c`yuWI+MMLX;U;j-ZwEXNtRGkeUfMhB%FzuCdLWvWjRYzwN zMwFj3<##c4ekL%}PoOq%;Qx9Y0IT_IwgwrTm)*!tY_zq=fYvu^Z4@Rd>S7OlHOJRt zPh?u-uAd?Tv7@v0+02O{nNee1M)sVCuveD+B(Gi#^e7;XoUShgrKNl~R=YO(L219@ z7R7KtEO2lVGqRvyZeDtM8R>YT;py2uFUCsdutEnU8Cicw`b?tEjnH2uHP)z8FaYcI zKh5A4>gBZft$XSOcyH3)dDLPv|159I4Ig8wPzyRN!+dqCTfo8v)=&OW#x>W)-UmgKp6f8iQr#&vG>|ZF-hwLhBn_&fXVKbAPp&L)Q@W zy}UX+%7nqgHk{KS`bfa<|7s_fsOeAxC~qL3#j6()Zbn=m1;}McREnYy;zKGG@G_ zXHN&?epzmpBDOZ7K*f>{|J1o0@XYm>?P>71G3T38{@h_YTnV^gnnMu!%qa>M&YesfUz)H&O^B0MX`Lh5 z6qodMWV2)}Kcs@_rSqed;~`%ZCo8F4AJa&YD7hx?S&C=DU zYuKScKvur??;fy3$E*TKGnPuu8m*1t22BO@t+jm=lwMmN$z7;K4e#|VOgHh$!M~FM z;y4_bpry6lP4$GH`$n6p0a*2ODVtPrrul>fYh?uig|n!P&Ag@%Gm>Nz;!YY&bJTNK zaWwFXn#NQYSj^9ddTC3P+KSfFr8-&G)4_v%7ERzm>m>YA@JbP+-uAWVkkDfHrM7wI z@=qoL=a9CR->~ZGl8LB185c;75Z`!Gg85*-IUPhR!FmIb4h$NaU$E+#e!|f8@Jf8V zcIV&6=^DL#Yw1etYK4OJkQ|L;FjA&4;kyPTS}SK9tUs?&_K~#<{#o)pM~H zOOnkEK9CoJN_}C?rL}4nN!^pDT9FFZa!KuAqzkAY!3|M zD)BKG)Q-7dFsPkC_<#p)K8|tlT3Py>dch&&yDKR8#Nr7Wff**~Lj`eYRZAfe6Em`H zQg4^$lRQ#O%{XgH^OIDr6E!oNI|&eWBv@vk1R_^xXpWeHkN(>s`{&U8+rn`Fu^|Jw zOVsE<#S*anzkGyP68Cr6{~Y&)@f2X<1GY~9v&nxuO8^q|-;UttpZ`6hy`=uX|B>HB z`lD$v^Ne+!ziIngXnUPEOqTuoTt6A2`71{O;)LxV*gnVu0~! z^h-nxVI%rx7&QRIivc@f0r}sB+)B70(25tlb@p8kIRm&y_#GCJoC` zA2Z!jsOfP;#C`-6Oq&;J5o@B{LQoPaKTcTWR)oXGI*xGnVcPv3G-w(jom+FWhq zzgD~#YN}OPCB!fQ#$B}QeqN-^pk=DTfZ^d&R#MWMpU4BW_4uOJ=b}$qJ&1~IsyWvg z^>FiS-^s3j9ov#cqbyZo{qeg9T9~nCwO%GlaZ|v**}$r>YC461fgZFy?cVEP>wEh+ z!fJAZ@8*8tc6Dy?3_(Ka{!!d5H z^r92qSX-(Z9(i_pw8yt}@@3ri)$nBIY2j>MNA~>eck zFX(nFKh*N%_VMG0?$N$R=}C_=eNjskG@=_;ZLf3oek~Vh$eXWTn*sf1yVm|Jz+u*k zA0ZfHJDu<6Xys|=I`%`^R;;;+XJizJS<1;>x7G7BG|x`T7QdxQ9IkWk;0~5-HQj2X z@Vho@_FeA27Ylj^h8MXt#_UrV>cVwOCtCSSqkS*m;i5nM%o3RKpr?shjUk&dJ3x8f z4aArr!Pr5Ljt6Bs+5nciy1F@8q#X7wz#a?hc`>9}%iDVpa;LP93Q$|T+?MJlj zwV$X=lms){ngrJ-=jA0v(?a%5LcO}P5P#^Z7mcu=`2!$4%Fv4A33#N0pMcFzqD3Wx z8m+yRMN!+1+4meM7X=(@9JN5^4gC&-OtRgZnD<@ zz#Bf>qpb>#`8lz{Z}(y0?smFH1^%m#GX2zQO%hvcKhn@7uO_^D)%LcxWxmm9=lq*s z%+5p42OlUEO`+06liTditTnQ8rQcTSgXN%^O-2IpMYxhj&KGC**95gJ;m}x_lm8>?eMk_rM@sqP0^geG@F* zMk!XODuZ*dc!rHxv*>@iWay||jRB9Xw zhSL`p&Q5Z4$%yZc>_`)pmML`<1-D#F?rbObag*(RKgd!G?ot}FLV@A=`?hQc3w6mi z(tZ6(2UDwl9;yz8G(DAmksUdcmPKQv8Jl9Fex3I<-z{#+RQd3)mOj3dzr?`ZM7a5Vy=7wAG_)Ljr+Rr6inibC$$oRI* z+79K{_o> z2FzY)SKrV>VZLcDg*7h}x!n`y22VY6o?Gpebj#8{<68A=X2y~T-p2yzh77gDa=7R;4PJM>g{$vNwIZo7+gcx5 zx?lH}7_M&uEzmzO!p~;`kQZM;POd?Ty_VxS2+Mk*VejbeGl5b)-F)AUWQQ6Cn!;0s zM?iCKk+c58V*x({^t=0R1ND{1a9!EAD+LcT;P4J@g1o z7Tz5g7{1MG(-P2?KOVWk=Tog7!-Dzn@J-~VV5RybmX3R?k@AIKv-E}K?6S~)AswiF z4aTdnqasF%vb$Mi&x5=dsnTIpZ=rx9wVIjUYOy30jds1f6+Ua+6v(*yE4bfU&^{ap zs3)KKPPV}eLFbLjU*%6*c>eZL7ERjT?m9IeW;wT#0;Vo5hl5dKkLSbw-cmq7Y^@G! zW37Yrx%aH4_F%XCVuSyg&lB;p!&4OnmFx8aCJqz_9Q^8PNpixF6;=%`?P-y!$Ab51 zeosflS6c6VseT~CiDW#ndkcp5W)0OW4)*s~c*LSV4V$ZX3l0ZTXTnl0&WcljsVHH5 zjHZ$$uOF6u(bXQiJL`1Osd-}%p&=y$OYHF%5ND^qQ1-gP5N$eH#2{Q*&B=WAyvgs$ z$S4Goka^qdzw##|(o#jdF?Dd9jgwY%N8-kOoeb#8K$&tVxN-J!eox}(T#NhF^n|mQ ztI!V)OeruVXkxrsVdp$E_S?*9bj;1e!tq$B4N4JO)V^3;We&^J?T4Wv_d{#}#rG(l zg~06%??ThZ@zNb_WjQm`+tst;uiiRlX6R|FYw^Gzv5m zN*jP(Un~-dBwM_A+DwI+y^N#~uS@%it14spAhb_O%sR0Bi%Qt-wmaLe5knGKW9-n< z+#p3blgH65Nn#I>gfy6W&mfw^iiZ*MIaAo_w**yq_~XaH4?^yOf&{OK0Zy!VOi;^x zpuKQwwaJy#jRAZ{Hh0{x(J?d}7$Q|lvACQ3)Y>gy*N4T;2O$7jEHe-lxn5nEMLO2Ni65x9`QZLKHFl9HPJsm$W9-SiZpfBXWmG;jjm_fQc%@yZ)`wf_| zt<|aTQKuS6aAI0+U*+BRunX~n*{cymB4H#QozT&6vxs4fYHal}4pwxNN0@M?(uKG|bKaH@ej)<(c#WwZ|8-3*!EJ zjZNQHQ!@+~Oz@CI{fHL3<)*AZ6`y`CNe~VVwFD`#<2+~K^SDO8$$Ko)`YG$QrbcL# z-b%m5Uf6Y^!se>Hx?r)ztJ%JbeY%!SyAZ78y4FOJ=znGXrN_Jo%CnjHDTPU!D@W-H z**a(-=#j6)UOs!sM}zh9pLY8r!TB}(_DOIqg&64N3!e@NY#@~{>X18l*Qht&OhB~B zmrP9fHu&TGjlX`i|En8=#9>b7cd<#IFJ<*!;tfYg`)J+sho#8--acW`DsJ|e?Nn$j z<3vQ=2{)bJr6YeWTZI1&!+XLq`F*m?$AM<1#`I_$+fq*XtFlah0~BR;*SnU7u1dLh zK7Y+?Hyj$?LOxTj5}a?}(4;5>SoOLEn$fY584_oWdXwy1b91vzXM@!);b8%PO1F0Q z_TGrob&-=#)mscC81>7UjSIiEVagrQ1D3J- zw}oca0hcz*p?*US4}Z$+)3n(X!lTu`k7@4k`g;6~D_PESe}8e;yj@$*6R`N3v;S^( zvh}`3_56La?`;8_;d_}k5Bjd_w_D}(0|WDkK={~;fRbo53-WzR6HT!6>)*fsoE+`t zH5qKshj&W!tDAnpXK%P0px>UxOjJTkjhYtM_xv-LgaJDd<3-@3tasF$1VJ=hw9k*@ zHJEq)EO|G9_+?=e9t-(?%wnc=X$Z3`jsYe9OOy(%LK z3a%K7quqZ_;7@F0d3+$c|EMjF_S54KYK0KAR<8fN!~ZXL@d#ta+gD%dKW7a15{x3W zC(?Ccb6u0QOEVn3ra8I3zIVUNaRF*JsfD7!N6H-GoOn3ZurNBe>eQg{sTLJn*NsQ6x+7?o;?oQEky zKIojB)luTVT9w#eyg4sTJAd6m#_!qJdA36M9F0!@*UNtgecIFkvWfmw99Bvw{a9qv zYmq9QIlDgF_TTR>JcQuMxSsVQucHlLq2$mQxBhddWz4t{{V>III%brx?RV9^USJV6 zI<%y7t#>uQSD~Z*J(MASwRU7y$kemY;y=HzKizEUaFbe^Hg<6H>pd6Out=>#Uw{91 zt=tLiMgPs^Wd)hoRnu0=o3pLNE%r(mP|e z?*Kz$e)ni1Z4`DVYPq2i0;yjcLJe7?qoq@X(gNOac)nU<{>At0{9-0D)`9g=%G9Yc}up+4Vq zz36SIFFw&5X&L+sh8&}SjQMKS)?u>%PnEZ5AMmrnfYW=&OD^J}sGwljw3O1^_@Grd zu30qhS!#VUVL!_P1dT_w^&{l7KiIel(~t_^D#ondK6xC$$VlrtP$Z#3CN`3%tLUSi zZL+*4;zm#7K)HDR;*C+Wb42)xsoA}dlij8E;#iS-YTO~o#GJLj^D1pm#x%8epk=nrw(S%iL$?WCv9bVQoZy?j) zKhT8h%pU)2ajwToG(B!VGi4YW@~7<$BtHl=0vS+K4x$Bk3@3K5cThB{{{{dLsP5Nd zOk##J+bRsqxTWzsxUKyOESP9=J-QuuGr7rOXjm%8c9W3zTgFJ@VYAlZ0hXDOoQ$Gn zV;lE3yBOTlid?_x8MWXh!IOOtAoO^fiDEodY-m#^BRqio3F70g%%XhvE;_KQIS9~~ zfu*=lK{1_z8{cC^OLJrXl%~OP-%exg?Tz#h9vMDZMxe|HwaAw?Q1BpR-D=O9N`QqF zW5XSeu}`Z$hu_GhLjc#r2K5H`1#?LpKtFvFoSPS0*c$g3E3+bvwTga)w=*xmVKu92gu>Qxc0rQT~zgUd|1FSYrX$_=fM#3Fk#9+biG~+ ztWguAY}Hnri`}`St!D>J1icO|X`Z)#msRYctk-9oz^kd;Ngp+}_{`ITlUoB{0Vk-V z{9mHX7i70;5)PED0t>sw)z4+dzAIFFEba(cb|!r!pG}wW@!rKrT-cs4o85SE0#hFH z`zM~99J>Sk)rta?09Z!&VO3UN$u%jqJabBufZ%_tWk=VJ6cgg3V)pN=@+Wous0oa;ZUcZ>e7GZ+6<)D#PeYpw)!q$G;|? zn(tqdV}U>*tmk+Z2f1BQfWgi9SD7lHRJXZEx}Y`>KVmB6Q#S#4eW;cF^yK7KW1Mz~ z>T5K#%}|O^ZIt0DU!Z;H76qxmtBHq>$kM zIZg}WO&mie%@f8uD%>+#jPl4*dy>;h&qsQEwXez(`{ZST>W~h9* zEaC^-xTMU8VzHmDIr{6+Vn0&jRqoMH>z}(#k3;LYP%ggdsGOw++tEE4GB!k2J&wGB_i@19Msp$v9>vF(0DtFd;Hye^9-q&EX=RP| zxLxt8@tN3Ce_(}P`#d6FI`QMV?jxUl=xyk$ z+C5UY?D=)VXEu{mQ!_S}RMSVqVYk>kY0TMwW!)Kgcf)gr0JQhB7ZlKJQ&}2lv(0*D zUkU&{dwF^_k5+5f*7}4yTh*Vo)jC3LY)G z_oKM;EBuuLm2}VRQwjR&Lg)3O+?MN^p~Zmq22?BCrQ6wcG#JUv#)neuIu8!>8_wTL zGo{1@FeN5DF;ILgMKmeb?!35P?ZMn=nRJ1Pv;X5cLMv^m}HW92|^MoN8Ro1eWLy1GgkXVJEs zZ$gCg+v~U5_vL%F-ZuC=WKZhVUOBHhCQd88%;8CInC|TUV(rD*L@D;)zlK)3 z9uj0?Ue|wnM#Ews)Jwgi-N}U$RyP&f6l2K6A;s={>&26}!@G#lp-hR3ot%zY_g!X# z{9PhxaUNn&t;1plu<-JjMsdi*j_zV#1t0`AHwMy7$qha>jp^0>F+`Lf1B`Jt{Sbl$ z53l63P220S9;QO3;z|3N>q(UmDK3j+beio;t<9fVSsmC!xWgQ+{VaMdq9Z_&ECL^o z*nNKv&#-9hjr~j!U+TyCMG7wGnsL+l`^g`GyaVGRPnR%a(c3hgz1q^;n7OF3I+;2X z;8UzttOW*xO--dFwJGpPp(rRHO_4^uGoTJq*yqE)JFheIr?D|JbKCMob|l0jmRDBV zPMTt44-yt1b^%31MkGG&|IXMrx3e5i&C04StYEXb(LZ!w(U?)G=(r)|! z(x*N}`b59W;h&#BUshYH2sn&y16{gtD8;FAL_9u<)diH-d$z6ub*Co)>;(ugpBUR& zBwlX!(TS`K7P87G)&sa049Te`YqM6MvFq7cV&D3cr~c`w$=>&K)2B7Op(gq@>||Y< zB^q9nEx~wrc)-N7uOGA7X@AY+^zzYLzqGk+g+>0n+1G43yt;Ni6-X=?_*1_b2f%3r zq=d;RPFh;m*@=N6hpxW9QGaRrM7;Q6(QS-~TU)`Vo|Lq+5`d zR6;-+=}u`uK^mnUQju5)kPex`tuy9`y4)zjN-nf1JC1cimYoUCy&- zKl^#_z2E!we!b(7k}xs(IBto9?RwB(X+3R!+wD_r1P_@UOJrSLt#B<{One9fE%a!8 z@%(VjQoA&&vMD^A0|Fqe(dq!d=TV+m=^GNifDoi-Epx?M8JL}V7ntR2ooPzlv7r?> zGlOH5i|-MWZ!N(WK284m74e2VFgVz1xM$K+uW+=kwhp4|uAkifn(S!bd)o>u(uq8s zLh)=ljFTO=YL_&G+bQ^>w{7%^7v82Rr6BZme*vL=c)Wbo!1s}tQe>m9-ovD$h!R;0r?fF8Ki&$!)IL&hH=FORF~r?okg`lrLY%sDN&6LZP9 zk`$&Vxkzv%>7siB*2810XyxKOEfGX)x;sGe&CPq9>`sqnTUEOYeTgF?CqLYoGzz`2 zRG6^%K_O%$`9nVS^c?`*Ps7NOZJ(MDaB6|Kd!;sdahE5v8K%8EBD%ynwO&~2G+wRC z8~;UmgI-OnM7_V1h4UA_EY9F0X|jxeK)7{!JYm=x?&^}p%s%+;F+2dQC6%{wKO*Rm zEiG;?u-9`2P1YMe@Y7p2Z^B*+BF-D#TyA~*7$CzEV|rZv=Jo5WoGb$S%J&cMh>MFi z)YnVVt=8=MD%RG41sT~qys=5% z6FE9J`U=_9===5oeMDw1bEozz=jV$a+HZl?{K_N)a1fL+a?d5>*bRH2>);RqBGaug z-&hd55^!5}+@1Z|?4mtbLTRbZ9=+s6smnJD{(nXFd=UmgOBM~hgKbw~0sr;gNAuB@ zf}E0Wvu%It?dCjgOL!}g-V4^)LhTak`L9>~&O3ojoJ{MRF^>`w5+sk{BlK$dN9UpP zrU(9bR9U&X#2t}O-@@ioly+uPpL+J0l+n>K0Cu}qYDh`q?JfXbrfA{o;z}^_DQq@6GP2g} zxZu+n1xt}p^LHaJn?|QCEjjz12L5saOka<4J%c(#)vrk)ulW8|oJlx+)qs@u4rZpCPcU#uEzjc~+zCr9-6rv~m@FHOdX7ci*}idn@xI&( z%zvIv{lKasx$yewJfhykhIbYi3;0X;m$!l(PsR6F)-C=$8u z%Hcl~MnT~cA)z^Q?yJ4Ins5;WYlhiaEPgq$*@tH^W=|UKh=_J~oI5-ues;dY4rX3iSlR4qJq%`Tq>bc0} z-Bl$?d8l@QnFa!tE)EW`nd17=wB~k`_ex7C z^PA7?28uwESYI;zfIln5*UvAUga_J_b(c$3)v_{2EVBa~cSUOXsRr>$6H;D?{+hB^ zGgY=a1seID)Yv4C*YW+>>R#Oir(~`|8o$+Q$igZz!^z2Ju5Jh5RmO}#KX>7)S+0IE zJ|=^@zWA~w1AYA_GM>v^ei33b8>0{ko_#p1-qHLGZ<3FMI7rmn1X;qPo10_Bv(0ue zJYljk!v#+XVWi1smh1+ph0RQb)$pd%?2}h((B@N5N5>=YD-S>jc!vu&Iqqj5whO6o zN#YYy_rJqo32dtY0gua9`_v1^PZV{lpdJ@v%&(f^>6~~cI(ob|3bL{mBVP?CqSCw! zUSygTQ>2yVx&s6;nxn8jxkD`Q;;@o4mNIGWdV5mn?&N8a zbg5DO#md*g3AmZ3{g zai-Ly)lio?x94ep9PvQP@yQ9ZK}o~#%QI+lSF@w&I|6)411?&bZI)#HK$56`I82Ya zAV$~-RxV~P8XC`iJFKk3Wsrk*eKW2Mr<-Itcd|w`D{@Z2YXd7{LVN-u`M9#TiUncqE-Ou7A zKFhHJA-AiHaC98QvtA0?nQ!k5tR0W)iUo5A6TA+lhP6vc(J=2c-Dw^53#FisX0CHI zpKR>algmEn9e$eo6irn1-SW^C?~VWfF@DA%W#9R7Y7yfUme`+z9%}e7!v)$1LP-uS zq%nG8N=a1fupv&H7blSta1gH-Ub;~4F;5L256JY#_1xQ!HrD2`8h3IzeUMRyLzXJy zmKYFQ^x_nn#21*Sl~Sh~sa(>Yu2y(WU5kBB4`2Enu-=y+t*BQKB|`^RD?>(V7JxdX zl6}VBalaQAxtux_$_fGhLqSK+uzA&Z(=Rxfmzj%;haRLKdqDt^*8-)mglYK4zm?oHBO@!19S6LMu;c;{QQMf#ac-b z_e5-ux=afv{QY8r!h*KPI*cbj=w^1yc&WV~ZaCuu9O$)6rEGYwGH15Ry05RVLbsTF zqR4V6Pgdn^K)WSBV(>1lZkfA9lfugf8Nk=057^M}jrEd_=f8dP@tYoIkWZFUPF1fR zMrJPTPP=V>g6&v=Q;mh?00P;$a&tRQtc<>oPl%cAB^uFhrq>M323wmXwW%Y`V1;G; zWc#NvZxTK}KJNBxG>7ReQ87^zr6*c8++eyTSZ#a!zP|jy9nHeYb;K2hTE64h-uIFlB=e*{Z2C5BjSt#wA1>P0HntSk1F=ab1n~qI-Yz9xWgWxbGGu%IFQ;ML-{E2MX2U7AH+}g#40` z+<5F&G!G2pH4xVZOqEZ}bv zf#33QivauOt+h0uKG(JoCwk(PN@0OwoOXK&0_J7&3%8wwBN{1y+o}t z)~@0GS(hrj^;M^G#nx1<`^qNI7B|V^j%)Pv?>ZPKabo>Ee) z-`o7FZFQSHkAs#Nl?GDbParna7n2QXUI>eeF%ZKSN+w){^uGlYaTS$J!y%nm+OZ`b zNdDRIZm0HeO}j)tr7=H#YU=FMr=A#=X;@v|=Es}2IP@D{yzB&pjX;>?GfZaAWH7k- zMtBfg<6&No8M>{$4j69OCsY$oof6tXij=Zl8n zgIQ*hRZ>#Ai2MHXUOfdGQ!J*%$RPp)Pv1$Pzd0qnl*d+55)^(D)HPy>JcElQ#1r28 zMNNq>F9{guXU>;Dex|LxDPG;(?Kr&bT1@l0YVR`gG&s{;lCpE!Bq6(h`O)&94>YUe zxF7)P3kl$PdlHX|Cbv3~J2a8&hZAF8w7&I-(VJ{OM0=JU0W45X;2k>OdMy&_qrlL6 z6`{aDPn)2B(m!d`G+95XPLBFJ$!3149M-Zs=cZe2BjPdN1}H%Wkd-7!MF`hPewY18Q4|j z60A)+Y#P+AHUGvD5)#y-E5aruBzz@{Y|E66Y!VihB)CuExHA)4#AhL9UllN|VQ$(vn#*E;TA#4v?NdN(vFROc&H4u<@>)EzKU;g%M2UPVEc~DaIa}IXpk7Y|@!&g2vg`B!g2^5> zPvo(A?AY~ywhnCARkm4S>H|^;4#l3BrR8RB8czS%*49KmkLgIbufrIS-tRhK5KROa z1-FZTg4Ckzjl^$10Yw$iT6CU2e}1f+;js_JbzjglQ7ko-xhn&0|)w#sqEU_rpY*raVTQ9EBX7wP?h ztE`Mx>r8DRS!gmFE*Jw|&3y$;645HE;CSYAp)gtP)-+dsn*6@`b#X#s;!K`JFdUC6 zKw|?Rk4pIRWlN`~j?A<+IqO5BG*gOnZ|m-7|s2ZDyQ)?w^*tA*bLq z(EvlxK>|`K>n$#wbXeGK>5kw`#nua;TqhH&6TuQ1x1^^{>gV-XEduqHK9}kvj(KuERD0foB1JN)|GrZ2n)u} zUJpeS@!3Q6_ZHhQkm3L|EaTj4JIu<;a@-yf5_;%kiiF|uoUWZ?;8G?&vAS5ru^MAJ zX7uq3(D>C0wA;q%Vhckl&wQE9;kxQWV3a}E_aL-nNJRK6DX;T2nEZs@6ane^GV2_$ zH+O~`_1>yP1l-2@ID~8q=TBYuYPOQ$dswPpD{uv;PIGvmS8bE_%%)?H!*6${tWw?o z#K+Iwb8TQQcSb=Gl-sKwjP!BaLE0ELYkj))khW9W<)UwT&_Dd*%L3fZ#Rh4+&DR@? zti22;l&r8mr?#53c~P+!cbKaja%Qp~z#*O1{FbXTcJ#?(28Bf0xL&MdSNk{ZJ>x#; zASRD6)x)^V&6Ma<1`-%^o8uqSa6RkY;)Dc|7u%!Z((e2ht?lyD+9ZNkZmCQ5KdaUo zVD?K(A8vh<6S9A6w>7E9n`abG;nj*_Xvz8P*bz40bB+tuM@Rg^FXXl#o{0omyHYJh z+<3rok0g~*=;6}R4Qiz`&r9{*|EEJVL*q=X%4T7Aaq()SllmMEu+xQZ+<;y2i^{}+ z5`$*dDg}#JQwa&5QiEnCNo2Bs$JM1T-o-It^{=zDPeVfyuud;>A|l;<6W=D-vGDm| z{u+ghiOKIZ31PRTV=YkWlKRMNp?CD~Xu4dvG9fm06$Xy9rB}$o(NT#%@&$Rm42ow> z>YXP*QHPTASifkVAGeweT>qWjd%QUw$fPMp8_8`qG1fT?pQ+s9uL7H=k0F%Ys*>Fo z`mDBX|7Q-46(9hT|A%b9S)I<4jTZur12S4%ESC);zx#M+-$ ziQHUWU9TR&-OUr9gk?h`>N1AF0XlOJQo-ETun{qjT=|X zXQI&4&FNA*YPfi#VOnA=8OgmpjU;Af=3J$ejj?PupN@WX&x>gve>|)u8T;Ss>nz#@ zQ|WU_l5}@kMRJv>Mc#eW8J!Hs(}|u8qZ0JonQ@AqZa}tt&stuF9&bLI-3Il2s$pmF z;6V)H-{}pXYz+kJYwLt7zfLtiJ~Svh)8I-_07YN6N~+@~V+tp?tHA;X$_~87pvg2L z8L7w&L>b2eN-jxYc;%=Tc7JCK0T$p3Rvj-=4i077I_OT~i`yVk>7Rf6;$o9OD@Y2= zdF6hasNIZ;ijsW9G#M48?x<@pVW6GD&cz%UP@q|Ce>AoK`SWLTK3hLhs`|+(XRED7 zo7AU4hULX_`Q3$ZJWOo&0?NX10cc-!rk_{(I8m;8G4U2qkxW2E9&e2cdH!DB$7c*& zA4sY9vO&%o@fNY3I4Qty4ARi>si+@z+{Ay)3u+Mh8u_L zG(qzCpi#}!cLy4&O9IW~@A~VGy>`yr4*e=2*{$JJf?TfaU%k6ec;&%QflJeHN!)p0esneR{5xAm_-J+aIw5fOX6AAM66HftQGsT-k9>Sf~ZX)R~6 z-Ys>6g7fLK5a*8{9Rmrcy@`Zg1Seb#-%Kug`lQ=e)K^BsMJ~=V@c_awFmO`O>EAov zubQ^b>~t42Ia@0!snh30`D$w+l=u#-+qF7Ra!rlmmlzey&AW1rt$^lHnmK;?bYZmb$0Rfd zwdeLMb?$rFEP!{s3z2V&xaOIKl2zL-)Ph23;rNWRMozOXmyK_SqdBw}q7v zunBu(*_M)a3Y#;P<)~c_8}1vFPCB zR4D=(=HddC2!=nf8rHEg^Pi6@qjz+)PireFD$-~8_~R<4@dpQRTOI>fwX_G%>dB5E z7zwqorK6~~7Xs=OJT^RDr0Ox}7g{(`6dV*L4BbBaP2a0qWiBPjmdFQx)n2g0XRFH^ z3m$$)wl3qPZyzRMKn?rQC*Y57Fh&vxUydfat0WUAR00RS?&>S$H@un>+ zQQ%LE=^~8wBY4X&r;2p>1aQw7c+{!3q7L8_%D%EQCU+u$IWYhJUX1q*-_;L1+r%QjApa;Z?ePWKhq znzp(0=Q)QkTVRD##3$p;)0XLs7St@5RFX$Fzmo*@LqY9LX2a6Hk1DLfny8|MrWYYo z@l%gY<-!rGTe&$y_lAdzAKtuq>5oIHfJFANqOi)=T(uW4-)ihSU$ebCx*HuE%j-V2 z4kj6tZX8Gv2%``#k}=r*)e%argB&6c?MT;aOKZMzxHhY3*W_(L>bk^BV81XyGBPrt zY~66?FhY4OgV=KQPaQfS2D;@iMaaRp z=d+GxfkxNL1u?sRSI=GEq)-`cm6#jehvL`W^Zpe2>8Z!7G+Z2w=eLfv6m=7$^CO>#;quct# zOrEMbMmD~~^w3ja62H1A|Ki!Mz|SO{!~1Kw<35u`QVvwT>}!L!Yq;Le@(L?HbsJ2q zwh7ELrFSoLRB|;0W?!-vsIxP%Gc`4(ZL!&f(|c^LH(ymu(q#aR?<`HXwaj^^V>(|& zpV5w?$oJ~tg)>Ylij%`4)dj&~2IFSPS>xiw_K+a@72C<-%J<3|znTODcF=G><#<*K zgr_9Wf`GxQrRBG6|iB(NGH=BMYi(&93-7CRx-|ZqBdX z)1(?y-t2ZzRsEf;TVNcaW7qd;+LD}_Fs9S!6MyE= z2Zn_W)TPA6GTiwnmD$~S5*-x#sNP&<5ZyHcyq#9 zI8H`Q4cpmCF~YeYIc(ks;t7Ozmd8PLOHlWym{)|cJ~u0C50t7|iNLwMygYa(t>Snc z2Hs+EesN(oSA~>}Z#LS%M)y(5?OlM?ba!{Brl!i7&T~wh6vfAr7#SJG_E$$vF+?*1 zM%@b;pP!!(Xf&_cklWlxvMEt)>p8vn)*9Q>Sqq=26Vc<@8goO3>5|35fR2)fY4y-v1y1g#zD?Y#0 zNM%sx_#i_%JfkB#$$hDV&B!PX@(@G2P%ED^7}O3u-}m2OR8G}mAq4GQn-}WCaK*3HwicBu!#+%w%)JKi@v~g}yhoA)D|2Tvr#BNQKny;Ez=M4* zhWrl6ZY$@||8|Dw^Zl#SY{7Z+fd1E))BQ;<0p>raWJ8S#o8Q24#C(KFcr1(BO_=)x zauriD9maQO8jTn2%C(DLxLjRkQ^*TAZEpL8&c?DM-jA+|Pzwj>H`_WG6Uj<~vY1{a zFWz4MC|P23o*oqsqhC2AM!$C(wHwf$;k9u-F}s?odFQi@<+3xgP(L|qU)wi0xOP05 zv@_i(!o{^QQ%P>uJeROkbAhwzT2r4Q^S|xAR>)pEdOfH+2a|oI5 z3nH>GF>${x;9_FdxWK;qV1!>@x_|RKogyYOOHoYXv3GDF{0CIA59YJ|S)x;@8%96Z zjAVMswAW1Uf>kL)3*cnhV^gidSi44qb90-2IKFBay%3z3eOchfx0gT-qM`CQ!xSEc#zr-RoOF^6deoI=Z~)y*ZuA$vtIR5kp_}keZ%_oWXRojfP(PtuaOQT1lF; ziw>+C)6H*keU! z=sxN8D?z?mVBwJSS?e{N;Q^;zaTdTfW zbP&?eoNy5;=gTu@S^@jzAN}Mm1t5&<&w?md^4|VWrHh~v4>vrI^A%UId#psklC*d4 z-qo+Q1VMrqSMYpk+%y#yLTdM+Tn~S~045fe)mb|RQ07+fLCb03a2tyTlU}@&u zCw1;Zjc4#ujafe#9yM3M+Bt>1Zk6>)kjSJ?@dS8IZ1`{WShleRWe}gm>##bwACB*i zaQE}Lk}aMY|97YUM}z#&=kZnuKg-=)g7EdS?@D)CIIs z?EhenYufjJTk7OLvi|RGvbb^tQ^veqX)R0br2s$0hlEnfLm(PuwBtpxngv4@7O%^m z>71{nIlu3BV;j1)UC?$#lrgSef>zO{y<8< z8J{IP?_9uk7u$|Nd|(Y1WOz6e{L|h!`suEuvk z1*%Kktyy#z8;mn`F&C$UolDm}x^f+HHn6z3SdOiCgIeVIVz+dwSYgGDaQ3+-MesVI zIHG{A@ffLa2417ILIm8RE8@}`_-p{ix_`5+P`5IOWVN~8(2#&w`eQ^~P8eloqg%c- z9_Ag_3!1=B49Wv;k%K)86R01w=+JO3mt)}NiTfgh?rYhO z7^!gQheUQuQoDr9!!|>~RILTNyC7Q~n z&3AO`95>#J%wyd&Xi{*2tx}Mb7(C1%R&ZUP2%V|S>-kJWOU4^Q#F66x zUtgBp919e1&XNuf2snabVr7ozTEGmUTWnCIhnymP{%0cA2fiQcTymE>En3jNP3I|+ z2ZiOx$E_b(&}Jy599;Zam1`ecB1Bj;9^m1TJ^r*hn5$JxQPRQbxOmf`R-B58|5+la z43MSK=|*%fzg48B z_V>7B$peWrMBaz9X@}X))A@X$rA=ksKc)}M1mPM0K@_MgcH5^i(tgu*F#nN?^G)|F z2A&%hs#BHL#fpOSd^P$+{^&&BY`?NJy~X5HE4#(h z1?1LiR99$U+aj{G?-NV=pd%Kd+)hDu_pP+*ZUmfPM5CU!(n|V~)=-Kd|FgQ6Vgq_H z?27lhqq}>iv$KX)ddG@LWwZIMV-U}jPK@OTtFF8AZrPK2{dlG+MNKXBr=rZ4(q;xP zFDR@rd6M{_0{w)Zn9Q@YWbGHr%|@ z31oPw^K+~e8IA>ag##nlwvBEesQ_uSvMRMU7~L*TN;;Nt8;mGZYPdU^4wSqaY4d+w z-uPqTdNPv~`sa$nAqr9LjUFQq~j{~`T215LX@Pqpo>~4*;pM9bBspQ zv;NeSPAgDw0cANDe~Q{~-EbRZY6ZC%_%O9+EH5f%NAIol#=)}j)4qmyZtUOpm?}LH2J5`o>CkFK!(Qcm> zmF1uHE*#w4NBoa|HoEcA(Pe$@xfbtz9Eqr) z-yp+h2Wskn{wxVaL?f}$N46~B&ODIA4sr|3+sp)XanT6BB=Q)R=Mc!}(c9o{PvCQa z!Qp&^emLe)g&nm)jVS1#Kj~iRf%oFbq) zSpw>AT*0nm5t6BHN_?n!AMJ@uxjOUN{h98KwmFwX#*=}LBa_M8K&Ga?(FGB3jfiE> zUN;9oAXO+|Iql}%>u{`JMfXS!q8|)(gmT-EQSysh@;L1(cwDGNGyLGB)vmOoX44Vh zngEA5kFA1{yO>=Qv%My2*eW1_U>UD56<`h0C+jCRmLs1oj?!)CWYX`|7a`*!#jGZV zqUas>9su#Ac`12ZfS?;^o$FtIqTy^P0FdRxI*~72O)XCv?_!mlMN@7&(Qx^~>0(z< zw>IkM&-G~K^-<;SIfLy))UUqCh`;O-%#37E&I!ctHc1!uSl=pTW|9Fv0{N)i#y7!X zb^ophf(M6O3S`9cTyHFhPfLV!65r@hIsv63fLxhUk)@?mw^uYBxfPRTu!thU@z`F7 z_9q1!Tth?x`l6}Z1R^fGDu=6M>9@c-ssvPx73?Gj|K#@CjUNnz&{@r-7ie0HG9+>bI`7hH zYnNfMNT#M%SP?76ixz6DA8$Gy7fjv0)=4Cs{@>Fj{PUcwz>ytFo-c({RtOmcn7;B6 zJ>zJq5Pi1KwSM{b#h>QQY_)nYkwz7(L5)fV81Z|y%kB+!_-vAGfo~0@nXHv97Pjtlqx~yLTSI<>C>ZJa+r7ti%GFH~oHbwz*=f1dfD3fP~zKHd_N@#7(bFq_C z%>HnzogIXaffp-M-E8wR>-#b&l8Zy0_evVHZ1uy-{mQvH;;XY-1c-HN0H!u5^3(@s;XlJgaBEX_5>J!_alNbP2Tr~=r#r@3-sdVp~0T#&$rcr z$~_V0aAyI};6U7p`u9~d!Ta)Sv*f^xL&djLWMJ&jfzkJb{2zGubIck5ZC4a2uD>jfn zTpRVnF^L0YJQ~`00Uf%Z+v$4aDc~6L4TZ|&-_a^2jnupwtLGDf(*5npOYSEHdU=B7 z1f8ATOszWTDeQqtx!LDPG3oI8Mk9mGsF8yz-THWKrwi)t&x#G%oGAwajDbBA6arN2 zzmF!{y=h$I7AF@6AY#6SI=Gpk+lk6-mE$49ZS)|6>nO;EZ&8FKiQF_C9F80WHA*R6 zFTd=}wwGM5D?^>TnAnfB)fSo@Pub7U0t|x6-OeJw831Nbo$>FKft1-*&aB+g2pVo% ziV<<+JJ?xVufPTy*=qkIVM9gP4P%mL@y^>{9IcQ0?riy#$Xjf~FB^y8(?BVUaP2%nRM{#nT_y$%Od zehE*yN2BLGw6_K1zMd{HRwl0YLF%thoi6-0xY)ep{VDR9()feH26A^h4v)sQxn0=# zowQL$x%a63Dvc~Kb14PYOlX-2z%ajU90Nnp3-Mer>ECIgd$Vi&qA7VEie(|@j(NNB zoPo`V-I)?4ioJtmUQQD13Nsuh82Cz5QgW_-v_SJUZ?FL>LJzHhE*A&eY7zi@y9S!; z>o&7d<8tJDLI34?Z=CvM30-7>0meN@q_|j$;42MK0SH7kFC~9i+iVV8592_-jPe zu73Kk_)F~fQ_CUIZGm5;IOO-+hSDYe>(xDDU^^|_mu-2!G3VAOCX=E^X$S&FB@&W> z)NIRfa65`TtALwlQKtd;|A0yCR?6Uvg?WvHUiAW+atxR=}OI zEi?>T+Vug-Z@Pa+Z{In6(+4qWb;L=alPw?JFI8>#SJum6_~_`d;8a5WNG^t!Hbf4! z(V2LYIXOT1<97xcU_5;gt@@Vp&m)&DT7OvV2yIP5E%x_rz>cofJrCw!VhXsAk5!XH zTnmkn3VBeQ3~cHECbrOyFwir=T6>lO0irKKzo?GcKxjflsU6Kuui3-QNQE5zV(Z6| z!!o>jHztp0S9*q*yR)=JY72*lrph!Lq3IL9w&aJ0ee0xAKks_-1`B&*BTJ2Nq)A+Y znK`iOoI(51VB&KwuI0u3Wm`*@#uMG|F79+4+vU&K>2Myaq_4maXEUB#2r5f>ZAieI#~*k>mqnb$?+EQ)O@K!ql5xLe5gWt~qp#-@FuJ=Q;a zcnmWWpn%aeN?Q&CnL!5|x$^2p|9@D?H$%ph&QS9KAv9!9o-kaX{Wz zj-l0n^+N^J03V!!HJ>-${*be#b{GLCpjHNEj(X|p$mY1?F-s_^grr6_d!?-+z>4y* zDrO82Rj5FFN0i)iAN1iMLPw-9V8)d38Np&1t8=3C|VxbSGA~^<6D;M zr;GV8(fgjjaygnoj2&a{^O>(7z?6O_AQMgwFyGResSfwN2#jTWNK3{Obaj;!06@5G zX8Ko7%LQR={LHn1Y^`DdLNE5E0hgM{=k?8foMm%1t|Nq|IJTi7Po6j^kcJjqC3_YR zPAtIn=<;CV<@P591^|G$*aqlPT#7IIOUyJMnh>!{X*uiW;6A%QyHA_dasPPyR6;mg z5~^Hkpj{q7uqz5!J2x3_T^P|Lp$QkqH1G>F~dLe)%Ra@y?5C`i#3u z#BT?M^qr>@v6-(b0In|o9u~_>kPruipc3*_@8#Y}&eV&tU{~2V4iVU}bU0>VSnGXq zd0}CoCs9*SyJYP^(%K3PnzG<7;MM~NnxteiLP|Y1$UsrJ)+4F473f-2OQ0&DB&}^5 zGd!r4!on5S)6sWFm6j&hF!6~0YGF#oZDURnCSk*EftC+>Opg=(KA|-j$ycP~jzVTs zatI>-y2KNj=sTb@X_x|@;(kb92fHgv?ckX3?@>z$RuuM<^|uxm=*M)Ox$8>#xdH0=iryADDoY0(8!ldm$43fJ*O)11BJ=%j+|RRsMKu^t%r@ zn+Bh{Mv(Cl{drPXQlL5m@>C!Gmjk0oqs>FXeVIySn^!Rj}^tS~bB(3xUlK%HoaQ1+Y zcidt!D3Ln<-am{wSJ350$*q8Z^%))k2S4Zu!QTU1R*9U1j4#I|l@bD2F#tp5QazYJ5)R6x@-YrmEf`CE+;G&}q6by8~V78f)bJ@MOwcL$@lPwnf z$kV5(xr=QZYiZ2PvfFBM@n^0V$J%6EH$hD{VAYT~Tv4IZj%;-ghC@%X_#J+*XVabU zNdUD#`%spJ@IRn*f3{RyY^ihfXRYJnM=)HHT&3ZUw~r8~fB2|&y})(j2ee{9C}7CT zvH-ajzH|Nv&ar3>xm=~iQk1Ep(hw5>Ze~KF+i=!BIIl7U?A%A%z}a)nM|qo+W*u=( zd7O~4=}DdLMk)bVI`Bw1Ra^r2ClUoT|03v^n4*@5SSbi(tcneBk~Ee5wA8TCd_lPG zeKY~)zgS&M&5v#xSjTR>Y`T(y&H&Lz&wTTZ<;VqT(z;+IXm9V+7y5rMktd#MOd3eZ zjJOAk26nJO#|y?91qI%snj}ku!ubga8krr@4relSCfZA^AyNgpF|MgfDYOX+f2_g> zX`hQ|WS!;cfjL@jH~IyX@Bb}%ah&OoXEDqBhi1!FyMOBz1h5H!z`s1NfL4Mb`#uj| z4;L{XXTik0|5wD?^xT~Hz)8*R$aK7`$fz7-Ajjo$#_4ec38j#=nf;5Gm#d?m`qKoD z%h}A1>@NXUD25qW(e0zz@*1U7Jm#WcjahexrlH2KR49poPWzZ7&rm2RG+RBlYfyRH zGAIdnw6xeTadUhyWZ8-9$xtns(CCH0y~S^imnc3{W>orxImB_piw|7!jzaU1>6%M9 z*~wgfZ8;u?!|0evAbg>BV{$NKe4-kBs~%$+;sho9>6Rv1fb%w<}N)``64i#%)9LyTt;M zOhja++`pj#G+hr2OrJDFIl5qm1-hDloBm4oUqosYN--*_*gpniueJ>|P=q+Zd4bOo z{TAX74^#&N8$}i{G!<&uU{yx-0xToorUm2>;;;tfFk-ne#*9+ff8+oHF8KG3!M109 zmb^KcTWQq}MtOFqT%aA^^En-OEJoG;-ufPJ4l3kk0v5!@ZgVCE9hVM-4}O$1;Z{Rl zjg>P35J>q{6qr)ijEU%CYt|Yu=x1{>9C$kvlb8T;Wr;aCR}c7~xmpSNIJG$TzxKC@ zC|(;@!y#@8zr@CdM*e-91$Jj1U-LsV;Im;R5^%x&vu&5fU%HmrudM&%x15fesYq|~ zVCL7^{)qWP{O0lIb4me~dS~GHcrF?6huR9j_&}qr!J=CqyV{>Ag3MCO209N=JR~1Q zo3G>`kP2tE{xoH;D_Kn6KeU#29!Ml=1&-Opo0=sG9&k0pg(IR4aA$>ElR$=KoZi4S z(;K(Z2BOBUO^zTm5JIn>gALdyz!$!z1HXFJF@&6|AN<$t^;$+gUELu+BXi+Teg~_LW)= z&+XI%{9~1TC`ZGCgWLRY&fs?LS0Njtps0e1dbEoIh`y85i03puW2T{r z7d!>@j-q^wfUrf>`KP->!;vlu@9`lYnvosNNLpTJ*Ms%H zDc2q}G_0eVuhyEU%A;K@9>WqZKuAM#fcr=r^!XH`S@3dBAcWLsXVwaSVXG+PN)M~K z9xJB_UYqh=JUl*5&KKbAxF8oPGvli>+toRSs;c=Yx(7T%uUh=be0k~xg>Z-u#=G;! zj*gthH$CAv+48r%D=RB~ecR_7-Bu_cg0e~wLIDL5&mKUHgi#4yTpU}09JX{iE=aDD z@gP-AowzE5Qs9f7k>f6FH9Webb7_njBV1XTRNs58{a**n06ON|l)YO-EuedEfc7&>y zBs<=MAKm4%`+Z{bvnkXcFETl#Bb4hU+Qrd_65D~Ksp)C()>R52s775Rs&7@*wof!j zPxwyt;o2a^KmRZ>E%OpCFG<(bexIYXwRMFPp@oDDW!>GJ@XcpdPe8QfvA^u`xC!U{V)42%lf(1Vp`^|s@!$(@~@#>U3tq9RXTGSoz|NX2t5M-MfA zwVAEDaSMIfqcDyGh?Cu*b-eHn7F^BN8e3gGJ|W%x{by#M$EII$~n6jzW88b4$o>g8e$q6*ZrG=Xedq1bIj^EM2$<=*^Z@9I!#zBNu2oQER2nk7EULMmQDahD@<)I1sF=}$x zxnD>854Es6pyy^5ypTfBBlEwHXAhRDGL;bYwT;cr=H^U^f#Ca2TEWp=<%LSC$=PbV zn3$Lq%Ilfy#=!S`27Ji9{&>{8^No*SqOVz=J^Kmv&v3@`Xa>b77gm(K=%^J)nv~ZK zYh3qDwkC>KD1(doQv_XrmJ$;aOH54s_ZhtJww>||2>AK?$5rg!7Squ9oSwc|VKHhp zn9BC|0dQU85HPTau4*0Dt}YLi-(%2HgGl{U;A88Fc?4GH^52o&a3g8T1iJtPtAc?0 z16;}fi@CP|s(StYg*PG{0@5iRN(#~-4GIbfg7l`PL!?6`Bn6}dq(P8wR2o6LM7l#t zDFq~iyEfN^S zwW5PQczU|Y|9r7M9I~Gi-LLKC6cmWsjur{ojs}K=^oWzAj#aGC0pD^SP}0J}!YFu+ z;@q#fBqSy2g6i=5dt7YPpf_8N5}p`Ueq{98@%|>%>Vi$GuB-&P*sBE|3eHuwdf$^8 z*&y71=V~5;^FsdV)2AOlek>^|*(1zRhos@chsDX+*-Lq9DgT)T;niTeZ<&S&urE~P zWwcTa>pjY0QjkhJIy$-sh|+ku*%uI*OX%nX`1t>M3ZyD^VqzkmTOW>UC44y%ad6tQOk1-cx5yyohOD_iC)_8 z#`bNLh$@qhw?)4NZ5|Ud1fyQ2Q(myJu+YxE6MxgHzf+v(Vxh&@@Qp>?Hnb9X3?UWS z=G%MLZCza61#->9t(nT|>iB;J$`Axfv(~}T%*>2%@Vo@dEEeYHLoG{{f5-C#PTu05 z-6cuy1JLoDpC}qPA(gbIrUqiLwzKsg{+$}9Y3okVhYugBtM?myPhdnt07{0ySiPZ= z>AxdlV#B9`n`RQq*sTtv(@;@$i6>+&08#nj!-ozI4iF@c`FBv74 zXN<5G9*0|W;#8=~hn15Y(*{GyyicD#Sv5*NSyR7#ySd);*D{4hOqF>T$(1XwiVk($ z+}+cq{TokFOTm7lGQWmR=GOfuz`{YUE|CM+y0SJz*Pk<=gqYO~H{lIWQDJD^++ z4hZq_X(1VOqS6W$I6FIgg^uDvwMLL6^SzX3LIA@Cva%Br5?~8Ge|=X2zVg}xO;+^ck;m{zw7i;HUvF;&`=_~FI5b;Z4mLLLGosrL*#QcL>_hmpE2^kcPS4DI z9Ug}EI$T`TG&FN?1Aro@mm%YauLhyIDJ-n>;K9jL&Q(6t5hkO7ug1o~>3W!zpKonx z`E;p+z!J~%U3RuB`~*wo_{AM2pvuFD8i>ZT>!G#UWLpKvp%bkGFC!u#8Tw6nx^d0D z6p}v;s7>TbfXI$q&BA^!xnJxc6tG9O+5N?*1q2gTOCqi=ka07#C@af!d$!>X_bCe< z-P;sl$AhgIeb|f2=wE4}DA@@iN#rw^vXoI!$OW}>a`4lvM*SL8uRQGGZNe@kBO~*y za9nE(&<;m>^8clm1C#I>%$^EN%uVd`;@GG;9UX1( zs5m%SkKK!E=?q3!Cvb{!C5`=i`ZI`-w)q?jt1BFjkHUj_?iy}j3JvOb5TRzf_Zo7) zI+b(YQA{=ATrK+g)o__!G0H!LkZjsnJ9@iOWP7|^eJg$S{+aKYfl0^TB{0Y89`G8|LX|B`1RzKV$&Bvb559CXwk z5vzO-e0K3WET*j&kIR$rgBy?fgeF~VI&qBr|Hjv_6bH79Q?B&1b>Te7o8RC3aaY<` z_JfO)Gg`oB`QEh1*#?blOnIijpp}BNd6Nh_JbnG~iIx#ka`GgJ4>qW;xz)?4w3tV>johtc~D!dF30PVCi|DQZg&GPEz-!WNR>w6or0t z1H)3vr_;XhaGT$jT%M`-&FgbI7MMU*{exJeZw=mNe~(isBI!<8`PCn^#j+yi7oRiA z?P ztdylEB%_@nz9z;Ctkp6Oe&VeyzFWc1 zWC-tG*9u4=pKWN%kRb|nng?2iZx}K2H4=E|3|2r^xkd zzu$Vw!E~vm#p75oM4{A}S(NHq;p%AgGc?P-<{|p=*nY@80=RJZDWT$E9Z6x zKUzu*%gSY(_!78ZyrGJs>(PC(g1a@{()(uc>q$9wPu!8qHXRL3PL7YzMpqY(S%=98 z2R=Q%ySsX6vkfBdgzj7OE$9(6A)F5eYfC@J+l~dN zGfYpr=(0S)Fu(Vtzc!34vzS2pTg1rLkBNwT^qFJ%hT;gAdR`ut@DfTnWSzzsw(>bM zSq>MdUzipsF+3O$q|jlD40!FN zG-L;x>Vkgtbp#3WN#FhcWu9d&9mN7o#5IzZE*xg1ML03uY|ej^g8xgNj~H(_DGZ-z*VQC3HEpD(XX_Ojs4upkeQcfKW?T1hv5!Wp zIN5i-C-Kh?S|rW6?%URzlJB*iDPLAotFj4d@b(wsL+wm=OhhnCUvf~PPSfe%Lk2`| z(z8Mx*0d*nr!O-yQYj>TUY;B_ap4eR28g*K%<->(^cG&V*Vc-Ti<9%zB2AN0=t=Mt z!y!*5$`en#EF&|K{$_p&T|r@_klp$9>cCB}*NsB{Qlw-z9ZHh>s%@#-QUkN2mtu}b zXFjPv=&3u355ka;{8dGUI_S%tlu_5dedt0%s1y9GdF?boZp8Q+uiI;H1|W^TAl`1tf^YF z>AF%e&kC<&QX$(cwUpGv$w||br*(BE{i&szm;C+h=F)ZpFwoJhtxd*w>b-)59p|@r z_;206Pb(6j@>(EI7cMM+f7SFf{vX_InCVh~=VwV!I*Gi(k5G<2R6buDFF06q?%q3J z?~(jxJccIrk--ED^9XVARPn^!-^L0GaJ^6q4OVgDr2--LNR$d!g9-bD{i%?tUhH;ukXobM^G$`T8+K^z(-oA2& z8J%L%n;$>s>ybTtXgyA}zk&Rrd&4Moika|Y zdx42uQcRVKtgz51q4oNG`}XZ;zJw^TXN~5N?C~rCBc@u;oaIj9*@RbQZo%@r= zJQNi8RL}3Hv&`NRniG2}FCX|>=1$O;g|@YsPY9A%=vaeI{{6Kc-Qn)r>bbat42lD3 z(q-|LHzg$ImtGGj;$?O!uWhvULE0A%YME8kmAxr(AM(>*ILz`a+M4xuhco2>7h+?h z6ej%kUd4c$pE#Ui>B!Z?Lz$QS7+Glv5|fl<-m8(oat)DkYvIUs`UA7`oeQIf%y9S) zrBQw7X0nZqZoTdu4McM+tH*C_PR{WP3}&S}<(Z~!>5n3U884_iY|4%Auy~!kWFx z*E#fKB1uwzRt9Aa6O*a2niBqFMy*`=zhzkccl(Uf!k3u@t+`kqs@&WcWKAt~e5TDx z-Zs$5q~VhZ1$|4QWVak_HY0FbwL>B1v(%^*Y6$Z+l$gK(Ah= z>u|?D=Rg@NcW!8S@ryBvxqeq&?SC8om7dHDK#0+=UtOJ@W8ANyRv?b1|Nq1?{O=R^ zKQb`?|Nb?cUE%@hB$Sl8;KbPlF~xBFs*aD32M1C00wf0TG3@CTNCfznEciIK2N&?u zim(fCDxZ>T5(hnCp)F*!qRMuxqyje(wIT4=dpL4DoH9Qrs|&RY**>YG{`O_qTc&ir zwY9Y;>%$5voG0e^tgN}nw%{&Sl)=lCWt!c|j*s~G^d|sE$;Es^Lh)DsaRK}^`Bf+d z1qCa}p8gouKsk8}|Hv>yJ0o&%XlVUgVdC>vos;209bV7&3yIgVBMk{eyA7Z~yQy;x ziD>(UEvn^1njQ$v)SH=(J=$IZ-q8I|Df^V=VgCt?fd@4at8G%dNIv|WsQ#D!$O1?sHfEdO$-ZiG(h{OaM# zz#M~y#>D6zCT&2cU2Z&AOYe_DX9MBY9yIRCgp$DTd;w8K1dCDpAsyp zYTG(5GGRw1DSCv>SnvnY-ae#d@SXUK0vv)!UjG!~QYtAQQy&R|^ToxTJ=W8|3rpzI z(w8wW-44Qaa=P7<;N%ogP+)La#!_BY1+eB(je&|v*U-@J=g_MHK*EAgV>TZfzT@nC zGnzqTM2aw4hLNP8O<*tq68S7y$iHSU@baJI#aT%YFSyMlJY!=ds_etlBsJcT420Y# zDSlLGdHj35b^5m*Y+{SI{)G~f4dQ@Q#LtbsA}*JWjMj!R=)_z52_HTjwijeXJKSEH zm86xzzjEct59rc5n2mKOF~Q*z>+2j16>4hgmX;RUSEv=x!X^s~T=?;*$SCr=K2vQ} z_{74>ZnpU_DaCAB1Uw3zQl~?nj;Q$iHHhYM%DV~I_=rkMh{BP!cuGnJADGfj!U`Q7 zy{?moA5G(4U2Ti}Wz$HE*Wr)k53uvC#K{FP}ppDZ>4>LPB1irH_w9Kp5HeMk`E3 z)Ly*oTEuNip;1)>Q?tadttohqSBKI!_ z%pp9iMydIL-wboI_RDe!w*$iRfXEZpzF1Wy z_|LKATFwz+S^ug37`Xkb94SclUQ8Md9xL?~GgwdwY3k zu&qAGY3Svrif?L0OuYNehK()XkcNO7m@MNHGWz-n-WS+~u`(-Cw+j2$cjWPt8;>+k ztuX!n1T^|CtcN~$XX41aZN0pD=4&^FDJ*;$(=u)(AMN^e01))5O-#Y$v3~6PXl`l> z0I+v@)i!TfH8gg1TPm$Q4)D*;b`H)2?tbpY|2btg&Zn!}Y3+`ZG-dT>963GeVEmS| zzi{{rKa!fX2kCiy?buA zvanF+%D%L{p^dN}1bAU>?24F}wEhK(x1u1!hQGV(^dv9@t3Q=9bZnA>w}Zjoq>H3C znRQ#;e6bx5ghA3fIyE9$h~JJf7++?hV%Ua9gmS6|dGho!hKwOyTZq~56G}?uLQ#~0 zCLz^r-C01Ntddd2o}Rh8RTp!9)|Ri~^t8kOoNTpR*T_h>20!EqdUF68r6@^Xa#x9) zu&~{1gxMDac+NMivMkjms!>rTtzlBt)}1p+YT@7t0Ujgy1etZ#q4lp$F0$|Lfhf#o zD$7`Nu-)%C`^#ZS&8%oL%%b(4C0*I5q4mACle3H|lVW7+a4u~QN-$W+FrxY76|*#$ zbMh8PL2!((X^`1(ZCMXpLdP^Ybnpz(bQ&HmbD;a)2l>t&5iMB?7<>u~tG=f%scx3J zvUeu{rHN94k}2k&SS**3-$9H0?k@OOu(&Qk6SA0CSg8`B4c`2o74>qQ*J>TJ@6o9y z2Yz$|^LMm_n-t)fy_Iq^EP9FC-`-OSTqdAd+%@z3f*PDw{i#-KbaGe}-R`MqiIGA) z7Y3Q8^ZarkEZm&G*^!H9$e!zG)86EuJhhm(z^m-@TTu`%rTZ!jnZYlx#v5q1XEXhL zi;TFpApr^Q?wHdnDkR}Ug%tg1?U6LrgBd2-Tx@dr1zgXVU(u46n!3uoRzp?6!B|ysK&k}SgR<4F1#mv!H#4_tBFDTNc}s#3^bWy2RrQiXXi& z_HVCxEb;`qyL&I{`@6^1^NR|My?!m}4UQD3?0U~V!rY8V`bb#kg-h&fS78Ky%NUel%z&I#RXd;5 zXvuNa$jX}7gctDioog%!^3-0uy1#IY?>MJe_|WOe=Y=)}Grgx$EG!wGuJwKd_wQ>Jb8TB+ zh*VTP8$J@5Z$y`^WVUkWafnuGZf;BKC5vv-ETiOklaUevTU&#AY*toJC1y1w67)=1 zSe<(*$IK!se5%~Im&^yt(UZ7)?8ZpvZyY%N5rnzil^v5H)}7zwT)*YSx=2SbC#8`~G=XxK}M8P={bijxE6jF=@JMuy5&5plEcsx0h1;-n@W_<#P) zj}(x^a3lEmTP7;&+zLJJeM9*svB47ie81OX`2VEfeg>%UA;I}sI5G1wRR>Csw0`fS z6xB>acFu;8B?(`VlFHYRo6WqCSg*PJ;JF2a;zThhjg<(5=9`1Af((IuTwLi>mWb!X zRG}N+hCaHf_~bv*qb@4WEQvZfT%p57*(aHYf4_e7{AFSn!DDDnq>3Y8OFmoQGTY#S zXR@Ki!3kRP1I9I_urMd*XCcr~>9sotN(ltbx_hBsS-=TZm(I@w@CtI)<`}Nf&rjSG zo$mVGPX~5RgZEHrdQl3GL`P&>%L^FHz6ClRCMyw1&CWDx4MIls;O&t~A3PkRArp>1d3&mZEI?k<5G67!R%phyPQ zweqiJlee2{l}6Rr@6P!P*px@x4qs*>c9@NGr@fe^wu3L0_hFWsttFl&P;|7k;e7ni zo36dGJ&Ww8ov7CBEY=UvuXS3FtzWwEE{j+W_yBdoCByAO4-GGG^d%;`mThH_^A5lm z0_vTu+s5l(vDVi`o+!^cUu@kS5^u_bAJ5)%c^xKL@UadxKik=B4X3EE-s~i<_X(Y- zv>19wEMQN4FZH%=(!Vc>V`0OGQCZ#3FX$3Tip}@+j=LUW_{n@|G?>78mX@5n!nzQ< zyj&)pD-{EEHN{n?UJZVU?cvqhrP9$7Cm$BJHco6caHk>j6$7Y7O#EmsIRyw~j+e>& z-)%x{a`Tm!;;=}e9=YCIq;;N~w*J#R9QD!{4X*a>NErL3uxasgr zitteVhX`iQ4>lvW+r#Rd<-*hggIn9s(XsB7hh{m<-%b_7DA)aWsRTq>@XO10_MSI1 z{CL6nL9Z%$tFmo+CNn3HkS=FjUSlj1orGlmC-=YK|Av+%c&augE$U&hIfH7~w|dd7 zX&IaMRU7W^N^QaFnz0Z6wg)^z>rVXgb80W@B6bwr#fSvN!=gvl*!Z9&&cJPxi4$ia zO{LJ!-CfX@t++cpl1(i~>NUz0&F@70@)3!UO{Y{WN#8gn!l)tk&Pryc$3X<-Buq?{ zhoC}89G_qqI;8^r#wwQGaWd<}0J2C(W@bRj$cddjF=eNc6iQ+G2I0sdkv3dl5FWW| zl@fAu7ibx!3ff^~U>JO2`*%jK;j6|<-|%(HL{fu2iyQg^Yp9G-o{6)6r(f93Eh*)$ zGus6+q!A|x&QG9juPD9?9PC8iT+LsC@AMr(NMf=!(VZxkie~96yC;?m2#8ja z`F%<$A4Ic0OP5R#?w>yzTtXD@0~uc{vutC#WEJ*^UIfaFyVCeRx1-H_xQh-BcmLVJ zGq{NGz?ZU-$Y^NH$}G$4B>odMH0}~b4vxfW{8CW?uR_r7WuBTwCELOF0LS_NOK2tY zVR6K_(V}M`M7tCEJZR_%#l=TTUnOuOZNenIfx8LV08wa}GjokYP%$(wphNtx(G|2( zV=FDFI+HB;tU+>PBxe0>kf2aZlI{-kNxLZvo2>q)fWp4G=m4slCdg+Uk>FQ~x$XR3 zcPG4xu&@BT$L3o`UhqdZaH10x_9S`!iR@LN<}eto@vA3cp#>eTfE3|!3%uvg5w~vy zXEQs@w6=xR`O+lvWNNbJyh9JDylH3OWGCg!q*rsgcZ(5CQPJZMkxog5w>v&v*qU`5 zPvrh48W?XmIVR(LIXSm)D1M2;#(DM%%7tE|Uuhu$3kY{{i;n(UmEgPb%i&6-;)M%j{PpjZLk0J1i z5JwCYgeq{KFm)?(PMy?=aHc^Qu_0z_8_S4$<+UL}7zl$8&&A!#Y$<;K_A9Cpu;8=0 z*I|o_mCUc)9Vt%HSX>U|H$%*KU9J1nHboOcDD#L4fk`0SqdP1`WX zC%ez<);5>~we;dql24`}`kSxQPJ$#rG1BQL^~a|{DKwuKBQ&nRqDT9)LE=XL*xEdY z77-VRxW>&L^6duq!U_h0jg5*G6BDs>DjE?940G@}yW>61&k8jJhuM67U`Rzmq$OJK zN?TtFVz6#EG4?w^76)eWYnyGKFTY|yk3M%E`ljGgQN{N9LJ2EOcvHr{&dg)`^!=Tx zj1AXl>6fn8`5NfnhkzL{G9uLTdasO;lRTF`|J;)}I0AR`hhi!^0z|fw)ek!t0ephY zd~7u}z%Cwt<#T1T>}&0e(XA!K6_qH;n#XBtMsd6JGG#gG9%P;@u40uH6flZXaa{hQ zu=e?7SnT6TlryN6WN{ymD044QZ4Kf|OXPP|gL+8P8LZ_sYU-!_N<#e1Wd-)?mc4p4 z*S*w$Fx-2oopon#nHR!Nwq}fZEUn%@z@p%}CP)f+@0>re~Kzu|IeUkbRNF7WZv9`}tK*|y4CUliWc3BcRs6*LaQtCEui)F^oj5C{y6D%+5W zN@B{FpFc;qe);6?IxDZhx->18lFtFr$pr4qti8WagV+KeS3hd-MECbc432#{-X9zr z@VJiNT43{RZt&q9=C_ykH_@Aqeu>_QeUj9;es%my(bK{rwVKUcq7CrK5=;gIz@_zj{|w;4sC>Nkh65tq<~d}_e8^XI&)5FZ|#Z2*{o zmUey{z(|Z3cux23wu~8^s=f{LK6F|N{@30RT%K-=Rx&NGXO3gX)9bSw3w-G!)$O(IKVfC52C{u2XDyZTcH6{4+u_h)`K(Hw{3``BD=Dw8S z_x0m-?pFXixX6Xcl4T1C%ruhBf4?{>G=cTN#qblRw>RhL_B_JTk%bLq`0(7o-=1~8 zgjxcPryzcJ?pTlFkCpaFCB<&;S*#NVKc1mB5|^`S+wJtGL|Ed7mr2I}tJxUq%F0R{rM!`Sta6 zQJ$BfGDmykRX8uOdPH(!bKM=~6&6}CsD!K@hl;t0oE%+OOKu*F_T@$CJJHwUZ5hP` z?Cd`#n0rj_q}GW50O$ADd~EUbNW>}NExs`X_oiw^FK~EeCP)rLfCXOb0G|CyPtQL- zLB1RsL$YB=EI6&B8$3H5Rpu5^*hC!tr_%vDu8+f4#^ zd7mFOP#b>ietkzw zdYP4VB2bIz&!6UD688JxJ4m6O9?(GI#pW1f|KCUKu|C=UG54ww5~wdc$%TWzP{V{s zQPIh{B}1kHr$Bz~E0&bh&h9g1mUM)v8PfC;M1DmW(-F%YjA(#WYGSx=b&8Xs1WT5; zR%PJ1&9}i)20(re1@O8Hn{PvA|3u|V0ykoCxD|kWUuxUYPHs%kH~i>;uKt$?g)z5{ zL;&!bEVQuL8d66VG27t0O>fi`+WF;|(ZS zHBQG?PO4*2xK^Ln|XrX45UN10x3295ZTYzHs{pk5A! zYnw}3hSoOS;;7X7EGDK>Gt(ze%DjZmpqC82X5_mh{ff((nd7KTP4momSUNf$o|!tl zFWdK`6m7DyAzrPe<+M&xiGg~WA$p7+Qzo1o8}CJ}FezCsg=YEwv5=}NK%7+^w))F6McaV3iHV{rdnUCj;k$03~KIduEjmwf1ti5#I+A@7-wEv*Ch- zp<=xxL6zCe3)L-rNH`dvud?I(7bMAGpowJxZ6xg2Iz~ap zrOcdT3N`jbM3ILM_>|YS)f-ov&iQOe!M59)Q8=5J4GmqJrAy>l*g-y7XWF&~X?+=% zcglj#&!4Xk&(C)d97(HW?BL-(55j=_T?5y*uFgLwj}H{W$wizmZEWmCu{C974Sp2z z@pay!{qgfZOAwzPh!#-FyYa0pj#IZHr&*5cp{6FJ6liMB{{#XDKwVZkrCF|4lvE4L zDq5vH+tz;5ui0VX8X9#zRH&5dU>Hh1M~@X2I4_XERAC|FQ)jZo>2SQ?R;VN5`;o%} z5207p`~pi^H87s+-rHDrLFu)obBJBg(Rq0C>!a?F<3~pf8iLkDmc7lqP4nIHmm=Qj zf}n#~_EJ&O^oYkXSXpB7^1^=d81i`vXg*k>Q>O>$%ilkOHZ)TXVj}M!#=BqO{R+lX zAuJZulGLBvAtm6H4MHdi-?$C{bGmme)F&WwQIdbf;{&FgiQOo&mZ+`jAWkR6uCPV8HOAC z0Vaz90fbsd2#?W9rwx7Eas8`_!NyGhnS~D?fXgKp;&deNUJJNJD6QRow`H1ae}t2g zDpTd^WlVBqUxo4Yuh!piuj`op&;?XHL!JqoEvtd~wU?Q4fIdFmlD`u+y{Mwx+;k2| zJ=ireFF;ioH7t|kqvW6ep~))^5m!vNjpkAE`)Q$jY{$mh%t+=pD#R5>$ODOA|MXjIVHWzv)0Nzn07CkDS$_;jjEI zEQGWHh1Zh>IrH_xBXaNZ`}^T|rjp5a;js zh$m|SHTD4VJ)05QT#y3~503)YZsZ#z&MR7G6+GhGzqnbpaHWulojHHXo0t$0>Zg4v zk74Ay0RF|<--QJ6h*KYhMz-%RL6)9Q2^dFfo7rx*pv*K`;X=6+Z(@RQ+n=_1 z8Q%8E9hw|!Y3({-Du}?~tosZ=e=_?rSXUlL9rF(I2~V~u zrRjSTWFRa$L~7dDh>d;wHUbhp_0{xg^78nE6#9}Y!Y)`zjY?rS=!kMkJPOlw%^L;~ zMhP3($;s!yr+rBy?6h-jvL@bL5G8{0n#?nSGN1@y>-IMw)6;E3`IAt=RPB`q@3L&P zX{Gy-7m-e_Ub1mlN*AQ#6@UMc(knDHaCI^8o6e)8d^11`9r){-vy7IsPgcWbOx0%l`Fdhq44g%-qVXON33A5-a79Kr zK3_3aRlrk7u3FG4qy+|=bdv&r;JRAocD)|*%29r?Mkq4`jf*rj0Vzn}Rq_ylh;4s= zS$T}k4Pynj@Ayx*K7VipS7*BkZp?pzL-H)_d$ggYMG?q0aFWzgV2ad&0YMlKi-6~} z7rMG22MQ#NY^ahwF3xG2etJ6T-#4)h&E9)^iQ7-zg;Gx z2)Ua$zY&HwdXM;J)JvH}fw@pyR>nb!(-uK>6Ze~_7cv}`9k8Yk zF>}bnICK9pEb6>&EG^z`SCA3vZJwkU*Bpn(l?2)JGLc6M7c zpV&|4fhM9^_>db233e+$#;GVT*Ld)tP^)0L_sz`*4<1-BObv*8>|_)d7uR3pVlj0` zUP1ti1(-bvT-r830)nOrsp9U?D)G!xL|i0|=}sBk+wJ4*-*FL_+7mfuT~fVPk;I;EiDT7zUYfP6(Q zXw7HdNyN)rN)5>E^$MWZ04M3sP_~l)*^$-gUi@_(JzZV9i3$Z4l4_vOaB@Cg{2c0k zy8ES;jyUKU5H+CgnyZ0UwjzP3U;V|7duAf8v0i*^3-a>50~^Te;K!RsvC+|dR{g1# zeaRD3Q>A*3e&9Ty; zTBRu3^4i+-Es-o(V)wPdg{i%}i9FCOAyl4e!5*^FUcY`_YTT6eg)U9n-_P9Kd^8rC zG+brB=i}ZsGBN_x#zR;`;E@66PuyksrTQ80UZG1StT*R_yegnc0k!M;!y+On1{#`_ z!|gd<-LmGE7PMxN3|PVskH6*tscaVdE^bD%sKlf1?CnuY`P995Ay;YH7fe9&eSQ7S z7fCpQS=(ZsdpVHK1e*cUA@*dg43-Z_jolfBqJ&3m#~QeZWFcE}N=nogfzXuerY?Eu z*VsWi&~%Oc6wq4^w&(nh_AC%U?Q1$awmsTgfky9;%#)d!$q!6X5I*3>Lif;?E&_>J5Vk&y(z!|_7SHaFka*4~HB^_-Z3f5Y4| zzz%#PDhCS{PRIeLW6=?vhrx)W6nTB z827_1Lo2IQa&A4^CR*~+|)D+EaI8Pu%sKX@$AV+17ab4>Nmjcb)3UZsrk5_hSy+Fi) znl?K-E8Bm~91s2rf0AVnU@0#z2bw0mh?5C4u7jq1!gr;H4;7`QyaB_&y&fDKBr-w(i*4pP2Hpnb&$%JAdE!HYgFD*W-F}RhL;pHOMJOC| z@9%25zm}HPUPUFGqPR~eg6)q9*FQT8K*9wjwFU`(d_-c`uHBx^Bqn2`a#wKy?xU-# zezlD@4CeWZ7ocJ$VfTveqZ9WBV_0;@g}!=)HNUzFy*7$_8!tg-ot&K9!dy~u@y~3f zR|pR;FQ5f|?eD**ObJBYL!jgNu;WXsC@FoJnUTC{)!9o9yf4@EOT4ADp_e60(H%g@sl#HzXzLZbSPs5Wva=TN@kL4K&y}kVMu1*y7Pb zoq_3Ti5oX=0GAz(ma*~N;2@v>0^5x5Ej543hDzE z8vbBdVXk6eFbx6-Dyqi++z$b1YOwlnNMOc70s?o`)aa!B>SJS}@YnVDxHwd5BeFHn z_v8Ra4U`kGe|cVD5`6SNEa5Z#0Ou064UQ#T5ATbl37K%>5P?BKkF8N3^MuLqaVTQ6 z2fdy@I2PsRl7Y6YX1R6ip)w=fB}BlgCMEq1zX!VEn>U``-rinbXR3$K^YZeZ zJ(C%w!UH`GU7u^p$~wJS--x0I)SKh|IS)Mp#33>=vNAy@U!8H&?fGdJ^r#1NFK7|C zYZnW3N`Z=d({WaUnHj~NO}C4KOAW4*T;OJ_w%^bxNsWwzQQN~RvKh3Zbi6W)UWIvR zP0g(UPDvK}6FK%_(! zzfQ<|_YMmK1CY3t#&Y1t!cA#uY@`Zx2~0Aawv6&rseqH}?tTcG+1Az;0Zd9@Bun3X zCNl_cf)QC7wMsiL1|}pt1cCR zaHOlN3*=LTukT+F4QcNK`~Y~RlhaNh&ehb^)X5rqaai$F|2)=4`e#2@SKno2#d@lh z!LkF_o1z3qwl&%@t5s3u{{2#5lhQY2W1a9Z%^ zzZL2L`LSEX)P;V%f;(=u=?ezPaLBR#AaVK!8g3HR>PP zo@+|qXT;~$t3vPrUzLI)5sm>6k{vci3ZN_muTIJINWp_uU_F-lp7Vpf1*+TV(^Gq0 z-K4ZMBPiAJC5rB3ra1NOTlcMLPq4nG9jCYo_CVhc0Gk%d9l*iT)Y5|M{CKmpX-^Uz z74sL!1sTV}vU>Y51G2n<-HlQx?oLC%6|SjClk|Q%S{%qY4Fw%)A+B^22s=qXXJa3& zZ)||Jg-5OWwOF`Z1lT0qp)^FL)wL8Z?f0Oo&AEYseg8Y0icL6+D|JY)Fe@GMv$J0p zrx{c^&PgS*OtrKyZflc~!r22$^%cFOn7-#$`Tzt76(cAa>%kIs9{mPM(9HPyy`V%u z22xi5G0~bYG58<=IyaI=_?GqHwMsrVHpMy*tc<5WCtLd#Z#rUw@?z=z27Re`bV>}q zKZFI39+ZY!vwd)zK$|M;>{n3jLI{=!1CM-%ZCxvnqf;Dl;YyT25`RSnSh17-=YLV= zilgH;XfAvPDM;@w0FxU5)-5pugS7wIDM(F&=PygB;{f-rAAoZKPzqqK!F%k>7j_mF zbGVaa$rm;^PwaJGfOAk@zMG{G9tbIbk77a0PE4M`V8CWYarJFJ-TQ6U_^ZwW#*k&I zqN4H`%v&f%&;*(PaH(Gf0D82kH3jz2gw;9)=kgLxJA~nXA z3rq|U9S#!A9on}j7bRbPF~eK*`EM8TtN-uF%>UQ8CS5#8Wfc|aP3p9S1fEKA70eQYhVSFK-rRWCr#ku) zi~n2~64D+{hN}h4q4o9COBX?V){487TT(AS9$q_sW);z^$MV?w_n)qO<1a@?M~T;U zc(}MWnu@1%POJ5XyTOJTy7*K?M5ETvFBZs&uhL_;r^D}EeK*%gL$)iba4Ryl%6?`m z7TPqL?_|(yB}OrbB2328e4U+l=StqBrqYs;#iymw!@px&c@Zf5AciqKkddy=lbwr8 zjO@0DeD_c9Aq;UYRz^l-yK$j@m4l<@qgw4L!gKp6#t85AiFfHS1xL`GU0^KrmTLn1 z{72;h!sm?k=;Vj=5B=(5?WxJ~yN88aV?l0dO&&;d8>4p)=&wT3`e^m8(D92UvLP-@ z_z#^VqE+80(yMatKkoft?4M#{ZnITkTU$0VcRsPBVP;1IPX2pokF}R$rRcvZl&t#G zwnr;pX-iItxzYUoDZTc+So}>mbh%|2x#e)ma%ejisoGa;PmLQMq1p#O`u;r)tnJY$ z5fBhMw_hY(uy@)y34Q%e$mYBH)MTFifYjEQ%hVhwdxDgjD)%P>l|@s%nfnRFQ!Wv| zW*bfrleMo|}^+ub{B9B8LLJvs!PH#a>Q;&+^7AvgK!`ki~c%AznhFZfpm8ghZSC!tirX@?K-CF&=dfm;(>5gU2<%uFfz z+K+L5Zf^UX!3C@<4BW;g(=U;7vT|}tQB<2Nf3X9?!f*rr1O`ff=gwEp^dZRJ_#haQ zoGfOyYz?ak9_0Rp9e3e}$SjFNT*hkJ{^~dnt+J_wt2V!7PZdwO9}~Dlg&y^1 z6s6Q|MWhR|iu~~%<9RFYwz2kW#C`3hhM(UV0-AG|d-YA0m6df6{FG8s>`%UVC7>d9 z*j+j+CnrZnF8HmOYfVj%0>!&~ujr+zL@f=Pe^JC9xUZ*QKi6wuyLHRqa~b30q#pFx z+n*c#IkBOttc;C!1xvlaR3S4bt5GUo?3v752^GcR3LeqbtFM!Eqm3`rZ*1Cjy6ss` zj`{94+ZdeC#nL+^{`5QHjj>tzh>iPq_Wa3BOzrA@))yzqzuVZ@oEM#pefzn{>+9`q zH|IALD~+Dkgzb0U=#qP9_7#1IlDxI8?bMWTlh4Zgfd|G=!^cp(2Hb{}l$6N7_KU%4 z`e#$WOS7IcAE)4t|I~SspKAeBnBgol@(7zIJ#wJ#^^{n)O&-HM|fx6!rEK+V9ON ztKC*lL`MV-!^VM(^>z0~zq31cy2-y=(u|#F=j1>oq)qQb27^YIZzG@oO#LNz^>V!0 ziZ{Uj?5xnQ;UoIVp{?nhOWXJJU_z>*{QRUxR+fYr@AJ6o)gL@SCqminkIQwz+OYxz zqFCP8SeiEm1&PP_s7NArb|dk;Qu1?te=ArhY?cS16sxZGYxL9B{#MEU3XLed9WEO% z(=fo)$3l>C#nl!V4=iQAJE`9M>%Z>3zDI`mIa&2kU2L1UeQV+T1|s0YqvGy(F0_1mq)~S85HxQMB#?w0FxW5>5^CJ$kr8=o*2BO| z#mEiBEr+x1N4Iaw=}!SBFe&WFR&{pc8sc_g0(YR0xcJlh`KO$`>6(VNg?Ov?(3I8G z)b2p1r$(ReJU*YAU}N9q=7!2=K?7yH7#;rnQ!&1^_8uJ}HMK^&o%650L&CfjVd3F- zRaJXyl-FDbhq(eSv>LHW{K(bd+H;5US964UFlBv>u@!lnHepP$56~w z&5-VG@0jd$@uypp<*quGNb9vP)9&w-YD)&vn_wsBFlz70N(~P7KV3TZN95$@qH!30 zj4QJoed?N+8e8Gh{vMiN2L%OLkXwFgd;I+jS6BdAUOUcKsEv{Vh8Hb_96UQ;_cN94 z_>=YLj|U?6j;~uC)V8Uv&OO0E=^Uo@7_ZiA_Bu1A18L&M3XeRyT8}z9gh~wD=IV}` zdUNR}YxT=K){Dj6jq81mPTTPs1T3h9s@l#6KJ@kVvZdRv`h`mE$8G|7hK!fR@&Y&W=dO6<^6j=@2^n^24xESD2U zlKL5fm&PFN3q9Q9ygbE* zkE4%lzmAM(>FC@)M^Qn6WVV%#%CaFQ!&GPtkr4^;JGf!eRk5-?7lM8G6J-9^*oM#p z_WPIT$IILPV$qM4%pnH5%CpO~3lj(}Evup4)TjLo{U(PQTVV&N*5&ZKb3u(-_&AfT z9AF`>YU;l;7s} zNcc{d3FdPvG&mz4{eD^YkP~utaXL`&qOA>KufBzO>=I6iFc;3A9d%`t2nHnVMeSYcr!Ou@%T13%FKb~r4 z(??kB|32RT+2FnHx4OVQRpan;b$fe##jv!*4G-%iI(qP%?dYYouZ8cxn=Z@jhjySY z6V)e+D?PIM*^2l&+S+}3DatIv-@XCR*-P7(n~{-GSXh{$zWiDnb-l-Cbf#Ci{TTFv z$pS7&C-OvB<Q@E>nWc&N}8)H{-ff?-nb9^$}d+bUk z!vTXtTw2=M3FK7fsU8opX~o6YI?S3lSd!Bd?|%Mbb;eN1^{@yVD34UPzj0|3{+OAm zQdv5=A_3B-U7Wn&pK`0p$vW+XUM*eV@!}un0?q zeja)9y${e_t+C0C2fE{+FkB*#Td9w(`q30ZwZr+rg|TPP;9mG--_zlEe>c*0Wc*-` z&Glv_c-`yqoHsoe|3FmMS3e~=S#NYmNj6<>ivg-Bf*g`$+1)f=4*@{<)& zP1|Wh{7~Ph!Kk$y!qS!aKX`i+XsZ9WZB(gLDox^7p+T9G6qR9DDpC%aa#Y47jw8SeYK zulu_1&na0^1&r-*CEtQX=4=#(`t_UF6O)r>85e~~!^R>Jxve7; zcB@~BF5m%FZ*Ol0&hoZgM?Axd!vE~2Bt7=zZ-c7aTAu$ajgZ*Xubz2$1Cg$&0uBK@uDE{z6737brTzfXLqoUzJ(vu0qjWtwwy@*7#l z25fcf>fHIJrq7p6RoN^zR|4R3FRgCDdNeCd%_!BVF{GgX!OEo~JA?~@NX9_Y`RCfb z-}Kjsl19VS+Z zvaha|{B0l459pSJtzXt^$}b7=QK?= zttg*t5EYZJkdmUmOjZRj(>j0=D6Ax&_V(-{_WR!RJm@BX`gW6h+7*+s@Cx=SnyQwXq>PxkH>g-;>j<}kQ+A+?9%vxWzFw`yrc^) z%Pcu*7tdp@pE-7cya5kYDWfZ|)fmwDol(LyJn6J0<9306_TBra8jJlf<2dNiDsw}8kc*4O=RZf=dk zbN-IOcZ|V)v!mdXC$7v$5I?mn?}hioIONHnqJoo9+mcRc5oK25~4b z%qZ>M(vWny)DRqCzM>%B_H!zfbFVq3*RUt*Byl#We?i{~!0C z6PJCzaF(;)f5jL7N~@1=(N@z<){*v!JWx{AWWqP~^%yY6^YnpmWyh7U@N~}2%CT&> z#Q1-VoPNI@?ce2yBww7d@{)xf`CU{eix+zrxu*A2S|Ge&mL`N1-i+#zH(xIzV(n}+ zP!*m@;ie@$v>AGLd~JpA-=gT-cjtlD{ONAI7pd(eH813GPNi!BbI;8~k zK^+W7OUqnPdGb`&rrCGVXh8jU-cRBCJ9h3o7RrZ&VAjM1Ex~{9t#|3m{+;vxqvxNX z@-gAX%m3@4{$9(Xh!6Beu0owQOOCBGt&v%QCwQ8d1_*6$P6JWo?g(S35@^m zY&XX_M#6%EyDt4Xh<^ZwDtJ2m20^>5Qvwq|2$Gu;Uh`s48UPZ~sd6Lb(YlHmdo9jlK#oM_yIDAXX)Fer5*6O zAD^~H3Tg8o&0vnKi66hvVpKBJIedHh8L9c)++2KSV_wGir9YAP!0m&zZ7C_S?fv#D zE1|omCp56UCW@3CAD@FHjZva$mwYY~WIPzcAK$-Ajg-4Sp5?3=+=vDB`anDm3Pp-* zwWa2vLl#tDpSn0hWXiF2e%BZ$VW%Nchn>gvHk`jjOkx;bZ5?2}$LMRMh&T?Cl1P@I zcQZPoqA}=v!yddbk&cUr2@P}-Kj-A?e*RQN8&jkZ6psRsp)X5qf{CJ+*DR90 z(b3V1x{zMSB5?Wf_6WuwF*AFj&j95hqE0x9-ZlcM$@G*IGMkw`GKHI=Eim#xVI5~4 zuHS>R(Ht>Z=t|qv)TFX!PlPIYM_M0-)eQj=lpaW(E?%se2%Su}^Hj3LA$yLRrp%-uN$`Nr2+U|3m*S-M{AaV`AvWzGet z4WJ?7LE=fxx%rc~@i3iE!VehZT`sNOxaY==8xPDND~&Bm#$TZVQg*DrfL00U7&~*7 z0s)V`wY4iUzlac*uHCuNO5jM)Nir-nay2lPIA!lA(xymt+0ACa6zR3c#KoD8bJ6Bo z;@Gnj0G~w8O{)tnSuhMl()Q8s#Rt_r7g*+d_o0h9UKxO?GueM`-x;3MHa1TmJ!E1JANaL$q9Yl&u7~DM)caS>;J$b@rv;CaZ)YN27zHjU5+LAjtHb$tM^qD(orbmuQ zDJfk=ZUbT(^rUsaa>X2xACf?Gh=esdUE|r&e--I$y!D!E>(g#57L@jOci|HdZO(L| z;V*>j2lF5Fr^c#_r#ezC2hFCTO2BEjNZ8z&E~O$YlxgMzgLHw_c#`w zwzCVBdkE?Qq8`ln?SrqkS!82;7R;N+vm4#T!gknf)r~aA&ceiFfp%Li8| zw6hS2I6LQ`|NgqIt*w%u7a}F(+Kr5jAyFAXK_1~^h6cnkZVTO$A;W(h*K&YgflXEA-d`&$+v!#vKif7u`H1>kK^yLU-gv zU7OP(vdKr2{l-Qzv=h()Kn}8W*)pS(C)cZb5E;wu=StwokZ;8^L!N>3DbIGVsln!e z;iEy9LdDT97z#-kYAik}_%Q5Vq}3^t{f6D1_*I#=#k*r+1hwev>kAK6J}3iA0eQu0 zoe8A$#ne3K71Y+k$vuZusjy)H-xVYIvXo$*&Ys({jQ4-6nfyT^6-C?ZSbLvJ+{ z@*$5NVXbDe8R9dji6|p&pOHlO#WN%K4t?K0YuB#%DoQ{s*=V*EMidJgDH7O74#b!PM_ZHJv{}x?~A2@6D*j#;83V z+_D$#c7@eTEcNtKkRuj=$nL6iR&p{OPqkp#`cu$9L2m$UoUyJh9pC3c`~9Ym_O38j z)6>(ifa#EfLPCOso`r?TH4u2+eSZXSS_R1ODmd2GR?%w#0i_Tl5XoC8F3!Y$J67Mq zL&UW(DhXZKCg_mH0~dqgT+{}mhkoOb#R~BA^DkWryZ)o0p=R{H0n~#{TVKC^ovQOv z>Q~AoVuG-DV1;pkuq_9YZ#*(*=l%~tM4C7Rpt7>ZJR*i_Z+3?5-E;L)r#=}PgI&9J zVX31pSo6n^J7JOGEJLIWx+UBR-R-$PtPn!+*lmVL5`LXuNO?Zgabb?{ppURP=iuN8 zzYSCYi;EK(JiNSAcJGGSMMtjpQ2sQoh3St&Tc++hOBe`9T3~OwMJ*%txh^;Nf;ykq{XeLox_{+u z00`Hv|Ew33^t?xR+i1^CD4h_8*L!PtM9>wq8qeF?o0ypaHK{nBv3#e^_3k>bHSEzN z;#3vvN0{VZEbeWOt|CIzQo<5N1a+}J9oK}&DXpbOZM^^Dfdg}|;j=c?YpZhao8^1Y zVA!Dg+q-+Ws@Ie(JbROeIKj6xCXK|!#@ehK;3NY0Vjax1=6OL)xH9RiI~oYCb$=mm ziVR^o{0am~`1bKy%E{F0IqoH7AZ04Wp?&PvDw(9O+F zwaug%LgJ{aTZ3q+UB(N`Q;b&1!umT9CsEnFT-BE)%oTe!G49ZiBv&a=raA@H?2L}f zm}M5u35bog#}@rrSBFi5M*S7gYp&6H16MAt2+T@ee^U0`GCX>PQoWY2)oE+%G;ID) zpLXl&>Ow{3qAOHaSLYYJ+2Fu|0}pQ=yww!H5N*_;xCBm)$8$sR61n@*0W-7Nrr~?J zvt^v?H`ba@JOK~e+tzjwB2pk6+ciiW%1BAEM@z*_8cv@0`iv0Ck>Mbr`M^m9t<}{@ z`}}((d;9t(A^Qbx!s!wD1bWpB+-8-pYX1YL9=pIzHY*PU86ql&g6igt8_>8Ycuie^ zf~pd?|NUEZ@0ItMq(bkt(9K|)=JWAU!tZpyhT;$^La28ZFI#ri%`MND?fEeNPVrfR zzkk~6kHK`KT%|DaP}=wU`epq5<6~osAsSZWB(Q+;n6S5uGufsO4tlFvdlDN{4m_Xu_^}>@Y&C<2I5F}plJ0a_ z6~8u=T>4}K4K_Ail@dM(rINLuN&}JAHZ%xDZhADSSZ1|En+K|8 zmviT?Ks1L)?69V$Bi4LaSfUj*AM&#-mka-EVy*5>-E9?z2*K6;e+V-U$y8mk6$R!Q z(3^WT*B8f3)Qbi|OwIePp@BOUqKW4sL4gP)3w0JoIpQeGgbl!3_}~@KdeB73!1BO` zJtcP}R+eK+K{kwYy***<`1k^FYKphJ*!U?jI-mi3)-2HV=wKgJ{|Ir znsp_zQ-21~f)IuRrJu<)0>q*E1QTT;g6lBI5YHkI4o2XI)sJWha=o`$*DrW*s)3oA zSx#0~aZwTMzjsd7&6{5^bGT7gZ!gRQWKViv;Lb#z!>%D}3jiwN$!3{MTT903I|rdw zVvLVF*2O)|&St_k-?(uD2^yB7*Wn{#fdhZoscWN zs$2;jDA4Z(Otx;6I{0;Yt9%?qgA)-%pj;e4eBaTbh#r)nGNESQax?_;GCbtMj>;%i z@}|lsobSR1{a$5m5rQMbhr@VXU~(WKyEM|aKUVz)L6g1T-T2{*2ELcEUp5TK3T1(j zKwNmP`X8J&qXMi1APB#RN3*f9sl$s`9$AL028ThTR8>_e%HPLYmpA`(5eGjN`ZQTV zH8uDv<>ku_;JN`RA(C4hx*w$r2;$kDWvg(I5*#mfu$);29QI)g3pVuA2%ac7a70`C z?#1h4b|sr}U`i|=gOpu|Q%7zMWSD@3=+oT1=j!08dj3X|11uV}w}f4j#i=#0CxD}< z6-nK$B(-&GH`a@~x;i2Y*g7YyJD40uQ4ulFa7V!Kh3iCd78DeZh>!s1PEmdHaFPvF zXq{zGC_CR4Ms8PyuR(AE3+0Tk8Q6c=f1 zJh|NFId~hx%B97=+i2>UGCp6T_68Oo8vmPi4KO>wossC2v}P zo5Y#!$SK_x7%&a-DqNolH-5>&g}+j5?p;g(7X_6dE|&CmA|gsq`Xc@vs%~r#9bH{{ zgLlVZI3VsG|CD%27Bs~g&bT)$0i^N1@9jDJDo(ZUsb@XdW{=@-i(%10z*eO#G>ojri6J;0Bi0<65VFSF6z7wMC ztCufJzyEdj;{msqh9^%#bd7?Vu#k`i=+l%6fj7Q>)k2LFA(2Nk zrl_D`kE97e5O$vCePNWQY%nkIz=DD()j(e#IVND-;IN&cJN*-PXsGasVhYHO+xz`;a%N%X zd4S1bjR6jIJ)_RGBkSdkS3wjB22k)?em>{r{+p_5YLK#^-7X>y#JaL8@1PPZ8w>8; zyO1n`ZN_O&KWb|Ma8MLy0!$A37jWy=JU%|NQ>W0_d$=hh)i)GYxi;12X!}CM`4~_v z2oocf<_o6A#!$H7tgjz*^XARGJT)HdCu~*3v!UQ)K|VqY zpVeR@sS3Q-Ykhu47vg%~hI!C)GR(EM!C!uEqCX=!bQ zws6h}M8G_#quvn;ekdg1J}wK~A$`-=|MhSe<;~Gxv1ea!x&%r`H2C@NC1)6)7#SZQ zXwDi3#zaVG*uGsxQu3gtrocL-!3Haeo2%Tl6 zu!Tj-GLgL?`o|%%588(CXaqgk^q}U4l?jV=0&xd^4#FTWyc#TpF$Tz$jl{uZ@AnQk zZ?WyCuLTC`wvximzB?h(i_hYF#Ic76(}(I4n=)lQ$Ykt72+H8*u&U=mRX6ZEeE>E9 zIbPh<3oI1*^X6^ahOt<|GI7A~fa0CNzKWW8dG~!mK>_%KMU6L5nwO5kAtV+MX6EPT z69q3YY^J8aVKo&SW{^UFy#)A!sdBclal`wvxRT%sc9&k5V?_zwA+Y=!Ax`h%6SszE z2L^fpeh%&j(=yxpee)|;M`o*g?7 z?Kd0(J@^E>bo)nGw$hKn2Pc ztpyYn6=$6@csTox9^a@+Slm6ZxF60IFMh0r9S&(%)EDDZq+W_(42qdsIc~AzH7Wt;Ly1cv}4h$iMiTn-QNgQo+M%5e;7;Nt9Rjb}W zX^+yKwzdx#7;KXZLc1Z91IJAeOpuS5`=4B6?KiS76vK}&Ym7M4sJQCgKgc-CZQ67|(&g8OS6brI0Fl?Pe@3k%$PbjC>8G4Mbmlu;hMyiE4X0;5 zQ?%2;%7y8G^M3qz#Wk#CRJao5LvR(?;3jDXLH_VOG&q02c5o!JNMazE1oORW`SKH( zfLB89h+;szAZ`KD#ON+Kec^(v{cSDO|G+Pz1eOPPjs^$^^uzH%#T6K?ts~#CZ-sTyGMqzXe17_O1AtmzIFeF)=aU>YN=N z0ohIX`1sms`tPv*0m#wnlxqlP2MDKL{?a17-2zt-p20(HTCc*xFSHGSz))ZRImkTZ zKd{m=2^vig3K7^r7C@EIG1NBV%Q)iI1T-H2en>&tC-A!HLceDl z^raBot+sSTC0!Ly+BP7t@=5_k{2djZ%1g84?~CYNDFy&MFC1ZzD+DjYT2s zuxUxuhCV#G1N;kES=<~^!DW~e5T^;;uEktE3=&uq#Ab6*h=~Iq%2G1mFF<7?R#@bM zI0fh7;cvw7`%2Cbb2viG;jQWGdzN4gAU1?}Y}w+)K@(jSeBk=p^|xHZ1L7|jO=wMo3Iq7Vh8ihDAff|@i8M{&uV1Gm zkD&Mq<#-qYguvY_RQ!HQ5Lxm2_ixNW*Z?~EFFfkX)hlCKq<;LEfVwTLhVl@(3I*x0 zKmGlSPot_6!-M@(DH|9VD6~ztq_h-Or_wSq;m@9VH&s33hmAlo`BQc_@h|7kLzt!t zvkDd;*jR2P?_~lGqFBFj<)-GG!zLzK@I|0emn>UW4@6rzgE}9Omas?T|JJzUZOAy| zCc7ULi#;UU*z$IAppL7&v{=_ef)02je5UoT>?q~r!KdN4S@YhQZIYPc<42Er&5Uii z2J6<-MZE@=1$G?Ow`TC+2rOm!y+?rX$^Yuf}kCUB($K65hDIAGjXRj?)kRo2ZsdxJnH5rB~g&ngfd&%HMDHE zqsGRW05BjJQQ=o^z7d6t&stTW^qO+rb18!HY+>(iaRnGI7 zy-GyryB@!l*s{e2;S8|antR2d7T_|x$GacAdUXag3NJ4&WEX%4h#&p*=)h1A<6$#L zMn)DUDB=OIU*Ijk4?vdx;sr5-@IvzP@;nd>1M*d|B0J{;mB)M#8CXB6oc*cXK!DJ0 z&A}O^YHE3K`HM4qR%0_Bh!f<{_eM^~#ki z`Cg6FP4ty$z&?rR!V*Afp|&fxm*70?@)1CM%FAQp(eXM+%Cvzr!f?RE!h(V8NKKsTJC^05Ab7DuS1`uuu#2TJIMSQ1eWI_ZqepJSeP{-x$bT@Ye9mTII8vl2-XF zEd7?|W~ek0UcNkmL>R8Iu+|R@^&~tB6lIlW0%T85khtbJEdtvZ5NEd!^CMbDT)8p< z78ljHcF2P@JCK3W(cj{do15W@6X&swMU$5;gojjvhwOPs{a#!d`nmVAc+rNHcd+Qt zkb)=$oV+mB)Khq3MKSMjd^Z+ji(sTnWZ{t(p!NjLvC;Vc=)TZ%AX2@iN6rAyiMQB3zWV^27y#gI6_pC&&|hg`zq4Az;b))^ zb^cF3{!g3{`d?&4xg*Xx{7Iv)xuT`36YQ|l2HTD0ZBxH#4saxPHVpvSrv5Da7;GF=1TtxB?q~-)?zi-(xG!2r86SFG(&Vq3 z^JzF|S3;IofrnFIzhicnJP%DbxkA>(l@Xy^UwHD8X@J_QnC!$VtB8rjotHeh^M@6c z$=*5VTa{|2p2&M=`iBRSeg&!+$J4$`B&+wx`!h9klSuuci5F)gOLY{tZQT;nm8>J( ziMD#;)3k@VpVAAzx@{6{RJ#;=tU+qi{Nj*em20<1Z&WYw5xwNl_oJTryx*|OYH69h z)X^mR#hvz--_hct0!!s(KU9}+c!F22=DI$2D`J%EQR7Kga&u0(7cn||ipU!M-8ue~ znm%qHPd?mcZ}=u?s;0E|^mMGdSO?FsnPSN1&OE-ysA)LjcS3Zf|HQ@V;s^jVV zW5=x5PCTPbQae8eM@6&-cI$LEyy z!7C+N%eNf*UOC5YnYpjvh!RsGKKelK>>>?TM@8A^nk&B_u5R`HHP#d$*KyWaj6ACu z-_RjPle?tt+4VH*a%Giikx6yu@ZdEfQ|!ba`lKjM&rmb^B0tY#zcHLh z&kIYB4f`Eyly*1wL-fn3kjoI0y8=bRl0mS*12m1{K*Zk&4FQA=sz`UkcibyJd!`GA zp(g+?LhM9kPc^|;!9n!=4Z!J`Xq$BN#KVA`}SQ!H9WirIG!eW3v+2cJZS(j709$SHZ~G{Vvs!m@4aHpkvNh&fYgPC zq=>LE6YOnhvVea!1(U!nfEok_b--4Nk9UM^BGgW8R4ykCy(XGf#l;UUwDj--phmu= z?%$9el83J>rD1ee0ndk|C!w-~b9gkMoONQ+y})L6FU#d>-l64-wz$(~b~Y)*6_zhs znz1{|^Xle-@<4S}`lXAu4=CBj>?ZrRlo^*dB%X@`6FV)Ncncn_x;Me?U)W~g-Px;H zQN zN)6DJMO;g~3sQ#Dw$Xf;i}pE~dOm0k;B7rj&l*WRO#fmzr``U~1XY0faeC;53HQwd z&8V=PIz@GbH<{F_0&8xBXNXCA<#AC9Tg&p_MJ+K=&)J@9uCYJ4zA;F))FoC{HX0kYWc7Av)%<++O#KX=}C?HY5tILxw$7Mo;|rw?U7=d)|nrP0sbQ9t6j%W*bb~s zAL{UF_S4piLs1cg<49-uWr8>00ju*9AP2cnAT@NTX{oJURe;Eh!omuPZ5z8*-75ykI65(bJ>Lb^GBF_m%ubYA#dMuLf_A(i z0p*LSg|sbAw<5Lnx~3{9^eOnkzuP3@?5}kQeZVBkbxj_o2g}fcfDTPT{hJYUz<_b)cf|6uVzn(o{XE#2-WO`trKIf z$l7%qWj%%1-|cB956$W&S?b}iA-lOsQQk^jLKWkbbhA;%n@2_I#v&t+ zf3X_+8-ALpq-?xqP|=jYtkRMEm~8gFMdp^BQ*jN$>&;3lkM|81o~0jr5&W{j>*4mO z{Y^;&r6LBkPgWh2?td(lX@4rczHj@2YnT1{ZDy+lOcW(YyEJxLuRCHE9TR;xi~fCR z`{8&!cb@XD_$_&(GlSVja_n7R$?dWZuZiIvS>z)%Fh_4}&GO|JLDH@lf|li1QSmK%QjC^toJ&xh#F0GkP}=mM-b+O1&&{HtY>78k?if6n-hFv+{$ugm5<#PPEu|fb)~_C2Ix4&&a?{;zq0NQytBRM8A6=3xOnH|3 zymz2)rS)r#xeGUy&)a@Ec)yO~`tHV@oVa>ZL%NuILmu6-t|ceRj^0w&GCP~q-5)y~ zlQR`J&bDCmJt6I>NOjNc5D_4M)u{Ha&3Lg?Y`}}2$55k_YsRCv-ZSlQL?_?dsgH+g z$MUK5?$`DDUE4Yqk|`OjK9v&4?oa0QGbmgJWhRu8rQg3I(4$C&Hd7tNor&T`(q}}O zbu>;BW%iD)_iO!r{`@w6(Wwss?8Yc|BSf8?=MA~tNjbyJXe(y41Zk#BJ64SON-TT4 zTE%MguJN~Er^uc_3af)Q;;$^O*erCt%Ic@QcRT+B1p%*LUxM|cSY=VHa^w85Y85)` zhdlcSbuyDvAIRwq$4c^^F7On*D>movD&dJ!=E~ccaW5G zC}tX8DCW+PxG|Dhp6n#+2I`eA$ia5F~0pV($T(uBz*rQ zb0?|3f7HpHo5f}vsZ)-DK7-arCe&veYkaF+BVBRBX3v2{laQx2GskHY$BCg1)~8Gd zP(J-RW~tV;RIIn5#it>rWo*Q(MEOS(&8KI8WNGO$8tOf3?d)?)zu$_+Q45@HpoDVz zdbpk3P@6`yTJdcw`ID;O-;>XpjT{+sVNGUo+Z9P2_nbwo^t3IiOBw5EFN2vW@}5cG zyH_ZypOBh-KdUxD%j;|4%;s>Xr*zpj+)7;A4qKa@Gbz!~*H)W(73ftQ&PeS3{Eptr zFr?44wU3Ls%H*-zG-J2}NdnxjX;IU`rE2@6CSSYw{&D0CYE0-R&!h>oOeO`gOT~PD zKOuE+h`A1kntV>0{U+l(AkXO{F<;EKoDhx>)S=P+4@gZurF5*Ibfk$+_Xg132WJoM z4|SoGf8n(0s(9MjR;H7B*}+^G-deJQk> z31mgz^=0+xc3C|j4V4*NjYs|Nmz3Ei6+QSs;DolHuJ`ooMgF?8A9W41Cw0G#tL&oZ z(DTEcv!!N6UJ3X}%7dIcwslqXPcPbRYXFJW{&r2U{vkcpk?R89_)P=14orWr{*$t| zPECw_M&3@pBdW~rfkxUbK8zM|;e>P$=$s-6^-IglGKMZppECILomX_byB?`qA+Wt` zolEZHWkpS`QyIb?c9f}04lQ=R-F7-LwT#>zHFBg(wCL=w(sHX7cbboOtTJ_`iX2Jm zJCfG8HF2`Xr?n?iHJ|SNNS8>xaeEt$z@)x|&9_Eh4ft@766e9SMJVqR@sX46AWbTa+=y2qv`wAH4|69w{jG-WGu>Y5^_ zUur%femUXK*L=11X6snHyZX3dzNAS{w6}NZfTdfH7tNdAx$1b6cS0^xN;%3wspopA zOP1Z(=(mIM$)suDre7l|`8^%c`u$Ial`GV|dRiD#(UBwXXTLYe8t8ibWVbSh17)IH zSq}rdjYxg9bY1RSx2deatVt*Oj0Yt`*-FZHQlNE~&Fy|5#_auBYOC*WuvJO5?cHPj z{x4$kj3e7Ga9+0_&{3CrK6WCoXRJ4UpibMN>D}|VjJh`+GxloPZ-X7m-xyM#i%Lgx z`nhb`;$;~bdc@i+Qfmv+*A{Dj8SL=RxS{2x+H1;<$br~B5jwL{AMQ}7PG-?3O+u7g zJ>IaqcdmVElQn3p9h;pL{%&T{iaSZ0en5K^b<}UWXyB1Y=KGY$2@t_O)i+M6wD|ev zdvn(jzUTQ;5jyAUpfG=YTJrAM_T>KT5YtV!E^=pSQMA^JUzXCndsJ#JcwN1FT=YDh zO`7K8Zn!m~L7LEr8k34To5-$rVHd<~|+Q?E` zLAyh6M$z0r-EcvfnB@nhcs#bAg~46uuc@1x-sBbqQrsFO0T$fUx zJVNq&m&ZNMzu1EFwT9Y9ZJi0_vdki<8$?|PB$Y`FnC=0t>wcTo$*gw$q~L7Nk1mUb z*X6&@eDq}gO;HQWQp<+oQ#RRB0yC>jb&3u%<3|Ofe@dv$N{k1ndODwCc#T}x@?=)W zyi9a6jFS4U(vHz9d-L*LU>Xh{5bq>A-zPI`qjdBZS1wgvU$gI6Of9nEj{l5VrleRO zpL?yN!FXTDsxa`fd2Bb-YPM&k3tTsMwofW(8xs1d=QZ6f0 zb%WCRuRcw52B;0k=)sexm!8?8qED1X75zWY)y7oZCF=(;)x?p`izo#xw z^%-uq%%{AeD(rP61>;>6y%W8~@7Oh5wi<29FpynNRT$SSOHcVAQGemXMVekBCcN*~ zh1VA6o(N5@xtw}>yK?hdQp#ZixsI^_wG8jlOA?RAB3$&f1vAxriv!03)DJhRTIMyV zjTrPk@tACttu;|rS*bpHCOSCpQsR+_yjwco1iagLdwd*-z2rFM=vA~?bo6&azS6CI z`wO0wb>0+XZSb4fwz(pea4n2hFL`&5g&VyGt;MYUmRqBSFZb)!x4(a;y!&>~?Zyql z8LYK##Ry;CP#)eo@lFP9?g-37^I@b~skJZCeGlxe75*mJ6C0RPAy+!OQf%g{X8ntI zEyhZ&FXQ9%we`M6>GvOZ@vMbUVNaLVIDgG-zFc7tITlRgvT0OM&$UsUD?7TxaIbDM zftpUJ2kG~>deQ4lpJ>=uJvBJ}BZ|`(6(qe|Rn@!7(Ng2{fq`VJv2JB*mi==H_0c;e z&m`3Pj?@rFeKNt$2k3}ENG1)FDy^drHP!m{v}4_=vn`Bf#rO($5PN-B`tu}QdYW3V zHlK-$ll~`b!iqaUF^q7PZ7))sIjD4%;adUIQ!E~v^#uc%B%IL-Vqn}*Bp}f z&VicA22s{CZjT&wwoIAytFN(^DbTS?MpN|`^74__COxxTTzR(+^e>7s*rzg{4-jgKT5^CWciV~kp2(ty>ppVjm? zxMvQ7%B8V$D9WONx`VAFHda$Mlrag)m9_QR*~+=L~Bm#WKxFH zp7k?@sn@N21`;z{vG!a%zO2(>bstx5nW#iq0q8LkZ^iD%rP+T?JAvoJ^ZUZJB)9zq z+v@CE5<80Zgfhl#pmJc4IE1G{Zj${p*?SAy(W1eHTw$ipfJ?ZbWNX)0aDD3lOEB4^ znAjjH^xW|PAZWmgnEU|?`q_Bq6uyc8gF7O?ee~oWfhvgWxi??ptCRKs3Suxgtu(LC z{FWDw%TT7jh;jaj8vTbfn?{;_r<~UZXwowt$xy8iAl5J+WrUB$Or$XhXheiSGxeGI zlf=z6M8_WSk)}WEQkb<6cuW)u<&WrcCv>g2%mxy7riMym+xpWxL>arIe^NL#Tob9u zB=@{~x}@lzc0ToXE##hvzXlgbjoT)`*>EQyGq+35$M{RX;>>J5eHQLcD8gg4k}Jb^ zOhb%88TO%R#whYtg!+^}k?-lClP7C(`*<> zsk5rxeFxd{Y%A{QIJYH@JI!m~7gMU%_So+dcPfxu0r$sdyHiKKEZK@y9!*xW_X6|j zo1X-d2;}7A{Uv_b)9w5piMTIxpUH|f{=zN0vstgc1m;iqxV%bD)Q)YSv8jz#DWMnI zPSGw~Uuxo&r;ZM#^+g6WbKBBLLm{Eu(9D*Z7%|TQFLh20XGpB)s_jpo&mBNaE;)T< z(m3XOa zB03#N87{JFVOCPEcGTvNzM+o&p|W1dbG}hLKKJTHjbF3luI3JOCXse~G}oT2XWY1< z*qsvSlTgo0%F#)RxeANR&7scb(6f3X#n|7)d`9nCB?NnN#-pfvkHL%(K{Ab1MPq4F zCNxQyx67p>?zfgl!XLW!vMdm{!YI<`hG?z56+>e8aXToiK)T_I+cV?m*;b}>#njTKHAw~_a1Jc?i0k!BkSKav(XDa$Mb2DmTa)2pE! zOFRoJ9&xi6tHgy>BE}sRzV5-YkDASj^1@q~50q1wF%;$#^}Z+h?d$T}i4C9M zdoRBi;l#?AoP;djvP_?*Odoawg&oNL73fnI>Rm?T&H}(TaYw{B8X-2yH1F{WE6x}} zR)p1fcM~iGsb^13dr?=Z+VCZLkI(#^Ix5qcAP}ewFDe7W@v6UhA#KWuHuV-BXB~BP z9ko9`zn?H`_pDfdh(Lw#X*Ijfgw@%N5gZ28mr5fan{wcb|@#AJDu#pjqLFqHs*GflBNZ@HT;V^EID1Y*;2}6 zM$_{BDpX{1X=563^`^s>3LM@d~r zdZ>z1FXUN{0!%T=*#3MnZ8SpHv-4)M_rN`?aRF{J$$KP`&uT_l^!vv|{qo*HLGGZ7 zDzjag(}Bm4pOK)Br&^H*+`T7bNIi7bY-*tp5O?@Hc4`m1+MV^&k=ssV{uE-h)ckuL zA1dRsmjriE%)91=cNn^aV8FNImai; z#ByhD>U#aTc{+Vatg-B{MpG;M4F#e=2^w>fl>2z2hykZ~9j8Cqiu+BBnJE5c&Wlyl ziKEnshv2I)nP)N@9ewsja0&!i1q9Vo@UVG_*mMD3{s=#HCZnCoue8XI_tLF)EZYPG?R^NM?P_M}zKx+ijZdOjonkYO#rDuiAN_LM+$l^q zcbZ#|GS@s~L&a$i%P|sVlsg_nohjueTS-=m`e(QV2{fztFx^@8BDBd)d9NQz)yaA7 zP7H6h0GC1Y>Rx8h+El-PL4<^q% zPbMOpkzt@DVvlfv7ejt);oI|d$@PR8@5=F|cP2Z(U|6xg*5tQyXTMO!&0en2sk!O= z52b=ep3votS#)_yeGL~cnkuD?>QblrqL!q82u&~?IRh#rukUDnmj-vLsmar|yI7Ml zBw+@_H!1)7m(@JPk{!fj80YmHI@*$=77>J+qviN>%0wXNd;eFDiy7VwcM3x;vpKaa z>lXoUxf4;GTEZ~I7+pIZIMWAsSPC|=#%I`-;OVSpo?4O1j#_Sd6UodaxQW7*FL%~= zA_$+Ma;pWr>G4)mLSjC&GFt0~S9fS*j@&=-VzYj1BY`&N^l&o!zL~S6Z)XqpT<`U_ ztj_pW{h97VG*+VoMR0I~<1O(yJUl!-B_LZe6E?-UI!9c+fL~M#E?j{B;7@^m{m_2i ze}DdTZQu5Pf8q(5`}se9I#s^N&0c5l^P-G^Py{XwY_s*fEptnmH?Hye>@T>Uzq{>} z)p8xzmzk@?)&Ao;AvYe0|NHF%{SCtZah;>rPyfeHvh!~c4~0J;R{xL37=(%6HSwRH zHd|aCH<$Rx3x#B#Ax9GTPN10QYMvKM7asZhYaX8e{wn|TxBJG+6Tfg{iEx1E{9k9z z-aoF{l=1OpaNIz4(m<*#@&Bl}M!24w$Jki(&i?Fug_3u7HqP^G+9{X(%Bktyoe88S zg*#ZJbe8T!dKc}&Z=+)2rw+vuuE`ZREzl&yCagZ28+X$7e%X%VsBbqE{HesR4zr}q zrjB#gQfh_Yq-WGSI4w*}PcPZiV{Wy19$ZVC8|;uLU}&f}SHog9 zJ4s{2ZHTw4sz}RZ$?@7-%b(kBV_X|sdQSgVrSk1d!vd45R(f1d6G=U`t7_C}`v6_c zw<V7N8emYiQ3UjN)GN6^9aW_`AOYKEby zvbI#>y5D>y!fP+rZ`Ay?^YP_vLF*TI9_@IX%unhx-b&`jdGUim*LzAI7ateS3~6(^4*Uo_DnBtCjfB z#(Z-uHZ3+h*g(gjKtrlGVXFZJd`erD)xq|0vfXG3}T^l494s z+Ej&iaTwY7xGUS-tGmd6Ja&kbC0f*96Dk-oGpT4iM{Ww;2}FFa6Gg zD%2ppYfj9hg+D6udvwpfFjRJU-ne>q;+=+7)+4Q6V!4cs^XLQ14=n6U&Sq+*{nnn4 z|F~~Ko_?HDpp#Es?AkubHrkkQZ^t3-UOsw&-)w;3TuNc#wUDx0=O+LAW3u5{B3jlT zHf|g8;Hch`t$0YSXLn9#iVK`lU0!eMDC3yunw4I^r{{%R^*`2rjD2nt=ZuxH%J&s4 z1h;PHK0Q%b$CHl7J#CHU@WE|j$Tl)kBA1~MB>9XT*sK29&w3!LXUt4*JlI#$?PN}R z)z90{CXH3ux=FmdT$*c}o>(fH4m&lT4caBUYRQD|k?G~%ulo8A-#mP9v5oej#B}}Z ziCn>CeZ4DUej90 zykJBlmWZ%%{rkiWIL=lq(gER%^L^(9Zr`)byFm?hb5>)hFKG652$rc%>cO z`N-z(=FlJkso~q>>&H|Xks%h5%1%vJW<(zzh|A9HZ&dFg$uTfQJQqcYsf+%o_Uora z!<=X-FR|S=Vf~_R6#bSWUX5`7-8yMFQbr5Q|?Wky$O{at)r~h4SCcct=gnP~-lK+VvW@#RcE*buct+ZGcoTJ;ra_q5VbM z{e-Qxe(-t>vFFIHLtB`|qs9aMR&5C4zFf5^KQ2y3y7C;a+#}of1Lb0h7491v4vxhzN`{;L6CbHOd8!fF z6Jh%7sm|qUJyWs@|L|*ijc@3CA)~Dd&x?G{+<&ZhirwrlNZ6c$y%^|=!?(}Bo}aj6 zWj4j((~IvZuCMG5*4)bCENb<#9KP}Rq2Ju6rccvP)Lqt`vwI7Fi`1WsrNV9@^OPWB@iphQPApuf;jrMb-(}YP z>iscKUL`q2diBIri|p?TvTa?YH(_toVQ6b=aD6Kr#o?=$+T<^i_&n$1R`qjk%(>>O zSt(X?dC@6}5s9Cq>p3wgFcaR?lWvxIVK{|7O->9%7@NZvN_N%ltp#)zxRR^I?eM zZRekx+>WC%cnX{W3<|PpGvxjm9bB`)@X_i6(ZZYi-+!yzb=>h+zW9Uv#Sb5+1dUJd>KFMxLLobkV%LTbxQ4$UCQSCpWPeFO0)P34gHqC zUqX_aSHEP_)vn#2{79>lmvYU!4}HC|aEtWNl02`eC+8$2Zd_j$Q2*RO=wtOKYhI}U z*^2WD6+3V4A@mkPnb~`snbX-LJ4D%gbH>@(GvQ>U4>BUkmYF>cXP!7CA{jY*gru@1 zg_5Lxuj~8sw|l)`@8^6xAJ6Ce^{Pgd@;54;0OevR0W!k#ForR8^VhU_zOkaf0yrW% z-YRY=nuQD*Bu#WnnY(BDJmGeZ_oTS<2McHlAxmd9WxpxKNlrVk=e$2kYIPSKCCez4 zGISg{R?$&b-C^*8E45-a9Zpw1wNZel*fN8sn2gIDKf1Y1tXSSYeWycqoE*tF(cB!N zk$UGrAsPOuthwLtQvnGc?@ft%8cZsj>;X1WEVOQumcuNR6(A8(V1<6w2Uy+oahZsi z+I?STiSgEk{S+1riW|d;7VKlK4SvoD0Wv{zOESFwmn5cGY7^PT8QA_gKQ2ouk)f!* zs1RX07IHV#ibY$R_sLPudb$w)vEIvPnThn8x37Um48#BfD9DHyW(m}KmcAH6BQNSa zgeTg2Ep7;xH?9=-%Kv6W&7Clm>9!+5zC!j43ac~`;`nRL;v}d9bvss>1nMR067fXX zG%_L_Bm}I2JjoLD%9@!V5C?aVqB<2iH!{p9WWV+2Q@Z~Qd1sYLI;S)cWt#A~p!y(1 za678nTi=i|?)EJIk(;a14JOe+w%2CPDQ?Xw$GHi+u@(b#)0pkJ7 zJI~zRJbxh5FD_vrPz$nlZjC9N^l&BcMkb&1N!WuA`bg#G=L1ix0}2_+rCAc?#HuT?Wshk$Vk!$wQ7wZ;=`CFkyWGuC<*LP9 zcbO{QH*?WS!+MVMT7h!U5l^p-*78PH40P~KQW@OYA`i;${M=ikwRu+LskuuPj}27e zl{fx8CN?qekKli#Jp7Op6e^wT1SgzmzEV1VyF{Bi!nB~Qz+`+kR3;xZc1GQ0zTMDU zB&tz8;i~`MEmkWl`kL|wxg91iAX3t+5|mLGaiVnO19Vd!k@Ns_%*tpE$KDu@I>F(UuCSgqS zo`^Y7mCEEJ{+r6mnFpNN7W(^%35|6iE`7elK(k(c#gdFELTwEa%qeFKcU2pmSX+b* zVFr z0aq^U8Nk`FI|F;n!`}LWON`1J$bE?mc!$}9O19(FywdlzERC-YD}5M|@wXue#F$&P z6x?^oVUbndJ>6HG?FPSCz>e{ybNYI+NJOzM`0wQRuND&+ZXpoEPsJc2u2Oz zlyYH~D53U+fR)zgL}_avlRP5?lBL|t&}h~%bh&Va-IIh}IgzX}MqOHM$vd-K4Ezx= z_cW_)eKM{@u{(7bFWG0Y4Oxxf*H+NzMl8|`&Z@;>yfSC2b?<+2m^s6wan-JJnvImkX~)sWSU)6g7J4KUPmZ8)X0y^xGgheTcrG9 zVERP_ahU|~!CtKWP0#;N60TY=4JTed3T~A0w#cyE_OFAHl(0fdRfT}Tx{~?(6d1vb$~l+;7~QF>SJ$Ygw1FmAaxLh$&{B44%CSIwN zUHUN{0fw{*#=2am*@Q+X;J?lCOLXi?hNlMZwJYx6X*>VD}Ib$V9z)JXKa1cVR zjqIjO_GZI{##1jrPGLckm`3+9o)=#S;dc4aw@?9%68ci16O}8~K0Qr!P6}Oqy^A7O87r^!D=LhUHH+|a((q;=;tYWPb>N@h*h%7ugf6uA{C>S{P$WuOO{|3XbF=A!a4 zwN_kgG3DCg+9f3Tq~(!l0m`N{Q`gEM=Vf{)mY7{vWAsXMUeMzk9*V+Mda`q@j~(>M$oUA^Ty zXH`ZmcLaywC@S=LiU9+GUOsdpgiw+6Bk(3!0HJZTavsb+-k78mmyjatXyr<9)i zjU4v~r>NY{E=VR7of>kawu@Dh1- z?Y%r_aN~PTTt{};o2sS3E4w7C2v5%IP9^(t#HR7p!)~+d(}}mAamhA*6F;w=5#`LH zy>ub$=8&j(y1o62D6lFv2LqTVr=+E6Pe(RmxSid=Tc;UcriQQff0j*NZN_tw=+| z?%><~ACUF)s%!$=(g6gK<`f$d-wxNe_3eDjE91CLPUL9F$_L^>*FiH5ME?UVbB*f9L4@QpB;TFi0#i4rirB@I4k|NWV}S3~m)+Ax%;=ms!v9ic3- zk1H^#Ia!Em(lQ=}|9a@G)OxFwmTWzBSNH9-zR4Ml14|B@@RKfKolh-K?y!7nSxMt& zjNt8|kKiq+_F2&xBiSJZi?f2o>Dr{alziIX#A<>_FOzRlV~NgKK4a_Tv7fJ!KgSQQ zH=$@i)V|<%tADW?!6h>t4EyAZUd6@AaOlBDN@!NAVi!k6iGu;+?H~dcT@MfW`#bk2 zvujpk_KgOtdUn@6ST5g8ti#6+U#V%AJLgg+md{yA1nCrXw7Ey$DZAPf`aMhOCmuv^D zq-#teEzJd;)G`4!@+V=SaY=G5?%(8mD_N#MEu&o~Za8L9D(TOXyb=BG)zt~A+8K1> znhYc?8S~NCOyg#dUF8>agMN{v{k83*a$&6sa<|lsqnNdiw@4u|0TAF^qS5a)UN>9{ zK+I>2#vc{d%u=3l>A{-a$}sy3w`R2z?yS24Ou18^P%XWdWFqy(mm|t~4m)|=3epBN zO@}P`SbRKHI`qQ)k6fdI{sWKhu z-nq%_`=|c^*gS-W8fXczjPXH!bbTmoFe)VKw^@lxu^kNZF3b0rOagD3`3@Qn# z>AUaczU_DihlYj&Q5ik1ohF{QqlFDgB4QV1Plky=#9nmH&5$*nM;$mgAq1{Dl?Ico zEy7uG$od7Sp+~lGewp7zb=Trbf8iwvgq<9?#C)^L!{V07kX)3_wfU%R7bu#@$Lzn}m_x}In}|fYDocng=JRLR29Jc@ zhQx%F|5t_$*zW}=kEY(2O!yq_n41%#Fd5?JZ_X0&Wh7v!qfxredTJO-WuZYN0Ad52 zusW>pfhFB78fgW&%ldTjV3zMiz6vD*5n|ba7Tq1Cv@hwbk32jDnmh=U3UIvs<7L>AQCk#ktL3P`07 zBImPd>2lru5;z%g0a47cP9)TQkwt#nJ_vPC+=lNehq7urEmnL}>xdI}JP-U&qaj{jhDcY4`Mn}y|YM5^U$XYXtXJpK>v8K)>Sa!5v*Ho=7TDX{H zA6Mcb0g4f>H1xM{ZZ|75!tou#FW`Bm%6Kb-+_hL8I)ZEePj3V25>NGM!Vl{nX*k+= z*jdCKfB3-2s$1?r)~e_JGSng`$A}gHPCO-U3><~{s62D98RKViOR#1DOpo63U+$QY zX#{lnRu?_&h#tFu)Rja6&AG^e3>^TI-r3a*rVBCwpp-jD3z1nn`_df*V6TGM(3i=0 zQav&VzA&sWnTpvhJLhrdwEgTGZfc|-Fln=@>N`iGf+$9f^?vxLXo?{~tWt1t z(ldr7iT?4Tdb33oB8NMZ#SP5!7${4aP04bxrISMZC1%E73*KK*u#&DrHvm!;jDGBc z!VGZuED?5F71IxU9+2LZWMnL!1ttlA#q+KbpUFJNo;-dsJs?*By=jw54bmP+UgoE zSaf?!;+7W)jTQ_Ea1I|POQ&2TnEtvd*7B)(F&i5FaZ)6!OT5NIAAjjfJ3~Z3kPNtt zdMZIb5)j%Q0X~oWpCZv@ha`1mevT#Gz@9@@w`l zKU^f;s`Rmc_N;hRON6s!O#s`3pRDvnqDw{5)h0e5Zq%nRl>!_$BC}pH z@f-KG31s%9`^^m6Ok=egV>+b?ghT@yF84vgC_NpY_1?wX`Qk_J#`KxQ1OwJ%pFEDp zKf6-;ZDHdYkGxHl5LKF95lD>X^eL(2+;0OBbmbcf>sIku<=JIh zzuvpBpU_8NsA=;IDp0*T z{fV4F9&wFQ`KTC#Jq>d=idjQy*k<10g@OQQmzhOYh;^OOkMu1zTJ+*3EulwZLRgXd zfIr)!?PLRU#`)d)fX-!)qfwS&4>7~zZSNn4*C72xIAgm_?eE%c*w~z%j={v~= zl#WXy3a_+1*JHB&{U2QQ1zr~cBcV|oh{4IsqY#VPwPsPyNeuPqy^oJQMetUa>oC!d zHpRma9b2z}E|>$KGQH*-nPLz5|Aa4c8-TTs9j`(I8pn@Uk@hS~HYQh|0mDZ3Q4uN+ zAyCNRsq-I-lRY>^A?9ay+v}s{_Me*}5JIe?hoIfpeIxKjyqf%?(!+?IG!lqE3ofep z&U?8uZ0w;ZXW3rpi+%z-=M35-wu^Mr-w8vEe)39h$&%Oej=S3rGu^%~7;xrzBj%H2@@*t;?P_JCtxzJ=4N)M$R5q$AY*br3t zwFb>VR@oHA*VIev8Uq7q?P_8Fr6yJaNWUGtJywKNV5xZUKw1edviLSiH&*|I(o4q7 z3V2$~Jcybhu_z6ur*m=vi}k<^hL_^6V4}6c@_!;m@*7OE!yV}(S zZZ#{HQl9yfdN2^+qtB#(C2(L6!2?y9YfBIBh8ldgQ0mGb696QG z|1BEs2Jz!U7>8+>h*dWJ!XnzM3Xy=WWhI?Q}LN zXqb=Z0d~jMt$jgxc&A8ym`%GZyPW2dkr9Zp$pp2Oi9PWZMdKm5r)-UF4C0=`5Xia~aHcLb z(>p0t>!}HZ>QZvtj#Cpc4{F$`u&ZjKAY-uBap1TU?HcECCHiD3)+y47Qv1Ie2q{)+ zOpXN7Kl*Po0d@D#O=z@uJ&-k38Afbx(K=V))Vtz@JTfKZgO$oFB8bq-Yg7e(rIB@O zQEm-b>=(X)DLLa3fT*9UOe&=ia3>XEH&gPc5?0X+Co&6a1R}>{x3}g#wZ1W2d45fhxa_1v#^m%DcZ4HZt`upb#b10L6bj{l!ar$2;Xx@7z_m_kf!ZBCiJF3K} z8f83h6^%ex@h4LYd|#9r86kpn6TcIv6?)L)7HUF1h|02+GCL;=zG~J*w(9%;QmIRS z_jcpRT?Uq+#xXg&n_Wa@B|Uon$-&FYj0{qg_~+e&F`;;dR-=%b1QSQ$Dj7dxJDkeh zSs!y2X1{=350);iuV9iFz=RG4pm@>FQBHR4Dq!FM=4!d>&HC6aBJ6nU$$ar@AyE%u z$ePaq?SOJv|j;~x+V#qpHHE@s9{6k z7Qx+6i(FZ(*sBTX#4suMj<`i@5sJ>dI{`IE^ok9 zTO8VsZvf0zC0n(nS7Q}Z{akr$Na+VB@I;V)il|D4D?V>Z=5$GpqcLKURg>Mhldm1w zYj0437#!1<@Wqf&*0n3(5s1wqvEmS<-*#|VxP=wd7enxtz@9AC zBd9A$(J7*Q^H$e$h|`6=%J$c-lh(T`WJk$oXf$tMg1G#*^E~*GbSAk}wal>Mb}v0j zb(RL{rxcRmv5X9jA&11c0c)z`(o;W~!1k$%<`NG`vClC1Kt ztx2IrRG_rE1kf9sOx!~z`fLm1Su^%+A6-OF(HA4f6=A95R#qvuOAu(fDGYxeee zRa=|}FqTmJD+w;CEnqUh{qi;6#hAoEz`=vCYfEjyZFH2gj}vSHgvne*qj*jB$(!P3 z;Za~05GnC@(Wxpl@^@pZqmbLKQws&?U~Thk5BR1(tb$o0IeL4MSLfC){W_T90))d(y295RLDrhH?e`vJ`p_T{c7Ap5+pxJ+#GE)H zh$Nv-mI!Bh{7X5BVPs+*u=`#!G#jo@jjBO8-%AfmT;GkLp(la3sQtXb7B|^y)ILVB}$)pZ%gjj;NsV9?rVhIgTUp3Q!~u6C__vBxkz|5UYfS;*}!_v-PUI z%D9ELAqho8A_zN(8tMEJQGs1*1KO*54?{I5GPtQ>;ReVZPMPe_CpF*szOqB46z{R$ zs(5Dk>`}Rkm(OBoD_{bl5Ku~d*Hf*)`nFzJzJd)OgZv7X0kN$XD@ zgDwLoeR4hYRhK*ExmbuHF5|G<4=hsnv-fbKhF*@a~rg_Udf5}z~K?-svlw z*Q3KQmCJR~!zy5!M%z49r>%|wUFn{hLHq6}D&~L)5_1elPVT{u8$x79ai>_}I5~m0 z>(Rz2tD0F6lw+wN?Wix~6H>^M=&x&4)jqZHnvOPEqYW1!snU~=roFcmt17ou?uNR} zA+H&0`b@1Z&=vr7=K)nJEj?H3ivDd-NA00B?o<86AH-dRhrrI=&_qxb`g6IzJ#I|C zu4Z|@Ek3?%ZQh+Y{cw4ty^<}pWsMz7a2Yr+Vvql+r zK|Q1NF5g~q3cV9S1F++gMy8H`_FyR%gHFla?HKmRNS$6l8hAdGmP^}Oo6H`JiQ<8{ zJOPZD2zRQMDLCK*hoNr4)T7ZNdj^#G?M&O}KWxLB&i{dLM7WTZv?ayDTJn2ztxmbU4&3PAEIdAX$; zXNgs{iihB7d%=7{kQ2c`YNN~%=FZz9Sij>NMuNKF&q{0l9QS@Y-x&rv8{O14rsZ$b z^;j>*HKE6lx>Z(f;uUmL+KeNNu8o?%*#4)`*Xc>IV>JNJ-Ab=jUm-9GH2^VaW_Gg5 zJr^?=1J2b?m$5qF9a}E9kLy^lv$wbfJWW~cc|g6D&b!t0L0~=qyi|Ia&;*5#YEXK$;o++$J|mjDF@kOB8B7=Kh)i}GzV%LN_SV8Bu4 zTKWbqZ~jfVuQD?QQdW#*%hww^Syxz+=Q<8P(?NregxvGP@C(=^LfC$6O^ny(fCTA?Wzxn1SN91*Y+3tN<0aiUO*Uj0!uy&5o)^tR)h2_FX zznlw3_0#t90ELV4K_!LDi$i8#?GEEGVD!Lml0UY3LQg{6D3teqi@*j-v4I89mklop zv5&uG$vxM2wOKvulbT`JG!rLHe&z^?al91wf&V3zcsJ+X7lM$EN`ecnM8^jIsl=(( zNWSnY|;wQ4frA&l;b?s#(=}Eu)1YI6nH}DN<2X@ zvjo8?!t;Fj6V~gd!V!p#+okIj&@0i8sQJ~wbgm?)g_Kzcxq%A>EK?}Ahz2%G=AOH zMl#O#HTGlJ-h9h#^|q+6W>7-Mv_XLEXF4QyNe>Cm&URiqfQCI|4V{2f_zu(VI*v`SwT^ zo9HV{xiz5;&`q2grC;!xPe2jc@u(x9+hH4r0dyp2U%KHKH6>R3@vQvAseN zL9SDg4r;5L+_}RHI>7Y=cLNhK!OIa)1A_-!eRvW&5P>=vB;~eyoBVDY<#;HM%WMOG z52$YymmQu-%RVemr-j7uki~t_NQYkVCzVN#gB1u33^rv-bMt6~)_Y=z+T~zQGchhV z(FKS$BRGav#-mXpWi{&kvoQ6`*C73Pf+}HN%EqaHOUd1Imk|Liz36(B~pseZ(Ntj{t4mp&5KcqA5!TG!s8-J9Am9T-^IVjq8s_) zpq~2`q?frMIPPmrZFPARTDXnDfS47)h1)l1A}#_&!8_u(54T{MC58BWHFN!}e}TNv zGk0`Rjy4a$6FCQpX2*wXd9qjrjUc?dahAt2w0Qe0{PU(G3qP>`ZZ)aTvIi;~lAzqP zU;*6TVw>b7Z0c{PE0=_t1c2O^aTQ_L-hdlp8?mPf@%-^>n_D2J?Jq;KJUHY-$G12l zEkWP_V?95(JO23OK2q8G*Xdf2pSYg@y~4_}z3p5sm$(La%_Yg5VZCc_@H>+~dG}R~ z2f~9tfUt-SFihhlsKUE(_n_Gxq&h$1f#%7PX5>-otQgkk%6pB}i~@<1Sb#NR63lXC z-???0mRl4Aw#l(;_6GJe)uZ_nTpHzv)5BeE4>ofFs%#$u7I-MUVzIgkAG1t31qWcDM<%C_k+4`fUQrM zxijhtOefjpn!8eB;OZuc&OMa>kY{EdjdO{c3e|=)1EzIS5F97*PJrlbUYj zB*h5mJ;y~5m&EIFn4O%~((9L%0aXGiqUzhXh@f;cXq4a7d>$~{0p7iF-`9lu992bq zYYNzG3>h$LO6uZIMHAd{0Og>;Ox#%Ta&NAc7$I>(@Kj|SzHV~Z5W#?Sk9;t%%fp`U zk}%~AetAF?=h~#_TbAj*V{m?k`+-21)Ckli;2sn{r&R{fZz-a4F8}#APZ6Y_FL7mFvKJM$%nxl7VKSDN6&& z;;D(?1dJcorj%87Z+`@}%xU01#^YULU{Cv-Q(NB!>%|SHbY^hgD%osD;{xZ$|9>(` zF;|%@huW{$Otw8-^-CxG$p*>)#Q^3`5;-psvv0|E!nO2s%r$6HQCxushXmAZ zxkG-KKm>G9Q6xskkdmA+3r{`ty`Kl_t-0G}Uw%oD*0GfW`(LNYxZ#vPyDCR}##UR% zwVTi?ZYOT*hh3;-YhNG<1pa?Q5aGy|8eQ`mssiYh=OI7(0N5GQlAuw&2UK|3osB~` zC3rv?eN&w`aW}NU12l)BJb)w=ts1$i{tn2<#V`gevC}d~Np3+48@2^~iyQ>PQiVxC z*366i7v;?vKJZ`b4W;350i2jwAC$Ga9@v?ux;{&ydm}1v;cZ%ZX5t#y%Ttm#*UJnr z{gN40J(!tZ=BfODBXwcueSvs7@NG%ZRuCu20Iw5{Im$WDW7olCf+HMoz7GmekQfP4 z@XK{;b=Xj3?k9(+%X_%-2#gZ2_EkNdh zg3$H_WR#yo;0Pe72+!ZSW?$=f7_C~I+irBnU@Z!hSsBSs^aOC1iUO-P>SN^chLM4t zj-G%uLGR9qR22w%fiU(Jz@H9Af--nbVCh){HV6O}*hTCS2<>NCwS4r0Qi6Q(bXU!`8N-{$sRAR-WViP37gh++XTFbe}(3adg7>+8yUz?`W+ z6dkQawU$P)NmN>HLsQm zO?nV2wtx{bcm5<#CIzhOiw+z{iI)--ZuSDf#B_5^$V!I+)~>p^jpe!lZ(`Xc z5Mo6@@K&?2mQT-$Jt`R;l?su3NIv;*IZ1wp;d!x+o9QMj7@w?$W0L=Mw zG0lm(0?#NQMp$tYV8Zy#BNp=2M=DZax0lC-I;_SElYqxX;|0Z^c0e2G*Wfpv|VY4V{DGGJd zptyG`_(3f$mWR;}d;vBE0E{|#OKD_mK~v9kGEqS>DAHj=9RZ3^N`Xw^h{8Vwjc&O5 zAyzt&QB0I_ZJMVcb&{1DF#rt^p8I&flx5!7u1?8@MAhGu*-D!HvDvNn#DqNI5AC}M z_V%U#qoNg$rNajFinw^}q`VtfI_8mLU^uux2!u(dnU8rUB!MSaxap+hM-JG-$EbXO zUrJHjJFnuxmbkjl)zZibV5j70eaL^o2=SSL4;q%L1#NbVya$Sd?)!oV>S0v2$HO45j;MDM6BaW~A z7$K$yG2^(N;i7QYEsvLcwU-D&0*8l$p9cQvT4sOe)~}EjDscUbizwA5e$F$oTEGY_F;#l5|Gd)IlEk3# z73G-a7F%<_(;Bpi{CNpavr;Qi;^;Xwo}I(ic+p6M=v5}+h!O`2be)=CNa)a4OuVk(RW`g4s} zqX8{C>6ki7uAWE?UR1B#aWur39y%692QyoiEdf^De39-6 z>a&j--h#ClzwN#1mh3&3#;U-n&K8pY{irk@er8toIYBZBWm z?4e&gUyl~z-uqMwkNiQSx+DwEV!=o_=7MVC zv*=m+f1N>Hrbkc{oUQo19BO^T>Is#(EQ!6tqJ(qZKg}-^ng8zmx^9qMJp0~8Y=IL0 zAmKGAcZ_qcW2>1H*~mFReeYigYi(|3oCq3JiN%zeWPc&^OA7(5mf>(W8tkXltwvB= zg8`jx(rqcTX!s!r*mCTf(GS0Mf^dIDRkqUjioQS3Ss4;8Wc^gV9uX_2Ug_AH?jd$_ zg{L)0{i=GwgJkng4MTBvtZ4dm-?x$bH5b#-17;gu2 zub8~vv`?2H4y3^4DP?YT&qSTLe{)JTV)A4roagYqWEn?J&|Q-(E?;ec4DUblW+ZZ#s@)}Zl<;u`&j~bgWKqp7_uoXO z;`YePYOy@b4dsb~>N4giKU3MH+{+8sb6uxd(JGs9vUyRId9NY^v{;0q16MPsA231Xd z>KkW#LwsMrav=>i8JgPTJ%s5PIQay~Vw0f&8ANTgeMWT-iRG7P5vu!6N( zwpdi!?zWwE?5!t({D2Sw+6}9Fx{fv|$0UZ)fQj7IsRydEin)nonqa%lS#H+SHdRKz!&j@wAZHgrylf@gK=)A_r$vq%I~xFo|-$p zeo&Uw1Bsxf8pzU>9G7X7^^4AXrcg-38QBc_Re&dvoX)RNSvlyg^g4`wM;8A^ysPX| zlk`iSNhy;Sv_Zi;OfDb}bEiFOHTgr{sr)83kIg3ad9@6`321V1)f!I6<;pft??~ zDG7bM)rEn5XIsE^Xj65r;7)*Lbe*+|H)8c)(iGp;qz@B$S2Z_9_A$EA^YysmHOqDB zau7LPK=-ht0N$#HKTPb;LF`LjDiHMfs$I@ye_df1!Y%g1(C$yBB_7M8(wf`ZhdbR^-*AB;#J%+a*x^_-51jf!^V zg{xk`kKl3C*3*)CJln9Z>47M8`0_DRD4L3{xtr_6bO6ws&dpu zUsGW?@=#Op2iK>W|3MP#EEQOIM}v~&J5BBG2(~?2^Un(v0{c`C3?4k?Z?xluQ(w5i zC5y9d`>7=sH=6u1THm~HUl^m0@DI+nA{EI^DEoau(y~>Sp&Sbe8kr$IYt4_UgH7$t zW?$jHt{1)VG?PCv2Le>T(|-s@C1v(zYe`Ono}TOT!NB6d6(fZw3qG{cf0mm+^~8+y zltQ@sE7})nnNcObrI2$`oB96e{Z~yVx}&0FTZOU>Z592= z?#CJa$yVuJoX`{#rXi^8VXuV&z-kqt+m%;8PCX!lIDo;5j)MDoGv72TitC#=Uy4J0 z?$*Qau-v~zbyb|znI&P4wTDj?vbg7JR;(>B2yH!Mc+5cj>bRLv{J;fsX;OqX7|~c; z(q8SE9tYbGR`i9b0#>A2`E~f%p>1myeD-kmhRykzNSJ zryBScK#8l0$StU0@v@W!?^W!!U3csU8#pq_WZ&DnxPI#RzKzeZaKAKbukZ!CAgzPE z`Fd5TZA-=ZMzM*QcU@hz9t&T?RZLR&zO~v}+(4D_^epol zG$3RWlu$F!tZee~JHgWkr}QwXiPn%rxr;J+&CFjNJQ8~JBzcoBfPaIj(afR5`!UvW zs}iF}t*~!#a{IC$Y`wQy79tOiRq{BfKbMcj-;OQl#_|_DKAlWHr)n_$n;M6Na+i>l zSEwaq?gXTkbsvaN0VitKC#XHal;VDWD9VaNiEZhauW?O1F*+K)@U`mkEz10$E8-_Y zC+uu9KgQaR{n~0C1 zT_EDe%j<*ACWn} zXVllqNL?;JIs+Rn^R$obTlLMXP50lDkq(N!Zg97EDU)aZ&0_5)*{>Vdu#IS6SCKc4 zEWP$%jg$EbWbi>u+5j)P(+CbT|E?;8+dGAza0-ILEd=TceSNbF=GU*R$c5SkTZsA} zhg99nZ;a|q@gctesqf;bj`Z)nGG?9R-WEJYuX+XHb?NklTAX^doZd)_8Dc(Ec~d;D zukhoLx&ELDnoQEf0P8VI8FF^yaVBpkgZNXXMc24wF2d-q+liMR?&mzLOvP7D zb^hCik53%M^(2Q*_hvQ(t4|cx{DvtpC{0x+^f%Q@ z`)(6@ve6%$%2_4FCkrYIXV-QPg$4Fne5Z@le`W8O27N+X5BWH}A0>adVS8el!(2r? z9rhvY9{!}}2ebB_lENy&`N13DDhzN_7`R@q$fU+X>O_9<$1}Y~;TwX(s~rJ`-=1uk zb3{G`ohAy>L2JuJtelvLtNO{Gq9}&Izn{`=p-O`~Va`fl|Oj25)HzZ*)8N;jwP(8nyQy?*pUWF?SAyG*$%BiZGPWs&##Q!QR`tEWO= z4m!ryz8&jbDsWip=kZZ425koL2O6+_NwJ?I!Zaipl06xGUsR1RoJa0R1HJGAy`-I@ zQeTrhIG#hjjElR8Ij^U;@MVN-XC_$0vSo^|4Gm7wXO&tZF~268jIV~oKQxOC&SK~UhbeDB;p8Fn` zkVQ=ov9X0r@joXd9E9jj;~TjlnC`bysyVpNY-w@RO(uom>%}Pq51frswzQ!_CxSez~Vi%wI%PDbH>ct^@Zl z3tCh`iyB@pI$=E^{V{rCGxJt01g0+&_UKfJ)|S^h$lzy#Z8h7*zRZ$;)j0YyA6Id2 zL{Br4^%-TTpkbwpxAq=h;XTt^YN4AMk(X%#I{Ov-FYW^NpHMibC z5I=hF!<`vRXU|*WZuv>E)B*xs>*#C6lNFgT{e}wf$rC%z4W(^xk+@bh%+UG|^U>3z z?S_4(3BD|yjmMeGg5Qc_X8BAPS&KSLtV7`&mOEf(ridn5fe)I?qG;>DaBKW9J^F=1 z?x+uA-zyk_oPZFb7C)ef=msn)88}M1H|bx-_a(X74a?@g8eh257=-yT=qWuJ6RZKh zgb*qiK#mj9sA#lW1F)N+;A{S#TmTIhD2TnJ?+Y`(LVed<5Dg!N*7PA=2Ach^>JRky zu`kO;ch=~(qUKf$s?#EzN0WW&1&t<9)UDqK(yk1OEC;9co*zJ;27d%y65H78pOYZ?_y7rq9fSKd!bn2N_pJSux)kaqz zS1=`J;|I$h5hHbx==Hzxo_F`3%!$_ z1@>8o9J~Y!;!8s4NbFGUv+lHG?ny#_vfvQ~3xoDT^Dlo5zx5I98Tg(dVc=rx^-M_p zhRhvrJJoU{((4|Neg_9{2D_GZIIF_t|9iazOK46Fy!pJLD&vF(0tMZUM#1>04ht!* zGPdR7`aflnF#T4e8h_2`2kS|VbkN|HL{t`54Sl;j z74OixL|Yf>Eru%Boz1x4Gl0`gc8nA6UB2~4jQZ+upQ#J>kCB|w(Td0IyToh4xI3?w znwe2MWne6T5GRdixphYCV zc$e&tS%H>8Eg47W3;)?lyXtNGmyXU)Cqx>3MebL6#Zcd}$Tt7zIuaSuKN`7*+e-WR zK)8O^VPEGvip71VnqwzA^XPd-0u8LcYrIU3u1W`Ac|;HE>PWLQR+ z% zT7Xf9+6$@NM#m%9KbH9gx`f?;=`V^ZAsG#Z5;7GH5!U_LcLFaa{I@kTHwMZFg=|`@ zlgeInWAv$G{mO553q_p|MX%XV*ULXp8IsX^vREBYqK-spPq`0!gDnYzLc$bF}WDql+F50&q5ZSN5bQGCVwVoLH)L;`*!q1=?^`L+B3Su zxp*S|Nk~i?VBmctqx|pRH<2S8y<(A2!&>=5 zD{AhAtLKl>bSjv)@l|K@cdt4;OZuQief#?|5Z|2XSvqRphB`59ZIJI;lBlA-=_{0f z0b8=zqWg=u@A@YAeTwv~-01hcQ>RiVfBENG0;7bhI2k?jC$58gLT!I`ylC!j{=1@}phg*A&8YhjJg#e7zB zInxAWcTS?wz6QRo#`J?46aAtOjoX=Nv7ak_exj_)BlRa+?{XdR`|LDbfquMagB2fF z#O)Sjmau9dLnR#bhoxy5hY(#oR zONGwNnnH#H!y_RW-q4bGede6p#N!(r|3}n&2U7k1|NqJ;S%o-8GLAj7GE-LP*km3% zdu7WmJ3_~?vLj@V!?DRq4%s^+B%92H((gLnug~}A&*Gft^Lk#->v~*|$Nh1;&qS`@ zf0j5Ree{D$mXr^|1qH%Ht;p4}tD1R&CKSX(;t|9pH#ali`#7M3wxt+~QmYDWI1CCj z%IVUTerOd0gwzFLY&Zz@IBZ|k@O^*wZ1hR>U}4rmI3edKtp)F|Ln&#Q1TyU;y`BEZ zHUz-^6zk3k@W3z1SNi4|;BKt728O6l73j_3?^pjjK^LY0+E`thAdB$ua7Cie19U;f z7U(b_Qp|J}(5L?7B{gsuI8y@H$88|u!v(b<^OdyEcYd={U}2QA+JQzHUheQw{IKlR zjA840O1dXgVr1}1HpFMCslyG{zZWykf6AwoH>mTH_e$&~-EWqEbeOxtdJ7=!c&o^D ztZk?C*tw}NH-4{!$hF)%k5NKcJ!N#6upAiiL8Q(C$@h#eggOHvj%eY@rsFq*z=gyv0*fy~+)l4FS3D;cF&N75)&=Z%5JMwUiu`w*Ugy=b{D zhAeAwcFEgQxpZfVmC^q2)h7m(G5V|-Ol|~IVz-5om6?uXnBtUMCOq2x{XX0Cr+bcN za}(s#opEzXo7>QtOfrpPtK^0D(oY3Bn23r`CgprA)bnMAThY28GNUw@0LOQMQ6Fs?olt6G zN9`X4iY#uwxMnG#aeq$xNBfY>Pg#bt z%$RnT`+ppFMxWGfKjLnQMi_q1AN2Q|7XA{sv*;}v7Fcm&r!Z9?W_+tz={%}OS>r8q z-{N?ZJv2l|81<2|S$Fkz=p%m!Yy=D#4XMXt{<+MH!cV8nwGr5{W?z&m13V>EW~tI! zHc-Ibfltbiq?HD7WQOdY$rC4uwt%hE%9aA)k}zeH3=zh>jc{U%jk^_8sQ;G@P!#eS zA}z2T`ZO={VY=&~=5xC2?hd)ycGuK$m2`5C7%9Jo3M(($!(_99s3j?RlrNq$)nSsK z$lcnY<@n_dJ@QSinB^lTqMt#__?AkwUl@%fngLM>#rJ1m#_8=92+m^fN9j4%u_?#J)LeUlh{+sY^wcca=^&lnwqEKaPX_(p2%D`0ZzM!GV)pcvBgR7zTRfaU$}3Gb zNj$*$`;E!~a~A2=b;w5HyT4DSyi;synZNr?PRTp3xCa?JV8+BpQ)w$c{Iq1o2+_w$ z1jib%KG%H_?jaya3`dbIGD8M4w^DBW@!YU{#X6+O6G-!i1V4u+n;|fvPgPk!Q#zlv zG|N>{p{ZblUMXiIn*q=7`|`Q6N#%1OO3ReQt*%41D*TIC0lrn-nv*YMb=EZ%D#UH3 zSv6^Bl%>m<^vwC8%2k;HrmfyV7^EV+ypqmPXm3HNhmViFcD0bbrgOW^ZE(U{J)_b2 z!XwuLhRU8QMnO*~oMdL`pANL$Tufh7jTYo>Em9fX9LpGDU(1v^0pzTN@D}DQ5A0o) z#rJ##S99_}SOHu=*j`Uu9g?L1qis&+wVg`pNMd$LWY?x+mEXHxQ_tz=z>nt6bbeL#4V7e zFRc`zE#og2z*3Q#eWfD#Vxu~qSCiEnBlE(L6N;6?I!=v18I7!CqwmpMr57d?OAGUT zmVBHVCUf%p=>1Ppk9tWe;gKcaMyc?I1kddj?g-iIxLm)TY$i8N1e35(j)Z7Cz*ZkJ ze;6uwH1|s@Z!R(E20HW!W!A^DX>0>W-J1Om8zT;pvlfTHP^7YM_Xi8gVv*y(k&J!c z7BwM5CpfhCj#4^LHM!rf=a$a!*nN;!7?-jVPOGv2heEf=9Ac7!PR!uvLkjGx4>xR& zSZJA3eq5=rl6o%(JImD2eo(CYA>_lDgU1=>a2|qs?Ntv8O)!)sl@-8iR{|25(v2u6 zJI$t>!mO

@h-WD|E(7)hd|9uci`hCVtAOfTHOvwdxGQ(!8+j1Qmx_ZwRy<#L`r zbdrMOtUgCZpYRiB(**oS-YGP6)^Mr{Fu)D@ea-hG@sZQF70w^ipmg$X>R-L~$h}gQ z8-k*lZ+2R7k(>#f)as1>XakNegio`sb~|lGwIO<&7^CyJ)&p&i`(Nmm5K^{|7Ef$D zE8GXIfBc~qpf~`IwEBsLDK4{vsXxM%yvblxX=F}LF3Xlk4WjrS+YAXR7PvZvKBx&< zoUz>JjpW%01Hp;DbiXjdWpw|Y1AUs;Ba-r>$~-BC*R<9e^4iogi#GToJ#O16CNTQh z1b>#-IX)3NA1b!qla~ar$&wD_t$4fn=R++#TLf=w)NUAg9G*DHD!b8Rmoi!ktbm@(wE*qM+`*HwB?_gc8u=yB?l-VspP(dDqKzNk%nLg)O84b_E>?sz<~p9}?$1 zi(h|2o37C$UVu0RaO|$5}7`r|Y-=JCF>-AoFRo;ZraUodBESy9YeI zNT|kUviFbk;}<}SKv@t(0r)K@r*;E~B9Ff<4{79Xi>ZvuUk^FE4+0X66E$aPy6>Pb zRQ9IQgu(7mmGe(_vSZ)etk<>r>Qm5g$Gg~I<1f1H#7x4J#7vPj>?5R^Kn}1=Sdc_Y zmFJ1F`+Gj~$f?3}#y(lAML0cP=dgH4G;7I)HoBXiIiq5or}{moj`ZC(slPX@ZoWZ zvvFT%W0;12188G^i7MAeh#0I%o0l2?Y=HkQ#$%SLb|#{V)g&&0Vpa*W*L*U5{t6P*mI<2Qfr%Yo9 z?gD`kDekEs3e~#I)(U;7pru1WrHQL77Udu6vcl6Jj25Y=xlED@L*b4@gPkV-I=&4C zAPrg8%}!zQ+$!T&C3SQKXN`N_^?w|#I5L7$Y3(M~dAfI31Uwg>R}BrZLUJS$@1#IA zqK&;yo?n$9i=J4h->%dX>MgbS^sQ;PF*brXS0oNU(Xcwvtf2HI$*YlAJPMuXJ2{oE zGWaMpiiGcExXxT=+?LQUk4i7pb~JeXI_Ex?k(2F5$T%^&6z{_oSVq_qXT6#M(k&n) z*!2o~{CIj)S=Mubk^hXc+j@WhHN*D;RIX|XgZ z!;;nUFvG)@q?+3#-#Emd3CCK){#F+!kLn(Gm#cNyDZjs!m8etnGt>5FfUAJ$tyJRw z0>Cy4dw=i5NXV4F!r>O9NLPPS1@n_kzf<7feFPuX(&u*^p7c3>;p>x`T8!ZO{G&!! zYpQ}q-<5gn&0AE+hDYeHq!&lsEZA!HrR_2uYjGImHo3T39?fL+W2#$RFM!xW;MGNA zLM8<^ZB9-BEr8(OBBYD|{Jp6{Z|Vo~_BPtTV^+P^oENL5Z ztIvaiDKMs4yU<#>T!Qq#?zYlG?&}noCcAY5v(;?JR~%34U7VWQr7#Gb0gG zs3W+*HX>B}MHQAU7b)-eZ@PwhvNx*v`l(ak!)@qpeI`Lmxa#E9|9gClqHLYxO&_E> zi>4`P^<^)kI||gNEGFDEW(@+`zZYkB8vIt~@e=~@US7d%gOnY^jH&O87}*2w!TB`i zy+hQSVvh?#zZfG&Ljy>l@r2o$rYVD}Iw3NoaMptvreYhPQ%7)SsJi2m&f-wc9V%lP z5pX5l690(od2tMT|0EKc2lLYs3{{_XUie&qDSR#9a1~+&%^I@mRWyBXZRBmUOi6)n z>eXUeX6G}~)m~5D=l#qs4b%+472%HP>wYW?b)$GL+WOuJ-u+BOlf=;o7`>odj7XOx zDapiDVj`Ynkt~g3hkG`HZsvBvZ>PJE5X%p@D0pr6PpOJ&Ce3Ry`(+AjVLcqBBJAmw zv;z&MoHI+EJhGNoo6~-3^!CQDQfEI7ze!pAuo4h;2=hr2)1#opw^T%*C9-KQY*tM2 z5EGF{I5z7B(DQ3v$@~#4^TKyfeqjF3Jvad#1n42k3vW#z$4$Q18M*TLLMSO2LJO`= zJ)%EZOi%qWlLr&^!?zyWX~d3rxm-yk zOQ;~)0Fm;$)gTg&8y599LHC_;&@xCTzdQs&*wPIWqF{^m*5Rl)RCkvf5CV83X_hm$ zGRA{EOVWO>e~~1mMtOltJKt@t%sFWs?s7XD*q;57#+6iyU^tR9muPX4?KtbRwuDjA zsw~6JGkX}UB4mt3pwVmz4FlIf^X^rR)%^udT56I=V10%SsO&2ZN{U5*%A6GaLI;j}EsnaOL}jX5+y=2sxA=s^cQ~v!_xk44blKh6=THg_zb!8h-XFr< zkIJRV|B|i~ZEm$Zh0O#mh~&uQc&5K&rnNhm=N}*#E)yO=Uo^pYBm`q~ZOC;=T7lcMM z>IwKyXg6f!DD2lf12y#YY7b}YY-S#sh30Vqyo$XjqOK$&ZGaJ_I76}`x=6yZ6_bPR z0;kH2Le+zahLX3tDzwniJUO0XI+%s1y26w%Yt*34zibhmp)Sxr2mxZ{@<@8YSM@d5 zqH3M4F)l>HR~sd@L(Cmmsz1!+gVo1bt%@!|=FO}CUi4V1Lx}ksDb$S;9nQ>y;iIt# z%w(;%$CxjE?AD_lg6`k*36XFPx`WmD<~eEnIMwn zinshNMZ#@L^OB*Qq#HM*sfS8bBNOT#kP!Xi-;+#>hOXvu`N|%I!xzxD@2Fy*y*Z#a z7g=^DY+W5&)gju# z-q<_gdqTbiGLmrO5eoA}@~4pibZqHwxJ4a8BG}4Eo>Q|Boi(t3V&`GM0+r2JZ7Imi z=wM`Ie9Mj;nS2%ODY^*ljWOpY;JT**%Vr;B+?ZIBpiyyn_mlUQ`7>iqEfQ-?(XqQhZFZuZa#bO<}ktekB)#3HTz1j-~UtSko2_`r(6YH zp`bn?I=bqd-mAGhmXR;5NEzE5`Y4?i*xt5z0CzuLgO%F*oRV74>#WHLerjfO{kVBY z!+^P?{ZoQ3xt<1EycP9eD##0c+9^1v^)wfOliYQ5JX#KcFN{uB-#|d@lzO374i+;N zGc*jI9#~83<&aMtZ_f z01utaVpN^2OpRRab9X=SPZ&{Jxu@kNk4K=eR|h`M9An zoyD(6Ra5?7@*iR_(uHpspFKkA{0eHft9}|61Ue*{TWbHB+&mJ>&C-goAySm_8$YKsc2eb~ zKDJhCi5wB)|I5!h$+_7QH`@Ae_S02v_}A5BSmJ9!1)Z!K2l`*(22!HpC&RF0i!zKHrdM?B`&i|B|I$8&$;w zz(3`c`zA^*#!x|k4zz)DzH^VlEV=f#!_>aH^)dT3?y1B9gKw!dos2cRnItUKN+Wgr zrG&l^0rCSGqg6AYJ^GzmLik_?^|D}Ik6hzC9*zJ=Gs@2`T6TG#l^`VLTSZCXDoeJr8^W* z6SQ==Q-@LqsU&0T42C(VvOe|-c66yW{W*xNeH}Vz1Ec(FD#a>n>it|@wu-#G7rtIf z*fu}1BK24COt&E1J|g33E^K;)DmsYdPt$wQ)H&^qQvtOdrzF9oKG5x2VhKjOfRC_E zDLTK8(|FWmScDtdIkw|0TSL}b<)&qn0)JD}U9$Tzdx@zbo)HvEz)ALe-cS z^W&|BtV7(T(3>6(T!Lv$yl%OA2o`Be^r@nr`0C2!{!4g^HqDJFXx5S$;cQr@wE2=H zvdGVJ*xYWiKW4$9BMtVfAc-1K7pg3A7ZyBP(*%)X*%y*nYoMo0)@%(t6`s;azSCVq zuQNHJj}jme`^FyQc^k&$)^t@+>&exb*Cjc0fx(Yis!NQ7dN2BTQQ4=^C3TnwD6?zC zP?u}?529n+hDGD5}ZDnuTep~jW1+B-l8J0I?Oq5;_|kfW|1G|{0eJ_`jvfFjNV=!oZl z5$xa2VnyLbF3!dC&=3a66KN>}z2xyH%LbBP4^@R&8vkQ|7FH z21(~U{PwMuKnJX7?i~31Pb?sxCCbyf9vXAjPX)+jX;AUkZcQ>7cE~^Phz^cyyVxd) zUirG_n*C2ai+!i%JVM|a&b^8VbX%4haY0tttlzn3$&U67DgGo(4^O$1h3x{@b>Hn& zw~_g~;mQSE&5n{(2MRL8?0{#5&2bHmI(mNp|2;uNuAy~ zrB%3o?b@}>?;T_`l*U^0erYiOLQQ*BZYFN5Wy9e7HJa+d3zPk<6&cCW*R1JxMmsaGR;~2;j*Z@zK4(R@)WrKCyVsB)0)*{y=37Pr%BD%2y2a-JVS3Md_ zO0cZfOXI5h@jk@OE2_H`0ejb1 z2z@wUF?ox-&Hy7*_=m&sk%n!58NQhrAVIxJg=xeYy>>oWU*=16u9DoTNp^BYsx)OtEfoAVt=J3VtSr?JhG zew;^#N;Dq`sL(GZr1N^-dhQCN*^~lJJuA{9>0O~PX)%v2^y!Tf#q2d8 zbKe;3DL}3;=GS^l9B~!9cmH^U`eVCUjU<@p9e!o`YY-(=UY0u4x?K;~8YYTt8jLzH z|4}&77?k(VTBK~4cq>*#dGJV~X3#lli>0zG|4N=r*2+wBS;%KK2O;WNeZAXRR$i^W za`hMD9P$x@n{5G>$I&eBf1d3|*tR_%AWs;eMdc~r4}buk0K~wyOYiT^hD!F2jn!it zehTW%FENqi)1Z6?8NUzyBB(6cLBzT*=?c3SsxCU48r}9XBnf*SFIr5C&?pt_D(q=K z8Y*Wp>y(nW$LD)O@;4#^YGD8Lp?RWgkp5mWd-L5@zcxNyhZ%paov85pF4vELz;17-UbU6X^8clUunY@e2{#^dK7d=083^$je7|>E?+`B1zw$;=Lv1<%B!K6l*Ek2 zZ=u_R96(GP$x*JJBG0pCH0=y}HfApTR6QVPt0TgR{sE+~#0X>meC<>>wx@j-IpO%I zt)kF$-0r0T23x~Rg^6xFXK^^_mXfPZ;-9F>Yr5((m1ew~clqOFpJ@1 zJZ($miDW5i*a${dKs{knlvzXc+kLgk3{iD)Etd|hM9VVQTFl{{ZlaF}OfPxfZWNH# zN=ir|m9W+h!V59hvPr+%rwl-T_|xw*aiIh-ifi{^t%B>#x_CltSA{FZFSTV&{KijM%hgmZ0`W$ z1Dz)2RojdRfh)xAHGO2RcPbUm_VX>_q@28KEd8ziS$n_uR_Nb+YR;2+v<3&s6lkTs z7s;B?5J&uh(zJ9|B;iIW{MTt`^Ha9z3`#ia_?QR8hQN!$Se;T+?Tu{7qhcSCD~PXc zqt`i{A{E|DmohUj^b5bPt%EWH`yc1<;D;jrOoiV(ja}QvGvl&du|NN|0TOZ%<`Vs( zJK_sDf4}unVAd~`rBlk3H;?9irOQ^n$}{raK9Dh=zRPy6;oLY?@lEg5hKx5iwp`~O zJ$+^*6AddU8>6n>v!=aJYK$vm{C$q4Zy*7V=N_?( z_xB@P8oeL(`?mFm(jYI1M!SByW@qt^_powAW2 zFk*3jF03O2RnIEeVB1h7ba>Z)r8evHD(U^ItP`fa9A-4%_vq<~&lOoYVQVu_7H%BG zL06NKZVBWsHvE1}!I<{kdFfb|)FT|A%;ClDtMjY;G7orBRTsg0 zjMZQ4ozj)-E|8uJI`ku{_XB@a=$GsCoVU86wW6(-Ec_P2t6QS=dephR- zrqOvbfyJhr9lsv}phq8znhX1}t3{`qnQ-1`8{rRw79Oc=6eT3yZ02etKT znAHCMtADLxWxDujWA6nL%m{kbaV|K1^`rdPBTS)yCVKEXAJL1BEj&CAdUtA~GR`Jh zNV98Lu?ST_jjLdoZ8(2=&NKvl5DP;j^@<+7`Y&B#YXsBbMPPRsg-fzsBD5dNLE%L4 zH3;VXM&`AWM0;O(;$|&kwhhvOSNa6Oy%CW2lj#G0y<2Unw^)S;mf`&xM+Hx*g_ zJ@Wm9EdjZ=^XeHhQkbGBwX(kF9n*oo{qRl#Dmp!I7=3(0@-%chOsRuhRBi%e>WPkn^B>_ zk2>kS{@WUHFWPm9Q-EKKaggqr1?Bc568{I<=ZaSM+|_$ukvL5A9}01?aG$flYwaYu zRWEXF4!9*9M?1gcr>+Pc-51jKcS-GPFEj(gL`?g+5#`M6!bZEDr;~4W#g2%x8&dr9 z8ip|GAbRLZ3Ou7R_y-+If^MbF^cJcc*&ksQr~BNeI%GGqv?NBqvrW2~&OSsYh6LWn z1lw5Vf5H?bcC6Y!!nWDcN7acW= zO6dDSI%WPgI&#bON|;$PnO@tWoPPD;O3i_hWk)a8qEUO>mF2y8MaaJx%C*RBP5)J5 zgSxJ1pAX4`oK<8fs*OAK8Xr9jna%v z4@CdzTzA&HzbRj6%N9g8KIEOj{xU|2Q%$^)6TMcxI+0%~^WBg+n4dVGySaGGf=9E$CKkrri{ z_+iY5WkUXb|2xb=b-)8W3%N_-;H2@GD5BW*&TCKU1gbQwvw#tkGgq0_bB@6?s4Ir+ zhJ$9#TX8r4X#qHm{jx~(Kai2o?>4F?Qyi;rkz{;`Mu=FmelqDv+zU<&_##QG$RHh! z@oCAl){vES#O|GCC>!6;(xCn7H|2&`LCfUkA%%T-XFQf5ea4do@bi*5zWr&=s%iGWGbO3p42=eJHeD!$JtJ~I27DK}>|TMAkzz6bv8b8c%W8P{gi>^zS< zkvsM4LvQoaAs9Yo^&fp(aY1>qzR2J^inyutN0Wo9?>2|DW=~((R^%tn9XC1=4o#?( zf_=SRX8_#5i1(oIx0lN5_7AxcI7SEx=T>4O@}t4V`mvR8+8r_}(ZQ8KgD|Bk;qFG; zRkpX!GKr+oJhOWV%Lb=_ zi6^2zbi)%b8swU|zVO#SouXa-LO!Gvp+rCN2~kKRVb5RvQ?D|}(Bo6}x0h2uIcVHfAnvbI3=(IQL>$>|jCsFCY2H~&JCz2<@hmSj^FM@;rVWWwAt1QR zw`eHzg{Bo^$6TYS6zXK84Xke)G^(*do8vhA4O~ryd7W2h4Y2BoJ;>TG5~?gzmeATr z>mhKoKQr#PPM%D!Q8`dDt^Yj@_D<*0{kbZyY$O7_)hHh(knen@_GHAg^mm~HiZ{W< z$uig0zs;3COGAeTY6+w6KM2OnhL$bOn{pI}G+*f7BYCPNC9?nfu>9R6B1 z=|i!BzJ&~2#t92h=?)tT05)OjbTwESuC|cR)tBxVGNoXN_>&?lWX;;|{56g^^1=zg zLk)YQHG{~HIK6YFGT{EC)=Zw?y4p#Of-o>=C|{ypZDRSlJ;(b~;PzSVw5AQ(%6teJ z{8lWHR9({a%4nC~K*(<6qjUWd!e^g*CcID0-uutgXPUNqj6WE^8Uq?9p#Ho3zjZ#r0X0#wsjS!ddT3-`5i?yTB0>R~M1D?t0HoM$+f*nXbG*frLb)0HkdKrlhgsQAqVwmVZNYGmleP;dy!Ay=1C=^n-GrguJHA+dip}rmL=XOrwl&xB- znZM%X!MhyzIoU=Jwb`_r6D-XM>CSwmVvR$DK)``od8^E2tt9u*imhk<35lApX;wFM zkNP;8f$L)xS%8Amo#qtmffzTP-_zRL8B@-!39i6}LpU}j4>Zxk6-*Y=JRTjZ-ILAf z;gNLL?SEkTz3i4!O8V&+*K8J!7gsqy^i8(>6+&f+9Mq9LNS!uq?}t|iNuQ}r5rxpA zDy}<5D<<*Dr0QqNt3XalIOR6khZ_|$+U;&#)8{{bFRi#dk;||)9QL2snLgbZ<$4}} ze#d5J@wxx|K^ljJ5l+$SbiiKKp?)K<)6(5*6~XbX#7urO`n&dPMfv527L<2MJpWhp z{kgY?w8o#%PR14^Z`i5w(gz%x+0^TKNCCe{?q00C$Hn@oyfStxri*hZ^P;m8kMkFF zkCz|hrNB>uEi)(F<6yhVd2zm!P=Vsce|oyJ(A)su?K%WtxA!FeK`LbABXH=Dk;HVR z=rxv%N(}2xi`Wnl?Djt5NLa3YDEw{+bQF$^+-g1OcFO~Nm%)^XC+BlFtfaEgh<8=@ z1>W|r_*JALGgYH+nEw8dny{jFXq;Bs-VbzR9aWlbgdCvtf9=eIRF?HWiQLFx5af5X z%=F8_P}95rdXOR~B^jKBpkbS)cUzU7Z4Z4btLnGdz6{@yBp<_<8*GBV99Pbj)Ci)Z zt~f54BsA;#gLKAqmX+6_6mzlCEA`8U-7bM9?d0#rQhk5Qd;=8$T~%b(*9Ovu(m!v^ zQZ(^;aWDbRX4j==(_(#Sb0=-T)h@%6etLZg3FU^M1?Z%_%HK7Lme=l~e7z7Hp6bd< z{biB2ZBwX)Jp6JaXsuR;6A0?|?1AwA;_QA+{Ykb@8--M>M>yy{7|%~T%q5nR63z+f zbJ=f%YC<(SG&MMj@19*0wHd^uIv~+LJRg(YRY0b&rs(6Tbkw*7&c+`2J$32VXbE}g zAMYpa#Xle?eii;8@FASvI#12fbZXle`6;z%lFZl_9EF@u@`vbWb;|91KQyVW>k~Ae zL?x+_9(Hy%yuW#s!6(RCmpBTF-6q5;`m#%Tbbl!oUG z>2g`9X8mtN?VKpJ2)j4eQsq$n+iJ)OWJv{g!;!gI>p`)!{|(|d-%MmE0W%QC7OG3n zwvfnnL)>|Ye!V?bmq|R@j?y5Sc&+F$829;mNlEYE_X|?%iG&2s?}5VF<~ciW%>Ecl zy;bC|DyCI34GjIuL}CO1M1SNB%g1poJBrzIe=OPPgQ^hkT4<$GH=CH=d@iF&FT@^Z z3Nz+w3lM(T9L!r0^y6=!O8iYTZKT(N_5Utp$_FM+eGY;MmD@7AoRg|gl5;}-si*bf-^P?U&NPUv zv>8>J&h&jTF}+H&vV`yaybBHPanNMmI$*4*~YR8rP2;YvnR! zf4=(Akq`?vO2TzvtTXFk`zkx;LJZ&5PFE@ZFSG&0r@!wGP4neM)PMbu$M1xQ9Qjs; z*u>2?$h4^K$hy1y?Y)tWacQc1X8jO4P|nUmpz^XXiHJ#Wv?7F7KO73rVy1X%&I5`( z_szgsRXvp%@;I2|@sbp5<>p<*_}sPPXX#yx-49Xl8+7K2~_4xf`d80~F9m9j4xiNX#3frB-HY%cxw9XW3j5$g`3IC#Lqd=MpuFhbV z!aZnmpmPz(kRLkNH0`~ivHa8MO#)Ej? zy1Q~Zh8RJNJ8Q4&r3 zQoSCN&t5rnB(24_GLKHMS=uWhye_z9v{A5f0Tq(W;&0!l#n zPL;}bKx10J?dzG<6nWOuQ}3VQBH_0~({rAD2uoJ?!qyXq-s(%i5CzurS92S6X7UIHXJM|J?>e1YnxucNf6zpZMy?X+e$j^M;!P zo6ml7R0esE4>lX6pLVJQNxL$Xwbu0$Sz;ba^Az9jBkA|i<#l}17txmfm8cpz5&9=w zV-P%vMC{$^6DVKO4Xo3_+6r1G!q+l4tU^%bN9AD9eX_VYEG{^`Y^nohxB-mY8Xd6c zfqG-4#^TM^O;X*EmgjN z=5M_<@&DwG-u;*2Qg-6ciRQIE_8}C19Ld!CEb*-hZHI@HfxxTdM$$|TW0?7$!=t`o z1^oY-#XzTx_LZ6su-VS%4XV2XxqPA$Hn%SnC?qu zHsQ*D@U_wrDJOdV1rEaFYmA`D{iSC(N2K*R39v8ufBg!z zlPY(bA*g)2-+&Hk@n%U^Y{n{J9emdR+ErO0)-`&7d;{q4=+ns*6C>|+_y7Jc4pZk4 zPZ+j=oInvVf{5j%r_BQ0D5L^x{fujJEMj8RVw#@7*4F7fKK z%X=WD$JG{9lk`?iSdJuBu*e4~|8jco3k?U1*2~&v^<-W7k&qjVavU5Cw?*dFFATxa zA+F;I1L&%5ICd6mMMC)r@bggM!iR484I#Ej|DUos8ho;&i{eK+@D$FYL6ge=u;!QZ zD)ycZM2kG^R`k$yHSo26!$|DS)@m`h}n+Z=gXWlL%d0utoSx);JEM z@yoK;b4pgE!DQ6^BhDY94(C+xaz1oC)JxdDye3u^I$#}!;TSr+@|hT~p$Zo<0#tz> zs?6I|>{(Z`ULrBi<+Ed%E^Y4%2_!)ihuvGTQicI2)a4Uen+qgYc;G2?_r#pRTXTFr2N1ke5JQpSC)WVc5zmM47p3awt2Tmr zM>pApm~n0Lm!OqpOSeIKg@1Gcr7EK8xRNAZGi`wMJ3iufDt$Wijwa(vIpFT z^p(%49!U<_YbMK`5Ji9vCnCaCZsbu-ZqSK>j_Rlo)#5>dut|DtI9N4)cwvb@hjtLh zj+nx9h&Vy~a9Pf7i+VKh)KD_21FR^)*2FLDmd2OnNRHRzBn=IJ9aL{gIH5}V8L zARnN0F8@SP$gM|2T_OR|>-m_8=zzc0h03aX41gx>>ZJ z{%c8&ALJaH>p?##$y4A6exLLT?gjRVJgu!w1N1#3)%$)5|N5wu72eSP?~KN7UBv5G zaVX=OUgv-_8m=Q|fxK5}R{&q*6E^5tcG~lRM5WKuaVKyt{>oAz-pdfM4zlA60N+R5 zP*;*%!+B=rJ|G45XR-ikhgZexwjeS;o%Nm)UeRy)7C6AX%E!@J8{XlVqj}kpb(2zd zR$sWiJHA@D?3poGP5a5>z_)m1p~mh$f4>6L9i;^6&&hgXINWn2j<)$$^G!x0ZIYwH zt9y}v9rNSb2W-@##0DlEd!Q5kl#vuw|2v?h?}X{;efBd1sEw>Ej9wy=pBWgbe&O=6HDg( zhxdwbD9p=WUPon%IJsP(Y%Fv!E*JS3b|OYhe0Jj!qe%q~f;8ln+{>pa!=yY-aRWvc zmpIuDD(#iJXS0PavwZ!a2Fz$Dv^WB0?JU`R(M*7vw{GLWVcE~avN9gr-Xet4X$*D= zfd=e4{-9~S(P7-{`*Qwi8{3DaI+sa$6j=|X+#UDXfD#-(qOO<*aha_%DnzRVs64IG zcY3yqTk?V9-HOyCVhDI12~O(Jhhd100ZqFL2UB!|>5M^~_+RMg)ouOs+_nU;{HrOE zOk&{GBdkh&i@d!TlgIq`__`3l5{Jv`yl zRw=G45ahCo^%Mwy*JP)IgUz6FqSXbQ>3Cm%> zr@Vh%(#OS9!M=cc!oVf{a@UQNSH4?Kxh~TMn$@<8mebxvd;?x5kHG79Pl6%jE|1Hs zW6)f2T(;CW-MHkGm)73pR!d4~{EyMzI(COJ!4_esBh=mslHrV(xJDTm9>fYps$~RC zfZAo&xJW-Cxz_`NI`(@#EcM_G6o3k|ggi-W90;W*GT9KoDu>zruT~S!a%U{Bjps4z zNXxi81#>w=pS2ci2YE0qvVc18*8u75L2HwzBx5tRKjl@N0|^Tl5?LS^akPrubb(X>--?70$Z&Q@o9UnzusvB z;=XKf8RnjHfjpS7lT}|9r$OH?UPdk_k=N8zng`^!`I}PxJ+hAH0S^SI^+#t9GB@SKtL;>&q3-k8-5db zi6~1n15o(kUaiKOnOl~|+fQspTMb)>IBwueY8*b?6T+1cKSy#;+-0^KV&zqd=WTkT$>@WU+(~RMV~?w zvzrNsIOj}9uCqOZh~ma)v9sg3!)7_WexoIf zk(8?xSWqB^Psyl1Hcz`R}#CA3q6zj066@n!rsC)r!Av@YmnJG_K5aM zqm)Mgyx;C%VVpxFaYI*hvQ|l3!tSsI5*IUKuSO-i;(F=*hI2Q-heBdENFt3!YfjpL zWpoJ}1%6PX*=^jW=cYU?n__0uYo9u|eFI$0`+Z;M^6E+Ka>3cR z{*>!G9l)yKxv8WS5$askJ^G3YvlQh4ecV7W6u{C@#&bE}h;Z}mN9#l&51_c)7{x2w z7-<~J)cfey3_-d++F*pn7r_M2K^`YGT+!fU0DCZayo3d^Z(u@h8+!hq$mss}Z~->W zgIh~pq;&_;IKWX~l=;nGH0BLtd?s+x${6*=jGg4jq^H~io zxU?g|vRoJAP1toxSh~3CyQ<}uP&Sy0*=UNP0V&WDPQRZu&f>`vb^MO(;4R=zxUK7u z9+~p^y}R}oQG6S|J$}TJ2 zp!Ajg7gD@u_f_TuH1VZrehXG2Q-%bJZD3h{j%jYjm5#{Ks zueAxUswPKF4I-hB#KGXFNp`efgcmx1yjdT+m$-uIl*_DHqlc0=4gYS&lp?DAUDFy( z@=C8KE%19U2$c5V3%@D<_LwFD>`5=5$S)vNyaqiV|E1~q9+tge}0( zi*Rv8kMJY1dEwA-%DM_ua_3h3f1G`JIFxS~?%4Nz%}xl7 zh{0I18*3TsOwri4QQ1OvB71gOv!=n=DI#Pi`%ZR3lp;zC=hg2z=Q`JQ{yFD-e@yDi zH{bhy@B6&ZeLu_XHg8Uy8&x$R;?M4xwS3ErS;tNEDtZqUQ5gJ|aseoPsy=^)P2X zpB6}6?n#$?q_P)s$OKdO()ziKbwRvS3+|y>;S0BZNhJH%zQ}W1})TFwwi6vx{t5# zbqCUzTwC$RS4m&90c_J%i`ta+*vW6PFz9aJHR5J_o;kn;a#FxR;2-aAHO%rw{BmBPrm3m|5T(REt zN)KNW$UY^okJMD8kIKgzSf?$9kHDej`P2l>qc!6=4_#?{XE5#OIYEnsZPUTkOzrp0W)}^2lhHnwuf)a-K9x$vmUq4KLvWnYNx8S~ms|w-mxk96C4Ytm; zoolY~qOsxkN(32W1tem}{gjGEja9&yc)^&+e;JYD8@&?011T#Z&F$N_fUW`|j}HVg z_T(VT`!e^_zT)wptC-wv5-lOnJx9!HsXFR$ju-^99g2A^;;mO;*7sBayruXsv6mcn z<%43SFX!ujETeAdiWHg|h40Qp|0&%u9D)*uY~hpu0O5mPigDU8*}ZgozU^aX_78KZNHN8$r}P2~ z9H?qdLaF^JCEo+IM%q^@W1%)>a>*MtmU9VM)v=n~GzB^tcJ7kBk5~5|rEQmbC5pxW z*b6B5D-BsK8XwgtaCD!#7si-JM@xJ7!MM?^90!{IK{|pVbX_E+0ShjNfYdd!9y(Z!ClhW%FF_1TCi8C*IZMfcfAI{jMsPDiyFbGlEV< zJZlUPt`2YPWq&iKK?Sx6xe_&7vZgpt8+Q$J9O+*(;)!~3c}U9opR=@5V~SWUFnnu< zc4KVp_=W&j?w{&m_`9ddpW0M!0*w=Vp7Phq#_!}nt!MA$p0p44mR_TKoDLrZa{zQ0 z*}8PItTsbstRNHnD!MTv-}!@{vtQ`mt~Oj~7XD=D&s(U2x^aMv;mwBtt zB5he}8n5T$KU%^X^#P0=9#ZloY>o5sl3+sZ$b6B=%O9Y;{2KJ1GA12ZmC=F}fw=>! zVa=PCY(@_kVg76}CGlA}5^@kJT$bidv48y_*DHeve%d4tJbJHrfv9C#a90*53HDpO zE>|q!yAD7!pjCwO5f5^N$3dyWZYBKVX??`4piciGb|~40e-B58fLi2 zc*(%{iBBJ-Ukw9i?LMpSU%8RPBGq9fU;q4nPB!TOAfDt1?8L~{R3A;S1*GFGA<%o} zqtgH-5yooCh~8&P)mqeyr>p{7N8`Q-LDegOP+>a}8@s(`Qv*!RAqK|5%N3tDq$?eN zu}%NI8%`i|ZXlQE7*|Bj-+-S$D_TtZ7=?#jPOikeC?t5j@DKh^g=;U2DGep>O@d7D z+O!b?$G?6i@GgnU3lw;siUxGb{{tIO!!Qu9F>iGKwkc(dK-yuext@$6V8i&Q8eh%S zJx1I)y;zia8q1w>mz~2Wd0N{RCEw1^xFB}$$#1={R76k%ZbaA)78g=LF zmuVB|S)aV^Yme1^!ufk|&H7j=;%Sumny(Q3vodR#l1&*Ppz%Tk{JM5=)|0ivt*dtg zM6cY{$nWX`#`2#^YucJls=FYMZTxlZSWQYGE|s@!%BVT?_Wr(mDE8v*A;rCiXJITx z!$-E@k@|{3Jw5QX1|9_H+wEkKUx2Bnx+D>;R3evFewroh~DenNdA$}9R5QExN+2- zeeAn)GKTf;>r4Z@LkHGobD(BNMEx0w&4==-!HQ zh8;b_6(50C?Er{qT`w{k6%?Ce>kR)-jP%F>Fu3cpcJa*F%~^#8WhTESU!T&FTkviR zM4&Q!oWi|ESbCxZL3S%1*!s!cTE+Z4>T(l~3++sC!JPYY132<}Bdj#=MLZrnhyh-* zu%&d(d(4*PJ}UqZr?q^$z#xA*4giTZH_nf4+yE6E2T9)eV^Gd||IKHRMo?wxcHKsr z`64^vdT)~0juLhKj24idg}CEORjutDkImZXVm2c#9C~*_xzcOCLv?yUxMLMjG9IlU z8GqEe`o~f4J4lvLYKA}p3^-CWtATg^Hc)=3?&jaAn+`a439xk#e@07Ja=w`i}9**6KyP)l+8$#U3 zleU|4MUp3PU+hz5M>PcFrD&9epw6=8=*-8|R~B#A)<+bfLAUQ-2*c!WgZ833j(CaK z55G|vUgVv>TgqMM+sFmL7i{&5G6E(Cq;Psa09BK8pb>sIcb@K*u~rIkK{`>TdJtk7 zUN3QKuDO&WaaxSA{57gh$?29IrIFG)+xx7gDz2!!JXb7kY|>eM(TZi5ZtVa)7r{lK@WioXjwn)OTJGVmuKTu1S- zJJUXMij?C@nL58xL-VyOy)tB4mpuE?#=;gq^MzvAUdxNY*05hKJ5!X$;5pv`lUc) zmS@)G0*rQiPs5Qw47%;dvYR{n`mff#Be3yZUw1@#$-e@T3f`wWTGu~Gp{rlfpRJ7=RF&W8PjmRBHF$Wy~shefd6T3}#2g`B7p z5cb@WE(}hIs4c|^_;%og8nQZwRDYo`M#y7u@vg#?1U)6Nmg4lw0ohNYQf(FsT+2}z zGKLyMf=~@|zy~nq$)j~gx{|9TdtEWR=fEZB-Q&2E5Mdq`W2q7dz z#>X#RgfV;UJtmHXFDEm8YLV@wKA!EAf_FXUY!L;HTWPR6&;-o`9cwxUW&CkO-d3}I zV}y{XQYPx`Lso6jFCPZPXg0p7S!}z5pZYewIh%qOdgIO|AKg4xg-Ic=pFwGLIV9!H zGA29m{sERJH6`2k@_*BZgJwK^>g4|U0S8a@6COC2!2njt<<%Wlev}TFl&0H&A1DsD zP13$Rg6S81V$<&q_AlP%0!cnP{f8vUX7y$#hvC~VCkeUVDm`7EotZHWFA<5QCp`VJ zOT2;23O)uNKb)rSV?MOA51);19O2$pj-wqZVLChCU-G@fw9QNQ^?u45sOl|EEiL9B z)~|n5F1E~$;(ca}ou*sBA7&LZ?n{fVZcD}JSFe9$6cX0)H|rn%-r(P+AQmMg_|tkf zXnttxOY?Sr%YpJnT<0(Z!h1?DS<-b?7A92^Qc1+9@7Y~-HcwYSk7XoI5Qz=vyI5Aj z3EHr%_m$t&1~*>Z60yulX&WViNU@-*2IyQC;E9>r8YZ=oM*Q2^fwXZT$8Z)y{mJTp0(qsm`=^#-3c==eTP8b>e z_eLCJrE!>cHkWR-FD5T-$~Aw(kJYXqh!dh39!j7>^;>3E`i-E6(kO~d9FesEbM?W3Ru9W+)w8!v zWmn|fN0Pim!}4-pP%rV|0cI(~@}^)@=z$PW873tC2}Md>PTbgJ;;ju;V@6$$=s#4| zZ*Rv+gBK@TTr3GGkrW*-L0-Co#?0r*DCA|xNPr`4=lGG1P9NMvtWcj363TUPn~A4x zxyMB!9MsoIBN7{>o(;1K#HkmiEihHa6d4{ijCrz?(m-?>&KJ8Bt}ymF20D%tQnA1| zL>0lKV%~3Ms+~XNoCq@N?YgbIW%%3-ceE72Q?vaAn3{VnzQ@sO%Q~xz=ULgbZ zc&%3|#WBs*S8HUTDJW#0RHv!$|6xu;lM0dq_qUdvnZlWhozfBIfr!1#b1+FRtgEFuKvvMY5KEe5P|uj}`dh zg6b+Ksu=|dc9Ugwv zFb}TL7P@D{j;XqUCg1ii9`EpPV;uKAiHN8Zv(mThsf0J`_bG;Sv0_~cV2?3*WQ=N*a zgz}W;#%XE0dLf2}iZo&bBk2me&XEwcJ*{-#RD_u_5}}Fgpl?qm;3w|Lgc|+e-~IGl zhOIxF)(<@Lcuh=RvKN`+$kH!w%9P`xW*te#vJUiZKGBO=9vNrRpLe3_G z(z0{E0X~spzgP>F+L3TNd|QIq%q2QQteBZ>sjh)Rd?#k}^w~3ZmYlsL_t6&?wzpFn zvj|ZfkhffiF(##9WWa>_$WK`Q=BeGMgyz*7l}+z{hoU$~kMv6Qe(5C-#shZ@(sZ#; z1+CWg>F^qi@(J2@wU*43w&UTz!LBH#iF)=@WMsi#RZMe0fY_LMp2LLz@|Jj_-|^wq zFdDO4xHM=U_@6dm1Vby8lePJX+GHaAvEHLrCCQ{mLh}c?U;vC@0H4N9*W#%#e^udL68Q6ai>z^%~4mw&+LE&Z(cULH&;M% zBB8(b-UB-{H$HZ_HgcHq*%!R@OQeLP2(1YO>8=F|XIf{V3%Y^XBlS0)k*M9m!9~#+ z)N>LwdaYE`=y|NMh`}=$_bQdWyZtSksoulMunoTi?O{j5={;_c$A_Md^`p8Z@6A5Qq0a*vjG^waC%AA>{#g~u^`w+IkFU4%Zv{% z7LP;C;UAWb$_V!Tmp;0LbHJ|c;qH;n358?Sx8b6N|0-h$NE^? z67kHPK$6sIQB!kCL=TTg9W~bd^JSFk!bV(Ws>hwDTxoJH>4K_4lcu0&aDsY%0!RrT zF*Szl@F6K~G1QwEED{v--%)o<%#2jS09%RKG!;wSCQ)0}92X?lR-*^roO+?4RGr5J zWn_3H&1*eKzsTIHu)uZGnS*|X*ID(;dbjFF5XeLlym`f>OCvhZydss5bn`Ya=N{zj z9rrd}mMSTh{A6jvdg!pMX*I=+;Fn$rA2;Us%9oIu@k+A$28f^wjy?(H=eEYI z-Bw|WJC7UrLYpG9M~F&Qt`DYQVBkeIaZqb#{LYHDywmBCtJl*b!IVG^*){+36s&Ah z5tVowj0W@fM{SkvY88{a7sWGNT$bUoryp>KAO?s~qwv~3ik zO}jRBV#b_d$hRZ{bsuA8YjGCyaF@6q15K>7i=C)e#2?FDaaq#-@0)k`Cp5Jus9msg zDdwDup#_sJG936;JI1(6bI~LxH)L)3gV(^;SXZHB-&|P@=liPkytk1-3+yxurF&9M z97XO2T$_oF6*>p$=`0$dzGS zH4v7GV@=a59e*hW9CzKYO1l^H@5RDlPgD4^4T|{4nX8W?rZOVI3OE*I7w;z%duhrw z>NWFiP)Wb6!_o zD@?_}Bp+>$9&Bz3KHngGYmaaTA9FD4N4)sy6ZJwJOBbj-|H`>tQft8lYq1PRJ z75Y>XqPh^hC=^rK)(gIpg;HznT_lQ(PA~6_lX5;STOkjuCpF;2=ri`~N>`;o)#GWjfQ9vd_|6E;Kc-L{HR6SkS;8sbN zlCeGiz$v4^@WSe8>SDEtPru{#CutxMm&;*awD;pI%O*Xzws&^=6~anfg*G#T_CK1G z^UzAs&*%?hW$O`OA$sQO;y(geuqiW9;V27GVV=#Vl`29xJ`R>_tuKSyZF;SAX6jVv zcdZb?P#FCcbWL!Q8HeG1B~BqPKf$Nz1#)%z(;WW@@X5fJVLC)TSb${z(<9dHnup9Goxxgphe+~`;Qpb7|yWSM6;k+Qv3zn)a z%|{@ZgTG32np-}ToCZ!CW@7N4*b~rd;KmqZuvZv~)%39cfQ1%Ik1ch^{7pkjNRs|! z><^&4zF&_suefoR30%#YaRjjbjS4fd9UfvXs`bov7+S#IcngHFl0N)qFO?;5`7-e4 zvdy&c=8ACmOH(mTswC$9FP8(l5UHiHfN*zB5PoU{U>R8IwDgmazopZ~qY}yr^ucJp3 zSj5Dt;~#J|l#ua~P$L7th-m5P*tD*TSJ=Wh#98A~Q<+d@4|{Hs2nrZA6qo{Im8w^o z)_cXW&)kzV0$ssEO{(GTO+DJz-Tkt}NT*0dPpH37!ww5mWz=Qb>*+*OTb{@yL`9O- zin3WEodAStM9YS%U644gk?NvaROBG2Jn@vcMX>f`BrK7YU5&m(W$R~NXt%CaO3Al0 zvGhTEDcx^Gd*b5c9QYZLSX+bmu*louj|Ih&mfP}=r({O=}LtFR+>gS=c!im!5rnxYt3fLg;IAKQ! zqf?W~-n2@NBJg=#eH7Y?i-Ar6z+VwYWlqJ4pBOQP3!l(p+Du?sLt@FRHTlXrd5w6hd{6J7Q#eqErIOo z^X{z1cMI!fn{cCIW4A^Hw>pwCC<-7l<+!gXb!|5jqS2$)z!s%IMglOek8iW@&xI+D z%$Y!LuN(AmZi@m&CYEcc18fYGBp-RW2xU0mj(06pa z51d_6@(UC-39Wd4p8K8Hv)p2P9UX6d^_B0wFC1fL+BTgBqj!zt(>xeFLmdY{| zzHa0x=OS1>_pyne#S4~ z0+*&V3eLzzG+0n`B2K`bk2_ns%9b8>^*j2%$IOC+^U!X=F(C>*4HHYR^PXm{LB~j^ z2zz@}DGX3q0D_UoHmRXA1jtUFwq93LnII(~vR9!vQLJm?e}K%FxthOWisb^f?36AVS4>Y@Fxno9CmZ4X-hnO3slgwy+ z%+t9KI<>|``jfq1TeCOdZaTt3Eg-4fo*YmlOi0UAv6?tW923KvF$;jeE?1koX&TdO zF~zY`1C%UMNmc-jG$zahxaqnmPN z73|g~nK?D~<&axaTO!1_+t^J%ptGR1s5uDS!^F(NOASI7@o_ z*>yY6mNhdzh#)20Dm)1rO0ZYhoevT~=!x;>mt$)IJ}q*V?C$OcRxl!&gjSnqloLOx z(koJl<9rWT$x7P|UdxyCPJSY)A3GL?H>FGKLQppa8?(0@9r2{KRIVpCvHIZv+14df zi?MKS7uX*0>DKf2h;oU8q(z{KK^WESwf#2ex2%63c&g!6|0CJ^%ek#f1S+~5we(Kf zrZayRF02E`X(h0I;5lqBuA&Ps+9$rVgOgf2G0qs5rdi8Rum?a%5pz<_38gvwk&xPp zWXuuZA?-CX{PUe^334u*RXALah}yc1q4NPeSj8ILdO3lC>s87 z-%{$teXhplFk2Xsa;;OJ9uQ$3xk5a~%x2{x^PxuRdOfFEILNX zD1x_|NJ+kgj}0SVU#`Q`2fDwtq6goq@0u*cESi)$|VB zP3$L_i29yk(w*}Yp^~Ax%-o?sIZ!4IyejCEmc8ZzC)0ET2DsmHSIQ#2Z-NRJOs25!%_ndP(jefP<@eNpFY4!x|-IMaEXwXZeRNitcwU$v^N)>Hm?2&6T z48^Hqbp#rwY^i5AzT3Pmj+0fk{|dbBSRz?SL7^8TS>t`5BLO%e-N4VjZ;C4_4X8kR z^T*Z>vq4Q&8z3$Q{9V`fSSTaYbPcl)f!79D)xAm_N4}H%+>}zaqsa@#p#D|XvT60g zpF22W9i_{F_v?QR*u}fK#MO2s*1Db^QKI!EsRJd`z1`iKIy!DspV%Y#0eabnFJq!@ zW_9Z*%!b4HMAF4h!=S8GisrevRK~}z2^nqn7?nnzo%tyzr&X@x`K(GwXi)*q8H^u0 zj23r$Fjn_Ky<)oV2cDS%!zB#?&BqCdI?4tn<@JBKqRxekfnxy6g4d(OV~Ym5ClEQ% zj5&4CNZk5!)$>$HG3Z06>WrvKX|M;9OrTWC zsMjGK#Cmocphwwg{F=Y)y8J^P!I`dTEjEut5jq6OG}2FsCD6!arh!~C%}q3@tKx>{ zOh1{T!3Ih%>HdgW39HXm3md4ioI3UEGX zCoXA(zdG9bOfl%UnKfI|-=ScDT5hFRgl3UYW77aD3UCuP%P?V(RI&acsecE3S$!(^ zIUKC24CHl8hlo=@tVCbvq}u!G3|6*cy0sz7DytUtdD^wuyaEv30Qot%)VzpHQWtsDjeRZmu%7Cu-s;u|yfIsB8_W(GDR5a+kJ25;Dhp#RzkC{-wi=Np$p_PmW0M zhl8doO}YogwhZc*xkDccsm=S zIFWem%YB#RjeswNGZjtOE(J^`In)fMzZyzFeYf)G>`q-g8CPErv)Ie&+P0WYmz?>J zNX?Ez@K+AmUs)UQy>LCf@+6Ek2{G^#?p6X%Ve*Axj-v0E#aH)JYwss=$@eC5=frap z>A$wbcBwxr31j<|+70y)2PalwIwN6of+am>! z5e*&#bd6uEbI^~qAmLkz_44n&_Xb&Xlsou57F?jAbYdd-jERY1ATDmKNVzWDTY&W7 za*B?L$!kYi1FY3#X^Isjx;Fk+HO5=l)}3m`NV63^TyHvzT#&(=4^33n8desD17~p0 zTcORGcUz<>5oF4RUIz#-v!5mXk0_3aV)O{GUMJOP_l%5EWdP8Qt7_29BmhNAWf7~v zvNR)oW?m4V8wRT7FNQJ3K9qd<>!>G0zyCM$Kr+@;^$*EIox8zh&xfanw5M-BVGZ0# zDKU4m+zU@DL1iCA1JCq<^j?$Tv6SA>4M0`eI|ICi%!54_>cbfbC;FHnMTmcm&Bq*j z#I|K@_~$1iWjNjhe3fvj$Z=3CFU0cAjoK%?Nag+jgW5}0rD?R-EG<5A79{JNvCJRB zDJ{Pcsk8%%=K$%7TK0YPp@o<4XTd+n9i|c?Izou^S*h5WC^$A}mxvCtVwyJC;z{*`b5u+%yobhc$)M z9OUD}X=ZKmuL?QSMv&C+fcjzHSn31j|qPy_X zdLBJj*;vR{iqXw2bkr#A-Do zZ(L<_>?BS~wTWDW7FG2ahCezpYGn`^!9utm)V$Of{H$7Mq&Ly_0N2>L9D=8YcAjY= zIm0UrSG$Nc5ARTlnPGY8WZjdIxU@&~x^G0%^;o^?%n4a#!S75Ug(oX4q!f3K%SFZlDwM_>Q_)n3Ssw_qb@bdd;WAzwnQ@Ch^LB8GboQ>RT{#Mf;!J(%x6v0*A^#7wdm`_oRtBv5KhGNkX@49qpRQjJe`JG-$*bs^MgFI`VGjISr| zl1hU*!k>9lIlB_ix&vBMK+cCR80h7@-PIiSv=i}ihfwUH>|;0;G8ZEo!tU6?YN^y= zOXT(@jIT7!(FSmt%*4r7k0}9m64NpkbVFcdbJF$d1F0j_GsYIILy8P8pJ}zi476k! z#5f)Kwk*tVKjHm4p^1Z@a`48bY6~NDj769=S?7yigDIo3=k}MVUE~Ffl~e)XHFUWj zOy&n78A1&-f|poB%qNN2N(uxXSXVg-KBa_~pb@B#cr3CO*R(6cklb3)lMB5acZ z*arcDjAb>PsQ?;*Ivn!F`K!gU&fNe99^TxGD`ze`Q$pXQ05tLd8@3xzQl^DDAstxqvL z+w}s^zu(1~vQtwwy(gX0t3R$0HQ!3D);p3Is8E}c%vTvXr4LaS$_AXK2}LjBIdnJ! z9&#J^vP6mHd9V^23ewMMi*WRR8hcvBwFHZ5$IX#Ky6ZKZl>OU!Xl zC5bI0W-ZKne(;NX1w7FL#K07qDAcpl(<844#vu``Mg0-PagorvRvOBlCp4RH^f|lz zV^wHL89%_!uGuSkCf$A8M@O`-RFUMMg?M0hn-grOQ)hxY;o*X(8nmU*(QK?T6*~e1 z2!}!*E=fH9nife0X8J6t&2okxgk_OJ)#Xb21OTQGQBXeuXK7_U*3a)}1Q{JT59gKQ zaXL6tkhl18^!dubK+FdGgJgt6?w`LB?Z?-im#tuBz3yNSm~r|oFu5E0`yEHQB8Qcu zDSYG{3EAQ`VUrhO&hV#9Ed521ul?6|HeSDDq#Ho1&-tjt>8@i9c>4sK%#lT(sSowN zT;xY019E9L7rUDk&pqOXuTZ9$16t1Iod7H6SYO!Vsn*M!U)~Q85{+cXqg1-(Pxu%0 zOV^p}u2X#XfP>AEM7sV^*Y+GxWFQSKQr-CV0~^_P#nKT$u?afyK9B#&!S^)}kbMt zQ_lBesTMUjZ29qWSJKiJT2T9!>$KS>NGE`p(;DMmx6oe_`^+jdZF$T47A}?e8*h0Dpqx}3yb3*$5PrM`PG!?R`n)Il z0A%JMI4Qj4Y;OSHD{%vUks7mPYCaz`L&h$I$Z5lW|T=YPl1(CuV1PP1s$RQ-4DzgKD2_Xy9sc&Fpqj#>F!sHUr2O$MJ&WNZ#U zf3RP1+EQb3b<7b>jD*3(9+h=c)vK=|xZww(h-DJ18RdKPM7-2hc;3n6Y;2}|A{_R* z)1Gn5{{`Ni!BI37w)ajjk^q{WlucDauI4)V^XL3i4u#4g*3^OS1?}}$kLhtGL$;?c zqkg8<%|P5F3x590RyhgJN3_^f$G-~uqy1Ki^e6p|eolJ(H+p%|mWZ!53DBZdLFNPf z>6vz2e}KaI#D#nYP5e_VaLnw&Pbk9-^<=3K`;3Gb%ZYX6It5JLZ{5%C1usNzIHc%o5-lE|J`3|lQ zVPZgcfeXX*Xy%0*S6NEkDPaj;g0}ksSf~)y-cVNH@Pey`g5ck8LgqkL*1vzjf<&)> zmn=W#zaMhwfmZ;o|LyLPJ+`&J-2eWSFAwAk{rlH`&P+QwzZZYjW}vU7Jm^x5869W zY|xip$sC<%LSEc6iGN-k!#>%*F?MZO;PSf?nVY=Jdy{qM99-z)>6_1v;%<)!8hXmT z2UMrf|ib#Z<1 zUpW5xF;hNu*wJfYiVPHx)mlO`N1j_Xdat!!O3e^|7WBRP1^{#nR?U8APn8$X-CX3V z?fq~vnHUQL{+Zlo{P`YNv**w& z-1N(yGyI;!xmS_i=O~;NFWBncIJ+kYE6vj-4mnExN0Cucxk`uI`R+LiKX!8Bi=eDS zyLL~8($06tzAWX0D#}%7LQSqYlv6?O3I>o;iz}R5Jlf}KG^>cI!CMtU_|D?J;cfhU z*40;J4BW~!N1H2T!G9b~yiGUdK66lW$o-w;eY5=gWs-Q5?;JZZRtHiZuJk(-KOfS{yb7}@Nkrgx!g$<`Ic_sbzo=PZ3_}Lkvo;% z2lxA$w+7n+Cd(rfWyR74ht6|z505?(kA7Kbcap6}-+AS-*OoJDVAUXeH%y%(BjiVK zI=)@}IN+~(@wj=FS^HvWveM{~o6puC7ojpSi&Ei$XVsXs;LBG=&TUEpITJTN1x_FS zENr{54f`TuVuyo~UQ3VY`txQZqCa?UVwVc{mTs#Er^ZCTcRX7DoD+1n-P_yKb+R_6 zUf+DhLgHMd;ie=92uKz?;+3eF12+zQozJhm;*!67SzyZ5(B8iL)7{`NKiij4xmMBr z&XW5JE!TyGAMHg=TRl2?WSunW~nStp<^OzJTP663ho$XC?5J zObM>Mx*F#&05P`ODr$t+oo@0>m#;k7d3jLZ?D(VkQC2kVd2N~A@d8m)LKV9H%L>^K zretI(B?YaRpS9l-bSkh&RT`$>4T8@WA+DlMKj>y8^M~bVKX8K4a0f)2`FXzQ5oMOK*aX z2D$>2+b>Pr?$*}tAbnC z)Ah{Wy5emeuI4L#V_U3s#vKX`nSr&YtEKG!;7Z32v)^A<>^&ZeqRT(LVseys8qVl; zom034HXYsh+LbF#PHhn;(+Uas%B|18o5aP&{%q0bwRn}wHG0MJEhvy1H>OQA;_7qo z9g^|+wxEqy=oan%u~pxCN)>!)dD==zQPdF*ScDp?h|bm39cN&bPY<1ZhjcZYkNh4# ziKI8KCRoW3vC4=(3}B)5=k1o2&Kd5}Pe0wP&A$H8nvcS+x1z`?7$n<+In^ZPC`R|{?gxvHM0_%wLKkf%vbd4?5*$pJMw%ve24{Moi9orv5z)>EDQ{w4(NY7Sk1ga zhwFT3I?4RJm!VI0v?=j(Lm~&uy%RL{=CfPojUew?TdBAr1gk#dXk0Wr-`8hoAcG0| zeB-rI5NUO;+~?I1LxxjqOy*R>y^+vSfyv; z?#U=F5p&UD|iq;^qxOOY*?I=Lkg7r1E`DKWAsezST*b)6;QLFHtXc^u*qyqT`O*-#bekR2JAnlg#;V) zPJHy?mW1EddNc4*7>upyMu*PNHc${Ec8FBEH-qHs?QhOAReQ@T7WGeP6H&Rjc?T=4 z+L~ea|5oSU$opQz0dw;`n_7{(_8epn0}qIa?1$fj#lD$dlDPEupN%nlpM(yGR+L|V zu75a=t)KrcPv$w2ov<-Ycd$%PvUYgm%E8J>%ymsLPI~atf`9Z%NC_Amtfr;a>(>Of zTa32vEqys@)Grc=iXv_gK7&08_z-OFf5`GR=qybeGi&+k`T@D{3LPx>4RNvaZoo|mTn4XHa{c>-Mby5s$4XWWX%Jo{E{ z*8)H=FrITv2qyC2Gp|$3&ieeu;tX`dPsM`gDRS%Tb9lr3k#f9tapCxsi70?vfO6NJWwOnpDnoX0#FrJniR5A5( zw({?JvFXN%ggKsY)e5>xB`Ha{{>NR^G6}qvFbl-TxX>n$y;P&oQ+Z85>^ZjuR zvZiXfM@$7?51AE*S(3Z&NG2E{$VNe`T@H(X(0 z8nVFIxN)Wqz*=tp9tnEF2ge`ixA(zJ0pDiyc=0SpuM_ zn0lsQsA=RiT^GoYXa6AR{{~+F4{X?7>be_tp9t9)c=0_Dgc*)qlKFkr@Ya|7M_Xl1 z1ZTm`AG{Xdu;NIZGCkJzrLN^#^@4$B{v@5Py0g#_Jlv-EN<(8=5w4JN^(ff6P~&Neo!{`@6ke+?z9DZ{14{dJE*Ds?H6U+1!*dxBA_5rq$#~y zq99dJid0d0?;WDDX(}KhARwSfM|vj&L`0ep0qIR@fCNHKfIxB{e9!&ed*?lK=lpT! z&RsKmCR>uV*0(+N^L(CpDRfFoN_JOLnbUgLyO7qJiGCD~;= zqCH8(f?2gQiq+l&HsYuG3~YW?apynGzlqS|P+IJ%Y<|JhAZ#RVmOW53QF=Az7R|-p z+XV;S2ia+T8g`Y z8Q(*uN|Ap@e0ES{X>l~Js(q&aX)jF6VZ?o9JY~dQZ}(ejlC0}!Z%cF##j*tFJ8=zWkTRt0~4I<&@u`)&U#!}n^6rZS>S_)!F;`sGm7Ox>;5 zCh{NV$@?mV^O;c&$W=Ml`%sL>b|MY557*IF+Vw_-yUn>3nU)dJsH%whrQ~NFtWN zlHxiNz8Ea&7Q1{9+GJp0>;F>gwf)4!Mf8#2r%#tSw&gRe=~(y$gg_lJ(9v9TaPeVAEvN#jTv^*LYuxkq)6bI)B+7$Z_|wY$)m_fW zLN5=GCv*{~-?n}K`%tF8hgpvHL_juF0bIC{Au%48-*5kGYjmjBwX!Y{_v3RIqh28y z9nh@m&`JLDm^A8pQY6h}NcrC|HT!Q(O@rS`xRHqn*SXOUaDu{j_X1+IRpz+=-dyzT z1_ZnwbKbo9T!^i2sNAU{AhY?2@9J)%JrTbqfv^xi8y-$?bum3zabPdNtStNFWMTsi zvqFGNw8H5_lZ&%z4nvPc7d`yW2r+d}-aeY~ln4Dry{$L@{&u?r%$Ul4$sOFkuy6?F zML>J?t$qCJU4iSzOyN(S-6H;a9w=34w&yy|#RVzq^O18qeQI}JCGh>xyK}5W5 zrNa=e&t7zE#93FM5C2Fk{d0O2=H-(9aE|FxL~Qo&f`O0z1DYC1lBAlTz1dgTYuBz# zL|qsvHlMyZv4L`J!jelqvxz&1JI+P(8)5IzVZ;8d8?fCRC^S7z$H*q(W&8`}eUvJ_ zvTX7=yDPhHCy1B?P$_}yb3$P!*fbH(pFfYS=k$hsjyZ6+o?X#x@XgII22Rh87;a09Fh3;U zaD91-(=BOUjuXV4zI?9cmyzccyc&0FicISvFtuyD{9@F1jj6ktj#Y2B7QYb73uI&_ zC{LBr(_hi%gB<+on-@QMNl-oDhV~LTt{*bUR3ESaOhikkBt>1jzqdE4B3iL=+ ztB)^EI|RbYy-j0HRdAOT6fC#M?TNjj)fp@0T(oSG56B^x6{gBg5w!$kw43izs79{G zi-=_TRdN$@kAEN0!CGj{%T_dip__wegoK1n)OfBJJ>>GIvKUja2T|Ut3uk!OOULEcK(GVuihADlYHLC`-!=ez`ejXyk<#ewcGV2`#3C zeQJpt;|t8s2*a3^^2}b9;^T9nXepVX{f79?cD&=43i_4-^M@Zp+jlzWd*O$m(z2=P z+c0v>ZY!s|GpVYWbR0_a-ixG%G?F%Pt522pimV9@Q5T$tYCCgcn#RNGkYr+KapOUw z3McoO?}=%1UDWkI&mD~hu=wg``vm6CpHACsy(iDA-he^NNPMJyA zzE!m*-FsmDQx>_fxUHs7t#PRd2ze=Fgw&`s6G@$>?qQ8H+vAWEC9&c0!iGN6p_sf1 z90R+eVTMgfH+N|B_wVw)YZh&=@qQZl?CdrtDpAUxY?mrWa}43|P=pisZ!bP&TZZ$h z5#jDVzpYUoBOTMCkacWmXvp#)n))YA*+X`$uBI$v9#^>}zcy=BwIju4Sn|ZTvvwBaqQlK2uBo; zspgXG<{-N2Wt;$n`i~qvn(5WK&dPVcFu<2fz$8oE!oq?~WRJ*EPb>xeHwL3WtbV$= zHK$eZQK`#?gsTbU9StK{ujRLY+NEc)n=3iGColsIlJ4SHi{CXhjmy__@0`oK&!-wMu)f$w$0fTv@QLfrOG(Fm z<4hf2UlzN8l4;JPL}u;vGX<5)H%_xUq7 zX5FmH(R+S{*{k4+jOUaAp#SXpa`Wf5^~|c;aFfgpZ(!6?j>BBYe?-uxNv^=lhJ6b*4`l;Kd{+u))AK%JOMLzv`si3u)wtRh&i1~%y77;Uf=f@3XAYgbq zEgd!8zgsB)G3SlJRSWVa7gvhuM6R0tfCEH5X1`Jb1Iftg9g51-(k}=7n^?XZdT+c%pwW9;xj8EsGlt!o!ENq+ zZ*yne#lP)q?rzJ~j=e%6jPh`W+`2Xy)`s#X$h+cfTa)fbs! zyJGeI2EW{)<5~;ct9_P`P<|arIS|Icp;GtZg3Rb3Kr=6K-F|&(7PnJ^A97bwp79;; zZi@&9>8O({epcy1%5?RorckXEao6b`&%V3?hw*evMJ>!B{U7`j8vRQKpZfuu{~|b8 z{YtFz_~6xnC(SRjm(N{!{+ms;Gr!EL<2>riT;vgMW^~Zr;t%>XQ0l59*!EdXL^@yL zmgNv^>Zj?SS!fS4pZmfpWlNN;+^Ov>d8(W&W&039AF`e}37bK_t|MT)warq@zvgFK zQ?^3t0?C!;PrWPg_^A&<_L5w*0pxxg@gO;$X8~$vW)m;R<5c5?19xWHsGIY?i{B&Q zA4*X*nfQBq`qWgVDtB`KCYw}QLuJ9htMIj%$h|j54j(yx;z-ef!$Y@$T?~xp-MwAI zN=mKi-jSsSC*4m^n$JtvOW2QJe^ur_oZ5+u{~j$G8$Ti1MmFHta(bu6n1;fvwhiFM zP*NRdL~8s=mCY8CFAqE#xhF5bwz)Xwz58C9TRGAdv-NrKADXExt^-FxUI@+|@(XLZ z^c`h%7qO=2p4;S zetpEdqWyh>YpE@dsX_64MWqw^h&!v`>#`u?mxcQJ`uzUFjzlqigpbdjtc2xkX7|0C zS}ST`hgTYIfFk4WwUD0X?J!bm*Xg%?=A!bX)3}swe`I| z3W~#3Gm{kUy$C^jU8#!e*iXT@fiI6xtKMF{vSX?{%I|x&*seU{%s(^{>9#QGPLer^ zL?g5>C|am&o|o`mQUDGRm!01cwz0@JshgXD1-&@4`r?rbkTuK2zM6mguBB;uyDYn> zX+p}$XM5_-cpZn9WpA1ik|O9_AAkUmWri0K3UG^m{Sq&9ylx;E|MBAu{Q-PIgHze# z?zcrF6%!54W-qQPJ$l&xWhlILfF$AHdRHRD+E+oILy7V$Mge?+$D&N#1#D7AxCZg~^V8vqsuen>r_? zypq3u{o2~9!sX`KS?K)>RM2GK>U!kTL{G8n^hlM{INBXM)XTfMD5bs{t17E~dSAUtVxQf2@@RAIO@&0E5WfuTSqVs3MV>CmN_K$YhQ4*PE`qJw(R=vU#hHB(iB zPi-F<@=H7!Al04;rpBT#c~0ZTGbFaq1f09@d1Apx&x3&?69rfN{9*+gMR6CqX|Eq^ zI+uG&8RcnXQls8|h-yEUL4p5&Kosnugwwn^9>Dz1ZO|J)`iizi(oknrO*J5?+U zno$n3ltBMG+I8Nf<>zRB!*Yi^2(ZP+Q2tT^0x!cQ9Sw@~@=u8v7j)PAZC~f&;u7lm z9*V`JRoV{iPK6-vXrG{E@Y~sPO|A1iWxU;%UT)~PP0J!ht%qFV)a)x(G7-NsBTVco zzw-(_J6wMupnO-?H`x_ETC?bJ;d}`wazU~bBrb!=ByYlKo2&7k5pP@Lsz+j3UGHUO zwgQXWEFi`cIW1lFL8jb}o*5mmDlZ_A=hvC!?KCFqx#nYG@f$pa-~*>}%r_S$D7))_ zrdy(K-@XOLeR}W0;7;X|T;aB39|T(t-yE!YN~YF&J#qf_LB>6f?~xlA)1an2y-KX3 zQ)WBj5qFdW<)AUGWr=|U+SCm)W3vN}{Y55w%S$8k^YgQ6tf*QmQlT+u0rVc3=(mbb z9C83>;gQSZ`>AycwZ508OE<3%`AMnvG+~7|E!;h!)=?@T;LuHJ*WLG+ihKa>F*7qC z2{!M4Z5tfMaMd_Nh5AphRp)ySq(_*L5+-GDzA?}UvqRpd5OLQlcmGh(m7UXG_UfS- zoYx{9$I64|-bHxHh|>i0uy*tCTXz}1ilNJP%K*vhiG!0ztnuX=;!KA*XDd*%ftloM#H#Wh2Ph&jaANLPs)md#2(*&h_Ik4q7P~F z8CTE0wd{aG`XS-S^T-MpN?op)RpxBl4^j&y@YAPHp7wgN{kz~^f(v-O-qiI3$90Fs zx*-LWx5W9#%#L@D>KCOXFvzj2l%gWSGv}R*6v)RD$r-ylg9&ZIs|@kK(q0H z_)G9BH(VMd@Cut%dVmLK*Po|A!rr-Ezc5g#yB=TA;MM+i7L2^`9Pl-cGUAQUcC&vz zO_Qb$THl5&jiphqajz6m?UpZf(h~?hWsJ)c1^~qn-;g$uvwAAOFUL~! ztd%;20^f(j$--~?jp>w_ZBh!s~I4k11+s_m6azD zrM>fNZ-Qur>jg5o2&tJiSnNoO+C!d?{IQv@pQN6Zd#WY!tXR+hXV^pol5q%{WgT~dCif8Q3#Ne_SDW(RX791SVLO4Ofys@zjC+M zlCa;dfEsBiovJlyp`%hD2qkt?v_D5TEWFl>V0k@JXZ}Vugu`^JRkXS6dOxk#lw}{f z**cp_Mk=Jp>4eF%-(W7C+6x71T&ejkrwy{rwoMBt` zBl`TAr=rlUJ?erS0DCd1l%`_B51Wc%%`v|VgDCPw+f&5D3ct?7?p6vxTNe}jmpQQ_ zVDgxv^_Cb4l0cuaOMCnaZ+_6vn{2*=-wLxCEBp+4O^9v46O)1p3jWd9_&m+bkw%67 zy`F?_K*@tcRtG>9(LvfMu*7G;$yK%6i!3Ei4xPv(w&PPn^`wuG`(nWO48Q-<}NxE^SE?ja^FoK?s_y z^uNLS*1>k&oN7xFrX8MTz9shDa?g^KL z$A7-_A2g?^n$8gj=Kaaw?-e$yN$<8fFXbR=`wc4VmrVGf=8oUg{4)dpC0zqKS50xf{WS=#*Y@MB0M(jUmJuBVpMoplZDo3w5T+_hNwn!@$4*et}7$>JMj`v+<`*tdAsl z;)fs{A#Sz%uk#W50^@=+%!iL0Vc7OUGz;{pC6JbfvZ=U%lA7frQG8 zrlWFclTE@BcS^ALjG0oy_}-Cs1@~*zw^h+QLDN`*7UJ&bRaE!1>bqQs#E*3vRY`2+ z4UGpL4bN%R@2n&YHhTrEtinO!a@-6^<8?~9>HoyRtrk>g}{Ej=4s zK;IXig|*0}$}Q~ZwzP!r+RQ?;T0*u~qVdNY6dfM5*Jp`KmF#lY9s(%#?KhJjU_Xub z{qyVuf8>E9ubQvB%oo38X*WarI1vh$s1$$KrXLevSm>19t@zv$RpvNeALu=Xy_KJOQ+RwFjM}iHo(*77lt|?#2#zL7yp6=fIA4RTxKS#x8zN876s^qRy zG@y~F7##lk#V_QaKR+VR1IDV*y@lab0ww*vKzOTOV&zYYZEC)KC6X%R3CIRx8TYQ; zRbH;)TE~_KRuK-_V$$HGlJi*YG!~0-l_Y!c__X5%#~YB;wZ8mQ%2a`Q-9OR6NMWjH z-&DxDGuil@lGVco=ObvJ?QW0qs^!tPrRC+h_!I>}qsk33>5`5IlqG=J&FkTNe4p?1 zQ4-;t%&t-7-Ra4gD$S)NDd&kx>2n&%913199M?(RPY=`iv0T;jn0T(FdRm0&nIA6@ zm*?{778TcTiCtE3U_+Zul(kSxQMD(#zntsU6EsVb$Oj^m2OVcR;iYdQ^b3vY8Kck2 zT6{DuavuB`6lvEt7!~m@@ni9d=uxz3(SV$cY**4X&+VROgvVBEw`744ODkdF&qq$V zIl17*H?JRDNnZ&McMdz@hKYNH-3Gv?*YfXA58F<{i`?Av)6Lg2FF5n|%^N0bR_g_G zJyupO+90Z88xo4#x7ZtDi5vx`XvC11qe-uk4n z`wNx}u>rxzTvmGzk8&UYaSps5uT&UuwD5S_$bcBit!E(VT>;}WbcXiVm6h)4o3}$d zlde^0;PfuEbiUv0%^xU=Xl?5*(d#^~9L-K@O(wws70c4N#8q&2q*}X)r9m*li~-Z4 zXK=dNG_XF`D4V#-jg*mO$tH$c#)}lzTkuY(I7Sy#gxVu%O;;GwI^7Fsh>8!dhI;{5+fk z)o~Wx3srWThMzBfkRfBXks_DT>Je=gc0*Zvd42YOsYpK;O;k_DV1mQX_ZL+WzH%~h zDlpK~r-(T}{|((!de>p~5f-9nD;+1c+aAQ@`ZGHyN(`y8fp<@X+$<8eADUZ>%SU+3 z;g#rEg)XHZ*jC8b&)+I4yk97Mm}BKzxRP#v)K?W_fa`Ox_XY0l}u$aV7{ z!s!eoN13pRJGHOx?xqC=Y%l79gBsD=-S|m!$uObF*l}Fmi_(iPEX)rwA!5yt3Hb(@ zw-cNs>>qNZHL@Z4g2^_n^wweYqNL5wRfrg@6FIj&Kbf^d z5%EhCRaH8<>Q53+#;I038F~v}hI`%ZJMXzXhHCJcl;>LPMjI8Jmyq?^Hv4=1Q&+ka83$m%gAMd6a&hIwS(+hXKqSNK zDcTo0pB}S6kEg8RGCTAo3#x|RugK)}<;j_k4#o>5W!~K{ci`{9V6;BnujZA-9&L4$ zwjcG8Obkw?!|D{y2Va_dbI+pkl1w^Zf8NOB$#fpjUW^=y!}U+~$7C!1S_NQOycSpj zBZuumi;!8Z&m3L`C+X0>IMq{XxxQTKzspGR+NyD<{xDOr+GwSEF1=0a`>O+5&t5nG zizIm*5D<{jMiM-u2DwoP2`oS@#j`etzZT*}eh_Wt&?AcGjr`vVmg_qZrU?oa0%MhR0JD{zWSa-Kpa<0!BV zB%Q~8=$Y2|72L);rJyzqDg#2wEy0c{@S``Y3$V>j3i?*FP7MM8N?R$5eV=2s&w_|x z3NVhER@>YwsHInXm%E}{rsq#$4{w;xDVavUeD{U|xj`pGeFce#bm^!4S`s*MKBE%V zhJS(&jMtTTN4}*c{5-MR^?k_ldznA9PEflF`~WxBKC0=sjaIw5|M_KJVfVY(`wk`f z&6{8K6DJl&P}r$7tdh@)&6fA)HD90Kd5@m%{N%LOOf$^0hzK>4aPF^Ai!*^gA&DOscKheAX za|n5a;oRJ;?nj5jp0&WHZ{Ns7FvPsIh;JTP=0$%MX(*HrB;reX=*p}*ch=K0Z`ZSz z__yf!>o4}MbY;9=*&ocWvfc#`olw2t#>W%wG&hcfoC%#dM#pG|vbz&Tbc8fHb?%oL z(h{>Q>#_1xzQ%o^NIyFeRcTa3xR?$G*qN(_h}?%~`2P4(8xL~T2Fxx-vPqFiOHj(h zi?1kh2BRsAp8W%_*7DBgqPWxGO3!Qu!UIQ4vFpn_E&MPXyM1I6H`|`LPT4!^N~C9G z#KiJe`4<}Fm87chMCst)rM?}NF-@WISB?6`%-cD$pa9|4tJ^-nTaf`=SQjen`uR-j-lOeT)5hupv(*=LdYVctRVV6vlf&Ps^56`> z?~PN9(`E{Kf8FtS>*Vdw$X#%cC#o_~M!);=^tKmT&h<2bE#x=TsfO6Z^~+ERA!HKg z*~62+7sLp+Z{_LcpR}I%_3Ib3%c!uO$i2I?6CBFm50WlPPeI}^1TxQ_ewgKfM`OM` zGJ(|}t@G@tuTZ6}6XWNu6La-CN5a$4C_dP+lL>jasd!mjYk9P%>5H&%BVpo{_O_cP zhU)+6t#@-6BWIkjiCc5~MBc+fkE!1aV5yQ8R~vqG8usUyqHhFgk#*&-jd(8yZEUT+ zy+aqC<6D+hMHqGL&!Y!)^`fuR-g`_FcNftxMU$x+rtd!AJFq-^3j7HG=mG9gxrYjw zfSkO<#mZ`J8>yz3Kq5?x6x_NO>V3ts-1|zGrf^Ey7^ISbYda<;2J*zj0!A7O#IdvW z#(=>6bf2la9PXMV?>RY@`MH6Ond1zkjq4!}>oNbk!hb;cd}Njz58ZFYCp`s4+V$Nc9Q5~Q5q`L`+rSzYeIZWDIdyHez1@2V<(^z#w}st~%j@3- zyFQeDG264RH?%qZVhH-`YT9Hc!i!`M}lwCp1s~L$WHSn=;Gt87Zx2rKqhVA zclPw@4?;$ITmm}PzoH5n%Hm8Y1Mc6dOEjaRn3-cVH8l}4fBrCWaF{31@;WVUBvL<6 zy$33VeAjW z@4JZlPloe}@`V?uV^_Lfb1O9LFYq#cZLagG5P5rhsN@RG8&~raO{^K}xjF-ZPAZH= zrVXZrcx5GjId6iio7+yF>us(c+-8inHsUlh^W@CC<)mU{66#MaCUR^ocX_5Oi@BIBX!!~$KN zKPidYF@1eb>6-Jnd|$q9#rk%}2T)X{D_9o`viEao{L)(h#5{M(ZIQAR$)^;zE;$e z{Ui5=v8r<(k`v%GFzoqC3&_^m#i@#AYk77aGkJTA@bmu=#Qs|=pX3>XlarHCC{$qn zyXOx3mVlnhczuXFzy3ydrb8eN9>|LUp3BB#SVa~9SSR*b-jJUpDxkY-xs)RCnj>o8ot7n_-A`70Y7;-kHJ2LMvejq7*3 zmZ*-TR6uOs8|_ceZ&v?0tR*ZWIbL{MBlrFVia!ZeMey4=JF{d`@3YDk@y-jFb3bO`87(j{*!5ZewAMr)o*CJ&XiS+pmL zS-%`OM{`}_FUD;_g|fTNq2y;+f9o8V-^R+lh_}r%m>=H(8Ou3Q*R0PjTVARCRT(n& zf@{^HT_X{bFx>yijsE&<;8L-|RG3*BGcpNyqCm)3kmS zo&OumqH*|JTYMvHmOv1RG(X13sLv5(u3x0(F*#ON)>JLc zoKqYe%UEpK`%4f07EQmgZNnm-bl+Ml#KJw{dK)OzDr_F>eNNyQAQf(}J&A)5LfOy< zyd9ZL=Jhu_lXIDnP6gf9MzN=Ru|nA+Tjq{-pCg zwYmesDdZF-?`srOqM zt#NuLN9iwYF2)rFkPjZEy|6AaDRM*6Cxc!rw*-R8od6k zV_X_?zrRguDWE4PHAy}nSL+VD#$?;tb~s=YQt6B4X{nao36L)C_LATE!c64DG$uo(=e*AYmYB7uBnHC2@7`- zb7cc!{%d9*X6BKKCl5l_SeW#INA-+91;39krtDfmPp0r3kX0924j-zEA?W1lgdC=` zUS!|!%Db*zny0ZJoFc8$<(T9rDdefYyI*0}KBF=>XN2)_m75_V?0-f34Z+|>Y1ePB^$LHTbM94wm#L z2btLp2%KupCw^AeN}!fnQJ;4%FV|?MN|TqWroEQNKLYzKY7|ZW>LAH_Rc}yJH&lf6 z@3y`O!w4AMx*Lh}T8J) z?{>-v1b%|V%e+TGvTLO%`fLo2e{LaG+Vy$N|DxD(3SBmEEP8Pmtcj*8YG|L;m!?G> zsbji%lJ3;T>5X5pwDmV=Uo}4=><5CLrARpUUlSEo`4oq<7Q4_I(AAuIW=K_hP9rnt{%{qw_Mj`rBbqa`fH)NB41 zZY}QAS|E0}oDWF}NRh45n3f&MBMwV65R_I}9|=Z9F>p7q;g?JlH7ihF3sN@s(+n%} zvjamW>B9EaDqIdw&GYxe{|9sq|H)|lcYcch7ew7YiALA>e)y2)8TCg;N5{DCGm$$H zCwY3*zKS}HO8e_i2Ftk54Fb%3|MJpHM?^qiHtW?hb8yxd7SAi7`+#hL=3+erLucz5LD;InvhZ^c`(=a&2J!(nu8uo1yWFzi*`S`fYfjam{s$ z);EBaR8N+4Yzn1^xG*H5&r7@f*^dw#lvUUd=;iB0PESRZR>K1z?%a_eLLD$ec0#1G zFHd*D?yJXz=^v5g>1d^s0qr9r{<~YNP|ULbglE)$xzkhRywC96f4sA?;GheyC!m}6 z8(cZuKoP5;!7o#ZWLeL}t`s@v(dyM+Ev2>rOtMswMcdnay#nB6gAj+<*%RD%UvEu@ zaZX7gGN6cMEdNoN)e9{a_;nN#TCc%lGVVj*>=>ZM24tnB@q;Cw1IfgmB!Icxi@gG` zn6H^4OLs;PIQ1RS(a~BBL3?(;bF?8I@%-S?@f!EAa4*6*fS)~ce{v#syGKUIU=H9X z!P{E=ox9!^gaD9`^KiNC?q*9AyM13?+^tWK8I39(fC3ovK_>S5sZ?1{I~A4i01XwD zR*)BpHJ|xjf=BK!P-p@J7%8_cvmX$yq`+Or$H!s*&q>%l3_e1CUcye?d)b&sS|t*R z>+3N6hGVs!5jL|0{W8eP;xH$-L-)K%HO$2PFv+_PJK(N>kw*zDnU{Gy-eUx$@f0(P&Y$Fay zpR^_arQr%#f76Db6otB=z1pfO@7?;lt}~s_@AsZIK zak$)XbMfA_+bSw5XunOE)AhFZ=Brc9F!seU6p6Sr0-IP8L`x-|T+Cs*GaLA!b=bYI zRKq1tTOv;JhRn0x zh0GS_=ih&~pJ(R2`%cKHzojLt`Glb1%IYdaJq7I#74vX$J;8RRfEnoAoel*7%4Fh_ zQMpYC>ULLw zxji*C6)&b&<2K7C<;2ttA_Z&`!%S6{GorA0c3?lYF`QM5ef6p%_d#~Xd!W37%@Vi8 z6b?^)nQl;Fr)#Ux{F242A*hqWfOrV>WefbgyxA)&D~CE(H#XkA2+KH~Dq^OnrG zhSydmo8S-vU$q{b(7RC=ws$t|D-|^vEx2e;ocJJWehdBza|1#4EjSS>A(sbdM;?p% z`1`}k%2sG`UW`!OpzE=#@C4cMAe#XgbKER{zQ6Vg##?G}+#h&7+q}yR1QDmtolAsp z;kF-u8}`wddB!zsa9qOYVXpxcjZZrr@%ZuM8#lgg;s)Gj+h-RR66+RA^Ot`8x^El# zw_}6WPPp|MvEKzo2@=mP^MKptr2{+p%@57uh8y?qGx_Iq+surVSx*D#ZC5MA-^b?* zNC-Fzur4N*4v#?ag9!Q!&&)`O=;>;xT4CT+3;^p1&jfPJ4(1akU)Z?%tDm5h(`rv>9zSDLOOH=%u zJN@g|8~X`bTxJDGPnZoQkeO3*aVcl|0?s+1WWV zg3|tcx5BRPN!5zi(s1T|KC8~8*_jyxLd@!?%~0v=>U7J#lv`M+vF*+J7D87RbjAeb zwYQ8Sfa6zY*>QLOsG62p(W5$iL4Lr_cN(dT+-uiL61N2*MHojvWqE8;ACaRH!!=Ur zhzcZof!39Y0yz*9Na3P6t_}H23Fq+u_}8@xfcH1QVl6Bzj35|;*#YZ%&(U{sauPIv z^}sSL>D7lhzhHx=Dj`h^jb*q%c#Q7Ptu@WT-i&*(a5mJcR!B=D&@jgUR*CxX*sF7I z&k8YV!>jqG0zj(Ec@_^C_o=tT^d!M5$%6*0_Ck$pr=MU1m{WCy6b|M)*A(TyI)$7oz7SKYROm+uwuK2o(#8v^9ajC zCK1vU1N{sP-ox9%c}Wm5y22Tn{1IdVWvv5>(}b6LGWY91U~_E#fht)S-C;bT4LRX4}7@t)8qGzS^?mjAZaD7z-$xE1DnpCSG|?j-$Mb zs;WpJvuHpf zE9Kqi)VHltWIdmZojJw-AsnzQQQBSbw*1TABTnrb%9{gbS9S9)7o1>Zw3Z@-=YTeH zDD|~S8o_{v zrqN@t>3~j~Njq036U46z_!F>6!ALFRn}W3UYVvtlcjKJC$^z!(l z8>gE=(6P&T3HE8S|7qAp)LOzjJX4RDuFh0TfHSx`Sn?DME7*Nro}L8;B~zm{?h5kq z;jLq1fxw*&IsvTBBG^?1#eYB!4yk-o4BP=T!0*?`n;@qhM=FMKSQa7}u?xPb!Q}_CFx$2O`0LU@CY+@2~qI zBO)Tg^%gsLScM+u{j*2UH&OV4Ub4r5?!Oa96Dxe+$Tx|07u|NhT3^5awg8_}=K zSCEQ^>x~EONFLR9e2+>q&{q$b9fEu;To$6?XD$l;jb_6C{a1_?V_7 z7{k#-^>}ePt_;an%_q;EwJG@{0cwt>wY*38akYyKAd!kY2EO&;!_ATmJ=y$N_+4k_ zOE7}i?`^#Rn-kJ2b(meOu*Rf_yi6|;f{Z#OgW*zK4~Lu3ym86|To5s3b>fxzm9p90 zkY9gDw}vmAemnW2)9Bz4MzlAfJ~`RQ#%3O{!(Ay2qK!fT3$(GxX?}8?o}T4UUDp0x z5+4{II&mf)Nuj5wKNZ2pXCRHdcEZGB;$*V|AvNUdEVI%LR{3yxbt-d@4GSPAJfxhKhOk8hI@@`fLg(JnHdzB#{x7( z%4T$hb0skWCC0fAO>5cZ+9x|-K#j@5noIi zVie>9=f1#A;`00664;)K1}LqDm9pdmzR`E_^&_1yE+Z^kOY^IQ@SHaN$nYG*>J$eD zG{2p$zzs}OJGr@)>axHlxORnh~{3p24M$uRHvh0uD?j$in^E^wseXcW!sk`k#_K=lT+4k>TF$5&a#bVojmuWRD$$4X4 z+;LC?3nr{oaHuh<<|YDI3&PXWKO^5?7lLzm7)^WG?U?pya*)^x%oW%wEbR%&tJ7Mb zc&~6lJK|PIAU+sa=G9CsG_6n#^K$G;p;p;Pn@&!qoo!x)V)HOf6Gm!a`Td(k;DDQ< zz0k^ppafyFdsAjl7epEq{dO}TQ3zZH(lQL#C5ES_O@gi6I(sEaSHMac@ z0? zD$YIVgN)P997Qzn06~#>M9!nwD~OGi?x$u-N?dC-u3~)BgKcfOnkmCAS|92Hv4x=U zet709M_y2L-+36@RvyqSz@@{li?EPP#`Wi-?aA>G6VYljOfcTH&CLh759KQN>K~75 z*&~f6_hG8g&@{aN%Lo$(vo?T2p$cCkMScNkA_sXMua-iEh_voSPh9XsgNSbF_b$ie z1Ytu_i(fQxIyF{fp_eH)N8M+KYaG{|uLdoQl^tbv9H|f9pAebc?i-)`ilH5<8UhRo zSIJiq5!wNQn#q6R!4?e0cAOud$Yj5q{h(765P8?#hRVTpJ-HLw?R5&t?PCr@ffeQF zq)Ev)*u9!5ioFGD*rzR8Ox^mC=kt6iu&{`kS>215#lol94ob^&RiM~`?%^0FMYaHLU-vB z0ul(NR-X6Tw@=30{k)Im#S#cuLK?4@6Ble&?(`R?rfZ#$7K1KvuXry1B^1B3`e1(# zJv+g>bh0Y*Et{0{N$tj4hvhsS%t~(gBB?Z7&;<_v2p~3bbyuNb&d(%;gzzMZJ>Xqu z2Jp#j`1}&eyTb0F^Zg>H+><<@NuWvEu9v?KL!;lhF|>CYeQdQaj@X-x$zTVIMj5tm z4Esv+-cAV>tg8uszf3l?BJt8y+wC7?WhY(P4(<= zA@&4yn+z$nx`i`=lyA*YAq|l57B75ne#^3ge>8{OcsBLx1t%vY#Z&safZb-h8Gwu9 zThDD02CM<ORId&#f1re)) z0JmbX|8rHDdwdg;lP8_(iF>=X{Xb}X52&iLbxT;qGAm|K5zK%nP>KQ)jsZlDA|)t@ zBvAp$k~2rK1QVb{$qF72NlKI~qLKt8OFZP9b2y>rw(fne-}w9W_(%6>$Guf0VV}L% z`quZ&Z_c^a#>RUgjg-G1pR&G-ZhUd3Kh>7@0GpL}OSy!E7Tsqu&~%58;pV||i!peV zDt=&I_MZUQUqIVzT9+apt|b)`nrz(k(Wf)xTAtuwYy2~EOf#$SOxl_1LmmyGX0sX@ z*#asz1?W@(zNqd+a+5sZov5vj8A{wKJP(KbpLCTKHyh+ajOvbRxYQ(nc*1H zrF*uvyeJq?_`=-_)5?Mk>)yG*shYFM-c>O=-}JN_)|t!n=SjL{na-tA4%S8_r@6kl zCqCUJ{tX{&tO@INAK|x{8XfHMXY^Hes%dJtgh=~6URP(%VZ7wKFV7*!*26XZ$uLW= z+xZiRe_UquJFVSzRnU{QUZLP;IP%NYJ}??{I`3_&l?Qlw3od}_iF)c)~ta5zUwy% zz5nBIfvtR!P;x^vrR@bG;28Cyy=vS&JRcgC62I)&s|mhETDvGT3r41*t0EexXt>5< z@VWZp;Kl7{Hg@^Q5O;9;(8KN6+&?t`><^MquGKvG`rW%|xd-ZFC5-!Sa&TyzBc|%K z-u-{8G$msX!+ziTp6NX|@h$xMKRkHn^Y35&pLFM~U>{R$ckiiQvtdJ;?Qi|0Et&5u z9GIH!*KfqvK}Ojcr4l==Nr2BKb4UKa8&v-9+E$cDE{J-G`{ zvt`dwE^(*3fmKu?jcgIqF9&#e3*l_UcF4U2=jRp{Jh+%9o?NCdChtCG+%;3S3un6Q zjvm8y?dFk@WPOEzy1tY~5hED3J?tky9k>4-UT^ioISa-nx&`2s&$Kp_ zHyWr(uwN{10c^XPa9!iDpJ7FLJT@r%*L}+BU9%qNnjx1iEX<#Hr>*MnUaO@M&ioBt zkNLFX`tliBvDf0i)LC)`6dZNDd-pnhV*mdA*y*b@%SAlul4C)N%hy$C=N^av!*Rh# za-!#8M1YVs4s+;GKI}IyB^4SP+Hmg5R>JmpcHTl}YUmpdrx+^|FbSzBc=%W=+b*Wo z5X-)3Qc@k&kJUHsvu0b#HzCr#W8`PjctINe@GQy%y#2ks-nYe}esH?VLE^@zyTwM@ zc?X|u-+V?%s$GWA|cIKp@HZBBRU*&oLI4K6E=TbDo)NZenUX zyZoZo^x!dr40!$KCwm8LQVRnq<<8h+Fc69tR}o#s?33+o^!drk!u{I;H-O!IT#QJ% zWpwlM%1++8#CZkmpSV$sc_kZ#Nm1ThJ-oJS_CN&g7oUTm#vwWlPq%jnsHE@_fb-Bn zBX;7tFDjk(-5e_l7Dd)a6GA-!Yu`-5^)u1cREuw$KbWz2vSvpXO}5@KR3zkiXL?4!&mbJuVzR#yV!&KJ~h|H1c*?^X6F}rr1<;N>66R`h6v%{ectF zzr*x}xzj0gbyjXpboncsyHJjG8OrEwZlIKxI*z^#gx5{>qJM3|wf%<=51@OjTEJmk zG4?J$?=!dXD&n=K`mp!`9a}E^AmPY~eGzc+4z*C)n)W^%A=&W_M^qcCal6U;L>r_8 z@f>>IE9Ut)>T>vw?c22?>^>qy8QFFQSnjB7VLiOR<>VKhPN#0(ZHQ32YBlwpCMu-; z@i4}rIb08`@YRYF&pI0>dzYHEF3+~NocgtTej%CHD z@o}p$2Y&tfk&FIE14g|`dA;oq7oYUCcNT7NkWv>nDS128kc_v*@4Jig# z#1On_c)dko&2Q%yuC#uc+t$DT`3By)@!JfzWKZ&$wdelpPyADXkL_B%j(^oZn>6p7 zT*ayM&*6wmii~l>>Yjhx3ojbry*%^J3q5e5&o5kWO$uv3$o~3xleY8pHNV5x1|U^I z>;_Gp91Z!WdwAFxMDJFT4b&}_M6FQQKiu5h;4Sz9)f-|Z|MerciR00KK-w6#7_Tj=_S zn*+Ngc5sY5xt?VM64E3mCkJj!pCUaF{1?x+Qy9~#kd8Ty-lzeY+;vR|4#CGeMeepi z1L=6gyKj6_p&@`6o2sKYr~(9PayP;Xsp1H(43QeSkwyPPDA}n-s`!Ztp~KVzonJIJ z{;iEm_fBqX)kN z$oFh0OK=?w*2Gl9IzcEw?yay#V$R;HCQ9G@d>?iQM@L56udd-ekf}Uq69l^tPmQZ)o@EWq~zt1)Csv20veClz1Nr;sB8-~787T%#Lx9E zb`@_iGaJP~!yfepUV|y~fCK7}o<)2DbKX-HtO^kcfO9Q};PXjS1k?Q~#mLI7sf6AN zh8-0jriZY7j|Qg~U_=pJvnhNh$Py6r6ll55>833uty=$w&I-{& zXHK&+a}61puMmgf18{`4j3>h+EkZh%vBH;NT=@vqi*TjXzOVgz^%fjw_-+r>%y7x3 zFzC!9&EohA;a{mw2#sIhO}w}76vhj36sUSg1v8LGTOeP+8od1(6*G#}bSMHnDQl`) zu~-1M0ZjZ-G8F>4AC#C?>(}oVx%;B>?g8{&5ZJK5QJH%n@nnAb1S1j7D4dW~l)=ys zwB|^}Dk`nP;$~oiU0~#aA?Wfe;r61J`(cu~ME-60ZRxO=#)w1Z+#dmz!h>6C0nvl& zwWb{u|`FY91{L?a+K|*4L2LhUL)TKa+nW2UX$^1JS zeK5a6(t14=u18u?jk~_r zB+YKT12rGo)9B)89&ZFRxg*-Sk~^laY`y{!*lojZ2k5BC$7t8gFyyeK;Eka#%~TOb zn8j`UN^cR!;uc*u&qdfmOs*eg` zu%wW0wPUh~V=1v(V3biudW5)|V%%`7fwB*|^a3v)?Yg1?Hc1O}UJenfL`sMMVsj<( zD^_7uia|BxK@sC$P_Kqr^hqz3*rS@6DOR#_GRWG`JU!9lCT0IZG|=gj6;Sa|E_g&u zL(WI4eY*E1eyw&f+%sp-p2dvdVQV0mVNm9A@o-C<;!i(qt*n4&hMtldD$hP$zD_S0 z=c6v*Dwx;HV>SLq)Fv?F7$6~HWwc8FOTZHUd91ce;Vx%{$KEE^p<*4T=*f^B-?j7I0OxsaH z34NQkEJtg!zrR>;3M&Uf|9YHmGJK;lLN@kl%$HUz7zkXT;?Z$(2X+Ab*bXRjRt5yW z%oNb~G2N02kq-uIV*{T1q#S&?!26KzDU26~Fz>(e^8DF|e&A9jgjuiM65F?L_a{+YV87WazMq)3 zi-%%gym$f5YWeczH3^!*j}cpyca5R&L2OLYuYgO-8A%LF9O?Ne3O-?%!Adl`J~lJs z!lc_@xUfoK43!dUcexL*7o86apw2KAc9UWxECenjeEnmAOvWtj>tN5|pgNuv51Eyb zQ5&PIkBWluWrX-JLIju<&v!^iH!5}G^+b?DvEPAt z8VapUSATy4w&G(VWMySxc8P?1jjLV5yBfJJ9H5~O`VBf=(Pw0^RSjxtK%(uvdyk(#KM;Y;33H>@;{>go8*k1&+I*0_e8q|t z%a?29xky22U9Dvfv)r6Ja;qQvYx#y;^CvZ=9oR>5bo(Thi;L-6;6A8RD5e-Y*1PpeZ>)a47?`jQsc*3#VaO4n3a1p~A zzVZiXf?vXE)o+>z9@tNy9-+NCQ7iD6E=)aK+qPYU2@jb7r*5@bUiMeIRb=s0V}aty zO15oXYT?_>Lx~<~=@tyVz-x7OZO$5*zusOjTXTKmUqfpyC@){i?;6D8y3F zSGn%HcG~;rExu)YU%Xhd`d~&=|M5IhSZHsUv;M4w{D|r7NNzZ3MsLcN*XTl9iWY-rB2ykT|`z3lMM8qIG3Lb&5jsjNY(d5R(eCXQedi{qnC&>y-s~ai>wl@0ysLP*5INe?x18^MD5oo=4Mu zb;ywxUOIggjj0mIwNUa7>y={8b>_`|6DpUQZPsQCs5(#NV2YM-avr+$>Rz*XVP(=_ zCcHF12QnsyKJ?m?W3=fV93dx4sz2eYznmt^^kMVRE%8;=vZuzzUVJ8}M4r!0EE`Kp zfjMwQ*7MZByIK|X;r{4wW+91rCi&M&u6w5`S3`I-xNLsu^oCfLZ&8%oyI;nM=GA$< zu2SCn@7at_F^QoM1)8=k)>9oR|qF*toPzHL|Mqo)13^-Mq9d8R=> z+;6}pCN85V2-c{gr016#D2zevQt9vA4b1NQZ@G0%HwtBsxt6f`sDvnpw)jq*oBIVE zsh`oUFsjNND_|!kH8vsB(8y>|UwSeSUSvkAb+Ryr@Hq@Y2$4#shMB5y9n{ z_^6Ox974i$Y;`Fv70SCbSdla|oKB03PyJEBr#z42(J?r=QCHIIZWw_#;q|=j za6>QvR`Js<+&9$Kb>GdvPGXQhnGMht^nQkL-&RP?YUqbr0}rL|i@eU9dZfV{*H*ar z6yqsrypYe z{fgY~YtQSwA}%;wD3=(Oq}G%OBx)S7PpQ{(Y_Y76!Y_#CPWUAf>rTozuj|WNp|@ZD zBFB(e^A!~4WVoLa)+q-^Ylq)rE(IT@Vbh?C;Z%;gG~YU#n_lzhj74{BTv7?{&%U1; zK*XNEZhY2AW*)s;$VgAPewP`0xevFBOa31m^of8x$!=d|KCMBYaQdB(e!ov=a9f8= znKhJDiU+90V3>D&m3_9o#bGcWle?Ggvv~-S*dDCpzK#lfvdLJPJ_nm5w|&=35_8;Z zaU{!(s}QCnMw(S;kv#-m$55;iNct}`mw)B$%#tiHOsI?-0U+Ylxt zhEuAaKEEaYyc0{&ec*?nTGHQ!bpq77>$g3=N13LjrVay+BgqKl9m5QrsJq@7WRxJ@ z@pwN_#VkzAG^=U_$I+ZwtS@n_10)b;7cc40b)D}bJr^1(Tu$PPCTR_B9Y3_4#$`)< zwCYTRgYuiI*wp-C346?D7KRvye~^iK(qF&5hRDoI&OM>gdqmL-SGit)RS3X_4`P*> zxfF=A$$K~B;}oUmyYO{M_@fa4Ja%mlLb}E)()TGvHU7Sqw=ApN(EX_PfnJ?Aul9@@G2&U1a zt{&cv5TKKTqIyGZfubfaOODGtRe&|Qn8-&MqZMa_(Q9kkG;|}aSn2EcB3f@_9cbzoA%+Nf6)YyTT%ZL)x! zl-=ch@ltj6Namo!NZ)VU)DpL&sv73s&aB*B^Uf@<(^Eo4Mp$I8v(Kp(iZRB3#OPPV z(=Qk$=A_zMY}7(k@Eg^jEJ`6c0+`#1Z}?fKU` za{pp;{onr5)W5F2q~;Iq|M>m?_H+2pd+eOX{i6BRY0F#4nQh&L&hsX(Ip?2s^vBJO^x3A)XMe@=9jxBKyB?a>;4Uzy9@_*(HB*{O1?{b=ympSpWM$ zaby2_%Ks-nnAJ;(HgG86kG}4&%jss)WE7uThdC8N1YylDy6rL*V@y>$&Yvh$vY$ zF=G%=0?W%5{DDwm+WH+qW2fD9>?&nBc02UFfiE)x8Lrz8|43 zl|0!1LAC1hC+anlfkcP^EB&Z*&g)1iH!59pn8UzBc4^K(Z+!)P$wghf>*zN>?nR*& z@G$RT0&+C$b1Y*8(w9zBXB zK8{Z`Zn~Hee_vW3=gHdwPhz1Z9odDQH-ITQ`73r!O(YuxNzXQ0T8%sP1nQ@?*fi-b z7&gCN`Km{}?vot5s6)4oSm*QOg4ttV!s%VSMDLA7;;R%JpVnxbx7gB3DND-D)3*Ue zghic>Vx&ZVq+^Gzk30Z&-=4ixGDJT@{3>|hS~1VTc$4T*n?^u`MZ=~QClst-Yt|D$ z0qZpHkbHzM*}E$dAZ+PMuD?=@!tHwk377`FAQd-UwBIX4(%vH6_}9iZ+`cn?ypa7= z-)qZq@m?eC#YXTOl;PC+nPJtf6$gb6?0|(|mP6C3{Sil)UB?s9<&>#Pxm`K31uq8^ zvP_Z5#0TS&%K<*nw`-tybz5ZMRfizg(MkI$srk2HgamT`{B&kkpv*GP+>6`vNWs+? zgPFl5X**gLd&`35wiE;2^Z|tVMD@$@IB#`p4?qINjv)G6A*mGzQwzleHOd6@jASQQ zF-qz7Q)S{w_0-a;FH1-v!P4f-ZNGc&Zkqhg1>^$4sRbAjfe?c}fvBT#^J(yV6NsPq zKUlK#U$@p$YlsM9E)0%l4kqM|YFkyx5)}Z5fV_p107EUdwB52VO`5_Nn|rT|u!{!u zo(88i4IR+iYelJPl)E`DVZ?F&CbA_nx>a>MARKXbE8(0i^ zQ&gd{SAXu8-&`#MQ!|r8%PwE|d9A88qwOKPsewT|yEG^hcFD2R{sd@n=XU4=aY(EQ zkVEKH%`jW@L;56jx}d$F0vcV=M$!Al!1Z9#SpcA1@6J42gDEpF+%1$xw+M=9QL%)dC1A@Kx1Oea})AcOel4Qjq>dHbS z593{An0;06D(5GjBbAdDuDtaN|OGQt_c{;i6>(rUt{f z*dv+;Tw_$Fb5g+);tDYjqIvUWE4l6VtSjJd4Ug&v=*tFm7>)o#I|;+PzwS|MEb1M} zv#pLS;LAJ%*@PW0$CDEm$*1E#vA}?T#@I_x7qVO!H{MSq@MOAQ}KJ1NpWah zOp#@2Eaof`qKD)s07`vmlL!IaYP|M*iRxwXZeQRabmnD;NhR9Mi2$Sbe_V4G9ABzL z&YA0Yc8BNwQa!XgM78IS4zXx7h|U3S2L~aU!$i6X#gHMU(@_L-i4p)jmWxh-=xmRd zi`~W!Bvh0prFuHX0e*cW0$Q7xwA_5So`8K0j64=Y$kRISmRcQlwGi+VX7(dm+1L3` z=zLjTEkZ)OmdBh<ksYo6V9ugD%iyR1`w)<>#T8_h8 zE{nB)DA8Ex#0s^O0UtXm5l89t@NFJXDYdfz(P|DTR*Y_Tdhz5;a@8p(p z>hp7N?Pm`1eaH* zU8KiJCGs!|WYzipKw>=YdkKEW-{-YADw11AJ8@UDGztZgi^IW*trqB1(L7I~7~S0x zg0G?1<^-dR9=7V-h3<_}tJ$Zp+KCE6G&hK-_#G=rfW$&h?Q6*~SjY>r7(A?AUnS(; z2Bhc2oEd`gt`^;#FsP??67fqkGx%nIx9N^sbGeFz)S`<*IVoi)oBkU6Jkw-GL1#9E ziJMAP)poXS!GboR`t@l8&qt3gl4B6dsXgod8rNvFC_OrwH7ZS;y-2M>nphA%ziBEA zmodit`sxXz=U|tfyRSt96j)cl2{8!0G!q{XP}&XLf7|bx${Pw%IJR2<)rDFmJokrgKg7un(heE){1YSp?vMxQEehH$)@$mT{fB{T= z0O;nI*`El;MWGsRRLGtzx8#!JPeo+cuH6LsxE&uz^!Wd@B-j&C7ABRIlOv^2-D*Vo z&%D9oaFCN&1^5A~!2+(Ug-)Qc;??af>kB}B_^Sy8MFmC~u#e74CdS8mXc>UjaQ@N^ zc%Cs?EHe{2sL}vhktdzM(uN7JK^BaXGkGO~3~~C88H2KJLqE-lA*bo|gExB&sNq*C-yyrSFaVhJriIm638pu|L78 z#R=WmgL}i&$jE2SU(C`plR9^L#QAZJoe01iM)A`f&xAaYK66A@irF^SF~0X3&O z4h`D8yhG@IPqg}_-ATRnX1Vh{6)rQ!9V)F`P~(CGxUlhLoA{~%f^5k_&y&kyj1mc% z=!UVH+18wRGq(CC#wemJ;!cg-$XUE46qxNd*`d#w` zxOZR?^(J>-@6;QjNyG{R5DaTDhz`Z3)8fIQzCJPJCd6_Znlc=hGL2*_C_B4BG-%+o z0F}?;iPtx3Zu`1JuyU-h3PgRb>yL8V+@;77RzmO_X{}@d31>cz zV{J?(ORw2853B>JSuxd!tFQrd z0f<*=F#o{2A9`jbg;u!|^$@?p0;p)Isyc?;<&azsz5K8rnY2*p)%6T}{`(k*+E=tF zinM=JK@H#KKRi40eU}-w6FsKaIDdnt>~p{#@WW4upv=q!mv&naZHNTo<7jCk2n5pj z6Gh3f&uruv!~+z1nDk1M&;tY##3U)H(J*#C7vPmn(1j>t;8#B*?DjoN|69inl!gls z1`?f$+Bf0n?UDj@2Ucbt+AVGnRQ~GEbKKoTenx3lsGdT|aAF0!EDnp_`uZ63vnvf- zl|lfSJr9vU1`A8FnLfJW$_hp9|pAjJ@|k9rM0`zADk@QN>ZdWgZkb&HFW6Fe8tt3YtnJW*y5x@R{2 z9K555QR+h)QbSjF^4Cq`C<_9_kYk8FW}?;!RU!%YC4Ai{Z0Sn{I{-_Gc$sZV`*(Nhmp*g{HpdBTJNOWh85U8k$#yqK@Txjko*6~=! zF6v~{gONZKxm2hEuY@0H??9INmLW}a8jG#7zDfkvq6TPXHs}Rc?jjg0v@05R?T?}i z@Cwv*O^vOnP40u4zwF^F+>?e$wH4(mShZY(POZeimo)0_+-R~ zpu9Kou8?)(Cp$rYD$*t@Igu&x0aLwmr$xc@a0~I+Hk(7iXd(jc$MFiHwICoo0ZTC_ z2j!2UVJ8Y54tU4pio`p9-6Tf`EPWjDE;TjPeWx4Y+_?Txa^ffI0m&>N1-pqL4B56Yx_ovo8WSJqn@< zJm)alCMnSBAl}2#Rf5T_1F5D+^(Wdqf^b3>5>>Foa}9S7EqSoZidAV`n5loyi~E+< z%1F_TjVlU-t3;%5dz6{vaCxJoUmPpeDcyUW53(T*X>A0^%%gSV&rv!ikOh*H{zTC! zqH@==g-sgi?(AZ91tC$ZmWDvb!}%lez7FVoiB%sr*L?i$<$JH)#fj7kj=cF?vZC;v zSD+0+$GpK|2{-S4SSMe>CDxauT#3^mW9}(V-|Bz%$?q{C<=VY%XO9qzZ z|MT4?@769SyxRElTJS%Aa7snk$*w-JhiNr>WNOjc=m1Z%OJ#HTaJX##edqmukqMrT zh()N)!Z&HCh(a!CDJTF7D~5JGP&zE{jdpSZL$&LJSqL(kin@t4zPaI-sH;vX!QLYv z&HhmBux~|~${>O=rSiNxc>l=?5{E)0dhh)R;4M#4`-wHY8NE>7L@O-kGuUq}}fPqboB9HX&Q-g1m zQ0ubiIhW~+O7&xNYez?BhVx9Ux`u^v$J-}N3hJ~OgSzQ?Z*%M_*9#3%1|FVyb%b{6 zj8C;dep?QG)QdZNVT<-+YWoI_Lh8qrVRv5DMW~nW{$REo8zK|$ol&j4#&lx^Y8mLb%a~M6xio2brF>*q~W-+dEcx0H=bi3u6 zgo=}6n$$=_#;7$v_42|OJ%8OWCJleo#Ak#IbpA z^idaj&seDH$|*AmxRusXYICPje4pB+?$qr*s%`_Z*q~oHbD% zph%}j#UC`vP@OQ>|MYF)wy2t;mHM+B*QUwxFB2=D3B25srW;~2)g#K9^iZW~WxK}y zp-@!y$GQ1O6t_vO9XY-Aef?Rg&b=+Mxj9|5^c?3N?&Q3z21Gwk_V|%J`{K^@T<5wl zGbhVPQObu}$9no6wd{NS;p7hXQD$B`Wum`<7Bn>z=y=4XOEEjme|?&eLI}N*mN*bw z>H1sPAlgmKv zc!zSB1d(P{B~NfVZLVu`XLCkF&A;zZT7f!qoB%$=Py3)9!8%)C@IuQBZ7ed_2#A}d zyT^okh|U3|4l$7qnaJf@C>F%8VM;%tq7f+`DP5W&u&w|=_9j910Rc$Q7b|K(z3W=# zfZ9Pwp9Jel5O-MMskjUwa>y%<%Lt*^zxBCH%q-egtDk395*d|VODIqTDnRvLH}u$@ zA3i$C{f(Fmm?yjunczZx%ArGE;Xnvy znS{26n%|5t0#EUoTP4BQ0aqY|SC}kJK&GI{s1m7D`rg`g4vGCrFHquT`(R4jGVk%X*|Y)m`9%OnN$u@&SLJq;MzrT?Nf zs(d^$S2Fr{9jG7Q%$%?SfG#o4ef&p3!gb~}G(OPRnU5R{&l{C!;xXlWsD)hhLoU&+}@Sl#&fjHu&T zlEO%-eKTuG4sGxMbfj2LCy(r!?oSlV?ruKlMJu7TruCb42eMa_Pv=ckT@?Ceo=qSt z<E|SDww1UM6UzL2!j4sg&+$S@S?g-wSp)?r5d(ooC%-A8c;yn?fXWl zByCtAF`HmDDMKF%ndr3gP;UdYZmbp~%mR=o>cCwQ!h|HKT-KO7iGYGro2WGae8e4q zA-)Yi6ouZ&BR@ekaKNI%mpW0ik%F^u#uFWfVS>{>Y#Rz>m2cgGI&9Y_C$&x-4+VME z1)WDLbO(af!g`+l%*M_~1znPdwQm82Xey03*9oj?EBY@#B@T^y`RMMa;c6KK)*>PI zBh5F1E0pE$7V}gx#Jv!bW+>Y<7VIe~It1g&gjl`^iDZn>X)r{8mX+j0wJtdB@$9_q zDc(a6NN7AAahtgyNeIMfKM}$0rUSk~oM;b;VE~hbY*cKuITy{K9Jl>jT|%3Hl=E`TLI!x6QCdQH{byC+$ z`Hy%^Xb0D2wr`m56B+xyn|ZrLzwa0G^K62Y_*$s~6TF+zsPPcRVXBq(7g;>~x+A@} zB8)VBkvUkcUe^@W8l7~2IoH1H`w+upW;eSES$xZ(6=r4hF{Z#NX?rne)l!EhC%aje z>{>0JzWeXaPog*htVf<;r9vK=%vR-@AeaO+mCzL?z7doH5Y-j32{0pgf^e9%k%lKx z5ka6h0Z1IYVh$oQ6o7=_*^DyB{{GjJ=fO7Dq880RkMRMxh#GktyG}%04)%vlSdc4Y*n>T$E_Z z$uYnTZ8-CXf^zK0?Z{O~EK$;Qj3Pg^oW!E9PoMX<+r7A>CT?A+Qht-{j~@X(BpjTLEn)_+9j z!gisQ^9nB##kSOS4CyDE_$nPfN{RF43z@YjJ=LNnMom4bOnzjkJA>17FEueS>9$bg zKLJt+BBoD`tJ3?chlvGhmdCIeSB821FS1{0_mNUWQ*!a4;4#O(R^p&F2g4>CN5@5^ z|K|FFoy*saeT=#nxq8ccmm0vUO+sEfPdwf56@lb?px9sChV@Rg$@qB1q+OUbisghd z?m>l&Qev4fXqxA&n^r7RMu-^DjOgWlDmR{-3v=6sao~pb4lW>wZ7pjFd=T<7XRc;J zKr|-o`+#OBre{ZT;h{H%)-$y66Z&x*=tPhr@}{aTUfDVfmN(mtU@l-A($w@@nf2vrlkD;DgzW%|NK#EZ;qXlP3E~9_ z40P2XIR**G6gv*yZinm+>kVYp`KpT!IRV>r+SsSi=p{5udxqy`-@6=Keb(>{f2v{O z+%<{Ahk8vZ&8({dV}?l>j)DPHx`Vks|$l`0>w`(t||d+;=K zh&^Z9noHUl&dpyY=z&qC)jAo5*pYocSx}p|?+w*-2S#ZO54zldETqZRwh#QgaJ(p_v6Hfvt~Kl4IsNXG2%-_f^tsx4Gcv5<)IXFby8V}lYDej;W(7BuZj@I<)fN}@9bf>+A2#V zolY@kv7$>77|_NvMrxw72kUoV3!gs@`*|8c$gujP#XzN}uD+g<6Bjn9Mi%jGw{vQo z`e5O`-s=5E$t{%y3*Cz+qyhWksp|g4Hc`_(?+NjbAjg&Vy6ap&G6)~ng$)h+{&524 zUY%X`=V#T?gJ=ezh|mCxNXz57faUD?TFQEDI` zq%qcS7Edx=-<0B+$>0_E^fKkq#V-@D{H5RhsFqyU^-)X7(^qM?iJydVG>-t4Y-&3* z`B?HvMl#nU^UU!p>63XKBI*WK9%lnc490T_9>>Av&a_5{$nm2I*4?2(v}qZSlI$G) zu2ri1sc((m9sI;T=9RfxE%TNQ>4pGK%fBFP*(yF)6b8O?zg7t{VD2iHZ}OB`f3v)TEPd z-W9%C7!=1>#`%7=n)`uWkArSS&hgh`fnXn zGVV0x+j|mQcT+b#duS@85L}&f;9Zc@yx2SXQ#t?hLm$3J-zkkI$9&5O zY_071;}0J`30x&xKmXag-hBz% zJdyJSIpbb-`n1X{SM$vJet?xaa(N$f5Sl#oN$~90UfXF8-{C_Sw~GrTUQ9f2ar+K8 zgRbQLTc6+Yx)C#S*zJ^+Yk4>+FHf-yTSsP+O6CYXt;tG2xY`RxP7$?Ow%C)k9x`trKOU zi5f2oJ$9x4%U%Nz&;VRi?2B!F-h-Lo4Bk2&pw>jlKD$)yn~`@>a~>Nl{FDflKgJac zj0DCRktj;DJRZ;W>$WBFbhn9VUG1 z2TLX|ZMrai6Lq|FZSP-KtKw$$^XjklSENqA-_I$;Y7}kbN81nUJ>1U zaW)|>F5Sa-sg&@-@j^dsdVL|g))t|l-kgc9FY4tcY7~i*JMR{!<9F}3ou{v~RTvJj zQOq7QVfjV(C^D)X>lrU-ZF$lyTz@vI+1u$#8X3c3&vws{K+50-UsM-QzdEK%=5!-k zQb5KQxVi~t7^?Ec;np#jX$e<<4%8vyn}qd`@D4y7vJBfqxSf-Zs7I~N-dntcYY-qk0csTDAG9~ zb!H$LVCut|yWH9V-iz|P3;dUg{k#*Z4N?rObt}C5HpmLjz~RK_Zx@WanniTPga-hg zG2emx{M1vYPD#T?t%#F`4Yx_6b0!oJqW#dlF^^)O0Te`-cCjk#0M5aU%G9~%Xy@G_ zZV5P85u!LzWAG&6uhDxBcXgm7OB*!+d2DCMj9)9V&Oz+eVOp$n`UjERS}2TkKF zhoFWXIyntE1<5f;WHG(M+o(10lh4a#lZU^x7&Jajt&~{h#Tib zrY{qgN}{m`YW@O}wv%9p;V~ho-8caPeG^I<7tH=JGX20}_z1*qc~eoSUI;#WG#+7M z<{bn{4*s0j+Yhxb-lQpEkO{%$^?@Ejf&j)>2W8IBzwc=Tr2P=kLBu8@5K8l-Fuk$R zXx3@kYEo-J-4fPeoK!jns!a+`EN)mgi8i(CTxrq}*k66QK@ey}=MVxgflfH2*Iu;w z9-HIUAP@d6=}8KM`SsyZ(YMv*v?ybW9R8%nLAO!oMY903b@_wO)jV8Ajh@rLYHEM# z%1L0K&15=eY0s8Xvd6=Bu}j%DUE?(n;G&&eNOGpkRWwu+jBNmYQ8BM!*4NC~u}Gcg zs90N+&qe>qri##0z!ZlrOWms=xy-i7E2Bqs2dkDLWtDSwI2jIJ<94xGs+}F}V!LRqGCbEP>qY6#mQ&qLvq7^-Rg3aXy40D; zGo0BT6q##ZoJX2`xG0q6TxYwXVQ!8gdL&^i!pt~0szF~dnwVwI~&^ov)0KNGuF|k)!6uaNtu!Gu_5!eP1CLJN!A>_q5j8fRc?FJ zeBXGL2el`sg#ArL?ld%8&KNY@a~}^U8J#{Evx`Nk$hIu*Rb>^^Qr4~fso0D>O<2T; z)V^-Rh^l0T1oGkrv9=|@Ua}QnuMt5f5H?bR)@e(W9(WfC!QT?f`uee1*9TuGkrlXDYJTJa{<~!#g`f!k&eHYG<#YNiYT}N7T*4`40!8Ai95ia7*r=p(^zWDSBf43t$!XfqAf2*70PWk4&cqiz&=xjlcC!y|KsZKR^(G`rxvIQP|JKMR&~^|iBkP7CYE zZm_a+eM=nYi9e?`H+uix(X`~hQPBTaQu{N)TYKK@-+H_J8MSM~weCyzVE00^d2HFP zoNY^{*8hie|9`pNV(7ftlD;Qmm%R@dOsU*Ia|!>x?5~^h+ILNX@VaxfS^TqL&#Etb zRz>hVkKpt45mE3tAn$!Z!AH!q-yldY+wDkcmB^cDHUES1N{qzkELpQyR=G&&Rqq32 z-44I^g$_Bl))ChbDeH(-a?@3Fdo#D~#)l)%XnN?{i;&@-3DGqJ&)2Kh1k&*IBxvVR*YL19r?~lBhuh+oP?PwcKHT$qxts9wFT&3|`PN0gn$Yw9 zJBhPO#G_M8A=ZMOk9PWyY?0H$12GE`x6UpTx|jA)=bgwf_aD`Zi~=t=ojYoJLFa>_ z+;lk4&$tBr8$qcQeAo+ zT{4XijrSO}iYQRG3M@6~5YYW<{{7Sg<-pji*l7V~EeRNq7!o??$F4j*^cw)HP7|H{e_dGJ3iu&|0A&TBmL+abDW?}KyO+|HE}@cn=KEVQ2+cGTwUf13R$T637E&s$D`vQV;bkTMx@u{|FAm@1 za<4m=SiNtQ%ffrB?H_j#zW14Iu1oJ9qTZFSb|TF$KOE(9$6$)OV zhy+f*&UyEUn_uVa+L)g?CUF8qhV@hT4_|num3w{EkK1Oe`j3>515Z+-m0$K9@ff1! zzDmqkl%;O{V=I6Aqn}|G?=~M4=XC2RIyfx*RH8$2WxaLeWB=Xt3du*xCZ)L6#FR|F z7g!9*yD>s+S9?vme{;!e`QkMnViXhD$jY7(uax~`My`Yvaw*mBAE~?aM$Q1QuM;1f zIdC;BW-fCZT@J~KZx6R|P?r=L%F^q-SC)>sMGO=jf0e~xKYwlE z*dLtMwofom)}dmaT!jkWHRiWA+oh;-@oq5^_CC=5)+A%_HSxllDbe4f5+rr>+Bxrg zmn}CJu9pAfz30y5XSO~%v+-{onYH#$T_5W_^*Qh)b!v&0qdD7viF%IiNqxt;$5X3o zoYaM^U7L%7>XLh$_;Llqx7YtHkn4IT>lg8DZD5pq!+Lw)mkNrz3+1boxL5l}89e!} zkbh+=|G-D@1M_U&Ecx+h@G zwuaMd-|qM#{5UhFOX7vm*~Njqw{1@r@O72n>2Aw3P1t^z>cds|CS@XSbF=JdqSIw% zp}0%lU#prq{B?`@;%=FqV_%C8Ic-LaIBip<%!wO2?sl=r86j8FE%z$OR{Z^*x%$f1 zaSQI}HAlC;t9c*y=>34in$K&Lszdi}u=PF=rStWF5cTDOP_NP7V@(W+!B~=IFxDo9 z5M?(rmdx1KOxZ@3tfj@Z4k=UgyL&^V8WyzfkD-Si#op5Mh}n-Ao4 z?Qd84P`CPN;Q^H}d5nxgItl$U-@B`rkH@s|*Ydx$k2n89+bl{9&GmS?98y0aaeTj# zhz^BKYuc~0!!H^}-rNzd3->NTnM-)dHhIHtk(58X9rAdnZ#g9Vblz~JIsbMH5Z0Ls zmqxziTlkMD9C7bT_5RoT?@Q~0fqrTB8BP_Gj+H0IC#>d%S^i~xRF-Ou&Lwv5E?ELR zRrB+7YHp(cU)*(kr+S;T`f%ZMI)$Xh_Z^M0)?l

{h!xnbmb;bC$p4WIB&=zQEwm z7kFsrMw%N+Rw$HB+CT|`#Pr10+4Y0R|Kp8T7-|SqM?90tQ!@#089FE-!7Rv3%=4b< z@YVIn-ETw`i&mdFAMkxJ%3i3`qP9vU_q7=|?bCzF&N5u!!EJ-L77Vj0#I9(|Jz%d} zMgp#Q;8G=#ArFlibh$X^t$g2+(LF?~me!N75Kh4foO!)<_UcF#{RJ-h?Ik7Gms8i5 zomFP_Mr}J#GS(ckK4eP@a!M!)X*@r2=$o~tZsB4I!Kt=Nr=#&QCic{Xq5&ZqoiN(@ zv-xH4PL4d;!TvnRYsH6u8cX%;tj_LgzT#;x!A*l!lHv@EbPqN80Q=B1E6_8}%Xnf$ z&rRA#nkdLW*VbWCRR21c%i0}u8-cS%P?N=J-cGIjZJ9Wv$@tdUgGe5VK$M&c5u&Ac zKW9);ya)JH$7i zuvFtIbkBln5LY?P;ujX7$)jIq+pf5KD4F|5y$>0@<@>ZXd~nIQAwiE8!DD-P-e+0K+l|oOYFjH=|CJqm0C69JqLYOWO*|K z*>&2NZh^5i2s4NQ|qmDw0 zcCQIB44nAH$rh%_hwz&X_|Zv2N3HYsb2SDE?58SngctEcr+YmCaz)ftSEB~Wa_wy` zRV3G}JF4esb9LeSeOkAhM*(PgnCy@6FV0Iz7MGYeWFN7;`}xP+m4waJ!cqCJ{_Ibq zM23^QexB@DxJ3PgZoRB&$b{BVa|)%TlS@uT2Xf=jjm#L0Nmgr$wtZ}AYpbF$irnk# zpVwPS#xXDie>h002;}yTBKLEz|6C^ZlTmp=0w8(a(VZU>{5pML2`%4kCI2{uHMo6f zEc;$7r-s%YioO#z6=S4a*q*k)XJ_`CZ>mLZa9N*N_r`GVjjMjPzd5>stu-b^az#|Nk#aIK-H*gLUb1Tq2G1J<02m)p-F4khSq z3u#dw9z-3P)5sC-!_7a#kMYKJ!Loqi%Ca`Tn{16g5!<6i6yr^N6|7wlNEr)w$#hj2EtW(+1;X>S=47yaP4F%)o@wfvXiAs`WKi zdT zSW^MYLNu*PlW+u;nkQp=1Sj{Fo+@`X7|kA8;_V#0kBjwviqi=6*w_roIJ3Q)a_)Y< zPdjQbisPuEPOI5ZJ#4ygM>vGj3&Cy4bPIu zQuVpuT6e)j?5D*4{0I_$q+k=5`#MPs4W1l0MJX$VXAgeqhAOUvP@b9n6~~nwB4UdU zWiMRH)^n|m<}v%L@1xABx+_T~BZ=AGv+K3isDVQzqsgD#m?ANFJ?h1kjJ`N8;l7&< zS~-Q3th)l+f598<2ME-6#p`cE*zZ9qAy20K^R}gEXy*TX%ar7+K+8LE%8?i}BuLmK z7_naj*)flX{_h*fkd0S;c#Yla^`F3p1OT(NE1jf6LHxz6N9V3eL?@$Q-wRBYelofbs0iH6} zQSR%Q3Hlz| zqo6wDZjat1ZSftfumik?G*&?OIN8iv-jD22AKkYr2j~rUI<89N z+D@b0Er=?95%Hu0Rb9uZ_br|^H7L}g%t!u{$lpjlBQpIy8(Zj;c;#q`p{>PlKLs$^ zW#2&IS>_+n65Y-puWCg`TkqB8Gtae!HeZ@f!^#@fBr4&BSMqt~^2MEK#_^m3fri*> zf?gVRC|s`gqT(+b!|~)Sb)%e*F2+{38{bC{hJ3G{ND(vU;`8A8-^Fu}=x}LIq=lgQ z%|$6O`l0o?+hnXUit7TEX~xDGR5ULY<$6Hry+U~_H=5yeBsOj&aS5}+-Cxyp_TbI% zb$P>t;EqCnu(v#4&dV+0|hmPgjY5~Xf@Mgm-K)5arg7-D!GbG zj5&&KE;m>(X}_kFHIF$re9J5pjH2|pG?O4)|C?Nl-y=QAj!|#9!MxAg8s8s3)e`Ov zrWTxWYTcp@xy~i4)0e04a8vl@W9jA^pq&)BK|6Wkoex}<&h8`iDvuofCnd?tFu$*y zVpWLeSuq)s4|fqh7sC#e)k4q_u@U+E6h?0w#@Mca9U85A$oslo3vTskR2W65@qGbF zrBTjIvnn=tt}XB?r|);FfB-+vXXrJf?Xwfw*R7>=i{{u`k-HBSOyl*82xQL3wH{am50-MoPl+3hbch<|iS=JQ-jNicH5h zYHdP4dB2GdGcSE2wl%-QKWr6v=&-U)_ zlcg&Of$Q26%AF|eMvOt_ddFOr_RLG z%R6M^CM*{VE)&0|r@ma0%ZC=TIF~x1%b(qq^01!>^?dbZq~CCZh*R5z_@^#Z)-oph z?o|SJm$qnSe{kPZZSo+0=<0G;1hcwI)PAa}!oE8R1B0PM_&H}tkK-;n7H~UlR%-~w z_j*g+F1hO_8AAa-=O?8)N#lC`V`UzHT%Nm6cHzVhA4&E9T-JOzQtUWIT|<;`A@g*H(7$>&7+FQK10VzJXL%fw6bBItMDQ? zYi7;+;9NqbZOQ6L<)>fmK>1HkPEwSjBG5n{ZvLqJN%OkSp0)1+O2JLqHaoJAn-QvsKLgC*e$*CtxEH1)kLH+14{UR9?0|dXsPg$1=a%!Mwl-%Ggae7|(k4DH9`UrS( zQI%-_xg~tR0QXaXUc>Ga*Cl0)RpA<6!ajn4z0SLs@U7DDx{IU3B;IbSK=#(s#WX2i za~~olQ)P7u*^`wkA#-P^ZiXW9gP~*CT+>G!(F@sER>fCeh~jNr;B5<+rBglG)JvPE zSZp6tq6ACNjLd0H2S}NDYg^iF)uzt=0 z=8)@~ulQ5P5y-%*_v5&s;u^5|a{C0%M0A@i307b+b-7QyTD3>nx}<`Kx$TL1olm_} zGeB(m_yzbUrf-@MqK{r((dz4t2!&bhhLwgn3(vMpKRsGb2n2bg5*j20(hrOwHf28~ zlm~~&_=CjqBp!4RU&m^?g#@=-4xEiy6M@+xS#;1ziIt1tQT^1JqaQv5& zpjt&b-k~EvW0-%$JF~(N$uquzJ>9!;W+aYH0FR~Vr7WWCD)#q8lh%1930^MgUp|6H zSr)H%$PLE(K5YW+wAdxmDcr2p+(6O{8EwK6w(}K$Q?3AaJSOs7t|U4a6UC^)?E7xt{kpax6%B^V(6MfsqR~lxHaW`*@Fa2fHEBIcbW2bLp~{wnE>TE51+-an zADyp`&(Tf5GFtcFQg>AoK)-uVSYc+BukjHpw`~L4{G*(F%Y|&LzP;}zqtzc=oR$_| z$etiFu!VKqpX$TszL9)5$a+zAVu%WkKr(12ynfhmC!ruQ9x{q7m5xK6d0oT1^5~I{ zMvhgdU_jnb8tuA`9WyPw8pXy1dpH=FW(qt_eg%gFub>1`>zxvTHHi;lcwE9Sm+Q{5 zb$l8HR_c!iCY1S}Bg~p9dhN+3S!~)ZF9PFPQ&_bAzjpy1+=psaCX+c4h5fHLK4iG9 z&KED`s~yTA(MP$!yoBJL{h0QOL{aN$$8zpCFM*V#!!`CnueZ)zE!pvcrueqCcNo6i z;-LK`PuNtjr%enmQ-ahgYQ49PZ!X^F>GqT|7qE+IR5X*M^6B%kq0aSuN=3r2St(Fn z9wL9vW|~`wf9*BOg{A&46`fFOyN9Hmhlvv9on~jhvfo8jAN6D@2+OM>%92`MRy$l} z(-{3!k8i)$F)2W*FBpSBf-{`8ka?l zIcyLulA<7BF^PvZR zWgy){x^GIyetK*Gn>J>-*CSJDd!UPApz?$T$2N(n5!D^^6t`>mUaf>?kC$0`>k?(P z;$<-Z#lI6Y3s2S)bor@dRgrv?PuPms;4G(glY`^YjUDnbrYP7Od6W0a^iPs{ebqy4 zHwcwE^x?H3_A+#C0ZSx(lcnes+Pud*h!2BL{YM^uS6q71VAYrs)|nefL)$#si28AGhVZg!LUf&@)RcHhBfsne2w?m#Ue5oovEfXs?a zjmD9`^1KvQqp5Q_DmTODz|uvzv^u`|YNiJUa~z(`Zwr3(@f5DOK*2r*q&ivg0tA5a z6$I8hDoI&-Gq!?#SY)ctQA|RUGE6iD4rYht5j2~c_OoKsy9u5^Pn<7DiAMQ8ZTmqVq$ZdvZSyjX$DU)YU|k%BZ4w)v-nb#br$oAbNUw<@pPx z2$>80ZR!$PLE;*EJF6+8ymvvTR8#Ei)N{Ot+5r^?4m3!Iw7w7rTH${Wh`ql#tc-29 zNc?j45ZiF&cbXw_{>81_{wOQDg+`C5d=iy!+rCNe=f8+Y@>~>J-xxM7{HxO%hHY(W zZDm+8qOVjAmG@vyrRzy!HDEl(pH&l+<5DCq0n7I4ko zF=1Mm<);{PRYT!jwW*6HVdBxU0vd}M=*4vKN@=<(tFHv>YW-Q)#YwmmvPv;`4~lE*L0(gM!E{_PU7sG1r_$#&~IYAZGq33fzV$E1B@&)u!y8mjS4jMKf;Yb zFa>#eR;AYGSIKC4r$OzDd$+qr%-?J?NAsGaZ4&gWJT!X*A(&kj3{1}r59Tg<`9g=e zXl|t9o`wR@#9bo3-u6?OM`iU;|M%URorua3d~*Q|b&A5~FOrOdYRKh>ZTI>i5tF5JtJ{j6l?-XA<<&FAFC5+cZ{y`!#e!Q%)D9 zbA3IwarWxz7V;^cmjj*0H9pNFlB(}Z39^orovI_Fpul0Syx1~D(ZTs?c1*8-XX7Z@ z{{J0Tg7XTgTRv+&io@;x8stOjId+?g zb8Yu0s{My-cEAa-b^+Ae6X0pXdC;3@uYNZ!8udd{^k?AnH1hh>8Cl0dhL}7EexcaH zvQp%FR^EPTt;Wm!Dao^Ya!)6ZXuqKVH(>`2C>unZ_FwkVDH_FyVPP(jNwkL3KoxfF zQJf8*E21aMsr!`EH{gwh`xYIAI=h@m(rw}2?k;rYtQ3-}o5JVlRHsF;-cg=|pvC{V zt%AJ{Pdcy~+6ALaKokmEf(}RWy#DMaG+mOs{Ew(560z!SWm|;LSy+1#2IU8(;7ET= z$O%8u*=Sdha$emtg`N#2nFYMqO^cyKp@b7{hB<)2b;dP+BwA}j&`^a@t0i`aZ08G4 z?QY8DG3O+))4IS!X^pn=j5lkK#>?HlHdi^BF8cA^lMK2@L($9a_u%yOg9mmfTc?~<(1H#e0ek!%{oqFMZPeB&`;b8N9| z(-p}vWpDl92KhP1W|D3Dd^ABLX7z_TntzjUV@3Pw?b|v|XT?qe^gJu%3sp`8hL!8I z1q(xCvDq|FH-8p$f!F+vVa{jJCgAip_2pMEj{CxO{#me~-HfClP|Hmrv^uLJ&P7YE zi$qZ6Y+pPs4&b4IF5*GpVzF(}c8oN9QXn5Q(8v*G9ffwor_xx_%lC2ib2i&RHLw0z zYY|%}SdWTD@or{JNf7W(_hUq@U-tIISw&|8diafqgQahPOLkDLXoyj@@= z4B50OI;__gUVaed10pRwf_WJj*HbO&zmn4@7Y0L%&o^ZP%9Auo|tuwf>HTG!pP}8 zpk1bdWlaSTJ;yUz%uxd+Gr4T(!({=Wc)F(6 z1N5w7oFAmH!?N=Ohe@!Gf(VM$)=`d738&pS8YL%9S1f&S|BnBD$Ld5&aH-JtI$flz zp__&Pqsl-{-AVaWwCGnR3Qom(VmMWv;c||&-mM!K8x+UIQJT^U>17)8y&&*eN>8&p z%j(1TH@%-i`86`w)yb)j$Xgr)}$LMcBaEAS4` z4Y@HqVa}!gl8Z>6(^BGg_8(b1K|r9ns|z?pkTK})2vlosWHR?2R~-raW=02Ju=@X; zQEXMRc>!^&B5#2lJ7yJtIu80;JeY+P8W?^&G<-T zX@2t%PxhA_8D?PK#foRST&-?g2urj8zmv7wHU`(-&)dw+u0Mz5n}ynI*FGdHML!31 zeySzvm;sg-vUV(A@mHg)vFX$|KXStN!>v8wjyqa*AOc*&`nk9MGBel^eh7Ym4s}ty z-5k~2hIinp&zEu9-i3K zT{=$r{Nb&cw%iE3#K3t9i|uk>5t$m@Ir$qRz(CWqxX9HMY<>Vch<2EQs*cIB&g-)9 zT>S|kZ1w)HxtIR=Te({2-$a#~;;!a|(%rm?3fTBqx)&%)*?{L8)#vt`IsXXm!!_BH zZ_}BQq00GrkczfHiv$TZ&e2v*M|i5Z5&i=3iS@Zuf`wjeyN|*ZC!G^{5dtZ>Q&&2d zxUXk@8gvpuB&k{*>$foiUCM86HX7@>&_rMc&bB!WBZ*=)b;EbAs%LOLHAeJ^g4S9+ zjuU&S57{iYbuhp!n-lgDpb8DTo5gxpCq_WLt)}#trv51`gptM1+^QJ8Qg5YHO-30x zqOBoGLu;_EB_wmfkVqM&(7cGT9L>KAyy;C2uORq1GFc2wS-bOKOcEPe?xS-eE>7EX zvKVs<={Mh!!O^XVQ$l>yb!ZvS3{0o4CLCDOFwF7RNXN%9*ykGfY*UgM(lW-t)2-Tq zA0R=|uA+%q?{(-Nh!PvzDM6a&5BK%G61%Y2&DRpc52XLlygP}k!M$JDekxXOF3t|% zOER!d1Jz#Y+k!pam~7R<2C}9wqi=N&thet&(TnMvb6^&;O=zm7FsJQ_I{Kp$mEIGgkAU>J)Y*SR>5r7k3Da7Xx}K_0n(ZVK_@u) zRu@pwIScqxU^OQWgMQPfJxvyMIvjj)w$+;a=j7tZJ7=(h=XYKCwzG)~fjjwjjfj_J zI1OYN3Ib!WOQSjz%^#PfQ+OoW>RDUrruP3_Z>*&}zsA+IAMyhVc{5xkQE=XcY&_s4FK5VY0AKqVF(xWDO_0+CT#SHkVG|+FZ7^2u9QIxlI zB_`G2^^W_vFK1tGoLO5f!E9f`4Jp-!#DZd;Ce)dI&0A%lAqz15Ze*?(VG56Y$z*z{ z2cn?`1XiQB4`A=ZCnsxI#{FH6PO`kPe=e#ud~tKR#Y% zE(E{&?3rzcM<8pmB2(~seO6i-X=<;EjW_e8dia!!1>3pN(o#{d7`72LtD$W$Gecrwo63PKOu(;ybqh{VnL;D)9;|&_T#l z1p#&1_(r#az|if|@vCovB1C@*DeCP;T3JZAR$`GU6B{G6*&}mNubzzE{Fv5QAv_C_ znN>&#qbqm%mnB+)`GmPX!sA<}uOUUMEIPSh@x|&#K19;8(r8n6rjM9)`N}sAKUIi1 z_nwI*&Vr7)mES+$ibCL?CHtfOZl1pKDQ5a)?}iDK{LTq+UtPNfg_19?$ljR&)#DT3 z$y|XTV+gnhKVX@t52{2);B_{QUJki6n^P4X8VLVf?p~ZvuCvdJG!C07klM3l@_Ose zi@rC`GWe~yh&Mqz2s82f>B|*ZLFnho-G>wIJ6>YwFs0>8r+P3Qq$qc@EDS*LsEZ=| zxW`tdjF7;Kl&+liD?TICnI=$sSEC{lf}sLN!nK9jfqV@xWDf(hty5cg5h(&?JwTJ% zjN$=p&pDJu`v6LZxJc&UVsS9fwU+x5$6Qfmt~AEkyqnthbf8s)^P zbTepDh#voKDb!Wxk{3sI@X&Z(Zw(E<`H@&1OhY-hx^f!One7xI>N-zP)?ko!oY<`a@(z3WgCRz&INp(RW8p$5-*5w8_Gi2Kyb5DbJ zf-2VxP3*n!E&3QM2=zS5CkxWy#`w(9^m53gpgLLZt7O^ifbl*|5#B!0jO%}~mXt1b zu7G+^R~$;?sJs8sNR=G8m$aOsb0N0(VRmBbjay_ZXWY)tx!~)%Sx^^*pr{AkW zctn7p2o9U4IDkdqJT3KmvrG96=?9&yg7YpfYM%>~q&NL4|<=DtH@8D5+3zmI5Qm@HmC) zP_BEJ5V8rsQ}*u7dCZF9aW3O}S#v3myNloNz5^PO6tbzPP9I4vj6^v;(2gu z$&Il=^*sf#ffiu6oO5DF1S&p+CjP(y|R4cb0UP)_g7I)MFe{L#wwC zGc*kG9hB=xJtYb=XY(=?q!bhvwom`Qy1Y6PX44HH)G6c{%Xs?0PF&A&JH2}qH!hyqbz zZ4;m?rn?ChVl4e;_W(01aAtgisGVl?qWgOJ9N5cLc^&F{9;9xSDCjI=b<$iPT$tfQtHFS zygz6sCWfR3K971xE-UW-68&5SoDDN^(08a>>5EnCwN+Yn3x@3Pg#QVQ`@Lc+vTsX~ zKx6b|_5CVcFr+A%5bGX7EKmJH#ibWQFR;!MVrT_Oo*CnZS6bzz9pHj8ss9(n{d#M3 zNjE_PyW8v7b4WH8qGc8_okb#`1I3NbGU{#be-;&hfh61pGeXd{c^?iw-bd_M|G$0iXexa-*!?s zp z3(D4={<~(bD(pyIN%*6iJcAgRws6hrp*`jhL51}CB!9HvM7X*XvPxziLxd;`_dx-R zEZd3z{#JdV4t!X9=T{2WB@jiW>XLsPA__8|vr>zwwkEZE|WB zK}@89Q1f-TI?1U~RR)b<7s!p072Kj%;)^(QgHwIKfc5^)`@FOGef4DPKH_}${WwV6 zsY9XZZw)1S{h_qQrY(FS1&IiOP&0Ty+Ri1wKlrE+C9Jq3AU=LG<@jMf zby*jccScL`#SnpYe3Os^M2v6zo@WLSpGXS&bEx}j3{}-EQM^=9-h`?!QC_Y)juj~; z$m~}EfWBCSn5)nngpKDHBypStZ9yW5k#$-zD{L_8|8FM1bB5Gpfr26)I@i|r`Otdo zXTUH#$e9C5+al#JqcW&HW3v$056{?7GdOh`EHvl%5P?t0GCZxkO07N8NV3atoQ*k{ zH}uJWH5^(Q_i-&+b8XkVFEOBaD%`e%#DidobN;Bv)q!zIw{*k53YTT^_=d3XEL<$t zqJBg33u!r9GMX2IMo!`HPsICXoTuRb|Bt;qXNXZhvB&8j;JXS-!)Yu9_zr{&w>9SR zj3H8a3Zj2i71!9b+4$-YaZGcls$+?8es4ca4^UJ~C7b|4s=`y#(?J3rO;WF~?yzi_ zCy>VUow=ub@z*KC)dogiIDZ9mlgbgNyEZPYU{CC^LNOqZP;TJ>U!lE8vHDeSo>lb@fwUOGJN6s42NKc4-{1I{t$c%0yfw-c$F z#tIy1YQV(~-7zbQ@~P$e^Qsk6BQep*5jZ6)gCz^OPQTg?DVg@SK-#@1cQk}Zy6d}) zqpToC$2XO(XqB3^YoxmF)zGD})e0U&2uc*0x?OC0Nk<7yL-C`Ze`hX6edYz>+h3P% z!zU%D>)c=T9(q43O&WmsP}D>BNHGx%`q@tEVt^~xHr_se z3$LG(w4Ut5v_8di3u$wEz7$AI0%2FC_+<%t+6VF3*<_EHIa9#8GlI%?Sb({b;pXjA zuoUP2;8FIpAY`5i_SSm#vTQe)Ox%TXyS>ARECR5jiYK}59@V4O!g&R+xI_zpo*V5tdY@u zi9T;n$7a>OX-{zkTS%{Pc>bl(DDsKTDYHF))f60QMQ{#NfpZyM)WzTLBPued%^!l7 ze^*KB&0|Q5ePB^^WNdTgh=hyq{wzomD$uo`YSB6`fBq_56Bkbk6{mPKJ0$rGl8R;o z`74%>S(|GElciVI>O5q%YX(CES4z{0(>c7D6O;nz7)Cj|Pac5_19!ka^Rl3kxP~bt zaUuI|dRbSkJO44s7{XZ3-?nF{7a_(DCVTi+kMltvO}JNnc>#KFT5Xa`l&%%I09Lh$ zy+KvudFd7qx(l|an!Iox5EYAn3Y8g*LZt(3V$KOLLV8E7$q#9my!Geu#g#C1`;RFS zkE|aajOu?UEXc_QWuYgLp0ARVZ+wg@0m8n^k>SBn`$$WbbJha|GT@&iQ3c`JQzEo6 z;*f0K-(lzoNk6~L_U#(E`ahrDzj%wN@f1StgB%YW&08MsKA&elg&zZmDgc(e5=@7E z$m7PvPPQy^$rK7UXyY9>;IuX@S>OeN4Jj_eA$O_SJ71>kg78DcM8{1YkY&!VS}+zJ z(wn5#<>+bAW#?TP;NIaCy<8{|0u-|)%UMI4c~T&oW=#&5;}pN*#o(Z<5)Th!(dtlV zIFfGpY*(SxR;f+o2i^Rw(rbSch61I-U1K_zue1i87F01-p&dGZZS9(@L1YbC_+Eu} ztru6&tfqc@^Q(@>kUNRUh#|P!*#dwYYbwcQS>Xaq9hzeT(uo+B=N*Y|fm_FoDc9KDohn zZ_!`(6^9&A*!@oJVR3)=MD0f(4W-YW^L_$-BojtpF;pLJq5q$V6tDJ)S{Kd}ytw-m zW{{>sJacVz-3r=YrcSA{H6F^@z3&WB8P#q58Ny;uBf~r!JG^qANLy|jwg?rrY0 z(QdcA^UdY@)b2e)L2m;HUo(s<>ULSoUWuMIc~I@>FJ)b3(l9`l%6Q1Xl!fhi%X1}& zP5VQIbMnPW-4kYNJiBe{L?=msnU^Z}Ef~7}ilJ|9x6rQo5P)+4RMdy#q#ZXFy}U5D z1A!I+6`?w$Gh+Ao@fDR3=c}536Qbmr-%hW7^fh>9))GBb%J#HRDGHx^qGgxp5?6s* z8#8XcO+QgfV{x+n71gwUREcy2Elv4f;ntE}Y%Jq*9lSwt7Rp=dw_M;_=1UMa`Z#+d z)cn9Y&i83yW#c>A+oN7DEW)eS^9F~6T6{{v~hqAvUSvlo<2?>iK9?inIoo8$!HrY`sU7|=3|p2 zc`d>yg>IVEq!zwrQQq;LTKcImCqq4zvy@(Mb>66NQLWB1|4!|+Rg-!)5xgVy zeHXqFlBAeS$tTEJ<#4~fk8HR9cTV!8_~HaQRbirguSR?KyTSt|;fw5%KC3~OoT4(W zCl^dd>nlf3dI~jE@KKWR&u~b%dYkY}x~Ic2D}$2x((>0qc6N3!FBf`{56OjEzYsmS zmR6`}9-4fEf3t1bU-qZf0@kSu@9c&KshPl*JWy;m;%;kpn&%@L3G-oQf# z@A%@q>qviHNu$v82BKhMu28hRmm}e6fuP*g$|Vh8Zh4b0L{W|$oT5J_eIJ>f;LZxE z!Vf&!D<5DQGa-`k>5ENhlez1ULsdcpTW7(?d%Z$Xb4<&h)Ztg0Y_*qjGV@s@thQ3{m&G{42}+F$Rb{PnuMO#R}6V*`B?Fc&p+x)vJ$ zT#z@NjeBVmUbt*_6zVxQN)o@uF7K=Hcd9VtO26#Iw9a#ZkZ*j!N{y>(stkar|55Tv z?H`~0C%;4-2PV0b&!_l!_~rv0J?mbxAeD1zZQO2Lx&=2i;o2$hl_S3NZeJ}BQ6W(0;E*lp ziAkMaxTJZFo*;sEog}b)v_t+jO54;Zh#qRtBBh^7DPN@$u4%3enFX`a$zwWvj;JT| z9*g@=9}F!HF+ZLwaoRB?1ma*6wJ*AbH?`{1St-fC_+`5LmVO!^IzHn0kd&^@nFWvUonjzcuiZ80ekoe536c5IJew0`okU5nhflK^0m4r)k@dj{(g3gq7lMM{uP!w@3eecohSy{ zjFWOioOkMqX}!_%e2^P~W|-md`m-O)*VuWkGw#_7BphX9y!LOcyJxDT&q}lJkLn+* zr^w|6-42+U`fs*KvZg{k*LRksTbb10;aK!=o%_F_I7QXpYo!?vj}~Y~YWMt2h>x>` zv40h74?DzBN1ZoK2veQxm-g#Ky53jak;9iy1|75+TMW8BD{)jp=AhKJ+HBSix$GiI z!~WV@#QSXHowWI%ai-^c%CqC~-!AFodpRlX^ZoSpK*{4bRl({KwQuRFXx2ZBp*iL@ zQPn7)Z$Tz zja*CycWTdd3>UR{A$NLD_PLKv{8jp6v71|@e0yK+Lr+KDN9*<>vq59?o@=j|l&WRa zZ8v_B?k9fNV|Y`n zZp2OQI;$Q@B-Otc+V@d~yi}v~Yrdc(=W)6(zgrzuPJP=SJLcff%l)ZT{pgt4HzChB zhwnk9=a0Pi3F;3GjG2v@ot0>vNw&+&&(`nW0ioxf-ONL(tMr0U>2 z%YWS(@-TGMBF)=U*oVkY`9b|p`x_!A9`ICs`E}%wu^6&`He{i7IC-O1|CqOi@T?;# z$lrB1%T6YSiXOQn*LIW{x=S+K?jPL5-X$JmFjl7oCu;vu3)w7_%=$rME8*n)T;*C- zC0bT*Pzg~*>v^uGzghSe-?58lTdzdDV&A#+x7HP;z8Z?ft9*6HKfEuvY!kh210t$@ zT(B=2ab)bMlf**DWPawS0?%Z6_~s;T4Ih%9E|XvYgFb!rSP8|E)sqSY>ubDYW$KOQ zOD7-&qB@Egn0Hcb^mjyyg#o*$e(b`T`VS{g!q z*dtKl&<>FiMtqDYFD>po{nh11P|3ZCbBgpAL+VSE)jzS&zgoYx+nTKKnEzRuubl4b zw8?IhgBMvR^#40yD8%m|fm-PQcct=FfpGL+BU26Id_33q{A!E)CfpCtu6#+?6c;lc zZ91diN_HJH5rse;e@b~>`H!?Sq;~&m4gG&7yHD@EWBy9&@;`SgI;A@1k}eGAJs+=6 z)Zc1wyK%TuWI!~*&p+Qn$0O9f+SnCtjO-Kv9dZ`Iju+we5d9p3vaw?Yl zpiJlb!}Y54`IFh9}WYZSTYF{6(*XxYbB@ZdiT8$bUG22u!SP!0VI? zRuxHdk7&!CA6ZXK9PZKHZfm%Lw=1~k66EluN%20p75+l$paYSK-}#W=6cj4E2Fd*9(;mvPZ!7by>*+xu}-Ta|`8)yC2nOc2B@ z$+@=9+bLQ>p^}vLZ`jeFM-u|hAK3Q0%vZ=Q)>_X!%T?ZRJKf%pCCaE;t74-9Q>||E zKE7Ds(!P?A=!mADZO!wMB%a1x+0DzPnGy5^DfZiXaV4R;#+yl&ab5gz;9h5Sbv0EM zqV+)>Yd}S_X-ZcgM3uB3Cg3ee!u(@B@ftrQ17;LNhUz@WY-C2{C>7>^uj;>z#ArT= z^DV7*ODA2SU)obdKj1FSxNO_Ju-+`=7AMZ%DINB_S)ZL&?P|X|J`PYgIB@@dt5fF*cv#OSPA4W4&tM1(dyfl)*TmwQ@*yMt+Co zzuqm$?JX^YR)rlpPTw4_NinRZOFG}YRxkd%B$vbG9ThN{XVvC&;YSzV`M=(K2Q2Z- z@{KE1cwb(bJH-%^72vgWR?c@Sqy$)IZEvpsK0kItQh;{r)@gG={&&=}$F!-Bn*>D_ zIq&;I^*J+-=F`Hh3b&;4FAp+B$8DrcA#W!76~XI@WW(6NWJ<}u!LmiNq?`JYZtWT8 zD>8$K30(z`GOP+(R_dI?Tcdx870zw3qQPI8tUWSSoZo%=Kv}e_uDW#uk8pmx-D6sA zp!6ka&9;QF%pVE`eA6Oc{T2+A*B0k4$tYD~ zyT3pmL=GITY;enrmEJ0&W)jfa?0d^$E%lEmg~{6j96}$cIQTumM?nTScposMJUl!s zY9zFD5>Zj3`@qI{8*=9Wrt;^~)8WpK6&F)J(rB|yd$qN%b8<)IpL-V^RS-9r>~=l8 zH-Uu=`E4Vw8{VcUC{50l$T+p}>WPTe9UiZZOQfw7Nk;#QhNh0!n`aX8y5`RH*1@2b zK3oo^(emOIKOXGzPRBf+0Tl+#d38LS zpbg&L+54>T#k)kB`q%uo6Dh8q!kZ2Cs(z@ip8OD2*yPv|RRL~wbjW>s)Jo%AZ%c7x z_H2LrIAp)qR76*C+15iXB)`g@iD-CYD(Z-G*?t z-~FrGh9=Bf^a<`sk|vhN$^>}9r5%Md^@LJ)`*l^#VpBEV%Ow25qvfWDSxh4&Ub1i{ z&z8K{1qFb2_FhS7jgRxoGAXm(ITV_lmr`!m^GMG1W5CVu%P56iZblDXP!Gy$@q1 zA&v;9i!bu|T)1`H6Nek)zK#{Si)xw;@>sTZd(c-TjOpTsI8iRQoojqX)2Y1 zKCJ=6jOfk=BA+C(We06#y2NAY46>;NnRl{BS!s@SKD(*#^~T;A3x>zGCmXn`ikZ|B z2_|!iQ3Ab-aY%i;0{QSc_TQnQy$z1)d#zPeDvdGfkLJ1t5(dU!8M@MWX21D3>wVFg zZ1Aq*uhz>oMMB0I*wsnPLf5C(x%hh-aX_)mQWQIG;*%GvAQ48?YCO@?`w z6zg03P>7`LA#3O4$1GPylSxK$jI_EK@sZ_*+!;7R(;aj*sh{Mp9))qu(vO2cdu-H`k=dfdNv>zQ8LpN)!Nsp4A9e z7f;VjH>D4abMIYZBoFZpUuw=}Jo7VbUwB-kUYiQ?QZ^NjRf-P;k-XcqeO{0Nh^40h zJEmjjWF$BZkdkU>4x9(@wAVyD=RAz!BYPTt%A^ zk5y_Z9A`|6{&OH;pf6%@1@xqrYsuBfPr#)e+hKBvg~UK#>>!gT!GXt+wD3ep#O0wb zmmeJuDBfv3gp6p1X4-A)B5vnWVZ=#u4#qGCXj(IW?Y{De8^XxZ5h#zK@!?FVsqs+x z?3mOXc(x5p9R!-h?f!wpRN!&~u~)wzBFv{Zz}^*q;pRT)??Vyy@2$!oDoKmL`bE*v za5E8s5$3w*wGS+p`U=vxgTkUKS+o;ULI29HE^dKG8C7*%pSsxEgP`CIqmUB0TLHw4zHIP70|qa+sG z9m*FD3#e=d`P3*W5O24xG`(LzKTa3nS?J0T?7SQAulN|B3zZ26@LZ31Kjvs%HRXNT z>f>d5*Ia73j%FtDI#VR|5B3q(MB2{Bmu!{V?wrM1rlML}B*{NgL+f|(IHrEF>oNy7 z>+EnGJ!@b;r2kjDMj)h;n{)_bRI8$ug4{En)n3xk#q=BgK}1{CG=WnbV1C|QlZ_Gt zn2TzqP1zy!c*g3tv=&=xy-`*l^L4$fP42g6*{LBsQVTePu%?3@jZLDt`LV z6J4SzK!6)uw#hGFbNvdLbxdVLnQ3)s`x6efck^Wc3$BgSE-j8v5>^Y%#Rm{ zomMxX&6}+@%iNC4p@C=YevPhqwl>eeUhEtKDXoBU)K=M}njd=F%ncU7lo9Dl!E}wv zkQkI>%N(t%TQMD6HKgD~xjo;{T{4X|4c^!4`R7?XG_PDSLJ61ajAOWMTvL2<(BML3 zK?0gM((xmfqDlq|2QHAvs+Th`{_(_pxd`@_t~jLXLs$*%J;Di8AFAev%Svl5bOOh(d&JR#EVJ_<`Eu}a2- z=WFgl^FutUpD#V^8>y{@L16POzjUKC%)DG2C^K#hX|Q=!+OSC{FM$qFS;---4We^# z-}Te=sixud2TX*DCqf?9>$|b;r}c_APbSeP@^r^l3_s7eRKaHoOxbZ zcctie#r?@a#qy*>iO($eJ1J|s1RvF>X1DJtK@@K#%cB`^BeQiTunZ_;=)h9|;2*Pp zdYq zP*w8E^dC++A~6v1z|PhiMJN#p@U9v~V&x3> zJ^!IfrI)rCKgLa)h#qyFY@1Mpz{fn-OJKb3P}%Hu$Zs=#bY*go=7G#!lt!Q~A2fLWqa|S6y<>Q@b!M4_E?%)&)dOjjuxC9h5@lBh;NsPJ2m}OP0u7=V3JtlK$ufy3w#2wkS&^ASp@Q7Xpp}xf+6D z0imXo-r34=E$NnT1*!{dHC45+N!#~J(-$=GC#OSVVrtvB(S^la4qA#cZPG+h*P%r)lEjL%eK#7ilfnv23Fc3GlHzKqW7Qn=#YZZl?N%BIq+W z3S~V${+W-;La8FDZsD}enj*R)E_fMoM0rwa^GAoGHL1OE!Hs||tn2|4)q2;j41tLO15Nmz9cyH1TN)oNpNFO2 z=lk@X6tr3|7bRLGTWseE!0^(!d)R9SU^tbr8ZZ4IC0)C-6Vj-^%DADY?d<6*GjYK5 zxV1b%IOgyYw7Q{F&>U}Qy3`T(9YWFooXqok>3X=6z|@&=N9Vjp4B3+`4${bee)Meu z!da})a1~je?=(AF6cywAqlm_sdrDOO>&u|a8&hzxuB3ldnT7C!X3C3mcKv?M)JBX~ z>UW;W#e{H8)I+G+oYGG>xmjTHNRbKJ+ZZy_23ZJEWXLwUdlUvBif9mUY8DTMp$#WO z=>RC1k3sr19Xk{A}qOu>DWQ>I4Ty&K;ci5Vt9%QV4AFD zE0MX3X1D@5%j9aEE8W8FhE5iRa>xDMS{u2YV{3xywVRL`aw;#4Ut}d~!-q|Ea zylZb`4r1J;rC&JGBmw0Ec-L(Z?p3fo>{#E%2TzqN!p;a{7GiqhmEj7mxyr5$YaS|>XMV7 z5D8>o!>Ut&CdC5@twUzqR+GstEQ+aGh8U&o$`%HO#54_dfJk)PLo{p`oJEYXWgRoD z0{U!w`4a9whq&fQguL-xaqEUxR5Hs-)c7YgNWcID{2l2x$G@?55zn}&bVG0SJe77w zW>C;u&D*REkJsOQgSC1;1+S$il;G?YolNzaa%;l=rwMkT*ROA!#=Oey^c)-J>#if- zf8%Pw)P^5fQ=wI@rAE#5kbX3yERec(&*@C@Qo#0TvY6(1Tq~cn3@3Yg?U9&)7)xZO z`InQOjpLh=)|nUsZ&rRkv!LBoW`1>+8D9@zMBSJvr{VW8Dr(!7c260^Y;_+_y;KOM zo08yUoHI?!ODd>~Tw*K|tOIC@s8X_~DE`@7b)8~72m>p5^i;ovG<63;l4xP21;+g4%ME zJp*K!j~tP@3H<4<51RYPVFEbPPqhfSOR;Iy4BIW8CRyc()LKMp!YsR^^ij(b3WYCTA*qdM=;I2P&c!i5Nr-Xg!xaP0~|bi#fRPz0GB zWKFCRf(mNp8_W~IgN^K9&$U?!C?qBIUykESTL`c@N8kd|b5f`MnanX&2oWN{C`nDc z&L?Ut#~l~~vZMyl8qA%SEzf7YmXehjM_|RyEcqRJf^9|g`J^5PO>2Nk7uY$dQ}NFc z>wSq7_ChvYA!cGpRVrAAgfm$@p@gjdXr72h*0}V#A{RD3Y>J?&4ZT( z2_{;jqne+7fY|~IOh2&ovuYny{p8GTPO3q&Bw)`A)Yhy+^ZdtaAhDKS){>)0x?4A;Fv)CI{q zO>~8J+Z8$OZN{PS2n!b>f_ZD6r0qf@VK+eK8vU2g7mluM{3$|+EAzDmln9;zO3*tQ z_p~MdFM)MKyqL8;Spu~ERm6vMXFS5^u{dRD*Qm%mpf2gZ1?ZLsrxzAl97ZVsptlrf z&YcVuE193cQr6`=MsX}b;8m@C;jsfa{A3o@Ph}v7GmS>my79@z$NGBLa(PYJMtA6vV zDQo*p0j=c(D!{SfO|)j*WWbzM0Q6H7MyQ5&|6~y4-Kx0jJC1cISu$w8zBag(y%4g~ zUe*rKLDIgZrC7?ode-@LoqDbaX)6j=(iN-<^&6`B`eQG3rS1a^0K2i|nQ zLDTx%@69XI6|v|moI6G~+no!#Qijh;K;6`^*K=78bvEy5gpI`$G|pkEp+p*$!YTHW z^Ef%v`sNC|j&J3&&8*TQa#wG&Q4-5L3lUwwF3I&(mn>N=?P@-t&;GLCeTOfZ-m zdqfn8Q;-$PzUGqp>vI{7w2-_67>>rE%SkJ76h=3i@=H|}N9}<%;X{oAdhFc9TpXcN zoXp190Q0RYDm_ZWPxw&((1tsBcczGU5W@L6ZLJnpSyGesj>Tqj<;Ap59t?l zGX_O=Fxa3k>{ShSpd8P8faxm?1y4pD9I-MrSkivPpE#*pLeZ)<eW6rV1L~6`S~ms(wD;3oXJA)bB_MJ>ZQ|S0z-qj?b*Zwk2tu-tpD&If}8_o z+($^9Qdko$-AP2DqbmTy_`m$uwZ#@)7C*Q1uESBynD7_6JXLGKr;9)sjceT~q-BDR z$n<$FIil_2l;kvb8K$bH-R+#u+RYCOppuuXaMenxLOoF}a7Ln4aTu@+=LY+jejMDJmL>9N*Wxu3cMYf6dF*(?#G2+7Zba268yx!q^TycE zDw{Oc60oFT3YZp@E2(Uu;hC)OhX|$H-LVRO^WexRNflCN+#8k8j~<=4vc7zAOy_!h zcR~QW<}S&N`W(iKd$6gdN5&`mK$k$NI>WpPQzW*{MCI=L z(&(%IiW^^4Ph+iQJ-KNMikq%EIpkQ%-W}dephjMmM%5SU zsoUt5jn&T3CI*n78TH+U-&B$Y!923%PQ z0bq6!O8+o?t#Uj$ZArlZO8d8B!wI9$0Ozh8q$UZ1F0|F6R!$8Ds)Q2Qlg0tmqSjNc z<{MY*)HB?!)#i@J4{+LyPasnl&8hh26^(-9c0%LtMoG0OyEgiOjFtWg<-Kr;S7jWW z$gkm?gH>6BDkYwwpBT)VC(Xf@^pR!wxQM}r)rJ8Q)FYYTdUA!a?gmCtr@<%z{rh`3 z@_?MJ&13u!Ykkp6&4Q#=(kLpdZe<(iwP$oOmlu^2jJhIKN9qLTB#~Lx!Z<33N>PcE z=EP-j*Q}h-*@wZM0Cs}>4imydcO8>PEd2xP31pBNw~=dYZ4DT9yD~FVJbZQ`QAW{V z;m4kyq`i`##L!G#xzDR-PA!Ywy+m}S*^W*zB)lp2%WZP|5od0l zhV=YPz_4PjY2?1)R9q2K45S(RZ_D367LqwO)H1%;WGZ~ikTTcr@*w&aPS!o&D z_ecSV0Hzbwlh&G$C(9*(xo-G6iG)&-cHhM1k!@*XYOIWk)yUN+EMN(1B5e4o+$Oyd zCb)M#(D%*wcTSvgC%7iKiKT1U?S$z!Q?U7XJyaunBF^>+&BZ1+P9RbO!(ZHBOqY`n8Ec55B)TV@8^lB|dwRuZ6&BuvuN(iZ|vIr+^ z`smmC$+yB@o}_ESo*BW7m9bXE(GfmljmIq`h~k!Ml}Nldj`Z#dLame|fGpv#9^raP zZmyuBhycG@rGalTk7E$&I1cd%v0NWkWG}nL1`J+eUd5Z6n|hR=?F8ejE}L1y0f;>R zGTveiUCH2Kmu-S=xzdkslYaiDQQz?qcBUYj70Y@kl&w7L#^Or-u)UKWN_oWJUv&O9 zp#+&vEk+P{EH{qpxvPdurTz(ZjVNC2H&}zf{d0^iPRX*M6%6f5nz$*_$;KH3$B(yH zk$^#pP}0`P-ZkD$a~vTL$qFnK-rr)(n$N&f)0wfTy{_WbUm)~XsLn(TSFRJFqPM3Z zl#l6m>d-$No<9c|=XADrsaA zcmA~uNlH7Wk(uw2qbi(kU-7HwgGe^s70$6HUdGUp^i;w^*KjXE9_NT{kiXgriksUyJ9$?zx45WX8m{ zFqva^i}3OdbhHek*)^LL3w6~f#AhxV&;w9}!DAPTl8U;PYfR+55EYDYRoDk24U|i*bGzyjn-l2d zhedOC7aPOo(F9i_zsakNNC|YpKxsjGaT^?Aq~sO!?Rf@UmwFjC__~Ffb&}P1IVc;gXWcP&}zZ87}4U`lx`sEe75dZA4 z-!S74mv0lOZi1_rzN}ATC!CXmeSq{q2Q4rrWaVg?;vgrSlx6 zUl+y1Dy0|sDdDFHO!)*g3Tg=Gn)vY6pnYq&A0X(1ND`S`B%f6SE(Ks5R;cIFFHB7K zyW`SF_hC#nIy5zQcYj3UNZQ($W??TP14)Kk##75UwX>q^cqVP6eK%rN_)$18isph? zB?bpT>x-<{=}S|lv3D+U+Wq2PY!@Ly*R*kQzS_w3W9tN(RU$oy>@=smg9$sQa-&fZ zqLVK!IUxqbU()OzN>yP42mO`jnCDe6l{Bb~x9 zQyRCD$*$=@73Y_?X7W(?R>i-wsEF@#2g?w9OiDFicF4L6BGvezpoe1ad^roOb+y%XrywucM|2k?@e=eCDDH z*cpTIX!~58oW=G1^Lo;my<=$p@ArgPBu`BW<<}v){%$Fu%CixC2M&Lzr|n8UIZ>&> zYVV7-@17%AqlNNCs1h~Xh13OY=s7BJmrHr3vPy`&ly}HYqt^iH7ttL+URReivUw}G zAl^f{Opv{@PrDrcaCd^3{@N4a#Okm5@UlGGz6l40hb;6EhlNW@NPe*1Y{lx4c5p3pHQbQJ8;|L5e1Ib2wkytFrKo{amdLwiWe!mYbb298Q8Z zF#4m$Tdo~VWllfp=EBG>F(0>d*(Aq~M2gB1%c@HH=DZU5b?+C+Y1z~jTW>0Iitn7P z(w*g3>reg8E>$*)bWMD#3!p%ji*#4*0taV%@y#sr~ zVri1G0iAps2y4ZCC>H7#<5VD$g=e0t^hkA^K$vF>3yx<+^9ml%`Jxkvc$c8Py=ZA* zuWmyT!-gM*1So*pgMze;>swBU3faGhpyDRYFEdIO)%k@vZ!(u8A}=k$qU+83uh-QC zs9#gZ4FmMU-8(2KFF{O=%&RiP2d(Rx7OJa^8RTnlpvBn?Guyy7XBU`M1Z1D6a|O{= znPHOTfx8|>(+_C}*(6j71s3QCT~@9!mwu3<2N(zcBNdY)x{WPUpA3(wT9C%CPD*{&Y`PNPvNl*q3lOWR3uMTR zMwk!VJU0}dKb~EV?W`kB^Cy564BGB}B*Mh7eRW6Um{p!_V#t$GvZqPd_vzOajP0L% zEaSXjh}X=5ZrfS?1t66Y@8Dp8qLqW^rOg7+mb9674hx249(1n0@~f9!5JJR9n|Y$y z^=!L!?JE(0gK*-}TE3%KeMp3SH@9=y{u+d&g_r4;TL9Xd5~Q%L2u%gm% z#1!~C^r}Kr`e^pt@}uMr9_{Cz1uF36HcP-ujj+6|iEbUge-GL{@>5?Nw2+f8DIOmb z%Tw;6o->%9zy|iv8WTQRnT1D$+e>F1*-f7}rI+u@x!crwzPc1aXL)uJ$0+U^@{3Kp zPDbtr*#U(pw@K+eF4c6XSb!OUp#0+Mu@nmlKqJUme1|CBXdhw{zfPk8pgp1r$)8== z-&+5~)Gui=*;>b!e|f|y=iciA_GNkhVs){*cT{!L#N?wiAbo&Kcb6Ph*ovUEg-C>! zX|_rVXUWpQQ?kMZdsc&GiLWLt-Hs;61%N0W=}PH>8|>T8to?CREhM_814Ihfn`@Tx zoO)t6m$Q_DSWbxIAaCJe=Y!|pI)ui%B?EG{l)c@tZ7XQ!;r?w`& zqA-R_O{J&cV*V0GSg7?)&lHV{YCE@^}3A4Gnk-XDyBM6JRMO?2)kS2QflIcqN6=~{@}=C28#htwoE zvP*q&V6L!ay-3R}s6ow}qOax-`wbns&SvpAN6`C`MhT>jK+wsgC9|bnZe%DjUAI_L z5E#xyb?|gs9Oi|3cJEMP@1a1=LeMLIKbQ;Eom3w#7SJIOxcV*<_x0IplUHg4F~NuJ zI&%GeZs8P)<*?oex8f>-Mu9o57b}4aR5Og1*s5tACuG=V&+j_R5VLm$a+AT^a?6duiHDBsuWnB{uEsUg3=Tsd^v>`&?0F+}huV+<{z54s+Swp&vhWVCy0J2VC z9F5pl;avbU`v6UPx%)8)-{R7ynyH zF)ufIr5ng(Vvui2LTJa?7)ZJS;%1ro<}NTICP^r}T+SSLd~xIinC#ls%e^JB-4OAO z{QL)u>Wz>&6M)=z7eiD`C-|>*$LiyA`Y}7e1oa5Mt+8Yq@*l3$rd(=`Y6NXAlP}dX z$$#$~kS`HLQpHv9 zY^#tjKf#iz`2Xj3!yzgr)yS`Zm$87ak2xIRQe>u7kgR2CAr z%ois=sHR{|7WTa_&#TS0$$)C`%-{jv^}44%qr@fPgA-ToEAG{FS<_r=S6A1Wh$dII zaWO$MO`5ET_!Vi0Y_mEZB4=T+Ul4E{w2A*3OCaLnWn#o>^HUv2W4QHj`OQKx@pbuH;WT7s}ERot)JoAYX%JEr>TAAHnly|7feG$`ImyuuEl*DR@p5 zOVL0s(CIcmVaLPnsfDEBv6Qckshx0?tAH_8@lf8zy-e0`F3q?aHk7?U)Q z1=UX|soF<(Q-dgd<-5)$beTrdx^>--z>_5bR;J8a$p*7!SoKlHQtL;@B!aw_K(#Vi zOfP#$mMC=3LHC>4bZQe2qyB>zh`gDWosU!#*L?jGJY~-GbGr-4#vtn}NgZtU;l*N} zlBN7-qxVd{X7@D$_}Tcbg+AD1?B%}!X1l18MCt_Z`H5aWBspQd?R0T%_*fdP=@`Vh z$X;QKWCPE$Ck?$`JgwV*VlG<*Amp;8iMk4OE{jWfj}VlzFWBi^JnOzRVrs>6(=^oG0Q- z35Kr%10W%stF7RV+Oyeuhln1rpgoBjPRLsMoi+LoJ@F2&N0gMN#o(zhU34Q#md%r9 zT7c9uno3UD{qq}dK_f@CXXLuOjX@wfLr2&{B91GnLJlE?avMzo-PhkEh_t($j+V@t*(Ck;V%rpy z$cmMZen@{Si7uM7(KD#miYrI+CN1}#_T8N~xN zGrzu6nLrMt+`_M#*@7jnOAfg5MOS)Qf}wTG_2bNlP~>*y!Mf(pX7?}D7m#Zzo6yq0 zpHT;|w_U>Hhz1ReA#zxPzqFl=c6D69Y+kb>vS+)1yfGXr<`j|_`kE%{v@odc3ob@c z0UNM*tjA-h*Gn2skex6bLt@nobNd= zNKGkpp-|b^Hb8#o=>AKU0R`iiwMo)Wr~LZE)yCxJ+36x@F+D z;k1-pYwK}-M&pZL#zcx*?6@4e-lW*&x5lVWW)X~6sJq33@l0o4Or39En@&>J^OZ!O z%H$~HG#EV4txNr$(Tyt_$-)31`r_Zb851>P4kC`A7;GEnWhlQ~i~CV9KL>xd@zAuD zg4S_Rj;Pf$wtjPw)d~I7+HBZ4^Ji{tsSt-S>bO?%VqWhu%p zMdG4YU(AG3CDUJvllv#`8W7)GYgKz_25J`QU#8|nn)(HBLT}A9FdYYih4}nxbW6IyFs8vnAgR5r7T}KeqNM(${Ip0q*Z*Ny?z>ipwca4~)xIerI+cgCL zYVZk6r4sa^8gtD*E}AWj3sL*~=G*PMb`}*q*NdF=3aP|Cl`gW+_3pFTFdpC~7YAND z%i|Jl^_Fg0B)r1JXvDx&(bFcN3+>(B)V=Z1OaGoBbCs9N*PG3!99?z-`e|q+!zCK!$VS$BHs2TH) z;5OdOMIR=1`9x6d&&W)aYVllu^Azz*Eu>B4>hFB5%*Qmp43>1=CZ=7=W86Xkfh6!? zx~lK)yi54IB9z(|TuVwl~LQ^|Q)U)zP8x^aDov!$usjs6y*2xqy#@WL_s+`Idftn>Y*pI3vDr zy^&$uL2hfmT|VLruefYlt=2DiYAf*bxx`ShEm*OAI5w`l72&(kboqwn8RfP~sKTRw zNnqJr0RnMvi*YZE7+QUPjS|8^zQ1^#P7~SP&jR!C{(@e(mKW9Qc!WQBb(&gMrcmxP zeGm-~j_!TGHS_e0XlP05DrqMTurQ~*qmAp!y?Gd^)c5K9%#Hf`n&{%XWaTQ)B2^2w zPi@T2-4GRZU!((nr|72)T&)ga*^KM(v0DsR3^)sJ&AI$xgrmNn2W8sP{i*`~mE-{m z`OR{LVe$9bT$Mlws(rKV4v*QIX$I0Y&7b*M*#O;NXe_;)YkgwT^ie=jA`423T)hC>v}JO9J6-p_9#Z#R~U|5hnl zk~!JZ#H8YjOd;lNkBzl;!O43rN^CYb1u&e46Ha*M`dm`qz(|2uRNkV@V)rrANc_lW z%jrkIjOQeH)ue3i+hjO`KR+7SMuVHNE zqtoY7C(l{)yr3xAnRRVbX@DnbCIgPn`>0#P2)~8PbLs@+z7kOGOPw8bvQN%jO4sDdroz)VD2vp@-seEiP%MQHfLU*e%8 zO2x(rWjFcT&K05$N?29eyo;Uo^M5HtH+i0alaLgzL6hbRh7p8)27sHK|9)i0qF+|F z$TN%)8-0m!#`0+{!jqEPnJ6oH%-+>!SV#wX-`j8Qv9JCqngunX*#FIZhwyu3yiBX7 zzlJet9@BQ~m6DH2`wQc_(CE+Bj|&q8A+>%wnRK9>6Ln0*CsDvzl~34hetx(F!flvG-j&in`o^IgQOOG~RKEtnmG?8i*wRBSL5A!y zl1`^$_owvlr+DC#TCC=|u&odFO;YyS%S-C~hDDkp1=;nA#`;wPBQykPwDZRsZtD5< zN(Y_a|LP>sh*1u7fn9fU@6jhJ`L-~xLC%C`zfS?vQWylyvZ$F|X#Zf9ag`C^5ryqa z*o4lZEVWgC5xo?%|BXu{evj{p2Mc3f9B>`Fa0f6->KZ5A`^BfJNRP9yU5`qmU2-Y= zO#)LJ3ULnqSQZ2_4-~p*HOU;KQ;1vm)nFH&IkY14`>X&|uZp2(I#)(Q!I(SVrODy} zm|HDMvDhoeOdw0fQI@tYg$swMD-{@jsuv3~+}((of0IfJ$K8+HE$ge7j|GV;Kq93N zk7d^ZrEu`!@T@u^)lV+&7vaOoM$U*s%|jSZiL`V+of8Oo)2LAd>+;8JoksVppgE-dLc>lahh{8S|y4pF883&A%) zobd?uA-`P7%&m{?YvzYDm++g+^e_ITku0xODVB`=PkG)1pEJq70C&jRy&B_N>j4F6Tg{&fQIUPCM@*~p&)H}OdX33P%T9E)ZY4KQfaMzZ zzK680v7G>x)EE`3UmA8gLa}kNzgb6t!GL`iNO<zJmHRR=Q!lh zf+p4vbGIQ`3|DQQUmr^cu&s9m%d71z!Hu+@dS5b?3&nB{cZ2Kctd|cSwJWI;_UQY4 zAUeW+^&TH^s_BlSKlquvhV2Fe{J&mS%P=fo^-j)<==q*Lu1$_P>vYOdHm(ZTVvZ>) z#KgqLaKOZh$$P()HSj21=4aEQ7zRew6MKaefFhP^b>|^~lVFr31M~U!QfF8MVIvc_^nSCXyb+hSWC^_T7W3phbcy&72s4K=^C+}dOX7W@8fC{ zq>12BXPUsD%Q?_|&xnhG7P>nvNn5g5VpTx9nAke7Fy$^F!TGm$D;IpkU<%XRR1<1| z?5DjtE@F;d*R35PnkAor;@aMSxGX*d`fKnVq5C>?5GL)T$ zG5Z6n?z~%dvp?N*H=vQMsma?-wBkjMA)yZJD?$JeJCDzmrQ>CM=+NK^2#;0{6h~Yu ztR-Q+dVl}lQl#hm6~QwRQT9{7r==}-a}N_8*!}@B1F6d=c%^U=5dMD=U%F#XRY@Y7>#X?CyNwbNh3b-w zJ}bp2620?sz$s4Kno!vRzlt7cp&iJ9HhmQOpA|!)a`o)C23T7jV3-FWi9C-S`k@87DSyY%ky^i!yW(uctRKa-QzHH5(G?$be_P%6&0<*27% zU{hgj)(=uN1^{C7zg$2U3I#G+kvKN+zfEnl<+dRgtU{r9q};F-X>6C$=YVY)Eq;Yr z3P?vXv`u$812M$q|D~IlktkGSP1KPbAf7)+^#MLqj1>dI{4YXGD2c+xE*LC|;*x)2 zO9A{p2ZeL2cI#PHna3ky;HDl2`yo)Mg8hWtP&8RTRxv;|Rm7%-1lPsUx5Y}Yo&3+G zmnGhQiDQOxURT!U`SZsU9<{h zorPgVYzj?+k*G*~mA!mq93|Wf3q$b(Mac1~28_RKFwrb9a0@(aMgq|)#V~oLfsdb2RDX<`Q7K#eF4MZ z5GY}zFF$*e+bj;>5b#QZR4@7B(BZc<18Q_s(YJ3z|Mw!yNz|S*;>ykRr^5N;O%*3F zFj8#>J`y$x7^{u8Ca@F9hATm#x`T!ffP2$fkOv6we=)g7A)?TEr7GZ~ zgjI!o+n$6w3&)MC$cAIy!c`3oD%2Nb1n)n!&L zkt&ou!pX#f03(SCcnVzOe%D<#@F7R;oC~M8I)IWdWxk0sAOUE;hO(+>#k>S!=_sda zoX!_G!!nrvW5gtoTlW|eO5)1s3-;UV_?^U}ZUMF-YxXD`BYTu{4G#2q9N*G&Y>cW$ zcjkAG??WC`#RLjn3i!nR>BpTGZ1~`7A9)iD476?uwi0weWw z40#83WJm}lW~9J4UFJQE)Xx#4_|%w@Kx5fY1O!&*2Hx5~6RZCNtO806Z|7^5nQ%bb>R@*Yc(1E{DKMxDE&Q7AXWQHPAal>t-( z%CrWUVew0m z{bB`X=b%YTI+70_kr2{`yiaEO=a%Hj@u5(tG1LO6Mg9o=v=op`Q)8RQzxN&!V<()| z{Jss*Hh8L|&OIJBjRGX&>vK=O|F^WMD9Y2rj<2@`-_pOxpxpeYB!yIg-$^rAYRHQt zmC#)Zs2it$D#G*Tj0gh;MCFC2zR@97L9+1uLpW%qwZauuQnb&2!8#Q+f;ca8HjE9A z#AA+3V0!%T8iZ2l1ZP2@0gTW&V0wrntkH9%RgeS}NX=-9fpM;E#Dtq>2sM;4H-JKS zMYZqF^e$QE-d3i1ZgJbQf7=5BuXp$HIf-me11GZZ6AA@LW~FEqDk5~{zjKJEgYN_-pitWdftJKa zw?J_TD0*RMH3IV+Ff>&zDVlt6Wd}v1!v1-Cocna=U>;io=)&g^OVho3q3h0q1QF-O zICr*!gg}FEO8{Jwt~5m2mS|SALBqF^iReDiCKstn8q(uY_NQoE~5FsK`1wlH9H0jb_DPloU zoEyL2l(lBo{4r~?-glX$kUY7$&pG?-z3(}S;ibd8E2v?0MxG@P33sqlxG@)2%d;eM zq%a^@I(_haC~#<%)cPOgOwQ?{+`He)jRqurZlON9 zP|t`d7R*x0HUSIC31$uRvBmPw&*`e4FYc9WAy<;Rwb&4=5Xkj3#4hK}8J67|4au7u z6tlGv2|l&!N1_m&m49)BQ^?Zj%Eg?=_c*mlBntWR>S3EQrj$vo`vVN1^3!r1F(MP8 zpETH(M^o$gU6acwAm9#DcYULz4Gzc8C=vK{DO=*IRE`ugFE~t9FLk%ml$l76d*=b# zaL&=^fQSXb>a>q=r360Hoj+}YQ#WbA^E?;s9It4YzY6-4%NDKG1>F}h8OP!#L(gp2 z-5>~vIT8N-T3pNniM^<@CwBY%3Kx%?$@+JNnpUadwLTpTGl@~mmZ!wgc z*r=hK*JvQQ63(7X-69O7hQ0HwcnNz7S#d_KUfrGZQIr8^=hR$)$K z79MFA0Jl~_l46e)5>b%Wx{`%E=@d569&AW@$i$O#GcRq})?Z-To}5_uc&xdv<{ z3Uad<^pKhpH2QShSWXlBmt*k31N-3v5rYQb*{1seO65N11%N-BJFhgJna#HOw(dCM zaxOT^0P;4IkT7iy#SVvVU5qH*2`}}%5LqZIM9nA=xTdS};B)ox^N$&u<7vEbG~yoA zkAvc7Fc}YVSg{?l4~k8OX=HX;H-aNzeX*9(YR-OOyxFJAn)B=T9$0G3wKxkU`GKBr z!i1_*NgY29;0!0&AlWylUqa+kYOr65CqP!vw+HsUfug;j?=tQ&>W|0Nx2Q776Z=-V zFe+-hr(d#t5GO=0EMHeEob&Qq@m;cR|)B1S@WbGpfQ zpuW3O6L{g&f66=n1Lw*M_AVCONlH#<$h)T=e?gE5 zhL;5u`$RYbJmSbQ(kdx2x&>?I?l|iD*iqL9YRDwt+(M{|DBV4315=jyvwGQ0(&!7H ztOndU4}eUJE#J=7Y&xhKj|_IryaO(oW0;^OgU@uOzC!%J71mi^x-|Eq-6dxCsDNKt z8QAtNqAkz`uvib|wJ|kFyL&&EQUI(S^8~l!tYRW5oN)=tJ<2LEMslU@L1F41G~6}G zuRaT=v!wMHs|lp#Z&S!5d`T35jO(eM5_TD&r>J4)&m0s@iMY>5h{Jidd6lr4hPO2F zhOhdypXoEi*vH2_h|P8@e#sZ6r?$Xm)FO4pFHlo^91hO8IWqMI9&Un~^9LyDS6>F0 z!c<3XEUu5k(dVChPzvUgpk-ru=mIdC+$6iumfusBo0-ou83hv?g8S-uu>9)AYgC{);l~ ze+=jFMbTW2<*B2M$Z6BH;+vVg61hIJJfaICN{e}n)Gz334zY$(YM4^#6#*9p*7PZM z4vFhF%#3WyEf^5&HJ~Q#;%C1mE6^mLCMnfkF#7Mz`Q(kOT(6}~2f5jd$Jn|8z{p4~ zOs?y*?%00N{aOFMI3#990+U-b=R;++lU$12VLYlDVN6N#IKo^y8QhF+Z)xbtZlsD_ z(7S(5kCfiKaP74X9n`%%b^4u%${sj(ANX!h(PPIhz(Fuv0|GhcH1bOPQfyFYQ;L^x zl8bQC(DPCN_9XQJAF@G$oWP-xOac2T;H!DA_~;{nxE-OZOEWy(6CG^zP&EJUT|&H! zrd4)Y7qv;ZXfOB40x(%bZQt>H&5QQpY~wTIZ*&t#-mO#YoEW_i02tR=GzZSE9l+9&?d@Kp!!pc zr8ZnF3q}z{BxJiyQ{yd&5{maO^frtE7K9;*QEI0?`u+62{?<7RIn}JrBAY4e^mZk4WdJu2>w%U zd1}h1K;to~YBTA<4NyPg!`Vv$`PF(M;#rlab3sGCCgD{I*^}YSE1#;x4XH;Y0PPNa zz$V_d)14N#xqFh>&s?87c7_H5c{pLi%ZgknzD4RNHZ7W(W&;-`7~ zZ})o0>K4rz22d~I;gc(tY#WZ$%7a^FbunB*^W?caL}@Ll-{Ws&*6wXW)gw~pw|JhP z?eUY%DW}ZBbbd2HvhjT(e2z@1iUBVk*O3S=uuIUu_V!U)X}}qXVVsgB($-UK{B`u9 z7|Z4bom`lhJpd=CUtBMKIlzy$h%XJnSI|S3`>6FPBsWD|mz=JjSboNiC``9*z4i=4 zyX)WZ=OP3`k{Zm0d8%};oneV%?6u(Q0kdwa7rY44Aot@$dzqcSvJ{4aiDxjPwJwS^ zP6%JQ4Teu9xun#(Li5zWlR-TUV%K%3^J0at)BEM$B#Pe#WoUi{sgz~#-pJCm?d9%~ zSDrK&s)xXG-L@Tl!Sxj;OUs-n00 z^_P=i*dX?ZhT3y5#dB+-_IaCIO>w1Z(qgAj_3ntV+A@N?dmH&yLd=?9N{lbzyZscp zJ^-vKlwX}wbNQm3!xURr`X!JNx>byaDoL{5TM!r^M6;bnx<-WXH?GZD?;y@uJ22J9 z^U{Ai3X8O))~nfeVag=gR2QLE$u;o~m5?qKN$$K|gI`#Prx5r`SZYC20L2@GH~Vqh z6lem|FM+;LF!?y3h7MvV^}V=NG6MyzI*Gp|N%7mOfsMm8LPNdJ!T!V2i0^^~s~Bh4 zUY)9cp9r`U%@SRl#oaJ_Dd?H+U`s6Glj%YQ5s++&V7;$(uS~#>Sj014W!$ zDBq?~qgw!nCbxt*77~L@jCl~a)Wx67NFr@g27(Fk9nHeSK{p)?rj%x{^n_?ZAY6c& zfRT_Z({T3jbUp!r8q%A$Focgej$M5QFsWd72nBM+S9z7L2mOd6k(xetuaLGuJ%6>} z@%9y@ST7*qVz5ia&PZy8M?o!iaEl#=9hDycp~0u|^=og}$RP!5f016}2&_3k-XE8i zOC^#riJ-xST|w=(D&PmY!n~t7@7`gEc_SSvMk{z@C5RO{sv(@QMR_xQMWb3?T8mBx zOukFHB#)SRJNL*G)wZjUsxV+=o`$``S9}2)R>Si|k_!sb>jsKBF(BL>liR$lV{yH7 zqo;)wUM8Qab+Bj#ijEP8k6(NCjDL(KTX<|%B*@DRcc5R8Su*+q^_?qs55EX8vX!pm z%UZMNUjT&8i4bEVSHhxG$A;?P>FN>Z)PBeVEF|kaBW4SDq6D#|H?nfJOpI(IN_+Wg zI|1ur49en|+^agEYK8kiv4&6v-&BW(BzeFE3uRF&#xJ6@NtcX7zBhlTJ_Hw4AZc@wZS5T~o5nRCaNr%b z300f<)&-+Lb!CTu3rSUuz;{VxorQw=dvFsGTC{e&5sG}7@7?WBo)Y)`0ovY4ye)L% z*|pXCl|(QYQaGvOxgf%s3okYDzh2^vtQT!r(c5;q;FfDz6K}~8rDEwQC>n!e1=N3e z$0H;R!;N2ooAA>idi=nEFfbu3q@Ei1&Rzb@$kvy2S8&YnD;T*p722h87}gj3J?xDd z?yFMY@-LfLFN7iGj(`X*@zPYalHgW_jX}Ns2gZx2Tz?5nQO`NG*vy%KG@Sy-{t{eR z>PvWp6Vk7ig-ERY=!}e*O$w z*GTvqy_=^vA?CNZRCMSsY69NRD+0!|k1Wh0fTy;K0#-0g70Nn(9btO7DEyJh6(A2~ z%+G~xMP;#?+HH{hsAGU!2-3;{@3`MdganWN}>*(QB^D83i4U zi}Qr>tg(>*()CdGBv2u@XFGU3sYne#ADJe0OWqBcheC%B{&86KJ~$kjDQ6--i@Hrs z!#JhiNbXMc6W>K6zV$z)fWs!azkq zPoGFnPoFC5Un|>s8AZMGoZr&uhutdX2l@G$^sfcHp#>i(PKLiB%Pt5|v&sd5v>sOi zu?D$mgN@=aU}ZW55kHOB=PdhkAd=)frUs;7u_QfoPD`P>h{j?cJ%k;HZqJUF3IFi( zy|%?TJfutp)7^_IYIo?M)t>bt4dg8M{YOX+E54Q~_~Y{On00^+ACTcOmnd5W~$j8yLfGs#>(;;tst7Sk&=QN#W3xNN7En45$PKZ(sh@Ia~xp zs-MsjRKAB~O87hb(U$UZap4RwDLcPZqlPPr8|vKm{Rprab^wZn{ibYFkno0Rf~BM5 zkaEU-e*ye|E6}&)FigjJT`=oQ#dya~9BNarvL6;6j=t5palKr8Oa}mB4MUQ>pQSO{ z5bJtb#bi9?5^HmEYy~s7T8fH3ieoy;00Ie>8LadGk^8+r2J^P;&an+Rd_Sc>=4Nw7 zQSra`kL@fha!90R=R75-?H;o?t-urH5hn)3L*Ja|L{ z2{IWol`DGrFjZDUI3qR3u<9+p5R~GV@^cV5sxbF9MpBTvRrGTYtI6AmiHW(eZ(Ea> zOT^#zXp{*-DoEZi9ywXrkxYI3!H<8%jvO#z=InAR!gK+0^U~!e!KlhOlqN9aTn@bt~3(ZLZ zQcYswH*c{##vj2@Z5 z4#{S<{9#JG;eb=`8ZlApoSbH1Ax1t%jl(kqLL4}qWSN~l>VXP^{VG8M6B#E;M+PA ziAUjZ4pYYC|BzAR)2BMvkWYx%C-?UB#&lVF6+vtoGmGF*o*33}8cWl=`B02C{6o=t zIOq`NJCI5Q0YGR!%zphHG!m|EGG4C;{h}Xc#UufQs$2$c{AFxJOrB(*HXZoB-O%H3 zQ{Goq%=XcDSMCmVP#CcSrv4 zu^f}c;n=-ChaE)HN^{$;T|rNAxF~u169wbI4wxz9cZIwdY`D!-=y-Zj)mu?kU5_{> zL7qocf%TGthV$ZqMq6}=W@%-nVSZv7L?@Gr(bmF2Hy2DA7aJe2LD^3&iP1wXag$Ae zOmef~=B?P))CSDvBq|O_A!rQ`lW|Gzag}$@?hPF;0o}%rE^tGiVDDRGgk)F|#FCsF z@nD8nzvSq4GI<+Bwn(E?tspXnMJVnz(qW2S|Ak^#AQ&AAP*n`V~ix%)c<$)wVm6YQX!v{u2Mk2kn!&O%c){der zEV$`w9iBN2i4xZhD%;kC+EvoSkN}9U6fHe4N+tFMp@~XUp(-X^koq$lF6J6qbbnV7 zfIg}6+6ywj-N9rQx=1fl1S#h>QiI`N3T|oGI)n2Yp`IaUSA~bFveU)&o@VZ(G^;A{QtNg*#{W$o|P-sKbKU3AU0L-0JEY%SeISk`60m zn*Rjb*VS%U^p5<&NX?fW%Pp?gnSwNji|k&-qa^e#fSu)1@NPyL23N z9EYDiTRVN526D~-vF?bVFO!|5O24>ww>fYDpAa;8_h=yUwLnDNOJ~2&){@029+)F} z@o1f{VZEiv7m;~wHnYkrb4565g(7zAVv438@D!F=HZ3qh;Ia>F^Z4CnusGf&UFh|Z zkgmm_gAv&7rrvN`!&vDqJfAlIO$G@Q-DRV=xbU`a43p1Li7e}2#Tyyk?& z*vHZkp4!AoU`Xqmu2&kXyxMky+ZPd;zO$tsd)=OxT#?=!*mNkbS9AS^6C4m>F;y`+ z*~;w0vWf(8c@N$v=2zSR-v zFsx#~gQkkvJ}i3lBKb7sH}^mE0V0v)<*nkSi}h9u(Xyyh5>`sNRG}-k-c6v&%3Mh? zH0M}v$?bs(hP<6xJ4}oRYmjo1qu$4YOR2AZV2YMnk(V!Vy}Dmgn(^q&i6uSizFev# z0VD+;v|>%&k~v{CRLK{(Z+rwS-WlR11C}K@ghY}roD?C`DP_KHC4eFD$+4Xf^8wTY zjE!lkKr^L*8(%c{%G?|jXqQ-7S@SIAT?~Oh1MtoV#pOjp$m@87SLYD7y(tuvFi!dC5HQ4$n*@v$TdgGBPfWU%j-fAdl4nP){Dn>y8o?igSN7fT; z<}zU$Dx3m0MGE`YVH?ki1a@}TFEWGFc(bk&ZbOAQycTzr?5i@84T4Izw#-zF)m*>) zt>VU`*d%jw`L&oKkW+~_iai?Vi!O9nW6&@C)SNZ5yj(cc$Bu9_f0?vW&S+C z;oQ#FT{oLYMTcr34Hm*_)nCo-6Q0Y`IIjX_mgY) z=%S1*I|<&5l+r9>Qwjzs(>!r+F5W!$T>g0V7f2-la~a?x(UM;PM#n{nP(}O*oDQMl z&M}(YA`PL#2l$jUHALnkk}&7#J>+mG$Odqe2F1%boar-pig2?uT=OKDii~-nhj1Q& z&#xLA<`}|lsmqQ>VR56O`o><=y6k^mQ6dRABZhnZcP=fq$E0Zm-)9)8S07xqz!0rz z0%;*W*f5n`+0Pt#d<>k+sSQ+~}lYl1)EL1Xep$r?reBE1;Aa2pnWufXp zv{!PNS`Nfn@If)%%yV5-xJqhY|AmABE(nK{k{f;H1u6<*Q1+wVb^d#J!sYuFQ-pm% zq=S5@8Dtj%eIP9c)vHbKQ;MI^hbLt9NAj(E#;S=2TF0*-0H!WDnPpc31>Xxa*-$D7 z@s8_c9t3h0GuzdEpAncoD)9S;Xf!&<5Y~G33Dp|qTmdSzP-tX_a^#jWhUb=I_exIK z+}ZRZ>H^;&NMUke8YmcT(u65P0v9C1W^7L)@cZyQ{R1)`s1dpryNBH2!g5RrFC;I) z-IneH-Z>v!<%mB7w=!qos!?>e1~jZ8B)Or(zsHf}3*2Oq#Z@I^U>chjDO{4tRF;bDre zG+^O%q?v@uIq+7Ifoes$HNr_tKpE#YU`=w_hzLiDDJWRQchI>?o;6^~T-nod1&D-B zwV_y9Y7a{fs*9317aDA)$V z_@YryUAQnFMFbZGLaY6M(-yE7!Iq23&})Z;qS^F`BNHALR+s{}Hk}Pb<~F6t zH!zp?-wBdHbaBJ$k z4G+4;?|0b^eVnC*^qu;*JjHCwHHrggt&M^}2@F~2;+STU){n)m+_VX>;v63gA0S(WKP@)<;pe&mT9l*;# z!49uMgN1OEdT961=%lq9+M*ft$QS5x2YeoAi?|&6dSVU9d;c4H;l~^#>jo5BX-D7Z z1Pvth-IssEhtFkQBC+avHK9V!;eaVZHvuqH+o15inSzkOmgx!{@UnL;s>GyVCoOSO zV(ZcL15N6)O(hUc$7N>D){Br;xgp{kD~`ke${HkquSzrvIchVd-lY;UzxWQsNSN&O zl^g`kG(i2{+_VgRaRJ)#Lg&QEl7J>Yx|pL3KCuGGPuvPuNq_?aV#5{!z?mjry8`BOGm$M*R^LwRir>1&KDed0rTnyEcvC*3Mx47>=;jlChjg501 zMxFs@_EK{n?>F~6FTUX1G!k$nPpWyLgVKHMS!ng-3({-h zM@ot{uRK>J+XIs*0_MIhEO$Of(=~ZQ;lWWvn%wskgNAciP6=nkEEuUC=&cKto0DbZ+U0$$& zWB3IF0NNB8L5;iNhg{WW`ED58q#>;HttETcD$?+QX9tL0GD?-Aw@y&`f+)^-qO5Br zV-`;>4U2@8d848TQUHtl_TTZ#ZP){+<*3)~TRdG(d{5^+ATlUf}J_7r^FQ zk84Qe*1U+(21U@X9DpI>ELQ$^mLiQY%zQq5AzM#Q9jF$%y0PSK0C8QQEAolZ6TPE| zR^WZX;?&q?n?xP5dR$e-8g)=@W~s|5{#eVK(qOYLIYf+fl>;`FV1F~iK)jgnX8eAO z&OJb4$`~!DGcA3s{XIpkg(U|ABtLQ+K)G=^uzkY04DX+VM$YsKmTi4q6VKX{RR^}o zTmS`e4h~cHQ~dh*uBs+ez7A6&y`ds6TkUat`LEG)FBTE0Oi6NlzkvHfN6Ro(AWr*w zj|z{Cm1K_Wa43h7oF}EUXc=@?f$;$Tq<~*NhKjMiDFS6oOmgTbgcNCsdnt3ZHr~7^ zkluinE6yti@4`kjK4nk@LWUQ?_dnGF(|3M?^iTm;kXZAgmNUsYgh*7Px&xe+I2;TgK4fcgpz%h;ic{@4>{){9tD$w@hd zUEsD)fYaz7LeESdWmirlL&kVkc=^!Z{&b(e0%XZ8kdA|ClU{pMU;sOhz8K{3ZE*m=kv@Q%Omes2? z+r)>)_qg&!&T4S;x-!HNa^g-*L+u#;`UX9uKQmM3MRD1al@#;7j)In$tGg1C$fpQZ zcppR8Ahk?-AljGzpQ3%BNcF{gt~P*M0f3sX@@CHiS`s)m*;lVPKqmINM(e4(u*~QU z9dG={bQ+JlJ}ZJUhyv-QiK~<|qh%B_3FFz8S`wopx48?6h{24`1Xuv-$7Fn!nJvjh zDCrEQK1Bc=c!sQI{$G`RA*;-l6Ic^A49|faKX6M~cXJ|bXPVf@8%28YJK921Oj` z10uaR++_=F4j%XJi$NpgDJHl6K7+?m$%RR{32+Aq2!J~a$c6$y`YZiYCo7+2vy3%N za@lioMcC@J1M4#D7$;_V?yOCRbxPM;Z?Hr4bGtV$maQG=Fb&iB6F`r{MyJQh+qPyg~_i` zz(t?a1~hrP=Dbg(wAXX*rk>xFuqhBVSYrWVM4*Z@P@}K}U&ptb6A(+mZ_<=9c>Qv(8GqUw{MOEAx0;md$q5C8+ z&144Gq`0E@=|d+VF(QR+9Qp!gQ2=dt3LS5`@^s?DRftIl5`CxZKz7Y5(jxSSC{WeSCxyCi5yB##~K&7~L zFvsvr5z2VXy<^NY+^E|QVKTD1lMzOqYoH-u#_F3%a@Yhni)fvSwdu``@~OXYsD z1p!>9gVCe;4=Yh@&}H5zw1wj3@z8%ns)lV!?@EE&!P#V%0)A`UKz$e&8v2UI%)Pw) zYQ7<747kGFJxnBm)|xM$S2%DtmK>a&ox&3LyGFP$_*4I&eNcxAn^HTCesi)e+a2nG z4s2eG%lkT;JQ*{`w#abyQh;V>S(RI#-uZj19(G12bWe^cQCet^Nh%sU4xeo@84oBy zorCoTF7IDe6v*mb5n$xFVn1Fd{`s0Qi_!V8()Dnb;X8}w5?&FqI$iC5+?R}Ne}j|d zo0?PZpB&@6Ri#Co>-Y)gZw=c3c7rD+GL6JX7(&U16QGt)w4_yOY40eHfK3m zMC?Q1vttb4(@m1JJUdSoGp=EI;6h$61+UwDk{cjY1^TpWSeoQrV~F2Y+8d+>^(2z_ z(!~?19=67|hJOIH)KV)=LubHhRc-pXBiI0t91UdVn61mVS%5j1M2cns*3gHF@CQ~F ze7KVFgsD|N463b&qa&7aVOTXjMdWM`XNrQj-B{_}>f>j9Cp`)B5j+SmY$!hd#Q171 zD?_1;3L7xweHp1}+}foc;_(Fsn>^D|D&p?J3@63-aX5SHNw@>nsXTSFl*I$bBX>mu zyN4{>K9|C!_b!8D-PwfXl-WnKm3-Kbk;7+o1~Yd$&>+oaONM}WNOo}r7PM1ydF|su zVX0~cbFP0X1^Yj~1TJLTr!D_{C!ObSy#EXiW+PwdmT>NGc;{f`gx;7S*(Al*iT(w? zvwp;@@!Lm^HLi41-hjL^!Gi(erUo?tmL;nHp6W_DKdd&@#NVO+}kGKskPQe`9`h# zCd0v94^JU#vSDunMfQnHUi?MhA>YI|&)#WPlc=qsU0~aKQrmGlQP|kQaKdhJ=tPc) zT**as*@p5e>B~LNz{}BkB50AqZofYmXg}Korn4(5h{)_+P7ZzwwT+38G#$^obUHBSc_ooWL};&!Zpw9~ zNJ1#2^2C7mUX5X4|F@Hk0kQO>F_kVB)FVXH50_R%cy}IXHETTn^;9M(&?y$IyB!zX z`7eMtm-jE|K|g4Ti0@)8czCry^rHSUypG6qB2iB;CL$YdErGn2yj>@iq0SS<1Wj0^ z_;es1qI4N@U^>8LZ0v>EdA&y!XamqxBX@#rWFx`M9++Q#`9)gxcxWT(fCX>i zhy4pnJy^7?>{L7d0*It@^@@SvxDeN7GAI(0@}BXypVf(4szG?!stMu@*0W0wBrsZ<}OvcoV^ zrfe?Y1D9#Q-A$PsCr5CkDBi zg;#64!E)cN?8M;*2r0mk2-xSK=kDXH9E^Eg%;WH%w-ul?@2&xV0TLh@AN%H#hUO~+ zD}^EkI9zWiZg`xk+?|p9{3{yz2GI7a*6gI$KkffyTua&;o}dKrrQ*1>;d-?mt4mfl zNprC}y8T$yGp20fxoefF1F2pFP~7KFh0Kq@OZQ9q;=3i&5fsCSo&X(M2qZ&Slz|N+ zB@B!KV>(qMBOGLRoCPIz{ro}Rb(;d%a&bApQ{U$8x#bf+p%2?mx7qC19G*0-KapXu zB~F=e(XORSKz|R5^<_AXx zv1k^lYy9)99luR+>f@u5%e%aokddRY?VdP7Zv)RKb>4sG%-zcJotoQb2n) z{DWNDX%4_LRBN&csSKPHW(y>BYxk(z4j+U#@c!5>erL0 zI)B27$cz4nP|hO(67RNSkB)S-g-tfT_a)T$u!~>m5A8gRht2T-ct@+2LpLDJm4lP z^8WT68IM{6TNq|dQS@_90N|3xGlsF{)WV;GIQ+Cn7L;>7R?eSjvisKvAc^4=VH09NE! zd}kVKr+I1Wr9uOMa=vW)WF54uSS+sUcz8;rms`8Vp3XXAA<1B9w z=WJbV?)q}*(PyN|qoCXlQ&uv13DpXQau4|}XBoieMw!qi9sfDf{`>pj`A5E(lKhZ} z?m~y!eq{AhazA(sP#=J4dUo|)(_`nFT>i8kel%_}OtGNP>Ey?__3*YQd&UeUM0GuI zxM;a8(U~i!YFoI_Tfhwn`>XXgT8+5mK@@=lakOHz()#V!EeHexnsS&2<4P^%0ZC4wlzQx@|($gJ86M37wJ=3&8$>h84Yjo7Gl^Ed5Xe9XA+49?`%YTd# zdifG9Jg?|%(K?NCI&H4ozd}B`SY&Ii3d2mX?>KOw3FtOSfDMDxIPG{U(zg!4!H<|$ zAkUQ#;Uf!x!-L$fpAUOmmD*v7d?OclrW#myvjwK}?5Bbq4AP^F*dcLCW|0lV3H`8c zK93N7DbcWENv;Hz=V@kNyDISA=BIN)WRZsk7JS@_SaL}=#opp_Z1vf+U1P*rPSvJv zcDCV7zITqMm6*ZrGJr<ChiEZxcGe5{u6#h`>%S}iULqGYHW6S! z|1lSY>c%)(O|lnK53}Cg7HMT1A}}EG?D{uefD@7N)Q^mLFvdPd2Z&5xg?gVyB-dGR z0yuMJ14z#|vu{lvw06I3+kkW3`ocm)!Ph;5M94JjVfl0%nVKkHMizEUo~cf>zkZYU zV`Rb2$eay*rf|KDizTUuH5Y& zOwG9rU9%|AmIH}G0R*@$k+P40jqU51q%DLg^i-8XzbYbgU5e#JVxD3kRtGhAjjNky ziGyh&P~@YrY{jEfZ8G+=y6HwxXifwFsNQ3)^<~UvQe0lBROHT(hFOb`-l3S-D*--N zD|9hEj{iPgzAKOF3yGSkE1Ob>cLHT-!5;sB1;-!wjQVqcT??Y~n3bVc2FeKRT2x!) z)Dxi)*e`Z)z;e)>tAU6=z1+gy(amr^g-c}Cw9?Zhy?y_(m#hf3MME9j^YV0p@D`Whggt8nTQ`gFg7N?{M*Lwn8~-aVVy0dg z%4*~W^%NN7sTb%WCeqrLQXvdygRYn~v)DCrg6=ro%%#?(M`M_{Oa+R4cRZN-&b;mN!%5`j-FA`B$to z&%>a7btPSXHQw$=hPkNKWIw4w6DDPIl>^7V8Ah-f!({>Or3!g*oS2sifeS4iG)>f1 zc?qJ-b3%ZL1{?tK;F3V-W;=u02xviGR7p^ttlGQ6m&t`31{UF6&{~d|IKzK2N^qVP zYP)!vD*KR0B_9k~mDBh@Ew)AJ;Eo6AvcEozpP2fV%~Ce}#y3!kK}Z#rpmPW2g|FRs zV}xJE3Ie#y&rKXWqMNvHqG^5Eq}b@KWJ;tekJhHYRLNJTf`?)D%WkFS3iaOtQYQnx z;BxNB&9Sj`hLP+KB)&zH>YwTZ6mqRH!%r!%a`=i5Nt5&8u_o!z1ca4)EKdYddF6Rd z?k9s}WG@ptUJCS{w6y?VcM~i0YI_v7i0NKP!rr#4Kd2pmp^8fLBF_;F)F;fWK+tS7i8Dwyg?p%OWC{GJd74>2VFIN|r>Hp@^fdT;VX2<$Gu@ zi(+PEZZnIn@r#Gy>bRmQS0y$jNFr>N4jSL&?rMM*ZEe*_G?*--2P7sh>!HRt7-ngj zmXB|+yk8<&gw!BFDWhY^bt^Xe`?FQMWbbYl9VnU3|19J;)&Q7>`MkQwcd&{R@$kG- zponzTgKj)`j(>W1dC9H2)Lr<$aK(n88c~VAbmsRTPNxXY_01nhGaPS)Rd;qs&b>0$ z3!a26gm4x(k_(Jkj*Iy5w%r5q!^#?QJ|)-`zM?-cu`EtcOCqv#SQRtYkJXTV8zO_7 z9_G5JE;xXZ*}*6zOkpURPA$FelBf7?_17!%2n71*^ zRTnSkR_;Z*LV0fBzjYm?hYMMbJWc%+s2$&zCOm}ayMA%azJ|mk-u)-l(jreHv^V7s z$k`KF@UDzQVA=QLQMx`nqq4*X0 zbTzz7GnDiQjb&zsirSl%PVbb_%@0`@nuaM=sUV`Mq)dVgAD}tpomf3qky<`WxL2=BU3qw#z3wHY2`_cWqZ)v}V@I zxE&f7{0DW5rffvP2rAC~hoM8D+q`C#g}HVqLxkeZ+a}}1i5CUtriu9Uli6EJKJ7G| z;*TrMWB)Sz6mF88G=A`UaBt99yC<$_ZjoZzJQOk_xP&(vLndGnXhcj>c4)aJQZ8on zXI(8ZU8A2COAI!b>a#s+! zFojewKtgeVC>dw=^=H-9J!J6|t9(azYy0%%p||D&Z)@}A%H}g~gMthtf&7}cedTPVg5u+Qr)d5XQuu1=&X}z=(wMIkCRPb zaDsndx$^kxdver~&kNe}4%?;>ucmt=dsIgk^>O|GSl<7aYs*k|mYsikToc9k_s8zS zw4b5Fe$<^Y{`J%Cg!0&^D z^j+`0ZsDR|-xVH8^&E+Y@GboQy>iPG>3ld~2KrgfHtr&pHu82~f0G|R*s9h7YxDMN z?df-i$Mo>MmZ`!N#xq@eHKC z{@Ix)Qxa86ltq0VS-oMc=rB>`ki4gsH&0nieFn9Kaeu$jo!{Rd zNoL;r@W9Fg>OEie`Iu4kcy_qKeo6`m9FApQUiYK{1%p3B`!J1!W{}TK?<3 zuh)`AgO_(-5B>E1CDUdG+=EeX(r-+YGCc9yl-*Z9`nFdu^!e}TZU38_?<*EK=+67dw(K_D}-=NIOSJ>Ok?Qhg_6{-;b?;D;KFbxA|WXean1_&AW4j65jHA1K)Sb-HbT zIZ;M5^7rveC*=N!Q}nOW-PYx=*FJ{-oKBMwi`dO5OiYF;Is9HwD!d=w+3d0|cHwyC zg_Ftb5K-D>HgrSr`1$9N+EhV#$Ctl8o>IQ>dE}KDK_zl*%_I1cW7PMYTaFQ*Dqc8h z{wUfn{y1;85E%{L{$uoJ@yd^dsHNYp&-(s-F|F+w{zg?M{nX7Fi6dm_0&KP^&-Q3# zs_ZRk>`t{rWW(iQ$3l~crg}(mj8ZqUkC8%VtgZisfv;cOjqhsUV0$Ho6rZ9^OvZa5HF6pMI z=I2*`CCuivTL{;#@!_P1C!I`Q|M#BJFY0U0>*%AOJKM^|VI8;^TKh&WdD5rbe)4cf zsr^}b;TZX=G409iug|@b#cur)wJv)d`4$Sb4Eff;`BC%N#DN)N-SV04@WBpcr<22h zvhefkscQRxZ+B&OpB&dbFjoFL*LBNJ*5=N3m-76dclR#AzPyWn!9AK#VY;L3+2?qT zv+ZCreSIUf%J-^MtLyag+pJd;53Ht)XJ7yQr0_#XeP_wK%JJ5M)t$fl+h2by7|sms zHhB1qdoW^*LgLdk7Xv^$l&nUR1<~VQzL{=4kA6l|Ekivk`*EF}##uMF#vbFgn@|%E zuC3>O+fLc|fcNm5tv<~ix;+{^KQl?S-_CyBc{w`u>qpsZO33yo5qAT+^#TY*X@Wi1 zRQEk~T}J!)2mTiSsNSY@{ScmPxp5w9iG6PNS2U$2s&`0gdy02-;ox{<-k-4x^i87v z>Yd}#*@w5Yq|%NhiD*_qA8qHo<-Z8$t0RwZ4+}@jLb7kyp9Fnpw0HJa-aocg{}X@y z(cd}=$xDw&w&ouyqo({`-4dE;v;j@+Bm5?B|IAH9!NJcYe|~xAhf>oOKDVO|{TDkg zR;ss`<_4JV5oH=5Oy zHIDl2fQKzZfRFuS)avh+w|``CvI?^*J5N{an*1Jl+s=HxkZI-Uc*Xo`CCU5I`_JT- zBYe&^PNSK=<-~&vp|gj6l*?I4KE0pBPk^Q)M9g^Ac3PU6mbcI)R}5&|BSQ?obNv0a z%<1r7qeo8RODEfsQ{EY3?O`N68yhc}@R}QFZe*gMG%qgpdG6tC=~4Re_eaCK#gSbv z{o6WupM+|SLTn0tjyZnMelNs5ZkqPfe;YA;AfRmf zq22V*|*hKFE~DgW_e$H7(Q^Nwfsgzo;yD09HDBqzyzgG zaOT+edzOD9&k20!Ovpx2W9`~y0e*h<3;`H<=j*Ab!86v8`}m`31TGs_jc}_ z8)tp&@G7XEm-DyU?)impBE1islO07#RnL9q3kt{moN$x+ncAP1-cGe0d})Yc{117( zd*i$41sUJ#`j|i$i@c1sD~;ub7L&Ql&nNw<-R#Kcd}r;hM0Wgs{gPp7MYOm0*YRxE z53&1{Adn+&Io{p`o3zn;zA0>eCV8i{74(lVH*XAE(%YBM^aTc@MC%XkFWY8)re4%F zejT)SEc)5?LMy^IpOWLLt6SIlzcKgb@ldbv->{BT&N+qCMky3Z9ic*YQ^%HsBxD(q z>{|+hF*7ZvY+0fxYuU5!gJGsfl#q2W#@GjAW-w+3V`lD4=l8mQ_w#%1=Xu@FbN_SC z>%7iO%=i1fuIqbUpY{EDe_o_^eY@5O&wGgX=WnZ}Mg_Hb4i!+das2)~FRM0H^$%eJ ziCxvl&;QwHuFuZkeM(o>UH!Pi{Za`1R1wi5iG7iz#lNkxE^DC)qjN<~i1Ec|OxRbG zZe{k};JkqNE_B8q&;^vYPMv`nUu`o80h2w+%Hd}ZD85nWU7_e)_IP4&x?{mmdCBdb&mj(hhrTUwwdSxH_GWdZIQ?H@JxVS~YZhhAwMr`WYn$zbrqLp%_G)eZ?x>HI0cjMRzzc-g=>J7mhOP^u5n zRkm4B;A3#lL5$>Bch`=oK2)z?$me}HQ(5WV-Y-@*_U*yeOItqzZ0yEO$7X-O0r&3c z1&p#@wAdl$sLPFh@9s48LdL~Tc`Ddgb$6NH+`uyZ#vb8uJ1RaX+jqlKLpVutWzJ}T z_e^PH9o6>DuIkkZ<*bX6%>Ov@$KLc;rM)Ac6kokg(Wr-x!z z%O0+9aJX^17D}Q(9G{zfHXrG@8ko{zH7(+PKM|9GoS6~i2v z=4t!u&34YT8!U-rHu;`8+el}n_cl*fTXHNy5#Jt`Q`Ef1g4P=Z+m4Y6EO6|J2?W7qD*1;2Cq|mUB=9mMP9X-%O zPJ^0CiweyClkwY;G1#(>ck5IIgnfi`?u_rv!P-UXhsdSzGIH@@Q;!Jl+x`16 z<3+m0W)0*ora~e|>xrzVye%$SyP{;?%HI$A{qT0E+=zHDu+dPpHvA!e-9 z&{SqCfJaIzt}{K_2V^7DBq{H)w&>9YMq(ZZtdQx_8gfQd3>HFvJHpQnL>gX9(2v*B z^U?;YzAzJhK{=q#j9ym1m|1@m)TatbBCl^`0n;%?G1ko_={fDf-36)FHy;48zMSN9 z;M4?IHMx{s?>==#KYAR+f~=2JK)8vQUafSyn&TGQJLsGRYAPXkxf$Q^w6KL^q=p_q z{ZNWLjvBLKn>nFFf+-=?1g*L8>wUlMH%SWm#6&|P2u!jh2D{>viGA}*wnER<;{x+MEzXR(Gw_mTs2133 z2Dsi$uUJ1$%S?m13h$qXN+e_h6w~BN8wE_OAS#LsRWfXUtGAwY#?XhIkNhgkL!`5xN@v*Zf1& z>ZU=`mjg;omj)c~?|cc6 z*aK<~fu%nIE8XDPeI~_14PX9sXa8S3-rOd|@p|v}YAqrtVZV&!EOyYd5lgEi6gMA8 zcVMFM2%;LS&(SR$R!MGZdc;7m3UY(ijwf|*8G*v!L6?s|YF>j`aheE7C5QST6AVkT zni$jc1eS_hp{qeJd}eOZ}Xi9l?8@ z<*+W3W7-RWr@qG|9EDv*)QJn6up_c|eDuqQox)Z(X}aM~4`<}Cjm&z2@bOy4@5YiB zDTUIScQ&|a=a8dKa2vvJ0&$W%uKT0Xs&ifSnJc=xpFO)<)cX$Wu)mLK3k@gFrWYWX zGfP7yv=-NXHVIh+Bm;pZDWbO?r5~@6h9CDBQR>JH`c~~`3JK>EtE>B|u;ZblLwL0J zP+F8t`E6vInk{e$2{}2ZXGFDBx9xGzcKpO9 z7w;^ET0kNf(z=v+)?%{L@x zVDD~TlRWkNPyb85}UzjSAmM}CG*PWXH=(`aZ2B_=26sroHNf)(Y~lLFbg zz-L$=xhnU!fmynme9l$5v06J787+3J>0!ggkq;{SgBORnb7%P+8l~d7u=LmTrHNef z$m+qUB0mfZvu~6NR{vzq;jUEf3Q=9$DP-tlX2|KBy-F=>m;l^7)6EV!x$0q$_8%w> zdnlvDeDTmJoHwdj;~qZ(cTSR%kvQ*pKAQ-~A$ks|7p`oK#U8=XS{ z3F$J0V_0Y}zS$)x(RgaIHc66#ru0FF&nW~wUUXYD{sJ% zT{dN{>_1-~PB^&V!u>=JcyUU0%~8_Df8+g$(Vnzy9K(!I@3Fn2tK2bT16UnEUmCqo zw&JS&5b5gE!MYAHg6P=czF!_a+KIV6TJ{{R9-12N*~O|NB?@8xIW!kNV> ziVkk!(JVDy*X&ko6QyN(O3k>CVYsAj^Hp;XN==Qr(X(HH1Sf};0IA%H>2eN*ZI-y; zv>%lM!RjiYSmim*nH$0|K#2f^2ZYg^p*Hy`^te`~zO<1yx%sTE5a z^{Cuy)c?18c5)Jkm6sh_K?{l0=e|n+UcY|cx4bYb)b~^C?$L0jADU2ZmR=vsJyc`} zDlcFYm%i37`>Ow1ZDBnmDYWl}QSPXJ=X|TvZh?0$KIe4zfhV^MvWYGCr&|4qnCyoW zyR}7*Ts44nZ>&yW=ot`xwwR(lwW=elFf*(m=Zrcj6cJ`O8mxZ$bWYs3E4r(qDSSi< zE)~3SlC!KGKK%XdUh(Z?vF$xN?D74wEG_2jL66?7;6+36WzD;@b?W1{Fn(A{JOQvq z9J`PUo)U=oy2Vg#3T1wKO>gE7e|udbqPhNK##|#hDqds#9k)aB{>uAE?nWCMMhfFR zLGk0>yquCX%7RZBHWlQzSn14_Yx^rGIpJ%)YNWAzf9hCWu5p4Ux3nhObEw$b)-ill z`Jp?EI>G&3)2mBaw<2*TXMm&`q+v?p`}_NKpVJ<2&Ctyk=h_YVT=d+qLpJJ@A)0h) z5%&y8p+5rN2EilaC4pxVfI}>N;_x;(bWn8MmY}iJnVWD@1|KZlz@89=gjO_!LQ~73 zy_)nVkDrFfb!egJx)9;LYZnrd!w^BlK>eE=H?|^yu3Qi0f-WMBO_i}kMGOfnx=-c z8>fqXv_h7GmMtSiC&KVn=GvL~M*alceS4$rpasOiIH1zo7K&UHqT}TKzg2Hj6I9N& z(J4Iy`P5gX&d!c&l-`FE*;e;VsB2x3B;^kfI%q_c745SC9Y1GMC-O zI`)e|ky>0%q|0iKVX9*g?`6kvHqSWw#VBfflblr-LL57+kdhrdcGi_G&F`!21uuvI zr#CaF8T^vFvc&~0s&mgC_Tuh*E2)r%Z~u;-2nB@hqIzq$U0kNRll@DWKt$g%qgOVHxz14DUhB$G zcS4*R$ef~Dq-zCt>=NX&r@PpY^%WHhh-LuIQKo6}xli;$rYH!d%_xeF!$kL+JP5(c z7Z)GsTer{)Tj}d?4yVRx)_6dJK84vrVce1cEV0&WK-@VrD++{Kg8DYe$0Isj2J|>< zSRv`~#dh(DFp8o~I3o@}f#7rtNrm5AT32RGcnktz<2icCZ9{k1l0ypR)`6Sqsn5-U zv8qjdd|J`4eA6yP8oqQ?CY+w@H6wpqBVeN0M4c2g)t(xmL9Ouv4qj|8#DduAQ_;LV z>XEI-`fOFv%o~NM(PV_MN}UO3KJ=}b?+#}zYfin>U)s|3A6&TL3Ses9{do=vQoz_5uQoI_Nvfw1**Lz~pMBs= zW9CvEsM_AOIcIdkeJ@N+J|!V3!ZH#qq!Do^M42fXkYT(ys_u^nTun0MukN8x*$waS zwG?}SVzsmaut=>K>4?u{@78rMD>Q`&U!b)GnH|&ik0|WPv{GMNthHbfVK{#x93apJ z%pst}Tc+BVLj-SVDlZzuIxg&+nYA*9wuBFT&vsvK#5Q0(s$ z!H#w*AQEZKTs4*+y;043607w7J5E_9#vnKHct~iM7zj3@DBbGj8jG_ms`i@#&DFty z;ndP>U7z{qNe4`l(p3F&&RnDGAt1bUhEq$p8--|XrN;lq4ncGAW_i43LCeH^1wdxh z#Nf&;!fme_%@r70^U@ZlX?+cN(MS)lsDH%97}07#)QM*V7|fv|bJi8ut+iT<0xX}V zo$4537(&_&08T!K^pnK+=py|N^Xd5_E=x8CWS)ZQefs00Wez*ar(}r=PHLL?zP1xN zq34`9er2WBC6}fTtC=aB`%4C-i|*OQI~=YkFJ3z8k%{O&oC;wKCTr6#-djhyHU&}B zb~XT7L|FS$kgQljnO)6TFtI1|fJeDQK#Lg>+vfde?)xCKfQg*2yLR}}21A2bF|InC zRSHaogA}w6J}c^YN7w?7gXL~fZih9>nHU@rHTPelvNb}+H$GTFq=2cBxrg4&snlg1 zzi`p(8SqKaKF$}W>U{A2ACB2o-EEI*8Dl&ca5r;M4Y*=ZVpCdqj`Drr^mF`9v+sDp zJ^uK@UXN^K_-tRt_3TT2K)iuH3dIC?TOk-sTV;!|sSh5!j_i;ElKB4MQ3m;)oJ6b zP=N3WId<*ZbGn{fZA*B#m7+s^IBoN6jXQ6Bj_f=R$7d?HY;4OWKn^>7s;91K$8=1C zQZ$oeRf-DD=I5kjW!hAu7EsFAai1>5p73hvH^kafoq>+u4{0#5i?n*mSgW1Zxvc$1(a^iY&!DL% zW?hdjAjUcp^g>C^5dKw;Cr3oGw`HN#B|IUtTUE1IdMlp5stRZ)?8|K zO@DoVfA!kh8WlDIrrQ(7YCntH-ex$z?idGQi}1z%Q`7UUL5ubFbzUt&fp;^lOKboo z>H*8Hl(L-NePN8hklLnV@?X#|Mtmxqky#8v1+Irrrt3aWC}OHxBjRTq1HZPzcma=;QOI^ck#tE(@{gtHb4gP;}R zGfgf$^!+K&Id*9#^WF*_$A2NMvIWf7z~dgwv?2aB)3(Drvwxzo&+KhMBUeP5M{o2{ z_Z_Lc|LKXk6WlG=;%hmAO$cBXzqZiZseE#%4qYAf$lxsxSRAkfZWt;{g%C`H@Hb|L zQk`wAgNI^XGso+XRp92ge9G->LgFY*HOMVcA3p3yc+&*vamBYCY;5Yg5>Eo}ld*0A zIjg7$-^@Q-Vw>+ZrG#1Dnt;={nRw@RN{n;^JquKov!(A(eoBCBb%Ll;l^wqdfFzJuPw<1`CDFnXIt^YmzVRbrlUO(aT=(dZ9pxuE$O4k>8#xo#FL9`)=nfF* zvUGCdeQbfDeMh}NzRR2lXE2k^H9|gZZ)TCZvmvPN;vq@xxghr6yl?A1JJ#!tZ9%K` z9KDHWj@G!3*6UJ}KLA!ea2qVB><`sJ&M@&=Az$oT>c`jeq6;TNxxpW5v@hB<-Mg?K z0U`I+@}r7kk~wJ~__n`3+VjqGpNTgZG?YC7SxiZ%w{UuZMja37ASyc4%H^cX6Iq@5 zqT_*tJm3&XqxwyRu4_a^8PU~QAW^4xe|rQtUsAwKp=t*O9WRWDOMu*I1>PEi%O(+n zEkeq_{kdaGC^yZneML_+`+iMoZOws3(k=lt3UGlp>7_lxN;u#qR5U(V*ONt$+>JeG zo)+5kw$SteE&i0}XwKZMHm!+Vt}Tt)cwf++POs?&+aB^Vsz;05Oex1c*zm{rH_qAE zIz5CYpEd*Hsue?rSLzW$KDDQgKWGycqCvjYQ_G1>q>GF7V%-j^K4ZRxpPpLi&=!)- zHJLzyYml4w`eT^G!uh9ijpB|1@$B?%q%k)^7rEG))^E%_P)GVr(nQXpQv8t*w6}zjd~-uru{v@s2201a|aR0B+MLFmSc>pI*z)_x`3JtS>(ma-Y<)<7_6r$Cx!S$_KzHcvO|!u?#6MGD!{;x_%<( zsi^L9Ch#nlUq)JJ&wSVNeohV(C82udr4c7g`s-WZ&Y_Fe&kPMoNbVdJJ-)!xCd|riaJsd=42lP7M=)T2}EG{%)cUbpS~_ud!#TP_;WhCc%m1HYq4g1fzA4 zd?q>HG$jM|bxxbE1<1#pQA7C;-Me=VPzu2pvnjmik(SI5)F0zKujq@%A|(BEIusq6 z1B;QC?RKsN3gzUpC%#r)%m98I$M<<1yOqG{Zyaw{^QucKAQ7!^a^tnVNZc=2HY?piXKhj;0JVZ| zZZtlw!ODu1B6*QPVr-qaw^xt3QDf!wc5NGgiMStUdOv^bN`R~-A_?^S?R9q8{#~1% zqy70U@Zhf@3K9)QpvBguuXEzgL7y5u%Iq-i%q43JJ>JMeuS~$1)4$7hGj=nxr3?l`sgJBi&i&hW1z}=r;ppZ3(5&V zG)rUc%}hn(a!v(qV>@r)K+74yJtF?2)ra>BPXu=^zg3A^_#o~?gbyMORPPsl0!r>C zMeL9m4a5OKLe+;2G}oVf77!$^BdSH!Ou@VV-fbJQaG!fCU#a%hb^Vs)#)EPYiMguw|5gBb6>>~P4Q(rgw< z{=!P{3xgPZncq-T_nSZvv7^+>Cg}F5njwpdOk`Rdu3VYgVp5_P7>n80H%T?|u7Ci^ z$lgfC(g`$7$V~%ykk~Z20{flRCUs1$uK$5RQSfg^PS}BPu0MuR4)A&aOy}8G?Y7!f z*9@Gja>K3^5S|An*hUt(xCAoSd%+4 z9Y85s+StrD0Z@*`Xz)U%5)`9Zt}+uzk6L&}&ayd_Z{y@y2o2%7^^`v2_sHisqp}_n zx&abp9Go?v4tp>gYx)TqTlMJylWHE^^_ishSip21NvEH`^J7fS2NdLWm_v1X;naL_ z-QL*tB-4;eEVmwHk41e6PBrdKU~b=3=myj(|Hg-#tH_-Q-X^8ORU{MWGQMPp&ele9 z5EM5X<5PiI>j!}+K~Y_q198LGy1PN_7aB;7OU*#poRb3Ja2<9|mrgHcvq_;>i~je3 zs;K7l0@?}1tg$FO&^HNvK8~csp}M&>P>zT?F~k;FO(5AsxZ=w8>Z_hR>ye=Xl2Gze zu+QL}W^5hcSZIU}Tc~_lRx3qBinOuEeY(@E(@DMpMSDAwZplmdeM#6O@SoyE>SI^LPMY4-{RkM! zK~!9WonxvgY-ta)+@ZHo#UbnOg_>8FObynj;Pmx{M#BQn_hQcbL6zUnh-+uOOPjIC zF}(*2^g~{|?e(VBG$412>=^T2Lj1ukgaJdMI`FQu;vNseLok3Q*#pw?IA zH~WtHH_f%0Drs*Hs_dcayo$0Zvk%^SOI`*i2vIM6|NE-EqW!C)V8~42+cskw#w?p^ zkrBGYP&PlR_rMtop zN@@ok!FJGu%rz#TomKW*lLviB60dL)0rTEYkI}W8y1Vyb>Ddpy*lE)$xi9G0aPI4~ zqzDFapjQWl{vv@HtHw|5xnq8 z2%r_YN$2)k7WlcZv9#&Byw6r_;=_*A0f^Ud?V=qy!6?b2&GW{ z(4*nwBNt+F*2nI~$|h(m&)GW91n2ZETmYe#VmDKnrqwMU>F~)35JEeA_;B24z&apE zx@G|3Y2@bJYdcz)4OktGmo+2snXq|zo2u?wCnOWVC6C~7&Nz_Pb_!p8W-J-jKxwir z#c}-gLyl?X6;-YbM$3nlbEq&9Ut^{aC4O$lKL=DtvGd)%nv`|)1QK6OU8xx^SMVRM zecLeFY*(WO!VlBNJFmE}DVQ?6=Wf;Q`gB2oun)Y`y1i+-VWoF^Iv{b zOoArRh+&^pyTP_pbin%RFYf6Gtfyl1L+j?cQed?4vjavEq*S>NOX5Ib6k$OeCB~!| zab&)SLm3?C-FX3D=n%kZLJ4S| z!x_d#6%|$9Ur2;PvbIdf=9yRyFOpKxk`{HnZ5(X8NE%K)SUWAnvvNkGLkhK4NrZ6# z$_S8y>1Azw-7OBi2AV5S(v*f%=RR-34il~ey1QP&RTIqpui9-zwN{|hh3a>p%fD}} z68@_Hkz>eE^BImx2hO%vUrv~s11PPpv?M;I*csH``V_75}xrJ4?k_N7Z{!&#Dli6+Bk!S3gkxYtS~?mnT<<%2ckbKk96U%>^Jkwe`KpwfruKjcA?)tl zPzfDAd9tf|A2G@y+QaXRqN0+L7^OFxudMueW)(Q~V4=%>@JbYXh4X>g+%pOa4w5m# zw@DK%9zAJ_{ylFrYfw0(buED4@3b8ovikVrgKI}{dpm=-<)iPlNMuhqOU0HsHYQ0+ zXx{Bb@!s;+6e{A5>c-HILj0FwUMm1F%gfv6>OK3WT`k^htmG!h`&C+(q%@U-O)XET zhit6O&`}#zrOnOv+MYia47qiS`LZLc(wDIJp&WZf*waAVEY4H?;jS=r!Qpo8>yn%1wZHy_xTMD#ToDfs*uE zDq462V78!%kz+8G3Gyyd6%pOR>ff_;BcqI{81_~YGdTeFa5d=^9n0o{Z4^AXKXRzG z15kinHZWLCES8_yC2;Si#jdu=%srA1emoYEirC14Q%k(Y{LtgID>y#z_9XTn7CW?z zCH#;^W?TpllHq%mq$o;XeW|$rw|I+ zUYLoLZVrge(tS>COfB|;kuGd@qbDW;fAmx|v$jm+kIQ#GP(vTR6h4Uc$PWGzChpYC zS!uB;w+|*kVSo1hvpQ2h;oUU_r(#IFgIXaAKR#F)cLYXtR%Om4Xl}7Ev_+sK)JaJ4 zCl2kBS#lJug(5kb-r=F-GQtd$dwOhQEi|Le4c-qs2mvfaQgs)CU? zALH;)70<~>_T0&x+Vq{|>ilk-3a88+Me3B)y8FbjJze!~Ceu|GV7imw!i{As>)_L1 z>!|KAO*!{c?gQerCgjBOqH?JRwrDDk>_nZ7jPE$~(DQ`E*egI0^jw6%h{w>*Dl79J z(wfgIw_NmP6Mw30V7f$Dqr9vsk;TQL<8ZdHI%#Wmp!ZV1un`3SQkCuEqlR@8~o!f}*bcO@{JkqpbV*x3NmyBdO& zK&SD~S|EwicuDP;xz`G|?p3D*5^+D#B&xRog5qvtLxV=F;;u*A&vr=Nzy}lyJ(sYr zx;xj`k@>^5%TKc7Yly?8Pr%ZV4DFl(8orV|k;`MZ#zv46RFGN=KS0KNdVwPqdbP9> z$=!Mne{s9=zDtUITIIjn&K@TTlh6q5+!Tja`$MPKZjY(|ZD0XmjIX@p_!D4}z*|DB zOm%!Xro+!M4fky<+dwmZxG)(;OOZ1gcv5IrB9Xm}o-vm<*9jh7n;Z4F44z7VYel-= zjM+xOKJYkn+_POfouY*|D;I&n(giR`ai<77wi98b9=!fr_^~?5mgtcPfUZ@tZ{A4u z8i2D&tKbN)BT(NPey<^KFE`PoWLb8#o);=2zRuZ#{n2LUS3wV(=tQv6L&x3%dDWYd zjT!|2A&^n*?A6tBrvXd@i>bA* z^B&UrGYj;zxPkcXi1%Hf2+^bNKttb6=B{;BAUNw{>KrCj#iK{rK(6beXz2r}Mato> z`ikbQF^~e_tk?8Ns!vUlvrs1;gn^VDY7-qN07D1c2-ey>Q|sou`h#^#`7a~Ah zWfREGOzq%nW9}eMlmOwW@;(9~=eO~}Bcl#%P!Jj&j0IY&!oH;`AX_h@CXC?@)q-U) z7`>x@*1?nMCXMUepE0@32pHhA1RW`5t3#lHH3i+DyXU*pKnx%ys-8}wQv$F+du3_E zoU-)@y6Y>s+QJeVep?ej-~_kM+JHo(cz~bx%`a{!0raOZr*$_9(~ZE&YL+tVDYBF zj+qks$6k;}NH+fPPw_CQ1(C6DaTn%KM|ln{4Mljwx2da`C`G8O($?6-Q;UZr^y_|F zK)(vCi9eT9lR?1QS(-le4Vtd=-+A1vXSShWa)5-GHR1E?e#6|peh^$At8~$~v~$ewu3YH+G{X>+r6YSGZRm-6b>Ifr_k zV1ox)-U9`}>6gmdALWq9SzEnMgbASdzEaVb7z}3{hp!J(15~WjirNs zB{??vj_&wgj-bCF&(&OeZnm^GWUMedQ)6{7Pa6fwAH=%T3lXv$>H9x=aoZA3n?|KzpK0Z=0u;euaoY#IA1 zDk=Se$lB?7yaW0(a`~EcDoO6-ogzI@m%HyLx*0?#{w>ngloN}E25t(&MM$XSH*FK) z6mFg|NXFb>%*RtN_0^>^DcqS zUw&E@BAfS?Jn&G|TbG4QZP>VL!pUspR(qeZ^j}hvI-wtj3F{a4wOEnk9sEZhPPOuj zMUHH->B=epgIyxB34DHNliXjLL9~9asV9b0#_S_{uf$_w@xAG63m_SXU>t#WtDlbV zEQjNiM*K_zXj1~(fB=4+C2!zSi91Jh?>Ckjf3J_(@;f1*YP7?ln=>Z{qw}ga={BB=D>&)`+&LZU8u&*{KA+lz%B9dD7Fu&iB z|JP`|mn0YcOu~3?y`Q+aI7r79yIVK|5=CHdx;37jk%)pkeyP4TXrG8wud> z#tB-Xvr((+#GuHT27_7`{l&wnr1KF8RdBlQRF!e|*V?*d3oUZ%-iqQ#cNA;X_MnJH zIQo6Axz1`SimY5}SHqTuZ%k$yt>LZ+vYVO17|Q#(AD#d1sb;07l_ih{z0P0G+^eL+ zs0PDORwh-tw7s^S2--YMM(Hr)z(EV|&x3TS^AR>S3rNZRPxskm1u zDVagjqb>EzZ`<8L68SqYOxTNs(h-a@GT)(Lqt2Ejlf$9I2kSH06R5?-LaETrId%8% zZ(cWjKu?6DR+I9XKj!DUY@LGV%)tMq)_W<_TZS%0l~x-i-!encHX*;yWM6DU}Da zmEoQ`st^9%OlLa+MRnI7>5J-KPMOPuNuBBH2f-rT=6r`={b;o_ZlbvOIAZ-|Hi}p0 zW(OoYW@#ywQ0pZbZ4g2XF5KiR?F%qY%G0Jdd7+6R-FIsWkvfR+21}5f0L{1}B`*PZ z8RS$><{nWE%5rppSG^A@gg~tq6K6gPro?22uWh;}=r$wmk?pU=S#*+d${Y|Y95(5h z!L1tingqR!;pVU!$&dwGNL=KXeLtnMY<}^#e*`;N~c=g{f>Y>1GCMK9}}~WwC$g zTV#bVtulW+5ctV05l8IL?<#Me{-RLP1e)#KKIeXEjxg zj&)y88TTCkd?jPY6F}HIgga8jVJ!{GZm%CzaR*W>beYE)_Q{RLa%df?PT@1#40HJ$ z3DtA}lp#Yg!L;EPQ*P~13C({WI@TP<&v@`{(<4(iuru+K0jMFk`+i{f_Bp9IgQ!k6 zmvEY-?fPXILvMUmLG!TnuIcJJXjSD}6{;#RHJ6kmT$qjgB<8TLG4nmS)dKAYU(16l zVYVYol9p;lr6X3pmZ8THe8FTwU4X+=_q_m){0Yl>QIvg2@WdT({5_jNj`qQ)Ol`++Z|fbyHnFGc*LM4&Pc=&W@OF zG?r??_?%zxu}bd-Fv%B3YlP@JmF36wffkQoP^wONsP^Iu6$=^(m(bu*_(^b{Uo#m= zM?E;sUhT*lbxnAmR?z+C0wt2};e=|No5DT=T5J>O7q{FYuQc5Uvr+(b^{LuT;gavj z%J@TNYD;gH!TP#YZ$P74Z<|Q^pGYNjIM`mi?N~MlL~eChIpfxf_dci9GjbD#{E+=j zim89g91R?bM(YwWHBT0YZUIn%!pgnbIM5yBok?cHo}tU(<{bYH+~!oXj&U~paYDb888@};KY1`_4#xHk*3sGB!;ma*Np&A}f)fjVhREsUd=Rvw-JuP*svnNq~5k zlvNX>i~xIt7s2m8{&xNR{}!dq|FVJwqx_#T<8)%lS+XWMnXrPH_+0s{6TnJLc)ss_ zIn-i_}U%VQF*y$o_i5&1}7( z&Xcvic2t}!nDe7O`e3;P4oo1^?dz@q)6(25?MHK19_3KNz$c-BEO>v0=3t|WU*joa znEd{}s*e5yhz)FpJz+}Dwyylr5l2P_O)c`e@b#+TQVc1*X+`k?V_r(yMV zIL)NV+~D4kpdQUyf5{exxiFeF>Jh{m4Vr%&v^fW=jrLQURpO*AQmpq1i?K$Y4wsd5 z3;zOcaSxcNC&a1Md#Tl??cJCBqKf8SBmJAA<)VAEmAmx|Gyd-L6jo1PD$MrWTDK=i z>^xNo^yZp<{A_P@6nMQ2tsT-uA;UuJKpPBh3h>T3u*iU(zDe<>A^ zP!^0I+ne|72Fo0*mHTSX5@FK&`}Qg)Qu=?FPxKpjvRn0mobb`VAI|^q$nGNq3>b=^ z8s;-?!#Ca1v|j6RUg&Kn`nSA}@+2V+?rQ6h23}NB&X1Cce(+W#VLwD6APf2pd94Q^Q4jQ(gVyHKem_#w}^qp9IEj_ z`Toz*LEkTc<*IcN%!DiLw~Q52b>FAUxSm6r{Cp4`g@bJNk|ApQCDI2*G9&n#!hy=I zW%Sg}r9>aUvGc$%z=1hio@}18d#`8P9tqW``iV3rl)zPvn?q7UjuZNVvN)8Kb5acU+!psqC__wFQ$XYf}XCk%Yq;X#@=zULvGUt0qv-18Um#dC|v~)uwQa6~I7A5Qy@e=%Z z=$UF9Z&TjYX1M?67K;$0SbE#?rMein;R$EZqG`LbU2J{SG;Eb3vO(t}^!BN&Tp(J^MO0`V}4eX_$dm zXVZu8fV)1*0IdrXV|&1r%H=`5IR4EV_glAKtpY%<+i{5?=1|_5paC;4sjcbQ7b<_& zk36v{&ohioqVuy<%Gvr*;>Hn(*0LF_39l-uC*Z@)0teLdfu%jhkfzAKZHbilr`+CD&f zXO&=Jl^j(=&+9S|F==L)<-fJ**I`6M3q6xm$D}~6Ke4E&sDIy#oUx=Kj(06TUp43q zwN4xy4#7P?Tq66#9DcmJa_i>bsZb+>^`(;N;~xybep&A`1_B01qvX@Il=AhdNCN}; zjLWjd{r~)4%Vnt|w1g!LND=z#6ITsXkvi1cgK?6kf0hO`y)&J^`edLc~ z=qp9V&tlsYcX9c7TB$rLsQqRZ&+GdLM$8qdUvbk7n@%X8HB(5>$fNW z$h5Ez&Ymf*H7h=``;9q}fs_3arJqF+kJLvo zuziDL-DP%19splM+u+*>0|ykM)-g5rJHX>1T?-r+9enf5pf{ExjD4z&Zj3c zU*fvb)F@?=AQyvB`MR?xZxY<>3RDDoRu*y-fQzJ&Z|qR(xeB24S}K4o?vcZtYX_PQ zIV>uj1Nf)}D%yV&Y&N`g>lVmd-wa;|pmnr`1z7jXGDb%x6|3F5$BL~LAUF)3y0N(l zG8r-T;G)EHe+V{TWz&c=Gc)wDBhu2O7C7u+Ao$C;fK}2Y^cO+F3SiK3QXV~eq+nbA z<{Q`)lha3B9>KgwSVU8hNdO#$LZxVfUvKXPw=xCM6i}rVM)7xabOb+lw}ZWPKlr-I z6a4QYBJt7DCe4zdql=`Z-=&=?D7gN!%c&5=y?8ucQUn5#9X5Y-VCU(M60-YNw6sH( z&wvokG!;0qk&%(d;&z_ie`eo`(dOEm_QUz>yzNbkOf5GpfX#Shis8uEJc`texLs(z z`0a4}m!&Z^g6&Hg$)1d(k_Yy&A+z1r?QT4&UDo|Sw4BbB=r zi`+C+umM_)kn-Z_VTt0q?YEj6{?@l{sQl#@+7XlAe!BshQh@RUn9uiYJHQ%B=!nZ22-s1sc>&`FFcKI)^dzodGW}}AJ=8_H_FZCX} zRa4+Scw=??uiHQTErPZiM5G2QaNa~Mzma!uj<|g<(h44|{V;~cw}}#{(L0NG@Yycf z?6O_=oA-7#>y(ObY1RpP-U`1b=uGDcfKHEXd$?OADd+myV~g#qEOhhh{a!1iChwRj z8b`tNC;LxdG+a7(O9RDIt79RAUZMYhT<%#<~< zdM0?*vzRvBK0isV@ z1#GMQGn+z&Udnn87-gf%BQ~!Kh2N34D7IBrgcc{AVp7q?Hr+udz;38D_0SS{S9FHC z@IFl~cm`Cdn?SZFfA_o6-zG{a%vMEV%bl#_B*IyC(58D@2f>k)xX>!lHe7Cv326$o z3}}82BUyiWQ6Q*!*|&7hz8~`0XKat~~FNb2IZFr*;8i)ufl_)7w=1MZpQ>G+IrjVgB zmrx3&5QQ?&GRr(9MTkwxJY?ABS;o$__xt|7zq8I+>#VcR`R84|_THb*Gv3dAU-xz0 zPqL*B!CY~qNvb37gbaCh6caGke{|?5&b1ClrMTQ(m`sCDo0CZ7AEG;*o~GZL z5?bicq+=AOjBWheOyJ5;ZfV0RRASbV<~D&9X*My~8^Y=j5+=Hw^cpk7-$yV0M> z!08og26Kfe4+QmQzws4Cd%4nrJ|X7qLAr`8V}?wW5rj@#<2ForzZB zyKOraWk@TA`e|g}l?c(X@9fr{vib$SK}U1#Vw@&soB2o|he{iJZe3q0TbYqLWt1*< zA5WlMp2Jk<+}gAA9m+!PG%n;nXgYgieab^Gw>Cv3DqQ=h{H>Ymt!XKTrCZY85Mg~b zr5byBtET_3Xd9oKyS(0QTx`+E%GN1^WTzrVH@YAw(G`xQp-&5LyO zaq_|V(#-~bmg;rgkoLURjrH~dwJ^Rnq`4@j$9Gy%!vfga8Y1MQx*g~OcQdD)HPy^Z z9cmC@0iJ)4`v-6N?PJO&RpJO4hnKdL{+EFs=M?rm- zrFx)vA*6Ig-j|kJhlhak$#V(QC^6dBX5?W$roC)uP3ux*C)L+SwFCO9_!x}GRMjhv zpA|nd^@_ClZhqf?@UWY+(wlhL!B0>A{anhp+)KpnVn#{)=B=91kBFR+V=rXp8$_K4 z=_4epMZl(-KC38K=>@CSmSIUa78m0#qV8JErKKKlP%pERJfup09{W9)CwP=;B~wFA2{@$^Ij;^86LMWY9zOE% zJgT|bQe0-Vjgt6l5XwWWqE#B6mhhq=j_W#;pNx+r(+$)r9wM1joyk>RIW@uOR#V%L`Z;(*`RM zzx>)r!$z@~7e}f$vE$wRR9l;^i0X#LiDr>!BC3iUCz2&MzSjoF+vC?NGREh_mBON@ zZG$KZ)nN@7Ffk_Y&E|YA zdAB=Nha2MU&pVHvsQz@TTa5fenSq>S**7=qe~+cGdGJSU4T}BjPtPc8h8NO&+P<~9 zw^1_JRe0`&foxm4Uhx%%)8XVtGT@QC*5PMwYNlw%+xOZ`bkw!!@R|LeKD?hL;@Cf0 z$-`lC*>R-&BWf*qZdGG^t`~aRvTnbC&216N$(CsEAL*^RmgMfyN*kZINHaPurKp7)SeLsZ*TS+-||v{riR7g>^_r^I@^L`M-@ z>q8x+nHEkwD64$^MMv>32bk&JCwNToDSbMo`Q#6t z71f!~Fv91yqlH;^M|LtA=ndAA9QWScbQGN(ttlcNSEc>X=2vL*K#k*EJh{ySs*JWa zPZD}izQlnh_N1k@+*?i@nLpXQH!p?!Im175Bxzsqj1Q~(6=1bf;(=^BlbuW=Ei+-0 zgDDxB2?4II4vVQAs4YX4Ku^)>ngK#QI3#0NCN9UWORO83~Q09fx7^D0D4#JWXG3epL zR}Cj@LcDisWyhnTqpvuyXt!*8C6IFUzVt_XZbK1oZ}SBUIlm`rJ*Bm;rxvy)cwTtt(h#5b;Go)GZ(4yz+udEXq#{{gCt9)NN{TXBsu!tRP)ha3U+uvGu zx+8DuS4<77M%Y;{6pZOx*>Le)+dV1u`I&rZKqVSF9$4w=QsnyUK6fOWhXGQ@U2dZ% z9LEz=bXt>x1#h=}_V;^A;Lea$SABjtK#^%_`%VHSn{4xRU;BqX*3huACaE+%V!MDp z<9qwzoZAuZt4d1YBkclmD+@XT=FuUia@|XQWVD3~v=`|FDMx8Xh;V7iyDZ*Xb#j}2 z0I%e3k8p2-tr4p;Ii&lyF%Dm+U1J@HE$<#&yr}sxE7DaRY1iQoaGq?9mzR@pT^v)a zNlnhZOb-0x;kMROxyHqO4a30#hussF`m@@zU}r>?ehQA0(-^ zPM#8J_ukKYqc)aVXx!tch$9EF_kx^W_T|>(uQkVsO~a8dW2u>6afjadE!)n@n!c5i zj>Y<~*b=*?cE@rmX|x_=v|wr%#^R;UUAgv-BSJKx;GAJmyr<`l#gDGP@;XjE5wDe& zh_9&;H7B`v(_F5lHj{)~!$5JMO|GQF;S^*2#^U_Rz7R$pa)5fHnaAwZCc1D6aSNm|U~I z4Pa1sjiz;U3<*Nrst!3rjct|QLGvDe6Cy-05j=+KmmYPy>t|+i8C-#DqavZ!6ur{t z>msS6`6^sUr^<(lR_AMwT#tH5q5GPePCBQULD<X1Yy4`g4>{%fmr)J@jgGaYx-}r`<{8b13BP3I{;fkx%BQ~c``m(j; zrS$hcJWbV2fL@HyYlh9ckh0e zu{wXrQ*kYBdh8T@?jM+StQb+!icw*e6LI=}RLSWi3VdP7`M%KC|xi5Uuw6*FpzS*30=;-T3mZ_`c(x z55$9ugxr)uCjpkqbQ(78_ZMdu1S0Uxdj~HwKIeM>o>c(1V;(+2j{xtV|IDxatwb4r z-g}W%zGVB&PBOe7crP;=wD(m}?c`%YW9|X$P(RAADHF)WUg1E)F}&g!AyH52i|uxN z66bnfu05|FADGqcmrK^eDfquE_;=O8x&*BhMOg5|hcy#-GmwoKEVxB$Qq1G+q;o?n zMKeF0xch&dCyt}H7A(4o&2dQ4v1JHY!sh<(!^UxxQ$(*L{|!ze>+aN_SIFy=4oygL zFdzI=If1vk$!6a~&+!-Fy-%SL-Bgs^Od?*h`^B$sZ&)4zU6)uL8XwABr0})xCHQ9@ZdILo zzMdvHL#1=^Q7x;H{JVB^fvt^;ezcWCCsx)d|AI9hKf9ZhCGjugPve z1R4A&U(c~l$u?UU%Yes|NkxzNI$nv7JmVr*Ks6z!k_~kC>`B&dbD+z}u?hATYabzEVpB|+X_-`9e)Q~ zG2^qc*xFr!&!Zzm7d|sPuUZG6vLwGj0*H;^HL(^U8ypN7S>?qXKWbsqg&Q9-R5D5bN1}uji{t&m$Xv+x!%XnrX z#RNOH%uI_PiV4am4nN#sh*>?V%Gk?U?Pp$1HSzO3@RH}(MEpe`2kN9~!}4-#C(7<2 znB!R-047ue#iH3fpzB4P)|QR{wEemXP!#QqA1ac>7c&1w$;^l^ZSlsL8m+>d5FV`` zsqut@ScTA*#1}w!n&XnMb~zv}@a({QcXi|=I34_!geBKqjPJe9Hq*+8+*ncI*10o- zsHgW1;zTn}qpQq#0(ESN8W z9K4=*$6KD1brbB=(H}~{kXXfw=Ue1-GDd;Lq=ZJg>!oO)IQNqbkG#qpIdJn^WZGnx z)`xdP-_*Va#S_job(dJQ;_3w8Oba71@dUK6b0?sS7(-F(9v(?at>oAz`!HD{8D-jW z@>EpKlU<}W)yvAci}%<`MR?`*X24r+);r9`63+)XjK5~WjZ%$c2)OQNU%dD)aKb{h zRk8N@1tX6%dTxj5cYX|Qy%mLaRSw5oirfviQY7AxW06d!=bo=-m|Y?xONUyoRBZ~i zT`%<6)imA5rF!0^Ci%iv!h`YOp++qq=N9dU1&dd2PXVA>nUCKJmm_x1kzKYs4xGK! zdw!xT<)Sy%ShygzT0EclYief9CX=A%`Eqr(C>@Ybwp-&D1vvN&WSgS(GB5Joyw%8x zQa*8dxn3#SVGXaG>Ox_Uv@+7YCxod?BS{t8V<`@OhVdVokqWN@xp_39J{=`}J)5Y(0 z9k3FGh-xIWhSg=Jnqi>7|JgCjg)HBG@v6a25Pra01Lc=h)XAzpBu+uV>h)06Mtzi-}3Q9Rcm+Tc&} z{LTYt62=9`?n7Z_hmRa#bN|!Vk~}e?{E6WK=1S&{M ztd3{DTKrX{gL(M8i$x@M{1@U1nEv|kWP@L7!>3Bz4c`b7+l+FZ(f}7!Kjl)ps3MV+ z5UGCUiUsYRh0#_tUBq!zF-oF~{RF1=pl9W#w%K~2p5pmt1(`tB`j^KW#qVxR2mz?g zwamQ}M|1EwLJa!7(GRx;pN^uU``|WM7wWSqL1pZ>T!cuRMuNl8Gk`EN1K52*n1vGO zWG#i(loZ>()%%WoH@}Vi%p3p_5tKtm$sXY6y$*B~1<#pHohK*F2OtO8d46pE| zMGp*1k}caN{}j3#9yrUX5ieT0Wy}0DTr7<%SK2=xRkvx36gRC4VG(zphl+Ik`Efk+ z12;Bc0(i2#1eSO2?aqRk3QD$Ol(`coodVT;552r_CG;7){!dDzke_t= z(c)-R_%oi}*nJZ3=)zcAzD>{M!0(-Mcx@~RNE!1i6<7WN8(mC7Vk5WhDj&-yPpe;9 zZh!iGjog<98zMxN%&@*>S)WE4t6N!FJ=n5sWpSdT%)^7A`|ZQY6xZch@QHsG*u{{? z>nnh$WB!P<@w8x_t7lj~A+ZycX;O2$J;%KE)rt6QG@Zk14+Z1Cefu=toqv$u$X^9OBe|E4AwM8vSd7kroNtm_E zREZE~8+9_(87F5y?NhPpDqQ&fUJAp&8v)yN{`#ZM4R=wV1uZCrY02^SU_yIP8Ssn) ze69=VoD~Vs?yK^p+_nu=Hv4$50&L5jq%VUR2TAxNa+~%d3wDEn$6))89S0;_|KP6m zYa&md7D8r#YGhoRiRITxkPH=CuHnyk=CFRA6%@|=ET0@rzdY|Fs7OW#*>t1zxBx+w039@psLhN{S0BcWSOtlC*2y2xX93UhaY&!2#jk zD6Qn0?fi(Ek&zKWx3dcMC8VUJnE7w<-Uq;kb@{z-`%u!*$ZesOlFjwJzHAB7 zeNY_T!(ze879cn3G#a(bZDn3J>&Dmjpbf>v#KbG1R_SSjf)976uCl?aCp}H5pTK-@ z5>GR!31H#9w#m9r;u(wRBIX85IgYg2%N3CQ@Ez+aTEdF|HeKm9Mo4IUbwUGFx1dw{ zg5;;P0W8i`jug`^uXQ9`-8SV#ya@-L_R-Tj zwuLP9c$8hqyq@DtIXp1H=QREk)y427{(qXjJCj)YSAuAHP;Bb6;1BxqP4=lPh-6d$W^0>bAIckn?%X3MPlTb2bebDd zl6#WN4%JbcX^VX0Iw8#ZRRydz0Hr>X6+;W^cX!T;KjV ziD%g;bL8;hw~<(i_jbaEznu=T^7tZb1M0|mKYDadC}8*;dX5RWvpj1mmrsGcys*%f zX(EGm?&3gbebsXh#ZMV|1sg(~0s|A1^X%`h)6=#I6tjoe+1clY8)>Pi?cXFOc=vh#8brl;U zMW28}ud!j|N=r-gZO{N}YHAdUp|-BzsjRA6n&{9;y>bRQakITylLvq^0zh)IU!~MF zGU@?^zj^;SHzWx~ccW0BOPd^al) z!qvy5T=xZi+lM+vZjP0YcEF}SA(UTP0`PLMrY7qo;Xv;Wp?# zPGL}GdMm9Prdu>t*6#v0JHch)8{7G@C_+NKN#OLM#d!iC{7eBuSsMl*;-f{4~0)zP6GZJEPey=6tTa$sy5$B9W9v!K!kr{00O1t#r zNxT;af?yV0OJw%Z8N^iLsFtr}li}K0R>((`ymt$t)NsoT#YPDWz>Fdx~a z+?*2H?cU+A-S=1K5e_leXd>MSE?spY8<92Ly~qDidi}0Bj%CB(>j4yd7(Bo&Q(09# zjzwrdi6oa7mIh(Zq5a{cKEC_^&Rh9^$qMO|Ym5Yb?9V*r%|N%uZYd@tZe^jhq-bKY zInH_GK63W+#D!*sBsae#4an}1v0R=7atXaO&BUO>JTL5fMB`NF7+>%epN+9;!3 zi>|*Z==JT|Mu$zYN`M8IhkvuJjB-m~O4JwX=!#cr%}L_$eiXad8|9vS!7JywH|zaN zL5L2KzWjMcD|!Cf`xh20A|rE+Q$J_t30(Su#M@2LL9folP|$`T$ZCy|v7jxz>EE8C zR^_Aj;rvJY4nbDY`26nXA8E-N!#$--5qE*C^bcsPA_^%tQDqeSG&8^?d3EzF6$#t+1W?C^qB=?M-NFzILX2f z>)h#GAtkn&DzUB^D86`}@}unqnSjpBILkIlkwXSGZAN3A!fq=>7Hf93*5tegPhDo0 z@?8+^_l{nv>fQ0#U*pN%q;R8+3d-6r4bi*D{@z-i)yq>vpMT4Bgg6j`B#gR?lPbKl zQeSgvYkE;2CrBi4AG@5OoMfr9{0GEOIY8?r&cLtDE$Y9#;4;+nXG*ft%{7mC%5c~Hf3pwVe@NqL3Au>?(0ZTwC5_c&t;S$IH`RpbKwG@h1JEQ;yK1%7H!%e z?{MjBH;N~u=u9jn;{a8oqsYHI8(X@jF69$zF^mdc8?NeX1$8<`ClvN0-%;t^8vDfk z&xMfS`H}Jm;52B&-x|{f8HLruz|Jm6^POk+{p05O&2a(_BV8&YmLZ6blC_Wxukuv@ zzr}gYEoF7AItgwAcNs+|zr8h#B<5x-+!*4b!ud7t@Q8T_nF63j@=#SQ- zDK0LqYVlJFMR&)(>@oZp^eu*xhSSY?uzuC1QNSTF>qcf54b6|~zP!VBbzmuo>m{gI zE?BUL##2yZ9ykzUfA+^FahhmGGYV8~O2guH5a<~&cvnn_FTKUfA3R8EU+%EUb#w9K z3#|=p>n?Hb+XnD|II1--Eu25oZROdMCZslJ}p@D#mJ-p7T23V^HX1!t_#+y49i`0(?X_yVGPhf>7R>pL&)w*%N#Gu4h+n* zA-!zm{SDR<*X0YL1+ukob??t|8j64m&{wrk!v-ozQ0emJGiOFSbb=5<4d6~jj-wo6 zBLo!w!2+q@3b+FL()S;Ui* zG&DLTRaogy1g%M3xbRg)tI+0&nHH8)P@auW$jRE^x<%CoTL8KJ%3gF80p>>aG_va_ zk+3pyRtU^tk&{lkTY(O;9eXL{D-|PxD>rbDkSy?roo}4wM=lUdkW%H(Kq$q{*Zai6>(y4fO*Xpexpak=;dx2>TZ4F>f zxLOi4InDZXLT|9HZ>;{#|NHb)O7+pwb>wk%vpDR}z`yU^{5t6O zXT2QcxbM1dqewQ_NHYlQ?|g6BmSd*kM|n_maq`r0<)1uts;RHn)-}ZRJ^E#SjVFVUNbNeFL*PQBXn&ou(k17U^2*kGj@sWmb%kWD$A0APxcLNW?8kfL7_M+?D=LIqjA=(m zq&bdRl|A-Y89PmVAUa&2HSq-pfVX*KYucN%bc3>VxXOW{a7qC%RuqBgWA*FRX`|Ud z6p6dpsQonvhIg>meC>~DyrA1kl89roed7ro9)a7b;M}f!#xY7vASJKrQtsMx^tYce zL4FcMC4c4=YyIHU8IQLEpIjIdxc6cI$?sXFP z#a9yba#Z8wlhviEnG)UI)@K#`QZ!s$K@_2)m!Gkc9jvo#WNl4Na+*Mf6s67mnE)vR zfLxrgUfp#d`db)mROG#j%z~UUG-+4U0wscC6QZxWiP$kDica z3XmgC{)GKaJTFdRVxtQ2Cv99nhK!nd!&+G_>*B@vpC)pl zYT2QCjSv#(7<3Gw_P-yaM^|3A=c|Z3HO&e+Mdn^_Ji$~ljnT2*nH)qC+Gsf*9#GK}nz7<6>a9~9rF^oaIca`J3bKn)YT^G%!n z|L}?bpZzjW#fqy7?DPs-9Jbpy_CIPCE@fK%BdZ?BO10hgkNy3;FR~r9`v5a+xy<;T_qhuuHH$Sy_MlZOzNhtrVUmaRv2$#y^3N-6&eRGDq# zrw<+K#U)np8hAoJ@_uKpU%!q51JcIRvLv#`b7XTkj+{MvmYw}UW$z0Toz~#vbrHmY zw#@is0FcOq3l|Wfe;^+EAbA-i==0&KjL#7Fm& zzOYMM%;kfp=O#HaWhChmUQBcF+-_FZ+alWz9_Hmm;D(o5fGKhc(bU8&UR|*K{~le zKLNxphm?>vV@{BrwR-kHU!kLD3^`+Rnf$0^q|baGJ$iIRCcsu%t8qUgqpUH81}!XS zp<$CaxNl$8b}zG?yLR26#@drTiX@(GZ~0~fT;IO~m+3u^IEnmZ&1l|U8oZoS_1$?@ z)#t=RwueaBP+4*OqTJ8Sru-ELy5fwkU|^}q?4& z0Phvn693C;Sjd%J&*b|P;py`Y(MyP4{4-Ge>bkq4kZsq3-Xma zyekw`NkcY@wB~QjUh2I?id1qBWsE)Re`T9N4orKQv3N5n!+vN1QClCKTsM&WO`FsM zD>uCXFowzAfY0m0)YRY4%r>^s9DxjUbeG7TzhFFV0XTkcX{pRj z0Z7D^l3r>I7$<281UPqHJ;lMn!NW7TcnZmc_es3A@#W>`e~^}M3&Q2A6N`?G?(Xi! z#>OZ$BosoSOlC329>pxSPk6HfL`87UAG%Fhn*6d$#?wwm6+&M99vjogVsTy=4GY+M zoJ0ik)VwYql%Jv=SG%#kx^M4ZG()R@V8CN_A|F9=SG#(mDmzc<=1rS6U~<)X44WGw z{GK%BTIOY>=s=wW5}nVk+E>JYjFY;cUhb7V;xzWdzi(TkphZg}z^v|EJFJ}i+l>*T zp`=OK_)3NmZ2ux-Sn38IFf#HrH0v8h)-LBgw`_g6!Myt&9y}=D>|s-X`D-wDf1~7B z*?e=;W&exvq1v|+lR~%Dph!$@$f=I9@$$c5oO<3kQ`vhn-H_lz4R+7i4liPocGR@YZaO-(9FR-t$!mm^zHspF0LCtd6DEGpP|S@A%UXbR&km_`4vhr zq0o_1q?j-}dtX!_?g^`1eu4MsUwc#`KHi-MGP!eZo|T&H<76!joXFZr!Q35vy?{0H zc_xaDy?$imyCE}@tfiHoUg)Tir0#U@@N3Ativg5ztbTvAW8b+L1ckEuy({8CCqjhik^X3av8P9hkG(mo|zz*Jt7p179|AntXOlK})jK481ucqh=fp+&` ztK(wo+}ZL{=y{35cb6`GI%`_{9kAqic}8Pflp;($UDgsUovbE>yiov4uy_$sF(@|M zOu#-K9`}zlAOfdLLEl$2yT}Cdn+h;I--fg&|86Xm0XJ$@JxmIC@ho=z{N(G>V1_;V zXQDk$D`mcNXL|E~j=`_cp8Y6v1ee#WG5NM3+>sI&rJ9R+upl`Y>$bL~A0cmqZ0=k} z5_|eI=Pn*VJ*_dmTsk3vfk+ZP+{UHb>=6IT+t;#{Yke&{-B9D>9R%tg>!u?6)YDv+ zCi9~>PhhzF`uc|YF+w^r#=IvbuCjTge&9{hPteT$|4q$WkHZvOY_V!EX-cB2i!nIEA*){vx zAI5^P-|H#js2<8Q`u&0{m7^(+OQ`yvx1<}2l9A{XlSdW5K&4+@oSh@g6K`i5uqz>= z)=&X=5VQu1R>g#ho`wiQg_nqf>~ZB0-0k*@&t`d6aIjYC%ZnIn#fJ~+=wf@XnSMon z0Wql`16U~WQWyvZgpbKKWm*|mN1LQ#WVm_1J23e^@q-!{8c0HXlA8;`ytp;p5NvL6 z{>QW@e#MO3lB*3>F*c^6foHm(XBV;Eu||s4)c=W^L(bm6%q(#Kaa9+`Hp&#m16rwx zvx69A*888vcW!}$s>=A*tb7y5e>?V(IOv?pXDnWUWnASu<}U>P3)7i1;ldgk2~d_` zE2Zip{7=@_da@T8ykA+!1egy$e*E}Vdb#aHI8+vWRp7Uvpwrw4?qzJ|EV=<1Y!iBIIcN(ZLn;b&1~2a(f!s}4vfg^#x@g~t*JFnxs% zY6G9)xmQ2!nD|3UYcX&cy!xp!op?p~$A(3IJ7G0@sX2GiFfgYGTIV?D?mwjt_6^}& z*V;1X?UcU7sOIwO8|4AJ+ubcW=1hW$E{ln-_ZP-V+1ikkG_Ic&LitVo5Xg7rB{`Uv ztmlb1wOH!};Ea#N>3SuUC&^e!`m)8KeDG--5r1~eABnnrpdd&;r8{UP;!k;i@A8q$ zBO_{HP3ssrE;3uy(D)xrl6y(FH`;w2K##JU6IK{M9R;PIpL&{Jq3s25LZEFkf7YI- z5_acR#U2MWw~msdQvK zI5?^WGK%EimO{tw?)``;-=P40rU~F1f?8*1rin$nrgeLwxkkjf_kj{;f58^iCG=H+ zx5w(HsUr41Ug=}I(7(OgJu_p`bkSuAy`$$rZ!9_0F4l!w&fNq+`9=sF3n{SW%Qpp| zOqv@8VX}0kHq(;6nZr`sZN?k;2=d;#>RTyBRvZaonAiy3qhB{pDonKFu11stPFAaO zU}Bi7J8jW?sjHCq#s*plZkfPj>7DBfp+3|aOOqs466dy%I6zJrQCX&h{^}f4q(R^y zEw&Zq57k5Boa23i|JFNm!u&-3%9Mwaj2)XoNTPx8U~Sy`Dy-F5-ZnW23Ghr-7ZExJ zb0!dY-jbj1Sle0Z$dNnT0MIe|YnnF_v&4BwEd( ziPu3lgd#*v9J@R>)U;o{y2og47>H)cpJ+5R`rWY)AU$1VmU0E-?3ebkp3S-<33!Z*PykkeG_G^h({&z7O2t?lV? zpp&AYX`y6;_@=S!p-g{k>FQEj`2!LW&r|c-qrL~lOOZfP64m1E3(8KHexE9{BysX~ zJ$d}yb2F1b?OeyhSPJTSVgeKYmyViO7-)uPyq#ryW}r5#^!%N#<%T7Dbis??AyN6Q zDhzwfxi&Tr?0xjydncB1w02sKQ`+Lhz^ChIs|Z!q`WQhb+YFN$V$)uOW?xY)aW?<5 zo1pe~WJn1II3XeTZpOohSU%Asw}MZltU;_v{}(pZ)?87pP-VDrY%F7;=kKg;&F!_)JyMf~+QQU`l{PSWvV3M&LAeK?}ZFf6%e3(eMuglS7(kA-7iU>u(=92*nJ&G#JTm= zvAWYZVa}RD;xp5p)6@{rUS@RQ?CXH*vIfXq=*;x%Wnm*Cc0w#ZSyRIX@y6-WTOz*4 zofovA$((Cc8n!J4NysJ=cLBL{Jmm23@MlK8w0Uir{8BpJ%I7zC?0~Eq#un-pnHg0m zzn1UNM#W#(#ZeS?He9IDfsSGN(>{1+z*Sj1D$v1&>A#KUc@9a@(PZUuB>X@4jYb{< z5mhg4)f89~j==y*j4cs4_L9l(;ut=#K$x+Hd@ z1M)Nyjks5t&WHbTU>h`Kzd+KR`kobMrmPw!qD3qIJM74D+C;g1AQB>XI1Fd*ZZvOf z#b9mT-Y&oxn0`QMu30 z;0;^;c#R+IWJtber4m39E`RR~k5ef13KjjpkIZbLpK5Ac(*Pe?q-#iB3 zK$AhM=X#=d<&jfGS6|K!9)&|i5FY2Svyh0G>uF6r!N!($)tbb>{lko>2ONYJi$pH3 zdLuUfJnKf&izB3`UwTv<3;Xevc-Uj=Q^$^S9VjRpnXl zIvp!N(QYt>tTMwzW7)JGcMTjQySr;%QdPZ|dizpHtFns3XWxxEv!U~85xMiI@|tH2 zozGYsVhSn-!3e1!hn1Y}`!W&|xBTgfQm{g`K>s&-2K_t4SjC2ztt{2+*qk|2%vQO z^PxFTqvBz_edpJj8i$9!={OG=8IeGi^oNztxz3z{`q)aucS%kx;d?N5yuH{5KC@SR z33U;sb+D@sfB*TzX>&C~*yMlO^NBW9S(JTfVGP+Dt(17ZTryib+q%s5g18G8$0Y%a zM71M0zH5Vzn)hEq{IvI|RgphaoO;~+cPTl_@lIW67}SoyzPqnmF;miL{zx5dk>l1L zX_B#K$P9R@QRWT<5X=QfQSz@<$MVlJ;)}0GTaX`wfR&Qw+(yD7;#5$`m7kq@4moGd zoO)+GmZg!^b9RJx8ZEsrTPNZuZ&(BcFd;#qxw`~XI}GmHTZX- z^=7Faw}j?6c)+zGtzTZ``JX7^o8UQjrKu@E?_iK)N#*AWD^z)E1NHa{y zG;v%juy*Ol(<|#<{60Ha8`>!NvCQSJR#MRX1pGb<>dAAx*w_VX@V@46LHa{V8!q+e z+)AAAJYC8uDJh4li%5e9bHj2Y9Xs{|YR8~LCoLG&5Imj1Yk99*HWvnLEifs31785Edhn%fFl3o-^*Dfp& zO^V)}jzYxLDl}A|g5c=OJ3;G~l+X~K+_ZyG&BVzknQO=;Up1`uBX-!t(RGIf>^}HS z7g9f4dO9D?Cr=?Qn_p@U~FYi^)&=ctJ|0Fy%GxK#^ zBQF(+D63T5&7nU#q@?GfmmTjz)0&cD(Z;ilqJ7kNABX#w#b=eh1$%tpA*HpmaKDan z;i;#TD^Bd3>`xdWG14{SIqy#LAOA?Zd2)o0c-{!D<$!GBJ9um>0ljL)YX*zeOOcV`N?*9ve&u2)RPPSF)}!S24_CAz3P5 zd@2K~gpc;L&Tr?2kN#3#3qRrupWK;wydz;@?B6M39@|mo0Ox+IH@g@cT+(@+t30lc+=r<>{EsTAY1ZAH~Q#IXS$-VpGw!x{4i4Kitbi3an zN-uJ1FA}0ZUB&vgxUSrFn6${)ZmxH;EBHi|PQLtbP z19Ij}m<)fLr=d6#dn??PK4^jXjHco+?R}}%lVQF|eD7S1%%qW{Yx8r8! zbrE+1cxB%X0A4vF6U2%rTR(qv+IW4&cmJuC{9$Su?coN%8*WPxw>zK5&JA1hNv$ko zM;w5HAeM;^(DvUEM$>hyvRD)Rh;xiRt_Zc&6~sd<;vFM_Z_F(^P+!G zo;BDzqaAIy;o`0hi$rRhQGQ8Hbd2>8#;;gi?8iPQz1*}JF55faSowZ4hJ<=~z+x;S ziO!C#DG6??;8BA#WYKik+b3E3)L^5>(hdvfCGD=l(Prs|QR3*|H(qIQEXCj>CK$rAzZ(gNAzX0NNrxbdbaSZD=<+aFDG$9CrmG;O?)Yj!#h{&Q zfvpeuo0Tp#>v$}ibbn*#RtZs&9A3?gbf_DGxp8F2D?*gm<8fM!NI?2B*CoF;RA;I? zk3EZ=#p6Iu92Eqz<7-0z%#8L(5>d`R#X8^MBMuP^kuQzhiQ@m+O@Rwjb=TPCkia6R z!8(c$F*HScZ>(wP=i%6rWfnyD5q0>o39<}U>|#lt8JJ>Rt|;5S6A|Vpjw3h@<=dsW zATd?vBB`q$4BY?rDi#cm(O1OXlLMc24AxtcTnF;O`H8*?ba$MF4Ut+a^R*%MI1RN^ zU*GP0#}`|RYok9?({tbapJPoi?3HAhHG9%i*m}{sqxn7jByD#9-GQWNX(itezMlfi zC~5IDn!Ro0+<@&Koaa>3j%k(O%gP$wHW_&kf0-c?i2Wk#?E^_GEmVxX*E(d3*NVI- z=^0-q?<3X8JdWuONuj4WZTIJcFCB#(1lS#d1oqB~#ffxBC)noVgK$eYcU`{sZM#=2 z6pc=4=oox4k_Uy3k-e6SQjYV9a6*TtkDWP#jBVqIU04$34`LOr9@1@U!sr66CdsQd zF2g68wv}ry?XumKk!AjpLo4})h;Ib{twVBSKf)UN+V`H8=_<(V13KTkpGwLCHBelf z()3J4e1&v#<*T1^qpkwyPbgC z;yQAN?QMCs>~p|=ZBM~PO*GW!)^V<02sXUnS}bTaom2TeIXK`T!U;HT$z4Top!M>g z8Arm+r}8;Pt|h2DD0oGsmX7rOr5*@K;s!iI(4eDhlJ-oo3JEoA5M`1aVeg$d zQHZ-~zQ_0~M_eb`p9TGYsth3oDf?O;L6%KcM4Z%%#V>wk*G3>h2n5-ZOAnIu`|Yn{ zFQHC{k;JiMT@y(OIUbS>RTP`b2NBl@D!t*d)Q*33t}|AyHjF^f7VAO9Na!?uY0)+L z!$$#riuXujBKkBCk`AB(HWk4~*4j~v7tj3;AzQ>AliLy(AtCXeld;S8SRYzySN~J3ha@1X(Ld4wn8H z0T&vq!=v;TaoR9II$-=4w-JO762BhT0-fkyl><%NG_I?+-38|R4i^Wgg{{r^(v>uPP$u24ypc+rEU)LNf`(l{42L@fa-@?xT!TNMLh^4w$55t^Nr7S@9Vvt)0SV8~yWl zbd_Omw1nGyw~L7VX9v0^+++(W;Q9(~>cs#Rc76m<0y{n&g{bi);BM)s{AL0FDe5|R zY6P*$I^m_X+R-~%;T;QH-9{i8NGS-l!iHSWf89811$ zUmscT7_k}33OWi=S+O@86<&~2kr1Gw_)2fBvQj8ES*mmX+(Y`px&y$CAA=AH0b)@P z=Axo)n;m=$mM0J(Ci1V{%iOU-6C~Uai@rDluwb*hJ6L1y--E4D6A-3sVuKnySH-=3 zMr+v2K|3kfN9DyLw)O$A9~`Zj>wtgizlCA$L7pI2W}$xY0K`Duti;RI6J3@f{_6+D z5+TB&i;YvDXS^kXi-=^wVE(1#OM|buEZWj1xA;BNL>lCWl93ze9#6qc^k*WDsGZUu z+A($j$*L+JU`=If(c{0A5-`i=mw^1G=zy9AepsR7w}OKFpI5*Dl9aUWs%;OvVem1! zh7}pLa7iPj@Yl+7n~***34{ZEN;leb6G=CCzWvP6ueY~sK|z(papN;m$$)1N&{lfq zc>g}ypk&c)a7ypiA6tU?WZQHf{!i(R#qu4}-ktQTyrzPaqW_ZH?2}_Ef-Kd!(DK`q zHszReX`+SlA%TuyqyL263^;rHtzN!1)Nsx%4s>m}uq<2vZbFXtDpYCU z0;1TD;(=U?xOgQ;hzAutMDGK$PtxTJ=8BW@onILv2@S<4y(=C-ueLCXiJEA~IxyGI zOVkqpW{zYqb`?G6JNQAx_W^%_TC=DteL4Kwz5_{SQCe4C=iP37D-6aZP@Zo%!o~xT zV$euH`87l^%esEE*SA%+s zLIe<4Jvsgsn{3m$&ERDBRnac=`G$+@2Z}={PXxf&gLRJ3<{`EK{5v$=nND~tvsx-ybnAC@#o0{ozI+1NJ7ZU7lYaC*B6flFO7 za*IP+u_ei(r`v%}CEqz0m4D`mPy~AK9LucHEGuOaP`CZSLc_`eVI|aDdO5N(fm|BL zQ2zoV&x8OL8n|PmG~a#XX^NZx2lzAgEqlCxpw4Y&WOXXn_C3Vv5uYtTeY!QIh}9TfwQCIXlwurJ>Bne9gK$o zm3Kwb-qW{&j|UDlXl<+^DfF!=#^RTHX68yCB}vchpO0`>Ki`6I48DC=5k1Q(x#iiU zTWWEuHSKZSI*aneEn5{p7Cs^az~$f6nfUBuE*;RmoPX~zD}!^@OU1dX0R>16*!4oA97LZ!8eve!zK=rxZoAvwG$KjTMg)7>7F_ONaTEHA!BLf3y(dXpr-z3$My z#q2wN3g7O0Qh7}=mTyWUq3lzYPtv6wZgc+q{fcIVp+B!6l~h@YniQy&ly>F)VQEFP%B^Hl%_~(#1L>Lo zSllQHoW$NcEq}`zf+3}GCpre?;)@rp#`0)cTlT5oJ(T+o9)u9alyCyzJ$E#3!E?p1 zmV`jKjlC7X9>BZ(IE>r~@k$U!9k|9wsxHr(uQ(u6hi%8|ynR%}vxneTeyHEb z)(EVvpx41z8XHe&-FN^grRk&M)I?_z4GX_{5Y>Lom&cIAZCK`>&b34)1=aFY+3QIf zXn^~ew3tAVfnR+$Dk>^S%s0>9wta&6nRhe71yc5~^&I6&zFQ3pK?xG~%hV9O=O>)A zyExWnSw#Vo#Y6{rTN7TYaGm^iTV->b!^rpd{nbxxdP>E`#5&0mCA~F8%JUZ6wy{V$ zTC9(@Ry}2CTUmgp)Z}xb%iZM_U~%45zc%y}pWhkQ23KSY9pkN{J-gp=>o_covS3+^ z{ITn3%QQhu9E73?aq^=*Mp`LcX3B(&0}cN(+1B48M1|~!zM+=n?2KB9Hco_~lL`>& z|8?(ZBO68rp4d^(k9U%8$RW815_ndu{HyO(w4(F$<&I>^kUZzE+a@4grRn9)&dhvt ziwO+0)I|&L^5VKyIHoO%k5eMJ}_TE7wuYpBnXJ=>Ko^yufpU9J(oFfqRkx74{ zQpUDhA+So(Oj1YTOC`z&$3dXF`3T!+ZJ&2V>roC4k|YoJ?BKaoAZsE9JxS%2LgEfV z{TmkekBmtZAGQxDEUooXl9C)8v3L~e%j6--;@p*f8xMvEOekVYBsgTat6SRG)*Ma%LE1nVl-T_>cU;LjTkyKB-A2>E*upqQ!a71-Z5T4Gk?NF{FTLyGZ`V^Ai>QG%PQa;GvPDQnsWM|g4 z;sf-#4C1&jJ~KgZ5Rc{oQJ zTJql-AhqK$D472JQU(0Z3@Kxb*k6RN!Wc1bNF8NmAESFBB#3QV1-60@RY!-8R6OqO zy|!O`>n6`4j8Yxwpb&0--l1ju4~h2I8-?*P#rQxZhA5+cL@p#Z> zsAN}HR+#*@+SHIn`Ok3@O;!(MKQ~xMx?~vF$_SY^-+-i711TdGs;eAaTo#bg=1<=4 z$eY0W1s74V*k%0ZnYyfF%5QHSL;lE?S|Y?Wtf99A_jHpaeG+afkeA>1_FAnO8dXua zDFT+-gr;8l=xBs+C-1c-UIaNaDhgMhy^7Vjdf&7qQSI{o!PZ9;iJ(05ioRH#zu@3AYY7(Xm|J2^Q?6@1uX zh7vl+^PF#Txx_#!1&O&rn=dqexXGV=au~B_QJh&LJ)>g&EjJ~_78SfNPwkF(ovD=y>a62GgP(*#>M-<&jI!`f zv4}XV+WYtKuXofj+F<>;!+R$jx{pH#sTzNetrC^^P(Y6BEa{;l_Sv)F&ppNkc4BV+ z4zUwD@}@d$s*^JMbPrlAUFX%+Rg9jID7fxBP;Ca;L{I-19VH8@^N5hP9)h?D4e~Zz zbGde62Auv2JH*9{7jM8&QiaXv5=!T7J9qEcank_r=;GhdVBsboOI--ys#FqZZ-U_U zb1Vflj&RbI;08Se(#P8vp1SxeMvIcUW-V2V``M$AdM?clLz(z85GR+-Yf*Mq3^{y# zk7&eUsSGu185Z8{^*Qw#30aQnNw(c8m7KOi#C3bUJrj-tH9t{&7_RnXr0T`D@lK~x zby4z1bz62S4bB_laXsI`E0BvEOHfE@5=f$Ly*c0C8#6Q5IC)Ig>1&13qn*`hW9xe&jKJ%Dh&_7!pP~XluB%J!Y2^+A7r~Xm zmd+-T%>cGFgx)+ya08jQVeSRD-QB!dg!K_2$I-Z|U|g~-9haP%LKTF9g7W(5j=E%h zq$1D?A&$<^x)gk^Ee$DHr2MjNH%R;cl8KS~!O!Oy89!sB^T+Ce2bVtLw`|~(U(HZh zzHHS3wq4jD(WaB{h{*-8JpgaU^DVueaES2@e?O6=YtcHSHZ>fN4g`qmBnb`kGziqf zPU7DhQ~y*32)3bngn0$YVeunOK>S!yt-~n=^ljGx*jtg@7FwLUEn5=B6|psCxyd?7hT89ZvV6XbgXn=@kC0DYZH3JCxbme? zwLEiJNS-rYmjw@*#oy@H~uXUtNE znFZAjyU)MPcb&f2Fx9(2{8URuz0QTekmzXhx_>-Yyahad-@f~uZFffPQ^>#mQ_84? zH|A+yqM#P>ECpRRvN}B{ohSS`!===Dz24dNXV^wNjk!!dU6?4{TDa=eZcz8^Aj(3m zQut@;XDA3L{^9=It+7qM{wYDc@cHZvo<6-_MHmsHpnl z@3>YFBAia(RWxWz7xHwQqa`%P^AE>IgQDBP1_L7HhA=VA2xK2S_BR#e61LH6!j2l~ zxz3E#AB0*iLK-;>oRkXlkp8SL0p8b$5W1ivuecQ+%mUT-IPJ~Y#7&|jROgudtR&iSmUtHVxM@eX){Xa zytiSXm*#+cZ2(F9@9ix>lU(ozW4$z|;x^wHkMbr^%e`MfEkD``dEWk`FMdu!!puAO$+LTzr$Qn%?v6+?DjoA{6Ws$+s(f766l^b^)h z1COZyVX)S8y7o@i8zE-g3f4dw=Z5VocJCFqL2?|AFw<_SC2K~>!9^o0f8Ajw4sJ=X zCdFU(Rfd?#&;8R7Gf`lo-15a=S=Yd;ZE4A>IxX?xFR;K(Y|l*Mlu9H^C7IVIrkN#? zrG%@~42Yf6GwDkU*#ng9+btPyVv>RclXi1k2hNXL;3h(k>xP(t7Y-ICgZrWNcyHHT zTWk3ivv6unKZ}r#qE5c**4En{-^0ZQ13J6hDLa@mBnfGqXfjDk9J+;{UR@Th4415( zug|p)G4-4eqm7GEZn!V#gqZ2)7@1X>!kLA-M$wXT&*IlsTs74W$+U^&$6?RE-k25Y z+?`qVB9-#mX(Ee{DwLUzim6MXMJCn5jTnIzvaQbDyDk0IfN(S)?P&4mRF>tEcdKBT ztP~Wk)X5Pok00u217?i7*eiI9mv{7sZ;;n=b%-`e!o#JZPI1M^al|ygGd>Vh7$6=D z;j@Ftb3o7IkEM^Yh+8${b>QUTKC*3BVuBV%E96)zpH<}^DoS%RMw(K8_TulnR+~wQ zl|d7CSEw|qg|cRl!20x|j70astNgH;N~#V``hH{7mpAd#=>*G0anp}})BQnf_YwY0 zb-WV25i-|q6=K?7M^M&u$)S}k2xqfxt4^znNDbgpo?0wk1EbPzQ}Ox6{riy6Ircvx zR96Otq@*y4%x$a__wTv6oKbl55x=&WMLSQpjKq7Lg0;NhgWiPYB2P`d2z&wB7D6?P z;tdkYW*caR#7y}N#{_=uZXz+TcfIkwM$69QH^RhpN&MRW2eeO|2r&Zz1kv!^Y@@P! zT#;y^Vf3tif0cHUPMgj8OmlNGA^2!;hJETy&6$D9!3?$qVdfCC+DNNd6$`U!^H_CT zxC+y{JS$?H_xlKGS=$Di>gSuARo+}aZLJORF~s!z-`~f(3M}CujB3??4(eol)J)0Z zmNdD58hUH0dbHXH*2j+yPGzsdd&s-nO>AW@lwbRhl0YYq+^Hk#xUm{BOg@^V^+!H9 zkMA=ydgIROqZ>?3OVl#VXruRWKC!<$7nGL~0;fmc^v8OiyS>q71-fM^AX;8qg{sjA_Pbw`X z&A%Y2J$F^9Bu&p~oW55Tl1WgGu#L>Q_H&^2EtQmPm|cExb!u-vB?bDxwsb%kj~?QP zgov0A=hM(F=sSK^ud_9O;P0GYQ1`M_`k6p2l0RZ`v+0H1&2c^I$<>)bu)?sYh4UmTC7 zt&lBcyxstm$xH~Byk7nU0A;k-ZTDnq>%Z|`v%}#EIJl-4T5XHlLIwp!tM(JV8~F(j zQxxaw&n?V6?dqwOg0CHq`(@He)B5EJ=o}^P7&XX>s+!l-mv|q&TIwXFU@+VO(Aje> zmlouiZ3Cf$=UgyMAG5nh+AIj}0!8`YqY7NJgtzW&S14MC;VeKbtJb5iqYCLA{QBB$ z4)i(R1k77U{wXhiZ&D%7RebReOz**JB@`PePfc06`(yu}(<>C^Qox=XeB7`hB{`i+6qFQwzOE zGbl~s&w4Lq6#_O7>bdJGq@&lDvLo&LAlA#W((WKpLv_17) zuH6xVFY_&zFgPEyP~$G7!fICacE1pUxB~&vM}kn?LE?uPY3t6eRpf!n=l*&h>Z1JQ z!Pfm;5g!=GU6P*XrE5Ri{(19;locT>c*Iiki{dKX;>n6W!+JK{rY{?Z-$2bb)FUZT9B%&&yN>6 zsk+bet=FRp$6$5*dGzOn85)}@W;@gwb&=O&E;J15?PPsXOuS9Bf=lYzt^HNOHRqy< z$-B9MWU`7h)r5V}m8z{kQx+^j;%cUZ6{xh?6x${u@H7rF*wADYUOftfAsSQ!ST=un>x;G>)BOND5eBg+ zf22xC-8WtV>;pcD zDu2YYfBiN5+Jo-U|74>Vh)&^Bz|8kGjfpM*?kUAKvXN#E8A;_oqTOa0mDf)*IwMb< zs|ZZ6pzK>9E?ELuhssWE?2JLk46RnRH6hCu1ES`i&L{5dxoqZ-LS9H@w_>EltYtdZt)?BfR{ z(pnO8O?>m~NFXh)zxli2ao`^K73y%bu9+{bEElNz;GP{(yf2S<3!g_aD< z5Yy)OCfuit^owmWZ3wcN-;z?S+pOc`l;;=MAve07pLPB1zo;3wo|OnzG}FPdAtqEr zyE9+6ElpcH?S_Pv4!5j8B!6XE;#6-^gmh9(SQ4vnqH@r&N;ulWjdrwjD~v#eGz5Sp zTlJhqg*;Rkqh%~XM8^;_e-0BIz1Ya)XP==)Khud|SFEypw>zD$GshU-ve0)lCs^SB zuP5H}oT=rO?Qjf%_%J)oEIo^mZxX#_d*XO!5;I3)79mMFh?F%!wJ4l?fB=dgB(Dyy zrzur|Sadyeq_ZtqKi$ByxhlFFts;(I`LskKCThedcIGE3@n-TRsB&7(qi?%z){WRp z+8km!$iiwPEc>g>AgQS(TWE(!Sby)F=yDjgj+axLUp(^oX~xoQ>e1^kqRhhkI;xpb z6`CF8B{RH0YT#^6OY~YzT$)QdAOS)MHD^|flW)_{#vUp2?m4E6o<~1TVwe6ZLkE@S z1ZXDDE&H84*PPhK&1RVw&v3T!t&6nxj?K29j64<;x6*8Qlz?U+{s&;#C%eXDt#s(Z zJ440uO8E;Tl!Y$FPFP6Y?Tg9*SiDShH5vSKN);5H&#ohNM=rIbnt=38H-E1$yF8eh z^6(dMT{fN8Ryq6-F2HLfDV+9~RaeZ}tsuFL%NWAvPe-sPl6*NtPaiz!tT&KltsScp zy5ey+|Dw~iy?1yaO3!mtCY%9i3>T0469xzzcj!$WQx>xr=~I66XgD@>-yYIW8_-yj z=b#c3){O|IZQ8mWECYCTC@|a9Y77%!GOAISo$UTc{r`-08VDFpsjfkepr^m?ZYlQz z4eAt^_V5=_!>h~aTN4)qfKJISxAcyc>ySL`pIHz{mv{Smc(}xa2$^jCO9i$%#izgV8Gf_kW`o+5aA_=?jf3WZ^<> zMZj;H-vZ-As7l%5G?9Fd$(`qSgZ@cl3Co)dsH%?SQ(FJ_@q-6kp`U}wzg*bG8wJ%) z$U`4VvB+|L#P89)eO0koD3sT7*Z*ls{T8!RaiL>g&b3t4;_g+@!Zu^ zwwUxl=_+cd8U4WJAtjjxF02U;JNWa#E4Y?H&KU84z&$7_+2Ziap9Gi2%}n>uB*Nn2 z%2+UJ=o`=6bMN&B>tLf~e1}KDw?ugI0@!F^S*Nz7%VNNK;=n|rS}BKHM`QyEyu-6& zOUK!toWTsNf6rC+i2D?SJBP$|aZ9p0yv0QM0h%z`JSk<=VME|r{vU8rY_e^qjjxpO ze=e~N4sL~}=tt~;0qHr4Ux3wGeorCeK3@O7e))G!&T1}}9Jc@dxX<_RHwIij`2!(- z{y%Tv_$=cFVDY59_zSQ&CTBqEj9;ON9>afb+-51$LMg*IV&hYA@jFzY51d{YW6=|s ziLczzz#q}#tYfz@8@QV{qNjdCTdPO^+h5^;0rEmA75btMa6@|(?qh;~;${rS5#>9w z1(usO5LL!tj6d3$je;-T!^~=Djqh>(p#Aq5iHR8Jdu03!yefN~ZslTfK8MCP$`WPv zaLdbclsbBJWQ9JHpqT{U$15NPa`n)t!v{ab_)*kG{>v!RE;0ke4puDJ(V0PP>1I1Q z9sARf!87%+#_>siE)Kn4;5yTQiGaUAMdKfP_FMygY|1;murN{o+B;~5$r7F1{_|p9 zp!l@~MZD2enC=>Q&RvhM*wk?E2zEY*^1Nd*Y&vBab=B1B7e~tSEOJh%jo={|uk2D< zI%#Ef)eucS4!Po|?k%I%^pybWE!Csf=7m9J9dq_!G?4qx@4SN1Yj@7w-WGj8*C^OF zgRP$4Uw>ZTSfq^WAz^Vrvo+gJ)r%L6TsGOwfkXK4z3wI7ohrQL?9R%}E2CUlXmk-qH!@%*Ewp z7Oqp1Eg@$g?SMOC=joGJFyR|k@HyEp*~>p4mT7nDO~(GKpO?|w8(0nV(1%LAG@gBL zsl}&^_vroop9Xkv2y|Jt8~pzLyKXNtyeLfn#r;m8Xu=yTf8W2Tm`DAp-QOGS-oiT5 zEVQmXd*BUC(Co0|FlnB)vyx<1(VUT&({SNvGUMkPtKwN9??=)7wwwmSk?C?~XXC>(?#=Kg>E1^?g&-Z+a&OM!R&EGJwqd94Z|u#PN6z1j3WMkmwS@SZm~GyPn<{$zicAMkq4{=pDuPU>_Spt(z?V**I(a0%oK z`Z=#QZzU;&hYAVu@|OLk9#P+(2|O1l@q__X`>4#%&w@pc|A8GlPI9qCy+t?$_Qc=M zFW*XbOOf>9Um zO745+26hOo|5o%lL69=t3!OQ1ICZQI7Ii01ARj`h!MW{-D`d3W%fBE|K*KZ$ za{M`O*MDl^O0&SRIG)HT@n2tCPRg%Q!h|${?W}ugqBi7YxPrC@jXGK>*qJ7{(%77U z@JiSIUq6jPHz3CZY6I6D#cj;&lW^x0;N$CPFZJMq0yn~!&FAYysG{C$0Dytf0W}L? zD#MXJ#6+*~1Dx%xLRgP$S|`E-T;=$1YQ^P>2fETuV>iI>*>;*KFjkv|%)OYu|9*E} z2(4=9c2MJqS)e{}X@2f|)}KfIJPrTHIxdVkYKz$*NY#3OjnQ7JvPS_FQPqm)k>6pT zVG-r}#TpHzj3VOq`;etjUcG_VB1y0xKu!o3x9PqUku{GuZ$>SgW@3G^J_;>ins#~< zk`fU^NsLU-Tr1ow|3JUTJV&b-ZEO&(QDXe?aYQZN)fk7L4|TFzsJ02$_#4CY%HC@n zkho-0aj7Alg2Guj*OyyaMBoVruiKl9M|#%Baflhxmq_Og?>Qx5ouNr%v|H7^cnuC& zYm9b`ozC#?-m(AMVBJsqKd+Jg;|ihtpKs>}ihlkA=^2K#CT#Af9~xIz|Iu+yF9H?` z6}(ohB8arm4+YNmVkHkX;%BM0Sh#}fBR|ihqIGEK?Ru2F?8FLPvf=6xJ_|Loy2PHI zx9b93E<*=VVFZAfH4Yt?iZNV0qBeT$=5t z)}z8sB~3WCx9HYY{(fnL#$K`3kPzndq6Q1e3h{FCnrZQV-M?RJK8xo)W~0fDN`UpX17a7TX&JlG zoM*}LZp#lZ8Z}hMpJ4zw_@Un9`x zIDrkM#2ZCO30jChsbA2DBRd%%SzXe*1__x|ujHd21FHVq3F~WGA+cw?(a5F2we$f& zEA@6TTIDDOyi@1h#r3t@yR&kh1{kO@e)RKijM513TAISm$w_HuNWf0Qnrfrz5%w3< zbqHKgP(TGxq|Ezs+gWm(K*aD+H1 zJ@Tz=(n575|LaAVC0aIGiv_^+Y_tVl8?h>6wU7skF6aNgHLeHJsGAs?k}}e&6|Z)i z8bbUAPE1_uIP=uhQt^7@yQ}D_2%2dR|K;t|4#d@1BnU50K$BxX0F0*b{+;+BLConm z)y8=mV8hNlT%SPAYimnJVP=6VC>|86d%8hs30s7;Sk%i#;sjdpR$=oosX(`#d0L}j z^gCx;I6PEt7=b`c*>bEo#a3n)pR!f!NN*te?qC5YhC#_#9b!XNXe{%Ua85?Oyw9#3 zJARfv!0rCkD$EQwLwgG-+_&!3a}Ig5J>NkY6{grIb!~^eS^7q;WS+5+r8O|iBnj6b zXq!=W95Rx5wk1M7K{;r)QV?l4$~(~x6Ot1#gfhy>uCaq?-?=h_wplc)6{MLDIJDD- zQ0z}1T& z&IH$GJI0JzaCR#q5DqbM;|H5JV_a>((In}D`?wU07V;>Nmk`lo0DnIi9(G_oeRo@S z$u)m4lL_gC<%+vA3^4dr@UjI|_wj(U-p6YzgG1M#CUZv_Xe?b#FE35@VJUiX*7k=CMcv4HVgZOGOR*DmoAKh^73O1 ztHYU-Z5Ld6XYNCuBa~LqnG)LGpxk47DFFbAeH^4>IgHwVD3W*gp$o;e$9oRQ-GFUL zZ-vC2R}Am@%6DmI5{@5G;(OyIoH?N&o-}3{C`>AdTxA#0yVCptbHtBzxKX`-!dB*Y zQezSsg{vL~rDZ|MJv;Ni=WAhYoZAi)3N(W-1?H-1VaH)?jB^U&Y_xp*@B&I!_#-oJ zRqBKxyIK^us>i;Y>q_uZET}luu2Wa3+<*hi{m{k)hKjSp z=s}Xc=mK3`{gqmBJgR*>^BE>Em6>QWrp|l*!UwoK&yuQwp@ZMJ9-F8(JEmXeZS^!}Xa=;Dq|J^1KwQZ~FXAO=h^K&5qLrK%|<8Pdb?PBbXId{<{m7d&0O|MfqAQFhBCZeLoX<4&#><2>HnqZG z>do+JT#L(s&F4oMT7z{F4STClxZ6sE+NMK zHGPqeWo}(hRsfykL4#O3k6}WWaYj15SnTd*C4PSm3F;` z`7vpl%he6FKr*rxB%Jyg7GPrE7ub~ zoZTv{`{k$wMauNwr3G~TKn+jh)u-@iMF$#IVMJdmtE%x+3eTTo>4(53DaS<;HX)yX zJo+(fiIvomi>fJ<5ib36XFrUWaAx!dcID?SB&0z7j8ny*m+~z91WMO0_`|<7L|DTm zhw)-_gezEm-W`TKdC`T?{+3IcYeDXga7Zj5;ybSaC^01Qr#cZ-bHh-2V1#k`R-iZXlw*2ZKJ?`VRYI_=i|eQ0271F z87*-ePm>&0^MK=`@A46irlP`)+c~HdsVQ;r3>2s9UP(93=uTchiA37pt}`ECdpYGh zEUtEU?gL1X2U~SA{>|z+3pNEt1zj*`MjKg1|DjL2q(6wxPNW!m!9Mh;#+Ne`CM`Cu z#Qj2iXcIHr4mwUTYEAA4%^Q~8 zSu42o;tH)jT(-4nyHEbovkWMxsEaeNSd!>-}g zJ$R(ZC`G#eHcnezfTCH_X7*u1F6^4YkW~brqO|eq9m9?DfCMkCG;;|qB*|Ll*`@WV zF=!+Wt7Ubbd@dFqn1i5-9ezMj@;jKtBL_BFcPF=JBKcw!CZ(hdi{}^$s=d3k9}1f5 z3Gnvbm?%e=pKiSz@eT*B<59wr0nYLQ!!_WAII5%zSJEKE*u@zh zuzA^am>-2Yo%#5&jju;-7iLC23nvoC4>FEvT6``=D4S)CLwj(tZFm5Q6?@a!tBNMc+>wh{; z=sKpT*l1~F>$@Aj1plT3(y*Dqd#l+Z>=q@iteZ7NwjJBQ&EPGO2-_~yz(gKuUYh;Z zmL4t}n$;5q3)uK*i1URFZ5VZ8A_Hqm?8ye#(Nwfkr?-MCwWpvw!xFeS48Ca&i^M{>EF#b|9&cAWhVOdW&g~wNwUew+MQy_qf2`B`U}gnyj*>I;R%YB zNc6O)UNr5&Sgr*%rapMSUC5-u{O8|N9$W6U4Jfh)$Q)f??RK3!v+%@tZ5LEkb1HC;VeC6s@YXX(v^msAB;kP=U&;;IHUrl2LYVqme?etlK@ zZh4l{YxhVeo!!!W&K=JWo@?jX%G54NDG_zD&Z3HBcSre4j=Xt3WRG8y?GE#1sgo-D zHh-8n_*gJ)zFm50$b(*uU(dD*w@yT2Y&fR+Us^j@7+x`D`&imoR%o}A-sWouVWPI~>*MlD=+wq{elci&+ zN};NMN6*%X;XXaqm+5_iR33h1yGGTvn|=|_1_~=baQE`dv{0+$O5XLu2R02g4hXX= ztPZb?Xu0-may}eCvpQOAUiz@Z(|OrltFfW(>Id zzz=EWAi+UF#mYg&?zK^Ax7ackiu^)fT6KHWA^Nw#lXu@Jy$sr~Rl`>FtT&LdS3QGq zw-!7BqFh9ewO&Q^7SQ&X65)PaWHlHm=INH4^QM*~PMuOEG*~!MOCfd*u+RQZT-Mbz zkmjZTtCYK`-np>t5CX5kw?TG8uW#QMaajD3J6*N%g9R-tW}*j&spAbqaE(H0Wu#V zlN!Z@(=YFqIV>0*@m_<@?}=fnX)xT7N001*hUwS|W@l zJJR#Bt2h667c>27OYYE{tU;!GHVa=$?v(F$oo^c}CQ31HQBJEA&~~Ri zR2HwOED)Y=t$VIC&~9a4wu?a7#MC*u>w$55MucL~{O2DQKAIVygG=`jh=aVBOc}t`)L*7VIwxP46NcHKihh#oZjT+P=6t z7o=W|nJ(QKHM;$D_q1EdWOi}?8^5%PqE1QXn)z>m=F@#EEZ@3ohR1?`Q@!ttk~`JH zVKNejE;tP0^5=X^+swqgJn-q#onizX?0q|>&P<5i{zs%xTq7<+emj$c-${0X%U?g; z=a9*e^}R1m$R`L~9x9zv3aoRYn)Q4V>G-9PV&KkSd7dp(ZX1(B;7Rr`ft0V$wTJwp zf7zvf>+`=)IYYz5@=d3P8|~@+ZLWhN+7SlU^{=cmuY1iUY)q-E+e*Ht)+xR5JanP& z5u4p$q<-3Bt<+$DXnLcXe%h%*>euHSQpvP~Os{ln&Jx4e&7U6);ULPb+8blw+>3eElemmZk65IkPzB zy1txmYUWm$lvVYYp95__ovHpDE|0sP{Go07+=ge{z)jpeHGT)@YyEux#G=cQ&Tk!}h=V2u9az zVfl5N+mjz<>64|AM2@m%EsOH~GXc&H(`mEZR2%a0c(vhrJL7)K2lg;$cZKtD+IDot zeA3&+{_Nm6J1QZvV9a02yirxwRdwmbABWQF&(b?5HfH>Hy$3(mmc{NEg%}ns``*vh zb*FU@iGnH8cW+8k6N6gcs&j}Rp76?M-0~_0!+X2)I^Pu?i%7NIKA{FxF@femsxtPf zORxN;#!vC>WGci3jT_!37CJ4cZ;s7ut$V9%8Lp;#r^QdMI@2?s)V<L`zss=Y7xWGZAvPx1eD&%mb6 z$Yr`R@#9&cpiiS%$#Qhl!DDlay~=4-MxOY)3y>8qL;%+XT1OW5v)Y*+bz|y%hHiIdp_~esofjT{nW^) zMARxf;$-aJ5Vx@tvaT$4WxF^ej;=2CKM`u@F)MQT8TA{Je50P4^0dwqo?9(8ZAUEl zRims0C(iinsy^tOA^%yo{;|6N)9mE{(T_$$pF_9LREs#N>gO_44?E2@v@|I8I9=%~p{6t9@=BpWz$eyxl5N7} z=Z6++hxA&Bo*MB3_2EU-l8Xl z>Fm*lXy@R$>G|y9u0a0E*5atx(7K}KfhwVNstt#6&347FmhonO!qbnA^HnMfntAU^>kZ(L5qBRCs|=FOk|pRAO$Q@Fh;**2 zf358~r)Q_^)E3*DzMiv2zFT#K+kMWQY!>N=D1$>$0ULnVs?4~*dnLbX_cCr~yug0m z*`Fs8Px{l(!e7F!^tW``0^f-QHRa8(7Od8=>ZFus{0znAJ7NMQ-6uM0BHF}N?VGOF z{^ci5QS8Q_>HVq=9&glNanf3nH4Dvsxve>+b`t3K~J!-~-24(lRU97v7g zwar>m6};V3Q_l~P51tEGT+i(dl%+LU1F~9P9&1d@?i$;V*w8$;zpJOBLA*AOESZqh z5^?9O{cvpUZrPf_44T_ixqzieVaDE&cFzU#KzSMC$hyq3-L)#mB~%;roV0_)L-kw4 zRmEKGu)(EQi=-FBV>eu))jCeyHat#xqCmjXYjrfvGTbVKKpN<DpCEyP!L;|Z}f%M3`fUU)g<>te&bR`*>dkcW61AI?1?<)EDJW zG98Ycp{z%Aub9uLG*eC;+RpRx5t|xmeJO>RgB0m>i6op|lQ%jbs2En*v)mnxOr>P| zpy3+7GGf_tK75sF)#j0oGA-)U)rRIXwK$r&$(#yuTKdgX=u<{aMt(C@0lk6D8Y@@~u7unuMeU+ndh0J`iOU_7Jf9Rn-J6e@ zx!b4K5@({~)zwK}tJA~kgZH;jle_sUO#=JXiB)JTV>ga{9l-uVS4oC$o5o~V*1>A7a_Z!KA7GglCC(Nm5nk+sl{ZX_|3l& zw@~ELjm8pGg!!>+*Rs-=6>HpLPmVg?QJ-JzrMG0Cw7j-$cg7>EqG^FC*+O zIdw@Z(v=nHl>vd2B?|qZuj!|Ie6Ib$iCH?V+I;rclZwlBYdf{A^a-|^lV?Xm8`UY7 zgv|131FoxD++Hqms@0W(gFpS!TXdvnYM41&Qp?RkBSWLVjs~m-4%oK0sFGXmiVr}^ z_&i`IlUo2%`3B`M(Sv@+6$T|{Ic;7(KQt2*-G{7mmz3#b5BBU4lJBW7Krr_jgN>5BBPN_JhY^tlvSwN$cMZvRO;(=Sh5d*>+fQp@9H{AjJD za$1{ty5d~yag>>QOb5AAsf>7TZNa|hKI48gb&e>ntqi&@hU6|)8=f>e9b)KJ@?Fc& z#dy;x(Va{)G>PJm`hVIiOm*1jE{~6WJh!wEz09@}y&e=MmcG8$b~~ZXnWE=T>m#aj zg-^^`)O@{L(O;ohcU`NWq+&PnUfaIb)nI)B#!a^o3^%ILr$c8Z%Uq|*fyI#Yh^rmN zvBdRU-py`hyw%<0VoR#T=*S~AS8Yk7(}6pEtWs(P$bH-sFPAMAz8^|RBmwI{oaW1I zPV}HqlQk}$vJ;1uH7x7*K)aA zGf;!|HNz+`x2~3T_wIEs0ktug^)C1Hl5Lm~`Lqy;)|%F$5M3MP#=bC%=??q!VzW+1 z3=Pg1xv9U$b4=#VYHb%y=@93Oh#$N{I7pW z_dV;dQ>o~$PFZS-$NZ+R49d^JH8 z@3>7hxf5Cj(v;ES+f(JT0WY~-O)J>cBg1`p)kusen0Mt*r2R-3CWwY2cI}qA3thm3 zDj$4Iak^Qx`810}g?152?Wkup0i`RnROKzw548N z^4nZmmfDSx9&hq4$)SgiGT&;Yq|W(>Z`me3;iiQpUMt)64KLpQ+xhh3p5b_MY^Y6? zt9hL*ARTgvH+gN9oLyW9NDp|_0DX{5T+SA-L`o$6<6*$@0t6$Q5{tdkzu&BVb*6TJEHxO>IQ8C} z#U?~A4OrR0y)EIkUg^r=xMc1`(ZEy%S>5AJn-dj}ZcfW2*GdsC-CcD^C&S;cAIMYTk?{q{QXTQEA(K=f;CLXz% zX5D7y-XXzEz`Hn>cgrR>ws*5PpGn0)g`!tsxI8T*!bxZ0a<xC}xqj-<$*( zL?r(hBATM*PGyIy6KcxKpZ#stEp zK#jUFdSdJ?IzoYRq~S(gj%}AB?ano8v+9di9}AM57>j5g>XkWR#u()4KYaKWy1Ppc zPs#|Wj?{#j38ly_b5JKWJM06ofrjp7KE{MNt2fZO9iJRc?EQ61WKX4_nBxb=zmmn+ zSqFX(Kt~i=?Mn$N1Z|>i;J*Ur*SF5x245c8-+UV%I4qU!BN3-R$dE=m_|p z&2aWtLe~fELO1=)VuSya!2>s-WpW$8vdCRuOU%2?YLL^&>d~4{>YJx--~!7N6Ey)_ z??ow#eq@vB{hJwLC5#dWu5b%o~}j-B(} zv8{^~CWh@Yt1XTuz44J>%bs$$_B|ix-8PUcWaC&|&idefw&~B$1(RnlJiW!h(ryBJ z*(FtjxVc3)pUD2w&ms!PM}OzUc%NBzrz8%o_6#+HkRp?nMa+zCEH?lB(xxpZt#5n@ zpsg7jiF^rEN2=>Ks%GGnPdjOH!6rW`C^UA0osqRs=YMAaOnaD>vpO|J>ouNolWQfb zNh(g$X$T*ji)DMiI5Gep5e#?WVecOSvM`sCV%cJU!pT}|mB*P%z!~LSSvBj%BKJ%% z+5yI#o~0e$#$*C=K(B>-1^@-BICrikd56y*hfaEV#HrCp<-?sosP1iQGY#56Sfmz= z*u@Wn*s$K!E@h(^E+9Kr&+iw=6^1yieE$Y|wm5~L;8@y4sy)-`RxMfj6?cEs(VW6F5QfhP!-AjycMr~KVZo0j-MyD>9jbqiPivHd~n}Y()cV8*nC$a66Bcsv=WtF_2YUcV)AAU=U{sVnw>=9^;wDuVL0L6f3gn)f0j~MF5HfK42C~ zYfl=>Hms9Nw~{(zYu9`o`A^J+Oa??5+Pc%{YD`4Z8LM!C7$xV5>a;*F@j!)etK{M7 zUEgGZN#ks z6#ICwVs=-%Jdkb?>`?t<;_%vLFp&c)O- z+w!QvS~#roCl+!NlEa@LvadSQnIX-31(^R#UG>^>Hsfhiq<;0`128<6o^QP8C86de zLLqJQuyN{7W8eM;75j<8&g}E7rxaI=m*%P~6uTJrRET@REnrMWQzm?j|5QfQFFok6 zKoD~7mQP5DvL)v0N>yz}yzQ}&57KLcGW*ZhH1#EPZNg5NYKUiqy-UR@rRC$)4aw^ zuivIFvfaRkxO{Ef1y;Yh4E&=R}_&Msjo5E%WY?>cT`c z#Y+-V+uD3KYnGZil16$juYKQh#!Dp(wTjo0>3R4BfaEQv&V*4iK03N3(y{>%+7#?O%{yoA{Ufy^ZR7)olnB+jvH8$9=T(}mwq-1c zKl*1u9rBO*a{ycVo-3u*lVw&eOgIll0_rJVB_O0?&@Q`dCD|F=NvL7nc{a!XQ?mrc z#WO|onR4?tmKJOYKHGU~bgb~U(kTi_wWkBqC<=(%0U|Ko3(56YAKCG*`Q4GqL1|wO z=*#T!L=hlDbS{rOMU#(01nKp@C-2?_X-$6kZ9w+kLgyW+%M-y zw5)2k(f%e4Y#2+)m?lb!2o-GB6&8?P0YU=ZL-9-o1=q={Yg+Z9!!_RRmsb_iVBC62GofoM;3CUPx2!(E1a(i^aAOcVLH;3n0n#D)5-6bn3nQf z^Z&S9>?J+@b)|cqj70j4>rf2d3Frp(_mB7xEa-MuS;qHgQQLVQg@EXba!-!un%;bm zwU&w#ETw>SU9%D)`Z*RFgCoVM#mp0d(evTVlCSEvWN@Oay2p@s&L4Kj2mg7%s!zqR zDzpeU!c?E-y4!~joHTtvO3dWUq0d09lbBfm~TurW(m;C3ceB^u-o^sZhfxLN=}A`H)bQrzBQ;ka`_s zFR2<0SCH4WjYMFn@0@EBADTZ94)z3zl3mkP#(I?hvaA2?+#Fvlv%5Mp#PrX;~ zA0XQ|s)COfSQ%EwLIC^(pbR4h{@Hx82k?+k3f?k7eG9(t5k(TOK2c#h-3LNBQi9p@ zD@fiZjTt0jRQm#HN_D()k@SfoWxMg*+nbmSdWE@-FHVD))rhXz!eUDA zm$l+sdYF`52heG@ks4)U;4y9w;HBfP5`s|L)gfyziH8Ed=G?_;y)obeNjOD}%hJAG z0Awa>#zZ;&QqXAeTnFI?#!X6inwt5$*--co0B;thp*7 zi;kRQvmt5?zufD3zZY-*52e1=;yU5%x5V}ABKE?L*S-vjaW;dOES`rk z%<>D??)zlQc-)aVcQ|UT4M#I10&Z{X=$`G^D!-k#|85+a#$MP4@ZqVL-M8~QZwvDw z%ff!4u=K6N&59Z`8N%&u?KR~W;hwC+bnn_A7Y4G&8*d5=IlEKFOy2Fa;_8ueK7C4x zRqybgr&|i9FFCdG-9|Izcg>6v*93W&Z>UN0`*JlfmXEiijwl~}SRFC^K!7ZE z%g9W=!iw2pL>9=5>Q2SbuF9189XJ2{uiV@}HhD*o&drP0=N^m9aJMWrQ$6=2^1IESN@T?%?VII0Gh59UI%jy-pDz?(QRS5*9qnEfL9Cchn;uIi zsF!u;(g(l@b*;^Jc-U(i7Sb%4jLtOj$MY4Afas2LAHbv%RQTq|`C2eW_b~tCGX;C&{!#6&F(hCJDED)Tk(c|1jhX>aJo`` zfUS6_fmWX`H+ed%TLA=Akh+cg3mYkiTQ)fd5rr6+fzl$VHx5R+ngausrM>80(-da< zZK`o01$sFO_z>^gaCt{E}|*JUa|Sa4BU@xx_+ zBbO(l)xAz)EuLp1xF2A;VdO|dM%YeV*K;uj2mXcAxvb?@Ew;JQq{ftJs^&mB+su8I zJ{%L@&SPV5kBSSl4EI=SMADAaD?lNSF*=)CRZB!oE;vzzr8-{#9CI6WEZb&TQhnS4 zYi0h1O<}YDEs|+_2F_Ez<8@j^jQw_}BYjGa)Ca(;%N;vO#N7*P^k^ysQmh`G`)qK6yPN0ad+9e2D1B9bhpBPaS^nw<1-sgtfStJfd@kbo^p z`_^l{J7;`YZ1HSE`|~E8qqc18dm@z9M?yj?w{$C;5Mq99e0^;q zcWnv67s>B`h8lLwpDt|uf7-k9ucq!i`Yb~^)(&y>96SyP)}`tc0g1>W2v%)HRJ2$n z>?(@^0|-dKkVMNAjzdL-8rBfT4Le#;C_#2pI#mQjsca@dM<7uo0TB{nSmwU@D|*g+ z`GIptczNIV-uL^t_kQkuFX*uSylkRK7h2Kg65U};CHy1W>6GuuIXb>q%ctUB%X_ zy~_}f!UbDo+gc`6fBizisbRy%cyt-=f6Cmuye&Vf&z%t?YqCW?^vlxtm*>$pqoY_x z3p{K)`1D)?1I*m2l$pRQLOqY}cM^}b3lTUNKEEywA%c??xMFVw(@m(uUH-Pq0*4kJ z6DzM~#QlB_Oc6HlB=)NHd9SdhS-B3^ij7)c~lQJZH^`2Og;BV6L9Yt_`MzGM0= zKUXy(m6z+4%k5S=4ir{)Me_2EAAV>)g!mkz z32J0*&Pm=%>eUUu+Gm5SGKNMg;$Qs}y=ip+{C(^sP$Y-_!|JEv1}~shE-hg_MWb$6&LbtYn3{~_^e=A+ZY)fRL!%>Q z_%nUCa9>sMURjEZPJtB&XQ=atiW3us(RP8p@HEYijd`F1KvufSp4e^|wkOkRmHA2Z zl$!CcKQF3y5y@%)6ICB*xR~WKGY*l2;lN?5eP~SqB6wqB=&hWf{C}I#AF8;Tp&XcZ z(+x;gBtDumL`;t>1Uxx&>d?My$qf1s*;eyT>k2Q`jv;kJe=L880s89sEu_fwZ84a$ zmS=9M+3RtPFk90t^UDMd?)vj8Gj2Y&ZngU?yR_B@tAF2?W;jl^LN@QFrBc_}t}IX$ zrZD8s1*-1cMq_h((Us+1=2k-CY0d z?oo*B!ZVSG1x1zSfkk9{OWk}JxP=RK6flsbNbtcC#|Uv43p}iT^8uiPasycg1oU;D z8MHA{Z=ZY9h{k~^$c5XV=cYZu%kRGp2QT4(Wr|&}ehJ0V$_o zXs`Z~&g&!`jBao)h<>5r*n|DdH=dMBo!1NHFxmWVZ3k9a4s5C zeBbtu(pcJ_)fgwN^1DaTET#(}D%_<{P~HQQ!}dGLSc83$uxmh=7{W(-6`erTD|@4B zVawv~nyp6vO|F;jYkzibwLJ zaL}x--JJ z;^PgE4lyRA0v=k8S#AhCfm~X`SK!)S8?R*8#9d%B95{Y9jjqUUz03f_F5k53+81GE z^=7&AY(;Ve%&*){bXyFOy8f+rXYhjDauK!Onvz|*&Yq}{48OXDt!O6(Ez)PZ0B6V9FGlWdn>l@46T(H=F@TqwFD=r8a1QcNLb`C@5$Jmw& z*QMuP_4^SztmwDFO21p%yUstlR~t$_FlX6)b5sQSggol?Y;ZW|x4(cNa2U`174hD5 zj#I>+XUtQrnKdB_i&jIl08|ql^MFbD_E#?&k3q|=Q#UlHB>Wt`}!s;mwse3 z*RUeQAR_Y@!|7qT7h;r-4VU3P@II~S|B0PEF2j|x6gV;5HX9^p zw&yWb5wZnFex60vC4vNlMW@h0(l z+}l*p;Z^D}I^d3!0DLJdDodVqK}-=0;@iQMZigAMWA*x1Ujj{>=V#E!PaeHz!7cQ| z&7*B2Yh`v`jRJH#P~0$kfp{QHJJOXA?k$tAIo@4qnAgZQj{go{O)xX)Ivrx_UN>le z7a_}0e(O9EJe8=^|nm0xMUWQ}-kJ+dCGxjH>Umb%GX5XBfIj%3 zRADJ}T1N|R*ndOu!@8^vO>LXwP-e&>y~flCQst7m%uc>mbpU^g#YF?^jY zukfeE)|H1uzQ?pyIRW;w2356+UVHXy5wBbpJes8H6mFZ?ZJ83QRf7Gcn!sSm4&<`8rcBxD4 z;;!Jv&PF^BdB#YF=Sh68D7B)f^>Bu$;x%~nY*q+f-SZ7@FC{OM6GA2fSqP*ukg7-! zfuJElLxP3`4G9_&G$d$9(2$@ZK|_Lu1Puuq5;P=eNYId=Awffeh6D`>8veo1F!a!y zTNGvS%5&%TL&Vl3J}>dOiNF7Um2H@wS9hPI@N;&h@Xf+4>oa_R#vj&4@k>O1owFAm z`&f%7sXthP>JYLJUx9J=?;vyJ-^d9eGlZ-KQXELNB)C9;l28J|kqBud^qA-cqKt?# wBFcy;BchCmG9t=|C?lebh%)-4D5H;zdQ3N6m6;l_EGf@j`*)Up9`en90pFO8e*gdg diff --git a/sdks/python/setup.py b/sdks/python/setup.py index cadc4f34c86d..ca585ccf7167 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -349,9 +349,11 @@ def get_portability_package_data(): 'interactive_test': [ # headless chrome based integration tests 'needle>=0.5.0,<1', - 'chromedriver-binary>=100,<114', + 'chromedriver-binary>=117,<118', # use a fixed major version of PIL for different python versions 'pillow>=7.1.1,<10', + # urllib 2.x is a breaking change for the headless chrome tests + 'urllib3<2,>=1.21.1' ], 'aws': ['boto3>=1.9,<2'], 'azure': [ From b117ac8b08374c2c4ed2c0e57c691e00e6dcb92f Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Sat, 7 Oct 2023 10:02:30 -0400 Subject: [PATCH 30/85] Update executionResult (#28884) --- sdks/python/apache_beam/testing/load_tests/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/testing/load_tests/build.gradle b/sdks/python/apache_beam/testing/load_tests/build.gradle index 144f7d12ba6c..538d4a01bfee 100644 --- a/sdks/python/apache_beam/testing/load_tests/build.gradle +++ b/sdks/python/apache_beam/testing/load_tests/build.gradle @@ -59,7 +59,7 @@ task run(type: Exec, dependsOn: installGcpTest) { ignoreExitValue true doLast { - if (execResult.exitValue != 0) { + if (executionResult.get().exitValue != 0) { throw new GradleException('error occurred') } } From d5b8fb8295827fed6cd13c2767e73cdfe4af1ef4 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Sat, 7 Oct 2023 10:29:26 -0400 Subject: [PATCH 31/85] Add edge segment size to filter out change points that are observed on the data edge (#28780) * Add edge_segment_length * Add issue to remove workaround * Move is_edge_change_point into find_latest_change_point_index * Fix python formatting --- .../testing/analyzers/constants.py | 1 + .../testing/analyzers/perf_analysis_test.py | 10 ++++++ .../testing/analyzers/perf_analysis_utils.py | 31 ++++++++++++++++++- 3 files changed, 41 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/testing/analyzers/constants.py b/sdks/python/apache_beam/testing/analyzers/constants.py index 8f8bdf13300c..09ab5c595908 100644 --- a/sdks/python/apache_beam/testing/analyzers/constants.py +++ b/sdks/python/apache_beam/testing/analyzers/constants.py @@ -72,3 +72,4 @@ }] _ANOMALY_MARKER = ' <---- Anomaly' +_EDGE_SEGMENT_SIZE = 3 diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py index 9c7921300d9d..15344ab13b3a 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py @@ -34,6 +34,7 @@ from apache_beam.testing.analyzers import github_issues_utils from apache_beam.testing.analyzers.perf_analysis_utils import BigQueryMetricsFetcher from apache_beam.testing.analyzers.perf_analysis_utils import is_change_point_in_valid_window + from apache_beam.testing.analyzers.perf_analysis_utils import is_edge_change_point from apache_beam.testing.analyzers.perf_analysis_utils import is_perf_alert from apache_beam.testing.analyzers.perf_analysis_utils import e_divisive from apache_beam.testing.analyzers.perf_analysis_utils import filter_change_points_by_median_threshold @@ -242,6 +243,15 @@ def read_csv(path): metric_values, change_points) self.assertEqual(len(valid_points), 0) + def test_change_point_on_edge_segment(self): + data = [1] * 50 + [100] + change_points = find_change_points(data) + self.assertEqual(change_points, [50]) + + self.assertEqual(is_edge_change_point(change_points[0], len(data)), True) + + self.assertEqual(find_latest_change_point_index(data), None) + if __name__ == '__main__': logging.getLogger().setLevel(logging.DEBUG) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py index f9604c490fc0..91c339a766d2 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py @@ -143,7 +143,22 @@ def find_latest_change_point_index(metric_values: List[Union[float, int]]): if not change_points_indices: return None change_points_indices.sort() - return change_points_indices[-1] + # Remove the change points that are at the edges of the data. + # https://github.com/apache/beam/issues/28757 + # Remove this workaround once we have a good solution to deal + # with the edge change points. + change_point_index = change_points_indices[-1] + if is_edge_change_point(change_point_index, + len(metric_values), + constants._EDGE_SEGMENT_SIZE): + logging.info( + 'The change point %s is located at the edge of the data with an edge ' + 'segment size of %s. This change point will be ignored for now, ' + 'awaiting additional data. Should the change point persist after ' + 'gathering more data, an alert will be raised.' % + (change_point_index, constants._EDGE_SEGMENT_SIZE)) + return None + return change_point_index def publish_issue_metadata_to_big_query(issue_metadata, table_name): @@ -231,6 +246,20 @@ def filter_change_points_by_median_threshold( return valid_change_points +def is_edge_change_point( + change_point_index, + data_size, + edge_segment_size=constants._EDGE_SEGMENT_SIZE): + """ + Removes the change points that are at the edges of the data. + Args: + change_point_index: Index of the change point. + data_size: Size of the data. + edge_segment_size: Size of the edge segment. + """ + return change_point_index > data_size - edge_segment_size + + class MetricsFetcher(metaclass=abc.ABCMeta): @abc.abstractmethod def fetch_metric_data( From a3bf007cef1f749b2759309c14ced1c5d9e38d76 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Sat, 7 Oct 2023 11:35:14 -0400 Subject: [PATCH 32/85] Append UUID when test_name is None for perf tool (#28882) * Append UUID when test_name is null * Update sdks/python/apache_beam/testing/analyzers/perf_analysis.py --- sdks/python/apache_beam/testing/analyzers/perf_analysis.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py index 0074625d3b47..109e5bfcc286 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py @@ -23,6 +23,7 @@ import argparse import logging import os +import uuid from datetime import datetime from datetime import timezone from typing import Any @@ -146,7 +147,7 @@ def run_change_point_analysis( datetime.now().replace(tzinfo=timezone.utc)), # BQ doesn't allow '.' in table name test_id=test_id.replace('.', '_'), - test_name=test_name, + test_name=test_name or uuid.uuid4().hex, metric_name=metric_name, change_point=metric_values[change_point_index], issue_number=issue_number, From 92d5bf877969cd00a229a97d35575f7f5d1fc09b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Oct 2023 09:10:00 -0400 Subject: [PATCH 33/85] Bump github.com/aws/aws-sdk-go-v2 from 1.21.0 to 1.21.1 in /sdks (#28891) Bumps [github.com/aws/aws-sdk-go-v2](https://github.com/aws/aws-sdk-go-v2) from 1.21.0 to 1.21.1. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/v1.21.0...v1.21.1) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-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 | 4 ++-- sdks/go.sum | 6 ++++-- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index d1c98259f1e9..b8dd03de6b07 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -30,12 +30,12 @@ require ( cloud.google.com/go/pubsub v1.33.0 cloud.google.com/go/spanner v1.50.0 cloud.google.com/go/storage v1.33.0 - github.com/aws/aws-sdk-go-v2 v1.21.0 + github.com/aws/aws-sdk-go-v2 v1.21.1 github.com/aws/aws-sdk-go-v2/config v1.18.43 github.com/aws/aws-sdk-go-v2/credentials v1.13.41 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.87 github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 - github.com/aws/smithy-go v1.14.2 + github.com/aws/smithy-go v1.15.0 github.com/docker/go-connections v0.4.0 github.com/dustin/go-humanize v1.0.1 github.com/go-sql-driver/mysql v1.7.1 diff --git a/sdks/go.sum b/sdks/go.sum index 1c39718964d0..f177dd8debb8 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,8 +81,9 @@ 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.21.0 h1:gMT0IW+03wtYJhRqTVYn0wLzwdnK9sRMcxmtfGzRdJc= github.com/aws/aws-sdk-go-v2 v1.21.0/go.mod h1:/RfNgGmRxI+iFOB1OeJUyxiU+9s88k3pfHvDagGEp0M= +github.com/aws/aws-sdk-go-v2 v1.21.1 h1:wjHYshtPpYOZm+/mu3NhVgRRc0baM6LJZOmxPZ5Cwzs= +github.com/aws/aws-sdk-go-v2 v1.21.1/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13 h1:OPLEkmhXf6xFPiz0bLeDArZIDx1NNS4oJyG4nv3Gct0= github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13/go.mod h1:gpAbvyDGQFozTEmlTFO8XcQKHzubdq0LzRyJpG6MiXM= github.com/aws/aws-sdk-go-v2/config v1.5.0/go.mod h1:RWlPOAW3E3tbtNAqTwvSW54Of/yP3oiZXMI0xfUdjyA= @@ -133,8 +134,9 @@ github.com/aws/aws-sdk-go-v2/service/sts v1.22.0/go.mod h1:VC7JDqsqiwXukYEDjoHh9 github.com/aws/aws-sdk-go-v2/service/sts v1.23.0 h1:pyvfUqkNLMipdKNAtu7OVbRxUrR2BMaKccIPpk/Hkak= github.com/aws/aws-sdk-go-v2/service/sts v1.23.0/go.mod h1:VC7JDqsqiwXukYEDjoHh9U0fOJtNWh04FPQz4ct4GGU= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.14.2 h1:MJU9hqBGbvWZdApzpvoF2WAIJDbtjK2NDJSiJP7HblQ= github.com/aws/smithy-go v1.14.2/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= +github.com/aws/smithy-go v1.15.0 h1:PS/durmlzvAFpQHDs4wi4sNNP9ExsqZh6IlfdHXgKK8= +github.com/aws/smithy-go v1.15.0/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= From 76fbb8e9686b97fd51af5b8207c167f901671012 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Oct 2023 09:25:22 -0400 Subject: [PATCH 34/85] Bump org.nosphere.apache.rat from 0.8.0 to 0.8.1 (#28675) Bumps org.nosphere.apache.rat from 0.8.0 to 0.8.1. --- updated-dependencies: - dependency-name: org.nosphere.apache.rat 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> --- build.gradle.kts | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/build.gradle.kts b/build.gradle.kts index a4a9a09e504f..fbea1a59b282 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -19,7 +19,7 @@ plugins { base // Apply one top level rat plugin to perform any required license enforcement analysis - id("org.nosphere.apache.rat") version "0.8.0" + id("org.nosphere.apache.rat") version "0.8.1" // Enable gradle-based release management id("net.researchgate.release") version "2.8.1" id("org.apache.beam.module") From 604629798a363c2a62a2aa46a1be89fca4a6b784 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Mon, 9 Oct 2023 12:21:55 -0400 Subject: [PATCH 35/85] Add dataclasses to perf alert tool and refactor code. (#28889) * Refactor code with Dataclasses Refactor * Add pipe to add extra line for test_description * Fix lint --- .../apache_beam/testing/analyzers/__init__.py | 5 + .../testing/analyzers/github_issues_utils.py | 43 +++--- .../testing/analyzers/perf_analysis.py | 144 ++++++++++++------ .../testing/analyzers/perf_analysis_test.py | 69 +++++---- .../testing/analyzers/perf_analysis_utils.py | 141 +++++++++++------ .../testing/analyzers/tests_config.yaml | 49 +++--- 6 files changed, 284 insertions(+), 167 deletions(-) diff --git a/sdks/python/apache_beam/testing/analyzers/__init__.py b/sdks/python/apache_beam/testing/analyzers/__init__.py index cce3acad34a4..136d9f5f5d8a 100644 --- a/sdks/python/apache_beam/testing/analyzers/__init__.py +++ b/sdks/python/apache_beam/testing/analyzers/__init__.py @@ -14,3 +14,8 @@ # See the License for the specific language governing permissions and # limitations under the License. # + +""" +Peformance alert tooling for Apache Beam. No backwards compatibility +guarantees. +""" diff --git a/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py b/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py index 82758be8f180..bbcd2a8b11b5 100644 --- a/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py @@ -21,10 +21,11 @@ from typing import Optional from typing import Tuple -import pandas as pd import requests from apache_beam.testing.analyzers import constants +from apache_beam.testing.analyzers.perf_analysis_utils import MetricContainer +from apache_beam.testing.analyzers.perf_analysis_utils import TestConfigContainer try: _GITHUB_TOKEN: Optional[str] = os.environ['GITHUB_TOKEN'] @@ -140,25 +141,18 @@ def add_awaiting_triage_label(issue_number: int): def get_issue_description( - test_id: str, - test_name: Optional[str], - metric_name: str, - timestamps: List[pd.Timestamp], - metric_values: List, + test_config_container: TestConfigContainer, + metric_container: MetricContainer, change_point_index: int, max_results_to_display: int = 5, - test_description: Optional[str] = None, ) -> str: """ Args: - metric_name: Metric name used for the Change Point Analysis. - timestamps: Timestamps of the metrics when they were published to the - Database. Timestamps are expected in ascending order. - metric_values: metric values for the previous runs. - change_point_index: Index for the change point. The element in the - index of the metric_values would be the change point. - max_results_to_display: Max number of results to display from the change - point index, in both directions of the change point index. + test_config_container: TestConfigContainer containing test metadata. + metric_container: MetricContainer containing metric data. + change_point_index: Index of the change point in the metric data. + max_results_to_display: Max number of results to display from the change + point index, in both directions of the change point index. Returns: str: Description used to fill the GitHub issues description. @@ -168,25 +162,30 @@ def get_issue_description( description = [] - description.append(_ISSUE_DESCRIPTION_TEMPLATE.format(test_id, metric_name)) + description.append( + _ISSUE_DESCRIPTION_TEMPLATE.format( + test_config_container.test_id, test_config_container.metric_name)) - if test_name: - description.append(("`test_name:` " + f'{test_name}')) + if test_config_container.test_name: + description.append(("`test_name:` " + f'{test_config_container.test_name}')) - if test_description: - description.append(("`Test description:` " + f'{test_description}')) + if test_config_container.test_description: + description.append( + ("`Test description:` " + f'{test_config_container.test_description}')) description.append('```') runs_to_display = [] max_timestamp_index = min( - change_point_index + max_results_to_display, len(metric_values) - 1) + change_point_index + max_results_to_display, + len(metric_container.values) - 1) min_timestamp_index = max(0, change_point_index - max_results_to_display) # run in reverse to display the most recent runs first. for i in reversed(range(min_timestamp_index, max_timestamp_index + 1)): row_template = _METRIC_INFO_TEMPLATE.format( - timestamps[i].ctime(), format(metric_values[i], '.2f')) + metric_container.timestamps[i].ctime(), + format(metric_container.values[i], '.2f')) if i == change_point_index: row_template += constants._ANOMALY_MARKER runs_to_display.append(row_template) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py index 109e5bfcc286..27f8398a0fb3 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis.py @@ -34,66 +34,103 @@ from apache_beam.testing.analyzers import constants from apache_beam.testing.analyzers.perf_analysis_utils import BigQueryMetricsFetcher +from apache_beam.testing.analyzers.perf_analysis_utils import ChangePointConfig from apache_beam.testing.analyzers.perf_analysis_utils import GitHubIssueMetaData from apache_beam.testing.analyzers.perf_analysis_utils import MetricsFetcher +from apache_beam.testing.analyzers.perf_analysis_utils import TestConfigContainer from apache_beam.testing.analyzers.perf_analysis_utils import create_performance_alert from apache_beam.testing.analyzers.perf_analysis_utils import find_latest_change_point_index from apache_beam.testing.analyzers.perf_analysis_utils import get_existing_issues_data from apache_beam.testing.analyzers.perf_analysis_utils import is_change_point_in_valid_window -from apache_beam.testing.analyzers.perf_analysis_utils import is_perf_alert +from apache_beam.testing.analyzers.perf_analysis_utils import is_sibling_change_point from apache_beam.testing.analyzers.perf_analysis_utils import publish_issue_metadata_to_big_query from apache_beam.testing.analyzers.perf_analysis_utils import read_test_config -from apache_beam.testing.analyzers.perf_analysis_utils import validate_config + + +def get_test_config_container( + params: Dict[str, Any], + test_id: str, +) -> TestConfigContainer: + """ + Args: + params: Dict containing parameters to run change point analysis. + Returns: + TestConfigContainer object containing test config parameters. + """ + return TestConfigContainer( + project=params['project'], + metrics_dataset=params['metrics_dataset'], + metrics_table=params['metrics_table'], + metric_name=params['metric_name'], + test_id=test_id, + test_description=params['test_description'], + test_name=params.get('test_name', None), + labels=params.get('labels', None), + ) + + +def get_change_point_config(params: Dict[str, Any], ) -> ChangePointConfig: + """ + Args: + params: Dict containing parameters to run change point analysis. + Returns: + ChangePointConfig object containing change point analysis parameters. + """ + return ChangePointConfig( + min_runs_between_change_points=params.get( + 'min_runs_between_change_points', + constants._DEFAULT_MIN_RUNS_BETWEEN_CHANGE_POINTS), + num_runs_in_change_point_window=params.get( + 'num_runs_in_change_point_window', + constants._DEFAULT_NUM_RUMS_IN_CHANGE_POINT_WINDOW)) def run_change_point_analysis( - params, test_id, big_query_metrics_fetcher: MetricsFetcher): + test_config_container: TestConfigContainer, + big_query_metrics_fetcher: MetricsFetcher, + change_point_config: ChangePointConfig = ChangePointConfig(), +): """ Args: - params: Dict containing parameters to run change point analysis. - test_id: Test id for the current test. + test_config_container: TestConfigContainer containing test metadata for + fetching data and running change point analysis. big_query_metrics_fetcher: BigQuery metrics fetcher used to fetch data for change point analysis. + change_point_config: ChangePointConfig containing parameters to run + change point analysis. Returns: bool indicating if a change point is observed and alerted on GitHub. """ - logging.info("Running change point analysis for test ID %s" % test_id) - if not validate_config(params.keys()): - raise ValueError( - f"Please make sure all these keys {constants._PERF_TEST_KEYS} " - f"are specified for the {test_id}") - - metric_name = params['metric_name'] + logging.info( + "Running change point analysis for test ID %s" % + test_config_container.test_id) # test_name will be used to query a single test from # multiple tests in a single BQ table. Right now, the default # assumption is that all the test have an individual BQ table # but this might not be case for other tests(such as IO tests where # a single BQ tables stores all the data) - test_name = params.get('test_name', None) + test_name = test_config_container.test_name min_runs_between_change_points = ( - constants._DEFAULT_MIN_RUNS_BETWEEN_CHANGE_POINTS) - if 'min_runs_between_change_points' in params: - min_runs_between_change_points = params['min_runs_between_change_points'] + change_point_config.min_runs_between_change_points) num_runs_in_change_point_window = ( - constants._DEFAULT_NUM_RUMS_IN_CHANGE_POINT_WINDOW) - if 'num_runs_in_change_point_window' in params: - num_runs_in_change_point_window = params['num_runs_in_change_point_window'] - - metric_values, timestamps = big_query_metrics_fetcher.fetch_metric_data( - project=params['project'], - metrics_dataset=params['metrics_dataset'], - metrics_table=params['metrics_table'], - metric_name=params['metric_name'], - test_name=test_name - ) + change_point_config.num_runs_in_change_point_window) + + metric_container = big_query_metrics_fetcher.fetch_metric_data( + test_config=test_config_container) + metric_container.sort_by_timestamp() + + metric_values = metric_container.values + timestamps = metric_container.timestamps change_point_index = find_latest_change_point_index( metric_values=metric_values) if not change_point_index: - logging.info("Change point is not detected for the test ID %s" % test_id) + logging.info( + "Change point is not detected for the test ID %s" % + test_config_container.test_id) return False # since timestamps are ordered in ascending order and # num_runs_in_change_point_window refers to the latest runs, @@ -107,15 +144,17 @@ def run_change_point_analysis( 'on metric %s. Since the change point run %s ' 'lies outside the num_runs_in_change_point_window distance: %s, ' 'alert is not raised.' % ( - test_id, - metric_name, + test_config_container.test_id, + test_config_container.metric_name, latest_change_point_run + 1, num_runs_in_change_point_window)) return False - is_alert = True + is_valid_change_point = True last_reported_issue_number = None - issue_metadata_table_name = f'{params.get("metrics_table")}_{metric_name}' + issue_metadata_table_name = ( + f'{test_config_container.metrics_table}_{test_config_container.metric_name}' # pylint: disable=line-too-long + ) existing_issue_data = get_existing_issues_data( table_name=issue_metadata_table_name) @@ -127,37 +166,39 @@ def run_change_point_analysis( # convert numpy.int64 to int last_reported_issue_number = last_reported_issue_number.item() - is_alert = is_perf_alert( + is_valid_change_point = is_sibling_change_point( previous_change_point_timestamps=existing_issue_timestamps, change_point_index=change_point_index, timestamps=timestamps, - min_runs_between_change_points=min_runs_between_change_points) - if is_alert: + min_runs_between_change_points=min_runs_between_change_points, + test_id=test_config_container.test_id) + if is_valid_change_point: issue_number, issue_url = create_performance_alert( - metric_name, test_id, timestamps, - metric_values, change_point_index, - params.get('labels', None), - last_reported_issue_number, - test_description = params.get('test_description', None), - test_name = test_name + test_config_container=test_config_container, + metric_container=metric_container, + change_point_index=change_point_index, + existing_issue_number=last_reported_issue_number, ) issue_metadata = GitHubIssueMetaData( issue_timestamp=pd.Timestamp( datetime.now().replace(tzinfo=timezone.utc)), # BQ doesn't allow '.' in table name - test_id=test_id.replace('.', '_'), + test_id=test_config_container.test_id.replace('.', '_'), test_name=test_name or uuid.uuid4().hex, - metric_name=metric_name, + metric_name=test_config_container.metric_name, change_point=metric_values[change_point_index], issue_number=issue_number, issue_url=issue_url, - change_point_timestamp=timestamps[change_point_index]) + change_point_timestamp=timestamps[change_point_index], + ) publish_issue_metadata_to_big_query( - issue_metadata=issue_metadata, table_name=issue_metadata_table_name) - - return is_alert + issue_metadata=issue_metadata, + table_name=issue_metadata_table_name, + project=test_config_container.project, + ) + return is_valid_change_point def run( @@ -185,10 +226,13 @@ def run( tests_config: Dict[str, Dict[str, Any]] = read_test_config(config_file_path) for test_id, params in tests_config.items(): + test_config_container = get_test_config_container(params, test_id=test_id) + change_point_config = get_change_point_config(params) run_change_point_analysis( - params=params, - test_id=test_id, - big_query_metrics_fetcher=big_query_metrics_fetcher) + test_config_container=test_config_container, + big_query_metrics_fetcher=big_query_metrics_fetcher, + change_point_config=change_point_config, + ) if __name__ == '__main__': diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py index 15344ab13b3a..5164c8d8fd36 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_test.py @@ -16,7 +16,6 @@ # # pytype: skip-file -import datetime import logging import os import re @@ -33,15 +32,18 @@ from apache_beam.testing.analyzers import constants from apache_beam.testing.analyzers import github_issues_utils from apache_beam.testing.analyzers.perf_analysis_utils import BigQueryMetricsFetcher + from apache_beam.testing.analyzers.perf_analysis_utils import MetricContainer + from apache_beam.testing.analyzers.perf_analysis_utils import TestConfigContainer from apache_beam.testing.analyzers.perf_analysis_utils import is_change_point_in_valid_window from apache_beam.testing.analyzers.perf_analysis_utils import is_edge_change_point - from apache_beam.testing.analyzers.perf_analysis_utils import is_perf_alert + from apache_beam.testing.analyzers.perf_analysis_utils import is_sibling_change_point from apache_beam.testing.analyzers.perf_analysis_utils import e_divisive from apache_beam.testing.analyzers.perf_analysis_utils import filter_change_points_by_median_threshold from apache_beam.testing.analyzers.perf_analysis_utils import find_change_points from apache_beam.testing.analyzers.perf_analysis_utils import find_latest_change_point_index from apache_beam.testing.analyzers.perf_analysis_utils import validate_config from apache_beam.testing.load_tests import load_test_metrics_utils + except ImportError as e: raise unittest.SkipTest('Missing dependencies to run perf analysis tests.') @@ -50,23 +52,23 @@ def get_fake_data_with_no_change_point(*args, **kwargs): num_samples = 20 metric_values = [1] * num_samples - timestamps = list(range(num_samples)) - return metric_values, timestamps + timestamps = [pd.Timestamp(i) for i in range(num_samples)] + return MetricContainer(metric_values, timestamps) def get_fake_data_with_change_point(*args, **kwargs): # change point will be at index 13. num_samples = 20 metric_values = [0] * 12 + [3] + [4] * 7 - timestamps = [i for i in range(num_samples)] - return metric_values, timestamps + timestamps = [pd.Timestamp(i) for i in range(num_samples)] + return MetricContainer(metric_values, timestamps) def get_existing_issue_data(**kwargs): # change point found at index 13. So passing 13 in the # existing issue data in mock method. return pd.DataFrame([{ - constants._CHANGE_POINT_TIMESTAMP_LABEL: 13, + constants._CHANGE_POINT_TIMESTAMP_LABEL: pd.Timestamp(13), constants._ISSUE_NUMBER: np.array([0]) }]) @@ -77,7 +79,7 @@ def setUp(self) -> None: self.multiple_change_point_series = self.single_change_point_series + [ 2 ] * 20 - self.timestamps = list(range(5)) + self.timestamps = [pd.Timestamp(i) for i in range(5)] self.params = { 'test_description': 'fake_description', 'metrics_dataset': 'fake_dataset', @@ -123,30 +125,33 @@ def test_validate_config(self): def test_duplicate_change_point(self): change_point_index = 2 min_runs_between_change_points = 1 - is_alert = is_perf_alert( + is_alert = is_sibling_change_point( previous_change_point_timestamps=[self.timestamps[0]], timestamps=self.timestamps, change_point_index=change_point_index, - min_runs_between_change_points=min_runs_between_change_points) + min_runs_between_change_points=min_runs_between_change_points, + test_id=self.test_id) self.assertTrue(is_alert) def test_duplicate_change_points_are_not_valid_alerts(self): change_point_index = 2 min_runs_between_change_points = 1 - is_alert = is_perf_alert( + is_alert = is_sibling_change_point( previous_change_point_timestamps=[self.timestamps[3]], timestamps=self.timestamps, change_point_index=change_point_index, - min_runs_between_change_points=min_runs_between_change_points) + min_runs_between_change_points=min_runs_between_change_points, + test_id=self.test_id) self.assertFalse(is_alert) - is_alert = is_perf_alert( + is_alert = is_sibling_change_point( previous_change_point_timestamps=[ self.timestamps[0], self.timestamps[3] ], timestamps=self.timestamps, change_point_index=change_point_index, - min_runs_between_change_points=min_runs_between_change_points) + min_runs_between_change_points=min_runs_between_change_points, + test_id=self.test_id) self.assertFalse(is_alert) @mock.patch.object( @@ -154,9 +159,10 @@ def test_duplicate_change_points_are_not_valid_alerts(self): 'fetch_metric_data', get_fake_data_with_no_change_point) def test_no_alerts_when_no_change_points(self): + test_config_container = analysis.get_test_config_container( + params=self.params, test_id=self.test_id) is_alert = analysis.run_change_point_analysis( - params=self.params, - test_id=self.test_id, + test_config_container=test_config_container, big_query_metrics_fetcher=BigQueryMetricsFetcher()) self.assertFalse(is_alert) @@ -176,9 +182,10 @@ def test_no_alerts_when_no_change_points(self): '.create_performance_alert', return_value=(0, '')) def test_alert_on_data_with_change_point(self, *args): + test_config_container = analysis.get_test_config_container( + params=self.params, test_id=self.test_id) is_alert = analysis.run_change_point_analysis( - params=self.params, - test_id=self.test_id, + test_config_container=test_config_container, big_query_metrics_fetcher=BigQueryMetricsFetcher()) self.assertTrue(is_alert) @@ -197,24 +204,32 @@ def test_alert_on_data_with_change_point(self, *args): 'apache_beam.testing.analyzers.perf_analysis.create_performance_alert', return_value=(0, '')) def test_alert_on_data_with_reported_change_point(self, *args): + test_config_container = analysis.get_test_config_container( + params=self.params, test_id=self.test_id) is_alert = analysis.run_change_point_analysis( - params=self.params, - test_id=self.test_id, + test_config_container=test_config_container, big_query_metrics_fetcher=BigQueryMetricsFetcher()) self.assertFalse(is_alert) def test_change_point_has_anomaly_marker_in_gh_description(self): - metric_values, timestamps = get_fake_data_with_change_point() - timestamps = [datetime.datetime.fromtimestamp(ts) for ts in timestamps] + metric_container = get_fake_data_with_change_point() + metric_values = metric_container.values change_point_index = find_latest_change_point_index(metric_values) - description = github_issues_utils.get_issue_description( + test_config_container = TestConfigContainer( + project=self.params['project'], + metrics_dataset=self.params['metrics_dataset'], + metrics_table=self.params['metrics_table'], + metric_name=self.params['metric_name'], test_id=self.test_id, - test_name=self.params.get('test_name', None), test_description=self.params['test_description'], - metric_name=self.params['metric_name'], - metric_values=metric_values, - timestamps=timestamps, + test_name=self.params.get('test_name', None), + labels=self.params.get('labels', None), + ) + + description = github_issues_utils.get_issue_description( + test_config_container=test_config_container, + metric_container=metric_container, change_point_index=change_point_index, max_results_to_display=( constants._NUM_RESULTS_TO_DISPLAY_ON_ISSUE_DESCRIPTION)) diff --git a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py index 91c339a766d2..2b89ac9fdba9 100644 --- a/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/perf_analysis_utils.py @@ -31,7 +31,6 @@ from google.cloud import bigquery from apache_beam.testing.analyzers import constants -from apache_beam.testing.analyzers import github_issues_utils from apache_beam.testing.load_tests import load_test_metrics_utils from apache_beam.testing.load_tests.load_test_metrics_utils import BigQueryMetricsPublisher from signal_processing_algorithms.energy_statistics.energy_statistics import e_divisive @@ -54,6 +53,54 @@ class GitHubIssueMetaData: change_point: float +@dataclass +class ChangePointConfig: + """ + This class holds the change point configuration parameters. + """ + min_runs_between_change_points: int = ( + constants._DEFAULT_MIN_RUNS_BETWEEN_CHANGE_POINTS) + num_runs_in_change_point_window: int = ( + constants._DEFAULT_NUM_RUMS_IN_CHANGE_POINT_WINDOW) + + +@dataclass +class TestConfigContainer: + metric_name: str # make this list instead. + project: str + metrics_dataset: str + metrics_table: str + test_id: str # unique id for each test config. + test_description: str + test_name: Optional[str] = None + labels: Optional[List[str]] = None + + +@dataclass +class MetricContainer: + """ + This class holds the metric values and timestamps for a given metric. + Args: + metric_values: List of metric values. + timestamps: List of pandas timestamps corresponding to the metric values. + """ + + values: List[Union[int, float]] + timestamps: List[pd.Timestamp] + + def sort_by_timestamp(self, in_place=True): + """ + Sorts the metric values and timestamps in ascending order wrt timestamps. + Args: + in_place: If True, sort the metric values and timestamps in place. + """ + timestamps, values = zip(*sorted(zip(self.timestamps, self.values))) + if not in_place: + return MetricContainer(values=values, timestamps=timestamps) + self.timestamps, self.values = zip(*sorted( + zip(self.timestamps, self.values))) + + def is_change_point_in_valid_window( num_runs_in_change_point_window: int, latest_change_point_run: int) -> bool: return num_runs_in_change_point_window > latest_change_point_run @@ -81,12 +128,16 @@ def get_existing_issues_data(table_name: str) -> Optional[pd.DataFrame]: return existing_issue_data -def is_perf_alert( +def is_sibling_change_point( previous_change_point_timestamps: List[pd.Timestamp], change_point_index: int, timestamps: List[pd.Timestamp], - min_runs_between_change_points: int) -> bool: + min_runs_between_change_points: int, + test_id: str, +) -> bool: """ + Sibling change points are the change points that are close to each other. + Search the previous_change_point_timestamps with current observed change point sibling window and determine if it is a duplicate change point or not. @@ -105,6 +156,18 @@ def is_perf_alert( for previous_change_point_timestamp in previous_change_point_timestamps: if (sibling_change_point_min_timestamp <= previous_change_point_timestamp <= sibling_change_point_max_timestamp): + logging.info( + 'Performance regression/improvement found for the test ID: %s. ' + 'Since the change point timestamp %s ' + 'lies within the sibling change point window: %s, ' + 'alert is not raised.' % ( + test_id, + previous_change_point_timestamp.strftime('%Y-%m-%d %H:%M:%S'), + ( + sibling_change_point_min_timestamp.strftime( + '%Y-%m-%d %H:%M:%S'), + sibling_change_point_max_timestamp.strftime( + '%Y-%m-%d %H:%M:%S')))) return False return True @@ -161,12 +224,16 @@ def find_latest_change_point_index(metric_values: List[Union[float, int]]): return change_point_index -def publish_issue_metadata_to_big_query(issue_metadata, table_name): +def publish_issue_metadata_to_big_query( + issue_metadata, + table_name, + project=constants._BQ_PROJECT_NAME, +): """ Published issue_metadata to BigQuery with table name. """ bq_metrics_publisher = BigQueryMetricsPublisher( - project_name=constants._BQ_PROJECT_NAME, + project_name=project, dataset=constants._BQ_DATASET, table=table_name, bq_schema=constants._SCHEMA) @@ -177,37 +244,32 @@ def publish_issue_metadata_to_big_query(issue_metadata, table_name): def create_performance_alert( - metric_name: str, - test_id: str, - timestamps: List[pd.Timestamp], - metric_values: List[Union[int, float]], + test_config_container: TestConfigContainer, + metric_container: MetricContainer, change_point_index: int, - labels: List[str], existing_issue_number: Optional[int], - test_description: Optional[str] = None, - test_name: Optional[str] = None, ) -> Tuple[int, str]: """ Creates performance alert on GitHub issues and returns GitHub issue number and issue URL. """ + # avoid circular imports + # pylint: disable=wrong-import-order, wrong-import-position + from apache_beam.testing.analyzers import github_issues_utils + description = github_issues_utils.get_issue_description( - test_id=test_id, - test_name=test_name, - test_description=test_description, - metric_name=metric_name, - timestamps=timestamps, - metric_values=metric_values, + test_config_container=test_config_container, + metric_container=metric_container, change_point_index=change_point_index, max_results_to_display=( constants._NUM_RESULTS_TO_DISPLAY_ON_ISSUE_DESCRIPTION)) issue_number, issue_url = github_issues_utils.report_change_point_on_issues( title=github_issues_utils._ISSUE_TITLE_TEMPLATE.format( - test_id, metric_name + test_config_container.test_id, test_config_container.metric_name ), description=description, - labels=labels, + labels=test_config_container.labels, existing_issue_number=existing_issue_number) logging.info( @@ -263,13 +325,7 @@ def is_edge_change_point( class MetricsFetcher(metaclass=abc.ABCMeta): @abc.abstractmethod def fetch_metric_data( - self, - *, - project, - metrics_dataset, - metrics_table, - metric_name, - test_name=None) -> Tuple[List[Union[int, float]], List[pd.Timestamp]]: + self, *, test_config: TestConfigContainer) -> MetricContainer: """ Define SQL query and fetch the timestamp values and metric values from BigQuery tables. @@ -279,22 +335,18 @@ def fetch_metric_data( class BigQueryMetricsFetcher(MetricsFetcher): def fetch_metric_data( - self, - *, - project, - metrics_dataset, - metrics_table, - metric_name, - test_name=None, - ) -> Tuple[List[Union[int, float]], List[pd.Timestamp]]: + self, *, test_config: TestConfigContainer) -> MetricContainer: """ Args: - params: Dict containing keys required to fetch data from a data source. + test_config: TestConfigContainer containing metadata required to fetch + metric data from BigQuery. Returns: - Tuple[List[Union[int, float]], List[pd.Timestamp]]: Tuple containing list - of metric_values and list of timestamps. Both are sorted in ascending - order wrt timestamps. + MetricContainer containing metric values and timestamps. """ + project = test_config.project + metrics_dataset = test_config.metrics_dataset + metrics_table = test_config.metrics_table + metric_name = test_config.metric_name query = f""" SELECT * FROM {project}.{metrics_dataset}.{metrics_table} @@ -305,8 +357,9 @@ def fetch_metric_data( client = bigquery.Client() query_job = client.query(query=query) metric_data = query_job.result().to_dataframe() - metric_data.sort_values( - by=[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL], inplace=True) - return ( - metric_data[load_test_metrics_utils.VALUE_LABEL].tolist(), - metric_data[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL].tolist()) + # metric_data.sort_values( + # by=[load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL], inplace=True) + return MetricContainer( + values=metric_data[load_test_metrics_utils.VALUE_LABEL].tolist(), + timestamps=metric_data[ + load_test_metrics_utils.SUBMIT_TIMESTAMP_LABEL].tolist()) diff --git a/sdks/python/apache_beam/testing/analyzers/tests_config.yaml b/sdks/python/apache_beam/testing/analyzers/tests_config.yaml index ec9cfe6f1ac0..2e72cd5cc301 100644 --- a/sdks/python/apache_beam/testing/analyzers/tests_config.yaml +++ b/sdks/python/apache_beam/testing/analyzers/tests_config.yaml @@ -19,7 +19,7 @@ # {test_id}-{metric_name} pytorch_image_classification_benchmarks-resnet152-mean_inference_batch_latency_micro_secs: - test_description: + test_description: | Pytorch image classification on 50k images of size 224 x 224 with resnet 152. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L63 Test dashboard - http://metrics.beam.apache.org/d/ZpS8Uf44z/python-ml-runinference-benchmarks?orgId=1&viewPanel=2 @@ -30,7 +30,7 @@ pytorch_image_classification_benchmarks-resnet152-mean_inference_batch_latency_m metric_name: mean_inference_batch_latency_micro_secs pytorch_image_classification_benchmarks-resnet101-mean_load_model_latency_milli_secs: - test_description: + test_description: | Pytorch image classification on 50k images of size 224 x 224 with resnet 101. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L34 Test dashboard - http://metrics.beam.apache.org/d/ZpS8Uf44z/python-ml-runinference-benchmarks?orgId=1&viewPanel=7 @@ -41,7 +41,7 @@ pytorch_image_classification_benchmarks-resnet101-mean_load_model_latency_milli_ metric_name: mean_load_model_latency_milli_secs pytorch_image_classification_benchmarks-resnet101-mean_inference_batch_latency_micro_secs: - test_description: + test_description: | Pytorch image classification on 50k images of size 224 x 224 with resnet 101. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L34 Test dashboard - http://metrics.beam.apache.org/d/ZpS8Uf44z/python-ml-runinference-benchmarks?orgId=1&viewPanel=2 @@ -52,7 +52,7 @@ pytorch_image_classification_benchmarks-resnet101-mean_inference_batch_latency_m metric_name: mean_inference_batch_latency_micro_secs pytorch_image_classification_benchmarks-resnet152-GPU-mean_inference_batch_latency_micro_secs: - test_description: + test_description: | Pytorch image classification on 50k images of size 224 x 224 with resnet 152 with Tesla T4 GPU. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L151 Test dashboard - http://metrics.beam.apache.org/d/ZpS8Uf44z/python-ml-runinference-benchmarks?orgId=1&viewPanel=7 @@ -63,7 +63,7 @@ pytorch_image_classification_benchmarks-resnet152-GPU-mean_inference_batch_laten metric_name: mean_inference_batch_latency_micro_secs pytorch_image_classification_benchmarks-resnet152-GPU-mean_load_model_latency_milli_secs: - test_description: + test_description: | Pytorch image classification on 50k images of size 224 x 224 with resnet 152 with Tesla T4 GPU. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L151 Test dashboard - http://metrics.beam.apache.org/d/ZpS8Uf44z/python-ml-runinference-benchmarks?orgId=1&viewPanel=7 @@ -74,7 +74,7 @@ pytorch_image_classification_benchmarks-resnet152-GPU-mean_load_model_latency_mi metric_name: mean_load_model_latency_milli_secs pytorch_image_classification_benchmarks-resnet152-GPU-mean_inference_batch_latency_micro_secs: - test_description: + test_description: | Pytorch image classification on 50k images of size 224 x 224 with resnet 152 with Tesla T4 GPU. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/.test-infra/jenkins/job_InferenceBenchmarkTests_Python.groovy#L151). Test dashboard - http://metrics.beam.apache.org/d/ZpS8Uf44z/python-ml-runinference-benchmarks?from=now-90d&to=now&viewPanel=2 @@ -85,7 +85,7 @@ pytorch_image_classification_benchmarks-resnet152-GPU-mean_inference_batch_laten metric_name: mean_inference_batch_latency_micro_secs test_cloudml_benchmark_cirteo_no_shuffle_10GB-runtime_sec: - test_description: + test_description: | TFT Criteo test on 10 GB data with no Reshuffle. Test link - [Test link](https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/sdks/python/apache_beam/testing/benchmarks/cloudml/cloudml_benchmark_test.py#L82) metrics_dataset: beam_cloudml @@ -94,7 +94,7 @@ test_cloudml_benchmark_cirteo_no_shuffle_10GB-runtime_sec: metric_name: runtime_sec test_cloudml_benchmark_criteo_10GB-runtime_sec: - test_description: + test_description: | TFT Criteo test on 10 GB data. Test link - https://github.com/apache/beam/blob/42d0a6e3564d8b9c5d912428a6de18fb22a13ac1/sdks/python/apache_beam/testing/benchmarks/cloudml/cloudml_benchmark_test.py#LL104C7-L104C41 metrics_dataset: beam_cloudml @@ -104,7 +104,7 @@ test_cloudml_benchmark_criteo_10GB-runtime_sec: # Python Combine load tests at http://metrics.beam.apache.org/d/WNzYt13Zk/combine-load-tests?orgId=1 combine_python_batch_2gb_10_byte_records: - test_description: + test_description: | Combine Python Load Test 2 GB 10 byte records Test link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_Combine_Python.groovy#L76C24-L76C65 Test dashboard - http://metrics.beam.apache.org/d/WNzYt13Zk/combine-load-tests?orgId=1&from=now-90d&to=now&var-processingType=batch&var-sdk=python&viewPanel=2 @@ -115,7 +115,7 @@ combine_python_batch_2gb_10_byte_records: project: apache-beam-testing combine_python_batch_2gb_fanout_4: - test_description: + test_description: | Combine Python Load test - 2GB Fanout 4 Test link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_Combine_Python.groovy#L52 Test Dashboard - http://metrics.beam.apache.org/d/WNzYt13Zk/combine-load-tests?orgId=1&from=now-90d&to=now&var-processingType=batch&var-sdk=python&viewPanel=4 @@ -126,7 +126,8 @@ combine_python_batch_2gb_fanout_4: project: apache-beam-testing combine_python_batch_2gb_fanout_8: - test_description: Combine Python Load test - 2GB Fanout 8 + test_description: | + Combine Python Load test - 2GB Fanout 8 test_target: apache_beam.testing.load_tests.combine_test metrics_dataset: load_test metrics_table: python_dataflow_batch_combine_5 @@ -135,7 +136,7 @@ combine_python_batch_2gb_fanout_8: # Python Batch GBK load tests at http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now gbk_python_batch_load_test_2gb_of_10B_records: - test_description: + test_description: | GroupByKey Python Load test - 2GB of 10B records python | GBK | Small records (10B) Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now&viewPanel=2 @@ -147,7 +148,7 @@ gbk_python_batch_load_test_2gb_of_10B_records: project: apache-beam-testing gbk_python_batch_load_test_2gb_of_100B_records: - test_description: + test_description: | GroupByKey Python Load test - 2GB of 100B records python | GBK | Medium records (100B) Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now&viewPanel=3 @@ -159,7 +160,7 @@ gbk_python_batch_load_test_2gb_of_100B_records: project: apache-beam-testing gbk_python_batch_load_test_2gb_of_100KB_records: - test_description: + test_description: | GroupByKey Python Load test - 2GB of 100kB records python | GBK | Large records (100kB) Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-6M&to=now&viewPanel=4&inspect=4 @@ -173,7 +174,7 @@ gbk_python_batch_load_test_2gb_of_100KB_records: gbk_python_batch_load_test_fanout_4_times_with_2GB_10byte_records_total: # this test looks little noisy. Remove this if it causes too many false # positives. - test_description: + test_description: | GroupByKey Python Load test - fanout 4 times with 2GB 10-byte records total python | GBK | Fanout 4 Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now&viewPanel=5 @@ -188,7 +189,7 @@ gbk_python_batch_load_test_fanout_4_times_with_2GB_10byte_records_total: gbk_python_batch_load_test_fanout_8_times_with_2GB_10byte_records_total: # this test looks little noisy. Remove this if it causes too many false # positives. - test_description: + test_description: | GroupByKey Python Load test - fanout 8 times with 2GB 10-byte records total python | GBK | Fanout 8 Test Dashboard - http://metrics.beam.apache.org/d/UYZ-oJ3Zk/gbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python&from=now-90d&to=now&viewPanel=6 @@ -201,7 +202,7 @@ gbk_python_batch_load_test_fanout_8_times_with_2GB_10byte_records_total: # Python SideInput load tests at http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now sideinpts_python_batch_1gb_1kb_10workers_1000window_1key_percent_dict: - test_description: + test_description: | python | Side Input | 1 GB dictionary, 1% of keys, 1000 fixed windows Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy#L120 Test Dashboard - http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now&viewPanel=8 @@ -212,7 +213,7 @@ sideinpts_python_batch_1gb_1kb_10workers_1000window_1key_percent_dict: sideinpts_python_batch_1gb_1kb_10workers_1000window_99key_percent_dict: - test_description: + test_description: | python | Side Input | 1 GB dictionary, 99% of keys, 1000 fixed windows Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy#L133 Test Dashboard - http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now&viewPanel=9 @@ -222,7 +223,7 @@ sideinpts_python_batch_1gb_1kb_10workers_1000window_99key_percent_dict: project: apache-beam-testing sideinpts_python_batch_10gb_1kb_10workers_1000window_first_iterable: - test_description: + test_description: | python | Side Input | 10 GB iterable, 1% of elements, 1000 fixed windows Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy#L146 Test Dashboard - http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now&viewPanel=10 @@ -233,7 +234,7 @@ sideinpts_python_batch_10gb_1kb_10workers_1000window_first_iterable: sideinpts_python_batch_10gb_1kb_10workers_1000window_first_iterable: - test_description: + test_description: | python | Side Input | 10 GB iterable, all elements, 1000 fixed windows Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_SideInput_Python.groovy#L159 Test Dashboard - http://metrics.beam.apache.org/d/-E9aGlFGk/side-input-load-tests?orgId=1&from=now-90d&to=now&viewPanel=11 @@ -245,7 +246,7 @@ sideinpts_python_batch_10gb_1kb_10workers_1000window_first_iterable: # Python CoGBK load tests at http://metrics.beam.apache.org/d/fK0U4JqWz/cogbk-load-tests?orgId=1&var-processingType=batch&var-sdk=python cogbk_python_batch_load_test_2GB_of_100B_records_with_a_single_key: - test_description: + test_description: | CoGroupByKey Python Load test - 2GB of 100B records with a single key python | coGBK | 100B records with a single key Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy#L32C25-L32C76 @@ -257,7 +258,7 @@ cogbk_python_batch_load_test_2GB_of_100B_records_with_a_single_key: project: apache-beam-testing cogbk_python_batch_load_test_2GB_of_100B_records_with_a_multiple_key: - test_description: + test_description: | CoGroupByKey Python Load test - 2GB of 100B records with multiple keys python | coGBK | 100B records with multiple keys @@ -270,7 +271,7 @@ cogbk_python_batch_load_test_2GB_of_100B_records_with_a_multiple_key: project: apache-beam-testing cogbk_python_batch_load_test_reiterate_4times_10KB_values: - test_description: + test_description: | CoGroupByKey Python Load test - reiterate 4 times 10kB values python | coGBK | reiteration 10kB value Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy#L96 @@ -281,7 +282,7 @@ cogbk_python_batch_load_test_reiterate_4times_10KB_values: project: apache-beam-testing cogbk_python_batch_load_test_reiterate_4times_2MB_values: - test_description: + test_description: | CoGroupByKey Python Load test - reiterate 4 times 2 MB values python | coGBK | reiteration 2MB value Test Link - https://github.com/apache/beam/blob/5e38decf9e723a385057131b01bbd33d8c60bda3/.test-infra/jenkins/job_LoadTests_coGBK_Python.groovy#L128 From d17063412ea4a3417b5a4e3f2d14fa1640c8caed Mon Sep 17 00:00:00 2001 From: RyuSA <12961775+RyuSA@users.noreply.github.com> Date: Tue, 10 Oct 2023 01:31:46 +0900 Subject: [PATCH 36/85] Update BigQueryIO Documentation (#28591) * Update BigQueryIO Documentation - Updated the description regarding failed rows for Storage Write API. - Made `PCollection` formatting consistent. * Update website/www/site/content/en/documentation/io/built-in/google-bigquery.md --- .../documentation/io/built-in/google-bigquery.md | 14 ++++++-------- 1 file changed, 6 insertions(+), 8 deletions(-) diff --git a/website/www/site/content/en/documentation/io/built-in/google-bigquery.md b/website/www/site/content/en/documentation/io/built-in/google-bigquery.md index 7a31b63a3c96..26ca0baec0cf 100644 --- a/website/www/site/content/en/documentation/io/built-in/google-bigquery.md +++ b/website/www/site/content/en/documentation/io/built-in/google-bigquery.md @@ -883,10 +883,9 @@ explicitly enable this using [`withAutoSharding`](https://beam.apache.org/releas ***Note:*** Auto sharding with `STORAGE_WRITE_API` is supported on Dataflow's legacy runner, but **not** on Runner V2 {{< /paragraph >}} -When using `STORAGE_WRITE_API`, the PCollection returned by -[`WriteResult.getFailedInserts`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.html#getFailedInserts--) -will not contain the failed rows. If there are data validation errors, the -transform will throw a `RuntimeException`. +When using `STORAGE_WRITE_API`, the `PCollection` returned by +[`WriteResult.getFailedStorageApiInserts`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.html#getFailedStorageApiInserts--) +will contain the rows that failed to be written to the Storage Write API sink. #### At-least-once semantics @@ -901,10 +900,9 @@ specify the number of streams, and you can’t specify the triggering frequency. Auto sharding is not applicable for `STORAGE_API_AT_LEAST_ONCE`. -When using `STORAGE_API_AT_LEAST_ONCE`, the PCollection returned by -[`WriteResult.getFailedInserts`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.html#getFailedInserts--) -will not contain the failed rows. If there are data validation errors, the -transform will throw a `RuntimeException`. +When using `STORAGE_API_AT_LEAST_ONCE`, the `PCollection` returned by +[`WriteResult.getFailedStorageApiInserts`](https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/gcp/bigquery/WriteResult.html#getFailedStorageApiInserts--) +will contain the rows that failed to be written to the Storage Write API sink. #### Quotas From 66bae3fa5bad949250688f45d34e22575c0ac1c0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Oct 2023 13:18:14 -0400 Subject: [PATCH 37/85] Bump github.com/aws/aws-sdk-go-v2/feature/s3/manager in /sdks (#28890) Bumps [github.com/aws/aws-sdk-go-v2/feature/s3/manager](https://github.com/aws/aws-sdk-go-v2) from 1.11.87 to 1.11.89. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/feature/s3/manager/v1.11.87...feature/s3/manager/v1.11.89) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/feature/s3/manager 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 | 34 ++++++++++++------------ sdks/go.sum | 74 ++++++++++++++++++++++++----------------------------- 2 files changed, 51 insertions(+), 57 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index b8dd03de6b07..d5b7d7e3e41e 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -31,10 +31,10 @@ require ( cloud.google.com/go/spanner v1.50.0 cloud.google.com/go/storage v1.33.0 github.com/aws/aws-sdk-go-v2 v1.21.1 - github.com/aws/aws-sdk-go-v2/config v1.18.43 - github.com/aws/aws-sdk-go-v2/credentials v1.13.41 - github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.87 - github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 + github.com/aws/aws-sdk-go-v2/config v1.18.44 + github.com/aws/aws-sdk-go-v2/credentials v1.13.42 + github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.89 + github.com/aws/aws-sdk-go-v2/service/s3 v1.40.1 github.com/aws/smithy-go v1.15.0 github.com/docker/go-connections v0.4.0 github.com/dustin/go-humanize v1.0.1 @@ -97,19 +97,19 @@ require ( github.com/apache/arrow/go/v12 v12.0.0 // indirect 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.4.13 // indirect - github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11 // indirect - github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41 // indirect - github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.35 // indirect - github.com/aws/aws-sdk-go-v2/internal/ini v1.3.43 // indirect - github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.4 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.14 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.36 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.35 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4 // indirect - github.com/aws/aws-sdk-go-v2/service/sso v1.15.0 // indirect - github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.1 // indirect - github.com/aws/aws-sdk-go-v2/service/sts v1.23.0 // indirect + github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 // indirect + github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.12 // indirect + github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.42 // indirect + github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.36 // indirect + github.com/aws/aws-sdk-go-v2/internal/ini v1.3.44 // indirect + github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.5 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.37 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.36 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.5 // indirect + github.com/aws/aws-sdk-go-v2/service/sso v1.15.1 // indirect + github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.2 // indirect + github.com/aws/aws-sdk-go-v2/service/sts v1.23.1 // 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 f177dd8debb8..7891171f2095 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -81,60 +81,54 @@ 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.21.0/go.mod h1:/RfNgGmRxI+iFOB1OeJUyxiU+9s88k3pfHvDagGEp0M= github.com/aws/aws-sdk-go-v2 v1.21.1 h1:wjHYshtPpYOZm+/mu3NhVgRRc0baM6LJZOmxPZ5Cwzs= github.com/aws/aws-sdk-go-v2 v1.21.1/go.mod h1:ErQhvNuEMhJjweavOYhxVkn2RUx7kQXVATHrjKtxIpM= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13 h1:OPLEkmhXf6xFPiz0bLeDArZIDx1NNS4oJyG4nv3Gct0= -github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.13/go.mod h1:gpAbvyDGQFozTEmlTFO8XcQKHzubdq0LzRyJpG6MiXM= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14 h1:Sc82v7tDQ/vdU1WtuSyzZ1I7y/68j//HJ6uozND1IDs= +github.com/aws/aws-sdk-go-v2/aws/protocol/eventstream v1.4.14/go.mod h1:9NCTOURS8OpxvoAVHq79LK81/zC78hfRWFn+aL0SPcY= 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.18.42/go.mod h1:4AZM3nMMxwlG+eZlxvBKqwVbkDLlnN2a4UGTL6HjaZI= -github.com/aws/aws-sdk-go-v2/config v1.18.43 h1:IgdUtTRvUDC6eiJBqU6vh7bHFNAEBjQ8S+qJ7zVhDOs= -github.com/aws/aws-sdk-go-v2/config v1.18.43/go.mod h1:NiFev8qlgg8MPzw3fO/EwzMZeZwlJEKGwfpjRPA9Nvw= +github.com/aws/aws-sdk-go-v2/config v1.18.44 h1:U10NQ3OxiY0dGGozmVIENIDnCT0W432PWxk2VO8wGnY= +github.com/aws/aws-sdk-go-v2/config v1.18.44/go.mod h1:pHxnQBldd0heEdJmolLBk78D1Bf69YnKLY3LOpFImlU= github.com/aws/aws-sdk-go-v2/credentials v1.3.1/go.mod h1:r0n73xwsIVagq8RsxmZbGSRQFj9As3je72C2WzUIToc= -github.com/aws/aws-sdk-go-v2/credentials v1.13.40/go.mod h1:VtEHVAAqDWASwdOqj/1huyT6uHbs5s8FUHfDQdky/Rs= -github.com/aws/aws-sdk-go-v2/credentials v1.13.41 h1:dgbKq1tamtboYAKSXWbqL0lKO9rmEzEhbZFh9JQW/Bg= -github.com/aws/aws-sdk-go-v2/credentials v1.13.41/go.mod h1:cc3Fn7DkKbJalPtQnudHGZZ8ml9+hwtbc1CJONsYYqk= +github.com/aws/aws-sdk-go-v2/credentials v1.13.42 h1:KMkjpZqcMOwtRHChVlHdNxTUUAC6NC/b58mRZDIdcRg= +github.com/aws/aws-sdk-go-v2/credentials v1.13.42/go.mod h1:7ltKclhvEB8305sBhrpls24HGxORl6qgnQqSJ314Uw8= 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.13.11 h1:uDZJF1hu0EVT/4bogChk8DyjSF6fof6uL/0Y26Ma7Fg= -github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.11/go.mod h1:TEPP4tENqBGO99KwVpV9MlOX4NSrSLP8u3KRy2CDwA8= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.12 h1:3j5lrl9kVQrJ1BU4O0z7MQ8sa+UXdiLuo4j0V+odNI8= +github.com/aws/aws-sdk-go-v2/feature/ec2/imds v1.13.12/go.mod h1:JbFpcHDBdsex1zpIKuVRorZSQiZEyc3MykNCcjgz174= 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.11.87 h1:e20ZrsgDPUXqg8+rZVuPwNSp6yniUN2Yr2tzFZ+Yvl0= -github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.87/go.mod h1:0i0TAT6W+5i48QTlDU2KmY6U2hBZeY/LCP0wktya2oc= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41 h1:22dGT7PneFMx4+b3pz7lMTRyN8ZKH7M2cW4GP9yUS2g= -github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.41/go.mod h1:CrObHAuPneJBlfEJ5T3szXOUkLEThaGfvnhTf33buas= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.35 h1:SijA0mgjV8E+8G45ltVHs0fvKpTj8xmZJ3VwhGKtUSI= -github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.35/go.mod h1:SJC1nEVVva1g3pHAIdCp7QsRIkMmLAgoDquQ9Rr8kYw= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.89 h1:XPqSyw8SBSLMRrF9Oip6tQpivXWJLMn8sdRoAsUCQQA= +github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.11.89/go.mod h1:OkYwM7gYm9HieL6emYtkg7Pb7Jd8FFM5Pl5uAZ1h2jo= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.42 h1:817VqVe6wvwE46xXy6YF5RywvjOX6U2zRQQ6IbQFK0s= +github.com/aws/aws-sdk-go-v2/internal/configsources v1.1.42/go.mod h1:oDfgXoBBmj+kXnqxDDnIDnC56QBosglKp8ftRCTxR+0= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.36 h1:7ZApaXzWbo8slc+W5TynuUlB4z66g44h7uqa3/d/BsY= +github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.4.36/go.mod h1:rwr4WnmFi3RJO0M4dxbJtgi9BPLMpVBMX1nUte5ha9U= 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.3.43 h1:g+qlObJH4Kn4n21g69DjspU0hKTjWtq7naZ9OLCv0ew= -github.com/aws/aws-sdk-go-v2/internal/ini v1.3.43/go.mod h1:rzfdUlfA+jdgLDmPKjd3Chq9V7LVLYo1Nz++Wb91aRo= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.4 h1:6lJvvkQ9HmbHZ4h/IEwclwv2mrTW8Uq1SOB/kXy0mfw= -github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.4/go.mod h1:1PrKYwxTM+zjpw9Y41KFtoJCQrJ34Z47Y4VgVbfndjo= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.44 h1:quOJOqlbSfeJTboXLjYXM1M9T52LBXqLoTPlmsKLpBo= +github.com/aws/aws-sdk-go-v2/internal/ini v1.3.44/go.mod h1:LNy+P1+1LiRcCsVYr/4zG5n8zWFL0xsvZkOybjbftm8= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.5 h1:8JG9ny0BqBDzmtIzbpaN+eke152ZNsYKApFJ/q29Hxo= +github.com/aws/aws-sdk-go-v2/internal/v4a v1.1.5/go.mod h1:kEDHQApP/ukMO9natNftgUN3NaTsMxK6jb2jjpSMX7Y= github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.2.1/go.mod h1:v33JQ57i2nekYTA70Mb+O18KeH4KqhdqxTJZNK1zdRE= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.14 h1:m0QTSI6pZYJTk5WSKx3fm5cNW/DCicVzULBgU/6IyD0= -github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.14/go.mod h1:dDilntgHy9WnHXsh7dDtUPgHKEfTJIBUTHM8OWm0f/0= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.36 h1:eev2yZX7esGRjqRbnVk1UxMLw4CyVZDpZXRCcy75oQk= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.36/go.mod h1:lGnOkH9NJATw0XEPcAknFBj3zzNTEGRHtSw+CwC1YTg= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15 h1:7R8uRYyXzdD71KWVCL78lJZltah6VVznXBazvKjfH58= +github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.9.15/go.mod h1:26SQUPcTNgV1Tapwdt4a1rOsYRsnBsJHLMPoxK2b0d8= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.37 h1:Mx1zJlYbiUQANWT40koevLvxawGFolmkaP4m+LuyG7M= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.37/go.mod h1:PjKIAMFthKPgG/B8bbRpo3F8jfr2q2L+w3u78jJ12a0= 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.9.35 h1:CdzPW9kKitgIiLV1+MHobfR5Xg25iYnyzWZhyQuSlDI= -github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.35/go.mod h1:QGF2Rs33W5MaN9gYdEQOBBFPLwTZkEhRwI33f7KIG0o= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.36 h1:YXlm7LxwNlauqb2OrinWlcvtsflTzP8GaMvYfQBhoT4= +github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.36/go.mod h1:ou9ffqJ9hKOVZmjlC6kQ6oROAyG1M4yBKzR+9BKbDwk= 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.15.4 h1:v0jkRigbSD6uOdwcaUQmgEwG1BkPfAPDqaeNt/29ghg= -github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.4/go.mod h1:LhTyt8J04LL+9cIt7pYJ5lbS/U98ZmXovLOR/4LUsk8= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.5 h1:sAAz28SeA7YZl8Yaphjs9tlLsflhdniQPjf3X2cqr4s= +github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.15.5/go.mod h1:HC7gNz3VH0p+RvLKK+HqNQv/gHy+1Os3ko/F41s3+aw= github.com/aws/aws-sdk-go-v2/service/s3 v1.11.1/go.mod h1:XLAGFrEjbvMCLvAtWLLP32yTv8GpBquCApZEycDLunI= -github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0 h1:wl5dxN1NONhTDQD9uaEvNsDRX29cBmGED/nl0jkWlt4= -github.com/aws/aws-sdk-go-v2/service/s3 v1.40.0/go.mod h1:rDGMZA7f4pbmTtPOk5v5UM2lmX6UAbRnMDJeDvnH7AM= +github.com/aws/aws-sdk-go-v2/service/s3 v1.40.1 h1:FqIaVPbs2W8U3fszl2PCL1IDKeRdM7TssjWamL6b2mg= +github.com/aws/aws-sdk-go-v2/service/s3 v1.40.1/go.mod h1:X0e0NCAx4GjOrKro7s9QYy+YEIFhgCkt6gYKVKhZB5Y= 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.14.1/go.mod h1:fIAwKQKBFu90pBxx07BFOMJLpRUGu8VOzLJakeY+0K4= -github.com/aws/aws-sdk-go-v2/service/sso v1.15.0 h1:vuGK1vHNP9zx0PfOrtPumbwR2af0ATQ1Z2H6p75AgRQ= -github.com/aws/aws-sdk-go-v2/service/sso v1.15.0/go.mod h1:fIAwKQKBFu90pBxx07BFOMJLpRUGu8VOzLJakeY+0K4= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.1 h1:8lKOidPkmSmfUtiTgtdXWgaKItCZ/g75/jEk6Ql6GsA= -github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.1/go.mod h1:yygr8ACQRY2PrEcy3xsUI357stq2AxnFM6DIsR9lij4= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.1 h1:ZN3bxw9OYC5D6umLw6f57rNJfGfhg1DIAAcKpzyUTOE= +github.com/aws/aws-sdk-go-v2/service/sso v1.15.1/go.mod h1:PieckvBoT5HtyB9AsJRrYZFY2Z+EyfVM/9zG6gbV8DQ= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.2 h1:fSCCJuT5i6ht8TqGdZc5Q5K9pz/atrf7qH4iK5C9XzU= +github.com/aws/aws-sdk-go-v2/service/ssooidc v1.17.2/go.mod h1:5eNtr+vNc5vVd92q7SJ+U/HszsIdhZBEyi9dkMRKsp8= 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.22.0/go.mod h1:VC7JDqsqiwXukYEDjoHh9U0fOJtNWh04FPQz4ct4GGU= -github.com/aws/aws-sdk-go-v2/service/sts v1.23.0 h1:pyvfUqkNLMipdKNAtu7OVbRxUrR2BMaKccIPpk/Hkak= -github.com/aws/aws-sdk-go-v2/service/sts v1.23.0/go.mod h1:VC7JDqsqiwXukYEDjoHh9U0fOJtNWh04FPQz4ct4GGU= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.1 h1:ASNYk1ypWAxRhJjKS0jBnTUeDl7HROOpeSMu1xDA/I8= +github.com/aws/aws-sdk-go-v2/service/sts v1.23.1/go.mod h1:2cnsAhVT3mqusovc2stUSUrSBGTcX9nh8Tu6xh//2eI= github.com/aws/smithy-go v1.6.0/go.mod h1:SObp3lf9smib00L/v3U2eAKG8FyQ7iLrJnQiAmR5n+E= -github.com/aws/smithy-go v1.14.2/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/aws/smithy-go v1.15.0 h1:PS/durmlzvAFpQHDs4wi4sNNP9ExsqZh6IlfdHXgKK8= github.com/aws/smithy-go v1.15.0/go.mod h1:Tg+OJXh4MB2R/uN61Ko2f6hTZwB/ZYGOtib8J3gBHzA= github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= From ad5c2f55bfefed2e91339b1d3ff1aae67ec6b5f6 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Oct 2023 13:19:02 -0400 Subject: [PATCH 38/85] Bump golang.org/x/sync from 0.3.0 to 0.4.0 in /sdks (#28856) Bumps [golang.org/x/sync](https://github.com/golang/sync) from 0.3.0 to 0.4.0. - [Commits](https://github.com/golang/sync/compare/v0.3.0...v0.4.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 d5b7d7e3e41e..1014531ec0f9 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -54,7 +54,7 @@ require ( go.mongodb.org/mongo-driver v1.12.1 golang.org/x/net v0.16.0 golang.org/x/oauth2 v0.12.0 - golang.org/x/sync v0.3.0 + golang.org/x/sync v0.4.0 golang.org/x/sys v0.13.0 golang.org/x/text v0.13.0 google.golang.org/api v0.144.0 diff --git a/sdks/go.sum b/sdks/go.sum index 7891171f2095..dc6318b72025 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -558,8 +558,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.3.0 h1:ftCYgMx6zT/asHUrPw8BLLscYtGznsLAnjq5RH9P66E= -golang.org/x/sync v0.3.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= +golang.org/x/sync v0.4.0 h1:zxkM55ReGkDlKSM+Fu41A+zmbZuaPVbGMzvvdUPznYQ= +golang.org/x/sync v0.4.0/go.mod h1:FU7BRWz2tNW+3quACPkgCx/L+uEAv1htQ0V83Z9Rj+Y= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= From 5491b5cdabc37e84fb836cfc610e52b7702fdc91 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Mon, 9 Oct 2023 10:49:11 -0700 Subject: [PATCH 39/85] Update container version in BeamModulePlugin.groovy (#28879) Ensure boot loaders are built using the latest Go patch release. --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 3a332d8d4d08..9f4ae8bcfb29 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2154,7 +2154,7 @@ class BeamModulePlugin implements Plugin { def goRootDir = "${project.rootDir}/sdks/go" // This sets the whole project Go version. - project.ext.goVersion = "go1.21.1" + project.ext.goVersion = "go1.21.2" // Minor TODO: Figure out if we can pull out the GOCMD env variable after goPrepare script // completion, and avoid this GOBIN substitution. From be3909a62d74c32b2b317918b6480db5fe63cf46 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Oct 2023 10:50:28 -0700 Subject: [PATCH 40/85] Bump cloud.google.com/go/bigquery from 1.55.0 to 1.56.0 in /sdks (#28892) Bumps [cloud.google.com/go/bigquery](https://github.com/googleapis/google-cloud-go) from 1.55.0 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/bigquery/v1.55.0...bigquery/v1.56.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/bigquery 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 1014531ec0f9..bf55e376467b 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -23,7 +23,7 @@ module github.com/apache/beam/sdks/v2 go 1.20 require ( - cloud.google.com/go/bigquery v1.55.0 + cloud.google.com/go/bigquery v1.56.0 cloud.google.com/go/bigtable v1.20.0 cloud.google.com/go/datastore v1.14.0 cloud.google.com/go/profiler v0.3.1 diff --git a/sdks/go.sum b/sdks/go.sum index dc6318b72025..3579eedeb512 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -13,8 +13,8 @@ cloud.google.com/go v0.110.7/go.mod h1:+EYjdK8e5RME/VY/qLCAtuyALQ9q67dvuum8i+H5x 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= -cloud.google.com/go/bigquery v1.55.0 h1:hs44Xxov3XLWQiCx2J8lK5U/ihLqnpm4RVVl5fdtLLI= -cloud.google.com/go/bigquery v1.55.0/go.mod h1:9Y5I3PN9kQWuid6183JFhOGOW3GcirA5LpsKCUn+2ec= +cloud.google.com/go/bigquery v1.56.0 h1:LHIc9E7Kw+ftFpQFKzZYBB88IAFz7qONawXXx0F3QBo= +cloud.google.com/go/bigquery v1.56.0/go.mod h1:KDcsploXTEY7XT3fDQzMUZlpQLHzE4itubHrnmhUrZA= cloud.google.com/go/bigtable v1.20.0 h1:NqZC/WcesSn4O8L0I2JmuNsUigSyBQifVLYgM9LMQeQ= cloud.google.com/go/bigtable v1.20.0/go.mod h1:upJDn8frsjzpRMfybiWkD1PG6WCCL7CRl26MgVeoXY4= cloud.google.com/go/compute v1.23.0 h1:tP41Zoavr8ptEqaW6j+LQOnyBBhO7OkOMAGrgLopTwY= From 9eac03027bcfed79de751d0d2f1dc05f87b19e5b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 9 Oct 2023 10:52:05 -0700 Subject: [PATCH 41/85] Bump github.com/testcontainers/testcontainers-go in /sdks (#28761) Bumps [github.com/testcontainers/testcontainers-go](https://github.com/testcontainers/testcontainers-go) from 0.24.1 to 0.25.0. - [Release notes](https://github.com/testcontainers/testcontainers-go/releases) - [Commits](https://github.com/testcontainers/testcontainers-go/compare/v0.24.1...v0.25.0) --- updated-dependencies: - dependency-name: github.com/testcontainers/testcontainers-go 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 | 8 ++++---- sdks/go.sum | 20 ++++++++++---------- 2 files changed, 14 insertions(+), 14 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index bf55e376467b..a2b5ed097642 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -47,7 +47,7 @@ require ( github.com/linkedin/goavro/v2 v2.12.0 github.com/proullon/ramsql v0.1.3 github.com/spf13/cobra v1.7.0 - github.com/testcontainers/testcontainers-go v0.24.1 + github.com/testcontainers/testcontainers-go v0.25.0 github.com/tetratelabs/wazero v1.5.0 github.com/xitongsys/parquet-go v1.6.2 github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c @@ -77,10 +77,10 @@ require ( github.com/go-ole/go-ole v1.2.6 // indirect github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect - github.com/shirou/gopsutil/v3 v3.23.7 // indirect + github.com/shirou/gopsutil/v3 v3.23.8 // indirect github.com/shoenig/go-m1cpu v0.1.6 // indirect - github.com/tklauser/go-sysconf v0.3.11 // indirect - github.com/tklauser/numcpus v0.6.0 // 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 ) diff --git a/sdks/go.sum b/sdks/go.sum index 3579eedeb512..516adec93875 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -396,8 +396,8 @@ github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46/go.mod h1:uAQ5P 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.7 h1:C+fHO8hfIppoJ1WdsVm1RoI0RwXoNdfTK7yWXV0wVj4= -github.com/shirou/gopsutil/v3 v3.23.7/go.mod h1:c4gnmoRC0hQuaLqvxnx1//VXQ0Ms/X9UnJF8pddY5z4= +github.com/shirou/gopsutil/v3 v3.23.8 h1:xnATPiybo6GgdRoC4YoGnxXZFRc3dqQTGi73oLvvBrE= +github.com/shirou/gopsutil/v3 v3.23.8/go.mod h1:7hmCaBn+2ZwaZOr6jmPBZDfawwMGuo1id3C6aM8EDqQ= github.com/shoenig/go-m1cpu v0.1.6 h1:nxdKQNcEB6vzgA2E2bvzKIYRuNj7XNJ4S/aRSwKzFtM= github.com/shoenig/go-m1cpu v0.1.6/go.mod h1:1JJMcUBvfNwpq05QDQVAnx3gUHr9IYF7GNg9SUEw2VQ= github.com/shoenig/test v0.6.4 h1:kVTaSd7WLz5WZ2IaoM0RSzRsUD+m8wRR+5qvntpn4LU= @@ -428,14 +428,14 @@ github.com/stretchr/testify v1.8.1/go.mod h1:w2LPCIKwWwSfY2zedu0+kehJoqGctiVI29o 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.24.1 h1:gJdZuQIVWnMJTo+CmQMEP7/CAagNk/0jbcUPn3OWvD8= -github.com/testcontainers/testcontainers-go v0.24.1/go.mod h1:MGBiAkCm86yXQoCiipmQCqZLVdk1uFqtMqaU1Or0MRk= +github.com/testcontainers/testcontainers-go v0.25.0 h1:erH6cQjsaJrH+rJDU9qIf89KFdhK0Bft0aEZHlYC3Vs= +github.com/testcontainers/testcontainers-go v0.25.0/go.mod h1:4sC9SiJyzD1XFi59q8umTQYWxnkweEc5OjVtTUlJzqQ= github.com/tetratelabs/wazero v1.5.0 h1:Yz3fZHivfDiZFUXnWMPUoiW7s8tC1sjdBtlJn08qYa0= github.com/tetratelabs/wazero v1.5.0/go.mod h1:0U0G41+ochRKoPKCJlh0jMg1CHkyfK8kDqiirMmKY8A= -github.com/tklauser/go-sysconf v0.3.11 h1:89WgdJhk5SNwJfu+GKyYveZ4IaJ7xAkecBo+KdJV0CM= -github.com/tklauser/go-sysconf v0.3.11/go.mod h1:GqXfhXY3kiPa0nAXPDIQIWzJbMCB7AmcWpGR8lSZfqI= -github.com/tklauser/numcpus v0.6.0 h1:kebhY2Qt+3U6RNK7UqpYNA+tJ23IBEGKkB7JQBfDYms= -github.com/tklauser/numcpus v0.6.0/go.mod h1:FEZLMke0lhOUG6w2JadTzp0a+Nl8PF/GFkQ5UVIcaL4= +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= +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= @@ -597,9 +597,9 @@ golang.org/x/sys v0.0.0-20220408201424-a24fb2fb8a0f/go.mod h1:oPkhp1MJrh7nUepCBc 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= golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.10.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.13.0 h1:Af8nKPmuFypiUBjVoU9V20FiaFXOcuZI21p0ycVYYGE= golang.org/x/sys v0.13.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= From 4ead788840c32669477efcdc73b1b6abf01f2cbb Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Mon, 9 Oct 2023 14:00:30 -0400 Subject: [PATCH 42/85] Reenable TestFhirIO.* on GHA (#28435) * Reenable TestFhirIO.* on GHA * Propogate last error * temporarily only look at failing test and better error * temporarily only look at failing test and better error * temporarily only look at failing test and better error * Restore integration.go --- sdks/go/test/integration/integration.go | 6 ------ .../test/integration/io/fhirio/fhirio_test.go | 19 ++++++++++++++----- 2 files changed, 14 insertions(+), 11 deletions(-) diff --git a/sdks/go/test/integration/integration.go b/sdks/go/test/integration/integration.go index f3cffd176110..d3eb1c3ee7c3 100644 --- a/sdks/go/test/integration/integration.go +++ b/sdks/go/test/integration/integration.go @@ -304,12 +304,6 @@ func CheckFilters(t *testing.T) { panic("ptest.Main() has not been called: please override TestMain to ensure that the integration test runs properly.") } - // TODO(https://github.com/apache/beam/issues/28227): Grant github-actions service account permission to healthcare.fhirStores.create. - var user = os.Getenv("USER") - if user == "github-actions" { - dataflowFilters = append(dataflowFilters, "TestFhirIO.*") - } - // Check for sickbaying first. n := t.Name() for _, f := range sickbay { diff --git a/sdks/go/test/integration/io/fhirio/fhirio_test.go b/sdks/go/test/integration/io/fhirio/fhirio_test.go index 03e3654d5c49..01f6db9324f3 100644 --- a/sdks/go/test/integration/io/fhirio/fhirio_test.go +++ b/sdks/go/test/integration/io/fhirio/fhirio_test.go @@ -96,9 +96,9 @@ func setupFhirStore(t *testing.T, shouldPopulateStore bool) (fhirStoreInfo, func var resourcePaths [][]byte if shouldPopulateStore { - resourcePaths = populateStore(createdFhirStorePath) - if len(resourcePaths) == 0 { - t.Fatal("No data got populated to test") + resourcePaths, err = populateStore(createdFhirStorePath) + if err != nil { + t.Fatal(err) } } @@ -127,11 +127,13 @@ func deleteStore(storePath string) (*healthcare.Empty, error) { // Populates fhir store with data. Note that failure to populate some data is not // detrimental to the tests, so it is fine to ignore. -func populateStore(storePath string) [][]byte { +func populateStore(storePath string) ([][]byte, error) { resourcePaths := make([][]byte, 0) + bufferedErrors := make([]string, 0) for _, bundle := range readPrettyBundles() { response, err := storeService.ExecuteBundle(storePath, strings.NewReader(bundle)).Do() if err != nil { + bufferedErrors = append(bufferedErrors, err.Error()) continue } @@ -145,23 +147,30 @@ func populateStore(storePath string) [][]byte { } err = json.NewDecoder(response.Body).Decode(&body) if err != nil { + bufferedErrors = append(bufferedErrors, err.Error()) continue } for _, entry := range body.Entry { bundleFailedToBeCreated := !strings.Contains(entry.Response.Status, "201") if bundleFailedToBeCreated { + bufferedErrors = append(bufferedErrors, fmt.Sprintf("Bundle creation failed with: %v", entry.Response)) continue } resourcePath, err := extractResourcePathFrom(entry.Response.Location) if err != nil { + bufferedErrors = append(bufferedErrors, err.Error()) continue } resourcePaths = append(resourcePaths, resourcePath) } } - return resourcePaths + if len(resourcePaths) == 0 { + return nil, fmt.Errorf("failed to populate fhir store with any data. Errors with requests: %s", bufferedErrors) + } + + return resourcePaths, nil } func readPrettyBundles() []string { From 21f822f6580f04c3bd581005d93846e555982367 Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Mon, 9 Oct 2023 14:14:07 -0400 Subject: [PATCH 43/85] Catch EOFError along with StopIteration (#28896) * catch EOF instead of StopIteration * add stop iteration to close * catch both stop iteration and EOF --- sdks/python/apache_beam/io/gcp/bigquery.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/gcp/bigquery.py b/sdks/python/apache_beam/io/gcp/bigquery.py index 986919fd6b82..184138af7525 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery.py +++ b/sdks/python/apache_beam/io/gcp/bigquery.py @@ -1310,7 +1310,7 @@ def __iter__(self): def __next__(self): try: return fastavro.schemaless_reader(self.bytes_reader, self.avro_schema) - except StopIteration: + except (StopIteration, EOFError): self.read_rows_response = next(self.read_rows_iterator, None) if self.read_rows_response is not None: self.bytes_reader = io.BytesIO( From f63e9ed28e4889c2645bfaf5a3450c10a205cf62 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 9 Oct 2023 11:19:21 -0700 Subject: [PATCH 44/85] Additional context for decoding errors. (#27024) --- .../apache_beam/runners/worker/bundle_processor.py | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py index 935ba83709c0..c7fcb9587455 100644 --- a/sdks/python/apache_beam/runners/worker/bundle_processor.py +++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py @@ -227,8 +227,13 @@ def process_encoded(self, encoded_windowed_values): if self.index == self.stop - 1: return self.index += 1 - decoded_value = self.windowed_coder_impl.decode_from_stream( - input_stream, True) + try: + decoded_value = self.windowed_coder_impl.decode_from_stream( + input_stream, True) + except Exception as exn: + raise ValueError( + "Error decoding input stream with coder " + + self.windowed_coder) from exn self.output(decoded_value) def monitoring_infos(self, transform_id, tag_to_pcollection_id): From dafe928a15b5ef5a865f5889eb98848770b462a6 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 9 Oct 2023 11:20:34 -0700 Subject: [PATCH 45/85] Add Java variants of WriteTo{Csv,Json}. (#28380) --- .../schemaio-expansion-service/build.gradle | 6 + .../providers/CsvWriteTransformProvider.java | 145 +++++++++ .../sdk/io/csv/providers/package-info.java | 20 ++ sdks/java/io/json/build.gradle | 35 +++ .../org/apache/beam/sdk/io/json/JsonIO.java | 283 ++++++++++++++++++ .../apache/beam/sdk/io/json/package-info.java | 20 ++ .../providers/JsonWriteTransformProvider.java | 142 +++++++++ .../sdk/io/json/providers/package-info.java | 20 ++ .../beam/sdk/io/json/JsonIOWriteTest.java | 145 +++++++++ sdks/python/apache_beam/yaml/standard_io.yaml | 7 + settings.gradle.kts | 1 + 11 files changed, 824 insertions(+) create mode 100644 sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java create mode 100644 sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/package-info.java create mode 100644 sdks/java/io/json/build.gradle create mode 100644 sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/JsonIO.java create mode 100644 sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/package-info.java create mode 100644 sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/JsonWriteTransformProvider.java create mode 100644 sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/package-info.java create mode 100644 sdks/java/io/json/src/test/java/org/apache/beam/sdk/io/json/JsonIOWriteTest.java diff --git a/sdks/java/extensions/schemaio-expansion-service/build.gradle b/sdks/java/extensions/schemaio-expansion-service/build.gradle index d23330d73c22..246c0c155cbd 100644 --- a/sdks/java/extensions/schemaio-expansion-service/build.gradle +++ b/sdks/java/extensions/schemaio-expansion-service/build.gradle @@ -32,8 +32,14 @@ applyJavaNature( dependencies { implementation project(path: ":sdks:java:expansion-service") permitUnusedDeclared project(path: ":sdks:java:expansion-service") // BEAM-11761 + implementation project(":sdks:java:extensions:google-cloud-platform-core") + permitUnusedDeclared project(path: ":sdks:java:extensions:google-cloud-platform-core") // BEAM-11761 + implementation project(":sdks:java:io:csv") + permitUnusedDeclared project(path: ":sdks:java:io:csv") // BEAM-11761 implementation project(":sdks:java:io:jdbc") permitUnusedDeclared project(":sdks:java:io:jdbc") // BEAM-11761 + implementation project(":sdks:java:io:json") + permitUnusedDeclared project(path: ":sdks:java:io:json") // BEAM-11761 implementation library.java.postgres permitUnusedDeclared library.java.postgres // BEAM-11761 implementation project(path: ":model:pipeline", configuration: "shadow") diff --git a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java new file mode 100644 index 000000000000..4e07a06197f5 --- /dev/null +++ b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/CsvWriteTransformProvider.java @@ -0,0 +1,145 @@ +/* + * 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.io.csv.providers; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.io.WriteFilesResult; +import org.apache.beam.sdk.io.csv.CsvIO; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; +import org.apache.commons.csv.CSVFormat; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for {@link CsvIO#write}. + * + *

Internal only: This class is actively being worked on, and it will likely change. We + * provide no backwards compatibility guarantees, and it should not be implemented outside the Beam + * repository. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@AutoService(SchemaTransformProvider.class) +public class CsvWriteTransformProvider + extends TypedSchemaTransformProvider { + private static final String INPUT_ROWS_TAG = "input"; + private static final String WRITE_RESULTS = "output"; + + @Override + protected Class configurationClass() { + return CsvWriteConfiguration.class; + } + + @Override + protected SchemaTransform from(CsvWriteConfiguration configuration) { + return new CsvWriteTransform(configuration); + } + + @Override + public String identifier() { + return String.format("beam:schematransform:org.apache.beam:csv_write:v1"); + } + + @Override + public List inputCollectionNames() { + return Collections.singletonList(INPUT_ROWS_TAG); + } + + @Override + public List outputCollectionNames() { + return Collections.singletonList(WRITE_RESULTS); + } + + /** Configuration for writing to BigQuery with Storage Write API. */ + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class CsvWriteConfiguration { + + public void validate() { + checkArgument( + !Strings.isNullOrEmpty(this.getPath()), "Path for a CSV Write must be specified."); + } + + public static Builder builder() { + return new AutoValue_CsvWriteTransformProvider_CsvWriteConfiguration.Builder(); + } + + @SchemaFieldDescription("The file path to write to.") + public abstract String getPath(); + + /** Builder for {@link CsvWriteConfiguration}. */ + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setPath(String path); + + /** Builds a {@link CsvWriteConfiguration} instance. */ + public abstract CsvWriteConfiguration build(); + } + } + + /** A {@link SchemaTransform} for {@link CsvIO#write}. */ + protected static class CsvWriteTransform extends SchemaTransform { + + private final CsvWriteConfiguration configuration; + + CsvWriteTransform(CsvWriteConfiguration configuration) { + configuration.validate(); + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + WriteFilesResult result = + input + .get(INPUT_ROWS_TAG) + .apply(CsvIO.writeRows(configuration.getPath(), CSVFormat.DEFAULT).withSuffix("")); + Schema outputSchema = Schema.of(Field.of("filename", FieldType.STRING)); + return PCollectionRowTuple.of( + WRITE_RESULTS, + result + .getPerDestinationOutputFilenames() + .apply( + "Collect filenames", + MapElements.into(TypeDescriptors.rows()) + .via( + (destinationAndRow) -> + Row.withSchema(outputSchema) + .withFieldValue("filename", destinationAndRow.getValue()) + .build())) + .setRowSchema(outputSchema)); + } + } +} diff --git a/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/package-info.java b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/package-info.java new file mode 100644 index 000000000000..646e69b7cb8c --- /dev/null +++ b/sdks/java/io/csv/src/main/java/org/apache/beam/sdk/io/csv/providers/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Transforms for reading and writing CSV files. */ +package org.apache.beam.sdk.io.csv.providers; diff --git a/sdks/java/io/json/build.gradle b/sdks/java/io/json/build.gradle new file mode 100644 index 000000000000..fe1f607a3696 --- /dev/null +++ b/sdks/java/io/json/build.gradle @@ -0,0 +1,35 @@ +/* + * 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.io.json' +) + +description = "Apache Beam :: SDKs :: Java :: IO :: JSON" +ext.summary = "IO to read and write JSON files." + +dependencies { + implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation library.java.vendored_guava_32_1_2_jre + implementation library.java.everit_json_schema + testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") + testImplementation library.java.junit + testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") +} \ No newline at end of file diff --git a/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/JsonIO.java b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/JsonIO.java new file mode 100644 index 000000000000..3abb29a80427 --- /dev/null +++ b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/JsonIO.java @@ -0,0 +1,283 @@ +/* + * 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.io.json; + +import static org.apache.beam.sdk.values.TypeDescriptors.rows; +import static org.apache.beam.sdk.values.TypeDescriptors.strings; + +import com.google.auto.value.AutoValue; +import org.apache.beam.sdk.coders.RowCoder; +import org.apache.beam.sdk.io.Compression; +import org.apache.beam.sdk.io.FileBasedSink; +import org.apache.beam.sdk.io.FileIO; +import org.apache.beam.sdk.io.ShardNameTemplate; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.WriteFiles; +import org.apache.beam.sdk.io.WriteFilesResult; +import org.apache.beam.sdk.io.fs.ResourceId; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.utils.JsonUtils; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.Row; + +/** + * {@link PTransform}s for reading and writing JSON files. + * + *

Reading JSON files

+ * + *

Reading from JSON files is not yet implemented in Java. Please see https://github.com/apache/beam/issues/24552. + * + *

Writing JSON files

+ * + *

To write a {@link PCollection} to one or more line-delimited JSON files, use {@link + * JsonIO.Write}, using{@link JsonIO#writeRows} or {@link JsonIO#write}. {@link JsonIO.Write} + * supports writing {@link Row} or custom Java types using an inferred {@link Schema}. Examples + * below show both scenarios. See the Beam Programming Guide on inferring + * schemas for more information on how to enable Beam to infer a {@link Schema} from a custom + * Java type. + * + *

Example usage:

+ * + *

Suppose we have the following Transaction class annotated with + * {@code @DefaultSchema(JavaBeanSchema.class)} so that Beam can infer its {@link Schema}: + * + *

{@code @DefaultSchema(JavaBeanSchema.class)
+ * public class Transaction {
+ *   public Transaction() { … }
+ *   public Long getTransactionId();
+ *   public void setTransactionId(Long transactionId) { … }
+ *   public String getBank() { … }
+ *   public void setBank(String bank) { … }
+ *   public double getPurchaseAmount() { … }
+ *   public void setPurchaseAmount(double purchaseAmount) { … }
+ * }
+ * }
+ * + *

From a {@code PCollection}, {@link JsonIO.Write} can write one or many JSON + * files. + * + *

{@code
+ * PCollection transactions = ...
+ * transactions.apply(JsonIO.write("path/to/folder/prefix"));
+ * }
+ * + *

The resulting JSON files will look like the following where the header is repeated for every + * file, whereas by default, {@link JsonIO.Write} will write all fields in sorted order of + * the field names. + * + *

{@code
+ * {"bank": "A", "purchaseAmount": 10.23, "transactionId": 12345}
+ * {"bank": "B", "purchaseAmount": 54.65, "transactionId": 54321}
+ * {"bank": "C", "purchaseAmount": 11,76, "transactionId": 98765}
+ * }
+ * + *

A {@link PCollection} of {@link Row}s works just like custom Java types illustrated above, + * except we use {@link JsonIO#writeRows} as shown below for the same {@code Transaction} class. We + * derive {@code Transaction}'s {@link Schema} using a {@link + * org.apache.beam.sdk.schemas.annotations.DefaultSchema.DefaultSchemaProvider}. Note that + * hard-coding the {@link Row}s below is for illustration purposes. Developers are instead + * encouraged to take advantage of {@link + * org.apache.beam.sdk.schemas.annotations.DefaultSchema.DefaultSchemaProvider#toRowFunction}. + * + *

{@code
+ * DefaultSchemaProvider defaultSchemaProvider = new DefaultSchemaProvider();
+ * Schema schema = defaultSchemaProvider.schemaFor(TypeDescriptor.of(Transaction.class));
+ * PCollection transactions = pipeline.apply(Create.of(
+ *  Row
+ *    .withSchema(schema)
+ *    .withFieldValue("bank", "A")
+ *    .withFieldValue("purchaseAmount", 10.23)
+ *    .withFieldValue("transactionId", "12345")
+ *    .build(),
+ *  Row
+ *    .withSchema(schema)
+ *    .withFieldValue("bank", "B")
+ *    .withFieldValue("purchaseAmount", 54.65)
+ *    .withFieldValue("transactionId", "54321")
+ *    .build(),
+ *  Row
+ *    .withSchema(schema)
+ *    .withFieldValue("bank", "C")
+ *    .withFieldValue("purchaseAmount", 11.76)
+ *    .withFieldValue("transactionId", "98765")
+ *    .build()
+ * );
+ *
+ * transactions.apply(
+ *  JsonIO
+ *    .writeRowsTo("gs://bucket/path/to/folder/prefix")
+ * );
+ * }
+ * + *

Writing the transactions {@link PCollection} of {@link Row}s would yield the following JSON + * file content. + * + *

{@code
+ * {"bank": "A", "purchaseAmount": 10.23, "transactionId": 12345}
+ * {"bank": "B", "purchaseAmount": 54.65, "transactionId": 54321}
+ * {"bank": "C", "purchaseAmount": 11,76, "transactionId": 98765}
+ * }
+ */ +public class JsonIO { + static final String DEFAULT_FILENAME_SUFFIX = ".json"; + + /** Instantiates a {@link Write} for writing user types in {@link JSONFormat} format. */ + public static Write write(String to) { + return new AutoValue_JsonIO_Write.Builder() + .setTextIOWrite(createDefaultTextIOWrite(to)) + .build(); + } + + /** Instantiates a {@link Write} for writing {@link Row}s in {@link JSONFormat} format. */ + public static Write writeRows(String to) { + return new AutoValue_JsonIO_Write.Builder() + .setTextIOWrite(createDefaultTextIOWrite(to)) + .build(); + } + + /** {@link PTransform} for writing JSON files. */ + @AutoValue + public abstract static class Write + extends PTransform, WriteFilesResult> { + + /** Specifies the {@link Compression} of all generated shard files. */ + public Write withCompression(Compression compression) { + return toBuilder().setTextIOWrite(getTextIOWrite().withCompression(compression)).build(); + } + + /** Whether to skip the spilling of data. See {@link WriteFiles#withNoSpilling}. */ + public Write withNoSpilling() { + return toBuilder().setTextIOWrite(getTextIOWrite().withNoSpilling()).build(); + } + + /** + * Specifies to use a given fixed number of shards per window. See {@link + * TextIO.Write#withNumShards}. + */ + public Write withNumShards(Integer numShards) { + return toBuilder().setTextIOWrite(getTextIOWrite().withNumShards(numShards)).build(); + } + + /** + * Forces a single file as output and empty shard name template. See {@link + * TextIO.Write#withoutSharding}. + */ + public Write withoutSharding() { + return toBuilder().setTextIOWrite(getTextIOWrite().withoutSharding()).build(); + } + + /** + * Uses the given {@link ShardNameTemplate} for naming output files. See {@link + * TextIO.Write#withShardNameTemplate}. + */ + public Write withShardTemplate(String shardTemplate) { + return toBuilder() + .setTextIOWrite(getTextIOWrite().withShardNameTemplate(shardTemplate)) + .build(); + } + + /** Configures the filename suffix for written files. See {@link TextIO.Write#withSuffix}. */ + public Write withSuffix(String suffix) { + return toBuilder().setTextIOWrite(getTextIOWrite().withSuffix(suffix)).build(); + } + + /** + * Set the base directory used to generate temporary files. See {@link + * TextIO.Write#withTempDirectory}. + */ + public Write withTempDirectory(ResourceId tempDirectory) { + return toBuilder().setTextIOWrite(getTextIOWrite().withTempDirectory(tempDirectory)).build(); + } + + /** + * Preserves windowing of input elements and writes them to files based on the element's window. + * See {@link TextIO.Write#withWindowedWrites}. + */ + public Write withWindowedWrites() { + return toBuilder().setTextIOWrite(getTextIOWrite().withWindowedWrites()).build(); + } + + /** + * Returns a transform for writing to text files like this one but that has the given {@link + * FileBasedSink.WritableByteChannelFactory} to be used by the {@link FileBasedSink} during + * output. See {@link TextIO.Write#withWritableByteChannelFactory}. + */ + public Write withWritableByteChannelFactory( + FileBasedSink.WritableByteChannelFactory writableByteChannelFactory) { + return toBuilder() + .setTextIOWrite( + getTextIOWrite().withWritableByteChannelFactory(writableByteChannelFactory)) + .build(); + } + + /** The underlying {@link FileIO.Write} that writes converted input to JSON formatted output. */ + abstract TextIO.Write getTextIOWrite(); + + abstract Builder toBuilder(); + + @AutoValue.Builder + abstract static class Builder { + + /** + * The underlying {@link FileIO.Write} that writes converted input to JSON formatted output. + */ + abstract Builder setTextIOWrite(TextIO.Write value); + + abstract Write autoBuild(); + + final Write build() { + return autoBuild(); + } + } + + @Override + public WriteFilesResult expand(PCollection input) { + if (!input.hasSchema()) { + throw new IllegalArgumentException( + String.format( + "%s requires an input Schema. Note that only Row or user classes are supported. Consider using TextIO or FileIO directly when writing primitive types", + Write.class.getName())); + } + + Schema schema = input.getSchema(); + + RowCoder rowCoder = RowCoder.of(schema); + + PCollection rows = + input + .apply("To Rows", MapElements.into(rows()).via(input.getToRowFunction())) + .setCoder(rowCoder); + + SerializableFunction toJsonFn = + JsonUtils.getRowToJsonStringsFunction(input.getSchema()); + + PCollection json = rows.apply("To JSON", MapElements.into(strings()).via(toJsonFn)); + + return json.apply("Write JSON", getTextIOWrite().withOutputFilenames()); + } + } + + private static TextIO.Write createDefaultTextIOWrite(String to) { + return TextIO.write().to(to).withSuffix(DEFAULT_FILENAME_SUFFIX); + } +} diff --git a/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/package-info.java b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/package-info.java new file mode 100644 index 000000000000..1ee191835713 --- /dev/null +++ b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Transforms for reading and writing JSON files. */ +package org.apache.beam.sdk.io.json; diff --git a/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/JsonWriteTransformProvider.java b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/JsonWriteTransformProvider.java new file mode 100644 index 000000000000..9e030821e5ca --- /dev/null +++ b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/JsonWriteTransformProvider.java @@ -0,0 +1,142 @@ +/* + * 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.io.json.providers; + +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import com.google.auto.service.AutoService; +import com.google.auto.value.AutoValue; +import java.util.Collections; +import java.util.List; +import org.apache.beam.sdk.io.WriteFilesResult; +import org.apache.beam.sdk.io.json.JsonIO; +import org.apache.beam.sdk.schemas.AutoValueSchema; +import org.apache.beam.sdk.schemas.Schema; +import org.apache.beam.sdk.schemas.Schema.Field; +import org.apache.beam.sdk.schemas.Schema.FieldType; +import org.apache.beam.sdk.schemas.annotations.DefaultSchema; +import org.apache.beam.sdk.schemas.annotations.SchemaFieldDescription; +import org.apache.beam.sdk.schemas.transforms.SchemaTransform; +import org.apache.beam.sdk.schemas.transforms.SchemaTransformProvider; +import org.apache.beam.sdk.schemas.transforms.TypedSchemaTransformProvider; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.PCollectionRowTuple; +import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; + +/** + * An implementation of {@link TypedSchemaTransformProvider} for {@link JsonIO#write}. + * + *

Internal only: This class is actively being worked on, and it will likely change. We + * provide no backwards compatibility guarantees, and it should not be implemented outside the Beam + * repository. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@AutoService(SchemaTransformProvider.class) +public class JsonWriteTransformProvider + extends TypedSchemaTransformProvider { + private static final String INPUT_ROWS_TAG = "input"; + private static final String WRITE_RESULTS = "output"; + + @Override + protected Class configurationClass() { + return JsonWriteConfiguration.class; + } + + @Override + protected SchemaTransform from(JsonWriteConfiguration configuration) { + return new JsonWriteTransform(configuration); + } + + @Override + public String identifier() { + return String.format("beam:schematransform:org.apache.beam:json_write:v1"); + } + + @Override + public List inputCollectionNames() { + return Collections.singletonList(INPUT_ROWS_TAG); + } + + @Override + public List outputCollectionNames() { + return Collections.singletonList(WRITE_RESULTS); + } + + /** Configuration for writing to BigQuery with Storage Write API. */ + @DefaultSchema(AutoValueSchema.class) + @AutoValue + public abstract static class JsonWriteConfiguration { + + public void validate() { + checkArgument( + !Strings.isNullOrEmpty(this.getPath()), "Path for a JSON Write must be specified."); + } + + public static Builder builder() { + return new AutoValue_JsonWriteTransformProvider_JsonWriteConfiguration.Builder(); + } + + @SchemaFieldDescription("The file path to write to.") + public abstract String getPath(); + + /** Builder for {@link JsonWriteConfiguration}. */ + @AutoValue.Builder + public abstract static class Builder { + + public abstract Builder setPath(String path); + + /** Builds a {@link JsonWriteConfiguration} instance. */ + public abstract JsonWriteConfiguration build(); + } + } + + /** A {@link SchemaTransform} for {@link JsonIO#write}. */ + protected static class JsonWriteTransform extends SchemaTransform { + + private final JsonWriteConfiguration configuration; + + JsonWriteTransform(JsonWriteConfiguration configuration) { + configuration.validate(); + this.configuration = configuration; + } + + @Override + public PCollectionRowTuple expand(PCollectionRowTuple input) { + WriteFilesResult result = + input.get(INPUT_ROWS_TAG).apply(JsonIO.writeRows(configuration.getPath()).withSuffix("")); + Schema outputSchema = Schema.of(Field.of("filename", FieldType.STRING)); + return PCollectionRowTuple.of( + WRITE_RESULTS, + result + .getPerDestinationOutputFilenames() + .apply( + "Collect filenames", + MapElements.into(TypeDescriptors.rows()) + .via( + (destinationAndRow) -> + Row.withSchema(outputSchema) + .withFieldValue("filename", destinationAndRow.getValue()) + .build())) + .setRowSchema(outputSchema)); + } + } +} diff --git a/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/package-info.java b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/package-info.java new file mode 100644 index 000000000000..312454f8733b --- /dev/null +++ b/sdks/java/io/json/src/main/java/org/apache/beam/sdk/io/json/providers/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Transforms for reading and writing JSON files. */ +package org.apache.beam.sdk.io.json.providers; diff --git a/sdks/java/io/json/src/test/java/org/apache/beam/sdk/io/json/JsonIOWriteTest.java b/sdks/java/io/json/src/test/java/org/apache/beam/sdk/io/json/JsonIOWriteTest.java new file mode 100644 index 000000000000..71fdcd6b3d94 --- /dev/null +++ b/sdks/java/io/json/src/test/java/org/apache/beam/sdk/io/json/JsonIOWriteTest.java @@ -0,0 +1,145 @@ +/* + * 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.io.json; + +import static org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.allPrimitiveDataTypes; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; + +import java.io.File; +import java.io.IOException; +import java.math.BigDecimal; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.common.SchemaAwareJavaBeans.AllPrimitiveDataTypes; +import org.apache.beam.sdk.testing.PAssert; +import org.apache.beam.sdk.testing.SerializableMatcher; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.values.PCollection; +import org.hamcrest.BaseMatcher; +import org.hamcrest.Description; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link JsonIO.Write}. */ +@RunWith(JUnit4.class) +public class JsonIOWriteTest { + @Rule public TestPipeline writePipeline = TestPipeline.create(); + + @Rule public TestPipeline readPipeline = TestPipeline.create(); + + @Rule + public TestPipeline errorPipeline = TestPipeline.create().enableAbandonedNodeEnforcement(false); + + @Rule public TemporaryFolder tempFolder = new TemporaryFolder(); + + @Test + public void writesUserDefinedTypes() { + File folder = + createFolder(AllPrimitiveDataTypes.class.getSimpleName(), "writesUserDefinedTypes"); + + PCollection input = + writePipeline.apply( + Create.of( + allPrimitiveDataTypes(false, BigDecimal.TEN, 1.0, 1.0f, 1, 1L, "a"), + allPrimitiveDataTypes( + false, BigDecimal.TEN.add(BigDecimal.TEN), 2.0, 2.0f, 2, 2L, "b"), + allPrimitiveDataTypes( + false, + BigDecimal.TEN.add(BigDecimal.TEN).add(BigDecimal.TEN), + 3.0, + 3.0f, + 3, + 3L, + "c"))); + + input.apply(JsonIO.write(toFilenamePrefix(folder)).withNumShards(1)); + + writePipeline.run().waitUntilFinish(); + + PAssert.that(readPipeline.apply(TextIO.read().from(toFilenamePrefix(folder) + "*"))) + .containsInAnyOrder( + containsAll( + "\"aDouble\":1.0", + "\"aFloat\":1.0", + "\"aLong\":1", + "\"aString\":\"a\"", + "\"anInteger\":1", + "\"aDecimal\":10", + "\"aBoolean\":false"), + containsAll( + "\"aDouble\":2.0", + "\"aFloat\":2.0", + "\"aLong\":2", + "\"aString\":\"b\"", + "\"anInteger\":2", + "\"aDecimal\":20", + "\"aBoolean\":false"), + containsAll( + "\"aDouble\":3.0", + "\"aFloat\":3.0", + "\"aLong\":3", + "\"aString\":\"c\"", + "\"anInteger\":3", + "\"aDecimal\":30", + "\"aBoolean\":false")); + + readPipeline.run(); + } + + private static SerializableMatcher containsAll(String... needles) { + class Matcher extends BaseMatcher implements SerializableMatcher { + @Override + public boolean matches(Object item) { + if (!(item instanceof String)) { + return false; + } + + String haystack = (String) item; + for (String needle : needles) { + if (!haystack.contains(needle)) { + return false; + } + } + return true; + } + + @Override + public void describeTo(Description description) { + description.appendText("Contains all of: "); + description.appendValueList("[", ",", "]", needles); + } + } + return new Matcher(); + } + + private static String toFilenamePrefix(File folder) { + checkArgument(folder.isDirectory()); + return folder.getAbsolutePath() + "/out"; + } + + private File createFolder(String... paths) { + try { + return tempFolder.newFolder(paths); + } catch (IOException e) { + throw new IllegalStateException(e); + } + } +} diff --git a/sdks/python/apache_beam/yaml/standard_io.yaml b/sdks/python/apache_beam/yaml/standard_io.yaml index 8a9e0c100496..c4748483b04b 100644 --- a/sdks/python/apache_beam/yaml/standard_io.yaml +++ b/sdks/python/apache_beam/yaml/standard_io.yaml @@ -105,3 +105,10 @@ 'WriteToParquet': 'apache_beam.io.WriteToParquet' 'ReadFromAvro': 'apache_beam.io.ReadFromAvro' 'WriteToAvro': 'apache_beam.io.WriteToAvro' + +- type: beamJar + transforms: + 'WriteToCsv': 'beam:schematransform:org.apache.beam:csv_write:v1' + 'WriteToJson': 'beam:schematransform:org.apache.beam:json_write:v1' + config: + gradle_target: 'sdks:java:extensions:schemaio-expansion-service:shadowJar' diff --git a/settings.gradle.kts b/settings.gradle.kts index f4901d7df92b..c370c5da27d1 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -244,6 +244,7 @@ include(":sdks:java:io:hbase") include(":sdks:java:io:hcatalog") include(":sdks:java:io:jdbc") include(":sdks:java:io:jms") +include(":sdks:java:io:json") include(":sdks:java:io:kafka") include(":sdks:java:io:kinesis") include(":sdks:java:io:kinesis:expansion-service") From 2bbb3485c78ede3c4acddd462158814157f2b46f Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 9 Oct 2023 11:21:07 -0700 Subject: [PATCH 46/85] Populate top-level display data in yaml main. (#28512) --- sdks/python/apache_beam/pipeline.py | 28 +++++++++++++++---- sdks/python/apache_beam/transforms/display.py | 23 +++++++++------ sdks/python/apache_beam/yaml/main.py | 17 ++++++----- 3 files changed, 47 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 042b483d50f1..14177cd603d8 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -88,6 +88,7 @@ from apache_beam.transforms import ParDo from apache_beam.transforms import ptransform from apache_beam.transforms.display import DisplayData +from apache_beam.transforms.display import HasDisplayData from apache_beam.transforms.resources import merge_resource_hints from apache_beam.transforms.resources import resource_hints_from_options from apache_beam.transforms.sideinputs import get_sideinput_index @@ -108,7 +109,7 @@ __all__ = ['Pipeline', 'PTransformOverride'] -class Pipeline(object): +class Pipeline(HasDisplayData): """A pipeline object that manages a DAG of :class:`~apache_beam.pvalue.PValue` s and their :class:`~apache_beam.transforms.ptransform.PTransform` s. @@ -133,9 +134,12 @@ def runner_implemented_transforms(cls): common_urns.primitives.IMPULSE.urn, ]) - def __init__(self, runner=None, options=None, argv=None): - # type: (Optional[Union[str, PipelineRunner]], Optional[PipelineOptions], Optional[List[str]]) -> None - + def __init__( + self, + runner: Optional[Union[str, PipelineRunner]] = None, + options: Optional[PipelineOptions] = None, + argv: Optional[List[str]] = None, + display_data: Optional[Dict[str, Any]] = None): """Initialize a pipeline object. Args: @@ -151,6 +155,8 @@ def __init__(self, runner=None, options=None, argv=None): to be used for building a :class:`~apache_beam.options.pipeline_options.PipelineOptions` object. This will only be used if argument **options** is :data:`None`. + display_data (Dict[str: Any]): a dictionary of static data associated + with this pipeline that can be displayed when it runs. Raises: ValueError: if either the runner or options argument is not @@ -233,6 +239,11 @@ def __init__(self, runner=None, options=None, argv=None): # Records whether this pipeline contains any external transforms. self.contains_external_transforms = False + self._display_data = display_data or {} + + def display_data(self): + # type: () -> Dict[str, Any] + return self._display_data @property # type: ignore[misc] # decorated property not supported def options(self): @@ -914,7 +925,8 @@ def visit_transform(self, transform_node): proto = beam_runner_api_pb2.Pipeline( root_transform_ids=[root_transform_id], components=context.to_runner_api(), - requirements=context.requirements()) + requirements=context.requirements(), + display_data=DisplayData('', self._display_data).to_proto()) proto.components.transforms[root_transform_id].unique_name = ( root_transform_id) self.merge_compatible_environments(proto) @@ -970,7 +982,11 @@ def from_runner_api( # type: (...) -> Pipeline """For internal use only; no backwards-compatibility guarantees.""" - p = Pipeline(runner=runner, options=options) + p = Pipeline( + runner=runner, + options=options, + display_data={str(ix): d + for ix, d in enumerate(proto.display_data)}) from apache_beam.runners import pipeline_context context = pipeline_context.PipelineContext( proto.components, requirements=proto.requirements) diff --git a/sdks/python/apache_beam/transforms/display.py b/sdks/python/apache_beam/transforms/display.py index b52a8fd5b6dd..0d1dd552413e 100644 --- a/sdks/python/apache_beam/transforms/display.py +++ b/sdks/python/apache_beam/transforms/display.py @@ -45,6 +45,7 @@ from datetime import timedelta from typing import TYPE_CHECKING from typing import List +from typing import Union from apache_beam.portability import common_urns from apache_beam.portability.api import beam_runner_api_pb2 @@ -101,7 +102,8 @@ def __init__( ): # type: (...) -> None self.namespace = namespace - self.items = [] # type: List[DisplayDataItem] + self.items = [ + ] # type: List[Union[DisplayDataItem, beam_runner_api_pb2.DisplayData]] self._populate_items(display_data_dict) def _populate_items(self, display_data_dict): @@ -112,26 +114,31 @@ def _populate_items(self, display_data_dict): subcomponent_display_data = DisplayData( element._get_display_data_namespace(), element.display_data()) self.items += subcomponent_display_data.items - continue - if isinstance(element, DisplayDataItem): + elif isinstance(element, DisplayDataItem): if element.should_drop(): continue element.key = key element.namespace = self.namespace self.items.append(element) - continue - # If it's not a HasDisplayData element, - # nor a dictionary, then it's a simple value - self.items.append( - DisplayDataItem(element, namespace=self.namespace, key=key)) + elif isinstance(element, beam_runner_api_pb2.DisplayData): + self.items.append(element) + + else: + # If it's not a HasDisplayData element, + # nor a dictionary, then it's a simple value + self.items.append( + DisplayDataItem(element, namespace=self.namespace, key=key)) def to_proto(self): # type: (...) -> List[beam_runner_api_pb2.DisplayData] """Returns a List of Beam proto representation of Display data.""" def create_payload(dd): + if isinstance(dd, beam_runner_api_pb2.DisplayData): + return dd + display_data_dict = None try: display_data_dict = dd.get_dict() diff --git a/sdks/python/apache_beam/yaml/main.py b/sdks/python/apache_beam/yaml/main.py index eb0695f337b4..e2ec8df9cfc3 100644 --- a/sdks/python/apache_beam/yaml/main.py +++ b/sdks/python/apache_beam/yaml/main.py @@ -51,19 +51,22 @@ def _pipeline_spec_from_args(known_args): raise ValueError( "Exactly one of pipeline_spec or pipeline_spec_file must be set.") - return yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) + return pipeline_yaml def run(argv=None): yaml_transform._LOGGER.setLevel('INFO') known_args, pipeline_args = _configure_parser(argv) - pipeline_spec = _pipeline_spec_from_args(known_args) + pipeline_yaml = _pipeline_spec_from_args(known_args) + pipeline_spec = yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) - with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( - pipeline_args, - pickle_library='cloudpickle', - **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get( - 'options', {})))) as p: + with beam.Pipeline( # linebreak for better yapf formatting + options=beam.options.pipeline_options.PipelineOptions( + pipeline_args, + pickle_library='cloudpickle', + **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get( + 'options', {}))), + display_data={'yaml': pipeline_yaml}) as p: print("Building pipeline...") yaml_transform.expand_pipeline(p, pipeline_spec) print("Running pipeline...") From 3844972d681a317c1281685edc5f6fc8783c8043 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Mon, 9 Oct 2023 16:15:10 -0700 Subject: [PATCH 47/85] Revert "Populate top-level display data in yaml main. (#28512)" This reverts commit 2bbb3485c78ede3c4acddd462158814157f2b46f. --- sdks/python/apache_beam/pipeline.py | 28 ++++--------------- sdks/python/apache_beam/transforms/display.py | 23 ++++++--------- sdks/python/apache_beam/yaml/main.py | 17 +++++------ 3 files changed, 21 insertions(+), 47 deletions(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 14177cd603d8..042b483d50f1 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -88,7 +88,6 @@ from apache_beam.transforms import ParDo from apache_beam.transforms import ptransform from apache_beam.transforms.display import DisplayData -from apache_beam.transforms.display import HasDisplayData from apache_beam.transforms.resources import merge_resource_hints from apache_beam.transforms.resources import resource_hints_from_options from apache_beam.transforms.sideinputs import get_sideinput_index @@ -109,7 +108,7 @@ __all__ = ['Pipeline', 'PTransformOverride'] -class Pipeline(HasDisplayData): +class Pipeline(object): """A pipeline object that manages a DAG of :class:`~apache_beam.pvalue.PValue` s and their :class:`~apache_beam.transforms.ptransform.PTransform` s. @@ -134,12 +133,9 @@ def runner_implemented_transforms(cls): common_urns.primitives.IMPULSE.urn, ]) - def __init__( - self, - runner: Optional[Union[str, PipelineRunner]] = None, - options: Optional[PipelineOptions] = None, - argv: Optional[List[str]] = None, - display_data: Optional[Dict[str, Any]] = None): + def __init__(self, runner=None, options=None, argv=None): + # type: (Optional[Union[str, PipelineRunner]], Optional[PipelineOptions], Optional[List[str]]) -> None + """Initialize a pipeline object. Args: @@ -155,8 +151,6 @@ def __init__( to be used for building a :class:`~apache_beam.options.pipeline_options.PipelineOptions` object. This will only be used if argument **options** is :data:`None`. - display_data (Dict[str: Any]): a dictionary of static data associated - with this pipeline that can be displayed when it runs. Raises: ValueError: if either the runner or options argument is not @@ -239,11 +233,6 @@ def __init__( # Records whether this pipeline contains any external transforms. self.contains_external_transforms = False - self._display_data = display_data or {} - - def display_data(self): - # type: () -> Dict[str, Any] - return self._display_data @property # type: ignore[misc] # decorated property not supported def options(self): @@ -925,8 +914,7 @@ def visit_transform(self, transform_node): proto = beam_runner_api_pb2.Pipeline( root_transform_ids=[root_transform_id], components=context.to_runner_api(), - requirements=context.requirements(), - display_data=DisplayData('', self._display_data).to_proto()) + requirements=context.requirements()) proto.components.transforms[root_transform_id].unique_name = ( root_transform_id) self.merge_compatible_environments(proto) @@ -982,11 +970,7 @@ def from_runner_api( # type: (...) -> Pipeline """For internal use only; no backwards-compatibility guarantees.""" - p = Pipeline( - runner=runner, - options=options, - display_data={str(ix): d - for ix, d in enumerate(proto.display_data)}) + p = Pipeline(runner=runner, options=options) from apache_beam.runners import pipeline_context context = pipeline_context.PipelineContext( proto.components, requirements=proto.requirements) diff --git a/sdks/python/apache_beam/transforms/display.py b/sdks/python/apache_beam/transforms/display.py index 0d1dd552413e..b52a8fd5b6dd 100644 --- a/sdks/python/apache_beam/transforms/display.py +++ b/sdks/python/apache_beam/transforms/display.py @@ -45,7 +45,6 @@ from datetime import timedelta from typing import TYPE_CHECKING from typing import List -from typing import Union from apache_beam.portability import common_urns from apache_beam.portability.api import beam_runner_api_pb2 @@ -102,8 +101,7 @@ def __init__( ): # type: (...) -> None self.namespace = namespace - self.items = [ - ] # type: List[Union[DisplayDataItem, beam_runner_api_pb2.DisplayData]] + self.items = [] # type: List[DisplayDataItem] self._populate_items(display_data_dict) def _populate_items(self, display_data_dict): @@ -114,31 +112,26 @@ def _populate_items(self, display_data_dict): subcomponent_display_data = DisplayData( element._get_display_data_namespace(), element.display_data()) self.items += subcomponent_display_data.items + continue - elif isinstance(element, DisplayDataItem): + if isinstance(element, DisplayDataItem): if element.should_drop(): continue element.key = key element.namespace = self.namespace self.items.append(element) + continue - elif isinstance(element, beam_runner_api_pb2.DisplayData): - self.items.append(element) - - else: - # If it's not a HasDisplayData element, - # nor a dictionary, then it's a simple value - self.items.append( - DisplayDataItem(element, namespace=self.namespace, key=key)) + # If it's not a HasDisplayData element, + # nor a dictionary, then it's a simple value + self.items.append( + DisplayDataItem(element, namespace=self.namespace, key=key)) def to_proto(self): # type: (...) -> List[beam_runner_api_pb2.DisplayData] """Returns a List of Beam proto representation of Display data.""" def create_payload(dd): - if isinstance(dd, beam_runner_api_pb2.DisplayData): - return dd - display_data_dict = None try: display_data_dict = dd.get_dict() diff --git a/sdks/python/apache_beam/yaml/main.py b/sdks/python/apache_beam/yaml/main.py index e2ec8df9cfc3..eb0695f337b4 100644 --- a/sdks/python/apache_beam/yaml/main.py +++ b/sdks/python/apache_beam/yaml/main.py @@ -51,22 +51,19 @@ def _pipeline_spec_from_args(known_args): raise ValueError( "Exactly one of pipeline_spec or pipeline_spec_file must be set.") - return pipeline_yaml + return yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) def run(argv=None): yaml_transform._LOGGER.setLevel('INFO') known_args, pipeline_args = _configure_parser(argv) - pipeline_yaml = _pipeline_spec_from_args(known_args) - pipeline_spec = yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) + pipeline_spec = _pipeline_spec_from_args(known_args) - with beam.Pipeline( # linebreak for better yapf formatting - options=beam.options.pipeline_options.PipelineOptions( - pipeline_args, - pickle_library='cloudpickle', - **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get( - 'options', {}))), - display_data={'yaml': pipeline_yaml}) as p: + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pipeline_args, + pickle_library='cloudpickle', + **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get( + 'options', {})))) as p: print("Building pipeline...") yaml_transform.expand_pipeline(p, pipeline_spec) print("Running pipeline...") From 3d574b485abab1256b5597db2a94af70b858fa1c Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 10 Oct 2023 10:02:33 -0400 Subject: [PATCH 48/85] Fix sourceJar task dependencies (#28900) --- .../beam/gradle/BeamModulePlugin.groovy | 26 ++++++++++++++++++- runners/flink/flink_runner.gradle | 23 +++++++++++++--- .../maven-archetypes/examples/build.gradle | 7 +++++ .../gcp-bom-examples/build.gradle | 6 +++++ 4 files changed, 57 insertions(+), 5 deletions(-) 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 9f4ae8bcfb29..705bcb978e12 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2395,7 +2395,20 @@ class BeamModulePlugin implements Plugin { // TODO: Decide whether this should be inlined into the one project that relies on it // or be left here. - project.ext.applyAvroNature = { project.apply plugin: "com.commercehub.gradle.plugin.avro" } + project.ext.applyAvroNature = { + project.apply plugin: "com.commercehub.gradle.plugin.avro" + + // add dependency BeamModulePlugin defined custom tasks + // they are defined only when certain flags are provided (e.g. -Prelease; -Ppublishing, etc) + def sourcesJar = project.tasks.findByName('sourcesJar') + if (sourcesJar != null) { + sourcesJar.dependsOn project.tasks.getByName('generateAvroJava') + } + def testSourcesJar = project.tasks.findByName('testSourcesJar') + if (testSourcesJar != null) { + testSourcesJar.dependsOn project.tasks.getByName('generateTestAvroJava') + } + } project.ext.applyAntlrNature = { project.apply plugin: 'antlr' @@ -2406,6 +2419,17 @@ class BeamModulePlugin implements Plugin { generatedSourceDirs += project.generateTestGrammarSource.outputDirectory } } + + // add dependency BeamModulePlugin defined custom tasks + // they are defined only when certain flags are provided (e.g. -Prelease; -Ppublishing, etc) + def sourcesJar = project.tasks.findByName('sourcesJar') + if (sourcesJar != null) { + sourcesJar.mustRunAfter project.tasks.getByName('generateGrammarSource') + } + def testSourcesJar = project.tasks.findByName('testSourcesJar') + if (testSourcesJar != null) { + testSourcesJar.dependsOn project.tasks.getByName('generateTestGrammarSource') + } } // Creates a task to run the quickstart for a runner. diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle index 30fb922e9c7c..c087575f8023 100644 --- a/runners/flink/flink_runner.gradle +++ b/runners/flink/flink_runner.gradle @@ -46,36 +46,51 @@ evaluationDependsOn(":examples:java") /* * Copy & merge source overrides into build directory. */ -def sourceOverridesBase = "${project.buildDir}/source-overrides/src" +def sourceOverridesBase = project.layout.buildDirectory.dir('source-overrides/src') def copySourceOverrides = tasks.register('copySourceOverrides', Copy) { it.from main_source_overrides it.into "${sourceOverridesBase}/main/java" it.duplicatesStrategy DuplicatesStrategy.INCLUDE } -compileJava.dependsOn copySourceOverrides def copyResourcesOverrides = tasks.register('copyResourcesOverrides', Copy) { it.from main_resources_overrides it.into "${sourceOverridesBase}/main/resources" it.duplicatesStrategy DuplicatesStrategy.INCLUDE } -processResources.dependsOn copyResourcesOverrides def copyTestSourceOverrides = tasks.register('copyTestSourceOverrides', Copy) { it.from test_source_overrides it.into "${sourceOverridesBase}/test/java" it.duplicatesStrategy DuplicatesStrategy.INCLUDE } -compileTestJava.dependsOn copyTestSourceOverrides def copyTestResourcesOverrides = tasks.register('copyTestResourcesOverrides', Copy) { it.from test_resources_overrides it.into "${sourceOverridesBase}/test/resources" it.duplicatesStrategy DuplicatesStrategy.INCLUDE } + +// add dependency to gradle Java plugin defined tasks +compileJava.dependsOn copySourceOverrides +processResources.dependsOn copyResourcesOverrides +compileTestJava.dependsOn copyTestSourceOverrides processTestResources.dependsOn copyTestResourcesOverrides +// add dependency BeamModulePlugin defined custom tasks +// they are defined only when certain flags are provided (e.g. -Prelease; -Ppublishing, etc) +def sourcesJar = project.tasks.findByName('sourcesJar') +if (sourcesJar != null) { + sourcesJar.dependsOn copySourceOverrides + sourcesJar.dependsOn copyResourcesOverrides +} +def testSourcesJar = project.tasks.findByName('testSourcesJar') +if (testSourcesJar != null) { + testSourcesJar.dependsOn copyTestSourceOverrides + testSourcesJar.dependsOn copyTestResourcesOverrides +} + /* * We have to explicitly set all directories here to make sure each * version of Flink has the correct overrides set. diff --git a/sdks/java/maven-archetypes/examples/build.gradle b/sdks/java/maven-archetypes/examples/build.gradle index 56b4a7c84285..1edb55a10f95 100644 --- a/sdks/java/maven-archetypes/examples/build.gradle +++ b/sdks/java/maven-archetypes/examples/build.gradle @@ -72,6 +72,13 @@ task generateSources(type: Exec) { commandLine './generate-sources.sh' } +// add dependency BeamModulePlugin defined custom tasks +// they are defined only when certain flags are provided (e.g. -Prelease; -Ppublishing, etc) +def sourcesJar = project.tasks.findByName('sourcesJar') +if (sourcesJar != null) { + sourcesJar.dependsOn generateSources +} + sourceSets { main { output.dir('src', builtBy: 'generateSources') diff --git a/sdks/java/maven-archetypes/gcp-bom-examples/build.gradle b/sdks/java/maven-archetypes/gcp-bom-examples/build.gradle index 541c91bd6adb..f9fabcfe19b0 100644 --- a/sdks/java/maven-archetypes/gcp-bom-examples/build.gradle +++ b/sdks/java/maven-archetypes/gcp-bom-examples/build.gradle @@ -71,6 +71,12 @@ task generateSources(type: Exec) { environment "HERE", "." commandLine '../examples/generate-sources.sh' } +// add dependency BeamModulePlugin defined custom tasks +// they are defined only when certain flags are provided (e.g. -Prelease; -Ppublishing, etc) +def sourcesJar = project.tasks.findByName('sourcesJar') +if (sourcesJar != null) { + sourcesJar.dependsOn generateSources +} sourceSets { main { From 152f9c52b57e8cc3c7d791ab787217c333e71f4d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 10 Oct 2023 10:17:26 -0400 Subject: [PATCH 49/85] Bump golang.org/x/oauth2 from 0.12.0 to 0.13.0 in /sdks (#28912) Bumps [golang.org/x/oauth2](https://github.com/golang/oauth2) from 0.12.0 to 0.13.0. - [Commits](https://github.com/golang/oauth2/compare/v0.12.0...v0.13.0) --- updated-dependencies: - dependency-name: golang.org/x/oauth2 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 a2b5ed097642..53eae7db4539 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -53,7 +53,7 @@ require ( github.com/xitongsys/parquet-go-source v0.0.0-20220315005136-aec0fe3e777c go.mongodb.org/mongo-driver v1.12.1 golang.org/x/net v0.16.0 - golang.org/x/oauth2 v0.12.0 + golang.org/x/oauth2 v0.13.0 golang.org/x/sync v0.4.0 golang.org/x/sys v0.13.0 golang.org/x/text v0.13.0 diff --git a/sdks/go.sum b/sdks/go.sum index 516adec93875..5dc085b859a0 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -548,8 +548,8 @@ golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4Iltr 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.12.0 h1:smVPGxink+n1ZI5pkQa8y6fZT0RW0MgCO5bFpepy4B4= -golang.org/x/oauth2 v0.12.0/go.mod h1:A74bZ3aGXgCY0qaIC9Ahg6Lglin4AMAco8cIv9baba4= +golang.org/x/oauth2 v0.13.0 h1:jDDenyj+WgFtmV3zYVoi8aE2BwtXFLWOA67ZfNWftiY= +golang.org/x/oauth2 v0.13.0/go.mod h1:/JMhi4ZRXAf4HG9LiNmxvk+45+96RUlVThiH8FzNBn0= 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= From e01de66459416f8f40dc3fbe64243fef9694ffc3 Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Tue, 10 Oct 2023 10:42:23 -0400 Subject: [PATCH 50/85] remove unused os import (#28914) --- sdks/go/test/integration/integration.go | 1 - 1 file changed, 1 deletion(-) diff --git a/sdks/go/test/integration/integration.go b/sdks/go/test/integration/integration.go index d3eb1c3ee7c3..5b7473fb561a 100644 --- a/sdks/go/test/integration/integration.go +++ b/sdks/go/test/integration/integration.go @@ -38,7 +38,6 @@ package integration import ( "fmt" "math/rand" - "os" "regexp" "strings" "testing" From e130352d7c7eee8503e82c5e921126d12b88a111 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 10 Oct 2023 10:43:10 -0400 Subject: [PATCH 51/85] Remove issue_comment trigger for load test, performance test and other jobs (#28915) --- .github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml | 2 -- .../workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml | 2 -- .../workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml | 2 -- .../workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml | 2 -- .../beam_LoadTests_Java_Combine_Dataflow_Streaming.yml | 2 -- ...am_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml | 2 -- .../workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml | 2 -- .../beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 2 -- .../beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 2 -- .../beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml | 2 -- .../beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml | 2 -- .../beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml | 2 -- .../workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml | 2 -- ...beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml | 2 -- .../workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml | 2 -- .github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml | 2 -- .../workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml | 2 -- .../beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml | 2 -- .github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml | 2 -- .../workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml | 2 -- .github/workflows/beam_PerformanceTests_AvroIOIT.yml | 2 -- .github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml | 2 -- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml | 2 -- .../beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml | 2 -- .../beam_PerformanceTests_BigQueryIO_Streaming_Java.yml | 2 -- .../workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml | 2 -- .../beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml | 2 -- .github/workflows/beam_PerformanceTests_Cdap.yml | 2 -- .github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml | 2 -- .../beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml | 2 -- .github/workflows/beam_PerformanceTests_HadoopFormat.yml | 2 -- .github/workflows/beam_PerformanceTests_JDBC.yml | 2 -- .github/workflows/beam_PerformanceTests_Kafka_IO.yml | 2 -- .github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml | 2 -- .../workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml | 2 -- .github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml | 2 -- .github/workflows/beam_PerformanceTests_ParquetIOIT.yml | 2 -- .github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml | 2 -- .../beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml | 2 -- .../beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml | 2 -- .../beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml | 2 -- .../beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml | 2 -- .github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml | 2 -- .github/workflows/beam_PerformanceTests_TFRecordIOIT.yml | 2 -- .github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml | 2 -- .github/workflows/beam_PerformanceTests_TextIOIT.yml | 2 -- .github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml | 2 -- .github/workflows/beam_PerformanceTests_TextIOIT_Python.yml | 2 -- .../beam_PerformanceTests_WordCountIT_PythonVersions.yml | 2 -- .github/workflows/beam_PerformanceTests_XmlIOIT.yml | 2 -- .github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml | 2 -- .../workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml | 2 -- .github/workflows/beam_Publish_Docker_Snapshots.yml | 2 -- .../workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml | 2 -- 64 files changed, 128 deletions(-) diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml index d90b7a5a4cc3..0d4c9ab48d27 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests CoGBK Dataflow Batch Go on: - issue_comment: - types: [created] schedule: - cron: '40 23 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml index cf355a2ee98d..18238c796e47 100644 --- a/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml +++ b/.github/workflows/beam_LoadTests_Go_CoGBK_Flink_batch.yml @@ -16,8 +16,6 @@ name: LoadTests Go CoGBK Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '10 14 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml index 423290d3fdc6..172f48a83e33 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests Combine Dataflow Batch Go on: - issue_comment: - types: [created] schedule: - cron: '40 23 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml index 0226b003b58e..e5dbef1df53d 100644 --- a/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_Combine_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Go Combine Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '40 6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml index bfdb19c1f5d5..ae4528ced8e4 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests GBK Dataflow Batch Go on: - issue_comment: - types: [created] schedule: - cron: '40 23 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml index 7c4d95738a09..a26fe19b79b7 100644 --- a/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_GBK_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Go GBK Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '20 1 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml index e4b4f0997d4d..18dbf26409ad 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests ParDo Dataflow Batch Go on: - issue_comment: - types: [created] schedule: - cron: '15 18 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml index 34211f9270ff..094005e2f532 100644 --- a/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_ParDo_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Go ParDo Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '40 2 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml index cad15e4eae0c..6265fe428c00 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests SideInput Dataflow Batch Go on: - issue_comment: - types: [created] schedule: - cron: '40 23 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml index 8600c5cd3717..20e8b4d54dd3 100644 --- a/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Go_SideInput_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Go SideInput Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '40 23 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml index 0cb601522a81..3ad2cdf995e7 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: Load Tests CoGBK Dataflow Streaming Java on: - issue_comment: - types: [created] schedule: - cron: '50 10 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml index 758d196f273c..bc408402c792 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests Combine Dataflow Batch Java on: - issue_comment: - types: [created] schedule: - cron: '35 7 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml index b0df52354722..029b7024d6ed 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Java Combine Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '25 14 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml index 28c17ffb535c..5f4b8594ecde 100644 --- a/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_Combine_SparkStructuredStreaming_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Java Combine SparkStructuredStreaming Batch on: - issue_comment: - types: [created] schedule: - cron: '15 18 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml index b490ce69323e..9f7da2c00049 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Dataflow Batch on: - issue_comment: - types: [created] schedule: - cron: '10 6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml index 19cc12f91eb1..fd718e23fd0a 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '50 6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml index aeb8006dfd53..318f157e57a6 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Dataflow V2 Batch Java11 on: - issue_comment: - types: [created] schedule: - cron: '10 7 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml index 23f9f34d9747..543cb86985d8 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Dataflow V2 Batch Java17 on: - issue_comment: - types: [created] schedule: - cron: '50 7 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml index 9a0b236e43e9..fcd55f761c04 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Dataflow V2 Streaming Java11 on: - issue_comment: - types: [created] schedule: - cron: '50 8 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml index 3580aaca51bb..5df7d3249192 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK Dataflow V2 Streaming Java17 on: - issue_comment: - types: [created] schedule: - cron: '50 9 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml index 039a9fc3b752..27505e9e88e9 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_SparkStructuredStreaming_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Java GBK SparkStructuredStreaming Batch on: - issue_comment: - types: [created] schedule: - cron: '10 10 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml index 6bd52d2ebae0..406cf18c85d1 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Java ParDo Dataflow Batch on: - issue_comment: - types: [created] schedule: - cron: '55 9 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml index dbee457b6ec5..4b6fcb4ad51d 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Java ParDo Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '10 11 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml index afbb79caec91..89a903cc146c 100644 --- a/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml +++ b/.github/workflows/beam_LoadTests_Java_ParDo_SparkStructuredStreaming_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Java ParDo SparkStructuredStreaming Batch on: - issue_comment: - types: [created] schedule: - cron: '25 8 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml index f7d7a056d595..d8b97f2d6032 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: Load Tests Combine Dataflow Batch Python on: - issue_comment: - types: [created] schedule: - cron: '40 5 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml index 165b23703562..d7d80e164283 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Python Combine Flink Batch on: - issue_comment: - types: [created] schedule: - cron: '10 6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml index 1dae586de21a..66ac3bd2fc08 100644 --- a/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_Combine_Flink_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Python Combine Flink Streaming on: - issue_comment: - types: [created] schedule: - cron: '10 7 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml index b74a44c647c8..f3e2181991d7 100644 --- a/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml +++ b/.github/workflows/beam_LoadTests_Python_FnApiRunner_Microbenchmark.yml @@ -16,8 +16,6 @@ name: Load Tests FnApiRunner Microbenchmark Python on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml index fa3459992d0e..edf2e97857d6 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Batch.yml @@ -16,8 +16,6 @@ name: LoadTests Python GBK Dataflow Batch on: - issue_comment: - types: [created] schedule: - cron: '10 2 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml index da0cf8eefbb8..d7e17c2676f9 100644 --- a/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml +++ b/.github/workflows/beam_LoadTests_Python_GBK_Dataflow_Streaming.yml @@ -16,8 +16,6 @@ name: LoadTests Python GBK Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '10 4 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml index a6c56287da41..14d798bb3e72 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT.yml @@ -16,8 +16,6 @@ name: Performance Tests AvroIOIT on: - issue_comment: - types: [created] schedule: - cron: '10 1/13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml index 7393e0e39b37..b84d56f34b9b 100644 --- a/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_AvroIOIT_HDFS.yml @@ -16,8 +16,6 @@ name: Performance Tests AvroIOIT HDFS on: - issue_comment: - types: [created] schedule: - cron: '10 1/13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml index d29acbfc765f..7f6daf23d05f 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Avro.yml @@ -16,8 +16,6 @@ name: Performance Tests BigQueryIO Batch Java Avro on: - issue_comment: - types: [created] schedule: - cron: '10 1,13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml index 067d0e4b95b8..13c60e107834 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Batch_Java_Json.yml @@ -16,8 +16,6 @@ name: Performance Tests BigQueryIO Batch Java Json on: - issue_comment: - types: [created] schedule: - cron: '30 8,20 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml index bf10d4be522e..75a68c65a8b3 100644 --- a/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml +++ b/.github/workflows/beam_PerformanceTests_BigQueryIO_Streaming_Java.yml @@ -16,8 +16,6 @@ name: Performance Tests BigQueryIO Streaming Java on: - issue_comment: - types: [created] schedule: - cron: '20 15,22 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml index 58958de69c89..ac84517443e9 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Read_Python.yml @@ -16,8 +16,6 @@ name: PerformanceTests BiqQueryIO Read Python on: - issue_comment: - types: [created] schedule: - cron: '0 2 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml index f0fcd20bd3b5..d2e5ddf3d07c 100644 --- a/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_BiqQueryIO_Write_Python_Batch.yml @@ -16,8 +16,6 @@ name: PerformanceTests BiqQueryIO Write Python Batch on: - issue_comment: - types: [created] schedule: - cron: '0 1 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_Cdap.yml b/.github/workflows/beam_PerformanceTests_Cdap.yml index f45419a1223f..269809121449 100644 --- a/.github/workflows/beam_PerformanceTests_Cdap.yml +++ b/.github/workflows/beam_PerformanceTests_Cdap.yml @@ -16,8 +16,6 @@ name: PerformanceTests Cdap on: - issue_comment: - types: [created] schedule: - cron: '13 4/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml index 0e82c0fdf7d1..ed5e19454d26 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT.yml @@ -16,8 +16,6 @@ name: PerformanceTests Compressed TextIOIT on: - issue_comment: - types: [created] schedule: - cron: '10 1/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml index 78c9d3e8ab7a..f43f96634459 100644 --- a/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_Compressed_TextIOIT_HDFS.yml @@ -16,8 +16,6 @@ name: PerformanceTests Compressed TextIOIT HDFS on: - issue_comment: - types: [created] schedule: - cron: '50 1/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml index d73c6f6fb5f5..f5514f765a57 100644 --- a/.github/workflows/beam_PerformanceTests_HadoopFormat.yml +++ b/.github/workflows/beam_PerformanceTests_HadoopFormat.yml @@ -16,8 +16,6 @@ name: PerformanceTests HadoopFormat on: - issue_comment: - types: [created] schedule: - cron: '16 7/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_JDBC.yml b/.github/workflows/beam_PerformanceTests_JDBC.yml index c5995480e9d5..350718e94449 100644 --- a/.github/workflows/beam_PerformanceTests_JDBC.yml +++ b/.github/workflows/beam_PerformanceTests_JDBC.yml @@ -16,8 +16,6 @@ name: PerformanceTests JDBC on: - issue_comment: - types: [created] schedule: - cron: '30 1,13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml index ea709238b8f3..6bb79bcc5b61 100644 --- a/.github/workflows/beam_PerformanceTests_Kafka_IO.yml +++ b/.github/workflows/beam_PerformanceTests_Kafka_IO.yml @@ -16,8 +16,6 @@ name: PerformanceTests Kafka IO on: - issue_comment: - types: [created] schedule: - cron: '30 2,14 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml index 5834bd8ab3e0..7f533c157234 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT.yml @@ -16,8 +16,6 @@ name: PerformanceTests ManyFiles TextIOIT on: - issue_comment: - types: [created] schedule: - cron: '10 2/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml index 03163a41dcf9..96bb0573a911 100644 --- a/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ManyFiles_TextIOIT_HDFS.yml @@ -16,8 +16,6 @@ name: PerformanceTests ManyFiles TextIOIT HDFS on: - issue_comment: - types: [created] schedule: - cron: '50 2/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml index 0ad21f99f8d4..83e0a73a9c9b 100644 --- a/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PerformanceTests_MongoDBIO_IT.yml @@ -16,8 +16,6 @@ name: PerformanceTests MongoDBIO IT on: - issue_comment: - types: [created] schedule: - cron: '14 5/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml index ceb540b16b1f..03dbb650c4ab 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT.yml @@ -16,8 +16,6 @@ name: PerformanceTests ParquetIOIT on: - issue_comment: - types: [created] schedule: - cron: '10 3/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml index d0c40599eb62..49c2296340be 100644 --- a/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_ParquetIOIT_HDFS.yml @@ -16,8 +16,6 @@ name: PerformanceTests ParquetIOIT HDFS on: - issue_comment: - types: [created] schedule: - cron: '50 3/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml index 3a16e482979f..97082ea8aeda 100644 --- a/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml +++ b/.github/workflows/beam_PerformanceTests_PubsubIOIT_Python_Streaming.yml @@ -16,8 +16,6 @@ name: PerformanceTests PubsubIOIT Python Streaming on: - issue_comment: - types: [created] schedule: - cron: '11 2 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml index 06014a56e682..2a93234e94e2 100644 --- a/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml +++ b/.github/workflows/beam_PerformanceTests_SQLBigQueryIO_Batch_Java.yml @@ -16,8 +16,6 @@ name: PerformanceTests SQLBigQueryIO Batch Java on: - issue_comment: - types: [created] schedule: - cron: '0 7,19 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml index 9e464ef58900..09e165e64e52 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Read_2GB_Python.yml @@ -16,8 +16,6 @@ name: PerformanceTests SpannerIO Read 2GB Python on: - issue_comment: - types: [created] schedule: - cron: '30 4 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml index 8cfce643f749..b1f7761b1133 100644 --- a/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml +++ b/.github/workflows/beam_PerformanceTests_SpannerIO_Write_2GB_Python_Batch.yml @@ -16,8 +16,6 @@ name: PerformanceTests SpannerIO Write 2GB Python Batch on: - issue_comment: - types: [created] schedule: - cron: '0 5 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml index 58c561f6ef64..8f6c72afb415 100644 --- a/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml +++ b/.github/workflows/beam_PerformanceTests_SparkReceiver_IO.yml @@ -16,8 +16,6 @@ name: PerformanceTests SparkReceiver IO on: - issue_comment: - types: [created] schedule: - cron: '15 6/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml index aa96b7e2bf81..93e9132e1d1d 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT.yml @@ -16,8 +16,6 @@ name: PerformanceTests TFRecordIOIT on: - issue_comment: - types: [created] schedule: - cron: '10 4/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml index bbc4a79aa0f3..dab14af741b7 100644 --- a/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TFRecordIOIT_HDFS.yml @@ -18,8 +18,6 @@ name: PerformanceTests TFRecordIOIT HDFS on: # TODO(https://github.com/apache/beam/issues/18796) TFRecord performance test is failing only when running on hdfs. # We need to fix this before enabling this job on jenkins. - # issue_comment: - # types: [created] # schedule: # - cron: '17 8/20 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT.yml b/.github/workflows/beam_PerformanceTests_TextIOIT.yml index 9daa2b29dd2a..c313731206be 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT.yml @@ -16,8 +16,6 @@ name: PerformanceTests TextIOIT on: - issue_comment: - types: [created] schedule: - cron: '0 7,19 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml index a98379b281a9..118605fe1026 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_HDFS.yml @@ -16,8 +16,6 @@ name: PerformanceTests TextIOIT HDFS on: - issue_comment: - types: [created] schedule: - cron: '30 7,19 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml index cb2b7fb34a9f..deb589b18c51 100644 --- a/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml +++ b/.github/workflows/beam_PerformanceTests_TextIOIT_Python.yml @@ -16,8 +16,6 @@ name: PerformanceTests TextIOIT Python on: - issue_comment: - types: [created] schedule: - cron: '0 8,20 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml index 53b157d691c5..82069f943329 100644 --- a/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml +++ b/.github/workflows/beam_PerformanceTests_WordCountIT_PythonVersions.yml @@ -16,8 +16,6 @@ name: PerformanceTests WordCountIT PythonVersions on: - issue_comment: - types: [created] schedule: - cron: '12 3 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml index cd0245f269d1..5b2a64f1a919 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT.yml @@ -16,8 +16,6 @@ name: PerformanceTests XmlIOIT on: - issue_comment: - types: [created] schedule: - cron: '30 4/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml index a89f0b5dcb0f..3358085a0120 100644 --- a/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml +++ b/.github/workflows/beam_PerformanceTests_XmlIOIT_HDFS.yml @@ -16,8 +16,6 @@ name: PerformanceTests XmlIOIT HDFS on: - issue_comment: - types: [created] schedule: - cron: '50 4/12 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml index a29454ced4bf..ee84de4ac618 100644 --- a/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml +++ b/.github/workflows/beam_PerformanceTests_xlang_KafkaIO_Python.yml @@ -16,8 +16,6 @@ name: PerformanceTests xlang KafkaIO Python on: - issue_comment: - types: [created] schedule: - cron: '10 5 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_Publish_Docker_Snapshots.yml b/.github/workflows/beam_Publish_Docker_Snapshots.yml index 41262a58b1e7..75ff5df3de47 100644 --- a/.github/workflows/beam_Publish_Docker_Snapshots.yml +++ b/.github/workflows/beam_Publish_Docker_Snapshots.yml @@ -16,8 +16,6 @@ name: Publish Docker Snapshots on: - issue_comment: - types: [created] schedule: - cron: '0 13 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml index 9a4ff4144ac5..ce1d824df6e0 100644 --- a/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml +++ b/.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml @@ -16,8 +16,6 @@ name: Python ValidatesContainer Dataflow ARM on: - issue_comment: - types: [created] push: tags: ['v*'] branches: ['master', 'release-*'] From a7abe61288cd39ad8fc4baf74e782ef7cefe0b69 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 10 Oct 2023 11:48:55 -0400 Subject: [PATCH 52/85] Remove issue_comment trigger in most postcommit (#28916) --- .github/workflows/beam_PostCommit_BeamMetrics_Publish.yml | 2 -- .github/workflows/beam_PostCommit_Go.yml | 2 -- .github/workflows/beam_PostCommit_Go_VR_Flink.yml | 2 -- .github/workflows/beam_PostCommit_Go_VR_Samza.yml | 2 -- .github/workflows/beam_PostCommit_Go_VR_Spark.yml | 2 -- .github/workflows/beam_PostCommit_Java.yml | 2 -- .github/workflows/beam_PostCommit_Java_Avro_Versions.yml | 2 -- .github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml | 2 -- .github/workflows/beam_PostCommit_Java_DataflowV1.yml | 2 -- .github/workflows/beam_PostCommit_Java_DataflowV2.yml | 2 -- .../workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml | 2 -- .github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml | 2 -- .../beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml | 2 -- .github/workflows/beam_PostCommit_Java_Examples_Direct.yml | 2 -- .github/workflows/beam_PostCommit_Java_Examples_Flink.yml | 2 -- .github/workflows/beam_PostCommit_Java_Examples_Spark.yml | 2 -- .github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml | 2 -- .github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml | 2 -- .github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml | 2 -- .github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml | 2 -- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml | 2 -- .github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml | 2 -- .github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml | 2 -- .github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml | 2 -- .github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml | 2 -- .github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml | 2 -- .../workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml | 2 -- .github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml | 2 -- .github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml | 2 -- .github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml | 2 -- .github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml | 2 -- .github/workflows/beam_PostCommit_Java_PVR_Samza.yml | 2 -- .github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml | 2 -- .github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml | 2 -- .github/workflows/beam_PostCommit_Java_Sickbay.yml | 2 -- .github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml | 2 -- .github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml | 2 -- .github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml | 2 -- .github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml | 2 -- .../workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml | 2 -- ...am_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml | 2 -- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml | 2 -- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml | 2 -- ...am_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml | 2 -- .../workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml | 2 -- ...beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml | 2 -- .../workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml | 2 -- .../beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml | 2 -- .../workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml | 2 -- .../workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml | 2 -- ...PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml | 2 -- .../beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml | 2 -- .../workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml | 2 -- .github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml | 2 -- .github/workflows/beam_PostCommit_Javadoc.yml | 2 -- .github/workflows/beam_PostCommit_PortableJar_Flink.yml | 2 -- .github/workflows/beam_PostCommit_PortableJar_Spark.yml | 2 -- .github/workflows/beam_PostCommit_Python.yml | 2 -- .github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml | 2 -- .github/workflows/beam_PostCommit_Python_Examples_Direct.yml | 2 -- .github/workflows/beam_PostCommit_Python_Examples_Flink.yml | 2 -- .github/workflows/beam_PostCommit_Python_Examples_Spark.yml | 2 -- .github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml | 2 -- .github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml | 2 -- .../beam_PostCommit_Python_ValidatesContainer_Dataflow.yml | 2 -- ...am_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml | 2 -- .../beam_PostCommit_Python_ValidatesRunner_Dataflow.yml | 2 -- .../workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml | 2 -- .../workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml | 2 -- .../workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml | 2 -- .github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml | 2 -- .github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml | 2 -- .github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml | 2 -- .github/workflows/beam_PostCommit_SQL.yml | 2 -- .github/workflows/beam_PostCommit_Sickbay_Python.yml | 2 -- .github/workflows/beam_PostCommit_TransformService_Direct.yml | 2 -- .github/workflows/beam_PostCommit_Website_Test.yml | 2 -- .github/workflows/beam_PostCommit_XVR_Direct.yml | 2 -- .github/workflows/beam_PostCommit_XVR_Flink.yml | 2 -- .github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml | 2 -- .../workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml | 2 -- .../beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml | 2 -- .../workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml | 2 -- .github/workflows/beam_PostCommit_XVR_Samza.yml | 2 -- .github/workflows/beam_PostCommit_XVR_Spark3.yml | 2 -- 85 files changed, 170 deletions(-) diff --git a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml index 0ea3207b505a..bccb2c8e78c6 100644 --- a/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml +++ b/.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml @@ -20,8 +20,6 @@ on: tags: ['v*'] branches: ['master', 'release-*'] paths: ['.github/workflows/beam_PostCommit_BeamMetrics_Publish.yml'] - issue_comment: - types: [created] schedule: - cron: '24 2 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Go.yml b/.github/workflows/beam_PostCommit_Go.yml index dc1180314d67..5fff2d334a5a 100644 --- a/.github/workflows/beam_PostCommit_Go.yml +++ b/.github/workflows/beam_PostCommit_Go.yml @@ -16,8 +16,6 @@ name: PostCommit Go on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml index 21dcf7f8e72a..3923e9d74f94 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Flink.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit Go VR Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml index 90e107ee0c17..4e0663be0dbf 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml @@ -16,8 +16,6 @@ name: PostCommit Go VR Samza on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml index 07cd627059b2..85262335c9a5 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Spark.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Spark.yml @@ -16,8 +16,6 @@ name: PostCommit Go VR Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java.yml b/.github/workflows/beam_PostCommit_Java.yml index 3eed85bc2026..0943e2dec263 100644 --- a/.github/workflows/beam_PostCommit_Java.yml +++ b/.github/workflows/beam_PostCommit_Java.yml @@ -18,8 +18,6 @@ name: PostCommit Java on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml index 1bd828d08ee0..63ffda3864e6 100644 --- a/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Avro_Versions.yml @@ -18,8 +18,6 @@ name: PostCommit Java Avro Versions on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml index 952273e810d2..f74b429988d7 100644 --- a/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml +++ b/.github/workflows/beam_PostCommit_Java_BigQueryEarlyRollout.yml @@ -18,8 +18,6 @@ name: PostCommit Java BigQueryEarlyRollout on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml index 55e4f99afc74..b6cf668ddfd3 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV1.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV1.yml @@ -18,8 +18,6 @@ name: PostCommit Java Dataflow V1 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml index 7ca2f57ce78d..f99c08c36f8d 100644 --- a/.github/workflows/beam_PostCommit_Java_DataflowV2.yml +++ b/.github/workflows/beam_PostCommit_Java_DataflowV2.yml @@ -18,8 +18,6 @@ name: PostCommit Java Dataflow V2 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml index a998e0242254..3d2523654fb6 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_Java.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Dataflow Java on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index 0c87a81c5bed..67780ae4a848 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Dataflow V2 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index 2526f9b56531..74e1787945e3 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Dataflow V2 Java on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml index 6fd1150aecd1..9508dfe22617 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Direct.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml index b123134cd239..c359ae2f9e53 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Flink.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml index fa28cdf402bf..78130d9ec247 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Spark.yml @@ -18,8 +18,6 @@ name: PostCommit Java Examples Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml index f9c7175ccd21..189fbb083a39 100644 --- a/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml +++ b/.github/workflows/beam_PostCommit_Java_Hadoop_Versions.yml @@ -16,8 +16,6 @@ name: PostCommit Java Hadoop Versions on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml index f048fdc6f1f7..7f1d76a13936 100644 --- a/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_InfluxDbIO_IT.yml @@ -18,8 +18,6 @@ name: Java InfluxDbIO Integration Test on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml index 3ee009747a88..d5b23c095d0b 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java11.yml @@ -16,8 +16,6 @@ name: PostCommit Java Jpms Dataflow Java11 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 398be301eecf..0b9a366211ee 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -16,8 +16,6 @@ name: PostCommit Java Jpms Dataflow Java17 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml index 44b89acb642e..97f03ed59211 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java11.yml @@ -16,8 +16,6 @@ name: PostCommit Java Jpms Direct Java11 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index dbeb84ab660b..fcdbd63f7e97 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -16,8 +16,6 @@ name: PostCommit Java Jpms Direct Java17 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml index f4ba8ada32ba..832e211a4d61 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Flink_Java11.yml @@ -16,8 +16,6 @@ name: PostCommit Java Jpms Flink Java11 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml index 9758fda66eb7..14b5e5380921 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Spark_Java11.yml @@ -16,8 +16,6 @@ name: PostCommit Java Jpms Spark Java11 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml index 74b786b599e1..44bdb4e51ca8 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow.yml @@ -18,8 +18,6 @@ name: PostCommit Java Nexmark Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml index 3b7836990b69..50a72af7909a 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2.yml @@ -18,8 +18,6 @@ name: PostCommit Java Nexmark Dataflow V2 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index a0957f643279..d78793172898 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -18,8 +18,6 @@ name: PostCommit Java Nexmark Dataflow V2 Java on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml index 4daa13da8b13..e7c6e4655671 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Direct.yml @@ -18,8 +18,6 @@ name: PostCommit Java Nexmark Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml index a03c447416f3..7dc59a1a2e72 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Flink.yml @@ -18,8 +18,6 @@ name: PostCommit Java Nexmark Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml index a43e7e6d311f..fd1f9745d80e 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Spark.yml @@ -18,8 +18,6 @@ name: PostCommit Java Nexmark Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml index 991e4f71b1c4..efab6427642a 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Flink_Streaming.yml @@ -18,8 +18,6 @@ name: PostCommit Java PVR Flink Streaming on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml index 041c031f3f76..942028ea3569 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Samza.yml @@ -18,8 +18,6 @@ name: PostCommit Java PVR Samza on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml index 2a375d71981a..6b47a64572ea 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark3_Streaming.yml @@ -18,8 +18,6 @@ name: PostCommit Java PVR Spark3 Streaming on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml index 8d54c7707258..82b132522c12 100644 --- a/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml +++ b/.github/workflows/beam_PostCommit_Java_PVR_Spark_Batch.yml @@ -18,8 +18,6 @@ name: PostCommit Java PVR Spark Batch on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Sickbay.yml b/.github/workflows/beam_PostCommit_Java_Sickbay.yml index 7f914dc35880..5d2da81e2ab0 100644 --- a/.github/workflows/beam_PostCommit_Java_Sickbay.yml +++ b/.github/workflows/beam_PostCommit_Java_Sickbay.yml @@ -18,8 +18,6 @@ name: PostCommit Java Sickbay on: - issue_comment: - types: [created] schedule: - cron: '0 0 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml index 40a1dc8faa64..ac8b5cacf40b 100644 --- a/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Java_SingleStoreIO_IT.yml @@ -18,8 +18,6 @@ name: PostCommit Java SingleStoreIO IT on: - issue_comment: - types: [created] schedule: - cron: '0 */23 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml index e19831c60732..3239281fb938 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit Java Tpcds Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml index fd35fe4ec776..c6dcdcfaa3a4 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit Java Tpcds Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml index 2284e7fa06bf..73fa4e5ba143 100644 --- a/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_Tpcds_Spark.yml @@ -16,8 +16,6 @@ name: PostCommit Java Tpcds Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml index 596ef873c964..0a0095e941ae 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml index 19c2f3f8cb16..e7d9bf9ccb6b 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_JavaVersions.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Dataflow JavaVersions on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml index 013b34bc807b..094133af2bb9 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Dataflow Streaming on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml index 3a9e0140f818..4331772d32d4 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Dataflow V2 on: - issue_comment: - types: [created] schedule: - cron: '0 */8 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml index edf9a04f22c3..e0f90156f370 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2_Streaming.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Dataflow V2 Streaming on: - issue_comment: - types: [created] schedule: - cron: '0 */8 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml index 79447f610809..1020b3869b97 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml index 8e5dbac0c4cd..12bd26d070ed 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Direct_JavaVersions.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Direct JavaVersions on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml index 2ff883dafa75..5de57835a190 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit Java ValidatesRunner Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml index c0339100845e..aa5c24ac2b21 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Flink_Java11.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Flink Java11 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml index 97e35490c25d..62e003a1660d 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Samza.yml @@ -16,8 +16,6 @@ name: PostCommit Java ValidatesRunner Samza on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml index 5800c338ed16..083e473a3a56 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark.yml @@ -16,8 +16,6 @@ name: PostCommit Java ValidatesRunner Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml index 08504316333a..74bb0b15b8de 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.yml @@ -16,8 +16,6 @@ name: PostCommit Java ValidatesRunner SparkStructuredStreaming on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml index 1cc0193b1e69..8082f2143b38 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Spark_Java11.yml @@ -18,8 +18,6 @@ name: PostCommit Java ValidatesRunner Spark Java11 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml index 21bacac11d59..530c360ea346 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_Twister2.yml @@ -16,8 +16,6 @@ name: PostCommit Java ValidatesRunner Twister2 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml index 75c07bc49783..61e0affa5e81 100644 --- a/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml +++ b/.github/workflows/beam_PostCommit_Java_ValidatesRunner_ULR.yml @@ -16,8 +16,6 @@ name: PostCommit Java ValidatesRunner ULR on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Javadoc.yml b/.github/workflows/beam_PostCommit_Javadoc.yml index 7185f588f463..8413f39b3b9d 100644 --- a/.github/workflows/beam_PostCommit_Javadoc.yml +++ b/.github/workflows/beam_PostCommit_Javadoc.yml @@ -18,8 +18,6 @@ name: PostCommit Javadoc on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml index 3f682ea57dde..18941e70a881 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Flink.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit PortableJar Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml index f9884d651978..55cdffcb86a1 100644 --- a/.github/workflows/beam_PostCommit_PortableJar_Spark.yml +++ b/.github/workflows/beam_PostCommit_PortableJar_Spark.yml @@ -16,8 +16,6 @@ name: PostCommit PortableJar Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python.yml b/.github/workflows/beam_PostCommit_Python.yml index 3fb650606b40..97299644e5dc 100644 --- a/.github/workflows/beam_PostCommit_Python.yml +++ b/.github/workflows/beam_PostCommit_Python.yml @@ -18,8 +18,6 @@ name: PostCommit Python on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml index 40c508b38e6e..81687e498b8b 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit Python Examples Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml index 85d766b0575c..903101e5a01c 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Direct.yml @@ -16,8 +16,6 @@ name: PostCommit Python Examples Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml index 17112cf18e2a..aa1404a4aa64 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit Python Examples Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml index ccc7a998df27..7716ca3e67b1 100644 --- a/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_Examples_Spark.yml @@ -16,8 +16,6 @@ name: PostCommit Python Examples Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml index 0fb2302b8ed1..7c8b7d2526ab 100644 --- a/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml +++ b/.github/workflows/beam_PostCommit_Python_MongoDBIO_IT.yml @@ -16,8 +16,6 @@ name: PostCommit Python MongoDBIO IT on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml index db95a48a3007..bf96c1cf74be 100644 --- a/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Nexmark_Direct.yml @@ -18,8 +18,6 @@ name: PostCommit Python Nexmark Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml index fb7102a5b52e..713f7fa94e20 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit Python ValidatesContainer Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index a95682e415d4..18b57aad8057 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -16,8 +16,6 @@ name: PostCommit Python ValidatesContainer Dataflow With RC on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml index 572f477773b6..6ff8edb9cce8 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit Python ValidatesRunner Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml index a1652f3a18de..de2ed09b9614 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit Python ValidatesRunner Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml index c41605391d40..b86fcfac218b 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Samza.yml @@ -16,8 +16,6 @@ name: PostCommit Python ValidatesRunner Samza on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml index d42835727d36..65fa97a5b5ea 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesRunner_Spark.yml @@ -16,8 +16,6 @@ name: PostCommit Python ValidatesRunner Spark on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml index 31025df81ddc..6c2255f60f1d 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit Python Xlang Gcp Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml index d17d15029c2c..36d100473f98 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_Gcp_Direct.yml @@ -16,8 +16,6 @@ name: PostCommit Python Xlang Gcp Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml index 386d91518795..eb9665d39d7f 100644 --- a/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_Python_Xlang_IO_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit Python Xlang IO Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_SQL.yml b/.github/workflows/beam_PostCommit_SQL.yml index a753cde6e56d..3cd72fcecc42 100644 --- a/.github/workflows/beam_PostCommit_SQL.yml +++ b/.github/workflows/beam_PostCommit_SQL.yml @@ -18,8 +18,6 @@ name: PostCommit SQL on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Sickbay_Python.yml b/.github/workflows/beam_PostCommit_Sickbay_Python.yml index 2153e9908898..7e15ac7e8873 100644 --- a/.github/workflows/beam_PostCommit_Sickbay_Python.yml +++ b/.github/workflows/beam_PostCommit_Sickbay_Python.yml @@ -18,8 +18,6 @@ name: PostCommit Sickbay Python on: - issue_comment: - types: [created] schedule: - cron: '0 0 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 9387e9f74831..491688e097a0 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -16,8 +16,6 @@ name: PostCommit TransformService Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_Website_Test.yml b/.github/workflows/beam_PostCommit_Website_Test.yml index cd06181c456d..bb12d5804b6f 100644 --- a/.github/workflows/beam_PostCommit_Website_Test.yml +++ b/.github/workflows/beam_PostCommit_Website_Test.yml @@ -16,8 +16,6 @@ name: PostCommit Website Test on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index eaee2b669873..8db745b8b217 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -16,8 +16,6 @@ name: PostCommit XVR Direct on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 67effa70b8b5..640555a03ec1 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -16,8 +16,6 @@ name: PostCommit XVR Flink on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml index 2f8a2eb8a3da..be10e5fb1bb3 100644 --- a/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_GoUsingJava_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit XVR GoUsingJava Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml index 386fa304606d..4148d0a2d468 100644 --- a/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_JavaUsingPython_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit XVR JavaUsingPython Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml index 2432ddc30e2a..6c8e01124d0b 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit XVR PythonUsingJavaSQL Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml index bebac78a6315..486fe84ff45c 100644 --- a/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml +++ b/.github/workflows/beam_PostCommit_XVR_PythonUsingJava_Dataflow.yml @@ -16,8 +16,6 @@ name: PostCommit XVR PythonUsingJava Dataflow on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index 8cf16cfe225d..8a6c9bda6539 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -16,8 +16,6 @@ name: PostCommit XVR Samza on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 193526ffe7f3..b61cfe3251f0 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -16,8 +16,6 @@ name: PostCommit XVR Spark3 on: - issue_comment: - types: [created] schedule: - cron: '0 */6 * * *' workflow_dispatch: From 2bccee14cf9261f95db5a433a404454bb28b0a9c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 10 Oct 2023 13:14:11 -0400 Subject: [PATCH 53/85] Bump google.golang.org/api from 0.144.0 to 0.146.0 in /sdks (#28911) Bumps [google.golang.org/api](https://github.com/googleapis/google-api-go-client) from 0.144.0 to 0.146.0. - [Release notes](https://github.com/googleapis/google-api-go-client/releases) - [Changelog](https://github.com/googleapis/google-api-go-client/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-api-go-client/compare/v0.144.0...v0.146.0) --- updated-dependencies: - dependency-name: google.golang.org/api 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 53eae7db4539..0bc60b630bf0 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -57,7 +57,7 @@ require ( golang.org/x/sync v0.4.0 golang.org/x/sys v0.13.0 golang.org/x/text v0.13.0 - google.golang.org/api v0.144.0 + google.golang.org/api v0.146.0 google.golang.org/genproto v0.0.0-20230913181813-007df8e322eb google.golang.org/grpc v1.58.2 google.golang.org/protobuf v1.31.0 diff --git a/sdks/go.sum b/sdks/go.sum index 5dc085b859a0..7743b2bc3247 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -670,8 +670,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.144.0 h1:01xgplvIwdMpnrlenPHMgRAAgAH9N5Zv21Qu6XwJxSU= -google.golang.org/api v0.144.0/go.mod h1:OARJqIfoYjXJj4C1AiBSXYZt03qsoz8FQYU6fBEfrHM= +google.golang.org/api v0.146.0 h1:9aBYT4vQXt9dhCuLNfwfd3zpwu8atg0yPkjBymwSrOM= +google.golang.org/api v0.146.0/go.mod h1:OARJqIfoYjXJj4C1AiBSXYZt03qsoz8FQYU6fBEfrHM= 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= From a859863d3926db18dc0f2112dd0b7e9a5105923f Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Tue, 10 Oct 2023 15:54:02 -0400 Subject: [PATCH 54/85] update dataflow containers (#28904) --- sdks/python/apache_beam/runners/dataflow/internal/names.py | 2 +- .../www/site/content/en/documentation/runtime/environments.md | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/names.py b/sdks/python/apache_beam/runners/dataflow/internal/names.py index 8347a597e968..579764aeb7c1 100644 --- a/sdks/python/apache_beam/runners/dataflow/internal/names.py +++ b/sdks/python/apache_beam/runners/dataflow/internal/names.py @@ -34,6 +34,6 @@ # Unreleased sdks use container image tag specified below. # Update this tag whenever there is a change that # requires changes to SDK harness container or SDK harness launcher. -BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20230927' +BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20231009' DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3' diff --git a/website/www/site/content/en/documentation/runtime/environments.md b/website/www/site/content/en/documentation/runtime/environments.md index 624b5aa6b881..452fb6141e63 100644 --- a/website/www/site/content/en/documentation/runtime/environments.md +++ b/website/www/site/content/en/documentation/runtime/environments.md @@ -115,14 +115,13 @@ This method requires building image artifacts from Beam source. For additional i ./gradlew :sdks:java:container:java11:docker ./gradlew :sdks:java:container:java17:docker ./gradlew :sdks:go:container:docker - ./gradlew :sdks:python:container:py36:docker ./gradlew :sdks:python:container:py38:docker ./gradlew :sdks:python:container:py39:docker ./gradlew :sdks:python:container:py310:docker ./gradlew :sdks:python:container:py311:docker # Shortcut for building all Python SDKs - ./gradlew :sdks:python:container buildAll + ./gradlew :sdks:python:container:buildAll ``` 4. Verify the images you built were created by running `docker images`. From d2420ad6ee5120e13fee9bd4061e9906f858e2dd Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 10 Oct 2023 16:04:10 -0400 Subject: [PATCH 55/85] Fix buildDirectory (#28925) --- runners/flink/flink_runner.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/runners/flink/flink_runner.gradle b/runners/flink/flink_runner.gradle index c087575f8023..b1a459337e51 100644 --- a/runners/flink/flink_runner.gradle +++ b/runners/flink/flink_runner.gradle @@ -46,7 +46,7 @@ evaluationDependsOn(":examples:java") /* * Copy & merge source overrides into build directory. */ -def sourceOverridesBase = project.layout.buildDirectory.dir('source-overrides/src') +def sourceOverridesBase = project.layout.buildDirectory.dir('source-overrides/src').get() def copySourceOverrides = tasks.register('copySourceOverrides', Copy) { it.from main_source_overrides From 65eaf45026e9eeb61a9e05412488e5858faec6de Mon Sep 17 00:00:00 2001 From: Ritesh Ghorse Date: Tue, 10 Oct 2023 16:15:11 -0400 Subject: [PATCH 56/85] fix lint (#28917) --- sdks/python/apache_beam/runners/worker/bundle_processor.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/worker/bundle_processor.py b/sdks/python/apache_beam/runners/worker/bundle_processor.py index c7fcb9587455..7ff0ad258bc2 100644 --- a/sdks/python/apache_beam/runners/worker/bundle_processor.py +++ b/sdks/python/apache_beam/runners/worker/bundle_processor.py @@ -233,7 +233,7 @@ def process_encoded(self, encoded_windowed_values): except Exception as exn: raise ValueError( "Error decoding input stream with coder " + - self.windowed_coder) from exn + str(self.windowed_coder)) from exn self.output(decoded_value) def monitoring_infos(self, transform_id, tag_to_pcollection_id): From c62a81b1c57e771875aaf4415aca76cf9e4c2783 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 10 Oct 2023 15:51:13 -0700 Subject: [PATCH 57/85] Reapply "Populate top-level display data in yaml main. (#28512)" This reverts commit 3844972d681a317c1281685edc5f6fc8783c8043. --- sdks/python/apache_beam/pipeline.py | 28 +++++++++++++++---- sdks/python/apache_beam/transforms/display.py | 23 +++++++++------ sdks/python/apache_beam/yaml/main.py | 17 ++++++----- 3 files changed, 47 insertions(+), 21 deletions(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 042b483d50f1..14177cd603d8 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -88,6 +88,7 @@ from apache_beam.transforms import ParDo from apache_beam.transforms import ptransform from apache_beam.transforms.display import DisplayData +from apache_beam.transforms.display import HasDisplayData from apache_beam.transforms.resources import merge_resource_hints from apache_beam.transforms.resources import resource_hints_from_options from apache_beam.transforms.sideinputs import get_sideinput_index @@ -108,7 +109,7 @@ __all__ = ['Pipeline', 'PTransformOverride'] -class Pipeline(object): +class Pipeline(HasDisplayData): """A pipeline object that manages a DAG of :class:`~apache_beam.pvalue.PValue` s and their :class:`~apache_beam.transforms.ptransform.PTransform` s. @@ -133,9 +134,12 @@ def runner_implemented_transforms(cls): common_urns.primitives.IMPULSE.urn, ]) - def __init__(self, runner=None, options=None, argv=None): - # type: (Optional[Union[str, PipelineRunner]], Optional[PipelineOptions], Optional[List[str]]) -> None - + def __init__( + self, + runner: Optional[Union[str, PipelineRunner]] = None, + options: Optional[PipelineOptions] = None, + argv: Optional[List[str]] = None, + display_data: Optional[Dict[str, Any]] = None): """Initialize a pipeline object. Args: @@ -151,6 +155,8 @@ def __init__(self, runner=None, options=None, argv=None): to be used for building a :class:`~apache_beam.options.pipeline_options.PipelineOptions` object. This will only be used if argument **options** is :data:`None`. + display_data (Dict[str: Any]): a dictionary of static data associated + with this pipeline that can be displayed when it runs. Raises: ValueError: if either the runner or options argument is not @@ -233,6 +239,11 @@ def __init__(self, runner=None, options=None, argv=None): # Records whether this pipeline contains any external transforms. self.contains_external_transforms = False + self._display_data = display_data or {} + + def display_data(self): + # type: () -> Dict[str, Any] + return self._display_data @property # type: ignore[misc] # decorated property not supported def options(self): @@ -914,7 +925,8 @@ def visit_transform(self, transform_node): proto = beam_runner_api_pb2.Pipeline( root_transform_ids=[root_transform_id], components=context.to_runner_api(), - requirements=context.requirements()) + requirements=context.requirements(), + display_data=DisplayData('', self._display_data).to_proto()) proto.components.transforms[root_transform_id].unique_name = ( root_transform_id) self.merge_compatible_environments(proto) @@ -970,7 +982,11 @@ def from_runner_api( # type: (...) -> Pipeline """For internal use only; no backwards-compatibility guarantees.""" - p = Pipeline(runner=runner, options=options) + p = Pipeline( + runner=runner, + options=options, + display_data={str(ix): d + for ix, d in enumerate(proto.display_data)}) from apache_beam.runners import pipeline_context context = pipeline_context.PipelineContext( proto.components, requirements=proto.requirements) diff --git a/sdks/python/apache_beam/transforms/display.py b/sdks/python/apache_beam/transforms/display.py index b52a8fd5b6dd..0d1dd552413e 100644 --- a/sdks/python/apache_beam/transforms/display.py +++ b/sdks/python/apache_beam/transforms/display.py @@ -45,6 +45,7 @@ from datetime import timedelta from typing import TYPE_CHECKING from typing import List +from typing import Union from apache_beam.portability import common_urns from apache_beam.portability.api import beam_runner_api_pb2 @@ -101,7 +102,8 @@ def __init__( ): # type: (...) -> None self.namespace = namespace - self.items = [] # type: List[DisplayDataItem] + self.items = [ + ] # type: List[Union[DisplayDataItem, beam_runner_api_pb2.DisplayData]] self._populate_items(display_data_dict) def _populate_items(self, display_data_dict): @@ -112,26 +114,31 @@ def _populate_items(self, display_data_dict): subcomponent_display_data = DisplayData( element._get_display_data_namespace(), element.display_data()) self.items += subcomponent_display_data.items - continue - if isinstance(element, DisplayDataItem): + elif isinstance(element, DisplayDataItem): if element.should_drop(): continue element.key = key element.namespace = self.namespace self.items.append(element) - continue - # If it's not a HasDisplayData element, - # nor a dictionary, then it's a simple value - self.items.append( - DisplayDataItem(element, namespace=self.namespace, key=key)) + elif isinstance(element, beam_runner_api_pb2.DisplayData): + self.items.append(element) + + else: + # If it's not a HasDisplayData element, + # nor a dictionary, then it's a simple value + self.items.append( + DisplayDataItem(element, namespace=self.namespace, key=key)) def to_proto(self): # type: (...) -> List[beam_runner_api_pb2.DisplayData] """Returns a List of Beam proto representation of Display data.""" def create_payload(dd): + if isinstance(dd, beam_runner_api_pb2.DisplayData): + return dd + display_data_dict = None try: display_data_dict = dd.get_dict() diff --git a/sdks/python/apache_beam/yaml/main.py b/sdks/python/apache_beam/yaml/main.py index eb0695f337b4..e2ec8df9cfc3 100644 --- a/sdks/python/apache_beam/yaml/main.py +++ b/sdks/python/apache_beam/yaml/main.py @@ -51,19 +51,22 @@ def _pipeline_spec_from_args(known_args): raise ValueError( "Exactly one of pipeline_spec or pipeline_spec_file must be set.") - return yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) + return pipeline_yaml def run(argv=None): yaml_transform._LOGGER.setLevel('INFO') known_args, pipeline_args = _configure_parser(argv) - pipeline_spec = _pipeline_spec_from_args(known_args) + pipeline_yaml = _pipeline_spec_from_args(known_args) + pipeline_spec = yaml.load(pipeline_yaml, Loader=yaml_transform.SafeLineLoader) - with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( - pipeline_args, - pickle_library='cloudpickle', - **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get( - 'options', {})))) as p: + with beam.Pipeline( # linebreak for better yapf formatting + options=beam.options.pipeline_options.PipelineOptions( + pipeline_args, + pickle_library='cloudpickle', + **yaml_transform.SafeLineLoader.strip_metadata(pipeline_spec.get( + 'options', {}))), + display_data={'yaml': pipeline_yaml}) as p: print("Building pipeline...") yaml_transform.expand_pipeline(p, pipeline_spec) print("Running pipeline...") From de48383bdca60f72d95c5889b281eed3be0fb5a6 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Tue, 10 Oct 2023 17:41:22 -0700 Subject: [PATCH 58/85] Fix typo in docstring. --- sdks/python/apache_beam/pipeline.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 14177cd603d8..f52616307e7b 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -155,7 +155,7 @@ def __init__( to be used for building a :class:`~apache_beam.options.pipeline_options.PipelineOptions` object. This will only be used if argument **options** is :data:`None`. - display_data (Dict[str: Any]): a dictionary of static data associated + display_data (Dict[str, Any]): a dictionary of static data associated with this pipeline that can be displayed when it runs. Raises: From 99c87a2c68b09020d9d5fa40da18d432b501c39a Mon Sep 17 00:00:00 2001 From: Chamikara Jayalath Date: Tue, 10 Oct 2023 21:09:58 -0700 Subject: [PATCH 59/85] Updates the Transform Service to accept Python extra packages through the Java API (#28783) * Updates the Transform Service to accept Python extra packages through the Java API * Addressing reviewer comments * Addressing reviewer comments --- build.gradle.kts | 2 + .../core/construction/TransformUpgrader.java | 2 +- .../python/PythonExternalTransform.java | 39 ++-- .../transform-service/docker-compose/.env | 8 + .../docker-compose/docker-compose.yml | 3 +- .../transform-service/launcher/build.gradle | 3 + .../launcher/TransformServiceLauncher.java | 135 +++++++++++-- .../TransformServiceLauncherTest.java | 185 ++++++++++++++++++ .../transformservice/ExpansionService.java | 114 ++++++++++- .../ExpansionServiceTest.java | 7 +- .../utils/transform_service_launcher.py | 14 ++ .../expansion-service-container/boot.go | 102 +++++++++- 12 files changed, 566 insertions(+), 48 deletions(-) create mode 100644 sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java diff --git a/build.gradle.kts b/build.gradle.kts index fbea1a59b282..ea1b4e6784e3 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -310,6 +310,8 @@ tasks.register("javaPreCommit") { dependsOn(":sdks:java:testing:test-utils:build") dependsOn(":sdks:java:testing:tpcds:build") dependsOn(":sdks:java:testing:watermarks:build") + dependsOn(":sdks:java:transform-service:build") + dependsOn(":sdks:java:transform-service:launcher:build") dependsOn(":examples:java:preCommit") dependsOn(":examples:java:twitter:preCommit") diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java index d657bb31b184..5e1609f27a39 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/TransformUpgrader.java @@ -108,7 +108,7 @@ public RunnerApi.Pipeline upgradeTransformsViaTransformService( } else if (options.getTransformServiceBeamVersion() != null) { String projectName = UUID.randomUUID().toString(); int port = findAvailablePort(); - service = TransformServiceLauncher.forProject(projectName, port); + service = TransformServiceLauncher.forProject(projectName, port, null); service.setBeamVersion(options.getTransformServiceBeamVersion()); // Starting the transform service. diff --git a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java index 4a5f4f12a07a..5ba3484964c1 100644 --- a/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java +++ b/sdks/java/extensions/python/src/main/java/org/apache/beam/sdk/extensions/python/PythonExternalTransform.java @@ -495,6 +495,20 @@ public OutputT expand(InputT input) { boolean pythonAvailable = isPythonAvailable(); boolean dockerAvailable = isDockerAvailable(); + File requirementsFile = null; + if (!extraPackages.isEmpty()) { + requirementsFile = File.createTempFile("requirements", ".txt"); + requirementsFile.deleteOnExit(); + try (Writer fout = + new OutputStreamWriter( + new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + for (String pkg : extraPackages) { + fout.write(pkg); + fout.write('\n'); + } + } + } + // We use the transform service if either of the following is true. // * It was explicitly requested. // * Python executable is not available in the system but Docker is available. @@ -514,19 +528,16 @@ public OutputT expand(InputT input) { projectName, port); - TransformServiceLauncher service = TransformServiceLauncher.forProject(projectName, port); + String pythonRequirementsFile = + requirementsFile != null ? requirementsFile.getAbsolutePath() : null; + TransformServiceLauncher service = + TransformServiceLauncher.forProject(projectName, port, pythonRequirementsFile); service.setBeamVersion(ReleaseInfo.getReleaseInfo().getSdkVersion()); - // TODO(https://github.com/apache/beam/issues/26833): add support for installing extra - // packages. - if (!extraPackages.isEmpty()) { - throw new RuntimeException( - "Transform Service does not support installing extra packages yet"); - } try { // Starting the transform service. service.start(); // Waiting the service to be ready. - service.waitTillUp(15000); + service.waitTillUp(-1); // Expanding the transform. output = apply(input, String.format("localhost:%s", port), payload); } finally { @@ -539,17 +550,7 @@ public OutputT expand(InputT input) { ImmutableList.Builder args = ImmutableList.builder(); args.add( "--port=" + port, "--fully_qualified_name_glob=*", "--pickle_library=cloudpickle"); - if (!extraPackages.isEmpty()) { - File requirementsFile = File.createTempFile("requirements", ".txt"); - requirementsFile.deleteOnExit(); - try (Writer fout = - new OutputStreamWriter( - new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { - for (String pkg : extraPackages) { - fout.write(pkg); - fout.write('\n'); - } - } + if (requirementsFile != null) { args.add("--requirements_file=" + requirementsFile.getAbsolutePath()); } PythonService service = diff --git a/sdks/java/transform-service/docker-compose/.env b/sdks/java/transform-service/docker-compose/.env index 5de5982cfa30..ed27b267fed3 100644 --- a/sdks/java/transform-service/docker-compose/.env +++ b/sdks/java/transform-service/docker-compose/.env @@ -12,6 +12,14 @@ BEAM_VERSION=$BEAM_VERSION CREDENTIALS_VOLUME=$CREDENTIALS_VOLUME +DEPENDENCIES_VOLUME=$DEPENDENCIES_VOLUME + +# A requirements file with either of the following +# * PyPi packages +# * Locally available packages relative to the directory provided to +# DEPENDENCIES_VOLUME. +PYTHON_REQUIREMENTS_FILE_NAME=$PYTHON_REQUIREMENTS_FILE_NAME + GOOGLE_APPLICATION_CREDENTIALS_FILE_NAME=application_default_credentials.json COMPOSE_PROJECT_NAME=apache.beam.transform.service TRANSFORM_SERVICE_PORT=$TRANSFORM_SERVICE_PORT diff --git a/sdks/java/transform-service/docker-compose/docker-compose.yml b/sdks/java/transform-service/docker-compose/docker-compose.yml index b685be10a329..39235533b9a8 100644 --- a/sdks/java/transform-service/docker-compose/docker-compose.yml +++ b/sdks/java/transform-service/docker-compose/docker-compose.yml @@ -32,8 +32,9 @@ services: expansion-service-2: image: "apache/beam_python_expansion_service:${BEAM_VERSION}" restart: on-failure - command: -id expansion-service-2 -port 5001 + command: -id expansion-service-2 -port 5001 -requirements_file ${PYTHON_REQUIREMENTS_FILE_NAME} -dependencies_dir '/dependencies_volume' volumes: - ${CREDENTIALS_VOLUME}:/credentials_volume + - ${DEPENDENCIES_VOLUME}:/dependencies_volume environment: - GOOGLE_APPLICATION_CREDENTIALS=/credentials_volume/${GOOGLE_APPLICATION_CREDENTIALS_FILE_NAME} diff --git a/sdks/java/transform-service/launcher/build.gradle b/sdks/java/transform-service/launcher/build.gradle index 83c5d60a1ef1..0952f37109eb 100644 --- a/sdks/java/transform-service/launcher/build.gradle +++ b/sdks/java/transform-service/launcher/build.gradle @@ -45,6 +45,9 @@ dependencies { shadow library.java.args4j shadow library.java.error_prone_annotations permitUnusedDeclared(library.java.error_prone_annotations) + testImplementation library.java.junit + testImplementation library.java.mockito_core + testImplementation project(path: ":sdks:java:core") } sourceSets { diff --git a/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java b/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java index f52fdfed710d..c0a9097a762f 100644 --- a/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java +++ b/sdks/java/transform-service/launcher/src/main/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncher.java @@ -17,9 +17,11 @@ */ package org.apache.beam.sdk.transformservice.launcher; +import java.io.BufferedWriter; import java.io.File; import java.io.FileOutputStream; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.nio.file.Paths; import java.util.ArrayList; @@ -28,6 +30,7 @@ import java.util.Locale; import java.util.Map; import java.util.concurrent.TimeoutException; +import java.util.stream.Stream; 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.io.ByteStreams; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.Files; @@ -62,9 +65,9 @@ public class TransformServiceLauncher { private static final int STATUS_LOGGER_WAIT_TIME = 3000; @SuppressWarnings("argument") - private TransformServiceLauncher(@Nullable String projectName, int port) throws IOException { - LOG.info("Initializing the Beam Transform Service {}.", projectName); - + private TransformServiceLauncher( + @Nullable String projectName, int port, @Nullable String pythonRequirementsFile) + throws IOException { String tmpDirLocation = System.getProperty("java.io.tmpdir"); // We use Docker Compose project name as the name of the temporary directory to isolate // different transform service instances that may be running in the same machine. @@ -83,14 +86,14 @@ private TransformServiceLauncher(@Nullable String projectName, int port) throws ByteStreams.copy(getClass().getResourceAsStream("/.env"), fout); } + // Setting up the credentials directory. File credentialsDir = Paths.get(tmpDir, "credentials_dir").toFile(); - LOG.info( - "Creating a temporary directory for storing credentials: " - + credentialsDir.getAbsolutePath()); - if (credentialsDir.exists()) { LOG.info("Reusing the existing credentials directory " + credentialsDir.getAbsolutePath()); } else { + LOG.info( + "Creating a temporary directory for storing credentials: " + + credentialsDir.getAbsolutePath()); if (!credentialsDir.mkdir()) { throw new IOException( "Could not create a temporary directory for storing credentials: " @@ -124,10 +127,84 @@ private TransformServiceLauncher(@Nullable String projectName, int port) throws } } + // Setting up the dependencies directory. + File dependenciesDir = Paths.get(tmpDir, "dependencies_dir").toFile(); + Path updatedRequirementsFilePath = Paths.get(dependenciesDir.toString(), "requirements.txt"); + if (dependenciesDir.exists()) { + LOG.info("Reusing the existing dependencies directory " + dependenciesDir.getAbsolutePath()); + } else { + LOG.info( + "Creating a temporary directory for storing dependencies: " + + dependenciesDir.getAbsolutePath()); + if (!dependenciesDir.mkdir()) { + throw new IOException( + "Could not create a temporary directory for storing dependencies: " + + dependenciesDir.getAbsolutePath()); + } + + // We create a requirements file with extra dependencies. + // If there are no extra dependencies, we just provide an empty requirements file. + File file = updatedRequirementsFilePath.toFile(); + if (!file.createNewFile()) { + throw new IOException( + "Could not create the new requirements file " + updatedRequirementsFilePath); + } + + // Updating dependencies. + if (pythonRequirementsFile != null) { + Path requirementsFilePath = Paths.get(pythonRequirementsFile); + List updatedLines = new ArrayList<>(); + + try (Stream lines = java.nio.file.Files.lines(requirementsFilePath)) { + lines.forEachOrdered( + line -> { + Path dependencyFilePath = Paths.get(line); + if (java.nio.file.Files.exists(dependencyFilePath)) { + Path fileName = dependencyFilePath.getFileName(); + if (fileName == null) { + throw new IllegalArgumentException( + "Could not determine the filename of the local artifact " + + dependencyFilePath); + } + try { + java.nio.file.Files.copy( + dependencyFilePath, + Paths.get(dependenciesDir.toString(), fileName.toString())); + } catch (IOException e) { + throw new RuntimeException(e); + } + updatedLines.add(fileName.toString()); + } else { + updatedLines.add(line); + } + }); + } + + try (BufferedWriter writer = + java.nio.file.Files.newBufferedWriter(file.toPath(), StandardCharsets.UTF_8)) { + for (String line : updatedLines) { + writer.write(line); + writer.newLine(); + } + writer.flush(); + } + } + } + // Setting environment variables used by the docker-compose.yml file. environmentVariables.put("CREDENTIALS_VOLUME", credentialsDir.getAbsolutePath()); + environmentVariables.put("DEPENDENCIES_VOLUME", dependenciesDir.getAbsolutePath()); environmentVariables.put("TRANSFORM_SERVICE_PORT", String.valueOf(port)); + Path updatedRequirementsFileName = updatedRequirementsFilePath.getFileName(); + if (updatedRequirementsFileName == null) { + throw new IllegalArgumentException( + "Could not determine the file name of the updated requirements file " + + updatedRequirementsFilePath); + } + environmentVariables.put( + "PYTHON_REQUIREMENTS_FILE_NAME", updatedRequirementsFileName.toString()); + // Building the Docker Compose command. dockerComposeStartCommandPrefix.add("docker-compose"); dockerComposeStartCommandPrefix.add("-p"); @@ -136,21 +213,37 @@ private TransformServiceLauncher(@Nullable String projectName, int port) throws dockerComposeStartCommandPrefix.add(dockerComposeFile.getAbsolutePath()); } + /** + * Specifies the Beam version to get containers for the transform service. + * + *

Could be a release Beam version with containers in Docker Hub or an unreleased Beam version + * for which containers are available locally. + * + * @param beamVersion a Beam version to get containers from. + */ public void setBeamVersion(String beamVersion) { environmentVariables.put("BEAM_VERSION", beamVersion); } - public void setPythonExtraPackages(String pythonExtraPackages) { - environmentVariables.put("$PYTHON_EXTRA_PACKAGES", pythonExtraPackages); - } - + /** + * Initializes a client for managing transform service instances. + * + * @param projectName project name for the transform service. + * @param port port exposed by the transform service. + * @param pythonRequirementsFile a requirements file with extra dependencies for the Python + * expansion services. + * @return an initialized client for managing the transform service. + * @throws IOException + */ public static synchronized TransformServiceLauncher forProject( - @Nullable String projectName, int port) throws IOException { + @Nullable String projectName, int port, @Nullable String pythonRequirementsFile) + throws IOException { if (projectName == null || projectName.isEmpty()) { projectName = DEFAULT_PROJECT_NAME; } if (!launchers.containsKey(projectName)) { - launchers.put(projectName, new TransformServiceLauncher(projectName, port)); + launchers.put( + projectName, new TransformServiceLauncher(projectName, port, pythonRequirementsFile)); } return launchers.get(projectName); } @@ -200,10 +293,10 @@ public synchronized void status() throws IOException { public synchronized void waitTillUp(int timeout) throws IOException, TimeoutException { timeout = timeout <= 0 ? DEFAULT_START_WAIT_TIME : timeout; - String statusFileName = getStatus(); long startTime = System.currentTimeMillis(); while (System.currentTimeMillis() - startTime < timeout) { + String statusFileName = getStatus(); try { // We are just waiting for a local process. No need for exponential backoff. this.wait(1000); @@ -226,6 +319,7 @@ public synchronized void waitTillUp(int timeout) throws IOException, TimeoutExce private synchronized String getStatus() throws IOException { File outputOverride = File.createTempFile("output_override", null); + outputOverride.deleteOnExit(); runDockerComposeCommand(ImmutableList.of("ps"), outputOverride); return outputOverride.getAbsolutePath(); @@ -238,6 +332,8 @@ private static class ArgConfig { static final String PORT_ARG_NAME = "port"; static final String BEAM_VERSION_ARG_NAME = "beam_version"; + static final String PYTHON_REQUIREMENTS_FILE_ARG_NAME = "python_requirements_file"; + @Option(name = "--" + PROJECT_NAME_ARG_NAME, usage = "Docker compose project name") private String projectName = ""; @@ -249,6 +345,11 @@ private static class ArgConfig { @Option(name = "--" + BEAM_VERSION_ARG_NAME, usage = "Beam version to use.") private String beamVersion = ""; + + @Option( + name = "--" + PYTHON_REQUIREMENTS_FILE_ARG_NAME, + usage = "Extra Python packages in the form of an requirements file.") + private String pythonRequirementsFile = ""; } public static void main(String[] args) throws IOException, TimeoutException { @@ -288,8 +389,12 @@ public static void main(String[] args) throws IOException, TimeoutException { : ("port " + Integer.toString(config.port) + "."))); System.out.println("==================================================="); + String pythonRequirementsFile = + !config.pythonRequirementsFile.isEmpty() ? config.pythonRequirementsFile : null; + TransformServiceLauncher service = - TransformServiceLauncher.forProject(config.projectName, config.port); + TransformServiceLauncher.forProject( + config.projectName, config.port, pythonRequirementsFile); if (!config.beamVersion.isEmpty()) { service.setBeamVersion(config.beamVersion); } diff --git a/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java b/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java new file mode 100644 index 000000000000..4ef84b02061b --- /dev/null +++ b/sdks/java/transform-service/launcher/src/test/java/org/apache/beam/sdk/transformservice/launcher/TransformServiceLauncherTest.java @@ -0,0 +1,185 @@ +/* + * 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.transformservice.launcher; + +import static java.nio.charset.StandardCharsets.UTF_8; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.io.Writer; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.UUID; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Charsets; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +@RunWith(JUnit4.class) +public class TransformServiceLauncherTest { + + @Test + public void testLauncherCreatesCredentialsDir() throws IOException { + String projectName = UUID.randomUUID().toString(); + Path expectedTempDir = Paths.get(System.getProperty("java.io.tmpdir"), projectName); + File file = expectedTempDir.toFile(); + file.deleteOnExit(); + TransformServiceLauncher.forProject(projectName, 12345, null); + Path expectedCredentialsDir = Paths.get(expectedTempDir.toString(), "credentials_dir"); + assertTrue(expectedCredentialsDir.toFile().exists()); + } + + @Test + public void testLauncherCreatesDependenciesDir() throws IOException { + String projectName = UUID.randomUUID().toString(); + Path expectedTempDir = Paths.get(System.getProperty("java.io.tmpdir"), projectName); + File file = expectedTempDir.toFile(); + file.deleteOnExit(); + TransformServiceLauncher.forProject(projectName, 12345, null); + Path expectedCredentialsDir = Paths.get(expectedTempDir.toString(), "dependencies_dir"); + assertTrue(expectedCredentialsDir.toFile().exists()); + } + + @Test + public void testLauncherInstallsDependencies() throws IOException { + String projectName = UUID.randomUUID().toString(); + Path expectedTempDir = Paths.get(System.getProperty("java.io.tmpdir"), projectName); + File file = expectedTempDir.toFile(); + file.deleteOnExit(); + + File requirementsFile = + Paths.get( + System.getProperty("java.io.tmpdir"), + ("requirements" + UUID.randomUUID().toString() + ".txt")) + .toFile(); + requirementsFile.deleteOnExit(); + + try (Writer fout = + new OutputStreamWriter( + new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + fout.write("pypipackage1\n"); + fout.write("pypipackage2\n"); + } + + TransformServiceLauncher.forProject(projectName, 12345, requirementsFile.getAbsolutePath()); + + // Confirming that the Transform Service launcher created a temporary requirements file with the + // specified set of packages. + Path expectedUpdatedRequirementsFile = + Paths.get(expectedTempDir.toString(), "dependencies_dir", "requirements.txt"); + assertTrue(expectedUpdatedRequirementsFile.toFile().exists()); + + ArrayList expectedUpdatedRequirementsFileLines = new ArrayList<>(); + try (BufferedReader bufReader = + Files.newBufferedReader(expectedUpdatedRequirementsFile, UTF_8)) { + String line = bufReader.readLine(); + while (line != null) { + expectedUpdatedRequirementsFileLines.add(line); + line = bufReader.readLine(); + } + } + + assertEquals(2, expectedUpdatedRequirementsFileLines.size()); + assertTrue(expectedUpdatedRequirementsFileLines.contains("pypipackage1")); + assertTrue(expectedUpdatedRequirementsFileLines.contains("pypipackage2")); + } + + @Test + public void testLauncherInstallsLocalDependencies() throws IOException { + String projectName = UUID.randomUUID().toString(); + Path expectedTempDir = Paths.get(System.getProperty("java.io.tmpdir"), projectName); + File file = expectedTempDir.toFile(); + file.deleteOnExit(); + + String dependency1FileName = "dep_" + UUID.randomUUID().toString(); + File dependency1 = + Paths.get(System.getProperty("java.io.tmpdir"), dependency1FileName).toFile(); + dependency1.deleteOnExit(); + try (Writer fout = + new OutputStreamWriter( + new FileOutputStream(dependency1.getAbsolutePath()), Charsets.UTF_8)) { + fout.write("tempdata\n"); + } + + String dependency2FileName = "dep_" + UUID.randomUUID().toString(); + File dependency2 = + Paths.get(System.getProperty("java.io.tmpdir"), dependency2FileName).toFile(); + dependency2.deleteOnExit(); + try (Writer fout = + new OutputStreamWriter( + new FileOutputStream(dependency2.getAbsolutePath()), Charsets.UTF_8)) { + fout.write("tempdata\n"); + } + + File requirementsFile = + Paths.get( + System.getProperty("java.io.tmpdir"), + ("requirements" + UUID.randomUUID().toString() + ".txt")) + .toFile(); + requirementsFile.deleteOnExit(); + try (Writer fout = + new OutputStreamWriter( + new FileOutputStream(requirementsFile.getAbsolutePath()), Charsets.UTF_8)) { + fout.write(dependency1.getAbsolutePath() + "\n"); + fout.write(dependency2.getAbsolutePath() + "\n"); + fout.write("pypipackage" + "\n"); + } + + TransformServiceLauncher.forProject(projectName, 12345, requirementsFile.getAbsolutePath()); + + // Confirming that the Transform Service launcher created a temporary requirements file with the + // specified set of packages. + Path expectedUpdatedRequirementsFile = + Paths.get(expectedTempDir.toString(), "dependencies_dir", "requirements.txt"); + assertTrue(expectedUpdatedRequirementsFile.toFile().exists()); + + ArrayList expectedUpdatedRequirementsFileLines = new ArrayList<>(); + try (BufferedReader bufReader = + Files.newBufferedReader(expectedUpdatedRequirementsFile, UTF_8)) { + String line = bufReader.readLine(); + while (line != null) { + expectedUpdatedRequirementsFileLines.add(line); + line = bufReader.readLine(); + } + } + + // To make local packages available to the expansion service Docker containers, the temporary + // requirements file should contain names of the local packages relative to the dependencies + // volume and local packages should have been copied to the dependencies volume. + assertEquals(3, expectedUpdatedRequirementsFileLines.size()); + assertTrue(expectedUpdatedRequirementsFileLines.contains(dependency1FileName)); + assertTrue(expectedUpdatedRequirementsFileLines.contains(dependency2FileName)); + assertTrue(expectedUpdatedRequirementsFileLines.contains("pypipackage")); + + assertTrue( + Paths.get(expectedTempDir.toString(), "dependencies_dir", dependency1FileName) + .toFile() + .exists()); + assertTrue( + Paths.get(expectedTempDir.toString(), "dependencies_dir", dependency2FileName) + .toFile() + .exists()); + } +} diff --git a/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java b/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java index 17fe5472f9fc..0a2e65099e7d 100644 --- a/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java +++ b/sdks/java/transform-service/src/main/java/org/apache/beam/sdk/transformservice/ExpansionService.java @@ -17,15 +17,22 @@ */ package org.apache.beam.sdk.transformservice; +import java.io.IOException; +import java.net.Socket; import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; +import java.util.concurrent.TimeoutException; import org.apache.beam.model.expansion.v1.ExpansionApi; +import org.apache.beam.model.expansion.v1.ExpansionApi.ExpansionResponse; import org.apache.beam.model.expansion.v1.ExpansionServiceGrpc; import org.apache.beam.model.pipeline.v1.Endpoints; import org.apache.beam.runners.core.construction.DefaultExpansionServiceClientFactory; import org.apache.beam.runners.core.construction.ExpansionServiceClientFactory; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.ManagedChannelBuilder; import org.apache.beam.vendor.grpc.v1p54p0.io.grpc.stub.StreamObserver; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; import org.checkerframework.checker.nullness.qual.Nullable; @@ -40,6 +47,12 @@ public class ExpansionService extends ExpansionServiceGrpc.ExpansionServiceImplB final List endpoints; + private boolean checkedAllServices = false; + + private static final long SERVICE_CHECK_TIMEOUT_MILLIS = 60000; + + private boolean disableServiceCheck = false; + ExpansionService( List endpoints, @Nullable ExpansionServiceClientFactory clientFactory) { @@ -48,10 +61,65 @@ public class ExpansionService extends ExpansionServiceGrpc.ExpansionServiceImplB clientFactory != null ? clientFactory : DEFAULT_EXPANSION_SERVICE_CLIENT_FACTORY; } + // Waits till all expansion services are ready. + private void waitForAllServicesToBeReady() throws TimeoutException { + if (disableServiceCheck) { + // Service check disabled. Just returning. + return; + } + + outer: + for (Endpoints.ApiServiceDescriptor endpoint : endpoints) { + long start = System.currentTimeMillis(); + long duration = 10; + while (System.currentTimeMillis() - start < SERVICE_CHECK_TIMEOUT_MILLIS) { + try { + String url = endpoint.getUrl(); + int portIndex = url.lastIndexOf(":"); + if (portIndex <= 0) { + throw new RuntimeException( + "Expected the endpoint to be of the form : but received " + url); + } + int port = Integer.parseInt(url.substring(portIndex + 1)); + String host = url.substring(0, portIndex); + new Socket(host, port).close(); + // Current service is up. Checking the next one. + continue outer; + } catch (IOException exn) { + try { + Thread.sleep(duration); + } catch (InterruptedException e) { + // Ignore + } + duration = (long) (duration * 1.2); + } + } + throw new TimeoutException( + "Timeout waiting for the service " + + endpoint.getUrl() + + " to startup after " + + (System.currentTimeMillis() - start) + + " milliseconds."); + } + } + + @VisibleForTesting + void disableServiceCheck() { + disableServiceCheck = true; + } + @Override public void expand( ExpansionApi.ExpansionRequest request, StreamObserver responseObserver) { + if (!checkedAllServices) { + try { + waitForAllServicesToBeReady(); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } + checkedAllServices = true; + } try { responseObserver.onNext(processExpand(request)); responseObserver.onCompleted(); @@ -68,6 +136,14 @@ public void expand( public void discoverSchemaTransform( ExpansionApi.DiscoverSchemaTransformRequest request, StreamObserver responseObserver) { + if (!checkedAllServices) { + try { + waitForAllServicesToBeReady(); + } catch (TimeoutException e) { + throw new RuntimeException(e); + } + checkedAllServices = true; + } try { responseObserver.onNext(processDiscover(request)); responseObserver.onCompleted(); @@ -80,18 +156,41 @@ public void discoverSchemaTransform( } } - /*package*/ ExpansionApi.ExpansionResponse processExpand(ExpansionApi.ExpansionRequest request) { + private ExpansionApi.ExpansionResponse getAggregatedErrorResponse( + Map errorResponses) { + StringBuilder errorMessageBuilder = new StringBuilder(); + + errorMessageBuilder.append( + "Aggregated errors from " + errorResponses.size() + " expansion services." + "\n"); + for (Map.Entry entry : errorResponses.entrySet()) { + errorMessageBuilder.append( + "Error from expansion service " + + entry.getKey() + + ": " + + entry.getValue().getError() + + "\n"); + } + + return errorResponses + .values() + .iterator() + .next() + .toBuilder() + .setError(errorMessageBuilder.toString()) + .build(); + } + + ExpansionApi.ExpansionResponse processExpand(ExpansionApi.ExpansionRequest request) { // Trying out expansion services in order till one succeeds. // If all services fail, re-raises the last error. - // TODO: when all services fail, return an aggregated error with errors from all services. - ExpansionApi.ExpansionResponse lastErrorResponse = null; + Map errorResponses = new HashMap<>(); RuntimeException lastException = null; for (Endpoints.ApiServiceDescriptor endpoint : endpoints) { try { ExpansionApi.ExpansionResponse response = expansionServiceClientFactory.getExpansionServiceClient(endpoint).expand(request); if (!response.getError().isEmpty()) { - lastErrorResponse = response; + errorResponses.put(endpoint.getUrl(), response); continue; } return response; @@ -99,8 +198,11 @@ public void discoverSchemaTransform( lastException = e; } } - if (lastErrorResponse != null) { - return lastErrorResponse; + if (lastException != null) { + throw new RuntimeException("Expansion request to transform service failed.", lastException); + } + if (!errorResponses.isEmpty()) { + return getAggregatedErrorResponse(errorResponses); } else if (lastException != null) { throw new RuntimeException("Expansion request to transform service failed.", lastException); } else { diff --git a/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java b/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java index 298bce87f901..9905abd1d9ba 100644 --- a/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java +++ b/sdks/java/transform-service/src/test/java/org/apache/beam/sdk/transformservice/ExpansionServiceTest.java @@ -60,6 +60,8 @@ public void setUp() throws Exception { endpoints.add(endpoint2); clientFactory = Mockito.mock(ExpansionServiceClientFactory.class); expansionService = new ExpansionService(endpoints, clientFactory); + // We do not run actual services in unit tests. + expansionService.disableServiceCheck(); } @Test @@ -131,7 +133,10 @@ public void testExpandFail() { ArgumentCaptor expansionResponseCapture = ArgumentCaptor.forClass(ExpansionResponse.class); Mockito.verify(responseObserver).onNext(expansionResponseCapture.capture()); - assertEquals("expansion error 2", expansionResponseCapture.getValue().getError()); + + // Error response should contain errors from both expansion services. + assertTrue(expansionResponseCapture.getValue().getError().contains("expansion error 1")); + assertTrue(expansionResponseCapture.getValue().getError().contains("expansion error 2")); } @Test diff --git a/sdks/python/apache_beam/utils/transform_service_launcher.py b/sdks/python/apache_beam/utils/transform_service_launcher.py index 33feab9bf29c..ac492513aba5 100644 --- a/sdks/python/apache_beam/utils/transform_service_launcher.py +++ b/sdks/python/apache_beam/utils/transform_service_launcher.py @@ -86,6 +86,7 @@ def __init__(self, project_name, port, beam_version=None): compose_file = os.path.join(temp_dir, 'docker-compose.yml') + # Creating the credentials volume. credentials_dir = os.path.join(temp_dir, 'credentials_dir') if not os.path.exists(credentials_dir): os.mkdir(credentials_dir) @@ -111,11 +112,24 @@ def __init__(self, project_name, port, beam_version=None): 'credentials file at the expected location %s.' % application_default_path_file) + # Creating the dependencies volume. + dependencies_dir = os.path.join(temp_dir, 'dependencies_dir') + if not os.path.exists(dependencies_dir): + os.mkdir(dependencies_dir) + self._environmental_variables = {} self._environmental_variables['CREDENTIALS_VOLUME'] = credentials_dir + self._environmental_variables['DEPENDENCIES_VOLUME'] = dependencies_dir self._environmental_variables['TRANSFORM_SERVICE_PORT'] = str(port) self._environmental_variables['BEAM_VERSION'] = beam_version + # Setting an empty requirements file + requirements_file_name = os.path.join(dependencies_dir, 'requirements.txt') + with open(requirements_file_name, 'w') as _: + pass + self._environmental_variables['PYTHON_REQUIREMENTS_FILE_NAME'] = ( + 'requirements.txt') + self._docker_compose_start_command_prefix = [] self._docker_compose_start_command_prefix.append('docker-compose') self._docker_compose_start_command_prefix.append('-p') diff --git a/sdks/python/expansion-service-container/boot.go b/sdks/python/expansion-service-container/boot.go index 90a97c35425a..ba56b349c4ea 100644 --- a/sdks/python/expansion-service-container/boot.go +++ b/sdks/python/expansion-service-container/boot.go @@ -18,8 +18,10 @@ package main import ( + "bufio" "flag" "fmt" + "io/ioutil" "log" "os" "path/filepath" @@ -31,16 +33,15 @@ import ( ) var ( - id = flag.String("id", "", "Local identifier (required)") - port = flag.Int("port", 0, "Port for the expansion service (required)") + id = flag.String("id", "", "Local identifier (required)") + port = flag.Int("port", 0, "Port for the expansion service (required)") + requirements_file = flag.String("requirements_file", "", "A requirement file with extra packages to be made available to the transforms being expanded. Path should be relative to the 'dependencies_dir'") + dependencies_dir = flag.String("dependencies_dir", "", "A directory that stores locally available extra packages.") ) const ( expansionServiceEntrypoint = "apache_beam.runners.portability.expansion_service_main" venvDirectory = "beam_venv" // This should match the venv directory name used in the Dockerfile. - requirementsFile = "requirements.txt" - beamSDKArtifact = "apache-beam-sdk.tar.gz" - beamSDKOptions = "[gcp,dataframe]" ) func main() { @@ -58,6 +59,79 @@ func main() { } } +func getLines(fileNameToRead string) ([]string, error) { + fileToRead, err := os.Open(fileNameToRead) + if err != nil { + return nil, err + } + defer fileToRead.Close() + + sc := bufio.NewScanner(fileToRead) + lines := make([]string, 0) + + // Read through 'tokens' until an EOF is encountered. + for sc.Scan() { + lines = append(lines, sc.Text()) + } + + if err := sc.Err(); err != nil { + return nil, err + } + return lines, nil +} + +func installExtraPackages(requirementsFile string) error { + extraPackages, err := getLines(requirementsFile) + if err != nil { + return err + } + + for _, extraPackage := range extraPackages { + log.Printf("Installing extra package %v", extraPackage) + // We expect 'pip' command in virtual env to be already available at the top of the PATH. + args := []string{"install", extraPackage} + if err := execx.Execute("pip", args...); err != nil { + return fmt.Errorf("Could not install the package %s: %s", extraPackage, err) + } + } + return nil +} + +func getUpdatedRequirementsFile(oldRequirementsFileName string, dependenciesDir string) (string, error) { + oldExtraPackages, err := getLines(filepath.Join(dependenciesDir, oldRequirementsFileName)) + if err != nil { + return "", err + } + var updatedExtraPackages = make([]string, 0) + for _, extraPackage := range oldExtraPackages { + // TODO update + potentialLocalFilePath := filepath.Join(dependenciesDir, extraPackage) + _, err := os.Stat(potentialLocalFilePath) + if err == nil { + // Package exists locally so using that. + extraPackage = potentialLocalFilePath + log.Printf("Using locally available extra package %v", extraPackage) + } + updatedExtraPackages = append(updatedExtraPackages, extraPackage) + } + + updatedRequirementsFile, err := ioutil.TempFile("/opt/apache/beam", "requirements*.txt") + if err != nil { + return "", err + } + + updatedRequirementsFileName := updatedRequirementsFile.Name() + + datawriter := bufio.NewWriter(updatedRequirementsFile) + for _, extraPackage := range updatedExtraPackages { + _, _ = datawriter.WriteString(extraPackage + "\n") + } + datawriter.Flush() + updatedRequirementsFile.Close() + + return updatedRequirementsFileName, nil +} + func launchExpansionServiceProcess() error { pythonVersion, err := expansionx.GetPythonVersion() if err != nil { @@ -70,6 +144,24 @@ func launchExpansionServiceProcess() error { os.Setenv("PATH", strings.Join([]string{filepath.Join(dir, "bin"), os.Getenv("PATH")}, ":")) args := []string{"-m", expansionServiceEntrypoint, "-p", strconv.Itoa(*port), "--fully_qualified_name_glob", "*"} + + if *requirements_file != "" { + log.Printf("Received the requirements file %v", *requirements_file) + updatedRequirementsFileName, err := getUpdatedRequirementsFile(*requirements_file, *dependencies_dir) + if err != nil { + return err + } + defer os.Remove(updatedRequirementsFileName) + log.Printf("Updated requirements file is %v", updatedRequirementsFileName) + // Provide the requirements file to the expansion service so that packages get staged by runners. + args = append(args, "--requirements_file", updatedRequirementsFileName) + // Install packages locally so that they can be used by the expansion service during transform + // expansion if needed. + err = installExtraPackages(updatedRequirementsFileName) + if err != nil { + return err + } + } if err := execx.Execute(pythonVersion, args...); err != nil { return fmt.Errorf("could not start the expansion service: %s", err) } From 88139bdc9a04b039d58d2669ee85914223ef9bd6 Mon Sep 17 00:00:00 2001 From: Andrey Devyatkin Date: Wed, 11 Oct 2023 14:05:35 +0200 Subject: [PATCH 60/85] Retrieve workflow runs for GA Post-Commits Status dashboard (#28844) * Fixed the GitHub Actions postcommit dashboard * retrieve workflow runs from GitHub API * fixed throttling and out of memory issues and excluded 'skipped' runs --- .test-infra/metrics/docker-compose.yml | 5 +- .../metrics/sync/github/requirements.txt | 6 +- .../metrics/sync/github/sync_workflows.py | 400 +++++++++++------- 3 files changed, 250 insertions(+), 161 deletions(-) diff --git a/.test-infra/metrics/docker-compose.yml b/.test-infra/metrics/docker-compose.yml index 77f07bad13fa..3d847ff79676 100644 --- a/.test-infra/metrics/docker-compose.yml +++ b/.test-infra/metrics/docker-compose.yml @@ -85,7 +85,10 @@ services: - DB_DBNAME=beam_metrics - DB_DBUSERNAME=admin - DB_DBPWD= - - GH_ACCESS_TOKEN= + - GH_APP_ID= + - GH_APP_INSTALLATION_ID= + - GH_PEM_KEY= + - GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH=30 syncjenkins: image: syncjenkins container_name: beamsyncjenkins diff --git a/.test-infra/metrics/sync/github/requirements.txt b/.test-infra/metrics/sync/github/requirements.txt index 14a64686e55a..5b231565459f 100644 --- a/.test-infra/metrics/sync/github/requirements.txt +++ b/.test-infra/metrics/sync/github/requirements.txt @@ -13,7 +13,7 @@ # See the License for the specific language governing permissions and # limitations under the License. -requests +aiohttp +backoff psycopg2-binary -ddt - +PyGithub \ No newline at end of file diff --git a/.test-infra/metrics/sync/github/sync_workflows.py b/.test-infra/metrics/sync/github/sync_workflows.py index 0b4ddfec3802..25065878c4bd 100644 --- a/.test-infra/metrics/sync/github/sync_workflows.py +++ b/.test-infra/metrics/sync/github/sync_workflows.py @@ -1,4 +1,3 @@ -# # 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. @@ -6,180 +5,267 @@ # (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 +# 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. + ''' -This module queries GitHub to collect Beam-related workflows metrics and put them in -PostgreSQL. -This Script is running every 3 hours in a cloud function in apache-beam-testing project. -This cloud function is triggered by a pubsub topic. -You can find the cloud function in the next link +This module queries GitHub API to collect Beam-related workflows metrics and +put them in PostgreSQL. +This script is running every 3 hours as a cloud function +"github_actions_workflows_dashboard_sync" in apache-beam-testing project: https://console.cloud.google.com/functions/details/us-central1/github_actions_workflows_dashboard_sync?env=gen1&project=apache-beam-testing -Pub sub topic : https://console.cloud.google.com/cloudpubsub/topic/detail/github_actions_workflows_sync?project=apache-beam-testing -Cron Job : https://console.cloud.google.com/cloudscheduler/jobs/edit/us-central1/github_actions_workflows_dashboard_sync?project=apache-beam-testing -Writing the latest 10 runs of every workflow in master branch in a beammetrics database +This cloud function is triggered by a pubsub topic: +https://console.cloud.google.com/cloudpubsub/topic/detail/github_actions_workflows_sync?project=apache-beam-testing +Cron Job: +https://console.cloud.google.com/cloudscheduler/jobs/edit/us-central1/github_actions_workflows_dashboard_sync?project=apache-beam-testing ''' +import asyncio +import aiohttp +import backoff +import math import os import sys import time import re -import requests import psycopg2 - -from datetime import datetime -from github import GithubIntegration +from github import GithubIntegration DB_HOST = os.environ['DB_HOST'] DB_PORT = os.environ['DB_PORT'] DB_NAME = os.environ['DB_DBNAME'] DB_USER_NAME = os.environ['DB_DBUSERNAME'] DB_PASSWORD = os.environ['DB_DBPWD'] -GH_WORKFLOWS_TABLE_NAME = "github_workflows" -# Number of workflows that fetch github API -GH_NUMBER_OF_WORKFLOWS = 100 -GH_WORKFLOWS_NUMBER_EXECUTIONS = 100 -WORKFLOWS_OBJECT_LIST = [] +GH_APP_ID = os.environ['GH_APP_ID'] +GH_APP_INSTALLATION_ID = os.environ['GH_APP_INSTALLATION_ID'] +GH_PEM_KEY = os.environ['GH_PEM_KEY'] +GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH =\ + os.environ['GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH'] class Workflow: - def __init__(self,id,name,filename): - self.id = id - self.name = name - self.filename = filename - self.listOfRuns = [] - self.runUrl = [] - -# The table will save the latest ten run of every workflow -GH_WORKFLOWS_CREATE_TABLE_QUERY = f""" -CREATE TABLE IF NOT EXISTS {GH_WORKFLOWS_TABLE_NAME} ( - job_name text PRIMARY KEY, - job_yml_filename text""" -for i in range(0,GH_WORKFLOWS_NUMBER_EXECUTIONS): - i = i + 1 - GH_WORKFLOWS_CREATE_TABLE_QUERY += """,\n run{} text, - run{}Id text""".format(str(i),str(i)) -GH_WORKFLOWS_CREATE_TABLE_QUERY += ")\n" - -def githubWorkflowsGrafanaSync(data,context): - print('Started') - print('Updating table with recent workflow runs') - databaseOperations(initDbConnection(),fetchWorkflowData()) - print('Done') - return "Completed" - -def initDbConnection(): - '''Init connection with the Database''' - connection = None - maxRetries = 3 - i = 0 - while connection == None and i < maxRetries: - try: - connection = psycopg2.connect( - f"dbname='{DB_NAME}' user='{DB_USER_NAME}' host='{DB_HOST}'" - f" port='{DB_PORT}' password='{DB_PASSWORD}'") - except Exception as e: - print('Failed to connect to DB; retrying in 1 minute') - print(e) - time.sleep(60) - i = i + 1 - if i >= maxRetries: - print("Number of retries exceded ") - sys.exit(1) - return connection - -def getToken(): - git_integration = GithubIntegration( - os.environ["GH_APP_ID"], - os.environ["GH_PEM_KEY"]) - token=git_integration.get_access_token( - os.environ["GH_APP_INSTALLATION_ID"] - ).token - return token - -def retriesRequest(request): - requestSucceeded = False - retryFactor = 1 - while not requestSucceeded: - retryTime = 60 * retryFactor - if request.status_code != 200: - print('Failed to get the request with code {}'.format(request.status_code)) - time.sleep(retryTime) - retryFactor = retryFactor + retryFactor - if retryFactor * 60 >= 3600: - print("Error: The request take more than an hour") - sys.exit(1) - else: - requestSucceeded = True -def fetchWorkflowData(): - '''Return a json with all the workflows and the latests - ten executions''' - completed = False - page = 1 - workflows = [] + def __init__(self, id, name, filename): + self.id = id + self.name = name + self.filename = filename + self.runs = [] + +async def github_workflows_dashboard_sync(): + print('Started') + print('Updating table with recent workflow runs') + + if not GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH or \ + not GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH.isdigit(): + raise ValueError( + 'The number of workflow runs to fetch is not specified or not an integer' + ) + + database_operations(init_db_connection(), await fetch_workflow_data()) + + print('Done') + return "Completed" + +def init_db_connection(): + '''Init connection with the Database''' + connection = None + maxRetries = 3 + i = 0 + while connection is None and i < maxRetries: try: - while not completed: - url = "https://api.github.com/repos/apache/beam/actions/workflows" - queryOptions = { 'branch' : 'master', 'page': page, 'per_page' : GH_NUMBER_OF_WORKFLOWS } - response = requests.get(url = url, params = queryOptions) - retriesRequest(response) - jsonResponse = response.json() - if jsonResponse['total_count'] >= GH_NUMBER_OF_WORKFLOWS: - page = page + 1 - workflowsPage = jsonResponse['workflows'] - workflows.append(workflowsPage) - else: - completed = True - workflowsPage = jsonResponse['workflows'] - workflows.append(workflowsPage) - for pageItem in workflows: - for item in pageItem: - path = item['path'] - result = re.search('/(.*).yml', path) - path = (result.group(1)) + ".yml" - workflowObject = Workflow(item['id'],item['name'],path) - WORKFLOWS_OBJECT_LIST.append(workflowObject) - url = "https://api.github.com/repos/apache/beam/actions/workflows/" - queryOptions = { 'branch' : 'master', 'per_page' : GH_WORKFLOWS_NUMBER_EXECUTIONS, - 'page' :'1', 'exclude_pull_request':True } - for workflow in WORKFLOWS_OBJECT_LIST: - response = requests.get(url = "{}{}/runs".format(url,workflow.id), - params=queryOptions) - retriesRequest(response) - responseJson = response.json() - workflowsRuns = responseJson['workflow_runs'] - for item in workflowsRuns: - if item['status'] == 'completed': - workflow.runUrl.append(item['html_url']) - workflow.listOfRuns.append(item['conclusion']) - elif item['status'] != 'cancelled': - workflow.listOfRuns.append(item['status']) - workflow.runUrl.append(item['html_url']) - for i in range(0,GH_WORKFLOWS_NUMBER_EXECUTIONS): - if i >= len(workflow.listOfRuns): - workflow.listOfRuns.append('None') - workflow.runUrl.append('None') + connection = psycopg2.connect( + f"dbname='{DB_NAME}' user='{DB_USER_NAME}' host='{DB_HOST}'" + f" port='{DB_PORT}' password='{DB_PASSWORD}'") except Exception as e: - print('Failed to get GHA workflows') - print(e) - -def databaseOperations(connection,fetchWorkflows): - '''Create the table if not exist and update the table with the latest runs - of the workflows ''' - queryInsert = "INSERT INTO {} VALUES ".format(GH_WORKFLOWS_TABLE_NAME) - cursor = connection.cursor() - cursor.execute(GH_WORKFLOWS_CREATE_TABLE_QUERY) - cursor.execute("DELETE FROM {};".format(GH_WORKFLOWS_TABLE_NAME)) - query = "" - for workflow in WORKFLOWS_OBJECT_LIST: - rowInsert = "(\'{}\',\'{}\'".format(workflow.name,workflow.filename) - for run, runUrl in zip(workflow.listOfRuns,workflow.runUrl): - rowInsert += ",\'{}\',\'{}\'".format(run,runUrl) - query = query + rowInsert - query += ")," - query = query[:-1] + ";" - query = queryInsert + query - cursor.execute(query) - cursor.close() - connection.commit() - connection.close() + print('Failed to connect to DB; retrying in 1 minute') + print(e) + time.sleep(60) + i = i + 1 + if i >= maxRetries: + print("Number of retries exceded ") + sys.exit(1) + return connection + +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: + async with aiohttp.ClientSession() as session: + async with session.get(url, params=params, headers=headers) as response: + if response.status == 200: + result = await response.json() + if request_id: + return request_id, result + return result + elif response.status == 403: + print(f'Retry for: {url}') + headers['Authorization'] = get_token() + raise aiohttp.ClientResponseError( + response.request_info, + response.history, + status=response.status, + message=response.reason, + headers=response.headers + ) + +async def fetch_workflow_data(): + def append_workflow_runs(workflow, runs): + for run in runs: + # Getting rid of all runs with a "skipped" status to display + # only actual runs + if run['conclusion'] != 'skipped': + status = '' + if run['status'] == 'completed': + status = run['conclusion'] + elif run['status'] != 'cancelled': + status = run['status'] + workflow.runs.append((int(run['id']), status, run['html_url'])) + + url = "https://api.github.com/repos/apache/beam/actions/workflows" + headers = {'Authorization': get_token()} + page = 1 + number_of_entries_per_page = 100 # The number of results per page (max 100) + params =\ + {'branch': 'master', 'page': page, 'per_page': number_of_entries_per_page} + concurrent_requests = 30 # Number of requests to send simultaneously + semaphore = asyncio.Semaphore(concurrent_requests) + + print("Start fetching recent workflow runs") + workflow_tasks = [] + response = await fetch(url, semaphore, params, headers) + pages_to_fetch =\ + math.ceil(response['total_count'] / number_of_entries_per_page) + while pages_to_fetch >= page: + params = { + 'branch': 'master', + 'page': page, + 'per_page': number_of_entries_per_page + } + workflow_tasks.append(fetch(url, semaphore, params, headers)) + page += 1 + + workflow_run_tasks = [] + for completed_task in asyncio.as_completed(workflow_tasks): + response = await completed_task + workflows = response.get('workflows', []) + for workflow in workflows: + runs_url = f"{url}/{workflow['id']}/runs" + page = 1 + pages_to_fetch = math.ceil( + int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH) / number_of_entries_per_page + ) + while pages_to_fetch >= page: + params = { + 'branch': 'master', + 'page': page, + 'per_page': number_of_entries_per_page, + 'exclude_pull_requests': 'true' + } + workflow_run_tasks.append(fetch(runs_url, semaphore, params, headers)) + page += 1 + print("Successfully fetched workflow runs") + + print("Start fetching workflow runs details") + workflows = {} + workflow_ids_to_fetch_extra_runs = {} + for completed_task in asyncio.as_completed(workflow_run_tasks): + response = await completed_task + workflow_runs = response.get('workflow_runs') + if workflow_runs: + workflow_id = workflow_runs[0]['workflow_id'] + workflow = workflows.get(workflow_id) + if not workflow: + workflow_name = workflow_runs[0]['name'] + workflow_path = workflow_runs[0]['path'] + result = re.search(r'(workflows\/.*)$', workflow_path) + if result: + workflow_path = result.group(1) + workflow = Workflow(workflow_id, workflow_name, workflow_path) + + append_workflow_runs(workflow, workflow_runs) + workflows[workflow_id] = workflow + if len(workflow.runs) < int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH): + workflow_ids_to_fetch_extra_runs[workflow_id] = workflow_id + 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 + # Fetch extra workflow runs if the specified number of runs is not reached + while workflow_ids_to_fetch_extra_runs: + extra_workflow_runs_tasks = [] + for workflow_id in list(workflow_ids_to_fetch_extra_runs.values()): + runs_url = f"{url}/{workflow_id}/runs" + params = { + 'branch': 'master', + 'page': page, + 'per_page': number_of_entries_per_page, + 'exclude_pull_requests': 'true' + } + extra_workflow_runs_tasks.append(fetch(runs_url, semaphore, params, headers, workflow_id)) + for completed_task in asyncio.as_completed(extra_workflow_runs_tasks): + workflow_id, response = await completed_task + workflow = workflows[workflow_id] + print(f"Fetching extra workflow runs for: {workflow.filename}") + workflow_runs = response.get('workflow_runs') + if workflow_runs: + append_workflow_runs(workflow, workflow_runs) + else: + number_of_runs_to_add =\ + int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH) - len(workflow.runs) + workflow.runs.extend([(0, 'None', 'None')] * number_of_runs_to_add) + if len(workflow.runs) >= int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH): + workflow_ids_to_fetch_extra_runs.pop(workflow_id, None) + print(f"Successfully fetched extra workflow runs for: {workflow.filename}") + page += 1 + print("Successfully fetched workflow runs details") + + for workflow in list(workflows.values()): + runs = sorted(workflow.runs, key=lambda r: r[0], reverse=True) + workflow.runs = runs[:int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH)] + + return list(workflows.values()) + +def database_operations(connection, workflows): + # Create the table and update it with the latest workflow runs + if not workflows: + return + cursor = connection.cursor() + workflows_table_name = "github_workflows" + cursor.execute(f"DROP TABLE IF EXISTS {workflows_table_name};") + create_table_query = f""" + CREATE TABLE IF NOT EXISTS {workflows_table_name} ( + workflow_id integer NOT NULL PRIMARY KEY, + job_name text NOT NULL, + job_yml_filename text NOT NULL""" + for i in range(int(GH_NUMBER_OF_WORKFLOW_RUNS_TO_FETCH)): + create_table_query += f""", + run{i+1} text, + run{i+1}Id text""" + create_table_query += ")\n" + cursor.execute(create_table_query) + insert_query = f"INSERT INTO {workflows_table_name} VALUES " + for workflow in workflows: + row_insert =\ + f"(\'{workflow.id}\',\'{workflow.name}\',\'{workflow.filename}\'" + for _, status, url in workflow.runs: + row_insert += f",\'{status}\',\'{url}\'" + insert_query += f"{row_insert})," + insert_query = insert_query[:-1] + ";" + cursor.execute(insert_query) + cursor.close() + connection.commit() + connection.close() + +if __name__ == '__main__': + asyncio.run(github_workflows_dashboard_sync()) From e3c1591bbb8c2b91313267fcd39d77720c768b75 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 11 Oct 2023 09:27:52 -0400 Subject: [PATCH 61/85] Bump google.golang.org/grpc from 1.58.2 to 1.58.3 in /sdks (#28938) Bumps [google.golang.org/grpc](https://github.com/grpc/grpc-go) from 1.58.2 to 1.58.3. - [Release notes](https://github.com/grpc/grpc-go/releases) - [Commits](https://github.com/grpc/grpc-go/compare/v1.58.2...v1.58.3) --- updated-dependencies: - dependency-name: google.golang.org/grpc 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 | 2 +- sdks/go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 0bc60b630bf0..761798d57c9b 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -59,7 +59,7 @@ require ( golang.org/x/text v0.13.0 google.golang.org/api v0.146.0 google.golang.org/genproto v0.0.0-20230913181813-007df8e322eb - google.golang.org/grpc v1.58.2 + google.golang.org/grpc v1.58.3 google.golang.org/protobuf v1.31.0 gopkg.in/retry.v1 v1.0.3 gopkg.in/yaml.v2 v2.4.0 diff --git a/sdks/go.sum b/sdks/go.sum index 7743b2bc3247..ac862972b2ae 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -712,8 +712,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.58.2 h1:SXUpjxeVF3FKrTYQI4f4KvbGD5u2xccdYdurwowix5I= -google.golang.org/grpc v1.58.2/go.mod h1:tgX3ZQDlNJGU96V6yHh1T/JeoBQ2TXdr43YbYSsCJk0= +google.golang.org/grpc v1.58.3 h1:BjnpXut1btbtgN/6sp+brB2Kbm2LjNXnidYujAVbSoQ= +google.golang.org/grpc v1.58.3/go.mod h1:tgX3ZQDlNJGU96V6yHh1T/JeoBQ2TXdr43YbYSsCJk0= 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= From 62efb96788455d8b8cddbe9e60335821bf735f65 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 11 Oct 2023 09:55:42 -0400 Subject: [PATCH 62/85] Bump golang.org/x/net from 0.16.0 to 0.17.0 in /sdks (#28939) Bumps [golang.org/x/net](https://github.com/golang/net) from 0.16.0 to 0.17.0. - [Commits](https://github.com/golang/net/compare/v0.16.0...v0.17.0) --- updated-dependencies: - dependency-name: golang.org/x/net 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 761798d57c9b..e5baa554fe6d 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -52,7 +52,7 @@ 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.12.1 - golang.org/x/net v0.16.0 + golang.org/x/net v0.17.0 golang.org/x/oauth2 v0.13.0 golang.org/x/sync v0.4.0 golang.org/x/sys v0.13.0 diff --git a/sdks/go.sum b/sdks/go.sum index ac862972b2ae..f1d0c53d9685 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -541,8 +541,8 @@ golang.org/x/net v0.0.0-20201224014010-6772e930b67b/go.mod h1:m0MpNAwzfU5UDzcl9v 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.16.0 h1:7eBu7KsSvFDtSXUIDbh3aqlK4DPsZ1rByC8PFfBThos= -golang.org/x/net v0.16.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= +golang.org/x/net v0.17.0 h1:pVaXccu2ozPjCXewfr1S7xza/zcXTity9cCdXQYSjIM= +golang.org/x/net v0.17.0/go.mod h1:NxSsAGuq816PNPmqtQdLE42eU2Fs7NoRIZrHJAlaCOE= 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= From d437afab9615f2d3983da68cdb86a98209375ba6 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 11 Oct 2023 10:10:19 -0400 Subject: [PATCH 63/85] Run self-assign and pr-updates only in create event (#28928) --- .github/workflows/pr-bot-pr-updates.yml | 1 + .github/workflows/self-assign.yml | 2 +- 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/pr-bot-pr-updates.yml b/.github/workflows/pr-bot-pr-updates.yml index c882c18feeba..f3734e0adcbf 100644 --- a/.github/workflows/pr-bot-pr-updates.yml +++ b/.github/workflows/pr-bot-pr-updates.yml @@ -18,6 +18,7 @@ on: pull_request_target: types: ["synchronize"] # Synchronize is the action that runs after pushes to the user branch issue_comment: + types: [created] permissions: read-all jobs: process-pr-update: diff --git a/.github/workflows/self-assign.yml b/.github/workflows/self-assign.yml index c6b7cc69ce97..29ad240cd0dd 100644 --- a/.github/workflows/self-assign.yml +++ b/.github/workflows/self-assign.yml @@ -16,7 +16,7 @@ name: Assign or close an issue on: issue_comment: - + types: [created] jobs: assign: permissions: From a54afde4a05769f88826459168c9fa17c9bb00ae Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Wed, 11 Oct 2023 10:15:46 -0400 Subject: [PATCH 64/85] Fix missing backslash in arguments of job (#28942) This is causing repeated failures (e.g. https://github.com/apache/beam/actions/runs/6478627017/job/17590709062) --- ..._PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml index 18b57aad8057..44df23f0296e 100644 --- a/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml +++ b/.github/workflows/beam_PostCommit_Python_ValidatesContainer_Dataflow_With_RC.yml @@ -87,10 +87,10 @@ jobs: with: gradle-command: :sdks:python:test-suites:dataflow:py${{steps.set_py_ver_clean.outputs.py_ver_clean}}:validatesContainer arguments: | - -PtestRCDependencies=true + -PtestRCDependencies=true \ -PpythonVersion=${{ matrix.python_version }} \ - name: Archive code coverage results uses: actions/upload-artifact@v3 with: name: python-code-coverage-report - path: "**/pytest*.xml" \ No newline at end of file + path: "**/pytest*.xml" From e56d1a1157bf4ea12f85a0a13f3f863d31ddff0e Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Wed, 11 Oct 2023 10:49:18 -0400 Subject: [PATCH 65/85] Enable missing-timeout pylint check (#28920) --- sdks/python/.pylintrc | 1 - .../portability/flink_uber_jar_job_server.py | 4 ++-- .../testing/analyzers/github_issues_utils.py | 20 +++++++++++++++---- .../benchmarks/nexmark/nexmark_launcher.py | 3 ++- .../load_tests/load_test_metrics_utils.py | 3 ++- 5 files changed, 22 insertions(+), 9 deletions(-) diff --git a/sdks/python/.pylintrc b/sdks/python/.pylintrc index e649e07e5ef6..a67f00ff2f82 100644 --- a/sdks/python/.pylintrc +++ b/sdks/python/.pylintrc @@ -116,7 +116,6 @@ disable = missing-docstring, modified-iterating-list, multiple-statements, - missing-timeout, #TODO(https://github.com/apache/beam/issues/28240) Enable and fix warnings no-self-use, no-else-break, no-else-continue, diff --git a/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py b/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py index 9a40a55c7601..3b302e334a5f 100644 --- a/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py +++ b/sdks/python/apache_beam/runners/portability/flink_uber_jar_job_server.py @@ -80,8 +80,8 @@ def executable_jar(self): return job_server.JavaJarJobServer.local_jar(url) def flink_version(self): - full_version = requests.get('%s/v1/config' % - self._master_url).json()['flink-version'] + full_version = requests.get( + '%s/v1/config' % self._master_url, timeout=60).json()['flink-version'] # Only return up to minor version. return '.'.join(full_version.split('.')[:2]) diff --git a/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py b/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py index bbcd2a8b11b5..cbbb9e5d3a2e 100644 --- a/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py +++ b/sdks/python/apache_beam/testing/analyzers/github_issues_utils.py @@ -61,6 +61,8 @@ _AWAITING_TRIAGE_LABEL = 'awaiting triage' _PERF_ALERT_LABEL = 'perf-alert' +_REQUEST_TIMEOUT_SECS = 60 + def create_issue( title: str, @@ -89,7 +91,10 @@ def create_issue( if labels: data['labels'].extend(labels) # type: ignore response = requests.post( - url=url, data=json.dumps(data), headers=_HEADERS).json() + url=url, + data=json.dumps(data), + headers=_HEADERS, + timeout=_REQUEST_TIMEOUT_SECS).json() return response['number'], response['html_url'] @@ -118,7 +123,8 @@ def comment_on_issue(issue_number: int, 'issue_number': issue_number }, default=str), - headers=_HEADERS).json() + headers=_HEADERS, + timeout=_REQUEST_TIMEOUT_SECS).json() if open_issue_response['state'] == 'open': data = { 'owner': _GITHUB_REPO_OWNER, @@ -128,7 +134,10 @@ def comment_on_issue(issue_number: int, } response = requests.post( - open_issue_response['comments_url'], json.dumps(data), headers=_HEADERS) + open_issue_response['comments_url'], + json.dumps(data), + headers=_HEADERS, + timeout=_REQUEST_TIMEOUT_SECS) return True, response.json()['html_url'] return False, '' @@ -137,7 +146,10 @@ def add_awaiting_triage_label(issue_number: int): url = 'https://api.github.com/repos/{}/{}/issues/{}/labels'.format( _GITHUB_REPO_OWNER, _GITHUB_REPO_NAME, issue_number) requests.post( - url, json.dumps({'labels': [_AWAITING_TRIAGE_LABEL]}), headers=_HEADERS) + url, + json.dumps({'labels': [_AWAITING_TRIAGE_LABEL]}), + headers=_HEADERS, + timeout=_REQUEST_TIMEOUT_SECS) def get_issue_description( diff --git a/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py index 2296588ec496..bdf6f476212d 100644 --- a/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py +++ b/sdks/python/apache_beam/testing/benchmarks/nexmark/nexmark_launcher.py @@ -420,7 +420,8 @@ def publish_performance_influxdb(self, query_num, perf): auth = HTTPBasicAuth(user, password) try: - response = requests.post(url, params=query_str, data=payload, auth=auth) + response = requests.post( + url, params=query_str, data=payload, auth=auth, timeout=60) except requests.exceptions.RequestException as e: logging.warning('Failed to publish metrics to InfluxDB: ' + str(e)) else: diff --git a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py index 01db2c114efb..1ff46a3f7d19 100644 --- a/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py +++ b/sdks/python/apache_beam/testing/load_tests/load_test_metrics_utils.py @@ -535,7 +535,8 @@ def publish(self, results): self.options.http_auth_enabled() else None try: - response = requests.post(url, params=query_str, data=payload, auth=auth) + response = requests.post( + url, params=query_str, data=payload, auth=auth, timeout=60) except requests.exceptions.RequestException as e: _LOGGER.warning('Failed to publish metrics to InfluxDB: ' + str(e)) else: From 9cdcb036cc904885254e8a58b75b32354f23ed6f Mon Sep 17 00:00:00 2001 From: martin trieu Date: Wed, 11 Oct 2023 08:28:49 -0700 Subject: [PATCH 66/85] =?UTF-8?q?move=20nested=20windmill=20state=20classe?= =?UTF-8?q?s=20to=20top=20level=20files=20and=20organize=20di=E2=80=A6=20(?= =?UTF-8?q?#28506)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../worker/StreamingDataflowWorker.java | 2 + .../worker/StreamingModeExecutionContext.java | 483 ++- .../runners/dataflow/worker/Weighers.java | 2 +- .../worker/WindmillStateInternals.java | 2830 ----------------- .../worker/streaming/ActiveWorkState.java | 2 +- .../worker/streaming/ComputationState.java | 2 +- .../windmill/state/CachingStateTable.java | 275 ++ .../windmill/state/ConcatIterables.java | 55 + .../worker/windmill/state/IdTracker.java | 253 ++ .../worker/windmill/state/PagingIterable.java | 129 + .../worker/windmill/state/RangeCoder.java | 76 + .../worker/windmill/state/RangeSetCoder.java | 46 + .../windmill/state/SimpleWindmillState.java | 42 + .../worker/windmill/state/StateTag.java | 109 + .../state/TimestampedValueWithId.java | 42 + .../windmill/state/ToIterableFunction.java | 79 + .../windmill/state/ValuesAndContPosition.java | 46 + .../worker/windmill/state/WeightedList.java | 54 + .../worker/windmill/state/WindmillBag.java | 209 ++ .../state/WindmillCombiningState.java | 170 + .../worker/windmill/state/WindmillMap.java | 449 +++ .../windmill/state/WindmillMultimap.java | 732 +++++ .../windmill/state/WindmillOrderedList.java | 305 ++ .../worker/windmill/state/WindmillSet.java | 146 + .../worker/windmill/state/WindmillState.java | 68 + .../state}/WindmillStateCache.java | 338 +- .../state/WindmillStateInternals.java | 171 + .../state}/WindmillStateReader.java | 530 +-- .../windmill/state/WindmillStateUtil.java | 51 + .../worker/windmill/state/WindmillValue.java | 160 + .../windmill/state/WindmillWatermarkHold.java | 259 ++ .../worker/windmill/state/WrappedFuture.java | 64 + .../StreamingModeExecutionContextTest.java | 2 + .../worker/streaming/ActiveWorkStateTest.java | 2 +- .../state}/WindmillStateCacheTest.java | 135 +- .../state}/WindmillStateInternalsTest.java | 282 +- .../state}/WindmillStateReaderTest.java | 6 +- 37 files changed, 4725 insertions(+), 3881 deletions(-) delete mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/CachingStateTable.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ConcatIterables.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/IdTracker.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/PagingIterable.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeCoder.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeSetCoder.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/SimpleWindmillState.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/StateTag.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/TimestampedValueWithId.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ToIterableFunction.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ValuesAndContPosition.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WeightedList.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillBag.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillCombiningState.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMap.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillOrderedList.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillSet.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillState.java rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/{ => windmill/state}/WindmillStateCache.java (92%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternals.java rename runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/{ => windmill/state}/WindmillStateReader.java (70%) create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateUtil.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillValue.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillWatermarkHold.java create mode 100644 runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WrappedFuture.java rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/{ => windmill/state}/WindmillStateCacheTest.java (75%) rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/{ => windmill/state}/WindmillStateInternalsTest.java (97%) rename runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/{ => windmill/state}/WindmillStateReaderTest.java (99%) 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 5d4c0288c838..32b272c67280 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 @@ -106,6 +106,8 @@ import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.CommitWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.WindmillStream.GetWorkStream; import org.apache.beam.runners.dataflow.worker.windmill.WindmillStreamPool; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.extensions.gcp.util.Transport; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java index 3f4cb0893755..c8fa6e6dfb78 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContext.java @@ -49,6 +49,9 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataId; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.GlobalDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.Timer; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateInternals; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.metrics.MetricsContainer; @@ -82,7 +85,12 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext { private static final Logger LOG = LoggerFactory.getLogger(StreamingModeExecutionContext.class); - + private final String computationId; + private final Map, Map> sideInputCache; + // Per-key cache of active Reader objects in use by this process. + private final ImmutableMap stateNameMap; + private final WindmillStateCache.ForComputation stateCache; + private final ReaderCache readerCache; /** * The current user-facing key for this execution context. * @@ -94,20 +102,12 @@ public class StreamingModeExecutionContext extends DataflowExecutionContext, Map> sideInputCache; - - // Per-key cache of active Reader objects in use by this process. - private final ImmutableMap stateNameMap; - private final WindmillStateCache.ForComputation stateCache; - private Windmill.WorkItem work; private WindmillComputationKey computationKey; private StateFetcher stateFetcher; private Windmill.WorkItemCommitRequest.Builder outputBuilder; private UnboundedSource.UnboundedReader activeReader; private volatile long backlogBytes; - private final ReaderCache readerCache; public StreamingModeExecutionContext( CounterFactory counterFactory, @@ -133,86 +133,6 @@ public StreamingModeExecutionContext( this.backlogBytes = UnboundedSource.UnboundedReader.BACKLOG_UNKNOWN; } - /** - * Execution states in Streaming are shared between multiple map-task executors. Thus this class - * needs to be thread safe for multiple writers. A single stage could have have multiple executors - * running concurrently. - */ - public static class StreamingModeExecutionState - extends DataflowOperationContext.DataflowExecutionState { - - // AtomicLong is used because this value is written in two places: - // 1. The sampling thread calls takeSample to increment the time spent in this state - // 2. The reporting thread calls extractUpdate which reads the current sum *AND* sets it to 0. - private final AtomicLong totalMillisInState = new AtomicLong(); - - // The worker that created this state. Used to report lulls back to the worker. - @SuppressWarnings("unused") // Affects a public api - private final StreamingDataflowWorker worker; - - public StreamingModeExecutionState( - NameContext nameContext, - String stateName, - MetricsContainer metricsContainer, - ProfileScope profileScope, - StreamingDataflowWorker worker) { - // TODO: Take in the requesting step name and side input index for streaming. - super(nameContext, stateName, null, null, metricsContainer, profileScope); - this.worker = worker; - } - - /** - * Take sample is only called by the ExecutionStateSampler thread. It is the only place that - * increments totalMillisInState, however the reporting thread periodically calls extractUpdate - * which will read the sum and reset it to 0, so totalMillisInState does have multiple writers. - */ - @Override - public void takeSample(long millisSinceLastSample) { - totalMillisInState.addAndGet(millisSinceLastSample); - } - - /** - * Extract updates in the form of a {@link CounterUpdate}. - * - *

Non-final updates are extracted periodically and report the physical value as a delta. - * This requires setting the totalMillisInState back to 0. - * - *

Final updates should never be requested from a Streaming job since the work unit never - * completes. - */ - @Override - public @Nullable CounterUpdate extractUpdate(boolean isFinalUpdate) { - // Streaming reports deltas, so isFinalUpdate doesn't matter, and should never be true. - long sum = totalMillisInState.getAndSet(0); - return sum == 0 ? null : createUpdate(false, sum); - } - } - - /** - * Implementation of DataflowExecutionStateRegistry that creates Streaming versions of - * ExecutionState. - */ - public static class StreamingModeExecutionStateRegistry extends DataflowExecutionStateRegistry { - - private final StreamingDataflowWorker worker; - - public StreamingModeExecutionStateRegistry(StreamingDataflowWorker worker) { - this.worker = worker; - } - - @Override - protected DataflowOperationContext.DataflowExecutionState createState( - NameContext nameContext, - String stateName, - String requestingStepName, - Integer inputIndex, - MetricsContainer container, - ProfileScope profileScope) { - return new StreamingModeExecutionState( - nameContext, stateName, container, profileScope, worker); - } - } - @VisibleForTesting public long getBacklogBytes() { return backlogBytes; @@ -304,11 +224,8 @@ protected SideInputReader getSideInputReaderForViews( String stateFamily, StateFetcher.SideInputState state, Supplier scopedReadStateSupplier) { - Map tagCache = sideInputCache.get(view.getTagInternal()); - if (tagCache == null) { - tagCache = new HashMap<>(); - sideInputCache.put(view.getTagInternal(), tagCache); - } + Map tagCache = + sideInputCache.computeIfAbsent(view.getTagInternal(), k -> new HashMap<>()); if (tagCache.containsKey(sideInputWindow)) { @SuppressWarnings("unchecked") @@ -455,6 +372,10 @@ public Map flushState() { return callbacks; } + String getStateFamily(NameContext nameContext) { + return nameContext.userName() == null ? null : stateNameMap.get(nameContext.userName()); + } + interface StreamingModeStepContext { boolean issueSideInputFetch( @@ -478,8 +399,84 @@ void writePCollectionViewData( throws IOException; } - String getStateFamily(NameContext nameContext) { - return nameContext.userName() == null ? null : stateNameMap.get(nameContext.userName()); + /** + * Execution states in Streaming are shared between multiple map-task executors. Thus this class + * needs to be thread safe for multiple writers. A single stage could have have multiple executors + * running concurrently. + */ + public static class StreamingModeExecutionState + extends DataflowOperationContext.DataflowExecutionState { + + // AtomicLong is used because this value is written in two places: + // 1. The sampling thread calls takeSample to increment the time spent in this state + // 2. The reporting thread calls extractUpdate which reads the current sum *AND* sets it to 0. + private final AtomicLong totalMillisInState = new AtomicLong(); + + // The worker that created this state. Used to report lulls back to the worker. + @SuppressWarnings("unused") // Affects a public api + private final StreamingDataflowWorker worker; + + public StreamingModeExecutionState( + NameContext nameContext, + String stateName, + MetricsContainer metricsContainer, + ProfileScope profileScope, + StreamingDataflowWorker worker) { + // TODO: Take in the requesting step name and side input index for streaming. + super(nameContext, stateName, null, null, metricsContainer, profileScope); + this.worker = worker; + } + + /** + * Take sample is only called by the ExecutionStateSampler thread. It is the only place that + * increments totalMillisInState, however the reporting thread periodically calls extractUpdate + * which will read the sum and reset it to 0, so totalMillisInState does have multiple writers. + */ + @Override + public void takeSample(long millisSinceLastSample) { + totalMillisInState.addAndGet(millisSinceLastSample); + } + + /** + * Extract updates in the form of a {@link CounterUpdate}. + * + *

Non-final updates are extracted periodically and report the physical value as a delta. + * This requires setting the totalMillisInState back to 0. + * + *

Final updates should never be requested from a Streaming job since the work unit never + * completes. + */ + @Override + public @Nullable CounterUpdate extractUpdate(boolean isFinalUpdate) { + // Streaming reports deltas, so isFinalUpdate doesn't matter, and should never be true. + long sum = totalMillisInState.getAndSet(0); + return sum == 0 ? null : createUpdate(false, sum); + } + } + + /** + * Implementation of DataflowExecutionStateRegistry that creates Streaming versions of + * ExecutionState. + */ + public static class StreamingModeExecutionStateRegistry extends DataflowExecutionStateRegistry { + + private final StreamingDataflowWorker worker; + + public StreamingModeExecutionStateRegistry(StreamingDataflowWorker worker) { + this.worker = worker; + } + + @Override + protected DataflowOperationContext.DataflowExecutionState createState( + NameContext nameContext, + String stateName, + String requestingStepName, + Integer inputIndex, + MetricsContainer container, + ProfileScope profileScope) { + return new StreamingModeExecutionState( + nameContext, stateName, container, profileScope, worker); + } } private static class ScopedReadStateSupplier implements Supplier { @@ -501,15 +498,156 @@ public Closeable get() { } } + /** + * A specialized {@link StepContext} that uses provided {@link StateInternals} and {@link + * TimerInternals} for user state and timers. + */ + private static class UserStepContext extends DataflowStepContext + implements StreamingModeStepContext { + + private final StreamingModeExecutionContext.StepContext wrapped; + + public UserStepContext(StreamingModeExecutionContext.StepContext wrapped) { + super(wrapped.getNameContext()); + this.wrapped = wrapped; + } + + @Override + public boolean issueSideInputFetch( + PCollectionView view, BoundedWindow w, StateFetcher.SideInputState s) { + return wrapped.issueSideInputFetch(view, w, s); + } + + @Override + public void addBlockingSideInput(GlobalDataRequest blocked) { + wrapped.addBlockingSideInput(blocked); + } + + @Override + public void addBlockingSideInputs(Iterable blocked) { + wrapped.addBlockingSideInputs(blocked); + } + + @Override + public StateInternals stateInternals() { + return wrapped.stateInternals(); + } + + @Override + public Iterable getSideInputNotifications() { + return wrapped.getSideInputNotifications(); + } + + @Override + public void writePCollectionViewData( + TupleTag tag, + Iterable data, + Coder> dataCoder, + W window, + Coder windowCoder) + throws IOException { + throw new IllegalStateException("User DoFns cannot write PCollectionView data"); + } + + @Override + public TimerInternals timerInternals() { + return wrapped.userTimerInternals(); + } + + @Override + public TimerData getNextFiredTimer(Coder windowCoder) { + return wrapped.getNextFiredUserTimer(windowCoder); + } + + @Override + public void setStateCleanupTimer( + String timerId, + W window, + Coder windowCoder, + Instant cleanupTime, + Instant cleanupOutputTimestamp) { + throw new UnsupportedOperationException( + String.format( + "setStateCleanupTimer should not be called on %s, only on a system %s", + getClass().getSimpleName(), + StreamingModeExecutionContext.StepContext.class.getSimpleName())); + } + + @Override + public DataflowStepContext namespacedToUser() { + return this; + } + } + + /** A {@link SideInputReader} that fetches side inputs from the streaming worker's cache. */ + public static class StreamingModeSideInputReader implements SideInputReader { + + private final StreamingModeExecutionContext context; + private final Set> viewSet; + + private StreamingModeSideInputReader( + Iterable> views, StreamingModeExecutionContext context) { + this.context = context; + this.viewSet = ImmutableSet.copyOf(views); + } + + public static StreamingModeSideInputReader of( + Iterable> views, StreamingModeExecutionContext context) { + return new StreamingModeSideInputReader(views, context); + } + + @Override + public T get(PCollectionView view, BoundedWindow window) { + if (!contains(view)) { + throw new RuntimeException("get() called with unknown view"); + } + + // We are only fetching the cached value here, so we don't need stateFamily or + // readStateSupplier. + return context + .fetchSideInput( + view, + window, + null /* unused stateFamily */, + StateFetcher.SideInputState.CACHED_IN_WORKITEM, + null /* unused readStateSupplier */) + .orNull(); + } + + @Override + public boolean contains(PCollectionView view) { + return viewSet.contains(view); + } + + @Override + public boolean isEmpty() { + return viewSet.isEmpty(); + } + } + class StepContext extends DataflowExecutionContext.DataflowStepContext implements StreamingModeStepContext { + private final String stateFamily; + private final Supplier scopedReadStateSupplier; private WindmillStateInternals stateInternals; - private WindmillTimerInternals systemTimerInternals; private WindmillTimerInternals userTimerInternals; - private final String stateFamily; - private final Supplier scopedReadStateSupplier; + // Lazily initialized + private Iterator cachedFiredSystemTimers = null; + // Lazily initialized + private PeekingIterator cachedFiredUserTimers = null; + // An ordered list of any timers that were set or modified by user processing earlier in this + // bundle. + // We use a NavigableSet instead of a priority queue to prevent duplicate elements from ending + // up in the queue. + private NavigableSet modifiedUserEventTimersOrdered = null; + private NavigableSet modifiedUserProcessingTimersOrdered = null; + private NavigableSet modifiedUserSynchronizedProcessingTimersOrdered = null; + // A list of timer keys that were modified by user processing earlier in this bundle. This + // serves a tombstone, so + // that we know not to fire any bundle tiemrs that were moddified. + private Table modifiedUserTimerKeys = null; public StepContext(DataflowOperationContext operationContext) { super(operationContext.nameContext()); @@ -570,14 +708,11 @@ public void flushState() { userTimerInternals.persistTo(outputBuilder); } - // Lazily initialized - private Iterator cachedFiredSystemTimers = null; - @Override public TimerData getNextFiredTimer(Coder windowCoder) { if (cachedFiredSystemTimers == null) { cachedFiredSystemTimers = - FluentIterable.from(StreamingModeExecutionContext.this.getFiredTimers()) + FluentIterable.from(StreamingModeExecutionContext.this.getFiredTimers()) .filter( timer -> WindmillTimerInternals.isSystemTimer(timer) @@ -601,16 +736,6 @@ public TimerData getNextFiredTimer(Coder windowCode return nextTimer; } - // Lazily initialized - private PeekingIterator cachedFiredUserTimers = null; - // An ordered list of any timers that were set or modified by user processing earlier in this - // bundle. - // We use a NavigableSet instead of a priority queue to prevent duplicate elements from ending - // up in the queue. - private NavigableSet modifiedUserEventTimersOrdered = null; - private NavigableSet modifiedUserProcessingTimersOrdered = null; - private NavigableSet modifiedUserSynchronizedProcessingTimersOrdered = null; - private NavigableSet getModifiedUserTimersOrdered(TimeDomain timeDomain) { switch (timeDomain) { case EVENT_TIME: @@ -624,11 +749,6 @@ private NavigableSet getModifiedUserTimersOrdered(TimeDomain timeDoma } } - // A list of timer keys that were modified by user processing earlier in this bundle. This - // serves a tombstone, so - // that we know not to fire any bundle tiemrs that were moddified. - private Table modifiedUserTimerKeys = null; - private void onUserTimerModified(TimerData timerData) { if (!timerData.getDeleted()) { getModifiedUserTimersOrdered(timerData.getDomain()).add(timerData); @@ -804,131 +924,4 @@ public TimerInternals userTimerInternals() { return checkNotNull(userTimerInternals); } } - - /** - * A specialized {@link StepContext} that uses provided {@link StateInternals} and {@link - * TimerInternals} for user state and timers. - */ - private static class UserStepContext extends DataflowStepContext - implements StreamingModeStepContext { - - private final StreamingModeExecutionContext.StepContext wrapped; - - public UserStepContext(StreamingModeExecutionContext.StepContext wrapped) { - super(wrapped.getNameContext()); - this.wrapped = wrapped; - } - - @Override - public boolean issueSideInputFetch( - PCollectionView view, BoundedWindow w, StateFetcher.SideInputState s) { - return wrapped.issueSideInputFetch(view, w, s); - } - - @Override - public void addBlockingSideInput(GlobalDataRequest blocked) { - wrapped.addBlockingSideInput(blocked); - } - - @Override - public void addBlockingSideInputs(Iterable blocked) { - wrapped.addBlockingSideInputs(blocked); - } - - @Override - public StateInternals stateInternals() { - return wrapped.stateInternals(); - } - - @Override - public Iterable getSideInputNotifications() { - return wrapped.getSideInputNotifications(); - } - - @Override - public void writePCollectionViewData( - TupleTag tag, - Iterable data, - Coder> dataCoder, - W window, - Coder windowCoder) - throws IOException { - throw new IllegalStateException("User DoFns cannot write PCollectionView data"); - } - - @Override - public TimerInternals timerInternals() { - return wrapped.userTimerInternals(); - } - - @Override - public TimerData getNextFiredTimer(Coder windowCoder) { - return wrapped.getNextFiredUserTimer(windowCoder); - } - - @Override - public void setStateCleanupTimer( - String timerId, - W window, - Coder windowCoder, - Instant cleanupTime, - Instant cleanupOutputTimestamp) { - throw new UnsupportedOperationException( - String.format( - "setStateCleanupTimer should not be called on %s, only on a system %s", - getClass().getSimpleName(), - StreamingModeExecutionContext.StepContext.class.getSimpleName())); - } - - @Override - public DataflowStepContext namespacedToUser() { - return this; - } - } - - /** A {@link SideInputReader} that fetches side inputs from the streaming worker's cache. */ - public static class StreamingModeSideInputReader implements SideInputReader { - - private StreamingModeExecutionContext context; - private Set> viewSet; - - private StreamingModeSideInputReader( - Iterable> views, StreamingModeExecutionContext context) { - this.context = context; - this.viewSet = ImmutableSet.copyOf(views); - } - - public static StreamingModeSideInputReader of( - Iterable> views, StreamingModeExecutionContext context) { - return new StreamingModeSideInputReader(views, context); - } - - @Override - public T get(PCollectionView view, BoundedWindow window) { - if (!contains(view)) { - throw new RuntimeException("get() called with unknown view"); - } - - // We are only fetching the cached value here, so we don't need stateFamily or - // readStateSupplier. - return context - .fetchSideInput( - view, - window, - null /* unused stateFamily */, - StateFetcher.SideInputState.CACHED_IN_WORKITEM, - null /* unused readStateSupplier */) - .orNull(); - } - - @Override - public boolean contains(PCollectionView view) { - return viewSet.contains(view); - } - - @Override - public boolean isEmpty() { - return viewSet.isEmpty(); - } - } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Weighers.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Weighers.java index d2231b8b47bb..eb4e0f4885a7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Weighers.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/Weighers.java @@ -28,7 +28,7 @@ *

Package-private here so that the dependency on Guava does not leak into the public API * surface. */ -class Weighers { +public class Weighers { public static Weigher fixedWeightKeys(final int keyWeight) { return (key, value) -> (int) Math.min(keyWeight + value.getWeight(), Integer.MAX_VALUE); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java deleted file mode 100644 index d4edc0afc0b1..000000000000 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternals.java +++ /dev/null @@ -1,2830 +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. - */ -package org.apache.beam.runners.dataflow.worker; - -import com.google.auto.value.AutoValue; -import java.io.Closeable; -import java.io.IOException; -import java.io.InputStream; -import java.io.OutputStream; -import java.io.OutputStreamWriter; -import java.nio.charset.StandardCharsets; -import java.util.AbstractMap; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.Iterator; -import java.util.List; -import java.util.Map; -import java.util.Map.Entry; -import java.util.Objects; -import java.util.Random; -import java.util.Set; -import java.util.SortedSet; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; -import java.util.function.BiConsumer; -import java.util.function.Function; -import java.util.stream.Collectors; -import javax.annotation.concurrent.NotThreadSafe; -import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Triple; -import org.apache.beam.runners.core.StateInternals; -import org.apache.beam.runners.core.StateNamespace; -import org.apache.beam.runners.core.StateTable; -import org.apache.beam.runners.core.StateTag; -import org.apache.beam.runners.core.StateTag.StateBinder; -import org.apache.beam.runners.core.StateTags; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListEntry; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListRange; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListDeleteRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListInsertRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListUpdateRequest; -import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; -import org.apache.beam.sdk.coders.BooleanCoder; -import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.coders.Coder.Context; -import org.apache.beam.sdk.coders.CoderException; -import org.apache.beam.sdk.coders.CustomCoder; -import org.apache.beam.sdk.coders.InstantCoder; -import org.apache.beam.sdk.coders.MapCoder; -import org.apache.beam.sdk.coders.NullableCoder; -import org.apache.beam.sdk.coders.SetCoder; -import org.apache.beam.sdk.coders.StructuredCoder; -import org.apache.beam.sdk.coders.VarLongCoder; -import org.apache.beam.sdk.state.BagState; -import org.apache.beam.sdk.state.CombiningState; -import org.apache.beam.sdk.state.MapState; -import org.apache.beam.sdk.state.MultimapState; -import org.apache.beam.sdk.state.OrderedListState; -import org.apache.beam.sdk.state.ReadableState; -import org.apache.beam.sdk.state.ReadableStates; -import org.apache.beam.sdk.state.SetState; -import org.apache.beam.sdk.state.State; -import org.apache.beam.sdk.state.StateContext; -import org.apache.beam.sdk.state.StateContexts; -import org.apache.beam.sdk.state.ValueState; -import org.apache.beam.sdk.state.WatermarkHoldState; -import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.transforms.CombineWithContext.CombineFnWithContext; -import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.Weighted; -import org.apache.beam.sdk.values.TimestampedValue; -import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BoundType; -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.Iterables; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.RangeSet; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.TreeRangeSet; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; -import org.checkerframework.checker.initialization.qual.Initialized; -import org.checkerframework.checker.nullness.qual.NonNull; -import org.checkerframework.checker.nullness.qual.Nullable; -import org.checkerframework.checker.nullness.qual.UnknownKeyFor; -import org.joda.time.Duration; -import org.joda.time.Instant; - -/** Implementation of {@link StateInternals} using Windmill to manage the underlying data. */ -@SuppressWarnings({ - "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) - "nullness" // TODO(https://github.com/apache/beam/issues/20497) -}) -class WindmillStateInternals implements StateInternals { - - /** - * The key will be null when not in a keyed context, from the users perspective. There is still a - * "key" for the Windmill computation, but it cannot be meaningfully deserialized. - */ - private final @Nullable K key; - - @Override - public @Nullable K getKey() { - return key; - } - - private static class CachingStateTable extends StateTable { - private final String stateFamily; - private final WindmillStateReader reader; - private final WindmillStateCache.ForKeyAndFamily cache; - private final boolean isSystemTable; - boolean isNewKey; - private final Supplier scopedReadStateSupplier; - private final StateTable derivedStateTable; - - public CachingStateTable( - @Nullable K key, - String stateFamily, - WindmillStateReader reader, - WindmillStateCache.ForKeyAndFamily cache, - boolean isSystemTable, - boolean isNewKey, - Supplier scopedReadStateSupplier, - StateTable derivedStateTable) { - this.stateFamily = stateFamily; - this.reader = reader; - this.cache = cache; - this.isSystemTable = isSystemTable; - this.isNewKey = isNewKey; - this.scopedReadStateSupplier = scopedReadStateSupplier; - this.derivedStateTable = derivedStateTable != null ? derivedStateTable : this; - } - - @Override - protected StateBinder binderForNamespace( - final StateNamespace namespace, final StateContext c) { - // Look up state objects in the cache or create new ones if not found. The state will - // be added to the cache in persist(). - return new StateBinder() { - @Override - public BagState bindBag(StateTag> address, Coder elemCoder) { - if (isSystemTable) { - address = StateTags.makeSystemTagInternal(address); - } - WindmillBag result = (WindmillBag) cache.get(namespace, address); - if (result == null) { - result = new WindmillBag<>(namespace, address, stateFamily, elemCoder, isNewKey); - } - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public SetState bindSet(StateTag> spec, Coder elemCoder) { - WindmillSet result = - new WindmillSet(namespace, spec, stateFamily, elemCoder, cache, isNewKey); - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public MapState bindMap( - StateTag> spec, Coder keyCoder, Coder valueCoder) { - WindmillMap result = (WindmillMap) cache.get(namespace, spec); - if (result == null) { - result = - new WindmillMap( - namespace, spec, stateFamily, keyCoder, valueCoder, isNewKey); - } - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public MultimapState bindMultimap( - StateTag> spec, - Coder keyCoder, - Coder valueCoder) { - WindmillMultimap result = - (WindmillMultimap) cache.get(namespace, spec); - if (result == null) { - result = - new WindmillMultimap<>( - namespace, spec, stateFamily, keyCoder, valueCoder, isNewKey); - } - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public OrderedListState bindOrderedList( - StateTag> spec, Coder elemCoder) { - if (isSystemTable) { - spec = StateTags.makeSystemTagInternal(spec); - } - WindmillOrderedList result = (WindmillOrderedList) cache.get(namespace, spec); - if (result == null) { - result = - new WindmillOrderedList<>( - derivedStateTable, namespace, spec, stateFamily, elemCoder, isNewKey); - } - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public WatermarkHoldState bindWatermark( - StateTag address, TimestampCombiner timestampCombiner) { - if (isSystemTable) { - address = StateTags.makeSystemTagInternal(address); - } - WindmillWatermarkHold result = (WindmillWatermarkHold) cache.get(namespace, address); - if (result == null) { - result = - new WindmillWatermarkHold( - namespace, address, stateFamily, timestampCombiner, isNewKey); - } - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public CombiningState bindCombiningValue( - StateTag> address, - Coder accumCoder, - CombineFn combineFn) { - if (isSystemTable) { - address = StateTags.makeSystemTagInternal(address); - } - WindmillCombiningState result = - new WindmillCombiningState<>( - namespace, address, stateFamily, accumCoder, combineFn, cache, isNewKey); - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - - @Override - public - CombiningState bindCombiningValueWithContext( - StateTag> address, - Coder accumCoder, - CombineFnWithContext combineFn) { - if (isSystemTable) { - address = StateTags.makeSystemTagInternal(address); - } - return bindCombiningValue(address, accumCoder, CombineFnUtil.bindContext(combineFn, c)); - } - - @Override - public ValueState bindValue(StateTag> address, Coder coder) { - if (isSystemTable) { - address = StateTags.makeSystemTagInternal(address); - } - WindmillValue result = (WindmillValue) cache.get(namespace, address); - if (result == null) { - result = new WindmillValue<>(namespace, address, stateFamily, coder, isNewKey); - } - result.initializeForWorkItem(reader, scopedReadStateSupplier); - return result; - } - }; - } - } - - private WindmillStateCache.ForKeyAndFamily cache; - Supplier scopedReadStateSupplier; - private StateTable workItemState; - private StateTable workItemDerivedState; - - public WindmillStateInternals( - @Nullable K key, - String stateFamily, - WindmillStateReader reader, - boolean isNewKey, - WindmillStateCache.ForKeyAndFamily cache, - Supplier scopedReadStateSupplier) { - this.key = key; - this.cache = cache; - this.scopedReadStateSupplier = scopedReadStateSupplier; - this.workItemDerivedState = - new CachingStateTable<>( - key, stateFamily, reader, cache, true, isNewKey, scopedReadStateSupplier, null); - this.workItemState = - new CachingStateTable<>( - key, - stateFamily, - reader, - cache, - false, - isNewKey, - scopedReadStateSupplier, - workItemDerivedState); - } - - private void persist(List> commitsToMerge, StateTable stateTable) { - for (State location : stateTable.values()) { - if (!(location instanceof WindmillState)) { - throw new IllegalStateException( - String.format( - "%s wasn't created by %s -- unable to persist it", - location.getClass().getSimpleName(), getClass().getSimpleName())); - } - - try { - commitsToMerge.add(((WindmillState) location).persist(cache)); - } catch (IOException e) { - throw new RuntimeException("Unable to persist state", e); - } - } - - // All cached State objects now have known values. - // Clear any references to the underlying reader to prevent space leaks. - // The next work unit to use these cached State objects will reset the - // reader to a current reader in case those values are modified. - for (State location : stateTable.values()) { - ((WindmillState) location).cleanupAfterWorkItem(); - } - - // Clear out the map of already retrieved state instances. - stateTable.clear(); - } - - public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder) { - List> commitsToMerge = new ArrayList<>(); - - // Call persist on each first, which may schedule some futures for reading. - persist(commitsToMerge, workItemState); - persist(commitsToMerge, workItemDerivedState); - - try (Closeable scope = scopedReadStateSupplier.get()) { - for (Future commitFuture : commitsToMerge) { - commitBuilder.mergeFrom(commitFuture.get()); - } - } catch (ExecutionException | InterruptedException | IOException exc) { - if (exc instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Failed to retrieve Windmill state during persist()", exc); - } - - cache.persist(); - } - - /** Encodes the given namespace and address as {@code <namespace>+<address>}. */ - @VisibleForTesting - static ByteString encodeKey(StateNamespace namespace, StateTag address) { - try { - // Use ByteStringOutputStream rather than concatenation and String.format. We build these keys - // a lot, and this leads to better performance results. See associated benchmarks. - ByteStringOutputStream stream = new ByteStringOutputStream(); - OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8); - - // stringKey starts and ends with a slash. We separate it from the - // StateTag ID by a '+' (which is guaranteed not to be in the stringKey) because the - // ID comes from the user. - namespace.appendTo(writer); - writer.write('+'); - address.appendTo(writer); - writer.flush(); - return stream.toByteString(); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - /** - * Abstract base class for all Windmill state. - * - *

Note that these are not thread safe; each state object is associated with a key and thus - * only accessed by a single thread at once. - */ - @NotThreadSafe - private abstract static class WindmillState { - protected Supplier scopedReadStateSupplier; - protected WindmillStateReader reader; - - /** - * Return an asynchronously computed {@link WorkItemCommitRequest}. The request should be of a - * form that can be merged with others (only add to repeated fields). - */ - abstract Future persist(WindmillStateCache.ForKeyAndFamily cache) - throws IOException; - - /** - * Prepare this (possibly reused from cache) state for reading from {@code reader} if needed. - */ - void initializeForWorkItem( - WindmillStateReader reader, Supplier scopedReadStateSupplier) { - this.reader = reader; - this.scopedReadStateSupplier = scopedReadStateSupplier; - } - - /** - * This (now cached) state should never need to interact with the reader until the next work - * item. Clear it to prevent space leaks. The reader will be reset by {@link - * #initializeForWorkItem} upon the next work item. - */ - void cleanupAfterWorkItem() { - this.reader = null; - this.scopedReadStateSupplier = null; - } - - Closeable scopedReadState() { - return scopedReadStateSupplier.get(); - } - } - - /** - * Base class for implementations of {@link WindmillState} where the {@link #persist} call does - * not require any asynchronous reading. - */ - private abstract static class SimpleWindmillState extends WindmillState { - @Override - public final Future persist(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - return Futures.immediateFuture(persistDirectly(cache)); - } - - /** - * Returns a {@link WorkItemCommitRequest} that can be used to persist this state to Windmill. - */ - protected abstract WorkItemCommitRequest persistDirectly( - WindmillStateCache.ForKeyAndFamily cache) throws IOException; - } - - @Override - public T state(StateNamespace namespace, StateTag address) { - return workItemState.get(namespace, address, StateContexts.nullContext()); - } - - @Override - public T state( - StateNamespace namespace, StateTag address, StateContext c) { - return workItemState.get(namespace, address, c); - } - - private static class WindmillValue extends SimpleWindmillState implements ValueState { - private final StateNamespace namespace; - private final StateTag> address; - private final ByteString stateKey; - private final String stateFamily; - private final Coder coder; - - /** Whether we've modified the value since creation of this state. */ - private boolean modified = false; - /** Whether the in memory value is the true value. */ - private boolean valueIsKnown = false; - /** The size of the encoded value */ - private long cachedSize = -1; - - private T value; - - private WindmillValue( - StateNamespace namespace, - StateTag> address, - String stateFamily, - Coder coder, - boolean isNewKey) { - this.namespace = namespace; - this.address = address; - this.stateKey = encodeKey(namespace, address); - this.stateFamily = stateFamily; - this.coder = coder; - if (isNewKey) { - this.valueIsKnown = true; - this.value = null; - } - } - - @Override - public void clear() { - modified = true; - valueIsKnown = true; - value = null; - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public WindmillValue readLater() { - getFuture(); - return this; - } - - @Override - public T read() { - try (Closeable scope = scopedReadState()) { - if (!valueIsKnown) { - cachedSize = -1; - } - value = getFuture().get(); - valueIsKnown = true; - return value; - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read value from state", e); - } - } - - @Override - public void write(T value) { - modified = true; - valueIsKnown = true; - cachedSize = -1; - this.value = value; - } - - @Override - protected WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - if (!valueIsKnown) { - // The value was never read, written or cleared. - // Thus nothing to update in Windmill. - // And no need to add to global cache. - return WorkItemCommitRequest.newBuilder().buildPartial(); - } - - ByteString encoded = null; - if (cachedSize == -1 || modified) { - ByteStringOutputStream stream = new ByteStringOutputStream(); - if (value != null) { - coder.encode(value, stream, Coder.Context.OUTER); - } - encoded = stream.toByteString(); - cachedSize = encoded.size(); - } - - // Place in cache to avoid a future read. - cache.put(namespace, address, this, cachedSize); - - if (!modified) { - // The value was read, but never written or cleared. - // But nothing to update in Windmill. - return WorkItemCommitRequest.newBuilder().buildPartial(); - } - - // The value was written or cleared. Commit that change to Windmill. - modified = false; - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - commitBuilder - .addValueUpdatesBuilder() - .setTag(stateKey) - .setStateFamily(stateFamily) - .getValueBuilder() - .setData(encoded) - .setTimestamp(Long.MAX_VALUE); - return commitBuilder.buildPartial(); - } - - private Future getFuture() { - // WindmillStateReader guarantees that we can ask for a future for a particular tag multiple - // times and it will efficiently be reused. - return valueIsKnown - ? Futures.immediateFuture(value) - : reader.valueFuture(stateKey, stateFamily, coder); - } - } - - // Coder for closed-open ranges. - private static class RangeCoder extends StructuredCoder> { - private Coder boundCoder; - - RangeCoder(Coder boundCoder) { - this.boundCoder = NullableCoder.of(boundCoder); - } - - @Override - public List> getCoderArguments() { - return Lists.newArrayList(boundCoder); - } - - @Override - public void verifyDeterministic() throws NonDeterministicException { - boundCoder.verifyDeterministic(); - ; - } - - @Override - public void encode(Range value, OutputStream outStream) throws CoderException, IOException { - Preconditions.checkState( - value.lowerBoundType().equals(BoundType.CLOSED), "unexpected range " + value); - Preconditions.checkState( - value.upperBoundType().equals(BoundType.OPEN), "unexpected range " + value); - boundCoder.encode(value.hasLowerBound() ? value.lowerEndpoint() : null, outStream); - boundCoder.encode(value.hasUpperBound() ? value.upperEndpoint() : null, outStream); - } - - @Override - public Range decode(InputStream inStream) throws CoderException, IOException { - @Nullable T lower = boundCoder.decode(inStream); - @Nullable T upper = boundCoder.decode(inStream); - if (lower == null) { - return upper != null ? Range.lessThan(upper) : Range.all(); - } else if (upper == null) { - return Range.atLeast(lower); - } else { - return Range.closedOpen(lower, upper); - } - } - } - - private static class RangeSetCoder extends CustomCoder> { - private SetCoder> rangesCoder; - - RangeSetCoder(Coder boundCoder) { - this.rangesCoder = SetCoder.of(new RangeCoder<>(boundCoder)); - } - - @Override - public void encode(RangeSet value, OutputStream outStream) throws IOException { - rangesCoder.encode(value.asRanges(), outStream); - } - - @Override - public RangeSet decode(InputStream inStream) throws CoderException, IOException { - return TreeRangeSet.create(rangesCoder.decode(inStream)); - } - } - - /** - * Tracker for the ids used in an ordered list. - * - *

Windmill accepts an int64 id for each timestamped-element in the list. Unique elements are - * identified by the pair of timestamp and id. This means that tow unique elements e1, e2 must - * have different (ts1, id1), (ts2, id2) pairs. To accomplish this we bucket time into five-minute - * buckets, and store a free list of ids available for each bucket. - * - *

When a timestamp range is deleted, we remove id tracking for elements in that range. In - * order to handle the case where a range is deleted piecemeal, we track sub-range deletions for - * each range. For example: - * - *

12:00 - 12:05 ids 12:05 - 12:10 ids - * - *

delete 12:00-12:06 - * - *

12:00 - 12:05 *removed* 12:05 - 12:10 ids subranges deleted 12:05-12:06 - * - *

delete 12:06 - 12:07 - * - *

12:05 - 12:10 ids subranges deleted 12:05-12:07 - * - *

delete 12:07 - 12:10 - * - *

12:05 - 12:10 *removed* - */ - static final class IdTracker { - static final String IDS_AVAILABLE_STR = "IdsAvailable"; - static final String DELETIONS_STR = "Deletions"; - - // Note that this previously was Long.MIN_VALUE but ids are unsigned when - // sending to windmill for Streaming Engine. For updated appliance - // pipelines with existing state, there may be negative ids. - static final long NEW_RANGE_MIN_ID = 0; - static final long NEW_RANGE_MAX_ID = Long.MAX_VALUE; - - // We track ids on five-minute boundaries. - private static final Duration RESOLUTION = Duration.standardMinutes(5); - static final MapCoder, RangeSet> IDS_AVAILABLE_CODER = - MapCoder.of(new RangeCoder<>(InstantCoder.of()), new RangeSetCoder<>(VarLongCoder.of())); - static final MapCoder, RangeSet> SUBRANGE_DELETIONS_CODER = - MapCoder.of(new RangeCoder<>(InstantCoder.of()), new RangeSetCoder<>(InstantCoder.of())); - private final StateTag, RangeSet>>> idsAvailableTag; - // A map from five-minute ranges to the set of ids available in that interval. - final ValueState, RangeSet>> idsAvailableValue; - private final StateTag, RangeSet>>> subRangeDeletionsTag; - // If a timestamp-range in the map has been partially cleared, the cleared intervals are stored - // here. - final ValueState, RangeSet>> subRangeDeletionsValue; - - IdTracker( - StateTable stateTable, - StateNamespace namespace, - StateTag spec, - String stateFamily, - boolean complete) { - this.idsAvailableTag = - StateTags.makeSystemTagInternal( - StateTags.value(spec.getId() + IDS_AVAILABLE_STR, IDS_AVAILABLE_CODER)); - this.idsAvailableValue = - stateTable.get(namespace, idsAvailableTag, StateContexts.nullContext()); - this.subRangeDeletionsTag = - StateTags.makeSystemTagInternal( - StateTags.value(spec.getId() + DELETIONS_STR, SUBRANGE_DELETIONS_CODER)); - this.subRangeDeletionsValue = - stateTable.get(namespace, subRangeDeletionsTag, StateContexts.nullContext()); - } - - static > - Map, RangeSet> newSortedRangeMap(Class valueClass) { - return Maps.newTreeMap( - Comparator., Instant>comparing(Range::lowerEndpoint) - .thenComparing(Range::upperEndpoint)); - } - - private Range getTrackedRange(Instant ts) { - Instant snapped = - new Instant(ts.getMillis() - ts.plus(RESOLUTION).getMillis() % RESOLUTION.getMillis()); - return Range.closedOpen(snapped, snapped.plus(RESOLUTION)); - } - - @SuppressWarnings("FutureReturnValueIgnored") - void readLater() { - idsAvailableValue.readLater(); - subRangeDeletionsValue.readLater(); - } - - Map, RangeSet> readIdsAvailable() { - Map, RangeSet> idsAvailable = idsAvailableValue.read(); - return idsAvailable != null ? idsAvailable : newSortedRangeMap(Long.class); - } - - Map, RangeSet> readSubRangeDeletions() { - Map, RangeSet> subRangeDeletions = subRangeDeletionsValue.read(); - return subRangeDeletions != null ? subRangeDeletions : newSortedRangeMap(Instant.class); - } - - void clear() throws ExecutionException, InterruptedException { - idsAvailableValue.clear(); - subRangeDeletionsValue.clear(); - } - - void add( - SortedSet> elements, BiConsumer, Long> output) - throws ExecutionException, InterruptedException { - Range currentIdRange = null; - long currentId = 0; - - Range currentTsRange = null; - RangeSet currentTsRangeDeletions = null; - - Map, RangeSet> idsAvailable = readIdsAvailable(); - Map, RangeSet> subRangeDeletions = readSubRangeDeletions(); - - RangeSet availableIdsForTsRange = null; - Iterator> idRangeIter = null; - RangeSet idsUsed = TreeRangeSet.create(); - for (TimestampedValueWithId pendingAdd : elements) { - // Since elements are in increasing ts order, often we'll be able to reuse the previous - // iteration's range. - if (currentTsRange == null - || !currentTsRange.contains(pendingAdd.getValue().getTimestamp())) { - if (availableIdsForTsRange != null) { - // We're moving onto a new ts range. Remove all used ids - availableIdsForTsRange.removeAll(idsUsed); - idsUsed = TreeRangeSet.create(); - } - - // Lookup the range for the current timestamp. - currentTsRange = getTrackedRange(pendingAdd.getValue().getTimestamp()); - // Lookup available ids for this timestamp range. If nothing there, we default to all ids - // available. - availableIdsForTsRange = - idsAvailable.computeIfAbsent( - currentTsRange, - r -> - TreeRangeSet.create( - ImmutableList.of(Range.closedOpen(NEW_RANGE_MIN_ID, NEW_RANGE_MAX_ID)))); - idRangeIter = availableIdsForTsRange.asRanges().iterator(); - currentIdRange = null; - currentTsRangeDeletions = subRangeDeletions.get(currentTsRange); - } - - if (currentIdRange == null || currentId >= currentIdRange.upperEndpoint()) { - // Move to the next range of free ids, and start assigning ranges from there. - currentIdRange = idRangeIter.next(); - currentId = currentIdRange.lowerEndpoint(); - } - - if (currentTsRangeDeletions != null) { - currentTsRangeDeletions.remove( - Range.closedOpen( - pendingAdd.getValue().getTimestamp(), - pendingAdd.getValue().getTimestamp().plus(Duration.millis(1)))); - } - idsUsed.add(Range.closedOpen(currentId, currentId + 1)); - output.accept(pendingAdd.getValue(), currentId++); - } - if (availableIdsForTsRange != null) { - availableIdsForTsRange.removeAll(idsUsed); - } - writeValues(idsAvailable, subRangeDeletions); - } - - // Remove a timestamp range. Returns ids freed up. - void remove(Range tsRange) throws ExecutionException, InterruptedException { - Map, RangeSet> idsAvailable = readIdsAvailable(); - Map, RangeSet> subRangeDeletions = readSubRangeDeletions(); - - for (Range current = getTrackedRange(tsRange.lowerEndpoint()); - current.lowerEndpoint().isBefore(tsRange.upperEndpoint()); - current = getTrackedRange(current.lowerEndpoint().plus(RESOLUTION))) { - // TODO(reuvenlax): shouldn't need to iterate over all ranges. - boolean rangeCleared; - if (!tsRange.encloses(current)) { - // This can happen if the beginning or the end of tsRange doesn't fall on a RESOLUTION - // boundary. Since we - // are deleting a portion of a tracked range, track what we are deleting. - RangeSet rangeDeletions = - subRangeDeletions.computeIfAbsent(current, r -> TreeRangeSet.create()); - rangeDeletions.add(tsRange.intersection(current)); - // If we ended up deleting the whole range, than we can simply remove it from the tracking - // map. - rangeCleared = rangeDeletions.encloses(current); - } else { - rangeCleared = true; - } - if (rangeCleared) { - // Remove the range from both maps. - idsAvailable.remove(current); - subRangeDeletions.remove(current); - } - } - writeValues(idsAvailable, subRangeDeletions); - } - - private void writeValues( - Map, RangeSet> idsAvailable, - Map, RangeSet> subRangeDeletions) { - if (idsAvailable.isEmpty()) { - idsAvailable.clear(); - } else { - idsAvailableValue.write(idsAvailable); - } - if (subRangeDeletions.isEmpty()) { - subRangeDeletionsValue.clear(); - } else { - subRangeDeletionsValue.write(subRangeDeletions); - } - } - } - - @AutoValue - abstract static class TimestampedValueWithId { - private static final Comparator> COMPARATOR = - Comparator., Instant>comparing(v -> v.getValue().getTimestamp()) - .thenComparingLong(TimestampedValueWithId::getId); - - abstract TimestampedValue getValue(); - - abstract long getId(); - - static TimestampedValueWithId of(TimestampedValue value, long id) { - return new AutoValue_WindmillStateInternals_TimestampedValueWithId<>(value, id); - } - - static TimestampedValueWithId bound(Instant ts) { - return of(TimestampedValue.of(null, ts), Long.MIN_VALUE); - } - } - - static class WindmillOrderedList extends SimpleWindmillState implements OrderedListState { - private final ByteString stateKey; - private final String stateFamily; - private final Coder elemCoder; - private boolean complete; - private boolean cleared = false; - // We need to sort based on timestamp, but we need objects with the same timestamp to be treated - // as unique. We can't use a MultiSet as we can't construct a comparator that uniquely - // identifies objects, - // so we construct a unique in-memory long ids for each element. - private SortedSet> pendingAdds = - Sets.newTreeSet(TimestampedValueWithId.COMPARATOR); - - private RangeSet pendingDeletes = TreeRangeSet.create(); - private IdTracker idTracker; - - // The default proto values for SortedListRange correspond to the minimum and maximum - // timestamps. - static final long MIN_TS_MICROS = SortedListRange.getDefaultInstance().getStart(); - static final long MAX_TS_MICROS = SortedListRange.getDefaultInstance().getLimit(); - - private WindmillOrderedList( - StateTable derivedStateTable, - StateNamespace namespace, - StateTag> spec, - String stateFamily, - Coder elemCoder, - boolean isNewKey) { - - this.stateKey = encodeKey(namespace, spec); - this.stateFamily = stateFamily; - this.elemCoder = elemCoder; - this.complete = isNewKey; - this.idTracker = new IdTracker(derivedStateTable, namespace, spec, stateFamily, complete); - } - - @Override - public Iterable> read() { - return readRange(null, null); - } - - private SortedSet> getPendingAddRange( - @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { - SortedSet> pendingInRange = pendingAdds; - if (minTimestamp != null && limitTimestamp != null) { - pendingInRange = - pendingInRange.subSet( - TimestampedValueWithId.bound(minTimestamp), - TimestampedValueWithId.bound(limitTimestamp)); - } else if (minTimestamp == null && limitTimestamp != null) { - pendingInRange = pendingInRange.headSet(TimestampedValueWithId.bound(limitTimestamp)); - } else if (limitTimestamp == null && minTimestamp != null) { - pendingInRange = pendingInRange.tailSet(TimestampedValueWithId.bound(minTimestamp)); - } - return pendingInRange; - } - - @Override - public Iterable> readRange( - @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { - idTracker.readLater(); - - final Future>> future = getFuture(minTimestamp, limitTimestamp); - try (Closeable scope = scopedReadState()) { - SortedSet> pendingInRange = - getPendingAddRange(minTimestamp, limitTimestamp); - - // Transform the return iterator so it has the same type as pendingAdds. We need to ensure - // that the ids don't overlap with any in pendingAdds, so begin with pendingAdds.size(). - Iterable> data = - new Iterable>() { - // Anything returned from windmill that has been deleted should be ignored. - private Iterable> iterable = - Iterables.filter(future.get(), tv -> !pendingDeletes.contains(tv.getTimestamp())); - - @Override - public Iterator> iterator() { - return new Iterator>() { - private Iterator> iter = iterable.iterator(); - private long currentId = pendingAdds.size(); - - @Override - public boolean hasNext() { - return iter.hasNext(); - } - - @Override - public TimestampedValueWithId next() { - return TimestampedValueWithId.of(iter.next(), currentId++); - } - }; - } - }; - - Iterable> includingAdds = - Iterables.mergeSorted( - ImmutableList.of(data, pendingInRange), TimestampedValueWithId.COMPARATOR); - Iterable> fullIterable = - Iterables.transform(includingAdds, TimestampedValueWithId::getValue); - - // TODO(reuvenlax): If we have a known bounded amount of data, cache known ranges. - return fullIterable; - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - @Override - public void clear() { - cleared = true; - complete = true; - pendingAdds.clear(); - pendingDeletes.clear(); - try { - idTracker.clear(); - } catch (ExecutionException | InterruptedException e) { - throw new RuntimeException(e); - } - } - - @Override - public void clearRange(Instant minTimestamp, Instant limitTimestamp) { - getPendingAddRange(minTimestamp, limitTimestamp).clear(); - pendingDeletes.add(Range.closedOpen(minTimestamp, limitTimestamp)); - } - - @Override - public void add(TimestampedValue value) { - // We use the current size of the container as the in-memory id. This works because - // pendingAdds is completely - // cleared when it is processed (otherwise we could end up with duplicate elements in the same - // container). These - // are not the ids that will be sent to windmill. - pendingAdds.add(TimestampedValueWithId.of(value, pendingAdds.size())); - // Leave pendingDeletes alone. Since we can have multiple values with the same timestamp, we - // may still need - // overlapping deletes to remove previous entries at this timestamp. - } - - @Override - public ReadableState isEmpty() { - return new ReadableState() { - @Override - public ReadableState readLater() { - WindmillOrderedList.this.readLater(); - return this; - } - - @Override - public Boolean read() { - return Iterables.isEmpty(WindmillOrderedList.this.read()); - } - }; - } - - @Override - public OrderedListState readLater() { - return readRangeLater(null, null); - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public OrderedListState readRangeLater( - @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { - idTracker.readLater(); - getFuture(minTimestamp, limitTimestamp); - return this; - } - - @Override - public WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - TagSortedListUpdateRequest.Builder updatesBuilder = - commitBuilder - .addSortedListUpdatesBuilder() - .setStateFamily(cache.getStateFamily()) - .setTag(stateKey); - try { - if (cleared) { - // Default range. - updatesBuilder.addDeletesBuilder().build(); - cleared = false; - } - - if (!pendingAdds.isEmpty()) { - // TODO(reuvenlax): Once we start caching data, we should remove this line. We have it - // here now - // because once we persist - // added data we forget about it from the cache, so the object is no longer complete. - complete = false; - - TagSortedListInsertRequest.Builder insertBuilder = updatesBuilder.addInsertsBuilder(); - idTracker.add( - pendingAdds, - (elem, id) -> { - try { - ByteStringOutputStream elementStream = new ByteStringOutputStream(); - elemCoder.encode(elem.getValue(), elementStream, Context.OUTER); - insertBuilder.addEntries( - SortedListEntry.newBuilder() - .setValue(elementStream.toByteString()) - .setSortKey( - WindmillTimeUtils.harnessToWindmillTimestamp(elem.getTimestamp())) - .setId(id)); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - pendingAdds.clear(); - insertBuilder.build(); - } - - if (!pendingDeletes.isEmpty()) { - for (Range range : pendingDeletes.asRanges()) { - TagSortedListDeleteRequest.Builder deletesBuilder = updatesBuilder.addDeletesBuilder(); - deletesBuilder.setRange( - SortedListRange.newBuilder() - .setStart(WindmillTimeUtils.harnessToWindmillTimestamp(range.lowerEndpoint())) - .setLimit(WindmillTimeUtils.harnessToWindmillTimestamp(range.upperEndpoint()))); - deletesBuilder.build(); - idTracker.remove(range); - } - pendingDeletes.clear(); - } - } catch (ExecutionException | InterruptedException e) { - throw new RuntimeException(e); - } - return commitBuilder.buildPartial(); - } - - private Future>> getFuture( - @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { - long startSortKey = - minTimestamp != null - ? WindmillTimeUtils.harnessToWindmillTimestamp(minTimestamp) - : MIN_TS_MICROS; - long limitSortKey = - limitTimestamp != null - ? WindmillTimeUtils.harnessToWindmillTimestamp(limitTimestamp) - : MAX_TS_MICROS; - - if (complete) { - // Right now we don't cache any data, so complete means an empty list. - // TODO(reuvenlax): change this once we start caching data. - return Futures.immediateFuture(Collections.emptyList()); - } - return reader.orderedListFuture( - Range.closedOpen(startSortKey, limitSortKey), stateKey, stateFamily, elemCoder); - } - } - - static class WindmillSet extends SimpleWindmillState implements SetState { - WindmillMap windmillMap; - - WindmillSet( - StateNamespace namespace, - StateTag> address, - String stateFamily, - Coder keyCoder, - WindmillStateCache.ForKeyAndFamily cache, - boolean isNewKey) { - StateTag> internalMapAddress = - StateTags.convertToMapTagInternal(address); - WindmillMap cachedMap = - (WindmillMap) cache.get(namespace, internalMapAddress); - this.windmillMap = - (cachedMap != null) - ? cachedMap - : new WindmillMap<>( - namespace, - internalMapAddress, - stateFamily, - keyCoder, - BooleanCoder.of(), - isNewKey); - } - - @Override - protected WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - return windmillMap.persistDirectly(cache); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState< - @UnknownKeyFor @NonNull @Initialized Boolean> - contains(K k) { - return windmillMap.getOrDefault(k, false); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState< - @UnknownKeyFor @NonNull @Initialized Boolean> - addIfAbsent(K k) { - return new ReadableState() { - ReadableState putState = windmillMap.putIfAbsent(k, true); - - @Override - public @Nullable Boolean read() { - Boolean result = putState.read(); - return (result != null) ? result : false; - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState readLater() { - putState = putState.readLater(); - return this; - } - }; - } - - @Override - public void remove(K k) { - windmillMap.remove(k); - } - - @Override - public void add(K value) { - windmillMap.put(value, true); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState< - @UnknownKeyFor @NonNull @Initialized Boolean> - isEmpty() { - return windmillMap.isEmpty(); - } - - @Override - public @Nullable Iterable read() { - return windmillMap.keys().read(); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized SetState readLater() { - windmillMap.keys().readLater(); - return this; - } - - @Override - public void clear() { - windmillMap.clear(); - } - - @Override - void initializeForWorkItem( - WindmillStateReader reader, Supplier scopedReadStateSupplier) { - windmillMap.initializeForWorkItem(reader, scopedReadStateSupplier); - } - - @Override - void cleanupAfterWorkItem() { - windmillMap.cleanupAfterWorkItem(); - } - } - - static class WindmillMap extends SimpleWindmillState implements MapState { - private final StateNamespace namespace; - private final StateTag> address; - private final ByteString stateKeyPrefix; - private final String stateFamily; - private final Coder keyCoder; - private final Coder valueCoder; - private boolean complete; - - // TODO(reuvenlax): Should we evict items from the cache? We would have to make sure - // that anything in the cache that is not committed is not evicted. negativeCache could be - // evicted whenever we want. - private Map cachedValues = Maps.newHashMap(); - private Set negativeCache = Sets.newHashSet(); - private boolean cleared = false; - - private Set localAdditions = Sets.newHashSet(); - private Set localRemovals = Sets.newHashSet(); - - WindmillMap( - StateNamespace namespace, - StateTag> address, - String stateFamily, - Coder keyCoder, - Coder valueCoder, - boolean isNewKey) { - this.namespace = namespace; - this.address = address; - this.stateKeyPrefix = encodeKey(namespace, address); - this.stateFamily = stateFamily; - this.keyCoder = keyCoder; - this.valueCoder = valueCoder; - this.complete = isNewKey; - } - - private K userKeyFromProtoKey(ByteString tag) throws IOException { - Preconditions.checkState(tag.startsWith(stateKeyPrefix)); - ByteString keyBytes = tag.substring(stateKeyPrefix.size()); - return keyCoder.decode(keyBytes.newInput(), Context.OUTER); - } - - private ByteString protoKeyFromUserKey(K key) throws IOException { - ByteStringOutputStream keyStream = new ByteStringOutputStream(); - stateKeyPrefix.writeTo(keyStream); - keyCoder.encode(key, keyStream, Context.OUTER); - return keyStream.toByteString(); - } - - @Override - protected WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - if (!cleared && localAdditions.isEmpty() && localRemovals.isEmpty()) { - // No changes, so return directly. - return WorkItemCommitRequest.newBuilder().buildPartial(); - } - - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - - if (cleared) { - commitBuilder - .addTagValuePrefixDeletesBuilder() - .setStateFamily(stateFamily) - .setTagPrefix(stateKeyPrefix); - } - cleared = false; - - for (K key : localAdditions) { - ByteString keyBytes = protoKeyFromUserKey(key); - ByteStringOutputStream valueStream = new ByteStringOutputStream(); - valueCoder.encode(cachedValues.get(key), valueStream, Context.OUTER); - ByteString valueBytes = valueStream.toByteString(); - - commitBuilder - .addValueUpdatesBuilder() - .setTag(keyBytes) - .setStateFamily(stateFamily) - .getValueBuilder() - .setData(valueBytes) - .setTimestamp(Long.MAX_VALUE); - } - localAdditions.clear(); - - for (K key : localRemovals) { - ByteStringOutputStream keyStream = new ByteStringOutputStream(); - stateKeyPrefix.writeTo(keyStream); - keyCoder.encode(key, keyStream, Context.OUTER); - ByteString keyBytes = keyStream.toByteString(); - // Leaving data blank means that we delete the tag. - commitBuilder - .addValueUpdatesBuilder() - .setTag(keyBytes) - .setStateFamily(stateFamily) - .getValueBuilder() - .setTimestamp(Long.MAX_VALUE); - - V cachedValue = cachedValues.remove(key); - if (cachedValue != null) { - ByteStringOutputStream valueStream = new ByteStringOutputStream(); - valueCoder.encode(cachedValues.get(key), valueStream, Context.OUTER); - } - } - negativeCache.addAll(localRemovals); - localRemovals.clear(); - - // TODO(reuvenlax): We should store in the cache parameter, as that would enable caching the - // map - // between work items, reducing fetches to Windmill. To do so, we need keep track of the - // encoded size - // of the map, and to do so efficiently (i.e. without iterating over the entire map on every - // persist) - // we need to track the sizes of each map entry. - cache.put(namespace, address, this, 1); - return commitBuilder.buildPartial(); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState get(K key) { - return getOrDefault(key, null); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState getOrDefault( - K key, @Nullable V defaultValue) { - return new ReadableState() { - @Override - public @Nullable V read() { - Future persistedData = getFutureForKey(key); - try (Closeable scope = scopedReadState()) { - if (localRemovals.contains(key) || negativeCache.contains(key)) { - return null; - } - @Nullable V cachedValue = cachedValues.get(key); - if (cachedValue != null || complete) { - return cachedValue; - } - - V persistedValue = persistedData.get(); - if (persistedValue == null) { - negativeCache.add(key); - return defaultValue; - } - // TODO: Don't do this if it was already in cache. - cachedValues.put(key, persistedValue); - return persistedValue; - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public @UnknownKeyFor @NonNull @Initialized ReadableState readLater() { - WindmillMap.this.getFutureForKey(key); - return this; - } - }; - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState< - @UnknownKeyFor @NonNull @Initialized Iterable> - keys() { - ReadableState>> entries = entries(); - return new ReadableState>() { - @Override - public @Nullable Iterable read() { - return Iterables.transform(entries.read(), e -> e.getKey()); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState> readLater() { - entries.readLater(); - return this; - } - }; - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState< - @UnknownKeyFor @NonNull @Initialized Iterable> - values() { - ReadableState>> entries = entries(); - return new ReadableState>() { - @Override - public @Nullable Iterable read() { - return Iterables.transform(entries.read(), e -> e.getValue()); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState> readLater() { - entries.readLater(); - return this; - } - }; - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState< - @UnknownKeyFor @NonNull @Initialized Iterable< - @UnknownKeyFor @NonNull @Initialized Entry>> - entries() { - return new ReadableState>>() { - @Override - public Iterable> read() { - if (complete) { - return Iterables.unmodifiableIterable(cachedValues.entrySet()); - } - Future>> persistedData = getFuture(); - try (Closeable scope = scopedReadState()) { - Iterable> data = persistedData.get(); - Iterable> transformedData = - Iterables., Map.Entry>transform( - data, - entry -> { - try { - return new AbstractMap.SimpleEntry<>( - userKeyFromProtoKey(entry.getKey()), entry.getValue()); - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - - if (data instanceof Weighted) { - // This is a known amount of data. Cache it all. - transformedData.forEach( - e -> { - // The cached data overrides what is read from state, so call putIfAbsent. - cachedValues.putIfAbsent(e.getKey(), e.getValue()); - }); - complete = true; - return Iterables.unmodifiableIterable(cachedValues.entrySet()); - } else { - // This means that the result might be too large to cache, so don't add it to the - // local cache. Instead merge the iterables, giving priority to any local additions - // (represented in cachedValued and localRemovals) that may not have been committed - // yet. - return Iterables.unmodifiableIterable( - Iterables.concat( - cachedValues.entrySet(), - Iterables.filter( - transformedData, - e -> - !cachedValues.containsKey(e.getKey()) - && !localRemovals.contains(e.getKey())))); - } - - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public @UnknownKeyFor @NonNull @Initialized ReadableState>> - readLater() { - WindmillMap.this.getFuture(); - return this; - } - }; - } - - @Override - public ReadableState isEmpty() { - return new ReadableState() { - // TODO(reuvenlax): Can we find a more efficient way of implementing isEmpty than reading - // the entire map? - ReadableState> keys = WindmillMap.this.keys(); - - @Override - public @Nullable Boolean read() { - return Iterables.isEmpty(keys.read()); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState readLater() { - keys.readLater(); - return this; - } - }; - } - - @Override - public void put(K key, V value) { - V oldValue = cachedValues.put(key, value); - if (valueCoder.consistentWithEquals() && value.equals(oldValue)) { - return; - } - localAdditions.add(key); - localRemovals.remove(key); - negativeCache.remove(key); - } - - @Override - public @UnknownKeyFor @NonNull @Initialized ReadableState computeIfAbsent( - K key, Function mappingFunction) { - Future persistedData = getFutureForKey(key); - try (Closeable scope = scopedReadState()) { - if (localRemovals.contains(key) || negativeCache.contains(key)) { - return ReadableStates.immediate(null); - } - @Nullable V cachedValue = cachedValues.get(key); - if (cachedValue != null || complete) { - return ReadableStates.immediate(cachedValue); - } - - V persistedValue = persistedData.get(); - if (persistedValue == null) { - // This is a new value. Add it to the map and return null. - put(key, mappingFunction.apply(key)); - return ReadableStates.immediate(null); - } - // TODO: Don't do this if it was already in cache. - cachedValues.put(key, persistedValue); - return ReadableStates.immediate(persistedValue); - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - @Override - public void remove(K key) { - if (localRemovals.add(key)) { - cachedValues.remove(key); - localAdditions.remove(key); - } - } - - @Override - public void clear() { - cachedValues.clear(); - localAdditions.clear(); - localRemovals.clear(); - negativeCache.clear(); - cleared = true; - complete = true; - } - - private Future getFutureForKey(K key) { - try { - ByteStringOutputStream keyStream = new ByteStringOutputStream(); - stateKeyPrefix.writeTo(keyStream); - keyCoder.encode(key, keyStream, Context.OUTER); - return reader.valueFuture(keyStream.toByteString(), stateFamily, valueCoder); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - private Future>> getFuture() { - if (complete) { - // The caller will merge in local cached values. - return Futures.immediateFuture(Collections.emptyList()); - } else { - return reader.valuePrefixFuture(stateKeyPrefix, stateFamily, valueCoder); - } - } - } - - private static class WindmillMultimap extends SimpleWindmillState - implements MultimapState { - - private final StateNamespace namespace; - private final StateTag> address; - private final ByteString stateKey; - private final String stateFamily; - private final Coder keyCoder; - private final Coder valueCoder; - - private enum KeyExistence { - // this key is known to exist, it has at least 1 value in either localAdditions or windmill - KNOWN_EXIST, - // this key is known to be nonexistent, it has 0 value in both localAdditions and windmill - KNOWN_NONEXISTENT, - // we don't know if this key is in this multimap, it has exact 0 value in localAddition, but - // may have no or any number of values in windmill. This is just to provide a mapping between - // the original key and the structural key. - UNKNOWN_EXISTENCE - } - - private class KeyState { - final K originalKey; - KeyExistence existence; - // valuesCached can be true if only existence == KNOWN_EXIST and all values of this key are - // cached (both values and localAdditions). - boolean valuesCached; - // Represents the values in windmill. When new values are added during user processing, they - // are added to localAdditions but not values. Those new values will be added to values only - // after they are persisted into windmill and removed from localAdditions - ConcatIterables values; - int valuesSize; - - // When new values are added during user processing, they are added to localAdditions, so that - // we can later try to persist them in windmill. When a key is removed during user processing, - // we mark removedLocally to be true so that we can later try to delete it from windmill. If - // localAdditions is not empty and removedLocally is true, values in localAdditions will be - // added to windmill after old values in windmill are removed. - List localAdditions; - boolean removedLocally; - - KeyState(K originalKey) { - this.originalKey = originalKey; - existence = KeyExistence.UNKNOWN_EXISTENCE; - valuesCached = complete; - values = new ConcatIterables<>(); - valuesSize = 0; - localAdditions = Lists.newArrayList(); - removedLocally = false; - } - } - - // Set to true when user clears the entire multimap, so that we can later send delete request to - // the windmill backend. - private boolean cleared = false; - // We use the structural value of the keys as the key in keyStateMap, so that different java - // Objects with the same content will be treated as the same Multimap key. - private Map keyStateMap = Maps.newHashMap(); - // If true, all keys are cached in keyStateMap with existence == KNOWN_EXIST. - private boolean allKeysKnown = false; - - // True if all contents of this multimap are cached in this object. - private boolean complete = false; - // hasLocalAdditions and hasLocalRemovals track whether there are local changes that needs to be - // propagated to windmill. - private boolean hasLocalAdditions = false; - private boolean hasLocalRemovals = false; - - private WindmillMultimap( - StateNamespace namespace, - StateTag> address, - String stateFamily, - Coder keyCoder, - Coder valueCoder, - boolean isNewShardingKey) { - this.namespace = namespace; - this.address = address; - this.stateKey = encodeKey(namespace, address); - this.stateFamily = stateFamily; - this.keyCoder = keyCoder; - this.valueCoder = valueCoder; - this.complete = isNewShardingKey; - this.allKeysKnown = isNewShardingKey; - } - - @Override - public void put(K key, V value) { - final Object structuralKey = keyCoder.structuralValue(key); - hasLocalAdditions = true; - keyStateMap.compute( - structuralKey, - (k, v) -> { - if (v == null) v = new KeyState(key); - v.existence = KeyExistence.KNOWN_EXIST; - v.localAdditions.add(value); - return v; - }); - } - - // Initiates a backend state read to fetch all entries if necessary. - private Future>>> necessaryEntriesFromStorageFuture( - boolean omitValues) { - if (complete) { - // Since we're complete, even if there are entries in storage we don't need to read them. - return Futures.immediateFuture(Collections.emptyList()); - } else { - return reader.multimapFetchAllFuture(omitValues, stateKey, stateFamily, valueCoder); - } - } - - // Initiates a backend state read to fetch a single entry if necessary. - private Future> necessaryKeyEntriesFromStorageFuture(K key) { - try { - ByteStringOutputStream keyStream = new ByteStringOutputStream(); - keyCoder.encode(key, keyStream, Context.OUTER); - return reader.multimapFetchSingleEntryFuture( - keyStream.toByteString(), stateKey, stateFamily, valueCoder); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - @Override - public ReadableState> get(K key) { - return new ReadableState>() { - final Object structuralKey = keyCoder.structuralValue(key); - - @Override - public Iterable read() { - KeyState keyState = null; - if (allKeysKnown) { - keyState = keyStateMap.get(structuralKey); - if (keyState == null || keyState.existence == KeyExistence.UNKNOWN_EXISTENCE) { - if (keyState != null) keyStateMap.remove(structuralKey); - return Collections.emptyList(); - } - } else { - keyState = keyStateMap.computeIfAbsent(structuralKey, k -> new KeyState(key)); - } - if (keyState.existence == KeyExistence.KNOWN_NONEXISTENT) { - return Collections.emptyList(); - } - Iterable localNewValues = - Iterables.limit(keyState.localAdditions, keyState.localAdditions.size()); - if (keyState.removedLocally) { - // this key has been removed locally but the removal hasn't been sent to windmill, - // thus values in windmill(if any) are obsolete, and we only care about local values. - return Iterables.unmodifiableIterable(localNewValues); - } - if (keyState.valuesCached || complete) { - return Iterables.unmodifiableIterable( - Iterables.concat( - Iterables.limit(keyState.values, keyState.valuesSize), localNewValues)); - } - Future> persistedData = necessaryKeyEntriesFromStorageFuture(key); - try (Closeable scope = scopedReadState()) { - final Iterable persistedValues = persistedData.get(); - // Iterables.isEmpty() is O(1). - if (Iterables.isEmpty(persistedValues)) { - if (keyState.localAdditions.isEmpty()) { - // empty in both cache and windmill, mark key as KNOWN_NONEXISTENT. - keyState.existence = KeyExistence.KNOWN_NONEXISTENT; - return Collections.emptyList(); - } - return Iterables.unmodifiableIterable(localNewValues); - } - keyState.existence = KeyExistence.KNOWN_EXIST; - if (persistedValues instanceof Weighted) { - keyState.valuesCached = true; - ConcatIterables it = new ConcatIterables<>(); - it.extendWith(persistedValues); - keyState.values = it; - keyState.valuesSize = Iterables.size(persistedValues); - } - return Iterables.unmodifiableIterable( - Iterables.concat(persistedValues, localNewValues)); - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read Multimap state", e); - } - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public ReadableState> readLater() { - WindmillMultimap.this.necessaryKeyEntriesFromStorageFuture(key); - return this; - } - }; - } - - @Override - protected WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - if (!cleared && !hasLocalAdditions && !hasLocalRemovals) { - cache.put(namespace, address, this, 1); - return WorkItemCommitRequest.newBuilder().buildPartial(); - } - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - Windmill.TagMultimapUpdateRequest.Builder builder = commitBuilder.addMultimapUpdatesBuilder(); - builder.setTag(stateKey).setStateFamily(stateFamily); - - if (cleared) { - builder.setDeleteAll(true); - } - if (hasLocalRemovals || hasLocalAdditions) { - ByteStringOutputStream keyStream = new ByteStringOutputStream(); - ByteStringOutputStream valueStream = new ByteStringOutputStream(); - Iterator> iterator = keyStateMap.entrySet().iterator(); - while (iterator.hasNext()) { - KeyState keyState = iterator.next().getValue(); - if (!keyState.removedLocally && keyState.localAdditions.isEmpty()) { - if (keyState.existence == KeyExistence.KNOWN_NONEXISTENT) iterator.remove(); - continue; - } - keyCoder.encode(keyState.originalKey, keyStream, Context.OUTER); - ByteString encodedKey = keyStream.toByteStringAndReset(); - Windmill.TagMultimapEntry.Builder entryBuilder = builder.addUpdatesBuilder(); - entryBuilder.setEntryName(encodedKey); - if (keyState.removedLocally) entryBuilder.setDeleteAll(true); - keyState.removedLocally = false; - if (!keyState.localAdditions.isEmpty()) { - for (V value : keyState.localAdditions) { - valueCoder.encode(value, valueStream, Context.OUTER); - ByteString encodedValue = valueStream.toByteStringAndReset(); - entryBuilder.addValues(encodedValue); - } - // Move newly added values from localAdditions to keyState.values as those new values - // now - // are also persisted in Windmill. If a key now has no more values and is not - // KNOWN_EXIST, - // remove it from cache. - if (keyState.valuesCached) { - keyState.values.extendWith(keyState.localAdditions); - keyState.valuesSize += keyState.localAdditions.size(); - } - // Create a new localAdditions so that the cached values are unaffected. - keyState.localAdditions = Lists.newArrayList(); - } - if (!keyState.valuesCached && keyState.existence != KeyExistence.KNOWN_EXIST) { - iterator.remove(); - } - } - } - - hasLocalAdditions = false; - hasLocalRemovals = false; - cleared = false; - - cache.put(namespace, address, this, 1); - return commitBuilder.buildPartial(); - } - - @Override - public void remove(K key) { - final Object structuralKey = keyCoder.structuralValue(key); - // does not insert key if allKeysKnown. - KeyState keyState = - keyStateMap.computeIfAbsent(structuralKey, k -> allKeysKnown ? null : new KeyState(key)); - if (keyState == null || keyState.existence == KeyExistence.KNOWN_NONEXISTENT) { - return; - } - if (keyState.valuesCached && keyState.valuesSize == 0) { - // no data in windmill, deleting from local cache is sufficient. - keyStateMap.remove(structuralKey); - } else { - // there may be data in windmill that need to be removed. - hasLocalRemovals = true; - keyState.removedLocally = true; - keyState.values = new ConcatIterables<>(); - keyState.valuesSize = 0; - keyState.existence = KeyExistence.KNOWN_NONEXISTENT; - } - if (!keyState.localAdditions.isEmpty()) { - keyState.localAdditions = Lists.newArrayList(); - } - keyState.valuesCached = true; - } - - @Override - public void clear() { - keyStateMap = Maps.newHashMap(); - cleared = true; - complete = true; - allKeysKnown = true; - hasLocalAdditions = false; - hasLocalRemovals = false; - } - - @Override - public ReadableState> keys() { - return new ReadableState>() { - - private Map cachedExistKeys() { - return keyStateMap.entrySet().stream() - .filter(entry -> entry.getValue().existence == KeyExistence.KNOWN_EXIST) - .collect(Collectors.toMap(Entry::getKey, e -> e.getValue().originalKey)); - } - - @Override - public Iterable read() { - if (allKeysKnown) { - return Iterables.unmodifiableIterable(cachedExistKeys().values()); - } - Future>>> persistedData = - necessaryEntriesFromStorageFuture(true); - try (Closeable scope = scopedReadState()) { - Iterable>> entries = persistedData.get(); - if (entries instanceof Weighted) { - // This is a known amount of data, cache them all. - entries.forEach( - entry -> { - try { - K originalKey = keyCoder.decode(entry.getKey().newInput(), Context.OUTER); - KeyState keyState = - keyStateMap.computeIfAbsent( - keyCoder.structuralValue(originalKey), - stk -> new KeyState(originalKey)); - if (keyState.existence == KeyExistence.UNKNOWN_EXISTENCE) { - keyState.existence = KeyExistence.KNOWN_EXIST; - } - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - allKeysKnown = true; - keyStateMap - .values() - .removeIf( - keyState -> - keyState.existence != KeyExistence.KNOWN_EXIST - && !keyState.removedLocally); - return Iterables.unmodifiableIterable(cachedExistKeys().values()); - } else { - Map cachedExistKeys = Maps.newHashMap(); - Set cachedNonExistKeys = Sets.newHashSet(); - keyStateMap.forEach( - (structuralKey, keyState) -> { - switch (keyState.existence) { - case KNOWN_EXIST: - cachedExistKeys.put(structuralKey, keyState.originalKey); - break; - case KNOWN_NONEXISTENT: - cachedNonExistKeys.add(structuralKey); - break; - default: - break; - } - }); - // keysOnlyInWindmill is lazily loaded. - Iterable keysOnlyInWindmill = - Iterables.filter( - Iterables.transform( - entries, - entry -> { - try { - K originalKey = - keyCoder.decode(entry.getKey().newInput(), Context.OUTER); - Object structuralKey = keyCoder.structuralValue(originalKey); - if (cachedExistKeys.containsKey(structuralKey) - || cachedNonExistKeys.contains(structuralKey)) return null; - return originalKey; - } catch (IOException e) { - throw new RuntimeException(e); - } - }), - Objects::nonNull); - return Iterables.unmodifiableIterable( - Iterables.concat(cachedExistKeys.values(), keysOnlyInWindmill)); - } - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public ReadableState> readLater() { - WindmillMultimap.this.necessaryEntriesFromStorageFuture(true); - return this; - } - }; - } - - @Override - public ReadableState>> entries() { - return new ReadableState>>() { - @Override - public Iterable> read() { - if (complete) { - return Iterables.unmodifiableIterable( - unnestCachedEntries(mergedCachedEntries(null).entrySet())); - } - Future>>> persistedData = - necessaryEntriesFromStorageFuture(false); - try (Closeable scope = scopedReadState()) { - Iterable>> entries = persistedData.get(); - if (Iterables.isEmpty(entries)) { - complete = true; - allKeysKnown = true; - return Iterables.unmodifiableIterable( - unnestCachedEntries(mergedCachedEntries(null).entrySet())); - } - if (!(entries instanceof Weighted)) { - return nonWeightedEntries(entries); - } - // This is a known amount of data, cache them all. - entries.forEach( - entry -> { - try { - final K originalKey = keyCoder.decode(entry.getKey().newInput(), Context.OUTER); - final Object structuralKey = keyCoder.structuralValue(originalKey); - KeyState keyState = - keyStateMap.computeIfAbsent(structuralKey, k -> new KeyState(originalKey)); - // Ignore any key from windmill that has been marked pending deletion or is - // fully cached. - if (keyState.existence == KeyExistence.KNOWN_NONEXISTENT - || (keyState.existence == KeyExistence.KNOWN_EXIST - && keyState.valuesCached)) return; - // Or else cache contents from windmill. - keyState.existence = KeyExistence.KNOWN_EXIST; - keyState.values.extendWith(entry.getValue()); - keyState.valuesSize += Iterables.size(entry.getValue()); - keyState.valuesCached = true; - } catch (IOException e) { - throw new RuntimeException(e); - } - }); - allKeysKnown = true; - complete = true; - return Iterables.unmodifiableIterable( - unnestCachedEntries(mergedCachedEntries(null).entrySet())); - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public ReadableState>> readLater() { - WindmillMultimap.this.necessaryEntriesFromStorageFuture(false); - return this; - } - - // Collect all cached entries into a map and all KNOWN_NONEXISTENT keys to - // knownNonexistentKeys(if not null). Note that this method is not side-effect-free: it - // unloads any key that is not KNOWN_EXIST and not pending deletion from cache; also if - // complete it marks the valuesCached of any key that is KNOWN_EXIST to true, entries() - // depends on this behavior when the fetched result is weighted to iterate the whole - // keyStateMap one less time. For each cached key, returns its structural key and a tuple of - // . - private Map>> mergedCachedEntries( - Set knownNonexistentKeys) { - Map>> cachedEntries = Maps.newHashMap(); - keyStateMap - .entrySet() - .removeIf( - (entry -> { - Object structuralKey = entry.getKey(); - KeyState keyState = entry.getValue(); - if (complete && keyState.existence == KeyExistence.KNOWN_EXIST) { - keyState.valuesCached = true; - } - ConcatIterables it = null; - if (!keyState.localAdditions.isEmpty()) { - it = new ConcatIterables<>(); - it.extendWith( - Iterables.limit(keyState.localAdditions, keyState.localAdditions.size())); - } - if (keyState.valuesCached) { - if (it == null) it = new ConcatIterables<>(); - it.extendWith(Iterables.limit(keyState.values, keyState.valuesSize)); - } - if (it != null) { - cachedEntries.put( - structuralKey, - Triple.of(keyState.originalKey, keyState.valuesCached, it)); - } - if (knownNonexistentKeys != null - && keyState.existence == KeyExistence.KNOWN_NONEXISTENT) - knownNonexistentKeys.add(structuralKey); - return (keyState.existence == KeyExistence.KNOWN_NONEXISTENT - && !keyState.removedLocally) - || keyState.existence == KeyExistence.UNKNOWN_EXISTENCE; - })); - return cachedEntries; - } - - private Iterable> unnestCachedEntries( - Iterable>>> cachedEntries) { - return Iterables.concat( - Iterables.transform( - cachedEntries, - entry -> - Iterables.transform( - entry.getValue().getRight(), - v -> new AbstractMap.SimpleEntry<>(entry.getValue().getLeft(), v)))); - } - - private Iterable> nonWeightedEntries( - Iterable>> lazyWindmillEntries) { - class ResultIterable implements Iterable> { - private final Iterable>> lazyWindmillEntries; - private final Map>> cachedEntries; - private final Set knownNonexistentKeys; - - ResultIterable( - Map>> cachedEntries, - Iterable>> lazyWindmillEntries, - Set knownNonexistentKeys) { - this.cachedEntries = cachedEntries; - this.lazyWindmillEntries = lazyWindmillEntries; - this.knownNonexistentKeys = knownNonexistentKeys; - } - - @Override - public Iterator> iterator() { - // Each time when the Iterable returned by entries() is iterated, a new Iterator is - // created. Every iterator must keep its own copy of seenCachedKeys so that if a key - // is paginated into multiple iterables from windmill, the cached values of this key - // will only be returned once. - Set seenCachedKeys = Sets.newHashSet(); - // notFullyCachedEntries returns all entries from windmill that are not fully cached - // and combines them with localAdditions. If a key is fully cached, contents of this - // key from windmill are ignored. - Iterable>> notFullyCachedEntries = - Iterables.filter( - Iterables.transform( - lazyWindmillEntries, - entry -> { - try { - final K key = - keyCoder.decode(entry.getKey().newInput(), Context.OUTER); - final Object structuralKey = keyCoder.structuralValue(key); - // key is deleted in cache thus fully cached. - if (knownNonexistentKeys.contains(structuralKey)) return null; - Triple> triple = - cachedEntries.get(structuralKey); - // no record of key in cache, return content in windmill. - if (triple == null) { - return Triple.of(structuralKey, key, entry.getValue()); - } - // key is fully cached in cache. - if (triple.getMiddle()) return null; - - // key is not fully cached, combine the content in windmill with local - // additions with only the first observed page for the key to ensure - // it is not repeated. - if (!seenCachedKeys.add(structuralKey)) { - return Triple.of(structuralKey, key, entry.getValue()); - } else { - ConcatIterables it = new ConcatIterables<>(); - it.extendWith(triple.getRight()); - it.extendWith(entry.getValue()); - return Triple.of(structuralKey, key, it); - } - } catch (IOException e) { - throw new RuntimeException(e); - } - }), - Objects::nonNull); - Iterator> unnestWindmill = - Iterators.concat( - Iterables.transform( - notFullyCachedEntries, - entry -> - Iterables.transform( - entry.getRight(), - v -> new AbstractMap.SimpleEntry<>(entry.getMiddle(), v)) - .iterator()) - .iterator()); - Iterator> fullyCached = - unnestCachedEntries( - Iterables.filter( - cachedEntries.entrySet(), - entry -> !seenCachedKeys.contains(entry.getKey()))) - .iterator(); - return Iterators.concat(unnestWindmill, fullyCached); - } - } - - Set knownNonexistentKeys = Sets.newHashSet(); - Map>> cachedEntries = - mergedCachedEntries(knownNonexistentKeys); - return Iterables.unmodifiableIterable( - new ResultIterable(cachedEntries, lazyWindmillEntries, knownNonexistentKeys)); - } - }; - } - - @Override - public ReadableState containsKey(K key) { - return new ReadableState() { - ReadableState> values = null; - final Object structuralKey = keyCoder.structuralValue(key); - - @Override - public Boolean read() { - KeyState keyState = keyStateMap.getOrDefault(structuralKey, null); - if (keyState != null && keyState.existence != KeyExistence.UNKNOWN_EXISTENCE) { - return keyState.existence == KeyExistence.KNOWN_EXIST; - } - if (values == null) { - values = WindmillMultimap.this.get(key); - } - return !Iterables.isEmpty(values.read()); - } - - @Override - public ReadableState readLater() { - if (values == null) { - values = WindmillMultimap.this.get(key); - } - values.readLater(); - return this; - } - }; - } - - // Currently, isEmpty is implemented by reading all keys and could potentially be optimized. - // But note that if isEmpty is often followed by iterating over keys then maybe not too bad; if - // isEmpty is followed by iterating over both keys and values then it won't help much. - @Override - public ReadableState isEmpty() { - return new ReadableState() { - ReadableState> keys = null; - - @Override - public Boolean read() { - for (KeyState keyState : keyStateMap.values()) { - if (keyState.existence == KeyExistence.KNOWN_EXIST) return false; - } - if (keys == null) { - keys = WindmillMultimap.this.keys(); - } - return Iterables.isEmpty(keys.read()); - } - - @Override - public ReadableState readLater() { - if (keys == null) { - keys = WindmillMultimap.this.keys(); - } - keys.readLater(); - return this; - } - }; - } - } - - private static class WindmillBag extends SimpleWindmillState implements BagState { - - private final StateNamespace namespace; - private final StateTag> address; - private final ByteString stateKey; - private final String stateFamily; - private final Coder elemCoder; - - private boolean cleared = false; - /** - * If non-{@literal null}, this contains the complete contents of the bag, except for any local - * additions. If {@literal null} then we don't know if Windmill contains additional values which - * should be part of the bag. We'll need to read them if the work item actually wants the bag - * contents. - */ - private ConcatIterables cachedValues = null; - - private List localAdditions = new ArrayList<>(); - private long encodedSize = 0; - - private WindmillBag( - StateNamespace namespace, - StateTag> address, - String stateFamily, - Coder elemCoder, - boolean isNewKey) { - this.namespace = namespace; - this.address = address; - this.stateKey = encodeKey(namespace, address); - this.stateFamily = stateFamily; - this.elemCoder = elemCoder; - if (isNewKey) { - this.cachedValues = new ConcatIterables<>(); - } - } - - @Override - public void clear() { - cleared = true; - cachedValues = new ConcatIterables<>(); - localAdditions = new ArrayList<>(); - encodedSize = 0; - } - - /** - * Return iterable over all bag values in Windmill which should contribute to overall bag - * contents. - */ - private Iterable fetchData(Future> persistedData) { - try (Closeable scope = scopedReadState()) { - if (cachedValues != null) { - return cachedValues.snapshot(); - } - Iterable data = persistedData.get(); - if (data instanceof Weighted) { - // We have a known bounded amount of data; cache it. - cachedValues = new ConcatIterables<>(); - cachedValues.extendWith(data); - encodedSize = ((Weighted) data).getWeight(); - return cachedValues.snapshot(); - } else { - // This is an iterable that may not fit in memory at once; don't cache it. - return data; - } - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - } - - public boolean valuesAreCached() { - return cachedValues != null; - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public WindmillBag readLater() { - getFuture(); - return this; - } - - @Override - public Iterable read() { - return Iterables.concat( - fetchData(getFuture()), Iterables.limit(localAdditions, localAdditions.size())); - } - - @Override - public ReadableState isEmpty() { - return new ReadableState() { - @Override - public ReadableState readLater() { - WindmillBag.this.readLater(); - return this; - } - - @Override - public Boolean read() { - return Iterables.isEmpty(fetchData(getFuture())) && localAdditions.isEmpty(); - } - }; - } - - @Override - public void add(T input) { - localAdditions.add(input); - } - - @Override - public WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - - Windmill.TagBag.Builder bagUpdatesBuilder = null; - - if (cleared) { - bagUpdatesBuilder = commitBuilder.addBagUpdatesBuilder(); - bagUpdatesBuilder.setDeleteAll(true); - cleared = false; - } - - if (!localAdditions.isEmpty()) { - // Tell Windmill to capture the local additions. - if (bagUpdatesBuilder == null) { - bagUpdatesBuilder = commitBuilder.addBagUpdatesBuilder(); - } - for (T value : localAdditions) { - ByteStringOutputStream stream = new ByteStringOutputStream(); - // Encode the value - elemCoder.encode(value, stream, Coder.Context.OUTER); - ByteString encoded = stream.toByteString(); - if (cachedValues != null) { - // We'll capture this value in the cache below. - // Capture the value's size now since we have it. - encodedSize += encoded.size(); - } - bagUpdatesBuilder.addValues(encoded); - } - } - - if (bagUpdatesBuilder != null) { - bagUpdatesBuilder.setTag(stateKey).setStateFamily(stateFamily); - } - - if (cachedValues != null) { - if (!localAdditions.isEmpty()) { - // Capture the local additions in the cached value since we and - // Windmill are now in agreement. - cachedValues.extendWith(localAdditions); - } - // We now know the complete bag contents, and any read on it will yield a - // cached value, so cache it for future reads. - cache.put(namespace, address, this, encodedSize); - } - - // Don't reuse the localAdditions object; we don't want future changes to it to - // modify the value of cachedValues. - localAdditions = new ArrayList<>(); - - return commitBuilder.buildPartial(); - } - - private Future> getFuture() { - return cachedValues != null ? null : reader.bagFuture(stateKey, stateFamily, elemCoder); - } - } - - private static class ConcatIterables implements Iterable { - // List of component iterables. Should only be appended to in order to support snapshot(). - List> iterables; - - public ConcatIterables() { - this.iterables = new ArrayList<>(); - } - - public void extendWith(Iterable iterable) { - iterables.add(iterable); - } - - @Override - public Iterator iterator() { - return Iterators.concat(Iterables.transform(iterables, Iterable::iterator).iterator()); - } - - /** - * Returns a view of the current state of this iterable. Remembers the current length of - * iterables so that the returned value Will not change due to future extendWith() calls. - */ - public Iterable snapshot() { - final int limit = iterables.size(); - final List> iterablesList = iterables; - return () -> - Iterators.concat( - Iterators.transform( - Iterators.limit(iterablesList.iterator(), limit), Iterable::iterator)); - } - } - - private static class WindmillWatermarkHold extends WindmillState implements WatermarkHoldState { - // The encoded size of an Instant. - private static final int ENCODED_SIZE = 8; - - private final TimestampCombiner timestampCombiner; - private final StateNamespace namespace; - private final StateTag address; - private final ByteString stateKey; - private final String stateFamily; - - private boolean cleared = false; - /** - * If non-{@literal null}, the known current hold value, or absent if we know there are no - * output watermark holds. If {@literal null}, the current hold value could depend on holds in - * Windmill we do not yet know. - */ - private Optional cachedValue = null; - - private Instant localAdditions = null; - - private WindmillWatermarkHold( - StateNamespace namespace, - StateTag address, - String stateFamily, - TimestampCombiner timestampCombiner, - boolean isNewKey) { - this.namespace = namespace; - this.address = address; - this.stateKey = encodeKey(namespace, address); - this.stateFamily = stateFamily; - this.timestampCombiner = timestampCombiner; - if (isNewKey) { - cachedValue = Optional.absent(); - } - } - - @Override - public void clear() { - cleared = true; - cachedValue = Optional.absent(); - localAdditions = null; - } - - @Override - @SuppressWarnings("FutureReturnValueIgnored") - public WindmillWatermarkHold readLater() { - getFuture(); - return this; - } - - @Override - public Instant read() { - try (Closeable scope = scopedReadState()) { - Instant persistedHold = getFuture().get(); - if (persistedHold == null) { - cachedValue = Optional.absent(); - } else { - cachedValue = Optional.of(persistedHold); - } - } catch (InterruptedException | ExecutionException | IOException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read state", e); - } - - if (localAdditions == null) { - return cachedValue.orNull(); - } else if (!cachedValue.isPresent()) { - return localAdditions; - } else { - return timestampCombiner.combine(localAdditions, cachedValue.get()); - } - } - - @Override - public ReadableState isEmpty() { - throw new UnsupportedOperationException(); - } - - @Override - public void add(Instant outputTime) { - localAdditions = - (localAdditions == null) - ? outputTime - : timestampCombiner.combine(outputTime, localAdditions); - } - - @Override - public TimestampCombiner getTimestampCombiner() { - return timestampCombiner; - } - - @Override - public Future persist(final WindmillStateCache.ForKeyAndFamily cache) { - - Future result; - - if (!cleared && localAdditions == null) { - // No changes, so no need to update Windmill and no need to cache any value. - return Futures.immediateFuture(WorkItemCommitRequest.newBuilder().buildPartial()); - } - - if (cleared && localAdditions == null) { - // Just clearing the persisted state; blind delete - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - commitBuilder - .addWatermarkHoldsBuilder() - .setTag(stateKey) - .setStateFamily(stateFamily) - .setReset(true); - - result = Futures.immediateFuture(commitBuilder.buildPartial()); - } else if (cleared && localAdditions != null) { - // Since we cleared before adding, we can do a blind overwrite of persisted state - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - commitBuilder - .addWatermarkHoldsBuilder() - .setTag(stateKey) - .setStateFamily(stateFamily) - .setReset(true) - .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); - - cachedValue = Optional.of(localAdditions); - - result = Futures.immediateFuture(commitBuilder.buildPartial()); - } else if (!cleared && localAdditions != null) { - // Otherwise, we need to combine the local additions with the already persisted data - result = combineWithPersisted(); - } else { - throw new IllegalStateException("Unreachable condition"); - } - - return Futures.lazyTransform( - result, - result1 -> { - cleared = false; - localAdditions = null; - if (cachedValue != null) { - cache.put(namespace, address, WindmillWatermarkHold.this, ENCODED_SIZE); - } - return result1; - }); - } - - private Future getFuture() { - return cachedValue != null - ? Futures.immediateFuture(cachedValue.orNull()) - : reader.watermarkFuture(stateKey, stateFamily); - } - - /** - * Combines local additions with persisted data and mutates the {@code commitBuilder} to write - * the result. - */ - private Future combineWithPersisted() { - boolean windmillCanCombine = false; - - // If the combined output time depends only on the window, then we are just blindly adding - // the same value that may or may not already be present. This depends on the state only being - // used for one window. - windmillCanCombine |= timestampCombiner.dependsOnlyOnWindow(); - - // If the combined output time depends only on the earliest input timestamp, then because - // assignOutputTime is monotonic, the hold only depends on the earliest output timestamp - // (which is the value submitted as a watermark hold). The only way holds for later inputs - // can be redundant is if the are later (or equal) to the earliest. So taking the MIN - // implicitly, as Windmill does, has the desired behavior. - windmillCanCombine |= timestampCombiner.dependsOnlyOnEarliestTimestamp(); - - if (windmillCanCombine) { - // We do a blind write and let Windmill take the MIN - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - commitBuilder - .addWatermarkHoldsBuilder() - .setTag(stateKey) - .setStateFamily(stateFamily) - .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); - - if (cachedValue != null) { - cachedValue = - Optional.of( - cachedValue.isPresent() - ? timestampCombiner.combine(cachedValue.get(), localAdditions) - : localAdditions); - } - - return Futures.immediateFuture(commitBuilder.buildPartial()); - } else { - // The non-fast path does a read-modify-write - return Futures.lazyTransform( - (cachedValue != null) - ? Futures.immediateFuture(cachedValue.orNull()) - : reader.watermarkFuture(stateKey, stateFamily), - priorHold -> { - cachedValue = - Optional.of( - (priorHold != null) - ? timestampCombiner.combine(priorHold, localAdditions) - : localAdditions); - WorkItemCommitRequest.Builder commitBuilder = WorkItemCommitRequest.newBuilder(); - commitBuilder - .addWatermarkHoldsBuilder() - .setTag(stateKey) - .setStateFamily(stateFamily) - .setReset(true) - .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(cachedValue.get())); - - return commitBuilder.buildPartial(); - }); - } - } - } - - private static class WindmillCombiningState extends WindmillState - implements CombiningState { - - private final WindmillBag bag; - private final CombineFn combineFn; - - /* We use a separate, in-memory AccumT rather than relying on the WindmillWatermarkBag's - * localAdditions, because we want to combine multiple InputT's to a single AccumT - * before adding it. - */ - private AccumT localAdditionsAccum; - private boolean hasLocalAdditions = false; - - private WindmillCombiningState( - StateNamespace namespace, - StateTag> address, - String stateFamily, - Coder accumCoder, - CombineFn combineFn, - WindmillStateCache.ForKeyAndFamily cache, - boolean isNewKey) { - StateTag> internalBagAddress = StateTags.convertToBagTagInternal(address); - WindmillBag cachedBag = - (WindmillBag) cache.get(namespace, internalBagAddress); - this.bag = - (cachedBag != null) - ? cachedBag - : new WindmillBag<>(namespace, internalBagAddress, stateFamily, accumCoder, isNewKey); - this.combineFn = combineFn; - this.localAdditionsAccum = combineFn.createAccumulator(); - } - - @Override - void initializeForWorkItem( - WindmillStateReader reader, Supplier scopedReadStateSupplier) { - super.initializeForWorkItem(reader, scopedReadStateSupplier); - this.bag.initializeForWorkItem(reader, scopedReadStateSupplier); - } - - @Override - void cleanupAfterWorkItem() { - super.cleanupAfterWorkItem(); - bag.cleanupAfterWorkItem(); - } - - @Override - public WindmillCombiningState readLater() { - bag.readLater(); - return this; - } - - @Override - public OutputT read() { - return combineFn.extractOutput(getAccum()); - } - - @Override - public void add(InputT input) { - hasLocalAdditions = true; - localAdditionsAccum = combineFn.addInput(localAdditionsAccum, input); - } - - @Override - public void clear() { - bag.clear(); - localAdditionsAccum = combineFn.createAccumulator(); - hasLocalAdditions = false; - } - - @Override - public Future persist(WindmillStateCache.ForKeyAndFamily cache) - throws IOException { - if (hasLocalAdditions) { - if (COMPACT_NOW.get().get() || bag.valuesAreCached()) { - // Implicitly clears the bag and combines local and persisted accumulators. - localAdditionsAccum = getAccum(); - } - bag.add(combineFn.compact(localAdditionsAccum)); - localAdditionsAccum = combineFn.createAccumulator(); - hasLocalAdditions = false; - } - - return bag.persist(cache); - } - - @Override - public AccumT getAccum() { - Iterable accums = - Iterables.concat(bag.read(), Collections.singleton(localAdditionsAccum)); - - // Compact things - AccumT merged = combineFn.mergeAccumulators(accums); - bag.clear(); - localAdditionsAccum = merged; - hasLocalAdditions = true; - return merged; - } - - @Override - public ReadableState isEmpty() { - final ReadableState bagIsEmpty = bag.isEmpty(); - return new ReadableState() { - @Override - public ReadableState readLater() { - bagIsEmpty.readLater(); - return this; - } - - @Override - public Boolean read() { - return !hasLocalAdditions && bagIsEmpty.read(); - } - }; - } - - @Override - public void addAccum(AccumT accum) { - hasLocalAdditions = true; - localAdditionsAccum = combineFn.mergeAccumulators(Arrays.asList(localAdditionsAccum, accum)); - } - - @Override - public AccumT mergeAccumulators(Iterable accumulators) { - return combineFn.mergeAccumulators(accumulators); - } - } - - @VisibleForTesting - static final ThreadLocal> COMPACT_NOW = - ThreadLocal.withInitial( - () -> - new Supplier() { - /* The rate at which, on average, this will return true. */ - static final double RATE = 0.002; - Random random = new Random(); - long counter = nextSample(); - - private long nextSample() { - // Use geometric distribution to find next true value. - // This lets us avoid invoking random.nextDouble() on every call. - return (long) Math.floor(Math.log(random.nextDouble()) / Math.log(1 - RATE)); - } - - @Override - public Boolean get() { - counter--; - if (counter < 0) { - counter = nextSample(); - return true; - } else { - return false; - } - } - }); -} 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 529bb0a41907..9858666c40a2 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 @@ -32,9 +32,9 @@ import javax.annotation.Nullable; import javax.annotation.concurrent.GuardedBy; import javax.annotation.concurrent.ThreadSafe; -import org.apache.beam.runners.dataflow.worker.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java index a902d2b13a77..9d7a9131f584 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/streaming/ComputationState.java @@ -23,9 +23,9 @@ import java.util.Map; import java.util.concurrent.ConcurrentLinkedQueue; import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.worker.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.util.BoundedQueueExecutor; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/CachingStateTable.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/CachingStateTable.java new file mode 100644 index 000000000000..bcaf8bf21a2d --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/CachingStateTable.java @@ -0,0 +1,275 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.io.Closeable; +import java.util.Optional; +import javax.annotation.Nullable; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTable; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.core.StateTags; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.MultimapState; +import org.apache.beam.sdk.state.OrderedListState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.state.WatermarkHoldState; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.sdk.transforms.CombineWithContext; +import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; +import org.apache.beam.sdk.util.CombineFnUtil; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; + +final class CachingStateTable extends StateTable { + private final String stateFamily; + private final WindmillStateReader reader; + private final WindmillStateCache.ForKeyAndFamily cache; + private final boolean isSystemTable; + private final Supplier scopedReadStateSupplier; + private final @Nullable StateTable derivedStateTable; + private final boolean isNewKey; + + private CachingStateTable(Builder builder) { + this.stateFamily = builder.stateFamily; + this.reader = builder.reader; + this.cache = builder.cache; + this.isSystemTable = builder.isSystemTable; + this.isNewKey = builder.isNewKey; + this.scopedReadStateSupplier = builder.scopedReadStateSupplier; + this.derivedStateTable = builder.derivedStateTable; + + if (this.isSystemTable) { + Preconditions.checkState(derivedStateTable == null); + } else { + Preconditions.checkNotNull(this.derivedStateTable); + } + } + + static CachingStateTable.Builder builder( + String stateFamily, + WindmillStateReader reader, + WindmillStateCache.ForKeyAndFamily cache, + boolean isNewKey, + Supplier scopedReadStateSupplier) { + return new CachingStateTable.Builder( + stateFamily, reader, cache, scopedReadStateSupplier, isNewKey); + } + + @Override + @SuppressWarnings("deprecation") + protected StateTag.StateBinder binderForNamespace(StateNamespace namespace, StateContext c) { + // Look up state objects in the cache or create new ones if not found. The state will + // be added to the cache in persist(). + return new StateTag.StateBinder() { + @Override + public BagState bindBag(StateTag> address, Coder elemCoder) { + StateTag> resolvedAddress = + isSystemTable ? StateTags.makeSystemTagInternal(address) : address; + + WindmillBag result = + cache + .get(namespace, resolvedAddress) + .map(bagState -> (WindmillBag) bagState) + .orElseGet( + () -> + new WindmillBag<>( + namespace, resolvedAddress, stateFamily, elemCoder, isNewKey)); + + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public SetState bindSet(StateTag> spec, Coder elemCoder) { + WindmillSet result = + new WindmillSet<>(namespace, spec, stateFamily, elemCoder, cache, isNewKey); + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public MapState bindMap( + StateTag> spec, Coder keyCoder, Coder valueCoder) { + WindmillMap result = + cache + .get(namespace, spec) + .map(mapState -> (WindmillMap) mapState) + .orElseGet( + () -> + new WindmillMap<>( + namespace, spec, stateFamily, keyCoder, valueCoder, isNewKey)); + + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public MultimapState bindMultimap( + StateTag> spec, + Coder keyCoder, + Coder valueCoder) { + WindmillMultimap result = + cache + .get(namespace, spec) + .map(multimapState -> (WindmillMultimap) multimapState) + .orElseGet( + () -> + new WindmillMultimap<>( + namespace, spec, stateFamily, keyCoder, valueCoder, isNewKey)); + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public OrderedListState bindOrderedList( + StateTag> spec, Coder elemCoder) { + StateTag> specOrInternalTag = addressOrInternalTag(spec); + + WindmillOrderedList result = + cache + .get(namespace, specOrInternalTag) + .map(orderedList -> (WindmillOrderedList) orderedList) + .orElseGet( + () -> + new WindmillOrderedList<>( + Optional.ofNullable(derivedStateTable).orElse(CachingStateTable.this), + namespace, + specOrInternalTag, + stateFamily, + elemCoder, + isNewKey)); + + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public WatermarkHoldState bindWatermark( + StateTag address, TimestampCombiner timestampCombiner) { + StateTag addressOrInternalTag = addressOrInternalTag(address); + + WindmillWatermarkHold result = + cache + .get(namespace, addressOrInternalTag) + .map(watermarkHold -> (WindmillWatermarkHold) watermarkHold) + .orElseGet( + () -> + new WindmillWatermarkHold( + namespace, address, stateFamily, timestampCombiner, isNewKey)); + + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public CombiningState bindCombiningValue( + StateTag> address, + Coder accumCoder, + Combine.CombineFn combineFn) { + StateTag> addressOrInternalTag = + addressOrInternalTag(address); + + WindmillCombiningState result = + new WindmillCombiningState<>( + namespace, + addressOrInternalTag, + stateFamily, + accumCoder, + combineFn, + cache, + isNewKey); + + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + @Override + public + CombiningState bindCombiningValueWithContext( + StateTag> address, + Coder accumCoder, + CombineWithContext.CombineFnWithContext combineFn) { + return bindCombiningValue( + addressOrInternalTag(address), accumCoder, CombineFnUtil.bindContext(combineFn, c)); + } + + @Override + public ValueState bindValue(StateTag> address, Coder coder) { + StateTag> addressOrInternalTag = addressOrInternalTag(address); + + WindmillValue result = + cache + .get(namespace, addressOrInternalTag) + .map(value -> (WindmillValue) value) + .orElseGet( + () -> + new WindmillValue<>( + namespace, addressOrInternalTag, stateFamily, coder, isNewKey)); + + result.initializeForWorkItem(reader, scopedReadStateSupplier); + return result; + } + + private StateTag addressOrInternalTag(StateTag address) { + return isSystemTable ? StateTags.makeSystemTagInternal(address) : address; + } + }; + } + + static class Builder { + private final String stateFamily; + private final WindmillStateReader reader; + private final WindmillStateCache.ForKeyAndFamily cache; + private final Supplier scopedReadStateSupplier; + private final boolean isNewKey; + private boolean isSystemTable; + private @Nullable StateTable derivedStateTable; + + private Builder( + String stateFamily, + WindmillStateReader reader, + WindmillStateCache.ForKeyAndFamily cache, + Supplier scopedReadStateSupplier, + boolean isNewKey) { + this.stateFamily = stateFamily; + this.reader = reader; + this.cache = cache; + this.scopedReadStateSupplier = scopedReadStateSupplier; + this.isNewKey = isNewKey; + this.isSystemTable = true; + this.derivedStateTable = null; + } + + Builder withDerivedState(StateTable derivedStateTable) { + this.isSystemTable = false; + this.derivedStateTable = derivedStateTable; + return this; + } + + CachingStateTable build() { + return new CachingStateTable(this); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ConcatIterables.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ConcatIterables.java new file mode 100644 index 000000000000..4bb806bd70fd --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ConcatIterables.java @@ -0,0 +1,55 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; +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.Iterators; + +class ConcatIterables implements Iterable { + // List of component iterables. Should only be appended to in order to support snapshot(). + private final List> iterables; + + public ConcatIterables() { + this.iterables = new ArrayList<>(); + } + + public void extendWith(Iterable iterable) { + iterables.add(iterable); + } + + @Override + public Iterator iterator() { + return Iterators.concat(Iterables.transform(iterables, Iterable::iterator).iterator()); + } + + /** + * Returns a view of the current state of this iterable. Remembers the current length of iterables + * so that the returned value Will not change due to future extendWith() calls. + */ + public Iterable snapshot() { + final int limit = iterables.size(); + final List> iterablesList = iterables; + return () -> + Iterators.concat( + Iterators.transform( + Iterators.limit(iterablesList.iterator(), limit), Iterable::iterator)); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/IdTracker.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/IdTracker.java new file mode 100644 index 000000000000..5090626ae8ee --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/IdTracker.java @@ -0,0 +1,253 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.util.Comparator; +import java.util.Iterator; +import java.util.Map; +import java.util.SortedSet; +import java.util.concurrent.ExecutionException; +import java.util.function.BiConsumer; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTable; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.core.StateTags; +import org.apache.beam.sdk.coders.InstantCoder; +import org.apache.beam.sdk.coders.MapCoder; +import org.apache.beam.sdk.coders.VarLongCoder; +import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +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.Maps; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.RangeSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.TreeRangeSet; +import org.joda.time.Duration; +import org.joda.time.Instant; + +/** + * Tracker for the ids used in an ordered list. + * + *

Windmill accepts an int64 id for each timestamped-element in the list. Unique elements are + * identified by the pair of timestamp and id. This means that tow unique elements e1, e2 must have + * different (ts1, id1), (ts2, id2) pairs. To accomplish this we bucket time into five-minute + * buckets, and store a free list of ids available for each bucket. + * + *

When a timestamp range is deleted, we remove id tracking for elements in that range. In order + * to handle the case where a range is deleted piecemeal, we track sub-range deletions for each + * range. For example: + * + *

12:00 - 12:05 ids 12:05 - 12:10 ids + * + *

delete 12:00-12:06 + * + *

12:00 - 12:05 *removed* 12:05 - 12:10 ids subranges deleted 12:05-12:06 + * + *

delete 12:06 - 12:07 + * + *

12:05 - 12:10 ids subranges deleted 12:05-12:07 + * + *

delete 12:07 - 12:10 + * + *

12:05 - 12:10 *removed* + */ +@SuppressWarnings("nullness" // TODO(https://github.com/apache/beam/issues/20497) +) +final class IdTracker { + @VisibleForTesting static final String IDS_AVAILABLE_STR = "IdsAvailable"; + @VisibleForTesting static final String DELETIONS_STR = "Deletions"; + // Note that this previously was Long.MIN_VALUE but ids are unsigned when + // sending to windmill for Streaming Engine. For updated appliance + // pipelines with existing state, there may be negative ids. + @VisibleForTesting static final long NEW_RANGE_MIN_ID = 0; + + @VisibleForTesting + static final MapCoder, RangeSet> IDS_AVAILABLE_CODER = + MapCoder.of(new RangeCoder<>(InstantCoder.of()), new RangeSetCoder<>(VarLongCoder.of())); + + @VisibleForTesting + static final MapCoder, RangeSet> SUBRANGE_DELETIONS_CODER = + MapCoder.of(new RangeCoder<>(InstantCoder.of()), new RangeSetCoder<>(InstantCoder.of())); + + private static final long NEW_RANGE_MAX_ID = Long.MAX_VALUE; + // We track ids on five-minute boundaries. + private static final Duration RESOLUTION = Duration.standardMinutes(5); + // A map from five-minute ranges to the set of ids available in that interval. + private final ValueState, RangeSet>> idsAvailableValue; + // If a timestamp-range in the map has been partially cleared, the cleared intervals are stored + // here. + private final ValueState, RangeSet>> subRangeDeletionsValue; + + IdTracker(StateTable stateTable, StateNamespace namespace, StateTag spec) { + StateTag, RangeSet>>> idsAvailableTag = + StateTags.makeSystemTagInternal( + StateTags.value(spec.getId() + IDS_AVAILABLE_STR, IDS_AVAILABLE_CODER)); + StateTag, RangeSet>>> subRangeDeletionsTag = + StateTags.makeSystemTagInternal( + StateTags.value(spec.getId() + DELETIONS_STR, SUBRANGE_DELETIONS_CODER)); + + this.idsAvailableValue = + stateTable.get(namespace, idsAvailableTag, StateContexts.nullContext()); + this.subRangeDeletionsValue = + stateTable.get(namespace, subRangeDeletionsTag, StateContexts.nullContext()); + } + + static > + Map, RangeSet> newSortedRangeMap() { + return Maps.newTreeMap( + Comparator., Instant>comparing(Range::lowerEndpoint) + .thenComparing(Range::upperEndpoint)); + } + + private Range getTrackedRange(Instant ts) { + Instant snapped = + new Instant(ts.getMillis() - ts.plus(RESOLUTION).getMillis() % RESOLUTION.getMillis()); + return Range.closedOpen(snapped, snapped.plus(RESOLUTION)); + } + + @SuppressWarnings("FutureReturnValueIgnored") + void readLater() { + idsAvailableValue.readLater(); + subRangeDeletionsValue.readLater(); + } + + Map, RangeSet> readIdsAvailable() { + Map, RangeSet> idsAvailable = idsAvailableValue.read(); + return idsAvailable != null ? idsAvailable : newSortedRangeMap(); + } + + Map, RangeSet> readSubRangeDeletions() { + Map, RangeSet> subRangeDeletions = subRangeDeletionsValue.read(); + return subRangeDeletions != null ? subRangeDeletions : newSortedRangeMap(); + } + + void clear() throws ExecutionException, InterruptedException { + idsAvailableValue.clear(); + subRangeDeletionsValue.clear(); + } + + void add( + SortedSet> elements, BiConsumer, Long> output) + throws ExecutionException, InterruptedException { + Range currentIdRange = null; + long currentId = 0; + + Range currentTsRange = null; + RangeSet currentTsRangeDeletions = null; + + Map, RangeSet> idsAvailable = readIdsAvailable(); + Map, RangeSet> subRangeDeletions = readSubRangeDeletions(); + + RangeSet availableIdsForTsRange = null; + Iterator> idRangeIter = null; + RangeSet idsUsed = TreeRangeSet.create(); + for (TimestampedValueWithId pendingAdd : elements) { + // Since elements are in increasing ts order, often we'll be able to reuse the previous + // iteration's range. + if (currentTsRange == null + || !currentTsRange.contains(pendingAdd.getValue().getTimestamp())) { + if (availableIdsForTsRange != null) { + // We're moving onto a new ts range. Remove all used ids + availableIdsForTsRange.removeAll(idsUsed); + idsUsed = TreeRangeSet.create(); + } + + // Lookup the range for the current timestamp. + currentTsRange = getTrackedRange(pendingAdd.getValue().getTimestamp()); + // Lookup available ids for this timestamp range. If nothing there, we default to all ids + // available. + availableIdsForTsRange = + idsAvailable.computeIfAbsent( + currentTsRange, + r -> + TreeRangeSet.create( + ImmutableList.of(Range.closedOpen(NEW_RANGE_MIN_ID, NEW_RANGE_MAX_ID)))); + idRangeIter = availableIdsForTsRange.asRanges().iterator(); + currentIdRange = null; + currentTsRangeDeletions = subRangeDeletions.get(currentTsRange); + } + + if (currentIdRange == null || currentId >= currentIdRange.upperEndpoint()) { + // Move to the next range of free ids, and start assigning ranges from there. + currentIdRange = idRangeIter.next(); + currentId = currentIdRange.lowerEndpoint(); + } + + if (currentTsRangeDeletions != null) { + currentTsRangeDeletions.remove( + Range.closedOpen( + pendingAdd.getValue().getTimestamp(), + pendingAdd.getValue().getTimestamp().plus(Duration.millis(1)))); + } + idsUsed.add(Range.closedOpen(currentId, currentId + 1)); + output.accept(pendingAdd.getValue(), currentId++); + } + if (availableIdsForTsRange != null) { + availableIdsForTsRange.removeAll(idsUsed); + } + writeValues(idsAvailable, subRangeDeletions); + } + + // Remove a timestamp range. Returns ids freed up. + void remove(Range tsRange) throws ExecutionException, InterruptedException { + Map, RangeSet> idsAvailable = readIdsAvailable(); + Map, RangeSet> subRangeDeletions = readSubRangeDeletions(); + + for (Range current = getTrackedRange(tsRange.lowerEndpoint()); + current.lowerEndpoint().isBefore(tsRange.upperEndpoint()); + current = getTrackedRange(current.lowerEndpoint().plus(RESOLUTION))) { + // TODO(reuvenlax): shouldn't need to iterate over all ranges. + boolean rangeCleared; + if (!tsRange.encloses(current)) { + // This can happen if the beginning or the end of tsRange doesn't fall on a RESOLUTION + // boundary. Since we are deleting a portion of a tracked range, track what we are deleting. + RangeSet rangeDeletions = + subRangeDeletions.computeIfAbsent(current, r -> TreeRangeSet.create()); + rangeDeletions.add(tsRange.intersection(current)); + // If we ended up deleting the whole range, then we can simply remove it from the tracking + // map. + rangeCleared = rangeDeletions.encloses(current); + } else { + rangeCleared = true; + } + if (rangeCleared) { + // Remove the range from both maps. + idsAvailable.remove(current); + subRangeDeletions.remove(current); + } + } + writeValues(idsAvailable, subRangeDeletions); + } + + private void writeValues( + Map, RangeSet> idsAvailable, + Map, RangeSet> subRangeDeletions) { + if (idsAvailable.isEmpty()) { + idsAvailable.clear(); + } else { + idsAvailableValue.write(idsAvailable); + } + if (subRangeDeletions.isEmpty()) { + subRangeDeletionsValue.clear(); + } else { + subRangeDeletionsValue.write(subRangeDeletions); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/PagingIterable.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/PagingIterable.java new file mode 100644 index 000000000000..73f076d92013 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/PagingIterable.java @@ -0,0 +1,129 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.util.Iterator; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.AbstractIterator; + +/** + * An iterable over elements backed by paginated GetData requests to Windmill. The iterable may be + * iterated over an arbitrary number of times and multiple iterators may be active simultaneously. + * + *

There are two pattern we wish to support with low -memory and -latency: + * + *

    + *
  1. Re-iterate over the initial elements multiple times (eg Iterables.first). We'll cache the + * initial 'page' of values returned by Windmill from our first request for the lifetime of + * the iterable. + *
  2. Iterate through all elements of a very large collection. We'll send the GetData request for + * the next page when the current page is begun. We'll discard intermediate pages and only + * retain the first. Thus the maximum memory pressure is one page plus one page per call to + * iterator. + *
+ */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +class PagingIterable implements Iterable { + /** + * The reader we will use for scheduling continuation pages. + * + *

NOTE We've made this explicit to remind us to be careful not to cache the iterable. + */ + private final WindmillStateReader reader; + + /** Initial values returned for the first page. Never reclaimed. */ + private final List firstPage; + + /** State tag with continuation position set for second page. */ + private final StateTag secondPagePos; + + /** Coder for elements. */ + private final Coder coder; + + PagingIterable( + WindmillStateReader reader, + List firstPage, + StateTag secondPagePos, + Coder coder) { + this.reader = reader; + this.firstPage = firstPage; + this.secondPagePos = secondPagePos; + this.coder = coder; + } + + @Override + public Iterator iterator() { + return new PagingIterableIterator(); + } + + private class PagingIterableIterator extends AbstractIterator { + private Iterator currentPage = firstPage.iterator(); + private StateTag nextPagePos = secondPagePos; + private Future> pendingNextPage = + // NOTE: The results of continuation page reads are never cached. + reader.continuationFuture(nextPagePos, coder); + + @Override + protected ResultT computeNext() { + while (true) { + if (currentPage.hasNext()) { + return currentPage.next(); + } + if (pendingNextPage == null) { + return endOfData(); + } + + ValuesAndContPosition valuesAndContPosition; + try { + valuesAndContPosition = pendingNextPage.get(); + } catch (InterruptedException | ExecutionException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read value from state", e); + } + currentPage = valuesAndContPosition.getValues().iterator(); + StateTag.Builder nextPageBuilder = + StateTag.of( + nextPagePos.getKind(), + nextPagePos.getTag(), + nextPagePos.getStateFamily(), + valuesAndContPosition.getContinuationPosition()) + .toBuilder(); + if (secondPagePos.getSortedListRange() != null) { + nextPageBuilder.setSortedListRange(secondPagePos.getSortedListRange()); + } + if (secondPagePos.getOmitValues() != null) { + nextPageBuilder.setOmitValues(secondPagePos.getOmitValues()); + } + if (secondPagePos.getMultimapKey() != null) { + nextPageBuilder.setMultimapKey(secondPagePos.getMultimapKey()); + } + nextPagePos = nextPageBuilder.build(); + pendingNextPage = + // NOTE: The results of continuation page reads are never cached. + reader.continuationFuture(nextPagePos, coder); + } + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeCoder.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeCoder.java new file mode 100644 index 000000000000..0e11531226f7 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeCoder.java @@ -0,0 +1,76 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.List; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StructuredCoder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BoundType; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; +import org.checkerframework.checker.nullness.qual.Nullable; + +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +/** Coder for closed-open ranges. */ +class RangeCoder> extends StructuredCoder> { + private final Coder boundCoder; + + RangeCoder(Coder boundCoder) { + this.boundCoder = NullableCoder.of(boundCoder); + } + + @Override + public List> getCoderArguments() { + return Lists.newArrayList(boundCoder); + } + + @Override + public void verifyDeterministic() throws NonDeterministicException { + boundCoder.verifyDeterministic(); + } + + @Override + public void encode(Range value, OutputStream outStream) throws IOException { + Preconditions.checkState( + value.lowerBoundType().equals(BoundType.CLOSED), "unexpected range " + value); + Preconditions.checkState( + value.upperBoundType().equals(BoundType.OPEN), "unexpected range " + value); + boundCoder.encode(value.hasLowerBound() ? value.lowerEndpoint() : null, outStream); + boundCoder.encode(value.hasUpperBound() ? value.upperEndpoint() : null, outStream); + } + + @Override + public Range decode(InputStream inStream) throws IOException { + @Nullable T lower = boundCoder.decode(inStream); + @Nullable T upper = boundCoder.decode(inStream); + if (lower == null) { + return upper != null ? Range.lessThan(upper) : Range.all(); + } else if (upper == null) { + return Range.atLeast(lower); + } else { + return Range.closedOpen(lower, upper); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeSetCoder.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeSetCoder.java new file mode 100644 index 000000000000..291a83e81ee6 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/RangeSetCoder.java @@ -0,0 +1,46 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.SetCoder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.RangeSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.TreeRangeSet; + +class RangeSetCoder> extends CustomCoder> { + private final SetCoder> rangesCoder; + + RangeSetCoder(Coder boundCoder) { + this.rangesCoder = SetCoder.of(new RangeCoder<>(boundCoder)); + } + + @Override + public void encode(RangeSet value, OutputStream outStream) throws IOException { + rangesCoder.encode(value.asRanges(), outStream); + } + + @Override + public RangeSet decode(InputStream inStream) throws IOException { + return TreeRangeSet.create(rangesCoder.decode(inStream)); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/SimpleWindmillState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/SimpleWindmillState.java new file mode 100644 index 000000000000..bd7f8041c680 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/SimpleWindmillState.java @@ -0,0 +1,42 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.io.IOException; +import java.util.concurrent.Future; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; + +/** + * Base class for implementations of {@link WindmillState} where the {@link #persist} call does not + * require any asynchronous reading. + */ +abstract class SimpleWindmillState extends WindmillState { + @Override + public final Future persist( + WindmillStateCache.ForKeyAndFamily cache) throws IOException { + return Futures.immediateFuture(persistDirectly(cache)); + } + + /** + * Returns a {@link Windmill.WorkItemCommitRequest} that can be used to persist this state to + * Windmill. + */ + protected abstract Windmill.WorkItemCommitRequest persistDirectly( + WindmillStateCache.ForKeyAndFamily cache) throws IOException; +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/StateTag.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/StateTag.java new file mode 100644 index 000000000000..13c2a9e66baa --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/StateTag.java @@ -0,0 +1,109 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import com.google.auto.value.AutoValue; +import javax.annotation.Nullable; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; + +/** + * When combined with a key and computationId, represents the unique address for state managed by + * Windmill. + */ +@AutoValue +public abstract class StateTag { + static StateTag of( + Kind kind, ByteString tag, String stateFamily, @Nullable RequestPositionT requestPosition) { + return new AutoValue_StateTag.Builder() + .setKind(kind) + .setTag(tag) + .setStateFamily(stateFamily) + .setRequestPosition(requestPosition) + .build(); + } + + public static StateTag of( + Kind kind, ByteString tag, String stateFamily) { + return of(kind, tag, stateFamily, null); + } + + abstract Kind getKind(); + + abstract ByteString getTag(); + + abstract String getStateFamily(); + + /** + * For {@link Kind#BAG, Kind#ORDERED_LIST, Kind#VALUE_PREFIX, KIND#MULTIMAP_SINGLE_ENTRY, + * KIND#MULTIMAP_ALL} kinds: A previous 'continuation_position' returned by Windmill to signal the + * resulting state was incomplete. Sending that position will request the next page of values. + * Null for first request. + * + *

Null for other kinds. + */ + @Nullable + public abstract RequestPositionT getRequestPosition(); + + /** For {@link Kind#ORDERED_LIST} kinds: the range to fetch or delete. */ + @Nullable + abstract Range getSortedListRange(); + + /** For {@link Kind#MULTIMAP_SINGLE_ENTRY} kinds: the key in the multimap to fetch or delete. */ + @Nullable + abstract ByteString getMultimapKey(); + + /** + * For {@link Kind#MULTIMAP_ALL} kinds: will only return the keys of the multimap and not the + * values if true. + */ + @Nullable + abstract Boolean getOmitValues(); + + public abstract Builder toBuilder(); + + public enum Kind { + VALUE, + BAG, + WATERMARK, + ORDERED_LIST, + VALUE_PREFIX, + MULTIMAP_SINGLE_ENTRY, + MULTIMAP_ALL + } + + @AutoValue.Builder + abstract static class Builder { + abstract Builder setKind(Kind kind); + + abstract Builder setTag(ByteString tag); + + abstract Builder setStateFamily(String stateFamily); + + abstract Builder setRequestPosition( + @Nullable RequestPositionT requestPosition); + + abstract Builder setSortedListRange(@Nullable Range sortedListRange); + + abstract Builder setMultimapKey(@Nullable ByteString encodedMultimapKey); + + abstract Builder setOmitValues(Boolean omitValues); + + abstract StateTag build(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/TimestampedValueWithId.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/TimestampedValueWithId.java new file mode 100644 index 000000000000..e180efafb65b --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/TimestampedValueWithId.java @@ -0,0 +1,42 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import com.google.auto.value.AutoValue; +import java.util.Comparator; +import org.apache.beam.sdk.values.TimestampedValue; +import org.joda.time.Instant; + +@AutoValue +abstract class TimestampedValueWithId { + public static final Comparator> COMPARATOR = + Comparator., Instant>comparing(v -> v.getValue().getTimestamp()) + .thenComparingLong(TimestampedValueWithId::getId); + + static TimestampedValueWithId of(TimestampedValue value, long id) { + return new AutoValue_TimestampedValueWithId<>(value, id); + } + + static TimestampedValueWithId bound(Instant ts) { + return of(TimestampedValue.of(null, ts), Long.MIN_VALUE); + } + + abstract TimestampedValue getValue(); + + abstract long getId(); +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ToIterableFunction.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ToIterableFunction.java new file mode 100644 index 000000000000..3db058c79a03 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ToIterableFunction.java @@ -0,0 +1,79 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; + +/** Function to extract an {@link Iterable} from the continuation-supporting page read future. */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class ToIterableFunction + implements Function, Iterable> { + private final StateTag stateTag; + private final Coder coder; + /** + * Reader to request continuation pages from, or {@literal null} if no continuation pages + * required. + */ + private @Nullable WindmillStateReader reader; + + public ToIterableFunction( + WindmillStateReader reader, StateTag stateTag, Coder coder) { + this.reader = reader; + this.stateTag = stateTag; + this.coder = coder; + } + + @SuppressFBWarnings( + value = "NP_METHOD_PARAMETER_TIGHTENS_ANNOTATION", + justification = "https://github.com/google/guava/issues/920") + @Override + public Iterable apply( + @Nonnull ValuesAndContPosition valuesAndContPosition) { + if (valuesAndContPosition.getContinuationPosition() == null) { + // Number of values is small enough Windmill sent us the entire bag in one response. + reader = null; + return valuesAndContPosition.getValues(); + } else { + // Return an iterable which knows how to come back for more. + StateTag.Builder continuationTBuilder = + StateTag.of( + stateTag.getKind(), + stateTag.getTag(), + stateTag.getStateFamily(), + valuesAndContPosition.getContinuationPosition()) + .toBuilder(); + if (stateTag.getSortedListRange() != null) { + continuationTBuilder.setSortedListRange(stateTag.getSortedListRange()).build(); + } + if (stateTag.getMultimapKey() != null) { + continuationTBuilder.setMultimapKey(stateTag.getMultimapKey()).build(); + } + if (stateTag.getOmitValues() != null) { + continuationTBuilder.setOmitValues(stateTag.getOmitValues()).build(); + } + return new PagingIterable<>( + reader, valuesAndContPosition.getValues(), continuationTBuilder.build(), coder); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ValuesAndContPosition.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ValuesAndContPosition.java new file mode 100644 index 000000000000..a1002fee4380 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/ValuesAndContPosition.java @@ -0,0 +1,46 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.util.List; +import javax.annotation.Nullable; + +/** + * An in-memory collection of deserialized values and an optional continuation position to pass to + * Windmill when fetching the next page of values. + */ +public class ValuesAndContPosition { + private final List values; + + /** Position to pass to next request for next page of values. Null if done. */ + private final @Nullable ContinuationT continuationPosition; + + public ValuesAndContPosition(List values, @Nullable ContinuationT continuationPosition) { + this.values = values; + this.continuationPosition = continuationPosition; + } + + public List getValues() { + return values; + } + + @Nullable + public ContinuationT getContinuationPosition() { + return continuationPosition; + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WeightedList.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WeightedList.java new file mode 100644 index 000000000000..1b39d07c6fe7 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WeightedList.java @@ -0,0 +1,54 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.util.List; +import org.apache.beam.sdk.util.Weighted; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ForwardingList; + +@VisibleForTesting +public class WeightedList extends ForwardingList implements Weighted { + private final List delegate; + long weight; + + WeightedList(List delegate) { + this.delegate = delegate; + this.weight = 0; + } + + @Override + protected List delegate() { + return delegate; + } + + @Override + public boolean add(T elem) { + throw new UnsupportedOperationException("Must use AddWeighted()"); + } + + @Override + public long getWeight() { + return weight; + } + + public void addWeighted(T elem, long weight) { + delegate.add(elem); + this.weight += weight; + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillBag.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillBag.java new file mode 100644 index 000000000000..7cdb3776dfa1 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillBag.java @@ -0,0 +1,209 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.Weighted; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; + +@SuppressWarnings({ + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WindmillBag extends SimpleWindmillState implements BagState { + + private final StateNamespace namespace; + private final StateTag> address; + private final ByteString stateKey; + private final String stateFamily; + private final Coder elemCoder; + + private boolean cleared = false; + /** + * If non-{@literal null}, this contains the complete contents of the bag, except for any local + * additions. If {@literal null} then we don't know if Windmill contains additional values which + * should be part of the bag. We'll need to read them if the work item actually wants the bag + * contents. + */ + private ConcatIterables cachedValues = null; + + private List localAdditions = new ArrayList<>(); + private long encodedSize = 0; + + WindmillBag( + StateNamespace namespace, + StateTag> address, + String stateFamily, + Coder elemCoder, + boolean isNewKey) { + this.namespace = namespace; + this.address = address; + this.stateKey = WindmillStateUtil.encodeKey(namespace, address); + this.stateFamily = stateFamily; + this.elemCoder = elemCoder; + if (isNewKey) { + this.cachedValues = new ConcatIterables<>(); + } + } + + @Override + public void clear() { + cleared = true; + cachedValues = new ConcatIterables<>(); + localAdditions = new ArrayList<>(); + encodedSize = 0; + } + + /** + * Return iterable over all bag values in Windmill which should contribute to overall bag + * contents. + */ + private Iterable fetchData(Future> persistedData) { + try (Closeable scope = scopedReadState()) { + if (cachedValues != null) { + return cachedValues.snapshot(); + } + Iterable data = persistedData.get(); + if (data instanceof Weighted) { + // We have a known bounded amount of data; cache it. + cachedValues = new ConcatIterables<>(); + cachedValues.extendWith(data); + encodedSize = ((Weighted) data).getWeight(); + return cachedValues.snapshot(); + } else { + // This is an iterable that may not fit in memory at once; don't cache it. + return data; + } + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + public boolean valuesAreCached() { + return cachedValues != null; + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public WindmillBag readLater() { + getFuture(); + return this; + } + + @Override + public Iterable read() { + return Iterables.concat( + fetchData(getFuture()), Iterables.limit(localAdditions, localAdditions.size())); + } + + @Override + public ReadableState isEmpty() { + return new ReadableState() { + @Override + public ReadableState readLater() { + WindmillBag.this.readLater(); + return this; + } + + @Override + public Boolean read() { + return Iterables.isEmpty(fetchData(getFuture())) && localAdditions.isEmpty(); + } + }; + } + + @Override + public void add(T input) { + localAdditions.add(input); + } + + @Override + public Windmill.WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + + Windmill.TagBag.Builder bagUpdatesBuilder = null; + + if (cleared) { + bagUpdatesBuilder = commitBuilder.addBagUpdatesBuilder(); + bagUpdatesBuilder.setDeleteAll(true); + cleared = false; + } + + if (!localAdditions.isEmpty()) { + // Tell Windmill to capture the local additions. + if (bagUpdatesBuilder == null) { + bagUpdatesBuilder = commitBuilder.addBagUpdatesBuilder(); + } + for (T value : localAdditions) { + ByteStringOutputStream stream = new ByteStringOutputStream(); + // Encode the value + elemCoder.encode(value, stream, Coder.Context.OUTER); + ByteString encoded = stream.toByteString(); + if (cachedValues != null) { + // We'll capture this value in the cache below. + // Capture the value's size now since we have it. + encodedSize += encoded.size(); + } + bagUpdatesBuilder.addValues(encoded); + } + } + + if (bagUpdatesBuilder != null) { + bagUpdatesBuilder.setTag(stateKey).setStateFamily(stateFamily); + } + + if (cachedValues != null) { + if (!localAdditions.isEmpty()) { + // Capture the local additions in the cached value since we and + // Windmill are now in agreement. + cachedValues.extendWith(localAdditions); + } + // We now know the complete bag contents, and any read on it will yield a + // cached value, so cache it for future reads. + cache.put(namespace, address, this, encodedSize); + } + + // Don't reuse the localAdditions object; we don't want future changes to it to + // modify the value of cachedValues. + localAdditions = new ArrayList<>(); + + return commitBuilder.buildPartial(); + } + + private Future> getFuture() { + return cachedValues != null ? null : reader.bagFuture(stateKey, stateFamily, elemCoder); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillCombiningState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillCombiningState.java new file mode 100644 index 000000000000..98359913c703 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillCombiningState.java @@ -0,0 +1,170 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collections; +import java.util.concurrent.Future; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.core.StateTags; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.BagState; +import org.apache.beam.sdk.state.CombiningState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.transforms.Combine; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; + +@NotThreadSafe +class WindmillCombiningState extends WindmillState + implements CombiningState { + + private final WindmillBag bag; + private final Combine.CombineFn combineFn; + + /* We use a separate, in-memory AccumT rather than relying on the WindmillWatermarkBag's + * localAdditions, because we want to combine multiple InputT's to a single AccumT + * before adding it. + */ + private AccumT localAdditionsAccumulator; + private boolean hasLocalAdditions; + + WindmillCombiningState( + StateNamespace namespace, + StateTag> address, + String stateFamily, + Coder accumCoder, + Combine.CombineFn combineFn, + WindmillStateCache.ForKeyAndFamily cache, + boolean isNewKey) { + StateTag> internalBagAddress = StateTags.convertToBagTagInternal(address); + this.bag = + cache + .get(namespace, internalBagAddress) + .map(state -> (WindmillBag) state) + .orElseGet( + () -> + new WindmillBag<>( + namespace, internalBagAddress, stateFamily, accumCoder, isNewKey)); + + this.combineFn = combineFn; + this.localAdditionsAccumulator = combineFn.createAccumulator(); + this.hasLocalAdditions = false; + } + + @Override + void initializeForWorkItem( + WindmillStateReader reader, Supplier scopedReadStateSupplier) { + super.initializeForWorkItem(reader, scopedReadStateSupplier); + this.bag.initializeForWorkItem(reader, scopedReadStateSupplier); + } + + @Override + void cleanupAfterWorkItem() { + super.cleanupAfterWorkItem(); + bag.cleanupAfterWorkItem(); + } + + @Override + public WindmillCombiningState readLater() { + bag.readLater(); + return this; + } + + @Override + @SuppressWarnings("nullness") + public OutputT read() { + return combineFn.extractOutput(getAccum()); + } + + @Override + public void add(InputT input) { + hasLocalAdditions = true; + localAdditionsAccumulator = combineFn.addInput(localAdditionsAccumulator, input); + } + + @Override + public void clear() { + bag.clear(); + localAdditionsAccumulator = combineFn.createAccumulator(); + hasLocalAdditions = false; + } + + @Override + public Future persist(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + if (hasLocalAdditions) { + if (WindmillStateInternals.COMPACT_NOW.get().get() || bag.valuesAreCached()) { + // Implicitly clears the bag and combines local and persisted accumulators. + localAdditionsAccumulator = getAccum(); + } + bag.add(combineFn.compact(localAdditionsAccumulator)); + localAdditionsAccumulator = combineFn.createAccumulator(); + hasLocalAdditions = false; + } + + return bag.persist(cache); + } + + @Override + public AccumT getAccum() { + Iterable accumulators = + Iterables.concat(bag.read(), Collections.singleton(localAdditionsAccumulator)); + + // Compact things + AccumT merged = combineFn.mergeAccumulators(accumulators); + bag.clear(); + localAdditionsAccumulator = merged; + hasLocalAdditions = true; + return merged; + } + + @Override + public ReadableState isEmpty() { + final ReadableState bagIsEmpty = bag.isEmpty(); + return new ReadableState() { + @Override + public ReadableState readLater() { + bagIsEmpty.readLater(); + return this; + } + + @Override + public Boolean read() { + return !hasLocalAdditions && bagIsEmpty.read(); + } + }; + } + + @Override + public void addAccum(AccumT accumulator) { + hasLocalAdditions = true; + localAdditionsAccumulator = + combineFn.mergeAccumulators(Arrays.asList(localAdditionsAccumulator, accumulator)); + } + + @Override + public AccumT mergeAccumulators(Iterable accumulators) { + return combineFn.mergeAccumulators(accumulators); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMap.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMap.java new file mode 100644 index 000000000000..43490a725ac4 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMap.java @@ -0,0 +1,449 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import static org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateUtil.encodeKey; + +import java.io.Closeable; +import java.io.IOException; +import java.util.AbstractMap; +import java.util.Collections; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.function.Function; +import javax.annotation.Nullable; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.ReadableStates; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.Weighted; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; +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.Maps; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; + +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WindmillMap extends SimpleWindmillState implements MapState { + private final StateNamespace namespace; + private final StateTag> address; + private final ByteString stateKeyPrefix; + private final String stateFamily; + private final Coder keyCoder; + private final Coder valueCoder; + // TODO(reuvenlax): Should we evict items from the cache? We would have to make sure + // that anything in the cache that is not committed is not evicted. negativeCache could be + // evicted whenever we want. + private final Map cachedValues = Maps.newHashMap(); + private final Set negativeCache = Sets.newHashSet(); + private final Set localAdditions = Sets.newHashSet(); + private final Set localRemovals = Sets.newHashSet(); + private boolean complete; + private boolean cleared = false; + + WindmillMap( + StateNamespace namespace, + StateTag> address, + String stateFamily, + Coder keyCoder, + Coder valueCoder, + boolean isNewKey) { + this.namespace = namespace; + this.address = address; + this.stateKeyPrefix = encodeKey(namespace, address); + this.stateFamily = stateFamily; + this.keyCoder = keyCoder; + this.valueCoder = valueCoder; + this.complete = isNewKey; + } + + private K userKeyFromProtoKey(ByteString tag) throws IOException { + Preconditions.checkState(tag.startsWith(stateKeyPrefix)); + ByteString keyBytes = tag.substring(stateKeyPrefix.size()); + return keyCoder.decode(keyBytes.newInput(), Coder.Context.OUTER); + } + + private ByteString protoKeyFromUserKey(K key) throws IOException { + ByteStringOutputStream keyStream = new ByteStringOutputStream(); + stateKeyPrefix.writeTo(keyStream); + keyCoder.encode(key, keyStream, Coder.Context.OUTER); + return keyStream.toByteString(); + } + + @Override + protected Windmill.WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + if (!cleared && localAdditions.isEmpty() && localRemovals.isEmpty()) { + // No changes, so return directly. + return Windmill.WorkItemCommitRequest.newBuilder().buildPartial(); + } + + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + + if (cleared) { + commitBuilder + .addTagValuePrefixDeletesBuilder() + .setStateFamily(stateFamily) + .setTagPrefix(stateKeyPrefix); + } + cleared = false; + + for (K key : localAdditions) { + ByteString keyBytes = protoKeyFromUserKey(key); + ByteStringOutputStream valueStream = new ByteStringOutputStream(); + valueCoder.encode(cachedValues.get(key), valueStream, Coder.Context.OUTER); + ByteString valueBytes = valueStream.toByteString(); + + commitBuilder + .addValueUpdatesBuilder() + .setTag(keyBytes) + .setStateFamily(stateFamily) + .getValueBuilder() + .setData(valueBytes) + .setTimestamp(Long.MAX_VALUE); + } + localAdditions.clear(); + + for (K key : localRemovals) { + ByteStringOutputStream keyStream = new ByteStringOutputStream(); + stateKeyPrefix.writeTo(keyStream); + keyCoder.encode(key, keyStream, Coder.Context.OUTER); + ByteString keyBytes = keyStream.toByteString(); + // Leaving data blank means that we delete the tag. + commitBuilder + .addValueUpdatesBuilder() + .setTag(keyBytes) + .setStateFamily(stateFamily) + .getValueBuilder() + .setTimestamp(Long.MAX_VALUE); + + V cachedValue = cachedValues.remove(key); + if (cachedValue != null) { + ByteStringOutputStream valueStream = new ByteStringOutputStream(); + valueCoder.encode(cachedValues.get(key), valueStream, Coder.Context.OUTER); + } + } + negativeCache.addAll(localRemovals); + localRemovals.clear(); + + // TODO(reuvenlax): We should store in the cache parameter, as that would enable caching the + // map + // between work items, reducing fetches to Windmill. To do so, we need keep track of the + // encoded size + // of the map, and to do so efficiently (i.e. without iterating over the entire map on every + // persist) + // we need to track the sizes of each map entry. + cache.put(namespace, address, this, 1); + return commitBuilder.buildPartial(); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState get(K key) { + return getOrDefault(key, null); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState getOrDefault( + K key, @Nullable V defaultValue) { + return new WindmillMapReadResultReadableState(key, defaultValue); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState< + @UnknownKeyFor @NonNull @Initialized Iterable> + keys() { + ReadableState>> entries = entries(); + return new WindmillMapKeysReadableState(entries); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState< + @UnknownKeyFor @NonNull @Initialized Iterable> + values() { + ReadableState>> entries = entries(); + return new WindmillMapValuesReadableState(entries); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState< + @UnknownKeyFor @NonNull @Initialized Iterable< + Map.@UnknownKeyFor @NonNull @Initialized Entry>> + entries() { + return new WindmillMapEntriesReadableState(); + } + + @Override + public ReadableState isEmpty() { + return new WindmillMapIsEmptyReadableState(); + } + + @Override + public void put(K key, V value) { + V oldValue = cachedValues.put(key, value); + if (valueCoder.consistentWithEquals() && value.equals(oldValue)) { + return; + } + localAdditions.add(key); + localRemovals.remove(key); + negativeCache.remove(key); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState computeIfAbsent( + K key, Function mappingFunction) { + Future persistedData = getFutureForKey(key); + try (Closeable scope = scopedReadState()) { + if (localRemovals.contains(key) || negativeCache.contains(key)) { + return ReadableStates.immediate(null); + } + @Nullable V cachedValue = cachedValues.get(key); + if (cachedValue != null || complete) { + return ReadableStates.immediate(cachedValue); + } + + V persistedValue = persistedData.get(); + if (persistedValue == null) { + // This is a new value. Add it to the map and return null. + put(key, mappingFunction.apply(key)); + return ReadableStates.immediate(null); + } + // TODO: Don't do this if it was already in cache. + cachedValues.put(key, persistedValue); + return ReadableStates.immediate(persistedValue); + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + @Override + public void remove(K key) { + if (localRemovals.add(key)) { + cachedValues.remove(key); + localAdditions.remove(key); + } + } + + @Override + public void clear() { + cachedValues.clear(); + localAdditions.clear(); + localRemovals.clear(); + negativeCache.clear(); + cleared = true; + complete = true; + } + + private Future getFutureForKey(K key) { + try { + ByteStringOutputStream keyStream = new ByteStringOutputStream(); + stateKeyPrefix.writeTo(keyStream); + keyCoder.encode(key, keyStream, Coder.Context.OUTER); + return reader.valueFuture(keyStream.toByteString(), stateFamily, valueCoder); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private Future>> getFuture() { + if (complete) { + // The caller will merge in local cached values. + return Futures.immediateFuture(Collections.emptyList()); + } else { + return reader.valuePrefixFuture(stateKeyPrefix, stateFamily, valueCoder); + } + } + + private class WindmillMapKeysReadableState implements ReadableState> { + private final ReadableState>> entries; + + public WindmillMapKeysReadableState(ReadableState>> entries) { + this.entries = entries; + } + + @Override + public Iterable read() { + return Iterables.transform(entries.read(), Map.Entry::getKey); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState> readLater() { + entries.readLater(); + return this; + } + } + + private class WindmillMapValuesReadableState implements ReadableState> { + private final ReadableState>> entries; + + public WindmillMapValuesReadableState(ReadableState>> entries) { + this.entries = entries; + } + + @Override + public @Nullable Iterable read() { + return Iterables.transform(entries.read(), Map.Entry::getValue); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState> readLater() { + entries.readLater(); + return this; + } + } + + private class WindmillMapEntriesReadableState + implements ReadableState>> { + @Override + public Iterable> read() { + if (complete) { + return Iterables.unmodifiableIterable(cachedValues.entrySet()); + } + Future>> persistedData = getFuture(); + try (Closeable scope = scopedReadState()) { + Iterable> data = persistedData.get(); + Iterable> transformedData = + Iterables.transform( + data, + entry -> { + try { + return new AbstractMap.SimpleEntry<>( + userKeyFromProtoKey(entry.getKey()), entry.getValue()); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + + if (data instanceof Weighted) { + // This is a known amount of data. Cache it all. + transformedData.forEach( + e -> { + // The cached data overrides what is read from state, so call putIfAbsent. + cachedValues.putIfAbsent(e.getKey(), e.getValue()); + }); + complete = true; + return Iterables.unmodifiableIterable(cachedValues.entrySet()); + } else { + // This means that the result might be too large to cache, so don't add it to the + // local cache. Instead merge the iterables, giving priority to any local additions + // (represented in cachedValued and localRemovals) that may not have been committed + // yet. + return Iterables.unmodifiableIterable( + Iterables.concat( + cachedValues.entrySet(), + Iterables.filter( + transformedData, + e -> + !cachedValues.containsKey(e.getKey()) + && !localRemovals.contains(e.getKey())))); + } + + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public @UnknownKeyFor @NonNull @Initialized ReadableState>> + readLater() { + WindmillMap.this.getFuture(); + return this; + } + } + + private class WindmillMapIsEmptyReadableState implements ReadableState { + // TODO(reuvenlax): Can we find a more efficient way of implementing isEmpty than reading + // the entire map? + final ReadableState> keys = WindmillMap.this.keys(); + + @Override + public @Nullable Boolean read() { + return Iterables.isEmpty(keys.read()); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState readLater() { + keys.readLater(); + return this; + } + } + + private class WindmillMapReadResultReadableState implements ReadableState { + private final K key; + private final @Nullable V defaultValue; + + public WindmillMapReadResultReadableState(K key, @Nullable V defaultValue) { + this.key = key; + this.defaultValue = defaultValue; + } + + @Override + public @Nullable V read() { + Future persistedData = getFutureForKey(key); + try (Closeable scope = scopedReadState()) { + if (localRemovals.contains(key) || negativeCache.contains(key)) { + return null; + } + @Nullable V cachedValue = cachedValues.get(key); + if (cachedValue != null || complete) { + return cachedValue; + } + + V persistedValue = persistedData.get(); + if (persistedValue == null) { + negativeCache.add(key); + return defaultValue; + } + // TODO: Don't do this if it was already in cache. + cachedValues.put(key, persistedValue); + return persistedValue; + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public @UnknownKeyFor @NonNull @Initialized ReadableState readLater() { + WindmillMap.this.getFutureForKey(key); + return this; + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java new file mode 100644 index 000000000000..1c0b3df44c21 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillMultimap.java @@ -0,0 +1,732 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import static org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateUtil.encodeKey; + +import java.io.Closeable; +import java.io.IOException; +import java.util.AbstractMap; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Set; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.stream.Collectors; +import org.apache.beam.repackaged.core.org.apache.commons.lang3.tuple.Triple; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.MultimapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.util.Weighted; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +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.Iterators; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; + +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WindmillMultimap extends SimpleWindmillState implements MultimapState { + + private final StateNamespace namespace; + private final StateTag> address; + private final ByteString stateKey; + private final String stateFamily; + private final Coder keyCoder; + private final Coder valueCoder; + // Set to true when user clears the entire multimap, so that we can later send delete request to + // the windmill backend. + private boolean cleared = false; + // We use the structural value of the keys as the key in keyStateMap, so that different java + // Objects with the same content will be treated as the same Multimap key. + private Map keyStateMap = Maps.newHashMap(); + // If true, all keys are cached in keyStateMap with existence == KNOWN_EXIST. + private boolean allKeysKnown; + // True if all contents of this multimap are cached in this object. + private boolean complete; + // hasLocalAdditions and hasLocalRemovals track whether there are local changes that needs to be + // propagated to windmill. + private boolean hasLocalAdditions = false; + private boolean hasLocalRemovals = false; + + WindmillMultimap( + StateNamespace namespace, + StateTag> address, + String stateFamily, + Coder keyCoder, + Coder valueCoder, + boolean isNewShardingKey) { + this.namespace = namespace; + this.address = address; + this.stateKey = encodeKey(namespace, address); + this.stateFamily = stateFamily; + this.keyCoder = keyCoder; + this.valueCoder = valueCoder; + this.complete = isNewShardingKey; + this.allKeysKnown = isNewShardingKey; + } + + private static Iterable> unnestCachedEntries( + Iterable>>> cachedEntries) { + return Iterables.concat( + Iterables.transform( + cachedEntries, + entry -> + Iterables.transform( + entry.getValue().getRight(), + v -> new AbstractMap.SimpleEntry<>(entry.getValue().getLeft(), v)))); + } + + @Override + public void put(K key, V value) { + final Object structuralKey = keyCoder.structuralValue(key); + hasLocalAdditions = true; + keyStateMap.compute( + structuralKey, + (k, v) -> { + if (v == null) v = new KeyState(key); + v.existence = KeyExistence.KNOWN_EXIST; + v.localAdditions.add(value); + return v; + }); + } + + // Initiates a backend state read to fetch all entries if necessary. + private Future>>> necessaryEntriesFromStorageFuture( + boolean omitValues) { + if (complete) { + // Since we're complete, even if there are entries in storage we don't need to read them. + return Futures.immediateFuture(Collections.emptyList()); + } else { + return reader.multimapFetchAllFuture(omitValues, stateKey, stateFamily, valueCoder); + } + } + + // Initiates a backend state read to fetch a single entry if necessary. + private Future> necessaryKeyEntriesFromStorageFuture(K key) { + try { + ByteStringOutputStream keyStream = new ByteStringOutputStream(); + keyCoder.encode(key, keyStream, Coder.Context.OUTER); + return reader.multimapFetchSingleEntryFuture( + keyStream.toByteString(), stateKey, stateFamily, valueCoder); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public ReadableState> get(K key) { + return new ReadResultReadableState(key); + } + + @Override + protected Windmill.WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + if (!cleared && !hasLocalAdditions && !hasLocalRemovals) { + cache.put(namespace, address, this, 1); + return Windmill.WorkItemCommitRequest.newBuilder().buildPartial(); + } + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + Windmill.TagMultimapUpdateRequest.Builder builder = commitBuilder.addMultimapUpdatesBuilder(); + builder.setTag(stateKey).setStateFamily(stateFamily); + + if (cleared) { + builder.setDeleteAll(true); + } + if (hasLocalRemovals || hasLocalAdditions) { + ByteStringOutputStream keyStream = new ByteStringOutputStream(); + ByteStringOutputStream valueStream = new ByteStringOutputStream(); + Iterator> iterator = keyStateMap.entrySet().iterator(); + while (iterator.hasNext()) { + KeyState keyState = iterator.next().getValue(); + if (!keyState.removedLocally && keyState.localAdditions.isEmpty()) { + if (keyState.existence == KeyExistence.KNOWN_NONEXISTENT) iterator.remove(); + continue; + } + keyCoder.encode(keyState.originalKey, keyStream, Coder.Context.OUTER); + ByteString encodedKey = keyStream.toByteStringAndReset(); + Windmill.TagMultimapEntry.Builder entryBuilder = builder.addUpdatesBuilder(); + entryBuilder.setEntryName(encodedKey); + if (keyState.removedLocally) entryBuilder.setDeleteAll(true); + keyState.removedLocally = false; + if (!keyState.localAdditions.isEmpty()) { + for (V value : keyState.localAdditions) { + valueCoder.encode(value, valueStream, Coder.Context.OUTER); + ByteString encodedValue = valueStream.toByteStringAndReset(); + entryBuilder.addValues(encodedValue); + } + // Move newly added values from localAdditions to keyState.values as those new values + // now + // are also persisted in Windmill. If a key now has no more values and is not + // KNOWN_EXIST, + // remove it from cache. + if (keyState.valuesCached) { + keyState.values.extendWith(keyState.localAdditions); + keyState.valuesSize += keyState.localAdditions.size(); + } + // Create a new localAdditions so that the cached values are unaffected. + keyState.localAdditions = Lists.newArrayList(); + } + if (!keyState.valuesCached && keyState.existence != KeyExistence.KNOWN_EXIST) { + iterator.remove(); + } + } + } + + hasLocalAdditions = false; + hasLocalRemovals = false; + cleared = false; + + cache.put(namespace, address, this, 1); + return commitBuilder.buildPartial(); + } + + @Override + public void remove(K key) { + final Object structuralKey = keyCoder.structuralValue(key); + // does not insert key if allKeysKnown. + KeyState keyState = + keyStateMap.computeIfAbsent(structuralKey, k -> allKeysKnown ? null : new KeyState(key)); + if (keyState == null || keyState.existence == KeyExistence.KNOWN_NONEXISTENT) { + return; + } + if (keyState.valuesCached && keyState.valuesSize == 0) { + // no data in windmill, deleting from local cache is sufficient. + keyStateMap.remove(structuralKey); + } else { + // there may be data in windmill that need to be removed. + hasLocalRemovals = true; + keyState.removedLocally = true; + keyState.values = new ConcatIterables<>(); + keyState.valuesSize = 0; + keyState.existence = KeyExistence.KNOWN_NONEXISTENT; + } + if (!keyState.localAdditions.isEmpty()) { + keyState.localAdditions = Lists.newArrayList(); + } + keyState.valuesCached = true; + } + + @Override + public void clear() { + keyStateMap = Maps.newHashMap(); + cleared = true; + complete = true; + allKeysKnown = true; + hasLocalAdditions = false; + hasLocalRemovals = false; + } + + @Override + public ReadableState> keys() { + return new KeysReadableState(); + } + + @Override + public ReadableState>> entries() { + return new EntriesReadableState(); + } + + @Override + public ReadableState containsKey(K key) { + return new ContainsKeyReadableState(key); + } + + // Currently, isEmpty is implemented by reading all keys and could potentially be optimized. + // But note that if isEmpty is often followed by iterating over keys then maybe not too bad; if + // isEmpty is followed by iterating over both keys and values then it won't help much. + @Override + public ReadableState isEmpty() { + return new IsEmptyReadableState(); + } + + private enum KeyExistence { + // this key is known to exist, it has at least 1 value in either localAdditions or windmill + KNOWN_EXIST, + // this key is known to be nonexistent, it has 0 value in both localAdditions and windmill + KNOWN_NONEXISTENT, + // we don't know if this key is in this multimap, it has exact 0 value in localAddition, but + // may have no or any number of values in windmill. This is just to provide a mapping between + // the original key and the structural key. + UNKNOWN_EXISTENCE + } + + private class KeyState { + final K originalKey; + KeyExistence existence; + // valuesCached can be true if only existence == KNOWN_EXIST and all values of this key are + // cached (both values and localAdditions). + boolean valuesCached; + // Represents the values in windmill. When new values are added during user processing, they + // are added to localAdditions but not values. Those new values will be added to values only + // after they are persisted into windmill and removed from localAdditions + ConcatIterables values; + int valuesSize; + + // When new values are added during user processing, they are added to localAdditions, so that + // we can later try to persist them in windmill. When a key is removed during user processing, + // we mark removedLocally to be true so that we can later try to delete it from windmill. If + // localAdditions is not empty and removedLocally is true, values in localAdditions will be + // added to windmill after old values in windmill are removed. + List localAdditions; + boolean removedLocally; + + KeyState(K originalKey) { + this.originalKey = originalKey; + existence = KeyExistence.UNKNOWN_EXISTENCE; + valuesCached = complete; + values = new ConcatIterables<>(); + valuesSize = 0; + localAdditions = Lists.newArrayList(); + removedLocally = false; + } + } + + private class ReadResultReadableState implements ReadableState> { + final Object structuralKey; + private final K key; + + public ReadResultReadableState(K key) { + this.key = key; + structuralKey = keyCoder.structuralValue(key); + } + + @Override + public Iterable read() { + KeyState keyState = null; + if (allKeysKnown) { + keyState = keyStateMap.get(structuralKey); + if (keyState == null || keyState.existence == KeyExistence.UNKNOWN_EXISTENCE) { + if (keyState != null) keyStateMap.remove(structuralKey); + return Collections.emptyList(); + } + } else { + keyState = keyStateMap.computeIfAbsent(structuralKey, k -> new KeyState(key)); + } + if (keyState.existence == KeyExistence.KNOWN_NONEXISTENT) { + return Collections.emptyList(); + } + Iterable localNewValues = + Iterables.limit(keyState.localAdditions, keyState.localAdditions.size()); + if (keyState.removedLocally) { + // this key has been removed locally but the removal hasn't been sent to windmill, + // thus values in windmill(if any) are obsolete, and we only care about local values. + return Iterables.unmodifiableIterable(localNewValues); + } + if (keyState.valuesCached || complete) { + return Iterables.unmodifiableIterable( + Iterables.concat( + Iterables.limit(keyState.values, keyState.valuesSize), localNewValues)); + } + Future> persistedData = necessaryKeyEntriesFromStorageFuture(key); + try (Closeable scope = scopedReadState()) { + final Iterable persistedValues = persistedData.get(); + // Iterables.isEmpty() is O(1). + if (Iterables.isEmpty(persistedValues)) { + if (keyState.localAdditions.isEmpty()) { + // empty in both cache and windmill, mark key as KNOWN_NONEXISTENT. + keyState.existence = KeyExistence.KNOWN_NONEXISTENT; + return Collections.emptyList(); + } + return Iterables.unmodifiableIterable(localNewValues); + } + keyState.existence = KeyExistence.KNOWN_EXIST; + if (persistedValues instanceof Weighted) { + keyState.valuesCached = true; + ConcatIterables it = new ConcatIterables<>(); + it.extendWith(persistedValues); + keyState.values = it; + keyState.valuesSize = Iterables.size(persistedValues); + } + return Iterables.unmodifiableIterable(Iterables.concat(persistedValues, localNewValues)); + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read Multimap state", e); + } + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public ReadableState> readLater() { + WindmillMultimap.this.necessaryKeyEntriesFromStorageFuture(key); + return this; + } + } + + private class KeysReadableState implements ReadableState> { + + private Map cachedExistKeys() { + return keyStateMap.entrySet().stream() + .filter(entry -> entry.getValue().existence == KeyExistence.KNOWN_EXIST) + .collect(Collectors.toMap(Map.Entry::getKey, e -> e.getValue().originalKey)); + } + + @Override + public Iterable read() { + if (allKeysKnown) { + return Iterables.unmodifiableIterable(cachedExistKeys().values()); + } + Future>>> persistedData = + necessaryEntriesFromStorageFuture(true); + try (Closeable scope = scopedReadState()) { + Iterable>> entries = persistedData.get(); + if (entries instanceof Weighted) { + // This is a known amount of data, cache them all. + entries.forEach( + entry -> { + try { + K originalKey = keyCoder.decode(entry.getKey().newInput(), Coder.Context.OUTER); + KeyState keyState = + keyStateMap.computeIfAbsent( + keyCoder.structuralValue(originalKey), stk -> new KeyState(originalKey)); + if (keyState.existence == KeyExistence.UNKNOWN_EXISTENCE) { + keyState.existence = KeyExistence.KNOWN_EXIST; + } + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + allKeysKnown = true; + keyStateMap + .values() + .removeIf( + keyState -> + keyState.existence != KeyExistence.KNOWN_EXIST && !keyState.removedLocally); + return Iterables.unmodifiableIterable(cachedExistKeys().values()); + } else { + Map cachedExistKeys = Maps.newHashMap(); + Set cachedNonExistKeys = Sets.newHashSet(); + keyStateMap.forEach( + (structuralKey, keyState) -> { + switch (keyState.existence) { + case KNOWN_EXIST: + cachedExistKeys.put(structuralKey, keyState.originalKey); + break; + case KNOWN_NONEXISTENT: + cachedNonExistKeys.add(structuralKey); + break; + default: + break; + } + }); + // keysOnlyInWindmill is lazily loaded. + Iterable keysOnlyInWindmill = + Iterables.filter( + Iterables.transform( + entries, + entry -> { + try { + K originalKey = + keyCoder.decode(entry.getKey().newInput(), Coder.Context.OUTER); + Object structuralKey = keyCoder.structuralValue(originalKey); + if (cachedExistKeys.containsKey(structuralKey) + || cachedNonExistKeys.contains(structuralKey)) return null; + return originalKey; + } catch (IOException e) { + throw new RuntimeException(e); + } + }), + Objects::nonNull); + return Iterables.unmodifiableIterable( + Iterables.concat(cachedExistKeys.values(), keysOnlyInWindmill)); + } + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public ReadableState> readLater() { + WindmillMultimap.this.necessaryEntriesFromStorageFuture(true); + return this; + } + } + + private class EntriesReadableState implements ReadableState>> { + @Override + public Iterable> read() { + if (complete) { + return Iterables.unmodifiableIterable( + unnestCachedEntries(mergedCachedEntries(null).entrySet())); + } + Future>>> persistedData = + necessaryEntriesFromStorageFuture(false); + try (Closeable scope = scopedReadState()) { + Iterable>> entries = persistedData.get(); + if (Iterables.isEmpty(entries)) { + complete = true; + allKeysKnown = true; + return Iterables.unmodifiableIterable( + unnestCachedEntries(mergedCachedEntries(null).entrySet())); + } + if (!(entries instanceof Weighted)) { + return nonWeightedEntries(entries); + } + // This is a known amount of data, cache them all. + entries.forEach( + entry -> { + try { + final K originalKey = + keyCoder.decode(entry.getKey().newInput(), Coder.Context.OUTER); + final Object structuralKey = keyCoder.structuralValue(originalKey); + KeyState keyState = + keyStateMap.computeIfAbsent(structuralKey, k -> new KeyState(originalKey)); + // Ignore any key from windmill that has been marked pending deletion or is + // fully cached. + if (keyState.existence == KeyExistence.KNOWN_NONEXISTENT + || (keyState.existence == KeyExistence.KNOWN_EXIST && keyState.valuesCached)) + return; + // Or else cache contents from windmill. + keyState.existence = KeyExistence.KNOWN_EXIST; + keyState.values.extendWith(entry.getValue()); + keyState.valuesSize += Iterables.size(entry.getValue()); + keyState.valuesCached = true; + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + allKeysKnown = true; + complete = true; + return Iterables.unmodifiableIterable( + unnestCachedEntries(mergedCachedEntries(null).entrySet())); + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public ReadableState>> readLater() { + WindmillMultimap.this.necessaryEntriesFromStorageFuture(false); + return this; + } + + /** + * Collect all cached entries into a map and all KNOWN_NONEXISTENT keys to + * knownNonexistentKeys(if not null). Note that this method is not side-effect-free: it unloads + * any key that is not KNOWN_EXIST and not pending deletion from cache; also if complete it + * marks the valuesCached of any key that is KNOWN_EXIST to true, entries() depends on this + * behavior when the fetched result is weighted to iterate the whole keyStateMap one less time. + * For each cached key, returns its structural key and a tuple of . + */ + private Map>> mergedCachedEntries( + Set knownNonexistentKeys) { + Map>> cachedEntries = Maps.newHashMap(); + keyStateMap + .entrySet() + .removeIf( + (entry -> { + Object structuralKey = entry.getKey(); + KeyState keyState = entry.getValue(); + if (complete && keyState.existence == KeyExistence.KNOWN_EXIST) { + keyState.valuesCached = true; + } + ConcatIterables it = null; + if (!keyState.localAdditions.isEmpty()) { + it = new ConcatIterables<>(); + it.extendWith( + Iterables.limit(keyState.localAdditions, keyState.localAdditions.size())); + } + if (keyState.valuesCached) { + if (it == null) it = new ConcatIterables<>(); + it.extendWith(Iterables.limit(keyState.values, keyState.valuesSize)); + } + if (it != null) { + cachedEntries.put( + structuralKey, Triple.of(keyState.originalKey, keyState.valuesCached, it)); + } + if (knownNonexistentKeys != null + && keyState.existence == KeyExistence.KNOWN_NONEXISTENT) + knownNonexistentKeys.add(structuralKey); + return (keyState.existence == KeyExistence.KNOWN_NONEXISTENT + && !keyState.removedLocally) + || keyState.existence == KeyExistence.UNKNOWN_EXISTENCE; + })); + return cachedEntries; + } + + private Iterable> nonWeightedEntries( + Iterable>> lazyWindmillEntries) { + class ResultIterable implements Iterable> { + private final Iterable>> lazyWindmillEntries; + private final Map>> cachedEntries; + private final Set knownNonexistentKeys; + + ResultIterable( + Map>> cachedEntries, + Iterable>> lazyWindmillEntries, + Set knownNonexistentKeys) { + this.cachedEntries = cachedEntries; + this.lazyWindmillEntries = lazyWindmillEntries; + this.knownNonexistentKeys = knownNonexistentKeys; + } + + @Override + public Iterator> iterator() { + // Each time when the Iterable returned by entries() is iterated, a new Iterator is + // created. Every iterator must keep its own copy of seenCachedKeys so that if a key + // is paginated into multiple iterables from windmill, the cached values of this key + // will only be returned once. + Set seenCachedKeys = Sets.newHashSet(); + // notFullyCachedEntries returns all entries from windmill that are not fully cached + // and combines them with localAdditions. If a key is fully cached, contents of this + // key from windmill are ignored. + Iterable>> notFullyCachedEntries = + Iterables.filter( + Iterables.transform( + lazyWindmillEntries, + entry -> { + try { + final K key = + keyCoder.decode(entry.getKey().newInput(), Coder.Context.OUTER); + final Object structuralKey = keyCoder.structuralValue(key); + // key is deleted in cache thus fully cached. + if (knownNonexistentKeys.contains(structuralKey)) return null; + Triple> triple = + cachedEntries.get(structuralKey); + // no record of key in cache, return content in windmill. + if (triple == null) { + return Triple.of(structuralKey, key, entry.getValue()); + } + // key is fully cached in cache. + if (triple.getMiddle()) return null; + + // key is not fully cached, combine the content in windmill with local + // additions with only the first observed page for the key to ensure + // it is not repeated. + if (!seenCachedKeys.add(structuralKey)) { + return Triple.of(structuralKey, key, entry.getValue()); + } else { + ConcatIterables it = new ConcatIterables<>(); + it.extendWith(triple.getRight()); + it.extendWith(entry.getValue()); + return Triple.of(structuralKey, key, it); + } + } catch (IOException e) { + throw new RuntimeException(e); + } + }), + Objects::nonNull); + Iterator> unnestWindmill = + Iterators.concat( + Iterables.transform( + notFullyCachedEntries, + entry -> + Iterables.transform( + entry.getRight(), + v -> new AbstractMap.SimpleEntry<>(entry.getMiddle(), v)) + .iterator()) + .iterator()); + Iterator> fullyCached = + unnestCachedEntries( + Iterables.filter( + cachedEntries.entrySet(), + entry -> !seenCachedKeys.contains(entry.getKey()))) + .iterator(); + return Iterators.concat(unnestWindmill, fullyCached); + } + } + + Set knownNonexistentKeys = Sets.newHashSet(); + Map>> cachedEntries = + mergedCachedEntries(knownNonexistentKeys); + return Iterables.unmodifiableIterable( + new ResultIterable(cachedEntries, lazyWindmillEntries, knownNonexistentKeys)); + } + } + + private class ContainsKeyReadableState implements ReadableState { + final Object structuralKey; + private final K key; + ReadableState> values; + + public ContainsKeyReadableState(K key) { + this.key = key; + structuralKey = keyCoder.structuralValue(key); + values = null; + } + + @Override + public Boolean read() { + KeyState keyState = keyStateMap.getOrDefault(structuralKey, null); + if (keyState != null && keyState.existence != KeyExistence.UNKNOWN_EXISTENCE) { + return keyState.existence == KeyExistence.KNOWN_EXIST; + } + if (values == null) { + values = WindmillMultimap.this.get(key); + } + return !Iterables.isEmpty(values.read()); + } + + @Override + public ReadableState readLater() { + if (values == null) { + values = WindmillMultimap.this.get(key); + } + values.readLater(); + return this; + } + } + + private class IsEmptyReadableState implements ReadableState { + ReadableState> keys = null; + + @Override + public Boolean read() { + for (KeyState keyState : keyStateMap.values()) { + if (keyState.existence == KeyExistence.KNOWN_EXIST) { + return false; + } + } + if (keys == null) { + keys = WindmillMultimap.this.keys(); + } + return Iterables.isEmpty(keys.read()); + } + + @Override + public ReadableState readLater() { + if (keys == null) { + keys = WindmillMultimap.this.keys(); + } + keys.readLater(); + return this; + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillOrderedList.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillOrderedList.java new file mode 100644 index 000000000000..c92e2e93ddfe --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillOrderedList.java @@ -0,0 +1,305 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import static org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateUtil.encodeKey; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Collections; +import java.util.Iterator; +import java.util.SortedSet; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTable; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.OrderedListState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +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.Iterables; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Range; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.RangeSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.TreeRangeSet; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; +import org.checkerframework.checker.nullness.qual.Nullable; +import org.joda.time.Instant; + +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WindmillOrderedList extends SimpleWindmillState implements OrderedListState { + // The default proto values for SortedListRange correspond to the minimum and maximum + // timestamps. + static final long MIN_TS_MICROS = Windmill.SortedListRange.getDefaultInstance().getStart(); + static final long MAX_TS_MICROS = Windmill.SortedListRange.getDefaultInstance().getLimit(); + private final ByteString stateKey; + private final String stateFamily; + private final Coder elemCoder; + // We need to sort based on timestamp, but we need objects with the same timestamp to be treated + // as unique. We can't use a MultiSet as we can't construct a comparator that uniquely + // identifies objects, + // so we construct a unique in-memory long ids for each element. + private final SortedSet> pendingAdds = + Sets.newTreeSet(TimestampedValueWithId.COMPARATOR); + private final RangeSet pendingDeletes = TreeRangeSet.create(); + private final IdTracker idTracker; + private boolean complete; + private boolean cleared = false; + + WindmillOrderedList( + StateTable derivedStateTable, + StateNamespace namespace, + StateTag> spec, + String stateFamily, + Coder elemCoder, + boolean isNewKey) { + + this.stateKey = encodeKey(namespace, spec); + this.stateFamily = stateFamily; + this.elemCoder = elemCoder; + this.complete = isNewKey; + this.idTracker = new IdTracker(derivedStateTable, namespace, spec); + } + + @Override + public Iterable> read() { + return readRange(null, null); + } + + private SortedSet> getPendingAddRange( + @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { + SortedSet> pendingInRange = pendingAdds; + if (minTimestamp != null && limitTimestamp != null) { + pendingInRange = + pendingInRange.subSet( + TimestampedValueWithId.bound(minTimestamp), + TimestampedValueWithId.bound(limitTimestamp)); + } else if (minTimestamp == null && limitTimestamp != null) { + pendingInRange = pendingInRange.headSet(TimestampedValueWithId.bound(limitTimestamp)); + } else if (limitTimestamp == null && minTimestamp != null) { + pendingInRange = pendingInRange.tailSet(TimestampedValueWithId.bound(minTimestamp)); + } + return pendingInRange; + } + + @Override + public Iterable> readRange( + @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { + idTracker.readLater(); + + final Future>> future = getFuture(minTimestamp, limitTimestamp); + try (Closeable ignored = scopedReadState()) { + SortedSet> pendingInRange = + getPendingAddRange(minTimestamp, limitTimestamp); + + // Transform the return iterator, so it has the same type as pendingAdds. We need to ensure + // that the ids don't overlap with any in pendingAdds, so begin with pendingAdds.size(). + Iterable> data = + new Iterable>() { + // Anything returned from windmill that has been deleted should be ignored. + private final Iterable> iterable = + Iterables.filter(future.get(), tv -> !pendingDeletes.contains(tv.getTimestamp())); + + @Override + public Iterator> iterator() { + return new Iterator>() { + private final Iterator> iter = iterable.iterator(); + private long currentId = pendingAdds.size(); + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public TimestampedValueWithId next() { + return TimestampedValueWithId.of(iter.next(), currentId++); + } + }; + } + }; + + Iterable> includingAdds = + Iterables.mergeSorted( + ImmutableList.of(data, pendingInRange), TimestampedValueWithId.COMPARATOR); + + // TODO(reuvenlax): If we have a known bounded amount of data, cache known ranges. + return Iterables.transform(includingAdds, TimestampedValueWithId::getValue); + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + } + + @Override + public void clear() { + cleared = true; + complete = true; + pendingAdds.clear(); + pendingDeletes.clear(); + try { + idTracker.clear(); + } catch (ExecutionException | InterruptedException e) { + throw new RuntimeException(e); + } + } + + @Override + public void clearRange(Instant minTimestamp, Instant limitTimestamp) { + getPendingAddRange(minTimestamp, limitTimestamp).clear(); + pendingDeletes.add(Range.closedOpen(minTimestamp, limitTimestamp)); + } + + @Override + public void add(TimestampedValue value) { + // We use the current size of the container as the in-memory id. This works because + // pendingAdds is completely + // cleared when it is processed (otherwise we could end up with duplicate elements in the same + // container). These + // are not the ids that will be sent to windmill. + pendingAdds.add(TimestampedValueWithId.of(value, pendingAdds.size())); + // Leave pendingDeletes alone. Since we can have multiple values with the same timestamp, we + // may still need + // overlapping deletes to remove previous entries at this timestamp. + } + + @Override + public ReadableState isEmpty() { + return new ReadableState() { + @Override + public ReadableState readLater() { + WindmillOrderedList.this.readLater(); + return this; + } + + @Override + public Boolean read() { + return Iterables.isEmpty(WindmillOrderedList.this.read()); + } + }; + } + + @Override + public OrderedListState readLater() { + return readRangeLater(null, null); + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public OrderedListState readRangeLater( + @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { + idTracker.readLater(); + getFuture(minTimestamp, limitTimestamp); + return this; + } + + @Override + public Windmill.WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + Windmill.TagSortedListUpdateRequest.Builder updatesBuilder = + commitBuilder + .addSortedListUpdatesBuilder() + .setStateFamily(cache.getStateFamily()) + .setTag(stateKey); + try { + if (cleared) { + // Default range. + updatesBuilder.addDeletesBuilder().build(); + cleared = false; + } + + if (!pendingAdds.isEmpty()) { + // TODO(reuvenlax): Once we start caching data, we should remove this line. We have it + // here now + // because once we persist + // added data we forget about it from the cache, so the object is no longer complete. + complete = false; + + Windmill.TagSortedListInsertRequest.Builder insertBuilder = + updatesBuilder.addInsertsBuilder(); + idTracker.add( + pendingAdds, + (elem, id) -> { + try { + ByteStringOutputStream elementStream = new ByteStringOutputStream(); + elemCoder.encode(elem.getValue(), elementStream, Coder.Context.OUTER); + insertBuilder.addEntries( + Windmill.SortedListEntry.newBuilder() + .setValue(elementStream.toByteString()) + .setSortKey( + WindmillTimeUtils.harnessToWindmillTimestamp(elem.getTimestamp())) + .setId(id)); + } catch (IOException e) { + throw new RuntimeException(e); + } + }); + pendingAdds.clear(); + insertBuilder.build(); + } + + if (!pendingDeletes.isEmpty()) { + for (Range range : pendingDeletes.asRanges()) { + Windmill.TagSortedListDeleteRequest.Builder deletesBuilder = + updatesBuilder.addDeletesBuilder(); + deletesBuilder.setRange( + Windmill.SortedListRange.newBuilder() + .setStart(WindmillTimeUtils.harnessToWindmillTimestamp(range.lowerEndpoint())) + .setLimit(WindmillTimeUtils.harnessToWindmillTimestamp(range.upperEndpoint()))); + deletesBuilder.build(); + idTracker.remove(range); + } + pendingDeletes.clear(); + } + } catch (ExecutionException | InterruptedException e) { + throw new RuntimeException(e); + } + return commitBuilder.buildPartial(); + } + + private Future>> getFuture( + @Nullable Instant minTimestamp, @Nullable Instant limitTimestamp) { + long startSortKey = + minTimestamp != null + ? WindmillTimeUtils.harnessToWindmillTimestamp(minTimestamp) + : MIN_TS_MICROS; + long limitSortKey = + limitTimestamp != null + ? WindmillTimeUtils.harnessToWindmillTimestamp(limitTimestamp) + : MAX_TS_MICROS; + + if (complete) { + // Right now we don't cache any data, so complete means an empty list. + // TODO(reuvenlax): change this once we start caching data. + return Futures.immediateFuture(Collections.emptyList()); + } + return reader.orderedListFuture( + Range.closedOpen(startSortKey, limitSortKey), stateKey, stateFamily, elemCoder); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillSet.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillSet.java new file mode 100644 index 000000000000..4afb879e722e --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillSet.java @@ -0,0 +1,146 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.io.Closeable; +import java.io.IOException; +import java.util.Optional; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.core.StateTags; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.BooleanCoder; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.MapState; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.SetState; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; +import org.checkerframework.checker.initialization.qual.Initialized; +import org.checkerframework.checker.nullness.qual.NonNull; +import org.checkerframework.checker.nullness.qual.UnknownKeyFor; + +public class WindmillSet extends SimpleWindmillState implements SetState { + private final WindmillMap windmillMap; + + WindmillSet( + StateNamespace namespace, + StateTag> address, + String stateFamily, + Coder keyCoder, + WindmillStateCache.ForKeyAndFamily cache, + boolean isNewKey) { + StateTag> internalMapAddress = StateTags.convertToMapTagInternal(address); + + this.windmillMap = + cache + .get(namespace, internalMapAddress) + .map(map -> (WindmillMap) map) + .orElseGet( + () -> + new WindmillMap<>( + namespace, + internalMapAddress, + stateFamily, + keyCoder, + BooleanCoder.of(), + isNewKey)); + } + + @Override + protected Windmill.WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + return windmillMap.persistDirectly(cache); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState< + @UnknownKeyFor @NonNull @Initialized Boolean> + contains(K k) { + return windmillMap.getOrDefault(k, false); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState< + @UnknownKeyFor @NonNull @Initialized Boolean> + addIfAbsent(K k) { + return new WindmillSetAddIfAbsentReadableState(k); + } + + @Override + public void remove(K k) { + windmillMap.remove(k); + } + + @Override + public void add(K value) { + windmillMap.put(value, true); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState< + @UnknownKeyFor @NonNull @Initialized Boolean> + isEmpty() { + return windmillMap.isEmpty(); + } + + @Override + public Iterable read() { + return windmillMap.keys().read(); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized SetState readLater() { + windmillMap.keys().readLater(); + return this; + } + + @Override + public void clear() { + windmillMap.clear(); + } + + @Override + void initializeForWorkItem( + WindmillStateReader reader, Supplier scopedReadStateSupplier) { + windmillMap.initializeForWorkItem(reader, scopedReadStateSupplier); + } + + @Override + void cleanupAfterWorkItem() { + windmillMap.cleanupAfterWorkItem(); + } + + private class WindmillSetAddIfAbsentReadableState implements ReadableState { + ReadableState putState; + + public WindmillSetAddIfAbsentReadableState(K k) { + putState = windmillMap.putIfAbsent(k, true); + } + + @Override + public Boolean read() { + return Optional.ofNullable(putState.read()).orElse(false); + } + + @Override + public @UnknownKeyFor @NonNull @Initialized ReadableState readLater() { + putState = putState.readLater(); + return this; + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillState.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillState.java new file mode 100644 index 000000000000..59fd3f8a1b37 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillState.java @@ -0,0 +1,68 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.Future; +import javax.annotation.concurrent.NotThreadSafe; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; + +/** + * Abstract base class for all Windmill state. + * + *

Note that these are not thread safe; each state object is associated with a key and thus only + * accessed by a single thread at once. + */ +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +@NotThreadSafe +abstract class WindmillState { + protected Supplier scopedReadStateSupplier; + protected WindmillStateReader reader; + + /** + * Return an asynchronously computed {@link Windmill.WorkItemCommitRequest}. The request should be + * of a form that can be merged with others (only add to repeated fields). + */ + abstract Future persist(WindmillStateCache.ForKeyAndFamily cache) + throws IOException; + + /** Prepare this (possibly reused from cache) state for reading from {@code reader} if needed. */ + void initializeForWorkItem( + WindmillStateReader reader, Supplier scopedReadStateSupplier) { + this.reader = reader; + this.scopedReadStateSupplier = scopedReadStateSupplier; + } + + /** + * This (now cached) state should never need to interact with the reader until the next work item. + * Clear it to prevent space leaks. The reader will be reset by {@link #initializeForWorkItem} + * upon the next work item. + */ + void cleanupAfterWorkItem() { + this.reader = null; + this.scopedReadStateSupplier = null; + } + + Closeable scopedReadState() { + return scopedReadStateSupplier.get(); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java similarity index 92% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java index 700c7bbe01c2..6c1239d6ebd2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateCache.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCache.java @@ -15,12 +15,13 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker; +package org.apache.beam.runners.dataflow.worker.windmill.state; import java.io.IOException; import java.io.PrintWriter; import java.util.HashMap; import java.util.Objects; +import java.util.Optional; import java.util.concurrent.ConcurrentMap; import java.util.function.BiConsumer; import javax.servlet.http.HttpServletRequest; @@ -28,6 +29,9 @@ import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.core.StateTags; +import org.apache.beam.runners.dataflow.worker.StreamingDataflowWorker; +import org.apache.beam.runners.dataflow.worker.Weighers; +import org.apache.beam.runners.dataflow.worker.WindmillComputationKey; import org.apache.beam.runners.dataflow.worker.status.BaseStatusServlet; import org.apache.beam.runners.dataflow.worker.status.StatusDataProvider; import org.apache.beam.sdk.state.State; @@ -84,14 +88,6 @@ public WindmillStateCache(long workerCacheMb) { .build(); } - private static class EntryStats { - long entries; - long idWeight; - long entryWeight; - long entryValues; - long maxEntryValues; - } - private EntryStats calculateEntryStats() { EntryStats stats = new EntryStats(); BiConsumer consumer = @@ -119,130 +115,52 @@ public CacheStats getCacheStats() { return stateCache.stats(); } - /** Per-computation view of the state cache. */ - public class ForComputation { - - private final String computation; - - private ForComputation(String computation) { - this.computation = computation; - } - - /** Invalidate all cache entries for this computation and {@code processingKey}. */ - public void invalidate(ByteString processingKey, long shardingKey) { - WindmillComputationKey key = - WindmillComputationKey.create(computation, processingKey, shardingKey); - // By removing the ForKey object, all state for the key is orphaned in the cache and will - // be removed by normal cache cleanup. - keyIndex.remove(key); - } - - /** - * Returns a per-computation, per-key view of the state cache. Access to the cached data for - * this key is not thread-safe. Callers should ensure that there is only a single ForKey object - * in use at a time and that access to it is synchronized or single-threaded. - */ - public ForKey forKey(WindmillComputationKey computationKey, long cacheToken, long workToken) { - ForKey forKey = keyIndex.get(computationKey); - if (forKey == null || !forKey.updateTokens(cacheToken, workToken)) { - forKey = new ForKey(computationKey, cacheToken, workToken); - // We prefer this implementation to using compute because that is implemented similarly for - // ConcurrentHashMap with the downside of it performing inserts for unchanged existing - // values as well. - keyIndex.put(computationKey, forKey); - } - return forKey; - } + /** Returns a per-computation view of the state cache. */ + public ForComputation forComputation(String computation) { + return new ForComputation(computation); } - /** Per-computation, per-key view of the state cache. */ - // Note that we utilize the default equality and hashCode for this class based upon the instance - // (instead of the fields) to optimize cache invalidation. - public class ForKey { - private final WindmillComputationKey computationKey; - // Cache token must be consistent for the key for the cache to be valid. - private final long cacheToken; - - // The work token for processing must be greater than the last work token. As work items are - // increasing for a key, a less-than or equal to work token indicates that the current token is - // for stale processing. - private long workToken; - - /** - * Returns a per-computation, per-key, per-family view of the state cache. Access to the cached - * data for this key is not thread-safe. Callers should ensure that there is only a single - * ForKeyAndFamily object in use at a time for a given computation, key, family tuple and that - * access to it is synchronized or single-threaded. - */ - public ForKeyAndFamily forFamily(String stateFamily) { - return new ForKeyAndFamily(this, stateFamily); - } - - private ForKey(WindmillComputationKey computationKey, long cacheToken, long workToken) { - this.computationKey = computationKey; - this.cacheToken = cacheToken; - this.workToken = workToken; - } - - private boolean updateTokens(long cacheToken, long workToken) { - if (this.cacheToken != cacheToken || workToken <= this.workToken) { - return false; - } - this.workToken = workToken; - return true; - } + /** Print summary statistics of the cache to the given {@link PrintWriter}. */ + @Override + public void appendSummaryHtml(PrintWriter response) { + response.println("Cache Stats:
"); + response.println( + "" + + "" + + "" + + ""); + CacheStats cacheStats = stateCache.stats(); + EntryStats entryStats = calculateEntryStats(); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println(""); + response.println("
Hit RatioEvictionsEntriesEntry ValuesMax Entry ValuesId WeightEntry WeightMax WeightKeys
" + cacheStats.hitRate() + "" + cacheStats.evictionCount() + "" + entryStats.entries + "(" + stateCache.size() + " inc. weak) " + entryStats.entryValues + "" + entryStats.maxEntryValues + "" + entryStats.idWeight / MEGABYTES + "MB" + entryStats.entryWeight / MEGABYTES + "MB" + getMaxWeight() / MEGABYTES + "MB" + keyIndex.size() + "

"); } - /** - * Per-computation, per-key, per-family view of the state cache. Modifications are cached locally - * and must be flushed to the cache by calling persist. This class is not thread-safe. - */ - public class ForKeyAndFamily { - final ForKey forKey; - final String stateFamily; - private final HashMap localCache; - - private ForKeyAndFamily(ForKey forKey, String stateFamily) { - this.forKey = forKey; - this.stateFamily = stateFamily; - localCache = new HashMap<>(); - } - - public String getStateFamily() { - return stateFamily; - } - - public @Nullable T get(StateNamespace namespace, StateTag address) { - StateId id = new StateId(forKey, stateFamily, namespace); - @SuppressWarnings("nullness") // Unsure how to annotate lambda return allowing null. - @Nullable - StateCacheEntry entry = localCache.computeIfAbsent(id, key -> stateCache.getIfPresent(key)); - return entry == null ? null : entry.get(namespace, address); - } - - public void put( - StateNamespace namespace, StateTag address, T value, long weight) { - StateId id = new StateId(forKey, stateFamily, namespace); - @Nullable StateCacheEntry entry = localCache.get(id); - if (entry == null) { - entry = stateCache.getIfPresent(id); - if (entry == null) { - entry = new StateCacheEntry(); - } - boolean hadValue = localCache.putIfAbsent(id, entry) != null; - Preconditions.checkState(!hadValue); + public BaseStatusServlet statusServlet() { + return new BaseStatusServlet("/cachez") { + @Override + protected void doGet(HttpServletRequest request, HttpServletResponse response) + throws IOException { + PrintWriter writer = response.getWriter(); + writer.println("

Cache Information

"); + appendSummaryHtml(writer); } - entry.put(namespace, address, value, weight); - } - - public void persist() { - localCache.forEach((id, entry) -> stateCache.put(id, entry)); - } + }; } - /** Returns a per-computation view of the state cache. */ - public ForComputation forComputation(String computation) { - return new ForComputation(computation); + private static class EntryStats { + long entries; + long idWeight; + long entryWeight; + long entryValues; + long maxEntryValues; } /** @@ -297,12 +215,10 @@ public StateCacheEntry() { this.weight = 0; } - public @Nullable T get(StateNamespace namespace, StateTag tag) { - @SuppressWarnings("unchecked") - @Nullable - WeightedValue weightedValue = - (WeightedValue) values.get(new NamespacedTag<>(namespace, tag)); - return weightedValue == null ? null : weightedValue.value; + @SuppressWarnings("unchecked") + public Optional get(StateNamespace namespace, StateTag tag) { + return Optional.ofNullable((WeightedValue) values.get(new NamespacedTag<>(namespace, tag))) + .flatMap(WeightedValue::value); } public void put( @@ -362,43 +278,137 @@ public int hashCode() { } private static class WeightedValue { - public long weight; - public @Nullable T value; + private long weight; + private @Nullable T value; + + private Optional value() { + return Optional.ofNullable(this.value); + } } } - /** Print summary statistics of the cache to the given {@link PrintWriter}. */ - @Override - public void appendSummaryHtml(PrintWriter response) { - response.println("Cache Stats:
"); - response.println( - "" - + "" - + "" - + ""); - CacheStats cacheStats = stateCache.stats(); - EntryStats entryStats = calculateEntryStats(); - response.println(""); - response.println(""); - response.println(""); - response.println(""); - response.println(""); - response.println(""); - response.println(""); - response.println(""); - response.println(""); - response.println("
Hit RatioEvictionsEntriesEntry ValuesMax Entry ValuesId WeightEntry WeightMax WeightKeys
" + cacheStats.hitRate() + "" + cacheStats.evictionCount() + "" + entryStats.entries + "(" + stateCache.size() + " inc. weak) " + entryStats.entryValues + "" + entryStats.maxEntryValues + "" + entryStats.idWeight / MEGABYTES + "MB" + entryStats.entryWeight / MEGABYTES + "MB" + getMaxWeight() / MEGABYTES + "MB" + keyIndex.size() + "

"); + /** Per-computation view of the state cache. */ + public class ForComputation { + + private final String computation; + + private ForComputation(String computation) { + this.computation = computation; + } + + /** Invalidate all cache entries for this computation and {@code processingKey}. */ + public void invalidate(ByteString processingKey, long shardingKey) { + WindmillComputationKey key = + WindmillComputationKey.create(computation, processingKey, shardingKey); + // By removing the ForKey object, all state for the key is orphaned in the cache and will + // be removed by normal cache cleanup. + keyIndex.remove(key); + } + + /** + * Returns a per-computation, per-key view of the state cache. Access to the cached data for + * this key is not thread-safe. Callers should ensure that there is only a single ForKey object + * in use at a time and that access to it is synchronized or single-threaded. + */ + public ForKey forKey(WindmillComputationKey computationKey, long cacheToken, long workToken) { + ForKey forKey = keyIndex.get(computationKey); + if (forKey == null || !forKey.updateTokens(cacheToken, workToken)) { + forKey = new ForKey(computationKey, cacheToken, workToken); + // We prefer this implementation to using compute because that is implemented similarly for + // ConcurrentHashMap with the downside of it performing inserts for unchanged existing + // values as well. + keyIndex.put(computationKey, forKey); + } + return forKey; + } } - public BaseStatusServlet statusServlet() { - return new BaseStatusServlet("/cachez") { - @Override - protected void doGet(HttpServletRequest request, HttpServletResponse response) - throws IOException { - PrintWriter writer = response.getWriter(); - writer.println("

Cache Information

"); - appendSummaryHtml(writer); + /** Per-computation, per-key view of the state cache. */ + // Note that we utilize the default equality and hashCode for this class based upon the instance + // (instead of the fields) to optimize cache invalidation. + public class ForKey { + private final WindmillComputationKey computationKey; + // Cache token must be consistent for the key for the cache to be valid. + private final long cacheToken; + + // The work token for processing must be greater than the last work token. As work items are + // increasing for a key, a less-than or equal to work token indicates that the current token is + // for stale processing. + private long workToken; + + private ForKey(WindmillComputationKey computationKey, long cacheToken, long workToken) { + this.computationKey = computationKey; + this.cacheToken = cacheToken; + this.workToken = workToken; + } + + /** + * Returns a per-computation, per-key, per-family view of the state cache. Access to the cached + * data for this key is not thread-safe. Callers should ensure that there is only a single + * ForKeyAndFamily object in use at a time for a given computation, key, family tuple and that + * access to it is synchronized or single-threaded. + */ + public ForKeyAndFamily forFamily(String stateFamily) { + return new ForKeyAndFamily(this, stateFamily); + } + + private boolean updateTokens(long cacheToken, long workToken) { + if (this.cacheToken != cacheToken || workToken <= this.workToken) { + return false; } - }; + this.workToken = workToken; + return true; + } + } + + /** + * Per-computation, per-key, per-family view of the state cache. Modifications are cached locally + * and must be flushed to the cache by calling persist. This class is not thread-safe. + */ + public class ForKeyAndFamily { + final ForKey forKey; + final String stateFamily; + private final HashMap localCache; + + private ForKeyAndFamily(ForKey forKey, String stateFamily) { + this.forKey = forKey; + this.stateFamily = stateFamily; + localCache = new HashMap<>(); + } + + public String getStateFamily() { + return stateFamily; + } + + public Optional get(StateNamespace namespace, StateTag address) { + @SuppressWarnings("nullness") + // the mapping function for localCache.computeIfAbsent (i.e stateCache.getIfPresent) is + // nullable. + Optional stateCacheEntry = + Optional.ofNullable( + localCache.computeIfAbsent( + new StateId(forKey, stateFamily, namespace), stateCache::getIfPresent)); + + return stateCacheEntry.flatMap(entry -> entry.get(namespace, address)); + } + + public void put( + StateNamespace namespace, StateTag address, T value, long weight) { + StateId id = new StateId(forKey, stateFamily, namespace); + @Nullable StateCacheEntry entry = localCache.get(id); + if (entry == null) { + entry = stateCache.getIfPresent(id); + if (entry == null) { + entry = new StateCacheEntry(); + } + boolean hadValue = localCache.putIfAbsent(id, entry) != null; + Preconditions.checkState(!hadValue); + } + entry.put(namespace, address, value, weight); + } + + public void persist() { + localCache.forEach(stateCache::put); + } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternals.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternals.java new file mode 100644 index 000000000000..c900228e86b0 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternals.java @@ -0,0 +1,171 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.io.Closeable; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Random; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import org.apache.beam.runners.core.StateInternals; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTable; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.WorkItemCommitRequest; +import org.apache.beam.sdk.state.State; +import org.apache.beam.sdk.state.StateContext; +import org.apache.beam.sdk.state.StateContexts; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Supplier; +import org.checkerframework.checker.nullness.qual.Nullable; + +/** Implementation of {@link StateInternals} using Windmill to manage the underlying data. */ +@SuppressWarnings("nullness" // TODO(https://github.com/apache/beam/issues/20497) +) +public class WindmillStateInternals implements StateInternals { + + @VisibleForTesting + static final ThreadLocal> COMPACT_NOW = + ThreadLocal.withInitial(ShouldCompactNowFn::new); + /** + * The key will be null when not in a keyed context, from the users perspective. There is still a + * "key" for the Windmill computation, but it cannot be meaningfully deserialized. + */ + private final @Nullable K key; + + private final WindmillStateCache.ForKeyAndFamily cache; + private final StateTable workItemState; + private final StateTable workItemDerivedState; + private final Supplier scopedReadStateSupplier; + + public WindmillStateInternals( + @Nullable K key, + String stateFamily, + WindmillStateReader reader, + boolean isNewKey, + WindmillStateCache.ForKeyAndFamily cache, + Supplier scopedReadStateSupplier) { + this.key = key; + this.cache = cache; + this.scopedReadStateSupplier = scopedReadStateSupplier; + this.workItemDerivedState = + CachingStateTable.builder(stateFamily, reader, cache, isNewKey, scopedReadStateSupplier) + .build(); + this.workItemState = + CachingStateTable.builder(stateFamily, reader, cache, isNewKey, scopedReadStateSupplier) + .withDerivedState(workItemDerivedState) + .build(); + } + + @Override + public @Nullable K getKey() { + return key; + } + + private void persist(List> commitsToMerge, StateTable stateTable) { + for (State location : stateTable.values()) { + if (!(location instanceof WindmillState)) { + throw new IllegalStateException( + String.format( + "%s wasn't created by %s -- unable to persist it", + location.getClass().getSimpleName(), getClass().getSimpleName())); + } + + try { + commitsToMerge.add(((WindmillState) location).persist(cache)); + } catch (IOException e) { + throw new RuntimeException("Unable to persist state", e); + } + } + + // All cached State objects now have known values. + // Clear any references to the underlying reader to prevent space leaks. + // The next work unit to use these cached State objects will reset the + // reader to a current reader in case those values are modified. + for (State location : stateTable.values()) { + ((WindmillState) location).cleanupAfterWorkItem(); + } + + // Clear out the map of already retrieved state instances. + stateTable.clear(); + } + + public void persist(final Windmill.WorkItemCommitRequest.Builder commitBuilder) { + List> commitsToMerge = new ArrayList<>(); + + // Call persist on each first, which may schedule some futures for reading. + persist(commitsToMerge, workItemState); + persist(commitsToMerge, workItemDerivedState); + + try (Closeable ignored = scopedReadStateSupplier.get()) { + for (Future commitFuture : commitsToMerge) { + commitBuilder.mergeFrom(commitFuture.get()); + } + } catch (ExecutionException | InterruptedException | IOException exc) { + if (exc instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Failed to retrieve Windmill state during persist()", exc); + } + + cache.persist(); + } + + @Override + public T state(StateNamespace namespace, StateTag address) { + return workItemState.get(namespace, address, StateContexts.nullContext()); + } + + @Override + public T state( + StateNamespace namespace, StateTag address, StateContext c) { + return workItemState.get(namespace, address, c); + } + + private static class ShouldCompactNowFn implements Supplier { + /* The rate at which, on average, this will return true. */ + private static final double RATE = 0.002; + private final Random random; + private long counter; + + private ShouldCompactNowFn() { + this.random = new Random(); + this.counter = nextSample(random); + } + + private static long nextSample(Random random) { + // Use geometric distribution to find next true value. + // This lets us avoid invoking random.nextDouble() on every call. + return (long) Math.floor(Math.log(random.nextDouble()) / Math.log(1 - RATE)); + } + + @Override + public Boolean get() { + counter--; + if (counter < 0) { + counter = nextSample(random); + return true; + } else { + return false; + } + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java similarity index 70% rename from runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateReader.java rename to runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java index 192a40299e0a..07d652992c1c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillStateReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReader.java @@ -15,11 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker; +package org.apache.beam.runners.dataflow.worker.windmill.state; import com.google.api.client.util.Lists; -import com.google.auto.value.AutoValue; -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; import java.io.IOException; import java.io.InputStream; import java.util.AbstractMap; @@ -27,43 +25,41 @@ import java.util.Collections; import java.util.Comparator; import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; -import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; import java.util.function.Supplier; import java.util.stream.Collectors; -import javax.annotation.Nonnull; import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.worker.WindmillStateReader.StateTag.Kind; +import org.apache.beam.runners.dataflow.worker.KeyTokenInvalidException; +import org.apache.beam.runners.dataflow.worker.MetricTrackingWindmillServerStub; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataResponse; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListEntry; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListRange; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagBag; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListFetchRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagValue; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagValuePrefixRequest; +import org.apache.beam.runners.dataflow.worker.windmill.state.StateTag.Kind; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.Coder.Context; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.Weighted; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Function; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.AbstractIterator; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ForwardingList; 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.Range; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; -import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ForwardingFuture; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.SettableFuture; import org.joda.time.Instant; @@ -80,7 +76,7 @@ "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) "nullness" // TODO(https://github.com/apache/beam/issues/20497) }) -class WindmillStateReader { +public class WindmillStateReader { /** * Ideal maximum bytes in a TagBag response. However, Windmill will always return at least one * value if possible irrespective of this limit. @@ -116,190 +112,44 @@ class WindmillStateReader { public static final long MAX_KEY_BYTES = 16L << 20; // 16MB public static final long MAX_CONTINUATION_KEY_BYTES = 72L << 20; // 72MB - - /** - * When combined with a key and computationId, represents the unique address for state managed by - * Windmill. - */ - @AutoValue - abstract static class StateTag { - enum Kind { - VALUE, - BAG, - WATERMARK, - ORDERED_LIST, - VALUE_PREFIX, - MULTIMAP_SINGLE_ENTRY, - MULTIMAP_ALL - } - - abstract Kind getKind(); - - abstract ByteString getTag(); - - abstract String getStateFamily(); - - /** - * For {@link Kind#BAG, Kind#ORDERED_LIST, Kind#VALUE_PREFIX, KIND#MULTIMAP_SINGLE_ENTRY, - * KIND#MULTIMAP_ALL} kinds: A previous 'continuation_position' returned by Windmill to signal - * the resulting state was incomplete. Sending that position will request the next page of - * values. Null for first request. - * - *

Null for other kinds. - */ - @Nullable - abstract RequestPositionT getRequestPosition(); - - /** For {@link Kind#ORDERED_LIST} kinds: the range to fetch or delete. */ - @Nullable - abstract Range getSortedListRange(); - - /** For {@link Kind#MULTIMAP_SINGLE_ENTRY} kinds: the key in the multimap to fetch or delete. */ - @Nullable - abstract ByteString getMultimapKey(); - - /** - * For {@link Kind#MULTIMAP_ALL} kinds: will only return the keys of the multimap and not the - * values if true. - */ - @Nullable - abstract Boolean getOmitValues(); - - static StateTag of( - Kind kind, ByteString tag, String stateFamily, @Nullable RequestPositionT requestPosition) { - return new AutoValue_WindmillStateReader_StateTag.Builder() - .setKind(kind) - .setTag(tag) - .setStateFamily(stateFamily) - .setRequestPosition(requestPosition) - .build(); - } - - static StateTag of( - Kind kind, ByteString tag, String stateFamily) { - return of(kind, tag, stateFamily, null); - } - - abstract Builder toBuilder(); - - @AutoValue.Builder - abstract static class Builder { - abstract Builder setKind(Kind kind); - - abstract Builder setTag(ByteString tag); - - abstract Builder setStateFamily(String stateFamily); - - abstract Builder setRequestPosition( - @Nullable RequestPositionT requestPosition); - - abstract Builder setSortedListRange(@Nullable Range sortedListRange); - - abstract Builder setMultimapKey(@Nullable ByteString encodedMultimapKey); - - abstract Builder setOmitValues(Boolean omitValues); - - abstract StateTag build(); - } - } - - /** - * An in-memory collection of deserialized values and an optional continuation position to pass to - * Windmill when fetching the next page of values. - */ - private static class ValuesAndContPosition { - private final List values; - - /** Position to pass to next request for next page of values. Null if done. */ - private final @Nullable ContinuationT continuationPosition; - - public ValuesAndContPosition(List values, @Nullable ContinuationT continuationPosition) { - this.values = values; - this.continuationPosition = continuationPosition; - } - } - + @VisibleForTesting final ConcurrentLinkedQueue> pendingLookups; private final String computation; private final ByteString key; private final long shardingKey; private final long workToken; - // WindmillStateReader should only perform blocking i/o in a try-with-resources block that // declares an AutoCloseable vended by readWrapperSupplier. private final Supplier readWrapperSupplier; - - private final MetricTrackingWindmillServerStub server; - + private final MetricTrackingWindmillServerStub metricTrackingWindmillServerStub; + private final ConcurrentHashMap, CoderAndFuture> waiting; private long bytesRead = 0L; public WindmillStateReader( - MetricTrackingWindmillServerStub server, + MetricTrackingWindmillServerStub metricTrackingWindmillServerStub, String computation, ByteString key, long shardingKey, long workToken, Supplier readWrapperSupplier) { - this.server = server; + this.metricTrackingWindmillServerStub = metricTrackingWindmillServerStub; this.computation = computation; this.key = key; this.shardingKey = shardingKey; this.workToken = workToken; this.readWrapperSupplier = readWrapperSupplier; + this.waiting = new ConcurrentHashMap<>(); + this.pendingLookups = new ConcurrentLinkedQueue<>(); } public WindmillStateReader( - MetricTrackingWindmillServerStub server, + MetricTrackingWindmillServerStub metricTrackingWindmillServerStub, String computation, ByteString key, long shardingKey, long workToken) { - this(server, computation, key, shardingKey, workToken, () -> null); + this(metricTrackingWindmillServerStub, computation, key, shardingKey, workToken, () -> null); } - private static final class CoderAndFuture { - private Coder coder = null; - private final SettableFuture future; - - private CoderAndFuture(Coder coder, SettableFuture future) { - this.coder = coder; - this.future = future; - } - - private SettableFuture getFuture() { - return future; - } - - private SettableFuture getNonDoneFuture(StateTag stateTag) { - if (future.isDone()) { - throw new IllegalStateException("Future for " + stateTag + " is already done"); - } - return future; - } - - private Coder getAndClearCoder() { - if (coder == null) { - throw new IllegalStateException("Coder has already been cleared from cache"); - } - Coder result = (Coder) coder; - if (result == null) { - throw new IllegalStateException("Coder has already been cleared from cache"); - } - coder = null; - return result; - } - - private void checkNoCoder() { - if (coder != null) { - throw new IllegalStateException("Unexpected coder"); - } - } - } - - @VisibleForTesting - ConcurrentLinkedQueue> pendingLookups = new ConcurrentLinkedQueue<>(); - - private ConcurrentHashMap, CoderAndFuture> waiting = new ConcurrentHashMap<>(); - private Future stateFuture(StateTag stateTag, @Nullable Coder coder) { CoderAndFuture coderAndFuture = new CoderAndFuture<>(coder, SettableFuture.create()); CoderAndFuture existingCoderAndFutureWildcard = @@ -393,9 +243,9 @@ public Future>> valuePrefixFuture( * Internal request to fetch the next 'page' of values. Return null if no continuation position is * in {@code contStateTag}, which signals there are no more pages. */ - private @Nullable - Future> continuationFuture( - StateTag contStateTag, Coder coder) { + @Nullable + Future> continuationFuture( + StateTag contStateTag, Coder coder) { if (contStateTag.getRequestPosition() == null) { // We're done. return null; @@ -403,45 +253,6 @@ Future> continuationFuture( return stateFuture(contStateTag, coder); } - /** - * A future which will trigger a GetData request to Windmill for all outstanding futures on the - * first {@link #get}. - */ - private static class WrappedFuture extends ForwardingFuture.SimpleForwardingFuture { - /** - * The reader we'll use to service the eventual read. Null if read has been fulfilled. - * - *

NOTE: We must clear this after the read is fulfilled to prevent space leaks. - */ - private @Nullable WindmillStateReader reader; - - public WrappedFuture(WindmillStateReader reader, Future delegate) { - super(delegate); - this.reader = reader; - } - - @Override - public T get() throws InterruptedException, ExecutionException { - if (!delegate().isDone() && reader != null) { - // Only one thread per reader, so no race here. - reader.startBatchAndBlock(); - } - reader = null; - return super.get(); - } - - @Override - public T get(long timeout, TimeUnit unit) - throws InterruptedException, ExecutionException, TimeoutException { - if (!delegate().isDone() && reader != null) { - // Only one thread per reader, so no race here. - reader.startBatchAndBlock(); - } - reader = null; - return super.get(timeout, unit); - } - } - private Future wrappedFuture(final Future future) { if (future.isDone()) { // If the underlying lookup is already complete, we don't need to create the wrapper. @@ -452,59 +263,6 @@ private Future wrappedFuture(final Future future) { } } - /** Function to extract an {@link Iterable} from the continuation-supporting page read future. */ - private static class ToIterableFunction - implements Function, Iterable> { - /** - * Reader to request continuation pages from, or {@literal null} if no continuation pages - * required. - */ - private @Nullable WindmillStateReader reader; - - private final StateTag stateTag; - private final Coder coder; - - public ToIterableFunction( - WindmillStateReader reader, StateTag stateTag, Coder coder) { - this.reader = reader; - this.stateTag = stateTag; - this.coder = coder; - } - - @SuppressFBWarnings( - value = "NP_METHOD_PARAMETER_TIGHTENS_ANNOTATION", - justification = "https://github.com/google/guava/issues/920") - @Override - public Iterable apply( - @Nonnull ValuesAndContPosition valuesAndContPosition) { - if (valuesAndContPosition.continuationPosition == null) { - // Number of values is small enough Windmill sent us the entire bag in one response. - reader = null; - return valuesAndContPosition.values; - } else { - // Return an iterable which knows how to come back for more. - StateTag.Builder continuationTBuilder = - StateTag.of( - stateTag.getKind(), - stateTag.getTag(), - stateTag.getStateFamily(), - valuesAndContPosition.continuationPosition) - .toBuilder(); - if (stateTag.getSortedListRange() != null) { - continuationTBuilder.setSortedListRange(stateTag.getSortedListRange()).build(); - } - if (stateTag.getMultimapKey() != null) { - continuationTBuilder.setMultimapKey(stateTag.getMultimapKey()).build(); - } - if (stateTag.getOmitValues() != null) { - continuationTBuilder.setOmitValues(stateTag.getOmitValues()).build(); - } - return new PagingIterable( - reader, valuesAndContPosition.values, continuationTBuilder.build(), coder); - } - } - } - /** * Return future which transforms a {@code ValuesAndContPosition} result into the initial * Iterable result expected from the external caller. @@ -588,14 +346,7 @@ public void startBatchAndBlock() { return; } - Windmill.KeyedGetDataRequest request = createRequest(toFetch); - Windmill.KeyedGetDataResponse response; - try (AutoCloseable readWrapper = readWrapperSupplier.get()) { - response = server.getStateData(computation, request); - } - if (response == null) { - throw new RuntimeException("Windmill unexpectedly returned null for request " + request); - } + KeyedGetDataResponse response = tryGetDataFromWindmill(toFetch); // Removes tags from toFetch as they are processed. consumeResponse(response, toFetch); @@ -605,17 +356,31 @@ public void startBatchAndBlock() { for (StateTag stateTag : toFetch) { waiting.get(stateTag).future.setException(e); } + throw new RuntimeException(e); } } + private KeyedGetDataResponse tryGetDataFromWindmill(HashSet> stateTags) + throws Exception { + KeyedGetDataRequest keyedGetDataRequest = createRequest(stateTags); + try (AutoCloseable ignored = readWrapperSupplier.get()) { + return Optional.ofNullable( + metricTrackingWindmillServerStub.getStateData(computation, keyedGetDataRequest)) + .orElseThrow( + () -> + new RuntimeException( + "Windmill unexpectedly returned null for request " + keyedGetDataRequest)); + } + } + public long getBytesRead() { return bytesRead; } - private Windmill.KeyedGetDataRequest createRequest(Iterable> toFetch) { - Windmill.KeyedGetDataRequest.Builder keyedDataBuilder = - Windmill.KeyedGetDataRequest.newBuilder() + private KeyedGetDataRequest createRequest(Iterable> toFetch) { + KeyedGetDataRequest.Builder keyedDataBuilder = + KeyedGetDataRequest.newBuilder() .setKey(key) .setShardingKey(shardingKey) .setWorkToken(workToken); @@ -760,7 +525,7 @@ private Windmill.KeyedGetDataRequest createRequest(Iterable> toFetch return keyedDataBuilder.build(); } - private void consumeResponse(Windmill.KeyedGetDataResponse response, Set> toFetch) { + private void consumeResponse(KeyedGetDataResponse response, Set> toFetch) { bytesRead += response.getSerializedSize(); if (response.getFailed()) { throw new KeyTokenInvalidException(key.toStringUtf8()); @@ -885,41 +650,10 @@ private void consumeResponse(Windmill.KeyedGetDataResponse response, Set extends ForwardingList implements Weighted { - private List delegate; - long weight; - - WeightedList(List delegate) { - this.delegate = delegate; - this.weight = 0; - } - - @Override - protected List delegate() { - return delegate; - } - - @Override - public boolean add(T elem) { - throw new UnsupportedOperationException("Must use AddWeighted()"); - } - - @Override - public long getWeight() { - return weight; - } - - public void addWeighted(T elem, long weight) { - delegate.add(elem); - this.weight += weight; - } - } - /** The deserialized values in {@code bag} as a read-only array list. */ private List bagPageValues(TagBag bag, Coder elemCoder) { if (bag.getValuesCount() == 0) { - return new WeightedList(Collections.emptyList()); + return new WeightedList(Collections.emptyList()); } WeightedList valueList = new WeightedList<>(new ArrayList(bag.getValuesCount())); @@ -1013,23 +747,19 @@ private List>> multimapPageValues( private void consumeBag(TagBag bag, StateTag stateTag) { boolean shouldRemove; - if (stateTag.getRequestPosition() == null) { - // This is the response for the first page. - // Leave the future in the cache so subsequent requests for the first page - // can return immediately. - shouldRemove = false; - } else { - // This is a response for a subsequent page. - // Don't cache the future since we may need to make multiple requests with different - // continuation positions. - shouldRemove = true; - } + // This is the response for the first page. + // Leave the future in the cache so subsequent requests for the first page + // can return immediately. + // This is a response for a subsequent page. + // Don't cache the future since we may need to make multiple requests with different + // continuation positions. + shouldRemove = stateTag.getRequestPosition() != null; CoderAndFuture> coderAndFuture = getWaiting(stateTag, shouldRemove); SettableFuture> future = coderAndFuture.getNonDoneFuture(stateTag); try { - Coder coder = coderAndFuture.getAndClearCoder(); + Coder coder = coderAndFuture.getAndClearCoder(); List values = this.bagPageValues(bag, coder); future.set( new ValuesAndContPosition<>( @@ -1082,18 +812,14 @@ private void consumeTagValue(TagValue tagValue, StateTag stateTag) { private void consumeTagPrefixResponse( Windmill.TagValuePrefixResponse tagValuePrefixResponse, StateTag stateTag) { boolean shouldRemove; - if (stateTag.getRequestPosition() == null) { - // This is the response for the first page. - // Leave the future in the cache so subsequent - // requests for the first page - // can return immediately. - shouldRemove = false; - } else { - // This is a response for a subsequent page. - // Don't cache the future since we may need to make multiple requests with different - // continuation positions. - shouldRemove = true; - } + // This is the response for the first page. + // Leave the future in the cache so subsequent + // requests for the first page + // can return immediately. + // This is a response for a subsequent page. + // Don't cache the future since we may need to make multiple requests with different + // continuation positions. + shouldRemove = stateTag.getRequestPosition() != null; CoderAndFuture, ByteString>> coderAndFuture = getWaiting(stateTag, shouldRemove); @@ -1117,17 +843,13 @@ private void consumeTagPrefixResponse( private void consumeSortedList( Windmill.TagSortedListFetchResponse sortedListFetchResponse, StateTag stateTag) { boolean shouldRemove; - if (stateTag.getRequestPosition() == null) { - // This is the response for the first page.// Leave the future in the cache so subsequent - // requests for the first page - // can return immediately. - shouldRemove = false; - } else { - // This is a response for a subsequent page. - // Don't cache the future since we may need to make multiple requests with different - // continuation positions. - shouldRemove = true; - } + // This is the response for the first page.// Leave the future in the cache so subsequent + // requests for the first page + // can return immediately. + // This is a response for a subsequent page. + // Don't cache the future since we may need to make multiple requests with different + // continuation positions. + shouldRemove = stateTag.getRequestPosition() != null; CoderAndFuture, ByteString>> coderAndFuture = getWaiting(stateTag, shouldRemove); @@ -1187,102 +909,42 @@ private void consumeMultimapSingleEntry( } } - /** - * An iterable over elements backed by paginated GetData requests to Windmill. The iterable may be - * iterated over an arbitrary number of times and multiple iterators may be active simultaneously. - * - *

There are two pattern we wish to support with low -memory and -latency: - * - *

    - *
  1. Re-iterate over the initial elements multiple times (eg Iterables.first). We'll cache the - * initial 'page' of values returned by Windmill from our first request for the lifetime of - * the iterable. - *
  2. Iterate through all elements of a very large collection. We'll send the GetData request - * for the next page when the current page is begun. We'll discard intermediate pages and - * only retain the first. Thus the maximum memory pressure is one page plus one page per - * call to iterator. - *
- */ - private static class PagingIterable implements Iterable { - /** - * The reader we will use for scheduling continuation pages. - * - *

NOTE We've made this explicit to remind us to be careful not to cache the iterable. - */ - private final WindmillStateReader reader; - - /** Initial values returned for the first page. Never reclaimed. */ - private final List firstPage; - - /** State tag with continuation position set for second page. */ - private final StateTag secondPagePos; - - /** Coder for elements. */ - private final Coder coder; - - private PagingIterable( - WindmillStateReader reader, - List firstPage, - StateTag secondPagePos, - Coder coder) { - this.reader = reader; - this.firstPage = firstPage; - this.secondPagePos = secondPagePos; + private static final class CoderAndFuture { + private final SettableFuture future; + private Coder coder = null; + + private CoderAndFuture(Coder coder, SettableFuture future) { this.coder = coder; + this.future = future; } - @Override - public Iterator iterator() { - return new AbstractIterator() { - private Iterator currentPage = firstPage.iterator(); - private StateTag nextPagePos = secondPagePos; - private Future> pendingNextPage = - // NOTE: The results of continuation page reads are never cached. - reader.continuationFuture(nextPagePos, coder); - - @Override - protected ResultT computeNext() { - while (true) { - if (currentPage.hasNext()) { - return currentPage.next(); - } - if (pendingNextPage == null) { - return endOfData(); - } + private SettableFuture getFuture() { + return future; + } - ValuesAndContPosition valuesAndContPosition; - try { - valuesAndContPosition = pendingNextPage.get(); - } catch (InterruptedException | ExecutionException e) { - if (e instanceof InterruptedException) { - Thread.currentThread().interrupt(); - } - throw new RuntimeException("Unable to read value from state", e); - } - currentPage = valuesAndContPosition.values.iterator(); - StateTag.Builder nextPageBuilder = - StateTag.of( - nextPagePos.getKind(), - nextPagePos.getTag(), - nextPagePos.getStateFamily(), - valuesAndContPosition.continuationPosition) - .toBuilder(); - if (secondPagePos.getSortedListRange() != null) { - nextPageBuilder.setSortedListRange(secondPagePos.getSortedListRange()); - } - if (secondPagePos.getOmitValues() != null) { - nextPageBuilder.setOmitValues(secondPagePos.getOmitValues()); - } - if (secondPagePos.getMultimapKey() != null) { - nextPageBuilder.setMultimapKey(secondPagePos.getMultimapKey()); - } - nextPagePos = nextPageBuilder.build(); - pendingNextPage = - // NOTE: The results of continuation page reads are never cached. - reader.continuationFuture(nextPagePos, coder); - } - } - }; + private SettableFuture getNonDoneFuture(StateTag stateTag) { + if (future.isDone()) { + throw new IllegalStateException("Future for " + stateTag + " is already done"); + } + return future; + } + + private Coder getAndClearCoder() { + if (coder == null) { + throw new IllegalStateException("Coder has already been cleared from cache"); + } + Coder result = (Coder) coder; + if (result == null) { + throw new IllegalStateException("Coder has already been cleared from cache"); + } + coder = null; + return result; + } + + private void checkNoCoder() { + if (coder != null) { + throw new IllegalStateException("Unexpected coder"); + } } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateUtil.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateUtil.java new file mode 100644 index 000000000000..3cac5c3c5724 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateUtil.java @@ -0,0 +1,51 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.nio.charset.StandardCharsets; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; + +class WindmillStateUtil { + /** Encodes the given namespace and address as {@code <namespace>+<address>}. */ + @VisibleForTesting + static ByteString encodeKey(StateNamespace namespace, StateTag address) { + try { + // Use ByteStringOutputStream rather than concatenation and String.format. We build these keys + // a lot, and this leads to better performance results. See associated benchmarks. + ByteStringOutputStream stream = new ByteStringOutputStream(); + OutputStreamWriter writer = new OutputStreamWriter(stream, StandardCharsets.UTF_8); + + // stringKey starts and ends with a slash. We separate it from the + // StateTag ID by a '+' (which is guaranteed not to be in the stringKey) because the + // ID comes from the user. + namespace.appendTo(writer); + writer.write('+'); + address.appendTo(writer); + writer.flush(); + return stream.toByteString(); + } catch (IOException e) { + throw new RuntimeException(e); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillValue.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillValue.java new file mode 100644 index 000000000000..1ea6e56435d2 --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillValue.java @@ -0,0 +1,160 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import static org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateUtil.encodeKey; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.coders.Coder; +import org.apache.beam.sdk.state.ValueState; +import org.apache.beam.sdk.util.ByteStringOutputStream; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; + +@SuppressWarnings({ + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WindmillValue extends SimpleWindmillState implements ValueState { + private final StateNamespace namespace; + private final StateTag> address; + private final ByteString stateKey; + private final String stateFamily; + private final Coder coder; + + /** Whether we've modified the value since creation of this state. */ + private boolean modified = false; + /** Whether the in memory value is the true value. */ + private boolean valueIsKnown = false; + /** The size of the encoded value */ + private long cachedSize = -1; + + private T value; + + WindmillValue( + StateNamespace namespace, + StateTag> address, + String stateFamily, + Coder coder, + boolean isNewKey) { + this.namespace = namespace; + this.address = address; + this.stateKey = encodeKey(namespace, address); + this.stateFamily = stateFamily; + this.coder = coder; + if (isNewKey) { + this.valueIsKnown = true; + this.value = null; + } + } + + @Override + public void clear() { + modified = true; + valueIsKnown = true; + value = null; + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public WindmillValue readLater() { + getFuture(); + return this; + } + + @Override + public T read() { + try (Closeable scope = scopedReadState()) { + if (!valueIsKnown) { + cachedSize = -1; + } + value = getFuture().get(); + valueIsKnown = true; + return value; + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read value from state", e); + } + } + + @Override + public void write(T value) { + modified = true; + valueIsKnown = true; + cachedSize = -1; + this.value = value; + } + + @Override + protected Windmill.WorkItemCommitRequest persistDirectly(WindmillStateCache.ForKeyAndFamily cache) + throws IOException { + if (!valueIsKnown) { + // The value was never read, written or cleared. + // Thus nothing to update in Windmill. + // And no need to add to global cache. + return Windmill.WorkItemCommitRequest.newBuilder().buildPartial(); + } + + ByteString encoded = null; + if (cachedSize == -1 || modified) { + ByteStringOutputStream stream = new ByteStringOutputStream(); + if (value != null) { + coder.encode(value, stream, Coder.Context.OUTER); + } + encoded = stream.toByteString(); + cachedSize = encoded.size(); + } + + // Place in cache to avoid a future read. + cache.put(namespace, address, this, cachedSize); + + if (!modified) { + // The value was read, but never written or cleared. + // But nothing to update in Windmill. + return Windmill.WorkItemCommitRequest.newBuilder().buildPartial(); + } + + // The value was written or cleared. Commit that change to Windmill. + modified = false; + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + commitBuilder + .addValueUpdatesBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .getValueBuilder() + .setData(encoded) + .setTimestamp(Long.MAX_VALUE); + return commitBuilder.buildPartial(); + } + + private Future getFuture() { + // WindmillStateReader guarantees that we can ask for a future for a particular tag multiple + // times and it will efficiently be reused. + return valueIsKnown + ? Futures.immediateFuture(value) + : reader.valueFuture(stateKey, stateFamily, coder); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillWatermarkHold.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillWatermarkHold.java new file mode 100644 index 000000000000..a800c2eb6dad --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillWatermarkHold.java @@ -0,0 +1,259 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import static org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateUtil.encodeKey; + +import java.io.Closeable; +import java.io.IOException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import org.apache.beam.runners.core.StateNamespace; +import org.apache.beam.runners.core.StateTag; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; +import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.sdk.state.ReadableState; +import org.apache.beam.sdk.state.WatermarkHoldState; +import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; +import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Futures; +import org.joda.time.Instant; + +@SuppressWarnings({ + "rawtypes", // TODO(https://github.com/apache/beam/issues/20447) + "nullness" // TODO(https://github.com/apache/beam/issues/20497) +}) +public class WindmillWatermarkHold extends WindmillState implements WatermarkHoldState { + // The encoded size of an Instant. + private static final int ENCODED_SIZE = 8; + + private final TimestampCombiner timestampCombiner; + private final StateNamespace namespace; + private final StateTag address; + private final ByteString stateKey; + private final String stateFamily; + + private boolean cleared = false; + /** + * If non-{@literal null}, the known current hold value, or absent if we know there are no output + * watermark holds. If {@literal null}, the current hold value could depend on holds in Windmill + * we do not yet know. + */ + private Optional cachedValue = null; + + private Instant localAdditions = null; + + WindmillWatermarkHold( + StateNamespace namespace, + StateTag address, + String stateFamily, + TimestampCombiner timestampCombiner, + boolean isNewKey) { + this.namespace = namespace; + this.address = address; + this.stateKey = encodeKey(namespace, address); + this.stateFamily = stateFamily; + this.timestampCombiner = timestampCombiner; + if (isNewKey) { + cachedValue = Optional.absent(); + } + } + + @Override + public void clear() { + cleared = true; + cachedValue = Optional.absent(); + localAdditions = null; + } + + @Override + @SuppressWarnings("FutureReturnValueIgnored") + public WindmillWatermarkHold readLater() { + getFuture(); + return this; + } + + @Override + public Instant read() { + try (Closeable scope = scopedReadState()) { + Instant persistedHold = getFuture().get(); + if (persistedHold == null) { + cachedValue = Optional.absent(); + } else { + cachedValue = Optional.of(persistedHold); + } + } catch (InterruptedException | ExecutionException | IOException e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + throw new RuntimeException("Unable to read state", e); + } + + if (localAdditions == null) { + return cachedValue.orNull(); + } else if (!cachedValue.isPresent()) { + return localAdditions; + } else { + return timestampCombiner.combine(localAdditions, cachedValue.get()); + } + } + + @Override + public ReadableState isEmpty() { + throw new UnsupportedOperationException(); + } + + @Override + public void add(Instant outputTime) { + localAdditions = + (localAdditions == null) + ? outputTime + : timestampCombiner.combine(outputTime, localAdditions); + } + + @Override + public TimestampCombiner getTimestampCombiner() { + return timestampCombiner; + } + + @Override + public Future persist( + final WindmillStateCache.ForKeyAndFamily cache) { + + Future result; + + if (!cleared && localAdditions == null) { + // No changes, so no need to update Windmill and no need to cache any value. + return Futures.immediateFuture(Windmill.WorkItemCommitRequest.newBuilder().buildPartial()); + } + + if (cleared && localAdditions == null) { + // Just clearing the persisted state; blind delete + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + commitBuilder + .addWatermarkHoldsBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .setReset(true); + + result = Futures.immediateFuture(commitBuilder.buildPartial()); + } else if (cleared && localAdditions != null) { + // Since we cleared before adding, we can do a blind overwrite of persisted state + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + commitBuilder + .addWatermarkHoldsBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .setReset(true) + .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); + + cachedValue = Optional.of(localAdditions); + + result = Futures.immediateFuture(commitBuilder.buildPartial()); + } else if (!cleared && localAdditions != null) { + // Otherwise, we need to combine the local additions with the already persisted data + result = combineWithPersisted(); + } else { + throw new IllegalStateException("Unreachable condition"); + } + + return Futures.lazyTransform( + result, + result1 -> { + cleared = false; + localAdditions = null; + if (cachedValue != null) { + cache.put(namespace, address, WindmillWatermarkHold.this, ENCODED_SIZE); + } + return result1; + }); + } + + private Future getFuture() { + return cachedValue != null + ? Futures.immediateFuture(cachedValue.orNull()) + : reader.watermarkFuture(stateKey, stateFamily); + } + + /** + * Combines local additions with persisted data and mutates the {@code commitBuilder} to write the + * result. + */ + private Future combineWithPersisted() { + boolean windmillCanCombine = false; + + // If the combined output time depends only on the window, then we are just blindly adding + // the same value that may or may not already be present. This depends on the state only being + // used for one window. + windmillCanCombine |= timestampCombiner.dependsOnlyOnWindow(); + + // If the combined output time depends only on the earliest input timestamp, then because + // assignOutputTime is monotonic, the hold only depends on the earliest output timestamp + // (which is the value submitted as a watermark hold). The only way holds for later inputs + // can be redundant is if the are later (or equal) to the earliest. So taking the MIN + // implicitly, as Windmill does, has the desired behavior. + windmillCanCombine |= timestampCombiner.dependsOnlyOnEarliestTimestamp(); + + if (windmillCanCombine) { + // We do a blind write and let Windmill take the MIN + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + commitBuilder + .addWatermarkHoldsBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(localAdditions)); + + if (cachedValue != null) { + cachedValue = + Optional.of( + cachedValue.isPresent() + ? timestampCombiner.combine(cachedValue.get(), localAdditions) + : localAdditions); + } + + return Futures.immediateFuture(commitBuilder.buildPartial()); + } else { + // The non-fast path does a read-modify-write + return Futures.lazyTransform( + (cachedValue != null) + ? Futures.immediateFuture(cachedValue.orNull()) + : reader.watermarkFuture(stateKey, stateFamily), + priorHold -> { + cachedValue = + Optional.of( + (priorHold != null) + ? timestampCombiner.combine(priorHold, localAdditions) + : localAdditions); + Windmill.WorkItemCommitRequest.Builder commitBuilder = + Windmill.WorkItemCommitRequest.newBuilder(); + commitBuilder + .addWatermarkHoldsBuilder() + .setTag(stateKey) + .setStateFamily(stateFamily) + .setReset(true) + .addTimestamps(WindmillTimeUtils.harnessToWindmillTimestamp(cachedValue.get())); + + return commitBuilder.buildPartial(); + }); + } + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WrappedFuture.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WrappedFuture.java new file mode 100644 index 000000000000..035f6ec8e93d --- /dev/null +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/state/WrappedFuture.java @@ -0,0 +1,64 @@ +/* + * 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.runners.dataflow.worker.windmill.state; + +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; +import javax.annotation.Nullable; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ForwardingFuture; + +/** + * A future which will trigger a GetData request to Windmill for all outstanding futures on the + * first {@link #get}. + */ +public class WrappedFuture extends ForwardingFuture.SimpleForwardingFuture { + /** + * The reader we'll use to service the eventual read. Null if read has been fulfilled. + * + *

NOTE: We must clear this after the read is fulfilled to prevent space leaks. + */ + private @Nullable WindmillStateReader reader; + + public WrappedFuture(WindmillStateReader reader, Future delegate) { + super(delegate); + this.reader = reader; + } + + @Override + public T get() throws InterruptedException, ExecutionException { + if (!delegate().isDone() && reader != null) { + // Only one thread per reader, so no race here. + reader.startBatchAndBlock(); + } + reader = null; + return super.get(); + } + + @Override + public T get(long timeout, TimeUnit unit) + throws InterruptedException, ExecutionException, TimeoutException { + if (!delegate().isDone() && reader != null) { + // Only one thread per reader, so no race here. + reader.startBatchAndBlock(); + } + reader = null; + return super.get(timeout, unit); + } +} diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java index 57e29bb21dee..6620dbdaab79 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingModeExecutionContextTest.java @@ -56,6 +56,8 @@ import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.NoopProfileScope; import org.apache.beam.runners.dataflow.worker.profiler.ScopedProfiler.ProfileScope; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateReader; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.metrics.MetricsContainer; import org.apache.beam.sdk.options.PipelineOptionsFactory; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java index 1f3dee4b76ba..12ae816de829 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/streaming/ActiveWorkStateTest.java @@ -33,10 +33,10 @@ import java.util.Map; import java.util.Optional; import javax.annotation.Nullable; -import org.apache.beam.runners.dataflow.worker.WindmillStateCache; import org.apache.beam.runners.dataflow.worker.streaming.ActiveWorkState.ActivateWorkResult; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; +import org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache; import org.apache.beam.vendor.grpc.v1p54p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateCacheTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java similarity index 75% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateCacheTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java index eca431af11a7..cc6633f1b704 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateCacheTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateCacheTest.java @@ -15,17 +15,18 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker; +package org.apache.beam.runners.dataflow.worker.windmill.state; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertNull; import java.io.IOException; import java.util.Objects; +import java.util.Optional; import org.apache.beam.runners.core.StateNamespace; import org.apache.beam.runners.core.StateNamespaces; import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; +import org.apache.beam.runners.dataflow.worker.WindmillComputationKey; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.state.State; import org.apache.beam.sdk.state.StateSpec; @@ -38,7 +39,7 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -/** Tests for {@link WindmillStateCache}. */ +/** Tests for {@link org.apache.beam.runners.dataflow.worker.windmill.state.WindmillStateCache}. */ @RunWith(JUnit4.class) public class WindmillStateCacheTest { @@ -153,10 +154,11 @@ public void setUp() { public void testBasic() throws Exception { WindmillStateCache.ForKeyAndFamily keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 1L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); - assertNull(keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); - assertNull(keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); - assertNull(keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag2"))); + assertEquals( + Optional.empty(), keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); + assertEquals(Optional.empty(), keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); + assertEquals(Optional.empty(), keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag2"))); assertEquals(0, cache.getWeight()); keyCache.put(StateNamespaces.global(), new TestStateTag("tag1"), new TestState("g1"), 2); @@ -177,12 +179,17 @@ public void testBasic() throws Exception { keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 2L).forFamily(STATE_FAMILY); assertEquals( - new TestState("g1"), keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); - assertEquals(new TestState("w2"), keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); + Optional.of(new TestState("g1")), + keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertEquals( + Optional.of(new TestState("w2")), + keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); assertEquals( - new TestState("t3"), keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); + Optional.of(new TestState("t3")), + keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); assertEquals( - new TestState("t2"), keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag2"))); + Optional.of(new TestState("t2")), + keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag2"))); } /** Verifies that max weight is set */ @@ -196,7 +203,8 @@ public void testMaxWeight() throws Exception { public void testInvalidation() throws Exception { WindmillStateCache.ForKeyAndFamily keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 1L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertEquals( + Optional.empty(), keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); keyCache.put(StateNamespaces.global(), new TestStateTag("tag1"), new TestState("g1"), 2); keyCache.persist(); @@ -204,11 +212,13 @@ public void testInvalidation() throws Exception { cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 2L).forFamily(STATE_FAMILY); assertEquals(127, cache.getWeight()); assertEquals( - new TestState("g1"), keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + Optional.of(new TestState("g1")), + keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 1L, 3L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertEquals( + Optional.empty(), keyCache.get(StateNamespaces.global(), new TestStateTag("tag1"))); assertEquals(127, cache.getWeight()); } @@ -225,8 +235,8 @@ public void testEviction() throws Exception { // Eviction is atomic across the whole window. keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 2L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); - assertNull(keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), new TestStateTag("tag2"))); + assertEquals(Optional.empty(), keyCache.get(triggerNamespace(0, 0), new TestStateTag("tag3"))); } /** Verifies that the cache does not vend for stale work tokens. */ @@ -239,35 +249,35 @@ public void testStaleWorkItem() throws Exception { keyCache.put(windowNamespace(0), tag, new TestState("w2"), 2); // Same cache. - assertEquals(new TestState("w2"), keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.of(new TestState("w2")), keyCache.get(windowNamespace(0), tag)); assertEquals(0, cache.getWeight()); keyCache.persist(); assertEquals(127, cache.getWeight()); - assertEquals(new TestState("w2"), keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.of(new TestState("w2")), keyCache.get(windowNamespace(0), tag)); // Previous work token. keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 1L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), tag)); // Retry of work token that inserted. keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 2L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), tag)); keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 10L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), tag)); keyCache.put(windowNamespace(0), tag, new TestState("w3"), 2); // Ensure that second put updated work token. keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 5L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), tag)); keyCache = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 15L).forFamily(STATE_FAMILY); - assertNull(keyCache.get(windowNamespace(0), tag)); + assertEquals(Optional.empty(), keyCache.get(windowNamespace(0), tag)); } /** Verifies that caches are kept independently per-key. */ @@ -293,7 +303,7 @@ public void testMultipleKeys() throws Exception { TestState state1 = new TestState("g1"); keyCache1.put(StateNamespaces.global(), tag, state1, 2); - assertEquals(state1, keyCache1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), keyCache1.get(StateNamespaces.global(), tag)); keyCache1.persist(); keyCache1 = @@ -301,22 +311,22 @@ public void testMultipleKeys() throws Exception { .forComputation("comp1") .forKey(computationKey("comp1", "key1", SHARDING_KEY), 0L, 1L) .forFamily(STATE_FAMILY); - assertEquals(state1, keyCache1.get(StateNamespaces.global(), tag)); - assertNull(keyCache2.get(StateNamespaces.global(), tag)); - assertNull(keyCache3.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), keyCache1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), keyCache2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), keyCache3.get(StateNamespaces.global(), tag)); TestState state2 = new TestState("g2"); keyCache2.put(StateNamespaces.global(), tag, state2, 2); keyCache2.persist(); - assertEquals(state2, keyCache2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state2), keyCache2.get(StateNamespaces.global(), tag)); keyCache2 = cache .forComputation("comp1") .forKey(computationKey("comp1", "key2", SHARDING_KEY), 0L, 20L) .forFamily(STATE_FAMILY); - assertEquals(state2, keyCache2.get(StateNamespaces.global(), tag)); - assertEquals(state1, keyCache1.get(StateNamespaces.global(), tag)); - assertNull(keyCache3.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state2), keyCache2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), keyCache1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), keyCache3.get(StateNamespaces.global(), tag)); } /** Verifies that caches are kept independently per shard of key. */ @@ -343,28 +353,28 @@ public void testMultipleShardsOfKey() throws Exception { TestState state1 = new TestState("g1"); key1CacheShard1.put(StateNamespaces.global(), tag, state1, 2); key1CacheShard1.persist(); - assertEquals(state1, key1CacheShard1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), key1CacheShard1.get(StateNamespaces.global(), tag)); key1CacheShard1 = cache .forComputation(COMPUTATION) .forKey(computationKey(COMPUTATION, "key1", 1), 0L, 1L) .forFamily(STATE_FAMILY); - assertEquals(state1, key1CacheShard1.get(StateNamespaces.global(), tag)); - assertNull(key1CacheShard2.get(StateNamespaces.global(), tag)); - assertNull(key2CacheShard1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), key1CacheShard1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), key1CacheShard2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), key2CacheShard1.get(StateNamespaces.global(), tag)); TestState state2 = new TestState("g2"); key1CacheShard2.put(StateNamespaces.global(), tag, state2, 2); - assertEquals(state2, key1CacheShard2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state2), key1CacheShard2.get(StateNamespaces.global(), tag)); key1CacheShard2.persist(); key1CacheShard2 = cache .forComputation(COMPUTATION) .forKey(computationKey(COMPUTATION, "key1", 2), 0L, 20L) .forFamily(STATE_FAMILY); - assertEquals(state2, key1CacheShard2.get(StateNamespaces.global(), tag)); - assertEquals(state1, key1CacheShard1.get(StateNamespaces.global(), tag)); - assertNull(key2CacheShard1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state2), key1CacheShard2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), key1CacheShard1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), key2CacheShard1.get(StateNamespaces.global(), tag)); } /** Verifies that caches are kept independently per-family. */ @@ -379,22 +389,22 @@ public void testMultipleFamilies() throws Exception { TestState state1 = new TestState("g1"); family1.put(StateNamespaces.global(), tag, state1, 2); - assertEquals(state1, family1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), family1.get(StateNamespaces.global(), tag)); family1.persist(); TestState state2 = new TestState("g2"); family2.put(StateNamespaces.global(), tag, state2, 2); family2.persist(); - assertEquals(state2, family2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state2), family2.get(StateNamespaces.global(), tag)); keyCache = cache.forComputation("comp1").forKey(computationKey("comp1", "key1", SHARDING_KEY), 0L, 1L); family1 = keyCache.forFamily("family1"); family2 = keyCache.forFamily("family2"); WindmillStateCache.ForKeyAndFamily family3 = keyCache.forFamily("family3"); - assertEquals(state1, family1.get(StateNamespaces.global(), tag)); - assertEquals(state2, family2.get(StateNamespaces.global(), tag)); - assertNull(family3.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state1), family1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(state2), family2.get(StateNamespaces.global(), tag)); + assertEquals(Optional.empty(), family3.get(StateNamespaces.global(), tag)); } /** Verifies explicit invalidation does indeed invalidate the correct entries. */ @@ -450,13 +460,17 @@ public void testExplicitInvalidation() throws Exception { .forKey(computationKey("comp1", "key1", 2), 0L, 1L) .forFamily(STATE_FAMILY); assertEquals( - new TestState("g1"), keyCache1.get(StateNamespaces.global(), new TestStateTag("tag1"))); + Optional.of(new TestState("g1")), + keyCache1.get(StateNamespaces.global(), new TestStateTag("tag1"))); assertEquals( - new TestState("g2"), keyCache2.get(StateNamespaces.global(), new TestStateTag("tag2"))); + Optional.of(new TestState("g2")), + keyCache2.get(StateNamespaces.global(), new TestStateTag("tag2"))); assertEquals( - new TestState("g3"), keyCache3.get(StateNamespaces.global(), new TestStateTag("tag3"))); + Optional.of(new TestState("g3")), + keyCache3.get(StateNamespaces.global(), new TestStateTag("tag3"))); assertEquals( - new TestState("g4"), keyCache4.get(StateNamespaces.global(), new TestStateTag("tag4"))); + Optional.of(new TestState("g4")), + keyCache4.get(StateNamespaces.global(), new TestStateTag("tag4"))); // Invalidation of key 1 shard 1 does not affect another shard of key 1 or other keys. cache.forComputation("comp1").invalidate(ByteString.copyFromUtf8("key1"), 1); @@ -466,23 +480,30 @@ public void testExplicitInvalidation() throws Exception { .forKey(computationKey("comp1", "key1", 1), 0L, 2L) .forFamily(STATE_FAMILY); - assertNull(keyCache1.get(StateNamespaces.global(), new TestStateTag("tag1"))); assertEquals( - new TestState("g2"), keyCache2.get(StateNamespaces.global(), new TestStateTag("tag2"))); + Optional.empty(), keyCache1.get(StateNamespaces.global(), new TestStateTag("tag1"))); + assertEquals( + Optional.of(new TestState("g2")), + keyCache2.get(StateNamespaces.global(), new TestStateTag("tag2"))); assertEquals( - new TestState("g3"), keyCache3.get(StateNamespaces.global(), new TestStateTag("tag3"))); + Optional.of(new TestState("g3")), + keyCache3.get(StateNamespaces.global(), new TestStateTag("tag3"))); assertEquals( - new TestState("g4"), keyCache4.get(StateNamespaces.global(), new TestStateTag("tag4"))); + Optional.of(new TestState("g4")), + keyCache4.get(StateNamespaces.global(), new TestStateTag("tag4"))); // Invalidation of an non-existing key affects nothing. cache.forComputation("comp1").invalidate(ByteString.copyFromUtf8("key1"), 3); assertEquals( - new TestState("g2"), keyCache2.get(StateNamespaces.global(), new TestStateTag("tag2"))); + Optional.of(new TestState("g2")), + keyCache2.get(StateNamespaces.global(), new TestStateTag("tag2"))); assertEquals( - new TestState("g3"), keyCache3.get(StateNamespaces.global(), new TestStateTag("tag3"))); + Optional.of(new TestState("g3")), + keyCache3.get(StateNamespaces.global(), new TestStateTag("tag3"))); assertEquals( - new TestState("g4"), keyCache4.get(StateNamespaces.global(), new TestStateTag("tag4"))); + Optional.of(new TestState("g4")), + keyCache4.get(StateNamespaces.global(), new TestStateTag("tag4"))); } private static class TestStateTagWithBadEquality extends TestStateTag { @@ -517,9 +538,9 @@ public void testBadCoderEquality() throws Exception { keyCache1 = cache.forComputation(COMPUTATION).forKey(COMPUTATION_KEY, 0L, 1L).forFamily(STATE_FAMILY); - assertEquals(new TestState("g1"), keyCache1.get(StateNamespaces.global(), tag)); + assertEquals(Optional.of(new TestState("g1")), keyCache1.get(StateNamespaces.global(), tag)); assertEquals( - new TestState("g1"), + Optional.of(new TestState("g1")), keyCache1.get(StateNamespaces.global(), new TestStateTagWithBadEquality("tag1"))); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java similarity index 97% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java index 9f2d5eee8f87..8971c39ccaa1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateInternalsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateInternalsTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker; +package org.apache.beam.runners.dataflow.worker.windmill.state; import static org.apache.beam.runners.dataflow.worker.DataflowMatchers.ByteStringMatcher.byteStringEq; import static org.apache.beam.sdk.testing.SystemNanoTimeSleeper.sleepMillis; @@ -56,8 +56,8 @@ import org.apache.beam.runners.core.StateTag; import org.apache.beam.runners.core.StateTags; import org.apache.beam.runners.dataflow.options.DataflowWorkerHarnessOptions; -import org.apache.beam.runners.dataflow.worker.WindmillStateInternals.IdTracker; -import org.apache.beam.runners.dataflow.worker.WindmillStateInternals.WindmillOrderedList; +import org.apache.beam.runners.dataflow.worker.WindmillComputationKey; +import org.apache.beam.runners.dataflow.worker.WindmillStateTestUtils; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagBag; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.TagSortedListUpdateRequest; @@ -113,24 +113,21 @@ }) public class WindmillStateInternalsTest { + public static final Range FULL_ORDERED_LIST_RANGE = + Range.closedOpen(WindmillOrderedList.MIN_TS_MICROS, WindmillOrderedList.MAX_TS_MICROS); private static final StateNamespace NAMESPACE = new StateNamespaceForTest("ns"); private static final String STATE_FAMILY = "family"; - private static final StateTag> COMBINING_ADDR = StateTags.combiningValueFromInputInternal("combining", VarIntCoder.of(), Sum.ofIntegers()); private static final ByteString COMBINING_KEY = key(NAMESPACE, "combining"); private final Coder accumCoder = Sum.ofIntegers().getAccumulatorCoder(null, VarIntCoder.of()); - private long workToken = 0; - DataflowWorkerHarnessOptions options; - + private long workToken = 0; @Mock private WindmillStateReader mockReader; - private WindmillStateInternals underTest; private WindmillStateInternals underTestNewKey; private WindmillStateCache cache; - @Mock private Supplier readStateSupplier; private static ByteString key(StateNamespace namespace, String addrId) { @@ -141,6 +138,67 @@ private static ByteString systemKey(StateNamespace namespace, String addrId) { return ByteString.copyFromUtf8(namespace.stringKey() + "+s" + addrId); } + private static ByteString encodeWithCoder(T key, Coder coder) { + ByteStringOutputStream out = new ByteStringOutputStream(); + try { + coder.encode(key, out, Context.OUTER); + } catch (IOException e) { + throw new RuntimeException(e); + } + return out.toByteString(); + } + + // We use the structural value of the Multimap keys to differentiate between different keys. So we + // mix using the original key object and a duplicate but same key object so make sure the + // correctness. + private static byte[] dup(byte[] key) { + byte[] res = new byte[key.length]; + System.arraycopy(key, 0, res, 0, key.length); + return res; + } + + private static Map.Entry> multimapEntry( + byte[] key, Integer... values) { + return new AbstractMap.SimpleEntry<>( + encodeWithCoder(key, ByteArrayCoder.of()), Arrays.asList(values)); + } + + @SafeVarargs + private static List weightedList(T... entries) { + WeightedList list = new WeightedList<>(new ArrayList<>()); + for (T entry : entries) { + list.addWeighted(entry, 1); + } + return list; + } + + private static CombinableMatcher multimapEntryMatcher(byte[] key, Integer value) { + return Matchers.both(Matchers.hasProperty("key", Matchers.equalTo(key))) + .and(Matchers.hasProperty("value", Matchers.equalTo(value))); + } + + private static MultimapEntryUpdate decodeTagMultimapEntry(Windmill.TagMultimapEntry entryProto) { + try { + String key = StringUtf8Coder.of().decode(entryProto.getEntryName().newInput(), Context.OUTER); + List values = new ArrayList<>(); + for (ByteString value : entryProto.getValuesList()) { + values.add(VarIntCoder.of().decode(value.newInput(), Context.OUTER)); + } + return new MultimapEntryUpdate(key, values, entryProto.getDeleteAll()); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + private static void assertTagMultimapUpdates( + Windmill.TagMultimapUpdateRequest.Builder updates, MultimapEntryUpdate... expected) { + assertThat( + updates.getUpdatesList().stream() + .map(WindmillStateInternalsTest::decodeTagMultimapEntry) + .collect(Collectors.toList()), + Matchers.containsInAnyOrder(expected)); + } + @Before public void setUp() { MockitoAnnotations.initMocks(this); @@ -203,9 +261,8 @@ private void waitAndSet(final SettableFuture future, final T value, final .run(); } - private WindmillStateReader.WeightedList weightedList(String... elems) { - WindmillStateReader.WeightedList result = - new WindmillStateReader.WeightedList<>(new ArrayList(elems.length)); + private WeightedList weightedList(String... elems) { + WeightedList result = new WeightedList<>(new ArrayList(elems.length)); for (String elem : elems) { result.addWeighted(elem, elem.length()); } @@ -660,25 +717,6 @@ public void testMapComplexPersist() throws Exception { assertEquals(0, commitBuilder.getValueUpdatesCount()); } - private static ByteString encodeWithCoder(T key, Coder coder) { - ByteStringOutputStream out = new ByteStringOutputStream(); - try { - coder.encode(key, out, Context.OUTER); - } catch (IOException e) { - throw new RuntimeException(e); - } - return out.toByteString(); - } - - // We use the structural value of the Multimap keys to differentiate between different keys. So we - // mix using the original key object and a duplicate but same key object so make sure the - // correctness. - private static byte[] dup(byte[] key) { - byte[] res = new byte[key.length]; - System.arraycopy(key, 0, res, 0, key.length); - return res; - } - @Test public void testMultimapGet() { final String tag = "multimap"; @@ -796,7 +834,7 @@ public void testMultimapRemovePersistPut() { underTest.persist(commitBuilder); assertTagMultimapUpdates( Iterables.getOnlyElement(commitBuilder.getMultimapUpdatesBuilderList()), - new MultimapEntryUpdate(key, Arrays.asList(4), true)); + new MultimapEntryUpdate(key, Collections.singletonList(4), true)); multimapState.put(key, 5); assertThat(multimapState.get(key).read(), Matchers.containsInAnyOrder(4, 5)); @@ -887,22 +925,6 @@ public void testMultimapLocalClearOverrideStorage() { assertTrue(multimapState.isEmpty().read()); } - private static Map.Entry> multimapEntry( - byte[] key, Integer... values) { - return new AbstractMap.SimpleEntry<>( - encodeWithCoder(key, ByteArrayCoder.of()), Arrays.asList(values)); - } - - @SafeVarargs - private static List weightedList(T... entries) { - WindmillStateReader.WeightedList list = - new WindmillStateReader.WeightedList<>(new ArrayList<>()); - for (T entry : entries) { - list.addWeighted(entry, 1); - } - return list; - } - @Test public void testMultimapBasicEntriesAndKeys() { final String tag = "multimap"; @@ -950,11 +972,6 @@ true, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) assertThat(keys, Matchers.containsInAnyOrder(key1, key2)); } - private static CombinableMatcher multimapEntryMatcher(byte[] key, Integer value) { - return Matchers.both(Matchers.hasProperty("key", Matchers.equalTo(key))) - .and(Matchers.hasProperty("value", Matchers.equalTo(value))); - } - @Test public void testMultimapEntriesAndKeysMergeLocalAdd() { final String tag = "multimap"; @@ -1389,10 +1406,10 @@ false, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) entriesFuture, () -> new Iterator>>() { - int returnedEntries = 0; - byte[] entryKey = new byte[10_000]; // each key is 10KB final int targetEntries = 1_000_000; // return 1 million entries, which is 10 GBs - Random rand = new Random(); + final byte[] entryKey = new byte[10_000]; // each key is 10KB + final Random rand = new Random(); + int returnedEntries = 0; @Override public boolean hasNext() { @@ -1429,10 +1446,10 @@ true, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) keysFuture, () -> new Iterator>>() { - int returnedEntries = 0; - byte[] entryKey = new byte[10_000]; // each key is 10KB final int targetEntries = 1_000_000; // return 1 million entries, which is 10 GBs - Random rand = new Random(); + final byte[] entryKey = new byte[10_000]; // each key is 10KB + final Random rand = new Random(); + int returnedEntries = 0; @Override public boolean hasNext() { @@ -1477,10 +1494,10 @@ false, key(NAMESPACE, tag), STATE_FAMILY, ByteArrayCoder.of())) Iterable values = () -> new Iterator() { - int returnedValues = 0; - byte[] value = new byte[10_000]; // each value is 10KB final int targetValues = 1_000_000; // return 1 million values, which is 10 GBs - Random rand = new Random(); + final byte[] value = new byte[10_000]; // each value is 10KB + final Random rand = new Random(); + int returnedValues = 0; @Override public boolean hasNext() { @@ -1497,8 +1514,8 @@ public byte[] next() { waitAndSet( entriesFuture, - Arrays.asList( - new AbstractMap.SimpleEntry<>(encodeWithCoder(key, VarIntCoder.of()), values)), + Collections.singletonList( + new SimpleEntry<>(encodeWithCoder(key, VarIntCoder.of()), values)), 200); waitAndSet(getKeyFuture, values, 200); @@ -1509,55 +1526,6 @@ public byte[] next() { assertEquals(1_000_000, Iterables.size(valueResult)); } - private static class MultimapEntryUpdate { - String key; - Iterable values; - boolean deleteAll; - - public MultimapEntryUpdate(String key, Iterable values, boolean deleteAll) { - this.key = key; - this.values = values; - this.deleteAll = deleteAll; - } - - @Override - public boolean equals(Object o) { - if (this == o) return true; - if (!(o instanceof MultimapEntryUpdate)) return false; - MultimapEntryUpdate that = (MultimapEntryUpdate) o; - return deleteAll == that.deleteAll - && Objects.equals(key, that.key) - && Objects.equals(values, that.values); - } - - @Override - public int hashCode() { - return Objects.hash(key, values, deleteAll); - } - } - - private static MultimapEntryUpdate decodeTagMultimapEntry(Windmill.TagMultimapEntry entryProto) { - try { - String key = StringUtf8Coder.of().decode(entryProto.getEntryName().newInput(), Context.OUTER); - List values = new ArrayList<>(); - for (ByteString value : entryProto.getValuesList()) { - values.add(VarIntCoder.of().decode(value.newInput(), Context.OUTER)); - } - return new MultimapEntryUpdate(key, values, entryProto.getDeleteAll()); - } catch (IOException e) { - throw new RuntimeException(e); - } - } - - private static void assertTagMultimapUpdates( - Windmill.TagMultimapUpdateRequest.Builder updates, MultimapEntryUpdate... expected) { - assertThat( - updates.getUpdatesList().stream() - .map(WindmillStateInternalsTest::decodeTagMultimapEntry) - .collect(Collectors.toList()), - Matchers.containsInAnyOrder(expected)); - } - @Test public void testMultimapPutAndPersist() { final String tag = "multimap"; @@ -1582,7 +1550,7 @@ public void testMultimapPutAndPersist() { assertTagMultimapUpdates( builder, new MultimapEntryUpdate(key1, Arrays.asList(1, 2), false), - new MultimapEntryUpdate(key2, Arrays.asList(2), false)); + new MultimapEntryUpdate(key2, Collections.singletonList(2), false)); } @Test @@ -1615,7 +1583,7 @@ public void testMultimapRemovePutAndPersist() { assertTagMultimapUpdates( builder, new MultimapEntryUpdate(key1, Arrays.asList(1, 2), true), - new MultimapEntryUpdate(key2, Arrays.asList(4), true)); + new MultimapEntryUpdate(key2, Collections.singletonList(4), true)); } @Test @@ -1709,7 +1677,8 @@ false, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) assertEquals(1, commitBuilder.getMultimapUpdatesCount()); Windmill.TagMultimapUpdateRequest.Builder builder = Iterables.getOnlyElement(commitBuilder.getMultimapUpdatesBuilderList()); - assertTagMultimapUpdates(builder, new MultimapEntryUpdate(key1, Arrays.asList(4), false)); + assertTagMultimapUpdates( + builder, new MultimapEntryUpdate(key1, Collections.singletonList(4), false)); } @Test @@ -1731,8 +1700,7 @@ true, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) ReadableState> keysResult = multimapState.keys().readLater(); waitAndSet( keysFuture, - new WindmillStateReader.WeightedList<>( - Arrays.asList(multimapEntry(key1), multimapEntry(key2))), + new WeightedList<>(Arrays.asList(multimapEntry(key1), multimapEntry(key2))), 30); multimapState.remove(key1); @@ -1753,7 +1721,7 @@ true, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) Windmill.TagMultimapEntry entryUpdate = Iterables.getOnlyElement(builder.getUpdatesList()); byte[] decodedKey = ByteArrayCoder.of().decode(entryUpdate.getEntryName().newInput(), Context.OUTER); - assertTrue(Arrays.equals(key1, decodedKey)); + assertArrayEquals(key1, decodedKey); assertTrue(entryUpdate.getDeleteAll()); } @@ -1870,9 +1838,6 @@ true, key(NAMESPACE, tag), STATE_FAMILY, VarIntCoder.of())) underTest.persist(commitBuilder); } - public static final Range FULL_ORDERED_LIST_RANGE = - Range.closedOpen(WindmillOrderedList.MIN_TS_MICROS, WindmillOrderedList.MAX_TS_MICROS); - @Test public void testOrderedListAddBeforeRead() throws Exception { StateTag> addr = @@ -1897,7 +1862,7 @@ public void testOrderedListAddBeforeRead() throws Exception { TimestampedValue.of("goodbye", Instant.ofEpochMilli(50)); orderedList.add(helloValue); - waitAndSet(future, Arrays.asList(worldValue), 200); + waitAndSet(future, Collections.singletonList(worldValue), 200); assertThat(orderedList.read(), Matchers.contains(worldValue, helloValue)); orderedList.add(goodbyeValue); @@ -1940,7 +1905,7 @@ public void testOrderedListIsEmptyFalse() throws Exception { STATE_FAMILY, StringUtf8Coder.of()); - waitAndSet(future, Arrays.asList(TimestampedValue.of("world", Instant.EPOCH)), 200); + waitAndSet(future, Collections.singletonList(TimestampedValue.of("world", Instant.EPOCH)), 200); assertThat(result.read(), Matchers.is(false)); } @@ -2266,10 +2231,6 @@ public void testNewOrderedListNoFetch() throws Exception { Mockito.verifyZeroInteractions(mockReader); } - // test ordered list cleared before read - // test fetch + add + read - // test ids - @Test public void testBagAddBeforeRead() throws Exception { StateTag> addr = StateTags.bag("bag", StringUtf8Coder.of()); @@ -2282,13 +2243,17 @@ public void testBagAddBeforeRead() throws Exception { bag.readLater(); bag.add("hello"); - waitAndSet(future, Arrays.asList("world"), 200); + waitAndSet(future, Collections.singletonList("world"), 200); assertThat(bag.read(), Matchers.containsInAnyOrder("hello", "world")); bag.add("goodbye"); assertThat(bag.read(), Matchers.containsInAnyOrder("hello", "world", "goodbye")); } + // test ordered list cleared before read + // test fetch + add + read + // test ids + @Test public void testBagClearBeforeRead() throws Exception { StateTag> addr = StateTags.bag("bag", StringUtf8Coder.of()); @@ -2313,7 +2278,7 @@ public void testBagIsEmptyFalse() throws Exception { ReadableState result = bag.isEmpty().readLater(); Mockito.verify(mockReader).bagFuture(key(NAMESPACE, "bag"), STATE_FAMILY, StringUtf8Coder.of()); - waitAndSet(future, Arrays.asList("world"), 200); + waitAndSet(future, Collections.singletonList("world"), 200); assertThat(result.read(), Matchers.is(false)); } @@ -2328,7 +2293,7 @@ public void testBagIsEmptyTrue() throws Exception { ReadableState result = bag.isEmpty().readLater(); Mockito.verify(mockReader).bagFuture(key(NAMESPACE, "bag"), STATE_FAMILY, StringUtf8Coder.of()); - waitAndSet(future, Arrays.asList(), 200); + waitAndSet(future, Collections.emptyList(), 200); assertThat(result.read(), Matchers.is(true)); } @@ -2436,7 +2401,7 @@ public void testCombiningAddBeforeRead() throws Exception { assertThat(value.read(), Matchers.equalTo(29)); // That get "compressed" the combiner. So, the underlying future should change: - future.set(Arrays.asList(new int[] {29})); + future.set(Collections.singletonList(new int[] {29})); value.add(2); assertThat(value.read(), Matchers.equalTo(31)); @@ -2480,7 +2445,7 @@ public void testCombiningIsEmpty() throws Exception { .bagFuture(byteString.capture(), eq(STATE_FAMILY), Mockito.>any()); assertThat(byteString.getValue(), byteStringEq(COMBINING_KEY)); - waitAndSet(future, Arrays.asList(new int[] {29}), 200); + waitAndSet(future, Collections.singletonList(new int[] {29}), 200); assertThat(result.read(), Matchers.is(false)); } @@ -2527,12 +2492,10 @@ public void testCombiningAddPersistWithCompact() throws Exception { Mockito.when( mockReader.bagFuture( - org.mockito.Matchers.any(), - org.mockito.Matchers.any(), + org.mockito.Matchers.any(), + org.mockito.Matchers.any(), org.mockito.Matchers.>any())) - .thenReturn( - Futures.>immediateFuture( - ImmutableList.of(new int[] {40}, new int[] {60}))); + .thenReturn(Futures.immediateFuture(ImmutableList.of(new int[] {40}, new int[] {60}))); GroupingState value = underTest.state(NAMESPACE, COMBINING_ADDR); @@ -2717,7 +2680,7 @@ public void testWatermarkPersistLatestEmpty() throws Exception { hold.add(new Instant(2000)); when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)) - .thenReturn(Futures.immediateFuture(null)); + .thenReturn(Futures.immediateFuture(null)); Windmill.WorkItemCommitRequest.Builder commitBuilder = Windmill.WorkItemCommitRequest.newBuilder(); @@ -2743,7 +2706,7 @@ public void testWatermarkPersistLatestWindmillWins() throws Exception { hold.add(new Instant(2000)); when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)) - .thenReturn(Futures.immediateFuture(new Instant(4000))); + .thenReturn(Futures.immediateFuture(new Instant(4000))); Windmill.WorkItemCommitRequest.Builder commitBuilder = Windmill.WorkItemCommitRequest.newBuilder(); @@ -2769,7 +2732,7 @@ public void testWatermarkPersistLatestLocalAdditionsWin() throws Exception { hold.add(new Instant(2000)); when(mockReader.watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY)) - .thenReturn(Futures.immediateFuture(new Instant(500))); + .thenReturn(Futures.immediateFuture(new Instant(500))); Windmill.WorkItemCommitRequest.Builder commitBuilder = Windmill.WorkItemCommitRequest.newBuilder(); @@ -2880,7 +2843,7 @@ public void testValueClearBeforeRead() throws Exception { value.clear(); - assertEquals(null, value.read()); + assertNull(value.read()); Mockito.verifyNoMoreInteractions(mockReader); } @@ -2956,7 +2919,7 @@ public void testNewValueNoFetch() throws Exception { StateTag> addr = StateTags.value("value", StringUtf8Coder.of()); ValueState value = underTestNewKey.state(NAMESPACE, addr); - assertEquals(null, value.read()); + assertNull(value.read()); // Shouldn't need to read from windmill for this. Mockito.verifyZeroInteractions(mockReader); @@ -2984,7 +2947,7 @@ public void testCachedValue() throws Exception { resetUnderTest(); value = underTest.state(NAMESPACE, addr); - assertEquals(null, value.read()); + assertNull(value.read()); underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); Mockito.verifyNoMoreInteractions(mockReader); @@ -3087,7 +3050,7 @@ public void testCachedWatermarkHold() throws Exception { resetUnderTest(); hold = underTest.state(NAMESPACE, addr); - assertEquals(null, hold.read()); + assertNull(hold.read()); underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); Mockito.verify(mockReader, times(2)).watermarkFuture(key(NAMESPACE, "watermark"), STATE_FAMILY); @@ -3109,7 +3072,7 @@ public void testCachedCombining() throws Exception { value.readLater(); value.add(1); - waitAndSet(future, Arrays.asList(new int[] {2}), 200); + waitAndSet(future, Collections.singletonList(new int[] {2}), 200); assertThat(value.read(), Matchers.equalTo(3)); underTest.persist(Windmill.WorkItemCommitRequest.newBuilder()); @@ -3149,4 +3112,31 @@ private void disableCompactOnWrite() { private void forceCompactOnWrite() { WindmillStateInternals.COMPACT_NOW.set(() -> true); } + + private static class MultimapEntryUpdate { + String key; + Iterable values; + boolean deleteAll; + + public MultimapEntryUpdate(String key, Iterable values, boolean deleteAll) { + this.key = key; + this.values = values; + this.deleteAll = deleteAll; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (!(o instanceof MultimapEntryUpdate)) return false; + MultimapEntryUpdate that = (MultimapEntryUpdate) o; + return deleteAll == that.deleteAll + && Objects.equals(key, that.key) + && Objects.equals(values, that.values); + } + + @Override + public int hashCode() { + return Objects.hash(key, values, deleteAll); + } + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java similarity index 99% rename from runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateReaderTest.java rename to runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java index 1981a63c0ed5..b8c4803a8f34 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillStateReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/windmill/state/WindmillStateReaderTest.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.runners.dataflow.worker; +package org.apache.beam.runners.dataflow.worker.windmill.state; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.Assert.assertEquals; @@ -33,6 +33,10 @@ import java.util.List; import java.util.Map; import java.util.concurrent.Future; +import org.apache.beam.runners.dataflow.worker.KeyTokenInvalidException; +import org.apache.beam.runners.dataflow.worker.MetricTrackingWindmillServerStub; +import org.apache.beam.runners.dataflow.worker.WindmillStateTestUtils; +import org.apache.beam.runners.dataflow.worker.WindmillTimeUtils; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.KeyedGetDataRequest; import org.apache.beam.runners.dataflow.worker.windmill.Windmill.SortedListEntry; From 6a456f04b68b3e19c66cf55937e6cf2be4cada19 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Wed, 11 Oct 2023 08:42:58 -0700 Subject: [PATCH 67/85] Add user counters and msec metrics to prism UI. (#28929) Co-authored-by: lostluck <13907733+lostluck@users.noreply.github.com> --- .../beam/runners/prism/internal/web/web.go | 40 ++++++++++++++++++- 1 file changed, 39 insertions(+), 1 deletion(-) diff --git a/sdks/go/pkg/beam/runners/prism/internal/web/web.go b/sdks/go/pkg/beam/runners/prism/internal/web/web.go index 7bfbe19a910b..b7afad35aeee 100644 --- a/sdks/go/pkg/beam/runners/prism/internal/web/web.go +++ b/sdks/go/pkg/beam/runners/prism/internal/web/web.go @@ -188,6 +188,10 @@ func (h *jobDetailsHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { trs := pipeResp.GetPipeline().GetComponents().GetTransforms() col2T, topo := preprocessTransforms(trs) + counters := toTransformMap(results.AllMetrics().Counters()) + distributions := toTransformMap(results.AllMetrics().Distributions()) + msecs := toTransformMap(results.AllMetrics().Msecs()) + data.Transforms = make([]pTransform, 0, len(trs)) for _, id := range topo { pt := trs[id] @@ -224,6 +228,29 @@ func (h *jobDetailsHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { strMets = append(strMets, outMets...) } + var msecMets []string + // TODO: Figure out where uniquename or id is being used in prism. It should be all global transform ID to faciliate lookups. + for _, msec := range msecs[pt.GetUniqueName()] { + msecMets = append(msecMets, fmt.Sprintf("\n- %+v", msec.Result())) + } + if len(msecMets) > 0 { + strMets = append(strMets, "Profiling metrics") + strMets = append(strMets, msecMets...) + } + + var userMetrics []string + for _, ctr := range counters[pt.GetUniqueName()] { + userMetrics = append(userMetrics, fmt.Sprintf("\n- %s.%s: %v", ctr.Namespace(), ctr.Name(), ctr.Result())) + } + for _, dist := range distributions[pt.GetUniqueName()] { + userMetrics = append(userMetrics, fmt.Sprintf("\n- %s.%s: %+v", dist.Namespace(), dist.Name(), dist.Result())) + } + + if len(userMetrics) > 0 { + strMets = append(strMets, "User metrics") + strMets = append(strMets, userMetrics...) + } + data.Transforms = append(data.Transforms, pTransform{ ID: id, Transform: pt, @@ -234,6 +261,14 @@ func (h *jobDetailsHandler) ServeHTTP(w http.ResponseWriter, r *http.Request) { renderPage(jobPage, &data, w) } +func toTransformMap[E interface{ Transform() string }](mets []E) map[string][]E { + ret := map[string][]E{} + for _, met := range mets { + ret[met.Transform()] = append(ret[met.Transform()], met) + } + return ret +} + type pcolParent struct { L string T *pipepb.PTransform @@ -244,7 +279,10 @@ type pcolParent struct { func preprocessTransforms(trs map[string]*pipepb.PTransform) (map[string]pcolParent, []string) { ret := map[string]pcolParent{} var leaves []string - for id, t := range trs { + keys := maps.Keys(trs) + sort.Strings(keys) + for _, id := range keys { + t := trs[id] // Skip composites at this time. if len(t.GetSubtransforms()) > 0 { continue From 2bd23b166660a5a6fdf78f0615d3cd6a101122f2 Mon Sep 17 00:00:00 2001 From: Anand Inguva <34158215+AnandInguva@users.noreply.github.com> Date: Wed, 11 Oct 2023 16:20:31 +0000 Subject: [PATCH 68/85] Add counter for MLTransform and data processing transforms (#28927) --- sdks/python/apache_beam/ml/transforms/base.py | 39 ++++++++++++++++++- .../apache_beam/ml/transforms/base_test.py | 25 ++++++++++++ 2 files changed, 63 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/transforms/base.py b/sdks/python/apache_beam/ml/transforms/base.py index a45928f5c8bf..49ce6e9ec1e0 100644 --- a/sdks/python/apache_beam/ml/transforms/base.py +++ b/sdks/python/apache_beam/ml/transforms/base.py @@ -25,6 +25,7 @@ from typing import TypeVar import apache_beam as beam +from apache_beam.metrics.metric import Metrics __all__ = ['MLTransform', 'ProcessHandler', 'BaseOperation'] @@ -88,6 +89,13 @@ def __call__(self, data: OperationInputT, transformed_data = {**transformed_data, **artifacts} return transformed_data + def get_counter(self): + """ + Returns the counter name for the operation. + """ + counter_name = self.__class__.__name__ + return Metrics.counter(MLTransform, f'BeamML_{counter_name}') + class ProcessHandler(Generic[ExampleT, MLTransformOutputT], abc.ABC): """ @@ -194,6 +202,9 @@ def __init__( transforms=transforms) # type: ignore[arg-type] self._process_handler = process_handler + self.transforms = transforms + self._counter = Metrics.counter( + MLTransform, f'BeamML_{self.__class__.__name__}') def expand( self, pcoll: beam.PCollection[ExampleT] @@ -209,8 +220,11 @@ def expand( Args: pcoll: A PCollection of ExampleT type. Returns: - A PCollection of MLTransformOutputT type. + A PCollection of MLTransformOutputT type """ + _ = ( + pcoll.pipeline + | "MLTransformMetricsUsage" >> MLTransformMetricsUsage(self)) return self._process_handler.process_data(pcoll) def with_transform(self, transform: BaseOperation): @@ -230,3 +244,26 @@ def _validate_transform(self, transform): raise TypeError( 'transform must be a subclass of BaseOperation. ' 'Got: %s instead.' % type(transform)) + + +class MLTransformMetricsUsage(beam.PTransform): + def __init__(self, ml_transform: MLTransform): + self._ml_transform = ml_transform + self._ml_transform._counter.inc() + + def expand(self, pipeline): + def _increment_counters(): + # increment for MLTransform. + self._ml_transform._counter.inc() + # increment if data processing transforms are passed. + transforms = ( + self._ml_transform.transforms or + self._ml_transform._process_handler.transforms) + if transforms: + for transform in transforms: + transform.get_counter().inc() + + _ = ( + pipeline + | beam.Create([None]) + | beam.Map(lambda _: _increment_counters())) diff --git a/sdks/python/apache_beam/ml/transforms/base_test.py b/sdks/python/apache_beam/ml/transforms/base_test.py index df7a6d26b47c..2e447964541b 100644 --- a/sdks/python/apache_beam/ml/transforms/base_test.py +++ b/sdks/python/apache_beam/ml/transforms/base_test.py @@ -27,6 +27,7 @@ from parameterized import parameterized import apache_beam as beam +from apache_beam.metrics.metric import MetricsFilter from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to @@ -244,6 +245,30 @@ def test_ml_transforms_on_multiple_columns_multiple_transforms(self): equal_to(expected_output_y, equals_fn=np.array_equal), label='actual_output_y') + def test_mltransform_with_counter(self): + transforms = [ + tft.ComputeAndApplyVocabulary(columns=['y']), + tft.ScaleTo01(columns=['x']) + ] + data = [{'x': [1, 2, 3], 'y': ['a', 'b', 'c']}] + with beam.Pipeline() as p: + _ = ( + p | beam.Create(data) + | base.MLTransform( + transforms=transforms, + write_artifact_location=self.artifact_location)) + scale_to_01_counter = MetricsFilter().with_name('BeamML_ScaleTo01') + vocab_counter = MetricsFilter().with_name( + 'BeamML_ComputeAndApplyVocabulary') + mltransform_counter = MetricsFilter().with_name('BeamML_MLTransform') + result = p.result + self.assertEqual( + result.metrics().query(scale_to_01_counter)['counters'][0].result, 1) + self.assertEqual( + result.metrics().query(vocab_counter)['counters'][0].result, 1) + self.assertEqual( + result.metrics().query(mltransform_counter)['counters'][0].result, 1) + if __name__ == '__main__': unittest.main() From 7a1245d401132d2175dc2fc0efec9b9c704e8e84 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 11 Oct 2023 09:27:35 -0700 Subject: [PATCH 69/85] Bump github.com/google/go-cmp from 0.5.9 to 0.6.0 in /sdks (#28940) Bumps [github.com/google/go-cmp](https://github.com/google/go-cmp) from 0.5.9 to 0.6.0. - [Release notes](https://github.com/google/go-cmp/releases) - [Commits](https://github.com/google/go-cmp/compare/v0.5.9...v0.6.0) --- updated-dependencies: - dependency-name: github.com/google/go-cmp 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 | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index e5baa554fe6d..edb524545bd5 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -40,7 +40,7 @@ require ( github.com/dustin/go-humanize v1.0.1 github.com/go-sql-driver/mysql v1.7.1 github.com/golang/protobuf v1.5.3 // TODO(danoliveira): Fully replace this with google.golang.org/protobuf - github.com/google/go-cmp v0.5.9 + github.com/google/go-cmp v0.6.0 github.com/google/uuid v1.3.1 github.com/johannesboyne/gofakes3 v0.0.0-20221110173912-32fb85c5aed6 github.com/lib/pq v1.10.9 diff --git a/sdks/go.sum b/sdks/go.sum index f1d0c53d9685..3cdcc9ef8e99 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -259,8 +259,9 @@ github.com/google/go-cmp v0.5.4/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/ github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.6/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.5.8/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= -github.com/google/go-cmp v0.5.9 h1:O2Tfq5qg4qc4AmwVlvv0oLiVAGB7enBSJ2x2DqQFi38= 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/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= From 2bfcb9fda37be8d5b9550a1b118ed87d03fdf12c Mon Sep 17 00:00:00 2001 From: Joar Wandborg Date: Wed, 11 Oct 2023 19:00:04 +0200 Subject: [PATCH 70/85] Remove unnecessary a space character that got through Javadoc CI ;) (#28863) (Tongue in cheek) Spotless caught me when I did not end the first sentence of the Javadoc with a a dot, but it did not tell me I had an extra space before the `` tag at the end of the sentence. --- .../org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java index 971b91d89b74..f6224cc24b2f 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigtable/BigtableIOTest.java @@ -768,7 +768,7 @@ public void testReadingWithSplits() throws Exception { /** * Regression test for [Bug]: BigtableSource - * "Desired bundle size 0 bytes must be greater than 0" #28793 . + * "Desired bundle size 0 bytes must be greater than 0" #28793. */ @Test public void testSplittingWithDesiredBundleSizeZero() throws Exception { From aedfa461fb354e0e97d75694d2a117d9ad505602 Mon Sep 17 00:00:00 2001 From: Minbo Bae <49642083+baeminbo@users.noreply.github.com> Date: Wed, 11 Oct 2023 10:57:24 -0700 Subject: [PATCH 71/85] [#20970] Fix gRPC leak by closing ResidualSource at BoundedToUnboundedSourceAdapter.Reader#init() in Dataflow worker (#28548) --- .../UnboundedReadFromBoundedSource.java | 14 +- .../UnboundedReadFromBoundedSourceTest.java | 180 ++++++++++++++++++ .../dataflow/worker/WorkerCustomSources.java | 7 +- 3 files changed, 198 insertions(+), 3 deletions(-) diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java index 67697636a363..53fad782da96 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSource.java @@ -17,8 +17,8 @@ */ package org.apache.beam.runners.core.construction; -import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkNotNull; +import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import java.io.IOException; import java.io.InputStream; @@ -288,6 +288,15 @@ private void init( residualElementsList == null ? new ResidualElements(Collections.emptyList()) : new ResidualElements(residualElementsList); + + if (this.residualSource != null) { + // close current residualSource to avoid leak of reader.close() in ResidualSource + try { + this.residualSource.close(); + } catch (IOException e) { + LOG.warn("Ignore error at closing ResidualSource", e); + } + } this.residualSource = residualSource == null ? null : new ResidualSource(residualSource, options); } @@ -465,7 +474,7 @@ public ResidualSource(BoundedSource residualSource, PipelineOptions options) } private boolean advance() throws IOException { - checkArgument(!closed, "advance() call on closed %s", getClass().getName()); + checkState(!closed, "advance() call on closed %s", getClass().getName()); if (readerDone) { return false; } @@ -505,6 +514,7 @@ BoundedSource getSource() { } Checkpoint getCheckpointMark() { + checkState(!closed, "getCheckpointMark() call on closed %s", getClass().getName()); if (reader == null) { // Reader hasn't started, checkpoint the residualSource. return new Checkpoint<>(null /* residualElements */, residualSource); diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java index cd4b49262fcb..31f6842a42bc 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/UnboundedReadFromBoundedSourceTest.java @@ -26,9 +26,15 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; +import java.lang.ref.Reference; +import java.lang.ref.ReferenceQueue; +import java.lang.ref.WeakReference; import java.nio.ByteBuffer; import java.nio.channels.ReadableByteChannel; +import java.util.Collections; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.NoSuchElementException; import java.util.Random; import org.apache.beam.runners.core.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; @@ -69,10 +75,14 @@ import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** Unit tests for {@link UnboundedReadFromBoundedSource}. */ @RunWith(JUnit4.class) public class UnboundedReadFromBoundedSourceTest { + private static final Logger LOG = + LoggerFactory.getLogger(UnboundedReadFromBoundedSourceTest.class); @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); @@ -280,6 +290,38 @@ public void testReadFromCheckpointBeforeStart() throws Exception { unboundedSource.createReader(options, checkpoint).getCurrent(); } + @Test + public void testReadersClosedProperly() throws IOException { + ManagedReaderBoundedSource boundedSource = new ManagedReaderBoundedSource(0, 10); + BoundedToUnboundedSourceAdapter unboundedSource = + new BoundedToUnboundedSourceAdapter<>(boundedSource); + PipelineOptions options = PipelineOptionsFactory.create(); + + BoundedToUnboundedSourceAdapter.Reader reader = + unboundedSource.createReader(options, new Checkpoint(null, boundedSource)); + + for (int i = 0; i < 3; ++i) { + if (i == 0) { + assertTrue(reader.start()); + } else { + assertTrue(reader.advance()); + } + assertEquals(i, (int) reader.getCurrent()); + } + Checkpoint checkpoint = reader.getCheckpointMark(); + List> residualElements = checkpoint.getResidualElements(); + for (int i = 0; i < 7; ++i) { + TimestampedValue element = residualElements.get(i); + assertEquals(i + 3, (int) element.getValue()); + } + for (int i = 0; i < 100; ++i) { + // A WeakReference of an object that no other objects reference are not immediately added to + // ReferenceQueue. To test this, we should run System.gc() multiple times. + // If a reader is GCed without closing, `cleanQueue` throws a RuntimeException. + boundedSource.cleanQueue(); + } + } + /** Generate byte array of given size. */ private static byte[] generateInput(int size) { // Arbitrary but fixed seed @@ -298,6 +340,7 @@ private static void writeFile(File file, byte[] input) throws IOException { /** Unsplittable source for use in tests. */ private static class UnsplittableSource extends FileBasedSource { + public UnsplittableSource(String fileOrPatternSpec, long minBundleSize) { super(StaticValueProvider.of(fileOrPatternSpec), minBundleSize); } @@ -323,6 +366,7 @@ public Coder getOutputCoder() { } private static class UnsplittableReader extends FileBasedReader { + ByteBuffer buff = ByteBuffer.allocate(1); Byte current; long offset; @@ -370,4 +414,140 @@ protected long getCurrentOffset() { } } } + + /** + * An integer generating bounded source. This source class checks if readers are closed properly. + * For that, it manages weak references of readers, and checks at `createReader` and `cleanQueue` + * if readers were closed before GCed. The `cleanQueue` does not change the state in + * `ManagedReaderBoundedSource`, but throws an exception if it finds a reader GCed without + * closing. + */ + private static class ManagedReaderBoundedSource extends BoundedSource { + + private final int from; + private final int to; // exclusive + + private transient ReferenceQueue refQueue; + private transient Map, CloseStatus> cloesStatusMap; + + public ManagedReaderBoundedSource(int from, int to) { + if (from > to) { + throw new RuntimeException( + String.format("`from` <= `to`, but got from: %d, to: %d", from, to)); + } + this.from = from; + this.to = to; + } + + @Override + public List> split( + long desiredBundleSizeBytes, PipelineOptions options) { + return Collections.singletonList(this); + } + + @Override + public long getEstimatedSizeBytes(PipelineOptions options) { + return (to - from) * 4L; + } + + @Override + public BoundedReader createReader(PipelineOptions options) { + // Add weak reference to queue to monitor GCed readers. If `CloseStatus` associated with + // reader is not closed, it means a reader was GCed without closing properly. The CloseStatus + // check for GCed readers are done at cleanQueue(). + if (refQueue == null) { + refQueue = new ReferenceQueue<>(); + cloesStatusMap = new HashMap<>(); + } + cleanQueue(); + + CloseStatus status = new CloseStatus(); + ManagedReader reader = new ManagedReader(status); + WeakReference reference = new WeakReference<>(reader, refQueue); + cloesStatusMap.put(reference, status); + LOG.info("Add reference {} for reader {}", reference, reader); + return reader; + } + + public void cleanQueue() { + System.gc(); + + Reference reference; + while ((reference = refQueue.poll()) != null) { + CloseStatus closeStatus = cloesStatusMap.get(reference); + LOG.info("Poll reference: {}, closed: {}", reference, closeStatus.closed); + closeStatus.throwIfNotClosed(); + } + } + + class CloseStatus { + + private final RuntimeException allocationStacktrace; + + private boolean closed; + + public CloseStatus() { + allocationStacktrace = + new RuntimeException("Previous reader was not closed properly. Reader allocation was"); + closed = false; + } + + void close() { + cleanQueue(); + closed = true; + } + + void throwIfNotClosed() { + if (!closed) { + throw allocationStacktrace; + } + } + } + + class ManagedReader extends BoundedReader { + + private final CloseStatus status; + + int current; + + public ManagedReader(CloseStatus status) { + this.status = status; + } + + @Override + public boolean start() { + if (from < to) { + current = from; + return true; + } else { + return false; + } + } + + @Override + public boolean advance() { + if (current + 1 < to) { + ++current; + return true; + } else { + return false; + } + } + + @Override + public Integer getCurrent() { + return current; + } + + @Override + public void close() { + status.close(); + } + + @Override + public BoundedSource getCurrentSource() { + return ManagedReaderBoundedSource.this; + } + } + } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java index 872dc1e89a79..a9050236efc8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSources.java @@ -776,6 +776,9 @@ public double getRemainingParallelism() { private static class UnboundedReaderIterator extends NativeReader.NativeReaderIterator>> { + // Do not close reader. The reader is cached in StreamingModeExecutionContext.readerCache, and + // will be reused until the cache is evicted, expired or invalidated. + // See UnboundedReader#iterator(). private final UnboundedSource.UnboundedReader reader; private final StreamingModeExecutionContext context; private final boolean started; @@ -862,7 +865,9 @@ public WindowedValue> getCurrent() throws NoSuchElementExce } @Override - public void close() {} + public void close() { + // Don't close reader. + } @Override public NativeReader.Progress getProgress() { From bd46c76dd086f2546006460d280cbbe7ae19d3ee Mon Sep 17 00:00:00 2001 From: Bruno Volpato Date: Wed, 11 Oct 2023 15:23:46 -0400 Subject: [PATCH 72/85] Bump snappy-java version to 1.1.10.4 (#28655) --- .../main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 705bcb978e12..19ed6aaa4276 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -860,7 +860,7 @@ class BeamModulePlugin implements Plugin { slf4j_jul_to_slf4j : "org.slf4j:jul-to-slf4j:$slf4j_version", slf4j_log4j12 : "org.slf4j:slf4j-log4j12:$slf4j_version", slf4j_jcl : "org.slf4j:slf4j-jcl:$slf4j_version", - snappy_java : "org.xerial.snappy:snappy-java:1.1.10.3", + snappy_java : "org.xerial.snappy:snappy-java:1.1.10.4", spark_core : "org.apache.spark:spark-core_2.11:$spark2_version", spark_streaming : "org.apache.spark:spark-streaming_2.11:$spark2_version", spark3_core : "org.apache.spark:spark-core_2.12:$spark3_version", From ebad3deb5a0a497b0a4a46ca38f896ad3bd7b09b Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Wed, 11 Oct 2023 23:26:28 +0400 Subject: [PATCH 73/85] Add GitHub Workflow Replacements for Jenkins job_LoadTests_Python_CoGBK subgroup (#28764) --- ..._LoadTests_Python_CoGBK_Dataflow_Batch.yml | 127 ++++++++++++++++ ...dTests_Python_CoGBK_Dataflow_Streaming.yml | 126 ++++++++++++++++ ...eam_LoadTests_Python_CoGBK_Flink_Batch.yml | 135 ++++++++++++++++++ ...oGBK_Dataflow_Batch_100b_Multiple_Keys.txt | 28 ++++ ...n_CoGBK_Dataflow_Batch_100b_Single_Key.txt | 28 ++++ .../python_CoGBK_Dataflow_Batch_10kB.txt | 28 ++++ .../python_CoGBK_Dataflow_Batch_2MB.txt | 28 ++++ ...ataflow_Flink_Batch_100b_Multiple_Keys.txt | 28 ++++ ...K_Dataflow_Flink_Batch_100b_Single_Key.txt | 28 ++++ ...python_CoGBK_Dataflow_Flink_Batch_10kB.txt | 28 ++++ ..._Dataflow_Streaming_100b_Multiple_Keys.txt | 30 ++++ ...GBK_Dataflow_Streaming_100b_Single_Key.txt | 30 ++++ .../python_CoGBK_Dataflow_Streaming_10kB.txt | 30 ++++ .../python_CoGBK_Dataflow_Streaming_2MB.txt | 30 ++++ 14 files changed, 704 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml create mode 100644 .github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt create mode 100644 .github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml new file mode 100644 index 000000000000..284ba2420654 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Batch.yml @@ -0,0 +1,127 @@ +# 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. + +name: LoadTests Python CoGBK Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 11 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_CoGBK_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python CoGBK Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_CoGBK_Dataflow_Batch"] + job_phrase: ["Run Load Tests Python CoGBK Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB of 100B records with a single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + --info \ + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Batch_test_arguments_1 }} --job_name=load-tests-python-dataflow-batch-cogbk-1-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK 2GB of 100B records with multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Batch_test_arguments_2 }} --job_name=load-tests-python-dataflow-batch-cogbk-2-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Batch_test_arguments_3 }} --job_name=load-tests-python-dataflow-batch-cogbk-3-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Batch_test_arguments_4 }} --job_name=load-tests-python-dataflow-batch-cogbk-4-${{ steps.datetime.outputs.datetime }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml new file mode 100644 index 000000000000..313ba94f9df3 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Dataflow_Streaming.yml @@ -0,0 +1,126 @@ +# 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. + +name: LoadTests Python CoGBK Dataflow Streaming + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 11 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_CoGBK_Dataflow_Streaming: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python CoGBK Dataflow Streaming' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_CoGBK_Dataflow_Streaming"] + job_phrase: ["Run Load Tests Python CoGBK Dataflow Streaming"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB of 100B records with a single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Streaming_test_arguments_1 }} --job_name=load-tests-python-dataflow-streaming-cogbk-1-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK 2GB of 100B records with multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Streaming_test_arguments_2 }} --job_name=load-tests-python-dataflow-streaming-cogbk-2-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Streaming_test_arguments_3 }} --job_name=load-tests-python-dataflow-streaming-cogbk-3-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK reiterate 4 times 2MB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Dataflow_Streaming_test_arguments_4 }} --job_name=load-tests-python-dataflow-streaming-cogbk-4-${{ steps.datetime.outputs.datetime }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml new file mode 100644 index 000000000000..6d668c1b2566 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_CoGBK_Flink_Batch.yml @@ -0,0 +1,135 @@ +# 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. + +name: LoadTests Python CoGBK Dataflow Flink Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '40 12 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + GCLOUD_ZONE: us-central1-a + CLUSTER_NAME: beam-loadtests-python-cogbk-flink-batch-${{ github.run_id }} + GCS_BUCKET: gs://beam-flink-cluster + FLINK_DOWNLOAD_URL: https://archive.apache.org/dist/flink/flink-1.15.0/flink-1.15.0-bin-scala_2.12.tgz + HADOOP_DOWNLOAD_URL: https://repo.maven.apache.org/maven2/org/apache/flink/flink-shaded-hadoop-2-uber/2.8.3-10.0/flink-shaded-hadoop-2-uber-2.8.3-10.0.jar + FLINK_TASKMANAGER_SLOTS: 1 + DETACHED_MODE: true + HARNESS_IMAGES_TO_PULL: gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest + JOB_SERVER_IMAGE: gcr.io/apache-beam-testing/beam_portability/beam_flink1.15_job_server:latest + ARTIFACTS_DIR: gs://beam-flink-cluster/beam-loadtests-python-cogbk-flink-batch-${{ github.run_id }} + +jobs: + beam_LoadTests_Python_CoGBK_Flink_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python CoGBK Flink Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_CoGBK_Flink_Batch"] + job_phrase: ["Run Load Tests Python CoGBK Flink Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Install Python + uses: actions/setup-python@v4 + with: + python-version: '3.8' + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt + - name: Start Flink with parallelism 5 + env: + FLINK_NUM_WORKERS: 5 + run: | + cd ${{ github.workspace }}/.test-infra/dataproc; ./flink_cluster.sh create + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB of 100B records with a single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + --info \ + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Flink_Batch_test_arguments_1 }} --job_name=load-tests-python-flink-batch-cogbk-1-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK 2GB of 100B records with multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + --info \ + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Flink_Batch_test_arguments_2 }} --job_name=load-tests-python-flink-batch-cogbk-2-${{ steps.datetime.outputs.datetime }}' \ + - name: run CoGBK reiterate 4 times 10kB values + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + --info \ + -PloadTest.mainClass=apache_beam.testing.load_tests.co_group_by_key_test \ + -Prunner=FlinkRunner \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_CoGBK_Flink_Batch_test_arguments_2 }} --job_name=load-tests-python-flink-batch-cogbk-3-${{ steps.datetime.outputs.datetime }}' \ + - name: Teardown Flink + if: always() + run: | + ${{ github.workspace }}/.test-infra/dataproc/flink_cluster.sh delete \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt new file mode 100644 index 000000000000..d5ba43180738 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt @@ -0,0 +1,28 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_cogbk_2 +--influx_measurement=python_batch_cogbk_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":5,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt new file mode 100644 index 000000000000..47ebf22dc835 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_100b_Single_Key.txt @@ -0,0 +1,28 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_cogbk_1 +--influx_measurement=python_batch_cogbk_1 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt new file mode 100644 index 000000000000..13161125b570 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_10kB.txt @@ -0,0 +1,28 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_cogbk_3 +--influx_measurement=python_batch_cogbk_3 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":200000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt new file mode 100644 index 000000000000..052c2464a1cc --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Batch_2MB.txt @@ -0,0 +1,28 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_cogbk_4 +--influx_measurement=python_batch_cogbk_4 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--num_workers=5 +--autoscaling_algorithm=NONE +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt new file mode 100644 index 000000000000..4b8a2f72010b --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Multiple_Keys.txt @@ -0,0 +1,28 @@ +# 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. + +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_cogbk_2 +--influx_measurement=python_batch_cogbk_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":5,\\"hot_key_fraction\\":1}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":5,\\"hot_key_fraction\\":1}'' +--iterations=1 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt new file mode 100644 index 000000000000..3aeb927f04ee --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_100b_Single_Key.txt @@ -0,0 +1,28 @@ +# 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. + +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_cogbk_1 +--influx_measurement=python_batch_cogbk_1 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1,\\"hot_key_fraction\\":1}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1}'' +--iterations=1 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt new file mode 100644 index 000000000000..e350e2d29944 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Flink_Batch_10kB.txt @@ -0,0 +1,28 @@ +# 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. + +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_flink_batch_cogbk_3 +--influx_measurement=python_batch_cogbk_3 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":200000,\\"hot_key_fraction\\":1}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1}'' +--iterations=4 +--parallelism=5 +--endpoint=localhost:8099 +--environment_type=DOCKER +--environment_config=gcr.io/apache-beam-testing/beam-sdk/beam_go_sdk:latest \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt new file mode 100644 index 000000000000..a687f0cf5de7 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Multiple_Keys.txt @@ -0,0 +1,30 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_cogbk_2 +--influx_measurement=python_streaming_cogbk_2 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":5,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--worker_machine_type=n1-highmem-4 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt new file mode 100644 index 000000000000..9141182b90fc --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_100b_Single_Key.txt @@ -0,0 +1,30 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_cogbk_1 +--influx_measurement=python_streaming_cogbk_1 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=1 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--worker_machine_type=n1-highmem-4 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt new file mode 100644 index 000000000000..7250f073f25e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_10kB.txt @@ -0,0 +1,30 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_cogbk_3 +--influx_measurement=python_streaming_cogbk_3 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":200000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--worker_machine_type=n1-highmem-4 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt new file mode 100644 index 000000000000..59723107d53c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python-cogbk/python_CoGBK_Dataflow_Streaming_2MB.txt @@ -0,0 +1,30 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_streaming_cogbk_4 +--influx_measurement=python_streaming_cogbk_4 +--input_options=''{\\"num_records\\":20000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--co_input_options=''{\\"num_records\\":2000000,\\"key_size\\":10,\\"value_size\\":90,\\"num_hot_keys\\":1000,\\"hot_key_fraction\\":1,\\"algorithm\\":\\"lcg\\"}'' +--iterations=4 +--num_workers=5 +--autoscaling_algorithm=NONE +--streaming +--worker_machine_type=n1-highmem-4 +--runner=DataflowRunner \ No newline at end of file From dfdbcb9006b92dd7b896d09c050726ebfb44446c Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Wed, 11 Oct 2023 23:27:04 +0400 Subject: [PATCH 74/85] Add GitHub Workflow Replacements for Jenkins job_LoadTests_Java_CoGBK subgroup (#28742) --- ...am_LoadTests_Java_CoGBK_Dataflow_Batch.yml | 114 +++++++++++++++ ...a_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 132 ++++++++++++++++++ ...GBK_Dataflow_V2_Streaming_JavaVersions.yml | 132 ++++++++++++++++++ ...a_CoGBK_SparkStructuredStreaming_Batch.yml | 114 +++++++++++++++ ...oGBK_Dataflow_Batch_100b_Multiple_Keys.txt | 27 ++++ ...a_CoGBK_Dataflow_Batch_100b_Single_Key.txt | 27 ++++ .../java_CoGBK_Dataflow_Batch_10kB.txt | 27 ++++ .../java_CoGBK_Dataflow_Batch_2MB.txt | 27 ++++ ...w_V2_Streaming_Java_100b_Multiple_Keys.txt | 29 ++++ ...flow_V2_Streaming_Java_100b_Single_Key.txt | 29 ++++ ..._CoGBK_Dataflow_V2_Streaming_Java_10kB.txt | 29 ++++ ...a_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt | 29 ++++ ...uredStreaming_Batch_100b_Multiple_Keys.txt | 25 ++++ ...ucturedStreaming_Batch_100b_Single_Key.txt | 25 ++++ ...BK_SparkStructuredStreaming_Batch_10kB.txt | 25 ++++ ...GBK_SparkStructuredStreaming_Batch_2MB.txt | 25 ++++ 16 files changed, 816 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml create mode 100644 .github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml create mode 100644 .github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml create mode 100644 .github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt create mode 100644 .github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml new file mode 100644 index 000000000000..25feaaf288a1 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_Batch.yml @@ -0,0 +1,114 @@ +# 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. + +name: LoadTests Java CoGBK Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 9 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_CoGBK_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java CoGBK Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_CoGBK_Dataflow_Batch"] + job_phrase: ["Run Load Tests Java CoGBK Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB 100 byte records - single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Batch_test_arguments_1 }} --appName=load_tests_Java_Dataflow_batch_CoGBK_1' \ + - name: run CoGBK 2GB 100 byte records - multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Batch_test_arguments_2 }} --appName=load_tests_Java_Dataflow_batch_CoGBK_2' \ + - name: run CoGBK 2GB reiteration 10kB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Batch_test_arguments_3 }} --appName=load_tests_Java_Dataflow_batch_CoGBK_3' \ + - name: run CoGBK 2GB reiteration 2MB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_Batch_test_arguments_4 }} --appName=load_tests_Java_Dataflow_batch_CoGBK_4' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml new file mode 100644 index 000000000000..35575aec0bef --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -0,0 +1,132 @@ +# 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. + +name: LoadTests Java CoGBK Dataflow V2 Batch JavaVersions + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 10 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + (contains(github.event.comment.body, 'Run Load Tests Java') && + contains(github.event.comment.body, 'CoGBK Dataflow V2 Batch')) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{matrix.job_name}} (${{matrix.job_phrase_1}} ${{matrix.java_version}} ${{matrix.job_phrase_2}}) + strategy: + fail-fast: false + matrix: + job_name: ["beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions"] + job_phrase_1: ["Run Load Tests Java"] + job_phrase_2: ["CoGBK Dataflow V2 Batch"] + java_version: ['11','17'] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: ${{ matrix.java_version }} + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt + arguments: | + --influxTags={\"runnerVersion\":\"v2\",\"jdk\":\"java${{ matrix.java_version }}\"} + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB 100 byte records - single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions_test_arguments_1 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_CoGBK_1' \ + - name: run CoGBK 2GB 100 byte records - multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions_test_arguments_2 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_CoGBK_2' \ + - name: run CoGBK 2GB reiteration 10kB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions_test_arguments_3 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_CoGBK_3' \ + - name: run CoGBK 2GB reiteration 2MB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions_test_arguments_4 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_batch_CoGBK_4' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml new file mode 100644 index 000000000000..46dfaa986920 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -0,0 +1,132 @@ +# 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. + +name: LoadTests Java CoGBK Dataflow V2 Streaming JavaVersions + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 10 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + (contains(github.event.comment.body, 'Run Load Tests Java') && + contains(github.event.comment.body, 'CoGBK Dataflow V2 Streaming')) + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 240 + name: ${{matrix.job_name}} (${{matrix.job_phrase_1}} ${{matrix.java_version}} ${{matrix.job_phrase_2}}) + strategy: + fail-fast: false + matrix: + job_name: ["beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions"] + job_phrase_1: ["Run Load Tests Java"] + job_phrase_2: ["CoGBK Dataflow V2 Streaming"] + java_version: ['11','17'] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} ${{ matrix.job_phrase_1 }} ${{matrix.java_version}} ${{ matrix.job_phrase_2 }} + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + java-version: ${{ matrix.java_version }} + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt + arguments: | + --influxTags={\"runnerVersion\":\"v2\",\"jdk\":\"java${{ matrix.java_version }}\"} + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB 100 byte records - single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions_test_arguments_1 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_CoGBK_1' \ + - name: run CoGBK 2GB 100 byte records - multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions_test_arguments_2 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_CoGBK_2' \ + - name: run CoGBK 2GB reiteration 10kB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions_test_arguments_3 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_CoGBK_3' \ + - name: run CoGBK 2GB reiteration 2MB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:google-cloud-dataflow-java \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions_test_arguments_4 }} --appName=load_tests_Java${{ matrix.java_version }}_Dataflow_V2_streaming_CoGBK_4' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml new file mode 100644 index 000000000000..3d18345589ed --- /dev/null +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch.yml @@ -0,0 +1,114 @@ +# 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. + +name: LoadTests Java CoGBK SparkStructuredStreaming Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 11 * * *' + workflow_dispatch: + +# Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Java CoGBK SparkStructuredStreaming Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch"] + job_phrase: ["Run Load Tests Java CoGBK SparkStructuredStreaming Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: java + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run CoGBK 2GB 100 byte records - single key + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch_test_arguments_1 }}' \ + - name: run CoGBK 2GB 100 byte records - multiple keys + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch_test_arguments_2 }}' \ + - name: run CoGBK 2GB reiteration 10kB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch_test_arguments_3 }}' \ + - name: run CoGBK 2GB reiteration 2MB value + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:java:testing:load-tests:run + arguments: | + -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ + -Prunner=:runners:spark:3 \ + '-PloadTest.args=${{ env.beam_LoadTests_Java_CoGBK_SparkStructuredStreaming_Batch_test_arguments_4 }}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt new file mode 100644 index 000000000000..6402c43d1567 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Multiple_Keys.txt @@ -0,0 +1,27 @@ +# 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. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":5} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt new file mode 100644 index 000000000000..5d53c3b3d7ef --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_100b_Single_Key.txt @@ -0,0 +1,27 @@ +# 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. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt new file mode 100644 index 000000000000..501fa6b3a57f --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_10kB.txt @@ -0,0 +1,27 @@ +# 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. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200000} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt new file mode 100644 index 000000000000..0d5f57fc47b8 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_Batch_2MB.txt @@ -0,0 +1,27 @@ +# 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. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=false +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt new file mode 100644 index 000000000000..24aff12bad79 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Multiple_Keys.txt @@ -0,0 +1,29 @@ +# 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. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_cogbk_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":5} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--coInputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt new file mode 100644 index 000000000000..eead04aeb7e4 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_100b_Single_Key.txt @@ -0,0 +1,29 @@ +# 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. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_cogbk_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=1 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--coInputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt new file mode 100644 index 000000000000..4b45c7df3f9e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_10kB.txt @@ -0,0 +1,29 @@ +# 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. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_cogbk_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200000} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--coInputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt new file mode 100644 index 000000000000..f522fd7ec36d --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_Dataflow_V2_Streaming_Java_2MB.txt @@ -0,0 +1,29 @@ +# 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. + +--region=us-central1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_streaming_cogbk_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=4 +--numWorkers=5 +--autoscalingAlgorithm=NONE +--streaming=true +--inputWindowDurationSec=1200 +--coInputWindowDurationSec=1200 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt new file mode 100644 index 000000000000..747f495b144d --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Multiple_Keys.txt @@ -0,0 +1,25 @@ +# 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. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_CoGBK_2 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_2 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":5} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=1 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt new file mode 100644 index 000000000000..b17549a702f3 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_100b_Single_Key.txt @@ -0,0 +1,25 @@ +# 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. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_CoGBK_1 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_1 +--publishToInfluxDB=true +--sourceOptions={"numRecords":20000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=1 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt new file mode 100644 index 000000000000..84f53ee120a2 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_10kB.txt @@ -0,0 +1,25 @@ +# 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. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_CoGBK_3 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_3 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":200000} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=4 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt new file mode 100644 index 000000000000..8f8bdb899551 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/java_CoGBK_SparkStructuredStreaming_Batch_2MB.txt @@ -0,0 +1,25 @@ +# 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. + +--appName=load_tests_Java_SparkStructuredStreaming_batch_CoGBK_4 +--tempLocation=gs://temp-storage-for-perf-tests/loadtests +--influxMeasurement=java_batch_cogbk_4 +--publishToInfluxDB=true +--sourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--coSourceOptions={"numRecords":2000000,"keySizeBytes":10,"valueSizeBytes":90,"numHotKeys":1000} +--iterations=4 +--streaming=false +--runner=SparkStructuredStreamingRunner \ No newline at end of file From 09f30537015c2383402fcccfe688ddf428b1ea7e Mon Sep 17 00:00:00 2001 From: Aleksandr Dudko <116064902+aleksandr-dudko@users.noreply.github.com> Date: Wed, 11 Oct 2023 23:27:32 +0400 Subject: [PATCH 75/85] Add GitHub Workflow Replacements for Jenkins job_LoadTests_Python_SideInput_Dataflow_Batch (#28826) --- ...dTests_Python_SideInput_Dataflow_Batch.yml | 186 ++++++++++++++++++ ...w_Batch_10gb_1000window_first_iterable.txt | 30 +++ ...ataflow_Batch_10gb_1000window_iterable.txt | 29 +++ ...flow_Batch_10gb_1window_first_iterable.txt | 29 +++ ...t_Dataflow_Batch_10gb_1window_iterable.txt | 28 +++ ...Batch_1gb_1000window_1key_percent_dict.txt | 30 +++ ...atch_1gb_1000window_99key_percent_dict.txt | 30 +++ ...ow_Batch_1gb_1window_1key_percent_dict.txt | 29 +++ ...w_Batch_1gb_1window_99key_percent_dict.txt | 29 +++ ..._Dataflow_Batch_1gb_1window_first_list.txt | 29 +++ ...eInput_Dataflow_Batch_1gb_1window_list.txt | 28 +++ 11 files changed, 477 insertions(+) create mode 100644 .github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt create mode 100644 .github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt diff --git a/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml new file mode 100644 index 000000000000..eb58c85e92e8 --- /dev/null +++ b/.github/workflows/beam_LoadTests_Python_SideInput_Dataflow_Batch.yml @@ -0,0 +1,186 @@ +# 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. + +name: LoadTests Python SideInput Dataflow Batch + +on: + issue_comment: + types: [created] + schedule: + - cron: '30 13 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_LoadTests_Python_SideInput_Dataflow_Batch: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Load Tests Python SideInput Dataflow Batch' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 720 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_LoadTests_Python_SideInput_Dataflow_Batch"] + job_phrase: ["Run Load Tests Python SideInput Dataflow Batch"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Setup environment + uses: ./.github/actions/setup-environment-action + with: + python-version: 3.8 + - name: Prepare test arguments + uses: ./.github/actions/test-arguments-action + with: + test-type: load + test-language: python + argument-file-paths: | + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt + ${{ github.workspace }}/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt + - name: Set current datetime + id: datetime + run: | + echo "datetime=$(date '+%m%d%H%M%S' --utc)" >> $GITHUB_OUTPUT + # The env variables are created and populated in the test-arguments-action as "_test_arguments_" + - name: run SideInput Dataflow Batch Python Load Test 1 (1gb-1kb-10workers-1window-1key-percent-dict) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_1 }} --job_name=load-tests-python-dataflow-batch-sideinput-1-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 2 (1gb-1kb-10workers-1window-99key-percent-dict) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_2 }} --job_name=load-tests-python-dataflow-batch-sideinput-2-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 3 (10gb-1kb-10workers-1window-first-iterable) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_3 }} --job_name=load-tests-python-dataflow-batch-sideinput-3-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 4 (10gb-1kb-10workers-1window-iterable) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_4 }} --job_name=load-tests-python-dataflow-batch-sideinput-4-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 5 (1gb-1kb-10workers-1window-first-list) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_5 }} --job_name=load-tests-python-dataflow-batch-sideinput-5-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 6 (1gb-1kb-10workers-1window-list) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_6 }} --job_name=load-tests-python-dataflow-batch-sideinput-6-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 7 (1gb-1kb-10workers-1000window-1key-percent-dict) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_7 }} --job_name=load-tests-python-dataflow-batch-sideinput-7-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 8 (1gb-1kb-10workers-1000window-99key-percent-dict) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_8 }} --job_name=load-tests-python-dataflow-batch-sideinput-8-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 9 (10gb-1kb-10workers-1000window-first-iterable) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_9 }} --job_name=load-tests-python-dataflow-batch-sideinput-9-${{ steps.datetime.outputs.datetime }}' \ + - name: run SideInput Dataflow Batch Python Load Test 10 (10gb-1kb-10workers-1000window-iterable) + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:apache_beam:testing:load_tests:run + arguments: | + -PloadTest.mainClass=apache_beam.testing.load_tests.sideinput_test \ + -Prunner=DataflowRunner \ + -PpythonVersion=3.8 \ + '-PloadTest.args=${{ env.beam_LoadTests_Python_SideInput_Dataflow_Batch_test_arguments_10 }} --job_name=load-tests-python-dataflow-batch-sideinput-10-${{ steps.datetime.outputs.datetime }}' \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt new file mode 100644 index 000000000000..204c07bc16a2 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_first_iterable.txt @@ -0,0 +1,30 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_9 +--influx_measurement=python_batch_sideinput_9 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":10000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=iter +--access_percentage=1 +--window_count=1000 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt new file mode 100644 index 000000000000..1ae64bb4a369 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1000window_iterable.txt @@ -0,0 +1,29 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_10 +--influx_measurement=python_batch_sideinput_10 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":10000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=iter +--window_count=1000 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt new file mode 100644 index 000000000000..0759517d9c2e --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_first_iterable.txt @@ -0,0 +1,29 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_3 +--influx_measurement=python_batch_sideinput_3 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":10000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=iter +--access_percentage=1 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt new file mode 100644 index 000000000000..c555c0d32d4c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_10gb_1window_iterable.txt @@ -0,0 +1,28 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_4 +--influx_measurement=python_batch_sideinput_4 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":10000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=iter +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt new file mode 100644 index 000000000000..4b3cee817f43 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_1key_percent_dict.txt @@ -0,0 +1,30 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_7 +--influx_measurement=python_batch_sideinput_7 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":1000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=dict +--access_percentage=1 +--window_count=1000 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt new file mode 100644 index 000000000000..00ba6feef50c --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1000window_99key_percent_dict.txt @@ -0,0 +1,30 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_8 +--influx_measurement=python_batch_sideinput_8 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":1000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=dict +--access_percentage=99 +--window_count=1000 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt new file mode 100644 index 000000000000..07e4a5ecba62 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_1key_percent_dict.txt @@ -0,0 +1,29 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_1 +--influx_measurement=python_batch_sideinput_1 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":1000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=dict +--access_percentage=1 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt new file mode 100644 index 000000000000..b565598c1a16 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_99key_percent_dict.txt @@ -0,0 +1,29 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_2 +--influx_measurement=python_batch_sideinput_2 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":1000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=dict +--access_percentage=99 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt new file mode 100644 index 000000000000..de5b02198177 --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_first_list.txt @@ -0,0 +1,29 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_5 +--influx_measurement=python_batch_sideinput_5 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":1000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=list +--access_percentage=1 +--runner=DataflowRunner \ No newline at end of file diff --git a/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt new file mode 100644 index 000000000000..078ce2cc74fe --- /dev/null +++ b/.github/workflows/load-tests-job-configs/python_SideInput_Dataflow_Batch_1gb_1window_list.txt @@ -0,0 +1,28 @@ +# 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. + +--region=us-central1 +--temp_location=gs://temp-storage-for-perf-tests/loadtests +--publish_to_big_query=true +--metrics_dataset=load_test +--metrics_table=python_dataflow_batch_sideinput_6 +--influx_measurement=python_batch_sideinput_6 +--num_workers=10 +--autoscaling_algorithm=NONE +--experiments=use_runner_v2 +--input_options=''{\\"num_records\\":1000000,\\"key_size\\":100,\\"value_size\\":900,\\"algorithm\\":\\"lcg\\"}'' +--side_input_type=list +--runner=DataflowRunner \ No newline at end of file From 6bab4b579923c52a04723ab0a9c99874bc9980c4 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Wed, 11 Oct 2023 23:27:48 +0400 Subject: [PATCH 76/85] Add Community Metrics Prober workflow (#28842) --- .../beam_Prober_CommunityMetrics.yml | 77 +++++++++++++++++++ 1 file changed, 77 insertions(+) create mode 100644 .github/workflows/beam_Prober_CommunityMetrics.yml diff --git a/.github/workflows/beam_Prober_CommunityMetrics.yml b/.github/workflows/beam_Prober_CommunityMetrics.yml new file mode 100644 index 000000000000..5095f8d51737 --- /dev/null +++ b/.github/workflows/beam_Prober_CommunityMetrics.yml @@ -0,0 +1,77 @@ +# 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. + +name: Community Metrics Prober + +on: + issue_comment: + types: [created] + schedule: + - cron: '0 */6 * * *' + workflow_dispatch: + +#Setting explicit permissions for the action to avoid the default permissions which are `write-all` in case of pull_request_target event +permissions: + actions: write + pull-requests: read + checks: read + contents: read + deployments: read + id-token: none + issues: read + discussions: read + packages: read + pages: read + repository-projects: read + security-events: read + statuses: read + +# This allows a subsequently queued workflow run to interrupt previous runs +concurrency: + group: '${{ github.workflow }} @ ${{ github.event.issue.number || github.sha || github.head_ref || github.ref }}-${{ github.event.schedule || github.event.comment.body || github.event.sender.login }}' + cancel-in-progress: true + +env: + GRADLE_ENTERPRISE_ACCESS_KEY: ${{ secrets.GE_ACCESS_TOKEN }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_Prober_CommunityMetrics: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'schedule' || + github.event.comment.body == 'Run Community Metrics Prober' + runs-on: [self-hosted, ubuntu-20.04, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_Prober_CommunityMetrics"] + job_phrase: ["Run Community Metrics Prober"] + steps: + - uses: actions/checkout@v3 + - name: Setup repository + uses: ./.github/actions/setup-action + with: + comment_phrase: ${{ matrix.job_phrase }} + github_token: ${{ secrets.GITHUB_TOKEN }} + github_job: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + - name: Health check probes for the Community Metrics infrastructure + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :communityMetricsProber + arguments: | + --rerun-tasks \ No newline at end of file From 104c10b3ee536a9a3ea52b4dbf62d86b669da5d9 Mon Sep 17 00:00:00 2001 From: Damon Date: Wed, 11 Oct 2023 12:39:38 -0700 Subject: [PATCH 77/85] [RRIO] Create Caller and SetupTeardown interfaces (#28905) * Create test Caller and SetupTeardown interfaces * Update Javadoc * Defer Call transform to future PR * Rename package to requestresponseio * Add username to TODO --- sdks/java/io/rrio/build.gradle | 7 +- .../beam/io/requestresponseio/Caller.java | 27 ++++ .../io/requestresponseio/SetupTeardown.java | 34 +++++ .../UserCodeExecutionException.java | 38 +++++ .../UserCodeQuotaException.java | 42 ++++++ .../UserCodeTimeoutException.java | 39 ++++++ .../io/requestresponseio/package-info.java | 20 +++ .../org/apache/beam/io/rrio/CallerTest.java | 126 +++++++++++++++++ .../beam/io/rrio/SetupTeardownTest.java | 132 ++++++++++++++++++ 9 files changed, 462 insertions(+), 3 deletions(-) create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/Caller.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/SetupTeardown.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeExecutionException.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeQuotaException.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeTimeoutException.java create mode 100644 sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/package-info.java create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/CallerTest.java create mode 100644 sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/SetupTeardownTest.java diff --git a/sdks/java/io/rrio/build.gradle b/sdks/java/io/rrio/build.gradle index d65df370e0ca..32fbd9d22e38 100644 --- a/sdks/java/io/rrio/build.gradle +++ b/sdks/java/io/rrio/build.gradle @@ -25,9 +25,10 @@ description = "Apache Beam :: SDKS :: Java :: IO :: RequestResponseIO (RRIO)" ext.summary = "Support to read from and write to Web APIs" dependencies { - implementation project(path: ":sdks:java:core", configuration: "shadow") - implementation library.java.joda_time - implementation library.java.vendored_guava_32_1_2_jre + // TODO(damondouglas): revert to implementation after project is more fully developed + permitUnusedDeclared project(path: ":sdks:java:core", configuration: "shadow") + permitUnusedDeclared library.java.joda_time + permitUnusedDeclared library.java.vendored_guava_32_1_2_jre testImplementation project(path: ":sdks:java:core", configuration: "shadowTest") testImplementation library.java.junit diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/Caller.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/Caller.java new file mode 100644 index 000000000000..32b514c43a15 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/Caller.java @@ -0,0 +1,27 @@ +/* + * 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.io.requestresponseio; + +import java.io.Serializable; + +/** {@link Caller} interfaces user custom code intended for API calls. */ +public interface Caller extends Serializable { + + /** Calls a Web API with the {@link RequestT} and returns a {@link ResponseT}. */ + ResponseT call(RequestT request) throws UserCodeExecutionException; +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/SetupTeardown.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/SetupTeardown.java new file mode 100644 index 000000000000..2bdc8113d98e --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/SetupTeardown.java @@ -0,0 +1,34 @@ +/* + * 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.io.requestresponseio; + +import java.io.Serializable; + +/** + * Provided by user and called within {@link org.apache.beam.sdk.transforms.DoFn.Setup} and @{link + * org.apache.beam.sdk.transforms.DoFn.Teardown} lifecycle methods of {@link Call}'s {@link + * org.apache.beam.sdk.transforms.DoFn}. + */ +public interface SetupTeardown extends Serializable { + + /** Called during the {@link org.apache.beam.sdk.transforms.DoFn}'s setup lifecycle method. */ + void setup() throws UserCodeExecutionException; + + /** Called during the {@link org.apache.beam.sdk.transforms.DoFn}'s teardown lifecycle method. */ + void teardown() throws UserCodeExecutionException; +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeExecutionException.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeExecutionException.java new file mode 100644 index 000000000000..3a4c002f52e8 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeExecutionException.java @@ -0,0 +1,38 @@ +/* + * 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.io.requestresponseio; + +/** Base {@link Exception} for signaling errors in user custom code. */ +public class UserCodeExecutionException extends Exception { + public UserCodeExecutionException(String message) { + super(message); + } + + public UserCodeExecutionException(String message, Throwable cause) { + super(message, cause); + } + + public UserCodeExecutionException(Throwable cause) { + super(cause); + } + + public UserCodeExecutionException( + String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeQuotaException.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeQuotaException.java new file mode 100644 index 000000000000..f16f078927f8 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeQuotaException.java @@ -0,0 +1,42 @@ +/* + * 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.io.requestresponseio; + +/** + * Extends {@link UserCodeQuotaException} to allow the user custom code to specifically signal a + * Quota or API overuse related error. + */ +public class UserCodeQuotaException extends UserCodeExecutionException { + + public UserCodeQuotaException(String message) { + super(message); + } + + public UserCodeQuotaException(String message, Throwable cause) { + super(message, cause); + } + + public UserCodeQuotaException(Throwable cause) { + super(cause); + } + + public UserCodeQuotaException( + String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeTimeoutException.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeTimeoutException.java new file mode 100644 index 000000000000..22b067449858 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/UserCodeTimeoutException.java @@ -0,0 +1,39 @@ +/* + * 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.io.requestresponseio; + +/** An extension of {@link UserCodeQuotaException} to specifically signal a user code timeout. */ +public class UserCodeTimeoutException extends UserCodeExecutionException { + + public UserCodeTimeoutException(String message) { + super(message); + } + + public UserCodeTimeoutException(String message, Throwable cause) { + super(message, cause); + } + + public UserCodeTimeoutException(Throwable cause) { + super(cause); + } + + public UserCodeTimeoutException( + String message, Throwable cause, boolean enableSuppression, boolean writableStackTrace) { + super(message, cause, enableSuppression, writableStackTrace); + } +} diff --git a/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/package-info.java b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/package-info.java new file mode 100644 index 000000000000..cd9c11c13f86 --- /dev/null +++ b/sdks/java/io/rrio/src/main/java/org/apache/beam/io/requestresponseio/package-info.java @@ -0,0 +1,20 @@ +/* + * 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 provides Beam I/O transform support for safely reading from and writing to Web APIs. */ +package org.apache.beam.io.requestresponseio; diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/CallerTest.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/CallerTest.java new file mode 100644 index 000000000000..5258573f4283 --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/CallerTest.java @@ -0,0 +1,126 @@ +/* + * 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.io.rrio; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +import org.apache.beam.io.requestresponseio.Caller; +import org.apache.beam.io.requestresponseio.UserCodeExecutionException; +import org.apache.beam.io.requestresponseio.UserCodeQuotaException; +import org.apache.beam.io.requestresponseio.UserCodeTimeoutException; +import org.apache.beam.sdk.Pipeline.PipelineExecutionException; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.SerializableUtils; +import org.joda.time.Instant; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +/** Tests for {@link Caller}. */ +@RunWith(JUnit4.class) +public class CallerTest { + + @Rule public TestPipeline pipeline = TestPipeline.create(); + + @Test + public void canSerializeImplementingClasses() { + SerializableUtils.serializeToByteArray(new CallerImpl()); + } + + @Test + public void canSerializeWhenUsedInDoFn() { + pipeline + .apply(Create.of(Instant.now())) + .apply(ParDo.of(new CallerUsingDoFn<>(new CallerImpl()))) + .setCoder(StringUtf8Coder.of()); + + pipeline.run(); + } + + @Test + public void canSignalQuotaException() { + pipeline + .apply(Create.of(1)) + .apply(ParDo.of(new CallerUsingDoFn<>(new CallerThrowsQuotaException()))) + .setCoder(VarIntCoder.of()); + + PipelineExecutionException executionException = + assertThrows(PipelineExecutionException.class, pipeline::run); + assertEquals(UserCodeQuotaException.class, executionException.getCause().getClass()); + } + + @Test + public void canSignalTimeoutException() { + pipeline + .apply(Create.of(1)) + .apply(ParDo.of(new CallerUsingDoFn<>(new CallerThrowsTimeoutException()))) + .setCoder(VarIntCoder.of()); + + PipelineExecutionException executionException = + assertThrows(PipelineExecutionException.class, pipeline::run); + assertEquals(UserCodeTimeoutException.class, executionException.getCause().getClass()); + } + + private static class CallerUsingDoFn extends DoFn { + private final Caller caller; + + private CallerUsingDoFn(Caller caller) { + this.caller = caller; + } + + @ProcessElement + public void process(@Element RequestT request, OutputReceiver receiver) + throws UserCodeExecutionException { + RequestT safeRequest = checkStateNotNull(request); + ResponseT response = caller.call(safeRequest); + receiver.output(response); + } + } + + private static class CallerImpl implements Caller { + + @Override + public String call(Instant request) throws UserCodeExecutionException { + return request.toString(); + } + } + + private static class CallerThrowsQuotaException implements Caller { + + @Override + public Integer call(Integer request) throws UserCodeExecutionException { + throw new UserCodeQuotaException("quota"); + } + } + + private static class CallerThrowsTimeoutException implements Caller { + + @Override + public Integer call(Integer request) throws UserCodeExecutionException { + throw new UserCodeTimeoutException("timeout"); + } + } +} diff --git a/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/SetupTeardownTest.java b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/SetupTeardownTest.java new file mode 100644 index 000000000000..a8c5c45ede5c --- /dev/null +++ b/sdks/java/io/rrio/src/test/java/org/apache/beam/io/rrio/SetupTeardownTest.java @@ -0,0 +1,132 @@ +/* + * 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.io.rrio; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertThrows; + +import org.apache.beam.io.requestresponseio.SetupTeardown; +import org.apache.beam.io.requestresponseio.UserCodeExecutionException; +import org.apache.beam.io.requestresponseio.UserCodeQuotaException; +import org.apache.beam.io.requestresponseio.UserCodeTimeoutException; +import org.apache.beam.sdk.coders.VarIntCoder; +import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.sdk.transforms.Create; +import org.apache.beam.sdk.transforms.DoFn; +import org.apache.beam.sdk.transforms.ParDo; +import org.apache.beam.sdk.util.SerializableUtils; +import org.apache.beam.sdk.util.UserCodeException; +import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.UncheckedExecutionException; +import org.junit.Rule; +import org.junit.Test; + +public class SetupTeardownTest { + @Rule public TestPipeline pipeline = TestPipeline.create(); + + @Test + public void canSerializeImplementingClasses() { + SerializableUtils.serializeToByteArray(new SetupTeardownImpl()); + } + + @Test + public void canSerializeWhenUsedInDoFn() { + pipeline + .apply(Create.of(1)) + .apply(ParDo.of(new SetupTeardownUsingDoFn(new SetupTeardownImpl()))) + .setCoder(VarIntCoder.of()); + + pipeline.run(); + } + + @Test + public void canSignalQuotaException() { + pipeline + .apply(Create.of(1)) + .apply(ParDo.of(new SetupTeardownUsingDoFn(new ThrowsQuotaException()))) + .setCoder(VarIntCoder.of()); + + UncheckedExecutionException exception = + assertThrows(UncheckedExecutionException.class, pipeline::run); + UserCodeException userCodeException = (UserCodeException) exception.getCause(); + assertEquals(UserCodeQuotaException.class, userCodeException.getCause().getClass()); + } + + @Test + public void canSignalTimeoutException() { + pipeline + .apply(Create.of(1)) + .apply(ParDo.of(new SetupTeardownUsingDoFn(new ThrowsTimeoutException()))) + .setCoder(VarIntCoder.of()); + + UncheckedExecutionException exception = + assertThrows(UncheckedExecutionException.class, pipeline::run); + UserCodeException userCodeException = (UserCodeException) exception.getCause(); + assertEquals(UserCodeTimeoutException.class, userCodeException.getCause().getClass()); + } + + private static class SetupTeardownUsingDoFn extends DoFn { + private final SetupTeardown setupTeardown; + + private SetupTeardownUsingDoFn(SetupTeardown setupTeardown) { + this.setupTeardown = setupTeardown; + } + + @Setup + public void setup() throws UserCodeExecutionException { + setupTeardown.setup(); + } + + @Teardown + public void teardown() throws UserCodeExecutionException { + setupTeardown.teardown(); + } + + @ProcessElement + public void process() {} + } + + private static class SetupTeardownImpl implements SetupTeardown { + @Override + public void setup() throws UserCodeExecutionException {} + + @Override + public void teardown() throws UserCodeExecutionException {} + } + + private static class ThrowsQuotaException implements SetupTeardown { + + @Override + public void setup() throws UserCodeExecutionException { + throw new UserCodeQuotaException("quota"); + } + + @Override + public void teardown() throws UserCodeExecutionException {} + } + + private static class ThrowsTimeoutException implements SetupTeardown { + + @Override + public void setup() throws UserCodeExecutionException { + throw new UserCodeTimeoutException("timeout"); + } + + @Override + public void teardown() throws UserCodeExecutionException {} + } +} From e8e38145d96a7b0561a418c19519917b0567591e Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 11 Oct 2023 21:16:13 -0400 Subject: [PATCH 78/85] Setup Java 21 container (#28833) * Add Java21 container * fix spotless * Update website * Fix jamm * align configs in recent change --- ...a_CoGBK_Dataflow_V2_Batch_JavaVersions.yml | 8 ++-- ...GBK_Dataflow_V2_Streaming_JavaVersions.yml | 8 ++-- ...ests_Java_GBK_Dataflow_V2_Batch_Java11.yml | 14 +++---- ...ests_Java_GBK_Dataflow_V2_Batch_Java17.yml | 14 +++---- ..._Java_GBK_Dataflow_V2_Streaming_Java11.yml | 14 +++---- ..._Java_GBK_Dataflow_V2_Streaming_Java17.yml | 14 +++---- ..._PostCommit_Java_Examples_Dataflow_ARM.yml | 2 +- ...tCommit_Java_Examples_Dataflow_V2_Java.yml | 2 +- ...m_PostCommit_Java_Jpms_Dataflow_Java17.yml | 2 +- ...eam_PostCommit_Java_Jpms_Direct_Java17.yml | 2 +- ...stCommit_Java_Nexmark_Dataflow_V2_Java.yml | 2 +- ...eam_PostCommit_TransformService_Direct.yml | 2 +- ...reCommit_Java_Examples_Dataflow_Java11.yml | 2 +- ...reCommit_Java_Examples_Dataflow_Java17.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java11.yml | 2 +- .../workflows/beam_PreCommit_SQL_Java17.yml | 2 +- .test-infra/jenkins/JavaTestProperties.groovy | 7 +++- .test-infra/jenkins/NexmarkBuilder.groovy | 4 +- ..._LoadTests_CoGBK_Dataflow_V2_Java11.groovy | 2 +- ..._LoadTests_CoGBK_Dataflow_V2_Java17.groovy | 2 +- ...ob_LoadTests_GBK_Dataflow_V2_Java11.groovy | 2 +- ...ob_LoadTests_GBK_Dataflow_V2_Java17.groovy | 2 +- ..._LoadTests_ParDo_Dataflow_V2_Java11.groovy | 2 +- ..._LoadTests_ParDo_Dataflow_V2_Java17.groovy | 2 +- ...it_Java_Examples_Dataflow_V2_Java11.groovy | 2 +- ...it_Java_Examples_Dataflow_V2_Java17.groovy | 2 +- ...ostCommit_Java_Jpms_Dataflow_Java17.groovy | 2 +- ..._PostCommit_Java_Jpms_Direct_Java17.groovy | 2 +- ..._PostCommit_TransformService_Direct.groovy | 2 +- ...ommit_Java_Examples_Dataflow_Java11.groovy | 2 +- .../jenkins/job_PreCommit_SQL_Java11.groovy | 2 +- .../jenkins/job_PreCommit_SQL_Java17.groovy | 2 +- ...ommit_Java_Examples_Dataflow_Java17.groovy | 2 +- build.gradle.kts | 14 +++---- .../beam/gradle/BeamModulePlugin.groovy | 37 ++++++++++++++++++- .../core/construction/Environments.java | 6 ++- .../core/construction/EnvironmentsTest.java | 6 ++- .../arm/build.gradle | 6 +-- .../google-cloud-dataflow-java/build.gradle | 6 +-- .../beam/runners/dataflow/DataflowRunner.java | 7 ++-- runners/spark/spark_runner.gradle | 5 ++- sdks/java/container/Dockerfile | 3 +- sdks/java/container/common.gradle | 14 ++++--- sdks/java/container/java21/build.gradle | 30 +++++++++++++++ sdks/java/container/java21/option-jamm.json | 12 ++++++ sdks/java/testing/jpms-tests/build.gradle | 21 +++++------ sdks/java/testing/test-utils/build.gradle | 29 +++++---------- .../jvmverification/JvmVerification.java | 16 +++++++- settings.gradle.kts | 1 + .../www/site/content/en/roadmap/java-sdk.md | 6 +-- 50 files changed, 220 insertions(+), 132 deletions(-) create mode 100644 sdks/java/container/java21/build.gradle create mode 100644 sdks/java/container/java21/option-jamm.json diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml index 35575aec0bef..ff172cc338b7 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Batch_JavaVersions.yml @@ -95,7 +95,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -105,7 +105,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -115,7 +115,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -125,7 +125,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ diff --git a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml index 46dfaa986920..efa61373f2ab 100644 --- a/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml +++ b/.github/workflows/beam_LoadTests_Java_CoGBK_Dataflow_V2_Streaming_JavaVersions.yml @@ -95,7 +95,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -105,7 +105,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -115,7 +115,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ @@ -125,7 +125,7 @@ jobs: with: gradle-command: :sdks:java:testing:load-tests:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion=${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -PloadTest.mainClass=org.apache.beam.sdk.loadtests.CoGroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml index 318f157e57a6..5c333013e2a2 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11.yml @@ -93,7 +93,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -104,7 +104,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -115,7 +115,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -126,7 +126,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -137,7 +137,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -148,7 +148,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -159,6 +159,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java11_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml index 543cb86985d8..4fa5c364728b 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17.yml @@ -95,7 +95,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -106,7 +106,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -117,7 +117,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -128,7 +128,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -139,7 +139,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -150,7 +150,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -161,6 +161,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Batch_Java17_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml index fcd55f761c04..6432a315d8a1 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11.yml @@ -93,7 +93,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -104,7 +104,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -115,7 +115,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -126,7 +126,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -137,7 +137,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -148,7 +148,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -159,6 +159,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java11_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml index 5df7d3249192..cd8c34279593 100644 --- a/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml +++ b/.github/workflows/beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17.yml @@ -95,7 +95,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_1 }}' \ - name: run Load test 2GB of 100B records @@ -106,7 +106,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_2 }}' \ - name: run Load test 2GB of 100kB records @@ -117,7 +117,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_3 }}' \ - name: run Load test fanout 4 times with 2GB 10-byte records total @@ -128,7 +128,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_4 }}' \ - name: run Load test fanout 8 times with 2GB 10-byte records total @@ -139,7 +139,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_5 }}' \ - name: run Load test reiterate 4 times 10kB values @@ -150,7 +150,7 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_6 }}' \ - name: run Load test reiterate 4 times 2MB values @@ -161,6 +161,6 @@ jobs: -PloadTest.mainClass=org.apache.beam.sdk.loadtests.GroupByKeyLoadTest \ -Prunner=:runners:google-cloud-dataflow-java \ -Prunner.version=V2 \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ '-PloadTest.args=${{ env.beam_LoadTests_Java_GBK_Dataflow_V2_Streaming_Java17_test_arguments_7 }}' \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml index 939d2646d352..8add1a2445c8 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_ARM.yml @@ -111,7 +111,7 @@ jobs: gradle-command: :runners:google-cloud-dataflow-java:arm:examplesJavaRunnerV2IntegrationTestARM max-workers: 12 arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -Pcontainer-architecture-list=arm64,amd64 \ -Ppush-containers \ diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml index 74e1787945e3..ab862df1847a 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2_Java.yml @@ -86,7 +86,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ -PskipCheckerFramework \ - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml index 0b9a366211ee..3f6352b0a948 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Dataflow_Java17.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sdks:java:testing:jpms-tests:dataflowRunnerIntegrationTest arguments: -PskipCheckerFramework - -PcompileAndRunTestsWithJava17 + -PtestJavaVersion=17 -Pjava17Home=$JAVA_HOME_17_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml index fcdbd63f7e97..360b9c757a02 100644 --- a/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml +++ b/.github/workflows/beam_PostCommit_Java_Jpms_Direct_Java17.yml @@ -80,7 +80,7 @@ jobs: gradle-command: :sdks:java:testing:jpms-tests:directRunnerIntegrationTest arguments: -PskipCheckerFramework - -PcompileAndRunTestsWithJava17 + -PtestJavaVersion=17 -Pjava17Home=$JAVA_HOME_17_X64 - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml index d78793172898..5782928a8b4d 100644 --- a/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml +++ b/.github/workflows/beam_PostCommit_Java_Nexmark_Dataflow_V2_Java.yml @@ -107,7 +107,7 @@ jobs: with: gradle-command: :sdks:java:testing:nexmark:run arguments: | - -PcompileAndRunTestsWithJava${{ matrix.java_version }} \ + -PtestJavaVersion${{ matrix.java_version }} \ -Pjava${{ matrix.java_version }}Home=$JAVA_HOME_${{ matrix.java_version }}_X64 \ -Pnexmark.runner.version=V2 \ -Pnexmark.runner=:runners:google-cloud-dataflow-java \ diff --git a/.github/workflows/beam_PostCommit_TransformService_Direct.yml b/.github/workflows/beam_PostCommit_TransformService_Direct.yml index 491688e097a0..2ba83506d395 100644 --- a/.github/workflows/beam_PostCommit_TransformService_Direct.yml +++ b/.github/workflows/beam_PostCommit_TransformService_Direct.yml @@ -84,7 +84,7 @@ jobs: with: gradle-command: :sdks:python:test-suites:direct:xlang:transformServicePythonUsingJava arguments: | - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ -PuseWheelDistribution \ -PpythonVersion=${{ matrix.python_version }} \ diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml index 03f4d32861b1..186d75674c71 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java11.yml @@ -115,7 +115,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ -PskipCheckerFramework \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -Pjava11Home=$JAVA_HOME_11_X64 \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v3 diff --git a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml index cf8cefe388f3..cafc8cf5ab6e 100644 --- a/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml +++ b/.github/workflows/beam_PreCommit_Java_Examples_Dataflow_Java17.yml @@ -115,7 +115,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -PskipCheckerFramework \ -Pjava17Home=$JAVA_HOME_17_X64 \ max-workers: 12 diff --git a/.github/workflows/beam_PreCommit_SQL_Java11.yml b/.github/workflows/beam_PreCommit_SQL_Java11.yml index f1c733418b8d..3412365bf13e 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java11.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java11.yml @@ -103,7 +103,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - -PcompileAndRunTestsWithJava11 \ + -PtestJavaVersion=11 \ -PskipCheckerFramework \ -Pjava11Home=$JAVA_HOME_11_X64 \ - name: Archive JUnit Test Results diff --git a/.github/workflows/beam_PreCommit_SQL_Java17.yml b/.github/workflows/beam_PreCommit_SQL_Java17.yml index 7547bd396815..2215bab8ddeb 100644 --- a/.github/workflows/beam_PreCommit_SQL_Java17.yml +++ b/.github/workflows/beam_PreCommit_SQL_Java17.yml @@ -101,7 +101,7 @@ jobs: arguments: | -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ - -PcompileAndRunTestsWithJava17 \ + -PtestJavaVersion=17 \ -PskipCheckerFramework \ -Pjava17Home=$JAVA_HOME_17_X64 \ - name: Archive JUnit Test Results diff --git a/.test-infra/jenkins/JavaTestProperties.groovy b/.test-infra/jenkins/JavaTestProperties.groovy index ce7446a6e71d..5403cee5cf9a 100644 --- a/.test-infra/jenkins/JavaTestProperties.groovy +++ b/.test-infra/jenkins/JavaTestProperties.groovy @@ -17,5 +17,10 @@ */ class JavaTestProperties { - final static List SUPPORTED_CONTAINER_TASKS = ['java8', 'java11', 'java17'] + final static List SUPPORTED_CONTAINER_TASKS = [ + 'java8', + 'java11', + 'java17', + 'java21' + ] } diff --git a/.test-infra/jenkins/NexmarkBuilder.groovy b/.test-infra/jenkins/NexmarkBuilder.groovy index 044b0cbb9561..69fa3dcc4277 100644 --- a/.test-infra/jenkins/NexmarkBuilder.groovy +++ b/.test-infra/jenkins/NexmarkBuilder.groovy @@ -145,7 +145,7 @@ class NexmarkBuilder { rootBuildScriptDir(commonJobProperties.checkoutDir) tasks(':sdks:java:testing:nexmark:run') commonJobProperties.setGradleSwitches(delegate) - switches("-PcompileAndRunTestsWithJava11") + switches("-PtestJavaVersion=11") switches("-Pjava11Home=${commonJobProperties.JAVA_11_HOME}") switches("-Pnexmark.runner=${runner.getDependencyBySDK(sdk)}") switches("-Pnexmark.args=\"${parseOptions(options)}\"") @@ -168,7 +168,7 @@ class NexmarkBuilder { rootBuildScriptDir(commonJobProperties.checkoutDir) tasks(':sdks:java:testing:nexmark:run') commonJobProperties.setGradleSwitches(delegate) - switches("-PcompileAndRunTestsWithJava17") + switches("-PtestJavaVersion=17") switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") switches("-Pnexmark.runner=${runner.getDependencyBySDK(sdk)}") switches("-Pnexmark.args=\"${parseOptions(options)}\"") diff --git a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy index fc7f39d28a0d..55501db4429d 100644 --- a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java11.groovy @@ -188,7 +188,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava11', + '-PtestJavaVersion=11', "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy index ca8c6689ad0f..8fb09fd07448 100644 --- a/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_LoadTests_CoGBK_Dataflow_V2_Java17.groovy @@ -188,7 +188,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava17', + '-PtestJavaVersion=17', "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy index cc2d5d2e5554..2191e448fade 100644 --- a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java11.groovy @@ -253,7 +253,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava11', + '-PtestJavaVersion=11', "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy index 7405f9154b83..2520f68f0178 100644 --- a/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_LoadTests_GBK_Dataflow_V2_Java17.groovy @@ -253,7 +253,7 @@ def loadTestConfigurations = { mode, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava17', + '-PtestJavaVersion=17', "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy index a4535d52e6cf..b7154e840e6a 100644 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java11.groovy @@ -160,7 +160,7 @@ def commonLoadTestConfig = { jobType, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava11', + '-PtestJavaVersion=11', "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" ] diff --git a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy index f0f2179ebb3b..df6c66e02aed 100644 --- a/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_LoadTests_ParDo_Dataflow_V2_Java17.groovy @@ -160,7 +160,7 @@ def commonLoadTestConfig = { jobType, isStreaming -> def final JOB_SPECIFIC_SWITCHES = [ '-Prunner.version="V2"', - '-PcompileAndRunTestsWithJava17', + '-PtestJavaVersion=17', "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" ] diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy index 6687ae0e6f8a..6229f7c48a72 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java11.groovy @@ -43,7 +43,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_V2_ja commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' - switches '-PcompileAndRunTestsWithJava11' + switches '-PtestJavaVersion=11' switches '-PskipCheckerFramework' switches "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy index b275fe9276d9..7e52a7e09789 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Examples_Dataflow_V2_Java17.groovy @@ -43,7 +43,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Examples_Dataflow_V2_ja commonJobProperties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' - switches '-PcompileAndRunTestsWithJava17' + switches '-PtestJavaVersion=17' switches '-PskipCheckerFramework' switches "-Pjava17Home=${commonJobProperties.JAVA_17_HOME}" } diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy index 4e26c164319e..f518985ca7a8 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Dataflow_Java17.groovy @@ -42,7 +42,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Dataflow_Java17', tasks(':sdks:java:testing:jpms-tests:dataflowRunnerIntegrationTest') commonJobProperties.setGradleSwitches(delegate) switches("-PskipCheckerFramework") - switches("-PcompileAndRunTestsWithJava17") + switches("-PtestJavaVersion=17") switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") // Specify maven home on Jenkins, needed by Maven archetype integration tests. switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') diff --git a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy index f31373ecaada..04c31389ecbe 100644 --- a/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy +++ b/.test-infra/jenkins/job_PostCommit_Java_Jpms_Direct_Java17.groovy @@ -42,7 +42,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_Java_Jpms_Direct_Java17', 'R tasks(':sdks:java:testing:jpms-tests:directRunnerIntegrationTest') commonJobProperties.setGradleSwitches(delegate) switches("-PskipCheckerFramework") - switches("-PcompileAndRunTestsWithJava17") + switches("-PtestJavaVersion=17") switches("-Pjava17Home=${commonJobProperties.JAVA_17_HOME}") // Specify maven home on Jenkins, needed by Maven archetype integration tests. switches('-Pmaven_home=/home/jenkins/tools/maven/apache-maven-3.5.4') diff --git a/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy b/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy index 0d7f58e71706..03d29069a52c 100644 --- a/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy +++ b/.test-infra/jenkins/job_PostCommit_TransformService_Direct.groovy @@ -43,7 +43,7 @@ PostcommitJobBuilder.postCommitJob('beam_PostCommit_TransformService_Direct', rootBuildScriptDir(commonJobProperties.checkoutDir) tasks(':sdks:python:test-suites:direct:xlang:transformServicePythonUsingJava') commonJobProperties.setGradleSwitches(delegate) - switches '-PcompileAndRunTestsWithJava11' + switches '-PtestJavaVersion=11' switches "-Pjava11Home=${commonJobProperties.JAVA_11_HOME}" switches("-PuseWheelDistribution") switches("-PpythonVersion=${pythonVersion}") diff --git a/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy b/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy index cabf5dbdd457..9f733a42ccce 100644 --- a/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy +++ b/.test-infra/jenkins/job_PreCommit_Java_Examples_Dataflow_Java11.groovy @@ -50,7 +50,7 @@ builder.build { switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' switches '-PskipCheckerFramework' // Gradle itself is running under JDK8 so plugin configures wrong for JDK11 - switches '-PcompileAndRunTestsWithJava11' + switches '-PtestJavaVersion=11' switches "-Pjava11Home=${properties.JAVA_11_HOME}" properties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) } diff --git a/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy b/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy index 9742ab756cf7..34154b37209e 100644 --- a/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy +++ b/.test-infra/jenkins/job_PreCommit_SQL_Java11.groovy @@ -26,7 +26,7 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( gradleSwitches: [ '-PdisableSpotlessCheck=true', '-PdisableCheckStyle=true', - '-PcompileAndRunTestsWithJava11', + '-PtestJavaVersion=11', '-PskipCheckerFramework', // Gradle itself is running under JDK8 so plugin configures wrong for JDK11 "-Pjava11Home=${properties.JAVA_11_HOME}" diff --git a/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy b/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy index 158fa683c1a8..472521cb1b08 100644 --- a/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy +++ b/.test-infra/jenkins/job_PreCommit_SQL_Java17.groovy @@ -26,7 +26,7 @@ PrecommitJobBuilder builder = new PrecommitJobBuilder( gradleSwitches: [ '-PdisableSpotlessCheck=true', '-PdisableCheckStyle=true', - '-PcompileAndRunTestsWithJava17', + '-PtestJavaVersion=17', '-PskipCheckerFramework', // Gradle itself is running under JDK8 so plugin configures wrong for JDK17 "-Pjava17Home=${properties.JAVA_17_HOME}" diff --git a/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy b/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy index 3654a4c75edb..387b707123e7 100644 --- a/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy +++ b/.test-infra/jenkins/job_Precommit_Java_Examples_Dataflow_Java17.groovy @@ -50,7 +50,7 @@ builder.build { switches '-PdisableSpotlessCheck=true' switches '-PdisableCheckStyle=true' switches '-PskipCheckerFramework' // Gradle itself is running under JDK8 so plugin configures wrong for JDK17 - switches '-PcompileAndRunTestsWithJava17' + switches '-PtestJavaVersion=17' switches "-Pjava17Home=${properties.JAVA_17_HOME}" properties.setGradleSwitches(delegate, 3 * Runtime.runtime.availableProcessors()) } diff --git a/build.gradle.kts b/build.gradle.kts index ea1b4e6784e3..5bcfbb3ed06e 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -712,14 +712,12 @@ if (project.hasProperty("javaLinkageArtifactIds")) { } } } -if (project.hasProperty("compileAndRunTestsWithJava11")) { - tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion") - tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion") - tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion") -} else if (project.hasProperty("compileAndRunTestsWithJava17")) { - tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion17") - tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion17") - tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion17") +if (project.hasProperty("testJavaVersion")) { + var testVer = project.property("testJavaVersion") + + tasks.getByName("javaPreCommitPortabilityApi").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") + tasks.getByName("javaExamplesDataflowPrecommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") + tasks.getByName("sqlPreCommit").dependsOn(":sdks:java:testing:test-utils:verifyJavaVersion$testVer") } else { allprojects { tasks.withType(Test::class).configureEach { 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 19ed6aaa4276..c32717aae725 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -946,6 +946,29 @@ class BeamModulePlugin implements Plugin { ] } + project.ext.setJava21Options = { CompileOptions options -> + def java17Home = project.findProperty("java17Home") + options.fork = true + options.forkOptions.javaHome = java17Home as File + options.compilerArgs += ['-Xlint:-path'] + // Error prone requires some packages to be exported/opened for Java 17 + // Disabling checks since this property is only used for Jenkins tests + // https://github.com/tbroyer/gradle-errorprone-plugin#jdk-16-support + options.errorprone.errorproneArgs.add("-XepDisableAllChecks") + options.forkOptions.jvmArgs += [ + "-J--add-exports=jdk.compiler/com.sun.tools.javac.api=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.file=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.main=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.model=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.parser=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.processing=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.tree=ALL-UNNAMED", + "-J--add-exports=jdk.compiler/com.sun.tools.javac.util=ALL-UNNAMED", + "-J--add-opens=jdk.compiler/com.sun.tools.javac.code=ALL-UNNAMED", + "-J--add-opens=jdk.compiler/com.sun.tools.javac.comp=ALL-UNNAMED" + ] + } + project.ext.repositories = { maven { name "testPublicationLocal" @@ -1492,7 +1515,7 @@ class BeamModulePlugin implements Plugin { options.errorprone.errorproneArgs.add("-Xep:Slf4jLoggerShouldBeNonStatic:OFF") } - if (project.hasProperty("compileAndRunTestsWithJava11")) { + if (project.findProperty('testJavaVersion') == "11") { def java11Home = project.findProperty("java11Home") project.tasks.compileTestJava { options.fork = true @@ -1504,7 +1527,7 @@ class BeamModulePlugin implements Plugin { useJUnit() executable = "${java11Home}/bin/java" } - } else if (project.hasProperty("compileAndRunTestsWithJava17")) { + } else if (project.findProperty('testJavaVersion') == "17") { def java17Home = project.findProperty("java17Home") project.tasks.compileTestJava { setCompileAndRuntimeJavaVersion(options.compilerArgs, '17') @@ -1514,6 +1537,16 @@ class BeamModulePlugin implements Plugin { useJUnit() executable = "${java17Home}/bin/java" } + } else if (project.findProperty('testJavaVersion') == "21") { + def java21Home = project.findProperty("java21Home") + project.tasks.compileTestJava { + setCompileAndRuntimeJavaVersion(options.compilerArgs, '21') + project.ext.setJava17Options(options) + } + project.tasks.withType(Test).configureEach { + useJUnit() + executable = "${java21Home}/bin/java" + } } if (configuration.shadowClosure) { diff --git a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java index 31a555989afd..f531b5be344d 100644 --- a/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java +++ b/runners/core-construction-java/src/main/java/org/apache/beam/runners/core/construction/Environments.java @@ -94,7 +94,8 @@ public class Environments { public enum JavaVersion { java8("java", "1.8", 8), java11("java11", "11", 11), - java17("java17", "17", 17); + java17("java17", "17", 17), + java21("java21", "21", 21); // Legacy name, as used in container image private final String legacyName; @@ -119,6 +120,7 @@ public String specification() { return this.specification; } + /** Return the LTS java version given the Java specification version. */ public static JavaVersion forSpecification(String specification) { for (JavaVersion ver : JavaVersion.values()) { if (ver.specification.equals(specification)) { @@ -137,7 +139,7 @@ public static JavaVersion forSpecification(String specification) { } } LOG.warn( - "unsupported Java version: {}, falling back to: {}", + "Unsupported Java version: {}, falling back to: {}", specification, fallback.specification); return fallback; diff --git a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java index ae429fb1fe6d..b71a654f1031 100644 --- a/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java +++ b/runners/core-construction-java/src/test/java/org/apache/beam/runners/core/construction/EnvironmentsTest.java @@ -291,6 +291,8 @@ public void testLtsJavaVersion() { assertEquals("java11", JavaVersion.java11.legacyName()); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("17")); assertEquals("java17", JavaVersion.java17.legacyName()); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("21")); + assertEquals("java21", JavaVersion.java21.legacyName()); } @Test @@ -303,7 +305,9 @@ public void testNonLtsJavaVersion() { assertEquals(JavaVersion.java17, JavaVersion.forSpecification("15")); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("16")); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("18")); - assertEquals(JavaVersion.java17, JavaVersion.forSpecification("19")); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("19")); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("20")); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("21")); } @Test(expected = UnsupportedOperationException.class) diff --git a/runners/google-cloud-dataflow-java/arm/build.gradle b/runners/google-cloud-dataflow-java/arm/build.gradle index e79eeedcd828..71cbc7c58e86 100644 --- a/runners/google-cloud-dataflow-java/arm/build.gradle +++ b/runners/google-cloud-dataflow-java/arm/build.gradle @@ -76,10 +76,8 @@ dependencies { } def javaVer = "java8" -if(project.hasProperty('compileAndRunTestsWithJava17')) { - javaVer = "java17" -} else if(project.hasProperty('compileAndRunTestsWithJava11')) { - javaVer = "java11" +if (project.hasProperty('testJavaVersion')) { + javaVer = "java${project.getProperty('testJavaVersion')}" } def dataflowProject = project.findProperty('dataflowProject') ?: 'apache-beam-testing' def dataflowRegion = project.findProperty('dataflowRegion') ?: 'us-central1' diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index adc1f2e09bc4..e4f34687d31c 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -279,10 +279,8 @@ def createRunnerV2ValidatesRunnerTest = { Map args -> // task ordering such that the registry doesn't get cleaned up prior to task completion. def buildAndPushDockerJavaContainer = tasks.register("buildAndPushDockerJavaContainer") { def javaVer = "java8" - if(project.hasProperty('compileAndRunTestsWithJava17')) { - javaVer = "java17" - } else if(project.hasProperty('compileAndRunTestsWithJava11')) { - javaVer = "java11" + if(project.hasProperty('testJavaVersion')) { + javaVer = "java${project.getProperty('testJavaVersion')}" } dependsOn ":sdks:java:container:${javaVer}:docker" def defaultDockerImageName = containerImageName( diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java index 26548038a1df..891b4c0454c9 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowRunner.java @@ -398,10 +398,9 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { // Adding the Java version to the SDK name for user's and support convenience. String agentJavaVer = "(JRE 8 environment)"; - if (Environments.getJavaVersion() == Environments.JavaVersion.java17) { - agentJavaVer = "(JRE 17 environment)"; - } else if (Environments.getJavaVersion() == Environments.JavaVersion.java11) { - agentJavaVer = "(JRE 11 environment)"; + if (Environments.getJavaVersion() != Environments.JavaVersion.java8) { + agentJavaVer = + String.format("(JRE %s environment)", Environments.getJavaVersion().specification()); } DataflowRunnerInfo dataflowRunnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); diff --git a/runners/spark/spark_runner.gradle b/runners/spark/spark_runner.gradle index d0dbe453ddfb..74013de6107d 100644 --- a/runners/spark/spark_runner.gradle +++ b/runners/spark/spark_runner.gradle @@ -63,8 +63,9 @@ def sparkTestProperties(overrides = [:]) { def sparkTestJvmArgs() { - // run tests with Java 17 using -PcompileAndRunTestsWithJava17 -Pjava17Home=??? - if (project.hasProperty("compileAndRunTestsWithJava17")) { + // run tests with Java 17 using -PtestJavaVersion=17 -Pjava17Home=??? + if (project.hasProperty('testJavaVersion') && + project.getProperty('testJavaVersion') in ['17', '21']) { return [ "--add-opens=java.base/sun.nio.ch=ALL-UNNAMED", // add-opens below required for Kryo FieldSerializer / SparkRunnerKryoRegistratorTest diff --git a/sdks/java/container/Dockerfile b/sdks/java/container/Dockerfile index e0fa8d4a0a6f..9c266ea132b8 100644 --- a/sdks/java/container/Dockerfile +++ b/sdks/java/container/Dockerfile @@ -15,8 +15,9 @@ # See the License for the specific language governing permissions and # limitations under the License. ############################################################################### +ARG base_image ARG java_version -FROM eclipse-temurin:${java_version} +FROM ${base_image}:${java_version} LABEL Author "Apache Beam " ARG TARGETOS ARG TARGETARCH diff --git a/sdks/java/container/common.gradle b/sdks/java/container/common.gradle index cc427494ed6e..9f6cbe64a294 100644 --- a/sdks/java/container/common.gradle +++ b/sdks/java/container/common.gradle @@ -29,6 +29,7 @@ applyDockerNature() if (!project.hasProperty('imageJavaVersion')) { throw new GradleException('imageJavaVersion project property must be set') } +def javaBaseImage = project.findProperty('javaBaseImage') ?: 'eclipse-temurin' def imageJavaVersion = project.findProperty('imageJavaVersion') description = "Apache Beam :: SDKs :: Java :: Container :: Java ${imageJavaVersion} Container" @@ -71,19 +72,19 @@ task copySdkHarnessLauncher(type: Copy) { } task copyJavaThirdPartyLicenses(type: Copy) { - from("${project(':sdks:java:container').buildDir}/target/third_party_licenses") + from project(':sdks:java:container').layout.buildDirectory.dir('target/third_party_licenses') into "build/target/third_party_licenses" dependsOn ':sdks:java:container:pullLicenses' } task copyGolangLicenses(type: Copy) { - from "${project(':release:go-licenses:java').buildDir}/output" + from project(':release:go-licenses:java').layout.buildDirectory.dir('output') into "build/target/go-licenses" dependsOn ':release:go-licenses:java:createLicenses' } task copyJdkOptions(type: Copy) { - if (imageJavaVersion == "17" || imageJavaVersion == "11") { + if (["11", "17", "21"].contains(imageJavaVersion)) { from "option-jamm.json" } from "java${imageJavaVersion}-security.properties" @@ -97,10 +98,10 @@ task skipPullLicenses(type: Exec) { } task validateJavaHome { - if (imageJavaVersion == "11" || imageJavaVersion == "17") { + if (["11", "17", "21"].contains(imageJavaVersion)) { doFirst { - if (!project.hasProperty('java17Home') && !project.hasProperty('java11Home')) { - throw new GradleException('java17Home or java11Home property required. Re-run with -Pjava17Home or -Pjava11Home') + if (!project.hasProperty("java${imageJavaVersion}Home")) { + throw new GradleException("java${imageJavaVersion}Home property required for imageJavaVersion=${imageJavaVersion}. Re-run with -Pjava${imageJavaVersion}Home") } } } @@ -124,6 +125,7 @@ docker { buildArgs([ 'pull_licenses': project.rootProject.hasProperty(["docker-pull-licenses"]) || project.rootProject.hasProperty(["isRelease"]), + 'base_image': javaBaseImage, 'java_version': imageJavaVersion, ]) buildx useBuildx diff --git a/sdks/java/container/java21/build.gradle b/sdks/java/container/java21/build.gradle new file mode 100644 index 000000000000..038064102dcb --- /dev/null +++ b/sdks/java/container/java21/build.gradle @@ -0,0 +1,30 @@ +/* + * 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. + */ + +project.ext { + // TODO(https://github.com/apache/beam/issues/28120) switch to temurin once available + javaBaseImage = 'openjdk' + imageJavaVersion = '21' +} + +// Load the main build script which contains all build logic. +apply from: "../common.gradle" + +dependencies { + dockerDependency project(path: ":sdks:java:container:agent") +} \ No newline at end of file diff --git a/sdks/java/container/java21/option-jamm.json b/sdks/java/container/java21/option-jamm.json new file mode 100644 index 000000000000..5647ff66be5c --- /dev/null +++ b/sdks/java/container/java21/option-jamm.json @@ -0,0 +1,12 @@ +{ + "name": "jamm", + "enabled": true, + "options": { + "java_arguments": [ + "--add-modules=jamm", + "--module-path=/opt/apache/beam/jars/jamm.jar", + "--add-opens=java.base/java.lang=jamm", + "--add-opens=java.base/java.util=jamm" + ] + } +} \ No newline at end of file diff --git a/sdks/java/testing/jpms-tests/build.gradle b/sdks/java/testing/jpms-tests/build.gradle index 6321f874c903..2a25463931c1 100644 --- a/sdks/java/testing/jpms-tests/build.gradle +++ b/sdks/java/testing/jpms-tests/build.gradle @@ -23,10 +23,8 @@ plugins { } // overwrite javaVersion before applyJavaNature -if (project.hasProperty("compileAndRunTestsWithJava17")) { - javaVersion = '1.17' -} else { - javaVersion = '1.11' +if (project.hasProperty("testJavaVersion")) { + javaVersion = "1.${project.getProperty('testJavaVersion')}" as String } applyJavaNature( @@ -42,13 +40,14 @@ ext.summary = "E2E test for Java 9 modules" // direct compileJava to use specified java version. project.tasks.compileJava { - if (project.hasProperty("compileAndRunTestsWithJava11")) { + if (project.hasProperty('testJavaVersion')) { options.fork = true - options.forkOptions.javaHome = project.findProperty("java11Home") as File - } else if (project.hasProperty("compileAndRunTestsWithJava17")) { - options.fork = true - options.forkOptions.javaHome = project.findProperty("java17Home") as File - setJava17Options(options) + options.forkOptions.javaHome = project.findProperty("java${project.getProperty('testJavaVersion')}Home") as File + if (project.getProperty('testJavaVersion') == '17') { + setJava17Options(options) + } else if (project.getProperty('testJavaVersion') == '21') { + setJava21Options(options) + } } } @@ -120,7 +119,7 @@ plugins.withType(JavaPlugin).configureEach{ // JPMS requires JDK > 8 project.tasks.each { it.onlyIf { - project.hasProperty("compileAndRunTestsWithJava17") + project.hasProperty('testJavaVersion') || JavaVersion.VERSION_1_8.compareTo(JavaVersion.current()) < 0 } } diff --git a/sdks/java/testing/test-utils/build.gradle b/sdks/java/testing/test-utils/build.gradle index 50c815dd57f7..6e30693d8894 100644 --- a/sdks/java/testing/test-utils/build.gradle +++ b/sdks/java/testing/test-utils/build.gradle @@ -43,24 +43,15 @@ dependencies { testRuntimeOnly project(path: ":runners:direct-java", configuration: "shadowTest") } -task verifyJavaVersion(type: Test) { - filter { - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyCodeIsCompiledWithJava8' - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyTestCodeIsCompiledWithJava11' - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyRunningJVMVersionIs11' - } - doLast { - println 'Java verified' +['11', '17', '21'].each { + tasks.create(name: "verifyJavaVersion${it}", type: Test) { + filter { + includeTestsMatching "org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyCodeIsCompiledWithJava8" + includeTestsMatching "org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyTestCodeIsCompiledWithJava${it}" + includeTestsMatching "org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyRunningJVMVersionIs${it}" + } + doLast { + println 'Java verified' + } } } - -task verifyJavaVersion17(type: Test) { - filter { - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyCodeIsCompiledWithJava8' - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyTestCodeIsCompiledWithJava17' - includeTestsMatching 'org.apache.beam.sdk.testutils.jvmverification.JvmVerification.verifyRunningJVMVersionIs17' - } - doLast { - println 'Java verified' - } -} \ No newline at end of file diff --git a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java index ad29e8b6a1d6..a6b5d6dca6c1 100644 --- a/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java +++ b/sdks/java/testing/test-utils/src/test/java/org/apache/beam/sdk/testutils/jvmverification/JvmVerification.java @@ -20,6 +20,7 @@ import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v11; import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v17; import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v1_8; +import static org.apache.beam.sdk.testutils.jvmverification.JvmVerification.Java.v21; import static org.junit.Assert.assertEquals; import java.io.IOException; @@ -39,6 +40,7 @@ public class JvmVerification { versionMapping.put("0034", v1_8); versionMapping.put("0037", v11); versionMapping.put("003d", v17); + versionMapping.put("0041", v21); } // bytecode @@ -62,6 +64,11 @@ public void verifyTestCodeIsCompiledWithJava17() throws IOException { assertEquals(v17, getByteCodeVersion(JvmVerification.class)); } + @Test + public void verifyTestCodeIsCompiledWithJava21() throws IOException { + assertEquals(v21, getByteCodeVersion(JvmVerification.class)); + } + // jvm @Test public void verifyRunningJVMVersionIs11() { @@ -75,6 +82,12 @@ public void verifyRunningJVMVersionIs17() { assertEquals(v17.name, version); } + @Test + public void verifyRunningJVMVersionIs21() { + final String version = getJavaSpecification(); + assertEquals(v21.name, version); + } + private static Java getByteCodeVersion(final Class clazz) throws IOException { final InputStream stream = clazz.getClassLoader().getResourceAsStream(clazz.getName().replace(".", "/") + ".class"); @@ -91,7 +104,8 @@ private static String getJavaSpecification() { enum Java { v1_8("1.8"), v11("11"), - v17("17"); + v17("17"), + v21("21"); final String name; diff --git a/settings.gradle.kts b/settings.gradle.kts index c370c5da27d1..d1069ec2d352 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -175,6 +175,7 @@ include(":sdks:java:container:agent") include(":sdks:java:container:java8") include(":sdks:java:container:java11") include(":sdks:java:container:java17") +include(":sdks:java:container:java21") include(":sdks:java:core") include(":sdks:java:core:jmh") include(":sdks:java:expansion-service") diff --git a/website/www/site/content/en/roadmap/java-sdk.md b/website/www/site/content/en/roadmap/java-sdk.md index b65424b57a3d..a1c85e139193 100644 --- a/website/www/site/content/en/roadmap/java-sdk.md +++ b/website/www/site/content/en/roadmap/java-sdk.md @@ -17,9 +17,9 @@ limitations under the License. # Java SDK Roadmap -## Next Java LTS version support (Java 17) +## Next Java LTS version support (Java 21) Work to support the next LTS release of Java is in progress. For more details -about the scope and info on the various tasks please see the JIRA ticket. +about the scope and info on the various tasks please see the GitHub Issue. -- JIRA: [BEAM-12240](https://issues.apache.org/jira/browse/BEAM-12240) +- GitHub: [#28120](https://github.com/apache/beam/issues/28120) From 9c75db4760ca3440a0a2ed12f031d84bc011aa96 Mon Sep 17 00:00:00 2001 From: Robert Bradshaw Date: Wed, 11 Oct 2023 18:19:29 -0700 Subject: [PATCH 79/85] [YAML] Avro format for PubSub. (#28899) --- sdks/python/apache_beam/yaml/yaml_io.py | 21 ++++++ sdks/python/apache_beam/yaml/yaml_io_test.py | 68 ++++++++++++++++++++ 2 files changed, 89 insertions(+) diff --git a/sdks/python/apache_beam/yaml/yaml_io.py b/sdks/python/apache_beam/yaml/yaml_io.py index 3321644ded57..b2bf150fa558 100644 --- a/sdks/python/apache_beam/yaml/yaml_io.py +++ b/sdks/python/apache_beam/yaml/yaml_io.py @@ -23,6 +23,7 @@ implementations of the same transforms, the configs must be kept in sync. """ +import io import os from typing import Any from typing import Callable @@ -32,12 +33,14 @@ from typing import Optional from typing import Tuple +import fastavro import yaml import apache_beam as beam import apache_beam.io as beam_io from apache_beam.io import ReadFromBigQuery from apache_beam.io import WriteToBigQuery +from apache_beam.io import avroio from apache_beam.io.gcp.bigquery import BigQueryDisposition from apache_beam.portability.api import schema_pb2 from apache_beam.typehints import schemas @@ -146,6 +149,13 @@ def _create_parser( elif format == 'json': beam_schema = json_utils.json_schema_to_beam_schema(schema) return beam_schema, json_utils.json_parser(beam_schema) + elif format == 'avro': + beam_schema = avroio.avro_schema_to_beam_schema(schema) + covert_to_row = avroio.avro_dict_to_beam_row(schema, beam_schema) + return ( + beam_schema, + lambda record: covert_to_row( + fastavro.schemaless_reader(io.BytesIO(record), schema))) else: raise ValueError(f'Unknown format: {format}') @@ -162,6 +172,17 @@ def _create_formatter( return lambda row: getattr(row, field_names[0]) elif format == 'json': return json_utils.json_formater(beam_schema) + elif format == 'avro': + avro_schema = schema or avroio.beam_schema_to_avro_schema(beam_schema) + from_row = avroio.beam_row_to_avro_dict(avro_schema, beam_schema) + + def formatter(row): + buffer = io.BytesIO() + fastavro.schemaless_writer(buffer, avro_schema, from_row(row)) + buffer.seek(0) + return buffer.read() + + return formatter else: raise ValueError(f'Unknown format: {format}') diff --git a/sdks/python/apache_beam/yaml/yaml_io_test.py b/sdks/python/apache_beam/yaml/yaml_io_test.py index 72675da278b0..7071860a7bf1 100644 --- a/sdks/python/apache_beam/yaml/yaml_io_test.py +++ b/sdks/python/apache_beam/yaml/yaml_io_test.py @@ -15,9 +15,12 @@ # limitations under the License. # +import io +import json import logging import unittest +import fastavro import mock import apache_beam as beam @@ -167,6 +170,48 @@ def test_read_with_id_attribute(self): result, equal_to([beam.Row(payload=b'msg1'), beam.Row(payload=b'msg2')])) + _avro_schema = { + 'type': 'record', + 'name': 'ec', + 'fields': [{ + 'name': 'label', 'type': 'string' + }, { + 'name': 'rank', 'type': 'int' + }] + } + + def _encode_avro(self, data): + buffer = io.BytesIO() + fastavro.schemaless_writer(buffer, self._avro_schema, data) + buffer.seek(0) + return buffer.read() + + def test_read_avro(self): + + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch( + 'apache_beam.io.ReadFromPubSub', + FakeReadFromPubSub( + topic='my_topic', + messages=[PubsubMessage(self._encode_avro({'label': '37a', + 'rank': 1}), {}), + PubsubMessage(self._encode_avro({'label': '389a', + 'rank': 2}), {})])): + result = p | YamlTransform( + ''' + type: ReadFromPubSub + config: + topic: my_topic + format: avro + schema: %s + ''' % json.dumps(self._avro_schema)) + assert_that( + result, + equal_to( + [beam.Row(label='37a', rank=1), # linebreak + beam.Row(label='389a', rank=2)])) + def test_read_json(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: @@ -346,6 +391,29 @@ def test_write_with_id_attribute(self): id_attribute: some_attr ''')) + def test_write_avro(self): + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + with mock.patch( + 'apache_beam.io.WriteToPubSub', + FakeWriteToPubSub( + topic='my_topic', + messages=[PubsubMessage(self._encode_avro({'label': '37a', + 'rank': 1}), {}), + PubsubMessage(self._encode_avro({'label': '389a', + 'rank': 2}), {})])): + _ = ( + p | beam.Create( + [beam.Row(label='37a', rank=1), beam.Row(label='389a', rank=2)]) + | YamlTransform( + ''' + type: WriteToPubSub + input: input + config: + topic: my_topic + format: avro + ''')) + def test_write_json(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: From fb99630d1e0924eec54239ee369b2042b15d5187 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 3 Oct 2023 13:28:55 -0400 Subject: [PATCH 80/85] [Website] Announce Beam 2.51.0 --- CHANGES.md | 18 +- website/www/site/config.toml | 2 +- .../www/site/content/en/blog/beam-2.51.0.md | 210 ++++++++++++++++++ .../site/content/en/get-started/downloads.md | 14 +- 4 files changed, 226 insertions(+), 18 deletions(-) create mode 100644 website/www/site/content/en/blog/beam-2.51.0.md diff --git a/CHANGES.md b/CHANGES.md index 0c2c2e3f79f4..43ade8c530b9 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -87,16 +87,7 @@ * ([#X](https://github.com/apache/beam/issues/X)). -# [2.51.0] - Unreleased - -## Highlights - -* New highly anticipated feature X added to Python SDK ([#X](https://github.com/apache/beam/issues/X)). -* New highly anticipated feature Y added to Java SDK ([#Y](https://github.com/apache/beam/issues/Y)). - -## I/Os - -* Support for X source added (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). +# [2.51.0] - 2023-10-03 ## New Features / Improvements @@ -104,6 +95,7 @@ * In Python, the [VertexAIModelHandlerJSON](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.vertex_ai_inference.html#apache_beam.ml.inference.vertex_ai_inference.VertexAIModelHandlerJSON) now supports passing in inference_args. These will be passed through to the Vertex endpoint as parameters. * Added support to run `mypy` on user pipelines ([#27906](https://github.com/apache/beam/issues/27906)) + ## Breaking Changes * Removed fastjson library dependency for Beam SQL. Table property is changed to be based on jackson ObjectNode (Java) ([#24154](https://github.com/apache/beam/issues/24154)). @@ -111,9 +103,6 @@ * Removed the parameter `t reflect.Type` from `parquetio.Write`. The element type is derived from the input PCollection (Go) ([#28490](https://github.com/apache/beam/issues/28490)) * Refactor BeamSqlSeekableTable.setUp adding a parameter joinSubsetType. [#28283](https://github.com/apache/beam/issues/28283) -## Deprecations - -* X behavior is deprecated and will be removed in X versions ([#X](https://github.com/apache/beam/issues/X)). ## Bugfixes @@ -128,7 +117,8 @@ ## Known Issues -* ([#X](https://github.com/apache/beam/issues/X)). +* Python pipelines using BigQuery Storage Read API must pin `fastavro` + dependency to 1.8.3 or earlier: [#28811](https://github.com/apache/beam/issues/28811) # [2.50.0] - 2023-08-30 diff --git a/website/www/site/config.toml b/website/www/site/config.toml index 6a1907e60591..c7b0cd3412e6 100644 --- a/website/www/site/config.toml +++ b/website/www/site/config.toml @@ -104,7 +104,7 @@ github_project_repo = "https://github.com/apache/beam" [params] description = "Apache Beam is an open source, unified model and set of language-specific SDKs for defining and executing data processing workflows, and also data ingestion and integration flows, supporting Enterprise Integration Patterns (EIPs) and Domain Specific Languages (DSLs). Dataflow pipelines simplify the mechanics of large-scale batch and streaming data processing and can run on a number of runtimes like Apache Flink, Apache Spark, and Google Cloud Dataflow (a cloud service). Beam also brings DSL in different languages, allowing users to easily implement their data integration processes." -release_latest = "2.50.0" +release_latest = "2.51.0" # The repository and branch where the files live in Github or Colab. This is used # to serve and stage from your local branch, but publish to the master branch. # e.g. https://github.com/{{< param branch_repo >}}/path/to/notebook.ipynb diff --git a/website/www/site/content/en/blog/beam-2.51.0.md b/website/www/site/content/en/blog/beam-2.51.0.md new file mode 100644 index 000000000000..aaa4142bae62 --- /dev/null +++ b/website/www/site/content/en/blog/beam-2.51.0.md @@ -0,0 +1,210 @@ +--- +title: "Apache Beam 2.51.0" +date: 2023-10-11 09:00:00 -0400 +categories: + - blog + - release +authors: + - klk +--- + + +We are happy to present the new 2.51.0 release of Beam. +This release includes both improvements and new functionality. +See the [download page](/get-started/downloads/#2510-2023-10-03) for this release. + + + +For more information on changes in 2.51.0, check out the [detailed release notes](https://github.com/apache/beam/milestone/15). + +## New Features / Improvements + +* In Python, [RunInference](https://beam.apache.org/documentation/sdks/python-machine-learning/#why-use-the-runinference-api) now supports loading many models in the same transform using a [KeyedModelHandler](https://beam.apache.org/documentation/sdks/python-machine-learning/#use-a-keyed-modelhandler) ([#27628](https://github.com/apache/beam/issues/27628)). +* In Python, the [VertexAIModelHandlerJSON](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.vertex_ai_inference.html#apache_beam.ml.inference.vertex_ai_inference.VertexAIModelHandlerJSON) now supports passing in inference_args. These will be passed through to the Vertex endpoint as parameters. +* Added support to run `mypy` on user pipelines ([#27906](https://github.com/apache/beam/issues/27906)) + + +## Breaking Changes + +* Removed fastjson library dependency for Beam SQL. Table property is changed to be based on jackson ObjectNode (Java) ([#24154](https://github.com/apache/beam/issues/24154)). +* Removed TensorFlow from Beam Python container images [PR](https://github.com/apache/beam/pull/28424). If you have been negatively affected by this change, please comment on [#20605](https://github.com/apache/beam/issues/20605). +* Removed the parameter `t reflect.Type` from `parquetio.Write`. The element type is derived from the input PCollection (Go) ([#28490](https://github.com/apache/beam/issues/28490)) +* Refactor BeamSqlSeekableTable.setUp adding a parameter joinSubsetType. [#28283](https://github.com/apache/beam/issues/28283) + + +## Bugfixes + +* Fixed exception chaining issue in GCS connector (Python) ([#26769](https://github.com/apache/beam/issues/26769#issuecomment-1700422615)). +* Fixed streaming inserts exception handling, GoogleAPICallErrors are now retried according to retry strategy and routed to failed rows where appropriate rather than causing a pipeline error (Python) ([#21080](https://github.com/apache/beam/issues/21080)). +* Fixed a bug in Python SDK's cross-language Bigtable sink that mishandled records that don't have an explicit timestamp set: [#28632](https://github.com/apache/beam/issues/28632). + + +## Security Fixes +* Python containers updated, fixing [CVE-2021-30474](https://nvd.nist.gov/vuln/detail/CVE-2021-30474), [CVE-2021-30475](https://nvd.nist.gov/vuln/detail/CVE-2021-30475), [CVE-2021-30473](https://nvd.nist.gov/vuln/detail/CVE-2021-30473), [CVE-2020-36133](https://nvd.nist.gov/vuln/detail/CVE-2020-36133), [CVE-2020-36131](https://nvd.nist.gov/vuln/detail/CVE-2020-36131), [CVE-2020-36130](https://nvd.nist.gov/vuln/detail/CVE-2020-36130), and [CVE-2020-36135](https://nvd.nist.gov/vuln/detail/CVE-2020-36135) +* Used go 1.21.1 to build, fixing [CVE-2023-39320](https://security-tracker.debian.org/tracker/CVE-2023-39320) + + +## Known Issues + +* Python pipelines using BigQuery Storage Read API must pin `fastavro` dependency to 1.8.3 + or earlier: [#28811](https://github.com/apache/beam/issues/28811) + +## List of Contributors + +According to git shortlog, the following people contributed to the 2.50.0 release. Thank you to all contributors! + +Adam Whitmore + +Ahmed Abualsaud + +Ahmet Altay + +Aleksandr Dudko + +Alexey Romanenko + +Anand Inguva + +Andrey Devyatkin + +Arvind Ram + +Arwin Tio + +BjornPrime + +Bruno Volpato + +Bulat + +Celeste Zeng + +Chamikara Jayalath + +Clay Johnson + +Damon + +Danny McCormick + +David Cavazos + +Dip Patel + +Hai Joey Tran + +Hao Xu + +Haruka Abe + +Jack Dingilian + +Jack McCluskey + +Jeff Kinard + +Jeffrey Kinard + +Joey Tran + +Johanna Öjeling + +Julien Tournay + +Kenneth Knowles + +Kerry Donny-Clark + +Mattie Fu + +Melissa Pashniak + +Michel Davit + +Moritz Mack + +Pranav Bhandari + +Rebecca Szper + +Reeba Qureshi + +Reuven Lax + +Ritesh Ghorse + +Robert Bradshaw + +Robert Burke + +Ruwann + +Ryan Tam + +Sam Rohde + +Sereana Seim + +Svetak Sundhar + +Tim Grein + +Udi Meiri + +Valentyn Tymofieiev + +Vitaly Terentyev + +Vlado Djerek + +Xinyu Liu + +Yi Hu + +Zbynek Konecny + +Zechen Jiang + +bzablocki + +caneff + +dependabot[bot] + +gDuperran + +gabry.wu + +johnjcasey + +kberezin-nshl + +kennknowles + +liferoad + +lostluck + +magicgoody + +martin trieu + +mosche + +olalamichelle + +tvalentyn + +xqhu + +Łukasz Spyra diff --git a/website/www/site/content/en/get-started/downloads.md b/website/www/site/content/en/get-started/downloads.md index 9a753dafe32e..b564a5801cd8 100644 --- a/website/www/site/content/en/get-started/downloads.md +++ b/website/www/site/content/en/get-started/downloads.md @@ -96,10 +96,18 @@ versions denoted `0.x.y`. ## Releases +### 2.51.0 (2023-10-11) +Official [source code download](https://downloads.apache.org/beam/2.51.0/apache-beam-2.51.0-source-release.zip). +[SHA-512](https://downloads.apache.org/beam/2.51.0/apache-beam-2.51.0-source-release.zip.sha512). +[signature](https://downloads.apache.org/beam/2.51.0/apache-beam-2.51.0-source-release.zip.asc). + +[Release notes](https://github.com/apache/beam/releases/tag/v2.51.0) +[Blog post](/blog/beam-2.51.0). + ### 2.50.0 (2023-08-30) -Official [source code download](https://downloads.apache.org/beam/2.50.0/apache-beam-2.50.0-source-release.zip). -[SHA-512](https://downloads.apache.org/beam/2.50.0/apache-beam-2.50.0-source-release.zip.sha512). -[signature](https://downloads.apache.org/beam/2.50.0/apache-beam-2.50.0-source-release.zip.asc). +Official [source code download](https://archive.apache.org/beam/2.50.0/apache-beam-2.50.0-source-release.zip). +[SHA-512](https://archive.apache.org/beam/2.50.0/apache-beam-2.50.0-source-release.zip.sha512). +[signature](https://archive.apache.org/beam/2.50.0/apache-beam-2.50.0-source-release.zip.asc). [Release notes](https://github.com/apache/beam/releases/tag/v2.50.0) [Blog post](/blog/beam-2.50.0). From 6b3b3eb603fbd6feccf72a093bd520c5ca973cb5 Mon Sep 17 00:00:00 2001 From: Robert Burke Date: Wed, 11 Oct 2023 19:43:06 -0700 Subject: [PATCH 81/85] Update go_tests.yml to cache (#28954) Configure caching go deps properly in Go Tests action. --- .github/workflows/go_tests.yml | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.github/workflows/go_tests.yml b/.github/workflows/go_tests.yml index 6884dd692522..66c57d219a83 100644 --- a/.github/workflows/go_tests.yml +++ b/.github/workflows/go_tests.yml @@ -46,6 +46,8 @@ jobs: - uses: actions/setup-go@v4 with: go-version: '1.21' + cache-dependency-path: | + sdks/go.sum - name: Delete old coverage run: "cd sdks && rm -rf .coverage.txt || :" - name: Run coverage @@ -70,4 +72,4 @@ jobs: with: work-dir: ./sdks go-package: ./... - go-version-input: 1.21 \ No newline at end of file + go-version-input: 1.21 From 4a7c4842aceb6786040d5cd699b7916c4509bb7b Mon Sep 17 00:00:00 2001 From: Kerry Donny-Clark Date: Wed, 11 Oct 2023 23:35:25 -0400 Subject: [PATCH 82/85] Add new resource hint to all sdks for number of cpus per worker machine (#28848) * Adds new resource hint for number of cpus per worker. * Fixes minor bugs.` * Go fmt and removes unwanted .python-version file. * Adds tests. * Fixes typo. * Fixes Java tests, adds URN to parsers. * Addresses FindBugs issue with int parsing. * Applies Java formatting corrections * Adds generated go protobufs --- .../model/pipeline/v1/beam_runner_api.proto | 4 + .../model/fnexecution_v1/beam_fn_api.pb.go | 67 ++++++++++----- .../fnexecution_v1/beam_fn_api_grpc.pb.go | 2 +- .../fnexecution_v1/beam_provision_api.pb.go | 2 +- .../beam_provision_api_grpc.pb.go | 2 +- .../jobmanagement_v1/beam_artifact_api.pb.go | 2 +- .../beam_artifact_api_grpc.pb.go | 2 +- .../jobmanagement_v1/beam_expansion_api.pb.go | 2 +- .../beam_expansion_api_grpc.pb.go | 2 +- .../model/jobmanagement_v1/beam_job_api.pb.go | 2 +- .../jobmanagement_v1/beam_job_api_grpc.pb.go | 2 +- .../model/pipeline_v1/beam_runner_api.pb.go | 81 ++++++++++--------- .../pipeline_v1/beam_runner_api_grpc.pb.go | 2 +- .../beam/model/pipeline_v1/endpoints.pb.go | 2 +- .../pipeline_v1/external_transforms.pb.go | 2 +- .../pkg/beam/model/pipeline_v1/metrics.pb.go | 2 +- .../pkg/beam/model/pipeline_v1/schema.pb.go | 2 +- .../pipeline_v1/standard_window_fns.pb.go | 2 +- sdks/go/pkg/beam/options/resource/hint.go | 37 +++++++++ .../go/pkg/beam/options/resource/hint_test.go | 45 ++++++++++- .../resourcehints/ResourceHints.java | 63 +++++++++++++++ .../resourcehints/ResourceHintsTest.java | 11 ++- .../apache_beam/transforms/resources.py | 16 ++++ .../apache_beam/transforms/resources_test.py | 6 ++ 24 files changed, 281 insertions(+), 79 deletions(-) diff --git a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto index 2483103b5794..db958f183c45 100644 --- a/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto +++ b/model/pipeline/src/main/proto/org/apache/beam/model/pipeline/v1/beam_runner_api.proto @@ -1982,5 +1982,9 @@ message StandardResourceHints { // SDKs should convert the size to bytes, but can allow users to specify human-friendly units (e.g. GiB). // Payload: ASCII encoded string of the base 10 representation of an integer number of bytes. MIN_RAM_BYTES = 1 [(beam_urn) = "beam:resources:min_ram_bytes:v1"]; + // Describes desired number of CPUs available in transform's execution environment. + // SDKs should accept and validate a positive integer count. + // Payload: ASCII encoded string of the base 10 representation of an integer number of CPUs. + CPU_COUNT = 2 [(beam_urn) = "beam:resources:cpu_count:v1"]; } } diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go index 1d547470ea1a..9d14cff3c7d6 100644 --- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api.pb.go @@ -27,7 +27,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/fn_execution/v1/beam_fn_api.proto // TODO: Consider consolidating common components in another package @@ -1883,30 +1883,57 @@ func (x *ProcessBundleSplitRequest) GetDesiredSplits() map[string]*ProcessBundle // first_residual_element. // - The current bundle, if no further splits happen, will have done exactly // the work under primary_roots and all elements up to and including the -// channel splits last_primary_element. +// channel split's last_primary_element. // // This allows the SDK to relinquish ownership of and commit to not process some // of the elements that it may have been sent (the residual) while retaining // ownership and commitment to finish the other portion (the primary). // -// For example, lets say the SDK is processing elements A B C D E and a split -// request comes in. The SDK could return a response with a channel split -// representing a last_primary_element of 3 (D) and first_residual_element of 4 -// (E). The SDK is now responsible for processing A B C D and the runner must -// process E in the future. A future split request could have the SDK split the -// elements B into B1 and B2 and C into C1 and C2 representing their primary and -// residual roots. The SDK would return a response with a channel split -// representing a last_primary_element of 0 (A) and first_residual_element of 3 -// (D) with primary_roots (B1, C1) and residual_roots (B2, C2). The SDK is now -// responsible for processing A B1 C1 and the runner must process C2 D2 (and E -// from the prior split) in the future. Yet another future split request could -// have the SDK could split B1 further into B1a and B1b primary and residuals -// and return C2 as a residual (assuming C2 was left unprocessed). The SDK would -// return a response with a channel split representing a last_primary_element of -// 0 (A) and first_residual_element of 4 (E) with primary_roots (B1a) and -// residual_roots (B1b, C1). The SDK is now responsible for processing A B1a the -// runner must process B1b C1 (in addition to C2, D, E from prior splits) in the -// future. +// Example with three splits of a single bundle: +// Let's say the SDK is processing elements [A B C D E]. These elements make +// up the 0-indexed channel. +// +// ** First Split ** +// Channel Split = [ A B C D <> E ] +// Primary Roots = [] (No elements were split) +// Residual Roots = [] +// +// Say a split request comes in. The SDK could return a response with a channel +// split representing a last_primary_element of 3 (D) and +// first_residual_element of 4 (E). The SDK is now responsible for processing A +// B C D and the runner must process E in the future. +// +// (A B C D) | (E) +// +// ** Second Split ** +// Channel Split = [ A < B C > D E ] +// Primary Roots = [B1 C1] +// Residual Roots = [B2 C2] +// +// A future split request could have the SDK split the elements B into B1 and +// B2 and C into C1 and C2 representing their primary and residual roots. The +// +// (A B1 C1) | (B2 C2 D) +// +// SDK would return a response with a channel split representing a +// last_primary_element of 0 (A) and first_residual_element of 3 (D) with +// primary_roots (B1, C1) and residual_roots (B2, C2). The SDK is now +// responsible for processing A B1 C1 and the runner must process B2 C2 D (and +// E from the prior split) in the future. +// +// ** Third Split ** +// Channel Split = [ A < B C > D E ] +// Primary Roots = [B1a] +// Residual Roots [B1b C1] +// Yet another future split request could have the SDK could split B1 further +// into B1a and B1b primary and residuals and return C1 as a residual (assuming +// C1 was left unprocessed). The SDK would return a response with a channel +// split representing a last_primary_element of 0 (A) and +// first_residual_element of 3 (E) with primary_roots (B1a) and residual_roots +// (B1b, C1). The SDK is now responsible for processing A B1a the runner must +// process B1b C1 (in addition to C2, D, E from prior splits) in the future. +// +// (A B1a) | (B1b C1) // // For more rigorous definitions see https://s.apache.org/beam-breaking-fusion type ProcessBundleSplitResponse struct { diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_grpc.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_grpc.pb.go index ac9e402750c4..cd53ea805705 100644 --- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_fn_api_grpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.1.0 -// - protoc v4.24.0--rc1 +// - protoc v4.24.4 // source: org/apache/beam/model/fn_execution/v1/beam_fn_api.proto package fnexecution_v1 diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go index a24609b2fd05..26cf245f7206 100644 --- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api.pb.go @@ -22,7 +22,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/fn_execution/v1/beam_provision_api.proto package fnexecution_v1 diff --git a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_grpc.pb.go b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_grpc.pb.go index f9c6f5681399..9064b348b4c0 100644 --- a/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/fnexecution_v1/beam_provision_api_grpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.1.0 -// - protoc v4.24.0--rc1 +// - protoc v4.24.4 // source: org/apache/beam/model/fn_execution/v1/beam_provision_api.proto package fnexecution_v1 diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go index 6a7663d77e9c..85bb2e368970 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api.pb.go @@ -22,7 +22,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/job_management/v1/beam_artifact_api.proto package jobmanagement_v1 diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_grpc.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_grpc.pb.go index 6b381b96f3d1..28e43e21fbbd 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_artifact_api_grpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.1.0 -// - protoc v4.24.0--rc1 +// - protoc v4.24.4 // source: org/apache/beam/model/job_management/v1/beam_artifact_api.proto package jobmanagement_v1 diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go index 0f33c7ab9e3c..8f7ca43ec0f5 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api.pb.go @@ -22,7 +22,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/job_management/v1/beam_expansion_api.proto package jobmanagement_v1 diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_grpc.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_grpc.pb.go index e2cc3c4f77ec..f1c3782f5fb8 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_expansion_api_grpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.1.0 -// - protoc v4.24.0--rc1 +// - protoc v4.24.4 // source: org/apache/beam/model/job_management/v1/beam_expansion_api.proto package jobmanagement_v1 diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go index d93130d26d9f..62e0b313ec2d 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api.pb.go @@ -22,7 +22,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/job_management/v1/beam_job_api.proto package jobmanagement_v1 diff --git a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_grpc.pb.go b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_grpc.pb.go index 08da7e4643c3..38f2c85a1c1c 100644 --- a/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/jobmanagement_v1/beam_job_api_grpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.1.0 -// - protoc v4.24.0--rc1 +// - protoc v4.24.4 // source: org/apache/beam/model/job_management/v1/beam_job_api.proto package jobmanagement_v1 diff --git a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go index b20a5dccbe05..49df2b5c2e59 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api.pb.go @@ -22,7 +22,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/beam_runner_api.proto package pipeline_v1 @@ -1857,6 +1857,10 @@ const ( // SDKs should convert the size to bytes, but can allow users to specify human-friendly units (e.g. GiB). // Payload: ASCII encoded string of the base 10 representation of an integer number of bytes. StandardResourceHints_MIN_RAM_BYTES StandardResourceHints_Enum = 1 + // Describes desired number of CPUs available in transform's execution environment. + // SDKs should accept and validate a positive integer count. + // Payload: ASCII encoded string of the base 10 representation of an integer number of CPUs. + StandardResourceHints_CPU_COUNT StandardResourceHints_Enum = 2 ) // Enum value maps for StandardResourceHints_Enum. @@ -1864,10 +1868,12 @@ var ( StandardResourceHints_Enum_name = map[int32]string{ 0: "ACCELERATOR", 1: "MIN_RAM_BYTES", + 2: "CPU_COUNT", } StandardResourceHints_Enum_value = map[string]int32{ "ACCELERATOR": 0, "MIN_RAM_BYTES": 1, + "CPU_COUNT": 2, } ) @@ -9223,42 +9229,45 @@ var file_org_apache_beam_model_pipeline_v1_beam_runner_api_proto_rawDesc = []byt 0x65, 0x63, 0x75, 0x74, 0x61, 0x62, 0x6c, 0x65, 0x53, 0x74, 0x61, 0x67, 0x65, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x54, 0x69, 0x6d, 0x65, 0x72, 0x49, 0x64, 0x48, 0x00, 0x52, 0x05, 0x74, 0x69, 0x6d, 0x65, 0x72, 0x42, 0x08, 0x0a, 0x06, 0x74, 0x61, 0x72, 0x67, 0x65, 0x74, 0x22, - 0x8f, 0x01, 0x0a, 0x15, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x6f, - 0x75, 0x72, 0x63, 0x65, 0x48, 0x69, 0x6e, 0x74, 0x73, 0x22, 0x76, 0x0a, 0x04, 0x45, 0x6e, 0x75, - 0x6d, 0x12, 0x34, 0x0a, 0x0b, 0x41, 0x43, 0x43, 0x45, 0x4c, 0x45, 0x52, 0x41, 0x54, 0x4f, 0x52, - 0x10, 0x00, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, - 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, 0x61, 0x63, 0x63, 0x65, 0x6c, 0x65, 0x72, - 0x61, 0x74, 0x6f, 0x72, 0x3a, 0x76, 0x31, 0x12, 0x38, 0x0a, 0x0d, 0x4d, 0x49, 0x4e, 0x5f, 0x52, - 0x41, 0x4d, 0x5f, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x01, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, 0xc2, - 0x05, 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, - 0x3a, 0x6d, 0x69, 0x6e, 0x5f, 0x72, 0x61, 0x6d, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x3a, 0x76, - 0x31, 0x32, 0x8f, 0x01, 0x0a, 0x11, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, - 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x7a, 0x0a, 0x06, 0x45, 0x76, 0x65, 0x6e, 0x74, - 0x73, 0x12, 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, - 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, - 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, + 0xc2, 0x01, 0x0a, 0x15, 0x53, 0x74, 0x61, 0x6e, 0x64, 0x61, 0x72, 0x64, 0x52, 0x65, 0x73, 0x6f, + 0x75, 0x72, 0x63, 0x65, 0x48, 0x69, 0x6e, 0x74, 0x73, 0x22, 0xa8, 0x01, 0x0a, 0x04, 0x45, 0x6e, + 0x75, 0x6d, 0x12, 0x34, 0x0a, 0x0b, 0x41, 0x43, 0x43, 0x45, 0x4c, 0x45, 0x52, 0x41, 0x54, 0x4f, + 0x52, 0x10, 0x00, 0x1a, 0x23, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1d, 0x62, 0x65, 0x61, 0x6d, 0x3a, + 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, 0x61, 0x63, 0x63, 0x65, 0x6c, 0x65, + 0x72, 0x61, 0x74, 0x6f, 0x72, 0x3a, 0x76, 0x31, 0x12, 0x38, 0x0a, 0x0d, 0x4d, 0x49, 0x4e, 0x5f, + 0x52, 0x41, 0x4d, 0x5f, 0x42, 0x59, 0x54, 0x45, 0x53, 0x10, 0x01, 0x1a, 0x25, 0xa2, 0xb4, 0xfa, + 0xc2, 0x05, 0x1f, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, + 0x73, 0x3a, 0x6d, 0x69, 0x6e, 0x5f, 0x72, 0x61, 0x6d, 0x5f, 0x62, 0x79, 0x74, 0x65, 0x73, 0x3a, + 0x76, 0x31, 0x12, 0x30, 0x0a, 0x09, 0x43, 0x50, 0x55, 0x5f, 0x43, 0x4f, 0x55, 0x4e, 0x54, 0x10, + 0x02, 0x1a, 0x21, 0xa2, 0xb4, 0xfa, 0xc2, 0x05, 0x1b, 0x62, 0x65, 0x61, 0x6d, 0x3a, 0x72, 0x65, + 0x73, 0x6f, 0x75, 0x72, 0x63, 0x65, 0x73, 0x3a, 0x63, 0x70, 0x75, 0x5f, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x3a, 0x76, 0x31, 0x32, 0x8f, 0x01, 0x0a, 0x11, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, + 0x65, 0x61, 0x6d, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x7a, 0x0a, 0x06, 0x45, 0x76, + 0x65, 0x6e, 0x74, 0x73, 0x12, 0x30, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, + 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, + 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x73, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x3a, 0x2e, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, + 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, + 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, + 0x74, 0x72, 0x65, 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, + 0x6e, 0x74, 0x22, 0x00, 0x30, 0x01, 0x3a, 0x3f, 0x0a, 0x08, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x75, + 0x72, 0x6e, 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, + 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xc4, 0xa6, 0xaf, 0x58, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, + 0x62, 0x65, 0x61, 0x6d, 0x55, 0x72, 0x6e, 0x3a, 0x49, 0x0a, 0x0d, 0x62, 0x65, 0x61, 0x6d, 0x5f, + 0x63, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, + 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xc5, 0xa6, 0xaf, 0x58, + 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x62, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, + 0x6e, 0x74, 0x42, 0x78, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, - 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x2e, 0x54, 0x65, 0x73, 0x74, 0x53, 0x74, 0x72, 0x65, - 0x61, 0x6d, 0x50, 0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x2e, 0x45, 0x76, 0x65, 0x6e, 0x74, 0x22, - 0x00, 0x30, 0x01, 0x3a, 0x3f, 0x0a, 0x08, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x75, 0x72, 0x6e, 0x12, - 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, - 0x6e, 0x73, 0x18, 0xc4, 0xa6, 0xaf, 0x58, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x62, 0x65, 0x61, - 0x6d, 0x55, 0x72, 0x6e, 0x3a, 0x49, 0x0a, 0x0d, 0x62, 0x65, 0x61, 0x6d, 0x5f, 0x63, 0x6f, 0x6e, - 0x73, 0x74, 0x61, 0x6e, 0x74, 0x12, 0x21, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x75, - 0x65, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0xc5, 0xa6, 0xaf, 0x58, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x0c, 0x62, 0x65, 0x61, 0x6d, 0x43, 0x6f, 0x6e, 0x73, 0x74, 0x61, 0x6e, 0x74, 0x42, - 0x78, 0x0a, 0x21, 0x6f, 0x72, 0x67, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x62, 0x65, - 0x61, 0x6d, 0x2e, 0x6d, 0x6f, 0x64, 0x65, 0x6c, 0x2e, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, - 0x65, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x41, 0x70, 0x69, 0x5a, - 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, - 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, 0x76, 0x32, 0x2f, - 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, 0x6f, 0x64, 0x65, - 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x3b, 0x70, 0x69, - 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x33, + 0x6c, 0x69, 0x6e, 0x65, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x52, 0x75, 0x6e, 0x6e, 0x65, 0x72, 0x41, + 0x70, 0x69, 0x5a, 0x48, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, + 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x73, 0x64, 0x6b, 0x73, 0x2f, + 0x76, 0x32, 0x2f, 0x67, 0x6f, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x62, 0x65, 0x61, 0x6d, 0x2f, 0x6d, + 0x6f, 0x64, 0x65, 0x6c, 0x2f, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, + 0x3b, 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x5f, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, } var ( diff --git a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_grpc.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_grpc.pb.go index d5e65f7b768d..20a30cf4dd01 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_grpc.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/beam_runner_api_grpc.pb.go @@ -17,7 +17,7 @@ // Code generated by protoc-gen-go-grpc. DO NOT EDIT. // versions: // - protoc-gen-go-grpc v1.1.0 -// - protoc v4.24.0--rc1 +// - protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/beam_runner_api.proto package pipeline_v1 diff --git a/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go index 74348ddc3b33..2dfaffa2bff0 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/endpoints.pb.go @@ -21,7 +21,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/endpoints.proto package pipeline_v1 diff --git a/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go index 0bc21a56685e..edbe82264f5e 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/external_transforms.pb.go @@ -21,7 +21,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/external_transforms.proto package pipeline_v1 diff --git a/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go index ec3e0d704a80..60edad2363be 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/metrics.pb.go @@ -21,7 +21,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/metrics.proto package pipeline_v1 diff --git a/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go index 717fbbfb7a69..4bc6a57044cd 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/schema.pb.go @@ -24,7 +24,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/schema.proto package pipeline_v1 diff --git a/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go index dccd7d427503..e0522806df73 100644 --- a/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go +++ b/sdks/go/pkg/beam/model/pipeline_v1/standard_window_fns.pb.go @@ -22,7 +22,7 @@ // Code generated by protoc-gen-go. DO NOT EDIT. // versions: // protoc-gen-go v1.27.1 -// protoc v4.24.0--rc1 +// protoc v4.24.4 // source: org/apache/beam/model/pipeline/v1/standard_window_fns.proto package pipeline_v1 diff --git a/sdks/go/pkg/beam/options/resource/hint.go b/sdks/go/pkg/beam/options/resource/hint.go index 1538fe65def2..d823f4feafa9 100644 --- a/sdks/go/pkg/beam/options/resource/hint.go +++ b/sdks/go/pkg/beam/options/resource/hint.go @@ -196,3 +196,40 @@ func (h acceleratorHint) MergeWithOuter(outer Hint) Hint { func (h acceleratorHint) String() string { return fmt.Sprintf("accelerator=%v", h.value) } + +// CPUCount hints that this scope should be put in a machine with at least this many CPUs or vCPUs. +// +// Hints are advisory only and runners may not respect them. +// +// See https://beam.apache.org/documentation/runtime/resource-hints/ for more information about +// resource hints. +func CPUCount(v uint64) Hint { + return CPUCountHint{value: uint64(v)} +} + +type CPUCountHint struct { + value uint64 +} + +func (CPUCountHint) URN() string { + return "beam:resources:cpu_count:v1" +} + +func (h CPUCountHint) Payload() []byte { + // Go strings are utf8, and if the string is ascii, + // byte conversion handles that directly. + return []byte(strconv.FormatUint(h.value, 10)) +} + +// MergeWithOuter by keeping the maximum of the two cpu counts. +func (h CPUCountHint) MergeWithOuter(outer Hint) Hint { + // Intentional runtime panic from type assertion to catch hint merge errors. + if outer.(CPUCountHint).value > h.value { + return outer + } + return h +} + +func (h CPUCountHint) String() string { + return fmt.Sprintf("cpu_count=%v", humanize.Bytes(uint64(h.value))) +} diff --git a/sdks/go/pkg/beam/options/resource/hint_test.go b/sdks/go/pkg/beam/options/resource/hint_test.go index cf24b47b6c91..7c2a1df79294 100644 --- a/sdks/go/pkg/beam/options/resource/hint_test.go +++ b/sdks/go/pkg/beam/options/resource/hint_test.go @@ -111,6 +111,38 @@ func TestParseMinRAMHint_panic(t *testing.T) { ParseMinRAM("a bad byte string") } +func TestCPUCountHint_MergeWith(t *testing.T) { + low := CPUCountHint{value: 2} + high := CPUCountHint{value: 128} + + if got, want := low.MergeWithOuter(high), high; got != want { + t.Errorf("%v.MergeWith(%v) = %v, want %v", low, high, got, want) + } + if got, want := high.MergeWithOuter(low), high; got != want { + t.Errorf("%v.MergeWith(%v) = %v, want %v", high, low, got, want) + } +} + +func TestCPUCountHint_Payload(t *testing.T) { + tests := []struct { + value uint64 + payload string + }{ + {0, "0"}, + {2, "2"}, + {11, "11"}, + {2003, "2003"}, + {1.2e7, "12000000"}, + } + + for _, test := range tests { + h := CPUCountHint{value: test.value} + if got, want := h.Payload(), []byte(test.payload); !bytes.Equal(got, want) { + t.Errorf("%v.Payload() = %v, want %v", h, got, want) + } + } +} + // We copy the URN from the proto for use as a constant rather than perform a direct look up // each time, or increase initialization time. However we do need to validate that they are // correct, and match the standard hint urns, so that's done here. @@ -130,7 +162,11 @@ func TestStandardHintUrns(t *testing.T) { }, { h: MinRAMBytes(2e9), urn: getStandardURN(pipepb.StandardResourceHints_MIN_RAM_BYTES), + }, { + h: CPUCount(4), + urn: getStandardURN(pipepb.StandardResourceHints_CPU_COUNT), }} + for _, test := range tests { if got, want := test.h.URN(), test.urn; got != want { t.Errorf("Checked urn for %T, got %q, want %q", test.h, got, want) @@ -154,12 +190,12 @@ func (h customHint) MergeWithOuter(outer Hint) Hint { } func TestHints_Equal(t *testing.T) { - hs := NewHints(MinRAMBytes(2e9), Accelerator("type:pants;count1;install-pajamas")) + hs := NewHints(MinRAMBytes(2e9), Accelerator("type:pants;count1;install-pajamas"), CPUCount(4)) if got, want := hs.Equal(hs), true; got != want { t.Errorf("Self equal test: hs.Equal(hs) = %v, want %v", got, want) } - eq := NewHints(MinRAMBytes(2e9), Accelerator("type:pants;count1;install-pajamas")) + eq := NewHints(MinRAMBytes(2e9), Accelerator("type:pants;count1;install-pajamas"), CPUCount(4)) if got, want := hs.Equal(eq), true; got != want { t.Errorf("identical equal test: hs.Equal(eq) = %v, want %v", got, want) } @@ -223,12 +259,13 @@ func TestHints_MergeWithOuter(t *testing.T) { func TestHints_Payloads(t *testing.T) { { - hs := NewHints(MinRAMBytes(2e9), Accelerator("type:jeans;count1;")) + hs := NewHints(MinRAMBytes(2e9), Accelerator("type:jeans;count1;"), CPUCount(4)) got := hs.Payloads() want := map[string][]byte{ "beam:resources:min_ram_bytes:v1": []byte("2000000000"), "beam:resources:accelerator:v1": []byte("type:jeans;count1;"), + "beam:resources:cpu_count:v1": []byte("4"), } if !reflect.DeepEqual(got, want) { t.Errorf("hs.Payloads() = %v, want %v", got, want) @@ -248,7 +285,7 @@ func TestHints_Payloads(t *testing.T) { func TestHints_NilHints(t *testing.T) { var hs1, hs2 Hints - hs := NewHints(MinRAMBytes(2e9), Accelerator("type:pants;count1;install-pajamas")) + hs := NewHints(MinRAMBytes(2e9), Accelerator("type:pants;count1;install-pajamas"), CPUCount(4)) if got, want := hs1.Equal(hs2), true; got != want { t.Errorf("nils equal test: (nil).Equal(nil) = %v, want %v", got, want) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java index afd6a6ccb151..85cb2df9deab 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHints.java @@ -49,6 +49,8 @@ public class ResourceHints { private static final String MIN_RAM_URN = "beam:resources:min_ram_bytes:v1"; private static final String ACCELERATOR_URN = "beam:resources:accelerator:v1"; + private static final String CPU_COUNT_URN = "beam:resources:cpu_count:v1"; + // TODO: reference this from a common location in all packages that use this. private static String getUrn(ProtocolMessageEnum value) { return value.getValueDescriptor().getOptions().getExtension(RunnerApi.beamUrn); @@ -57,6 +59,7 @@ private static String getUrn(ProtocolMessageEnum value) { static { checkState(MIN_RAM_URN.equals(getUrn(StandardResourceHints.Enum.MIN_RAM_BYTES))); checkState(ACCELERATOR_URN.equals(getUrn(StandardResourceHints.Enum.ACCELERATOR))); + checkState(CPU_COUNT_URN.equals(getUrn(StandardResourceHints.Enum.CPU_COUNT))); } private static ImmutableMap hintNameToUrn = @@ -64,12 +67,15 @@ private static String getUrn(ProtocolMessageEnum value) { .put("minRam", MIN_RAM_URN) .put("min_ram", MIN_RAM_URN) // Courtesy alias. .put("accelerator", ACCELERATOR_URN) + .put("cpuCount", CPU_COUNT_URN) + .put("cpu_count", CPU_COUNT_URN) // Courtesy alias. .build(); private static ImmutableMap> parsers = ImmutableMap.>builder() .put(MIN_RAM_URN, s -> new BytesHint(BytesHint.parse(s))) .put(ACCELERATOR_URN, s -> new StringHint(s)) + .put(CPU_COUNT_URN, s -> new IntHint(IntHint.parse(s))) .build(); private static final ResourceHints EMPTY = new ResourceHints(ImmutableMap.of()); @@ -212,6 +218,46 @@ public int hashCode() { } } + /*package*/ static class IntHint extends ResourceHint { + private final int value; + + @Override + public boolean equals(@Nullable Object other) { + if (other == null) { + return false; + } else if (this == other) { + return true; + } else if (other instanceof IntHint) { + return ((IntHint) other).value == value; + } else { + return false; + } + } + + @Override + public int hashCode() { + return Integer.hashCode(value); + } + + public IntHint(int value) { + this.value = value; + } + + public static int parse(String s) { + return Integer.parseInt(s, 10); + } + + @Override + public ResourceHint mergeWithOuter(ResourceHint outer) { + return new IntHint(Math.max(value, ((IntHint) outer).value)); + } + + @Override + public byte[] toBytes() { + return String.valueOf(value).getBytes(Charsets.US_ASCII); + } + } + /** * Sets desired minimal available RAM size to have in transform's execution environment. * @@ -264,6 +310,23 @@ public ResourceHints withHint(String urn, ResourceHint hint) { return new ResourceHints(newHints.build()); } + /** + * Sets desired minimal CPU or vCPU count to have in transform's execution environment. + * + * @param cpuCount specifies a positive CPU count. + */ + public ResourceHints withCPUCount(int cpuCount) { + if (cpuCount <= 0) { + LOG.error( + "Encountered invalid non-positive cpu count hint value {}.\n" + + "The value is ignored. In the future, The method will require an object Long type " + + "and throw an IllegalArgumentException for invalid values.", + cpuCount); + return this; + } + return withHint(CPU_COUNT_URN, new IntHint(cpuCount)); + } + public Map hints() { return hints; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHintsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHintsTest.java index 3cc522176374..c7643f718aa5 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHintsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/resourcehints/ResourceHintsTest.java @@ -92,10 +92,13 @@ public void testFromOptions() { .withHint("beam:resources:bar", new ResourceHints.StringHint("foo"))); options = PipelineOptionsFactory.fromArgs( - "--resourceHints=min_ram=1KB", "--resourceHints=accelerator=foo") + "--resourceHints=min_ram=1KB", + "--resourceHints=accelerator=foo", + "--resourceHints=cpu_count=4") .as(ResourceHintsOptions.class); - assertEquals( - ResourceHints.fromOptions(options), - ResourceHints.create().withMinRam(1000).withAccelerator("foo")); + ResourceHints fromOptions = ResourceHints.fromOptions(options); + ResourceHints expect = + ResourceHints.create().withMinRam(1000).withAccelerator("foo").withCPUCount(4); + assertEquals(fromOptions, expect); } } diff --git a/sdks/python/apache_beam/transforms/resources.py b/sdks/python/apache_beam/transforms/resources.py index 7bb202ab5660..7c4160df8edd 100644 --- a/sdks/python/apache_beam/transforms/resources.py +++ b/sdks/python/apache_beam/transforms/resources.py @@ -42,6 +42,7 @@ 'ResourceHint', 'AcceleratorHint', 'MinRamHint', + 'CpuCountHint', 'merge_resource_hints', 'parse_resource_hints', 'resource_hints_from_options', @@ -177,6 +178,21 @@ def get_merged_value( ResourceHint.register_resource_hint('minRam', MinRamHint) +class CpuCountHint(ResourceHint): + """Describes number of CPUs available in transform's execution environment.""" + urn = resource_hints.CPU_COUNT.urn + + @classmethod + def get_merged_value( + cls, outer_value, inner_value): # type: (bytes, bytes) -> bytes + return ResourceHint._use_max(outer_value, inner_value) + + +ResourceHint.register_resource_hint('cpu_count', CpuCountHint) +# Alias for interoperability with SDKs preferring camelCase. +ResourceHint.register_resource_hint('cpuCount', CpuCountHint) + + def parse_resource_hints(hints): # type: (Dict[Any, Any]) -> Dict[str, bytes] parsed_hints = {} for hint, value in hints.items(): diff --git a/sdks/python/apache_beam/transforms/resources_test.py b/sdks/python/apache_beam/transforms/resources_test.py index 939391b7adcb..939bdcd62651 100644 --- a/sdks/python/apache_beam/transforms/resources_test.py +++ b/sdks/python/apache_beam/transforms/resources_test.py @@ -46,6 +46,11 @@ class ResourcesTest(unittest.TestCase): val='gpu', urn='beam:resources:accelerator:v1', bytestr=b'gpu'), + param( + name='cpu_count', + val='4', + urn='beam:resources:cpu_count:v1', + bytestr=b'4'), ]) def test_known_resource_hints(self, name, val, urn, bytestr): t = PTransform() @@ -56,6 +61,7 @@ def test_known_resource_hints(self, name, val, urn, bytestr): @parameterized.expand([ param(name='min_ram', val='3,500G'), param(name='accelerator', val=1), + param(name='cpu_count', val=1), param(name='unknown_hint', val=1) ]) def test_resource_hint_parsing_fails_early(self, name, val): From 835bd655948d9a8ebc47277dedb25fb4c619f239 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 12 Oct 2023 09:25:37 -0400 Subject: [PATCH 83/85] Add docs for per key inference (#28243) * Update KeyMhMapping to KeyModelMapping * Add docs for per key inference * Add piece on memory thrashing * Whitespace * Update wording based on feedback * Add references to website in pydoc * Apply suggestions from code review Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> * Remove ordering implied by wording * Lint fixes --------- Co-authored-by: Rebecca Szper <98840847+rszper@users.noreply.github.com> --- sdks/python/apache_beam/ml/inference/base.py | 9 +++- .../sdks/python-machine-learning.md | 48 +++++++++++++++++++ 2 files changed, 56 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/ml/inference/base.py b/sdks/python/apache_beam/ml/inference/base.py index 90d43cfddb94..753e1468137c 100644 --- a/sdks/python/apache_beam/ml/inference/base.py +++ b/sdks/python/apache_beam/ml/inference/base.py @@ -482,6 +482,12 @@ def __init__( from the cohort. When model updates occur, the metrics will be reported in the form `--`. + Loading multiple models at the same time can increase the risk of an out of + memory (OOM) exception. To avoid this issue, use the parameter + `max_models_per_worker_hint` to limit the number of models that are loaded + at the same time. For more information about memory management, see + `Use a keyed `ModelHandler _`. # pylint: disable=line-too-long + Args: unkeyed: Either (a) an implementation of ModelHandler that does not @@ -491,7 +497,8 @@ def __init__( models can be held in memory at one time per worker process. For example, if your worker has 8 GB of memory provisioned and your workers take up 1 GB each, you should set this to 7 to allow all models to sit - in memory with some buffer. + in memory with some buffer. For more information about memory management, + see `Use a keyed `ModelHandler _`. # pylint: disable=line-too-long """ self._metrics_collectors: Dict[str, _MetricsCollector] = {} self._default_metrics_collector: _MetricsCollector = None diff --git a/website/www/site/content/en/documentation/sdks/python-machine-learning.md b/website/www/site/content/en/documentation/sdks/python-machine-learning.md index 5e0cf483ff3e..0076fa370b0f 100644 --- a/website/www/site/content/en/documentation/sdks/python-machine-learning.md +++ b/website/www/site/content/en/documentation/sdks/python-machine-learning.md @@ -215,6 +215,54 @@ with pipeline as p: If you are unsure if your data is keyed, you can also use `MaybeKeyedModelHandler`. +You can also use a `KeyedModelHandler` to load several different models based on their associated key: + +``` +from apache_beam.ml.inference.base import KeyedModelHandler +keyed_model_handler = KeyedModelHandler([ + KeyModelMapping(['key1'], PytorchModelHandlerTensor()), + KeyModelMapping(['key2', 'key3'], PytorchModelHandlerTensor()) +]) +with pipeline as p: + data = p | beam.Create([ + ('key1', torch.tensor([[1,2,3],[4,5,6],...])), + ('key2', torch.tensor([[1,2,3],[4,5,6],...])), + ('key3', torch.tensor([[1,2,3],[4,5,6],...])), + ]) + predictions = data | RunInference(keyed_model_handler) +``` + +The previous example loads a model by using `config1`. That model is then used for inference for all examples associated +with `key1`. It also loads a model by using `config2`. That model is used for all examples associated with `key2` and `key3`. + +Loading multiple models at the same times increases the risk of out of memory (OOM) errors. By default, `KeyedModelHandler` doesn't +limit the number of models loaded into memory at the same time. If the models don't all fit into memory, +your pipeline will likely fail with an out of memory error. To avoid this issue, provide a hint about the +maximum number of models that can be loaded at the same time. + +``` +mhs = [ + KeyModelMapping(['key1'], PytorchModelHandlerTensor()), + KeyModelMapping(['key2', 'key3'], PytorchModelHandlerTensor()), + KeyModelMapping(['key4'], PytorchModelHandlerTensor()), + KeyModelMapping(['key5', 'key6', 'key7'], PytorchModelHandlerTensor()), +] +keyed_model_handler = KeyedModelHandler(mhs, max_models_per_worker_hint=2) +``` + +The previous example loads at most two models per SDK worker process at any given time. It unloads models that aren't +currently being used. Runners that have multiple SDK worker processes on a given machine load at most +`max_models_per_worker_hint*` models onto the machine. Leave enough space for the models +and any additional memory needs from other transforms. Because there might be a delay between when a model is offloaded and when the +memory is released, it is recommended that you leave additional buffer. + +**Note**: Having many models but a small `max_models_per_worker_hint` can lead to _memory thrashing_, where +a large amount of execution time is wasted swapping models in and out of memory. To reduce the likelihood and impact +of memory thrashing, if you're using a distributed runner, insert a +[GroupByKey](https://beam.apache.org/documentation/transforms/python/aggregation/groupbykey/) transform before your +inference step. This step reduces thrashing by ensuring that elements with the same key and model are +collocated on the same worker. + For more information, see [`KeyedModelHander`](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.KeyedModelHandler). ### Use the `PredictionResult` object From fbd147f83548f7586eccb0a364d7804318fbac86 Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Thu, 12 Oct 2023 09:27:14 -0400 Subject: [PATCH 84/85] Use latest released beam in notebook (#28801) --- examples/notebooks/beam-ml/per_key_models.ipynb | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/examples/notebooks/beam-ml/per_key_models.ipynb b/examples/notebooks/beam-ml/per_key_models.ipynb index b529449555d0..a5dfa0d54e31 100644 --- a/examples/notebooks/beam-ml/per_key_models.ipynb +++ b/examples/notebooks/beam-ml/per_key_models.ipynb @@ -107,12 +107,7 @@ } ], "source": [ - "# Note that this notebook currently installs from Beam head since this feature hasn't been released yet.\n", - "# It will be released with version 2.51.0, at which point you can install with the following command:\n", - "# !pip install apache_beam[gcp]>=2.51.0 --quiet\n", - "!git clone https://github.com/apache/beam\n", - "!pip install -r beam/sdks/python/build-requirements.txt\n", - "!pip install -e ./beam/sdks/python[gcp]\n", + "!pip install apache_beam[gcp]>=2.51.0 --quiet\n", "!pip install torch --quiet\n", "!pip install transformers --quiet\n", "\n", From 7449b67a6d1019c82d852a69ae8cb093199335e6 Mon Sep 17 00:00:00 2001 From: Rebecca Szper <98840847+rszper@users.noreply.github.com> Date: Thu, 12 Oct 2023 06:54:39 -0700 Subject: [PATCH 85/85] Editing notebooks to prepare for DevSite import (#28949) --- .../beam-ml/automatic_model_refresh.ipynb | 8 ++++---- .../notebooks/beam-ml/mltransform_basic.ipynb | 6 ++---- .../notebooks/beam-ml/per_key_models.ipynb | 18 +++++++++--------- 3 files changed, 15 insertions(+), 17 deletions(-) diff --git a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb index 9cbab0a14178..3bafa4f07887 100644 --- a/examples/notebooks/beam-ml/automatic_model_refresh.ipynb +++ b/examples/notebooks/beam-ml/automatic_model_refresh.ipynb @@ -248,7 +248,7 @@ " This example uses `TFModelHandlerTensor` as the model handler and the `resnet_101` model trained on [ImageNet](https://www.image-net.org/).\n", "\n", "\n", - "For DataflowRunner, the model needs to be stored remote location accessible by the Beam pipeline. So we will download `ResNet101` model and upload it to the GCS location.\n" + "For the Dataflow runner, you need to store the model in a remote location that the Apache Beam pipeline can access. For this example, download the `ResNet101` model, and upload it to the Google Cloud Storage bucket.\n" ], "metadata": { "id": "_AUNH_GJk_NE" @@ -392,7 +392,7 @@ "source": [ "2. To read and preprocess the images, use the `preprocess_image` function. This example uses `Cat-with-beanie.jpg` for all inferences.\n", "\n", - " **Note**: Image used for prediction is licensed in CC-BY. The creator is listed in the [LICENSE.txt](https://storage.googleapis.com/apache-beam-samples/image_captioning/LICENSE.txt) file." + " **Note**: The image used for prediction is licensed in CC-BY. The creator is listed in the [LICENSE.txt](https://storage.googleapis.com/apache-beam-samples/image_captioning/LICENSE.txt) file." ], "metadata": { "id": "8-sal2rFAxP2" @@ -424,7 +424,7 @@ "cell_type": "markdown", "source": [ "3. Pass the images to the RunInference `PTransform`. RunInference takes `model_handler` and `model_metadata_pcoll` as input parameters.\n", - " * `model_metadata_pcoll` is a side input `PCollection` to the RunInference `PTransform`. This side input is used to update the `model_uri` in the `model_handler` without needing to stop the Apache Beam pipeline\n", + " * `model_metadata_pcoll` is a side input `PCollection` to the RunInference `PTransform`. This side input updates the `model_uri` in the `model_handler` while the Apache Beam pipeline runs.\n", " * Use `WatchFilePattern` as side input to watch a `file_pattern` matching `.keras` files. In this case, the `file_pattern` is `'gs://BUCKET_NAME/dataflow/*keras'`.\n", "\n" ], @@ -483,7 +483,7 @@ "source": [ "### Watch for the model update\n", "\n", - "After the pipeline starts processing data and when you see output emitted from the RunInference `PTransform`, upload a `resnet152` model saved in `.keras` format to a Google Cloud Storage bucket location that matches the `file_pattern` you defined earlier.\n" + "After the pipeline starts processing data, when you see output emitted from the RunInference `PTransform`, upload a `resnet152` model saved in the `.keras` format to a Google Cloud Storage bucket location that matches the `file_pattern` you defined earlier.\n" ], "metadata": { "id": "wYp-mBHHjOjA" diff --git a/examples/notebooks/beam-ml/mltransform_basic.ipynb b/examples/notebooks/beam-ml/mltransform_basic.ipynb index 820bc3400b58..fd305bddb3ba 100644 --- a/examples/notebooks/beam-ml/mltransform_basic.ipynb +++ b/examples/notebooks/beam-ml/mltransform_basic.ipynb @@ -65,7 +65,7 @@ "id": "d3b81cf2-8603-42bd-995e-9e14631effd0" }, "source": [ - "This notebook demonstrates how to use `MLTransform` to preprocess your data for machine learning models. `MLTransform` is a `PTransform` that wraps multiple Apache Beam data processing transforms. As a result, `MLTransform` gives you the ability to preprocess different types of data in multiple ways with one transform.\n", + "This notebook demonstrates how to use `MLTransform` to preprocess your data for machine learning models. `MLTransform` is a `PTransform` that wraps multiple Apache Beam data processing transforms. With `MLTransform`, you can preprocess different types of data in multiple ways with one transform.\n", "\n", "This notebook uses data processing transforms defined in the [apache_beam/ml/transforms/tft](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.transforms.tft.html) module." ] @@ -423,8 +423,6 @@ "source": [ "### Scale the data by using the z-score\n", "\n", - "Scale to the data using the z-score\n", - "\n", "Similar to `ScaleTo01`, use [ScaleToZScore](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.transforms.tft.html#apache_beam.ml.transforms.tft.ScaleToZScore) to scale the values by using the [z-score]([z-score](https://www.tensorflow.org/tfx/transform/api_docs/python/tft/scale_to_z_score#:~:text=Scaling%20to%20z%2Dscore%20subtracts%20out%20the%20mean%20and%20divides%20by%20standard%20deviation.%20Note%20that%20the%20standard%20deviation%20computed%20here%20is%20based%20on%20the%20biased%20variance%20(0%20delta%20degrees%20of%20freedom)%2C%20as%20computed%20by%20analyzers.var.).\n" ], "metadata": { @@ -607,7 +605,7 @@ "\n", "The previous examples show how to preprocess data for model training. This example uses the same preprocessing steps on the inference data. By using the same steps on the inference data, you can maintain consistent results.\n", "\n", - "Preprocess the data going into the inference by using the same preprocessing steps used on the data prior to training. To do this with `MLTransform`, pass the artifact location from the previous transforms to the parameter `read_artifact_location`. `MLTransform` uses the values and artifacts produced in the previous steps. You don't need to provide the transforms, because they are saved with the artifacts in the artifact location.\n" + "Preprocess the data used by the inference by using the same preprocessing steps that you used on the data prior to training. When using `MLTransform`, pass the artifact location from the previous transforms to the parameter `read_artifact_location`. `MLTransform` uses the values and artifacts produced in the previous steps. You don't need to provide the transforms, because they are saved with the artifacts in the artifact location.\n" ], "metadata": { "id": "kcnQSwkA-eSA" diff --git a/examples/notebooks/beam-ml/per_key_models.ipynb b/examples/notebooks/beam-ml/per_key_models.ipynb index a5dfa0d54e31..53845c0b3e19 100644 --- a/examples/notebooks/beam-ml/per_key_models.ipynb +++ b/examples/notebooks/beam-ml/per_key_models.ipynb @@ -70,7 +70,7 @@ "\n", "In Apache Beam, the recommended way to run inference is to use the `RunInference` transform. By using a `KeyedModelHandler`, you can efficiently run inference with O(100s) of models without having to manage memory yourself.\n", "\n", - "This notebook demonstrates how to use a `KeyedModelHandler` to run inference in an Apache Beam pipeline with multiple different models on a per-key basis. This notebook uses pretrained pipelines from Hugging Face. Before continuing with this notebook, it is recommended that you walk through the [beginner RunInference notebook](https://colab.sandbox.google.com/github/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb)." + "This notebook demonstrates how to use a `KeyedModelHandler` to run inference in an Apache Beam pipeline with multiple different models on a per-key basis. This notebook uses pretrained pipelines from Hugging Face. Before continuing with this notebook, it is recommended that you walk through the [Use RunInference in Apache Beam](https://colab.sandbox.google.com/github/apache/beam/blob/master/examples/notebooks/beam-ml/run_inference_pytorch_tensorflow_sklearn.ipynb) notebook." ], "metadata": { "id": "ZAVOrrW2An1n" @@ -81,7 +81,7 @@ "source": [ "## Install dependencies\n", "\n", - "First, install both Apache Beam and the dependencies needed by Hugging Face." + "Install both Apache Beam and the dependencies needed by Hugging Face." ], "metadata": { "id": "_fNyheQoDgGt" @@ -144,7 +144,7 @@ "\n", "A model handler is the Apache Beam method used to define the configuration needed to load and invoke models. Because this example uses two models, we define two model handlers, one for each model. Because both models are incapsulated within Hugging Face pipelines, we use the model handler `HuggingFacePipelineModelHandler`.\n", "\n", - "In this notebook, we load the models using Hugging Face and run them against an example. The models produce different outputs." + "For this example, load the models using Hugging Face, and then run them against an example. The models produce different outputs." ], "metadata": { "id": "uEqljVgCD7hx" @@ -350,7 +350,7 @@ "source": [ "## Define the examples\n", "\n", - "Next, define examples to input into the pipeline. The examples include their correct classifications." + "Define examples to input into the pipeline. The examples include the correct classifications." ], "metadata": { "id": "yd92MC7YEsTf" @@ -387,7 +387,7 @@ "class FormatExamples(beam.DoFn):\n", " \"\"\"\n", " Map each example to a tuple of ('-', 'example').\n", - " We use these keys to map our elements to the correct models.\n", + " Use these keys to map our elements to the correct models.\n", " \"\"\"\n", " def process(self, element: Tuple[str, str]) -> Iterable[Tuple[str, str]]:\n", " yield (f'distilbert-{element[1]}', element[0])\n", @@ -402,7 +402,7 @@ { "cell_type": "markdown", "source": [ - "Use the formatted keys to define a `KeyedModelHandler` that maps keys to the `ModelHandler` used for those keys. The `KeyedModelHandler` method lets you define an optional `max_models_per_worker_hint`, which limits the number of models that can be held in a single worker process at one time. If you're worried about your worker running out of memory, use this option. For more information about managing memory, see [Use a keyed ModelHandler](https://beam.apache.org/documentation/sdks/python-machine-learning/index.html#use-a-keyed-modelhandler)." + "Use the formatted keys to define a `KeyedModelHandler` that maps keys to the `ModelHandler` used for those keys. The `KeyedModelHandler` method lets you define an optional `max_models_per_worker_hint`, which limits the number of models that can be held in a single worker process at one time. If your worker might run out of memory, use this option. For more information about managing memory, see [Use a keyed ModelHandler](https://beam.apache.org/documentation/sdks/python-machine-learning/index.html#use-a-keyed-modelhandler)." ], "metadata": { "id": "IP65_5nNGIb8" @@ -428,9 +428,9 @@ "source": [ "## Postprocess the results\n", "\n", - "The `RunInference` transform returns a Tuple containing:\n", + "The `RunInference` transform returns a tuple that contains the following objects:\n", "* the original key\n", - "* a `PredictionResult` object containing the original example and the inference.\n", + "* a `PredictionResult` object containing the original example and the inference\n", "Use those outputs to extract the relevant data. Then, to compare each model's prediction, group this data by the original example." ], "metadata": { @@ -505,7 +505,7 @@ "source": [ "## Run the pipeline\n", "\n", - "Put together all of the pieces to run a single Apache Beam pipeline." + "To run a single Apache Beam pipeline, combine the previous steps." ], "metadata": { "id": "-LrpmM2PGAkf"