From 9ad679f709ea233ac0b36d8209250a2cb4b5684a Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Wed, 7 May 2025 15:22:16 -0400 Subject: [PATCH 01/49] Add a test provider for specifiable and try it on AnomalyDetection. --- .../apache_beam/yaml/yaml_transform_test.py | 36 ++++++++++++++++++- 1 file changed, 35 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/yaml_transform_test.py b/sdks/python/apache_beam/yaml/yaml_transform_test.py index 1a99507d76d7..c5a57b2e3e88 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform_test.py +++ b/sdks/python/apache_beam/yaml/yaml_transform_test.py @@ -74,15 +74,26 @@ def raise_on_big(row): return {'small_elements': good, self._error_handling['output']: bad} +from apache_beam.ml.anomaly.specifiable import Spec +from apache_beam.ml.anomaly.transforms import AnomalyDetection +from apache_beam.ml.anomaly.transforms import Specifiable + +class SpecifiableTransform(beam.PTransform): + def __init__(self, spec: str): + self._spec = Spec(type='ZScore') + + def expand(self, pcoll): + return pcoll | AnomalyDetection(detector=Specifiable.from_spec(self._spec)) + TEST_PROVIDERS = { 'CreateInts': CreateInts, 'CreateTimestamped': CreateTimestamped, 'SumGlobally': SumGlobally, 'SizeLimiter': SizeLimiter, 'PyMap': lambda fn: beam.Map(python_callable.PythonCallableWithSource(fn)), + 'AnomalyDetection': SpecifiableTransform, } - class YamlTransformE2ETest(unittest.TestCase): def test_composite(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( @@ -155,6 +166,29 @@ def test_chain_with_source_sink(self): providers=TEST_PROVIDERS) assert_that(result, equal_to([41, 43, 47, 53, 61, 71, 83, 97, 113, 131])) + def test_specifiable_transform(self): + TRAIN_DATA = [ + (0, beam.Row(x=1)), + (0, beam.Row(x=2)), + (0, beam.Row(x=4)), + (0, beam.Row(x=9)), + ] + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + result = p | beam.Create(TRAIN_DATA) | YamlTransform( + ''' + type: chain + transforms: + - type: AnomalyDetection + config: + spec: "type:'Zscore', config:{}" + - type: PyMap + config: + fn: "lambda x: (x[1].predictions[0].label)" + ''', + providers=TEST_PROVIDERS) + assert_that(result, equal_to([-2, -2, 1, 1])) + def test_chain_with_root(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: From d0a1b35ec480e7f0e4e09ebcdc4079efc1e8e819 Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Tue, 3 Jun 2025 10:06:28 -0400 Subject: [PATCH 02/49] Add specifiable providers and extract the code and test into separate files. --- sdks/python/apache_beam/yaml/yaml_provider.py | 2 + .../apache_beam/yaml/yaml_specifiable.py | 70 +++++++++++++++++++ .../apache_beam/yaml/yaml_specifiable_test.py | 68 ++++++++++++++++++ .../apache_beam/yaml/yaml_transform_test.py | 36 +--------- 4 files changed, 141 insertions(+), 35 deletions(-) create mode 100644 sdks/python/apache_beam/yaml/yaml_specifiable.py create mode 100644 sdks/python/apache_beam/yaml/yaml_specifiable_test.py diff --git a/sdks/python/apache_beam/yaml/yaml_provider.py b/sdks/python/apache_beam/yaml/yaml_provider.py index 0609c4ef8df5..47539304adfd 100755 --- a/sdks/python/apache_beam/yaml/yaml_provider.py +++ b/sdks/python/apache_beam/yaml/yaml_provider.py @@ -1475,6 +1475,7 @@ def standard_providers(): from apache_beam.yaml.yaml_mapping import create_mapping_providers from apache_beam.yaml.yaml_join import create_join_providers from apache_beam.yaml.yaml_io import io_providers + from apache_beam.yaml.yaml_specifiable import create_specifiable_providers return merge_providers( YamlProviders.create_builtin_provider(), @@ -1483,6 +1484,7 @@ def standard_providers(): create_combine_providers(), create_join_providers(), io_providers(), + create_specifiable_providers(), load_providers(yaml_utils.locate_data_file('standard_providers.yaml'))) diff --git a/sdks/python/apache_beam/yaml/yaml_specifiable.py b/sdks/python/apache_beam/yaml/yaml_specifiable.py new file mode 100644 index 000000000000..f5b2045edb24 --- /dev/null +++ b/sdks/python/apache_beam/yaml/yaml_specifiable.py @@ -0,0 +1,70 @@ +# +# 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. +# + +import inspect + +import apache_beam as beam +from apache_beam.ml.anomaly.specifiable import Spec +from apache_beam.ml.anomaly.transforms import AnomalyDetection +from apache_beam.ml.anomaly.transforms import Specifiable +from apache_beam.yaml.yaml_provider import InlineProvider + + +def dict_to_maybe_specifiable(d: dict): + def specify_helper(d: dict): + ret = {} + for k, v in d.items(): + if isinstance(v, dict): + ret[k] = dict_to_maybe_specifiable(v) + else: + ret[k] = v + return ret + + if "type" in d and "config" in d and isinstance(d["config"], dict): + return Specifiable.from_spec( + Spec(type=d["type"], config=specify_helper(d["config"]))) + else: + return specify_helper(d) + + +class SpecifiableTransform(beam.PTransform): + AVAILABlE_TRANSFORMS = {} + + @staticmethod + def register(typ, cls): + SpecifiableTransform.AVAILABlE_TRANSFORMS[typ] = cls + + def __init__(self, **kwargs): + self._typ = inspect.currentframe().f_back.f_locals.get("type", None) + assert self._typ is not None + + self._kwargs = dict_to_maybe_specifiable(kwargs) + + def expand(self, pcoll): + if self._typ in SpecifiableTransform.AVAILABlE_TRANSFORMS: + return pcoll | SpecifiableTransform.AVAILABlE_TRANSFORMS[self._typ]( + **self._kwargs) + + +SpecifiableTransform.register("AnomalyDetection", AnomalyDetection) + + +def create_specifiable_providers(): + return InlineProvider({ + k: SpecifiableTransform + for k in SpecifiableTransform.AVAILABlE_TRANSFORMS + }) diff --git a/sdks/python/apache_beam/yaml/yaml_specifiable_test.py b/sdks/python/apache_beam/yaml/yaml_specifiable_test.py new file mode 100644 index 000000000000..6d9e0de75f97 --- /dev/null +++ b/sdks/python/apache_beam/yaml/yaml_specifiable_test.py @@ -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. +# + +import logging +import unittest + +import apache_beam as beam +from apache_beam.testing.util import assert_that +from apache_beam.testing.util import equal_to +from apache_beam.utils import python_callable +from apache_beam.yaml.yaml_transform import YamlTransform + +TEST_PROVIDERS = { + 'PyMap': lambda fn: beam.Map(python_callable.PythonCallableWithSource(fn)), +} + + +class YamlSpecifiableTransformTest(unittest.TestCase): + def test_specifiable_transform(self): + TRAIN_DATA = [ + (0, beam.Row(x=1)), + (0, beam.Row(x=2)), + (0, beam.Row(x=2)), + (0, beam.Row(x=4)), + (0, beam.Row(x=9)), + ] + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + result = p | beam.Create(TRAIN_DATA) | YamlTransform( + ''' + type: chain + transforms: + - type: AnomalyDetection + config: + detector: + type: 'ZScore' + config: + sub_stat_tracker: + type: 'IncLandmarkMeanTracker' + config: {} + stdev_tracker: + type: 'IncLandmarkStdevTracker' + config: {} + - type: PyMap + config: + fn: "lambda x: (x[1].predictions[0].label)" + ''', + providers=TEST_PROVIDERS) + assert_that(result, equal_to([-2, -2, 0, 1, 1])) + + +if __name__ == '__main__': + logging.getLogger().setLevel(logging.INFO) + unittest.main() diff --git a/sdks/python/apache_beam/yaml/yaml_transform_test.py b/sdks/python/apache_beam/yaml/yaml_transform_test.py index c5a57b2e3e88..1a99507d76d7 100644 --- a/sdks/python/apache_beam/yaml/yaml_transform_test.py +++ b/sdks/python/apache_beam/yaml/yaml_transform_test.py @@ -74,26 +74,15 @@ def raise_on_big(row): return {'small_elements': good, self._error_handling['output']: bad} -from apache_beam.ml.anomaly.specifiable import Spec -from apache_beam.ml.anomaly.transforms import AnomalyDetection -from apache_beam.ml.anomaly.transforms import Specifiable - -class SpecifiableTransform(beam.PTransform): - def __init__(self, spec: str): - self._spec = Spec(type='ZScore') - - def expand(self, pcoll): - return pcoll | AnomalyDetection(detector=Specifiable.from_spec(self._spec)) - TEST_PROVIDERS = { 'CreateInts': CreateInts, 'CreateTimestamped': CreateTimestamped, 'SumGlobally': SumGlobally, 'SizeLimiter': SizeLimiter, 'PyMap': lambda fn: beam.Map(python_callable.PythonCallableWithSource(fn)), - 'AnomalyDetection': SpecifiableTransform, } + class YamlTransformE2ETest(unittest.TestCase): def test_composite(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( @@ -166,29 +155,6 @@ def test_chain_with_source_sink(self): providers=TEST_PROVIDERS) assert_that(result, equal_to([41, 43, 47, 53, 61, 71, 83, 97, 113, 131])) - def test_specifiable_transform(self): - TRAIN_DATA = [ - (0, beam.Row(x=1)), - (0, beam.Row(x=2)), - (0, beam.Row(x=4)), - (0, beam.Row(x=9)), - ] - with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( - pickle_library='cloudpickle')) as p: - result = p | beam.Create(TRAIN_DATA) | YamlTransform( - ''' - type: chain - transforms: - - type: AnomalyDetection - config: - spec: "type:'Zscore', config:{}" - - type: PyMap - config: - fn: "lambda x: (x[1].predictions[0].label)" - ''', - providers=TEST_PROVIDERS) - assert_that(result, equal_to([-2, -2, 1, 1])) - def test_chain_with_root(self): with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( pickle_library='cloudpickle')) as p: From 37c8aeaf0bb2008fd278d172cc64bf1f1faaf0d4 Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Tue, 3 Jun 2025 10:37:38 -0400 Subject: [PATCH 03/49] Polish code and add some more comments --- .../apache_beam/yaml/yaml_specifiable.py | 23 ++++++++++++------- 1 file changed, 15 insertions(+), 8 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_specifiable.py b/sdks/python/apache_beam/yaml/yaml_specifiable.py index f5b2045edb24..b6bcced5b558 100644 --- a/sdks/python/apache_beam/yaml/yaml_specifiable.py +++ b/sdks/python/apache_beam/yaml/yaml_specifiable.py @@ -25,6 +25,7 @@ def dict_to_maybe_specifiable(d: dict): + # convert a dictionary itself or its values to specifiables def specify_helper(d: dict): ret = {} for k, v in d.items(): @@ -46,6 +47,7 @@ class SpecifiableTransform(beam.PTransform): @staticmethod def register(typ, cls): + '''Register a specifiable transform''' SpecifiableTransform.AVAILABlE_TRANSFORMS[typ] = cls def __init__(self, **kwargs): @@ -55,16 +57,21 @@ def __init__(self, **kwargs): self._kwargs = dict_to_maybe_specifiable(kwargs) def expand(self, pcoll): - if self._typ in SpecifiableTransform.AVAILABlE_TRANSFORMS: - return pcoll | SpecifiableTransform.AVAILABlE_TRANSFORMS[self._typ]( - **self._kwargs) + if self._typ not in SpecifiableTransform.AVAILABlE_TRANSFORMS: + raise ValueError(f"Unknown specifiable transform {self._typ}") + return pcoll | SpecifiableTransform.AVAILABlE_TRANSFORMS[self._typ]( + **self._kwargs) -SpecifiableTransform.register("AnomalyDetection", AnomalyDetection) + +class SpecifiableProvider(InlineProvider): + def __init__(self, transform_factories): + super().__init__(transform_factories=transform_factories) + for k, v in self._transform_factories.items(): + SpecifiableTransform.register(k, v) + # All specifiable transforms will use the same factory + self._transform_factories[k] = SpecifiableTransform def create_specifiable_providers(): - return InlineProvider({ - k: SpecifiableTransform - for k in SpecifiableTransform.AVAILABlE_TRANSFORMS - }) + return SpecifiableProvider({"AnomalyDetection": AnomalyDetection}) From a003a92160d20ac471da47424bed0db09b2f67cb Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Tue, 3 Jun 2025 14:40:00 -0400 Subject: [PATCH 04/49] Simply code change by overriding create_transform in provider --- .../apache_beam/yaml/yaml_specifiable.py | 65 +++++-------------- 1 file changed, 17 insertions(+), 48 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_specifiable.py b/sdks/python/apache_beam/yaml/yaml_specifiable.py index b6bcced5b558..cd2e44c0734c 100644 --- a/sdks/python/apache_beam/yaml/yaml_specifiable.py +++ b/sdks/python/apache_beam/yaml/yaml_specifiable.py @@ -15,63 +15,32 @@ # limitations under the License. # -import inspect - -import apache_beam as beam from apache_beam.ml.anomaly.specifiable import Spec from apache_beam.ml.anomaly.transforms import AnomalyDetection from apache_beam.ml.anomaly.transforms import Specifiable from apache_beam.yaml.yaml_provider import InlineProvider -def dict_to_maybe_specifiable(d: dict): - # convert a dictionary itself or its values to specifiables - def specify_helper(d: dict): - ret = {} - for k, v in d.items(): - if isinstance(v, dict): - ret[k] = dict_to_maybe_specifiable(v) - else: - ret[k] = v - return ret - - if "type" in d and "config" in d and isinstance(d["config"], dict): - return Specifiable.from_spec( - Spec(type=d["type"], config=specify_helper(d["config"]))) +def maybe_make_specifiable(v): + if isinstance(v, dict): + if "type" in v and "config" in v: + return Specifiable.from_spec( + Spec(type=v["type"], config=maybe_make_specifiable(v["config"]))) + else: + ret = {k: maybe_make_specifiable(v[k]) for k in v} + return ret else: - return specify_helper(d) - - -class SpecifiableTransform(beam.PTransform): - AVAILABlE_TRANSFORMS = {} - - @staticmethod - def register(typ, cls): - '''Register a specifiable transform''' - SpecifiableTransform.AVAILABlE_TRANSFORMS[typ] = cls - - def __init__(self, **kwargs): - self._typ = inspect.currentframe().f_back.f_locals.get("type", None) - assert self._typ is not None - - self._kwargs = dict_to_maybe_specifiable(kwargs) - - def expand(self, pcoll): - if self._typ not in SpecifiableTransform.AVAILABlE_TRANSFORMS: - raise ValueError(f"Unknown specifiable transform {self._typ}") - - return pcoll | SpecifiableTransform.AVAILABlE_TRANSFORMS[self._typ]( - **self._kwargs) + return v -class SpecifiableProvider(InlineProvider): - def __init__(self, transform_factories): - super().__init__(transform_factories=transform_factories) - for k, v in self._transform_factories.items(): - SpecifiableTransform.register(k, v) - # All specifiable transforms will use the same factory - self._transform_factories[k] = SpecifiableTransform +class SpecProvider(InlineProvider): + def create_transform(self, type, args, yaml_create_transform): + return self._transform_factories[type]( + **{ + k: maybe_make_specifiable(v) + for k, v in args.items() + }) def create_specifiable_providers(): - return SpecifiableProvider({"AnomalyDetection": AnomalyDetection}) + return SpecProvider({"AnomalyDetection": AnomalyDetection}) From 0ab00aa18102145f53da7cd77a7d6b0198e82216 Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Wed, 4 Jun 2025 14:49:48 -0400 Subject: [PATCH 05/49] Add support on callable in spec --- .../apache_beam/yaml/yaml_specifiable.py | 19 ++++++- .../apache_beam/yaml/yaml_specifiable_test.py | 55 +++++++++++++++++-- 2 files changed, 67 insertions(+), 7 deletions(-) diff --git a/sdks/python/apache_beam/yaml/yaml_specifiable.py b/sdks/python/apache_beam/yaml/yaml_specifiable.py index cd2e44c0734c..ace6f412779f 100644 --- a/sdks/python/apache_beam/yaml/yaml_specifiable.py +++ b/sdks/python/apache_beam/yaml/yaml_specifiable.py @@ -18,6 +18,8 @@ from apache_beam.ml.anomaly.specifiable import Spec from apache_beam.ml.anomaly.transforms import AnomalyDetection from apache_beam.ml.anomaly.transforms import Specifiable +from apache_beam.io.filesystems import FileSystems +from apache_beam.utils import python_callable from apache_beam.yaml.yaml_provider import InlineProvider @@ -26,9 +28,20 @@ def maybe_make_specifiable(v): if "type" in v and "config" in v: return Specifiable.from_spec( Spec(type=v["type"], config=maybe_make_specifiable(v["config"]))) - else: - ret = {k: maybe_make_specifiable(v[k]) for k in v} - return ret + + if "callable" in v: + if "path" in v or "name" in v: + raise ValueError( + "Cannot specify 'callable' with 'path' and 'name' for function.") + else: + return python_callable.PythonCallableWithSource(v["callable"]) + + if "path" in v and "name" in v: + return python_callable.PythonCallableWithSource.load_from_script( + FileSystems.open(v["path"]).read().decode(), v["name"]) + + ret = {k: maybe_make_specifiable(v[k]) for k in v} + return ret else: return v diff --git a/sdks/python/apache_beam/yaml/yaml_specifiable_test.py b/sdks/python/apache_beam/yaml/yaml_specifiable_test.py index 6d9e0de75f97..c8a3d4e482c6 100644 --- a/sdks/python/apache_beam/yaml/yaml_specifiable_test.py +++ b/sdks/python/apache_beam/yaml/yaml_specifiable_test.py @@ -17,18 +17,35 @@ import logging import unittest +from typing import Callable import apache_beam as beam from apache_beam.testing.util import assert_that from apache_beam.testing.util import equal_to from apache_beam.utils import python_callable from apache_beam.yaml.yaml_transform import YamlTransform +from apache_beam.ml.anomaly.base import AnomalyDetector +from apache_beam.ml.anomaly.specifiable import specifiable TEST_PROVIDERS = { 'PyMap': lambda fn: beam.Map(python_callable.PythonCallableWithSource(fn)), } +@specifiable +class FakeDetector(AnomalyDetector): # pylint: disable=unused-variable + def __init__(self, fn: Callable): + super().__init__() + self._fn = fn + + def learn_one(self, x: beam.Row) -> None: + pass + + def score_one(self, x: beam.Row) -> float: + v = next(iter(x)) + return self._fn(v) + + class YamlSpecifiableTransformTest(unittest.TestCase): def test_specifiable_transform(self): TRAIN_DATA = [ @@ -50,11 +67,13 @@ def test_specifiable_transform(self): type: 'ZScore' config: sub_stat_tracker: - type: 'IncLandmarkMeanTracker' - config: {} + type: 'IncSlidingMeanTracker' + config: + window_size: 5 stdev_tracker: - type: 'IncLandmarkStdevTracker' - config: {} + type: 'IncSlidingStdevTracker' + config: + window_size: 5 - type: PyMap config: fn: "lambda x: (x[1].predictions[0].label)" @@ -62,6 +81,34 @@ def test_specifiable_transform(self): providers=TEST_PROVIDERS) assert_that(result, equal_to([-2, -2, 0, 1, 1])) + def test_specifiable_transform_with_callable(self): + TRAIN_DATA = [ + (0, beam.Row(x=1)), + (0, beam.Row(x=2)), + (0, beam.Row(x=2)), + (0, beam.Row(x=4)), + (0, beam.Row(x=9)), + ] + with beam.Pipeline(options=beam.options.pipeline_options.PipelineOptions( + pickle_library='cloudpickle')) as p: + result = p | beam.Create(TRAIN_DATA) | YamlTransform( + ''' + type: chain + transforms: + - type: AnomalyDetection + config: + detector: + type: 'FakeDetector' + config: + fn: + callable: "lambda x: x * 10.0" + - type: PyMap + config: + fn: "lambda x: (x[1].predictions[0].score)" + ''', + providers=TEST_PROVIDERS) + assert_that(result, equal_to([10.0, 20.0, 20.0, 40.0, 90.0])) + if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) From 19b9e287586587cb806443e00bff21a816ec7cd9 Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Wed, 4 Jun 2025 15:01:37 -0400 Subject: [PATCH 06/49] Switch to use cloud pickler in the stateful dofn states. --- sdks/python/apache_beam/coders/coders.py | 9 +++++++++ sdks/python/apache_beam/ml/anomaly/transforms.py | 8 +++++--- 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/sdks/python/apache_beam/coders/coders.py b/sdks/python/apache_beam/coders/coders.py index d7074a9d1972..e1a5192a0867 100644 --- a/sdks/python/apache_beam/coders/coders.py +++ b/sdks/python/apache_beam/coders/coders.py @@ -59,6 +59,7 @@ from apache_beam.coders import coder_impl from apache_beam.coders.avro_record import AvroRecord +from apache_beam.internal import cloudpickle_pickler from apache_beam.portability import common_urns from apache_beam.portability import python_urns from apache_beam.portability.api import beam_runner_api_pb2 @@ -93,6 +94,7 @@ 'AvroGenericCoder', 'BooleanCoder', 'BytesCoder', + 'CloudpickleCoder', 'DillCoder', 'FastPrimitivesCoder', 'FloatCoder', @@ -854,6 +856,13 @@ def __hash__(self): return hash(type(self)) +class CloudpickleCoder(_PickleCoderBase): + """Coder using Apache Beam's vendored Cloudpickle pickler.""" + def _create_impl(self): + return coder_impl.CallbackCoderImpl( + cloudpickle_pickler.dumps, cloudpickle_pickler.loads) + + class _MemoizingPickleCoder(_PickleCoderBase): """Coder using Python's pickle functionality with memoization.""" def __init__(self, cache_size=16): diff --git a/sdks/python/apache_beam/ml/anomaly/transforms.py b/sdks/python/apache_beam/ml/anomaly/transforms.py index 5870878ec69c..ef5501b33786 100644 --- a/sdks/python/apache_beam/ml/anomaly/transforms.py +++ b/sdks/python/apache_beam/ml/anomaly/transforms.py @@ -25,7 +25,7 @@ from typing import Union import apache_beam as beam -from apache_beam.coders import DillCoder +from apache_beam.coders import CloudpickleCoder from apache_beam.ml.anomaly import aggregations from apache_beam.ml.anomaly.base import AggregationFn from apache_beam.ml.anomaly.base import AnomalyDetector @@ -57,7 +57,8 @@ class _ScoreAndLearnDoFn(beam.DoFn): then updates the model with the same data. It maintains the model state using Beam's state management. """ - MODEL_STATE_INDEX = ReadModifyWriteStateSpec('saved_model', DillCoder()) + MODEL_STATE_INDEX = ReadModifyWriteStateSpec( + 'saved_model', CloudpickleCoder()) def __init__(self, detector_spec: Spec): self._detector_spec = detector_spec @@ -227,7 +228,8 @@ class _StatefulThresholdDoFn(_BaseThresholdDoFn): AssertionError: If the provided `threshold_fn_spec` leads to the creation of a stateless `ThresholdFn`. """ - THRESHOLD_STATE_INDEX = ReadModifyWriteStateSpec('saved_tracker', DillCoder()) + THRESHOLD_STATE_INDEX = ReadModifyWriteStateSpec( + 'saved_tracker', CloudpickleCoder()) def __init__(self, threshold_fn_spec: Spec): assert isinstance(threshold_fn_spec.config, dict) From 13e34acac4fbf73c4bb33097617044eb0581c3b5 Mon Sep 17 00:00:00 2001 From: Ahmed Abualsaud <65791736+ahmedabu98@users.noreply.github.com> Date: Mon, 2 Jun 2025 07:44:43 -0400 Subject: [PATCH 07/49] update container tag (#35097) --- sdks/python/apache_beam/runners/dataflow/internal/names.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/runners/dataflow/internal/names.py b/sdks/python/apache_beam/runners/dataflow/internal/names.py index d9e225854954..82c1fce7c337 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-20250418' +BEAM_DEV_SDK_CONTAINER_TAG = 'beam-master-20250530' DATAFLOW_CONTAINER_IMAGE_REPOSITORY = 'gcr.io/cloud-dataflow/v1beta3' From eda379d1fb52fad972eccd14977318d730f8b7e1 Mon Sep 17 00:00:00 2001 From: Bhargavkonidena Date: Mon, 2 Jun 2025 18:14:26 +0530 Subject: [PATCH 08/49] Fix for issue 34772 to include user provided providers (#35033) * Fix for issue 34772 to include user provided providers * Fix CI checks * Incorporate review comments --- sdks/python/apache_beam/yaml/generate_yaml_docs.py | 13 ++++++++++++- 1 file changed, 12 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/yaml/generate_yaml_docs.py b/sdks/python/apache_beam/yaml/generate_yaml_docs.py index 592177dc2bc0..ae1a04661e62 100644 --- a/sdks/python/apache_beam/yaml/generate_yaml_docs.py +++ b/sdks/python/apache_beam/yaml/generate_yaml_docs.py @@ -32,6 +32,7 @@ from apache_beam.version import __version__ as beam_version from apache_beam.yaml import json_utils from apache_beam.yaml import yaml_provider +from apache_beam.yaml import yaml_utils from apache_beam.yaml.yaml_errors import ErrorHandlingConfig @@ -263,7 +264,12 @@ def create_index(include, exclude, options): with subprocess_server.SubprocessServer.cache_subprocesses(): json_config_schemas = [] markdown_out = io.StringIO() - providers = yaml_provider.standard_providers() + if options.provider_source_file: + providers = yaml_provider.merge_providers( + yaml_provider.load_providers( + yaml_utils.locate_data_file(options.provider_source_file))) + else: + providers = yaml_provider.standard_providers() for transform_base, transforms in itertools.groupby( sorted(providers.keys(), key=io_grouping_key), key=lambda s: s.split('-')[0]): @@ -557,6 +563,11 @@ def main(): parser.add_argument('--schema_file') parser.add_argument('--include', default='.*') parser.add_argument('--exclude', default='') + parser.add_argument( + "--provider_source_file", + help="Path to a YAML file containing custom providers. " + "If not provided, uses standard Beam providers.", + ) options = parser.parse_args() include = re.compile(options.include).match exclude = ( From cd32138afda780d2a539d2d181c2033f9439bc0b Mon Sep 17 00:00:00 2001 From: Derrick Williams Date: Mon, 2 Jun 2025 09:51:56 -0400 Subject: [PATCH 09/49] update test your pipeline with TestPipeline with Rule (#35105) --- .../content/en/documentation/pipelines/test-your-pipeline.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/website/www/site/content/en/documentation/pipelines/test-your-pipeline.md b/website/www/site/content/en/documentation/pipelines/test-your-pipeline.md index 82f5e33ed07a..512b58735ac9 100644 --- a/website/www/site/content/en/documentation/pipelines/test-your-pipeline.md +++ b/website/www/site/content/en/documentation/pipelines/test-your-pipeline.md @@ -287,6 +287,8 @@ The following example code shows how one might test the [WordCount example pipel {{< highlight java >}} public class WordCountTest { + @Rule public final transient TestPipeline p = TestPipeline.create(); + // Our static input data, which will comprise the initial PCollection. static final String[] WORDS_ARRAY = new String[] { "hi there", "hi", "hi sue bob", @@ -301,8 +303,6 @@ public class WordCountTest { // Example test that tests the pipeline's transforms. public void testCountWords() throws Exception { - Pipeline p = TestPipeline.create(); - // Create a PCollection from the WORDS static input data. PCollection input = p.apply(Create.of(WORDS)); From 86213beb8cf5c3e723c294f1551974e844fa8e89 Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Mon, 2 Jun 2025 18:29:47 +0400 Subject: [PATCH 10/49] Add license for kafka-python (#35116) --- sdks/python/container/license_scripts/dep_urls_py.yaml | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sdks/python/container/license_scripts/dep_urls_py.yaml b/sdks/python/container/license_scripts/dep_urls_py.yaml index 0fe830b7ab6e..da10163fdb4f 100644 --- a/sdks/python/container/license_scripts/dep_urls_py.yaml +++ b/sdks/python/container/license_scripts/dep_urls_py.yaml @@ -94,6 +94,8 @@ pip_dependencies: license: "https://raw.githubusercontent.com/mtth/hdfs/master/LICENSE" httplib2: license: "https://raw.githubusercontent.com/httplib2/httplib2/master/LICENSE" + kafka-python: + license: "https://raw.githubusercontent.com/dpkp/kafka-python/master/LICENSE" keras: license: "https://raw.githubusercontent.com/keras-team/keras/master/LICENSE" keras-nightly: From 1c174b1ca54db1283a328e24b01cf1f3fbac86bf Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Mon, 2 Jun 2025 12:01:36 -0400 Subject: [PATCH 11/49] Update dataflow containers (#35118) --- runners/google-cloud-dataflow-java/build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 3aa1713fb6ed..9e24c0db09cf 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -53,8 +53,8 @@ evaluationDependsOn(":sdks:java:container:java11") ext.dataflowLegacyEnvironmentMajorVersion = '8' ext.dataflowFnapiEnvironmentMajorVersion = '8' -ext.dataflowLegacyContainerVersion = 'beam-master-20250326' -ext.dataflowFnapiContainerVersion = 'beam-master-20250326' +ext.dataflowLegacyContainerVersion = 'beam-master-20250602' +ext.dataflowFnapiContainerVersion = 'beam-master-20250602' ext.dataflowContainerBaseRepository = 'gcr.io/cloud-dataflow/v1beta3' processResources { From bdf31e78c0df19a2775fdd073eb356f9f704d19b Mon Sep 17 00:00:00 2001 From: bullet03 Date: Mon, 2 Jun 2025 19:16:54 +0200 Subject: [PATCH 12/49] [Website] add akvelon case study (#34943) * feat: add akvelon logo * feat: add akvelon case study * fix: remove white space * feat: add akvelon to main page * feat: use new images * fix: typos * fix: change order of akvelon case-study * fix: update text * fix: update mainPage text * fix: update images * fix: about akvelon section update * fix: update akvelon card * fix: update akvelon header * fix: update code tag * fix: update about akvelon * fix: update date and order * fix: add link and change img * fix: change CDAP text * fix: add bold weight * fix: solve conflicts * fix: remove unused code * fix: delete whitespace * fix: indents format * fix: add bold text --------- Co-authored-by: Bulat Safiullin --- .../site/content/en/case-studies/akvelon.md | 173 +++++++++++++++++- website/www/site/data/en/quotes.yaml | 4 + .../images/case-study/akvelon/chitnis.png | Bin 0 -> 140126 bytes .../images/case-study/akvelon/diagram-01.png | Bin 0 -> 105049 bytes .../images/case-study/akvelon/diagram-02.png | Bin 0 -> 73850 bytes .../images/case-study/akvelon/pikle.png | Bin 0 -> 502346 bytes .../images/case-study/akvelon/terentyev.png | Bin 0 -> 815004 bytes 7 files changed, 173 insertions(+), 4 deletions(-) create mode 100644 website/www/site/static/images/case-study/akvelon/chitnis.png create mode 100644 website/www/site/static/images/case-study/akvelon/diagram-01.png create mode 100644 website/www/site/static/images/case-study/akvelon/diagram-02.png create mode 100644 website/www/site/static/images/case-study/akvelon/pikle.png create mode 100644 website/www/site/static/images/case-study/akvelon/terentyev.png diff --git a/website/www/site/content/en/case-studies/akvelon.md b/website/www/site/content/en/case-studies/akvelon.md index b119bafc770b..7ba691869664 100644 --- a/website/www/site/content/en/case-studies/akvelon.md +++ b/website/www/site/content/en/case-studies/akvelon.md @@ -1,8 +1,17 @@ --- -title: "Akvelon" -icon: /images/logos/powered-by/akvelon.png -hasNav: true -cardDescription: "

Akvelon is a software engineering company that helps start-ups, SMBs, and Fortune 500 companies unlock the full potential of cloud, data, and AI/ML to empower their strategic advantage. Akvelon team has deep expertise in integrating Apache Beam with diverse data processing ecosystems and is an enthusiastic Apache Beam community contributor.

" +title: "Secure and Interoperable Apache Beam Pipelines by Akvelon" +name: "Akvelon" +icon: "/images/logos/powered-by/akvelon.png" +category: "study" +cardTitle: "Secure and Interoperable Apache Beam Pipelines by Akvelon" +cardDescription: "To support data privacy and pipeline reusability at scale, Akvelon developed Beam-based solutions for Protegrity and a major North American credit reporting company, enabling tokenization with Dataflow Flex Templates. Akvelon also built a CDAP Connector to integrate CDAP plugins with Apache Beam, enabling plugin reuse and multi-runtime compatibility." +authorName: "Vitaly Terentyev" +coauthorName: "Ashley Pikle" +authorPosition: "Software Engineer @Akvelon" +coauthorPosition: "Director of AI Business Development @Akvelon" +authorImg: /images/case-study/akvelon/terentyev.png +coauthorImg: /images/case-study/akvelon/pikle.png +publishDate: 2025-05-25T00:12:00+00:00 --- +
+
+ +
+
+

+ “To support data privacy and pipeline reusability at scale, Akvelon developed Beam-based solutions for Protegrity and a major North American credit reporting company, enabling tokenization with Dataflow Flex Templates. Akvelon also built a CDAP Connector to integrate CDAP plugins with Apache Beam, enabling plugin reuse and multi-runtime compatibility.” +

+
+
+ +
+
+
+ Ashley Pikle +
+
+ Director of AI Business Development @Akvelon +
+
+
+
+
+
+ +# Secure and Interoperable Apache Beam Pipelines by Akvelon + +## Background + +To meet growing enterprise needs for secure, scalable, and interoperable data processing pipelines, **Akvelon** developed multiple Apache Beam-powered solutions tailored for real-world production environments: +- Data tokenization and detokenization capabilities for **Protegrity** and a leading North American credit reporting company +- A connector layer to integrate **CDAP** plugins into Apache Beam pipelines + +By leveraging [Apache Beam](https://beam.apache.org/) and [Google Cloud Dataflow](https://cloud.google.com/products/dataflow?hl=en), Akvelon enabled its clients to achieve scalable data protection, regulatory compliance, and platform interoperability through reusable, open-source pipeline components. + +## Use Case 1: Data Tokenization for Protegrity and a Leading Credit Reporting Company + +### The Challenge + +**Protegrity**, a leading enterprise data-security vendor, sought to enhance its data protection platform with scalable tokenization support for batch and streaming data. Their goal: allow customers such as a major North American credit reporting company to tokenize sensitive data using Google Cloud Dataflow. The solution needed to be fast, secure, reusable, and compliant with privacy regulations (e.g., HIPAA, GDPR). + +### The Solution + +Akvelon designed and implemented a **Dataflow Flex Template** using Apache Beam that allows users to tokenize and detokenize sensitive data within both batch and streaming pipelines. + +
+ + Protegrity & Equifax Tokenization Pipeline + +
+ +### Key features +- **Seamless integration with Protegrity UDFs**, enabling native tokenization directly within Beam transforms without requiring external service orchestration +- **Support for multiple data formats** such as CSV, JSON, Parquet, allowing flexible deployment across diverse data pipelines +- **Stateful processing with `DoFn` and timers**, which improves streaming reliability and reduces overall pipeline latency +- **Full compatibility with Google Cloud Dataflow**, ensuring autoscaling, fault tolerance, and operational simplicity through managed Apache Beam execution + +This design provided both Protegrity and its enterprise clients with a reusable, open-source architecture for scalable data privacy and processing. + +### The Results +- **Enabled data tokenization at** for regulated industries +- **Accelerated adoption of Dataflow templates** across Protegrity’s customer base +- **Delivered an [open-source Flex Template](https://github.com/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md)** that benefits the entire Apache Beam community + +
+

+ In collaboration with Akvelon, Protegrity utilized a Dataflow Flex template that helps us enable customers to tokenize and detokenize streaming and batch data from a fully managed Google Cloud Dataflow service. We appreciate Akvelon’s support as a trusted partner with Google Cloud expertise. +

+
+
+ +
+
+
+ Jay Chitnis +
+
+ VP of Partners and Business Development @Protegrity +
+
+
+
+ +## Use Case 2: CDAP Connector for Apache Beam + +### The Challenge + +**CDAP** had extensive plugin support for Spark but lacked native compatibility with Apache Beam. This limitation prevented organizations from reusing CDAP's rich ecosystem of data connectors (e.g., Salesforce, HubSpot, ServiceNow) within Beam-based pipelines, constraining cross-platform integration. + +### The Solution + +Akvelon engineered a **shim layer** (CDAP Connector) that bridges CDAP plugins with Apache Beam. This innovation enables CDAP source and sink plugins to operate seamlessly within Beam pipelines. + +
+ + CDAP Connector Integration with Apache Beam + +
+ +### Highlights + +- Supports `StructuredRecord` format conversion to Beam schema (`BeamRow`) +- Enables CDAP plugins to run seamlessly in both Spark and Beam pipelines +- Facilitates integration testing across third-party data sources (e.g., Salesforce, Zendesk) +- Complies with Beam’s development and style guide for open-source contributions + +The project included prototyping, test infrastructure, and Salesforce plugin pipelines to ensure robustness. + +### The Results + +- **Enabled seamless reuse of CDAP plugins in Beam** + - **30+ CDAP plugins** now work seamlessly with Beam pipelines + - **Integration time** reduced **from hours to just a few minutes** +- **Simplified execution and migration of CDAP pipelines to Beam** + - Enabled **seamless execution** of CDAP pipelines on the Beam runtime + - Simplified **migration of existing CDAP pipelines** to Beam with minimal changes +- **Accelerated delivery and validated performance for Google Cloud customers** + - Delivered **rapid development cycles** with standardized plugin configurations + - Successfully processed **5 million records** in end-to-end tests for **batch and streaming** + +## Technology Stack + +- Apache Beam +- Google Cloud Dataflow +- Protegrity Data Protection Platform +- CDAP (Cloud Data Fusion) +- BigQuery +- Salesforce, Zendesk, HubSpot, ServiceNow plugins + +## Final words + +Akvelon’s contributions to Apache Beam-based solutions - from advanced tokenization for Protegrity and its enterprise customers to enabling plugin interoperability through the CDAP Connector - demonstrate the value of open-source, cloud-native data engineering. By delivering reusable and secure components, Akvelon supports enterprises in modernizing and unifying their data infrastructure. + +## Watch the Solution in Action + +[Architecture Walkthrough Video ](https://www.youtube.com/watch?v=IQIzdfNIAHk) + +## About Akvelon, Inc. + +Akvelon guides enterprises through digital transformation on Google Cloud - applying deep expertise in data engineering, AI/ML, cloud infrastructure, and custom application development to design, deploy, and scale modern workloads. + +At Akvelon, we’ve built a long-standing partnership with Google Cloud—helping software-driven organizations implement, migrate, modernize, automate, and optimize their systems while making the most of cloud technologies. + +As a **Google Cloud Service** and **Build Partner**, we contribute actively to the ecosystem: +- Contributing code and guidance to **Apache Beam**—including Playground, Tour of Beam, and the Duet AI training set +- Improving project infrastructure and supporting the Apache Beam community—now with an official Apache Beam Committer on our team + +Backed by deep expertise in data engineering, AI/ML, cloud architecture, and application development, our engineers deliver reusable, secure, and production-ready solutions on Google Cloud for enterprises worldwide. + +- [Akvelon on Google Cloud](https://cloud.google.com/find-a-partner/partner/akvelon) +- [Akvelon Data and Analytics Accelerators](https://github.com/akvelon/DnA_accelerators) + +{{< case_study_feedback "Akvelon" >}} + +
+
diff --git a/website/www/site/data/en/quotes.yaml b/website/www/site/data/en/quotes.yaml index b424ece4a1bb..936b7472d333 100644 --- a/website/www/site/data/en/quotes.yaml +++ b/website/www/site/data/en/quotes.yaml @@ -81,6 +81,10 @@ logoUrl: /images/logos/powered-by/accenture.png linkUrl: case-studies/accenture_baltics/index.html linkText: Learn more +- text: Akvelon built Beam-based solutions for Protegrity and a major North American credit reporting company, enabling tokenization with Dataflow Flex Templates and reducing infrastructure and deployment complexity. + icon: icons/quote-icon.svg + logoUrl: /images/logos/powered-by/akvelon.png + linkUrl: case-studies/akvelon/index.html - text: With Apache Beam and Dataflow, Credit Karma achieved a 99% uptime for critical data pipelines, a significant jump from 80%. This reliability, coupled with faster development (1 engineer vs. 3 estimated), has been crucial for enabling real-time financial insights for our more than 140 million members. icon: icons/quote-icon.svg logoUrl: images/logos/powered-by/credit-karma.png diff --git a/website/www/site/static/images/case-study/akvelon/chitnis.png b/website/www/site/static/images/case-study/akvelon/chitnis.png new file mode 100644 index 0000000000000000000000000000000000000000..13608fd325cc5ffcbcdeade57ff93a6daf7f28df GIT binary patch literal 140126 zcmW(+WmpvN+of5MSh``CkR^XeNypN?lpqR-gn%H3(z&p-bT><U>zZqx`;x*ni0|l`_RR9AtffK(|0b2nkuPxs^KfX`#^U=45O9zTq zoW`S4f4zGDMVPSCWJtflIOh2{{cncAr&U*%8oZ68zt=RQ>F$3sQXE^4kKHvrlUnM= z*QBX{Es)3j7OqDJq=Ej-3;Kks!aGWbC*29@Rg1fO>D9ZXH3}rbX*3+4* z^zHFltj}He+S3iyGiJ-_xoOK&jcL}+LPgZ`#p^!byQSOLJ}mnu`)xsgOzvk6s-9nt z&_DJJPnc?sJbyep8DszBBP?~VYi;+3ef-}=lv2Seh;XF&mB^!qBB6TAtn{hJ+6~?N zipxl*fX3d)s~-%945F*zXS8!)B@`^@Dsx7WEP0U~?nU<2t(G5t{`RGeDh~(-$J2Qh z(s>ks$n|Xhr4%1${@hPxX4p%Yn16*nQy(FVyV$1!nX8R_)9^!2h*%C1{f|R$LF8BT zl@P%}0wVQMLvGPLcCXa#fE}r>b^V%9CdZ^1>#+QItZ3y@k|tlwPRn@hNUml4XMp7R zDCUYDP|J1p>OgE<4SGA1dc-he3`|&E8Fr;2&zCCI7e3M0X1n&5)29D|w4i@WE!4>b##yrfevuAP3D})#{ z5NvRSaPjr1WG8JMixSL6A6Wg$cM2P46^%|?nf(LkbM{V6#&E!!!q_GjOD^c2$grSiYX>Kio(+gkmFD+4C&a! z(y5^6_IFs&(@AFFkCqV=a|>-QLB)}geI3eCx@>5!PV7$fJkq4X)AXA;rHz5TMb`;8d`an%2uWCX$fz(l|`zbqb9 zI&}jTIczC#KZDXejYHa52*+Bb1kY2sr~4B==>Mcb3pa?vxiNa)#w&8z6p^jNx=c%CJ*i;w`r6L@t=!LEz{8uC8on*@!RYp<1o!T zGfuob7SFyYGV@dxdj=B@TB? z=dFZ-tx`e;=6P9CU`w|xf((Dv;v}0@BA$@nh%!l;NS}jQ8dIRX;OEtAl|ig*&NfB>D2)OI0&Dg86@uRl->0l1pX`08 zjNa^#+=n`@os{M{9NgnFo5!8!2L-sJ^&i66dan10A_T=C`QJ2S=Bg zp>vIWTHhnUeR2iCs?8TTY(7r3#e_nZ6H9eaPKZ1n*z!yNzL*jm;>p6uF(MaP7zQmS z4&snhAXmv=cJEpnwVfXbj;teQB1cAl<&VXm51uH zf;Np7i+5&G-lu$(cwY-nk9XuHzngP2zL!@-*GA7fh{I4wuz=OI zva|c}hdH$n#J>41qX^jP+ywG6^BDO5{4vR>2pn9WU~#SmA~-QD0SeP41jc!C$%!|` z9@g@a4ddvOY8*de&hXesK4x$^wS{vUu)dq3&4`vaFZmpwIcNoW0OAzI@+or0wGAY2 z#@yqv>#4-)mtbn|1t7mrLskYS8|C6w`)|mCRB~}L=ZF$bh8DL`b?syfZr=~0v41(fP+B-I#~!lYmQntlgi8J6HjsA zAXes&u6QI55i*Ar@k36?fhK_&mgoqjf%Zlvqvuw4um+!vOSGsdfbf7%*A7o*L9n42 z4jV8)pFWe+r+S;jmj;?ZQ}CyWwK#4@tc(>NzU4{_w8fa1g7ZD1Icq|gKr{uL5k>$N zCvwhqbC^m_M^|e7D;a(4bbb|zKwc6kNq8#*z*0C?ct&n=-rVSVSjTUeyxK`A0YiI& zOAcs@r*}Pgl1aA-vk29r^va~~gcono_Ue$0)M91gAdH5UtLJRk?#+5b^dcqnnTdEPa3DUGF zhz>`S@nRtpCBZyy{*t1n@93d4!m-0+Isgd{M2=d}%Xm^}jJ!N*A4Co%BM;&`DFq_E zOE0Qu5GRQUDb^ehVkJSGbPW&6uWtgYwS!T;w{j9R061e6Eujd6XGB{iqlUcd%Z)O!K$ zs24%3n2?%Z14RkcU3t5|t z{}=?C-YncX(E}ZjB-s4Ue@UkoqRJZuRMnYFi{FeEG{(s}w?M#Xa)2Uuw756qi5)ii zQoDA7P1xy`Dm;Qy3YAJX#UiY9(@sSNyxyEYATJ&)03FHr3VY&~2d6~TE@&7%tP=b_ z_z}A;1)B()oBaRsq7%%tv3a8YeHw~cykDr*um73jYou;QA(U6p%AsR^%sS2g{g=&$ z`YPX`f@B?=Tk^fF$9WOC!01_Aihnud@OtbHXH*5Oda6qL4&EV zbiM?p>K*MVyFfjiZnJOIObL-#!D=X;!3KqVb5jJ7OP#xbEcC2T;=u77s((+0Ws+m$ zQWRtGdGVk$%@@`J0tnQI!kL!&pckG6f(5}~N(9_kW;k~Ivh$hgEU6?=r{6z-gH>E4 zMGBf~9|YV%Suj-rHA@WaMk6BZFgL35sh(WJwxMNb$2as{-U%;#3q7p=amYfxs{);f z77TMRz=fZYM9G1mcynzb&v0l_oHW8m zN5ZTCJ>U28hAe|%Wm6iyLID6_`%U(5kaQUJ1Qa;9kt3qcrw*)!cU)X(&29o?aWhE~ zSa@5}=LIq%A_}$)sbA%XV_> zk9@C6Hx5FmDDiuI1VE|B>^;ZY5CA!drw=I^hRt%whG52%+_E|?N+rZWA@NdImDzg0 z`l_9)J;Qj`Oj}VbCk$vjczy1bpd2tqCz;5{7tb z4n!H9V)$AoP{%b;nS|<%Re(-aL+Q^nR$cbCSGU-SSZ0bgVDe;3R*fN~mzNbFUnc%N zFCldA2RhuFOi8|vKsH%Ih7h?9OALYTR42Bhb_COdF9BGaRc@YNa3~&2T~XUgi_I!i zM}!l%A-ej+-+?oiHU*f*pFxT!UY(eprLZ~R5tOti0qA-ojUEJnB>)LY01ZJ3fE5rb zvr9&WfLZCQtw4eJ(Rz6#b|py&l|ea%gJOK>D|s- zPl*jzx~HLPSd<+iZ!uV)R4GI}3c(^8nJjAg_c;dVXy%4PGx;DnMu}D%L~CNfw$FaGMuslFkcHHK2Pn+7`8iHd>uaF8-;v63D9N#;Bw&Uq72){F z1NVoY#!w*va-B~Ak0!>C~c40F_LF7bE#`cli*4+DpSQ!P09M4S+ypa}uQgX8fm za9LX-L99f#R3ek<%TOHBEd&joh3H0qOd_&tB;ebHvK%=u#D7kk3S$4dKXGGALPa_r zb0Ysb@+BVq5MTpdkLNxxISJOQ|2e){2#2ju#sS~VUVe(FgklF_gYsA`AO3xdTa&#Q zOsDT0&f+3$3YV2GiGjz1EMBAvvew`&??uwUbF#IGbf?N4kdZW&`OS(npaYf2`{E$< zx=Cn}{~LvP2}unFkgReSG5?qm_pStUEGx!t%%~Q;h*Eo6kbwRRjp(cd-&hdYukwz; zMO-5Gq?2=sJ50Mojy38%!>RF!*$J|du0C4asO)7vKGQW{>Yc{ch29SL=~{xeS&8qP zGPv(v5C@Z6ggM45{#~#V@M@xEPOMKX9;r3*kDq~95Kc)2W08y=1T$dt9VcrD1coFpl6r0h@599>Y9oUEtD}PV~gp%cGVuJ%yQD30F z_hfVcKGZ+%7q$Qc!6ZawLqc1y;4~^9@ z>nA5Ngw0hl9<&3=NpGy|)FiDB(8Xo7i7|^lRl{kp!TCiF5Po&!grnTU10wsP|*Kf zGZnUkOGMkKzYbRIV=$e^Um4I}N7}F1>Gqo}eoX1nF#}+7&)4Ws1sXg%bMpXJVZHvZ z6it;G@iP9=?Vg3;ye<(1?repWxNIyqEr#yKelb)zwlyt1$;uDrC6-VpDfof|6G6&T zOr)b+SG>Rsd^rM($Ra>ervOn9$P*kcGoK(~7^EbXxe5);mw^PZ+y2vTpY83^VcQ75u=!jq^a#}8W`6LdzSy9k@N0)esH8g%2a?U zBQ35LKC)z@@b!h>lqnYUNDW^tTqfD;epjXl8ne#|9F*tj%p61uA|$t14P~dxk(*aWKoN(1nIhQr1ZoSP&57QIXy@P#Ei60xZZa3W}BzWU(nC|H57L z4j-YaS!AXTcSII`JhIlIaq9d|rLhBtmPJ@G!`6|TJh^%RJV0IYz+`Jlk5-}|EaATH zs)o=vA(AqcYpTzs^UK~x4ERbw@VmxdF;DjUU}D#6EmvX<=c`EPs~;}T5w-PKbFZBn zoQVh{hp#TQE{N1!E+d`4M`aV#xey2oUSANX=Z;TGjju>9PQ6x;wUe7mUgl7x*YPO| zj24N9CFHfAq^p~@gEW~z{j+b0zmv)y=ERPB)9Ck zW^0Q=iMirSrGiLS=XL;*{wl;n3I#Ng&>d0UPgqb2&phE`cE$W2@U#!t>55)UHjmdL z`{f-lxX5NNMdkqy)8c6E^~|!=@}lVCs;Ea%>;3L06Q750)3#%{miIObvF`y~N$LU_ zeAKw#Dt_F2y|m`hXzOdN_U-zL%0=p7pzn*@oPCn9$4EWbn)8hQ>(#IKxQssk1lHWn z#Klb(3_Od!7|wvGNcZYghZfF=TBPz^8Mj6fpTgaicmmg5Mlfd{u0` zPTu0*Ls<1yhyN&qYo~4S5D4)Bta=r@XnEXSX5i#?Op-<&d%!z?*!SS-R~~rKJ8m}? z(6Z@V%j3)1xKG8M}1ZuL!)Ys!_=DBWzyIF zJW0l$hdjrPSASgByzVdOmcJh@P?f!{GrJtTeVE&UVA$o_gx=R z?GGr5XU$S4VIXCcdH*9aQzQ~-Wo>Je2lT|K!C|@;jzxRPB-*8%UTgrK!hGof=La$adM^{0@;$1IwmQw_Za;;;T<u+%U3)0)bTQ*BJuAy*)GH#9v$h*3NkB@9gUp$}h zrOz^$BwVwLbCE62A3m>@-qY0gJ0e1v8Y%KM~byG48?qbzB?@Mkg@K147Kd z!izrB^>*;>1@HtMtlAv1Km~0tPS0v_z+l9KPQ)x#kY?+qUgpC%&svF#u z6FuHbalGg5mW-9(`g3S#&%&Qx$c)G}&b#n%5$0GVKR4I0yln(AaUkbw8mhFK^5njg zvpGAl$HVhjT+iqP&^xyr9{yDLx}TT(w&oLP117QT_z`ENMklNfg0m=MlZ-qNF5k6E zm>>`!OGZ>Wi_CnEMMM*JlOU>b!yf1K=r>DA-5$v*0wnipEw&B@DsXdi#dQRoe+arp z&c419Y9JC4x;kET|0kX4b+_s%)wr`9ezv86ZaThy1+P_x&=SZqxJ1otT0~i;jeK;S zPj?fd9N-%MkCPLE6Z~Q9?`dNOZ}7;M8bfeWS14^#*Wvk%oRlv&`DoP6hFy$f^Dm2? z`u#6ndrQy9GIH;{->s=N5$nx3)7$NJUCT1#Yn7Jm2eQ>)F$C2JxoAvEI5-;`?1|eW-meXABV70642+W+?B(b zL9D38)`v?CYrl5bPqRDBu-;FxKA;wZtzAB-C73=gUbmj?6f*kUoN7rw+(xZF@4oxe zc%H8D?gd%CtX99dt2p>9*Ac)&~qBl{lQ&=)?4^xFaHF`$*W zqK8D|Axbg;gNsJVA6?4v43vOC|5sVk&6k~O$G?x^|9Rbhj*@&zC~UpG5EwgNHdqPa zt}ST|U*RfwKI!;l_CWbwwtc>Xv6aS$xXGMg?a+TkHo`fjK7!?@%!U@R?iQjZ3ks<8 zM6At3%NkBeG0)PFI?;$sZzdsNFQdCf^*no?2@y1PK)4Y=SFiwugj&OIbyb>Gm;SWX zHrU*qXg%Eow{ZeXs-+f)*N70?h%>ZD6)-D?B@tey$;hvd_|Tls=m2n|!Iv{de|&_$ z+!cf3Uyv82l(pko9Xyn5rR!Xh{Hq~&QNr38~_%zo`Lze2+Kdga>K=hpd?y-3B! znYTNZ6kG;nirvW}+xN!kNi(~G?-*W&uULOiR(GqVB@~WZu&KL!|K&pvP!FI92*faR zW8lMd;*)8b3Br&CU_YV*PPxvN!c|1ucP~6zz*kfpJjP!q0+S>!LX{XF5B?rMj$>v$ zpXqvDy`H7tv$mzPwPUhG|3tN4UwoKawwvC*g zdnM4lsS$4&mo$mqlEvtp5N#}{rLSo5EvfOPO`H_vn~@e?TM*awu6zt0YxWEBxo2=S z^_rkKZSI7DXv0}Qc(`vUV~;+K3OM0@S<_>O6?QOT&A0E&o5pYj zVA0s#NlGX)G_LX)xvTlF@nSE^_d($Kc$ex*|FtXeD080OURud<7N*Y(!`665=I$M% z@MFYCl!{9W{f7n>^HWYja3|X5rJxWQQdz^123CF9{>tr!&^gs@#4c8e=YLqiw7ukq z6W_a7|0na#iA@?9b1P{uWvK_|Ruh&KkXVuw zoDvtA5#U=`dA&G4`IyVp@{pp$)OJwKg^wRbibpAf5jxqKGqb~W&QjIO_YLd|a4;IcLQf|rBmt@|$95~KG;_?Sf7eS-I; z{>URhiSDjom$3!XPPZF0Z%0Wq1$iMa-uqU|rVqg1l{zb1H4Vd43hhkn!?vVW{RL zq*~j-uLdp#A4pb$-8;wd4&T3=bkh{>ik-^G$GW(IET z^Nb)dgSGaTcUsruIK$lu_-)>kt8gPcjADlHpi;JzJYb2)_9rPzKwW~DAlpoEt(MVc z#pBDfB$cQ1cyud4$%RJ*pp)wJqDS9fU0}&UWThTg!Dy`w8`qi^X(fA7>cG2K4QWqI7?ui!2gpOZx?lV8s{AB)e2cB1fxHmk_yptR(; zxa7FReot~*47MTyn-Wx}htBY<6XtQ4foZvhqaUKF<1j_vR<}eKkqmXD9k4D^R@h$W zymYVE4S-2dRJ;_8Z?naR-hTd^u+4Yq6397U6l9_!3qbEYn)posw9Vg)&IR;|clMP1 zw+k<>)O}g5CeT{wBmh2*=OYWwhmL=}f@m%Y4LjToe9cLdiGHQS*?(gG`c7i`l}nyC zd4j-$Uf)tsWc5-I5tC*3?7iXL+^kSzL-px^gV!_5weQ7VX$21#*ZI8!QPYgYbQc;4 ze&@)t3!MaTgGnUMg}A*PEP%4gY^UE6{2oT@K!WE2uc3`51*fs~0_`H&Q2)%Lnu@jTjzD(t%O~{q&Zv-)wgA zv!v;VxXKJYuSSl4@9(eOJY8kzd)m^v9*WmKCJZt(eb#n6O&Da7YT8=(GH*H#Y|m+Nu4uk&2TUz7PJ?@uGGnB};s9=5ceC3KC~GtOvRlDOiW`{LTG26J^4 zNL+P)bimHza76m2Ko#zNwpI2?k94B#PDX zI_gkjXKVeT{@?wV_f77c5DCXm&V}Qp`W&m26XEwu9(_bLD90?Pzvu@_Yo@@U^5-+B z{tZMpS(gODIQ}Cr)3Hf0J70fq{;apuZ#Q@2wGVvP6{+g-X|2N@Yrt48Sp2TQc3}}@>72fAlQ9c`(*FF~oXsh+ zlHFLnXY$zmyoOUHQ&oU`LG;$WDEACD_Y{;_(rY4dBsqC%731A}@b`WQ_6} zcj+^+pdYock%@9C?HAc%n208-)mud{A0BvNofOPU%-W(eg$H=P_#~yNY;A+G`@On^ z5-68yq2Wvuf1sw}6bpn@V!VLXE2_|nAQiQC)9ExV0|(yzP|VdV#AJL)`HI%? znxHQV*hAy|d*Qd}TM;yNRJO`jhD1x2+*@^=2`%{4kP+);R|ilkq?zbq)A+{PODi1Z zlQv2+GX11|Q@RoYG5jP0W*mz`Y!Nl}{UxNJNCX!>{lg^ozN#AYJ=+-b+HQ?nxjGW( zop(Cj3%lH({UX0=X!PDB*_1S#wX?&JHJ&Ph^s%Pc!-wIx66>2mXfdZK;Q3d+w zL$FqFmwG4dsq`1OFRDu43-epkm5|p)sKctuJ?`r)*RK-VSh`h<^wYQS<`-spZ<7vd zjSK;__~Ul`v`v@<+1NQq>{+-?P0ENB=5uaYRJG6_IF95893P+vVk-uy#lQ2%Uj~g} z6qP4v`~<`C6pV?$(DMS8MHym;DU%<%ba(RtEE9QY)kRWz_P8P3fi#_@k53?DDb=vB z*Iz_RKkk$WQIwKY@COiLnIj$PTbUw3!51RwycUHjRhjQ_o0$DR0q2fxQ4KvtP$%3%K$*&~vifHdt6$8(TYSVoLB$BplxDuD?_SF;y;Wt$4`A#> zesDL(L@7T)`4$Ud)4V}q`7`N87m+=FHTVXQaEKX4(COQnV8%gavkRPbZ2m`Lj0H_; z{{iHcUuOnZXA9z)-KD4fPHYXmdg550lN4E+5LjIhpIDJTeYo;=^2_;dZ~I8K9e3 z2>@LcPP-8L1ih0ywVvcVy;On1g(Q?Y0!uvjxji$u{BK$+!B}0*Mr0WpGg*Zx)I%U+<653!BP6#$FBAL zI^uZE)OF39n1UiHS7Xusi=bs3JkXt-%_!n!#R}t#yD;+@e?yV&)AreM+YLMGUr_5} z>tz{i`o@(nO)Eqg8#dFI{sP&Sm{v0V2s>C->D*3;mMpqvaz;i*&vW9bv;-UXSA|NW za?L_Af^m7_=unH;)yz6OQG6)DpE5g(&?4MXH=opyPY?vc^uibig#`W`dIq7*Th3cC5 zM}_^z6mWz|HqW5wM&@>_dQsUIbwcw4wV+(JN`;xf@VQ`p!@inGc#)c8GL51-W}6Q6GjOM^`g)#m+1@snt&! z7`8+4{`-P6)BSXt)VyqN$MGO#?cXsN8oumAn`0<1Tmn?@L!0Rl|5 z$0Lw*bh2=-L^BZgbhcvnCqd?9fe@!WV=e_dhfswq(cAbdecJ3TQj z+CKH^FZH(#0!oBMsN*jPZRD*)J2^Zh*wWY|?bTe_kbhYaeUnNs?Hd~?_a_-HVd`&a zAtp(IE>X0lna+nmx^=7@WV!gkn^$+ZwbXiO28Eao7}6h9u9US-FGFaUqhnPZlLL~f z6Sp2Pq(4rU^;O91@W=7etwm&Ra_vUDrE5d^v7V0tq}k zW(}TD>deoyI)6GUCpw>YQF;o>5i!OTz?4(6C3f+-_mplt|4E!kp49ZdeJ@*4dfr_R(wS+41%WGB6cVE zBni7cGkE_y*3Bos%Wp%enudAlmmGGY7L$FF%Y%KUJ)(@-zc<<4`nOFnl%67hR^ zpz!(~2>`3!Zs86!aQj{b8eHqYHelE*6YEAZk84iDbz%HxA^8f|(`03?wxPgVOXE5- zbreNxU6}Tkr^bJmR(XChzb+-*>n)ERtl*4ep*PAuc!8t8#>T36Vgob`kcgb{(0)>l-iwHeqRNww}Rfkdqle36~c-n8~z+4&S zlc1>REyHO5An@e7`x)`4_N9gOV06=vh?q(<^{0{Dp>b-K9KFJ+2wzI9zkG`1GUi`? zW|q%argkv;+lMvSRHJh<*f`3{PV5QMtvXM6{*QwnF02{4XSXz3{+n4Os@0}UV*L0z zyWEC){@qgD^@rot0PE_EGKSYKhxP{xE8WAe7bc2Cx-Nws#}^o2bSSXvp9ozlXnhy(=r4y&)5sZFkb$$~KmEUV& zO@l_zJCedZRu5IM!W>`C#HAz%kcrmXa_VF?sKb~LQ+(g1l$!15 z(%QK0fK_90S}XUh{th)EDx+PLs>?|fH|9J>s7WGQ_9JNP`YS?C{4 zIQs$+sEG%C#a|@?S5tb$0W0dAxK zSwG;x1b`J^M7nErtBdM`u4D6)vLH#+U)%t*=pa8DpwaD*Z}6PtDzr~RG{ZNeRHn6u z1MF@0oob#j%x9-vtqi#LuCED>?Wh;uu4&Pe!tFXWB$t{L*!D#9Nn%xJ1pBf$5uD9+ z8`du|raziwj}0sjm)H7A4JxzW)&CQD#o5mmFv)QhZ@?JW&FT3v%0&wj z&T9SML?~{ zo3(*RnrbQ0cVrNJ=yu#XbxAjiMv~k-K>chh)cN!5&tgMDfO!l=1U+oH`FGzWT~A66 z`(&TC?0*r8It8i$>4MaN(!MAS`IyH)6qKwdBt0S6n_pp6fS`>?zskxYf*AC#A!X6( zj1%Yw(;tO5*T`(_`v3OWjCe4N(i8T5Y2B@7Z#}y#!L&~Zp66LHl^TDXgqz3!1eRyo zJ>JaA>;BxsxeRlr;iO>12dA?1oT7|w4}BSko18G03a}qEy|~Uri?LIU%IBTjK96%P6fVM1I6U}E|F z(2YFyq2U2&z^F+p0rw8=cg0jco?(qzC)UNT}e@DnwK-WIKj#bwpRPz z8`^Bxfqb8S5&8i3RV}C5D89w9iR6vtC}Bj6swcj=#0WuT+e$p0S^ngJzwA`%Et+=#AJLS%Cnj99Wf5kRG!!R?>O1DknKa&6mEozG$8 z^~&v&?{n|9i~j2xs};UqHAJ+%EF#NXvZ!^PnC5`@nh4C$yNCN%49CjV zpw8#dH`Du3-&V&<=LW0?5KUc7bj<`>U6LsY56!XJ3l4s!|}Fz7HjRVwZl6 z%dpW246+r4kQn05)2^?CA3vYVp^KRL z_&g0JP|%GozOT>uczGAmV@aWW^&!;9G|0ei#}4Bdw5ijabB*U2!gzk-TD?5s*XF_0l`OQJDOjuO`N6G|_7i_LI6 zn!N>Xt&xt3sh#S&tK~!O&=2oIdIe57O}C<6s0)8|lKE9#%gTsX^IJ-ImcpVz9R3zU z9@?{cdmCtJx$}9pjEhGu1g8C7hjB>l=0P@i2oHCf=A!a5Yt@N<+t87-VF&o|Q((^Rs4v0ZH!>f~dT_df!JY3#HIj&9tueLvZcMehAHC=oL)QH@XuS=UwW z)ge;Cn>d_dxZe{q6uBE~Kye{`-(hHPojf2AJuG$gl%*SK7v)#^Eo3Bgm&V+yIs@QI z{JJ*6-Z%fQChPussA1(Fv3!J;E-#a2W!l1-iy%owhDPZ2dH)YHX&nn{Np#QS@c~6K%iZ>nTr!G;fO2?hK+5}Oh^cbE~>H>!y*-3nx|KZ z(pLB#0}+*o#kZ-44o8vHq$CK0n}^c`-cJ7YGlc%~ruj`1Vb)0B#oKFw7hLaMqB2G> zUjO~$VHo0I`L1NKCCS+3G3oiRsj1{0`V9Q)wB*kQ6=z!gvBT=rxbe)g>6Dn^R5~+X z?tP0lpe`>%-4;N>12C(ggrYC)%DB;LCdbIkem%Pak<&A89@1tV)2qqOcVHOR%L$Jv zYKAzy8&etWrx0mvzB6>W zT7`Iuqrwp!=Dyo|DmQid3_`_P|9v2KbK_Ht;REOhU@`_xK` zBc3}ZM>WTQKw|wIjge87=AG?u4Zl&P!;a_AQ1^L3)Aw%K+~!{>M@++UFwvvvQot1! z`>*HExcvIBV>`1(dh7MDqPHW^_-RL8;B@iUBhv}Z#CffSg**Bg&)h%_8H9h=!`&tA zKA{z^km0d^X}ujSY&|s{Ycaf?{28R`a|{-|*sI@oW#nB?}6m*;H z)Z_871?2yn&JPZgFJ>KiBWr_bDy2;Xza6k*lg&vYW>LEg(r6n(Pa(}* zKfHnD2+XY&%RC{gg1}WbAdz|>{se^m9@0f;S4f-Y=p@mRh=z5u=m@3if7;}VRjK_U zm(@a0opJ$+{)i<^7(#3m3rT$GP)JQ@5C(&gNA;0~VvGY`V#1^Q=q$D%MD7QI#yBBPU#C9p?%-}T?6%P;5{A6uNrMI15w7VL%BR)t5p#nVyLF9thpm1( z^|445n~3M9jiO;Vza2XHPtGCl{9kNF!$mFxP54yB{aPV?~mkcp^F(Df|6S(LsWe>FG#Q!=-)mJeEHW_VEF2FO_3-1eC_4Mq&UV# zu`L*|#Qq07ic&1^Lt@NP0g%TnC6tQ3fTQ94EjTsU!TVS;w39#8xaSS$$sy}Ho@@i% zZrB(1pHga{289+ihW)>Lv5vIUzJAeb9n2q+10%O4AY@d=3>7SW*JqD!uh*S!!GIwR z-V7UO{wBv(XNRxE;gB~?5>*-R$QMssqq4oTqrega6Cn>l1dt0>YbpoNSPD&$zAHs< z3$*95Gn0ynPv9BA))c%^(H-uoE=ac=f<<9`1z>c5L9Cv6j0|_io8n{$iyxY$9RHjt z*fzOThXz$u(ZQRO&4%b{Z?SqdYn51rSV$Uza(PXGbES?V^e3KjveL&Ib!Ema8<9)L zPpj9ND+Vi_{*!l440Xo;G5uQ{X^1I&GfKTR06v^A|G;6QDkZ54Jko5-C?fq6AB`?YjMZzpaJv_&T~+MWpn+0jX>(+>ixGn$nmiPY)$ zX;bYF=;fe>`b+LBgvUbv2cAG-zuR3tmz}ToqixX~5nsS%vFVm&Im~vq&E>K>t!6fy zK%6i$gc&)Wz>f~2sFmE(%2)2u zG%>c3Hs3Lgx;ojS3b!W|2u8z^dYqYRE zg-yura+VW>h$;!Wq}{Q2l{f@}#37-GM+(u&DwBrEJ7%+%A;QLEE}zzi3k>9SSi(Mc zQ6g5cqHI<9%95f|5EzXlLg6^fkZna1i6KCR*!reeemGX(3q^eazuWD^#e-(IA(_1F zk}E&|_*b8KcGpY08`+K1nn!ll_rVXv+QPlby)d*(inswP-7}{QVcIZ5Y9pzoLr7W* zgfG{yJ$zrRY5|8@%$5S-X%11mLlBCs4O_?NR1o6h*gY*qj<)trwDgR34IOUoocxb( zJ-6k;ZGaG0ZobOaj5ls2%TbP>GKhy?|r1fGaUA-swZ zoi8-gc8P6m2zBw*x@|oZpmASS$NTR3SiE3`g>9x{ z&B;41SD}Ni@GNbF$-DZK>!FxY zv#noTMw82+cjCTVD)~sB$Va?i=D^?{EFlLqwhJz879ES1|@gf$Q<`o2y zC5H0x7!1V_%L7HX$7g55kWLh?zW&B9fBjpPb?v)q+p1dz(9qe8nU+Cth`5w3Yid*? zQhCgf3Q`{_TYe!p6bmv0u__T=GKY$@@uvbp`FL3hfvKVSG;t^F*iqB|N_87ka{1Es z1?Hy7;K?tsr&aAcYnp%e`#+y#oKS|4HX4P*ESZx+D5IstZ$SvtMa$>e3i{qzg0N?3 zzWPAd2k-r4L1CG}WJFVgodpBFu;0hb&1jAIqpM0Q5LMFRT_p*mNsmFcskR6U^~wAY z@uxkxNg<_JG|tBMER;NuB+w$dC`HtXN!!`kdcB^yTjZD6PEve56q`rxu(8R}PCLF8 zAFpHM1hl6UMU{ATo6H`w#b+@GtV}3)ER0OQ-O36Uw%d)3-_H<$Wy>onIEq9do94nV zL)1&0?03x~2#G@(Li})x&H#4?gykirY+`LV=Jy7W0JY~7r-?Q9Of67-F@!^yK0)M8rf2H+kFvmcuv^L)t?=Z_w*QYjc=vMX{n) z#jC6aiz%0l(c{{W#?5*`5_yqukZq&H2MlP@5RKX+It77r{}vi{Q(B~fIhYDIay(H( zy`7?uls2uvpFVwNDR~reNK?RQxh;Y>kH-@Zhx4K_w$q{CN1J>jvju5hi#2Gs#XPS3 zfUhtVC=3O?W}_4KXw)M>^}Ai7Gs%zxTVq_wmWa=ye*yAC{QD$rkcFlhozRq%y zj4LOW2;ob$UAqso{_gjGS`0$5s5cpfvWYBwE{j5ony&QwcxFUPD=maQBYb9d5<<3b z#qfOdFx%Owdw70u;%Hq{?4h;@7L zt(mlGw^&)&wr8!lmV zZ&T`$@p}C*L%5&Y6YwIu2*g6cnBO1t`~A3@#9}dIjPOH)(He~x-Ta=rzWwy}m-ZY& zPzn^4on|Bg<7M-m(^MBqXY3Ic^>z)lLGAW(*IAp&I%we`wNH67du7llxS zqv-@5RzaAe@eD!OJIWR)QJEkl4*N!5?--cv8k~OSxmT~a;s&2P1nN8vE9?W<(A2Y_ zC$MqNhRy3Xhuk5v&IBi9`&n3(+in8CoSYms0n+0M1Okyrge_X-!|`l{n(xvX@VLd@ zCai4FvQ+Lz(Wc-~boS&(>uzxv8aj@Q6+L(SWIC&4;ffA zz!(U)*hZoOmn+ZjEslg%<;T_(CCcNWqL3%w?+&}HZnMES?>~>A^rd}ua6)DEU?u`j2|{taiolaXsG>{M z%Wu~nCh#Kx!k4SmeJ!Mn1NUkI`Fg^gHBAh{#dAV&+X*qBQ$R?WVOoAD-8}H^Kq#EB zTd7xeRqGu((KRr0puXe&k3A9%7dx!JI3l=UptQJzCS+R8w({Z?8&-Ks@dL%IEYOpUDAD0?psnD7jpl+YH&4x2bV_@Q+WD5D#0g0-MuDgRQb4HYkjJ19 z>-Q(jxWEIb02})3Wq70lE9>+Yv(av2+nKWPGT_aRhKk}*pja4-K`11`;dsE8=XJ;2 z_Nd1iby?zGI~25;+@@TYF~?!dcA0aXCRVVUbN~{c1r*_pWQGtqp^i_l)j~+u1&F2x zQ12_xidpv+TRW26m;>P`i%5fEm)i@4f@}+YKF1A}L0sTm{Escw4~8vPdm_L1U%&8g zduyBa*LCi1V7p!HZS1LR?qia2yu~3@Nk0CRAk-2lzj2|%WCDSo5{pU@vcmpmt+c(S zB1}y|_}0R`qe%jl8=eU#6zqxN3euT^P)NZWQRoP}kJ8}CiLSn>A3nR|$}4a5dGg}X z(z4=JCHckp9}l|lzZ}it{ByUIm#uW#T^^S|i~z|SLK=gov{N|)JD-Xi7|puwnTzZa41j|fv5=ZsILW7s6?|mDL-U8 zBl4ahm(}EV*&}{WBIqlOga|@5gVW~@yPaXD-S05?th%tnRN!;ud+m8HQ^=wBnzQ^i zJ>;_+A*aPa95T_gZ@xQ@*dT}X*=RhFMBj>1;^fRY)}w=90l!+W*W)TiqZDAw>GIgD zZY1Z3KlAeP0s$W%Zn5%vPB{^{vVCjp?n^Ge`WxT*VND}jT)(fW7x?XO>F4)6R0x!v z?W7Q@pglbZ>FH@esPv0+2nAO(Y7gYJS_s1J)$O}#n}7HFKfLu%w%eK-KSj#@&oV_oQ49!821KDcqtWc(e=d(ZnipHUc5P8nQ9(h0-EODlvlN07 zfq0}Tn!*W%)W}I5NT4DRRR+E}el)qvn4PW9;x`gSh^WsIvKkl|WKCAS?8sp@KyI_q zYc~39W*_TzV$(H)Za0F`f^Z-|>@)3xj5-Yn zRrqB*Y!L`w+MgPVVBMsAz)-7<5He_M+jk#mQ6Nkokx!|2im7MvNcY5i*ZASC@%irY zV!|vp?nwCXNkG9`6{K?io1@HM9*q z_u{_Gx7}D&wCe0l7nGG&#G*y3R&KcP!Yj9IxhxScW>ffYy6x6m-h1iASDd@~yz;UN zpC^b*xn9R^T4b|1&~A{`L?fXUWhJXCRu-_qWfxl)$oChcrl!Zmo{zk-xiY*rKV8Zk z6gDeG#GllTmoGPDWdSx*jxuwa6Ue5j5DYYR)1qTL%v<3nCJWu{o(^EMUd=k2F&8H0 zv714W-)i=l3{ImCI0c=ysM{HJIl>NGNzkz-KX~qn{7r??)e-Mm1;I^4;j(}&YS$$^ z=F+gMH0F)_9X`9>Y0dH2jdrtvzJR!oB4v#D!g_wI3<{o!I(D((WBP0rFQ2SuWYZ7y z{E#dYzgL{a#0FH5^@aJLKNx^4Hs*d{57#w3A}BSR*-TWIgKgpl!(6?7!;|0o&+7Wl z>ejx>rtUol+bU6R>H(I@ihBjBfqnIZOk(eC&GO&9(ti5{;UfHG#7VAE`5~PF$H-?B zoiSXkN-SF0{34*@qT62!r8{9NWW`f{Kr{48Rp+ki=3oE*kNb4f+Sl;NV_$_0o_FrWt5&WH24dk*ej;A#^+s7Q zIqx1g@7%2$&pHP-SW;9PizF;&hc4G-HnY`ke3;9{Hfu2Ikp&WdB2E)MD6&|0Kc8HZ z!}h&a=42`aDs#S;<>ruYFu~}IEZI{MEtx|AKs$DO?64^-yv}LVdn_i%Z!tgts~!s4 z46rx^nZ-e8dBk5FbSFHvJddr=?<@-kD)J)h;<56O3tAWRZp`y7iELdFJv-mK zF6LgJ@NO&&tS<_c$Gi!@4Tc$VntVh?CNmlc$>J7*P&UA&;3sfWLAXl_O=ap-?~))?5<)nkxX&4z z%&qZ2qERL=2_e(zK-fKT?(ktkoc zcJs>ewczlk8*h8y{!iX>&qp_I+~W5|fMp`iwo5?>8t?_#QeK0}Zgu!Pe$>!BHM`j& z7RXY9Br+u;M}%j>@mTPg#gL=HhLjM|WiK=2E;H(ulc;BsR`JmoqqvU+yOo92KwCnD z-Hw3WIUiDF|tK!slRn%&?IR6Q9YF zy!_`R=T8!ZvIt1g)ychr556hBw@loyo|%k3$6?4t^k~=TLJot@WrEkSyv}d81Zsp5c6Ib_g`EP zxU?V$U0xKvrZn&R^871S#xE<4UQrynx-@Wgk>~1S=glj9x37=B|D3`PY$jOi`EQkjh%`It{>1DsfsBgy~|p zr`J>_Q%4?rGW=?k88%#R}|FSz8!8*crVyYBhSqmSQu>z$~#+t}^BLZL`uA-mT!5^u<3H0`dqg`hrlan zHiazaFhWYZF={jBIgDuTg>FlM%Z%1v>~R%1EQL;MsoS>7=hzVTotFqf8-vcX{Ps;j zC#c(+=Q}s(*yOdG_D{0=91c#ujxW$RcBFe~roDTlvv=$hkNj&aR&2Gn zqLIY;=Wf0H(rssNI`7_l?!WzxyEkvX(BomTXhD9->eXxW^YfAKS z;~(1`ee~j@FI~0biEEhtc~5@l zzjp3#c(tlw-@*2M2Rmz<*)m`DsNA7Nfl%CKeh~;&U7AY)QVXHhPUW)71|z>v*$ zNuuE|Q}PC8lmemIp$VE9Y$>r&Ef)c24(qox>R4Z;L5G~mZ((+q=P(y}tmS?ilC-lz z_Om07b7PM4!lv^h=B*LS#d-Eimh zczZ?cwsnd3tk1uFed3OFi96Ra-Mv2XfwKxebXNR<%>|!4zw}dEOFy-xzLd2f}LeAkChEP6|Af%2N90>^92BrXE zTkm-P@XWtI_1#2axyfip%J;78Zo27)n+pL=8&+op3 z|My7VY*u5)>#8U&TvJ?>@Opg~D*`5#{_%C_v{ZuPQpE=G$Ze1~WSxh+z=!hKJqg`r zgO`up0Y8LrVTUE(V=nPoDk6@Jao70;u8WG?mle4#FL7N}s5od1Ch(f6;9eDJK;JsYAQKCc9XJaow_=#!UJJaXCU$F5lS zxhvN{dRfKix2=2PnhjsQ=B%$=v+gU`uK(H%8=k!N?5E#%-t!;4_{9%i^y-H%t$OI{ znuo8h{nS;BkKb7R$n}+3`g|`H36WKXJoZPu#HSi5t)U(oN@l z;l^_xy?)atu3E=YTygt_6*q1!y?kxK=Ca6&s3+(!@~eyx~W@yVEc-aF54@WWnc;eDJ;px9_a1 zu5Yhu=-gS|c(83?A6o?2q3sa+Wqdg;O>oDkS15>X} z9Q{R0&-mc@vBBZPgM*XrzxzYsP@dK7NaPjX_n`-_xa_K}TP}=-V?loiu6XnHw_I?} z)`D1}&B(T9;tO+~Y)~25%UWh!w#<^X+?u`I zmXl@A&9>{7Lk@kGL%+;wT;?(71Wmaib56vn%d_bU?1q&N%UZ8#bJ(#p4^gG#rgHBc zYY;ky0o8pQ0uO8ofBf9Mht7+C;=I^Hn2x+wq4 zmy|wn>59iLE`I#7vM*dw{`h63|9WZ37q2Y;>a}aWcKy14yQ<=ASFir=t>-***M-l$ z|B{zKylwY`*S`9|Rr?>@R{hY`2Ohqz?vWds9=oOO^Y7{S;_Xb2zx)65_8wqSu4%e% zucd^l0xCg~bIwsvq5(k#2?`Qbf&>u+6a*xLNRA?4LJ>i76f>Y8L8735AknZ^&&-~4 z=Iq&L=JfQ;p8bCR2W3g2Yjsc0S=V#b_18~Xc3a%_Jnt9Y?`7h#-it?{Mms!+uxkrH z*b;HDInw@qq}{!<2P#AMR|H#Egd8XjIhf~bmFQv=f5bRszozRBsiP*chjhe@r6^h= zf)f0^k|Z8so}U@px6!t57x;Obz%~RV5Gc5f_+sLNuOvQ}07YC-5N)cWqJ3ddB7y)m zC>Mo@=5(k5lmN0pmawR>Fbj%9GFgC1V=$N^mX^DVOUjyBA3W_F1?^x4w z)Tp~R|lec*q<_}tpS`1}6XGXP_eLHm}jq^dr$>b=%`hC)UeELXTi}OMXpHsM+^#%8kTzO zy5VDa2YH=MO`vUKs9neTBM)PaJxlQFPV#!5==nVUXiuW+%M|ziG|$0|ueVpsvs3T>k59FP+Om*3D?x41x zm7=eeqWdoCLt4US@(fK8vIbL7j>0Pr{{q{#3H^-5pCAsvKNS2c!M_zR&rc*CbQ5hd zM(1+#iV{tEoH+ccm z1wm8!fp9UFhqJaSTA&tiH^7gq>cB7^9$%F2}hKGFs zA($c2sC#qU3%3625ORXYNhACEHzEA76K*c~zUn(K+Y&;|A=?JO2O;VRx*;@gv7zrl zh*y?wCU65nPA9}{{w&P9jm6JHu*WMS$SW_zr$i&EFi6@=-ko9;D@noqGE}Jd9F?g@m4&=i>8Es9 z!3JWHrc&T|Sx(wn4(ge9D*4V@g@-kYowdu38Qk#NQF&r#m5=E?AF~=?^Lu_)_x&yF zPg&HTGHVDhZ#liYd~G0e)`_gF$cs5WWE>b5Bj>1t1YWoi0B8wj|_j@!6@u(5d3E z=_?VVg=cy%`8`bXdXVH@d&cg%|Nip81G%22myQ}-I%*iXPu<^4!Bt1vPKC8cmbp`c zrp_S4KLva%QMUa7QaY5!5J%A(AinL$A^Ca86!cFDfgk?C@yRp9A_%nF8>&GH3BoEJ zVvCHFEYuFbAykRriu(G7`9)W&>sp{J?0AkwW$=O9n8W{nAY{8ACxqO{af=HoF1j}B zJ1_elgjjzigdF$LkJk4+E#MH#u^n;a{^SC@8{A}PUHk`MO}y_Lo1I?xGB7sVH!$Jo z=%%QkipJR}R8ym!D)P#Dnz~wQ>avnjEMat{Cea>-qNAY+HmIwqr641VxA_8of<#wb z5Su_e8MwaJj&GX~?=~9GPY6H0?NVgS9zU?BP9y0s$p$Qnk(i(^3@@N@GRJ>9KkzL;SP=9MI0W9{X)eMK-+xzhdbZ@u+?CMz(y;d> zXJ-n}yem30TN*iCa&{;y;6>_*?$qPY5{`F7d$eBgsth_<>Sb}&+w!uLQT#!@5K~28 zLn(J{2}d)nBTuApDv?IrRG>#78x4BJg_I zqrJHsoydrv!k0SwO``K}6XE|^f`7X-nO9MOS6PTxgUYW><2PiGb_r7~L_<3wmo+h;1z|6O4q)=W_l)@(sA2$-A! z0*iyO-WLVGFF5@!FK{M5Xs$SHwj^vkFK{Ts5Ay9mx_@u7|C0otx-$+pye+PKnO`}! zE63S5-bUq|h0JMVNgq9NCuIgi07DV7CXG*u#3M%94n?vsg_lL)12M>af@B^snvl3K zokd5@IEXfIsKGIrXbg#a8sP3&h( z4ha9Y12Y&x_F(BI2-!CHJ0Sd?xOv-B>#whE@2_uvz7FBo>?+&+5PKZ$WHt#Sw28pK z0zxc+5OK)y_pZO&oB5GbivH%bmmT7Ban)mlDs53YaaD+5~2zU zpgrIKo2sIMnvxuc=Ty}CJd6fF#lc_>41cwouuFq8Jd@(u#d8czZxqvTlpF8j(=FniG(?G2Ks~9`1zG&N*Q8ojyhei?|hY}q|Qk>shc6*!cJ(=MS zn$Gf?&OH7u@6=pj5YPpWvju_p8uXz!biVk^Y;g!?5+Ry*>TQniOm@KA>{IV@0sv9q z6RdI~Cty7L)O5k=$->aV%;3jy?hU~w*c2SoPdlU;U_Z~<%Z?FaVCQoPscFM<$Eb(0g~Klo~5ZgggOU}EmcixHcHP7J0PgCVS= zqqp0{TvbLyS zA)mNgF~eFV%SNT}u;vv=GOj+@*+zO(+MdD96~P~%Cn_MqKQBM!cZc0f93 z|1|sns3+pk%k%a~QHQYO0feBzxWhvifyRRXVgH2#L-B_&gzs{^=JHQq2;b!#2OxpE z`Qpa|k(YkTZq;cszzv|6LG{zFh|4aQHp32?V{GTIX$} z8pL}n)}L})A~${*PW;aJ@BK>V^kVnOG$hiOBX1j9AMe;{#bk&n$SJBRs;WpytINqt zu~=YwkTB7)B*fW33Q|(Ibv$uM5I@?|4vYy2hzo-&3ZO}OzV*Io$OCz{k$8Us0>J~t z$=jp_(8Pl_op%S5Y$nXJo5gP{P1voJ zxLZESN-o`6vEYCT5O~d1@0N#A)p6rmU(+T(vt~lArz|>y_dE*S55D&z`fzWQefPOT zz2^=fMcBR!x9vY?`zrbnSYQ8zL$59#9*B1UK_Y!|{vf;t_#BLJ8n}35Fwy08vin$? z*TkjcQ<*+9xu@px1L5&qjz6d1=7-`C5QcN1Bm|h8%?kj{1AB$1<_ZGlix8+pd_DcP zFc{46O<@pdB0pg0vd{B)*N#ZLj)=n#qnx{9oa#^SyY69l#aXAwQ8(kD_IWdfQ+g7I zWrVH7g^XB2np6QjmWZyXh?cmBJOj!Es=OFeNkU8w$^jWUF@`WY1%#LHr=Nbp{X|eN zf{lh6D2!}FD^Udn1->9x_f8N*m;&n4RR}eQcMBRAVuyGT}4J0R-o8WAIHptdG zy5EkDVgflK$^Lt}wAUe>yy~ z^kU@g@YF(fewn(Kp@g`siju04uAzdMxPrKZqO>$vA3z8>6M)41S0qJQ*z{2C2bh7^ z2@%`&1W0H+hIixKOeh6;(0B}kgeKo*1bG!`yc!Jt9jN>h+AGGtUxI8ejq>L)dAhd} z(^rFaN?SZoMeag*wkJL~+-n*%Ib z2>iN2_B{@>0RmrMaC{Z#+#l-%C<0nY=MMHo9C&s9z+en=KjeXlM}`xRf(9-+L0JeU z*%x)VKN_%f8BTJ;1WsH&F`4N*mF@pFH()yF6g;9DuMo@-)q5bYIQ$fx5Co*5n|VPC zMQ1*g1idc~nlA~SD>;oyM`FMRvgq5Akja9;v0VR=Y~R6*lP^;|yAzH9jeTj}T~Q9z ze&#nkjjnr|6gV3s>{B~$p?FeD@`#eiK4}Idb_1r69+PSyDWW4OEX_cZ%(0T6!2l9W^yk zCIgTI4kblIRORJm&_-@_f(jl2bA1U|fWJF-DE%Z#Srd1VCn6{-9{ zpuPy%LX2u7K|d%#wv!S#A}e%Eo(cq>R1-d>Ef%CF8Kf^7Vk8#6QzFJvE@i)R%6`Sv z{fd_lsT4YDUUk*K>1}++$Ml}RS$%+IbFdW%2m}v=49|6HD`(hjcUGxVp9vw<{8%+lSeL!QE{U)+bO=hF1 z2#AzG+QQYd^QB?$3WM>q#M}JSGlc-ySBjID==eoCduw z4_~+zIeRs1rZg0sa6Io6m?3cyIGpL*pXNQD9oU!T(H^?LA;7Xec<=3#=J`&B8TPs{ zW{RhG$a-o^>=h&LWRUc!ye1NiUGkzTqBLn58U9I8e^e0`h7cm?dh=C2qV+h@!h;8& zc)_!M+aLZwB9TN`V(D3VjSrqZ?i=gqLESDu8#I0$lG)PwegGR=>*FyS+EfJoby@Up z4&mSb-~Wst+(aOz?t2gpOwMycNPxF>9Q{oQ(c4Zg3{K1=6p5eW$%($PnU|yQ9(E6T z`<#}LP|(uU*HPD2kylm}mr#|F)mBx-YZoDW;w%a%;^P@8_^2@6?Wk#z5NaEN_eo$5 z0YYM($94waHn2ejL4I`_Nl%z$B0||CN;xPhgz#G+19M z*nkykED>oc6$3%kMmg0+Dcx2v%U-Scs7{6Fj!K_h)&3SWffjW^R?Q*1Th8oxc*Yta zd~(+IS;WEKsKW$Q4iHBNVjTwK9N`gA1Oo&DUne-h<3Peuh@C@8u1JZ;1`}Md`r};( zliWwryhk&95q`PB(*5$~>?0W+K~hId1n$`2gN@_n6gVl>15b^6KfXveNF z+ehc@n$H}#b=>soF%z`HZm&j!h1xMy(S4!;Mi6f4ew<Tvx>|<18ae=>vZR!zf+F}H;D<*+A&$aR_|)lw0u-Vz z6B0i!Kqw?g!Lt?}q0kf-^o ziTJ6rd{vpgYQjM}5@9=JkPYsVK5s4)YbBp*qmq6=CF7uSj)TS(SG{Xq$Oa*~)toX1 zCu}-x1rVZ1i0}iTrx6Ex0Kzz@m$8n$7Y+k~Ll>Pu#DxPJ!q+KopyA|W03k$BFhDFI za46a1b*dLY2m}Itb63wVR9tvh68^UMEC^pr7lpwEfcL&48Z>|H!u<7^4>w{#@Z$N> z^Y2T;Ab-x4oqb;(Iae0(;d<24%?k_H&d**ApD8{2wj^|>ICQoo3@`@N0l$${pNUJp z1Bq@=&mDRag;o(X1n<9d!o1MIAlGq6>S2>0V`XOrF;go44u*h{DBVDup+qBz^86%B z;*(&|h_=OO+aaAw!>2N`A;c$S0E6&|J|+s~n44EkQ%Bp={^qCsxEkEl#A!2`n?0r4mGxVn8Nx#)zI}s6S6-5nYIVCj(6@(Fts0m4fMRI(1L`hv0$3@ymJp)B6)xZV4Sgq$Zb{(xAl&kmSX@kQ3Fp`QXmfr0vp@dh|ML|> zZt%8*5OMe&2sarZTip0zalb!y+Tp2%=K~X0uHVzqHx*_|D#)rR$to&H$r-9^fD z=rqWgGyw`em7XamD7bN0A96o3KM;sM_4Veq#EB@|(Slz9EA(9@0Y|NsV zi;?$Ak`K!WxX4nE%27O&X=pG;Qw%r^)Dk-^8{P~glKH$(h0x=v-GYcVhRWZ z4=jzGDLW5(2jR0k3fyn;P7+d8GHAIvb)`CWsWNfCBKG~2sD-N+R<6gd-bh$1k6pfY z5ia0y=|(&tI$L^ft|an9Y1Cp_)ceA-Z!!ZX()~wMPrzRa73tGx2Y|3Q*8Ryjms)?D zYJc0R0J{R$J#l*s+?B+wMd>CCvW2YZPHC1ZLy$$xz6$W7{yOl9h!IEmu5*ZX*+Ut~ zw;i=>McuIgLOvc|Ia#Hw+>+*pz3tBsfgIfpznLfGwn6qFC#TlYVi$qmLioLb8BVk~ zA>qdF$Ku9v<2`=_;jjPWpZ@fxKmA?^znUQzgd0UmOQN^ z6{;_L)=>W3PK7uN<%^ao36{#KdsNc*DrH!!WFJr~IHX$Uq;tzf|IRVPYB$3=@14#5 z=IuedyPzfvv%wI)jB&ycLL3DMUnd+HNp^jm;yRpsWH`wgLpYk?JeGWPBGq*~JvG*1w4IFjP=Cd+TGIE<(OBO!dgfh<}Y`Tp94#akCafFE3gmaEd1tJ7ERU0S-E zhIA)&{$}EAdF*^?%wqY)kGE1lOBL~e-`dT@wd)BVD&m1dc>19%X1?Uy`{Ky=h2gXL zVc>)V@$LZOWTtQL#iP$Jxb($)^~QR(1RZP)wrdJ?s`NRS>tG(JBX1{3H>L8LiqlM` zMYY8l5&|R|Kk8V6GYBM7NunDK;!uzxh|Zt-3L)yR#mmPMkw{F*Y<}3?`lKI^i*8hg zXt~U$IND5%h_WHv($>*1rB?2*0P_3*lGZzMYrfvX;XH zazMB~D7wzx)I#5z`91}=?d;9` z^M~L9obXkQ-9W58_#VK97pXx61&yS;jb3seO@qqN6$l)-=!95B0rl8mg473AcTQ!d?a=zQ2c zgsAHtX?Zr<{1iPB91a7VoHp3>bO_Y=WDto9!f*VL>;AW%IRt8WI7n!nLpC0mNp74r zX`T7+2U@I+4NU}i8x4X{^Rv zfY3zp!Y-*8Q|VYUnTuvpNmg=~_99DbH5zii7sGqxv^pbZ)t7S9=&V`R-~D zwCXs$8z6idc>o9m0f898zUYI%AxfkPM_$D_f`$@YkW!F_QwfJ8W(!}%9PW#@8%^;T zPd_o06*QI^IA0ESckx|$?7M4mb2k%!y4AYO&n%Uisgvv$AmV{O6Q&7|3D2_J4GFIT25-Aq}!mAY1yv08n3^>!)%2~R;Et|u*& zUxd^O5P}B|r+AK}cmaf?$zDV8Zhf)N;OY;<>^sjoHitSj1UuaDuu9*r6KW{uEKS`f zC1@roq%A5SO(TO7;(=(09^~!3f_x+@arhjAPK9{F%ex)4otUfO`RNbvN;MVLl+=vd zcN_24cfxOK{i9caU+eR+w(fDF@6+p!=U*r8e-py*iHpDi4g&vr2seovKdf)uZ{t{B z(B0M-wQbK=|M(AoLZ>o)2g2{s$cB(D&J_9=LCBsbBp_ek;`H#v2vi6sgU)FiS*mL2 zE6A&W6ROC`X(-4kNQj9NC)8u>1Kyw@#MGhBeuF@aAKL~6`S{Rjek48#K|Wa;1t8R7 z3hWRSG!qxJmKJi5qq!_&*+GU>Pnr}lZ-GxJL02vN@B(CLR;fg zcbUj5!2ke&07*naR4V|7m-Z><>{lta*Q_|A2N2$L(YouQ*KpFLEx@8Z*y_QVy=*7M z5JDWqOJP93*dQ9AaY1Wh;*Sm{uWLBj1)E`itV3_qAs}!v!*3!ha4PT2`|`NArO^ww zl7PUOTL~-oFMn*v{@h%!TAwq22Rtxs_EySDZSK;&?Ae=_-dA2;xRbSTCug;`2((<2 z_2F*D^4-g;)tO6oE`i{BsVZaT4#IEccKRyZsZ2wDSROM|6ghU;?{&&?aKg#-lfdCf zvO7RXq-)1V5l+n^hnvnEuJ*Gjb~cXNqj6k~Wh+PBElblA6Of|v36Xe+Cf~g9#fKrp zt48s0sQmoAcz6a)7;gUws(K>>!|a@bn!5Jth6mNH-2frr*WNP$k6?@dVQ2R@=MerE zA!OfS6IkE1IgzsA;l}L8$Z^8Uk({5=p3PQGW) z*ApI#2ppLF&@(*!=;i3y3#mqC`<2ynm6gC2S`50t=7KIONP>xPshfrL|TtWySbd;wf2vwLTm4tm%7=ap$ zGrFQcV7R_iq#^1_0Zs@6#+a-_m|`UlWg$SAYoiJfmOE?TII4}D&{MzBXD5O%(EPz^ ztH)vco<-PTb)P%XbN*m=gbi@`JbXVq0)elSj$sH#(mh_MxeuqJiHea_cM$vtV(P@D zlOw4oCbNU43d5(1&yN;F&R3?a)e+WrKkLK2jK!MEAMRaVKu@!m?qz}jF5b)iP@S_> zQ?OiHxKdm4siEvkQ^n_&vbDxy(8q@2<=Xs}8YJ+^)jJsH2gqsw9~Wbqn(k?pyo4nx4bPg4(bQ%OF61AY!#VC z(o|VGKZC+UrJ$2-;In`@N+Y9PW*9;`O^8gKxBAmh=+HP|RfJC4X>5{}o!`{@xTUKX zV1PK<`Qi;gh(al<00)Ty8;(&KqPxuZAlx+lunEF%U$#ylp)I)LrjGpr_BC?0jhtDx z5j(lXCJ?UK1Xj2AfD_hr^sN2e|M=@6+%ycs9*MyOzMOo|4dJVa_aIIP*Cz=#Alzts zvGDlCXu_pJ{hfOyWt3Hv!2;!#WMtIjWTZuybO9*&kngb}WS=F-t_Ih)?QZ)6g?Aee zNSv>NTFpaV;)KvblKRyNRhT}?B7SPZ!CJy$`r;7=(vgNTQO447yQD9g%D^MKFq2BK zkV)Px4^EhAqf&5Ct;k-z)Ip=dN$u7#t=i*85Jy{2nYRU)bp~5@h3tNaChqrO8oP;w zqc6@ML=Z+E0>;qbClrLqZWEcu$1i&$UGl=3$nXXM34g(!Vh>5Sd*Id{71ULZ*u?@Bmoqy4!#p6D12-i8} zg6%JYaD%|F5Ms2r*<<6!hLBAl8?)czgq#o(?eSj%gy4k#-@pIIUk@SYJOn3%K;VB7 z!mjS&g0iYzyAMH4sH1BL5XwqQKo*r`(lLa%0tDoQg@o2Wa^l+;;KbW@`G~GWLL_wb z0+T`#CL;)mJvd|~+BZ&UFOMMfR2BA97dxfF3f5trHIRrjl8rK!kJ%*`Zz-Q(E(aWf zzy>jd>DJ1b2UH6WsRM*1_G;yhssLe)w_&4?F_>ZNDbtQ13*Zn7ID8tu|9PY>lzIIZ z(7Mo97aYOs#!@}TQa!kG8AOQVM~ProaT{ZbDlU(QNh z&O*)QwZ_7g`uq>oS?_M9PF_o%y>l7#p*jydaINvmQf=`x`+KuPhK}a9YJLw8XXKYO(5gC@SlrV!u7lQ9vwiDt>LJT1u zkHG{YMESNO4-}%Hb5{rmrG@z9X(R-ph@i0q)l5=oue2b7P(|2Vo#n3~>aQsrs3USl zU;M0*%sFG33#MNo#7+ntCR)m-?Nb5>!3m2GYk(78b5;ik?|JFf9XA9u`|X58%7ze} z5D0`gikD>%#T^+*AO=N~-GM+P0y$t02%O6H#T<@Z@*d9&c#{)6T@>*;D`fs=%ICT~ zfDn@BVtwAHwksc-iXdCgS7psrWiHkifL0qyK)De z^?Bppzij|Hk9Bxl<4{5qI>s+V-F)7+RhxhA9(1Bo7IKA8ve%ewkjgp5~wVs z@@6p$j-tIw+t!ycu^}W89ikw@$jQh$IlJcMm)))JtZDDP-__UfXsGe=FmOmr z6%OM#%ATm$62jj^k^2RlH6b^0e-VM-yT1uP;`Eqr4yCB;8UzS`qb6kQzYoGKYr>J~ zrDy#UH>#SgY@FnkG_`dMAhd`HGa-x0qEkN^3?T|$kA_xe2%_zR#DTIT0FO9C36+9q zJcgeKY*3gg01z@sd@KqNKqyD$Q>K#v!W|M+b15o@@R+KIm!_z%1`F?63f2<|HIO(< z%umEx$i|z=0)eqRrQ%E_aUxCKtB4^ianLM3qIuIrtI|#Tu7^&omp-V)*Q6uBtP4Ew zwAG`K-OtWiLkfKvb?C)8yO-zfU&p(QC%Z!_IGhM(=sB9=fsq?cM>GP2(>Z>?AzTB2 zZ*oIMvVzC+A{K9_&sJ1c^UQaELOs{ZzAl9a8u>TDtPe5)C zWl_LyVy=(KqHGO}zX!bRtH((riEOd{D*=CZX?&H)v0!Yhs%z~P-^x_8}ltKIeLy!7i&7`6EC0yBhA z`rx$Hle5;SAdEWnD$cPl<}j`XhZ0<%8l1dz95@_F^Bhb%28nbgKVUM`_iZlP0*@A~ zq@EnjJT;LQI$0F?wmg2}ZYCOBX(?N6Duu`QwfUfx=CZZcij~%ig@!918_Pbo-T0-m z^7Df`YY(bc9#pS%)vP|O{oK>}%ZtumUq1NdW&1C^ZNEHk`sH!`=dPMBZI!5EtS?@G zny@Z^tuBA1IvX2BFjLMDK5m1jzA=#BcJFLFhlbC`ayj36(7&fKci*bz#&6OkV$8# zYv_bV#sGu}xt=iq49`*kFl|rP&n@Ikq(rUw&6Xw39THoXg+D-)K#pA;B3Imm;*S~o zO+GiF$f+81oKw%@C#VVka|;MDhurx67{Wfn{kS2-$YB)O1R?|6n4*BEggv5J3V;x< zUyQtK?Cd$<=%J*ptE#32p#^F}zz-nAE4G0#{29YFe9S;2-ZzLLL=%G~5|oSJgme-g zjUS!rE+W7qCWtu15E@9(%%mA?2*C^q2w8!8VrO|LP3Z}M}b3lgyIScLc$4M zfxy>z$Kd4?gnXuRPJ!NJ`Hg4!4yXByWd=^>hrKC22M|I9_>sU5LD*KlP*=E6R}2uY zwq09mzrNI5_NC+2uMckj`l$MsM>Ss_-T(Bs?(@@zU!J#o?rHzh+ky0=?bjFW;EI3W z+xmCU8ozW@L1YEzgb@3&0gaum)F4ZI4>e)wxtZd#v-u&jIf3u8{eZwXL~@4zwP>G5 zlKYDUw`Xx~T~RJgVGd=!dy@|v`Rq`zRT0$_r^COn5QP^*2pPW?M>U1xFN(Ga^L(t5OSi( z4Iw9uY`Qq_bF;@5IB^5);My=<$keJRJ;}K^z6%I3a|B5D&{tbDKwlgn3^$R9 zG?9%qmj?)wcgrK~RZQ8Z1WMbdkYTNqZL6Ah2-SoD;Z-MffDoJz1Q6n~5ZmCxGkd@Z zfkXC0#j7}Hzz!)eCbs zlA!|p+yY)#vD$L=OXtnyrmKq$S5{lf0m4t6x7ONke1?$ORsH4Ry>$Yg)B}Nk>}dlV z{PME%Q*Z0Xo|ezOt-rp2;;{J(@Yz-SsiW#s+pSe%oCb}WHkPi|70+Lfn=Ly(Qxra% zA3B#GJezxJ>M|Oef%*_bIF#)1GSU6{MfZo%t}S7XH%{5)d02+;)wa_R*OQ>iG6c}s zS7cr^Jxif51O(PQzCuBWHu`Rd|Im@Fyy$QXnh;e^UfIptFT3day^ikn^FRqF1lT&C zj&u?y@_rA(ue&9K z|Myo2F^53lMn}+ZYC`r?u4xHO;8zGYnIW6BO&XY(gU9uitJCWRAsQW>`tW@4&HdJ= zE?xl!CKgZ=Dk&%egpfjkD{&?h*aHZGJ-CAjLjH39rEX|U6t?%9HQxm zw(FlCR3Z)^Rjpod{Y0h zvwE%L&T`B3h5IEdjb&ee%$q8a!PZ!Dw> zqSFplR5iVP0s+F?&5s+O4TIK~tv!9+^?a=J$#>%@H-s3xZ)-mU&ks2KJrHuf6~4JC zg?>-S?%H~`Bopr6ZSAdYeKx-eHQ|4(L-=)|a^k}#2sdfdoWyTJ$o9a^`ql|wO)T_} z&x7Ev05j}-HW(ZpXTHZ4Y6gg-P!oc=fdE3ZZImC4i?Rzs3?V-I8oqSl3m9clC`gD? zsc~yVbVw})afl#fPyj+>2_Z`WEvyU;v(I#ILoJb*#f)nD=(HGHn z{jm-ZN5KQ}s!@P&B-wp5*=yp`$?Q)`j%xQN!x9#!o#hkVQZDHY0U6 zL0Je+`0JB~j}K}-J-D~hc4xkhn9pdt@v-IFhsxCXYl$-@(bKtSrm}*jGX18reBWmK zO@4QZI1`O2O+yA{2qkt zNcjUl{Ocg(e9aFy+;GA^&NbZVU~S@rZGaG*@GFD_dz&G|D@8f+!*{+VaDIb81Rf@E z6NFz!Vm5?}!{I3czw!5dBQrqVi;>xvqjLZuj-#!Q`of};_So1%O(-QH3E!qT+W>fI z7cYeZ5W=@C+Py|1p;@U72;l;M2cv@ITL?u3cp;I>(aFlf0$O5JV@aB&4Bb|N;iSTH z)euJz>Pi5Fr+3Jn1^i4Eqf8Y*7(x_8!3GbgXCG3}KA;8=X4xne+NrUf5Nr@bNX!-* zxA^aZM2h1mKnOMnakMW6ZM5%?by{CFn&3K`;sFqXCNq4dvI3BDgQoI>$FlqZ!pVZO zP!LX+$DvV~mMb6It}QlQU2D1a%fma6KUZ3B0D&uQH$M>&etBB|@lh>U9ENbUt7f^Q zX7$1SO{kE(3b<>01$k;UNgN^{9w4@Yjq`eVu@b)FlB%p|| z(QYPQG!%}vR=|A>A&#RA(sp6W_VqZr0pSh_s)aO-4dF>GQ3RopG(Z@>OFqh6Iod)g z+DwrXLU6*YgKFS}nfsLiLbPJWNweap7C>0-t_K_fgkXaJA)YPl3fawGF+&uDh(O>F z2t-?WColWI$vg!RP8Xb>$`6^y3jqixiqB1z#iC^~4JA+k0)$|I zU%GCu5z~DiJ1W;YDxnno^x*DCV5z-owd3xmE|f?C!j-O?HKIQJ2*^FD2Nb_NYXreh zSK99`ci#Q{u=Z0|ErimK?RVE&ZY zDk3B#Kq8@*Guwzog@P0THFfP%r^Cx`-M`=7-TD|EV1_0MyT_ZK;a!E_tqD0dD}9Hq z&2GjmZU{M}CYxbSalU}t_aJA2aMj0#kj);(59^ya3J_v2Fo)a_Zql~|va{%SA!M_M zQD;NQo}z%qR}=5S2789zbq~FL`f36oY=83V{KZRVyKO|pq#(2agiz*z4PqyRYXA>~ zzixp#3Lk3e1(gOsh=VJt6-l_u=yn?4&kWMf5J$lYrD^Esb`6%Gp_rh#6xCXe=BO;< zqAqq^`zwUtga9FusRG_SfosBa8&!y-m-j1Q+N+R5v^*?7s&m~{2jVD(u<7Iu@IWAt zGmZj;XtEy7PaGxKbH)Rt{jtty(EsAG@zmpEX(vb1PHsSW=6z}8Sl;P3h2d|mM7=4G zo2^V=tSee>DPL+XTWY=r`4b345O&@M{C;^<0~{{5-d=3J1&=@@L{p?^^?!sQ`V@@u zKBDmndioD9+8~Ioc2qABuUu`ryVQcF6A^@UR~G9^7Vl&MgzrjX-xfr^%?o{(8v+nc zX8Mk$c|jH(PH_h(e3j(c6X%X0ym4YLK$v{k2p~MDC1s=_rYa?-AR&g^|Hz1mK`DVg zK4FHq7>kM$DXQKfj*@wKXhKw|`Olt@yIbGc@~|JQ4}1)qu)SwuqYdeAK)AkpaBI5$ zs}Qm|b7Elvt6RJ8wmgSNj3RL#v4`=iM&PZhs-@>EgrieSXdZ9kJr0>@8XY(p zMWZHcVf^4nTsDa6K&<|82$OR}rzRx0#`E%@q)1H^3pAkozVobdVJ^ppO{M=!^* zZr68)oKM(gX|1TFCMgb1NEH{RL&#wYQN)-uxE7`gu;@Y}G(lW)kT%*|0EDuf)h$pc@^j+brG_GsDPQI(0&3!aKaPXQoj1qfrfIYjpajk zDu5YA?ov2sDj#F17-K1a(NZB{4;moNuu%px%(YW1w1b*Z`HG_&IAP^6ox5K8_l_Ih zKd}RKAN4nHIc4!6WFNcL#mg8cw97v32yR_56z_&LoPZk1j9o(Ky^KTMl^^gr+kYhI z)NDoMWKsBJQ6xCwo6^|1%Jh}`!nM{i;P99B+YlW=fDP1ykV`*4xV6%LeYN$@r;h5+ zo%cR<+*@s}T4{l&_kMkdJP<@=R7k3In8}p~$UGs+F1K7?ZM*)d{racotE+WrmSXK* z)`uG@Z;GPEa>FL`!)6N3PJ<_34j4)G8BFv94J3HHj7M?wag19_xWmeeu}b^ z`nhu#>zledx?XiW9_)CE)@@?~@zgvgeoe$oJy!F>fi1PkxY*(*5Q{ruV!FQPg!LU? z2}CdM>cbn9s@wiDRkfgl?5EzZ{^>vd6E%9;1R+j|oWNiLv9=@--0ycn$bQ>lq9rMY z@a5P=loPto;zVHcA3+8zc7#aQC-`2l+vLJ1LuB#R070fc}ee6AS6 z^@VIyAz>O7H3CL^YPJiIwgZX;guE<41feXQUx`W4U{TO~y%cr7JkwE4)J;puTU+v^ zuB5-AZ14`bGbZu?VT7qdgsI$lGt>}1)=D99FHsgARE4rI*IpBDqE*1r&sUAMmr{>a)zMJ0R8z22ox5`PGL(h0Wf!N5&rcUc%oaz? z6o$>@of%8_8&38fO7a2@*$}pcJ6;bul;du3;efusshYi(^e$y79eGJ5DRCtk8D&{{ z4HZ>cX*ppAI=Y8JqodD7h$JY0T9BYAB_2L8ahW5o-W50QwLj=*Lx>pL*f8-u2)S`$ zYfAz->qgGkd=J7+9{BZT{{;vkg0gkHsT&M)bb8_c{S(o56j>j~7I-#<7#Ow;b)mXQ~7~YMIqz)K@$bRp-2iXI4*99=*5~#+#B_YsYWwZA zjys^0wn`9OeC(`3cL)fVnjzwruQZpf)E6(+psm0F;c`{R(yi3_ii@+QQFA4cbH(Aa zMQ6bbAux}mo&Xg45 z1fiUayaS6 zwIzX@IAr6;e%X3J{T4#@K3#6)wj>ZzC>B6?2mWtN%>MJg{u_djO~TiU4S3k>eI;;X z0orfs2M`kPjC>N_aF7`L8GN(&YGR>p?ESNWsgCEvbsf*MimyAldI5wIVv^z_EDRyA z2NwV#&;cPfOMDu$UUgqtZwSSX-VuPx+b_b4V>tESkh zXB7h|Von-6{{md)`L(j~mw=HwH7rtu8QuxEps*)M4Nd^)e(N1Oi8s zy~a|!0YcDlx;G@!u`J)woKqn1!{NN((ZVxehVO4AEmU6mP?Zg4^`-sBr-xN*1cAWe za$Chp`?Ze`u7eipia*qsEH##*Wi-tdprwYZXso5~>QX~F!1=ND=BLivU%IM3w%=K9 zMkOWKDVpa10voR^L(O?FZ~1P{YIPY>*3?X}&^*8!<6)2~$^>GvUn-F3S zi5!c@WOi0l*MvYLJjG11n~uT-5GX0if;ftH z`ttCg<@{(j6S6^;Aity#zbr#QgTW63?viF$$+K)#L=UTryJ$B*kZm+>)_IkiI` zLwMF)F~VFa(ozu!jIvaS-K|U*3$_ldL;pAKuX`A&B-dsR|1jiVB!Y z(f28`4yj8xYfHQ8$#@#cpEOqTHIRcW3J?aF$U_{3ED8{ww*Uwgqb(F-td!#SDyQ10 zK^)CFtW)TuzkVj6i#~BEn(;laoxowOw@Cxkg#PBB)&NU(=e6$h_G}1;zzLH)0K&1f z<7^0_CY;RiCo02J?~2aMUyXQsC2Zzuw_X4B!R;>{x7J#& zfgr(xpdbVYmuvG;bDFzZOW=W38H=~m7H%ZGFOR|NNb%SV*dU6dNnU+%?k{58o=3ZN zML1W7I+UNZ&Tux1+^gqpqUvO*X=A8up`$4)DIp^vsidGND=W`n2vaB&K0ZF&ml=f4 z4;6y?kfEV%6cwL-ulXSm*!FmUSh3B~{P*#92WIg8dhR>lgOGiVA;f@TH`{_V)Fo}* z>o))NFNBaVy>GN(hCNly&tAWIk7~l<86u7n5U%4iIsbA34Xdy@{L1GKAmnyVLhD02 zToAII5DRBf?1Wvtqis)LT`sJ!vOXj&qX0z(1RFU?F&S|dPzUaZT@fC^7|BGBZT651 z?uUj3IRLh z1C13zOcX+QDnytnoVQSpvQ&z;P`Y5L9BZYNxJNl{zgp%Y?K}cPAh66?ui~h|&0|J) z+>C+4YEP3IFXJXZGr+Iqlto*h6+p;t#SH}Z#WyR$4B^COAK-8z+YcZF z%@&2uUkRVe4H~#~Vmvqa?Ujg`vKVl})w;q@t>quvuB~+300@D=)sBh}O+||hg-Z?j zAL?>f?&p1MDE!=XWv#vlsym|3Mc!&%@ux<#K6|yH2&%r-rm|1%*8#%M?KeNRUWdmo zoj1`6I6(A%!AfoJN;Psq)P&|vIzTvAc46j9BtZDKFccufZ7*IYd%ueF=!^5{iSc+C zd897X>AJ5?j*IzuYdv>E#r^6sW@-vXDvB}^;-VrfX-O#w2`M_A$%c?Yr!pBd3W-c6 z3&_eT**m+X=akg6Ji-KapozhC@W9_%9(}(i+#nEFkL;Tb8v~)71hQ{(JK+|Xefy(L z;)ZaeUB%Zy7fx~H&#$`O^t`I27o2c-a&8?$TPsIt zlJhl@4>DE`GEodMQ983zF>IG&q!}2XN{p3CjD>QnrAoqXm888YY5Uc)4rt}tYZn~W z1^lkL7~VLxqtYD++zC!td)&0{gju7%SxcZLs4aMRSJ?h1kq4iiJJfRl5r}Q@b&?w= zg!ot`barPp+C1^T<{+56>MQNIfxC7zPk7RHgw%YprEyvcCQL5}??AZK$QB$S0|Cx;TeU&VX9 zi1T`S-nA+GXjOpS6))=q`<-4!$`JYL{#^zzK5>>*YJ>mO2}i9W}m=&c`;p zd))k>7#V{O5g#w)+Hlz(Y0{|vA-mBZ@+JfQAl$IaD_8)#m! zxniZM9M_E~tb!BXFIYvp3Nu%0GFPiFFI6Uk6TU4De^-cF<-f@dnam1)lNmUE$$vD( z=XHutf1LNDFsG(7hwH6JMIcJ)7YyZqC}@iG3YEBjYcNZDT1POrU;#ZS8wx^ zw!;@CI3!ewLR9AdLk@)%w;Q{9I-a9au=xqmD*M@B%kv>LQ%@9x->ebk5H1C|#iol> z4I3+6*buVCjo+3ut|LeEiQ#OTh6x05SACq~WD?U@+4!uo;o-o<%s>CjpTB{SJ+%N3 za*wrOvG21XHg=Q0RloQO^{!JAB~6tgy4iM3Q37BXdub7Qk6WUCx6sX$=yiBdxz2q z6BXZ`Dn2I4zQ&6FJC%dYR8E_#hFNNyF;@#QRl*R4TObIdcB=t|@m9)-dsI`b)iMrf zW!dTEI_T#+>6aee0T7nEm;i^jyms9?X;~kzw<&mkQ}Di)ko_H@2OdV)KaF&Fe%`4! z`pC-|msj!bfEFS!L zUHVXyJ71H(*iZ~%a=E1fk|fXtUI+Ni-AKg^f9G$dqe0Z$nTvO_ma6mD>PrAx01pD^ zFP)XdW?!U_owwJB#cqh?<_ZKeQ8W?|<`Kc5y3 zwB9N|%?}`a80y%3#;z&+$n7A9Oc$%umYUWIEJYegOh^DS3$ek4jQWlWp?=LIeqJ(( z2a0wvmMDt_50ZzE94o(bzxmM%5I%?k?~T9^;zY`g-?})eKJE&TTVL}e7iK>YCxKX; zePY-I-fdg2^nNV183KXG3>zQykG=h0|N8G+I3d1ZPxo=+w`m&i$M|gtAv@M?2_bmk z^TFx1r-QX^Ph(Sa%y!!-DXT+GD8Uj12w6lmh_fh~xTgwGDfnX+1R#ZJf>Z%M0lsZC zeqIKdUs!-7Ekx25XYALIcQeyIx!1sNk6wVKZh*N)u$A`d-8w<$>Valzp;j7UyEV^P zs)z5^I%}yBYOVqt5~HKa*a>4S6`>%6I0_J^+iGUo>*P5am-QC^Y-AF=02#}Bjh`YPH;qK7gG~IjV zoS8W?Gaufy-utP7qDX-5*)!*yz5c6KrKn^#-TBpZ-SXVejR1q*Fq6S3i-BnKp;*iD z1ly@(hxs(;rA)WgERVHp&&@oat$g3T0)JKjGw9YUB_X#fLhe?D-LHv&W+1C0Spm=Q zUWftR;FGJlPufdf_SV0FHmo(j8-oUNK+c<<`d2rrUv)w!roCt@0gu4m%Xa9%n72L9 zRrUbt&l7FGOt*iSx%qLf>yL$=-{*Tj%yhk8Ar2=>AX03dJX$(?N9NWCr&n0F>182g z=1*??a+2AxuxlOb%agQDn&f=_>*MP=E`ykaT2OXA&B}bg5*`~00)B1%^B~ak{^{kN z+yC;vSnGsL3}4z&<_ry|H%@-*`;Rygo>X@{pUH6|JVu>0CBs4p%3%NG>aF3~?2=kN zV@pXXSsIm2#NnYb$efH}1sV_v3$PB@gAPg%7Jv!z3G#6<5pp5I!;e2R#!4wn;xr80|*oJBopF2?gj}x6AXK#I&>;84V@8inQuZ#V^Ef0bxAipgRKvy|UcfXzLf=VLOA=IW0U4mwA zdTQSG)G!fN0)#Ky3toacd?oYArDWE~khM?&Muyg zI_dgrN@^03Bmo}&bEjDdnKNh5tqKIrodYsO1O&mD5R1i&5-7$NcJ)^}hG*Bn=!11P z7&by^jK{h(=>)(3C_>9;c2{{FFBK%bAzMhFWo)ZXJKXdB-5i=(E3dy|X6-1$P@s!T5U@BbsDaD{dLWR1L4vTxga~N8UI5NK6qtEe`qMCO zF3^aQMR=7-7$-HkI6K2^FUu@<<5c_e$rfs9mg4mCbJx~Tg z;dCwCahs8%lB-OLLn{_n67@Udr=dDF|H757@}_-^>r#$_v=e58NpT+ARz|C=59) z3f(CRJS-0bBd+_karbIs?^efv#|IY@pR{B?yqxvoM)~vZ3(xzT-cDS9J$e1-*{+xU zjQ}CDfvkVlUjDeX?0HAkyPn2(eJyYLTi%Vff0*t4ZE5J&<)L3!M?b8O|FSy%>)P0d zwXxsU$3L!(e^?p)d2#6dZ2!;GH$gpQ9=--$DEw}q;dSqY*WK07QNrC7F9E{NVo(g9 zU(13Hyl75-+LQ=BDL~r{E!h|O1BACrL-z}UxAQzU3w#%HeEZ{^8huTZ%~ULt=1n46YZ)7MZ*7DE?IG+n3gm51WK0hIZ9PU8~K|4CVdI&xmi>jM+jVvfs2{KU(R6;x& zfnjbR1qewvG#QTpuYeX5gA@iu5+DSv4)d<`r{TP3F#Hptu;u;CtyhyBucvOl7;1hw*!+4BT7`er-SFa8-K*}4ZwIct z9B6$t(hjN|VD@oy`s4QeZ##=0cbCB9hwX*mw&uVqzwa#k0bbvj`DJyId3}91=uUs0 z?0{x#hA%-26@7KDda9WS%U<0mVHU$2Pz)i57omj{fDlx|+a-aY5Ecd=6#MQJ2dw4? zj-|M_g;*9l>w2q8D53a}yj;RuXZW~I1A$;P2zpUyZ#_Rh8jiqYMFBz*nPy?@eC5W# z@a+2N{3g2{{mcxSVgX=|+xwi8Q!=a^lg8PRvYsESdnXDYJI8Av8zJZ1%zp?W+aW7p zOgOQ)0Zq&tJ@`uqIUTaTCYF_BL;mIGZV5C|TlwOAWvAbY@JypCj|o^+xiBi%$g z!&D~ELcY*iq10Zb!dbQ2MIEHx{d}XB?qxs2wh;5qD4U)Hr@>VBkqob)G>`FY-|5@{ zkh$Ez#exu!g}k7ZoIsGZ?7)rO;LW^{?Sin~g7Cew zY0i4qlJoRh{$6eJ{nnDFUG;A#I{sK2c{A7ja{A_*srHv6Ezbupy&Ad<27d2HT3-*f zz8Y+MJ$dWhd><%%A9v^eI9mDJosBL~m)GHi%rA%q;32Hh?V0S;LRcS{1dii78~ zeFoy3T7s++%~VWjC=zH-xXZIqC7h$duN;LQD{yzxe&e2y0ckaD_ z{BW!daw7a9@UJ0cm(A}Xgu;9Uy2PG|5Cl|0fKXXOmq?;g#l)d~zyus{D2hg~0w`jb z6Qo#_2nsF;O_MTb0>EfcR78M^64sZM@G;iO^>MC>^s0;Ttc`N34t1#TvnY4bFSOIj zG*e13lm*5>5)5R(Xpk)sijE|B1-uTFrW#2CgxMByMK+3M4ypiQm8*KSn+8DG*F>p4Lzggb>0 z!i}Pk&EnAQ(#XAvn8WIL;P7!%*7KIUCs*?h>e8Rzs)43Mr*8ah9kh`HKQ9lynuPjN z(4Dd-Ip2(4e>>LsZsOLv#lhb;CjQu8`2A?*kGmWHaBusMJKMkRZ~VjU?SH(p1M&~I zHvz?u`>P-KSAIWO{e5c&IsPz1b)`MdB9JHf1OW$^tzP(uj zis9qSX-^uVTlqrgAXY-Fg`g>A?xBHhaKB#`yk8NqUKl)`=G7JJRP18lu1u9g!7#k% z_@Ik-p@T+wxViXvxdB2!9syAt0gb{FN#d5aZY`bteG^L)OM9%@(Z7z6bN%qEwZs!W zC%Y1IhW&=_ug^zg=X({BbA5uvkCl`5*aA6wQ8vQXzBy0{N9R@{gm<32-?{UAlVd)H z^~l*na^hi)2{~!8*5bdEzT02)G6Q}%=kV{Vocts3^Fc`*o8Ks@zNDn4N1;heOUX$| zNaJxB92O0!LkN@cKqDNqrX1!UY7NqqzCFp(zhm59-oiZ_&rGmwtalK_d; zmrO8}Ow^Z1HI~V>RxYwrFR)R~u}}n$<<8nw9=dhDMwfyuu0`5)#=G^T`V3?S3}yum zXZV1OX8BF#1WsfJjb`|dX9dpWM=TbEE#!yJ=LCbS7KCmUM{X8}uN8!V0ED}x5eF5~ zw`=3?HKaah&U)CAcdsS)UR(LYo~9R5y}xeE{kpmEc6I9Q^7!i+=2?cbeecF@z8dLx zGu88EVd&-J&~JOoA9k1jINJJoXXE3YgAcb4|9Eu&w}*E>KDqPT!=qpB?)-AJ_1m4T z-|y}Gc4y=L{_?NPHuabF>32&bZx;sM&h@^T>UuHS{$k+ri{3_n|5+z=!*4`hwy zPF6flc$|y%oMS@HV#qehSq4uUW9Qfu=$96BY>gGxt~B=2-${PQC?0coePFSDU=IAA zL$jl|AHU;lNB=J)WI5yz_yj73yALK8cd8m&&+C~>$|%T4%hTzS&@Lz>98^4%hyV$P zVj+Y&QW$ig5bIn$A?7*16r_NX7}ir?In~p;Jj$ys!LKUPtts9YAT0H>E%h=lay81d z(MmE^PB4~F1|`u{2{>dfR7ghZ(1AwaFu_PB(U<`q(~TJ*878tBW^zDdl7S?6%(74@ zbu`T5$gqE%Q-=-`N5!u0EeJ2J*bMgQ!ocmYu9(7he9cXzyJMeLP z@#D_&FB|heug$!hANX})2q1hvehWIKVx|`;dbc(Sp@IW9}$9<>)g?I?ZH2?cyK+`pK1yDsjqCi1W<{Gc-IPIbh++Nisgk+;jkZb1g(^W^rm{%`dV5u2xR`#03km&4?h>T2sd=kENI>fom+p5sbG|OHm#Zo=VTs74~E!|2j%~CbNOfkwpCRCdqsUsP0 z#7H(%07)>GP1cuA(D^i98m&!>)S`mNczvl%Th$^r-D+Rc3jr1v0xT~E*|bDBU5j>Y zi*~ve>ChJG*ct13BhKwcoX4#Mul{7ep=96Dw1A0Bh~iRy7;w0f8@!O|znljg2CfwZ zZIy)WRz~gD#@%jAy>}_|!PWd1UA3=A+TPC(vk?BgKKFiY_QUe{hs9y&7`5@PS7Wzc zP4&H59(lh$1qA-Kzy6PR_y4$k@V9$+f4g(^`@?%sC46+}!~KI_j&^?D-}rE_2}X&( z?5)1pTzI!S1Af}Om8rK2JfChv!7qSBqLElpM>n75wywUhMKJmpoZW=-Ss=*B+}7yqSCx=u zk27q9EP)`$q@Co)i4dxIC;73CtKfjg4zwWgErfp>HbRa-S5ZtXZZRijJ|kq+L$*Cm zir*B-e&?rB`9p*!y?rwP;nL2%<=y-9o44D#N9VgMnuJ5-R5fP~Ws zI4T|shJ{#!h=?%1un=^nB`AX^eqJ${fEopFqbd{YXq*@9R37C45LQRKH6{99jPk{b?l6C>y7swN%f!344KafUCaqy$PQY_3tY+#TrCXREDhVKh}@})-Mf%*P?x;l zkoNRO)!VUlQ1^aYhuTqK2IM0&u{ro=q8k!8(e-k=@72P{o0ahZra)Q0Uw4)OFo5uH zcMkrzdj}u{l@K`m&Q`ua~CZuFU=ne&*8DtJ#s~ zlYP%7dS6WRJsIzKJl6hXxb@zxhP8`%)5USKC2^az8TYOgKWZy^aIN65F?F{#ez!Vi zuPS=CBIaIQ{6Sgxeree4%831{u+`Gwv21_P6hxV-Dd7Z#xR`f|WZpm;WNS~@1W_EqbIDZiheLV%IT03;U|ZmS1DKsIlk+!A_M}Vb`(1P@F8&6H8krL z7_Dz&rJ|}ur%8)qu+RIt=P||z|$hz#W>wTKgCuj*-A6PTrJ*AHO@jU-clVT z)?78(NHIcRE>uS*L|ZycLo!&E7NjB`q)PKw757o0_^F8pX-S3~%f*-}#aXDtn<;~z zkZz-$YpN-Rk)5%2?1s@0CR!mPg&Giae}|*r|@3 zEe`2T^G~zU(IUb`xK43{Vd0rG;3E>cryb8Z@Fn=f6crT}r%8oH#C#l z7C*M4PnY?f5Xk9}{pA;j90J3k%`;;ZtZc_s)Y`!?T_NQMyJJic0Fx4pJN* zheC_t@ls+WfKU_xLkmJD{KAF#SRE-+Sbzu@k`+Z*DoKXf=of~%SH}g^B?Vnd3#p6q zZH)J?i}tRM@~sSYE%CR{^R&!zHBEOiNVd_Ax6p_&RgE!MjR*D6Mk~f#EmTj=PXpRz z3gWLu_g12K$dlX@KE+d&>Zc_cper4sFBhyQ8>k}#-Vc64f|+Klv0A91N|2snkd8u# zK9nF`xe#6Xa08`ObB#=E?LsGmDtD7QZ?h&}i)+C)o#Bo>(ar<$?h~m#^SOab1tAOh z!OKMvYZb9;F%EwhkjWe zd%rviis8Gp={IXLKX0zQSzms)vG#6j{nx#1Fcf^bHh*_w;GqBJeqYDYz>Rw&T@NOD zAB^^c+#BgW?C;#Zb#1e~dAqY^zpHh=wPET~)$oPF;i|mBvdsR1gx;*E-pufx^x)Cl zsJYVkmCD4m%7o3Tr0uGt-Kyk$P!cO*4l1G#D#Q0GLwBmf7fZv3G6Qp+4D~1o#5o8d z&)J^@dAJ05&w(8h(e(VM6#z(U~5NT|JVX_6~+7}G(EEbWoUk5WNu@0c4KsU zowFAm=4eO%6yXnsg=~RP+j)ZDF@(qVh_c{uBK(UES@t--Ty7tj1AlSm#LU+Se_RQ_ zE^%Ku{4YevI&7w+f700}R72ZDT2_ff7AIl}q9_cVKxBwhXm}hRE`k+io_8qB2c7pU zEPxde6hjFskuY}ZGSIwexKC|D(8Z+S=Cts-IRA?PWVCN%Y*0;vS81SgEN%Yo!rsq8y;j@KU3DsEB*2(p(h9>}2p(5@;JKtUZHy6{4b;hbqlk zf$XMC1p!aoRm6R@82(zaUaFD~3KI4TGzUexld`07El~X^cYL_8cX#CgT|H8*zF7HcP|$|y;a9NRHp4w z&*cYQm!1u`Js)g)F?9XKP}{SCYfpz-Uyk2+Io17qs_W@Y@3WbK$I}C#);*pXe>Ok! zWOnNL(%h@{xfjdR52gk-Zd@6?P|{tJHBgy5-Bht~r5?z;gv)8Mtyu{-a^i1gMcqgbyB_P;nc&}-7CfFEHk}(emmj%O6uVK9v{w@k zn$rF9sQrqto$}y~^5FU6km2lrDsM|O2^@y!CqAy9c$qhBg>JFTyeXOxAFr^W5E6y4 zv~g^{25lZ4o`L3O*a)Fz!|x#cW+h|`JXs0ZMelnE*<-w8IVq46;mMp-_h(sY>zf&x zUV*l)aUx{3qnzWqlh{roWaU4J@RLBO9fiu^+{PV%(A6(oOV>I2hl!yT9x(ZQ*2o(cU5yLrZ%O`nQRz?TZCxthqgkMgNsEzTx5am-H z?p+t{UlHz+?`NOsYM$iUoC)w&I+3Ns=(H3eU28#aL3?B_iAPU4zj$|%@ zF{UEGqqU5v4TAt4K`dxUz|vY0XD*Jiph4H^TT0=KDJX3c8bptRGp37z7}4-15~9`& zF-scKUJB=>C>E?K8LcCeWF!k5mOJY+MzxJ;z`FBZjY)u!#$ z=Nw%rd2*}nX>Zfr_6tYXYaVqsJ?U+EIn)O7u&3$v&5L)so9_0t-08h?Z>a6gaQk6@ z$AhunqrtAbBmIX1-M2^jcLzJS`mV3GHv?%m^HQ6W!Y(C-UQLT=&r0gZPQFo)ezP#W zGe5O0I}xNKFXdWp>ZOc?+JuOTsGyqYpr(Y7mc*c|2>~66fgSMyHxqn&Q~ZZg0)|rp zrZPeobE9{wtD?Y|aI+$0u{3O=AoOCeqcwws<3G)R?lh==03mqfItxAL z=Z4PS2BSiN@M?Qk|JdB%)G`Yp%OM-#*v!W9)d=<~1;=HCoXZj{b;kq(Y$p-2CSW?p zz8x07V+bLKC;0tG5q^IEN^9>l_>2GdfBDxh>xAscmti@Oio2WI>tk_^o4mDO~p6iHDW0SpUCC^QgA z$Kfa#G!B74!9=hkLKtB|JY1N97Lmk>XwpQzjkWTE-0PA;E@#EGX2mrph15oRg8~R1 zK`J7A^ZXoBTuhUkj8mLVlAVkb?2O{A^%HD#qRlnJja4E{R6`7ueKcj96)5%$qOk9|#pp7XQQ##&69Is18X^4tw;RFoCP~a6~ajYH@sfia+$3v%j%b`W2kb=@EA$d`x z3K^?PM5z6oST}>M;z9h=>kHN{SMJ?xemHt-uBm$EYQyTai{NqLTK#B!X-|31t&;Ti+{6p9ei`l-krrBU zRyt{}X2n6yRZ+e*G5+O|exP?K3G*%p^(v1EtcnXR4)x0Qbtv$20-s-3Qo}nFgIlA0 zTf#lsqCC420tS+U`VxF^MSIK^gwGWOZ%Cxznc*a1kn%=H}^lvv+uEd1nH;o)9uNI=9Q3l7TMporm&8-LcGX zKua3XbPh`(^QNCuJAG4Iy_10A=CSQ&ERdgAGB;kc8qQ-AE$nnpY=D5ilyffhoB094 zA1qz4#)JbC0O7y?{Rr8kK}aBU-2GSf*po5G?6D4#0pXl$Gj(kL(`tY zF}fx;4A71esYFo%S(HE%B}ftpbWyw*8pXOf3=Sp?9E!q(K^bHak@^g3kd;A6gl|)N zWJ`AJ)r^?N_~5EAkMdyG%20Rkq%6!c$HyKZOod|1LYQc47-M}t*htAoTgFpU+C_zC z%OIPIV~oU*hT>R#5=uuDu8k7`Igf*@V?|WaLh4u{HH@GJT2LJ&pn?)qLJBCN1QalW zvKY8DQdklxB#nj1;NVgyVJU==3`#^1Bcg_dgCAv0$2&`lddLxi)hV%hGMOgIrFQ2p zdYWGGx4aezDBAYMd5om`PiBS97shVaXY4d)A6zNgZO*@awe-%_ioGkP4{lw2I?(c@ z?+U1do6TjLmrG`=GbSq17cUmAU8&u?-q=-;-dB=6Se`d|q4auQ{I%?urj*dCNRM0} z>v%g|@YnRxlJeJ62r^WTG}lgWFivx`%=EO$^0LkFuuXHdNq4tNb+!cmP7q>jRuFg}AnaxLpZxX$^6{9_HE^;XauYypSKdUL3Mr7Q9;-wowr=TNpf? z?FSHoUY*2uM&ulHB&-l0H$Zrf>kQXVr$9aA;o*^%k&lc?Z0{Z(2c>UrowMisG&RZ$ z=j&gU#1jaiBeW;CSprWYJh>46&Dl_j;nf>M)=oZ(YT8tBDH>IhLL^Cui7}|+vQ&x$L6n3<0DjQj z4&Wl-2>~HQ$HJAwux85AF^(4CgZz47a%+C#mGtO_*uaVqH!uza4gtc_P>*bH8^8}F z$(G*1}V}#_<0&*yX6hed!7osDCX>cJim;gnXpDH3C0TW=r1wlR3 zCn78ipOLhnh0@shvbfcn zl$Gk#-KPA*mg22Tg%55tfMR&>X5)Tq&0168Ohw8-PQ+MQ>P$`cctzG|MNV6KOlNjN zM@HPOoTN*s0rd&q#ld#zE(T!S8)>Ezpey61BIW{yuZrRh@>Ew9Nk1J$@Gl|ER6X2S zElf`}Twg6*UnRy!J;g#h$HBPB#k|PHG~eDZ&(0vvUcbcIq}J2=qQ6~}zkQ>h-9V!E zVnNtOS>#q(=yp}adU^O%LEuP^e@nC*_&6i+a|ttVSqI&-9$F^k;yK4HARr_l2-7z* zFR82p2*>6(M&~!ao@n_rP0Gx1exJj<5`Xs7zEIXV_H2QFitu|&4($9Deysc_5&pM- z{CY`2f9!&6l5Y4B7$FfG&CTz)sRj0ux&_<>aNHI4lk!hf^lGZuzRt;BX9_c zAs7&*yO{!hi4MBaR$6|#vhM1V9_lh~>XNo{Br^$|2@PXH!x@TUK$EFUKbU|J@IKL)F1pEYBaf~N}5Tqg=rA$fImdP_$sdUh(b2Do8x4ItX(v|4b z6X(^N*ziykXU z?8%Ag%8qSMjqFU1Zix%J8W(&q)~hVgF2!Ca)Ii=>Q^HGy>Ieq36gX&%L7g`yBh08s zduf8JqL_yg6hCFUzlubdrgXfaa*myTiK}U`iwUroZL6DXq8h6w4<3sgjcUAYYJF_0 z-7Q;#Z02%9w<}}UOC#1wLKlicCi4TPO2V!u`hZF+CJfy@64+xJG2n6;(4Fm`QNHMsu6h=ftoZzCPoD=AF zH7}{FJm*?&QbT-bWvFMdzjJ|)14xmdL#m@8YZeqF#8}ZoL&8y!;-pBklE#Dn)0l!X zqv0(i2&UqALo(Wsgf*tZEa@maX`F*J-ky%Jp(3p*2-c$|QN%_J;VMn=RG_#kQrwg% zo~krEDWbKcs5up@(Ai1m}cgcbo7k$jLnZMZjCPQF%h!DcF3f7Vl>E_3uVJ36FVW&zhcm=l@HD9Qj5s$_~2`Wg}!c{2s#H{l|;T`*U+!?QOj& ziCHH4W=aeN2{9TpA0vtf0>$wtG8#re2@{Yo5TKC&7ZFFnV66@naDuTU={G5Qm91ojJS4-9uXoRU!n6UzA%v@Bcwz5PU2Em+;0|?D%SZisr zjf@!6A$SF6NrT%;pq!+zF48zRDXhB`)l>g(4br0f z$dkPk$UvqI6>UU->EVSmu>c;w5(=h-6j4G70fb;AsRubknvsz9BoTLURFE7oR*RNy zELUWwS><8a;BQgyV|FRfvNPJTBg)}coKsJt`*d-{YHiZtm9oRuiv6o)%XQh4B?&{h z;cd|_S3+%DW8At@{cj`(w#NBgi}kw{>R#q%QRrY4XP^?SCgme5=FK2`$&lP6@Xi#h zlNiQ{hVqgnhN(+|;+ABnm}ab;Wv-fQrBP(3RqCW$2W+g`pD#!JR4oG1fXX7{3Vf zIwsat6i_e9!z)gq=H(XM>>lVDp1w6a+dsQ8wthIcc)-fg{5}Ul=&A_Lu;CpO$X*pZ zVedx>K~ADNsgC{puOno=c=C02SR1(iUW7-Fp)j8x>mmX;@>uyh@tMlK16c>jppL)+lG-usO`BF2JrK$O-g{1vYxg`bv?SGNDSeKshn+ z7_A{4r^Sd;lMGd)MyQC#X-TIUD&?7}6zG0KIlHm{M=^+c_EF!q?&?4U2Xlq;Mm0c`uNK3(DH8I!VYAQorN!* z=Z^(`T?T)IkhLJeiSWnm==aE-%t>r)f&a%5vgEKWaftdVY;~LzLDu#jgM9w3FZjI|WXjE*z{ zVmrsHiSM6GBTTS>f=9MPVRbD-f|Xm~do&WnZz@f1V35#ja}Bos2jh9qn! z26F}#Qi15DN@0~pK-86vvm;}y#c*b1tPv5VA;POF%&jQIB`9| zKqXwM%bqBXg&NUF$EzV$b-pGS13-fHd7DQm-g4Dd#H#x z%i(RM(3Ui$i5Og;B&2+Ak4*4mf~2TR1Yhx4Hp18 zFU(_%6}AvVn~5PjRV0FS;INjmkZ3a%_we^)+M8tAwj61~4Q&7!tNL zF-;UqR+wKx2%4~=!G*|zyb>a3}7$FNH67-5eN@8((lIa$*878v1=5hrVa#hZ1 zZDHo!iB16FY+3YpNyd>huUjN}x2sM;z-;gu99g`%uxL3}T`xIYV0t@GCS^t#!}?skYIsFjp%# z)hg0g$kU+btJ5oW6)s!r-taW*3$*MHw;haf>WOx|9%c*r-fFO>c6(xG zd~W5|(&nv|-QLBm$;rd9={;!2_WZ`s;>OUzCbTj!v&*UOq()AGoDJv6u+?#{!b1!3 zCs2JeKS0O|r$El{I%bc(ceHJ2>H6SY_wd|*`yc=Q2_bv4%ioRgb76$+u@8d+S>Joc zLb&`HAv7l3eY&#yZ1d>#()N?JqgQK3z~sF%rRE#Pzn4n3w=Fa}!xv zB_c_QOqRimf-xbL426hAVqyFMA%%bzLnBEDVFm%EB|&yK&`j}mxssROQ&T#6vFc_~ zW>ZRJRfJEmzhkbaB^#lymbkkT$w3+mT1|5*+?WhAAd47~z!QWi1qB>h(nLXDYAKF4 z6T<+RI(Q*rrMiS z2YXfrcow?Y6*$;rSeWEE*rwZA`Dm%>QA8yKc}V9@(}elxA^_xBNr5vAK`sqApAlZj zfsXW3#D%Go6LsksMp8K@(v=P>Z6Rho2~Hz9{!3NyD^0}*9rbr^HSM)lt~R7k6a?Og zh3*DX=c!+0r=F}Y9i>i*R1=F;r^c#^MJW<|=}2#Jq$dUGPKLXYVBkrJEHOb-I^SHq z#>Ke7$L4a7OIw6jd$?D7sC!4K`;7?S%RY7$mYNxAw0IeOx`J4(nR2JM;f(;Z?l9Y~ zNc*;M2w^P~VWFdTfxY|%U;U0a=i#iN>AdjyypYBGfYn03k-UHsKXVNV5(g84!4SgW ze|+39o}Z*Kf|*{fgJq>F-TjkO^Sz53y(>FCv+HByJHr#(!_yl>&~nAbz}z~-Y!YI| zSpZq^*f0Lw2)}HCIl<}g%YO~RP3BqJ|4D?LVIySD%{&4EfkU7XYD9ORuI|2A-+#5T z^L+jG>t)cRZr{GMfBRr@<^JU0;lQoY>e6B#_W&(bO9|?EB2HBlBZb9@p-}`ZiiF1i zgm~x}d|?6x0T2>kf-<5=9cikMscvR~=e45jp^Md1O|>@)GwR|)N<&-=d~9;u&C~1+ zqD|C-bY*-sXs+@^2MM&5IMPB42}&8T$BGFVZA?NLiJ^_eF#1HKHckYL-$368x<@dU zQ$`4>Aca*?a1AU9JUK6l)+C}CaDHhxA5D-Ogu-`LLWD~JC!|HeSTMxBw3OqlO^bZp zYr_J|{k$rJd{Z2)-PIK|MNv|M=Op;fkCm!_Br6@1uZNoVC%aA;fL=7?pzGq}k?YS!+V6DL?OrWds*3JQa=994R%WM`X($z= zLJm_9h1yOzLVzUNhXQlJ^4nwhY%%-}cp+B`(uY9|QPU$m$Vr(kFBmP%>`sZi9_iWSZIy4R94(DYRKS%ONi}=uw+5JXgxdgw%^}wHf#%iT zCS`8=)t;L5zIxZA9mcYQp+BpU$haC6a5dDo-rF@oL)n2&Hl-5O z&@gEXMhpd=EXsC(jA5OWIJoPq@i^$5`B7)=Z` zji~_RmEz}8L@P;%oT?$DJhn7Q%$RdR40^HJYXbGOkcMi*Y4teej?A%#6H<$3a zQwYAFR3u26l47c2I3sDAqnc8Ty=8HLPkBf{p0{hfjhVZuygmt|0_T_DK26~|O*(g0 zl;L!6+S6fQ^spKzXO2KNrS2Gm;H=$O{CLx=?Uu8Xhl+>B+i2*;*8^W!3)^oU{+XRbBv&sD8im5 z>LN>ZmZL(GPx6v>auS~B)dCFl0*wq3?d{4#gIiLQYoo$0Cq;H;$M$4KHV3)nnX0B} zP}8)CwT>E%Uiw!8&A|AxCDiJ2sMV!ltBV2Vmjd)I1sH%K>u{R)TtVn!LC9KB@K#yq zQh981w1+W+LPEl!^R|&FtdIanm2>#W@Qt&65LzeKYg@3mb#et7GHa!;{dZ z^w8?x{1$XSdSGng+XC6ULr>-;zhelGb)+Y+pWxgp!wTo?C;1&4`+W}~n;+|XLKecV zq1mqxvhc9@{T&E7Uu5lifkuP7kJk5IKqJ7zM@xqfmJT6=>$?v(cOI|pyjtIV4Ja<} zKi@ofbpP<~-pJ_9?Cf-H6+bdIUS8^gy;Y%^QJ^BjR)V5Sz{ugzbPSS&MgV~nERKvp z0f%@L4E(1h!iD5W7=SR))}$gavbU~c;9_-WL3UGe3_w^O>p`AGy1sp=7KAeael3x+dBPYVc5IQF##IKIW7)#QOr0Hq| ztduZ6O@L2QSWuaW1(CrZC~#qdkRXa*K=>TD5Dyvnc=fu75}Kc{NY&JtJ$uX(>I?CU4GDAx!ssDkm7d9Q#a2<7K{L6 zlu5x-ST7R73C(Yh;&Z|Znqc7Oc$6_7{JjubcoZnz@)#Ihh=(r7F98!WQB*RLSCm0x zHN@%GYMN$>Dh?VdQT7&v!5+1d-j^bMnnT?x9CeblC`Fd?l@4n4?mCxz3;{);t|Q8> zHPWsp-ljLfelXc}D#w4NEONIx{s5#VcDW*2Q84vQSq- z&}e$hd_(m_Z}-qFK)60Qw>B}kJvt4wqtF8295Y}XIQ3O8%IWZXImwR|&WV}t;ruK4 zPatzeXH;64#A&J6(aDyQBvm3t8Ve(# zM2IK^1%rlKQ@99LL=Y*!OF#(8Q*iq765-BP4OvN`4E8rv0ffz|adol5)uA4RUY6;0 zItiw#F?#a;suUk3k|#sdRTAq&L)lW0mS8AIgn^iWVIdK&ixmbVKovODf6Bvn8Nvd} zXawM=h(XW=c*Ob6fhPu%;!fw)Emf7y)2T{iq9g`O5P<`P0_S*y!7z_bSzlW^*k1Q{ zcS((o>}|U;(9xO|8WL<}W-Bl4q9E<1#Bi3PT2VxS4LJl{jGqU|bsEKe7SDf{#(!2( z_?$L^-<^htP$FjQOI~zR?~k;e&h*`^N&*O9%yhnA?0Yxc{e1N5Ku_f@c7PFn&>(Fa?F8q0t1GFiL53in@=1AYrjkcSzlJZInwetL`uIt_7H*OZjM?2`K zm`IZx~%|N96M6$0Iz}wzdwfOqOoeXyVX{8!Tim(+4;HQ zxuvoBH4cQE0AU~Vh^KENWH~&^@5czgmy^s`;oMcz#d!~=Zy{t0{3{4q2R#9Vt^G4! z5VF*6ojquK+ViU`QVh0YT(_R6vW#=3D%c0p#w zHi~j4GSccKk~9j9K6ehq%`F1Tny`R}vqO-tmw~osdQ$wx^1`D#cOLHUu8)qk7Z>Dv zy7?$E3=l%+1^Gaz{9HtyGX&lNTW?-K-v6tX%DEot>E) zoS2)O1qe3)LeP#v#+awYun@B9-bsfi`Tcc--&-^MQ-mjHQ~z#+SNrF$^-bR#nEu;8 z{qIwUk6v#aKKgt)5;P^lxxnzPd>i4(M)XrS5Q1SLXh*kppKR|w0gszIPdB%o9PK}P zvbD8-ss5sgQ5qZ>dFoU=7gx;LGl{}Nk!X>41@U+TMK29;XDumpGE5pHNCtI@Oq8Qg zB=C4NKQ9h00NPP$0@7MjDJ$5wtt5Y_p?2(2Lw9Mx_3YG^q{vEt=Nw1F3`?y9LnZK| z!Zj#C%7g$#La+)sP?_W-C+aDK^zbk@9|>iD=a@aszdFY`Cv4A<{BB(Ii49M-4p zT`pLxOuH25TwtwRWN|)DMciKumLN|p^)fGTH3`;X7>dDEFalBtAv`}1oOv&Cx){;h z)z#P2OGb(zEh%%U>B{QL=KZaeyGt{pP4z{-?h#rl0n$`JI*^LXRTZx@QM%!3JQ8F# z6Ya8+>a&p(u$33IQ{lf?9d>jf{;)3j@Ioq725VvuYa(YW61#JwozxlPD5MyMKto{} zg8Z6-XJcWf8)Z;yJ{Cu{#Y;D?Elm$j%ubHXtxq$rAsn3F>Yd#JWpHo`x;}7VdY3f^ zGw}5tU09vuNr6m+D;(aA@0&RpPLSWm$#Fpw8`w#EoCrB{6Fh>~0m8vWkapd~@v;=M zoH#`cjl>A@LtAH1A`BARN<%R{zzZM*zd#_pehNfT75x>(yyZx4QaEOJiZUf43~|D`7y(_3kfA8FpT>%cx220ZN|RmX z#61)xeAVRq)a5;tWUS~!9UMX#Aq4tW5*Mgrf>L|}3NVD9fk8n~&`9IO(dJ7%b=6n0 zvoEEk)J8=Vd%0y;ng+J0+2in{oicw_|O;%ys{=(DU-MIe$U}I5w=5>5JK4A%S3ql$y*jeCa^DK!efhm>>S&h_GKdUBtH&> zi~A3bAp|4Cm8}PxJCBcc?>w5F?+6P?Clca+I-PL(OzclT#d4jE6BdlX!s_gei<}K3 zO%?o&l#QfBwZyR+G?E;dAW0M@qTonjeozT9Lc9_Lpv(Uf&@3giRfblEeJ4c zG7{RkCPnm+C3`DS1C;4Os*)g~8Zu#;j1V=3k0RYgTFizK+kw2rzRXcS2(x)2XVfLl_CS5BDw zJeLh?d0{8%no(>dz;;AC|=cgm2rcUyodSH+}Q{T-UF&oxhAQ6m?h<4gQf1D`W1~1A%dO8&mE!rr*7o2`B=P_v#bYs}lRO!=04r6k&cE7B2xu z$nl(ZBqK|uk)0~|eP4^mHO1@gS5~G5C+DUH=GVq1H-=|7`{p;lLI_=|`_(!2#}Klb zRZfH_rft4`q5WS+cyf7=4UALWNs6ETh?#F91nnq52(_c15Pqt15JKRJ*>E1`#|mr1 z=P`tbpb?r>(4z(` zQ-YMleC0{*(s&0d!h#@diW4%y3OZ6yPE@2T9RqNBN{jl+kpmUMo5e#^sFCXQL><`- z1Eowu)igb&R9(eX1LblH!%|Db1{eG5!TyuESvyS)hpkuVYAQ#{iWcf>W@^fArX|#e zc$a!Qq*&+&s4{G2#nth`68xv>0$hp+{_`jSV~l_eUf72$7$=7-HI%vGWw@B?dAlP1 z$)#L?@Xg4z_ft3C&)oWLzU$+3+q2HH?W%-@!sx|{`>kU;C zd1?KL5#3P%9ez%2o;J7q9eRSC#v)v&W4%^0g7%AJ!9UDFMeI>k{N1|bdyR4T8;8SNyEmcb z8Bht06=mZ+oGMaKpc2++5^Jz12w{K+#?BZZL|JIW>c|6!A&NrG1cTS-!XZjRL5dIj z<@r1w-UIc}QJmZA!A+xkmvy<$YjHv`s%M;LG}0SOp5PGLDa>CH^HOnF`T#VnzJ0-&_fm)?Cq_l6(nfh8!hRkb zyiS6@Elzn?SFqJww$nxi2tTwJzoignGGhnOA>GKJF+xH&BBI3InqaDm*OyOLdzc_6 z7B0h^q%48fe3YUupK7LxF;z`6P(~PQzzo#T<_5Xm&INvMWg&j8Xm|$(Ig*t&mICjL z4QdYcYzcCudsxz(O&Yu{+XC%70_}Rj9YC!2rL3ss?9k=x@C8!T zTvpUPnUgQll<=j^rKq@l;k z3i_IB0m7Q@p~i!$4+x~RL!#&{%p6{rVUc5fe#jxS?y)0eetVSJp^A4XOoY&S%-=xx z&GPsup9L}zG6gabf=c*JgwQqj&&Ce(V+MF*{sJq1rycE|_zEG&@WkrSiIe&;6?*-&G_2Yo>xX z&4q7S33J*CbJ+`WItbkI5WDRzeA8R_R*=~35Hap>@!Rn-_fZN$Y5EUSG$abGR4bee zYkh4g-gXUP-Zdefo$=wVF(K4opBxt(q?t~HhJx=iNslK&y8N7qoEIKnJ}H0WoI3dH z?*3}QbH$$bs+Yi}a4F6d72zst<-u_Kg_IBoVIg*tp7*Yy1VY$Be&15ORh_(#NqS zCa@D0!N4QOaELxc{0J7&6CcnX?%5vZ))VH@5#-$LXVdIw)#7j66>2{a?aDy-O{auR z;esYpgF(?-&53(i5VMjOvy>CFkR8KBI7NVdVHh1+7UZaY|Ax@@iw~|{k+^WiM@pd7 zOruFnq*ac4&OzgC24=XtsJo@ErK7#JYq+r;+AskSlG`9TKwwStFsO&$L&*As{X-VQ zLk?-*Uo}0-?+`+Ef!{=U^ec9ROn&SLi)fu7d9)4)A*JEtU;p_9A+xLjgg>r?OoS|l ze{*4mi4X+ZPBD4#$H!OtCf5dM*5}4%TQR9Ig2Dh{__1R?96!5raJU>l=6?Q6gb)wO zR6R~r+W(1wi_kq=-dmRUxcr{UT1bgJ8zO0Jx~enb6%6+<fI8{re&_W@{=xM&`^L%slYEMgwyCvPzq1?rqWNn;mpdO_r z=PM`fEOj3=w`#X9DPBLTaQ%$x?eqG)mo4vKa}>DdDa7d~d^tvj3lA!xl@dU>gbjI> z1tEM($=<3f-D{`pwO8#n7VOlJUX>GOvk?qJ>@YUA3m#VKZHv*Bi&qeYD+nRvMH3#2 zA?2kJa^mpEA_#d2q@pxl>sf)dezvI&+0~{w$g{@Vslnf^!QZ_xz@s(9r#;-gE7G$k z(z7qp9i$`J9tdm-u!IoCxsRg*0KaKs=xk=>Qcmn@K5RWdb|o)vF((!voXd)tNspXL z4V%J+3?)Yv`8z(pdsUq4n!uHdG8fMv)SuSc=?-X1^(gQxI_bX6BMyX3MRsXSrKYJKj)LczjIIN4-m3r`<}o<6n~7cklFzu zq_hHrzy0~2&~0nXMwD68K2e-{#o7;XC`S=8>*rzInq=qz0Du5VL_t)0M-cW;d_f2@ zJh9$6w$d}cG(WM_kcf@sec*rm1cdO|3HM{ioqpnQzjQ89Ml{zS;P<29s!!$>6&Xy=}qXr!u0=<^2wPx$~spT|5NQg@xjxgCUW z+X&pW=7$da)f3^=5xS`-c+=p)O*4TTR(v;Xd2cxLbGkp^^bzC?5aJ9L;*1i#4S&R& ztRRF|zL%)LovI;BR2R+AlqRT4XX`7ZYd=lXRK%()B&o>7Da%DF%lgYpI!W-D@n2KD zb^iH{b1Iw{^mwnD30$`oxbE`chK~qmkm$A8N4Ft_7S9L5>=rShOoSVineS>#_B*Hm z;Z8%rZf*8ka>jBYc0Lz1hKCKHW11qpvnSs^l+eVB_Fn&7r&ex33}0aQZ#5s0m2sa)Xm0PSZy%{^g*Z_r?&I+dlo|GmBYjK4g%e}4GCQCn_6P-qG0W28l8O8dvI^}+VomY3@Y-TV9?J$ zL+vOBhu_a9oH&koU%8xTpq*i$j?-6!sY!-O3wQ~0y9nNP=I4%4QgD=(km0@|F3cm$ zbzSX|xVp56qA-t_u{Jiyy9%FNpPgQpilimQmWBu9`Z;HMS>c=v;igIv8q(3~pdA$f z1rT}!uZBpZx^RS=P?*XCPz(?1A)g<#4~Qp7md{`Mo`=XSJ2B7$bAeINQs9OuFVv=* z^IW$P-~s^4Ks3J{qZ9uPsAP%V4v^#umgEe7$Q3PnH{sd+_^0>cAKy!mQYEeX_%4}Lj5rYjA}*-@!B#0@`CoFT&8?iP48R)vATcFN${4d5SNGGEl8jQcckbo z_(Psdb*Xv>ZAQ4$3M%MLTFg!k;!SD#W)*3-w(xy(+547~t-9RT6ykarX1xUWA`3p5 z7B`X-(GcvIW2T(=R4`EBT8tz=sDWrXiR7n}7)2?frd)=;Vus=KVmqS}d*cdcbGo-Z zm;k$D1N&eheelrED38`Kmj-{kI$x`Lf6L}To7P~P_E5Y2IG3RWPzn9!iQy|*F{`B5 z_3XG;xv=FN*g{V93@L0PGk7vRbea%3of0#JiXBaarCOOO-{%tLz0Py?&qQq-0|XM)F$_X)ht_F8=tshQep7YZ&^L3m zJbtqs{d6&~@yoNX59jl3+(&EOH^X9icn+-XDz51SDWZ22R=0x|G@sfEl1*s`uSJx` z4}bp0KS0Q$$WnK-41Ss<*^aSb4?qZBk0KnIc+Et(G``dTONqWO8glYv#Bq)w4i4XA z$DEHJj}?AUW@SP$(IyzEAT(s+6eYaGxV^wp!ZRm~KnpGe-9X_jD=jlVX)+#h(|neasn^~Uih>7N%H(jPx;_-{4fOpxQZA;Lkgw!7_BRZ zF_cH@KZ#Nk2P3GXD1d#%5cwL2GElQpxC4fw4zy^`QjX@517V3!# zqQOGkAdzCc36BH;!c=uxg7(u?9l1PntwJl^QU@c@D%blvHio!1hq*U}y3xHXsqSWE zS0jp>agCQLD1&VwwjE*iKp+F@GlL6W$cS9Yj$O-+0|?hhv5Ta*1rmgCG9!2*Eo6!q zIfad$K*fwC#Nf;f)$Vf&a-J8wdD>2jr`5r1&f0iYMVO&2y%6cLLCfu>m9}-XQ(Jp! zonv+F<4lAsY(QXL``GsovVQzK2#>a--?%9MuR!>X#h3$xhXm$PT1ezJ@Cp#V|Kopt z7aR@@ieh3-acD#qFx87N9f-@R$BBCqpA z+{;4GQuPTcz=IGOSd)pTq7ujnQRU%*dHybG9#+ZD1_>4_G5Swpv?QW5#6ZBC7%ed% zFi4p{P>C;GzyzxMzngmAoG2TQ01Y;zd zs4W@(oZt5mx2wc0XR(_uA~&3buDJ-`@(}0tlDO?7#_1=1D@2MbR+2mE5l@zeWSyPn zaIo!Mg3mfGd^0oQO(A}>ENz>by<3+LvR#t{5N?tQ>m|vHd5H_TNg#8X@EJl}U%YRn ziwQzWGD3>i@4?L=LC$C?-b6WJtgQ+ z0jY4*p}HA1_?d$m*c;h~8Bl(6v_8cyTNBMn5c(lW0^`A%cMc_X|2<)*C z7LptGfB$!m`Ip;=5Q0L-j*$I6wWIb}VL##bn+W@-m)H>wPeNNLAcUif!+BMSkDhz~ zd^+r>pMsBZc>ToTefeCKooSAxe#8?oUlATRo}1P@m-Vlm)VRQ*b(X{U*VEBTvP4gN zXB`D4(fc+U&+K%RqCD-rE%i~M-XvH=4Jnn3h_8%~C=K<`^>Zb9Sz}xb6Rg#u4dml> zWe)gBMC(XKX-j|_2n2#es|v-ci@-F*;93$W-G?a#k5i512u5;v16iE@V~oxtq^eZh zQ{gaaK7WzBUILu1_iq4xcH9>n?p|=YcgdCak~`leuLoB`gfB-)-UI@X3IgcoLMd93 z8OHJ?OQkeR1)_y~im^Ke`?8_-?=>9v>;dkIP5uj-T`$e@U)DsoP-> zx#OkpB4zIbgf%v5{r;BIF>b5qV9=4iDM)!&O4uf6Zdd1QS7&e0NL$p5Eo#~p8NXhH zTFQklW+yI?;469PB@%KJ9o`Y?Ugl(gR+SEuxa%)?Gf?n)xcKduhdhbT1koyDc&&#S z267~0g={m$9P{V77D@${Dn-_6*{CXbz zbrEtSKWQZ^?h8Uf5Ca#;zy*(|gflRagR#M&XVJWMR*C1lqml?c%xN{&4dQ#QOZy`fOcinxUoNrE6iQ z&iWre?(y@n@O!uFJ#9fF>LJK&&wbVM@;R-GC-twM)4Fh6@60jdvnK<^h4Hp#iH_!; z2AaV(CYT66Y-~^_Dh3}LTtP&WiOHZ6R>nsah4|+9xn}y=6TB^v4%%_1&ttTtBGtt~ zm=r-%%Ir&-0+V#4FbDk7Oyo1o6?3hg=i4h6Ick(TX%^XO{Ms%RcCkhgCk96!p89pO=mM+F}3LZMNxvH!;i57j-^(#w}}cPIz;m|=So=yzwr9BK!1 zCj7ey|4u$_T>2J5XnE!c!v3js5b*7b>Gg%~(GtJVz&m_Cr_Q?lbj)bB{(D*1FwKZQ5YN9b3;xHZQcy-W% zih~-+6bO=IsaRyMR_>xn_Ryty8_>Lss=bV8p2ii3`m&k>Ekbx#mxE zES~0C$`#ouR@o|%?UXC+mC25(bPw$YUxONN1B#n=q3!bm8wKz?8OBn0T``QN5L#Ub z1bS%O{8g3qVL0e*Ao3Dbn=`AisKIi3qS9HD;-=FX?$i_IIt23`Lk3J?LuYWIFYsX? zvxKM>Qo?!xay2h$Eg!K_oV-zh0-vm8!9aNgKF4u^gDCGol<#12;1DLLE7&1I`fh}R zU~!PuP*&WVQrt^e@RXJDguU@>xX;UijJ}fmfwtDhmhPsGvAWiA2w~f&#TY1U&;sc{ zM0oUD_RA71fn?TUho1$q93JJzLU@SZHxaV-%d-SR)$bsmo=wjtkR4$z4Z2wWX(l{` z5aI+~sQC2bQHtziA^TTH5q|n)^2_lS>Np4Lk!XRTO3Whx+Z$(XZd}m&<+$eQ<2vU~>YO@e`O7K0bEj<1pYjsrEemza3J;7dv=g3KnTXdc$4Qay+<(`lF{lCaaz&< zVYI3U2uw>dL0tmOfyp|L@PoD_Ius{0z_P+l zvBX9`-~4fwp)_7w9IGLcpehn2FA)0pe)tpq$fpk?Wcec>^F}@51qhK(c(T;Q$YzRd zE;=KDHuDMIZ}5?CGGSYJ=)LmP_moT^aGRXIT}1%drKRtXiSH@tJLUM7xk>BUi1l3L zVg~F*TJ&5->_S%jY-a2qBWOFjA{;^K8&33P zpaQ2;!sdw4iy2U%0|9@WDG=CuS&RmKFL?8+2>mh-v78wX0uVB=K?6vy9)wpf(z_Ss z*XUu0Qxh++*BHQuty2hl6w=G2s7ZU9@c@t2ROD=NPB*!%y}iA*t+%0b92y8)4-S|C z#@dEJ4iGXxjxzfmKX!zN1cESsWJmbpO2|^je%b|lJ9>!XS1l+TLZ&??KbAmtgx{!y zhY&K=G2bwUL+0w}(FM|P3;aI9uJPs3#m&CSwW0B)dB!ZGl8Vze^0{`?`t&J_6UR~% zq|?-7BSi07T|Q}b?H8@n9BRioG*5Eqo#wDPf5PSbX}9aY_>1vWhPsp|#x;`CJE}?m zzuL+YY5}RCIKP^dR+ovVVG_$?LJLB@bAsKVO6aH`XRHXP->$bx8`yMltqPJ{r7kfcGUS zuMXZoUln4O0MrA7<0--Y2&g?5~o@&sv}7aO6CTH+bG08&HXb^;DJg z*V0=lA^Kv-rd z+vK1z7-%^M_k5WWyqTG}SAh9YmG+UA`Mx?6ng%HuJLL3T3e=Q>dbmv?ZdT&om8NW$ z;daY_AJkfYBADes)^ZcpauQdw;m{zL84qgRi`3|a^!TOB1n>&V=5kijYIf3k0eYhZ zzfn$jU75B?$pm>*mA+Y#_PR9nRWS~{zNt)mTa^xeVXYW5n;y564qHx#&Elhn&;jiU zZmkKf9f_{ZUb?NmM#J&0D*$0R{lRnn>j%ENnPXD zi!`67!7lJWxpH!o>9A;>c8IdR~Htp3C*?_GeCEMTDw!RT3oc+g(4a zcZNgf1c%lM4owaY)6*yHE}pTv^po?w3x%#GO$h-Md?FKJ84-i)8X4ht= z)n^i`FiFMnVfhh0U@RnfnZs=~(WXjqnvY^sq~g^c#;ZvsXgmZ0;hHi?oyTbH$5@>w zOoUnba)su~RSr6}?s^SghEP28TfGhYf-Q%_ZHGdvM zL?WN^hspAWJ?06Qxfd>dH(csYj07jNI;tjCZuF$lUcKMn`~}Q?9UHWrjodHB{Z7sN zolbgR1Fg#JP%^fw(zYs7A&1ZkO*(jgfDpS^j@zoly)DDME&+8c`E?0+gJE`(8wJUr zUtP~dzb;I9Q-s?nz;5KHye&$7U7Y%+jQF-PW0R7#MbBYg2-&I4+oF&*t1@3#5J3P! z@cODO6(C&Bf!wZU!U4iTlz$t{y(P}M9p>B@Vl^D?Fop8nC`7!eBz&wdTFb?+rlrgm z6ik*^4Aj)Mwsh5W4$(Tt=pExV9piN^LjYlI`yjnzfQj&+5{`clA^TX!Ch&{8f5`8j zMaWe5Uqi^=i!$Hm(GGS|FcB`l*=KGhWb!+B&?nTLGV2~|UGxw__NyBI2q9ECp>s#4 zq4YDKQaHW_%HY`C%kIhff!U?u>BSd}*}2x9qLAp|2ZF8_FGg|Rq#I}^i1FHAJEL=o zL+9jijpH0@92^FxPMDoPWq##^hv-dexKlkmpqhy6C6^A=&^xMW&D6?NI*Aw*{DY1ekXRLIN2vPUEr8vk6{vNxqAyfCY5G z0w!=NC3q1R0tFv*;LvXY;|ns2bf17Z0*AdJrrkltt-c1eu3F_b%DEr#sBm6uBGWq=z2tfzie;}}LdbRJ< z-P>!>X2RLEu89|Yb4!5Y2xD%QG2fC`icnPb<+>Ac<9fS|d7=cLgeyC8;s4^{BwQbnF17e6Y5rtA^fMO@N&i`*_7gxcNmmx3_XK0`6d=Uw%V$6W6?4s= zm)dF4+>NST^_%=myFzSwLu~s)ZHFQp0K$nlm#J8{Ik@)%(timZw1f#>!G$fShBGsd z4Vp~`GW`z_dQK)djm6sag_?m%*yN>4anvZXR3z%kAf5|HKfWLK=zg&DJ%34VUvVy9 zQO*D{t{^eaU@^`piQ5?Y2YI@WXqL)NPFkQ60`@QQ;V+3Xn|UZ`b+nqaQ=JX6O(X46 zNv}(B?<$F)Dnd|)%sxW1_ z2)UG#w3q>(N{Q-)`!_~AHALFAM%#n`OLIt{HA497f<*9Fzpl+&svx~0Lyc%(T_dv$ zQo6<|T@$p33osK8^`MAKHI?-Urqu3XA!a%Yz}4Q03oo~Jq?A45ISIH5_F`{Qq91^ zO6TN!|HQ&9=!)C=OFaBTM5KZ^x%%9l)07^$-MMOb@ubFyb zEiV8NQmp_&oqm7I7ckFtT-ZhyY^xaiftvLJKw_3AD(M|16U<$&D^h_&(2MR=qymAU z9o;TVeqWdSp)vormXZ&xCGT5GKw10HSo*Q4>_an!mHkG_Zhh5W1NnUub+4^vzrA+9 zqwYg@-EX}O;B~*9zSm0GZYTkT5h{tbh2U>|T@DpPFgmW3q8IZJ^BM3-TufhLKx3qH zW4LWwwA~EcV=38xGb?Vh82x8^`7$MErmEmYRY^~6Q%7S*Lwi59Ypk+sys~rh3&Pef zl@J)KZDSlo$h@-@Chgmg{ng5PAwanE z;V%f`*evv*CFa2vpAbR~>S2VR^93RMwTh$s*b(+lt#nVVf%HwT4Nt6(jju8$*Zarj z2B#PLW>-3=7kefbN5&VYMrQ|eOA|EoJ#KTC8tRg?6utOvm|Q-iefEUbF%I1mCp1r- z)I5FMocmINmuXR;Rc~5!6Df6&T6uu5wuMHit0<~3E3PRnqGUlAD#)oAN=gz1n^1*{ zfgYV51l`Pyw$hH*e;%**46ZAmq%V&$QbZa)OEG?iHIz>^dX{OTm~E<*Z=qafuLTe? zchhwQ+xJB{4M#bTMneGz$0J>)VP1=aMr&Eq8$%9uIE1Ke*}k;F`O@HE+S| z{=&C{MQ=xlaU$gevbChi7S92~z5vUqc#l83ibiN_Oku(G*6g)F zuOHg$b~~E(x*Fg2H2>D$_FI3;$L_}W9e^c$r=c8Vx1kjLjXU&wW>fm6jIdUfvQnJ9 zn2mfvgbyM^nxkEtBJ4V%>_D?HpXl{2Gj6vm<=;B1W~*{0sAXf6s`iHVmX;oX4V1x( zu1SE9+5xqrO|2u)X=?`v>3;(u>v7qj5*}=l07b74(fGB7zlzOdLiR1g&DD<&QxCXl746e7uM#d*=-;%S}Z$Fz?h z*ZTQ{=BX0~*UzNcY36zvx1odDa?^$>mHl+kj@GnRQ|l^9YRZae1^JY0A}u?uih!v~ zK~PiRG%TzV8JQj7o$cpl;5s212z-^1@G=u5ag7LD zNsR#x*9nm;_|RE|_gJjMV1zY5*zBoaZuLAv_YqP>G)k5q6hJqD8!o)pUGLujZ$Ov` zK_v_kFyttCS?S88?D9}2O30ry_Tg}usw?^dM#UR&^6Q`!55vc0+z$YEXn zTQC+@fiW-jZ87FeArfS>7zrG1*W_&1=kK%>?{`%0cTx5`X&*Xh@7rmA^f!S*2N~zi-Zz81?``|g*Z$i;=O06zzYny7&);{{ee9_H&`RBFs@$nB+pdG;1NNH~$l*(B z>S`HwB`WM#o9jXYFS-`dpK)Y=E| zP`f88yQj#Vquk!Xagv5T|+fp2ltM)kJq(Ld=KHFned`<24jAHaH>7K2%)Rzdgn%-xmJ>@j59x{!R6DYXHM(=d`kD! zX`QpDbuRr3rnr0$qxv|{j{J-vaz!t#8X#<=)9R{9>7|9_+??|Cl!~;JLR3N}HVG8N z3RFx5A}Tk^H{Hh;>u8*8t%J7ELYt_eO;u9NRdH78OoT*}=PZQfP7uO6AIo|#izZ*o z?oj*TDA%!Aw@H}COrkf`h+=}*@ZkU=7U4nQ)HIW15v} zhnF!U(h1C;FA0&a^Wl4ygx~86KQ@*Eft&P#O=`|2jr6XX1YJ(3AiOC`ewCN_Ixp#M z5dyTMJ75lMEZl1=f8R}h-&_4#Z|!g0wZC=M{4v<}`#>vH>-s>^Yx!fa1CRsN@2`=* zzealhV)Xxyv0mU1DE-*g@JDyUADuOzaRmssYKy>R2-t&87?j^N8ia5;AG45!Vqjys z;(b~p9lB#2212c0z}z?R;s3KHf0JG`UQy6brVZA$v^I4&wGDzIM(v)g>Y4&GAroOs z+b9!ZT{rYV81M`A9TPu5$eaLKnh1fnnN{elrx$BO(<|T=8VK3i(L-UGJX8i*8x9YL0qsBRomd974g|b`ev`30 zGPXF#SnL~L=pLW%W-RuMFF_Bk7+;xYycloqC51$IN=Xn6v1FF47zP=nvQ8ZYZ$5n{Sj_-yI&vnOpzq;JN3Yt<+ zK`35AH=yMi3GT>;cTkGLBz;-%r@8{IKn0#h2E8K0yv;^@C{OznRKkYR{o0b9nj+8~ zZdd0)mkKKhp!eI%hp%TwuVhAViun`|KngU$X~+)e~k}AKMwc(K05H{=-^*Ny?^v|{NCO2@t`DrXszCF zss#9fZHPTBYmuD1P9dz95|^^EjO5sUcmUPcurAcNKfrh<$$h&3@dq_;tGslwsJw$x z4{Bh2=O}bvXvYY6({OOsD0qcF>>6Wsn9Q(Wf%s-vx2GNC%fif2X5S2JVq~v!tgzpw zcId0J#=bJ9Opv;svfA!)dKU=u_B7@lYOI^o4)Oacbf_IIuW5U;``b4Vf?D@qL&%;( z2>T}>e!a|-xepKyGZvu(*A5T@fqkD5_76?WF{W!WNRc|45pptR_9j^6M?Sn)-EUsj zIl-ZG@tg_w4I93j?xI{V%Hm~S=8c)S&a$GW^71B1RVSSa5YmftDS6o}ghffQmB|S- zTw*0Mx-c#{KgccJ(-!Aw0-91V941?7VXZVVmKt~yHN3Gh(L^cBTonk+w^j!TDIR73 zA!tY2gY0@j!A$7Ni1C<&c}~N<7nA(IAWR7dL*Qy!3~&fumr^5VF(K1PzllU|Mx1+p zxLv!y8O>R%*i-?hBo-=u$BFj}7;8;#UoyLM*^~c7W_l7o|E{x5c8Uhc}FAc)MaforX;Y--K&0hLde{dF@jWenwWwgYu%wOF(A3t?LI}b_*uch*CGaS` z{~E$?R>dD6WZEM)53^sHV5PL7xAZ_@S$z*XLYBH8!DCA<75cT(>Xz3#A2~KQKR^hl zAbvyB&?5?=_utu1$eu$8`x#5j9MU*E29aZ(jSoF2n*n|JO@z>IPA?6OP7XJ>r22RT zKYf^PsG6m#0DpKt_yMOI&&_~GqLHdk{GSM0-@E7{be&|U+mMCrswir%ENiH$XdzeB zmln_qb11o_JWOI40aKQWE=`7mcC;FwL`KJ!!o$mA1F{2L2yRv>4#pT8-4tt`RBK(F zmF59L)ie{;Y)g$oTkR4Dy$Tm2s)xnFKxo$$>f9IZG92X&5Kbrff?jkP83+PhpGXc} z!iFv4!j}%}AG?s2PmhUvz>~}PL=xJk81l8}av625VKK5_pV}Fi~fc(W61)uym*ar~)*46rZ zPZL1+zJ&(Wg%4 z^tT|p>5)w36O}~0gt<&PFRENQqj2ez`t=LO_ilkQ=qG#|ltB<*;hP`Gk?qG9p~(k^)}hqb3M(Yo+NQJL&&E)(r-^?UtH%wdK1tW$(zjTa~1( zGUB_Ul+C>4H`&Q+naE`#;&p!VMnUqceDuo#?Ca9hO>)K-h4hX}dP6PVX{g$3tJ!X^ z+wN%kFwg_w0fCS`#^}EtXk;S%qqp<-?zTUBnjwTu)cv}O_jTm}A!vg)DOsykDQlJZ zl_KmM3BgE->PhhFL3l0Z$FG(qZP5t3^qiMv*)yd@jOx0crtZ3?UM523Rta{8f1e*S z->Za&wlW>H_v1>)>Ozl}LFTug8d0`NSX$Es!cxaU5nqF-g~8wf%A#^ym-2wC5DO+gRFU;u|LZ8%4VuqToTS>arD*$m@nwINO|34v+$ zdZCJsRv{vD!@LMywm4_g6gz#KjUGUVHPc8jQBT%a#^^s!H&M;C)+)Bst8g|Zds@}H zTQqvxvWkIx%1@-e)M*vm?TZ z>SLPkq?M#4?JvV)diS!@)pHLooRmCw{MohhMm)E?1a2}Nf?m{D=$4f46WzZl*Bdv)f2El>YG*1Oq4e@QKRM=1af z-<4->l@PXz30p;k*EzW5bo3etxssW*n3k|Wgl_U&J=lD-m>&DGFm;=nzt>o?(*(NH zT2K$6=5uTWbfIhrhrmquXMg7(J?#*})>`OtVhwb0f4`=9w>tkFHD{#)vs{wAkcXPj zMKV&O8zY^&5kBi>=$91y>ng&_65>K(#$;K^2)(|mp|hc(hgkzzTb5W3p=*PkL-dY8 zR;t?vK#uZbhFR;_5wdRJX7W3H$M)w3ys%F7{j~b>ol405$yW%=Zv7 z+fh&np_vdu2)fV%gwPEx(0gcY<|w@HA!K4>b(?($y(KH`{MhrEAC$haF9{Vp$S7vP%r>*AT z<_WO5)c60 z$jrz96X9=N&3|+>d~B}StF7Fnm+n=U?9q$BP`XXceOZYI2SSd`KD7l z>A#nw2oKo<#qCglG!V8=SOOu0tPIS6@p6GNxyTq`^irx5EX@OCB|RT-$0X(Vl|~bC2l$jTn3z@b0LaVXRU)TPO2x2bqfFdQzTf+2*c;8|qgOp^apQt)^}z;KLr zccfcOsB?vn1<_suX7t4IF`w4Gt5TQF2%JB2=iI3~r;bTqI;C^>x+@=-hak6?@Lg|_ zJ0QMdcLJq&LmvsC)TGn(pO%`d)H><{gp6qSMReG!jD$C2()*6;e}6IbAFH$fxwh~h zi_?FNbnmyS zy)B^OY=a!K*F&bmx{e``zmM?iV=TVhG{fF%vLj@v`*I2X;0gr$wm_Ccc7ZH@pEvJ9 z@R(uNI`+2n5W+*-dD)Xg`EYJS-2&bSsNdIdHkHY&wx)^icMA$zK-R(UxwLH$4 z8EkDQW6*F@eeXxYevB+Uu6Pn$)>l)O%Pqc~~`jTDEyxcl+6bVmKb@F&6GL9^>`` z?z4al0tn}kL6AVWF9YTe1onoxwgx#>`&#F@7$liJ3(^ob6~C>(c|qvHDc+0cZl6EP zb&5mu+)rwp7o2!*y4>e<ClSEdRfL4 zE@6%UpTot08FeVaaWL455o$RdVY`6xT}=&pm6P&fqH%J=DI0O3BZaF1HBQJD%5&g3MFrA2_W!M&zP z@$afh3pJ#Ls*LG^)S>*0?uwH3=9b2e0U(gxday&{I|z?DJj(BLpo@p>2oD~L#{OKi zueLIMQy|n;vLR$&fMM}t?$Cw!9em5Id(5z3lVA@^;13XjN>~6LgWt6Fc9-Mr&PNF0 z#QNy;2J_;?(B$&Sowrs|aX%79JEsY8tu%mrxMxpA+ht72uriZJ+9FhPBm0n`^)gmEgMa z2;HX`eR+a0gfPcKt}?sa+A7PV-%j`BxO>wk>*|NBV$`!@Qk>cZF6Ih(Z^Th+u(a@r;(>vdK3 zQVDUK2z!x?dn z_CEG^eC%uet*7ZjXZ?O_&2BRl@Y}5^1HI_`nzHw`C41C7P;%dur@yNtzQ|6P$wQ21 zL=EEunquuIGGjMrS!)&PbEO%5*@Twdtd8pHwwBJ?mOke82qwQzJLo@y{HDMkBK(wZ zUaUXr@aTZ|>63cs>1E#xYllS1!4m~Rpz~uszdr~>v4q}PLT`tjH+Ybu>MoE&_l_P# zm`88Vr?(c;8du-!alG070A|9$@wMUUSD*w9O@Fx`?VI^;M>w_C%~LOip9ki0Ppabp0RU`ny#%vsggey)(m*6o)7*sAQAsZ2w zofHcY7Ge@B@JKQ-nVyNIr6&W5d6B+ZAs(bamrP$r(2gcs>m-_}#OlbywB-`DpP+P} z;0)vmhR?E0mGi7LN*wga4u%v*qiSa(Cc^FjyS`xife?q0PzOeYGw4XcWH_DR3j_|w zx{t(p3`BZ#ggG~bI@5z3@|+CedWxRU#H=3iD)QeEzj5*2`IC1cho|qIJIQHdjEG=ralDJZqK2<>M z%)+-6rsjrYK9T1S6#w?Wb{? zvM`NDNt%z6b!1a?<%k9fBr}x)8?91X-AX%ss=mwZBcdw*|pkJ3>>|`KgepG|y8WPSI-@@BVV~?)fu(moM;x zoIfLQ_LSts(=u0nk-qwinFx` zC&vsC;NzL7$qWPoA2X2P-x==I6>1MO4##=Tq(rXeVP02|-qMP;>nnGgt2bNd+bz|* zt#nZRfW57nQV>uBx5x#Xm3i;Txx19yeOmr5HG8WfeG{~@McAdBq^ZofF+yk`%8Tr8 zG)9PAEJ;}^&1B@Jb&%3q%St;NS{s^ss$2SMyP@;cSdA#F9sL&~Jal=GwWsOOuy+*U zA&t!UM>k2Z!s7R1gx_13$%SrSYc3+!ufE#hc(wBhVgJ;6@8lwMMDN6>XSOgAvWCEK zB4i5uS0bF=0HZ1Mzdte#6~m$Cw$`G8))Yi(fP1lvWtod{mZf^Kp(55wGtb+$nvm3x ziwBjkA`M%LLj!@pVLmo7JuWg67EMZwDaWBGX_%T!e0d7I01;gn7f1^A1PIf->@W^S zNoMK*VZ7e681+YSY7Z0Cq!Ahq(VCBNI#1G#6hTv3V4+cDrBz|4Lvt}|@HA`lwd(Y< z>hibl4YckHvgr%49SF1U4|D7eweJYBX$`Qd@iMLQG%EEp&2=+GnW+Y=$~Zn1Hj%zB zcaKZ-#^w8$&hlRRh5!1mLN~4mUb*z(;yKWl@?AW8|Ke$$^Cv;>|8i2`{7KO(zo^}~ ztjl@D`u?o|Nq&U#!+dj%MnAj3IKRnw|B2XuDOlJnHgO>vzg(EURFF2Gg9qht5FSwL zXF>KdrUuwF$NF`r#15n-5%rB8Rga+=QDx|o6A6~ z77;fpb0CMc6`Ku}+l}Od+W^Y9XvOa+1<;}^rSM%v_GWn|wDGqpYnM#gsRCs%?M+GQ z>-^-^Jmge*EQ1g_fbt~!7)+$cEEeOJOLNByNZp0`9aLIdb60)a0H9LS#rRC_tE&|M zO$gZ+W>^T>ul_+}-q&Frns#)*{HHg}BV^d7zGCJo=s$@thtg6&X)da2Sb4e4v9bGc zbY|@k!ojIe)sHy`GUvlX^^m;`{tFTIjIQ*KeZJG5UrF8XDS&DcNrJ>Bd?l zL)CN>DDdkq_s*ZW|Lb{>yO+-KTs#Mw zRZuPOUpOOs^^(H1OWK@QfkQt@eze--5<9&Hf4lw=mw^!XfiTa3n1C@%{B&0GbS8#@ zOYDyiE_X6Y)q9d|qEzYq|MPa=!EK%Cz4)DMvKwdDwrt62EMo7y_uhLiAVClS0T2XH zN%Tg50N8u)y>}8|?_IsiZ5{6>n`|~cH@Ug@_uub5I1~htlr1}(JM*10M-a)eXZX?c zyyd(nYemA`s1mCxdCE%BYOQpoM!ZrjSgR5&m$8?`bfuWSA}CxDkvB^jhn0dy&5D;j z_1_G&emB2bk0_=er_}ih{?*^)&qn|ny zUv|nLH1baC7<&r*BsKA2i|Ew=Xh*vq^!MzI46n{CEGReT)}TGv6Dwz9!prL6PeFL0 z7d@Mp(UifnsZlM2nlcE_#JsESU3^scELt<7XLB$xw(muFP9R(fVT4d6{PuTe2thjv z5JD$1-Fu-CsEzPkJ$y^xk!J79#r!yg8;8&p{Ct3L^XSF?@zaz2(}R_j$+FV2l!Ovb zXQ``gRiyVABX6lpuwGZb++DZSQ$OBXIan|2k_nmwtmYDKLkYKvMXxBPHVQfI68>mi z#bAXLq`OqmBcwGlaP@^bwfR{9p)e_k6XnGWb7KWMvixj0zScZnD}k@IB*3mb+!ZvX z&54l1&a~jZtcbDPn3=r91!A(2n6g}uyjqyLPR#&WrepwtQ-p+3Omuf{SPLq&HaDz1 zCqk4QM2+yw^>IjaH1jvnbJe|JbM1=7=bwT=6t8`3dHo7((&Y0i#@9XpG5GXXpkFom zEoeQ5jfhaOjd2?mY8A=SmKEW)!RJsgY=!2oqUH)(>9InT;pEoB04i)?BSDy1bo zsx>owC?B!Jqi-lmR?4N*63(oGw^%J+sueBP2$eMgWu;(V#$J@Ml_d8bv0Z_RIetG?^4 z{7oNpx+!%0VYmG2Hrb0-2|##MP1#fA%@wCTX%~Mz()8s(@1wzigNezN1=ZZr=JeV= z^yGwcR~zBm_3+0bg!#b&wP!~)gTS}eC+6RJ97!9Yb`8|_rY-Os!newx=4lBngs{UO zLa2Tr421CSpxM#m2Vd@d|J!U~MTt*ec|- zO8Ipnb{&sd&u6tsg`gcBZ>RzS+eGYICZ&yCP({j7;L^%5sWL>2ATf{~=~)!w0wZMm zKyRMld0UG8Y|6r1YGS0GnA z^_k`EtESgKF~0t>>CG!9H$E}C`YYpWADMpkE3;2NwE5^SJU;!)sM}X^&2?EG*77iS zQ0uB9+-st}n=>Q(Nm(Ps_(2M`3zyZ4%O1!_PSWxgIOHV(cezx$T%lN0lupSxi?t=H z28pu11Z1IF09ek;1yf?il#n(ppeRd=_iMzD+seP_t@*mI4vZ~--COr$9pY=lM%|?dFBJ<|@{wJcm$o#!TA;gs^*b{^hSShH)fMcJK(3v2L!j3HGHHaKS4Tj~b229yxlb(Jm>!MZg4QP?nc2LdHq5;!4sYc=7(A60*YG zK|9I`um_cp>thKJmiXHOgtf8W_3^%7aM+q006Nmatng7(^aL8pEGBx9kf0S-gnyQ&LyWz-pRuls&UIJ)o6h>TfJ3Vr zSL}5^w==k5qyM?ttt&=1KQ`0-%v|rP`Rz|Fbw0DzzX6n5U-{7K$}fU%d<+c^oy-M( zj*<{Zagcpklt%|TVT^^F;F9`jd0qINAqswkj-O$ZR3i3Tsbr&~T2)ptDH2X9I7@XB zWs`KVp=7RBI8(u&lyRqJg1IvOtc*1yDOSo@J5@qZ3?H?~9yLjxwR`zvU z**C3 zcU$>ij5Yn$@X&+aUVw0OWovGAXL4m1Ae>)^)IqNVRp*a_P^$)Rk1T9y3)Dua*#z^3 zpE~F4^{X3Ecqu~Lp$1|1%u4t48b}Wq2TpH*Kr4Pz?}bqNtf$s-_1w(p;)`$o_JiO4 z-9Mc?dbM}@3=|*uOi}2neiHZ6z{fYi+tW0(Hjm$l@EkVHQiS&RYG?28`R>89tplhp z1r9fk9&hhI+}*ybs;!fzrAneBhw$iiesOPJCf(P87UbH;Coi^D_twa}Dofi+dF>^v zwh~SmtFT7EXq2!UOIRIpL2sG3Rm`qqQOhWV$^w)OlO{pNmt;kY(!;oMzQy4#j8G?L zkbQB0HN(@4?rzNTG~;@ki34rQqoB#snmC`PB)|5w0AQ>)!@obne>^963LP?)7cq&8 z7(|D5BK#Xty=zl^>eBrb2>%LHSOq#lmK!BTMR792@lihM-nQXZCf-JRPI@;T^l#f6 z>Dn9UfE&j*mCaO%2c=9fDhAgYItAP37|6sS|wk#OJB81UN-R`*RhT&$cGig zbun_Di`cGXJ{@g(In)i>(WBAnjrrAu)$NImo$<9@=waNoW9XQsHOSj6kh!v}K5KLr zMtCurjos&>jZk}=sP?8aP0@n~gl`o<4Z_*;9j5lE?_9;xE`6E+a$V!l%U&)GyhNcb z5DsWZ+ebhpTs*k{>c52$b_nCS$nQN7o};*R@N{SY>E`~EjU%WSuAe;FIe4_cdw*+a zx|Jyy=Zm%#@)-)TA}Ww*Z%D8=07JWRPy@>)-BmK+ty9VY6svf&YCatxY?Jc`YGgyz zvd&U|GrzckMl2&B0K$^&I8kOKFFA-6>&*yvp@%p!{B4T-t>|75!eVz5mWMIl*Fqld zR37b78}D0}=+l(w-I40oo#xw<>N9}w>reCSN%QK;@NP@|}9n3S?k; zPG}h_Oqvr>f{Nj0M-k(EGkomAt&Dt(b)EEYINZ7EXr$+0sOxN~=W3+yVW{Jxcf&#V zs=eM-JKfLhbv|>{yXIT#htHYFDQzZ%b2Sb%!4}i{RZgR{Fg0)XU+U4jojxA+?NdkU=JV! zff0g|_^Jgc6+N!yJgnm!R?&9Lh%18Z1zy%(9p_PB-Gi>ylb+t4;i=`hmD!c;vDKZ? zm7R%|!>RKK_h5u?&fb1Inz7%x{2)SYf#+M$*8!jW1cVxUntTsJ?J@|zj|ooO*mB#j zvTI}ki2LBLfB(()kOQrz@LCnlXoH9JbVfqg6rDR)3uW`ga>;M?(aTb9a`M& z99a^VR>wrqY)$aSSBVzaE6}Nv)zZPb^1-I+-fC%=oYzvyt>DuXOcJPv)jV3OlnZKO zvycgT(Mo!OEH71pP7-EEb5nv?3BJr24@QI&E!d9gZ_V(xVtAWV-Hf2OBe~pRd76lW zY%5|sYZHBH6TE9<-J23U+f%(eQ@pxTy}MK0+mqeeQrsI-+-1=YoFEHsxQ#T~Q;G za1;EkhrxAUqgw&SI)P?-u?`lg9uC=lZp7$ddO{>7&@0x)G}`nw%GI0~@7Gg|S!i-m7LlY~UW(7VRkVm-$GQ0DV-;yWduM*w(z;-McY9t6bWcUELdB z138#lJ%tdSO~#xxoB$!bo$bcfDx*vWT1ueef1-@mj*6?+7AEw2({Z$ z=(~=f8&C#UPM@qDKiWLFzqfb)WarVn)x*t>{uVS&=;J}PG(_L}B}VTfNov%1nFxA# zx1kasY%Aq7N!aBAS_Ow9Drepx-Es%& zO!fWD4FfEVgDj1LZA^k~jRGw6{mgWO&Gf@9??hV}MpzjoyV_zxe8>slholQJS)};r z1Q&-yTf-c8i{fa{779{XFJ14fS*(}MRthG{_~2%{yAed$T(R6*wbov@*ib%KC7!7e zj>);B3hqR;Xu3`^S}Ew27PpJYJyQCRqIgVEG%BHt3Q54>ObK~Wgx@GFIFJ_}E68`N zX;39>;ytZlebFTPs!;$3I>n<^^r}?|O6C1J=4mbKpo+F#MqK9Qtnu@XtAr=DWxLHy zo4x%j)5>|kZ2e$z;|N+LT)#K9qCWKgH9}Y*v~^}#{rQDlRCht(n@!sQ;kgNzi}udt zQiSL8Ho~*gri!ji%`=>Y9rL{NH6kxPlOtNp#Owg z(32;d$4{VHnUiPBCy!T-AF2`FJzCw{9-Hao@Hs(2h1TYHgPZA}{v!3$pOh#7eH7Y@7$Q=-kj{-lH$>r z=v*7;B#W?Nt6%g>39+WfI2Faa0)C?GU?DP0fQ;Z|hZkjp=Er&=gPoE*ETbKb0g;{>iC=M1z8z{SsMk~8H75SL^zp-+8c#h>qlA}#M_%BJDO*@*yMUU(_+I#D1zTb)GK4QiphQQ;^8vpfRxrREEp0NjEM@y1jGq;&LR)J zF3Q^~$vY?`J*Y2!*2sNQ%X-xe+54hN@T!sjs)_qW3;$Is|9LA9e8GMtbyHEWA;YV< z$W#>}S!nO7 zA>dg7etD(_$lKSlZ+(im{b?;Kb*Zrew4+_MitZ{&yNugd!m1R|0m5<)sf)wNpdz$jw<3qIVR;lUn-21}5lBUo`W9L+I7K4Q!B?E!-Dvyk{+( zhmFO%6{K}JenW->m2g`~d{A9_T3)`>(zY@*GQFT0S=|{~fvVQ*((%mVF?3Zz4>YOs zW+*tR+`TjxtDklad_6UK3Brp-?;OI*?fq1Q7e9mpBRuC2M);;3{o8-|H*JJx6kj9M z=6B}ja)IxQP_q6FADFhpB z{W$sBM=3Y2lw~E%RF`%r#7znjK-g6#XpwMgg+;X@dKI4{FCvr|6KmLo)htpC7$s71 z{G5d1^hj!a042(c9N|U|cOi#3P=XyuewO3_OOm%K(cOURZORI;fZtmvi*YQEcLAwP zbZJQStWR~Xj(4hxbCk!}3q!3K{wDceMg(8e{17`*gbOjkm7e6w%nTOfMu{-7?Cfw- zf_F}^LyEUK5Ex-+7;32(Y_1b(sS|CdAMa$4=xPw-WEk&ilImrN@U}#_nPoef;M}cf ze$L!*KVfWe11`ItL+jztniwSDus9=y6ctE`@Z}{0)#Ee9rR?>N`klV!m9|P%bJaq9 z84x&CDFOC2dz-iWTR|lpE8`87vU(*&eWlESGETpO-C0`HRzmBL74?;~hAX*)@}fZr zWk^Kob1O@O#Bj-gEjF9uRnfaiO zc2HTkT|wNG;{n1W8SSgq+Gq8Rhn?MPqvMlHs{^as0O5>ke{tz(L5*-$c?dh4d@T_C z@gl#UfDm2}yjTV==KB#ss|)Xk@KTfSLZ}gVz7oDC!Z(o4+k>M1Hkb@c!EV-R-^8?d{X)fe8*0lVNR_X=sq4b2G>2Hr2(Zy$}t45&P@P z+A5_z)zS_*ua-wIr4v9ItmKhp47{9)6gPUB079S<6u@pVy;Df- z64QH1i-Eu{S#eJ}ySIV^5caF5X9h~hpd$qc`&fDHh4H=A#2HTJssy`JS#VHGJ#8p@ z)W~_-#8baqP57#r_p+Jus*M99JgFicR2A%25!YpyB@W`ayy&~W)-PJxPI~&*CMGA8 z%YCa`!^^v~%6;Y1(bB^4{5ud{Tp@&Wk>5px+MCqgh4Ac=+_w>4>hNrd?>!J|m$>(d z24UOC;#*_F-~SVYaPI`#B?HZi-iHys+PMoI3i|!1Gzd4|HhF2zH=hSjE)A%Gpb~-r zhv0tg`0>WUqm8|XCp!{q8$r-Ofatc z7#~xNzcp0H{Oz!j9<atP4VHsX^9sKTEcswKT-FGRm(tJ)$=^afF0ild;w+gkyYKKeMon zifhKFS0NKX6W&e6O$ynoHHy8i`lErCz3!&1u7;(CvdMDMe64h|vlf&`(2LGjmyF6d z9fHCR5xG+WjRQxk`2%uxx3s9Ugx)SJ?BEvkaEZNKVh;<~Q-tlIr*x1L2ALVNV(hYv zxL;TNxI_G?UG$<`{<5p|WryThGyj>o7k%E!ebiL6UxnEzN3Th;mqnQi{LG^|_IIN_ zPue?<1_st9CZ<%&qZ_;9>xTg0(!$Q_;_m$7-rVBev{DUe8M;oXpg+Rn!;1(nU6~k% zD&$*7Il(tjJNo)iQ0NSpw?}~Qi4czV-)dIg!6(}5eixZtOy~F#NY{iC9wKTGwht>i zMwWUfm%(rN9|z(4fj38@Xgrg$Cr6R?(I^ClY>D!JC z>A;7#Ps3aRw)9AQR)ifd%&s)jwJyQ0JuR#^D|!T* zJXwfb7135oSb#>muiN7h=XZwAG5z)1KDHV|@?Dx;L7tR5fyVU4Fa0 z21Hpeohsvw%Q%A)Mi;-ZS3(^sXO7qK#%lzfLUJQJuZDrDqGUHPa=JLUL16(%KabE& zOKl@1^wQF%1ej$R>9C&jv`g}=Tl%s`@uIu*X`A>-Gw)F&3j~Y??==)1Rugv0Fsmg< zr3f+0&D^an{${NE#X$dFUq7gXGpg0`t^J9Oqq&tsm2!V&X>U<^Fuw%7icsSUR6_MT zdM`(KZljDA#qUFSb_UGbWl)1qdsgNmzjF@Hea(+W2>8L#7T7wx&^ENxG61e~GwTN* z{QW=w`@N@M{RD&;nSFoGA>2H?JX?n^HV>Y!A3TEt?l<Ou*3x1s9kaPOg zdo9&#HL~SO@kX6&tEp_YRywcXOiPP5TFMt|MMF|*zl1z2r;k^#hvh|$46F>7C_qNC z(}RS_h+0x=2P?OmhwfzM^f8g0sFr$MM?0=1Z7Xog;@o+D)<|*6oP>DXS##Rcw%*mhGCnh}+8A5k8-orMJ)U3Q zUs&2Xd;605&Dekwyk_{L5WX&k(7|fpI;uWXRD%#YyGf0(e|FC-rTPlV?jo-Ggl z$QJZ9LaknO7@Csl8e8cYQGsGOy?Xc)5neR*{W)I-zX##Lb3krm@9DZan`(hjPSsnw zPoREv^X}8#yH7TDj~7*&^TWfl16_R$u|Ob*gp4UEKuQa8 z#DpvXI*pbRR+tb39Fi0K2oY}BAp6_^>l{BzjHfBe%?RsZ3I>OT{#FDpW8jb$WW|oM z6(u@VAbeWz5#7{;zM`}aT1pcsu_iaVBrSj*;Q~g3;D->RJg6yw#fZq_%n({?06y9y z)yp!_{HF8Gk8G~~%I5P=+;pz_+_@EOsuyK-C(gzIP)u_)%ycr$bv8gd8RR#Q zUr!7^9c+Kn-~6bz@wBsMzqxFuLAF^dUajOWC|FC?{3$u3k4Na`;z7FExK=t!hKgsz z`{ai@VFPT*QErl)s77jL8#}k1gBlYM2RWD#4t7>dS}kWBHV7WHOCNX1U-VZ#>XhDZ z5#DR$p4Kz>s>$0G1zVM*{i?z>30^76o93bViZTWTxP=DUaz~S@cXWAbX+gC;wz@y8 z-mM006IE`5G6=tv7ha1$8xyKulzl0}OM6ju&QAD+2ZV4vyogZa5J1v6)F8a5?qbeu zPx}cIGw_YJK#=w!6)1-7!;9cwmJj~$FaPWQv#$ZdJ!pT-4vyHB=u zpMrpUkma36D+f=Z9`)XfjR&tb?><}Exi_cU1I2J=u)nvVqD>)aDB)Cd8D*?OK_L=U zLeP-{iegf(kbvM}(m1&(+?-^3YB)JDn4A{gu{+(wE6zhr&P>>g%zJ0q)^lwOUE7o-IjM!Mlcorp1B zjPwv5Do%uoElv$a1~>$o-Z1~D`oo$DStpL^QgyB%ZeKy_e1g*X7=Qa$bkiFWSM%Bcm#+Ar*}Tjh zHtC*>^R%(zL5<=;UFFl(`ez-DAdlMW?=@E*)Jcz;6{l?#C#_}s4btrz(OM;MrJOTg zN}H8X#`$=VE*i2D6~j&N!~58z*cil{>m=Fg<@wu-vLYI2$X*d~sFX5MQaC9g&q%4N z@}liJ{y~%YZmZ;Rm*R1c;%Z%di`ixJwpS%7F!D;{Kp@QP*eRbMF?+3JBLu) z;ro=qi|g{|cE-FxWNPj-Duw9s`QdkJ@{JIz2NmFwj`l zSs`wg@G99f2@TJ}r*jEey!>n-2?ZnMqEng3L{?S;EhVBLE`S*0hmZ6`hd7~w?0`db zkS*5FI>*y2$HN%yZHD$RLb~Xq-1P~*rbSV9!W54xR7i7vY&#_t2<+vc`}nvnCZ?${ zOP(9YN`^XHN}@kECt6BKDM7`t(?T&p&R_&+ef1Z&fAPVmfA+!0fBwOzAAIonpZw|V zpZ~f3hd(#`@E0b({Dt|)A6k9-E6*#x^84(QkZYes-~2rJ){P9E>qy<}DBWvVosaOk zpU{kM(hYC&%WY}MtzfQ{Qs(&hK2ll(GC~;TO7t|%veSvy`^fLwU-(}AMTqW|EH^V|Vqi5f zt%HLbkQ9yz3Dc$Ig);hj4d<{$a@;Dt+gkdltNdP@^te%QP{%#2W$o7$Z&y)P6!{C% zJe3qTFT%|TiGy51zl1u~QKg(3T~?};3!7jBs9ZjpTs|INIT~I+8ecnrHfk>)!0NQ~ zp6#eM#fu1`jid8tXQ*i-gwy-Fw|rlO>QfBi*%XbabK}7GK?nrG2w{O5grE|F0GUub z3J^Yc{tb)}TB3OUDAW5O+}7HV{l`KGn5`Z{Gc=p~Pqz1-?5MN7`*?l-F*IkRZbvsx zpKgIKTR)zk-IyDkn(6BsY^dx42us)%EGj@)Oi1V85qu&NAQa>y`M3-|Hl2-11_+B% zqeyW9_(*R|sB4bDEz-{h6#xb8XO-h^j`B7ILAo0tT=a6>^zwX+X<@dUcxP#*ZxuSC ziI~{M$nN9j4G4%m9DI8bx`vo3MJMpHq3vlvU?nwMij4!KENYy0wzpNJrLKqWr`DhS z(&*z~T>sNQx$(0<1-bdN4{rV0pXmJjXZj!hnbBYV-072#+^&4$^V!wV&p(g7aU=EC z&1}7!xp!{i46ozvd`>ggq29S!q<>3fX;S87+ZyOIm=ryj95qw1iPYLf#i)>GeXif@gNe=Ev4{u5fu8#NN`dAX}b@Hrlklc-^ezsT_qX_*g zp*kOh8+-yD17m~#_|p)o%i!`Ggo998`WIUV zAdO&T`@ui{+kgHjgxaeSZ!P6tzJmYGoiP{r!GTvMHjW-|9X#IMhqANxXnXe|_+laQ=T;8q77ykY zk7kxnpqKJ)oQ$g9s(ZFGdudk_SmV18UfPR}t6z%^!}~#m+TMNwLU4sw3pEJgD=hE_ z5dwkd5W+REeQ2d)82WH>`S62({FncD`1EW{xO)#eZB4yo|4xM3)$gsP{I?Nm;DK-1 zJchP|0!15IJ7b_)_Zl9&I00_9j-IW-YlBCRHxHpe6c3+(fFa`c(M$DDcm)2X*g1Hh zRBq2qE=>FaZ#RmksjHBcG>>65JFEgq=#v?y9o$E<{aA#C*GOm0RQ zTnoMZY0&4N1YG?%;Odp2&pr#jdNuayCyCcS&AfRXqk9{#tCM%@I_AbT^!2OQYgh0$ zu41qJigf)d{l*Q6u}QgwO{1((-iL29Oh6LY*QO-(~;^oK~7akh?}+S#d1bvZWJZdKG(~b7;GoZ z46Y(1HWp^~a&Y4%glT#Hd>KJmPFkxf-mVjG*OqKl3Af7y+ZFsZ1${}9KQF;4<%A^# zaZZYxt&YRH=EKoBq`g(PK`}N3B{W!1s4HNK@zJs%8uZU3R%_H+~Exx}F zv~~LeKR6mi@If-?Dj{4OH8{_~>x3Q`JWCfCMye-5&jh|s$KV1;*YMKp%JB#P^soQ< zKY>uoAtVq+s1f)*2-Q~KLI`IE2;7H$AV9cz^k{qk;p*!C#@gZZ=;TyS&rn@?vw&93 zBGw5gilRIYHc3dx}T$JvY<^72=5SwMP0` zWvbx;eqdOL@-hcEKwzqa9tap5=6Rcv!yN=^{xyVz7J4Rd*vH2Xm5@hEX&`+fQYSaR znTf5V+!d4B;3B4 zbXzC=_Kj@a+c-mgs)ZTX&RXi?RORd06dKqO71kLUR^#O+wY6gE>9Tb7dHTi`R!)^x zcC~g6Rkn8ZPR=df-d&+#W2xzrh^z%9QiaN0&Ot6@rq84%PNc*Rr$qK8hPFjIcg4C7 zr1>w9)3zkUeK4YH7A{p5w~$grss2T=Zmc99aaO1t9bJV>YNO|jiSd)A_*pq_K}J}T zQ&bApN*Q;xg1cKK+$d+ON(&dndCF4UN_oL@8F5LLKUad8xn%+T=M()NtH zbsSqcff3HCwm@c=Os_`x+M(8QX_q}J z@jVDN=Y?vpfg6~g_J{9-5H5q~9Ci*Z0)gEl$}@zIo_+J95T0{*wsQEr2w_+6gAfe% z;J^qqe+vYz?mgK&eX)A<2>htS`!6?kPC+r89-A8L?i{L9w26yqSot+i6SbG)p0NJ{S#6f_p0 ztI1haq!bA{f|KS=im=J`Hq3Q5#Cn(lhd4(Qw5@w~okJL*%SvWN+qT z_lcwf3jH3Be_tTpqft+aijFDd4Rr2eddh5S{A6;}a6(vryw5lyOjVG2C@r|(z`fTl zIqr}y*RXq;=qg--92EulmFLA(;Nq+C$pGP~2scrZH(QDY2p0f3DNQM7t^kBJyj3}E zUWA(!qM%99vivm#ak&(#ibKq-RvNNR%v_qBnOWMISvwhD)<(DkBLsxTRt~{m0fCcC z`#_)up=Pb(VlAMXb#G2hd+Q2$_0LBc@0ndi+e7db+0{iyc8i2s6nVr z5f%tCtvdYRpa1QD1B7Q|!q*7*?|t!O5pKPHy5hoU@W&y9-j{ZU5E?paz8(mKCQiY? z^5i)vh8u?uS9k7C%&bmLOb>Uq_t(n$EBH+UN(CcNQHbQB;@Ie97B-cUo5IRN(6W<@ zvJzp0l;j{nj5iPn9Ad&<0Ky!9+Zv&ruvs*qw5P(KrtK;6^xcrr^<^bD_A2kdbg0&!pdu4+s*!;mZK_euEln$TCRa{IS55%JiRI&I=t039kZJX~qd*|M)8+jU zPH4TDUOVU7(f2`kxxBiynjgyzk-3MEDU#=ZITv$8Y zICwO_vNt?5KHS~eSF7k%@EZB#3M!_Gik1=5S?B}?DuI+4OGPFZq{Wg`qlxiB1PW%(hOg#OdqQZPjjS?b*_&M($g~A!vZAB-8|FPG{eONB;Dx_6lc9$A2WKa zyErSj0vFpvP4DJnfx|HwZB!0zfbQfITiH0^u$Gn~Mnx1SdD3EB3PNmgK4xeS6Qt7} ztcwZS$p~e42W_K|vC_j@>gQQNAz2s{nCeqZ^_dn%CAQ`@u8v&+zN3*LtBEOFX_?#U z2yop+<{sqbpOP5>;lpCyA-QM+Puit2Px#^ova(n8jbApmd{tBTOj>%%DB2?s)^f6x znJKF5lw}NbZvP$!cUMk(+9G_^DZSS%zuQ-_+g3VL&KwXByScb_Cc251T}w(=4-3P!6gs2nT{AHkTwP?QDSfcyb3QfmHyVvF!Zzp(3-$2tZHI7rMpYmH;pp7<2mkW# z|MmF!um9%|YNkWq6XDs*L)8KSJRnew@cHu2qusk-s&?*!pR;opI)ejL%bt<>hQ_v$ z?w*1AijGoFJ*N;AB!$ckyKqL4gBoNd?=R2uR`YHCh2sZ;_s68{$vm`sD5+B!0 z&+1_1_6rFElENMV5%8<0qv~mB87_g78A4C=p~rcVBb^C>HaH&(w3{*3)fD4mf^jy+ zIvL{}jR{UB`Oc;VPR1mA6QZ?2frT#3T%T)gSmt2a;^Exq=P~Z*HysqT6dAdenDh`& zd|t$S$rHceOP+|N5BQQpj^Kf;{L7}+@A~@xFg5)T(=-1tJpAj%rUwGy0l9Dki&7%f z7IRZo_{;-AJ`nh*p8KL({%tIh)R3{ zv=&NF8|Gxqmf%4pT#!*`C54kEgh?@KT7sUH*BQrt550)E~w7vIVv)r$lR z)AQ4F%X2IH(2J&5?~W)>M$QldOAtcXAtX?ZaB5)(3Oq;pn%w#4m|i!WnrNH6v=VB~ z%DjDM;U5X%#WQK(V)#7>H4b5Z@Kg=Fe%Lvx99h_T3!(b#?e|4^A?Fdo4u34dt+UzG zGlZZ8T|ImOqTG70dGrFBKHYl+y5E(Z2X$@31gfB+p<`-fY_O@eMZ&4)k;~|LrG+`H z>}X<2FexLFm>x;VPRLJ*EJzB&$N1+*`{JWL2r-^%-WJ(`b_uQ~>0VZ8Zsr;87MX6w z=`MyK;3mt%6kOAs4N{%%Bs=IO+3P0Q-b!}R1;vmQ{@P)FJZR+FnJDl}+{GgRfpUC` zC?}qm70yfxVx|N#ll;jMZuvp>1V7-=3hQcyaWO>$gf1p{S2LoAImO$G>SImzupqmd zQJhT~4n|xXBZ;kXje}*Yt9`$R>r`;Sa$K}3Hf9@v*w4#*#ALrLDSav}dsbHUtg7yt z&fdSBnEL(P{2!K<{(5@yH~sxD8|qI5f;}2}r!aq;g4<^ik44mb70icqya&y~$33!# zePyS;gfje+m@v)3^l^xF#pKS~dZ-L8tb@*TOtn9}ayqO!(JF>- z974~_FI|H0B0o6VW506_A%5z!Mm1#+TAa{&b=gIPmpHtjPDAUQ!;1(tH|HF7z80uK zsO?ZKaJ6ep)ibdI5RNYH{y#$adT94rAQ&RP1L5o0)Hev%4_<(59lumUy9rR2;;0Yc!Ti8INP_L{c$-_XR@Z?; zFfz;wu%bjc@lt$C(GlhN*hXq*7a!lr#aENFcv+E@IDb}pCj{A_vNHbPf(iM>gcomrEUb+5PUSfI~* zROoDU*d{V_2bXuupdB%bAC;E9sA+iB-2UC**zf0;{-9d;SGhPRR8hS-Qn0cJ*=KT|@QEXf}gqKElelTrxbgb*<;LN1mP=0$|jqMUXH zu8Kh!?;BoJZO<#WCKon~SLiMXW_u*wgXeCeyA)LN|P&?Wh->P_v+Zu;& z1TJjBixcqjpymxNe-wn8=JV2aRQvu15yB1uLLjhbVh!*Em9S%E395wu{=fe7$;)p* zBMO~mc>3}{-RbS!{{rqdFZ7&05NOT@IyM`cs?p}Ou6}0sY)VFJ$^LcJYZB_O2M$+v zA3|SztlqSJ^kQ`nS|Qv%e71M;#q!qU)xBq(14|)M2umC9TrA`G=;8j>!C-qwtyoaU zFIG@6LTri7x-M!YjO*$4c} z%FwY@g@}gYymC?&Co`N9?^_V-PmT>>B!$uw0&#)P_yFfDXLGENEylwl$Jsa=AapXq zx)~EZ%!poQBp(ZEpcNy;wkXt&8EP*Kb5ex6)rNRB1bTD?y7q>6Pb7vQuLqDylc+}DI>*>j_=4XGiy84}J`LC7AuO}y8jSM{LYJ1q)@UXG+ZjJO|bLsQ$ z@@HL&r@a*qI%Fr!;-ePvafkGRmdh!4VF)qN4@Ug=@^tc#3#?KiOWY3nOXXKbsA#zNF8Rz4>DLHaHwzIr^YI^?G=4c&-v^7sHDh)dJs~v<>Z6Qx659 zca^<1qg~1%BwFd3TGd4RbM0tS-S+`?kV(Erk^12JrQ0;@k}6$5dM}GwL@rFxc6l1@WuM>BlyvZ zf${a!Y>JMdqlvj&RbAJsm){;A-tBMeY>-G=rTnTQ0zk;ki4|ZHi;=P9^hjJ%2sR;z zm>3EwAu`Yv6X}}aZyn=inBZxi=x!G4WEkgY7-gjgBLwQ=?et=8bbv!R2>_wBPOK#; zhBp%JZfCgYWBtuZ;kL9WXI7FAKO;nhiWOrL*odgYcwb_a2R_o980kZb_9=+;$_sG9 z`Z{F0S?0J~?R!Pd+WKnh%(#M^xUrmg>m>T=Fa`E}p#M6<%XCs4OPmaDA z>V4kZ{<6F2S!d0Y_KJrcvU_dP)0UFEt&)>A$#J{nu&rdLwPd4Bvffs@-BS$^j+Ye! zgbnm;2|B7M*{3+gw-OWCR2bh`l+wq_80DcyxfpPr5avz@P@`M~=t$>en0|Kp2p==V z!8H>ztEh#8&CPR*E70y1CA2kkdi8j0d3 z`Cm_8{d({IOKpVelhpp02;rkEv>m<+A?Qn?SHT>Ep&+y>2)ahxV5r$hqr4Ixa7-M}KzD8Nz1UJ!Ex4=!b z`3+DHfkTA5AvVwgA7n+1b}dTqVI=$068#BL9;hG(WPn|6uq!UqEic3s8{`ZsA==w6 z*V{V7**MG55UIXq*y^TP-_Ekt%W>2P6tkTTb39D3z7_;Ot3rRP;y^n=u#+UjsUp;) zCeo)ZIifQyx;rDbGd-?3C8|3+X^@mRCuA+iL^CqcT66VcjeNdBy4F~A(%o{@-gw+n z`=GP#Nq608n_{;?xZB7-Xc9s-uw8P}AwB4j?6yfZ+ob^EW>+O>M@M9I&`Yqa5@&2R$Xoo#17UaWf_aSu@gHP$&D?Ii1Dm8UmuF zL_9q-tXzW@A-PbqkpKXI07*naRHl_X^Xeyb)$pJt3TR9SZJjv=LqYX^Q3#=i-@6cM z3)DD#UxaV>oNxYkfmZK@Wx#hu8{x&L@-KbPP=ipjFsMPOaR?6!2WQtm_|O0Sf1Z8u zEsPKh3lE?dl08!|_nim!KDiVjTn4`f;hQpe0@Y0LU$Uk?etYNs7tpY9?KqUkSW8F-m9Svga?}V7&0@7e?kT_(XBH4xx1&G`<>#|#V(NfAzzI8SPvH#yD+6Y8AdZJp|2kqtV}AZK)-Q?9=Q zF2p(4$0jepA>GL+)5#d=W}4+OqhY#}fjV~(E{13~;{tDU zs=sxyzoQ`Bvn)PH5f>;9_hR}w&^)Z!{!WsZ5J_UVFfl@q5>=j;QC*PR%*gK-70;H7 z=gNeO3jRh-$$qnZqei$?&RnhI9=1vMnuR-!d~iMJDBbH)Y_`c(noE>T(%Bl`kd)dh zEa(*xn~Jhau`!@2Ez6H-q@}bKr?fFrdx{XlEYvs)InB+PWFw|{h!J)gXiD4ZDNWSO zN+P0;jvs1iS)5;5Sl=60Z3AO-z|6unv>Xp=AP}fLhN3!}QXav}5|;~HJg@Fjgc^a` z2;qf65cPEES*6kz_&o^UCvTO(*^P4u-#qU0_Bx^VF&0__LTDVQT?~6CSHXYz2mkxO z{_jPE>S>kpz}_d9BD}N`p4Gjx8VK()h30a$)rXp@_8tO+TX$cs>^@%GeX_a#Y-#mD zQ|m-Ra^5Xn>yNMKeR|a(H68ox#c#g*?)P7Q@$K@$N=sF>oK++w=5o-<^t4b)au7bo z8z9V$^+SYvWCpwDguCZOd8Y^3fJz7uM!Oh=JLrem=tS7+2ATkAHzLe$f-6X*1<-gK zT*FMEBVj^JJ`Xm&7Ho7C1b_q$YNn?N#@~_{=1>sjN{V(XjP=Y5cTM-PPH;6zbv4WJ zv(F81METk01v{a#g3=t9bcam)G#9QjdnBRyt zyPjZuE5qpy#>cWC(6%thsW{4umk=OI4(7-DQ3IXGzV?hD7k0QWJ;;L*=3N{Uz=-h| zX2;f3Fzp=Dh=e&NVNI44&&$}VO5Uujcv?}k+$h*@mu@u)*Xr0Hn@z&aR>^t`G&Q=^ zD4D6|f=bvgF6XmZo_qm%(cy`mJUrjp z#_#$~^N)T25JB(0b^D`FbV^FAzy9{`e)F5Z|IK%Qd$@B1I?@`kNX{UM^AU{nFhY!H zew;5Z!9P3FD>cYDE7Tnw<%Nmx%nozR2(pcFHwtsq545@&WN{%f%GwW{z|=MK~BGTN}ok>qQ&i009(}?exJ{ z<^|f}L!AnvJ*aWMf{X|WGFFfhAx@2yX2eL-q9v(OmAOfExO72km>iwl%EWi_$b%xr zu&8K4Tr?voo-1X|loriautCM!YAe}j5v@1yS86#R%k})#rjm6~3|r-Mwfs>zqfbNv z?PxtM6Lh3ygoN^fgepox4JE#wn$ST{hY`*&5p!JRQVDi|k=9(0*hE9{aufOa=;79; zrNz}*)z+vInz01ED71llesg;Da8?DKpA8&Np0%UE;Q{DHFY1CbzPJr?@pDbiA=I9a z(X7q`E$@EUr}omtn|IB>E0+oc(fs^#2+y^e9V5_F^WfSxtORKXP3Z_Evr}^}@Yo9c zh%oqf;XJ~_d$0bu2*Hnn5uWQp--Qsi2P&cZ{e(|951t>~{c28iB$agqg=Aj8VR`MU z;V*xALr2GSaAfw^-~IjH{LMf9?eG5S;k~C*BNO#989*p4$l)Mk@?*WAN*L{p2zP@K zq9Q!9p&N&cV7mlwvnW@i5L=y4>)VlbdI9P~(856_G{1RP_tXh8x*BBqd4Tcfeuj`h z|2v;S4vnuxTHJ`YzYXdk($@^>Yk~B$%=EE>rg@xA5*!Q@oeWalOwv8f(n0^}Wtrw` zl;LiY>1vtqT_msa5dP}W^zt9r=U+rAK+7l_|$1BYpG12Eaxv*i`JW@ zTkVRCHt|}EaJ7-Is^dUufDo>=mV!z+RmleJD74dnnOjHAs3E5ThgGDcio&>xg6KL* zd`EF6KsdrkpJipu^KwB`+D}if!bVjUX0Wjt6(Zj3`1sP&7U)99m!Z9#pc2k2p-pO2 zYtRS~1i*tfsVzfiX+z5oOYbazFv9Pl_;wlm9)xf1VK(RZouhd1!;AS*2sHv>gpj?_ zWf09#`Ea$=JhKm~?ioUOc69&V7ylCo;bGzXAlz2N+rIY-I0V;~y@#tikJopfwD-=Y zWzb#RqOV>v17+|p|Kf8C3#Ylojc+%tlmG6Egb102!hyT#dms z%ga2;Q9skeG}*}zm;?yXzP4x|8xg!;!nxTIJsk1w_DDyI6g%TI7Yn4X z6F$n7o)%b~879Oe3UZU!=`k`)W+NHbOu_XqNu9L3E-I#lgzV=O4hpEFQr2im(RfL* zszS8YpxADd@3&VT_SEinRjs#+mm7HtHO$3oCR_$r8cUYyg%joMK?wzvL4Xic!bV0G zD2Bjc1u;R6k1Wj#ttQ2^G19wf=>wGHsp7O5E_#@e*-OiiqasT2X?!B8v$0OOxHJpu z-tzAB`U$9hGfO-3%B{K8y@?et-rGJSw+KA~4{a9x|CzfB;JDH(PxRI0GRR_vX2%f|<9m z75~V%N@c1$qU+b^d~msN2wS%bkOIVw_fK9!J={6@8ZS*)uwff> z$zsjfslLoXYw;gbwqhl;~p1j|>|K{+~ccYVA zX&JTFW+7xLO;IR=*918^giK5v<`;KAeD~Ks|K*>dBYpe!ht;L+uKL!xoUFR!sFD!Z z1ZU%L8_i%_wJ;~G06PsIa}{qx6%QRn9|I*XJq1raSr1*Bn>NKsUD{TWh!CPPd5P9? z=x9(2nz%V#+=Lw6~jEIve$y-m(-$)6{ppU++rw+|c zlkBP??V?V0R7G)Cr+R41dTP?#)krSNQXVSO5hiN!*4n{(GC{iZ7*mxv3)MJdr4(}& zbpE#qXrx4HNhg{pCYh+@yBJml+Si4+R0TLy1$i_^26iMy^`$4>%t#)~N*T#Wn#xa~ zEy}f!;9MK-P!sQ;7VJ}1STr-UG{@KhbNcX>HFRzTKKY5yxUhs>*STQ_$q~GYCLyLc)H}u9G|FBl)LtXhQPba66(KZH^D$KOHB|Q2R|JIaI#e8?rK~tQ zmRF8o#kc^0W;8KVswlh~k_C(?g2q%Kc-1HI=@NM$2GaZ%bWwXHNp~%@@eOwf^$^~1 z)u1@2NZTn%!K;fp6>6iasS?=Rn?B!A!;5C>OGnSP!nwdD2pL(|_Z8A50 zIzMryC~37i3k5x6Fn6n^aJ#)^2L^;)WozvvyM5KG?WK#&g>wxBlQr2BHM!$e*~2BN z0|ki#h4J99CpWS)BeX3ouqny6DaEHL$+IQNy)(tLFV+1?{4>u83{NTx?;tV3Od zV|lc1c2w{{R}VVsZiDGTF4&=I|3L_`upmT}F-!661B7SURgW& z_WS?z-S7Y9+n@gPn|FWS*?v6IKi*VWUJ(~t5*m>0V+F&)2nUT22X#LyMNboXZxbbG zM*$&JLJwUT5a^;Q?WjhwQIG_LFz^F`mNH^+feC~%2p3?F8KJfWmyRU2o)j-wwUL)_ zR-<_7$^ybb6BR#0MY#6RMr)4h z?I=U}EJyt`YmGt|!vaU$3QyBqJFPrNy>c(}N-v8lZ_8RgyZS(<8b8NESDPX?+qyvS zws7B(jJVr5NpmF`%T+nEB`J4{lb5P8mMT*h%99r=Qr7CSHkxu*n{rkfvsaq(7V2^~ z+Dew13g;X1XKHiDN>fM5(r=Zf-7HOmN_evj1u9`zW>`l?FjT_2IM?Px4=973Nv>Uq zj(v#^14#~piMG87j%~5%&cjT9r-s6u*{QpWt9$e7N9+hOhwS`5kML*bOHXG+FGh&v zXAzFiZZkW)s9$AAiz8$fIDBXQw;{y0o1P+k{`xz#MdksTnAtmi_1h4Bys_}J2tOSP zo@Vm_BaCMf!k=0@dbP6q;^5Ig+@3j#iYaw=j@8k0QIsP%ekq=?wZE(`1rz*fGl5I3}LXXs+XCpt0C3XSPmRQJL;(` z@1{)yfzBEv2URJo5<+EzN(cy{)dYdYWMM-R@DtXj2;k9`=F=i_Lv%>IU==>^s3PsI zCF7&72o9knLU#(hozy5+3KC{A;t&w-i0UEPSCj6eK?~Mb3^h;)(56NjDaKo5{ zwjBxX)lu##e$G=vgG&pmGmG1E>r9U^Tm4uDe;dNjFI})#Lgse#XDcDtV-7PaJeV%* znw=jrLcDRs5#s8YH6oA~AjB&nYDfRa|NOUSuf9dA^Cw>eLKvh0r+*JZ7KcnTG5F?e zEH4;2L&y+_9!q%mYUkve)!kRio3HbVdabP@EzLspG~MYGT@fJ)54SLe(9XescK+by zyFb17hd(|4?oUtO{dDr|>&=b5o83KC>B-qazUe;JG0ys7w(3DPDqbcsuKHwmLm5va zaH8O;Bj=_`Wena(Fs`##lCo8hgfeIa146nehR~2Kq$kC1AT0>?^vL`;LTxD?2wWIR z3t7kz?3AQ|oST*mz=i?~ao3e`R7VJn!82XVLQWj|Qut5tP^Un`ja8$~)nPyg9ch%Y zQiPFwj+k@2xuNuQ}|kT5Br2Tb?#umU6c;{dP(6 z6kJzjjh3X0mt~AqWMT*z1H#z0^x)Q%K&XW6=|L?C9_>jUPzKxM8~|ZgynS1&eN&`$ zL$qTx=$Ay5aN@cI6rnppU*}7aGBVc5Iat0j6o5rLAjB^|B1FuN(UX{dGeTS-%g7MjjC&vd`bT~{ zj9?D2;UaoI_AxpJfB(UU_1(A4?RTABlFdv*^)Oun2T%3X&*Lb+OMXl|< z$M38^e*NR4_dh;-_v4fIKRtW*{n3NRGq-Oy78hiO1tj~}L^>IS+G{~2^fj0FG^Bdy z(>)DjKp@mE7`B1H(@7K+X>bSvQBz720fbP&jHx2}BmqDOT__9-v5_GR1)&jzz=eU7 z09dt>leASNIjT}!HR&EYau8Q-83#48g(A^Zj$k5-iXr@ED7Mg%x~q{R%r)Zebi<4l zLXG6(tuzv>HF7(Ja451-AoM}O!Dhb@ac~8ZV7j3 z4R`NN3hYk}ohVGaT@XK8k-pwsv|O9DP=#)OU#!m9ZY=_P``uML9p#&CWlQz>Gv(=b z%2Oeug$cLIQYI=gZWSfnEJ+d2(zPJOtu!&>-uUR;-Gzmf{iU5pQ_Bb;61dE`U+EX`fcYiCCT4J)I6|xvVz)2- zl1g|kLS|IIsuHsFq7bYXJr^OmS?L0VpNt9r=l}cnr!PMsAq<2&M=y|~e;-2Td>$ck z$T&;pv|!$UgAR1se|u|uCpf&s&@kA-BwRzuO_63pB&Z7VlQ^#nT<75B+}+ra7ax-x;1um@9_FYQYOfJsuIQyt!)9V!)TvIY z01v7Y(OFFj25cBYY-DIc5dwtfvIG;FCLECEQlq(O z$T+K0L7TgrwZ`%>; z(-ZI08|QVWD7r7nb0RN%wj^P$JZZ8ZdN?a&Fg^;W z${mmcc-x(&`@Pji19gXebqBq*J3X~)ZRLv%CDWB@cdF9IOOvN6(UbuU9xKBrA z$FM4_-FVnCP z=Qnl_fy5DFgjfVJBmB&mka?1ZMIiG5+boUf=PMyz3{fS-5MluSJqVer-$e*_PH)%+ za?f{VA#AOJ zaV?6)lqzN|PqbGiJE%~gJGE4jG?Nzxer5_pb9qz`F@%f;m=a*D9BHKyW2>3vX;v6u zo91R%9qw2jXjkNE-jf*66X(|*<1tel(;e?Nk`;WnG;X>yb|g0x_{~~_yX8(96SfNaE*;2clW;;zO@n>nrmVl zq^}*Irr@S5V<$^CCP}Ia^GkDHl_?kg8yyix1a7j{_gR+KRtN) z&F0?W_~`Aj!t$h0-{c^#IB(lXd%ZvtC2t*?y9S!2v{MuZd$tN<5N8dtle)B>vZR9= z$zFwMDJN=56GWAejut0OC?dvWVH2tV#1zyqmh91LAz29FBner{N|@8dVd!W?5w=n$ zL0}wcr7C5sL55c&c`*}tF-sL96h`DwPQ*c30{&}2C4`PN&DE$X+$Gb^xH8DG%-^mm z(7qwuwKdAUC*Ery*{46zt2@?pG%IMTAmUDO%xqcwLT&1JLDWKB#$tW;Ty-Wm-07>i zKhm-{PTqzkfbL>~4C<4&%fMD1$qU+j*B(o@Zn=T37_>>Ue9Zc`L~{$x2(u zQ%wn?3POBnk&xpW_oc6{iAazWGfO&e%naXI9lN)8cjNK)qYrye-)ndk9~Q>o5Uo-mfiyvE z=b8}-hJt+1qk=s)=e7vDp;YhDoZyLq z@Trm*Ksa3*zuKI;(p0e2T(Htsvff#_)>(OQqv2$v_4rov{h=lhxZP8^+EzSUlYO^3 z=U#2$LVf9ML+)H--ee8qnb=hsw`;O)Ri+P@rw*1S50obL7bW%N$1@I14QfpdXiM|& z%?xZybni(?+E1L@|yK}1t01pO)crS_}WSZgJVdr-m zwi?I+^8ZYPm_QbUAP`5$5_l5=zdaWr3&@WMVNCdI5i*lIoB#X-;qJq?_!pU1=h3OY zjK&qz9X)x!y7O#f|K;Av`}Mt7H-q4O(S}TZyJrD>55E@ZH2*W~bdd5OV44a_=W9Bkwa}2hAXeS*JQ{9lS@LpX$ zw4?LQ1$P_srt7kASEY|sq>WZ*j@M>`!@lB#{-T85f`rbjh?dmgmgE35AWZe^PV?zZ zbni~|Xo_;Ek9IAK@<|KwEKG{Mbz^90acyZ8?G&9|LpwsT$(Yk#^b8?>I0dV{3lK7| zOkgGPY*!lM#Fqa6glyO@$~gj=5#j>b5#jkolccghYL}p)lImEhlCrD{6^0 zmx@ADYK+E*LQwsHA6#IIfqKxR(nKtjq#$OB1Pr0I2H9MN2nb=6Xsxir>->RZ1NrDeWzY>&OfTgl(xo?dd@sss7M{ z-pC4o*NPzP$|#TAP|wt`z^=NwxqHjYYx{Q@edEj;I=%@gVy6}29VRY{z303rpFxNf z!~Z~pzorasj^16zGBUG<<<^~52n+)8U=1hMeEt%Ic<|dPP7(gw-~ayP#d{1Pnp3&| z@;?M29+m}q93cqYIeN9R_Y6G>`|$PV-kZ(6uiLt2?H%F`4E>dr?G+VmsU$NgaeXRL zpCGIxA|TDfDRiBK=jv6CD_1$by39eNDdp!^H+J4?>YXWRnJnp;>6+eNIC{1H==J=@ z;pD{K#_IZ{kdP1?3tuC3#zq#Rv!iy&2H-)U4C0j#0vCYLf-Y=H z6~SVHc95c3Qv)dhD>+FhgD@b3G6gkhnBCe>Ak;;BdX zHIj##?5RWZH;{?6QUi)fPP&PXI$7?wr>OywFsz z*j&8WUNYO155;h@E(^NQn-yteHCZqq>@AFeO4wfz2bHirEx0o?q$e}DD?Jd(;9yQ* zOT24UxN|{>TS}l;O=14siK(Tv{l$%=JLu-x!|BzN$t4EA#U1=;CKge@A^!&=Wah_$ zkXhho5MH>ShG}E>M}%AV(WvZKA!ObS!_E&2_J9|5hg%0P_Kx3xz?H2>2M@m4KmIUv z_c%Jf#=sy*SI>t|H>FUFq=^OuVNEG94N(C(VF3xwYrN2lLM6O%<=Uk$uL=r?1^R_o z)%G-YPUY8*Cln7Bwol($y1(`49aP}uwcX(xBgN_25zbED2AUo^3hp{oXDul=Z5ot8 z5Qt&3k%#f2qy-&qkufHtSs9EXW)en;Fmy!iC{@IoE(QUvPyoT9E|JehiDV%s0c8+| zf*??jA_NFg^;42CRwAG%h#Sg@feG?p)g13y6XRMH>zx+hoE;Z2dgI2*()zvC{d;SS&XaLy z1t3JvoLJgru7oIzo^xv9LWlUx5KQ+&p!{z}h_AM@^TQFc)WDn5D-d>m?7@t*9BXg)(Rmq@n39BV#5d zr9%)?6Bkw?2rCNlkwy5$xUchrK**&_m%sY*GRGBeT@C%j2&X{DOzq?_Yon(J$p7u%T6Wppw zL-hlPmKxF)>Ld&y3=zShrLvfvs)Vzaw4;WUi>9=X0o~6?HqcZdz(g+0LLt^(J<(As z%~`j|&!Q^Kp*qx|G2F2`$!9n_d^j(9vMl{hRqk$I<3?xQdQa2QX#aX|%Su<{N_WFb zXYJj(0zkMsShv|*$w)a=!kvLCK)BpqJW-tijVKaWl{#LVeX}YZ${=K@B%v!iyeZiq z1U4u6V+h+)yldm#>yy2U!kvVN~dUhc*Z4VHpk_(N9S}Tl?~PpEZ^CFdvNr6 zdFfzeXr?^3EHN;|9~DDY4^6Uz5*eCLa0s=|gaTct2=ta{F_4B98z69jZQ#Z#o+(8D ztxOPkjfs3{Rz~V{OsG#0GJw{UE(&E30s@U;Y^fk=B1b?`1b##lMS_XEn7IOKN9|Rm z9Mnio8e}(Z3XBQ8^=Z(N1{uo-8p;G1&?7BX5*)Nso%9NP%`3wk%0ui*18gfo9eZ*k zN2)Szlx2)m=TA3R%y-o<_13QqG;9tvuHUF#>?xn^C|&BS-5TjQoaoyaYF!;@nr$mv z>8aZ2ui6=^S?tK4s!qMrkUiayJyx3m0*5OzhRQN-6s7iN#CN4cHb?s2NDuEz^lOT6 zs1LVoiFYcCbIOi%O$hTT%*(uWb7*mCWqxJ--ulkn)&1$!y{VPm$(4P`#PaD;OU!*H zhKIe?!~^eRs}7hz<~9|32m9^?>l2>@m(2Vo#_!ymyYh~5HUYyDj2~VAafB?>n=Ga8 zoG?ax=)m8JHOP(Ir%wwV9A6n2TgGxe!kuI0T`wOIGI9aJ^NJz*fz=UC)nU#}@!l=T0fQw;V|BT=s&dBbipFaS zN2;sdT62i1HoCjH(TGt`Bi+i*;{HbZ?Av%8PJF4tCB@i|_C2 zSz20ITi>2r-I!V4!VsSF!@_u2Jg$hv+qvZqe;@OPe;2Ee*qhjAeik81uq`sb0U=)T z*u#zxauLD{9sUx8|N3tT;pYBRumoBdd&3!Np;=Sla2$=;!(BlLjzS@5B zdhO)d#tFLKIC_D0X&=4bJ$Si$@OI$FQdn4#u~CqLexSCNkGz~MjbcEQR1y)9=Hn5E z2=I#VaPf0<@<189^3_#FU`zN-N2)?@uo6OepOt>%KE|@4@W$ zn>#Cy28QOUOB*v{68vo}?9~+A)n#mDVH_w5eI}#a3F}LtE!rSTR~p-~jVMB^i7m<- zfkN|-Qp(4>r znFI)7Eb69B^D&e|xBnW@(W0du6%dA+DFzzT!_5`q?9}7!HIkjR3jNKhBJ7*uJo>Uj zo05D7i{hr63T{dcwOqQ#ENwSlIsn_W92JX;sVb>xW#w`=MGv)XRNBT3Pi!qs`%vT zE@$`7pZj6P-MTMPEBlBsRde`i@8sRq!Hbhe-_9>S$jWJTcS|)i2++~+R#9-4rJ0k6 z8sg$~AwdZqZebp7K|UTqZcZ+&3|_u;1w#n@uwr=q$|X)tPP&}3Z&-XmZEsHPU_wbx zYV}Y_&-~EBAe0j^r=gl=KoZiGQK2V5mbm%LL2Y&si83aNWdm2}Io9CJ; z7MjcF>I&~x=gc+c-D}F9X~<=)6QZwoVBuKDMJQ z2-!KE-R^LDUY=hFVe_2XxN5LL^=*V{*5?;pQkUVYNsI_2w=ZDSp6 zq!*~E>Om!0P)T|cVv53oQhdC^T%3Fm9xh%Wc>NkDhLGX#6d{ZZuU`5Ru7yPirj{;= zS(OF#!zmSmk$K%Qg*|mQR&FhzQ};$k=X#ravSZ?d>>RxGv}_cmVN7UBLwnGm3~Cbv z0TKw*l;DM68a2S1I5z|kLg~{d^6LRh0lTqHUGa+_iK4j4H!z>S7#wvx9~TBf7IgfZ||LEGUB4uI%uZJfG$-zeiI z24;k~B4!)xY;aV#LmV)3A!U)q@*_)_nT^bDuq{hmh!F8(gncxKg=JC{zsB!=V7GS> z!W+{Y%m}eZh2jD~L5PnD*%2aR*o_WnXU3d|@cdvO27xddL~9jCuMZ!6w}1R$cJUz$ z1w+G1Y;9unwf(fzJT;YFDMVwEgocP9x`UpFTi`ke4}|MF_w{QW*RF8j2tgf=kWml6 z0;_V0THe74xmCT{HMhdEI((9<(yDHBj<3(GJ(z|6;m7Vamer+%#`u_<*{Lg9C=yL* zf@mWsNk~UZNJm;&OHx2nf?tEcryUt^AYcrYI|46O72%crGjO3!6wqWi)T4=m zKz%wvUsl3MK?-7w?rD-VQy^L^NY`f} zc_Wp1w<>b)HkD7;m)xl<8Y@kosm;Axn+p{0*5rf3sp^d3lK76Ckor`g%0$>2Q>B2LpB| z5kh!2r8&ulhg}pF=8`zYG&gf@7s@UT$9As3-+++40J4`s3?cT;MF^SC6lJf3EMr0p zAtvw?;j6dbZ6Bhe(3mG?a6-%iF@zYrjiYCfpGCNS_!RxY$8YwYe!YJ991!l@e>X9+ zmzCG)>6>L~6=`M~uA$M|y>=#D|CguqWrN)U9Zi}QjyEJ|YB5ETM1Ru93UuB4!jga8PH-=Z2Q z&aWzlUNs~H)rmrCl7bpi!rEjph=#O?CRtdEBBD(dh4Kh(sy<^S!&pYlOhz0iI;xUg zG$}yQL!0iXMsm@Vc2JhER~GZur-oW8M%$_;IB6t0C}w(SmIj%%B)axx1>7u-n5apf ztWD|74XcT9F9~p{jr8tLkM7M*Y)_AAN($}GjqAyY9WF_qsLGqH$(^Xqnr+Cv*OWI? zT`*OVJzko6lX0|RYr21RqFZU4eOwfhW#2O~?6)7$xOFYJLmOdtzFW^CuSmsqf`Gb3cFqMs>x=f3$H z5MsK{%|!^$b;!Ip@rlE!b+kEVVs&tQ1)GdHU70u&_z6NZT042M^8l4TKnO!PtTkmr z2NoP7a5sZ8J<}inAYqTU6x!v&^EF%xqN(g>3Do{y|TJ1+ArM8%F01c#Y{mG z5b6LIG z@VjY|d=2UT#`Fj)rFc8}!T`e@PwlEutBzDpsE1?aF%z|keR<*4(QXAkHWeXmt%;${ ziJ?_dz9k{9?P;N{DZ%Y&;WtZChl&zM%93xECXAFP0m8B3w85O1?lg3>Qe(1bMZ9Bi zjBRO@eQ~&beu#6rk8_H@UtWCL%FOETzx$_GufE?odbPOw{OdH6N5lqO)qt27g?G) znW!g4P?r!>7T}ZM;S#;h!F%m07t}XY1Fu}Wd>Ku~Fy~YtqZ0n^3p6Z*Hw6U6G<7Z9 z{1c+mY9g|lf>LUHW6SdEZw*cFO)j2H-8*RQzMYty@9E^}YHA8isjf6ZizuolE(i|6 zm4XPjq7aX=2%nNLx4bYQGAfTge|NeqfA)LWp4=2Xdoj9Z7%_HiS+D7K1dsUYI40j(+7Q=1rUN)I%WPOwu> zuv5zR(5a5Fsf%)GPW0%`3}{XFEsLqhcn6&vP;hWDfQ# z<{8n?e3&KJe}UWZpLKTEJj?y^=bU6Qdpi8XT20&-ela|So)-8SfmrYzGdGy-6~y^{ zw1>aRD1*?2;%gFpjDztpian!?5L5{P;m$F#^y?9xIz+AP;gjv-=lhRd??4NB@Ob;^ z*}=n?`ww0qnFr5y@4wyJe>*(7m6Tj>W*)4j?hFWx^!&Bd+?5sVX(VGQf+h$Q6_VxS z5$EO-;<(0h^$I735Oeq$gkOD$euVMiH4bhPRmIT4!z(m1B&9kcqtQRUBC)Wmwr6%= zVtZ`vuyb%Ox3D(IKg`R{&R$p3K$fgSC2C5EYf6eJit#Im@X8A#ges!EAW&YE8=@e_ zqbSacIRr>5XZbcBU`jFo)4t5$KKSy_m6b+k)!vQJ~OH?)zdo`%Wp z`q}>GrQ!CK(ax2TPPMUa)e%lr;f}RYE{$;>%?Vzhu{Fu3Co{A&BeWwutT8dDD%!Iw z%C#iIp&-;EFT^(8$1=muF+145CNpf0sF)q%pJ(|NOEJXXWI_1zxftQOkV;4}52 z%m^`opCCll>$L9bk62)z0kgpBE#udw-2 zXh|=9d6ic{g084<;~Ey4Se{za8J*P_mR6ry+E>|mr+a*>{nlDd>u_>feu!U?i-m=u znu?B`tQuKTnE+q}6~zPqp_~vmXat0EB3yJq4tRy&6rn-I0`FkKtzw@6%n-y+TmZoq znrC8RXeraauP zBEr5Z%CRQKxhc`FF~Pqs&c7%cJXK5X9)|ua~)pvr!a)< z2m6kGdR8y~Cd*cse-A?3Au~Vh(Y)CG^7!+5doMF&|Hg{6lrZIi(v1vQ)f!oFC+rALNi7V4LA< z0ZDPU%(>HUhTp@6_#6XRfcIR`4mN=eaKOVe9Z z%~g(SjtLYNBnt}=1$f1HIMHEh7(y5aUOxTjD$1EUW`T^3^fHDJEmvH+{JSr&UcJsw zp{uxfMWpA|Hn40I|9b{%|t)-$SPa-N1 z2nrHn@?!in{_9je4&VonA(UB&g6L^&1d31| zVYsMH5>^u9)g+1NQ^kyEVwUof){0UNYE)ZgDSKs-izX$|Of}R}JJeb)&dn;x%OTp? zGTz-f!OJGz!y?K>Khi-f(q1FlUOmmjAj8WjKft;q)S)26F2lz>#lskq6X2BX@0jjm z4axGiN%OWybTdtY8t86Y6yj4E6Ok1VSe=nK*xEloyMA!+?D*OHM{mA=@!elv|M0it zSKrTXKSZ}gEbKFyPeupGCgfZspZV3fVRp+N_RqtA8T?>hc5>`vLR=vG^24vnCpRdu zA!H9WE5nWu8xvyr?Fi3iN6!%MKi|Fo8nUtX0{YM6hu^JkJnb8tOV4QWa8ENd2vApZ zqtk3?(w3@)k@WHoujR(K}Xh81)dch5H8TxlMdDXHy= zO3HV14KUC%R-)4603m^(C@u_yz#brEltFHa5GPd_Aw=~MI?uDV6I@{kF>D_L4qppACy@LT;Ktd8f0-*qM#;ORF5WZra-b$p+HHrRv@8Q6={1_iifU}kBNr2 ziB5oxX^^c&sGWtkxt^Pmy1R**kC{e*rFxjPMx>o~jIAaV%w#u{EMIG=nF+2Y(GL32 zjz(!d_UXQMpf1tFB+b_{!Ob+r$vDN!wk#r`Iwrg#Dk8(rzc?StA`VFr|W(gMU;EHklS-RXyWk#C%-D}^O>i!niu@> zj{`y`gy;66pGSyK#xRHdO{{p{`t+t=W`r!0rFigm6cdQej`p0*j`oc#qS;Xl;qD3A z_W6qt9zQu3A!F?KYWM!z?Sr?Q+plKtoisGvj*Kp`v5wZ!_Eu4Lm6fq06V2r5Hd^X# z>MD*h6hlcdRS`iMApsIUk0>9v5HGg?CkI*;#O6hr5i(aowp@Wm6uU;9>J_w-!NbL` zprGa8;-8vRlU>mrnce6XQ{oU&NBiUdkQh@G9GLFqlM@_Xn^V%=&@nYKy}Ek1ckt}Rn;$>?_|G4H|DT?H z{nN(5^E-=slgp>u$}ohJbGwj<+0&Ih=8|>p(C*wYf03OZGeUNUI6|PsjPSyohY-Vt z1H*GJKkU0K2yuJNw>U8eUnFE+qUau7WL+ox)d(35nbrvrLJ+uh@B*6!9Ub2)Eb8+Q z$g{DH*3)p-!ef)Hy* z@r6N_T)T7?M+n96GGog0>g8*Ed}10}W+5?|N%;*hEcA#ebqp(Xj4Te!=`8JC=)Aqt zb$g?_Yce&jHYg~{*4$c^N>?NjWyORM4+cUINE6_pFgaxOrJSb@8FdUgOeJxshfo!T zut0~Y0u}OXOo$@{+n7VRW|%}_IOJ265>l2FP!#7jkdd%frI^S`SSd+ctH`+OsoAK? zS}M_OHI>X&YfvsR@!Ysh-&DEb;GdFd#6>na79Xa<;S z`I~45Sm=e>nuXXHMZ1_qyO>0~8YFm`#=4oM`#9DmM-3F_G$h6q1_cxZg=7bXl_zD^ zhaH-G%@_y6+#r++?q{$XL`WO8M9V)+_;D-Dr@06rGcf$O%ntG2Vt?Tx0K7m$zl7(W_yl1`*fY8a2(cP?S_%LD z_rHJm;SbvV05>fS2y?n-iwG_o00%1By5UrE+lj%G+CC=*2JA_CGNkcU%< z`#KWH&53rXUAuPe>eZ`fQ!=LxFEP5zi}G>z%o$WDiJaGYDHH_@JFmc)yomI=fW&I& z@FE-kT<@gDxUzxj!TI*_m73nk{QCaLj9Lf(6e9~aDqVvhPNqmo(M3gIK&T|bp}==Z zf&aP;F9%(KQ$~!JBFrT%c%3SIoi218T2}ZBT3Qt4Q4!@;6XjDE-sC)E1t0h= zUARt&lVjazcGgLvQ5=&@Tp-JzVn)k!!|wL2xs98%>k#~uG-hDz2-*34rWiu71Ak8u zG7hIW3tJ65J%bb>#P*A#(csz2MEA%&JQ%`1eS+|aaVryJsU7d#my}X1Zryfs;YV_$~q~?*ixm9$wWO#QEfSz zDV?lGBq)jrQH1$PeB5H(*9EctF(8nWgNuWM<6MXA_Sgk7S4(JO`S?X-ly$9~L%bsM z{Nl>JV=6qOD!dbFL$g}5>qn~k?^gCq<~QC=5_V~Bm;ReNHbQVVKG&aF;|m=z=e@K#Z;MUu1Ysk zqMFH*P2`9$l7!yXK|=;gqN|oX#7$elTVKWBSOX9Snd^i(7)5)SCwQ8td0LkSxOXN+ zj}_&PmlqBdW%uT%w`QhRB*f)M#b-yQWJRS{K(&p2V+xw-JxBUDH%*~?>48nADg49$z733YE3zel=GB(6$ ziVCVwq;x4nO-V6%Q9&|ekEjUeHU8^YdC)e|Yn%`qA@&dMkewehLiYF3*RFE%3ri_! zncKKWdqorlBvgeYH-x0t2PV}-=C)=x+$?Cjo!vN^QgtJ?sMjyG#XY*r!X-*g(}G5l zp^J(tAxr$4qFidi*HuKWD~n#26XK8)xeo0pO^^c{;4wR7tn%}#hzO{P31A2{B!sny z!rD?Iy3%6WQlEw>jzxCp+l9_r%M^gkc{NWQ0NddB`WlsF#NMplZC(q zbf;FzbZAx~mdX?hC9;(Y)kckOr$z&ZPFiw+&|OE-%Ro8MTr1pKKitMJ-oq-{&o<4+ zI^W-+IW}Y{Cw06uf2^W#yryKVrfjgHxG^U?FETDAATlK=E+;OxG^?VsxqoZ(@bR;E zPhWoc@YCPke*2g0hwo-KPDbW-NAK;8FC2_89+@&V5A|?o@*YBXI#q-9q6~yA{LUrx zvk0+LiNAw?7cZnNhgPs07J6)^o-6! z&Jg|^hVV7oT78C)@nnc6h#w=T70_fqLXJizitreuJ z9jvMDr=;jCPq&eghJUAxiQ;MmQ6)iM84^KVN?ch?h{{-#AV3+sewB~o3O9}rJIV>) zE5ovFjUC&$2;rS;*8m|02bUm0T1nf))ioe3EU6|Yt2H{aIUu3ZKd~|*uQj1!AhCKl zzH&IW{6=`;aAd(iSXzU-U!tMDjl3jPUQ|>E{+~mPS3{IXOPohdoKsQws-oC+WpT8Q zFAIZ12v!e;xaAoXA<9C0sv`Vqq5^nRsx2w3D4p{MAnk7jE^ ztaPL8jpLlmQoU_50_}5y?5iTZ`Z8i~m*h@W7Ejfd-EFR$X>A;DYUnB{gLX8@FFY+M z7K&j(QhrNS>%!fo`$tcnJpbnXKm7Im4}W|5{x54suMoevy|MX&(Yf96SycL<3&m$j z(Xs*7H!h;tRAzN7gxG_xC9qdQ7KGUQ=L%%x<3YXb0@)p2l+Pf%pb~y;O+UqXPCLp# zc)DE%u30J}8VaJicmLyp`zHdAUhJQ|UfX*NL5DaUzPUYfP*yV>7FB9$9&2b2p#%KX z0u|-mY0}m-X>$d-HC@__Ow1%{Ql@! z0YnIyI!5MRlm*_o+pOUod`*uTA@*JDf~}T*JWiXLA3HfL%!~EEkRslTGIyjnLbfsC zIRbHdd>IQ3S1gNR{swjCW$-2r&TT!J zAGZkib$%ESa&qu6!iDbKWfRDZ@LY%I=8G@BKuzqG>sPMu2#Qj)4IP7{a&t<$vr5|& z3t9t`svJTKoTAEn(^?`5`;)6j@*1ZLo2Clu#|kR?3yRxPlM9_}JY=NgDIx@_ppb$n zzm}u`I7D~%5(UuRPvSh9V%$(9VbG{9!VA$5<%5P>3qHw+rU(xfZ81I_aei%rpf-VD zM?yf4C}cnqF_acHrl25%6oMs10*@6{(watuR~x#NtqcjS;lfT<+CiQIaaN$&%aLpp zNOsB;sDAdU6h~F6iyGZcOAg|xs|W&v%``$SG@@*E#pb-_~lY-L6b6pG?QE>r^2xE1gj~mU4p3Tbeq5wzN@NE$DhPB^ zp}MKjUDaiPqPvz%fU#M49H}8J@;jjPs$AA2< zuYdR(+Ecpr0Qyo0j1R}=_Qzn9$b@ig9^LH6ERY!?_9Kh2UxJWjnrH0Z4iiGAXGKh{ zuscNWpZ$nku<&Ckdd$_2`NP93+tz+gARep?V*D_8kj|U;umHk;{r1N{>^wvVTkM=b z8GM8x#OmHzV66BC%lgjC>AAzkj@uD&WoFh9ntFa3T7jzS0qW|WDk>iGvQBie4V7pP zWsoePDv4Xl$bew+6 zRCd)!M&(FS@nB$5gORD5EM1K#C`9AuR2AaT5$6Pc#w1~LvXGgypox^AF;T!s0>xNT z7-B*cGL{r%{3vKjg3ky;;KEp1>@?_%Q^xb27%`_4tYjqZWTilt1D$9~Apkiul86OS z*is6;nn?(n5(MDFT3XD8EDjpY;R7^r8zmA#s7!HIrnsrlytNfPHRXJCcN?q5Yih>on{K!C4A*tF71cH6RMw}L zl*eY2MyJ)J=9a~$SElCG7F2dNbuBM#J$v@eH$VL4hrj-pfB4IPIeGiz{LZt{MTBs2 z`Tpqa9)=K&0~fb2gt#IWffz!ZD3hoyY*hIfgqTD8^Y9-E4p|VQDVeDiW`{UJoD(xY z7IOFq5FZCJBRu`g1rswDg?U~S6NuZx(lI>u#lQaje|`JMKWsgEy!Ghm-s9(6_aC9# znvP!YpS(jw?-13)tpgYezS}wcdUo-+vwtbKu+P&c*Tf`3ODjN4#am6$6QUyT4h2wA z)eE(rF4d){f!P>4`aYp=k{s(Up!tdDfmO z9$|&C*&R6*H}b1)71mGWRF9{W+)S;UNUWX;DI5w+Yc#R)P?Mq2g|3tMIAqa^ydZR> z=txqMkR_4Nnj~N@&2K2hXCTRNC@la!Ey=>xD1w$0K`WZDwTzH0T^!zoUs=<|Y-J^_ z=>!X!xRs2Ajf{ky9MM661hJQwf?wG%-n60;EXZO=BUuar@F12H^k861?7K8kTUmmO zvXrL=*;kY5uO}O1s1RzR9AmAOa@m#@5n?hJxD0{OZ=+s^;|K=8W9B#I*XP^vbyOvc#WA}V9%N#?fiylBig%dI^$o9nXf${Sb*kVKqlIT^ zdgaE{^5Epsz{DaJxW;k=kwYICM!VHcKgo=6km=c=*wGc(-4e`Uo{Yf@Ai(Q@*R!A< zGvEH+ITiTX<=74h0)ecDc!9tJpR+PEaRP@RHfAmiLevS7IcszXQ4Y65 zh)kf_0U_gjnw=2Ry%8;(q7d%;{2(unpwbZ)Jrf59T%>(Kij{AQg?oa&Ez!~?#w$2G zB)&8F zwgBsSAyy!be!CbRh#rxCWFeB|2N}u;7|03$c)%MS zKS86zLdNpKAQ-?yRN7+PsEfkjyaX@u;^2N#{CWVYtPs2cTx_N&Y^5yjpd#s{iV*0j zrR1k~f@q=+1SU8b#M|owhuPjXg?PtmqB||Zvn4K|lNOCWR%Gz!=6uz+lGF*wsc1g<#M)Pum#*ge+PKZ`}4C@KwG8&dpN5bmjuLAX1}#GLSB5Y{3) zYHtc-4+U}uhbVy{?C!jnHfdy7w#z8Y19w2kXod)ckDkApT-g|1+L&6sIJvYwvr69~ zd-WEw$;I5Y(HZ3a)m3$)A>jo^W`U>GomEtjF1&(*g~DM=Bngx-2K=POb*02~B!te2 z395Yn(H4Gjofy^-Z?-Nw#o_vT=)b^(A`~ zDTLTkLPDiaWSLu7v1??d7pV!K)f++SOU-M|%q$M@3Q$*76XO;XI&km^C&vkTz^Nn5 zWgyCFB*kSU&1)9i!w)i%;Q^V-@|w!=nH}agmlv{B6fr*{Vx}N!Dvun^ zj)<8mikc{hz!ApgO5)fVkiHB*7##=(fE)v^tyUq@V0+l;UEP?qNptwJ#5HX$bddiSg+m1$L)Lb!8{EWhXU|vr6J13Z-O6 z7iLBm79@a63%ufKTl z&D@oH4ZY*-BlGmc41oc_=o+5y9GXK`7tWAw;HVGX$cH5SG;TV*1M;f4qPE;;Y#WWNXmrwJVcL7pGUQ zj?7(}Shza5aARolazpD>61mR7IZi_hr+V7u*fBdLC2K{nJ~=Zv853!7eR?-SM_f=- zOi*2f|CBKQ31PmY0^ABb?9v=8q6mRZd@Khykpm84z{SpoLI`Q$f5;)0C^A-V7zx|~ zAqWX#Klu!~YsDvk?Ed}x4zaOv@rlYRoiWrh_qOzma}7vw_DXQ}B01rbo&8dLh`G41 zeAnPS$KV3{kYeZP8o%_enEZjPiovq_{=(w=(2zK-)7lc8JhB|@N4Qu|@vxs0;4~8D zG?wH74vnR`O{Ms(Fsxlzk6VlelC9RH0SgVLz9T&4u7BN36Y^Ee+dPES8Foq{uD2c++`J{}S z+F{qza_;JKK3Yfob&sJ0#@Xuteq=WjATZn8D&O0tGSImx!lN_Jw>v2axruL1LI*Xa zJ}tX2HZv=Nnh{x^6)cw<#$&eyzCsED{Jb84<&sU;AzDA=sf^P>N*CI7!`%Ei5^S>HdsQqCu4su z3~B5YFbJ{E#BK;N4p9^tQ#*{eKY$=)7~uHaje)U?B^3h!!4w1Ipwp*aj~{bVKI#DOCnsYjEomqtp$G0K zDSBQ^NK;f$LzwTR5TA+w&rt!MBLcjN{9JO}>=GO-!a(4m16&Y=BEctpiZ<}Z&ce;g z%)xSq9fc4TisFIKnL!wYbnQnAHkcSWWQ-I! zQFm2I_tVmDr=(m@N;;nq2M+C&#cYp?S}JS;X{#(jml%@K(L+k`G=K`wLg-SHaXBsR zbW+k@Ma=e?h~05fhZEwCsuFM+dcE5jSuYKFkoQ?dyzX&=@yTfLKnL9v7sGT9({y(e zP?nc@p|5Qm^6a#5b2zQ_P7=f5D8{7dQM%|t?q7Y)RRW$eSa6C*XdvF0NltVhGXh{aS zuL~yhE=wqNn6Tx7yCDQ^gYf-(fbjM7DzY&)vOE3q&H2?kW3xBfx|Yb9jbMP<+Bh{e z54BSs%1ZV!Ql<(r=5mrI^x*THq?neNu!gXJngH)fL0)Abke^$Tk6WIPTaJfQijzJu z4YEPh1_8N4OsofxeMFgIjEec-2LjQ-CWQ13J_CnLj1X?;a0i44f#?pUD7^@fXs7Yh z&-d?V+Q-DoB`7U^|+LHuXq6IF++CS6AKie)i-z~B{ zEVDVaYABmFS=PPSF}~i>GfpO@+vuB}kdRQ}~%mS2i;T` z!we7893QKqAjk3`M_PnC5IB^KA50DECWo|VMzvBBD#;neNjdq6rBqUFesX<9Mjb%d zl3UcAPAQ3q%?=&T3kcDd{Gm}OvhLx(fe^hIgAk2rF<}Iv9f;jFhzhkn zY|Q9*|1E@cZ6lC=f2D!4#(q?#v_8ghNki`r0zV4j(-&_b100*bF)@E@X#8?p_d-!c ze=vb+VG*XO>7}mbrE<(kQPxsk+FU^jWF{|VEGMBaEvAhuj1W*2`17-?z|2>&s0m_OP%tJ#j)uOvp@v-$stlQkug;n$`X= zm)lW3*P{Zi$A#QZiQzP){j?PV&L0UnuSC#2?x%gk^Ng(9NohdQSyd8X1JYpZcuWkU zRtREk6a^uIbx;13aI?FlmgBvgM##qqaqlb2sJ(x3HVuS#oL}EIq4)h z>n7Tt1Cd<~vTd4dGPNYVq9~)T zIK93qtFAt)qB@0AMw7$0XiJ{SzsfBA-zIb-`>D!m@ z{_*0QpN6KF%$$5>j%$l4YG@cbS~>;#5Rx(r>H&%Brv9d`vG)GymY#{m&QTa6D}%fD z?oTkD++79TVSt#y?uHQMZAbl+5kt) z&u$1&4jCUJb4L*fSH2jXy)ig;rLJi_F{R4EHSXLw-;*a@Po8i+sq88*WhpOdaYV*a zR?G;QzafkiU_=E@L;5GgcTAZ7I7pC35hTEU81+Cl7C|78|^HYBdV zFEZafDA&{}PR};X&?(BqHP*%_#Va%qA5-cRQx=fYgs1e!lue{oj+NAnc6QCRHI7u} z6~z1a`JB`8Q$K+}tK_LF;c|@MQ$+}OLfGTDpo@xti>iqG83}JKSp-5ommE-?YzvLd&t zxvsmtW3sJtvUg}$0_|-f3T%I>~WfwleFRdyruP%4&{CN{+2QNY@ zrM$GZ6WBw*>l}q+Ti+A{8h=4I#5Nv2d`cVY#mJT66ni zO?6jXM3kSSMUd_J2#ZtEx`!i9@~3Ku2AvjkQx|jAmUh;Va90)eJuMxip%{H$HAd%T zwDt+2x)T1he1O_v9H6Kw>3l}g>#Q8^teo2^2^Uo{4>c)ot;0Ur@_y%JeYNC30p}Hh zbr3HNJa;5W`$&Y5N~H0L1aq}iYwZ*p?Gy(cvWHQsmoeGh5V^akhY@&Sp}$R8uwzw- zOEuB8DcrrErou+R2v&ul@!yQo7GuS(Nb1fTUcC?Q&gH(Qb;OF z4@f3^hGw|pitzaA#H5<6;_CdShKl|g+9-TMH?>b)n7F)t?fLCTZ|*&R`|zuGi`VW( zB~wJ@POenedE`0*1u#-=U-LGhWC>iU-9j{dop?x}`01SE8wA(AWh zjWrESfLe#A&>ALC*RpkkG+N(8L+?1cR*-SA5;{)}T{l?UHB!?#T-`BL)qbI}Z4iXn z8s>*+`yO*SOekjMO?@!Hnt>RiaGsGsj7jVvb_T8lZJJ>dgBaFHF!m>kYWqOjtE2D# zb?@Gbwzhc^slm}DUPs?oQ`<}Rl-seR_DCC0*5a^~shqg6w1|O}ke-BqjyV5WF+NQZ zzB6bg2AoiUPYECd14P6n!Obbg$u0sAva#|4hrk|2AgcZQ4t@H!{TPHOho}%5k=4DM zC~nOuMezF*2zPL}sqNKNOpqKRv$_wlac~QW$|$RzH8XV#aScp&2q0T}k@W1M&sjwn z*vDA9CwK&<5#sYgNX0=3#kkl)m#9L|xJqJXM_Sob5pAt?0J;BRYinT@=Z+J#l>;>ty&&0B z5qDRYc2koC{G5*oyQ+wKsmmY{tgce1u?oRJnW(QEX`&i$rI~1>m0+nEXRe-Tt&wc4 zndzXD<6@BQqMzk%oaJqn>1~$oX_(_@UP7?12yrYYI8+cE>cd={qrC=50V8RFGuim* zEdQ}gzm{lMaLDp7uS!y6D>bXPsQl(| zIc3!aw5p= zaYaF4MJX9oWtH=~2KKJF@Ti2`{IXWyu(@Xny^k+=9dd)T3sZn%L+>cIjW-&CB1YGP zXg7YS06#_ocSE>IUpjSX%c%NS|?M9V3CD1N-T_Yk$fFzQ=qBOp=38 zMyn{ML7uyuL3V|A2EO_)&}dyX^$q#u1@V-WfTVc0uviCNwug63R3t4ei&jw8 zP}&L*mRFAeham97&Vi+g`Ri-f9^88L<%3t>&98qE5uYP1ugS(O$H^nh!Y;C~ z$HOPh!zU{wsw6FUO8KO&sWmPrEVZD#t*&JlSp!9{-!#&rN?;FdZ#3?j;I=1$AYcy_ zBY}vv%9xDdXx)~@{W*l#$=!^hm+kl|oju6Z8m z_PBWWz!ZE`Mo@g7YiPEaZ>o6!#fnhu8ru+(Gnh$RXd1mSv3mF7)%zQ37pHo=nsZaB zL9T^94pet@u&DwU{X!?bA~%CVoFUa)FV9Oi$Ne0|?L5U(H_J^o+2L%Q<(XIub)x=p zytbn68Cl#ZDHmm7J9%Dv1wPkfA_3|${%52EHRU1=PC(e1;iv~nchJpp*2{9#r8w)u zub{Z-=6M2&;jw*OofZk+5| zjW;CQtLNfOYe_*J)RYS~Wf$t}x+|;ebBpq#(o%fl6I_Gi?LBk6eQIK28pxTA1vM=- zJq^`;B^4JcYsaeVC+eG~I(nCeCNIye-nn@5>Ah#)Kxj&98`ajg=i-&);g@IQlwgNw zh+CYCN0OIcMnLE=pO73EuME(rbWGdC$}1owiBd>w>=^I9un1x5=DckPQyCif2$ns_ zq|iMHq<74>O=Mw23ndDTFl)D}rXPb4bv6vb-5fGPxSLOuK)Mh!WP}jKZz~){QT*VC z*g^^r0%32@w4niB`KXhEytS;X39ttc%1RqcNf=0ooD&m5y6~a`r$zZyMfgs@QJ5F$ z2Y%>Aa7!a%2t*kO+(Zk75akeq5RNbg0wHPK6T%(&Ar_ZzXL4^R{1}A$_iuJJ4l%QH zaEtH?D~QQx99A|yspG9@9ck{CWZ|A{<(X{lnPB4)XXh2;8AS3VraR)(Ec`Mo19MFR zax8-LamkH|6(behtAn#QRHOtR%!P?CjavZY44t#+Q9L5a6niLZGX-V&7WV?yyV$n!BQ@i(mwvT6x; z>W}jnA^D6Z`2vA+nE{L0-czYgJyE7qXVq8}#SEA8HL*S&*$G3HMHi|oJ4?#yGN}bo zWU_Az$s;Jy$%E?aR~;Q)ot)lKP~Fim(AG3mQFWoFZnTy*L2H@m>{}if*?^!FIK27b z>&IXIw0ikLU}%b%;hWEq5&d13s3CWCr#3?xytJC`4CXTW8zUi(( z+0FrJRvvK{E)n2|wm6cdSF)9Frgb3I)IY-k3{pRx< zm%f-A9`CN8)u$xY6Mb6(-JAVg8~hy`2)49v%Z3Qcx(KtHFq6sP(b@a=7l1FdapLrYv4fGVqi{keURHBXyNXW~a#38X0!yAP>xT(#vty&v!R2 z^)avTx2o{BtPHfSz*`h~8|8WEmHHdk6091DwrvsigK_SYsXntA0gII2Ws3h|mdAL4 zMF&wg-|1Mg`H=#gPI%f}$Z(n2sp;1Eut6;#wP8Cw^7q6pd^scwr% zF=3UM-Ccfkstt?}z^`K$;^33x5mpwGJAXvY=A1>Ktxpmzoa#l)vGXQbx<;Ei#Thxq z899;6ywWZGDb|6Rj-lC}ad`nLwNaE7O2vhy-b({h_h&Xc+kC(dGVzrfy0~ zcLK3365kdb(3|W(nC{&kXV;nF(4Ju19BW<|W?UI)SmvW!;;C2cX_W1FF2zzU!Av#Q z@K}_NVwk2ZQC&LxtPG5kOjWb(wF=zy3q6g&`pW$*%WxKDI5U8-#M`tOXHpSh4%S!Z zXIg_dZw|BXjCSgda_WtB8cFg5161)ZZ7af#2C4GWrRhQ6pxgdn{Hib4qZ32-ZbAQj~2ISe)kc_16BFblIFy)nqge30iL z69>J-f~0?k4k6Jdng^l~qIQQyq<{Nl9|&!Jcu#45T zj?uGCFmg$?@J@Fo=J+J!`6p3vi4}ecl@aMp+2s=j4NKL%S4Wng&tHDEc=7SVnkN1DwEAmX4`90opnGt&N{g+x;YNIR43g6SAzl%!$MCZcp|u8 zMSy92kY#(M!(f8PNQ&2Fn(rbd2nbxt4P42_uVi}9rPz(c7!E|}*7&Mr*eX(7)Qa(T zRgpe*iBWaQq5k7B4qjcWO{l~0s|^iGCOKo#|GyXXRpjGTp1i*YU`YC zY#y(wA1*HMuW1Hf_}uOEN?cTh{ugs!O*kB~eYj|?}rFfXqt9Kj3$LM|>5 zc6LGfSe%zf3Q&Z+kpo!ekrojZ;#Ed5ri(~*w6}Lc zUCoMafSMAbXT^lU08b$CCtbq)$2S8}zN5m(Ed&pvQ79L?1UsuRE3*LeLB4|rkd`2- zLkBtOLd*~?%wP#2mKK5-A;fwZblcplmok=2(W>bV4$+8oduYnoz1WOPcR1vJ`hqD` zpM44_va+xV@e3aol0G3Mt9DG&#>6SYB`5nPh z5tNh{N-p$CD)5Rc@=I*QCv^v=^pi?wt1sN{UVbro^U?Z^dpFiEt&R=%Hk3CNr1a+} zER-hqrFr)zx{i`O=F)wp)4fMh+F<<+GUPJRGgpn;>M}JUn7t9KyWZ671~4 z5Htaha0c=hZZ1Gk3UV0$5+W%`&ZQO9)X!PCc!#D_tLoc^zzJ*HF0^zZz5e?4(S{CW z9~y{MYg+~z5U93Iq=qphiU^f#nA~y-`?jAXP(j1)yDk69q2oykypU05>_;{5O59Cz>p>tyR zxMh$K6uF7gCV_{TxIx^M5#8O;bje7*mIkdr-)JdwC&hNg|v37s%#?w3ZpFO^N_r}HL#gU$o z`oh7ywkl2-Im;c<5H+3~K$1>jO;d{LShE%;8w=V_fcSSnY4x5@z2U?=hVjxR6U+ z&Lb}65EirWi&_4w6yPuzAY9J$noYI`2)jZ~*ZUvKvX+TDCsg3AUmRdv6y#Y#B$h|S z0fcSo8O^E5b@9=8LHHzh4{|_IbynU;$Ay)-E9E8KYR50mxo{eI556q<&5VPJ<7^0&cY$g#vuak$H^|l#UaYcDGCsx zW(eSMaUvY@^2h*?aGr}-hLcAcq7oqq<&zr5&YmHFVq4!7t!)@`#pa$V;1DoFZ*bk% zxw$|IotWA{UpT*e!Wtuo{}qJvh!o?H(e99(a5r+h&-_~m(a@BB%^(6{U*Gtd)8?W= zYJz;HgauR~{si%JD+7mU9|E0^#?LJSLOX%zyif=}5pChO8$z@?ib4p~VQC?H62e9L z)EOr944OT%Fmp0;_(2SDDPwesX%~kdqEmFd8MW?b+P7~LNThcUA?pD)W;T9)aV4?C zr=^v3kEmH`n)sTz#Xxds=bdEi5~pt;t7R3VWfi687;Whr?-@Y`GbE-~1gBJa$5h(~ zm7DvOS_V~mC3S=sPvl;>*0uWR;=?Z=z54pet7mub+*q9$8m+G$E2B&nrY{wzUnIZc=zrI$HqX@Do>qC7p*F{b2T11^*DW+k6}xI zX&b?^E6lbn)V4j$t~K1QCCnC%J#nsM>HhP1q3cEAptbza#T@)Xrr%PQ?+V3#of5E4 z4cN%_11Ibc({A)VTHz^AwU>+67tL`$SLAP1;O~|n7+4w>(~wB+%*t*hr!*u+=Mw^w zJU!Ech;@|W(XP?e`OB9!?p)k>xVUiZ!r)S6c{eqOMxiujX3}zUT3T8cdWSZK7j9m< zkL<{H{oa?UIaQ(xY5*Yzj|4yn2FSr7!p0^<2TvS9kxL92aKKR#2&B8B1Shu?2m(|< zk%wPSQbtwJ&>@(ZR8rO2&_3MQHG$e4#HDBj6`h`j2>?T}+!3bkhI2>W_%R6438M5M z69EiM0(Xb0AJM;oaLWeiP6*-X(9q0T4J%QhQ^20Epo$>>F(Co)z~kV7f?$ArN9g+$ zO7d`sBRlr83ZNxWdLIG>4`F6c0#Vc36GGq(B2s`5{E)F>2oUbg;r9G+PYxMT+>s@s z5F**&frE${?%#i4A5y3J+ozv@_Bpb&LO@JWM)8cS@;O-*BUK$YU2B4YLl`oj+ZE~H z1Bd$dF@{bN=AO~c!70A66auN7kW}p*U11+wVi{0siZ8PXulG*xkEJbFPTrZm_44ZT zuOGaA^Wy8bPwqdywRUNyg*H)LF;kkqR6$*>%-*O;zfwcKQbD?_TpSDl03!8CL_t(k z8Mjg#F-IlLVfu?OiW}6~dz{$QShu(OX0h0T0DsC*@cRa&y zni9BF7`9#(y-^gpRuHmG4O-0dUCi{M6X=I(DZ_In$!;KA2b{18f1=1mF-(Ii$L(wh z-l{Oby$~N*5gyk>O7BQdZzjc8Mg-;tdT03iQ={W(1yw_x!}HS@0m8M_JF_!adV6Lo zO55{t8geO(*%>q{wWFjZsK)7`M+4VyvdF8YoY@4&%j$w4049o-T zwieN&effW@k3ooPt4xZ`(%yz2BZQ0*=`Q^n2+?%EH6S!i52I2&si4G%{fGfxtZ>WMSsOB2q9z z^b8$B7K zyj+!dwI=amMeIgt)LL=)T4C67UdU`#z(fjeIMD?(L~^;1=nNW8ave?em`KM>Q3B@j zf|rUyS4zT{i^E{BSV)B9G8Mm+8?cn^w}27|2AGH6$njs!z|AGQOvGCagr4t)aMV}1 z%tJZiELXP6nX(|8;y_OnLRtd3g_Kku6JA2}1_*OP2&KuHZRL#@x+bP4*OwNr&&}T) z8D4I087VJrF372;W;f(y)5?l_8yY7&`c``!s<<6IpDf!~^YHWPc zEL;-o+~S?Ex(A*Ct5C#Z@^Z*qVW|(jUP~k@)J|0PN>0|2W%-sDVlCvuS30lu2 z#HC<^kSK!C_44Q{4H(ok59~&6hfrh0AlywOS{-Frue7sQu>-=7p!jbfLM8yE%M+rnB z#Iiv&{R89<(j(Hp{_kh#-VSyfAAEq5(*qJcvK5NPKY?}#Va`6gO=B$|69 zSbC8h11Qcx6h{yt$B~$CM<}o*lsSe~`;j_h3WoCA=G&*Q&0T+S`{|DlU;K3c={GN4 zzP@wg?#ATIR7cZDP5wkx`U*H0f4YHuK_sBqtGhKdaJ628ro;uM>obd?q9zD z=F#h4pM3pG$MB+-i3_{HVPnBp`B-LILRk1Dy^cWZyy*z&Mu-=(E5?guw{VOcA)_R(dK>_H?|KofkSS;w*(`5 z4Q-?Kts`}9Bk1juKwF!{?1XTOw9R!AdlI-i9K{UtKZKB>W6_UnZ!s}GudQhgVW+Ub zQ88gsj$9!Osn$c?%mMj!^^cI|`^V-FKr5d{wf2*C`|Mg|G2~4&1CY!k>89Bw9dZd_nrr7wWI1w_OLo(gM^Som!{o)%! zle)t*dJ{{A3R{=k$L~z6f4Ta_+slu?z5VRn-6yZ_-F;e9J*Qt)CSSPYJSjqAz`B^1W*KHoK0~ViLx4xu^5Xn>LIEX*oziB%SN5!h&4T2 z8emZq>R%NWUKJKs8WdI%6jT=MR~hM7853HSOm57p>Z=$;!KSnTba@9SUgXq#%P z@2@Utt|@J)DQzptZ!RtFsHqvDb-UBbyMR%Q zXZ}M7kso0YzJLFIYI1Q4!Xu(WM|inp={J!VMZb#J=>!6Y9ISkx9r$e#$XF5GEXM30 z5Nq6H5P|>(fDnXr3-^Q&Ir9++_Y6%@2zLjl$Vny+6ha6}fxv?YnfD_QGVR|712%qP zIYkv6IaNb3B_k;n>m%y!s(Su<4nzlhlm{`WBTsMwP%-}e0%HJ56@ox@a*xc+Z$^a zhdY;Pg>yxzi#fz4Y6wK3bo>GUyqPTTsSNMgOz)Wt92~*i0M3mP;(9Rx9?*EPG!!Ke zz+0gB&1d`0W%(@S;o%kF^miARNj5c3b8(jWkt1v&WG@rm>Zob2u{D=8P zj>##VRnxKv3{6bVC;}A0{lEiJ0fgv;8^-+b{|-X5@3#ZOa{9U_#%?}}C5Vy9{{%vY z#yuUvx!GlaP()A(f=_NvaUL#l4)~&C;|HPOZEaG>gKQT1@0<|rC@>v>5R?^JA4p#- zxYayFgax#4gpWShAV7$207Hjx^JP)?Fd(So@~wx{x$% zqO|N|b(|ATJ=0u?1)<5cBuaBiPIF=!Ejpz+G_@x*do(P6GNyDoyLGK{^yb)=SL+YH zyYckh)33gJ{rc^T`wwm`Ezb3JPnPA4Q4^*!{1&r3R)V&BXwy|S7XZR;T3=^fC#bQqvyfU>T-a7pF;LgN zGPLq|V*MdN`1q@TKKk;P;kg?oHh%0phdFs zf+D*xni(>Ro1304vI)b-5@J+1^6^Ot3CYRIt7)CL!Urc5meP>8v~2*?+%??VH%9Bc zP}SU9-9CV>2?o-TG;(W)!rInRP+be6`qr`P=3%-%a7c&kqti$KT)Ul+!SN9Mu+9V) zj)EW@{cUD?QR9puAGZt;h=u|1cm5cJjQDLg!<`(WC~kK7J`lL=xj^aPKG_ElLR<CZ%?T$1ca5C*+jW`BoDC( zA^9P_D2hNxXAB{bLxhz>5G8OohhUgUyh{IA(ie*hAq5*gDINhC5ecQ^C(qkD`oxeZ zrPXanMA~tI);`$S1wrXhZ6`AQ1+l5d0Z0YW%Ia25I9k&(QqwdH+701WlW>!(-PZRp z?HP`K3_?cz<&XEj&P*?yKB+ zZ{M7om}#mQE=eDu#*bx%O{e(GBsk3`I!#2{Ohh`2$GQ#1y7fi748^-mBznxJx`P(e zJr*)N=Q9x$fx5MV5V}1g{fYSu@A*tG_$g$A6LEIqv9>UnOLYYZ8(dDOo@LLq;wbds z^gqU3=%v%0hHps>ttNyQcm-2Dys5sP^@*XKxoLf6mA!Qx-7TZd%~LfE(-rko74?%< zHN(yI7XZSp+Ro;(rrM&0vb@^DyrzQEuBz6#j`15~>yM_^?=M|@diBBEFP{B0xpZ5{ z*cJXUSvkZHv5GLW0)!A}A|8k_h7iau%)%}J?133#+JPbvhwK~Mmoa3AQGJ8V{M@h9)<%8FH zSDtP>{QlmnpC7(>_tl#pe*Ed5@4o&1$)m?tmse(b`-ZB^2Xm9gQvIhB9TyVqrlV~q zBb|q%fk2PGSkK`^@9{+3N`}``ng@iMOBwD*)Nb0bVsA?Xf9{sPaR4#JXGSv=Pr97&GuwRl5b-ip*%RGz$+-s>xX-?&RQB+;Le%&Xc*$ujJ(fU1ZmrZowE(scZf1^jW=*g z*0oR6vx_rwO0e`uvG+~)2+0LM3{9^krZ)s#OmUx2a+ynZSxk3Zqu|!5{ulG{mkNRa#kE}j6^id-CJutt*<{zT z1pA>F>+v|pxik+*HOHdu8(lS{j~`6ZIauZ?;x6}jg0Xx{ycaDlu#!k9@((J;6N`gF z>q(JK8L6FxCA~H6gKc9MdY5{;ms(n8YiUz8HAA%(-L$gShNAka+)7Y=F^yK$UEcs7 z^u^YZYtxrsj;}pfxccx=6EZ@G z+8~+}VjQ9^M0!z22wszyUz)y1N?iJ|s+NvLXk-etu(qKcF~e%Wsj(MnD$?8fXsHG( z$oE8n>Zy}xF=hCrP$z7ZG6b~ z_$-$Ys!w9MXMCAgVzn>1#W%Ihi`3$s+!mD4l~g>G-?&iOxz;{*W9s75EBD^sfBF5r z7w^9K^5=(detGit*T-M}^u^tm8?%d3w2D4TOka}!K%(C$2|u0`FhW8IoJz$5g!4)6 zb4l(CsqSl8-j{NHK^wU~>p8wa<0>3;eAg&GOX*(IB^T*|=TuOF?g>MP5qm1S2|=2U=c^XpnF+qzmVbac)%b}u(wxIDc4aBTJd?A0e%9)ElH z%b&pTuiSs*OGshkJ9+dPla zf?~|*r9cSa2M_{^03o~wltadZaEJCpan~E$iQ=XuZgGeSTQm^i6#wNM1j6Ac`x<+-Aeqky z%707LW5yq4@>Ag|@-eFp^=ym_Ylw-diy{>f;)=pUE8`P83d%;B`=|Qm=Ep8hj;{4| zPS;iUL*$vCQJ$NeSDIN?lULhV*4$m+Kh!;WVQ>i`Z0O(UoVYo*`f%p*)3tkV@4Wi) z;p<=Ty!^SmaY#&7jg3Uzua9c?Rb3at{&LLRhrgI{t3U=fnK!}!V z-~tvF0T6OMH&SKg;F6S+SJN?a@ClB~D5$Dz>Vdcv@kOt z>GW6QLNVKVIs)Ne{_@G+KG_G@aIhn91BDQC<3EQG<#5Nl+8vi3qE~J(1H|ydoRFK7 z?+_C+jG6W`fA;C2gDku!PM)`M39xnx(|3qaHVl^23Q#;BbW$%^O&@P$O?2=i`4Dov zLvox0(*Z(buM~Z+WL>XRv%nnth!XeMDxbvqkc{^D{K4ds;iSURjEb3zs@c58<*MGR zw2_;=OAiMxK3lr^=JNfwx1PVd`{j3!-v0f?+kd=x{^rT;M^~pt#%qfPGNQ&)gC|H{ zL$S_d@osa;KJzL5iz$A~X}+uJxaBmrl~mV_4EL)!-fL8!#Vn8IOb>9#g;ckZXd8fV zF2fU9(VK;vNOryuVOwc^I!I<8LHW~IZRThVUjGx^`Ci5VVS92^D=DENCOJPaHjfZe z7Ms{zR6W%>vN*E3HhXPx>e4{ZR84tTWl?i(N>N5!R&ja>1fp$KUHvV?gPo%T{R?&N z^VMBohL^@S9?xBW0TAAJ`tH%2f8BlckCFMCC$%j1vj`zI7-l{mP9fypc<|o}(H7m` zQ1BQ_GTSAH`RAtKAtwZ=B5W+kDgw9$dB`DzEFk6-Le@s{%J2xu%PDIb+qx6u(x~M$ zfUvH0OK9^D*&t)Iu5%0s#2{qo8PdzDySBXmF<`b${}w{j^md;|A$))R`c1Xdn(*m| zuPFGZfe$_MUA6V;-~Dml5d`jVLd*lP5RpE>Y!HEPtL}ohA~U@<^FROd*Z=+BPnejH zd$wRUh^D3=vqT1}krEF5%(gEG>??rr9d3J*+c{)q=HOt1P@J8K-pz;i#>*pePRGL0 zJ4V+r?ub57Nsp*>-v5NIpPEsizFDB7MQsAuFO@Wy88MX=wvg(#NOE6Ea$irwZDjebXZx;X z;8xN+H!?ggXL(-A@!ZICy_n&8DbsB}(QYW*cs$+)0@RgUzf}rhiR{zqd$!j8c$gxq zx8(oiTZ=@hF^8V!CYzsbj`SJMPUuZfs*Ox6@=wUehgK)1ca_yoc8#n}tY4hDwm7yv z+&foS)LM{Mkr`i@lT=!sQ{PmY__Y1m`1KWffg$ivy?6?I5dRuH{CyV9_sEV5g8CBZ2sS+9zjv^4AQ9Jwp zGlVc^gb)Y>9XN3CPaxbA#s3QkvG>UcArOd)<e3Q)UNER zu>#s`Zq0N?*;q>PXl(vaT*-KR`BY}hYJUIqhK0w&SHE1p|K`ricQ4-k^7W6uzy9I( z*Wdm2;;Zjpz4-dkjcb?2FU&NSkK`v$WP~ne1}~FwtLZ*#nE`9r_;pIa2F33ZD8~m$ z4zqDUo$MG1e(O8?AH21ZFz|~^H3N>sh3D@AEo^P(;E6wC6{FgLyq1bbL5oblx ztu&gW{4V4q^=6anVv~!4lc+)A6-nvs<@Mv;Bg^Bfs}mQe2IhNON2>B0^3zJQ6AN?6 zWo6Wc=IZXAmZ6@u3%zZlZOx;NEo1d+g zQ+$%Vf-;haRn@di0z(tC^Q#+L2V1+xn!879Tl#C7`;a1P>%g9>Zs`NnwDyB)+Xm>h zRz&cu9fFqvl{a*NDrlXc%Em5GRZ};ptf2!`PU{3!GUr2y4%H=K!)-SguoWj;Uv= zu5+@6Q-ZcLN!KmW+&A4bsyH&MHN9+zQaP4gGeN1HPAVP=qxAZuv=Z}%Vk>7;o7eLO zuD35eT>Rpz+t1!1PWbJwUw`-OH{bvI@|#~?zWw>h^Vhd--d&qr7;mNx=ckQlMbD9a z=g5BZY5t3ufvdTM4Ju(HFKC?_xR#5oU0NV}jKTrCK_SQa%~7(A6t7>IQ% zb32n|s^l%t<05@9%~n2IPdH3Vie#Zx8S2xX5!aNSSP@Ig3rQ-DNvX-qYp-q|=^maR zo}V9H9O)WwhxjV5IxnL%H?6pkQdLpZK&$S6lkJV&&2^m(bzQWk!N!j9w*L7GvzKQt z-dnnIfAQvvE05pZeDVF&M{gFdJ~6TN5V5Mle8R ziyB@bUOpjlF&PPQSt-fGvNDPa^2$m_RFsdZ9#cN4dQ$z=X-%~=S{jCKfU$f8R+)I=b(qrz5+dZ{x#_Fi`P$HzIpoU zE$GX)--5pS=DV-oegFE~A3$%u`w{fh-~Wl~??3|MlB@RKNZH z9)a-ItuHjxwNVJs&;9?45N-)0a3_Qyu${mD^=|;-=bt0B8GsPY1{opTJhPq2{}4j- zP40fTj1X?#A`0=w&5dyA-yqz~2YE!KPpKLD>NzB6Swv`>67+2%4DBQI?L$FE4q?`~ z1jq^P{L+oxla1WSK;Stil8$q{saFcvBO$3aA*YjEG>}<6lG8AoSTYomH{eBT38r2M zE18I|TS{%+sGGVwd;QhTCvP8p^~19_KfQYW$|YjF}Bh- zzA-+3ef8@7n-8AfeF1uSTJNCr)^pha?6~F!q`RSh8y7oU6_6B#H-}$<` c^~d}F1B-fhR6;cW&j0`b07*qoM6N<$g1~pFu>b%7 literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/case-study/akvelon/diagram-01.png b/website/www/site/static/images/case-study/akvelon/diagram-01.png new file mode 100644 index 0000000000000000000000000000000000000000..1b960d4bbef00e225feb3824114a07f382375c0b GIT binary patch literal 105049 zcmeFYXIN8d*EWp92+q*PTct?T@m54S2uO{JijEW&Lx%t{AYEEQ3lSMXsWys~L_k3Z zy+a_8p$P#Zy#$DWA+#h?LP$cAFU)&9-|zR&_vbsF{bQ58IrcuTwXW-0>pai3_RD)V zmJ+|8{#{5&NaD`z8xMqpe)}jS^sDAE5y2hR{Lj8ZM{>{HxpD1by>#2n<%G0L_J23yH;L-}9rL$(Mq!~ox^L%R4G!+Gej4k+j3{$qT81c&dG!u2 zT>RuOf>HdR18V2%l)HPfV;dk6Pu%ZI znX!KObj!Q*|I)uIc=keCdi?b-N^jyDdFe!JcakS26vpP;?BwRg_9`x2dP)+ zfx)xOo#=Dv?4d$ZFq^Gk`$+3(N8PzumxXB0xya?_-f+HQJ1sM&-5?k196<0VMESEg zA$?}&t9U;?>D(8X9h1-gr#^{?U2Z~19U5v@aD?+hyc)jxCZp%QmRYI#J7-HT4X-c{$;S)YcOYA=5AFHM+f}WZd zX18eiguOb6b2MAZO}D)!e9sBt*hH=_Q6DNZtxs@{fcipj=lPAjKMa`KTaPU3{t`iL zcMs+Tv*NZm^W+G*i^fbXtrzGl*VBCp@EZ-pZTN@1?CnS66%j?M{sg`?Lm!UK$BLd9;0JV9gk1v_j7C z0hzk?wh^EZB5^DC&~e|CfK)B5!*y8Kn`w95y950u{!`!RW;~{ogu&rdEDMm%T{l&l-hN5O))oHRZQAMDNemqcz%**Zc-zm zQMzwddaP2x;bi}I?NEeU2)Q&F1yKLpnJ6ahq;1TBXeourTaKCv*G^n6U=W9cPJVP| zt*4Qa)sp{o;I^hUXsvIzy2bYI{W29r=_n(=FPo{Yy7X`;Z|X$08yo4_>=UUCs(x1u zp}ke7xjtDG#ZO&i65ETW_tNrszDYL06X0ykjs#Rric8r#iov|!*wL!kr;MW* zj|eim733G+O}#oH?dg9IHJ@Jr>W@9t`U80MZ|}vz2hy+1VGdCo00_pqx35ZwLpNiq zNr;>0zS;K&_Di$Q73kzVFUWU4Q>yS)Pp;2zp}n<>hy4^8+pPw6_|XK>DVgAXrl;)S zHT_i@+HJYD=NdN%oQIr3;C-1{l;@+ zaLrAxrKO~2CN+w02YO;3)wH$&!ufLqD^2L>9zNG`_&+~q!xoIg=NqRMwAI7)NAJ9d zfH-b^qP$~NMf=p|hZF(}d>f3HZbvk}+aN@Hq1ofZm$ReTDbTp<{>XQ^W3l$SAM8nn z{*x7oO>;7VCY{Zy{#ljCvDrXEsp>{NgCli0sG5`&(dv3D?9!+oXnwyjI5WOxrN5|^ z#tz;7_}xCs&@VHhvZ_rr-RNQ|ect z*&19hizj(AQC(!Blw$?F<_r5V6FKy!Jvt(@Cndy6t1WV?$P@pSvQzQahP2p9vyu+g z{Wjr;7$=8CTHjyAKO5;PoUK~Hy43Bz`W$SB=KP40!y-fz%}{feEZ6m;{lytZMUf;^ z$num?4Id^J-o&?h%T{W31GGcH&?Ez&HSd1F-_tv`W7Sz$cGfl~?l6lfT#|iMsDc4kI+;w9)d5N0g*)hlG>vK;>r*w+AdRB76w=cK+!*-Bk(=5xQX&&L@Ta9F zhPOYrrw4)#x0D>=e(O;VKG}>sO@|YrERI&K7Wlm>k5Kv`QJfzd^YzKrlEuJq_yoAzSG2tTeEr3lay5lR zkHuI{_S8Xv6VIVnoN8OlFi1gE=^GdZI-Nr&P1h}i-Mqh3{)nsQ|JPbkV8qAGnyiAh zOHKf1SCjj7<@d><^#(ad(DlP{A+6Z;rb~a@Kd&yY0#^#`@EgARv*mutdVo1|WN(Nz zkn1sNX*XsKV!(cVWjqz{IH}F`5_Ab<&u7+MV^n= z=O#0fFh0@xPV^l-2hj+v^TYB4`(<>o9(7N*_!42WYODVpVC3R0Ld4*>C5u37}ew+nNwcjP6J zv_~J|**+?gs0)l&OIQao9;VDYfIy->t_$0EtOHh zpYS)GDXWP46Fk{ar#W6;wVOEAp=%DKN_5%WuKeXDFs<6??KNn(v+nhzI&pO4`m0R+ zQgPn`mLPJtP32z?`!7%%jjgC%S8Y0PFU~_Ywr`Eg!%E6CXf?yg)^n4VdCo^M+YtjB zp&~B^PC3N2QT4z2$x{ryt7UzO8*a0@C#Pc->(0&lp0kP(kh3o|J%|eDUDOWL-NUG4 zvaF&^&TDpF@3#y&vUsmIvyzQGzb?JeMvw?+9jC|3ROoM)e;&bL^o0a6hm~h#QYh)ps?g$4_$E~ zy@H6?)x1au`61x#ARVc zGI$wCqDPIcr!_hqqQ&W*@Y`TiT`X(=_fv9%ZR9L^-zcQHZ}bz4c5n4F>bGW-P$Lj` zy}7xPk{;^je;V&zFLtzO-Kt&O7rwfbL46_uVSj6q^kJ0U+|Z!d+1#1^3-|?X+zMH& zDDbgQ|FKyQGF=M1>#xBOB@8OKh}K7|@#0sf7XzmL;_8EHv&H@mI-6}3J3ktdqed4; zA+YXbGK_xjt$%if-imQhD;eNCDF@S6Xfk2+pI=_BYHC(=8&^S}el*~`urSA?m{Q~{ z?3%sj4N6Z989$oz+G#R#Q$8vrl(YWp=yYkL@v>&hmBaMha*#~eY!GVnHYEHxc-2m% zXIs3@47m9hP@B5qf(40G<`xX5zTbQ*Y+>WX(5! z$jQOF>{YrF1hbLbS(l%+(|hY;HMwoCw*K}Z=`CrS@K2iZcYdCws{q|PudFEjWY$g- zw=@RPsEYd!nRc;szCw=h+uNZNMk;TjIF7L9ipG^xX~vAoHus$XJ%&Cu8ta|$;V!n6 zJQP^qv3Y30g{7>h$PAR$&nZbQ1A zR13w_eF*6hpmOLynb7tdyC=x4(Xhk_D?C-s-%v-yy5XP0Y5(#lOGRoD^SwDYS5NrH zMCy^u;uMZq$LAEX`jNxwPBaEl(@ht5^sC_;l8mqU|Je=cQ*f`^m}_?bV|k*&CtZIv zi!tnG^!n}Vj>U*-^^LNqf3`Flr;L~pBFDp|lhJ^vm-hs8|BKR@SH0$+X5=cI-uWet zgr3G{qZmmRinv+*n_$gSrCToT`>$FAR3h86wlGl9bKooV>z`jIQ0S(nY=BzR#c|Yt-e;S? zJ1|?o7wpdjTG#K)zk#3okx4H=16bpQ@3q$%l=UB!;8^Y<7XHv-EWCHz=}EchfNrWi zz$9>Ol$bOhw?JHQM3S3SN>J@3(^=6z4dO`?Vj!uck>NrAg{FXadK%f9@(b&T{)>1w zj5ophuCvoawOizkR;ajL;Ro$i@0QXsq2#qcC3L0rg{jmzhKTDgZqtN&l%P%A-lhO=?rn zoh^1+`>+`(Az)zE3;85}v0lIvJy+QkdgcmtT>`ver7sBmC$`1E9FjQCDYDbch`ZSf zMw|42JW_+VN_@HZEB}v6mS+1|8NR0^UFzkZzm2W($+8PgpBt%+A}%=}r|d5$+0I9? zj@cy{Sj4FSIDg9yZA%m1j{V+%-f@k#0n3|*Z}s)Qj)`Di!PK>jMBy&!48zQR#3gymU<=$NSDswlx$l5fn)9D7F#APxEt z{#p|-Lp4!X$*zq})%(xGSsl!Ez3~atkrtM5Q*Q6&?2&x4;>?x0UilzA)tDa`tN@+g zCR5RHfw|hZ&!lx+z^(fHp^@O^gWb&cJW_3{w$&gb0eD+edOzH^i9#sq@gHX4r{4#> z2Q;J>-K(pw0WK`G69lmF-pS-%>5qN`zu1X<4V`y-f%!SFJnhM91#LIAjoF$&R+*-H z4esv-Amq?I;=F17wnUn*X4q^(8@_sfuqIG07M>W~M>FOKXiHE-0mpGioF;xsItt5B7PSbX~UBK>7 zGeU-jT^(F&!LYihv~RN-&6|!sgSq6Yfp23@Hzg`)jg)+s$+mTZDzo2zIBG*uzulhW zy|r|+i#Pn!zCL|QoSXL19Jn57{I`Y+w67-Gjj`{oSRYjslDL+uKf|t&oY3v9qo&B) z#m6yvC+SJsaM*iV3a{z8X!!q$>-Rh_*QG+naVU%)* zYc_OSvW_BO4}ZWD>rchZAHBI)?tZ1e4?y|h^A1n&RRZzZ4q0c*qWhZG&0DRVKKKDY zwwC8#|E{W&>)KbQ73);nVS+5XQtb$p_SrTcqWMSJ|1iYHQEN7~jZo}o^~szn`KE$a ziOS#au-ZeD$4WqeJy@c~ISI{WT4B0xE6*=4OmSuwOS8BcPg!za+$>qz8z~V9q>=nu zGt}HUbA5&j`_xe7AG{Xcj)Qo?{r5es-jN#3jm9bZC>Ucpg-2k8%=(u8n{8ukke9Dy z4cljmU+AE=8}_cHy}fUTZgZPt>UJ|GiUc6pHM}p7Z@x*lXMC;LAlK;%?JeRL>?TpU z`nBzZ5liJ(>Z3j_>OGW6)#qze^&3;+>tY1I8HtMWyvE#0kB_I_2VS|>uS``elX;bZ z>J{FdA!qjv&1?d{?ylD9HtLJwxUi%?2PVPTd)V(j$)R#Rr)+}rUz4&*jVztfrsi^; z1=n%!Je6_Pla-NqB^WXJ!KzNRar@BF0VjSt(+-J~m2J-*=~R8uY49IP=8xk~T4dG^yp zLt%o?eWb}c)SehEhD%{D#urRz%b_7R=-qUozPFZ@hEAG@GIKS0B4& zz46eG9t1hT0Q_N28~tt8YLzx$D)8*Z6q^`c4ci0#1&ID`|EYYDT5%!k+9!cHrW-Uj z8Sn4#i?0)SaOQY#8)8^(Z)hfM*e{~;AIAtM%fZtgT=U93+n}kJDg~3F0LpJg{H=~^ zaju)xM}1+ibt@rMIqhMeLfHiuGXwt*3{L7pu!8EUgV}{(`TR^T%2;;X{wV*1fLRX; z#`d8@z`fivdtM%DY3fa}%O(D|B#J+IUnzUj^UAeXkd+>NcA30g_dR0f6J0e$!CEbg z1QeOA`}wBl?XvT)gP|;kQycwD!%IEO!4zah|0`F`ZK(R(HW3XmUl+#x^fc=FiqEZZ{4G`ZJ6_`T-0<^z{(SjC(C}Sd!*% zg5e46AuGlnNvPOn4I0}{`rrD!_)|r^8*uU;XoU=FCNc&o2Ail4rv$Q1Js*$qTZL4IGogP zXVZRDC1f4=-gkAxK;4`kM!Mdwcx8muFrj}5DD>n`pOJ+vZOYG2CadB~r zZv7+RhX9YCgw74G)wWLgF7Ppm@*68= zYHZusKr2$P=gdouu3#adw{9R1C}6t5h{uZ=TV6==?#{A?*Z%zWK4kATe)(I%UPe__ zRaOT2*?}SVCw89$$w*COR-~{*hmv8aa0i9x?{VkMB@azIFculK)*I|ilNWxfIFzfe zCiG_OKQS&-7Ru?vG_6~F4YlI1wUyO2jGFQGQ%e34+L}@Y@94LT7+ihfsI50S6crTS z@x6fDIWs(LGe7E)ti5pbl3idsCa$ZbWpj@UCXX_Daz6jXmg*jCE!g_ZlIlj=I^MGh zc29e`&1_f4`2@BvyKs8SF)_tB#LuSmrI0a-bw;UDs6#&UAmJB}KXsIT6$qrqZ~H%a zdg_DzQ8~e`>0X$OASHp>g@!#>NeLsQjtfl-9R^;$!Qngy2P+q&`u#U=7aOoIXj_Ph z{)8IE`V`cTbWbk;i@nlf`~f3sb-WjI6;hrC)B78>T#&~Lvxwc`J>U8^5TWx;=L&C4 zb~S~SveJ7;)J#^zG{}4ACUk6N;R;i%pmr%>kBI(k~c`Nz z9tqqHRHyBH-+iGmpRf~_UJfvBI+^5XxFF-kKY)rpJn~evaNT=P1=j&;r%iLb7SHMx zql?c?2AzIksj-qsP`vWT7?2mfyAr0lJ`O0x|!C=H6oEMxgeA3OgSGYo*9zUg5 zbS0XdvmhIJz^!d$@}_?R+(uX+y<-2F)Kz!joX>FT3!|O-`PpyF^&6JEL20~Tx8w7K z7e3etp_*X1iz4@;oG@{A>hzowVHB!^0ApE-r&k-NfDD+o29+G; z5lm&V_ z%J-pxG%YU*P?gXhhp^$o2XSPehNgTnY3d}vPzq|ek#l?`!ecA!+6~IJ9`dWzRo90H zZU+lMSK~c3&H6txK0US9X}9$#;FkkIuFs0<}LjF{%cUJ(Ck1I)QP_pZ8`PDo1dTu<)V$Q-Ov4 z$AoI6K0kZRX8@#bQHsuo(A&J!8w;yt%fqt@hND+ty%p(jU~@iDJI9)R3Pzrl>g^;f~w!p zY(&UB1Ej>}2&&8%1dze%*Pf}I&FIPZ03QkHO*P!?qp#|iPnBAlV*|b)lTA)$R|Nug z9stg?HMaO&dlwe%-PbUjzDk#$KKpKRWCWy8LlrYK7qkJK>GFayJTBj|hu1-T#)b#RM2> z({QEL;w`7dR7rHEn}dg7;AFsd}I0wHpF@ zC?p$y^>o0_0@Lj@2)9PM)j=q_nbdvXQc+2zU_WoG0gMt)k4q^oDHuDnmA9m$*pVTOxeBVA$=sj!29RrXB1ch6ho{(dBnZVq$&5Dc!h1#I)p-Fb1N z^+=}$AJESke{*HLL&b~pDKdS^`MiIQ)0vFmShcv_WmH4EhH@T(^7Wi#27Cq<#B*mU zY_kkBTx-toHYF$mYW(HK_s_ov)I+Ilj;-6s`}DWXr09)|rFC?*YL~Avf0dL4VEzq= z+(PvRHcLn+o9~U_^a?HrUHznnft?h+EXq#C&n__;2?ZnUsh+C=a_t(J;)s)?<|dR= z2(8pr(}u{oqD~Y9*2qmJO3g?IVf2hkhSJmX>-A0-HgHRB(pOqXjNsmzzfC-J0(W3F zF=^dRR7E@k?8JMF4{$e}3?vm$jtMo8W+OJ13K^m|=#vYbXD6;A)Z>2a%C;aN3VPsc zyBbdSco-0uh+2NCqHL@e9YfE6nPLFZDIpI)q&eMJ_FR{^gZpM%@_j!pBmI|$C&av# zXY+gO_3TOYszJ{BhDxySW?MO3lBsT)1szrGY~`fq~!n z<&^qb=_>zh@j|z#iPEKbAlkXgB*8X{)nSKp5gE%R?(Qz9r@(uv?JHiGgknUXp*XVI zEqKNwk3jA52_*CO#HVUEH;V3ueMTNue_K(+3L)47RtmrP)}+{1wVSwamE&F@SW6SN zDx2yGqC(n0#sc%66~Ke-p;kS4c6K-DyGj~Z`eF^0*&j(4tr01? z*>^3hh+mc9HUAtwV&s2kaPzcyF|e?KH^i-79I@?J@{hTp)ZM}N0i;(p$6abuW}|Wp zVr&AUcl$%W2J=V0g*#VXcX4SBO7k5n=_z6OdrdS~C(o$y%tEs&tVPXqR(LCNI`l6+ z7R)lW($U$bg_t;Y!K!;Se0QD$&;jplMn2)I^c*bPrf{Xq1Cshyc#%J`m|}WUb z@;`c%VXf?tDXE~;H9mv2>K29=bbfNX0W%rKCNzpP+qNEpvo5kM;frm#4Jz1jm z-Mdj@UescnWXpp+x)f1K==h33PqnA#2)qQ0OwrfoX#s#ZoZ`M*Ci)C?G^!79E&=uvLU%)OPZcEs+)fEMBUIUYCdO?mL#gRl2_5YR&3~txe#kZ4 z%T~NPK++S|_djF25|Ad54eQSGZ@NJ#!meu3FuvBH!F4+wu`)FCN5~Z5#@s<^?fUuV?uwh1^lx_p@L3u z&rC7!Vx1V6zoZ)EMzW(NtjSlU?a!EiCp-}f^}O%JUC$tL|G^6n`Xd@IOSbIIY<*5o z`mBK2pszu02smI0*#?AP^j(y^;uW0pdq1|QhOw8G(n!dgRH1cG| z`>lHcaUM2pK&R>NtyyaneR3mPBdXC}GdpeUjE(gY;S7Xz>qa;4_R`9eM&jr&lso|diB!}q^%z_`kK0QN z?u7;O+z+Y&Q8%q1a^E>WfP6eOtnuZseXJ z-555_zz1Q%tTI4$HorwPb7&@$kn{^vf-2+nz~vctfmj8!bO;ZJ-D1uIQn6$}$^w?Q zRIU@-C@0$mNA6p43KS6Gsn?VanV+Q z{gDxUat4{t- z;t1P(C#m+Qs7!AnOQl=bM>AugdF}+9vT%Zl-9>nDH%oh;5qW2q>`JrZ>%@yPF_l-o zx36Ub(i3=VQ^jF;wqw*1pl^0+PPqWls%y}}!nQCocqRf`>a2c1mKfpmkKy|y$KS^3 z#r&DPxR@V5RQvO7Mk*r=p{5kOHLx0k@2Fiwkd&G2o7uJAyx^OC1LTpd#nMa`?kEcW zaM#-8j6H9`)~gFOue{a70W;7k9HlJ*l=zgl{M1&_Es|6(P<* z??4kzYoyvUH+{_XhE14=Y|oDc{om~M=a(DnvMSgjAo$P zy^^$@&kNhO;%*U3sk0?pvvznw`eP$Ep8?+o{Iyv^b2|uwT8>-@X0?rXJ&y4@tgK3j zDvo;Or<RA0@?m4r1tII;1`nKr5($YBrY7Z%z3+YUu@*j!n@!M-N<~#h05Ulm-Hqx`zd_#@4B0~{pNZxi^t1#K{0|NoOy&DIQ3s-6~@`8p~ z@b`pk6blqbFpZA8y}Z=|#A7P}=Z)?={n=}?ZX<9~;E2)A)9GDyX6*+Y&Na@+%*VlD zkX2c~ypWLoGNH;yX9dBQt7YV)BqHu?&}ivNJVyDIT?*(0Fe5f(kX>xsor=X4exKfC ziQb@~`T7R%11Fsgiec}zGQv#Sbg{dz48Y`U*8gd63Os>xl;aHLm|fO*%$nv(rB(sB z-U00ICyFUNtsXPxf$oap!HY%t1({lYV`BJBr6hO-{1UCX;Nj01ECwj%p((9a>J)!Z23)>?%`x%8O`n<> zxdbV#-V!Oz)vef>@dtox;EqR|bobxdokQ+doNyMI!Sd3AGl{Rp;_cH-muj~-r5!a8 zpGpDle7Er#hGC~Iqoi>NwpdchH-)v@2qQRH1iy~&XPEP0C=;(Q>X5IOEMnKzdZO!v z9Wf2FC36T?+mEn{7=Dtb2Ev=tree0W)fI$?vx}AaAM@>ZCK`jFa=f-fm|9v{;f^|g zSGorcVkGN47w_}CQ%kqKiNR1X}@h5cAe%3x!aeHB^JKVm+$roU7C zb%RqMZT4V?n`wtx*i8$TjByui0I(osE{}_+C{7&v{0Uu!yGre?BV7rTL%p-UKC%4_N9JvoaTxd zKW$|Cn$DA-*C4W$t+Y!|_R!0u6P9tTj6aQ=yLSQHfo~5BhbS#9CwlUJPpQ}5DB@Cj z?r$HDeCayR*Whghs(~ZuyH^6@K2}`_vX%iax|s52 z-C6ue!h174`-L!sy7B%~xAnx_#d4FTqY22}zvuHF=q7j>F?`G#-qvwuWw86?{b3kR zCOBpTBrdk@8pQx)O&LNQ*XCuofd`#yTIvbbnd0;x58n?(Jd#~?P_q|*K5G!9(6KQo zgWV;!6i%1eZ**|e@krw2gezVw^qm6q)~@|g5l*eZLu^~ex03*$cY^+L&y~iD96d9 zj;aQJglBCt2AJT+^ygxG%FZhpal0tUv9MKaPfWXW@9QLffa4hrivY4ea1-HqnC;tt z6$JCTWvBT&D!fTaCW~#&Z?cdL94Sh`Z%j`9^GMFwWN%ufZ!8`DV6$AjxMLfLsO{R# z1zI*zH{@k?QnLhWn*L(s$96TAo$GnTSfI09PyE={R)i@Ah}c)AEW95p0^`z`@HUp> zgT(*Tmg!9v;Qq^wyKG{d9WY}MK}q*)7#>ca)&=7nw@j+jgl!I%3x_?awDhc|VdKjy ziZze--{n{kN1WVoD{{dpTmZi@XKxwtB65?jT8(2)YW18Gjb6rO#KkB$&owI@L%$P! zmx9lAvQ3G~LDYR0JfR^% zw4bnE&vO+f$|U&rH$(UlwPpTcg9RorlW?xVFZhzk5GVsa9IX?YFLP3o;A@u}`HK<#W=~>u0d_Q6AnYU;? zAg0TR%-vyMN`)TM0upgjQ7yr8=9(JD2@>hszjPDwRK?;Z|KY*Bt7T$}?%?ix9!>`Q zy0i6Cs(pk#p0Tn=W(znvUp>yQ02C@%0-ZQOKDVvzZ?;E-F z*GuqM)loqf&_7Mqx=$=g*&wd5oIwKKI?(5B*tQv)2cV=n=+rGu(%vg6aw3zxRv^^8L!tWo+{w`7tsrHKy(I?2CjS*6 zfSgV;weza84}Y>iGiKo&@?IR67|VerR&8)Pea?BG6fH$YK{p4rvhz=8@F$o{g&bOP zf@Dm9+?hNB#Lufgzg#RJc@NGbp7=BzdqZZpnH9rsyw~YKi#Kr9l@x8$CkZ;$^6gkP z0a%|(=n%bu7!Gkiuxn>`zF3gcTRLtcM}KrtgFk_stltR@<;f%PgsI+j>#-D)37A#Q zUdGCa_IcKxpVdgXeonP?*_79!`^YY$d|*u%{mMDhtqb%xW(-7oeeoWtPwqqc^#`peUV9`B!7RLs|NcOPZezOjcZhBE`^8G}=uhJI z1;Chvy8|?Lu>p?>WY$LO_kv($gXcc(*`D3v$&e=-4i<&2om0(%#WJ4DNDr^a-|F}jc59x8|L08%b>pCR8=ge`%N>K51UCPx4l>o(s_ zyz@kfA0dOwuiC;D8QRXD%y0g9T%P{WCsyg$de&>Y|mI_h??9#VNu*kk5Y<@!HOA0rV8_8WW6Azi&8qWh!Ur-&xK)BqHao++M5 z-utofZmeYGArkmmBO%u?c1w*Mvw|VEeFV~}a{$J_a(eGyXC$E4pi*e#oV1gF`JcLl zessHH!8dTe>FR7GM6|vQ7d6$|_0QvFod?a$o4xi&M;|1PI?qc9c~q-Oo{R`*+~RXn zygcTQ9PjCzj3R8j7$U5`96}NQa8u*_DIG3pGc&Wk*drm#6{yy^OO$F_N)!U&R}r;F z)}h|FP3G}0rYDqn_PsLqhl+;Dvk6Q#P&1Z5lIHV0)TgD9Odp2wJoVE&-VP$9oE)d;fD`v#z%ynx*f`ASC&)9 z5}$WZR=Fu3N+lO>XLVLIb%LZG|z%+;mb;U&6wNW@KkTYS#K%<(aIb3=mz zkGs1Fh+Vj?mgKwPOgUC~s(7@E8Fue+M4K_yvf{y1~R|?C_g#PD2%T$;BEWJ_moYOX|2{ zoJ_)Rc*QVlLVxcEa&?#?W8f9}=pxvCnD`v2dF&=ka#sf2tf9ksll@NARa8V|Y4Od* zY`d@B4N{3?qc5&-qm%%+XpCv79*`zXi7v zD8#=1ptf7OmY#Ac5_#A#QzU%pHw(`^wQBbG=gz8fHDI}|Udtom$(KfqN9dmq^#eVl zNas_Qo<0{#oSZ2)u@`A6-;i=?|9lYjqe=6Dpf!Ex^89XEZGqL3ZoxMeu(>E7xj{eo zAY53D^p)Am8Ai{&2fj~PnzOg=3xh!d-D+7t)2N@B{qQ%TfWXJcFK}ozTtn0L=JuZP zq9I97MpY040t_Y2>bv=9{)?zBJS$H$Tf>~_w_&4tD5DDbx!VPc>$zt!ez}9n z@BQA9>A+ZpJ~*!@0RuTKse< z1+lpuKb~b<2;TLcAU8NNdAf*{g#?swICsDTsd(RXD2ca|r=!$L<&qBWu_T^!m7x@J zpEXQ`ts~9ViUg4F&N?kdIhEKnn4(-=AcHp;N4-uOwb_ zY}9!8PO-QAiM82yR(#_n?W~My+cC%I!J!4fd#V8U?jSPdgoR?pnfpaa{4f zlhknLVx<==iX|vWDoaPpTTi52!aU^K)w(f0I41UOx1#vNV$O*Rmsko&oW{GX#p<~Q zSH!6meBeMIj&G6OD*-wPWw{;}&n#UR?hyBG()h7fjn+eytoiA3_&qA8y)usPYFj_% zI5ANfoervg&J=0hsgtc1ABIy_ToN@~2j});GajC8C8=S)DLU}nrS|nL!dVKI>*@Ap znq6+K2sFjnj1Q`mReCl^*_qeD%G}aJ!tJHH%~&rplz;C5c79P2WCV+52e+eWxJR3f zfJ?)2>P|qfC64NPWQ_%vV4ApAYE9PURxVbwkJP>~=Q-x5#(o6wa2Yq?POhb1^s9`_toHb)3tZjJnK^=nfJu-y7iOeECv^hRIa=hN2XW7kO zN{%`rWOZFpQ88fo^muD~F|SOSEPN4Ze$O)YvM3vMnY>JNx`q{fLJ$|R_0Uc1q zo#2NLMI?-A#jVyAp1D7<<-fQ2o_skiQ@j?=%yJoEz!y_Mnp$#>c$}Ktc=_bpjeYJI z=w9N;z|$#Um$3U#U;2NuvkNM>S>$(lQT=^Eb2Yw?UFyLz!)^B51=WQ323@Nuub5oo zUY#CDcZmx8qLH0>wnV&}%tOQO;vo+=tW=Qusi6p;pJh>ed(KjY7dOGPSrq@&!+kxY zf6AvT(OSp8bs<*6xgTQ2k0QuRj}J^~fe7MIGL~@FgGPzf7*N%*bVMZn+}3FY>27A> zTfbKmv=W}nnaYcAx!G%~AqeF3$@w5tXcJ+SKJo1R#Exq_HR==l-6vJyLtye2wZAVA z7pH;7A3HTqs|sb=g-wz#qXe+mWTJGm3|5HvFC-JgXE9n zBb|2&58ggREjaZU!7Inc%-U!OJVR zg`U*5x5rT>=&Aao);96tPkdTj=+@l1$yk2d_S3M1>NhybwxM~AsU)j8yZ35{g8#-- z179KSnLtOJzWor~Y7WLP_XH=jEi7OoANKWL33t}9S>2CP(DCt?^YA~PT?_FKK6oS$ z&g>Fk5ma|Mub5lI86%W*-7=w{B?-36Z3m+sx%fH*0 z(*h|xy5cjF3HBx6i%V0e!e+n~?(?q6*PeCa!@!Nt!477dO?q;YXybg~MrD(}g&o+< zUZh)BA#)@6leqV(4PSHS>ntF&08PdHm6(%@|H^fXut$>s!Vhc#v=+rIRK^emNP+?X zP-ikupnm>6t=O6`XoSclECw&I_Oj@&W7Ab8N`9)BLa4H`5nNUdJLbO@f}|Egx%Z+@ zoD}F{pE$DJD64T`s5D_7pG6*^JAXpZr6R4^E|J^?K6@woZ46!mMh|iWS~5D@)4xkr;dn)_4#J97+GcwNic}`|4?FX4D|2Zq=W~twV1Pvb<0@er=PUe{7<=Xf2 z^Mfl3w!lkh=?8|Odx@n&ref2U&7XQd8#32G)OmpXVj_ChYmU7Ue1rMDVah3Hgc+_Y zE0iETukE^6R}~rpHZK8hLr;+DfUudP}g2G?%Qrc>v4ZcOL5pQB(%zc2nAl+*qG$XOqGM>$mAouSQ* z5MK`U>7UQ+(-%qGUP`4)8Lp;}iKd?Bm86sw?chPOY=*~Bj}?K-hum&mL84eCel@5e zy$ytg&jrk6M$WLmwFoX~3=rQ>r>>f4X-E1TiU)^-JniwT{Z;dgv3u4)wS#Q9x?O$j z-5fb^*y&TywWVwITBLhn>!a182B_O8x^rk1M*Im_6lZ!MMG?msYkBFQ^zyd*%n?N8 z{6p()@P#q!GalXBe%Ys4A&-Q6-GKrgwhZkOJLi)}@HZG$T0inJMKHhJEMyobfjm@M{4R| zuTx2dGrGuQug9J$kLb2KA9on7a+uF}Kt_eh6NX$;9jkR}K>8-mG)qno6YU?hf}JRyb&lg6qeuIX>tKTjh$_{ z;EUVzEBl>tzYn`PyR1#`G6#SJN<1vN523iE_9@u`8u|hbN!@OtEejdZkiD+*`a3e} zcL(yJXDvx*=X7TY;f7ps=!Fi@b7L;Xf(f=M8Lw+Vg1GS{mBx;6AUzSf|JLLbG2-~h zgAj9z`B*e*mjMhPrpKiWh-6DjC4(WVJXzsylnj=(hrEdW4@39jFUXxg#qiCm&1*m9+gfrR;)!pK0oL@@H<5i0(pYLu^yyW5LDKIYu(3DrPSB2IqQ-SA<~1 z36M!c>4`j_(EJ?y%cz1n)?V zyCi_k$_GGx4}~P zy;mVSd1I%Dhf`VEH*O{~We{s>Q3dWFf+GB#0-HL=+^onAK{%2KXNY_3W(|}g=s4RF z+>gCP-T~($TylB_0s%4zq03FD{n4Yr{bTuS<|9=SpJzT=_H}f$a#IOAh`D}k^K+ta z{!uQbSXi4IQi_?IXl*g+J4`4b-fbm3R}WNoH=SaeN>mHf=H$^3gogNGRt)sh3&zfR z+q_nu;!oz4>nZ+dyAc8|;Y(l`=y>sjY|;%=sH<~U%eB^eZ01Fme)Ga8N)2ds{-Iqek@PSviYWf9`ML* zuuTxhyb^0RR}*MLGWxsF+i81x+Pg@^B0 zxADz*uBgaV;T~OhP>_*c!k(+5GQ-#;)s`i0hT{Tbx=9|3F6o&X5x53t%X7TBc)&t8 zm-+HxPe+^Ge1tb{h$cb25B?3W2NqvZuoao-$4g(Nqnce?Q z+n3)?S&}i_n(aQVJ{cWZ6^y({UO0jP)fU*CUR0{~>;>BZ0?9S86LQ0JGyoBv8~? z`z>jnv`p1e9;H9uFpO+5%ep*=Zo0`Q&7wER%c!qDBK9k9W9%rLULVvJd8}y{vL&7b z?YPxIR9mr`E*^8;R6TPW-Rq~g<2%gRcLeuff-|74k9b7EjQ~vq+W5Rdc1BoE%xN{; zTFC`Z5f6tq{rDPbtMmHKVrQ_X#WC&{v1(pL#%mbesy!$_&PEdb!a25c>X>`w-HF!J_5FtgKwP4SAI`xHmr*;-qvMulqp|beiHlkhh5TWnjbb>Z(xM z&fVx$rvXL#f}2Oal@um_Ii5WXa)=^ldoT|2b}<@OD1jI*O2>lkx9+0^rp4oGry)>D)@sWK`R{oA;W+bzU4= zmuN<8lE|774-fI%Q&Cn3DN)DKRN5~mOBNCZ0U)eh=?!UpG>-cN$05{i3*%D%$GfyI zsZr#*%>lp@6VUhXH1~f({xKA1w>vVbNj6(>?~VReU@obDN`Q# zRp%B50B+JPzPaGc6J%pJoUN>R?FQe?^{0T-zzcwUn1`qqYTk{bScMh7cVfy(G*ntV z^N*|T7souFHhUPWM-JGuHUgHr1~zw=h^*h!o&4ZuX;uJhDQ$jSDxaR%Hx#;;Ae|E7aDLD&%hFr)6Cnd5<@{F-kfE|yarYm_}2yHVQ|0LwU7lMN${$FTNA#37G!RJ{ey5f*9XX2 z5E!hBr!94xRq%1RDV!5LEXoA)%#u`8SULZg<-g(G5!Q1Tpk5l3ECdnnTtrwewV#d8 z?T^p=coXrh(f&S@$utX?n_$BgVxue1Qvv_i(B$IWjrT*rO1D@`4Q`Rrwv%$2vK!A_ zaBef4Cn-tAcl&=1UUU;3y$Q9Mek!#9TodDwXWjXT=VUQOT?>~u+PVn? zG2}A93W-Q2(gK94byy5@{1?&lsU7)0DSuD7C=~se{$JlF3;nX}d!g!mI@#~Ny3A>R zq1Lz_$V!obL@rS~%0+(1zXExlE;GY}S1oS4?Jcu&FLQhUqVDpZcx+w>9%ba7jQVRJ z2w*cVsNb*tL2VQ@bJrm%;gaFEJu4bNVr<1j)41)VX?`3dwW#Qvg?)rRgXbTnuGZY)e}-yB(R2AV(%_hwt&B5kN+#Z`ad)A|DVJC zf5eQb1=i{AUh<#0xw)xR0ZMdXYin!gB~nRjo*j3E5!}h3B@K`1ft8$T4UO?r&3!qi zSOFqiirkaO;tASM^a;%DR53}JRD5jfmQjCzy;cqnx;Zy{t%zET2l*E(BY$d3%4_#4 zW`kpN))7>69HP`(AkZQ;-DY59L41$FcgkNF%`hHnF)Ipo?&FgM(EZgcB(04Uh(D@) zXq&fZV>?tFxDNO}1KTkz?#Y6c(S^@uBC8lH(=@fdHsMRA3W-Fbi5e`Y1bbb6s*VZm zwbfH(u@*4S9WsgB-K{R3PsWh-vvf8dr9)H@3Hl*dZ0iz1;yyfU|m=y?H zoSg*&a~r=2j)<61JiU@7kexRkPNCMfBygC#_Y3E{LeZot(fb(Wtu;Z!B} zxF@MfV$TNIK*Z?sl6jVEHUh3ZqUAq3fQz#4A@FRH-*J(sQIE43ca~)FEiqHW0xtso zBmQ>JhS5cCEliLTcO}iTQN$UTAjgNazTq;LFS-23X$BNQ)C6WgUE~t+{F)P0nrY?z z>@nr2@~2`nS=Y_NVF1&e9qBU%1s%Wy#vzPnOD%5;A zXH|c_Jh@`?s&Y)#{uwa(pv%!4S(n_+jhaZhYiF2B&G+z6cvLyR;s;~PzRCj7Mf6|| zXipnNj4H1>C0m=Y0iK>nbW?cLAbu=)q?O5rrT6nI>_y=G?A$C!rsBbK@m=WQbbdSL+=7%SYSAh`}9R38>rbO~y&m8-m%OCv@Fjm9%e>^{9)V93@_IeA8lVWQuS8my4_$PvHO zvoV_s8{Gzp%_CK|!?vgyU1t=LD!}t=w_YsL?}S{%Fwwf%NL0j)p$dHATEJz=r_({j zZhE)@z))@0fX$YeZCw&peg)nOrUQP2dDO6*grUh=%I0@JU7{~}i)tJjWTT3y5Vrw? zx5t|RPHs=%-h~DyF+o>ou}| z1-Lm>chWQl0PA8`{F@1rbcIojq}sJ*u=cZ(wUSDLh4P-xos~7Q_&67a-36wdf2LcKVT#7j^l7I*Jb1fa@0PR={GRVn$wF9=<=*fCFx0cl*VnCQ6O}Jz&DZoT&7lR>t_0 z$kAslJ--g7(t2J}OwVo#p+`3kmBsz<(=4SJKF#n7AVf14AR`5y(P&v!!|sqSw~H$) z?H{SAr$|Og6fjaT@4vodA71XRXRX(4D$3aM{6EuXDWEJTKh=whnKs{By?1z}i6SI* zX&&R%fqATmtoJEl7UKbl0`5E@fcvbSDy?8F*hatw;)SHAF5JAk_y3MG+d4n&%N2N$ zz%MxydSFGzl-7YGsco!fQ{n7C`F>2rrlJzWTnUU75byf0nJhfZ6pX5W<8)ulP)Y^o zr%sclkQ+v=MN5G6Aax7Edkki)0u^F`S-F$>KeGfHcERPpqsm9Y_koK9c+k+`LDSi^9~=OB0XA)%TM1L~ zHiR|nj6{mN!gs3JGr9r(AX!QsrThm_d}fzjJs^MU2E=tvshKmk2F(HUs2r6c{R0(P z5S`XLB71RP1h!FB@3)&*?}S~*nw`2cz~vhx-(uG2WN8u6UmCX;x{#R9JHUJKnJVBr z)Ac&s$-a8c2LL1)Z=Vvb*ZF&Tn=WmD=t+KkdRD@)LdY;_+DX04AOmMYpuyNy>S@!U zqsMCpGuZiFLz}^TUZI|EFd%z#JCTi7VQWCtI>NZh#5>Tu<$YJJz)u5}Joy`10dS4| z{UNc1<^s~%_$naAb_3qlV=%#mK0D9hl5h|B167g3bTAe#has2~3eXAnRu^p7tJfi+ zu3Wq*|8WhLX~OBXxtV4L%qg`yks%Nw;SLgg6r}b(s;H z+`hxNypIu9T%~yd8i)a>57+~QivHamkUnc?wod58WAEd6QHVVR^mkSpYMC`U9-Saq z^PzDxBnZ~2`ya!B4+b~^!CZJR7Ely)69VMM%7~_b^q)^yz(Hfd<8x-{)4=;#xk-WM zl_?&TV_k#$r>%B?*us{kwd@*J`EU#V$qw^*z2s8uaYBE3m@h4i0tjg0Gg{N_;f1V$ zha;;ggL}i7`I4Bc^F|foegPj41*sJZU|+4BlfSgw;W43OhdQV0uf1@)U(D%JbrkTe zYc2yImp-vckJe>CYz|XK$ewGw1SBGEE_t+)-A&l5eJMAD+V5~x?t+O{wG;` z1+^oKtN0awpj(Uh9g3DsM0nAYGS0nF%vR^6y)}r@I7No+mc!=Ase-PD8Ay;hWi#>O zrWWYZ&UXEpMg{U^2ubCUW*E%q1_U^+BP4mf&KFwOZq$Z4itk6@J%zn?Cl%M0x;wvO zY`g&{iOZ-HF;MXOF6ChRg^&4xPe47lR}fi{&l2w?;YMe|VrR3a2+(QP^KfC{CBG?c ze3`nkz?_KUQ==)iOnLXiM%Vl%#}K|O zuB1S9qYEAD@MG=Rv)^U*r7SW2)wO^)+R6Aoad1GK?w?DYDtNvD&Lm*TTlX>CH4~oy ziN9(_mk9tBH1~WS3#~ALt?j*bON$}SgJMV1z&X-7B_&S#RcGcm?()g3|LEc`o#R?o zcKnSl$)QxAqor#jOc76pUOT?p6;zD`M|tRm0O{~#fG`p;O|pn2XCe?U7Ki7mOo9VX zd0y~C*8P{~L*4d|dDClO$EK|uY|ZZBi6bVRYle|l0Wq49hc&HQ^Sj_~t8>}aup5Wn z?e*uaQY*6{qA#6Q4jq(t4{DX~xxDk98wE1-y!^5W^nQ@WmBeoNwcI)KwI$CsX(>awyT;MCIkq8n}+&fJ`>vFH1n0rGNYjh zA~p{Z_rL*su?D;}Gb+5%@gqTR?0!KU0JhZ>kTJH3)sQ$RL=(F!&qHAAaD54y`}1?%XpT!o7}!0bbwXKbz!AkHO99*3FGHUG`>kP7>ep zgv~+6`2k}6v2L)&Il=nEUo7{g3Mqjr#W0i0-16d7FgCeX4>ADqJ_TQKW-ho83uebL z%`$$~hT~4h25fPZ-XK#5ITUGl5oO_Gv9(svn#2fTNo$ouF9V!cDwXQC;cFcyYbiGbB{rY_ zLf1kI-Fv&g8@Ue^A};m#<+-24B1U;0P{~Xl98d^Ui}(R$>6(R>Dk3Yr3L0pVaHE$( z*)L{GC)RHG_?W}+1UKNwTU%tw-Lwl8s-UExxjSab!O1bUKH@9OHS(~$q~@PoqW?3; zA)|x*mkk2C0jXe)=N)Sax+%f!>l~_Bo%N+1j5>P-AIf|0 z37i}yjs_v2MVpZ%fDXCwtve@r9KZ7zwz}z75Iqrw zMHdL#2V=kFcErS4l7G7*v8vJGYwac8sp8DfAOSP>D*?xkmVd#aokj>VOH!O0pgzAI zp>!v1j4|uH!Jj-2F0jm(K8e>2`qN%d3)(wxyIo!Bel#j&}d`r!0g(7Z$Ek>7x^IbBch^{La4o$-TsaH zAUt)^I5{tJVJ>oBygK5_UTDI0{rCPuWnwiOqi_+@Ed*%y@i-r6GlvCw4Ls5T0cX}3 zed!3S7_HXIkGPITO=yHGZ|4z9Vuna1WRa=0Zg%+ZV1ONzPhDI z9zM4BF4HpPwU)stSp`iEe=F8r9=WIJQvo+u|0~fU^*cz|NoQVOue1qckdb+IAn|=t zUy51)9#}cBl{Ic@`g7iw|FQ^X(o(u$CSKtR6Pk;jk>bS1(?1N|WH z(oeeueSfJz7dXYJtyYQ2+4#xL0Am??|!`c9B8(MM?#4WTB@E0{eaVgA}N(oBqE z@Ny?BNkrN@bD{+bG*IZMcrq0qT6VZx-cJGYlVK5r@~4uY{r&5C5855%YGAsO<5KJ| z+{w8T@3O`oI6e{2G{>S*;U5vdW|c0iCa!q_OdMR5aip~|Zv6-lQ?Y=pR(On{D|OtE zZWkGLis~VG0;drpvrx+Fb5JW=?wO(^e~(d0t(Ur|xu<bh6VYS#VF1GW5K&0SE_?aeZd}qKo@vS!bzCBORVjk-`)@XS^)iMuIhTW477N?zofb13fL1%b%5t5W2D3j#8yQBG7zzsx$Z;H3(C?z3 z-&1ugEjQwJtMoTOg@y*g&8@rei^9CJlL0AevFSbnQ$~og{J?z1kB!n&F<~4+2#)83vU@jWpj!)~Oy&V}*}#^3XG=8gI6Af^wmai%wAZDw|LK=dU)biDyoUe{lME2f z1nXDi;Ef0AV9G{CTzPo%d`-dUqCs6*cF7Uh`>(P6@^MeYB${E z^nNwuR$-yQar(N z3VckLL3r>hW#`eW4D=XigHQ{fQ8VW?9_B%wQKTS^T!VI*D8H_xS zDs%*X(P)A@kNojtvz8$D^NbWXo~of3Ew|Y!?wJ*3=1x!z`<)hr{d9D<6Tg1E2+&HY zXXW$L4M5J%_FK}R(w_9ndn{10Xa&`bjJ2Pi=OoxKGd@A0?Krq9=qDx4kySa(_h<|) zD*~DsOkD?pD)W4_=D2c@(D-7;ImZt*rReU*tWK?t~MrlUL#ecWq4`MWYnejW4 zNM7rY!+cEDC(Dbh&_uVEFrpldn@Oos3)Y+;=?si=c)A&8j?9WdX=Y;F z+r17?w~232|0>O$*GP_P&xYBj$@I3+a+8ZCwkmejId`EI z3WUhSaN;mq#`w^YzdVGk;^Q^5P-w>cRuXkH{Fia@4sIZW4ed zFV%c(QNg7l0F`zM=V z`}Yr&Aau9|`^?`N|Yy$v27$!0r&yHU!rbC#ZO%so4HSJ#(h3+`ZLR(k%E z7IT?P;x@>01*_=LArjV1me(QkUBS(Z&7HZihY5u?dMv^fh12{Ag?%O^QFeA^ql~$j zytfC4ZG<)DfT{%{b8MP=+CvYzi3{6SOe{Py6*oneid8U;|9}ZGn+aEPrz)K7{K}{P%$eo4pG&U1|5Gv<)M~|C6-blsy1v5w zOfNY~cc#OKv9YDhJJ0_(F%B-`BK9j%2k*Ndr4?J+P-LXDn>8h4MHti^LZheNZVl!V zMvV_$pMJw9wKmKc@p@SvwFm{tIqaRJbL;nuLo9isBBH$&T|%p~PK+npJ6WgN<% zhmpGmKtCXf!2zj?p`Y)Y_^&R_%D2@1d>b?6!UW@QlvxyD40<@1m39TpFK@jJ9SJIX#LZ$29popdQl-}7F+CNy$LHziTA0PbPaE} zK5TJf7bFHNhY==kJ##B6w`wfKe%o^{pZ_8hWAd12NrweMSk?&z-PU`Fu8I|JW}K($ zWh}b@E#%lPpJ0xv;aLxFJmuGBehBJtNXLaRE1qf0e88>ztWI#YATz9%JOUjRUM(=J zs{@fYwS}*^1Q&#`3zu#Ums?iaSb;z^frhf(AD?-+yoo$_c9s_v%D0Ef9>_xs0fhNb zJU``loFqYvR4l=|!;Y`G=%-!A72)v}qsN0ayjnsCV|QJiZBy6k>sL(cOkth=dLj5v zs0t^`;M_g>T?n|L5Qz< zEFB2)lEA-4gWEaLg}$rj3mL83u=)9>JVM==iQ*6v*&8qKVGrFry!fSF^Qg6}!P#7{ z>Ju)FaaFoi`C^o~=B2z&t2(+54{4q(03M8?vte_(+_ z9ICigoHRx87aCowAG?uV8yq|4SxI}-*K95=U=k~`>Lw}&fcK1(rVVT7J;ZuLq4!>J zBHYHNFN;Le%vHw(f_#s{kL0^Nh{m1gC^7GaL>-R$`^`{&H>U|9hMN!l+kQRMwYm&c z9e=0}G{qN#9G4wLhjPP@=QnkQ(M0Xw*acN)WdllnEwTs*pKN9~zC_qJuSSs4v8aK= ztoWLUH={xE;XmTXR!-+*gKa86j(P##%Xs}N0zs9bKs3t8-8qCeJB<1*UgVeLtb_QR zCy$~pL_KI!vN2S7V>Ihp@p{gjkJL6Blv(~WDA0j$XY{H!y|9-bR-*`6ll`EaMVLp0 zrwG_kRYtt}n}c6N|2|3k(b~Z%*StV4zfu**j;Oa$c1X`_s1Pw9*_A(hm+*Q0=UG#^ z^(m?~PN(XZxchQVzzf8zLzVqbT&2#ulKg+93sw*EIy32LJ}N@W(e`K4VY*J*Ej~aZ zgu}ag3$2%o^XkOs6wysKgswPL;&sw-7w{M@7I@4Z)!2@PM&kgldtPnjJVzY60L6VS z;dKc}3o04Dk&sSoYQCe-^0m_8QAKJ0F_K0(xVv82368b`|sw{Z#-07T5Lh&9N;s zGUSVWD0a1hzJ$ur4yb1_WJiChVCj~O;45ifT&TopA}SRuHhzpwymgu9p`diF{{Acz zEyesxJN?x?Qhu)9*X!TONNaNXlLb$9H}7JBK8M#%0%a+g72Vul?X5HW8a@r5% zMl_`>ef-K}<{B_K;!z10WOK;N%Eu~|cozr*`a!27!e=RHsE+iVu?AayW()b|J2aK1 z8{vfUAk_F^C&P4S6l*sh@hg@zhgo3nuQ=n-KVF%WR^vj=eHrCgbo8PIhlAd%rvAxy z+Ae17HlY6W`9~$ALil-6q+*SYE*Um*U&B#NxT&GBXFj{xo_DGmkfohlUH<}mBa}TA z@02A{`eo^KJwirG*>TF`iUS8-g+k48hwz*McKgSyd+4|en7jG;0&E&mV<`X6rbjZ_ zAalwYwuJnEvrXUw8LR-22GE+r9rc z&h(~pK@HN&8JpOI5HaDguwkPUps(dvIGPrq-)J!!{!zP6Qvv}i zqfH%mL?d;if$9gdWD5twv2$_8EN(cE1UeTKLlB%`aa=T-X|>1R$wF0oNt$4#xIiv@TKz+r91zj4<2 zmR0+jaE2s02Iz5m8P{1jH7D5&a8yc<`g=Fy_w^X`4rC|OI#s9N7U>KZw1awXkUnq{)7gv0od)YTl zQ7rp?Jh&k2+5Eg+u=jpFDvJ8fvcZ=^o|2Ao&xWyfDmCGwR;H!XQnPYf;g}M<*Oc7xS+}`DzWC+Nq zPUZ^uARMachtBvgNh~V1-J4%SW8+=Lj zUHzC>+<=%cT>NRoVPPr>d8HL=)ZnWx-Mo1zsx}`jSV}bvF*Zg%vykt%Z8X=TGeZ5T z#3et>H7HmU%M`E00?i-LJ&TYigV6O-PNNKSr^@IZRJj!WJ>tUSaL0J536!JH|G4eR~Qq$-nMi}{?J}ik$*>- z!FnU=iM#)sCS=odfWSC4be)@ZF3xb>oZq%*!?9DbjKX1zqf^pHztS zY63JmNV5-LR6yXgtMp0-rh&+aW|Yat&+KK%dSU2A(WDY25bd;dU;l2X(rvc5oucSe zr}z>yhH{v%4I?b#bAcj&jMLIGlZ(G;&iPWaETjK*L>gFE5okF)^^fqXInsCR)#hT0 z|98%j^I($IK#FoXlJOS&y%>O9+{?+fL`Ynd%C7@j!A5wTX1Sih7)uSQn>At$o})T5 zsJ~al=*K#u^bH^mZ(Ikk(eBEuM|5l62F{oxmEGRtXbD5)N^^{ zi7AUSV%GkylT3Nb^HlqP(WGY8~XjpeC7sqckX1_N_r8}Z)bCmURqun8KeXH zwfhN0f3>y)HR;n)TyNIkFrf3WuC^jhvf*K@xTuy09NLRi1@7pC(gen!<)tiGJqSK1 zuBlvid;!K?yv25)6ej}BWGt}>twGleJyLnTW~EBHOPXnh3PnRPes<9=Q!lw6Hx0@&XY_pk z{ZvASg&frqKZP{OC7Vyq-E+f;Z<~d>xz$D%=psMO6r4s@OA}3eBeuQz>Kl+%&vT%2 z$1|>GA%9SZUH-L?1&7lSTq;4j_Bj@nbKBrN{Z`FWp1OBVYB?_VDYF3Q%s zUZj>@=}TSFjizn<`{?e2nz9q5k-X5Am%-zxj{{7^XP%ZrsU!3WLvEF}Q&5V_>UtUj zy3L3>ch0C@cq$yXHY6JRy6v+T_l6fF9ySUX?tO#}H1&B{9voz|A%Q_N3XD3LAjzx7HX z>|_^q)BHo_-;vlNMS81xQ!C|m+aem?G&to-38MCVs8(uBnP_piDH=KdBkS=;Ox61ift>0K?6diMckspWluaXEUBIr?UYFV`$C}NtD1D##i18$<$SX>AEiv` z!-aJ=aw4t;Z`P_D&!y#A20jmj1k<*Z9ZNrYWRzp|?j7Z9@dAvNyCo3f#rk&$s=Rv# zeYUTM=zVhQ1$1stLARi`iFHM>@QuN*)<5*3PgCD;ea4NkU2MFPyzUs7ck6kgW99PN zVMZi#=2W=I^tJEFPa^sQBgO}#j|Xof#SUorZyj{(W-Nx-)*aZGaAD;AJD}dT>6eFg z)!KyMBmFE)-RKb$t(op&3b>2$uCN=E{9jJzZ!`# zi;13@2g98dA0u&us8kzX_)CEgf$blhD*Ke7X?XvW^LD>j^PRGZ3{F4qryM{oa9kMs zsKWI#oBs4deBDfwx2=K;uB8%tSR(It3!CiR`mf^7RtnAx!z1awa(-;Ze8m#Njb=_( zGv|Z!-}q!qOz?L7Q`}AgPcPYq6olg=;Yy;ki zG4tE#)}`^yropjdsx-~eH(_cwg?|dPQjWXP3|0S%Ig*$5EuX6+>!c_eg4Jr*_@247 z>t5uH89_ag#LTYu{L?I;bBhnrD?L%XGx*+I&zzX8WO5fxEPuaQk$h+IkaLWj=_IYD zR^7z(dJ|;_MR(5OjWvyTcC|!*2YvN2HCE+l=$y8ETXH?{WtFb^BVkbDDcDy!J^dAyQ93J{OTpi^W8dlb>F5N{7BZ#=#Vxq4w$8oRGl0Hed+<)qG!w45&UpJU#pQc|Tdwo@FZIA< z9SZy$!cEHgwovuv@ovqXlC)z_=(I9fsx`1^#P{Sy;2db-p|#Q*>&;VNWS1W4p9*KS zz)v|cU58#vh)`$1~)Z_(Io>$6VmTlP ztJ+p>T98h@^yhQw75@oS)_dP-#woK(cPh3n@*c1ERcwVM2jfELRg|h-!|wl4L04O! z&Syz?;NI4MC|odhoN!vcM}U0@B4!u&)r=Q$G{_zp(1;wgcMJ;?d`~g zrg#_ipvT($Gx*5%y%MY9^6BF3b`i~s7TM*CqhWhNlt!brpMQ48Ypp+D%LH0J1%JQp zR)53Dmm^a!(t7Q}x*KmzeESN_x)K+3A*yDK%~>HFuYjes6-Q)Odds-j3W|_SyUT~| ztLSjvNwDzBz_kzkU@-aTw#bZZM@O>bOC+taIS+=jBxjZ=5~gx^(>%;+4f>bhw`U6< zKC?jdilz88xl)5sZgC-oaE)7(1oL1UAts?F?teefsOX0K?;hFRa|t0IMur~F0lJI) zg6sE3!%4?}BASnx+BS_fqWLxo_Nbkwrd5nYY^-d(P!TMhQsu47{r;rjHlVqHdF5CkoWO2`Vo{zwIvjR5p zJ*G=IHO?&y&fO7Bt4hO(gaFN^=tAMU*i1Q@8P-0Hh(>3AHFJSyHsZnGGscWZ%#6&Zvm+icMuSI%LFR81}jWnXO zHtEv;-f3*QJ#Cz^6miiQIbHdJ)wZV0$LJ1qzQpvQz=ail=0+8d$j z$afpC{ec2Ey7oxhtcIUPLb_(HN{@_&C&1tPFdbncKL`4<_H|gc+q+GgG2`Ex!m+`akMxSn8-3CFayQxN|5+$uqwiiL zl)}!^VJ>R3bmE>|!yHXQx55@pxyEhJH|ed{1YDy`Q*k7&cTN^+IYKI?tlt{wk1uvV z*%p#ndz+h)g?Yz2&qC`l4FXrZ7C}u-(C^NQ@O|&w2f~~{@9XD*n$3y42d4pf!|$5c zh~i=lu57s@KPjRMsuO!3?2SdkNMoTMDH81Im9&A+BHV=x(^1*rtA}nLUr!|u6WP2Y z@I5fsP(d&L8?IhA(1($8-AWSy79kH%?e_0H*ckgxysPjql}_mHCxMYo*FTNb>Dad5 zsp4`-#+amaMp;ybaRr07Ee)6MxHrZ6NX!7@u`LeRG}xD7ZGZEq29 zCCjqSufC!D_U)c!yQTqDcw)CylqlT?Nb{cls+Fk}e&*SnYIm~t4@yE3W{+P^ASB={ zw9k|x-@Jd?D+728y;|Kf$er&751q~}j(vqsLA71SX1?WsBd2jNaN!+FCd}r@cI|54 zNW&hTx;m6Amdv?aBv#M*@mQd5)7#mFltIU*pX&%apkyWD^BIy7Dr)@*pJz&_6m3F#h4@>>-}1Mxyw36UK6F*(MgI)c!hYY#zZw3zZ-i=*tew33psD56>sddY zV}T;eodch}zbjVZA$=hQm0SLXwHg-YuRNsB&yp`(y%*K$9>4WMuo&*@5n3(|s%xt? z2&>4O%Xx5MZN%TY`nuIwhTkc)Yq-l#?Wl#0roWry zGrYP=v7TLSI*}7_Ax~cKSUSJFrMg^=dp!efPMi2imJ@YbdZ%QA1QqU}Nm>@M7YoK4 z%RzI2R8ymbJ5-JQUc@WI*9YjSMShKa+py%M5wrUHKN-f#oYR}z=jXa?IA-&2f+ZZw zLC=euHWDKb(yapDz5eJi^;5X>{9u*P(O`k~WLAINp^=30eQAI~Uy03xsT!U<0aT_ufiMm670m3elWuFNDu-Rq-vUyNa4;rW6V)s!$91i6z+ zH6_&jLRtj&sDM7pakmYuK!-AU)N_)Z1Fe{)b*O-)3uLMlHxPWg$3ypq&vi4eR{q`8 zvZP@tyXvPZ>K;G@t8~kCPx_9L4Slz9d#-%*)}mF5_9@Ek8>e$l{|`@J9thR@{XceP zt9KtHSz48K-X(iPX|ZSD#!}f0*|U>SsVG^p7TMP@gTaib#MlSJ3??xo#$+&KFy?nh z@9*#SXV=_&pXWU1Ip=j=uk%!#m<}u%c4~K){mAB{FAcpxF#wL@+(hFI=dj~r<-!uL32K)E2{R0fWjc0_Zsp%*T)%{%2?w}516h$7G zUkGEuZH(m+Y%iC;`Jj}!#!>qSUOZ$~h3}|)8@-eXW3x46_c~*iG`!cs4ziU+9kzc8 z3_I@!w#JXES3d6x{di32QnnfeeX}|IEPB2bwQBWKV~X!t2aYwH3hh+W2&-0=julcz zrPtkjImx+5*j?~q#F6X+n(!NN%yKzXnLnBGyHk8woVDp_`;%T^EAK-t2G z3*5kffCu&g0r&aTYq;Y94Z^?7lSWQfbC=rLi5Y7ia#2~(eE@~Hh%AsCQ5$S0L>GYH zdVA!yc>G?aBo98`mB{0|suX8;AH@R}rx+af7%G5*O07tb-t+wn%G}>?_Gd}A&Zeob z!;V!VirJH1Q>Cb)olSDIAZjEqG7QDg5fLySB|6XG7bv6x>PY~^kLiGxfpgSqb~)KH zhQl-6S>Qr8cmJEcAC1(1IyWh8$Fr|2hsDr!Kn&>h-J5_mK?|7QCcha3vI@m0XHO#b zU30-@Kt#EO-GGl2al=peM}b7=fEQOj3cPr*n`!0S#kgbO>IOrhT_M)3%~NDHBh{G@ z_QoP>fdQmYc1T1rpiNmm1-v(O{OZYIcJAJ`9j!u~uBE`H4<082tUl1wdjF&N+HhUc!@-aQOJg>RMhfi6YI?0Qrs>Uj_5^ z%|FJqgO=W2MSB8b)6Rv~BG(6dPuVkz+an@oG~V5FL5%C|nhdiUq1yG*9&&>RRtccY z2G)!(OAmnc+-xjkJlP=T;4`R8!>n&4YAYrlT#jGNo8y!K%UJ?K8HMl*VlMU>yK18MLCWJ`*3qEvI7h!dk&})m zK71?)%pNG%GF^CqCtDy!oTKqHIY|-aFU8y6K+#iX1Ehi3fJW*}nv`u|+M(mtF+cP( zq+K$3=h*uN$K&Gd-ZTeckVE@I*=~a@QP(DGMYMM%Cm40spM1jReg`;}1!n@nm>v}v6E%%P9BN7^u)49PmhaoWUIK6XS=dR^qonxav>D|C3PdbBuP#)FD1pWYUE2J4c2h~sz<}7K2vi1x; z`#+HTos&!>3W2HT)p=o(2C8@550yX1@-#yCG$uEJ6V#}TKkQGVyG>(vEOhx@N*Cf8 zXSsrwmX=^ypwinr3MV=LjkCMElBD>!71!1&Z_uN&(c{I;Rs0=I3qb4s6}q$^chu z>#h0P%;yVi00p&&%(RYW7DExxx+RPsAA1IRIFzDQcrzjqc)1a>nhDDk_2;#yl92F6YJ0yjY0vZd^KHNBFAD{jS{}KO`@1ECQr?E=sOq_ zkL}2VlbOGe%y3QB8B@|s?T~1vaxX2#ZSu4~`5Smy&9#>R2;pRw$+1X&;v<;XBKmEr z>aWH!hR!~LxheLH>g>(8*nDXh~q@d2Z;VV|i2-avU4AFOWv}2h(3Sru)zzA6Q*en6Q z?9JTWZypGmT(Y4shIH;j@IKklL*QOYbQrQuo|Z}^%zj~w1q}FyX-VH?4E0>Z!tx5u z0GDC}$PABOjOAR?+0oNjw%>hS;8e$WxVjUci<;dkR+I7{W{azpViJRhsiS)V`1r?x zGou?`yKi;ceRLk-;jYjYT^>De^fXm_tkl9U1qb)awIR7U;y3$3!*Oz>EBV8x>BqD@;?vIkQFR z$jhW%O05QFUN_cfk3plwPBZth+Sg%2Kx-Vlc92(opPTn?_1g{Tu5V^{(}3wNYS|G5 z->Ojvg5s88x@h1O&=*R;&{Rs;^3qeA_HJ1_)grXYhpE?*d4unzzQUyHi48=G)ErR@ zJ?G5>LJ00OF>WiC*=4TRN_>|s!>f=2Be8<-$CA+93B>pkTznob{y8Q3gfdx|hYu&& zd=)G5*ca~v8zQ?&@L2X?+XN9!5{~83)*H(cnPJ(xuyCxPJkh%_b#Ft>(;+x|7EMa` zBR)gmg4s%qam3Uv^QsHI9`;kx!& zwQksbR3{C!|L)|bI@G~A&TEvaDo?mILB;Ox@9yeU#4<5^-t)xy<{9jM;VW{iwo!Qd zWoB(zfVc2ZAdFa-VTBPqXDJ-)5FHVWa-Wn3-G{CQOSlB#c}!XNvAGA6P`;3*u7q)N z$c&nVb6kesdCvHDA54GqEqtI*jz7u8 z0O}x!8-#J3gN08dT}&RpRzUZnPQZq2)yX(5ZiVak&+jM(-MWPxfjjg|S!g@Sv|e&p zvRz=QZ?}V7M$&|{Hz1ELff8HpWst@el@`{O*mO`m8q8J9cHut3{y!4-0K%?I@ojW8{yMirT0Q1P}5wM>J zvdR}bM{P8}Tkc;-N#I=OakdlDZi0Na++D|0?eRkNz<2nVBUwC;y?K0bl3|P=brO#} zcjsn9rHdVFYdIjWIlQ1UtR@W4_7vVjXA&<7*_YxzS39aR;Iu1+F+1(!Nxf1+8pCF} zH?eS|6?4f4CzY9$MsrE!-e#zFO)XhhI~BLo5&FI%?ay;}U0Q zhIhsa8ZD7zSszt627D69A!c!Ovv1C4Wnv2l-dMv)!pORAjA8_zVS(xA5C6i-GOt@>+*PvYT;t?;LAG!IF)Je&{AF2${0N61L?gL6ydd9D@7w}`RJta)HqC2 zs0Rd#fw>nGZfNx{?i*Y7d1$PMxr3@zLtmuv_k%ys9tdFWGJ{CW-w{~t6ylDv zN(qU%$BeSOPsl<6vh{;~LN?a-o6z^|&mqJNo1M>G^3M~pnSX?s!z+WTlRBWrB9hCN z`0O8U(z&>cZ<@n49@^XX-f@k)^M>6sJhHj!jisW{PHLxcSEYajvz6A8@-sb#spVGneBCkJxG^LL%&p%UO-T4(u#W?^=>=pc zgRF~Rsy_*L9CVc>!;J5n1Kxb|`jo{D=y6uk#mW%qXNEtAzBi6JQnw;~v9qbKyM-DH z!u0gsa+x5;ztei~20^aodh5*%%i`gR?|V2@k9JBjF>g^cMyad@5-}#JXg8<9CiAb= z!6<%H0sD!la7w_&GSRh!(qd8}*JORUU)cw3@3bBCFX~g0z}4|gsQQKkHK@apzT~1l z|F&@1H5oGa5Joq`(@Mt_B}5v!SUxa9IyV0J^Gxw@~ygNah8dZu`#EXQTHuhs2c z6Yq%?cQCck6#Ls%Q2h8e2}`ajY82ECQLX&ZI%$J@VE5i41`fqAbmzN2~ zAF{qfwof+MOxuwj6^?Q&ytA_}13L_bV1TrY70Ex%&9cJF*988OIODC!tDBc>c$cj-m? zzUg>l9|>%vRyK`XhS@(Xhwno^RM|hQgyt27S>Bi9a7>sWH=JUB((TKHuT#Q5?n2O3 zyGZ^GNrA=z>roMww@(sl;!i;Or0+45${I%*tkxH^KkK^sBi6mN z5X1<#iv>rOtm@U*q@V-wOJrkG^o1TDhrvKe=tgZv8C178upJi>%wQ<=R3&C3nvC1> z+@S?9pjcZ>Y6AurUm2|60r4(e#_jddE(pC9IQ(>cBIpfZk`Ez;>i=bR@q53e<6MCb zJdN*trM!TK7bxA&pg1oT-{a*fM%eklJh-d{WT<35%Pma?0&d7-gyQRAL~lM#fx|N6 z1^#>$Aq>Km4jefnFb@}6#TAKLyqoKHnEX*X44MtR925m#{#tpzwgs;mBsvxw`!?CUl&^qUSa8X9ti60E-HtcCOqxl8&Kr%= z07Nib`_-jw;+$1t4icn|K~}qvp-smTD2eBw2kSSYVE2Xj?V) z!%v>!lIr8HnL^-~eMeA0?}=cvQs;NMPS$-OupM9-H_AC}6?gf7g>DrDJcAyi4XH1Y z0}v1(KI=UL#AnOiWXp5LZptpl!$ogaaISIPq2{0Zlz&;b{)OCbvmN8X#p_ZkpMN6THV^up%yzo?sOlvrD{WVN^7`OM&!?X4 z=KaXA>PCwcY~p};3y-V<>9-Z%naL$Kqvi$;AQ1tsbfA^W4T|h7Q1|yL&MCmN)%wbn zRHi*X48q9yuXvLWhV5V@x@{A^Bx5Pz%aiol=1>CxR0?r2wx;ES8jGp9VyvHv>t#V- zZm`r%)UoPoSxtqvcV9Mo4B`BN9xA)MF7OIgpQrym_Pti?Z)r^cO=~EsJlff0af-)9 zrG)T(|KN_l+x@eI-vBtU!uM8fSTOim^vw$#H2~Nwbe6a5;Pd_lil=0(;)%l{5@170 zv{U8zDgemb4Eij>dQRs85J<*1AYKE2Sv?DGw6MO-;SxdoJWq*@gFPT{fFoCT0hA)- z04uq41x4l&0>7}lavNxm6#A)7Gm`2{xWB6%w`_&)!IL?P|0Jole-6-y6Z>a(Z;}Qy z9jaP-p7h$lCa!9SP5#!Z643p&Ox)DP`%ocUf9k}K8&fQ?hi zJd~m|DedXmBn1b*Wz5&PVw~TG!ZQR6>y=^16>;FI)T>vC8 z&;H2&mNy%e@R4y#(u5=XCV&M|!(nh%qe0*T#{jn8*=zaZ9#K~y&dZY9f1Vl4;i3V6 zkks9SJ6PE24a%`f_H&)!QRHco&Z=e{0K4A=kelq7@N+-*0Ph4`2jEQx4<14n2=2XJ z4NDIJVT_vd0Ys~L^WcVqx9R)sDOk%xp?&h^W{I(Sh zT}5Cyu-qup1kC1w$Z%DLLRdHaUAQpC+rftH{AUQJA9tz1QM^V8dtE7e>K~N<-f|kS zVp;U5g zi+ae;t$_&zvKQPRRU3cjEra{_)HyT{Y6DISaGH87)6;mD<4huP zuHe)k{aRm)=aE{Q(TQ4k=L9Cb!uvxfy7u;>f_{(54ZOQtr>NZ=vT{X$-!NM^TR6NK z8V**d>|+Nr_HG`$$^rQJrtpD6VeuM=RQrdGt=27@E8?n-GrqgSN;Ewuah3K~jp=8{ zl(w#FhwddN#wtOjNt2(*%DYl~qh0MNLf*SVXUYzBrEVqhCoWS=uQm0EJ6EY4RLY~U zrz8-dKILG_oV}uf-s+3?8lkI+?lcc79Eo&w4OjJ#xm9=i4;)JNcll?E3@i^|RL=wR zX_%o6a3OH|=Ne7Qh7tz~uI|*#c5C{z!CE`a>GCH1S8ge4?T;HqJ@zAa&}fwnA`}{ZQgL`fTHEWtI`AxA z$;;zXz?0E~#~l z5*`Jv2Uwm+QNho356OEqeoYi`5XF_uOFwpB)=QyZu2YdQiK)X5_P3pW>Gq26jOBvc zbcXPk#F4+RheiNyif2N?1@&k|cqw{!ZB6syBlo!n)19^2$-%$J{$QS^XO3JoS2X!& zt#t*H;@WgXM(h4Nbo%3?A9D)}{U^+;3Eurdu+naG;=bMmj%;_L_Mmp^-Dhw-VCKuP zsWp6a85vf5iy5@kGIHMDM7#n0ez|{OU@P;A--zNXJ}{F0vU@z#J($>C=! zD8suTV`=eK2$EJ^pui`?-)7Pvp-IhmXC^q?N3{p__$%Y#9d#P3tp>+UX~s$>e@YZp z0TPV=E_Fp@Dd4<;1T6sO@3oZ0~3eC3>oH2{W*0O!oYhMHLNjXf`Q5p6&E0mBkK-q*-WQ ztStF_L4VhWJi9B}28j*$ppcRBV}gksE>&+DR@4aixf@OT?0J1{Z~Nsq)!{a2@Ca$j zGv({1pk!1L7HNoySQ9se!%5yn`C_DI+tLTjeWfc7Dc;?!V~S|3>vg5I_qt@w! z?+0_64z#_yF?wfwOu%Zdj?(j64Ptpi&UEtYN_gk$jsY$I-+q};(i#`s0Jfx;r)D(T z`(@$PDTMy>-@-k9tVj5*XYkD50KOJy`3y_h)^@XN_js45LSK5I$tWA9 za>DpiwB)=Cz_Yt2tzeV1VK|c1zHynhiF*fAylm}WGMP)dK=I=#;LjFR?L{snQxoHN<$^?g+Sd5aP+Njv0|_H zyve~u<7~7Po1}rFOH&5ZL1aes=SC=gHdZ_D%Je41?dijDu+`Zjx$<_3ecpw3OTa@v zx^oB|}II;Zw-CBz$tGEUHDlU;^oUgbVXZMx|W10M5 zY;kCe!((LdVz=iAx4hhnAjWO60#m0fB;SPX3!N+#=BA$c4p;DT&R?0>&p6ku52|&9 zH%N%j-*jkg2XQD0qtgEi$_@i%3pqgpF%z+Y?1269^VEpfIZeEV3k|J3&4Vn1Day7gG1 zj{RNX7mA;1D3xxxs;CZZhr7nqquLWawTVn;bt!{gy^Z0n73k?{k--p!Er$+8osINQ zcdy=O(&=9h4ccFMHMvMXgeBp1rp*AmUe@kZMpu0#&g6BMYvK9wA?2?!n>Wm1__6cP zvF5olIH!t%|8a-B6BE7pG389?+P1Bd5gb{9-*DB>`Y`aCFV#6}y#)EZ=~EP)OKjw3 zyFY=fBxqrceYqQO^z*d7qUZu8I%*cu2>vgC0q7a`!>=^i+gS3;m}JiEtwU<+vQAYI zFI4F*0N%(7Rg^Q?o9Onm(|St4O)JY`@KSpurghoLKM=3OA5h`S8Lm zamthjYJA098>d}n+A)eP34f~MV}o7wnM$Om$m}ZBPa?Lij?WcAPkcrcJL`nSe=hlb zuV5;Z=gr39ceswf-Vf+k#Yr)<4{oFFtkh|?AdQZ)`>aU*`t!OhS2K)Enn`_cVF`b< z`I=*(p3)~0@P;(}p(j_uF64Q(yVJIq!Eu-kEfWAXm46j~);Qvo2P?#R(Kz!`+m5rfhCmu12hX zp+)L8m{~8JIBNpFDSD^k(zmNLptW^JRzyZyQRJZ37vT&XTv)i{=Yo?Y(thwzZ~N|0 z`L%$5XQC(i3+<*3b*t9umB4woNU?wBW*L+4i)oo*bK%{gGa-WablCde{O|U6IX#?E zV1>T8d|iylT(##dzv)LI9{ANib7i?5H0@#0z2~8>oNM#!NuI|h5Q@%nZq0cR(YwJg z#)POV?6L>vsO?pHqQk)l9OQzAGa;4RpAfTp5@o3KiyZ_v;yN14=I7L0bv1Dl5#Aw*hY+WIZ{;WmgYLrNpnRHfBvyU@!^kODH1yST|eh} zXNb_wpz%?~rrwx_`fBK0+aCLKWd&QH!^45QHR8#VNWB&wcU2-dtSDsbE#w~so4k5M z-HKm1UfuNC%f+W}XdxL?hg{}1?0U_NVWt&EP+;VWyVxVSvb#CXpEWY$>wXnf*sS00 zgkF(_WyomHKUw{>`lWnLg9lAJN)-ixwULfZTR)2u0}kVEg)P$)h49*Agu!XYU|nZx z3v|BPl=DbfP{1i($7%p7hy*&}0&#yJEAF3VSni*!I`i0`Cnfzc86f$)-XdGoJ@wPL zo|hL1thikNnE8hrqrfp=^g6)Q z+T`RWY(P#8w+bXsYf2?)0UE!O%{H}FO6d0Om0Y`X!6r^p26!;Mb5 zEUZ_;E?53-eAAg*1+^wCQ;Beg*--feXnl?)tN;98K7)_EP%)8bQocU79`8>1s$uw(Dl%JBxW^^DnzvGA9#V+s2A)X&^C}&-t9?K2?u25;Q zAS~;>;?#JRFGhcR?O0o=P0uq@D{jU>za*_m@MsP$mAI=60m~T z6m_T?eVrZ;{5_T$K^n~UpMsrzAp#w{1QsRL?-eC_7VD=eh^|<|?ZVdDKgEx~CC`Ug z{TCk8dfHkL*+Q)sx4Z5%2u1x#LN1wD=~>c_rqj4Q$Y zI!ZP2^ELfeUlsx^$nBNEp=pff#cWwPuotTuB1B)NKY@;q6h-JC4*0sMO&O8sSXvZi zvC%tiX3tq%ER~SCfC;`Q7VGBf;M7Kn9w;>?uU-{}WEQBcGiDp2H^fbAy)#lJ`0#E_ z`my5W2he_puzYknRm!bJ*d?dhp}9632q9tbjz@MJzJF_QCRN4W@$3;&Zp($!s zxQ^k$7Y`d{%&Vrt%%46>lcSC6RSzI+@lz9AKBG%6gWWqDCW6 zLOUAx==pr4uL(Z3=M+RmdadXmcnR{>Qxq-6ngt z0G3^X&L?n=@xFzSN_v)W$jKd(!Tr2}+rm0GKfFZxy!a^*Z&2k&YF)92p1B|T>{tkG zIx-ffIrxy)1yGFPzOAJpCyo}RxN_%DrmVe{)Tpl$)sp*qXOQIuI=ldiaXC9}%tq)6 zZHqn9rdiRV9hn1kEF#x@g|}EsLoc``^2#g$00_=N&+|o_Cqaw)J>I)b7Pdj1-des= z!z;e=d^onr2SuMw&v$J)E8Fok;zky2%gbrTM{1Uq9H#oPVgc$X3jOP%O3^WE{LY0~ zr^uStF-k4<^pi}#k#pm@vzw(!BhTaPERUx!=RZ2+Y2_8h!qh`5Q5s)9%`0sp1dWRv zN_FRI=ydvGK?hAKbY9F4MVfF6lRJS4U;kp5GA?JHWhN(_Fuj_NyrS0<1y;}sh+D~KDO@?9Rc zQndjjA{It-40aYH?~%~FC0~P!>~Y@D(o=jw9{&p`^;TC6vw~G{o|aBE-x-(8kKo7A zlIboAc&n^Y?`Bn3+mUa^x=k*Gzs`U~Sd&h?ZGJ(aF8iBjzWRo@v#qyTv%!Xk6QmvW zsbmFhZWJ_BV0tBNZMJQDngi%PqZPjJZgD1;Fzbuk9Gne0I*ka$E2*;b(RU(uz0On) zox2+JprcW>Fc6T?aOrZN%<%|9#E96UEK3%hCFzxj4=03Hu+HOElOe0yVG*eo#0H<{ zjyf##?o+V6SoQN^T%cSuX*yuOE0{pon_Y{{nO|D6(U_p@omB_0phZ9_l&K|!_uVM% zB@cT4Z&eOZC*bZ}d{O#FzH{}@E>I5xHm(C2q7(6qTCaxF8a^y6XlX4{q}$~#tXib} z@XAu{Z?3A-j_`j*qL=Qcp8Pd1{}q6mDxc~9Q0Vd(S4Fl?UE-RtlhmA$s%)Etep3g) z9gJcfw-_bv1VASyHynIaTyN0S(76vg?Npr4KuM-b_@0!EHHHMNZGT`zs9t%I>;ag^ zNk~h_BT9+;et%WRur25J#R=fcSGTJ0vjpFnh=F@V<=;Tm?W8m<3VD0;yJDS64`uKF z^YIl-25TIe{fw(_IffH61kU>Vg z>uw{JXGuqI+dRtRnacO%N}Bs}5-isTf6b8U+-$7-3jjy4NP{-J59=jt53yOh;7W#L z4zF{78GR}6J*U6W9$-pm7C7bV|3N4LAaCo?0mJR_Ulf1^tMV@hV?deL0qokEzHcnL z1Zw>HAIQj>GqYJ?;Z6g!J3K~>?I}Uyr`X)DPeEl^Z+F0(HBy@t;Qwu0lI|k8GhEUbv_7q z5XK205)FOHW`~B)k!qosiX*#>7+ut9K>jhxed&_Y9M}Q zI6(FcU2FN|O7xx?I|z5c>VT(suk|{I2LnQU**}B$SC-6SR6NggcpC{IU}T?C&ZpboJd?+G!&$4lO_8}tI$t+6A3*{+i}uWAQSBKeqyKW6mNb^s99=z)H_R|b_%WwIvXzrs#> zRfR@QpVYN1t(M<_;qf)4b8BAKA*|%Q9{&`AzLFAOyA@=!g>X74ilQooDa)9)kF_y> z0eUpFeIo#_?mg{`pb4gzx{CiUFl(mo7k? zfmq=L($t%V@yee$6-$13Z%!2tsd@~y;~d*ajQds+B%(z(mk_yyG2uS{tt!`|%*l%t zdrcvTR|O0OHk=Bz0X`3|kG5X<-|As;hOE^(6V(X7P=-ku2*40M2ROU?m;FvFhq&*Le@G_8Fp|X)z!R#p)$F}u z$)^wO^=~;KT=Xox*XqMR?+uYd!-+@1BpgDenXu9vzM)w)B&EG)M3`sgfY#f-F)>Jb zuc;GM#qT84g)~+nRnvFr%gz(l3_Hquhy!mQ7*tV1jg%zMh!Y--+d?;p8XeE$o4h6^ zeQGAC2jQ5@rGpjqA;)I>yJ-vi81H>h<5}2C3ivU6v_v@mV|jJx_V>%t86V(&t)(^r zRo2g->Lz)QDyD$AN<`{bVrBWX^>gsd+!<5Vo>#FR>#YO1gpuiojF5p>na{1g@{Y_X zeS7E?&B|ldgS68`HeO(9(_0n4gYVJrsi)tP0`Tdb{5Gb}rFjoKd0Qn(i%&O+V zhwG_I{qwsi=(iL8OFjOM4KMF^ypo7@?bW)k4nX9v?MJ{isS4|v_RJ;p1V2D}HJICLtc!Y2ai%rfb?UKUf= zO*9`-z@Buss=WLjC%_0-NjJll2!DQOx{8AcIOv1r{e-l(9|EFEAu9V#v@m9%Oll@M z)3u3rFABgDcOQGwDKkMo_ZQL01?7s@pHz%KTDQmVd<4#UIT`t@GkTmpwpjSZr<;4U zb_g->B7BX&EQ>zX6o?FqS#+EvQRSnAEhpUU>K24U^LG};VOx><0mo|Ga) zspNc^;BN3+qXn`LO?Be|&L)9-jgl8WdB@U$&fqekJHpG7?BwfjJUWQoT;qW}uez^Y&B$w?EGbk?ym@e9Fv9+bfgHmXIIVnP;0)t zUU<(dg&Dt31)`;`p>uWR)z*B)L@JqS-u>{u!>0~(2PJaIg!)5Zewl?hyAMUHxTl?W zCC}nmi<^(Izp)6^KN`Z&Z6YVnAF+P!VgevH$mqz-CnH6>U!$^Zll--550qM{QmBaX zx|9LSjw)lzhUZ()WPMY$DJ~n+T^d?2@XLmjFRc1RqvNO5e*q~3={f`RO4fs}QCjIS z!g#=62q(ufsLq>KH#D0SkYmjf_qKoi{I_|j)|58DWJQ7eaeYGm#c%TdJwji=qfUsB_pJR;h)DZ?3sPyAMF-4qS1) zr7Vei+YBE)8X(=j5zKZ6s^Si-MVCXc6$4EOL8+M1x;K*tCA<7LybCw}EG{~PDkOE9 zOZ#R&w>YYtQp=al>kP$H(b3&s_KG8{@t2;k@?5SQYN3=OB6l^1NhM*ggb}tX_#no9 zo;kbwf7TY6F`G7`$#!?nHPmF7S;)s)x*48+V#WRaUOVqsqHf*Uq^mKjiAa*T@aKC| zKab>V3aX5$Th3zK#_cMOa|H9vgf)o zF~{z8RI93E=+l}*GCwR4kV`A8KA3^~Ikv=l&qYrcMo*maY!29^v~}i(E*B|4LjGef z2m|52PWf7&zstGR%W+GMh5+ZZb+@tI57`@5%{lhR6)skVF&<=O)|A#c zj3q+fB#%&*!X69>S;e(VHqd{;jW8!{_vq&DBPNw@KcflfUN8t;Wmx;N{I_n+V}=)(%hL6e6jz zL1e~#alGBc6486|?Cj8VAfo(5n$u2RzU}slkCg$R_GE^X3@znX2IjxF1{4L_AD+A# zxvBTHDZ@J50UDl_Ki7YbU(%$qp=#0cBESt~*)yxSz8n`}$+(@Nljq2ldA&s1MdYKO z25ChxzumCA^|XAt;rLhcY%jC4^OZ@&&t<%}`sL-;CBHW-_`1IalUW7W63Vw%A&#{T zRU^D+j9EEnytKOXH z)KT{q?x(IX^e!6%Ecar{_!5oeB|&9*|J0~p59i|wT`qtS1=0w(jhAjNm8+CZ86U@i zDjNpS7qJvPGty3(0^SDLty|TmN9qbAYpOX#yTR(_;x-}w5f4g$<^c=BtYJxdUHCRA zP%t|*T>K(|=A{bJ*w~nfFV^~D5&CuVJfi-EpThMgp8$ZSeJTKH_!Zamb03S0C>INh zBzMIAs?~_h7}0JF1Iosm2%z;tN2k?4Y?WG`Ar-#Ulc*Pvj5U+Om{#(-IJV9yq!rMI zHeCD?wztjKv8HArTDNY0c%WqNUUZSeaQkm3U$VWM_BR_by@|EIXwrdAYP76X4^OVCHO^Jy3@K{shWm-3YbBEpZQmf){pz}eurn%3K{rI;7JbDAt>6(Q zQ!R4?9R9_DtCL|l3CWVqQSGcku5N{jv-ailE_&0!@MsRY>QzXHRK(9Ov|UMm zvCo&5GUkiWgWFMgS_B1;eX;&5g~!q1Vfo})`4=j8fSMBG_;u+6Gdoas-HtJv>9umP zg?a0OFt@XyKJz%CtuSFaF~QU_S=07OWl_}^#k>E?CpHzGusy7+|Na+HR@pT5F=RFg zO6)LI=YKoG8u`#t_8JiQU6IS+0o{+r>3JQM?1yMR_S4@O z`qt_e{h>m^bU*xsi`x_K^ng{tj8IrKlF-L8tb{1vX&qflLV2?^QkrYLq5U*Q{j9U1P z88Db};{0`5+7_Uj-@AHd4CcydwfwX3b*%Qr)?h+Nt}5iXqPBCH)Bat8vb7e|xI2;7 z(!?T40oVB;!a0uI(^bIz~6 z%M(?ic^ZLht&>Oqo5uf)U=;_GFnk)^!rl{Ss?&zE4l(x3nx^#akdD#hW zZ0V%Av&A>pYr2MLY|h7cP+0J%qsD)!I7^iFk|<+tVb!G8o8z>b9J`&oNM`PnBgfNz z2DE3QoI{ zqoOnZl0f502*!DE7b1w79_j<*eI{wGD!y2)%*JSL0SAZ72Q!Ee!>*F{1; zqV|q2iuW}swtim{s=40rfy3(*S3iKVeXO@3M2XIBC`q>XpB*20v)!Jd+lh-rxw*Dn znr?nGQ?1|3c@?}gTMBft!j|m> zkMaEWfy6F$hh43G=`3iG8w<7TbE`Bi^NH?r>rIB@&lB6d=lL&p)P~f)w7`ch2hf2# zD2fjF%ns-~%A5~6t+0LBeqXon8GH^uQJm+#KUFJ`uH*wPAnrP9XX&ij6KbKh4yo`V zSA1B_G&d9gPK2=naieC8aj#-TOF4v`f^mv==HJf}$?F@AvIc&)p zO#$7tY>Du7mAq?LJI}qrqG@D$83@xwo>A(9_)~$*@9T?mZcGN^OM7#Xk?~AmJ|1+%RhCN_8+`yuJLmjdAL8ND;-+(S%g%F;HBIGQ^;eXGVFnl0fo?qm=ax#s8Mtm5{!2b_gmTS(2X#_) zMkzs`h_~Z&f+<+o{3_=!%GkIaUmBHNTW_o#1&yibv&256Z{jBWv*Tv#G5s87kd2cQ zVAqVy_pTVSik zX`w^DXO&c|SSyBUd>U3EM9m~$rKkD#z?T&~#K%grl8N5lF8d{%u!r%D_ z^X>0`?#rqU@ii{Q=LYM9tYq_aa3^LVx@_rc5bq7oTx%$TuJM&pc5S?V!*Hm?i~mGu z_A;@q#A{+USq0#<6S-%&2jq;;ut$c!SK>dsL@az4HHAy+<*9{|D z^0eO7hO0%ZKU2sVzaO56X!t!@|G6IL6P{KpVpR3AIdAB}eiXEWLVt^Pc!d{v>s!@6 zu{A5S5rJ28ivt&WNEY;S0LQqF+|h73LDcp5z!^7n`su;Ufu{gD_%*R>&(iCTuHMnj z(8|j+>0i`+TRUuVLP_Wc|FNCkET4014N>-P7s{=-s`H#9@{+fb-VMDcjb51zWY>nw zHR3MuAJaoSOR0Rp3RfFos<;5*hX)aD{mRB6DR2kM*5La1l^4Bj0_C$uxjZb~-@jy~ z?ToaJG4am8co$aLyC?{)6T3^6i8I}cG_)5WwELfdA)C(!<+!Q2pMYBGKcwU|bS~%8 zi#Il6d^U~&`di-X@E^w5_R)GM+Ip6J7zP`tQeouie#j3D_Zb>5wR+WySLB77^QDXzD7D$kC(A8am;fpxN({cqTQ@g~uDDX} zn<1og;lh~6uZulU@lv=T6kmN()j139>LcUmy<&;<=r$@-=^(aLpSz~+YgXs83wwzb z#u?Bs3L#KDd>Te9A83T;Q+~(MA41}ra7)UuNR>MKtUPETU!0dU9?Joq!Ns&4WF7~i zgL09_cZNzH8cW)GRUw0?j%2`bg1>~sFP-?`)8)fN{uBPK==@|vlE82NNWJ^5-2y>d zF*efma?{qx2{^ypp{%cwCksE-f*kRH>`Q`2!BD|vQUtWJaw7QoiK+?<%xw+l|H4NJQ5d$2-Lh-v8&U7Lq>c}w+kFkMOrlJa(p?LfnAz`SiC-AmoD|Hsys$3ywG z|4&MZN`10b*7~$qMwE~}6@~0Wma!(GA+ql&DTEeFcG+i`vG2Q3))0fSBn>f^5YrgU z@7zPr^L$^g-~Hz-=RW5;*E#1}-q-uU&9{xWPYCXfDpfKc)f)Tk?iXR-p;ssMR8yjq zvACUxd1bCyy{YkTvIrE%uBj2S;QR&z0~G1f{1n#4MfXS?qO`sTZ0$A31*_ey9T1mf zjWEcRM(JWX;&N*xJ8zGbc^&pqPKB3zWq)UHGLD@c$QnCTc?6YKOVrHfV8YxfPc%UB zsG$g!U#Md9zRxKduEAaQPYQj9unSi@S3LaItLj*vmN^g{nl+2LvxOyw*bOju@GqqT zS*!x$(1=zVQotQ{&V)J54f$+m?m!i70zb0xJ$A+(2S1;BdH$F1d$LQcbrjZ)^;C)d zsWM+wCBZK0+O;xu@mEAg+_fp}y5#K1dX$%;wa;OyH=*DugDd7Ls9TP1sywE=S5(^f z&--b>Nfp0F!r4Pvli`BNuEQE(BR2&_2kC1IqErDD+(|W;jPse#cYc8Kbfc8_J{){0 z(#yK5AggdP?p#N$Ha)TtxO^sRKdfQsHuCW_$d%;h18~PcA_)GINRPn_BpoZNhS$@~ z9YJzHB@=t)=$?|%+z7JmSA}1Vr#P0J2ayj23FFxG+y6?L+a z$ualPFPfPC83tWCm|>Eqd~9UYpPJt#E$#f>r%QF4*gd|*)D8hUB>VY;0;qhQyiS!_ zGN#WCM)J@!m$o6!v`R~=TbHqHw4Zx-;(I>nP4Ik|cDm}f)?g?4+K-pd^H*2bVZ6E2 zuhpTewV{@SPX}HvtuD}Zx)Vw7&Ut{j0eF*BTCM$1IvS+I-7L=Ll|Gv8#B@oFfNV*Z zp1J%G3o;)BiygkqpI_~-+-?r#xAEmo+1gm`vW}FKE-*ehb$*=!!i9P#=&4nx-T=HK zsa?og$4E$7F$BaZxfNDAzpox(l^-W17v{U*jjzhLB8|-QK$K#*+Rcmdbww!P&Iia1 zt0aMF=weTbS1I|uocC`^U5~ZCMi^EJ*u@PEvnarzbe|fi=5oqS-RE;Aze`n0(`RL% z58wghB8Opth3}M^f|#bx2VHdz3q>ANqW-{+h2eM;x zze-fz_2=^awun=*+2xa)Gw*xVsJ?%rI^YHZ7_8O63?8z~CXfoUUhw^anu)%-OKVR< zk{tb{}Z8sw_kKm_Ke?C|_q#M}dfn z%;;ouz@XIPbe9wENIlTjFiW)WFv#xLnSBw8v4N29qY65;q`F||55j%G$9(Mk$5-}- zTY)X7V|JL>cp6S_oZILw?%qtq#80^42tZmkV;c-QFS|Xhvgz>gCRm%^%$ZLMUhrIn zm%|k}$2^WOXNcGRMRV<}&?{*r%jB3>PLAD)(wsNc8#8|ZCQ$f(%;`?HC%FlZmH8K? zOLLhVZ}JPHv6+tyCmdM%LrtEA^ZNW~tV_K+j47_GDk#*nP)mMtw&P^vDy8gm2<3BAkqQ&eG$v=8eLBp}@w1S3ZAHPB11aG9 zjR2)z8zjY=io<;+no0hdR|?pdCW?(3oV3ehruac{#Z-e7tFSkHQ7vY2XlAFZq|Ob* zc*5VyMWtTq{^LXW8vJPZ`C-HIBZe&L!5%5Brol|w5QbQt!AvJOUoxMf_5gL&-m>Hl zOGoipmWwx2X92jY_+F(?^teK?6eiRiQT({AEgpb{(T`C@$=`sr>0^Ak3-jUGS4*-uM?)Vdu z?yji!Rn6hw1|yQBB=P<~w`TZy;73+pBCDybF9Qu-kDWjosXUrX_E}nT(>kGxPhsV$ zz>)v-s`Acn@iiyPnV%3**7xE}qSRPz%E360M*hy0#<@X5m0t)Dta~=N?}JG<=(e}> zpBh33Q>r|`A+P3p zoAtB&UO?WZ`BI6Kd2HKZ7DmsO`+PXZ(1SuRGEqEbem}RmGA`gV^Vr({g!PtbL28#fe$;S_HPfnkqxwKg|T&Q}hF(&RPKM_nkRcvp6>>eTB4fVd-c4=$4q=&J6 zx~RdpbE(eHLrQAGg-}V9UFo!v#+GjztraX78OBjo_+%9q!J<$Qi{d!VW+>VwW;I?* z;jHCx1hAdC4OQ)>jbQQ|1p6&<^coU<7u}B?lf3$*WQ`PCz6AxwuIkRNNO=7KnBQk9HzH}hgni+D|+ zt_~o6Z>cl;L$^2}Vx~PXu|fNRzPl#J3}i-iaktz23?O5Ii*PJwbHYi#0 ztlYaXr^nROcjX5;o+#t|As{hlKL`8Bey`=0SZ}1AN^l{YzWP+GX3|{1;wutvE^$2u zjxWKkW*)VyzQaPe>gHtZPM|y-MB17}SKbzXK)9^1qU2IA@t-nAuq-q61OFy}G-Uox1Te z>*^S%+74V5ARQUc%j}79`RC|e2QNg4_IA%mC~OFXx@!M;IxeEklxzX);G)qdd+ufv z0bGgOL|ru$y;sGl8Sk6(2Yl=a5X`o`7G%$BaNh|KIdZFxE8^ssdzR?!Goze{z7MCq z{#!*db&SoaqzF_sZGPUwblJ-|uVmJ>J!P<4-pDX~lCZyYF`m9ZKsHgZRL$h!F4`YH zTJvYkz}4?JECBzQ%1wUD->13DYSS@Y1M#ZZLu_utUv%!|yZPUYE(K)_yx@+K1AV7I z0_NkL5MAaNFr{L`A5#~~MIY8L=ZSGlR_3_GqSL_oa&?mZF)(KE&(~1YTlDy0y~agi z-iKj({WG%hZc-=99N8rk_Mxagle<}l(@_Q$;PPc3DHu#8yGEKHp>h0HoY}GP_UjtU z0s4bzgjxs2BN)VAREtfC(%D*DIl~n}U&gjUY2p@NTE*S%sTs_NmMth2<2d9$He@(H zQn_j!_N3%DEzF4j(b+LK$MN40Y2zJ7muBAQDUVFE*Q8W}5`kHaCg;)}5!NYOSxWg| zr6B?1i|evrV3bF8vd83S*O9V?tuS&r^^`}O~U4T8RzcpbGI zlevSp1Ino|n3q!hC4aZB42`=1Nm-xiP(5Ol007R zOKo)@tEDE_kK%PXFGs|tY&Etw~k;Npi9{320y1D9zJ^-A8w&Xa| zs`-6jOU~doX8t!PLh{t<7p{<|7*vEt4dDR#*(@0Ax$yCGEQj!$vpUgvY zPH3ko?D#}o9~+}82B~?GBg7ZtCO|&fQWc~@^(!0irO(1>h~wqljkD;?h}sV>BeVTD5e0x=U>OS-uN5<6eku z45)~sz)i-BD-;CECZSqC1N6Gtb3ORj3sC83J5biu9gI4mNXHVr0VR0+DVZv(@OC@Ylqxym z_P)a8jk&ZZt6}(6G2@}CU%vW!rwsOyK>AG`E>u5eCgQMSrDG||3BrtRz=07p@V=OC zZ;~)hr3@^RA4f42-L$EJtQW6ILO|UwT6S>nIeq5jn^Ijir5ccOz|J+>D0&#M!tQ4M z|M9%bBM!JLvBvfvRDu&L_-JkaMO$`8mrl=^qo${^_S-Fk4l(Q!#*2T5zj_Cz zTVv2j9g1*O>LH!i!a)d~S0Ys>?|ff|;K>LFAAsNvLq0H511P*&m~p&^zARz%*r6~{ zRmDmN^7pBtu)Z*Mur^V~$GDom0E(sBS3SIDbq)9S6wP89AjI*o!>(3U6^rsI3(?# zmIqRkPCHn?!-SBfP&VifF#c?1$X(3Nd$V_HemH%C?h6}l30J|!ARKR`)WeD@yxX+2 zrvqs3zxZ@QQj_0(^reCE(_UVs6S{1-(T2Uw7 z-%)-5H39LF0#EHFe`-U;OGo&bIDkl=@hOLDzNdRS$z};Aig*BajTlFfVjR4iU%g#p z=^%r$*;>(?z(Ahcbcw)UqDQh|Z$Y|3`0^f@NdNzz?%H#q6d$h+$GzY{x@8eaxcc#$ z**i}5fMw$^82tmMl3BUq*mqPu9i&U|9;een#5^Dsy#($_uy7{^(6>HV9QRJ#;j6su zCJv(+I2;wy)3-e_LH`Q`@PnkH%X7fO{fFg`kN@}MV@j|%WYd3&X(2cDd>l;dRNr=; z0ely*z)x@B^HZi$DomrC4IGxc*o>Xm%DspRIkxXOEX&Z0r)!%RdMi}m11z#K{hNE^ zYcfb7i2f{_ab59(PVEFz2;^cu!I@pv-fYIbDpQE?7>%G7A7~r5`qV|fN zYG;1`7h?)gpFm;|jcyBn5c5@7@im)W8{<4Aw7Y8TeSLu89T1q*rEuq zFb$lGVoDW?E>DPV35X(QA0Wpd03OpV9{TI!l;~2+EFzTEuxJ4!w%IW~H~O~rSK&W5 z+j?F^iaYAi8Q5kP;HPK>k0$mEfKg3*{5KFbkR7@Pp9*ps+p| zx(_;iPv;SV-7%b_k3k*uKMwv^M{d(ce0)zy4WXOu)=>>u++myYF8ZrAj_gR5_r_5* z0RI;BOkAh@&ED+@s^35WUc0}+^(t8JYhF}e=x=`&R1oulNbBk8k;2t)d+%*jpj2|0U6z_^>4NNo>VfAd z0=8MygCN(T=1bhH=AE(c_)nhpOc} z@Kt3uD-%&2vE$Sc63Q|Yi+Zh~1(-5x+IwzR@;FTWSjD1xeBXDhotiE-N-$iU*&FnL zSaxAXhvf{&tB)EG<*F9T;fDSbWLlpe&0&dDu*44a6!!KSeP;Do9{K88eU7?jl z>6u)>&faYsxfBD^h^^#+&3YQeU2vjjWUH(P&j{RZo)lZhdfn2v3>!g zG$5ol5TvRT3_mxqOa;eRC`_`Dmt7eg|x<0EeR(Jqc>Zg0<6maN2J z{95U&Yj@X3lG{AGC*)Gx>jNjhmtG3jVJXOy$kSo5dY2XsIOw>u4(Wf5$>zusH18oc zXW46(!ukr@8?;PdSwMJK?Q!@f1Jr8I-jLE+SHX!@xf_W6Uq%=-!OC**DGRO{$ufLi zV?SA)v9LhdoEL}2NpuUfcWi2%f%d1QQdCT&vrwaDse16h zK6aH85?}`?Z9}edNz{)>I7u0{H{)c zo4hYN@k(}(mr(;dEJsj*BXqJBDQiAs-|kfe1U!n{ZM3@0MzVkaAZ3zC;P`)}^3c6uh= z`cr2*Y~g+*vxGmKesqZq>)#TFUO*aiJg(V`%}{tA}<#_%@X_-k-gi|*~K8+UpP{R}$U3?4O{vJj-ISiD;a4B9%@ z;Hpmb8vBb-q^~c7egAx42PiTrn1)oNHT{3Hk8T1DIj*F=$6{k(eHHL5K_6>y(0{Ls zj)+;Dq3sp(Tk)$oO7~S7e(adP!VCw!xZt4VLM+Mt+P%iz^N(&h-EH6dU^bv}U*IeV zwaD!x!6C{i5C9apdBGAOk7_NHnJ?r(WN+7rl~ zKzzFWWFI7`=v~^Z-FHC|#BD`bzq-c`1pp}5qNhWI(sSeXiY5HT*6l1JIJk|$_(;R= zj%xcIs|b`|fqo6@UbrT`2VDGtbRVJZ3}k0gjhWBpfL2bm^T*HaSbw|r+PP0^fqD}fHlrvY3> zlt-`c`us3mSfF;0u60>?FS3+R<|RX=vELfc#{pR+*Qb_AAS*cyV*O4Ld{j*w^Iga0 z=TcH8<);0k&eZ%`YVr4PX>GC3df$M6%x-R>;x0Wnrsu_UhsYf>^%rOi>Sb2dn;r-> z+3Pe4qk}>Guk#H)1a7@*G4agpwkIV zOC>*a)NTwHk6PMEf$qvPpY4{mz$@a5g5ji;;p%^1l=&~B$Md<}muk&_x{=;HU{1RW zb{aA~j#YLwI!Q_Z6%ID$Bz$(>l&T7JVy6*EQ)@Da?ZfQw+DJaU6Tbv zbH!t_ykDdT$i3nmy4fm1wFlno^`aa&(WfzFxX6oC9mq8ZM;gORkF)d3t|Fm0o9ms$SVtDUX?I06W56o5w>Y%Z_k_CJJ#9Y zzfVDV)_JHZE#yy;Z$qgWF)8}@^YUv0L2RLgY9Yx{z%6_ z99S_&JpQ~N-(2~H>~w%?wJ!PUKl8DmHT#XZ#jK7L5#?)zTm*zVGc6rRCqJn7m*h8f z^VIh~B&^r%H&AC5VA(w!fIryUXC6Rxiodx1*)ZNIfJ(c&GEzOhrj?N3RFy{gsk%M$ ze1P*>387L06pC(nbgWM(8i8NYvf9_X zuJ|Xj4i8mM&wSjf6;)0FF|L*wb5l=%e_;wDHWo64M&kmk0RGg~VYLd7LrEN{U-5+M zwnVY3xq6BxrEyF%aBP25RmKdV&dnB(v*fR?MGHMH&;@W1=y{7m?zNs?4RB30Ov7-f zuuN<%g7P!kaE}zN!%Yf)P%~I-E1^e{RvxgOHV#Mo6v{bybqsX*6pPBtO#7z||HcN4 zz#5D@o<68@w7z>RuU)uW=z&hR-`)xfv|c+78`5y86>PBvy_VQ@;6jnixyNE>$w6k8OcI1#Dyn zOppN-0CSjgCb?(JZ1vU5c2PMz{U7ABuP}@J0rV78rg%7%Cj*s!Ad+q-Zaj`CR=Na@6+&@YKTPzeM4L$^-PWJpsGS%aAPKoF z`UbzO-(IKn%B_7(Ap;VPTks&gOydEGnDYz|df`H_a5d)iOeJOJ^Eq#t#ZG5HWtsN_ zbY6=znf7K9$9-nv0c{taC4`7XKoxMsr9V$!zF|_b?t5|upuT@AJ7hvT<8WbtNXQZS zaUbp{1;@cEl63M`b*iiN;^dq4#7g4suGbK{04t1TNy+%J2z7Qs1dmS|=Ph+objWbk zG~<-=(cRO2w&-o)^62A$VX}171DzxXQPSd1;8XU*EB+Z>T5}}RmS@n>)$MBb!dEm3 z;F_*x8n8vHww}Y)HMWDPL8C2scw_h(*c3dt*B;Sl!KHI>0TiWeR26A-clMB*kWlb0 z=GmaZ2M!hW(??@`K!Pnk+H>VsFaVl&T5Z{vnbRg)s8DpUAw&1XPM=(Iqgsb9RFBNh zox2odU-{GQ5(6qb?5Msg?Zm*>H#r;Z?098;lQ2=`<}8oeT1xH2*1aYFIY|k4Pk`qp z4Y2!9u8rcm4ehAwO%iWBYDp>US>~c9qCo~alW>6An%DxX$bZR)X5=(lw#?@@|9RZg zZflBzL{$y=t?h*DE3vu?Zrt+_&FfcOI3?vMn8!(2Pv@fyOdqc=PM5*;T+~?o;Za}z zr(&MtN`AnfO!D?>@T_l5xnm@}^GveV>PxZ70~}=@4+(2V1qHwH6Wa;F3oD#qW6!dj zC!+JOV@-KsSqYAWySYrZEejC6!4vG%jBp5%*Ynrp^j8q^gi5(9??Oe}VHf@DuWqc- zMoFQoP(f0^*dh{7Clp(l?rTuka@tl@e|{B&T2SDY$&@9=dfsht zFBpqOW`O~(t8hz^FV(joZxDIE^D^9NU+MVDLO`EMjEi?6H(&L#`C5#?bL#JU%0u%< zqWM0qk=`4`dW^e|_oW2GJ9P_I^48U(;Ob(>+lUg%>Yf_UI`|EaJq}Lc{ld;0;q|V<%rgAG!AJGk|kJG%afUa z?)ASWyLEb=GRGclwIY>*AoUe>fE(r#dT6~*&R#2Ka?P|bVn!01q07&3MZ+fJD_R^W zVM=#T6ol#t+Hk>JyV}&d5$CCdtPbefH7BEK*{ZVeN2Z7R+2=wFhRn_H{S)%ip`Bui&%P?Rk#H_vRxZ zR^-}$Oj(M#JQC@%?u#~j1-}Zhs0KV<27otjXN6-~pai*Y ze^&IO%#O2^01OY}w&J$wDoJV^vvZ(Om@c<0z?~|qIOp4|#}6%Uns_4})(y2@GTU|> zG`U$JoMj$5_dv@A8Pxg+cy%}K3NhRj)Hd8-`+Y5k!O$d`$z;oXe(Yf4zczkSzi6G~ z^Npg_iDAsl?cM^sVdvFuSpQ;OW)dev9Hel>6-ixl_ES5G+gr7U8x-pSc$Z}cf@Iqf`x zrK9;?F2l%ygnfPJ*7%6q3hcI`!8LvQ?++Z-l&uPI4>xz)G&z7j@zPGb-&u1@PG-TL zIoprlfBfh9iENj%WhFISztr2ZrBM#vi7}aKPPds4F=y4kdjmfVm%qB6MV8uo4?!Qk}~e&+FrqJYt4tJ znA5@@s&$c6E2;>0-!a_p@mqZTwC!}#luLzJ(yA%i;y^>PAnUo`WHL);Z(mESzwM`0 zbaI8id6uZmz=1hkX(7g1Gv`t9>oHv^wE0HCOK&~4hC{yRt{ypcDAe!!eB_~n zm1hIi7g<}KFDjm62|{{|y}Dk8d)xX>`5~oBa5Dy7O>8V-{b`*4L!78jxZ@Qcah-Qg{aUrF zZ(p2>HCm@N%LR64H_Y+ylFfN*WoDtwTsgg)B`Xq9Ewd6H(Mj(`i%j!Hf-#ytZ8YwG zRMZ79`tRWj_$6-YhoOG9?E81x&faO+Xy)7c7o$XO19~LDKGRo{3Y)*W@92Tji)ilW^2kWM zqy&>;kMeRWV>?a#oxSZSo#2G~wI1C#llzLTzmJ$siYP5ER4uW<136{Vomlld7!W_*-iK2u-cPH?4Mo!P8dX~zg-JTtWi zK4;emn&8k@hW0ZU&LqwyZ}Ncy*`GTzzZoe6&)D`*Z}`qn(d05d+ zw>#U_>$+92iFIVJ=dC37YdLV!_*m_%Cz}W%&;F!ds2}IVtzmcg+pR$)26=Yt?vVXP zm!`&8T$8q!O)*pi0i2n#{h zL}1}f{#V%7O}CwV`)QvTZnflcceJzj#xs6(zmM(`YU8DTZ~UhJ2A$;hYb#)IeZ9q* zd-+|nQ1b%}b+9%0973fdvMXD0t)8(5#-)!q+x8VTpQo?$(9kJ`!8AW>`r%o|&(ALp zPGajXt9;l(%B#3vMBoFU7sFa(#sfv2r>WaQEax)|@R*1I2LnVN_XX-?q`!=Gt@R?1 zT<^E3$^$$F%^PO~?!ey*xfAQwOT$i0)fBjlLAy=XwTcw;fYl@YDw$>e63W&*>WT>cCNu}8bSsN>QAusn;5P8k*oZ7%_l}0?P6S< z{79!x;uKFRt4F3#N6&<4;NCZ(fCbF$GavSR-_Y}^rET(osS=M(%C}XvMC!T?NHHaf z``L5ks8L^U_yZ?WN@1g3QLMso?lP`m&@g9TY}xH=C9lyxBnR1qTQXOZ_&Qc(X62)V zI(YP%xfu$m9eW=TXJz*cZ5N6hWz zwQGlsJo77%%q7Gut&|Zid{{C1k~A}v0+F1fn)gHiO1BkOZ^;VAY@B@pcJav?gDb7Z|3 zJ(0V>mZ(lf=oW|s=k>{eUFJ3wS{eOOc8ZVf0_rGh0WVqgpOM@>1i6RiO=B}tCsfM} zyZ2i3=;TF!gfS2Zj=US6u-5AdG+%oTSa1Bk@aSGCG;ykScqG#y;WkxfcZxibyU<^z z<<09!1TZh&H$9kYebo^YX&}}`MnWnb_}a2vv&w+kUH&2_ zdBC4B|MjT9VQIZ^n?y`Tlo4QdVxJjbA3W}WzjF=TfNbyhd5aZ9&gZ9Ba!1g-dkcKp zabzzy=xSu>X+5Xs&cNfmwN}sNE0<^|`}TF7>j5PX;pfNcU&8_yL8RGrbVh*|&673P zxwW}r{?V}B{qwg~{W~a?Fgu++Lw*@{GGPPG`}$$-PC`4ZuHk-J^ZR7JyWxBgzCP4< zNrK_)=&6a5V4Z)Ew(?mk9`JLRDkCh016YaIqlBkNMij@_y2}s1e4hKF%h=SwTt%5v zR}<`Z5)!Q-0G4_tnLK)C3b`bARe?-lIS{+`Q&q9c5dp5H0#fg>s+xNM^LZv07-e_^ zn3~E5Vb$fEGAUa4`Kq1R^*RXB^SnhT)q+hj+{t{s_77!E$~L-hV`K2p^X>@GcFX0_ z8Q%B6|LKLTi?|`2HHHrs_#9WJuNW@szP`W+nV+e9i`49vm|>p!W+$@cP}xxo4atpb zJ1~*?IS?cGc}eg^)I?~%J41o3_A$B$E#{$2Oe%)`sT~6|tokC}6B2CT*i$IkxW-7-_d`fInI6aog<`gac2--5@Fb%K$?@xshKw`&X?&h;6Q_QK4g>2~H zqHy`05un9_pw7UNGe{EMTWVE+@?2G#W@pdGGBDQy7&;J1;bxWj_KtmT@o;B!#L43C zY>tq;>T7i4kM2aZh9LA5a5G$dpGocLBu|IhCHU^2ym1PQS9Lqw$_k|Oj?!w{6n zQNp)BWkM5S^W$XzlGA+2a`$+o=UcFEm2F$T8G$=VEyU7rINYTQ@UM4RvI8DFHaT)J zkS-%J$yZvyvu751OqLr;a=>BC3uph>wp(Y6 z3sc6^z2ZKu)ir`nU@n}KHky%76#7^)nBqyN#Fw|2Qmejp8wzt#WlX|zQ zJ?0=PW%_%4{YWK!BE;6|)8n?Hbe*^6Q?u=DuaiCK)LYC~^n>rXv`B`c5VZzmZ^X|J zqvlBdbm=> zi3G2S7IFPrzOX6bI^617M-OIWl{_&PK%>Nw#&^gg!%;ih>%yPTbx-iQCAx{fr9d6= zB7=MALWpvFaNL}-5V0_p&oKDPTp3PgKM{m&yPonA{V+0)#4l>di5BVyw6;;@SQW5g zZV~HN-R2msBP`<$CiAMM493fZHW6$7f}$7s(pP*6x%+ax%_RsuXuk`*BoK9KMV6Z9 zZ(P9j3rXm+i(q`T5VcP20koTLPc8@0tii3Oz7JQ{`ij0!tyIzVEu4qm*-RIU58PQ7 zp2O#Yl-vZOTw1bfB5P__q+Jk`u|kMzuh5bt25q9XHv5-cnRAHHJ(1`J-Q`jjPjt;4 zWC40e%gYx*H65+P*19O=2p#=8cNdcNbuVtWJ-KMI)_`>Ug45-e?YRhR%p<+Z6!aFF zg{sekvr@D(Zb5(19at72k3BReb=wtNn+_OwnQv|>qD^SqtG|rKRb}yXpb@N<4!)|= zy?Rvt&_@Xt!xCn8V*6yG%dZcP8VD33*bUvPe+Z?|Xo6lZPKwe;Ot`T}80VWGIW)lN z(@xcKOqY9J1QY}nOD^{0Cbb&!YE|~!ry@!fyUIG+bQ49(M$Nsr5!BbhA;Mv8xr;T1 zj78`e{DSJhpU}1so7=)d&m-Fj20S5F)d=(kTE-nQllI5E@NfWaj7*;B|M#6i&5q8x z@91niqjG!N3%SnMAwg&c8Mm|-Rljauz3^2;f8b$F%lgV1x=CbVzgGL6)GEI`LY*j= z{*eUzV6C|zA>xz{C$~e^oT)xaFYnfXrdYOUxL$bxqAkNW0<)pc{;_?*fH5#WxTSqc zcuAE~gST6+i^yRuxx#XIfH5;P;3;CM*2M?0RBY<(-H&!IITnjy~&VB%n#xwx?zI;?J9Gy zOYCo)eOo^fMr>xz4aJ=;R#}%@9et!VXE2YH*0YHe*(+Y0Cf;M$=jW=w~wk==OcpX3CF&O@L z9kkxAlWf(LpjTh2RIzj7v?fB;O|(teKD3 z3DxLc*}mVFhwX7!wq4j4W-O08WlmW$n5nDqcTO;vNfXM8R_vJwXlX|xri3C37dP1l zwzdXJ5v52(YVKmKz9W7jV2djZ7yW8_BEYNaYfY1d_&Wnf`l$u^At14hMI4d-VHwP( zDQs}NgE(1I1-tGwSA&M#F3dM*3`c8gj7?Q3gjzdr)CrM?MULtXs8()$XM2gD8X}N+ zqQCXzSQnnDq9afUrrZk0fhiaE-1}`=6@WY%VVy_9U^k2@T(MO6qIvOAEsV9c)N?e< z7^8q`HD`a0W_iL+#tcjtP%7BUg(J_L9+6R85YgJ)@&H2u-Slv97p1G=Pe=N?&Ifbi z?A@)8%;t*7VP5|lEEW1Be5DK_kg`%S#~@6U96{k~Qiv#%;Cmw{Opqyv#?1R|IJxBI zsWz#@OC|R8NzRglHmiiLK|zdn{X2ratwz^+rhQ7-*7=qwtzz;ciq}|OEa74#E(uRK zXmdq&+HESAMTy-UchaQTx_@qMrD}UymP$_P=<%skHmfrmn}E|Nee4chkcRB@7Y>NF z3%t00EdpI|-8<2ck{dsc)hjhCKB}Kog+j-0UgvddJDW0Hf}l2^FfT@%BJ{46 zF$a|v=b`@@c@%+=rRrfvt|W?cX1~ifIiD+|+TxzT7}?(MVDPFu$q&b}zqI&H^~@Eg z`o&uOyl464IyBjGz^88_z+-zWc7jF`7a3n6(z^xuHo$f&b$OA=z40j}cCRb~-6Yzw z9xCu2E%uZ)x)~K}_ZAvKOI8 z9Hkrc(F!Z510zfqW5}84rSEa8YJP4r8NPX#@s3J#l<7d-;Hyl>G4yhDTlA7s@st7f z{=maSMECZ#(g6oI0)CAUUQg?$ZN<_w31f7wUJ(0(7E9|pxeL07#Px&2-&BcdB2zOq zE4Z;2ob*a-gr2=o#IcmKHJ_jqWpKIJ&(ziMbp@HWh$G;_d_7gm_~6!f#&+}|Ms~U- zP!ExsRiWvnhz^oOw0&6gjOL4NL&`}amT=!CTcV6)r?9ehZUUv>@3!MO`J{YU2NxW~ zZ|J*iV_GMKN(?wnh4vR3*JiKI5XB9XIKf7^N;a44zI`>(;Nk9Sp_?V*X`)K{oI-*j z{^-G67V{tNjqlms-4=LlMQw4`NA*86Zzmb%_lq#;x=rNOqo)vN*?($$AEK2l5!AOh zlQrXNffR!&2U{bGGg2uFajGjKz>5z(k|$%B-=tnyWo1xyCHmrPAMRg&xl!0{uR<|?Zet_)%On0~-;#znW?AFyGJ<~y< zk8g2r>kSZVxk9EozWIn>XPnAw{0mdi{5oUXG0j(r&IYe?5lk6IpK|8zetlJ5<5ziF zuh>GHG9*Q+M^9AQ-|1={p1Uh4i$##G_*^U-$y$^*(w8cm-0vUH6L!B1>sY@+B-Uqw zjdTMZnd^IKb4!)ziCNDlY;R+*J6br;OGnDA8Z#=oPkT8VU6uYi=Zy4mTY!uPnfSGV zhhz6i<=!aig(KyWXbX+v1&o=~KtOSWI!1f|B|EBs8FX;lo*S_n@W+s+A9d?gPX&l9 zt+w?!EmzjlHrx{O!KEgXCDgmLZTe<0RLhj%_8zQxTPfOJ`6zWoHOJwiv(@zPfSg>5 z$5Sx?L}xP_ABCl7G9lrOY4YcHfKL%~SwcHTGB7L8G1wtbGv0Z2C(ymZfG%R_mumd^ zEZ~<5?KpNU{US;5p_0@;w@3w6IX4>iJ!rkMQ&RvMaA&aN(Z{;}`;bB6t)ZSqd2%u@ ze#f?l!BR<}n|H;-2PY=@U~ZxP)GX3X<{gX!XwdcelaeDHd(R1)ei%9SMZ-S{P)+al z;HLp-RhM1pnCPB(LBm@evHuwv3;554{kKBX{%QOtm1ozh2c0Vior}^x0k4-rp8s;B z5wY7)D&b&;o03<2Hz0*ElxN5}4NUT#$5j@z^y&J87VY?`gI_`&{(k;t0q{*5iHFA+ zstDKZ*Usj;A*@LyjqC|8Tk+t6aglt>=zR(2eo~@iu*p{$d@C?1I;0=nln<*Od3Nx) zsSj)jmC;l3dk6p2@QI5ryvpS$JYbw4%*OXTBWgyOG`_v zfij5x9ePag6@E3j{I?lb zVMxe^H`F5$G7QhJ?}VvA8$VURtY{xfAx8IV5Dp#+A<=m)pvGT;#+95$Gy@I#t~9Ux zZGY$Yc`pnO#pHPIE4j|~f2KF;Krfg>3tOgwzrP>5#h?Fl1vtmeAl?CW8a$~cU;>|5 z0rHy3qZ)s;w(6yl@c|+tVxL-I(%H@(Yja0rq5|*smz=3LVA=^wfCivO_pT`2c5+&Z z(tfFZ@{2~5%8zLFuxC4%GB62_0#So|ygW1N>+4;vDn)Oed_6JKC}H_`HKdwAN^G`# zi|g=%n>)1>P`~Jk#T@;k+$Hl&pbT+V2Dv+_pusbsL1O5G7K@Wgms_-rucrP-x$Gdl zK9WiV%}{-^PRWKjD|hkE!-33RQK1)EX}fw%Yj@<<_F&b>g+bakKYUkQ~92 zS`7xG&HOIIq)_doH6G5g^aA8be1B0+*d|I^J;`HaaSUa|kG+%M{y4ff9$W3XytnY) z=ciA5A{D${n;Zng%P{LHs_Zy+iuJ%wEa^YOgTYh$aa)8~teO0r}!y#Q4;||%tYt|y> z>+I3p5=k>AS~Bw0xVJ*FnFJ^Y*1hq|=V{^Xn>+R^G<$3N)3My=#E~;UADpikc*P-F zPpO8R|N>Vxk`FleV5uu1)eR=PP zb>6*RPuV~AvBiXODsr(n)ygY8?xsMJ8(v`>-S`%w5T7q=kiqO!xl)oIYwALY5*w-U z3TZLoQ+O;^5fgDee(|#6{x%*DjYQv-grLIPpoDx~gwyoVwe<<6^C`YA^vQ#Tvjf$M z@wY&uy+2xC_S^U!Cv|b`NaU2+cAw|iQoWzpyBG#D!JO#96L&yAj`}RgqlUK3yodwC zTbmTsg@T2%^vqaK68WWo-kYp7ExE~oVK_q7r}<@D+k;`B)-TK`I~uEk=lq@6kyp;dV!k6LK^hA zd+Yl*;CHy)pDJzkcratlN1M4@@hf5hr5V|i?BQPWW^kfxEUB;xchMw9*(JMX>yEQk z!el{>vhjdxQdoJ&{2R6#!q0o#HFI;rW6*i*Pf>2v6rm2{z|hDrjqQ0yTsd zy@6x>wd8TxKYx;a77x;e6j-PX7Q5~r5!Tmi*-Q8EUiN|(rQU1tjctPW5Q!Yce~fjQ zV~M0IEdIo9d5|Ic)7K$PpnNm;E$JP6wfwSJv9IJ67z4Mj@9$pO{e4;4Lig0IKKcLX zk12C-PN1d?e%@=hFi&e!*q+T3GBK*7<>p`RmWB^q{hvc}?y$^h*37l_b>^7xpccYY z{q$U+a0A!dQ)?{33??@XH^0TtD%R3Y?t6 z6OH3U3;$;k+<*p^`fo~Eaecj_$vOQZ2c#}@h`iglrT9rrxk-TjA#k?xlp{|`nbB%_ zgn}})4JOueMKBpc_P3D(#S({SfIja}$)3BPyDB917k4y`zMaePv%X;y=E;3i=8#c<4^eB(WBVsT!Owf%UeGr>>qla_U)7hdGd`M% zaFSvy!@Y_6@A2&m{c%o6Pu2B56;lzprsolw)VZ58hYw~F30X(Yn6qjZGCVvo>_?3F z{GF!Ur@s|y_WNdSU3^aJ+vdz>$*kNQu|%8ib|KZ|6|2l?#@o7XB{jMyBgbsJDOTKa zu)xl6II_ADr@Q6($Mw*Cr5LuSw%21)JU634QFx?>b(X#7X8&xV5?%2744u~(J#a7w0--68WTcMvpN)J7fqJKnb%!@~kC zu-nee3MyvI=9#mXX;TEw`vV_k;CG6RzTypeKDA?B@W|7Kd$DqH?gOkByVtJ<4~O>Y zC-ZH7QB6bNo@$vh#9gY}?hDM8!YG%vEb^Ygaasc*E~dN|kpgkwK4B z3E(7AVvhO&c0iR=S662gWh0UmH-8O2aIg(+XJp|`@^+PG?`vPN7;yv3DX$_XBUe_| zEzbbAcxt7Fu=nYh3hY0&!2PUwZkv{)#%ApB0MW!j9*&FSrFyqSLDE^EC&gI&)lYW1 zXa!Yf%9tKz+S}%L1{_K&dtLTuGUZ1 z$VK4;x$c9(oc?~FR;MC-W{;3e%jHW3(V@dl!5$y!V2RIXW2KNwRk}=$DwZi7?Csz@ zkVkLi%81HNU7kr+{6Xne_0YO|WOud#+f?{p$(vOw51)-SQ}`Dwh#ucDlGe+C|HsvL z$2EC=Z=+VJ6I(%$sdXS5%2XgY>O{bT342C1Ax8EH!BRw-DzZmZ1SE{GH;D=aWre*V zN(6*33@eb30*)r3A>WiBqP*#gxV4t@dd^jfwUO)h)obFHxW$#R zFaeQLg8(kRetFIxBpkHd%SJ0y#>Zt>dxXF#QE+?zzs>(e2&xbfpQ*g{TMdKet~cAg z+ro@zAy5hZizTP^C8L{1lX8!n22K>o71f`q%l0*9nQBNTT}EKFSSPhTs=f_q?~&32 z#>8NR7d5jS%^Ez!BzSVZY!6H3Cbei^R>s(f&)8D#r~)memm=tW~N7!Od5^NkLU?+NR?VPZE?`YrY=&GJZZu2IyC)Oh{P;d`U$eodcM zkQ1LPXo_a{;Pnd&xz2G!fGNz2-L-nW)ADfUz?iF>Jetpo;f#z&Oqxa1A+!;dBbZdn zT5s2AF-^rf;BB0$?v`J2Gs(HN_k%>>*Ow@=F5_;Tu!4nPg!ujIe9HGZ>qxE!XBx>l z6O_KlW#YN#zt0%yFZ#unFYn3)&)l$A8-ClVZ*o0$Rqa`$>q?}QRs1+PSp_i!4onMg zkktQ_-^(&`pACFkB(!PJ{4bPBOD-+be|`J0-Ch7ZRq8-$lxWaHho21|eJ|n!VS@%h zE>9wEL?*=k`uUlzW$MWCK)B!CxHynG5?XrS1^RzK`Fy8KLxbUtQF*g_y`QLAz_z?7 z5U~^b-c$qJbZz&BU7wKL%c^k3O3q-H^v18F8?_Zo|(F>V)ok; z639U-6DvO{0IFwx+<2%bm@E)W-nesa=flQ|AKvz>6Cjk}*KG{`gkJe)QNIua=QWis zucmL7XKcd%c@d;I+&qqWKJ~K9VWXg`$}i%D=mnww;?6W4p5KQB4da$u_&T2BkIdG% z{~N#=0Jebi$l-MJq&M-$*Np?{5(Zj__$Dd?DQo-Ug;nQYh+k4gNOu)&o&Pg^`8%Zq z3@|Syxe(CW;7tMAy4g-O=mGq`|Kd6j@-%{w2P{86wfG|80FlD#A0BP{KW|IQ{(0=6 zNUYA^PX7g3+csVDj1262tDMvhHIoA`v-3iJgoL$^)h_eDarrH0 zvFG_Oh2*lq{~Ri>_5JsOc(8?s-H$((y>>x+nHKS1jLnM|LuGUxeExtx_7~vqwyz6S zzknEC=hidc^Tuw~K3xPl`F~MW(Qmvk?Zs=6Q!@O0e7OH%D{g_uy;Z|Me*h@k|9sbW z)Q8I%QgxIrR_*a=toUIez8(dO@p(ko-#0=(%fhajoc`Aj@!mM~XwQM9iYuQmqQpD!n#-G{36$A z?|cub_uMcpJ|&R!#`mXkV_}ycL-=(M_~qM&GWRC%Et~~+_Frtpiz8(&7-!;4eAHef zELXh$?=g6L>UMgLDyFFQm#$cc7tsIwz6LeyPt!y3f|pOfcfF8MXj$dNfSPa?I|pO- zD^?$01eo~+Ys`zR8y4r>nf`;_dBWQ&lJYWpZZernhq=By70*O z-vnMRblR7j80o5i^&|%ll<^p^?Mvj;lYtY>$`(Mz{ItAA8*BlpR;lA`;y|0N5d+#A zd~~#RsjYu7Feu1x2x4kQ6!*x(k@^6ONY2kWqhRVKtHk4;ry3=Y{nc#+s?q7L* z|H)~A+|wPGE&^s@D)bhLb>A^JxM8vqdNJ3jzqXZ{pKlBu^z7egjkM3hh2G9DHab_H zP-SX^*UGtH*MYO>kEt_1jFY_%nUiQhHzJ~K@MQ5WG`HxsO5Ph!&{7Vdph=KiZR$$}{ zFouUu%fP~M`{{kqM{Ytk`c-NYb`#gip4{#)z5jEO`Mo^1I@}zrgQ=i|*H)a`X98_` zGjz2KE$(9MQ-H%kONh8{*4B@Tstt{dX!{UTkhxCFP4VoN@pXQ4vvtqH8o zq=YIN7)3Us8~2wRNFA=i?~AB7ax34BIIMyjZGr6|$}VJ9;Y#jElr5i!QrnZ%_xB!@ z?lKkba-TtCi7+7-58wLykq>d#B(>!Bbb<5Hwko=LZ0X6RA$VL&eo?KgFX=>t_^(k% z6}}U0p`M^=+3%tc%1;SQ2wrw>wj>yqT(UU#>ACySB3VTx21Rz&M7T3=6kGe&uz#h# z(DyGBp9dwd2W7sy{+!|fvCVnRhnCa=v#Qz(tYPCcEMn@QAM$Vu%Nv%ix!BK{cwe>s zWkREhN=C~sYy%F|I4z|t`?sMh_RUo6!$a4waWx`#@}}426qVMi@_~f=_5&z%5WG2h zOeQ+<(my_B{BCg9FVL5-b*!E#$>BBAmH)IzG#!<7)BNeVX&}yG!cC_+q?Mh(s2jVM z|LIQ2f>Q!1k~Xy)%oL>tc>J%C6?liLqe~IgaT1y3JZ>~?V^Y%|d*Gz<7%gPc2 zqFA?u-s6~QZ``Yf1M&iGgvMmo|6I#-%^i1|$@ffK4g?CKRh5pTa{#-0U)B!3I#Qd> zdZ|Zi$9|>yduAXRyA>SHy=_DQn|lP>#uDzd+;2eGw5R|#0Yd*X=Q?OH>Rq2d#1 zJ(>cmLXC-8psI~;NPV$pCR1zfO_sw{YXYKB4ypb2x2n`jD{b-Jh=VKa<2icyPS70*C6dEA9SF=TvZG0 zEDdVN`RWWol{)Tqwwr6m>*FJ@u0(y`LoY1zo($M$qy`Fz1TDM&Ev~{hAJU_ij%D`c zS`;xfF)Nzi!9!OEsKlTBBfNZ)nrD)`=RyMLcj7qs_|9n%IB4+>wGOrZvBYr86-f>A zj`S=k_Sx^gr8Z@stMze#?2mhPcD!~)9|5TaLumGzE!Stk-IlYQ^xds*ld-o;-v7BT z_2(46g?oh{-O3dAm+y3n^J4K9U^*UOK6LKXKi`-kqpg26sI}9fIcwFQ6n$*RZO65_ z{$cM7nGdNE=GVMN3;~RI`_-)GzN0um(!=Oyf4R8g40$RZxa{O_pC}BAH1V0%3_aiM z9u7j!d4f}C`nlIPM}jCU8~cAg)($>)S55Y{P?*aNi1iFO9}Eu0bjR(}&>J7YP%}Qd zKyR1;z2mAW?pdO|MfpF^ONyjlyp{*>LB6=tZjfHbdqyf~g!dh#gqDC|eAfNuN#d6X zUFelIx!2h&^!l(gAFjZx#G&}Sh0O^CV$2>;ssH^7g$0Ji-Z)Y2@Z=AT!!9|&|KgNw zNdDaPivErx_5fK-3KIW^!2(K3sLu6grsrGpCHE%tb@c7-w)(EV{loX}zN5WUa(}P? z4_mqoU0V}+{=OOd`TxU0uMgi)`5V-4-?_1Cxklxk(a;! zi=KJt?d)#-<=Yk^yWYgxsz#Fk1CzjZ4V>8Q3u+F1EbDqfu9?pQwa@6ssCga-3%gl4 z_2#t;ay!3oPaf@iO*eX1!qy0)-%ug%ikoEaw5S}N{a$cCJ{On6Qh7p&}{5B_YL&gJkC<_#r zD8Lwnj{{Z?Q6zQhWQM$ce(_^=d1C9X<--;b$6OgkZR1=&y2StEuJ9Rm*}=L`@9_eA zBYBl5zX)WVHnj>^dotskjmJt0`eN$Ekp z3%uWk*7it13!a^^>tP%Bmx2fF`LqI}-$zE7XxRL|{YMWJhWO5XM2ZKe%f5ba{O2y7 zkC`{@akgN3Y}-q_yF9QDY{g8(_+sLF7o_g&s@eFUX7#+ga7Gn+uKnqt=~wAB6(BN% zrqgzJ-6*=qH}CmB!snsZ{H(la4UOPcDz@{IYfTwsYr;Of+xV>qeUjfnUPK>0eq?Ge z=Kh->3-3iQi9btG;0Y+wxrRjI^DdG?y*kh7fdkFw-p%BuZ;?m~1`1XBy`DB32^jY= zuETZE@N8;Y28l6A_n)L|d#seialHy`8qNS>&g-7}zDGPVSpdQX1oM?T<41}})f*>j zkAP6!PevxIGzZDY_|r~xFZfTxQAlDt0wo~^p<{%H0`Nc&x-Yd-uDnC7ye-Y{>x z;2*5^ni&BIR+|y3MQe1{7@vNouSl+xf!&d_01Czi5DL`5Qi21HD(*nQLZlpi*uGBi zjOD#?gA&yV9;L^gQe+LJgaI7#N8P8tkWD~Af?xl+@a$Ter|T`=6nB|#u3hr^k61`4 zE!!2@9}{V>_W9Xj{3SD2T&LB(EOcSa+}$KAW`@Ki$GqQ%0egRwW}O*6=$mk7xKQM( zes6TnoJY-r2btB|ODQW`1!iz1b*HbkSS0+*^Tzxlv43{N1x*lAht~3?%DRqcBYkY^ z*IxHV{L~hk$(?U4hMC*k09+y)Z{zJb=FetK;-qZEHLbX8c|q9o_wH8!KUmgl zl4Ghlj%~}9+QwZ4X2S|Gc{&{~Zj6Gm3GKTDuF*h7}agE)BV5Gt?S)nSWzPsuI&}r9S-GLveAk9J4#+P12**Ov-d7}rIv~Ys?3jNWVc#_ zvSLeALEZx4{c2`(5pFKBNaEkY;|YDO(AZ^1?_9yJHy;$8XLSGWfVz5!ESHKp0pw69!8MUhmiR ztXW`Dc>o$yQaLJl67SEx_n(*J(Ff1vvi4^C;E(l69rVt2)mX`s>-El*&h!uNwS(;o zM^C4(h>cp8U4G+kx;X@x=If4Cg5LU6PoYf9;!inRGh<4r&%6$x8h#8&?Nb?TZJwBu zFAgFJ@3^wgKqL@n?O?9+(W-`GR+)dB2La+uJ(jtqB^Q{5^R{tbdhE(y z`-v(WTDxZ;eiPDHeF_fPhK!$ueQ#k6zImHAGCuV5=fvoPG6g=MR&ozE_lfDbuh4W2 zRReU2r(=HFt(Ti-KoV6k7f}cxAIm+M_cigo*HoUiFok=8R#Ieh#5>O6UImcw{+$|- z#t<>}`B(e$k}TX!3tNM?@&*+1%z>3>`h+7h=_}boLWVMTuINX?4X7Kp;zN)PYq1jR zRN&g;w}F7Q~d9-t1`UaUxdb&AxhQXfId-C-)b4i9#{|>H$GiHNZ>rz zqW?`Tv$MdhYNW!mb(`I{H+r5#Kh$Z8w>A zr^^L5WDOHi5_3Z>_7zXkIoMe)<{S5H`1+oe7AZNj|EWs zk`UALJYy39j^|JT)R-OP>3iF#B%Q-(w7bK^FA*(3wZB$nac?|kTimA#=4ksbKDg*- z{)2|kKJ1=a>ZO*umB9<|Pi3v4_wrqhU#i=l?~|87P_KruGuQ!~1E^Tp>3TsG`~hsV z%g*l$OOf}Xf0Bjv6l)=*e=L%>X8mH(wV*Tv?|CYEPr1Im{y*<*XU1G8`OSi(HNHOu zfKJ*U=lXu+!gMKsU2qCC&IgdQd$ew$G{Y43Kd6hh@P2;E*9a}b{ru0@Z6^pl3fs&OfsPx~Lc^NBD_^&U!T??rGqxdJ691#nj|1mDWVBOh$Bl4YE==sgH|1b5Yi$cMU z&-5+`A^Cikagm$UKa~GvTS4BLMBnJB_UqqM?Ep&E>H7E14Yr}e({16TMp+b)n?Ea9 zYZ?~|@ulAA4_F~b2@xBVBx^%1-l|`zG#mHdiryQu^fHE=XuHN}B1AxBU{VmJL+9tt zsd#aB`GS*{wlCg1@BKc{rfmWN%m+&2Ju6#0VP@Bh)}CyA_*gY+SRs6uu@*w@@=l(P z5_%UDYte_YFKRroZ$%-@SN&lshP zP^FC`gRm=;$sw&+l1&g<++)9;-}H~kJ$zm6SjPnml}Df`t^Csz|B>LqK=gLe@~kWu)3>|Vs~b3L5}J)yAs zW>4R>fQ0f@t#NXSDhN9TbLZQ0X=;wrU3-DpoGU0OrWX=?D};+}Dji*X#1V8@T#9?D z+@-B}CKh{m*EPvs;<7QVSGR1&y0H&pFvGOzA< z^A;AO;6%=BK#xJBzXS2BX*?5a;>&z_z$M$Pt4DoT=ot8g42+LJXU z0=+Yd4Y=tgFNbNRL6SL@5g8{@ry#Cj(SIJaL(d~(w^oVNITE5{#3zPL5fvN2y9aBm zJH4x!Rn1W+sUPN-D#wU(ai&c#BgCKW9yAW`)&(D>O_Icr(R+`aaaxMMED&(1JwyAS zEHU5HAjj@^OIc;{fS(n{XY@YoDt6Y#^2F!VqPCZ%V`z_W z#E_pA#+M-9DA-N2J3Fj3P7O|JG z%>p682%W92%OyUv#8#gqrBRXDBVNssj;O(L2I&F6gh+12R${Wc*R&*KbMxC}ftD#s zoh;BW#dKrXA@mijFjA|;3($c?OzLi0TA|6I3yX)i#VvY<+cwnB4}{+rXCoq3vu<*R z1b(@ox+##@90Z!f=~^-sYd$k~t8CV$!Gn!fF!RL~^b{4e74lXJL|4+ER0iLCx&oiv z!GDE$%Dirakc{pnI}j;bzX3!p?9Mb%FU^#XUN1_T2Is`6Ji*u^>O(dcdM6W%Y<}Ti zR1HXiN$Dpv7>g{J%mhPteArOEtI6G`MhafS}y>EHe zQDdvAMRjFMzGRT6b?Drez2Da({CiaWOy|zC5VIS45g>%2nn%!S&7-bq>8DsK0 z478&iNYSpSjUxgdck0AT8^k8$!$=E$V?z3p!CHw5mZE7I9_^LpH4f&%Tz=CQlQ^wc zB^|4Ft6yfmu}nG2oS=6H=^swU-GNonEE~=*)-4gL7h>z+6tr^-IP<|lhl7e6wXf~R zay%~Te0$4Q@fv{8OW`)k)4N32Sx!D&yGAOg5PLZy3d}LdrFYwxCqKW0f#Gg&1Ixj-jRdbACyKpUP$f{+x`V zzptLjtaGpwDU@DMJ+IFB#IO|ud6Aw+jdA(yiHW(gCK#^)7B!vBruZ+=iDG4p;g&mt zS!Xk_2gNi<(t19gNB7-wc3pnnF!^HFp+^+fcv4HefS$+o4Y%GL1A4wK`wSsyD$d~} zTi;S*WP40NclCNYbG_<2_oVF0BL_=;H@rfZn>}Yt3X`VkuIbuoeNoKyqG(CLK2#1oL9R|jP=ammSnC}bA z_L&HpZ>lasUK;b7VTybmR%us1+)z@Av??%bkaaS|{_cP^nAKh`tQwcB8V|8NdiN%@ z2WOy>3`U|rXg=BWlOXVu3s6-d{?4-bhEE7$;vkX zs0oE+wbQdJSWUf5&T8%)pYzH&eq#F}qE$jtW_LE<6TKkOFSO&sF59$4ZS^y{oGlSE z6$o=MCd!5Ceivy()t+g66ywL%aDr?#iBv7(JxCM1yr$c(l`qI+o~h#b9ZeOx+( zyM}*!&TG2dnypx|CPmo{0q2fsq`9q`UxVn5hG;GCbJO-Btju1}L`;YeDr^K;VWsF6}OLUB6v@?xXabY1!~DIjP$UVuD6`hIDt`X-iSV$+FEa#0Q6v(|gpBra>w{7Ss}->1`(N{b`2*7r8BpZyUIKINR9eQj_~wzUa&2WdmR1%W?|nZYHPZg z5^1}|958FRe}$^$d2DPjXt74KsP40H$r8b9(#TL(W<65{{!NPi^sGN-+RHXNLCzGU zS*cEv&57 z$nX8aB@Gyu7o3)idO*VLxiyA?7mNm?qOlrfQ%sXHM$z=b{hrMi;4Y@)4W{Clp&tMD z9239VSLT$Fim*6oqs%c_cwuu?(%t^TS25%k!FgW8hqB^7&sWMESVdB*PaUAf(Ka9%67Qb}UQ*to#iL%wcr-aPS6R2!GW!bI~+nLV@U=$t8c?JaBd$qTd`aq50mM_uaVua$9i9hQ=+pw{vYnU3kkE_pYGWYQYEnnyOi9AtskwRQHnvPlRQzgS!dp+eQ* z?i^K*5OifMpd|xwZ7tKQ?LE(*;U*97TM zRXlUpv@FzivbmKB!$)FR(^U=w1u3X+V_~Jmf7*1Pk8v@Hg2T zoj;|491-+Ws8vcFj0D)P~UTpx~AFmfYsTVBZ`&_u*B|n#2l(Jf;3iX~8qgB`t2i zc2e})#dqAv@r%uQ=X(W=8rdiI4wXH}bdhtngq(H{u?qt~Jxs2ie0RpYICH49Li&u< zQ4}`}|5zbD_VGa)j5HRe^f*uKD2bF!g=}Octe? z7DYbDoZ~(hd4!^Wn(RQWNXSj5L{t>|R*(+m#R;~ip-AM zZcOf`zvQk>h{(9gYmVK6XR$|1Aext&x!inCJKa|M+eqIENYSP zorbmZdsS}8|8oML_DSrVCJ}aECc>GIR0-PXU*SeFmm)32s!&E~!-8>)dHeaf{E{UT zuG*oHPF5CUuU?cR%UUe{(70$COM1y$L;>m@$Uv1*K^2JAR$CgZDa6b95AxgTk+rG< ziimApz=$E5*g?88ByZrD!bE0cXcvxd;RpsDm+mxw2CU$M*J_$DMh;eBmT2IY!qi>Q zZn&FY`lz^oY5oNTXJL8geO8w@A}M@9`6aE zn_}ohho09r!qC$T65d@GQ6z1yR*nG{Vy3c*a}TL3{M8!HRU&)=$vWiuGD68mRV=++ zJ{pv3(UWsj*d4Z&3Ba)=@{Bcj+e{c*u^`+WH8{{|YRIc%7j~U`m%%A`z7~ z=OMFN)~gjrH8kabpxj1Gk7Dzmk8>sV*v+>eRo5+#FNLE!!$=m~i$Q!1nqHaTpKdq1C zPJ=r`N^%y|Ps8LX8XN;LO;VVEt{}gJzg;hxm_(d!M)j%AsMi)v;6KKnX))!jrImP& zkIZjIAL}h+1qoB6ToZfm$ar+lqp1~m&iK1L)hOd;yx_*Kxa)*8S=0~twegz;T57I%8M1xqLYgddL zjueucw2Y}-tY#6+D(*XGu{nqtWdkwoAj4b9LtA<$g2edmTl$oD02tde)ZfEVvL)zf zcn+mQQTn~95ZVXl1A7IJPPNQ`$d&j-&y0D!z_w1YsAhV^e-iWpyKYwD2K7DAYv$CE zDq$2fS3kn-)Q6{lqAHEu?&k-#r?0q5H}6dmm-1c~DyKRW2i!8MaQqs&n$ z{oF0*TNAbF`>O?~5|RTMsS;?hkR#MHQd;xM7UphB;aQMx^(-o$P=)juHq%w04s`sY zpHvu$c3zTbT>HC4N}On2e=4Zn$-wrgNMNrR<#t`jNUpTr<~LGSqx)bDZLzKjdNB%N za%WN1eW@U_A6V3tV;W*@PIvKx64>%3Rp&*C_%lYWJ6&h9HeW_aeXyxxuh%)O0t8WS?sASB$ zZq%Al$WgH>&*-27t4cbKb>q>v@=@mxw~ri2roFQ?pIknWj1Z{9UeMFwf`+ zR-glA;Ii0nG4u^n(D;>}AKqOR2pt$asL4G)ac`1V)&hN3;0qDJY&7P>*$;h&sW;-K zTgqaCosJgBdY*Ik4GJ@OSzwiVNNtv8nPIRul`vqGu68a>w!FW3J@xk*)MAeFj^gq~ zTz`I5Odg|q!KpoYpggABTI2NH__r3rq)U(PZI))!{iL`NWe4=EPU2uiq8fA&L1~ z+ubg*c#r4Y>(KN_0bVsn#}xD^NXGKTq)EQ9$s}A7`82b9yfX*RVmuTJ>re0!mJEsXVou$033haa9SBtoE4bu)x{|Lm8!uCfAJpF-tN zL3Pf^nO0~bCKC(eU?3j#9EIc@w(#sF9W*a47_#m`lbqKj(_G~dKGE>LbWwL8IPpc#-##4L!_S@`aHQjael{tDp^(HvoUNqj`efZ%=?_E#kF^ zy5>QnJA(j9i1hXFtQT+?l>BXbZ;}=Hq_nsTrQYW&Icv& zb;QT2xTIXJ@3}qLW7*w^Au4N#)zEnRVE~OLTuG$A(2?`NFsmu|$LaWsKB1sWemSSHIfz%+~Q#N?aV zd#ul!53_rxb!)mgmDHz~+%(yf(=JoFdPJE@&*1?W5^iM_*zg8;5 z594p`#d^5`JW_9~$`nov3X(i%pM7$xxS?Ney1e!@f%d-$OF@#1l>#I`|Xg|+A)!E3syFV6)20`(yg|24{&flx<(Ti_5e06Y2R@R);INj2gA7u7TpW)NAcw>k%4MLvAEQ$>6cQXXX33ObuK)UMGwkpUSvl?v^F`+A!i&4ZDt4ZVLzb1(`NI zV@W;Y(qm0Zi*H-^7j1cIgr{&g@!WJMUz@ZpdWcHr6T84?0|tGCxD0XHpT z%HBxzpH)$s_*|w@h$XFY38)*Zg%$}37WyWCO0Km#-=#s=u0~NQs?N=kQ*EW01a4Sg zr+z8uq+Q*fUpekGV;$ogSXN??QELkKh-T4RlnJh<|ii;7={ewJ4>J@%k`ywrM)H#z=R1h=Cc)F^!z<3q3*s+KW z-mnit58N2-(4KrW#0_OnW?qSM=x?2(3(SO^;LPn>%GWvS7=X>g8f7pr)V?Z?Jy2!x zE6IJf`+Boo%?4N3nn4@~A1-w?PiVj)j7Mj6SIV%Iz9EW%xno^c=4Ri!!4`y+ZEB!u zgH%sn_7uZRbaZ)24uB1b{E`JCB}B!3^0oYzDv!-b577a^_JWb6EwhQLjU5!)_G%g` zd;>uXjWdB`_>Hw@e{A5k^~hJnXd|)oU1_Kj1qOA!x|V($i6K z0;DRdz<^8)YL1&6tdbYsGEAoQFq495dXt(}W3}wM3r6#*L55qMB}yI4Tm7Y(->P?& z$td?S+f3zSFdM7RWyg&-r4qqPM1I#^#_RIz(%wAsN zgA`b^)tgDg3Hw{K>9!k4N;EGNQxT7lynkI>i}CgY(Pk`!l`%{(NLIH0pk;y)fNb1O z$GjVyjI!mpSq0BJD&!g%jJ{@kz+&k|$z@BN;T3*SDyxpjZ2_Kf)^2plD`_dn0@M8} zsMM?KYp3=k5EV}xRjSdmI5+uC+HomvjW$3MCV2m;(Rm+QqZ7SO&By%881pisZme&! zURRWY-+l5goxLg+XVP2)Za6n=Dc(x*x4P9*Zu`!Aj(Hi*kcCI{A>*lpjrk-(Y6E{O zCM`f)IC#nyKHPKdz2^6NOkZg#C+CESnyB8SP~%N#|FdN&ok+dD=Yx4}Wg!3|@c`J6U{X#trG(mhJ4IpNfZ* zuUgQrx!I95L@7xkIPTN7lQ2C&Xwue1=7XVHH&A)X=>0m(LX^l&op=*x1v%l_AZH+L zZ7Sn_{Lpxnp3Pd{kn8Moc95&-*y^jP_czgCT-oTH@%#qwcA?IWD&(@D(-H7EMM67}5gwE`^)f#ifGH;^IbK{9r$ z1t+n`G}+6VG4#R?rZpkhe2YOwx|MUg?5b(nz;7zD*ZmxZOS4NB!DHYIWlZjHO&EU>7MO}oELA#4_ItaT9E|AN*$J|f+` zAIq&z)<9TPPW*+XJV9XJBO-L^UH+{mndvcWHzIUSczVt|!%i2Xf)@$2#SNbQ0GK;sr3qZC*(v^p{?&Xw{gnbAfrUFdB_U zH7bP&0j3`bqP_HjWwSp;dlzERwV(32NA*0?iQcg$V!b&XkCGa`>;{!3m9Dq&oR1i1 zgh8gxX&dGZL=Yf{B{j!_^$QjN-pmh#XW>B)?ne?i{aXxuf`sYTx0Yd*6y*?y zdivjk%@6n&^|YozNk@|s#>?FUQBMf$7$L34Q=drdgnS^NS0?qT&ijj`tTbxql7Szi zGO-3b(nkD8OlF#uLIU(x)z^d8mZwnHs6#pQsoodUOH@32RFlf2AzLT>hFK-u>n54O zW;(NvjSie|{E_0)t@@(A^RVk&dbe?AqAL6Bv&CKk*-bRTre6@<{MPi(vb_CKv_1u7 z1tO#Ni2>*rN%b7C?eBZkG2)_ufah1HRhVjWoX~>#6d^hJFAGz8p%tSrsJ{Y*AmSox z8vaIGW3#lh7L5;V&;mVEYx@0hoy`~GU6u2hMYT$vmi^B8^uCzF#cDzFwSI{3Xd4;I zu{al+h>zdNm-qgvN&)Op!@^AyQN7BYr?NKDj}eKA%!}gn%%-8*O6FSOt_Bs?vFVMz zbok_h4t&&1k)FK6)&((WQf1RSK!p|Rc3ghwYWy(sEy&$mX_3OKnF2G_gB&z zsFUaxMzczU0lFP)BJ(`)(r;XP#!|CUk@aD8zNs2>@v-mCn+RIa@k2UaKW%_GS`;W? zONX!-0ok|};uao7RjTCg+*2u0w;G@glnnXX>Ko{nRnljSQh#lcTzTcc;2whf!1$|g z;G}+1bAauJu{?U?rF?beLUv|y>7Y%CPusvyx$CG}iuuoZC?c1`Pm0jH=QMB?QV6&C zNZZ;^5G4XDJ^Vbon=L4ZD+ zT_HSScdSR#VoV#hhWy9fZ5s-?B<3K+-z1n#Atvnpx`&9@7)VnNdH-oO!`!W?ggl$x ziRF&@kET(yvsy4k1;3y|?Li?++71$pHNQSO%;@$mOedt=yU5nRm}aBBY8?Mgz1GE%<#KAqm&W9 zN*H!;eJ!Hr8C+miab6OT5<~8DeaV31sa~pzf=`3^^)mxI{MDWL$qbyd^loE}Xmwc5 zR1vx5H!;9VEHGcINKwscnNxQcUCQ$@KdaK9lzD$BesbEDTdVRAMXxpGde$#jC-G-| z{*hY-x{Tp)`y88#&Cfl5riVOQDpT3(dAPuQDAx*{E3sH>XGOo3KDm5f3s-ML@Z7vs zh@%gLM@7CXniR2Q^I1s3QRa|=Ws#_vH|BLdwAWI-tuwW5+$fqLMS2P?kF*`jUEO6t z=dW~nNr3Gro+Jn*nKlTocv(Z7r|fLXsAd(IO~@BSdC$DgokCNc&MCAUq_fd(id|rQ zKx}}GJ~NB{Qye*7p6|23R~(e)^|kG!WU46J&-2kl=F8sGo)TXl-=J!eu!=$ETc6(5 zsN~L!RJ$#ce`AiiR6E1lnH`V2^-yqaxt0q|{v*5GQy+GVVqH`gUVNnYH2S3{qMYyc zzZb>U;L&SKBtSt(D>4U>O()wDoiAdYUY4)wx-6xXO@54!_!WtvrSYm!oqd@H3_s`h z8lv^v2_8$;98wy5_GH}McLjY&Tgu!!pCB<>&XZm#_B$H&gEIskVFz<_r<^OJhG9Yi zk49V6`=dNYs!!x!NQ4^54Y}jegXDs+N@N5x^L^~q@E(zbIar1 zLGy|Vc^l0xDfh}a8C|T`+lycx*@jjd{TR6|_}}mSYPW*^I8f-BY!fu$l_}9HO5I=r zFLp9cN1Po4e?Ia-whhs?>`3d#RF@86K$YU)nHcYZtCH#Nrx$ubtVAskNIz!Foo+_0 z;DxVJ=K4!KJ5o=yoD46EoIy+%4AWnMRth$2t)}xswjGSEzCjLaVN@#<^Xm-ya~r+u zvVhZ!;y5^%m*asc&9f96H9)2}nu-VAs^9XnIscm7=M>0U^y8q?*#g+d5n_nfGXiRr zQXd4|sqaGgJ7ZKit6w&Vw#?Rbxb`aRo@m=eeo2uft%U`8LKFuvY6io%dlF-=5raC1 z{fPb@`LR)jAh>=%>Pe18@565=N4+)6y=3vx1+@EXR0^ml_s+*fC+6=DgF^4g;`R@y z?f>vauP7nR#0OEy3WUBXlI^=QF@p=W$iFvnA$|@9T}X_(Xb#oNm+k6bJU{gU+Mk>j z{;`$nrg;ZMwoXfFIMB{9*EjZBK)`qRCa#Id!oVYVU(x~a89jTZ`OXbO-b}%i4PyWI{G}kyJ*bY zu6=gAx?vF(v`3UiuAM6ViY->W+WI#TsfF>hxEXQr%a>p$PY8r6Sl6lnSw9!m0+R1q zh2+Zi+3#ZSG8Za-P-yAE%bV(r^}k;~4=C;4v(wP7)Q^KVgc<-UVo=w$7h`?IG}M@5 zu+ttRIeBesx0F2X+7C2cCE z=_0>`pqyyS^eAvN^CGXi#{awqQO%@B=N_!Ya{aE)QAuQ}- z8~&}IjiQf)a4S@5x~~!z?@fVIEb5l|^`dRr(*yn+PaV6pLq-GIh%46uK{jqJhn=DfnU*ZJt)GhH zDCZZzs`NI`Z7%P&jj3B~K(a8{wYv!Mc2 zQZqLc!UoECq2EM>!dKx@j^ZJg`fEmkv@|#=dX}C>8T(a}GdaDg5w%H1`--EQO#bTp zhTg7QE<7y9jT8n>$gU1_F5?C*8k|QYkom{0 z${3~r+|)JGLEns8RDx}r)Xa`@hrP5r&m2iU?K|cey?}A?y{UbXM`2Ouv(dinUB0p+_bsZBwWo^#`+igo?U5180*QT zKaa>Z6dQlzmH45kg}bYf0IL5VET*m93IVwyEsH7-GgU7!i6*2rwN{&s~A<5ESvfK+zt6<6o7Tl2W&c@+S+>w zNQWP`YsNC`%OeEmyRed3OMm1FWC&e^x9W%eI3fSnO~SsY^Zc3_tW#(kCzqAB%b@GU z-h^&=r6s7PWY`7pO9a9|G!d2(-AsS~?SR|{_Zwo@x@wgyt23o2AW3*?z!+maVDNqs zIUs|z;HCD%HqY%i+(ubeV`k)s)UNog>#XkOw$rsZFj4uGO@Pjq3cd5pEAqPUX*q8K<~XCkk&QO z@9=806g4x{#Q912LL-?GRq9jbP=XxMOo$N3Lcr`UDjF!0c2`oakbd3`bWc!Cgocka=<4p6S z^Y`iWaoqxMI~REfxeF9P_ZRcuauKT9YOdZ%?@nL02I$QWUe`*73e%G=WETu#=T+>o z6wt3%><-AFBucz&Maq?B&W);UX@jlO z>O2Nx+SejFjeF}J3iBsfX;hwH*7givUl=6^%$bD)apYIv_taIt9s&$;WB)0&t-JUQ z-SE!e9asIs2EJ?0Vp;&`P1tp{-?%Eo-1VGonpcE*5*|CVlJ(bixo55rR37)!1P*sp zKH{jMqJf`?uKZu$2&I!ZMk{jX9LS}65+#%s6Ypj56wFDz?!110o)L4;XZg4jMx>?7 zU8!UBmU#BGUC%Nu%{p)_&?Foxbin`n6h9UCH^7 zBTL;_!gcVk{l9#lIMh0cZX6my)m~Uz_lmSTY~W#i?ag`FJnbMzBRx~;OmJ7+yv{Yu z!>BAX%F0Bd^wZwRO zUseYU^tEDhr~VkYK=Q=wH^K7Yhc$sMMGsuXe!=;1-{PCWJUMe6j$*1udc?Z`Vn=Qf z5X#jTGx#70^6Ie}XdqySMl>{Usc^u8rr>;|87tMGgYd|sIQLQsOPz_9TaKVk#E6zp zh;9}S4&Dl3Hjw9T1IEGNa8{Emj`DV7**G}H3^7#dgRIa7k9~0N7QAl=geXT$%Wh2L zSH_OuPdCZ?xmKN8Q~zZgueJ47Q^WOtRn&0{j&$u{8D)i@OF3q#7r&gsy082lr7;BTE{XpQq(j<0y z{U;ax+}7?eTsg-UZ!);t-gO-`ARzkKzI*%tV$Jhp$u|qPV$j!8KV4dKm0y_F&l@5Z zaQ@pFER_YHvW?O?Z0?4?$dR(S{JF5|g8CW4BNdTpB66eA;d zEhiIX9xJ<&{KpLXsaJGe7pm+CHJ{b;=>d|rS0?~-#h__1JWbx_=APAN ze)dYKdUj90X~W{)p*w@sS@G}Yj5OQ%T?h3u!CDjcyzaVbsM>N zcMSX8D;>KI4l8dDX0zr~^Wn&bgHB9TgMJqdKIBiy0i|HZzPpTtA?A>C0Szh^?cm?F zHo;%u9o9}KkqX1}x?-J; z|FMU$Tk`o*)V|oU7k`q+@-^H}@@fRvGME9)oV6bKM|?lPRQBge=G`HvVE@4+j$ICC z=Q?h*9b3=7jTHi%n14QKc>?5~{kd_w1k_9VbEA6fzsQ|{#|ZteKSptOc6J|ELLlT1 zy8uBJ;EuW3y5}Q*?4$eU*YGAgkB+tN*n5$~Y4<-nqpR@0v1fn)zHilr8k`An4$f;q zD$W`g3^&E{Tx>jX?F;yhwXE}?IdZTAm(ohZh%Dfb+NdVGGgU-`QI^R937_TtF#(ja;Sa;|=E+zyuZ z>(}O(0cAX5q52GIJ$n1agZ@1(PX8VXzz6+jSh&8#|7%?TZ~SC*7)Vl#9epsBlHb`J z#X7>3KnLi=d^T4*4N|wulnB81fc^?RQ$5mh>nZ8iDulyq;dX}oN~))TK>nnY(Zqb~ zZS`!U7+m;^>n(nd@b)XZX##XGTyIxy^l;lpF6zK`ntaWE>!u8&eLg-u?sRT@r=7Zc zZABhHnq(8yXkY6-t|X;=WX3uE|&rmgjzxu0Xvae#hG}jq6pN*jkIU_Jm zm`l`^se`SV#=hfB0`X&84gA1ogp2b;p9vF#Z~owKzr+oWu6ne6HGBUjkS9^9~u z2i-gu+jNn4yUjYzM@l#M#5MucFlHZb@04jvxXdoMF|MFE%`3-sy5d&^Y-^ccj?+^C zPO+<|0Z}8fj2-HW*l148vCw02qVH;iO>AWMy3wDIqD9VdW>JC}r>%e5Jl_eFa97rK zg>-Y|Es(?!?0~K0WkYVa#nSDiK^3Uo&BgW0+C}#A!ToOZ9a4U`VNOR2&nCYGA846e za^!gTRUxt0?E3jX;19d7jmPBn?~uE+Z>XmYy+>^sKk{^`HiD^>VO z!8{s ztyGDBvj^HY%=j8dB-^5L=~A33xcA6giz}(uUyuDkOC_gBkXavEspQq}N{v^DH&ZLF znw%h4w+Gvz`BtRHTgW{x3BSb&9}xfwMFUTGW!E8O$~7#}1^3+b59b+eoVs8t2m~S} zZ%Nee5um$@TPb$iOoVRR3Q@>=d^$}gldDprK=p&nSxy&zvggoO9r?KH=y~C5K$F(u z4hOfjXuAiwyt0>YJ;H{E*w14aejr7$e;0d`qZ_Xo-ku)q=eR^+Kr{jSYU5N_qmwiG zxrTMDWw!NrqOOXYD^{vYak3<}dR5tVGdO5@l)pPe5$gkeZ7^_DYkl#KkW=I8(##h% z(vc?psI?nGKcO)LKbr>!2WRF-=;sxw_|4A~mgTN1I@#~)@6RIo8ujjdpP5;$ipX+w zBGq4!1@gMs&;QdeC={9njA|d1_dlO}KGwAVj%QgU8+lEvo2h6Zw{Rp4)(YJ}VPW0O zz+1U_d1(gfsDC2a_QCvOu?b$gfFc{;e@vH|!ID0lUAwh9ASQNee4$AAk{Lofewzj< z;W7?TqwtePyZ|xSc%rd@T#nvK(~Lp%+(Jrc*Wf*z%ij<4aP0w?J*tck-mEz&(ozK8 zoTR!h-6UuUQIJmb)_A*Y{fMdV+KzG8NZRi^R{*gL4mYO661vE>66lI$36&=T>^D-? z-JT6fuPRGGffY^;l02#X?kdmJ6T^~vcC-Bb4oEhj4map}hE9L}rF3P9c`$8>;+j`y zv1hgGcQi)7c5Ob@lBB=f9Flr$I+%*x5R^#MA3I(<%9yYsV|?fa-;OpwR+mCF2Epgc zb|$MX7~s78YF27qtlB#Mb9&aMTTv`polj=0DUx!kK>dDLMy;1(!Qdsu&wHg;`{O*K zzX_Nh9r*ZI+S}H4Tq?fLn{o>N{P*C)-5MICsLhd?_9c}x7gE5luo5zLb!#Sc3D|jcURXkSfXRCDtGK?n*~i}Xso9SUH@yw zF`t5#89C9{aIXH|1PuJz>|#qPb!8l(ryzlJ|hda;dtP{vOH1%q1;E{k;@CF%w<-rd<+qgJ@5 z(46aj&J8oQ@BcVHNX9O$S9hOQc3x}$krjtPQyXbt5#8BT%5>KpU7Zjzlsen~n{HH- zqI;i+u(2~UqnvZquLyNQ4%<0`YU?Zln)(@ie&-?d`C8ZEy40J@3LVGV4A+5sjA^^e zQ-W0+C8xjM-CUt}i&So=jc+xfVST_Ka4*JP0eM zI$=)s@2d%#fme zyN50XS7M+B138Y^W}%+&seToWc-FDi<(WaQ!9Gipi{4gtt&UWs0#;|MpG8^wmM?hz za@u&a*BeDU+iIWRA(Z7w@F$AF`sRpvkbj})`K($>vX&0ci7-rJN}o?NSo~EKA&Zbm z<2PUyTRo`=QJG#+P1PGGVf1?$wDxse?VAnm`GdN0+Jv<9p@Blmq5^=EG9qznI4 zM0FF<5%+ZYhSRCb=dmT;*Q;FEdTLu4QGse1jPmm z8Ib9lH6@0k8F^BgLKz7CZ`Z67+HR3L)1snlQc9km_ZRxfoTsFQCLpa>ON2r7jO&(1 zU5`>~I<*n}c0^Ap{q@BjQ;75XmqU_vil5`v?lz;m-iW!bEp||KQpd;3uXzR>CMNrU z^0Lo13RBb{*`)7snSlhbWh0W9}L0bd9D{-TC?mXNQ`W znWa@+w%+)|?hodH0YnQuL}X&mEDKkf;nnkwpTM6l`qpzkl{UN>VX|W4OZhIUnC9$5 zW4`!Z6&370uD_1z>>%WrB=rW4&phnY>?C~w2UrB99eA;`yWeA*UxxnVhkG8@bm!@- zjW^S$|4HoFbzO*$wXoEi3VQOa#meQd0+??mRa#om&Dc|pU?L~PP$0`zmheIo`&u8L ztcZ_3RwD8dPJ4FUGVY(r2ZAn;_|<8=Ws0PagO|cCmoRaF&o1n%0{D#7)+B0b3lrp6 zHW$1&NtHmR9UTdmQTl{3r5JQ5mn>>ST&c1xTtjyQsmz zm$H|NKh|@%L9{BD_#y21-ijijPBp$MNpYl~=#pa8Yw4g-ekLkK6&xMqE)_!|POY@o z`(wX4{70$Pb9Q+0!Q}8Q7||=fzA=*WBkv0Zk_=4F>o{bPX0-2;UY2Wh5M|R&?FMiH z*jnH}{%?EDhF|8A>IGCzfJdDtv!d4*TqN+yxMViP3_JQDrrvY04(sVJ&N$d>S~%u5 zF89G!Pr=AI>B?-#aKI^cW&J7L-0k%;7VRsGF}&X-FB(vsfyi2urvmun_2gv zeMu`tH&T;N)K-DmPVoqC(fw|Bh~lZkU1MD{T;5HrVjagA_;@R63-(k-!!oc-z%ht@ z?O0$1a7~ZkqZ4sH3)5BxQe#u>_EXbTX=|hK+!cgX4$^wmlD)3OPc@=k=4$M<0+ZUe& zpbTA3J&v*(D=&LF?5C(m3eHW^9Tk%U`IW`%hJv*OaR)4rA+_q&&)j2{yg$!+UcPeA78g2Jzo-olH!d5zpGVIOG{{!? z6nxaxVQjwEuHoWSOsTI95!5%|@^Fz=(0c)M;?ZMwgh>)KCqUqC_40>Aw5mU=%U_=@ z#ck6>RUAI!RIPo{@JCn1@>lb?8mZ4<4U-4+6PKkV;=dltSq7|1-#xJRf7>BMZbd?C zg_47h$s8hTvbUJUlX+dxjmWzX7Q%69=Uanfn57tv)ZgQ*1?jTP;JInLY=@>72$JnX zUq!}})Kt`Bp|l+~-%)F2?Pv{-ytJIbk3^5>ze9*npYPP25+J4kw#ct!_1ttacjZ^m zY^kragYENYh68}fm>p!N_$D`ImfUw-OeCwBIiEf_XCvgzp5^MWjnOJAGMcwjck>g; z54-?a5$8@y=ju(Lx4dlP{=oD22{HGLu8lNburhSSw6OGsxpGCXt|F*#&QIul5@61v zQxfq8R5&kGvCaO>T|(vN!ZjaOMnyc~09BV?XHkvd-^JZ~wX( zrDUdm?Fzzn7w5BvU5B?ZA^pLS5- zWN$YhZFG3}u8_E5+v(DN>$uSazVFVvyS$h)Omoyp(D>jT;Ea&0ot$j;peAzy#w z92U{jOmbg(sc@Tb8&eZVW5YGStl_CP`YH{;ZW;qv?r)i$ux8D0N$z~!%)(03IEX&0 zeyaP$@fdaQss|ITW!)KrKtV%6P@8e-*v&mPn@ekJRo&*eqr(Ppx#g;~sM|KI4qcf; zgTuFUP;V`3_)>a9tIa223t06bUM1rapQXil{{t1UvWs?|ItV*;u)=uw*dvQR{qxRr zM2(>y^M_q_1RWETzY84K>*7Rm*)wSEQRTF!DMYOX)RUn`or+rOc1`XOo~Q>hnScFQ zYj3(YKddJq@_o{NlVKC;??fUr-J9!6%zF3P@nwe|LEWFY&?kG9`yA-M;~|cy(&qz8 zyG(>`)ZaTVAo4C7uv#0u>UqCMt}ySGO}yuo1WKXV0OIP#)rMn zm>vmIDMzxFe?+--m&$b zzNPLc!i}Vt_deErB}xI#x7hI--!DmfE$3Z=Yo5a!$QSA(qp<c;llcnh)hu?DZ8Y+8Y!Vr*2l6`8T7i z;*~D((pGS|4d47{;3o<`RuITZJ;rDE ze9;WAt-hH`-Juo5gcQ3lmVrT3mDaG8m7-C2>7zmF7Gvdc=`|5NImOH$;`F*L(O^6* z2dMO9mVADOvbmw55feQy8W%hJ}FuQggdt>nuT1;hqRtXocBaw}6T z$#`Ta^y7#%*QpL|YR-s~)r3D(Jv(53qU}C}Tx+V33I9n;{cmAu`e^;TOVC1W69aPr zO%o_pDB6L9Ci*@cV#Ad(MtmncN5Y~S7iSzLkpA`LHXGz?@G+BixMEpZKHdsaj++}K zzsx6wZ4=`%ImywuOqe>C?jD08LaJEP%Oy~P&H(krdP(nr= zo6TNe8xN5(npy*Q6TD5R-Dl51FJ%-prjQwoyXs1ZKsm}z4^R!!zAK8r^1P_O3xaWY zLmjPk^+bFPS)A;}r>#Ut(DSpvp+28oeH-=f|H$9}#YTT6FM6haC`{rbsw(1s)c65& z7fGc>3P;*4Kj$NQu?xGnR3C?qhZT>oF=L6|*+n1{QeAre`(BBeli-lNu$lUlmA=0h zKTo9j*O(;BIPB@R=Bcb~jJ&(VHkVy42UqrP4vJk?QcAZsfWr!9%=dU6D+&1zKPGBv z1gUX3ZPU0q{k;*PX(d;jHJ_Rf^Yz2fHV!(_?2l`%j8R}TUy(P26DG#+m<>vdr(51J zfvnoJlzMRyR>KVK-EVr#Pb_v!pxZxsDSHNB2s_F%0H9zdX%CugL zfQGxt=?GmhgAn_VKk^j1l7yFDZP|NdMv(_Dr&YN%GXoqqvi4d=dd)BK<^Yq1)oJ7f zc@LT?y(~>w49UHEBU2l&a7v+iz$EE@EEri@tqp!+_|+e2nh3%+Mi^}x9cy5)fjXD^pWf9Unir6^0CM+`-o-~GFD z88~BH78#hdw?mevb{sx-N=#&2PEv8crXu3L)eMBMp#8X!=WJN*g;&4d1c{K^S3eT zTp^Zg-Muj@8ywK{4{itT(&UQBHrUJg$qwdx$OHgK*LNUe7zfZz-}`rs&+cKJE>dL7 zwl8f5kQbJXdr)u>tjiAo_UKi_A3%2-IL{T<#a;9?Gcz;d54|s-IoLn&eT;s zx-)eRal~Nw%iWcA-XGQeTZjG07vc*46BEp1f4|f;3%+xrA#V)=4xCWygGxMq0~kIa zyYcPg+cp?(Qp;tR6&kHBYP#*J(cDJQ8x_EDe*-k}_L1^&qL3_{_ZyB71RisQ zen>V>*zn*bHm*%#t@my_2NO88-N)tIJmJ$^r<*_gT?EhEmK+D#_V*gt$GPQdV;8b8 zq!-9!Jp-7eUK?<_#I@s|czV!i9IYk=u&1AH){UD9-F8M~a(V^0Y_4SVGgi1xD`S_o zn?4CN)gt3LwmtM*8|-^S|H6=5Hc8b*W(0j@yMgV&P@-vbFITMhZucYMBF7PPRV08T z8^+H!eA{o?_8(3wgYc;m=7m1^MRj$>qJTr0R*=jP^4)p51ldO}(!ZEdc& z)Bo54)DECO;mU$?>l1=8;yhxL)x1URO9TR;i2Tw^k@hLjai3wq@BV1Cc0#0%2eARY zg0c-sN5inauj$&GfxhV-!{{_G^lPMzlq+$yShDwAL#hV>0u4HrQoS5StHW)W2SL1v zB?uy#UID2=I7$SC8{q5-!K*JVeM9>ZL&;%Z zUaPdpqJq;2>fTfH1Q)!vGjSH&90I|SiBNfDZ^1#^FiFCUCov#2Hw`RPf<~gJyfDRt zWeu;$^qyJUaXOW^##|?KbZet=3$rjnh(0-1%NdJbX`Efzg6~`b8X|vNNRH=pm#Qf- z9QEAziJNhAE`t^6i=z8d>ZDMGy*Ho zuSU;EDy!5h`_L=~9n7E;c3DEG{>HSU#S}&s#T$Q%5MX}jk!c!wX*Ae1qiZQkKf9)! z6=S~=L(|U<-rDI8#}K#WtG(pY^Gjc^nEu}RuA>L+`d=rk?TZ8&2A6ZO%C&0ds4^!< z5jgmFVbR&mU(S4AUVY45XI;I}Oo;BPg0kez;ij*f{V^uwx-Pg4iWb7-+!c&dxH;Q^ zN8@d0;iD#{Aq#RmZwND2AO_4}q;oZ)-iNqV5Cp{g7I1+eFy@1XAs<$`I+U=O0Ygb* zSlF1An1!`(PR%yId z*GkwaINE~g@G|XWb5hU{3dR`rCVCSsqsP0h+t*DbNB~*yjH+OsOu1ZAWIY1|eo(B7 zB%k>Y-+CNIL_Zw$ z*G~8qwK)!?3$P53Cisx8?FH-_Yn(W@&kJNUZ;e}#H?j;U9mA>TS&b%w4^ zA!#be%^NysLUh9;>w&luLXQJJxVO_q0#|V6HLH8YnRpq}h+_7KxZvE&AV^xZS934P zGD~bg*|oVR4YvXK3Je5cg&2gRKP6u@(u@DowL`J+ltLc^<{1k< zwcvP1)9Sz6@9YZ=j*mBWzL{TfrdCOZw(3I)mAs2z##N!BhD21egj=)7b(n=Jbb$Gi zfu+xiVRJ6p!-qNzBU9_PkaU0(h16`G1_szVtcddLy#TnD*g*;{*FqGYi*~7k*UEh0 zS`^D1cO5{6kd(MmGs%z3>qtXV$3;P_HB!@yFG=Ve2y z;XtxK!N=aQ*5^z5Bl~JUpMKP6Pjt~fQ1^#1hZ(bam-=f;y617?qgy<g8;K&u!6* zIcR5E;~!I?lY^}|jl<7KFVsl_0K@6R4_3~O5YO(<{|B%kkD$$(OiT-56E>KImE5dL|Ub$TUzi1^H5 zcTjo_u%HZ=qJ3%F^;^Zn%fquQtUzwY&6=w#@*H7oaF%T3paTD(ktgamBiMWJ8RM-5 zQ(N#|4bRmsu4~O{4rx`Qv2dkyXwu?0BNNC}cF48r8`n%(J-v?6dFVrRS)l0si~vi% z6J+WP6;;}uU*n>6nb4sMIaR83#nvyTpMt6)6H-gpn??W178X~)!i2T6@j3UlfRtwy zd~ff~=*?}=Mp;RRo2@tKd4aG~1KL>@*J0ANk!1kO>&Dh}^ReYa(>$KxDkFGrjv!-a0F&pzFlm z0tJ8P3;Bd*LQmVdp!e$IMQ{vRsrav8MoA@%mOJl5g@H2B1S4z6$8tjRz{-GA8bRF0Jx@5Z z%Ye?Ook1yw#NY?ct$^^EJ|xT*D`Ja6RG^mz#^{&>o6x_yVzIa1RQ|LzJl?OtNX*0V zy=dfB7L~8#K2KcH{tpFn7n#dG6D3}cP7}4D>qk*ufDB^oh-gO3jb=Jy6~|`|;jThc z0cge=?<9}+!Kjnf$Fq-V+WHkMp0KS@>kjsmFssLURe&I*fl-^;BQq=hQrEmOd8mNF z1!`n;9T2~^RV)f)`hWCe`{#ixQ;-d}-uaOIu%-grF!V^5NkLt=A_h87yuC_jHFLgM zM9?hS&=m}^J2 z0_ijObM6%gwb=)?+VrBaR2CMy5`(W=5M@6N@ZvE!F8_y_MS+(1z_16$kymb%eSM`u z5`CK~-12!)Au@Otj=tenOSQ-0XQQ-cyxQcekE?sp%XjQX9J5!)+Tnd*7wZ-lV%T5e8ostY;Ozg$*8e-!O_{BXJ69(Ewj4OOGyjg2 z`#<;-H#yca&4(JGQvRU8NR1I%J0%VaxWavBjkdBezHY1Hm7qw*UYD literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/case-study/akvelon/diagram-02.png b/website/www/site/static/images/case-study/akvelon/diagram-02.png new file mode 100644 index 0000000000000000000000000000000000000000..996dff421e6b30ddf727d020704f288d29161107 GIT binary patch literal 73850 zcmeFZ_g@oR*ESxCfPeymfPxe`M^U;Wy{RA`l_qFFN)SY9KuYK#prT0Cqk>Wuq&JZg zX$clUL3#-gLWj^2dIE%ycM{P1exLU*_KPdgmIpPg%c>PM)immmGn`4T-!O)$5*q}8kFE)7t%$3eOe1<&oFWXC7!;>6J zx0Iap1d!3!yrsKjg5@qHn3iGm?wvC}%lg+@|5VUl(DVNuc+RZHe15K!-TA?GJ0U%N zuIWZOgIv;-HVrgCH$f! zWBxRl@*f9*`X7!;(9bqV@cr*3qZ{J)@&A>u^70_P#cE!sPM=OmC@U*t1r^0tS6BOO zug}fw#aJmuFl`9>TN&^J1!BE74}FyZ*{6@uKR~JMW*>#?PjDRqH4ObAJ21}eh2yb0 z(>xB4yTViU;Us$W4mWd-y-mM9D;4<+q^Q3{yp66{&z-rpMB}(oyp4tDw5lK zP7SiK zeP8t(pMgGB-FJ2ty29GhK|c%1`Vqyry^~%2FWv*u)9xU9g2hrJpBV2VARsX4ii` zlD(SOOrZ&+XxZ{kFP3}C4w#f6(4W>k5*V5y?!5Iiedw? z86Mtbm^k+Meh17d$rCoO@)YSNMHwcI>(Czr9+?3H$ z#e%(oeP;+Wh`PiA^l$zl9G1bb8n@>DbF^v8qN^GY@q zFR?@)VX}udQa^_>PR5@7ee2-hVB$y|6WyvmiLLaD4S9QucevQJSZtQGcW9_h2Vez> zmXJ@&jGGl(_S=41|jQy)Uvo{32Qv({5{(|ARpx=X$An+G+gZ*)F zan<9Dz^1ovGStrl&p7#+>=lKXIOwnV?da?O2%Nqx4tV+5Z=u9C`jtOc*2xpO+mJ^$ zh>xY73;hyE*lBNO&Hht+N&i4JNKyDP!xw-_uhxk|2}(@HN5iH38-Ek59795 z`!1CANm%JCGcyZtV*z=3#ha$gXnpJ{#EApI4*`2T{LXQB$}o%vRFt-gk{w`VK(Xh3 zkFNaHU-iQHk+`GM%s`8m8rAj3cDdz8K;2Sb}~n+)Qe3__bRO zM3tA9b7dHKXMUyoz%+FLzUSjk`wcuj{e#7crx~zE{zwMHwXSsCW2Qf4`&vhn>C)!P zr0dd1M@}wk-A^NUD%-DUB`kP*wF1_9%N3s{Ze*B%dP5?HD4)|wBNrEGgplQwS-d>AfsjY^m zp2klPYmcs4)nz2#K`+A}vw!_HX_C{SRd*VSTWx3UEEE;^!7%ZkPQq`K?sM1w&|Wv> zqrNCVu#ru`*rs^aPd8O~PhxxCrMwODF>{fqomu?BY$$^jzWXff0K5NESYEYwxos`c z4NEdX;kP3($l*}OAry6JXijJIeTv{$QS0MS9L9xTSj%b873?kIPSgAH?sT;T-}YQX zgmcg6?x09Fl}F1$wOBki6y$$ZyiU}y0?gjo5#H*4J%BB#)-duyX&uURY_|L7X-pSe z$snnOK2B{Ir!VmJ0rALf!}z*h(a3ihjhogol8^nf#Px;WRnV@RV2r_s#B9{H-%!Nf z2$81;2d$(SuLVi%r}M!Ugr900JRfiJael^+P^@*ZpQa1^7Ua+avc?(7$=8|_#7#^H z2n`&X^1f+ZA=}!s+S~pzi)25tB53VsWsiZWJq5O8guh&IB1>3QbNAXB_5HVvFAFpP?!j&x(yuY~B%JBnfp|Avq<#EtHT zp+bn%!0Q)xoyA6aHdLT*lCSmjEajzz;oGsvk<^sj(yM@vV0Vf<|B4u(1?fMuEUk-& z-F@hD?_t~P&V~qpW|CTVjkAZBs6UF(wGFP>({TBbNLF|xZ~GcJTO(z zB{yPLoXCRVh0oOZS|QR(iEY3Is#S{xKV28tlZH% z;j+Z$l^>;(BQ`n=Cyk9>Vc22)12@;yN&N%QiuA2M)B_GSG(q&7W~R7}lb+;n$SqA$ zq}_Dw&U`PmM%Iw7g^;yt?u{c990r|SB&wzXKXMn$@NzY8up6qWge6P8hCaKrB^zAP zK{=afZ7g!}@izkw`-Q~9o0>`lYOQXiv@)+`Y<_|x)f z(}`MaN=CVp14f!aRc@8&8eZ3^f1HiU*C8>lqwGePWV!i4=iU>DYT8R3v+Wa$vt5{9VLmb zRgeJa+FGEjvqdB&K)3gk5#X+np&&do4jO_8o*LJ7HJq%rX`+YF87XqKc;kZ8cyQ$# zU37;5b-D_OnI1|wRqGk-=Y04Qxonh_E*IZ{JwcHpSBM{2w(bz@9qk?vDub*kMH^Nh zHgvzl2FomV(TxxAKvbrwVMy|3A2YDex(@nimW_z0+6{HLdo5BIGq-e14+eZ!Y1a6z z6b-G-R+~|}X^`95LS&yw2$EH;Izuw40BbM3OZ%(}Ug{wB(1SyR^nT8v^TXg-AEvX| zZDl=0wa7%~!uhp(*Eu9PE3+DOqQ@@_bn1R~247#2&QVUyKKgu;KuC;lC5Zb0v%MtWl>9ZHcwhbB&lMeZn(1G55 z-P~QQIyx|1YUKerEPT&NG0RwB!0_G4uL6COwogtu(ENIgV}oB{pKZiX(7I73NH*RA z{78G6xUI5=o`sF`M=ZI^V-MCj{s}V zz^aard+GOIlMt661Q7iCjRU@QpwZJ*AYy$m_N)$CnFT2o=vh4vo0`2o7piU(%WDc& zv!fPy+O{a_S&i8}IaQS^=hq>1)B>%5#ofHoV|>GRp^ad8J8Hu-psvt>JgtFrFT|np z<26hV>G@%et04shtxDQlB&Mv6L#(&jlTzKmBffioeyT*%i(lZ~X;{`CwiKJZS3<9U zSZq|A{rTqgKt(syF#k%&pGduVV3=?N`sU7Tu`cJ&j--Kj8#M@6 zPdtlFcDkjlxN*(!I6+^5#D4e{#Y4x&RFKfT>cIL)dRf;-*4bR>0D%yVU-%xZFS+~W z5qfkW-RlCTbR8!~@?t&N;yYQ5PhWDYc$__?9z4|;=^tNb8mp#MiBf02n6|nJIGYcrr-lIu8`ARGGY0U#)(H`{V85I z5;{WPS0|j5Sc%}lrFssm>^7;`C8rP3{D2k7W{HGFV#uoyTt`!|=3azM7r*?vE2OoQqjtnoRC!mj`VZ$5Xr+8YmUG_GIJW?U+u6$^Mi+ zCi7oeK8kjQz~v=ZV7TV>fx)=yKf3MTHIwR{T?Q#mbdKvfnH6Lq<>=ik+z~#3F%~H{n5O1<}hReZ2 z${1>*NKbEX?^kC&(b=a`ub01E<02)9!tT|EW+o}SxHJ9q3`7^BSYNZn+ zsY@L|F$)~wW?9xp>3_^-eUebJ`{v_;^@p`maJ2~SV*O)cQ4`zq(yW8P9IB{8NW~iS zW^wA@4Hn32o&~LlPNLB-DD)AIr5YTUJkVJs+rCyA+lQ@0BAa%k9{ak;v=lJdae~- zl^yf^tB-{tYm_4i$ensc@4ly~q}y5)i4#J{E~_F#{TJj^{XOsx95aeOq>1>~p{Ivq z)>N6rVJBI;J(amR4_GRWWDtTTc)M9m`F}e^7un(f8E;xHqDf` z&FJTz)mur)OIhrjE|cjYx@F7M+EHJ4IYg==K^3HCPL} zKZCpz?~5dQE~bnQkO%#y*EMGz@jXTLbsIm{ckqibc0j$Z;Kw^}4A)W}15`UYL@T6` zawy2z@1XiT5myNv{!?^+0bSXT2C1i`k67hfd!;LUkpIT=-pa++{A$>@c(UAMwstsa z_x{D;!KR2u9V)xz!Nj@D2UPmnV-c~;MR)7+~e`K!LIQ3e0^bn^H2@0u+3g1bKqQ+u%d)^C0G zdK<>vd?7`9#_LX#G?SS=Vrn*KZgT~kyhi)pA3(}phF>t?A(3?zM`PfzpLoAKojpVw zC<6=TJbT_?PRXzv0N=%wj=U}z4TB!PS2ltU7#lo#A|n6|RLo{fnWnl4tfuMbF&-h9 ze+jm`SqS(z>;~;WRs50Rb`288P3V8hBkRyNmq?1;tlNy=?E_HGK@}0_WF@pf_>NtmV)7S=*xdQ13ilYN zHpL~W7)pSyfG;TWE=4jxrEs(JZ|OIOr%x~I)spqR=wYxTgnW5!U6>*YL$zXr%cEO9B9DFg+HXjd9*X?d*Fw7)gnqxW#1_0Jb?en{h~_ zKNkQgu91f+D~!?XIQf?;+ktC*(wn(}_r-~frh_h_zyNAXKeAjrjFF7k%w5KR+W12s zFXDcTAuHPiUuj3G`pc2+e&nflhB|R^4?wYh(gRznuvoG)M$(O(*qy|E>>!ZIHmnJE5pzYot75&TupD z0aj0j+*sLLbju7ZC7!aK)RO>UDEHjsqEVQ;l~+#i?xinmjq`rFjA56OVCR?!s^mPv z6i%Eo;i7jzcpo@>Ahriv3Uf2V>F56dh&n$GJaf96dC~wR%ur)vY7GC;GXRRH_?3PE6h6lISdRlR zGsa=hyN~=P48_SyT-`ef`rYYSv%?(v;SzRuXOKN~n&MNt4+!V&bvRt?Rt#SxfY5z> zMZPb|=(xSW@8|h=HT_*L!qQ-WWFKMON3GW{3P>h0J*-h=0)4ERu#=T$ypwKE;DEn; z#OxHv_!C%$i+?|>skzyDA8VYy0`kdVAdSCa@bx@fOTD`%LEKL`cq|Or_NLw6dmtdd zJ7;qooC?p-+rx){W3h(Syk&Qu=NUQzw%$sY)ndHy56lYypQ7SkRNs^dec-ycNj#cw4d!*?3+0wHX%V({9*zCUkjg9W_x@Z2!6yM ziq1HEF&jy_zF!~yK|f*hm6pJrA5{PvxVBW z9Sl?VstuaA!&FxHe!7-6Q(R)5=5qpY5}^$aTU+D9_ov_ko`YMk+-bQBZ8zshPPwjY zq+1xIXLaACm8uagxFQ6iJwj-IJ%6rcNiz+|%-LZ*{cucM_06i$wi>@X8E)9L_=~wq zg`<%*jw7BpTUrSMgFuwfLesxXkwUagZro6Y;K?{qM|#V{@2aR^h!6|<-}W4-YThS{ zpNO(Wo$OlWy9fka-i&EX5NcXB>(1stKF4R7=&{PEhsCLn)E(GRQN~2&!0aNJJwJ(hZg#auhZKFdercnRDwk&S`lG2 zLm@u3{@3~(etnXz?pa;)!OE4}F7*(G@)lfcWiJ&AUdI%N;CA{Z+U7MX7V2wCW~)6< zw9aWBCC<3E;-^J(YhW@-)^Z<9QgXJfa(xRtafLmvn{z!ONOxq9O2D0r?Eb=STUd%_ zrB&{5&Znzk`Mx=xv?`ejc}%T_Z5RZJ#b#iFD_dfn}rxpQl~Ybj4e|H??+GQ?6q}7s)Fi{fNow!MhzT z-P@1G{$~>}Oz6q~TNHoRG+yI*t{en&bX@)xw=FqHb*)k+noy-W|0&HM3qckcYaC7K zKU(fTBp$TupMj~wdTtvp+_t67@1*s=?+LCfxA?Mgv$XXgwq_%v@Ql1d%+p$R@X%3M zfbXvSt=JKQj%_;`w$SA{GQWj!tuG+>4~+s-2)%%&VGtKYpFd*Rf`xgi(zf>FJ~!u| zRrUaT3r1bIv(1I2Z|icc!-QQ-zNQf}^-{kNE^)hP8+X$n&#BDacH9m3sV2x48ltZ4 z6JTyX>}kD5CfI?og@Vh8Epuxs`b-{`Ld4#?_4A2cb8iBA>}p{}*7rBc>lUX4|4O2~ z^V~`+Nn54(z*osIyjz-o&-5j-u{I4uNSO9`1i0hUHbjflaL_Yss*9~!!d#rY(DAfB``J=6lQlxCc4x$TOND%4LCcWnmJUIVjUuj@UhX@oH{ z^RC^ZnBBk`m3ik@X~0K{YaGU>J)LV+d~2O9F8RbnfHTcs1m!{(XF`8q?%^`JC~&db zU0Yh((ENH3@(FcY0VA1k*7ONmOGhv+sdYanb4TyPJO;Mo_^GLT07mky0N~U>YZvQZ zo;FeW7Sd9N2!8kx?c;dn#=ul{-&0H=cayy&Kp3{4 zK65oj0n~By9q zUXo#l{}LdmnYjQ-$K~OaNz~%V9+RrE-<%hjKq7O)+bf>p7dF*Ff7PFobzNHMZ)VV= zLT>KQUVCr4vj3)6hN2vF9{l60EavGK?6E+5f#W~)ygCp{u3EeTdMP=Y^lkGq;`pA~ zU!vpbIhbBTLDr#H+9JS+7areB0e^c}XBNPt&-hkKn*pQTpKcaTe=ul&K%AYF9Kwo$ z%*--`*vz=Fwtpp|#V+nBJxPiV@j15a>$T^Ye;~sFO>$rBF7y;q)% zF+SIyi2>C6M_{nDxJV4kW$nxAwm-;TwRi*cLW+Hxl|uhBN<18f#)7uUKcKlm2vlxNbg$Dsasqkk5+ z^ffwL&$jiLQTc_N?Mn&a&$;+oK@k&lrraq?f!KO!zxMbI@1G4he>)Y>!W43^RPWc| z`}PU8IAKN49TbIEuj*Z5^sON;2|&w3++R;Q5M`4uf!UG(;~6Mxdf*HT5HE?+kau(K5x4&8q+Mhwv@`&hDN6$oN8Id+^^g5Ds|pE)m` zaL%;5e8pZZc)js{$Hw{d=-2TlbVpWkO>tf(*2uc^xZu{fHq5%`5~?Ey-guP>rs7#w zacpRSWRCE)H{`R7S8KjTr>!w#dXH_xNO*tQx(qt?GnGzp%T0S5bdC@<_I_p||Ik7Y ze+Cw;i32bNWAX^)StL1QXR1WG-s<_2@mnVjNspG}a_J5iZgz6ttKRW?FaP=#Dv9%5 zqQ)?@SP6u7Xze;^JXjuF2N;kf|HVK=rmE7-h8Nc);kQD?DO#7<02~5L=a0N*$F9Ua z;l;k((B$UJaxC)FIxHTH+Ct&BMDL5yzE)Ea%GsBnW|SUc6?Ln)C6)_Nghq-*z`#vT zPI4{zhSO~Xz?;9T@mErZr!akH9G2Pu`%QHu%Wj8$>)oDNE1N8>xDi6G&u}3f(Y9kbm4By zW#`CQsUew3vFexIi+#q>0!iqIx9aD#^Ws;=D6gX(u|LD zW03LY0_5E0j`{@Nix<>*B_q!9@*frp&G9BGi2XD9gX@Zjz4C7-|HHuxo=mJ4Dc{aC zB++FGfoLZH*r0=92iZJ^H%WdAZK^qj0C&xOsP`xs=?f@4+D>j>JhISB@N2atZ#|?D zhnD^#Ap{x>U%(g1THBWg$a9805_8QY|H2e%$~Pe^Ac2L=F0aMt$do?n9(VX2A=fG_ zDcKLpU(1Eyug_vC``tCEHaz|5%XA9?8ZkgEKtJeHcT%01Wu-{X8*0H7TBu!~bW*R= z$YqSK(;%{kX`j$oFZNsXpeT6>_!*jPiC|?XJKy7b=$QUE^^oRnQUw%SD`zZ<~x^XT%w zWf0<|+2x4~(+1Qda%%uU)3u5kE)B9?4g0kaY~&~o@T+jS){=C`H7F$Ab90M~)P~Hr zFIbXBnu3(huUX4sh7oYXK(KCrxK%3oeJ}D3lt_ zSlcU%{}a%s9KNr98Im>>d50OM1a7hWgJn-G-AQCEVKwvXS4 zZw5GkY$&!wzJGdaeLRI|DJ`a8V7(GGLhv2S8g14vlnlbDAMK(3>L6S-3H5}uo||~%#o)KXYq}25y?slx zaU0N8SX$2PQcaML9peFN#t8*5NM$Z3R>R=LYJ5+Zl!8jf(;7lT=Xbi)C!|CfQ(lHb zB4pdYI44I$6|C09Z$uM@>nY}PZ%dQ^bmG zM?cRvTJePWU@|9YtD!N3XYNyv#A{r;sw-V5vl$s4=r%Tx0nP;9JaMi2>fYWrCAN?4-fP` zoVFzUP*s6|ZRKLuPX5vq>LeK`@XQc99cpHQTOeJDAWKBSt%$j^WjdTFBm{%a<0O3d z&gVLG&l}Z$v>0L~kPl9Pvzn~##)_@9eqC?uNe>anT+7vK3*DSbH^{*Ip1N`;?+M#P zXuh3P+w+zi6li;E^h2s!83Kk|8l~pP6Mm7m(X0M5hU{ucnxpju&7=SWeryyUV0k4} z2zsmZ%jcQfM;tN(`t&lw_n@cW2`5jB8CyX|5lWcpU9W5uZ7Zdl!EA(A+w#n1ON+&i z43*E#1$^kx+R@vlmLin2X>p@QWWvzkO^FlQZv%QfOJ_o~GZem}?lx1B7X-9>M`3s< zSI;mI39qftvbh$-61Kz$3Pu11o15(n$TB((V9KCM9boj1rEkrfQqfzIUyr15<|f@F zVx{dzd++y#ubtr}I!B+}9YD=K(+QqU5Lvh@C0tI+Iusj3}9s^I5Uai3+iQ5BiS@4d_I`&cx;*KQoEb+w zAP)kpYA{{>aR-R6cQ`wI_1@W#-;fF+>8=M4S+E=}gHCHnG9Ocy<%{fsgwDIWR%cpL z02xr%ry{5>40eaBZ)6MsjLn^qW)lA7&cH}>;_L;q$eLulb{qWI-DiRNrtf7*2~H>S zQ&XnNZ7_u^ozf5!Akc-rYR>mq_`#)6R>B2T<$)uLAamM15qCUKiQtJBkW5Xe#>rOeo`#s=KH9Usi=A7zz^-m zU;}SBD+eGebmkZsIBdr^}p4}2`b9-xdcq?djxq73eaeMll&qs+W@`JKN zVB-KWZ;9_$1=HFy<_<&kO(@xbxTwGlY=NMX+^t6xU?Y|8_+&s97K8u@3a(N|L?R^z zr>&URlxS zu){#i)VO`H>(#}$=_o?H@9MzH)>ejlMg5UJy^DkGzOUVPXI~@@*|Es=Tqq&?H<#Fc ze%g0yB%$>IBToI%vjU2rsAxWR^TWLQ5gXK6@jKG%9TV_%6l&7ry;r3K%KS@UOm>KmE~Lc{huz48>M@yD*p|}CKR~*bg!%hXk`Fn9M!(+iF=5oK`|L^j>$^;i&%C(J>UIHQF*_r) zgA{4^%a?~J8LX;c;I%`D#ue7T1LJM4Hz#$mgbIY>q1E@XwcTj}BUMvc*AK8)c~ln+ zu2Cgw7upk4VwSTwbiU8>_wk)Msk}_rjNXwk)Kc?beC_w4j4pcc9WSS^{>vIy1O|rR z=p_8LG7JnPD|^YJ_xj@jXF#b(GJEXP(r_tutW#lHnhB`Bw0`uIq|I+o-!`qpL+`Y_ zdu%@|OYGTapMDG9by_4J?)(~y&Kz%5sn(=$u^=VumalM0p@k$26NTq?Le{D4`-!wbOY_fYNci$tj}`AHD7O&Xq=go z>FzIHx`F;U33qjKBd|#Jq?Eb@iN6mA!1YP6ZL{=5$Crr9acdP?v=(OC>nJ{rmKSVd zu7X1u8NM^7)5o#B1{V=&bYUyDy~!v^{?5$1r*^Z(wk~>6j5R~pV3LuT%>JX^tru1- z?F}W2P)&o$AthdOwsrF^ip6RZx-Y0UeLQ)%L^*f?LNdY{49w z1vZ**ZYLmULn_U%)cm$`289nkS7(kZsEL=JUq>_~tuXslN49h`bVHOeW-C(rV z%XC`PQVv}jlCtCl$a*6<4n0=OXqnjAnxJkqw64!IN^!hmdUM&}hlIy)uk{PWb?Z1- z0oNhX1#{D!VCt>X4$oVL|4L(@Nf#~Cv{ogNCqg;;aGlX}H_F}h&DWEu}BhY+oyWjxOJ z(~E4c{Hjeh=!3Nu#B+Mnh$fXc#GBohj*LH@jnG-591>xZ68-qx`3Byf;fh;ZFmH7`UDD^K=^dYVfX27{`Tn!92B4s+EA*eZbB5qz z!(#Odg4Efnr2tDlrcw**09BKqxXSxlk1g`{&Cmb?2=aK@6<_oYG@~v!O zDg~)ab!%0AWA*}x=-yCqgW0Zcl-E5=E-2jxW22Spc?`+MS^fBFgZBaQ%kk+uQondJ z3~Gb4Gwso`^F#w#Urb4Dw_&us{$r4s`{E-@@~k$3wp%J!L`f_LU%;H6shv&~5n?`4 zR)Rz<5}QzzuU}nQ&7SK51xj1cB*j{!Cxa*DZ2h?5(o)~FL1&g$|JLSre^Kq#?yn@+ z5OUh*6Wh%8dn>>aKK&~#F`>IV$R{R9S+)*5U2k46|qY;18J{v}ig z8TvFzh6Ab0bfZdQ&(vJJSQ_F)M1*EQn4gZH=@}h`By%>+%&{HQ6xX$NKE>*3Eogvh z@&U?G?~*E!ixO2{Gb{a*sC=K2QvJvGP8OMoWes4mIeNb59mPFy@cH4m8z`%TcR+)` zsZ%|2aTg#sgychEK8v~gaLbTOjA{jR=J)0ZC$Amw}zY}gB9p?CKGB33FnPA1wtg8W|g_h%3tO?6SB_hjl1wcu(kSjIMe;3rweT)*0%)Ia`5P z>`0)Q&BbqmW_(_!okBZOAx$v1b6;Hz_ur(&v(EyhS@v z2*-7cE1-9L){xMh_7gzTr!%qP7I*2RC$5I!?1~%~Ocv6|S;9#ZV#mnWA|G?}qiLW! z)uZ29rY63f(=!cgAF}@E%Y$12T;y)!Ux>D7&77iQH{?B}YElddtoMHA8u7*a z$sg!?e|wyE0UPT?eWt!}s7SP81E8tz^RL0xin|>=W`m{_Szzuum2Th`M;6QEw# z8|*a!>6kL0T&Dxl4ABwj3mp$O2ee3YZArNq1Hrdiud=QiF5(C9gRgs%N?pXsg(sf$ zv?`!ePn|l|y}22<##j2y?ZCmElKXEqHIM6QlS7_7qiBP;70}iPAF><`1FxWv zW|PWp+Cfdf`{@pu0_=Tk&f{3J01}zC>lP!7N8s+OeAyjlS4ql zc45dPO?z8IUk+JKhfXP zmOndG>xCD73WL~E_L14&xs_obGq52BAVymBAShj~Jz_FzpQziGh~{g3!TuN+@sGmC z7-W?2pPh+L48c;7b`~?E^}qI2GCI5hfC*1A2+IF(iF7+eoCLVSa|5TR_DR$K;`IU@ ze*k;B7_w^n-W#XrL-Ou)&^;ytI$^r1T@$Kul9zr`F?t_R){gq97bdL_jQdU1ui@|Z z(5c^jVt2BX5IV`p3vBZBM(h!6Z^!7jgMi(oe>Gh;p^Jm&_qINQfWF%QNYjC9r19*2 zrVIQY7#|asrg=_yNi%K%Hr>Cu!CeW6QIvP%OtOFK2Fj+X!|uBI7xB~V(r8`5p1Xt{ zXV>xaErsI=Wj~eu#~bH5PLHI@2lwz~JL+zNyZ_c*#dJ67ZeDKt#~Iiny?YlB+9m!Q zbEdhu!{mv2SVnqj6_foLr>hdQ`tPF*P6`M!7bVmbnSrFsCwY?x0nFReH-mdDDc+|v zb~H(lD8y9yPJP9AcJs~crhCcP7kXqwM$+yhR;A{?BI_#EKV5t*09Y_h^myP5x-N0I z4~@94;Xgmj(;3O_&UZiUL<;Shh9*|S=&XikdYY!{Xs?0q+#^~KxjRuj-TcGB2E88_ z9|rn+F^BgB06M2g46%`Ji-tUrl=F7MF?8|b-SaDWo}87Rr+XH^2z771H+P-Lex|IV z^rHLkxXZezPv69S<7|63r6^=bs$KO|ZqN6Q-HbNf2E(P2-Oa;_z|IEiBzsHl$tCo~ zqghO%VJjBfXq58n`zH1Mn?~xl4?l(xtG5Sj@Jp);WzsEmo0%1Bxu};VJMUqxC&Sj5 zWoGsxK4Js|c&e}pIkVN(z zfLB!bsPl2e4cf?CWhK?bQ}WdkZPbnPbH8dKMx}6yrYmV#KRJ;2uku#2*R{z~$n$QC zNpYD|9e#ZtlNm_!?UBH1rReAKBLPqi*f$zx=c(5?5IwG$b*@A@L#`V_q8#E^oPJ^E zT5A%Tx?j+OPipUO@XtAy)>z)<02Q7RQb;s-XZArDar&UTsVSJ(CKuS&!F3XQ^ux^D zbtf!g#vQriV+G52jlBu>!@8ohOmIFLP_X^(un~ zn9?{zY*6t^!oAbGvPL^Ix&p_6iESWo;|kx8m$Yd)*U&|q_Z>C`$Dr@dabCQ(v4n*S zo@%P{ z3%2lSICE{j`gT}9(K2zg65C(1QIg&>T>&%rMrbd~uF9$@n)hi5YIOo5D{U{rbKy4B z;f;(tRWc?^+a@uIOWUOoOwFV@F42>;x1@hmnrMh}-;LY3Hz{stY@8D_$ukW`rQa{z z?pfb{ru$DRV&UPd-N}@mDMaw~WUVmoYYWF|ioPpzO^{yqtfcX1=?87^G*;dYv3fu+SiN&nc^*)b7>XGp;;$SM?EZ-S+9`?$aoZw>sOZ)KdrS zVHPnLTI~QySj(pe<(}Uo+$8TwhoqLspk(n27Q=VabcI@P;|t{rhnwbC6U%qEC^b%1 zGV5&_n2nnHf4|Q9AIRUb>@CLJG^p=;a1~ZxfQ8_YeIpBn3n{uwel`V^q8i;otTisN z#(`*zy=9v;Uy;4~=5|N5@mlW&+bHq43%J(?iY9JW$kddI#rO=lS@$idcZ?G23aa|% z&hJf2+(rgEA*gk(5buWDWh}kIM<24ty6nQK7Lz>q)2F))Tvf8sff{Yp1ch9T2|5); zHsp8R(X~rGbr|yM6dLzt=HHl`HB)U$pU(dse`Z z>+HJ^E+@P148(qlt%{_*wr{Wzx)?EE$K}<0s+&f8DwGS^S(%*sH|BI~^nr#Tu;9ff zZf4${3+(mRU7HjXFEZ= zMLic5wgaf(B*L}{6r-(>i`zi=GNd5(y$27+)-{Ru9cW$t$lGyAxyNa1 zLk4qSS2pIpif^Nokz3HY^+jw?zHY5-_Nev?`CT$PtvfQQB3|!-kBaT`M3P7`U6#oYu=&TX&4b(B0nOd|u?{d|R^yk57Rumx=k7=^g!yFO z=GUiY>B{|q=QS~+0y4N=FVzq1qwsS~4Q>U3(zjy-*A_lCl$S_XmEhb~+-pzHesac} z%6Z7e$S0lsP$JkeGkrUxr9@By{3wLLX_PdC00fsgo8_J%1&u{GL5%;|#9}N_%Zu1p zcjvKnm95;563v|JNBshZ4AVG20Z|~>qjmZ<{uYeVq#@AX0W({+1zX~X$$8dSHYjf1 zA(ez0avJArO}WxkBo4J$0WG2R6Z`13qGa@j7Y-L z1v_zft&D4JUCd4LZaYV*3d-qd^a_|H?xH1cdmPV*`I<1ed24X9EvO_)+73u1)VDOzK4;s~t?X8sHovFzoJUL=cK=~Oy04Mfn(NoPNo5M#1AuPty!v~IOJVcN{Wn>HoRwO{9qv3FD< zwJ-iIXf2V^@kGY@sKc<<{6Do|{ujl^mP z&1g~;zko=dIduxpb^g9{&+x$Jr58=&2+izfPq;P}*|he3b9gmTi89(vTQ1!-(pXLx zH%X%YfZSL^TV>&V$0L?5^p6`N2;#YAh^<+DS<;%OL5vyJw$jV~C zziRP6@ZD+LMa##e4Ool_ezkQT`RQ$iYqw)`ut!H{s%p-Bkk9SNsFabF!<2byFXSv$ zr1yOtm|uq(eC*q5g_X!`dm@bqBS{NGB|=O7*jqsK)p7+}+RA5nrfD1ngJXQLM>Xl% z^6ghWx00-EOKVO}8sq=DEy#^;aDk!x4=i0ThTyzs{pw#WX-2Zof8I66&e_uP3BmZe zD_h&8sQ5YG21#8;^oi8~rc0;VFQw>)K&{g2ZDwrAPjxs89o8ejA_-i_Gn$=Oc!c?# zGbYV;m8cs{hNGi_Iz2m~9-*>r>2IwC#DqmspgSy^Z$Ic_))p2Zw1BxalTh#R`Y{kL z+F|E(`X*s!;r}po-r;PvZ5TIRbQraYT77HQZta=&ZA(#mwRU;!RU@d8P*rNSW^3;~ zYHz7s6pbxGY$0MxNF@34{`WbK{O@?4=f1D&zRury-Zu(f>DBW8dN6Qn zliSIPLKFI(y!~Vn2R!z+1xAc#eEpPJFEw&T1i$*NaIA94EWCHM*T_ zF0Y{Rb@jZJ|0Rb_k7O>c9-JoS2D-ccv7rgzIFbc;Pr5X))922+whkFb;J(X?-+jLx zojDC17QjRK!?j?4+I=oIUT@pJUvdH{rD6LPCl|pd@cV-VkiR2|lD?|(oaf;c`CYm5f3>B2jmFH^-6UeOgl%ylSzwqUcF^Ka3U6*#t7cD@`5B2 zzTe*6gnENDktJ+B>=2`P`WQDW`_~8XHfi)?gTNzLk*H%>*aLOkh3@p?_skjV=d{B2 zAl$pd)4}vs=?=4z4d+_Y1|{FK4a$yq_zgD2Yl_vb#!dn*@?xePk1qQHhSI^xJ(bGH zpO;I_&67zQe=fan8CpcAte%~oiT71K4S5Dk#Q5+%1c<<7sTnus(450W>dE)HX2f~8 zEfV0v>Ue&#;ji4rG zar~DpCKgY^#ILQ2*LOD_BO(t+=vg4umN&-$v+WrWYBZ4Uj*jF@BAspR13k&VdZO|>g zRsD$6HoZvGy6N!5U6naoJoubp+3`rm z>o$Enw|D#BgkQ{8!C*HQP9ZK&rwMyj^&aB38U2_)1w_7a!}WIVV*>0h}j7FWwO?n z9)(|9Sk%50)Z~c8`*!tX@8lyEFBxVJ^VB`Z5RxOR*S5(BwID(e`x0)0T(Cb5{?G0x#+3>@9`&)2Lk$}rTp^Sh`{x* zh_zQAZ*2Uq^9u}!T$s6zr8-ec-$s`hfW=|i)1HRaH?7qr+ z-S*H%?h*gJU+a-eCQW;xfYV9NCVR8z4TfD)nfrZdeJlcO&-+5zKqE-%^TV%Nd>)ve zq)IzdOU4x78cX)d?j(j29RZky)j#;qZ>=<+w69NOi8tifa`(JQ08KW zu%pZTGX8cXb{auCiJx&K?coB9HWgi(Vj;@YKZ7@2_N-L`{iIfo4`=pN5uY~_=M{=# zc0_+zLqOq8KLv7BfMU5yzD0rGo9ZEXb(dX=r2yC&xO~&6Q$p|eWGOIlJI77JM`9vb zLk1cw|9&lKx*CircU!SSwEj7~x`RdBcrnjDVi&l?q;Pvz*LNx@^qZ0Y&I|iBZuw#f zJwg!XkVe0*@i&$*Gq=+5C4lY*{kO@4$8p+9c%Eq94=&CDtO_dL*9Z6Cru8s1| zG|9XasC)fAgY}AaRPCNd;Iz_!NJ2VyKF%1u-T~SW)q@YDb6)&*J$6^jj=|YWrY=I> z)#pX{FUVEB7bXTkpM=a7E7)9%VrnH4iAVxqaWYqRbJwTmA_EO?n<~@GzH4A|KWntk z6~?rDrU&nGjJbF*zUZNlR`ccd@+?@D7Ve3gu&ec$ju2z8_FG!Gl|q{)QE+x{kRSd) z!v(b*zS3ebSU;X3RGum)^E|vy#_h@O`<4}hd>j6@l{WsXE@qD5G_~U}9Uv9O#n6>T z?rnYY=y!99plYnWFUU~946WCEB;9@N7gj%_9%%fYmo=7_6r!Q>X{9)MlS4A$OkMaf zfv|dS1>y^yY7?fXh@RiUTG+X+g$C|tU!q{#^q2E5>w0lJmwXlTCZD;MZ_^lxoAS8Q zc%kR#zcwT(+O|@M%T5rH?@-7^pC~$=FmEI=5YEp4i?n;yYNL4Z4L$xE--Xq(Sn}^P zUkZN{y6e@XEvr(~)HJcGvGGh{FSgiIqJp(5wL{d$i6!qHprw{7?9+~_I_|(# zqken;gN^)BY|FQiV4r8oY{=S?5reB&psf`hmE@MiVJJu=CzCDPpa*Um852^W&hf&l z{W?c_2Qh;!%6L=0?dkJq?x4-GIuIf^-Aa+D6Ny-`_SXx0WD3ZXa(aI3T;@*Q8dTka z4xV`YU4i>@;BO3AstYdl{#p%R{KhC>ZeKdknYMQKi-IcApsCeA=+ zf@-&>$LpkS+?iVw9cqXA^4oSvHHcN$9p!dx1!zvZ(A@}2{;V1PV5LhtZW|gdyt1i- z3+2OAXP3|um5bM! z)ty%vUGa>9HGy-RmxW)ivI}hn{^Av4ldTYou)ObEkRG7ll0+Szv4Uwu11=561A;8O zc>k=`3+%P*MV&A=qPVqSk%_VrXCp`7c-eZs^7dA&X1Fy`lV8};C@hles8M_T@Ltnn z2&jA(STalbj)ULS{3QoZiYWC|q9;IcKh~Dn=DBe}ij*zV-m50OW?rp+boyB3-3wH<}v_5t%3bsBv2X=-9sBwxEGm zT&O{wUx|hXhBxyN=;5#7nStG|v+!*14iQL=*@7)IVBwKzRc$Jfxkc1+{t74oEXB+! zR3`o#`h^$O=$vbI5<4=0+VH9WK*9baGte`A<}l@)FpUMiJ!dL(hdIc>H^AO<_|h!L zJFZS*163?Fop*5Ln1=YrHF__@pU`X=s3QZpm$N^s#6gdsD1MDYQcL8>WNYL~Xl)Zp z20zD*KcKS+dV{Fzv-TA~I@4iXY;rN+`2{5h^~sR^f@;WusDm~Lp?2ciBLRJxWAt7q zT;tu8JN%lT*fViGQ|#q33q{9iEN}rm=q^r>yQ=8idH<1%RxHtHWc+3-Q&r(vt{u!+ zJG}GeikVS#T zVWG49T~NgokOjn$Rv8Zte-P*l8j>oZG-r|5f8p|5c)zxU19F{1y|bOXOd2z;QhS~q z(VR6U0C{J3^j|$8*>r4kky6kJ*kwhxfFV`k&QecT!dq8RKF?iWS z%PA0M7mhSk4ZszkL(7lPbl8w+bW0SIM)33L2!qj^I($eJDn5!xSW9k2VlmI0Va%j$ z3HaIal&9akXV9#px@btx3_CC9Z&-cM^&}%`gE{*&AK=u&mS-36wLUA}$Zgkt0bu=l zSTyKuk>jt6ivTIzPSL%54Q~TyU?85&lL^Z@t zR4FnPmG!~TSH1yAQG*XCE4awt;a5#-FQ2xg)*1$)cRBC3X*Vyrwik5I0{Xf`^Osf= z(ALUuTH?w5@!zYN@jrB4-uv}0%oDK{Rf~1pg}=QGNQE$K+~rtiZ%1pIa4ek6Xip0< zBDd-%Ju6co`gPiUD;kozye%^CG!#|9>o3@C&JFD|5ZdN0{~rDktto7NvUR+n-7sxU zc$SNCNGq+DnERogl0w_;X_Eu>@a8?gIz;kQv>ei-`0s(fteikKUTu6>P9=L;6<{3({~>DrE7y;KuGLlm zJ{8=YPZ;;aua=j=>TIa*26B}fetoQ z&BX5HLafuJAp9l%Ik@Se`S*@oPZaXoO+o%ApsS>x|R% z;W>Y=+vm|K68s{dcGAXw_AxNg2)n$jjs8S(7mclxjR61R5R_s`!=M;uC$8-n_x zk&Wy78FBSSYO#mU9umt;qrGUJbhV4*G6mTE0`R}=c#0!FY3k(P*8JC^`?b&`y|+WW zwei(bhtkD4p{{Nzn&d|{X{V{qN|)6TGLOx-&W2Z#wDNcOGkpj^h)264dwWkD2)vR z|1<7^K}x{OF`ABe6(c)_GI8u*Rg8ryx9W;HXw|9(e9dQXVyM5drJYyGYUGm+mJ1`0 zB~#)^b^3yR-m%UL$bxl5m@tIaqZcoOnjj5e+ANRH) z`0UFw#s=+F;l!6bvCa-h2XL2><*wmYo1k=FIj>HqqHVO<>62ltq*XDQp_g1 zll@m)V_@@X7kcT}lMiLT%PdT4u~}0L4h~)QD!0I{R{4isM}9UXOTqTnSL_j%h0_#- z{hdKz^XF&Sfk4gVJ*a#?>6+6~(q9)k2InNM~vg1}onRU1!nZ`FRM2*sa$rdDB!_7zY6 zHI&cG47Nq>j&#j7W<`3ag00v%%qQ2{i_Or4Cd+g27RHMznUYP9$(+5b_8$hhC(+kE zBRZ@)`?Y!KJ5_TKItn>{7CQT=T)boxxQc<8Tyf2G*O2;i|FhEt_erAn*B1V}(y}4#gYp0Go9o>Pu@8yIE~J_J234^lS{Ry>b0N zeI`tVq~wdjU&2G3ubDnu%Ts$5t88h{;82E_OvTG8m79ArD!H9V$14P(HDJ677-M+G zq;*z-d5{f5wp!Ke<&=xjD5)s)+dtRx+7V&cvQ>QyLT{E!dAaLr!Q}Qr)cbr(Ijca?CHqIf}PHx%$TB91(2 zPmm6h3LM?9{h}7JGYo7VE=E1LC!qr%c<@Nwugk%^ds7Gi?&wp?w)YK>l~Ty96u&+) zEBsAk+@sjbZ(h^iarC+HYarpm7ha6lbi=l?)$D#C?Py|Q0Y~*du0MrpGp$6zZLxne zRqfvhS;O~XnUJs*6LWl-SA1E|zerg2JamP7I$U>RYC^ZZS}y>6QdSrW$tv&4->a&M zfgd8Y#9B%5D_XO=3f{Ok0g{IzG$VPKtA5wq{wdUDyALui6C%+vv9zl0)OIwst?7cf zT9AFV9xng4FNNsf72l>1&$a1pwZm?aJ1G#wXX5>(YQ4Qc1~bTk(F^%Bxp#PBNpl&) zUO(OuWB!og?vBsYKZfU&^i8^Jqkz->fOuBgf914Ux6k9RLG=|=`R4NfjI_ZXsew{)Ix3iDY}M?e zFY}~V`{Ndk2nsm>xmkJxFKB7u|8bD?Mxp&orU^rsC!8pR?he>sCO5lk!|@SIg*x|N zUIm{I+Xw!^9DD{5QC9PWp!k}jnz{L_Jf^L2!hDpJ+9hw}i>D&z!S&B7_Qr1VWaUqD z`7YHa>Em{3X4r1unxXlgX#1mE)BAW_Fgl%eOpoPdS$RGM;bz%?QP`<2$q~9r5jRFI9)<)_a!VJG(t-7SiQBKrTfkKV*Vm=(U&0)TxJi ze946!Px z33AX{!uYS%&i(B3Z1GwW7^IBT`bF7eeB4VVF&AS>p;dXGQLoEoiZ5FQ=-wZ?r1Tj$ zPBV@^R`%a0iCjLQtyRape-fDkomw&xhUcYnpJ~nxlW(Nj%`3UBRaIn?bXVnj<)h{U~nzr@k7&xfCp zM3ZTS4m4CkqGjwCMEnn4stJ^_1S8pO*{b| zz@YqvHFuEwvB(DsQnOd>^UI}Xyl?L^ayA-!qod7SV-m|sw;TFO&}#Er>U#TwA>+gs z#+eSKdB-Vy!2E;Lv`}@Tx*P`z7}iJ8)WT(>Q4_pk@xs#}YxVw~v>Kf^lY`lGiC${p zY7jL-MZM3jg&frE7nCFSW$?mi250zKy)&lr{bj>zB5pr#gBlAi_8z4!)WLIZH;1-^ zQ+mbS2v4=+A!qQC*+*+wKv$>Vqm+9AQhTyPxC{zQrq9%pBAuM88S*GK`>0UH#>lp$ zINQ|azp(7QISV0;kXYq%|A zX;PZ_hwL1+?70+C1m^VX4_>!ZJw0?7chywAsGM1&i}H~+ZhRE<7^K|!^75>Z;!rV8 zA0m%rlc;h05pWOUScDF9AYfU`Q7A+@cvW@m)U1xjbo~@8f6kO_tx8@mw^VgHlF-;% zHh}iPF4ciNPE#3Q$#QE|JpINU+SV&CtI86_UIWt0PIXQ$NX9ZC&G!9JIAHac$i6EC z=T!)1D?uxQ)7FS8*N)XZLz<|{EzWb2Z3%A_r+N@GTVm~U{P)BDa;XnR$^ilT4Maqq z=L{_G$1C%`tEYD~tC z6@e5p5xwv3B*cQnYiww8>!j;ft_!o`1=*`N&b5*=-aeoMQ`M5Dnf=l%e@ms)7qz(0 z;l7AV5$Jo|f-l4@yudOyH1u@a$=|lV5p1S6I=g3mkoAYWGU#)TQm?eB4m)%c$TelN zeuC(<@& zr+okS-z(JCpvg>4i7BQmf7IrOo}If~KmM5!q8W^t5vAzm!#1USDD%Uf-a4A-07{r~ zJ@B%d(AF+^=|0|nrOwIZY+mEgrBUzBu-;JYD7Lpa9$xm&t~LRt=muUe=A>_{_#JxI z8FWC$=KiJ8gu~aT);i}%gmx19?Pb6JxV9tPNQu&I@WiB*2-eL z{;A5j+`$|*DiFSDxB7UOqirA~k0Hx0Ub67`Nfp^|iTeNGJz+Maxz!H$XDd?uF(Ufl z*$crrLB?2q`n=gnZLzC2Nk5{NWEpeWz~R8iy=n`D!$r}wo*OR{Sdal|eE3-^t>aYt ztxNOl(2Sh6lVNuHIF2v=_fa!5Xh6X>`^$@*lC1lEAvfEcnn&(OVA~C$(xCax23;zI1@`6r}!W3PsM115>hA=Am-`_RL*7<{l_UX zqIG$iic?Z_ELu~s4Bw`03Q@iwtUjh1k{hei>;Xo(?DZye`4la%w;v*jros6&3*7Hl z-!yKWx2v@6$f}8fPYR0OX>_)|Uuh9P?X&I-^c!`uljK~o!S3`wc|S1#kt$vDF;fd# z=x%1Xb(}&hguWW*YB0stHlw{|+TVQ)Ul}+187!%lGZ%te=x14hgA%?Z%Bu|03D$U9{BpU356G z8{RG#r{DJ+#D9F!Wvym962Wko<$&q>?u_=DpYM?FtjR4=|A$hh*%gQH$TN=jj~frv;JEJ6cA~gi}@TT@#r7GpYGMOJnUj^+p0S`|3`#jLR)|&eo12L znr!2uTXAUJjTp++8KH4|r9RdP3hzY|e>%=%^ua_g8t-dN8>HnKle!$y<|b8E`EBI8 zVtEg)7=BPY;V=loZRBOYpTLDloyzl$={_RNbLk=Y!dun#?-^dyc;<+ys)wM1GHaXb zT#0Bj+FaKyJ-EsU{a(Ftwqt(gB5IaYbrE~Fm{}NFmcC2BhoA%5g>_32DSxep0dmyt zOUW4RlKloUpzD&}W5m}d0!4P}3hut@&lkK*U#LEm(TiNUMeoK$`v7GM{EOS0!TekA3+9A>dzD(HUzv~%Keoaq1o`?ml zyqe;VYjuj|P~)GLy9Kgh_B6OBX@S=0UKZF0-=sT;|8SyjbKY<*2eZ3X#;wk=cBoqY+^I+|4lH(;x|A6?*H)ob^Segr6Y|$b zcVjK#JZx%;wQl5RK)1oj1|+EUY6;MFb{%fuc3)FSm# zdrYO5P(ld}skEyNErQ308Jc$0X%nV?_`InOU8E3JAb0vtg|ly;g9-Lck)zJJw>I4Q zL)f4SY5d1<%4~(M8Aj{J1A$<0TT4&#!sD3d`4fVPW3sNHSaj25IloNxvyGpwn%WaK zvG}

A*`4(d&4QeNR!&6`=Wmgl^?l(PnURLBXqGGbk=|V=QYv3nsZo^?Fg+T)|U$ zj_W#nTZF-tFr^yw=4s5q)dPVsxyFDG$yu+PSkve=vakI!Vnz=12h3BjH17-tpEfgi9<^x8 zGPNZoRLjZa$jK~bG(6PedpolO{k^Ql{sqr!Ti5@2cSL_^qXM-i5lmDieWM`z?iA_vx6%!4sM3WH zIJi%z87;7jd-|`G3|Ou|f>eUn>ckYSo%cWgw2m)FMTKU$jZ#=@f&VlcN4|H9QbT5a z=l^ED&-Y`#?e&Wjje1-;S)|(mhKx!!HuAh+4OM~#9NmbZ6eWsgHS$N-N1Ut(l6`h!>^n!v?J3Q9 z4P0D%Cq$WuyoWfL_wdS4x6fFb@m|UQ<{e5T2*OE=sQE|1FVD>bw>D^tZXdykfNJz* zt+GLa=vA?PQrtf0j3o6{Sec8xjb?xasD|4J$8=9cf-%na8$ zelb4<&c{Ms?5bVugwsLpA0zP2Yf@sW#36)CM{!UE59+y`%iz%5f_v5pyrPq`1&emh^IhA?;myVj&0i1atjQF6&49MyJp z!O65khQS@=!9fDG8V-lk6l6g9{`os*k>tPbmwi%~cge5JYXoTZ5_P+u7dl(3?&lYS z?(|F7t9;f%`G(WxuQMN7R0qJ%s^eNCE1}gi^zCQd9y``T{A5gY{40J|>ull8ly2hM z-K4O_SP-AgjuL{&znHVSpPV~==f510h=yGvy28K3E;2%pO70eJ&e32&D$Xo~{wOkS zTG7T(d%!c1ppE7wGU zkGBT#K1&Oa8!>&2=-q&NK|~I^3E)-TQ!Pyi^Y<-FKWz1HBQ@L`P$4Re%?8c<{v(LR zq~&}oyDA!r(s#XdV;p(66={nxG1v*VffKW%prK6m_=l<1=|vmV{Z+6(uBoao2xT(8 z&vy@MSFbD|dF`82rpyHkGnr1u+^e-5W?--0TFWR6Ew!EQayiE5dOhZfB*gN&tYzBC`omGRe8z%MdS!^NW>=R;av*ipy7zl zY-D#B+mat~NVe^j$bpPmNXpzfF_y)00ZJR4n31uqVr-dgnGmOXZ@t%SccR5b=~T0n zGI$;Iz^x##AN=G>;MzTZ0$cUcHSKP?cXn*9kyj>|exIt5==%~Q`Q>ns)gVDSY+uAO z`GZVZAg>$2rx(FEMr87sN;r^ceL;X?q8J;J><}(q;IJ|VMjp{Hz<#k9Wg!Ft9z$Y&$K`^8y z6uF9@r%x|e>#^4$4oQ)O>*{BR1Oe}jIfPRu2Z*kobH`d)seWP%(fxyz>75Hhv(3Z? z*d#*p;_s10X@+K%N$h9iOFio~JZY-O!t=0Ej(fP1c0i@?8V9Harey5?~# zv;hpyu=Qd$ulE=<+0d0xLaIaG>vhp{(7X5TheL;v$a|^zrbd&oz1r9IXU1OT;Xv=N zYXC63HQ5z@uJB#n;eO3I1wpf0x&-Bz&KG(i19L0_v@2-K`^brt&q<>FMx!IA*DQIE zfwN!31;!>yKhi>1YW?M-+Qdc;qQV}Y6u|KM{xLhL&$08?+wZ(cW7R%M)GH(7^Y`dk z=5GYln~yfTNWD?>)~4QIQrb;1z5do>@!{6Zvv@%Y>L<12c2I1C4D&Q4q)?}^$w-2l zfqef)@zQ+<>Nm8KRdY8vLPU!b;~=2c>poVtOFD#I#0xSwEFC-W-Mj4V6?w8 zk5bbyKF{Zk)tGUUXH9x-B*wWHEbS^T(Z+>yU%nFiy*wM8@uRt%boIlZ*_Ohnd=7X# zU6HyGP#0`1e3jK<>~ywB6?ZNtZ7QwlX8xpKL% zhn4EH=CCPHt9N^T{}#Q?<=p94{m9cgM|8zX|EB8i#(4lJBmZpZvLmUXsrdGRHrKQY zBOA}sFKyoSM-}K!!o0`03T>o$Q^Y9z8Ee<`EVm3XzgQ%89P&Yo*$^z4)cvmrxF zx!;*iUYodtX^^^bsVYI%%ojG-f~vW?KyP1ynN37MI9o0&;jE_YcIxf1y@5-@{oH)q zhv$|yU#z+Ry3X`7XqI~0VS15GxNTL;u(qJqk_ck?Fxla*3h*_B7h>M<3qD#U_eHe! zp6_}Mc9N}s-}tZYX0#T9#XG?UetDV#1$OEudwdH*nc=l-e3?`e0Zb;5=ADWuY_sON z1o?Yx4&BsKx=^41Z31T*%2CJjgwD|aX_5$ondTqDYy*q1i|Fx>96U_=FLv3T;01$7 z=e{dGR_(*1W2Q5Rc&~E}Q6c6j>TOLjV9i`PohpD-OKwk__Uo_6w^_Ur^5553c>Pz7 z$5JO`r?oT(I4rXQILD^q`22NE*|~MsBmDQXE2lr&E`YFfvd-`1C>#?9Bh8`80-Y?Y zyo7@|3~pBGeSu`^XHXzLots<9LLc%gH2TbXX;r~w+04(1VUN7YrJw|)$#Jd$8cVtSSgXG_ zf4|y*?Ju<5+Kut|VxlV8D}jwkf8|nTc>^KNcBg36dh%d>l25Iy?p)G>O=z@D$kN=^ z0^7$@pr2Q%kt=&z>9)8`x9P)rxvbr@^02bCmXgQc_vxd_wHm?{gDOHn9-Z3Fl@~PYVNm z(l_BGC^7`1N8smJ>;(t4wV=hrQ)dh~-w7LJJCg{vU6%g#-^p^f+2dF8G8$=~@LPEu z?w~Vk5q_5H0xQ&cjI1~ULoGX-ng+oSm|gJKCVP~7 zP{pMeW~s<9ltt#WO8YI#&Pt^$&=;~Jq@7r$nvAdO3M5ww*krnSP%}EDop~e$_eI5G zDz`u&pMsTn7avgPo;3Le#6m_TIPDIXYryp>auJ$OAt4)Qsk8wTyWReoL)6U5Q*=+E zy&-0tNbwYkJLbyY_-fCN!;&Q0cZ?-mCA;s+!6K)wx91-J6wk3}D2dfH`1Z2c!wE#@ zDxExjFoEGCkA3fV^H)1Ua-O#SCtvg(StFjw{s*)PUWe9&KGCL5p3lOGW`BsI0v)`Z z?y2xYg3KE(7W{@oBgWsEl5#h{Dcv#8;hX|KjKSS&;frXSXG5GMcw3i~TPg%-vb$4A!7BbA2jZc@b&r*`sUV8Tmlk%%`|njBw2 z{E#%G)1PpGR61H=gVt9r=rM*%%5!7=jNG!5sHFdTTQw?A7LK2#PTlqb)D>*8Y=L*@ zD2?bQnQ(TH-s2D1Qq%1Pc)5MSobXAG+fqqFwJ}n}gw|P?M=(aO{10=tGV<@dEL$T$ zg&8Xtbbwpz5v}_$aZS%+OtFeKNL8q%%_<2H68^2P5<38OV_&XgmQ4BZN9NT?i1C*q znDi^mJ5#QINZ*dfhuuh*B&5IGhVSLde9&)15YAe?mu~w8qA8ZnfRbLFZ=B`+)hu@L7t9yR7H`D8)UUU6bG!O) zQNR80&%@5((kph9qJ`>%PdIWi23OME)Om}p2m70N#cfLaZoyoa3Ua#mlg$FLI}w&J zr2t4j^y0J5;cY>)-+YNOl!c4-tm~m5;O~xA0;i%EOr8&{;jhVstPO<_fsM-Y^qfpb zf3tH-Dtls;S?|A7?_Q|f)W>cPryeRIc$PB(;+==D4?wG|&(bY);#!*H*Gt1hO&tgNsR7nP!u}A6&vbyK7==>_ zsl(^i8PA@a0XPwxo@rt?$!8$1n2%bUV~2`x5?zxz{5Q__5q<>#TT9|78i{v?5IUBQ z?3((BXeb_%Tn4Wem1x2QI#GWE>c{axr?gj`yA)p>k*b}_CgOI!cl~{sLf43I@^$!7 z*8x!nerj=o)tdL;$3#0N7~U_J!o4PT=p)@YR^-Ioja{@v+-;D$x8uwsET&y=sw@HC zkrX%L<FPC`VWDiz`Gb|?0hz2})Q z-7^FDurXTD5^QF8L;ANnK**+MF3dZA)ufTCRC=aTFBPV)7&1BCPMEInuU6=l(!Zg- z>Z3v{mycrEX%buaw;Jbs+rd|*OiNk7{_u{Yz5P$n0{HtSm|p)LXmLxb>0yUfHlK-qawYi;EDb(12}I`iPN@`cJJB}H^NBFfUzil{ zu0UsaE$ZU5zskU8&D_pv$-XtS)Pi_mCe5M-cm71Z9nCq}kf%JfdB z;3*335nYdr_rfV{$i}fy(O6NmMp;!9M|Lt8JVkRU^j?W`WtNG`y8T%C*6MA#fMYvs zRK-rxDKU$7`FQTMS4XKn@n+PU&nX<&f;U<=pH~R?$SYySdVKg!>$%LiIWwyZI@|3* znbv_ocN+SZaP?&J5EdDYwKnZ@#$yzX%PrEsr<<8}K<}5YEC1cSz{V(7?XL;jEQto2 zA9wW9)=8BBLmdcAw{6Ama%8ZHn~bj6m;TNt_1UkY?sXFOM{4|Wfc*9)~7#JeV7)vxSUg& z_M54}emQd~s5LX4rzDFKQ?BEV1XEzd9CUh!9yqMyG~s@a=0$btn&$8yn#ZJn@~L{q z3MV7@I=}6ztYkqx^rfiW)Cv4k%m!2%D|D(fDRRPkgJmI%>zFj%Gx4CM^kevFA`CT_ zRthY$KK`ze*$N&o0)>wqOSo9g_KEipb+v3-3e7)r7r&uv$;E@(nMpHA6M~1T2Vjfm zURDwv!!)R_qZhAJ>SkNhx{k!$QkxwO5$VRC|DpKf4uiwJ_TTb1NA?a#Hl2~f?VPk; z&D;eFix{APr~|?v=*u?G<(}W^w|>$1(K{E9{It($uFA*ROf^9_a}{aP1(8AUt%Hac zD^ORW*R$}YRwcLls*BF4A;D%-Y#;FVbexh3`;SL3-t~cA}F}d6hC#&+K)N)7vJ*}sFBA+5g zdx@21dybu}_Iaqn#Gt=J_u!$-Q8t#Or(ypp(h|GUCfpF$WPnxtuZ^C)`ZdwEY;)(l z+(U|pw+wy}6{;Fl{-OZ+(u;e~jO9^H#r|cJS44#Sg^B4ljD~=^Hua{-vF_g* z_x%&5^u^Qr{>b{-*Rr!Iu2PmAw(CQ>&FU$rYw-?tXl~Fho@~)hLgrufQyR|YpRn@I zmEXcUYtQBpZ{YR?Z1~ifScT{KXRcQ?pq-U=-?`=5-sM*$#xp!t`6?)6ScP5*dUAvh z*G2a8R@%o})ff)it|Luz-&&?q(G4Jt3E!A2z4iS?_BN^!^iqIj+rF`MsQGUkZKc0=@S zeV4V*lr{Y+eVg%Za~RXfa>~|}{jw^drofGRI>q{B6 zUP2kKq3W^!FFbUD^p=j1=SNb~38CCTemBwb{H=Sq6z@1_?)hFmRfOmbBQx)1(P3W} z9s%<06f%d_(-FF-)DiB{k=B{Efk2*L^RlQ|r7!sCG$T^DcI110fRi@Qk2zMG+eU^K z%6*!hhfS_fyPc>L`TYL~r!f0BO8g9>U)hc!`!IIsE}74>iF7p>y!qErd=yld1|xK{ zO}ltIrnohaj3K&%aOYxA!Tx)t*SvzdBd@xNKdTF! zmmPRs7}&C_87E?FGgoU zrD9AU*{cF3ElvxqdT(b15uOUYZ@V-bb>qVw zS~_|KbZB3qP+f88L;ts6$r1v;Iff!4WR%l#!Tu2Jl?s#ae?1HjWMC$r$s;Hov7{ayd1oW0vvOBjH7 zqqQDV$_u@8L~c)xdMT*y2emRe*9zbkbhJVbp?O@*Nv(az3x(_e!!C4)0`8@=j6UZt zLb0=woBa<-N3e=l3U3%1G00B7BZX5}O6IE$899(kk||97i;AM9fTE-1Cm+q!!(dBF z`?pRXzI^!nUyHS)b;PiH_LG2!-8G5hMiV-fkO#3JvE@UQjtJ1!Q5@~ETWGU}8>Q|o zi5Zo2?gKEqc425-&oZBES}Lfc#7iL`d08#1)KtCum!SF1Ip^Kib`)J#ijtH@!XRo5F~M=z7&{25ixACL(Q(cveiyBV?i-U-4$)>~T-6P$!C z-a*w=KF~f!2W}Uu1HdnN%pE86W0qOGE$XJ6Dq+*(O4)9VR0nnyv``THVf6`6NocRqPWgQ!2)FOr`5&#`rJ+Rv}wG!PP$`yt{vMePb2U>jNI@q z|MvKsY+iUZyb*RJ)v#AS85)A@NLh5A%yIjVKJUJmx0;7SnMrsvgdV|>FKyrHo-e{TxrbPs>JC);~x$}Zn%c`oosb*sWJ%5LJxXYMw= zKPB|9=+CYG=KVDPXhMQjofhQMiodEszki>M#+xaGJRPO-f`)z%&eEY5dn}GyT>l=3 zZ-ZRt@Lqquy}`Ztmn*rs`dJ>4;0ErVh2a!n_D4rvN*V&c9L;}};>yupz88_yp>$p{ ztPmc94j-+qo-=8>8pmP7=_gii$-ndQLTGejk`swrK%JUm*wpg)?_B|MyOuc#CBC=B z%2=OLP|SXyN_2np=L1!oJA15IJmt)%w(5+td;GU*##^-g83Zk=18ZIWN7r}9Q~mw% zUpuQ|M0QzaR|*MN$t)`?yKQ78BjesnA_^r^$Q44?O|CsI5|ym1xVSbE7uUY)cW(3Z zef)mE^Oy4We!tE+uk)JE*Xv-F0!1OpQMMZwO{;7-UdyQARoq1lKYR3adt#Vx`5DmX zcbFFDH-mc0M;vD{aD$XWP0GjO@)_sVkQUPQ8u%3S@p<3 z@@p`;9#);K2v&Ue;y88aM+MQ6TIikG_=+*C@-yPY3%VB_+HFgnl1%4gj18!3n43j} z$8!UPoRXXzq55HBFHWl0X1%9)kF%*KUh7Zkdf-i%g)33j`(Plm1{ zIE~vS4_&j*(eBmqh)*|hU3lxz)=_qrN~TX>c8h==hdhcCeW3q=W|=1=C7?||=K#r* z{@|dNf65yl>OS$A{-7marD9O0(BO0sc`yxmx` z;~Q?;@N(JnTzIbI+vvRM=b#E0S2t-~<+oEsz9JgAx zHiPlOUFkN|vv!rsokcb7wxpqrMXw=HmmSN^E+omPlp%;H0bKP|-Fpn-mv-lspKg42 zijYRm0oo=BuAf(xe8CG*8nwSq$0U6$A6(NmX)=3N1_D-smezX*0#-kM_}24F?|P8W zR_H*qnLwA`5WWmdr!ZQ59I$}SV8yJB*nbR`wt`@f&&6Eh6B(Dfs0@f}{}oL#yXRib z3iOV@eZTuSaLhIRe|)V24OL~|HK z*ctSV%My>iafZ4Fk^m4u#}Bj->WuiiI>W-kwzMa})Fx*Ek|l?14ZDnK(Ak8T02rRPz^fKca&`j|C{~Ztb2B`qi)$lFO&cyJBxoHb zX~U9DPsty1{2kbgSnX(f%H0>fK#$S}!s;xIc-k(Jg*KdknG0G1Wf6_gGi zzYfruK>k~tnT-P=lq=BV^GZMe0KC^HYrm{mcdScV`wh-HJiel@K(x%G4uxf9k1PG~ z(4-{j+uQL2!7@RrvZ(52TA}r6&2bWPp8if ztr!Q7QM^FqvX(RSH~^{t8hK&aW{(aGR#qMU7=kTVDMTr%m^%b)-bq!#&t)Rj@FZCe zg0JF-&H)Wd_WBoC^2@gw8`^7_Z>w&n;j@_!pT*lvK%Jc6jn#5C-f1tLV_&Ggo!`L{ z7Up3Iryfq4h0r=`b z*OxmTbE5fHSrTcJcxohFd&IBXT0xgR-Cer3jO*%9t>SYiyG@OS1-xP)Dpxvrm-ouK zw#nIpvGsUt?e>RiiQT4AM2*d17sAYb6K6UldtUux zF90(FzH#itEWZOCQM6h*EnT5m#Y7Nee=`!&HnNRFRg&n{UB{7vY#Oy2C0l0Ez{u~P z*=wU&OgltE%ZlLCWW+{>zh2k4JDubd8)$;dy~r0Tw^DO|-X5e`Rw%N`L)28pvH|-g zerj9gDxyO&e^i@ZAcZyhH`_vED-k+miRcWqLc2|e8cL+Pt^|iF+i9oGt%dS&z0PkR z_!KH$8+9HIbkg1^ok=I7%@?4r2zzs89Vz8!CUzld`$zrIiq%}QseyRzByL&UR-e)={qi>e8YlJ*N( zMe4MS>O^R)L|rH`l?hY9#*_#rB}#7N;Gijmc`hgRjZi4LhGSZt2znGewJ)yOh*7nv zvh{==!!$YH1i~G}KL%ybg%9{~j0EHEt!(}P^(|#pzWD`irn@C~PS^T7knBCO%To>~ zhWOG7b34nlVwiR|ElDpYhjb;^R2Dn6Lfj~uSK!qBoRYhDS+aE9(g%*$&*9dFAR zmcgj>q2)sIl{WW{Jtl<;-)5q9k`CSI)I5fP8v{P?8#?M6FJdh8AdZ`f+^JA{>YVKa zBC9zudT65|aC{iR^dZVBOqwWBwvDvovvrSPqI1Fm;I_!i!NMC+GE5J`dq@1f+@AYp z0}#ht1Frpr;<}7z_iB)Ix=TBwy`jn*qiozZ0tT{@QYWenDU|Qe`vdf&RfFanp^`Z1 zj3-b2h(8Fcm3>wI*yeR3En$|F8mNcl{?ldS2kh!*p3H3v`uABY7;sEDh73vHIy8h{ zDG5OY!{Kltd(8C(#l<*v(FYaQz05dUvdgdnw`JXCd^mA&WgM5Jvlko?gAB#W=&fO2 zMcvln1L89k9>DrQbeo%tM$_(jg!&Tn>`))(7Ta?$F)V@7sskZbf<(So=6@Gx2hvi_HtqXv@c|EJhtT zo@4E>@72(D?`@41V(^2ZUt-k7(S*ZN@NrOFc(L;sz#$c|4fmBLz3jf)Usxi@pOGSV zLz2@1n^QJ=!^1?9AagS)pxs@WSe~$0pr=G4DAuOIHQ^MNHcDE(%H^*ODtKzkgGE!W zu(+^NP_|pQ`WZDbfEIQi(A+P5px3BK=D0_PR9p2vj51VCP_#LEMMK34_Gzhhff7Fb z^6l8SK!+i|r^ zSrvYBO_R^5^0Z%Q!c5imHg_iEfTkgF_x8doe@mu)RfGcA;izO^FfK0*ii zmYd(bpDI!9lHVfl${O3({`aW33`ieuG7`y8%D?V#8|2(^@gOS^_~M|7Q>%PuQPeGA zwhezC$P)2J^u7+gEU&|FPxGZ|3RS8D_L=aPwK~v*lTs;~$t9P%_LxkTch~!ikLE6_ zcwp#9MqoYf$Qw_B@pjyJCpa!0lov`eOz;+>I$hyApCG<7wJMHL-o)X0JZJd$i9K*? z)jBF-NT*jSiScF4mO~bx}m*3T6*Vhe*kpNoMO9E7{XIM|+$y2(+I62{DPdoBrx4T=E zcaXdte3GJV{Wh|Aa{LwLN)K$FI2pBUIa(cSTY9rYIX?7^gks2j!WnC+h zf=?(__`;I3xh8agr7GT^XT#0wv5R`-fTY@ZgO{M%BHHB9b6w(8Edh1@7AhM;_P)#Z zWmBubne0uW5O>Hs3tkninW2|ivFN#{MjqsVejd53n&2Q(4R$npp5D^|ET8s5aehN& zM#6{k(I&yqtzUw15B$8_zK?7KkyP>FvhuFPBxRpghdoodCbqe}PW>fYS2SOJ7+5HC z2QKj%ZfLKDrF!;pR-LpXb~%X0RT<7?tzROKiTeoVR%zPy&%erkwoF(M>6Bj> z4fYh;N5Z3`hAu@j3?avTlxuGO&<)dBeEhT1Iy!Q;4jRIgVRg2#0rt3%K7&_6D2@vg zdibydN;+1z2x7~(RtUA$AVtvW(cyyZ0~UG3>D$Q>yS^K+&h=MIwA+y&o=!ucb_Kd* zq(HQtc(7u-!R~|DxVO0fYa=i)+|L2fl$rwm3<48ogG|M{a8&V z9{0L?C3&sR0i#&~GiY_JTsp;^u+^HX6_57YgK1k|ULIpnddRjYZ~A?_hO z|Ld7bg;P>Z*?T1m4}Z#dZf|)HtH(%;>KXQ#RdGKidf}V)M&Ijhir!pBrzH%XF@HLg&s@sIXjYqqQmbRG3C3ZprLEgy5ccNVaDw? zImJ1f^4#-$=0ru4=L+&q&NOa+*hqDx&iAQ&Ajp+F@f6mc*frnTPGyP4=IAiucATfk zL1w`<-HUuydxuGeN^HSZVcp}$5{6uAU2*$~qi)n9q5PI^4hzYHfWj4ik*B9}+6Fz7 zNnC~nHW}9tO0HsQ$bMAsrpA#K?EojxiO0iwYg6a*cCdZ1h!>@OwcrUSp0^`VbFf}#bd=CF&%*KHjirKR)aNl^cy9!sYx^q8+tuvU6-@f5w~DQ{5l zt?6}nVMb64+I+Y$&H=SR1Qg1Qq+QWy6r_|Gt?xIXM zKF^+j$ql&=?QN3M5rl4h5T+N_RjpvR^F3xJYl>OR6}nJ!>{S1Va#eF0dnVFrirf2Z zz21;s07n_pE_umC^8=Sa*EbTQmdwxV!Q0&n_#hhYR*Nz%RnBIyr$a}pj3%M$qm^G9 zE^M0>KGn51$%GCZPbOKe*$8PMMLLNIZN(idUhbNEibB6wfj~xIg6{IORIjQUG1K8X z$32(3%5~MWsKcEz6?Tn{p3Omg4dnJi3zIp#i!4l)vZ|HR2fe?wUnG%px_ik`mx29k zn*F<`rR7Nv17BL}68pWR7qL~oY_eKz=^)2?1xE3mtyLL4$*LD>J%La^iB3EBP*cQO z#1v7QH9&SIRq1^~HNaBi zWDeW!3q=m^ME0FcRL(TyEIWVlYso~t-6`EBKcD=f`S}Im{Kw;SUhSY&Su##RSTxyw zqmP7>lSI#{8ssNV9Nwo*!bd&3IIo_IzZVx!1}?+()by<4f+>43ln3*xj>*UC*WxZS(9`Bw6%@D(v?HSQHjY+pxstV5>nOhz|JojPFR{>&+ z)?hm%Dlf_Dg7$}J2k{MS^q1d~zZKOfPrlQLw^fW?LyZM9g=_6EJ9h%BFr<~=KPK(y zl5QyCtxdUn1rZS~kg9^YR?3HwrG^AnZ?;A=RJ-O0;@ob+9le58u(KVZn55Er%E}b# z>CjD}>?EX~L}0U#Lz^GyDes#(RYc2LUydHNk#oj;3n^c(zwBPyjPQo_lB(fZGt7>G+(nyo@r|TyjNYi{4YW z^|0eC9wt?bPthXNY?!Nc zCq!Z$2m@EISw@Msc}hzrls7CjDsd30^cxx!pgx!YSukOrY%piSN3E#qwf zR+ct;5T__SOTh_*=GaUiB5?7S&T*Y+y2ccmjJB8j8eyPAsdh)++QTK9xNE;itF83~ zL6P7c2f)|yc5(H46nq+rI(OD_id*(-#3pL$pwS09<>;Sr46urM2NIZ59)?GHRj&Z& zM=d%$-sJT6wSAwMvi!R5&q_!gJPaAFXlEZb$;$ABC;G>xIU4L;<5U^GAzN&p1ee>H zGFSJcIPHHFj`Dw#y`wFEL@YE=V>F4scd6pKBrBime&j}UGnB}ry&9ogw%N`eP9it6 zCqrK4?DO^ZV_vO$SdBG_p^41J%(MAg$j&$B(GPfnwFcA?<=a(aWUgBOC5Fz`Fm?=?gK1vT68X9j zA1AIQM%sQ=?ymfy-b=misyVKGeKf!r`_h7})4B~B?g8A9IkuSb&28=yizi}7_^Cg! zNW@D<`{SQWksPHY(Jsi>MMl6919~bah?6+pL>M8~+VU8Cu-p66b2w2m z;meO)Q-wUdzCPH3DOG8aZF=7I8GD9C7U*%X)ONm97i!iD7$p)|1*)LLi<#fc`M#9i z^k4BGSZx}kD!bL>rU&(x>$I7Ne|RR3nvNH4HR;WZ=zni%`q=5#0f(`(&rQcVrO&&~ zEns=Av4>xOYp-`JxR|jrN21v^8CBcV4ndD)U&*F!V!WYBDEJcubKE^VKb6i&6{( zgz@<@8TgcPCtM#z>_z5&($&m;Pp=}n#r_6E4@z?EP5YK+Paj5Khp!}rRSqFcFz!5C zuQ~U67iQCGxj1U$*wDLW`5r~mH}WZJe$cE~>xs@H#V2{ALzrTS9;n650`ctFYg$&l z8w(<$hWz~b$HrJ{i3_=|1BG`bU`yqVzz{Ju?fJKC69SP9EuiODbDp%@w1l<6v%d-SZ9^N(a zo^Izgy|4R|LNiyy?W1hpFq`!(9z8k8xCNIfSks5$Lza6JB3d22(2Ac8YzK!BmPch? zEpBkz`=4~4XaJ27#MkT6!=Ee%FyUWQHCVY%2QApMX7z)x_Vkwe=9zm|(GV$mE}D-E z213A0EVDp$45mjgTYe|QZXeQp+sF2l93 zB$95$%Zy$uZGF=O)eqZnbLVr$%tvAu`Rp8HpQ`gbhtIwQ9!+0$vic6Jt~q*l{A_v+ z`Qqbm-r-19C6c}NO8%}fme$T9?XL~hez>v)ugWa3%TN_P>eXIjLxfpoxI0$28)uD~ zVpdiOfcbYCtkS}hDY!sp9~D`qr^&~60t?V&?gh9r@XAddOB`)Cw%y|MNn(;bd%AYx zs35z*U56OU0DW#_RKt&_pYv?@T3vm9@p4mBVCt?`*QzXX6xQ4zyuE`}IfhcVSrq-0 zIV8VWNo5C{J!yhd(K?X5pVS;#`{S~_k(lOjp{?1;jaMQ01Ls3iRD>K&m zUHY5YO!6zgfVdYbrxQ|T*tW733Tz{gs@`8}=ebyvoq+Y`k$sI{>#{blklP27Yu)9s z&z#B{kS2j3Hn8aBaq&njEF{aHM0nTH_g8ryh72(s-@K6FY%?hGXlJC)SsZDdS)jij zCIN#4sR{&TnJbkbl{41sY5l&!Mh)foWj)u{x34p&5Szxg4S_ zks@_?pfX|_bhbOb6G5f&oDF7Tm7PuIRxeRq+9;2iLkHG}lhxN{lKHJVdv1uq=A2W5iIYo6BytX}+ zND0SPc-}pMYd8KNU>Q40siih z2>Fx9^uhZCi{N&V{5Z28Vg1e9w1kn1qlP2omSk9I?AZ3J2Cv+k;f#cw6+NW&6{wTp zDnhqnbN&3|u|;4!m_FXS(oV#fZOko~<(#~;km<&fWZqDV^v;loSPtYk;&Y{rFS}@r za66wznXPPqee~qB{AXQreXKLg3r@GFb$e@b4G(cQayPEz;D?8g=G`8bKI!*V6(RLA zJ4QQQrjlSU^G~Y{M2HPJq)X@4+{>B zf%xzX(gng@Up*0()i3f-Ypwd^KdYH+^?Z3D=E0PB=C4op+I#H^U!uPAL~~hCySRUA*Br@E#Pn^Q zXr=rPxt6_HuCg(O2cjvmAL(yMsiN;s(jU78TavWn_P_ZUE=<89K z@r^)Ho~(Uy4v>(5fU6QLiMN}=uM#@TL|0Kr?;jss>jqAe=3Z0_-*kJgq;_MZ8b9=M zRP#vdy2|j$E1HJ&D-58*vSbTw(tBMKcAYcS5jU8fEm~LZIr22Jf4rj`cKg>dPqAQ$ zSvo3gOoYg=uZHxy?!5(t)UJ(qw$tg;Or$9BrkcBn+>Ccf#c~BG={ujx1%r|(xzb*Gcze_G&Fjf+mS~CWRA)Y+Iv<~ zF0bzr#!ybWi`-0IP=p zFzu1e*xCRoGF#J(itrK#M0xp24=A+TZH$k8#%p!yv8{Zt+!XzTC_PW>y`Wc_sGlAY z+Ui&(-n}rsfzhNz)%HsBvDDDpF&kmB{KL9C{5F7 zLatHR*&Xpo1@`)0wE0WA!}ntH&L~uh}w}7zVC}L6I%9x+7VZ0FsA{@ zy%?+FozIfnvurv8AD7l2vj>XS59Tz0T+n{2JzFkmXw$&6#yv+|>^%wvl~qOqliE|r z#hGoaddt=cXSQrsrY-$vZYn;w*L~tPIWUG{gTXBd%|7dcfyEIFs0@^ z4>YxQ=?+xqts-FctDoJiWNl~P28EYj$_gPiV%?G#iXD!=(j|1n_G;QQogrFy<74+W?SO zi5yj=Rt~J}6A?FPa zS;5rU1~65-kEq1H`>Z3tR8Pctm6bkQ(b;Q*r}UpsbYr*;yTcK&n1@e|V*q_YThB?# zJWHeyt!;Q;52(jiPH|@(_z%tlsRT8hXf9usP5uUAWu+f-Unbbj*r8+}Z;Phe^mG|8 z05Lo{@q{8A`%yFH$#k0!enf>A{er&iJ!DzwBi5`B5tX%RthLDP|7CbWd(|-Va0O#c z_(x0E%Olm&9<^Um)!)5?mgpd+bneZ^ALqMmROT66os@lmjIl0%(Wy5CpG#^XuUif}7G#aH)9) zMX|gVoR{!QUNKx>--#=y%G|xYmU7NZdgN~_sx35a<<6j>0rKwH@&UasR5eBIGg(`* zN9ok^SKk6`!V1A~$cdFde`YN2LSOkf%)05b%@^rt>dU^^o;@$UZ%rJ1FtgEb=XFU> zFY5ZS7o_(lQh6#neJ)b=dcWaPDdQnj5vC5Pe^%ZJRuk`!FL$uEuaeoVkQ|J!IV@c(R4vWzoUA-n*$Met_7= z0_knNfr@GPkv^f~c_WaHUPEJeiq19~Zk&T;2ZW*{2^|h8@OuBntIW#ce7$m2P&Y`~ zZIJVxLDGy#P8TVO-HfNy)vLTYAW*iK;(o=9DXe6zpN%>5;*fF;b1ex=#xZ(zccJkgi+l0F%&F(%~`(UCbM5*jd+l zfLtJ+1L^e(QLmshL>-_D>YhObQhfgYm`+B$vF5gT`Q`V=R|x7$K@B3oBm*K1c2itj zyj4b_!0alPg8TW%cND5!s^m2nxtTNkOaX=r2#gB@4mC}*s-wA`24gJxdoCj^^Lu3s z0}VMumnZ!4xDY0svrF!83^+d(tua}EnOq{mRx1qEpV!-{QUfH!IfnKh@C;w@I#sfTVAy3x3Cy0 zZY(zU{cf_KpGn1C-}rKH6797m>3buGE_o~o9mmg^mH8CSFk}ywT-CF8HDzU7%6()> z-`M-+BB#s#;LRn;OZy@8^B2#*IQ9W?# z)ew|tZ#b{<@Edu{{d}aM8SucbVfq?vW_=)*7hn)YgJJ__yDaxne}V5r4!_K98sn8IBES#VqGebJx4E{ZP5`t>|C$sU@?#WMV!3?hRq z@o-0}5*o*X@jt*vcKth9{59RNtl(~SkeQpklC0D4)8YO3;Gw@EFLQIqApk3e@WWx5 z2&aR)e!dCF0?t3H3BCupax%3YlQX76&rJu))+yW13CDgthp^K9JAHB3G#xzRDllST zO+T@bv`5f~{)LXNUIV`rh(dmuh24&$2^=uG-2U-#qSd9Joq?5^lMiS3mj1k}7d!%$ z{fAaOs(Tzjpnbq;ke~?y+Hkr4FRg!@4&2%V!j3|V(u619e#fjbc>ltsdZGM}^$!6I zU+mf0>02j_zW#az1o7-YyO=FOmjM2hAO8QDr=+eTjfU^1+pB1SIWVB0#rm(GAbE7U zM}JQZ{7=`vOig}vHl0dB3Gq7y{JJHoqgA0w@q5T`Kr5su{BP*U5cUqiZrtl`aqwqK zb@mE%_>0=#5B*yTIyleO@O^X;7miaZQL{HYdVW)2yYKJLaaLwDFN54HzfQ z|AU|X&x-JiA7FrVsV3zAi8DhB{(U2X{mdQh=ttLp6us)h%kp=ty7K@2n4fqu<|r%R zK+e!sOV<}aD^4a(z$Jz9!NTLU;F9a0 zZ?OoBC@nARL76VUtF~bCgM2&JfN1|FG@&e>nV?Z%6`=sXyftPrp<)@EDOlZ63EzcC|rOqB83NGck95b+YEkB=^s4p zS9cZ7_0X$?y)M`As7Cv7!cEwNHk@Qtpo4oL0mLR`ykPr?=r~qhmP2g!z$=Soub~xQ zSaJ+gViZ{0(JcZ;GA~TQ)rk+A3kzNb1b1#?PU5KE!A+q_cTV$S|17JE?q621G-SWc z$-qb6e8@!F=^@fRyWDWye6RafHRg-A##ef?HrAZ;?APQ~#Ex??D%`9eS6;gNRe8KN zA;-j4I@cn@?>VEn%!2`^R&=Q4EqEfEzOgL|?Y|2v_9nI|!?871S;Uxci3N_6Vp^(c zn8;q|#IZ9hK8<_f9ZI@RY+N3{*)yc@X7T;BWSLnB-l4K{n?e9&JJsOfTegxYhF@z8 zEpQQBLqUH3>%xuBbPX#ng$6Uaq@X3(ZVry-J$x?TM}0BsOQo)Kv6s7Unv3OVd3gRguQm%+X6;un0#ZC?ko76qCsLD*wF@EU==Fp&15`Z7gq|Sh!xt|u_zW_`D$RAD8 z)!TB&2ReNHWHP^!s`8b0MncQnY^3k++k7bxI?QGG%hH?&Z)-XkD|rqD8)#=xW)pPZ zC`fOekVjP6?7j7xz&1OJf`2W4q^pKaQ1*BeY9Uvt>MGncom3P1n!c$g$HWeY?%a~O zROk3Zy@U8pA+m4QoErA2jKRPmH$VPJa1EkUz7ZZt%;WZaa0=@N1^RUDu#)%vz zUFi!>x70q+TgkFMThFyN&2`6QkBz#$X6kw-m}qN|aoapKqOJH^y({v~OqqG>nKv2B zhW(vRj%C^l(|J<-_21B0YLPKa5z_6+c(A~?2VSQPq_wd_ujRQGo?_XQpMGAJm^_znjyW4QE0JE*R8D}HM*88hg|c< z!K4nYG;2u@l@wPQk5iZ-*y>ir+f#7t>#I6Rsx_VX6)#&hu9Y1^x~+21Q*H zsHcAK+3IwlH#DC>XoUuH^s@hv+z^3tQ9o;6KswsieD9lZ9!rX)_PwjT8Ne91Z3!E_ zJ1Y!IX&RnO|55o+@z4Cf7#)$~lOx^;KVb9I zEU$#$W*5ZS=I>YEG&X93H%yv514mpd3J8N9@fXd+R@H z{~%w6{^LVHgWdmU8uefP{U$e%83C5hJ$31eOur2*XhGH=rZBzAW_|~}QVBM^t(w|C z1OIKeAgpx$KJr@Wa5o%i??<99Q%C;a=GW`;@^X?dW(a}Q2cK#KasI~;`e9%TfSpxv|mh#Rt1zX754Vn-rrB4 zL2H8FKLT4UF;QZSi=;pITPps~Z0YyCN9gD(1H0~MrSGv%zaN3v^8R5`UWhFi@Tv6m z+2a09;C~Em&Ir2bM{XRTv9!3^AM$Vi-5NKsBL)a(8ZTP^G-kg1A)@au5xsP@1EC!G zR_51ADB0Bom1&UIdYB^Y~k%@Mm?|lP|BVg-_gA2>DCXm!M6Y4bxq901D;G>NUX3~lpe*pv7 zgy!G&YAMzWGXf)|;+o@8aPk89I;RCqP=W!yC|D4~Q2ix`jUi~ul7)8tUBebIG1p9! z=zc>UY+oiY?N_kprzV06&Qyy}WgtQFN%!yPbdS9_PuknE0KRR(R?s%`F3VUUk<&Mcnq3rbeRv?|DFF&6^s` z^n{bPMnQ+qAE0BSGOZ@Q=-rwh5XFqw! z3T`LOD0yU=^lOC#y65`}7Qg2rjYFOAp|k3^qu|ZJx85!FlkVTXb>X$r!L3(Dp0rIN z(41@9=~DOq0UA>5Zo78OX97V~voc_$75)AEGl51x?RCs{yH|`c=oVeu3fksMs&9?m zpZ4c@pb$iOS%%MA zvg}s>p+pmnlc09*9nfYWWpvg)(3Z@32=6bxo4oJ|(kc>MNFLt7qes$~f~tTYZz-Sj7`i5%9+*i7Nn!a- z8U!ID8Qb0%C=96C-O)x{2!s{dd;E-`Jn(j)gnxbx9h1aP7GHdyehb)K{83r?l^Hr3_U|AD4A1AvJ%UGqf^i?IurEjPrNzYx42bAvAmXb| zotOBvbAaT3`C$eLwu}QHSc{$Siys0-ZnjQqr_o#Ji$5kFKC?6Ftl@x2{q?sY=={G) z_n3l{V=RZb5+6kW%tuDEQ$H_NE`&8_hXtmzBGKoN_HOR|r?2F_Y)^GUXU&o>kyJtU znG;I(hJn?mjsJRl+1(FE&hjLFe9-b-pO&e%TbW+q=4P(0$>Af74pj6kJ;=eE7$`ue zK^PIlh9+x&r}i?E{Qj+h&d*Ywmd+7(s5q$GdkxB>y&A(O!Av^JxigWO5@8^l#x2bO zLRpvj9MHp{Hj<;P8%wOF%hO8O`9%~2ee6H`A=a#9h8WX1&nR18n?DvH$z&fgz8XG| zDDf5~Tbn`gsu)PgaL7h+?H+{%=eDPl6$z$!RZqOp*E2=WUMh=P`he?A7mgxr z@g~l4ThXCu$-kxBf4}sm3Bj%}#Lxc`V+^o#)qV@V!h}RSc1^{KmYw zx)o!W?02!o6<)P&FoYz`2z%s$>^HhLUP6K0g$H&U>Ewf$|H;%~vHzo;lA6SH;CHTO zJUOyFT(O#OR4A~}A36ZSsBs~XYA;%`XgsIZPRYof$kg1=uB*Ua@Z+e?lV)wl{U zL-{IbIr! zuRo%-3isrpuuu4Xl-x6XV^as8d#P@wJA832Zs@%S_Whm_UaAGwL)cBw6o?Ir~HZ0Mc$Lg+;WaQ=WJy z&E~NbhZRz|eKtqXdI*5)pTp&aEUDm}B>+%xaF9sc5}u{meJw<5*#Ud3c2sW&;H(sE zTGO3{mEo2e`qFFkiR8&BF2OJYaF44NowS2#B%L5tkd!u?cNq6&ZdL52j1Sb zT&!#Zo2p)+uuDpc7b=Q%Zr>wK@S%okopoS((Hvz;S4PU~%=)X>J%d9n7II?fhmwnh z*L}O+wEdWJ5uNvyPJ3LB4AY$iT`#e~%J?<@$FdHG+W0^*46$zhollrB@&3x&N*NAx zv#!-O?0O&ZplUwI^g4sqm_wUS7C~!0>n5Q`-p2b@<4&X{zZL3kufRDk)4s9Yiga1e zD~P&ImnLXi53033D%L2v=~-UuI=h(T#`Lf*h7+3Aqju`)rzvjoZ!$8HKy35pfLii| z%q1}I(rwo1?m1pCwkdBXD@w+gD+J=+NafY>Tke}%5-2{u{9NT2bq3tw)bU-rE0}tC z;HGZ0%&tk-jUNGnE=Say$pIp0P!%GgA^O7Q-9%f#4g z#pf~j_4tIxNcDA9?K7;*^~KJip0z&LmD@|KKHEs_el$@1M&9xgG~O>Bu43F2=$V&t zuMo5pu&+B~DT2_Q;v$l=H4|tVE80LGinREx8X0!mVT{P%RR1L#&#rD1}S=L2ZZnOUrT_n z`e(UIhI1V?^9wMk0Nd!eJQO2~iWe!+Wqj+7rcguXj|DGPCI0|LesY2EsnfbJ3Q*OD z0~djOId-L)YD>6>1x2ezZ8!M{802H~#Re28J-$a!s62M`xsx_guvRC+En@NUvp}C6 z*ZT|QQ-$mo2RuqEin3naO$+?`U3?=o%Z(#0PCUcx`*0xv+*CI3i$Kuw;(zQ%=R@Xw zVKg&BZy*5NG@6cHDlS$%Y_B~ZIy$$(CSl!0`Z`ydDHfvPR66s7L}WYlv*f?nMa1yr zw;3iRC}MP##oUhWOMQm#cWqD6%kubkK+6df+4g2L#pYl@CU1Kzv@90;tjZTBipgP& z0N;9uo%$XdZ}}1eZfwPnJ5Mi!hhRTlP$_-{cXN6(LiV$}GyiUmAliYtXQ$#Wh+fBC zIzI1LGuh$QLNv2HB=WHTLK6Q>dz%L}l2c1T-*cH;0HHf0%~6@H^?Qyky#IXU+l^5a zlg(2~`wlDB0W~qvJsrySXp@^+$dcdNfFaI^(K~EswKwvnUPyV3d-X{0<^@00Nijp0 zuGK`PF|XF|4to z*;|}i-ez)YS)+W(phh*1*(^@gEE}a{c5P%5l`1KBbF_9dR&_r@chZ^(EVlli6i1ia zvEO1gL+0{3h<5bz2MYDxcTWe^>takw0d{p``E0<49x2fA_=1WfFH2(-PXh*6>$k|< z?zcvPZBUq2zQrx>c=3$((foHA-8((y=X&=CVTQ8lZUVeguEGLezL-*spWubB2)oDx z12oF=R;F|vd7RyRm!$QX9VdY|H~z{r(4?C*iiLpKtWBt{`sWs1E8UB z%}k2>#O%&?87j0Lcz3AxzX`SYAjt__{)iZc=XpGP9o;M_<~$9W&ytwtUF9lXf4v_2 zp}KD(5YqCrqs`*)QqNZ!0vC}sEIyg+_3k{f#tH%klOeyQH{@Tu=tMT;TmtY-L0(Ln z$O=nK{bRw){PPv=Yd?Ukf_L$2oszC0+<6W{I=@EIfBKE^ck_c-#zaT^j=>cgBomDP zO?N^M(E>f-#+NR=e-HdiB!WW1hCSycR6$s3B{}g+>py$*pTMI7@w&5pfWj7}5!Z5O zTUyp1Y6bp7Z68oeAe3;lvRP9jkNJ0q?7ty2er*~dkxG{lQ-7ptQOw`eneOm|mL>3B z0d9joNtplI^Y+V^L3%gd?MJ-lqsySd3U;Pl@c<;Fe67L~e)VXA(G20~j{fp*PvAfK zrTj#z>xhizCrgz8tV}57`xY(z(ikBN>4iGsjA^dM1g2<5@ZlNQAyETb%Lv-Az4SNe zn!>KL#=UEov0S!Jm|wis0WJJex~GN)UglINU{DY2hOQ9Lg645xiZDa?b1<=Sloka? z6016JrXq#3qD^L}&qp3D`}JA~TI%m|39RZJ&zXQ8giHLOwK$o-1Jst=0M6TPjd0>; z(-@Oa%(Pm6_3f$n#Pg?Jt^>W%;#oPqk@Ypj?m@c$0tZYV2 zue0u&O9CEsrkX{^LmQD9^4baB<=bAOL+bmjIu58lP_IM-&K&YPse89v8f!WJuh{n= zbncq`Ck2D%sArpAoIE`iqI}^%ef2zDW*7N2?&A#8uOPJN27KzdU{oE8C}Prj6aUxO z5_24_WRqe6l)Vkge`PUYI#9cw8>Va1&((lFB|hnD{(wsjT^_gImvFMM&dLQb#{{BB zA=qAtGN4S(<#%hLPp{Ul5BSn0KEY-R|422-ZraOq=ta-GbeNp<>eE7G@CeZvwi(_4N6>~(nB=5NqB^2UE8 z<~UeNB_c81PjBNqtJ3V-4Ph zS#iQ!i=p_L@`Z00$0Ox$!{6Wf3WOgwJ2J8+cd6+kAal!iOTk4z-6z)tb>GR*ecb`% zAz+)|s)i}T{2EttCqdx4Qfl1RPPI*y&(mKYRr0M({-2i*009`t>H_EP6Vq}6A*3Xm z3F21R46NCsDnGYEGH$w!0b^^+u9S)as&@9YG~tQek&gFM{s)Dtm}&n-8;H z4C`?TrQ@?O@;qn=qho~Yg2=~#D~V=0G1Y+gh(9(3PqupqR28J+^iEd(lc1PA^IJf! z^F|*~nFr$6NfZh&^G%*YlXDmUR9Yg~O>><;ZToAh84SvAaODV z)pHpE00H!-JJ11S+=|&<@$6@R0V88A^efg&mP=LHF+G1zCbYZM4qbO^iFUe3;nRNq ziGucn6P+PeU|bBVl+pHexR2EF9j3OTk`pf?^ew~@4zvclm~JL}n~@Q|*gV5- zEfC}BxO`MvS1bTe5alsiCs;pSnjmF70rnyJuKIG#(Yy1w{yObhZ(*arz2%M`E>;a6Yby_Zr$z>*zoV)uZ?IdmfAC)QN^AZ&v$%d&{>0o!ssSwFMy5jz>TaEa3fN z6HbixZ^|5<%O3vu>P6}E_1YkOfq56zfS|%=e8fD0zvHLA2LG(#14{_57BNZXu&WIZ z*vt=;WDnIle{fvm*vVjkI_Cc4f_eVIv0HB12d^(SkAHFYK+VTp8@uLtGW7r?llS@l zIeB?cgRr?Xfv{Uk0f#Ge!`Gjw9|E9Pxeb`v)26quOsPBI9mMAAeOKP~RP^^E`!l-i z{)tTT8BYV0;-y=M#e%vbliud+Dwt>%IcJtx$}3@WCE})HmV}Tl*+z6`Rx8lNzp4Vg z61@~Oe|!OV&Rar$V_rE6cK8I}jOti%K8Ok!le3O_E9b{%fA$PyeGGsn?s7Kughx%z$5ge0 zAF2}6WqaOB$=p)AVQ^_9Tdn813$Ld2d5@etB{z?ljQE3ZW7}s^PGz>vh*m$E79a6C z?w4zqPcd>@_gf1M1~at03uWt<-<@0)14XGSJI~~Y6&#?=d7186;M%@XffBw z{g`+=YB*Q0BB1*zsg0HP`o1kFqg;&BtOc2xy5y4$jqtdTd>UVG%XO9Y>(h_Qp7`aEB&K z&z@ZshehB~@W6@$ZYyjU1hzElCEaz2GoUUD)_Sgrtr-xZ;u7)tne%H^{z&^W^A%FP zwcR+*w>qj;u)3e@3Q#mX55CYW_n}~!ky)n%zC-;ybG)$89mTjDbi0)t7my`RtG#~c zv&nmQtErXQs9yjdaUs;F0}&vZbA)poetF?CBdt3|W>T`%M5XD{diZJO6LIQlKc5RQ z;$321%xnWz|q}&%Wxn#O~*;=#s=pSeOl+3;;o7QVK6oz_eLS-9B zg2M??zPqz1^*%o3o|6lysnb-vNe6X|E2jVC^_bPxGofPr3%13^=dl8&dn*r1!@FFf zxcMJlQuSTt4jVS%9j{(`Nm!j8zp>Fca!2mFHCr zoTAA+-eV**s|)*86#JDTDZyM`is(b}4qWI&^jcVS#S?v5z8Ki+?$zEuL4?)8$ z>9J9ay%xnNRMJFJfAd$y*JAR~y7CaRZ13^tsplKa=jel`9HE(Dw*%*iQ(s!)e*Rz> z!0toFH7P%@b*xoG-6)q}tOoYcbmNDd?m1-QD$v;yxIT~iyx;ma()6Np)288*-Egms zi04aQ4`O65p^!N!lh5jAvh4n3N3FaHr=vgWS*1X4L8_S%UT_mq*tzU_A{L79pIQ>! zMNE;;+_#pT-NT-maOdnM*UnRKnQJ0K4!I->++7)d>xWdRCXV9N9rsfB)XitK6Xhj7 zct~;3DDpzK`#T|_Zw-VJ?GcRyvP^xbIg(Yem~3oHh~>0ouPuezBP^K0L{-$e@s77_ zm6cF2YdkG}5WM1@BW8*bYIPj+=dY>u=ugma)Nn^7+piQp>$jRbb6LoKl4{uYRDgKQ z!B{$e&kEzLaAP_YUwxDuo$^jf`Ey**$^;I4k?Y9!&Usq6GJ`orQEe8uj$eW@2AFkzYNpvuRP!`HMO~Ov?NNuXuFT}-R4v4NUtR7qMxIJoxCd{u zrS*|4xZk06M^C|?x+0~4!wb=_wpT*^U*S^Mq_kdf6;ZBio<3>jp5D^f%_|IT;ob9G z?u2dp{pX(Hiya|zS3-rp&kqG%FB8y!N32#MS?sZ2_O+U#{>aGC2jkg?PY}!{bO|+~ zxoZ)*opU6jc+}>Aqxe%Woqw&)b#Sxh(t&BndrxE*CUkn(tN)x$FY!3F(WUP};rL6r zik8Rw;6Up6x`;0QO|p zWvNhGlNO%#RWzmeJ&5b?<0UwXa6g#IuaTR5 zbW%brX7kPztmC@VHF#1{8jt%`RITPlZSYSYEAvyQ{-mC7a};Q8rdtUeAS#ndn{cd?%*1F_J!!hrg#+(Cm_|NS;$xR+Q?Bzh8ITt`~97$|eMzXLW_d*B2P2YBH5# zv{Wy)SbprC8QV)P7}XAknT<69D#(3faI_YAjbE=K@79xRFFV{wnTV^q>UkRnrCH7c z5OT%R!X4mL>`HPE)z{wq(cqrxlfvuo$g|hfoWA`pQ~4aH{ZvHolY5$mrQb0&x2fc& zGSY_K-%J**qc&!P3;W^&AFSFq(!R$F>6z#RFI0IQc0>BpAo}YGTG<214|faxAe=J< zG1)DNs`q<77&yOuC~Gl(YXeW@zZQaEDSw19GCmO=3l96SyVGv;o}U-w#;nbjjIADK zf`=+zs@m6-X|f{r#BD}2T{@_NZnJ;diX)EJLPX_{dT3|JiQVLNQGb!5CC#;p(ph~& z4-K{SRci13kT|HcazuPC=y5o!HkmoP6qZ0)`SFXc`I@=Edj1Qc2w^7L6^Kn_HvU2= z7ddJzzwpsvf1F#v#0P&0u{!n?AxUiB0SriwF!5WyTD{DOfH@bSs+mb6r53%o zW6Wm|bw7epf9x)LayUv~QXEE@ux!@?--bx)(X+K*>MuTvZBb4aH`HrdM@lfP&RGcF zvK*swyAbH9)%m^0Bw^s8N+NH$vS+7(X6P5a0r;pLE(6E?9Y>9rHsut&4@sP`{qy#i zNt^bSs-*^AGe<0}W4hz5p9qOHJwqzMXay3)MSVz>K5s(W7EhzBq zRbdoF2X*gHd){5YP^-gh5DQdPe0hzwjMyCV!qt}d;zYW>y*of$ulMQD=a}nkpIYka zWLg|mM_5=#B_VZ#0A;kCXa^t9evBGkY18x2;?+l-z*UP8bL5_B@bH$5nG z9=9=~=DvRaD=aydH9>WLec{(g^lfWFGllY?;K~8tw%h=;o*!8uBcAfoO+Rko(^Ny^ zyI(rnDBMk)&~{n&wrgjmav8ZQ;cj2C=5uBJW~;|YI2B$H=0kX~<}qre>yIN!rX2i; z7<-5*3WhID$dVl@Csv;IPjS<_Q>LV9vJa0{?_-+d(U1U(pp|P353=C+NEzwl7>Rr1 z+p?NC5!frOeKLKsFYA2PK22PIBAAu^OyG|7jK0|`oN0ieHOkB`ZOFRaM}T>VSY@v< z$_QL$MKl%Xj7Q~KP1qP|XH1-2@3+37W0x*CwzesjOq4AwL61ja*MmetC!N;? zh|8*q{a7zFaTIUvNT592J3SmuYujx>g(@JZnPdtzczz@M8}n7O=4^-yP3HtEND3@M zBX^%LM`i%02$525_a7vD-!fUC>t#C?Rfmo%9}>8nPI`WE^g6Oj-0|LWxa`+K0mI?E z3>N8GKhRF!j5(mIYQ1+6_asa2=ybxBySu;L(P3?@c#&<|_jJhqN;6}Q0%oUb!P4*J zAsLy5-qf&Xt@<+ecqrMjB-d)-CW|NsDC0$ z;opyA98e`IVW=2r^qr^PRp5q`dtfN4d#JukZ9e#(F^u_cP`X_xYO_2ne3^FA&6rZ` ztB_oX+c-k{7Dm(dwpg_kTFtrtSaTW^F)!LUu|I$yp}o2=R*YpH!r1-r-k4V>blp;T zE)v+J{^DZ%pjK9qby|j~4>**ZLYkc^E@`&Ct$h21!cV6SYCMD{RtqEaEJTEIrVRQZ0> zmb27@+SmwsgFlZAgR6ua_?#R{L9$rGfJ=0CC3JUst3cT9`0Vo-+MYh$@Z-a0 znKD<1N>{}0Op0KCH$2-wR-$w9F1&8@Rb|N@lbI@eaQj6cS%?M)e(x8K=MUqq1lgOP zc2E<@62EArCNTcGRdR|cNUd+=hdW<(w(!uF3>fXEJt4_Mi*zNSq<2B>LKJ()f^z{0 zM2yC|9&0i^^`MK}{mnXPRg5v7oO^7sG6K>_RSxY>pL%VtnR{4@t>-^@8w2UISIdms zUY<9Re@pI^DqjxK=#l%B;UM=>l-J=`x<<=_ZLo0?WOknfHdv-+6M`D-*E|cWp>`Nk z+y9u3wLWRdXUyX56sl;)mm)gW&u-$64W;lLf6dWEQ%Tyjp$Dm`Ekh(qFk-!=f4Igk zAwm;nZq@=aHD@dGdJ@+0*D&4cuAm`R6f5M|p(LEtq;LQczVV^G<6b&4R-z`PgZ#7K z8_qX9?wGicOtn!!{4C_}X;r>n+;lbmenv`=)FYXsaD8@t%h^s0g0UG@1$hm}JYkHy zJk5}c-}NJ0Z|K^na`A8l6tgi-jUho+7MC%|zUU2orN?`PsN2Sv)Tvd(Ajxg9LNA!U zI_ss?gc=-Q9`Y{YlPs#Ed#&YT==s9e*QRRS%~bgXb?Z*>&Z0xdn#sMpMy&z=9#iK} z!ebNB4uy!#4NDSHa1~~5tQjKhlCuoNB32`Xsi}AJ*^K8QieA;cp6-J&Gs)$Ne+XG# ztRtTxP_7nAj;&33>D*DAj4*S)@abUxg4$wKna2k>M!~G@V<3|c^#NrbWhLZlb$zR* z@)~rwqIl@Db7PBZrO#052f%nn_D@?+XF8zK7W#K4rb0))RNNYh|fmmnrWudL?DMfG>?Ohe7FTVHQ z=5qD9!Qu(zsdf+gWhHm1o#n!x|DE}OzMyBZ9(rT<3*}I1^&C;TW`nxM895u5 zD!U)BW%SUt@n>*CV9eH*D&YF<;(5?KzF8?`I%N(5(SlAy9S}hYgSwXiA6`@+dBpMX zRP|Ql6_CN$)?bjqEgsoil$pA3@b_v&xF*kfXEykwdCMNBfXf-1p=%9~3BZ_rH~}q* zJY`1P-lz(C{}I?`2V9@!c@RXBYex)B>ui; zPOHpj`{mji=jh_ntzSj1S?s*}lHsXC+_$B7xxYcMimn5KUk1>55M%dKtwXWy#RNl3 z5BRo|9%Bt?9ydlIY$OoggKkRxEljVtIONm0L7$Y|xV9$gV-=9^nnNwKks%91*LQu^ zPp4El(DL`A897w?5>(hO>dd%HXWin3dDQpwDMnG!(mXs3O2s{K3Fy zXJh8gcH6x`Ipa=J{?i`OQ(F&lvRYYk*2VGu0g8Mz#n`T+`Uv1JEIPerYp7!=`d}0)8z)qkb9RP8~Vv zUEPU79=}Aod~yPTZYUFaw;op?_HO67pvY0{9a#C3JI}UfXpq) zxb4-@C;In$jN(9_wVOjqId&4gf3FgbN1)Gi`PChwRR8{RsDO$d62%d}^-AESUYUOn z#!c+JDaW!d0tCvBCg0xr&`Hnw?|?~!$7jHQ8t3aK2Cyg3Y3tMftgll7E$%=K=#5yp zna5-guI*e@eOT_!gLCEhkDrnPewFOTwNksCPW)$B@mJ+K-+u#r4trU`xjm*&8ovkp z=gPAu6SNpU*5)2i{v+FME<~>Vz}Rp5x-Va5AI1O^4NUIJ)vu8|5R6mpZ^l|5c*uEs z8Cbkw8yP#X(LXanllx|Nc28ul1@JL&eU^~l@ioz}CmPNA?KOx7kQ~c4TMW2xi^P+;@DMYc~%M$mwb zo))424_^RAWWa6;36H=UU(w&14bbD4MvsbW*N0lbh2JO$S~B88^yAr*jU&j_&{Jx1 zy!*R?DYpAwyD7`{FWf2|cB@38QBbHn60?xtoqp1;qmaRsdd+pj`gU`?Xb$ z=OE3Vw1}W0+L@RN23T)&DMq6QJfZszV8xl#3v4hK=rejF@#Nniz?B%8Zdt^eSkXIz zUOqgpr;G4L?=1{*xl5jv1!yVv!Rg#gUu&rfUG$1OOP7CS8t?7xUB?8vU!sfhCI6&p zTR=UuSvd=0>I>DgN9FL((8=C-bJ}Qy`XnBzJzaxK)Au9g73!0D{FAY-V@uKsYa^H| zl+xNwx);#0sE~P6nJW=bM+`5&dm7lSq9Oi*L104AjZ83`fzbtf0^3S~tNr(yzIcpZ zHCJHLhyypJZvFl&!tXg+ZJ|Vdce=85;@LNevJ05grFMxOBf?iLu#S5&TZ3LVh zsy4NdLWV7}Q26~+3-f83P{V~n18+NlkfRF<1k|yoLuhv3iAU(Qm5CqyKXK#={j}8xBx$11+qdhE zz2gf9V!-p@B+}FgEvy3$k9Jr$W;xn~cdtORb2S_wQ^jiH=7T2y>IqCRjfv8?M1F~s zLQJ+Q?qy5yg6eN&0m_r!O8CINqYr1wrZNJga*U#`X?CdL>npAV)YY!qcI&s|PW*vS zV$AV~SRUB_6YR}<@AA-kp+S)D)lO5q{t3Pk^uySbd~ND$cHLf%>$3kT?P5)imYvQ|P)-QQ7jVbJ4lVl9t%UtKl`1M78t zfsT!>>Js2~OKOndE`#}FcyHcV^;!)jA3a}&Ck+h?DGPdzE`R9RjT0Ic53!%PpZ{XF z8?n2+keSzC6!IXWnxh*^M)E3^J9t~PUJm^9*cRcOl^`U0B+)t<}Clv8qRAkgAY_v2X$1}As(;H99 zT_ROLlkusOl-#KwQ?2SxcpK1)_@2V6Hkp;cSNsEQoHQYr4|)_WvIF)yTwTfD-WM}_ zGYvgb>ezp-8_-fgX+wlyUp)Pxl19|Mg~F59M7`1F7a{g%S%tB)pK=Qb4Gr5Vj>6+0h5S9biD)Hje+XtJn6} zTc@aUiBWJXOYUJKhrHSvQ)r|++GZWTqAEHMMU#es(B--|3|a&=E4uoy_&#^}@D z-yTH#N@sC5?kPutsD`gtU2>7C=e>I>Kcu`R&MHP1&DQ3wzkEAUSnoe{`GH;9{b7rR z*Ep3+r}#DVh*10~&SKs{h!{Bs>JInPACS^s-}RFxWnZe`;|W*UhWJPi{>s=+kXt zP2E#W*p^>be>7k_V46bso@TEm#h$EgXn|&*DObd+#joW^{|qpk3NY+O!*bR{K0tFmZzJtIt#;{0iu=OBy=X zkKEERi41mn-}>~AtSC?WG~Qy*?p&bSt1o_v1z=B~BR1c}{{~i5vV^WF7{9;m6!84B zKMQ^|Rhxegk#YLnR4d4>@=X8(hs-~H>{iUxwDl3;VTHy|7P0{v3x31l4y0u4IZvm1 z!|If0!>zEF`1Yb$wX1+aN3Tz!^7LC76{Mk7>PQc|W`U*xSgwVe6jvXh|2XS3{_OX% zD`@X7r;Q3ykTl+p*N(`X0wlSboyPDhrXmK{bAz{?nol->^9fv;{WrpLN^?JtM>IaZ z?NOo#()u1~@wNNH(An>S4%k7VvlirJXA*khu6VZ*L$&rY)RxC$!I$?Mk#uP#w2oN_ zEv@bE4Vj>a+gBgDQ}~Um>|~|m>!GyRl9RQ`L+g4M4p*MG*OT9o(qZ(+xjM%aBcoSU zpNoP(-j4vkDiiY`cHJ}zJzp7tk-s!_=nRx6K4~Qg`|upTp=62?`s<7Q(^Tbt^d%mE zfu3DLUQjPccks7|MrlJ1d(!Yj5Z^q{N;D;jobQQ&lFz+i7uKQtG|NAXKn*7Xh_d_$ z^E2RDNJf6{0zj|O2^`ql+t|9g^>!M3nq*S1Cwy3qs}FDVeX8sk+Ilh1r-XqhIqJd0UvhAjsz1RvDjf`Fz@XdnR_< zW&B;+14J7;c-L>j{vRLJ4jnh|IBKsj=b9d&ZYbEVj(xWV7e%skh-*>5t2cqE3?i-0 zb_=V8Zsa5RqX$)0i8reKD(h|)$beu=ze;*5ql4l8#KgH8%IH-UgfeDxV(t-hubVLq zoR4E9Df;E>uXj51Hj6O(KF$&+qDd?4$_V#bcK+UliZJ3U;TMk?^xQP^;dC-WEFI6^ zOH5`Bm3mrpiJtz=D1d@Jf(O8XlY7@F+#W@J4N=_B5G&KT>v?YGm%$Mm)xCs>JBYG$ zm!5TQ*F1HTUvN?q@5i!Z@q6?mM)G2@0^_0foXe_(vEF-8#8xo9MI`@~fT^N1eX|Qn zPQ6!qlXXx1;@4)8+LBwyad@J)n#tRi_BS>+(%d1O%$%K|bDNZ~X=XAT^9E(^qlU`lrkAM({XXs>`bY6DHpSZ z!1eo!;j{BFp)7+*S4n0^lc|iA<`rKyt&x4lXRQ81Sn%V;;D?&In(H&!xeajxmQ{Xj z+5jZ>c3Wb$+1pnj(-v#8BCv?= zpCQ-sZFm2=BPfGP<$Gq{X%Lr50~P`MGisuEcx7-;mV<*dmQ|XrP zNgUA^n%*-dE9PWnxei0eHD9~=Xhv+1KYBi9(U*cyzPjlAkj3|iTn?U0%bvJ?2`^c^mES;+KdBo~-2_LijJ8mu8fd z@I~aq$;x$i-OFCT8lzL(k=do`%;97Fc)!zbds|E~MAd^2QY@s}txu`P-4QA=H0it~ z#&!r99SUV)YwuE>QUzMEFrgB@NvhSU1GJ%E8Kj{baIwz92`m>S4jOOQOPagJacvP*UdtSRsIgk9YeQ0n8=`G30?61H`T-k z4{W;QCQiB=_p}~gKlj~(f_3NR2o%)BtR<0RO*y6S4|if!7={rU_eZ+DusKO3{MAEe z3S55})02G}Bj0YnZnxS!ExFP4MmB0w)l@+8qkH}Utd;NJ!?g)9tZQphd2bY36YGAT z;f{l~jebV7?USGdnCxmY9jrr&(cooQR4$s7DlRU_9e({iUL-?4-l1I#XBPuSy2N-o zhS$%z?4)M5slJ&fqsBBjQK7>RI8R*^HH$(H?n|uQoERfT?0I0nyACyg z$W(GVFZ(M`>h6nb^03~%NyK_0V{WanPf_F=ptj(D4QjR@w$f{oBQ3{0y}P@{73GO2 zu04&$6a`&l<{E@45ONC^Di4e6yPb(!pTdeqt&sK--{;T#YV3!3ZRp-#R`~(*Q65S~ zdPy#L@LfQv~d%8u&j z0!i+A#L*IMlWL`rb)ClhQD~Xx74F%!(u;Z}`whVVvVM_4Ub)=oP_Ky|6M`n~oE?H4 zz6VS>X@M3pz&Yo8Zj1hS8kENU5v93Y0<GR@L+#lAc6!8Mq%zzZbGxUy3Z~Pr+HV8NGK$!U$aB8x zY<8O0Y3|cB-}UvRU!ChfUtuTpgCiQ`${z<7Szrm5IuQV$;!Nhmv5BciIm6w=)gm z+6ONoJSJ-|iU>Y{JZWs^o}YR@qvLM8ByCm#wrzX)S)ll*xqCE8a0q#2ZPCv6ux+k_ zP|EZ^zNt14@{lxLQ^e~0nX1cFpIs`}hM(?`6#M(JLP%~Br1lQBB@k=YFe$Q+Atge7 zht0v-Cf-1y$_0pP@e4rxg?@_gsY_@kLTvEHi?bVW)>*(B{ZK#WkaxSd-E>fkfL;iR z#aTyTjZ8c1Px`TjThzyIc^n{WyqT!@mKT87m5=<_I5M#^o2)^dMbWy%EB+6jw+ci& zsX>gMjrnvZ)b0l;ToW9woW!L4Dh?HV%M=)+U)6HiT$ZV*O_2&w;#jAlv(9IJAZP0d zDj+GsKL$DewA%f2FNH$Alb0lSSYM-*J>8BLgP0&L-)cKk(n9)cXr&=avz!)`9(_kbF#4L%gk}%XQhv_UX&6%havTIzq0|`+zwM{sK)`v{pvM*mt@Su zeth*j_FbBpjxs?fyUqp_f^Q^*RHQU#3iaP@^C%vQmy4YG zJ8Q5-K;K*L7uXrHm70Hmxj#(HzG@-lqx*h+J1|&t;|TPJxh}$m9W_SbHZ1D#n7O?~ zzr+n^!Kf>(V2MzNcJ7WcT<@U=IOxar=~X*USLCKq2ByBXlv zN>XzZZNU*Jv25S=Hc;MWrprde@V*fMBCt_j@G z=4c|Ks7cz9m|G1^R&oN;bKKXl6Pz}%oQz%4@uK6*=)os{6wm)D&p9k8gvp{edKFJw z@B93kckg&uJSCXG>o~i931A5=|?Jm~x&DRY>*?w_F>X4{A&FCn;)Nqg+2Ebyt}4B}(z`DsAe3 zG!^jQZs--WKhT4O_$MDq0+o6a8(j_*{Ulv(LZ5r_&xpQgZVL9&-d2^1%@@t#XI8Y$ z7&vhyp1mhjE{qx2CqJ%LT1nmCe%&8b`)?QaW&ni1Mv+j0rwCuB?$5ROaO=344QSu1 zy+eGp<)YsxdoJ@7-kHhw0V;SIG5xE2+M(?XDwQ`;ID4RIy&-DiBwSm}7STFC!}dJJ z(oDATK_G(G+QoT~@*ya{hR#>TD;5s0>b3%am{z^bUTnVIR;ci2z;I}kiJ13p`iq$t zH9u8Ujq#OFIHG%E0}2$)$BakOKY8|MCjUFE<2^7oV{Z-8=jc@y8KZWouDxHy_2EJY zfsG$Cq@~^TD%Ptk`5=T4wsDZ5u^sp3#{B#r{1LP5&Qdx_IP+$tD{>(zY9aRw{ZA3g zCkL@dRm~n?4vSILc*V0ab3V4)pLA$GWR<=pE-a?yb-VJKg0-r6cp~L+GZ(KXxiuO+ ze3wF;E|sQ6sN)UN8JYxW79VXTfbU6KkdsQ5@{4QpyH;^A|HMcjrzg3Bqc~-wKEX)K zx64ai&&W3Yj?~%q&M{+&=Z*_96$qU&S;!V&xYk_ z1hysErH2_IDrm3V&;Co$jLef(bjVXZ#((jpc_Gf23DyFv_LR(~Id{);^zB6#ye2NiWxA(}J_>W!;2YrkH07H)P|*R*oD=qO&xgtxeJL zBg}Z6#{TWzs%f)pG;GY>Q05g(AP{@sNV0P`KaP|Nc`#!wr=1kap3)&vWzN6h_)eRl zTd3n|JU4VXR+ahCA#jy6Lmyd=LA!GMGrd;xA~I%O>=*eg=F-OafJ}-+m%_f2@lN5Z zR$?|h3Fe>mIL3hS9KNof^r$1eO;-Rl1w+4cZQATT2JW{~3jiXfb4mxX2UWR!GRy-E zB`Of|yyF^!u-3J;^YlNDlU z8WFBNq0V2?Q1a}s*&M28edEYOp0qssxMyNVN|h@;Ug11;Df~N7%Iri#F?M=)mH@4E z_?2D*{)4}iNelhF;CMU49Hbr3_d}3+$A-LF-pDk0s?&gXkQt7kzNxnP6e)FoL~uWb z2}raU<5A4BC3m;cYa@W=mi-AT=Lr&Z^I|0lFgFRiO28L z9fJ24qP7I%waKdOscVl&$IyfzJ`+?k3pK#*XdBK~=@u=#>VTW?eg;m2<$m-X3eA#VSb=Gtm z`@>=$CH}~vF8Z?Vy!$(M+mxR51;oIRq{TWJ@o`Ai&g0A@-2sV`vv(hX5N*0kWrpma zgd9l}6A0RzM@=~c+FeF8%?ZqQ)u3HXml{8TZcG0>M*LFa_)VN-Bo1Egc4myV2^=H1blRWqJK(K7;M(&?wsXu(hb3FrS=PcAO3H= z>{E)a6C40_N=mrff+Ch!?&J!@hIYfWyBX5`T)YAMds6D|@seoUUu?i5E1H&VXPA}{ z^$Sy9P*RE`OEQkEI;<2U;qN)<3+erPPyIt%U%^&E#pkP8SAH>*$zq|(Mq{B?^;>r9 ztg4|9pDqwuCp@kDp$iziIUwLTT9-&Ex!sEYSP*RJc#{EtEZh}AioZ^6oQgh4kH#r-$<7^ctXTfM}<@`j3*G` zb6O8Me`ox*M@0g($c<=j6|CF}PHgi=&|D9W)bH#DRF`joPm0m8PdA1Hh>L(7oU>Y( z2o-wJq#)=_nh$+qwKy@%anW)NWcVKhv|c-JD?N8{4CHLQjkNw|+-lOlFdd*Q<^N@{ z=x-|j_v5kue|a%!Sc{l$y$i&g(8<_J=Ka3vwW;4U>a0EsfWl7H;U{z?8nlY||D^~( z2FkySIe!1(|66GLZ=(NuS^d{1HvXH{I~@4mkVE=!R{s~Ux|q!eiZB*G;3_=3_4Es- LmrV+dobLY*RPy{v literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/case-study/akvelon/pikle.png b/website/www/site/static/images/case-study/akvelon/pikle.png new file mode 100644 index 0000000000000000000000000000000000000000..933a68e47ba0dbeeede94343aa423be6c4747e5e GIT binary patch literal 502346 zcmX6^RahI{7EMBMEfjaB#e%z)LZP@j1b2707Afv7Emn%VLvVMBJH?^6+ok_~m@k<; zWX{>L_Bw0t30IW=gpNXl0ssKerKLV90|20^*Q*r?;q{l>kGmJIA3$g2Pal8_gG~Of zH^>fBTFw9fD$c(P2uRN)03a5hNPiSl^~gMG^Gq$6(;~mkzVN+KuQLfk4*>=**x)42 zE2|Bnx!(A|(~ZX4OwK!ZFTtIwJdwlICef zCm#Y4q|jjp1(p@)td!}`=soD)`0b9gWIO*oK4|;ZQXp(EQgl-0-SKcdeDXMU^3;Yt z^HV9WuZ#dw@Hq#TRZ|g=)@0x5hA2BO-+39lhT%T$OD{X_+mMs?0j@gvflmk0C zai4a6{ko9j#u4gACh|~G`NT6XT=fZkcle@zo8%tZ=SAGl3?s#VujQFE`!BzXK@IE2#8<{wGM2N2P>bK?RmNzk7Ih%c3yvd43?PW;bC|C?^hdF zKd$nw!0xOSJY^!bp@5B?Udg39KX8*sz{3XV?fo%?>w!PjGdAE}qkCo<=rP)6ecEFl z;@$fEy4IhLD|tclud7~uJnQaQg1J8~Gj5cJ>La>~-11=Mwht>%Yi6Vge1RRG7(RsB z{dh73yOH-@Cm`LfMnF%z&rG;yZxtg_MdVeywCdVo0P9J3Zll*N?T=jvsiuls1@^ z{=}c^1;u{851m~<^RM!K;?9?ReLf2bZqMvbrLFt^PdS+Z9eyWz4mZJTTSdFV0FD}e zeBfaPKe_(`--{^thRorwEmFA!!wc>_-Jj487?NG4tmpNNm#rfV=3K(`M@SBg!V^47 zMZI$b(yOPg>zjwCc@4(m0f!hLw*1I|$0YS9f%{GGz)c1bWq}RPN1(?udzZP&q5quI zt0ZRgx}0k^EIYqFME$6MU0aCrp0AzPUurwE>y%~a?i`jB>4$+!JFt|!*y-ha8w2hS zl=^6WdZ=CAfjxfqn|7k=`k?mkDK{O)(Eh>6vmaX_{g_Y;w|alF&Z8JZZ{m6ZsM%k} z^|1w@&&CFAfWh)EbMe2NBKpq)#`c??BEE#nd^cub$jxobE=JePvj2ve{7W<3jXNzN z^0P{hs6V>W#IU24zufRI3O8~*hOXd?)Wu;`{%jZBv)fHagxfcn3;odhnencfBcXYQ z$-fy;vup0gE$ZRZ+oJkwk7MiCI7F3%uX^2cV>9pVd;=A)=WEhFyi4V~v@S>O*wk&s z-@v3*9i<4nN}UQr8JINLk(T<0A;dZ=7MwY_mhvhc@VLX#zZ zL`n6sKqvF|*wDsTeN9mv*8c}p!4}SBcgZRTy!&|jxZG%5!L)rNfk-sLF5>CZyV%m( zLwwwxzbj5Cxy?)LH%KGVh*_*Eyj$L|-WaM}U#z7JO=E#87BZ@SLFr$lf6KN| z#nO3w#D(#79GR@pty!w!){}t)W0A-{gTr9q}f0EsX zU%30+4i}S)yHnW)*W=;8DOs^8rHa*JKit|0omKxb)-dO>yBfZ`8JZiMAr`Tf-v|0UR#I$9iJYv=fg-^BRHQ+34Ogxh`JFtXL?|J!D=SW>-ii9_dFvfcJ#SKDouQ>&;_v;QDoc`Hb>A$RCorQ+OmIWS1 zf-yTsmMi#9UUYguH|spAqU-g-~bWi;=wd>S5=#0MIEAH>N!`Q0Z4(STrxkUQcC3E1NbF{TqIr&xQD zh)bX5|LeU>_5HK$FxPq0vM-(!;CBHLJ`f3Da3tzzdNPBL;5qbKkyW~xU~zPfk6)_V zxLRM0nd@H@6Mc|O<#!=oc0A>MT-bbFc`8igO1DC>)Nt z=S~fIEP*0x6-VxkB7e1;y=qR|lsx&K5%qN?FZU3x0>>Sb^Y2OHep3f!}M%pFpiw0MH$rX)UKx3Emw zo&9Wql6Om3ku8-_Hhb&tzL_BNQsHnO=;tBvX|}Skp7AjJ_%E-QN5lrT+=WkrA-<<6 z(0S56h1Jkw9ZJ0x?s_8JO9pSZV6jq0gDkHZR0qK6IgzJco;)tBUYu&r+q-UjyKU}- z#FmHWZY*l#!f9^itOIqXWOFQ#I)g3iv+#9w<{qx?eO2KsYWZ^E%8k-9aRdF7GIpmMbxvnuDQ>_FI%p-{u7uh3N0;U ztsL^>=o@7s&fw=;o@U~{@>#h5To;e;A8*5*34dhLM^vAIXWn{& z(+sz{{VZT4TO}tqNvnUZx+pP@PA&lhisH5gf-yy7^x$SL1q0q;vHx`a#M(DFN`$-U zb|)os9N4rsNU<@vl^I9mVigyrkhTx5KC;&c;Hp@k8Dy7y_SB6odO;r=;y$;x$#88i z4}7yX@bDw9YrRfU4XU308j$7 z@%~Mk-5%jdr02JYVJ8-o!Tqc!P>-2e&hzve*V^LHc(~QE`m4*n=xXww5oU)6^WP3~ zSAiw+i|HEo9sx|-Zg0O!2A^wptKXHcY5n8B0Lh^Ic?;v|I216n`ny#rc(#N@hOt@X z-l?xPjT4IGe^0I^Ynd`{Q{jAM7tY9y6x7Kq?Q?}@HBKmq&qAm1|iHFn-_lN{uz zRciX`Fj|Q9`&^6<$Oocgq*Ylcr$1Wp0c|oexubqKZ#{-u!<2R;*iTsqw|-LSh?;ZG zI*o+$5CzEg(`;hD4VX61$36q+Z`<$6VA(}_lUO5Y=LMPO2zgRgw74KqnG6t~uT?Rc z?Z4e4#Ib#N>mh;~%T;+8;R|S9`wYn1uy(n7az(6yr|a1JIJI}*f_gI-)6)AauFGa~ zu>QVv_w-C#_Z2RQ$^Vr@N9wKT@eaa5jAtgMoO@fJQa^A`X+@~@nM2fBkm&CyKw%p) zC6zJoS1zC$%tCAzOBARGs!24+L!%0&H2j;HV3`Tjg)67Nut>a`HiP2t%3N&fVYRM{ zLh7ehQBjYgwwcl?8!TWmh<1~r%pf&jKB)=!0?c-D?}35I(WOrnFW@)o&y?fJadb3n z2j}5mVh^Nj|{sVa013%uy@8*fa8Z$$F>6 zcT>&u%C+O;z(^GfM>I3(QP>fEz{*2U?UKXFS^(pdm?l48=-e?6?D9LU$I)qco4(>d zHuPWJ-10)?LtXYf6;4fmkn@>(_I>;#H?Ln!f6tQflJx<_0)q{0-^ z<$0@1Faow-CSbH+M$gNbftjGtdf;W1X#j{^g_SzZOw4BJ6-jX@E>Ie68T114k4tE9 zb0O%Z!?IRYg_f1e^D5d~WAvg!BZci=Mv=jmcEs>ssl-pge^9Xw(aW&IqN@9C{LPqM z^kcxIxJN_&gabkHziEqnU;a70^lt@iaAj zjA`LD2tC^PTkW>sAVLqm4s&9d_UYcg_(@)wi0^-lxLjsE;eOS7wG&wzC>)FlyIHS! zJS-pnfv$M`>25x*104HjBJf^rt4Uq6(`B2DEt6ZW(;Y$hXl zHsmEqdK><5q9cm<;it?AGL8*X1^bUyw{{bM)Bqw5R7)?-1eO5|iuxJ?t2tk06|a!| z9t*luqOZYrnsF4d3L(c5D3sF!>_;~U_3*@!z)tyDHOMbwy%WvS4l;)RLu52TIc!BY z_8)&ScZ&6pzv%syK6P&%MXqQm)g$Y@q&U!J5M~Ih%EtT)&&u)SwH$WRk1?ol_ik!@ zjHL@7gS>5$QSFuPVf~Bms+_a$A8t6Ku${uA(p(@IQZnA&eYS~s>H zRD>?k1zubE&?V1-w&{_F?=+?ocdiJWrcQ|(Im)EfZro8@-|zc|As&D3#DSVo#aufI zrs!xx+DUA+&8ciWf!HwRk2$k(Yz|Cs(qrq$WI5g=o2>#oMWJ{mq+e%nV{PN%%)Vs} zp>H>$6rWx{22ulyVg~7DR7#QqMx#@FN@hi9+8Ll)pF+Y*U^ND zm*TYk`5LJO0&l7$cb`qPTDkuqU?&Eb%v0t=P|xGecbk;lpYM34z25XmP|TjzB89y9 z!Ug_D+D5*)sCbAKZ=dLH|nB6l+*6V)}6<~RoIjwOe8n3*F?H_|SuGq{|kcU~H zJOE@zN&Dy*=|z{}-5Pp-%rhffUy6C?^VSG9PjGqzjLMV~o_35nMg?=9X9?u#e@V>^ z>?G83tfzd3!sW4QHuvpjvVysLKC>h5bEbXMlv!i*-uL|3zOnp{sF&_PVw_huGzyqT=cz3F z;NJ)ug@zYJ5ML}W&kAztTrR;sw{ENrS_fD7nIjiz1ywp?Y|esuM&Td5-z?XUgz6Lf z*g8J9ZSV(eyY3n1U0Ix-Agl@K8o=h}%fxa!kg-D1ZFJB)h(QejQ2c=m|Ik2QIA^aY zap2x%!*owA);#yfJK}>&UK(z}`21;?A?d+%eHr|aL|*g$egsMzGm?qP0JJL0)Zo>>&0_G|2bYE zk(v`z3CuTkuKJfguD9T3Fg?ALX!F$9+7HsRK$NSA+B;uIhXu>C1{zHQo zHj%EKZg;&G!mcY(iX*oF1&O>rs98dnuKmP;aOIpZbB9yjTe46hmZW}+GDHZmQaaZ& zFpvXiJP!-W<)vT7-(Ul3D_7G1_f&fR(zomisYA?UOtzj?=HmR^-G!4G`M zPSrQk$aBFPiP6BG{?bIdPR!hdHqu;@$Y>FoKe4`Z^JYYadp|4d6z13wcr=RZVc>Am z`OrfmJKl~~NPdg`dh$pjZd2y6YK{GWW#AURz3V`jMtl+Qfe1zV`_k(caLpEoTJVv6 z?EnOD-2@H;>vu<&(uYf}jEo}2`!l2gROauJBHyw(m&Lp3zf-TU0zGAzS8%Hdt|$h) z2fhatu3&)FzD{R*q-$1)K{U71n> zH{IS0If7H^9Ts;f%GVh|Yf?FEd%Y(q*J zd&i;hYvUrQK(Uk9#U^vZ-6JKNPq(fRe-aUqu?wV#TA^D3aOWioV`CXoByddt+-i5l z1hQWKs|FgrsI2kA5EB5ogHK3<)&nXGjR^r_EPlv#m17TyI?G>>!OgxRivCL>34@<^ zYhipkw}h@ATMxdS?ZqZ@$L0Ya;&IhxD&zDGbM%GYleZk=dEpN!wd$7%hAVW zqRC#TApKM_hh_*>AS^gQy4_4-PFJJAJ-~dfeUk z35|$P2%fXW`V@z$dvW6Zqz@zQt#{4t zEH=;qgo<)fLEs++SVSrw)|rCUp@4{`N;izT4A`L{8q$c}X6-G%Xr-X&>C%Qh|06nE z3+L+nrA5HUrU~)|wYV$Q zW1pYRUuv~BcZB82^5c-hckAzues@Alf+zIv~BLxgVVQqzQxW7IW*iMYyX{uPTY zC!<$Mr1kIm!>Qp##p(W|qfchL7ps}KQrkX@==q0fdeRcmLSv4hVaAo zh;!lR_wUb?=%y#9vDe@u3h0Vqzzo_f)JfD!1uS#BxTGZ!&%PpMH{f#%hHr3 za*B~D0wjM8|FUh2h1U_5BhpRro08X-TVAY&uGG5569KqS2+pG+=Izd%v5&WblBLx3 zG;3(np?3(_Fmor9P4H9P9&PcIFV8FD9C>>K1n2DOp2N-M6^8<&J)A16V|CE}AFK?}O#Q zzpWdy8C$by7#&InaieU5b8EkkW081+hqEsm^85Ah9asWzmzrB|&=#fJ1E79aoM>;3 z>?$%PO6ATG!7yHCYY#`lN(oVKbTO84XZmyS)RyuGqCtUMdtD7sGmSkRdO1yYab3FF zb$pE_XPkYCvyTG5ZHvX2y%E+Ukyy(ZPTHPGBaA+=`5KHBp`FYC z)sE|Es;t6tLtuno=#6A!K4|iRD!*08D0GiK z;u}_;(h`jMBX=HhtgGh)nmrK&EgM4ubL$!$@wCB3zF=FSsR%nHsl(&a&W{b{qRu)* z9PK^V7^f}YD?4cP2$ZMrpGdJhB*BZ|0UKhhuno>~V)kI!Yb{}LF^*)ov65B98Lat2 zS%XQk%!O9^$tC!h2R&$xA!@hyA#@ffawwpsKU{|M3G5~ndvy>O{d?nb=229Ca3Np? zt^Vy>dr=r(C{>k(g~16E8?4TR9n{d%UOxghHV=fcz(F5LZm79HjbvhF2Gk+xmmTJ* z#TeG+x!ottqwr?mse{aNX?Tp{+>w^;J{%!5l5PHghU>VYeASdvprH3kb6vEPJ*>+* zX`vU<5AWGQfL}52L-JyM0E~>FZ=l!sW8?@3HvY%ONB@&vt8{!$MSRo*xNgn}x?aA- zwBKwdoS1sI0DONLIHYpY;ZwmyZOZ0KbFDkY(LJyL!5Xj;NW7|whr*!0z6iZsUK1(q zpa7cUbh8#N-E`GA#-H?v0J(B@v-s-chj|9dcT`wJEIA>a=gyG)KgCHnnQw8Evx6`K z4|ba9Ur}BKqUnRuL2Y)$8#KAK&XcpX@nW{id0C$mx(X0GTqg;hl?Gekh&vLdqV+`D z)wCxhG-H)kCnvY6!_j_~4)Kdv>g=^cRzeR!f|=~;M7Fm!&YYKjD~4!pwKbAaU|q@k z7#n|rs+ZVJo~Vistxeho<8i>&V#loy>k>s($rrE+S{V~(aeui-v(+UR^ioYC=}xJA z6L;{`pqEDv?VPSG?m>d?CQERXR1jm1w6v7>kG@ejZzirdcj!P=qgwF~Sa z$+f-X&i%OyeHdNWg5)xvV=sc; zxSF4r-Q#5mR0&IHT`!)cejPsk>Ms;bx_Bw9kfXo#L^eid>Q}OL*PgMvt-nNU*T0JDL3JcL3E|ls*S@hz?9zXEi!QXZsC$RY^{Cc-d-|pd9Jziw5Tt~-( zg!Sg=;~8F(=RTIb7eRzVQX(5}1Wibx(l5oh*tm%9sgqh5U{BGhVt+k=m`YRcYqBU7 zC>tg%Ne@g`*gSN`!eTAv-SF0WO%R_~$u zTPh}?S}7r1S~{4BU0h58tnxhwj4g31KD6Az(TvZd_=?!f0tqn5_-V0iA$CJ5dakbK z@Y}<<%#;N!9?2oi_x;(p7fF<6WU(`L(h36P(Er{R=}$9OS{l<37%A0Q0};+*_FZ5i z({AemR4YQ8zH>iKOs$g$erpE_TCT}n%4&ucNQz9b?k=z%A`lkSph>rm71o?(dwSbI zZpSf9#FJ7fF4k|3D}WGR@5Rf06hDx|n&>jgx69#4X6^$U8iQg`j%x6z*!*C$x`A3+6Go_$(O=0xaHU-+ zcn@#cQL}6NuhVb|Prn(pF=G;Vc(K+Sx(Qpf0rwpkVn;tKSv!3koqWHbRZu)MErlwi z$*23g3~NGsM}kM0WBGOX3wyh6CBX^VIOad_ZvPZhjRmPWn1MB18bT*}9)7xt1`Kb$YVbQxT~EF@){ z*(c?X8(V3E;rp2yBHtz!1d!eko5G1ZMlJZTyC1r6cJ7?OzByPF(dktcmeo0Ld&#KX zAvMf@_Y>;5&U=-9Y#nYo&DSRta@xljmGk=KURyu2jdj7W!^`EO)8ea`vkC#I0wP+! zm<^wqq>v!2_1)2m@ET-Vlt5z%QA|gM!Uq(6=BWgcC-~UFh)1o;%6)q;Xa>Hyl3Y0bbf~4OvzM7(-J4TG z^gXOUaZKGgU}#{QkRGKylJaqQ792vRufkW5)bQtplWYJ|#$OS{fB@ISqXZ`SdHyH- zO4ox3NY72>Et8H;6FRtov$eGjhQ>_z7e_|`IenB|t%#^BUG)#m`<&}AKTXxbW09p(9Qlei=l@2| zbkE&{hpT6n(!&)=|4p=e z0*yK?;wo(iyi~kK_E^Zu@HmAu?vP2N(;rRjx}bJU2uO8!{6{#k%RU_!z|<6n;oC3f zWK(XmW<9I&Hw(drYB{qqX$BA|S|$^hUozL&wU{hN)vW7N`E0EQ39#E29ty~bR=gH?&m^x%WBejd+{4N@bW3V_J&yG?mT!&R3| zlKtZuWo=cq-`l8WSvmaG+S`c<(&3NJy?WbNmr&}Q|EDC}4xMf@guOvWSuVonyGH;| zH6s9h;~TBo1SgvcCy1kyT6>j%(_WfU*fLy)xX|2<9jGFn94l%Q$rhK_)5UFiuJSF_ zJ^qRA3)<$9)B-!JgVjf*UEk!uifPvXu=epOc2MNCLY3*%&4G*71uBYg@Mj_vN&Yf( zgj-Lc>JayjsOWu#&9#w1BcmpT!JNu=fYv`jID+o6jfy!mBue+6qX15m8u-2FST@|t zTHlR@q@+!wP2rYGn-IYD0lP)iFFJO^>oa2!ByR@z0y!HwKSRQ&<>QY6+FYeo7zh1V zu+Z~=xy@disJ>IslGlj&=vQ=(zO=Kv-@5-JXo0A~Ln$AeAMW? zJQC^4crbiCly59|8RGReR@G#Z(qt8Xl&Caks4B3@;+(P4K!9#{GI0<%%T0u`1CfF8 z0!ML+{fN&d^frw9SX=RlYVbbjs#Lf@Yxn5OvT=t(KA?t?7Cpl7#9Akw`u9*+5>vWn z2|TUN2Ub~QDSpqimvFg z`>j@e`F#RJqi4m5{%efq1E@2+;Mz0oT5E_Lu4vjM;(gH)Q;RmR8uUo8InXM}ij*SO z;8qz=HcuPGvCBlae#C1T3$w1oNOG-h9SKd-F;3WuCC*~~6TRM43c>Bnt?$Gghl&Gi zba9xE9i$mcm5SHuR3OWlp+J0hP53$6>QBsB-h%4icSRla=eXP{^+1){&LgIZ24p%F zS)Znn1(I-jGVlkK2~vfSrkxFd)1ksK(0Xm>-yW0yhEDV|9bd?O2xWyX9b400Sa0Wa z)(?EESx8${5jJ@-&l0Co=d8cT(hlX#c!SGFxnpH12mlVsaEw#2tvNk0UND1rgJRX@ zPZOWDR>k*RtvnRWZd9`@uin<%*)`7oMoVe=zz&MVVNTD71NP6Jb$NeiSUV}oK zPN|D~)6H{GmIUwJM9@+FZ+T>@XA&v5IUbFrgQOR{K% zQ*6D+&~MJz;ln@i8P@3otN24vMk_MOn*0b+RE~@SjHOk4ls}zA)Lo6~(Xf(dt*a2P zZ=<^Zn9Cp=S2m$x0GX3U3N}%@e@{bTb3iq>ZfC)AR^jG2m8*_TDd9x~Z@*({p4*Y5kA@HeBmLMby6I1QR*a zUTc_t6yJ!lXRlNXQBOXRC|P6(;J&VZY&(tcY|A_Xq~nu+N)r0GM9Nap;v5Tb=gU7) zW3>jWq_B4yyBW1%uUQzAx@ro+B&36szuMxYZxDG;a0iWd*pmqa6s;}^~V-DxDJ+UrjI*^UQ4A+4Rm28cLYKmU)2iYaVg>wWC z*`oI!lkx{D+bb_duvD<|X{Tzdl1_r@roi4O?0MD|2!%t2R;PDdTqz-LGN;@9FbN|g z)OUe$*hztr%_GSsNKw^Qs{k6mS?+s%Un%6h1NZX=5`SYSF)?!Kk7$h{o2p?UVD$A9 zw{N@?`3+3w4vT{upP}&L))O=Mkw53|me!CT7w^ocl`dG?cAws4v5yv*Na)w-95O^? zvcRnhrvSV_$Gul^O@?s&@}F>I;f_spDL-rm>#^0%<#WWBD>@m;5S?VX9%8?TPJDfbQcmd~hexsvxl2J9UPyb)bN&nXIW!C_6n%*DZeo7?xm~mR=!K-c< zgXf<6iEz66t?a|+WX-WBjOPQKqVRb8l=yq3N0BBD1ezrGnJtYRj6yJWO z_4q)*Su$QFyI$Xjfw+t(8~;*rhy}C4WgyOj9<`XB#WDpZo-e>%KROQJo4TxX4}C;htU~Q1TJZYluFbzB}&cA)X+EE*K!L1*C;h~qYnh2 zjo9FHtVHhK2ykr_dSAWqVajFN5BtepsW^fxLB6w8X}_hn;4i#}OT~PH2jAaC@~>$N zCn5Xd{wOiRKIf7ZuQM2P*#BT|qt7?SFQLQxy5V%CVr%P|xLjGlZP2YQSN(Y;fKuR| z5qjSKCl7EdHe?Ure=I+&w7q~F(9o)*?{-op)Dz%>juCz5F=J|o0R74Xo90Nadi1}wTuF* z@14%zkrrDu&+9`bWu~kK5eqUt2e-Z5UO@JfZYoEV>~eX z8I&o*OG$VvOndgY8Ra+ zc3DNd;AxE(809+~o!DbwS)ZYc4jOPcuOYY#-ST-HZ4?l*~i5q2p^X^mxMD>o)hS{MIvh>ttx0n|6;4CZT(y>O*=lEy{i(imirJU~@4OywnEkk%?1QO* zYx~w=3YNLPNJgJ70=58uHEAz@u$Ie$UsZ9L_{Nw{->q9r2hYw5CuaDd+Vw2AF2N*| z%tT%y;)|;<)uDo~_BiJzbT6$Re@U)A zOmP$;SU5=mRHpdaFD&Jzb>KDZQ>Lc9jx(Rdi3IkG)YavnmcC9i{OjtnX8s%6pJ>19 zChyPqd$MU@ADprN$YnM}Adn)gf;Vb@Tke+jS!=dO|YvcziJx!_-EY@7faJcA*Kc=8XFK{F_O8dQr-T^~t{T`Mgd zOTwQKx9hD7cT{zaFm0X0^${2JWJ=h@K^RiyYMZ67L`kesIm32nTtMWga~)yw@-B~< z*>G&z7s{_QnT$75>daRqi>DT#lF^Xt#NY+=)Y|b|pAo==Nm6LGd_K2h4N&Bl4;&Y9 zLbDj+5N(}z@lz5jhdc<;w-4GTI;%G0VDK*^l4ef_9=B<)(ii8myMO6vNX72CUqNs? zpVd^JKug9N$o|^7NI0)0gx+0OGhti)p`iv;wbXOykkDD`4##ZV(%nm<2{L3)IivOv zgIH&gpq-NT|Gsu)D86^Hdb9{Zoeo7fJ&5te$i5x5YvB5&v}G1Si~)L` z8P42WyBRoKHhHLkqqXI8Pl|@4H5pRy_fzF3WpU}F2hr2<)oi@8`qgrNQ<^l5c5PAT zH=?f>>gP)q9f=b5rwO;Iw6H&P2P_Pw??$=AN{z==7kr?3Xv9d4{-7z0V7CpHLH5$w zg<_M#5>|*vkZuL$H=N3ugOtv#aYJNsjhB!bsorj&g;fi)Nxk`%tCkwcpaL1ol`G;A zWr$3#`da-As(l>r8(t%}P0x)5Bv!TApD>q@zLiQRQ{UNFVI=%;SI?9^VRDM%`E!|{ zu%2vTf`Z!0)BaR5RVa>}H~9^c13q*Q{q2;nrQTFmJWVH4&xe@}_vu;fDb1>=%L8Ni zU+huV`#0&IF&alkzOhr<=LGiA9b5Oynj$Emym!S2SL2dKm_L2m`%UGGB|yebp4}6y zwT9O}l}tL_Gi~>vjNQ;DU zF0xr^UW(?rWsssZvJ>zTD{ePO1OBr6wGUw+3Qdh{)=5(nA!-UNy&F8Lj`4*_Zn^ zpFT|Jyr9pHP_Q=dA{w+yJWTt4yaWD4(4r8q&h8HoC;#>|~g9mu| zQ3b0KT_ajdR|x(o%M)BS>;NDyPv~~_$(<5+Me;S$O8U(X#a^&hW8d|3efziow+ekj zKlhRCtNFl&tIk_k5OA`KVU2O=ov97+v)rW|h-Zfc`c2~9m8GasTC?h+ittX`-j~3u zW`DGZsyI!4JmlZBPlX!m>56ik5NoE=z%|M0Mc=o+AH_Ns7gIHuvk1}Y3yK*7+4AHQ z<}>%yI(k^(0IC;bL(NT+~ORnMJ|xY$1neP>m-P z+P{TVbM{=vPQlP=e3FVBB_t|que^G&yR}_5r;sz!KAhCxB$XxTNxw1jB2V-+!$*|b z=C2;Dtom)~dc;dSo!-4)Ft}Tq1!}hE5|3;X&@4JWB1*a<+`O8EFRL@#<&gVCYm;|7 zS-tdW*~EW41jdxqvZA$hW3%-2L}zO5Z>jhkI$>nJc32OATggBlsc2oMOqMZZd{v&` z$WHguG4^5R`-{LPZQit144BjjD_w4e0At@Z4}7w##{4?&YCq!g$a3KAV(2r{^JN3V zYa?6kU&pl3bL+Xky}xc_ZBy>bzMCtjxUL<1Yd;VWKFs9&>36Me0~5w1aeX0a>p{W zKQLN#fxS#WX4D>|v0soW=4j@K=Kp20E6`I7uP{;2dar7RLD|#OoOG>;KwL(pwePon z*&2U!gAAy?(r`L=Qp9`XfD%w69tb(Ajb{E64QH!SBHUmpfEof}n`xHY!j5rj`Q$kH zQ?P5H$>hRmX9U{sqBqMEj^Vf!)z9m!%e%Kys`{v-&5oO<-F&@)>d4zMi zw^3AK8~yq=hQdGcv=g!LseM*~)QjNcZ^t+PHWya2cOMhbhsD~*1w4D$yp0!5?;B#w zJXR4y;Hyl`DdFaS8h3GXkIYb23daCYacTDASgk`bVaJ{p>I%U+!U!u}9J)f*0nD3g zqxY5w6@J7(J59K~+1$>xItc8}1<4|Rri7-E^QiwbjBDMx|D=HXt;j&z{s}t`<$Cga zEnLNdvn~n*j(l?CmhtI}gV~}6gmW}O0w|-HG^l(_l-CSmQlO6uaeFt2lxWI4$H-2B ztQt66KlrDHpS>)--&3lMxlhv)WaP|mWkl_%km5ufo@ECOo6^Iy;5J zk@Wz8+q(tv}GxowTYr<=tP)TqOaXj4@n#KG#9 zOq!ZoFJGXCPHaC3D6~p?PD$t~?Gv>ut_m^oG7Z$JKr0=K znH9?4(g?6wdF(T-hBS}do27-;e>ll3M_K~5Vd(t0)YB)=py$0R|9#wP-HmwcAI};1{%aHE!@HX|1M)^0q7#c$VaExQ zv$_}vs#3g%wYK}T>Dxl4pCL2XIfy{Q2b&SCZ zSp|~k08rq$aGh_obIUZmyNK*)R!~32%Rbj=t7MeWf1Mx_$F;l)NB5ljwX>bOoDG8p zJ&Wp+`s_18*yoR?%r*GM?USLAbb7poUYW&bmxJQ#bt7eNR;jC7R@#4#o6+rjfnDd$ zNcsvNbOjMsE)0UQIQ)@=YZQVEgfX6uopr;Y*ryu=JXkNvOD1Jxme)Qbb=z7e=CTf0 z>gB}otM&tO^SzV=Qjv~td(ta@mD}8PE!l{gs**VfADS_l$evIV?(ceM^|>T!M_UdzMZ|6;v3x!T+etm|9<9tf5?3Jcc1qaLv zmXj)6%qdGGqxE+0ivhIfT#-KV`;bxukgn(7>g3KM4{Lzb(LGl)IILBfvJuT(@pU1Q zcO$ZjW18;BG?|#u7aJ(>iU`ux@00D3RUa#N9R22EFGEYRCpN%*FHuS#Ex=9h$3w}z~vv6tznlm0feoA;SFVyW3H z0p4WK#4rmrtjVS?yxW4~`m*c_2mx?-gNPMF9N00panwR3<9zw45-znX2m zmBqkY7u7TsB1Ko@M$>)s#SMieElDXy=?eSD_)p!U0YsO*Okvcl0py^4CDY%gQJ+Fr z;Z;$%G}+5dKf)XC3eoq^ixqJ&5uTO0GTo{a`Q^|$aGA}{4iZ9?xFJb4N$8ulsDov7 zP_t=OOR0HHL%4TndZ=bxgP1NIwCknFe0le(a6fuJqd1(yQ;dlI7F3FYqEdLNN>81! zRMa{5_5BZ+Kxe;>`muWbju|;Ses;Rjx|kDzS)6&7WD`~@*BP)^VBRKI1)GOYr$0+6 zhAEv%fIfQ&3y5twnM_6?M1m4Vlhk4wUIr03Bp-?qIJN7mD_)E9Y`4y+b%{!AG0?pk z(-|Q>i?j?oemqhlXNXBek`~tnQ+PO|OF|**<8?gBK|jbmDm!;LDWyyQvG*-?atHwP z$cyM^SyF%!9Rk>4p%#_8aE^=h^=w`;{#u~9p3cdKf))=r}3|^xuV}Xr(Ub0ZUgd^ zEi7wN%O0i~fGXn#phJ9O6(vt81+WxBr{~(1PMNsCnmMZ5-lK|I+l$by}QLwj!W(u`$ckMt`;;tb zz?K-(DDyCu*DG){*8-Y`zh!UWPRh9IG3IY*z1v2?Ya`RkiGb7M^w23*lxxX>Nh)Ge z8mIFBvb)7NFPrv(Vb788M6+9VaGGR8W=l*bn-bPJK`fE*r7l4)eFLabe{09m7T*HtJVD_N|WTzGb5zwy~g6gV7Fa>dgX-9f5s|S74{%0hpehgzxy- z{|JBl8?{J6jq4f}#MZdJ@xT|}ig$TpU;h^D#eKE}R2P*O=n$?x$OJ?v^O=BfAvrLw zB9jadc*TeML>#~Y(!p)bgI_rX6YHk{7LUUAsV89T^k1;!K(l!Y4#Z`c>{d9VxkpCn zFhr1GL&ox89Y%#6QBseR@jX9SgFb{6Tsq|!?a8Z8Zc7$QL*70oAPGQ>h0w6vRigkF zWKkJtK*5B$qEeUs##p{sJWtbaAj6p4vT407tYl}6EwgNYNYCQ@XiwnG`YOe?cx55k zQmU2dTGyoMgw>Vup~Aw`zUdM$c3{UQN``sklC6`RA%Ov|081URZ^gEXfGYz*8^%}g zDj-_R^Lm+IW3SG1yH0^}OjcT6aSJoTI8D!htXX9_*}L8}lgglrYk|rEpwK1TDPv%i zPbXnuIiZa;uuV#yV?Y;lzUf%HpW|O& zxi8DO1c((NVh>|`P%4uVGCgEE#)dPyJx|egTDEd)8G37{Aub#RH~%yY&io~W1xnO< z7FqwgQwHkSd^U3Xp2s)Hq{SGYGW81SU;ri{ve73T_oF_7EITw0#D#Jp`Q0zCs?t z7Y#Hdt*-dO3NxLT&Ah7ZWmy;1!M1IW=gXyoU6z?xVqnq~r+pq(QsC22e+gc*MOnHi z<$FKcacXROorgD93Fp%ge~6=$t<(LN{`R>O<cdi=o-VP01#peK0oqY8yX#p{uzX*S@kLN z-ZaBmhe}MCX7D>5?oZ1VP$I0Bfmoa`8k zT8UL*i~+3BfB;IJreGbjbAYaQavN|eEf9d*Cjjd#nM==ybnLal6|u}gB@8-kI#%f% zklo4d13jE40Z#-Z$Rv+%#O{#--rYOT+%WNxK)V?WDx8VOYhP-9@Dutrk@6v zOZRGLL{6*GK-N8uZw%3-2e?A$C%wuZOKquLyE`{P(v0Eqd981@N7ABqaH) z$Xf$r#!HD$zOKi;p06v21zp3}<7(fFVD|A^B%#KBM+LDp_Ah21c{$8{__y$;_K)zd zFT0VeBKF}i4{ArB9t6>eH4!ChQPwf(+DJCQ{*T+s;8#vSvwEBXYnVR*;oQ^UmyScb z^*l@rS6K7ryA4U=AjXC)d3Y&3Fr6z$$fYEs0yHZ%+IRs{tqLz)!KexQsX9|-UVPpX z{=Z>vONL3E*GIGhff($8toy<_RZ2vCXe%yIV3Hn?1{_`foe(`4;H1oRD3~^I3RE(X z&c~s6-Q~M+TP~*{Pi&^7Q{c?M%3K?C`czgGM5k(l4fQRv@_D^zGMJvAQm9)Z`;niEuw>M z)h5h?*@$`l8^El+AQ*ERQ?iyGv;;)$@0J-|84CC6b1a=K)tK6Oqs(|rjtq8;Em5oh zQo3CRoD)bj9g67!Vm8{clr348M%YXnZE(sOr!r;=j|yFj?W2%DZFx?mv4hmH;`e&K zaN&qB-obL|7p1gDrRTz&lgy-*{Y&fH^!u#m?e(NshAWP@X`)Gf(CGQ&sdMaxtc{<5 z0bB9)$>9KB3B>l4b!GNdJ5ydx)O4NLp^U{fmoK6`+H(B?EJdQe>3;tKm^%0^@W#16 zs6`TL>{nC}TVsFXLtprQyvzR|`0g#cPcUM(a|LZTh1{jXKulhC*8e%Rr4oNu@`2n0 z*j|IMu?lYO41}|P25#{wfW>3rS5Jc9Jk1GReJTg5rC^QGli_~Q2V@@?(1Pn^xIWdK zSLV#TJ*_mXQ@bOACp~C|QB&^G2S}BOczbl1o7q&IvEU2a#$rF2+!O3UA0@d7GJ9q$ zr>ujrX^9!jT0>p9%r+WWE`X=*ZPOu+I1&`cqac*e^ZSIfFAA2`)L&LIDo-0V z3DG1fP76r1dK~-s+EXT9I-FF$tHn;`1g_(RB#AjG&cU(-5V-*X*-2t%SwlO-V?Ipx zqsEE+sl#~;U7IiKpqn677}WvBblU3g912Y1ar3Sap2k?XrMHExBd zAhyQd#zTMp61Z#blX#~l_VxA)Pk6OA*9^|eDvYBD$tOlaB4pN1hbx805)Vg8*rFl5 zNn%3~och%>&@G*W;rua(3r|CD>5I^;Ab?#z3;i8R*jndoV*;;D(3#G&Rn*W za@pDoN+9lR*jtf1bru+gn3O;(VwE1M37E*a!T_Sx%zh^%7=g!Ln&WqU~O;LN|5 zMgY9bSC~7i+GQOP(Q`I5Khtoke%QPuq9!x)mIWP32T-X&hJwPn`gBeh3p;X|x~!3P z#c>`7v3Abc^5xW$LLM2p>Pv}6ixWPbo7&;>mI$exuaqR_AUQgC+drDznbkH9LRyGV z?#fbSD=^FNocik0{sh%ZJD09mR#uaX0fEn^;bg5~-ogl0J(ZEhR|2v0N{$A7N3TOp z_#$R@g5(DHS>R(#@}{AVz*y(0D7F%1t)&lOd$>^`-smwnp9t7{h--5!JJarVyzV~W zr)FOor+)0;!(+cQ-1EiOxMib)*cy8m2Oob6KK1@jeD{t^Pkgc4=cuXvNQ-9dV>&TQ zY%vsK!)`jgmxnuRMqQ5?6=%coVZar3NXL3$u*z-&e(_oG3r~Tc{|jj5zXZ*~lh9i| z0h5Cj1i6&lwGDnqS~ij%L^d-|?5eXer&1Nkhg?F`z*UHO7@Qye`kwWUQ69VmxPr@j zq2x085mfFT9p2@#v&_tyVouh>GrJ@cXR9Y0RyDn5t8DF@rpkO1(e3HMFuE+mODD=_Yg$(&3ypybJeysHoa(B`-K*=#~l&b5(2S(k{hrK zfVmnpW4c(A5oCroxgXf{jZf2fXSQE-tU9r3Bi~UMQ#CKg4h9HNTV;?{!`ZOpp;ho| zR$3Nv{_>gPEcvWew$-!lGpZGpm2l4B*Z{82$4nhKV2ouicko>vMqJKJWRZv^9_mGmqCA9u{UAlE2gpiVlR zo|5BwXek&K{w2J5eFiSudEjEV%8N+` za)vc%_Ifd93Z-3^IcB-^AQiT(RSTRxxd&`7vSa|3Pe52W%1K+1Y_e94!NlhC2woTA zHb3BOV`5&3b#+15F9H^^8!{UF0*BpHJ>${kQr0oBv5L@|E9@HWmC)T z27U0wTq(&|bsJJ<+c4)dEOLchz~%icU>bDz-L%CDQXMEOr-^aS^r+dxm94KD(IPX; zO5hcNZNJ}xNlJdg*9rQYn9xRl--wkG@mwM;P|#M-7%0MxwS?Iu}vB(5e#?r9g; zd%E1O!WF$AW@ZQQ>UaKs@Wk(3>}Fr1#wDYI*c#Us4?KPc-Uxpm@0;&~G7b1r5ifGn zbQif1I(YyKa5H8_f0G_wEG4npYs*>-7+X_hlRUB1LHNecKZ`Lu1+X3;<1bA?I|~lgc()`C4ki zs*7hX=&N0Wqq=69XLK1j8q+(f<22}009YtHW+0khI*yGSslWzcz`5`$2CHaSRGzbP z$#M}SovhQv`XccxaA!1X@9g|0=c-(*QLo)bZO>R6%M3RzqqP#gnaQ?d8;$92o$07e zlHqk3F5M#prXrZ_5X1%ovf^Suc3i|P(?0K*O(yMakIUFfWjI_;7^>3&S%dBH=!kQv zYb_mZWv-*L*?QBsz`b|(7f`OV7f@<{7wcum}fW9d2{@@W4jDGmyYR4-Y2*mYY7TBaE)1fv%({Dpv-9$1eA6QIe?1iHPVSJVvnR`je*V>U>}Rtme??}=8GSx?oh_LNBzu&a^-T9+up!UXP( z0q%JN)h@j<2<2%MXe+Q4aGE_|t(>eV7+sNgM)Peo`Ai)u_@H+M!x*FkfYY*mmH;b) zSN4`AAJ|?aHeKqXU=9kU`-08ODaGEf_7fx%Ke6@ZN)Zb7*2V)|{$A&uqX4zGypo=U ztZ$C7(2DCM+`&c!?GSilN$nCVuH#+80VGDBJC_br zt1KOdmceVIz*aQzf(^acpG7_Iqk(Z9Q?kx$%GQdknI*Xa>t%yE7wkYkY>UTix5a<^ zDEBD#k##wFG4Rr*w6(a=W!75W`_T>K=yE6IGML%%5u7Uku@MW2cLw;KK;a% z{IEt1iwa_ETzRm3;Gutp_qt!hzxr1-4|kC^K4CO;2$>mCV{nS-jOA-IL8+*1PL^5J zx>dHsa%(3cAb9QOp8{BT3Yt{}uNx^_27*_4P;e2 z%H|dj3y`VPY~z%l(*-E>kRF`D^k@naWov~!hL{t=5}RbytOC?vNiZPQha{|wWmNNa zTbY=4t-Uha*0`cjS;#FUmMZt88Xf(Db}ZY##7$ zoEy_nL+*&B(FHpz6JX9MQ-)@I0Z9aFwWe3CTqEy2zuJFj(y2EUA8j>Wu$-8Wgs z`H~tdv23&3O;(UDbvw6p9zE3?J0}dRvBtJ-bWK}l&aMsRbd^cY*S1uY>)2w;Gmr)L zs3q^}o*!u8gbkR44Jn>L)xThyJL@!?j->@HUnk6#({hLmP6b}dn{7-TQn1edWrEl} ziV0lS`(S$RhY8Yg|!O5L@HQV)pS@;T`ec;*IB?YnxH|(7M>ytH%+%J`V>F;EHr$ z3&_Wj9^yPKNsj>M_1SK~QCQevBq;%8w=P$#tuaMTAFKME*NRe!2@7J`WJA;Zc`)r< zW=kbeMmM?GdklgNfTo8%6taa3q_YG-0jhUn2}4oYMMy*21|V&CS@dh;(( zTIN+A#KuU=dU;)uoLr^=HVS~sU{_e-BoVNLDJ{KP>Omz-Tpqe?F0bH37&8NQpbVty zO{sIEvV%q$Q*#gyNr7YF$;(R&EP_KTSYUvY){TL$0LO`5A13hiKp$Tj^!77wHGpji zUS59NnykUVVhlyXm-)~(g{Iaz7&%eRyUdOg6Dtx3l|l3bShI72*gJ?PM<)PQ?71Y{ zUXHf_Ly{1Fs5l`HYAKc-`eSj=iS4ge#{l?h`RV zKn07L*i9rSp)n8$*{_D1Nxlz32&1m z-whqz!x|Zr(2H_pAIIDoqI-{x!er^sg?WYxeSVekM(7RyCQLo}@8PkB7p`qqHEyM- zAhyQk2gwIwB=>=Dy7rlskFAT{0!t7J^3XK^>qc*(G|W%KXIuq`iVtir0j!?|w{{xZ zi|nSE95t?1}tBAj#L1mt1lG(_mK#g&Ibt6DH*kbfds z`C6XUGTjYGDi|5jZbD1|R+y+fqzpw%jH)^|@MozinY^+RQ^A{6(aI%aor2b)7(mjy zwmdA(`#A;$y>c$J-n6O0y2?2sPWB0?DjUhp0>qlyuzw>qRulbJ8B(u~0><=O0+xVV z8YJ%Na5OWlzTt!`VqghCG7F2KgCqc^efl3lY(B>wkuVD zpnC{mtM6;l7VRftXq&hNJ<`ci=bI`D5*w?fYm;u50c@*XVgP5$irUkS8IX(*12 z&XvW8Z0@eZt2(3~l>TyRWyB1mm7;9SIttVZ%2-u-@Yy4t6EeEMNn(~2orhoT!5S5u z=##SWH;r0qNVI}Z4frh+TJ~Y9jsVO8Hid0Z55Q(|UQ z*-PJP53+wzYAsYH}L^zzASkeWm5G^dboG zGQc`&u~le|bw60yTb{Nx0IJdklQWnc!oYRvY^bJ}8DY`BdX+{ruFx(jpd^q=)?C8I zH(9p9*8zRM1B2~tW^5C{Ny&Z@AQg771hCw362RK=u@AbuV8i4}<67mkj6O@`IePtb zN6)S~1zsWpRtq)Vhe=cJ)K<>rGQW*ee_Ufp?V19o^ObE0+8H~qL~{a{W@Dvm8Rd5@ z9{@<2U9bB!18$cXVVY4_YHw+u`D`ec?uRC!VmmJ#eWJ~_;T2G6hdQQE0vFxOs8seO z-UAQBP3~l{r66`zSZct_)Nv02#Xc8hW;Qw6m==s!lwBxxzgJ!L=mJ_}ELrh3ZO zxM)-mTjSy}`^eXVAO1exSblJCXM8>bJAP9vlQ%LO(g>5F8THIcut7K#D_DpAU;&!d zlh7<31;6+-zCHu(%CpS8-WFEbu7=9W_87RScca&vHr>=&&!#0Xol||S4q5WOKXUdl z0a%v+is&ENXetU@Cw`>@3UUzvpB|3Pgms1Ct9>5xwp$Vu=#Z(weW{kD4{Yy>`3rEi zWUf5A==dBhSNDr3U-MS#9ChDc#)7?I=8%s1@3$@sE3&DY2Zyq+b9@T<+%Ds(=HSjW!u-< zS*AQ>Si|QTl#@T-ZpF6y^*E6s2;)x1kYg{-U_`RcX#adtbNSGbD486 zE<$ydR_Q9~E~)k8<{KsRwhued6015`f_D8hG%LrUCE$wfz}mBL`_@VLE5mc_HqdI; zIw$3@z)+e)Fap>FxQcF(l52WSfUZ}9gjr4nTLP5y;o+UK!j3_#0XiEZ%u)jbD?Twk zyjjX(CB2$HKl3xl|Rp0XvIcr=65b|(uQvlYqk8Mal7Hp-L{)Tr7Nb`IR zz)FG?B!Jv11IM6~fuY)HwF+z-pBPSa5s*YFP1N$@SitXl{i5QQu(Exxnd#7^^Vn>? z+NMpcrvWKpF|Bjq6X>;h8X#^|N6VcL1MQHR;#Or5zR=E^H>nkQkc22Xp@Js=(0iW} zoItCSrF1DN(;Gqoo-$Z1ZK=c=AmsDbkewwLf=vfmdIH6H3;>pXWPj}z#(WysWy61C zNK;2GcWey6mTnkMnNY8cu~^*+Q%qhXQVP1Pq_@j@xbd#H0pNnUExbb+d96IE$xt#*oosDLg>xO`B{vYp$!S}cW?m$MPh>4vH>wIsdieMx;=*KP5 zzL2rp(&AJPav}8$HO_vlC2X>(#Qy7Fl?ODuHH+XH=3@>cAI5 zjbN5zvL_i3Hv0SuT8`Dn1mYN&0*HoCnI;2f#{MN1Sd*X{pDRmQ;u-@qGy88_{azSJ zuS|ENESddQ3M*t|ee+T^J0u2atxNV;%!X<*iD%QCeIXYiXd`vdnv& zprb5mPqeP4ceM&yreh|?mVuxp=vl|Xn{RD?O-y0Ikb7mvQ@L3ya?rL3__s|7A`3Ru zK&|)5*2yZR+4hRzzOa3kBnYYm$skWO!(KIfkdmJ@(l`j5)OMT1l;t8t8W;J|x$-x})twh)yzyDduRJZQThGbK zy)T;I>vZpBOYBM)w`9INmu&^W+mdBwC^4ZnPjanZLi&CV)`J#Ob%@6JJJTOi!*sAI z@{&1x_};^es%3HuSs|2HjiizyEdNQ6(#}I!X#GS8mfMg|@6J%)GR#^UU`-6aVjK+i z6yamb?C&hA!Ujy3vj(Sv&Y~%I`a%oIBm1AWP7bEX(lXpV2J?4C!;3+bRlA!0u-A+Z zQAA(>C^o#W3@j9fx3I=|yZsI}%OwOOtaN%PFy4dcH-OSfNGS zGO;W7uRFI5b`6}MiQtyQd+q|@k0J!l&9B%Wdtr65b`SYFynsmQ3gU{LDU@j zUuSt|Ix-lT^>3QOLU?~8nv**D4Z)<8I4=>YL4RaX6Ju5B?BUJ_Df$8j=y|@ByuT|` zx~w4f(yI>$vpG5yT#s zwy6Gflm0>N(LVHcZz@Sz{AK^fTeQw$D4p-uj(2xU#*3FF-~F--x1X2c#^+?JAa=U> zyqsQqS1rVCL<-eEXi!K+@4E2ZS{cNc?k8umdf^^V2IYi4a}$| zV$ow$Ki>mvBV6w|m=%GR$tHe$?QfDc34kjY5FC92uM*X|go8BBSb=*%L}@{!1beEl z=hT6M0u7&-jDQ^2K&eDZ4FJY81OG*66URVMHF(IL+oSCezt?00L&r*z0UZJ;v5FC! zMM~7~4pEYI09%-cmbm62b7!e%gtZMshtAez0=n4u#rM;*{K*0j!4Ml*RpWY1+{!4| zIt~D&xdYue$Iz78xd*r#pttq?piCm;3OLM|;X)A%D{a5X!cL9xjO#M+s-OX6SxkU> zEhc$**RKv8vzkJ)YHb_)F=QI^i%qr~>kDsHWZoAro(lN(&~?*}E+8G+L1XirlJxtj zV@mtXe71FI`za{{#I}2(X~r8{h^86Ok#mTf!gZ*wNu3e;Po*q;mgBx6o1b#m7JEi` ziER6~IBD%&4;Y!Sva|`B*I?2;BEktX8P{O10f;l)AABO6Os`syT$ksa`B@LuTk%2&8oip*-pXxHNsA|JQ3%2N07Zqnx{vB+ASpy8uerQU$z2axzE4)WfAgr5 zJudxn1hL1(mp7n)4c6=d{a90F_TM#*Ks>1+Ejmo^4~?op$X*aW+nH}=-rkjU1*)sN zFG#-mf~4yez}|XJHuqkX_s!R2y}Kd9_NENG3$qd0)xk>HSAA%CSnbq(P?)V_7i!EB zm{@58sF?kr<^sO^lF9*w%DJ1Ap)6kNy zm>0zCX494`5FA+f3fj&Do2$O&JoUr^=|o5Y9K^1u9UK}M?vuS}g3qKMtx`odhG28W z2YqU5M`o;#LxAiN8rYN7+Sj4XY-nHIvo=G2YS$un=T4S7)hR$U!uEQNaGfa;amLo! z;(<~U0Q&&J7HZBY_nQM~8p{g%ugr40Lyoo=Nk^H9M&ijr%Ea!~M#~)pc6)$W$T7yj zofd|W8YBbqsspx31Q>J?2{2r6&(6@&t(wNchY8hy@s_1Yf(KfCIaoV2GR4u-wotSYDr##t=c_h=N1X6GQ;U( zj-nCc$rvTkAm2tK92@uS%zAt=*W>b%+f&kqH||-r<+u8B95uY^7{|Fl+rU8ueE%8-O0ZoAN=V5EuZ|v z^RItn$K}1Y96{`HsQ~tq|6%P!{;s?Z{i_4H-p^2?CwJhJy}Y6_l*Kz-2c`&zTfEvC z(QNg-_a2@t7_{}jc`wucLQeK~Wp#PYw62d_|9yGn=F_se{RNrMza%HS`o1Q1m3tL& zUYI@5e%@99TY+!c`Ll)B=Ekfr!#6t$F+D7FWs&L3K@Sv9396?%2PKrE_S#@xf^))5 z5HHS-*!NK+fZ@ypUchII_2)U$qAY* zm6awXFe{$OW!u9wP5{ufw{r7-*QHJgj4gqBp#{_c@75teEB_AgNw!#+nGfa`_EBbQ zWm;NP>ta%u4z-z#8O~2a64;QUs6cNkW`_RyJA!+%!h?ftFL*GTWMCQz%#$SphHlY3 zX@@4cNc0CLyPG?ggpA1qWdOa68ZF0C$TgVpZ~Hbvny`>4lx17Vv;P*8FzSuN*VUTG zR&*NX`{C`*084#;E^e2m99${@pHkvjd>;!cx(!y+*BlSPDmxHt$zLp#Vf)H$u~Wb- zYu`l+e2!WI81)f`z%y8nkn~XqZ2IJm+X3dgNtp;l!yfT*;o#@v-bM!Q^ROVTybGOG?^iuaT8Kf%!AKiX(nA{Cg*a3vtH42BZPQ2>_ikLn3l>lKO+ zOrw`|9TDOma4!2X(EwqI?F^Bi$1HK`2mNo#6eg5}HAf0)SXo$rfNB7^bkE&RZUKfj zryOnto&-h*0tpRkN&TeAVVIp`Fv=lm0I(h)Qy>WOri!-1bpa7Ncx+`^y?8=Fdzu3< z3w(YH8Nk_}nelFc?ClN=CPLY!DHda_GiDsRJ&kJc_Y_G)C9D9a-u(L>!}gje6s{K~ zKIu9Z_A9_ObG#3rn0yjB1P53a0G4)dnTeFVozYGQ8RI^6=MDl;R_$tWqyVSVCR}}Q zm$09Uh>MWjABDD9O(&ahOb%T4CB>S<<)oKUr5wPRpf5uP1)h~kS%<3t+E!>pL_$+* z*HvXym$lD4m!Se#t!g-E5||czP2Y zw`3-C3auhjf@MGk76*f#7fD{6l-`EIP?+`ngV|e|cOkTms^8LL1nY~J zWV-VONjIJqx&FBdWIrz_+m}rjo3!&a+n7u1ltoJ}YH3V@Mq%bQ!^$}-<#4bMTOZw@ zA+^pQ$Cvmw`qlt;d``62BIAZ&W3V%*Ue&X@{9G8m^r43yl7(5v80!+*Jl#elQvt*Z z7&;U}7ywXBrwZ1SfN{*!tae}*c2gs>_(fvCw(_^ewj~rdXy72S>unGNNWgg_;2!$0 zII2vIAqDXLvO!XE_kGtuhd=Ae^{G}tYx2-P{bW-q5=9YGOSW==y3E{Xbtjt6hXVjA zVhyy)8ezc|5`z-!2$zUFgnrjSN&=gW0tlJ%i0(c?o5?&!m<+L<#6z4B3~@!Etfp%& zaKo^clQ4NmsmT;Fods00x^vJ3%F;s824D#=N&rK)yMp`23L==yK$E$$X@#BXMd1!) z^T&iP?y~{gN&v9z8?X;T2~01AX|T`%{}Ky1Lz&|C5w^paG9keYK(TqWvORaqA19f) zv$Kn&w>BZSJCWh}R3Z}7Uso2=Ex1m-rklM)>MpJx=8W^^8vD*z4EkK|e~;xtw$a;v z;n@2f^7EM6#4bS)RhWC~`emD#n$=o?rrEW^vsM+83sKR6R|VS}8Byv4eP7%4eo&Vd zKOeSzn*dOn;WGrDPQk<|RL`M1-m-p6}*l&pRXYHb28rgoUHDCLC)rDa+1$w(qaX*gPw+c1r4g?@~Sn)>VmzLnPE5| z1m?Mg<2Ybg-Js9pJhy-Rfrx-fXgH_~UhTmoY42eTe!9GgrfQ0xdvmyo^ zd11d{2t6{!hp23oVE5I)BS31BNH}T|MrxZbczj5zCXUT*hI(ycGay5ec)n${9`vU^mAxw4p5HYsb(50VeP}YIU(rKOZHIvlX6b-lRR-&W7a-A=*H0#{bbq z$of$e&3)j#6Ya^@L%_%6zyLrBSa+U@pM@ZnQiLw)%h8SlD(Ga*kz>EJ-_TjX&u_b~ zOe(Zvf;X~x&Dzf80BQcGYeHEe3f!`|S8EH$$K92ry>7iqcb+f_B;1elR9btq9h0~o zi{ElWD+lNIwU)F5-~Ibs*C9wtk@eR|#z??BI41They!UUn0yxxY<-?-wN+w!g7e|; z)JcpzR8Pw)fwSM&kb~&ME^Sty7i2l-FrbE*?vhLov+K8+?I9TR=}|{s~Gh`Ov=D&2fW`~Sc(8P68W`CswT1+ zyTrtjB{0wdc#brGiG&I!f0X@JgaA*-deR)flNg9w->^h&!KukMDC0)m6^sleY)@tScuU5vDA$%~`iPhS| zo`{-T_}kJEVQ8gC$xy1~U_#>9#L8Q`&2fyP2P5PFULmW*WMx8vm!EG~pz+)csM3Ys zN{=gq%vqa|FbS*o;ANtn{TxyP`)p|Y-?v?s1tzoEr|1_0F1wZ(g?o>sk$XT&i9-Q9 zCy=)2_Z8$v3D?vndNXY11d#+^r)B3F?ZuJ6vL@Vwopr%OvoLBEjArRzjH)wrhN7*XPw$ksDD z6V4ve?ib3S>p~({<(B;+X|JUURT0tT3j>utz_~Yl(B$Wx^82;=?AO!s_>(_g|NWoq}5-~s2K zm9sL#^%4vqwU+^2qX`jU%x$s+@H&t(2Y~E>QDp4{kUE1-hS<*q^|PFS+3#k+!kIkh z+-$s3({{3`1_7`jL5|UhdAD!Z0mo+1 zkYH(fuW}mLc+Q@E<+o_@C8@P-zmFM4Z<9)#Q0#ICV)eTy9+CoN!fhFu@@--u0BsTs zH1ok`+#3XT%L}=963pJcO(VKMP&y3Vz6&PQQf%_wBnQ54=18I}dwkaVs(I#j{PUeA zx>k_7+f@*{tLxi3JFnNpG|l_V`mEnMuaCPG(B7@>sDjsR1+?dPMRpfbw&(U8lNhAA zxn-o4_9UKACQSI+n~cQE;pV0jQi0IYM*O63-482{QDxZr78l zw1Sm_zsVd8yyWU&z>uy2k-1NR2_}?4w^hhu$e)_s9Clm-w3jUZnjK|D`(bsZ0earh z^yVy4rZz0Ox6skNg_w*yEw2t4~}0)cSo!L>c?4OEzehHp&Vchv#9x&h1)gvsy{8+MSDTC!LDRjbqpMZ(2fJw4XUMOL(dQtFOb}5V}N8$h)a7}T` zEpQF>#GG^nJ8j`?p1;1&kzlCpu#kwSI^V*CtL#12zVzO)ESP|O!3Exz9%jp8f9=68Olg}iVaCkc!gdkP0ZB9UpQ7v=Gp&Y>O0>pZG+7j{&{aerbu7cP7wgOoF zt^rsF#rxgPe?M<)7kc4N70TG|&+S>A3z8=3cO8 zG%Zup?;f_&C96fzhOR&2wRnp`tTmCkJ3lBNc;Y+dyMFwCKNd!K?=MFX`(9rxf$Q>@ z>#hFpyk+Z5VCSTX``h42xSA~?z${oWP%0qO0MuAJyRpvxhl^KaJbyvPThGXJ?Nc&d z|Fo=bKPTTHH|2f9rJUy3y2=3qrw(}Y-ZQMVxAwts=b}T@+z1$K55;9(i zab5#g`G!lkKYTkKUW~6VZ_B$()jDgsd`0rD=ViG0w4`glC#xHuk@xMtBJa<)DEl1xYK9!k7HkImTYdbSt29Nf&>KOQC1az*uk*BzcNGpF}{}(-esxX6ad!s z#Vjnq)WY1Kt06@Itw2)0LLB{GUE}JEuj+2;Oki4Afa$T{zufexg6$1pg{gmSNpn5J z1U+VU#@LeydCw3Ki^~e8Y^*jD*O*MFzsZakFab$A5DV$XmE<5Hd#``FK0Uz_!KN6& z471RLgD4i9@XB4D?GKXDCyYAj9Nm&ifl@KHHW|Yj_d{|$rhZ4Ojp_Rrms`79S{hj8 z1mrgplEhf7f%gL&Z%GZ@(Z9LD9m3j#iG>IgCNs{xGrI*7)}TnmHOcUWw#h&aZK+*l zypH5g^lP0G!S|;><4&S>ePHVeu~{59`xs0!ip0Lrd7!bYweQ+_cwT?&Z0b%fx0iBJ zL8OAz*_iHK_m}}-F9B$0D^92k?B$k#w|;Jq3Sjq_=Apf<>)x8&V6W<|+LzYzzpE3F z%liH8-3nyyO1Y@u_55ai+^Zn=u1s~sNGA*y zZ!li_gOm6$3w7Z4Tax^tx*z|6eDH~1J|=vr9hru1xEt9lV9XoL!i~PFT`H0e~&k+(9206>J(a2(ri75q7fOQ6-^;kwgcn zrCWvTh!f+Xc4FPA78Jl&0t7L;n}BU16I*DHMKCnSAT40E9Y}ik-Pu7Vmz7q`K)S$P zl3+iL%?$9&L?(9z$dCc#VAPy>wEdRO7d1Bn{~lll{*Mg0BEhoIfS+M+vZD)TG=~<@ zDh^>gv^$Nye zf-L}KhSsAZmbA$oZ;+NvPM`%xhGBs3s%vs1OrHTZfpp!Y)n7AeV4>#HX(8j>OGq3% za$nGVajh&`0T%;qgDw&|$G&IC9s)K3m@0_XU$WG!U|kxLcP7>EQ68J#kCzox8CV;g zIgW%$wqaT7h~qeL?q*ELXep-qJ+O`Qc%`x85lP{ULY#SZC#oFBT+k+Yk2{OZ$4u+n zxaT=Z^1g%M8rwsM5`9C5%UaSHA_{W~z#0}?0ifq#Q5CEzXq{0VVyjH)zJ2C>A3$&M zzECIh%BEgk?(Dql^X{^~cd@s>wwhES$mFQLwJMS*dd)p)j zs-0Efe0d=i-C9ZSSs&Zm#^~C_0~U#g{ zA|H6-_vFb>eCl;>;kdjDmLrILFD{RN^53l;!q3ZFmcVtN#rh8gs-uOG+$IZfs*9B; zP27?TS?}*kzIRRLTQ7=K;F`YjlnghYk@e-5YRC-iiOC75XbS-0yPV)h)Ps#h@N63JOCf-piax*DbkwcEv?!5S(f!+||>V$c$v+&;Jg*F3tU!m{L31N~k$ z)f<+dOWFs`G4MzBcWUe^6Sk-^P5|#cfVDyw0J}aZ027GNIxAgKZ|OkY&u`3tI#d(w z3CE}FnL1g(LY9(Rn+rWG%IUUp!#M8bIzih9l?)6iCR@~VbKM2trG>srV}=2?ee*fm zZu4tJo2Q3w?0f1M#1vpl7fra+eHQ3Obv?!uLo4gakbC!)9UR17TwWSb(tWV9&-6%L zu$^;ko9*9kKY;eo7NmA>bls2x#}cvzpP-gT>;<@OjCPSW=}ZB%N@#rBW$$c$zih=7mTMgG=s+f{uv=+wL&^(5CS%U)FWqAFe?5w~s*fy|)}e?0ao_ z{He#q9SBx$>vk|W1F1I92SzEIck4ZK9@y+}%KF}yrQCQ<%JpYty#5(k-TaK4sCCwG zA&*Y8oJ=ks&`vLB1+oLe@@l%iMJDaG%lh6qBVpw;cOJ&tV;^)k<`!lOFe1|F;aZRB z8RSa|L1c&Px(0?{0cJ52worC23!vKofaTDN39u3r3nYX?Vj_r?7H&p3JW8?V&eBVy zF|w2oz{AxUTQaA#uyQwn38wy&hOn6=q{j*0$XrTndxQ>m!ZQ=I1~8^+ND&6N{fh6T z1sLM-@^?cA$XSfR_MKV#cJk*gF^#%MIy!)TlV*JtRkUy(`jWP$Ax=4x(1xMkR|V>6 zagJGHHrUl%;+(VD`*YL4=1iLj0xu*lr2@t;gH3JW{siz`5+_}J4-=TdWiK;;bv^{g zSO_AyA7tQjWc^G0W{$)mI2!=mGPK_*zR~_2$l7E{i%kq8j?I%MGaMOaVOw|2v`Pnz zEx$9VJ98D-Tl?`+w9Z}{+X^i#S6m|D zt2q*=`+!dB_q5dlXcqI+#<@$Kf3`g^HOIuWKxa?0L|!S&XQ-b~qej*)tbd~S6?q8C1xg#ePYHwDmiLAh?&gQfBH7X^TXD|X9R=cnnKNGr4KL#dN%OK8ktAbNH z@M^m!WCr~&Nes;%O#4e4_Lv3FP>V%@y>aUQ1pl*2%dfJg$T^b2WI}$G$FwTasmR_;@UI=!yLQ=)D6KH z4zPKmy_1!`1-q)(PS-z8{BBo7VWf_B3}!g&vxXc1l}xQethpuPhy$M%nk`M28tct* zUK#fFzq|7XCfdsJlA)%w0DwPZ(!p?gxJ4uuB`5)EQqQh-_XPngGtbF@wgj&JyApP4 zw$9vGTMM=FS`aW2FpDgHvnV)`8KO@Mrk3IP8CuU^KAGrb8Rfn|JnOhIoN1oi&0`nG z2HL!%S@v=L`#eu=T}?Vr+K*aemBnrZhs<#La$~ZHH0%gKVH3!5)LKbv4+#Nn zv_#0nHfI8LA*iR^*i?JWjq&E*Bp}(D+YrZ>E=BT4l*49`ZD}CZyrs?8S)EFhJ=gNH z_sm~*zPu}gKDOt6{ppviHez|SvfhG8Mz4uAb$enWn7sM|_J8sB>tA$7J}i$tej!hN z;vc_}6&#m$-f{%7?~!GE{Ga@{rKF#@0?yv-g3jYGgWsc)nti-8gZ}|ZTk89&wYbMR zLstMRw_gys_N>UupO$p(X<6TTUQV|!%jtL_Q{KyFt))fRwbOIkrfFE2zp84oOh!;H zEd;jLjy%nv4m{jdRJXNv+pT$D8fDNuIJS9KxHtE5 zz(!I~HosG5^wX@L;e7>_5eiQUtQo`jEM$-{XwZLDLjOt?neHd+A4Ss9Fa*PxQH^Q< z3#(;A2Z9Y9vcTS$EJNW#>`MT^;xFr51McAQCV_dv#ZG-rWGc7i7;0JL6DlLdfJ1zUuObUNe!q6(o7)- zvZ)bl@))5o7C3y5`Ci8!wkvi3o?G`5@A2|TNCSdhe{wzj-BN{lc)S z=VevDwl1=%pKofvd{(xyR`-Oos=wVJ%HTN@CBvma+BwJ6RH9t@A;7=i^45%Px{@D| z4?Mn+CqMD4Z^a6a%hz@}g4lQcQoD(deCXr#-u!DN_1*i!FB#e21%t``7SE(lO{33& zT$(rRIH+grJsEa4WV-m03^$*bp#s;e@`2mW$?3(H<&knzPKPZjOO3U2Tv>*8xE;)H zp|v*+?ADXE${cmrXA8NxWsEbms4(Ru!h-N%QiZcW9`bF5<`}@qf)cnUXJGBT0HBNn zm|&1HGP$hCfOH94$kq?0FG`~Jex=lN3d*huGTpnv0J#J>HT$e067}J!-7KgAhOj!5 zuUOV~8U6{(E;pRXA|a?zE}vs3mv%TmYU*>u(|_Y{+%-ObAqg52yi|G z$Q{9y+9Yc9iAMO6>T7Uo z5L&1qsdH|cX74LHJ2Jo9C4%e)iB%%OHqtf2zSR+N9U0p$I~e9p%8m(a3azyF2N{I| zmy9bH2aQ8S2bLP=9#mudspDi{1&6h%gd=!s%Qi9ei5{anH1QHx>7^0vKP>v%vtDPn zyqkQ&I(Zvktp)>|t99*PHx+=^zt?r5d2+Ir^~q_SbZ+Wre(&o0>-zenK5r@_UF&4D z{+;W+t@rq*K2Fy4H|tIP?MeOJMmF|5$=T_NJX+UxRsrs5y*EwRiq{yoj6wHSlckfTyxyhJ=l_~~;IDj}Jo!_9U*xa9v2`7n zcf@i8vG3;PCw^^|@4WFp)!X|=OUj`(dOeE+*nI=B6c*CuTmMHrb>UzWU9x+(dMB@S zwl8Je-;&|(7b2qWBK>wiy7OPwsS&goNS|kLnL{bRu=2N@ViO|4=t>jJs%HXYSa2L{6~1c>GU)sxmT zlGF^C;=L6e2+XCx%jb@{bQ1vX6AD2DYVyRpDH6tf;*l`PXzj?a4c#rMziv#w2T|#q zVYXTjRx60HL`8tv6qA}}P1d_ZH0!jG;sUU;g3}&Z;4P$W_EOowCYi|w?!e#*D~7oa zQ<5|F!DbqeO9(APi>&tsKni6o1^&tUyxZ;~^xt&nbCU{S->ur-9>qflT7AA4U^MQd zLN=SS2y|#zr<1~Ds+~C&YL~H4(qW#s&@8joGLz)|djp8uVj6tm_10$ZzIp&Ch! zGEjPQvayE(*427#09NH8`q0nU`h9(0BVr0($CX7_tN_TWf?EZ?CuatLPtH#3dQK|n zJ(bh-BlU46C;FYbE|rG8Z@QF6$4glct_Lr9lDR%oUU}Qqb3O0uzSiTL*$9Lnm5l-( zg+RLcSuU82m9Akd!fk9!_Z#P0JBzQS@+al7Fa8ViO@HC<%4dJ}K@#cX@(y2)Aokt5 z{IyT7<=Xi3^-lbuH_E_r(O;S^R*VJ(ow{!^O*zZ!7!y z+4^`^#v9Mc>h6oOy0{^$bXh?_?My83qr$>Gh4m8?gt$S^qNQz?4}?0>oB6j|!$3aZ zV>b51{e!@0EH!c^1}+TeY56b)jii-yo~dBA%z6q*E>>S2;n*l6%B=oi$Oa1&H*IJO z`#FcON98gR+8Pirv(zPTOc35ahzrHeGeAK`iP;2D1$I)}apeF%wC+bTn1L-nJ_7I2 zSk}pXQ~9iu3;@wFfO)5TO#~3dkJCop5>>Op#%gS`1ZQUxlB`$|!7F$9)QGZ)!Z}Er zcmQ0Kn23O#NzDh>&l?$Xu(7lmBcsVlhlS*ume!Avvb^8R2T4IpmSkZGp-d#V3VuD2*Kj89YhKD_pK3)24z!q8qy^n?mgHi0{?OMs zmdy@OCuLCA(6PIQ-HVG16;Q4WcsdY0sgDg9)C~ZrGpZY7TPwgkt)R7jN1snC=sZ#I zsX+GhL^c(yt~dO?va0^ReWu^roPcRv+3T~@N9F91_to{B`S zs_WS3{?*6nT24=-FPQ->QG?gCtPXE#NW8L#?A1{Y&g zocC8>4!;rCCYQ(L{nH2J&-_=vDF5$I9;$2W^ zguq@O%_Om9zZZ#&VP#7*IFNa7r3k0pZ5i%;MTQ%n6?yq-Nv}LD)2$ceWP43kyE~S( zI*!RgR8nXATE)vV5r> z;Xw{65iSf4Hnoun1oYsP0g@rWyd*ED*)*EX5nvdcfhD`y_Gj)$gQlD#qn8rNGzWwN zU7(Z$&{F_Th66@9cTxy|k_=YI?X(bBZjwO*aqb^WJ1$SsB4C|Z3}9RZ5DehYoQz-^ z8_0Bd#c1r%03b`?l`vUK!OHpH1J4cEu@-{HHR?k71y!1cGz3Uzq8G#O=E7jtBDoe2 z8bHbjGQ#mb2^eINTqo3Of*}Wk>%hHvyHd}{HoW9U=k%s~p2P|T#^RfgZ27Xg4SCetOwjncp<_TTm9)DL^YkxMpa z2i*$@%5&E&_xk4PGLNzVu|*Dp+6QF90di^#pu~!)+18!b@cl5kOvfY!RuIFZ8Op#a zcvX<9;8TICvZM+~RTiLN)Ig~LO$VY1IQ2L7eUlKZjk)zC%*iD|CMnp2#J~Wuva*}0 zzIRdq>8U-qT?NPM$u;o)yR|XCx;_Qf(<6__%0TW)9^DM`zIB$9ah8?3BUoRaWX`J! zV%N20xDU7#XZk7AN*&8eTf_|M4d0*J?EdPBzRz;y8>xRz|Ds%+|Eu!YfBLuOsh`@u z=E{!CJ8C(C*mv9VqrdeHvQGc7-h6-Nty!Vo#e=lFT04e)GyJ!AoMo}jf6!tC`is4c z`@1r2Z_04@OCr}kYt93zf!*BxyqsLVEbHAZ8Ow!Pu6Tu_$$;$0ic!JB3PO8xBx&tT zMo>bZSHQ?vX++j+mj-~yYLHd(PfHvjzsDGK0yESzt~M5$|EXXUZ5mXM@-s;!HFhj} zM)!zNb`Blp5I~inm0@7_)f8iO=@aUp$t4fQRdm3Dkh|AD`aYvrf^hH^FN51Nmx^t8liU`Jrt<}s%c**V`xqHnh5|qmjW_N3CvPLnNlQiWk?&0g)8&& zF*jfziCZBMKnOaw<01AjsR2Ds$@;>}luif(JZ@bRj0c%Vf$kQd*b{nmJ2PA<=I*+% zouIAdCJ$KLH_}*D>dTX#lmPoepbo&C!9)|7Mrhxz+X$w&F~tsa{e<+z6b_Ta0-y#E z9s{5Y0A1)Yf&uL&Rrou+QxmnbV6|bT<%x7D*nfiZ8Rt*$TP=5L_yp&nP2c*v7ztBM z?!x_MiEpX@W{Q_f`BaKqoU=MkW|C(hdfSuqVQ@9cd~)l6z69?a6LF=TMt2O zue%LUE-XYLKDmbsXpIC(1(9RD|LlHLV0q%K>B;GtoK=A6pz_qdKh^$Jzi+H8!0Lv8 zQ`yz}-IcMg0I>0JDMGNi1iL5r{fPqDGk{lr5F9IjHo&YP*FInS=XHCdeLhw2I?6Ym zjPlK=709lVoUI0vTs>>mYO;_Ik zBX!*S*DH|yUyeZb-L)J+?7M0C%TNAu^=|$@>#KkHty`(3_K4rjDDR1`+`y}KG)ShY zU{oCiR%s`z-CZjkJKcFv()9{pD|j92^Xl#ka<+R#HuD{GDac4bN(33oHmJTZ0A~r7 zO2h14ezBO}p;xqNgUJlrtyDUscF0n8;IEMJz`*DlUb3$MvxUHc2J#FwcHnm3v7H#q z8;ETls2M<(VeOQ=%=(4KP8`5V0RWtZru!+iFk>)W*pIQ3FO106AFxl>t=Eg>#WU)^I;XY^k6k2u zWnCkS_FzblpqND-a)2;T3Ip?+JB=Wm5NN+;&J6H}4Y>f4=`u>Mn)cLyY>CMf!_(9% z2UE^#$99(Znc$ev_Vj&Pbp=qA25Llt_07>%8pBN{)?E|EV3`vucpZ$$^Sm6vDRU9w z1j1Rp1azGSzu%XfB6cy@{uah>4jl~`Hs=MK4v>cxGNxoKEm$(M`j+2BrQ3E$EZQv* zvL0)pkO!IV(_A7>bTB;?PNHlZL)&MvzKqED+{v$14`QR+2WKTv*whLPP4~KX?JC()-k0n3CKy-!e4Ly~_3y4sj-XmtCT!_%RSM*ZV05;~ zaes_R&@^2wG`1_SsP>1AlMZS$krq@2l^}roMkR4f6hVeQXAKq&}Zn zg4oKAMP*@Hz`?B%G3Fp|L9bU!FvjGMK(@sj4sL-%g!+Ib$s!7MA0x_dHz51Lzxuc3 zlYis<;Z}NFzSheT#J*b=1+aTjZ@WMC)~?u~gftLaJS44_JEm1a$Wu+<%cZR6a~Za` zMDD#L(~ajUaD7_lmp?7}#`AJ={-T`DH|0cB|5{8kkks2HInY^6S=PgB)=?RN#RoR8 z_Cx7_4isAd+mowP71h&zb(}~XxLf3kQDCn(oyn4S-0rQNv_J(ky+pIvLO8%|<{xgW2gBdJCTnFqrK@`(WQY9WcF}$i{kdkpy5a$mkCcDMeW1!LCzw z7l#3k5Y|p9m9e((0Namu^fr%cXfP2px=lVYffRsn$^=qqm~o#rcgL2Um?1LKT5 zfVi%GB$KifK(38tU2WFhChE0Upg3CizJ8`G?CDv^18mYZ!Me`mI}T>20JrNVrGd4U zP3-VJ|E@pmZ)I&YQEPo-n%HTzaTgNn6IKh&E`jc}@@?pQo5A~R0YD!T#2#Lz`~U3Q>xKV8dF)4jL7w^_ z@4UVhAD6Gias;vOhUFu_`G;$7rwr_Oz2)nU*iFZ6)ZTSAPt{B(Y}dZ-@5*L(Lx%I0 zCEfa>4A&}nedRMEn!t7bWjUE|oBP0O@GNsnKN>vGUI&-WR3C#^n<^N@uoGZ8JQ2jp zFzzDk$I_QG!DVhBg3z+DLK4T)Gc6vX%niUIOdEPc!7QBx6XwPeO~ImqMGMQR5$qwe z$2r(1WY#x!FC}U#+aPB4SlQ(iBN)*L(U&F%h>&@9(3+D2Ku@}2)hP!<8uqtAXQB)M zx>nUh0(6Z46m$nLHdZs0HAzI-9NK41O_MneWMO;oON$`54UnA%R_O~E#4*GBl1UoO zLJ!SuaaJ>u?mFgggbyx_|HFfpP6Oi@noWLA=7{=UKj&OGSZo|qYyXP?cQ9wI7(tV* zklF9SVbfH?HXh&cTdT3j3IwHNmxF~3*~kz;hRKi7?&H4jJAmhDgaud3-?XiS9@LL9 z!xNSu%Yil~7!fT?gxuw&Gf6Cy+2|2K9yi^6K`2mpmG2U zCL8yZxjFI_H=uwxT8kn1kK(Og=n(bkT}F zdNDMRO8ux>_L-NNy^7HaffBFby-{H#<#J;8Vz766S(z3GmpVgxpynv2xb&6kj4}dWHVvN6jUXJ ztdanO>(ZsMw|Cu3v#`cg>gQ96EO7^h zEo;1m@0-q%#{BjW2fM4)XqH7}V0C9L(Wt616`|z zB262b!D(P{7Waj9-Ro%2#&AwxI%B`V)u)aj9azjEJ5ZkJQ43%k7FwwT_|CRvw1<&^ znCB!rR~6d?C?=D`zlV{s0JnM?VEg6zOgCL>EN`4Rn(&#WTSp zEs;7V*1=;B6bM*5c?<5Z{vl}d&Op-ENkSuQ0GpB<>?`eoh)`4fw=4v}^ph+u_wQ~-uYTp<=KT_YlEo9SRb;<(>B*S_DMG?}x#Yy~rL$6DWDjn{h99rVX&JQk5P`?4 zz(R}mJIQ@VC&&rFJQ{FH&5DU&k6_E>7zn{zh#u^cdCZP246fl@^L_<8YzbHZpKjU@ zD=dAjU~}>;d^0$oG-P+;#^md@Y*x_K9WpKeD!Lv`;5u1BKZo$US56xmz!Yh6g3M$v z9LF@ELYV}h5lo0^!DW()?D7!-6KltaL3d0864V?}+BHfJd{IYMs^h$<)va5SZT8ClABWK(NaK zQm0FWg?62kMtIF>^pZkg4lQJVgOc@f&GJT=Vq*Atc?@1F!92GgqH3s_heIU%j)7K z8E$=E^7UsWU3*6I%@<_4`;x48Hx0n%Iv}YAb_I5Ag8s^9^Q<)F? zAePoz(7&45y0EelI%&?piw{()Q8PbEWpVbb`hzM~p(zDmO%i7N49i=$JbU2)D`&g+ z#4&0%4PXuM6Ghl`Wl0SaDnL0fZ{=cJXxNjV$hxwCfP+V4HD@>4H{h62kRW@vJi^@| z2RQc0!G_E#>~SmsYvsn~8)PZlK)70=Fu~9 zytn~%xqf_UpcDYv?8RLZQewe^o{?>=GS8=9T23BHeBZ7iizGxc8Q589Kil&VEHEV) z-tMTW#Q={&0tBHggx@DG5g0-1klLUh;LT}vVd566J5i$I+Dx?nM8M5NFA0X$S~CGG z9+@dpWRTH@oT2tMn=v8q_e<%7_S@12#4h9I|V195{l20S-F8nOrMcd(S98HW2o>g7cE zu1}G~t4qjLb~(1qViD+Pyq6>-cMzQ#kXpzI4y#Vx2gF8B7e1p+n(5jteDBTyqvaB$ zkcjAaDBUo4 zHzRQCT%_p4w*GB}8nR1v{5l$CZtKv=_qvW9tF*QCXs@f)zS90e%5LvTw?mFJu+Nou6x;}@!v1i=1lK((*V)uFdm^KraD z3Lq$V3)`$t_nL0ZlEW95NZUl?Xj>35Xbjb?}6V;c2w#?@6)Uy-?5F{p#R4 z$pilE&-LNo36r2`<0Fc{Bt#=rTPn@9BB24OtUxWUe9{8n?Bp^W-rIT)>ABu+x90f5 z&nOpQ$>iQ^nF+6bwK+6a3#*jwn%VQvIbi5xqzi)gZy6JI>Tr{XI?P^dRWY*KZ=Ig1 zj6(IayZYU_uiHx-4-D`oR9`$exoEjD`G`;6QV?tZenVa756NRs{L&G~zVntNh<&Fm z3Sh;OxW479vb|J7r=~-O)=-(srHmE8ZniH;zWzBGZagjHjb~+Z?~8J}y(XLet@`<0 zQO7@)3C!q#GyEfs8IzWYF3{rb5V`l_0R*gq0d7VH@D7>5)Wyy%Nei>`Vv=1jvQ`~v z4}j~P*oZ-a$tlKs7H3R*ZDE3hpnsj`-Z`LYQ5~Enuoz@GJt?VGoob9#Mi`xXO(th4 zXl4?bZp&pB(i{kMKe5gUVS8mrRVJ`mZLETiItYXXRscQK$RdLnpokr{lZQD5MPqA~ z{V{LYAvCAsKs{uaSe7Ih<-XdLGo7%zV)&hGZ+6{mF!5yXATdbLVmtU{GMF>3YaGu+ zMtJnWaaneCXpCvc3NjSlDuMAFXdB#zKDmU>)d$=dfT^_%K+8}n^qbm_!F)Np5AC=R zba9YwvH+JaG#rpp(?>T=FxgUqwb?VaXQ3c8fREn+&`kqKG`EGO%eA5)(u!?(ARUef z1Si-x>X?o79WtCT`Xop{RtGJRij3`g3hyVOETKyh0|>VJxK1|tJ&Rh{HV1L3YVnfN zDmj?_HU|(Vc=M=rq!>whIQN~&lW+fZ zfY{u9nMJ}GZJ9=ALsQ6KoYl921Y--?Ga{gUp}PodcyciDDqhChnApMFdq@^?Bz}dY zDm8H2A`RJW*rGZf*>-jUx-KU4fvp|s8|*hSCiMa*nl@1_7%P;uHKty*v1^gjjY^?P z1++;vqdU^JAb@&>wte-b``RN0WWVq6Uy)CK;>Cyi z^>O)XEJqOgPFguVu{Z~f|Rv|u3ffofDe()ONYwa~i#MUm@I%j(wW$w8h?OiF`3zHz2_kebGM$>6kAkB(hg;vq)u7gRfy}%yz(V1M-1Jw36vo7X^ zHqBUu)OF?n98BV3h|b*;xU&n!S|n*J=!X?f~#4 z2^s9vlOg0hVuBGmM-FB=cTNioYo`F|1i{YCb^r{vPazEm;1AXsz&Lc9A^|wUC4k^K zqytU=%Hk23%vqdEKA`cn*JB%q3960sX{>8zXjH&}ang~Y8g8Ps6JT3FrS zApvoKgR!|afbmKlSXi)GsmU4oThoCQnNx(bOA#96hlC(EcL;YP5Q*X(F@YLFbK9H_ z1c{O*K!$H?sRd5bfFk7ty>GBz8;?AgFG=Z&ci4Tr_d+Eq5u&6+Tyl^`XBu4_zxN60 zj07r5Fl^u2q)fr9v$kMrli$<(MKWju-t0i){O+5tZ(0CTGP2YjcPuVGo!Wh(ah(wflA)qs^!+OV5?-S4U5fW1AAcUb+s-M|$}HK`h^suH7x~ zoe>u(gY5=z%q<~|-m`AqHAdtnLDEn^ajB2>EsRLaKJnGOHAA>NJ2~U9my=;Dr>ntA z+78p1lwl?3wYl6=<^ysA_xtt4<0LGL(yB~8=u*7V10rW33K$2ea&!+P&4 zA~!xK<=Qh9ylVBWFUyGn*m6gP-FfZBw@6@uj!r>J1+J!d71wJS1S&r8Da*Q>y*{i} zywQRM(pXqZ2h1zBxaVc7q@zn4?7gxUI@AOcbk)@Env}q^t5bx=62SoM4CZ-`^+S?o zEG~$tkOq)_^@NJ1#k4YzQwvuUw31( zVLyk~750fEt?hB0WHo!aKt?8e>O!q#F9o67UTfQjy$f#Dc8{`MGh*Pb4fsq1t_E;Q z5VE`7`g2Cn4$3+hBF;7;E0}bVsqQ5Xp@$s+z=r|%BhCgknBYEP>p|j>U|&P)3h-+c zmMGQH%?!L>91yuDb@uyMqcH%$mQ^3@CvK?H&cs~egrFp>Z-W|2^M6jf)!=Btv)Etb z#2fPJLME2bbLK>XZRH_BEPqcnylt<6TCSOiRGg@==mD#`={=&?mwiH?w8eX#I$7cR zL}GpYBjLjLAU5G%7fwo}U0DNZ35bwhE~eiCJ_Y^@Y&rkvZY42Y{ZCy zA}iU-Sli0UYGY0ex=;5mwsQC0d7YeOtvLg=wO-fJ&s&jbZ+yc&!4C(~{`%?{hKB;W z_xZ}B^(_8*`M^hiL7w{Q>#x0rgK2AeY=Ym zEsM_H9 zgYBiI7K4`A+zyanGdqy>>WL2IQ~b`gwiy;uW^;og1o~`Dtkp>$0Q#J5VYM7uO?mXW zFO6BkF~P16lAz3vYnYJX4Cz6Z22{b&b3iTDe#8ecJQr>&2D~AzQ$nH$LA4i|aG>oO z&A68OJfPZEp6vAm_Sqc;n)hxChckPgZJSJ#AFq4RQs-S%oFX)t&vETjB*}37#v-vz z3Bu}j_yDhVA`w#0v@!DbsoC~qsWOxKVvz5^fdL2T4fBBU7uwJ8E+t5oPSzj+a#)Ji z!n*>HK#&pr#}MxoJon{SS)q-Uy#x~sGLK#^wPmOi&@h~A6D(r_dCYys<)9}*{@7>F zl_FzETc^pHHEl<#Yo`}0!6UR7WX5TSXUS>M)SDKbNu7I<2;>wp&8Agw4{G!~j@`P8 zEp77BPcq0p0}4y;kAPmPn;-$8#^kQ8hcGp^B`Nww(+rDG@c9)zR_jX#cZe|0U=*96 zy_ptA2se&kRi%MhtVw^P@*zvCO#}MV5w00SwB>~RloL}jxqb|EDxkxG+pkSsfsy1V z+wX>{&jS_pkSIHtuCTYUyIF@|EOL zSDIILkLZzTlC4`Z>Os}Ekan&8U=m~FUc$I%7fS)~S|~B%{UzaA(H<#Y^TL=rMFq0? zL-N?ieomhH_-#2ZU(@9XV!zhQ6Tk5X>Sg%#dS`q`36c8iUlL2k7WllzBl(RQh!06j z7&NU4*ZzjAFJG4F_VZT2V5R=Ax1W>K%df~u?M$Y!H2~{Tc>k?-IvSBHHkbmuBp32i zo8{mKIJs_%30%s!`qD`ULSC>e?YJOJOk*`hCvCso`#$NqS?S3j04ic7Fg0Xe&G}%S zBdq6OoA``j^9xh{39Cc6meQpo`3EKQ#`xse?7kuCWXI zvC`Cpfx@csbO}qTnbD67mtqX;2jMX{hJ~eZ!%`>FJ_ihTfY=nz&badYx53#@T~E&~ z8OC6fM5Kky0UF_BIx@s^L?T(0m+l2Y7Hm@o;f_$I&aH`B%@cV6&-_FCEc2AiaUt)-=%or1abOOy^Y zxlD@tEY9h{b51SMG49_;=4-N;(7GowzJbF1{jnu-SwgS)WLGD#HX#(3F=-@2r}e6% zo#awZ^G+UF6*=3C=DKozxvP)8+)*1YDna00Bal8gcxG>ob^^V+XYg{V6DKRRu7H%p z55elerHNNOuDJ5wSdaM!E0Fz#Bar>tE=LgiwOl^(JKt3Ax(Zc@}CCVZ>ur=ndel}KcsuF?I?aPvHeO~sjJSXMGb0W9CAjA1}D_*d&$n@UB z_A^;Nbd)w$)yYr`gWPd@iMh zx?3hWvZjF+!iN%JH-VnYlG@O*TY}Wkj>$vrlZ>kdY*UtC+)Q!+w$tbA1lXz#+ykh$ zeH(HcC{`VtYL#|SGhJEIX>#rDv_e@}FosdVDibtymShcM%0gC=yU=}asGpYSmxlF+ zU^dRDw81Wxy!>t)CkFIjP{ClvoAJUM(VUC^hJx!|JKKXCRbuAaSVO zO)kUe0crnx-+W_X6V3AhJYD`UxA3w+*9!3JcyDfrTpgGv8}120*jA=wzmIoX_fzaU;7iL)-fSqL{T1NJc ztS?@YlY7rgy8c;_n=ig17EUd35 zXAVRqI?z7B%f8ix_x$e;81Fn$#H)Pm!K4a62hL{hI!%BqfmTogKoslrrjB);5FVpJ zK3{uy4`%rENz4v*8i4abJOLmx^?)GHr3lo$I@_{>Ef&i~Zus|V0{c4#(4u~npp=6{ zoJSmQuq_>1!Vvrr``xp8%qZ`&+*6;Z)&tb5Ahv<|au zV061SKror0+sy%1dM5VZ)`)#$f)zvH4Zt!vDTXM5dY&i9M|yU+fz-xaF4!y%_L2|I zb-!!v9L4EefaZ2?h1=;s@#3BdfTv$=NjrzvIue}#c$PpKlgO09wSmmFZ-P-2ZiiaO zcAee3dGGxM0CZs|(g7Rm2a@~QTAZTH(6*KqTDQIn;1$0yX}grzhI&Nrjq(-WyCCBT zuCh&hskv=7Tg!$0O|K{I&oiY2S9)l6JXmLZp1MA)-_F4r{9fTcNF-0XfAkOq5OjiT z)%DCrwoUS)?>(4c^%FrRhUM=5qo%sZ)xOpeLotikecYDSd?8a^&p3ED=Yvc^VXI={O79#!locH*G=BuV^n&e~r`}k8!60?tjkB^~ zMLbNZz;(CZT44m!xM^{MMAk6}OBA#Fgi@SI!i~Tl%2vP&)yB-C4wr1}0_b+3@T)A} z!Tw5PUxxz_+mZ%z4818d6Q#W-c9_0crh%6wh}Fp-OSsZg*T0YZcy)rWf580Q**ibFUHCn3eqW{?&4{`p|N;C-qs2 zlSKLDaTP8VDKpV8$9;du#P$H6QEi46uf9&J*^d#&{sOAss4q8HTea#C;wEcf>U z}gX+q%&1_9TUptBQ{!-T4>$1A|isUJ?e6MGQbh#h<%J1?ym>^?y zcg>1t@66DUo4=y#EZe~2z^r)np#&Zgopw}_B0I>kv|Nt?%@vZ323Qy+p>5NEvX=wE zG84QE14xq_MEHCf1F(9z*&G_!RadQt$r8PvV}L(t%yKGyae)T#+(`izT%Fdo-&X=7 zx5dis@6=@^3jit(?m+-9w52^$DXBZp%VW4NqZ{qx)j=%xIogIuXsbDpHlU3`odp?! zo#HXJ*Map?+ZNenKVN0>{^C0wbA0$&uNOW1jBF#X2d{G0kZ1F+P%=F}L;Ie=mVn3T=rjAO8s08GaE1e2}e{;vjI z^Fus(rLuAe4nY<<6brnR6?>qFfH zS8>J?-6F70sRN^AP{k+Mi}QJ(_Ra66?mMY#Zyx#JYrh7Y3SPBe*qofmNqv}8K}y!= zuGUjXiekb(G-BZp!20jTHUVB-|L$JyKx}~596P%Fjr46@qAmAjYg1~8p!}U6f5H<= zJptA1ToN9lpuR%X`BKJmA*=e;{Cep)uJYQ_HU{;{Hk^ao83)W-T9&nH$Nw-J~Wx@k?oh{ zWPekJ?H$ScyON7)VN(&4RK^w#N zIl#Fks%0+&Dw5@n!N3~hNnJbp%tC+!jV!I8 zJB7s_#S?fA$gTCjihrkg9f(ZghyY^ewEWFg(Ev|xn;^R8I$ix{BHPRmy`OL1pC_jq z)3B~L8#z5Y_42ObAXfGAfcZkJKVvXE5F6Jhk>PJqocm42TH8(w9W49qBx&)xzWpzV zgKj%9S-@iVWLE^nB0ehrs;`w@^*R^Y*UnaoR{hIIBUMs8<~rUekcV8ZD2UzF-|dR* z<|O;N&3y&275$o>m#~k-{dCoZ;31ayDm&2SODx}=uYchi%9Ee?2M@Q>64j$hFT%zW%I?cfTO( z%dg031+c4f&vmr(PBOc4AGNJW!36>fbCNTrloI&yi$sMZ z3}dZpbT&)3q7#d0g!Xk>d7%Qh9E>@`qQ;ZsRKqo(NW?UBjA%-obAZI4k~{W=9cd;D zlHG!9q6RiK=K-GkT-q4DWHEw$dnD!XdQq1<v6rtGYE@d$xkur)9ePd0FpXk@b92HgZwH);87=;W*uu z!NZdU0~QVCDJ2-XEPjTq_&C(WE{--WS=jzNoY7}=w6u%iGaV23-CkokkYRy$tXo^5 zPb==n>fe=+5tuij8SPJAk~I?5atCM!u+P~G7-S!;ZF_Du3-%R&Hi?1K)!+=PRohyv z>wvU!##PT-%Hj1oHp5c8f2Rn2CM!fP5wY#+FF|aKRi8tmm#k#uf zpkp6|VQX~FoY!OtCZ;`_zLf)BuSg__?UTa@pkkm;hwHrhSR_$NWB+KCRakrjgI`M+ z!?SpTV_P51-X#XK`vK?AsvW|5OII(`X7?qhKQ)p>BDv6&Y(A4`(oe7A_x0V83gcKVu&A-gka}*4mI1n zX%$=LzS1gj`AY2dOdGK|bw?pBF$`NS07q?qT}t7YvN65U%ww_^monCFQK{6}*I;|Q zYZBK~4%!33pj=&2pr8_8$k=xy3u=F{K3yYH;}S3Jue3kWO}Gc{>{eWyTuPf5F|oD> zvFua(w#xq0+9IxD!34qY{~Y^&2x9%-=~EVEv9#V0uvj>@8W{eL>1g zpRSMJm+5u|u$M2%RNr6M&({ONm!lLP8?+-C&45}GOLL}RB%3wTcm=?6F&&zP{dd%^+98+~5~9A{rw6~Jy#F4pA)wmxjnBpFQLLh_ZDY3#jH$@CT^ zvoV1yMWR?ZD&!FfX)qxdcCg*~E;AUX)CTslOd@Inu{2Ah0}w@~u8@fm2}61g>|~;t zXhcv;jdQ=;59IalAAV;Du;w5?V+ROcGZVCe(N0YskojFdH!SqPvlZJf^1%D@wf4VN z@4CO_ps`!rH|Sx9+?{*>yoIa_YS1@8+7i?_NyrP=2nVnSK>6VO7Df9&`6yw)?n;oL2RF$WM6JKK#B;eZpSCzSZ-^q7;)& zw8sE>%EBs$)$i?getxHr5G*Hfbcs168hGvdwi~Y_kqhmd=k1`akI>Ctnb1i>F^zUk zdG4<0a#3k3Ehc>8Ns-B;fkGDoi>S@A%DXyt6?xy;T23}ziucyNbGdz?jI54ZlV5f1 zllQH_Makx$VQtUc3WE~n=&O8LsCDuDfz#Iv?ojh%`=pmU<;?`ah=$Uw`1Z&_)QwZe`O-A6!TVvQiF(Hfk*Z_mZ|KwZ?C)XZX{ks0$K^+ z%mNX47#HpJQ2ydi%YmxTz!zEW%i;dt7p%%vU1I#cJElWvzh zyx?{5eb!rkZ+(HYx-%xLW}bv?7IqETC)a4U^LRfAlkb0rnOC2xbJKSZBmLhg|gxOB&|Pxp^-n3G#D?=vcQ}~wVAWaTl78a!*{z)=nQ3Qgvw=+gTRXl; z;VHzxzycx5mso^Z0LM-&ATR(wp$gf0WnmPD4DmN&700T>yG zQd!D2fRbJ*GO$Sg;(j_)zVurK!H&+lFa7d=SP&1B1RjQVK?!*Ute*vjK{MED#C? z-vdqT{q{?jKAdasm8*au+WmdNQh!bwqKd{9G78?Dz4Z~n3B%|8a&^BOj~Jz&alZ)J zU2e2c9o>QcZ~(!1f`x2UbmG}9&hit=#YCH@ndPSe^Jawjqi|n`6iw?&z*@R8u*E+k z3v0VWpomME8Yc2~dH`bM6vzDHDu}f%NZ2p6Lni|#xK~BG$sB(_drjP}=OXDd0` z)Z?#X;O^qm9^Ne{lhHR?Lp^L$*j9a`<>u%Nk|8w%#U;0IP z>Stbg@DGm5n^}$^_U&4JQBgV?XQNaV7^@G_W zu6Ka{YXHB&3;#OXgGpEB_wtoHkjIdvJFfO=Ylm4DAhrjoulDe@+$RJXNa+E`oxAmL zQgaBDhe7he_J+9bS6SRE;5xN?twlaqY@h?$+Aoy;o@AO<(^p6G6-cU{yLOP$zT*>J zFILdBlDMy>gteA`=HP(^Ry@B8>q!Zd4`0Vz3NqX6u5{dsU6O8c=o77k#LuhkulKP& zoOHV&s_UCXmf()R5ebGl1=8&RqqOp~eT!q?;UJCI1t2ds26kGn%x>%K>wtjDGt<}M?PpAe-SKtq) zu=fM8hd1hLkJI?YX?^vB^2h)3-;#g&@w>12qvP_rmm`RMJC+~)wM`mNf2FqK?}@?6 zmWKhc54VXcw-w%){(tt~E!MU)I}aNFKj&KeaxPWfCW#US6okQ{Q|*`}NOo=zTO_g~ zhzLSLLI^uuZreP8cz`$ig%=(`iXcdKcN@nLBVhzWC~<@ZNJz*J?F)351Cg+>$xTQ*qZMWJbwaQfW{HO{F;ASuw z09GZ(tgDoQJp}ujw;A0X;JKw`dqE3eT<{(%(@6m>x!ZtZ{~>rIt=dAAz*FeV67X=NU#xL%SPA+de!*CjkBB}D~xkmrL% z90$#7Srd#M90Cl3K`hY#@>FLhfZrN$-Nsf9_N+xM7-PmPlYsemin6u&JCyY}BBhWT zY~Wm!y)%W3Eyo)7Jql`CwJj_5BMOV6Ple2bfr!tl1!t9$@HdQ=bdPLcM7dRiQn`Oq zKo-iy=Ies#MEO>2N@Sgo3R`4~-ZKHf-!NYbW1Q4|2C_3`LrfnJvLZ6jtYe9#K3mkj z0_|deGQcXIH(5SdWlM>WN-K|7OKoySmE49&c~!wC>Q!c=UR%0`yh_+&*;w=0?f|xz z@sYwS%?UCf%k#ArE99UuQ;hhkMN2qK0jJwy5G%V6ZFV;%2hgRTuYAbv7`tN?1nC6< zQPLQZ85(oiwefc5wO+E{HOYu%;6Eohk9iKaDR(#VInC{<7JR9OHG8!(h~)q-uD`{M zE9)qB7dVtrQ%E*TF-A914B^2U=hQtDF#P0ZPtPU}_$nIg%x-e?C|L^B6)EeXZv(`7 z=8D^o^zmmGLH-x%d;Z4Xq2K!QT``1D__zbHpY-w5t(Fho{iS6g{-8;6x^*Gs!-Sbw zoJ3Y8b}%1RgOo-0?1qK}&e*PFuy9yRzGrlA{~ldD`zCGQU%>0*-=?$ce?;5E+jLeJ z06R?T!zw8;Osri>M4@FL&{CnCG`q?Bqp@;u8Ib(4&dgenm!!O^v0YZ- z{BOBtIWqy;WLwq(I4gXINAZXCYFq+h!NS^osn~b58diW&V;`U##6ND3>q-!vFg4pg z_{?7=%BWTy8||anm1DK+0*WzfXQ1$*jH@X$8+ZOW+fRTwi2CN5H$VUZ07&4Q2Eh5{ z?0m_j;EFcbFCG5VK~NntX3s-jT6lJ)skl)Aw*Fq+6-EJBn!{#{A)ty97{px6IDOrY zQw-M;sD&|*z8_P)bdo{AEPRp4gXN%L^Ydr9;p1TapL3ix3dM{-vbb&`t+1!x!Q1Fx z^?6TyUJcXHOp{^KDb<{Onvo?B(V1x=2A|wuanT(K)#Wv6F$=L8A8)lTPHm%5dqx6vH(M zK>5V(KqAiNzu7Un_PhIdyOEZS)IMFG%3TUQRCS)VMTWAw+hnY&KJ%gztNO`qm%s0y zEq!ywnP4{bRh#?JDcIL9DL+cLP7cP+R~ zg)FW)ct!%`&fShtvUYkCW!_i52c-QLAFxDT?|8An)|FN?rDZK9R^stb$m1`o=h30 ztO-Rg3h?JHEl{Y|uGx$x-9nzi<^wV*aIqM~An^5QTb8mlnG|R<@CSye`Stq1tIWg( zTNN%H0FX7vYg-2i^it`30*krmkTKq#&dGtQ_JJ4{*owzu^sFR{!cO23qv@=sgO=*= zd6wf6o7}uxQOMHk)xocY{*wX*7*$x5qz^hH>jMCm7OwnOZbXJ}nx+6|z*G>GY zc?YS~y~ql}E`p7>QXMLN*8o=LNo(EiNT=ObD!$WeIt~9>dgq%j)7S2bA$*d@9f9p)Pk#gr2^HZB4)xP{v2T{ecsac%0CxNAbsC<1i#FG< z(`NS$ZKkI*ELNxL#X&)A!8Uny1*uy@`qE60&6!ke3Bl(OK>?Bu%2})MF=GV;@FA^D zOjY`?|4Qv!l3kR0C}uv<0@; zSO@`>Nde{UO&AYgC>&7ncZQmzDlks4oGqzed{3Ed4X4oWg`(AYXCT&}w+C>}R@yi_ zfGxcuhg(5F1M+Kjps<38tp4qu!8GvZ?TnxxA21vS@CNK4P_Xu$KWS3TQB)#c13>T! zykdUi`LaK9%>Ck^wovI9Jb*!Rz*j7&vaEosT?fxZ>3x8U{I|Kkbd{Ox{&9Zz69f1~ z-=Pnn#0~h=a8sZfjH9~0R3os$04SJqzQmI&QETRlTrth`Y`7edx(y(BOhDM7DHFTA z_RPlms5dtb<{2Q<+Ud`sGAH(Mj2VS3!2ePZUP~YknHwCymVW%)?=wb;V)kMs1%xdQ z=wMhD_b>HX?pynMl{%;_*>2A?z1H&c{9+q(&|g(#DnO(6yan5?-Mg3|IGAe(h=FxZ zXG(t`(TEAiMwjQdz@Wvti2LWZ1K8^;>yPTL1~rj72r|@+8sjo&idOpKxP{FkN0-;- zgL~4i>>qo5v!lIaE-1$24rauuya55A#*Mj3Ipyx7-92iLR9D?{^7r=r^u0g!27T>I zUtM?O-SP2`I}rOx9k2ezzqI`S3k2!Ep3xuaD4H3-0*OjH*vo>Hk}%;#HqHmy%r}zn z_5S`%x_|Rc8n54^di5r4caP|7eoEs4UUWm3O-L5LK*JFNAtY=Tlq<+gz66fK%z{*VShaP#-eZvj+^urdD#c5BWmD znDU*aTGmE`ucUZ``A(`o>>z&&gC|1Fs@)Xwz#%dw9R6$kJ!W79#Cp(H(M$vzV`FXG zA8iRYg_4AR_@cT`YRNx~q=-=v)&g*W$xKQFbE*Sd2^b&e_RWBR>=lZJygy^iL;0?P z%Q~Q%e-;OjdM7<{wKoLQW$+rn(XLi)4jBU{Z7$yWyrksZ9$=FLZynPfh-ZdO04$y{ zF|fI9fuzNmPd%5Q2x6SZ^IoQ%Yq<<+tTg?414O&0dKVd|(Ym%h?+p3$A7{lHA(QB#UmSPK5u?RUk13cz2hkdSti7nwPjImm? zv02U>XBlEP=I~zKm(tI@O(g5zIjT;5B%f~YZ+)72ZN$MS(Z#a{~R(2L{ z_A_1WCfc!2>~Ow(cWd`h;L0V+3kVq7JaG3{ZZ}^Wr{}?4|Ni`s{tCVF>aWt*zx3ND z9{le3Hjg_H`$-(H{@V90v-FpjnSY)n;K#TSuMUp99Molib_)ny79teTVCk=zIa|Q( z+5C*Q(_=}c_2A|kbouOerIhUE>TTNY-(SG(m4MhHfUBxgdl|@6=NJ&8RX~qow3)ix zk2L~-SQv~ZNCN=r;aj6C5DTlnq-TZFS|9!$^Rb;BOou>m)(Q0BVnNhY(I+bo78Q^ds4Qm3fux=WX@`q7;-gXCG?E zT=o5;{Wuw80zpsir#ch*zjR}>WtxWavqdx?XmTcxf7k3kj-a~xI0LGbJbBtH`02Srb#=BD6UtfJ)%f-!p%)JWG~TW? z`764cM@ioW&shw!0s_tU9$sV_LQ;jj*8Gu+86yr-8$fGhM~DC~E{fOau;0t>3@7&& zxDl3to-?cQeVAKvh|&IH0Ig}I!|e%A<#VQ_KEZWc^D)E9I0a%G=8Ubs3}O{)m%Lw~ zENs=F!ymP-kjb2lkQ2j@Wq4J>BBh?gGpsX|IFL>Ay<2OLPYIyT zD_N3%$5g=_$}T|U&)oxF9LIs2*gZ)mc`4L&md%SIE;!`Sx0hX6RntJIC3Q$k+cVBI>EUKye zwFSuj-37?LL3hV@blid1@AP=}|NNuNs`_7+nfV9$4?Zr4)vdK6;8i_<;AB3trO8W< zGZ3^Zx}4vq`-gYwa{m_HyZ${od$NGo>$hkS0Q+qD{*C$0lK2#rq5==wm1CW)zIiG| zNR&iISlldV2P<|^Qy~B-d3Pn4;cJHAE+T+gkANkxd=@ZdZOqJor{$Ta6fCB|kX(p{ z0+ONy8wIQSf?`tFfx+A)3?~R%yV+_$pS4i{I&WG6#cNlIz}`J$pXVOtljaXrSXS&anfWPkM)=j+d@~Tkea4{faG1p( zomp5p*ZNX39%!EI{b4eHQr)=$gDNbYWk#4BfvPP~&8$X z&{zI9B^Q$Be#G)t_Ae!@Rh6OW7WJ18ANIl4T;5G#+0_L`%I{HlAg8igGrT@FCxB%b zdZ~RSK0AJ$2E!7R1);y1BS!}6slW5Nogt>>z4_N^0ilyHr5dcd<+umltz7izX8CMj zSF=UVv%AUl?m$=9JXdsH6c~fj#^pSAb!o+S{J70?A>U~z2obmq#Ae`1Cy%q`S^i~u z(O&x;ibsc-jJ9XK`$=lE`^8KckCj z0kHeG>E7-Qy10IwF0Q^wo9j1db9hYUumITUW)aG>3AD|!_%_PKRz%-2xU;1BqbA;m zzi1X>?0Y6UWB@}sM0rz7K}9W6X*ElGl`|A%00mG301Fp`%*bY+SO6@cF6*i7pjHT^dvhkdQum+TWHQ&}+En&F|d<1#H-9W$+UqcvJ zbssp}ZlnSi`Uc=ts&fs+nG(6yPy*N*2!-GSLpv4&Fc#$>RUN>Lo8qwuefKM_y;8we z`_W9xW88vSBg=>Drs7P%aegL}HSQvlY#mxSW&-&#X+hUQ?^VS+HBh90{f9 z*R`!aMACOkbBIg`wW3N_P20o)#@1j9eNyegdL^oBlRrgKiS;>jLUA|vFjr$yliNkQ za40a|zj^=Aoq;Yt8uOa3wGk-GS`9T3>jE&Yhs3chhQ2 zL0tmL3uIz2*G9w}>0W*8k#)eUD2^KIp!2)N+CbaqU|VrK&m|z-%WQ`st^4k2ZvnB{ zkXi%M%m{-V!29dP(lHTe|n~ zK5aHix?DZmNp-Jg{;y36AgEgc8Cg_%??sZ_U!OJ-CF?cgWWgj z-p#k@oLvZR-lXB?5w-oZ<+N)?pTpKpj%07wTD25oV*M z=Bm{hfLeN{0!0bzK~gYQ4`NmE*g1Bt`Xj}4g4K;QUCzW(N&BA`+-7Vm4}~(WlVo2I z;+fqC%%uPzi8B+NZMOnG#ht+12Q>GsbBZ3d&xiQFZiWuXZ_OoPJECkX0AI24vSYW^ zH<%FtAKVBc4<#Vu1Gf(dbjZ&VX}$ECDSBVzd=E<20JZ~tc0c%O6muf>)LtdBQ8)v) zQi8Mjj;s|Vt^O|Qz2aaUeO{f31<<^0{os<}OI=cM8+m}nk&mrs)$keE2X^I`jel-G zf||etWkxhVZ+8v+KIFdZpR4&Z7*McOR#GB!=(@Y_*8~=FU;BH_z%K<*jIro7Z%gY5 zKCS>t07cZI`7luR>%L*{#YKt@RE)FU&kY1TcVKe;UW>kG7K~&L4hF`QiQVrH8rYQp zDcDth22Q~>AxMdkSwc62nSDj{y(3Wh7?eTrxs23)I2*1X0IX{^HXrUvgIz2luN38} z%>YJmH2_2~*SKi0e9_qbkY=aOGV#`F-rc z=#e`CKBKsfUh8;m@!=KQur_U7<+A;Zwp?JlbofT+jAUdnm#P8=dN2d%+XJ!BK?DEz z(%3&v|Ju*~(p~DUk9XXG*pGW?Dy<*&zmK%d#{;odQUqxdq9!f`Qm=$qj1CKcy_g@< zgTvc&?|K2S*WXxP-=eb{bs^YYe?ar@T7mFP5+PrtD%WC<^6$b{4I&hoh23(5zU7Ps zw?av&WP#=ZD*_0HPSD_h-z&2v@T<&)59}nuW*Qg`-y+90p83H7z)GoEVMa)kOU=oW z_m@@F8Z@VM5#ZVkRFeF@m1iOGQ$1Vy<9rePU|qzl3=kmch5sU zYw)+Sjbf}I8hho(>At)a21Y2_WMB;02e0@RtK8ds4VzhcdgTE1)wa9kbG}#F-sN+= z1Zs(@PXe}VvPo76V`|AkaIob0I0anSYq9vf?LYIe&G~SpcyJkQ}Pnq_&*KUW|YKQj{>AXevrrj~+#s_mv=M~h+& z+3B`BW{hc58Hb$Q>Qp4amhHuD3_|6L7UDdZW=Op&j9mrN9?HOYooBR}lWq^UEo~+EM>L@H%ugb4)mwvfi z|>aPXNSn4m!<7VA<<@+hnmI z>4wgkf!)1D_ih#dd-Z$bN^rh=hsOP5qW!Z)fZynLz6p(Sb0)s9!oUt;xei6^O*0#- zfMpZV0DenHjSB?iLW-yZo?fj>0K!>S{VXZq}` zi6Em^x@j(evm{n@HPRWd0N_x1Z>Y#Z5CAOC0n-JP0l344A@$da@$sm00{*u4HMk)p zfQ#S}0LVN#d_7C?R9bqb+<(3-bo)8ChbV!URo}D(Oz24Adv_=B-h;~wC{7&=R=~oo z`{~qyw?!OujCkDJdcgX3GWHYx?4B{U2V(X@rL@!IxOIgFF@9pYR^3(>_a%{6`mn+c@iiDQj%Y#bXr* zEAlrYIqY!7kwEOkF9xxRg>{A?3GA5yz`6^8vlPvOHD&|2B;Rp&0t-Y^flOX;OTjt0 z5)tOL07RmoLCPjj^*f_69>R7M8wcrzU`UbYHM<4?T#hyF5tX-e7K{d@OGVn71nOMX z8hEV;oT5&VfSHTHs_N%@ut6dB0bL9GW58NFup-l~hR^1aBJ%v14mo0)XD~qM`(R*e zIR@4O?z1nZ&$$*C1q4aJR~3sI9n$Vf4nAG!moOIc?|QEQs;J(jt_E;FNUs*aO}4tx ze~G;X2*J$(SRAb5`O?$n{sSODg{usdyKe4)!ZDCt1BPse>h*Y@4G!|-Ga~p?t1&lp zbEo%9##l|tl>2T-^7Qm(D^#nN{>nib*0{+GWhGbhDYXl`0RwOGQmWo1GDF+fj5TF? z_0T3er#gTYCbn6`zi#ZPYe5>9LS&_E?c5TKc?PRpKO6d+J+&J-4@iWWmYC@4I$qnPr@LSIOsZawo`ZRasfE;+0Uf ztCKGssO{%UbkwTfGEgpXr6Kp%Mw^@Yz*dU)^_hx4TcpV!$$(KG6Oyo#L6@MFn#_< z{~P++SKj~dv+oWZcOdrLIllPSOPcH7Uw->1k3RbZK&%KXS!~VNi#=_pYr2@;qX+x9 z=;8ib;zDqF^-Xah*e>9;ET3~ltphU+9O+PWi-)QP9Z6kzRB<9x41~Zj6S?fN00s=~ z=jzBMCD+dk#RL$0*lR@{{=JKW1;a_$f;_#pL&4#vDdpKftR#PgPaZ{(l?ZhWiV|R3 zjakw=(nuEUgX&rEg{@#S=Kz9WO`%BmddY1-Sn8wp0D1JTt^OaRQZbF(&({qHcaV_rp%kjYj6SoDF* zi6QNb$n^%FQAuhfLw-muIM8kQod{0X+Ll2B2mRVSr<-K7*^X7_2J5Sv-L+)CP(Nzv zDUwpPW{jyZpNxr*0Af#mc;Z;jIA8Ae7upZ}#Lv;M{mlHKU${HuxC61@&hgLv#Lq7O z|M1DrJ^>J$LE+FQI-?u9Y>y?4)`R`)^zizdbh*6FuHT@u!(*!RP749dYO#vXsQ6l{ zFk>H1lofSF2Boe}K3*~r_suqQ2WFtqT^RR48>q?6c^nUWb1OJecGgi4e4S()zfbwn zWhY1p&jFGU6eew~aE~bF?^+TtZRQ1`j1UL1sSl|YDA;IaE)j~RG29&>F?fUdnoX-b2u_7(3m2Nva)+ukbdhm>ON=^M(()XOT4D*p+dAsif+$ts+{l0L=8N@_4VnW#Z(9Owd5ql%n#r$+eh@$cV3+2329?DD`JkWz zrKbz-SZ$t8viq}_U7%UY%ChKCMztuIotyn0AU4LA?7DjIx%BqD=i;^J*Z;NA>+A#o zv6(C=I@o#*ccFkBO-5cGOS>$BAafi5|0iXpYZV5TGp+O>c4NCjI{xar8UPMs-htR5 zKxpsR+RF;xm6OJ$wIK4Kb?45hv@g~xt>&%#vwM@Tg_>^GY*(b$VKDkzMOA_-3_Kc# z6oeK^>p_#cMwwvn?-*upmyer7JN;1S1?wavTGAM;l%GJ zJ~~Ax9M2p2BV|AT!S=N;{ZG$5`R?%J4#a-cekOc2@p#pi%=kG+tK-a zMfavhbh!XnuExbdT<14$(02b=vJ`Ob=ffh653Q;X?X2l>HUh3JlHL|<&y-a|K?9jL zygh;`H1j9Y_T&HuumGiF*D@0;wau580$3|h&;cq725ANQM}&H@pdqj=k**6{oqBU& zWyJMiHsxymU4xdF)qo+rj#N4`=9xjNx(*DMMoax$rD9hhuhPY?2a*prF{{DS1#?%8 zA##8^q@XQ#EvO0j4qd)k_=19k!}b@_H)-u~+d40}XG}As<$|k2v*4Cf^R5f5vm*nb zaQCoRV$u4;*pZTes`Py@(j71?7A%#Rlui-PXi`^8MOBs??xw)cO7SGs}ibGGxs#bN37w4khhQ zq;{#HO%~Xt*$4{3ms-b~=e-m_;E416P7P{jcva#^r9W{g(7z9ct9@ijVn_=rd)fC( zKW+t7D3;1C-~}9FHLwa_ZA$XgnMjObHG(0#e2}^qfra*AGJwh;_Wb{91 z_M;)wPZI_1g1or1Wc*GJ|5! zn=dWL|Dbx80MtSKuK3*TBJ?+-ZOI9k)h#woo;@N0ma6AqWdC8VkTZ;$TULC}3JNR? znr#4E!$m={R|0~ofRB&031GN{zp!LOuw1mN0HIidR8r2C?gEYXDHs zI|BR#(+6N5Af#t+262xe%78fgzv_;Ff1>Xd_#6)!z_wyYaZm3H2eIZ#lR)e`-zDa+ z&&$Ms=Erx#3|r#x@hnlf4t|8%reJ^~69;wo!wDFl0IrnUl$c%F#_gH&+kk98P5>(F zvcl`}jkc?JLaH=FsVh(_25TY9?~0B1pewbj8s#BHW>f`W@PCCLxV-~EOUS0wA1Em+ zp^oMw4+9WY{3JIl2a)bMF5qChKq^layej*uKD0Wf)5<_ACZEMsz`=Q}HL~3VFvkF5 zIRkR;)p!|;N|CuE7dyZV9Fup;jojBBZb`hObbhu~;5w+h+HOXdAtNdY z8i;M?n-#`R%*8c`wZZUbn=o#s3urYLb6TyVxntXDLt2MjfLYbG?0)Mle(&5FI)KgF zk7kavMP|zwtE|*rE?iq(CxoFdlPKkLP8~K+vc}k_czb?97nhfEt*h%D-R#&ucBBJo zT#35UOgi)}vR~wJUnAGsR^IatJNGZs=U)Bq>9sGv{cU{pg*@&+?1vp+{OU`KsrUDm z-~Mp-zO$M3>O_~M*jGWr?2@c|K5wJ1rrS8f0#?Um5e;J96-z~PBgG8v&F|Aohj-}Z zn>QB#`z<=V(mb$b;^H2YEJhaUby|d&3ARnur18u&a_T3(i-0Zh>9LA5iRA+M6vUIR zI*q(-u8<%#orw$zMwH6I*jFhD*`?G91q?9ZBwuNwc*rVW$*r3v6QdU8PSv+9+9)h> z(e0x~6v12OR!~{Vac~x2Yld$z_l*WXa5^lhJw8@f8d7JRt42v=tiIskCzgCskUI`% z%XK!gRVUXOv^?-=nGuAc6JRPu3+>uAT}p8Pqyc72d+M0X8f#0-zH|zrg9#x6{b|sE zHB6xtA14KvfC{2K2KWFQ;{$DI%8_h4kIYEXbT*bk$F|}!Nx4?sglDXlnV~7`mYKNq zQ5$f$f>)IO%>X%X^JiADcpqmUy(ZDfI**YGjJ#lg0ut*w_!1A=U(ML?W@!Pyvca$; zGeXo?B?0!TtdY8|tcW-_`x73j$Zb3J=)t;qVKG0Tltef>O9E`MBfXzWb&;*u)$<~*EHD*Xzwplr2q$Us; zSIq@T`n8#?G*DBF1^`QF)wp;*iJTkEFe-YjgRro(l%n134qDY~E)fJaBU3?90BdV$ zz0)w8>~mKNig_UOxN5XJyC^Ut%-5ML(XFqG_z^JQ<^uW*3}#IOIJiL;J6o}9p+YcC z-9>hwk|1d=>RK&hitg$Rqx$^wYh!?RF#QmWtW6>d0$b-c?u_+gSvdRy(KmBk1*Yuk|e@%DCt;ZdR{jlRt{rG>k%&tFs z^!~~U>kqXcMQk>Uj};cfvX@r{Vv+%tt7Gx8w49MG~1&cYPgVF*1%uXX1O<7nosY*2< z+wZ1iRu>=}ZYAz#%Ak>VSH#fPYym#Zq;4|0D`S$#`vIm2wr*pNDCpO5Mby1kQ6iN= zvJ*6Wu>(n`WvRC`6M<$P;5JI2S6Awmynpj}dJza~mpmFZ2fN5Lu-fG4!-oyj@qC`8f-?9bA#;|A!C6Xg>cvMpsyes`11*_3slU= zN$bd1-SrNMw=u6((qOKd)~N@AwJ;irTYi}V2NSFFvL;`7zrNgK#DEZ-uc4x%jI)AR zZ4V>FswpkmAX&~iG-YnL+kr06w{-8~Lef$`yE)L~r@IB@@tFuGlO26`$Z-c^pLe|a>;K_0m45Q%XCDe;Nf(WP zxk)3XH@ahMP3Au>Xx1gc=uTagOi0z({x)5=vkmkbuCG)wNc?-Ei%1m>Me&~ z4FOf!Z+TuTqfGhKz zysbh`dSmdC04FbJ9Z>k#915)xfhZ4h> z>ln`Kdpd<3;2=%%xIe4fmgacAZ#{l~azNL+dH8U;n}q@K6)-1^2?NI3-_PgD&>IrI z6q>s4#9F~~nbtHA8|+|!Wxsz0<9_spOhW>C>E1A`RGv^5VV^oYUTEJuCorgU+kuT6 z9oo{CH2n-_3hbyoB7>|w8_Z1^Saz#oHg?C^QCJ4?E)MAq!$HzuRn6eilxf$}t}!3I zkI2AjOJGVzSk~(d7bglp0uTctF7L~!jEW>=kaYk-`Dd05F30>v(H2ZSSwBPZVgm)T zNb*^4oy-^v#JaI&L8{Eq)^S*k9prDb^@xH7)m@OP`Ai?roj4qr!{=8AJX}{OuQBa= z&pcscgkD5D#H6WlKdgy)K3>&G%LcxdfVC|I}WKHx({ITbgbdQ{c`{bDSg`}$daUBFT0HQ| z^TPsk77g-Z!C&zYTjIuIgUJ{L%r5V3YGPqqq}(d87Vgx1SkDPiR5B8sa!Ftf z%8jz2l#H(-fj@sPq7J~SMhaz=<_*+9D*$w%EVLQDZ=Nd;j8Vs!qHlD&-GQlHbM|ym zcKrlWJ#Z!W8MlkwfvSQFj}vh)QzaGL$z2RzU-JnNfxaFCocXxnde)ad5Ad=bBnR~Y zQu$c71^2?)xL$4xMy2JTB)~1B!%X7}^ z`X?@KlcX%Wytt4ZxMw#v^!VvDJ>4&Ju0^a1*=~aFc$mS6f-u>#Ni*iCNWv{fK<=C6 z_#t}b)&G3~vj5+$C%?eQ9f&=3XhDPjdHL=B$>`1KFmzPsc4)16}KY5p0QO zfdQ$z_F2rJWKR2Tr-Tr)5`cltIX~9`I4j_pQk5#DCn!pd#`i@~NmmtPR!u;G#jM2( z9q>z;54dsI)|>fqNtLe6AW^(1S}>Tf+gMJ;p$wi?U^Ih3&X=FtNF+X1zrXvLnsPyq z1~+d#Mgo}SuGfv0!BkY$SK33+SrOXEpS#KyC%SDbtQ%%nrh%f z+MLTxX4dc)1$5ig&+KTY@N=61gh=CoFvU4RCKo1~`KhTs&=KERVUV@c0qK*Wp zgT9yR1JpzDXdii{OK*8Ig+8h>AF_|Co;8)P+X|CUI7oi(b%y2 zabj)xb(>j1Rh4RH=iRq75i^YfTpB=?nKDoq1*n1PkwXJcg-Aj_}aVJ84~KWjje*6`D7%wSVjKsAw` zL!|N7ou&+FLP$t<|Y3+yU^FQ_r^z|=2{qS$TSjQcR zT_0ck>SmddT#@U$Pk!zT=s6%Z9gT+GY8Ux!zC<&Bx}%Hslr9gC=)vx7`t1GB<1Q6L^JB9@^I{sp7Y=0if8t)z6hFO1UpR(NVgD^+8^g5;)}jlDA3q z9v%p{0&~7jY%;eJ;B*cC`>1dDmzz&&U5%BG>G94;T8Z65=@z(j4rvOwY?MUhC9ll3 zs*Y5Ao+V&p?`4o3Amq4IwA!O6+DhURn`Moo#uoWDlQ(h#%NRecy3~x#oh-|NfLKxx zyW@-m65O?bj2%~9+R5u?w^KI*ORFViNGSqE3-HqFz{)KNe0XEij>7;*Y5n9LqItd8 zJu*0UXoIz2>jeO|1zPt$$AvloYaPy+Xw=Zk75^+yn8@dV7#vnSM$&R!o_lNnw;AMz z8-LOsfaPqQv@mj4yEVDZos7WsZ2tKAJX%XwFG0fi4oH?`J_?k+lVu|GU~HEN6)V5`g(7{yG2rVN#+cfMQD-5g!pcR z{cno#^gdgEtkA{%<-31|{@hRgox4H^{c#6k*T?t$_6ha}I(M+C7IjEE>H z{A^KNKrFIp0kGE+#P#z2EqdwZjRnYllg<|~JM14*ot`eQYq9#2=`qE_!55aWf3(Xn z;0m85T}+u7m6jSfu#LO;Ke0MQp(tJGkt*gwIq0Nn`mIR=H~?bxd#3?xT)U}1k?+4A zgJo$`+D4OW`kqsOK4D^Ebz54M(SsaRux~7C7XWsH+XKx`aT|Emd4KFje99<*9Dy%v?n8h{ zD#!lB>vQ5~4@9e1Yg+jgtFQ>y5`T^a9$hAw`$kdKUkqIN?#wEM9#Qi?&@^1LgR*8p zHBP6tVA~{s(&l&fT+ZChZoC0}vZ1nXv$+-QH9LWT*qs1b&U|vg>Vyl5HNxsZUx=GnnO}TGAoRW#)N&xIQfnES?)` zyBnH!2aU2N+alzJIhY$kB}>q6tV;?oissvv-)tzb6t}8pxBkht^Fz3jOXMtofd%hI6cwTyxL z!Y(JBZ?^(sxjo`5R*Z$UbNF{=CN>+*2PiY;ph#H;T**s_RDSfK(lmFLz@2ghMG7uvLmFO>>6;2z1<062oYp ziJ_s&b{LTEx{aykS)YB;k3e#Mb??>H-}tkv1;46ZN~ z5ql7?4$KY#W)aX_tEqM$Q`^=o0%=tyu=^h8UN5?Hb8&GlmC+tQxuVC!k%N&67IUrVr ziT2ZJ3%d86b%vHG?m<}(7{g&$9MPo$gh&ex6h*(bWiv^pUP-OBRrVD*VwJ5mCW=&% zjLG634FD*hVY)cjoqSk3`53*dgJ$5P0Jxf>0bLRxFh*x4lb)3$b50gh9p zA@N?Gr(M8RRxALT2A1j&0Xe6(!H?LLTaNI(4dt)~#T9e7peTf~$3uWQAMph0T58`B z$TlFGx6ZpXTtxvp*qO}E!i69+Q796yzK1jJsV~4pIh&l=tW(co4-tR6ffAq_z{L~m zr{mPPtmlI7*sS08^M~%9&9_d7+;xIV%-bXbWmV1q16dWvJjK+Osb16|bx{+^X7P^w zZWI=gfsN<~m07V3;aO3rNOl4a-xCW{UHcLHLq&5P>!q4p<#GpS#N+OLrhPj_>=i|8n{N`@8qL8UO8oSU{L+ zivkXlGjaWo?ZJFtIgsn{7QM88lO8M}_G16uB8Z1qMF`KFfVqseomn;6^r$Q> z|2ayA0^2qO16#vl58t(ZTXIHl2;OiE+x(}nEavmt3C@JU*d<^-Ov2VSW{aoUSlifc zlHUt_PVsx(3rv6pTBKl$5~+4n$5~%bQIz301^l?N1Kb9N-bdF?R1icK zC}kiC+xulKo$G;qMc0D6?3@64C}!c$nYQzxEFFl}xQ>iX4W-$Ve<|`a$JV_rl*~Bc znb4+&dt00e_37Fe+ z&55NO>{?CEqx5|&UJGKis47+MdFRC;E@qT>du!xDIDU@ZoBa{@R_I`FRl07Bkv&X% zGrsXHbL+C;E@@oa=}B2@3BHZqPHRs9l+w#VjBRRS2DA8G*m_z4ServC_nOP*Zmhj! z&HDt4-C`68&8xOcg==qQeGOKUc66}HyL%Hy;v0Z-5Nm#-0_w0!X7exH2guzL^0OCf zTnB7xn(HuX&^KnxVjlKD6viD~fwWUg4xZ|||1<#BSM;B_rW7Ki8myj;qndkw*3q<0 zOY2Jwlsns0y5P(-XXjMTAJEgM&*ixC60#{MfJm8_WMc_1ts54G>$>=Mk4ONgmj3yIKJ3W4eEMX92Gq z#Pz16)nY%`y8mDi#ybsO5IhLQ9Vv%=E2kQhWx+=x$cCP^%TdgHD6=_xG_mK?Ox8aH z*P2W`KWfQYc~-8@x-kXwb!xoUE-FWJ z9^k7az)huFApyWtnW4hjaT~~unHgj6;0~ z^?SNKjy=K`0pbg_I zKYK~f?YWO#Kza8?7o{Lbe-7#mYxj-ye(y7OtPRyLf(lhfI4~pZc}g+)19u?!_1X?+ z)q)6=?#XTimsQjB(f!Hs8^$#x->+deVUEF%k38nr#5ZlaRRG^~)T$OeUh`pxs>B$0M z-=&B9x9H^q2XZX{cE14Fo5wWmZ)mrlbs-hq`gE&tsP4l!7Sxp=6}*|{nYSs|Tgd@# z<`9-aQ*=iLr0(OI0fT*o!6z#aW^MC#^qf#~NBz?xRgi4?&6Be$y3i9~=9dvUYK&J+ zwPj!sA_2fvJ&zS&N=mJ*3TSaBSbl;9_FS-*N=t+qGZ%xp4sf4?m8}D|;s8FBiB+s% z02*nRz{48=sHyL=;=&k7+cPUV^sHP4edx(x#Q<2e-wZO@wpR9UwEV#pc_9a9Ac|eQ zK=4ECu0T-+$EOZgc3_AykUaqup+ssqeLw0L*yumF&4gQkvW?9ESc8i~Vh(i>bc@|7 zmIP9xZKfm2verQ8I-;(M}*Z76cKjtwT(F|PfbH4qET zYg4x2UV$JZ$r4rVz4#nEy0M^EgS2!1B*`94g?p>Un zFS5`;?DE2)Zc`Q$U$f#NU|D5SrFGp5InS7@27164zxOR&&N`<{9D&T}6uv70F;Qsh&8$uh}0K-SzZp zPuIIyw7j~jKzD_#(EzY8OFjV*%f}DV_x#A;rho8rPd>usKDEb-4Pt+K(e{sD|D|PW zymIR~CjqQ{K(~QdJ}_aqR&%8T)%iwp!JZu+(}Vp2Vwr*Ee6WXiXxP0^huyVB*hciE zM%yV1R^NfkwTO>PqSfmpx)oLCINhJszFB0_0n83`C%Dmb;Q1mX+309p2FHQ!Hu?!iFp z#<%sjeD5eA7QR%YWPIS1S{p(+)MDGk4qnYK*0<3WLVFua>bqR{K<5ZDEJXhgrZh4j z1ZXL%IakW++NHSS%j1@-V67QE!j)I!bPxlAV1ip*>l+mW5pt>ZM zsRNXDPVL9I?`s>mBhYCE08#4O6#yvyU;2D~jd&J-uyytjv?F#;V$SPl09UgxfBrq} zx|k#x9E`Wl{e_S0wxGeN0G+dB9XQNt^o!dQr>mdThyUPy`dk2Emh%p+Ie?vJFpA2^ zD&yaKF|L6I<0@r!dngrvvUEKO0|0&LC6-&vzg(22Q7$@=NrPC+4rE)5m9M)R0k0BW z&s*QyF>+mRqH>oI5AJ#u<14QcJ07tE810nGR7Bo>7J}|xfca>19x+eBBl`jjS&)nd zpU<{6(9OXk`FA^z#R!^HPG=mzAOQ=jE=wdkdc!iSHdbj<+O)|3Hg2t01Xt(UOUd7} zi%SXqJuF@E-Urw8`0BvlTvXM@+h|#2N%L3dpv6xN#9l0&_zJ!DmH+A^T<%kQyx1W2 zyZ_pMuuO{|zxDHO!mNdT`WvQYwi=6C(6tGFzV0=KedPs_Vx6`@}zq1B=WjuULX^)g67>s>vdXeLKjSDuR?w z0Fo1cMuo90eh#jfG{0+RZX+ELbs(kBCp)QMR&posNOK5z0{Bgt6DU9zFpTt3DtI92 zv}*CJ1RUHIerOs%AwI7Llr#m`=oQZ#kO~Oi#8zk#I^Bi2LFCyA3Ld6o2+fzS(F$zZ^A%Nz8W3O1^@hByhDo#qDNrjj zHG$uxC%fgI4K?{Ww{9)zXW788!SLX|_z9-m-^25IJC4;=6u8KJjZW%ik%M#eo1Nn0 zTx)ke4Pp31fdjJduDA4)@=t&O@(sO}E0#?HSQQ}2Iv4-?a3A7;84nIKgVNhg=G$)L z#egY5thQ-#c|!tB`Lk(O$iP*+_&Z`GGbymVzni%D!K87rvjI_I`YgBZ>|CAISfuEW z)qEfTp7@i}44`hZ+xuN<-lp%>8YNw|pWyn?(lRK91^G)U&DeUrm>d~wdg%3at&JN^ z4<|mcgO(=edrndmLRo7N6Y@KZXxsf8ZTD~3Xvu9&F{o`5dCGxaL{56>8nFII02ZLM z+1iKPA(MpyYEkuyiW_iY9O2=xa1x*^am}3JLQk4)ZfTYbO7dElYc-=lIrgDUbT+8& z6<_j!%g<^KZPuq7iGk30e0Fw04q+*#{yg*9==5ncdrY8y?1y@n|4EO5hl~LTNcLNq9V=1%s_ z$yw-nl z?nWp(1i=-XixRbMepc8Li@>r_3JN@C15e~_s1{)lpHeCisKFd&_VqNoLEZb9W2$e0 zT?FqU28x5VDqF|;FI+jae^c<5Z~cezN4q>KzNFwA)so@k4JH)E|L+b)ER zLBnnkxx0?EsTjHhfWe5*0M`ItD&J(MiGkVSpm}0M3lv@22R1gBIVYIf+&!IDp)$5QL!~;^RuCkGBBqI zUWMUZ<3oL(l6ME!y9w3XsB3o$+e(zFOW_Nw-yPymdzu z1OhMR?WWTC*(lHV^w~2>z4hc~(S>D$+&dv-*k6|0&VI-M)_H>YY#~aH6@HaCGs`SL zevrQRi~l2i{Y!7Gr|yezyqF;NUw>^&XY=1*e)}U>a0C-Ai=zUYToWL>GC(yBkuI8B zPWti*|ytQEfc}$g#)5`)M$ZUqbed{5t{8_V*&)g(d{$<)<~Tb+sVopV=5t# zH9+j(eOsCdV@yRTtW;9BRDh=d(L(~ls#BsACEN-kb&X#8R64Yrekp^iHGst&6A)&9 zTi|!^iekQ_1C$sPk>xBkM5x9~N>#Pay9(fwP^7oT{SwiK<{E$iCzf_Y_WLw!q)K59 z1<{FuBDVlro>}l>pXqpqa>qFcZ1HVNXFiFt8^IdH+VXW@$Fu`as^U_=CeZ83zT|+h zqI6`2xuG^z@807g1+XNTQO^P^0M?YewNYdO%LdXw)fvSPY(eWfu4c3j9)u$s z<^b?T*Ozt$vKg3^F$PbXg=l(_b32hWoI935thf_QO8iXcl=#f8MRyHmg} zB4m6~_lTM_L^6c19{SD=!-DV}Dea!^O2Bw-#)?Y6BqNy8xMCW3{(ig1Xm=WHE=EC> zLqBe;3nY_66Cc^ZD!YxyUN8;@c1H)WNa2Y2%$~*B81nU5OdwP0Pm_@mE0F+a=?%x` z3(Fj`I-mIXU3Vq@CWJuQoEgRajcwc<4*I#I(#Y2j`lu>B#X4OAbv4Lqke{WSzg(_% zyIlKh%SCjyv}*%BxjrmFmi3{r*fu{!!_Cdy3!4RehuqV+1oYZ$GciV(jEI7%x-zsl zXWE~mKk@H>g?{UAP4t2tFD8infmi>dWhVS}{UZsTq?~mD@=tNWx1W~#tsE${-(XCM z0t}*qAT0rx^PbKR&orIZ{%yLqdxI|dXL~G#4mjNw!($fohyB5s4%Etqo?e3o6@a}G zgis^{jJ_>Mx=pm-@7M@qH?y^uyubq1zZN-Q7FGoYgUyX4+!x?laNb~U1S-77lDG5B zXUyCQgw2%zsL_KKM#=Hp-0F7>1(yKF_4=j?$-#p+d|P-3!73bp^$&b-D+O~fl+|=3 z+{+ZFfyF*;mVy@*_VL+JQ%+O>6C&RZM})o1_7w|z3cRjC(NPL9TMr0e0f?1*n)0g# z+7U!Wa5JdYTw_vphSl%I=N38mBMQNuQ%Iq|1>}_CO2D6iR?kGmT>=8cCI7{WV6y7h zhe9VPatEx%+d(Dxxqi#@SoQ!@=Jj{mxp^GAaVjTLsjUDrzUv^CeM{##z#{d2u(_WB za^V{ckm-#0Y`Xz1O2&Go)CLfbH4m_n?qfh-jCevOf!RvNV(S|DgTs(w`NqJNff@&W z34rCT^X1n22|2R)Jk1$9TYEHEtH?%Jq3q5Rb0m1w{Mqvl2eqZ|Yq%%nG0KFb$_1Tk z)DqaUMj%TA?q7}d)7^Xnk2U&=4SU%zx9Sh#m>u7DSi!pFcXO|D@YTxM1cOKCd2YJoK0r7k=rdo86?mRY1sW;`OaPMnT z@FLA?O0wW+r$9gWLsYJ}^x9W`<%*v0zsmUTqm4Th*@5y}EY#|Esxbggd*sjeWID?1`WYwbIE3;}0 zFLKJ_JMMSON@6k~eJ~$60llHHsd+4bJWnz934sn)#~^~s&5d9jD{V&uvE%4Itz5B2 z{9p&I!ZmnMmHEn!L#oV`OQfn0gT*hRYX&RUbRLM%x9U%pV#`6STh(gxl#+s*06I&6 zE7Yd1NubJARzL%Wii`n4(f5p8tz-sJFXnos(qH!oAU+4Yy5C#Zru02BrE_BYFoQ)X zMPYh4V8VbYz{`5BW}9QRxDx}FK^PG7CRPdegy20mA7nA!NlJbMK$)*ai;xdPke z_2k(VJ=-1VhJ#$#c(6WZf}}K*x4|5*(4v9g*W&{gf5(eROH$oc62d)KrhYj(09?> z^w|Z(KG?rS7yC!FY0qfnC`WT25`V1&Z-v!G0PloxfI!jW20Bc+grI_~7(Qh`CUGhk z0dB1B1OQ6`PG>`m@O5(Q+mzngFE-y(?q%0uwnKodm8S)WHL<8{%p8%Sa3>JOacI2& z#7LMf6d%~hfb?7lc~t>oC6C{xXy$>96(<-5OTCprZ1n)I!E(TE#?e9V1}V0DHZ%hX z;cE;K3yYk~YmXaCxz$S9nz@ryf`9fY2D++G zhZ*9tY+HNmc|lb|xJ_wL*EFdtl#djMpXOV9=XZ6VIL)TzDSM-zp#CoIX^P0BIp+}rdu$PqLE|x9E$1LY2 z&WtP4YDJr(5W=wmxCXW^N6_kKvc=}Hx*E*1UHa$V*_IwYd`Ry+dVc}3J9>Jv%*lEt zbCsWqeYG_>Ph|nYV)1sX-)?Ehe_?tQ6OdF7Jm=t(|3&(fKl%$VQZ=s^3&ftP%Jp0j zi(iaYllL@=5qaOREg*V}!7Zmjp7(Tdct#KBNA#KY4!u0TNiQAVq00rpp0_L7EFgDK zw;Q>R_-pOC`W4pzzm|Id#4=dIRxjQQMV&3<*xs8ws0%5ureJ3q%J1nlP{joRc1o_<7jc7^(`2~w)HHgJ%%h_-XxETlya+Xqwa0PJe|H>>Z z1K7=gGO-36`I^;$tZb20XFlb9+d3NtkqTvUAV6%KS66HY1aB^2aO#!W^;0a*^T6yW#?N1L%e9Ja zZ)X2bY)w{)Fe8+F^*J0Dy0tCVpboI2V{Zes-u83}%+(Gw3Er;Jez-5Be#lB0N)Dj4 znwvof`2GKw)A{Jf!;=Cz!x$&7zmxreyDK{IT!Uei7d@lyeSvGPje!J+%c>3flhV%u z-vS~6h>bpIiRpz;?Leag-F3gj?<#}EC2Em9D6y^MGiu7M8HPGCr8IRV*iW*<#UPel z3b+ZAl9OHc>SPEY*#lX;cMMRiXP&cuu%!lyUp*_ z*k#@Qv2)yAVDQvg+K!?X4Hxiwets@MR@Y~YeNiE)=MUJnk9H$XVKb)G!CxiHZf|gn z@P759c||w*tHu0{66@Y~t((s=U(oLOJ+g7cIo&-dX;}yOeU8Ck z;zs>%b=9)|=lBfvFTUDM^#0R5J=@K+JE&G;;4SLZg1{TB55f-bocaedr84v4+AD%bNsY!+j%1+5;pyd)NsbSAR*%VC++`vs^zqnD@m7ZCf-@_Ks# zuy0F&gY)^>0$?ZI_T@=DXyWw4B4l@0i=e$xP{=oee-b<2Z?{oVB`X&sOlY1~t2r2G zHsg^Te81aSu0&0G!r#v+Ui53qmgW~gtR;?8S;Ox$z_p+7g~bB2rEV?&${I9dYZ>tB znk|HY4PLuKN*D7-MbxMb!GNgtrv5?_@WddNYgvy9Qu$nEVpZ8Gh#d&QPLemVXf^_M zD?oItu?xvvK+Yp8+R^2j2W| zyKDRe6H6*L(L%V;Kn~RwOn2sX9Xx@L=i&wjb&K*MBsS*{;`e{>3V_0EwWg`Hme+2P zv0RdNw{MbStEB=e@~hpZ6r_dHmSAjsGM)gidCsj3#QYiG%&?`c>q;dVor4+9=mZC< z`Y`5Woa96a@HSJeZEOD5coqZ-Vw#1K zT0EYjK?WoM=mt?KK0CcX09Xyc7OOA_hF`CvGElQ4(8|Z3;`?3p-s}R90d4G@={;~x zHi)x(fu6&QT-fXaH!yDNY`SG5yKM|!&m^9r(Mt~=EWoubAod_^?DauYy764)s&M=} zN{U-r2{I)~F#Mv_XHEfjS`ME$UZ`qbF9wMH!fT&hmiaH!$*Nq>0kQJlEL#ZGq6aww zms7tefYn6LBpCuYQSzR~c1@QHfc@;A!RuRee*v=hruQTt>}Eb_ggiTYFAH(5MOcOX zVzo^RfSng1e3%u)@>VxjR@rQ}lIw4is;p$D2m-dX*~OUig62|iIP5JaVASTuf<&o7 z5d+4)0=(w3v#4r?YbvX&j%DH^V0{Um0=lOe8#}|-7GRhAbB@J{yqFPLOQzL}SM-sf zF|xHkm~Shh@~OCD5Z_036DZ1t4t7p}+Ce}twtln!sk$PJF3|X0ltvY{cVqsrc1;Ac z=4vJT=|2kFVPtpF~rkuwpTI@ZAU z=(@>`5(O9507;AzM0p!?nU@}X{5;!&eGi}pATfi<9KhAJ6+XEoI5Yn(0k~6cGXTX< zZrCPukg{^WK>67MCM-R_Uzei)JW|~6-THQvGz(UJ=!Or*BSr16kK4w|8P_z2)_suW zU=IXN8!~7&r9MVD&N9=A*a6skd6h%!hcG$Tfk<;Tu|s-(-~H7x995w=xI&bo{t zL2>L(aI@ozN_(k>xHn&XD-xJMjopL5dBwdYSjqee=JuRExaEjvj;j`|J9O}F9< zlmTpS&><5=HgA_m2>52&-frjxQXD+$C7w&gBt?4TJ8kb%Ght%hKO-KD^M zvW8JB-_%isp)!q@PEDta9EkSUf(R{AEVZ`AZcf3q91KIv;$grPD`kXfy7pr|1!7OL zX(trPbHK@M-#_{e3=s-YSyS}2ma`0Z0>7U5kV0mM0gzZ$eZv8rD6a$-ky|&nJO9KS z49c~2wnABbr<4R@Q~Uk%1o%5c(kA|?z7{S%f!DZwWFkRcOvs_&B#)m>J}=bCfRkYK zw&MhA?d`ked-=Qo<6w}?l;d-dvr134+o_By>3Ny~FO{)Xvz0h&i@xa@WR`JU6s8Fa zX3^aSW;kF;Sz4?mbtmBB2iG_I1;`$Rt<|(&B@X_Q$dK1S#@8tYHAt> zVClZh^{BdgLa(+xy<-VxRU`_?g_>QWu$vX|CkHl@J#$&XBgiC4+bA<-phH>UEVHrv z*~}?3XVE+7=NEK#`GDSkdP8r#^LPQXyaQk$mI*AG-;?18bb`&Z23e^6u0{|>^;kB? zr0R}`OJn{Jz4qn*;pBw;6dj*_5c`GS_^Znt_>XV>{6uu39tcFh3zMyN`RrKwjSKKR zpRbnJW4f5%r-z3(=rh;fkm_99`RVeypQR{{pIBMKgr6akvIB3slGClvs)ND5hCSFoi3JmJTrjH5EFRzmg=MG+LB z6wi5w_ZSox_}+rq#a5Ut!c+PXfm#x@rQPiPor9bJdevBk;^S-t#o1I|{}QYwA zJv;#mIopdawb6lodwB+wxphpwxD9YlSYc^3D**z)_`RF`PD*-DQhdR-hTA|WBaBqk{e0;& zZ1l1!Zr{Ip<{2*l(f}dA-ywqt0MNl#T8k|L*B~c@`85A&o~xb}#poBqK44;zla}TE zZ28F%8!&FpMBh}&8N2oQFVBkkEa#5;0EUu}=U8-5G5OUQeLAQXEE_t8_07T4yCIW6 z?1V%PbGTup`G?FGsqfI=&cN2dd63ijDmiAM!C!F2dgf;o7|K4b!&cc?epVq-Cec64 zYRedG>3BOz(a0RMNa@b_{(^PHicK{aVrLF?Vus*mPcvsFVD^BOrC7yV@lWEd z9^JwoTo4tj9gle|skFG%VoPbZh?#zi=l*lI1iJ>G%WNzdS!NEKGNXfKFTm27 ztylq3u(iTh8tCmUWJ-*qDm#gi&6S9#gbRDIA_!COyCwIi7Qke{bVTmeT0Gz6VA5N7 z@@hb~n9o^_iZl_}W#F>fdlvmF%8LcP2~Ej8yU~fm}b;d+>4uZdT{6?~w&(c2|H-IQ5;pwU>cm zFb~JT+OI9wwCBkRjjubd<}lIfemx+U_fox3w7~�&~s`10>^`HFgWodS?DCZe$Wix|M zJ^?UH45p~&nxJZn&en4bI5hNZ0BV&Rle2i<79uHNpt)h;0;Rh{v-r>+)CHkw3VA7Y z-PD~xWl-{{v^g$bi533H0X$Xq7Y9-KwRWB8UN#mMGiGx9IBj7!etA&m7@J~k0Z3CB zqu^D#Z;sreklir)0b)^&P2CB2zK)tNdt^pYD|5MFYU^5x&tT`ocN1ox%j@;lY^_&o zi<>tBU@^DCoj~*y+)ya*7NSVY;A=4Y+ZmG(LKrLR17Itd1elv>=aCF5w)Ud?*K!l& z#Kh^dI8}{Q_$o$r%~BYF&tqfY<x?1Mt+wZ+k&#tc}y<3AXEEP*rW}sCkm>Kn1 z-ok%8o-rTKBOR9GU!c$ZTR%g;_4B)v<0M&t{+i@&z#ssB2D$Y=&ai+^mvyKzv;7 zZ;7RIf@wziMM1}iAh()D7Jx92xgQji zvDFYYEvislX@I4`vlsN4Gc5oD{g!HG22X+5+dwJ4Ppj{q`h0!P8WgSHgB=2p1qDqt zE92t-oBmCTzPMs4_*cS-$5U{@hy##K;PVt{z3rL1_vl1l0FdK3Y;IIz4{3F6bVBym z#WSLF0A7y(ZONP;sRE^Xvj1nymD7Xk{e%0T0`qAo;+ARKex^PcEM8d4bL5dGRNb50 z*j)SidfF_nyjR?K!8OrW(GS*fY2HiQ%x5zEr4( zc3m{NmCVH=@#RSSWaDP2(Z}FK*Wbp9#n8muABN;&b7DTqT*LR8g$x%1J7Bttt&?U3 zfg@})sm&me(%gl@g+K&Ogsthkwg6UHPa74-)zUy&sacQ&grG$ShtiKYi-oek1X z=ykH(vFF=SiWl+0E*d+lQu*kE$MpEwo({AzV_>$N#}*uJp(SnK zkml_@`UhY7`~3Lyg4i#9_0KO0`9DATSvSznSNI~xJaZj6h%IH3idz@+HQk$@ z(8HM-*yTm$0u(v*3ug_G6jcWiRJ^Toc6MPPc4J$| zL{XbvwU}%zHDyJFK{Wt7p`vh5il!gL)`3{Coy~z;tXERQ?w02|(S>o3(m&&=ITRP{-^;rt z0+v!g!Pl^b_SmAo|13Y2IcsIcnJ%@7 z`2E)M;dZ0FSVoE2c4*wVCF4>{k$E>(bv@(b;@%~_{F#^O@snrt`ujI@bb8{99CXj`wP z>}f4q*+v#2AOT=p(Q8^R5de^x0-e7LNZX##<@qI@FCdn;t^rypO=2jEnZUfZ$OU^s z_;*0o9JESe$r^`wO!CcMV zKqU%jIZIGoESL$IW#t;$yLg7t0>z4f9NpdSQ){V4XFqEd@{@;AH#3BC#PIV$>3aMB zwsUU-uWKV>eXNa^lRpC=b%*v>$qA%l{^WZkkeBb9LFXx;dYiyspHoh~=YDjN2F$^N zDQ8WK)-#tr*7gdp810^uVMqB{TXnD(E3*3DfF%bE>ZOYYk%4%}y8w0Xp+D zfStv2E10A>HvxP8t=e}r&~xtYUA%2XQO3`J8o}?g_!=x`#7|b-Mde|!q6g})FHEaA zzgGEdp0s+^UUrVisv{y0%s^nNQ8Yr4uUMh!6tGtHft<}3lEGLoI~emRCUDDQ3-?ep zpzF~k^YzVGXaJvLzH`j~$8=fd=e7FZZ2uli+7;5`v=b4ryCxPE|IyrDOw-OxOR%c3 z!I?nKn`0`((D>aI2$Q`NsUT!>B>-QiK1c$Fc7RiP}1IYd+jDa$?F{#uc( zM98`oDDL>q3h*ZId0mcPsjweQvvaldY|np;rlB{JE652YAohd6DWXL{N1OyJ8$I5TiYeWX~Q{ zy2=-&-mdanfBA2Jl5c+&jvqY``}6Qn)*gyg7qe~SOU27*31>=hgZ+n@rJ`#j#1rd9;*f1h6qA<5Dz`1R%;-GqE z%aULO=lN&AXpg}VcUtS~urOLYfUBt^6+uA*oR4-If(z7RH+4qHvY(75?sldx zv|X89_iJO$);i>Qjwd;dq4yL@{ian?p9W~n4BOs~0KIH8*ukh8@U4WVD%cC=cky)nqD%h9|Y;?DP1?_EXMRoxy*T)D}7LI1Hh06yO+zh z5&fA=Oz%dHI}l%=)aRbck}T_Kf|JAz>TwkR-f=!bteGIi_B10R)e%Q z^ghvGtTl?yREacW{cy_wMtZMQPHY#zn1ZastiSi6nKuB|gLw5?t1++wCR2lZH{kW? z(G&d+l^Duw&F{Uk&md#g<_i2=Cc*BMQM761VpvdcVr#dpK8Z z!Sj1T8~C60<=!*gG*peOLhD$N7#*-OE`u2#CR*%-d1~ozOjezYFYtxBU(7lcx~KJ{ zX?ykH-5yYQca&#OU&xE=O_7?O80DfCH(DWxz->fJJG#Q`>vt3{=fs1rVR(=C>;EzN z>3{#%bb3|6yAsKWiVkIAkMd;g zey|_N7nbj8QG>VIQ+cD_ZUA;gTBi%$z)iR!2r8M)7P1+*pC;}<9ISTr;ZgxDBf?jw zVp|jrz{Zd_-^TMC2#la5HoP53Xt`}8=&7mS1_0#cU_xk0YKtA%=u7eE*LP9+kwd_> z=)b2w+~0FT$yXmY00`aq%ybVcm`-ZjEUfaGWil`!+XKC(DclaMl2$~(Tl%dj0D?$t zI8&55)cqPDx>ly@qNZMgT1;agm3b+}bKg?n)1S(+SX4DxZ0@vkH_J>7=3s_Mc`MAUnER8};5O z$*O6%j$~r(`zC8e-4A-P1I==v1zkGSwqiD`16P}~Yl-{n6B?!o0G@h=GCW{LC>Es< zOqLWEF|LgvNU_EscpsIJx!KB9CDwlGdJhAlb5l+J_yzKgl?*kup!@00mZ4xi%rQR@ z)VLzGllj6{EyCk0Jqt^Kuqb)M5CMRV^oJ%V3GPjX>GKuKeNLAyx_j+Q*1E0-hi`LZ*Pji8}jt( zR{rsKKa{7}i%geKEOXIH_YYHM%P_+jTz>xecv7&RA^?`k)Aemw}_Q3A>(MpaY6k4mx?e8^M$HN&#%oLh$y|t8%?B zPs+7iPDf=5>2IoDtEeEeH}i*InZO6=AQ!VSu{}G%+=IHZ3Aoz#)c3Wq7{Q(m5ff@P zfNW+}EDDvWN`k zFM;j~q04|Qn$~z9)vlSlRayX8|4#VDM$p&f*A1{*;bS}5ljCtGw4pGa7nPcx^coop zg02f@Zh3uv@$$KW;fKJyP6sEMv44<+6JS(;R9LRjt-}$|Vo{{ZgU|n7xCw(=H8#gTi1*{B!w8rzy ztRW@M0nL5c0suO&+qI8{NSO1Isnmv8`Q3Ouv~fKQ@*`E)z8~|0+n-@sX-<)+rW-}< zzq(?KNo@ejm{skS?mH;)9jk=MXK-{2@N2BC#BJ>wg6soOjB^$fNE|Eba=$3B?N>Kz zkg>i;RPb6|J~g4c%Yq!9maEywht;ne{bEDNR;LDL=3}K< zISqy$tHfZ~oeRp6dbS$2)weXZW9{dJIbeH&g4lupD#ixj9Ok5D$=Ytgnqlr2 z>;WVfC@;KU`m;|U20YWjsI*busijURgG1hXMjlK8_ZOz3=TWu>VBK{^_Hh7s4?va5 z3N3E7o{*`5ysjWl)y%JT5>@j$Ge_~Y`Ezx12(?l=G8$NkFde*EZw*kAaEZ%MuX zTib7+f7NMuv$Z#5m;o*2E-e~)-a*)fMGZ%Y~b~s<+tVY8<2goeISp^ z%?4rxny9g?3OEEuP8J_WG7b~U3|2_Qs_xkV-df>>)aggRKk)6E=< ztVZ2w&o`>3=yd=o*-x!NKqgYe&+$#1EDso%mLib-{B?CepoI?Lw$M+ZdC6S|@}3H? z*pFZ=Fl3EPDC7z@jk%N|c#GhbX}MGgyC9V)XuXo|^Dubnr1Dsnf z$R$bLU>$fZQ+ATc&jp{2pB87Bf)%c5q7EPq7dQw0&dvV^k8{p`h2!)v`&W0@%vloJ zneRHwTJr`m0`PNyHMjqxp7X)B;+~%GvlQR_ZNmUq@y$uRjkcc`&z?b+uB2zMi(?&V zI;RyWi)=2jxzY9(*hILo^BJq>KIZ0Igl3~s)=sMc0_ZcEVQI&j9#p&5Q_BZ&3%PtHZ{)S z7)1BrcajyJ_RyY9`(TQGS2vatZ7FC(SfpUlnvD@wF-=R-G)u6aL1-8QY%yTL zTRUM~ClCV7TV7pRoGWtBPLszB6e@K#QS_e7PQdjq=rhU!`i~dkdy}t|*jt-F_j!ew z@}pQAOOm%qCiy8GCznrY{a~$mFmGavxlb|plVheS91o~Iw)M&VHu2tndMzJ3y;72F z>VaOpKLzH0_`$XU&&ak)%#`u?WKvN6xcvTq=U3z({na0dAg>=C5c@N~@SkrZ`ER`P z910ZsL(UC7!;DN8EIN^TLgPMB551_t@IhTI;sVHo=3>=abMDq+=A7oPiC7e7LnlfD}``ke#&+V=f5P zs=G`CQvqv$Yy||owfd&%8!Q=)9XRT5TSfva_gJO?Ui7t%s$Vv5Jh66~VqqGS;rlT* z#kF|$oNM+D3kdiKA-0wk0D=lYjkO}zunJ0xssgpUjX>QuegQ#fPwcK z2|dW0T6GVVk-~MvUUfwpw^3y~VgvMf2t-;GLeSKI>e~;Xv=E%~`OKeDeI!#U?j!tX z`*{j#oloR5hpE4%X%k>G?JZ#K>C^9Ewt*E*^4E#LT7*GK+5ge{Gv(7|_u(2$1!qiV zW{xXTLE_qwo30eY?C)6#2)v+>(@iQXYh+*#7qgZ{r^SK92nNL!mI@0a%f|BBoH%qr zVc=u7zOiRLKY6k7ri>iq1v(}0X7T-VFCSV`|9lMKi|g)cJ~oMTC@6CbrKU+sd#J5Q zX7b^ZrYnI!KJ@*g2Y5}@`t;e0m-4+2x^HaH9Xp$~m%zS73`t{pF;`$tk533R8%&Axe40K z<)wUneP6!R{@M2WZOsY$W_>Of2aI zn!4lwe@)f)+8SxNRt76r)U8#_5@CRr;lVY0)5u($VZj^_@n^GZt;`?Djbn!^rzR42; zxSEy(^m3>U3>YBB=RGYNpvnaLK==Ft&dO0m5~#R-4SJ(yG zR&JR9Mf9O9F-Y8`N+Cd>27ny%d<^p116cw$+blB6JEp|9}oJ#f#*S>`ddb?+{59#9v_tTg?B34CdKRdBSd7SNNJHwUA!@GOwWqqM90T{}vm z42k;JC$E4mSihyEN|xEzH$e2cIp9D`XmYjdI@?PYx(UKyil-Dx?wd|m(gI~s{@9#d z@CXsD(Bh-<^(*}>S0e8LY)GD0O}wyyzvv5#?l7hmc{#NrBE%o9PpiqyBT93x0I_6W z+YZo#g{6d2&u5`8V-xpBWwk)Ec<%D*fYHqCR$Qe(0iixjIh_cB(=UfEH z@AIGr6U_Xt-iIYVBOyUR%g5~fd4GSWKExIGesp0u%!xHMozye~z)|Lz6^Q^=!E%{W z27`dy&aA@~FV9tizsbL7{bAFBGN2_nmE~8E1cj`9tC-O?9xM9=L4#6JzyWM;1A0ih z9jrAKV-e(puPGGVV4&DUbrXR20%;%bZtvvk`bJ*fEb{DnmAl@y=@6C<{Zchzjnhuf zAD zu?tz3U{A>2WhSK`#sY{Y8z&g(fgHKZtt@)9F=~E8D3+ZWUpI->{YU9GVBpmRqcW(n zIxy@pvJ;t4h+Pfvg7B>qePzisKp56By_9SI@M+6n#XhGg4A)Nptl~S)LT3|{A%W;S zPYyn^k0#?P=4b>$?3vgckkzEvSSqen0c@AEnnN~$ zSXW%7hH}atZ&ZK90Sb>QF?VcDi_0%|h$=mh1N1*dFL;cBgPAtYXCEE(i@QJg9uRI1 z3`mO@hze$v@-11?ARDGMZx#iy_xicY-)&y?lIWdb<R0 zXKn0@CM+!C{nO}L5HegpS!3M|)-4gAFDy|VrDh3uYxq6mtj*4WV)F%ai z@*Y?9^@ed{ATIYzg%gt~$#uNDmi7LqAojy&H}d>eKx}uI z&#V?rV@E{lu4~GF$m&{)34CRK#DYSy`H$?Vv}~SA;B=(`TLOrPDyy-C(k8-1I|Op6 zkFKSv>Hk^zTL>8n%t|3@&(SYqKE2SO*(VUZ#f4NEUi~^3UcMFPs#-j#OMOU9nbrte z^*^FOg9*_}(S}e|KciYb(wBi{)buq~QD@a_Z2l(fjR9y4W;ggf(gef^U+u%~_qD}l zVyR>@SZ89Y=0oIykDHRG6szc)nm|l|x$H7>4*2Bn4lJy1YNJoXT?ykK??``uYpWGf&5^E7PuCJ5pI&=`|wOl((D)qQKR%q9KkY5nf* z?vd8cOz+2D`nwgEseJ&fKtdul0RA{%wPlpEGU(_w2l6}>rur*L+#e5QzRnfO29xYV zrB*>ADtGp|c-?hCEm$SmBx8Cl0$2?VB6ceXbXy~BSbuh%15h-yeD=h35BakisGH{t zWmca&enZ>*=<<=~HSc@Z<@Q-aGO>cxRNV)5f`x(BJIejQli)>}`Nie0_2XfxE+-58 zPkBrz5w5wrOB-dklMpelHQgV8yrnM_>`mGFk-)0UvGMo3PPpa=dD222RJhUjT$)Uo znl{Dicyle+FR$d~&2a;K2C;wk|GwN*n!maI_N7;yR$TDrjdwib^f0kd+r7N8T*>F#Q~9#Ix83Nw z>PGOWJ=2O^S`7wLPooOgd(oQWWr__-N-*tJNs<<)BBZT|$X(%z5M^N<1UA!U zSLC79@2$MT_%z%A29WFhgTRQyf}Pvw2gcPk_+VmJ26mN#9FON&wa$P@VjENtVDV25 z)Wku6et-M>kkZA>l`bO1^2ipN=Bt9;vD|FG_6neDY^&Y_zd|?YV3xfRU zi}>BM9F&nEY$d-BU_2RUErBb-m4nx9gr^xt7T+YjKU)Ta;-F$bX8AZZ#nbDJEZg{5 z1~un^<2eR3&Z}qpL`F`WDR?4OluXaD`-CFGVkXG;KK5%1sq zcU4NjFLGR)4?JUQ;np))LBL`)2B{Hv+SLj+WL|H`fKuZAahn9y3fnKk=WcK*;^4^o zBF(96Nw9ZQyCQHYo?|bSy(gl7pcy)-Tlo_6iCwK7$7Te;J@Q~13w}hLfq^;>PFLU; z%26Ta%&kdg{>Iji^C3D1@~A|BY1@xIDiLy=@>-$;!QNlg-LyC_iF+x7dHV`59uv>U zPb;2jv4`ek(hrR-m<&XzVg*p$m4R+CRQI>T!9lEA5kbPNdEO>DYHdRxGZZRQ)|$_+ zsFLUEnO&pgJ}o8}og|sWV$FiJ4f$vO3bU(3~tmm7$^lb0KSeQ_spz4olKvsUp^SB1W>rD7@%9Uor; z?!Q=mMt=Xl{msvEkk@A$#Qve5|8v{F|K3NRXew9M?8l>dRW82z^q;rsbh+Nh=h}xG zczs{KRK6!)*Z}Ms+uw)v+F6fffp)uEa&O}mLgFh^dc61ihD>)dAwAPH&k$t6I9OT5 z=IjLhVOwKxSky1KCxFnSTPrK;NSfe%ZLL_Ct0aJV$}rgm91C#hEp9RjvX`jbwT7Uw z{Sb|zwknYO`epJ5J0M%aw@^VFvOTC-ZZhVzlbl-l1}+1|8s#nm{pW?6xI&T8Dqan~ zuayzr7F0GN;44g6P`Y7`TG15~Itv)n1)H58rmcOp0tZB84B%p_Cj&7MT7pzg+0=|1 z4%gxsmHm4oL-{JumB&-Cb>uzRu75iI`!<|2-v=ATk>9<4285p5AU+L%J@?EX^?SbW zIUsA66t*vvVsBTeDYc%&d9JM(YtI#K?stVG3jiSeA-SHoLNl;&pnCeuq14)4PItCb z5Igp}48qYF+&}^+eQb&>1DSjli6`pEobrMgLS>8nC`j?AXmU`98ydx##ChrpGoKU3 z+KQ)~wF}#g%q?qG=5<+Ez7?(q0I^mP8v(N|6cM8`Pz}qlk;r}z!4xk5lQo^HN^_xK zK3yT~YsSDIgq5f4d6UI%flu*Xap2ex)Z)GiTip_PtuJrW~p+wlU?~GZK$5abHcj!r zaCJy#Exaepg5V~y1RhFeH^XJX4+gaiV>g1>ES$-B=<}ROv9k}W@6(ST$3Sx2sTcb-_*|NlFYX9w+Ot6 zAux!jX6yl=?+d7Ar7}>3X?qhkq*qTWr?TSZp1)WT@ z7dFuSk}?7+N)7o&+1pxTL2Jd_gDm7jw^Laed~+F~fCp7_Epi|m*FkdM5gAh68MrR_ zQRAF5ZaS*0XfESBdAR}DXV0F=)%CU9^*E7j3^pt)FK_SUUabg+xbG0uczS#vAhv(` zUy+~qxxXym`S#WKbD1C3@!18jU;pJ_*?Rw9{#fUpfY|CkeDkhryw!TgEb_+kLO#EK zDDSine!_qMr4kk!Tv! zUn>;A4gRl3YkmJ_5-v;4GaHG9H1%o=Fg4oQIXcMAqwY?RE5iquHQYLbikdazeFrIzITqzmA zrvA_7cy4}~owPy?nD&JMX6M1{2>yzvzZsd{Q{`q4JYUIt?yAh}S=@8i989Xad3g@# zo|?Z;!Lomj4VpvYzq^8A?=>)BWMFVZfY$2Y;rq?oWqn2yr``i^3j*l!`&c z6b5{TvOu8`@((f*r$nX{fdxw--wePW;lHoXyA}hi+uuiL37Z4m4rl>)2R&?>S)vsH zR>sKnB9td!;-(2>oW~MsMk%NeSS=-5!T3vLTyS8&>ukg4fl^z(K8}iNg}!$%O0&^7 zRuEG#yTb1dV0aL7w;BUR3*>sQE~zQBAOrO-3Fw;Uk$@c66_leDbHH=h!he*Cj6Oro z?qNth%{$I-&QZ#~CJa!F7X!e`kRie@Jq@Yse5`OQOQA=0*#4W!6yR6S8MC-#kX_KL z`nhRZO5VJ9@lrnAKQyk@D>wpUdvxZ>?yXtU!Um#ZGa=Xdr zLaw$>J6hmC*Pb@jsU0mhp)S(ysHtg1BFeM4Bs8GYGZ~n0o86gR z7qv>3qRj3QWf9%4C<*VViQZy?){ju8T-dpey9)W#jK)Y55De}^Hbj})_bgovH$n(IXv$Wy)qKzn?xI0P0cKo) zz6ru{a!o(9`z-j#XxMiq|2by>TPpzbSdnJ9R+?M(EOcooGWgT7&ya$K0+M6xCbZOM zNsKhdxRi5X;~^F{e{TepF-uObG5n6RPQKsBBz(m8&$R2Km|6D~l<`^j&zu4HiL^fh zz_XHr;6p-wVC?{zgE_~AIAI? z#GaiG8I_&yCjaiBe1PTSb8;RMvbVHe?J%aX(kE5UjbRyoCMDinVv|f-jlpcN?l8dz z15p=(fMNk$nnj|6h2y>Y%W6e01+d5!OOPxGCaZidLUDgG*D<2f6c|_;*Bo6+RuqUJ zKwJH4GJsfp2`pRa24`ZWh9FJ-U>|Mmm@D`i_8mS4!0*2vCW$~+?vW+=cq6mCQ*k~z&imm4YIl`^^xAi2qY9ToT%=i&&u=0SM%Oc>8!-pJGE0I`c@ zGwJ8w%Pd#NNv`g<-`B$i31)jHp9H7J_X%SA6}~P%{qz5e{MNVs@%MeH-{s@831WZd z&;0f6-`_X9_u%GC0Gohp+J9U`Hmk^0i`{qJoLc;NCpbHDV76@CrSa0TE%-Lpr55Bf0 zZ~$VN-b-SLm|lhEcMFoBZUSH@%=#yulTkAe%rk(c^^J16VT z`7nSzWjBkLM+|_LnYdFxc>;>}=af*q{J!`{F+l=@`7#f&zWa6z*G$j!Fi?+s?}2r` zxBC^Qc38g1c;>N>2ryUQ(A@M|Mj&h;c1nrSiw7I6P|TQ(Ot}FXF&(Sd zkx@8vjapl~-0K(UV0P{qV7jcmxUfA9jI6qd=pKj^hwlH~f|Zo!74ov2<6O0I-^Ccz zXbTkvUtgTl5-wFiB#=o3b6Sxbp6@03>&h|5gx18=%Pa?8(;D(Qk2ff%?P-uV7S_v~ zZI6F`buG^}5PN@fBX_qirO&QMj~{Os;<3EAZ}RL$`>{ zS&)4|3Iwm-7qF`^s#XxdwPO>oi1-xxnMT(x?$=f5M_a@NhzZdGmxc&trE`jB2@JjScN35nXT!H;T#3W}45pI{57OVaY^I#NA5#E;erFt5uaXpUSLlm;qun zs{{bFq?9g%*Tpse0BQv8xo<5AI8AqWA8lt9t%AS;tXEvm8GuHtNqJivz zSdw@gRJ4OR-5(qqcoJ#*joQZvTAKp{l>yD=gb$Q{p;?r;zKSKvqk9mtRi=G;2VR4P zqSuVic0C_HorUg3i!pr+P~{w7>AGQ{(e!fz4+O;ot)6ZWF##+bQ5i3^nx8#^q8hgZ z@{qw()dIl!d3T+SMZp7|2GyqODg>|={<)$JYn!xqf{j)5B|$6;3NXc!{SCiYY2LSX z2PzqKomgE~(h6V-Rq-{(KH*w-cco#qa?Nv|R(&w-mnB@cZ2Kr45ZlLlS?(R&vr`m+ z6*9I(OQUz@wU1jrughDte9e|eR?|}UeGt!Ilhq0WgOD1}Fa?9{b4y$qEG1brh4WnC zD#3lBxKiNq(*Z2Oux1GN6n`x-&fN7F;MZ^yYkzVdDzFrm2{0q=(*Ud_swi(Lqbv`4 z2e9}0y1ITT&o>Zzdwt!jZEW9ba&g$cG4}wlo^os5z^pnpns}N!i+i*{=`KrD6N5s- z3D6fpU&~HvRACy+k8%8d`N?1UO?mg5|M26T{(2mrEfD+f{`w@3zVPeYsC?!82_f*u z!~9Vx{nLDe)G}DrgR(kUeO#W(+uQ5+wBMF5ukXpFTyG2VZUeYSOGCU`zPp~Hgk~t! zq&=Ne)dbLFTyr_k0zk`!W56(*2}fWtx%QsSOEmlYrqsr2W4vyP^v(I~XriIHa_@u4 z`}boX_^~2ytzKVU9#(8Tt57<6t#7o10dRU<3U@ED7|$=buDcU>`Q4HtR^k6jfKo)6 zg#Nndj6hBP#%RAK={69{^jh?DsuXTRU=)R)1(P-O*@O+mOk}fdOfUfvGzhi+!T^BS zZ1!#gEVWylHOU~hIXErcY*2%oKGn6CLq=d3!K;sd8pm^BXMfK4-l&A^&&k*I$+kbv zumh!xuYL9b0@%2|jra-UD@kWf&vC*$r|%9MAkRsE7*H($!TE&z%;Mvd6T%OTDPYY? zq&%7cf}5L<#aZvRy99wj4z&nPeD!eFYGX`<+?{ zVt%n9;a*{241V`u7d^PMxc=bzSM#wCJIevBUYrH2ZKu|HA(igtGtE?x z7I9CErZ~e$x1yFTX>lbzm!ztV1%sgkEGcntKZ!xU<6L7PvIcY^wH8SY1+krpox2RK z3MyR5k#K*aeA5SJa=f|K9MZj_WSuv8(aV1x zW_fnA0om*BQt){DUNe9S?4b8V-h1ntu0#XCa?sMGD!KiyK7LYu;!pe~`OaTIC)Hoe zB-^h&hqwR#%n)14G;T#YY*1wbU!!nmE zHAQ}h$LL2DW=RhSf_AJRdY5?Jnb^Xf8bJ`R&u7sJVrUypPhM)WCHDlfCM5)YNqu4G zEY1*wU@OsP1$8RYg_=A|&i1}mAUm6{s%H%-Rwe8}5Ia-YBtT!p1z+wblU3N1DKyIz z0cS8zso701erekE4cXGLlq5uv*qj2smEm{Wn4(<-VRqlifwQ?Z%q}?Tt9ect;WLeq zvf}g7mri*&Ro33Te{!9*4cFUWdw;!97@}SKa|+k%nK>>1F|H7%XLd8-umJ_&+8<5E zJ{bD&r0wuxP4gx9wO3l&wP&JSO1|9iAwT>IFWR*r2sR z5m6l*H#g(>&+o~xN_`*wI)djx5fO24$Nte?_LP4Bs!So)ny_Adl-EqkSRoM!S-qA} z`VnDIg}~AhTG?4z1J;$uRS;KHJ}lzpMf=a5p}>4(S1pvFDXCUe#)=j1 zGJ>u|IY1~FGAgHnam*8tp;V(X(ptDvXyC1kVLcqD%*3*EthUh_z?tEoKPziCwTwkq zcLF$E5YYR5S|S5BS^N0O8*;IMR=sD>RImZAo@T4N6zuWLFy-w@}qEN3$+^M$Y;BLhBK`xVW`*PZ~J* z_6cB?*X8~wch@%?h-Tpx=nTefdcnkG}N z$7qXMIGD1si0>tQuF{4X_ff}JrM&e|wInH)NRl?}*l0JdzC zZSAiPUKfjO)wzS(&}Ig&y6Ct>wEEUD<#wfM3~mY3ifxm$i9f^ivU&i*F$x{HsmYEE z9(esV0aRr!r&P(Qj+?LNc;$6uT!_3!erY?zxyyD+Bbkk|9L>B0?Up=R?gbNJhKC67dKi9gXWBhogCns z5KO=gSt;Dlvy_v5900OkKmHE3+~#c7=$&04pvHKvi863oNLppQ2>;LhUzPkfo z`}-+C^uhwH%+>D#t4dmvE+-sJ2M{+mk%IJ7f%+Q~rFJpXt*vV2X8J6cyt@NbCIOhj zYH4BB(#v-M+l;~O;MD-^s(@CRS%A9Y{Y#}bz(bnxn5wkUol!)H_;+g~e15H;T;BWi zClYCc>l zkS&Kt^5XhVo?TsU*Xk*^He9h*eW@F(92EfjQk0UOooX2y`Eif(v+}cl>MzRQ`?cdI z_|gyU`0Rk#Kl*3?YDVvDfy~_7m8@>wOsUcTI=`ZYwZL1 z>hV4KynG;!rx#l%-EI^2sC2(Fu*e*s6<-!x5cJuM=<&%J857`3FbLFmoq_E_BqioY zjp^QOM$vXQmOz#!F)ru~W&<=bG8&H_v)s*a4L(l`OltHgtZ9LdLNx){)k-|Jb#Xtn zwlaXD4c|ip>_-i=v3`=7TClA%fm(s73*XnzJs&K5ttK+DbE?io@CP%Pf0n>+iE{`7 zqM21UOO}bP$UHD1lC_jsg$=(z6P1>@R9jjFduptyx?(dTtroaB0LYb{+X2{Kk+H3{ zBzFX|8f65ldjR2XBs&1|5M%4BC<*cX`~2Prq)tKexg&$u{P%v2?a$wXXgrFU?yb^SeDo9FvIcploP^I~`gDs1Wb^Yb;fii&qA zM*9sqPh0|CT*BAbshb6^k%6e+yLj19U2!$o(0CQRn#qyZGVki+vd}mts}`w(ijA0aLbkN%OD>AM#3-@R-gw=icJthEdcKu-0 z#P0jaxoas3YpgX1x#yXF^yKPp;+DZ?0!wdm$p{DgtJdfjuASDRKMFo)aFzo5Dj{zr zfmqJ8{k|B;AL$@%&gZ`LHQHb;=mb~?u`gdJh`qkKk^5~Pt{cXfH&Jn*tHpdN)5WD+ z-`&cy7uz}Qu+8ZWSFAm-?to|OCDA*G?R#Ty@wPW`M7Dk2mFR{>&6W_-ugK zU;fRnY_spzrROty6^JbleHJPc_ZxV$AlBK zOL^25*YeG5)dgGeElCyQ64B|1IdIlzT~C3AH9P!ifI1VzQO|q?%FL+1lDOho4q$Lg%pN05ITXbn(%A-S zA@uu8vkX*PQj>FfPsvj(AglXg&FY7R!Uo9E8vdotd}M3&Yk3lCwu>8ii{}rJ5Q2#A z1impr(V&1VZq_RapcWUT7Q9vcy9NHR1SNi50OI1CQ1yy1rA$7iY;*JUM5jkK+wV8f zhs^!MJUH|BDI@1QgM((Cw+Fj>M&q24^iVtYmH^(CYm4LMES_&nd}i_Uc)c}Q4D$UV zqIi670Nvc!_Wgv0WCo@thQ|(ovjqCGjPmzNaS)K#bD@&y%4P%)#$%2QR4A6gAZ75( z>(l2nb;!w>#6cjL-f1x|9xMuzz4bXyN_|^sB|<+?Zu+AjuoF3T8T`tKI9=&j3rDSv z*qmiucwY&{fS{GY)t;o6r{#O%nF^L>;cY97D_z}SM0 z!{?b94*8Mfl)E>CTsbetOh)tGE;C4`&@UZxP@UoWUU_>bH&-t=5c^!NZm;CF^&s&( zxi}Pg-1jpk;M@APTZ^x5S9x(aZ5Ut!uxn?yFU=)sIvN|NeK`fh+@_#5NbEFH-h^0ki2tG^rz~-Ei{r46%-BVr%w!q#S0H_jq66i-sFkqn_d6SMl z+0J?*u#1%l*Ep|VrjPwR0l>n?oom@KI>4trMs?wz-HOPxikO>1Ep<2ltf#wA?BZ3- z*!MD;>LFX3gwJlFY{PX4^j8M531kNbbaJqXfGEt9Bl4hopyFg{wH`fsBu}0^v9ww} zEoCoD+o?pcbdEC$L-aNtCUY&I1<_d4SWDB)v}6J{yM-GS)>`*JDC zzLJzOT2@RZl!WaVP|7tW7%Q9i+`o$VWpO+_D5(4*0X4EYU=$6+u1#GB*4rDodHF(~ zKYuDOHV}K$Edtxz?5TYZ+xIWD|9T$YgLHcK`CXBxS0eWcVz>9EOI5n0++q6n1+n$< z24b6LF6fP0P?Y0SfLQ5yy|#p@^lYa4c`*{69>@WTOuWkeMtJ`n0z3~uGO-J7N zEQH^_L)NAXDW$qr^>ux`yxain2l6%fP`+9|l+U*h`o>utkQ9tx zPjA*IEpxk0C>~+%0-fP(0I{7F7q|gfkXTKR5EI&kemcS^+$ zl$rIQvhd@os3H@J#pEE}0B;6%CFZGvqXMW(u+OU$A!b^-tPzb{&MT+_^q< zg@wL(rTDyNdsdEIi_O;Wt0xjvp6fI5d!OqgoRz>4{!W0H=TI&-oS#7z<&0lH``8n( z^S+c{Ye|X)SVZ>mYR^pncs-xsKOWjB27+>(TGf=kY6>L%&@}^;T_hNJ1bwXt;OdO5 z`zWK&*VMkYhH|cBvYOf6(sDJpEkvdRW+>3PQY00Zv4JCT5jkq1BumVN35u=Hbt#_m zDbFeK``KJy&}mM5PwX$O7bwPSX|C*ex$QxApsOWBTz=mPgo$I*JW$wY4uY7Uyzzzt zS%0sC*_27aG8zDZKz_d*eEn+>?leg#nVAu7Xv{jy`WF8lwSCttQ4FCv70t@-&3<#fddov$op3y zcr%+)^5(z=kT$jaJPqkjVWRIW?A^_+v9LpWt)fY$D*+VDIlN{+2A2T_ge}Z5yJO4N znB3XNhrMh7wfN@&fGtBq^K+UD$lR8^>5~ufYxDO8fnLav7OObEWNsah9Z2sl^z_n2R_k@vDG6-5+A;Imh1&K<8}N%!su`wb?$t334cq0jOL zl3ih5eS}i@(7C59FyH4Jypqjr9m`!iLP1FL*xP>k962&NRJn0p3jo>}J{4_lXZrbp zE=T~Aj9-k2E^8Tm;^SJF=OK2$vW?GAkj?8gGFBWjk>MFt(HC>OBHKq6RQ>Y-Fr6hM z*p8qD38gLCS$jYW{E98G%dW+_PWJ%*&MJazGev9!bkVPWkpjeEoKTP)%%*DhZWq`q1Zto*i3;#lnp%k<)*>4 zDy>`a^AT4O2Q#`bu(%)UI_H@U#DVD>Po8LU{?52+c7p9YcXxL_E4sqoT@H+?HG-ux zvONO7IoJyT9Tt@=U($R{z@LD$%X??pje6A-R?PJr7DQf;e#Bxj8ZtS%CFS!S;18dP zzaMM67^YwhhmRKL4l`u*`FWJ(ri0jPd8$n8ZFgAI@w=EWRnHt-^tzmHb=KJ^gA=hy($yi+Kl^d_o>X}^D4~r%A?cgZ^^sgeyz*VzTP1Amwxk~ z-v;AXCC`gj4G7FlrviJz=5K#CootJy&3a>fE?-)n$?sX;m#@f&@_BhKhjO)@cdsC= zb|#=r$_Ok9V%r8{kLsg(gm$IEU9CBF)Cg-VMrRAWO}nN8SatS50qizY@B8cCUi4cn z5ME>g=`ui|MN>=^7)U}B0bkf}d;w~rb`c2?0gNxFU?4NG>j@J}AvGm;egK zLJRlM4C;H1{b#PG;d7+9UW?^>wdC5>BUjB=wraIBrrVkd&Gb13slZegfY}*LDK6Lw z0u`7F##mPZSenYrrNH+Pf>c-D157g!z5yTr?~;lb(D<7eFKZv&2`1kViTo{@e1PQE z9$XzC3&iH%&jHx|=P_eW?{mtgoPPdM?TPiX2c`5C4S&%63ph-V{QJ-rcNrN$?7kh= z50h`6Ttr1uP@fH~uyz6g8r@J>0HGi1tyThfG-SJYXgu=Kz})3PEBc{mU!e`Qv7xPn zpVK5?3dN1_Zn%y(vn*6;bN#T8k^-aIc<}#=`n2c-y4Vg&NHEbZg$EkaZK=xv`V_icYcb}7HAU|RFbEJE<$V{V@CugoxDbFxO09mLx)iWntTXMI z9VYXOfg%e^_7#0c5Ur;`c8z-lkcDPhIAqQ9Rrlboo~7XO(Id?r+go^df2+^0fK~lw9}N}``psI-Z5tzH zr>hyGEi6kuor^X8^X3ht1oH zwwm(Nd@R)!qBNcV$7S2oZ?5F(>V-Ue_EfI={;rQ<=^zgso9e#PbMGEBxa+Wi*tRKi zS64^5zFFkvzUL*FP}BfTY?Cr{EK}ZHC2eJYT9~H~_;vZQpZ_cJ55E1ur@Y7y{CK@V z?9Y7TzuNx&`uOInKx}cq;|GO~^hv^qgNg~S8QnFi{86WWDp)XXkkqo9(u`v7w=2|Py!ta&(+-; zV5)?laliglcsA2yi3hy(xPci0jP3-+WC#i!R0p_V?>+=>KdQai0fLvf1BUo`)hvL8U zeIbyb+-d3a#!|mB#Y%F@sxrRz?=4we4P%ZmZTlF*oE%0u=O=N^Nzvg{~n&Yj9Bw~q0A&aE`Z;yaOTz`u)jaQ-|b zGqXHBXQ|Tb$?p6@`N~}K#R3M8_i}rECD$)s$kS)f<@WAQ*K4=1&48S<`X*26FFqeG zHXwVj0os7-i>rJ2@a2u%^>kYuZ?M!H=7nuTeZZ{v!8!T-364IPZ_B&i{F~p;MShUS z>kDGP@f$zAt+rp0frahIt3fQQC5`9P^kaNM-N5I?aw|`k7sv_wzI>^?CvR;a_M-G) zywx<%mFj1Kncu+Ia=h!5saPJiV8JH!ML8%#Ac;W?A3pKlt+~aUXu!^@%`l)#iwR_b z=l(z4o#0>v5t0eBA5nOqVYA$^CW1#oZdm$o`THWNijapUgw{@wtk-8n9&9!{1L?52 zwlI@>#j#eRicH8H_fs|Y6sbwNMPb~^A#&o1hp};xdODZ?;Q|gvml5ymxAiK%Nsih*E3e3 z|K3uh_p8~|VgBLBjLH79bARuF&$;WL1Hb#tfj{?~Lsq=<&)`$yXMUAJPGt94$`fT% zcl?aZslQi}A9GF@#B2HeB6=Mpq{QGb{m;G+M|L@bT1Iqxy?1=CCAO8HM=5I=Ue+!d z0BozAe34b7p%)vKIu_kvimMX$r%9P5q*ta2OylA#BACq;Dfnbf;J1Y@aKm~;kD*Nk zWnSU03*gpJ6ws~?G{p?Ex$RAx@s8l;(-YIN!mC#oq;|OP# zBv5Dpn8q0z#gz`5UmdD0clYWKdwum%nb^DS@2U<2Cb4F);o6rIneif*Tb~`ae!jZ7 zm+yXftwj)IezeV3g}vHe6{|;3y-Vc#0kPUE{Te_0IWz!K*>+ zp5cIf1ryASLv_8E$8sy5TVKjo>NELr{ZQVK_vLZVNYHL{Dy-|QzALJaHzfAIKWc61 zroOP9iCu(AqPuUSMYSL7ev|nlwGuu~8A#f~p3(+^B;cA-%L2OmJJ~#f=bY)6$&Cot zITyIKtYcP#lcQxo9iLM>%|O_hjKso*J3-chNASVGkTq={R^!|@IBrm?@u!g{kSq`& z9mLMga+F}AtEDtiAS*I}SW9~*e&M|klM-wOqrH3n!8(sXwr?0RMOrhO2@A(FRiKds zkh7_fKCuNLgIkRWTmre&wkFBlAPaREfGdZLpmt;nK8`1pga$U_5g1ypiKVp`dN)O(j`3w zh5fJ3fA|Uz%a2FMr>vW)D+;CX0Ad2_vy&o3K*ou0}UHxT_yW3x{b4Ni%8m0G)$^SOYlGOa@ia)J6bHaAq1OHiyo{-WwB} z!K>vBwMAixrDqN9Dkb^Z?E<;Xr^%QQZixF0y%dK7vEb4sy{6PUrqheF0c2J(h;2#e zaf38@!pLyD7fVKZ;RWpc)cp*=PGDBMu&@BALl=OC_*_s7d0ES-F-MNo6fsIB7Uvs_ zo&bV`&{KB;0N4UhD~bKqzQy0ng*XOr6~CtkVn|pED$q6C%<*9V>)FV1P_PNPWg*S= zn%ka$EXVFu$2nG%KQj=1=sU`dP+;4eLJ)qQNr`h#fiC?h_w4HqoTlK&S3MK25kZlm zRH5U}f?lqdh@P%q1VGOt6WfZ58&A$(IO*9BvEf?V#kG$Nm~GAh)QN4jtn|fdU$I6B zlvAIk-dV&p>55cP`8mDm1i-`*dF^71^xjm60R#!`ggh=~Py^E_a8DvYTnT1({}S8Of20-PK?YM5@pNf@4_S2d22ZL3`6CoF?Bz|`p0VP0vYytXC zTrnb8Og%@rd6JDS920^lwxI&}V%(g)rE3Rd4%teX5bP!R8z5&b=^Elu;0x3E5mbw@ z#w!5%v6(|e|2qmBSb+rfU2Fzuk4V*Yzx{sOWpfYkML=rf?+o%2@v*U>OY_-}(iAVBSZN6GPEvYo9ZU{2%ZL1YTt$!?d}E~!``g@u7=tB=XMtM^3kPF!Q*24m!&p5Xhwn_!QWfwfAN!-hxlH^Aa0kIB7rZ?6i$rFl4SC8OFZDSN0jiyT5`sVibMy|U% z!Oe}le0ilQJ14HuzT~G+axUi!1?j#2%uQ{UN0)DGeJt|9i);Dd<|x;#r`$T|_)g7M znJ&S}+>MH_^G|cY%A#wi{2uwvxBl9vywDHqc>O@^FaEu+Z@vCs%C6U|?-yh`>WBoI zZSWKo1OX~YX*TFA?D1MYC(q?;(+At@pUIc&cje9XLz&ha_$W^5Z`Cw2f!$XFjs^hB z4P);1mtf{p2-rt`5_XVxzZ{XK#Td8<44N%TeznBJ9WO6E;&3~^r|;^4U267P3>Z$- z&N~>Lt7C+IPmk(s1?f+0DLY70h01gaE4qEK$TE`BpWE z(bAmV(r#Frk2%>_TL{+QW@;68pP&$@u-*HY+ENI1bN|^1aLs0H$$<6rJ!=%f>kh2e z1ZttC1egl5Ij5>%JPcG7|49H)1GdNg_Y(**f#8WC*48a+H;o(A$h26{-c;yUE3t9n z@jSQ+${)+K`o@*a@J2KiUwe$i_f53S7)N|Y#fZvOi8<;nEG4W0La>@mo%GxGF-;jZ z1j*1_iTTey_x|LAHt%cbsrDVPlo}0P%)Gm58QNgc8AdCd)s4RI`2Om;cQBd@S#$Gy z<#!7+ztlLNzRz}^wYR@-mA4j`TdR`@;m)*4SaXS}#h7I==1^147i-JiYVy0c>8P^2 zM}-@(UE%(;to9rQGc&*o{9tu1&0-*5;A=l=c9RNM12wZ3_oE&}SIn*Dc6+`LH{9@a1G2|h6Cb&gZKL`z`Kh1(pXA+d|C3LCu^-&=`hnO#@{Rvib}Vc^IH|-zY>7`w zF0*HoO0%OOrI9&WH+24U;;U~(-q{#5~y08;}K?)_EZze8|W znTIvp0qmKPVW7fCwg2pyCDQX`@L^TF!i`{}zp}JYYJ@`BKpULO1hV%whMvVq9C<@ZNd1tjOyw!{9Wl2 ztAH2*WIcbZ5)71$=5ruaJ(Xv6ZUtjdsNmWARs%jHZm0aCiZ9lIyxT~|)w6M-PH=syV1O6JDF=Jw>ukrTz#S^)o zFXg?fMc#XHE4Rn350*Z@79iH@Hujb7BGab?vF+rY?NPoa-}&}y#m2r~Aoc+k)($c- z#V3)ekXS5*+jvFF7W17lO|r-C-OHi760{fcPB9SsrNc{kvi&`;x9T@bVPy;`&E#Rq z@lCAKWF0^z(p}BI5ekI630M(EyjN6nnvrX?N4(zO-EKhkR_^Z26hjc(zn24;x-jTb zpKjvT1sn_ZA%8;m!K%}#A7GxVjV8ljpIeg%Eb_0Z-o4Ao7n*avzy7QV#I)G_ZF4oo z!hKUkQp9KY9`z?Zl+{g179|V+;v$iqKxo{gJBH_2FmRkU_NknLFb}XwFJrdM(O!X9 z8cRmmIs@ZAx8_=6zPpnGare#AXPW>}*`E+K&GAT#p9oBo)@CUIM#(sF4OriZUm-T> z*dRw17VE2>n%c&e6#viLyED&ac669yJHROfUU2it=LH*FlY2?DDFQ9Pa&qH=X5|cM z0`7UP>k}kb0-raYt}cRPd}_O8BUr`(j90p$>=+{l90B+;=yn$s1)l|qMCv0>kdn5c z5+Fy?&j6OwiP^MZ4m$Yen&LPN2l|hx?~q${I0N6K@=6)h81w}2$EtpIy>+Mrwjza~ zn(UXOf>;fl()XFsg<}{%j{{lO;$uQz*Bnf?!d4!N46cd3c0*~Si%B(BwwMCM><8{{ zu>F49+o}5lWo=R}T+u4XS^}V{;~#4;r`&UPuap6J<#R(ZhklPf z1KtjkY0_f6DMLNhV3>?G=3!QtT9$WMBfKUQDOdin?+p#{o@Gx*u5sGGuu{ZRZ5DI0 z(=>@bP)O{#VT|M524b(Tm5J5Y2539ztzLODf{9teoEa*(cPfj2`9#XaWBKrAmG__B zZ0l#zECsdav@g|)D_0|rIzxLFQhn5M{wqHwKmB^Kv9A}1eSn44Lxn^#Tl%JKzmwr~ zJF}nLHjQSHX#=W{*K7GgeJNj=p6l!L@=PAJmol|`V@;1HoO&QvuNB>v2n4YlwM>a{3p%2TO}B-;w)cb^ez392_A*ahs!*B<{^z!V$9TZ9RWk`KblfV1ygF* z>wbgGAuEY(IZsVypLqj}6+<`;y9!g1G{~M^KW+cwm@!V9LHT=8l#%SX{#n z_6-&s(3b+(Rl(~Ka&JZU17$#uw301R6>Ckbofr$bw9NLha#KUfEP|HR*9@zCMegg0 zys${MMc0OgfUoXSa9q54Vj}OX&*-*f+6m;qA6$be>>$9GkvllSxAoixSumH2&^<_R-fgZKi(c)SYsI4sC z13&BDck@M0WiK~(tGxIAmE0{Iyk1DpZog>LSCj&db@DrK^hbYNzVq$ZCR4%d17csv z!ivN);}IHp2xpi6Lebya8f8^Oh< z^)h17@7=~C1kCNXXJXlI`pSB`8*T(cRs)n}T$XrN9Na_%8$xE641_c}HB(w!y@VrK zErKr{QfqTff`WiJG7qe%yjIczHzsXXa25iWDMzLR0aPLAY<7-36>}>vWj&eycghK? zbO`~-+33|6_h@_aYb@eSU$7ix;3QQnb7*to0^)+78xH96J*Yex1A^)dutX9d$mfk9 z^xScJzp;%z*LK&+bA@0n_n}7<`x+YWL(%;w)IU)TCB}me}{Ceo0{oEPGl;0iBvke^F;@Xt36*4- z(|TI%Ui69WWoREgeq`U7;iHW;$;<=<`dr%tynVcnl?^B(&KhIBcbCDQ6ZY!LTnTP^>DikbJ%8T7{S0?C&rMEu z0;JuV0p<&`HuP*c7l%h0CG!9LfA7i7eU+Q_Aj@>A3Rz*^+7s;beIGwAfAHu2n*4)b zegAjzqM!D7eL(E1SXez2j>ca)!D(8S7$^jbP%!tx2AAzxo-EJgo$^AyJU!n4><998 z`%oUY=i6s@(i75XIu%rmYMY>HN8}lVf1@w1I&PTalNmuOV&;8aCVf(ab?nQZn@cFc_Wl%VROU|kM z-r8cWX)o0}{C##crq3!8lSi-?8OgCgK#rC#!nIUSUakZf;uNx4vEwEtvesZ9hl8fl zc&1vbx65B}W67JRwykla1L$<(#eOGS8v#_j1`8v^Gq>GF*>YQ<&@}w+V^Vp4o3~WT z*_U4a6l2_cF68S*BtQEIs+{!=0_Ii(6&9g#D+y?_9|Krq8DXsCqHGfVA=M`m)~uuE zy}mafs__8;hez{))pUZIzL4KHR{~>PlSSxg-_fisbop{Jlv!XboJl|Lzt!UGtB|Qg zD84B|U|lJXih)=y+bbr17D=Gn75h#wM8D`%fVo#0*}^=g*%gUakF3&QEsCTvxNZ2% zTvWnsq=`64qU#vrAjli4OcEp`D*;|93o=O5ZW79CTQ{bhgo&TBRlhGLzYTDci8Zj* zt4MZ&MJ-=-Bo~)6G9aKHCO~Y{a^lm85~n+Vqw7nok#G|UKp8(zL98qF zR3Hlif5nKO%tE`U5d;RKmvV6DP$*}Ly-4xHx}aL5z@o6o-Tj^1Uf;^~%WDnpy1l*C zSMT^s^r5>FXkb?a*-h9-)FlW$%ZEcz_O_K-o?Wl<>_+7I{VYqp)Jk5}!F;g)-_Mbx zOTXaf0uhb9a?e+Z))NUYFu4R@5f&fAb=6r!bG+b-~fWjm;&B8rJ(l7Bn36`|uGBpVx zBLM|m|NQRuPPMaMey&&S%H|q6#di>UadD}dZ1+LZOJfmtCf9_@iVY|>8fS~9Qzn3d zf+b)VW*ErltZ|J24C#HMG77<}3RYB2VN!823>h>r$+!aTCK3TdV!;v!|7F*5L!XKT z>6rjRuo{D_O5|E)(60HC(%e+sL^Gu<4SUc=@MJkUizVUaCXzsv0w{vS1cWMXK2?G` zaRJHaaQ0frGK=54ff4gaT7qBQ6Ch}H18u_eGCtPnnZ`6xJnt053R_tC(>APE9l;`I zmec;;vpqb2xGy)78sqNggN1GT?}!X@dQc{lVx_<_7NO4(mdVV_t=`xBs`%wHed_1d zG#+H@rx66#w8{E-4j?u@v-eXV7nJ~hhjWGOY-*ZiUML9Ys)LeAu)7b$;m>S_&yb1$ z$ymV^tSA`GMqkr#Gdhwrb@!*E`n0O~{}=$))BdH3U&gGOAM6qG=ZLg|;fqi9_2{k! zWNpn_WIm~6F`cDtO^E3&EmbJQ2=UV27O)FpiU**@*ce0G{9eZuZGfMzDL0|C>7UpV zR*aj;f_az|SvKdnuxGK!z{*Y$Ww8kjfa7Wz7MNZn`s#-}vHi1C~cFl>& z%@#__QeXtjp%X0^GpvKf!6*rbZUM9X%dCpCrVnenWg9?m2nOBB4Pl__xV{|pyNqQ< zBKBzPp_wpXK&^lf85Ik_RrX&U*I;f}#k%7X!{pw01PAN2jOyAs^q+^Qe!&GnT6 z*zQKyFWd9z&Z&(21hU1!EHYXMSm|DVeInec2RW8Y`QG!Rymz(AU3V!cmY9* z-DQ4C{@~BQ7HsV60b>8?FaH<)-&dcMO?~ingeDE7DbdXaa(nT-%XTAgE+5D{@>IS& zy_7GO59O`(J$bUe*gm^g;A44VtEEdxKMzy;akKvZ_bqcAd zUU!X>z7tGIn)a+NCQRw0Ix~Gsb=HBwS`*B8bOG_`^Tep`Bj~knJkmP=3v=`tD{IN< zo21}ArrN?LLd{vT4UiJ9b&-M&|L8Fu%k5HF=g1kaBI=&0@Ok&Rem@OnFT&Fkbvy{gGjk^Qe7a zx6LgCmXC9HKY)@nVD@)*De7}{N|EL}x1#)>w<)(5GHWfl56F@g8+U4C-Un0w$ZE%R z9cb8tZEGgnnfRiKyp<{1;1v)WEIX@4=kuNfgT3F)?7v#j)?y&Do7t6d?SF5d-))<7 z|NCBlzgHQ+z#q#*5bgoTg})o~%fM32R)B!2=I#e~Df+z*GKX#mEq84PphvS(^%Qyi z$@Tl_WBW`%20)RoF~1gJaF-}!+}X7LvxC>Vfeb1HbP=+>VD*qvgH~kBgGk`qAOL3{ z@7*_ahr{^$oL%Xodrb2pWV-@JSPfU5Pqvoo<jit${hMkSRSb1dn4NwT3*^xw@?B zI+pjmyAdeaa+nccfH_={0S5E2gIG=C-_M)j1~wr@RuO04DIX@}5@&GJo}{5`n_J=p zdlU9<`q|QISD7=Q5!}VT!}7#Zf=6}$4Q87#q^FToa6e7L(zF(HQNj(d*yn2HJl<>I z_WcH2Z?3Q9<%<^^pzZtpz24)HKrET<4q{zN1}nQd+_8N_ccS@`+_hQWd%5WArU!X# ze+y(tC?%f`&1SX2udxZ9?YL;~e*15Is*C;(9zT2#`wRcq@7;Rno7-=PSD!e7Sf|Ph zhLTCJ`)Z07ym@?2-dvx`qjD#;Q~tF>-C1F=E(n;c z?z%Qb0g`-)*oI||oPF;6Pya70EM*URlI-pta9mV~^#!XN$zo>rF2Fi??fK$t6KKwU zb~AzoAzfa`;|;_f)ZYk|x@u+#EbxIsj+-mBP>+U{n1osPq>t0;b;w1iqf0U0U4n1! z4y_TmO0ZxA?M&^m<|<_d%H2m8AeTAI^7=IyBQY5^SRD-(0W&W!3b6V;3ojGDCo z3cxh;vk5h+C--437@=`#XYVr5qv9Oz=-sU}JX zvz6{i*qD4%68s)Mq#h7eQvi;G%yv763?Y z0{e5h$v-F$*zW-dxkj}B2R`SfKz2j7`vDL}bic1}zbVF%KWiQPxI~~Q+fh&< zWxA8wZIdQjnpsn0NX6U5G+Sz(KUx(mwFWJbUwTdM1Pj~_2#Ou>uBlxK#2PMH-I}l> zwU+X*04n;dkk!Vp_G=1)!03`tJcz5`tLKTmJKj0C)@%j~(;(V7uO37uX`N-IF^<6k zR3a7Luof^pnZb6S;8NU$g7dl7xNn}bzE)~tjg$3;%%#0&GFFX`l(UtAs3EJkz6fG% zorTMTh$+>Rm(Q(WVgc9?{8cAQgK2I}oV$^tDzm8s{&bt;p4Es4r1`x?A~wUiGq0Iw zqpyG#Of8m}xcgTcLw)Y-hD1izb6(qsaNlLNBTKJ!tsu4sbajQSD}enz-0NA%lT571 zuY)i%jRkb|`>vWoa_SInJMVr2tuOD3JiQfp`tn}x%$Tg-%?i$^IUa<*rTnD);a~U_ z`TO6#`Mz%OeI7r25c@NK=0Dqd>0kfI^IX_+;t}P*XOX^m3Tf{*5PK&V8;E_fbRXFF z<;(S{ytRHPkM7^wfb9#pC`{y!LZ)kjNo;lhQnPb#VMQBE=*09lpBI7uSo0+R3s{6j z>lBKcQ-nTu%|Ptc^^M$IT`7Y>vwGK330S+I(Z%H>GuNm9s1P_ynlMLZoTGe z{z=-uCnl|mA`-nVTd$*{Nk#o_x@a(Zv&`7kA%HuWWld^(pMqPU1Oa52basKyOa+!7 z)|ddYU!KgIEWNXF$X+pr(4`kF)B?}*N|0e77OZUoyu8oEsyrOm2D2M{s`dKVANOF} zfl3iKD`vBIfiD_hHResQv3${|&#p%SKKVw>b+HGp!@bJBJ^4Oy46GzqeH%5Mgo$u- z_G@+P!|99B|6qFkIwd`7Q>%=DNd{}c9jB%ZHTB;~%A`FgFCjA=X!1O=a|&whdj5m} zsQjG$&=R&x`@-bNZBae|!d*uAVu-m{UywsClgKz_b!r-ntus zF-{i1($9&YExG`ZeKr68yZfW(q*j@vLABz900Axk?%4s<AKh#WPTTSU056p6|mJPgg_9h*Rnj`78GVcSO9KdmvTAjbY_WlwFjy-t%to& ztgZ1HSqI=oKsTHI`Ei#$A_J?uXPOekiU(j^r2(8)03Cq73RbZe2e6YX!;n{{0N{o% zXPD0;V0cb00R29fJw46UCZJNDmqxWdwIb_~)ucOvDKhvsNO;pf$^4tjdxBSal`PLI*E3Gr>_=Shx)K zz^?fs0PH+_42^+U^NptKoMs+iJ4(-}zI2IA*}hx$Osg}N zbFvj!y!`iAF7$1X6)YB45*WagN(eRJYZ5MLYckSxslh@1EeHywd=xtU3aTdc8*f@)*NJW?(;l)oR61& z!tt=-)^QWGYYiq7z%iM7$QZf=r$jU)^0H*n8LyWFjpOQSvV zCuM5$e9indrO~K)^T#ghVqepKt49KMBqWLp1dos?-%*@`_DG#aC2~0 zuN^DZ&b-Cr(|w}-VfjOU@jsXM{;Ts_f5B(;`0WL;|J=Xzm$yOqQ=jh5yb-1LC%s#0 z_l%7>%h6a^l{ep8cdoB(AolaihjMj(F4O%Bna3TxPp!9l~lg#klnh~0;Y)A)~ z11k`Q=in2nNf=rtFmPh0BbSVnf>KS+QpP{U!y>wY|j z@&UOcaAXhTZDl|BZlF05hHVgp)TE63BxO;%F0 z4}&wtC~pvRESL=CMkr#RZZxN&1Kk$Cdh|0IP1x^fJQ1LZA>QsiOQ?U9;;HX6vG_At z+>Sb))>+M7E8vL02Eb`5S?Oe1Gh|CmttW7Kt6_=Fpb53h2@Kh_EE5@8$uS+tieP|z zd_~v>vy<@+? z#>Afu>^NC&spDKg;Qo4YUS-1$9S0}F6@YV=>H}C?qtun`!srB+Zo`mpMQ~_<*1;RuRoA7S zid?{1&-?B33K|{k-r7%U^eIX3#M#0QN92czO=QVnqs|MGO_6j3rr5}X;ZlT!+S)V0 z_{q;{apsgEavh)r+Fj~u-A*=0C47v7{geiucJCSR zQ0TPQ&R)e3?2hS6EF)rE-C0JhVGLUA*)yaQD>STt9erE6-kT_rD(#AHC({Q-0z` z+WKqq|9s<@-ueqZqsMO_i2bYo>nzW&|GVwCcR%%K8jB8N$=XJ(ed=9XV^V@~HYveX zJIT#7`(TW~?7$YeRWQN;`Gw(Ru~jp@WHMIoF6QUgE<#28S%5F^wMj8% zgp~JTvmM~*WXmda-x~tJ+V}{=QLQl~HN~dUc|K+AJ(M>BG=`uUzgw~Ok@qXU_YpA4 zOnvdYl`KX9SeMKW0VFq`Oaj|yTJt*}l^O&HGUF(06N!T&hYmJaIu7q^Xn!{zkpfY$s$%oNoOqRp)c_7XS)g+!@rr&k_Hr`dn! zPdX)tB$ntX8P{qX=n&@jQ45ZH6dq=imw~nd}E@fj&aq4?%4@@#>Qk*lQ6EA&j zoNmS0YzMHcwM){3k-%tW4_4d=tG0N4h((|drq9*$IH?|H1GD|=UHOgoU(1Ithukd( z;G?}B`_nw$0`~p^`O2UD1^FkxxPI@qe)f;wJ`no{{_THl`|qEByASE=T0_Iv0P33b zSm=7IOba_+%a3gU_Dk}Se0lvXd2;$t`g*%PPun3@FM+R}Mz))8%)sWc%Q)LXR(ih# zWHFb0#{UgXtNC1FCizu@JqINT;2P{zCQIm zxFQQ8loPaU8|&}G_+y{p)x3Sz@ ztOJDTngj#$5RfJ~rkA0Kp!_rjCEK zJdc4nGgyg2aI? zlmNO5P^utSL-ZQ0vW@sI|8}Q^+rhqi^#V!-ssl`Zk9NzWK|26joJ+Gqts@+JiiB5N z)5{W-4zdnyN(GcTUd*PhHaG0Os%ORPtcktSR*`wr`LVilq^s(@Ku%zFK2J6-dX1*r zT~h{OAd~mWz=ntO=?JH!B}33NoZa{Y&g+pC5CYt(9WK#_oGaW@R@llBNrCIiYibWZH?T+edvqR3jGx)eGKfC*I=sU~&dlDbopy%yTg&D;qPN-pCi$m-18nH|1-~Kb0@8 zAIS0iQpRS0jd2}KvIeKl)U##y-?J|^H&HCU2~`CqYJkjQ6&)yQAbOdS$vmTJcEHMw zQH{r8zOv0ynODtFBx67?dPY+NTO8`)Ss{Mc$5jf3ve;-rz!z)P*uFa|OScEBb#FYVZ6?c-qdmq?hgE)xqrd_;*8XKL2?zqinkIz1 zs2RsK;@CcrJ+8s>Jvzhh)ROL>(afWOO}?&%-tGa5vg0SqWC@+k!uSyUrF3M^R;rOc z^c<4_#26E{nI>E35FXn2!EmX!2kxd$q2^NiGtfiS;<+p{^T2Aoy1 zx`Avy6>T<)rYWF*vz_}J1L>~$JsAqModrOUI!FVEi(i9Qs!@W%fc)K^F>B|OC$xH5 zD45gc_8@ASH>|5jBpvAhhC|#vq0DSBHM*nZ01#|BldR+8-LCdbY&cc0EHb3L=CZ5V z%32*l+7wBJ;?s477=j>fuxSK<;0jrXt*<`edQbv!ds+9cdEV>LxgRY<;<}C~6RGdL z+CHOWV@}vOLjr*Y8d_4Zm84lIj(2e)S$RSqk*F8d3s8|_m0bH$_RlpVt@+GIfY|`N zf7eZs!`Antz&g&T%ENOS=J>sS{YqZGd?_zpyg+4wlMTA%)1j-x&hWzD-dkoecWvDlWlCbC19G^XRlXz{^l&V+j^h0i`ThZ$-U{Y z{l2})P1tr1SD&?xE11B?iBu3+aWEe|n*7cU#QyD%f5Q*&@!JDpf94!ua<7fo?O% zYAuBLgzWdM)#mIC>$2(u_2vX4GNKuJw*JuclZK z)bueu4S$4-YJqP_7V0q;&CZ+hfif9`rINitB%7uN^U`qeoG-OSq1Tf|>I!fG(BnoL z`>dW|NfYUwq068an0%6J@Md@ ziHOY=I|kWV&L7{1iZjM*DCZD~Ddm{Pdbb`4TYRrvobLnXnmXMwQgIz{h&e8d#E6jH z6N!4gy@#_V0B}0C>ki0Ex&BtHI*?MJxTPp>&F3HB)GC;|$lqn1VsbPs?svo{b zHH9;(*sNGS%V2W@BzOtf^-_wCrL7{`0G5r-QoIdrk_=JnR}-$8QRZf7OQx^>=oi81&dkDrpC{E9xPsWqxYkf>U36SrJpT*_rgUMK z9f~oDt%Sx<|8DmNTNaEgP!*Cl(m;9l7cc7HT53P_^fT*!l&n{>n+^m#gO+JZ5+Ubg zC$cDQW3Ss1p5?t|n4Y^xmt+WqfF9>IIAicoX3=auS`B^o!jvQMIaa~&kBb{~6@`6CN zwxn~)>`6-14wx+)p75m6 zsO9X&9(!}loWKRcUwTh!J7tXcmTOl`y8n+2*PEl^Fg=0FpOCI$8 zScL+BSZmhe=0g_Blp7iJ7FuT7wT?(nO*ZV5CSf{fl#DwNnZ_h7Y@UKxxdax$a?@!e zaGp3PvDkU*y4!^gmUMSKGcwH57n)6Xx!_#beJ^W-$y#o!K`N}!FI%IU4mm+dT%J)o zTt@1G3pvB+1y{PBi#cqkQzZXYU>eEGh;lBMKT&z(1cxol#huKFyQj3KL`W+zPVM!1 z_ixvoLtB1I*`EGo64Km4g}SpG}-K9t7JaCx_+Sm^kV0*8KnCGU<`^5y9x`Pv3x zzjAnK?Pd?_y|gv8VX6ZSo&-X1_RppyyHk(YQdF)FT{s$6J7W>C*maLgO$SyQwqdy} zE^LM9sW>jSq6w5^(KPX;soMkVy+P!*F?4&afx+TfQQW@dd;#o>`ihm z^WjE#my8{+t_ofG<3kQ)*vNQTj?dXnBb|dw;FROa@oL!L%~3 zS&LOSW`+^W1rprgb-g*DF*eu9wH z6dNE_AiCz?IBIT}Xd?#+JHM>X+=VHQ7a(?W$D{Yu%@dr*TAf?i`j82rIswZ&?#Y$} zQruGvc%DyD3X+l01#`Y8EuUlE;NF+Aac3qto=)ZU@}3jHF6s?0S+!$blWkp4s%}AL zpYti#j(%PVE0q)Ip07_5X}QN{X~uM(*&dzkW_>PsAIM=`tV$=BK6|dWCVR=r*jSyD z_gbakELN%D{g?Oh;?2pVRo2H;<&?UukEPuAx+icCd+Wzf%Rm3;zbU``&FA0Ghri#) zj~>MS(VzWyw}JYX-uiRM47PfZU}0^Ea$(6L$MbXf!t_FZas#kGd3-Kk>@TfjS6}YT zUFIYkOz?rH#||=V7n^KUPA-NkTo-nS>H&90BLw)_R&{G(<(H;S~Exc2^LTq z^sTIa*i$b`Ff&9#A8A9#O%n_NnXX3mv3V#jO7yVY7ppa_2ovQs^}rn?bj7TGy@Kg7 zCUi(FVK=~jP1Mafh+R-2!Zi`jngF0m8t<_Zys@n_qf!2WfGL46l0aNi_~4VdOf3+ z>Y3Troarz^;{mfBa>_<3c&>v8z{^_9QQs1Thbwa;;kt1IhlNZ@=x z$s)I3BeaXM1Ea2Wwc4>r&^@hS*kW&)jCHk!3dVX5V`L+0H&_BolEG!j13-2{y=f*W zGU>{I-gFURfPLbcvwHyY!|tnOt^*^3m1yB>tlZ$AN3>8zNTXc(MkfnjuVeuuTkv=y z>20JRI3s~;FbR!7c18(EtbaGJw83kMrses18*tx6*a&0Ks-`_RV5uxDx~AFn!JY+c zx@Hp#Wk!>QJtd$eASEli1GkXhtV`PaLcnlbqsdSql*sI#QDZq1x?TgwzP3gOODR!i zHDR?r?2&ED3Gh9#!33|+=)>)`B*F~~t(aT0GO^p-Jxs$pp(*RC>*eZ+{MvV4$p_Ex zw)1YTrx*W>Z@R!QtM9o&x`0mN+i{J0@OydhoBzw(eCP*#{AfY!&;HVrZFK&we46G4 zmk(^3eN6m^g?gmBS9#nv`gfM+@|EeieC_y=e0lm%o-VItKHW+;*3`Wy{Ykw~dZy`* zXsmaNx}V9wa3Q2F%#HoJrV$Ac(0k~Ds&+;nWMVbEpbLOu`vcgAtE;1jSm4_4Gf$W3 zJaegqtqklX(5(Ni;gtZj%en)^eC!zLi{phRws>p-+=PIq1x8)LcAHV>wGG||Up!=D zsfV?V(|`C_3c+nM4AHKT2@x#w#h<}EjYvczUrZ32RT7fI9Xyw&z$*wDlVq!301)-}(UtXHys)W??fa8YT%|Av1LiEQjaLqunC zbzS$04F39)PMU$%dz8{5bBaV3o9Sz1f%iIJI-15xq8t^wb1g{fF-^E`sE4#?(Yl#A z0|rhNjNVvhK?9%wp$aNl%Bhoeosla5a68Z5iu8y&+Q*}pU-AxzM+35+jgRD1aXqzu z@8Rem*jd``^V_lYNpS1TEtY`pv117)%(`6x2rY0b&$O4+ns@JtrJwfPiOq39SunS0 zf--Lf#I_dmx_S^d6>?5fo;nV0j`nRVVj+;=*OMZ&_0 zBC)0jM#27-Gl}`lAqfx*`{ntLUUxR6T;fh%NV7zWJdPlCANkqsTgwQx8|JjMGUl}* zL6S+T1JaHGUGtK}gXxx;)C;1*>Lf!OUH8Y>=SYMQFOc{d5?5KznRYqa9V_W|x;GY9 zD-u{!qubjUbKU0;hY8)fa(&G+WrznzaOOymn}mC#mlho+;qjL2ZJbRLtBk9zF}*YQ z8(w<da0k!j0f-f6D0UpeAY)v@ zmT?aL9^2FgMgJ0P9~ZWjN@*jawwCK1@{Ya1Bf4S8tA1xAxZ{Y9loUvK_?68Tyt8g7 zBi?(46G0DO3!nRDb_Q#L)}71ALX)0g<4oBJwJDd~^Bx)EUi!xqbNuHDW|l?jhqdK| z!CgGt+8L#+lew(aED~5(Z33um)3#>!R(YnKkPnlnZCw)wfKz0-}cue6{Pm8s^Gold3}LiWv%s( z!$;aXZ!wmLCU7o#v?3!DRd!?ICCF;dW#f#lcxwXK1$w>#MmCxIb9BI3;Us}X%efG2 z^B&koOI2fF?*kyDlPg9PbU5V`DzrGG>Km+u`3+3eU8xmF;%0nzK5YkN8i>z^~G zRDmXdrvY97UA2MA++JTh06uK*)%WWLZa=s06M!PMOGlF%Z0E9lUYTD#rd4n;FgzQO zy;8RK%EQ}~h}js4GpEAn0wP1_C~@WU(+gdoyJGVsPHh%u+G5BK&SujgHxkr)9XZDd z`W=LnwTP^H=z^R*<9wp651X5^0?QFA4w}u~PRGdKcfJH;!x`Ax%G~!X39`hNh1BG^ z9#NZ&=n$+TDQ)r2WtPie;5Qvx*6fZ7A4pQJT6Hpx9XTX5S&fdOV2!$OheTaOJ-u7l z6j8iz)agX)J)a1ir!tSJHJ!pCs5vNXJisJ0cdm{eVbHn0&F`zjF_N%3W_&NlNzYa7 z{_XASXaJ)7LFkA}ql2(TOv1xCQbyUF)G++LkU_a}2eF;r%kn z^V1~XeWoCKyI<{S_3_z2#JzMTbUD5n%)N;s!*2y*Z)7_Entba=*Zkl|2x7Mp`TDp2 z&i2cf-}oKWEE8>vxaWTBXB$?GM22d zn2AR8s9j362D{C040Twzp(6C0hnH~G2TSqrd%N&j`8N+)1u!Mf;Cry%tm;HZ*jbVh zh<1R|0(7omH++UAOGv;%Fusg*2n>{NPfLU`Lw-W&?(E&qLOZ2k_ukrM>hA-QfPCE( z6qpR)q2QDEua9m>&iI}gCX0}=ClV}Y-nS|zNWEu$L{iC=Ve>jajBOHb1U{hAEP`3o zNLX=}ZeXKsasPY{_HZ`IzV>1W9SA~lAY^Uf_|=3xmc7FJ8x3kJjIxN3=OPdQZrj*5M`PU9DBulcc}@Oiq@p+WexaXCasd|lxk=xK-XSbTa-4kxw9@_ z=4iNF#Y<-`F+ZqVT@ki42kM-e*5;&nn!JGkgU_*k@_Tu%=8!Q7g(25-I=4EG6P$%8 zX$NOp%LUE|{VVvqf|S5E`x~;bhtRXmsF6LR^q@%!EcAbn8w0)$6aqLL{BO3=TL$|S z(iD3Q$5f{FS`)|mjFL@P&R!k-JT(ix+h*SY^2BXASZ0f)A!6i>4bC=Bi}+rn_j7?p zb}`nqZEK$iVZ*e?G(amNN}+RZTsJ+szQx+)oIzp?K#m2~xoU;6AaKwuHxHQGW-0l>Iok z?f`S6I1+YVxW)xCj2UypnEk^XYp~ijIuD+gawxx36651w&X~SVB%xi&k{ zaZyR4HCNL51zzT_Z;oZZ{(^k?{8qm6-H$dQX;g4EWm!4;Z=4EYg>00)Iii4 ziqxEOahX?)1THCzEK3sUg9gc2j$}Xdea1Q*@*GySY%L6GhPcPYu* z5gAOzl1S)c38M*PZ`x^qn+$V}kP!rHtY5OOp=Zl%{|sM}1y|t85N&|J+x?h9l-wLIUMzRBICCv^w=1uQk_KGFpy!iUoFI31 zeix4jL9U!Z5k9ErLFdf>3~h8h9De8D^s!=2MJraWdq@I|fiH7_wT|O81F~3`)DVg@ zh-+ErH`5A`dedpylqje>ayE22hA^Sd{C0-eyR_7;+-fzNf;aYZRW zTbtYKC$!^r4doi_>$0Yn-l+hVU-JlGWn$xZ%sH;Hu*=};rL1XlQuVwjC+KcR>m1w3 z!Wbn3@TB~sN@e&+y(-2y>?QURDXc7)#k%g+v^VIfO+x^J!7@%@PjycC)9c;N*7)Gc zSa^L+8WQ!6$1F`b9|8m`+kc~u(~Yz6SB2fb0_Zfys(?@XJ)YbA)VXWS|FyIK`jhW1 zfK^|+!SQbJv?#+d-xp=Wq7sqYKrN+Ns}tBuDjxA2r&17OAg5=ctjW9SvRJ=Xg)z~J#H=voWNgVk1%2ylFC4Ow=N8B4IX>0pt?}{>YPQoZ~yh6zIP{OI#2;nOj zxUS&fN&3V=uGkefVF07N0xR0wRSA%U#Z(C!{z7SO{-0f zsAX`Ir8A8$Dk?xG(Wn!YhT0---*q6UqcqvT+K~L?_rl6@4JLR&#<4<@-(+8zplVH9 zg6G!A;tJ;b8b7OJ8tb6906cDs)4m3Co8}n}B5>VPEjj0lv6I0p;eK~`L&1xYVQ32f z9<@}XJ4#a|CZ$8tSwUn%BHr;}4F18i&Ii+lUSHjqBt^SucF0s*TkT+!8c}tWo;-PC z&v&1`Bkw$UDmPExm7DG76Mb&4J>8y9-g$TX_xAU9KlkwYj{e;K{`ATAx$Rgt`g|(a z+wTT!w}0z*-dL33i68sg-}7`krbZ5KxCXAE;XceUpJycNd3U>e1F+hzcv(+2H~d~C zF%wCo513fWEv6t~C@T;+N8a*w0(_ldgI#7}m_n-;Y-9KOego6DO+I{iFR#z4^KRbB z3wO4!4K}%pcfWZESlz%wip39HRGx>?>1Nq|kNUoO`(w6(sFKR!zAyorn2esd4N(gN7AzM_E zdGngCZ{*d>mjPmHn10p_UG_aDh%F^X>LdDugY__N7fNaoEc)>ugRS5=!63#aS^zA6 zzXY)S+vV#O#PYK{Z@e%ARbOl8GDa9#-Zui6!O8+P10J#23G7^}VDT{!Ye`*efY^`| z^mYLRTqi^$Y#|l<$q)(t=ah_tPDjoFBmgI#N*?FLV@&Mj!(Z#dYVdwKyxzP%!Sq)dhGjg@9%Je=af4;B=u=UNH9cXG=SuI_luWgv}1hG>inqe7UTlk>AUT8+?Dr zU-;E<)^=n_nKBN9iJ}3nSCS%G4TV}&HW#Vo?d*5_q;B*Inr8>Lvs^#jK;#B0Cy+`p z-!>qrbL{|DUs>39-gzgU#*S_v(>2&&OE)m;KW{>^64Dmks}UX0N*VTgi3HSndYn@i zOXipjjE>NG&xI*X35E|BHLy(Ra+~c8zt+>ObG>R`Q_9EpcuRBde2F{+y(N&$={-JZ zlB*F9b2vw2Y%K~i9gT(k)!%$BFK)Lv+>dDH14oATt})-}fysJ15G(Qr zfA@!;;Ps;cvH!wf`nhcg{*_PlUJg_pFl@SgBuLW7w1L>C>l^vP24cTDeJEd&_bs98 zdfaUwX0`f$s$CqzfgyTRWNEzZ_rq0Yv3h9H^*$_j#8z?o^9^(@dLHbWGDDjag{z^j zGwpoE@~#rpNOA zTA7@{QbDmP1eqn#7g~6(UVFEJ4Zu20?GL7cdHArZEGadLZTHeutQm?fO<)s(9vS{w z>T@YW;MnrXk}B)U5?JQ|v6fUL9-bVivuFB6aj;3PA-i z>ykj2fSHVpZ5-%oCSW@qd;nlc-oKp_Ba+%=ID-MCqTa0})&W)&VSCz#KrH5qz zJ3zMmisM1_i9{x7;MOWXK)Y4b!4HI-yM;svO07b`_WP5k&UPt4P!@{}>y%Nb)mFF>5X6o}K)>!X)j^1_9 zukSFkNdFv?3X|*sP%22(pC&VzuQovXt~~wxyI!$Enc5AMDhPdg<4oxRbMfZpsl5B{ zJKOVJV`Hz8Ab6kz;My5n-Pf(=I`(L4sHd5U++?PcrOsS|o`AeW1!g9Nb=O%ZRH=@* zL|HL*yLGizT&&ZUoa8m>`TLyQmM=0j!a&0mpG_FysZiu1hkXoor)6fV=ZM zdxUK!RHvw(butgohV}h;1iLjxXd4I0yR3L$d}qu-$!nc}BhduUlQJ)0PeqBAzd*E26&5wG{4)ToI)PyK7QwC5i zY3s^Es0tcxs3E*^HtkrFS_~u*9Mq)KZj}QvWSxmC-a9^%0u$7djqN*dWXxCH)PWXT z@nEPK_j;c0q}jAF9Vy=5GfFTpzg?#;E-hHy(OR*F$i%91%etZ#g*lI%gvFD^;Bdky z)%LU0I#OCdhbwGiwJDenR$>p1w{-<$bV#Jg*eEY>M}+{FfUSX<)49lYyP1&bk}r<8 zuK|LZFd&}HJlv5Qc+JUx9{U;hE|!L3={=MFP!nrlblZ35qm6^kxhL;96H6xaW}9D} zdsZVjlEIFlO^v-9^VLgLq9a_tl%o)wRzbGdo8~v{+q|oBMEK1*LUU?T<{($G|UNkI-x@ zAOHZ$Uh9563s}XiUbD!oPA?;z2ycqFel=nmW}hB(`|P0Z)Qaz+s6oSH~Ue>a}C(5T$+Tbv#>f;+iL7vJy#X zi`8GFGbWjoDRegR*)>Q2mRN#@dEk@Xoxg#mB&`vwah}a!U+GN183tHL2RWTW8|Ohy zF?>yqb?-r7P@_zcv`cg>G77s6alqRMf`^}TNRyT|z&h7W)D7Q$KcHlm-WNTPGtpKHLnmfn?_dW-o%6{tasagQ9Ib`2hRsz=TkaIJpU%#uO#eTOnMS6 zav`&-PgHQ33GT~#rpBb z7_e&-$rm~vc)HV~=d~D+olOsZCAZ^B&WF#*x1Zh1uYLEK!P^0QEwQh6ul&TU>K_Ee zO8ct(p+EOO$a}x|?YI2kTRndGAhsrSeXlWy4S`R1OYS2$b*CX7(Ryh2^3Hf6KQ8af z*V^`+p2_FND>*Lr1^_hByO;*m7w$yR(8>fstHPxA1?6l!;l{EGt=q;61F-Bma-$3G z`WTfBxOp1*5oj^3%$EPjsuG|}%osOTGlg&`aDT8F4F>FoS5vZ&cHCF}O3!!v{3(#&XM zSccz+i5a;Z6Ku*eStyZ^J#?Yx??!QI)c!Rupbmu2#0>T$kGhxGB(dAz^^U*w95*;k zd~_jj!2VG)vwIz$!v{+#{RR{Su%k9iG&q4rkS$MthuBN>s}xQ-2v(E{hDT@FLXZt4znt}N`TAcFu_S+DEN)Mn_G z+1Ox#Y(p2{_`0J=ms9|GwV=gu-!Qew_;@%Ib|_|t0Fe4WC~9fA4$6mE-`MwWw?Q2d zwNa0Y=ZYG;Hd^eK7hU!I8y0{{YtS*Am9)(sU&Q7JjIdzBh10q^b?ak0HMB&|u3heo zvhML3j&;N-5H_gwW=R^D@B=oTHpBKXWsk>*3KMkVIKH;dIA<2>>BaX$+(R!*==0d` z*l-CS=4lcdne<8t9wRyo(X-T)+VH8(a6+cgr82?K&To6&bgb7mUKiN(s8Vs{}T=eA-@Qt(`j>8-PqvNdx9*uaXto_2wlb zt-*nGM2)ZHSHJVVeE0diygFGAH-l9+f3z70D*PC5;@o@Wv5yje60qC;_}(}F$4_{} z5B&Jyf>`a)HQF~t6S_Vnh~KeMU=ZAsRfD@4bE3E9C~xG)`tuFEDuDe+zAP`~W_crh zbp}JbRrsEu3mOb3bU7}_KG<>DElVG|@Vy7Jd#m@_S?l#1x!pV}mcCLiQMEdr$lwdn z+Y@@Sg0u_?kZHAI;-^z|+hToHZe%uD4-Q_BV0ZVSOsp+P==3pA=E_+Juu5a!j+T`a z?Am(YF|hOotM=;SqzJqFL4YC%+etYlndL#oc#YTh^;aQDSith(2b~w|Tqvo1UeFuc z3_h`45`?&Su_s1L*~anFH+ZH?%qPs;jQE{% zjaj}T1=I@}TuC5X*s5^Oq-9H8BB=dT%X6*QY#?@brdg>YaY+n&00OYCk@zRh###je zNu8$EDTRz+=MWTt!~0|3JraP&6$~ac;&*L8*&SdVtuD4E&K%_W=4g@u1+6#G9ooFV zzCw-HscgQ@?6SctNTJwdBxra0`d$Xn!CMZ_n(K5>bW(t@-7b81rPIx8VgnO8G6baV zM0sQt0N9XAO_qe-7{pU{5EKBODdwsGBt-G6kN0tX8*y{j3q0bMp z2J@LfDgc-Ocnyy(!8Z3PBRbX)1PNgz3k==_(~6M?fM>YeKqkOa2%Z$Ad0B^WCQ`RA zVEe&vhNFn+2t1UDfUKi+Du9;GpRlL1=aFdC?HaX$o%uv_3pQdMhvV4)|iPW&M zmLCAALPv5nw-ig_lR~}#ybd}cRjUrdfvkV*Bj$Tig2iOO2B4Vb_RMZzm_2H8l2?iy zv@GYnSF0It+*izZmaYpB+xJAtG-dS+1F&I#y%s?eN^5BIL2NYG-sZH=kgM%)+q#~; z#_{_v-pF&66}(zyJGE_2=y`2G*PJ+9<9NJ6!Op^c+~4=F|Fb{r1g{@1i2eFs`qRP= zU7rHPUcRNWa2-HR7vv--d8#sk>7{&idM;mT&*k0axg6Fzt61Pg$upepER}mMHc6tb zEOau`!h>cb02O0`GvP(OfPVt@ymqcqaJFYnNb~_TVD{D{Wkr4D9|cmw<-2lMAI7g1BS8vC$|J ze3F$D0A1Tu%U;}j`7Qwc2(_-LU_d~Hi&xN_3Q1;KBH01jp$QuujYepwSgUgOg6aoI z3XWhmbOSn~gNw>fyqt=&wCV_D=IRlpgANx=ZfcyS0E4W3c?eC^+~+KTgIbPL&-SUD zt9Ho%j%20ApkyQ2Yu4HYb`^$k%K>)DOqc0owmK%;avqHwRL} zM4rb%YmES09zOtEnJ*F!LmK-@6HXAq>V2g$pOu#xjs9n&AKt_(IUsu1l2C~KRPy8)Gj-GOK=K(lo=!F*k-{o0LZg5QR1Sn^WvK9OW@!8CS8 z2mE2jr?d#wJWTtFd7n*WIFIu#f$MeqHV9r@JQq}dAb_<>IMgq5-nS{VnNbM=_lUK& z{e2CY=WYOFt#%L5bG^pA0hsi_NsB?TLm|3hzjJ(sL6aQs!S9lEw2O?$eLT+beR)Po zUhMTw9c^)n@tx+FBjK64SeAXVa}4b?%5H7@sU`b0N_(=4mVW;N8HApa699~qV4VvS z0Qbq%YuN+y6tfGOJTkZH4o4?DBnsF|Ei%B!PML?krU7DKy^_~jUhDSGBC+8uNor$v zAhxBPz{hT1#e-58w<)m8WxT2r%MFzdhBua&?tq|B<}+jsNo#5B38*e#jv9=l;{}zrV+R|A9cP|Iw3d&T?2@ z%e(SIe!M@IFK+<$^Ws|AdD-sBK=JP3+2Llq8Vd|h5d!n@AB=}cgxz)WsTNiz;Hy_J zU7US?3(vP(nC<526I*l|($Hflb34HFOh}Ha*|RCLa;eRld(RLaH)`{2Jp^>X7#}07b+lvv>icF&Ou)caqe6zZ z*7suucO1(!g>#f*sh)KvtV|IN=+H^k>{nwDieY%b7LE~{vb7=<)@Gy;?*S4SCZjQ~ z#%Vu~_N@D*7PXc8{NQ{pL13AK^`2jX(T@X>ANP0=!ZLLJ_%SMbY_@}r&$R!{W5*wl z81Kt--?7hdOsd>wcg!6ah67PTvt!L>71WVid95M|%H{^k{#Y@m>OE;f%`ibdr6~IM ztP(Is2O^fvGC+HM+!cwp`8*~2MOzZirBJ^+a@@t4S4ucMAg~`-uG_qyO`^IFUT2kp zdd-hZT~Y&X=LQ||SSZ0@6l{t71(qNSU>U~DkGUd}=aoum3n$EPN}{~v*1a_>S?u=( z?X;tIFy#iVm$k8X1RH<|*T0?w8B&g0)DuRu(cOFr0>rYDJf4>n=k5BD=^Oqc}5}U3U4RUa11a0pr7ZvAS_k+jKr>*Ctjs zsKB5;oP9kbc`RGX>Vq5B<=7^m+#9Oz#l)sp3&yDx>v(aDVDc{Rr+@E?a!-Em^$b0v zzE4bo`oa!x+EVtAhN<yMHRxbjEPX~RP0O5}l4bQI zhuMz7I)P+i$zWKRRSUO+g^f-`b4m{4*>P-!)23t|vH&?9f_*aFZYgl{#gH?*n;t!_ zv2Rpal_Lz~1t_3?xB^wqoBZyqrg3>rYXbp?O~nf~xf!&!!q{ZPG={7L+c^H%JBY^A(V=C; z1qoM;O%0YMA}bDD7N`Myz7zDN^Wyf*iM=lD!%ipGss&6Qnx1Cp7Ike{Qv%k4J(?c& za9tt`hby0>XaYm+z5gIz$pk}$T`?Z6Q}_GR!8o=Utrj3QVK1XA2!->w0Jq#%GXljI zi*s#bI8cV{-piJm-Ca#;VB0kP3b48Md1dFxPy$3Q0IE@KVEDH#!&``-%F~*XoY9)L zR2E^zIiQ(6S7@5|>&YcY)Q`3TX6Pi0^AdVk;e5&|=HdDy!tdvj97Y=sB9MuWtPk!> zvIT2PIR^=bNS1m`=WLJ034`XQ68rWV)P@o#UEacyNPr7-DaKUsyuOHig= zqdoIAy(bDL+&_{!b@hB>HKst_jt$CaVQpec%0LDl8;Q6cV`9?l`b(Yun z>U#J&Od<0T`Cg!{I*oO1Jo5P6^3#9ezmac!bVZ6hdr5vk#f1Pm=gvjm$i zA$TJ%Efny|GdZ`vtC3f*VjVu7K1F8a97c)&){OrKMD;c6z;u8AfK|N&h&61Y6OUk8 z&xZhbIeRE&(7VRC1gkkbWOkW|MMG%q{A3SC`5q?y)X!ZuN~oXZqsj^J27jGxeG4e+ zZEv8)lI#to29|u+8Cr3J$TdN*Hiz_Z6tCYw;1S6w3QVu9J?7C$wLH;sEmt>|q@u;M z{kd;3Xvr&wtQnkbExlNFu(zR=bx+!oUD#a)?KY#W=Io!n3Se7{XA>kRh-EUCftRSl zum_-KI9IY>17KQ;C%y+=ID)WrOm!$~xLx8(SmVpbdcg+eJdyy{ zQL3xu##4`9L-Sdim4KD!xj%;gUVt$WKRdHs+Lue|!uNVS0I{t<`o2phmf*qH-Rj-A z3M)i=#C2(`;h>}FObKgPdZ@N`v?{eLgPmf%%NA0AI=?An@<_urr;pw0bWVi@vhf7q zs}(tF=zn0nd+W-9WG){Ya&=_K5EPC3UIsOOHUcSuovGg$N6tN?ItlA}2{z1GSZ#K* z6sH0X)#`SX`cSS7uK_EjA6(F0)@nbSTEZFH`x9D4FUrE6y=lqndRSX26|7keaqq}z z!w3mObqE_0O}j7&DV=gM#;hI9Yq#qd5}?zgPF~8)BX{WjxTe05;84o^4|`kggP+F* z`-taY1Ys0#Oh$JgSupYg4pHf)13X8wf$ep*$!aTodG?gI@tzdtHl=7eU99@ej=jvE zL-x+6In@BKasC(Uo%@vUH|b!tM%*O{;x%8WIlcn5ZfHGDA;*2-pJ?2 zGx>7=P<~>1Chwj^N6(q8uKui`u$J9QrUcCr)T6&YTXh80;-2pA-9TDd z6ZiHmH`?s*B67K5*6_sT*e0XV9{laWy(!=&j2XT0e3Ou}aP1jbnjx`Ry*{HLp~A<$ zuC!R%{!YV893=|J|z z-e+|$51z1POqAY3fKSiTMq&$rDC!l?2Zmj*@m{d9T4DyW0tL4>_ z3?#+aJn|lH?iU8Gu9X!aw3W}V2nlB;4`;9%S!=Rq;*4arRc84xvQINY$g_}zcA!Fi z_l$%?k4R|K0!!jjDFG`I%!pJRW})_)2_k+km$-i;oYN7^F#i@Kz;H9KQDWF*OnuA> zSlt0wKZbxw9CZF8nXxdygTTXqGwU>;(I{k~8(!l)=e-Ua2u5EZ;Tc}L`iz7+HZ>qX z)we7wYkvEC?llSc1dvO-4-gyC0wGI|c_FN>0O>&YDmM)_ja5$c-CPrk0p$#juVHI_ z=w|FadfesY4Oo)IK^ZFSH<1Vy=Tlj63BVb4r^rO*b7AvP)GqtE*dBT4DC93C z5`abo+qJTh!D55kP?FJ^Xm-R3&}|MK>r{YQUG{q@qmEM{h4g21V;hi$z&Y$KXKpgQ zuJ@JB^;QUkPH9_>NOGNc1lZe*JBZc9a6z4M+93z{N04b)7iGaE2u)c*IPI}c2%vEu zJ64|(7TN+FzkA}Hl7U+OY>|Wf7XN%knBeT zo63$IRYq`GcQr-a9h+K)BhFD&+r3*q-)R7xKx4l*nS;3+$ymjBww8_x zow2)Zyq^TSgFi<)Q#+Pk8W>K@HL%UwHUQZcHLq(h;;Ly~&_Kn1dfQpeJik9Xq;t0q z-+<^}M|fM;b2>yB?#*_)#VIPGbsHrG%vx>cwCrpwGwiJfHER3n=g&wK^!bEy!8(}w z1V%XRvjf|70x#LeUe?|MdBy^r=LCmpW<6$;C`y#wPYj3T>! z0SwPG~;(@3d7x<6Tm4b`#SSt6FRu+)nkj@RbF)J14ryJoMkMkB8VL;V|L@<5SiCPV|F#JdW@?ark_o+)jup3}Wuv!7@rG9t+9x%(i zM|?dgr|^<_4Z10xk(^-1S&#%FkQ3;-S+GHkb$9N@T6Qh)-Mn_HjvrX2D9nC1fQi3QWBLetBeIGZ|At;SRfQe0!$t+Hu zI2d7)BAvV$4%#$l);Dc~Z(WO24dzuS*kFXbC3N4W`p-s}|sKEg-go1?Kt-d4Z(FL7_KvppSL#BYl-K?|C`J)N8OeoH|Y$c>- zclgy_9+KAprY;pT!R_Nn9 z%JdoT1NbwiU}kk0m*Z6CmDL;U+yfLhi&`j#u*w7*j!+`Y6M1=@gRmTyI~FX(If`_l zRwXEUSz4Nwl7V?jwx7vc{9CQeqV@Gh_tII1?L0JB;clJ0gVzRPzw^=S?S>!CaT}A) zp##3`6WP_bd^D{Y`}aP>1KQ)W2eE(VA3ojC<~Kx3E7Zpi0AkIUZ$R~CeIs9JvKO z_aO6IMwA;$oz?#NFf6h8c?JLXqU~?>7p_LW7$2G1&oH^?> z8bWPgV#zp)AS~KV7wa{D#aLGdu5K80qrQSMH(-l|t1AFkYpsb&0G{B*+OTEApE|%X znAziuu>TZ*(z<&cy#4UiF)K?<0L6!yrQBvDb7iP2>i}X-9#|FO;0(mUL%sX=O&MYw z2yqEp^5TB39vYn**%dqy9!}W+Q>^ItOa>AN+j3s8g|Gt*uL{hynxHUPLV!QJpJezW z_~$dG{+pOl%HKyYa%_u9c9_PSpUY|+wz=YT2ElD*wVK4-WcyA2{efK~0KeEYaL7kN zwtw#7v0@DgmPY_7DS(S0PonlQ593K&31;dj?pTERQ)u8yboUiQX}srK`XC}idc@_W8-I)Hs~%qaDV{rCEpVELNci)q7ft6HnP3d zL9ovPKv?e_ytWQj99W<|U6HVb_Oh#&;9{eLkQ6AmT>R;O$J|7CIHY1e`F8W72k6K! zKIfZ12khyHeI+`%mAU3p0NA7rfaonanS{bmgV!#tOzXz6#G2~KvNPE>YmX)ok#@#! zh}o;s(a>Zsu6GKeNHD$*P+R00D5+#Q1sm|Ru|Kb1b!ktpLfYkf+vIJ81Bm)OxknQr zbsVBi>|qW5xHLp<60`G};4?(f-Y4*bS(iz(K5+F>((#~A>kS3_{sUd;Nta332m{t&bpXFfq zK86-5h^-p}9rD)q3x&VOIDQOJeJrGV`Q+>S5KWOInAIb*S4l8r&HISk-z-^T8=n;{ ztU1Lns6BuW7iB|hKV=%!NUITaaHc>}SyzJCD=iH)yA0s^i15FHOl3V$Q6RwOhbuWJoMPH95VGAR1}*XRe%R#7+ceF zla6^&Uc5BW$)4r+XA}Z6mZg1KE5H>_RD5~omp$AB9s{uW#wvoSAw&YP%XMU<`G_OP zH51B2Cavt=fFl%klHVI(QR?-9h2i}yz117q=9-gL8T=`ouUcwr$0P@G&hwVac7~DR zd;RZzp1C=eJB{lC_(Q`ICMnftF#qG>&)EDWL)sC7E%$?`U&|OSs7b3#rr+)s$J1sI zV`WCY1b9}kb018sYhta`7H*GM+E`H=)_+=gEn|;|Fy%9mWEwWp*(W33(sAxR8^mzj%Q|I(n6j{XE?Sn8EUao{*|5U*!b7gtR`=#~ zM!k-B)woV+6UzKxTW#TdflX%Q$?^=ulM0zEXyT%ov z#%Vai*zx}Rj+j~S739G9P1peIyk*Cr5ln>aHdZ@V5xO|L3KaxI!R)r&Ka(C#;E4o_sIovyaor-|L1VyfqicnA@~k_mMrlxvZ6wCI4Cxj z$0A6y5bxOEk1kBuYH)xkPr#-d!@0LjEgY{>jDJO)-PIdNNPR!2%E)E~=Cs6T-EN=2 z#A^F-XIt~{WMR)t>{^CZC$LHdt5{;O;5zl%y(+{Z^Ce=O5y9xWR>DD!`7Rj4tQBsy z6I#?ZI8UZ1n?+k`z>2a144`YN+wSh~){`VSliotc(!nHFJ*g8IYs^!(Cu_74&&0%> z)|g8IEhsLvh(%Iz+vf z>o50!YbNbr3m=YlZSaFw6Dx?UmoJ_hh-E%ZjIr8n4b_KyWRC^du!AkpY1e61#wd{} z$w*YbkLw#&*9E@e8#mj3&+bGSLZ9TkHhJ;tBp<#!%XgpO$*o*%cJu)3n#9Iu_&CUW z-~4Ys%L9J4Aol0~;a9eW^EbkEsE@YZ#(d(Z4sy5!&nn6Og)Rg?+wWXqY}qWDMC%0 z2sVXDIC@Sk!%COOog|Bnw7l_qya-Dc^nR=v_WOZi8?6VgmNv`yQ?# zzAH52>l%(T!60>}Em&-3GG{~}=8A?Dsa>728Aq#55H|5C62bhcd>HnLa=DH()atBb zUC()69@-Ee9oKZynI^-;BHuesyT~j!H7j~L7*eJGwj>!dxHHO$^9cVSH^~`%FoN=_-7@{LeQbFv) zy&mpf0o-yiZ~TIKBnLX~@oVD#+))K+vF04JJiG7mPrv-*+G7{a?z`)ulz?G1-Cqo7f~j=<<|Kl8xW-g%_f^5` z5kQtj-o1>_l=WEkch+MC$hGPK6KiCuhPKPDEi7aPUfRan8k%fjcGi@jm092u4a7vI zQ2_n~PB2-vWCF8JtBd(vD|-;;9#+RD&SRP(AxY=O!+Tz?W=439WoQiGUNl%Zge<}t z=HB-6k>5vFg8j&2!#Q*Z0Q%luLtgOohijQx>$lhI1z(UI)skH|<2_BgbE#|P@*L~4 zOk9yN=L=%?zjn!+Qb6t}nWA{lBj+K)&FcbaU*OsYGMxNF0M%MH%7MWOy`#U?#|qaM zU>weBII}&OBy8xEow#n1bCB4ZQI!!k#~Z4g!~UGnWI>8PdVqOzgu6KC#9JIFxdZLj zQNi(uOLoI0zD~%*(vE=vY!peKg?@_EC0i-RHU;Y&AQrM159>=L7TFx=3C;;( z%^tHH8|@1~EZ-ZiN!BDL6Hw=WXe0;Q2B#y-C`ZF<*@l*5pG+*7T9$hB3KJgQpQp;K z(vf!QmJ$tGesJDwG}psiWL9d!8%hM4ja1^OXM*xHp|!h$$Hn(uGO>p!u{FgSYyfYm zCFiB?*K?;9x#bwgJjD7n$UZ!=%RoKut0r_Sh*eFjMRXTd+*q+EqLf+z&`ccbj20Kr z%$oKMb}YcGT$a=-=NY90-F8^xW@EX1PA%;AH4yEXuH@zEO8(KWeJRv(41@Oh#GuBYI7Az}bj&T~&xtC_Vo z^5yoC{I2=AeCcp2PseMS?mm)szBLBJ*`S;hmVh&};ITj$-Q$KjsLOIJ1hK3Ys|L$6 z&(%x3F;7w1*CzA?;mfve3y8&^8P1aCNk}wSmBFMDs~x$#nE?t}uLO~7HF+*THbWqs z%uL+zwT1VH1enwZb*GOq8ukyqMmA&YTjUPEVZp3nj~Qfm;3X5NIf$7e9PYi-JH~0- z5KgOkfWG)WlhvLRq#%M!(ba3(O?`C`yZ!yf5>y1>vsog8)xVt4F{}H(XM4>h2UZtT z0c`h*0VX+79jpQ8{Cz6p89M4IL;SUzQJeM#q$8Z0>{5c#UW^O@XUz_;nf6Rpnc_2h zk32N4=t8FT1|6)j3B$C1_AIgX7=Vj;#EjX}{ibd_O-cwZ3~7tCOuaL&m!P~)td2or z%=Wvf)zsRL4%&P(W<6Bjc#1BpiG*kqj6;i`g;nBM(TR=lT_*C>$M~4vIX-kc?e9cd z`~3xL-GhY*_4)i=@Acs~G}QaNCN6?f~H=NxO?(3t+E*6Q`m zbb+J2Z)fWvYCX{TSPBdJof7a4&Rw%@SWk%BNJupX3QFX9?{&Umo(AcF^WkEt zO{%Q~J6E79GZemFrhYUK`|56zU-|Y2^89wSMhI&^z+u>&kzt#$krffd9!U<^mk>0M zBy~Ra(d6CFvJLHL2V(!t|Ks;=BmN(Y#H@e#;3lLuZeUQf*{B)k$xO&$8}eyU&UE(s zu)dV1(+l|>^M~>~+lTVS!%42j243&qY=1jTUs;aHIhkQ0>4TNS@-}a3#Aa-Iz8Pw% z6mGocL8+!`aRBR~iN&<93SzIXo?3laYF|&$eapRW1}DjAb&9ZU<62{b4V^4XwwzH$ zMm0|Cz~p4d*{+8h3-=^FhSwiu7JXR^*eIRd1L770ZIYELX^t>DNZl2U)_iS;4YSIcb zzMfqx6yrDl%*3QK$R;e2QH^q+EvbzfU3%$rVZ-gzWVmr_Wh3jZsZ4hIkg^6l4_q6K zshjYUY{L>y0Fnk$^!;;b(>Mhf4HgmO)zAl&X?+Gen>#=$0|J|h`ajMR_~24@;ZCYk zD5{89UhCN%d}Bx#TfK80vW;d+R zdJ0I76(y-oqAkZ8Krz3|$N{ajx_V*y^Kybs9FWDd(8Qh~7jV@FnCtCJPsLWdL=Z&bXIjwT@1CPOLCOP|^cf0uy2kw)d}O*LOCvyXwr2%HzkT@FJVAI|86ceu{H)^YN<~~pd2@S~ z*RSv8{f}PA%Tt%T_1c!()L0HsgjRJlQjIvi{&9`$AdDji?U6@ozgzzBpZcFI zf!IIxmws;h@4sxne*4?d_*HYu49-((?x-p%ZdFVxQ5dEkb zYA)e>->jCHv}`YVkK{4anOd#8?0B*dpGtV#T>M!e-|Lyy>_&zLeg9g5>0y~GK5Ce= zK5Md=zt4+nTNbWm#ICXVH83Bxk(AH}HJ@a^8+LM@n=%zv=TI4olZQ^#iD}R;TLMI5 zJ<0`!($9TDbA#jd^X6vSC1o8p@Oqf=d)Foh*ItJZ46KLd-KZWXl-W|l%JXG-sIc1( zup}IiVBRi0-VUG|S)vjfQOz!`OrVdiBs&jDg}{Sly}!z&`!vdgEJ`DMqj1TCC#}u) zJ9T6M-WWCv14@1FZ0yG2v2tD)+T9(K*}%%+e>_o!-fSsi<8#2C?R6i_cZ|Pb57KmS z<;MdSq9bh>anAyCROtM`lF`$>;h-m`x%&&ll!_R&W_FVY@-&yc#_@>O>V*YZ9e z!_f$p9l%7w8opsIYZuTE6(SH>GGMA@2U%}*Nmg`CiO??EQm&V=|cNDaFRH z44Z3DqpEBaONTQ^q++iDD8r68TaBm{8x;(9>{6EDJ81dgl^%#cc8fhYXS;4Yv`OIq!1`j{IaDPcs)iAsu52MHhg6Tu@VDH6wl`NVT z)4P1*a-Z2N^i!kH%9jB}o`Z=KGkh$z@Obao|k z_SWnnbf*)l7Iez+h9@mbI9IZ}Mke2tOSE5Y-)O8I;$x{DJ{3nid-lwjSgx_KSGL#n z*v|vBV%((I2|C)5lK|IbZ(3T|+;oB_R9^7!vCS1#7N&9V#&&*A5GtIntolr^^O1S@pru4<1+-Y zKleAc|9;he|D?@=9u7Ut2BX#%>F95*%;{oA?p}@?i2dT>wfyAqR=zUb%8lGw&Dgeq z*amOgCX4Nyt|_7ZZ!jbTW61Jark=&-ry7>~yA8y?xwRt2x7wUw`=7N5-JX*!g4QG4 zp#WCFm}{CcOPd?l*?{W389Ht;{OpPH0qS1~TsT1**^1Ppask&)PYIe6!1^<@Fl#jd zCi^@D%b55wMaVObXXko~Y*XsX*0ypzqWn{4z+YBQpmf4OtbzD@^fCusA!t$9V9cdK z%gjt#&(J;0t{FONJK3uPbgf>RMM3Nw4F$T@Y`s2G0}Ie3EkK`#^SyPe#74OW7>aDr zHTAfDpVEtta3A+--a)>%fZqi4dnY27Lxy0;YPU+;@(KtdsqgM!McPQ+X|FPeWE&YJ zsPcpfZEo2$jPoOti1tZm0bC2ntAMrkIIrI$WFK}Qp4VS>qa!wCF9(##DjbtKf9vKQ zAeOkemDgK_A1}}S@_t-~<@VZfo+mf7y^YItmERG-iED@7H416HIvXFG5suL5JGB;b z&ohljtSx8hd~z|wZ);aG^Vjo*ZCG}|s9t*wUs_#B%95HVmr-9CGrf)@wBdC=NL)op zw&N?dLA$M?-jWiHzNXZ|7(xJM2Yf~Qyc_Qkw<0vZT+~gzd)na;*Ib<)wqDi;J0Y6gWWx_8MfSe?9Fvq=PI2jA963XR z9WV1m$42+dyW3j>uhwoBAa?LRGx@glNHlh^m(g0DC!)dMBEQAS=kZwD zYsI$Q2U%N0ueW<+s7u+|s~dTF9`a9q^F#UevyIS|Cy2tq$uL2_<(L$EgbpuzOw`SH{)k1)&?m* zsAS_n6?;%6D1PsHZ2TtYkYvut)dsQ{%7h|~zEO)vJ5S)zV(!38=zetGKV@7W4o)~cD8Zp+wkFu{^|%jk$W0U5 zwYdFk09M~KS@iYP-Gz)Bncx;<4|=9Je*u{JeGB`xpM(3Y*KA!CFT6eY`T8{?mxQ%n zDQh}M3-Q{4@*Lg1luU{FjRnN9V(V6)%gOZmaj>F#X@n49O*VFeW7@e^yTn{Jr)c(0 zT?$@VBbM`b0GJFC4Y?JZ_DKSrR6>ZKxkrqf`|RUfNfkP~^s@&uGD{=#kJ+RrZw&1S z%(VWz`Ukh`JpTFXJNe>9k{_)g z_Lugz@_e~6V5hB|$$)gJ6Kmg9gVxb1&>A41Gzu8u9FDA;=nHSL#kCXzWl22y z1hK3K%2tGa4Gx~GVX5I9hGO2dkuP9yy~2X)#3o1{=4Fb-STK{Y>rIBa$6Q3xHk+qy zHWNF}c-YRgl08{Ss|GN5@|CqL9EK<*WuR?+&6y?JdDkxOqNV_LED22xV$Tfx>S?gP z>WJ|ED9J3?CO9t`ZyFERNww2*PQ3!096JIpyBC7F(~BB2n8ddLqhb)MVbVu1qW~28 z$c$A03A{T%9+(}e#~ZSaW6lMUQ-Byf-Vu6g!*-6m348_{FpQ7R|CB)*1K4cmpgmW| zvt{zyKxV(ZmyC8+ByAbF=I4_|;*Rtd5c_r4^ys>K2k^oM9SpLRdG5|~GG6}HsI7_Q z6RYMB`$5`6Tm-JGOJ!CB&w0q5tRn&65^LDOpF5v1H&4LpW`3N4puyRFi8-J5pm*Lc zAJ?JP4Yy_3wIgwsz`ii=u4^TLLf9Xg&swR4Nw%T6X<&$4(saxq6lQooj2!v?~I_% z=k7Tec_RdEUy)5;Y(hb=n|aIVm_DNIFdo#t8*4}1J;109O(i+t zTlrywEp}pR>ljFJMod3^vX5|Y;T++)gO3(Ud6r>`x!C9ww^3Mf)LJn2qui`>`(_&C z)y*J3|Kce3s^-|8L1t5&>{51*iS?K?If)+oE^m0W-c>$Yv=h()zHP9L5mFw$zjtu-+L<3pP}>o~D&@#?_3T z$q$aGiwez`BqNfOA;simd8M3h7Klzznp#tq1p`P1c(u|}=mw_GRZz)CZW?IzU<`wY zaZIyIrbGN(X&`MOPZw?i>vRyjg)~=MT`qj&W>HK=p-Y^-;S(gXWo|_W8~Abs?}P%I(nl6+r4U zxX1PJU(g)9Z?Un{D3YIX6JzUuWy++6u>9|y4CMc8ZdPV902GhqlgGk%_*}|7SWzMM zRhjL-bwUQI;YVJ3oikeing+KzFxep3#$yKAudV_{JCL26I#36w?$89Y=3H zXpIhMon5FDq}unnl!qdiojlYm)P*AA5NsL`FJuCT%qWHI{In$1Xu#@WQvypI&JAQc?>NKnbFFD^GVe(|&NT&th zBI8bmD{64R!TDm+uawFrn|Td@+QE+Vcu~&L4WZldHiUg6P$S#eI48=?4}zshk_IBlfmcEAfI1T||8+P4q0I*4P$nzD%{?)I4^*29$aS;3K|LZ5~&iL<2 zUfWwiEEk2-Jv%(Rf)jkrRykk=v6|i85Aw2K$@k=y{AdNSAMI}Bvfs$QO_uP}z+5wsfQ4PSi4{+E6DBBQL=6p%0)gTl?f3Y%`nleaU=fD9^>x;m0U(w@yMhoKd2WnH zp8s*fI4?75O*ToO6JVeT+u&}&k0gl2*c!uIYEjxXcR`@S%WElnYAsyb0MP2?X%t-U zjTv3c=$uD?MMA-j2=E0EX8ladU4EM3OG)!!iz~q5&*oI6p8;f&Puf>waD! zw_6{l;PsTLC5X+;=zMQnThUimu*|;^IC6s>q9RG4GXL!E|JlE#fJPiYfGo;F)tb^? zLdz47cQoX+I-9KN0L&t1#CtjwLDgoCq&ZO zOy_wCK(+1m<|pe_7qT2-@7XaCgSQ3ZJV(7Bu-$axSyu31Ozh%iwDh$dj(GR#GS*_& zY^0C%fwps=H*$jNps}T>=T`Xpg$$N35ewy)07Mp9a-KEW%x}(tbY^99pFKJOKsGc0 zt{i`o9b_w0R+NBfdBZ&>5cTyopd!RptDeq&&vTIzhZwIlHudsCo*0Nd+pHO$52Xca z+g`kQVY*ooGX`Quchc<>czwah`yNLdNG&XLOKd;Ix!W8lWKymZ|30>qESUW*_<>D) zR1>S2uV*yl;R#CrLnGIAKxS>Y9l0{GXtusz%F$`u3H^IW7e#O*eTRBNdk z9Su_$d$o(Ctj%_u;|HYq077J*1_CeZoWS4j4dd*l3UmuAN3ISk=!N$DLF| z6To?zB3X(IEPbJqGi^0r&%7q=Ia|h>`~Zy%1{T*ILU+s!#k!TC{D-j* zezIcq?}iqJM#hfuDPh=_9b1{sWDI;` z;_`2s{_zVv?%YhGD+`&u0J6LAm|3`-o0&CUD909n__{I1#uvs@QWkR(<_f?ieKRlW z`J94Rd?JX-1fdQ0Pfql{3-1NTlroTV9=-f+t08{;sA0S}f!O3Yc=f#9ERK(wt_=)} zfT=-(5|fNjcPht_pDA#xq82F1IL*%F&dU;DX%0qn4(Sp%Y?uEylVj$n237&sXfKbm zk`oiIV*v|0nRd2|u4(D)GYdUv;jU4inKA z-P0N)F{_Q6L8(_AbOV#Q^sMi};tq`t7B)}+m?Cw9Od`%T0QUeeygj!OiM7gXYUwQ3 zrjnIxVQ*P4);eHaUmH>J-i!AV0JHWr>?AY&?ekjB@!F1;kwor~0}`kjwXxXi_y|V3 zIDO%WFV!hq0tehf6U*EgCy~OZL6e)@=X)7TK72fOu&bFAcWS*z%AJxDo$$n*nZfnV zAom(dnl>5Ye(7u-cTNW0+|KgFt4Y4PZt~S#mxC5P)oy5`k5$+8)rOcCx!r^{3tSg= zk{1smp`5Fp!2kV=Ha_^pLF}iW{l)eFKe&1BYe6i2iq5nPGEWENV?}FnE(enmd@^3k z57)0>I=`1^!~Ob-I|ICRH|zY53$QoOb{~fj{xe&V8-_?4g1VU5%GAB2!8%(o97e#H zAM6WS0i@Q1qc&BK$z(4kSHvvt38cyH1cQU~CurV9#RIZC=`E9mwT>W5CYwF>Yt*n?w1hGz5jHc3danR-4Eg8Jd~a&utG52(;eaYE6J$UR?vCL!&spFf zKy?Z|J^$_u=Tw0*KxS%QnLm&XXBGf>Hhl>;xu-0FjxZa|ID31HYxm@?Ib>16?jjjV zkL41A1?RhDa;!BikkA@?5tSj@h(AyhJJGgzsV-xlJEDRG_i=5omT@>9q2I(>rLG(i z2s&U==HiBlouPS_t&>a;$izrC{mD6*Aa-MO(n7=@S-0`JCP-vt^?^Lk?fNdRX!rLS zXl+}fhB7_^oMxnk?`1jOK1xI#MM}34g zHukZBHiPe#V$W%F)9L0KHWF-c#$V^p{oQTI3G}=ps@lBTiU*QpMqEd)tD~=vSsl7cI>;5Xmxxy8T&%VZ^a7hYwoAL+}w@w^Dpn@dO|z|$?G<` z8%3dFF@x4QXHm~v0!7EHdW67ZTvwF*)8F{7AOH2;AHNug{WCv%F75cAJvz=?K&)L$ z2Y^L+AY)?12Yl}CHIB=zynq(=M;8zB$@x((+Jn`H4G`?;CbUT=Hn+b)zbVNinwKd( zQbyCLJG*2?QI-h#=YuiPBJe*KmlKrI{5iD zFi)P%JEIPxmlks-gyZI>62(hKG5Nrbow5W;FXB4EU{w|9uGjVC@;tnB?^K0Qk+SHc zcY#rxW>N*SG+zAOz<4pZgK2zwu*nSJdb!OhH<>}oMuFKBug&7?`Km=imDG`dRP}Es z;~$!`Rx@B>PM56T$wDs-D)r86VLCV!h!QCk4;c5z7%_rxDIgjpx44PgH7*MPqdzx- z&7qbnzA`2C5%3q1wG}{18sTsN7LKi0-8H_F6(;(LNzqTJMxQ{}H>!MWm|t#`Wn2Oz za1$Q#_p}~)<5)x{zIgt0HgSI(pHk;Z#*u~SC2MR#?`ms#@zQgiB#MqwbOx=IU@gLi^k1INJ9?$;v$5)|Spgtu%tF7@MT)@~F<{;w83NnZOz*f$+*M zWz4tPN2MXROA;VANQWUE_vnII_J`_h2Z$|>I&F4~(r%EdcyUoQ+v>n7yY<#jR?*~W zb&i#-KbJ3GALKWG_e=Tejy3pOl8nO+ST6ZjmlCj~jV)5o0T7zc7O%}OgW12*KmGB4 zDF5eQeUR_?_{Bi%ANwtHMJwq`psF2k_MS${1b4zJ`_E}VsZ zIUeO))@y7?1`rhd11T$JUpBXwS^Q^iXYzqyjOaP&F{XG(> zX6mRU0B$g;@T$0hxz~+Nhg)K++Ek1I6Kux)W|3zYHfzz+6DI@PJ2M=MmxuRmGCd1$ zs9C_=Onht?7{Dz9Kmg;02q7U9VJ2Y9&s$=XJECmavMF5dKvie3mD2(cbq3b&k%P4b zi??~>5iPZHcD@?1F^crF)XyH-^&?~jJa4j^nb0I$lj$Kh!94J>h;VIV2*=N%6WGKm z?3l~=EKWY(nQ(WvtgB#-=tIY9fCNH{VOq)@&9c&D5r+V;BsrSU2oGKyA9E_?FOnd# zHP!*mbBfU5m|0Q_Sd9%pgRU$PCN~XkbR4bb&zV}cov_F!8*;F))T6-&R>Pc3YPBH= z%CqIKgb_bwTdU^z^l{3@x+8nSZi(OTNVY1^*M!ej-m?%)Mp5|^uI>388Az8OAvsDn zmLq{&wttNTs=j16A59Ic?IlY5O{kCUc47&()SchOOXwj3-vnmSf*+j5r3yddW&)_zq+r+bvdL(8q>VC+O9$u|ZSoGJrm)B*52|5d^vqS)`;#UeXt@@8q|C|4!C# zuG|@P4GhjY#Wcyi$$o82P7?_|%E|e@;rDUZ<9SgH@o=elhRfJ9kOh z{57;QbMwW9*YVlZr_qeCrJ#xKy9>$v>*!rumMmp8pD+ZWEp#_enD(lyN9fn=^(|N_ z{akvXK9PDXqSJCksh471yRDurLKXnAUKpAk&w61p@=-ysu^@*BuSy^+S+>{$+9Uu# zr$b}R4k|T#5fX(70G?V~tEuRQ+5t@qa7q@|+oMX0(lSC2i4Y}>UtjF^c*|m( zN(uOg*9=>hI3K=EcRFq&07ep<8LI#gK-u&25)7hrmdT=wfO^hs{puO?sByi_1u!BE z&3Ovlvbg}PQLQq7gKISDO!7M=01^5R8G%<(F^SS~0-yx&>m{pw>RxFCp?Ld!>5*i% z#5nW-3_yy(MB6_!3Yc+jWRMwIn#jiLIiOsYbgoZCKP(Xk+F?2b{E{9dLI-$uAkt!; z%(L$e%gGp-W)!tE9nknCnCw7XlT$B@b3M>OSo(&IjXJRaCCWgu!_*RMs!+B#w*ij% zHmoeMX6Tr5K1PUJ@-=>)?9%0JUcF^$oyAF_n0zCD1N(^bOD(Sf;UJ&n5wU5`YO{d_ z_FL&ZuU?j9U9&>~)sD3{l$;LA<#e9yu&0Gw4^o1cFJBrH+X<>+*D1+!=>?)@Tu0S` z$NgdbSO>9ohc<*uz8*wDkQ;O$79(nIZ*S!0<|ZPxz9;Y89-_>bCm@d~naO<}K(!I9 z4rTl^%0?Wsan2j=WKO`>LHoA5BxPnsaNRt@ZhJd?qK2yhZe2K=$h4X!T;h zxb7yQS=Q$#-SD(})XcrD&R%YYh_G+wDPHqt5UWR;`%nMcfB)zW@A&xcAogGQJ7;n+ z|CRNpi${NZD~PpXCGG_v(Z=|%%#JIF{X#yuxRzhJyp#9ZjXW7r;s1S<;W}lP;tup=Rwi`i zT9lnx5bp0;si8iqQRO)`vXNyRqXi=x8+f8b8Uq3@G3kWsV6_dNgG* z%guHhW)`9wp;7OEi%c{NrVfn4Aap*qV6`SE`DP;l`h)-gdO(H0m_Uil#IQAdJ}=0E zWHJTkCeP0{TW&(Jp%N$-uChyh4vlGJr=H3XCE%-?+bTcUG;DyQQ|B@Xq8cJSZH!Q1 zXS}9uU&Fthem|Y}ytk~RRc1M+QB0@1-G1He@-*R|Sxjf#poKTB@_JCo-}OsXJC(>W~Pl5<~?U zRv|zcq$OBguT&77#U?LRSeqTt>9x@(!Q<2{Kx}+6IiYf2I1!{UoFhnu!HzNGd}lao z#xA#f&XU~#Vm0KdO%NR9vl>K4HG>Wi3!CWpE<*1bO6GbX*vyGKZ02&6$B<2|z$(l@ zq37y6bTFq3=N>X4num@(=R}&4L+FFmha70ym=Q1uc9bZFGW-B_du&y$RT@ndV zYZ&7h3%&DNhO+Nq%&pYgL2jl22rDWq+?UY2@ZL@=o6>Q3s zA&8BAqLpOZBmtZSb}>QMO>tNq!xd+>1_IHDS$FJ?>r)4H&er3-o}S3>e14GEH}@<0 zTAc`uB#g?4k4S7L6Py40_ryr2=liDL*YCgFfBIkkJ0kz)qxZba%t?Jv%ulTvLjtO8G6R){eu5#Vwu_ytvSwZaOER(5h;V~q$j=7bjEW1% z$s*e18#B}+2q~myNThn)wOF-HqQMLF?@%*!EJi{fS#6*Nc+DgZ=-w=GvB^qo0eG(!GT=-^m#7Nz-vHoG8Cj9S zCKRCV4sRWsZu0;Ye$z;EO-dR6CFj9X?sHr_oLn1gtt_x)sNJ*nVZFM9lkDwe$@*n$ z=mmOIYV%ds>TA%QNv_LS_tQx-*6hC%Ty_Gm>R2fCu#k5_sGZXu0K%v167!ypc0`L% z&c`RPgk5!wVqW7kk*x6b*ypSsV`~M8I{rCiwlj3uv9`(H#@NTY^Or82YpFw%%LKC} z#=>jJE>4Kx{Eo|vvRZUL3V6k9rw(mB&r0nWqE}{bmIO-q$S6S7aZ8R`S^K5_av6)d z-dNn9l?nJ<+|%`IvOJPr9P#{a&IRQ&3b1zrGVA@jt<$yzCL|OvV*(G0c!I7u5(QxD zTWxQWX3Y$=tR>k|j*)>|Z7elg;vd%?#ae{3bUdIf%ELA!0m7>4E^32Du%(O8rV zITiW@Wre{i_jJA)WwN?Q9gZH>w63k2+Z(yNyL|(QZD_9G=a|oD`29_!6C-H4|5=Hz z!Xfarw2x+VTIr)kTt0Y*N4$Lw;p^;|r$Yc<}D1xa)_RX<^pv*opXBwxrcpWVn0$2)n^9@e2d ztRCJv@XEL5^tQ4$^klnTvV*qK>S}rs`Z1Ws7Fv@5;f0;O*TY&^kj;b3OXs3= zHo?U5SR^CCaf_7<@T%Y(Wx!?v*aLKBM+bKRYnw1F7v10he29`CgLqQP9GlOinqh6f z_%NZaE;evK5mXyMa?<+V9RT<^G88T1E{Bfe(PS38!NHa?uV;I&C2PR94VY&fH2|3| z!NjsF+XBEp6a$e9S|Vv<2KaJKTV)$Oo3n$RS;FJx91u%{a2=@tWNJ+mfXJT=Qel9G@x^TqF<1})IF-j zYOKMOvYJB2m~eFssfA{Ulg0c6%$BqS>a0${`(%gho>awTQaVaxedj6-KsT?ftN|yG z$y&%-^SJ~2(gCsYekRZPcA$JfX3wr8h%D;G@ z9Pn=)aBw3wrk-O4DH#Y1lRPwE=bfw@*9HD9zTx{a#JQ;b8+dM*2l^fz7ft3;z-mc) zmSeKWlZ#7}jX6kdo&$kvR7u(M=P%^t2k+a5=$PqeYS~4uTjon}t=e_4%cg+MHm1@; z{%p=O(4l~gYmk&D4D=Haeb7UggSa}5*H>3osYuskt$f7#w0aIZ$O*>XSy^9i2CuAg z6b=g=NZ9lY;TIcM=J&vPF&3>VdmY-IqIPM3gus*Am*&Ka${ubWn*6=r`CPuZ>vF#+ z1AA%Fm_6onZ%p#q0mMqwihUOlYcrK(S?{0spZ(;2eDs2Mczkyz_S3)cAFm(%(VPDC zHpFdrMv|9n|rCiDP+AI0t{#L#}svh=XJ;~!bEIP4gWa&;|%_ewk zU=4J^OH1|<2F`9)?;BPkY6}*&h2K+eYDaaZV4l&QFbi;V67tw&9a=F6p=A-TZD;lA zEQ@{@BKcSs00FfvU4U2vT9le#vmhWrsgoT_U*#sFmZV|EZEqQNqpes}AiG0B-N{=p2alPx;w74XqYLPj{3vk_nN)$EB zxo3+2}l9Oa3X56 zgaIJ*R6g@226k(lPQKR%ud!`^KeZPT8tx@h!mr)~q_Hr+tqVqi2w;(*+Ud|N4h6Cn z>k`_1Mv5LDj$RpM1+I4w_m+ria)Ntz?8)4cQMAOPiLGkEm@aiKIOE>AiQBmEVl9ZZ z1jjmpBOcWrY6J-76BnOY7(gHz7y~*Y1MX3Db|Rx46UV>jTuN=Q6u|4CqsMF9Ihg6A zg+#AQf;+o6AT*4T29TDpw=N~Hv7rMZf+Q}N7FdZbyH9SvEt)s?U|=~{2k2ret8BoE zPAlhpkqxoo2mf9h^k7tCY+5VqcyZKi?vFL9Cqg#(blz?v4-K-i)*_)dZ6f4WEkBP9 zP#QKn&Wo%)%_t4+we8ALokjOW8+GjEYTKV^!=c@Nn`qfnPVih4yPiKYrglWPFWVDf zUmEfVf&>bnpZOka1k*^gHNQ3k+${kj=L@4Fek#zX8-cTe*qduPJb1ZH2h9T-@;Hd~ z>tZ>`eawSRWT-hi%sIeFh+_fCCO9$fiA9*ZLB}R)nJpv0CUn4^F1?&*x{^JnrPPvu5FZm;E&vpe}{1+f=$ZyF9x z%&3o8zPRw%M1WEOeHa^SEXnLFp~?lcbk-5C=`4GL+`YxKiOmD2ev6yXlb1+=3Koo7Oh7c+U&*!gD|)gFal({XV2s(6p2=6rbkdnXzc6#G{!eJ^2kAfF_STSyL|H(qsjtbUOXpY{nA%>n78( zNdkSAmti;zz=t|mh9xa5CvyM=CQrEz>C6jBpBlgdn5;Ax7~Uh8SU>I(uZ#2BT!Uj2 zmKhmQqjXP3)*O#+K=mw|8I0bct912h)u5upRv5g{EA?Oi255vBBJLb37)P@De!o#K za>?Yr9^*-E@xhYUYSK*A98k7)Z z<$*j$k3NjCIajvg)b^ZaLqB?)*wrn&x?O+0>PE{~y(F&EZf@lkh1F|qN}IpZ0LnV*Tws&ytU%EM(9y0L#9KkD z)`OhQn^@P{*|~vN1!h|HV6Ym^mn&eC6-Xa#!|ENZ+qyoD?r$!K@_RIfWWgE%G|mCA zue=G6-dHrf+1bmG^Pv|V5`h|Fn zSalw+uCL^Nbt(_2qQIK%!;twib`<`yjvb`Hjr&a{byN5OapLq@)+o3Vs-FFn@p4GFMZbI=AT{xp4@yl6d&1+L6vh|znGaNA}RZcMhJ z7vvMsyOWv1Gm?UwZP3JzdjO$>15L1twsnnOmScpb))-hVYqH-JERKaA494PGS_QH! zo*wmcw;Ma~i!a2%4ae9rBhz$3BTB*y2w^qWh26uW9RX-ZfU=S(Hj$wd{WK`8@e2jebdR}6i_i`1qRhVd8f7m5@`-^_}XTFF9$F7et_^_?LQfElMg_$ z;zi-Tg)tN5jA26uqPBX=?0&R>y%m5b-;{iAn1L>^F*v9&VzOaM_{snS;UKm+f;H?7 zN2!HA+sjG$dWo_~7BXIvHELAL^(e`M6ZVP~Ml-Nv9M}h&FJi2Md;hM_WShy&CVpz1 z%epz;?4#DNzYzLDu);k?LI7*TfjvOtl5AEu&>UgAf9{|}s2q%!8?eP3N1@GHV8;;7 z?hBnHcW}H*j?ZTWNjm3lZ|@AevYb!tCg?W8)aSzbtsFDcdbVv;g#dRUsMPSQ>3jxj z!L46<>en~7R<`T%=`(q%VSNCm*kq)^Mp5hSjz^WIwa{ZSWX-wGxg1}yu5oM{!U{Ia z;?K1%vB$CeQyYsj$fybA7tH#vYmNXm<~O>q@$a5z;Y5xOt~&kq5BC9H^~>A8@1kTR zOOR>PnsM)>hT48*V$YzdRg%a)7cA^pSzH~%k&(H)4=e{rLSE?U=@araW&aPYgDX0x;Jhkhj^Y$caArrS6pBGvCJjA_Lv@Y zR)96;Bx-zJtm|_f-=pm0w^soB`SrTq=d*S0yR1kZTiyG;W0&gw-C?(e2w-(jAKw6C z%?zv{_P>4fqHlP7dl35<{*=Hu|Mwzc`3K@@d{vn zaDF8p?yu#E+^l=rLwK|^*(Q+<3%l8L5!S8GXN}h^j!jQ}%TOMJ31JI612)i(3MY;b zXboUiQg%RwMd*FP)3SdfJiWa$u5A6NN1!oM4Up?iml!*KkM!(kL%mlYNIv8=2Oona7{N%ABlsR)Fr_`ZmjaF>Pm=I;IRfi(m56=OJ7+D}} z$4yB(9p5--{qL+pT3Mf=RpSpJw-7L-Gs&c#440PqD>Rgt5J0!XQ1n{>5S4|p4c`6^ z@Oq-RtDsfN&u{ulqa=@?!-MN7=21^3F8&d~V)M^p{&U(3u*{x-%5gy@1Kb~zw9K$a zMXPTh)=J;aksYsdP7A8v-QC-UuMX6_P@j$aG&<1l>zZJaqCj@jS%tX`=u8K#C(d7g zKj?fC2UFH}eSP8mYCB%98???637!Hd`nZ_x7IsNLQ!%u?ruPJe`aRa8 zCp!B26FZME7=9GODj|E}p;jI!6Nm*&x%8Zib-wz%aC_(S27le$(`GL$9S1c&tXAnBOqWMC;VP@wN%ne~LzHlM};8iA!xx?B{Tmkbg2E>fl`C2an-ljNYG2Acu6 z+V=NwVmZ(IR_n=8k7f%El?YgWAU<>10NCEMBw{eZ9TCGN?Fj4OvPQPGXooJjc4N%4 zk`Nf3+Mr}t<8(E2;P02xysh)4_K{y}E(+K{0N*TTmX6a`boHvry z43_Y$Y{IqXzD8{7@LT|}DXJv+km6|%r9i)hA)vz=q=b(eh3E+cmZI_M5N31Za*vq_ zH9~ZY4U>s%**=#w7)@K-T=ehPHn`Wj$2m#@*zDFbquBWznb*rE3|cBW^uQ<_u>nUohI5JM)2Tlc@d^_i#o}>{j>L?{t}!!tQ~R5< z&!eyqHh->6jWNc-{2pVzOXq&ra*k5+VMclm*^Kl8&?+y`X#sm;;Tg6&0-#HF-N7q2=3xVN`QA;*tT6Oxe@ubC7_kVVzSa%_9GP-zKkxP4IeuZ{StG>+fa@*#&8rY}w8dMwBTz zvw5?yvcSO5J^)!4)IVkCDc5&Cm|VrRoj2=-tY5eITz$>`-GjV(^;*7q^=bvO*Ct~y zf}ne*rfWhcFW@p{+v^$EPW#}StA~_BcfCkduIOO!2&<@P^a>hGUIEh-0N3pYqhWcY zCGBu?GJ(2a*4*{x9{hZ9fGb?D9J9sb1k@xS=dAcZhx9mm0-cD1V2e6Y>hz|0NJp~y ze+Co*KKQu5R*`0V$X02iZ~|&R4gIs;kvAC$cFOXhGV)< zo8*hF<8WgAx;*mc%xZLFgdJ8Oq#$$$(8{p>xwe~SOYEVAbE2rKVR+5rH`*}qswH{KhHxMt)8s(>aOPF9YE}b?Are%Kl`tK^U({x z=JBmT?C|Ml|H}H&|KzuQU%VeX>S7tJj@pOiVD(_Xzq^%B&aUNy@wGgWD;fGdHUMYL zHV-mTY+qOyF9KLHkQ<$A&+xGS{h>G+ zjq|Z2h)iH^=A%S5{X!W*57uL7sbw-x8nOT%%TYL|9W-Qe#t0+1rzWd;5`b=ad&mYT zn)koA!Xjdv+hkftNX(kWGg2EjbQn)&es%&lGHT6rh^&d~C5@oB)w#V)!5~YgCN(|dD73#!!J^na zt_I~a5-`M_O3=~WTQ>%7LcLRQU{({K{L*oMT2DL-FvId{+#GEaaa~Ey$6;H2AVO;= zhH(IJdIWH3{3Ay#WW?{RT_O^rIO7A*c;U zJ2jkS=M_|HczOe14_rd*?)wVQ{KTSU}>)dv=5pvDH~ z3QkZb7!5nhdj#hXYjdCl?*5x)uA|(L9%Dvj7M59L=Qz8!0FetZFgOBhcA!!r6y9ed zmpi_*F|leRKrz796=ttIVow=rYuN;fLZFDfWYaLCOnW-ZoB0kS;(%Ae*<~m{97^tK z9K%i~MBpgWxug`pk88(@Yifl6TGD})5-m9i(G%cu9H>X-dd~7xyQsKhtmgWUoxC); zOBvWF3Suw3&a9Q#>SA`#bg}#FUg@7TgyUec6k#XCF_>5qCSK{FB`Ffa-ZST9@%g4z z39hfNjEN=b6(CmUck|C~(5JHFW#^Q@9>v5~lOYBp>_kpGnLnH+1nRZcw%bbDhzgP= z?=R<%RH-KCl1C`-bw00X?Dc(<-~8PxxjL+4vVUUxk4YKV^)uRDF`AU1_Z{ZzH#4yn zj0dQ0@(<)^fBjpP+WPiP?598ex7Yta`PT0fP5`~3mbN1TuXDHD$%}p?pX~MPN?s0M z$$7hxwmbkFCjD<4_>$s%cJau>ne2R%8$+!>*yHc|yn@(TZi=1~L76Y`7J8-zgGzH_ zOSQL<5iDroN9M?7e9i#wk=(_tk*u&w6?}}WrX&F_Z>;2o>_Ypp_#->JZP-(ScCvD{ zts_?>HL1qzu_!Y__e^+WnA($(CX|uN*0Rwyv!_n0vL0lq%?sW6aRHnCdIh2C=FF?E zJMYNVEV<024W^BA0Lw}SwHpdui%8t)SraxAnrPUC>?OJobw7Vk$jC+zTET+aw`mKwJvX>|z`20M;P85XZ{VXPKp zGRb^m?Zuq#;l$9{2!@BUfeJoD3WCP~J$ftQL2QoPL~AvI2qQ)*Ae=>LYfGq+h1$vP z6{oT~mKjo9upa0*5X6v)cg?FOUM?Z;=<^Z`?!0(NI;1H8vkI69U`>aOdFsGKozR_E z34mO{Sl9LU!y<6FyiUpSI9me*1*w{lr4oVLy9c>_IJnOC{$M)W`@{YE+`T1^>1REx zf4{xIv)`|-uT55EoL)jme|4?>9lo^9IhdxQPLQd8j}9w9ECl#b&k2yMwFc@cqI0N z0K+K7*L$oHzNW4_?i+84pfnWZ=ieN}(g_)&4y#3cB*7xDw@KRJ3a;%uS~{YmwVd7i z8+C3(SF62)%60!#psHtecmH6Yb9r$omzPiE89=Od?;`W5_K5rLQqHSkJUhBWj)>+( z)gb1qMAaL3FEP8|oWrSAHdgXtcLuL&=v>LlB_iyoFJ)+1HXB2 zj0ct?ERiZb=aC&tNlPb7+F{#_SfI{YQa+*c=KiRA-kE%PHOv3^d#epN?xl}U4aB;m z&*Kfy!|vhnJyryKBZzfp^SS)&C*P{n*0%$(f9`MnX#F?^vF#gQ`^E>j4FhT@p52ae zvD~g8_R0Y4$Gg|^Y}43V48 z%$Qo=ct(aX?RATUl*i7~O;OVOye5VT{ktcHI7s4Ru&z_1FT`;qu)Qwq2HOr@^UfLO7 zFe;|!b^WtXCTP9%o*+Ctz+0D=%y{oKK1|a}mQx2$jp~BAtdEI~58v)s?i-^cqH9mBvNNdX!GVF5}wuW>L**0>Zd_ z1-7QUJshzj=aF&_kd8b~VK}UkG&b86&n6&H?S=Npvl(pvC zUg)sr0$NTJ9StgVMptU|%Js?vw3b|JTz{TSJBHK9DBOY#lL=uX&o@4g0KY{7CBd+Q zSOe<{Vp~I!Gg#Q&-u|vYUdzepex%In<5LZo=la|m(4BKAjT^T1NwP`>iL02) zF^#rds&OfdZ`CRXJ8@`rx?KUjh6{i6fC4o*><2U0 zG=uj+o-Wt&QeIhUtq)cJdm&e{TY>Gos8(#9Fb4^b(pXjK#%fukniRo1mIec#FA4Gp zep#59p^_+3fgQFLZ*hnP&c&XFiS0T&HIeFnYr@g4xg3qG=Rkc<2a|k2A_PA)lX18y zvP4Ig3Q9dBuQN-_)I=fnW=EnBJVY8$mH^Q&FUm~s*YD4Q@$pRhCBlCGc_QBJYYAOv zhFMW1qNmBCCN(DR0to>On{^2>z~8J2!QMJ_?F_im*bhSHpw1^SiAyFAYynq}&s-fI zwCcP=hZU`{so_&)kPfvZ);uHJl8mK?Et>%c_hjL<4>YbJc_GCDo{v^+Ul@#^m(%qbSISEGC#4lwrh{pxo)D>rtluy`GOhRob%@O zd1jupQqHshSc2G7nNs~;zpRUvB|U<@?LDGJy{8k=!NxYQ^ahR)+2Fi1CUZTWvY^Jm z8bj*NwzErT?riV8V~)-S1x^pzxodKb=EGr<+nd`JVBX2KRx(&YscKvGyH*-dc2$Au z?X9w~>-T$?4LGxUOL=vYZiDGAWV%EXk$fTCym=NGeYnyw# z%&DObPRZ6;gCq!218M2}LP8J>EaVc}m+m?Gum%ViJFh%&X2)m9{^Ih9JXswntxlkG^U|6l>;lBHWaMW2FgQkz zHQ6*4I(2}VufgB*H5?~jzmzzI0z5L$mSzg_{9Mq>HK`b$y1dcg< zoDvFe0kL{He`^J?fBVq^-tzcXAa?llvwwB{_N(9KJ$6=QAUky-267H^Cig3dUBPR6 zCEpva*6%BbZ8z)wdTZ>3_UUcbg^tM-ndlUWO87bSqs3r_=wBA8NnM)EbwtT67W?;e z1(Renth+E47!L20#{(;dWENv7nW@{o9EOF$sda@%Zu>HX&r_eklFhO4=|5&YiuBOv zGJ(lhScX8w6P%n0s%2%I9r%2MUpK+**KGyB|!Z8C?&wz*+M` z=uBpDr0Jn@GOc?D1;UKjA+lE8Q)jJBe;;-*`>ykA<~)T1=Fevquhsg)JfD#LOL>Vm zFTjoIHO$X-FvR_ep97?B!J!)q8pb_$S>cnAds2-y*Y+(t+8W1p{V{4iFq_ zgd~;mI3NtKpQ%4<=>1;VS7l1kz#t~Lf~?UQf0eSR(p1}?8WU*8nhmgC-`*NvH8xkf zd##}H!IH!duIoHZQ5NbrdF|o3!|b&d09EJWegz2%NRR7tqjpPr`BKi5iB-viN=cfp z3+&lW1XXmCB_13q?(BGO=XB|HU=SS4JJx3yE7vw~LvyV>_~^j4>418E03szhixDCmAEF{0hIW(OF{sF?te z(KqNc_e27ebf$7m@tjU;9Y@Xr9Z$?%*O4=v?CW}-4^hcLBte_tx+@9O%j*)vwgA%` zdCo8dDN)yg+w#=14|MPg8ywf)%Q(Pp>0?G>(Fi71A6AWEom={Tm8aBTsBy+W*iS@Mx_LOo1DRV>I(J_Y#`2;}*&<-|4)JVd?#!+h53SJQGUqS|UU=@vy z>KlkEi?>f#hj?7=@c!(XTpf1u|Ni`1KEGLK-R=y>0=W}VvIh}TA8(70)#nP3y;(u* z{~+)9_!b~`9kNe<_di;Hc=7J<@kat!36D|xhVNDodv0C2Udu=0wIz1F*RD)TAj`dN zyb5CX2blZ;hG$8}vC0WH3xu93yW}icl%6HEnDA7!K=djJQWtDW^pNWH7(0OT>4mki z1q0@lfYTmBf{o6yPGABAtPI0n7&*uhcIZ4Z#5vem1uL4NO^(GfC9vm%;czdTX>~sH22ZW|SCiA*vIU9yN?2-vRTCyg6>G?vKd;QNlbPWbxRsVXv z--S-s&oAqso_eGHJ&kKMxWS&yo_I2&IRoc`g+dp0_$iK%a6=!^;uncyEZm)2qJfuQIkyepY6qh>a6{YRf%h4e2nbyLSkyjoemnc+H9B2= z%kY=aW!E!%0>YAA>H_$3yKk}C<#^VNSTa0YAe(DEzmF|9z6OKJ-^1`qVLKW_XEl6B z7MAbdGOiWgFE(Hkiy^*|yW3tVw ztw%J}uqr7JhhSngDsW@nxV)vTvE~hOB$e2#PTsgbbEj!=&7~e+0qp(#(HYLWI|Hyc zcN$?p2`F~%g9Kt?txMMLT-TmDwPz~`J-c{fkpXKjyW896_2xBX$MH;d=lc0)e#`}% z?>hF+Uc4tCe*8W8@WYSggO9$q0@&1!l;ee4HJ;6gg1&Ie_r!>yTiV)o<8DX6jk7CX(1hJehLx z8qyCFhyzSF0uQX3vSvuK(0Db^ufPsKU`zHi);%};e#*p-MV5f*n>A&~HrbUuwYMSP zV>1@wB*1`U5!8?bsB#XU(@WU@ErU1&iB$KYQx=-f?aX;_I~_nBc1AU%YB$ZvTi1j> ztdhYxH_tE5t-RLLb*^dhSI2!L@Rc%_d3{6dr@7-65YMsUeL1xeaNNEA?FwR7r;$u- ztrN@gU|y>gj5Ia3LknBDalU)9z;yC`AM(Ano=Z3YW9ngpv6teA#6FzkuU};8Yvm9E zSxx)Zh^}^f+@Gx|?QW3&X9cletl#^1jv1Ewl8=)*Af<#?InOu!{FcZ2D~SE~zv)GM zygP{f^MB%3*D?Ip-ud--oU$NHP1dZoU7zHv-O95SxPG*|mJf#;dET#O-`9zt%-DRC zc3eIG3GBy^hKGH@31WkeAF?%`3*l#E)*m;8>Ahy^@zLTs5%sCYBw&I$3S7{YiQ)cZWM!KN#=5e*eRTNp@f^Z%Oz0M3C7_DNmbFD4 zj|~Bu%~&8@zqiDxGt|bDSRH1-l~K?xKx|`)#0B$CQA<@5N2v3hH&r#TXoA7>JmCOC z2sx06wGB$V-K>t;9*qqY&_)eeY*=Kid9U;lMit#<#v$2;0QM|_bM#Q{5se5AlQ+RQ z9v|Xf)Q!#-rWzsxii2b>tg+;85z$pKuSXj$O17p6D;Bg&2mO1KPRY{t+t3g#5Y zegSK~#OA_bg;Qj>nEk|2>(Ii8$%fjz>tN)O*cdwz1pI1%oi#vrvQDPE0zLxUTGb)h zB7R^jb68oH6zaV1!{d#F&PHX2vP2e~1}WvPA~}K^+}8LHB{cY^$^^V7`(7&@;69&% zn+{fC!&Y@4KLsX%_ ztv^+N3+ao=7{=7%8heX5^78%n<$FK)C3*kjkLCT3K9Z-;pW8gAWJcekcH;rI*klCy zrEIK8dhR{h)*66#Lza1Ben0_Af-LT^$yG)G?XlY3_1x?}AsSMrpsj&63qztHYwZ}u zd`V9y9ZYwWC0o=9MlI>?G6G}NkvtdW^jfH#xB+g~m_aD)ANd+dCy>rFfl$l85pZmr zIsJ-d>3fPM(z#DPZr56Cv8Qlca!qt+Mq3L>Njj`eC*9r~Nvg#0j ziUvgi#+#Qkj8{@V=xbZ|By}2JeU);8QxI#PHCE>u&P2I)yzj`$3A^M2UoeWub`Al= z3ghR>Kr+{Ql=!>QNbL$*Kfhk&x4t;Ymk)b+SVrIT2^`3n zt3*_fR{+H+Xg#gc@rK7s`9r_{lNHE*^-V9-$Gd~rU;Xhv-@bz$c7wppl^ZNfXp5vD z<+9z&`@^k#Z@iHY$1CgFHO#lGp+^$K>|n2Z^ejBkK4-&%WaQvMvQR$~CME?bYA_Gk zJ&C=qc~kRzC4j57jAIKruxycS2$KkoX2ZgAu?{V(1+WDrLj_z_p{oP1Ge!@Z;VL?J z(bvXe`hHbs#5`6u^0y zIz-rLd5mQi!u1s@KI}AOR|jY zQ*G=bG}BfzenQyq7`iD<(jZ&m+P;Q<@L-Sjsy)>9o5++MeM8=Z1#pJk%NrSpcsR)Q z?TvrkTv;nZElH$$K(D)bXQ5uxAgVUdGQ`*CZttCaTc7jI{9s{TlZteXt2KC?R>~PGE(3Z{bDgMUfH^GrC ztDMmX-ha%Ck<2qUUnHeCYmwU9?lsrIO2Fes2#}jWgOk9uTY+o%0NAX91sY3KLkhzY zV3zAGjf1cC(x?QEs3$9xN+@>yGu9yOgG4lgj;*dME1rMwd#-ar*+^qx^;j*8l7rS} zq6U``EY1!>b7bS}osYDFfmqGgquEX$zomI44XcE)-`Tt#bY82b@X3W-JbxwYUP0{B z=T-vD+1*P6$qi+bSSD)h84fa(+|$)Jg;hp6+-G4~K|)Z7#RCqyY&a?e0I zSZC=O@uB`T`@n*2m!lsChx*OutlYVSTsy* z722e19AK|nU@I^nK2-_eB*3x0LhZ7R8P0F+eq_7P=*DG-#PcM3*ed{Ct81{kdRwbBB9M#(;oSf2`p)F5n=W6j zU!Pxhxm{ewgbT3!#xWh{3GQnEXZT+ddqG}TT%WXhZmNs@-{l=1?+jwUs~+~zap=t` zD>*;NllCAV?{BTm?EAxwT+UY_hg&mLGQ&ev_GBaoG6Wl;Q8J)34g)}%X`#qeqEw7! z{#v;tuU*Iuu9FqSz&iu#p*v5W@C}yQ(OwJ>^4$>~r)BDv**FX|rl=iT+3#Y34Onv6^OE9(;Q6}L}*Cea75(`0Uig2NN}HgHxbtxyXM$IEK6*GiEZ>cwL}r+1U?P~ zS{2Cd%?Uue+~%?kcCfN)3dq9u1R6Lsto3QkjQIT?u{hN*Bg|^q;266d^sL=P@8W&! zHM85oxHocjb1m1`uU0U7<>2+s0jvR91F1*o6dy_}(R(EG+*yd#6R>6^kU5BH(A6?o zPmPzhcC};%RU%4A52PTYcZ z(R*sX=(G$)<2SktB0Uw?XGKTgfqj69H?!3OOv&Pg-Wy<)<`U5uL8TtPNc47D=_TCD1(icgK7d-Q!9sT22fM!` zs!kyM-x`vPGI>gwCHGQYrL|t6E?$37KHqz#{I%sdZA3$+4+TyC0pa_+B#xweYk?xAN=r_vTM;j zkNF68T!7RuaufWE3DOs57uMPFoO6u1t%HM>bD*9}2MR&i23- zXT+<6AfO$sl7+C-wBZA5m?yE|H#seOmLYX)C5U`jEAy`7bs@J8P4136`TJkq$o16Z zp_TiwuF;()Yxli5`LbmiK$5rAz&0k7d{%By$P#lAZe`{`$@QF!^zuYcRau4OSU zVJ{Eze7uv7RuKEq?n++Bwd|+s)f0CqL9{vLBoqQ!K{5JiqvPYI%?(-&4MFUYrJ}$# z3H2_L3yXE#eB%cj2WGFqyJxFSR!Cqq0$~Ue8@ey@o`z5|>)Y$Yb(ENy&^l5gDMtdB z)dmFBHLKatuA}QX7lIpmRRQ-R8K&8zDuIP`Y_WTZ8`czc?{Yov!L!*JdW^~gi~kNn zpuif8X|M*+st=OdFO0*mVC-#7{JGuW31??$im)H)@VFGiJo#3P3y$Z>#AKJzy$cMO zv94BEbB-37QAaiEf%!EI0Igvd$PmWQBaEjrUasSF@T%b?W_jO6T~7xkzTtWHE&>ZK zbrC^@j2Q_w2ZU9<^pQQc9^dM3J{-*n(_`M;-I$ipG~?ps*(`BunqjMb&<9t@VxMa; zYQEtx>Bt!h54Ca~Z4xpVRTxH$6(S)$?`LH$nQRxKc|lE7DW!$L^3;Cp!92mUAHcHa zYbqG`jjnD)v@M*I%m|N}$+f;=t#V1xnXQRic0C^JEhYP03+}aCv$UVvqLjJ>v(at( z3X@)ZLtIivyOg%rXjzPp#EFmgx(C!F?E%hNpPQ`UsU2I-X@vezb=?yxC{U+K+w$IfFXf|; zK9=X}_m}T~V2L-2@NG?c#d-<*0}?05mvU78ond^xelXYSl;W@v#*cNuH34hdVQVR?kQ$-! z3|@EeQ9^QLLLC`k32^4%u?OI?do$#I7R;e^M2{A;{{!wL^|!WPow;OSK$kam_zSS8 z$%d=r_4Mfz%rnTzP)WiDcNhY=)27%wfL@pNVi}TmTPb->WISnGdJpnI$(~oQzOsE_ znw^PfMFoOPVv{OosFKAp1vepg0%`10z#?|jBU(mh7dLHV`=q6Ynoz3hoYU@c&Wbau z8oe=X+hA34CS_wE2KlYe@8yg8b*}fF+araM9r7NPHw^@9@@|hxF~?im`|`8jcqd)# zyMfqW|C@hcJ->hB9Z%_N9u7#PP3xr}dnzMXL9CY6dZto>T`w z==JQ;o0@>#5*8-ZJe;7fosOk+89*x@{bY_N)0QDzf|YQp@+_(<}i4H~=P$AwmF6WT5qkb>I*GQz}wUT%u#Sq3E zAj82Y0QMr8(l@=xw8o_2ydIIZDzMbzl%PpYoTt|FQ z55^_|YV_b{-mq{c?@D5SPx-`q8nCgn>7;C zxFr~gcD6&qUIhknNl95ak7R1=$>8styY`6sqTIhE{>Iw3Q8Ka2>ljlWX9=#)pC5+8 zLM9XJ=Q^X*<6elkw9Mj}X^DG6D^zBobU^~aMO zlZbk#GO(;&zdH zIbVq=hsS8_SOT0!h{z1G!sm0V(m&XXDlf587*|+nl-OkiyEr6(!XVi4E<+>-P%5$CfLJT?fADf+6E>7iaLml! z3SuQ3KHOQ{A7KO@;&QTHQWggeG&W~|nTVzO*S!_A-bHKKJ=r&*##pS5>^P8ZBunVv zpBskXGyfb1t%pc@negAm{fu+XanWz=PhtHGz6`j*Um#>4Iy)# z$2xyaXSp6XJOUUt3-41)=qtonR!UNa0w$&0(_|<#as;LTjAc}Mn3uQ@oChQIUlptd zGuF!SYZtjO7z;XszPOTI3swzmRec0ET3B~?s2yo+04z7Xg-t4!VB@&FYaM5@vK|hd zH_?x99O=x-5cZmIVWA`;Dt|c3t-l`;X;Q4GI!u)jw4F1mO_`}PZMpf}zRoV6$oZ2e zD|kJ~7his5^=%>)qxR(!Dv7*=hJq2V#3RjoxticBRxJJ>%OD*=EVN|F;m!yF~mq}W=InxsY- zQk;p^Z`2;l0TSsN+SwF4?lZpV+7Vf3jV(q~=S#q{+3XwhF}9=uz5SUW9`m?)@GqP` zpQlUMI*FY(X+nm)p4gC;`x! zcwDSVT(6g^Tl}4C zpi|$?SNFc@!rgGQct=$hU^S1;D5(Z^l&gm>zjwXJm$zN+=GB?)>Tv8Nn`M;hKHmw% zO1+-{MSl8|pMBHY-wDJHpMLh^_1hnN$5VRaqcv(Iyx!tje~@SGPCnQv3wtHcr!QrH z{7TyKdNpGA9#&gCiNeYW50%L)C8f%S!v{kt+*}y$x2@e+&K8q5^f5*MQl$y{2!TI5 zufv0fea(QehAKlRU|@Ls9mVD4xGc{zEevaCsMRtm_X9#IWO)5iH`ruuP$w7lKUx08 z>)-muQ@u-F?8>GP$m-vY6BFLNf`f#2=NdS z%Sn+fBC}DW7eYhI`3Z>z6P#w~I>QD?G70PvUhRXBEwasK3<*IbelhgQWQ-be-E5|B z2eHBUHUV%3Sdy~FBY6;|olB1rG02R*6{ zXvE#iXX&xmwX%Xj1+1z|J<~2l&tJ%sr_W@6d0`TOySsyY{^gg}9#(bO3WP`K2}hLb zBTwd43bOKufh`uXXw9C#M>CM4={&oAu3nF7w+*lzT+?k`;@0DT@ZpE@$qzoUW(d!o zJ#*bS!onO=Cc{{x1AYGrxbN&sLF}zNP;*Z(1ogyT?SzsGO`ww=P_-pXI!RwL6iZuN z%Ho-A7@E}ZPTKt;lKmF$HLzN&73-~L-6@XykE$p0)U8Q`R+@+^g0Oh5sU{+KhbD{I z!%2iQC!6P6$jf3r!GYs(TagD)6B%-tU}6#ZihU((qk9BJf?yyBIR&v)5T)cU>9!2f z<21;!G8z$UyF?WsZBkG>qH+%t&m35_2+8}tu(v%yBME!3nv6JEb%NE2(+Gk|%GT#R zy6867&U@$QtRpgpL@D*L#1P|vULA^^_9yB7SqIv>2&q0L0ULq zZya`sJ7`}yju}18-&?^ZHPW=PCp^Lyh6e_U zX6NE;-(n&Wm^uX@=6R2(gSg`corzD_Ma1!kizBU>uPCvQbM+qqlWA00zT3<&0jfqGYGh6bjq(jma}p7GX` zMU6x~L8-YVb+!oK*%dJvuNYoueaGC;2}rr|u?6pr$sz^Jf-#B0;jL~qT*Hq+Y=q4E z)`A;XSF+F_Gp{~Z2<#=2d{D-Y&PR8aoEw-(+L(|*HJ%NA$9D@y9~Wow567rCdN4-M zEz6A1v})7Zp!HjmG2LHWqKntgI&!_bdTljVZ>_x6jH((tTeD1#ao|IS-OHv~a}Bhi z*5~!Qw9$gvhY8NHhIaM!CYM>S_4?W?Z|J#bDEIr{|3rTImw&}dYpF)pI@e*tr)E0Dce?W*4=G|pPDv8{G%zdGML`>p^+**Udi`}O;PbM`Vr zs7^pH8#00PwwWePn~i%BAzHsjbO1ZenDpp==k{OslnEK!-IDaIy|3C)fY=2fHd;gv z2~_FO(cyO;InD?DxbxgvZu9`Pl+ZANhmQr9O$pvcQilDYI{68B!j^%>xsQw>R={+`UJ*7~nb_5?o}c;L=AOJC5n*av6Q0Ac z$7>p*Hv)$}q(6&Gc=T<1lP!Sp4Zb{y5s$?6*h{kv7^60GabhMb(WoW16vP^VBm%&# zwj4>g$f+>k4twYJ!GDhW>6{0hQ+SZ{7q;C9`%tP83@wNl&i5`7O0$Y78=kL9ZIdp* z@o_uorMWDQXmfHWw<|Qb>pS`RSF5x9dfm^K^L5UxHldI1K#pYdhwTS`-Qx`)7LQB$ z>2LhyZ+iPTgV;a&cRpCB@n3rPGfKvzK0-UQUhJ?MgUfy^AB^|%(QqU0_p24eDu8{p z?z(rv5TCJ%pyr*;NESAkQD)T<;KJCHL?;3l_v%}jh+a5(GKNZW;SQ(6+3zbsP^QzI_QQ|P*lQ$F$HLo?xTDw-#$2yoDqfIHjz}kU@;FMAax2as5Lx!}33(?1! zp83e|iMZEpTG*Ypo7M9?8{N2wjj6!E%~8R?MOI$Y$(|dB!U_=df{E|c8bV(k;WcQ1 ze2`Ex&%-X3DrGW)#p+8QX3sRXjktRk69cJs{X4mQ`b;jKTpEDXV_B9mWF0n7U|76k zkpo!Po+P+(mPcF*w>v}WGi4L@{<{^o-$l*N{e}PD2QT$xO%BxSUpMskUcP4;;SOS5 zrr>?v-8wQA}lsB>d1J!+*M(DqfCJ-^6r-dt*#Bkx=icRzlFf>GP`jzPh?vf4(&a?8S>0 z@`E4zKz{hcA6az+eNGuPlTN|ORvYu6n(_4`RV zp=T>_y|C7zXL4a>i}qGZ$+|UJsCgGQGoxy>OLT5q%#GeWk`oREohyc40bq-QDwwyU z$&tK^;k;xci$j1|FSFGbYHE>s8B@SN&+TD4{+UQ933Qvkmy!vXtFW~jcdY;tqkEtO z_Pv0tHrHW-vwGc}=U9}&q`Bo}ZB&*}nnMHkE2_Sf&e?(N?UVn6+}|Ll4x|I)ji4NjzlA;N>{hzqSBIj-Q~$$Tx}+ds&M zawRY2wd@aHt|0b$9WUKjy~s5)=-eQ+(8@5B&W(efC2KEoGITPtTztX_;V)rkF#lH1 zUmwf?_07qP*V{2>zaDO!14~^*I6XTeMF$yktNfgdU0smz*$}>f?BL9bi8o;>p$vcH zy~`BIA511`LKdtCNRJ8!d+#dc@_~J9VtQ;VjiaILhl6YTASGZ)sw}hApgYXaE6Xr( z)AJ37NmVeE!h0bll!UC}?h*pmJc_fi@K8Oh)QZNI;6@D4Hr6oZ!p7#{S)a==DG>g>J$+qz%o{Jz(3jPX2Uyx;uSzfYlav-Vp5TyxF&UEa$xp7D%l6d%8Xw~O7W z8T|Q~*%6(dof#XX2V+`UwMo)(slZ%4c+22cJo6O5*{=WOnl?S=0TsCV-fA>ok@_7B z)Iv@$cq&Q(t9tudJ5S9BC`ev@e)jyCy#DMnIa6OxU7$zrkTkdaOg93bz2V2>KC=*w z3ule_!y*xObn-o`O2{1SWpm6O+ps&GHg%K@bO6oIgE22?8X_e!2KGB@#=shrb@g-b zd4erCKDS?Mn6|a?I-`i{BLm=+*7xrhh0~@-= z)FI}9o};JpXO}14yS>_trqvgKs5)u)FBW4-Z7j1Rlm#HOb-r>HK#dA{_ZfD5scxt` z9s}CYgVcV;J#R3~$1ayTdUUM<__R66cE%dqSzUm38oRgmI9tZ-7vK3pzW4nvz$DDD#_MzHV=><#7S9+Me_M+c)y=?OVB7fUAMo%L}<%e!stejCpTC?(+TQ zbSvl2&gAUuM4l}Hqng^E>G!j9cTrGQb_cc&jI7r=;CunD!jo7q9*2S;7;E+5ZF~vn zdS+HvH6t=+2Q2R9;_SP7{Z1w`GO5txY%&nLeu51h!Ls1o*lIfV92o;3(p|vr6S8Q0 zFp%R|r^YeBAI7I6i6*TvQ$hIv!E1t0&HU-V_vl>6HM|T1?CJdX%vy$qw9+G%n&g_} zJBW^nqtB;rSAb+iyFp8O~2sJ8N$VC?fPIeY1;u_m6EbWg4DP%lGVK)1>Wl)R;@a5a#ajBhao{4EtgM$x zWmL?lFR=>QYZh|3#wS`vB7Hm|ShB7=0I;Te1AtW!;f$zO$?oLz{JHsQzIgpwUM^p+ zKmWpd#A{OoJr>m=s#hRAP)$;@-QfK_N=={E7X8i@ysB5{xIHmnPL%{$H?{313Yl7U zkP-G=2(0*7n!?p|?#q|2m%l%8hHkrwcDTkSE0}+DTd;!_^Ud%tZO{}Z3sQ$<)|t8t zjdl^C{+M0F`1{pZ?dM0#o8kpN!p$s$I5Mw7BXXpOnG&{zpR8J9fYs;3+AuRExc(BZ zN&qNvtds}MEb5}Ec~*H#lWyM4`8P*$ijD_?Co3Uez-SQR(l> zjzgY81{-bxJS*Hs2u!@ZvWlU8Mt0{Gr5h#HD^84Rg#^3kkyWjb)F=6_553rxR zxxSXm5AWpS-8=dEtFPqEH($v&zxZkavTx+_{lx-iFWh5uxgMhhuHr4Iw}S;|nqheH z@|iqe0<`nG}C+rfmU^aTu6@{|(eBx@tEo|0XwIZx8fK#T^&7`14CjBxd_7mXt zI7j*4|ouZjXnDc#zLqJn!OJ|G<^n}`J9&STXI5V;^Uz^?XfMQ2Qt}@CwA`lNA6I>a*l%p=qnB9_N`%ws=>w3GE{1 z8KB(3jse&v0I$t9AZ1aGIJmpeWs(nQA)Yijl&K+*rENHWg$}%4BZtvy#`c|$kHLZSa70&mTuAgFKRhIVFEZpu~a-fW# z7FRvGU$M29RSAN!feX0Q_qe*eviF(UE;{5iGd5sW@3#O(Q4cWSsIll2T)SF4JG2LY z)hvm!qOV`SlGmRtU(48g`SOL!sFrc2(g9^)Z*Q*T;{99suz=V%-+V0}-oBN0@7@{% zd%J-5$NO7b&s7g=1zbb%wQD=dvuEcEfL-R=vjwm&pQV8HbQ#-T$a5N?MIBCDuH?sW zd&ZEU)fw7R*cq@4ED}r*5G=l+5C>y{c1qSRcYlbQ3Y1oAJQ2gaD6D?QV(9)L}*@mYN#G{`^{q?tT!vx$t4knGsFhp~3|8fL6%BrjArxZt>pwqRK57XAHy@&qjv0p znE3R1#Ujtvpu>>GA~Eo%DBy`mTKbKHDb$M8M)L|wbX_TE>6W0m$U0ca38I!H)DwA! zsZp2`rdO(`lphuV>zWei&hexqC z)X@Qjg9@mlwfWmRjl0B5LViCr?_i!v8>M4du!JW4OnZRg;{R9UFvMH$vMvN?#I z0?anF#ax1}lQZ+;ROUxn*k=oXeg5L5rR}u>TNhZeH19sVlgq11xARd(YvS%v)Ndw@BiRSdG-2p z1KYmApV{YGw=r?sE)7`1EBBBd!DP^}Z37zuY`Qt57qcPY8fSSUNE3E7$+$hTx;94w(2#R7Kc3^}>8 zFc!f+{;lBgbO8;jhkgC(wLDwKou>O0INxa3x+SRluz*)(U%&b4YkB+4H_P{5$oqG1 zmgnEe&GP%lM^C|Ss@of#E_3E|8T%#~I6qxL@$+R&zcetbUZ6Ys-I+-kwpQTfV3%Is zL+yLXqY|C|&pcO*#|)~vaa2mogVv3Y2>e;h2kpGCBVu-Cw!Xl&%nC&D{#>;P-n=GY z2>ZAklO=#kiL3+3mjpT`5jN@eax!1Z=fj2k;N(_bv^P@j-^#Rr*u(xothz~PCpDRNvLH=S z0KmM9RBulu;;kL$)nd>P;dv{0Oe8D_y@ixM>J!Jz0d2 zBR+_%y9-`>z=C0*#?sC-0bg5tQU4n0p;m;NTZ+YbtOUc}v-!k9EZ-LkpcRooMqnMM z0AjuO`yt0!Te;Ar-llu*s5T-dQKlghw1nk~y}&m?WYS~P+29;TY6qz?4!E!5hhqB) z(Xe1aPCXAl)5)<3uNCNGvBBu3T&tyvD%}wVOObkk7B^OXC}XNjtoD~bUp8wk7CYIU*!}bQ zpB4N}i`wFHcm&hX3{Zdh!#^djKL2bvHh@?yaL{7D!9i?;C6ERRBN}qBJ5uT01pxfX z2D$EPfK5|20`Z>R>_k-~{Z95HQjy{VD%XY@5qEUuD(F@>IGa}3{&Dbo(J$;XSsrfo z-MkGCey8+oIk z_3Lls>t8Iub@}(zhYt%Fe`kPOS=wnCzj_bGN>?7Hl-0nmkqcQOxZwKeD_YlLI#Z-14vuev#ieZlv2r2b-Q36zPwwRG@Qt+FH`b*~H)Kx* zA%)r4Hv*OPI%u>4^x+{uESXL3&NRp1?6ou#n8Az${b*9>Be9s2ZqU0$mUj?g1tf2@ znRIJ!dn>?zJxW(0xB{_YV<-5Hde3oBxmEz34XP!Fp+-T<8mQ$lI*?7qmK}^VC9TY> zv#?w9s7q-=0ki9@YyYJhNxu$r+_zdfcbb}uaI8Abmm@Cvc8vHJiQ*zhAXegqu1f*~ zJ3u?MLCTK>Uk-e+9Gh;>i@LZN^B#uIta#Cp_QIwA3?8GJWG?_p1Ee#3+D#AU*2u&UM^LKc;m7dUnF;SZ&?JO3#R+v*3jd* zE|+Y^XiSqADxipIhISVVe1*=+f}#aH`voqx(SP~n59Il)*9L$`n1T%>W~)9p92X>j;b?k#fur;e1C5l0rxH4 zhaT1nU>gC@6oC)@cN|E}emz?@dEk0y9knLc7B)zZbZne0V^u+{0tbC>wHi{l0tK*N zfAtIb=9{nO?VE2Lz+Qb=!1@Bxm#?O2$7NIBEo15VGnEHyt!V4~^h{nnUjXa^SQWHB zfB8&aywHXYXBIS`+T7SU+ot2NcI$OyZAuh)WdK%CAX-wt$+3tr!Aytn8a1vvejhEH za`1M(Gvr*x2+m$Sv!BqGcKLcRD?VQfS%k=9E)o(2s5w5Lz_JI4$Uaa?U_)>YE^P&D zKhl0%SHo@`c1|2z5Bkr}&XW#SZBObPcUrKgNt(phkc~;gUaQMmG5WM-l2m(L(N^3(hlNuhn|vR2JGm?@^I+ZEkrxl*F|PF<`2Iemwq06e?;`V9 zeAaq_Cxd*f6YxA%CiW5?yA+@3IocSs7PuUv9;o}g9wTTnIx33de7zg`F(3>`a?ir5 z5^}8GImnu4C4X!$;uv-(9*H1mEsIK)+1I~=?@tJVeXQs`*~;Ctm2a*m`R2CD&9sxf z>_M( zB|2&M3$(tH?~NDo<>{@QJ-(6o?!B?FZU^I4>;5;GY|T3*nZf(y{k4OMooBEJ&QREZ z7`l|B?a`C#HW5`a9pWq$CysY?@wTj+=qFAVmgi&I#=SFO`#syTq9MRhbh@&0YApss zm?WcR>K1Kay`H76(-mDX$Xi|@SqO^~unnfcsa+ScT4P``{gTC8ZnS^cA7TSFkl)=) z7UIl(R2cV_mRBMf<#9obm&9*V%Hxvn?dXxI0@rb@=)|(|HiIK{YXhrX=U@|Rfex3L zS~!9AT0r;f^___LlaWwOnx+1DP0X3VN3to!nuC2R8a>^o1y?(3?@8v>^r|NwbZh`9 z)#X7`ych4^8;Ip1ty)@joS>|P7E#6+scs94gpI&DnwHiottC@Mhpy`!jrjFLy`8Ru zPJQb9*$esd2S1RPuV0znmW_it0_1pO4n5B4;n51_bVHxa*V1i+Slnk-%FtsHz&Z<3 zjWHgeRTWqm^~?%g9j2D7CSgFxNs2S@WK``K(ZMG5uw-I=BM-3JS<|FL1sQv1g&C0b zK94CgXc(#ZA``()HG)on@J7jiputz?Qb52k=fC|EIv{~{lIcraFrmA{%sJUL7;(wT zXvI|(VBXS>wIZqZI^ zX9_|`e>V(>*eJNbXZ3Cpt+^IfnN~Jsu`JNY-_KSfu-k5tc^p9M4Cn^wWRxB!Gq0Ew z@p^)HTa#KArjnE*$eul|8!`g#+BsW89j%ABzrVF0(vI1jWQLRr>akR9X}^u*=>vlJ z_Y*9v2d^bbGXB1H`|=9(=KZX|U~~6&(g2 zr&<{>2unO)Bd8jL_Q9H}9NJ#a+Jl_!-^#C?T+0tn?_|FGV>#Skn#un3_+a2>tL3TU zCiB+76}CUdJOOlB3r!I>O}DY{8dM{^nLa=V99!7u117H*t2k5SHb)+*kI+mxBeX@Y z<^i7_O%_z|0xjtTmjqgU*-}^~s~x%F6(1<-t7z>{p{-NUy0dh%Yq}IX8{i1p!Gz5* zQ;~(0v{-y-MKpF%sb^)$OI1&qYZdJt8|I^T9XsG0lI0;oQ3L!R5eo5NAo3W7ds<-MVyqaPzV5X&NXUz?k2= zLAdb1J+;wCQ3N)A1#i20L%cU1+EOv z!dts#z2Da81MJ1{)cfn}XS7mp3DNZB5G8()lEc z*8&*(Ac7=2nI-1x$j(lkRvT=kId|x0Ry`*{;BNR8K*X*W^k){N>Dr@gDIukzUOwQfDYpvb3?$s7J_oWgmw)!LxsJcw+B=kih|Jf?p=HK~F5#g5*wX*-N?UCE48p&Wv2YiLTbjztjJ zOI9)nat$Z%tKVDq_V&Q+@!+~wyPUJ2p@_?wln>d@=~7_KtgVg667>NTgNUy6ERVsZ z8Yq^c78Erh5sU$>6acXS#b(8z|egI)qhhYuDsTc$&^u-4hmwduDDh&7VP z)_dRdjL}~9H!(tB!unU&g!HQs|V!GUK^har(NS*ol*zq8ZVApJah;*2x2GQJ^p*7O8|=P2@CBPPBxAxze<;Z= z*jM#2kivM8EcAlLM<-(QG_%>>%{$T-%%k?dzUkjJLV8KR9Vq;Um?)0rcd&b54vL6g6AExw2*6%hJ zm;2}ooY&Skrb?h#nR68SGUtL-ujecnX(ES}u>?tkLasB8W&ZrM{ zf6!6rZE6K#CVSWfBiq1~`}I2n)53tkbXQVhjVc(c?t42%=EPdkeIFG@r|)P4^( zW}KDhc;`IEFb)8VbhvOVH)9%|fa5wa$KM8VD*1D0^bEwZ(1)*S*0qm=-{(3&L@>Z2 zO!3jLAS_xAqbYy_C@t7@5W0Za z<=CH}tA_TOv8?)~#ajAxwmh#L=lr`Dah*Xza0&oRiN#Q}^(pm+1hdp-mXd24*9x-8 zK5lyOmR61Z{UzWbTTe|ri&wp{3+@E=yXH)+5(Oykn!}%WJx=>(2R0V5=2GUsbFeOI z94`SLb1jawg3>7l9YWdFnnC@jUXEs-v*v(3#%X`rOI~)l(K1&Hq z7a+A8znR_g%q1~L%4F67%Idvi^soPBLJ;n9Hzf#tfiwF%od?SY-X1pc)wS5yz1nQ) z>dFK_sZV|`K7BRzR^#aD$LI3be(yh%Klo3c#KoTmv48Gw{)Ghu{q|Fouy0BFf3mQ= z3>od2VFo*<*mGJ;0uU?L^4*hL`D|Pw^{a1Wy1((1K+`ny;Pyeq8AJzT0*ez*!_%F> z(=4tlv_XheL8J(zt@Lq?M^Av+MIr{8N@|a6q0=*hVZJk%j}8Dk(BWW*05hveY4_E@ zNSPfHv^2x$piF^>TE-~zVhgWDf+*B@^mBC2*9}YR;O9MVW zddD$uoBQZCO8~J1VqEOP+*uOnvzX|3@yp}|zol)Bvk1_?f>pCLWY;ao?QSX}XlRE- zV2wj*Na;2!M8eI*{vNRpkuu?N*!d%Pn9y8;*MGuBWSz<;R?|cCr032>)xeX^+=?zg zB5nsZpov8FG@N)qqAxIC%-dTR_%GL)vu+Ny&BvN+4;^s0T~At?^fD0Z2y>WL;MKnl zVNR}#wr>2dUcHjfUVr8dGr-hdUtY=kcU~Q?I>ism&+p!RBbV<#SVg<~cit^u$|UT~ z!2ZEqDtO;5SOKKSfpZi|;bBMWR=N%g}L8y4Z2)!7`FpD^R1M8kUF1{%~ zhU_|4j6jS<8vKOcWP^sZhhI<`X( zLcSwHgi!taPFr)T?UeP6ch+3n6_nz}zfF(7%qd|C0}^1Od-jlJ5LJYZ+3ThpFG-lEK}y~pW^6;`P}@pKFA{WwOm z4rN>Qb@sJ@(3j7i$xCHb)t2trxwEP&8&Ghnz}B1w=tQ8;1JIuNF)8>}3Bhig(k`y| z1FX2#)>&+tBG}kDetO`N+Swk&@;(wUL`TJv`uYQk>bSlf2x3)AaPYO|ewnWZVh>Xk zb>;o;$O!Quh6Am2W6v9kfQby!qOOjU44v&aT?)9(4BkvmnMAWBntrWVWl2L>M-Kbp|x5Kx!^Pw-wW# zsB49GF+6cM7=Ton(S(Ff6+*^1!C%@>tsQk5*;whM1ZJh?g0D8NClb35L^@_o7qUVV9S9qD6$Skn}C7GE~d zYr(==yg#-$1Hww}snL^9Z(M^t(O@Y;UC_7Nt+2>N-(NK~3T*7|S*NXSL~2~jVln0< zuPA0hv50;@u;u8%JM7GCRXe$jNa~7{c`c0Wapycu;UH}|UI14!itMEgeOIMcq$?|Z z8ky?j^Gs_AHlKq94qNzxa&18&%FxcS5;HS}L&xM_rY12*hvY1tP= zR02XWhdyT+P*GcK=YgUnpPyO4(64M9L$(li1vGZotUJj%4cBktP5NT8quOAPfD(KJ zVkuqmW{ZplH+?c$f?(DBheww=Rf5>viGe`_o*P=S33TS81IjaiR|40wo$Faoy{cPT z&J*}-_TH--Fs>&%mjO7a+`1E87*+?a?k-?oR!zQj zP<-P5_G1FDR{JdI2dT-CKEe_JHf-=+s~Q<&75mwmG}_A+GOMjp!#Fx{2=>QDqkr$2S3ly!V+ZR zxaW?H;QRIa$WxC#qc)(~fiGa)XT%vLxFZNB&gz4-__t5k_;=qjoKk`s1zYL5Ga_T7 z0o4FU6%tr>EeB_Gy(49HuUG@IK2AqulE?_#m{q-b$@3^k{Y%UF9^J<(_csK_(r`We ze(1Nob<4YT$Dd^#)isEFTvbZ-xr?S3d~abFW8St31Hd--EOg2-_uVs zoa;x#a~`lcPOg@}nlZ)<#46ENM!A=re09Bm*bjTTpBE6@HWut_rtVW;^Pl{~r+)PJNS^x|NZub~OO%L}T6lwg4eRk3?}kaP8jUBK%X z+beliucY0+lW765+FovC+L`T5^>a|u!}hCb57SVQS`pEWLU7YZIr@lVn3#{ryz~o3 zQu;jZW=w2U@gEM+{i9Dq+E*nLhz&K9sS(}9AVj6&cIT697?OEY_Rb1?&=1ZQO_6eKD-g>wAW8_}K@f_GZJ)jViAS+w4;uwnuHCzHlKjLJjKufY~4Zw-fw1X9iXqJu^6E+-n)*+B3 ze`e#ug0yC-(z#o}=>*~9-&0UbSD8)~tv_d&gJ(L;7X@9!ObZJ@YG(As$FKo{-OkhX z6^7UytJwosf?fq1%2u%_w^oOLc>mri!@vH;AIqDs7XbT>+8e!>>k9|16D-wc({*`Q zI2{ONwM=`cn*rE1I%t^tg%;oK+yz6^(xz*JXYUL%^$9Gvv_6f=bS^zR5fqKsxT+VT z1w%7>214v_oK0y6c<5hQVpHTz1MmR*Y3l)zE3ixtVEdG@&3OJPXKxpHSe0+f%_}H> zyEj?+eGFpz^sTH|Ijl!a<2GmV1fT0YD99*d{&zu}vsu2mYjD9-yGaC;MFoJ>maUqe zw^Y3bJLe8C&)LEiUzXNCJGbZ0na1Bb$b4phdy2*>E?qEyyBPsQclI7;x21l{26i{l zvF=P#V1KL3?gsLq!QZ3em}!;e2J|D&r*_~K##F|>0>Bnn^c7!A%()>LZ^>XAYuU^V zv^uOL)7EHq-LGgf?(6LL!&V>2lepd-;0oay*n@#uZ9Ds5ezG1U=UtD@LRg$j!A4sR zi45>dA>z(`k^7U4*UlMa1!QM*-oj}ev#_iV=*b9<*@ojakT|FNs5x)6mYpVp#m#*% zT9t=m4B>hWki_Xo#?_7oQo9NV6gn{ssQVZ;VEijRbxYVbR?Tc;{m{QNe{IMCx69b= zJ5_QoisEk^ymidvYee91krK6n`Dum4K9KwhBB(VGtJybzST9odd%PL8Xk=6azGJ!Y=M-_gOje3W7Fd4YeE}pKna8dgyAs(lQet z+E%`~Y4X+Oa_;->GXC`(^lOmT&!r;_*;}Oj$sgsb1;qYOPyOc8AokaP>_uHImtBIJ z!l_p(m{@%{+qwp~03Jm_>|S>5PF~E{^2K;7pO2Sv-Y%uxX`_RCfENcwnvyi>_l&}< zIa8^&s3_A?nRa6}Vn%p?E$*e<@+8mYQy2r`2HS~qG2yWb9QV%)!Yfggx99*YEPe_2By<2*vFJ!Nu zO1qKBBIykndcnIFQ*mL58C^Mvax$h-HU=)&Y;m@Z3$swCCzATt>HvKl(7DSb0vLB( z9F1u{NU&}t)58((W778iV1OIp1M4ig7iH-40CYf$zph(L0Lvlm8_P11Hz=g@)eY=n zF_2bqY_w^ymV?lf>ff^E@QiEj+S1~{74;bUO5d~x8_!!~-Z!k5Ocq05J28#rMK`4@ z>N72>7I9r(8vAM%M_+wq%&W==Zmuuo#)`Y{%+JqMqE^7Q_lj`?7ED`j4SXm#c9w4& zUEspk*cM(B?Sz!TfUIDFpL;DHHnl2QTvPr2uoba%&)aj*vK6E@^XCK~1akx;jxX0> zBrJ`vmiZj-0-g><*eK(M&;r^9#qV=%iuVr?yVBNj#Bp;k*em-qa*o-QCN;3g4U#1v zl&#WHC9h|Tc{8Nt8H(hTt=*mKy@QMZEa;g6&2x7HI6F!9_GIT00=KW)c{2eNQmGd8 z6d>1JREBPh=o+@&!rS%Sz_J746F9sSa~jcrM`Z?Ev;T7dI)=0;?T2iG2lF->z(p{u zL2hwvJr!G-{etrXg5OdZJm(rjO)r^322-_UIz9lFnBTzlduV(>xwL;#v#h5p3_9UM4TvJ;=g z^8_w24-m)=Ou;ugHo(HNu=^S3AweJY?7R;J-5l4!1Q#>jHyif{mq-n8LJ4=F%8pz< zeg=>ku@9@vYakf*xoYey>RNPsk6`Ip$i=-RZn_BMIb%rjnCVPXi>w##S8v_t?cue^ zc8|8iIwl4L>Mr{M6I+v!<^3|YxdXAs0JcDu*T=rc%)P`q)7E38rR8NmK6n?7#~%Mp zcN!spwM(FDlWVS3V(=gvxp=Jd?pEaLzM6MzDXNXFA|;Q5m@o(U(Y!1D3qSEtX!VKW zuBSol@3txzfBaubSJV{~3InC!V3MkCc9LA8J=E2=SHNd7~0CoZFbC7sjb`bM%3O`AXaes>OhG?n(8x1a3-u^L&#q1eyw^& zHwUSU;%8&SV&Y2cHn3{oX=~>SI}}957X8YM$!{M50Fl?ez!s}8P|$=2f`?i%t&x?T z73SvyD||X<1spT6ItnFy5rD3Cnqxf0f(3cYgf3`$A0PG)QIw@J6xY52fVURRxQ*{w z5cNwimkr~5hM|73ERO5J_5NuQ4X>>f-@vEkc=V9f^j=$ex}p571!Fch?L7~6FHV;+ zrPaI}fD+3LfK}UWcFx9~yM|F2JWVNEig|}Vi?OG1z>iu#H*IVBk6;rnT(SgcH~CHsU@rN+%%AfA?cmVr=q#!!{5Z(CCUK31X!qS8OfC z1 zytZ~0ZM=c8e&x^oqwx3rb6C3>OxqZ>pxNh9LpQT|!E36Ez1j~8h%NH|THOkCtvVBC zLO*)MvHgExV&yaWYrpq*o+|Eo8pQtTzw)cg$^3ik&(BGesS%Ov%rFT}6->(&q3vue z>_+Xi7VQ3+T*~L=LSBtma=L)n`To*$c%l)CP#eQ^H^AtO^DYfl&lCLLvO3l#r|lZ~w8LurOpbtSP(4+|B9`sw7FK4PGq@#JM9;#_-IhS{ z1_-{HWM{9wSOJU!jx(4wp|y?&SPou&Be7JL=~?`hQ({^>eq~%Nkg`ftwOG6NBENmM z9CpuLJU98+@qZt@JCT7&`!H{z93U!&j6~mQj86;Tl%=3|t6pz3B52bbW8Ovg#j54^ z4-W>M70l}AtL2&XqE~jJRx6$~#!NMbgEu|UlrdQ}E5JeOSqpO9WPXPR8kreLC6P-m z^aSQEKoYJMg^__@({QX97=W@+tj309x>rgLAS)@={=&hjjy-s%i zdi&X=B@S?n!8OUi##~lZLa<{Kf>IKILNcW9dt%1|U`u9qOm@#RFlU!!EfW_`Y<&=KDgWYMQG)M|IdJ;>Ib zaFy_DVvRxcIJO9eg@KdXRqB}4xnSDv4IBXiRu?qB@%_)7Ku}C30|6|9BjTT9783!| z>@A744uUli`knVM+H`jkD%R}8@ov%t>Z8KDl4KiJnRW3g50v+>rG=IsY>sD+!Tynw znie2wMj+(t6r`3Qa!PX-iK&fpH*e+b^+CS5Sk8Mokv5)L2Hn@h662?vge&qZ9z0Je z>vPZFZRD$`i@Tl%u|N8E{^jKj{_6TO@t^^5UX@X>hOaQUxaw}id8~Ln! zkeBsRHq*6~!<}!!#l76jyV1*WWM-#{S+FSih4o^R1g~LPfrX4m$wThD@pNYdP`0+a z;OjZ5riuF37B?G3(cR`(f(babzFktl9&<)F7<1D>b64g?Un zs7e5OWmQbKR$DN;Vw)u zQy-<9_0~6d(_n!CGDdh{#4KJwhoDA~Ml$gF=&e6ZFRk}MyP)ayhYgGMu%Fw?Fu!T> z9<>ZyK%jYOkPfiYn14G2gVEg0p;BrA_N4eIQ8t?Zr)RT~M7eK<%wPcy)5-*|v@xP=qtX3AP#N6_96A=#E`ODNo)ECcre1oxoAltVL)xeH4hGSi z9b>Fv#o#>P4f2`Nf((RToM;H9b_+n$ zH)-HKswMqagMr0HZ}(bK_OVLU*SrU@rgW@WDO~}UB&eH1BIFC2efm6`{7XTr7IHnB zf4SRhDI0fSCr3fJQ;Hjp#*s#7PySxUQnJK6dps%!$F{a%Wr}g8s1I59% z^=<@we{qJJ5~XA9W-!Ac+vvas=T3~z(mh4khu|+j;mp{<)-X00qN-s4$ctm)I_?{= z-}BKW2j&|KKxpe4_R$SGZ_*Kl2N;K494NC;%>9NFP&wc{D%9lljy-rXJ**Z`-`;wG z7q4j+!B&ZVsJmt%)$3m0>z6Z-S+l@#?-~{SxaLP-^|-L>7(fPqm0UmVe)D}8aLrDr z*cxvc&yCz2MtOI&m#;4-xt%x5b7w|pYDT>TNF;>cx$I)NcCm&_{U*%^%`KWW( z(;)WOfBc)v$Nk0r4L=64I57hmEr6`QmReZ{vE%$GyLK<9?M}WoT+3(mgPh5QjQcC8 zZ4b~}-5lJVJe`pcOrAMY^|WZR&2%U$yCz0y98~P}&33drkF>Zc&PE^|r!Sdi7DVks z-^hh>7`T8kI-l^K4*cxtZbSCRjH02EzzvNQvJIqf^aTYqUPN;Oz3;e6KMDp#I3J+U z=YZpN+bHkOU2#>oP(IFfPe7NjDPrgCN|H(%AqM(ABF7L!uvRhF>YIQj#H?6M*EUX~sM2l@)_o$%teCHMF&(5D&o6i#f zmK^}BR;DW(d&Vkut5PQ@(d#w?Yv-cw>}zMORp!|ATHL$3TA*3nC7nfHiE3rPm&4HM zzXyU=1Ss)uYzP(E6^`G+d0DItaR3i-KVV_pFmr5hLzW0o8AMFo!p=ZWf-wlH?TDr2 z_ce$WfK9F;5=<;dX`y~47k#!~Mm8d~pE)O(V$a43#r{kSz8svUUDgg`+t?}1hG6dP zHUl>N0qK8aNdN+N+f5YAY|ws~VAucNpW6)XG-6}4xV&NHb3+2{a4u1{ZAfb|GO;7b zdrMU#fHj=j*yO_R)ySYG_NXc9AVLRUCdMXyX`2%Q0~%}8%6kKrvm%gTEe7Oh>bS$W zY%&l_@CpxZ0NV$Z6_J6*eJyYru)`dXrD{&l&__Dwf8)SxvQSICN@6%DF9uL z{@wjvKK)qU^`Eq#{p24UKmQ3P_P73n<<0-h#~*(DaV63kH3qO>G)K2q+h(o=TR9jL z`Z1C>E^gEjWFSf%0`xY3RQd{fYGfN=iGbUV4wSu_ zm>ZL^iAAK>6ZvM$Ds|c|WgFfJDQ32?@V8V?9%2@MO(=!-%rvCz4VlY~owT_amW$b6?J%Y$y;1cpj`rYumd7Sj!`VH=Q|)vs;%{Q3Fm@!PfDdof*s*3z`InijSyRq@3EJ-ORq zw$3M(`qV{QN-(JI#^II;wSkS65|Rs*0-RWD-YUD%Q@dNFa22>J=v1avf#|cd6O#d) zY(3qp0QSVZJdpy%IP{eB1}+2}%K*49=K$RsuU``u24ab`FZxlsoF-ZG-mxPq>v8SJ z8k>3#YR#Jx44hj7O*jAY}f zdA--F$-}uCIK&*a;BoK_#4xh;vu8A!w?! za`Wq9ukF&=LrE}HNT1XMWYNwS*QrR}JC54!=3)KHO}%yjch5NLiy~vy7_fQP%trgn zHngsbRVjh40S^z81+mtib|x)Qu>RlzAvkg`>xg9S_-X(xuy8R)YGjjo;FDuAFd4?R zB=abFzOgRGLN5WFdRbkL>*JuR1F`WwgzCNqlH?9-h=6m_rf@}>}i)&18>&Rmp z$!g;GwYBRAGRm#qb{?#SMcc**A>S9e>wtYCGdn|cMPuP$oNLcu;7qxuxVF}vX%xKP z-rg7!OAqz|>)HU{{Qg0f`S>2JSa=2ja-8Mq)(v8zntx}(N|6rlWU#W{?-7Wsg7rMl zA`JKea_wgggo->&qg>uKd3!a>#Y2^QIW>kBk=sX)V-Wk%??3q)`|Vr#+3)@S@v~2Z z*nj@F{?p~dUVZ%G{f8xp1-~It@MY7`2c&}7aRI73d5~u+Cr~ND@^#WKq&;5Ae7Lv1 zy!)xT{;#Qqa}Pa}z88zgJaSqaREtnmMExuQtov`eCUnMvMNzd^XRar2__QJNo}e@2 zV(790cwkQ1Y=J;FX@9?a`tyW#gnSPdn{UFyP799=Usu;D4I4(lW2f6H5h03Sj$n)P zJRk8{K4~h>EkLYZAM2(wTbn#82r^18A5RW_dSsej7o&!I!G#b3OUtM<7b+|SF?J{) z=)f?p8!L@t6DftsLubXcUIg6J*iHH)H3cTW(7p#LOj^8b z(CNj~Q!Etlf*CSP;cZKR)ZhiH_hRyNrrrZH^PZg@XR#KwyiBQj0vKCe<3{i7$p)#_ z6If|^rT6I7;mR8g3+|F9&8Md7YuI4)q&n# z+K`VLU?rI(iG~<`F(WH)aL3w9n-n24l?6CMfauLRLW_lQGaUZ5ZrPB43XS;wvixD!NS7xqdp=>GX= zt};_7=Pn3N2>VJf)P^j?h--C3DzM61|D@(Vlxb#ytsJBMTnVt$fu;bbMBikNImSyj znk%ik<{*6w@qoRrfbnF&#-Ng!vdnUh7-Kv(-G6CEN$COs-UcoO?&B=xelbFtXL@h- zhuy*+5V~t^{t7##c%aNHuHn*%A-85TG;hLUbOeJnLaEa_F4}62kXgumFv5;Bfns5Ezseft-Q9O+{5ZnZiW5sJR z_pVC7%FRBC$(Vu}GQzE7?#3gXmZO)@P*H|LMqaoPfT68Jj1jVeX7_@M*$vXB(e|=} zsWL|e=$ixIA&hGRXN4>}GC$Vh%{}Zr)E^cB7Hx*CC~0u^RY9h*s|HrL-VMpp|7xFf z=JrTu@aa4~aSf{#&W!ZrM5>qGn>Cr=0lHb*G}VqZBLR?ep#-=~mXaIT6#tI7DH7~J zEt>!>QDidkI_D;mjO-BYB5VIUvQpTz=w0mKA8aMZaFYnO$fl5mW47YBfG6kldh;jn zaFD<`)>cdx=Uq8sZTMQwAWEK!i&TL9DW|<_U|s zg!}tPxxaffnL&bB@lJ-pgo(NDAk;ykg8?CdAZiloh&`s{+T~G9*Wv33C^U{40L|ku zUXV@?jj*h)VT}9r74>5;e5wo1#IIj3S;#D62bOuPIqZFJ1k0ZRU|X@m*`=+91dy+@ z`Vy{PqqhZTwv*jE6DVc?>04Lr)|IXmto^U!@cZj*-^aMf2L9TNZkT&xVim+vHkOjN zwFHXa*Nu#hNIJimk$H&a22mXY_3*vz9|afC!9*|nxV=(YV{ z&FFtQPMsuHf&QT9cpp8X=gEO9HM4c*{lc}Hdno}o!c1AlBpu4i-;SF&$3TOTJ)^c` zCf4`Cg3uu^Lmwq7` zA|)LFmT4>4u6UZt3t7yXXdK{q=}~{S3>0kN_*`79O+`Tx1Au9^&DvS5G5n6bux$g- znQ8y?Q}y{gm*;9)PcP0BH=tKt>!~rW?!~!bp-yFb*BP@7z@8OPRDwPS4cm=xAY@rW z-%Vy}rlC3j>1MLLh@8?HuF3rbqAwVry?!o*5 z$O;2cd~Po%U@*b-EdlsAP+m|o01+nOu`c-nn4!T`Dl0Meb0WxM>M z0H9aa9>(WPATEkfqBdD~JzakfvpbM@9Rh#nZ0y|vU~g{k%{O+!X09XPAYSyUK*aR) zvo{A}XFLDkoNMfu$9Cfp08&O$yJD--La)lPPs_s`v$&Q)yC1NT=}x^rD9 zZtoO~-kgK72%HZ(uo!GFpZ^%8DC0-RlPsc}GgzM8p3$W)1J>Lj3S>tR3-rw1gdEKG zTn1Y)=7wx)6vrsZz$R$}CktSq%#yIdHRewBKx9Hu_07!!VqtB_3~cY9)B{+uzyWfP zfNal{$NcvR7st=n@WJbr`(VlmSS!MNZ8MJKoOI-uV2=m$>MlP17ltt`)9B`@cZWf~ zzMACg>sfAQ1+gd6MLL0zR^-yB9^VdPm%|e$hYwF^K)qkN?v0PJflHvb=Di zrP$q$L5q^MSQz%p@0OdT1s~eE$H1J1$Qr3EH)gr1o9Rx@+qJxu_wt?gl{}l?E^mJ? z&9t1|gtY}|lG+s#=|vuE3a}~0rha2a^y?=72Y99WO(2jJOnz2enx>S1)GnG&hQst| zhQ$vHDD=i5qM77bus&R1<}ZA5eS8hVCQDyn{j97XMUph59TLm zteDw`?Hsf^;p1<_HfULGukrHAU8|z9sWiTLJ+NR@ZKE!VQr&TXoJU3Go^%p{R`;QN`?SSN&dXsmt7q z!B7q;5TPs}qv#^xKU%mp14!fk_k|?f6j&hU9isTNg95V$($Cuh#BR4pQSGcDK@<&) zyQly+Pz`3*i^B#;C){>MuahN4S<#lVvkG(-^ln^cFoMwFH$Wz836!T0zzL9S%LW%yf$+q_YV!m zm68Agk;)8#gLr@Q8eW`~z!;dQK6~v~xYsGmAVW*wETIw3Gxej_dsyk*Q8EC{Fa6`W z3v_xtyWd)}z+)YfWykUQ{2TH<#(E&o+Rbt8xGqt)#W_ytLm6rU-w3`+0Otq-N{v9) z&kbi%Isj9tJ@%|03t6l?)s%WzU4N~ROZTnWo6@MT*{`E z+$9^TC~K$tjRt@c5M3AoS4q&t41!=t_%UOqH3{!xTA4O7_N7QpNrDjkjL&r7KLD?h zkRZT7KsK``guuq%#lR{)mqEioaAemS4kIjL>~Tvld^mOySmBKk2)YF_B~8_&PR7^{ zS#(I2U;x`bY=+Rl+S+0I{;g?dd5={Z78-g46b@D;agGm2%!>?I1#=(^!*iZfI;LIQ zZmw_S_V&(;!!_$RFw?bWrYTojF|pk>5|P*+?RFLhARH|c*PA}EagP|xwQe34AJdLu zu&%c`j-4LZ*xd8lOcFtmxo8|f6sVkFdH#MG{BN!w<*N^Sxo$7zP)_4R{kTQ)0^%I` zJP%Cz*TgBVLA_i1{4d821s{Rf`lH|Z&E;bKMGsO52nyT=bg(!$b$F@0+aG;r7#OiQ zS|9Xv^Kx@$@6>fJm-1TP$mi`Baz1~s!WQqo<=Uyy2E>fTF||rne3A44)@(zZp)2H< z6*lctCmJGru(H@UeQ$ z=WfYQCyXbafP0>%>eOQ`u0Zp7dtvEtfC0~@(u z!Z-O5Oeb5S%yg0jS>JEna&9hSq9r2-pQo=-|aG`lUrTYI&qG0JXJJX~6k* z0j~>yJ=fmz&lUi?fLQ&m09Ng?Z30f2P~VhXn`CdrHbOyapU%sww~yIq9n+cfY-;LwdWp&2oUZ16p1zJBP*>ULiV_q z8nD@k-X~ATH+Zv8a(#7WEok?X6HMcVUG<6u;2f7d5fjCw9gtnk)tYzBFQnuR;AE5p zu>J8Spru@Z#ou$xGe3t(xF~vDgJ5Fz`#o3O8@@~w#2(~+`MYBJgXri90{IoWOG7 z=k4H|LRSAoqZ4{J@CLOSaeWr#F+PaQaA)w|2UTNNJ!yDtC^$bin8;lGh z5GkWYAU}T#lXm)AdZ(mhk+(xu`)>ya6joW?ytr!|yK3+1%_5M0ONDH6eEBv~6Sigd()^cz=keIYiCm_PlJfMhILGRgYYu0V)IWCoRDldnhbnOjC~q+G|q z7}g@ahMwzU0=T~DPtZ}(TadGJpLRy{meOJ@+`Sd{+{>M2${r8lOryhLF5bpr_?ARJB+Xu%! zo(E(Whd3y9<10r8xk~w$qoAlz7r(9}@wVT4xxT!%;w~$=UdBwej`ua7mo)TWebIUb zFuc~7fDv@l1HD^$TE80QV-=WmZ0%D&$A>nFU_uEHy#|d06+hTyplt!MvVhow{Nl37 z)$~#x>WK$07||l@`PC1I{E>kB=-^BL&3^fn{Ol+HL;sSGKy3N-AOG(17hjOcfttW? zTI1x{FHb($MmNsYz-ygcw`mKlEkIUf#)z1;oCZ-^=NIhmIb)P#wZl zH-y#H(BXoy!g%(s2hquh#`0?$l=Z+E2DUi#w7%&kVS&`&SpD~Ye?T-MVZq&@7;<(txGt$nT7g!J{Bloswzd(iD` znJDJ*`gn~Xx_17oozA>xg4|xq-65>e$HOKl66MegL^F`_@sov)dQEiyI-pjA<1E+r zr0kdli)*^NHoeVQj`bFxA~P{fI*A9zTFEikkAGgn0I@@k%lNwh&Bq~NiTl2Kj|xqH zO$IYF7?6SJ^?`iAin(;JABSA`#fcBip7B`B-fZ?J6TLlEUr_ecJZC`8aTyW__^@lc zZ)`hMROp-6%@5Xq`S@Dp*U@-x&LGdL73}H9s(V*AUXI`I=!qZ##^%?9 zZWLuIR%>G*b~%EF+1t*3y@1%u_IyEnr&f$8wZjJ8dG-JP=v<#VfwBUz(*k1uoBr2p z5c|8!(O&)0f2qk1)wzmlT~&@=9JH$bRjRg#-J3Sbv}n_EGv-msJRr22{qL;Ztb(6= z*)1UUxq{gCR$jGhIcpC>TdrnZr6!@Vd7^hLiu?h5#|Rp1($PR&cfeSS{#!Fa~xX@_<}~YPSI4IKwENXWUz3`SV@h0)M{n z%z19zMqF}IG+`w~7OU7M&dJyu81>-Ss`Ur2oE~6sZX4_9U4u*ui@xf{D%FB0x?qvd zO(kb`KT$FmijBW?_XSZu3xXr-y=vJJ57 zOp?CFZLaX8owH6Hpq``MtJ}itf?+ic?vCEA8)vGA0X@WZ^6DI%Y+k#5+vf&tj_9>( z*H5oUgN+gkX(Tnt6<{{5bAVU@V?MZUtU9P|B?|-x1Y4?!FC8n}S5ilD*U$kg+h`D6 zj;V<)bR!tIkkbHQjhh6rg9G2tTap?bsp1mjo>+OVl&1qOyu`sUNl>353 zD>ez=SFpDu(*FdwXi(yTN5NN(j&`&xjMl!Dx_&r#lx)CYDgMe$S}c;MpDQhyL~>2y z^@Z-ADSvwJ*)x~v(Q5e zT78ukaA|GBowl4+nb^bL+R@tfRNR1a;@WA$qjaSe_#AONx7E_7=K*`PN_Q7J^00_^ z^7gjLxV9M52^i%g5Zeb;%@>RgsQUb8h9dWvJm%C{H;@$~As0ed#$XtQ|xsOxu<%!SM515Z0(S%0iqwmK* z_s@CvO=V(zeHCO;G}F#wQ4qUdes_7>SQjyYdDp9(~| zZY*EVUD|*VO+rPncSMK*30(JKOohFbAk{HPbupvT zLgXc$mg(ElR zl7(tkNoS|$-X2eNm&ruD$gG$a`noU>Z7S>uBZy7Z)(`WB1VcJvmt zBmSK}7#ltcOjnOlJ@=DYa(Lw9 zcyV6`P{}@JsvAp!YjcIcP!;^8zMHcI)t8k4JP#5rT}}Ta9;HqRIYPyc_=J` zgBG=8VdHzqI8vMI1!g72I{&T*!9#q;QWMNrhFCRT(-Gx+aeZTTkJxC)!bJ-&InX!{j3}9k;KSFWddP$6f+!5=h z7lApnF`sS(bftK~A;*9hMk4q^HYtvg?;C;mvHYiW>nG^8=57^Q*mZw*=LOhW*uEL9 zVAsHTCo?<_E?1!SiQkzId2V5GP5JXjt5r8Xo_aj>d?{rGddg1Bk< z@37pztNUhMyFMIFLaXdjESA(*z1vusEMvCd{-;KFkxa) zNa#~%RAU=6S8EFTi^L!`>fZB}k#=#z{DLT}AAn4sZeaPbv5DoW%?cK_S~1Vr$r<#z z8`JaDuDV|fxNdK*r_~ErKj=DXg$YdW$Rw zCijeuV1Rzn|HYKP2Qnh`-sGl^h0sp{X=}fC`kotDca0-iMHGTm+D>7Uv^SZBg;x5> z9+eA`f<%*TJXPTs?y=U5%}YqFD#w*?*0?sK52JiT-as)D^gx{q6!hByCR>R$cE zM&TZ^7PVxMSj!ynK1}3Mwc!jXbd-@U1^_S+>H(iJr!u6bt6}^|x9AH4D7|?BSiz_Q zsH&GNy?Tqi@5bKSzG}z5j{JGFW3c-JMK22)Jypn*s7N2iSU}uuh74Bw<|brlFmU4c zlX#$BIB#HyR4IL$D?qKMcAb4r?RGII1(%{!zzwoJKK|yeAcxiZ3|5&2Co_>O4Q=VT z_DHa^_&#*wm_<=z#q&jI6Ij6PMr+vad+1*Ya61s4jDC%EhI;e_M!d#zFtL=ZvE?$e z2&O-tXRanW=eWW&3E5AqZ)5|BfrGxm&jhRmghS0^??G$?#e+mJq|dPXqdDP#MrAd> zf>_n4st#7&3B2I}3c$o&2o4s)``nrf8wGTSCoTaJNq|;x+ywm|WB|1EW3@?Kf3me4 zV@|ekya5FB65+lvW5UmD&|LRH>*|>7xLW*d-33doz2Wl0|F9?PF9!tIF_sF60e86= zp>^kHSt`3BsBXEA4b*u9=!Po+{H4tT>aYiBM12F@F*eZKlcDWg0*=9>YjZz-9=ndP z@Qsa@?$PW(0~sZrtnIS~`h@}I8o&lCTl+bjUnGeNgFR*v7?5E>up_FurZI@`p=zJ; zh;8wR1R~yX##MY?Ezcb)#+vaVuEExth{N)sS9h~~eX*Ah4_mpNb_*UnRg~a_5HW|{ z{%RI&tTXh_?uzM-e7Ez!J1AW0|0sX>lmCbRT!Gk^|LkA>SI6o4H-^KFl*66W>3;eC zs58_TnC9q>?J!5(#G7BZ*>%~@*4ihk#9+C3%io_**Ydhu$cyQnJe#kq(94)uwT(LL zja4n6s~uEjlx?X}MmoZCZCLbXuzM1Nz4uG>jduwBmTMw?06KWh!Y*o@`Y5!;#qFb+ zoVe0~JzLO+T;vsu+L$WoQs8oi1d94PGo}x-X7@Ser>h3*?*3M8R9|Cqf_wWOAXd2e z6@cxu{E0=WrhbQDFe13-Y_$W`Y6~m@9zjUq-+|F_)_^IZzA3{~<8BU*UtU9f+wDku zsS;6lrbDYSm!{c6+MfQARsp?AQPd*_hMQeZ@MIGZSct@;-D?JG%#zeTS7=O2pK9Ww zA8%iTi9uSFgeDW<#z1Y?C}Hf7st3wwCra_ErvDwSvU%3bGQm>9-d5r2k$p z!RgsLi1=Kav8IO#26#9#7XG$8jErDV2f;>bt1y1 zQ&&Y?NCy%Ob*+KjN0yQT4@jk!xc;8ym?F>>03$z^wjx&>vpNA%x#0KXnLlSk63%Mqw3HZ1OMWEa<-_&^z*m%c*Om)hbDBlP+v~rK4-I1vEG3B-o&8I7C0ue(~*vscHN{lOe(wBf<^L4pA6v;h1$5MP=j3+@iww!RT`>x?Uz|4ZpiH$dhm;fqW z&bN&wHEe>QHhl!KksW%15iK#6>gv2WS6HCNMnuZO62xl3`3CMfp1nvP@?+4Uc zw)A7d2P&EvNMvs2xHXxH3NX5wL+i&efaPOG;i6^n4vvMU3U6&KNI!Tc*O^$6%ezUw ze*Y+MZb!MBcQOs<%YfJjZhhSK#&oLWSn7_3lRuv6gN5Ll=cYDW|B$a)f!KfHU-_4Q z^Y!WNzxZ-@C(k!`ayHz_X?>85+{-vW$Z)t{0PUT~{#MGP0^MsV)2&K-my76ru>=8X z%b<`=r2{YITE1Ja7ZCeaP7ZGu;Ct%;c6wYc#y(hBdj4@Uh&GGDf~1@f)W%Mzo*ZzK zsR7LmK|B>{1Hrd+jil9>x^IwhT&d@+2w5o^+ZVvGM3hQvo3OGOs1@7VcQ20ESX)nH zm@p6>P$szM?0Karywv~27!ipH%ndlLv${>3469^Xv+pf$R}1HnmG&4bU@;T_LW2tJkVUaW z3BOZTjF-V34Yu_1tR+&GcFl$0v&Ba4%$_ggE+v@4Y@Wc?ti!krlx~3%>SJZKAZDu9 zPrx!5*o~N8iwfXz2=Gd4);vKGy`vE&c2)4EQoOPWn z(v*^(kBKx`8I0D6GIhlaOuO^%{rLITnFH#o0}qzXUV-09{+UROW9JO}EQ@pm&bP_hD2aqW2x zI=wI(1T+@vGNY%@b5AH>|K@D)!Zp(z0>nWon;|G0YjOg1f1CvO9W%Nz5CXm)a{aMz znnXz}gFQ`j%A*W*{IU>4iPwp7}i>mJQ&IwXt>AC$3S2#TOa-09BHpqi4^HPOa@%g0!6{xKx&e?yg9Ejt)HOi? ztI74Y8_4@MU~M}YQz@&iR}9ejbF!im&&>!P=-$8pM?q{#|Mu>bK*9uLj}3DA_oo|3 z4=DTlB3%OrUcU%RV$LC$XA5R7BHjC~=34i8{7y#-Z;|deOy9=Kp4262`TZUl>}3z! zFQfnR_8?zq=pj1ud_a*h74_K$w@ z5BU2Pi2b+z?O*%tAAWxMYrpdPQoekBEkAg9CEt5_E#G;*fY9ex^2Pauygqp^FLzh+ zY`B&4dL^gj{Q@?wMH`Nl`PM-0cDk4PcrDw*jhsyv^0K{`&)QpgKE0F8;X;N5#9A`N z()b5U-E#tXM-%q{omVGW;y*#;<9{EKP#0CKwIl$JC*oc-gEaFXI@Qa zf$PpRqFdYG`rzL2J6ZXU*@a{P-I*c*z8Cw9(N(Kw_l2^Y0Y_RGIEWYa6})CZh5A4T z1L$A}`n}Ri**71XC-C5{(2*0sG9oGjT+-09z&Gl2cWtqlb4=%Ay#`FVPy!IO7K>~A zZbcIMafD8rx~~qzB|8aSFo9}}+m59oGg^|R;X1%E-jC<#Vn%HrGB-N7d^q^}V<}j| z20bMdm88!snOHWWaJ?=wH>&_R0M{C9QwLppAjomhNMf!x$keH}0L(4Xda&xe;2=Uf zP>+DM)A&lPK>`pq#ARt9B5F#FQ8i!H}d26x*SF=jPLAxI5ofLL`Xc-%u1 zyEjlvcSe)Z>M@PXYGJ(C#cQFnk9AkbPG+zT7zm&daRSH;8wl|pJp;y-C4p)U)|4-a zaml1fN!djQgBDPJ==OyQ-72L4+$(%7Z7~qKyUGj+=L*Or)_9a|yl_~k3z5wUd;nKK zsK1VR!fqfPD8>FgK(=QrYi0e(Y(-V-0%PE{CR;m3rip>1tUlAP2bj#_N6%Y8KrMYh z++fp*0_;05vgUFFXC&&!!-ZwF4)n!lM~^-L6!=pItKL>wcwX~{;N_$Hd-QVzn(?`~ zH_byR^IKZ}eHh?vtjz2Lho$AU`{nEU?jYY>?B)IaDB2ZZ8ecewEnN<)cI$XIB7o>( zJS5{R*;ct5(XExY^0VLjeZIpA#QsPB{%3#t2QNQ-|NAfB%MV{&$e;TBN`9~a+F$wX zLKN8k@b&uz*uIl5U*0bNF7o{sxANT=*Yd^ldwF&KC@)SP<@M%4UX8c%TrT8A{U9%g z_wq{K$*cLDoF3lDR&Hc02OM@YT^}QSTYO}Qi%+J>mTC-kU~%I?wzq!On0&s;Kmh%_eV9 zq+Vw-oxE|*EUWg_CV+~=)&JI-EgYibmVs*pL1p}ZNqct~>9*`VZ0&tcRd@gQ$$4{l zDcFDvFi;@C3=}YMz(9cl1rr#s1uzhd@o*?8P#}O7hk^ssfD!|CCJv#%fD;D-Y`_i( zI1%8$3=DDrGZF)noXh(^`tkR(s?PS=Ykl8ZYoG3aNSgKEx?Npebsl@Kz4o`h^>QGU zEge;Uc|%G<>@7JZgj{Q8E~E8>Re}RJeVe9y{gCT zM(n7d^A_IgRTogRH7W!UbX$0WfnzW)V$D4)3Cm<1_|Rg;`vO&&Tl{zvSXfD&Rq_I# zcutA6V_-AJ%Uym`+9Znv^TK+fj8*l%-#v?&V2ZANGsq|kdfZ04vCr2o85Cr|7&0&# zdCHW`j8mR%&DIhCnxM{$z9&%yn72qIW0 zc6}A0I=YqsMuqIJ>*)>PfT5ikaRq1^VpPL>sI$bfVc-hXE;?rCNY0!Rsv6TAH$ zAhs!9?zsrP-Fx3t3d$z-&rG{{AU>LpLfpU0a2dzOLLjlsHX!zK1+D>NAKi1{vErFX zz%y4T@D=Ma06>=juD8u@4W57it~o@2>p07`%auHTxHp!GKg~#%uHz47QLLm*pR}2& z)d}zG;Qjp~Z{8o}_5GwE_BemGesMpDoRsK17$^7L7taX2l63!Z{km2j|^ZA4P!U|Ns``P;y%wEfPKks1nQodNh?RP(ayZ*dg|9&rDe7=I+ zpL;F8@b&fgul3*W*qgN!R{wl@~v0b^4a`OK3*>6Vp#!fd$i3Q7p4Ki zwgm}eEZGv0uuxo(1vT;(ZCTsg8sZI#GX)b|Y5hCKZ~(+2guogNpxepEXVomHO8eAi$py28h0l>DdVRv`8a{q7-rg#i{ zz_bRd)3||g@i*V}-Ff896xz!zwm=DNz20g`AuA=~@pQgg8CWUak6NV?0K0*;paLxv z$jWY85&ts*ATx1EV`34rw40E81t4yvuBsGy@N-LWnz5TiE0()z1Ww$ zl4bX&DVUv?hAL3nCYvmG)gS9&kldwOB>*dcYWFNm>JDXb_eI}x_LYfxI$XV4z{zZG z{eEvv+>Jl8wkp}*iZV2%Iym;lz@3WjLZWHBW|E_D{7T9;SK!KcRV#-jo{#`0&*RLC zu?-bj!DM{9k=KihyX%gLT&TAT#+5PSJ^8^x705`oV9(smM}=5I@A>bn!XPj}PY&|+ zeTILy!X{vq7@tBt6QF{MR+;XS$Eyaw1<;@&-Jo!dsj=uAn{i2i5$8l%GJm@1M{pd# zwvq)eOrV>bWo=++1x8;Bt;Bi7&tCzy^CmcMz!gfP5insx&3XbU zaRFclc5btsZiuLDqKkt7%^?6O;0T*8U;Ddz4BX(DbS^lzxrQbU?Z##|01niKp!flTcElP2OmA+cv^w#N{}7KB19h~W+vHa{?)B|?q3_C!LMid|#^F5%u^ zU_2~LipE}QdxeN3<~$P{A|z{qWzQ31UwrLJzWeEeeDU=a*nWKlt6zIBKmYj(dOy8c!R@vDho8Mw#(eh*$s@~+_6pab=GZS4%HO2@NIFr?p zF+g8~zfYm01VArhw+ixyIg(X_!D~(~sTKlJ1vO}~BKk3U!qy%Os$U0N+gj+tLdXdw z)9$UuzCNsi->o=J3;^gx+yVWr^=9Sfsli?5Z-bVLvL+DX44MP?J@k20E=g7P@3r ztD`(4T!0<3wDxNUs|8G}RhYCV*w+RoR>42dHx)NTwn|V++Kdu-%^Qy-P+yTSF@f!n zX>tZa8N-S5iG-6WDsylRdI9}iFs%#~nBn%bE_&8vVg~!}#$oDOmz#Oydvcb6GT&0x z>jER8F^1Jwae5%}0h>OJK2J1aH2^0K!(d`9WY^0z`68XoVBRfMaAKr_^&eR_f<&(H ziC}lT0YYw7x6P9SBN`?BTvPT&_;`Ul7?{`qaL=!0vmqA7Dn`7Z-dho`d%Cnpj&$Y- zBcZ+5Pi^>{CrQ09K7#A{StW^r!H#PczMr0gu#^CUIIp+e43dCTnXl7~fEn$@(Xaqa zn0nCQ;p@!~daYOku_uIY6)U5T@Dc_6i}*OU&3JK4^}HfOJf#9JlyGB0QS1o-EZ9dG zGR7M!G;D3NbAyZFUiP^qYyz;b47D@T15I{Z79$iZSHo%8a|Azqd|jSnPE`(2?=t77 zcsB_Ql7{lJ(FVgx&^SfO3e*^%=AFyLgqU#}x80D|Wa`;FG5B6Ns?8uvNCXVEYK6-^ z#;$u!kBEh4V)KD?L1^{uq~ z9#7FP2or{wPMiOTB`t7?R4z)LvF2IC_B;(1C~sf;T@UXU#N5~ z5W0vZY|tr=GCvM@49J?BZBj5^z$UYRbMMl@iUhsSHv3wK4Mb%cWxllO!l4(`w!X^^ z79#87=E+qz46Sau@Z6LfL98_KZ1*fEr5Mn1_7z)EKc?z@VOTc56C6=5m2pBzqa26h z?IMK+1q=YX=m`MpN2B%uqYRTbcYXYvKACP(qwrNF_|l7%QSFAl;|oSffFjc^0F(kC#C zQcbuU`l53V2|3IRFL-Sr8AJwlfE@jNPZ6 zq(qvkZ3AFIF*o-JXd+qyu_P3v(1b&1UQWC#cJHna)4r>FjpmCqPeKc3)#fELlM!1K z`;E&a=4t24#~jy!)OVO-2xrr=cylFuqSTiP1XiZ6SsMqqW~$@o@3t~C`pLp&|! z89KwNSu}sL$w!Qd0ed!A>W?7BtR&^>aPY*g9#){bBN?FC2n+J4j&*g1CvnlR4F{mr zeFNb`CDtHR5WJ|wNA_f5yg%8Iv>6U*XrQ4>pc98HetBG;)KKfKmOGB?GOZwXm7Rx& z9!_wQ*LPK}j^`_g{d~m-J99k<4RZq8s{6Tl&N3MghUK$IiCfplJpEhw$&deElX*OT z^uzL>)Uy1;sxHsts<01IPKR{>p4Km~*2MC-g4+jeUNFnu`n+17mw3Kk!R-oo%Z< z{gr%j@gSdnyb8ez#;*S2M&5sUFHbj%v4&t6+_Hd_8tY;1T8O$?l+6}GSQ2*g0orKg zEpAbg5!B=K&5)B=${3&J^9XiS-(P&=)b?ow?oyVVnWJ$+^E#z=Oj`MUA!!C;<35e3LDtZo$ID1{4-4Sxa$7 z3ru8X{9#g9P2BQ!D-O5kl>`)cHCWMf)mxDMZZ-hhUC{g84)K39WDHDq9jq0L)LBHz9;v5M0*SP|3uy`ASLm z;O_qfnmJ!I9s)SW2B7yjW%d;@-ZVnUT2=yT7cv?&!ErG{qQ|@*j^-&-5c_n18~H&yc3Ib^qqe0raM`qTm&VYx2up~~kh=wq7^?wX zC*}`W9>lVc-5fz|X*AX-tFpO%8#kH~{Vykk&k=}`STRx`?3vHKm?uai$zu=%{lUGL zhFvK%s*1bD2X@e7s#@beAKJC5P4eIbH6*qpVulEjOi&wt!l8Up%%=Q@BX;;Ij zt@J3pv`1H8?-jC>bwXQ~7^5B%i1zSI%-$Z0F}$9OP*{8ULm<}lSEFF~(%{DMJb_N< z+sB#vg7~`Sbb2y)IyjAaTLUoE7BkeeHveI<9HMT3ijiw~k4JfTdyud07P&d>59Hk#&pfdlA5VES=@cxXfb#U=E*x@3BE^ttKUa^Ta6>)=Hu=TtG}s}wpd--8J&a# zKuD^HcL$>D4vZ^Hr)xOiT)p>rBzIWs76al#SyX@za}l5PxoC9)h7Kre+*{Nh7gMmE z%|C*PtZ0WTW-WTQA|+Ism+Zo*>ANgKPE$>VF?H~=gd$TCULFcem+lX6!*CCzQ{Rdr z+iq@PtBS9)Y9e2`!CP(x%4kh;wt|y0#1QW_Fdqt1ipQ3cmDELBfcl(W!8zUH?teB# ztl(7*A~aJd8EBSxEdZJlVBLV}?zuaY%^%!l(;~|?0gyc3nYB~Exd8m=@}kQVN(-`H zX#6yEV3N$FiMlBPZ*9FADidc8sxM_5qY1I9Ebo=FybFPj7s0QCVs};x%!VMt+L#Js z61`I$KF6>d>w0E&ad&9ZyW5>bXhGj}rpW=Ha05ei1HA_;>7eX#Ut@)hMgUmUMcc=G zDb5U9V+hj$-o1=*$oQR&RCLJDI-3BINj*Rn2x=_+A(rfBe_Lq$YM2E=W)QbI<)k(| zr&Jcs+r_$(gAkAkgNJI_RWrvs6#N8TX*$Q|p5U5|Q5ZnMFY=H~PV|^ctZF7P+G$%D z1Nz?L?{|>4K)3c|s|UgXy2UETE!c(CX3Uj=QpCxR;xgcCT}N7=W2mIa37Vf zv#5T-lbSIane5m+r`O`(Xit0 z^a`9B+SRrQtMM{|m7NytT<)ph`MS)YGEWX;V#t~Rn6u=d8JQUJsSdh{-Z^OtatA2? zWutp0XIvRy&-nemzNky+>DjXzWVP~>{T`Jd>21F|i0w75EwET3W;zn#GSP(EFSftI zzBT-?KyC!hImAQ%Zf|cD%<>W5uqd(BKzz-p*g{asrx6XVfS~IMOk=H5mibj|{+0Y;e-o{bYF7>EZs{$&&QTvmt;eN94Sba`Nw15c_}j-yOvMiyzkStsnY( zLrm!KV1kZ;HYuqlX_wDNMTL019x|}(GP+XNAoj6=hJIsbK7l3r0Lt z6=RWlo6?Dy#135s*B1rGOSyoB^%xsk$=Gz4C3O-5b0iQ5PpE=l!2nYz7z*?ldlL*R zkLw_;FrJ8j0xec|sNqQ~73ZRRiJ;d@txO_e$>KTL=XP4}ueVp7tz*>=ET3*V_^go4 zY}S>kWCc?r#xq;4P>gzhENcT?V>}Wc{GD=HYwDgdZ^B428&<~o9N(j^NORz%M)S?# zMK=?l$;~vCXgWYZTa7hqZ=W-?-5lY6y;!8sc(Q(`$Y(;C-Eb}%!*%oZk7 zOfWm8kc9xQEIK|pSYZVOyROOQ#$^o~8x?$JW}NXMKoxj%Fy4&`r2-wsHJ|2)H4+87 z!HpE2Q(|rj*qFK8_&IC@OhDA}0s1S$dXzNxV|<`9Q`8zI$IemS>hM1NH9`g@v=WZg zB*0D9TL7sFENdBL+d(p9DG9KT5roXv5kaqik6Z4zr?}@vTI?kN6KvsK(a@rL@|vSt zzn+-$;)WnS3(7dwSX)C`sa659vN7RHCVA(AbvBu)(9tF<9mHx`C1qj{8Ux#Z=5T`k zdmkSKtPL@=#hxMo*!5rA&P|hOXCATP?lp5_#J>@rfjkth0SE6aj&Ez2apY#S&~PHK zqui$PNJ+_*)8Tr^{ML|dX9RtP3gjjB7sk|*Fv!G#v#Cjct4F3x0LcjZ+F6wXjju;I zQ@@#RIH=d}{7jR)G$wXG58)6IpPWgi&^N=_4unyXX|eEvE)RETQp7ppcekFu{`(F! zvVA@J`Yxf*Wt5E^GU+<3=I_UId!0E3dx;aik(=>tvqEn>UO6*`FO(FuPwT^kQAe z6aLO}S4S)0e+uBHLMk;Ko61>-7{Ko2K$a!3`mwC^MNAysF3iXmxESef;R55SxG|CG z5M5+pEpcJ6%M=$dA9aeZAOx{=yA?2$6u9iZXYJO`!fNsZHX;izI0?%FEKGAv-w$gl z%5HcVQi+aQ131Hdx@C{ndH9MmiSCAE44)?uG3h1c7BXfq0l|VX@?*JM6s5MNorBok zj&-+>@%7@*4e_L6Y^;@{N{L!?v=wG;SGu3*;x3r$X#>PEfHbgFf{~=bf)ElmvnW%9 zA;kA(kUJ_L%5C0QI7Stl?T~Z^usFLyuND(Y03%E^3!vBlY%G2_gCe#Njd)jkjf~zJ zv4WZq;Z+=><}s7|d_FuB1KI4wm=-B)ZEDKbyRflL+BA6>ffXB{y`Hx{Cv1P<)8Wc9cP*av=k zCUf~ogEx*7=NS#8*idu-c$KfTAr;RtpuUDj2=`hOsHGvaT^rQL-Xui~SY?o391u+b zgo?!V^?NIGIwmPOs;J`OBMd)rS=Bb+(VZ;})tdAP4XiS-bpkmFnX&bzdyff=&AkI* zJW#oX-aXf?y6KahN0+W+*V!GoK500Ch8mo#YnKAs#~xB})UjW%O&f?MVulyVq!b26 z*c)mLO`&HBVjshM9o?eLC=TPir59}Kz%Vm2>BQ%t;{NsjVDU$TCGEjl;xQRI5`d}^ zFKmo*Go#GcN-|Fbn%k!;vv&1{plBw1iI za%1e{J6iy|B>;_W2ev{U&#@V%D-xiWMw}r0@&4f;@9v)D^$KDyAJ==i_>Q#svsKQ| z48#^N-X8?98y)<@V*s*JzLcL>tytHZzx^BKpRYf@1LF)QY@wrN^YmC1f)){4TFcS} z>_U!*b9wW+$xpvr*o+qR|+b4CVa7LaUdRc!Y26y=Gi1obnRsk{JMV=P|t zPy%U-v(pP(jS86yydt}c4f*6uY|Ky2jLwYKGZ#>kVL@)eU0yi$8ZADZMYf9GajXUG zmx!R(JU1O(s19PyCDmhM&*nXty2;sGi*1Da$_!0h*E|>s1a_=aLG~tC!2}Yq;lJoE zPk>K3xy% zcpIh&!#}(+agG4zHDr5Aax=}bE+In%p)DB>W*9&wHN0dFxgltCqo_oeIZ3IiAV?zV z73CJ&xVdYmA|)4K5J7MDq^+LV^>~PVtdl9dKIqfaQOj*TJRSggV_pQ|FH%Am>s&O6 zYjb>(44*+4FTRg*f6HtlL5+jV66fbQn#{smr9VZxFp;3)J}F!~X>C;Q`$=qh-YwhM z0I*f=8mmAE4DHZG3_;o&aMkBC_NFYG{k4MAvz^EC8#vtQ`1Ej(E{A8mb60Oct2&XM z!|DKY3rZ=&(F^fq0GrpYmGC?|6FbB#mon&B;TTL|BE+ma?n zHbQ{+VRG6+hmsc_0*jerCeJpHJc1|{q*6v9p@Hq7IB#UhvQNT(IuXP!8WYyQ1lZq0 zQT6N9qTGd{2~dvJHol(7rS(F)3rK5;S}?w`H@LPX5}-V}YNm@iSn<)#O27~xVcR8T z0Ir2D{0%$iiZxtbET=SD!=YT_36N3Q1!%+g;%5f3nVHb!K|xpq7$;_FN3s*ZCC0)M zv?dS9u=lGoRP)Twg&VIp*nDA}>dPRORt4JKr5!{4ck)7T8h8IL&_#jXTWNKsY~9PYbRJ>phq2X#^Y9NHH(Te+L`NQd)d?a`~Cfc#>3v>r@mBAD*LwPOzY@;z(5UXB7FPpI2Ta^a~kSpP#Gtu(sY5RObc1%sO+G5Yp zJ(Tg|5_?Ag+1ShT-j{W4%quy$(IdkBWZdH^PenrndcqdHM+JLJ|6ZS`E=wK6&gcfo z0grW6(w@xw7(;lr05)Go27&~!WMbpq~#j zWzN*wEeLOhu~mRwPD?bK>T>w78f;g0#}&k$KUJM|%3?O!aQ>F;3 zAT|76KL}$(ELITvulfrGvF+c>`HG$X>tXV6Ue`MuNvxMv@+T9}>;s~f_eXO3ev-fV zY6Y>ct6W|0*2TOt8v0}b^9vSz-uw-q>_UlBxJtG@neJzzsvA<-os$Ip_bgTL<`#!+-$8S##j$yW24{{ii2~#t1}Y2 z(4}ie+*1M_>b@#g3T&?mD7)XOD6@+{0Nxlv@Ej}_cJD#?p61{Z8{-6b@I*j2JzEn5 zv0EnBfjb%j_{I*UXduV_Mt0veZ1zb9Xex;3CBSsUD%h*Zs|r*wuc!jBn^LmY?CyJ& z-+=+!Fv2w@dZjotGQ0!|1|F+qrg{jCZ~#kCJLw$TyzSHGm07wO+2wY9WHaj&<)3=o zTLrs3#_BSDO^k$MU{Did%`h=V!Rc>r6io2oJa`43Hp?^DkJXjA5qwt!gf0D%Dj znY<#t9{8T=AA4pu@^+&S5d0j#`le>+BVVx_9OAMyZlWW2r<&aAjS)!1CU5nGtGjw( zWDTJR*3z6T>3hz?xi2WKN4=RY zF7%}cW1yS6bw$sO%f@%Ghm|oLk4C4T-=8MO6B%pBY(wtL7EpPs?29TUqhEpCM4syB5e2>Vx6mhYJ-zL`+`LV$dH;SVUwygA+t-s^UCrx_)^&147y5}EvpB&DQu1au#&UAv-174?jvzQI z+Gzd}44u0LVe#j4Gz@?NU<=~sLUdOzx`WUvvq_I?0*r~xEUYsl>H4|ZupD}Wfb@S< zqFM`AcrCQHac{XZ76my^nA;rV!sO;PBta!MXdC9hM+iS-#6%5}(+OQr$*$vGtQ^z` z@C?LSC4^Ghfy>Hko$Yt1844w-ObWn5LtWHTEMUKelH_LDuXIkErYAo+lR7eossuuA&HeX@DcN(ioP;7 z9Y%$Ma+LhS{gJV?jiDjY)YhD`N+~wJv{7r3xq|YN@b-wWb9doo5KIv3n<&u*UDa z0DNn81ArARCWMGgDK5LEdl#&QJU;H8?fQI5$$u)G1q^WsSr$=%$~p2Y9E|eC0c<8s zoIS0^@EuquR=ahfKIqd?Ztr^A@CUiOeUOK{CzYuq7}Kt&Pn;L*JC&l=&%EE>2?dLbKI zT7ihf2KSaEj}}rjZw$RG*YO_L;1)%I%gU-a-wcGc)nC#GG z6%51!Az7z+)`y0zHQ zUEu0|{m4ghb6Mrh3ShtdvdQJ8$o<_UPe%i=R?eweXCrkhTVhTZsGdDY;bEz&%v6|C z>8`MP2zAWf|WrePJ9a-Tq zsf!AGt4Imd*oIaTE8_~%4kWn}1q*0%vlr-Y#+G?qO;_;G;~qm1vR-O+pc#OAMgof& z6*Q)VAVUIZdm`;BAe@+ig)@amwbX#)>^8Hu|waE>zZ2Ty*Es)zOm{EGIx*Swp5AY!D zCS~cBncW4LEq*TCM`b9%Y_GW{I+L$!b@KZ&!y@(xX)NDZupcz95uDws9M{u{x_U-P z!=$R|saGR7S_Og*T>9&t)OB}zub~8Ycm3ys%eI%cS{48Sob$L~BhSj!^bE_daf&xQ zju2#$=@Va)`3nNMZ(bfr_Zk7f;b|Mh8@0sE2DBElY}Y+kGVeG8bSE48^v7Ix!qR1e zy_OQXpuQN}wxf7WgaHgPHyV`L<5E2V+2p8$*jFFD+K>;{+@LeC9aqe~e4}6wMOnn1 zHcu%7fww(pl%^7Yf4la^aJ!QN*aNzE^(P4n5@LB>c6}&VvpH`guXm2)(uT>mJrCEg zNW@6z$4jg$-_O^c?=8n~TUS!iY6V@-X>nBzceC=Oy}`%Z>st*ecza*uY5tn*UVT#z z?R*u6o#}29EK2~_{r42$WoYyL!efI+ZCp#10=2 zQxwACBarNc!A`H_`f8TfZ;HHrUF7Cw1+Y)EGO-@NDk5}sKDqXjn}|A7Q`s?IC#w_aXE$+T;^Q$|rsFYe{_`TADjB@w z8mTrb72{HHFUG`HtxljyW;c7NfQ~ojR-B17rgffD#{I;M)@hJ|Eg*14k)Z)~R6Kd^ z9Jtl$0M#39loqUYBNGQDA`x3v(|2=BZh%5lu=7TdWl+L;x7*TT@vP2Bse1A71th{C z)@@|gk{dcH|M=d9sV!x2)iYs?3bwgN!_F0<9mUsL{w|MeMcXHj?S&9Ri!_}h-Jyyx zC<2C-@nl+hOA~oeIbxc^fD^=4^*cp~j_TQFow#BvvUw+nA=u(NbQLEX1)sjFuMpeV zDn$a|wN$T8uuYcRg4UM|9=_)c@URZ!S<+DRE}F?!IDg8usP@k!plzfk-qlXQ?t711!H7JxUty;+ODgTzQiA%?J1OH zFRssHx?!F6@tiwAg9$Gpq#)*$IW@|(o}C#c>T4Kdrd=Rc0NahE9;z7YASweG0k#>* zydF0oHh*R`^6b5SpFNa-Ahz%271y#&FgE=g*kS~*f^`{)Y|F&z>k{vMh;`kD1V(wb z4SB_L!x+|2DJ!rbjIhaZX;y9H^dw8K#IPD|?=ElTU9S^+-{i3WnoO_0y?&-MvAc-x zUI1cMr~qO?gc1*akd56Ae=GlcZ+M^x^k4o!es%r(-yhZE%MU*g7g;KdP%3SP{d%(> z%jM-x-o2US&6^HlXL&sA)CXOVrs;>uo4h`!;d}u~#|bI~(v9pS$GXX~@H15zD!?0D zI9s4uyhjdGU`&azpZ3|BnAj{pCJbatv7qS&(?!waMUGx3OF{4HlB`Du&yCIQdnw$o#yRp*GqrkO@|d#R4F$(5321v+FSbOCCu?1Lg&ZtwkI6(^-iA1& zyumu_ljc>c6Rb*QKU+A#`EG9+qs5e^H8@bnu7GbhF^jc1yIP8Sl2ItT$>5sgCWk`# zThP#OU;1}!kls%SsKI}+X^bihG|gIckM3RCg<`VfFBB>-N4@cL{m(1mRAUSk-?Jl73Cj@q#tb1Pm? zy_A9RtOlsxYpvcVy%Fe4Y>{ffAi(H?W(VTHjv&QY*;wnICd$t;>7gXukk=4&M}hEh zELbl#8XA-|&-KOUmi^#K3Xo02hLyp4kWo*Q{CaDY#kpq65@HxjT0^>BIS^UIn_bjG zd%{1v1sXJ52U(&_2x18tY67FfLO~d(y&J%*%gBTl$aRi;uV8RzfPo#2*a#yS36K=m zsPcya26^CQN>+8g$t41zSpy79ZxY??%&WFoD8b=~Qd@@=w001Czi#Yza(maq2@Yrp zEDm5736D;JPW5S}0ap-NlNys~m;mP4!U715i_5UfggzFZUz2r^AAW-j4E`quzyjJ;Px+;v80oHbaX4V*9+ETY#_|wkEu%z4086i}#|h^!w6t;YTz;sR-jZYN~AGC?3>P-ElBf*yUd?LH!A)4&nSgmH}A z^_KukzLzZ6gWM3@aSQ~?%S%s@?VNl=$#Hg?HC`&pY5~*&y!8@W9l$asmPtPffS@qD z(_)4jOB*1R768`cDXW7AvXKPWGu@aQg`4Ly07U+qDbtyqVYnIQ@gqR3A4{L-D zAaP$JrzCS;2|z+dypTOYbtZ(uSWILx7*Qv-BIdaQ zxDuc`7>l60+5*Wi^mhQ(ZR@SIKe@g z*gIunA2qx{<+vMKB^gbsBWQrznzH1xNaWlaBHSN?A$I9-a%1aC&i6@nzJm&U+*&v& zKCJLeEGqA6$dnZ8N?@HqQA=LPt-L1L7B{bCkSEt4tqt$xc>{v=dx+1(_nCNY-6Sb* z_V>=tb`ab3B(GsYvaZPU>I8W`W`;k5!R?y0VGWXv$z2`9c1E_p<~8Q)kLO0u4o^EL z3NvE<0ZAo|N^Sx`?-|H0ie)-h$Y)$7Bgx z$LtT#_)q=jwkC|9km2P ziD6~TO%f9aBJbSW9#>oHjipSyW%ZWlVbI*D3twDygL)! ze!((yS5;0(0JBwVWCU-NvNI9Bk(FqxTdTWY3pVl@akd^ZF(sua0R(AeO*MlNZuVnU zftNu3vyNSuB<~VdB5rQWx<-fs!GC0uGmg(T^{N0xXdHO_uLzTOI)D{t?HCW67~%zN zt18_SGY+HXaABk94NJj(=vZzZI)a#F1%^ql_WyadHyB;LPF6I5JZ8`*u35uie+ICM zUqF1mWN^}C1Z$)07{)%iyPQ)M?N-q|q!dJI6xa(RI32HNyv9LR2{3)L2XiH9;|CyP zQXlL6T5D=ETD$`tJT8-ml}H+syoMR>#cP~mJSzdTYLE08%5GxYFt-i!t9k$gg&Xx~ z{oCVrZH-$?TVtsf>S5fibbSR$glbu@5VV>E~RaFLWQW%d#a@#YzWtncT1iY$sl!W1klU1 zEr*wX@_XDn`%i+{p5WDg_G|7nYPaj?XT`s}jz*+?cw*9^i=&nSrmE?vH<7_8&@?oRm;0^%+E9Y`JT&y>?(%yS@3QpBrs?z zc2Mm3=Ltzy4gT;2pXdEi5$FUr!dq@8GAKz4#2$+ibHlW8iqc9fB(LR>)*GdDo`H9R z>*JY#DR4b_y}<_Bp6-LN22J~sVDgZO1hV*9byub__C@0sei1TtFBL!LLfV!tb4It$ zi3OR$9a#u=xiQkkUwmU_2J=pv9av&lZ+5WLe%6Uaf_A`UoT zQ}L<l#Zto4c9JUaK*m3N{zIS)dcGFtvxA_yS7$F_@y}@{vKD+q%>lIM;#- z^-H6iPr-iH3SzA|>*jlq-gYYAG(1w~uH(R9j0j@49spje%S)OsYa~+=I~&)f>#UR^ zmhmMp`x1Na=a3P|hD(@6R1E{03tTUEa)Ti2F3K1IeZjtw4Z^ZXx_#*xVGtZcNrtGz zmPiS}e$!aykQ}D!uN{Wp8ZUHwdjeqTUblmS*n`}5CieDT+1MTvYtDfNHWmg&3Wth$ zLP@e38-94=!YlbAh~;bjUkDXtmP{;J;4F~N)aw^=5_L9`c&>4e@M;D6Ra;0p#KmOJ z8FGmnlrUh6Ut_GdSTnII0kWhG*A#3ugG?rkon{9 ztk`q~vD02os-L6WnS#~IYewD)?&v99Je~vuTppu^Y#WdsUm1M{_XlW*4Watj=CFk+fmB@* zGz9Iy4o;~UWg@G^W6Ws|H-H%mdd?-bN~)Qg#?Utlg1(BFHEBY2p56-kUaoimwhfFM z;{gjdk7DtElELe2RTrijU|pu!D>2l{j##0ZY}~EHRyFis`k`kCLw9%-SR+%60y=*Jk+gJLMgE|wSvV}~+$ErCq|W+Eh}`1?%wbSZLxDU#W_?26o*{=cT*c2Z+&9+j;`<2XgR!Gsma45Q-NW27WqV^cIjqgB zF|jxA-z$i{y?xLGFAIQc*4P2tEOBOyAqhhaa%K)i-`=_P4s$J%-`j?&jSv5Z>Wu>Z z2HCP%)4z4y)Y|C1>qk2wr=zvyx%}w|8tOCSJD_@V6{B%pMx!~CCb=SmNJ?02sjOEG zPXG-g(`AzzOTB*V`Nb=3N6QLEM)34R>I3$UaPw=`qC`rV?j7S;hcVqSx76qKsxmH? z30gRV(dt_}j27Bj6j{pNKF3d?B{f!8HqkahH9P z$_ipXlB;(;x$8n+|3#;ryNcU;^8r-=RkWNE9YlynxQb~V8z`Mzv5~pb>uL*pRZZwJpgh?EY`WR0+aHC@yK?Ca ztQN3`;BpLDgf?v5;+_<;0kPa9qgzdK+Y_J)EOjc) zq||4`J~}u;k0AmBL%v`U! zCU;#i=zTW_!jmTh_4=TT!^~pY)?}=@SLSG?>nzcw!SXL0c4yqI832|*X}>}7svq=v z6|pb@1-}5nmi*wvtm`YfH8pNhPr!u3i2c=}tJUot`&)q1hK=9YO$(I~XGyCMmK$P- z)wFF4nZ$Yy#B%mToi0HWUB~p2h@N9k>m*`Zx#r2Xjhnb{v<3RQpO6M4_mV1ZOxtHC z*FCmob}rA|SS0wjlTN~f8qqao{pCHO5wcc1rq)UdR>*~vY~Vol65t?1*Z>>}a39yr z`nYc1R*uTRb(AYCt8NGUEfH#$J9qU)2Uk}&^8Wh0-1e}7-t0iE-#iwf<+7A;2s9j9 z(AlfCRPw1<(vyb2rSgiS7ZSvdhCP@tdYjuW5g{_p2vgT_x19?0fz6-CzJeSh18n8M zL}hL_N69VqMxY&hLqb{gHD838mQ~Eqv+692KZee7U^?#A&i*DM!kJ8X;^@(|WUeza z#Mghl^i~J4+HMwQz}TF}&do8Xj!*i&*ngjqU|90Pm5~Y0Of0Jq^oPmA!4taz{4)G- z1h00yacvD@4$|ujx2Qw_xu#xMJdwJ5DppcLZ$1B0e?B6)e3jLM{XD>fz?>b+%dtG3 z)B2gihPo}0CCQGcEfl;H0l91x`)XSh?P@fZMiPY^DbNg zA`t{WBfFM>KrQEIymhA&wX0U$FUj8K$J9X%WXpL$^vS z49=Y!6D0ubHk^Rl;&aBf`MGQ?7O>i2x!MUEL*vOKc?7`9m`N*}r3qa<6r-9eNkMF} zq^`cz&tyg2m7Zcx%Y^S`c4`Ey`a5IovI}(MhOgA%lEp;%a=x4TEBe zwQR2`g*iN9za0R(Bsr5Q(+pO`fN6mNS0-klSBs0(_-h!L6Tm%;0A+=dBw+Gq##I43 zTe!G*3GfC90e~t7FiK!@E7YDbbx}h!aE5InK3@cB7&a*^*_YOT$%!O+Y+?hOVCeYV zh$(>>Lo%PEZZm*_up4(*g3KhQ)pzIQrOt7$L!M1%QT2HMNL{=B2Lr>_s(|F(axA$_r%KG)~ohC6J%9)gpeqPF} z??_pJ>~T6}f&Xf{SemOuYpWVeFY-;d0J^My{ zGXb9w#M<89O&;!>c6T}+QNC$8VL%TEv`(&5utH4O zXucthKu`xCie`xM=+@v8l0e!wWGXL=rH(6DRRUD0BeLe%&vC7zJgdYNgIp%JNbA_Shvc`2r!!-c`)>RN}#QhTIEE`!a zslJf{uhCQ%Wmx-1XacJi-wD>tU z82!e@y%zCF^H?-;&A3KOP-d7x)R7IwG#ODObt=O5`1d8wl5;br5+njkym5{NEUeA( z2n0L1q+v$b=d_H~Zs+INuY35eH#xxCRJ?Gq)v~RF{o)?+0dUFvWefHRs0m_wOl$_R z{9ZmMzqdVKlGwDr;vLAuDaEzs&;W)r^yF3rutzKLc}irJ&M6q%F27G;j=L9GyU(q_ zwHsmgj}NOH-)k$~&b~gN8(9aj_p7|$_7=?5-ud7_XUDg||YQ^?u2WBbIS~w1s1+s<3_!WY%98k_FT{bstF(R~Dg2yEA zvO25_Ch|m*P_~)ATH*<<>R6z-zfTK?U@vb>nE|5^n@~j@tXg`sUO5%K%Kj21TU_BZ zZu105rrf#Mxw+GFdoCMd=On=8re#+2=|x4ELCAQF8pkQYu+WNSql4me(7Zag zN=_{>4$!Tw-xCY2L(Fjzi^<2V(%(-V_hD@tEQrxuo;Qf7eHm=d6>GgdIGFt(!%Pzkb)>$JJ|I?j(& zVYP(;FikO6Qfe}co&!$??+r@4!Wf#$vrV9o`TGEPle7^uZdL$p!p*OG?16PkV!Usk zFT~4=CVLfMcN%j|)^#x^wl%V_tn^uoolToJ4P9M%3pfLoHo;N_*A?G!kHprvz<4Ss zO9_Ik9N}^jan&@Ma+M6Bx5U?8NpgwJlY!?0!`xg?1Yj(2ZW+wh=mzP&7kbMsj`((tvaXt=Ub65`dIRa4Y zswEC;pX+pSnH%(6I{-hx6MB>?r@Qx~soy)4lg)Z$-z2s{GeGPjH3`yR8N^C^IQt)_-~PUQu`Z?`e)y9g2C)S|#sk2st z^vwzuWCZH-Cz!%2&NL~YnxZu*Gu``>Aqy0k47&Lw=w>_&$0yu;0O;a`k1Mc>SSa=k zQf_52EmK^`!O0Gevu*5eG_6~*crPTz;q~Z0bn12s)1Jlqr&5l zK$Z;%B!d}-2+$8-=7@vMWx1*)`Vhnty~UanKIt3)(&79&oNc zJ|?=Dd;Gmo4mrM(zgLt^XK6?RA1Qndf?gIkwDj-mX3}#mPRd$6xsLB&>9O{G?Fv8N zf-F#Qy6ViQp1{*Vtk#IVm3Qy1%~5HCKj>eJvKGhiDjHBv-$~QY3~!W3|q1Y zj61A75#OiXw2Do^zLS90xTlx&e*!bx6qGzj-_N8-%j+Tm-VEk47|c+YOl%nMF`fY? zd5t+HHpU<~iJ3t%;UL<9YsYW`2eD`F=_2^+dU@X)>?Hl40doHN5~-8NHNVf#GpUQu zJT}`F-*!*aaO^=JHR!A*Ofd?H;scUTz&ZM4B3CEV4)#;))~}%UbU4}bXmPKP4^~Cv z;qgJ^UG4b@W_AUs2TSVe46St{d(v>p143_($A7()A=`K`t%EpjXmuyRuEeO zVj&AkR2~23Ahw=#lXQB6@YP`C zye8~4dAuuyYTF9dS-d2WmaKI|K=cvyVQqt;%RhrxGp5XKR1mz+ay{NHqXbBEL1&nO zpv|7-rVSAE4Pbq4H#Y;YOLXgEd@f^;RAJ1qIXml0mp83ez9%JEVi6D0w{h115P;a~ z6$mQ2uqZI>lGK$TF10h9WNh^_MG!L@Y^)`U72ixV5StxLj@iu_uw_8&$~Q4%_L*?) z)+95$&qUaHCou2+HCXx8H}3$1nZb(~+nBqMd2f19E0V*iUyp;$H1}~VFxH+tGcqQO zVcT|eV{(`?do_(2LC7X-Vqn1&NX{F=&>ppb4MYrVS9%Mu2&HI_EK(c-fP?0&Q{T= zA#i2KE+X-i z6Z!hEN7lV&U~!LTZQ1ZZ*>UZ{14t4WJ4+=DZs6Z|4r5|xzjp_*#peI){9Hk7c&OsO z=H2W7c+^{IpOxnZE(xg*fm?cF`6P(V7-P!-Q_tr|uJ2CsmLPlx0mSbf6WdzyD2(;z znSj`_*9kzohX!;Y+dT;fu&kP|wAg%|5T!Z&=EFKjgmYIm+u?@S``ZOl? zr+$n8d@JI*9Ed1rDBy3p@)+dt*J-%q75ugIO6S&@CP@piQv1z>TyV zk6z_~`gsEDG0!YDWHC>)Bq>Y`C&(5XnOa^`2Iu|B0DZT#pHH_nt1&g+`bpNx&OJcwLcWnAp z@cCcawg3hLE5_zz0L%n6#&4?J@!qg$PB2=$)#!|KCMGu8?Ph}^Kp=sIvpObJLichH zJJMuq03I0{8dzIntyWMGM+rb~#}Tg!tHAYW^*$R~CTe`FEU^hEeYq^$CYv*`EaR$x zmO!gnc^(ygDCPy(DV0FY8Q3V%!@9m2OPbkkGUE&>nE>W#B+|W})_ouJrHF$-@i{79 zeIT<=9;533FbDbsFcl9gz=ngLBrO~;!r(D*835KJj!W@*6|%7Soy@TtY&Aj|tQ9ej z3w`C_7RLvGHSklMh39puc4^}7WZ+il$z=c@i^P86p#(_$={iXFgWXQU%E$&cxcMz^ zwP(aPb;EW5`~L1;ZmzE7?b~+>W^a44%L)q0G(#?o1wQ%E+t|^NQ}*78h!XXW@zDE8 zo(;|~ktNBHlY|}+?%!a-EBDuB!I%{Z2yFX@>E8$;)1UNQmLd@=E55eFBVoT!N!JPX z#f5b*B|xanXMku5Iroo&1rAK^=j<`BOv>t?&mgvc@ciP!>cqmc#eIRg|D{;ip|TCf zzC~q(%&O*l=Ki0<3Bw~Bo@OV?5@5DtJ_pV=uFdymm)B)e!n2KuT?DQg-+Dx7qbF}! zqSuqgaX%V}?VZIs@zw$ElZ78N+!qZ(JFiuTlExe)1F;8!*m@?1`E%KQ_Jve!J$tTj zwKp^n7;?|T31Ec&RRJspqCfxV=|?}1UtYiay$^m61L^}?sLWbYf@CrjnyfcAuLJd| zy!q-v{`}9@#r?3Chl40f>osC$TL{N){Vt0O0Nn#NaCDnfu&j@EZg0v(Xp1rynGLqx z!i0%u>4_7b+IV0lpqV`&ob}wi!n*z`?ND*JvpUUiP-mfDF{_?&^+OLp>!R* z=8%KAdUb-Vq%xQw8yk$M=*bxNunj>VD9#CB+~6Hl)CehHID;BJ)m;3(Dc1lhcG@tK z27p{sEVM6tPNj{dkGolSzENfcNjBM%%$8N6d@;Z)nK||9; zj#M=df-z&*K=t;swaye42YOb5Yo|Xe7&@tG92F@n4qSmc2G<7-z>-Q9@qJ;dXUq`& z`>|me!X=or1%sG=jq^*MnOYHx_nf6R17l=ZgZ(bU*pOvy9IG1O@0pB9*3kxLDnW5g zaK%f2Rrd;{2rkKp!OLDb7*2+h6rTgHBr#cA;D))R4%6Rhw2+|6r{(0wEa7eyJqgu8Ebw23>~{n_oCw(35?z3EN*{L$NRkw*ED(__}=z214Ft* zSRsip2hrSTWSr;c$Fl*-pf&^8oXo}FS@lVQDdJl#Mz^;{ezMRPP1?Fw8Ku{l&6%jI zP4CYU)`S_0S4K8!7{SS^R`luw$Nkr2|Jg5!eDp0jt)SNG#O??n#bQxTiU+IY>I8pv z5R3imf1Q5xU&^nnU;YO#eTV=gPgZl`>I=B~0Gn>GLWzrK(!zw85fwpydb zV}S_H`s~El9iTCU+UCb@wcuQ3q8J|AepscKcE#9Q1g&oM+?vH{X&F6^Gt^45 zR%r>j6`kC%%u42FYQj$Tq7ln%9zhL9Ve5FZMP&Y7Yw<4aW?auy$+k`kcol?!nNbB%g8|4%g0$`j_gH$5 zioW>Yii6mpz`*-VVK1`ZV6eHRF+9Bl)^(1HXvmHu`kVM z`ki{%>Sd(OG9Qm}oSH90(o29?1O3v@IDzbCIsdxMpM4?i{F`!IL9Fz4&1H`YIdGSiHKmF*3@~i91U;XeW zM~j2u1Y{9RnJxeWlQOZVWw#E@r}FlzUSdn+{$VeNCoz{>%C`22=^LpE-U=Yk_F<)X z=C^^@U5X*0@K8`rOSDzX@9VWDt4LiyjEgmGX9+Ts1$lS31FI2)RXSRQQ>wj4!Bi&^ zv-@GqcWb8?3~tW861)ObLD#4Y=+sgXb-|{9aq`7egk9BIOTdX6andzgAkmi8leo~y z#sK9deJ`FObf3iU`Pgcql?=3Hm>&Z<1|HZ+t+6@*nBE=T@nTIbdIFcnyaKSQTMt?I zR#yp{?k&XNes zb70D|BC&uqC#}Al4SF5GggBck1V1INlbO5_KL6Xm=8)|alNbNX24rTJc8(7&+s{%5 z;58-LK*REEbG5ygMi%931G@cl9Q*1nR{|J<*otvey)zpVYan+T#z{|sPy!8I>6?ej z-O1SBa zIBR%Dk}TNyYv_bLH{zFP2_Rg%?#hP_0Z@_zIvPX4!18%Gi>@PtLNMF~hC{`CRc|&h z1tVCihdcD|^m0*7Oc2NxiMquuYZXl~$O1yYFl31kE9wi! z`yNi?@rlG`~480|1&{@f0@jbbgl+fk8e-I!UKC2)u-EToCKG``FjDxz66Fx@GRjm5*fk>2wcg) zK4_AbmA~ph)v5`s0QTuY!wP!bE6aX8GWNBXAB3**g!TmfnQ%)YgWUsd4Op`&Yckan zyUte-`!y*aeOubuw`7?wwDoN5;MUh@mjjE5t!;{?K!05j+vML(|KbPo57u9P<-?y$ zilCGsCf2vHLW{8@>fO4CW;r~5DzE=yFRx!4h*@JzAb=_ zOE_6?{TSQAz9)nWcv4p*b4eD~`T(CA;!joETW!$J0QW4$v$G4e;&MOZEbQ4eNWrsn zA5**AMZ21)3qdWiafA1TggDugx1$x^Y++#p%ZJ6d<=G_NpI|(<4<7?p;{F%^yDP7f zuhHI{rss@bbn_|01{`eVoUSFgR=NOm-~e1XPiTd!SurPs5m-!L2dg`>KBf9*!e$7w z%LCecmN7JdAR!w8TZ@_-!4N`FBQ}(qr*>}v0CnK(Ed0nQ1}L=alL2T?@mb7YfW$n` zq0B2jTl4kH3xLvrcaEb?*Ywgg=4Vb?qY~xG()xF{6S9**Y>aDD06Pj|+*=aAbX;^; za+i5U$iN&iqA+Mn-62l40?=Sh2?VPfgq>CEacIZIK&+RpJ4INJ0KnpR-K1fND%b@Z zTd~(S+eyS4yQFml15)iXXAQ;O;8xM&`MiuEYqQQAh>aVjo$&=(NI(0mSWDvS#+Emo zfWg2@EaJyW$E5KGmbE}Ote$TPHno!$@Nmdrh@g1|&cRH24^2uTIQmQgUi`X9Vo)b}KcTaO6r$U>fO&1+fzHSdtBFpX;ore)nu)nJD;a5* zofw*wBpS z1??DYcEC&Bc?>h~#XLK)z#81D?@|*QUh`N{VH|@G0oXBiHNYmnmlL}PVz&%;;SI{D zpNDn70kQl&$KG~u+8b>2r-In~_ozHzF|P(-BL+6gep-Wwv$J#k*&9T3JY?aOyLGJQ z*k4O@^;7Y|x;QS@@M`HLwaz}5cK$6noqa==-N)-BoQWm=3yf#WO;9z_K`cc7{|6KM z2R)qNd+Rm*@`paye1Km9u@Gb`^?ozb+rRDP>G2c!=}&j^?ph*!UsA7+`D?bI)9}T zX|BpxEZb*}OF5tnMJ^z030fzQ%e70#CgutwGB&MgS#uB@?lZ*wwg@NST;wgVNt3df zoBUaGhvv=D3=sa!fDpnDi#Qpm2~*;PheV?R00(T4(`@%fm%08uD^!$Jv;D3iQJLL> znzCa3I~}B2(*r>ZN(RDDDjyntKqUs|(_>>dz^$Yk@e$t`Cf;PA#wO3?Ft76p7pcYH zq6}6V|3Tp;P&lpD=LBoECO3IyR(Q<{3%CH!Z4KNC*(czA>0Y>*k8bM-vrzXa!TAdy zcH7U~HlK|g4Qrnu*5s5MPGn0h65%*$lg9eUE@6TeWAKWBf5cQ4jfLNVjom>X%#zEF zS#m3sDw4T+(&)0T%hDNHCU!MkG1%0Ca&oqzMRNAPQ@N(dIy=Bh$!s1=8Oe@hf`tZZTH7s!6VTXl z0st$9qN(?dIWf6hd_ReJ|Dwk0ROv;jaJNPXn`ux1d9f(#+B0!1@j%A}_8dv;OmIO!u@kD)XbiH&zsoAtpB0mgjhEkNXmU)z^vT&S2In zAoS`Och+Fzc6}-%i)I9jd$n#{kMeZ1$^*9f7X|*kRqy$V!_F`Ey4QBEv^J7(PYcPd z=9qwX-wAB2D2Tm~)BZC#o_&4&`Aw1i3Sy@f#P-Tno(s`G3u8#aApL8DSdl-Ne(Tp) z5G(R`f98i@0I{LO^QP4oRF8eQSkeE-@)s+Jy}sPZa!fPg%@UB! zrL=M;AT6OGW3iP1)N-t-7y?KJ6MI6vx0=|)fqhL@JK2O#*wz)Sw|gzg(#Q+Jv#W(+ zY4-jkqf%W5D_h!uTP?9U8;Fgu&=7&iHhK$6F>tFOwpf%J z8ypbCb@|xsro_gERhX@EAD3Evyd=plnJ{Nm0lst0t1A(NI^^z9Gl2D0@d&QjkBz#q zJ)XPNaLxIu$7LlbBW_g4CPpo-L7vC&hjW7SN|LV+DM# z-{!#$xMuz5d+cjxxiqQk^_6_})f?^Fr5zjlf2F}?TD%?lf@xV`EHSyZNmQ=Mz>p6o zY~8Y=Ga;2&1i&ql<(*#rX*?E<;mxhJl(h?H?PNQzY;$}O=6uT>@mvMg=nc9;{lE{iuF z;OB@>h#9uWh~qx5R~w(Pd7(@Y+jQ(%e zAA7zouE~;|I!J@SvU8Qk;`M^kC2FH-B{9TVkVWy(f#qa{c%KeKfOEDO{dZIVV1T?? ztEMNW<{x#k0xkNGx9cmkI=Gq8U4As9+)r zib__(ZQ~ZJ3YA1$it1HkbTjZ^h=R=%Exy={3u?2DOI2d?21wdk%DTu@$y>Gp#6HuHb?!oMHs@M<+5Qdroof?QJEit(IN^$1i zV|FDB96c96yMm9_Fec9#1^<1cBn#N^x&f_rF{U`D8lf#ET`Mox@D<(c#^g|bM<{`N zl$=BqE@?7t3XrBq=3*v0Yb9?%tN|MPsZ~2PCibALZ1E}uUDoW{4B8dFXOGNtV{7qb zE`6R1bW{gE#ei_yxK3?B(t_Qcg+xf2uh9r7KG!4LVjt{R6h?G>W;;2OzFT9Y8|2Ev zS0unpHP24rc}j5>wASP?s1+L+*H00an2Id%Btx}^U%6IyUF&#JXB8V08m2BFV0%pl zLMD}20Li6&3*NMhg>V1l-EdCz0)}#yi+A@8LI9OrH#ik5w zFE7;Hv{#o`^7f4}u|2flgggK>a;gE?BM1vSwaM86fa2BPN^vGO#LR_9?{Z0F;nfm+ z1=0vB(H#sdvmLL6*HTQ!FxH*`mV^t%TJ=yOIP`P9Ne%Qlbv)0B;t{TqJiUWy*H#(CN}u~OKWYbN2@$%rw4Ym8i>su zyE5t0A46Hk4XEaMizEUsVGj!*xKrk}gV?L9D|vT$sjsyX!PBFM^11PZJfwb(y^=%g z-rd1#FGY5KW(|Cp1V^2%Yp}Wwq(_$V`^SZ`*z5gHXYy2D$#Dg-J+Vv9K9y718{;ZC z<}(mW-S}4ru{t)b{bc&>A83u(U;5yOY!ruy3}wn9Tg5jCY5gKUTCIam<p+ zBEp$c=iM5)xOk=IoP3c(q59@2Y)mlrDHaN|>glQz!;)UkLKZ54=2i?_?{-V67u#-* zARClb13;qXL!hxBTXjLU>EaC!cr6~ABbiYhpJC*O1?`xbn-E?{tQr?6OHHvP+0K=g z?H6#LG%GlN>e+qL*;b2(txNM>_7g<8V!`S@I7gjVbrDTFs}(zuZH+l(1A)=CPFK8X zWxAjkXJ-jFcJ2Mx8b|112@{eJn58*cIkssErU{#D10ZO${F%t*aYO{%ooR%O@LF@s zF_zW@RGZs_I?GD~ums~gFrz?HDfH_6dmcd^nufD0bng z1||mZ8oxIz00RVKVFZ1%necNqx3;>Vt5qZD8Loogm#M%lU8C6nF3SwUqbyIo*}*~A zAHxW20}K|n2IZw*3_+K_l=o=GDia>#cH( zxXu9~s4%E!bcpfU6dK?MqhYXpPj_9`YGhIzDrJD-7B>!3oT`Qn6ilJVyx1IZ4a^oJ zYhaBoegfP8=+4+GP>a7Y)PS+kL@QiZt{3%In#}|5fh7qcj1DFov;mA8_)8Tj9DpZWtNpEr4lkU@c1mm&2Na{AL}U8~C_UCjjUGwUWjtGq6>%j7rimrpv=ChT*Ra zS6~ttjS|jJlqU4bA;m&BG*7aYNN^2!k=)ia;vxxBT%SRQG@D0lrc=3piGr{h6$>gl zJr*S7TBg0}cugb+U}Nj^nb(}ar0;))?Z~0`am_R!=(XVWX(IYMZq_{G;I-+uKs7*Y zcoVlz)s?^3`W9j0~A4YjL4Afbm!jYrmaiKT>KpnmER-681ip(SG7u%g7o2(WG9H7>PT%!Xxe zas!_nyqG0&p#Zj1>K(+Y!s_6wL>x_5L;;N1GWR&Juzh?cFc!+NdPcr+PhOrx>{bov zovd4oSWIuYCYKl@2e8~W7i=hzT%&BKLyv61rv#!pvKi{iY5~9<+}amogJ3cRHew5k zpW&9-nX*SM)QU~mVhfL|K>IxPc8(n#?7IQetnj>o*hR6p*<%1;+Rk$>-m10FNmn#y z=%6+c)J;|ccACQN#H5ypuQLUu$Lm?`#VQUk*k4^w$d)z8Tm62$_3NB!lp+SO0dxsn zEvo$k^qQLt%q74R@rca;21Q#4nfTrdK5aaKnYILQ*;&ne8I|xyji5-p&OI~9p*EKUCp0Wm@Ql^gW zWjiC*YyNU`OiT|7W?7Yzt~-uT%EqeqN62j3YoS+wAxH~qB?I(mG(o}0JO0P&!u&0T zH9rYBn1wBRpIVth3QEWo2MWH1O7TU?P0fVE3|}`aCC1a{#`H1bfX1#zY#(ny!hPUo!G^e#MS!dkL9$c1oPl z=)yi@wPwj9HS7A?$=1SZ3Rr*ew0RL0i85on_XTUEcMsyGS0}clFyu16H)355PuByx(-0r@X zyT`6N!%IW9WmjcZ=6*zYcr15c?tX6PK>-}0@!Yb9jRElryGdvU&-C!_cERVo;St0>vUBa0#6aaKny3x*f zehiF5;zbC<_sbvtwfxogy?^xRI~eE?$}x6=3X2U&k{$}%aK_j2;nOpD{e#HIPhB1^ z>g9UoAt7v+v>IuDdicZnau20V8nZ^1HoN(wi17VGa8j$sNS`abwmRwFSbKQ7OHcQ7*&I_YehzYxsZ(MtUB8*7-5$TMsX+-#F`3Uf|;Q4VW4x z<$0v1OYl9r#{s2)4R4I_A`ReO+ebP>V@m)d`$_OR*eF8coK;|J^MFuf#%VqoC_FIa z&V~_429pa)8;QhR7(P|dDRf1+?64;^J(Sid=Y-=n2eMjzOL@!3!-DhEf}Isk)+lB}GVpARR~$WD>s?pvH&pp>W%%%c@={*p5QQdH2lwj}KC1ApZ zi7*R2Y;SbYGZ4GPwG$8-pI8kIHH_5IVDgY<+`}u&BqaiHL_SorYEqXOQ7a+(uzJOs z+35-wq2}R+sZC2G>XGEpGVF@aI%vpER(i|qa2RkV%WVNQ)ae?bi>bIrCXWf}LHY~X z4`GwT$5=Mgn39n?I=#BZd*7A-0a2z3GMFXks^=AJY3Dr^h7ReSCCH-exAd+8L1iTa zJ4VWP@MVJrvgEOQ*>m*2IKBiB70OwaeJS7-A^Gr9X1EFCQ$Uq4$q%xD6H`K6|C2L3 ztRANBBU)X6?P(Rr-T}n^^7Ak9=BKyv%LZa^u5X}A7Msfl1u~k%OA%=P^veJ$UE}1G zSm9*bcPy%qVwB!@?#(0CI=@7ULF2+5Ii3}?Qct+U41kfc6z2fttHJ1oc$n-Xd0H@U zz08m8ftY`lN$+ww^YT3sR=m1)R@Oro8At@G_u}H%BQj@RRV#_ur}5#nnkN?Yx7vX* z=m9BI?^KXZAA>rYQC!S)QA=LhuE%5Gp@CQCiA zDr@@mE^*g#dwUn>bX>%1n?1#c?zQm^$SU^H*1QKKYtr~_t;`emM*qH3G8-IzYjAdX7%d!-X7i2bjg{GQ09 zC^GEn8Kur%kmr~_0v+z)}1Ykj_?))Xz=TEK9ws#t;DdO zzW$UTl7j>#mtNZ$z*Z1ziX8XL!qL@RY2gWGed^@~z_%MTZ?j&Dg5AdCh>MpnN+Zdm zlmwj%JVHkR{`{b-u*+ z!vb%~PZfa9OulPjfuE=K_BR*(1TVmxWVM4#xc2}&>MoE(Jpqc44W(Vn%O?e z-$pr)&B+rq6e)jqacqeos3%$l5fXk{hMi2Qi*pN28LG#|O1XG~AUkq7E#qW=^Kxb0 z35xZ4$u%BWLk2|5Xnn96vC4_)(Ai@~o zooOQhfc>+1Tw{&JrdK?Mjb6o(UTUn&*oa%X9!igYZf`vBv((p3>g`0k6LQIq@w)Z^ zmY*L&*GU4%VpRC8`q(c^I!`=_#T_EP2WflG)$%iLZf@nrpI*z4KmH`|-@TLD?X|bW zJ{YdCww&gVJV8ZXN2RS-J^r4e%<1}5*nr(ThAo_c~91DS`k8oLDW3yfCkW|0=_ zV>Eh3Ay=%y+}!Zr>mC9ije-U3OrWPJW4(TZhP3n4S?oXdArY`$n*T&2nM2g#2DPsN z+_#2CM%jFZl!vr1vpVSZ{^5yV*IU9)mTGoa(zyfN`V8nAZC&3_H2*Kmco-MnhZW(g z1Uyex(TofL8HT6gZ{giakkSAs;@Tu-mz#NfU)rpg;~ z70j!{lVFu-l5D(rlpbT-Y>Sb%=)p7oY>HFE!NnYzC}nYZyDfp z2L?l4%OgP>jQ_y|FtO+n6N#2En0k!2Gx_?61s=Ry(|=g zaSx}8Ij-P=wYtBdd;NF>;m&zpW4y!j>TjfTt;$$a23#+m?G|90*n!;GNf$GETbyrI z$Ig``Bq)|&?rx1j@X=ZvE1lr8Rm0GR8+_1FWi-EC@)(^?SI@i>fr41|#*z=riV`$h zDT97LD<=upS;t8o6DRq+K9}1?LF|8Np4c-yvF54o=vF9AWDTtOd zvU(Dh);3+{`msEJdTZUYy1<)ID=5Ss(i$vHvE!Nt!>rN-ukP_#bNt1>8(;#~(Qi!Q z1YYZAm*bUm0xGpiJm>G^<7v2mVfQLfGrY+3QM_HRZw3Y&RLSUhFlXcfdKS5tpz&6; z9$JY7zObo)df}gWOXBXy1zx`~!y5#w?g>xJdI3k(-ajI{y0u51*h-a13wepsT9en8 zz_}>tf_3tj<}hWA@n_@#d`L)k!(0Y;jG#3R#UzRXo;imIdqwgK=T7o>&NCmiL>GVR zzerHvJ;4)Oji35A!J2HXd=H8&f5#ZX^S%y#H^cCHFPao+K-D;t&bd6m8*4md1F%d$ zI@*8%x{9zircwfQqf<}pMcX}Z8-_j4KA>YY_~q-x4XD04nC%#Na>G}EM;Z|N=B16~ zzS@tLe7#~k#{A)R4<#=JFy&Y_j1{xRg+>gN9syshd;DkjPcT#FzLxj#_mXc}*ENAd zNLm2f$MdXc!5R0SR}h{PT+6^kRW2j>uuRU0*NsM`_+}X!rW>9HfR~i@mGf*r=lp{J zJ|d5PN|E^r1n7z7XIjWIs}rD>C&oQGMB?lo)$<>TuC_nx1gM3zx z*xlJhVTG5zbvN7q`dhi+MN;=xK6^`)JU?aDIR$HbcblR@kUdSZVQ z#O`6y9>luk3;v15h5MDm>XjWUysz%r=8hOkI&n*sJQ0qA%*vXe09-($zqGdK ztDZzxuuci_J!xb~u~urrmfs0V7kE6CS7Th*s3PD%(+nyY7z;eO7bM~}oYy!sXL!bj z(+F4?-j9jrFt0o+*98E9=*vVJr{Fa@?mNiF_t2<6m1^v37f zwPEc(kl`U}cx|O+%EmDym2#~fV9y1Ai*RbLsUGV}!ZRIvTmT||zZ|nCI7tPRbOPDQ zgtRuPfY|s5;&$sjYBCO)?*{^AaCFIIIxiL3s>3&Mr;<3OrzSz@0IUcpn+MivJ02Xg zn#{U4R(V+dANDX6DC$OO+uhlbj;EDBni8V&0gTE0p z4lY>O>{*oF!DPpW8o$fq!G$$iKc*SgX7EgflQfxRe^nm9lB!4~T2lIR>*B77DL)C6REd z==nSs3)P%{{Tp3oECh;BKs$43*ulJS;=S5gaRO5gi-@lHTCpk#|^JSrA0B zMk>36vD^eho?J}UtxIVH;KLs1RRlN0t_yAB9-=NS`e7SQpjdq(QMG_x*jhJluSgpS z?K{qigG~t^13hIVr`LP-MGlaR?}t%?;bU2l@GuJZ$q|{jIn;dufIQdCJ*B0+geNvi zZN=wuaeGbJ;blg2JWwjcK<#;W$v6QpK6HS)VHF3gdc@)ehJx5DB#vp#SReb8Au57e z$U5mGpaVoZ&f={_3)4{61@zbPke&%JzC}e;9d9w@eiOip+Wu1VQ9+^xIqrApuDG9)$hU`eIJUi;Kkw@~r!%Nwn zS1KKKdXN6}`S9{IMlCq_YbJ5ALPX<9n?Mh_fpkGamo+-D#4bJGn&Z1-b65o(yCf1< zvg4l!Ig@l8M*@qWqC%|a^f>bz4fVF{K1*el;I4!E6Gl0`8RXY7t=?TfxzUgK58^?u zNCA7{v0H)1QX%;1*Z3|_Tlq%m=1otuGQx!pnC9!L6{DB;DQTMsqZe_DB9yYS7mGxaT_Kec zoDR-od2oHV@|uq>oFl*X_+02ATp;OoWlzezEJNn8CTPf%vnBOiMrKNGL{9xcs_xgy!W;8C=CXP!-N2_bEj;F|^WA_a__Qq2qT3gKC+(&Mr=N`ChA5J|QK&YP8lA=n@d;a{nRb#qB zKG1SN89kd@u33rb*&e@eQpPCzj$*lfC12VGV%yj9W%)BXFJBp*K%gglU9|wQUM^Im z=2bj?JT@foxKj}OKW_(k_T<+nNaeL)5<{p3V9OM#Dr17!=kn>pSMv5}4Ua$D#^(7p zfl7*Z5U7vJqqMfI8V;1^CAF|Q>Nzc=Q@x?<87_H(-CPW8p=a5n#v6gcQm?^KwOdwtM9P|?fvE6ei{f_yalA}1c4<;d@2XL z62(i~INENJI5idWN0l}=Y%tw}dmwPG&#Fj5A&gnb)RJZi6TOC0 zDHB4)WyiLS+(E0&Cp@xhjI0k5;oQSEMyJ!_S?|NGM?I2xc(pP?@y1@Aj*y?qM{dYG z_1b`;fK~Y|dH=^{2|&~_b_-h@_s9254|aHA0p@i6h&7m9X?QXcv=aPPkSrqcyPdzo zpU4`e>fyj{M$lKodLj2z2o-#D-eQm5DW9TnrYF5>V<4N~*Y-uyYd~N{v1BB~fnPJA zYK9BK`<|r9XbAw9&_m~q3FNq+t=a`%Z3C?| zDhOPdh%w*^>zwx`BO#cFv976?rPQjVUhY|Ogy)G3H9RUbz+8u0aIfcmxrXTxaj4se z-x~fFGL$tv6TAx66hiW#WYD8JFkZZNR$1gctU|sV#;lpI9iA_&Lw?uB(B*gF3={2T z!0RrVKpQ|0^b)NU)7#7mw?Ou|*gm8(Yze~%>z*&pjs0}}S>C^UFF$<$le~HT#`v&_ zM16f(sW1`S-~3)3q*++CXLFb^LZ)ZSa$zyYU0C(u(2k&HEIV@bzs#I^z1pfup92#fUNU~ok|dt#Eyi@vBtdld=#J!17V=r1;^m5bOL!PpT}2n z(|+%XUCT3}xo?}L9iod@ir(&(1>^T&gTNdmDmFhZ^u+!qh}{F&kSrkQ0I|atl(wC{ z-$3lm%`5r%;ibI)prxjc+uu*_jZuyo6a}p({pn@0KmpisQZA$jJS0sZguMs|>$eu^ zhm801s7QhbW}cE0%fzk&3hEmff3tej^0_Y_*nx69y(|E7U#w!n{XIT=PS2|EfKPa0 zDY$M0uBCnL+_ts=m}w>~F<6k0keps`$bcl0j&L+L-~gFz=9SWey&z+=dWRNfr*jr)SqcoghB8-G>E<}+ z#t{f61QdZq;-*LQP(f^z_F8CYNF0N6wg<8SN2cROuJ-iVK>#gHCefou6Mg3$B#~P^ zq98Z%AA%qf+W?FJLDh$YdIC--^SUnR8YS?$c5HA0kIiU3%GZ=d9w#yob2)<->y=!W z0V9B!w5unAwb;C=Sl4MRsIDW-CZmMhIUtzVyt~kgt@W7Kx{f5qn(JwV_nHQ>4AfdK zazKwU4JIjaQMrpFfM0sQM}YF2!hw=bQMb0COz7GI2|(_nl#u$72nv~v6kXn7{$Bg-}Lg0cLQ)NJ?|D_O0* z>;n>e&$izT)A!e(uB}_wk3YVacfY)|&#h60z}KkCtybfBusy1IBx3kJQWOa-QhenLPmY+#uY*AF{5g{4(`7l?d1VtjT@__xZsJM_dWc+c$@J9fOae6 z6MIK{f;P6>GxZU*?Jrqw%c|}~cPtor#dDiuk-9^Tqnq0sx!!>5=g;0+R^^YJ-jwr} z>&HdSYQbtm(_71CV<&ec9_<_#=nduqQ4Z%lWJuKT4PB9QecmwOm-a%gHxPTPAolQ; zT&(RZCAkNFa5}+oCy*Ur5}cjf$KtPV-~E4r*!mMMLP-Xe_1)BksU76u{<++I`Q5gd zuZ&J`zFcjSasn?7z^2m+CgH)nl;o^X&J7M{D-@iYWQBIVX|}Qfv=9jOn)b1THzHfh z9mDg=Nu+?YeW#Vwx;Q9xtJ}L}w2=#{a$yN#-2-(t5;hl+-Pby%iD~8gR3hsWEp#8v zhvc~1zZ!$W)81KKc&)XZ0&^;Kc+g@4Xfw(Dw-R}3BAq_GYJgKP7SxL6CQiVvch2!t zP;XwSivtPkjH70cW|ki4Rdw%;5x^H$c$}N(+u}2CtTjEjD&1DfHTRa@X6X^yG$q82 zUGRQ}B3gfs%4AAF~|0rCQNDK-;*3migpDb^8PJ62O7JvR1-< z1RPw0Wtl^CDj-Ns(+rGhtX6MbY&^q5<9042;4#x*B*o5^W&$Sjnq9!ewDXaJj6}Nw z3+E$&Pd)3#XUo>I4aSCdNn>dyc$*e%{M?vvO~z}qIU_|r7+ z6*LmOu4oh!>VW`RT7;<^svpFfZz4_7&}uPC)Iqs{H8*EW? z;ZY_ig~3NaS!o5VgFV7SvD2@9r54rLM2UgN`Ux_VIwIU{JBjPwE`XoywXy${y|_8D5=#CTEdMhwrF^NGW+ou|4wn^Opu% zl}2!N_1wWM#=#nl0LSRJ)0^%ZU{Dc;7y$F_0Iq({JhAP$T(2+96Z^1ywXOdv37<`v z`v9?e#Uy7BEOOb6@AR_8d+kpJvHSPJqN^ZQ2x65^@LWFMekDJ@ImpLr?bdbNCgEU- zUF!y7^`hC$Vnxwtb(51O5WRhI{s_IT1xP`N&hX4y($}u>0KK^3c>$ns;Cezf{VBY! z%&=GCRiLF!7rZlu3rp0t1t8&q+VfuA1M8r6(3%=q;)WZ0E#E`{;msN7aZXOmZ_S5` zR!B!I82>3)C|+{pX-^x0#x8umJLlv>RjBzT!~=G zss`*>5>yU=s0walOrZD|DrAP-LMcD7AxVO%gbgnLqtOeR0Au32?z~HJ+#!=PoDHI+ z9dNza#yMthAdeVy0x7()av!#?840nq1}?asi5$re8DV7*^O$%G-;Q4AS__uh?7q3e z(?`z=?>j=Xg(D8A<<$vRlPzocc?MlT%6K8!%)#yIiHemI%!xtfRwDoaKh?U`wB)J9 z6*Q{n@Yi4i9A&H`K#q<$WBR*DZXnT^qgQVmxbZRd>Dit=K#%7=_Qa3@&;%xf262Gc zobc(8H_6YG^XBo#q-V&fpmD}y5s2FTMpuWH^9b3G?@jAw7h0roMuTcFHLQGt-bj4b z*j6izGy+0gcdVJL+M*AS2oCeWo}*gLkk2=_^7ie|^2^V^%Ii08HZc2P1F`p@$Dx#v z8Df4N_m@j9TP!6e*TGh_5nH3`$5SA-g`p8fb5sWCg%YtwlrF!Y$J^_p^u&&k5AO2; zVh0Qx?*BUw8}cMAzT-M1ll>Oxg+n+^;MAIAk=n527fb*)#KAqWOSX))`$we6sbnZA zb>+?vduo*v>gJ_HB8nkJ0AGPx&BU+!C=H%U4G8@zG=#F;7M*F10Z<)g6=A#fdTeVm zYe`-AC2{>Cj2K*9odTTe{T_~9T|)Owb<7-32YIeZTvikI*;JXp^^7MSUWhrCoDUB4 zq+jB{8Bge&_nYy4JFev8<&}JvuaVgGtO)NDAhsm2mk-V|&)-K%@OEAUv47a!JIE7{ zM;k6*_e?U{gVVn=Ap8-U$zXly{%Y~9s6NLW{_CP5>x`e`dKw?#G` ztx^Kmy{9Oimk}VAK#Y&`{UtVEdK=y2dW>cVtX)Z8CFtSx6)U}?ZKPKl$f^_z!k>J9K~HTv9+Y8pDLVj2Egb-M$;{R881@o&XQ(Na zFF{UOO{x6#x$Y9KDsY9O?Kv3}>}loy^ft`~OT6FOyumka{T!?r|Nkc*cs-Z5O8#u*{p3 zal!iRW28F8_4*C2oS__9P^Wy-68v#J`?a+2Ty}6Bwma8i1+g>jXG$l8Gu8{9Ao*-i>fg z6~s1~B8jveoDQSjRqfoxTDQZkf@#cb?KK*W`$6NY680{7yBZtWBw*)+ z_y<9(1V9ZQei?5Ys}X`Fu=2vlsyPw1BFphfKS>$^5}N2BprPNZF7x9zaKs;D5=2-; z62$g&V(GuA3vcb^6T}*wph2g%P?;1Mzvh;dzsI;kW_d05$-!HWD*? z5BbSq!JM!rbjcgw7(8rn@@Aam(*|O1q)PU7Soi&xI8U?&QuuY}R}ADIg$m zOP%#-Yk4ngCA*di3?*xH0)*^8j%%kUwo)Vd(x>}M4MG8$=ZVMXa4_7?zfH$xSbaw& z;Ai(BC_kqP?f^>0=}GHz0hdV6ui!#G+A@$-F&t=UP0tAFZi`KMt_~1Zdu`Ub&eic` zZShW^E}TFEVTj4cW?%tV5Ze*`??@g%2cCvz2ISCE8KA3It=rd2#%p9_qu@T1j8e&G z=6odLQmN;TY}e`(TL7JrUVi^@f{6e#@of=$%PM#YFzS;q%^_tcB!`N3!gjf?E(OTi z-zobe6}ACZB%@uBRqyu-+CXX3%Mxe=v@Cm;<9`rFSiK$af(<@t1*C(R(p^|*v;p}6 zoPuvH@Wu)gct(beEtw06R;IazbF$%{^ARfd>7x(+Y?JU1h8`?)(npWEi1}(B1zPcs zM&1CO!0$l-27R+;^^XPP8NB*FtMEOD70B7(bz?)Tt(ZH)q~vbXuj=djkhCM{UI$qK zv5-whA$l2$c{3*fp{x;Umg}U4cFeN8|C}E!Jw4Ufpa(! zRT)rEt5WXN>uUAdb?-g;SRc37_BTsn_vooo0Xr%XF{$-}Y_QHm=fE5LH6%I4#Vu=I zV*{~Q@@f4_Zsk8SI)Qm&8zK&4$BAth3IAn)*!0NKR+1ur<&8#9?30hDK&+#73T}l{ z4Fho{XKgul`%2!vIcyX5)iy~l<-B?(w+s~X?>#IF2bTcC$mEQScoFm74V281orlPP zBCdhuaVY2k2uGHcaTmT@BkERHax=s2`XI+dfvfg8Ryx7CcG(bL=tI)&A59ek$evw+ zs3nfR$o8ElfJNU^5_U}-o=UKKQmACMw*CJ7cV_TcPg^~vht-pxTd!vylB5U5yJ96# zJ~(p2Ij?%m)4_XTtMbtIZ5$0~s26OF8ltY?JRGD8P}RGJHq3(|%{BrH1t~Z7;3c-M zkX_1Jv8niG>zLmBg&-;@1g`Lo*kGcu0RU?Q=#`~Sox1e3bY{sj89@T564b=X!-g>k zT$;qDnc*@*jv;|&aE!XXxZWZInVS_nHrdGt`e1e~#LocAh7n;^Q!ZKl34AgF<34x| z38Gj^>jJ*`2mr_-Y4D8r8W2!3oT`E51&p{HdWKYcPs$SL1>Wfi^BLY7DG3~cq%HTp zI#q>W)eG0s%VsiXG_nFfdk6R%MUe`GsmvqBVDy|L+L6=_ogar>`(d!YF{VP^H@#Ps zp%UdeKt@}5u6ibK2|YXsM5XA6!O~-mK(2YVwxmvx!Q?@LT_!OW>P7*vIVag`-o+8J z4^S^^7iMq^6uC7*11bCt_#Y;nGVrvCfj@W zbW-4Yn2>2AJlKkPWJj+jKy2!dy!UiK_;Vn9J?jA;qQ$%u(^~CZ1Cmgry#ui3d0n0W zG;1C69f*B&u9lR6R=_NsC=H+}J;}q2JkGq$Pq}B(J^bGN$%Jgb^t=WwuXyYpW%I6& z$7pEej#ve9t3`8zLyPkyg@Y3eKih&5%rN2*17s87;pEq~ML{p-J!Kk%&%tBLtU@&?|&$_3*!i@U)=x*}HVc;;yvDT z{b`)x!t&#La{~t;Ju?4~VOd*f z#*nj@G}3UYT6CT(`*BDgA^)&N-&-Z9p;%uRg9Xl%LeYtWyW`OMEW zz8lexux+zxr%CBsa!_x9l|yiB!Lno#>g)j+d3A?1irf!uzYbIcVeS>4VDKwf=UByi zXoK8vjqD@C+Q0m16+E;F$ioBLbwWoiiz&-a>Hug3Hm(SliO8@EppK&3*y^WC$t3P+3Q5x5Vi!(gsTLz z9nA@X0t29HrE)qwuG}i%v!NT)6KT3=oa5Y9=4WI>3{Nn>;R#o(>BrUTbZWcL zKmPF>d9!VJH=l0g=Jvs_4PBuQSI?}O$EJ|%eeG+n`x+xd7Xwet*We&Ou!y_ehe2Yd z;AH}o5WF)y+nsu+$wPKLb{)wY2T&D+x+afZ>Jl3pM2{fHsKS!4PqOPaGru#b`M$G`6i~NctGO>t;@S1(A*&|x$(^{5Mq{VCUr%n3cpsOBPJ)Tu8g05tlN$wZfG{Qscj`2JS z8+lFxiJIH53dqTCeev>@Jb(5=PMY}q{7PQF@ZV1WwdRdg@AK)D9@x&#q=#6mHvD>S zoYx|dt;c$$kug>*#*&QYILp_~Wy5keXZ6JXPOjzma^E-n*Pp@JqKgr`32v+tr}jpm z33+dURjS0MQtMhRh_VtA?{WA5Vu$I%iCviUccTZ z@42<-v!sUm2Z^K|JsxXPl84dUf(+{iVV4?Nkezps)+a-+)l+LB?C!@m&zpH+y~ytg z1%JJ$|J>tPH)gh$tu%%Ds4d~@W906V1(+e&OiHWa`3xtpQ2gWjtKo%e>D8b7onaVy zC2Odk$0IGkxS{N2T|ygs;yS}Z9Alpd|1Oo#GElGA%4&JZts|0nq|w3zPfU-1Dh098 zn3oiQhE*g`=EMQxwkm??oFvr!TZ6> K4OqWU z#}hIcZDhOqE~XG_?4DdJ60$CJ4b|T;oDnF6AsGfo^#bzC;T?GFW${C{m}_adDY@0n zHIqIjk6_S4hhvs{vHM5L>bk9Qu9X&R*xQae6+>VU^q$;er(+!8RTYU6HOrEnqsV=Y z_N6Vjp`!piB8~x?0F%13<*XF2VqHLyJ|trlX*_@TQlewWMLdJDOe@DrFH`OA9<3Y$)J>qlNB{;*Z72dW9|vv0??V< zV#&6Vr37&|#rxslRg(5Wx%cG85lO=<9V&J=i%|#^gj&)0PoFm+`%%96<{RV0e*SzT zcXz7;dGltek#-1RU`Ai$SGZ2{wLe()J@2nrPmLQ1)eJ$Xpkf&u0(7@J&Uwc)`7 zqq}`SUVa`{=}0}@V*EP{(<6ZF6+Eog=qK_-A`8IUW$XML09X%b^1#MU9g)~3xn9|{ zB=!hgUq(CK{Cf(At=6&Yh0zJFHz0f0UZP{yz&K*gN8!Y>M9JX1g*3|u&?>vX`ELqh z|EKNeA%%Aav9!GQl!KPTiVa&YLIvWuE%vKzF+Y>{zbbhBN-huDiWb(mmb`-3h2gPK z&?A8)Id)E>M+cxm1@lz-DQ>tWG!LP1a<^v!s!RRFiA9Mwp-<2sUW_td^T*OF&SQk< zrA%VO0V5-OQXG%xp5s=3?`3Cd<#T@Mkw9&Cy~#_E~dw8s)=l2bStHO|q^l!pu!_4bZq-#c^^kI9yGd*gbGF5Za1d>pnSrJLc;!oN4BT`8oXOBBVebJL z?2cEA;UF|FdZo;b9}^^rHm~xL;uWIAd2f>#HS=o5#?t=l;HizK_Yu6*OOt6*6jo>*wiHc2fj+SfT4wl?EjDHT^e(0s1NxP&_s5i3m zajAzZ24FQ4$}QX1TQw$Ydaz0raL(t24uXm~%#O&pUl8dcosZvLtTR{Q8ovi0i(}-1 zcx!lzk{&>Hu^a@KYtt7CKr7Fm6V8=20r%Lab$u)w-rIod$Mxj~WM6F{_J!4nl~xRG zG7HXp&&GQj?DZav?@uY_snDcaAd}on;ppTCSQi%)rU{I7x;bg z+*_?J5Y{nR6HwxM)|DkJ65KwuC_~6(7Ge|1o#F}L8=l?Dzrnq%GTj%C|65>{pM8pp zRK2(Rws!pPvZN=r9wTbnaX#Q353cfX7l9{9HB11K7A|K`$U=rcHyL_BgF|=rE^V^1 ze3>WSv3S3&C28?_&oFgMQbFwDbPPj;7Aixd^x_7H2UUWAfB>Z?76d;tyc_~AxSYjXVnp4I9b^l;_k-jTukXHrpuIX7I9Src9uY+|f35z{qh4l?Dg0VErjY50?4 z$2n=KPr}o~??(b*^C&^fN(j|!Rp&T)v;ZhF5Z0}NNEu;Qo=Rs)xr=tF)LA%R3(pQb zBjrAX@&P?i4(oeE=plKuvHN>47-a^td&fbQ<<@o{SbaUPHr50dxkBqtjM3_^>6u5; zM${uWk1Ku;PU7k%wubRx&(`4JmtTIB@4o#(-o1M#_jeEOG@w9CR2Gt&zXT-A_~BJiwMS&t<$qM{2}VXvomKH2YNKDFV+>e-+Sg<@Y;x1 zW(_*Z-(Xzt?w_^hovL^#v_g)RJA>Ao_(#q%>RAHVNPfp)c_Q_p?9t&?31c=&_Z^nV zSb zEj;7gt8y5A3&h&^d{z+q@8l^EOX>Ly5bF>3u5p1Xh#gH6yH0X_y*)qvPHu0W$(JvO z&GK+gh#qDsB`}O)aPbg}6#+gM(lfHoJ@?0VbN9v_q9hgtU}Y(Y(I6z56?j5P3H87_ zfPI#3^v(&%$^vFBnZ@*wVEtCcubAnis(1*n0iS8nT&?F28>&A6<3WhFx63KZ!HkI&}ob*V~ zJmbdlUFf6D>}Qi3Rt(q{pSI0o2bKd$GaVxYn;f}`HX+W3Jc4B!zw~6A0W0D zVC6u-y$)R>;*MiTS%5&FMbSTgj1p|Sr%3$4(m z_a!_*SqJs01P%Clj2?c8nz9~N_gZd^)d&s%vByKaCq19;byX01i1`*6Eu$k`4Fg0! zKdP6Jx2BwGa(;O&1hC-92E{-Mql)9ej36e~Sb`5I1J=;MrV`B#ti7_zn#h;iJ9+cwja4Pk>IB;G;O_q7K_$~) z=oEN31~4QA2PKG$$1Y7B>VZA{Hf>>JT|mQ_!P3gA2BX2uN*&Xn!|2fq;7~-Rs*PAT zJMbH_S3X2etmd|mz+dO8?or95F2|&vkM{&?0LNJh=q<;#`2*<8{h@@(ssBOiQygIB z8kH-mc^7nSKsTvw^OmrrCwLODUXaN{G+aXKT_$XneSPY>SMJ|1hL)JL7VMdDml&W6 zowF4*U(m3G1(u?MKPoT)MTK7@*s}>XHV{aM#wuiL-PJzsh zM)hx%ugcK<6v{8czV}L0n!w&;yXUJ^Y6Gp*4)F1m7uUuWz+SI8>+3FYt&JWOEQ%E& zLM4rxw{%$m8k(2=SPm&((LNX+frdo|#l+7MjMx_4+}b2~WvgHd!5$VUy&lP170kM# zW1Cufg~`!O{%qFc>>%?Cw*i0~@`ogoT!fzY9Pjj!5d1!t3GQz3CoH+D96ULGNQe%B z({8S+wLZ<}qm7(z6NoWOn~6UyK#>sZ*~fgHtf=}i$n^v$W#X0@2dMpO+3t8>YJ&)r zC~NT0PIQ+5wFI$+y3riE_+~6oHp}_27YEb^9(C=U;fwfGi806bD4% zmM1;2Cr0p&i0mEQF`~NO9BB_8eYNi#JxgCl)AD4@v32oU(A~M_hBvGesbht;MlM@& zm{-&mu8H830pC-dN+2!sSStO{Mj7`*Pb@zZKp1i-Fk~7cR#q{IJa0k8r&d-Y#?)yq zbb9pyax+>!c}OXDv4p(std-chyp&J!J2X2u!Qq3&sMhk@P*tVuOgzwJc(;jS5Fhmm z`A-3{?%SY1x}i{b!v zQm6z;!7KNEqI)-#6iZf@;9;N23Q=d*se$sV-MQ z;ZayUaYhOB>H#+p15dIN=^HIaJ;#MDF-MdoYmmE%=W3Yb5}Q+R1;k=8b8*vp_aft< z6);STa%WbdfbC}k(06b)58=s;&~{l54p^45PWB#dnZ_}`JH4E}_oi%027QlAbD41` z=X2*#og)LqTk+@;^83Mc4Gw6YGdz&u_%;$Ht$Ie*F%Hmo@Wic!WwWdUm-ixo`6XatQIyL4q?$1R&V*F@By zr3W^~Spt9wt{DdtZ0AVq@}Hvs)-3EUvFSvl!ekSMSDXJfgliVu!%C;(M)@a#*!;bO zlLx}Gw4x}__&Af`>i)p(VcsUY2wC>qo)c}oyc`m_l=r%V_YBXIj>B2Oz_v%cfB!+g{pMS1bnxR(ujS_E9sst9 zm(|+zt@fth)`5bCdjmjY0#xV}^*F!)-2v+4aH-dAx(StUI8T^&<8!kLM%kMks5=x-`F6RUuVoF8yR?#^ ztk==|1Y)^w=-zZVW-=7_4y(g@Swjc-XBI1cY1N8C_qA~5Sj6^nE;8l*?>YJ@j(0{j zMw*&4!9w8p@tSKiR^>YCnFEmX9O-#pIC}xK)hU>cxumg#ei7HXX&}Kohao`Syso6H zC9YvJi@=^b9vg_g-5}-14a8m#BzE*wNwG{0=^ep z==z}GP-rJ7=kQV+Ze4iyC-vZ-&)IB%?sd)HyV7F7pc7b&HC-qSrwni~o0G1sC-MEg zCxZ?4Fi)Vk2a*-M{zIYvTC>gzGmP@LnIBZOPFh%b?t1(^#bNIiW<6Lg;2P55IOT>t zhmUzY7G_rEPETxDh%2mBT8&(2&G|OZti|6k#?MNh#Mq%k)A@{AzS*a|A3zDfalHhL zj}0wu^YWIq!MqOZvE8zy*fh4{I#A1|aK99OFT9B0Lv@$)QA|L3(*`WX^s?5?S$41I zyk7@1_&wkD4N|xUn{U8_fE9}y^BN`Bnk%G$A`?I;_bf05TJby;8MMELCv-N5Xb`T0 z-h&G~Qc1axqbCO(kWipB0tKhi?P^HYRnA>l(h=_0ussm1QVDYEfZ#p!?q@Fq!%{Zfv?HEE=G^} z3~P3>9{&KUPD_3_WmrJV=;b&aW#dMtA<)7Qq#Kf++dOZ}v2}@x@Az#e$Rs~6lEC0q zjuxGQvGLug9GkN3#_ecD-g#EuSB)TWz3_Ufhdz!@x%Cx8jyLDOkrTm$RC^!y%E~YH zYysNsyGq$vy+VLMu4*c_4;ks`v+3OB0`0QdVi&=wdjgnbK zulcYgQ0D&llyhY-Z%)^mWj&J@kNH{Zr>JtkMbY0b)aF4QRXP*)M}**M&`g!nln+QSv%?6)yO{ zD2V;5?Zdwhxv7r=h9^%f8WOaydchmj>I55zJ+BA(^5v!c`s=f8p&#Y`a|an7$DWJX~9@)*fq;l7zN-SWp#GmM<*~Jya2FfN|#MqkD*>ywNlgr zJGGbXo2*&^n>9?#)&G8r&1oS)vz<%6eEk9Yy(|X*m1TO>12q0Em;kx1B zGG6Hf{K-&~d$D>E-Q?AdG9IZ+lMCAd%v0Vydvdj6%q_rU@XQ8FFxL5=+dR?`o0Lp! zKSd@2RPkr}qR6A>JuOofFui3v?^amL+@#`Y(so-M|5gDGWvH$@PZ%jzulJ&B1a%0N zW2W^m31SVvS|_gb#+uikZF~<)B-h#bf*Y6~rtAWSopux+y8g*amcSJNj|~$A$t4>Y zPxI6k5L3o^s>du(th+EysL+%0Gz8ZLIrVn7@c7}o1fqMMUIhZAv&^kH3m9lL$a++4 zh~&f_1!m&JmaOJ_Gbx2Vl6c+7nhmfKbJJWF%lF3!QpyCTxcM<0P5BMrD|z4h?<4?z z>`9zKtWaG9=Q&i!>cEvCHcmUhXQbbAg-kHnY62Xfb>VL|PK)ksU#vFl&p*GDzy0_B zkNou0PjdbFinq-|ua9npBW7uM7W4U^;VN_s{gO#%Eip8Z#MoyflItNNG<4E9A7^aUWrP zEuaGN_}uf{1AFqhu^QtiU;}z$n@4oDQA%q_B=)XW%$2F$bK6HHDRoghXkro6en5CpCpY!%(I?j z?S3cF3Cdo<_bA)4n z-ZQ%YZVoFazkG*3c9ewQdi)lMtzOtF%#}_s18OOoV=hjfgPvF!31X|)?zng@*iLy6 zcwG;uAg~>P#{h*Kq$L_Dj%1H+d$??!K+u-hi|I$Ml)N=68Fgu)J=$`UM!i* zGOgD*PX#pm4!vbhfmkoiRTbmoKrOS`gPSZP=7e>Uxxw=o?&%Ou>QLk4fO2Q2qw!&n z-~yVr6<}36#~Ag1RVogyC<6#ej)$!QczKvA89eB>>(0KkJOL)#QjbY*kWt=Q1s+{ zqOu6}jz$RVTqTl_fXS-@u+aWfdutf+1hK^Ytz4@(f0N$B@HF?Hxeo_-s4U2%G2{Ap zKooD=H%i^+P@F#t>m70^u6t2eXyu6&!wMrkExeh)101qpNw+WQC~r)R7c)} z@%IM|(F1ZgmRu{muMXa=HAlE?ApB@uEfu7@W5^pG@SN3=m5v(fHOhvse_OvQTWq9a ziJd6%XW%(&&C(NV$B+3+fK)Nd{d$m_^~DBYzmn_iIeB6O#3CYY?*YU%AI~<@;Ht;Z zv1AKyyN?gbiTxk9kNi?jZnAV0>c z&*5YpBLazP)#`&)A`X%0mG?(atnnlzy}^Fm7QD1b61w#Nc*fU=!F zJh8{Dl}EY)H+}-TDCpf+e&kJx zxwWMO;7|anFSnN5Rd_)vxxEc3bIuK>Bxq2O!blqN`mraLJg43D75SPzA7c!IQx*Nm zL9Forbspe_?FX+9tKQm1K#Mg#NH2)v7G;H-T65R7w{tEfyl8$qE5i4~GC>DQw+W0S zZ#4^UEP$3ZBjCKzVjW0&D9N05W^(X=EZkVjF^RHqV+6G!<_jF(#|k#blX|Cmw8)Ru z6V(PA=@n+DSCC~Z<|8s%}oU6d9uf0BP1a&@wo zvX1zb*ff9pvi+mO=^Fk!00cM13+4fJQ+c;`KhV34dXvv$M6-IIqz`^E_>(4jx z>o4!+&D*!~AO6GN%e(g<(c;_J!{NAr*DLqH_9HUeb)PwVwgFe0teIyKR94KnG@s`Wv1M#Dn<#>>H(qS5ZXU8!C8`8*s+&t zh+es_Y{nujy`gT9xqLbw*6Ru7ygmNg-xG!UDVrSrP5jw=sOo)-zG{C{4=?|d%jt! z2l@2rO78AY+s)CsJ;%V3*J2lgEaJ}4mXuBOjM5aBqzvfq1hGLy5P+2MI%dsE)|eaE z{M{3L501!V>m@x>PeYJBfUFz6TPTZ!>%B}w^^~p#*R+D+3PveNQpR)) zS4<4MM`^6x_146$3h?xE>ZLXc2NVv;?D~k})7>i&u!b)|tJUH~-2jI`c)!4!ZRwsf zPr$QUu_#V&eZJx4^Ol6pn^F(!5g?YK5PFp@jD7GHdMB%#(5;Yj1t;-4UpIzNQvr7* z5e7iuqG}9dl{$sRLUjeWgaQTAC)UU)3TZ#DNClsHw^9^@#JaXc&s_B;68T8ewxriD zbM`OXa_~eE$d($)Yfb;~Ge&tO7>j%PYh_$b$)OT~tuuH(I+MI$9H9Sk7{r78$>!To zgU6U&p?$IYu)MK(x{8=H%N?vbMP=ec4glEU-ogNwG<3x21kv)6WxC?8L+Jf#<$Acm z43XIM31U0INGIC0U4IFW2cO|qs+ryRyzAYT-nl>{1?+b6c6lwl!tp17SQ;<6=B;C1 zqYm@W#Bs~{MG|Oc=(oi>koa2&&a^RA<8qYm^0;-iOc5^q>^gg8x#@Z9Qh;oB0t#a< zWjy3$&4fQcj?ecXh`nT^12w$Q7dNW3)$C6{zTWn?pXBfV<2Q2s>5Gpf=^(Za5ZOWZ zqzmMc`Y-DpbMr1vj{uen#(J>Bdbw$=oxQOYIxFXuEj+249)VdQLy@$a9No@RUEoAh zlq<+<0Zj2M0G4x|Uf3qAhq{lL&U@sLaywEkNs{6RB}g890fa^wP*0@-zSA-Eyn$A9 zx>83=EwgB9LiZ%^g)ZdkdjO|8xmujdbmWSCYw1+C-d-IM<$?KHSs7p#=?QQTP{_y{ zBcN>UJwPWQ@Mo=CI)^N>U1DYyI*a?T&{e()J_R_yw{`xOB5G(D! zjzv%(+-F*gR)f`Q&V4>Xtny;p;Yx09UdZbo7rD88CinNp4f(^OK*%0fmn~MgsX%$x ziytU=@?aT&zA32--vBJh)62Mo>GCU#WJNw68XcEM_lB0sL?+z!QKgEZw-n0l$?mV$Nt_otUG+JM# z!piYE0E&U~9jwc#0LQ{LhvRf^@@t(7yUKt?IFE(ohTfV1)Lz1?4Iq{l_Yu~po{#Mb zJHTw^Urj(A_c(J53+K>RGd5en7{L=OEusY$8so9Kh6r|o3%2(jK(@p9EGb<91SGtz zROCT=hb%-wOUq58wvmiYgx{e7Are4s3SR;Qv;uyz3I)rk&F3?a1rKXm4)CCNFR``w zdxPb)76Ky)#XfLdc0VESY1@p;nv8a{7>BVxAWl~sFN_qfkRASNp;nj!C z#~imX*yv$x)nPKciUm#(W5jiKVZ{_Pv<1L}?P5iKe*3e0^Ub&N)9bhL!}mWKl_1Ng zxiJ?BX^Yo6KBK&n8d0Y2T7rg@qx>23-yPwWTaNWJ2+|)7Q1PTm*{6k$B7cxdU~^VK z$9ce5F@#ReY3virf;y_Ht3!vy<+Q*GojSB0qc~0G8%d+15v=6&kp?$PYq1_G<;9HB5}D!JYgoGk#8xhCW$87)Teg>G7Qa4NHNp}EM`DMR zIT~|q!V>nmf!gc32MhEq_HT1cbr?w(t*mta$my^Gs9x3rU{@s6Up#@GkqF+m@0SZA zJ=mEbXLi+{Kr`pr< z-v7Jh4}T&5avPC5hHcs>M zL4g@xXz}klFZPm^3jky_v~uei&aa29Q6Xo zsFb74p`<0tQb_&=H+F`UcLo#vXJN=lcnb+b=;1T33~KR)XjTls7f8U0FliWN$s^@f zNJZLX%tvg1RiTC#JvJzI(6Ux5k%EGs?u=SGB`U6FFF4XN3;=`{k#>t!7VHuUdBH$kH}2tcg=x%Rp5 z;J%g$^4TFPTwJ-Bc|Q|nsQLX5$d!a4nqFZEr-DqfzJCw~^3?v499-TVKr6<*R#VVw z4&gOMWeH}5ODlRjfNCq}nLV)~z~Y9AjCqYr*BtZCL(ypGy!-Xg$f1nH5h%muv|N@h za}2>k?nNAcTwEXi_|s4F-M8P#ufM*Rw{L%uhX;3}7`QpO;jZ)QA?9vC$C8HE6;u^) zA-fdAFaa!qbp`lqF*bdS3aFUoQ_|4j@3_f@W8PbU*-Fp$3LU;8echwHm86VPju~;; zcY54!mIFL%|GNV~<~uCxkSNeIMAQM~Q+(J;mwRQ$Iw)OC0%R^$e#)am7J5H87_%k; z+n%Izw1D=s9A-`~p=YJtSyEq54!U4G*5*7{lNT6lMio$eVw3@|F?=!peFwkVoo?p9 zDHFhOB~8)1!^3sSS*s^qJep0Qu1}39-+j5myg^rjgL@`;1#q?J`2ls1^?DvG z$}qU+blqMckfPe zbE9;EXXc6Z3{083{e-1Q8bOQh4F;MzfT!LNEC+7LE0Q;iO3+Yead8>}sRUuswjR6< zw>nUmc)duN@O&Z*+tyRVcW4a@x{UEB?}38=KBX%7fpCDmy%eRrmJIg6EanlXuU=eQ zvN3}I*90j1oK-AB8IkA!HZoKZjS7yalgrJOfG_yNzA@2g(0YjOT`gWIfZEoxZL4`= zACm5&=X$Oswh-obZqdkkdH@%+&>IzKrxF|P)ruk;NbDl0!8|HpJ)(oj(ZIM;4BTKj zF#JE9EA2oH8Sy-(@oo<#Gz;%ofD1u*oV?j3KGzX$@wrDx%`9{RSPA)yaO|j)*<=K{ zojbVu?|S(h$HIv9EIuy;#67~DrW~R<2fmBlIX&G}05rT{r~zbC&MgT6$p>TlUg`p@ zq7B5l_e0lQKX{T{2S3)vVxh+{fgA6O*F_LNV7S=QxUAftzdLLoGh~7(m&Q|n<0+aLEy?d86<}W;6i;qLY#!Kk# zGR@7PD&_=zR%jokp1fOA4_R_=UxMGkoqvc#MQgY*=dXfRH&pXl(<{rU5N<0n+$K;MHhE+5dFnwdL3F?>mZ-6pUchX z=bIJ!a#IA}^+G?iuA_ih_mbEW1%NR1X0B`s%O!+h40f?{3q^sGZFVgax{k*Kl6=4k zSzCa{@a&8cR!+L`alhRcv4smjC-4kfi4C5-73+mfp9Vb?7dTdLMDQQL=~ppk)Jkyw+OIrxDchI+3uZAhv>GZV+CQ z1nRQsV%roxI5~p9Uun|SHV@C9XXmcfyIz>j9@xn}0i_~^EpSWr%8;zt6!ao&%4=K* zVtcgVJ6g7>XQJvj9GBuHH5!0tfKFkpwxFw&gm)y*)2B{UOaP#VRk>aCh(vg$<+`9p zDrhmqvm#R77lA>DZDe3Gpj}faHwIS8sVv-F32zj0__?7=$W8jZXrQfe5{4fuh-G-F zf=$~%nUE;pZ!NiFoAO)TBgEt_r71*- z2-?=-+&=PrJGj?ABkp15+MLuJ=CTPkGJ3J%3GehkO-iiEObi-y+j+BvA(7~14|1jh zZ{{Y^SkHIYn83V3gL5a)!I)zjkMXlaYRcOGEe~v7SV?GV#Z1Aqaz3cqLvFG2(rf5& zi#+858p}1n2zgwTAQb5EL0V4Z zAg2d*^=P4P_zJgfHxT>z`dWV6K_E-7x<=&H%In_r8vF_v$Q*NUjATQ^r z=s>zphxgIi#4 ze_@m&s=V-DYXIgsw+dU$b1Mu}AFLq(yKFTWU15yk_rXLh=$vepArj@Z7DhTS_6@Dg zt!U>w_2~ys%8#ZUcJLmZ|~&OCWF?w6kSWrS=#7*<$~FaFYSW5oNOxSnJ%fwiH8>SpYXliW0~PVi;?egm;Dx4&Ka zCa&$MnG1kp(A+WBHl25&OIq9y5q~)DeBA!epHN4se?d5j!Uhq6>jlpUes7um&hKGYMHBb+heb8AbngHwUr7b-C zyr8ArDKgjfVO0#yyOKE~1;jqVXXU+MW3t5ztBRIYD%3gb8%ug5zoqUa!7~~Oq`frF z2yk7W)Axkd;X1v?BpW6yk&k8f(`OLt9m|pzTFLeGnDmyD!_39a@gebhq92^uqSh6( zDw3wE{~YQmbU3O~9_umv zGqbmzk^|{U9Z1o1E6Y~xz|9XLn|S2C*fIEf-WZP04GMYb;GCl9Yn7rdr;dgwtVoEl(>D55A)%bZzr{dvj||4Zi*M+YQA2Q9iuC zmb?42gRCJDVrL}Hj1JSQURPUC=-wO~vIf6D+OUsu&)iT1x`4?`GI9V1oB+a~0dGTM z3Oo<6N{34REe)>0^TDu*pi=Mgd*mH}O11B@l{b)bCDNqkE717JX&rg55DmVjr;%Q3 z`U}kY(Nq2nARBefn-qA(niF`h%M_J7(7AIc6>Z9XNQ~hnvDTmy2=(Ad&T8GS*$^hE zA=uNio7!CIC8ittP>6jJsTmunnkSlLdod-fnzu*+Az>?(er7 z`24$V(4TL=zt}cA7o^~Ga^Ns%QLKK-Rup82UaV*F85RV&u?)Sq1$K-=!0h0ZT%^Ye zF~7e9uskoq2zxL&eU=j)K%!a=Ew5D)P*f1>C8;jKyQ1Q*$IH!W1c2T1VS{H>xSYH$ zmXOK?eRy>23Sbi~`%eY24V#jlgDSt4E@6E;GceGWRF8eK`8Y*!=DaVFG#f_=NBQ6> zh-LY;pjTB6v|6TW>MyQX zvK4EKvORV@VZ+!85_peLXi{ZpE5kETTOj``2tW5&g1e4s5)))hcbs=?a7Rx)X z6UW+9B&OsPfu|0-&*o~>2uJ~mgW#QKVRWP>ZXy!Y#0bORQN?g+pxaMQt=mMM$Bq6SAIvV10er|R(ihK?x1qMhN*z~<@km{q)pFhjH zU*ByY_IvsEoA2er^|iGh+QsneEqo|Gd8bv^}qdQAkZ@W+}@tLZYK4Et9v-Xja4boL2A< zPQksLMRk(lhN#P0SuDy%{65P}sy zxAS*K(k2X4c98ZhM>85#&2MKS}mfP{wd ztzi3UIfmZs01J$T$x|XG;8y2*@0I7=c}@8wxRYwYane`jSr$4>iZ^7C`{w5kUTEi9 z74Z-@J3ILXXo)DzH?!El%ly!93?FPLO5JQ0ZG&hfYn%tWoDPxL6ktwNsATv|1~KlTp;&150l zWNCO@)}Y7`EcLo>2l&k_>onjflolR|!4ns-9}NboBsGb=fz%T-D3AaTv!1q}eNHB+ zYdyK=VNZu6S2t96l{8VsTbd8Kxw(;_fBsp1_~A$S;q_~|y}7rL z?Eu(d*C)<_)gdFCs4(|z_xPM-G>c*heHqx3=!D={Nwp0iEQtQ7OM0@&(_)&D3J zRc%>>8lG3dcbRWNr@{gXPJz7MK%p@~%w^q+=r=E_zrZ`)J0czeic!v~O|L=3d@+uUp)Y zQdKNm&$}GHuBLKhOhVSNQXqQ1H$&?7_BO8cbh?t4FJBpOx33J(JzCd{T3g32;F$}O zyeygP>@{&OCGr(z_rd`3F&iylYvL5@u^}j*Dgd$Ct?QNCtY2@xt0(s5hDVMPEvN&$ z!%tsc?Uj(@uS_TS0wLB;x~f>=wsX}+Kpbzjb3$>-~r^6|q4 zVjngT+fIJ77Z;Ao-HXsKfdtMUUAgY=9l)L+q;?MSf%L5L2qy!?(xX}}o(>*~k-Bkj z^ti5T)(cE8%USjyb?@Dnycjz`Qp;U9Cumky^_UdisnZyqULg5Wu4@iS#P^FzA#XQo zv@Se$)--+Uy3;uDhjlUA-J|hM;q_#VSp&Ri0c^=&+iNQ-ZIdp>v!fk$LGCo`63 zh6#Bs4tVhwc=u)G79!)Z9P<+9$N)Sz+kb~=S6))g`+*#^@MKpVAdlAMqX+L3T$sn1;W&_OX3S^s@oO|dCkv~3pm77K z_$k(L=Zye-P`-;H$k#twOpkv(dhRdul2`04<=1-kd~|6c$6*8t`Ja?E82`wD zFrdDx@-iSdH0HH)lNGXGy)@GG1|Z(Nc_ZI{|Gm6^^NaCeqs{E%JWJdESh|kf`2gr1 zrP3%z!eU2v$&N{RqyOO-)UcX<~Ey0|FgI#F5jwy_j$Ps91{EtCi zBtUJPJw28I;)3EbUqF}JgTni+(JJbOEscH{;ayUfnLVC z={0Ah$|gCYnH)ZY)Le_z1Dz2b{~VD_EAPqlIk#^a;mCc-WVFK*D;_z}y;VW1wmoJ7 zwcf{9ufDRqn2|FARns+)CyNRXFH?F(<0u2Mm!uQ$`);IU^_@t%DyuJKa8rOINt-rokX zm=yj4h;{SoY8$MtPkp^Nx{Pq>(f}{-zw5yhN*1ESr zp8#y{K_jqicnv9^P7p>vmvLKGUv%oFyE0xV%0{%bSj@$Kf&X&UE?l$X=XtA8Pav>t z{;X<1zG`ZnRmvfYl9<730&!;Kr-Gfn3rn+=Dd%+%#5z~#Xm#c4$@d^UU-%DlT&W27 zh?acF5Y;az-9zWE;Bsqs?rQhWv)hWSb}S8=R&e1NtSk9Mx9`5*zI|7-uuu#;mY}fv1{49mo+k~ zb3CvfIhOReA@sk(+iJk|;z?Qk0$RYu&u<=CNiLzQ)>P6hejnixo7DLjK_lQ?^KYa~ zJ>0Rg-~&Q=NmB@LQL_GpdW7>*^>1_TSZCrP^+bHeJ;(;$+68kbQt1kzx8y7VUMCsz zqqtEPK2i*ZN8DTldhqc(qF7b8G|CI+se)CP%@)QHP?cZ@i3!f_Zj~ZUK4uE!h1vKp z{8lFK4VNu|TjnkZ%F}72!}aw_{NXl}_}*kmD%h{iN=w`3@8gG$^5c&`Z6Njs1F`pa zXI#H}0KJrx0);cE3g$^eoo<^IBqqzsGIhfvl1n)fI!UiCN z=WdhUp?(bmjOE*yENvp%5RF|TA7Y8*Ew9Eyj;YGy;h62$so#kSxVNTW1%Tz)OD1yr zxlNG$-L*lbhDlbQHd`!e5)ZC&MvNI@GAB*<>(WcGI{Dk)s?{;R+};E}dGYMIeEs#; z_C0sTovkW?8eiRmc=I&(Z{4IwkCZSv@E+nD{Y%OCzq{_*zL ze<4pi0x&o7{!-9Fz{EFDvw6W+*zALC@O~%PA2vk(`PnvrueQO`T4rlh69}uh@%yRd zSUT(e;lXO3-NO@09#bvee>`R)7p*H7f&`)j*_!lYo;u5v_miy%2O7unoNaCCo!EO` zt5x@u7j{MXor@rWkXvPRzD=4KMXneSktZ z#`Ioop*!GR)zb%P!n|3gCQF(v-X~jk0q=wc?=2KOizmDH))ULpOAV?;!*5NJ_5$Sz z_bEI904kO-+T__b|5wkvGnj^|jwq|5G=h`0r9GGz*f_D+m;>|&7i-IgX`;L z{<0WiYT6TAD{Ae!D5q5y>&HW(GmU`Xv5lM|+iFna7?Vf8oF;%*&(G3pGPqhp-bMjS zE|)7@SOc3~l*F>%z1AY{Kh(6o74={bku~3=x=81l6WkN@c`s`FxoUYf_24TNWqWS{ z@aU`0w{4aK)dz6B3amSUzG1O7IK_Tk8V6-fqYB6fFYO3fO?e%`8RVMOUZ!4ewoEE? zL|@bRFitGKNgbpH7pX-|R-24gda8QzDI=^d;0@z*R>DuuKz?GT{D%}sO@f(cE9Iuh z-Uxfk;iAh>@}+qtD@2>gj1j58S|tFEDg}J6QIL7E24LB;GQil~2EN}TS!|T|nE9Rw zY`iZTZ3BtGq^U3=m$Jjm^KkoOyxs8t*sOJ7herd|3-iym_dK{^?@-s_0&!`kb* zr#{h6N!_#bUW{)Z9D7^Yw>bwE$>^-qA_07H$}oNcWK+LGFSs7ci{a9YM5mUKG?oO6 z_W)kK6ygDu3P59nu|i#cK)FPYCnI&0(ox5b>Uq7;Oa2%ega>|y!}9xxq7F$Xfk;G` z0}Byb06&B$6?B4|o6r9*W$zLz%a&Y+#on1!=XbyTUbFvA%CZCw1bE^B2O4mwfrlJu zKo}lqV0X6+8IWMumc-YBII$tX224SgL=kO6Ff}!7L-I%lJg@+I;tAUW85GIxe&2iV z`JYo&ncEo=Yenp=b6=A_5B1Ku^~uWIxp(Xp5i3>%P4WE9L99h$5rA;XtO=lX4)E%> z2Q-7$g4)93Sx-_SI5>>aZPQ#tn3*hYZ9$JBm0)hJ@%|uZa)p+&k97gD#)*Y9TylLG zC$@Pejx+W>gSMWW-D|}s+&>*Y`ri2M<(Xf9I_wggpMB=ZhU`amLj>O!p5t6p2kxa@ zUTWvAXGSehr6xt{d+3gaZ&FHD4CBWX2{bkRsK(>b0IY2~)jhWj6bS6GIk6}lIWD|V z2YmbvpP-iB2Hk*w^8oNV9lD;wLhc|)12T^y6^WCx1WHHk#SrYh1JOzW3`E*?NwU1+sxwpyTZHP zBjxk#Hl*WfJmzr|b3p;D-eX&s2Y%MmN2kkzQ)59MZ$tqiy&t=2EaLE@0nw^e^-4rW z{r;brbcrrh(ihCh(R7TG{4D}qt+=9(2AfTUs(>jLz@8S4Ye%s6`<(nej(=lRDmHyi zWO7)$$lPiXrq|!&Ui)Wrz8L{zIEtkcX>T4m;^xvIr$Yd>`Xa8i0Hxvo`}_lMDx(~z z#6|SQIj9@%Kv@#4$I_cIut~%a8%P~$Tzj~u;DtJfoegLf1)J*FYP523Sf~BY0jNH= zbL~|%azi6%@j0t!9!&-8(2#GNTvp5fPQYx2_uvs<2|5#Wh;a;x!ADV4fFG{OmQ&R3 zQx=ce^Kf~UN~nE&SyW7q?9<@}SNB0wenSi$B`SZL>ER1yC zLzST`Y;;G1t%dN6joP?|EDRtfp|^zI|&2 zUCM`jcX{nmT3$nx&oXintn=9o8>_7lEUw zbJrF31@c>G=lUPED97-3HwGZKa+yPRuogsn{R7bjG4AL39+HL-HYV0v@k|Y;Np)hZ z+a$C>ZE1H2mOet)$wu-l^#Aeq(@1 z!g-B3kM)_#1lZYEj-s!*Hnjk)o(UjL5cDhT+YGvRz(?)b-l#A)*VhZCx-kbzzwhky zRGvM5o|r}ivS3p<&WDx}S$4-{miT}uEQ`yz=WaE#z`#|E!Hl)oJfk7*X(6e#{5ja) z_f6>pFXeiEF8lG79LlKy+)|uAXmhd+24cOlAuQ7oNPY5?3qJoj&%Zr<^n>}s<)44$ z2@WpZshzS!@v|2S~*|P3tJkdt|4}37k72Z!meLV4g>vYTK$v%>+ko?IE#6o_INR`JS9}Hg+;8~I{XCRi~0koJF$C6lM{eft>*uIh9 zGUNFYRTOv)5fJO-QuBA+u%j6x^Z;4(+F31*6uw164X7^Otwt)BIinzUfBztd`+M0f zzxT`USu-$)3DQ-V`?*KQa0_Sx1eT4Q+L`nLe-yN$^&*}Gr-kj7RV=6~|#Bg3F0 zgB&BHf-wg?oR&PZ!x0CTAcP>NcAw)qjtw;DB3nfF(OVv?&yIPon04a%KxB*eI`6HS^WY1pA$ir$T&^jS4AXe?#n>Sy`AN}!9#>v|;D&(Ma0wMA`v!^ve(a#RnWMiZRoPvR7me@Le;ahZgbd^?W7%6shYz!qA zCiy?Eo9k(l0I^IPVVHWpZNa!xV<&;LOEWp0!^dC2zLC0@G2Icw&U4n~RIzb<_UxH8 zB&Y!94PVZ?pHa^i>5Iygnz9>8#mLyHm*UZ=qks#d62)$Vo7W$t2qQfdqsbJ55qhg?R!WanovgCsnp#*{PUe*3fARKJye*g41JIyK? zh~3Nm{hfUA`4{pBfAq)l*{5H~?cJk|4Wm>>yVei0a8wE?P}m#i)HXLxX=D1|Z+z%* zD$u2!r|oF*w$<}JDDB(u=Dq$ofZ%-qmUDpjpSMIDXuyc&bGQ$&&q_MhJZRWG+|;C9 z0sO-zIdD(-$D+9SjHO|6Ka9?CEbv->Pk{b?Z3#1f0?t==b` zA!KuA@LZ`tFlo!%Mk>MVQP>E)3GCF2H9myH*y0*!Hx5~hCWyqg8tdS`UC5P!*!j8a zh8Hr`a|5xpz`?X@)WBDT6-9QW{0Z~6JpJc#b$@U8=syxAvia9$EN)KT9H@kS*EgA|6 z&cNbpNt4j3;7i?^zSom3~+9H}9 z$HuW%$F;F``6^2r8#u16r>F+sK&(rl9rjGWG)qUT*XY8`Wj(*bSmvhZAZmz>+>0)X zrHVE4>SEP30IYG}h6))MI#SGWHVK1yy^T_MqGoH%C*!g5ow(ztMJIuKeA#eh+7xZZ*&HRRzX zdx0aZjOeOGmPWtl{OzE1<4#}k`DSAbfW&p!@URG_78zKCb#KKA3R>^)ZshLjN^Uh> zcC$Qgmwzt?`+hOlj|yfFdyn9j7X8s3^UM+T)HO}If$^-Afn-Ib^D0u8N(d-S!6Upb z98*}p5MYhub`gbvN+EQY^l8s*$I){kfn3(!ttV=w{@Irpv9@S?i;zq zJh7AgOo%-zZg?UcH)~H9# zdC&n=4;BmoRzsG{5vG)}vgSxg4dAIn3)u)RRL~4IVbdxLcIor-8tfjRu&EAlU6?H^ zaQa@Y)viq|e;n|40V?k2f=p)zrX9rUKH|YPrBy13Rr}}}7&nW0J zkb1C(cY#a&b?U|@n4KU7L&8T%wH}ek3M7b~(zY2LrkE2b3*Ft%z&0h0?$#BByqq9iLM* z{;V)*b}lXTObOZqz^%>nt{Cay=I?Lr1;u+1>qSg^OsFZw&SX5$NrQ!EK7r@x01^|m z_tNFSZ4%qPah>(@_d@SHy;wGiRr4}QBEYhXCUjroE z?4ApTb#}v72qX{Qyu>)83St+KaDQ`aEj<;y>fiTwclLN#UbCW3LQ5j8I z2Z2tT$(=0$CYkl2!oeysDH58ZjLdq`aW*hmjekE**!2qUC63Wr2dYZV2ya(>e8QpY0FT%%kgbKo zHieT|;=BD^h+=qbY~KCauw(V-kBD^x+L)(^Hsx~(REuc{qnMKbnGvMf+iluXB8UZ$ zI^e$T7NDy3Op9q0JbeDy7xHiZ@Q>u(yDQ5m8ifF0k`YC_cOz*>%o5G9U)UT((1owt zcMJWjqlNz^KSgN!+n@B-_a3g$2H@Z6AOIK^j3LLevq}AwTSxfcLtQ9&-&O$t0K^Kp zt)viicDQrmXoqSyCt}|in340(aj$ks%}vZxJ~yBU+|_j;J1IH#0b=>NE}(;}hN_9g z-u-^c#(SL12E0_(fhY?qbnhcAfXTCwOU**u8e^ssOQcXzYW7 zJ>0f>JqH6zU;5L(Mz8u=`&+{g{~I~{tsj*COn2&D=HmE6eG1*0GreVP)X0ofbs!) zqNW$>5o3+qG8&*8X^Zp@0P=Vr(=h$||)hjF>Va%!N-0l5to|k9cP_r*s(iv5scy zw4h7*&W{f}Q^#mEuThJz?A%8$7#J$g2S&9!FQx$LB{~3cUd^Vu0>Ny2 z=7Slr&DNu^n~fVODp72L1|rLxNiUs^8hqbnF-r-@h3lYd@AJyYzz~3r#>d~E;MafA zAZCn&k^19^ia8FP>$qlFZ4MrBCD6#NAI^(p!50N!Xb>{4mH$su9D$%Tq<%30%0qSx z6avywI38U8QE@-G=r$WwBuCr}u1s)Soxa^ZA4Oj6=--G?lB30VeLydZ54o}|mgD`YYeZr9(dRQShA}!i!9H(#?jIue_#uGKdAtUpjdqAY znsY@wbFwnr{=wu{a(?ri=}dy-;#*2ySLu$kiaHZe%~Y6}^Zs}D6)+ON5jmTj8)%t0 zt+|jEpdzZfv8g{idO??i*jxGh^DpF&e)1E!zPh!5On}&e0!)=evewI>Yne|;m5Kxk z1MaW?;C-d6<|S?v=4L8 zV#w2ogSR;N@x4kF9dJBhLnCEPTGJRZuqyqRBE+u2=xE^|4>9HZsd@gD8rffTIY5|eE`uR7j^$vfX#Ecz8>ZBs>*)aE(5>(-b+*x z&)B>bz;cp~5aze`eeQPq<{-Y%2S&k=2ACz&;5*Zk-G`ip>84G-?Kv%=GmhAV9}f<= z8j!O-r$aqyx(R@wkA4OvVDB3&d9`|;A-rz|MH?$Bvg1wj3ixvJa-#M>!_f0CYXAdo zcJxY*qqyW#Ru~70JZE|j_X}ekkq<^N@L9H~9T2J2=NP5c6Sqk0#?vvyqTmM4Tmy5z zGc^fQCzu?>V*9o+KSk#*E3mVcfYtOw>(Mz9P=xX5q>$7DPC7xc6t#X=9bH~)?Ai`) zt*cmim9A-&)#nC1`kFwFoUDjUGE!N z5w-9l4?*AVgDC}Abl!M|F2i39yl^K zkHz(0;RKK#%p#`Rjy8>%kXYjWlzMDS=}>g*g~0^LH8_!Li~B=>S^#je;KM%~TvisA zY*KVRqBC9LNuQx0IOe38kA8Gl9LDk+d>pQ zzal_D=XmGf=eap3$0km|1OVG-HX@?OL2SIi0IzUjytOZ?=^O5`-KRq|JD8D1_01Sf zSdeyAXI$s4I^z%b4{~#Ly#Uv11F;|f>=U`Uy|?!k0#%2|PG{2W+t^dcTC^VLrSuN7 zQ+IIx5bf&Si*B&;#&Im-U$8Zu%M#A+>V2Jmag+kG;0?zYZwy}u-4*zJMJsNJxh);n z9=zDXY+i{y$oH4Aqrwmg$ZdKv>Q{9gTxDi;mVGG}+Y z(E?0c+^bQj3V@qq7cv9W^ff8Gr-JTQZgo? zuy$s%nU9dzx-*=kpF6v_a1gt3I)QiIa_WFiZ?D&(%+?gLf3A%`MVGCq1F=5#HSVw3 z&e^2|-EpsQIhM~Tz5x|Yd9MfA$aQ-mUrev%PBaR8whY54tQ_{r1F9L*fu1b@8Wp=M z;Muyl+RfQ!Y83xQe*8QCEkJDj%gX?KJ2tXXz=)Aa?=msFN2<0R!qUTL6Cg{6V-GQL zn;KOR1)308%2ny9VE^}FK7EYwrNb273)4gfAoVlL z#;=Qj9W?Yh_O?Oq1AN9k5udc`d3vQTiiZHW*gCah(J_stuNi53?_tjYJXMeAIG52u zlsLc`lb#_c-VmIN8yQWvc!L7RkF$v*?+ttxqY-Skr$!4HSXph3hNe3n7eMmp3KZmi zi3dzr5vt>Ka<*MSkpm+ueCt}hM!|_b=t1jznH}D00zZiU*hKBvV>SMY&`|N2e8-mg?Rg%9T~hfzCPF znVH8}K4%mcom;W)z-2ErizlY!!cs-Ral~YhgLDB?G?3&K`?w}0!$E3^C>s5SCiMOM zeKCdtFjf)6#l|vs5%P#U4mGGTwF9pafvpZ?g`8_D5>x?KboP4&5ETWY1#C?ML9G9b z$>L(W4U3(5jyw7}m4=_x?;jITw;J|I2xb9b`e<$+P4PVgfSda|H*WlwG=e$SV?_6m zb))njBj?G5<(ivOi|FZ3dfi@TLg%(+u615mcRxj?xWdB%(I}Q4RF|G zjOc6D7mU!B9>M593SG!m$Ds*6w>tNlt(nniW^6gkHR#b%x9j2=(G#bySyLRWDgE2P z?U-_awjb-&+qBNL)-bmj-DG6j8Y$WOCQhp8+cIPSCQ$c?u>NhIX}uDLJKg307Xi)C zG1TGUEn0PsPn67BY~wWxyD+EDFV5xl>sJ<8CSAbh$6VY7LV>_E`ca28Pw23v*KvSS zODT|gD+uINvy?{7Qk%QHZ?HL0O-k#qylt-$)PehzQVK>3Rw|X?v*|tUlxsh z{|`O(o{P6^eC=Fz`%`)MuE^E(vZxQwm&d6i_9D^qd?=Z=WFdpz2u3^i7OB}s;gqGg zquGH|5EvcZ@Ar|~ujEqBUJqOP5DPs>5vxEAkt=d*Z;|NZAkJ?cN<&xvNZ}J2nJH~8s6mn9YaU(85LYA4lZb+}e2(v5 zWjWv^_jWNpC;2|k*)o#%hvU8PUX;!;)u>UdgDwh7u-JpR0EYPbK7e2+AimM}kv2Lq zN9WL|ni(xh@LE>a%hw9tTmrZf$ngF79w}e)H;;k8t8hMbjzm|fT`h+{+sDWdnSbrI2m z_tv-%mCjU=h{A#b3P=-_tq1|s>Y-WJE!pJ6>S?g>I=Xy^!YrrDYJ|m&o zDf9pwL24lI0<@gjm_zZ!umHbC{X1VPYq%_;{D20C|Hh zX(!H`MQX@;!|(k%6{N5Wh?Bj2JS(C|O101~Tk2VF7ms*j$`XKp;7J!aI4lA%P^d<7 z%!qiwH93eKqVSJEegyaF?EKUt!qXhaZN%JRDmqh{vKluUq6&F7b{~sQLMX8 z4C(9|5i1?4lpA^iV(~!_g0$6Z{<~y#JZEdXFL`y18#w?e1r&}sJh*utg+l|$00^9^Q# z{aL55T~RAlNj$}dw6SF6I>#&kW(Q_N-?im2;yVKMeX zr7Jr?3PD#SC9*z;IG5R`=g$vE+9v-Xmyt?}q=PCjqj`AM3J7@?6s00BBwgfd0h{S5qmfAcm z>bkqXmv@(!^5*Ru`Qpvn1<1aYN97}SLzp1GF81uy_KW~1GYfueWR^;IC;EADv{mO= zQ-Oq!qx*RbV*6sR(tU8%P({2pgb~Mqpqlgc=v>@)EdfrsheweWg7TC-WQsolY!+y- z{(*M2b`yjSKW#?nvR4m)kY3Z*hVV0Va6P-hZk|q!(}=Ij)QLr3P3p1Z{+F<2CF?SZ zKvIv~YoyeL+|U-zlD7~~ke2da0MD|{h7fQ&SK6~dYl}{a%YpI%8Eq&M%(!Ey7nLIph<@#nL5BKM?+dZ>LY{O;}6CiGD4!lKG zk#^V3%>>Ger_vR;o@`t%meGqoYN9_wP5GtPNO9og$;j4(s?l{Z1}?gR)5FPM@-zE= zDie4#wr9HmgtRm+1cK!3GOE+703?5CfRaLT%-nn27}=VIC>GH|gu2by;?~LU7#GkD zNNSJQ5iz4@NpN9BbBH9{=DS?${lh)HK7lfXs~s4K1)W1LlVow#RWv$*Mb+5K4#sBi z)X?eKshp~cfPz_#=+d+9uEjPb7?d=p&F-rJcQjTPlYFwWi{gN8NFfNw19=+(oqJYDY(O3gpP`DUMq@1XmE9AWef4KV|Nxrs? z^_*!|1F=vCvER^c7S4gf`8j`Gzg&8*^r#Evd+8|#7ZiaSCCyngwF0|q+5YVnD$Ih8>G^_7S@3!J3Ul_j>726&*ZVN4=(PoY(uH9acz7 zu6b?wymV?UGV?}a-pyIe#UImoMY{EX%#(sxmDjqwypk`z_(I;jTfpqQYuWFnE=ng! z*Qm~k)%nT&!R#O$c(dt#ZQpO87JFHIXF3PIC)EygBJ#SdYo}nO4YsSFvl5zKSTig=)_xg?o&7Gd7pJh0*450ZwM_lYNG&+qc-|yk6=O>KU)t~g=wCXB{eJrk z%Mt%lPVSN`?8`u`WVoG`)3m2PB&YJQ-!4YrYym_m%9r#;4?TdL z(b=b=Z#vqyYKs08(p97tMA{Fa*f}_LWeoYl_p*Bm3znjQr-+anM)?h2FfBl@%3JJE zSxt$q!o_{G2@r5T?jEBMj7E*;PrcR(BS%15yXcI2tZQ`y1^8?i$0#Jn#g@S|NAx

iJz5vj*FH4L(kL1tZsqGp4y|t0u13WL_ zb-#e-vy0{Y`q_K0(oj*KRP9ko78cv8)$wEtO#`VTT6U_FxB*iSJ#1my)-Hjl*U9(N zBY}>RVRLm7TSe>(WwHt)Qf_;>UOS4RI8$fyK3GcHYk=DTumyTpB&@mT>)=A|&~{_8 zrp?B+;HGhGtZ=Z$769;r&Ru&rgjOBAlglu5X6^KYp#xchVvf$#8yV1|yPIoL89mvj z_ImcGQcLbik`ZNcnuJY86O>rXBw%JEVVb1AnMRGL_3`zJ5Y)pK z-YXxL07&syI7rCzJ*E4ZAl96X0iC!EL$YE6o%ghtUfX*}yMYgIXJ`~AwDzARj7Gsm zqyx(&(J3M*&g4@nA@MmpR};#L@*XJ}XrXv#)XsIfmhfHN`Kov(Hs(f{Ec?E(-1@h@ z-2c8NU;6xg^2JB)lTZDXBJaQNR5o&ZIR~T*27+^s?2oyBUYNx$n;>|-9FJbg!p`y^ z4F9G)ap@cG(&Ie^V)gjLZ_9VS^>z7;W&D5k&0d!ex_?rs;k1d>Etpt5`o~7YW856Q3;0KK9DN&8d%!J;jhG&<1Q15J7uK<1H-7_ z>2uNnFEz3Jd5-22ERMhT%}^aRP1aICy7wB=I_EF6`T#oDSt6G*w5n&FTmnGQoDr91 z4OS*5fbQJMbEa6?<9#$GIX9`o&UCl>rRP3d0J8!eZ|A%&g+$C zF0`nQz5zJJqe2a}J9VTdvffo%gYCt#{^6I-UIJvVsDq7~Y0C)u?ii3H)$81p3~RCA11& zQwJ|sW=`+OH4dW0e)hTGBuGXs$J#kk#d9GD`fbRdc`pq~m6)FbcAdX-Ksnjkup9^M z`;K5+%<;hMB2hBUKi~JB#g$y1*xuch29|o*E@STOutyD=)tc7%w5({U8((~eE$SrK zu1uE9^>6&z`{nbW-pEHka3;s&t>voD%udcG$Jws|wImDcK)GO^(z)`zRm!?CN6*Y( za{TX`FUs#;`j@--c+Y`YJs$g}e0O z@r_RRa_iPkPS2Ee(TST{9D%nMEduEk^MHUV=gWC-tgVEE#FU{Q|s>o;apodL9eWo}UR5bO>&}+!VT` z;b*<-sddH7ZIeKFu~EQJ@&6nxZ}k`v!eXfu4qvYswR=VKK{N)4+LisbVeecgf>6eQ zv70fcX0XY6ZORmB?av7r$WxR`^H^(%IFoZN6(PCF45BB4=`~%waurPWgfLaZHM2!+ zjXJTWjU6+CSO49j)uTU;E&&Y8_MwipZ)YoDJfi#AfDSn|^|jeba)=5M3}uH?K!6N~ zRJaPLi;)Bd!GiVW+X`zrw{E_^o`IdFUP=J@0Kl!aphmR;V|mG{D$qnF0{wXhFst7) zTEONU0l2}iPSCoNMc#K`&emRdQv`gjN*AoO)}Fdu2eG@^li5P!SG;4_Y=C0q;NGVGL&4 zi3i;Iyk<*-oN1|J_pzD5npQtgiKKFwr+-UhO<8d=&EMQ$3DjO@8V)Hpako92x8g3 zF?61hXwZ_W?6jBp-U+O(Z%ULD=q9ckwjLX0Ai;Dn@u$z&_i*ek#+^w}3IH^OtAF>- zLk{Hip1s4%2p~7-^>Y3cXMo(x7t>%lxE`W$f@`6eM7D|XqaJkJxEz9AjL3e?00l^4 z0;_?7QxWxfd#SBS!hW+ z4z4LKcxIK%Y9H!@&Q6%H7)GC(LChX#jmergC8L)FW}PwyL97u4Lv~;#c19+g zIhgHV*8^E5jZmY;IhzSyJ)@K$b_$sX5*1O3X+nbpE0x7LKlf~{hkPAGIT#gga;?)= zW8>E&_3nO+GbA7~y9~sN@&|BE5UapdKV!F_LL#x-?}L!@-tAe1pn(PR<^#Do52j__ zn8rL4@I0|g2h%gz{bKjf09#c6fXE8R!bAHt*wxfNi_cHt6b7>j_NzpEyk*lCf&O&` z>3CxVykPulU1eoQDI|n&ozWYj^BlA6odMrP8`scFcc6#CC7-*)S`x!*429KgGBMCM zfh0-@(~Y?eXDOY}tXoT&SfsY)djY^!xe*ded_{Z&BaQDH=Iazv9K7ce5=U^qs@oh1 zBx|l^e*`7Y8|z4@VfK6Px~dwWGjJakHl!)MxRkB(z*>!=Q+6r_=*E1%ex+Y@)7M@X=S+VDzW$2KHj#8pPHsXO`L z{UTp{^d9-rqwkl8K6NDbT{~IU{cY8>NW>&cOK0j0j@9271tchcYZ&MsZN4CX{8nCg zw;k_E5Ua;yzb*gAVxPaZe0%hhUi(ggY(e*e8Da0bnT(xMmgD4PmXkYW0Y(d$Fm0?S zf~oMI^$)3gBv|cBDD69Kn}+$f7P4FWQ-xt5Y6_#i=;+}+mRABO7HlXyW=GO)v;nZx zh;zveqjy$nGcuhqR$QRmFmW}jP`Lx;($&gqD<;-9`6(F=e-0hk7+fbY}a#?MX zoL5<81OEzIr*`vwM(Lc;t2cBUW2J~_Zs1%BpJ9uVJ6U~%l{@Be=B5O&(9BxtHtJ?~ zV02EdZ_V1UE~*bE>Fd9=nvlHKQ)+Kth*FRRA*Q#>AM;u^mv7 zs3hR*IXi3Z9_5b?!O0L+1@ifj#jBx@H8zX>7&E~;yTo}SU~d~hlvxVMZw#o9TR5IA zA~IyXsY&It2{2tH>}um@2%}?JwyE*Ih%DtOq%Z07x&?@JU2aBAdNvaPHZX~6BdKTQ zI%JYrnEsCaEirpLGNomhoOMO{Fk|;9)Wfxew}wgnt;ha>oV=A6-)+Zx7sUR3Ir;ss znKShpKk2pa6vWa~^NjV~0#4>-qaO^|o!*}1_N^kP%VXc|C3=uAFv|dtGXp-mW+Yw& zIAQ2_M#5C}gcw}x$|x7Gtu?}}C^lc&j<&0p$ZPFFlDY^Y?7Q`ZjRKcgSy%;m`<*A1 z%&dJYGudFW4lGAGKRcHj*RR{y8zTxsW(@)c=$laBFc>SmChi~eVFdf= zoA4+EBLuMwc@w~rnYC+==hd-)U6K7rCgQHwF{vSSzy`+9u+BXz1qQ~*l8rhL>k0H_6CRE-=KJ zAq!JGP#o))P8Bzll9GW#u*%;Np!8rh8QS&lvE=OO3hvi7r*JeJ0+?I_TEHHxOdY$b zFQ*eSb#s44p*G9WVR{b$i$U6%5~S3K&nRfGb0Lzx7H954+s9sby;)5hvlg|QWeus! z9ufzWm}u3439bQlTjb03o&m+t47`-S^XVm5+Xy|uBTydKsVWjAKa_*|Mlj|vAQz$f+%{r$*9 z<0|2>u-ZxCNXME4lV6gD?;Spq)>vYrgXsoGfa4n_{!p!=kliJ-4JU`=&?nN;2J{CV z=|~1H0LsXru#AQZncL|4UY`>MWP-0hOABm!x%b{CKl4)}k9>A3zy7NqmY;p_LarX& zkWD%9*A!16oUF^}>|@zFyT}VXbUuRje(T3_D}PYGB>(Jfz3DDG-m@T9kH`KK`4?>` zzqNe(zBhUe!NS`<0BQ`JSRFPEb6=QOs*`J;wz50h$er8E26C<&!vd(xb1ES=*CVS7 zOi?3HuZdvKijj-2e(T;fBRod%iew0a*Uctbry3rmTOYEVY#qto?0PWdt{_(MI{t2t zvMtV(W&wSAmn@s33u^-gwIIxwoA4C6SC)cl$Rr_a!+NhEIs!1ev7ZC1vUF1~Y0%7W zOmC7| zS^S>vaJ%m-k-)#A31eGZ0i4sdy$}#jUAucS5F=c#*R#WE$D*pwWOiWbWAHSl^v9-*hjq`=dNGHk zv%eS*JCb0?+=jG8%>Snkmrzf-AxUrMl zcQn+fA*xMhU|SUAHg;y7g1ImdLY*Tn(!$@f_y6WKXCe8%4}E7)3XPh?CLQfBi$vl3 zc%PNc8G{Ysxn?9c2w2A681}Vitzh^D-BL=8WE28dGO;tVKj|4eXl-HeBO#4oU^+tnkQ7WAnvUd>&}GJi&JZK6cPKIT>5^|C!X)N{+i#Q{Y_JR?V|Z~*V2++XtZYN+ zSTh@=BB9rICj}Kpde*kqU^y!bw43el&^bL`0xNFL*5ts<3gjsMF~JHl`eWcyyp?!w zSER8nD;#d2!vb(P&rxTX*ANWm3cSOf<6ONQrF3(fVeE?buu!JV64vPS>H_g1&f#vG5iXEKt8lg9Zo%zruBTa8O6MRJFC zcDvWQbEAQQM=8|#^PHMC(@tK6f)t{12|H3FAdf;5P^`>mf4sHrb z6TNS=@lj{)P`und_Hy_O5s}MpYkkb}w};Qlci+M*@7m)%4Py1sl3Rc9&GOa7_WZp5 z+~d_?Jr0gH0=0)8Y%V5D4Yg;Pnt>ZF zAWl|-p*e1g26Sh5h@Am+_tkR~Sz3lHm0j-KLNCbISh*k6nbhAATv>+vI7Ftr9({H$ zffx@0<^f-p!~+)Z_4PC7F#41`q)`N$3tee3qvsj^45u?hMcKr5t8!NiNBnWEmV zzJ6oO>yg~Lb4PC7x(%>dO+KI@Rtt}!Tv#pHo|c=iGPjl4^`SS;Yd7P(*+v-A!$w0e z;gmzn)E*Y<9QPJ}_03JPj;NJd@=S5ts7r;Wj2W(;?dl*$QkU1d#69xFcX&mU&n!}tW7kOQN9BEqoEUpra(tkgxj6ewHC zS^}3gK~e3xRwB?+TAJ*5=Gxf(yk3V^c5OESY>k2e5FkNRv%odzxFhk~2c6=UHK0Scj5QdMfm(xs9S}|G0jx)u^aT@~cQzJi z_PZ7}Z8Ta7=ia=zoyiE^caO-=d_?3+U%V=R{jYsYKKj6^9BppO(Xd}|@obS;!CYzu z<-zE{^;*|eP2Dt6*ecQ{Nzc|F98Lb|=1cMim;dp-bG+w4tR8>xHTf6IWBMKa{;iU( zZ6xTJxUvm)v~PjWph71K3)KTg4kKkPU`|v207n*LGwKqpHS=OOr*ZAFFi$z z>tWM0n5iM`bU04AMp&i088BSzKkphN0#oW2%`+v4glo)vlw6{5wL~LWNIjowdiDObr@R)A9}wzRqi((9 z6(E$-3H(cj!5B&<;|*{XhLnbDU`HZ=+5zV}V4nCK$;gq}A;^uopZolLhCBOlG|e=g zv2SpFDRK$$#j-{U3TK}*4x}(O*!`z&27q-B6?>~tUjuuz#~n7+x_51?7VnX52m@Fc zN>8e3kf}kW9&n8HHRYd`fVh9g>-w+>enRti;a}=# z^ogN6lzP*SQd#QD=awgY?Zt_@5meU({}!7o!9VLbX%dQo*dxf4v@WbyC?Lk$vz!rt zqJa}egczNHMxATg#(M<&I@Bn%Z#Hm%v?=ucwF7!ZGNg>>VAE%a)g=UT%?sxH*tW2t zlr!tkeC~ctv`Y!@{H3mUa%YpeR_N24M?(0ljddK-sfs9oIP1Q)hb`jbY;YihPZ{*n?|d!2!N5?c8N}XBo1;`Pl!@-kZhR zmR;9jbFIBkbDP&xM^>>Yc9N*7Vo?-(in0_bFbqUNA|r7O#E7HFMocOW^b`NZq*bKz zlK?>wAd)HMBl*;aKN2KJ5Ws=qbp9}c0I?0jNaqNO(n+j(@7?B{v-e(EbIvj5TIam? ziV~0?zE`JsdGEXTG`m^Nm}8DP=3kTFf9wxGWRGV8#OjUs);H`v{rn#m#OmV*16iMK z9s$+Ku{b;7Mc8T_uLHUQ*+Z6x4=a#W@7P(>ewrFzEE+RyYH^KRhQ}tWEL(vKn1|$o zZX0xi1C+;7(wK~Hob8LT4tq=cY`r6m>lwY+z#A8g_Iul;))l0zH^yFC)H%DQjPXzU zULj#{JrR9QExJwa<7W#eOv0zm|1~x*vK1_r>C(IevR?EwA{|GNn_7zAKr_?%&3(Y^ zf$F&{%c|FGZRfJK*Xeob`8>F~vW-5o&w25UWLp8Rf-c?sL6dn)U8MC(VF`yg3m^px z8d^>Y3)}#d?4*N#@l8L9bbM1Q*c@H|ymZyU&PvWbs9}0bBv_Y)jFEIqN)iWEgLham zGi+g%zk_SHRuVhM0qJ6WzUv?h7)}MTb-c0(fL7BBWELa6g;V$vXK`N0Q~4||Bxr44 z0#?`+1`edy7Ltq?oeZdzCCF<2qF`acy5>%bz|FRG^J>RImS191Ox>}O;Z5A3*_)inO27x<7hEeJxS)0!uo6X!eMXQAz2Y4%3W9JT^PGO?77^l}0oF@aHufL(0; z4B%QCm<7HPqIJg9WD4!tcXPml+!GLc0?zE1;sh}N(6}VSPA{X_%GYa`wVBRSGJ>}z z!*VvXz+#Oc7ULn)$)$nVEE3na|4y(dsFl#dC6i9YwP9f@uA?omeu09`Pihmn4mLP~ ziv)g@bF{xNrC7ge&+x;?M;tFC_@-fukjp?S& zkwq>qM85XLjr`0{za($Hc}HG;VOP)T4=S?Q18|Q)tJ-Hsw~jpKgLZt6^YI@Az$Jc^ zp0)L_ck<`tugf#=cqTxsKECsp_Zo?7JfeveZ?>l+ipS*A`fIkR&Vx<7Bv zF6#(d%k|YH@4oxciihS|yKHDGJO_v}Z4fGBxzSTc#DO)e&2KqWw}Ap+<_? zHl!gspBQLKO-I#7$eIG&Y_|qb^nEN%$pVCq0UL7biEivk=sz?3OVNP_i$rLm$fDx8 z^b{qKhzenKNk~m@(Epvx31^>aaTQRFLjVoeg*EAcvf}!s%#>#O?U3E7r zUUyVY7jJ~HRulZ>%SX;_{*ZAZnDoB#5{?)`R?4V&T$|X~8gt?Uu`$@M`WTu)qL0t>mQ_-}3zCas|x$`qkkORmw zvo@j_*$bhur$9;(?{ft5aYxg!RG?>KxmNUrig9aHN}x=sf$rSaPyJo?tefuR`uj2M zWHn53?_M%Df}i{PUHQ@%UXWKmw8&<7P+#^?TEl{{l_-?$);UmqU2 zpIg5sxA(^5ck;RX=^y4|uKw}NfLJT?`iA_k^?p6E$m`o@Tu+uyItK_t~C9(l0(_l&CwZ4U-SlaCs`xSS{GPU^pb^S!IVBqcRVyzCu>KO z@#bUd_F2j_>i6|_WB#wU(dfUmXE+vf{oDEZg{9#a=w^+7J5A$K!v+w!0x*uv8?3NC zP7AR@hQN&oEQ)J0~a@X)luoTUI)xB2>j&%m}WcsN3`+N89%iX(op(}JS zMb`qY?pAQPUb*M4bp{NdQd;CVARbZNVudc5-maxBpu&Z;mtwHqm#MH4j5gEfzDzkY zW~|t9>>km88464I{)?OJ7cjVCuCKo{ATkCp2__NjAJdqAtPTt(Wm`*$f)r}wRL6aF zb!D$taB@)tg1dJvW3%u|aX$-XK6LyTU@k0Hl#WSlmCS4(9F7Ijq$La_SPa%Pq1^D6 zxFP@p7=LWZOuyV9DiQ(v$;KXk_Y^sU!G*zK?o!eKvkf4eZmQ1C65jY*9J@(S%Vy7F z)1c0`b%qbD-rfmbur3GirUnarFjW%^S&S8S;W=8=wL3VA9&Qx&8H<5V)wC**UBLP` z0P8&Prfcb~kQ-WVWzW&WIlXTi`&!bT1Pds}xfhNl_M z1kGSqI`M1^!eY2J9co`a&N0C?VP*QFI2r+6j_!E@=e z2W|cYbm``Y1y|YINmF9MZed1HOkHgT#-rLu9qkga=Z@nS%)i2+XAOBHZUvFfUZ|Z*ItLl+M5U>*mt{c)cD-2F=L|GzZ6C4Wzzk;gLwV)gNz@5w*-^| zBJlegyyFQFM$`4VW`~7N&4$IJ?6#pafTs_(6FrJHrNpA$N8^5dT6)2Y}^h z<8ffr93kj|>SQ0acq!7zagDOSZ*=Rj%GNm;Hd6!Zzp=T_;u}%6Ho^n1AQsvp?fsqwC*_mIg2194{>+<5#fPi-O(#U`|JS(PXQ;t->6TVf>P7niejUbpS8%_ITSR5fmKj zTuV0xoPmSpwF6NWYs&F4yALsC2of0thKEB!y16q0IR<60M`U}w7|U3HD=Y;7z0#v z5_2!nV~_Qft;|t-6P{Y$=1^;A=VtSzL-~ycxvlGtO;OL<@7oyN=Vx0l7&A#CZ02{ zHZ1;J6U(4k)<%cVb=^mXw9mJmg%f?0f%YQU{?uWmatA9LGAXXp0{ukxHsrT$( z5f!oa^hrlex#(vMj0^AsaSYr&K zZ_-U(uZw~#x)jjca=SS*ji07r6{IWR)stBbUZJe#-wGIfV=|4N1_GOw9_97)O?!hi|1O)xX!Ki`O?!67zE@Z`g({Vl9TFZm2$~^3g zQfqy@2ntq`gAoNnx&T`H-uj;h8xC<44{z!QxzT_@)Jo1Pk~ffYnC2{}sQMkis!MhW zL?wF=I7&k9KI?OH^a7`;(49^Z?U2ncbvjgwW zHCf@-VcOTA!@GEqHFV&S4InRIl-v)QmOVIT1Q!Fw&wvI8EXuPkK_*%7W3j-`w6h-M zt%Kb}kto=9$xN+siuiS(0pQ4jh1*GLSq)x)_)qw{ny8)bSUukNIodb)wU)+$6Obj2 zRcakIc!XRdBRH`&(E>q<`@$l!W^LQ)_J;j#=7Y%q5gwkU_ZifpEke8_FS1+&=nou1Y5pXomx>+i`N;`}n8%KL>vw7y`M z8Me|nJLE*^$`Fh_WkQ^8N318VdFB}5T6)5aGC1b-OX-*_%13KUJBRqrbp)R&Lhonk z)(fmLEHd-S?g5LqT!&GIGyDFA+O6_zljPpzET8)LC_nS{*W~k`d$9u8tGb^&sAFJf zOt^MHF?@lyI=cVxpiS2DZh3zYo8lzmTJSNqm+b1${wLn~O?e>C(Bqi{v3i5P^$i2C zKhHtdZfI&>iIe>CC@`LC-;2cfu=Tm6$?A&eMJ{P=0`K#>>2FuF-0mOAyN_<_W`ig+ zZ*{peu$xzl=)SM|%(d!f7Z-GUk%~YNpJL<9*yyRFO0cM+c%EtTY|aO3Xh8=9UYzu- z*yQ4|1`|t{1fc;wn1vxZ^AR0ba48D>ZZ-8>W*VN?_ZxZ}F5dtsrsu+9uP;Sd9t9L4 zC=3o(z~mPl`_W4k*cEKiqAjyk84Z*fp)$CE_??Tp^5TmxMq6LCCwlL__Wix<7R@cWz;qyuJX?VPD)0e(*$$IImDzpO-zdPp zI6pHcR$KIrdi-iiD{I(w}gJ2>=@4F6V11gbOUY6)=HBpMD3B)4?12&F8EG27AaNY7$B z6L!1emp{z`<04FJF9uHa9@;Z(NY5%j!u_m&Uk)bim=^VW-NU(J@>b1OtaSk6I$Hsk z24dd$p#E-Gzu)eIomD?spGT0$Bp9qi7poq^))WJsxM(T2(y?8?%v#F|zp@41d&(M$ zv+q`fx$#!Z)~0yvV_`snRtEE$asPO&82~Kastn93G60Fr7QPm+?JONX1_H(a8-Bm$ z8EmIC?Kzu(XaQ}pYtHwFgaMzeb5V+WQulU*rYjdpejNCFVcF0E9X$*yHuQfN>dE8p zIYk;9t4knx4K)~_$*Di(x7{~tx22qp&B9prY6 z1Ugw=*V7PtT&tC6fmX}xQ0rYnu9Pqad(CVV;C0X8=h^%S6bo`HZbc3oGrwM}Rj7Ml znm}Vw@7w9>wXOH6)b(*EA5%Aix3}_hU;nUt@}pD52k+LW?CM+TwVgqXl^8OCP?j#E zhW6k7gz`kp0-7zzi+1-Vj=RpPeTmGK*!U?8V49qhS3hcvv^gz%jLjWd zF_3MvU}0Eg%G9~N=HMDVgWuGx<;J8+G^3{U3StwyV!80^OD1gW zG}ZT8qZ|GN>Z3$qn=n`z{2t8euM75KQ{Vf%f?H!Zl!04sppA5WxBF^Jms-V|35{8_ zodsBR$%FXs2yWx5mJS?9FmyKkmX6U1W`&GMdk4WdGDh}fev^qheUzAUyhc3}Lmd0$ zb%M`ng*(o&%>*krc7?AY1<^oI0j#pH6Ney!ywwlwgjG z0)hFWtZ*Kp@UN(DT6L!i8sQ8wBD-Kr?C8F*rWY@*seyyM@~H!P72bb_BZ}Txy>@J>653@Hzm|?Qqt_`d1ADg5 zwV{wgFu*E}d1YYcQUG3yA?7s42tjll{K=fP&vLXAVTAMg_OFK>$;icq(7ksAu?H#a1qYBP%k|dcddy~br zG(Ol+5HkFX*xv|L2|R};lk43ESTuN4yU{VL{pqGW zn|g1CZe6mA?!M=1YbK=g|Dn8iFUeQFbRl2);yroe$Lcz|dAIJD*Y#XnY7hfGArJ`= zi@mt`|M|N4WboP!yj#~8TzCJjfB*aA=cYgX^f&n7eLRyORv+K`hWrn8gZmY3bdQ07 zKO~4v1hLX-fn>o#0G1}ZrIG1W)nQi8yu`-lT4Yur(&C!nHmqM zL=|5)!wWA+OQd_COSH);-8q~XYdi4t*HB_WO)J}>9tdJJ!VXQYZ%Ql_2^O|=>2s!W z#eqieV>_4Q)|3upaxi}^{vpdI*eJZJxEx)h3>K5s&IHlk@yZJ%qSH}v#=)+Ca_`DJ z7nio7-5&N}WU%S5XoldSe)$IK$3TN`YGPX&E7`T`JC(5@hs8C!z3UXM+8ch7dae^} zkR0S5?7hffS?V0RF0!x&coI{xzUfFT#u)`E6@+ZOAjdmtt$gFPV{Dc^9LrF5wx07v z&<+yakQn;onct=R)U{k=0R`3eEJG+(zY5UAngAn<-yu=g;O1lr;4L zZ3fFeqLUoaSHd_UC?OHin02wN5X088Tp)(;3;!S?$c zT~&Ig888)bZ|rABOF}PH`~2eT1(`zIN1+F%&e)pp=F}MPlpX_+WO7*xLXsr_cJ|ti z-j6(jj?4x-Lpv|ndi7J6?PN{f1nWO~+m2z|mST;Mw$95O^V{#ApKFjQO9|iLkXm4w zF`c+N38j_#xzx|u*m;y?C)@R{+&w$UTW_xAkN?8!^43pmE1K zt!>g)xs&e=yPV3I0s%i^eX}4w_PgJcfAsBdl-KIb{nF{* zj&w0U0K{4Z9&O`V6rU2oMKRqHL4xj+<7!*KF08Oiiw>uQR+XO1dV5|s4Y#u~qjw}A zHPUvm{@`Ae#XUtQF4_PcjkS|lApB=RkLzakEKAd8GNPsCj%_z>lRWJ0GsdB{7955l zY`iiTA*9ozT9g|YuVa@=JlWv^s3N+H!GY6Pvfe+u8i}tZ`cJhuzb@S%tAeOmz++rrLZ&>9m)z^KQk&(Tz^ zrU6Hoz_aq)`rYpj!md{m>|l!HvHLF_%M_a1Zo}3m4+ESO+~0F$G7+z2bw>Blz#qT{ zHxm|>4Cv6MDg2Thq*>ZhO#Cqqpse#zT>;P7SoM2p?wcn$#>tMfc?Psm&0fS4dRi1dP zb49@v917|vsP_=ycF2$uIG9y$SbdI42-J^uhsFv;B-R6!AE|vEwe)Lr`=drY31NU= zM_K_!kxa*m9)nnpf9b#uGf6%^#@y~-Q0dw3w&#!~ywG4CtcJnT<3$CYR%c^U%TU6#Kc~wk4!|Z7pema3OK30A8hNX*4ja zn&mO+OuCYn@9*V}Pbc}x7q;@{&)=61XyE1T_I8Mg|oSOq-U0=3r~zJZ*Fg_0+~^w zGjR@%nJO+2?JVf%LDq1M;wPJeV0rZ6MPLbqB8OmCl{MTOD3qgOFfNvMibsoAC)^iY zzm%G#i-Fl39c_BHCPR}(&`J$e$~nHsN|@!%KxY7?QuKfG1ZjXoSFXdfvxV|(du9ul zcQV@BqOv{Tl0=DgVk^Sv*gPzf{qK0p`dAjSX<$ef2X)4%k?|D4@i;h)j#mgyC?(MA zeS5Qan=7^aD&4LqZ4+$z%H}=WTdmnr2xi1Ji)&}thHSDG(@vVVD9+3BU-0u~u(WtU zW5+Rg+ec-7wEe6$j<7pUzPYJjPB2?>yfLNY*YxwdNNiZ_NY!^p zOWkobtFPn#I(oR>uL5=yAHtNgv9DzDyDnTN4FHI>*0Fof5U560b+Kgd4a6>Z%|dRy zxc_hRbDi}%Y2cJhbti|p$X+*Yap{t>scGd>6lG_1mAF->lHCDL9*gNRt@ZP`F%YX= z+|2o893#+`?t1umT2K^54^{~l-7JH$J5CVT#Qf_DK%G5oKXWb&DQ>jTvaTQ_X7h4c zAPZW8ZHz#)kWrV=aRw>SxhSv@jqF<+PXMq3w9kdSS!&Pe22|*#qw$8cpiKaN^ME>t z5p@intul*0tg|Eq2*{hC>=J>Iv$8EC<(m8jW4|kg#JTrHj-lNrxaE21y(nHU!uXm_ z386O~hBp3(#x`4%S`Z<^k{AMSYbIh$aH*fUm(BQ4e)Qvm{IRdR zB%k}tT7Km9y_}yttjBa)KXY%-%Q0B=x?-HslAy&&jFMt~!Y}1%Q1>Z+``uw!{^@sq zL-tR3(GUIO*#@!o#(n#n@(=5M{WJaVTLhEj{d6XgiNz|%bfiu7?7-4eI$%n(F;W=x z01V4gW3!v7f>UKyS9Ow%wy8`=jeE)li*lW;kFFothVND0DHax{lICC>JZ+|bW8}#~ z8ZOxSGdnL0NEw@U4fZD16{v1nx${b81kcJ>0#7oiQKW>Ys`y_Sqdh$=HBE{tdW)O| zX)Eb~Zv5QRYh-HJAuO00{LwZm6hm>*u~gbpT6d{nT9;sxZBw3|-R_xP+M4m-UO3@w z?goCllwlOlJla99?IHDfR*cgzZTg)u0$tFx5Jp)R2PQDCpV-r#tv$Wkni7o#i=~4g zNCNb;^>>h9SU0B8B?SsbG%f7X2eHLoy+sc*i_REKNG!%Pift@7!sevRuAcYV+19{{ z{$1Z=ci7n`b$;hkwr6K@ae22s=K_AmQWx&Qqye-i%g`7S;5h@EO}jg+##a0T8=rzr zg21CS8VJ6kX%;y*5!}TDO>KM~F%-pt8-K;D20x+cu-Of-IcAV?allO}0a+R{t^C>F z(W~eQ62Pou6K3jvf=yPmk*)Jki?eJz9p(VF%Fdco0iJUpLpuQo11Qo(!Hf8k1ve!_ zA|PJaB?gvOCNI!tCpNa%qj!9(r>tXgkE(h1o!(d=<#A#+3b*P2U50dSs6v~Zm}9X^@yVAbN&UANu3ZL#37 z*`jgmMsZo9zwjK_I{-&w2BER6!h#|MWem~^7v_X^+PV2iXJ(~RH_}Q!Mmz9{b1aJo z&l0U`XWSc%U0#0#zyxaaosJMLG7aq-@kWtBrS0Gf-4;_g&b?c2wO6fLAK5-dICJ63bB9E2KTXeVO+w@c9kd$F#> zI#g`^Ug;bf^!To1yLu$|FK^_FpS_TO?CY<}r#_M7q5{~oJghI6Y%bYb^jRHV>XY9U zP8yM;^(^JKmYx3X@rM<9{h8tI^2~U`KI!pHgIIlh=dZ~>{PwTPC+pq(DgN&0#=rj< zh=mIC0l#+kgU@k`DxawA29vC-N3z9#KPUPAgByAHa4$D|O=S#jgCofpE(HqH-qVB1 z#Og$*?ugMUwPB{EJM3eE8xw1$-Z{*hQvj)KOyJr1R@NJrFzC6Au<_dOb_PJ*t|mpB zMefTM&8}9>w4atyh`MF*_LnqXH&Cpb>;ipXSl+m3o$PEeh6unWbt3fCQck@Um-pSy z##}42rwfgyyQ0G}D>ul8lWh);5TeJwXP*SH>orNEhDI&ID{!g_35H+cO{Aj@tx zv_PZw)zf+I?a4)dS==LVgc-Sxk`eZ53Sf7;TLZCvT^z(w_e%4A!G>7|7L<_HNvHxI zqhKeGUZAtyoEiI1R>oN03OwDmtmzN)!0K8A6}XSK*{%j-+s-bwb>ZDe97@Vy^h*V= z7xuZE&9*+zucc-YoEb+e()9}MehXw5Zsu)lo<7&)1^q^xyEdLFw1*B@+Qv^!8^?VV zYX+6$ld-oyilU4FE?!wo`9agL+D_M)C2@^wTP8TyY0R`70a(gR2#A``u^;aYS&s#E z7(2$o+8FX8Elme6<}@(D-`JU1{oT}e?!xO7bWULBquB#8A*yYi)Bs+A5;}6}7+EWL zg(QH+`GN%Jf(kv;z}Dw$WT*oQi8RsxT7jEYdXOh_Pp%FcjAV~{0%(K{n zv!3-zO`4M76p$M$Ws-%12lF8#m|Mnz#6wi;r%o4X^U7sl3iRz9@H(@gBV(HB=#ruk ziWY}{xl>r%+1fP%fFDUE2ntKfpcEglUX9N&JA{5dG^4o3ji?klX(%6#lg%j4%G$L}!z0elU>vXD2%3>w3;F$x+y_&mp)zeU=|3EQa{0I98Gk(UAXXpm{EGZnWspB#e>^|^+xrDF31MuCa9bjR zY^hHFQ}#jQ431f!Otzp|XRMOt`er8&uPcbX)eTO4IM>!$YO+EEU+6U5Z_KDJ0@!FY zHO!PS^*jp&=cOwU(AUE?fR({3cI(mzP!Of26xJg%D)a|dtQKxQ0KsN%PJn^JGupol zxhqluGh{669G`Qlqq570w3;)4zH&qgryzY*TBUoz-&o9*;nZ+|wG|oDCJFEb{r2Wo zt}2L)qLQ)GU1c!p!NNPp_4k7JW!Hg7Uq?_rFX(bMyQIKa5HeszLfdPb1sGx7Xm0BL z*LJuw*ks%AQ;IY*{TstlO{DYngeEzAfsWn7wR+mBMCHQnF^jY8xDZU(^+>%Q4f2dq zzz(&IMOoeTW+mJ83Z3Hs%u$5o=>T8E)rw0sk}6->ZdD31IICwtPQrOXd#zv)74c-6 zsFk#UXrL|Bli65CnV#llK_RVwx|C`#qkNR^pSfdVOMK_g1O)_0&NPyZ<>rNgucc|! zdlr}UNzWifOVN3W@o9k%tl5|z^%N=p-k41I3E|r-UZi&ge{~23LU(yxkgPYBla?EV8+f$UtEG@_{oMX9j zB?yNC2LesE*s>&KoES(^BfFr}-GtO|uy@_H4_#v^M+_I&F?xAM!;;Rk8G|E8fR22N zpv0vSUG=>%$Rg{Jge{i|ya^(G&){oVXm$)j%h9=%C}1NaX&{r%B4q1=x$`fN2MKV= z#(LJ!3U~2u0<1zmG%nA*c0KFVk!9*^Y}HwnD~(z)r~! zb8neD#~3z$!F;BMmwO9Yat7mq@nTt0=>kceQ_B>3?RjAfZwIaJ_sYPk$yofDbF2xH zX$_a{8RMXY(?hwuxRuYneMi3Zg?sYZH_zq%-D?^1qx$llV}tC$VrXI$#73uqGFLs0 zeg5MKkM|{EMSeHu@+aT<4Y_&WXFUUt=L5w4`*QQ`Z^-Y|5%JHS{_O`0gf_+!0W2qD z?iM51WMLCaC!)KzjvDqc`j(E*L{&4jlq?Z*5wvH^8H#sj*K@$h;yNz6FW7A^_2TDZt3_sQ?Nrhg0#i}_MYhQG>o2J zT>u&cT3()i)IcFv>^pV!OoImu;yN;>{cMA7YSs3|S0|zKLpr(!-54X|12D>TIsa~1 z3b;}@xOFXEpT70uTcl^|`O4d4G0l{I$JtPE9P>R&I5aZpROZU5;iR$tCcY4 z=Pp3*nvP(Muq|^BE-Vnyan}c2r%ad+&;VL30BRc4LY6Srj?xw47RZ%oohD{^7r?f3 z-mM@qC?{~D0fEQZJb3$#R^(&s@L5bd5QMg`!MvVR$VgP)8t+tyJ^ zVG;mrr>$LZ?FYNg8i-%lp#EB3f3?Whzxtwl`3v{uGco`C_1b%xW9UYW~N zjBNzzjg){c((KB0pb z+pZ==|1Q{X9*VTK;i8;Uiin+2yKRVUX^S)%H<{k&Fqm$U*i4$n&LeY+uzel0V^hXR zkLBj}x?ZcDMv-U{Um%=;!VU3e`Gmf@ESp zMTezAX0VH>UBBgNVYXsbU8%`8bn}deU*hKi@XP-Xh6X^4a*hEONosesS#M>%-B=M} z!RGE8!LEKDKi}^wJ2-_wITn6_#@63=jOt+aM=#j$^*StIt(2itX~u34uv{*GOoqCf{=R>#Y3zYhlk z9jn?_*#WFxAGguU=Jh)GzqLbL@iDG|_UzmX@zfqo=cG?VPdR&lA}z2P@^R-scH@K1 z5y74w*D`y*_mITQERI^ZxJtB?6^;o{GgFi5;6VVuI=~b3VFGiA&tBLFM9|#~vL#%9 z3pD&dB#5pO|A&7s0b)y$0FFl^hXE-W&Vu5(r3E^azZ{J>faI9pr2tF`bgM0`Z-MyI z!1z43K&KB7WLes^h<&H&VO?^N0=yn+|2u-fZ;2nX=~fZ6o0Eu1($EBBbf9X$)f&F6 zASGDAS}VnWL4bhR2$q$pj1${0cYr9^^I8T^HQ%?pJR|EKm^0AK&^7)^< zkS~7zQhwscw${*KHQv+@KiK+U*(Pl#TTqb3RDd)Pn+gNIlP|1;h(|C{P~e*S5!d0B zB)>hrE#H;rkI*{dt$MZga}`&&(Xn3 z8PMbfG|EhQ6>yPAp>EzX0_q9Al^8f62vDr4#4p=VooC;F4)cT?B3VGd(8W zx#mc?-))fn1`w>)4A}vgRPQ)6nt!D(QmTEuy}j{snx?Rqa_h0d7RyK1*9Hjmm=zQ} zQnvQ`+FR2q_@8_5;XUn@*;Xy9zBgOg9uN?mZhU^sOR#5Tr1kh*6YDxvp&@vu5iIfg zP7g=6CP{N~NN8TM8lf}Y+I^ZV;5pcFn)cQ{zqqD4J1fV{Pcm)HMrTDOM+J6o7goT0 z7H-ry*1-xIR&svk?S0n@VpXz|y<(Xa$Wa-bBz|VVRY3AT+o(~U6&gBkZ2Yv~HF_51F&h{m!~ z?vZI%*2Zz4Nt@?_zcm+*xL+V^bUb_8{VEgv+uV7>wR^=miF&EDBQwdm3+2W9zow>tx{(Wz+hq7SQBUcCLR*|8+o zLb9^iqezb=b<>(-LtFqr(HhWP1?KQQ4ZUo)U~#kK^$2zp3)o}@Y&jNB7Dl<3Bo;u8 zJ5hxmAXU$YHauZo?IS zsf92K<|hOIx|rVF+!(OrrXd}htj9DX1w74o?_~+bc0h3)jqJ^0t)P;8+GBGmV8sO0 z=&foyKweHqBQabA%P}j<0m^i1hJUASJUZYG`$K?O-xyqL&326jVkIMpNhm_`McM)_ zr2D7Yix3E=d)uF4>4m{IUj3^U{i5YahS8KCDBagisQ%Uj2?^03}`N%c)s{f zgLN>lC{F>aG9PvDDsV?7G-(sEgoc&lh=pBG`{LtNpBP=Bpuwb^prAw44E8VBF#}o> z?THy2#dmbXb8D@sZT!<5gMIwHF9!Ro9T7OEFPdKsB;fgXZ$g#IJFu$>dvPW z1|c(wkTTeA_jqMyKp0p=z|=r;jGF~kP%NqfGaEK*%CJw+^P0j5x>V>en>z%hLoMhg zh$U-o=U~J~3&N;xHXw^YWdMlP?teo_sp@(TusaT|fk2by0MIWyw-P{pj`8k9g8SD_ z)`pqDRmquK1F&4TC)-PWTsy!tFukJJG_A@~Fsu>u>Zl)w2($|_Ieb0~W=jzJDZbcH zXTUm`P+Vxe$x!%5Mm`TpNa#(`{bMJ0&UW&lmk#oypB&}qzW%Cw=FKyC;m#vDTR*DT z;#$0`X9nrD*czEIwvT~m3v}KOaxenx;u7o<7eg}7`aTLU zoN?FdD+o#h7%4V&&<-)f$Zy7H8WvC09Z0 z0W55WX3fu~6#xpxPhJm5&DmD(%{3OcO)JzK*UwVd0tL$nV?y6w$F^3y>$@de7&Bx*&0;RHH()?D zkPIs)0$UWd`n~C$9@@m`Tpx35bTXpoGrB$@lhuH$OEe}jvH=Q9Y~b{VEdWntW}_D9 z{p;b6k~_9nz_fR)Uh+A>Y|}AQPY~PKv&f>ro{1y$C@2EhZq5l!f7{=4x(O6~aoqdy z4$HuL2br*!Su~v(8c3bqm?U>E%vaA8u z)!Xs1Yg@B>O!UX~(Iu{iLa?pHT}vrd^JhB%jLuL<(-fot+p1lHDd8s3r%DK% za1hRb4q$qcI9s*f2cx6K9eQ7^qB;*=aAq5zcx5TPWJ6~wuFr}d@goZa$V@h8gwR}O zFQCwkNx_DVlQn(U)a4WmKIAqPxG=B*AyS{w+I>vU>%B}>@x;uP^_r| zs>Su%9`;_;W{niIMX5PA_&sL_pp3M$pg_i17zec|O!rM-B5?2pQ-QJ;1S#tpVEw>` z4x^2sJC|PQrEI(&^N6BA0a$3kN`_V(W7W`R)9Yc!=>ki>?qu;U5{@0l*wa-*y0I$! zYzY8c5Eb=6<4C*sBw!mGG#R$J1I+}mlu1bFVke|f{hm)PqcAUUC77Eo5a3kj_#Unj zTA1fQ7aJRs=?LA_URd>>oN=|D)tU-Lux;?Hj**Ky_srXOb#`uytOh?Or6Hiu&n~5P ze?zkrf>_$i*%ewkHWVO!CSz{bg$0)=)Jr|^Um93MVAV5xR{WT}p@9QTIRbhyhJ@hJ zvs+GbK~^xk7Iv8&#QHc`U^|!_BWu^FNl>DDAc~TXGfuqss7c`8;yCKL$Pnk10>oo{ z7!%rx`^rSkYjc_Lu_Wx76Kr)a7hA1RY!y;2Ioh5z%8$)c8^%l*vV#W?IV*t`B9 zB@oxq1f~y>PC;ytE}e5sFtEARemMKuHky`5a9?7pZLMS7V}Cu*eO=>{oS*IF@c@Z_#nB$p2Vz z_CNp5Z^-<9@Axb}o=Xs`H~L@y3-aIAf%@ka7oCokr;e+qMNHBUF2P<*pFmyNCaYwm zy^s!NWw*}j#-f1Z!J`}59qQy8Dp084XvTY3GmqlwCu7Hogz>lnF7F_tHat7~SPJ!3 z3)>fPU?&#})KhAniLr>HqE^4#22+n?g{YjJ%KQP(MVMZzXW-WQsyk@(G!`vxyi$0G z6c`Iz4xajp-qremO;O}WNJ$)bPhu*KEP(m0>StI4zq0pG;N_N6$!&;cEZ}4`P=F+* zZ8)Lhge6u)^<0EI&0B^Ew(Dm1{=0FoBX&k~viGpeAr|cezR$r%I-oYzjEg@#OR=#C z!RO$Bbrt>VH@3*@XLu{yz1erMt*vRIgGm{q(wfCqwwbTj2E?t-9+C(>R}Y~4SO{WQ zTW=&`tttyLqKli`D`-Bif4_6->FNPiVB^};mR6{Ql&i7j#w?APF9}vS*+7sNtz`2| z^q|n^5G4==)z~H=fI~nb*cS;qC08OeEh8ldO!xMHIM`u~@Zb>1_z76M#DC-SN}D@< z$Dk~$S2vAzM$P2SPSy~XCIS7h2 zb{TMk^`1}0zKW4Y#paH&t8y!6=aYQuW0QR8vx9u?%dg1CKfaQ)&4W5FA4wXuR$>k? z;IcoEAF2O3KGdf_($N=x${-!Ozh2d^UrcYw@4erm8%<({j`={-IVs4 zA=C(5Cud!p^Wyp|#oL1dn9a&t{S9CVAUn{>=FI3iKrC5PUEr-XsMB9Z$885CPItDofNgcbzI6afEI^o{b<+*}Ae{o#!9F$@T3;=l zr=A^Wh8Lx+fX+bfS>$HL4<5m*osh~o%*VBefl6J(Y4RQh=2&PQFe2!nq=BDf=kETq zH0EY%T#s-E&sf);I*7HdLCW@mdC=!hy|s(L8&iQV*xmqI3hA=@cqTYR=zH6y4|Z$? zvvCDJx3@dFuAgaI*nMkUp&P8;8-iVZ?+o9|g5K<=zn%rHa*G8a3PP;&lDM6jfsS?S zvSqAhO2*{h+;{=mcAe2sV_UypSYeTUu9iUHnmK#7nB-tvpBI|P zl{U9IY4Zp!3=&O(1#acabGyGATT#q(JW}nLK&3Af=Z1j)BWx=VW=$_{UoW|GCUg+ zAO-1ern*taAan=uI)wI__W58{jUk6rgp9ciF5h5g#Ab&Dxg#1BPMN51M?+7EIqE7sT#uL zRsz5hFecnjbrWE~GNx7tx!8iZwz2H{krGY^lMlGUPBV-f1F+l8m3-*sB0u}n_vA}& zC;8D&jB@XeUe5?Td^QXnOgO=)$ysW5DI|R-ezsA@ZcbCAv=T0V`+@1-AgiD^Ac%nGU{Y| zB{oYiINi^QBb+JHc+C>bD+Zzj?Y#7Ca>wo(OFF^c39Q%T%)c?v69cZ!#b9Y{t3Ky| zu5Sm-7uT!Lrq?ziSEQw%DehssS5^ttQP=AHaAg{fdLD%NzJ*%=k(HX$g#FhbGYfuC~l-XN0aFk zKSy=z8NC}E_18*>*jVToSlfMO4=|hd9h!f*tR&Q7L{M_Fw}dRIt>88ATa19^XE=*m zObVlAfF^a(v6&8a?|39Hy)em}Z*Jx1fA&Y@jUQXf{d?*IzOSFTtFOMb>`)%185s5b zv-z>7WLFpboIYCNm(Rl^3&HmE|DpoeUwiDu&(Y)g2C?r*dFNl2zftG};*nO$aJs|4*;GHqH8;M;Qq)+Q7SN<_wtZaXa&L5n_z z_CVf=$d@`xB|)F11sF*vZva@y8Jrgjfl`3jKGnz##eT+rt`&7R1aFVH2GD`yfLcw_ zl$)mBiR{kMnjD1X4+`K4)70E}x&;uDUIl6T7|3{nwLL0OBVz}~HKTaJI*lb~XLzpk zY*i5J$HS1@!Kt#Pd32e8GOr2-RoBeE=n(OV4Eo=M8p(G7Z_>SW$LW z3oIdI#~A349WTt?wu#1Xq1QZqpX2ItGVTiqlLmy_uKKY9ZJ6FgJr z657Ott(c?eWp=j#cHgqizZf%Xe!nIikZv3IbGI&k_In2>%6czo%dFRIqm3xEi0kaa z=7h3v8vLr}ez)^tPHUd9pX?rbrTQw`h#ClvKo12%(!81*!|Ki(0dyDF?Jn@n<(v)% z6;?*B<#z^99W7W)6r2_ARe~2LgKaPTY`Rrj!wRgbCeFu#z^X6S!a61GUi!RfvM9?q z4Je3WpbGrZOK`^I8g|!tWq@n2uLaEOLhwpJ2ynR2bidfRq!!b@4bJQmKl_^OYq(e0 z1L?vmJSR2^acyukE#W!~wo?IaXyTJgPMS77AurvCgXUxGmXFr*xXc|0rB)flL`;#k zzBSKq%)#DWf^#i7IYMo&1?NYuI6m{|`M9kxSy|r)B-XkSatL2PNwW0&uFLvd>OhxU z&6d^k$aAb8?kwc%kB-l^1$F!Sr+4L*m-q7K8(aCQ&)t`=e(}Td(*0e1&b5KpJZMdY zx8@$neiYQxL`p%P_~=3GY0*~uH}Jygi;H}}9>AY?=iie@Pk#63k!osRd`rG0#H*H_stEa1*tGzsEidoA2A~>0{199M^&@+ z_8#08#tTLqOllE=>lJ6{eEqotSPm;P$?PBm@5*3>$-KFpSr3F7cyboXw4>U9U~H`m zdl0J+F7o>OwP}er=*s3`Foi{oV54RcRu*=;DwI3`z-aZl$_aGSu|*IS>txo=A6D2+ z&cTe2zl?FPm{!*VlqP+!|0p}bXX@S4w6qt$IqN@IXQwr!CW`@Y^JJ*k;qvkhK={hd z#ChzAsXu6SMNq}>R8{J)zx@zM6PjG=P}_H_{OxhiHdTXKug)2UC|5_U`CM(%5l=l2O*eZ04^&*zz(AE_mM!T2d4w<#Sl0dc!tXfGc{a`P)JLT*N-EiW1$Bm?k>@Nq0r68G^G{L zfl?{~-_o8v%+Sma1ZcuFkj0DA#$n8frH%EJpoe!4RiU=HBM3?dIIU?%X~kM5_bJT> zwpo(l!wnC1KG&sXoVZ5%5}A%!STdrCYyA*wACEg&-6(owQxMt~rgn{nCPRritAL!W zZDH$S{0y?k#%|KNd+-U4Kp`Pp;2pVipL$T|)Qx=Rt&RNb&%7pYzi}q7efUtv^Skxf zZ&grMU#7)e3o0 zKAzUi3v=$tj~@hL@jW9Lvx;G+`E_a!IvHn6({IlRio_9KpHg9)awb>lrdz?;~aW(Zra z<0cIbN*%<`wt(sN2wNX)N&?_IbelCl-?p1bu@#|aGhlQn#N=rE3EL}fayT!7R$d<~ zY=Wom5Fk^6i7iZ3<`vqq`e+HC+z9Hu356MiA13JRkQ#+u&X8BoQA@BUxY+hS6N=3U zVCi4$fs;6JB`bzq#|kds8#|!w?b-HBPt-t+ro?ZqZ|qv!yLZpp)2r)95~s$d{>!+XTj!*;C>Y^^qL!p)!gNR z-mQb`ODlj?J^0z>9g`Fk09`$D)sF8DGxX1HxuKP+Y)1- z-H^s|dx4!aHX4!>U4K?&UIAX24H+S0ip&QyKmbQktaP-<`g7vG&d}n*JA0Yf6s1I# z#)B0bAW;=hrnB$5m65TC&%rVDRsZ3?>MMJU5qiDa+6y_sG_@3ZbLEhb33vdWB-%gY zvkgmipO@n}xJKGpg^gQ@6n`7P&MkK56-z(1l#6qWHu0xx%oi~5Zjufw0&gmgFJv)?nnK3__+w?GwsltvqdWH*m(8) zP4x5K@0ml}@3jZry7SKSd-B5loqX)0i+trv_vEL){Ib0M+CkRi_v`$*t!wyXh1I&c z_Dp(sol7j(v8Kc_#F5r9v2t{cPr${IH@&S1zde3heqBCzd;ntkKsT>{vHp1H^lwiD zt3Lq5if5caDn#h%3kF;uqE4nAz+(m=WtwpXM}wLEeE1?0YlTAqqXH0nu%a7gT8guH0w%zUjwGyW z>?z&C(c7*N#KuAr7mePaNG~B}(ogsq(gjwpwu;@>)14dc&@rl3ZOqSBri6z3scjf6 z5)IoO0_-GRVTNzYV*Ta)h7Cxe5?Bh03Ub@j_`R_b{_oz(rwG1wdM`ZWewC)*^l_WWDEL=&b!J=VPIZMG-cg zFm@J2QNe<8p-zo`wRHh9h@5epLswMB;vh1P1ui3YTukeWy)PMQyRJiuRC~Ym%%JqJa4?rv)*3Iiz|Gb8|0~0fDp4S}rfou*j%hXH}<&>e|-c_k2{~s%(j- zuGdx^p>yRhGkx#K+1M}DTVnFFH#F15F|{x`D2yLcSBzBLztt8IG9;(~<)Q=xb_`rQ zU|^4>9Q~euD@y|!_PZmJ2k!?^(h4v#(mc-MFbx?IHdq2LdT0x=H*#pl-D`f^?M#H| zI0%B9BnwzSvL<{_T`XHFt7GoS6m^@5K3XeZiF5Pu08dDokDFB(H%nggW|L*o0GwY& z>ebk_NcFeg>G~1eSk0#q0F}}LiGmMTT^Z&=q1nP~iVbl@|8xVVXBX~0s_dFFg>^%` zy1JH!j~+UJQIJ$Yk-8RGIu_tyBtT%`AO*q6h;u1T7gtIIRizEs6DGC~6q-Fyk>h*O zm{$_IOE5$&;5xo5>-JnODsa8Hdr!{KFJ-kolYPCy2Yc-#w<@Da~@S}{n zb=X_G%f$-a_Po>H+Sj$kPt?=W5?O}*{??mm)b++Ssz}f-uJ`4<&IpXh&f;wq{-6B) zacgU4Y+~W$@4EFQ02`zWp7AMOLm<$p>T&5cf}}|pV9#(C=$fL~lAQ9<{Gw9=!Dbfz ztue$sTgwL}MzN-VX&j+#XX6F`44zxTy48Q>kiBqqiuDU?G6QV5&NaqUz*G)Ar@kXv zTU(VmHhVw@X8l@~d@cnOANPh01e%25lsPTkai;#d!r_4|cxa_NCN`SsgqLoUAUvrh zdY|!PaLKFDkE!D<9xMip4qh4aiaYG~l3I6C0qXhrwS4%)d->{@Uy!eT^+ox_M<%)0 zyjRyc_j^|<0%ZvUkN?2i8D?%6&r9Ic)5j5rebS>m_JfY-e{T4c{Jkf?;Q4!e0Ajnx zcmJCF&)@o{yikYJ*Yw|9OaCxn^|9Yczqkl=Ev-N%nY#d@OJxt#M$2~8gKW0RHW}Tl zc9Vjr3Tl?K3I;Cf`cXH4w1VE%L64d$yI#!?mSD)3L(|5NruA9+O+8X}1St~Vr8I4# zdo||htiujZ4oHsENekGHWudpKN5WQMCDJWe1Rr>9uz)T7YYQ0sU3-&pZ>0*0C|wkh z3O8Ne(!D3&qi_Mv!R{IHKvQ-y8RBg4kUGCa+)KY)Ie>f9pe5*5q0xIEK!3WZ`-Wf_ z-9V=h=pmp;Fi{t10-d397Iy*oHUC1kz#QBo*MMk7;gxtAdPa&>kJYQIo#|rrJ0%6B zq&7xG0uUJNwE_&cZyU|VZCe4)#+y`3YT5M2`@iq@NFgUV(~eOUP;J1rDAS{QV0P&u z`)bwENaGG0w*L#Or-wm!+|=#;}&glp34Y1JzRy%kT3NElq`N zJ=h|jndvu~geMIMW?a(a^$1NrU$B{QePHk&;7{866JRMeDG&A=Q{33x z!lFDDUxn+yp{?`Ww=zNY7$5C6PTk*J>nYYlu|lpfKah{STI9_) zHu9w}-jTOHy^_nb_ax8n$>3?40H1zO?B-I%E#a$fhdDpS!k*#<%ElhgRtmD8Ufk!| zQ2Do4KQ6y1A3Wmm0f;?$e8=4iezyMji68jnZZJL(=whywc}{6)zJgY{sIT4FB*{pH8R!`|8V}k6V1B7!%mm$WXg?*vCiv^3?} zOR^xOW9h62hH2=O;`8mj@P7rsW(ubO7ODBUm8b?Ld(QZm8)!>u+v~(x1CkMYb`@K0;YsjF&J+(Qzr~vl-?9A?=cDD1Dn_{5L?o)Du zb}!~}a4{Wr*4V>u5AvORQdWPz&2&$qg_+{)Q_B`@6F$rnEVlKkw~KP(^nk$wI5Ls_K< zGM1eckpk5Nkdt*Kc;z+4b?KV*9ABs7YsPi$F)ghBR-k|TTjRU(%kTV_Jo!zZ)5ixO z_T-@hrN%<~9?@7~f<3<9JfS6LaZ~gj?9ClxU@0%@bc=*$G_6kV z6}f$ZCTnHbz!P zQ)ib5NE_Xex(&@=&M^g@>Hwe6g5aP;{WGpviZ8M@9zJAacpKa5Hcj@t>};^fQY3x64S#(Hj#Fp;Vf}n@$4Ya z3EKunDNmvG6Sc37P*Ve~)G88;2H0hK+6Tv6_Z-9#VSkaCe=B>>a?qcXcJ#%1~iU=sne9yYn*#vFq&?&Lv_@uil%L2B*odui3>d2LrG-*H^Ny_t`ay zK7R*}&*H6{m0?!hr)GW>z|8<724Ja&U5!sn3443x6hMXJVwt-@Cf3?c5@d%157AlB zf_7=TNb%})_wq&i=u(883#9`}sRM!z&asB^JQG^$a*F1RCJHG4cmsaTf{^Q`Pdl!K zsndt%Zti2PDE<27L2uM3K%Y;r#%sbgqXhuWj=8rhGZg`5!_bsh8>}k;GwV6L#c_Kk zLG0YEg?*syOnho;JA-;1|-{^6&{ydafTIfY>KI-uX?r z`qnq)@6_R=gc$%T!YtbCmADwGxE0F;S<4W{J=^m(o3C5mszV%TSNpjsciuFVjMK zPAmh=wX~Ow7BHk--(o?J4a?J!HeQP0{Ba#6!7HBcIk+St!d+<{ll6+NeUCeZNfKPh;%dnvN7AF@-8pn~GXLf(4IN3{j1^M51Ti^~dL z@5-HfFUa=n(t;0_4?McQk*nQK9$a19zp1a*xvfqpR*PLvcR$f97c#iFeafKjX7CP0b*?5|yb zUpfGJlr_N`I>dW7ZRdcZHWrb^d%?=agPFq9iGbMFS_c22|CRvGDKXgQI;HOc09#s6 z$m|#AJ?D(KFG&E8DZqZL{UWi45QO$ja4_sCwc@){qSdSi;?$$JwX$h^7m08sb-(k?zS~?Bvs-244SE`f>UDPkhpI z{`dgIKIQS<@5%r6SH2Z8gfgQWT zB?1RFQV>+Y#-U4#-d!=1&!c*{z-1Znsq)h7S;o7Iup1%L29XY=sBYUC)OYW zE@L+%IDt(c={~aSGh6;nU|yY3wtu6r%^Jmw1F|#u7{I2-0Yy)w_1aAP9fQ;)O;3LU zW?9_jg}Fnp(JZ!dN3**qLQre*);{*RE+cuE2bbd={Wu1Q;*?pd$vCa^8CI#2T8=1PA5 z!Hrz+S9QZWug~;?jBz~3pw?TXGpXL1v_-{S*b){8TNq?Ufl%<0;i*?py=vZv%7_Un z$j2cxYgob@u%D)VlgJY9aqtd72k#c7K$b#$Wq1xVSr9RpASwcy7%2nw0NsuIM_ zU?eiqbH2Bs(i<$al}incd7*U=(^TGekrmGkz}j6|T)cc*^(sec$+n2wEsYJLF)h46 zWQ-ZaWNN%pudlO0Zs%oy z0|3?vxail*jR0fHZI!&0saoOav98zduQ;N8sY+e+y%a#WOvM3;R>Rw|tvB^+@WvbS z!8SuJB667kt#z7)ulaHU;CHMyHwJDGi?jgDzRh5>s9o*BYd7Xuv{kS{vijRuB4i!F z%o0S*080x^;;lfXc%y^GZNO%-M-u?-jD`b{npmAPzE3u9v?yyb0IT3tL9AA6@3gvJ zodqfgRp9#KD<7&r@~&*pF6?}-u5ab`b}tXEujNK{tBV&vtu`xT!Ot(w4aA2DBa!Mh9#BG{qy?z7w_Mf*I#>0UVH7s@}ZYsmX}_9!8Ekyp^J)j zI)>Ap>rfYjUfcbZovuc z`IzY%a5<&MyO$sR#F>HD&wb{ey#DH3=f_pugTG&&wzKmsv#+n}1|sI~ zt{v}?O%-O5VtH?bE6=eI#7g(elNUVkQU25PW97MSEBk{UAAr~&@ObB&@}JeA`b%|) zz4F)}pBg+-hgbr{MpoHU!xHQfgHN^+RFh>kMbL#e*9kMM>LeNK_wCB`aJRc8H@E6s zu$Z1Mjf;0rs+)$fu*2HT?&0mIbcKRe%Tz&ICG4y1id+Yy*vxz1KQk`R@QDP7jTF%A zmOa_q+OP8+V`<1rPum$FVG+K(yEH}wPciyddqF4ixPZg&;DKDd(Wn;Us}b!}{}W*4%h_g8XWulpI;SCs*5Ea=ceEH73vmP-rGG{D^5 zniuS7P6ivfcX=r&FcVW zSU38hEke%Jq)}c>3*a3EVA9PGFwgM9?RCIb_!jOBxl4&J!7$E|R`(+hw+uNxEtf+j z*JKC4?|41{|Ck!x6l!%5a3?AB$i`&KrLA!WZn;z?u(aYDYMrmSt%ZJFaUS9Yv^J*_ z^}Emw#~k)zH_I{s$gy~>bf+=%n4jRaYKMo;k^cNiEByTW7GwhkW)Tx{x&hJc`bI9c z*YfF4Y~>ez?o;yRFT5f@^7l@`#aW$r*}5YA{_p>O-LQ7DFDg7(o33%C zIAu^d!CAerD{B)v*2uq9kXwfOo@4g@*;&oU)eOHbuu<&f)#AE{Y9}Z`w-)Qt1nrzY z619c4-yK|6iy|#S@t0O`MhaEs54X4WHBD0kmFTEt%v86CQ$J`09R!H;0uRap<`)a4 zu~gGhWvWH+(!gji%I1CQ%=binSQad}NKNAodIJbD5aR7UrELU!k9l^p@FVm4#Z!H5 z`Bl&)LClVe=Iql9VS|D`1B;^p%sdWV>U3~`uLXEWj}wf$p4n5)J7taaIJFRK<=wH2 z@lqDd?6CCr>NPx&tcq@G9KUR^Fs$5%QJp6WuA@0zjNV5)feq2)--GG%u4i5ZGGLtO zrr}_D_CE3TbKZUMp4o4y7jnXDMtc2%i8jw;Wp?#^4FsoVdxR^W&|#-lHZOK$B5Sshb1 zH+y;b@T#s2JGrVrR_CP{3%f-jmx5UD%B8FiuB4S1r=hd9C+VgP@OoXV<8*X!?IdhsQB;rR$V23ZwX~t5!~C1jEZFuMrib5Tg5JFrfEf;K}Ha?5R?h#FIatt5)Ot60@~h< zgAxIQn8~F}Zri0Tz6U%0(QM^tj(?m+A5w;UN`^cuL!RTE%JeH+hR)XzUH1;K0y|9m zVD%FYTty~i{_Qoys%V%{$Z}@3{dGwN0W8_w0@+#sLk&1kgCLN#84ErTv=-Eo^#CS600)90sy{f^f{*AL2&RKsm9gxNvvkPq_0_C5LPXU^pFZ{L;AefDl0-@0=ApY?eU z>ho>@c(Xe@p(~zx>Vef7GG;C;d0wuzd;}dz4;4X4y2f>r=dQlY*kFhYyVn+iHV^`dRvYTgR+{UTmBNMP3%n z%&-=+iw~e^4ye>8O9vK9V`1@#LJ@4{WMv6vvmbg3oR)6W=W7ZXQ~vyfF*>&jX=-Hq zj0Vpog15<^GuimpjkmajpnmS5e(0@&8U?iV-tQ){Ggc=84N|q0?D@Sr=18!qfK+7! z`dkIL*H<_4=)qM559(&Hx4>nsvH{hK+;5S3N zsKIlhT~b(4QOnsJHmHBZg3??IgoR6Oo$_oFK|_vv)ELzQ>B$0Jv_~&`-Caixn>|Sl z#A02{Uf&NhR zk+e?HhtU=R3baMhN8lgoIEG^>wyW@U$$Rd3?Y-ApvvYjo8*|Qm?!6Rwc_k~zS>(Rw zoc&mPJ?7(^-}uHiO2w@8b}bv#9Z&Q*70wZ~BABU-H#ra`3nV_OC8B?Z?Ue1c;4U{c zDNJfa@3E?E7ch*;-v0dV2^Z_zrVjH*AA~? zQad>x<@8)v^4h%?`EIVPo5T#-x&+o`F|2Zb@6A&YeSd`GWbE8i)fkxy#4KE<6QE8+ zI$8D5HYVsF)gy8o$07@W7i0mj9YaI5IRI@FTV_ruqgyFmkS_puf(_o{Q2;P&&+G7Q zHRdOJZjOavn|R?Zv7oZB#t4uFS3sf1Jhi@`4C5tFBxzRvTt@pOY3Jp~EcLpHe>N26 zqx!ybP<4javzD3M!HZ`gQ9-8IJ}1^R7akWT+@M<21luJffMBrh;1+;zt!gF}toY)? zT`F8!G0%lD!RG$I0I_ylWfJ3r=LuQRZO$HspUcEzY)YtpnVgVtt!8E76=2=LJp>b$ z@0*5L_wzCCFIR=-S6VX2O!kX?h+1w(nAOgzE*yCE%k1k}&SjC(0MeS&RRL|4u)^^2 z6f?N~$RqfDvD||c;rx6pr>FV)9-qnC`rH^Sv22&U!+kkCJTxGCZ~)d+e}@DB45gKj z-K^KL&i6LWnjGvOplF3x`dZ~N39^`#X|YQzB7znO%I{qbqdazfmghemY?-T6OVc!&>n{`WV-XXRHv^v?eY-!4JyN7!!u zV)~8TbABQJ`gHsJz3}P1AnV-;KX-pdL9A&AW@8Hq#>s=o3{QgG5hKDQcv~JIz}#%3 zOmW|Ik19#X^MdLQH6zxt`W3`#N8ez0umRS9iS|VeB_a$b#=b54CBTL5L!UDMyQwv; z0Z_-PMP-QC0f=Y$W0D^$>srIWi5gvZ513IFbL$QTwDz$0*xw(^%U zS(yFPL=a1UwqbbvxJw=O2GG=yzR#tVrNC!g1MBBfv|~0|MOsaHrnQITFd28os_EqH z2}t-E$g+9`C=Xqe4VZaA#;AHKL8K&M0h4=W*^F_sDopTl4b)sJW1eRIZf(n15=_>E zfmkn^;Oue7#GmMi6%OD;3fn(Z{~0^?DDJ2qHjmVi=J*{MHU%F`WeM}K?xFIW=>u^k zG^x=wi$W$XF$@ggII*4j!pZwo?R7=-=f4x!&S@?nGqm^H!7C@;bn*U51;!3urLeD4 z0rmD?2g3%U5fo;W!;;75Qd9!=clr1MVii!TOk<=(2CsEUBw1z9c>Tr=x$)@Za(L}} zUYzdXwbKBOKykkeV2{;_;P?~_Y;ot%1fqRqQm@Kk2E5uPRI|LdZ|o_6wYCQ>MXQgs z0;x67zOuAeGWb>f>nEOiNC*`pWoYjdyT@4QMJ~R^4Q9liBvldv;M6|W~$)3v5 z(Gkv3{e-oRZmec*KD0sHAK4a8u5<0#KkfRNe%eCRg|!QH423OQ9%?4n- zeCPlupu@_YdZ`1jYRHle+Byl2iItkR#;juSX;$KHK;7#2O1R4zxgdp2^wssu`q!G& zvjOnkT1{+x2c->C3kxJNdgXV{n3p(a2P)c2w^x?3Va_o(M6z z=j`N?hsyGLF$0}kRVOQ`&bEL~Z&>bS;Hp4nWu-(ju!yMA7Rfppcy&82dd%1a$uh-y zyvEd+7SJob`GiTD%dSN-rb%CO{hBgw%e@u$XBCr1PGVvfLj?7NDn%or%*IVz)mWvDZg&he-gL>|&ruE<;1K3CN<1yJgv_vkw=0``z za_96+PT?dlTSA|L-h+L)cKw=M$sqQCiFB%uUM>bkRkUwxAoDiq3S(-vxLDdp6jD&SIv4a7P&?Vy*pb!&xHj?)MhrzEnAFL zXAuKh1lwKJfkuwSzC5aO>kI9uSrmJCy?6w(EhuO`&tv0yZ7i&nE^a0bC+89wQFoZZ zP}6}0?*~iIaGNRzgJm2>->x9m5}Z=w01-;kT<2c7(Eg_epi5ztEvh60mrdVr5~9dy zk~FuD75Qk3NN0e03{9UC;%ApHSWRAk(!g{VN-1Bo9ac-!s3X95*tdujqDq$V98{96 z?^Hm+#LRK9jrMb#Yn|{U&REoFTPh)Z4#wiyy7gg^DX*5pianGb z*!`(hWy~h1CnMWvUFi%Uo+Kf$B+#K+Q!;Pd4Xu}q!JbRVxiZCqu$C@BkbkR-lnh=w z1+B`!Y6dz$tho~CB&^p$6D`az-@a?2XK<-3#4vBU*3h_G26>3ClkG1w+Ejq<#0>jL z))tJfCTQh7nma-;WBq(Q08c4d1t+>Fzy!XpC24&z6evruJ`ou*B%A8!UW60j+yGT) z_qpg`_!*W%c>i1%EUn+{T%X5&o9AXnoi1Z=hk+8Nt`Dm;X$}s>*?`|A&7_Tay0gWk zJya$#Qf8ox>YnskVe8;Z29`Ha)IfcpQ@(cRn^E3>|F)d1w_rN;D;2CR^ZTz{yDEoQ z59HwR3Kld7$&)1q>kX~?223nU;cmCM#(F>YbKB9Rt>>Tnj6C(^Q}V>)kI6MHRDlX@ z);U~Gw9Y`*um8#EiJYFE%JFdq*7>1*OD+G#d}8Nk8#pT%qu3jmrRACfD!a6<)g~xw z`fF-{tVL}`NP&NXHp(_G=VfP(7H)J`)rV(`^w|;#%ovQ z#pn9muTJv*?}+-vcG_u`sh^yg=VTE5SPZzLg@!!8et?wVet@+y*X2^S53Lsc=fd;y zf8KY;m)kwJOAz}2+c&-?fB%=h78ki8|Fl2%pa2%1ZD^(7c5)a`EXz91re_6MB#Jb! zwUzz-Jn`)1K|3t-puB(!fKG%G6+G<80*u!H5MkCosvGN}w_CH0`%I=aP+OyumH{9;&d#zRm2?fEnNTiFv0t~DInOeERWHP5?0AS^trS@4W$SD8PCz5jp^vy8Tg2ptAo0>3! zd(NMpdK?it*le_jag(UG~YLgXY)@PUkzT|X=^bg@Bj z25_YlzUoW~KKBj|kT|FB)snI&C+BiIgGH4fj6D8616*|=xc0~m=v4QhRUK-7QfUba zB5Y($8?A)Czs2=k<@>N&njZGk&wNU5T)!$;G7#4162^gT6LPn(s<)GM9(PWTj^w@f z-jjFVy#>EoZGY>{2-(A=nc}f8x>(m?qE|H@Y1aQzxo-9dLdCmXHzr0UUkp(%sz4U` zI1WT2-3pjsI&EDeKiPtrW0M=;n}6|w7Fu=e*-7~H``j?b7Ul6a;BCkkv~~d zmw{NcWrA2Zd@v*4npi4Pg!cO}t~n`?oDASN6kO88v8Zzvwk{exEDx<{BN^~M?rd`? z+F+LeI8kwTq`O8mc0sJmmAc|YB29mk0vOvWTK3&!S;ckv?w(@BR=S=5xdE`s!geUV zZj5?4T=()6@U7JoUwppY$#54=xEY?4U;BXFUT*iYiRGGELR{*1M}B`(J*^1tt^t{N}jMFK%dURo=VcPv>HmxAZKBZ zNfW?WsuYZ=vYr~w8ZFGFV}{BPki3v+VYM}_8!WQ6UNV*}8PxBVWpBj;YSoky-U0ya za89UFlq~54Ol31rxjs|~+HAnHf!Irx8Uvw>V6AK=1S~EmNE}|w^g9)qf`R^Ee<#y$ zN>x^3fBG3O8c={}WRis*-(;+&1DV+u6FxZM{d92<)1g|TmxW+8q&k6Zo8(+WvZ{Y% z!kQ+(Y@8{bSuVJ=3<6laH?ZTP53FXLXW*&~?DFscEZo7NS}CmzBy`4*Ix`KR>EJ+& z`cd%O@&0HFi)yt+2?6Z3ydVSF@+noVr-P2sRtNw{XSto|a~*-s!NH#CF%QgKAJUX$ zg&MRCY4HPXJFC?KtgX;ju`o12cC@j$I}G>f;>y`ft9E7QRLa#hk^wJe%br%QiHAV} ztX<#D6;07j7p?5X)lI64t+{$(7V17&?gAxY$?Z_G7Z2 zZZO+4ov8Kole2R<&3#hAp(cU>sG^xfxq9uYT)loJ1K0!WcOpe@Gt$CI0q6PIDOlK9 zfoZY`%-+Hl9gkeUCeJ+cNhER|=KVnY!$oP&DsJ#x&7XnWLZ z4{izC=SLgIJUXNrZ5_*WFI``Kk=trqcGG$9QwvB_0g-bWxrm{UvoCO^iRg&6;UH2I zQj7N8K&<%MR|x2Y32fzhSYL~gmE(M=?G{-cVvn}q7lzxTpOZU5I4E@%wy}L|!8)p% zMK5mR%qqR8)DyZCz zFm|~QOIh?QfbtM&_{44Ue z=U)C){_{zF?&E@3ZOv0OOP!E9bf`GbD0twWKnSh|owO`dGUk zs=3mEc9{o(!~EDsCr=aMK$;m2v9Ka0WwqAn&5#_nNXZ6Ro$$``q-o5}whX?OF&m9V zTT~NsFn3}DRGS1afoa?t2m)AlbL>`U>H}%0Od$uVl7JLzleLwxhWwJRb=|rYA z?bVI^jU$bhFncNSh6s&3@HO#DyqhPhp?(s(q&7#|hiI?4&UVqqlnZcJ6^=KVKo+SuKk|VjwP2(KvfItkp>~a@Xtt@tjN>}!{%Jz`9jMy;(={) zM#)%R=xBRzR#CO3s((dgt_|(ul6i14Y7QxhMqm1FQJg)t(7c-y!4?sfHJk{18P~%L zILdNiddq7W2&T*9h`!k3!OO_*kg4VKl>siVMXN*=NfeZ%E7!I* z-n$-ZM=ER=YxtTwwB$6eYUV-idi<8CTS0vb?0u0~`dj;$3G#}MF15CJ-7606GP#wh z4bV~tT0}z~0;CMUf^Ut9{(8^Q1}x1}%}imMeRw^4wENzWn-Q^3y;5 zguM9aPOcu_&U4&RKHF0ru|S7#z-1AgK=K49D)+LztgZIrZ~0pNW**}IWVoi52$$P~ zY?mPRV{AA7BN@N(Z^(a>`}CjBe_s6v*mY;CgPvwuYD#4fmjd&0p2`3OVpbWb{CY~A zC41>4l*YUo9W#&>IoQ)dc$Bkq^?X!D^&szSFHcl^G?>@nO0{gJRn;&XqelWmay8s% zMz2<6Qt+xJC^zS8OaeM_Vv7OcVN7BQs(SMy)xpzQ%83lfM3(_e#<25*6AFQ{0nN#* zSu_btCwGjk42hM*0!undQY%j-@q!TD&U0W^XO^8!rhA2LQ8FX#akB93ngG^9{Bv;u zaOM{0GFL{|^rTYc1hv?OyA4$5gbhHceSTzhCj(L;1C&ut*Bc85i!LXOQ3lh@#ehIN z*s~Pu3bslr9a!v{Ol1GcRXNB&LoJu~0A}~FpwP)@Vv-oNcP#NcrV0p3GKFU*dle;2 z81{&6Db`F{3KAyTmWfZ$v^MhhD67{mwA#~34h{|>CD>c8Y|>P#xM3(@MnUiK=`rVy z01&1b@PUT%)7IoHijusj0zjrNDgZ_posfF3N@5@Y>d267l@!cwmDPA7J1tT{@LYj= za{I0ztt4S)vR6suDpQ-;T-8c*UR=S@ikCum_8toxkx~X(&q*t9o#s9~7T_xYgJ)Xc zM0M&4>cP%0z)I=wSfm7VB?$zSwoo!%I2=aC+#b#}BYuPXq`065Fc6ge)wXjrx2LC` zd;&iYa$B=-J?+%&Mk6T?2~fJQ7eJj9#$)!o$A6^Q^mYYu9>A=wF zLOXiz#=^GZN|eY@eF&o)DEc8aVa;E+@9GL^Q2bf>C{~G#Vo0j)3x2o2nKy0+X=IEh>gk%iU!4_6H>Zx5}-bk7)X}4?+-HW z{<-j++m9NSFlpEw<%zrK)1jrT;>m7xju7qk9spR;NP>XhA zl$)B+ffYW5JX!Z$2D1B_A#SB-B@KCC$(?Spa>|WQ?ze{}CAw zO&T*P0gl)rwohlGDAW{KstAi~R3!keXy~h3SRggm*pfD~dyuQx7nl!@idW}3l z#-J=~*8{*#l;#v!1|$R9S~9tj&6sqc2Q#lMsLK+xujyyCiq{5!_YBvllyRmO(=45I z;Y;5QJuDuh0$O!6P+!=qxm~H8U_e0%DIsycCLYz$HnYjcsx0yQ?;puqZ@(jFd2ym( zV+%mOHbyym5rI%b;OMETBVb5OmJH?RT~RGwix}MBxR+Ad%c~zz%h|E01KNB(zw^8x zR}WjY0aNc{Uy%2$U$azfa+e>fxH~}|v)k828b&##(oejwim)xzNSK{70S&H8-$y1V z^>kP;eYGb^bxb%b$6Bsw+|iL@u@$Y@idV)$d!*DNFfQ)*vt^U*dv~jmg>ZL-aVH7V z7qmMFkmK|0uhnbkOnGI%doZgNIFrRCK;NzIX5Zt!-I7!$qU~*)_ji`Zk8}CNW3#;R zqlfaSebnH?7_@4-$ zmEZYTd%4^$LF~iWe)U`O<}ZC+{;&M&U*x#(k+Lj0ha+v{UtxTd%%3`&`^AgT- z)Mv65+5tz{iKwnkfdIl+3O;AkyDl=gK+SX-T=&=ZR zdg>;PeH{=c_KG^@7dSEt*O;FzCs)sY546}Za*E;9xf=4j4AB`?XhK6-FW0n!5Lg-1 z+14E_VvpduRdX|dq#zk9>AG11({sbLrXm+e+!qP~Gc(w;n@cDdH!qgsOAJYQQkaOZ z>P>D?yk7Mybnr$va{=ou)qJU|LFNn?ddV)hl?cw7lnX zf%_8KPL$!aEa)w&F`-BT2j1jhAqLiSw@F+SyC{0c zYI$Q$++J$c0ihA=9g5dJ(82NlcHf%?^sPHf}8>2LK>?Db4;iqVq5g+_o;kB zL&ADo$XaLHs9nnL3g90BT}?zu<$TM8-e_rC0P9IyrOYtiJK5_6jF6op^XAe50}-zM zG)?wQ)(y6403aQJhql9G1zPYBozO=yX-Cs&BVwYOXk|%ZrPi5b1#*EA-=V6L4G&g`d_OSuxP0mB0FKuWGGg4l<&{mM7xANKmxGge( z+3lD_2wJ;RrgSkZYNAInUsj(;M_TvT?#D-n)^+Dvs zw=EF5f0Vu=_4?wA;XLJE^*$s2eHtf%e+ zy3pRntIIf$<#{uNS3+dEAW$tb{U6mjOW8C9D5$=r_F4I{zkh%dpZZwtuS^qrcp!t? zC=Hg#<>9_FhD;$RU{$j;H^7tU4iKxlv(9v9k)?0LoOUK<7{#|t4P6Z)EpDV`&N3KM z2CyuAf_(>)H~PxyTb(Ad`I13%R@RFj6ffDv*&tD(A?06BdUKZ1{!Uzsf|+%ff?!B) z4yJ!bdkAK~rj@w-ewq0{{d~~}7hvdz)jnJ(ph;EV=h@sYRCciD;#5M_Hu_BuJobTL zQh{LSOkj+N*g#5<7fV<3fUQ7G7JCbPCu#@9cdFSnSGl+c?AX?kq z`I_zo+OG)`ZNVZ`fe?7n&BGTMrJaUQqAoRB5DUn0mlNT=am_y3V`t}!sE~s03|>`A zus*jU4_=PCckQRM#L_a7)n?1b$$Mqz`*u+`UPpGvh*H}CF)2m{|mJTi&Vgv2; zpr);1st9z46%9csWoo*6Gd>kIQ<~!t7OaWYJ2rNWQ%JJ_EDJndv%&-?ChL`}7|H@U zbd^W}Kj z{AVBi)0f-5wo4HE5Vv3Xmi+I(^fkGW8|jxH1jJ%WtBYl`XhKyjoj8_O>4pq#Sr>XL zbudA^8PTifqh!#N0l}c1PeZrc{Q6Rc?pl8CO76f50y=7Y(#(M|&{E5+w<5uzfmjZr zo!KJkN?O0#O}x)HXI6yPpOlK9Sv1L+7VJF=5n_(;f2%QkY3^^6Z>?U`^Ss@}UICD`1q)^C!UQ_( z+-~;r@hSk-q@uI+89YT*Z>=m}gn#T-&QH%#IjUph)+#%?7+Y!LAiQgPYeVAJP79oa zhR!#t0iVtBz@?~ZHfuO#qD!^72AsElqYF^;>z<-s)vIU#rD6ER*;@T3f?-G2-hcT$ zp|Oy1W*GB-WQ8JzGM)q3_XtO(fcGdHuG-{lSFg(6a#;YLRshtCu1*`;&y6AOJ=NOUk=%jJqo7|60)d1zlOOSv9)p4LvwvGHpA zx#Ic`U5S_#se~A*QOk!{V!|K3ICDn>lO{ECC5|`lq|mVdf*oWhUQT-Ubzo?YsWWGq zIT>kUN2O%T9eBZh;gzfM6R$rhUwQordGXmOS6A=L5N_vl*ow)SCbqX^^IaN1=GeK- zejyk^^Y2ePk620!-*XE8o?AY>zt%s0SyJ|4ZkHhTA#dOKru@eIrhGBC_0L@lH~HqK z&aDTXcZ$+Mydwz*Ob4%FDwU^v^{ks2gNLFmQv>HHo7}n2W!a6gKLlAY+grzY4Z5D^ z(PE<(F~PK9>f^Yb@?fPO=278!wEY?5kS&G&)#8 z0*L86?U1{M)k=kB#(8xW7`3GiL#%dPW}zFT6cAwbmbrwWETlCSuTIs%j^b)38@60F zfSJU6Xq)nei9l9o-HAY12U36w40;5qT?M282?-1Q6ao}v#fY1y_jCr}=UB-rPzPs> zrGZ%09V>uaK~AuO2dw&kYE>+KD4{>LGQ?dH7jPzQ7!seZb4Amsn(@E&RY#;!Di)`go zfCvsAkrHq~v-Lfb{^$*j)YtKE@phdW5Nn`<0tvH{3)C&Ll(1Jz)Apzf*bR$RY-e*W zf!C_KNa(%KV0Iu=uWeMX`C0CR%6ux@ZP&g-rLEqp#YGFj_R4rK^gdGGK3I0REr88= zV3`+BH?Cbpf}S4pqF=yIb4xw1hV&D|y1GcdedkEtyM0^UxphnKoSfuE?S$h|S;Od( z3Wn0*Uy6Q1M-u`$UjW#`G?WE~ht-G9B0n?QHQ8}xoi&Q1|3fEic4zJ=b`l`5ZPFa0verIlFcFf(cp*)y?HQv2y@mE&@13wHFpq2W zEdqUejS)eV$kJrHt|a6ZjVp?pgc6l8fE(FggIF?ykM%Qjt5$gnlS2M(ZENP^szS2u zcp}T;R31Gzmmhg1$RGdGWAc?R-H?}_P4ejBQJ(jXkmv=_D$O+x&Lrb1@z{Ok+hA9@ z_d_?APc^i5`SapdO4!cw>+uEoOP5w!4|lr+u@8Bp_13S?z5IOs^O<|S#|M=8LW?i{ zg3U9g3aVA&pe*nKi!2--9O8FPGSla^*n!>$ur2iH)obsL0t#N0RaUL< z3btCEoSvP^d+)s~r^m+tA9|>$LY6_jPc9qUB#o}m4b;fdTQ)FJao$d~Eaz-?Sb;i{ z)G)D5ucA1I7o1?*^`tH6(CL?|TP&cJ?be0YLfs(0V@{@V56xu9J#iwfT54mydM=&} z5U7env*_Wh5v%}$%1(yd9`ydBVq5-O+o!f~)u68FIMT5Sl`c3?e!34BOBCSins3Qc zv`D5KOn}cdzYPDF#-caTopH;wuazRmEcV?8@YhCnU zaZF7DyoX^Ycvn(Zw@W7%4J0nSaATSu!|}02VVX9mH9a~B$AZ1&RrXy`#?@!>lfv}&Ykg} z&woC0zxUm1Id-p3AY_8>($)qmWk3e4OTDs$B=QZI@CjVEWc)1QX|^H(!N8hM*vfJq93JM$cL`?6^jdTx z2vQP_f@h_Am?@B5O~%|!w6X%ckFFn!oN&Ctj?kBOqE2^Z zuvLtOiSzdc0%}WI5NW9llMC<|9C$YjCutXPUs(g^ZjWOOuN!+?hF@|k|X**ouitfYj`PS#!^DVU9Q^#y<=Pcy97syJsP_U6)0 z?aN$;+eMeSc*LY;Y$vzYkWiTZ+TJ=aUu$2O%=ALez!8g>#R~dZ?OX60o=k&|q9WWM znjY9rSL(75g@V{Y+t*-hxB?69{mND|T&(g9O|**+fV90JoIe;)iKewbIX#nG@4YW? zz5TYl^Uf`K=hl02=lCQ8*%P=FsPtfD6)mAnlt&#(dDj>i5%BB4XiPKKMP7Fl2Vr6s z7Bd^Gex){m`Zrj1vGK$mR?sH&{TnUza0Z`TFFOmHz`?z)2kNlcaog-*Wul}F`^U$`PKy?8~gTsg`6csoB%^6}GO zy(`j_$?d{w6YD#W5y*ir>)H844t8d=tgFVn?@;=0k9-S1xw7(No)Ub zA7Euozt~z}V5ZHKJyPElhH-SZLEJ*CvAdEwI5S#Arvob ziGB`lFhPlBkn4GyH|dtulC0vJfxHNoPo)xC2w|ZTjmhek>-#q6=a3W}-#JFH4Yc6x zXvaf8No9v^q6>}~>|%Su)oje!ykp2tYqq&wmr`n$@`Gt|Us@^2Ot?PD0AR88X+W$v zt43X&0hAzKRSEBit!-4KZ)gp@xC@8-b{Z(^(6>-EPeI#E8!iR^0I+5Xub%;Gt zgm)%xMib2;|nuPj^?O9kI0DOVz|p&zfY#p(%e*xL)lFs9hZ$a>sE zlO{y9*-IN4*I(%g6U_e*bA6kLEJZmQ_OV zMdtkz^FGYhV%NLc!`*NH|H33cA6}L}{8(Rlx!rfW1hL=8?N`4gxBlwa6R$q#FU5X)YSuNj$t1N_gDDEyiYOLWYKo^)&^R8aaa2FHNOnrfXr{L|vcX zI&;bpyoQp^bJb0{Lt%T(W{WlGY4NqjWM^KtUJVF?rcln&ShpD9W*AuvEW2lI@d6bo z;a^+jcLj_Xxb=a%v&1HG5)`0Z+igHJR^Kd-9 z1uAA{d&2@r{O!IkPe`-%zn$GhiCY z#F~u2RJ=%9qU^#LYl6S6$vSM2A^_D|_V(oDRNlRHOTPQoTk?DV@H_JD?|cW2iaa31=V=W^=26rB*b?9qt`b%ECs87Olt? zCN1bQ%hi=C4BnIL`y2V-(^PzH}sWk;z~otW*LiY*p3p1~3o#(64N zWkMO%5nc^Yld@+8EP$a&m&IeRuMrLRr>#-o1r_2H$=2P5Jik|3mo) z-~P7z{vZ6I+C=2nCn#EQj-nYT#xi@dq5-tN_eydT%DgHIi-gCuI$2mWsKerHS;{T@!ZMWar_MA{ z$4xhlHrkSQ3L5MJ8{88Ao9(d8@Yid8QK#-DiMb*Zs%}l&KTZVT&JcT>uL)G}?k0$B z%jvcnUDu{|4g16kw`W( znJbFB#2Ea~|K@qCOJ_2KBYEuFxxDs;rF{9vZ^-Lkye^-9M#r+Z4Zx-`1K!DGuFS6p zWWCE)2kRY1^+C2aQgg3eQyUcy_w9LZ+f&0;cxN<)tYSfAnB$=nSO&1&NgjYUd6Mb#BryU=%yR~=X{EeZ zo(Gqh6yb&%OpyxE87$e%jr*Nh^2tg#VY-|Er38lsv(O_dcx_qN9_~%Sva};B%=l#Q ztgsD$0rlTE<6N`*o!JzY@)Wo1GIk0ecrsQt`6Lr;f_LRwP2dp&><)1K^}^;+`&e#R zDD16i9nJ64d=kyaurLZe-zTnfqiaFnq_Y?p&RmdbwL%BV!XE7HoBk2cEji#DQ>7E< zhMMF}(ZSvu#QJ1Ar# ze%S2P7SQ|d8yJldKxBQEL9j{*P@rKnK%~{HPBlRz0(9wT4-cqoW^$9%hb#EIw4WewiU%3h`KG9+xXIUEzx zWc|n%MzJh{Eac03GjwRrYa}OXVwEXj#oVg6l5ss|nY-it8PzcX9< z$-H`%;R`GsiUKIgg6HGYY<`t9DEq0vQT4BHzw?f~eJcamx89WZZoMn-W^j9Saw?|j zj98>9JFLqmOVTuV5`aZ6*m|^YrWYNAL^l0 zy|cCoiNs8|l>9zzN4iJ?kmcgJTl82os%wfJk3~(2v?)-3YCa~G_$9U)^6;?()7^2s z4Kr5;*2nY2e%dv%fxvqs9EXaYo(muyD`7=A+j!)xtAW&kT4MvSzWD50IYF$&1^l_r z8Cka{jG>eeuI;$Cb&J|~RV}-(xZcp`_ZPr&wdXQIB^BUnt`lar1;6njKfmzBb-ovI zX;mikbzbH<>Tqu@k6k~PFTQdhKld}w$`@YQm!}?I%l_g{KGrjMt9P3GC92ofNS-Fv z4xFD7J(|l@S*Oqk1hHibV6xslc&UFmJSTti!S;T+eU$AI#QuP`Z+uIB^OwGc`Q)qZ z^B<%TxVY8#hPyxK(XEoNWX!2&yTZyxtX!?ZsdXMGPV)dX!D|vLeIz}ojaJB5V~X4VCtoD~%d6Lse8=o19qJ542Q@8Q1)Kg(WRUs`m7bQ1uG&HL5~ zZFZI-z`ZKo|WAptgZl6CwORJwS=s?5v-~M0K)jRyfMjyG0o8XPL}xNM(?eZ z0Oy!!0$Hotev@gkl)Fx$fG)NMON*nc;gbdZN^onWqk37e#C{d7is7+SY1GayyQYh$C^FNQ-c=lQ5@Qx=CR_oJpIHdKk>$+ z^5b8+F0Z`2lutY{1HjJH5uA8nWrt;FkDo?NP(BbzwM21QYz49(1H`&yq`f>?ywB<1 z4KK)Ff3Q7YZXb2K1hGHB?N`1ff9IFKE|2G5-?;eN2Z34dD|bKNJjo7eW?bZ(@Uq2E z&CVou@heZ& zWz`83Ii;P+2s#!Cz=SZ(B{55=p{mSLEG#bcr21Yl06TF2SBA&5RNam)%~@5~2*S^h z9-B1*#{h{9&K8dKK{iIK+~SjixE)Kdgt@{t&odo%LXm6O@5Ul`&a`&sKG3m@q6Djp zPK|}QeUItZR$wz=b+oj?1`1-AWMz9MoV1nBtU!b%b0IUEc0pD~Rcct4+sJ|kMRV6q zYSg~A2_-$li~bri$HkKc4P-co-OO9_j+{*oYl&UPe3%KnCD%;!5H5Au^LHr=gW?sV zF`y%oz}7awQ}?PiV|uYpGKkXmqL9KoaxvxLx6LTWy_&rOl}8|~4VIEl zR!Ym-biO_SuX#bBc2;XJohGAE;Anc~4s8d?O39vZXfN5<*!UdZh zn~y7YA2m{=c3F4s9K#Rxj#lVWaH+{#`FTs+8pO**qgvN?jr)wu|G@w(q(!u8Ya;>C zL1!sfy#*7$A?vD=5|t60se8rg2|TGwRV-}_8p-Gf`YNx=HO4|u_o1a`H31AIa3|}{ z%0@4$&buup;A4x+OlA~=i5#NHCN?e?Gw-*rbeuf!q_B=*`kEA)3W)VEzC}X19TwL5 zYbTyi2unGUo!2@ph;2wUM%DayYviZZz_*bXPhOLTn24l)tLEFMnC4uXebm4ElE79? z&<%it<&{7}z_tZlAuNX?FCu~_{;pVvW6Zyw$o}53eCm_4eE!v?{OnIXB`?3!$+atY zWN&dKVZ4Jf>vm5f`Vt(qW`g`)`>G0KCnwK2CDNDndw|&cN%u-J>Ahe5YvK9ySMI;p z%k5!rmmu~Byxsg8^53U_B(LQ@@N)b7M`2`lKi8~$F@W}v2N+XH0bsC|zqoGD6IRrV zPdj)&=D}bq`^#Pq4>Xh`8A#|vaZV;nlMC|1F!wa6Hya^kX_a|3p*|Cen(S0~S~BEh zVL~P+B+Gh+Pih?So)kPWz$Wuzt3@#(LW??S0bkntB7v{HB^ArAiId^zKS%n7YCNt; z*Mj5E;u3)Yyz@{Qy;6n3j1of*1q{~7DmK83b-+D49Du~fyg+6<744c;KeNGHt05_1 z!((MXm8Bh;Dn$V5(EvU&v`baA$^aJMM`xQk3%MrtB8a^RcuN8Z%DftYbsx-fomm$i zKa)FK*rccWLr7D2y@lY{lc8*VWzhiun|zp%8*B{-YnXev+AEr1y%x&0nr1awwm!<* zirFNAV7JLwC)LDc85h3UF%o2et)}FHOnu_-x^)fvS`xrw)de~$RvV39yHp|}s2Skb z!XpVHu|ifvJVs+=QBIgd5EejM2!rK`_g4odQBk0R z>onT+rM@#jmmIC^^SA*6uJJgR?SKkUAokwCwoD-ZiF{`M*Ds41{9w0B5c>n(w1MCJ7v;aonD8t4&*%Jk?;93k zeSjT1AWIz{7M7pH7mc>dxeT(-4#NDVTjv39BdbN12M-_O253gEfUT^oZ5}#T7I&g$LLvi;tbJu3Vs#pj;w@|8-vUaLtI|xcZ-ock zw^fRCU?SKx@Jeq|Y7m3f*{v5^s7DOGx~{&cHnH-1idj8B8bS?A`Gn;mx#A5={;sP) zGBM1hAi_d^W|O4=M602xZIXjFWhnqcmWz^1WwHd5HEda#O98AhR}S2qrP*m3!NFtV z!~n2BPrrM&eNUTMQj*fh%J}5zS=EUWfe}e|Qz^@<*TbX(Yg^DvwJv=hsj-{wSOB^c zWiHfXDxflUb|4trhf@QJY~<%*Qa{cBILZKexHB@l8p&ycF~+b_Mq)t0i$^eFXigtm3+M;|O0il0|H>_1eszBfS zhaHoGSCi-jNF1iw%CLlJA#C)+*^;>sqR;KxfZAZPnMDPStvqF>ga80mndbGmNe50( zGk86dIcYVrCGF^RzlSKCu`))N{<^B8J% zwJ{*@oc&rbfvfE&(u^axr&T`C0z~XjDo}(~RdA2m0Sh;{ZqPQZg2U=j?TJJte-Rc~ ziPf&mfqigkRQ0EA8PgVY7F#HnIrvz`$QSyh16WeF9mmEZNw_eEsQ;@CyGe>v97%B+ zC0LXMQN8{?A9ZmhLl`oE-OAqbSZ-W9k5c#dnV>pYo9xO zuO%2oLvxY`_XMv!4HKJJtJz;Ai2v2z#aIbVv_gfhM88)WW_ulPv(TZGV4gU^1@X#P z5G9a-4en47O~XsT98 zJ1G!klth_bAyk<-un(6RL@CxeLKepE4zskCA?b!Pk-464N$nsQD{4_CdN^?|0LF?M zOvQ9u8uq6ES6-h)0NZ*_+G+1jK-Tsu6H_M6RIlw=vZPIuIVaEV-_m1r!ZldHkCY1o zGEqX7r?9fKS+&VzETQ_p0*tNHm0`~&JDn(#SO|GL1LVNVRm1G4RA%3)at)m*F~N5h zDtEiLY2^D@QWn=jfv;C4)8i{_dn${+c8hm`gdnbYOv$V1iD_Zk{w>?w_T{GXty4fC z7zr#xE8@~hwpDMU)CHR(hiM8YUOSav=((u(?*86Nu3x*6+t2~4TZtvgDbp$ep0IjA z6ST?tMOJ_URn$b8=z(l|Dn%c#=m{=qAj2jGVn^3$LO(CmvW~bn7N)oMtKs^+Oi1=vnQvp^U}87v8qz#JcrV@8z9? zT`y?u0rbtw6;=9FcF}-Q(E77U8e*y=&RoE>>B)PJvAdPM8Wy+#t!|%ZK%G(<9|tKn zflOkeg<@fTYuhjT^5m{i3)2L89nZAN2>dQh{PJ?UaNg*vl~=EBIh`ivoynEMJM!ow zNAkk+v;4%5-;h^-WF=2NekRurj`DH8pTBDjg-sz4Fs4owtQI>hq0L`|W2rqi2N1nL zlU%IU^}V(-PWq1R+5bn@=@-I}%lSvz+2!_7wo4HEN3`Ai4`g%m-m||9t%8 zgIH>t#3TYXkP^U3amlhtQRd>UivMS|FtHG)!h+Qz53Wn|mYu?6EGn|%7X}<+wW89{ zCj2RrFhpnIz}a3Kq|DP4RfnKyH7GYayXqc=fsE{IljTGf%0QZd*O>rv*WSpjTT*#= z&3g~wDu{(=K+&;rR^Z_4@_{zN!7Zff-__{e+1F88MN@3%6No+Pt7KszhczE%h2hJ{HrF$`&dWPaU( z$^f+8OO);mQgx^d&Fdln+3%q%)wb;WVzYiQ*8+XaTVn+EoDUBUtiXdhGw3y7m=d0| zn$)GGL$_vbzus7wSQEzvFKd~vxq6_2>C(g~O*Gn={TCSPXe_K(v;rXmFlz_qa?~mm zu>RG>;>2u|_0IK8wa=!Rpw`&7Bn|i_o8=|Mg4c8xUoOm&EQnq#443Cl>Wc z?$^tT6Jna_?U`+e1z8sMA?Se0}) z=qVu9`%aRg^R@mSibTny0lHaI)UH!Vt26awcl3T)82*_cwPlSf_;vfMRzgsLXI-S@ z{fWmt*L@4y*1deCeZQ?F$rt!V2W*vGL=$zRK00H$J6dt|gIlVZ6ilnd?I{DhxFt_K zekz}PWsui@^gv$y+)|!;bS-c@WxYVFbhvJ4gb+%6Luj zKyg2f+FhI3Q`6~YBUnVUoovlc@~$Rm+NQw*^QzsB|_0S?4Uj+A=abk7gn&6O>3ZG46&GC%1u0 z<~-qvS^k)s&~D)59LV0*A_}hOs{l~9naNdZRIjX{gStH}fN-$SFtcg$v_z-^5P)4c zU~J*Mp+8l7J8Ctn?K$?b-JIh+3;I@K5s;5b*<1aB4%dMoLyYArY-3j~=6T2$tuQlq~s;Q%-D5E$cSUa`faTK`ZCXc~Fhl@b9IReYpdqIGX(uA(;LKl(h znK}`i>tf-`mFxNS4nTk0PDRHlTu`7LX(HETkqVgSt0W-?*i;$N@-VV3BsXMA)#h%? z3V2#_c1pHiYcsaswJh@mBr+?OOLl#;S;_V2$0~8yGLcV1{#vG10p|%lU{5ljJwH2V z$z8KhQ?Tmcb9J~t)i*ADbkU=-1?_M8`P;XT;BBiwSoO7fKNaNftrlRdTUkR*bgFvM zA{RjTc`gZn$O%X_VlmHN9g2y8B{EYtXh@|>PaL>*Trrg}f8P~pgoo{g)dy#4`&Fic zwSNP6(Z}#TJ(Du)LfK^rROO1>LdBFy`**p7OK(`m+?eGMijQo$PIN#hMH473C{ZWO z1y;-gY%vo0dW>;kVM|VKfW6iu*Vpo?Pbz@z<;T8sU0!%D$P`pJX0_2GhWTeaajcXTA;dsg}L=b%^=_A zNhoDNp?WPHWNo#+-B9SSO^4c7D{#b8gT48p>by2%R$WNH*vtD}K)5cILK~2&ytQVO?VaQv50akVWrp+7h$ec(1?Ff}X)4?4>wv|m4PGlax zN`-}B(ExssO+ul_;gUWGOo%X8GyAxegPe$*KLo%-3X}h^~Y>%m)Va= zVT;}JtGzC}4B+nO;H$7h4Vax>JIX3x0PWOII(YSjs6>sPF&wc>c1YS|h%wN947ypb zyrfAyn%O?30vr|q^~<6S9a(Y}QUE>Pj+NY;5k_*`8kN2@@#oIb5sqKMjDkO1EVvR} zudf0Ajx5e>B}v616+L}f3!&=&CSU82O$OUk*=M&rGNRVdB^GwnhFUupx&+j?aOVq4 zE~Jvw#fMkB>tI=B46OAVdLXk7k^)t=zVbzZf?frSw&)p)E*JK7OoRlBt~OIV2PAz> zGZKL`Im`NzFXT+Gx~5hYt^>-N=7q%?)$``!hwNH)cndSapw;FO;$Q87{nWONMWV_e zHkPENl7ZuV|MWcW9G^f2pgLFuwpNAAz!Z{!21~wD*~lJ%vblPwpYkeyj~pT~JySNLg!l+vwwfo3PQuI|6=;aqPD(%i2bdDm&|Ao9o)!&QF$koEbr3->0=& z89#iH?2?2&U~C5{?pgV(b2*rSTer)?I5sHA)xH-o_msu7Xyv$BR>LTV`)hgj>2vww z=aT&BtB3OPXIFCd;7)$+SO(RQwZzWdSlbh$mu?GnWP5NtR9LwR3;Z0?(XUWt^u!Xg(pk4Ta+Ol_xZ z-=BOf?OZLM(H1Qrs-n7CCxH_wg# zpe6^w_REu|1OpwU2xKf^xuRk3X?McGrpG%}v+?riLiOCmKgl|a4*pXxhO&4Q(hD{f zS(z0hphvBOCJldf!HNkC^i>1{tIreES|&NYWD5gp%-5z#eFNC5D*}_|>>74pySHj5r50S#SVDl4*$NF< z*9c_+Xo*u3^>cO%ovIaA_w@C6Rri6tY}N!JQ$M9K#YOhDQ~Rte8}I5rZ9 z=tBY3Vu52(Q0m`eRqhhn?goi{mb5jl*Q~%bL#l(LXL;s9^{5$0>bXzr37|+AQAmtl zglO$lV$b$lwXYlXBA%_vo`P7Ei%yd*o_Yg)3Sv4s5%deQhdWSdh$iSMaMVPeV99@~ zZ`HyB>iv4WK9Qr-I|gP?PXLgQ!N6K!483aFH^Anq1zo--*AA}$xGJj(nSv%n?yZmz zc9uVT>((tfx_vwUehL;=g2@{!k!uM?*0TEv?$HK4%-xr&TzaKZY8%O=QgosgfvV-G z&4!P5RLQ_B7QEK+*sms9w$TIF?X=WsAezp1rIq@YdpJI27Pq{=dQE0(xwW~VSOkiw zFt0&`rK7v5zby-NFCC%`C3i=Fwr$aLmPD?U$UGBlHoz2zazVT2 z4g+%oOr24^2x7f7?9OYRG&j}f{5wrLQT4uZ?OSUFKjQ-@5(IQRD*-|oaO#SN#A7`x01q-nx0DB5P z)ve-+mSE10Ylm0m>Y>?b`9u4SN(YpIeNQWh9UWPlfj4mgL%dp9VG(6?Bc)H?U)=)C zQA;XO5Vw@G$ONjaHTPRe7+d^~b9^maei>_p3!%sY9p;~E&w+*2bJ;U*Xt!xo0E=?Y z#g(BZYDKzRT=ak~g$$Tn8AC0R+vGlpYkZ66v7ZB1Sq4b?U7Z2#4lL`yhiZxktCSV! zu?wGCFJ)^kAy&>-u*h0;Gpa>Ceq${!y)eiZKffoRe|eB+KOu7M%9uZ+?CVM1-v$vu z5eKLu#lIU&Y^vU`QhwRpE`Ztx*)FiQ3SfUed`5of!S;5!J?!lg#QqR%H@_wC{QJKs zzmYrppUr|r)6Q|)fjgGY;et3GZ85GbX}(n;GUoqm@FX8pzrKw%gGU}#IU1X zD^;Ddb?8%C*#NYb{p7{yP&@eNU@vl}1`H zvmihLwO*NPGWHeBY_reKaAsk9%)_!T#R|N&ddSijB9pSG%I4+IS`t{W60l-2jm6-A zKOp?Ak^wkeAcWlmnCkTdux7_FcBbQksA{xfXBPVcI^00MV`)sQ?n4KK*^rcdQJsmE7-;DA-PM~p11TiR*;nVR<0Q;vX0NF2nqp=HtG@}l62TH&H? zYWfowe#zxG^q{Un0lRNpT$V*m;$oU(oH-J7NTloNRI(r#A809Oc6E9`O-`e2rPb}~ z@pTtK>_u-`Q$4Wg!o_)li>1=i3EYF>R;$v;VhRdWm5dlVD+wEeMp+h!E~(};B#VT_ z-pHL`K^ckMVH$HZIX|FZn; z6vLm(UFcAsyIp zt%mu*ye%vMJE}EIgsqQGlv^>k12C(FNu5ja6t@vFYpTf5?rfUbDV2KaiHw-D2+m~e zBqbpPP7p^AQoI(;S5(2Pd2ojUG(!tk8Is#6-^gk-u~si|;b7H~Hlb-<40L2%+LEqb zLms-fB@;^q^UQVrMzig8vOis$K2UXi(0ziH$*+x@5=!1OJPVyDYRQ{kR3|~#5Ibv~ zoP})wtOGktxX!c^aE@d(cUjmAEd?L1ENihB;(s2lb}%$j+gHJx6h<0JTU>0EvZT=f zkOH5?uTUwOADdH<|rT6CbuUAvC`a60Ap6pgHAoAo#POK){>mqnP1%dVF_Ij-gvs=Jj-Qj^|V(tI1>5j|`mZ=lAzaSF6$l1+iE1L%nd7 zk+r^VHe<@dzWL@`8OXj1cGr^CBK77&VQhP^EXBoa2sNn=k_z{)O|bWf8+B_G{eVML=uK#^|j#f*+L_VRZ%m*k3QjxjD~?tEF~Ny$h{a^w9Yir zktHE7pqMa*P?Eu_s{ubp*MK<+r3eOMF|KoA0H1)ECu-_(cH9psBVLIT+6lvdQfDph;Cxd{RDwXX?{@pf- z3(Fvy)s{dL2PcBbO35l?oEVx-4j4rW5cA)Wxlhmw#BAa(0}0FkPfWzYgg|EYuCN?| zZO_bGunnOmWjHKJ)TnyWn1k}Ji#q_;KrAcOEC#Czu1yY;76VaheKvEn3b-N-$De>ywx>vaTK8ksHjde8DG`hV|z67(?Z(uP;EX zYFg`TV7|3p<%;`Z(F~V8n6=Q%{o0~u7!-^tc-k`$QoW`u8>&D9lm2XO3JT;Aid}ES zL*M%gV+NO{Qj|$VpzEY{m6zy^MVg`jU=mN8gB6?XChC|b+54r-0z!!_1Va>Ux3kZe zN(k2!0D-;vF?MSz9rM>TY9xzNluzWJvTa zF)9r%A!MJyVxZF)l~qTM0tUDjCfPuBITSUZ-djWbT0xENLKh*Y=ch>CQVT64 zUQq+97Fs5m8JW}nd$Pio z+__|RzG^+@VM`-b9s6#_C|)<0_Sl6b99EQyh;k--%M4(z+>y_GI>;AaS;~*S zc12!#KFN)P_wqfym5=L=XqBxHw+3L@n_6qghB;sd`hkJi@8qxi`S85__Cx=I%k5)t zmmv0sbSpqMgg=$L()D}x&-=)L9%MtX2~GwY8q&$J4u;MK!Nxj_D2$+>c9q|@TI9z{ zL%%w%c1U^}HD?3}aS*fI=`c}j69KF~o7@_U6Mb;cKLVO$VcU@*OgHqmJO&4)HLl-k zLJI&>FhWgi^7qETq^%ml^$`9=Q}C)|?F*Zf=EOmr$zWbW!Zo8lp-GhzbPae_Cd0L! z%D%=B$|MLDkZd0Guxc;Ggs!zGc5y-*UCXH8aIQ>|0~vp~eU$sdwwhQ^{_+VdHsI5l zBbRPC;C03pS;1a3g15vdS5Fo(*#OGp0-S7ELCGg(W_t%aKAorms5{ft;`7NoM0K%a z{+)yD?Hz#eQ&1)>tN|wiwQ!?%tscO*wd;p6qviXO>@hQaxv&HzOQ?&qC-d{6oCXR4 z>h9u<9oXRd!+i;Gb>)8a)PrAY|qr|H@^;G2(Z?}>T5}lSqFK0u*X`# zA6EO2-q+hlx52u;`R1GQ-n+NtG=pN4F7{#}rJO9w$cjgR=z7wOEv{pScGule#ExmC z#g^K^jda)W1(%hNr7K5Cq%y|P#~n}l^1=^2lRKA8xLa~;D3*HLvFXb!2`r}3Rwoyi zxVRjpqbo@C#VXp^1R0k}sU~_zcV;Z(?g+7tzn&z9b`}EV1TJ;baWLSaU{#f5L4Q~G zMg_5&5UjR$sv}noxh4wEn>+)Y<^{-+JpRauyzuOp!RtW=u?zY1(;K<6crPFK`_c!s z*jpP2POLsCQf#=&_|iHzhq+9C-!~9z3()U?fqhCvA*i6-(d@4y&8^d8!hABv}x69saDYmlx9>ttl0oL7>*@-o*B@*uz}y<70dLv ztF0C@q@~H0CV(AV(0UKeYGG;`=Cox`i;lQqUcEAxO9*y+WL>(!>)*xdf9ce|gJFOL zPmBROO3%wQTE_FxLHFn&*|mPoa;aoOfre$g4+dfe%Pu$OJfOINf&^$i31k&CI|D4f ze#!4u@QTv5^egS%2}0|z`>HUCMIzXq^*Mu#iAida?X!^(l+IYCCMk3U7ziuQ)$Z!j z0q=xN>?(^mcM>s{(3sm;63PaG+Fs)1wJRI=f*B-ws^7-tR`JLE2^7;*bMx& z{mts5tL;b+r@c{-litu`tfI}Qa-r-eXR)S#n&TF)0{Y4hXlx8FvUFa5ms&3$^) z4g`$mQEGrTG9eIm)cb&!Q0?qa!$X)XbfUeMwqq=+#-@)2i_CyorA?eOoW4*oIgJ73 zf$j|*#}HkREf-5(H&!+r?LKbk3muyZgRU2DlOzM9NnXgn`@OkO)M$&d%%wV+M+(XS z>`l?+FthI)$-0S}eI~0eIz*DfV4!#oCh`*63wCKOtNv839iGXjJ~7FwFAwtSOHqFG z)m8p`lp(&88{w_|b22aOd_8q6@N&GAB%`35nKCr#Xko5>Rla{9HlO=%=L@L}>~|jO zjxV=|x?O_UAM)+yH|4eh*$fK*ncPJm`yddT(%sK}@69ZcSPnpnzCbM5%c8B$*j2d! z%7{i~WzX{fI?5ssa#s&lan*q$U+GZTH2a2iksBhGAr_-V5s|9uD_Xq7Ke%tz$q_^{OiKP{vdfxYQ5! zC_glbOOv^_Eaaj0y{AXA0$2s1S5UoePp)0PB8P{E8T44c(f8|T)k^EuJGWq~rG*Wy z9A1%6Jn@PA{%i8cjYmv2lFu0ul6;&cVH1AC6cY43&zTH53zTRZN>c6W=F*<&6wujX z9IPtk0m$C>g)x1Rh5B!0Gxhu|5pHQoab$~!j{0wrBDZn>YFa_a`Fex%6#7%c6(-gq z1U7aFQVbH|u}7i-he ztl`KpG)@k7oJO8&+iWB`fdDq#XCn019&G#V{843K|L}qLaJl_Jwo4HEkHMxu_8Wg$ z{)>FK{^|VZlkIEw1FRn&$R<~H+ULYa6&nDRl0s-r=Y23qJ=hwXpjErH$rH<*2k-tVxKRa_HHi3uU61MsU z!7C>U*X3cf?woy*6)t1V}G+9a)ev9TWBObt-=i2~Pa+D-OK zO)}Tr?Tf_~f{2#~C>B$m#Fbc@l;E{!Y(0D{O|b>;6X2J2LcUKVI5G68p0k$FW$c<= zS9_n@Oj$YFP4S@h>);ESFqXZrU5v37#?bHAQ1e!clFTzC zC}-;}A|sI?L;Xm>K*Q+YEV1-6R(;DO4PLOK2l(qfi3=KBzxTGtEB?l00DYn3WlL2` z08O{*VO2hmfhQQ)3}8|IcC>;FlDwFKRnt=TRV#4q<;Ou@1Rfq(31Tb&AaB`1SF2iB zO%BrqihkdX>o?@lM;1+i4AeJ8S9C*{o}jL=p@Xg3Y64QH8arC=a$sT@h;ScTdiW#;thxPUprc z@v{vh#;jbM$ZDmb!o-%6iWaJM2Ll5cWNqDhF%|bv+9@@4#`S$ePdgn}19E+APjCxq z0kv|CgkhY`xZhkEeDXBs0nJ)=w}9e2P(qifUcXd}9{^-cyGdOWvi2vGZY``f3mVMR zZerKH%F>pfklZDH#y+>5oOLA=yWLV6;Z>vzJ-aT^vch9=7O?gfs|+M`09$$;51W(8r3Uz2yfUGbUoJjCOfe4FEgu>ur5#WBQkO+A7G$Z_y zftI}gcpVkDP^bWpHR*_QiO4d|t~IrV!q8tl?5)Xl7w4LMb$VeoT4$61KnL4*C0=Yc zQ(8(E=hx%_d$iSBEcVR7f(tMJ*JE`zI8gwLV+{$xt=qTc_O08>+A2d_o7P#c!MZB@ z40n(sg=olzQXZl5A3gWBPk{riYGYNhV@#!c7+bLtmYvqz0Ty^IAU4vlqS8J#rF%nL z+xP1tfHqkk2`SezL2SS7w7Q*wSoS*|r-GG9R%w5b(&vP-rs6-%K4jJ3DqAG6k6ER} zH1yEG#Z<)t2d&uwHb{z`U@@`C`<>4ZZYbXisfPNAujFuXDz~m2%jaGW8Ng2DjeP&i zt+gyB$5QoYC@L;xrH|_{7*V2^P;%)E7mNHIK1};qwl%*ET7rj6ZjYMz9@u_e0ql$N z&WHZwuKl*O9fb7gsr1El?7z))+u zD~-qMJ#vu;%uODk^Mo?X19shK(14od7a1_+8^DPIueGehS}TK@ZgT9W7)qAE>(P>j zkcc7_du(jNEE7c-euRer9mrXdLn^O%C|v)JLpaw%at16--)7ZnMjoEQl4>GZQ%!=9 zA+bykTM1VR9STO?jReQe(0VAnfFXXy7Y`1=m6h$gQo(8zFx&Ep^j2I6oLM15T?!qz z1(yQab#ZeTd(MtQt8Vj>_SmzuF-bbZ+EY8|U|9|F*KPy0DG~$QxmcK3vo*2`R2}Vv z^jO!XH^gqJ9x)^)5Y>fILlv)cxHqsot;-sEEt9+=%VZ!9;apvKbUoQ4J1>jHzOq>k z0^MQ_$693}(3h5^ssUV9w9@k9%($lkM*P>)(rs8a`NbsTGRMyTX*DzP%1}KSLV`yA zrcG~^vfP%;D=?XJgwn34B5h?`Rex-K6pOB!MN6ir3dOz-#umw7ZXf0PZ~!b6Do}ea zzSq91BYSKKLUsyW(tb3ygCt=0G>HfpSOVC(#iC%UR$J5j{A^mwwC*w( zKb2X1E{o}jJa_wCUU_yS-}$*oz8ASNe?Pw`1K92)pS&fHO{XM@X=wX_VNJurZD@Rt zkNCM4{IL|#Qu7iX8`-G9Dk@!?b>f^+kx0mSyLeUZ)PCdhu@bk(3c+PGmiwb zjMjK)T?{|@OG6;c!=GY|ZO{p}GLvhtw$Vw>jUZ{YjTqdZjp}4?lB7|5g~50wE8H|W zX`?Jda;+h!2G3*YlZU8XZyChMUtGcCO2I;_V`k2h?Ocah6~L;Q(GIFd zA>BbL5DM!llqU@C4IXmu%~)M^q?@gzuQ8Y%1K47h1#PXCgIyIP`xQNIk+1_DpFU>j z*fY>=KN-S}k-OKc23RxHojD6;DWpEQv`tbs0%&2Qh1b_At*~>ex;(OEk*rgzVbF=* zYuQzl@jN?MM!TadLjkJWjRJv-Cih}ZDm!-GS)a^+AHz;sce zpq$bMmfrUA#xy(R1Mu_&g%oCPTW-N%wrzS@kwHA ztgIFTVTCy>PwFf?oe=Q53E=g);9A*WQ3Ga?chm&0lnq~%%U}uyknA7Ynv0Q+G88S1 zT$@%?6S^7$u~=LTzNLM+AT>!(`^1vAYGmW_{^d%`;iBpkz~_ZZ99G7{22YTa;zTf0 z4_Fj7Sg4!;+E_g|y6`xY6Ed(T`F4DIs-PS8RFDq<#6k-TK-X z(cmN7vH(WV->P-{oEW!SKACKi0cl|2gVi|uJy*40cq_s-O{G!1g&h;f(iKEj`l{w@WhayvOo0MWO>j; zRkbi;9-Tu1VRnh7gD5ep~Mo?eSK`EE~C<{&^u zwjUlBSqHSB0`wi40b7t2TgaWonb2E3l!PwP`91>zXZ@9zRxlQ^1R)rD}*CAhFBC&k9z# z*l`QB!o;#1wD-TnzNhX4sG6oSf-_st&@sXpOV^14fDX5XNf|WpEQ9b2gmHc#BRD!f z(xfhIyHz`_nH*j@)TAx|U_D$Igf_6yI;(4&eWyi?KMyn0#Vv|0#g`kvB zu5AhCIANI9`V}RO(%GqozH*M6tA9fy7c z`a5sQf17?wZsk9JZz`H%=+x&NurazbU`CZvuAQTDUTIWGE$~xI^u&`aHlK}L$n8afS#c)MX z0Fwb;m%Fgq35Cc+FlYeHAYaq6)nvi!TQZ_$c#k0{4Y69i_*ye<+mg(sKvpc9R|^y% zd)`F@WimwQ+e^m?t7?aFq@pCGrcS-0)CbmKupRScPXK*e zFxKW1yINYJRkB5w+H}QGA`E)odIOc(f1)MHsTR3z20b#9IwS1}(R4xR(T&xH#ae8E zpv>>a?uFm2P6Iuqw;pllLJIeqS0I!eAh}kVP*K;EE}#1i$O06o31$hW``MWOJGAqB z4IJjr6UgT8o6YiHle2uxPvxoWYkA?xUY-Sjy>=qE5AMo-^9^a@`}w^Yz=oxg7#vaP z+)#oKMKLf;NF#FLL37sS$EDLJ@dRXECLEu`U~BiFttnX}e^y?W9}X|xS8;vUK1FQ@ zVn5~W&K;3A-;!Vc%9jN~us`6>e-Mbh{P)&VLAD2p0!58^6%YHzzQCeDjh1hK!#Tri zNCMkPn+LX8btbbGzOopE^MtTI*U4qQhDB0u%mCREbxD9Je&3bMda%06$>IA|KWi0k zk`815scX~fadO4S#U~iAaqb8bk0a~Aw^lOKy%cK#OHF7Qnb4;#Rx_ymmS>MEGfQ2n z0vZ`~q%sCCm}JyIoWPZnxrH|^TexDmN44|P%>Aa=C^@?{CLhf1t1m33w-Smb^OAVN z6VLnbw@<;w&nZZy|s^6Byyc_kx!70F5zAo-fIT< zCeKiZ2KRtn6C4M#Ey8EjY@T*Q8Wv_)RoUSTxB-L!sLbMviwB`>CI2RbyjA9aS8D?| z(8mIJs7@E2s>x&zu6;DSG_eInZCp17&N~i2yDH`34CA?rW9-26%Nhf}^7& zmR^STKALmE-hvtbt)6eU?(6;fOMQ_SNI#s5!e%Ef{(?|bOl^5(y|Flk=2@e#sFJj^ty1W>?Wogw**vPmq?n-Al6EV1AyZ^q4L&=JaKI$Pi5fx{1a<={`N|4 z9URG($vZNy-<4YK=J%iH?Xhg^mZxjjno_o8V2_dmJm765>Q~s?2Y-I-YklbzFu*?@ zUP(WJ`E%Dk-E9YAKO^mHZ^^HG^~XNKisHjKaVE zdZPJM;K+qAlG<_QiGH>>k%KF(9A25oe6Ljix?G;f^5P@|&$R+h7-w7Hmep=rXC7RG zhDixgpcVL7sWO0#9?~U~9=>f$W}b#Hs_qDWGwi9hQbwFcfPhLBzsVD&%lr z>mH0QJzyb(@i9u5j#*EJudW9Pff0D-;aDb4PJRxmJu6h} zmQUgZlm{s`LSTe8)Uj?jp}KXJf>~ytC$iEQN&;Ejz!0S~**a->avJ+pRWWfUlGW~g zkuvB9(fX0+tQWuVN?eIWE-F?anlx7Wv~A4BOoQsl3Oj&h-4Mts^SoiDx6ZV^NHR>i zt;Ry}IQRLn+Dy&n_C2-Lal)}NeKJ6aG2xocC@dC`2G?aj-YIg@4Wr{NfqKe8+g)^m zJbr$?7ArL|pp`{Kl;9-Xp?F45`9YU9I}sc)r_aY8ms+;ht~ z+^b(KBNa#v3+z9B92}&0v5*b3>0J|j!D0m1m%#lB1bEQF&RkBgSg0$(bTTzPG%Ao~ z0Evw-846m*$!e@twGI z{+;l0+QHY)NZWzf&rr)i_OE~C%kuSnk^XZs813EH#@DtpGY@!|JQBo~=OQQDfl=d@ z%^EA2=WOuMl0IJNfqgBrwwJ{`nes4E5h@wGXr62h8#fVMsFEXlJJ@gu*(%Q%cBaQC&jNTyATyQ|5z1sVB%ln8TVkjb&B1K3&>BzAoWx|b596e* zF=Ok^le9GSN;iSdlS~*EFStug}5ihAfa7}hL_Y8wkX&*ux(ZGtTX z070Lwsdtqy)S>s|ip?0Xi}XFVPUl|2Ve29sEkyWtEXR$e^Zp(gWEs(cSJBZ*DNdmb zs(PH7hT53&U?q6@U0_H}s~eT&2OFxPct{N@OW&;M^nlMQ@Y0O^N-OgDv2Ez>Y74T! z#HeW2YMZrcm(8(atx}GjE(zFo^=V}?mnLy#FbZkF4BwTnLx4FgjzAEr-m=T39za|Z zvkbPhqF4T`v!y;NxpXDTb|H`f1bQ?hmA}2fiT_=6>b%Y!P$oJr2t)nKm~L3GAq+AS z1+l>kM+Ex9nxtUpFK$I=fFBWqp%4kjyv>G5Urg+Zm&Tkpj3s6f<2v48+EaDcK#>_l zCmL~PU(;!8Dd0UqL*7hOyNld~>7orscnM;4yQ3tZ-c4k=I&p@!gtSKh zwgAuGbgXMKa8{ULa1&7QI%2W7gQbv;YC;L3Jj`lL*uX>@@C}6l-^#2ODW1Q`EZ5BW zK;wS)H)om9EPFc@rr3XbOV-*XoKY;nztgh4c8pkAsWJ%)tc(1&deu19zOoD|fEO4v z=rkAmdvb8F$iS&J{iK*?+dXgb+w_>R(-+G(Kkh-Omt{XGAb+^poXIO!9|4NTUs+O3ggS^%T{q#H`n@*`$+d(Km{gbss3FCSwUof z)@?E(F}^Mm?dqKMML$-qGupNqR+n?@E_J^MLIBGo$dx;OtcECuOCJcr?Haa4bSdaT z8!fnR1XktpMYhzY)3-iv1r00xm*v;;!ym}Ei9cV&|BrFe4};F5-aG=jAu&|E9Zq7ELuP9` zIglrs4U)7rd64gAKB?r&qRqEPt{%>03W-*J?d%T4iAcHnlQ{K@H?7NX-*VnF%LbhBbq`1pcS;zlds)GRVS>c| ztf-lER{_^87TE=lo9)JBc6IQl9VY;8>0qNT0HlPWO$XZ#AeO9irHMyj>vwsfi1NHk z5TL{4??&HpmUet>@LNhk9+T>XAlq2kWZ~T^GI`A;6k5qlfr<5RP4meBqh^IAbVN&@ z;`Zn>TvG-T>Bs3+kP>Y@!q&nk5g_>@D3M_eh3R%cYvoO&B?HC522NSE-Zjl8u@D}| zXnsk$pXyivmXt|O!YBkR$Y%3Bna*a?=4-&pk=nK7<(b_@LYetZr;m|wkFdWdDcPwt zPhSVFUS{?}f%DpaPi^IFt%w0^wC0-ZWMH@{C036KUQbSrm5Bu)R;E6GcK={s_TeQ9 zc0B)WYq62`b<}*{OVT5cmC6j#NWQ^^r~Q^BUeLr7Kuffvc)0c^E$AG~+V|UH#H_VY z*4Ug0SYDRn2_gulQ(jrJ+-gPpGD@C8V6Z3^o%?0Yxan9;JMT$~!tqB`cRx|fIZN6K zmbqI&+LEj!wk}LsU7%p$IGxn8n6+{+>*ebFM4bm-f2G#STQ5Czp26$;ayUDYX*kKi z;XLofx{L=(G<2a{XP4V=ABfdAALUEnk7fY-S3j|x*tOp=Z3kjMqwQ;N$^ZN2kH|0O zh4CNBe;$rs8(;qqk`vs|$UdlO&}2RA5C7(Pn#rr z(;$m^l!L`2KW>%jQ6u_|+1q%^P}0gw4$4FmPL%mlGwnoQNLo9oO95iXVZhi%7aHim z01g9qOo;HsB%`Mwwu*&83t<34W~JrTZ3f+IU}(-KLrV~2$d|$Xu@|a`D;?k#>nHRHAalC>nKM|uY)}+urLBiVJO7V(K4$e_;a1B16PE9JvpcpNFc!S?fV>Mz%;OG zRXs4uF_xqzt3V}NfHo3HR7{MDqX{X}ipXt;if3Rc!1XUeK#fe@?466P{s1@o$(Z>B82YR(d?_0lx78N|N(-g{QmAOl!@ zN6VrF0KjuvRUzx&kDarYV?B7}0?Glbuf0ohB+pCOTKo~j7Kuop)|LyUj-|3$y4eef zXtXKWA`7yXni^uO(A7{zL;qgLES5|9H4ip)?rIRqey4IBD|b$rkxE7)eWU3OZ1N8u zpBhch(nS$u19jd)qXx?PCG>r;ua#P7&DuQfPJ6ksNOFBYgVz_LyzyEiFFn7KXK$Zo zVEYYOv`12P_mp6;RYM&5A0$Qj`!tg9fJCcL0>o~SFk2utJX}hU;{O+7`h(#``CA`* z2X^guWZQw*&v3i*w!CxahvhG(82&)M(4QSZ&v5&o!DV>){`ZBgrw6$~x-^I}f;Gap zK|_LxdRm*fp)-ULdy*>$6S;mkmm4=0d7^7o7_eR;L;0>%;erklXtc_cf&SO)ziY{p zDBKHTr${*%dNYTD4K2h7H!bXG30APw0W&n)vjmY6-WP(veqc3;TFpL71^_!`ird8L zAgUsRMq*MgU?kcq5Ez=LvZ7NWew|6jmUvpWXQ8HsbK&V9%?r{XH26eQUSzJuj+bQ zb&eo&XAG$Am-4voZHw}u3};xZVcC36)Y*0$H1IfL4YJH zB!i)nSO#cF^gCGWTXiRWJ*svS4UCrRc2)k(#gnN}%xpk}5)=bTc3qH})vjrN9%C1W zz>930G&r#Co@_WuQli9#j`a#?>^><|hWD+hjfI3Hq2D#e*nyP`w4SmBO0b2`+7>#R zpp}mWw40Wotc9R70`jMPUY77V3R+-%;Rka5)I@AU}<5Ww9H>|uKLE!W+_SiVj1p65>MVhM-~=| zWSCH3`Bn9my9$Dq@bjbZ_@ky^pXq^NDnEK6vq2H-+aTe7OzJv;$NogiFB(5F(( zOhd3u!|eG$-K~_Q7zGrx15??EFsvWV|F`liZF=d#hD)ms0YJ4@^3#)&(9SMSwU5E! zAd+1K5L?1$_4q{ljxzT3#PpaMfN7x&`g6jM)-uw`WP+8A1BDbk>1K59q62rorV@{f z$ksU?GwfW7m2>QJ7OP~IpRWjN0xO*PH3-k#b;o^83S#yBGDu@wf2&#ZOUsyFF0INL zfaUUHsi3TFYqIIl!0Y*$vahFSSa6yr>>10%qR-j26;%fd5Zh?gs%f#G!gqq5JU?H` z$%z#;I6FDD#R+5_Riz+yKFdIs#RBX&b7>r{_=qlwx!^Mp*_PjfcY4tS*O~eP)fYSp z5=Wrdw`~loU0YcA!KFeHWd%(5!lz(VQPJK?r36@n@#f=c>bnAfr9`rZ>(0)xo!5@C z2fy|Q`o1b9u^mL3r*BjVK~h;kV05U-eS(R4xJK$;=JOzXvz3;Oed6kw+Gc&ni?w|I zrAlrez9ak7Z^*p8m#>-kbMIJM1vwK+5WEWeN>bU0<~=-wiM8(eP`LT%<_FEE6K{o@ zz5gTmrJsxE^2fs~a`Mr>bk{yDZ3kk%4ceWbmF{bAr@#E~{iwW?2b)Ja7d)uh7?SS0 zg;L|%ZmAen`{rZ0!MNQ4tiCUHWNe2)(W-GFE}g*xONL_Vri~IJ~mT{cC0T@~3q*l*Isq7_#B~_wZLdTM~iTkaIcrgx>9>Jr6B`ql$ zEes7tindu+buMW&D)-FnX%nToTP}idUM%!oTU95XFPvv%gV$!SM0Qd=TjS7Atjuj9 z>q^};jo|ILCu(S;j?PxNgnzSX!Omp*SC!0^YsIoE3dM+Prr(;rt>3q<=v zQt;iegsza0_ich9gdX(*Oe=g$Gk^mVs?0O@4-$a0^HT+}T37-1`7>Z(QDg$|!gYr0 zORqO|lCW(wwN=W)d$cG*2E%Trh0mi`RT&vV>a(Y^hp8??s$d}AK@9qqIcW4;C?cY6 z1q`ga4>oN$etJDPc+@^JoG15MEsMCZrW?l!@)9!t$*T(z#9~Qqi!zfFxul?NDjNP< zndQ#iHN+B34V0yUX?GTu5u`*mUIAqEF{3F(R+N2L2GuZRK+l*14~t z$OS5xb&mw9AB%~7D59%Nleb6lCb;0kS{O6TmVLxFY+rPUaI>ZY5=F zWqm41wVarVF=jiz>{|z$7|df76W3;#+*nNuAXWWgCkA3$uV=1Rx;!z) zV}YK-ksh-orj3|b5A%od`P3w>s*poMjdbMkFn#gJ z$5NcN9@t|CutRcC(#NWQq%pjZN%P+@B)trwVc(v6$_{-r42JW00-tDTcT&7DOVUo> zKlNDcHws7r+!B*|)Sx~xeBT?O#ez(en`Tq>Q0>{9H33bH`ps_3lcPLouT<~S{zs{W z4Ay1d-+Zy;g%E~Cim`MXU6@G$ZG@bfzX&D{O^-qf{EUUWwJ1UUPSwWhV#><+ZnQ$% z%94v*yIdQ?3?>!;eZvyPt=esg`3bMB;u+inz}5oDo-HpF@LIA`Z`T(zx0?Olav@-g z6dzZ>xMCY-sdcQ<1w7}Ghn7wVRaw9Vf>ZL+qyFzf=DM~jZvGu!epVSyXR+xX;gJC} zjdWD5`F0R%KsasOp`aTQy*!~SmlP5}NsWn%n>ZR|=BSdXD1*w<_WvCARUkYm)pmb6FlOL@TmBNu1*gJl+JJ7M&VVEk*aTq^?$>z$&Vz4;-M4QX$gS&pa{Jnz9PUp=OLGIv=1Def zEDJp#X~jJ{$-s6j(`iap^WRKN-Z4$=M8m`lGo-EJQ_YNUn-n}b#I&;2;3N=23B_tO zua0V66B$-^Gk(|GnJ#fH7w1c3+Dw#JykTpGT`SraZD~n!12ERia{}10q@aLU_jm34 zK}wK}k?qsqR8Ta!F0|uZ&eWlNiU~HBfIFY3eT2&iceV zTC`wW5{fCj5#&i91GLy8pc()xZH;V5i+X)j_FhlY?2dwRNk%!GXSd49noyF&0Dhe2)YVkdy}s zj3bYyf>3$_+l9ySsv(r%)L^mI6t;QrN0wo{7+u@q3zgC0wDWz`_f}}?fB#;-`%-vW{^rN{rd|6K zwjGH5Hf%q|gs;8|-<|(_>7#;J{d@Gd50{@c$w6^0V956J=Smn_%qui#8PG%vBmpwZ z10{^N;U(K<@Y?2qcq&)+F+pMCJ(J}U%*}~xR;QNug-q@YfG{Dg6zqVX>I#!;kyopv z1U5*MOy*|SgY0Z&V=JrXX6>z3YhR3T{##E#j?F%wJB*AdK**;o4w{2m2z#yFKqL!+CsN|AcM%PeYPi=!p( z6`$J?dt40Zo$X~(lm9N)^qOl#U6Pr4{??8*Mv)Ad16Q{WQ!pAlTbapg&dwQ{q!eIk zR$h630J_eXE2~DK(E136%9tVfL1WAF+txG`sSSRWE8PVhF zOtNrs<8A=Z7ALaV^Tk5;7JGVqY*>;Q3&o`aSn9lem0A)LD@HtOGy4ZCi$x>c3Cwb; zSLPP%Hxiz_gsn5FUL;`F8i)n6sk_>5M-MjiGcLc=J)d1(B39YVE#|s@s9@$6(s~y? z!7vu)*9N(Mu+(x!y|cuwP{tNoY~^dcMPijk0b(h_Q4wSYeb2D3W69C@8Xb>0UgA1X zgDsH>RdgtDt1Xq!=mH$T`ywwwk7a)j&)1E-@k%YPy&(Gd>6_~eY>#A;U!PPL%D$p7 zjCDB!ustnct!s4$N#cj`?R$ki2XKg zNcej5OY)bx>v5W=xbJ=J4|>>mJAS0i^`5EdWQXVdGhusE`i~|6hk<9plUm9a)n&S< zjHqUscV@?hYF}-$l1Y6bdwY2h-cK^$Ls{B9$hK?MX=!cv$U5Qb@yZgl)Dsx<1r1k4 z25q8Y;kKP=vW~KVRZW{FDOy5=Q-?4g?QY2Wde**zH!?4EZEgb&95B%s6T8uyCvOZD zn;6OzDc+#&8CnV-6vqN*Gb8nYgNLl?{!*#3by@B)W_R5RtR`^PN=nLpO_%`H>SWxK zp>FGn{+GPb-C=N3s0Vv2|Bf+p=Io%$NYYRsLHDlwRx)6tWVN2mAIfsBJAx>NgF7a9 zx$Fb|3V)lizk!>YI$PW{6f2@(bcEBfs90?_lznjGj7(?FhxS3wKO9uX<`h6w^;Hrsk&1Y0WY(9321C)8))BRX3FPNNU#}3C4`{NdM zfesPq50lB=yJD|%6apnb13U>})0uD zd?BbzZtE@;qsn#klkG>kSWlkv#euV>RT%oI-Fj+ZZDHPPnP#hi7Pxq-XtC8)RdFe? zh2bF25eKHeefIy9T*E(%pCw*GfU5Tdt&l`jZcP`tJ-r4t%ETV-_ww}ZB(FXv^7+>) z`Hq)@JoCg#uI`;?;CffubSf<*0%@&;0T%WNau`%T)TG!zO}kxNHoPO^mCr>FMjldw zq3F%_I3pjwJ!&IKemcA)|3zlKjA0) z!TsS8Ol-RJ_m)Y4&#z(o{o`$nLsWFm)B)XdpZb24?^k((Sm%kjlX)BE_Vs;v{;6y7 z+|!40oGtzn+5yFAdFp}Yb0=m|8Oh-`ulw8)|klNr?RL5Yf8n%%rwm+a_L64Mz;k&+xX zmL<5}w&C-|+KNo9m$BT8G<)S(Ga+iwrN$v#OW+Bh3cxUFTZV6^GRrr3a)M=?>dhaU zFRCStnD*B*&5dy~5a;LI!6j}R^>+=_hX))vC9OS{l~xtbzr$ z;MI?cPF6LZYi0NVdU1@9QE#B4R$DfN(qY7ZesNxsuqrZu({{?FGFve8tBr>6vFqK2 zEHIqj@*)f0>nths*o`hkDzk>FEh$d8xJhoZugtRnSxwq0L|#wHiY*9>=9=%*b4jNR z4Us#}#rp|jvkk-@E|a>hr%jrfh?F}XrDx5 zz7W1e*mgECpgNz@j!M~CV=)`DwOS^&9ZL1qB?%@K3C876s+PYHKTO! z>c^R2f@O78=bMJ&7d=Ojix|)ilUaW+*CxK9zwmSY4H0U?Gtg5yoH! zT`(=?b7g7a$vXuQ#^MT=TG65g@Dx^d6$$xVELky!A@pAoEh&LbE(NF(Q(vA zZ4nU*`|W)&CO?kOa0gwKY`XQ9R$CVfl_z-R8-70*p2ZT2=gfVgG481*TYRNyKG^HL zc<$AS0jy`G@)9pF;`(&Gmg71kB?}RX(}gXdLgYB@(YNsgAde$~&gI6<1G#zgirjwkn%uayP%lL3j%9UzEX(D_*w{6g z6Kmccu>UuZ_ zdet!Ds@LgyZOI#9$fj4TFA2khRYd|L|Lg#k2X@Dg_nk77YG2gQg z@5}(iUj7@;?dffvUz-~%rVMRM@Y@>mMqumA@i^Jc_r;rg$~v$Z>uy~$>s8J4{bWWR z0Qk1SfOTYCHEE4{UuWeU#HYYeIte5%>d@0-%JhczIqU-f{5Cqj)cvUzqMI zGi*$;0wR@#th7kNh83%nfp5%5RTG(1a?#M@O{h1eZn9>w7v&W-kBdnxQVDHK(c*By z6q#)3)^-*^R$yNex%jN&4{-?sSxc4Lm}@~#2GFG~rl$_&Sg$v|7B9f_i-Q9_=kShQ zudQ;E>2U45YGN6kA#9<;WG)u)0Go>b0_hGITAWvumqy=5UtDC+dgJ;Hxp^bs@;#Ve zWo=QSc50QqTuS4MAh#CqYhtV?RsUsbt4mcot&U|29CusLXo`p8$#Q#CC^(o(iFRlu z^-ZiufqBO2B6njfcBSXCrmUdWF-oegfW`p8srN`;O9}=H?WyOr)@oZ?O1sBWywP)~ zi!*Gi$@|jeYiEM@=7oeTW!jv{^@CG+{;7?;_|!_i@Jf&`er_f&J>SaBYe?+6o5#I( z@_t=tv`R{00%!d-p1}pLAJfZ&@c8zyv3gh#eCRR1Z2I!#Q6Bwze*C5IV*mF)O8Ui$*D`jVU$rD4h zmc@Q6hX)fm*n=PFxrEJKbuK^{dUKJ1#_~dfSDL}+@&r=NYMBDWBDpIAwK@ZtsQ!)< zD_5IhVYqaNb|%FuK8Z8Pma%Mw4s7ytU{^8WOKp29D|c&B1%|@Ao=l=mk#5@WpskLh z4Ua5h+F4ZgEr4N;8>AzOg;xyl7*1h5GAW-z-q*Rr4rTxatl9hlLh zQ!~w7BkKTmoWxduW`7@;jn&8^DQqVjp}AIZ%%}|+=^V0>fvpX7s-CDLPH1`}7vHNe zBn=izz*(*puG~YG8sEvZ)#Csr7EHB*Wq?%xRdsg|GL4oH1psz>eyX4jESCBVqf9QG zE;OsYCF9uBnky8^g~H5aWs&iBTQ@~Fo63?+%ETUB6Uh~mpKoJs4a5!(1B1?5uh&Qy z1A@-vRhEKTGey-winyO_3Ra?4lMq(`@KFv94)wQbdER8-Y1N|)eW=D5E(Wn?zpiA z{=3uc{sw`iuOX{r1>Rzg<{n^(FDv8P^Xv zK3}fZ_+dVOA7CEc+l2G;awb0%UXyzta<6vnGu(C{cGtEV45HtBEB%$P{FwZsJZ${) zGU_9PKI-GOkF$+r(pH^Qsz@6mFdERxRt92W$2EZ^*R+?LS1WnuW-CwM-ji#G8GzL& z9op$WsP84_A)RD1m|I-Q;O!s-jy(Dg_{=qTf_TkIYRUEbOb^x7am?ZHr!w z+E}SPv`IkQ*0cf|*du3Y#o5@(ylNf%MHW&tz8kYF znvr=0fG7-X2Cw><&rew)33}KG6TMsq>)>?^T(?-)5!32G(VrW$J>$=$kwkyMvBGbU)Ge3LyGI-H zV*q%ts;Gj81VJ2|)!H=0&MaFMwwSlSR3M2(!$On37S`9;&pa`(V$xw#`jOjB$*50l zH+3hNw07;N8-*q0DI<C%yM`ijE)95`o5*$|#D`1l^6!=+f{B_yCNK_{^kd;g*%dI@wfou*#O~Sy+t=Qb zzx|aTmA{q;p?@*|sXAF7kbXOX><~i4U#^)*ejliH!t~XFjrO z#BbqgV@ZQrxdGI|Hx=NxrBuVKk(IK$ltaYe2}1_zVl@Z@tqtf|b+XzdFO@llcp{YAsv)uv z?0h4d0rtLiJ#60Ya=B6ftT)*H76NMauiADg35&j8LTL(^*u=$-5X5?f#4NsQ zO#rN_R*#*z2|yar0F|nAKt=c=GO4TAMM{^9-G>|@0EEW#d^?q^`%8K1MlUZuCGy6L zwfyck=JJJC8ufp@v45J!zI#%q3|{42Lhh51RSLU~vhoxdjdCeLa#g`dEU>*GeXRD0 zFtH;LE9p1$MezOMmHxl~a67VV-!5$jVt4JK?Q3t#KYa5?Havu}O zesmBUO0aZM8gf ztBEwuu*FNBnSMUvw)d+8N3<>Z!w7s7wsiD)Pb6{Ig`pJg5Ed^!xWPZ++R>!UtAj6l zZ(mqlFC+oQpN7H3b)~T|+&G@lY$Ew6vJA2&%&1*cUD#u_@#-xp2y-?->-aYVg21dl z)-|=UE#(|Ft-YMVOtfiol9UbvTqM~%ZWDSS)0v9YVBF`VETamrwKFneVuM(+9Q$Z6 zFJ1LPhy=-Ei!TemTk_F{#UwU#Oh7^u%E;F2$EHEHko`o<&>ExNasdPYs}-~I_Tc~l z5S!R98tSUoEOKCy2(vI-uhzq21>aHGVogiUmh+pKq}7#vp?~YKv-4Xw^_J8nQsSbn z)M@Y#E(JgcAoYAoDBjK%6$z}u6$-R7ypXBdcO4zLQe3+Fr8TUSML5Bs%;H+ZZWdQHiJ_$L&I3w%PEk&1QSyEzt z@av&&z7$?w{vRK5FLv$QsO>=Pu067;PWCPNPrv$Qc_NQ3zvq+M?nec&u*ShaS4U4j z;KZM#80TvSG0JXsRy-jhgSyHuHmKYoXr zoER2}rH4s)wB6gXN~P-h9Zro|xl#S?obFS+Ncud*&{YQ4BY zy2MT<>SHmvPc-RhRx;7;xEce9lBvS?;y15XNapHfd46s{TV(*2p{|fMnQdF7evHT~5GA2}B`WXkdGg(5M;kLCLQxxDnW$alRumEZMRE1!Ed$Qv)z^3=7p9AOpwU(;~L9ShivOiDKAZs_SWHvF|s5(wm!=~Xq1*%cj z8-P|!kWHSu0u#N0#^@Bl5;(ioOo1vxtgx^F&}p;+k_@5E5D0h;I3R%ouVI2l^H6?)&fS@kE*1&1i9?$1s`Rw4;*;7I`R9tChp>IA1pg z{026^Bg;O0Bx^{rO=~RZPbcb0#m zG07lyDlb2o|WaD&t(sHw3ga5>v=6&1y_r}#A~ zN*HQ_KX3tiS@*s@Sl>S@*w{1*(dNuC$g7;Y@;FSo|UkY(yy4S`aWuS*^^=n+x6;qDXiq< zp~j+TZKw&YETlNt__*%X_8$}LN*P!)>K~!BqAL{wl5vQ%gzBr|@i-x7%lw@{wxaQR zRVr=80+`GYzGH=~A*}0$^3H@AQ!Fa0(4;521$2E^Y@59GhW()KB z0wNWVtYP=MD#wqj6jn2`g+L^5ugHE|5)wUMDgO`|%JXg*R`wF9wg9l|V=C3COllBV zPFE{-p-k~c$43fe!N4NZ zAA4MvBu49ZJ^NP2vcFoURzYm1g)jWKmP&NBE!f;E);0CEa4|39;&9=SfG}e3<50Mx zW+~U`t|1(OECgfYgD&`39nNu^)>X;l5Y+UrT7%<>7nsOPBh$`<^%eNbUgUA-OeWQ- z?9Wzm`|5?f{M1su_}Waq`}0@i^%rY-;@X8AOipB;-&3X2yp@amMJQO6k6C2d-7wyP z4dqW(Ov3Q0cA9c*g9F%94Lv8iAjO9uf1I{J=vE6uzq{{q{#xMZJyJ68BmYa)$@f(+ z%db6T>$|pV_q82}-L((Z?z|=M-T4Xm&r>H?^Pm6Jx8ggu7yY5I20kG2X&I(`3;&xB zVy}aDAI%gVj4a@11>J0Z$E>a8U{TBU!$xjgt>o}ZkiA*TfCU8})hDuBHr6drp=;8Lr+J1?o$r0p9{hd`3(`iW>t!CqRR1kloEd z!8R+K**>o|Nec@HKaT2a>IrP>+Zu{fguVv7Xa-vVs^@2C{A?Umdy2~RY9+6{hX7@f zfEfD>4KjAQsa+z%g^=$fL1!ovzk&4Rectav7{HWX0zvF6D zdCq)n#Q-dr<_tvNyL(UGzk64XPLCC|12lI0el3Q8WgcO}+>PUcFTTQgc zNCSU@be19QJ=vcg=Y{oBUVN^WFMe(+Z@jRP*Pc!C?5$3&?48K0LYdg5RxSi^)teWF zDVG8h#IS$|fQ@z$C10zs{|tXd-Uw@B&wg~7aeW8_yY+h5Vq~|s;jC=f+Eo7Y&xH5n zhr)NtJ0I``yS8gvZ3kj^?L#-S;gbLHCtY3;OkVJESd}6^C&PVVZTN(Gg^vhZ zO~4LJbRdHQ{aKWy$_ys$N*2>zuFQKmm~FHQSQR_9r>W&4PeLnYbE67xG0?BopEjvK zXEy>$7V7n+A+@-2%N>HCit>Qcu*sSUQPi(t8gy+ek%(=NERkCrdBT<3YuT}SK| zK^lOo=_3h30bJJ>p6v-2Te<){9o?@g{(`?b)Kh8D=-WcO4}TP%-Ft58fE588%l3 z09+g|fGG1!C8MjyZD7iwL)_%oq49L71wy!Rgs7M_7CoV*r}=&|hS2-4u)M4@SjCiI z+<1S8O?XJ#t+)n1UN3}U(F6T#ZI)fGkL*do>T&La{=QWP>0n;pdG{T8_r3Sz==em= zE|vt2y=^N{%dHCoIDX_Cw6oB*u8kcy0^O>V%tg{^)vq&vJ(I;`DTjM!^5pdlUZ38`OV6(I zZ7nb5$FDx!%k{lenbjv5{kJok}cGo^~gS_D1 z{c-ulJivpIeRP$tPpSoGriZQIYMI1`dwaD2-7pN=I?)pZTUaZZ33rE+UhKix954nTju*bd3Jd+N6qnmt!y49(G9&FRWA!5)6ixIxpAzx3)>WXAuF_{JDqb7`%f0QUFSOq6bJ0T&pXi{V>KWm1;?S>W7mgLq#pOKa%0_ zSf+SGd4yI*s|%y;Ut@`BjafX=7RCIsrDA#Gjq5g{8ejN;{Y(HG7yPE`y0&O7hPy?hU60svEU zEa=QFAy`!}_`u?IBxc#MSc`if<9Iszsuka+6OB@!^PZ^A))Kqc%2*o!x5UJ%Wca(^ zvMmlw%eqqAE?ac1>FVJYZ7wHpk7gNLbNQl+3pewQ^@?wCD}qsl(J}0LE*^M%)y{W_ z(&(-8hz#W(?3g&ylLi&xDy^6m(upkQOSyh!Er*LEdG^*)UV5&R*IrEW@-v-0b@Mcj zLHRxLZU($ZdHg!fTUn}Qmb+T$fTp@yDYUBfg-U_wpm`ne=Qem9##OtG0qnS??c1gC zceJ6=eYkHsmsr{VDgTf^9A1(C`GK3-wOxCRwga)d_OaSey(RCx`6c;t{WU>K>^~#l z#6G3NCq43$vkN1J%90W)y$KV%$WyA;tJ2bq#t2%)qMoG|5%R>u0QtaD>il?`pP$7} zb+hf>Mkdo`1|3UfJrJIQ_07d*soPV33 zqY_!_F1eC5SY;xD-Zew8u6@P(oH;H*RGuJXIE`fu4EejOhm&+PP5sG}@r~_CROTBh|IrdWel2U?6 zC>8M}u3%||#30VD>sc%8SQr?va?ix1Zzr?_(CV2dnL(B&u`@E zTW50f@Sa?qAIYM=CzEiNK_>w3TE`w#CJf0Ezfh1OupodITuD-{J9V8%whsna@ppcI zUvoSRTg2(0&>EQA$7vrdHwX`UdY$*}tMYRC;c#8Og6HoC^r1%s|FP2V5%P>?RsTK(!6RUy`^p5Pp4)@BrY5jyHLkkYfVrGtD6(JxM(vz3_yMp{22{h< z#?YyxBKdxcxcfp~^rC^xZZu{#SkjnR(!%!}7_L>&yLo{t? zU73pntoE$9K+WW}-u#_=DFD_aARUi?FJ2DW7hi!g9tGveB^S|7>H>l=CQnItbhyO!slUdgjhoXZPOt>wuZ7a6=B%PfP}X?2>H z+!@fz+CVB~12~6m$z1kSb`)CUhB~=x`R$SmmF7|zF?g!91j+i{gcwFAQ zB@gg$SCB#DVD~&~|9ifDe|S}X;m(~B9N4v8`^aqvVt4HmYhU}h{Vl+I}&~<&c=A2MuXLeE~WO21?W{T8J$SL%V6{FiEWQ1h|$~ zE@ZD+%Z>dYH}`wFIy=v|3%N46kXdyuvOboyI?t1sC4uSG63pbhttXaD1ta*jW+k7% zvbmuo^5_f|O#=w`gtm^Qph90+nN%>B3O4e@4fvMUo!(ce*5A|9_cA&AB9@b;U=a1< zQGY+CumKsbH*M#jx%WMNDLPtO20(qu=0KMLmkt6kgUM`g;IQJrqAU z9heoBGQ{h3(XeXCR!x>+Q3oUng;+|L3R&35x|#qq0LDN$zmb{kTE4VyG@)p3wvhRB zuJ>RAr_DqGj~nSbe7>jQdl>Y?H}?GOLKD8=rK%b2 z$rc@fstGW%I4O~k6WgE9Ce%4cv7!V$l@BacD?O8qLa`82KjL6EZMuj{qpK`f=~8`v zsGOdot(%pTP0R!e)Ef6pMq1~E49*}Du72~kzApFPzbAL^9m(N{{L`^cSYGWr{;4+EK)(5s2 z#d|`MQsK%RFEGNOU)1?nBn8zmFVHIJSHnp@|Mz6kyf3$}oyyaePSA943`tje48bdJF@H=>u6o( z;7MXVx9@t+q|B3r^OXlIwqb*%m@<69IZ12gi`!_GvPA}*sLIwh=FK{kA(g`Z8Q}Kn zSIO9O1yBvYD;92*UpQx;aY;$4!U{_t1dP#PV%BK_8ExEvCQiuIhbB#fO;X% zJ<-W`y)l#D{e`RYyS{K;UV1jlje~R9pB`m^dz{~YqKTt1j~hYDTZ@eidM-W5Z+qN& z*!WghHmE5N7J9rL;LOg=NE%(^c;phz!dv&+e60C5Biim68et9;KOrL zLibfns%cYgTECLEUdwEn0qm7VZeE+ot?N^{eLde_ZR8q&?PQZdYh^LteyI)z8je<7 zWutn%R97Q(50BEM-&5yf1Jj+*gF!XPW*DP;8^TH7I1X$SpLb^zae*zj^+4tsmWW-WF}KA1m+7()M}^s@pLZp{e79u z_H>a2J#1na+dOt_dg9i~tUEWT5#PIxqRk+AdTN3W(FVvgW z4}{k@f9FH&&aUm+W49fM-L-F8yYrTu{nT6XOMmLe<=0{g-YoYa2G@@#>FXq1B`d-zJg4t}$+Dmn+2vv7< zgOf0l2N-CRbcsg)*Yz4zWd`TE!2 zlcSS!P2fU*oMfOn%OJMRz;)7MK`_$=!hCO`Nov?1WsfCNhik{!bgffe$XQ}tt-u{X z8{c($aZZcb@)F8A=#7mqh? zmDe8n%0u2C#*gED*YD$eCl=k5wftsYTKp^RtLdlj{25L9-L+l&rnDW1-L-Fe`>CIk z|NG4!k)KbgI?O}h_gHN1ld%MRd?5S5AolX0O)00rK}P8<(im9Y)Ut%DJS{@U*5nCk zj*_;sm0VqHR#m9VR(ki0n-OsDhOk4HZL_BM=)sc_*4hlCrjSu_CgpXGj8; zYAuu~j)Nwz>y5Qh^%+B<02AxgQZ<8<3`wNToPIzuR=IB7kIG7{!c z=_*;4m3apRj?W=*CuaGmsATM+`abnAwq4P9r>3TEcz z8rM*6ddj0|R5e*HS|TKvUSU-+ScILOFXi6d6M5&oyBWydmGcXfL2flMtIYrw;5EjE z$y$vbSFo`2d}}JxAA5lTy(wUYP2F6XRCvUy*0nYNRk~dzA^7&UQ`P}nKp1XN2 z*Y@t^YyY0iCa2O?c~Kj1TV4wA8&PUh224)%Jvc`eFQ zH|jjO)pFyYlB@Gd8Cy(%b+wjB4bYM&sj5>3$ha|lAAu_@xYRGmnVF3;*vr*<21-kM zUD_nrBk@ETqFObBNLIoRLZXpZ*@7MuOc+9~kqH}(0TzVWr0jg$)}Pmwi-n}ZjD5Mvyp8}k=!qP2#4RJ&DV>Nsb7eLYfv-Xsvgw+Hw_)-b^q~G*y<%Ljg@N0b zNwJl(quo)L?$8Js`~Yccm$lr;$q9-ZyeIFy`@Xz)?>K|lbzV@l8SFNiFolH`Kvs=K zK?b#p#UkG?GPs>;zblagE_|E~-#^%w>3k;he1itqr2z)GErAU{wr9ehTF$8@mw{K+ zl((%VL$Vw+0f^y_DsOq;Vj zYMcwSKqyiW2WnllwYc1+_k=CVLG)Yg-|7+4tCGL9#iIHj4}!gqcYC<*_2IAOG5!C_ zul!(mee<(-{b^k&)@$^Z1eGe?1$IH z24;9838Ys|A(E>AIsk7PWuvUpktKjtP0~`3DV=8UwUjnbTzhdPi?)-!x|4lqXy+o6 z=0Xc6;J&FYGU!|>0ESJMlCB6_;`_izDqSkdE+JgH>Hw&MYWXk$tB!PO{*}13Ri3^cQ&hEllokSI02xqVefpi!!InP}L65<%+NUQ3h3 z3Y+R@VPIy(RflOPldECo+O?-msdDCk(d~`&^-{*wlAqKM6yS}vNt#rZdiwKP2zxf2 zD7yz8u}~r*9)iyA>6N)PQ0f)9z-Bvu^?j{aH7t^WO(P92=?s+~8 z5aZomt;_V6t5qi#=POO(dgtADv-4A5*;ryBE zwf>LZ`H%bG_+-C%*LLmEZ3kj^?OWdN{G6=6_LltBuY6g4IS+-uKhH(CKZ!y2gSYW8 z=K`;Xe!rtZBHNhC>jo0^FNU41GLDt4RJ$9x8r1m>bzkR6P-I>wIha9Ku$G&LohEmn zQqR=|!0k%*T9ml$WddK?V7{Ho%moWYBZR!61r`!n045Mp)r|Oy3x>bN{4t|}Rb^&l zqk2RnCQTK{PGrK2Z3B!#Qv>uUgrtdQdxVm4)}==et_*IEne6!l?k@47OD)4pFJwL95%?XcClFo>E&dv2%@3 z_@tQ*tgjfMnu@mBw9Z;Ip&ydr+CQ_@`xmF2?Izh0o2cq!^lcY{~ zObpS*s!=eFeKkL56kG_JJh|ysyq)LQAVFA}9O3_O@7;qe$?o#N?_}nE*So5_y1Ke& zdS-ftc}f}yVQ_5tkKI4Lj@S(oVXY&KO@Ppdwb^BXB@9O9Ru3{3vJrZqsqT>h0g@$Q zSrWzwnTKE{u)rX(5|WV#3$M(KG@9>udc4H?&+RBzmC&cH*em| zy!p8Ko!|L>-zRr0#0HrGb4<%5Sr2v7!Cbk6rN#g&(_P#%e}w?-rtn=KzSk0f&9J)A z!EEg+rb=6=vNf#f3dKmu*o42qodwNaXRZUG_lSGm4Zyl716Y2Nj>gXrj2|&kM&QHpS0X^mRD-xr4|wL$(Sxxz z#Nsq(W>W&U3}k_+@(@+iM#=W*#!P9-4q{zVLx2M=oSl>6G6H5iq|6Wd)D=Xjq>~x| z)CpiQP!evS6O6BNWjTw+m&yVPO0Kwu!JSmQNB&_tsVp!7QXN;d2yi1ryTIyB5L}eO zD$etu)%)E0{V6-o`^4#k#sx{M$bne}qt4bh@>S^3Sj&|`a_WZHJ&1L7uO(NE#o@EO zv8$HP^=w5;bm;iR3OpDnvWyll$RJkup0Qfmsuu4^U<7}*-QHFp+wF<$l$aejdV^Hh zXRe5k!A&|E7iZCD%Y6QlQeNDo`(z~hE*8X2++TsZD{e~1*1sZ8=3J!407MJqM5YZ2 zUX2^R=7O$Hr;n|z7GAt^761MDuVJG_``@GQKT-?ejE`ZZDsoy2OG`Mpyo___?!|)- zoyWrbA{zC&fJ~MQ<7?FyX|Y-X1JT*p83M6Y0kE8j<@3$i|IB5hxxj0iju{`{-atpA zjk`ulLI(rq$CFUfW~ZrM;|w=%xR z2+&4Z7bSF1kK33pZ)35#g;P^4Ja8h#BWHof?;GHORRXgo23Ve@W$67z(n2+6iL7lg zHi(%%0~&TBTo|KSINmpQ1DkH>aBYJC2Lm^ra%Daj)RY(qI1Sj? zfk?H5b-I}PjMd@lg6e=R^aMNz^$G?P#b7(YYcAL2c%&Y*%4-wl#bu?W>{>}F1+1;h z8l((po!eRB_?9e`Uv~37FLlh3b^FBoKl1!T)qAZpssf03s9AJ*VVuNON$q4E{lXML zD+5_Ti`zzyF2Kiv`M10+Iff=uvk2~B@b+UhKj7hU&c==$fK7*~bU+eQ$}`52-5sV* zZ&X330$45rj8J31=jb5{P)AOJZ0LZja(|`ZORDxq@_OVlvpe;BQO<`ztM~#yI_^+I$9(UyK;IN=g&WY^AA0Q`Po_Fy7GI> zyJo=HZnuRS+@<5ibeL(8;i~Dhp;Q7s$9!zqk*-I8pFu2Vvzbrq^>WMJsI0Tg(|-?d0)ej|&zmqC4w>W^t6F@Vld(=>31VimrbYtKc7C2=jeCxZ_upjac|?uAl5p&M)(q=q8(^4dBP7#&hRBN ze3;P#D~pFYy&&cxf0u6KLavytZK;*x6jkQTR#ME>Mwp+;usjVcOpP#CA7Q4JVY;l# zRO3t@m`J-YDNFN=q^K-qk#(``V(G6;9H7dTM5Lk0C9MwU;akpTv6GQxZ5`N>VRiRC zTpi0eatqYLG1YZ15>WvSG{)n!-XIM1kx`<3Ir#xCQkw1Pwh?qjdbTFRZR zWSBYencGrc!7f=_>jC~yhxY+lmbsa-yvQ#lPRYS(*t#sJ07)ny%$zbnu8-@idpUE) z$5TO*+L|gqcKlsEa1@~Fs0mB~@o9T0W=b`Ct5&6|P3k4ZnD5Ewh0j@JRBeEru!eng zT+{urvVUshtNWh0bLI=`QfDCH3%dlI!xwmIky)3v#jadeHZ~*^`}J>JMz`M=;L0Qi z=81}(GV_+{_{}daVEN=poV))V?t9>V1*QT@m5)0h;C^-UDz-b@*xGK%cI#79Qp`0y zLx7jSET2o^;)+%6AlHlS3yAH>j&c@SrO~R|+%}XpqKX36x!C#P^-@xij*pBuFm;1Y zZ6eAXBjKke+#zJSJKILhY-5TQ1}a-Pv)IS^QyIQ#)#Cih0FRu`@bK9T53ZyHYPT_4 z+d!?v(pX(oSz?dYSK|BHsDc2D$wy=KMsXQeNx6?O(K|$8#NXe0JMiWh%|zVgwKy;0+kbL!;crbghRO=FHMO_EWD#9hB`BO17i6mLQ-r>L49H(}e zHI2EBBFDslGFHvna^$EE4)ltqYwWVIoS}@>ezokfr%CRprNwK0_&Ci*l125@f~4du zNF80l>%JN@2Cy7HgUOLOS1cU?*1|TS7r;4&8Qp(zts;QN7O5)c?+eZ%9sYMi-du>M zMRA$#-$)G~I3g!V%wUu$2wcP$Nv5|XMSe|7O;3?fdK3|JoMm$ zn3FJfzJ6C2c6O@KMH)mojFot4f+zteYy=hQfL*WE$Rfe=swy; zBi_P7bsNi5Tex>&8xNi8BKuQ&(Xx75SROdCs$7Urs3sF$~d3MAkcIx=IN!<}e4 zLsd*X9mVRfjTj>sqxX*Rn(g?W+VykfxY>E<*e~Re{>&~NhJpr6xYk>ldHZXoNP7S;{@ z$ zs3!z$D_sG#_3{9<(m>uzsDU2R1_RNMWu?$0c5XgqT*>bl7;G{?)6ZwQG*%`OUy&34kK`JXVxw%V znBkqZyryvEGQ9#?Q_aGTGJT!f_Y%BBb=SA1&IIo)m3rg3~zev8N~Kxyg(w8`#*u|C3vrYQ#XK*}#Rec|qMK+Ezs|6_e1JezQ3m|pnCy~=iRj}#` zri^n0Gnzq_L35iQUdfoQf6%~hI#4wXi#oeGnKTIIx+b4yXLW*|;n<+2X3yCN&i znmSEZ2G(qHP)bB--)Ccm?<$2-S_CDJVef_ z5dKWaSn_3wUaWDn3 zgFfo%xCS}@+XHm>nIY}ZPf`E)b@8M4&)2^LAwt}m2p~4ZZ5>XI>$lckkB`vr>uYJ& z>D{@E{hEeLib)v%kSSy>t{M4L#tq>)@$p?-G=*n}RUoK9OrIF0-~yF%Pu-B9bzE8H z`K4le3Ca90|FSQ-v#dE;jyNkzfVR}fbh(Fx`T&cK9!}KTI5m@Et`TFVP9P&`-L}n(X~ecv05(nIOyr5+!K@T_@I#rwTP)v5vZTpa zLsYVu3`i4KG{$TzvS_}+jh&ghZSx0B^gN4EC9A^!s+C5U> z&BNY_bMn%me(f*}9p7^#8OFY>M@0QI9S-O3RA!xC?#N`R%X)GalJ}edFzbxQMl#f~ zi1~BQaa`b)a)H+Y`#=n2xuxQ0id0-odGvXAw zYC^y@?uiX77hzha>DYmQQ=$AgI)Lkgx{RuI9Cv@us-cL_wrUZmb_CrRKXOj2$(4O4 znpdIFh<3fdv)TMN#Q!XLLV{NT9{=w77tzGpwF5e~1DALI6I~vEv-}9&$G^Xa`;iP0 z;-!fIVnf^!(Y%Nk*4A<9$DYCl{qS;D_PPtda-JD*cmRtr2-x`u+4YDiaMhfWtUmRW zQK;Ow!m<%|Ha}?VcNT&$xp<3>5C{&5!a-zvQj}65P}_*xm}1GUavO`a0p?~B%ufT0 zQv=L5hN#B6Y-H&non+EPd_5hAbQil2MJy7}+``C7XX&`U0yYMrv4WRe#lY>YWNV!U zs}b&NqC8&_ua9#nB#SJUH{#6iP%Kjkyb`DzG0+?h93OT3M7F+o&W<^Su@s{z?@@q+ zi5-O@o-jWcITZcIh6wIt_ju?m3?>Ag*@YIR3A1k*m63k zsndRCYJk`YJ357k{<~r{E^^WgY$ion3}PA7s@|2&dBP@#KD-~A;Y&w-$=V8r%$Lx8 zVerRv32vMb=HOv_&LoR|s;7rQv$7%yXO-f@ulPK0q1bl2gU!|^u3Wu}&5ccw+8Pcs zF*rz5lN+CEu~;UVNxCdBHC4yyl@+WkGl;EH=O7XQ+u7d6WmY(N{smdz)y-|um}NVy zG6DVBxoI>SRa7dmtgqKkh0@UJwgtfUhJCb|6YLZh65(@K3G^nYmKpf6sfQH~d5o4* z71+dFjrpw|tj_hZI@=@g+7^|8GYc)65|EwV#EGe^Sg2k_1DC0zN1u(gV+i!<`wfH& zB(}4n9-!Bx!|ghtvN;GxfuGOx18mUC?<+r?J@xdv@WP>XAVi3lI0A?baYx04b$n*+f5*iv zi+JqvHQ4hS-ce0I0bY-1KMq>WVK%YfGC4ptF>)0IV{@Pjz&${bXCuo5cq()>s?|s+ z1t;c!GbbaQIaS8lx#cbr*$OyrFO*GLguIrq*=+7_6WQT z1>9w>xGZy(`QF^tXGq-_Rz9r&){1K8IwFHu)H;}#5df1WeFx+8dSlVgl_E0&LSjo= zcAQ;Fjc4laYt(L*F+d^!*>Qt;8CqfDK5$VQxMkcD6F??=pohd)yzU0tz`p z%IX9ov3KR~Z3W**cfdsX`#R*$l2EJ#%=}9>4`5d?UPgzg8552MY&*-&Qw)`2nQt>A zz*=ln8N9lTdghL$WL4?b;ps4AJ}S^^_p#CH5{PYKtJOlM-xE+d5V`^FPt2}$nXJz{ z$gy5)y)lDRr&h7Nd|HdPXdcUnu_D2XS6;*mm%om!_7*m~E$V)yd_YmE)i5)+NC0?- z&Ot-=o2drij0|U+&_*6#}peThx&wz&f>o+1eInt1T&%EJtl5=m`*xy*!@L5fVozcCK(JXt3(M zJfXVJeY{Jv)i2|+uH&`g_IH`-X0Irznwg0KYi17 zEJTQ#5CO!7xZ|SvZuBm!+fV$+Q}`u%w49>f>yekUEQ2g}J$D0&{~M zC${9241D38m$s6^ww$wTa-J|}HcKwM>g{bak=$bI-(l`(#?DfeYFUcw7+iOH0|k~& z*Ovi1XHj{1F+s>Qvzw+W3C3B~)D8DDC>3cdt3&%@Xuug%LZ3`i;A3X}@U$^*{cL`q z1%6CP5gxY5a#<}=F|cf7%*q57txivj4X$o%VPk7c4H|}4-p?ZCgr`F#Vx8Ty+E|$z;OyKM&d#=QcA<^a^V>9S zVTHi!LhUlOs{~>@s3)7Ku%4@7BZ3Zfum+B@V;$9Je9P%!bX1T%0O`*8J4bVT=H9?`yktt_os{~fl&+8NHU|eAIXRS@K9|$p>1SUI*PMwqHQ{4J}3*!^DuZuKt%=ahqP`6 zfy`r7y;H43Vp@WcA#u)VmLwZ2nPE2Z6~M-sM*1w)QwDSc0&qhW`_G2j*qfC0 zF(%YqV%{n1%t}#MMckY)exS?49jR8>3G{W!LDU$FknSIe)DyP^T~{Ks8bfzpd{$yCXmr~mw9f+hqJA(^=_j^om0{wM$U2FHQmWuqD)goD-#BS6BW_l) zFBn7QT)0Y(`9rWlTiY^tmEtqrhEM z$ppaOw>-qX%LA+|_Hb&hi=_&I+%jitw^5Io-`dve6wLLej@?LX8ss?1S#aOq(?oNG zDsPT0JtBYCSzqXP9m6~rZ>xCI?w5tlCDaZ8xQ79fSP+mJIhiggp5nqRap{tSxIH6)*bpzPX#N7S3m5T6 zYu}G`!U!Gu4Sy{?p(@2?_LAJX?V`vZn3!!8i~!@Jr*YyCS(IQDnOv_@vq;L#G%iCI z$&~GDQ*j_FVTi;IP_kWABX$PsW2!>Hwc1z5usQ>du5fAjb)(V|dO{;7XO-VZ%Wiqd`~x&P1(8K_h2uV*(dk5XO>Mz7WdWmc~L?u$q~S9jF({ zDP4yl@XNANQkn0n^!at>^kJEgs|9jA^c_oM{%a~(G3D#>5WAQ>SmlgLy_UFJX`z;? zD!9K{i#4+X3NuG70JMmThv!A?K2sn(=ntg8$%uhLNrufUVy>VCU&w5(UNMwxWuHEy z*;SvyWB4iBvKaSJto0&VN^?ku%8!*|u#qbo8(N`l@OxH~CS5K@4umza^fQLesf2@K2A;(aGl%6nG*xtyWGdwlRd01 zwy9Iqq4%jnMV+YGN)Po?4>hMI5M>=H(25MVp(+Vj)xNb>ncOlWkoitenyU^-W;`oU z2qu0D_V_(0ykGq7*e~{~?~*fj$c*k`e~+$3PjIRVE!yatlNr1&dBA>#e;*IE@(>~J zkO&|)#LF_8@5JE3I(~2MDLhL*^;7ga|D-E0zV7VEK1IiS#B=@{*&a6@5uhcIk+f08HsZ7eJGx43lMoaC-G&_!sLT4osf~?# zKQX_wB1!{BTVmC%s}xDW7&nuMRH$gCY6?&*9>>v}lZntU24ux~U92MNTV; zouig24%pbLvr?+dd&2<+v-}wbJPL4A$J14#g49(x@n|&D7rf~~mZ_11QXh>N0JPdI z0f*IUO}1SXXE1G%D^wHxi zI%l(`^W(C>R=LB>eT5vWRDk2`bb_gRiKY=|>nQ=)6jgSXqw8XBiaJNwpsV&9)Y-X; z#n~RtoQiP%Tm=u^TgH7W5zd?(V5$Bh78;i^U%w)aGp>t`E?r|?)VSb~uBRwv2fLA2 zG^^Gz-w6a>qf|OLs=p99g$xzJ@{-0(tIcT!CbFa?i0rz<^3!7(S z_gOwS{V2SDSP*;Qa)xeZ;GKl;zdd>-e!cnA7#(V*Awt|?5kPE+mvuDP(OFx^ul>jy z@d0`iuh6VVcP`^IEXNVY31WTtY%Aw3tt-@+6#OX^bHrAS2{d1baZU92Now7YDrcp{ zWhffbTSEm1QKtM_!;({}f?5Kp)8!0PWzLAD1Y$ER)`kQ=Mp&BSVcrfFr`wosY{}mX zQ*4d9EyZIE0=C>LW*cSOA&}ccHR_>8>#di#lAoQe2rwr@l<75_K2^8~D=|*qDsr-3 z9)OnwQ29bD8Nz2k%KTcVjF3s@QCtBC$R#q(cq+irIbn^I9v~b^abbvD^}pJ(iiDNR zaEdxWhJrzW7Dvf&d8FB6)|?fw`?|MtNUhow(6VrU%#vk^W_LMb%9IM{ zV?64oDmj*^J}FD6CBg-c6p$$lmU9~!pP5%}SXL}x<%j-=fG8J#anT&@D}z|8Xbve; zOq}afzn_WxSDU~rD-*Du>*bAY^ajpOSLh6p%W|q#aU5w89$#PF>6l;aVQGo3hk1*| znM`D?D#<1R*(<15t`M-@BJkS5>T-sA&&0U@OoFp?7oJ+AYiE|OA-YzkX}Rh07IlJJ z1X{PHAgg41)X8BGoY5Kgx~$sy;X+0&GR)Mt!6FBwk=Jz1S}6!0d#9#6-~CO9q-Tb#Or;=)YhK(6QRIZ#{R&wAdX7zNlOcxAknDTs|zj2w`K z)1&pgOE__S(Sc>~k_`!15TL~XRnsRB+C#&rMAtk4*qNy<0k0?L23TH5v9ge1bt%H> zMd0*OjMEDSr{^P_nu~C9#^U5OFkMektqw3%VHYw2x5+@n=he(%fD+?Sc84+w*YeU z$g13#X8X9HOw|ai)0S1|s7&Y&43@f`HioQ)>-1wQtcDP&CT*oub)4Ul&!-RW&fHr_g$DLx*OjTKAY z%<*Vd{;b{BqAo7*>a^Rc5a9-$nSf9!reF+HVL#|!hoJhi!5V0AUcnKJ{NJ5$5C zl`;X=0p@4f9(fZD0;i4YRm@IrW93AKQ%fn97Fgn~N9UX~_pF4mDK1-OyG`5biy8*g zT)0D}&R8+vve3Q&&T9y3H*RcC9N7YZ6&t!81|t~XBG?1OUh{q|&bo;Vq$7e@{<0s4 zWBYgFC+s?b*Pu8M;;x7QVnf_j@$@>rap599@MCYp$LQI<%wX1-Yd*$~6gTpq+bQeg zhyIIGUCs}!j^}eCRuZx(0ASyQPMJC&tFOKMYUKH~(RjO&%lUF;k7T(d`^A52<4i5k zB9PV*t|B{x)#47ON?pv$dQv_^Rmol7L&Tw{dgwrQdoS4gSa+cno8DMFu zPawFDS?1uD2biu5(1>+?)rbHs3)H7=M58T2UktQ51lX8w+oI_<;kho&W4>%E;Fq(A z>dY0ZvH=4?1J$^NvkrBKM7Xt1$5v#gSY=?)N59__kf(=54FtFt3(jqf=gCli5}{1n zsW9i2UNcR>J2vq&mcBtI#c5)9D-K<;@RAvDda%pxT!VgBs15?ntO9JFiW3+EczMUA ziW$@~ZaiNW<;ezK`M9CiWjt_C3FpsLaQ0+` z`6;FmbOpp#%Nx`)`QpA#gE%( znm>zg+^qcy5#n}?0AfSjm2u%2d@TpFATWz@X7*Oafs{7vNXX{P79Oe2X{acpzMtE= zJlvfl6^zX@Qu*=)A%vXI;4+$?Td3T_$%Sn+lKqHTiy*rFMdq5az9@A)F$hyUfKt3~c8!oSLVTGRM+i85X9wv3Uc1uGS@x)fLcO ziC7DF3o%!y+ilUujnNTsl$bsNr4h<;s>9MUG_Bx`tJsZbs}5zDJk9!Ac*ZosY(Rf^ zF(8099Q0&9Yp|xgEbGoP-&Vk3D$Gj;$`wM*EcP$lTca7#NGxMH(>cJf-;t~_vkR)# zGHR6)Ezdu_3_SZHlO>{Iynfo|h&69h)tnU)uBZYqk5IfJ*ew88WbRNn~^N6?UcwC|5v`O>&SXr`o=v*1E zdF2AW?XNB2uYdC#Ui+#BUiD}VuY9D2#~*4?o5E`zU!Zxjcj)b z`k~?j&vvX?BvX?)@qQhD9)kxc{eb_m4?sp>Diu$FNK_DHo?qT=SMy?-7$h+GDn|?? zQ&j2i8Ue3zoJlcQoj_-NtOSO(1t0)74GP)VDC(*ucWnCg!I$ z==F=#w#AK%Gq(%%4i*|cEKUz-I-uzgGvz+5qc7RqsS*!ccWB#NsGu#tFA@%LpQe2R zhB{4>K4BXIgtjAD-ZX0ym~7E>n_jmy1IwJ%!9a%W8K4>lvXN%68MKOht#B~emNgMt z00ZAq+M#39m%mw4m-)VI7GN114|@uh(=Kg8!KqD-Lo}1m>K~gr*=JBc;BGwQs`Q;F9um*_45^;b<81-FhPc}!fY=Zr;4-u8_?5Lc;OFV_R-;*uLsB<4FAfB;4uDz* zVvE^WCr`SQI_tt`r}DI_`@67|l+qU*sr=Bjrxzf~m*?tsKRj!Ndf+lgxi5g)(!vb_ zi^sgYkvd1K%<*egl|Z5EDQn5Xn{TJgt`o;xfjWy{Xc+C?Rm0tF|yx z-6C+xmFoinvR$zjJ;9mW*%YVdM_8G&l1*G)WDag7^H0vQWoZvf1auqK9;T?(2{hMB zo2ZpqsK#5g-7Qg9s1q2g(Uc2}stIeDYNdTG8PNW+oLGxM{}y4ZEzu}Vur003L*(>* zIMZ2AhM1~nTEIr1PtcP0%IGNXSH5!xQMx6ci?JoyYS|tG@9KyE{7Ckx#zOvdOd6uA z+L28-KN;wA52(uKSB7vJ$8XcuES^8NUGlsxgp6irfn0=uI% zg%l*DoXlMCoYzF>4-)1+V!hbCm}}mQq^CG_YQ^-uyGHMNUkOFdLD<7+RAT}!H1K?( zN?+8PGTP{el?T)_X}+2%AsM)%p?qi=eLA;BWX?++N_$y2*?tZVI}7=&?<&W5sVu_G z|6+o6?RGV9@Wx(|P5&3_Ir6pp_+B#-{*S&6bU=gyGfTyXm-V*5}P2TsygqHQy8 zcHfj~%4yf?3LPqZ3N|QKB>a_U=Y9Y@E#1g-Fg`G+)mUp<>^5%~2Aul>Pqy!g54(^7 zJ<8f-WTy*f8RE+HQ3_Q;V$uOIzhf1TWJS9&)l0=r$>4RC&YW6)fHq*DuEg#ceY>>6 z*9#z9nK9&z^1!s}xm@P%M{K2Q(8L!#Vu02Mrj&i8S`@jppfECZB?v4xVzMoK z_9)q=X0{h_fS5X|{|NRN`jMgM?I6%A3n-?^7u;l^nL+kN`#ts#XGzWZw$WXG4^U2a z1WHDBRdzqYOo;82A@?Smp5yD zr3RmQ)MD~^=OjJWw4^dYdF#C94lm`!J-6J4m5lxuuK1y$@{2|9GM75%vgeS!6@DJu zH8%?PlfzoQ!S%b3g?S7wj_AgCd&* z>k>oiJ*WQHA>edE+Ne`;bQ+VTC+u13(GIUIN8ti0&ayi>YMwQ@x9aMBU}z^4Gn94H)4nqlk>tm*!pQipz`pPk1spnxYqO%&_>!j0jD@h zkR^^^D4ltScR3`g>(mCMe98j&?U)`X)20@Fr$w<|qRYf|ho~$vZp`4gk7zpZWKx~5 zB2`J|$y$R@!6(ew^0Ns=pY4SW%4_iWJLYf0WM7!5bc8gFb5bAmP8k7To1mKMJ&QNU ztDe8X8dKmGO>cI1Iq*(1-Hmtp{zK=jN-cw&Mu^{gbeq3_5*s}veeeji%5gA%fLK`) zBiEy8(V>Bda~!XpJwP&45cb=HF0P)hml$*j7cLF;(~7Hlj@n^ zarv9pk$Njjh}aBLTaDh3AsSHUJPP(l#S3MrzO2o~*7MczD@DiHd@Z9tCUVlctC-%O zlwKivU7x(fAJ4$bN`Jl-b&0>Uz=J*@eHXT(Aa1fkp_IJEApt|aQZ51&%&jlcHU_c223g=YpHvwbPe`tine#Q zAgnB_K*Gu#+D=R?9SS?t4>sVg>0O?nRi$U)s5(bo(*@kr=}YkreTzQ?(!pX1q6%G8 zZV4CtwBo-zCPZ9n8C~uj3!5c?jL2}!Z0MwmP(R1w}HikZ*8p-t}K_X+0?!nY({kwF|v>$q} zY~?tWXs<7hE4IQL^4DSc2g`Qxv;hYAPGg=v?xG9s`NqWadWwUV^HS*z%x;kHM+c={ z7|ejmy#_%DmM2)?!FWcI1lBh-!&qk(CQzShST&lcU)1so`UZ>3i!KX^X`ArZSed`M z=cBiv++D+-0QJaJHp?VnWy%_Q{7Rp>sL#i`N;li@wMF;o*V>;(u|Yq8k+v%XB z(3NP$Y2o=Z%ze5sVU+U~V37*pD#25rc-xj-vSQ-b4W)vd16Y7G(WI8^xyA6;33BL* zlHwOTLhr%K@GwN{y_(Fz@r$qWi{8*$i^|yR86tgVL75&YzRm|Z32Nm31#plS&U0PK zyjQfhl)M}mb(|78MvQugex~KNJo;N-qMc)5%IgN*&bYRh(jhQg1f-hOW7^-$JFXrA zY#@#}Do=zQY)JOUO9tRAU{N2zT+)t;2C-#{-oxBF@Z83?_ zZb3AQ?&Gn`-xi8)>DB%b%-u4}%84HOQr~ZKoFb`G|5J5^kuM9|QVp>3v~&R*jv6rA z`@@0ISm)VD%bHTh2}fW2o02*wdGW}D$e}96>5Idlvi~gGf8MV#sXY7!)!xT3K*yYu zf3pGILW07r-`1zv`Bf*9a<2@ZLthD0`HAkJ;u_K{`tA*bOiLh$a zYr^4za`{lJuK&3*pBYEA4w;Wil$v47^+>GHu7}EGkUFzwM^=m>~Kerm^%! z&ehVtwmaR?>&|Lc{SCfjHRJ_-#d<=eiqWbo@_=C{+NQV+R=CsS>r9~{5i8+F(j)oI z#J^4Q*!`9yerT+k_mc?dH)z*w_J(uND5UmP3oDQPjZ0k*Z zz4g-M2E#ICMrlLCYI_M~?sD+H)SP*DasSy$m?PE+{Z6{`PW9$OFTV7?iZ8vZ#oH!q z%fH9F`*hiy_nx+A7i6;PLHFCZY=j9_*whfZxB#qwgc?9z1Sx(r;JHJKaq9kLbmWEKrQg?TR-txOtUADXw%WND6M zJYJ2H&%P3IqS+<3I534#sA&{QCt~GVKeS}BYc66GuWxWdO)!ffz$gWxf9E`4W7{3y z2;V074@-VvpDeE+g$yVUI~92i}DwBtF zND@mo0~GD8%n3$9IC|L&W{<2d{I^`W%7mA5<|ziZ*}t&Dkt=+(R}7|yx_X@J2z5wP z=Zm$o*6cbwdOc6XwIdY8rj77*qv8JCc<^b3 z;Qi0pVBa~RAO%B%h*occY(y{89Bbji`4xQG`SCvTdXijm6mwHsC94)s8}d(PFGo2^ z8T__7Rakvi-=T_M_xEG$=gigi=B%SlDo0!Qmk(~`;~FW6k^T`y<~f}$rGxhV z0|bU?CAD%7oD~g(Uis~=@WPNVQ*$XPqQL|Wf+F98&|qFnGf}Y~+jdo0JM{b9Q0(T# zHnC+#OVt`RSE!{h<6Wj!fxZEq2siBIz3GFFFuL8~!ssO-^sS6-;{bo59e?eX4kIi# z7^|AIA-_4dcFi=4WaxlB!OKC^zpT;2TSBBr*9x6FH;Z&bQJvFT$GHbTUe)!~?)-G+ zrx|J7#%j8?PaN{qq&>OZP8(|3{OAk58{N4*qhjUyB4=X1DhQ)(beWq-zWe@L-{aj= z)CoXhe0X%a`&f>`tH6X}p1hKx#{Q~Qi9P;|Wtok*QdOmf=X;`uH*aWEHa)ZtmFLMO z9P^;A*YwCsSxRYpd=iAv5$d=fCW%{E6hF7^fTVV4a;wJ3JpcDc*J9TGymvXeEgoow zmc&Xa@SBw26kX2D+C`dK!xqHh%x~z^;?p3-06S=mUZis4$RyR{_-McLnICQseYk66 zK-qeP=m(2>F43=uZiTrk7@tZ2X^Ix&x%FKwHsOB09@S?34i|MPPL&fC8S{Q1e<3*o zH@be%9OGSbO$V{)Y7;kcLjjpzovh5Mos3}aUjlbF5s1(9=qTizzE{gMBl<3q=MWyU zXO6Wx;lqsw65m83V7=}Fcvq3;{TGu`@eX&sbiu>T@OLW-Ub1D{CBE&d}?0>O36jJG!XR)faPY=wh?U;L`h*7(fcaBnA|KwRBx2ltTQ&zcP; zHC4@6CiM$8-u+FOKFY*^l9^~mp(|@AjM0+9!(k$iDJYYzFQipyvq{P0NC6`Ca*#j7 zi8Iq_MEtCpKqGaEAm6?NH(%W0R`-XWk;Crbaoa#+$Ic&5a(!`G*fxm;X7xp=a*=x6 zO?1N{=R4^-L~V z?ozRNtuGlTM)ZGtda3E~Q=qGbfxCNb=hB@s7GX5f$hZecn;uFH8mr_fi=t=XS22E7 znaC@?d9UBcvQs*sIICqG^h4Rh=8=FJ7o4~rBgHv3YPv)j~@O3$1aIR{Twv&hTVU-_%3?=jc;y6*Vx(RW)`o*xc)=0B-Exy z+nfB*3_0r2K=Bapi97m<>jAp+7#y@E!oq{^Exi3tle3a2$Y=tcI-f?Mjb=M&ZhA`b z!rX26bhb5q=!*tn)zIW9U&O+&ZWlafQIx+$-v*Mzj?QF+aT2l06Z!zZFyhyNDzFo2 zT6w-V27ZS+TPw)>?{X2yBkeH?wDPTOR|NB}eXkB*dm6w;zHbw|m9``ul+sJiq0MNR zW?0U=^iw2)YoU;-``Lf!I^=^I!AgX4A}7}YBcOYDulhHsN$MS^hH@0;QoafTG$24B z3ag?CX(EoZhVFLb_F`e9clS;53W)f-vv)!+=gH2GglI!TcBdN08*o7u|%X9VJSqYn_uI1&skvdUA*R^bY&r{mB6z|V#Rz4m&<&xHbLJ-2ajq;wnGYJaoU<#iRBwIki~mBUWU ztE=<$rWmEKn*QmF587AeU3V=s`vYD#!(#Cl=(%+ADsDRAt=sloYV%Aywu;Q%5@{Zn zZ_mw+6Y-epUUaky$g|=xHTyfuLy8nj*ba$#a2dkUnE`Y<*zFaXu|-Gi#*Y3eygBSY|DBBT2!UniPdDa}8$pDG=m2fSnh z>iEi*<4>o2UbnZ=OWRzE|I&cyYI=xN7Yc#uTC_Ob5P@!*1C9PNHpM^2KKkfn*@IqG z4^3HwNX#J(QGVFFDy=s(BZb6GHxU{)7l8tGh*x2zCzpyQ;w|D?j68;L|7K&)- zBH5s#Ce1Yr?*ZeIog_CspFYA;Xs_+;)`&fWL{YBG9f!ryT+%{QMH2dt1|GWAOI;Kb{J(Znw`f>)W_C2-eB4qCiF_x_8M7#Tf@IdA*J z+lpFiXkH2M)uK|*QaxoS9SfF>GLM%_yhuv7dP~KqHZ}wJ7HV6ZGrxiY{jzPI3roTT zz`zGDuh>R$f&HO|(an7Jq2X9QoUC8W`Hu@Xl!6Cp6{2USz&J>G#xrA;Rsc{ z>r4MrSurgY+$>9eq0Waz*LEl##iw29+)>UR-7IRw=LKkSU~tFn?uKMZePGyKj_6C5 zfb6m>uJ9%EXbRa&_QW^IJKD@27M!q_-ao|Qgrt*@IHuYUX{1Zs6#bOC7-F7gc1Z8= z^tB=6Y8ThE*e_=rm$>{rs@Y&3(tbKzRmMx7s;^=@+Hx(z@lbvhx+A_uvkTSSMKL+A z#@aF7Xkvk@_yyq6=Ar5ZXIi>(l&*OEJ$(K$C7yIRMgzFq6%%)|(vM95k2_VSF?t~f zso_jvi)!ai4##b?8{I2_nHQ{&J$knJv=qm*H>TwEIOp2#m zGh42?IgHu0VOMSL%qJK(Ya{s9UHF*$(zCT)!n(Mfup9o5BOwvf$XoAuU%QUhP+gu4 zn&BGZkXI2Nqi@wugR-WA9&t*|S{60m1PS!g^oLVB!!qvOZK>baZ#gom{VQz;X+Z)hT*T=Drhxh05pgUv@iUT`4E#pCWC{)=b z|Ee#1|1KS{U~yDy!HCQpY>f{XZDkB>`t;hBsou}U(-{{H7yh8V>R9xtq${kqTn6Vz zRtuJ`n*`BWEpgFGCs_@jTFR0HmJ$#IJ;_To%^(lUd75@KR(g5-4n$)I=6T&M$p5}t zTAJ1ho{q_$S0Q7L-WM$&4bZGBdzHNinuLo?Z!2llE2(?pG~AA@<)}NKqQa%#A=4&l zZvwE@=VXAA1E%wHi@KXUIr9AN&EUusPt=0-Rp7Y!Un7=t{vU)QiB26P#oP6J(s+*o zWiugzvT&M!)=sFp`l#dKRra{Uc#VQ<_}i}I>KgVy+~v?*MW^1)VohrJ#GK=Tw&=X<6eMh8Wf{n4~#q3kcsTdLXm<6-`yVbU~4Y zA};^cig$Ys8 zR_cd=QivS&@}Wk%Ek6eb_6&1U@utMIEFf=)z|UGN#e$>S3Gm-Dbse=DeB$&!jg1Q< literal 0 HcmV?d00001 diff --git a/website/www/site/static/images/case-study/akvelon/terentyev.png b/website/www/site/static/images/case-study/akvelon/terentyev.png new file mode 100644 index 0000000000000000000000000000000000000000..e164220353f874d07a5d3df777460d84f76c04b0 GIT binary patch literal 815004 zcmV)2K+M01P)Vjw##S2AvHM-R* zU|98K%vY;EGTm{scf;WR_jdGRaV)A}&7ZDT=c{4nvU2*f#WIyH2Ld`uh4Hvt@tj;b zMW%#LNgJj6%vS_UT^-kgR&^@j7_u))7GV!jDs?gK>)OqS)na|TSSt(|0xii`2`&bw zvv%?S^9YnoOdv@4R&vQiKG|JvOZR^lhcx;Ih1lT4$wsn_ZVfOZhtL*5>0nVH6nxQG zZw|<^0y`3vGP}??k(P>xG)`#&Q7*or)@#c&+BvRS zHL_!~^o5RgZ9BNNRA$9ASE?*|$Dv?LP?Nx%3R!~NnPCppwT0*G)MFMQIqrX%?JC<} z0s$(Xa*lC3w?wS%>xj!qH#?r3pPbE3PN&Bwr(4Qu-hC=3Qn*i7jH z1jz|gIM~;h1_-Qw>ATNd zs@9m7l>?K9BJE_5DqGC^-8xEj52a?+Oq4|~04dd&Kx%|SCdF!^#iP_IqxVLxs1h!3 zb8}SGEoy^IrK@5uBxd|SBanq;b4P3hBW~VDlE55gQJi30-{pvBP7+EDkq9Cp(&|Vd zV!z<}j9v(tjBLQ5Nr37HUm>!-^6r+4^V#(DZ2j;2oJzW+3fs z!ExYd5JD3Qm8>9di|o$b6iJI?PEcfhx=brFhLBLXISser(V* zOXtgQx9z~)L8oRQ=sOeXQ3=ufgI^b`&3zo=ilmfhJ1&+FR|I_Bwswb4NQiUpD~*&_$VqVmFyMB+|ajPicn1L zPU}W)RQl>P!6G27aRYNLFG_=5WVjv@>j9t!{49RPgK<>|38dA%kWhi-mwc15s4vn? z$i&jeD?r;6r5!{r@0?`#JX!+cmwrjwJ?>Ig;CMmFpysUK#m_*8dn4rMfwIWQO69l)70BJ5*YrP&UOiecuFId+(TA* z+d3Cx?zz_=SWOBhOCDXi<78H+rgBLQ!5lT%THF)I|IdlZVI#A3WLc{6j5;L*5s^Tc z3v!fNE=&%|NV{NWjvP)cBeAXAvc}BlCD$|+U8=cJWObIwSD}08DM{8AQ|H2fge7L~ zW2bmLwsa6GDZwQ2MqeYY3$+8U55dlk%f;FG>Cwsl;nDWPhqv$Dy{0#?O`(>F$_6G+ zQ3sinbg;xrQq@; ziVWIH=Yt#@l4dm>qoju6n8wG{>}gFGj;4-jY=NmPk*(JPF1^9NXh>3EH#iMU8!A;l z<|QGkf|aD>zyK)K+MER`L|z?}z^9>9b{%HI zEz_#=;7Yp-R%e0AKbfI+rZ4IS`q6EQ8V{K4Gn-)XKi3D zL6M9_o8rP!W~*+5uKKWdBbZwn0&8k^ft>bg{`ausl3iWy)M3)l-B%|ykxZlWPl`~4 znP`-t?E&osBa<<~5>QGORhIsM=x(-oNf|LZF#6>{&s`naRIV!c^o-$WSmG#B?(%wq zjcE`agf57d+{nI*5A{B(rdmwhk4fuXS-vOyXn z{trZ$-e6~J)NYDX>VnF3s~*JOt~Rhrw!sCAsVGkaIfRA(sDn@Cen$E@@d863%E7>ZexmY`QtCggS($UN+p@#T;2}5uKW3g#Hw6qbV(4Mx2|0xPO#{Iji-tJrm7tGc}r^HQHYzKb{4?1)KE8XdUE`Z!A>~NWW zo_t!tjRcA?bVQMkNCc6UIsX-Ob4h?W+p|eT4dTab#x#9Bj5%m+KPK>cj;d(}l2gg7 z%iQdGC8FBd3dEQ#;}2mJ$fHt=dZ(D(Dyk;m1t^6m_H2%9cwI+@A5ol{f`>ziCxsrn4z(j+lGCCE@G>W3VM57|J6z$Z$$}W>ObwwpsBohRs3iGfx|9I(N0m!1ja~$^%4+bbN9`(s3q!t{5(Hx#UZ8OZw1Me|Z$M*I0#$0) z-U5m<%4e4rrb4!SgFk;Q%EfA9;@gqg0@l)E0LY zPK`V@3hP-B+57`B($}nAX3RUHB*GS?AuuALahjFfO~@`#PhXK-K?n+VhFoZZzGxN% z*-?kp>aEemcR)-azXc^7Q?{NSg7~BL>horyxXw#-p9#kVV+FEOoK-UVi0IYR0^P7v zy6JC?G-ct4iiR!~!X^31(JvR34wBib%-zNb_AYK<<{F~kLR9;WZ^8IhZuy<;Qo6MM z%2L!vugf574l#iE4k<7j<^a+77@L*V$!F$RXD@o=II%-hwISV_as?Z{TSp<#u|2Y^h85_ZP8JdQ}k zT44UFvPf_oxJtHZV>($2tit_0(p)Juk#ZkaORGaWaYKeu`YlAg(5?2hw1+Z7aoW8Ozde+@n(ba{ojNL2rBXU5 zrW*T6+lO>P;i@d{;x@k&mziiB1zs?gbSm~zkt-cb+lKElwGE8wRMODQFb6V=YUIp9 zEh9z9A~@35EHaS^F#S`F zc@#5Z9a;e_jgSdvMfuxI=!odz1!ZVw^b%_Lg;qtDm&IBqhLi3=7HG~`P-sCZI-z|* zE1?q`7{_w0w+z1nGK$I^&uFA?T@1O>!uVkeyDLEnltm~(!g}WpuhF|Mhx(*VnC#_X zr45XRMX0$TI3s;NpX<0-Q0I`V)6CWa_L42jb*otFP)ux=U|%M% zHZZHQrE3&L8%50-Y5;vN2D55>ibW=H5-LYkuL^Lvuj~O*wu0Wuup%{5>IuuG`(#_w z3VF70N})GGswPO)^}FSA4KSv8jV==5jSR5MK^rc@plcK~fxN6nGmRCyD=Gv{$fF}z zM>|GTT(VOTF~@mj|5kzsGYyl|=M$vG(S-DhUt|VS<{&Dpf~61W{2)*{5p#5xuB_&Y zsR&AB4SXiU$EYJ|m7wzaPI}27GB@G8JTqOefVf1xbeur9%Yt0uS1PMGXulXRoIDMs z;3dguOpdGrX){4Gu|w-n?xj*JQofi8lI0|FPc_AOMt%jQk!gl5Eow$9Oo1#+cj>QE8`YKeU2yT1Gi_patIq2zG z2a2?SpUz=XKh{bDx_?K(8(7~QWwgG*&IB>QMtx0e#J2|_7%>^tpNCQBCcgVrEdTZe zZF?>pI5wgbw*|q`k7O(*LO6g~?;wJz4U9@>EgP6w#9q?ZG^(N=E+sOBxKo-UrkVlY zWQsH565$AyHBQh*r8Z)6dUbvx6q0nSXZeQD)Hg#bqXS%|t3jEj3fP@s)TFs&hEtg% z$S%b@@o^Ew!UPbiB02=jB}G8UqICjz#bBbO!{9rx%xf91^YWR3knODO7`g=)+`x1u z@D4>m5W(V|zf|Q9(Xrokh=P>LK@HB5uL;vdNtqmh+Q6$i3i1%{2LgHJzg#uutEI|! zEE_=NT$+^?S@tD_kC|hosHbKteu+al7qXYvgY-EhFCXqgpv)&1K9;|&5M1)kBcuGl zoF0FZh>D^dX~LkrL@N~us<9uu;Nb?~l~A*0JQWaB&F z?1C-L89kIyMv=#K+VjFO7|VxDBlzaBas#^nb{B4p5@pLY$q z1jll8WpSGM>87~2qm`AevZb8WRQ#XWBQ1N4UIjxRwGOJ%OCsyMrAsy9(vO> zP(j-1xc18y85p_VzpQ7-8(UqdE=Uvl>r z<-#-!M91`3QWdB3Inay3!QwswaYm(V9xDzO$zE{xI^e~=_AzfEqILwi2GJs$N6PPd`;6mj17+R#{Cb;+rNi(4qowr=P zvhp#G8MIcE=dYLng75ZzKSVPT0j)3WrJ_d*707i;JjRQ6SxsrAW|ee|uhA=^K;-}|0lLZ;QD3+#VA+yB zS_m}ID>H%Jl>$aIQR3IbRgk776fuE!DNI9xA0=f*uxL=fb6va<7tp#+snkr^)rB9g zk&&ZEvP^)E!UE%u~)c>_O#{0ft18*@jX$HsS^rioivd znwp~%LHbH}a+K~Um0Z|esey~~bt4Cg*y83|o|qJ8W6AxIqds)8eHwucRSz@*=mph! ziq9L0B%^QU0!F=orDOVnb1pZxBuJqW)sk~-n5xaO%tc`uD#@I%Hn2B159elJ3RMc5_|@>5O*qOXbvHd&3sBQ z$&F3N6-l-5r^*F0>BL8XTsrE7;5^;lNdp5poTL@P6>%Wd2!f-woYK5AUx-ftzCzd} zII)tBiY}G{qckd~-v-J*u;5x(wy5ClKfGrnU$1C1EKTib*Kt9`Y4_-I=>1GoATsJ#HZY0wI*hLfj|XH{fGH_ds-QM7u&@7L%ISvD5t;r#{a8#u zBJ`a#!N8MdjO}@b| zudsSQ&Lo-I)4V2NT7sYaC4zEi4(%Cqd7f<^yi^GVrRn%eu-Xn4s3!nO7)3{)lnkN8 zvZ6fgi|JQ@f|U`~n1(@Ua11R_K*g~UrN59HD@z+#*u&&1uR_c2=FW1bbRkk`I;Sag zR2pXO)QV(6O*%%!tK>pn(wJyu^#7YHuc6@oue3%0tSs{Ydb<+E_ytix#J zb9XbJ$1tUYwV=OcPfQfV!P$;29kIX|=ZzUua_y!svcVFP0jF%|Rli zwo{dLJ54N7E_9lzCi+j+SK7^{(1Zq#l#+eNb|f=jdR+vSTgm1=q8GAxn5DXSO&a&D zcX$G&Ze^fgDdEzbY=K-f{nRx=R8pN+uOK%UH(|54ji%4qT-rqI2tgOW(h=g z_f@L>L5gUB37PPy;|5Wub);Y7kg2OvwkT#e@T--J2gxB+cyo-FmOsn%J$b!qJWbdh3l++(N+RU~y-UnOJ4{T~AE&Fra*Xz)%vpOiVPE-Uq*xmz&0 zO(uzILo7uj!qH&5)`v8vS4rc-@ISDt(zAReJT%S;I14j_3Gu6PXl?gCNVAlMyWl zw}H_b;XfTIwg_dQK2i=vm)pRQ5c;yPjkVOOFS|>4Df(8(E~px7$ZV@A6(W zMdarwB}hk|q_?0;3(R~r1y|e{=#U#dno#vs1E4e>uurqn!kjT3InO#!iV&WzG>~Z$ zp`AdExT9|btet5z^;L7J$%}jgIn(^If2Qfl*iG|7TslR5faM~QCKt%c>&{Fh-*_6~ zGEr^Lcejw0LYP2r< z^G$K2F|)WIbJJdm<7r`Q>GJQ8$=XW(BLUUO%x4s7eB=tOgGSb;3d(9S9|I*+OH~L5 z@f|YxL{!BMuzHZxS_h0zQn1HCHPl>e)M zN?uh*W{5=&=KK^cXw;m1pTLM2KZ~1%CU>`~@H63Z2N}v$yDTMy4Gc6(?s7-O(;|ds z7t6)kxF1eDK0H$tr6&+idx2V0i3*#IBoSnsL+yV zc~r=1A){j*$8s<~6}}kE_9u}3*(70&Bgb?xjmpl`1Vu_{Zy%Ge*#DQRwm&MyS1RoR}`ebZRnKFXJO7j!{{5dy7RPbTeEo**xBD{rE< z!0g;nglywiMUh}ajxO1T2i{<%ZnXNpQteKK&bST*iU>%KY4A|l3>c@OH0?tEci~&^prkvt%WP%xXHbOtl{Z%8Rp+K1&&Y233JSRRGL725(gg?OkbF!= zX^wP30eS=DhTZbYO)+Aiv|CMwlv6-Y&gKIWUB6zG`*}M*Wt)fFeD(Fi*=-I8dq`y!&NLO zLdX(Q0Lx1mIb72MO7-!7#jdx0>#rS+D^49|%TO(3L$?gCdHm~2rK@QctfyTmlAMx` zZ2wYUs7wO#YRX1XP2^~@j^d+T=9{_Ik=&uE&xaY7DyTGU5E;JAP*0-*rH_QfiO1Zq zExru91giSVj_!Z8IeW~DlG<9uJNUE zTEVTyumsl*h2$I2S?@lm3f4hmrpWi=v(;dJKgvVrKlhsHRakH_Taprx_N-guAX(Fv zgH?w~;Pc1B<2BMWfq zFtd_vYZKf>da<-5g{6#A+M|x{3S(#;HmY8w+$u$qfOdi&(p8vJqX{Sume$d(&x`Ly zzK#|rRAq9Z7q*)YSB`7ivsDasngG*iK_#OS3!HrWOp8o8bA+kP_HYcXTDK!jG5I0@ zH?TSsx#;U}2HbAs-y5H-hV#|1-pTA6K7nw7;UO}VK!ECIDOB|p%`;h(YlXd~M z5*qoXWOPJPwUUeUY6*?7SIGig6u9JULAo92^}vB*)dyLS*oW-=k zi0Zw~;DS1vwNo^&drQL?}v}IDfQDIUE=9onwVu$7OsO8z_ihs?`uJK)+h6a_hvd*895K zI;ss}14F)a!J;C(cwbI~;be1;%rG-XK>{BuFFI}c8Li7e=7XgSqg1v(L6mlJ8PZ-2 z2v1b3)?0YVUk@(DkRous4|=b#q#a zX2W6)osZk1)`5oo^VtWV-h1!U2fMTWY;Z4wq@B--OM|W6fEU14wzt!LeLOYuFhSe3 z`g!t3cav{Ax^;ONnarb5zOJ(VGo(=3_5>*<>@HGjE_}yrz$@UFKkzz{PU&3_dC|v6 z`B-ftOA$=NS%F1V7s+hgQ81hVKfbdKj2vSlo3Fa?N4_S^pw8lfMU_UVaj;dn3RY8B zfI6~4dLKQtkLSpDniRQ$PX3}w)v;txYkKg_MrP*aImJqe_L;}8%`j63wFJpm!WZD(+x2}V(+c`VGvAuVFXZK*eA;my} z>D8vg%6)sx$32+!&%Jx~>DS+Ta5i0UUFXAM;_t>NNgvvOC|<^BBj{`KJf&WK4@_y9 zt>Bhpvqe^Zt)Ux6%Q4Bqz*78@BX!l~A0hVK7JS;UX{lJ1JT(oBicpyt7y_4XnTaW4 z(pp^Xjuu6<*3e^gvIjM5P>zOk9Mxj}mm7I#t-DFa(vI=fd0hlSgQbv9sn#Pi4O-P- z&t=m=yKD$E0a8{Q82w+W2q82hwSy2Z8_r zD+hA}%aFUI=BmnSvV~G&b+2|P{dpl=z^ZAp?EJ0RdS^OWEN|~0{^G5-uiU$LG@qaQ zn=1K%R(Pc8pDv!SRv+Gd_}!mA|Ixjjx!c=@qkeI?Se$!;dV`UxUvCF09odY8g7G~t z#pZTNn;|rZY&&$^2o+!d;NlT13Yj|{$0B=ugm9H4B_}*M5*$;%w3v>DFnvpWnX?I! zBC>00f2_nM)CLxGgfB~}=qTlAB8^}acgA&e{`2RGct%#Wj`sSZFPHetJ+8m3<_


AUJQ}A&WGLQ zqlpmG18qmJ$a?$PIXioBdb&5A-9I|s zKcB_Nn0`<4ZL%H=v*qG!IXs-rU;6O+OYdJj9o)P4H+J^ly|?{vzBqG#WG>%?#JJrE zsy(UJT$+6k%w#(U=773#p`qZ69!L%@q8b=09E)wA;T|C-f|e~o)grT$rF1?*fh|fl z!M8(^1LMhLCL=23oDqv_ZT+CqM79Qrx+f{c7ss3`E$EWQr4ove6mkpFQmM76AT4t# z=TKRE>k9Uy*gUs7%*mgLSN9JEb%T;^V1v?(&b5>2!+zo33i05GW?e>6Z&JO1Re;XUmNki)RT*!nfW2aPfdFNT=8_V zIPSXXFr4&@+q-+~J2QIAV_kGO>u)_g_~n(4-?{ne{j=`F`-k6q;jJg%x%S%aos+qH zEfa4->d^V*Ra2+EdW2~DC{&6})O)LkjGaqtLI1E-^GojLX`+Fnx)#eEeb04qGf?Si zm_~roaZ+Xias>ewBLT5sWlXEHTVRT~18YU@c)u(MGIL;eZC8&SOir}Op=Hv7d)}TI z2Uem{uq>9n98y*x)b>=ZocWzxd?g;k}dT{$lv)J0Jbow}1FIKX~@qgF~n6U&NrKp6uV2R8wcpFW}iQ{oT zEKe8nv*lv_|G9h5-VONu+5Ce$+uPmZqrH=_|Lmnd_{z7Re(l}(Hb=Jdmy%S;ki|WR zq?frwJ`!#N^Stm*ZFtGV3XmiUMGETP3rDR2Dq{|KKNsp8^;0X5rR@ebd5N~eQ*@*v zG@6i6P|&bg#z%;3RKX3FJT3J&)39I2m_55#=FbKTvT9pOnSe!tjMDd41j?NMH!rZb z(dZEhotB1TsYWBnyNN8CV<99{1$vcWxQK48%d(|JP~e`;u3BP`lY)F?!W;yO2jw9( zY+zbsnk*|NPE{o2znJpRIKla(<+Mw1L{(fWpOIh@X%IHhNLZBfRPVYx<^e-#d;Nd=An zXc%9fM2kykS`K8Ir*wdhd^(<9j(TF@{@oupJ{mGnx^9q{6kYs7qpIR{1X|&$U$%zI zp%O)Ib;vO+@xrC23w%rS(PA`&teyO>6*4ogu`4`6Sg(h))=;fSvm)TLqwutIB{s#X z8VwaCl?dKFFhvwC5{-!f90XO9D=U@hxUzxa$G0}5LOCnuC}0u#q6ijU40XZA#AH$C zl5^-*XEF@y4Q$dawhvC8f9w6feex%dz3}SY+3?1lhd+7q{fE=uG5r7w7oZKj;3mT0 zT7I&0pSrnko-GzLdI8e+h|J2}flXJ#S=_+J)eA*>g>th|bLxSkV>@^^$oHjLl!Sh()vgs@K#Nbu^7@A zF8eSRC?o0@L!^^EuUgR;KlIBH=)0NoR4XA(lDiC7NP4_cF|~Qhmx;EIcI{*d|b^BP}A7)boKw!7*({eY{}=e95<5J zI`oS&1o`_|GQ*aHOB}RFWBUAA^jhjZaD3$G;+H*nY0GL6zo=V+UcZC_SAj24c?8a& zYHoeEy4x9PyMm%Xu)3ZcX@s~{O!OmHmLPp~-Gx2WMYa#nDhQd7Lf&GbiF^g+w*@#5 z=94&JDS>ok7bgW(FYqOp8(0u2tdOHfD-vqvE#MH_N;8Eq)hoL6Zeg z-ZjY8yF1_f(X(HA{HgD~^xDWxw?Mm-tNh%AJX9xVeK`YFHX;=ZUZY$whckLbQ~7Di>bvl^d8Who*WzsL@1q- zMV^}TQ~J8oIN)3p=gP9w9k*SeG28`#&qSE{n9j>E#i!l@XC#vRj~u zDF-2PBpHnx0p||ay%d^?_!}|-Kr11+aMPtycV%-p?LNGH_eU?i_T?v@dhh=Jc0cU) zi^G0?HlO=8G#(*#exRPyZxN8r;O^V@rqkWC^H1*Y-n{>CGGF>1rV8JWKJOOCCzGS2 z4m8|U1ke{PMe zw3HC+<77!$`O;cteUTYa*P?I5?V24o*&wj!%z|PsR7ZJOkAwd?OA7|J$8zSL9OD7cI^RQ#vae z?8Z%PCyNR?ren!sE7JnE&S#-RWWI&9Ns73UN5)t$-VQ6MkJo0{>B$Vy66{vnl2N-@ z3fpd)tF5f=)504+5OxDr&Cg=B$j@>hcVRiHF{r5h){_x=F;OOM&=)`ScQ-7IEk0AVS zgd}wd(W!$Eu>OD9EwdlI;<`ykQ>_kDN7_{r7z*N;x&*VzL~_}1P@W8#FKl4+)(u;z z-BiR!h;a7|b}sE@!d50|3HI{bKo?Z#Hs4N0At$htg&vd-M$jFB%-~+$_)GewUZ`zg zNQPwSKt6bsj&g|B(+eDjn9PcD)0~Vi)Y2Uo9~7cUU&CG=SL{{tMMx%nRkj6{D7b;u zq578;LXn7K#zHA6q7%s9bf67PDQFi!a0hz{D(y;Lw9#kicA)dw>sPOT@^H^RC-jdl zr3($Dv*jYbMd)^xbuDh~9=>ty_VIFcvRq!jfB)9*-lx0!FTeHf`!_$izjwH|e;78f zs~kHGzYfz|{#M z@~cP=U0I2kqXZoaK@Ip@G-;8V8OG{Gy9cBkZX%f zk4}*D=)wM{<~MN-+!>P-n|ZwPPX=z!@+8GuvnZd7iqZ}dsk`{m+V4r zCfTfR87;j8q1wIBtSD=$b@_$7DwR4?T#X(k@EuaHScz4Dfm*;L%EEB6VNm(MM zCXzG9@`~0DGnO<`i-r7Q2U3F`38ubDK#r~;Ed3?N3X)4y&oL7p5e7F24n-oR%5J&E-&m@Jo{?Cjmz-9L*jYjkmV)0jD%FAlqYI=Jz6 z()G6vde_X?IDYXxQ{w5rs%KADCwb!h80M8fU4^2oN z#Y8q!+R@xRq)oY_A z1}nm9n>LrisH4BM%34?x$~l4Yi8FXW10wZ znGD0>e6inkQ-7P~_ORLd2J2+vzxeF$(UcDI4ezxT`wFTQi-bg?)emUnh`pLy=Z zzy0R-zWnXSfBWBk=8r!A#pm97>&Gv=@a-oafAyEoU-|IG>mR+ab#1bI|Kah64|Y$- zwy4qw+X!^2uB0l;vZ2iu%z7P4$qS7cL9X&XTvu4Rq?GDc*})xbV52$o6tPZ=;!X`R z(d5`N>GcFxHUA}XMdKtv0l{m!TygLM!bYB_Y8id1u1NBfz~qhFD}ICg3<0!BD2^6B*hcWl5P% zk)YfY@`zGgwt#v9TmsADlNDJijq|)05a?o0A|WCL>A3wlxq_Tfz7hP&Ad+!*QCT`g z7|P1HTAeS4owM_U^-a}ka6gLZ{}Sf*(aApjVN9yHe*WBa*l zcRoHiJ@Zxpc;*dxPVAH5~QxXI^>p=FZ-F1DnidhewBZ?moDA`|iy< zcVBqp_0Rm-pZ>c){^S4jr+@V4|LxCy`poyP#Mi)YN^O+)DKM2R=TKBga^9cWsQq# zgnwW~tF~LdCBN`IvmL5g%qpNAT`KN9>>pffv+lBagQjS)R3r!}Qb(38r0ud|g2h)B z>0a+p^kPS@wTyzAS#Ats!uR1!5TD0kCEE;nwhARX(oBR>)y~2NrA$F2jN-z+7?)d8 zsY#9>#+rao1p1|z>2laVot(|*{;JFW9cTY}*koAlP9{g){HX8uXVdM;^trdMym{@` zdV4!Oo9rJR?Cb!HKy$zC-QV86|KQF?H$ME+zxwQN|J#4_oB#TE|Kor8_rLhX6YstC z1MEYr)u`n?iY_wrD@3^-K+b_YV$ zQ5yL;98_~g5oZJf0;RM_8?6Nu9obMV8FFh#LW!I?sy^b+Tvu+ixbG}4hvR;6>b|(; z_ORpmeD8ef?#71Y;e7eym*4vO6F<53>GqZDH?QCP_`MI`d-0X$UwGkX&prS2pM36* zf8+o8o!|NY{`Kd+@Y$DNdFu67AOHC?U*3A}?%vL{cV7Y<+}~@#0U_5cc@3p5rj%C+ zs_ohx8JHo%5X?`*=eIujK0aP|UT!R=2T(yRE^vIDl@@AYnwiSjFw!CX_w9Tc5g3QPh;X3^y=(q@Y4$KIT$|nMjnpk~>|GR8NtxOC=S+@igGcpW{QHBMwdU6toN^6piqbHrMjSt zVJu&XrV!?X!CX0r=T}rof%5b>K+=(`&(Ai@Bu(fLKyVzf0J0a-v{!-C`I&w`3}?e? zcRD*@t{>k~7Uv%6rP?e@-Y(%4!TLHZ+ zsD>-mwuZ7xU5&r65;|UPU{*fimL=9sRnRdl#|u^}JZJ*@kS#;fuF_7k8oG({SF*Hv z%3Er)Vev%<6<Kf|?YGRp5cd%+dy?@64>+T>HC<_>WKWtU?ae1um>UOa_99uw+z~ z2~sx8Y6$I6kDKu=yFtj?Rwfx1qx`}msJR1U?bc%^=8VkbLWC;kQplCfUZAPk#gU1l z%W;G;9f<$i!S|rNT&TxX*c#!=DRu-W<9a8aGWZimE|OAn-+dH?LaSI6#GoTpPU%5( zHJzD{99rDgyUb)5KH1y9d2qNpUmP#p3mtU%+pUJZv&q#@?>+y{)n{IP=jzRScOUFt zy>|7L*I#(%*&jXeqwjz1n_vCRpZw?F`hWlL-}<-z;tSvS{4>vf@3mK+c=w&BwhrfW z+6jBLRH<&rvm~VHA!yP|IjSNgC|MI>p!n{W%#s?D!v;nTYDJe8%u|wa{2{2+oMS~; z7i*?0reu2cR%3EX9lU9Rg6@04c~*+g(^WY#IF?N_jh>H2l1ll=<);IR$ z1ay(yGpNe>75Zp{OLB=&Dx&oo`Go3Fv~MGn5xfGzHIs0pSU#MXDcD@BAjYJ$*H2CG zzQwy2r-Wn$S$77~5Plm7_bX}xGu76~sMbKHgPVOpRw+nFidgCW$`&Pys1r58%HJpN z^M|4m9KnzOTYLLgw|95utDAd=4^B?led=FLeD9MxZ(YB2?e@LX$?W9pbob!lwU4j; z^yklh@oQiG{XhGo-}^70`TfuR={LUrcQ3sBlUHAT>bd8?yLIN@Ys`|%GcF}p;Ygz# zH^iwuRijd)(cRfF_?Z9+#S z;+izSw3iz^(@?wuA}bL!Hd1L}1FMfTA~+q(_kzIjBZ_!QWqxg(5L4YqRg-l3V3er0 z1|ub(R|lD)bjYVw+zyNVC80RgQ{)q;0;EY_$}s&8;z(`;W+J19&w4Gs%YpfW@m#U~ zL`0?0=dYnE0$y9-fK>XK6j+pg2G5`QpM2UmpS^!)=cV_r+&wr7lfs1_maEmBhlh_p z{foD*USE?J!>|~Z=lx>$=;W90e(=5V78m0F;F8aB5APlC7bv& z4ajG*sw1Cb-PHzmfqD^Hrb-7Gs{&e*TxcjtghAJxv)wpnzlBuX)dm)3flm_E7-{#a zc55G_qn+T2%B2j-O69NWlDdAdRx!|2UJ{wIj7VMYPfN>%j+-ZmOnP3OjB%9CVQeEx z95vKsswpK8d6~GDs>~6r3tHL{pV4IsVdPtzsIp7n7MPEtrP}&V>!@G6{NB}{zWVz1 z$ys=c=!33X@8ER$+B+YvUM1`8OJI?!|kE;-rEP%a3vBK1+HMT9a& zai;|}8Bqe6Os4ivM)#SMes#+P?p|(_Zg6x=Xht(nX_u2L3{lIa5zf&=im2*PL^F!a zxej^#kts{dYIS#S|Hl0X>#gg-@#%K_O(~~9-QnK3U+tbv4yK*&thHs)t?$Ez;r8y{ zr-#Qcyz}8RFTL{M;GkR1`$f04+bxb4?u|yZQHPvlO{^7}-jXbN;oYvQq@PX|O%rB_ zWJw>TYC$!}RB41?!SfLO-PkOsxk@(IqJ$7pE^w;xW0aQf-g9Z`(v@jyQA572K0p!} zqhra2ph>2l#JzFc-o_d`?r)9%x~(dd4nnx|@?^|5i=DmXOY9g|Mu zPqc(yj_BhTJd_lL%bEqN773v`^4KRJ1$C;z&Imo|9n-p&w-_HFs7*{e<(bwvxk(qu zL{oxpWzX}`?(uzNm3ay5NZ-0eK?adgA&IcXN&^MuUtUxk=N<*A%Gqkr46}tzm>uWr zdH$;6R9Q(52Yd}74ra^YU^?5GOm?T!qrP`9&`;;5^ZChq{_t#aJnz?=*17*w%l|M( zce+?_UF(Ut=!fNGG2Gfcc=f#xKe+M9!u^uHlifP#-TU}va-7pBO8l@W#AEqkyTRuk zvhH#i($`&e<~K&ocj#(D@gT zxrnh*Lg)8#^qu>)W?A|Ryn>Iu%rRF4n_K`X)Lmo(bWA%0pVG10i3`znD+x|l*aM`p zGGZekFS5cV=tiTPZfc%~(DFX;4Za#_Ei83>p(&Ub>~tHN>6 zQUc1wjj?JU&kNTDm6jTj6<$_xR%Eht?_(vV=~Y$NfO(zqf*xc$vF zOQ1`@FhRhJPR;LGk}nEs4(BY6RjHv27H5>3*|1t~QOAqn;raA<;r=k#{&aeCXYbvQ z?|pb@`@!kiNxvZeI{CwXPQUpZYO(Swz{35JOj6sLEZu7%_3)!GYhum;+3~tabe@T9 z{Q3Ji`d*hT!iXl!+z70L*Hg(iH-?nPh6PI1p*~^m;s!>?b*Ye2z7jc?s#gnGc{B&) zV`+k`oxYjY9IwVt;nYiIQfgGk1a_a&F{CVGHeCz8eCiHR0r;|L9xp$&uh2{ixe)~= zziZjRa{aiBYMKd`x!0EwOF(y7%8h_NuiSdD zb2{ZoecsQ{=JWOK*Y$h%Ki%73+s#|at!W`71#9WxUUV3ZW+W&k*4!izW;xiGe_3Nw zv}8zgSWPy?;m3xQq%-Cp@=2GZ3=uO9tbOuSZeVOT`zYm*8*#1N6sc+=y-JfNJ`|ZW z`Z$+eaAlNCER}>(=rBslIhS+{sk|=`a=GW?EaS`G}P&(EcFmR4=1ydv(*_z z<0OSVRd!(=O02$tF(2(p%BP_T2B(0;w_Hj^=7jAFs~K7lQaKr8Uzio7B0~XaX%Dt) z=Lq6cG?ze5Q<^kdDwDBNUs9v%LU%mSpUIBZgjmi#WJ9GyQ#fBNwhxZ(?Cw7}JidPS z{?%J|{9pfCopg(HpZv_dvavj0&fUSVdT?}d{r-dXeVAYU+>0CX!ey(}0lih*QgUt> zfl)bzx_Je4S|s~p+`x$D_K(~JrHIBHENuj*MAb~E+`({ zR6c5TdWA8~gm7GFA`F_8^F`K%d*LFkp_XzurHaHyB4kWI23qLBkak{?WCWKvG!DDq z4uS50&sCO9!Q->wHwDK+Dc_aFfx+!y z%yRG#N-r^?j<-`??tx~HBi+l2i-(i5 zgRb*|hVVxv{mQU~nN?u47r^AG=1!+$WImSAE+so?qP-mNBJTBK7e zgbCd3SXPOb2pMu3t(Z+47@(8`$tDZ!8Q>0U+z5*4AGLPSszu; zdd+2#WSTB=xT6#bUKmp0kqB-K-`Ut2G7Yyt`vWXOID$Af#S_!Aw?KN~>@96zs8a&w z(~@L~Be&SfLRo^Wtf5a9tEo)|wqPKIti_~qoY&@FXf}S}7wqj&3hj+K=jsIENx!9IWhGj zPZMNN=y7>UjgLw8*1RxiR3mXwPrDQaMTJ&~5KVVfBCG5|94b}B%J;jwL2)p*3TdHZ zswocX;1zGs6%0OFq)v3ikz-bkcU(>~F_GM|ffWKFWQN%Lt}Elylyx_q%w}hOe>9su zI66LAEYF7F&e8Fw$LF^WCvV-nyFKks)^}t6&op<#V!qzM{5$k#?t5eXl^dVD`QC@; z{bC(uYkM|3?0etwS&L(SvQ)t?wH%RIiGoE`iWbF}3qd^=HnalzqQx1rd23v3qNSBk zvXhs3Q_&5$;*u9evmcc1K-40&5-kC%Y`B*MHwCua=msYF?r~yVE=wIJ*HgY9HIe#} zXM1dcVqa9jUtE<)4a-eky%$kUM1Ls;X%(0ek=9o?FoIgg6^%+=mFfAg*u6^N()EGa zT&~EgjQ5Z3E+2QucS&C8`vsdWP1^|JI!G@o6nE>#qoS`G@tMHiyIG*U%Ybj8$W*b$(P>v^klj- z?GOCtVGEze&dKTFtXnJWpG>ZPeCuR7a~s&bv&pghwsV;6<*LT0F_i>Ss^^1rl%MDd zPAQ66M5U%t*|RjNf&so>+UV`hF_yNfT@c3Pmy;^9Q$GItebQb}59dIOSFro{e?}Ln z%+ap`r7Ac=H7*mV@y+jJ9J6sRtNQj(zdNLDZz~pyY#Am^<=M_RkHQhBChcpWcEl|}@6(2%k=d0CmKfix; z{K>=pj~^ai-#vNq8ZPn z-41GffsB@mC_=W}LkQ)C;<8B3oiXrO>ih#hyg|`?LGVM-{*Z>oSS4Xa&W;;gdd>uH zj1-{-(gk&TYueVKWKHn!D>2jB0-B}#UD1%e#ba_p5#6${oXgo_xqCLdxpS~R={~r- z{naO*{@W*>zWVTRw_8m7AN-mwm)plD*Y4iGdvG}MRwlKH(~DOMe*vI zii8j)7itb$+i32uYYv&v7f*EgBM@> z@ZQe$$r-(AxaRDhO>W$OaO!`*gVeVCLv{}rz}~=I?1nw8t8QRubaqkO(qz*vR;$&E z(2`KF6TeiQI?`@uCBraNRXS>|M-D>C%F`Bw5;++Qgc6wSuCK-O3TKHeJTkOMVX0pN zmXIm≈Hwvs8tW{bH!>A_OB=IX48=kx%PYleE5?Riy}_$$1M)P>*wZ@@YW;h@*(g zoR8Bp$-GVKKz$u9ilc8IKLcZ$7I%ZTHnx?o(!pd(fht}hL!=FxjkL^Hc$pb;bIEp7 zvGP;*s$uW5UGGoZC#N6X-G1iPH{QH)`_qGyhm+}@y@UPf?AFfior6PHsp#&$$h_X% z+-r-=wWs?j`}V$9aJpP8C_;WH5*@5(f(2|WwyWu(yim%)A`?IR&`!ifGD0WmpU~E6 zSlV-g5C=+2keWIZAXj|dsrx!*?1>PsUeg|9@s&tNpN+NU&|kC)j!<5dIpI+q;>dbYKAb}d(jASNI9gP1%IMJ& zXx4|Oi7e9)gWOJ)4j>#UM$n`sT zDPPmx#W_~Fst6Ms6cEw{ThGkA1Q`xX6K$o};>@d*;mWNVJxvYZ5G_ocwls)_#@^T3 zbVp;bM5CNy{E`T^VR1+ZeXc?VtB;fd8Y__e%$HU(;I}~5B3K}2St(SgBe}@G31mqyya=zlVr!!#-IDL*myr}P4w zI~KiSRhcjE#p}41tX!JgQBs!3_+$4Fo?|%vqXGM_i>-a`;hu~~d!?o& zR5mavQnH{R0^c&gD2-B0$Tg{gnllPWPhESdeks)gEp=dbw?KtXJ72v2-UzDV2y}T= z;7{!4aqGSC2FBo=Y5kEce3_~tpHC~Ad~qXbk^t`R8u9k-{^LLW`S1PVpMK?gPwaM! z!)|`QST2TPIk=x{-1=bW?A@KSD~HotC)1O{TR>E8t~?KslmsF7x;!d7Sr*+gx($GC zq$EKID&Nd~NYx2yaT$_)CeVIUDpii=sq#fj3($*BIfMh;IHHbHXrW2y)ver#7eJTa ze#ykLYBr~hE~aD2K8mF2PhI0cA1T?3wV2tn7u&#OJIM6(>4oyFV06DJ?QryvBjrO$ zTJ4O`Qb4Hl5%Lnh0;I_ej2<>{INW|(kaA{6OQF4L%0*dZ(Wqu1lXo|>j%NLuIhl3Se!lg~+lO!5J-)Jket)_+i;F@fu!_;b&wSNs+x?b? z?~)J~+L58Xau^(69sh(X{AFH5NvBjw1mu@aOI}6byai4O;fq{@H{_gr%NbUH7Ydxi zrjroiQu1(D71AyOZBW5Be!aAU75V$@UOn~7hn6<+|J5lqD$lf^2+4NRVvkZwG!wnH*SsWY*(iyN4%7bDT~oU=Q;ayai8 zICa{^K$rF%}+BAX9a<0}9^ABXSSYDMwtG3y%%Nm7?XnaDIatPE0YMx;=G=I+u+wfp zR%=L=1nK6l^coAgG$w=1LbDI=tLmC99Fxy?w5X6c*2lf2K=bH=qk;)(EaicaYy&cM zvayubAJ(tk!90AscXr2HCMZR|TLrA5R2&R#S866Dii*z^|16k-u zJD;i$xPveuU7W#XB4g@*_IDj|vK+2&?|%8QAN+^E`0Ic9pa0-%k3IR5mtOnhzxm46 zz4Q6KNxwH+Oa{Gy%dS#aRVFy9+`Y1gJdhuWOLvq_h{H(R$Iy*EGYK@Pbqz%G$NU?8 zSi1nvmmS~Io+exydbneiq=?8yu?Xh-ZZ0banarHhb`az)wvsXiepz`GnE!<#b?6tK zI{2N|N-xMpkO-yu)5{uEE5*KJq!p9M6bN-7B_x;#QdVa*LAZB(Nh%2ZjU1_;rhf$4 z^+IyG8D=1Z8g1i0r#K7&RQaY}oVy2)(WO9=BnCRx&NgmyP`&PPdUno;M{|8fq{xL01OH17y!w3MwLF(4ZX zOE;QheV!y;TDFXV(s=E^AI<^^@)l)ps%n zl}{+;>1>7YTdCNC?UacR&8*Xg;j*g@5?g|Na}l z`M>`9Z~o8!=dW*t3#D9|g|fgEtw-!&?AYcQ4rtWz<|vBI?%=35z*3udN^7@+2~r*N9PJU4vP%>H*Zj!Am9B{5lv+7l!o<| z2L>W!9Pe0P%TPgz=pt~p0Xhm;%BOk&12a>AJx{xDj<6V6jez7jI7!?=WuDp9PvK&* zf|OqKy?X^I69P4b2oY$7Bs-ogP-R;`r@7u#9SWQTr2~V(1SX)MXsCcv*5|uuX@&^R z?TmR!urKk~?Qp(W9(VKo>FgWdf9yAY_g{YY3t#*7V@5E&dIU3QTE4S6H1Z zsP|C4w2x!aywt6rGFTDL4ovjoVh zuYzONr_~T{ORIi};yBz#P}IkMEVTwJ2UuOKLbDW(Oof)9SZXrSD4(4or;g-8UUVQ7 zGR=6bs>2+&L_mm)*>GJV#W{3p>KI6Qr6hCA7Q^9W_Tc#B%isL==fC>(I}dig^~96^ z^w<8`7FHDzs+3Eod0wRjSW(>ADz{W=tb~g)uNH0r3(OLz_0108E^ODd>zlWXgt`ZE zfK8_v!v~wx-+NQT7=Cuf^Rl=DFQ)Wo>g2YuyA!4^1& zY+x!SZt!(JS3dsaJ3si*fBoEF{o<8hu2!q7w{QQquY7qcR8{C=!%>CpBJp?mC?`c@ z$7GK~s-#)LJbuthj<`%PRNO39*{iqu9wE~<^vQ0`oH@}NzfR!aI~b<{T#qoH5R1}J zWNiX!6PbpZ#_CjAFiw;)aND@K(7DqIpMJ~A1FCG!=H7D~a< zKpZV9it=&?`Geokr;xLGs)1<$v`dt2WcDs$y?-}4*5JLU?k*X>sY|ue#Pl|g3%^006#BXa+ggk!A0nz6$-uRicmqElB56M zP{5RwT38y0415$seU}V>5=*DCbO8$WOD#rt`WJI3O&eUfD1p)Zv6{+~?B#w@UVMr) z#n*Pz$b5KpRs|h}y|zYsSR4z2%92HwFJ4z$X%x9^R)k$#GM3<~AoX)FN1hTM?FLqN z57SVQJwHuP2%+{E?1ho&c_0bO2iw_Q&=)>bPJ4zKi)58o{kAnv-mZdF!hOA2%yFe(pw4|zPN*`OM^fqivr!W>Y3s;b>2Q3vzJ6FM3putNDu80~z&=fR3 z+GV%OaHelEqNNamPi3HzJ(R)|Fe#YAt*mR2YBI9aT;fNebVj{om631iAdQ>I?G|*< zoQ5P|b0HJ$4rU;V@^~tVo<<}aWh%dd4J=E*V;%!t@EiM9jW$KqJzU38bk(+) z68B$Kvj|z2a4kZgoC#Tz?_dp`i~tSAmpgRK%b$u;1gxPKgPB_fZgB8Y%ywf5G8yzL z*?Vh=L_wU4sFMWaFTw=m%Gn$Tjx@TS6~1OrRiu<%EIOdysfmGjJ*`}W(->0;m^qj* z`KHdgtrOsGpYI$o?+r(JjD#*L3|-hAn;*T3-hU)x%))(fux>$-8wC5b{XdnORf`XdcE z63FrJ!#ZE?GcvUf%UZ2WcdrexqDX;I@s(Q_TVyHyaAIWwGfGji!Nhfuj`;>wf=d;K zwc@B!2eQW^6*`l0J6!h8bX^(kf$Wd5#lm;j@_m#P_{Y5vYTq@pLVKx>NtWE9UTl}0 z5@7Mk-to*=7ncCaSt~~oSt*3BV+8(YY3Ai$QIke}MJqvsxQF3SY@oivXv+thnJp-* z%n>LemC;v~qOalyd+aY9)rrG`)#)7gV9!d+D7{Q*iqNU`EwPLf4#M=$Q=+2E$84+< zYaN1$OgjItce-3&{p6ow@5`-Ad0U2iy4Udrrb?r)$^!LSLn!8nr~p*F*4l^ ziim|wM-~l-r4@8bBYC+1xg9iq8@Doc2`vd!DpFc8D=tkS_qdOa>d4lXNQAsk(HAw@ zv0JRMlzU_lEDcT}i>MLAq{S|F01*l73k+(Ob`2YtDIyiHwC-ziQa91UMD|$3>y%5i z9G7=xZ2@M2>rF5M?D|^DDjZEi>7D_LKn+of)q>g*XN6OXt(is%_P8qRq@d^}IB3+I zJG<*SniTVv=#j?q4w{2jCPBnz|rQ=cvr9jUx+*}(5`nAJQBoa0o zEmIRk;~Q8GY7HY>v%V|cBbq?f@pk%1Lfw7J>Yy(RD#%jBZwIPLqbij*!RX}(`HJBP zbigx+H%^G6qRZ|>5J3)mMB1XN>y?9*3D5o0>1^xf(PVcvpIX7N0u`9)wz$ju0)P{k zVrMpT;|)yQKM_G5s|^fM(#O&idUIqE<^KgvBF9oS9MxruLMXNJx0i!5R2uFyCu}On zVCD3q;KCjdjV^?kle7Hqo3IpK&&uSJoiz*Odh;^y`Y|M#3!;8f0HbV7*2f34f-@2# zF@0iaR~)Gzn~3)wSwYi_Eb4%;(FR5d+XJjnUp7ie0)DSlanwYUzN7lcZwIPLBa~+2 zoTmoEZnPmysBCr310{PDM;*z*=tyw=B?6(QOn`D=rWD`0wm;eLm!9E%5z$ox=Dd=w zgi&O)q@w&0XGx)i#E%JgG#R2Y%93*F4$Kx81SKWMy2&pAbtKXP4NR%;X#O%M4K)aa z2v0bhQ#GN&IUSG+bG~iSXK+^PuliymP*4t?j5+`RXYW1RWVfw!(fwENm-9UL-VZb9 zj?cN1&Uk{yQTEuDWO*!0vX#Tmd3R2g19MK5Lszcq>h7wp>b?NMVu1k3uDvCB&WF48 zJ}nlo*1`e^;Kd?Aa*oC=WrQenb-aEgKXNvt`3cLS6JdJ9vr-}fS*m;2)bR0rjeKn7@@j&dZ>L` zesj>)@I*X!2CvA$Gw4kku4D{_#DI>Ri0a@jg_2@?m1p#f7gkLq=?nY0wD6tEbUs+R zH&@a9cB)9)Zs2FIP3gIe&<%i2q15SN>~}KU2deHiYGv&(7@|R551WBqAdT7L9+#% zyWv|}f}0p=n^4%j2R6s5;1v{6-1>&u%B-(~>qe>*lC%J7hm{?=n~?i-5#!r;1;0`=YSMs-{?NwV2>KK19=&jJ5t``9<29w58p?z3Vx$y}2gbt`HA)DK$6{B?V+;~Y`#XE=y6Tc78Zmj(Y?`rGm z0LhT+30mGa7$HM@h0CDvjFxHZL52GkX4BIRODAYGMuqn3R{x=sR3cpp34dtaD(st$ zslrMOk0}vi{6Wp)n3HcgckW(bujxWlRVW|4F8o=~BTMXDpd#>t z5Ot`CZ5yEk5PUyT3xxKZTdNbg{DtXM%aa1Jz7Yr?I2(g(z*YCav!YaJYhj4<3N<4< zh;Tl%6kHft@caoA_Qi;5(g0@fSktTWbly6@oGg|x1hyVa-=g(HEz>f$IDHdb@<)gf zJ(Uo1jfBhyUhk)@t$|Kn^f|`;4+SZYsh|{M_{D=Z8&lRCTT#Zy;D`|jABQ(_ostH9 z3Zk%yx4zgdZ4%qUFe^6KJ3(Oug#yZ4U(*DJEYLg4p^6JQ9??&Vy^s&_MU#y{83~B3 z$|4ZcP*K1;a9{y}o`k{LE#Cj{3hT0B&-)uAP@{-O#(~y<@jT&!XfD6SLMuYcyjDkb zaf?^bNUWXEQr6S;ZH=>5p(-9)g*#$r%GJBI$pr|)LdTM4&R<5YayNIy$L6|G?s{2x zi-gH+0ytc$IR&5JX_d8ZwEv#x{>7QwP5?wq*2bUR|=KBIW||MCGk4ax$(L&@XZE5okvfYnra9;bUL29uEHuk)Kx+Z$%Lv$ zf$qJez-@MJ$)&mqo<$c2(<52{aqXRq(H852?tD3g0|^W^_tHVE6*_lAysqvh@`OvF z=M|>=LyPK3L<2>Ewiyaf9#5eTNG!c8BxL&QfOa^_U`$p#Hc{L}=DWo$1or*-6DZ|2 z`$!BD7;$L}+c_CE@p-vn0~@B_R(e$3(I^Ns&gvkh#i9n+t?(NPr|^v?tW_e>bTOdC zaD%jbptvxBC=iN-h0v;nBC6cTDpNC6H9DthrFRemY4rVfG0DIuVC&pDsES zq0I}^-4?#1olfi1#rXQ>9KMeu?!tGdm6&uIw+awld=qR~^xSPYSZ8;;4*H3!6Y#I5 zu0a_L1?>+k)NN}tV*Q?tE)&h+x7xf%aRYWDhIat?nZn8?9Kb=l_gdIQ5!&Ct`AuZr z0PFL$aupqH*snr+{u(!A>;;VnRnQhnAHJf$a)N8Al`srSmfi{m{tPW0?U7rjT%)xo zbTwkABE+0%|H|Gv1#7N2Eh>v_4NA+Gq0>VVFi7S4ARP8VH%PpDggUAqnqC(X(^1sa zfhLWWSX|%6os4oU3NdX$n{o?sdskS?K$YO6L}Y~|h8dv+Vw^;Sh)j$I>gVed6wJNT znJoc(O|fsO#ajlh-%^V>D}Hrm?qwX(RCL3|1DB8L2j}<)Sc9I>@-#e2 zyr5`Z+<2%MnHw>5X@ri7HvpP4==4zFt`b_IwWG5_Pf93=mhljjft=*-#=eRoFkf<& zj`$48<8Q+v%tGM>aoGS8SX4KrQ|PQv5YvP&PwUc_p}r4 zc8xmQ^#X)jSBpE7tLt-kyBM4U_%zt9hC&7K#F*xw75_50y?l#US#K0zi+d;jA zPN#unh?|a~Hlu(GNBI*#up&?^ViJV88xvUka%U5uMkB^{=ur28yBaS^Dx$y9U6clz z#eA$&Ry;&77LBXLH9&kKDhr(#6q8+DFPGOh2v@5&M0;YG^YkTg-xP5|X_>VVe|DpK z5%KeK8M18{x9>TLY2oUdfc9aH&_tv~x#}andysXBcOcAWd|yJH4d*aY;I^`8c{-6c zizI2Rb1Aoplv_tur~uxPXq9=;Dm%DB*iYFMU{MJH84LUe4(C_1i_7!#i|J@I z7!3QpVejl>8hGG(b$)Me3ctm$L^PCeYp#Ho7}M)QsSd%jhGo$*A@UfaEN_IUE*cG@ zIREFHIdE%$uF54czR{qY=FNcjAv{1K>_-(BnivXC=7wsG zRhtKdBdQ;o7z$Xgpdx*pI{qvN0NsW*qli2XuO%FY4X^WTA`>;R*BlduH5)-^;Oo-; zuCit(57%qlT*E==jwTRnU4d02h_PnpXXoR=pwn*GTkUG2UaD2CdesDjvr%(eHRrTm z>Nd-rW_i?aO()&axHBAeE@o2@rQws5cQ%p#^qd~7V*qa7Ae&_cyiI(B5*RG&2KEa0 z(={7q4TnD`AaCh2OUlzWR|EDndM~a{SjHTSGPnuNAYCYVoBudAv3#qMx!;(95wpnuxwH13e2H^Ri)P_Qu4 z-4)goZgfBJB!hnA&Jdt=NrX+x2*?7x7C`l-i-`1@vOp+8Ymh2n$Ms(55X>SCUVGC` z6d*&O@ws~OC=S{IZwy5EM$bEM24sP*4sIMRJ~k{4pKa-39A1|VK(-8J!ME(&_n0Yo z*NUbtpTJx@z}w{&`m~<6OVGP(wY-7X18>}g=6rd1vA8(9m<&gQe!qKq+Nd|m^?C_# zRjq869lhk}jjGkE*{!P6u9i+~rM8Q1-R{(_ai@AQY+Xz{lW}`6Xq`=mi|Z>;B6*4C z8h&L4hK6BvB&r|i4^*@J@MQ4D#1s)}Ap%3WsP2)2Xw^4xCw&C!5=6j(+jt9~tca}H zYHUUofsT=f0mX>+xiy#e0VS@>rE-0QR?x2#G#Ctq!~S^OA3^qN^m^6Pb_v}BTZvxZLSUYlUtUa)Uy0pn zFv@Q{H(mx8jwi59faGI3O{ir$JwPF=b|B|vyjRf{fKS1cHNWm$%@3=Ec?J?Be|V zY%&3ib=p3y*XpHm$#hK3F|?AU*2-3+>NIOkqiWYHR;^++>!n7;ZkFxSiqok$Z2&-E zHM?8422Fd~t4_zwi%EMjYWKR0@u)YO&yWv9D|riBtm3ycpnF2`$I<9h5Jh0P5&t-z zz+!+`#4p#YRIpGqeBw(9aWbrkby?9`Fg>iFZ;~5sjVd=3u+}Tf-odI(XvL5k5gbe^ zu6+(AbzF|C`Zs;i!N~v`|$O2emNNT`oj*0x<(M4@vuGURePs)yP?!dT!6qf z1YNPv%6RE+7OIBFYLpnEh*oFbPpV>gRdHSPGm^j}6v^~eB5fgRoP-v)&I&7(7K_RC zCjxZuWOLonEZUipP#V+ApJYXDCC#rKRaCu_-Lm>Gn z0)d-VDt(Y6Xsm439KBXD>J{+QY?iE+W1Tuy8!=hU>eP%u!y31pNvAR$)TYD6WY8RT zn}goz<>mQex!!^0zl#8`Y+UPYa1+AIa)XjN){-!BxmtWsT2PTXeenemJ~d0S71nQhs%Dg`O3oY|;V&nc&cyVMEm~ zg-{9o`K>@XjLzxe=H`5MKAH@MR*AUlBH?(rEAi4=Rkg=BF z;A%0yyu6rBC&OXC+if)))k?*(EXB|z%aF@}lPX573}6EGs#vOHNVX}JOxZC6#}cX~ zspcp(TdrDCSrb zw8p*Gpx2sC$G}N96ocD|{YPQ~6{lLsgWLFc!_vUBP_xI&`?LIO5g1gV#BgCmV04r~ zTU)=n$+bzB4!Zg~WRG4}!`ZZLi~ zhvyX-zw?W;>0mtRkH_87ur=&e2dAZ8%j(pnc9m^aicP0bHM6fBfvx>^r&qWb57v#A z4^9G{BHAk0!Kwpy|j z5OP&axomov!d!F+MK0GtX4400G*ak`bftj!3FjJcz`&>UY| z1kXO960-#Hf!n`wyA%2nbg03ABD(5x<4xkIn%>o&SVbfTRc;f*W#}wI%s=*PjoaVQ zP5_i{2xu)Ap~cPKfWU&iA_h@ooK6Qe&u15tvjL#4$>el2tPi@SUQ;`*iKi9rw90mB zT(2SYTgs?o-68srzhd%Jh>#ht2_!J;0@3Z15D|u>>5?cN*IKt4l0l&Q0jU^a1iIn( zRMffSbyx8F1UkUul~B+Y0@lKWVLPl#3GjjMGoemf_5}RfZ0=yDf!f4tnQM1F?yH2^ zi3eQJTwh--m+qtSD-I4TG+;ScP{1bg4fYMPd(bZVSuU<2sCRp!Rm*BGJA9h>YH{Pj zeX`iyE4bRO0TSl(tBcFo6wm_VN|fUGa|2 zl>rY~a>JHtmSpR^sj?-FF9U85o>rtm-5fS7$YG~|!>Z?l`dPm=?$`R=dcW7Yyf|CBq1eoutcg>$ z?gMVCu1@-@_3*aE6NyuX!xO;Hj%UQ(q>6g-`t-TSb2N2$-O@GUru&OF&$8<+;ls?G zaOjKDYb|`41=ca%&p+UonsytLv*E zkfUQW^(d8F6N#)4tf~{@7&SaSxHAly=r%t5BAt5`VR}9pk2;fab26w5J7%vbomP35 zy7+!m9<((;UBiwu>Q=}I46jB^d#!(>uIbh!wDO^#sRC~lTHcchYsz?a_!CbF(~Scq z6M0s`eA}d;TQv8@g8%@407*naR3%+rU(Y)P>Ec?rZ|IotcIK|5iH+|hP&*Tf!JqKu zjczZbo5GukRJ&zD@|Q5BhP~VlkV~ zrWa?!aj(~Jwc6Ec&9-dKG!+O#wg5~S@K(tLkN|2WwdSZ*TLA#- z0c@23mOyF0)un%Y#7_dD{T3deapb22EuciW zV4>FEu=m2=PNw6ao%GUCeW_-NQKxee=s9z9T@p9mIQ4$Uexx4HLb;*Bc(aEB`VY^7 zM;S2Dzs|fmDc=2y$8PK{C5p7bl@es$chkNS;p0$cy| zCqy@k4lwa9;&~HRJp%N}xK+hNur&$!@E(uLMCAWhv0{}dAyq|3s^==P5XZNxi90`7}v zJYnP&WGV0RAbeFTnBV8K+3W%gn8BcTdfIH%tJR8AE?c&(+Lmma@RpTjh_)e>AU~-t zMTuoYs2HLPp=wE0ORSh4KvqfT9gTBT0NznqKuCs=)d6n_fVVP$vFn2fzS1AEldf1S}1wuhfMk&%+1W*h|r{n8m z+D=T?Yhg`bR2@B5Ter{k#q44{>h`+jPSfZ#iaugI_t#(WubWG^e>{A3zi+J9bRJcE@8MqDxV*JoTwTql=ab>M-|cs5 zjf&&wmZ@l}pemf93KoDSmTd*ZRrQimbL5IGR&5ddtyyB#1iS@;Dz*%wE0uA;b&6c=S32+I|~WOBg*;TNAZxU|ZqWQep_ObUf0pHDE^L?L~WOP^%YiEJj3&8Bj> zEXx#_LMoTs&K^CL@;h>2w{Y_G=*5=@&p*o?Jy5y5x}ERVYc+$HV4m*uK1&0D^{_JE*XOXhZtk5yyliPD>tE)?x-eOpnMy76^(8?L2=7BAvcDJPQ#ezLql^f~+G|#Ny_9 z0oImRXJ==p!)|%hcgBOt*|2doYE4H?m%XZ^p4~q+?vS>q!s5&Jfq2VDct7536!{&? zCnVAiB}^L(wH%WLbI}26(2YABkrFDl{^^G`l%F1=V;#~IrbS9dcutlTZD9>J7cE@q zS>txNjk!iJH`E|nj?w8}F6A&@S(vp-fn(r$akadUp4u0}TPj#$ z4)BvB70L=z28mK0z)deW3TMl##^HoQSzYM+C<6O7Wy{^6F&g{=wE4$%DIMVXLgAPRsnDp^aM>Ah2=E z8MUlo(}MTL2K7n5Ht5!-(*fM81UYQs-4vvnlxV|Hth@1aSLvCn|j&+J9DwYcc9-VZ0lD=n#-WMuPG&j``|z`Wom$DCUd@S~3EjpHx|p?EX!7PT>^|HiHt0Bip*&;XULod2nIw*23K)e3ZSZUHB)Ffa?_UUCSWhA z3PLW8t?DqV0zbXhOPGApVX9335>BofJFDw)ot3WXfUaXcsToFsqwj1wgW5YdI8@l|^02=c69 zGDJ!UV%!LV5b&LepG zdMVel`Nd??nN9$E)z3%OY2O~5%KaACZRA?z6OzEz1yi2kpyg>GCpOfIvX6e!?Yr)y ztF;4R4YeArNZN6)=F{m>KnwhkP7`$4@2{J)3j+Qa)>YUhav!L;^;!Y%BEe(-C*DM) z!x8?|@`Pjzq8k6Tj6~_iWB~$ewA-4YTBcIAHOE$MTQyDD&?Qrs3|+Pj#nEI(70Mc4 z0)dytpi6=-Kd}`J#9I;oIH>4+MdREngbawc0BJc=&g4@XbF3Ej)xxfl-<5MaLi!b# ze9k4FaVO972amG5_fk9evPUm+Nf2fw_YRJa5}D&vKA9=zivr6jEGrg^3=FsgQIY_) z8j5Pksv{_lAeT7F5>-=_M3KuIQn6-=Ra3H59x$#d6$K`hPrOQQ-#vNxd3yI7t+3rP zbGcC-;chND50 z9`Gd86qV?<=i0tEAw+mW0GGT~7=d&$Y#tyCM%jlcAboyuHtC&@o6|w*tZxq6QoqS` zo0)bkQ7;`(_rM}-g&^o?0JN}+wWGmyp^J-3j{#xLYcI^s0L|aDimyffq^^oYfhcqnyp}p$V!Wce!dN!jO~)~66{}IUnq|9Lv1^V| zF|~@Nl^v~QD7GRv3U5n{A!Y$gId36I1;AA`ovVX*ODR;8d|Anrq^u>ROg^DAM@nv& zPrfJ~JuVzP%&1%~`}>F49LsVtC+ebNi)x8i%A9NoiXn(1&ty%dShuB;A({$rY9fTf zseGDEZKe0_AHVt{vHcaFd2Wh_RXyFb3#}5@tto@1Idb>ePy4mAQESw1_PWi<*>Ju9 z%mrT}3|}Q2(Y$n%dJRiS((*)NKnzO^WL*cdXp*DM#w5<&Zcmix3r8u=uvOr#po^m? zXcX05QVw_)Rx0@Pvf)&N0~Jj;=I-%Z%fy%stxgCcDjtKVwu%zCMIJtSd3G`Fo{eh5 zo;BzwgSOCb6+5+Tvvfid*e$m31r7q%f+330i#VEELM7=e9Awv_i8SaK z#oyf*vTu-5kB6sDrLW;O;JOx65uqK|m&Xy3g{xEY@t{4SL2JDM2%B&u`wm|(8m(%% zq}NJ%vt+a!qhV_`L#b*?RaZ(NEHk9CCV;rg67w3D*4UIGWK1z%(g%@ zR#o~ncscRBTfH2DrE7ECZ*@D3!LU1@pSyR(7OOXR1iBf%_6lid?ELyWp?#T9t(cS< z^_nq4l2%zK#>j_RBZ|PY5C?>Mf{R6gBrvQ*pTMGQde{0Wn>0cf(*%Lh-V$1b06f>o z2Un6X9;LMjLY;{b2$8?Rx+Q!*40PAo)zxG+>t9^7&d;i6qtc|Wk4|OsRZ=0g@LXbd zb=W;ET5GKyE#|HFDORA3M1kx{E(i;^+K);n#lrd%J@{vv5m<;Rq!p{^6%p8edJAax zeBT05%@F8M%s|DJKVF_7_Pn&GQC>}vDM_PVQO1~14&)DO~#e;{r{Ri3IdjOf8 zyP2JDa(nj+hmV-!CxFt#^eZ*Ls}=VR=Fnu1EiPemNt;huQr^;JMb^>(b6rOJ1t4Qsy1mm@P!@2 z=J~Ka?zj8B=5WwHKOcfcEqsQ1xd73a-)^BGS{R;0_kkJW{CZ~@ot>Z-A%=2^nRl)0 zPUvL+@!4f%hmJhf_9@Fzi!kIH%3bd}TOrhV6p&-S!wc7qs0a*Vm58qiVnDQWAS+lS zG8dDf=Av~A-)f&ukB9ptXt+&SXwc!n&*gf406$mPV1cRS0zSPtU))?Qt}m90^X1j} z>azdZ5g6V(ePIH$7A;TbqESIe(qgX#+Jbi*+3hz(Zj5_@5g08-V;0hHu-u5=1LYCc ziBQI8XmpOGgO0D==OX<`TYm^z54*Uy?X%QXW`%>0DJsIOudk^+e&^BNNaSpDqa*fU=k^ z3%RnG2e8GQB{Qa`NV0a6VRjOQofNyBWcE^QqM+=bo(x(>Jt*C>#F>X0$oyxRV2XWY$``)R3uRb1i=U3-&G{6@S z-=K%Pz3;`KHPHA-C@Mr&kO7NR0sAH^CFkyIR34<#c@4WjZQau5|TXo@C zp`8$QSKkw#AnBrGK@SjU(52+L#yKuJCrVMpO&F9bGr@>+$(wj+z#JK_ov_5G0pB5 zfx2ob{<2W~-x45@%U1>_5%#Kjw~}@P|*B{YN0=Vh$g3iDw|X zk~7;{Zcoo2n8jnG2m-7VQ%IR&$`Uhxr2rfeZtJkctRv=1a-l32z+bqNUf~Q)(_8M5QPJ&2U_`WxzdL>F2W6gv8d(xIJI9N??3d$rg1RstK3M6V~W zJj8Wj3VKfsr3HbF0)N_pfg0}f>X>ZCAdnsSUIJ=*xfOHxSftm6UJ7O3FP+FGAZ*cyk@QdSCJ41agOI^XM{#S6^D)klTGTkitr0SD zX0P4e8Rinv^+Q27;8mC@l#Up65%MLe2)EHI#gbZtPcbV+1V~-*0HnZ2#Nd8K zl{a*q6}9~|yK@4lOL%e2>?FCPtoUN*PmmRs&GSskRtr^~sTho{vz969hEy(>`jhE+zU&M~ELV8-^5M>ldzph5Olq%q zvX$Jr2RLl+;h(ZQ-{^&%hLJ_jijCUvQ!Hm)`1-_Yw|dsC_1o26yFNYZFX4O(A9h~9 zTWkG-<26^soWAu>>_m7c6w?YwVk$_$?*K>(@0?!B>UeFO^5F`-KxagOmz!(KP_fD`+IR`*BTy8#v%Az8U72+kQ$E{9H5_MJ+KsYwk+OJs<@_<)I|y z9_jQ414}=Aj1j>q;XK^tv#qadg{2Nfjvcil|imOu|FoG1XBUzd>5&=@{#*a<{ z0_Ye;9<-6y!iEG!q9cJYkNXFyL?UDW7^K=U%C~ zEiJ!eFh?eP0&jq^39WdnGf77-!UvWCA<1k-h4;_UTQ(dS9B>s40&I)A1QA@Q9`hD#E8MS9NLA}lCA}JjI8E2f@!EK z&^DtG?sEh}tBfveq7>+!due1XLm8prn<9LUng($MOl?q$wbttK09q#WbP%9tGnh!Z zM;AfodNP2(@E2#GrFV#2v>${Ru>cA6I5d|mkt!6Iq!WF>hKu$b7eh@7g>n z=0<1Ez4A^bz80eDxccckL;4eJXHp&0#Zw4*)X4CF(%lfgecDOFCDIw>!>V-joSb{@ zYYi|HU!0%Ws*pe4W)oXna*I!IOZi>3xUaK^00wtx@<#@DY;Y&KkW#s{#H9H`ip?hg zR3TS!A6o{o6nqp+6+h%%YsasQfbseDJutweL6uz6MeKzdO;DcgI^p!LC z|~!@<%V{`&_vt_olc~neuA?XErTmY3D6+M4$=zMC2SIkD1|04 z+76D$7K@D*Na0Wm*f7Uk^$+~jGp$Pi)f&09iDRMVt`VB(;?8)voGh2<6FYOvMPJ%K z2t7l1d1xMq5{U7S&Lsp0VLH*rCL;~+RrC-dey$}^Iyy129^MKxDa_r{>DXKg9xDEX3F z05FA&T0D_*`&@dDN$oM&Bes|>g6Fi%@HNIx>6`h0c>eU68u^9M? z6ODt=SAZ#y$pC@b?(3sqtfp`!jW-Pm5LkwlULEIN9Oj-L=AInoc9P8AN#UDEFaG66 zKmNfF{`&2AezWyzm*ZqnG(q5{XqKwkvSi4j2I4O^pH{gXAh4PtcB;mpQ|X>oYc;3U zX^-c#@%43mFiPjyhmW2;dHnGBU|ZnwrYb004sdWWvzIw}cC`KV%ZDGmdhp@#%g;g3 zSrSf~dcJM*rzN>lQM*-mZ|r=~xE!CJkJ_Vt`|NB4_ap-TS}x%$kAoZ*1^R^I!21ji znBvYDp?&xbdJzq@zVYqTy&5HkrpE&x3-gubO-vDrD8(f(Y&q-pa9WozLZ{yf-@Wxo zA@+#O>``a%bo*W&d zv)RLB_QhW6>2~t*tE1h+B+C>n4Zi%MYRWaBYltu)GhFHd!4P~mhx@%Q_ix2zFAs1B z2UC(+TjdQ+Qgl6C6kZ*rpC4qN?q{AJWOgBeR)AP5t6NmN9EWeZ7}O#y>-L+sVfvteU8=?n+$@nm$qSb#-mtv5(yiw_<>e)8nu z$?={fGNvx85-TuSu9)HqN4b-i$6NPyAAh=a?}Pm(9~TZE>iNBznr&H3%jVi8sb4pz z-Ri}lF&)&;M(yG7bTS@X&ClUe&M*vfLoxUQ5BNoDv^YS22?F!NtgzD*12{2#>n;>E zLSfc-VgI`YJWY!!LXnB77+#jJ@3JhmgSF z+k4@MD2wnzlq&0TSkX~ALlY%cNf)@Sqx9BsZWlfZ#-3z_WR~CBJHGqi$%98vw|4e( zMP8OINrn-aq?m9&J>WB01q8+wlM0h`q(a@`A=3;S7iZn+8C-3GnPqZyU1|3YlbL(> zA3S~f$RjXA(N$6A8J@|COp4DPW)7Yuw;!Cm{A&O4$GZ>zlzjP>n%%0($-16v+H9|; zq1}nYwlnHgr{m6a)Ex~@&(BBhF&4hb-XpNrA9p^!?L~-a0()b`cf4-{ zx}j#o76b~nHY^ly9e#sGz-vnT7WLg3JTWx)+h7xGz6@ZiJ7>#l_u&Az=)&0xw-VJw zhzf)*c8Ge?pjCW3p;1v30tTrr80x>TMW@GucK&sZY;s+g0A7WtGxB}ozp7jWypf{_ z>>6dk=wZ?6G%L0SNT}hynHNN0TrTnI*^^KH{PDZ*{r-bLfBItYBvDjeoUmWM%zXAV z`OS;u&QVt2g^HonVE6_90`jXELPbY6y#RahYxZ)(FTh?EodH+{VFmcXWldByEyoH6 znfzgvInMHltdJ`z1x`OqagSc?KYX^kbCAjwMV8fzMTOxthF6Nb!U?JfIEX6>#gtl1 zl*N2qCo`&=D!buUJI;cB0!sk2%gk9 z$yfveov`cmuVop(B0*?vVkA&;toKBX8RAiPVyx0xbo#dhZSxIy_qzhRrh)I*pr{M| zbF*%)7mHb|S*5cr(e7}&Kxt!<74Ktmzj^B zo_zJOKJmAQvrFP!uH(2rQf5OXUtz zg+!LkGwMl3c($AV{K598_g+1Eb$p!R8CGXFy~wFWL19Fd2jGQbA;V^mH6~G&3Qe8s z*NthvHXSuVkaTfzHeW8smsizhhZCe?u^{q{DGN2nsF$pYW4fP<5mkZBoE$v9_vM4H zK0erboIQG#KYWqizPI=2PXL+SZNjZd$x zrfzdy!;STDS>O^Fb_yVYtuEW3(cw`YN}>8s=o*)lp|t+D!pwaU$(uPzPkQL#Et7bXlb02HPtFZeWS<`LnPWZP3<2Qt)YzaxXM726y0Ncka`0lf|-k zF`FzFXg7z4U||qgokN{~4ZT8CqsM?O4zo{}6%W+V_&K1}LldKN@g-+K4BJ&7>N%b6 zGiy{g@)OslR!vH@>Kiojtw2Q+mO_6 z5UQG3m6?hPw;;Nk0J*B+T{eVU=d~h)T68ytxX*;SDUO1^u>#)+Y{(qX^GE62%cGN5 zhbP;|sr_U=krz+$(v#iPrw{i&dAR>%FO$ei1zuxC1F%>00e<8X8HWFBV`*J z)3&8S!#L|z&j5ivu?IxAJ1$z=M)i)RPBucShztV?B6a|{*yEt@*{tMJVB z%clo?Tye&d)m+=hKy~bz#9GNh@>;Rp%&#F2~S0^FbKdT2r8 zUlkGRQfOiz3*!tlhmsSkE8`nMTK@SUtgR7oN@x&@ZsJCY-v_*Qh-k6z1AeX1>|5=@!Nq>dgxdGgW6U%mUs&p)~QY&XFk6!dR) z3%|dA^oM)E6+(q zUSdR9R#X{yz~^N)1#C79zU?TZR%tqDOa`sV=yZB9x>#KGr9`=yvq zS2Ve7Xoe~pGHe$1MGdwMgx|Lz4G90!S*E9G%Yj;@yPeBkUCQQby_VO5#bR(ZKf7MS)(pOUg|_8`p3QMZT-Wn;gX@!c z-JoQwa6E1;Yaac*F1!l24hEtTU`@lS9}3zU0Yc1=%>ME~?GV!%QR#0CukJT+0qf3{ zA&)2_V)KCr3>?lbrmcz%*sI|vHCw6boGBJkiGxRvAAkDU-H$%Ld-uuiK}Oii%AdbX z|L&Xp_rBi#^x^LAQCb#J;N{-aQVWg>uL_oBxS661->(RF3lq)0(T!xd6a@@nMlP?}^gTMP3F2||7WU_+Fi?SfAysQDHQ9umF zWhEv7jMogI?P#NRExw+e+~rPgY8`jR3ij(0MtEjGKGJ$`bs^UcAtPY$2{DZl@fUfgb&nQm1a zG_+w`8=b=EokuMgijBwZFPBS1o3sicxh^7BI8>1^Ldb+_dB>}`ElPcJ5W0j- zAEDQg<4xTP;;hr>;*Akw3LngGFscdQ_ZqzvkHF9ZKR{q!7jmw-I?LK<+@RLu!`tZ+RJ$Sa0C`dG1#=lh2zX_@x}FPFgF zeHmrheP&tn2+WVe09~OwVeZW^bYqOIXky9GBw0-sSU_DT*4G4`FIz&&Qx!!LMTTe73ZJSPY}1xH6=U43o{dh=CjHCV*=#2eKOAbhEU@{8Z4R5Ye!JeN*p@0Pd{HRo(kF*|yRV8Y+v*K_lXIt9$rn<^{E^Hi zrNR-D-pU+2N^E^~_~Mh)&X;obWnD@2O5Cs}4eIbMz{7^sYulqibGZP^+QqtsNdhDP z;foCKsxN*jzjgPRQ(f_m4SpMEVs!B=Z~S8O2|hw+D*{7Ww5eOMxmM^6EgUhX2*ns9 z-xBebaSvxMVcj{sxEx+B&fVWo!k-$h16RgyH5_yqG;ra~wW0tmP>CoV4?;sdq>J`s z+ztmhKe~dbW$=j=7YJQ15RwGY#b2Tn8u7Weg6NbnPJT}yYE28!fZcE5#rF&U3e{?v zkA6l?poqY3Zf?#eqk72%1XefYnkiK^R%O%2`>!56xc9{u_dfsf!OPvFyr318@=iv6 ze8@aM%%ju8gD7AF60$HE#;5$$IlYGcaL6vk>37VPCT!R zCmpNMD|4foI%)y}vziTaI_<;m51Gq-MB!EfLw6#vGv$*RehHDr<~@W!=*%0M_+JY& zaS;UOO<9z%uR^p5@$)sRiJEtFdCXN}{BC}xkb9xAP$)tz2U;vBd}q1}yl|jtI@Fe* z9+ev7;CB4&Ro+mfEXRN_=mXL%c8iWy^noH04n}hWh;!Ma%G)|ui=nc5Ep?+s>t#TCl)vXfE5)^QVR^9NT!pibdlje z(8cjwv6zRO=J_JSLQ z+|VpK(Mus3k(xtqox6P?gMLByPn^JD?Wq0@Ky1AWU_qrsV7&@P&+!Bw<@q+itFs~S zRRXG@5BTmM*U7~y*g{VwV?W(E?Aj$Z@2*V9KR{r^ey6e`u!0sl@)_;nBfi z>fj`s<0Z4vvRi#dt?Xxn?Sst0L5j`i9r*e|_`q@*a@cCa;p!rx`y3c&!>@~hzzc1x zhxg1%n#*1~Z)v=(!A~a(Ad<=#3I#Tk&8Kp?0?YA$KsbQ|{}%wg39=+G#e6Q8W{O#! z%?lv*0#ugI7IP^!m(aLe#rOR4+foXt6VWFjygK6 zT?|eqeMn%vRt<1i+0^ps#P04ZNmcuk3lOm>rkPA1YpT$yl`EEEYLYG&4IyuEfX0rP z#Ea~~gXGTFncZ)s?8~~A?v})+BWWTCTc1gMes*zogYIy}MqsOxM*gFF4i0)Wk<(Tz zXu85vUdzWNY8_W1e&nkR#1ivYa&0^Li82iRPLgx9y+{e#s{ ztn~FUQ7-WlRfHz^Hv{R?>Lo@9tDPEwfA}2gO{q69WzcOy0<)Ed_X>bx5P`vmoekcSi@M0@5~l%j%I7laWHysZCE+U;GnsUukOz!a$Yr=fPGk#; zz-qFfE21v*Du~dyqNd4)Dk;E%%#}2$X&E(L29#-vFjSHYDV;0YvS@3vsmUeBuq{=C z&zv!*RqL$V00agy6H>a;eDX6p5=kRQJD`fdHOPtNiZ+fBN<3qFaGa&Y`3jWo?elVCW~YxgRCs zMo4S!Pee#$Rv2&@+;V*S=70|}!=teB;LnKDPx$ve0T2GXj)*ztiQzt=Q-Wj&Q>%2M zI#CtD%fe+gT(Cp}^fvcXfq2Y;Zu7vwd>erQf%(;PHeY}&kCp=Sd&obWz}&gsopgMn zTU~Pt{B&oj)x{rjih;OZ1xffls?C5qOwV7z8ev{V&x$mGL9doqop!x!DOF2`1ZF8E zUC<YgXpJZB@da;|pX{C7d$#*xYcHQon=)(5MfjOYcYi#*-KC)W zVSvEkoiJM}Itp7-xsuA4HAn!q!Z|8$N{k^E4GDlVWI^H>KvV)-ltotHiyT{!L{5=; zKy5N#R7JpEAY7BmwpKD#KqsaynTArbbXDXeu3)QTMU|_XTvmCdm{N*Koy*%IZ_9$E zh!S7m3K=GwETj+F?4iNuK^ ziLFYDIq3D^?89>cKj`dTSBwjYL;tz`nd|@yY|)9vAUT4<5IRTwS0Rc~(0qaf0ylM5 zlU3-%LL`zhtFNDvWzmJYKmCTni$G$bO-;KIs!lr=FHb1WSJV3@AsV53hqwR@Cusb} zOmwLP=-BAZ5V#&1eCQn=5RL^(#M~e}fo9!zQ^pe*Y7L@l-v)j`C|X*_SgWDy63=Qi z8ctwnynA!HpAHe=u;BEt`sdE(D{ffL^?rN?kAw4$cWDk0n1A500(?oiYYEO=?q;)V zydt6SYZNL*=H6BvRD%Ek2MKHezc%Z>hNDp_TXM}3A%WR?#neny5qOyuj7(8G%J7_F z*9ODJU?P`l2}asI$?YB<7jh{}VI7yi0N^GLwTO0Vz$cW&oGIiCAy?Aii|PSyA(VBo zqzM2OL#i5p4+KjVG(cB8qlv64v5E-ywd-!YWk3j~Dq5Oo8?s|60N}t-1x*zkLp4DZ>@!71^>B)*D$y}rAv>W9{#ctKWU#9|E#(-~qtQaETFi=5_Ns5`>+`*&c zmtX8W`f&T+AD@2u&ixO5_29$beD>}yzIgxL%<;iw(1lU%YQGFPm@noorI8GVdK!Md z@PBj%@Zm35zmv-z-x9Co08J2n_i{CuwygEgnMXwbqjx-cEf!Y@%q0bLuU$0vG0Dh#B86Z$oFvufc^=;W9W ziKx7|D)e6{fq7Z(v_cY?Pa$i0$O!J|3NDw+^Vw`TyBsZ-;P?tnO)f&%C=^6}Lj_{0 zgZ|-jB0*qR=vn5gS-oPFOtEV6H9#sRjB<2Y)-zHtDio<`-P*+=hM;o#l&%pgZ-U@gPmr(0oV-ESOi^#{zKgizj*HT z3U@7X1r0)kPOonUd@2uxKbgj)YzU7Vu&J5I_4vv4DrD}$6p*3TFr`hZkg`U1c6M&5STyZ+@ODM{$jDXnqST? zCTHXBpj+*COWjkmRuS4wP=a==WRW^mH z0l%kJ(q&!YWu6f@p5qlk&}7v#oswOynWc(RDr;7W<)wT+Bl0<20qjz+-OGj`6tmqe z^`;1>^&Dg`sR+N(>_%YdlPe(VvShXd-#aB*BBQZ+jVmhfZM->=$pPw;0G|nrB(jDo z8H%h)q9IAP3fN24MFCKn1#f)`0JbKVEgiryG)q$)LjeS)^9+E&F@V72T*ekb5XMPt z0lsbn1amxx?ut2@Tr;$q4b-)Yr8ldkTD8<{)_YyJ-=kKCJ2z?-2gF{@irpx~zwKK2 z6i{8QRI#)wd|D1rG6O1J)>u=_W)Gi#{n6X+{PgeN`r%){{jdMeJOBDOzy0A~zxSiR zegEhG@`qpk`2F9$_2mASPaoZV|Gl^W^W9&6{@F*8%p(Hxdc|h{(0NCvKF}YqF0t5P zrML-j9*QBPP`aX1Hi2gf(FoRmva4v@2o_oqMmRfoudQ@N&ncB^sg>b)g#iQ1fWbrWQppI zMvHfYDiABc`%mcGNboxoOZavjZ{1#2$HHL6JAbK1Q2@x=xhqe>v0(nPq61m00ATlIRi*=$q+e*x;Mmj|cKUb_LpuXeT6t(7}~ z!x|MpW+mW;Cc5CNwhUU}WcSHuAN=;6xBmI>{_6ku>EHifzx&bu_{k6c+gpG4f4dy^ z&%gZVzx~la{@_|q6!MeBkCl3GVC;B}fA2{?V zjsi|~^zeU!3GH=3Oq-$CY0x?t9UoT+>Arx_8G#-SVxPP<$4;2pH(e6hT|T1@Bj$@%$U zJUSip8r@c@Rn?m%um(j&0-#(GfGb3m|`3O@s83S}J-7^jP@ zz~pkNL@u4-n4&Clio~mmpzE?}Xz)!KOi|$rrp%zbS&qt<0Z}QOB^3=W=ZHl|gfE*f z!+q_Lz<^CaU4X;DU&q~)XaixmgAt?)y2L30qr%IEf+9;QU@T3y4YOifRm;*v5fCN_ z(cn!onM2eCVnlENU?>8B4q`7^P(@Z|^HM$o!Y`$m7js#eDX2WFiX6HKX~Eb_u@p&H zL{ozTc7!`YL{^eG6@-0;s<}Hv8Z~FoX#oQ3G;8f@xm|WZ_|<|tK5RgT&}Y6_ zGNjW+sa3NxiM?O^`~UjIkN)Aq-~Hmfw|@BRfBC_W|L6bt??3o&|M5?M_1;hZ?!BM? z^ZUR3`B$I*ap&n>CVOmX!f4#PhGWIuiRg7suQTHuhMze6=PEJK?fsDmhb=Zv>3M0x z=`ThsCTnxRi!?zy{>*~|XSEr2=_`Cr3ri0bCgwdHC=kR2kPIOV+m^U0=F7Xa+x#M# zr%rekoYL{4H$=eO#q}L98_us+Z=k@0Xw#vII6~V9f0Q8gPf>pF&xktDsAp+R@V zH6aCoMhkxZJ&=nC$Q@IQ#q9iIJf8H%gUv&+k~i}T@l)EV^ZomQz`2Mc_u zVsnm>*QJEQ?n(JAA@f4WJm*tSxztl8@i2e*3AK6r35b44T6|6zf|21qsYOc1r{~l2*o^K%o#EdIK$9*Lk6q^=t_1J*^z}ZWG@j2n-T-}87izL6f8bp zk{G~VCD|h|5Pq4G`^YmQF!vE=@Y5wQKqR~y@MUG5Rb)|-Ab~*+(*cL+wyHt`10*D{ zfTduF2KRcvtAboUBQkm5Bmlhp$mZQhj#asW$`<5ej?1KS$)jR6$rZB_Uj$K?%z?O2 z05KT|*8nP}1|qN$2+MTEP(?{(3_~(aMV2{D6>DYtv{7w=zzYyqy;5_`8i@8vCQxy| z#$p22Z&b|QX}t|%G(*{X_UOa^eCM-|{_~3ufA`xT|Km^o{;%Hp;eY?lzx~bozx?5! z-u>CV&ptTZev(fdY9iCDmo8^#UbkFh)h)yRg%X&bP7oLkniUY@XO;h2(6hfc4PLLG z1EZb43Y#tI6g-hkr!HLiGPp2YfCh^WsD%0d#t8NE4REK`9k`zXgFiqhcIA^2$b@<~n?F5hxKmp05auSc|O}WpH~s8~`Y;^5JoVi`4)?^ssrV^AjLayBR6jr z_J!AW5M|xK>wU{5YzB`k&{p;(T#8&?o}W#}<1@gro#C+AX@kYJQg!&UQ81*W!X3)Y zu2|UOGSAr5<6`1|_TbCZ_Q%PWe@Z<6@c7yL07p;%u>a_H`;Xq;fArh^$G_Qs^dEbV ze!c(bH^)!kIeGeS`sIHXc0b7P{xQ4r$L#J0eDX_M+3B~K`B`~6?=6>8(8Cu?cS#VW zmhbQ1>I}HY@KzjLzzn)AL*X4=w7qZ&unS|s{qAtfyB8_}elqwHeCs@4QpB<*RvH*NOQ%Id;jt{aYhlSKJlT8SPw9IB@ zE~{`L@X9f%696`w%%_iY>EmK9S;!;`=@TxUgz*_)biZSn(|NW8Dx%4NM|Bx~8(P-{ z%a9#QtJr3}WY^1f1KtdCY)yt+`BgyJf~^3um+ED+-K=y@8|_xT;#h_v=29n*?tb;< z#~*z9{yV?>)xUrIhhP2qy`TO0-Jjk4_`O$;zDymwWYQ;~zU@YNKAXZ$h`tx*tqJ4K z|0V?Hj%7F&{DJ9@;&lWArv=#I+{qW-m_@fg@EO3pHW0G3-5js09xCtO=<O!pYTg zWBld$<;C=Tba6SkoK0u5i}}^%e0~XHnpNa;?S)xkJFoB*d_W?uU58c}+}mLow|N}k z{XzM~{D zG6<@&a8ri#{}Fc<-fd(Fy6->c@$Ss*%HPPAA<-cRI|5m?>svmYJDlkY!sI z%d(i6nI+52?3nH3-I8p_v6D>C%lAE zm<@WT)iUXHI03_etY$zjt4?jzKy6wUmBOTy+jSc7Q6m=UWnz;`suA;bQlVBVPzu>9 z5nChSXk}cLkfo9ER3e^EE;gzpYMB6_3s^CkfCGT167ZBfj#2<3@a0^#oX1g%_&SNm ztWp4mIju&UNn1^DW_qm32@dv)-_~>|lZ*OaB zb9HTXVR2!4deY&rn=EFnUa!zJR>V|AEKR)gN?sUg(L#A=ZM&_yp5YsEq! zqg)D>3O-9M;%LMiC7-DhvcbB9JuYI5iCJV3lf)Uva>j9-alC**mU8J5E=|T~N%?F! zpDX3Eq#Pyyn1aUvBvWwNUD*rn#!PF9!4F2*`&(` zP86V*U8lAI-)T3`OiwH=&aSL1tS6t4zRvM;sJO^ZFXxtC{Z|3(@?M`mhacjfC*`RR^Xz-=yj|$aU;RMUOsd$ ziY{a9T;9X}2yYi8^8&%k59kLTpB^0^Y;3HoZ!GWZZtWlLUw%|RJWEhU>DlWdv5QCO zQp~fjJ5T?NSQn{5PqT-9=ONxMGe_^CN8D3;dB)Z=g9YvC>Arph)d%jr-QL(-U0#`= zo1bz|I_x%+&7!xMbta?62(l!T)(Wyqv&w8zSd0prNjYIv+w@`?mqZ?IL-m#nb>|Fs zWS}}TM>{joU76_43{*$@NIN7{dpfo=3kcJdiRsP)0$V{GY)=lMFQ3?-KZYoz48fSF zau&Lhh64VqXlyW_GMG1x%%=<&&`?D{3{(*lUBbbXa4|3rwv>af;*B+`g+miI-uBis z@QA0-KxZg3^6K*G{oLQhnY^qIC)qnr`I5XR*>hz)#dL5N9|M5R&rH~z7O4Jk2C`o& zyIx|~iXB>sQzN#ig%-8Yu9um$GPRtq0N4<7!6RVPDxG?*ORt*%4$P!;nKTxaSjneJ zIaD!+D&@135`jgla6qh*+YJgpF90n7FhDQBFe_Bor?46{Can?x%x=;y#W}z*;ANNR zrxs`2;C$LlI*mdo6wY?Dd)_Tb8^z*o^&oQ&V!i8Cuf<4r^3Snui)ngO@XFJ$b5&ly<%Nm2;`?(E zz`gH1o5xr3K?u?(M@NTy`#Xn6Z#{88Zjt&(Ub;XS1*knaIXpfFOK^0a_9{>D(VOh` zJohYx%UjQK+p`Q83OM^65UuuhceZynR#zA2=VqoSCnh}2CN_%^;L2>&0H*~lL3!Vz zwOceEX`NP`+hzcHhs&yQL9bHOOxd&!y;#mAP*BaIJ%xytl-~Nd-n!Vn`ndka_<_cR z{`&a7y7+Wuz@_7<}x=02reni^mZ~_}*MB zJR1wo#Pz1*;pw>U6l_l#zAux|mr3f&8tc!Y4(2k4^H`(#Y-|x5SH>aM$#?@ROU9FT zTPM(YI|mb_tHto^e^*Z9S%KjE#6b}}|F2xB^Ta(2J2?ct0hkG>Z^DFm)}ixEk|fYc zfstZqL^P#Q}3LN-mr zrb#$-5q+FbAqi<@0hP=q;h6*s6*B}_<-rvkJJ>aj9stC`_P68u+fhBukh;OLWw^Tu z+1Y^ZZzrP>H2f%ofa6ifym7LKMw7E47V^f&m;@XUmrMdM6ESFjVchXC4rNTtVW>oW z;K_g>o@v05>s(OJd+V&*G2^mLSdHF^JQio&z{IYv%)i}O*<4;&U6@&#ot$yo?G~L; zE72>sLI#eE=^n>)%Xy?Jn|fi=HUWOHN+uGpwOaYxoeiiw0gM;a>*&ZsuVYVP#!~|E zJltL@<*g1pg*wL`9UbpOwc>9#-fpk$Y_6;?&#y1etu4$h0Z>i4z!jaDaLi2Drrp*l zw*}IKd2+%EkU8zPgTn?4o1StlEX=H|EbhF0d-hs(Z>DrDL;U}x>tMK8|0rv|6yjY! z^H{%t3V)TZ9L$yQFX;L%7du`L{G9%($RFf6fc?|cz0=dL9)JDcLx1|GoS-|{?&_`e z`BR8tXHBtQ7<|U@c<;LpEmD_5F6+uJUrz8M5l?rFk9P57VP{p{o)kR%cz;$2an6ZL z`Jm^`yhmp*nD-bQ2q*8(^Ffb~dhaD5l=k^?L1~^52ZGe$$?5LN@%F*q%GSp0%F^WO z>fHYR4s;;rS?5_b;CZe8#W7tJEumcatR@gTuH&P_{k^wucQ&`T);2blmY3&dXWbJM zHoMheGwT7ROnMc__&_P%qF32KwrJ8=jY??ZaEr!iR=X@JAmFrspt;haSAJ-qw+36} zZnJ#CAhjqsQU;1X+Dz;(8)?rPXiS9H#BmPd4yhIf>Pb(V#9S48$!y&0^J zA8tw-YD@$gsEg~XiRrJ29c)Y-?aTl|wWW@=Dv83JPAN5OI&)ju{2 zCk}Rx_O>EA>PO%$=-xJDd;Jikx`E~@ppnkTk*>ypma2i)s-dpN;hq*O0!|zn^x}$% z2g%|E)N#NtHiAjiXTGRQ`Z7*`h^ySvNBC;R8jdw21hyq>_bgg$y^iUS7=jk^N*>F)0K=H|-M((J;* z^yH+&X*by|daFrmx9XfWoz1Manp8kGqsn2@*gZL;!|1J;0!yXCBzIaAF0;~URCwM& z1z07UFi9qiBDYcO)bY(KrddHXO37*-R!SdY<2%TMRm8qBd`~f|BWtiJ30@ropw(6! z(o_&wpX*zn>(`j)-&_#TSsDf~Hc%VeUlRjXz;dJ|g#gdOcBPNBCJr^nq1qDBoyq9# z6bw9-)SpQm%%LE2sKa^Gk$e`Wn2Uu82&H0Dxr|(+q&BJP?OIm1j@M^b;dfUp?@zZ* z-yIyiKRSGWbOh=6=snas@e6;a&``CHB6-?QE_7BD@qWBmxePE6xVN`6JLQ^kTBe+4 z05G>zYu1Y8d@7wZJc{VRkMuIfaV**xi%B1+P{zmUK;Ta-;F{G+w@C+eGBg2(DQ#M* zRU_6Z1ac8qD&VSRVx3ZI@l3Sgab0SFE}K^EH0VHxS*vt<25()^OQVpe6=J!VuN3p; zJhp%~#w3o?2_tm;Fl7`$9_}L!_TUG)#)jZD^x*hNAHKf>KL82S+cwnM*xOV!*j9&Z zt?jET2kNRUZZ6JlFVF9)f%P_&^){CFHC1%gmh?5(40pHT5xq3*Cx zkGC76Vo~V+Ud%wB*Ql|_hzug0j7AZMkyIRpLmuN&NIdEok4BaWIVPRT?JzITxVDyN zR%YA_Zr40;JT}Yn1fEy`Zsc zCapm&uU>3i&HaG_F3Sc9HvRT(QVUDSb=nIs0slXLA_^b zfab^t^s-xKoVFRKb;fC#aX~X(O@kGuW!7PxbJ*tWwpp8X#%7+eo90|rh;44$+@yVB z+6h3ru`<8Dy0E&sxV61;bb53QjkiB*XZni(`vQTyTfZFPuhz8;`a-`P1AmV`m;U8% z|J;#$&JKPq{&ii6d2xD=uYd3R57RcCee{HX_27^H`|+Ruo6mRuJ?#F!rH4EjY^^b= zC7Y{@CnracWq?e|IZHe^@g{NM&#r^#a`*+VuIr;+JSFZc>0GYN^nCQxqn|>3w4u6& z{oTzisBvg@d4O=bhLMH^mhO7?eXd1si$?sTVLV9^WpKy z?%~na@yX69FjyyVkB+wY_c!+T)^>K5cD5Eh!+Yn}m)+|Nj`ao0#++$&O1C;=+S{Bx zIogCq%|l6fEY+D~gN_lLq+?Gt$l>Ac?(Ww5I@CFEZf+6~%i%DZO=_K1qScBGI;q(p zw}Jf8q;^=fPOHXg0lo@yR6uU48iJN*+C8TMP)hDJN$m!a!zi-pg%&l>tmGNxEUlQL z77`U)w2XlgQIRY{Ck@jyj;bReD~X6Qd|xrTJ8!fr57S+U>BvL3XOFa`_g2RO`_-8L zx-$KFY4T%O(j&0akn0B+1^@;e7`iQu(49TjmqY5yBKBqwd(%mM8RUTs3Nni}lFvXF zaj+$PTp15n$|scZiRD64xtLrbCRd8bs-)vJ3TnNY(WGIwYq(t&+0e?A_;i2n-O1+h zyZw`Q2gmP@PM{f#KalB@e-M$N+vDt`r>6FD$F@&!>oU^34s36Cdv40*vKc+E=F+<@ zdYfLMmGRhA90fnjp^{kCF*cpb<#Oo^HkryM(>Zh&TPWaK)Jjj~A>_U6I*49ol|UtB zN%%|vFu8~iNT!tu^a_ztC9!CfR%mt%5MSpq>8*NTl(i&-w@E`?;VpE8WF>oqI$8jhs9>p5NidYmrbDU1b ziMVv5S_%NOJUy{IJ;IaH^FP(AaSG z1k^!PJqgW-uXkB=PO!8ZCLAV6fO9|&liO)_J1moS%d}^L<0)v^gK^rU8Jlt1Zk%zN z=G@lhY1hX5EMV={+T!NM^2+K0$R&<|CqL`34vn?Co{e7UFY?J7;mdKppsU?L-hBYr zbr}3|4E#O%T>58UeZl)*FaY=RzMt!Cay{nfexo<=e2Dkig@YREE_n71vS?pD`TExn z|NK9m-S}V6zWE2=oBtB>;JcL27cJ#EbRx=b)NQOmm*3Na`~7)G38=H8*A9CT^%07$ z>H~z%=D)Z?DR1DX0Ol!gL(ze$+B?_>+CMrtJb~)ePfm{y4tL+~tZl5%uPnPa*Jk&& z7WcQ7b~hKdHx{-x7PmK-w>FoyHWxNGW)Jo^Jjn69iv4iwV1Iq@?aJ2X+{VVt*4FIK z_WaJ){PyPD*1C7DKj<~;s|&8B8ON;CFzwV%JJmBz&6HI&>(Fd1JNC9`z%E;xE5P<` zZ*OdDtgNlAEH5uEF3vA3%+1bDPfodBxi_T`%L;Wm(sWPgpdbw36aTsI( zQ4@ez7S)993|emJ74=HDRpB&AY&xMu!#8VyAz&Mn44sUs6cJ@SteAxoQinLCej28e zG}K5KsK&y}hT97Vn{)dbGJ5M%yKCY*tD-xr-gE;yRTl>|*qk!loHo*&Hr$+os88&v zeA8MSTAS++OMOzD{J1*vRXZ#cQ5%nLPbc^1Gmu40WD#vJmyXP34Ck^&@>r;R_GmsA zTg1b`1o#pWsa!~aB*2vkaHU=e@#TDgG+e2OSRo}>1Cq&U4N7{ONj$tTA=urtog6L# zEB5}~9@I$ez0_w#iHjQ7OKp%|Y(al#pz<>5oKN157um?!FgQ>54NsMUXVcEk`h?2@ z3I`CtpsD3`4wFu+kN}p5cueLviHJt=S!@AM$Yk+oECHFu9;ecHT((XDO?Y7ja8e0v z8j)Ek&`CKG0ZqWB3OP(537@SL^3)Q(UM{q%6*i5^s)qV~IP@BmN^a1|^=i3RDO1VB z5&=ghTk#NwvKc*4z<^#;Vq-x z&HYW4y^R(9p4!6E?v}ykn%>$nfUc&Z?E3tS#)8c9w1m>MxU$TIimar@!b}j-R+isY zUD8`$f$3?(_H_<*wV?Yt(S03QL>JWQV`Pwm9%bP1<7m`qe^SuLBk8Rwn0)#)jKF<^Rt_1RdM-CkZ;ot>I@I~Hdq z0JPjr%figW^1}4A%chozD0nOpJ;I@n^I256h%I0caYOL#mfF@D7;(5q%p|ErEWM0x zRtZ7*06Yjbt;C^|I`uM_LGFSk5!Sd&TBivbU6dkJ&;8BxI{Fl)pg?a9{;5t1moxW^ufE>f;>wO6rT@jFh05~RcRAOg7y0L|^m8$< zhhH8oa6SI@-9LGd-te8nL!isAM84SS)mN|X{QILDzk2lTuO5H%>zCjCF5u3;1^@Ji z(EDGjZ$UMC`)~JlH}~JJ?5t0%%o!IZRLfJE%{jy7oNj$ay*Y2(TC#61 zPi!wwZZEkvmz>)hQ+vCMd%H_p8#8N5&gEI#%8Ye=(Xq8Uv9U6-wmh-A=vtn&&$~_2 zPThn}4+WcLh9_1)8A2X;L{1a=RWpw^Hw0rU0(oR$#Ei4ui<1>qgFDJ2;8kyWYodFrqx))NdMcwj%OabLLqLSuJpay$ zD0F+qSYH8Sq?A5f#27AQ4i~V8^0^}g{LupLNG@kMhdq+R8p&mYbpSJ<;XF1npE;Pz zMC39Mx%7b?#$XN&kwqQIVjy!^!}%Z@A6>*p1Fu#oqqiBPL-S7F{<`h-VBy`#+WYsr zXZ{r;m$!A~`rijfbxqzrY*zqwx$6(O=a1Ld7o9deIQj0g>D84MtsJVy6Z2(a9+yGk zv1oh_i^Jgwgi;<)GCoeDQYm5{TP5J>#6ocCf$_4cghshQE8&TNJ)@G@G>U)?0LD>? z`5GzTAO~(3+#~>a5XKBLiCQcG7R01eX%tdmzd%eriw3M7lZ*#02R{VI4Riux5&GJ( zz+H9LW4jyiy{(w;rs1|aL{nv7L-{~s1*)TAw5t(fSVK9yuDrLd8~_a2Tm$S^LqTS3 zUV2$td|7IIPHb4_o8W@@i1M_!%JhWV?39ku{Pyw!fU)6@Mr3;fqOBg$QIBY=1rCph z=p!Kqumf-cvVROUNJ0%_279rCy-X5T$QWmku;bX_F*K4eGC(7s`E)X$K1L&qGD#Q( zpKVboCe6kLmu<=I06uKaXtG5*pHAR80VL22Fqh398gJDP1PH%P4gjypjMpq0Xg3hgM+I@GTmi%OqWzG;c0U zZmrC0ZO(3OEp2bFu5YZYZ?5ki9zq!(l=WX7=KL4+_oM6RGxUPOFJ<%~&b2P}5Uvir zMYGQwz(?`VQ}ddYkJxZ|Xu-udekM)trUOq3-UsaRabAgbCEWW<54{sSUXOnbz@PkS z-ef@v`#_$N23OCFM_&a#yY=kuA0FNM*Jt1V{@JbHzP$PCfbV}7e)kWN_x>FH;JfID z-zU6!P@EXm(@@14Cpc{;;54jjcEtt4ZLY~(4^)VRV<5|V+AV!SK=`xA0uO)G-UbVADGY7M4{j@s90A5_u$VFg z8y_qpBl5`u*<*d_WOypIFP(wNp!TJZ;fZ7LL|jMAXiF5jB^uWjhii++w!~vw6L4(_ z#ExV_dlI%S0o@Wi+8m2+j>Wdd6FQSX3P6Brw4vN_WFDW?V3uMQCPdq-=F_8v_wTl# zd-7s92IQPRGxNpY2I|XtYf3uG@>@%C`|HZOtBULM z(;Eu2>hjV7w>m3|+RF-n0|QGCk{cJ18Xgeq|2)R$Nm{UPVQhG5QgnT8T1#;bFl0S7 zCCIi~ctbg$*FaNMe`6KCzneVVhwW=0g|}n+dhp18;_v_f7`h)mh8kuPaTIWb!wAd( z9NphT80x3tM|o5dYYfLAf`Syopb*=2idlzgdCIlqcFoys3r@${^yK1%6F9St`MI^( zsg-H>(zM%V)JlasENW<=rvp2JM8G@os6iH$NFT$Z`Z^Gu4J33gn~IXK$J8RGM$7^X zGb)5enZP6i#RaiVBetkT!25!tgGDK@Dg}0x$f*{)pry#E5dlhB)m)o~Yu9q^YK}w0 zap`z29p9#6+f^*9f?<$Qj8Yn4+wzoUeaW@8KE1oWxU;pgwY9pjvAVIfxp!~?br8Fj zSN;-RFzk9(`wR6;00w=0chw30+Is81?PrOy@%Q`$~#usRJrJuza>OTIZ zpZnowTd$>g5$@H#XQ&*|Cnt6KDkR{Z-;0|s9{kzo-XEXe{_VqW{^z4V|IcUN{DaST zzX`nkui(tFpXy+z$GfoY#I}>Ip^sXbF(5mGbR1mAw;6YMAbs9zm z-=F}2e1k@40mV9_!f65iN$0R=C#)KmNd|UNi^l}a5fK}!6w{0wz09GPJI-Dc z1z4!G8KgF_yB<9DLdaKXxMl^@Af@TV;|eZL%oyQQk({x97QUN?X(J+P&^@rB_JY2~ z?2f9W=F-@P!pOS(@Y?(kV5&Tz3aZQuD9i9I1E5NORSuNlSDhVLn;Te@^SUO>zb4za zCd;Qb+qbD8s1@p?5zv?)2v*ul!aB<$fV#>fI?KU&SQ{*?u`sj_P%t~Fp&-1sCJEJ^ zHPW6j)S3!}Y)V4Z#~^B>`fI`wb>W!ySRy=;&=ZetdjqfVZ_a<#ocFA?@MR~=zpE^; zw<>g?Hhi!?64?-is0oLc2Z8YRLf?*Jzut^>Ms<7Jzs})w30m(xkn{A-Ics)2dr=ewu*>i}L*TpP zgM+s#%QMq%v(uuq8D&$C2?l zYN-&!SBm%o29-x0qZ810B%CnRgXwESw$=MOoDui8Yx?on-|bwniUaPk`!H)nOKvX+EoMVtS)Y^DjIC9?x`*5t|`IvG>;8+lLq0~-tNJ+ z=E2q`bZ-~%WW(JZn0`1NH_9Yn$S5R!unz>H;GLuq1Yj7SK?e4b&lr;n=q9BIn4wv_ zW!7e$vsvaH)>)f*dC~>BvH97RDfgP^wJZ*^K_=p(hY$ljog;`oWM4NCi=q)RV<-dx z*+oWoGl+cx8cN2-D>x*LfMO6c3?jNtKr@OMMhPGg-5{aqg;ax(ViHkIVyZ<#vq>3{ zWDJ{}VUg3U3c6jzbgGyxHETl4a%=@ZLSuxDbo}7z1Csby4^WLzmCD}@v(kH}|?Fi8D${D6o_(#atdnbabaPHfRj?LeM68eA5Y z(+Yf*%4Lz;^g@kM8sm}2Qs>$`MF9>KZ3I^^9P^zOeysJE_yCMo+^#)!Q)lmWh z!#heNJ4z!u$|74~;ZTIq@Q#wOmcp=_?BL3bz}lS9j`GCz(zvF=h=#nN`hviQg5bKG zfa;7_HR;c5GoAnxVcX)U1L-tGI-xtEuO_6q@I_trldAN`HJMMEa-X*q`gFnkd&&a) zDnt4z!`h4eTk?Gxb6+&(zHBe_>n!x|uL>V-j2ozn8Ei@*Ba8TiI@(alIHHh8tX1=S zr>)G*W#iHQ?EBLVs5dNBjPs7&xO4-9k`L5n=hRcp=xqjrSal6xXN8}0c^Gzjd~&+C zySuuy=yEx=R+H4M6Nx!w5_%9d*n=JEqmglJCXGH$<dXycr59C;K04eIJ|?{BVZE6?w(F2S@nU^*HBy$0(``>J7m)g_&!`5h&BZN)kD z*{O9|DRtQ?fMLLdwUp!mVwI;SmZruP#76?2m8Hg|g}u&+ji}7dsLaVMN{Wkk_Q2=I z8&7ZjDeURbNrA6&--KjF2Ex+f0mE9$^V%x&dh1~=u*}xd97J<9aiC`m**n_Z-cwfv zudf;DYRB~V09*l$k0RM5ELh?Y0AL-!Fkz8!IuXrd0N**rq2l>8qKHXW3)nWbe8Q-i zcUtD0P!GV>8TZc8!qVi#%FN{YBJ_Gpm))cg3x)@J@uR~;?C9vwz))`|roW3k)I&pe z({Wv(=q4N=kkEz{j8Qckqh@2(Oss~5({k_{4pz-ZgFr18r{m)c0=z*;Fo}p3$+%5M zwaaM^1r^Atq)w=46Dq1(O_|hy^>LSqYL|_xIancOLZnt2^r} zI~!};TbnyOy9Y<72Pe=S_O}7-Z==`IwXge&0Q;n_8r?S-{h3Q?6lXhVLwF%+>G-7Ch+m^ zUqAR)pPzpF?E8Os^4#T>zB|(#89PdN(0KJEHJOsYWJDm(P z?D$|06shM-vo7(Ri$CY!&RE!11xdq2so29xE>^+E37Nxu`VgN!EM$yI*myaYEaQ+w zOe~i=!Xyt+iEuKma}3i#K((O<85>QUJ8l`p!xCQcr=t;KvszBk9d+6{wO>8S$^E}qNJCFNv{eL zUKGSXhb6s&CBG_71y1W_dB)4iET5`u|EjFlwb=pnxgKHxIF*LAmxKaJb(crLD*>^h zz3Qn13ytR&GF zmhiGT;dybwGg#t_;`nEUF%L^(AC@OPXfJ#{)}6*3$>)sZQ4tyFj(9|UWM^qmZPv5O z)Q5HHPaCtJx8%KQFY@at4eF~5@2v>$Dhp{Z4s0(9XwCO)&-d-E2uVDbIQ=KIrKsJGGC(|3j|Z=X|W5{I)% zOHR&iD^Ixy>Y#YmismgNz4MHcJ%R2oXqWwW@Apnlw|BPZ=cmo!f`XC?6+eU@?jvJ| zsAD)9nZ%^hcnl7o!;?zHI%u*bjoYAgXyg{P*shg3^a`6!u9fjsGM-v4Qc8qU9!Dn; z8RSx<5=aCbm_f!@@R@*Kz<~)E6h3vFiUV$An1RQzNEj9Yg-3K^;LXUk+RhqSM`dAu zZP`#`Rd+>kdr4kTWih<67|;u#3#g?qt1c(4DkG^fJpq;!U7MQ*uPLp{O3H``C`pMa zOo##utIbOACsAmTSPs5=Ygi#5LBInX=8u8+UY0hDqH0u{!jwQEi z)?u4-ITk!G%$#&s0q;q;Q4W(jIM7|3o8AS!dZ>$n>Ld)*5&CLq!%ZA)n~>No9q*G< z1{C86C3Q$mA5qeW6f}gAF{oha-EvP&_pVxyUaKGrx&BA`2MnvIp2y{)y~jn(b7Q%R@o+_FJ6`&Dy?X6xq`1i50Pte{t8_j z@b!>O-||%-q78v)?(ZotEeKDEdz2b;KRM#ZgpeQJ_RIxe-E0L z$}?QpGdYaMS9t(-$%lFRoqL+Vye(p$>PF}zG*vCwAIglpRo2Hx$8VPwoK~aIsA1|A zG!5`le3F7YrWVrma*j^MkZ`dyBAklvrV`*30-QA3IojVa&{+v@EA6b$Z>i2~C{3y> zNveS*R^-Q&XGc_IhQZQ88%k2?#D1NeVbyX?YL-F8fD#vv%lHH_8!cdtuqg;Cp&L5{ zm{;E4Qry>6IMP)~8g8ZFx+s8%s3uH*^$5HQ(OK5nnBP*FQCA#ao)cM-6IGQPSpguC z7EqMrmlOLeGx||#_`UdmJCQGMhCco#?8&#W{HdsVUQCL$!aAST@ zb*5ig%8RmOh(a(6T0l!t`-(3>iR}s-$71>)60WS;dE(z&|h4hw3)Mfj$!9vg->6os}q2|Pa+Gu3` z8*EzweK3bPoJT?CFfgSGPS2!?y0xG?emnc_WaIt2UFe?o_$^N}7}Q(HL!*nX^N<7c z)-Im6ejP!mJ9qQYv{k1^yKlD+_BKuc7vJx^KiN9im|Jk_Z7PABOJNXEq*25;epJ9= z3OFnQn0JHx#<94r75w1Sio{M7w5o|A$nD0Bo-${ zm!!qlW4ZS5Uq8?UCrc?egNEQ*p8^>|TqeeMr#-^JvXdF7#tj)6EbhynX zm)QWl40_r<<+4ddTppWBz>egkC)L7oF$1l*fyRNRg5KI(bVmuPw}L)g&%?Ef37t|> zw`>fqp!CbB{W1z%LF-l1`!$RK4HKaTVh?IKLt5^LL4XD@GYZirA=)m*0d!64#%FYt zSp#jxK$}$4v^+eUjAT*J3NdTiZQfd&-``%}+uqn-U)fw=+E`zCyZaWJjqqAikk`)t z7bES=xj6DOax$m^D(XR$@?cIn-fz8?3%GxvnI!x+tz9?@d{DWLa)>Sx$6GW;iT8v^XUQ zmKqF84=G6tDT5SRnikevnob^V7qU=N&M2RXV3E3MxDLWl<4|{bPg8DdZF*f?6P;QiQudr2XW zvfcnwxHR!mN!-ts$q#GNAJ=6*ZOnPmlIPb^6xap0RT9!u8VX>BtcmHXh-@hcK-9-W zvsE=G1Iq|63m>YB9q-K=ho|A&V{z><_?~$3KpF=FQ!zRxw3N+7!y&-fv8RiNXW;3{ zyM535V9#*nZ6WjAN9TR(z4h&$qR|<{z^0Rfjg`5@S=;)uYkz0%XlG$>b$VmQwl(Wm zpR;QutTD_WYaFkV@YNE&O2X4fMM@E0BNLf5z;dab0AMC)G7e}OUl~s>U`Y9FsfZ^P z@+5p{!dxYfqY?29O0fw5OeF&MxsXBTGsc9>aXJxA!44CVy|{r+{6ISf-q_y&>#WRa zE6Zvw$!dXRx59FIstWt-O2D$YD5E$bJkkGIYI0JG5~Iq};!9Fu^W!7SG7@X@GjrnJB!vbghXrLthb4viCx`jx#)K8dh9~$x zPYQS$`Ru;W?HfsfugcQm>hjX+^3pp>^8ptD1*>z@8jEw$-7TcSE<8X~7jTF5&}g&9 z%C2fa+2a1@YT_`Qjvt~EhKVD6{q2o?ZOud7o#_5vDi+PaW5-8_IAa(pG)d!tgh@8a zgf_LHcPdTl#lMahf!{^X}3BwEz!dvP_dYh3w)$P@ph^7KUZxy_`8|1Iv}ACsPZoBI4_>Wdo%kxx5HQ-|6sski~XO1iQ%cW}7pHDR9491y@f{_Obp zWN&YOb8jDdWy$XD_Rh}c=H}YU%HsU|)by0iVO3jj68R({_+C=*y;P{0?qycYtAa#eg90HXLn=-REKLb6 zO${kc4Xw$Ifj8t2b(Ifyl)@WwI;+!~N)jq_ql(gkGGbq(L_LWQz5mAlcIb;6K~KL9 zd<0m2E86$R_}BN6Lm#C@JWh*vm>O|EA?R+j-|evHH-n#i`})D3{qOxL@Zr}nK6kPr zpA^KsEJ}CDhL*tV1*5MCF7Zj2ah3PU%Cle*%^;pyzr3Y`EsX=1I-8IB-~t-e3mJbTx(x5Rwb z40dvSe0X$p;GJ>f+<{$g1_R}ygQK05ttHp&gnHU8Uz*S_P8gOZO-Ebv?+@1ICQWX; zcE)M6>*WSHUnk}2Bz&!eXVJ*)dX-J50sxyZXlGrMT&R-?q+FV}m54*ZQ_x5fvS+BHZlI|G*;0;ZD27+(^;G9I6{c5b zCpH&n_EZ;kR~7cv!MZC8%2Q)9LVZ&LUnTfGOAYqTiw-SKj;%;bD2$8BkBQ8T2uTkQ zge6A<4uNGwMnYlYo4mLPz_9APjO@6m#L(B7(P6m>kr{7-69Zp)ch8OrPJI3H_0Qk? z|NMQ_tH))TiN(n=Rawc61zBMCimc?Cy!6)c0z^wS2HryG>lp279_egEG}rXhmUY*b zHNbLOs)|P7Ei~L91&hQYdivX%dz(R#t`phSfkpHK6GlNJC>X>Ts*g0%%OZ_xL=1~c z=+diZ9i~OMV`0+nwi{hGqum0|69)iH4o=?~4u|YQ!fV@Wa!{QWY;4EaKoz_?Wu!fi zhOA_b)Ns)aTudVu*CHggi^*Lwa*v!0S5OC345XTk)N)3&T$Dj8?fNeO0Du5VL_t&l z>=#DQL+J%5r)qr3A>5o+ZA>dSW>xF6nkBc&s^JPbBtDm_RY~WjU7PERJKJkJJL}s! z>ziAvn_FwUhX>w5{q>CUqfGNZ4`5!M5$p6n2VmY^E;4bi1M>!c*@0cGe+WM()|swH z_#D74jr6*R08P^LakP*0*9?WetSiTJA#cc6TmeUKR4e2h0=l!UKC`7dskth?uIx=& zVQ5~ue`@@T^n{mbanCd2o~Oq=P6)r75c*?$zz>lxz7Bi*e}f2~%n22|x3M0sL zq&9;TY~|8M2RbXO3KR1Z1Cv7^NBiB0dIex}JKFE3xS)qAVb3!nUuDPm=EnKu#`_f{ zzAj7*fF%c&qy$!EMO0=-R%V4)WQA5_1sA7+2ro0EpQeOAjQ0N_{MC2CPrmlM_XnT5 z{{Y26I=}wvr~eD&^Yb4A9)26~;)lebd+8C6(jy5iH;>Z8eolJ*eRAOUInfU)GJLDD18VXDYjR&# zW%-t;`;?>r;6Yxh0OHf5?5M}Uy`_ZSPYnDqA>a;By#J4Jes|*hZ^ys>Ar>IY@B8Ri zH=|#E7ybOZsHfjVKl>);+1F7|{}}!3>*#0S#yr1~5q!Ti!M7;pMUvm`*yrCR`rJ(N zxdoI8NcQ@N6#tva{(x}brw8510B$Vgc3QxhVmmWN zT9TXc0t%z==Z4?O3Hu>8{D-`V9}1%Gz+!$XPkd05_Ow3pMPt@WfH7ddI!i(ZJ@W;l zT2tF#!7YV>gEg_EO^IEw;Ld`8-jcw;LCRAV%@BbAM-lF_@A9Hdh_zBVo2 zUALbeEWJP7f-nY6+Ii@0>jLQV%%^;KF{1gr1HGq1$LZ1O(cAY28}AR60ShM#f(gCI zsS_@_jr$w3yBpKTdmzp2cL!@5a}K*!qLZ-oGQLS6G|2d7rN{}rtWrDWnG(Wng5Hv) zm5Rh1rc}U|ia8uQS;VAkgaV^X?CD`IQhO%m&`9|_`WT-+CZvy%Q3I%+=H7<#*0S85 z>U?BV2>@6}Syo#~Rx2#Kqr3q4u*QP)){>ltytJzH`1+jGru>Y`wD{tLsPfdf%JjtY z)P$UiqP((<g;THyA)TNRvR%-CV%4NoH#cFKa+(|#y;dRNFep4aRV(K6s5sO>OIuCe zKog8P+Cd#^7--5uG-VQcN*F`6oY4jjx{*(4myqFdDndy|su@E%&ZvQhF^TY2$(U6_ zwn@og#UdeD71U{~Xlu!`zc#VA;(EL0+FhU6Sg<>ca*=>86tMLg<-&}6Yjedr&)nYr z_R;a)(a8Zcdi9IIxPq38!0R#pqAvD=zU~|k7aiFzXTe`Mo(o+$hRd1wN9|14^7l`~ zzs!Rx0J}KUi(~UlpMNE_E4zGd>(N;a$!BAJ^2A<`cKPG=q`uPXly-+nA?M;oySkbS zx*Bu38ZujJQ|l`dit{2fk^?f51M|{DD~sZ)isSN912f~mBk?#X+GLzW1BZd%p?4|4-o${v{*$Zg*JEaMrKLO?8Fsw^6bMF*@E)Q}EJ$~L{kq7_+nVz6SxiVnHyV>6qNqvd9?qJACKSm-$%T> z6Xp9;Ou)VPpodAJPtqe_0DqPj`>H6>w>Zfcmf~NM?q8DT2TS!WPVp^FdYK#dG%fOe zV#rU?e%}W@2KMV~h)nnX^y=;(eSZFrkSE_J2Hwq!eFjVRDN1~q7xy9~@>x>wgJ_@I zAy2;xe*8_W-w%0l&&o4iS7!%RWd@d|`4uHT%Zq!M8~t;3VD=Mz`~zXL+&O9-huS`cD&E`(JyaCKED|T4*v0tpod=vKl(cG;h!Kq04sk6AA=tL z2`KRXpMvjy9robc$S2>0KmI1{(VxN}{UPk(A0wZ9lO6u3B*7;u^nU!STd^;`i+}N5 z!t)ymFK#5hyqV&EJIVk11V7-_ZUD!Y=zBBC|5ozrTSO_~nHEm=k&@E93{DoUq$@;ddaCMgC9(3|ZoX@}!4#nXg*&gL*5Y zd#hsms^j}>;@gUX2Wnz5t!apw=#Ju`_WakK1^&IIuLmkT7^@8%td1ORN@R?d(?=_4 zBb9tYy^;mDsm2!FirqEW(c6X7!_D`n`)9rFz1=rFwTQ>Z`zI&+@1RYGp1{M?ll{Zp zwWIBYgU!i#m%^pv%^D@MM(K=MK5y5qOk4IhW)5~14z_0BF1uIU<{7KjWl-4E61|kG zmvW2>o>e1u8I+S2-K5p%G-|C{l|doV$b=fXP{5)H85E6(2V_!8ENYoaAvP(cdbyB8 zCDQPtG~5Ud0UzvW=%|J@!Lm9la(k-s+Dp^hOEO!FGrKAayDE!YU^$(YMLjho?WOrG zh1rd{87)QGtwq_DX$b&7u*B%ni%8#R zu+)T_g6!=0sKVrESXx{|aaKoVVP9?8a7*oIdjqniw!gW0u%jN?(+F>_YOgJ6sx0iN zD;2?*vWfZK;nfJFBPWCoV_cu-tHck&W zPWCtErtIJ%iG*B}Ui1HGdkgk9vV7gQKgzlLoO{mRvuCC|=#ZHiWy!LbS+W={CQFtr zGcz+YGqYodxx;a6hoSqfvXe|YPWSZeyH7vwQ%}{ZwN{m?o&T@iRSN*X_5i^4clP(6 zzkLh!4to#YdOmzwt@pd3^WA?Fz`oe5PlsQ{TnGk-i3j==-#-`mJsosa`gCuH)k3u$ zp8Z_&VV=X<4jUN16#x1Lz7GAro8hpA-=001{|l9mKC~}y35R21e(&l>3;w45uMWRC z^>v0HZrr?b?Z&0~xzVP&!onP5agMPhM_-tw$xM+MG;9tlp2dt$R4@v%P36U@nNGb$ z%To%mLT03p9wKFi$eFPPJ)!}m5GbX zBO8lDE3lm8?pR6LL zsDXeng2rI!_y{p2oP!CZBD{z(UZ5&`lm{!`U&Vx(MQFPm=aA!VGNMUDRC7>bQY0H4 z$c*Z3fRq>@Bl*e6ehR9;g63~bh)z@Cima?m4Y9<=$<0thjjm7D;`H5BF(iW!XNBc**Tj$667ohbmL8P7qrI)2xLlCZ|@Grj?~AMP?xzAjd4^ zI258xoe~@>=|q!&3jn5KP{~n|vECk-P(LXZYg6#eVotJBEN7CLD1?GV*NgZv7KM#N zvaoO&4XvtKpl_4C1J!Y3Il@>(r|XQR4e4jm_&nwW0ogal>$K1 za%WO`nyn%uxhf~6EGxM@@8DxqVMd1dHd!G^iFdy^y1?$Z-BD`!Ekr)$;`~yp4&^zF3 zhJC8`%jG`T|8=RaEBQL;|K1P(P5$2=9)4i^r)9sn=xnm3_S1xZXZLiL50>#XX_meqMVV<4r5Xfw04+5;S;I-uavW-=T}iXZh(iZ zLa>M&DkKJrNueTQa01p(MDQ^Q!gBQSg(i$m6vaZE$A+Cmg`PqM0~DP>1)WC)UqA<6 zKm}fi4>*tXKL@sP0T3u5N>twX&@o3LuFs~DE z?~_R1)A9bGl5=qZF8Bx!2{~3mhErpF@nLSnNH)j-jST!5%!3@`Mu~A_BD|P! zo^-ey9qz$GcrxK0Y-sB)r-W9f%ZHnQ{mL9|O&e@V?y0r3m1)Y;1y(UqObARs`2g|J zhg)AE#t-5d!C!>;6XN_d%&0UK(Jn<>gorF1H1S8Vm0p^}D7VqeljxOpMrAUy%E2tR zQA)$a9o zuOC2@DSv=QG#*T62KDlI_x|PUH+#FgPhRXjd9(lc(e176m9e$i-tGDR2Uq6yZ?C_+ zyZPeI#{Rvnhu4xbe&0!#7XwJ-EJkFk8yh=KT2T z%;?4CS)h&i$+g+hxv{>P;hvFhXk6EDSMykJ`)E)5%t+tNX#eD3_egi!M1N<0TSHev zWpjC9MNV3w!(3q3=UX*t8d0)>>rf`-nKh*ja~W_v(9}@LIi|!^mDr&WCd6DPD$GbuPtxb4 zn5zoX8_IG(I-nO|Sa)MZQ$>D#X--F7$yiVG(&WI#0{C}5cX4HIWpR3OZenSEYIT0H zxw<%6E4C?E*(QEXmZrVbHc*!_(ONLuS-Q|yy*$*gI?}W@(z-s{wmH$UHPy2{)4M&_ ze|Z7WYxMf+*p;QxD{GTa9$x+B7pPVFaHZ$Ng?K@nfAxBQVQG4DYG~`?^3Qj!KYMcj zIWS=_o&$g#^mzV9hrOd=^ox#0%%l4AIltDoq5QBp@vEDG+Ux#s%pRR6U{^zQeb(vb}BKG*-l&kycgy|FztJ=joRP*IXxTb5E&l3Z4pTvVJ{ z(NJB})|yvdmRp$9+FIM&*V@+JP*GLtNY-gpWRnJ!YDT4)5l%~NVH&z11)ZIQEzJ}T zv=^<;4sI?_Z>~<xiJ|dqd5Db5}O!uU=e$<^x=v1&0@xr`BeN7bp7`CwrHsdgsQv zCWo5G`sxQd%eq?%I~wv@YBK9eQ!0yWrFq7}40T?LJUdB}WfiBH_$fMWaw5yFVkF6F zMls1CCYoiWBn8nT#R1tAM5`Qcmg0VrS zm_>$8*OPKg)B=Mm@r(pJ38b7 zF3b%Vb^#X-32JzZb|=9+z?K;0L4|oT5MFdR*!r-5V!fHMUW`}|1uddFQ#IV2G1d+o zSULb$Z=JQRTwjqXHHqQ{Sbt8OHxT=?c=6tcB;#Xz`B*<47w%M(m5d0h7?r7|rYiAH zB`QaY&(qH2HU6J1#s=UX3KpWSCN8V58ws-{oXGhUwr)K<%f^24gmJ()&8Rw zPww8iw7fPmxV1QP@AAU^%Zv9f&E3B=`|RfG%b&O2JiPS!;iczy)_>lfyg1ddGTeG+ zeR}WiCBU$qJC`5b+%AV$$-j=$arfR?~05-5~tuF1XDQhgw&9&;%wenbTTKAVffP!I?uhn2`<3P@NE7Kw`p=Ah&FB(#u%k<;-8AuEwf(+XJzNrF0o zEum9nESi)}W#QvU@NlDqo2(LRxHMe?Bh#QNc9@G&%oW*=%AAza^rVU`M_oZ?T|q{9 zrae_JPu56^)03(TGl0ixt1B(Yu-mlqG;<tH zaMS8&>-t#xMQ9F=zU|q;OLN1Q7e}uwL*Fv5uTNcGnYem!erM<2$6w!mfF||+G|SJy zvbw+Qzua3`oSvQ;-nzI10QU6B{b$b}ym8(Hg!^2N(L`?$5U_ zU%fa!*1iM6?`rR8sjjU|&oG+| zw3H-lc|N5qA6Hq3YOf`A*E8Fz1wBpX>A}j)g`rEUGuvy>q)L#xTAkinnYy?I{8)U{4ysOq?D)eOH&gH?d&`& zGuuQ@)6lFEoR*7_(*U9(1o%)sK2%5u<>P{Q@xCCMbCJGmgdY>(PlfqVVBme(lLGSu zTfm8f&s1k{p(jK*&tfaFvq;#U&(AjzWE6h}(t{G~jtxDF@IM{v3ozyq=MQ*x78!6Z z&i`y&z&T{l`S{QasBplr3y9!zG5#)*zAh0yry_s}^8zO9RFv20C}?{s!sB?D`!RH= z8y6c%Lj)lF58D{e0%I2+bPfm|;))KvfDL!WMY!RiFKQlyC=XJUCjs(jo|I@WYBbb( zNs4k;GGl6UHA77qqiuj;X``*FL(Pt^>ZGbng;9v)paAN;n6XfUBm?Hb02G9KFcBWC zI8S!GHxKOx9+ZKPlv2ZU3@o!CUPX_v2;epm+#!Z%Dv<>`Y>5Rhj1K5kpPtZ`CvC_` zs7PiOCDCg$rJZG#+AKAAG4Cj|wUwA#i*(KT%JyPaXIWxzjd8LwV|l1#Yo`9zX8-eh zEARGie|Yog!`r6-WAESXy?OQQ?f&B@cc88r%aa4ZPVGOu`Q*;!dzY85tc-2X4}n+s zXSY`0J-+(O&dm=`FF)9txH{i|d1mPH{K&PHsq1UAcQ3EryRy2qI6gDfIRQLYSL;A~ z(_nk^a7WuvdrNmiU1x1|LrGD6Q9*NgQA=e}b45{WRY_Y_adSm}LuqbZQC4+cdP`YB zLw;tSRp(HMbX=NM%+1s(GBmOjxhP#N%QtJX^~!X$6r@12Zx|E7*-V`>$E3DN6B3yu zEtjIlpr%5Pl=C8AjF%+T&py}BxD=- z%yg}y*kQ`G>580|iY$AMRaXSeVs2VhPD)97Qo2zIFqW011)i+2Jio1`tRySNrj=UN zqU>aSd2T8&Uci2}*Oman0iJZyVD&BgJ{oAcZ2bL-19%X3o;)8j)uZGeK+ z`KfgU$z8R%gDpi<-Ia6w_45M_i$hJzBW)WKJ)6`08`A?9=Z3ZxMlLN*TwR{J0nOe! zdu3zh#^vSx{YTI&bq7#w%qNCCIn;89%l+TG@!{F`bzVe^u-S~WC->v-U)xXwv;qmFzcWe0d zpMR_SqEECtTFIZ&pZpbv8Gg8N>+0s#>d;76ZEapng|oKOSyz=+U71(gR6jfquyb={ zX{%>sd|+sxf1tOstE0Z5A}80BY^N6GlDb=Xoz1k4M%-98Wului)~TH6bu3I&UtAvA zSR7xSA6cCnTAuBjpX{C)>zo*D8S1L;X)SGUC~T zS%pbfVpBDQG(Fj=qd2wXR5c-4g*S=gHN02_D?&u_6B2!Oyyy%)so-GckwQBw-%8Il zQ9(+I8kd|1&65lAYZKrqc9fhEA*DtL2%%iGKMU#4fcw*d5PsBHA9A!OInsj==1L5^ zKniywM7ZN4JbLlP$J#QQJ&-|U@<*N(Ey(T76~5MQ?m%0mPkoJ`$|b6YI>A{9HM7IeeTng z@H`!^z(A}?;kM_?0KIBbS=A}5@)UMMzN)7>r79z_&?ace(=`_w+DZ(~1*-NkO=&-NeReSGWkwe^Ktmsa2G zK6?N1*~@1Sb|2n&cysg0>g4ix*V;ta{Y&%jA76X*^X9|tsT)hfm!^ldW`@?M1}@Hz zZ7z&0j`z-vbd7d5546_zHP?4H*7UX1^)=UY)|CTLbykbD?n3t*) z0sEC>R0CU98Et71_A-o#X*j83!49$%CC|kAMR^ zSeF-~-N%DjpO1#Z@9MA->>C};_UH8tV16$~e@grYV4qKYJ=p90!T8zVtN)v7ztQg* zAn4SW8jgI&I-2fG;d?6=gRit>f-odZ+&M=QFC2RbxCSTt~oDVlV+1z^<2G@t`ZX^T(p>j zmT_?!5lJJX853Dn4a=rs*)?qY0kIrvb}A&MBa!Avq^2a&!A`PEFv!GU!Y`l! zmcq_s!C{yyI^;Y$_#8I)0ye}Ikbn^B38(;s4R^sUqJ( zQL@4Wga8)O3mbX{;d>&^`&6vgNtn+`V8vp6Tn?!N?f9KR_@9IOpF;qWg+P5E5g`|1 zgU-SH&qn*7i}XDk>gf{daT*sL#Ky(KgWZGNjt9G*JXDD5sStO7H^4Ihywfn>Gq8ZO z@PGr82S^4ySuh}(D=rM`$_Q8shzkIwO+Ypmn}(ay$J){-+S4XF(neZSdTOoZ=^{Bj zh#Yl-6nTme?m~z-M~Jw94|5}gdq4spk8&3h!|gJXmW{B8aVjQ^8Rsb{hlz2)LUe$X z5Tv988#oa*VN9NmP@gUUKCIA4Dz#85lUeoI(vEUdXSuB+O;MgMZ!5L5mKdwjMHSA3 zhCE4qj-a(r(orO90Xqfqo=Ve1Tl&&q*~UcE&9&jDx7T;?U){ZbWB2~82iLdmT-kW{ z>e+{PFQLVEUhlqp^ZfOTN4t-1-n+bVbp?8nzqhq;du`&{!oc;#;h#6=Z?4U5&W@~3 z4NUa64t3U#bT$mM)^yibb<|XK)RuSGm3P*bbkr1gR28>Z6gHRS)a7SZW~G*AI4ZN9 zRk`VPg_(`T*^NcnRaq&;DVA)Ly25D#rpm}?s##@5u>z|uT_Z~pa{-oY5{`~Xkx@}f1|C2Kcq0XasAN)<47!X;VZdSW0bZ1-U=@Ry zBug-eIZ_6JhK%Ci;#3?OpNL5y<4{pyu+RW>Oc(=$5K-_FDnXaPlF-PE_&5cdVO7b^ zN|{wD)r)vy8jepyg8B`77QmQA!UK0+;!Mi2XzeO-wndw1R6F$wK&+xPOLVM`S2C7d*KVqtnxby0e2ZBcJiWmkP^V|i{xUTSq=dTUkT zSaG z=G^Gk;@I}`*oid`(U!VPhBBC{POFs_wWBaJ3YEM zJ9c?<`QhE0&mn+4eD!({%5ng(!=>g9NB$lSj^6~zcmF>=RgBSnt>_U}IAZ*K25MSlJD)!R3_FZXs| z?ftN^xi$|hQg=gHS!Qvuv%JJvQI=L(o(%vtzO*s3b$x7kYj|d1Y;tmVc(A9ptF5)X zuEtVV&FN_)Opg#3#>ulIg!vKj+z4m9Pc+t(I60WQwbrxy?C$>FK4jJj$W>%J&m2F{Vnix(aEm==ZF*4Fjj7&2v%S-{>%CRzYEX-^ZHPb+GYDj5X za+;RvP!R#WtWvCz54THVQxz~XKU9EqBSxJj!Om0R9>4}Mkls|d7a__O8wntBJ~rS? zjNj=fpA+F;$HF~-it;)h?R`Aj>v)XMiCDi=hyWLOfD72h2b~2%2VH==U<95&{DOrE zg#?Tav=ior4grT3aN%byB1Ch(q`z9-UM$NsQxe$`VsZ!40s3aK5LJ2knY+LFeIt7vMo|QGO7^sK_urDIO8z8F=A%kn2fE z=TATa40Asf;&v>;<3yzAi72lVF}|l`pmlRj!2?bs0#74@Tu{L-@jyYR&>^Q#At&R5 zj~j*Y9p#qMmb8hswCVQr$u{R$Yf4|Wxh$2dWd)0{7eqLB0oI!dcg2RD#)O{7hF<_g z2V5scxvQ9Pn~W@>L}@s2Lf~2B{iK9YJ~}{v2@+tS0n`#gfSwbXtEZ$V;;cfLLxLzY zQW`QP_1W^q0!@3FwW-M1Sg3C+HP`1V^DWfEBx-FIzdD^;lfkRc7Byr`TMIN@Wu~Dz z`%qmRN|Lwb%@87?C`{wzJ=RfZ~ zy7lzV<$G7wZf-7YFN|!@4cuCrzPq`2bzx#IOTFUd=DhdFlYV*?T3NmVP(?F~*aazkWlgrZWrKwOyzI>}b*Q6=30q!OPU!|lI z0Kl|t3NT(!PYn?#MIqEO$%$0F6)=NFFmPyjCUu5ZZkO=^z(7tdn`9O-^$AoJgP`Zp z%|f=CMM-2)RZO~^MiJmKgz!Lgpce}fspHX;6au}7BcNayXt}Dv z-I!_7WSZ0twbZ5%Wmz=2c70)rsU*z`lx|W1VkN0Wz>t+?*lY7Mzykrav#G>MiK4tz zQ)NL~TTM|D<4WqxW^L0V@+*;rrm+<4c@+~CH_#L~jp%EILA_~1}?Q%`GU zS5x_5Tg^mQ<6M91(n#m(MBn<<;KkVy;EA`FCN8f{Us<2Iwl;HfeeUMQ{Ee-J2X`*L ze+$hXelWSur>RyyK#}v;J9lo)OpPqfja|98^7!Z52LOBU=Ix8axmrFgZ}BGp_9eLg zUjo=SiAPTzo;_O6*9DKJ|F`Jtn%^D)*za8zO8?5F{?q(Wp5K-|n&BHkHUE0R@An^G zynOle>Ep*w9`5Ws{b6-&X%75xb=4P_q?eSY*Hq`#)Z`TwXH_>gjV`RuT)Z*9vNb-x zJT*HrJTlnV*WJ=oQeR_euBUgmz^8{1OOu#|aqQwaWqy=9(a#_07EKIV?%$dC_+bzH zk?rn1SzVvMdGqp(>z5`b2CJ)!ZB~s$Kx0r4#P|>l%r8FDD?Z#KKI}qV*m){CC{cj5 zYp6~GGsDQtGBNY4oI*PV7=f?Wdp%KL)%06y|r~_Wsx4FPsWTm^ zGwmspEw+i~q~SVaTY;$DMl7{pDr|&uJ3T!S%ZzhH2A)8No=1fOrk>?uLoH&mmWyGd zf|N`+4-+Ur2k=n=oH$=zd;kmK%Y=Iia6w68j4=TrAqFXFp{WX7aS|t6M=i8+YO>_5 zrN-J^b#0ESt<=zzuPRGnSEjM+vjrt~MtLf?HcQ%4sB6yGb(EMoiuKL;iLC|N-m2t@ z_M*kX`jyf4jp^Q%iLNKNwm-ah`u_Fv5AR=o`0)De+ZS(MKYz9Vc<0gWhj*?#xOwTx zovV*-ZQr@Hd}VoRb#icSpkuV7VX(EPzqz`*9$GZHqo$;#ytt{fsHr3mxT~hpoaWM; zrjqQ2qRg5+XKA{vIL%s=YR@opBS50OLNJ(Up6g0e!OG}dQ z910P#0T)1Y-|<>mtqCn94w=vd&x64?}^kZqL<0GTo@YQV4@n?6Y? zvdH+^R&BA<0vHBxY*gKfXI$^5Vc#fgKRve!X%1 z>eR&0^8EPqOY2V`+}(L{|Jl-yBn#o6W6nYrna(fOI^X<`Jp&nw;Y=`?V!MpIgH*eqV?Z4RF+rM@D=JeEPO;u5nNh#uz=ma<+ zE-*gqJTBTzK#fXLQL-$oJR7$lnG0}LoWd(kNhq>&3vGa5EWooYD<{Lsb{ZH~4Mnda znG&fs;MkPZBniPP#+!wBgAk_^U^NM-L=G~M6R%*!N$D^NEm}m5F!B-kW@5FImTSaG zsR4wj^XO35_z+hV*alyS3p^Jacs3@$CEEXVG_YU3ry{&hh60j#o&Y3+J6LKNe@ASc7FHjKdxBv$h>*WITgwPf3bt2jmqE|GKw+r;? z%llNcuS=BQnMmI=x@%v*4g%? znKtW0lWDYGKU%NtD;IYbu)6bE!)20*8r?vpE>lb3qP&3v10W~G_$0Ei3OZ6wgL5$< zQfdrfGw@zam0RSblZ5erTU@Ie+aCPhgy@DL6uuEfSKuyE>fRCT$D z^?90>Vnbby96+lfPu!3rsY*+z%a%0cD(iDKjrqE&3|UDszdk3iF;CZ7o;=i;Kh|0@ z)m1gqTeCRScw=?q@y+$Uhu7ZjJ$U=#$@|wkZ(lxn@%;Yo)4Q*pKYa1z=e>uwpWeN8 zeRFAcsC&4*p{Jp;sl1@Eyr8+V5JcXFvfP8i!n%_D+M?`+;_RB-)T*46imc@FOnYUv zqd3iyYtt2{n)9HE{D zqJ~9QGe{g{v_-_RNfQ)Q97xfz$$B=)z$IA)3^juQ5`lFC22RVNDrf{9z!;Y*BBF$Z zcrF@FgNIY2L)ge@4TqAVk(lK?Ig7+Z!xM-&F^w!`QbbfDJ|>EaLXzTQX(*VSNi+&s zdLdiRq*9Ow0gdX=s+Z*q1{lGncQ5M1sGPEZa2vi)FM`zMO~ViTvwFUT2%<% z1{%wAs|wOz@q7Zynfoo7(=2zEJRMfE_h|U!wmvJ6|K= zU)9&Qee`Gkp1!&6(fc`iOJABazd3s}<=cXvAM)>IgT7XNcn|dI?c3*jdrzM~1$w@> zv-e_W|0U3~=ev(~b|3vPw=gq4G1$^lQCN^%RA{TJ%mzNpnPw`hC>@=eo!{7ASl?b+ zTb-Sq7#Zp5?{8~rF08JUw>1$*`{Jht5fcN^6GO1^VdTgFcA%R)+{;;fCjB~_=OX(Uvgh@umaw0yiK0iVdlDcM*# z8?EGGWUP1zBTh&~aEZ}8Vzhu9WtC#eQ`wC<{5&&BN)M*Rc~RkhB$yvD){hYFgNgJ6 zLWa4)gU`VO&jK$6Fc#^1D$@64xHn+fu`t);AqURJ_w3(&&;96g?nl4#Klxuc76{Va zP67iI>3t^J_iUv1*>EqH2=6nIK4&Am&qn#4i}X2-jdag6vb(CRgAKN}5>2{+rr{&y z%s2)z5E*n1YT5GxWzR%IU#d<4#draojDbda{S@Q*6U_5il>3hWMG@`*NPmg+{A+~Q z--t0DQc5&2;sQG8I4S%TJ?0z>?#_yJPl)$QKzj?YK4QGD80RCw`{>!>R(^D{C?;J3 z&ru@t)o7qHBdOX-tx2NR+Zl~^Mw5e4mrScqV|JFvhw2TJ&F1M=%T%*@tR4W&G*YMO zDHSxP5gSvm{dwG(I>T(UZMeqLkflggkR_C8Ef2%ThtuJ{5^{6`E=))a=b-!;a1T<% z8B(+>HO89~<13}YWmFJ?{XkBOm}uam`S_qjMpT-Tl%XO!B-k>$u(?2Aot;>htEtIW zRHX~ridBud^6CsxOJQPjfx0c+yVRlt+T4io}S+=t*E44B^wZLIYH!3qss%)z!6NGh(8d@&Wrq55( z9T+cFx>{S4$ z0Q*J(D_JGd3YbDNmIRARAYcv_YsS;iaill|k3^uw!47AlvPgIqxj;xIk&y^7on%!C zZHZ!|n5zJ!WKk7#f|QD*AtIyk%0QwMEnw4xx6q)1+X+~ACGYPQ2v#PkGy120{w zFs->PbEv&~Zlr5@W^i$KaD8QBbA4`Vw0E?pb)vU*daz@Dv_Zvb|f^7Y}L(I;pf)N`2fj}U*Po=@rDWcwZx{-i#Y z`cnM!N2zZzeD@OHP5Gwm*M~e{l`Wu^CvTJK39j(OiAKAp}%J0wtD5iV;&|Qq-in9C2$Q_*X}lv%;BZKN>QSf(RfZ0*G*b zJj|B__XjY;MSJ4_%)&g7p{}t(=b{2!B7ILsLFUFK!t->f+bREZ$9&Hp^FI5Nhs%$i zEPrm1l1)e_<;(9v7^;D>vOPKqaFpo3go)FK%JWmi}ybElif!dVerqq@q zol{HKCSatDIB<1X(3uFI6VX1WVtmfPpb^T)BRzkLa{p_T>wiSM{RhnRFLB;~jr0Cb zwEtfTAwN>XPtc>!u;W~nv{0v#XibRIGb3yXF-|c&TY)N2(I>w(!n~@ zM3WiNYqALd%sN_^)K{S{He*z{Q;PT>(-Q)k9Q5I`#PNE|WK+sWowL2fTA7)s;UWe2 zPyr#7jrQYU0+>i&sFz3h1ww=?KHQCm4HJ`L6qqj`AEiscNhmSQI6nnF#>_|S*|8dW zl#Usbp{6zF8A?;+H8~pK!z$7wRT<(+C%-OR+EZz2DOA^GDauks6&cdnTupOvQfFmW zM@3$5eepm;X?JyAZ*6X0&B36eo(iCi@wR&#lXte}?q6No`T6SpXGNy7#A(kB~F`Dr?e}DPOU6eBS{9q zT`kVkE3ypAOr4^@q6Km)#mO>$l7wp(G6BGV%~I0vMm`hxGZ7&k72qXC!9{T}MpUST zh%yQo*+xZmTE1pp=`V}P*(^;;!eyGo=ikj!_0j|YD@d!&PiZR29&D?b8}D9O7+zT(y|_7ZX=83_a(HT>duF(2 zX1IG{tao*KXmft-66C;UuW!!XxHy0F;=-Nnr3cs7cOTu@eRAhHuvYJ0A1plx4P*vk z<=xwN8*3}GlcVd)bGHD%9^QTa?BU+-(~r;!%!e~QAFP9MK(oWoV28o<_b~Ql|GxmR zZxatd@@?YxSn>%2zy4!3rhlqG-@E-&{=>}Qkm-AgzsYcfZNH~)E#~Ln{tf58(LZ2+ zP{XS?Z+D(Q-+8{Xv-@lhxUPM$eY(5*cyIUN?(=)Q&+hI#zWMaQ<;Oo?{9$%>dStk_ zp|&X7ska+NPMb12O`DabDJxFyYOfd{ZC{ulTwfkvnID~*=p5{?Y;R9(YZ3Q!5GIB( z^CQT)p_u6**z7oFc7i-TN}HSDZ7$nyUGLxByZ`>ouqanCt7yy`*5u0gZG#4t{ zOO^Q$z#>@KAUZmP79T{33nU=|2(f;o1DKIw{m8(UMf(7Z#fQ1Zg`7tOL!I~F2QxWE zdY=mQI2q`6%-{964}>wmu)hI$oc^2F>A!oQ{=47Vp8_r%54dn5$n9i^hfBEEnGlZ? zM7UqELoryNG2E2VSg3cXXlf2dLW=_zg?t_`SYA*k3z*Nb_`qZMu#>c?(+P-k5{z3S z+1J7fP8CFCN@H>r@KPkB zU5b*J12vY3#-y1h%XD+nSfhQo-riB7%2dO|s2^3>6UiJeE6dlG5L{*>_m=BMYV0HR zsROmCtwrXtR9Q+QMZtd+M@V%d+1UA>h#z|Z%2PiU1wcsXLWIBWl0+duf=&axfx~Yj;d^Dc}7aU-JFx8 z&#~&VE!s4V!X_6uRN`c%$e|K}cx{s=I21zQvGUEDTw|h%&oU-3Z4$15N7Zww3JNBX zh6gq)*QB-x*|eBY87`h48ie+8XCWfYB1XE3?~t)mObFr%1bkZ|N2s)R-1Qwd@Q*{l@W)e?h%B_yIG6s(p@ zGYdI@VSIcXaEB=xnNtrnB-*r6uuaxTwS1YLywuttEMFWd)7U;IQoK0%uKuvpPSmsUmNryJ2Z& zU~Oq^b!qJC#f58|D{He;^J9Z^BfX2`0APa~b0e3QC$Fr|0DxWJTDZBjczb*4{`J+p zCpVryxwXB$eC683*RNmvdawc-~QT4~Ip*Vc2(&cA)Q|>kGFI2Dkm0OG5>ZB>sW^Or75r z`9ty zz4`3XwWklSKKc35qq`fA@2&kXH8n9f&{JDgl$&nKaHxv143!18ni6|QL-ufQ#q?;? z;&jK_eDC@~-_mr;#Bfn>kEy?#Khj5>86(b(5@v_d(<8{K(fH{x?DQC6W}Lb(t-gC} z=uN9&9za3_ z0uzRh_9aC7;-h?V;U3s<4|JF-)K4J<8bK8oa0Ui=19kK`0k9O}dE$U%$9&HH1Yzvd zUp-I$=K;+A3bw$d`2mu-o$z=4iHr;?b*cyIG6xzm8VbO_d}<;aEvCfcqr9R4x4e!a ze2=4oPVzC{Ms|op7@nhut+wFmZG^^TVylDLmP&4S(%Mq#9cj$23{FQnyTMMWu@LGk zgh~UZLXR#^gy+ekGK68N3BgvDzlq_er}=Bi{wln;2aMi*)}(Y-r?*#ScGVRPw$=_dRSz^) z^w$*+)))3vX4mIi+saZ#TZ<>VDo5MOCwuBwCVRJ+CO2lsm&b=EdpmoZYP#zxnoDzA zigQ|lGssRWNVeoy4PaZ8l4Ms&QZ>>vgCgCa%!MY8S2;DZRJ9Z^%qHiXM9|cQ7SX}C zF0C9e4D9GQ09|AQk7D4EG&G!!LDVzJnQBRnPOfKDhJk$Redmd3GsRPQ&WBbSVWZr(glYG;E5JPS6}Q zFbeTVKraS7f{BRKa%cdqiCh{NA5TR_N*F{g9>c}sh%qs244O~G(gEjaIJ;7yV3YV{ zl#)jm(Fjro$)*%rWqd793086^qHIytaY7PZ)>lO^k9CK*r7 zr>l5Wn@X5vN-VJH^OCe_24$*Ne$e4wlb2$G280!6lxI4!OzI3kHIt?^)!I~)-BO+p zF{~`RrX;hr7*b1h!AMWz{CLmG?9k@YvrvX30Q;nm zpSksU3d%!({ejhc^Zwo5i|2d$J1_R1?e78){qz3L-Q8!mo;|w!?BUkqpEn-gU4L?K z^V$9Fr}wv>J-GM-uwT7Bt@SmMF@j?tz^m}3Lvkv_~wKXQBsH8F%5AHvN|s~P8;JKI(_t-G6_^1FZEQExNAfuyb zm>3oT!6qTuWF&`zKsZjUiu?F&~8y)IfaVPSetaq!2bPSWJr(lH=&{VX(lnQNE|*LS5Oo2;g+#{w~y5UoIw;iwTiX zU}_Gw%&E^$Qlx4)CK1joKxOObxdukMnpl{`ZYk8X6&oA#wGDZO$_#yDiL8E9-42>~Aic8E9FZ>|31Zn;+?(>}~^ZYVDQ9P`g%H zUROl{09a|7EzfQ&PE9IwSWFVWLCA3eTqTMDy0T4)&^|O{p-9hTi3n&uHZGA#23#@l z7%6gLjzN{7k=dm@1CMMI(Sd(b664gQcoiODW|MN%l5~|&MZ?MPC@qy}<*<2^Jj)9A)sW)!baWy$&cq=|aWF{?9d#g6Xs&N>VMH#lrJZE_}aIr?fur#eaU8g8^m>UYS z8cTC)inD;~O0sK^KO*31lbEKdz?FHc|HT)eWmxUn!XKi0cE2^`qiA;4~Y z2H3qTt51L4e);^)i`{$ApZ~nK_vFs)Ybz^rH*Q|p+k1BR&h^QO;fb;S)x`+_u*dgr zKY9G~ix&{U{?UH;RTJVD9gU&?1^phPzB&6%p#KfMAb=fMkgoxDSoRMq|2?SuH|pz~ zJbL!`^rczy>*7ZN_RT{8_6z*qeE$6L^PT&;5*U;5j*W6HETbWm1ncd!4(BD}$)KxatSFt$NxH{LeI@h+o z(6zqUwYE?igk%?^A&O-{h{~v8{;T_kJu6xh_ zao2ax8Mt#YGbb~dNybi`#LTi~fyLZ{W@ct)Rx@{tTg)v+i!9la%pl1qGqb;H$@Vyw zlga(Qd)Io_Ufs2;YVU5@_3NkhTW=MeJTViG{s8~V-oCkW|L%q5m6?+#>YAclF{>Dn zQ-;lhWAX|M4jnzbC27w$NA_$27(2Xo^O1eurzUL6Ox&HFv^OvHz&gngkPK95HUhw` zIE_On^cfgU0a>k!Z;@i?=nQ<>Q7j|{o`0k)??_2@VqBRCrCA9jSqF-TcL#P5-t3yFVZP{?Ca&{)t$8xZ28#RqDGcH8nP-U4pcWP<9c*z%JBK zb99t!6FuL`D)jOS-Rvwq;jkKYKn~x_FWyPc-hxQ|rs%+zS->{!{u3r=J2dtCq%D5{ zI`ZA`4sHI=gm3>dVe|hywE2G?{`Nl(13R_lcSnExW5)I`GIxAYbl|JfLti74zaeCP z$0*#+hwhQ04{C^s2J#UTHQ7o%>R@E}c=>*QZa|O|6lRB{`PG`Tb`Q0`QaBJ13`Y1p z5m9$o(i@RLQ~suBWrg&Z zwg9)MO4eJg>LFDr}8GW7KbmMZNva;aJqyT48St+q)aW$9h_(`q~EC>SFbg#-Ov_Yj5;98a?GL zen*YN=unBADu6?E$fB`Gc~+Ugsgziye5;Ia60_s`l8`YRM2QHG65%mi6kLQy$|-mm z1*Zf@B3jELtwqD5#Dr2M1tvljaEdcHMd`faOhFm=vmz}#q5usMz>Aa^1i!Qpn|dhc z$4yE9_;dQEzZUKImXLOkSC++t=g9D-YBCCynvlKYJ4Et9Zdrkvj4^X*Dl%4rMQ{raVCRscv(IDjjo(YL4CLkOE8Q<(;)YmRDfYt zDG&HCAdgm7=`aF@Re8(7vD#N&>9vGih8mBhvo0{uRy#S^u{bkwVSe(=+_AZ1gOh_j zv!nfsQ^TichR@HAU09gBd}{j2+2ePvoqqD*>eEN7PaofV{p#8CXAc+VkB^KDOiqjp z_IC~Rwh#2S&P)zmzOZ=n+S2NcD~})Fd-eLotN2s0^@#iU{Le1{Y~$?TuhcKM`qcn? zH{8b`uDtgy8+-diKUe4DYkmB={;j3ksE=>*(^Ega%Eq>zoO-@t3h_hF>*Y_MKY#e( z-mTl$S8ravbN$?{D~q?ToVi&b(^+=)j8xMZ83E|t< z%S)$Xt^Sam$0FwOnV5^`PQ8u)1_93h^|RM+?ml^T^Zui=mzNsb8U#`%orR!~iwMYc zSnj_3@RfD0Q*&veF!+P(#(UU8T(2zc9&-CEKT1AJ^CX& z?MH0(R$~4(a=|uI!471`_XUUI`QvjBeVd)IIs4$IoP(Ql4s4D;>`C}859rXAg2Ued z;1wr-4@ugB$k;*5+efPXD6TRC0O5{N!TY%WEyKueCnCP=9IE*;mCYmlR<0za!*-&xY+X^Rg;Us741Z z7LttA83tTi-LsGNK2#_RXZUPOK0n6N1mbQmfoFXAZSco)i-hN!(K{g`GA|ll| zR1ve7iO3|EB&k_&6{F0hqWH8lAeWM&=OWEQe3Q@67xl&Jyxle4?rLXG)YBDp_0)U& z8iTQ@zcu1)jyPlW;oVwKV!|4Q@-l%UomAyVYWwLg>}WBNmNa z5hoU4m`TJnOZYkg6Id}|Ym^M4l!O7$Qqst321Uc5!~sUaD#$nm8LMYgbu5Yy3ssQe zazcp$Uo3{@@*o+^{8WBvmW~3qatSITREaB-B8o|w$z}U@747;l>&LItH-DMA`OD%x z-_i1pQ1X&FCD{f#UPHzZvQlz(ZGk5r6roDB6r7Gq)X|7)3Z4Zm;=qfgBs2$ELMX@t z4vb!shssI;04qy9h|5b86HsyrPE14*i?VrGn23l$=A;37q4Uxh2nY`Yr4;7^yeVl! z5+s*YoF^sY0O#a1JR4QU!6F$b7#ELWV^MSzj8Im{#=@0MGGG-4QzoHdRUEQO8ebfy z=FwCvvWSS{V8bM@TZy&LDBKD_qi(XHptAHRL`^2PJVE6W#8pITU)pB*0P>gj6i?`@u)8N7U9 z0RZggjg?37M2PXUzaJb}{OQH}6u|Go5q~<)es&c8x$=g`jV~2W52V>Bl4q|5;y`zoR z-b`w1!nQTS+8ZHVtgCnr6 zyyl>y+QHZHU@UYd9iB-lPD2+YBJ&TS3l3ol4xw}R6ABJ8VTnRqs*I4TAf{+3=@wRo zTbSz>_C>O&N181l`Rm-X=%y_edd4PG(nF(o?DEtyJ~b=zFV;u^Mw*g~6esAal1CBz_^l>~%=8 zI)lWsL;8ECBTp~ZJvv|e@M7b=i>(Wzo_Y^e!%CNs4$CQtN^)|IiO}iewtLt;L1AxL z+7*=7SouCRRnLaf%981@qriJ{F6ol7Y=TM1!WOIY^tF z8qjk*N}5%G)3T5%1~Oz2bwq5vb-o^;S`PqNN0q&^*40tv=&EwJRk}K={e!J_V?E7d zz0E@%^}P+1%^_#K$J`RI0X4WxwGLg_qzs$XL8B^c(YsV)qkv`>GBjMOl1-7)h%7V= zUzkfREtHY5@dyqE2@L%T3PFO$Nbx8&l^`Qxfb>kNRmf7)uv!{QMJUseU>ZUhzbuPc zbd(Rx){DCsgFB11Z%*Izm;CMjL>}2iNKaq`(^i%% z!$Fk{m7*eH4Ggjb0|zhU1hfDRl@O5<0+Nj=ra*Fl6(d7(0mG1)$%ynM zJ{l^gU=(x=4_C@Ym8e-HI;@ZmFQP#USjc!RKE14v0=PxN^6>}^pc)*apcC|bIv)$C zK}(3Gg*12>pN!{`@eB+CU6jKop;TNd2V2I$L4{Sr?8N;W7oR@7 z`sC5gXU`vk@#N`~2TPYPTv@(+{mRnZ%=lnm>&RgHiP@p$%fNwMzHxo|!NXf?!2$2` z*2e)BkA8mrE)a1on1181`&oUKVe5q-^vjjk;}m{Eg7>)%|23w^Gwc6k+U=80e$wIk zjNIBTdH=$|FSgov^%tLWPCxfd>)jPU())hmJB4nn|5v2WdG+SPF|uOiPPejSCzB$yiy?{ zz85neo6ALK2yvN8T9JxYD8*-LX?Y$=Sy&CP(!i^=uo?}lP7jaj;dNSQSYBdbWUGl8 zYEq7lo~LKz>FIe!dY*}qZvRI_ZW}cRwt)^y3amn1WeeB}h^uis~!fj?wL6ei; z9*{LQ%zD6eg9hUG_(f&UJdzZHh=!@K)Uu5n0B74tY@VVQ(3RFi( z8mrdy))~61^<9*-~i@T1Dk@oZm>X$&oxv0=4uMy6;_VJT>BK4|0P#q)k|8;$+AP`*q~RkPOmnCB=NAzOb?{q5>=rYAFUz znShW|G8w7N$R|eK zI>4=ghG18vyQjv}Q|oH4vb9v$I-}l>YEOH$cc`syw7Y4fyJ@hazPmo$;I~HI#wMS& z)n}`Bn4%72bvXc-%B`08baIn0Lezv(8yjzDMITfMh;G|@v zl8#l*b=1j|>5{q+T8A&W;iBZ4+;^N>O90Ci;XX4OY5`lq3 z65+)ZM6rrZ2Hy-9Q^v(Yd3c1DPcK&rwE{+bbu$?wXOe&+GXUGF656z4z$&AZE1+Wd zRE(5OQVSX78ZiKy%c!twL}sM`0L&ocdd#Y@%K!jY;WYyXR^c*+tQxOLS?RTOHdT!G zwap#tKYMcW@|n2{r;f)r6rCJhn2g6FUS6EKa(3?S^|O!fFFksA?b-7OV2pkI_`&7N z7q4DjzPfU0e)ibxXZ0nV|>xr+RkCG*COxVrLq6z zd>p@CpT*T@6(6{NN|6s3_^86CgZoc)X((4Q6GP_Pp|U1 z`&_^1#xEpZzMhBoqgJ0Z|JRGx@h40#UOc&f@5Y0BS0CQL^5E{3dv~tfy}k5c_0pOy z+`E4I&b5W#5TRd_OaH-wf2(303E42MRnl0Kd7#@rIUGH?2C&7Mu7&CL`I)-uG3P)p zr>m{3yS1dh6BBDAw>Hvhs!0_QLUkoE6oiL@MJ>&^kzwP~^5~Q2H*eiJfA0LmtvgHL zU)z(X4<0?f_vFdL7cXACkE35(e)#(7=~I(!jUKy|&tRdICiU>l%-Ji~F5X;SynJc= z#B^tWM}1>uU7auD)7E&T%^_KDgJCM>p6&Bbb^AtI9DPx9TUcFT<(Ne%At8rUco3ep zwKVBlMB4X=j4jZko6s3QkPCO~nR&HVTDwo!(~V z@k?EzvWNy%rAJk2U?D|GKnC$iirm6HJ15J+%ra0j0krh=0xdmPOUu?!GL(d)GE9;H zk-&!TBNuJM=WW4eZN_G9!esmtnfec8^55V||BcD`hn1V#;^xL8vQ{sr(unn|-~kof zD=jW(WoU?pwS*&9YPye?>*MF!Ss8jlqKTN~VrGVf`4y6a3TZ*Ds?I9bn4e-hrcY^|EHWCf5_VQhr)edmLB>??w&s- ze)s?E|K@+}|L1?~{pPoOzW&dBn|_<{?H>+r`LCp{e@@x?x17UUB$UEBw=`B~?rXHg z>MUI~mROymC+dks{oPStSJVSA*4JR4=&d+DQa{-0@){`|Yyu|tYc67|m6sf=pe)Zh zAFV{6U2l1Gxp8%|>fESjyjdT0kVLrM*o-gn8DH^AciPwm%{F$IPt+Tfclac=R(8lh zwFnSo$U#i@E_C*8QgITwG_~3(5155KLWzI`Emv^-1|bhyz{TW?i6vq}X+^oJv&Pfc z5bUY*c2zlJ)voSpM_ZMnrOMG(>+OgJM!Om&dRj+18wcB?-SrjC5l5rX*b=Zd`>d5l zRYa$#vg-UAnNuMMnN&8pK+TTFYuIHx1COR+5EL|gd~ZoU)4*jKc?=_$u4PeFbRvKj zAeMxP0d5RPPQe06i6}FVX5o_M*b*hK7;sC8Db^97S|UtOMpzga05BsNE{Eq+(h~?N z`>=_-(T8?o4(}o+@53G04L`UIc5r*?{vQyDJIPsz)SM)GUJAb?hhLH>LzgLWFf9pX z;gB>mlq8;~11`XnvS3AgbeRwbXT=|h#kamB;;=-%ar53ap<`49|}j~+f;T3T9JS-x;?@#M_- z#Z$A(XHKk~J9%U2Ts(;5#>$nY^Jf<4&YxL$_+Wigl~?ZupS5U?|04ifUr)Aiy#9+Cx;@lWA!H|n&&550KFDwT24+^P7GTHdRbj<@XqGa-VRiM7rDEg z6|JRJM2Nuv((Nv;tws0t38$t4H*YPze*5_8^VQ|$ldHEcK7Drg-u=}F4{yJG`JP^D z1OqW-n&$;ETW<`=q#`h!(ImrLhyD7`ju*v4!03I?Lu z$(Vh9IIuVpS{M$U91P9%`X<_KgY~+Okfh2&b4g(aMz)TcW?^NTndwIQQ5!eQBg&7c zV69Gir=QpE<|f$Re!7VaSAZ^!3s!)N`7%lHnLwFRHMg^<66TC|l_x{U|l&4=%i zpn>^Fwz1Q_qMWE1)8*&%Rfu|m><$;D#X)Sb6I!geCL_95U0f+Gh=}s4WCbBnuAP~p z#~(105}ouk;Ha$RBL-rk8hbzj-^qpiNXg!WP5TC&`VHd9-(iRT20Qds>A}AhC;UBY z_g@Zg`Tf4l|G9V5Z})Eg-JVUqJGkY)vJY&6rtg4f?8-Kc|E?lx@p4s71uigJ-t z%@R`)FIKZK40yJNgW=&J#p^+SAtvhcUC)^L*|xYfL_ie%g_)pEKds0 zlOYNeCeQX+IQc&~Uoz1LjnHdndL9<$O{u5PUh40Sh54R@Y7F@9!l{Nlp#<o+f-Jb8TM`i*PXuAM)-cw%~N`Rx3SOQ&xvoxQ$%{>IhI zx3908FFj9X;&p$Bf!&v{( z{x6?2eEI(ECq4h%CDy&&PfC8p8tq@`@Uzw4z5QIz>lSchKR<2z5x`!(1OU5#_uBot zOZRSFx_9&3of`|cub#NMGJWmx_{zDVOQ-vOQ{~1tgvsr-jJ_7hu^#i}pmTc2H#bsw zVytd%B6?z~@#JJ10NC7Q#rU9UAjawICiQh;dOP90o!FjEW=j*drkWB7qbe(rU0s~X zNz3W89e3|9zkd7t*^7JEuARAaZ|TXiJJ)Y4t=_sCUvjt>ix^)Hw)QjV#q&pV$A_95 zJQ@uHMJN$!W!=LQ^B1p7E-W^6H2DG+mrd$0asB1o8aJ;kA|7l|O?MgRddrW;tkYeV z=}zlZyJ@1$IM%Ekjw*U9#T@}|latospmeyYO;%i$7G9x&MO4sgJ*L&gZubkLcBWT> zvhs`7v`h|kFFIp0GUHoB`W8gyRz!Sb#+|sFUAWvG*xc=L0li;T(=V+^HF+3 ztlluY+fQwGkebU0Q6suSS>_ewS?Ea$)Lu@(7Fy2NlS7nKRgC%{9PWv8{^#eX{JEd?Bv*aKfn#hBvu%XGgy#2+gI|`0$%}?51 zn7*$h>kzsyg^MrJ@i8VT$s(tjq*R-V?KOyl7I}qTUF*>`hRm&%#O`uijrh zd+KabnFpqQwse`rLGsD(<=Nn$!=zM;o_2E*(g(+`qkYVO#^Pqb% z;0sUvhF!9!oRi6%T%4Shi;Nc3H z=z>azt|4G+ukyB6xw~pTz#6ty+M2_b<_ddDl{eN@d90^pxGg%;RzKKO-5zn)yLC-I zQ-jACHp$#-LC~mhDMbK6K8@6;lbOYAmqzN>D@+2WgoqMgVM;1a9ycg(J_ZOO!~sg- zfdkWX;`=7cDL5UQx}G~qNyqCsBm;+R>3~_j8$-W;-_I^)FPr#(?rR65e z&?Qn#iGlzF_RcM1yJbu>7q4exB)Ad*62eCpGhv0mU{QE zLzPsNoC=py5egb6zSfw6<>3)xGA2G57+(_|pM(Se1~4N+vf0RD4U4Sf(Zv)z1zy6& zB1B}IicJ%eFf?ePn26GG>7cEIiV+ZzOyKzlNRyOnRg07Yx`aiPu}C5Yj!#1J2yi8X z;8Y5IMukDd6j3oU20_Uo>xE3G3j7@KpS%ML6LS~X#(!Ee)qoQ8eb|X({fm>smrhQfJ25dm)OmcQd*#gR!WpT3GG-*|ri z-u)Zbub(@!036uT+4-B7&s|$OcWwE?jjKzyuV1-${mPx|D|c>Qy?=N0$&&|b09(sI zwY~|bzT!^|gIul#Q# zO5XMP?*R76SN-z+&Hx{7Y5ni=uEOir&z?Vd^!V=m2iNc2S-!J+;nwxV)vG6NEYDoM zbZqI&@WsW!-wd@XMmm%uovQI}{bav=a?k?+HaikIIbM5WvhKumw29x~T0f^x7(NbtS&3k<{BGJuzE#{l>|MPi{YZ_3ZJ}J4;Io z*RGwud-v+4OJ{D}SdPQ>T@3x&PCjp5J$d?Id@L5Nwiz^R0vXO%2s(QD<`x!*$A+Wz z;ebzPH*p;XX3);9c5@s2yq+4#P@{abNjcJ_7;RFGG$}_Lm19k+@n+4jX8o~d!?7ma zSiO3@K|NNl>In%NZ4|exRL9H-sBw)heydL$)RENmd?_}K2Tvg6d|!IxtCFO@!_zjS zvwk2K?qQT9(2Ec9QHe5Qs)C%Rq-1DlSvp3Jft6?B71@DT6PG$fB_45^PXhDHkUkkQ zphWofQZK z!NJz>Fi=Zqv@J5;5gF?Uj&}wpx&p^KeG^^2=~w`0qQg7Z?jCRRjI=n1o9zP)rojfo zP=juuL3{dG_2Zi}Z=SEbeSQ1wn|p6w-Ff+J_0he{OXntfT76z4%_t~VG17@e-)rcp z6D`Wsh47}@bg5-&GBDAitu$h2kgdgs{>*@E_e)B>!on(LS(hEx>n8TN z8SPG1XGqcLQ&c&`yE{mhufor&9wuK)g2LcgU1kcX`(J&g+<}h3M(wCfI(g^;|0LzsaMz}T%T6z z)ydYf$j9L%BVojANt=XNxD=0;6EPA3ijRQ`aWHUv7`7A8WM2v9{Lbdj2j zFwjs&8p=RH0&XcV5EY@s!6$h|bWozkl^IBIEv^&*%t}S6kcF(ABec|gw3I!V13wTF zw-ZzLkdE$0AKHmKvIn2M50|uuou4Wy%Mp}j152l;AORQU*it3%VH|=>LUT)KMkZE@ zE0vMp95jTFD&fIPIAw**;v7uoVMN+NCM=JGER>R9N*W?AI$F)dYdI7-4JW|DS#U^v z(`O76KpMOSaKaa50uv@BV-$3Pj>n)OAegoJMk%0me0^9M2VELpE5;zkbJf%ELV!IC zR6@sFRYH@T2LL8v6Ez}+l!@o#;Q~Bd%ci)s5{FtOV-mOoIKY?~_*5RXT)}q%hABnm zasiM-Bk`J4VTS>5D`L}EIE@t^Q^;j-7-WHRO;d%tvpy7$tsm%Em>4{NVjKXhzpeKC z$z!)xPCvST{l&{iFW$U(8K1v-`Q*`~J2$VNJ2gK)JGpY<)Xk-{*WwQB!p$pj2X=4u z+QU0HpFF(#{28!c&(^09*B^s@?7-GX)&JWV`#_G5$MFvX_lva=e?1&yW805A*;x2- zhjA)@)bq#nKfHbZD}eM#tBt+=vOf7T*Mm(qmj8m@#Vfvj^YRV&-SPb8tH;lt-+S_S z_2K=i_wFp+zH#p6)y3;eCx0_FV4UhVP4=0m`fbMty)#3;iMbqxyk6s z>H3A4#`&50nKAc3Ow!lI8t5kWwnKY65j|bRrUqhF1Q)F#b#<}_`c$Xp8y`N5&lx^> z_VD53)$7+zEv=lqeP`wB^`+}KSDwFox|V?>PO!B-RA1eDu-enpP!ZCbv@8xA#}_f2 zE^AXmWm7|-I%2k&xdt`Ws;9fnjA{?Ry;9a!uNi4Jj<=Y`TTN4)^G_Dx!>1 zz(HkFOOuNdzdO49i_~qu&p+@tM(F_^GuO;10BmyzNODivfsvNLP?LAK)jQJW z8Sn6qwFk!9g5&L>u@3)8yL-6RHPq}FY_twEntP*$o?2~hjXGAXidCs%m9p*%X zRw3yO@jFAjt`IjC;&uf&eN~dV9{2Km$Nif#ub!NL`|`@$*SFrhx%=wXt;Y|SZY&?4 z8j7}57?jKse8FxJ_E3Wp*A*t78qz+vQ1xV~{_&;8$Cp|jU1*%`(t8yU6DP~a&cYo1 zij@903*}&g7Sivg3mpB0q@ma?Qigy8a;*vkG|5P@TvusR%OsAv&ne@lPY9V zxqwm7NCP^#MZ{Jy2nH@qO(%%sQ4&yalo8Ql3|xqU3W4E3Kt$+LDYi_8gUYdGBH()p zGns`MoRTaNB43R!m7xk%xDq`Hu0TVi$bxb~mj>kupbAmivhTIybU`TS&6Ko zG;VPwzbsorECZdpgftbn774Bd_KpH5ixh*Kgch($vM?D5jM8i|u1th0)v|Cv1|AVq zlaSGX&>}p7T9O9@81{aru~NXNqNM%A;v5kX1&F2N(g=_oJR}EDN=n7?ac~|M3JOg^ z)_UB$l0lSFaU2X($tK&BLcN%+;8L_AriM?KP_dvB5fQ28P}ZjkRa}aM9ye#}$vc1p zbAWOg-z?!;r19-qJO)L`t_xUH5t}Y(*TqekS>@D-E9}~)kgdJOH`GxdXV~PxiSgdH z+5oU+SI*Bpd2j=aWiQ`6e;$tsc>VP0<2$R@&M(fMo;!Bs;wgZx8&@t|U%q%_<uy(=o3zkSr<<po?##`sB=++4`v0ticSATHtjSM3n(nZGcxB;v{p5iv{Q-u%lZtcmH?huD@pO{YUzqeG=fQSS3VuYVZ-ym(13C0}V!>Xwl3MRk*EkgAGOCzTA|w{` za0N7I3ZeKQwd}B%nCp_`>g=2bC$GsVZ1qUm{ff4rsx7Q(uhezbm}9k8pq@H=e}k*P z$vxQO9c~MZc0@+n!z1mH@mTdlZ|!Jj#Zaq%1Q4smGu+}CYI5~A*m|Po-a2EfRv)X; zbyWfEsoN_Qtzl_vP~7YnG<$i?UQV-@-Q;05uc^^RZ}BkO{j63mt=UCtb`e_K8iS5HUwK=lCl(F%)CbzDT!3VqHNIG_zo$L~FgDUw zHPjO6s`3DOHMsRHepA$?4eCW9y(D6m+vOauLFP9qJsJrxVF9hoCgG?VL>-r+W)VaL z7?6yF5@O?9cF~J+*u}ZztQ2Za3N0svl9fcyP3A%}z>$)fNXt%Q6&{r$^0oMQ%%P3| zD`()0WF$CNsrfEG*+fHI=~z7x24tZitW>0ng?F=X7DAZ{mLteZ5#^;w3XZDEa*Sw* z7M?FD%HU)r5fb-Mjvi!y{xc5)PX<4@4S9GMGcQF@oXN;dlECv6m||e;Obm>I2vt&% zG7>^UfGdee1s*0vL%7f!R#}#kgiujXYC6WqApwRdXebU60*ECeW7c9ivgpu)_@Xf~ zj*Tn<&Wuu)UzB=~g)RZ^O3$N%BRW5WRFVfQ7Z3|s0*GZ6v4KSc4h&$7i-w8_NWiN2 zA~2=MAY!R_RD+nU7cfChAs)fQ!&O|WT`h8G#99GeOv3_z@ra1{_A5$Zxs+=XvrTLD z^+KjgD_zeH?AFNu$ts*WmqzGR@~a*CrjWg}E;QH?J=WVYGu$=US>MxEb9R3G$^Glk zpWc1-`thqbPhY%zeE-3%tMO?-^#A~X07*naR4eCB&5oR&9lmz?^tF{s*H8p71)pzOZ;{c1#U;XEnLZ>wNr)J^=f(4L3Ibg=+s>TECNx<-bPn4CVUD?zOk^T?OL1KLd)q`OWk()y%kI zX2d=_>YEz__RBZk@9yug#Mb_@~P1W z*Umn=e*X5Q`IXZ%3&)2$TC3bntx?C*D5xSnK`LT~0^XWRzuT@+$>|aiL8oDQ?BcqR za-`k4Fj{$Tvi96$^|^`4v*Y2#VgLM)XJOENvfp{K*FN9tn2*_}T6E)$%86#xu_onk zop_*H)EyCZg{84dWp||#0Ic4@jOg(pB{HOhR;WuN8c0N298wm!gjsfOnw^*C5od=L zg%M>@L|q)x6#ErLKwf#Fg_|uv9xP7Uv~SaIw|@CQc76T(ecyhu_uDUaZThcWoBo)% z{jZSZ?}-IFiG^En`CAcLTVWa7P&vDC1^Wc}Y_EFGfu zz1q$L)~YR_?5#HR*O~`v%ZKWnLk(`=pGE+s;+`oCG}0CvYmZFEY9@Q@CVOkA`lH7O z8)t`_jt@1>jXEhvbZrrPbJ)>d;f~e$`|5+SI$uYXt1aT{uJI2wRgSgQj;PIdc2$et z*x=Rxvp`J01`Q&wnpb5}1oUE`PU6){ZE}H2C5k@-Q;SSol7UUsu?aFNQc4EitBeN8 z!eyo+lMa!ykCHNvzz^<#AJ`7r^)39sR(#4nPJSvi>o7exi3>@Sz;gA(G7YX&PlN+j z*_e1Yk5Z))grsZ>9b;z`z*{Gm%dw>fcmW_8 zH{&ojD~XzNfN?Z|mv>Z9m`+VQNKDy_OWaLJ-pegWW#t|bm1G)-Fb%duO(+usYSPh4 z3X+E^5~7PGC{M{Hh&dWIMM}Z45G8yJ zfISM}%_v~n+;phi>FSE&5U)OnHvQT?8@cySC`JOUb%Q{<-(n{$6$9> zuReHi`{nCr>i}C%$+5Pn(@!6_tdEsz!~MpGH~*it-r+lg_5LZ>#}z)I#mCI}$!%?GdLuXa}-$98^P{&U)Rp^vqOD`2?dc`erBJ3hVt-yEM3&rWD($89sC zj_G0B_@K3?!_-)-ZK~5XH7Kep81*%b?oQ>%fNOLxIMCy5X;e1UFskIDM zTY9R>J8E22L959qmCIN{K9R{lY1FdT_Rfy(m^a|z3+Z$QLZu{m9o(i$*;t2pZqPYD z>|GoREDU<*2i&tgwwYM@bfEUtDJ`;z z3+&>2GdD{`J<5h2fT#bMec+!N`!?kr{;@P|HzIpKB6}Y??*O?Zg@?{m(+l+60t3Iq zAb^=AXt#zMG_k8}q6VL)HEeDRn_EN1rl6s{(%MyH?XEWW)SCP2ZG#Q2krv;0hyPe- zWU{Mj3Q(%6X0p3xx+i+PuYR__VWu~Ftg{M~%niJU)^PKQk(T+<*7?!4lcQ}j{n7p= z2Ow71Ob+PKZUxM%ga=f}pc)ZWLW9b(pt3ZeC!f`uf%V*DoKwd~)~U&5KvgPMtW`5e?|gVl1FLA@@5YE7>Q> z>~>==4Crnz1fH!lzPZ)&;`+dashXasva3eYSFh~|%k2D;kR09O;B|+!oe@h<)ZgC} z9*FvTYu)_~z;(GhE1ke>0d!6GG)#2WjkZ;ERN8Bu$|{?@!KbfvsyrHwQ^5+GBoULu zDd)J9e7ltI)5-!mxnCo8$=Gfw(;=cNNpLx#Op1Zv(h`dIZHFi9!>1-=k`qey{s7Qb zxb2@MyS_sw>;Oa&LeiOeN$jFj5j0bcgP3UuBXCy?tcy$b3F*~pVTGKxMl6ztPxXr! zK`}ERX86Ta7mwuTk$pUJfJX`O$bL57NroE``RdXvEi?y6UX&q&WC;Mea#Mwc8SI>7 zUVaKIJCUAo7%a`58c34pq$}E7u-|aWPQP1iV%;NhT57EaArOlz^$_(be2| zstz`&KtSsF46BR}%H!FpNJtcfY3)&chV?VD#WUOag4_{EF()`S~V zNySIZ4P)s4AAJh2H5>79g^j5AP~l^@^vjB`BmQaIpWMTGCvV;@dH&a_^}E{8+lJ16 zw&!(yeD>w~E?FOb3jBPl5BluN=$~o$Ie^Vfa*t2QX2$drLxzz)&0tK`)v9i7D(~zJ zb#%A`L5|Iis0dRUqte=1T}7on5SCR|iy9l*l@XNHUTk)h1;U8tW?Wy4J~k*lIb}aR z6FNOzdG2`l^fA-oh;Xi#G2Ka@XrlLp@y#|wvl-W*!$tL^8Vj$&CU=-5N)=ZmVz9Y* zG8L&(E1TN78alhwL!(0t8gkghYparuwCbi~0EE_=9>?(>+wquXy307-WjP+R z&Gy)5Vz#*+$6U8#s@*c#VLl$SPqZ5cqKe)cS*%*#Ri$VO%4*%Bpc(u(gYj{hVnU{h zmTlnVnRz)@L4lc9VB!{7`Gq!7v0Yr|l)*g;q(^~p$ze80iA7kbVP?w7N0pQ;897Hr zDUeekS~kKaCA!sAkB08oF+)anrHNB#=Qnu;O+HbhU)Els>!~a6uXheKdioo@gN^>d zCjU@#V6Zhb+Fmi<0Te#g8J_H}ob0Lwnu>w3I6B+kI6Kfd)muN+8=V2K2b)d|Hc$7} zAM37}h}D6&vqNj>8i}K8eyn|AymMi^<9L5vZ`4}j6!`T7w-V}7p_KNbol6nqw>Jar-d_y_D#)PWWBdX$p)ftdcBeGfp zt<}M5^{{Fkv|3wMtt+e4Lo0Q#s0G^*)%M>ifnSh;p-?(|eFR^xCfX@- z`RMB4z2&~M)3pPQ#&S`qiCtJ_U^F`A9TDqbQ*fvm0L(WK^$s=q`=Xw%D#uV$c%q|j zyuEs)CDK#tin=u+lPK!aG}GG*?iXLrP01+_o9Me<%FFUc}*pkljBP@7_|nYcq7$KQRZll9G3GbCaZ{S#m_K z5>sFxmpWKz2Mg=u65VlD(d%^LI)fxAWqF0PIJzY4fP@{CvIA0PNXiUKn4lc+tXjdX z5HmbXtb+vXSCJW6U`7^NF%Ug0PhFa$D$P+8XDLduB}Hlcyd-c%ZcdW0AdQoq#0J(X z>#(>a-ApbsQQ!&;gk78=!H32!@?K{2(VD`&BNl4y9iRnV@l^jhM}pVcCtMO7EDw^Vpy z4V8m!b<;zgCnpC^PWGHU)_!ud`Q%Xj*@^ZGGd+vro%55u7Z<0moL^iyv$%5M^v&h- zcdlHzb>-43Fkv^>G9tctz6P+@FJHZiXNtXb=gE_2>uIIde%il|$8CRp(ELBFpBb7z z1;z*LZLIe6J+WTAw>zK7g7H%u^l{Ih*4$8Sqc&FlQ2*mEZH>6^(=4r>+PF;ZGyQzf zzX#aA^x@0-&DaQeY=|{JC?4&Xj`qt2W8#ihQCGKZXsn@csJ0@iGurV^55evttBh2U znjzED96nw}6{8}84TeeH5IGRW*4AP>TPY)bf|JMe=Z-tiOq8D-RL%AAkN46ix`@YG z3C9}oqc!-xAil$aturFrGMJ8!lnV$F0ZAyN2zX=$9gfGBXtkQ2!O`BaW3I}8P|4A# z8DW>Cr9wT>XqfIUpX+m+=y#tO^34x=rn}9#^Y_H?wz?w4~@E?!4XM4RT z`u$_A*8ZrbyGqensb~w!>%Ed1mpE+YJJnQ+7;BdjylPU=K#7>?6&6N?g;il@)s%DU z9K5KLU+WUoyM^^GewCFG&=Xt=ghfl8P5l+6Kcd&C&4F7J&x`l60l zwY9Ut&{Ja`Y_bkEmJc;M$2xXaaG|^Kx)fWYt?yZ~bsh#Sn zo9wQg>aLrL)lK)*ty!+dnSnI{Tn#kM4K`2qM#p2dQ{dE4^KpQ*p_avw*2R&wlOwGs zMq1~Fo5ni=UDf)iTNpM`{aTDi2DJ(D4eTrfGXpqFJtM`yjLX7KE9Yhco_WN8WCcJz zX<%pN7s5DT^oG+VCLz~@mm-BPLJ30)W!o#erOs_ozh!FWe?Z(X3f$^~p$OVC(tSJXPx4IX`kQS4K)!$whs zS!&?obsU^kOn0jU6=o&yUVb&-D`#8T1PLOaU6@A7ID|Q}C+~-^^MCxNc*mCFUE3f# zw-xQ!0^9pNYTtMGLqF1z_waHOm9Sh5y2wn1Tj@v-7atT-{CtW}NDD|=wK_?oMO6hH zm`WIu^TRS;ST2Ytgb}5nLdCC8@*;9>Siz0RITimeYj6Es$Fb;po_}HHhoSXm-rO~J z-o5996DM)vfMaG9GfP|A%*@Qp%*RSc|wMS&xFwSfTXDBlLJE&*}|gbXN`l!9fzTY;JJuw4LSavA^<1As?2uVR2+Dh@@; zATTf;6nG<#(5d1O^+KA2h6MnVP_bMrT*)E>GqXy0MiEQSA_5z;D+L-p&8d-CltP1? z>ozNb4xPgwGb{K(i#p@Ac86U9$>3yP`qxq_MbeMhtt1kzTcV(wEH51T*%_V z_x}YgG3xXLYtJNS za*WcI`<(1kd>Hwqh!vI;b4Y8ut&_0kn>fyN--F4~;nY4_F6c zqL_`8ax(ft;?bmPy4$?B*S@#cF&LpX?wz^V%V4u zX>&eJ)~hK5b-hvJK*BtjG!Lh3BWe3++Bu$ekL8?WImdX;F_yLVMpSV-+pj13bhwC_ zn)8VIA{tN`xYT6czpDroDzbkdzJD-2k@F2EtfLwGo_^n+e*fOV@PYZ;R0|wh$N`{M zqv`!)nSeZxrrw!MS#eS>kxzy_0thSPvy2mcxHBhW8UXZJu1REF+*jb#sy zWDXB!4h^M`jAjpwWv2TALn&jyFOFJiejP5L!T96|hp^2Iw31$>A?;8Rw<(BQ75L3c z!d5+{#LBF6ackYYT92SEBx_0O;28rdu7ZVR%`sJb(g2U^JL3BGjHN4Qh3D;XAVdBgHiFWqT~GG!CU9{UpqDZsJuQUcG$y^4XndkFVdqeg4W%dynnSjAZmd0}k8pMOWnq4n@mYUUq7?``Pv3 zmv^S0-`ey1*1m_AcAwqXcW^j8-4hv3xrP(=;h1G8Y#0pb1|p`Zyniwe4cX|A*z-Pp z$S90iBssS>W|H`n?0||B(($cgik^eBifJ|pBV>^H)k2q)W#^F$RHOvnK&{!1*tQ0? z@rT;)mQ*bLp#Hlh_1}HjvhrKl+8>acR-!ijfZw{DUcOGyP@;s@8*!Z$64J%Odw8Uv zkQNfr!(wJc#z|?U1+zA#l|_}JxLO)hNuvr;Oeu*f#9^5rEaSzM{IpgK@D>y^0wOw) zu!w*hV4~f`4i~=7iEDS_JDdcVlh|p;cg!=aUej5tgcw#LXxJ$P0BbDgHMOGtCc2RB@?l9@VZA>Vyo_{1m_@InN^Jd5!XrP3zEzT^e!Ju1otY z*?_et;h7pr9X~jBWcT1?Ph=t=0H2qGP+vvn*zSRgClB2G>G;i`PTahB1{m0_OF!Mb zeBsvBOLuQyg8&9O$s2${<2Zo9ym)cv&Yin=?>wJ$8Q1w zY`)L!#e8%Bw#EM)sd_hm+RDERlG%TDA^uT*>ET>-ZNE9iLe012SX|-VnhTrWbPWEQ zh`)P1zff?Y+CPu8xF%HR)h~qp+u#std>lJ9iJh9jfQ*m92Zj+nLyUA$;fX4>cD~lZ zGC0UGBaSHOLNnnMAy#3c>K#O@hivy!wHB1g-WiXgM*9H3uu}ub$zs=7u4ORQl8JTN zZEbp8i`(3kaMt8p6$N*F!3&GnFe(WKM`*)Q5o|tdrG#`E5Zynm`YFq(G{rd=bsz(h}MSATLh5TjveWW@gA zBv2wiiv>N}Ka$ut7#>brG9Gc<&PloX-9gzS!gtCXn3&$plz>~&u zzTUVM9DLtUWZzJ1|40(p(wj`?4^0$60Kg86Li7Rw;5j&&+B+28GZ+E1+uiHm)8pUM z8{E?ym@Yzt;O7|`*wY)`JDAwj7oF;hEHG@}U}9f?e1CspUmsu|;9qnkZ!ZL7aT_b3 z2Tq9zXaT_BK*DT-1`Dg&L@U#gOBA@R64XW|c8i`;YGu{9`1M{G#w6{nQ>0sni*?pS5NOgG@bwPK>vfACx3o<_vg8XfLjZHx36D6dpS2V2l4Fb+^ffPa}VcU+ z!u@N9&mHa=%^2-cq)pOgnNt{fe|cxdFrRBj^c z7>pWv{K~LiFJ zUV9Cz1$rkLu6!%2wOUI>+65FN8z;rLOGz*jpXgC>g?JbZR?o+FXxRiA8KI)%M1(F8 z0WKn=7^rp@8U~mIuinWUq+wKCikeHNA(|PeHoz?vhrBRq1C$r)8%1m&Z4215t3+lQPb;9?lmY;) z`8hi{L5nhAR=Kp2utlA4>XROQE@UYtoqNZ+kMAEpJlQwhpV~73zTV~cjTR0~_MJX7 zb>sZ8yO&PgzI6KbI(k+j`S=>3Fl%-bUxu zliBr3WrN#T5wn#?Emd|^t%%)-!8J9(YvH6WGMj|OcOp7!u&_ETq6v#_6AD>6y+UV@ zYE>+sRhSPc`(x_CxO^}!?Fox=0bwB|ErwNvkSZTi<$~H`)HIN?kE9*rS?5Fndh%&k zw|Bbe2g`{9SZN}M`3}HCkR&a131%Sc0#Y@a+CLoM(;uAfagXQh?!HVuI-IFzvh z%0|refS%~rV7&^YR|a>A+HL$sJEz9NtTZz!wd67xZkvj*-NdYN2pR*5wxkh}F(I>N zWZs6#TTxjHB4rsIMFp}2G8kYD1X+DS)G{p#uK*H7kN0f0S!1@da<=eZ}q z&tAWJ`0C}o=Z`Kwy?y%f@ri>Yv800=(4o4$w1Eh3x?uQm)PMDG@y^MyYljCPo!@=u z^w_z5g%i8dlX=fX&NG(v_D1Z`yHw)Nlv5cp@Y4=?w?~_?DPsmPpqF38whPGyE?!MX zIwW+Titkf$og%7*h0_u{nf2Q{*MGh3L;+Dc5sW(NzdC3T1pjUsrX5)YH2 zo0%=;*z!%Z`VxLe9jC39)=-Wv-y}dZ*mxw5l&+$52{Fx5QYVm66$4A}tfzL=&2tON z{76L@^!_sviULjJO9TeiQNC45#b`NH7P1}RTtn+@77~yg6b#!?$;WmHuy9&OlMs&- z5RhUjPA_5sdTB&VgPbSlkR>#%mQRz?adcE0boYyj0Uo9oG62{_6tskjg+_mf*a{X| z$tKT_;}Ak!kz$rrA<&DMT0X_876i>ouR-D0%3@Y+$gIft4Y`PUAR9P1*?;ojo z7cJ;nqPaO}Xxoby&#qm&a_01@+qZAezJw*b`SnZ z-aO>HX#KnSmyr0~%lPlkuYRvT{xyHeKP~ECRQp#I{?^lr%H*3;`LbMZX|As{)m;`& z*P6X8A_I!1AW&pPmYOQiGRPuaE3pMZ?Kz!24q7srWHX%(CBy( zk0Rw$SqvP3)J3JDIb4!JK-MZ5L8mky*7n8q14-RrQa_kB4QH(51=nOx5J<^HZ)CIx z5*{xEN3-t1gteWckIAq z|B>B;ho=X@*SDjS-A5<$M<*BP1yTe*9vF#_WUPgdI_sAKv**#noy@y? zBkGis>(`S!8oX1Ewu(CSyfzKHMMkaT<0_cQQYNCpA|?++9n(FrsqXMZJ~&wjfnx$_ zO)mhfFEQO8A1ek1GPbN=9<{Oq28vgM_sCIhNta#NZsj%EICW-bl>xdLR;eVFYN+Kl zL6cwEnJ}VLW=z}wPZ+y0R%F(OOq)BiHh9qu=mi0+&r2E%0&cMf1Ki<|U^pTij*3R& zlA)MrFeV;|N(SSy$!^=peLZ(BAANG?+`Sto&z+n+y07=-;o+Z7?YeRK=)*hbo;|+y z>c!pJ=Xd5_K6w4=$*XxMd-dwsJkRDk7y-#<@4kF;?eVQMmrqTc+?PH!2^`cmSul^J z47+mnW5bc7gTZTud#@kszkYo9+KJJVySw-H$NJ+AK(Fy!s5fFw+T|IiBIi=3ZSuHL z5>RqIQl?8xGjRwqLZ^jG3Me^#XeGnMz!-q^HXh!>FwlNw1{r%37KIg_er)fH{} zf<>1#Xmdsl09abD${ICEtun5bgC7eZkT9QC3IOKT3ZhOW z09Zci81GF2V*R*p;@JMlQ-`O2`f>lML(_m~M*zn5k6bu8ef{F$JJ(O%y>a&TwR5+x zUwCx?CXlcfvrj-?&OM)hq8Vb??CkTInJ2ez-?)DL>dl)sA3vRW_twi??CHK7XPxp-tkT0*Ly8K z_~P~N=5LWylg+!?W$3sfTDgAlY-bvN51G+uqocfb-nJ2{XZEkp(qj+{^FQV~_Gb{VZjOs*G`>qMkl zA+eT=tzsj~neb8$x{Qe^p}@A&5T$M%Z>%SIV50ZPWY3|A!hYaJ!}0xtk^KNLBdPgY zQ|b9}6S4l3HSJfXJhFU1HJCI{7rlEywUOk3(aipl%>JS5-l6oqk@T+q*kHyzn6v@U z8%~&uAx+jROSz;Gt2km2<^8&$lxwm(w5K<+ZvfCJ*dH~8O-u#74%@sE-MA9lxC+y- z5?!|fQS&{l;@j4eWz9R5c9egw<)Heb_Q^tcJQp0x2EcNjx4hFW?Ql!l z-O@Inygi`k@X1;{()xrE*6Sto2Z-~b4u`p;5z$CQG8~bNCgkHOp6XR=>CmU&mUcVa_`dZt0%6U+xOG)sk28XP9GUN`{U%*^9S$UIQ{U>`I(1T z=3dx2CUjUlyIdSs)SXVbE{HTNy;Kl7$qSU&&nlQI79^j zt{?yz<%ZP!pqyi6;jAo-9NEyd>FXVz{ns`B?SC!#?C&4Y?G7Qj$tobZr3@VdEyT9SDP2xEQ^z4O5zRadOoT`9 zu^mEuy8sUp6X8Z4MS$xfwA3i*1Y%n~V3>r2G6|U?A`;hB1=z;Lc5yMCh^i8xUs4iA zh(iFX0pH>h;A{e1FJ&8K93_{mV3PsB6l{`)N8u4*9CRDNq?C-*vIsg3^x7I36{}{E zKv~PC=y){1EvHIolroGGrdi6-38)%A*`W~yOscR^6*9^_dZEuG%7#t3h`m1(+%=dx zFgbAIz%D?qE2oZLI(6ji(LG1^j_jN4KD@X8-0`XF7Y^OKaq9lf^LKAtynpBF<41R& zJh}h$$%ENh@aH$X@KPtBUymN$zkBy4^p=dd+2?aF7n}eZ1-_`Ee0R+}8(uHuFOFUO z-G=x#&HduN{)Y!T(Qn3gEUNQ%)8duT`geC<{ARC3zvJTkQu)_EF5dBv*1x;;ZyM(g zk#@Pv@SRAvf+Af>S8k!mcC^uVRO2e@No|dkt`;%^Mng5>+iKu7?f7N{r5!_Sqw%|B zI=s_Ei$@tfdEQu`{LmiHiKC%?(}vyx+vCCWWpEg&wY;;Yozf}Qv;2NhKBex>8oM)g zhf_mjkvfP-G#w{Y2`x^Y#i~-M1uP+r%BApxEWKVH4%+&Pk;$RV{?Wq0iQ<9D?t@cc zS=>KSJUH2Zc-PReJ)?)F1}FP6!?{o~Y7LrrTx?C#_GLAjKCj*UdBe6ZT1%I~YgQs^ z*T8Gmpz7A)n>P?!H)0#tQeZp%CebjULT`8^=N`&92Gf?ogrPsGA4xkVy8V+qp^0K> zvL`&=9Vmp2QHLz*lx6+ufrMqe;Mvs|n(mKI_s6DsBcKjcvo|`L4-REKBWd?Q%#!g) zY%(klUq$WOiEZ9O>?m`oIXyA^NX`f9@9G6K@)biypN`Hz?5N%JN$ti@TDC83-|dylSOYLL7`cCh{RbuhEPj0Bou|GSwY|QV36W0~7DD;3CFxX$ zO`?FF>r&I~3cOv0wnz{rQI|!~ZsIj-nYA)fnUqwnpjL|s6+HA#0cIy3wVj9D%13Pz zptf?mHV82rgX)gH0BtD37z{J|gUsQGU^FHhiOEOeim{YxEUg|%Dn}BE(S&j|tscu5 z_YQ{6{W$sP?oYGNZasf;ig+o`*>^`-B@Wga+SGRvC zYRtQ&8JlP*WZ2ss*xel%OxXLQ)`6%!>(Yb`qO@I}u!ypDdBLg5S`;o3O-1U`QqdAj zhlNA+EBQgS(9R{>**Gf=iQT@cWXb=t^56fb_5bmI)O`I3Z0)z`O)Cl8){;v$Fsio+ zTPt+v7AFPQ6W;>V<%q=J_OJQK6xB4$FwjPj^a zHaW~7257h-4euqRokXOQh_GTiOz0LpqEXjbr|+y)wpJ?Jt3}Oa{KoAnWUZMDbMpuW z3Swb8D;cImifO@BY=&)FL9O4(ZLa}x#_6b`v;YZ1`V=g?kRm6-WaLg0pX5^T6f_L* zCouuZ$8_*8tx^(#gJ{t(a4I?xT?gIq5)zQjJ2v23YSbLEibJM$HQ`(9uuau0WILm? z1zTUqMZusq%aBk&v&1wko6sesV}U>Eg-khS^#NcjCrRCoY`Wf9f#MvcVq@44pr*>-vQwcdwtl4?XpK zZRYWvr%&!bn|U+`4D9v7h>ba5V^3!u&OCke^5yKS*RNi_y;U>c5JOGmnT6;@3ZH#;gP}Yp7EZ;Q-epQ zhd>TZ4;+~4-#5{>Z=(0$)WD&sfqkPrlfB8oY^V^n1uS9?zOibDj*@TMc6{9imZjff8aBC9jG?3p7}x^9CLn-?C%_q@Krq|gpB&4FasgAs zA`KdZ0X@&7X1bJAmyBqW;LIYlRRA~hI*gnaBd1xzXrQ!iYTdc4b>}x7W#7Tdzw0Rb z7FGr=+jcG`Hm~+;kOL9^NJ2Oe;q^!O{bA8yL_V6(j;9SHDcwj)3zlOk-FQ+rmexXQeTAKpBF|LUpxSB~F4f9UGT-9H_hIDL5h?2+k< zC->htf8^ekA0OX3@#5k6SI@8iJbUNmvs-g7@6Ww{`tsGYSFfJ|hRr;hu2Qu zI(PW)Pe-mCn>e;BKbW%h1ogu)>v-Bd7_%0@|3%1@_h_RgandGF+huW+u;^BIJCq?g zTZ(B_5<9gNq@2*@k#YPgp+iL1QIQ@#iCMF)YU%sy|MP!e@o)e0j*tJ}<{y?KHZG@@ zZemyO;MbQ(+p9I`76%m>0_YObA`(^<2$M>fou^UWY$)0wX!X1718zs3)7oRVfFHri z9uPZ(v<0daEXKS^pEv2Cdtf?sR<8#9%j*;cy&|iDR4gIq0f41}hsiiGArlx_ghvgt z$uTwsFf2qT_^4PP4dIIEEWw0s}3FhYF9UL5_ zU#Mmox&;)t9o21{z9&|1VK$Y~>PsZ3Ws@N33Et`UGl5mV-mPN`oOIZM8b`>GXPiU^ewpOFswzbr3Z>cG3uPukwG~rv23^}sKmOJI^261LPJ<12V{J_*o^A*9m<43S)DwrV0FM>7d!dq%r=59cTQQ=`RrUnZ0ZJH2KFm(X6j_4|@lpOmfoq-5pC+gE(FbM?nN zSAD#5#i!+~zo_2uMcw8vYd3vO>a0!&ZN2G$Q_rI!syb`ebW|>H-uV@wbxXj&?}^y@ z6VOcO{Yghp%$5rp<4#S~uFM7uJuxdl8oRd;f77(>o2KpGHf;MAUc1?;;SMA{3-lVz_yD&i0RpqW(To?Y1Yn!)jRW~Bgv~L# zB47~s^?aX}>s7Nn3YtSkwulK9A<8O1+Js0muTy}o=qO!YvF5W<@K3b*v+^}dD%N~n zx%P{Sb)Q$Q`;5@MF=!wUC*;t>b1_+OSkf0!49B%&Df4K`Je1H6gCq?jN&OgDrVJA~ z+o7@am9qz)0)Wlk2Ml{LbMwWM8#DJVKe}=L(e*PAE+4;l>DcXyN3WhecrgEYGQ!U$#Dt9a{2;@)Qjpfl0E!A0!#t}@{chWk*EtU`+dQ-^y$g=)f-Z9tc- zBiC%BH&2K>6}d}8fjh-iz%V5!)6s4@&!ZFwv7OZR2B2arcq6H~f{$(!;oA%x zl8TP!AUo#g@a<^Xxe?P?DQDopAFP0cVj$Y6uqIg9Hgs({U>L2V34l#X#{q6Z&nYwT zU@4{HjRK~IP0?{_09Oms0W0VPE~Zm}M~DarC6lOz#0$?SAi)xv!h=bYGl(WJ$1LG$ zcr?9`VGx1P%~G~S!uIN9PPNFb6*)CLr-qaC==zc#@Y$al$R3;;JpbeV>pva8b@|lI zizjbiI(g^H$y=9>Up{l_+>d)Mo;q;<=Gob&H)dyUJ)OBTGy7;38jJX9eo{mTV1Qn8 zub$5f*}@}YZ^kLkK@T{;UYPpfZQ)IzZo~PW!}%=qH!kjS{7v3=AuevUU(63Q|Lw|u ziVoCR|G|#`9r?{c{*Wb$*Zj-;v;JS%aq$`7uKCp^EjrY@do9%Y8$;PamNk+jbyP(= zNdj+Yv{&I8%TWzgn3f7;^XA6#4K>@hH+QDXv%X|9;0wD2DnSFXeN$cerux#-ma0xXTrB0e943#`#O9DX(d{S-mLi~W z;?6B+9c4(I*AnMS%}W4)=#{>(&Q zdaNfoT!;_mV||%OI^=Pi)GR`0-S(9g>z7upTUxnxY5AHj%hr5Rx_Zfu<)7^M;nSTf zpjNwN_2(pbLoVv=Ne9g;4jEAktJ%;}_CxLFrKtK%KE0qP>gupwyr^WbPUJerG6bjK%(iP7%pa4s~M4)i5`*|6PXQ4*2OE5G?<*^>8` zefB|Z`F5XOKbVV-72;!{eotZ`6H57QMi~RyxV3Wq7gg)O1gT#C6}aq*wO^L6g)V!? zs?Qrr)&SP`r+k1|BWVu^D2(SIdV!4Rf|G>^kg(y5zdLM8xHJKy*smA*^n8z&>sB%C z3c6KJbt)(h8Og*)sc0=scxg+?4<##?Y+wG#w(mdQ_C5G#SyHm%^U{?|N>?qR!gj`O z9H3dFY29#AI}pnF?_bxxadH(Uua}TbazJK-f-7BZ=Tsm>%!tomyj$S>z>+nQ4=jUfU zoZY?lQ+w0bPLDsid2H_K#n&(Iyq;*fIOkG^G#n?NpeDeW%@tx~vzCf-OPFD; z$SNSqi0v*R5mUB$`=|fD`rrSzP4EBLs;@t2U;8a?$2xlTR$)tp3|2c&FI13Cj`HaV z2|KCaCzQgJMw&Nii&kR+ilN708uHnPyp9349RTbty5{L+0oJ8a z7fkw$PLrSM~#92)6r074nCyexupyxAi9td)Jlw88l|mXN<@fo9hBB8VpAC(-K3@? zO?3t29;*r^sc)Le^<=`)CPA@xVDiej!?z%Aow{@R^u23m z9$Y^M3V>ecf82ZZ?4c*O&OLv4`T5hEFJ>RkyvZ9m&K?4g%*EW^RH*&*Zgy1dz&{iiM(?e3;Erln7_Tw`4jyOFYfB(G&c}y z+URYq)b4bxt}@tcZuOKUbPt6uk2^@gvj)_+wF7`AFj>B`Sb zR(!H^#m75Ve$K$cx|9BV(rZ==nD{PCTXlQ&=GyH);5#aUW<@dT>4|yz6F{?k{Yh8M zsq*SYL9;aF({;}SY$WR+%ZJDEk+DK_qL>;lCPxd=(OhsK?JdSVsi56$QerwAmw)x? z(vSY}<%jP#)|7@_rm=i%swXwklN{?wfNM?!EE+Mjsbby6??2l3-A5b0{czK_A8now zD6e1k;p(LyRBT;k(~1XD{trw#&^GK;S4vyPVjjq@rL^gROET4=U zR`JBBW)l;wz%*4XdvD#}|I6C{_%G#OzTdF?bJW&V)T*t5<_Z>k&pp4IWA-; zB;16Ymr{x{8hOE_>9w1>ZKi_FSah2DJ?25LWx#C%1~%XV5(Y5ln2){J@k@H_wxZ1f z^($ITMUw$!Ud1$dqbjWxCsl$pK%)|37?7}(f|rzXlM+s1Uc%x6dW=H}u}C0(2F^=E zdudoF0pY}U+R^O*V+KUCuB#sENQBkMJF0ZJX0L!4l+rcC4!|%Kwq1d0l_8oL^(B~{ z>+qGE1&9WEYXu+CsHP%JESytB395NI23kWyyJS4CO2S9O1egvL4Z}mVVC#3#Ix3~4 z4kM4KVd14Dlz~g7wKsHC?qI-MdFW1{Q*tJO3~K<~YS_G{b;m|RQ#Gx#Nkl>d_ma?Y z0x}A~Oh81+Xm}0y)N?5FffxpCOvj~)h)CdJ(0o*Q1Yj8OFfj!SvLImqV;U|Env#k` z*79j;4h5RRkpuRnT4g+kitkkOYzj7DSkNr%j@pL{p^5(F!Cn17o!Wcj(s2N=TNh8= zy#fIC)4i)_ubn@7;rQP3$97#gz4z{wBTw(2fBx*|%ehAYuRu`0^TUXq&Axd2;`xK; z&+osOeLVO2*-MaD&*$C=z#BxkxtZCyr!T?s<hCy=0OPc zM$TK>ZyVhC{L%Vf^4oU#_wu&RVobif?;joKk52Ix7_;+7e)EpBw_E?(^JI(jn~VID zm+pksm%!>a)@|QZySWZo(?)Ko zMb}g!Dr%6`Ex1~Ws7-CdYb->W7N<2)94<~IBrg`c@uXjE(qNgyHe6>#d+nC09c4{b zUHC4!Myk^(cw8!%OD0i}t?;HgSYsyv!Qj)l0y=|DVz5XQ8j(uHF{lVKrjdbf;E|dI zq!uo|iHWHtb(Il1D=jKccicOW4G$ONqs7EfKH8TK_ND@0Wh&@)n>9p4)6TVDm8}Ir z1KA8%0){PUP34-U6{|llTfL-g?bl)kzBd!h#oR_Ek4;42U=3|`JL=2U(=aUotGYYt z=}mZtvVpPga9`5xGfFKowp%BRyEXZ+r9a^sgVZNHo(qrV!V}$z$zpPR9$@_mcX!kU z0nDsIcQt?Wf}G6&vf0_9T6ApGhsIRc>AJ!{>irx%B-N zOW$Al)%&Zy`T)w(_f~!RkMBSGd-=B2R;{Qf4p`+KPC@`1PP>OdIqe%tc?S|6Z~;@r z_+ZkP_L_hu`V3N!PUzBbU23*R%kyYBHW|$#p_oJjJrAXzcTizvwVS@(^6kePzW#8- zS090F{QBd~%Rbt?>;u3s7P2bs5&?jXXMk@RfNudx0fvE$XFx1qIgznVWE^8@`*_;1 zt2=mfs^{9dBTw#KdiL<@(+5|e<(-QFTEM*?-Z%v$45-+HYo~5s{PFh16A!PSe{}W4 z$?3F30%IUH7=_I`UW7I!d*)7rzMp|kso*G^AeJw5&4+VP)X++S=R#%&CvHcO?@6~uiMh+HunL5c^&=py@-Hb{Vpd6;MM}nKpF7t z&5bdusn={Qnho71L*Agt>J=HCG@})Rq%@+$yo5pH5+&?}m>m}~f&9hzj0li0COOO| zhnd769Uq|M0#u9(-{mHBx$tl+24+ULsyphG?bXW88U?)8LWBE+WGfRbM>Yd`fu$PT zF7B)+R&2tSZIGavm~EBJ)+!YRZf4^_dmX)q2W`x_%2FE1FEL9e$#5O#5Ghh5v>4h z3;l;c!uUj_fQSN2QnH9rDi)AzA!-&GD4mdiLU1D6eD{^^T!X2T{(B;*2R;zE}R7J zb^qEq0I=JaPhS7&$fc8eF8w%p_0;t3ONSoaIWzP4$_w!IXy*3JqZ>01Z_GTt@$A`M zK(*Odk6yfd{9^Xeix&@Hym;{Z#e-+h?>~EXf9BcUCo^|uX6}N3n$Moyn|Zdd{=tjc zCv&f$4#|H`l)k8ueFbz6Z2C(&^5?Dg+f9oP_3kdeIl|lWA07NR>mikVck3U6=TGwX zm~W5#CkOe})h?Xz*ZVHcyH7WR{0;clme%T3)!UbsZw6Uky}h)(rmVAmdu!$TnoU5t zP#gr3gKQ?ZG?JR?admCvMyj++ZY7EJ1f~QfP*eB{lFGmh#T>q{hbMzfT8GATz>qDqZ8eoGRV|2CA`Q>xP`Nw?m(P?+_zEeHMeTw& zZE3Ii0bacVRks#Vvl>>jvbAz$XZ>cQoYfuo^`%3DxyS;*fXV=X72=*`(CIR(ab5L0 z)-5Yt_f`42uPWCst6IOT5|rnANG9UcikSdlL_{mRrMjhh zJDZFMIdp}XyO?kfr2Wv~#+cJ@mRl8kmrk7U=<;DJpwuW}Th>3C@sFefWBKT0F#!NJ zl=k(-UHPy*;j`Ke@{ZQpuRj0evk%|<;?ob2UCoJreIyHj7aA`_fV%Z0JW-E9!Xa*5 z`|Yw%{{G#P_d&k@;)CU1ez1J$2P?k*aMjlz{jl_dhN>+tv%Dwn8BTddlCEK>_tG<* z^$%zKgTO>nKm;RGJ&A#&FX=XfEWi{cUcK0-7x{F2ua;+*)6HTsh*3z?a?vts2f4kx zdi}Bu%RXAO^nzJdy}jk zN(MsLSjst;c1?AMj!gAjKYwKA{ye(wU4DN5>dc)>k8YfMaQ)Pyo2Q=KI`{PExkuMd z-o1Pr0@&4a4=*1-I-WG~JA}BMdQLOFV>7K|qf>@R*q8;MWH@Qq-Qzntp1N{;;_>z4 zki*?Nd-KY%tLOF}9MAU0Y`tM~E@()5G)bo{YUPKFys%MF^y{*AnMX>quyKUyEr^|K z*lksAF+HT@24!qvSA&7lrN+0^t@w1~hyVAECI8v5Vrl1w<)q53{I(hqtX_-juu{-Y z20jGTN6H31l-EcK8hH`W%cLrp)jc+Sug%c!wg7q+U4UNJ0iUxU0L)|WcP#`!%eCNP zpgibtF7zlCt)_01ejZ@@f(aOy78qDoFVE=28Jz^ESVAR$#?vc#3E4cuM63kxKOsBH zp-0)Y7>5>PQ=?2$h>i;a9+SF!WTc;haS@Tw&_q-l#ITN9Sx2=F*WzHI-8_N{+p0hT zdbMhBtx9Agy>=V6biDvpCxF+J8%so}WVc;JpsN@=%7$Y0!mb3gCp^StSU|@1G znu}~9HCNEuYsGk&nu#^=NLn^t!zEMTEy%`7a(e^5xf*~<&82YB9e`Uv!fH3HMAw#~ z>dN>8_yWL0RE(I4<>3)*OsA3!_(TBS20Tp3AOMj9Dkh`iL_`z|*)G5%#3Yn-eugR$ z38i2Xr8GRSF|Y)u07sKE2zo9}PQ|OB{zj5P1l)_G7g7PebON$d!-MALNVvv(NezI5XK(dI1B!S?p`~7?d-mvj*VSBIdT2Mo_jZsJ-&bT>4Tr1J-GP% z(bX4^ugyNY{qp6#*|~eOv-h4qzx({z?dQ*LKbyHVGjnr(dF#oO8#7ODJfFD*@(juy zKr#Tf=Pw=uQJZ}Y&7=4hV85B1aG~vfGw9%t+i_^?pSId>HvL}L+^-n$>w~-tgWrAE z^Wu}vEn?*2GyZwrJ?Xny+NlI-&0D=&sPMN<_0?;u%a-rl{PmXA z%eJoG+E7-HYAx@q-_%gDrKzMERo8%TDuY*5AnO{4tre)67D_8cf#Dm;6e$+T>_oDV z&A65>a+lh!Ra*32WWvto`pS;RoegChOE+!Uv2pFD)gbG)tld$zwX}Nsx*coRY+t*n zY;#R>H3pC1@fZq)Sfi1fj5?cHE8-F<$ZCAYR&4WnY~y-V-RiE|mF?B5uKw(+~t_1{)(_`Y)Eca5z!wBOVda}1`uBN^{-$~~C$3?#k7+0aBWHkuCuu=PakxsW~THW}5z`kE4eu20|p z`&UaoCSbeLLC0_kfX6#l2!YDINpHkscJ1VY%Rk<`a_sKa;}5T$zIWm9zQLeXgw!ya z<>dO#vhOjqs~uu=NJ|Op=sAz9H>8{H_U-EnotWyraca+%lhZ#P9Y4Ce2>s$oCjeO1 zr^|RXNryC`XNC-%tW(+R*Lh`B9Tg$RwzX{d4pp{ZM@B@Hyr`OIW?+Decm-ra#fB{( z{XZMu|F?>-K5YEq3;3qx)T(V#c%vHAZYCj}bexw%4g>v?a|=32QLiZI6a^zhuYyJ0 z>oh>!hCb`OfmsLK_5qK*&t>a>lLde+bSK*81JSF;Ztk`iAb?p6J!Yu?5G1EprnOQ) zudH60(u$KBQBo~TtAr^9KPlq@2}=k$AaMaZ#$(2~^f(t9(-C44f()#WiuO`aK?dGO z!`bk#d4{#>5l!l@dNr!SLhkf%@n#BKiE7f}T2-hRJbV`$1LI&|Tx_R^h*Z!4x6lBMK)i$tlO^(~pbSme!KEpvcqs)h zBw=NAqE*h*2>`$t0AmIbHEdI+eTM#&XWv-y#QuqE=a1dGc;e1wApIvn?%g>10n z4=>%nb>`Yn2QQx9b@9yP<+GDFF6_N~_3(omKR&&C?#07PFQ46f_43Zk**mk(@61A! zy7~O+jc5PN%#-Vor9HXv{K?H{PrxP+2xCCV<^lF>_SMVTKQORg+T1q)`|bL7^Scj< z{j&8>H$jD8Ka%yEz1|cS@$YT@KRWpD=FgG#?lJ!;@7Day?!Z}}&CM;$LANj>^mj!N z+IPP2w-u$O%S(2Aw`21c8&-U_X4%H-ZRKq>TN=yP*6vtavu#tu&UJO$R#t7^)=^W2 zY}^j3u0S<(ayqdh6oQRtrFNFX8#XnRZfmK8(ePb#N=a+W*818yL|a)?#i}iW(UCEHstdVs#RhDk}p>*>P zJ2x&b-?XxN%c`o)-&Jl}R=MHp@-<&ptXo>Pc4^JFl@_f8?3M6a^%9nVhGpP8ah;9G zmKwE?k?>jaQCD}&)tB%OqyvSBJ?c~edPN+XT*%xTw-2Saf6_gW@($-i!GC`K;eRgu=s%Zz@{eVo{{6ep-(UX42WVIw;M_neFqrlXXFMao zz(CSoV0(aJBiZ0YcWg8l>5aM5UQ^7W0}Kn8+*>ih+bD9jNQBZ3>etWGY@Z`cyRs42UmW) z59oFA@SRJC?_4?Z@Y?a)=l4$cx_m04Rfv!f>aexzh3M*liX2c=VrEX-Ehz@IePMl1 zP`|4?cxF%kk%|2DKy)PU?u%LaBj&tUll7{z9!)^U@hHhDtEkVfPnks~20}~jqSWo| z*t~)ZtMQ0vF*V;Srs+v=pMa_-c2$4#$-4jeKTAIS&&KaRhpqdTSh|7JSO(n7Oh(x0 zSU(rgiw-m>rxo`Y6-A@6Xi|5Z)!jC2w^Q5eHTDNBeF1B~&o=0B47r^{F8hGfHXl&1 zFQ99FY&|GgdL5Q|j6oypdn|@NtD)Cw`2R?I5AU{)H0{&>!S31fo$s5OoemDlq9{tD zC@PU+&N%}Fa{`G((+&6xqf)6-g;{sI53@ZNTV3i%189_L7j9!D;`io02@$>f^t!RyP#jf3y3)V zA`S#FKFh~vdfVu|97+$H+|4F!u7`oLlSyz;P!3Xyi;Ob2HfT|mie|W`wZ_IoJ2|aN zY>fh4sX$lATi^;5oKuyMAKSo1Y`^pqrt>0xg-%4(-GQI#%XQh0*19VQIJ(U zLW_io7m~385=O=(L8oF;Tg5aSXeOoO<#b{PlO!af1tgp>1U9{xtrs)ZJc_c72uxVO zr5@=qogVXjad!IZ4`1BA{P*SSKd#<5w{-2tm75nf?%V(n+E}}>cKhPe_3sugeRl1K zvzNa)dHLIuH_m^&Jpb*+%EhPmZa#m!_!Qd9c>D3j&4(K|AKYDdsrz^5A8gEryc$$K zT%Uh<_g08uF97!Jc_^R#zW}h`(hI!(#(@6&x1hT(dGJd8%{I`jH}1Y9%9~-3zoXyz z({nhKhkUe#X zopyh>bI{*6Iye;Y`8*y+U$<*~xO--_YqH-t(yi~Yh)oKbS;_A2Gy&Ija?}SjH{zQe z_RfxYr-ypS2Ri*dc8x+@Us;l$exfiV4yZIM5tfx$nt7rm^LSZCOmW(g^30gBjN?_s z>2`yBa%i`v3CE(P zT*kQ9b8-MQu+0oOW(Pc=|1P_hh(jI$Hf`sYu$^1??A#i@b8Ez|E&FzFIk;zARQUG8 z`*smgwO*$V{1Rq@4&eDt1{`z!?l~xezvuLD&zX_l56AjW4|h-bT_fG*K1+v7BXViP zF15(15?bY4@GVs*rm6)*4WFRqwF*h~NLW_JvHdaowjT$KjDQroe`oA|VE(q}Bu48c zw7CK3T+OqCz<$}!4!S-Z0(Hkn!!AItkA~dqE#dhycV7S8|m>h~YcpZjF(+Sj+wee~h5*(t*t zdCgpO2?t$fmy-PYwmvP#Zx#fsqFw{fYZ6ZO8a|!$etUZOlZoD$fOQ-otw%fJQ4QGI zN8PFctIXZO?$fo6INFCS61$M3p&^AW6}2hH7!^fE2HvL>dOLV}T8ov9bqgs}SWb5M z*6cl7V2AfMCLST>rEm}hk|urb!$JCN>oorMi1rBV&s2Ve=0Kg_( z+R0A+M5lhrW14YUW}Q|*uTxIjNvCbjVVm8Q69}SL2w>(Z+a|-HsQO8Ze%hj+v}ngn z>JbB!ZfX>u(WD$PClXrsF`*?f0i&;t7813$ zjS4W<#cb_iwz_E;7ZvRwH=A&EI#iXaxuOGAsV6qrSZD*eQPvDsVgSHk9cY-isfbgX zqop>o>Iz#5lNoizGE$?0jI;@E0rt2!Bxnq+YyphXTQxjtM;l4eMiSF7Y+OB;fFvWU=%`vg5hbEv0mGy;yoy5w zrI>=1vq-?k0dUG_cnJmk1;&DYcm$MQ$g;?JY9Y0QOVCRgT_$P2OEVd8etL4`hrfMz z>oRcs=YSTk{Qv-V7ks@%H7qAfOGd&F9F?My|lh|_3oW(8+Ya( z++TU}hJ#>9h_XYg{f8Fh-bNa0}zDa8+9r*L-za;4og=>cT1XR2K^yxd12?>$$ zaZ&Nd4=2YS&&fERpA}P(bv!@gNKWd}+_boo+=TL?cz9_%qAaPRED=@`TbP?vUYt<} z&ugm6s4h)K6sN#TV+-K}%n!#{;Lw`yqf012@(;8tC+PS>(ER1 z?3~o3!x4K9?B0| zV0(6K3ET1ho{$cN@1WqC{2t>(zkOx^7&yQ%7oZ>%!9U!0cGP=zv=0F6%t-fy*WT~c zbQz>hwGaZBM(j`tZNP!Gvw;KC2uWHYS50*M4uySiaQFLByFWO*XZz9c z9mgVe9^Sh>FZGyB!#g?T{BQ&U*qK4c**D(2xuqN5ocs3FXCDSnPjycXIX;~3{qEE8Z$Ao7 z`7|amMuacNz*D%ma*vV~&hW&v zfL#`_NT<4WLw0$$hTCrz4_ZY%N~VE{R#BR;MX8mEhb8E`PD$H4Z$!v<)(0)4*?qhy^jsVFC1lyjN&b z^Q6@Xr&6szOJovoe~k^d=<+(SZUC z0}iZAjxOWX=c&o{A{3%IKZ#JDEymSITkG{~JanF?gkcfWtYVf`%yB7%CIL%E#B`7` za#Ay=xt7&jC8ME*#3m^fYmu}8za^#O2sNENn`6dAkK*v|CX6+@l+ z4=4OzemHgc`!80m|FkxLe)+~vtG6%S+gP}F|JJ>YoA&^gRFp9l8q z`d3Tyf4_O-+ohZ5H zF9`H&dcE4u?|1m(GrV#AztpRcesLwF-+J9!ufMqX^DDaxe*OoPR`vPwcfw<1!eWjd zNH~5ZJ^n~mT6A{Cp{%qcx#@?qQvk!_N^%n5#U}u_V8zGF3Xc@#9W5w`Eh~#JFN=ql z#1!Qo%g;WZmlu_uelRWNXiiR287vKs$gBW@r<4`M7iJ|FWoDG+=a!dbmlb3d=Kukc z)u8G*p{(;RyWZDj>G!&PeV*Rl9=E64WH)tnJ7y;azxZhGt4~k=Ht!X#pds0IqA!J-rnb(_b1HlPW=eavg^ck4XHcDq^t z0Oq&Jhu!M2Zo@>cX$k<$Z=d!%r~93g0moFp0j$-8&pOy;?y`a}prqv3W6=?NBExoO zCB^IHqKSZKw%<7wuucRllRoEAufuLqwKP;k?F-uj*{^MTc0lwBG3c*R|(8= zu1?I*h$w0SRn8%>@%5Djsj-o}0lgx3eGs{8>!Gk6(c!xfhwqLK-`;@8@3VKz1sxxb z0FUK31L!pZogaO6$aQAO71}o)(ufD33vllAsQYhc2QK~n%*yo?D@Eu07ct8Xj< zdR@M=aA9r!$MfHR6dbT9l?)yi!=N@uMMSfn=dp=hCZ3eth_B7YAksQm&AmE)&@AoM zaC>z;uaRdEU;-B5=|ShGlYL*G9GnZ-N4s?6J%C}_fK?i_%f{SluR-7ecHAru7`SdR z!Nf$hW2!3>4q*#3bTph_CHAWLN@Bf{-ejgXa%*ynqPFMk{h;h{cwNF_a()^QQ7COh zXj&WHT#{ebHmDMgs3j1%aq6e-`Wc5A@ChwcdcM*0)w^ zaTR82y`7CTQfgILcn79Tjw)5+5Ym=14Y@&0ZlYG?p$d|CNVo)3qo$x7LK*;=Rm!kQ z*(L$g(a!gDh*dPaf{a#BPy*nb8sL1$gkkutjYc6u(?$_NX@6U)3(^6@7|0qmn<^$^ z$h8Py!m6@TAQM(m%tY6*Fb&YuAB(79liL}1Kqd|z$;LJaLPt46K%%uus5k|S$i+9( zQ8lDS7_Fs})7l6nttUaJJOk^-Ct(D@lTmPD;JxrDK`1&A(2I?2;^Ld7OuSjaHObj3 zE>X=RI#t|&y<@^>`}E}Scb}b}zwiy9*UJ3)J4=`E-&=U_@b-iIx9{J*`QXmN!}a-l zD_6l^+QsW%tuOv`Z}rN`;)Pq+e^{FT;m)n|t2fUtFPvMN|7r31kHElf>h{&|LAi7t z+8@0#|I;e?tA6|Z+9CkhrRAloODps18-QWZ-m)kEc=iOcVsEiZuMN#_8MQ+@(Ozxv z&t|W*fA!hl>gsp&@{xb05qa%Xh4k8*(|`9Zujx&IL3-`a_=vs+n=aVr8VYpzdfh$Uo?dUSqtj`%nFHSL4^NJM`T6XZpPl^b)48uc zoBR5&C%^b;>XVbhC&zu`1Mbm)b0XL^G0-*QbN9NfA}*_Z{!W`t%BSO)#1;y!8P`-x!!gQBhq7PW6r`0j0?DMW~20ARq1g>C;} z=hpWk_l31F2*Z7jseb2dza5|jqE{$$6<`?P)?CnYYN%_b-#rLEq)FDPm$! z;?Rn$N`XA-zLDleLlQ*t(Legd-6<4($41|4yK72Y2s06t+7mY*%XRVJfb^ z%PN~2bbd4n`gfilah(pa>hzHNbSPxw478Kld2-M(J7k+4vdoT}Pfy#v{A=Lqh10jM zeYQOR_453;>x<{1U0gTM-Mswu?6j{6$j=khA@C(EiC-_s3&B(_S-hV8b2_02pv!!>*1&TYI0L z>oW-l%;H`Z!@_9+0K*ohmM28>Q8k`+PQPC25Ry9x)kb=Q5?9`ue7vvS%F=%qX5G0l05Qw}Yl*Nnq3 zWdj$DGjSwWzuJ%>o{bnf zLJmwiq?Qh-M1xAvV28jb;dsR?z^y?UFWAlvNH_psUI<{!UN#h1-^s*#n7A$$4gd_8 zFkr<1#%#oT6TV7|ftx8cRz{tJ)1)I-YVZg(u3ST?>S(RhkQ(%~76G!nwJ4ocRU|>z z0GYWYyOd#VX99p(L@cwA?UDyks?17iEXihK>Z#4uZFnTKg2lkg*(3oO-9|ui@km}P zvJD5yW*MCT92FFNNJi6Is_3Xn3K9WK7#r8fC$vZ?IAFp=R9qVo#UrD+gce}kKua+V z2Rdh<>p(LgwUBO-v6Y-wlbC9j(t8ZzfTQEoSkG5~nY{4#Pi|fLZgu{`o#m_d0KgtC zLbHe)H}BtFfaVfcudd$wae4mlfL;$bZf&g3uPk0#UVt1ML@5Yh=a#O&2xa^wrV-rw z34BGn1I;d6zPogJefi4z+O@lPZ?3G}Tw1xkzH#T#lLydwWB+&xEs4D`-M2Z7_WHQ| zMzc5d+Gzgr$5;D#%d*q21tH|;e@@N))~RElpBJ!s zm{L-ZT3Q5@Qk<6xEK+`UVHJWY;_6HqhuzTM=a?Mqoe1{%dmP=sO!&;bKD(#W-qYin zn+bmR^@r#F{>k@Wo&DkKvp;XQl$-P+KU0qHSpG~iX zm6YY>z;d$Sx!Lf%9C&_CSx#C>W^!RhLUC3?bxCGLQ3eIyH0<*L=9!d2KE0LN+Kg?g zMpnaUIHcR4811!<_E{#p)=8gztk3Fq0tcpW=-Yd(9fK}4a07r|lYQpNJ_`WYw9h`{ z2eJa5ff4{L*kyJa6(mek?4kV!cJGJ`+fkUCZql@mdL6*AO$Myf1HiGl2fFMgt*jDW zd@v$x-<};2VLSHj-m!OA$cF)#ZQrtM>-$HeBLy7lh}Q{)Yz#Q(1_4nzPmOe*90pwJ z-1K2UGyR@Hk1+%=u}3fI)VF(dQm0mIR`B)hT#baS5;Hpl3<;Y=M%5Ii$3^eizJJ%2 zy*sw-+qn(!b^osI@kgS7VQr}`?Xbbpyr5;+uMhMo z`g+7&9*)gMbT~)@1ENzW&7XhP`|a1mmoI*_wD8T^tsgG@_=(e_sY1d^YD>$J6$MrK zc@;TnMaj9a%m!=~tUL#gs+6$NdI`yA7F#4_y#UvxZPW5FRw;3!TYYN4HW#o$v$frZ zkxuQ9OC5A{4A|uZ_Vxj*xZlJNTEspr-^Rz9*i9O0Lv4C2rX)+xA$qk!pN3~(H0vmJ zMp|8aOKDZyzTAi{h~xX4l8=!J)7#1ml+D#PdMh+@s1yKz0dkCL+sAaWaf5u?s-AIZ zXI=W!UFOr>W?;Xj9a_LM0I*4`cG9AmwCEud7FrAg-fMFWY{m(hut|p*0Blo$VIliv zfS_g8{6a7FSSampsMG@cHK36IbOoVUL;?6PK?Oh1-sYEb0J;X-feGV4Tbl(epWsD6 zhmS|^;ZnM}q;3x0O@ofX1ORi9kZv;4hOgD3%FV=TEwRE(tFzJ@)U5~uxkf{%RNyML z=jiDKdJin2HsUF-;}8Xn0{8x=~8Tv6|}$RWNdGd23}!3!)HTUk(5UNE2GY zf?_e+fzyJ{Qo*%hn%L-uHf)oKgl?y`0*on`M0!g#p&rIS*RZe+w3ZqUrUBe)r{U!s z3LnZ2+`>fHi%DnzHXaEDxJ5x$ftJA1i5aaL0aeC8$>}X75!Iz^^V(#S{q|2z1;6{^ z)b;aU0r0FXUcR@n@BoS=gaCH$?gGHo-IXgV3*WCSeD`4O`jdxi4<4-EyL)S6ZGPp} z#YKRwg$pY;FRp;{#`)#z=RgUZ9MI~*g*&$ay)K35b!mO&3h3(L!?lNx?yj$|EiK(% zTVH zMejNqz5Q_1&f|yoCd7p&BDVR7-{2}yes62oF*wjVhXb|T?Ga#~bc z+TpCM69vTyC1vRqfML}c|#Nxu#veFEANk&;gT5(=A3aQYmU2dDd*E2B) z?dzHdc6)m)9*@lHl?)8%28TL_M}4!iLx2C<$Jfq(bLHF@*DidybmJ%RMd`++Z~p$b zQ>UkVWBvBgfMeL_2=+R<97Z;cR9;dD0QQ1jdAW!JAW+XJ&PplCPOdG@sVU9jGVmk5 z&S9@h)6QX&F!+}GrfOI%ypV?71Q^7v$xyibgwNS+Qdl~Kc8wSUm`6L_W0>rN05;_X z4$LtVa8CPy34`p{Y`{LzXAO2*90modq3ZB~JrO&%?BBh$I4{Gjm5uqFQ=9n}2VA3G zr_W>6bco7|b0YWd-oIz(zCFA4g}u<;ojW3S?g-nqB|hdzyMQ&?=R5^%I0F=MpBm~s zIRqHCnMAk?S^yjEof&|3F?Snf9$*voQUEZQPGVP!O&tQg3<8)^%v6e)QVxmOT$!I5 z6CJ*7-;S*jJGKFZZ`*Qs|DOEx#Q4Me8Y*%F9`)%_PY7V0r$@ny+^2?JCx;-{H5IUW zT^$w!uU&>`^J<9Hf>v^79r8#CVpn~0G=-VL;1Uh z6P8ojh$yZrOD)YzEKJR>DJrQiD}?2;7&sY^=+tt%3_>xzv4e-vidseF>K;9J#%q}I z8OOWzfLov(>C^-ra<54ou!w^;$)HUfw1|RcagUOzr8k(^NI^3UU7X3s)VbwcpHA2* zrI?saRyI;ct`;;D6hv+-I=H(w?l3MZkzSh9-dJIyVmrAMuY}bv=Z$EjKx2A2@Lm(f zjv0q;&TTl^X*%0uopl?goLT^|Nt+hfuMoY|6K2hn%`ojSO*_m}R@0QtJZ;-dC}oAva^>T?_Y+NT3q*s+1H)Bl$&FTatB#5iA`!d>xS*+m4KRkSwTtO)2w)5YpJElW zjC`6+#58b7a&ohRf|C-PnN8*37b_*Th=2*BqSaiAh=P$ZiNrcYZC(bYu7Znc65?Bc z10&VINwsinRdF+-5HN<^Q~^vFlv9d=0RU?U6%x7))5t>AwP72%xTes07*)<70$YZw zE~OwV8R%LTww{Kpf=n0{r{vS+JQ|0HVqh9XL^QXxiHWXj!#4rQ0YEY^^}y9B+lZ<* zoPby-XSAq!tu8HZq|bO}s`tx}feHKTt;^rtxqTV9E+`KE-c0~7fSir>8+TVO1N*gh z^ZN&PZajvzB(FbuwEpDbo%`#Hcb0D~-@dYZ^CB=|i`RczzHxqO{`~D5=N9MBF9M5p z^TN`?xs_WNR+lcW0f24Xf;KWgd-CYfqe}<+LUt1D; zZRCY6h8jTL=#^Abo0S*g68}WS>s`It&p)*Zr#G*^(d^C7vx$&bBTY81L-k)j{_87m z_hFAiJG!8c{mQ#u)2je`S$Xq=y;gs%!8`FVSXxz0Tv=IUI^fgsgDJ-%P8`}5ec*%r zd*6%LyFL2Qp4iy1*to6H$9En(5gwNaEZ5eq)m(oCns&AW4$9IzK=hgJNNxpH!gna!wfU&^D8 z`nw0Zt#Sd4ibZ1UYnm!xb?{;~0o`lSjP=^bz4kG`ZL;4z>T`5kROSx8LoMvHDT1Ba z(NG?v$zC%wh3K4#DvLb;;` zyUq;uoQC2LyMg@zsi{L~RPeP@j#|P} ziW!nN5*~@jNj`ieVq4_yt^0Rwjo7v&YWL2Z#1pA82aoUHLT)J?>e8GY>HKK4>+Fc< z%rF3_BsDYan#@w5-pOLk|LZjVX-KN-dE=fbup<^C}T zzPkn$#}-s|^-9c69)?<3){vK7o>^R5SdFU4ttv<<%Pwv}nxSWtN!~ zY)KndDJDrdC@HgPz#$*&R8RD1$GUVQ9?dYcDhZlJL94`X;5kG% zEwxrlZNL?$w!*SC%+@}QsJDaT7T|&RGSTZ)1O%ZtAv5BA*wKAWDKV6y3`s+|k<{X3 z5!@U?pNQTsM|JHJM){Pv0|?l!lU=5epw+Kwr)~-Yn0m^tnYL+W?Aj@-X37dm z{iM|}Wivs>D>PfUNvs!@&>Gk#z%~Vl1*HQvLiQ{4Bg8WRuo0bnSS21-ibgs_!wSKs z24y@zuOSI6#YM%~35{lSl^I=O zz#yF9+W@K*7%wBOUPq|X6RSJ0a1E)}!onKqSP80%S69YwEN`nT6Sg2U4797A>5#GX ze3Df{w~85NKHbV^sOZ=Z3QkGI@zK@n#tJd6NkT%2$tb;$A)#P}Bov@dV_`M`7_+&W zi>qg~)V4w~^p$8tVSP~+rmBdEsRbqsI%5SmFeX98qwxsMY&4X}1C(5B6TcNHCS##s z4KkLAsUtSP@pYxNmTGEq4XL4mOK6s{Nm>a<$iOpjjVyEnARnW-iidAx1Y zlQVH@ZmWWY5)f)7bfjKN@j2wPL$1%xj(q#+-1YNc1N_`wyZ!+3Ui0@h0KFFOtGpI^9k z?&kILH*cJuzxKn8E8pF`{=?GE^D9f2*4GyvKfV{rocNE2j~=bAtS&Ese-`houibxi z|IzcuPkw&7Ik>+ya_ z)Z)vQZ+7^v7`~V8^LG#h&8EJ2@ojqf>`T29Q&5@LQ zfAyy?KRq)(J<{9XYquGd6ap3o09KR_d>Ep%2$bcZUYG|j%q_}EgB4^{m*v)16f4Bs z@%}!a+azftlTnSh`nu+7L<6FPN5=V`#z~)h!tWdpxW@eM;Xa4YZE)(PPOS(4Y_Ll^ z+H08TGfjCdK$Cte@FhT#0qX<+p5H#)YwdFzbt*||QD(&MEqk`V2UK4L>v0$+`rR`F zPG|%2VDCi06X>!kB%I9D6MMt9?cKdSeD}7!VcR42>&Oqsddru8@htzX+%y)L&KjCxtTGUQ$JCIv1cN^q(t<<6t0S9JK@U#-P zM#55w8SPvW8C3zxKAsi3FZ)DPUgD9$WD@j3BW@v=yi9YRzLvAQse#m}0 zXqxrwX1v-{{pQ&LtJT0mLM!!|nXrU}{6kU6+hekKB^2(8O8)nx!mTHA{~VR_zvJ@X zi^=-)vD80imh4S0-WQjW?P#ZJ*xfPcka|sReR@uxt}S2`1WZD|Su|i3^{8lC zdaaIDC&iSb3zE3a6&^XKPsQtMr`g+3Mrw_TTC2pB*QXxJIk2rR`6#(KL)=iV!qn*r zNHZ1VX(RcitRbZk3fwTXPn$btt?F5udd98+=mMJa=;t~OGj1KA*MwE|l3`O8^`u!f zWzkNUwGhDUA%KN&^~;>04pg8>dhnweN+LCG(odN5V+JUzFVHyfVMfh}UOB8&Kmb!q zMpWX_4$-I*Qc%Vlkn;v*{6Q&iNX7@~3JN(u{QzKmhED)Q0|Ef+<>9;Au-$FAE+*DR zXtH6dE$9ke3*1g^(BUh^O@&5koq<#X0H(qrbi_I<3#+HLbl~cRjpZU_1-}u_Z-RA@ z8mvNcX9vg8&NT2zW)amWpxLAx1Ba|(khDyq9I&dTR)DH+CqYN9==k&wCW(th0_Ozi zg+mlm8xZ9BQZBZ(wX%@jR7tFX)#Yb26lF1*YXGSD#1<(7FJ%%G95SDTVxjBUmiLDM=dWM-;p&BN=P!M?2z=PmrMq`;K7O(R@e3N| zpRcSe-MYQ7yu7%ybZcd8@!rF`Po6yjJ`6hh`F8@)LuVpC{Kv1tH#YI}ddq)Jk6$is z{R=(31jg%N3q_Q?*6e@$kv8>e53l}Vf2+-J>D6a9_0Ey(%-F)b6Gi!lGEyQEk3}XQ zjZTd@lzd{_!2^HTwfo&Y;qQm<{b1j|9fzWJMMrNraNtjS_WUs{?A^V4w?^*Yd+cyz zd~9q^RyqupQc@C^lM|Db6<1J@TviHP*NNQR<5?N8IT`VJ(9%~%QEqB(YG!U`8Sz4>+HmEk5Mbdp&Lt!^MT4r3Sgy$<)uYsMR}$9*|37# z!klz?QFeV*NkcVEB@>MW`nqje5sOI2p$Nz(bX`?bWto^s4!BGcP$pmZWT100&^6NQ z@H_RLMwv?|?z71Oz{b6%Nw0a6ULjdRvdsjXP#&osTc6XQmI(@T(jvmP?%nx8#I7yq zrW&8eG8u4B2OxmW4fT%sJ$|=I%poVoMThTtf6vY>yLW91+qo@#7XaAyy}Ncq?B20w z`v-{P{BEm$F4%o$2s(@T3~*q>F93FWxbM_Z->G3Q5csq`HjPs+v+Jb5d$|m9n?`I= z@+}>FgN&n-GPPowUP?2y(>vHLywhYpe&ejqFzq$W1#B}xhuf|oGEfyQ@M17Nl_kVvMa1TWCl&2Il={cG{B1|F z{v4V5AF=r#L}&bAU;O{x8~eW^V*bzG*#9@;#GfM*w?(If#b+N$FHLU1SEI>I4K0;Y zd7D-zRw?;XAx*?3sU%FjoMlnA^}AF-k9xvu9P3gIxnv`5*?>*Zt7CYz%w9DUFwAQd z1nts3J=-E|QBf*%v|2v0m{yaoW8-`}QMZEa5fOCcN&~sVNU4@KmzKxug~uEq6sB{l zih%uUZ>rGYo9v9%Za#HD&KXgQ#`UsElYGVk0H!?YP@i(?W?kA@j}Fjl)}x89)e zU`oKSDZns`avI2_oG^5Z8B~Bzlc7irfU8Y_Z5pi2%9v3%X*K|KfpXGp95?F64Z1Od zX3U@-HK<4QszEiN7j(MHuu3}G0WoX@08GJ$j#-fl2NeKj{K0nM#(0BbZod#XFos{q z^Z^qlpn8RrK0c|3hX+2)16&lL(T1)vwZQc)a1){0&S(-f>0WN7a&>DyhgC z4!*g)jiM5=frr!?p#(!h<{4?=GKPUHInS;*U=sEIfU(_UO^w zr%!>Adi3ns!zYjLKMpbM{`&Ij?fKg`E-hU9>Bf~GZ(Ke%f91mbmGjpw{;+WQ`=$AF zYb#go-Mjtd>3smOr_X+V{*Rv*Zr-?lj+BKLk4NTlxA^+=DgnEL9L+Hwhp>i3w&e6BICYpTC%Zm@5p z$KGvHI<-=_LE^P{40Y)My(arClV0GrY?C4H1(`4aFu!Bc=NRg?_Bo7di7-DaEppGU z{kykC@7qPeH3vKZb*^!*WxC%v1;tD{y-ovnGTi*@e_HM+s zclU39ci;9uhi~~~9jvg=rk@UUof_yqGYA+4<-b22V%V7x-{}$GnNdFkuz<(wGT8O) zR;|RYYj;3Xh*F1!4@t#yD7h9n(>R z^qSjde8w|D=UK>p0f3q3{N}lS+uWdIX2==rGwF3A1{>FehL<8zlCmOBr0+^E+82|# zCAB2%Ncx`-rT!r{Z);TQANMBwTlk583p@6Ib|3loy)o|{h}*U=c3X7P-lHk|^WiBC z%`h&HsMkxhTCqq>=krKnK1ruw8&z%Hmi7UUdZ<$kjNq^b(5rpWA@b^3ol3G-%kb*h zJz9>>#2>JUy3`akv!a7krX-g0kR<|al~cyARz+sn;V@)c z46t8JSPs9cSlL`{A*0=FQm>FcsB9b63MUM*DXU`IuAH^2PP??H-2h;Ce6BOy9v;1#u3^F7SbltAb=UPfL_3XfpW@X z1Pq(9n4skJz>^uZBRch{RyC$oj_IIsL@gUqLg6IPNh?q(0qR#k$2S8v7L>FNK;ekc znPEP_Fd@SyqV@1dJzTtpiFVOi+>~Y~q0Wq{Fk>tA*a{~DsYJuL)tU0aDD16|qW1sL_%dq^JrJ1h8URYpsgbWEYd&3Z_*;)$<7kK3U&JvIyw9HkyV_R!}fX zGERtU;-ecR#1=Wd6#z_1!2$CmrQ%qqI#x>!8lFd}E@rh7Y` z;M-CIwA@8PNoY7Zg8)1ey$N`!sy2KR8`s#@+QbU&591S%JUmoyttute!^w>Zd>xF? zfZ$WHx^}LVLk0k2U>hloRRD^_8h9I~k=as9sDZOkwR}PY4__-|viHYjF3n%NeBYa%ABm3fy9Y}}` zk3GKg=+WIV$0HKrBN7wBVq>-*jQr1?JKxW@W=m${TBQE8ykLP4#$uR#|C$R(c!&SXOFsc1mVWT4rW) zNl9LZTs#=)ni}q#8xNcu56%qxNBpjy&UQ~{v%{Hcv`1UrN4tIb7FRXDy`03UAhQ}c zZ3LsXeKgp0c5dkNPiAjk`*9OsOV@w8^wSreE<em;{h*khmcJE6VLe#f}qIn-zGv1#e8jWGvzg>U)y{X70MYUdx1hW$A<;@!A? z?>9d@4?ww(={&JLJQ_d^R{r-vPLqt3|zTd-T_u}V!^rlK87A;FrOGEvyP z^4b$cl}C=Jy?ZS6-I%Ox`xE{ccH%#F9Q(IzhyQ&~?E6tE;m32Lk7pe^ns(q=^1j^S zI20IIIasZVuh9tjB8o&x*Q>cMv)E&i^x5TpyL_ZeInpT$Iz>ZHalq8pt)lj*=>Y?~ zPsi%gumHe(rZ%gjMMf@BlFNiG1-zD0H4E!e@w$~Jkp3 zGh^_139Pa#aZRZfQ}3W+dw5izlr^a0j_O2{X8E*3HRDvB^k`4I)N?NNNw<2=4FIN_ zacTj*W`HcJX_I2o*ggRS9Ggxysh3Y0l~We|s{q@yV$-2a9GeSZA@60Fv>2zrtq{Wi zz(TgnFsj##Y19)s?YK?@NH(I8ht3HTgK|&_=p`LcN&-reU(OH6dBIRv2eheK!06-C zy+VdpNbBYTfRQ{*teb}M&`~aOqYYne!B!e^m3CT#iCoLA%HUO`X)tgjxmHK2F)bT5Et06E#-|n-OAcg`R_U%IP*K1TbA2(JY`@gbY28u4EBA=r|<>D<`1a zT53haW*H5uX(P8&u>ecJT8Staz!FS(5vnYYhi$@FmSM{CTPyQh$}({k`M8Qga&0-I zsai;cc0kMNcu-+Mx#y}F=vp?mfrDuPMhcWXTq7G@$HyTl^@!H05_)qbxv9Lhu8fYY zR|%OiE|rFFVbX?p%2OZ1LIiyH7&LrfeRe z^5p4*M~^lhJXn8t|IUMZYj^J~uPiUzx_Nc}#+4h_FVA1UcYbH`j~{IU42-s$$9nzNp#0V$ z`>jyt}xP_}f~()@*Z)?S)>x;yJCOY~8;z{OJDh zW08BJ_wS0@7Z!bB?~%y;M-Lu4arj8$(c>wx1@KY=gUl2#k$4P+!{LafrPZ~G+1bfC zSxK3xzd4%O;t(} zE7)zF=QVt;E%s|&nzXv=u?sEluY;JQ08B?7a6P|o% zd*1PISmM6w^aD*f2V3$Ep$ZPS!sBTTxn>DvtjBaJ*m*jXI|@2Eb1ZOn)VB#RfUYxR zf!X1{VXw2-p|xp6W{uFMm)W&4tCDY4FdZE%x2nx5V`>FtHJ8}I!qAW<6EL_u2i}X>`-kF`17<1pWWOCsbqD}9XgVDHtk zC6=Wom8Hh!#-M;s^HtC>CG1l2Jmh{7v!UQfnN70Bd6Pi8DR-cd-lyQbtf>DEH64)Cw-+ zl(T?jZtWa!U`}YoYuc)sv8tfuF{^6Qq?j_vr;M^m1HhPk(gf~8pqjAgrfe@3y*3%P zDRAp0&!(*ZToT(v+L%#0s#A|^)c{~)5Wp0J8rdMU9Zfm}?3Y{=R6uDG104c?JGWoP z9cYIR4D$+@eSBKThi$@)+Qo*pg}E3g7p2KXu6Gb>t%MpYrOv@Y>*QgH-2~(p2GkD_iP2?$6&5h<5ryPB7zbT1A!8*Jtdx!i#*5S1L`EW@Xbx08z*`%tp5NNQM%A!epdRqmMZ|^@ zS_^{KQo+F1N}0qC9-U2UrZm?O>JYT18alFuPz@v1mIF~6E2#~Y)J6m!4*-U4!#7e} z%2oWdo&K&VPL6yP1`1A3b|; z@#mjcpFO`HI_2^)G>iBI0@%X`>kk0H?%%n8Zw*UqmUefNx#a}=2tGB$O z*8urnE7bm-!%0VvrX1TBdvI%1#MVR6+m9X(i;ay)OgxaB9ub$g?O^2ldv`=?)N~J(5htBq z|5t6(f7Um@TaVs~B1DR$Ro$H(1AVqmo7$+Ax@@`+Pfwly>Fdkq{&w@~*EcSGH8J9C zs;|t+N-fAuFDb}^73aW9vdc@eD@t-Iin1yTGmwY^R80x373uGGO%3~97L|}oVUV$u zRty$dOF-2cRKF}HLb9CBuZFGEga%{@+gE@&uTdIm}TIqz}GaKxg9q0y{^m%|G13rz| zQjwn=Rg@4}k#+=`bCggVLr0uoRmL$Y;+S>G5>jc8zHQoXKRegxx8J{tSN1iP>407*43o=+?K*1w|7!Z|sV=9T&tO#-BjS#y?LC$e zb)q0X36>dOmK9f;nOK&YT%3kzEM;*p0APG>i(E!&Mz|cs{cz=#-6h zsm40xBTn&%UDR*jba#+^T6%W}+0%~iR#Lq>mfyhXQBjR;O$KJ2ic~3WMQB+Vw}RtQ za^3ArMQddTw$w~5*S3~O8uA(v4mKr3QS##%#YuvSJT1D~MaO#i)P4zLP{A3|3dYRs zfL_3F&AAPb-*P|zn{#O(fQ2RxAqNHk1{|10Ic0tcFxjM0F=0|o0YpKEe(8Z;Mc1Z) z1KVWSCfESU0JmNO3|cKS>PB^%VU21?-7%<^4{2mjOutGptPl^%g#&WIKnJ9tLf{AX zO9G|jfYQ~6@=N(d48NG!%OiJl@SQBIi_zkuAwAS4C#l{_thJJAos1?Up{fm*!mrF! z;}Hg0y^&rA&?N(ar&jA|4H{~rl33f0tq`N&DpH-9i*raQW)aE2Cs`y+lZa^+GSw`y zhD}nlhzbf;f^X(x>V$-5C5xbGqln2EKHwP*$41w;R+Ule5X72NR!bGF5dnxrsfS|` z1;Bv;Lq@5GF`KLSxJEGvC1c`wq-F-Tj)kq`;+ue%0t8|;*Ron_n8+%6V>PR}hSr2= z1q`c)QJX7>jpaNNN-bjX>8&($9lElFSdYM0!|;`*qJ&ykPN{`c8_L<}S}wLx zOvNxTRqWO(hngMeRL)KIT)y!2>e9`%Z`W zlipjG=AJ&h@cifH=l_7VAw$OL$%BUgV2|$Jhv;?Z;r;c;4>z7Xz6Ypw_wMT2>f*}M z!s5ckWdN}ItI%O!kOTYq(UZqlu3lJKxdqKb{`~Bhl!HG%e*Sdh(StkpHXc5G{Nx`m z5<&eB0k)Z){f(!ffQx^5sqEE*NZx$QYx=D=Z@u`N+Qe9>{sO~qJowrUf!BL@!y0bt z-s2~mr>?vc6?gbZ^3nY#4sMHzcz6H7oyTLtnS^$auF0;q3>T(+<$AX`IeD>3irazqPpPBTU4egci zg8a-BL{UyXyZ~KYOlT~lqY!Pa)dFIzh)^S=H_2Hjy_7W`=$antb6Hgq0fWON(#iN% zbR)I3$)c7F_c;K-fW`x!5wA1gF?CzjPQ9$lECT===`~IGY?A@UbiZ>t6tw{WHW~0h zj|98M0-j)xz1yy5kuVwYN77;r7Ny1z8W3*qZQb8F1zsBH0vH38p&p0BAa8Ai7bHd% z#z&SU97H4^{Qp>c@AtNjG~cuThW%xC=I+egJ3Y5&+|%7|yDi(wq7risU;-m@CUOQz z5QzYYoCy#B6EZ+#A}~`qN6SHyE$4QWEz64DI*?i|OS13W-F=?#Q%~`r&cQKl*N5M? z-g;}Ga?4=lUl7%=4ph9{SNSJWcbQwu&xPIV=@4MmdeRT<*G4M5o(!)hg3F1(YAUpm zivfUvcb;|Vrc6qgdE8}IO&JxA36XW2W0EnD!iuzoo>u^!PJTF@R@&>W5_55>j~ z$0diNlC_Y0C8Rl347`8t@MrIx`TT=3A6`3l`P|ac!wG-LEK{;cY+Ng%v#J+X*4|VG zYpCgibz*zE24DgKNx(zPBt*Gyc+vIX~ec1{zYk|n2P3--~l ziSO63fZTG*^f=9@Wd{x53U@|Dw~ydVT%?vcU=orXQZ}HM54cYWW1QHbqqJL@o#U8# zVRr>`@2iB0ZG2dTu)RWwXmcB0sw%e{MHk0J(km& z2UqOS0$A1#MHitMqs^Q$8oSxN&KCWS1QT3iB{G@=rRpaa83pi`S6 zfC++fo?pW9jWOp&s52t6S3q=gv0e_wNgK2c^+60{AtyNq6{cylx5_~8werz`UM9|< zmfmAxLjW_fF@Rp<3stP~G&w4l1Gp2XA8G`?XTOj87L*u~bAC zp{JGD3uB|<09w>O7=54}JRkJAxIQkvPt3qcSXdqjZj{plo{3b%b@}|!FF*U}tIt2V z_V)S9XEv^0JpAdkQ(wG)@`DRY=hl6Pv&Iuk=1;HX0DT@jy!r6qy$26~QM!Hm&i8;} zw{G9O1$@{KHvzitmjaA;??G()?&gxFU@xBKUv%;RwVpueSDyLwn5Tz5f92U5 z{|ij|+4x@_4A2D~uyXJI;{lJxfM4U^c>kL}y!pzjJ70f!`<8$G^B?~C;Hl1)D{%+4KS5 zOmZQ#luxY{v#Evf-0YlOp>FH!s;I55sjCP4YHp}*ZLDr-sN^xp5#LlX6X*A^wTa-fA`=;~`5S?#o1`6Croh@0xL2 z(+iw0|qh!u#xYJsp->xde z=78aI+qLpB05Fz-%V5(j-HVs=A?u3a#TQt4{NClHeS)E1l(GWnbslWWm*Xt zihk{CShpI}0gP>i##aO4wUBt#FV1>K(ld$^MgNBvk9=|U+}EF6`r?y|A74N7(fg+^ zUR>Ec6c0x|oKbc|f9Jt&SR=BxkBo*5!n%5!#KS|w0<=_u=kf*^^d3B_X_SL>X}LlB zSkNY3aLJNhMa(r8ca8=1+@OII)G=q3)EO1krvW761P#1dHDgLfc8Uom4$8#CO-fiU zIWJ(62Tf9ofS|y)sR+#mYTM{QB@4C}x&JkC%`QPl^;l269@jI)A^60MkenM<@)mT% zDVsd!(&Rn*q8GvyP{Cy=xFCQ5`&GJP&QBVOlZGXyp#(5pVN#oS=(5&{v>B@To3?7w zRxMQX*I`_oGy#U?oR6c3e}~zlwXsQS!D)x=S_xecy&O=QA^=#*Vu+i72~#KZO5nf% zz!DmH9IA^r8U+BG7z+ajrWy;4i^JoO1Bqd|Fsy(8HZNw(3=hqWQ0GR50K>d|yqk@g zq#a5e$j!b1XvL1Lo~43wPEEyZ_D&<21Mq#gqo>sE4I zO1@3Xos^5rqa3T4ZyDiexI-op+cLr)BO^yigA=?Vi-fNgG9@(JFbS#T3<+?9%)xFz zEHb>2iD>6xdjMDP9rZxW!A{Tz4h%T65eiDiBxr^iBQz`v+xMuZ1_hWZL?;K`&Bpv8Z#GNMxsWBscx*HG@s0Ps`1GS|?_M}_Z0YTD8()8T;hPW7T)S92vmQ9I zXkAZfPHeb7`SAFUcfJ6VWXa^*zIX43A0dX_ELpMd?%eqveDvtBEiifBy?^WbTVH;K087{wrt!tW?qvzv3F7zZU@P)}6aR&n%2VBTIvS^FRLd(r;h>!*5FohyVT0 zFaKf3u2;(T{O7J6FPA~vli#eW*jc%+yl!tlq6tTA>*}p&Xxd%Zu)DfuXJyrn_O^1l zlpP677t--UCbpW3ujMnTM09@MCzYw%+S{vZDq6r_;q6WMo>nTNor&$zEBJ|kyBH5G zB_e=c#bhWQo(ax6>{bbr)!GgJV|UL>2*k@2;_E!d4k>@PR#rdb;-n%D=m4+<-(qY& zI`5p22`vW2a(>~&;lg6l$7hr3tM|9m9USaxreS;e6yz8kJORUn=Q$6|etx-(UW@z48r{oSX8y zmSa9BgqWP)Oot9x%FymHltVON7xtOi9R>zWLu(tSwmM|Q;i#B!UM(-zi@l{KZMg(s!QivAmy4}JOmxi6uj25)`wAyggp z{Hg4zW64YB*Ai(zj!mxZ?ZB|9$RTuNM+Ke$AK~K-8V;QXtFPYzYuiI3cAAuIw^2A} z76mLL5$D)~TN-hSgBIben&#Iry$YgRMgaYJ4Qn1^7|T0OaY~3&60(VlvX2b8q#VyU zKWvuIssu_3T#j!U$2V$+T17nv*s#6$$~S2BWy76yYGj9z*ze{N=f@a+IVYqP#C4-d zi)_)QUUW_5rnE~QL(Zv#0)`O7AbKqWPX++yG!*Q*fJagzX zb{$aK21sUlw86RPa^!&zgAzANNgNIcVBks~Dq3Gk@hI&KD*?<1Ojy#OPU@5?y&|ra z#dUJfM%81{2}u|LOe+gdNJHaeL4`P?cmyy}NY3|-vFF81$b^kD=7wpr!;~2zX<9(^ zuyGD5%FaN#1Xw#~P)%sp(R%E|c*ii#A;LOG@W6^0StvaNtswMBuw6O^+Q7!hN$?3K z!Y-jtjq@j^Ji8c@Wtcl98#W1;20p_s<|?SDQGBn8O@wMnaVc^pVVE?iU=hU>G^4+h z*4u_}uOW9eFb7~WJReXK)y2eg2`GIdbi}v-XV>!m)7s)<@ZGo1e(~9dAANA;^s%K&Cs)7y z=+ajoo_+V?>iMJTvzy_gOP)i^p0_Wpe)-k=-{1cF&Yf=`J_NXeoYoJwzrB6?=8v~; z-udyn2lpQX>=DBN&wjjn^UnS60Ko1+sQT&KZ@;>B{heDsd{au5e{dh@0rbI>85ugt zq|^aC`=t=#^LjdN_AAVICcO517T?o1{?fNUKlte@PjXKl6_S5u!1IrK{`o%}@D#wF zA=&fyJ{kP<`#e4DH~;w3fBd(XUV_r)UwP&C+qMFFy}EbL@3+7CpF6g`zJKqo>WZ?O z1NE&HWJ=Gl7(c}9Xz$)zUAwiS>dlJEH#^%al?r|~5njuuS91%ii?QWgA{md&&dv%& zqm7LX4K)XQT5BnT9TGZ9BOn^Z)EToP74+ugfqXnv09ptnf;00olRDi94p-IQ`G+3( z@6m`qQVCmx%xxnqz_4nUm9-GC=VEikq`#010I#DQ<0!^hnT0t3u)`Y}jdG-^ro5@T z9NyB<2W#waZy12p4Yt)HJL*xL4aB}yI;NY9?9j;g>DU~AjYTUFip{ zvrZ$RSEdva1THNdnhvCp_nH z+SF8HUqkt}wuTrsF2$XUOT@qpNq}`2bK+ZBVM~vB}R1Aw;tHqU9|&QzYp8E z57ShJZr+V<-a%^LN$cG=O6iy}OEbZ#rKoRZ!M~RDZDv9nnaFB740*3y^zc&h$Wjsj zEIR9$wdovMg+(VfYounS$f{sXDroKrwnxJS2GpToI+ZN_2%d{=p!6KTwC?DyeWSN} zOUJ>Nn|A+4&G!FUyYoMIgwBw2ycnG*Z^`+rdA}{{(P$Mk3JHeBHK7R2 z7<>}}*P!&{Z;Ph7?UHt0fi=cC8d44G~`QGKFBO7svoZn9v?7{bU zqua6M4hFqXAtswNoch|oY<=^8bhPc^(0X(-+Kgp5I5`%u4FhyVTw^{nZ(h%tRnzAt zs7?viA;wIrs39YNUc;DGQ(aP=OGfm_p%RE5IdewIpI3__2I-WTrXsg12~7%I-2}O1 z7+yXCucX!QlIR|#g1euVNn>;PaUAYj7MHWM&mX@fdrRArzwFnL0+C?z7wqEHftG^CY> zH1aUy$;Lv8;fP`wIIw`MR8VS+{YZdev%^%VxD=o4X5k$}C>s^&_Ccv54I0FNxB=rM^sfPy5T!NZ9Fi!2YjFOykwnM@;@M(52-!{r~N`;dW zo}Ndui8)#haTo`ePzH@6x|&B)a`2;+K`9+01&(U4ivX*`wbc;XYbo6gOax4T?4Q=ZMkyrPHN07uA`zQ9dwTt&58nRp+NHNnZ@ztc{kxAZ|MmUTZ(mru_V&q7KD==C z-D6kZKJnG(?|*mm^RK`D_}i~PynpwL2M@lxbNA+Ls7xZH+qb^~V7q(o+XoMBK6nW9 z-Mxq31Kodk3+Ta5x53o<)z_bW@WFdO{P68Vs2c5k2w)GP(BhLx_A$VIJ_SENSwEh# zf6lEZonIRNv&TNh?9YO7&phyXJ+d;-M=zfYcwzG73-vg}_uTbg67;q&elqlrP~zo- zyFcB(^Wedq2lrpN3iz9U{?i}-`Hz46w^v{N_t&@le(Tmh@813L?w$X+{q>i3yz%

LzFF_*elI!y$u} zASC6&3NB5~CW={9xzIi+sQzK!)cV=d0Rph~(@y|kI|dVGNR9c5WXBj2aLyM*h( z3QOqh2q_g97Sn=Cv7&>=28?Dv7G#n+5pPK$T2aa19SnmB00z)S7KkVUAx$Kp2@!xr zC1otF?j4;)s_2i=R_?+U{j6wjgkhpYx1X$68yZY-LJl)jJ zwqlipD`auFbg8sZ>nTg^DNXGzP3bC0>Ml<1DNgMvN$)C7?=H=FQk>pdl=>Kq#hG2jncXE>PYN@- ziZY)RWpx&3b(LgymENCUL04IRXKBIXlERL{{Kq8-v>uff zJt{5iD9#7ydQg;abX|?9@%70uEg6Y-3er2ua~{_eJgF_}X(;P$s_1K~8ER`Bx>-MX zt8Vai!_e)fp<6)BBX?S!-oN=2IktRzqO<+)s`E3wopS>{^Mkz$f5!~}77{!hoE_<( z8|j-J>YeEA8h!Hk`S3tqP6h^VZH6&5!J1kTQA7rv#6m0=4uS{_*s%M>rEoYDE}O)L zfzv_&2Cg42i@>B~=p-yeB(W(37MaMR5SU~Vn@VJpseGzH!;{$Y0lm0N23rQPWH3w0 zU`kj>aaR$WDwax}1M~^SY2~d6`PJ^Baas>AmRw2^h-e}uMW7;aRV22G&e!PN^nuYS zuKvLcvDA`4u*BnS$Rsk8DO9UCGPxNR`-7$RRU0(K7rO_9>LU|9L!#__0^CAl3+wOP z?wM@p7|*GFkXYF49F#0|4)lr0h|Q|@3`vPfFK>P{mS1~U<>K!jp3rjV@$~%K+V%<3 zlpNq{_hMu3V%;Dx+1b0?+IU zJ&H|3b7&|ogyAv13rP&O^|GUIsW^y&rjV_Oq-!Yj6$>lyyRo8DVHE(TlV@m1U2EIJ zCtcMIHQpgUbPox;!JHx%XcWFFzl={kU~< zcC+)QSB#@R$vwS1rE|Dlp12+0HtB_C4p=qetp5zkBpw08oNPT1=#DRU@N4j(x>4gC72q`iNJQ;9}pAV3d zkH7Anzwa6ua`baWGw|jVlqnfwPQ_Z%2}A*lXR8$0E7=+exM)a17D>PW3j&5(DsZw> zdINxYy9WDvhw1gvfk}C(S!Fqyr5PEenJES7ktqp&Q4!7|foflGfs+Fym*Y4bD>@ws zBvOc$WTGh!V@|+Xk_grmut0->Cj&l=$mf%VLf~j1sfaG((FKCkyyBJ3l`Y&-j-F-kW^e1pI082 zT@sa+9~Pg9L=l4{-TZ?bJiXLT4ib%0Am;M<5R*w{FmW;|RilQKaw?yXgP14^jKNI4@*|{c^Xo@7bN9-gwd8b_CfvykyOSOY7}lN{ zaX%;KQGV=`qQoaf$xn)sfVzrPdP>s(x1N+FbrvT9!aXWXcvukMS(Nm+F!@nI^24Ik zj>41&`KcWRX%7q1ALgfa7NtGTPkof1)&Y0}-YLlJ$j^L`m+`P5>rqj5M`6apqRft> z%tysp4~w%qit~)DGU|R|ZhKzlo!pGuIq7W~$*t*0b%{}psWG?n(jJ!OJ+4B~1&mi; zV_AP=`CxO^U`s6k*x=3j{#*3}x0?rUw+sM)-DyFBhlXy;NZpM`-vIWa`{B35jRAyS zMu35FuD@^52(yvFh0(#e(Sf;=SrQuqq6I>t2;BQ+VzQ?Ld1}94JC5KB%u_? z6=RuVsz4>wIJ2c16csXLCt=W3B83gnxJ;%*K;v;O33xLs#vG5eB7*n1JcUZEvy<98 z+WQ7(S2o}5nykJvkX3y*E~nN>AMYBRoL76lspDB#QkjE)Tw&wG)Ur0Iy>CEda%NFY zOMB=1o81k7uAPgmy|7R+`g4vn4@ymkhoN`5Q0=3v8T#5z|lL#$0yE9;i=U5xc5Gv+rId` zeev7gyFYg?zaqW05P^5jmUwnFf~>=G97_}Toz-p?z?AJ8Jh?8C;esy6%O;LJrdR$gQbb4ZVa-w&5xShYB#MK2>DhYf(hQ+dh zVE`}_0tWmd%y1YhGSP+#u4y_z7jR%`m_gw3NCE*(ETS3G>nU7rQFT4IKTpm-0>5Ph z*#8g1{s+eX0+>-Jr^sg5-x0;blV1~0JgDn`f_pO&f|f?+P3V3AX$*#^5=p1wAGXS2U$y{l%WvvReo2I$R`>gCSrSC1+I#>N}79~Z>jP7Au7 z9t=o!H#6cvPRxVcctm-L9fb&@9u}p5Nk?JQgMy^Hx$$>%fD#Q#YR^r+lbdumFZn@X z8o*e4R^q*!In{q!l{a#-B-Q4uMdFkyrsrU0UItsJF%RAYrcXHDKuI}dN zwC82t$<4ZzodI+sGo>Xxp(!=CAvwA=BjI*_`s0fHuA0K0+G0cvr30;1LoHRqt<@uK z*GF&EjofG$yafQ(Fx1{Ua_`3Iy&C{s;~lpVfFbB=2jf&%2U2MsIr7yrH_&Gc7#cBV zWEdEihKFB`4lj)j&yNnx4G+%@3{3U+O%4xsKfW)O39efHXhF0BLesGnKAk1wGNn8k zAKB|7v$-TT8_YR!F<&L+DJ5V|L^dbMR%CFcp|LOl&mt4p6aX+Hhe{Q|R1r)ULu@6B zqh!NU7yyhWAd`7SG8YgI;;7W2=_NU}kJBnT5{jF&enDgj4_0zmN)}BBYy_Jo6j1mA zB1b@E^U*Zu8V3C%%E}CfvLK*v5J{kvh&4))M#WbufGOFt}^d@~_`(c{G_`cGE31EvLFYr{cC(Xi8{8wqXes z^QdopZRc!j?{Z@w_$>rofMkevkr6;^16;|6mtfr7e}~KufJHl7-#OXXIa=R2THD-P zTi;$^-&tSXT3y*(d9$%>NbWF(6AfvKOS8a=P0s@>);lzkoSUU~wN<-m9DE$Sf_?N6 z;68}UFE6=q?|#?N@X)h~k%_6{iOIp|lf6%Yrh1=EJsKP>t8aAh@*r_&1P+BHpc8mB z3;+*mUh$dfMb%YN$#Hz8 z2*)52cocvZp{r8m1Mr}hx=H9V$T`q&Y<2}aXAL2*Z`~}v?R6nC_FVoJPW+F?l?c+1 zL=Pl`kOf?Tua%~uR zGXfecf^aZ{E)lWR5|&!TwUr4Slw4aWLn#1XG)X|FNr_akjn{mZE1pZWR&dQ32LsO|XR5Av+gg8Vl7tdxR z=^K0=fy*Iq8CaOgl1iKIJlHvYj|2(d{&sr#_3ZL@gkfNQ@%ilSmvdzD1g z|H2rO-(esbvL9xE*|+_$W1wG-3^jj^wWbU{>~ekQbZTM!Mn_*!-R+#J){OGTl%ne~ zS;diQ1!2iK0kLU5VeuY;5l%k-S~qv8MlBZexm-FG!c!R}Is*^WQ35_iEP=&xwm`w< z%6VKdo6Uh05^il_^2+etjsDino_eH(O852k?wXY+)o(hhmLFHWdRV#mpnR$=zpEtn zPDWr`vj5GLz?&(-cQPXGX2-PW#I*xl$-%=lZG3AeKnZ)7I6 zfq711dtS<&?4;XS3AZxiZfC{c%!q4Ck2U<6q+6M(cd|3?WToFoPq~qnd@CdMR%S|T zT4GCDVp~R1i@|BNq$M?{CN?C+U5}5fjty;2i@5__U0GI7ZAou!X>VQ00I*z56^Qq0 zt{iTy0rVPas~x*pKYqJuq`hgVy$R53%z!Q=a`@;j5O7__$l+h~LhP3z^RO4VuD=1c zI5f04G`uuCvNSrnFfu$hJTyBrJUck_VsKz`c;M+!pNEI@71Jx%FlJZLCMa+zNg=Vl zSf~|4LJj~Jfyu@|5P`*j#eBX>##c$`LJrZ8=4VaCT9Q#{2#;fu5Qc$OwE&_^*n>c&Hm=T0bo1GMi*km_Aa*f-)$ef+d6!=d4%+)*gSX-00v$HZ6H4Ed;>W{ zcC@~AunJ7r#?HnXU>FiheDha0acOyV@zopP!xrXWFU-9H7<(}@fB*55=;VatoV2Dlq=*|F)DV>62*FBS(U=K7z{bdODx)Yhw99B{z$ z@*#={Ch$lkE`_ZS`Nzfv#3xIgJQz|Xng*NLU`C%;h4l|+R{t=u z`r)d@KTIsX2RyT|vLu0p1%oVN5=1P5h>sWXaY7zWfQT#+Q6wU&OiWfts7ev65mV#> zYc_1bpknw;vW!oW^GGt@ckaPnJhc!DQ&4m|iUDJoOdBSQV!{M2izHytWIU;ht)t!} zBq2H_FUKp`pUQ*KbQ}gI5V=&kltq)nBr%C1A=71WSW42=@{R#uMi>|?%^QjJca>ga zB@QHe9;rb1t@gYzNAc^5}PR!N`SQ;Kj%V%o^bQu%P!eH2Fkv&`NNZ0wW zor48(50c0YBle=W#o4zG+?(G&T08x9>+I9c#i#vCLtn9Xe;xyXAvqnNmv?sVJh@-l zQgyrY-t5ZS*74=m;l<`YSoj>SZ0s(tZNFOE8K0SLyw%cryQTT&&E%|H_kci|qZ1?s z-zu5JA#v3lu9gkS=_DZ)(3C1?!&(`vlQZpP`}NNgBfx(C^Yr~6$M1eWd<*m&_}5;) zaf2Zf&?Fq5ty*BG$HQDJFu zi3KV7)p?mE=@~`o$@!_s?pJh}lfSRPR!8FVtRb2ujcP@wfr$+ZLa||Mh+>LEn_|FX zf?!D`SW_t|2qtj2RH29}76S)H;BoOB2ARj!xVk;;8`(Si{dx`n zcJbxx{PXz*0x%%(^7IU0)&KBe|JRlov+9q3Jplka`sMiO!{On#toq&kcf0!lU}w)~ zSK2!IOB?U!*WAvkXaN9=%PEaaF9=D>*2kv#MI?F#ML7HDwJz==g_6hTFj-VOgG6Id z31pOw%~cZFMk43PbrPXkDAb5VYOzEqRw{%!i6Kja?dyX#)&an}YgfB#*SdiTs|G%7 zk`Akq&YdNH8rLsC8{|&vMD*DJ|VO*F`_LszBMhW zH8s(oq?Y8wrlh!*l=!Bk*!skn`lP7Zgz)R};b3e`iD^iV0>Z0D-vi|yt!N#g#;|b-~+R+>JV>cTBy`J7~9KX{tey?TZe(TtSo8u1*#_J()U3Z@v z#)(d2;>MGi-p-eOPiFg&tc{tTZoshr02rbLppg;ay=I31z=mE94o(jZP7V!BkB_9M zCYoAaGsT!&Q*20FB1gqkILoB=GP;08WHHfn7)7Jw7%+v$HiQuQOsRkdmgg**4FzkB zI4~lxUnC9{NrsZJ$RY-w3+^(c4-`Yn7TZY$S}{Y)fvS`ROMd|Gr)WzO|m zfywD~nGi6KtKsrgT$Ti&i^Y_(*m4d_$^j%JvMAO_A$$ymj;C;G3=sz@PNb0X6*87Y z%$AB_0T)fB7%a)GO9Ws??-7O}NPEA1 z_#U8cZSTTx{P`G(B5neP?X9oxuC497S=)YtG-22R410wzY;E!N%Hql+##n|MC6v@pH2^m1`|cIoxZ!mH^yL@(x+rskI>7hX*)EKSTWKb>73oqjd)Vrgh< zzH4M6zv{Z&$&tus0)SD3RJxGi8mupEZc8t(wbzGpHI5{X$Q(m5L*p#)1PdI&1ckY3 zh5FIb2Ego^75171>PJ8_Q_JtoEPk-ELea?-F^4P@kYo~~SVR&FDaiJin68k~xSiV4{>wl5wai{&x%+567g~&?q)^2mlPtf|2JD3j+T>x`ZpR zSJ?TxxdeL!#Ye?uraO4MqUm^~|2Nst2%LgOF>xdjjiu!Y9puS{#dGUNTj!s)&p#n) za!9%XV!{k0GvptBD;8ypA|lW-((CwJ3B=zv&wqKkv+dg=+vAwjm_4MUz+ufUOcbaM%YNC^poPGUO zZXQgT6we?rz>3mQz)(SCF>rwl;0Rf^a<;veWv}5lYIu%Xj)R8d1n8wx`a3Cn90F1k zI>smGcHS)?e_A{HvU&E~_TL*|NDW`4jPK#wum6i){{gUno`N~Tu&=;`z24f7$w(*j z8IX*_P>LA}DO)4w>J&_cNbPPP93K{u8si@m;u_)yT$f{@-ZLURI5|E%Jux^XIzBHc zuPQgaFfA!JB|bY*9~~@r(o*;=D;mk1jJKo`Z6FGoNkg+B8#aXEGH^VW6@~PpwH44c z6zUoVYl0_OQE5aj50c5~QYlp|V#uX@t=c^(q-Shu|Kj7}+s^=C$CtmKB8R|!M~HR$ z3Bi~_r@(0$PBfnweAvGo*#7|-5=%V!czW{D$go3TzYHhCj3uVlwoXPS7n|?(mNj+c z*R*Gqx2BcUCFWN|XBGi~0SD$EndBK1>Exx?IJpYtD!z!%;X)9TLZsoX(dHJG*Qhj{ zOeK=&q#PxmZ)BKCB$e@Ef<30YTi1tguJtyq_cm_yHmvsmhFxFlss&o4Z0o| zT%QQo6Iz=PRuL5dR2>~u7Z=(PA6^gMj0>)d52=q2tB;GQO9-zAiVts0jBHAZX-A?C{_WK-cili{YV}@v+v%T9`${(eYF+ zl_7xyI+AV31I>sBJv;#xaP%3#Q>gK1WQ;{KApk^f>|=L$ll&PEGj%LN8{;E6*8F$jzG)f zDcQ)0F$t3`XM>R?_E5tmRh$ol80zP1vK&fPigj60_ zr8030x;2SphQXQ{j#`@IFu;nLVNh4Ck=hFc28$t<5?EXmg-&DvP>V6Jh{%u&)NZ~} zX))<#iP=@RJBF9nPXM~M5%2YOSd4C;ARQ_YfB_S>ce%d%c6Ik0 ztj`RGppVwJklcEs6#2q17mKS?OY0L0Kx^YKm&c};`^Kk>uGcG_ zo#zHWUh)N;l?JAm0NDd(8&>gB9wk zrHzRt>PMuNwuKdiNRjX;a*+XGLK1LYa)2u(L!)A8G%TH#X{+MctH=_r4U2~6!dL+d zD}V_S23`VV#qf8)dto4|4Mao1z=6?mTnGry1GbLM(Moi_&aR=}ZXw>GiE&Y>DQXux z6cuAZ1lA8JzG*>1W7!mjiYsza@EuiYWtH>Wr|ah*x88o)!XiZa&W-z-(g7A`)9Mb*NDnZsXuW9|i+37%|8cK{`A@ zDRxLuE6OwQ*jf%#NoOhi)h;wJUSkXuD_k78i>(Sl4bjHjvq#&8-Th1NN7sIxSpRL7 z=;HtkgIlcNh_b(y-+!8a^V8(&`s}N>bFVh1mN!P{Rz~NRCKti`3@~iPJu^Sxb~iNN zK6dKh$uoxv_U44eMwx;`G_D}F6eA0$A|r~KMRXaRso?Uo64ay-T8+SUVGc9qFoKiA&>vHJGU)s0V2UycXI#ZUxnmJH>oaGqKr(5oVEc;Xg2wM*HmFL7%fWf$L!sgav+4xW$r$k5f6Y=nv+xtxRnJ+mROn% zP7*J|i^O$s0!ijMCSA%+F3V4?-j{XdU|z$~JvU4C-#LA_>CBP4=Z@aHSkhKn(ou1$ zvkHdFcUE5Psw(TQuIQ=RQddny2f$c;^}`!=_ixno0$r~K0swnRWF8WeDm`KIu=CI( z0Ij~3yPo^49rp%0+XlNkUi1LLCXT*NK(F?vT^&z4I|lFf-o16*ZZ|U# z23N`vYXw@D!Wg902k0<8&X9`9A`uA@kP!hxBm@9si4m?;Bvj*EOyJLnf=|T=mgvc~XrBRC;lmKHAEg-D~08F9gLQBdckJ;g*Omjvui-a z{;-va=@s`ZOaXRJFS{oeCfxHA?zxG{xiR<5hfzwT{r>T`hXaRBpVc_c z0yVZfFY{Vk$L;QC*E=5XJyU85ixunaWG=WTvjDAu{8(&Y#kNzaKa$Ab`;fl%_63G) z8^D;4-*>*gzC0#NDPgH`h7zaCp;#&<0GQBhL=1Y=WE5HTs8LRnaQsjvNz7-eFt$d< zP>WbPF;gS{h9cllP(B&ulMoIGH+A26A(CL62hcOd0eenY*R_>#(n2XUoO6%0JmM3qU#4})gqXaQY5wlq0kSYV^ z${?jC%%BZ5qZ)}Xi{Z;*v(=c)rNIMKVy8qJD3u0CFefH;h?D_hM~oWgWkle(ji+#m zgOWpYP97<1Y3ZF<99aBmWc63~`iH64zsgN&&}QKQ+*bFdU zZ$5bzz`)5OvEPdT7$nyJF=1QU{A=~ypDP=GuD$!*{j}d0=- zB*ml_Wu+BmrW9uE&Pj?+kKdh}mbR}T^Wfg3qO9ob#N<60MJ4<9oj6c%cu(GeqWH`t zi5}lh_xX|TvzEHQ!f9vD(t*_sAB>!y; z23ag-tHC3cUSu{Yolce0nOd|D{5>qb{=u3$MucB9oYK% z$90%2_1o%~#V`-O{$qaFtKU{${R#oL0s%&}a%^o2TYvz2KRUhA`k?Pz?VTf+8V;PV zFFa9^ap+9iK>)C$N%@E4Gm0XUvH`$c5wV7VK-^#y;!+;Up)trLGMx4#5OTCeg~qCq zTa;pxoTtV3YH(W=ixq-khw5(CiOG)oiPkE2YsExs#Z*h#MDwNbI~Uwd7u|Qj$tCxl zi-2dtH&490Rx)(8c%b&c)2h7>E){m2&1pTJe)~{jT|snJRya^aCQw90R%ChBE~uQS z^4!=<*)f;0Ak5BZ#-7iJxsVZiJ}u@#S~Sqv6lFp6aSTODuO??5-*8sw(fjT=n1zoDJ4n zQ_+37qPM>0enaj3n{^)bT(9kJ0INV$zqs7faQWfwI%3=x^j>|fw~6eiCJ(^+TDFRZ zKydq4XIp<)TYo3iKv(-f^)qXGz!sC4y;1hyvIzzC*1cDFnn^Bp@Cx0|4XUz>NV8V;oEZ zyZ^-^D#8c;V+V!$19-6W^ZU-<-;a;#f+BaN7sclvPd`w4`f_K*t!Edm-amJxy{x{i zwdcj?(ypvmc_ ziK!*`W8=7ty>bx)0qPmYXFjS=#I4G6fjq0x!K@yUMoMBms%-{@rj z*v!EAY~R@2^U=9yBeOtHhi3rE+8#YWdf|LjTJpuZy6(Q=j%TAl=j(0;#-++kE}B5% zLx%x2zD%w!gY83SdeIp>D0E=Le(>}EfuI-Uy1zeJ$meTS9IcA2QUY}GbXuNP3p)dV z_c9p;MuX6#2*6u0bxaD!B&y8wMs|N<}OjW6Fd~9Aztn9HkJ^ zh;ggZ8sP|t39^OTBX&onBy0$_hEZT>O;{^Q)cUnX9?8CqDVZMkKS z3qegXp-IM3p$wb{Oc+~*@(ogvO(hRBslsgX5R)dOO;huID2z!o?EoPYNxz}Uh&0I=T|H$E)B`LOu<*SYm~t@pc&&z`u~d*3~` zHo3Syxv=tLe5$&kHZ40gD%R-?FdD69C9z6G;xSOQkf-48VD9u{`%*+KCNAJAWr$i0 zB+#h9FWg_q^5RewcI&<~=PDZS-s*kc^KyQ0?%n9huM@9e*$@EiE4^U!~ z4(y8v9|j1w@#o6hkGI=9y(m^Jm`Yb(cpYpW@7?bnr+UshIr z{uHYvrrUTsb^Yz5iKh(c=qw_s?Xv zm!vfvh`Y5X`g&nheeSNyIRIn9rRjke(*n*WJ5MDzPwjS{+#PT{-dRGNl*9&{hzmTI z9Cki6;#_LP`Si%L?6``&#Ok8d+I^W<59QuCT6nu;AJDB609pqCx0=ru1LNgsHEg{E zy%(TYcg?x3+Kat)%RvFrVPRzU~uR1Ksz*EfK?uJ5yuXKVeNxG9fLja*x%FfysNFR zx9j$e`p}3#m050!Fj~X(rXZ~)*r>9oxe~YxMuxkac{I2)jL#Gyu#hMfp$e%;DHGrr zPby?%2u;BBV^jUuOpa6{)at?GBcf8VrD7J0loPRWF&h_g6(X)u$dn;WsemG6`*11V zY(JWahN}3OUL?|>5v4}+CPNHSuZGjL<$^XqQHjR8#&3HNYl#uU)vB+TrXpPZNjBpY?lj1tKmUo{G- z2Cq!PIFGUE7w+j7le7Kq=>hl5i;4Lc6AJ_Gg+cJF@Ahb+Z*2bg=*)w`k=6%K?)Q&8 z9iDzVJo|8H>So8o#Jv4_R|Hq0^rFGIS|2*co5J#@2O#=*u8~>$77!&7UfU z!3|QE1rKqET_((KR0Rc{zjkMM<>!glzfZj;(&*n4BJ9^9ocIMT7pxin3|tq`=fzEc z3P6ta2O}@5n{MSEJ80VF;OT?{y%-45iCGGSCgo6NT&7Zpn$@aMmp&>~7Zq%bbn3%Q z601z06A9EJwuI}$^!H&?SqcGS6iJ*iO@z)EYjUQzLNbFRG6SP?f|85E)AsF(%MA!i zvbthj`6o`)cRsm0F!jK_3IVqI+xXgtiS-ZDFc;B7-_Hk;DL!nvH=FHC$eTs?Vo-ow(XsU{U0r~tqYKZc)<+gM$5(%ydi7z7*u}inf4K2! zZUe3=?AJ~v z$%{?d$)@e(vc36yl0ZNPHy06~E=3qJ0S!k1!&w)kHkZ!a!~ ztyCa-o!D$qx?IN4@WW@zhG$+ay@sJ7TTL8qi4x-5KY*YE+aOvHUw?T020E};U&@A_ zM8yA7Mf}=T@8Q;0fO!}Oo>d1Y=5KdCJX3SCxa|7=3s-WFpHJFb9Gky4E^kk4RzXy1 zcG&KefT%cY=q{ZzSf)22G6|E%q`(0~Bw)pKMwQm0!A)|^fWztJ8aSgIo=7;MF75s6 z<&*98V|OdZTdKzIR!p>1Otw@`+^ujomyI_80f3c_HJ%-9JTumKX5`k1mkmdU>W>Uv zIsBsLU|+?arFs3Zky$$JFG- z)a1uj6RIsrs4d)Gzc=~%!HjE%fU+76XJ0#<(@>m$v!v+u$-Q?@?{7MNsPWXnrZb0I z&L0KnYP)ozz3gOL*~yNI(_PiTb)D<3xzK(2V)vEO`}GxoTONP`bUmuCd2|&3tfu!` zjc31?M-OiUXf-^(bM4WctH6OhX@aTq|JFUMTlI=BRb#_l~$}b33WQI0%ysj9Gp1H zr5q)OsAXIQ#*vEw#u!pA4P$a-ERmKkHHzeB8E(RX{Ss*;s2W3*LV{lYG(H27VW*C@D)}Be0xKHIVdVU|6oT?-_+deSpcvl2r!}x@%`*F zT>1hGn<01xFA_e?vl2E1GzXJA+zYSA=U$BihJC3k!sZUR*?EaDWN^##*yQ}^#N4QR zZVVo0iA=?rtq&*WM%;75?zxu}b3;?}gOhWE?l}Okp~;1z$tCd6NeG-Q4Nfcrk^!W> zm;&cZ;G96(T-Vd#eI*y{VR0h0(VNcN?oah1v%LHn-c*Jcow<|7@L_O(0L-?Nsc>P8 zm_hJQjiwcOVnm{Tv~G8-u=5)YU}~RdE7;e$%y7iz=Y^kv zKF++|d^9$4@p^SaQHnOifLWx7UcgZyK!`?&>S32WOD&|zcvSF?(W_-HyDB0?9T}<# zb>KF&)T~h0wKAg;kqgN@7|g}j3FQHDW27-4H8>rPuz>oXVzaXyc=KMoLv27Y7K@+ z&3L*H-w*)%l7hJO9xf%W0{tI)5qkCRp9H}Ex$^eY^_F(INzW9sxpD+iOGE~>Eh;=V zCu4VhZen46@}8pI1$i+!z;a~(Lza-6l9-#4T9}@>KPSJq=*anF2TvW@f8tPNN(?Y? zKT>_Yzz@%5@8ofI^0eh<8Ui zp4_UMZm%CF00w;6WNYQr-HJ&-vgY#fJEcIAz=}1V0|0}%eRAYx3GiWnVFQ;B_5+4h z?e8nw^Srd^>BWMF=W-sM&+9#x*KsPd{ZvNl$@J!u)TZK;rbDTXhtuvH&1ycL(|9cF z&WXI{6M2ota~qH6HJ-?CJYICGxZrllp2k!AZlB)Yc;>+E(+BRHKGb~nNXz+SK)`;r zUO3)fehP>H7(myhp37xDm&+eqsk~oT*>k0`_eymyFkbbSA6~0@c)jXDL-qY@)x8bX z4;qN2uj^O7>{-6{xbf=KriNz#V0UixwcH+Py))3-JOCV6OXJ|Z=D~Y+huUHNkXQg~ z@9*ds?Ccuo>>ljue$l-pqLrgeAbeEhs7!+=?1?emSDocPy zYf%W~C=25;gj^t&n9rAkCqf0LQvr!J3O+7n!R`qmI06GC(Q5_3b;)oJE=SZFvC$wh zXazc*NN3_HRSYS{kOC7X<*RT+4Fn%VP#uO_aJ56Ouu5ensYoLhC`1S@6ez`PDc_Gx z-bvjlQA+eylSnB?a2cjmsw_IOM#7Z}Ai!{;NGnxZbuxn*QNSoWfdZFkHFBd#VX>&~ zcA43X84RdO1p!8({^;+&olMzACPV)L`|-%%`S}0CoAeKF{~vs*G``pz7!jG6b@^&D zuv61ZZ=vN{B#5=V>EYJ=GAxPD!`7rt05D+1rU1aeaUS?E!ip`taWB3B7<13TWJNe& z3@~hN1#WhpU3SkdkIgQP%`A;gFOE$vj887O!Es`KYyzAwj)RNS3nP;YV-t%nCl^L$ zmR?RR4oxn;gkz&uMrK!EPXDdpnKf`6np*jCz5=M$?_PNFa{655HP^00xyepqbAR+B z?*#NBvjD($(7-)}wu1_+7{`anCMt@Q9sVR=HjOD0bJR+nUdz>MfcQooYBq`OX3S~E z9D0@tHkYzwJeC~cX)(kolQ?vIBhEMDs6+b=LxFJB7yy_+C&SE2iCF>cm)4;$yEOJ7 zOJaIb;lTshMSHA4PG2VJTkmb(ll=Sz0=`ZsaXO`efmnb;9vY+y4s^!EU2A*bUjJ?4 z%|}>5-1r2H!onABEo{Py9I#6 z*-C_?(3xg+Lvm zbtX7M(t@M%qN599BXXj`vm&B$LR0oeW*&?N$}Nsf+ZU5in0TzJ^5*@%w*Kkfk)>zT zZ-y3s9)$q=#r^8n8N!4uy!%AVkM`^fh9SRm>zk9yYd1TZ1L6ZERV7HfhJdfC3b+p1Svo-55S1UIS8;NI6wN&)%X7* z^f$fM0L%z_t#16a^yWiDbGzQ<b9^!nFN+QuY?59oUn!w_taE zc2Z$(QUP2KOW&K7wLd3wZ&v={f+J^+9z0c?e|WDY)ZxRSY-f_axGWz5-y21C@_9S? z+#NiQ7sByDxxOO4pO{a=cvKukO-hkPO%V%rvY6lc`TUR9kK4#3nn=V|sl*nGJTTA} z8F#7v*5va0#n-Td0|0F0&6ZX-{#bnjVD@?C4Y8w{Fkzn2D*sPGfI)O^wR?CHe!t{# zY=uh!fISWz#)K-YjUApCe(^ijZmz^WG0o|WDqHpJh6bz zq50GOX>2B@keV%eqs^eT>$MJpKER}N=#>_Y+NM<-q?rjJgUy$x+OGfy<^})*0^X~s z4A5(;r4pQsH(r`(E*)>WFm~tM=7~V`!HXX~p3%QkZt0Wh2>(0rd#uG(% zPVTvLV&9z;z;YdII(r!C4xzgjj<%Ja=m4(k(usQ)OWI3Mc2&X^ug>c8kX|*Hx~t2& zD$Bd8%Dbw|yDpb^)>d?1sqCq%>aMHmy;|K~_PWTUYO2uY1~bqrbIjkT70@?ac%Cn!txJ%=g|s;J^mj+h0I#_54k*o-QEZ#sIyB zV64@y!(M``DNulI*96YrdCI$mPTimMrKxqX8@m8N9Wc?X4gjNUX9GY0=OEPdo?<@ z4$gtToWK_hO)mG3Ez~!5?n=zkIRaTC$u?igc0ZaInYok7*hZ!O2q;dXZX?ll64gat zI&fzGJL$eO3;;zVFdKpQ0zyoBF>G_Qh#h*4R!oy|IZ70Wtw05OsnCRT)k2O&Bo8+0 zB7?r+>m;ZF*e`|Ds?r3TfV6>nTd*N8+?JJ}e)8<8!zIU)Gc%%ULD`82cfX&%^xw=nl;5n8T)dN4-JPxTg|vpt#)dIl7gahcEuIM#24+3$c_xlj0nvLi_Q(tJQ$mQ zEHU?J;+~VK`NvZ;4rQIFsk!xF;9lQk&&cAFsn>9TnKF+Rh4o=R#n3^4$oi4vs8=euN2*d+-2Ugvvdpa_DyR$dF@IZ1_Zqb2#=~>C< z05eyLP*4^P<z9k~hnIeG zulzbqR1Q7+m*JN1jn5u>eF4~KPcO{M`@g;d?B5v%0Jiq_Uuy)w+8*_#>HBgr34rZAwhsdAWO2rx zJlv@9Vo`l~OdmeiS19lkp>VJ_!rO^(b|UN@d^R}o5pu~=0aGPG&1#8V&sKr^g~Xe~ z`JU{%jqK~k=Q3~^YBWflc1KKnP2;_(l}%3z;_BN!0I`-||4#U@&jiLkuX@7i2?yq3 z)z=OVj}O}_9uoFzZT)u-y}s^f{;H+rpQjf#9>18VZEQbXbMtU%ZR){dQ{*n(WyY*3 zo?6J2b5J!0GawR^2)D>Ftz00**a(}-@b{#n}gRdcbYwzQ|Fw7aUTr?R5Q(|LFWati>g8=PDL-s|%HhT5Kn%MWhU zJ-S&>&lpx3p>Pz?{j#hX{3HNo^=qH@>_VW9YbAhFCe{uI{Lc1o_BRU@97$O_PDp{=H+w8^V4D@f=wZzW>=Wi8f?MTGOkpF zD&-=L8dWKIuscIc6~VqUsz^x3;6_74rAD{IXoiy4`Dt!k)7ctJza{_ltk?Fp4wlAH#jl^K05{_6F5E0w^{N>Eb`p25 z2DmQVuF(VtHJ7dA{tYk%N|Ol4m=M@ZDFj%6Avo9? z9^r`CWe?wF4GA-XD*=w6xRBj>@kxcdllLUX6~+gogg8=ygEPY7@*}ej#^)9%7M@7o zcQP~Qa8ky>>~mMI-MT-}3IH~|_;m91pa)=E#X9JCJUiGHJgpq>KMc*!mS3-{ym|f6 z*hu$C-+}TY&SYz7nk)ZkZe3$T`PHjQIr(vE8QFye8QH0>P$wTpeb|(pbU4%z{JQ8U zoWBRq#lTRe2&E!?5{Ki4v)*!3ZY1ElCT9Zbnl^o7Z81K9>-geo2ug2Y4?Y>(z(ONYD z7}iwghD%?UfJSd$1Uv&613V)-63@f^VYkiH2e4}cZuufq_W*AEYIxMx@c32(px2Ye>p*~BeJxFd_qyAE z@6OhGSbNipj(aaU+Xp*gWE2eD=!6=8Fzb5R-8t0NKGX#O29xPwgJM_bKzDb4cXwZR z_wd69<4+!S+^9cSye}yxBswZEGB(r}XcfpYL?M$H^g@k>uTXGgI2{v{QG|wxVZf$B zDbndNqY*P1MFs=#Uf>F<)yph)ZD5ElC{*PLQdnIwvz@Qjz)Viy&y)&Os}kvyd=r0Y5cA|Bfm*7v z=`}91(rHrJf#1?$8nsBJmS}Z=WOAckVFDM8EV0Ok!SJThy=e?D3J@LKyP*$*Bm?(x z23Mj|nVob*yn_sE8Chwt-0gfc<7qeq^jh1TUEQ2rBC3eXn*d$2E0Ah4EANP8DcGqv z?I9Kb*!%{tUsFVg;?(SGs5xSz*c|LvbkD>2EC6Gk6*4#p8j47wngd{4B`g{ostNlT zp<`S1a0~W^EWi#C0I>17*FfN8Wabs1*Z9I4aP$b6vls5Ao@XQJ1qUQr6I&$nrEz}r zq5R-Y{?QNm&K-o{Kb#JK2|M6kZ{Hm>ACiF0(?bUa7>1gSn9VA4Iy9jH$^auzC*tZv z0uzp!<$Qw-G0J!b3BZ`pfXf|vzDE9yD!`x%HW|ar_Gm|FQbc5COjt@-WJ36{Q%A2{ zyL!H~ET^bQZ83Q>{C7}%w~_q*cc<4j3YCagkW+*x8tEGX70fC7jX@~QQE*yzJusa~b8e15e zSDb$6)Zv072P2bHqmxq7bFIU&dB8-D^KjuS4$zLpMD1A^_%D{qig*5({8|F8}0d;`qCS zxP>wB;k&;8z*gV>x%lS8@a$^U?YoJE`vP_++G1kNk&(t-5!TqK;FP4W)a0P#-2uB} zcBLin&dUY>OWKp0Sd^W*H@l#CU-7x)p88>MTog^n_vN$vg@`YT_zF>9$S}lT0x%{3 z+@j$!hC&WE&uf%ag@mn9B1R40q++S%{ut`RXZmoNK0FpxjB*q>qE?5*CpGsxom>B9 z>Fp;NdIB66VZxwZ189AE_2%>1Ya$o%^#`c`0@#<-MEGKYUJzjaPOq;3TUgqhoPE>v zWVrNN)6t6h+>*1lxHzFzi(2JEt5j;2VitkKz){+TD!W8s#w98#i_fDlsT4YiMD`=o zC_E9OFlu!+qsd{i2U?v$R!5-8<}?~@I=x8-8qy;{>* zU)6P`>cQ38`whfy<(qYnh=QSqVZebszH|La(~ZZC4Nvaec-DOLS<}sD&9}ca(67l+juRFtb%^ zwnpa^U<|ga944t*gXvW` zB$g2_fSF8ct66D=C#XV6<0B+Chs0u&n80bVylG5tIuqQ!0lmBc)#yxtTxALf;bKy6 z3fYH35=dlq*PCV*-opCf>ie11cXMlhmk^;FTiNuqBTg;7n}V&0?_jnsoV^0j1;Y{v zq`@UIh_QJ9Hn=u6u?X7~iI=TU&b@-bgKPsAfj?Uthk`~8z5q@jdj)$y0PTp*Mi0Oy zU~&~Sd4CJ|zs0xig$;l)pwZd&p^4=uFDCbwTrdVj;07y=C)(ymg#h!RZ1)E~jJA_P z^QJSrnZz%FOz|RUz}sa_+Hvxo_t6!2J86m0w3+!QPX( zH=Ya;2(bBgA7?gxel+s(Sb2#()~E;&sl(OA7>gyrVNY`eXFFpHA_|Y?A3l3Dr6AWH z9uc%FDkUQ=H9aY8SBTP}_U8iE>_GSUgjE zBRZ!rFe%xwJKhl=XN!w9M@Kqhqg}C)0dc!R6Jz4CQWEnrk_)qwi?Y-98J z@c-$+w$44Y0}KO#ZVdJ)!d0>Lt%Tp@)n5R>CT7;#9t@nXzg=8epLP7SJ|aSBR$+FP z$Rd^5Fu5I-S-22j4vEqt$COeAmqVtLsdN&V>hDYT=LiLAlf_`OT3t3*kku7vas?U! zgDiHZ!D7PC9Lyf=ygt%eGu%=!(RO*d{mN8Z^+ap+M03?7aAVC-?v|>t=8DlfWxk@n;o)VMiX7A9+}Q5j5f1h=*tG=Mt4xUOzE!3(Oh`T}g+fB{iu zJ+v>OSUr!fw zT3wwmkD~)dOaXreomn@$4qnsO10C4FgZnR^JsItP<{lb&(A~CoZ=OPpqY|M=3Enm` zu1dvGD&brnTngS!6fw%cWg?SVVsnU07R+RoS*&uaO=>a=bw;7V3;-svI@LiTYFCKZ zV8x68a(0ef;g1UZQG|hs*%ApCd^HTCi%@_r62J@sSJVI?*>X8cF6Sy043Tgf#b*c2 zkA?_17$!4nfeB;ds7NQ%IE{uNo7@cRha3gQlS_pvC9VeyGbqe@g;|Fibt;QRqSez; z5e-4XSD^CvR4$(a1ZSS{NNl!0n@#0%IYJ>D75dV^w;}s8$ULE_wBpLl{08hiTzxma zx-q*7TRC6=2L#wpv#URC(QEM?EEq1oB~t2PP(3`toZolTi*Jb*55kizfKS1NF*u?H z4j6;znwo>j_mlJRKN$!9X%13q0)T2BPJ$+0Kn&Gf*wPlmAhclJ5`GIp;46Q*_!c;` zmy;{~qw{C0t~sNVRJI@*BH8Xw2MpUmqI*%8UNpu|8U&aZgW<>GkeD2Q2E&K$@6Dq6 zixKebD=_L&lR;=PDuV2qP>0N}6`K_T*tMb6hC0F)wX?h5-ACINAK@ zURiIv-<@@ApCQtr2+_*|HDZTKWK~F=3RQ?!83Hh-zyp->K#e-YYKe}t$0a!9(*V6f zQ**=83&PS0!cy}?Q*vCp)2uOxmdH3$Xt=_vBcpVI9ydn^TjE0lQzH@zV^jCWgd|v% zHe7DP6$W+A{(~LQhM&4u`e)XM7Tygl{XD$->-hSIDPr|&p4h%N5A^=S*y>72Rf%Pn zQWGRl28p%1w3YhaNr2o4M5q8r5^VQ~Gi*T)R z`oOueh|Ju;WC$=@T#Pj~#ugLph>mhbg$KrjMy13i<)x(;WoPcsOFxjGd9X0&$lm>@ zj~>5x`oPKJsMG|m0`o;!WHI6|7Wj*WWQo`x6O*N4@WjMY5ynfcWP(3+m7J+jvb9Qy z-J%M1N?b;%%Y+)$6fxglz~`ytBE1%cjm7$uyuD9{r{LS;?I#GZcYm(D{c8=b+x@ln z<_|!xbx*c0+zm!B?BCG!?*M~yoL_$?W;X+d!7XBdUdw=C|28dbapkA6skP>wXMo}W zU^yjc%~4SjvsPwPORYF=#gsOo(k@UrfCCfDO;SvOG5K5ygGOUg{b`gRy?6RE8FHP$ z;&cUu1_X!NTmgolAY*`2Z_wddoL^R)Khbk*xTR*~ZY2QNRD11od+p@C%M&ft1i&iY zO_fB0V&!mS>G18+;ajC+jpY+f6=S!~kK8yleB;!}jngk1P7O7j8M=1nCD7F~gI7-W z1BO)|dt6ccu)MhE(xJ9<``XSQ>?kd6FFW2|Rzh@V5Gz|1XMj3A&;pk0tE$g+*Iwwp za;dZCVn@{lVyiN&91;zO6}@$p-L(~+aJO=0_Z4Wry02dDX}I$62Ef&|`w(FD09{XS zUiAn73>dFRjn@IX9^JY52mq}4R$puL^Oh!X9B6L^ta{#d?|J*Z{?0aVeA?RjybbWI z1FEaD52&+aprd1`vlHeb5~IRi5Jkm7z%2kaXwJY1FlxOopFSRb{&eK|)5(#czGn{) z9Na6xrCdzF#o^dhhD=Ho327M0kO^4|DWcZ`bO}vXk;N*r!A=dC$s#vffg3~hMv>VH zgxj6^kO+l6K&Uf`4Q7s9PC|Gz5yF#8I1(6g&yz{n5)4L8i^agz(J&0SG#GmZte#5A z#c?kN1>7ZR0=__jOAQ)SEf*UV8kbS+Fk)IIs>B5<1y6wkfFTMQrcq&9l~5%Wsd1@J zZ3+wuh>X_Qoh%XN$6{0Y2$jd9^7$;0kclER9-G2p(l|^ilT4zLNEDJEjYOjRGkN@D zCoW9RuET2K+UD%qyP1_Q=pr1L2Vg`?{ks`JvgLPEo^ql`kSp(@6(b@%AkUVeQ3F>1 z)jV6rh^$4pnG7~DEJP6W}#Vkd81M1b99IUGN(b zE)aYOs0IKwIt{Cc)i+v#cV{T=Apl@LbdDDlZvTSBVsn8R1-}YEaAZS(`Osdm85Wg>w|E3oJ>mqi^Bs0^`-EGpE9V-~e>S3qz|oFi_R!~$;bY+o+jm&5dC(st6x z-`HcEuDB3aY)D{iNLWIsGs=>bzq{u80I+Rj?{9r}{_l?M z|M2nx^a2d?qEqQYu0*FY2L_j1s2*9|oP6^K0NBjN?{h>S1mPIq_AdfpFfVE=i~~m0 zgDW3r-@bq19z9fk%9aqUiLgptTGWDLR+%D5tB%{91(?q`?K-b*6-=^RE{Bm*WdiR~E z+%Qcrq7LNBU8p)-WsEahQyi}JfZUV$S6Xg#Jbl{K)w93kWLWI(u&C&??9}{%oWztw zt;Iy)alPT=2F;g6_k+VjnKUt%Awd`tlqTj;;4oPc6BR|LWIuX2=U)BIz4puWtKa5d zf1H2)Vg8M$edE(Cu_Nv81uzIOPYRc3?8_ft8TNN0qNk1i-Jc6@erta+7@xn#7P}j8 z%djg#8yac|4>v|cSR!{hqa(tTqLTAdQwq}33NvyK7GxhP$T*arcVy4OGe?h{Ey+E! z#}el9Ls&j2mxQ6fenEg?B8p7HkmGEnf~k}P!uVx!mEWKQ4(`mw7st||F zVGdYvb&|y-BK~$TnR+6jovsna{bJ3!do_v1!#IvfB#}&sP zmLKaWJ=$4%tfTZ8Kv#QdNeA4^b*i)SbXOIC)|qaAuF5kW?*-5WNCrPueYUIiLTBwo z*s=l3hUGn%D|&%yEAC$@e^6V|Q(MsmuTt-R8cQyZ!fC2RqsUiu&5ypSQIW$2K5O13frGN_BM) zc6AMPb-nEDgx;&I^+m@QfOU62@9y;2ujc?@Jv}cTJsf`eWc1myvHs`7{m&jf>@C{2 zmm?CfL}I=|NykKF0T1|B%N??%(@D-G8R4jk6nY!oibmqqqrl6C+G=DXv6y>Fj#r(z+S=0o_-Xx}=G+5^z&mwUe0)xR*1mlOi75#Ry_Uk| z`mtC(;8w_H0OQ4w3J{%4WK;R+c}?#73?FY!@x z8_;-p^!G+O05D>(HN5o$fDzl7xB3q~k-jj=_s?ZO;`Lu|wDsDfV^x7cy09>9Sg1NA zSQi?q4-2(LMg%8B#bxeJ-;Y@(D2MUu35`5sn|vZYS$@49m*g(pch{$Wnw4| zMcJ5CsyAvKF1;)8z{zu?^RE^+euoQS@SgyWFhu{Szt%VZ`gedq!90$?0Y)(F-y9fZ z*k?$ue*@URcW}%uzI!pg*wAwSMCG-Emum7)o{!BdGK7Rm%o+>;%qlYkD~usBwG+iH ze7RXH)`&S`KAp#*vY8|*Wv8F_k6zoD9Jaw^a5#)khc?iuvzz29jK`z<$`NYaseLm& zH{AeWO_eX3%ZFPl-R+kr?$x;ORZg@5<5l5?s+s_7Ypxi*T?zmOFgA9(bmC6g*v$)& zUJby3odtT)aBiUC-1DnvpVpmvQgiZg)yYQ{C+?S*^pv0It~k|2Kns=%D`2VcZ_5RV z1px*=1wQNs-m4lOyK2sNUB1`@JLb!}tIL5rA%}or4{ED=f$C}=)Lnjf?aG7eupz#; z;mTI=kZ3r3+<5J2)Agr-TYz4T*Pg({joZ&#?f^Z#+X&=|>xJpP?f3fHJ3R6*%yaHZ zogW}T1v@x^{p#%4N~VWvU~qpJbYMMQ&%3*x0|a(;0gUzE?|u2?@mSyU@qxbKf#-eC z9(Q(i#V058G2m|$EU6egm{JfHiAM(yf&#TdsMR4_tw67nm`zH%LuGf$ZFWsypguGd zJgcd|-N5OTS}d@o-fTfNDxN|r(W}%pV4hT{Le7^-5t&r1R--BvPpJ|a3_`tLXf%M= zE9`d6WWw|Ysa_{kE3{6FGb+d&X4i*UbU_xi!=$#GLhf^V|8I@Ae;kcluI%eJI}BNjsPbGb%13BReH7 zDOO=n3FH!$)w=)KnU@nw5G0;PM5xW#<)7zPewkYV$Dbg~Jd0r~p3aWV8P8s^Ww<=H zMX#@PS=w4KgU4k^nkix|%in8e#F#On%m}lCJ(Hkec$MdsYn#)ngwNYra)V#Z^X=el z7l_IClPmAtz@;s}8=G4no?5-pdOs>XAGZe6g;FL;jMyf`5{kixBsSZZ$^8G=dJDIx z*8T5)Ki_k981!_{FvE0rHxtaz9fH{1jf#P&2pBXNC>D!WzpFVc2 z_~z*gm*UgYDQs2<4i!$ufLCt8hWs41`5z%$ehCZSgbD$g3`@sH3RxPf(H)<1y}WjC z{=?A97nsWj6xiw)Bn5G!iWz3o!z7NsX4n6kTZaRh5oPwrm1FMG3(?AIr_4M}c& zqFW#D)cbAvc)LBr=g#pZ?2J#@n;er9>&i&9CME0qiH5ioV?vrKF~bm-YK%`a_*2xL zSVeS<&}0G766$3nE-eU;LPyg02Bjg^=E;gNC0h9=X_QLLl8WR;W6{0p7n860=GKN_ z*a@KP6M~J;Gq1nQK;a1JyLSI+t#6x47b}oD88SP^m1@sDye)ZGk}cVmzCY`7c~NOy z_05Xv>!lA4ojsS3v&|8g;Es#W&dq=TOGyx`6yUZI7s0%?}ZPz5cGfU@i#+82HlQ_xW9cA#C5AEqz@jjGE?y+Si-Z^+AH(C}1bl)> zNR`WI8aY)XrRrofy^N}pMj91-yPjuKQ$$=m6WnuA0l>%{o>*s4S?vn50MS4$zwP+N zt0Qx-zXL2_0KwmZq5fQc_zg9N-bo>BV zAOJQo^Qx#c3uH4F&S=eGDm!|^P zok%Q=1Qb{V79D~L*&4bTg^6HAF+?JkR>`&Lr6wg`$!A0o!uYg^dj*H4p57g*hEc8; z6_F~0D~^Xs%y}fSWyTWb{COik^=H_kL&JJnct^68b+jQ~*rKjAevXA(}EMCcVzU4Q2p;8tVN`9{bv z*lXkQwe!%f7CgFg>G8EofMGy#HQg*~xmDDB`$khqF`!p->8+;nl7RMV3FOKnSsO6u z1yxsRU~R9u+gbCV6SkbM>8z{jgze{HW5}?^#@;sQMI7kr7<|#)-O<+4-29-TDm!lnS0aa+i4%#U z;)Af^LHKYCokZnx>0%*WB4$V=ESVgvG?6$;sRZ1TIUE{~$LRN~T`q|k$TEO8fE=0J ztX5idVx3Z`RxkxnVl0B)&-TCGs06>7moDK)4?u*{hkDT)F{7jW>bNTyg4o0WI+%FWp9ydVPp zmxwLDMQq&?5gdV!2*ZYLMF*+O>g{`S6EhQ|<6K-hpUh#fWwPvD2YLo!DHxok0`9}+ z*8}$d09c?_Yy)zzIPCA}`VO#w8e0Q8DIk>)h9N+LT^0krM9xDd*QL4Y|@T+yj+XQIQCLh}J%otbf`PI$Qo&Vw z3}T0hsSz_3QZkP%H=6D}d^Rxmc4!_ZO&~3rzDy$quBvly|Co9Ed1QI9ptQgkuTXh} z_C&+(6T8dm9z1>CTv&Q_`{5ne@7;XZ*3#JC{kXNew5op3@#B`*Sfj^lkBLc3Psz{E zPR&SGn+@P^m&O-SID9e}s8nI3O2RW~gf;`;tQA`ILW_>65HTctqtAc75w-0;0uH^Xpjk(f0tkXUWWG9A^_g2kE08(3vQJt9d^D! zFSgm_u4rINw@Vx2aiu22=Oo8w#>HpFWo%DN$x2E{jgL!BOv*_IIxO$t&iKq^t;0m+ zMFFXf;j-{TK32%bi3DIpbAk4P^dgEx6uFeHRnWA6UJ|-dMmNYKjY1+1_1d~fNwp3?JOkY49oZ=7qoex?zsuG5IYKo{WF z4};^95eo>f9{>wTvA~aTUw8(Y@^#_KwM)-#TzPim>a*f&z#4%X10*XcZYjOhQU)Vl zi0Zo2f)pu3y9dWqwpDHz((~|M2asQN)m;sBoeede4RG=@?0MJ#gIhm9_1%St=&*;d zhi@m)XZ7{)Ik2ZxV4}*y9)w=N;Hw@XGgca(^gwIudG-td>}5wA%;tFB+1JzE($w_q zY0IO>&n^{S*P1Oj8VN(eM-VZ=xUg^%21_Sl!A%m20Z2v@2}yh&iO**#6hfm>Znx{a zUW3o41vhODyUJ-ZdYu-Z!|Jo^oJO_HptKl-8U<4#AahwH4nU5WE|w7ae5ynS04CI% zIcgnWtCtx}TBpy_Z7*2p+=0iwV- zkz&~77k2m+iewfWMIib|c<3fb0+jV`T?z);!h@bs%E z9evvlo>n^jBn}T3Nyf5>5s~xvAQGZzbTOBwmg$|2 zxb$t6kJ`tUKaBu@z5X(Rbn*?j5H|{{kY>JtCiE}>2D@^6n|iZe-O`-6dz(EY-jwE3 zC%E*VrlU1{FD1kW~KY_2aZ&mWhY=*>)vPS1@_-|5cEcV%vO!u@u8$~J3a zrpoQ*>9rhWkvkjgYn>QvcBPBO4JvAfA=(Gz|TDDBZ6w8=mFbUi zJT_^xS>?7%?Rt?>W%BzDT`FpLKHfFCIxzoX9BF$9Lvb4b13>-rM+V1_7#M7b98h2g zz!qT)DL}G-3-RmZ;#$j#fqkdXfKxY0qY_yy5{F&tuq)ls##pZ>Bhi-;=S_`C$c|6V zNs3MK*<4nW!{YF{+{wO})L2KHN1|8aBB>E{3R%P_iG_HffG8E?#UiXg2g0BmvfkCoN0ZxDb%FXDGx{SO7U^ajynFbnaIAJxqPfXywwpPpNJ zIXras+IgW#f@0&tS$GVa%+?6yP8DCz;c6Hn150XR3iS+NVk1*&6U)pJx{w(`!iHnf z!C@i41^>DwVpBLiECLsb#RUmDq-dSgqnBCaJPs8jXA>Js3MQVE4pu^d!DbwFMWX;< z4aH;iMWeOXM(coGAFe5a0IR+}RCyHu>?MqZUFt6{2+*td&bgl3XP)0a-F@q9=gqV2 z#pheEpL=!{qU*_3gh`iA!Bj&iwl)kf3~mZf)m=IPEWoga!qfFgx5I`)*zgcytnlok zE9agTU2eH`t@U+KtWUO+SAgE0Ianiim;xZ&aNkqpVrqs>F*n^t!s!+PN1`ycq$P~ z!{Zr5422j%z--2ZhY+#&C?-WHAPWR6rBY-vNh}tb%?ehr*(9}?m3FH&*6mF2Sp9aL z%LH_oNT=edWHbSXh?Hj11;R+Fj4Tw=qyS(Bk--|J(sI>0nZaUkc?~X))M!-NtR|TK z#{`n!fyk!+7sDXuU<@n}y9%hWfP{j{`U^12apO_&^2K+f zOF&n>8~s7Azu5*65lJ_(v<7T^2|g1tZ|=?D)N<4F;q6Dxi!ClRgNdTya2yhjM?y#8 zBN*ri8aj*;9!`%yGtpQU8W)AZaWF(Kp3Egs_+$cyf{r5M`3yj>P&x(NJx9<1vEXz! z03a+Au#iOLPzYQyo=ZkC@ez?gfu;C8X?}0IS7lX54dS$%ls)@)Zr`1kkd>@3>*%~F z42^)M6Ny|F16J^Hu#prDlS&bBS!%h!tQVOLO1nKKExYPz*Z8Z?qbpE;1^VGmzx{J| z?Hkh47q+zcKAjm(uneTqjV!L7ynfvfAES$Rt74sEw^`-4IkRHpcW1=zN^|A-ecNJF zb|Qa$sq*-RR+HGSlRLEnorn-ciU6Fa z;L+f(Jc>w{Fd1?#Rl<%^hsfIqXTX zd(U00Z0TzseTDS$|2*;LGm;GX4QXYvA-@7?^wTi&Wn+x>MlwC3!yvto`sJl}|Au*e zZ~vHDS-)|=%9ao-vzk~crNCmAx||BPOYisCl6)~4iGWhx6kkGCLVQ}R!LFyVX$&?U z*5_(vVhuR`L}WH2oQT0k(wR~*9sKI0;3hyuQ^-j&30Wo~$;3n{Y+uIE$jJ&3NiL+S zz;Q8F$O{MBi$O&*Xm}=#%w|%#9Iiql)oUa=tt%n9>T%1=>rb<*Q1Z^e*`Yv$L4Ym6 z%CA3QQcag(bJKW#%ZMjv>@BB?hk4AY#keBn}lt!H43)x1fT43;%5k zc1t)p98H1%<1lbmwb-Rs7!`a8hmsxVcu`+6_UO(4pcm}hdlj;({>EqnOh+688(0~3 zeWa#nq#7pH4^>?osJQa-{-yr=m-@;JUX+}J^eQo$bX)O>*5XsGH_kK{oq2Zc z%;UmSj|xve0w}tCy0!p16H2q9u}SqguxoG z7d$Px)O_n&OUd z4cM`_th44Gu=eWvowX2P9re|1NTPg4b!A&cWqV~60N8VcUY&vH7eH4%%%MkASHp8y z?^W0Js0OUyU@wx*5s+U2*CH$q1KetC>_TdpyTQJx3Fabpz*LUmp6>ph=gmzm_4SY1 zTRSJmr#gV1-m_OIk|7+Zp0>#=I1O-8Rx7j1zGOsix{gjCq@mjhnQQYz@+oh1r^!74S|`8uOeYt%a2 z)|hyW%_&f8g&LjK=`zLmtbV`Q>(K*E=Cr77CY99)k(yM%#Cj#rWJ0ZqDS^oy zM1E8dxbG)oLa9JE5JRc>a2g&(#bLmeokKLqv@kYVp97uP2M!yv#QZvyN1{QL2_HCPP|9e_w>^!JF` z54OQ58sHy*5OgX6q^&`hBfuME*xJYvR96sk^Y9Hvm)?&oy&ps7YED4m^=@=|ZG8DX zqR$}tAiyS9+Fp$1A3n#?T2L$|I+8%(kq81JK}Z1SAc}>KV4=`a5jZxAz(r#@D7*ki zlTjjN6sCgCR7Bx<)DSu@m`)92FhZ%6Pznjpq!Tza6q6JYNyM_r6aj^=W2>BU0++aj z6#A1r&TWs4HhC>VjYw`(Y~Pc2{KS#n`*-?MV?}y73%YcX&MOKrPp{-^6-*`2cM`T< zEw)({u4ttzT4u9*)3?>O_KvQ68h!P9V45W?zkTxqV6Y)Qw2e`g@ZvkS_N{AVDs9hh zp+zfk>g8Um%xjZ-Ervv=ds|}6_GC|XY+PPK>Ymi3ohkmDR8K}0L|4X6um(h!I}-qG zhczij>rYgATq=)E6>W%;u_Gv`V0<`=il+-$V!c#ikVQ&3G!dJllnB+jU58Hhj?Kfk z7t(QK3I^)F%|OrL?^AC+fsb^f^8VSIR~~h>j4muqEw9hM{xH1)e*JI8XO~7MX9vfo z`bQ>S4vqB;j&%=?x4sy@S=q4b^u=vQ&g36HcI4F2gNOGdWuzD!(Mq#ZX?BZsHm*{~ zlqvWIy~5?x#rbpzG5Q3bImz!xiS=Z}d$UsVPoBB=tfysYxo`f%$g59dZ$3eTA&^PO z*QjBq1F@JUlQI-?xvv+b-49?1~Uh)fbLl8m8K!pS&rcN~lh4kLv{P|;W_76n8YI!M5cO!T?8XQmw3m2-Oo^;*%}Kr<$*wY`k*fN#UtL4@0olUN~NP=5S5Hv4*QqhXIDw z7XpFx!&~?x`EVnxqp|2*)6I*}iUGZ@Hs8JmthMAubIA>$x>`zaHs86~Qg*8a=>^q% zH{e3N-S*&4D@;CwDI3sW+zTmK4ot0t8S~X$bv4hSORxsOtfQ*3v$CqYrtW!teOG;5 zM_pY942CrTX!RgXq`=x;Uk8L*&%@g1(4Y4L%1xx)j(Wx@``TWlMht7y`(p52K6g8vp~( zUs@ku{xAwR5S)m}dp9!kW@zeld+&Jefs>I64N1r$@F{p60msLY#6&y~hvj0iJS?7v zCJWFsF`g{OlBGDNiptZ{`FbW-%f)di!88CcN+^RKN~46)$Y>@N6GcO@sNj3UMG-X7 zN>7?OCevc@s%S#WPwvE6Ym7@}HHdXenMu22|L%*IFC0B_*q;{1RtablHcKI7DnwkJ zf~!|@^vWomf@9POY$mxoT8(sekl3C6Z95rv_{98_lwE1Dxrx4< zbWc_;pqD3ek0*PlCwm7lcP4<@E_+I@AudJXcFP=QsYyxU(?Ut%LHIC26h)+ynq#bL zn+A?Q;Inv2xm0I9a;9*2_RS0=#8;$KRG`__*SWPXgY$15cD|@>ZR?wvpMLuRUT0yp z_s5x6;PKAMz~ zsx(^oN~74|lo+E0Iy*;g5EzXrk4qctRmZti{%B)@&y}9!%S!cc%iMFm;C|DK*5T#e zx%UXbK7ub0X~qZ(58$prku%*S@KVaYCx1YwBUw4m$JG zG^&itHYnt7n?BYH{?WK?Y4O=9@mWd0TuC0jT8N7z2$e!xw8aL_CEmjBRm#Jqn}@V4*)hiK6~Z%4VhU=;N$UpJVVJ60!i#w z>HJciU#$0w6%MZ0z-GzlWG*R!jNXC@-5e1fgu#Xqu%X282q3(u7&IA6W74$-O-gFg zv7?7C6uGK*WG}frGWgZ zyAF6ZQU`6g=6Zki)xN4LeHD;ly>~CZ06MJfLeHIZ9kuQ6z<>g~Q`B+?Dlh=BmeO0zh&7endImOSw_3|f+V0+IL)O;2NHFX{AaMiw54-EC zx@xK*v8v&`%Fddq_R5ORs>+_)+Me3FZdmnI(@|g34#0*~G{1OQ-&0=)o(x_BUfa`9 z^Wq`ESlx3(av{Ba11>~leFC%R0|x2U)CBBhYfFE7>p)v81Xy=hTl2H}`nvM6^5^ZH zBYlIDqZ2cev(KJ3#m2^?G5Da+u&rSsTTtQPhKVi_GL%w|UJK2j6PgVQhgIuwy3&$; z*%>CkS7tRx%sQFPtgxBoW<4;K-6%7wnR3`%DoO?O@}&;9+7qjdi8cD-EHQpyK){I% zMwpstG6@X^nnWzr>r5Uu0GQt40P;($QvqUucas`)0$RoiGACC(@ z8=6Lmpb|rf*sTN<9HvMo2IC_(p@QIiU_$67%+^ilEt^nVf-%A2*szqG?3bf60q_8T zA-GzbUIg3%`;W-xZwv#Q#lPEosErU8BEZ%7JOt0w5?~vI+{D61Wc|*tMey>ElK_y5 zaQ`2FBjOtzTLyY-4M1xQq1VLn$I+#AK&)FcgF+Ern-b2#pMZfx*$T zv1}3mSO|j@L?dn``ZTVzBgyDH+yf)w%wi_ z2ry69F2J+sjC=qvp~c40DkG&FOe8*x91+35QotXxU1jl`6&3}BOQ-Q!Vy#+bcAmd_ zXLNRTZuQH|8`!RQ4r)P|&_DV5()>52aAOkZYZ#=d( z7LZiKh3{p{KYWG@i2o2iw81dwIb3}EZT`*Y@rBpVUOc~Y@Amc+`*krkhMY|l(upDl zQzei&^oH1Idt!_?H8vqT*`FS7^E;ws92|=%(n>uEo|LSll+1**ti*)0IHg&IXA;m< z6iX0kvg!Q^-sH3-n=2YcApP{)mjBus^v}?+AR-yXU_~&P;fzQ$i-k1c;o>}GI_ijR!+B&BxtkI%mOeeTWIc?4Yng8)NPD1XrFdo|m)C8W_5 zlwzx2VNS<)1@@y>?EeDThd>e7(hG#K{dY za+z7cRxs%TI*~~UC*rrFQNd_T7!DUf#6*DqJuA}ca2`5*r2Ou^ipt7MmkO_4D{AlP zn4TPcH9a=>qJ6xvaC@_0@sOD?`;q!?ndj^*4v=Zw%F38>lWEs=WdSU4kdn zTnE-yb*&#@4AEhbUgZTnWf!^uz)H@v-ag%2e7f!C`DaDv0@YoQ3QvK})9Yu!3RzWj z9;ho|kBcD5o?JTv==H4l0x+-v)_m(?%k9g}w=RLT_4bvP+gF=zUu%L*<%@u|lmdD| z1qK0j=VlYsVYdOX+V7P@fR&fD-@VguuN3^)R#6IsS3rA#wY>_g7420I0s!l(sSa#l zJ#s@Gl9w1r?x^j8GOG?kuDYtT8V+bi`utY+AOM5n?D6C8&FLF~bw7p)^xcT|3K&e{ zXm0LrYXb^wu)S@lv!lPOv#q(guCDIZ&EmEvPsjU)CI&_(heoHzCi{9{9zT9UDwT$z zQ9&pafyJbW1q_8m0LPsh1ZIQSYE(Eax)_H$BgvW&t93`~y-txb z&mfOd2v`bnq+CXqD}-jN%;8c-yEQJ4$>%r5_!Ks~&}ii8bRv^UWHQ1mN4-`T?T{Mu zGNS>=IK9iRuo$H#gGdK+I@l^XQzng+N|`b_O(X(0bA%`+hCvOd5P}JqAVNeiF(QL%oj{s2tfQ=(*7cedb83u&W`Z)A1!lyx)&42g-Fz_bdskU(hz7~2E*I~{f0<7E8qnXJCUUs4x~Gh=B{D<3i{-44Z^z zlOiI))rJ^BMRVj)?gWE7L2Y(Q1QPmBtplU=?LB4n&n}lfICK5xR7uW!EH_T zMW@HcZcjG%86{c7Nz!+1)j*@VNDy7P3zj3D;i1GP1NFzQ72_Ox>&A$0EvAjO< z>H}cv?AjmGV14)d+*=5+1@QUbe4bf)KQg;`r@G=);l-xsJp+?)M$XIeg|@!&t9Kvn zI#aa$#I?Pr3(s6BJb2=OFEI`ROen*!1VpZkDboXhDeO*zFUA<>wIq0AvQs>n$r`uQ z5*v58;7WB%Ps`9k&-A;&#ZP0%Fqb(bF>eX61=xJ+>-S048wO$apCkWXdjD_m{Q0+^ zn|phA9N%qCvP+ymD+?o)QDku>flsALnNq7t<2FgnFwU&BY6Kc^YEiL~c#fRsh>MQT zOvuX5$lj5jmYI;05oeEeM9H}%Hbrib#U#4YvXZj4W!OE@5k%rozis|$^Ok=E2XDm_ zBIryUn@8jcfl-8Fs#pxirl?e6qd{sk%8e$K#iB5om1YxAW8ox9Qg%L|K5}R96>tk` zEI3{IUvU3jeg)>4K^NjG5)WJb3fASfUthfg0Q*mKslP|X0`&Tu0$ZP3Sf84G(>pwJ z@a)k@1qYbSq1F3MYOhM+k||tbwMSwCYRsh&>qRUvm%?J=DKr#;7>>o_$rPSg;`GEE zICSjRt+JBarsDGfT?G*J?B<0h#pj+BL04fT0x<9pINo# z3|#K6s|Ij-URMjO2hwZ9psTQlvyrCAh`VF`ckr)_aPGQ_M4y61GXs>_*>uE;hS6kc5w$}dEw*L0E-j4S6=JxvfhHF=^Jgu!C>**iw z?Vor#Fg`FiH#IdqIr;F>!-SM%0-cVJWY9%|NVx>aFSbs@HE4JSHQ%5FH^4X z>=vWnYfp$ZdEF|jNoLY1tp*^!M0y!lC1%P5REdx%0;h~ZWU;8=xH!KpDbWz)(?q)f z!=z@5)M}MlV7auyW&r@>sT2%}P-4^>Tu!~q37{y}YXw>jU#H`$)l8X;A(c>tLL!@u zjbwzu&Jma(JUWDg3Lyc8g>S+H{~WUU*O1M>g>C*NY}3!7zitZO9D)wP5%4?r9UPoo zoLmC(3ubCefz8VL^fK6doPufzDyk`Dh5kdRfehZsGd?x<=rfLMZ9R=K#juk1W0$Tv#1~MPO?i zIgJbNU@i0X>(2g(-2I100v zmciFWii{jQhq#Fl62Zh{`4kk76vie*u!uM=3CAO%SOheKh^7;8bRvUKk!iRpoq#|J z`DtuWHm@8Q<|snO+6uoM>-;X)T8(iImV>JQjcYVF(X8XONc{r+D8 zfX%*x56!*(GW%w|y?^k+&6{a^ce}GQqO(%0DL!)o5dOZn+?1s4Im!7uQg-i8-hCu_ z_wl4%M-hx202Z5fASP!&z?d^_o6_Zp(y9o;NTQHIld_}K0+Cswa;r@7Mx|51Q}d!E zoG6h{qBh;W_i*;rr-iqFEC4DXng-D~kYRJ+^T5vuRsh}K=OMs;2Mn8e^Z$1j|`baBtoeI~CB{ACk(d@PF_ zL65?)xHPF;YI7L;KBM2`NRIVoCaE1p6oaaAL>E11XdRg8omw4Q{5TGamtilfxwnB< zhkwDmhW7#cF1)@oY%ySg{6l#42U_j-mfoI}ovGSbo!qP97=?5-nRI)_fE5?Ecvl#JNS z+?4G6EO)FIO(FlZY19AOyyd@wg16v_XeNiq6;T8-npnYajzJ3v(F&9F2J?8pylSJmRpwqq5!uV zivYR`{;t3R;_OM$xu?Z|XNAqT3!6)z^RT(}29#d_U3a0i-i6M?fEN)k3|jfE*7DnJ z<+l-rm9^h1Yq@s^7)(U0zSmV((OqBpysq*EY}$(etQG-SeO*sI;1k$@AED}kQmm%B zy|MyW_eQj=wyUA0`(aJj!&=yQ{t0*`OC4Mmt!ybCi(}bN5`he#^+~dhDSz@ojxs6tC(UT zOgNH=C=xzRE{IZz*=mv8p|vJ>ln%4ZYSg(M(TQ=+c)!Wx&_x3XS+q`*(yEmhR4l2G zCgf9K7NS^eHXCF8j^yO%loT@%VIH5s>(_gH+Gw}RZdX{XTBk$pv~e{mhD^fKXe34; zzl`9Z!5yu%S)?YDK&xfLNnlcjNCFgC1eF5NwH1fjf{h3!pu(uIAU6mXx)l?;85O)K zV(V|=kYT@uZVHYFB2XyD&J>K!zMe$rHNE^{dIem6Ai$8kd9Z$(S@{eZhS=mXcr>8E zAb}>9-;XcDm=+)tlK8$+F$Oa=AjuF4E`8kaAHt_?#J3>K0;ofrkdxqjR@TRs0lEUA zuMfiu>wr}NW5aXrhUVXNkI$ArdQ#QY{9Cu6rq45 zU{WQKWD$)FF-9ay$W$4LCL{24G=YH*6c`r>FAH02<_QgKmV&;O82k%nGmb~bNho+J z4JDw2vx#^Cl`N*=q5!>!VI=$(bol1b%>*)rFJRIbxSyt8eHefBad_p!@X{Li5FLHP zcPkzgTs*U5cP;=JmQBR)s3a+ut`bLTqymdZ8DmxY%p#{o9%GbyjeN6$qmt(zI?*}2 zFuL+7FpeZpSv9i?yW%3zuRtf?KWE?nH4FWRFc|?l?qIiHIPZA+^?K*H;6Cz5s@0hYM)NPPa`*t{dLd563? zyDW*R0+Wd%<43B6Y^_veQK~#TYof)I8EuL)2=xLAk4j|H`EqS}^;38O1-fbti7>4p zhA`|uZ3frA&cFL(;qC8>@4hU&{XD(0`eJ&*BUsS47EGpW!Q(?2bgtfLjEi$5C-^dx?Qt$T zpScB#a;Ky{>>PMG{bq1}eQf3P%6p{c!(UTKi*gdS>^j{R*#I6fI@yxl}cqERQ0I;fzp*JTgkn;_Em99fz%8FvJvr zh98~i_NB#ol4D}i;{d~ww@)7WjX?{kHt}Pgvdj<_ko$> zNaBFAn5Smnw!awIerOK>7|$e^dkji15Mg?)S1YqirDloLC{q|!a;-)vQ|c|wl=R%w z=Pp*)J$&5Q{NO?L<;&Oh?>n5D3jQRIx3sjqT3K0HT$&skpBx>U9UGYLZJ%y!n0j(= zyx}I`(;#ek2&5HY7{C?KVNh=YgCv9Wf}_H~b0EX2ufiDEgG&L1^^_qB?9Tc2I~Uq+ zoo~Nc(0b!SBcRt+=rcqF*2N~U-T+nrFb25w>;{BZKzKo(LDwQMaPT4&UU#mx-nrIt z=Q{KmmfwWgh5%84ag_IN!&2pPm_6Tm_fGr$vetVgZ7^@X1Td`gepy$=U2qW63mKTw zR#Aq;z$&|HD!QvHx@#%}E=1@*tgm|k83qJdHQ-rSbyXL18$x?tUjyV9Ot`Ouu`#eg zyo>eS50SJ(BzwNMv9b5rGhjVW8UqRR&jSh!GOQUethWWMtpn|y{cYWyPdgqp0Du)# zmX!^4z5omx?dcoq8=M*$n;IFP9vxqpnVX!RscvX+`h650hbfgrDkV`Gso12F*)%$@ z*&OFE`5Z=%!{l*VVq8|A%jSm$Ho9o5)SwcoB$JxWa%ANLj2R3@kIUe8$xJ4`TE$f;*)m0> zNCFgCIE5Ta!odI-4#+P|1PzC#<55&>7zrJWM+ITSg0LZ5u_0S9!6E2i3M2Bum78Pp zZzq@5rj|ZTFMpg_`S^YFDS)48uvz{T@EJl6-XhH4K-3t_uozo@KLO?1dZ6|zkYOKi zNFsXfJNY2DV6F!gVap#UmOsJ`3~?d048vVxFmdA}1lZ#G(8BuA{JTK_s+py(@$t)d z?|9PV;&KyzT?n4NP!6CQBf4! zriieAY}x#eUw{5>)6e1IL1YTeL4 zPQfxL7#c~GULkcEj7ctSoL%j=sA7x)lbk6N?Kys?XKZN%fc@3yahTxuI|4A+%y;e` z>>BmGiPWDP0kCyMPrUyMyGgBnnSJ}YZ+fZxQPci{EBU9-rXASl$w{}TB*f-!%Q|>u z+u_sOj-K0gtRVZyg{;HpG7p|f-FpHd*|GTiqjB4h__pnJrRS>Mz9^lFZ`FvMdWF|$ zO0;>iJ@NVe==5l%T|wbfaFJxW(Ng=gGa$b<8Z*LZ9F#RkZS{X6HNp_C;IqvkaGhUW ze?B^Rq~M6otCl+WmIQTlvTgsVqiwxI0~1TlJtL=YlxH8gn04s<_G71a96N0G*-3m_ zSR^5oMhT%(LP(_FFqmI)Sh85E^T$S~CYfU#Y$-Q_gh5j&c?XZS56uiMya)7x><9FE z{X5Ls2*eAaKn$dq|FyIZYrp0p&%XCO-00i;-`^ufe=V$jnOJ&#@_K>Frx!-cXc`Vt z&Ln_4AQ_V`j}#dBQoBfQ7pNR!g;k_+$X$ui3E7G9*@^y)xP+XfM6gNo#b^2fz>>Bl zCFLY0WW{FXCvV%Gma{7jh%;A`8$-wa7QFe_(5=6Qhy8}Zp_yE&SVfoW*lLTw=nz>P z3Kwklp>#TR9*^1Yx5UPR^BF^<{}L36Au}%BC0fZ}*+g^o8LF4tiIP-eYd@`EC8^Mdx#hDK15f0MS1J}()NnGT`=@j*;xhY^#Vw)y4voVYOn%( zURx7bJE|%>5a9(kHB}vTl^u0e?e(=C4fP!`Hdfbxq$&RJAHvR3PoDHXgI+|Kg829` zFn}@e&b&YU@Ozoc}ivuC)wccka#Xy3s2 z;PAxI$n41Y+{D!M)XdD>LUqGKcVYriKSGm1W;bhHc5|%Tp5Qk79gcX9BhF)V*)=w^ z-f6S?+|KwIo8PN68~I8|FOdphOexW6rADL7Vo^s&N2jJHZp-r|r@0f;Jjt0csoCD- zjF{vMcT$QoKG7Z*XYu&}&x~%bKH4QWnT1-7%4XI@J5@HDNT=he)I6n{ArN60k)b34 zpjQ|f4~>F@0At`V3_=72_EHKVph5`|;bb(5iouex0*T~SdDX-s?Bt6;>obsCKx2Uw z*?a=9nSS+o3NmaX?iFw+PAq?z0GerK9juefh#@J02(Py{0Zh| zEUW=^jV?oO0UKEe$S(w73+sckYyC5C`ezrP^*%pw?QC>fv^hazN>uu``-`gYRy4KL zwDok3Ew&FYJ?1r>R*KC6gqjl0+l|hZuqn`z2({KR0js=P&>K_1Axf zgl?u#@jp$z2Gsd51_%rR_P(mIj}UYPeP< z&|Zclmnqq8PIjt2dY)Fg_vD$*(YZlzAzuCnzQM^ipQa#|z9HS{k+S8rKNr>kB#@C{ zuoFjs8L-1IVA%BP=a*B<<&WErUoSm)`Np2}m$Hr=N!Ya~b>E@9W9RdZU)p}6F!#ij z+!KX4$1Y|azL0U?Y}($FDSM6sh9&Mi=Fi>l&E928&X7erq%M;t&S^|`Invz$fVneV z3cH*kq7ztDqs!gc(YN^S>%#kQup$js1S3VkaQ*kf`u_(RhP1qbsu8HPx4%y;y}Ev{ z*x*yk96XgG;eN*`6%@M(y(BZ$KG2wJ3MXZWc7^Boyp~Wq= zdlb=Ll^eXfM;{Yoj`b^Tb_$2PB?A4+)(~oxpyWXVxQmu%*{u0KFDoeFo^7fAtw+Z0W=7!n>LIcN4R3 zp0@YwJaSm$u&SLFV~kzrGAeBvkyg%;iS04|!^h9xDygoiYptnidGO%zg9i=e<&|g8 zT-=er#}CMslv-I)wY;>ly!>i*c4ls7a%yyNV&LWEVDH4sj?wmqW6vtaAD53d+!{ra z=6?Vzkgov@NHGAg9~m4_q5-x+p>}nkvJe0aD6p6JU@l@$*@dnkK3Nm2Wq;?|O)W+c_Hwd__K zkXaiODUq382*6-t!-|rQs`9o9z^C%gs=I9uO51?sD!<)Y3SM%j<3Tw97~(|)7`q3p zvI3~9p4#f(+Ugf|H9bIn)zyLZIUreWO?P!w>x2946%U{YtA!c%?T}tIfL@&s8z95# z>o@w-BLIUL9KfD7_B6sdmVfsm0zn1<*4NxT(AL)9(gJLtrTt}7Tic_iCk>CUUATuI5+*jQ_9 zjK$}%#<D#ww@7lF(*WT=1d-C?|+kNoxu7d}&cJ55c&GV+D zS^aU*$tkfJSurW;?u0aVQo1)a+n<&jm!9iO%XTNFMJFUX;u4*4NfvLM+3PoXVsy@E zt<$cwS|xfTjV~l~xOBb%oa13cQZVefjt(VaBPfJ$3JyibV;FcK!oo?|AfUWpQ;RiiO*>ER zF(v4=eyJf*Y)+Gy(lq}3q(g-l>$-ZI2WK1mXKLH~wjVkXB@vU^kr*m*GbZF$d=OSh z4cxA-h7;T^9g+7$yb13>t(eM_Z>fW>cXjg zNA`IVd<;<(I*LM-^F&UA+-H}1tx~T^;kW3MqFveX3A<9`^5U%V&g}jBYubCBPcIKI zz8eEyAFK$5-RP0}WjH(qunK_$kT^&uq#xB_5MbcK{i%0qsr*U%xss~m*Gi9Ey}kF+ zwe4pLcAUPr_k7Ww^TqjRigHh1-+rcO+ledLM=xg|x{$u_WZK?iDSMA5?LHd6;~)T7 zbVjZ@DN!E}$!gE=d2;;TYlC##{Y<>lCP<&&25y$5ZnIkDUK zChgb-?)xGl2_bZJ2n`cVAqJ62kYO}xSR|du=c}V#D!ZA)p@P3u6q!Kdi>{W{0AW7< z>h}rQ`S6b!m^|`zW)-PRgQI6)+1%d&wh%~~-vHQuBcs2-3cGJ0YV6O&cb^}2HQO>= zYzvQ}<+#%C=l$HdIEnaJnH^ycAQu9+XcW31u*q*a5H+%24+ylA5 zGIymVIR&em>`NN0K9Nv3;Uvh3LQ^Ni=e9I-LuB>XhclYtV@`t5& zYOVmZKufGUnVOjx9UbT$>w7WL*FE{7 zb)o|S*u(OPhSJe`$guB93vvrFn7aW)7!v)0#}OanRlqQy!1^jK2fT=V_b&w!5xYvx zKQF!5edl8PEeNo-+l8&SE(3Z61Q{>@Ga$v9VMq*#ia{6RrRLk0+U@{`U2VT}qoedz z`<+|Ow{NtT-UL8uyH^6NC7`;#8_-*wm1P|jWe{5TO55+3b=)uSyk7mC!_Rw1pT7hOY@}}hFl?%Scy4HHc4T61Yt$6t zkDf`*-<^N(@R4)pFBBEuD7#-)QCnI6q~`Inx~I(zPr-Y));(=`_^kC|W6Q&*EmaLq z$||a_mXw~leC5FDv*3k!`wnL0?@7wdPtM($p1(i;z{x#F&*knv3Vw{u$c@j*2g=Nq znC45%^rdIIQj%@4aVo2wE)W59p z18f5ig;TLOCXvV>Y0T!D##RJiFx-ViykO94Ly83)fZ%xG;4*@@<|lI) z5=SmXv&cj-i(^*Ed{#}8%bek}Wye|4Vj#eFr={*oi_ML5roDsOfS9hPkzWZF!?sGSHoGIRM_SW_@H@2O+mUFx?=kWRTea91b?(^p4xwGL(|=cB$gE4GJ;&1PZ;OCi$corTHQ3^-Pm3#`7gm0s zS^7M+@L^(h?d8bQV?8*$>s%~Z-xr>y(C z|8xH5oYp>4ABovZWb_f349?`x$UKWZ!(z=a8Iyosbm|xZ&wTo*wW0QOMq18=b611{ z$@cc{=GNxI;)2CuT3Rq!^ctf|Zc@lB3b92xYvFM%(+mp)7{xS7HjfgZfgVz-SH}S- z{V2x3Lk$N77&IcPSY0Z17XTQLU+M#fK|7+9_)J20CLul-6P^q4(5wOIHILBi4}ihw ze>OJ=wSN!d&ZizQ~U1SsK9>pa3R39^n^ zI6NX8hSqs>SRyQaA@q_WyH}*h{6pBk3`qtHV5pBnJ~JanF*Y{`LNU*i&Cbn@O^l6> zb$9gu1*Q>6^9y8`_06wb823T4WlN9E8d!Q|hHk@W*R;V)=|%I>8qX zPtUW)XDCDCgN&g*DyNS+L}E=)hi34M(JmtESs&$TANkJn?xq`eYA;-?I)AzH+{N-Bh;_)Zh8Ct^7aP^Fr@VLH$eu`1z!M!+R{3Rcd+XjCL7B)rHUtJcQXrAi1Ci1S+sMmaVs@%{Ru)eQ-qN7t9|T*52q=U+a){ zUy~wf-PLnxmFYS4sRgIg@*7jKP9hJLlxB&P$dB*s8MEiKhc>Bir1SJQAr3Hni2L~htIAff@KJnclr{g}c{sCzL zn2f-%{K)9ygy^EASX@d1h~lK!;`rd~(4fqafRv!X@R)f9`a2dZH7?!!_L3Cn+=>o19#Y%{^V3d%7&Ewj`?# zhixh^I$K)URG3|zS9kT&0JyPHFR52wo44NqfGxwZhrdB+z4&GMC9Hm}?*Dd>Lfl6K zGHmtbFH5^0WR^9;*v#$D-qUyPRb9PSb@h7vt$XJlv|W7EbLnya`Gu5MCN3LWu!-BXJ^$kPy+l;X}*6rC=?ohd0jUs`sdEU!8@ zGBq+RG2AyisP6nF&BEsDtIuouXipV*1Mn@ODNV%J_vK9jhC!pjN9)es0F#1iEGayd zU)+Q(K7&1X=PZ#;ofS)mr)R6rHhG0(oPxaVFrFs^JmK;YA6MsKU(d)eWV?BkYhaMQ zudfRx$SWc|ATH8BI>a|J&?_{+Ex0CZ&UWUaX*cx@5Q`QkHD14ABuiPfD?tJ@#fc0Qr8zVp$#@pfV5m1*gfQoke9ZV2So8NO*~OvV@% zvbn;sN$KRAYKo^CoszJI=4i|r0%3$g8l_Xm2m4t7VE6&ngL}=F&R;7kD5afFQzh*KnVP)uU$i z={N)Wp+Q8M4eEyn0Nc>eaQal79u=!Y&42;urJ%RU$Io0rbcOb!o^jc_|!+qg8QMkqB%6egLdec!#bv#9 z$-JW1Ei#8@iHupsn2b58U`;8wJoT7R%U3Td&70b#U4!+7VF_-)(k^c5Eo(-AH1nFy zw5l~OgClaX1gGs778J&1g=s|rIA>TEsLa!1&6G$pDb!5x)#E(X(5#FxEhbO!dYGdv zgY-vTga;ji_uKlfKWx3--0`5Jx9Qrg^mW7dIC#4NT%B;W zJM4Jkkb~`U7e`xnCwos92QOC#Z@0qA>Uo(CRZED#`he2QifjtB!g#l|^KNMyiZDc1 zAx=f8)!r{|ATZm6A`B4A0{H~dW!VB^3kF&qL%#|uWB_9Tbz5&yHD-b$48%L*#yi6* zq!$F(+8gc4OYm{AC#Q=V%Q9-yavRg~8))uXnd%xhy8cx#kqDQGAt zIfE-dUzS~w1>{$7e28bTf78{Q;3u*2^3&Qr>;QvVL;%r)fDfUS1APd|`qSFWPpf-x zgc|jwd)Klnv$3bJ4L42^I25JPWU_9EREA5pfAonAatidY4|G2n;C{l(&C%c6D;|z zuNZ%)5Lfpo&+wec#Ny=Ky27eU^>vpUE6>zaovk@_<@Bj5ja6r>N*YSQ@yv~LSMFW| z^ty8Y8jxR?f4tgosVS{6&n-Cckh9~rj&|R<*nRJ64^JE%4mrEJg+zp<<%Xu`1|?GB*xG(?Yje!r{@6)}^13r>^ZN46hvnTb9{NA!4mO#p!@~t&OcDT?m;-bbL&yTb6tWqTA(jlP zE+D_?QZ`-8q>GsFOQ<*#aA5ur+USQlBJPk7+73s>Bj6JskqmRC+z}LDvcH>!2rU~j z6k0Hgh&U3zbb|qODd*?Wkj&0-S*)?K5yHU0D4nH60EU?9RThP2L8)DYp8-(;>{$JG~l>yCP1OSP~mGpvcV%RGf;LZllJsyTcW zYfjFZS1_g|tdMNLTMZtUh}ozoIguxG=piKfj{9q^>Tbs30Oa z!Nt!9DS$cIdAQkmy4ZO*Ie5Cb`gvD3ofE0eXvPnq3ym+8un$(K#@<;Wz}{Lx0I>xC z25VYhDq$OtUI&3>5dpM-O8FzuFZ}^F6ACuOxCp%)n?Q%XgQYKoUPc)E`qh{Eso08Z8ZATr_HHlIOiZQ-w40Z#1OwIUuTSq zosYwDciUs0Ct+d5#~v`uA<)S_(7_&qkSy5E#>3%jn?pc>`6pn)bED!*Q{qZe<4ZH+ zN;5-qXl^tRWU(a)5&2Q!xsjon0ANvWfu1&Q_IAE5e}SI`vcP@;4!3=4-q;hV zRLmhpdw0j>TetGcN<2dRodP{EDUnfmNfFucq1kcK*tEDJ$grFSpu_T0D^kdjgz@s{rM-9f;nDQ6LjR<2OiDy( zW^5cbJ*gO*T~Sqe@n-$4rwuu8zlLM&&0(V^d-ZQDZe5lGC{Ivt`-kS&^v`2*3ia+<9nTe+~18h!YYOTj03% z<~LYy+W-9v%#eB*+tTT zzSN-Uq4iZ?u8hYe0MYG}6yOo#;}+!&!a2g-BgWS^DL5)G0Sj1lp}y)|T}4wBU|2;{ zH4t8P7wZ7Mu0OnW{lTrP_io&Hct-o!!EAh+BmNdNpwCPXalZ{JEmX{i^;SZGHr^*nWNGs z2Kv~8{cIwB2xu<`ZT#W=_NK;51=y0z^qkg~cA&TxElbN*>(Zh{uT>kgDziaj)F^dw ziC!u)NQGtz&%_%A05gvh%_I1Qae{el&@|eQnl+HjVZWN$3mEoCPZ&TKs=#!dUVtu0 zFGi=F2I-}s!Fe2r0&5ln5k`b5L}=XTZ=LCFnSp7;=K_2?2=JtU&@Lpj@dn$_o@JoG z0KEjHE(kDUFP}IdAmLGf2}pzU1f(hk0EQ3?-LQP{5Vm{~B@CjBL6Fb}1Vs2Jh^YNS z22lV3MiH@TLKX-*2+QT#EfY+( zUMz_+t)hP;f~!9P44Mz$0(3!} z#`YT^#jIOzm$wng1%37~qqqna6?%NL@F#AWkx~%|e`8*UQ#?%TuaHGw+e(vlgg<}o-o=zM0I>L+x@2r)R`s>Y$N1L%5%v=Tp|GjJ zBL-s^VDA{{EjgWV&h|X+#AG68(%wnjFVHai(LSeVs`%a_TcF2e8lO* z5trkbq~OHTv^ZQ^bU|uNae8cVW_VsoWI;+)VRAxwT3StJd`V&$xFb&q^^XZW?qc`V zu|wb6+x%q_d>n8Q@TZ0C_uwBa)tUQ<_(5WCPk;9h_wN-}R``U6IAXlrBLlqSg1ure z05EZd8HpwN3Ao(sdR%T}QA$NxdPP?K^$Y!M?!0zEyZX`$)GjiJ7MVr+3lc~T3;+WT zLPY)a2tokWUi`GU^LCIulva`ly#|HJaYfMdoLyd2R0A%o7i%uxslV|QV66J;kGQjE zvulgft8z1{vDvjnnc!H5&8aWStSL?^&ch^xW8y;MvJ+FVDH+9Sc@??&Re5PesSzn* zL2-dzApy6av{`pPKxqYW1p$WA3u?jrU)SEC0E0RV0oX6d6vt26<@E<0ZI|xd<_hLC z)@953&f+@wU%sbb-0Y<=QVR>bqe2}p9uEF)E&-nYQDG4&NlAHm+2s|v6;)ZKWvN9) zsl`PZ<)v8_<>@83*z63yh;VyPukyz8a^o5R*dn^S5-2b@LFyAiFLcu*9JU154*`aF zI9Cx(21{Rm01SD!@$#2VxQ^tf)xCEsdoQ%s#m7C(aRsr#8KHq`!JcuxE>WIt(O%Ax z9!?SN;GthqP#iWXud1M=p}e%AqO7q3aUq^MbK}g7N4M^`KKkLwoy&KwoxO3fys0`m zGr`{1^@yv(F%Q?{o^D4xoDO-|9rm<4;$?r--TtVX(=m6~WA5$%UEeu6e&^tL$ld|O zVFyPLM;)DxJ2~38I6HWHUAz0(vi@pi4}AWg|Ky%Wyoiw2pW!sD9l)@kS9gA1+4*^C zWT&p0Pu7@yMvdI4w+PpT&74HNUm;Ry|qF6hBecHt*_2$Mbd z2`W>-8J3QWD<%QkSo}f!$RL3`I5=cr^1cVNNF(C>wJX!{W7DOSrTS)E^lKKU(00xtXunhL47$%|unMLAJ z@NfnPWspzBgNGsp2^=MKf{acOQ+lU+o=tW)3&_191YmFy2b&521{=RPa1w`z!-RH3 z8Wql+q7r8*L>`@n7&nH5Y&JS~129ZD%mx4xjSRuj8>2(WBo3}*gnN)b90M_-7@w4l zBVNR*1Nns-5v9;r57Bin4sjlcFaR*V7yyjTo}HN__V>(>jB14vAi@B{j53uOQD93d z-Ll5Cq%|*TEepCuvtdc6wGNF4pLMYwbuinB6TJ-H07pU`Rnw-7%vm#I);ua+8xdML z{Dm>8by{W>YSt9yZNoAgyaeYnAtfSY1jcJ1Zr~6Dq?;I^0_`A%m8YE#OFJLTo6ujb zU3n$5Y|m+yCl!`)xoKFWXY;kR898-YN*WX5hvo)Y6TNh9Cz0MmVUmZ&#(B~Sft)%# zb+@^rq_Qp~BHF{x-`&^C&DX=t&&LB3c;o(Kp!UA}rSFjf7z7ni8mRCBS8S-6w%;u6 zyhGI(h(%;2@8UKJr}xm)umz3fkX~DF&ERC`-2!+S{1X5LAqXMBw)P>tzzYy)nKz-E z4?TgMVbECr8mLbL0GM@0WnPsSjkjCxC*$Js8&ZmzvQOP?Xd!kCa9OQ%a`E|7F3|z@ zVJ?p0&i27h4k4~CVV+*me(n)oPC?Em{T*z4YysPxL!2E09UU=F4gn5M7)K{?3~+Mt zcd_$6=@RS`m>LnDl@ME)nN*q|Ta+D*I!$9J|c5+4`!&NJ1r3;AhPB) z^mHH@Jm{EMe(@RnR3=5@3wN&N*A?bA7H8KNW>;g2YRfB5HI&t#t~_(K=JJEuYY(fh z{!n)QEViyBy)q}gGB2kNhixp!ov$dlP=;+R%&5qT%8U(32#?K3h)RhLi1ZJR3kKRN zIwK-HB?J@W?;VDD)X@iiek^9-9!D9E3>?@=L~e#{0R)`T&4A zN4k5&`S>RXhG#~m7G>mD6yh4n%bTj|FE^gPe)hthOShlhyZij%#UHO>t4gBM5<;1t&I37Rlj}XD3@%7k5nH!?vD<&3!l`dG8Z^^JuddoHl~c>(x(4^YU-jt)I**?=+?t z^Ww!RzG+%uo)%cZaf)Xeo7Dk&f#b-uhCL={jmj8Y$pCq}t%uvuH{M5_##85LLn79Q zlrth78kLSsDn`epMA8_EGEQeq;|Dl2>NttWZEq!XwiBD5^jy05V^ziJyc}F*McwT5 zoXKD|X!VO0Xv)^>)hNJBS{0CA8nI9(5$HrbgK*l20L(Z+Fi!w<5ly3b^BCSZ(vP$% z!zM5#qf5o;ft4^I$PoF3rVs(Z4D22aqeDq=lT%v(z?7608SyCyDdDMv@Jxh%CLDM! z8f+CE;Hphb>_8a?eTf5*Z4h9Hj{|5g5D;Lb9s#jWKpfzIVVDRF3M_5`u?5ft0Y=56 zi9`{TEMkzQ42qmV5|jH7srMWp5GELrUCcr@6k(DfrkKNo02^ZR=oB7>#3K{tsYEEh zU}|v)2rLl?(OyFwA%`VkvjjtI(J%`cJ;joaasbA}BhZ2fp*02qDlnuH28SX_;ShSn zK#wpC08BMMe^C8`SwuciV1mi1Q8s6GYJ!UI7fg(+1!5IntP{x$QY8SGMWI<#Y5~B2 z0$b3SExJV$3b0Y(lXlvj7V@)xZU4r?F7q_&LfyL3vS(a-{2p5)cc> zkN1{sz@GQ!%@39Xh9MZ+dS^z?7f|iB{T^*219HW*_1d_x4_<8AK;xZh6OmsVkX~@I z;_6G|3fz9FU*6SN*5~BP#_MNOaq)$x(rPZ3wi4P{GZTanZsVQnJ}IG&Q68?bp02ST z&e5K((S9!BJ|2<2-tqqaNrAzsA-=J2FN8~|lLN-yInc=^(9tc}34}|KlUJC#SC|_R zZ6|yjPx^acl44>DbK;8gBJ$G1a*_ixqXW_-LULj;>5;yPVa}nxhn-J;ef-E@kDvJK z3826Ly>`K80b6Lz>pKryn@g&2-M#GuQvdA-cT4JPqO;NilHvl>;({|{eG)=^6G9_$ zk`jt?6L5J+Wx1Jkg_*Va1&!s`pFCmBilvL2hRwH&yKoI7TvG!3nGUL7@c8OCIMZ(* zZ6Zcei+~U>KP_&(o)GY_KDbeFp&EA<&Y>@EEGw!lDXK0ls4OdOXsSGSv-0BY%8S>_ z&z`}Zs?4b_OfAbvDa*-ez?EF6sJ>QRb`FPYEGnoiicE=d3-ECEcD8jpY47FW9qAL9 z5uSvNjZTjUON#EHFyMTU7e6iU|GKjO^ZNcT>#!6DwBR3sSQF6{B7hvwYI~owOY79p zQISSx-GVn|@a}B=#lqU2(zG%_A3AmIR_WP`#r5@NbrrbEqQuO^;JAq7{G6iN8f-;X zW@%{(t{@4A&8#ZTt*MO9&kcx)a`yGHbM?CQphaihvmnE_tb3mUz0iRm5M40I_A`<^ zM0}D+93&54{<`%VdiLP#C3J#2#28dyTR^w%epuXmt6AJ=9UM$9$_!2k@k#Up01HS7 z^h*L5^KprCcZ>GKqy&a#ge759@+*tWPS;eQuLl5Yymkis2yQ&SedgxHtWvCJ2*%#W z&DO{HsE6IRPRIW0aOAMN?Qt*r<35gdL9WhWo^DaTUa=VGV6WqDj>nvxk2$*@b8)qC zbw|+UdeqVRu)X6kCzoSR&L`bHFj3LnBnC{0AS1ir7L`4?Hf8{A6GmZ@W zxf=}T2^n1gU`8%MKh&>9s$V)}r=^zJsiJo%XzfZ`tD4%XMC6wo$S(q9m>AzY-}_Y1 z*CNKZ3GuD`fff+ZSdVY#Biajq4e(5WXfGtd2>@6x2tKiwPZ|&+!b?CN6p;IclmRhy zP(&FJPzQO`0Rf!=xFw{*@|cuPRY$j47r@ob1Z0;ARmlEg<=`;* zOxiq+JWnMHXcRu30)6u=Hjl{?L57Wrxm?lkuz=0sF!av?&vP+Qgb)q)r1*R4${mL_G3qax!lPcy_*vSuXo zNj`IePaU4_C3BwlQ69GS-g(k`^X}s-H-BtAcL7^c5*42q5trCGKv~#;(=niX9?s+g z%IS?|8%i#SCZuLN(8I4_?BzrfU> zpo|c&L|*`~fMiTiYG6c0B(@q?cd4oVN>k&tv*&-jeC7W2hAXF&u^DbbzP4U2_I~b8 zfnN3)HyamyCJ_i^t%A>MyN0t$@6 z648gGjA0ppHrLfZ($UT8=%ROY(a7{kCVP%Xn<5cM@OVyFC#9>M^z?C0W5ZQkQ9Zcj zsjWUc!kt=Q+cAS{phjyn>MRQegHfYa%T-FLPAM}eq#CI}EuK>gXSKpvJ%3U^!#0eQ zOk;x!FnLHYk3ka$x;6!!i>QWq!Y**s!7><7X2>i=Hl&x9(V=0qtElZNN*f3zsRb|$ z9OZ;&DF~QFgzd~?LYrXlIjBxeF0Mif)<5*ks=qzD1wU{GL-7=sFH zU`(nA>1YBny>qJ7mSYnjI zfl`oO2*4~#{i51r(So1blF7KF(_1Nw=^q~tJZPgl=wP-JX1b~TUWNpqtDVAcCCvjV zMw!q9H8wD;>}E;GV|uP|bwaku6)sOIHirb3A--u=xh7wDp|-x#uf7I1_wcT3=OdEM zLyA&6A6Irit?YbS-bMqlTI_xV6oGc`?f2T%SMr4&k#2onxiq7&Ov#K>GQ+q~Gs07{ zr-a0zsR8C_H;MVIYgf_T8~^}-07*naRN!Gt=iMi*x9&f_di!qOne)}B&of467B^on zZoNShl?%H-U_i+QB^0dkAbaEi*uZc4_D6tE2r%S%6kw2BFcXMMuD6Is51zvnzMF3s zpnUurz!1|sq}>*E{i3y9?( z?*{~#f4r}6oDX=2NeqYpoJ)yx!gzmYXZQ69+pkZa{Labgh@0CXSC^xnE_VK&9$^?K zjL-K@_FvoBfY*F$WBb*y_F`rEx$^RhHRTtoi<(Nas`4@_ z^0F!lvntE;>(3OPI-gTr9h;XCm6M!UoSj)+gl($GKV6nylb;8)S7Tvbb#79AiW|n` zsN=Ci_D2sp96RiE+%w!KwI~H!otufvsyuyaoF`j;@ev)34~}1GihAfFC*Z>`07K>w z0)Txotgg2c`yO?)j`PI2rELq`{021jwn%4b?kCq?y;E`FM#H7+_a8sH|KLvTsrn3T zUM8+6r?MocsyM&4IHe>jC_OSTEiy7E(K|BmxU1s{7iZ7F(6#}pY5g_$tSoT(C(IwN z!2BWnOW}+q=-@yzgn#xN!g)&vlMnv{Ff@RO0l!o$JMWa1k#5;AMEZJ>fs#j=^E+d85iIkhw+KUfSV>84~OsUPJCx;^S$j! z@E_}&<2K)&u>Ib~{+I)hUoJ&mL9x5@uSvho5z@?>r(r zXr_Pw^a8?*F(m6FPD8K}C-CH{?p{u37o&f0h)5YHQzwX|(f&SmdmHi5!>(sf`z~L+ z3(%FHTbY+r(bGlV-rV2bda2VHHCm0qpfMS>daY8el&a(s05H8w0_2xUIIZMQX@xTe z{)Bdlp&ug|M+eMfgGdbw>M$U}0KI??gHz|(u+XJsbZXcK{mWWbm!93NXLahB9eSX{ z7@aCgyPDFjBDE^;&GNyg(!nP{fyv0NQW6X?p;bg^6CoG_3=<;SOEB057zP1Gga8we zAh*zfH85g71Q?kBLV&b_@yUbGi-=Uk1W=aY!IvadfGfbT0WqS(WK4>P3XukQ1_YUe zL4nhx;D~x6&|5+}Rm7l5Sab;ok_jN{&mmw!54~_`2nL5Eg`UG5?!*zU zXk-ML#DSE)P=KM94b;SeoG7MXC}Br4Y-&bK90J+Q+ysL?!5WtEc!KE(C2vkE6d>{o znmEjI05IKxQV#&Oq%kk)mzGS+W|LK`UFsR&+tl#}7~(#*4CpZ8 zsAh0j*~64{(uFOgxlX!>HfSbe2l*p?dC#QHUfVMC~0osn6_ z1RB<~h%!1y9GdK>aN7HbPdj@bJa7B)ar5;LYl6LBXwDGL zJtDddNkRf8XV?G|Z6CHa1AwjU8J4$+?4go|imUgp4bP5=G@3c7a+ELpv7;@mDnGI$ zKCC1zqAWSIAi+N?Dj+jDC^N=4DGXo&h_KkAgs`kI&uA}?2v5%_sJ-07T)lt@Ly#5_ z8{i*}aq#u{*1`U}llI{D7aYH_ISEMim5uE`oUr-o#PP3>9sTOa(XWpk|Hj4!Tx7n3 zmztB`*q(%wIFNt*2j$ZGus|Txn{+E{Gjh?BzLxr%XEUm?v54MEz-8q%mXuzq1I);% zDTvBX_Ke0PVABBM3Qv{QTt54_4?ir_NS9t3wm!hFCO8NY_BO$pOenyR(eG#=txfbrc{MaUb^eDlzWE0&!?4@1*gUZ zro|?g`YR+i%!Z2>`3mC-Y!-&g$zZ+?sdXLHmR|q9`TF+_bT$W+U%$W^@UK3vzWAtLUAfnO zKfNfe>P+3poWQ*H%DA$tTi%&f8t=9bR9?JYdKLie=A-83AMf0*KGjfMQOw>S z_VVulhB6Gr*a5%}%1A)B{k;0(r`5er)}1#Jz4=LRcR@p0cvh58g1=8SKcvyBsaC%5+Mo2_vL{>$9X;V$j#nTt=+&J^Y<<#OV=OC|>zOGKezRsb3&S8FT zk^UahfgUkIK1mTl8F4-dVXn~uuF)9JxM07e2>+z;z~m^uxNx@+jJ>xP#FeAVVLRu; zwobOrUT%INA+agN)r}W!Js6;ksw~@UFMi$H|EHa||FX6JpA1X;e3^9)7P`#ivj*;@ z7SM|`27^0gWR9qrTs3o8P2Bc*2(G`7$H(;@WhFZPG)c45Ro*|+s^?2W3pxk2Znlk=*>_2u3WlTUV1t!qXa}v z)p?OXy94fam)2EEwOXx&wnMGjpp@%m5}jNOc&yT6bX_^tD z8RidR7I9&MgeDRV+yQ_wpu>O&Q!(2W^mb&tZ#U3h26m5;(*ppep|=5s0f1>}ojQ7# zmfE2qx2j3c6~t$9;xh%QRZeb~Q#zz%AhtS0q;>(ZO-Mw%8%XsF8am+Ys7~1aMd;$; zyXNs-JSe~VVEzyw3UUjf7r+vYAfgcgnNZaQ#2FM`RHBqZ6qDh(j7}Cqo)JahNGD4f zR4JVzJlL^9640qapuHG05i(6mz=8p|HP2uI?S+Pr!v^vTh%g~s00x(>praAd*}i`O z3{_x%h%ksNAhYJ?fcApU!>K9h#3cBVVroi>4C8sGZO2y}C z4gdyi^k%tsL19=>8lVExm=|H{cL@d9vsU7@JDmVv_giWAo->~Ij@@k=dWN3?+yV@v zO&Njw>gUM&hvd)6^Bpt+endSiSZ2*HF=rPT(+mA9C1u=5pEA)V4IPv@=8R6L-x2Az z^=m+ye6;L*T7t83KEold;6ew~7^K%vtJ^;IgxsJ#LX;e}>>t#wbiuq!uj zi}mZX3d@*CJ2Wo?7^9EQ4$wH=Bud8szNNG0-m{jL-T{Hiw6M7k__TzKL|=kyJ&@Vx zAi!S@hy|01Z>_uUzEs5y@OHp3SRp&mYKTOGxIL(cAv35J5fKJn0O_^;_KRGDulaI< zGVDF{(}N4t=Dv9Y0?Y_o!RBs0dcd3*QJBmUorNdY%}A82nYnAt50Wdg!;6w4aH%2r ziJ0uzki2*x%lwkVeG5;L;Ns;-{9#P&N5ni5=US3gvWN=TIPqcqFdY#UL|Qp$7U5utdAE z-cIOgx=~kkp(L*sn_iilUX_zmk4>-0N-j>1$7aOk&fi5RbVpuPM7y`pj> zlW~cOg-OB55h1BDg{LZOuU*KhD)NpFbijBx270@MVq7DG+@pg$V?(^-!~IfXg0fP= z^Rt4p(*iOQ{L>Tt(-H&I6EUf=p&7{$+3CTl$v)B1Zh@gLe!+oJ$rYzAK5iSJaHeIt z<*nDhzWMxL-v9D{eE99Zefm%T*U$g--(SA}FEZ^mdt5y-ZJgp;$7anFyoK?3;{@L_ zEwGMGo2f%e@{j@?33TxQnNOsP2{aL)S8MlJN8flqX{M!<{j8HopwErYXjvl)@C<>@ z>myD+c}{uqjMUvT)Zaf!rOknp);4NKH?6Ci-rK`?^0?>9<@>cY7Ynd8xj7X%+2w7| z@g~E@=K8+Hw5(C6^lGIZacvkBKzNDuatR2XSfmt8EBNCI-k63zshuB_a>;V`fO(Q^ zo+MZRz{W}DF@kYq5X~i`08_Kz(l0fuL&faSumHfijZlGgscEfhS{n!ry+cRu)Y973 z)K(SwxtjbO9AQ$C+AXJc{{b)|U>FHjzW~F8B!I3CKraw;`1aXBxO16D?3%@Q%@O)Q zphKk4F}{2{LC7FMj8UMQ1G0@cAR^-dy`(^gfuImkH6{gwgFRs+$rlACppkhrihxNI zvKay=sY~A zfuR6XPS1cqwt~T87|>@R{s5R_4i58`OipsiBmC2&97_}CoVNs>DJbK!H`G?kr?Ud$T&eLx0vw_K*Pw96%hPxS(4yu4O zrUfFbmnH3Dh}&rb;)s?yVI1Nuv*)bT2{UO#*UOOiFyy3B18v$u9Muv=)Lg+b9o)KT zcE#p>q3MNU`K^BQqiN^!;)|cbo%+h&Pb&amJ3j+82K$SVKmZ1gaF+bS<~#lBzG`t# zZr&8>SLRd;Q!?X-P|2AUGsb3#Ebah_-rY}n-re6#qD!=kOE7;3MIHhzSmj!V@(!rI z&wxoVQ3<5g?)zl~mPiWiEeb6ZKS+)d>5SU`_$4uTVCz6qh;N}lMAiIy9_CGG zqerD1BFNBeBrKl6)ncnJ%_}biD&v$8+&e6RkQTyMHeSUCPcuPT&STEjF_Z+w{ZrF2>Nf+$^`gHf` zC_vA`N14g|L-UWdmn$pIm*&;wr zoi4)F;ch>EqO`0pzx=fN67DDgoY;8%%O-dyuYO;LL!=OTz4`@?WqJv16tIi=`X6Fw8&_FN0%}7jJA0Y4denM>Y z5c#N$eiZ>2QrY?gUB4rNm=EC$TsVqp9e%Lj_vgKJZ(qN$p;=q8Z0~{Fr4fPX)Qw9a z=}|5bUhXk|KJmdm2_b$-!I;#Lz_j4-%<#m*gq-TE)Z!HX*x-Qpu$(F!wyrWXCC)9> z&pjOD5*pwLET&f zk=Z$ExUz;Tw_5vX^HS5x5C8nzzy9}s`rrTkpa1KB{Nw-j@4x=b{{ZpZfBv5ze*JGN zJHJly%w(37!j@CmG7?il=PHM%^qdJTV?+g>0b+~JRkOx*;7FuP@KjMRVXg;1)6q8x zqP=&#t!MaYJG~n}Iy|8wGlf8n0r>^t^MuY>xc3X3bODUb;Co>Z`sc}m^VC5eoiI;>qc`SA{j&gU!~p?ufJf-(6Z)Y$ zku)eG4oWBlD8G=Z7tgbuDFdPl9yu^o$YU}Az<~A=Fb_1D z09p`-U@>e6$S*)I7F@70#1W$tI=DcAjlwzfD8LW}rkt9FtC=S!m56~}G6BB|N7PSG z!725#&`%G$n5U)?fRSgxpD;787D)6Gg;Az9NL5COdO>c40FxU4y%yAFliF<1nvDjl zM5MoU``NiG51YFg&-*ydecTrO^rPO12i;>&`==iFPP7naTS@cJ24`A`@B{!1KddE= z7-*9g@|dxkE^8sow^M|J!x}sn2sIU+s~!=pF=iIHqV-wLt8v-pm~4GowIQ~=kX!fl z>+ctLe}V(JkrX00TH%;o*k!y7F@~5vc0Mg`elV`T)h@r3nYRSG<$2BGwA?r@)^O*g z%+Wa#gF8rK^x&yP#s~ld>|)-2x3c#MNx8#H*b=O_eO>_ogIIzK7g4YQj2!?Bdg4*f z;d_K$pAmqe==zA3%}^bN8t&1=;~(@w2Kd6YB=8yNt2~fm7Fa5S(-F zUp&`BByj}tdHt$-^$na$y9@{ z`nXJU2A~0Ia4gr?xb$JU^?VD6=F# zBq7W@(my0EIwT`LGB+i@AT6mVHLoV8;ac^Xn+-MR%ZlrY@3uVGSho+>fT8`u0AR0w z-+-P&WX#}e0EVA8kXYaQX?5?tesvYkp;n$ND?eM*-1lr~c5+rOpOGr=wY9|OCxxU( z#}{F=&8zYU?U~cqqLr&ppD!9o+0B1Y%OcgHt2JG9n^#!5faxs>;IF z=Vg>8g{A}r#0EuV#3dDE1t-P0h4{IK_yUGG2YWjNd)WtjIfnW=h532MM@ARqM;8}| z7Z!wIbHlK?0A>-`+_<8`xPn6LsWT7zh(iLoa%p>Q|Ci7I^nd=#zx}T--1@%(w0{3r zaQ+{^|LecM{`ge!r#IieXFH?YSwfLNRf9cxTYU_g!4JviGlI5$8R3{V6V zmK)%Pb`ZlYoufd3J$%M^&`jwZ7#*NY0}=M9h1%B5?dThQ+Qw{dV?KFKZE2-< zc5^y9S?wLn{(%u9dAz%q+1^R{@%D3|y@2q_%_`5$t~%X#n@X8hDpoD#9qZzj4yZ4= zT&Gf+bQ*(3sg+B#Qn5}3LZ}h*)q+_i0x&smTnzxmC65g}Q;(7@lT;%YKnss#5m7e> zOeB7pM&AoJr?9%TETFx5we)TPFf=suRu$#BngX>KG;tg-tW8PokdwQV)E))YVLcK` zkBHPQB6T6rjph%54&xC!kQ!LeJh2;LSob`!Z$_7+l#rHYOdLkd05uC#Il#9tGGW0NA8>Vhn|r9O0G{7RILJ zv(xf9)JZ=pnwS_S6G!MYtwgF404c0ANR@g(FS*t%(=I3gz>JGZ(~`!#sIdSAX4Ws7 z49jyfinHf$*Pg!l@Hy$hb8>Spw~H=pAuP! zt?Xy3fc&D2oBEi_r~R`4V+5`aP_2)p>|sjBBpbtmRm!A!T(&c<+M85t&uR8%H9Mn{ zl{w9h(h9EfAD4E)t>kBjF*y9`{R)s!+aH&=VBQZrxwQ3haqFXb^ zm*y0v36XkaR>ByWCbNbIC@ea6N@ZF*=m6XO2t^po8N#~P3Y@`)U}^UgTHJzogUc78 zjT{>5p`Q?f2Tu0cfzSf+KclY2)Vb4@Zsw2z_=~D-1?DyR@}2?wo8cxf#1Rg=zJC7$VCc!0QS@*&((dO^4 z$roMk^3&Qr9N@c-tlC0nA;SD21Q>M3S+`%&Muy9p$|}y5_c8Davsu5gA-62BXQ!(! zoC-+9V3Gp?!@{ys0^(ysVxq7(Y+eaACMz*CGs-8)-!sn3CDPe7+R-=J%QwZxFV)vS zIlwF2$1~8+%@2dEXdDw6m8+cQQAK(VHJs7JmAiBQz>7l0sVil2GNPv^y&u&?0ARNt5J5Dzak>U32{c|G2?(;`e)3c=ajLU_ zti6ZZ+(y6m82_||GJqeaP-n>$pupJ8E%>X~A78k1_f*5x(vrr4yxP3%>Y9oxgnGFhqMXXBmtc24kMZ5)2KAhKKnaHf;ZbIYubF7|01=7(BwMQE;z{98Thxg7yvA5HLn_p zRBe)LO(?+Ra9H7j(gyw0o_z=D~@3o!kdK za7P0 z+lT7qTN3R$U$MxQ%nWjcjKQ&|Ey^cteAp$S}!i|GZSCQ?8 zwKw_&hX%tvAjF*B(d!+$z^ioYhwq4`1K7J8|MU_-uVuLE_;hLaE!&`_!Sl{A7Z}cN zzQ42s_rsO|#y0ooHg3s~*&3(K~ zJzb4lUH17!v3Qx4RGzLy|Av^lrfR4LEb_!Hcy!kH)03 zvz4U@g~?I5G2y_svLa%$qfQkh=M-mU6aj5Y$|y)af3>ivqBylMH##RPJTo&QGb{c~ zUfk)^FM!M=D!+eB?iEDnvxcZ}-~P|b*T1}c{kMnD|2Vt!U~GD!e^lK!sHAfgjDF?N z!~zg3LJyxv751>?6o!P(QFha%RHlO5E5VZm?dZ|#b<_)&y6Rf`I&h=q*RY^EaU*oL zj5P=#C#7@5434;m!E3{?n>(59Xif))(}`gNdeIp?I%6777^tnoU#V!hQqfd+_DV`} zUVLm$V%(Yh+zNWnlv=*Cym)V8?a{59k8f`6&#ARDN~CFCtD09UwNkNGBANvZg8&n# zgp+dKh=Mn)=8tLt$;Ozy%@xC##>G+k>KJ`_gt|1+vos9oMF#X*7^HyfED}PGv?F4) zEbJT`w*Vy!qo%<4qC%LQ;o_ z+$kh?B1wsDlh_t6u1!FOZS>Q`E*=RZ?7;z0390xg5*ie2>wu@5@j?oQN9+`l(PA=M zNJI-sSXe}ykcbzP2$F7+tQV>n@Gu#0F9cu`m`o1=CS*d{0)i!EarjIiWUOgs-xQEA z4jUA|pFPcH3cxb~W7w=W$^{CU*Zc-xBO|KOvA@^YxOyC5Y)mmRAqTkPPN}#^^C7?( zv@xy*W^lj_F!|IJe`L6iLh7ZEBvZT@v0_20S(4AJAZ>?BiWx-0=GQa}>$6L1b1N%z ztBZ51b2F>MLxR#vbp;oys#+=aC`MfuvxPKT+0L#)4R&<%8*yW`UBeC7u~yPl2UUP( zNXUcg-myj9;$ahEs)fXB?-l}p;aM6CLj|e{KLs8IgzM-L1HysZaC>ZG1r&$3**CR0 zCOZIB8xn8v=k~Q*Kv|xzAAANE@pSRKh=Bp5Z2-L81^|1i3+i3ky02NfCDE+$WbDB_K~^maA+``8@&3a$xz3nH_D}>mkvuv zz40yNM6(WQZ`KvxZIZ#RjlCyG1O5L3*y_92?mk@IeYk=!Y~?1b#jS(cjoScYz{3=C zD?-)m2w!rswiX_{N{aDHi+4_paf%6djt%ul3iC+`wF!2%@U^z|wY2fGGj_CiUtjOn zA02z|r~%;6hsH)$zFzj>!B)Zk<^f(NK5jBj*bR)wnxlN-#f1Vo1@17 zz5ex+Pkw#$=)b}pvJ<}ozaZt>Ru#TEsU$9;C^RD1FDy4Ox+o^JzU;@gaGX6<{{GT z2W1P672ej3Ki0Vg5+VgavT^uWG`rA7@MCImL3(E8I^cCKI>dY_W9Fkg&FB* z)6bOTme-VDth$tQ@qAiQanjk6q{0iSMJ1_ci&F}Ul5)?+=j6p^X8FfNIRyB4M2GpM z#s{RuMr0+Vo=XFNuQSi5oGHu9Ey+&H%OcX*%iv$e-RHXxUhX}3dGkI57>pIZr^`iz zZHLf!fSdu496Vgy-tHiyaxUgkM%dz&)gh^>nL;kORvwX+=o;IfiV>1}Ob zXJqSX6BHBR9~0;o7wDVd>kwiGG#NbV9^>wvjiqWqo($71tLl2|TiIQN@s>_bWi6S);PsEG8T}FpZHnAG)x!{Ah$BrMjJj58a|ffToelto zCk^*dCuwvp0pDL&htDgxdiqRR8TfNncsUgSEar4p`b9E+cwTdRW98w^?WcRUp6%~F z*;(J6Q7C3r(mAz!UJE=-0RW~E3uh#vIk`wH;VA_Zay|ku-iVezs-9pmS}SRdS2g{V zl@Z9WrJ-(wTisB|hI-})$S}IM9}h~)#?G)2fN=%-^A?xmtP}&7VI7?+3*8&*Ee>>cA0^K~;)i_1yoF<~Uxb`Um z&@wcfZ`loSg`UE9@yS>r1t%bNi79CCDjv2~OhkzZC@BRerr-r+oUoe+Pdx+i>ZJg5 z$yqcQG1W(v_R*y9>{p+#j|l)KWWiC(;EGfqua7y+>YL*9O|cm~4pZ3QC+74?27rD6 z>wk8&!_LQUi1$r9nzLKCWv%RKcDONH0+8DXwOES~%r{uzErz0Et?sd64Cp| z#(DCEjs3M-Zx4aNzWJx%(=BZBLio3K@aftc7Cw8EVS5|4S|D3iZ?NkX^kC>PRg`1iW~bOql33?^~tHq4Mcv$qZR@Er82Zc5AUJ=j~ndv|W<&fL}kV3>M& zTROMKmCszbQR5gC(A5br%*?~h$i+e5-s+f@nVzND z5i_%ojEvqlFo37Q^z?qUdGK^*bGNkzbLwJB>V>H2T>qd9pYZ&M=%U29qNIfLNokkT zvMaOGuVyBdrX`i6rIchO6lcU2WdZ{qMB^ejW(@U!^8&OrfQaZ$k; z$+7tviRUs>i%z8$CZ`l6<(6d?T*}KiU%(ii1pk8oxJ1x0#1blWrc*D4h=aK7_L*(Di(UU_BN z*~MvT1&Og45gwt=fL_l2PDhOOeskpG_m3U<#NhbRlgB?i{>jINM;yGJJVLyT985p4 zG|+dpGV^nDj0pBgj1Ndj2}n+h&o8D7Ol%)Md-3`&pMUf7^VdH;{^G|+U;g~$>tCLN z`r^lDU;Oa&)t{fe{PWt*^T7#iFH1~iiSXT%ILZ{UM~Ls{k!fO5FWl}TQAMO)Ne@HL z8JWj-3vSfWTRMjuS_iIP@4kGET;IY5FKKLN)wR$`y;ICVDT&4<^-ka^W1w)P@wP4w z3O|G<4!3pn;YcHR(g+&c4*=HPGe#ivU#si@0L#fOOGzt8O3Y14JQEX@eZ8Vf#9vui zyuZ8s?9Tp+I|t7XZa-RI-q6aWGfK&VR=G5zo>w6N6G4E@N`)FRPr)CP@rLBngKFNe zmNzmxJur-K!dyKk=tj-K*dzBhaPs)0}oU7w8_XVGIFbg)FL1>PGRbK_~vPBGan9BY2~7u zgfO4Ja{|*miD?Ch;t@Kg@$CWtDl$qy>5@=!0t#jt-wE#L;o6174iT|Kh-;G&Iz{*n zAps2lCIrZ(V80wGc$OIIw;1Z&89?)w| zy|6w5=(W5uySzNJyg0L>Q7-iM43v~sU%cAfjO}Y957uq;ZD4l~-(Bfv6cFW;&LOilPaN)uXR#o zfM>m9D@2YKAdNb@M4wz?POgE6Nc{_xp{1UY1?JT1m~wwyd3#c`r`~+DxCc*$ZQg-4 z2Dij?yJE;`XrN-NJCE0P9xtvR%4XJvrX(mr@8Gy_apM36A?`m}MF0k`u$2Q63l<>R z{tF1OeduH`fgL%y{Ek>UIK9Ida=;k^OjmEa?cbp5-Ai=0mjMM~md6uhGYru+k~|V)d{d)? zGh;jw0vsdU?898_g5C6O%|13d`LUt#`^QfL8hv1R;;46X)6s=SbILTJSnL(GN~*)xhyoc*f+e;BdowPG}k>W+cWlb zXl!DbJ3si!Yy<(HoZ06Sfhk#R05B|k1AH5ky#E!fq@-R?d8kN)+;-~8Lh z?;kPLGqyI-Gtt*G)3@_-u=TV*VQr*uZ))P{Xcge)92x49nBbS3oO7w7Z%VTN^vhS@ z{q^(LKYsSrkB?veaQDTZ?|%NngI7O2diC?;&wqUK`Cp&D{L{|E3lsqD=1-kxc#`u6Jb!yTYsyH5{pKi$7|e`Rh-Ba>*Al6j4C zNu!!m$h2aS4q%{wfhqXoGTtx%m}YuVJ2f~vJvc$>XerJf>8e+;h_geK1sECiZvBwD zFwng)1gEIzMyP05xEVHK78)2l6E@$EnP;N`y|lD;2(9iG7{fts0#^kEo@iE6+hn9> z39(5=X%P_{dDwa(p@k37)z!#FHGmS}+a#1O0ilDBYZDPW`S|uJObegT&L?*8Nu9cB z&OBV30N)Ot0No4UF2u9|hVgM70vt-#LzHxrU{6O6NlJ&kh$4D7k4oXusJby=z`c;X zC^#@h#9``+&!!95eF6@P-_PQ4`uKoh90s35NASj$a5%C7h%VU>M^`||kfUH6#qc2T zFxBXoW_)6n%bl5=)J|}fV{Zbb;J(-7q#9yuQqG-}0|Vm=RRW1%Y;tU%pV32_9v{_+ zVFw4G7g95eirFRQ{EB7?0!*{GI{iUoyOOCj}Ly3%mH4%32TwNr+hahw*y z(6tu&)h1>QYN#4D42cf3_}Vz`=#r!-+zikdw>FO^&Hglt!GG=$I~Sk-T||zmTo?PaZ{Uj7B&tv z%R3_VQrV40d*6T)cD4o%_AZfO-pNT`$?-m^u>tAP9tnXiF+R=_UY1^t9~&Efe9{OW z8#OTg02r8&!AIuCrao?#K|V%at^i$TK5k}kA<^litE0ZXRFi{x3SQ-w)oWw0NDOR?dHwCiQ#5^Q^}3|n9Pvqoap#-NzlM9Bu1YL_f2>AN%IKG z4De6$agX=#O!W1N53u)jK4y9{COwBWrIfB5A_n$k1p@3j02mx-3C}OI-!hXRj6)UM(r8yqaHe zCfLx6<@B*n$#W+a}?iZ3{oSdf--Ha+!h zM%uZI(--s3UMe4%mhBuoyLk`k+IRry1rZCrA7t?6XFK;{wAI4S0glOu%|7iC8sr%j z6p$Dem>TJm5at;l>XjJo9vAG95agK{;-4NJoE0CK9s^+JlN{_E>24Qf=Nai5nGqU) zCN?22)-T%gq_y4=qYunoOg&?~15O3IL^vBd862|&gfMdO_N#2ha8>KF^?S;#N3**C zV8FnhtR5Vyw-y;fTGv1uVGK95Fg>mka(I*4%^l!g%C+SNQd2^4Z1lOXgo_c0rD2Js zAxWj7@fQNa3%r6)yNBj@M&|e?or#FciAX#hlUbB{_R5*UD`x<@axP?~pG`^0OGwI% z4@(O24E68|^RjZa{m|f(f75&KUyuIg6B9iH8{?y9$Bpbv?Yte09n1`EP0YQVYyy28 zLIXXcqx}<-3a(riFRVZN=JzkZ|MTas{{ras=@&mgdhz4ovp?VY^e=Z_{Ppm&A0E8; z>x1Y2y}kcxa6(O{O;h`TV@)%_U$*gCI!A#aO}3$jag=FNueh5oq4voDz*-rTDyqBdTbVe@1fyR{VN3(lYR7U&y^|#BBtTayirv!5>ckG> zy2miY;g)tRlKFSBef`GlN7mhoA)jV-sco$yk_~zOFeI zTGxfBLl-E3E>KcpgAiLIBQ;70^LH1W{&Q|JaBwgmueyrXUs)i8x=0=N?3 zJEpO%fL?%AlU+@qroeMB&C|G60YWuEF90nbrbUA9042h=i%DG)GDg;e7k3l*MBFq9 z&T*Ci@nZH0=u{Xv&w*QCI&{HdU~v5q7#LH?X7X9IX(kOYOw`X7!eR;F5+Ys5W&kq- ziU!~(8(_-^Sc*ZmVyGWz8K9SRupa-IxsLJQ!XmBOMqK)xb=nXe$B0^;Re{MW`0FIx2#!MRxQoS z=0($TJg&F2yr!(8<$5C#MILS?^f%)ME;saCZDL$+<6LQC)}qH6@!X1*{yNNfJ4Mid zooFHPFib@!Rg7iGiT&Dks;Iha^m_YHP1krEMO2HPpbRab8EOD904+eTKJG@p@Fs11 z4M2-Jy29b_(#AI^gG;oDwZ5qhK(8UmEpQ!^?~TiE^$XU9BpZs=2P=D@ZQOndqo)Uu;twcy3UT7 zwLO7)>B6;od%qw(YwHgT4fSno?1F-PlM{l{6T`BTe3Qf7q5`aaTs}52d|&^>hsTXR z&^LNt-{?bwlOLO${Mg(Gn3jdFmpLdOPZLi!a~}_LA9oYzVRmp_u(g?+nVzlb zF&h(bJ#KFX0QPtp$>*2MEzpM8RV@|i`3Z@+aq;;{3FnjJ&d2y?xVy$%+l8AsMp!$9 zS=xkHz^TuX?lvBF$IVPn-rQJHUs-VFLP5o)vsKkaRgFbe z&4t$+^D1g{FW2N=sXl%A24vX9inOyOLGelMAt7O@NwK-9OJIamj_5r;4)E z&*z*eJzHE+F)mc?96sNHc~Rh7`a8g2lO$~Fdj=pu9UTuzOR)>^wf6Hc^R%}MaCM9F z2jvv*Z5QHc7wl;t=Is<6;FS^`kdYXYl@gMb6qpemm>wFM5fYmlnN%2`Qji#z9Tgt$ z@9k@EWouyVYHaIcWgldZ1X)@;2f@M6)^45^O&G3v1N`1p+Ye`NJz3lXl71*%)>65Y z8l|<1PfpT?man z7Zjf7ACn)PTo`q#Fh2ckQg-pFyt3@V%ei@FIoZYO09`Q|k@4B_$$80P$zfih-p+n5 zrVbVc)<(z7^^TbxbMSStbhSBZs&DCLW8-PBXJKGyYi=Ll;}ja=92y#xk<)_fS=>H+ z`P;v|`u@kyztQQ}FHb-F<-zkG4xj$@;OSp(KmGIm(?8vP_UD@iuO|h|fGar4IFUNt zMHmGT!ji{pny6I`J*{1Xz{AKi5sos2CX6?Aa4uC6N-klm8~SRSSl6nlm#&cui`#S0 zHeIz-v4ai_tee5_A^-%AVacN%xB(O%yc@Tmj*7_*i^$2#uObj8M8b`QxjU=N54JWQ-`sk1Yv0vEoSif|j2UAf@sw@*zaT*R~ zZUCT*G7rQHqN{gth`Kb~txKQ>0Ml^rS`KlppQHnrwhyhPqqMZn89GYY-3qt`C5+N6 zBQ{EKbv)D!F|JNdY!u<@B!p%Wu8EHU;#D`<)i8~1=5{qrpc*F8jnnuxKB0q)Zk|Fn z3-RsXetRWG0r1ynMe z76uHA-YsG?1S~oLm`Jw)h7_`dh!|KOZJI$9uwWFuhy(Y-;FH-j05I^VxE~lBL&9b# zI80Eo{yrJN*vJ4V<>)Yg7C_g`)D*a?C&z(oX{M&NeEy6;FeBh=c++Zl_6rVDQBLzz zJdtpME1a69byGT9nz+MbvoiI9d}axjW=Wx4RcJSqvzzL6$oyy}Oyh>*$r&VWwNjBFfM_kg(n{fUZ@>B-Ae! z?`HStDvGWqaOS9^%l-Tv=H&XA?9PLNU}>ETj225_}T;c;)9i2up55!0G{ze zHpbwL%5|Vwd(U8S#y*TiUDb7MAV4(S7-q$k% zDt64=#hBZMnOX&#+J#y; zMccZ>I(x=?*?Kw{+uG)rRgOp&BrAuSn~xW6J%zttIG*u2Vqm)U!#5K4_8>Dn5QgM& zfZLw|1Dn}87?v)QN2C}Iuahy}NbYYSGHY>^D=p21*GhBC&lgr!mNs>iHDWH-p^9rd z&Q`VMS2X5btv_?MHs^A6R_XPO^X0*@iMDPYPJVuFp#hFTUM?X%5g7^T#iuihb2BgG zo-QdYy;03m%x@n)-vR)G6TjeLVWh7Fels`kz1TQ-G$vDKl$1D!hd71?*@yT!g!(#% z`?^N>I{|2gdD#YmPXx|pzOFW5{?0Mso=LF*X^Da9i9s1LVcAis=ToxFGSiAvlM7O! zGa?hxA_@yqbI&9sriXZjIa&K!I|MnnhkFDhggN_p*?4$gZzpgyo6^<0s;vieK*?`E zTe!79s^B)!Zd@T`mUP9HW799RWz~1r^os^6J8xu{r(f+jpI4chR1}eSJu|H|A@p=W zV2)SNX`hf>|B&2(h`ivG^KscFX^3xSp1GKPrZgw3I6XNpDK;xAD8WBGB`i5VE%j`A zaAK&VuZyvr*)danV+Ru}Hw(95*VxQh{}_KuR}1jDPPC(0 z82kVV&8(*4P9gPWVT=QUcjOsr9eXI1hAm0|{-To!=RO87v()Z$5%U`#PJq?jB~P7Z3OhEm;Um0@Vl%Y#IyK7y-!y_*OtK38_;=?2r&qaxz*(>V)D&!AdANX*XWngA;b+ zb?JygDg^*cM5FMiq-h#OjDQMe9s)!mONfwREC4W;j$u%?ke)+b0i+37G!dIAZxHqstsRSR$W|PUr~##Z=+wX#Z@-<)Sx)mTKZ~GL%_f;HPXxL>7})_@_I%i zZt_O^AmCX&cA||Uz%dmBwi?e;A?=9#CIY_}!##VA*hJ=bF=X{vZZmPZXLyb}vIry$ z6qCEjM+4hDo<I=bpx+2~nX9x*f1x3+Q!^b1W(b_otVVq)@}qk13co%~SW z_(Oe@500Ax2|HqJqGxWbZ)tSg(#X)k(#+G<#M9l-)foWHz{T-|i=DBDvzeEhk&E4N zI}3dW>l4m)`VKb7>}>Sx?S8d<8xAysAlKN<1BxbS5PtD# zD+!=0r7$DmOlneI#>MN^l9|3b)_x0ULV($CAMy!Dr&x#dFuLc|=Dzhle@< zfB_JO2e?HC`y_;W#==q5jv<~FzAhI2o;IO@j!|JQF_B&gF@CABAz87}XX4Y&r=Bj) z$t=xEJa;NIEg~SoJMC2D)yjg(`l9&kaHn7g05F$O57$s{b7wmnFW)*`@1%B1vU*p! z^>FUi^ToXvOSkU}7v$}%x~s(OOZc?1u8b?_vmK1qc4~X(#njj{ez65Up_$(2D)ZSq zRvWcF@=T=9DKFm)FTZrJfGod=Ghx|f8D}f<&s@yODbCI+&Q8ux2}}%f3vu%ZbN38) z4~P#0zl!)XDIO6%W{#G6W+y%}`oz@5+&#i0{8V^SesWB9j7NlrrJMCpQ@xK*9zAM& z(%Q?b@ahfa;?~P={`A#%Kfe0rM}V%EU;Xmp%b%XT`su;5zutN9$6I&*uygnKTX%kU z^YFj+4u2<7>{I9hG;st=870%E0j0VKlh^Agm9?abY638@=1z7eZUjvnZ)oegQQuQh zMJ~O9FF4;>T#CC<$GlO`xOf?pd$u0DgwBxyb?d|pwPX8%&jJ0ykVlZNjZr|dPT*T; z7Pz8t{ayG09C4_+4u7Gn=IpuaMdxcy=Uz=tDT+_XOG!CbTwLGU#$xnIdHl5*&F#f` zxP%A*rZcdu_1zhzN+T1`sALOTCCr|eiPa(?UOcS?)RbB@q2!M$rv??1gUZPv)zpxZ zJD`|gkC3}C*DkSJ8bsY#6%#+(58Dpmb{C5{KL96z%?@B^IG8zjAPn3WH%syLB22XiT_r?S3%lwi*cKtCNq}jX=&YWA3~L7Z z1pqdVX&Og2abZtKGu-YXb@1_RLPDE}2(GPCN*CZ3;F*kq7Lq#o#4a&xsRzo16H!Sp zajKUrq;(5u6gWbRM&W}m8B{Sq5ep_C!fDG$AqEB`ImG=CV8~S$F7?L1L~NK?59lQx zV8{mhq#TBnMU$}^at>2Iz>*KZwM5lF01O(~H1`d?cz|9!HB6w_wa)WYNdF;!Y*H$e z(0XZAH*U~+C`zGZPOe#0%q=Kp=Vh8@xpqUYo}$tR@uVf0az(wcq*(&BD4$pIlw+L9 z_GSzKSmA}Lvg@eI2J)pEY*9rg09Y%D+eV&lCUR@g!&RMwRh`3Eo7ol3?6O+gwbp(B zFz8;e3sFfPm}w;o+PlTAWN{sKva++EGOES&$$+Avdd2jyMaK9tU>JRT6%=W3j>g^S z9$&+6AOm0?F5S8}v%TF;L1h(XlvbDLmghxeg!(19dBoay#M^qOIQyk} zc&E7gq+2gm5yP$94X!LCS7vw zCY+@L+v_*)JW;RT9+EDR#uS)AVHbzj(Kp$_9H}F9mo_$>yHQ!z*iwa|RAG9rb@p6s zr(9|wUTnYuj9sWjpRYm{TyH73+K^Lx)h#gmqvM9}AJc>0;OXQN)DP39Y^f ze(v#}zG;2|83CcE!=v-#3a=FwTrP+@7407v;1%s}6X0U#YGLYb=@{tZ66^-bCoa$@ zF37^uRu>ofiJ6{-r?r2Ae^hpKVqtQ8eu7tw*GYTBPfU(}WTf4`Re*MGCuYdgf>z`hH`Sar!f4TqkPlpfxd++e~HxK`?vH!b`{oiif`bH$*qp{`8 z0c8(UjIk z(@Zi4+eK|?;NrVvy?|j@4I8Iop|xz>3tD0 zfG#v(mk1AsJ%2;$Ihp7M6V49OO*ch-!QsFbp2~;=`Z~4PT&{ z1`HF-z=cGg74o9igt1aj^<_xxTj^GulL$ zY$Qz79)J7uotJ;ueDKxs!HZ>h-sU-Mc8Ai2q#Ys+7f9dW|2m8Az}897IyB> zZQYyOzBj;^ha_j1dU%?7dzpH87`wQfu(O37gI1QHJ~21@#Kh!%{S*Ioj`g#WiU3mFi6{DfzkjMnV47>- zDL20qH{Vpxpv=Iq+^~?W;Na9C=Rglv|A2ZFaZ0@=T{+O)e7FFAs=ov5-9%>Hrk5_j z2`(f;fPD@}S>F2${C!ld?hT5U$YToJpcvCH>|}8}7$enKa!GyDrRKI8Tz4&jQH^8X zK+~^vQm?d;%Ukdl8!;DZF~v3Lb5$LM*P616uR3}M{`--m|9f506Oy zpri;NAWsp#E`dJhuU^xv+}wVbYxUrzE|VVIvvKFi0FQsFC_gYY4j7eJa;#@!v{zD$ zPhymBLKvV|aB75aT%ft9-Ek)i6K_}R&_I{?XwMYbS`(g~9GjmOclK1;`OMShc^Rc= zq6<>pqI|7o4a=@*rVqK|WDZc0|}s5bA< z>^xoE`waOvbikkI)peJacb*|nkTo}#6zi(;`tw1_F8+yjesT7JNnUYzak-cBgOfvT z{Hz_q9Nl9*y%T%`(t?9CBEmAGJtDo$T+B~8nix2lop3fc_O!9|v$qR$vi7kDtTOks zHubPR;b^L7V|d)&__*CkOK)r6c%R7Z$b^D~_`Ep4S}PALLu+HV0RJ{z&*77=zx*vs z)_C=e4q!ig2HflAPj?>v*WtrI9NhoItvkQlKKR|n?cZ+he=pG?q*gcfR#cO!8mQGxw5q1w{EID#*_8z)9qpL0${OnF z!um^tj|`0fc3ZHi2vB2p*Xx>zXOKs%aU-AFlC z(vVwt<6LoVW=?rZ+J(e~qSWLIZLM^^V0Cu>-onbm<&7uno6okkKi#?c;?~U8m5ja;FTh-T!{d9@B;LXDKC5>Bc`T#c9u00uKSxWK>$0EC-|Q2Zd9!@oE-9&A@A! zI5h*Kq;;v`M3q(`UUFiSgwPk$z)lS~of&R- zQ3UiJAt+k6nAs} zxY9rgSiK@}WdibWJ=EX?iIF0H8-HxzU03J9|$<-)RZ zaY??Q=Bp*+qEY5>Lsiq2EA@5F#AYm_<7;@%3~g+MIK zr#Ar8)mwl)MIwEXRzpayziZb2eidAk)HNUKxP{;op%-%S8Sx)rVE5;??=vTao)Iy| zF3#rO9;Tiirtk=t^KpCo6ZZCcwl>FXZ1k+HKQT3X-_ZCsdWH~P`X=ukGx*5B;G~^} zxr?=_v!$V(iN3ALaVM)|4pv8Pt@Q0}PB_}=*_t1zW0;A!COaNN=1 zq^rxXuC~-)?`W-PX|6+~1_aY>6l_In^$k>Q^qJUzbl=FlpoGHEm@`3Man9Zeo*`Lb zQF&1@`7x1MQSKo=-r=DQ7{Y{dL9%j7y?u8M@LxBc@h!jrsNMn$8DDY_iQ#|(^b!tO zfhVq>sFwEzg-et%6@Ey99}uJ1Qyq-a8hmegGr9`X-9Tp7lGxRF&JAo|C5CuT%!rN5 zjPZ=}Iq6~vewCK~zV;DejIZ#|s7`DAeqSp?sI zpj@5DbGj&_c-fj(xiO3CYfUN&3C(g1%5VutbN7z-a1M8}^078`H#74vHTSl#4Y0Kf zw6hC#unTrH_pvg9lex_d9ZgQUSeSa-Sp_;<2Rd8&+nae?o4A=9gNK~W^c_u)*&FFO z7@u%91~2i7^$N=fi^_?NIUOCI7UmJ=n^Sy_C)YfA`JZ2Y_sgr-KfQYW^Q+gty!!e_ z0I(Nd{Pf`IU-l0F^X?;HV1L}Z`@5Yx|9SK7?`Bt@_j2Uzm;qo{?U)fXaRNh{xKZDI zwF-B&8dq6M>ckB-w)Hi&v#T3`kYTUYP_ETdF4qvV&evz0t-am=sMei-zNzFAs*5lN zK-fhX?Z6J=C=>0NL0tC)zGu9#gVxkeLlcJ@+JVC{TRIuwOAK)cT+1ptP8VD+DQ_(} zTb-Cv6cbwz7k9q0k{}juu5UiQdF$n^{V(?Je7S%4<-y%A5AS?=XaCjV!RLnupB~(P zvbnOYk;%1E@fsoskO=@qz#YR|qiPhyqyGM89mDBcB{lASLK$^4Np(JozA= z)YsM|?!_s9S~0t3S%6Gk8b;@AU*{aFa~1*EOg~x!gp7k%vk58yFeVnz3l<#>^L!!0 z+LV+w3BFlOXp)gyq}T>gXEnF6Vye9g=ocT?$nC11MmGs?t&^w*ZdW66AgozLM2SeK zX>2QOL_|0832g#G8+eEh3=G>QAfqHytdxcXO3}IjuJ}|kj|TfS_{<&=jP9jLISly_ zOV-bn_A#UkP;?QCCgw2V!XT?p%3?|25+WOXDT4aN08~S^#~{oQp7l=mQowrxA_59Z z`)G0&T>-e&*Q@BGD_Ow6`c;5l`v+0;xhE=1)w| zspnKm&BWN`^cYvc6V1ys3yK+FV7gk-F0H5+SCw;Xs)c3sA~Z0?!n|}&#a9VO_=D8` zrs|gRvZ{)zj;2m}LwoPldh(4n7T{JbcC@0MbD^4YslKKLkn z0HfDo#~TTgRj9!_>?poZi|d%B+6d;IE$$6x&T1TgH|U!Hycw`bq~^6a<2!0UIgo__b^!>|8*_thWo zz5K(y&;M}uv)}JO`;YBM->e?KTH60?@%E<+x8bxGM5O*{Z4k2R|H5e%s2eP(Uyx+G zut)#z&w-Rehy@?joe&MI5`639JRAKW@mfM+U7$e)A#gFexPUc;qj9noH$`*Yi{jsYvF2r($2)d z&cx8!>bRrT5gW5(ww8K!R`3Lvoy9Rb^WzQ{ChiW_{vKxDZpR(%4V;~RW$kNkWb!WB7KS{Uf!A6xwo|U=%1DlA;a!|j%3m!IZ}tPZvsfz7IL6?WB;jq@z#KF zxo2XAII6@CO40rNcIHF_iBpARGwU>jXo1Klj>*bn;#qGNrhcDo= z;XQy)T{a{9a{bPeo}uB?bNOLer#us59izkD6JorRV!Y!*{o;Z{lS4yNg6#txPdJ(z zxw+T|1v^AU*hPiACPw?G#RR9vMCBx&D$2ze8oO&+P*2lrn&e*}v)j7n+KG@qa z($_sH%qtlH%-bQ*$->3zxP@74#u;Kie^R>xn_spb&)j^x2#;C7(~fhSx5i}>>5^Ku zqHON2Pbdlq%X5h+^o%^~ACepBmF(jb;cOFNXXR&Y>1$yDhjUxm`T?~vHSsVp^D(#Z zwX*iNxAu23^|Up1w>I~(weYhy0rc{=HuJJE^RzMXusP{&W#DFc!qwc+-OMV$&L`0? zEHexVqKu8sNVstAnrdm|)pvjS;@h9Ud=2Z1uYrF3^!ZnReeu-~&tCp`=fNL=d)fYx-B&A7C1uTp#dWy_*V8l0lhZB)g=D3sUGD3XudY45ef!nn z-PiXYe0%@lH}@XAzJLGg`}e-Oclg!aJFo8Ed2wg|>DuDDMkW2MyQNgzFIRO>TYZrM74>#0h4fZ{bUur zOGSfmy)%rqnZ8af3x%i{vVO?ItNQSoHvq%JwZxuI6%|QA?Cw;M+vVLIa!Q+=)GWg{ zO0e}3VzacnT|j8z!iB^ZA+{Ch*EF;%*g7vDbbtZ?0|mb%cJT0R(|CZcEdpcl1WoA0l=85A=pDdGYRRX3yV@AIfsB@ zTA@e_0LB-p`B1;qLb-TKDB=mHC#Ttc9F0Pwkt$~-ss*_FHMgLeThhz{4_na!ZUO6B zSb-~ti@?AZWOG`fMmen%4D$fMT5DP_l~$KsX{u@@RyC3@RTD4QQ?IqLu`GE@H?IaW zjJVfeHEI~uE9vMF*Puokaokoi?`lh5XRi>=kW`@tfTGo+CtE4fCL;e@TYoco8qHMJ zV!1cEMu==Rk)y@4)R101VjN3Nn^>bxtT&Q`D4Lu)y4X9qgzA;PIqW<#Pn%d`POVYL zmV3un2F16A#Jd=#ynA?#KDp8}I*;v>PiYR9_P$=({p$9k|GNA9zwdwcr^jFZ`1JM9 z&%OmJ_S2JZe|Y}epPvJmefJA;1CZ>0o`3)UKK!0qvgj;1RcU~>s zez64nZ2tvp{eT~f;W8sq{{%1{(qJFsGVI9LEg`-+G`0e{^gj6e;`A#kUvn!zbBAC%8-M$(k~|WJNf~06H(d$I2#zU?NWKso zk>&3b<=_!y9USWwpBb5UCMh8+CN48FAkHr!Ato}48C2^*H9`-Po-(`~fLrk>GO#$+pF zqOo_hmNZyHVAYdYWzDGg{IiLL#pkQrO6sxaYEXsO+fJRmWbWwtzTVLjRz^0S_Q1d# z1Kpj1JZycPt=#RbJsg~UeHuEkE4vSLvt73CzS@MdG(X$Ahm0?|`&_-YRngWSnvw34 zl;9Q<_WbjG>zp%lM!Mr6!QHKJcL@O!;;s-R zxKltmlq>F1?uu)0LU0QqA>Q3HbIx=p-JP_2dl%jFzTd3hde)OlL3$EaUC*`mz3?w%&z-sS=RE-~Q$e2}%by@{KJbD(?Zh1i&zXMAJ*51Af5Y6zOH|WpmW(8N+$3&SPlyH=6yvMv$<f=;439e)9kM%d#L?)uli3Mp3nMpcBZQ5q zx4o&4y@j7W0z8pijPOqi^p5ilN{GxTDefJcd-nDN>=E_qhZiqFzJI>={jk5`Blmk_u)@dPPww6<5_*MQSZau}hH*6oyk$N-HQNrKO{8-Y&d$BQNQE zT3r0?fWUJI&%|4IE4jSUvGL6(PhPC8yjfd$ySnmbb@}x&U>F40%lVnz`PnT1u;IQD zwHOv|0D$S_5{*o#mGBk34rxcToC^%BUEbXy?QRlv)Jxjxq^))G_IgEYt-OUJZsOEd zlLSrt5yfC9v!*IHrx}A3Q85YzP7cXbuB4QzX@Fs6Dh5);tWeir)U^b4ZIy~mP_wXF zCRW45spvQ*+z+b&@0SwGU`R+Qu@oL;#sPZeiipL+>M}02xEoz8#FlYUMc{^yNAd~C zPD~L{t`6X2=%Nn5EnF$Ds-g>D+C?Z6kT60rUPQ(U$ru3{DA6rzYu=F{L3A{Wl9 zB*Lb>u=hMk#H5H=G!VF)$fSrlOsHHO4lI|4T{)Nn293v{KwL4%dW=y30s#nlbZA{d zIt6y0Vp4_rEBI5yG>V)_moe#b4o6;J505anwkiO<^kcwO-Cg?)58*;$celJ7=oe&| ztV__@)Gn9E!1EgpgRf!MO1i{qp;9eX>*Tr~rLI@0?N{nxA1T<316+Ihidnip8>Jf z-~6=x=JVQ{&#SM0f_e2BWaZVTl~>@!UpC(UviAD(^4`ap&2NSm_IjtE_DpZ}&uqhG zM#!FZ{lVmYO6}7NQmub#9gbF+UV~k)<~AX;ru9QrW>@>>;NjH1+3nt$4LGw9e(h%e z^p<`{4Q__*pLd1@fKyA_@yF`11;y}uE}8^n>kCUm14qlFcGgF1Y>wI6gOAOV&Q2$s z9FE%C9=5SOY-0f!_7@|=|2lr`KaU>$k3&cP^U%RB43C&PT9`Xq8dw`1u{?Ri+VFst z;g?p%2W?Cb+nFD*HvhuH>>yAsbJK&C76)xCjyqYKcsLolIsCx{aopJbn3?BEb5B!5 zl-HfS2lon6!_I{To(VX0J0|Q>kVk~Qf3$0KLO}HCz=+s@&}jdF2=CxizJ9S@esSJm z7lQ-N`Ja7owT36uO)Nky)sItwLrdY@QaBF`0_?GVF7pD2zP4W9Z5;w^W{t0%;PCp0 zZE8GRMU+<5@R1yDDYLtj$t`1Xi)d|m)r~n-4Os+E7LIzqsLVeh-Z3~R{?`2)*~qJz zWtY>7!jrBZG&1?qp)U=rjI9y&z`)GiY%CDA#x7Qd4wh!l_V%8hCD>}<%fs{fKOMHj zeWssFvOV@>V|-zK?D2AoP;&m>eMDrWXG~0RQbOd#grN9P??|tx^WlN<0Y)z1?-AIH zi;;(?g+DN=VArT1KsZYeo1-?zkJ%hILs&Tlxmmf}0D2v>JZa)$eZtY=sH3@|yPdg@ zn{#xCXMBWBfZHJpgFhZR5P#_wyHnYvn-&i}QH-r>^#z>$(ET?DXV?2?o^&Y1*Rw80 zUG+JAJ1F9cU+5LT$eZC|SEGW?NBJd%xt{WNjC6GfbF}rhwe+?$_cXWmv9bU>Lzr6o z*|kC< zQby8~tQvNc5)$$SRYiP4X&1T>FiZf{svxN`Zs#%`3D6K0POb$_JCeuHciTcD~CY8By_s6wno*^ z2>Tzlv?|)*2AHz53jhojZglaKU0gW;7#9KzmTz=$x$T`Yi46R2sDD@`(@44{D!xK3 zQETNokY0rjNEke}EC=Zs(Cb$p0GLp%<|$+ylCIizHi4O+R+y1ilv79mKRhLs>|5#R z+u2pOb4WnKfLGnlAzw^ECl@fXDjPG)n+k9p>1A~{GKu#K7%7E}Tyzt7+nroW4!Q{- zs<=v!T)+kZ!?2|QR5_SVBuzwa>;nYCvXzxIDd1l|z6(he6Y6>k@I3GerLm9JG6)Dq zZ62Vt^phHNwANl$=P(B_j5|SX9%6NjvpdF#^*z<~J*8w}DTR-wi@1vUiG{BwAHSPk z{Ayw2?~6PCv%LH5%JYxwdp~Tv{=EM7r**)mH@~dC`epSMyjgkm)6(9j<(Gh1pO^MN zFTeZ=WcBqg>u-PEc=zk_%O9sV{yFscb?@{}pPpfZu#Nr3z?|NteqSCOoP{VF{JnNj zZ(UHT^Z;9fZ=T!eo7?P#;srmN;KrN2z5Ss6JLnAzHc0H5T-J^+sYe&ZeUo=`i;vn{ zAG9$(?q~@N>9~XaaR&zjM+dl&Xm5Ai-d10fVSCip@_@PVUyV=x+34h-3=ID3=)u39 zIB4l+V~MaeurvPB;`m?9j(ur)@(W9&FRe`u+gTj3w*mHc*v9IpjTM06VOy)i_Le8y z?5zDg{@@U7Y2|BV>SgV!Q92l!p#P*AG+EYx!9YzI5~Rz zVyK*<*|mLu!D7gHIG|*5ack@ec!-$o9h+1Qj1=Jr!HH*XWM&l+FjrG=2A&FtI1`e1 zBhLSnpP{4aF-I#y4>wakANz=4x7Z-NAP-w#7c*Duqt?d{m>)b~dhm$((L<)kzBD>~ z!1(AF#>f5&uI2^?j#d`l2a>jOU! z@Zo3+{9viZrWtL`{qnX&Nnb#n_E=KvCi}XDo>5&-d9`EOs;Oi0V=@{l@ z8)#?eZ{rYb?+|3~9OmQ}>1OI~VF1?;&5k>o9Ck21Y;SbJ+1$j#&cfHlDgfaS>gRSU z%p*PmaVo?!E;KMHCid#ZN)ml``PqwiaHDHy?;`-%^OryDzW8DH`47*Yf4{x^-S&&` zSGPY-J^tqL%D*13d^7WCV|{d#a+BveEwG9wUqLtoB44UGm z*(o6wl#sHs+tB4gB1%cCP*F?u3`44Dr3zZ9l3uRmV0ATkEgPq0qE%G5a;RcpRE#Pa z1uLUqew$-ijg(ZCN{J;hQmLf6SV$=5;Y+yKB0jFT8V~5-3HkyM62KUs;6A-XfMg(y-*>-i7*Y;h z%3;blOeu@0sI5^pzzr~EOB--6Wf!1Vx2g*kA8NYz8n_3>Q}X!=u0Y-`;I*`ic>+FH zke^>fs-p1Q1qz;ABUI@CqU1e2(6@Skf%QXx0sZQQ1}5$4mFTo0wTdSbw+q_AKM0C|T}nY;NJfF1Y*b?&wk;Rkl2KmwpqQCb z#7r)rrxh`8Ws`1ZRRd$oMR(-mxS18rX(b$Ta}S24dRWm~PLovCYGG|*oemfnxUZy1 z@HJX$%Wy8b3osAEQs8Ser_I`(!}HUk7i%bE`GhZ@sEYgzb|h8&&uwHjh8=eLgj**1rW9P;`8tM^yK-c zrM=%@ff$4LOD})c!|dnP*8pQb&2IlMu(83}r~3@kZ;$C06=z`$$o_8G|5;9i3yCYR zsl^PGkU_ZPwb}=>J}|pAI0Fkt^jl^7Ym6KEGJ2q2i@J#?>ai!P(Z{V4&8e%`k2zQ# zwlg_oZ+67N>X?JQfs@k-XQ!j~_J?h44q90sv;<~mbHd)%z}D)BwZ#ET(*qVJ2TV^M zFga%8WNGMRanSnY7ZwIzSe`gwZLBviv%_|lM;+~sI@lexw>f5K0|0i^&i0stt%0+> znU~uiyiU2hM%vp3o0<8ZH1jjE@HaE^G%`dOI)pmfhd7>eIce->!}dBmnH0y_&sypo>g7awBn z>uBTa9(X1x?&kf|Hy-###UC*-J!Wof;c92;YG>|jW$tQifv_`nvo&#Zv~ovyg+)>8 zI)>rqoW3e@;i-Ob<@V&0wZ5rwu1YHJ>+czzWVd!_AuFqDYw>JK(yephi6Q51#wT2h zL4>%Sa4fvhZ@8uNa4UEvx*7jybe!ZZF+dC^BegXj2I|U8B z5B|hJiw6J`_YL(tni`s37@k?sJsP97)SbR_(JLX;?R1cPe2{wrxcYk~2Kb*1^gkQm zli-7hM!1E$xJSAA#e2I)x*EBd9J4bz?q~+kb;#Dxz{T9y)6UZ0-66u?B{~FgD#|k< z);%s#?~!5W?%!__D^_;i?Y;To#mgU`z4)NdN0?_HcXvPT?0(pM_HkqPhuNinOw4}; z=ru9(_kr=Z%7LdOrX;_VUR1%zDj))L!jZa=n5G9A_>%J4ydq|LHaWMDo>M@+b-(Q9 z{qp1tJfK&~!>UU+OHL)FpSziV=}vLn`Lru{i!0%XFFs&i1-=P2@EYM1xQhX)ix zlBVveyh2)DP8+&ZR1GYuNJB$vnUxA^2@DOXVWWC#@j4EnhlSD5QEDnmO~U!^H9>&sr!WKkf(+~8sk`|wJibOKPzgjzu1LWXcDHtDl$xoDnZf>1g;*u-6sZIX zokY_k)q(WNdU|ELK8UX0^y~ium`cHw@SD3E=?!(XhSDPJjawP%Ip|VMO->0dqm+6t zuli0dIS<wNg!Qnj=0PC?$l3iudUjeK&S-#4&VpVt7r_C7s%@oE0q4|7kypWpfM@y?G6&ptib{prc>=Y{8= z9`F7Lu1^5NUjDMU2jmO}U<`=b;)@?M$Jbwdv%?o=*b5K}IFOOr676z_1ClK}q zu8x2@#~qwbI=dfpa5`*jchJTbFzk>OuqdlzHdcqM&HrX@@};@aA#2m)b`~cbtd826 zAGWbL0*GaAr5^_mC}nrT)xp5U0W!?q7N83NOfO+Ju*%WN;Sc@^-kz~8&Jos@0fvU2 zM~r<83_T4_xScrRYG~kS003s%2gu!h0AT1wIjTWg zUN1)0i_7aoWq@Q%ZV{z3x4JpIij!PY5r6$MBEr+k%gV&f(kUb`?82q9(f|qo0Du5V zL_t(j*Y0>n#vC#>JY;+v_>GZ+rG=}Fxr?=htF5VTA)J>A0JiY- z(UWaBvT|X4bZ)Lg)=6t-sz*jfW*1fcW6eBa1-bhC?MqQf5f|^Ay_t47EFr+w+wP>Z zm7%+{Rgk}PY`8;&uXTWny}!Gev(0}W`P+Ye@h1aQ12bE*W2OdwId28E-q+0C;-s^=fwTD$C(~n2W`+nmb3a$xP+ynW5VzPckGM#W_$XI+ z=q)tv=B*ByZgcPNyRSbyfBA85?}z6vzTbWE5#-tPeSm#<`V0W|4=Nt!kt=YmDGv#^?v>{iGc&S@x9?Zndw|Nws(SE{ zkeN>f=t|BY+)hQEy_ySmzpiA*Uw9aECMEeH21n||5j(KNc1(3k35uOx$|yv#iYqv! z=$cYACl7d7DHV8FYC0;fh@6p2xSLk~Fs}*_>(Z61i#P@OB*VS( zu_489H>U-cnFAQsfrj#=pq8tdDBxZSIuZl`Oa~;41q=+MWni>4w3dNY(=ZB3rGkQz zQBV?6g_KkQ04Bs2^09eBe7*n&&{e=i!-i5kbb$a*4K%B&LPSQ3 zC>V$<5(sLaTS5{Fq6=ynRzSh=sL-EyaQCYkuu8_H$(U4#u6+rkQKWRLgh3VQJq&JW z(J2BdNkFa^kYMis!k;##JNO$Rs7*nql$7pX)cOK)!gVD%hXQw@lssZP<A zH-G^HgIixx-GHojK;GLc)d7I@NVF=hOw`Kjs_$s4?NUhxE76SG_w%y~tMZZb)cmUC z{OSiq)buh=A)&pvx-%15mr?+i3JdUUl{7&nvi??9b$Tf~AKRQ!$^i^3sp`x?)@D^U z0G%qp^O6fWsin2}T2&sd`$2I7im3nqBi8AvYc%wxK2p7wz)=Dw(OQQ|EkoqC;p(RT zQmQx?*I7Xm15g1EtFF_~S_f!t1Epj>k}RY&_k!<6s8s<-!39{RREX>DR@ci|GaT$wWLgN;YJF*C@>)E*x=Mr2(+$=rKPdFF> zyYh&Rj=FRO5f=KTAzZ#YVQ*sMYGdo=Wb5Hz?QU=5*#~<#1-aXMJ0CMQ`hWlW$3On{Pevvut!yoin;8E2$l?D!dhjpD zkNmg65df{f7#==gdi`5$44WM0m(dE#=>1gxU42hMaTBXX zD^V;A3~$WOzgS&)4d}J8_Ihpg_0sYi0I-Saoso&n@kg6ek2Xe!=G8I{G%$$(&`T|s zsHGy6s9UBl+>iihbv8lw0ykZa;`hz~a>0EsaK!z9{1sa6lL9&qgyYkNc*jZh`+67!q6-OYj??PO2?lZQF@Tlb4I@(DN7 z%C4jo-^@hc&Z$Z%VgP{UV_N{duBG9DhXLiPqzZ}%?f3I&fLj$5K8hy5FvJ9o9L*46 z+0sG+4-o4@3LXFq#ggR_xDU%)D;aVOOM#+E3NdXJL>`_cL(?U+mH|%p1dgLEr^)fP zI&6))ge1V#z(LQX1}(L@kJ{SL=p4cU--0WOeYjdBmL+F*j<7q18Eu2K=DymFp@H!i zGYel&FT9^y`g(cupPSFW-Gn8GpVwahvhwoheSm>10ERsU`C)$NhlSlA9zO%uAD=*a z{j{Hb{n9Td{yZ<}8_09Ow>!JDG ze!Y{yjjq|Peptb%e~gAU2A`_?;L4C5V6z*&vzst8TfNg8y^|}tM@x#~IdSi!%nE`_ zWRzoMn1#Q$v9G65fVZ)qubH2}bwHqnkFTMt+c5{nBX)L&ZEb;;9kaIqrghlP`h=Ua zWq_|;XoyunpqY=Kv4^LTJHi-&F!k~PG4^mj>E>*Na5eMuGWGMa@b|R|2(t7KF!%K~ z_wh3G^#Xs;A41Rj1|}k$g3T>F4ZNbAJfdBU+zk)e9X{%K?6|Xmsi(1(znNX2B_iC} zD->btY;(f+xQDOL#hbU{FI{_3P~I%*7Yt0v$7Z!ti@j6J{nN0`2^_+vUtJxV-Gt4) zp!#f$E^I-1!8#5IFxUujVM{f(0073|_5*;CJGG=X4Zc-|X_8=?WtH{Pa*hbeA~^QE|7gLSRS6473ySl+&aauW?*i%e|oNOVuI0F z8-F$N^p#Udx8g6{JsWdA!qLyp+RNU;+s!@%4jc126^4lLbpc3lcQ|Nd@E?Eq(_g;) ztGT6xjji>O69#`gaNy6!4t;58aKOaqOOumdni(80KXKH`=$MV+m!?O*Fga@IWMS@Z zAA05tzP4S^KdTsB(oU}G4Q#b{cC~*NRtgQm3E~^Wb0C|e;JCUgL%i)G z!TUpxtQ!c+{0|t^;_8~X~>KmeDcG}tGDyhGci)xqt%VC zA1}Nc8F{YNt#@>eH8c;2WeYt68zYal7oNOaU3ml0wYvIpW#!f4((C#8my=UF>x*l#+$h)Z$?oJ+oZHM)%g> zdpLM40|x>DM#d<}7&!?oBci25ln?@}L{MEKB9#iOi}?^&W#Xy|0j?O>m;hVA#}x3v z6)+68JuKtnk^5H>5xN(k7Y`7NgyMpGJ-}cUA_dQ*R&i<7kX}rRSifv2Wl-fTnu0}_ z0~=$|q#$%Eh?q_n(WoLSFfg*d?4HD9Pz4N{h{cf9utk7g9DPYWjU=K|Wbn)~MG62` z%aFqLLzc9jEo-QeH`b|In)Co`SGRYlJ3G}~-72_#*rnleRa~APU=lT7tm26!9eiF} z7r%>7r!uwb-ad7|q!XB#vRA6z$(N%bW28Q?Ue`ETbb3sw+b*V z`Iwgb`82?=0&Hs;vAdkiLy~wHrWnN#=VCemv96?7-!Ej9Qbp+%t+#Sm>By#hTw5`r zvjEjhV9D@oS#ecY1yxLK9>TKJ6*M`Dp)4f{Q%mbhh&(J?L8wzP+lT0F1MIF5N{jvk zbk_*0a|Fm402s4vu(o@Q-8l>ZHZ=Ki>hV`_4{Y;)Hg><;-21Tc>c{=9F94(`dVnqL zep+}2-D__9<2^YhxfUpLn>V0}B`XXV?4YAi(-&xAfWQ znc2`zLl5hjTGc&T){H$-3{Q6|dM~HvxJ5@hM1=yVob>fH4hk^z^EdJJxA5_|^zk$E z@;d3_d|Y46ane;EH%BucFZ=K?m)KKI(We}vPT7P;n)?S?`1@P<`2p-%2LxC_R{2`_ z`&tG1nfrO0`uJD{1lfcJ+lB_)hXvb(2U&;v{~;&=;T~q~9B2`JF*K_({d(3l3orA- zHV2Q`9X;VkI&YBC-B0Q4 zscKQ-nw6L)MPhO zg`KyxXM~4eT%b=3NRWSQsDEUzdyuDvyRCthS!BW)v3hV~eq(%oV`y%ze|lasHZJMy z?T`rsI)VC;TG%h@PVmRgQh0OtgMgOSRJu4J7i^g z%+~Cry}6N-jftzBwU4vCzkBqhi^K-5s2}{jEo!HhbknOna7T2te`Z-fgB5PH4$p1r z0R|*&Z4AzJUBuO}T%-LQBYd1Q)N+_5<{}t9j>d6x_+cT)JI+ z^=@fSF%^kv0Ld++<`&UFGV_RqW%TRlKQYR`9X1Ndg*497g7v7GJ?hXy^0)VMBE4u!z!O`83N6)5ab{Cgkt*t?TEw8*> zT6y(&VQ*&k`NZVYM-$tR#y7`Dm-_q0)KZ07Bv6a^TCq?o5kY_nyJg%KX;%Y0`_^6Rlh_1vf+M7PYa1o&dF-2 zC;>_&q2tuGL|t9Awg#(VV|r@vy#QcLtcHP6(*eM+N(xp10fv&0P(m_FNUD%hE2Y#5 zF&QZ&f>a0zND&^omk?jX#}te3#X>ybSqWg9fKU!1s;&g#!)caSJ_!U3qF+M9>Vty} zqu>QpIQ?sX>kDv8!D4``oJE&084?CVOa~aFikLLmk^^?-pb3~1VGUDM%Mvnybuqw= zs0M(F0bUex7;+9n!J+|xDQcPWI+nbirKn>o>TC7^rfg{j`ladY(sXobySQ2|51>oa z-L2#CReXU;AXf7vDy~%CE#bCw%0<$Nu}KcAmPTUm+W1PILMK%BNZ{PBKHy$5U5}!- zN8YQI=yc*&W+fxde$$8Yw5_Vo?IT2jr!vDFn+Sy7ds zfFQ`jbluNql@WPJGVm~EEqA<*JI?AD#<7)^OevvO4PI#!PBn?98~76q!YM}k5Ur`V zcWig^@%#DJZ`PiDyZ!RxHmsrlyaLd*`_sa+&w9=Jynhe3zJPMU@(sOd{kC8T%j?(Q z{z&@}gEqDa0M#hqF|qg3Gp{D672#-H?SypXiF#tW7nZNAfX`pOI{Z7}gLr$tC=u>I!TLk} zOiOSBemTD>ADL^C4Kf9TRPG?1J4os7C$;PFEoy9w8r`Ts)r!m6f>LI838N#gy5?F& zPH;l3Ylw%Hm$jLjg$2UK+S|?{(9u86C+=!w{I%$a3t{0&5n)L&VP~U5Pe*u%``LKd zA2K->b>@t`cNEZT{P9NL)RJa=S~W7R9Uaq+_Y3>FS-h$W_QO(IS`H@VTI$8Kx6Yiu zd+u85r7I~H60XJi#`w5L_;{ZV_fLolh!2mva4PEJX+)Txt%s|Dg~cDg`0J4qN9|o5 zENspHcJzz?I`riM6O$v>mWEFDhR*he;E4kklG+$L*;pW4U4s2xLw%frJkQ*^&FPZL zh8NTmi@NFMo*Df(75$4uru>5C6LFZ9;6`yF7|)7@{kV7}V|2KMa3*3P%9oBx`e z|E7QZ_2|_5k*Qa0+@Vrb9THPpiEk{&G-MZ2^GcYdsQS!&O3Fh5c>c{QpaHJ}$uDIU zl(BQ*FNx=`=SLxVrj!;R&GEvq$4w!y{`TBg3mBLkm6H zVWn86mI(E;D+L;GBN3`ZTq(C%+6@6F(F5!^8~Y7l(7mK>Y-wAKq@}jIww8{ib<|4~ z!cjEp9mzZs<)1;)pM(dh3^GnS81699kWWysM692 zDhkAyjEa#^F=7fzLID7)6jRED#4=GeQd9*btV~>8CMH&hs>;PxND-lohc51}DC$NP z3$SHy+AOg*cZfL?llu_X;0DNL=bmL+FV<#ZCbs%n|a299Ds zP4#g1tEoW`uy#0txxG`@)ve=lbvz!37Sc}`n@$x|L zFuy~j5^B^Uom$+hlMiU6eLc#d9`#6%dbCG5(xV*fRZsNkru%fW1AX&DgG&=rJ5!I} zj84Dr9edj@T)3B7btAbnB^#4kP@Qxu?{*d;AKOw?)d38wkkAfpQVUrb#q7)XD<2dx zvMTCxQH>y(B{evPxU9M>7uB3uUY~<*$w7ByRkkPHL*L9M7glu@RPi2`w>+$9%tp18 zlLdH&tcsic*rOkRU)altqN@l4>rN1=kGetphdPW5C0zYPC51 z+w>zbp^b66Mw#uyeP;39(&j(bpMAHr_x;wZAJ<=gUV@bRdEwbl`lZ7C zs*Rr(c7c|`MMY>`FM)OavikbxwKqS*-ihyF=Z>wff7<>U5bKwnuVJ2k{VT{8M3?@! z*c!q>~(+r!ku z-P{LZ;e#;qax?LAHS%&d3kp?YwRiKs3^zq zaF>V(m#7G*sBo9q7(`;C`I_yan{yTz>}uw!HLePG5OAI=nVG zurxHVJUqAz0M@G;R!XEQ3HV6msRaV9L?I0BhqL@6HWk&O%~M+cN+Biiv^zKmUhWaMXc;L4;_Ob@53w}#Ne z!u2w7ee9}%8d5Kt*u&C?Sq&Jbrr~8&teA`fZw3Gpkt+lcU`R0$(5pmTRVu1N3acsv z1SAkI5e_N9mI8Ev@G+$VbSZd|hbx0~EcJV1{3@iNx*WV808B{2z@^1%w1`ZQ(ufKc z&@TuuC7U5*(4`Ezj1ADmk+W;$91cL2jKh}Ka3rt|2LnXHfiM%-u%rz&09}%X8cBT( z-1K6QhvYC+kY0NC0_bXPRy4!I%Ya_6f}^!f)83)$>e2xZ>jnVhYlI?=P^uNk zl{^`@xr^VytFLX$e3;8()(dzFnW#^v93SkN9qL~g8Cn`2U7i?Ue>A>6F}?{hKDIqM zu{$~Wd~#}UVruWv^sAA{mwjU|HAAnZy64P>;oHf$v>ZZa5itu%zmbkgDWGLnHs+w4 z9u%`c0L<=XQwq_|H`DQVbIIvtoJ?eG5s&x!3e*$0I;(z9xV`PL~RHRdEj6+PceH0L>Fk>H2 zLr*tDgzHIn7bAp=xsQj1kEe;J+er^sBX2LukZ`-`Qw}kw9b@C2Von1CvyFe{#Okj9GSX6LmM36he#m3(5%!Nz2rRBHMvr4O2 z5MZNEv=jOf)W7#F1^|O=hL5*K0Kk5m3I^*qU^5PVcf?UR!DMG-enT+=mk`;a5xQ`M z${!^228msL)g3(mV7L|)s!>{AD=ua6is)?xq=tu;)o1S94mlI$6BFPW>E|8o=N;n> z02X^S=1S_NbGH*?E<{9~kBB%Q9eyr0BtG21+xdizp-;GP83C&sn*taZf4r{G>Z66V zq3Kz!vI|L0Pc2O>AYLt{-MO28HsN|?!u7Z-4=&%#xN-Ht<+C^9V=u5MAsL0{i2Dd9Xx;>AMBkJ9(XP+?0o3y8>g^$ zYYQ)Xhah*C5D%9iPv=1I)0eNbi!>X%|9tWKd)U};@B5v-k5Bi01K7^)$DL;%p6+}H zvbh5R2I;l)?b_4t=9m7bcjQ(7_}l)`-I^v9Y^sZ`d6-|FU&4HtL&?Y`rj*-7m?=Biu?UK6fSUZgRo3n^`yS;RBAo%cAKN{AIO=w3IEnIS5Zf!|Ez=@iH>EjTz z42+I}>1E)1nN>ZEDjlOrL&Iz6cr_KLAY)`?46LLFAyXr3%irPAPO|1l|7>gz6u*FPb4Guv6ePXTZgU_08F6a z3Y9!Da4#iKA!rx3H}UvgGCoh)*FUY(P4@N8kBx4Pk3St9dp11sbZF%1@aPW6(D2T{ z5XiH^q1~aO=YvBp`Uao(^zUl>o+))trRp7le5a*z0g2{ay@gE6CFGP-(~HSBGjTbU z4H;#%nH3FbB{f+U4ZyChrQ%caXeoKrD=Fx!Y1o{K`cguBKB^H-6_gP=^HD88%kpsT z$tCqSvlwUZqLPc4WfWd9k(-Tb&8}?9K-TA>TPukC-14S}C3XAVr-}$&c#b-)q%pOm zu>i*f29}Fz0x2W$0Hr8Ry_A-|>Uu4usgJ->1M31v1Nwz#NP)TmG*;Kh`zE%hmtHS! zeY3Lrul2o;>#sg7zxcESH0${%pkGg({|K`90+dI*fjI$yeOiC>Y4h!;r|*Ay z_Vq8%z5&Yh)6;jqY`^(=6R-{B&Cjc^;P%+>N1lJ)UxD0Td)$ZFKFQ$sdV}Niu;o$;s1p-KB?1?b-@iO-DF!F))vJD6B6%Xj;5|`)}f7bPM zqSL83hf~o`aZzsZah{0@-e+Mv6XJZ%CV0fhIYdR;gh$wfg_#EhmTb_f>_^*yEZTDIR3{_#`r z9_7g6R>e3=G)fbW&_yE@!4QczKqwYqZ^&=8Jq2WG@~1tQS=XRI+(fW8`m;IuV;nd&5OU18k2N8`og`W8{lz2^=i_M z#PEcmm~$}~ZeKco>&oeiXZ$0Byh6O4{ah^FtZh8(9DJOt5!PnTrWTH--hNIVUiL=T zh6Xm~hE5JfZf+(X?q;3{GY=ONgoBljt7DMAdsKw?sp!Ppcd0ERQU9C*0&Jycay>fZ(PJlcPqOP564Jv*HeDpalWBvgA#9@2EKUy z-ucio;V!`*7M>2~2s=wpIC#v--_0cu;TDK+5A?lw`<_TW@a)ZZ&tLu4U*p;H@1MT- zxcM9w8A5vPetf#~;px-wwzt3ChIR8F*LOYudM$5!H$L;Vc5rWG`fI*)rcyr*xsuR; zC%0n=ZFxoP++s$04l(5+;a(aV0PI#uSy~qUVJ_)@I__3VUg7=}`EF;&N7LB^!8H1-2%yn4FSPnVCnplZL!_BRlb8 z%9(Tb&RtBoaXatY&77iA>iojXmDO*ire8gpcsVxla&+?5_{^*M#rG@g|5)DmW@Y{B zwYB#f>+iQVzt;EQc(uIr1^^7uYx2=H02s)~(8@sHtV+=<69MJo19~a>a1~J{%00Gwin}kWa8l)X{`v4O)bNRka*deXXhiq)u5^C*#ye=^Pn@EoU*MOuCps z7c&99I3gfqb%0L|(uR6TeVw$SPFBm2a_I6Jy1b69Y^+l@!(~H#>3oZ#rB&Y40-|UJ z3~N_|w6rRlfsl2pI)sWYsf?$P3p6r8k5oLM(MtiIhBooI=Sf90$fKSzB8qenOew9E1+L{fJx4y7316T(M>tX z+6q#42_BHFr2yBKhOE0&z)3EyPq>6y&B}G)o3>1y6?6X(&xSjJCm= z?lE#>4}ckgqeRpBvc83><=2Z({{aBDw)g!S+yMK$`~t2b0(9w%HGa@98$$3vf7<&A zkZJ4f&ztW)J^kvZ-EV%`{rcBu?|*sv?&s~dKX1MHbsZSheo6gHz#(|vc}d^4V-c2w zz@m?T^K9R}VA(zRc zY;dwe*>^oB-~H78N7Z>iMPY3VJMZ0hb8nKH+{E4sh>8Nz1x1<&Qlu)PpcJL|4nv(@ zVCZe=y-%ap=`##4L+@Rs*^+yc*vtLT8BE^$*V^l>GvhF_B;lKH?|t^M)UvTsGQE1< z#PGEKC56j6CoXCp)l@yHt94Xc{jjFm5iO157cZVR&_8?4Sn-;vviS{FOM4A#M=iTs zT6T_?Z#(PXy?5Es>EbOrom+OgH|>CdsalvRo0$MEUbMAQF*Q4FV06aF_^h#!;#Fg1 z(`!oCj1}}Rebp%(=@gC7xq>JX%GlXh|MtZzx3n+WX=$2kshR5NSQ_bDni`p#TH4y& zu)kqoqOYN=edoSA0*Ubq3P-0EH1~;Hh2x!rQ$4a{+xqx0LF%=GmA@A3@@J4Ij3x@t6_OX z=f;)GmY1&GG`=6;5|S9^i@*^zjdU~&wT-RJ!{d{>xssKQ z*DLF=KYsPm+qJc~tE+F8WJyApEnIyCJPbC?%M5Js3%y>=EW93@ebX;`+QWIw7Cp+y zqejJLL?>iMCuGMXVFD15NC}J1@CzaNhrkgV?%qice3Af)ViR+s;&Z$M@weR(*7w4# zoI`H9M>~2%8QpMq^^T8D%0VFW5;27_NjX8`#E95*5OK-5z{7xh0eVHmrp6{_hDMV- z1Ck#4#=CjO-1CUAb$Dp&;Nj^VOQQ>B=Uz%iwuIt!!N3Mzw80-(AChd2O+1~Rdoelt zd~){L?A-IE#TN?;&vu4Ljg77k4y_7=3mon=n={Q}O|ttZ`uikp%q~V_V@qQlv%VHE ztgW#DUc}r`L$853Ls=h3;Z8RPfEieu;Z>FtT6w{rW9b=Hg;ja^8TkB)l2%$1w~Yn> z1`JHVlL1WJBj{^tOOJ|A^$(~>h;Ak(Gspm7sGe+WcP6%rjO|V*cBKuw zG%Ph04cG$|3I-{eMgpKppc50Q*!cS7*ajqmic4t10;I(?Vd6nFgB2YIz}5tNIue0$ zQORgvZ*|1PMhHSQH5I-sGabjsz_n$Q+JT2q1Je5<7>zXj$D3tP(IkuPRRF&ukX z3g>PB_iB|T4*|g1$|_sSDrrR()RNkk8b&*{yR(%CK0UBGQ*7QGTd>F-SmlZyv-oQ) z{^Ne$qh8KR4{N2Xcd4^yv8{W)wR5qhW0Br6M{il6(HEMVryHB6YU;-;Yep)nBqb$7 zIl01=)ZSzyBPh7g?E%ii8}|MLMG=!o6(I=)07$_Jg*#V&g(MY6VJm&3vjbwY17fo7 zdL_I1V?&ZCL5Q5txLgFbECQe|z91kz??FU{Ye>euAhL~HyhnK2{a}K#KQ<_d;)}=) zPAZBc)DW`hiNu;jay>em8bhdvPpMBxZ4O7*L}F{B@iieyMIHcq$wjza1}2xDQq)1n zZ;K<+Y zVBHJWx^~zji;q(PIZr;$u73caTHgM={OltL;9g5lVXAOu9pLBFG~gB>-yf4ZWEseJ zJH(RV5Ng~6>~H)42sgd{pJ`bk4aAh}%V%MQ?9<#+*Z?y3^wYxg|A2Mu(eD6b{He|E zp~W`gOq*bm$(1r#;+*OxE7u45_6~Xuc4rLrPH1b&>1fI8Y9G_Oa7gv+iHoXox>_eM zT|8~5f7bZwS(9t3);5~Gpkk*>Xl zhLwtO!d`uG+o>u#G`PYfnk{R0($>ot8lDC zJRxi1nCu^#`u_no&y`u1VC>5@dYD&)y$OE+j5AW%Cd{V`v#Ej{x*&%p%xL72>sTZJ zu&ORhSsMUYQXxGduMv@15kbsx368jW(?-|A;G&tHuDQOE{Z$KRbBBlawysvT_pJcH zY~5{4Zkg&?8EaqHJ9kle*MU7h?mrNZOJNUBv4#O8Ci;iQIg$zP@Hlr^LhEZL7DvTr zxkV6edBvH#gqYq9v~mr;Y3FWcWUG74-pIELS z2CACIYJgtGwpYw<8r$8ucK^QlO*?&Mm6PA^{r>N}fBO6W1OJpeaa==9#mGS2^qSf= z6Ad%d%huMHtgNi=Js=d+)prjvcw?}MV?cKN;17W941V5Oc@kK(gLS*Xv7}Vuc?0$H z#_C#D2A6D&E?evA-_X15=@dsu!sVph_HmWhJSVSlMoI4i5K}Fai(1AYE*ssjLZxO5 zPcE%&zFJ=!XJY=H`dC zxgUN@HuKQ>Zji&h(9jri61E75FTti%fDHgxBq9R4eR=vN`iEcw!*ISK zC{MoxXSeW&K5@64d~Z4V;0Q&dV;fV`&xa&iJM`j-0LIoOBijIAQ?t*e=3mS%yjWO# zvApzRer|hudSh&CeF)G?u)qhuF4@z4tjYe~@xH!s0I-gh9(qFqqrQewTg|Agk$ITR zz~J#Oh%VSe4+Gj*b0q*6gL`>{R5~TMk=n>n?n>b`CL^L1` z4In2CO@-}ErEthY(yh~oYkWdb=WxA zmq>yg98kPqplGRR*{CXPYbL%eivR%DnVZp`m(!M?+g6a<3L-zZr67+<$!7vG6%;Y@ z3mN$o7zKrNN)e+7aI1__0?<`VFD|A+11n^}dzLB8QaEd)r4)t?Z)L5e@Z@q!MJ2PM zno(XuFR!DQH@4L>JDR$BTlt(GiGVvP;Lq}fOMKB9Td)ebC0OGKpRi$UaD*Fd!8%8< z!QwsX;{uqi^{}3F^*?UwUTN=I1;K1xVze$bGv*tbXX+XzDyv6}ONX;^IA{zbETY6G zfb1Vi42ekzi%W~e7Dr>tl2Yqq@zub7!jPqaUVt>Nfh0f|-{?#yuVfd0tbc6IJwMEY zAc9|1CXh2(rm@&3Cf6k(&Do!1emCl2WO_()p-TWRFd-j7s*NMnW3m{yOj?Yr^B$c| zOGWsU&86e> znB{`^okv=_f$1#qAIC8y{vjVNwxF+uHl_6<9ypK^MT)P7lbRr<-?dY)7`HpM09x_&l@m zX>J=3>(l(xFa3%DTJz68gB5O`eV&Dli_peqwt&FB;!kaN49qdOV|0#$($wk~k9PD6 zFuCibWo52pWT<3h0Dj$_H@b36TluVkuBw@dnx(m#m6eL+4OJ@}4Lb*Ihui8lH#KaX zwCwL{+T2#RzNKttt#r*o>6-a@(`)LM=87gJrwolw85^IucJ1s{6F@Kd%a=~*YM%s3 zaOKK5L;W*XbWiDN1GoFCM?BsslC}v3le5yU-Z6pnGFCXDbL@!5(W7d|j;I`yQ$MA2 zNlC}#qMrF>O(RVu?F)|g??)vf-FyO(DLM2$afe{68%`#kly!mc^l`vR9Kf_-7I9>j z5AQzSLD&3_e#s_cK!COL$E(^!1q>lTR}LMnvSJt~rLLb;(@U!E#gw)q3mHiT&2ib) z(d5E_WP+owuYsL~j=8>p%~g||*RDC5*ty%-xmlavHof6&VeMvZk3K~WLUAjg#`qvzcEbo}wxmw(PaKr7P z<(<1%uU^$Sc;u(A|MB<#`u_XB?AohLPq4J#{@ZGjwY1@Z{Rc=9}g9w~K%DYXC&8th@&6 z((3DlwJ!nxNY*;Pxcqu%{?+8%Zxi#sOQ(Nl4L)yYgHOj#$hl3C@#ztWbU-gWxe6dB zAQT@M2Kxs6Lhx>0NdQ-2u^DklN^k@jfX&kn1XvBDZYPBZ+zYOZni5(S zv@rlMY6S(*3jz$ba=@xsQ)zJqHa#z+hS~rC#)kmw8sK$`cwGWPk8qGRDC*>9$0Z{l zJfK8{RVG9=VdFc}aXneYo=j3tR%%anYF8GiBOMP_$2X$m>X5ND$e3CTq5+@OoPwf}F?2GNIfy{W z#V-JB%>)2T?V;p!7v^_T3P5yF3R?5?nECmPLP|?XacfCgOGzn+FPmbpmO+3OmjHT! z6=)W-q(sKB;?~mA_L8#pva;4Pm@{OQl+j8`sio!AiW*uqwWXo0i^1ye6!TeQeC{-l zKgSm?@&rp9!3szCh%0)+7CdGNfZRXfh}JoxO#m?Nz!qP;%@c3)2cPl=wjj@h+bqFW zFL$Gtv(d|5@9ulj)wkZ>^N88L44kZyHeKI5SyMk&Rw*ee6((bATs;%Lf{+h_kO8ro zLGgKE$;B~n@O*u6Vo?mfIw`HmKQ`ARGzDmuYcSC@m=u7>yXTL)PQC=I`>|fpS@Eg0fNJq6wWv&LPy!_uUzM0r8=6!YjV;IKFazQ!VW?6> zN?j8Jlst;37KFn>vvs|+73>ZsKKP~{S zYycF!pMXWO_mEq_+W^2;-jA-lldizS&Lhj89Snaz-yQ ztH9LpuF@5QGdha)ijtH1m5F>erQzUp#zJ?Z82`!$;JP9lmh%nA*|P zI`Yc;XH~8!s_H8#=_p$}-3pA1aenBBz@$;T2io})-Qub4!RcN&5fZj?0QdS6V7yVl zum$euA`IwW5MVo(HG|i5@W-k%ufdr*1%4yXQ$TolPY_#CGE*D zi&z_zQ4*6@=pBbNak`~trmuIy$jHvv^p=T(hwV)dJ8Ks!tGkxw&Q^x@rkd9-D_**A z?7aLxcI`T;pqyJ$%NvW8i!a9yc5xSl{-t zx#wkj-^=>m1FKt(SI;XQ`}UuI`}?jR|F&n(w+9aHId$r=lF~^v4FzqTqe?3B>RJyY zlPHZn5MZ3qR^deFzy!QgM?BR#AiFa44|?sS5M=HNA^^W~S)WY{}%~m5sMcPhKxRdIMt( zrVUrt-avpYzgk#FXm>q zrzY3I>xZR}2E@xk;nIL$NyJ~|au+yUsA8=Ckq%}Lv%Y~=T?v9-Qv>PMSjYUM1_lBU z3jhq>s8U@@tAc%rRH$G1&41o(Sq?m`q#!jbi&WZJ)89_x_O%H(-C`bOn1J6U6!wTD z?4h>GCZum5!N(^rJftEnnu<+sPsMg;ka}~--C3!f8Km|!Vrv?{C4<1sz|+%k06zd@ z^fVdL-~s1ks?5M>gd_$D36zV9ifuy1HlpI1kWkAS(D8LB1c-VpJk$(Z5L1!>z361f zEkLg??v+Dq&m(u{Wp-2Yx(g{?6iP<{h{E=Q!q$R92BnZ88y>&Yg3$^~SrA~YrR9KO z09}mI(iX_7FC{KUX&JM$oKaLpFD|E+R5X`V(#q@UfLo2NT`jEso*^!4g3q0SnOoj` zA9uc=zr+%(u=%TO;UkXlF|aNW+<^_YXal;JXcN+F@F{=j8E*)#kYV6S(H2{{#S&}+ zjInrIeca7%&U$;_qt>3)ww@Ib?ft8r?8oi>Pg=WIQZpHDUWoyb1Q(yUki^{N^m4!6hKI(26AF^koBa^^55v>_V)Fvx3nH-k-%FR?j;#RleFQPOGj2+Dfb7fQ#MO7;mD7(u&aT7y+2R(kw@6Wm zUI5wpH2-Yp=om!T%qGk$0%A>Ve1@ZhWreiyH8_wO2m-naMQdBt;AE-4!ushXQ#u&`9Mu+qF?r)ps>JMv|v zWM*~Yy487;>t~Iwo;EO4(APVyf9bUT#Z#BH&Kc{SGch=8tPh@)zoc_qOXJjKozr@{ z^4i*R+8T1Y>L)L0oHD!w0oEyy*0lA6p^~rNHdDMJcUXPjLG^N^;aF|gKmk*j*CNVq8OUP{WYYwhRDMPi7XS=IN_8);v;$d4 zPsp!FAlmq$&FD{v1=v3QWI;~b@QGw`&fr~I}1 zawbkEO>UhrusyG0eCnj;p`%Ls4#@v_QgNS(j=Yw!@?{GxGe;wv`{s6TmJS{kj-EGM zeH`!kI(P&)czD@fyrlTkzVE-=yX(9C`@Y?K;BR~O0u}r3UAw;Cv-j&=dlb~PqHx*e z3|>Qzga!dN)hV9r7JmU4yhnA%z$9?aCIpydmNPV3P+e^4VxniGWoU2U69_1LWJ2prlkG1lykqlTgW1zp`sCZ_m|f{FR9<@MKqUJzi9U+oy!lC0Yh z&`Soe-(m7_?H!2umACUtujc{17Jr*scr`iuYIOQnzT{;a`%!D(QbrCvCLtpMQ-DvY zO3kW6;>&iX9Y!KDe1lM4{z=|}$cO&OhXEKjZ{)oP32t7=V7=uMAE)I?lYwJ1}7JnWadpb0ji;gGno)67Od<^j@Rr;}fGeEYK>z?t zXacV%qnYVAW+slAfosnpb>xyd@>9DCv%4s{T?P4_1(Z%o5ujIFKBYB}(vn}uEGS|Y z!Au|%Ho;f4m6f*?moke$l))7|UslE_FQ=E4HWwE+my|Y_me&`SH5Av-tEg>F9bGMK zR?iTpf0W0WJ3ya*VCK9bJA4eQWuHWa+&G4w`xg&;>9yy7FNJtZR^7 z<7=N}?)72j$;a6z9~U-1!KRPrA7ux{zUbH7v(K|n;hikgGGf6AAZxNh+GFr0? z%*mr_Dkrox&l(zBFf+S&!&b-6LCe}+-O@(g(pKHdR>jQxoU!p4!z=Q77fhx`ZQ4_5iu0 z!qMA9`?}6a?c>KU9+SI#>a@YRGlt4%4VBccYhJe3HE_6SV56yTrJ;LW+w!)JyTAXf z`wv25F}1BcI%m9vKiNJ28P+Qs2g#N!?A!;&lih#}lndT$DFwuWd)P@o4(HSJM;Cj= zQ}sOqg)PE@mO)C(U>;MPLlb7xVLCChiASzs5zD(VMJ>nzT0$-WSalqTjM97IQHBmS zm#nTD*f(NRtwVAb z4$3PXI;(n2{gQ%~p`y+;RbyK{Ge={KJ6CP)o80s;zvpf5=6}m4>~2WZy&HCVhfeJJ z$G)Gw+kfD@efz)Owd=pW|L4~~?f!Nz;MqQP{i~?#^2#<*WA6xqGu8_K-Jrboh-LKp z$1MJw(UJzazSnvhttjGwRZK>Te}+=?WGNksH_a< z0N+a%*UxBbYv>y%5i-YSw^r8QEIs;t8361Npx5uRj(J(X#>(%oV}50)cjL{%D$uXD zb4#yh7Jz&GJ~{tta{hNgR72I!Q37|Aa*)!iHA7o6hao?vAc=;9IY?wf4o;Op!j1usTP%8riDLK8}X z#=#peBLT4p4}BATL$Has;$&>8rynXPf)JCK;p`gb<(C*3hV=?ac6JN9<>VcWAc_VT zr)Qo^rJDd?1LAFgXoD}@5D#uiBwHgR>l2eOi#R#`Y+`DAd~|(m^vS67kz{yna3_B# z1OQtU@|Q#aVEjeCV2Q;VYh!dZS5-Atl~b!Ks8yAW+8Snk4YQ$&URO#5?ge~HR{EmC z{2@Hc3}Dt=Q9!GNLlK)wa_R7bW(c#=tZaNzMo9sK*~smIDa1io25T4aI|0CWfMG+d zp}|gWZX!0`#f|LgRTLW1fJ|VdVA?ZCU72a^8RXXVGzK|^o<^c)kZ5T{Mh20dL7=8# z>407&6e9%&orDB@0zn6Xji+N07`P-lKADO~HWN`zq~vB|aud8iC9x?biB3khq+xfW zJ(JLxLu$`S>851%6y-3K9`Y4VdfVx3jx4Np?H;)0eS)b z0(*coT452bu$Wp{+E`dzUsTjkT-jXF*wWD2P48oM4siQN`J5>}cb>;xfZXCOvS6&T zc#l{BT7pN=v_!DJ#SwsYlM6z$3F##Z8Mn5$gWKGpr<|duVC4=!11oQ6iw{1j3~mbs zpRxh22A^_)lMTNRjQk=R|3x_Z5`|SHpe44B{ZoRo5RFp(G$rH@b1dU^hi`m9HA-_RSGuug9z@SDY3-LkR*z0P)cAz zVK}DJFR|oK0O{t#WbmiHF?oPVLGgv|At?{T$e3(uBDp?^T#q6*rR26Hr!+(+7bEc1 zk?8VBOjQJ`!Y?Yl%Yy z9v?=Q-$@qUNEY9LkitPzZz020;LKmy*p5$tVX*lgwig2O!A`~HXCGHz%6jf!z$sI6 zPe0CX!PCuC>z^kceS+@^XI6nXd^^1KR=V5Xg;NcI-~EvxUr^Fu3dhu7YX z%34LlbHCyX=$EbTA6L=ae@f+VyAS=>4?lf(aR1)ZC-QeLdfIY27vyzS6fPQtZ1(;P3wVPR&XV|Y8QA~17;MMM_~T2wu?7CVNNRgC(q~{me)Ld;gX_>m4V@P-AhJFS4@!g<-8H)9Vd3WM;O6h(6Xp~e`@@E?V{-}@l5aF%%1@JbHg2;!jwn!%AmPUQqRmh#wePUS5B*D4VhY zXKk$gz6eKbyoUW6vbIAwVq>RwW9`lS>YKTh*Rbbs>GjkCMAyXZZ{st+i6prMof;5|i-<`Ji6r~UMn(Y%3yMhd z@Il!+h5`d~a*MRN?e7(s9FvrlfXPq7P+}6YfQN<0riVnQ28QGB+z<2gPfi3XhR6gk z3k<_~`^P&tdf#yl@d-%w^iOnj3A}mRJ14(xYWCUK_%^^5fR<=r699}STo;I*3=Xf4 zjBZR!ZcR=-9iQADpV*uj+ZY>pGBOMS28SOC7leXE5r1hwxGdx^@;LJ>)@)CYq=nkq z1OZkCBT2lqm5ll_KrdPqg<4Zct%9e)=yh-=J%AZt*q@0+S~;wT0f1E(P;g19 z1vw40dTuWhxYr=JQ_SsE-cvR00GIv&+b zKsA9NpvS z&!uJM(enyv1x1X)5_(YyY}P0&1|(}KEoT;2GK;Gk3rgz?D(Z`Bsg*#sI(yp%+`bV$ zXHv+U;qzwMoS8n>Y(Hm_#aRN%)z4k-;{xki>*ub)Dwm95T;ckcnwDUlBii^AU;s{n zp=UfmropHD;pdQE5M5xu&4&Qn6u|Wvf8>P#5Nqsb(daJ&mcG7?%fL8GtD8Lr_Ao)eoX_?A#N)A~T|}Ro;;~K2bTbc<5fJ zjHZOtIt0Ed8dZi&ZA?n34^5y%pvs~#RruTvOm=HDwip#^u<-qn8BK|4O>rp= z2`NpHsPcrAdO}_sKo>5zB^*`e5k&9~!If5Z0s|YL{rBYJo2gYOUgI!%2yE%o?E1&~ z&3B92?*YK(H(|5JPIdp9#=oBpsRB3vZ}oLnIkH42hJYZBX{Vh6Ni61 zeCX?+c765jKfe0*yRZKK?ca9q+OK%(khTB({iUbeVnXmCT<+|KO2hr5lV^F7~iN<(LJzl6!3 z008TOE#NaO=@bi2yO@=AA$|dvj2tk92s3yi^W4#S-q^fge1R>QVQ{3Codcz<;*!?k zqBaSob(qpRRMPBgD;zwo;NctA#gzbnaRI;}!{%iGn`KGF zCDhEwRL3Cfb>BpjAfz3kAgZ>lyo1+G>t*=Gda3B2Qq(zi;fkVzvu$8_ptFb5RSO-H z>*`hxde%-x)^|;8?wQ)TU3d1jeHeKAVaRQtaL2&NTS1Zb=GN+m<$wC|=$@aB?f=LA zU4P%T>+iewe7k?&H~SCjUbn$hYN|TKP5onZ&KNM)HsKTmm>2@=Pjta?h%ki+3~aWG zH&|LgMpOXpL^c9;bCqAGvVs(798$JiOTh)645X zx!@>`r8P*e<<&3BwYdEI!XJI}JG~q8u#+Cp>vw>z$@$+V=ODVqrhk=A{>l<=H?_^> zm-R=*=SCnhLZj1?utkZOf=EQVUoh4?5FHtx9fzcZBQipwQiH;YUj8Vr0F1jgKn}*) z9dXM!;+|*1Jd1nFVOy$x2t=U+k4OtFqgD+km z7HiQixHgsN0~G@p12AhWElI~^Wu;a$ z*Y@Q=GI*XyC*heXcqS3Yz@w>{BoH)wGM$jzl7gZUlA0mU&@>{N zN`fsLG$3lJ=+<;xTLwHC-42gP6FRbpUAd(0g4CYE%--VM?&1Ofu=c#X*6h62oPw6D zd}d}o02np5fJTAPVw9B8i_4me%BUq()RJmiStGNG(caM6&0zO;iTUgaA%C97n`eU$ z0qpsHHsIDmFMGb1v(&?0?By;4T!8@e>f^40b>|^RxDL<-pamsN2s4MUQ)7o-vH%ZZ z0E~&Zc!QfeAsF5kNuP>Fo{L6*2Fe9v~ZqGh(n6p-F`i=+Zc1 zZ5Xl?;3NtIeF#7<23zGDTNs9|c@Ums@0NJapAZ~hY|1v)RYGUQ>1T0p)pMLy)_R0JC4VidteVE_) zIP(M+#zvRljmT1kFo16@z8#UJ4M)F_49@%=U41{k3JYT(CLcrWEj<0S^87PAG4|pU zSQnptn%jnVJ^ydQ5X#&K$>RH+kS=`yF)Rbs@DczRv@xh(E6}}wgbe|(t-Kd6za0eM z&)P@uGSTd>V~;)*H42Sxc%iieaG zj;Ng7xdZH|isFe2=Z~wLJ*loJr+HTSirN|N^M{pA>_2m4|Ea?V6^{RKaNqy@@Xx<| z|IfdG^@kt-x@*^uawiU|p4+c>?yJ;-QfHq4vzv}44tDz1*R{+H&Kqi+x^iCbiqdHV zRdov;Lq{XyTUYe0HH~d`ui9R|Xry}0(%9|c1Bbi!oqa;`YFnFG0AS;t;#rtK9G+%N zLCkQZbC4NOydb&+V=Mg86&b@|X9s6wP6jaWr}Le{@%kQdWxJ@XZLp|yxUf}H+$Je$ zmln1RWjFE3wf(rV_Jq8~*vzV!^wOx5!YE>XI6mDs9(mK-%f!*f?6#%(9cw)+V;wVN zLmL|t8#|3F`e)SCPphb%R#rQsq#2b+=pUHik1PsCmW88Bg0VRnz~;EpkxE)ga#jc` zCzxE4P~B48#_8-42zmqqb#1h}K2GO!TvJcO?PugPb+61TlavQUg3BAqZ|XmY@-nlzJ6xfmrfo$wQs-t;UACe|8DQ@Z}#l| z`<~t39yq9KU>ZlJl(z~Rdq-%{z^2-S(;cGeKLNJW%JB!l=6J(1T*)L$EM@fccX9@} zgHxQrNzUK|OEl6a80=tkF&RXi>pEw2PMuIadBxJav7NK@1gO=o3u|9w>q~R|@~g$A z-**5u|E2WxrT=hl<<0csD_HlMe+B3@I{m9;@|S_JUpl!@imL^Qn6jw&?1v zRC-hbfL1|pWGdiURD4b%rX&ehj3Sf*h5-Wu0CV+BaP>;K=^S?5*88qUtb=oyol~Gs z5Gpc0JuD{0FBl!0nB^age(0MJ5=o3p&WS*fV-qtI(fQHw8NNYC-$0~qK$82z2>+nu zkTAU4!x)FVfzEE>>De{olUoy0FNUPsq9Irsk=0H%xuSIdFtG$0*yuQj&5^MU>Btix zUQ!SMU_)^F2Kb!|Lcp8jab~#e84hczuYa#AURx|1kO-Y@N5x=95zh84Xf7?PqEwU>G&NRschY$MZ3EmcF{@L+ z?i6r4dAv>@x1B2hpF~HJt6l&q zq$U!9N+i$-cq$G{Clcs*936+DVbL@knoh#dfNT+y8L41{p%F1u0(_E@f@P*)TFJPM zbbJQ@*iK{;+Oh~8Ii#+F)ZXIkp5naj!n{sOZhLMH09Z?Q9y2G8mX*_xk=2-6&{R-N zE3TlH)l$pqnYE18#?EdgtG`>sWsL|q69UeZfHTeG0N+~ZXD{|~mVs&Y!p%xA8|>G5 z*((5AEdFCztU&{V0D~>8(7OI?kr!_4(2KiM^!kHp;Nu8iyeSym5)N&PBu@uMo{L9c zfDnzm5RLsT7<(xg`x&MUN5J}vU=--r&p^L8qc8gJ^b5l1TB3&GC=R4NfcyNh}UaDhUMQ z6`L0jR}h1j9p@r7-3cJuxh3BZAo|DV2gFglqq0Mii-DzuB;*Gnazf(sVzK4vB|YeD zMkJ;x249EDWM)-x$tC@mY$hs`mXy{6a7f6erxkV(@>&y9nxZk)vG^Kv78RA%oS0UR z&u>X2*9XRC1xBaTHnBz~pN`G^Yi#cKiRCvFE3YS4-^@IIH}~Z2!p1vzG1y}OBmk#3 z09QlvuZQN}09L_l;o{rj`PVxD8(aP$yJiJ8(8JDq;Bb#W&Tf33+k~z5f1DWm2-_EB z^QERY;0zAf{Q(D7eH>YOKePaF1qAD(WEr6AJ=lNoE%@-=P6~14k9)}A^Q-SAYagVt z`&_0r{xiDv4l-=*b2)WLP0#*+zCZZ)JqHe-S30A2@to16bB4MKI%>x+oQC%UE6IWN zn98X`=T06|J|m}gUS9qD3Dq;lE}T--RXL@hctq(Wh-2sF4xKsi?VeqK`TmEmf86!g zfBy7Wu{ z2{RfvsntEWVtR6JZ9-;wBq1*pO%6s80}`>`v5B{Ryeyq>Ub|^^+48Ecxyf}$CkHnV zi(9v^n3-$o8>(ns(!OF!$}Zs!&k9GDgd=eLhYT;EF;~@Ba`AS~$jdCPtjwlR@ad^x(V^}SZ#%nMIRW>&YvkZ&>g0Xn zo}ZnwkF{%ntxuSpe~?vNtX)E^gRQm3>GMYq$Q{{z{NVQo_kOb*&iVbvfdg_EH2o6@ zMKm@97;A*d0|qt?hhq->uYv8{L<2|o56|*t)9BgaiC+E)dtib$G|d^BVV zXP4hhEdcj|$-~j9e@iC+B_98oBY95mnj>e?f+JFcBZ)cS592(1kpN?kF5!U@L||Ys37LRc zv5DE>-TXswf#HOZsFXYR!+?!N#bp48^A1Q3jU+`xr}zaX_y;Bhh2rl&h;eiY_Vi0+ zw2G$ZUPz={K)-~8TYOpBAJFTIfeo)q0lv@{?D1_tzN z7BCD>MC43$_ekl~p8Cp~`ifGRMXZ8-95Mrg0Ao~_$N*MEt14-(k{MWS5u>)4CZpH? zhO8H{xwN1>uduPQwW+GRt$_sq#_1As!LNQ0T>=0weg~J=&gOUV#QlSkzQM-qVnSee zW_V0lBDxAqsKuo;5Yrk-X-z3=#~>R}$#qEyP3R;h0o?)^mWl=XCCeWYI^mJ8lZ=8e4MjJ)K*y4Not_yAy|KXXQ& zvqzq@hPJx|8?@flissQcWQCVM#wQr*7lBJ4mPH~99)ys5qqAc0RlvXix}q_aA@PNN z(b?h1Vn8p~V4`1Ku77M!2qOPsDEVPzdO!jtF^!s-!9=7pT*Gs$?j<;SVS|#2-9yPP z{sezSzHd|}pcf{iF$z`e7oCC1Y>rQ<3r#MKKvyM_srcLud`>%%xd>z_Y+S@wlM6fW z*>vz%M`5ah5{g5SB}wV^L`rK$c~5F_2R4Tmh{*B{!&TJvNXEBErhXoq{cU{V)x`4a zsnyr>Pu?wVe3)DRF#QPF)!X5P*D!H6|5`lz`|!dWu$L^p-l=#&36mKZJo2^n{>yYI zSoDIeAHd$0pF)5w$Lw#V)JhFM2on=-v2}4?C0Zcc$8ptiBtZe+>wS$!^>)r}dW~4jnwJqHSSe za`*Pt+YUPBCg=6EpoFQOIjW*?@Z5=`N~h%2l>oiu)D@3kP&j-+{)GBzIdw>~6RM|A zshvA~M*jN)`~SXcHw4&^5MclFkV5(YZIE;+kAHnkv%kR66e3r41dW61%DsN3G2W{!4-HkQ|J zIyiaxxC8{<_Vlv9?`m<=QP0HO(%!kWjtOtok}mP3i^7owA^1+^*VhpGOXSv-hWO z_x||9!TqOHH9aGdh0Xo-JyL4_7?V5MDwu)*let%)80yy_0Gk8*<&RC5dZvC-=dw{M-Z{{Jr zUd=B3KE3dIeCF4k%pugTseg;cUiJ=bR5pq+gvy9ma+K`&S72yLBqAp=KG)3`ea9mn ztbyQTY*c1gOs1Q6qJJ2{I~eEa9&L9o+}0(`%-+Y%A9>F+&ekc|Ka3cUDhQ7yhenY? zqf#Q{vO=POtD*fv@DF^F?>~$MPX>n*{em$5!I-GH^w^{sNe*{{Zrt>a zM3BY9t5Y*CB_rG7;cbCgFOeF*hcmI38Jn#Mfd|Yq2RcSW*q1)R01MBBj=2aSbR`BO2X= z#x|ibU|)wp)?-oicuYM3Q-_m1#Go2Mpiy;+2~8*@9ftx8YfZ(qr$Kv9 z^QFJ=Bro~Wm)zl}tf4KQ-3lYS+XnbWH_&Mj$BC(~h#HxU}T(7W{FhqK3Rp;R7 z#_;&_v6){d7G6y*y_#8jGynMg+!IKziRHJFh1WaK8l3$dFbr@D&}(S%6?js1L)cE^ z#z#1`UN&xOdi~?fCSVmjD+cJb@(coO0X7-RrushlJo)(37?3Bxrxh8%fM6|u!5G-= zpbKCOnARv%w2xztK8-(uUoy73^ZuV^Ha|{pewcgudGYyw7N5as_p*fKea{8#mkz?20CiSmlZE*C}=95(l~!i`ScOx zQ^!_2&Y?}?-PP9ENMbl;DMckhwkcU1Mn=_^VamO58#^e$b~ylSd*{knmUt~$^`b8G8> z@TiA=K{s7IQVJ@WY^iJ-{S<3(3IL2fvdEFe&S5Xk=sff-SbSRIjm^obUyHB>@yop` z@TO*tWUhlJ1q`e094LpmL{TwQP{a@x(D>O6{aJNADdkLDQ3H-rht8`)=T;_Xl_jJX zC8QU|kaGPKP>$ZfgYMt;3-XSN^Nfgg4+*{R@8|64ZSU;v6B&kkHE}M2DkAiTySwRm^%J{}?mZxX zV)yZ*-|XM@-M$}xI&w(M$TSL@RmSAkbxWz72_|={l|R`ff=wK~15GwhvD^LK+8CTP{LT^p{DkBd}eMpPqOs*-wP|hE-w8B#cRjC zmLa<47k>roGE67_wzBqm`SDv=`dWQ6yYy;e=HKIhTT}nuiLvS5hNpkyNuRZ`7BUK2 zfK&xW;KLAP-(YNLbb1_;;vI-{@rZMIkl-Cc@(&~Xg^&Ux$nJj0-a+U)u923G0oHdz z9b6+FJz~7Wi8t?syZI)?VT%yS`3a~zG@&>Sl@}bD3UuwEAJQifbLU>9cK|XHkron0 z@OT*e&?mt=5E&GK^YBKvdn4|;Mci=>bGjc=P{Np*|9N!$>7aCbU}#%3xGflf43p{C z7GJzEFt|QE^khUL>zE%}9Tcxf2iK(HHHmmtI=Cu<0E6@bv6DsQv1eHQ)7{-8bXs3k zc~xC`DYd4&xu&AAvTTQ8^qMMWZ6&>?Om-==Y$&3PUeLhc>1GJ9ouI;&`aA%zn!>_{ z@+MCAP;aZSgUVrd4Dox0`MrQ)z{B{wF0P7~}$PHkFLf50v4Jd4F60$Zqxdw%*L8Gd%s5%^`9*3#J0EVF& z&`2s416-{cjiO=EEkt~43cfWJ-;$1R$t1OAliPB$+bDUhB}Hu&)$LVH-3@Jh^j>y5 zpWiDH_Kyiz6GGms0FE}EX7_`CmRS~Swx2cI$C~fsEV6hj{p_WF)ak3fE<+-T&<~+^KT$z)x)eW)aykQG-9e;VqeOJr_xzi6k#T z2!{c^o(qORyZ|8_0uNsbq%R?~q(1}k5=j2V8+^$T$r=}jwuNIaM&^GXU3fJ#`>IQ@ z9vq!(e>>R2FTp>W7?GUsAD!+Rfb$4V35+lBi^=ne&H*MAi7pR78K-_oxErkW70|v`0inNOGBHRL)(0oL@{g}S0w~I*lPK*7d~IlaaT2K^wV*ROwJ9>G*e5tOI;l7TSCvSrCCKVz38cD6TzLqx zNLF~O3QI0TWi|#S=06M}#w6!e)%S@ckHw=;hsR!wPX98s`0MQI+nKd@u$g}O&FI2w zDbTNlH-mGp2WEZ)(1I{qc>6`a{s`DBFPncj{p8c^#>cs>4-4Dxm!H00e)eJM>4$}_ z53`%_Ffzc;=;}Mk;v4A_teL^LT9g3{(D4uD0^AyblkY#nY#}hMwRf_+UEp!Fsr3(Y z+i*h1!ZSFeW9}Io(lNXBpBXsk9w}W%9Zn`1<`-hX?9g;t-eNj!{ z;Ebl)VMV!pr;ne}RWr1+uyk=Uaj;f1yaX_I6gZjs*`u0g6)q{O8fz;XYMs`F?RY;P zIr#nl{on1~2N?Fv-aSCY{87uIpUR8;2{_J7xZefK6Y#X^%_1ne!Q$0R0dOw<@P z_JSaN=tJ+l7ZF87R5~cV4a_jT_XbR**cAjD*g(Dazu$8(_kQ16d!2O{Ml8tu=GpJF zpZ(ic8@^rj)wgSZ`swR$zS*>Vxzo?vwuOX8g@i}%J#->Dx1^IHAqxg*5BeqArGhQzJx=2UdB%3B#_ z&9ssR(wUmh;@YmFs`k8!mfTBCS!E6B=W7#+Dq{1_g{5YBN5llgrUWOZ6Jp~0qoRDn z!@Yt-JP5&AXUclnG7cJ}9>F+@(m;S`fH7>lik>MQ>=%fuTdzfB#2oNCwE2%MD}PvH zyV2Tqqt*H!*9Jz1)?9CFY;CHjsyuVPEFm+C5ElIBf#3K3wdJJ8E-%7?(}ew=!G}CT z4juL0eawB^DbH;uPyTS^=nq@AuCsNuG+So7)MeRX0GOrOf~8BWS1do|9dM?qy^bWl zPF1wAlwF(w5?=!VLkdic(vbfLV0t{%A1|C?P#g&Y7}i=GVoQ*rqnpa@gI_j0H9R(h zD=h=I21PqEtsi}E9GwA@4Uf(Mz0k%lv~Br?VeEy*I5Pmk3+OfYR6g)TG4xcSd8W|J zh}6?`!TqZ1{P^_C!4dhfDJU==07eMS2_)uw5z+{u*-*a0S%9MO*s}rQxxU0KkAUPq z4iSF&!(;c6;62AePX(m95mI&@3h)if$t5?350U@iKQ^qI>Qs(zJdCOf_n&H z!h6_+(0x!~BJn+m__h>qCA}pEfPn%N-H-~$B!W?y7 ziZ5QtEWDhSSDBVuosnOck=K|3Z{4%%Q#0#QGip=P>eDhBGPAB_X9KQq9`dCV zJPFh!S3JcL&0~xweasY1vLusi*%Vv+kR^BkqD`cjlxdzSjn9?CFT~nsyus;C=B?8K zdHWBCg+>=drIm&yo)1sD;2%|dnpl9s3B!s!LJI@qFA`I(1jSznO1Kc0R~MdfH9Ds* zKCdw*8^q8>D6fbT->7rJ$yGiHR}K>Lw;zu53@;8#sSHfGbRsCnC!)v)zFK@~RB}aF zLRm=Mg|L*0*xWjxqc`zvWLkCVnbzRgiy^V4AyFmC*)=J74JieUAi02=gqZW7`2yn4 z6H_W8(khY)>XQl^h-sI-!VCO~IhU(@0bl~zU9nMK~Ip(Dpz%UNYB0oKT5Ow>V_72LSAAay=czkYne0F4F4$CjIg z1vYf=?chC>0t}_P{Z?`7jpFuO+0A!+<19t`NH+SeiE7yJ$e-Xef#;gZ`N)7dd2q}tX4XH zY+?H8Qj1T`Ej}?fUtngkz|{0ZQ_~MjP5+OH=>lu(e|)s?Qjs5R<2*IaNXqWw9;<<#uY#O@b!;Be7k1diZ8$WCXg5z8JGCS z!6U&jNgXskS)igx2C#*edIn}dU}6pqqLp@cc5JR0nL9?fsI!N5 zUr_|AcDm#`ncvXMuO)G7dw5k{?8|Mmvg>{48%Yf;`PBVzt9Z0P)iXGVH=szQWccvi|B;_hSz(B z>9|rIN20+Ln1+iMT@Eq?N*YHopt(0Z`phu=?0*3^H2e%0W*hL*Y#)gzhuv3w8!HZ32V;mMwKQ;laYc85e2`Ua&ZXI)ClDksJig+%8224%SW zB%bn)_Y2MddieyVx%fRH?Bi?ao1+_^T zmm}iN0>Hvz^TXqdVvfo{9 zoqu}%w#Oqi`;ubtfm$;qS3VTWCIDcV@Io(?|N5TMO{O2MSWpeP0rcs?4p|NqU!X5 zD=9fwfo&Oim05Y!IR$XNnweXbonMn%SXWToPM;^?MPP{>JQ}^HM?_yKW=nxAfOG)sCB;p@PH_Um642N@)d8Y>I zD=Ls51m)j@XajTsw}z)syf4-#M9@;BZ7l!BPRQn@W3!U6S?SF;k{ffP(Ko`8x6Hwp z4D}1~=)X&9x!-L+wP1Dnm@oNcsTusL&z72hVqx;Jxygr17Jp!}lE#{$!Z3rv=Lw8ZpZixz!k`q?MupZ#;;2mf09;fH1)FR=K;#KB~-&EiGopDkbE z^vl*Ccl@?>)7Kl;f3?XsI4mwDecw^HW1fWT-870&MHLUwrCKz?EJLe6{vP~#PY$3L z?xe>J9q68mJb_p+2a-f1JoPY3p`!|i`nalgniw$F2x^SPsqCazw2;f2doMPTN*lXM zYMRenZOkvP&N^R_cBU*o_iR)~URYvkKxCXRF#?3w>3|^5fFL(NLU>|&Gl_vfB}WPj z<0zJB1%<`rqlM9OT@Q;JlM(;pAK$F}!O`_62bV35D}P?LdW-X>ZL5FS{ndd}`}0eR zY8x7>>l-R+>QnOy4xBi=YtPTej{olN^~Z6K9jE;FdWIf4P) zg@`5jVi-d$M4VnILj+|?Va-DcVi;Qr@@tSO8tCW9W$Lk!v1i8NXL{ok!^ktxTZYjY z-RQJ#cv?S#$uCTS;phUM>5R{YG*1SASgI#d#UrU=3SJI?UW%a^k?JW!@Svepl37q2 zop?4T>3nL=mH70s7qy|%J*T(r_55*{ z+hMQh1MZPK4+r`ZbJFsw!=ei!M7Dd$8!D3OwJgS685%T*5Tr!u99fQ$&nDlNIwSz%zqfuLF^mZ1#lg%dc`E-ep zDdkgzd@`R$;?UcueNDZj#vbxD5~;4MtEQv7zN_zgFQtXj-$w0krNWigL1*-^Ib`_a zVlG9(qe?jx35P1@Gl#fL9fPJ}(6ltFj!f3{k+qb5J-OdNp&EKAI#R!mOfx_My84+$ ztVa6ZTQmC6aTn?n;%;~b=N6rH108yCZbRWp3%`fL`hj>m7kLPI@C_2;Nrv}XLUg|M zw^v>ueF)sb0QOKId&rkfa)4D5q_kLq2`G->AzSp2Bc0+%9&yEwxuVAaFt%_4iphV# z#C{i{d_t*tHlUjs&`-;CPi6XNV$D;I>M>I}UEiZW>63lnSk!T!_#-}XyN`z-@k#Lr z%R@o(iRGTeGp9of2{EPNsaM>CbB_9@dW7bk3d!}4Js*^CF*>6Pz!R2y*+2R`5Y9io z+%x*(u2U($9g8^`0647ljyQKRC^t0mf>%feRAAJZ_{_?b{Dy$2lB9yhgaXh~SAr8S zVMAsYiAfiIiTSY^m$OTn(~7QToNr4my5<*g#yu$ebV!~TG2bt$FeLFp>e(g$7zi@& zu$+YSau(kpl--mn#|QOK0bpw5jLP^@0ou#(60;hD0-J@BVHm@m?h?aG>F^8r$V=tu zYkcZia~sWyjo_Wb2^5?-{NUfZ+jy}3ET9hSI-)Z(>;@U-@sc5W&55yLF9amxE0KO$ zeQOqQWx%t2AA;(_(1m2&TjhpW#{QF#*Zk?yjUSsX`)G;f2a8QUu`n}pw6$<`T(Zm-n^SgJ>f&U#W`)z|YaKVP zT)p}8Z+8Cl=wP1niCksqI28?}V_Q^jNfAE3HhaWEfaKXY4t!!^B z`Gq$jFeM}B=t-{~dk>b^HTR=Pbv5QfG~)8>{n+w<83tmDr@X;MHHJMt{66Ft?Q6y* z8Ym<*C8p5R!~=bNMK?>_+|O(1W!H4kE88g*Exnhnca_z*l~i3Ts;JE`tIR!rIrH4* z>~j?vMHiB?i(*qViE)V`(XqkNF@cdWMHee6Tsa5jTEUl{caE@CV{9d^2oX)uiZaDu z2esEd#0{Xe{AWi1*ovQ*ZP>nY%l@x+9{I^X%BQ~NT4y(@sim!^z9}{T%z=|f_x`oR z-D9t}|306feZIs)Ug3w1c>K2i@RmRJfB)0A&z)DUz;J*u-vbOk1_OZ6 zg{rn*HjSs!j!XfT^acRfGqj5m00!t9o`$Q@IF0oWkt`d%@dBQv(LGfUJXWfvWb#Rg zY*H+n1cu26o+`C7(m|9R)yEvWaHTIHy(}T)LSAu0a@OUTl+uLMGEigQ0m+20%)s!x z(CDJb_!7`$hfhTu^Nc?3m+wPlvH(JBzqtfKM+dbMR0O{Y)FVq*c{P4f%u+Sat8rSdP9r@OoRX?7K{po zBYeTI5GDB{1;%1%SPTTPeu|n*R#AE-?Ja$cHMLE32w=^1$Vrc~eXrHGH`KP*S9YKf zsVg|WaDZX)92Kqg6>z;?dEwgC(x&RN=IXN6nu^OMXB#W)IW&QgEf=tqPSy(Zs3MyIJ-wP!}*FGpYm7=DQ2gs-fd&$~fvW`U2 z_x2k=VD(WAP^5lCAAO`3Bo|{C(_5o7HsA_G)ffZCKA}P*T)bO!?rjF=4wG|-!M+Ve zXJfVt%$@PKHREq7jM{^M87v(N0EP{6O$sEFeC)f%N4>#uc#tQ1$dyiVWK(R}Bd+KX zM=;3|P64YpqA8AeiY0=2#1TGX2_{*>hd?hDPA~C&`M?u3ysGM+s(aXW=N>!}dDJs{&+*W$`vU$r9(5uhbB|lBM`$4c%snLEFY+ugsUjk! zA}GGpi&${fKm9~tmQQ$5L~;co@@!CCX;=#4kY8+>f8y1HguI<6V%!1&V4$3?9`(!c z4KHx>Pw)y!_X^2M1of6(9hqE_Rf0CEMCVk8C0BrY3roEml2GauniZX19+OcK8ebZn zRT-Rg(LeGGA?i$6QdxBRm4N8tsEn(b=i7*BmwhA8oC-*fOTWkz7=-fMQuTyd2l{ea zX_!%f{4%_d8D3z%`PUNNtV9P2>=lw?NQB`KdaW3Fr5>Bp+j_55E!)zZM!^3yrhlk+%ZF99=yv9{YDY`}P-G-9Iu} z{;{e32aC;tVGFG+&0HMKmOGg`+nX-4Gjp;waj;qf#83=x8LZp#L3dc z$;M=v?IK6(#ST`c4wk0&7K^M+7g#L*z{2F8=BEEJb6C2_+61)MzZNfm`pD$t#Wp6^ zuC`0;Og~ui(Fcnce7NYtj~0En#A4CsU#$Q0&)r+MZQ1nYCb!cbY1uhm{(+mfZA;22 z?BmM&MJk$j5Iea+3oY>pFsu-t-Y;by<0#Q;X8bq`?ynN*jIbYdOTwYah;c{-t zg|z&lg3?RPy-X}w7}0cI5#saIz)J)(vNN z8Il@`NJ9}0VF1JYhiIGS{23TSqGMww4jf=C`8>dQG7VF#sBP)K+R&-g+&A8w(TzOQ z4nG5Fg%}3(GJ+tRHjK>Zv6@fcxbaeFm{AWsl`5yi@UkMEfH%T$BUL_<4?dM^riH2} zZ1ID}R&i{~#lVQ%@VLU1tjn<}rNp>GpP-aeK5@V>uYlALM6YvDzCoFw$^1exj(A4z zJ`%L^V8E^e{=l>C`#d-QcKj3}F)^duCoqK&mL3|N7amg(olqPWm3QP6(K|3XG7hy6 z38CqJfk}P=iOFfDp%GaDq3OUgpxvQUVL$zL?1+1KQwLYAolp)d;$##{mF9I~oi;@4ALyqj`^oaI zw*LC6nrpRJTNua%4L|_d?0fyCOM6dcwP|dX$aeB4XAX8#(MOkBI z4XdBSXG%pJz>K7?olWZ&@+dM;NIZ&|i=>i()5mA^a2Q=IS_gyDN~1K>sVz*nQd(H# zW+thb(bK~0Zf5qha=O~N-5vbSc3xLIkJKrI>gf=XIwd_FLQsovp*#MBB8Vxdp;Iq#_AV}y)9 z-C{ffay$aFK@52k3*Zs=kX$cfpy88z?m8ZiX94@i zoc9XLck@p<(Eb@&gglEQPT#8I7 z^$gCANGp%atB%jBPARC1&AghJTL*$KKDRzLr-2Yx3MdVVEzT^eqjB^?`5nc;q*@Dq z%^9Wsxm-V^&_9>yUx@V>z_89CZli}QZro7bL=Jn^$g3gLJe)P$e{Fp5dSqe_=w-O~ zX6V+O`UZLeKu$LN5*hK|V-IwgV)U(S_>IhnZtyq{&|v^D^{qF^!HDfu!6K&KVZ+YI zf{1YoH{4_NLRnLDVguA0>F`_W$UD*KTj9tXzVWpHK6UdQTlbPN@LYB0KLwZjt(X7& zuf@wgGO_*G#OhOX%g=0VOkAAIoSl}UqZWHJ7dta_wOPUep5 zoNQLun%J9t1`nEj0p!#Lq_rtZOgHqYrd$qae^7YP34Q-bj+bSAbE;qC^^-w8% zB~y-48UNOT^MN3^{K5gIFtDVmGndbOxBDyCAM98B?6hvj>TmXK+InpBuSb47+YkrclTav?o7`=v)}FH@B4Qha@*tOx7X9>_miH#0=^3h7DFLRvmKpt7)fnGUYv7RS!>1;tyg5dK83670+jI z;OKgP^Iw3W(K;D+`X$j&d9w1FYX#@8Fa;XpjTw#csdo6O4(ko#07KeqWX3o;qaU3? zU>lhRfa!*xtB0P+RgWd|DZY3dJB^qCfGGyhPm4@5&67{{u}036wui+Q6NuRX;aQ&k z$v#0Kp7H=-{-No?5qTj|1p(oCQ3<6<8CL?r@`$l#3E??^9t+udC~*5;?|nytj=D$f zJr?}uv0y@Y7W@*Qz+~T$R6=M5F*-jc;VdYs6P{70{Sq*MfVcT|}f*ZbcY(O`K^$#BirQ=*N@>}Eb>wnH&?(u|o1)|%0;g~=?iWC?+ z_d@4k=txXFCKQfxF@UjIdKODhXJ`?_sDreARXsbwi1d> zkx)9gj2k&r-0l`mPb-hq#sH^4SikRS;dQt2I$C&LZ2}-xSKB<#db^}heO*8=1%;%b z^eV}{>b^crU!S(OPfMmDXyNpt(1!6M(}yVxbiPGHVhfu%MtwqTmh1f#7Zg0s0m z@En9I?6~U@NG=9QE)fW?hip)Cf_oy_xLiG@(mqyd9w~;V6dC~7WBJfC=@7`TX`yzS zGw_6^enOKysccb)C0q%PKN}cRc+@-b(CI|4u!5MJ8o!uxarupjMb`l^M|{&x1?HR# z$aD+HbPve`1qJ{+?49BnoOjYM16V?cEj<~WcRJ$i>B!P!K}EkEiS~^wiOH@DOQ|44 z7kdU}y89;wh8KDUWCcVP`4S6>iRBR~mjQvNLUMz#K&enP8GSJnenxVuh)L&z<4RJB z>a#DjMrB-1%Bzh?f#+X}O1~PDT@#*uH8A0VUsQ2e!iBt&78+YGRNPh$O{#UzRKP9G zv;-inosomSF}#$b>nkz9O!pkFQUfw2qWNXCKLwc;HMd^t?;>ttg%+=9ZefeT#@@(> zU&)Lw&155QfnIQxjlPwPz7s>CZseR~?5*_1zd4$hA`p3_bE(D64r{k9 zFmqYB)McTC{U=MU7uwrfxGuL^xnk*Z7c&;PK8uqDeZ@iKF@F>|qj z>(W(r%f4`R*ywEPV!6=K>@z3xf0!+^v2cO1Ug~OXw#?kz)zWgMjon%YhqaE@%WX{T z%s#PL_#xI*G`C&4Vbi)Dzx}dx`w#o}?+lF~_ynE)<&U3#*}E+)HR?h`bvvC$;SXYA zz9>1Djj#TCPcKkiEUZ1qQR;bWBZ_oFM_>4<7{(QBGHiZU3{h>Er_^&%WCmL5$d+nY z;vuGZh{9L&u*IEpL0dnkxsP?7#AxiM)phpQw3Dk^d+XZC*Se@3{oH=Of(ftwI5^+a z3lt$zd-E;C5#T6SuI;AN-GkkozqWDy!DhuTP8)w;{li~h|90%BUl08l5aV~fvyH-F z_fi;LeUzrw?$o@(KTn+e`Hx*Y_WgS3_>LpTwj4gX<>cvIN8NVp+VkC4->$M*W^Q7= zXt6bDWOFkI2dm}F9XD>US+m;A$=N+1ys4K>Vk>&NDiTi(4C@mNV)R1&LkwVA%su~p z4h+2yFwA!mWoV5V7aMB6($E%}l+{HM=trOGj88TCry2t~@zNO{L+P=$A>KtqR2zA! zADh+=PpdUgrHV%)>7-En5OfS*f)3K9s>f>Wj9UL(Jotnqe$d*hicK!_2}%!&%!^OI z7!;oC9hi37Kk<}LtY<)?yHA{7NG9MdHn}_|=^`O4-#;`D-i_`%LELjJbmw3GM^8oU z-0QphFMt2Atk{%ufy68b z)h&&6t@SmnbyqtYsyZ95b~YmU_1<|{j_+k|t3x;QFvI-x^8;Zu7aOmZl@}NHwvqUB zv4EujfQdPSEV7u~#$u8LNP(e%L_V9uWp#5|T|7n?m)60iwlOKKjQ%!OZ!4p_m5$`s zb!OK!W=A8l6G+y~>4s|I0>ipncs*_W?lvB&UEJL&>~0nHw1Ey2V+@m#I%U0G^1f~* znKaNx8tU!UkSTg7WW->M_R~i&Xdzz)g*HNEj8QN*JWemvCj^a!a^mr>Ac~5|;&<^I z^zV#%hFA#i|C6bKWU*`liiZJ=D?;v2_%ckH)bWN&2Zw*Z>K#!`Ot18gzPR5j699ICkPQz8C0-1S zI~N*P;v1ah8=MCK^9wHsODv1cs0xfOAw-qA8(WFi4qWyIu5K?xU-Vv8>Jk2wqaFDl+Pua z8Hx5KWHYS2}{Dg>}sh5; zRkz+KZoENyYxoT+ETltsHSj#c(Y|}T} zk2$RQezBGF#}>8=Z5_;C%U4(|cQti(T(Zmt7`E8S0wfo33)Gm+ zIw!l0E>2&qbo$cOZiAEcT8E`;94yz_|HHx+RGW?E3R@tS!#W3>)piaWmbrYf+zFnv z!q(EoW|6hYhb9XaTA7$TEVXsE{dV)0+kf4>fB%l?*dSsw;n1nQdyefq=6@pVLP;x) zOXjPn$c{LO6vO-$mHz?^0|_WFG}??ng%lX>=J;>T0}KUF0sP=O=$ZIpVy;||c05Y8 zEYTocpr-MZ6t;{6vWzb5pz_9)_>a zQ|hiZUE8w%C+F|1mv6EEeAnt94}7)b=uf+jZQFhHx0Hf(5{*t{^2t&@x2o(kqQu>N|#vlLiE>XF9`E9d-bw)j!r7p5O`$KGb8{%P=;hADbD_K9Q>* z2_=(Ie9;6tX-7?-hl;_c1KJr-U?SC2KYOh7GATUv3?VElA@fplRz*ZyF)$1?lbd%8 z$SLo@WbfeAsO0m(QAHkp=_kFC54y!1Iu*U=C@3((k<(E}+#-J5c5Kf9LLf0GD*g;H zIxiL_(4S98Ee(y#@d`+cOgIx3ofkyRij2z-3d`^dO2L>AmV~b()X>zt`Twgx~)3 zIOQGFLy-;YCS?N;CCUkSEfY#n3y~*6HhK=cu%eeyTngNUM+CxqT>cHA_?8Hrgxy45 zM9Gc$7+-;rhXIVkHF7w{{{830ya)zMhpi5OPZ-T*NB zyNigX!SF(Y3}Nh2Q&kxbGtgvB)#aBMxCL(XT)g>03SH^`!hDIIJ^kAO>& za{7eq9v-Wk$L!+KySUU2Hoc2O@8tA%uzOnQJ#7q9E2HZ=wWpcU-OTE0VuIxA#0u0{ zM+>*B4dfTUyIs`X1|{rn7k9OR2$KT6dO8)PE@e-bs<(TP+^YroMJDU|D0+}zKrdW) z0k^1(8&EXn&3^hAauTv`(b)58Pf)n1#m~2JZq1`>o?D1*m|p`}HHl0ak_kSRy33Pd zF@{29aK~IW6Kuga2q~s$926Cm_khYp3%!sR135LOSi&im7#N1G05gW*Axi+N8iO1V zQTjkAf1nt8G@yF|?_h9x!H0^WCko9oM6ZD-;=yM^&2zryId^D=p+EpDZ_UP;ECwt-bo>G zXXCT0QVXy7N1gEvFAR>q;1hW^Eahrw^5ulQhUl!Ski;^#z$`aHx@T}UA*$FjBp06{ zi_NHv&908ls`QU7aSzS$3eOLSDo)C(qOuLrf&22IN!j3I=^zk#My>;Ty_D!*N&x`c z8L4hYqMHGL0k>4cub|ZE^y}5&*h}rrm&UuZBM;sh?!VRCMr$xtqq7PuBN3@CBTAf- zAs^$c%rGlA&dHF2aaOFy{1<47`IY+StOnCw#woOyS&Jsd-r^aFiW?~E5XdAk&Poh` zE~K<@`6V3zs=*^D{sS)+MmyE*w*zENaA!9G)3LRUfVM;)g4KhRn451ndxCQqt$q+&vK8S{=e!%Y{M_(MHBkEzIW+wWB zfx^>WsH^^N&o{0=*{%C+`M3K%-+FlS{?mIrLp(y_!>=?p^fP%hrl6nB>7z25+j~-S zi~cx%V)akoTdZ`jUuD1ktL5MSvSHiqZ-3eG#phqIuwQ9!YQNOf!D6Yim8qTCr{;@4 zFknNBGh3x}_D{6sC=K;pW z7BtiN%I0o{P(ExJeS(Y|`lmYmW3B#)M)w#2%=iSM3&HFOfK5L-V;Ff35=}8MDUms_89<7KKN9qm~8C<`E|i7AUPx|FE#gSOzMTmgtLI6Oi0H(UsQ6-F zSWIF`MC=(5TY+I2#Mu0hs9Y>Kswg}v+ubX|J0LM6A}2DgC_14y2=mY%atqzL-)q~h zlS%2Ns(}Xs8jxQPk^GW8fWi+~09UBy)1)vXBJU254|R(#yvY}ih{R(^gkd@K0?~*- zI1B*e@(f(Afz38BnCKXcK^p>wQ7LLFMcUEaU3IwiDpsN+j=4h_DR8@BF`qdg%FPlvfap_V4ODX0K2svsFRYvU= z&`BZ|NeJiy`NirH(7Sp4oorwjwSz@&WAuUuYh(9-4r^g_H!(nmb%GRY=5)4jafWrb z3!ngCovo;aIM1yP{3-)_^>(X!y3`~9SdWI>rvv%b3nGka=*L7D&7Xl4tkVa$|)HwoFuQv+8Z8KM!3|9~w9fK33L==@1K z4<3Aoxef6f5L=Tp;X?+fG|40z{%1IwPaB;Glc4^E*%ox`UKQzM~oMEe{naZbh>C^KyLhtbNr-Sm`2pK-a!sEW_{xK!Qw9BVJX2qQMi!F5z zE%1&w8a%5&@Y)*a3nWmJY z>%j>Zg5pb$`KJ1Y=Z7Yo!=ILzTN9IB85ncUC%h;u1<0CyA~3@{yzo>&T6D?<26tGh z9+wVG$u!R-gU`f6)1b5@2wKnOx)&hG5Yh~<;5sIf$jJ;OfoE^IN7F%WcPCq`s zS4Dm6EoRL}bIp>`IRq-hoY?qAJn|Or5Q;}gIEieu_PKuZ6C>&_GEmsq>G`J?b0v?XUYCH+;R~ z>+e>5_nq?>n;bu1ZNFi;{RS7?jV|_^R@i;9(&5Wh4qvRY+2CTi*6|-UYnEBAbg)`! z4|;2yr-+n6n~Ub5JF@v>ERn}7OZ=k6_m z!5$e|u^wIrKmT^c@=YsF5d7=A$sG(%#`$wmS=o&w8Wp&LI$$6pj7)qcOpc9z&DRJT z&%)Y_|6R5MH*$<^^7%$02Cz}S%E&_i!ymz29#}9H@-Z5?*hn%Dl@6|4L@5+_j}~bn zTwwqM_^FI+C=95$}lU+iADdEZxi+;(OZXVtbgHFR{6m>dRA zL}v@=Odf^KZffs|OUvGI@X)faHkz+=v|QeOo7y@V2~yP`K7^{hd98rOsruD z#gN0*fcmjOFNKi@N>rv}2-IAOK{xzZYj}c$7gAtw^JJb_7`=1`P-dXRo*B@=*vx?D ziBvg>+B-sc^~a`dgzz#kp&WXG$InaEkC}qomuslN8=nB=umA#uL={9Ol=y{Ycn78U zhozqmNI2mY6&#&Q2+um<9Z!fXIN%<=>u}JaQ?Wb$3fQsN`>zvWdkzI2J{18^4v#Ag zjm(QnK96m%EFy$v`UEBi5i{L9qfdH82Y~hpPC4lr;UAjfACeXlnL~g_LeqVMQ;xV3 z|2W|L``*)6s(ZA$Db>KFOf@M{JcJiDcQxRs#Dm}D(Gq#lK@(vx;o`O9deLgJzdJ~E_F}$U{B8=3AGTB z`LG9cSig}Y}`citb*MF(0FsOK!k zW{u$+feyOR*cKPw42dTAk}0-mf+-wl@Zmc$!Ndwcg>FzwaFU76w;r+hQ!M^NHg?tp z^x}%9*kUxB%#k2;!50+D9|~pTBE^JEJ*ga=l&i<(0}o{gU5^mY1|A`L4L(&2K2vI@ z6}oAe{<#1Y*w9M=7)$kxDtUUoT6ik35ag6sXs!=2|Db1*PgHSiUTsK9MOa#8L`Jno zNWuP7@u$P`p^o{b9}mbp=$(AjH|+!=BPi}XF}1=oywES^ynoF3$c&nRgsbkvk|TbZ zVJVkmvg`e#&z%l|H*JYtp_xH3MLwZ9e&KoH$)*00`KN<20;7xl!iyqPucj2DU0)d` ztw%kSDYJZdR;qs`$0wAL)$lbs_dUN{YHHkfCtY6u>A*{gPv8~c)=SQ?_jCo3(B|t z<+lE(Usrtlz2oQWZPq&5u6MRx?`*$mmHiiM9lu%!H&*LiE!R5#!+Mpy)e1YiRgTun z?UvY@nb<8|Vr#z8W{LSSOS{#M_N$!$U}g@M@To;MCRQ$%uIn9EZFK(rhfSwE4ke{T zxVi6J|HU$wjmtLg-kx4^wzR(HM2OF>6GyU2FLtrSRFQ@$HDCq}EL;lsG(V5^x9a-8 z0rtMo_DlTK3e!(q6sV|2M^-34ao*gB=PP23LF5PF!Arc#z(*1d^#<8MwfX1`EQV2K zM5;@TYcE_8qUWg2+#_1o5M0(zmHXWL@4;ybAQEU}8tm(v9@mO#ejiGg7t#Twgs zhzTh_{;}J3{VI!9&em&|+pJk(x5j1J#^o*>U2Ru6TDaJnFSD_9wY5h%%J%SXz{JkN z(#65T*kvXwR=VB7i1rW0d zVVS`Z+2Jt--h?D#Y(W4q^SDQZn|HLQe_U8hu77BnXFwuw(I+V7uVX=5ce$PRk7x7s z8r>tMdIJ9NGS!q={!k=+D1i6=Krg_s2>IFG>jQYfz0DEa;S25xMYn~bF&tn}B7hms zAmAGSU>wdcl3xrhgJobc3{0AiP90((J0b`dW=nlzb4_J)eND@?x{js>G|^m-wyU(E zJz!>*zQ%{2&Mtu+>|%3OS>u(mi)YSuG<4FsIP_i~iz?&* z!TThYutTq| zHePodl3z%Kbs&IscZj>%CEe{HycDD^6;xl(0I7SR3*;ARsE@4c?bX1Q+;1RJ3}osE zg?^Jl1AyITU{hRwXJ@dm_zWfrFho}X7}iciA-wZLUD9!Mz9pT&pf!#}k`(m}L3<&w zC73`ZoIo;5G(qQ2FnHro=rjui7Jq^+c*p>a1^4r+3%+IJCT!kYfegO;gHuS63_g;m;rc{|Qe&Q~&};W|%)tQ+d&xlx>={M$ zsDJwY(+S7@ zGfoBPMr2%x$*J*=Es4ymj!3JF&%G9ySa~9(=+9Gef$^n5iRDNAGEW9%`V#YjUf~I) zexZ5ciRZ%-OF)MO#T5F6k*Rc9bV)W zUKA8x7Ls@|BIQz8Vp&9L`Ef#uTX1@0W<@|`enR>Mme430cqkuuBG=A{2dAMVz_7s? zD8Lme?Tiw;QR-g+##8_>;|tBmOT(?#Aiwl?W`}ORR*j*Cq0Bfd(Z81HULnstW|Ws2 z=PWga^yZxO#v9@AEAc3Zwl|`Y zS>XuW&x%H0^9;|?{RrB|A{m3H%_(ocRo?~m_V2;FuiM#!LCHm5{`RNqx8FK{y}@>a zv*VX5oxWP*@Z~zEZ#LS0vBq+p%TfTC-5PtVXC%k2PcmM%7?4(8@g7LIG}T{k){bue4LYT3U1yQ89m{rpaB-tzg{?^b-V{p-KH zj-3cT{p<1Fza07_FfpNlM5XWt=n_4C7={3b(1Ljp(J1k}lm7of7b4U9V=An0h8d7r zLIL*pQ5Nb4@>IwK|2`%dvo)&F0UH}Vg2JNsDwOeoe+tPjq`eTvupp~Zl!Xr;N(VAa z3O4Wg=BESSY&-n3Ut~a2Zx2(VU`dCVpddv`hCs&PNoX7qg~_jN?(`)_uK)1|ixtbP z*0|cPUFooXt^N9SaAUL9)q1rHir#Rx1(G>`?rOW*&eGY^+{xN%g_G@?<=<@Go^`gO ziOg@Ki@MlS5=TMdscFJNEW`J23!)BbFDZ&e#{Gx$Q_RT0Vc^P*Jo$)HGp-wc3NP+B zx1ff!kFfsXJksz7;RZV)d#ctv7AvrZk7$w&Z-My_IYM-vq!@UN%uev5qJB!}-MU)G zN=~ncO+1&DQyH6F<{O+98dc=!pL)zQ_Jm(zU}Qm5;(4H2SagwhK$@Fx>@lCHU;pws z;+u5BH~II!2nXH5|2z^101F^yAw-2{#3i2tVnxQC35m)9G6BFq*hR&kiB2f?4NUR! zPauHy3eO6Q%5nFNgYpYWbMubbw#y9wcJ5N^p!VUw(4<23P^x?g^nyPy0E{QR&%?Gx z{oUi5Zy^fqaFL1Q4o`4XAi&Ti9uo*h1bmbrg)1-)$H-#pm{2S|lc5KIF{r4KNb6UT zNu0LpEkLj4hU(_V+Scp%99UgfV?9)RT}=nT49z%KwgST%D%%?>TWisBOKj;So{Lz9 zJ3E@IFO}x!*Ok{&+8LBC7Nd{H=oc}mVh%&jXRCzVK>>S^*)Q$u;*vVpeOxgyFB8{r?|3*AiUuJrx=8na$Eu2LR$qMDB)f;AxG$Xi~xoT zDKG@EA!Oo^>82&R=NQ0Va0h0X%4xFj@!3j&PxyIXVi6(wjDK{oZ)6D}uGAx}fDn6u zm|7K@eAOqi#62XR5OdByy2K~481xp1u%LvpAb?HUWgiqeRT!3Xg_v?BHoE};_Lq0Y z_5&e)QN=-t7rnyI0KmLMass1@LZZ%k2WEvQoQq1i5SLjIlU5!ab2c%%HZiv$v!o*? z69q>Bx&q@$K?R-;%>y0g6;TMkOLTTMNHI`go*_9ADHq)W(vJ~RLQ_hEV$Z~9l(B@v z5_r2d@I*53R6O`hIy564oI%&285zhgEh?qqr9zJZ3`#dMXq?gCd^LRcjp6ni&COXg zvdbfB1*{V5Ut!mm_zaoU_(rV98HRQ@W1o5fYHZ-iR({cfEjIs;qh`u!KnzS(HAamCU#j-bG7&7GH;IGBT0SZrhdnYrnr zrAt3GGyT|XF}(QNtO6zB2qMhf(cIeAYT0^6+m&`povfW!x@`Y_8!<9CGCF9_f$i&m zShe=&bz2W@-+k(jZAX9o>A;S|1fSx{x-O=qUo?moG~*{;^JK!$pRmFbG5&{M|MMxO zk)<%O{tGYz*7w8thnj@g6xS$DL3~}y-%1PN4V|3f!Se?6>W?)Zv0a+Rd4OTrzC5|6 zsJv|3f$h6a?(`!1U8t$1i=-U+AP3D_4X`8wOc9b|G>(MA7F1quJ?!Oaw|14etE1hz zRY0$08$VyR=}Wr}>#f$VvRb#&YV`{1RWA1HT%12&j*lW(I9r)7x3OI9;QHl;)1i@< zn|iMI@!DwOE|!eMQBwGWXpsswvy3*;i01*ulIl2E{vj521pw2cQ$49stiC;Hcszul zHKox_4QeI_HB)d848vQ9PzYdvF(Z;<8r>7cz$39@k|&x1?Zx5&z|aJh1mu_QnHpXQ z2A=ZdQ(Y8oRzZD8L_RUPFe9%fDdTc*L}745ffpg|glF7A_b37}KPsU#IrDNia zT?h=zJajzRn~)e1kxh)w^9xGxBqVtF#UJyC*n61pXydpmTR@m*J|6ZH*`WFfHk18u=<+zx~kSXELjRUJF2jh zdelyYYjZVf9^xXbrRLJ*lH&3+Wu)eQatDLf&8GM9=@cQeU(BY<5W{$bLjE9+qoz@$ zz1@6Lm!Pjh(BCDZbO^~ELQ0pozgt4VKJ4#=tBBk!?(3G2dt`mxxJzD1?pDBulwK93 zSKUV%AomVYdWXm)Ezk>!+^40G_5BnB6rC|jrH@i*qkYtoUMgU0v>zoxA%M}bN7#}{mUtWr%jbQFrkF8Rhthkoohc&ughV+ZS3gv!C)9%v(MnPPm})|ynvxGZ zmJdDwMS`Nv2A(Lev@=96?X*NQEz&;c56$ogpL5mES*jVb@M&QM=eS?KS4cs4@9-$O=m?kIJZoCq!pfd4?5ugyaRHtzZ|uBhPpe&x9pk zfnRc(SR9a8dDt)KhuuCP(fnf0yN48b1n2rk6cHi{10o7TqR)iKmk<-q#iU;fh%7qk zm+C_-a1YLlNUI7-xN?G!dxDT1mRtd(^p8G!A|QhhR}z>|8kt!c6nD-)vhajovTu0) z>EKL4Okr3u@GK)KyPP8$1&~WsPo#s>LiJOT`l)!}nHY(%XP~-(URc}krQGmBsei6U zh4K>hcV6jl&kc^gR)HkK)Rh!jEfB!48>XVf6|l9GVFmxG|@@hrIGSbm;XC zw3iaiJHG>T!86f_*f7E#aEq;;<_t^=b+5#RSE^fYHTN-eP5fIo{!Virt9xiUiweuF zmyf-X48IYfPU9PaZjP^+<7wtNL$j=bSFFKTc*GQ4^_;4l=~qlcQIsO~CR{Vgp7XS=In9b)am#$m(4>Jd|CAKCO z4i@IN=AW5<_Q59$KKSJ0g{FWpb1N5n$F(l@s~xSJt*n<>+O4o%>b%qfsJ7hMZpDht z+kf#1^dUxv9&+2iY3qjNn^%0l_lMo5c5Oej?U#c)wjJIdl9X20MI~_+R536NIokh6 zSN#`Mm_quyorrrk{ug7Iy&fMOoBy@&pnpHyKwu4Xkc&@` z&6eSvNz5Li!pt40GdawaYb%>uqcT$pD@w2RkSII}TQs*T<5ZBlk?|LUpQ>o;IL_f{l>NS8&*54U*r76y5(Q3ao)Jx zajmoM8Yj!ub~fuA9oDb-)6Ko4rm4Q0*+dp}GUQ!sWTppzQNA+5u z)r;0Kvw{66cSFOK=>+mowQh3A@C3M}(N1dMUJLaIlnP)B0QR?iW61DWJMv76MMlZh zkA+g8*A$yS$>fd$z(B;v)Ke&FP5Vrwdc+jnt-j7q$|&~_&A`GBuS6x40=>eb&jf|% zdJd+bVPVmEF$riNSU_kx=rgFWs9ca}fI4qN(vefd6P{7O@AKNR z$Kz6Uw+iH!?y+j{kwh^eQJ}aGp%?*-kD`0$v-Reecc5+~Ip*SNXC?1*>F|-%kk5pR{JCm=a$~ST~c8OKGPAIjZQ>}Ze1ns5ilB>I=>P|U@F5P>z z9;YR!K!xBA-+p;(zu5Ru1PZL>lR($cRrfbaddrV;HU$<1M3*My97!oS5t6(wA@3x; z26LOBnXCDQ5lw5FfTX?~yplot}0(cXbbrhZtl~iy#Ah~)= zWa$c@r0~@JDTSva(+-6v?~kTc`bQN6#uP@UmPaO+C1h5n7oEs1`y(OiU}#E3aPmHQ zVM@`Nprp#^jHB6mYNIl$!&3HbkIdf|nH!W;4w#J1s7lDH3QI1FqVEY$+7(0Jlez0; zXnb*cel=hE2mmHig97W7=z2w(_Y}b7xCZt?fw~jE{f~-PkYc^s)(`soA5Bj_nVt_A z9(@9OsnKePUU`y{_*r85^ke%o;TE6>A18wBldQF0b^nv;>9=;!Pp|)FeEv_}({Gwb z)XuPfD(-)Smw@)dSG9Z*w+wQ1J;Ih=&66)6vv8LfHI{|S=6Hq55LITzlhsE zBZOK8ct&{PAW#1pjx5~(OFO_&_cyCQHmN=|D*GA~y-X#{JtqG2+n5nh6Ku^b zt;YX0YWUy(@$>)xxBvcS#PCt$#+ch#+f8w_aGEgQZj7bFICJ}P!^VvmHpc8%3(L_q zHdDQ)Z`iphFgj?R@0yvbW=vW*W#($H6+4zK+O%N)rp0qNu3F!F8~8Lk$xi*jsT3O`F{aMEs5cZml~gg;*1C~f-Wjb zipV=CY#{;^ACYJ)cxGFZSb2*hWXe^1l|eufK&Z$;uC#?CArp|rI)*@TqnQy+OB!u& z`9GMlVLoAs^Tg?%j-Jz4ysHew4W9arP)Fs^Gi62$;`%qhj9fCxl3)zCe+dSL+g8BfI{F)AZ_vbFpyt6Bl35I=LW|VrDW{~!iC4-&gIDXf{6Hn zpoq+%$jsynoMxFt!!I;#!`7I2OSc2_*rGO_ zsatF8R%yFr*j(xqQ+91oU=nO0q8gwKa%3+p8^dH@aD~`K6p5b+g^xgli9`>D*gt## z@=GXa3sdA3pa+U27MeC z&eUH#3)<^i%_#ub^_t(IfMHkBTmSntoECPBisYy}cjEHt6P3I6o;`Y@sh-tzo!NAQ z(|nW1xWfa6G4F{{jWx?S3Lf48#=@i~WGZdBEpA5ec75rLV=3R~Wt}#g6g! zt?RhI7_;GJ9XyhYA(X$Rf}jXmP=1LN=<(r+I(VR;$imfkZlQsYD@KPQ8y*w^sNgmN zeKgp}6%#j&@C8@Wfu0W0YaXssQp;vzoQfc2R zbe&2)NUm3<4AnYaa!@&sPrSCiX!RN{;_4ji4{AecLA1S(hvGYm-t8H&ai~+ zqd^IKeZmSuQz~P#jwBbI3C}pbBX;k~ZM4|T!j`~r0S=pU+pTWueqaD0rZ zPuJFGc-U`xHrVoXQ1@U!-8P`U|5?$Fp;JV1p#qb(_7jYO`~selBzyQ00fzK4Jo(a& z<1YVYpoTI(B9kp~@$9Rt75y5(Epf|$#DoY0bWt`9s*JDDVXuAiP4(!T>dC(pkG@GC zd=|A02%$`aeA6Ju(9hIcS^u=pzr@2j;=jpa=mg8KH@g9>$yE~grbNb0_^vGXF z4I6J^W@$BU%-B&Q%|;Cy@sDA{hm9UP+S=CI*3lNUm${vp<;1aLZN>~AKk{d@Q9qlR zjkGj(ojq;o=H=`B)-T_-+-uD&w-wW-ube(>`3xxUwe!8!ESkAud01-p`J2o;LUpqY z0Y-&laFp$2RjeHy3M#U)MS6OaIO9;A!=YXuoIl{8aw7J3!nvW{&;T&)P$%#&0uEtD zv5I0LF(Q(e;My1=q89N9FBAa7R1X9o*EG1m##ftwUIJx{Kw%On+IZ4-o~(r{Bbm8k zZ39=d_eizdY|p=q9rm}eBYqig_N(=H%PA9W-KN@1b+dPOv2$~pIDN{rh29>E=1-a9 zc zQ?je#QqfYFl378^!7X0_p=qG5KxTp1%E&zs5u4{1OixHHBgsMotTb>+gX z%ja)isj0tQbN%9ZoV0@VM_uh1kYCqpps<|?nv9HGIeF##Ugm2c#r`;PaCgPA{U@61 z*$6O#vF3Uf;|{<1p0MejkkKe%Hc8nGC7YpQGt^v`R>(03IYtq`RV-)|3EG4_6P&}5 z&oc0s1`g8zY~!OgFD$#w4)x*oaa9Y6~irtSVWz(9VH7NXqHD>e3sj30!Ck9_S1uDXvYd)FlG zxg&gEP}#60q-0A--u8%s!1&$k19JW2c8Ajrf*1n)1jg?Ph%4V1kQtn?8&pw9(%xuV zRd~YQnAEC-j3eoVXTwtuZV4>{@WkgHi_SilP;f3X?fA;=>EWqW8D$s3>6HPorID!> zQMA4B83#h+OMT&{Ev_&qwlFZRbbCZmQr_{D!c$p$E*DhRrSCc$onBpfs6J!Y8DL&$ z!Y%+_YX0H0!sA)R$J6o;2FDe|rSA<(EDlfJm6TH*l2DRWe2gP}AP4CQ!b{UF)4i8t z+DH#5&Y{&WV%$)-_k$GEn0gEk`apiQJpHVH@ENsN>nCOVfDCgw5L$>U1lS-I-XU(m z2Z^ntUu^myZ|~PW8f<;>&o*k^>**KuBd9;+558h3RO@Fz7rcYWh^vh1#}vB$3Bn8e zgqT2&twM}a#O=&q6c7GXKKN61ACnIOU0mY;iYNUipchj+K=lck+J1Nhj))~v@HlnS zr0ij+`xt8A7V@k?1yZcHQQ6a^?rqZcHtG79`o3o6tH`v{DYNEIn&~!u`5gD9-c~av zn|n?k?KbJB5x@Q8*HObqjrnz)x!HsXkG2>yX8dRi%W*cgmR8p0=GJ2@9miTY zLyaFb!ED61QNzcZjkYtNI^T2d+WE6q&zZ5(d&cTnGuC;zEO41HdAz-wjpq_C_a(FJ zXG~kT!ROGqy4wQHJ+T2T^{V@9MLX_o!c>m?WS`T&7o{*`SM_iRY|2)OVU*p20@x3; z1}ZHoz#gZuJQ8RyBX}rxaj1xr(>~y8&^$tIMt-n^>|oHg5lFVJQ$`qH>$b%QMCB)D z?8mvx@r9u=xoNpoiD|ooBD4HM)57EO0bQvX70GG4L!+{TBeFnsCDB3Q;YQ}r=$wG? zOy8iih0Fc72heUdsC2+-BYabxN^O@^*+GropcZM!rFJOCh}`vKkBdX*OA#N(GXVxl zJr;-_ip02unN%Aew=lPo&Y=-7#$>jzad@Mi%QkXxF$_eErtz-e>cy*9YR+D}h;;)P zcD3dP(Cbo7{lyD)HRmv)qvk9D*zdrw`dYMdU{evkq&9&K`8Y6`<3v?e)vn6=nmdhk ztfp(tjn^6*>zbQyvYT%68UbT>1&js}qfyFeltHnZRm=tjyBSHw=jwS}1D}g9qe>!Z zmkJ+91otI^hjQ@~Qcq=~$5I>?^F%IrE|ok7TmiWSqGx>JQ@#NB4acWk-cvU3DTn`z zDmZ8JpYbuAX=qaypDOBZfbw&~2P=KU0dz?~pS|UXJ8+1J@C{G$mM7`NY$tIi4^8kW zzeutMd94_Y9=Kp8B^^k`zgM{cR8SbXsp|S`A(OP-Dh&Xv6SS97+o{muikCwB9*O`A z&hZGPBwa*}p;z6~ry}V}QuH783k>}{-A4{w2NgYy^1eFu+nhbOw@2&-N=4B2hbC8S zkK7fPeJp}r9hQ1vTSPH%D1uhy6OtPgUxuq$w5q6-{gFxg64DQ4mz>Wnt%;^p`$m-l z!2IIM!_yAMWSxr6Jh^sfR!Gvm)Z#PI>4&z5<@-k#hQt*|ChZQ2#v?87P+WRtOvWMq z*xd;v@IIrwHfQg}==6hODHXYUYf=hM#AhCirtJ-g&WoY%Ny|T)l2J*^s)|l7jic|0 zfE&BKgYlV_zTw&Qyn_HRnd&V_Pq?#`>)t7i_$_PEvVpD*C?pwn5fNZ5kNR7l3>qE} zYVHqUTzwmvw1VjBC)~oqUSb=D{}Otk0+XS5?bAFQFg^R$_WEC~ul}Wf_Kgf!`KGx4 z4d{h7jg|osybt_!4+o4d|1`ZOl8YELD0HcD(h8X$hO@-JO3FN5@FQbP2Jo1;}V@8a%9%t=nW8>^#?c`wN zU}tM@YiDm`Z*S$`WM%JaWjT4Q<&?2w?MIEZ9c^Y~Hrip#q`6bvmQHh7=;jWpYyIq* z>t{L7cKU6?KYp_qF=>{o(`+|OH)r<+i;54NxXn}D73rJcGL5^M+A(8-OuYPomhZmy zA%GDDHl)(19mf>3D1Z?c;zMGNM-?T|Jb@wrdqfH~83(wiYGbSKv+-@on-77Cl3&E= zNTH6hmf(Bd$CwXgn^4*I9bl~>fY=gU?Ty=;{C&pQnT@aoUfmY!gl)l9;X{Ar}df%j_8zNS0j0gzN z+PFP&OmuKSfO5$U>v52)ud}~kknndarwg8 z>z6Oop$qX`{iO@_munGVwHK~mz@1_BHRtiX<_|zueeEANE}j7$MvZI+Ze2QkXlRe~ zh2!UsAKg<>cH#J?hHI?Gt4$48?lsgkHe7FR08Mt2(|nuPgcuVx-xEVM-H|lj5i=TO z%qBUbNzP)bSWFFvse@uSYdH)(mkIK#MZjqla@&VQ7_VK(e*nfnIlllU;64!W9|N>_ z{Kp*OW47QihY#mZIlL!7=*1O0r;5deM_%wmLk-v;xf@j623z=&$$QQezGA?+>@Ams z?Xo1V*^<|QG+-E4^adwPO1pSut1@5OIh1oKMRZZzqI?-}6e`Exn1<)Yny|L<1!CL&{1Fkx)SqG?1&0OncH{E8~*uM=M{fo_1 z2ixhBtURXJPoKUiDC*?pTla+OMzM||H*yqhEIA377y{Ujh9RPbd`W39%qj$Qk-(?> z)aus{t05(x2*4tBecxgxA=LTYPP! z*uan&m=d6uk&Le-HhLn$m@*Jy0Mv(?mJWSOm)`In@(W6%>wtn2%7yqtfdRk_EhxW) zvJS4~EmPRh%AoSE;jzX3Az52?rf%|0-n27$?e+u^VLm}L-w^s1-(}x)M_UgG)7&LY93=v_V#)ihMkUS1F*25!L&YrF;+57u}Kki&*HPtbj z>Y5s_H$ydCV>Df7He6>k*0TX)jW>BsxA@Js1~=o)5tsXr!+pr+KVtEpKyd_!E*=?lf({qn zQ!bnfp0kl>FG$)U(u;L6`}@z}v7Ek_(UNeZ|n zly?hdT_Qyn+#Zn}DKa(%JsNL?3bYqW)SYmjg!EFt&#{vf?&nizaA!K)n!r)5d#Az^ zoTsWFi(XKGE~U0drF)MSL@hi49ULlj&*Q{!5`p+ZV*CIK3;@R0qyLbhL?gfrUN^1y z>Y5$dJ0psG!m@Wn7KYQRVzLfx2`vD6MbHl5l&|!IaoI=Wp)w9d)2bpRy->6;Nf^&nT3nJnQQ!@`_ zf0ni&x`+rHkhWmpucYMzp%( z5E$S#0suC^*L~t)^>cLnT*CmcN@yAswS9&Xw||k~c(AX+)TxLH{V#&{ z&sc2(g7!gCJ3h{D9RzwoG4vnqs=9B>JMTzerI(-dSiHh>$voEuGh7$XaGF1Dyob|I z6FuR|Yca-d+&Ei{v9=Q|9jvV!tjAedjI|m+-rn58alC`8)s*RW&YqUmE+Z|b{AxM* zS9{NK?n|7#)=itf*4=%Di|6vGUTZu(R=Q1av-oA~|NLV1i{)fn$5~SyW_ego^B8IE zxM=SjtU!B>9bsiY(sukHBb4Wkhy3NTr;rgv@=R z2_<1krOBC9@EZ#l^YKgFur*;@U>X2yWt9Pzo4Et@I$zp2LYi^i(K% zEE3^9Fce{;p&TjFLTur1T3DR6xaaejbQ+4M?)Aibm^|j}J0PGr^T&%fK zLxx*kI7ihr^wZ;@OUjE_SMz&a4LN~2dmIWLzjp4AL%XYv?K^qnQd2`6tD%lS3ousK z)Ku5pSl3L}#yAc2oW>i#FkbU5V3@f14$kxfbRod*DVdD`Faw)`Yjvz<9hDox!GFAw z$1!n1N3!mdddOuzWOE;|c#qfuDoFk*rpODP{UulaDM$1ic*cSU1urNMJsLRC2!po* zYyr-wL{p6T4FjGA0$1{yE9>A$0cLNwKjQyM8m~awfp!cz01WlL5Pn9gF0rzk)H{)~ z3q2a@P6-_0XNMCH-h%2P02@LV996n*IO4V^fR+a71<%LUndY5B{f?-#ZmMu@IOqmK zFQUNuBnG%T!*%l$)kS2f`kE9UZt}ZmC0Ez^DlbUles0QlCzJ5C+&@%_M1TR&;k(ut0bqR^_}-D|)Hb*tU3;JLaX-i}rRf9U9M`>CKS_*G zz%ZZ}0!-4fAmfMdAjEUnNEk@gnH?tZu!Eu6>!-Ub6 zqeohf8fgt8W7JrOQFc?UT)my9c-z~1jCY(d!E@P^Wjp3B+B$3Q#_4m`d(2)t&3oN6 z_Z3sDJuF9Bj~Hh^&fdd$(i{)x+0(~4P5fWO{_$UbALi&jGa;*}<`$z-sAfu9aO#zU zDqIF5D_*VKA@HCY1Led;H+(yv>{kA;`H_C>LT(dPFX+L44y`}ji zJA02wj??n6zi#BdV(?xAz(lgQDqW9E{r*;isbtRu zkY6!LrHS;t(MiSrVd*==(s5W{d=YRfF&%fhgvJ&H#}xVmqzA{Agv6Ej1kqP)j-0nD zU~^FFmcZ1-s{;Llv-9Alm9{S-b#G$YzR;Mwn8e})V&#a4F9?px_6<%2HI|-J6_Z$$ zNGlDG%>{sk$L8Vw<;cvfe#y(%hOXKWaq4^%XfK@+SM@YHxW>W#J1&3`yIL1DQ;J+h zQE#E}B9Xo%6p`v8zJf1>BB*Bq(IWu~p~nR<0oD%%#^K^b6&|+@8GvhGMlMUwML)f! z@eb$ar7QKd=sbkFe(A#X%Qa9pF4tl^@gidE2GoV~b?461*Wk3UYsf7$)?Y`4{d;8d zwR0zr?LV-$xcbWZyLYd#8KV<~8BO&p1lUb3<2D}{)^tbIa9h%F zTh@3-(R@$EXizg6G{CUt1~s!8(4}KG85p1xn>EzZ9f#G%!90;R7PF1TYG<(@Fgc*W z9^*Jl?h_^#hok_yu+fTq!dxis(4ILGdq`HinuRY|B!5WZlG(BX+5dPxUY zOl|z479HR4a69uG&|X+dq!(!$cA>XW2@fA}TXfUGK2Lh<&qewZ2!0jYLG33NGRr5MjhgPv)m6TKeQIAH~GPAuwPUk+1D% zDf<}m54U*T^x|vV!b>sIkX{*@yw@ibHxY*D6Hg zyTW4gGxHCnW>%$SR3*~(;Y1-?MSSML)iLRG}&KnA<|Dt*Nr~27H zmCyc!Qat?zC4cfw_V}Ch(N{6fbp9e}1Nqg@Fn+wJd4E^+{)Xi36~W8vqBqsGoMqcX zr+9nYyF1ys*-f53)n(!IpXRTdJ$0_Dh0}yz%}0(HJHp&M@>(%{)~Xqcx6WH1vV48m(v<=8m;23Ix?}d@9dqVw^PaZc&3WcT zx4G`q7R_+>cDJ48GR%DJ|N7Uth@Sd?0S5GO43g+>(yh2@6D7KX$W`i17k!9^i{7YMHfYXTN;4Bg;I`%=>P0@Z+Fp;6gEVHsgD zIRG%g+otXD3zqqXM&~p6ZCdnhz&Aw!46c2+hys^33SeXqGjV21afAw2_?%kcp|&yM zQT&Pk?5Rldm|6q-?&D~umcbBX+%|L};$W~J07D(dWT=}O1UE0&T?a%_yO=N4)L*Hk z7zSz#7GFzn5>&@#tnAk4O|an-4!zK2%7H*o9;@;RAsEjyNEHWNQYC0 z+pZZvG8L1d#Xu}}8=KS4VYjo{_W@u`)_q_Y(2LD`%;G*~gYXifU7cvN5_CE3x1BCHd=b6RxJ ziAnR2v;@1zHm?r&X`=Sh;&PW%-2wkO(VC&bk|{xKVGEJ;3{hvmiJa;yYPvNbzsS?D zi--}3A7x0dK57k2pzr6ZKd=TulG*}hHVYY+Znx! z0$5zu(fFLBn8A^_Cn}{fKH~@gEH&?UTK*|o{_jaSNBv_z{j|lqF z72D|k@w>BjUy8{%8k2S~qv!05ZAFYXEVoen4v0*4``ApG^tNlPRj5HDlB&{gUi130MkUsc~s;eCbU@CBe z^PuYSSM9TZs-OO;B-{dk$)9|cJ^lg!lRo+;dH5C3#Y64Y-=yokqk4B!)>$WdbA|u> z67ShH!RtS6sWycs*|~X+wzV8(J=)G=qSxAaKh0g^?Xkdp!X&HV<3^ZSjIy0Yj`MXg!pztn5xj2TO30Pw7*I{Z3e?0^3{{Qry?_P>Yy{J)3&^8fw(xBvS2Z{`jT zn*&1kpQx>8N-^zMj^1&+UMt&(^MYE+LX!ouYAEoKcrMV6@t?%{Po510P~kI?Uf;Wj z|2EK*wjs%5q`Vt{kz6RqNPIA6FycGIKL!;T5B^~ieFIloynnyXDE4y0z zhB6%f=pw=+DqV-Z^_{V;$AE*uFpNWwlT+ToO&q!6V}dG^_Xdi=dBx(r7R%o$)bH8+ zCkKw+NKV@eT%p5nV^(Ec+McM?vZUDkq(8T&Hxt5dS}req?hfML-|Mada^lG4iP znH2yqkY!=fIpHz6{vmWwV8JAMW3zAk@^vALR|f5=s#O_4Uv_Bq#F>T(Qe@!->4lqZ zB}&{yL^Trm64FQ%69p!wrcaTnVB(h|@e67+vxp2*5eObo8NL*~K#Fm>I7@}Y20_-! z;kEF1CLSAmi7ckJsX=t(O8vD9=Rs`UxCE*T&{g|ahFyopFV@u60J@M~*HMI>udh9K z9mA#2&2jbIagbv5HMqz5%9+#0t504$cl+K=eiLY{+dRfC4(m3Xb&~}EW8Po^z?yGz zKz=dr0=@Xe)*)%WE64s}qq6y)qM7UlYr3nX)EKUZVIxt)VCtHg1}3`|ip_23@*Z+H z4>;V1T;3xd|1p>Mn8$z06+Yt#Fw&4JYRQlV@{8DBP{k4~A(Lbnt^w+n8l8;9B72R$ z7e0X`NMVSS5JNiN2$k3~6e*~lA?h&FGwhOTy5NW;18$*WK&J+YvZMe8OzM(r@ew#D z0DF(!LRGgMyNIMfWT*)Ch;@Fbg$VQ_HhMMA{lQUKcwIJO3lT1$Y-KM~-ouc6sAIlK z&aK@PSO5?TOehaXDBB#8vn{+}V^EfVY*|3u?x4g9jMzxp8xUXa6P&vzELJa-DhJZ|XVPe5a`mPJ{xu^Z(_OIHJjxQue{E(!Iw1SiAc}Ftyk7gGg zP0u?Nn_L=8D^Jd;_Kz+Kjw=gGsz@$489}cMj4Ka_EJ@8eLeDyyoL(K3SQZ?ew>>o7 zFES@6t^h<>a6%!y=x|V6ZbWi%NK(<7ZShe_yKAqB(S;;=E>+^B2>1ku8}mK(SC#00 zQ5$=V_xrW2A5=&$6r&2$Cz-KdY6Nr*iVgh$Dh0?gBl(ob2r+6gOls_t8GCj2KN+74 zXdVGea3Mk^ZT%suOPpGValgpDRe+4}aK0zV9ld%pjD~qG&h7LXut&s!MsFLjcB6 zpq(8jiph;^iLS1J6`qplI^WaUW1{6$2lFX*<0so$x;olUn`A%D+0oqz6xc)$jN~8h zXfe*g+{VRe`oe`9cKUA#3Rvm0VcwcW-mB)gEq1e;Za=}@Zr;YVsil?GHFs(n#PwYD z9g((CqH7l8x6eY?Mhiz~MEAVBU9NkfH(}5Q)h$$MUg3n1zZmENU}{~5&eUxLdLh6t z1yQZXi-H&fSUXAjMh9pwRA8K!EdFbmvRA6;zEUqQDE%F{0#b>Vb09UZIyQZ8a&C2E zR@Kgk?CrrB5rC_h;=suK;F!VAKV!+yO?>i>!eSErU@2qud0forqq0Euyxq6@-PF(>h_6HQ%8HT&WNXpb7e;hk??)3G$w**ah zMa+95c7up@i_5&lX5S(hyUA__hTY-J*ZkCy}xf<1@(9sH0fb4G)sNT<^rncjVYjEE|Y1!mw^U4%y{9 zDS#oX;Bt)DVt`YpT-8a)glk<=WtU9V1%EWuPE?`q6wQk2F7m2(Dk8!(WHkr?rp7>u zk8)hp!37we_9I92fhq58mVLO+>`2P3S?imvW=lWKJe}6!CrT52zmEC>Cl+nD*H> zY#u)OQ~CIt{LxqW<8RW(UnLK}h~OV{|BImQGdd5ApV-F!CT;Ic>D%k5ym!B4_n>%%`%lL`o#@LUsooZ|8Y+*ie+{`7@ zmuy+#wQ8~J;stI?7JIE*HP2`5;_Yjf`>k2Bb;*KFbCztKw;^bmf86E>x_@j=NK|gH zFKyeVn00eDcstCrA3tgQaEo7uW5&nGkz+^yI(E!I#*Q97e%vT)%dw8OqwK7HF(3Dv z^#qT33xi{56-Un0HE^4RDwf;;x`&O}F!2@W5~T7th?9QEXTb-6sUC6_IOcIkc>P$x z8rlbqM*=+Dzx>0^K^gbSoJ^wO$T}I?Aj!B$uFAwg+d6OyB^rAD(GUnMF*FO6N6yu* z57;qjuA8mbBuh7tUlYbSTa0m&DgFH!j<YK-HmCzgBBsYjv-&d8ixeAO72d*oNVUEv@f$23-Ay>jKU%mUJM% z1aF%;ubS8|n>a5yf;Teddr&RM|7f7+RMN8cCDHd}6dX>tC9vG1Uf9e;Sw$3MN>FSWI zf+G_3GmR0~^R)VIt?r#tjq&qRTuQ+Wup&7eyQv5XN-GGYL-`SKX{Jo}lpqs{UrHpe zfMK`{CU}4VqY9Has5LO`9}ca7k>(*$U`7_($YLrR?l7-ix^$)Xd>!@=FMekj$}g*lovW`sd!rT@hR%+<^JnWWoTZ!{SI?igb{0g~>C;D#{eJAyotuay#yv6X zu8?zA$iB^G-{ElXaF{na&A_nR2r%Y7Au6y25u-uMxF?5dx<^Sd#2A;QXEU^HhK|dI ze{Bno4CG|9fyC`l05CrPp+NAE&wI$@JQM=MgipDGr(7`#F9CcSWG6V9ITaluRi~KP zD@ceGFy<~x^91?kC^OeRuPzK@X4sgR2bb!`975D zZjI)hTKirL^dbOL8T+LAUXc;^fsqzgKS%kIA?t0D_TA)mr50Y^7?{63swlbeL~6rg;MUTo^_=;YFbw7p?*g^`Ix zwERQi$-6`1%6x)zQu9yZ#ILmKz}Q`(AOxd}wgzVg#~1oXW`)M$q`f%$9^Z(p9pM>1 zAvB-h)WGQM;JAGM=z_@9io>UyIKro5*>kzNgUX>0`oA7zFQaw>7E^`oNglN|G+FtcBzekazw(>?lZc=A>K04?>Fh3kTTo!~IseEj%{Hulq9Y-UdR$!pnk zJ9p~|Q_LMb?d{;|>^Rom-DbgtMe{Z;n>gFk%EQfJ)=anMi{|^RS-x|_>VWkNHq3FJ z<=`~K)@8Qi%%yHiH_ct=zhZ05rmb;48za^)+%n%~{*(z*EQgQ#$Im1F{`2r(hK(5h z%dfxwJZj_!3yX116UR+<8a2_@%+Y$Zqs=H=TQh6>DKlno2@1>GQ+4ioLjzyMmKxyl zjS)(?SoM927{X?#AIPC5j3_Z0k=n(@CHf4|g}c76(};VbiH*LEtu$ePFPW1`;t+}F z5P5c=sc2&;C?}+eqXc>x8Db+-ien^64FT*%Gml;#U78X;++qm00%$)4Hz-!Lxl{1$wo4tD3>g}7>`EFe1vtsW0 zIqu8cCeNSjxoma_Eq(XNOQ){0(Z?as+{0xsG$Jyk29h9b;L2J-era2}Gyp0s;Hpcd zApnCf4Wvj%5dcP|Ad-<_J;s*z8XY!dL>Rb0rs9g<0>D6lHFH3Oz2u5Jq^jcDFO3XVE5Wm|$Zg*7L!3a7| zvy7ze4~Z`iP27`|dn_ofY;!ii4mT*_nlhBNB=ulS|_>_6NuB0zDU+uqQ6PIzIhic*0)a z@WRl<-9d4?;MNc-BBczT8J<`e77tbA7oHUoU*H>=;TN49mW+d*$?K^?8OJVFYJ^iM8_*vctA6>st*Q09xWPJ9g?&&u&@oP}^0F@Kc3nLS$ z?ab)cpw_(v`hGa_wI2mKJQ3-xHB%o;hwVrf1l_S_gQogFMq9*EGOs>E{~zncBX4%I=%8w>P9+_2SOU zyq7iXXO}t8@~f^*_F6IA!e*HHI7=4?$JuVS-mX7Q_I9$KYB6QDlly#ETUQ(FDOTPq zyq9fT?Y4N9g`3lOS7)2)?#>Hl&04o~>5g?9L$|Knw$y2corUw5aSo$KTMr*+H`3bG z!fW-+WxmUoZC^frW=!##WMMz{AESo-&+xzf&p-b5w-LYmebmTbCrlXQG;#DK zXEP^-)Co@6u{Wewj#b5MKW9dtsIpQ`z3EXkik!`aOa2S`+U&LwJX) zY7iQl#72e;FxHAl6RMA%y|^_reD<0Zu8X|vXE{&su(5=4w;JzaKEch(*3-_~!`{Nx z%3`Xeji-(MOlPOL?k)?xXRciCwQ|M$^=nr9ZeG1(!%E*ZD|W7!w{gz2rEW7;dIuyY z79T!y;ws}ZLw<{|yNjclbxf(ADL0@9Bl1ga?9doHRk{w4UurGj3IO&Bqj!K_@HiQm z1PmiPz}~gA;znkT9)9OLWpI_juF)HTq?0XtLjcy%$a>Yldd^1!bjOt&@}#tifbfjy zq+(iDWn$XipoGHJe(~O`{at3SbD6Ww*?X<6$4Z-NE3Bt3wQ*ZEb*|6!#eVK{w|dO? zS-CZ4duZmyoyoz`1=)qiX<5}_F@*rI^z5qCjD2w_<SzWJ0uy|c=TfyFwtI^G6ffCp^OX*Mu5EpfC{JGz+d>zIqu@L;!ZJrndevFznweVU#Hm{T(&uiMO80`Ni`aI0o~NYv)hj zxOkQdOT2XE%*jJ%>n^ey?~0omM4Uzi01Q)oL5JNDAi8ez;mEkdXWSJ)HDQU!?q&%v zjM1oIHmNvFEt>$0OT;3ZW8{!7D5b!7ya!x_)+0XuF_cjFgwKD%7d#UPpYt#rl5CG7 z%Y;JGJS2BE05iCwe^2fM;O3x{TqO}>gk)WE4eBSLmrUCu0VV-p;2dD4elJvmK6@`v z^zh_BGL&xkr}&b0N^Or^gHOZw3^lP4s)v*oCAp+GhIREhk8EGxb z)wutUS`|Yp2M9RLd(?2c_cDDC&t@BIbF0g}Yg$5i$<$oj6bI%vh$wnvt44KIqz zITo9HY(sF)w#cH8m|cEhMS-#97!{vd=^wo-KBGFd@K|ti zIj|}!{Q!N}AMv>-gOd*ez!q&zh|4~bT68ujaesXFQBZ6V31#U8hqnc%h9?w;MCV1N zl*iEb1;>}hr5}t-JD8GtJepP&5?3A+y(=WH6lfP3UlfsCwk5@gV>keh!n5YhIe|?2Lf^& zvjPBo5iOrSMXWTMz0CU@K?lUyF)XfaXSpcv^jNbUU+v<+el(KLXQR&XzKf3vRlp8EYQ zdG~d3=T*V$TJDPr&5wdpDl8|>8g5}d%woKio0IFJnNya`{K>}6(rKox=aQ+;GaW1^ zTX-z+T(x7}!c8meX1b1cwjS^5+CSr$#%>{t8vcO~XWp?(DbgVGr~fnHERGPMq``$Vabq&rauoBxKzd(Ggy zZeYG>U_FEDK1cNI(D7UGwDQo{-0--3z)V17#)5SLZi_ZfnzP1b?z+iyRy%vIa+=YJ)0f$MEVFfAYB6Qu7zgi(o-6#Kb9}?Iw+CjVWF7!|CD1Ao=oM+%2a@P}=~-2= z$%wA#q>?yV8D=4-?}7J^OWp-V%LI}YM#SaDrIaMl%LBu+*KLVixjy{Bv1=+rhYCBQ zK(B6G08@1;@hbKfuZ*g0Tq(h8ALbH?h62>P;1Wy$j5HM~fZ^3yBzuh|eF+LoD1HP0 zLmkE=%V50r@BYJ97ORE9G%%?!-)1d3BbqcU1{_6oR{&DyuV2OH;kAp{NW5_s88)QD zF4tg6{iT}gwdbL(T>w^{yHR_A%)LBw{lZyX468kZwhnYBUO02)?8US9?%k0z-UD>W zn0G}?px0ew81oK73k%fP9Rc&6ka1T?;yIv1%tjfr2>_<#Ff|+&E`9NF`6q zp=_VH?xKD#*7gWB?<8vM5Gr(i3O(HQ_lmW>LRF6tFb2=Z=HWXMa@aZK+90L_Hfp^bJ9o+ae3MMHKtTmWL$m3yj|#ML!T2 zRk|a%ARwv~(6uwXa7Sofcw%{cW|dD^R$yXTSn9rnyyKy?YQMPsfk~B1d=mX)%W;-i zYISt_LC|X9al4W-_H7GHq2*L1q*r9`I$c(MEqC{Yj9q6lcb!cm{lln7J`_lv ziWM&bU?Mq=O&7{u$uw^@Ej=1zk4)1o)xMWt+V4jIm_+wcg!Jl>8G4niA61x?IH)xB z%Z(p_@giL>U)3encB@-I8Xter-v1;w_KCDTveu7=XW#Tszp5Tmlf&TnMQZviHGL5o zJ`42&*d8R%f{zze7xg7T3j-2cNYrl|iH|Z3h%vcpDCvWAAcyFMOJ6ut3>Vep_tDKE z$CCj#C;pBv;&uSoAX)uFl7S!-w|^0|4)9H%K$C$oZ%q)Hr+dY4c|+*}8Js zj@6*s=B-`qzJoH4~^wuj?17kf9?rRz5(WfxTce&Om}c9THGm1AF#i>^f@ zaagpoIO@=`^bqEK3lz3EOB4#ZRZ8@%g%a+y{(tM!z@SV zITM}dI6BOWnKG5p?#^+cPMplf>)-wE+-YZ~K{GQ- zGBYzXJ7$tuW|4t_42_w^YzkAuNka~II(L10zsJeUf33aFl4ZrQ1?P93 z4bOf;&3*|nX^!W2QUH56F^voxp13LMV)m`=vbdu z(os^q5lGhDv9G>mcSYUivYNiMyy|>bOGU%_3ikTS`i*1)thb<~t)Q$ex40=iuZCIP zk(yN>7@nV1&@wPK2Y(c9)4)IWcZvLwMEX!9eL(i};zc@~;DBDFWJ1tIx;d!a*ZXtv zawQByfEeS6?-OId>oZ(*rk|d^Oyn1$3m7&sGCw>#2LU(6FlI-FWQ1X|(P6>alLP$+ z4jehOpFryX(CgGuP+<5ag|U;TR6YJU}Ffd8D!Ww?g%>8i;3H>3<4N7 zIX*u#wKy|%erDzpcjn5>^re~UEAIhzlf${r=l>=Yp(7#$kAE9wsNgP7bRRca&xLbK!nqf40_Dk{ z!gmEEXGpZ&J<0$Hi(Zrk6n0_!hh7jo;R6CNQsp8ELy}qqePgVeK{t7J}g-}Huj*g_edIZLuhh!7HdOVX>WL1ZE|4;04%Mz zudMO2x{hxvn)g*S?klR?0RT(PY0a(NR9Lkoue7hCX=hpe)^$Z~xiy>fYqqlcekf}E zJe{>Yk=YZRRJE?ChsoZZTD+mSemAROH;cW!wfD=?>fW}#FYDU(lsD|CZ27F1y|cJs zcX9nL2v+mw6-|3fYPZ+7e_md*2|g6;w%wJ@JFDOduW3hR>z-2f*0RQJ6>v$%#=({H z#*MY@yYtF6tl!!{IDQEL#z%nNg|{(YlZoz1XMYEPiEs!3E$X@QXFS;w0oV)P>{G$Q zGtuIba1ovy&|xnGbI;)g;wqlc$ezr=<;eWA*=uj*mtTp`FY#xe$}YTKxbY`$ZNB_U zfP-ShZ+`JD@QKjtEyT<$Tp1w1fL<7LII+M(Wk$+fcznmZ@K$h%@~J|A0lffMK*SHi zi((iGFNkXhFyZAl;;U3k{X4;xw`4E#TmF?d{L61~g!w8p-Ha{x!fSsDuKmHeg4xC4 zxusLWhsSyMj!oaH?EJ>aE^w{B`5IjVB?BEbGhG8a^N*~&>Hcxv4!+hZhDzFWeYbEQ zJAZdmR|hi>7n-k|ZLqIPM3{R_gjYgjKx%YUZgSYVC<|A*4ntSp##qx*&&b}=GdeIh zBPujAIy^1fGt5`ZOixWu)7*+?OgGcFG}bXUP&d+1G1O5q(o-@sRJJfywJ=jP2PtN* zPBT-dnP@W1^=)Vd06a&!ZGbEM0+ue;>c+atx;lCmmNWsjtb}dhi5i?@z3PqO1H2md*1-( zpb*!PaQET2dQK*WZZuPG8@j)XQ)G}|N`hZ9M4WeGly_XXXOzElh`U>u zhj)x`RAy{mby?%aEnB}k_}%d_(n+s4!|Bj zFA2IB+>^}Slg&Sr%sl{rp&*mqL4c7O80HUW@fwyV#;a*!G=6_#=E2w$rNHi-8@V|; zbqh{m`@TH`+$yhK5ArJ`pY0i#Wak}e=NkjuqPd6BJ;Lek;WU>JOQ4rSfSFUExoe2I zONhRWk7GbmN#mBxvd*-^)(ZBH#li%0=~*F!0m?nu)bXiY4EPz&ZU z(L@De49hcsuBZIv#T>s=qs_~?L=mZnk(DcOuAJfBnMN&gg8~@EuxXM`92=cSbd4pXGrd8~$~A z_{#3E;Mac24=XZkV*I__U547y#`XB8@ zWg1E6A<=EPNr&?6fWUW!0Ud=AVn2z9?$`IySE}B)KZ5a#L;34|%ma0M|e$zX4#4-QQH!Z)dmc z&Mj)KY}m?b+)-M$J)^iguX1y4)s}?3)|A4|ys9mQwcAVCJ6R1oYuom=_kB~>y1T4? zb4lH%qU!amhOK$5p3Ks=99BnhZC_4VTTyLqdBfJE%%+yE?*_)sPw}pDg||4sFyUQz zDTUJe^7*GAzhv`p4FTN3SA-{fE|4z?K#IZB0_Xy9HvbH#ycgiRD4nOIPp726bLEd^ z7hfr^z7Z}yC%b2=TSI zq8opRuK!C6apO<%4ZI;CHxO@EQH_nxEu9fPBu*ShX&!N^=C-T#jMwVvXqoD3ndyJz z6krz+>*L^K3!+Nj*3>IH&_2-9!qdgl+rui**FGfBDKx|>EZ8$9EHEW1G(E;IF@)h` zXYS6h@prQJcd-xg@JkL4OpowS4GBmLbq;V-HPrf4MM>Y(%$QEoF)>s(&{i|lQ8m_A zGSOEyH&n4SQKws|TUw}Dm}@XBwQMc5tj+c98HSDwQ+FGOP*3+rUu#bXT}vZnJYuDt`b0@tS>MpnFEFRHYSW%C51kzvoEDK(+l2|?rBT88G0_EVh)1jYi<1a5EPYYc zuL;~vf~_1wLd7vqRA2p=Rn_1c5zp`qatw}e4UKXOjdTl*K!AmXI|he(#U>RscQ$R_ zS=X~6DX+lF!`;-$*2I}%;9#QfXk_L^v-GvH3bMBfu%r7}+XmQK``H*dn;5v7oBG&U z1-m&%2YSRsdc;M!M}@dY1$)Ei(Ly}DPNUnG+h4X^s9zYB3m(D*xfGO^Yq<6&9 zJH*EK3B=q2+R71)LIB2pIK_QH0c>RA?(q2SUx%)v5d-H&@0K5m$(pV#W@BVRp`&-S zyOY2rvxj#c&U&yM{7cL*a->a}PIj3O2CujYwk^)o)J9Z_HzLv+B08YPS|u z0IoKc)NITz?XGX#Thp+;x%0E?#;wH_-2lku?!7flTiDIpST%ilMa}sot+~a`>G^dT z`E`kDmA=7Q>$0293@RkVv==?`WN_|7*9HM*PVT}fSSZEwFO^8dlbsZoabVY+gtQxw z#1{@Y3rUL|&5p2(^H${&&>;bRbC%X*7|jRa}>kYUac)f-?; zN@X1$i6sw2Apu!UFHAh-Btdj8dz;`y*_sekNVMe9#dsm&pY zRcXaN>x#Q0((B@KFol>_(wASgoz=9js`<;Zy4@u;+w)l)%bIrORRKVY2FJ$ zDzmINgS8$WpUK*oS-CB{W>0SIE|764%#MP(?dwYWk_y_GRU4SBp8S%IjJ$^Grfns) zn~G{S}I-Ii0eIk$3iQSJ81=Fbajw$^oiSz5QPoV|_Juq~^&H7TpMvVLoI z^A1+ymXi97MRn_|+qSbBH&rxkDP?boO|7eI`TX4I{1o>ZE>#fT;Rt|Xw%FTvuAxohtv7hVeH zpUE%3S-A1X+|_q-P)nEJQOb$8_?oxy27)SaP03#ohM@wRmcQg+tKl1Xo~b@Vs!^T? zB)jm2e-Z0mSkt=vMse*A8PQ$@V@NMTHC*F@H!i;x1FNo3$2a0DZzNZdXR_<>sNpc_ zjekjR5_H}Cm*h9RErEM({U6*Ty!K8AQta|uVAz1<$#&9!WO9YfbeyM_DFecTv+UbcY&&S7C5QIS5eQGW5^KCwYA;oi1E zZsuNg5O$#+uF(PB2_b=LQ4qe#p+50JR-X3C`dS~aU9(zU)4;;q$kJ5ZPwCy1*44kcuT&*m;?d?L`?E;)l>}l#ox@)x6S7@lM z(9rl)UHwxHjkUUZ8pg&nXUB-tl&0RkZ+HnvboSrxn~b^Pn5MSeUw47(1Ej+Ue^!7#ew4 zTZQ_#0Irgv{ga~|0zFI|Ee&kV%v~5ZK29zn-r;Gnc{OG1p6>2ldv|Cz-o1nY}NT zWBzb<{?Wn$n*8GIm;A9n^pHpOIPf1$adCa+=p^P3hsJK78@@ixzkB2qzZ7lYTbX6; zu_>i4ehIeTQFIRk6^3WHr3c=ybPKa|4YTr$q`8M%I0c*7_&N9{WLI=06*OiPx0cs# zs$gULVp&ZefU3G-M^*jyrj9Qf+dpe;-&0cAom13Y+p?{pb60)aj+&;;MdckCxpf64 ztp#OmSw)S>*)>6txjsReJ)8c?liU@(_s7HPz5F5NGsL5I^^!y)QD)?1N7hVW-?kXr z4Jd$N2&t+Vo|MU%`X2H5y~W|7trW-<0N8)^(_fmL!o^^tql>_>vGI9e1IZ%FhtEld z2ZiU(PWB%-(*MiPr;Z+iI1Z}o=)sf64q+U@df3S$hbVw8_u1pr^Dj8~eDD_(Vf%js zhMhd{^NIa9B?k9@`{j>c?>ln5pL^y6fAF+m$OHllf@+J$&UpD#vKJV?B;6U$AcW*wLfJF1 z;yI;*LGMzzN;IMs-N*kOFFW8>BiTYyzZ8fbQBCt?q>1WIT>h@$y{#10Ne|aCAi;@I z{qOj3fSJuH`AxayJ*73USm= zw8!VS0`1ln_a+ziCKU9gS8fYVs>)^c71eJ~ENG9;WM`GNr{uDeGODr*o9bG&m$5e$ z)ov)P>MN?;m{knAYJFT*b8*d%;_7Yd^4kk4H{`MU3QD`v@*1;?TGR6yTYA2#YTl7m z+LBw*k;ZIG?VM{%=50ii4HJ|W_mbE&v{`h6j5&h{yY`NZ zIb*-##KOxV>64v5O?k)UYtx)HOwIM_mfGefAJIK*tUPTr%rvzuv^^tz{Ne){UUqaZ zN4o$om#`4e=t%FF$iT$dkkmM@xDcQh&BuZ6=ScT+atQVDjt>t=jS9_)56p=6N(god z^ER@#T&t(~Hx;Fi*Q`;~)zLRIQZ>+7sja!nKwrtsRN35Ajb@>3Wu<3ht7mJgXK$-x z&(O3n(|4kodH^0AXkK;(Hl}NJHCAe|L`V3QTngPwz&LzUf zIm+86*u~h5p=)QVZDXuX*H@$IYFp`R(R8$Jb+zma4Bf5if!=oE0nQPD4nbaeRwk=; zl~-x6)wIxszewN4l;LR~mKt9Omsy*4?D+2Bw?{_~oZ}xGmz|yg1*RAzo%BZ%p{Dy?3;&cl0v7NLadsnz;m7xP}757zALJP9Y{h z`{ZI)GdgN#7B?4H_Lfxj)HZFasM`Qh)3hDrS8dbI>iVr^Rei-|9W#8kfnJRryDIB9 z7P30>i(B$aS_?t>m9@ubR{KZhCTFmZo)VKB5niwhNar-4Xaxa!h}_4T3S7wJ=0j|! zUQRU0(E5QoZxVAUbQ>YZ<(>|X=>D|e9=aoP?{Ii`$f7J{*fds*F2m`5dgcmgrl$f( zwlFz?w)DubK^gY1|H?UW%eEP_tAHUzfch}EH4o;mpDH=E<9yuo-9gvI-iYEuf6N93OLBYg1 z{scDJb1?>a5Mu-6I3$=Dk%0W7R;x^o5d}7h0OL$w*+RTA1NXm&7J?1Wq(q#_ zpW$#A{1D*-BEps(4oPtfUrNLn_8C4Dh_Qr8(F+1w9Hq-?L!!n|wEZWfWYMMku@rmg z-^(%uE?W_S{K6&0FN6y(dDtV4*{zxRS0KVBW?v8Ro>zAqh|lSYNMom!_GVRVOvrC7 zsNIpp+MHb2m0H-9Te&s6d{Z)}6SriSc9%75t8LrO?)aSD@l|EhUNq#b*_v6_S5Uj1 z)wVCYYFkRt#+0HB-jT)GWj)Ngt)Q+lS$)OqO=$)7soAxu*)_OMp|mZpygjF+HLtWQ zDYKE)_!;~!WeuNY758Qqc4riJRyFS_t=U*q*<07Pi&@bFQCQQLlvkTm-j6~w_s-q@Ha_uM7IvogPR6#jA3274ItII&I$1J2Z9>x`0}_J_?M-#8j2Ipc zPC?#Y(P04z(ZR{FVQC2fFi>4IKSzeYvr~kxPhwOkxZE`BCBgsBmG7Kfm+a{k}1YwgDlw!C|(+;r1bs z4k3|1FVC0+kC-^%mUDQhQ&_N5c&JNcXyCf|u*@WytIZl+Wlc*Ry0^V^q_<17w^e{6 z{E|AhCh80WRhqt%sg{bFrY22C-@(|x#lpnf(K!xFEs$!z0L(@00HK(~) z(cNrxt&BA+^ekNMqBApCZ9RRT|MSbkL;HW_9~qULnwdR2BL{#D^A`lOw{Z3gCzo-b z2m%ZTV(yD%_oQ?8BI<2 zhhTBLEUNLy=iK9Gw%)rHZt&Ub2vAEVO(NSj|0CX zmx#T5Q|Qn!J$->%tuj6`J31s89^junHFWdM=z(KLejz=I%adWp z4cb^XuB4kEx%ieX0aCs1ba&kyZCxM%10`+pohdrC5NRtlKJspMh#_@HcRSc(iA z;7<-9!zKs$6GMDJ*TfKSVi@QpnjDc#jZyT%F2tFM#p$ud>GAgf!|K;Hf#4a;fhcd;SK6?rtYH_kaszNBr?{=EaYq?@dscCGY3(*RrWbYPRBmF{ z??}O&EbGw=F{Li2yf=&0m&EMItKMAIw!6Az+q#_Eh&X0ZMOR66Z((IuE~X_m7gcU< z=>DOu=ZA*Af0owmDr4^~tpjbhqqJrNv!c7EeMccYxVn#7-J4RtE^F9WR=c4DT|?Km z_kB@Pvmv9Xqoi`vk&}Wc?sZ_;jNmFqcpa5Tlz415HLb3eLm%{~@m z0SsQU#~k={fcBF7J}G(t+?oSDghOM`7OuRTqv&;Ud0GpwgrzTd_7<>9o^)`0=#_uN zoqq@S2rywttQ0W-crPh+g@ei0-pQ`Lm0f)+z4As(KqVyHBHN$wNSYE+hfz`ti(yya z0j@x4p>D&muzyK$2Q;>O$Zz}`JrdER<4@VmKQV5sy*U(hDdw_wB{YNfQzTOET-m!s!$zkzDsqxGd#~?RNOVAbu#ts%PA>L6LNs0OC z5$W-+k$#K-XNJF%LztIuN@PTCa#VhDG&9LBH5AV84#Dn5_U1}@>Z`Ri{-LJ)@!HjY zS6;J1ouEip2LPr*hQI(|n$}k847!FDUCV}{VP&CVWuoV3Vdi0N?q;Q7uKTglr+-sf zyGBoQrH;xfJ#`gRT?0otBfu5d=@9JZ9OCX3>E{>Y?;YV~AK+l&PP6v3b_;g*iw=xR zOJr76w65Q<`0shW z)3Y_!wlcQ%^Gqx(uHU$8=Z~lUd3vUQM0{dO4gedNkxxo4%N8HV7qA;1Bo~Af7^Xm9 zc7PUWFWKBZx#Ax2l)?$H51#Zg0x%*(&~*u<2`PZxAD*}a02>&+`|I$H(W#sLC;6pS z-C0a_PEn&@Xu6Y6yuEKc!!wHJ0k{gca1AlTw!;vLUKVbEvcPGH!& zyoMsIZ*8n=*#S2y>o-@{Z33}XSl(4xx0zM5KEJrNqILrS4DM{{-dn-mP*l;8TiB4C zUXh;5&L{u`)`!HFCZ^TzKg3-ollZ|5K^KSWfF}&Yp)S%j#}{KRkw`JDpzzUxk2(ue zS+Zr(`93gx`oW=tgK-C)r8qZv{2QQDc)aTz4t70IJsnhA$JF%2iRIm3bAXD`A@Q%L z$BrI4eC*IKCyyLDeeCclh@*G};ROskdGvi(J&{}|5nzN~#4HilHy`-v#KE6V9L8^* zIQ+}uU-o~t>)W5cKlke?(a3-l&^0wSHwnBPlurP_hQ$*@LeODjl>8b%hK&vKMh3X! z!ygrQD8j5Zz7UKq=f7T;l(9=8Oypy zhI3um#tGoTIb~wQf%HPLg>8p2Y&)c4`Ad?x;}LQ!2LTwRz_5=4BpT_YC#FYa2AOCn zxUuYQ2%y4h+3(Z^2lcAqE4W-3Lx2%;sYm3set^1M_DDjKid0WOYY#N>O)GA+TptVcoWzicNVH zTk^}{Z3)qvUD(R3?8~p{P0C@{wCyQlZ!4?cUe>rfw`y~0QBP8SUox|gD6n2u>*ujK zEh&Xv70tUEyFV+dSs#~N$}H<()ggzn3tL#VTMJp6ifVQ+>vq<5eN$4m1N#*lcUHIV zDXZO>QPfi2u(_1I2~DKXjlQO+wl^1UGn=vs*rm0-1r$Xn+f9bn~SZ|QAo6X@y@7ZRG26jhKCnV%Sv8Q~G@ zL%KsD1G`5wY6&vG*wKs)y#D?7=~t^wl<+2wxJ#_;XXbw zL4ir(K?y;g;T|@AHVj{ebFgDTypMmZk6VblTac$mh>veXa7bK2a&}=!O-tvd-Fv?N zxwUt*Z&WPZ%a`UGWbPHf@C~sE46_T0a0rfY4h?sYjC79*^Na}(%ScWr%n8p-v2?Rr zqob^>uWI68=@8=Tl@Q>Y666{0W9bFIt&S=~TZyI%3{$7+>slKcJJan$y#v-IIY)*W zIN7L}8>(3v>ROx7+-;ovT>)SWcRN#Oy0MeFuDz*=o2`3HR8~!6*WMq$=pX)Nkbi7K zc6LfJ#GU8Q-jXlkf?AOr&g}Q#y(__O>%vZF zbAYK+fTe3V*1g=qOr1h3UBYQjp)|X|gp8{6qL!@E4tD$Ax~82~^;=7-*B6v`msG+f zL=)-ZIc(%%=2$MgUk?Y-%Ui^s<^K56T?`hjG`S?a>n#3Hfem>V0G4h>8nKXQ!p z&|?|wEswNOhGWVK(;fV=W(uZNeYoGxC!Xu69py`-4RPMp+|MhQw66F*bo3qsF7sp zGNmvvC0$MxVvypB5Nln}5p{ql(zh)ns*4Jwm*TmY0F3l{z)!<7CZ8-r5fJ|r32CE8 zT_yjoDQ5D^P*)yQry`mq;)+W$vrKwWCGdE7Go$}6sf;Q}Up@{pQ`2D2u$g{-iFhWx z#J$58-*Fc)i#VxxImmxj+i@r&xg{p8F)_D2B&8YxL|8-yJ20_4I;%FdxIHnyZCybZ zvwUMVv%Rpq7oxa!Q+`EHLRK}isyCygt*CBeVeOX6)_v)v8}e)RFxh*2V@k5i`pR2A zODkC)m(!YF+`;bHQ&83(lUQ7cn}asgH0?;uX<(MGFRk5C*0`s*ad%bw7ezJO07ywW zjoIa0Ib~?$3qQZGW_?mlT|!n(N?u)NNpo63!@68{baGKnNfUsrsCr9QVej`p&x}o8 zpP0Ei#RGudK%x^rYgAx3=kS0_G7uv4-y)-ObI+u6&#}E;{){7jJjr`F#k&W}Z1%z{ zzWkB&(kuC8oD9R^u*H|63$F#F?T};+UvuVPa~I!IODW;TG&UT*Azxx{7M@t%g;(73 zuXyJ{d!ei0a+?N7DA03P{t#gsJuy~Ze1WdPWEzZ5Cc~)q`ep5fqK!c7|E1TmRR;kW z@ksm==%u*%e;~!Eo(>`A58qG>6I=y=y_&iBdh)`X(fOC7ilvR_?apsWILWenxg?`nD!kJ`QftfkEl9;rYp- zIq^a15#I4WE@4hIcME+hJq;5LIO(hEYp5IQDC=o`s;=~Rb({cOtEauzNPn$~k&>D5 zS~Fu63o|WiD=ix`?I#|2g7|_i> zRZ;uviZyH1b&Sky%L>$k^GM z?(Jsd?_=O-t88Jk)=XcGZfsy@LHDq;^|FVs@~|^^wK8>~8@td<+!!=3m+;K&+V#73 z{e1eLXJ-0G#iu6}Q^KqA^N(Z;Ahxh|n4GndVL}LOpa+0q77=bp6ga<(YrqzMm*Myn zDI>w#S_tv`jNlROaN=VA5ER(CQOqBn8M=1j+(O^hpUbLx@=KcB1LEy{V(tAB7+%qo z-lBUVz${#Yu@TWT!qPp$+%=T$7G`Ga=j;`qmfsYW%E~NhZ|K_B-0^vR>rS}p%rEQA zDQYdL>;;*Hdns8R1!Zl8tj?;&Ez}w?7JB=(rQ}qW)%MjiZpkfbNl34WPOD8wtN-e| z5w74qA9H{Y$Q0MzDc6vTO(?~A}N`XyGTpS-qhE0yoLrjdyhX;8lj+{Dn zh_q}Fy+yTeAh+HF?97ovC#W?m{lx9}(Ed}051a*mk=A;OV1> zfBJs^_um{lbCQqw!(sWv$lS#6>;wW#3iO&70jdcnhEa!24DvCfILMzG5snY>M+ZdX zLz3|k@#L6{Fl=^)Boet(7rE1yW@avNxK}xX8(hIHuHZMG@Frh$OCY_6$wMU8GPjnq zc4!|3%ph1I+`>8*NgfJF@(|!8oLfSHMVd9l6s6{Ico`Wsw}kY9`)8NnTOtgg7mh7c z+o~W)(vl)2s4?Qni|XokGX6ygDQfL9DI*3$JAt-{8iOi+HZ&5P<#4e_GXkz$dCID7GRhy#eSIk4jtG6T-be1%I#qRq# zKCiW~ZhJ-R7ui*tfqz*gT`cy-bvbpQz)CB63(I;+tJW7*^sqZV&#&CVtl3`C_IY+$ zPZ@hVL{ZI_y3Q{unzj{I_f@y;s_xhg|5tWdM^1T1L3u|>buY7`YaO#byR0?4v@@-s zBe%Hk+n*+eC$3ITU!CS&;Yn_x9}RiNP&A`Y3i?>0+obG)Sn*Ick532!46ZFCk1_Sa zgAdCSIcO$$%o{+LOeX8{XJs*eUgHG zk^($pJOCkdPfJ4^T`eZ6FoIU-4*J}e^*_-QbT36uErWe9VHX}RYuw? z4764o>8~+1P&PHvW>^^6(T(ga;N))MZlljI`&4P|$E#Ma(o)m3)HnCAq5ERl$;gdi z>|tf*VQm}W;t}EN8ynys>+cig=^f+c9_i*4=^nl=I4mW=H`3kF-`?KG#@^dbpJwv6 zHEaIAkN@#MfBp1-K3VZUD^@8Ro0+@&S@{Or2ZcL_$2x|^*aU>y1_rrDMTVxO#^k3b z6lKQcr@4m*s2l5iqV)H*y2>`5b|H!3ky-I!88Lw=;bCb}zHtF&uIB2N+R7FXI2E96 zWp3zTWoT!m3!0KF25SUnT zZ}$9S`NAV~&+g_umKIlA8+d)Z{;0Lr&ngFd4dHmn_{NH%In?NrLV3gG_$t4mIhQ9TSgkf`| z!@^%r4W2mM51=}I?C|Mh{p5I<0E}Sl=%KSm`$3r<-v8q>y?fPf&}*a8_)& zDh5Ig{^6nhhrjvi@X6zxp#kaWkbE5IH8M9j0!LI=*r_oh!ayCy8wU{v3>)I3>C^yV zOfohkqK29$MirzHagjH3Nx;3rg~0y98(jF)1h<4@h}!}Q&JnMQ1ZR)`&(clO&i}Y6D*zz*K@V_I@yr{qg z?_K991@_+nTh3=n;f;+`UlhQI%z8xjhb=dzVm4BW$wLttiunpM6bDZ44J zv@55uJ-?)LT~0%OSvRrPZ3cye$C|$U%8dmzTl1>7=hf`VDBBX6Qj^8nSl#hWe(m<$ zn$6`6+lwo^GxFJ~Sv7^_-9;7as~T{wdE@Sb&3!*qx9+R$+?QY3*UwuBbFSo%V$q3~LG$(ei31A1M2CAp02S7g`T zO0K@f4Vf2T3ob4{#BfCZT+W0x@+nG3pID5r-x`sR0 z`ZG+OjCCx5UK%O}YAOaAswUc+phApv0AQc0DXr2}U8$|UQdf1gk>+Y+4LGjQ*Zhmx z+D|l8R892tt<5alY#f8!?R=cojr2ZVwdUhBYuD&$=~|o8eI0Cr-7L@po?+lhGjyiW zyzE>y+yd%AW69XbrgM;FI-GZH5f*hR!oOCS>|GIL;Up`s! zzn`r5>pxchO+`hUZbSD9vi1wH3krp>4GeY&3H6MPj>^hRV&)|jWydqq1Ck<*?P;qt z*Zy5)g|eQim79%UbYNgYm`jM4mA4}(DxkQgg$^iDEh`f>nu(gFxvq_sp{D#|O_zM{qZa z0s)5I8S-(QRz{5l00SL1J}f|bofAwA2_}bxlY^p(Ve$B|m}C*L9g#F5E^?;Mg8~!q zu2M-+9{(0!cuOGqO(eMk03ZN+?+5>Wp%l6C3;_nGJjGLNEJQ~l)LUXAyr^0ic509> zg&@BOv~ccn^D!&cqs#EL%ZODYAYAeX8c3$rhJn=U+-e99(#V06YI? za{kqzaEaY}G%~FtCapO!yOlE9$S&Vd*0dX_6O~#KpIx6h0NiTr^mz|zhms8T5UDlFO+?Ze0Sz5m-kJX!y*|cNdnc*?AVSsyuFS{!w zd)29RLL9W`zRMNiM;-w-hwG~0IpIj2ObZ`P@^HHlJiP!g5XXv(uN0TwO3%LrfZ@Wh z^VqvV6cYMEQ65o~vKN!GB?!vlaApov7tvm`Px%W=(#x-B3)PgekA05b&De~&hQAlZLQG3gbokO9CXH-KJ*ScG9B zsuFhf9ah4y9r4}tHrLmo8(Voh*#^3qxX=t8Or1hp0uucKl6>64oy?q!HO(}X^i@`AuT?SD z(6Kf)vbR(-(OsdT4Dt(ifN3ayssjMiP_fX{U>Iv!nJb$aeX6UuT2D*O+`!n0;T-H~ z<>9bWUG*<3SAU|UqHL&V0$l}0fc9Mf$@Gp3I0CO-fm&;PJyoabmLEyR{!PGl^=h)`md{2D;XG=I(b<8hJgHX z3JbRn4zdpp^i7D3%FO_Jr4;7H=B*1zj<9gGU8AqQMn`p(mhz_>Yn1fVG);AMEcG?a zbX1HqmGo8D=qRfhYXZs4Tx_)%mYNKjwlzb?nxJLJYL>=^j@DMbuGW6e zbZ>hLH-?R$gKM~_Ly(J&zb(y+&hWJhNJ+@9X>Q;7!x#M{hlivy(%W)0f_fmvkM1MV zNCxNYI|At)Y~PUM+7!h+FzlgNfdC^JMbIU;dE|;4A1{8$BdePxdG|-B?_%}qJ-~h) zx%o@KsIF-jtF|Y%xYpG#(!nR%&Nr6kL1Y$@UuMohbTYY2Ck^y;w@3h(nXPYlTwXz0 zdwd!TL>15r5Li*Sv7o#wEw4T~t2ztqfP2cTdutoFG_>t#?%D%l3$#~t{tzsd?;_{HC0WO>tT6CG5`+pI#W5#!bP*XZF4DA|&IW`_WxE7YjvqFaY7=djN8t zh;(^SPEmsUJj^|kY$5Ii{tv+ZTZ%1zN!k%P++~140KNG9n-swC)X%#CVS<;wdm#6v&hmxu691lYl! zj)MX_a_IQsBj0?z|Cjv(1HVd#2NlbWh@*<}5jhSi56i}e5o6ePI4Yh3ZVdp&1XIJJ zDaXI?;nA;T{4IG4HH%N))XKK~Y$!Gt#iqT51vKgg&e6FJ?#Ur0~B ze2&E}vNTzU-&&F=mP=XK*EfqfL-E`TVsQQ(J#$Ig5Hp9Q2F8P*GrJ_6e~AD?u#qWR z+@!p`cTpmL3iOi050R6sAW>S>GSUz962Au?(QUYPfEuNsHZoHwM2cP#1-6fi5sG9n z82T$N|3&h}7gF3C@&ZrI*u?RAa_-fLWH2mehi_*bH_?X-`A< z*Xf0=k*O7#CEXcCodp#=xh3s|tnR|fp7MsRg*6*X>NeN5@5w3aEW0*PFY7`Wp7S#Yf(if#Jaq?^umUm(&jW~ zLo#qQjlFs2(Seck6BCy>{Odf)U22$*bSFLF@Ug!Mvk{7ilG(>(Z#9{J5Ix2{+T1(v zSx_v#fR{tIu!Kz<=U-v+aNz}i{w0rWKf&A}@s*#H;U2Xq`Kt*jiZIUnYe3iZ>=I`l z;yHf-ofT!5UP~^&7G1`bn4(LVc*I_hdHf+HZHSX(RN9dA*<&jQLDy?3GLMwXuD->Y zWx_Kmi%4ON3JP5xz$PR99kLsLNUj64-idIo`A=#x46}$=A#iz^;M%`9SKm%ud^0xx z>Llk5tMj{$++#z6)YnrPWS{oQLjBzoesivl>rkauF8Xc7tn#x+1Cf2^L zcEO%@L2gc=p5C$k;Te&k>7mZSjt16xs>W(-bXAlMRP}94EWPY4+-;PNwLex_wNg`c zwT|WrZS@tp8f%PnwXDnyU2GWsp7tSurmoKFmgedfCZ^6d40n5NbAyl9toq-TtN*%2 zN!7@})WzDu(~jorXyxb3@N=U1I+%Ofn!8##_&NK=`3IsgZa~PokbuNMq*t83ZM)nTYK7n@rp^m{}?vYVmvC)Ak2?>R{NkzF4 z=_w(}ao#aOAOw|+;aA1h$5ncopc+3_U;7W$)vL9X0BK5kpf9yFjCHh3_4F8~+H_O2 zA-1BaGc43-W-8`JD&_`i7WyjY`kK}jrtS{b0d7|Q&SbL9769fR>FpKc;}+@W7~)_b z>=Klkm|nqd-tyI+pUxZ~mJ8?ZD;5!8f$`gelW@FyZsgYK!OLHKJ5*TKSo<)tfKm)?CO-9T2|c#kXGfj z>v5cm)s@L?%qVC8ja62SgI{G;J&oS*fRTFG2g?b}B@%$$pW)q|0b$3v$zRSQ-VpGv3wYP~+$#e9HId*3A20?m z<6P#TOB2S-w|94RR_H71=N5={-fXBg@* zlw!cPv0)KBW@>zPYJ8q*sRxE}XD$JKd1QSIpQ?e0?@6#Tf>K3~#b|^3OpJbpPsE6* z7h*W0!MSe^g%`%`5;e0dR6GYbN#9w3fMB(+H?4k(JIERC@Wam8^ z_)2G=5vlYPV1uJz%dL-?@1?SfBv5-{rjaBDL1aq^zz}A$j}c%B1lR`!CX}E)6BBD- zB=PK1>D-cR;e`bE5x)>DzThvu<`RZY$X{>RGZ7fm=owa;kky)$)tph(l~K}}i@opr za?86@nT@Fh&1r?LrR*)N`pp&fTk=_5)vdd#+x9f|epTDL2Tjqc)@POUWR~_a**nUc zzpm}xAD7jdQ_|bEaes1QS43KEVdaL3x=k$hmiUaC?9$Gfww;X~pVc<)=<55HRl5To znOWSCUfjy={G3^}K9AL1RJ##G8!(LB^?6}UUuJP@S>2|*lJ@-4wyMU>`K9ft+0_~O z>`Z0@$iakl?B2}>&JN8@OkKtoREm=%z@5AJS`O5iN8?I3;?Or*@gl<}E?iMQE5Z!Vbnd;?f*1)P}9vf-bsJ^Q+DhBq`&=}WD)-mU;mfz`kz2A>WFT?7v7A{ zKL7Hl;G>ZAq{!?w3Q9k=$h+j7;CQ8Q&u)mR@PTup}zWm*L?c1(rQCHYuCsS zkGOEJ#BkrFh~Tv7h^$zjSYInoo>uRYO z87LX)t=894F*VS(wlHwCVFY+NMg&;}co{m`>Dw~QUF;1QW-C?K{O`(7|7XRjzpPrL zW@2dWYGdkZW$JEi=>sruu>^p5*w8&}tv&4=gPc5~+`XbbJfhvb;(UB#eSKs7JR`l` zBfPw#{XHW6t-KsH%=P}EzV`2`Dr@xgOdK8T0s@@EA{;})oI^u>sw;KXSL$l4HPlr$(qE&ewNg`g6^7~>Ej1-wbqymeEmK_; zLygtCYO3Z2>U2{T3nOISyieO82s-d)eChJGe*r z1SE&}#QC^JxVVP7ho>jy)igG5`Qq~f=f(tA73Uwz=O4o-nY=q62_z4=!aG9g`abdU_%G z&0Cu~_aK}qy0QxzG7IW+ikiy_(pa_Ya|@dwShc;a-Fqt8>*1di*R*FBvvZ0YOKW{iRM}U#`4fKGZg8P0?*m55S4mIDN;oRgB5ys~uy+~Xa2`~iQ>padi z>`I@x#07dG!_H5QicTLref;2|Q%I>JC;Ca}At``S=|s4PQf7!Klx37nBDQ;AF$~9< zu^vWzr?8uG|4#?M`ReG&6QaR$bHf9QQNpnC5yaToFg2>I7{i_p+4P8HdRPP)1At8r z3Z@1HlY<0fAjrl6V2Y{nxhc|bGBt4?0ftiY1{d@t5L$RwAiOJ)!rKEUf-xxviU&?= zM1BFQBy%sN=U<8!mT(ew9*uEvE|>@`F;Q2Nxh1$qy!cXxr$3TQL<8UFq6ORnOdjw8 z0=3%gb7aXpe7PjX>eO=p7`10vjNBqY8XZM4V9zs=3>U8u`2}ZT(zSs-8o!fi<=@4# zk0dw^a-T|xQw}U-uxxq28P};O9t!}y=<18kNy76lg>c?PUkL=*^xT`?UBjLs)m~v` zNjdFl1)bSteVJvw*%ccLs<*DoZ%<>krQ|oK7qu4RU{-G?vpFicB)zC5r>uv?-k!~D z%`fR>x9%&c-j-LfF`3z!#oC@<{aIjaC9`5vL-)7qO8Vf=lG-g*4O>fUHl-D{vYNIv zc75L1v8SN4qig+l?Ze?F)Ne569RA5q0eQsH2 zc413iaa(akPhojyL0MaVX$zCpfxk~eMN|7XX9nft)0apB25r)~{0AH!CK9;9dt~H* zdVb&<_aRy9`f!?ee;U{({aprE9`JlFJXc(NEdzOqF7-=7P)pRX!8~Oa^;tMZiS6<66zn0;E*K5w=OD+T{bdi7uFX0N6i?4a- zUkb1b@s;orB1&}ewFLNd@r@XlexXkX83KC=0K+DGz#&R5>|m4;yD5rZ%e@=aKo}Oo z{*d3m4bIY=e*(S8K-iz6Ys8`B8py9dh%pDI6FHaOOf9}RE4cenOnzo$R=lN~`C9#z z0Omj$znZ2R1~fwrBh8gsYyP3Wa<$f)6>6(LUc2g3RaLsXyI*pme@dcXYJ5mWVnl9o zU<#c6=$hv0szyp`MrxK$bdPW^-^5_w#8A4s-CAAsHQE|V26`$c`f8>I%0@aWCc0`C zM(Q+E11D=^cROP@J3}WM1AB%J-E_5<^50hf{jaM&{maUge_gXi*+|dGfo5cHZscfb zxkUKWJRCI4 zbXRC7{X|@nce^*wEcl`rG@xnvJ!V}pn`T@dcnc30y56vcfR#si`F zE=O={ns;M{e*+X4=~;L%&bvP}eP?j&&bjg1XU2XzJ$&Qzz|}*?r7d0iD%rg`g>_C| zp|+mEbeABSa|qyy;T8@6vv3WgMwl(&PPcFiw@@=De+x&SgpAVk0`|Jx+Pc;q-J8B? z?fJZ#y$JxeE~h#*8y)4V8a7weZ-jdaN?O=0n=9D81!XOHrHz?IwHeIXyt0=3^7g2d zvha14F&Pc9X|2sY{XZUCgx_jp{MPssW)Y`(w{i9h{$8^3gD-s~k~{=9Lx@0tk|ZjL z``A`T+U}|AcOI!}bKv4sj2jv`g5?Ql$^(L$iKcRilbl;KyjwiMO}^kZiZH=7As+(J zb>lt2Zh!&{1Z@xOxe~5GLtYYw2+3S|w4httou^n-Ge0F*QMA+i=#&Hb2v9x`4x*f$0`|@i(3y7^q$!#uW@60OS#H`y<(X<1I#qRiwS=m?B zyscyXH_ct2W#%`vc7Fl1 zW`OEyEUD}T9hP0#l$KjtRMDAH&=?$9P*J!2#984a?+RCZn=g5QTm3k9XSjFZlR~aj z?+I{87{1A|=*xRB#d$C>bqlLs3hV+E&pw&G{8qg1Tr!V=Vj8zL_iYxlX3*t#y{x zwXC|1rF(z(+vnTg#>aPsM9N#Qz4m`pSKn%ZUL>1Cg?-I{G1OrsW$I`7^r719{sG}>sgao( zQCS($IccG3n_NRZX>2P~hMAGQu?^eaKQ?$vZc==~*6`F=8#l)_#v3t8!_rLO#>~jU za-*%;Mmuu@2TLO-D`Tdug@=<3k8SPEw)b(hX4BUgZus5m4}ZI6^>5a!{q6d7dRC?u zY#SQe#>&m!(%sS4hsg}~bPnJEz+3`2oKOxY%$@G%WXE-21i89~dwE3rdc_2I#Ra=X z_`8JqI&s}i9Ie)wWA&23^$AGW;=6g14|?SjW3w|sd*#(tCKY5dxbCY=^?z@qx5ms^ z&&op2inh_t*1+CY&&JZ&$GpkS#74j(cavU zX{ATE(08I4G3+f|S>|r8*501XkN~gP2yRT6XPCchfG6AEBWO!>T4_N>MPWo*wBHv0 zu#}kM#`^ZbeS1Fn_nE2rnT6*dzqC3yw^3e@?%fA+?EgRorkuelA~k%O>R*PX)ZCXU zNWYZirbvDhGzwr(AiX~!x(5LJar*8LQ+G~}-8}iz)ssJ5{PUON^{qgfp_J@0j$Z`J zH^Rv~g5eVd^kVa3;eZN`f=5U1NCz+YZn!-sz%wW=qp%?*r>?N7r>S#)@Af}+Z2P3X zWmiSrKxM;FX-$7^^X}S~UFCHHB~?9bJ^MTRJ}#_m%P(&!uIenT=*TN?FRtrLFK!J> zD2`39Pt55`FWR~L)3I+(UOIX9`q`Esr3NO%|nC1rnqg);cFe1Vbied~UxeovnNkE!_{6a6HT#nq5WAP9~ z*i|?HfJw!8jGhS5We{N^>=)LY{bBON@UaskBPT~;4O4@`h!>Icgpm^B$>HG>xW%=? zFbZI!?|8NXwqf}0*TaX8{O!-*ee>_JA5Uw}ozaY+(M+KWaTXr==-mf~X+d34whsP; zdg`2Fl2TnjuQT!$5jLino=^!UW(1R@sd*B9PZVK7{w0ABz1P#C8zSi)nd~mbFu4lb zT}g>=5tS3sPbj=VLZQ-HlmZoWq#g(WBWesWM$jckyb<|Dq!_^-{?H9K?b_7pY& z#y$;8tWL>mFRkB`T+p3b+*{VTv$1_&@6cZhDtZelx-063D(iNXRP{HuA1J9FsA$-c zoY$0`*PK;?nV(5nwT+zznmP|vx9)2n{Gy_5cVXQ?S=}J$t&D=^vf6&Q5zH^^sBYR` zR@a}J+Zeg2u)1;gvG1i*qDvwLo*;b5y(!UMya|)uL01>{NZ<{e2#XDmriFK5O$u(3 zk}Nji&0JX4UU)^(3*U~pCCMCuM>zXZpj!r90l=ozFUJ*&<0@d-;*@4lpuxNq`P`EF z;RCn=W95kaB8Ju6o&pH|Ut3$G&DNWkUtTy8N^1($65jlw<=;c?AK6 zJ3EklMdePdh_LIw$gbf|7-n-2 z5hl3s>Vwdx~8@i?@G@c8 z0uM}?b|xV2oUKe)Hs&6Vc787OAPzIgi|y-UWNG=k)ocIf!`1(5)#?v7ZqTzaHDy{_ zvaKy$Y-nzF)*f_*zng2QH!IMC707W9;W+y_+j`jAc-Y(dFr9d8Cm;9?oUB|O&6su$ zJXcU)7EF5sTbj8u-8CdQFfkq&7Q8hnHYY2oq%gC(B&9UhH#)??&f+6u{ne&M>uBa1 zZEQB$0=?`^=#D0I2V=UefdkFJ-fV-F;W`t2eG5ZVTQe(1OIy0N6Wh_k(Q1{k-a2bz zeY(Xudt(3?PK4pOTX}LE{CS+H2(Oq3@2C*3Fh3U8mBr;mB`4)p)l{~(=hT!&XC!1+ zmGtc0xBCx&{m0ioNtAa$erac)!I@0S3rb1gHh}g4&`YI*h4jMuA!%V&>mDl9aE->6 zDZco+Kypp0x-C)O2Y`)>?wt|b0eYPTdY!!v^6U8N`R#kZs;uj;Z0HHwl;-Xq>Ea*F z#QI?@+c(a|ZxfTqFK1uuUv@&6fe4FWcm@Z@rlsW8rsp>SuBf?`wavR4TKCj6?W}6t z-qf+bcks{6ogY^<4wcvT^=<#Gv12FDtEj3C=v7eBkyF-QSiNm?Mon}|Rc856YSH$R z#xFkqN_F(7t3OWNniSp>2DQBAFRq9Ju8Sd>3lGxJELL4e_KUXn?VPBzk$OJ&i% zw`@=f3^%~!?@NdeB-q8gVyPEOa6q#}c8mBpZYg9piSR;zQ3^}~Di$YfUBnWi@REps z0ocHw(w_Zsdh{p&Z1m*l=<$)GqhvN1<*G*ocI3#3qr>mF!G67t9sLg0=+SS74}XQh ziAT}gas24ffB)mVfByZ{>63Ca)1LuQ=_o%ve**WkrYF#s$e)}g4a?f;aW#$v!<-Jq zGy+V{Kc|?+Z7>CYTq_vY3CK>Eh`%7fbi@nO{7Zb{RSbazfRRF?0zO3~uYuSRpJ5a) z*}+i0}tgSyP0Abbs#3g#$=$z zK!*W1=P=AyKK}wotDMK#U^0yC!&cnGXRd9|>i3PVh|ldx%I`|f?@TZ5Ev(+v*!@Lj z=|FMg{`|UKX(e6BxlI}Q?YSi#nK)H>u%-8prL{Zj+76V~50%t!?rE!mb+3jjkK zJvXW#F}u06VNZU|uK4WMjG~UFj*m+!2a+@Ek}|5(a~nZ^<&}2AIa1lUqp9~$Ze?Fa zNq0_JS6XgER$)tP&%u_iKjfG76xV|g94x6DEN>VpsO;XFQ(IcsS6I>A*s;H+d1rmw zp3>U>u-M#^s=kral4*Zt{97W~ zeeFEF5l=Mdm$VCTYXWx_izIpLg=~)OY|SCMrqnMcRf`kK#Yq(^uqpMDQ1_CLQ?@Z# zQMK?ARoCTLz#Ao{z2;t0+=4|NkR7jA(uJ2)a~N)7fpWU^R(*-=cwy!tJpT&wrQiVl zhp$P(?^_Kp>?)AyXFwNxOLO&&=K5Rp^|vUt{#!hJ*9wM1qJPGKM3T^>4H|f(R2_c>RHEh>Hcw)YQq4=4?)5SQ*=y8rhneJK7ppS*$h!`L+Hd1N~J7 z1{*ACjy!MA$WV5mhpnf>MhC-nR(gi^D3#d3o)PKs$;H`m*@?W^KvM?ocLr;I5AZWL zG;pvmWjdO>I@35z3wOG?tG$_vovDkR8OPq%m(2+DUsz?f-nV`ga>P zY@nH%fkbn$v2wAo?6Z< z8>~(A?akKP8W}RIEZkUD?(Q~Tp6sw-@3SOz5=pPjs`-G&4_0@sQXMFx&ef+-Fe$!vraUE^4VX_mneO zU=)!{GhcjzFS#mG+!QPBp#l@%Kf}NK{p9VFXF+~lKl$V3?~bWEwtZGo*_n`5)rd~S0^qVpS*cSbcZj+nkQB~X5iocY-Z-EZuYrm7ENd~Ivi|90b5O6Eu;es zxW3YpOENg9ooy<@--Pa((-ZzBduGsJ} zF(Tp?7qKJ0+XI9D?8vvrhQA#-@)fp%4S#cT_`B01BS*eI^3T7GoH{N&`x7WI5LMOTbl;=ms^=Qc6Uae5E#v85|1DqC!jZpUA!x*7R^R$)Z^I94>Q|{6bLa@T!MH zBdNL~+3QkJx(ydfF@n06Wct(#yj&7{g$nD3i*oIX?~zy^Nmw806~k#>-W{ReVUGYT04vCcjr{}meg&}E$PlL?aC-<$}DUxB{S^` zD*JNEyNc?zXB79uq}Aot?5XJZb9L9Bx8$~GmG{+mep=P?hlHHA{Hp$K+y9c0*Os2! zR8-m3(6+C@)6||-0HD=^D z3JkX%7qEp~2?`9&if^=+-fD0}3y|-%0;Stapp@duE7Vxn$PDy)JA3UvTFg#-t-bzM zcjG@ZH~#|+BV|JnXP7lbl@K*J0P|-IrzZ?kUH@5i4Kt`c zYr%y#AK19ltX#~j*%szbrUq6 zW#evVi0$)R$CYv(5)>! zTtM$x{&7JM7ICN`w zM_Z1glaDjQkL|=`(0!eq1KFG??~v5Uh?HoiuZxkL3HF@X(2Q&?%^j`H9BB?5dcfw; z`23WZ%y@Qym#w=i!#{u%5y_2?{p!X{(IauQFO^U~NRB)9CJWh6lsY+IYSBMstR( zkt0pd&cu*zZO(Q!W3z0%y*y*1xmz}Qf(VNW@eU7g@pX4}XT@$!t7+@+-ShdjeSfU( z?Caih@XuczJA7hFs=lRLcseup9PUFUDojP9!~}IKs?@v#*en7}J41=E`wI1ai2}zq z@^NU$O}^y1Kz3auzY8DhN%4a-(|1pg-8^;X=7}G!AOGRfr++z9R^4Ax)j{kV(LgU3 z-xxM88Yso^iE!pdG05&0gBMHZ;uIAoFWSjH$Uic*q-I-gNjsdGwIoiyqPD-XZm7Jb zzp-s!&)^pWyZ#393(j6pW`n!`+S&g}QDtXi$8K0<4Fh>qI3+#1yelEADLJnzwQxsK z{TGM+E_$K?_HV^tWbMHc4f zy-RwOJr+oDZ-A;0QcijlWU~9jWDi`wFCdmJ0<(J}98V1Nx+Nn3lS@H?feyPNli*I6 z4CU9=si_MB!KLY`g~`cz{`3W+X>=1~(vzd7j}MO=9|eXTJwANoIJ!2zJBh7bSV+X8 z;!z|S)&50APmzk^x1)!@!8t5Pzr}dpBVV5yK70z@9K-+q$MEPd|EC|+6K7|q#sFZT zx8?+s=Ye5Z4Cr>Ig*olnoTxjT60;wdM#e7&CP=Zzt(%q#6S++z; zF}Q^%+h#9Rq^_s{Xw9#n4oi;sJYF$X*c^r?4r<=YuK`BP@hi}R1GVn?isnXoVHFXL z@-t5WS1PI}46TVz$p`|R5QZ?rt`ZWKh+Au@BjsqALfEk5Vy>+d- z8#+F2?)|i=cBra#e?i4SMo~|G&7ShM&nr9r6qnJQQqWOYy}h*Qn!c*$ zT}2h!;J?i)?TF8;N-t{8FGo*rWEO#Hu@0VpDPMRk z#qNbyDk|p?UXQx%we~W4H`FM_-hv=gUV5b>GHd3>FC>QJ?aa0R%;Nf4a|J;47NnT! z>TAFok#T5EeBTy^&ydtaSjww!(Nz!FgJY7)fdISmv+T+*GK}|qBe)0*d;0;)m+s7U zVt6}R&@BwDj6li&42*2dL4N&q{o3DbSpR!N!!;Hb>nv$&ElfAsn%i^i9Xzdpc?LEc z^(_tEL%bt1HU}hxI|sO#FwED|HmovR|B>msHRc=FS(zC-JKB1CI{I>*{Jb6gJRJQv z_B=OxUsrk{he5`&u*1CELcQ%c^z~*N*BEUuwYRi&cedd;TYJzQxGaV*+a=K5Bf`rw z#>ab;Z$MILWJYXUZoEf`j~;E~8siP?%}n)aG-Ep}8q3Z-)F&=GDYZN|sVJQr6Xfj6 zWd(s&@BVw!JZ(X6R^PM7J?!IGMY+ zIQjW{Z;JKV9OoGw+4^@QGq_Bc0p=Vv~!qOWKmN>)^h!sdImA(~gFg-PH|4`6Zq8 zt^42(bZF0)eLMf$(*1Gwz~}wD{#w(#v#_GGw5F%IXMaWW_Tsw!!rGp!($18^uFY9p zsm1#`cb@p?=)%c~+hgMU(=yNmWcDsVYwp?1`Nf&}MffAYmt=CpQyic^`($PoH$|w6 z7b&iiy=&D|k?e_3N{zO9pMOZr8b`VVEdB*a561^Fev zrIceWQ6{-AewTboj`zV@FON8vz|gnwqJ>V26(% z!JV+MyVKYK4R{;=>dWDyhsV$SsF^r3vl8ihZjL{W z45Jbb1rz51VDLOipx2Rd;tYRWOQticrZIF%F@9D77@MBZ3Z@WXfLnBP2rrX-EUbv) zKNEm0zpGR|QczY7?K3&tYtP{A_OHzGxCbV|99YuAJhKG2LWxE)rI1qO8O_`>f^Gh# zoLsypfT`w}$z~Y(K2*9F>J?w&3*`**Og)PWQ!^-sJy~j8&v3iz9n2_q2DLFpSvF9d zX~>YUm7L#S?SYYJ5RJ3Xaqn$@SvLPdvhWH6XV1S9oyYjTN%c}y$3I*GayTJ*;YoE- zDRrrZ-MN)R<;@2&%Lh`5dUC4;Y(%U_eZ3Y3)E&%kK8UKbO?)u5CY1 zTsfFkw5_b=52Y=iR(5=mnA07*wI-*cFFvn5Hlr!Cq`RViM_EljiT&Ku)ODbwuD`5t zsI~u-%GNyvzu zOv%M<=B-&Z32Bu;yWq&I9Lqr(X50$9T z$f&S)fxpTtZ_%cB1r-<-UB8lEPaf5zCrozrt?cR>jF`Iii~Q;@;>)1Je)+&NjN=~U z%HlC>U2H5FmNqVScCHQ8*ervki!n~$4uP8s>rhtIN z;J}nH-^8H6q>zY=sL0G1-B^5*S8fSix^=N@b9!b=|DIi+{p-*_N5437`k%+o9{E8ikYCjR zz2m%y6?$66=dbVbly8A~0x>&G4e)&W?`$T{Oa{>j1YAl9K zVUB@sF}y?F{i8C98xwPC(+isG+V+&!4V6~)S2t`gtM1D$>%>thy`R>#?5%0u)zH3g z*MYD4cm1ugeQ#aMj*9xeg7Wsv;+DeNZCR!5iP;Uw1zoY}9eK5%9r}l4^xVxef_p$O z39&?L=LoleUgsBe3ro85O9W%jsfAK(7PrLYWHTQ(a!583>0hRxg+4FBa`1j7ozl*3{4!jDS*L}OK-?XL_HP8K?Nidm|Ya`=O)G! zKb;ysK79Px=*WrVz_F2IM=_9|GI1O~a^&RE;giFFG|E-K;y?VacY_RWK8fk|nEZPR z?TEwQ{QKy)U;lXKNA>tw9hx|RQsZ-E8*H8r;%@@iiWGw{QHLSS1Sr&IrpDF$3Dxwt zaz%uVtN4>M!fBjx$;X=Kg796ch)8@*1Q?Uu27s-25wZA&O~>+C%;&|fFXGye&%Kb4 z`BGvMHjk1EZHkzGNR<%LF;Buc;L1fchbhi4OILQGUQ$VkXh%f3_ELd;&@ZTTJ`&JJ zxHY?o2^~b1f#8;d-XcZC6{Hb>A-#xYgGBm1l~ZB0E4F(aY(`DxR^w(HW-uyd7S(gh zWP|4=6&)*@eEyVNY^?Ygye;dEH=T-5}hzRn!jx1)F;g0?Ar>KdEj1xVr5i=)A1bp5nT} z%#t>^pUWv}FRbjTXc(ei509laeeoHU70pA*IkmYZ?P<9UTT;q1@|(ir^0sEx9z7+T zmRy|_T!%Z<33wEf*_1vO*eo3l*D%JqKo`Y-> zXgAQ4##ltcuM(11osW|FSHkyH4DosYzcn>Z!7+Xx|e)H zFJPEJwQAz%9Uy<;!;T zaIj%pS+lGdUQSFOmX))ep|!LP%Ix7}!{vI9OV+tmr>O26P(>S31p|PIF;euo>nqPNpvQCa$*T?v8d`7v~_3 zYq*zhd|*u8rY$9jfm=hJ1Kq7KFWAx6gGu-Ia0%ym#Rd6p3H4753)~v!mlW)^$u~GT zI5Hz5BsDBxbD*u8-R}*sLG43hgH>h*mM#uK@ex}~aOgT=$S*_pmVc@Nn1AFqe=ZR-i9C zz?Z@GV)=Qx1$a68xHx$*Y+S6ZU9GIx7!he?X9@r_b+$BiwuFC|g&WhJ>%)nR@Yxj0 z-4yE;9qAq(PE3Z*y2irav+ z3E{n;CT}1A>Bh+E%OfW){Q2)w9~KXty(5@DQN(`;+6w{3 z27viQv3U`W9RJWQnMHNIsRfO>r5)AHJ4&kus_J%>*A5g{_13lRZtXr;*{}mn;Ii8O z%7*RT1Aij5#69&byP7(87gn^xIayHMn^DxZHLpD}r)x{rww9rz-=4noBmeHS6h{N& zj>z1Sc7ACFpmpJe?gHEc!g@g-U+UoT{0r^;qMRUz*Z=^407*naRC*S#eI#KB0fzQM zbRNFX-N5siWC7z&7%3s*Hked)p8}Xzau<$euoU;GQ6nHqbY%G0@W}Cz!za;3Pa2tr0bOWNJc$Ly z!=uE1hzl!S52H%Vg&F6vD8oDoG^-E0zM`>PV+AaMVG|V8+gyId;kDb zP?mH|-@yK5R96I403Nvx0JbQlI+uUt)IgXKM;>n?fyKY zY%n3WBcrUpvgJ@s^TG7I4p3UN*ZowC!&i{99$$ zpUPT4Pt5L4%4#iX*i+haD5rLJX8E@2mOaJQ+sf+(07d2XgJpFCZM}z@I`>yM?y76u zU)y>huc9xnvahIqFs-O1ySyW(v;*WbJ+IkTpudRuyaLrv?>^n%8U`hm>C zmW+bt{PM1(%<9C<%6}g@H!Z$0ExIu!yfG=f#ur}~%5O`Qc-9M~xBy@if}4EtT_uiH z!^(t)Y(`2nPetmdqM1d$c2S^tA=JDSD3_*Wi{sM8G3m1j*|SO6vnkoLY58MNX&~2B z3on!lXvV>J9Q&PL3A8V!RL`f?ivTe6Zp^+AYM;Z$1GE3}t_3-#IR8QoaJu+f1JVlc zg92_Dw3qhM&nmRkzfxRW#uWTZrylViu&iu_vH z^r9Y#^N`YE3iu)KOdQfHZzvH)Mg2;zqL1SPM-QgGo1+y73#PdP$J)iunG@*l=@xLssu52^*dd1ed?3%a?1mA(bT*vZ@4(c9AA!_>yz+?-`$;AFm$ zZez%FuyAMD`?y=WF-+-p#!faS&bH?8fAL}1`MNSgd_1E4{S!h%QX(U>Vxsdm`z3`k zLp*If={6j?6W5Iu4bBpqIi}3Y~@%K#(3D_Fpmk=DD5+0oq6_OGfuqD9B(_tM= zZ?(DpYBPh6OxBw?S_E$i&nU~wt0~Q{Du~UB=f#Ew#cv5uPKHOX=x}C`kF5v8l5K13 zNHcb_26|b$vYdQ8fL?4rKhN+8ZcH2}GRiGH!ZR|`CpO9>Jj^x7-#yfq8y)Bs#$y3= zeV7a`%g)2emg8jY=3vgWF=JVqvH{5;zg$3m`NeMrUWO#cb7LZye%@AWM>8jDONN~* z&!@b({lH(p`TV=ne?K<%&G(`y`Q@2~Cz?698(oQpBM}lf4GTT=YV?kS0#jq>@&m=p zU8&}lM14yN4ATO@NDGMc!IbDO+=xvH0bn=8idzEdok`){A17`d`|;|LlM9DObO%2@ zQC!iLo?Gb?9OKFhb;Ug}P+;NC-l0rxB*Q1t(K~|f6Nv!xi)Ha5nLeRRufUXo+RU=n zjG~r?_Prh34wY2)R@4mvuHcNtXZk*A>fBe~w!5Ky4}96&^>Jx6kZiEHvOBk=HLI{W zyR<#GvO6iKIjvw@LiV=2nos`lrTFyZ?P)2x36$Dr+BrZMfNGg~)GfS%gV}{wM;RTW8D%{qHWOroBhbk40h@=>%#ttFSRq(f6rs{qQqz>5tBRl^HHX_3C`FRZH zpp5nSUGj+Oh=g9~1|-me3!FqpC`iJkuyRQ1hiL0~w+)6nUYPyyTse=Ugvr~AnH?(Z z5~E_S-lf`WF#LA~U}RV`200Qzrch%$Hc7w&fGsQMUrW*9y-dvq5zW4qE&SiV56=Z{ zY7E+3pODv+4w|W;8&*O6o|KYpk*W3RrTw{8+pAkX&MWOLuN^9@9?U7|C@kwKuN&$d z_`I@lS4s6?Wz(*Ly6x57pXJu?D{A^Ay##bpdrrkb9?+|DdroD4dE<_pk`7Q#a7F-= zvI<)OV71M=(2iKrT~@cfxOya^$@TC{}lK3;>fTdcT;+ujmP+?W=l zFA?P_jvaVFohDMvQ;Fu8bOtMa;+YpBq}L0e*M#`_xcK>)`0<4F$+YYVU-m?#cnq}D zEP!lURxDs88SYo$9TbA<(kxA@p7T}D1ls4A=ZiJTMc|eSsI%}=xqwZ0J)YV%BHix0_Lvv)Iw#PT>EpV=!t8ej2EQ{j zTxDvyo@Qa}Xam=M0|%OsgQc+ph%-YYJ43pUom(iK9c1t1V`c45Gh>Ce{78>di8SWVw;S~|)5gyEPKm7Cf#IbWqP*l2wr>fZ}QcRmB?s=lXR-DA@`%-iq-Ph_KtF`wO+B-5e0PLnz zjR2!$2LOyOxj!kqGs(Xp5MLL`Z%_dHe&XiH_m>WzIREXiW@y*HtLnBTXB4~ogs^== zTzw;1+%P6Ln8^)s_QCXC0x+thIhx6faNzj+gvMnSH^*mG^2RW;_hY`1`dusIqmGT~lFqP_#MtK)R z7%aK$CIF0D7-=EAA`xDd3a^SmgiXzjkI7G;I(K3iTft5rI|jHqfwNSOP~FW`f7l6} zb%`c>;ynBg7)G4*NH5A^KT1eO+#g3yjEsKu&*8)0o;!DTW?~%EG)O-((u*3W35$<= zV7T6w69toKvgc2N*qTGphP}_~=?N7T%psW63Hew^6au}b7bf8~f(3!-GGB~WK(P#% zeP0QmE7c>!$t=oWT)-zX8wEGIbnvGp?a6b?c(CxA(qUwy3js!f>b*phlTx7y4lq!P zh&D9VgJJ_ZlAf@4J`S>;l726oT&2M6E8;?=9QFiM6t}3ymDGH)2ZotpO41ERUMW{T z27s;1dBGk4Am}UcEJ=~TvBIxJx>umUM#t|Yc2{=QjzVBt>)xWOZG{!xa2wa!a}Z7w(&qoCmaao>J)eLMt8Cm^ z-n6@_ZGTDK02VF!{!q~{R8BQe*7jG_59XD2B&Annkd&CPtx++lk()B2BesTy#QX7Ly*#75JY#)f`NaAA#Q1H> zPRwg4$*Rc@ONsH03XIE2i%m~=@$?oOAwJQe+!)NJ@{bMnj|$*L`g_BQ4(5dW(|uhnIgVy7 zb{6h*TW=3mF#M8n0f}+ExM=_AaAz+!eVWla(~XAK78Xu+OfL`5z`(?;{9S+e>sQCd z&I#r%{&4x|$%QY!7I$v@yuNuTE-{+;Ap0+?gA}@{MBq#jyP1Slmdacc_D#UqpNsoK$djl~ncAx5C|GQ(kFvZPQRw z`@X9Bp_1xt=t!&|0EX4J?5bG?+WA(;p(ZuO)pY5y!dkF0<2}-#aFOqFTDnU&0c`@3Lim}Ei7vY!&G>+MI!*Q zQ)6Wn1Sr`FqXG|Cn!Tu=Fw%iQa!eEth>83@>4myaX)p44M<%oKBW}l5q@DMCbQrk z7?qRR_zHSMZu-VFkhM$RXA$kxmf#L zs#_MSfL`{NGPK6S0TTY{}6LMuKcVZ6-D`Fgcg6Z>8v}|IY_&Owl)QVsAqCbin#yFxxxa!`Yj;(b9CSsp(oX^9`1khW4PnEKTTk zG`6!D!(PwIc)huvHQSOK?a7PfxQ9DC1=`zi?aWxT4ORy0EDRhx-TmX@Sbp4fmWFz^ zCNwv?J&)rY#Py5~^-YKf+#2bh9O@C{$K4d-mk6x#1Eu8>>TSWY)weRWc5`tL3-gJK z_1zNdwP0xXj4tw2+9@ps>`C(6sQdjL7)Hgv_d(+}eVa zlFZ=5a0ZV>_h$P>hSFCXI@%QtK ziRDH{y9bB41_ir^hHyedJi>y!qJq6+g8kzo0uv$wH-(072@g+*@{JAgjtb#~`8)cs zY3`0RcPIFF0lnP9L;d4p0~2EWH%EB{@hqII*PH2Wv@|xgx3YDiyZU-~`13e^0kO$h zyFdAdSbcrw{1e^WQ|%0nPN&+Ns3lh17b`FYQLetH(%zfVJyL5yyW9o^2J627lLNr+ z6M)?wpT0gRxF(d|pzXmTxrMH`diZ4vUArv20!>!y}009lABADmAAzGrys+ZD&PYZ$^G?R$)VaNn1tj zKuyyQIB7v?RW}W`^?uyG?GW677FKoTm9}OVHN(n)+sV?-jN*>8qOQd3&P}Q92mW;K zC;n~tRZ)K7p2u_T{Ii*bB^}xN(p^}d1#S^~5rComf~C9g3NEHAH;H6n^@;5EzXRCH z5KD?%xX{7zP^v(H0lnnFEua@D57k2@0F2O!th-9p9ku!%(o1$7R&EZ>AF-?vs8-!lVFq-UYp=j zr2={_fZkd$%B$voHTz$$hZ3WtkYQ8+glZP8iD<5McyoSr^zvA9RWrH(SH`s zy@oY#@Q0wyO#yM$QK^mbd0j~by$QKpDJA_OiB%Dam5Dh`Ipy73^P93u+l#7tvx{1* znsx%c8afWN_I}dX`Eh#__m;OF%&y*%S=I~ms&3j* zP!6B`4!Cc_zW&Yw+xPuz`^R5*ZU0M4@27dCUFrEP#nt@<<-L{l@ZEiI+XuKJ6Y{!p zOWR?=mvH*zm9>`DcPD0)#3bft7qtN1(sCPi9QgN7@a~JQ@P(kmuF4c}|8Y|&yNj-4 zkYCf+g_3(p%~O%=z8tyr9BYXxoLdcoSfX1J&n)v5OOujCpw~I!voZ10N$Jxm(c>v0 z?w=`j&(W`lZ^x2^xa5UeP+iY}UgPpd)2gRfgd~Hx$v)_kiezA}glCfap&UOgq|pND z1?#2y;%n`tH`BpuYYK9LTS{m0W&}mBTA$ML~ND-$C-GaAE+?&;|2&-97l z1jPE%J)HF|OxKy3Zm^&k*xH&n*&EaCOd0m33`cz%^EIY=hBk(dp0?a*4_+L{Ba+1o zcCzEznzJp|)AW9`{=?tv>lxeH8Q55Eure`mwz2kL(F1%qQK8(;(cH~p-f{jOFBGiTei=K8_8L0j0p9P3}y#-xdeIp#)SkW zhJ_{u$EJtJ=fx)G$0QcUCKkmd6m8yG4mVPn@b0ITWpJbXoxJFFuC@-Y4)#okHTvs6 zT(@?Ev8kS^*#BEosG zk-SZj09UV=0Pf}x-%Y^*aUoGj(NW2r5%*b2i`n9dFWwMHCJrtF%*BtiGyBH4@S>dD{QScb z)AHbEu)M0Fe_-e5tzG+STi}#y&ns!KZQ9v0@L7BBp~}X=+LocNfj{&NecIN0prXF7 zq^1*APB8$iX=_fyrqtTFl!mY^wWW2RjQn_UQhZOMe5lqv1A5H>z2={3&lB_E!VBHP zGDWQa4`4I@1sL4yFCoLw*tSUBvttN10E}eyzMDHtsV?lF!>~A_y;Q0PD%E|p=8;rvYI28hjZ%@-G0x z1b{Ij!o(5)*i9j}6WvGBD<8=KV>tHM#?}#%)Z3=!c<=E{F~3emdB>|My0kU6?AXO?TAWg z+?3UxUOpI;(HI(E5*}Z&HNQD7t+KRXu&L+cf{LEP%5AyCh*;aYx4vzEdE@qyrk$X- zDmwp|R=RytW_xr}Rbll&LG{l3nq6sy9d&KHYnylG6t|^h*MX?2ZQfPiy0@w8P|vQv zSG6Cg>pEE9zCW|D4Rlyd^X}Z@&Vur8z*TF{!ScERkXbpU?ZwsDMwVaJo}O1zP|=c> zR~Z$b9iLj7ThbPnSlZNi@bnoC&`X53Q#arPDUrk5cbhMUyVYw`!keVH`WP_wezQ=n zeu`5`XO@K8C4u_Iv}|ci{QRu&>6qx*g!JjS_|cT;p-}QbrhI}0{Dox&#y2mC;Kegb zeAV-5CCIO5Aio6aXJQ;pK#ed(fKl=b7Jg1<7v=MdXh($42BDYw61G3jTzRX5b@?p- z43_5d8~hd-=Awkh3(G(?-Ibs5TbEv|fJql#Qvd^EQ8D(^!Fxv{GVID*Bofh=)tCn|qiiFV5d@bAVfjmj%;twTZ!RH?02O zYghed{rXi##(FfGiGwYT;b7`yYr%4K4dVK52?M^OK_Iz!v9X?!5uQzMc+VY?>?GlH=+e6u^nZvCct>aom_dCm+^ED?@W98q=H2@MPJ$Iyt!09XWId zH#*(TEh)3;`!V_4!V}#*oXKSPGI0qL?L}IcRb&9M1_4H;AS$%?$+`;+!+2i}x@M;( zcPB&uuzTbD+h-@Q1H<^z+XCsGG2!jgV>d@nU;OuXlA+yS<`y-E#3VDl0@*&n?tT$2 z+;Fx}7@Hg6>=VJ_MZ5UMv3Rj`FZ`0}6#_ajF}<|7s;97`qif($_rNFJ{hxI9f6~@_ z5YEz;t^>V;p999~T6gvgezN=He{382tg?Op@K##gU0Ty!SlL-n(VduGzd5ZwDxo$q zv1#9DXU_6(38eRw+Q(|$6ZrQ6y>#cFzXRCPdw~6V>D~bhYgxo-fSX|p=;@#eP~?)g z5>t!#Cz(UoEd;7d^+>LKpi)0nX&$OI4^?U$rKZusF{V#F&o4 z*z_b3VU)4{R}7;B85KH(DTru3#B$=)xc0pQ69K(Iv`w5xlJOUCycu9jcv&pL-IQt3 z4Y&dlm(U|B-boJ92UtQDhC;zz0!k>jHsQAXdt$xc14E>}`xd3f6o z5d)uQmgE|0(h7Py0AOm(Gn}_VB01h~cM(NM00xL6ehp%pr^a1AA#dnIVsXMtwgU4+ zs5Hf=@B=j@3ts^uY#s*+X87#QE!hJh2@SCst;vPk;&ZwZ^SXjvUZQ+^umuyOs~zz zZ>gx?QBcuaRx{YvgB!-Rt-Fe@LH^r*Bjs2)A^h%3S{3S-nWQXqotXFwWWcrwVelCZ-b*VqLcDBrxe8}7sYQahU@Lt-0IAnnyf8( zo83d*Y~5`fIra=sM~1sS-Nnw<$=1Nke3ib@s*T3$jm`AT%r{!l46SWUm<&dMpJz;z zM|31G%p)?wD>|AJ5#bRP#*GW}-5l;47v>Wa%#8zjMFl3t1SLd8ZjA+i1;zw>hH~jX zu8v+Vj^3`eo-Q;u7rMV6$S=Pwn*e+6;ar-low0+71ILNYb#wN1aq@7cd$Hhv4hoFR z>)rVW)$HB5^G|hiaBkz~HfeUDOdO9Ss)rKQeG+$ePo}vm)80|)aCW6q`#`3FE9@<@ zH};5fa7>DD5A59Jt+SIi&rRW`F03&jD6pF&r!W5PpOd`O^r~suQCid2 z(6+a&`(Q&WwljARe7bA@mwiKjYUw;sR@+xqKUiMdms`@3Rnod8vo>~XZN!$Evc^w` zf4B%hm8$M*W}i@NW*&RIs1hQjz}_!Phs|Dmt-H7)zqIGkQ-DpfbI35t)1jSPdY1#c zQhibqjA5^s>akk$SfgEuQdDa(c}uN(sMOw9!rg!dgZ!0RIJm1OiClm#wd$T)2@34C zSa}ovyi(as37QfqU*c7f=n8-8{N#k@`;+6xhmVnJ;V~c=sTsa29*&Nl93FXDcJbu!=|=i3qS`qes5`ZseP9fBNaHW{hwP|1XpXo0tQD{rVUdx;#kA z6d@Tki}`{`00+8v6-7(|Ml(Ia4>>2sSgZw+ka8qm#7=EuX%kBCQc0WCRgbhb6aq{^YKbdqY#zP!a6kc!NVg@${1Q&0!r_^VYTc3w zfAS*oMoVmm7}&95JAAkGg~fBZ_9^H>l6plNzX--qa8qBZo`CECfT`Zix745r!`d?Z zK9afDu*7p{L=?=t{z?2Sv+QHvn5u}ajR|=@N%_60#r<20`=e6pqf=^PQ>wE{dx~m@ z3Tg+6>xME*I!kK@fnl&J8g^7S?=GnyN-OCqsNYl4bg-!5U{>k&xWuZwVxZ3M(#E~< znYE=g+nPG}Rn-q<-TcyP}@YFc-d*7p@v_mtK5=a#hQmbMgDbf)Fj6;*a-6*YhY%Pwk(O(?CX-#vOt zHaUGB-aCm5KaJCZtCLgL#;4#euiMa=TzgsOqUu^-Q3A&Q~r@$rsOw zpZ_F!az^xMO#E<6cz;rKpD(!&e_=J|USjy6K=TYF*M#EfB%llbtLM||=VILwpi2aA z2R6JfN-<*$2du!CQZ056s?Nhb4Zg+d1suu)8jJYpSKtJijQU8RZX?nS2eMpz0}2d& z2xfM`i{Vc6G7eq=1@>NkQ4FKLq!ieS#p9h35lFUDPDJAdJpYUAGS0W8MA!#jAzV(7 zmwT``lgDK77(QVhzLCDREXId=8$a4;u-eFIgN3=iHPQe0?00(W*O(e_uryz9ZnDwR)X>J-*ulZk z%hi8Va9nm=d|q5aPE2Cn=JfKE+#0y#=A>7qg(L;pxm#O0o7uWpTQV%Hoh|HG)-(sJ zjTYt~=^K8iZ?M|X7)Y#dLEA{9>04Vfe0e@`F>aAzF5w~Wn7teA9UbWt9p)1k;vEwR zeD#SA_T3Z?%QG^F9pLE`?g#YZhWWbqx;uHXZ9SNlZgexYlL^bo*4x7?Ix;jlA!O?& z&*%VF5QsAd!<)tCxiP%hj$UjA*WCp~o3~fQ=H$`u`STZ^>flnYd#aux9bgz4PR>u# zDrWB7zKxzPwP?{HWwp9p78KtNbRDu~(K zhI-)EwnLQ-gRmMq_SIuA^FU4Gj<)VYUHyM->p9rkwZE-PIzTu zWmjf#TYP3iRANnBYV(0VojJq5#g~DOM09EAurXOX`&2XcY-S$Yma%i0qyS(R!1)z` z=_p$VRYF97y}|*ExZ8oXcnvWwBEv9$4}AeB$Y!WPh&Y*Et$jM9dp?6uygs%GvfNn^NTg<*FU>RVFf4XNUWTzN;Xfa}67ylP0U2?SRJ z(--;E^J71XPL7;BK1@Opv6hJOzOc}ph@-*?sE8er}}!G-CY_)fGr9EESS1F%f?r zb=Wj4px2BL4yLfWh*K{47e&(mIW#2-1qd*{;4+9Xkrap5V6X*AwZJ_mbWFhg`*WOi zN$iM3^1f7J93tsiCdnOuE_BZmL{SQinpKa%9K@`MB8(&`;+MqbK}}jgdJ)s3Mz^Ha z;-nSy*29$%l@{@0P*O~TJ!-J3iP6j=9Z{5H zF%w;gWa97}zIJ&+xm4EtB`>Nr>iS=!G`GD%#pP468ruW`^ z2V-Mg;8q1rG=T{$2D`>54IbKn7 zu(+ZttDvE>{$P3Cq5RU0hW3xT`hHo_cr2^9J-@s=v#2${vZtW3H>a#SHM^;}>LAEF zOs=Xslvx1S>#wTs%P(oGY&=*}-3?N#xVo#bs;jEuNJ?fyY4x$M&P#_!r-forU`t}j z9f1%4c56g%8_VY7k43WkeDM;dgJSUzy)BaQb+P&x&`Y3LAC+$m$u_QtSFcH)jDY+S zuLz_M;JrpClx{-|JK%{`!T2huu2JQhP_-c%e+o;2=Vu#Ic)_(eaZLr<3+OfdOgV!n zQcjcLW~%WSST(z)z41~r_fkzIAI?38CzCIEhKnRb!-L}s*!87C$dNC(i8=Hnkp73U z1q2u-E22}8;u+G5GIXfs@s4tSyM9QO5Q!J@_1Ntn)VF_B-TDzT>2Ljj0P~LyWQO{B z1hHKFTxe_uuV4?KP#7a85{2~Hrj1&ve(99ueH@~6Z751mV3>ujcje~8P2}p zzCj5A%n%RvKo_?_XB#g|6BlC}hK+@jmA)mQ&Tx;FnY|B&ZrCzu#*WrD44S*2w|5Ya z?$74MMMPvK^5VnnSk604_P)1k_xn5de7JY-2l~5r80+gHG~@!kQp%Efv3|}>TYGm)JG!L>&D6r##LC6e(%H(u z(rkx;A^eZ*GBMo&470S*x3MyGuwjM-a1&#_q9d455v=G)uc$~)Tufk6G%qRKKQY)Z zA<#D_z$={V8RGBYO*gVPGjp<}`+0hD+3tQUCpH5C=H+HecQST&w)Xb$3=axOh~gzh z0Koj?f<1%T&TKbVAG)g#!^wx?3Q~;2c4hliHgqW`7IiaECZ|@lldEdd0Y=3~!H1Jr z5Fe3Y@DE73{9T25X$(Y|7Cjw`@fErHo?Hz(Xp8|w2L}RdWte||h`%%>xWkt$38eR~ z3-4YUy8YJy?eS0k1K1uKyPw4k_wWzJqc6WuH{T!+P9y_Mh`_BVPi~xtUliRp!p$et z#Va5wqr9lHC#R&ns=2?t@6)EP|{i5^W zC$-IeC9s98>xKPePDyKKaZ5&VM_N&LQchQ8^A~5oo#xB#OI44k$f$|QbtQ+0oP9amR+ZKQ0+Q;LV<-anaxvvIMs|A^hK2P<;ifa6>LWL&{3e`P@3NG>YP=v{D zi^aDE!UdsVX5^ZD@a)Bl1A~Lx3JljbgBLEHAH;63vq&lI^dcu-Bmxl+z_v{s=kSp1 z!WoR}xQG^fTm$FM{pml?p8cAC{X5<8b?wNIjtDR845I*s^ulwmd5T_eD8!G2VF?wP#jVxtAD1_M zl$cSMnpIm>*H>24o0L&iQG2MlMIH{`~3#wH+T9*Y*L= zN^ARC4}8|t`Egs%r&zRTMFbX3VX zyYNbfD8l$C%5z8!WWjPG8HTAO`HJH=sc1xa8Xl6V7VspDR1sgR@4%P8hNZssHx=Lw zM>GGRy#2R#yrQ_Qa9_`0rkkI$EyKdq*NGLxvY^?1XlU>OC@?dim#ML{jj^-6fvw#> zbKAXU)&|yAw(gF8VSa%zTxYgD&CkJ=N3&vD8qy3c-7KwWR@P3oCN>s^b{532?qqCl zw%5XFr?J6K14CGQ%}p6x-+=fiz|w9Dqj&e{y|-%*tUYF?rcO5cww4577JF>rA9b>( zd)PBsZd`U)a(GgHTxwx_Mp;Tubw++eZgyREa%pmCY7jfr(>s9f#c_9K**kbxTf3T@ zJDC{S7~z<)y?fr@r~iS0(ffu*JIu`VtSpV3ZCT-gyrfv4xF}XsgjY<2cT5B?Atod> znwu2t8}H{4;Ys7VI{LUedApf9S^vynhn~5*MJb z!tsg@V1@a6g!<4!eCR>m&VF=jy1j*qwGG|D!rjio-NC}uneNN`^}l{KIk%>rUeirs zUM=NEczXaQ$J_zG863@Wyt2E2wI=H>BBo##&wt_9A1yTM$AiWO&8xcJm=C2I# zm#>Z95lHUwrT4D!?_9cm^Ur4#{U`smytXSSGKt9v1%P>wJ}|m}hzBpigB!*0kMQu1 zVsc_U{i5kU;jZ3+OkQMOd0Tk{eCm3NYrC4dPc(I%XlOfL*L17{C#U@G=tuw3ckH*l z{lDry^z+^$Kkqtpy1cf#s^L&$$BD|u!@1?1sRd0@>9w(0&5_AXAOGT;A>lodVnvSg zs4#7WS`=d@r~u!|4GrAKisGhb_8BmVbcSJD7`8U!#{)g|@PPs0H32+>+u1G6EVhVI zRY%fBwlz8X9GjpKe(+7U57;y_8)|rwr`I&op!x9l4c3J2kyiUar(MxZVuv_2>Utbr zTjioswWv_ul_{5yUdnqC*=>pJmQ;RMh)vBiqeH3-1K$pu1Aq;J0=sY?XRKU8DK>yo z3~Pya&_xvtu>*_*(!)CYSGa#>5GRLi4|Jy1#es8w{?9Z2_2(-Xia0bmiVb1Jkw|7# z^7+$3;Vh=h3vPf2g9j-chKao(zd(2iNC>@X?u}~$7)GQR9*j)^;SgYNq#J%AiO7;j zZ;NG%A}QJd#ESbQi0iRj{a8Nn1hkHVG!ju!9V#>;{(mLJamtQ3^#a5eXfHeon|nzu z<l}IO_i>F>lW?oa1E&qLBKC|NIQ5jwP zOO7TN9L%gZ+SK9o7{cG33 zFAg33&9PH|Xzu*1w)tdz+sX3!!#Snx0LZ4UPs?f#p?3r4m>(#r?9M1?tZX=3Ro`D; zd$6*hH^024yuLfHtT8>mCa1V1F0~>hv-$UbxI8pEhkFk3EurWZU$7t$-o{oiC5~nu z72cI79-?xkW(=u8S3Sj)4dt^@`LpZNjjNK???jJ=#E(YB4}{Vcd{uF;upu2=6De2u z$~9P{a=7zEpnNQXUjq%QE{Z@>%_`6c?qDHNyFN}#98~d8J@2kqhy7)XEf)#RPOp{4#7plkc+i)1ll6;U>`1z^Ky z#G9|M75e52)xrz-TFo}JzE>}PufFqIbN6qsba($pv-qQ$1SF2#`QaVcATN(lZ)PZy z9^h)nw6bGb(|lcw9V|XH(f`oYSkKzjz|q{y&CcA-(ZtEoz{*z7%wo5>sj0IyE6CeF z+|Q9^NAq*02hnZ4ZOz;*Y&>o4=ys0oj&?3~rVbWH4(3KSrut@vd#GsO9lL+JW9LtH z?J%&mv~+jbYis$TnehjPhCkc8?_Iq;2DVnNTyI;Jo4$kP9$QOO7n-e?mxC7@0OsuD z8I%~3Qk0lgnVwghlUJ9MRh^k$l^&H7}AL1x%&rtGJRd#+0M=^M{5@=Ge?WP z7KS^G_kCz!u+!LNr-|7v3yWQr7Uu3W@5o?oa*SU>6gxK3CoaN2Au1p_hMN$|j`8=1 z@UZu_w)M1fWH~x|I>Z0M2cX~VEqz0RxFFX;0vSOZ8kg1GGaHg+&Ka4^%iw=lG|%`UAIDHcff{F+9$ic@Gw_Xat>rkcQT zMmSDVS3guuP_vjx{SW}AS&>gH%hY#e>U&bv1DvoRT^^u z>fFEoMcmSTI=8siKRC+6FBr5J(96?5jNuaeG`&OZfhVjG`ld9ncA6E3j20ZVO=8V`|nHg|Ts*_=O`>aE2#5IJKso zeTuPCq?SlA43ju;ZDP*~ITD)y!p*(V;W^p1*W({YXhgb^g5)M;4}mp-c1Ktn6H{vx zz%;tYI_*OpCBn21#^4oJFDq0_3gsP@YDuZOCs!f8qzWA0ELPkRN$&{7HwA*((P8z# z`ES4e>g?e83xfj#1Lw}e`sTs~5Mkd84qO^I|IOg}OM~YI$#K`k^Ir{~Lx6$y8lcjC zzosTQ{}tJWe}(SEul{oG-~Tvx@uG5QXp%pIXJB~Tg%!dZLNvjXaACm>0UyhNA|Y|c z3jt;zxiB~iL+Alf{22*`h5+Ga#3HOK5@klRees-(j9DR7L~$UpFYq50HU1_vr9G*jFRS8MPW6eBme1q!I^%NN3u}(2 zX1Ar~wiJ|i=aqETG@Wce@MU@Jk-Xy0+|thauG1YyepgU?EUn~VLH#GG#Ya*K`l3?n zQ*&D~ihF81zs#*TRM&R0x&2gm)qxBWpV86#IgqTO^<>9^FPb|)EvxA(sqRI96}M#; zG?!HM7FTvBXI7Wi^b#}u@sjFp{M1w($SrBhEUZh+D9JBxPRpxE&#Nk|=t#}3OU`Kc z_5ZmzGd|e0 z{E|;@$Z-=xpaq*qqQJoLMf;puJpd^{tHb&5Vr9O?DaS|77>BpX}KAvpqO-d8e_Fo|Uqqj z>)lqCJ55dYSeRNdoIJyQ8DV~Q-md2Ej+S&+D-VW+o4bLXt+|`MZxlB+H!iI_BeOC) zt12tADm}e&f5`qoH;$7t%hr`?M`PGHxLH^^TbS6Hn%Y}fy4V@pSr}NG>RX!WTUhKd zGuJb>*kx{R!EpDE4);%tg2j#v_e%)pC8KZNFFuqV$8`&Iv-h&L_q29mI?$NT_H+mM zCJqd@pqS{u_!#f-AVvt+J(xoe<$A{ib5o<)@nOuU0QUfIYnF?h4};G0ab(dgT^uZ& zZOoi34IRz(Z7pbC?BDzw$gfq*o}Yd)u6;t;H$Zh!4K5ND4r!Ea&lG;B7=NH3?s;Gs zS@-2?0NA2@0uyBf@@2kcd023Nh`#~=yFR*nb>traJ9l~NH~)I2uJu@IX0exluqUW4 zP+)$co_-+0Lg?I3cmGJbUo^u%hRKcf4}?K#!beX^voy{M$Ms^M@!MOSgnp@P~&u{n*g zIjvFYZ3j;L_jU0-F_nomf!k^lX88Xt^~4i0#Bx=weLSH9N%~Yfg@r*43F)AcINo>> z$vN5dmUarn*2}4R>>Jb0J*S$WsXOol^e$2in;}XLlNFy)YD`T^h*(iXhCS41SJV^B zW9s{=v3m;Tl0tD;sl2O%`>G}2mQ0C*NF=hmBKe|Fc2^+24FDS+8vo|f(7Ce!ufahy z&<~&v8@zB47n1DaApG$>2L7JiZvP@p&6wGNsfpwc6*NUcr~Z2J{MTpC4E*5_182`k zu3giO4ukw6^diUs%_kDx0D$rNa{|E)K7Upynis(m-VhU#;nM_jQo#*STChl#2j)oO z-x9a|_WggHpUsFwa{_ooydYQH6^R$c(t9G=vQV-tl-IY!#4(9MD00luwv_Jv{!o%BdA z(h7D}vDR|rze7_x!uPjDrMG03A1Q77c~a4V*qpYkvcoA^t@)+hxy9{86$h&uKWgs$ zImo7}#v_egANL&pZTIotS9N}#U3CQ8mg_#rEIk?&RhpXJR8ZMpSbr+Hpd+umtGe!R zVQEKfN=ZRQM^)otP*?Ss)%$T{`zMuk{eZ62>{?jpm9Oe8t?JD$=_sz~si^DQpHrWl zS(RUgJtujjgqW03l9E-HkX{tGzaVaZNy7e$sQ8l4fBDsQ{_Lm_-nAQ}qjN&hO}T0r z2iVH)jfl`qs@NV-{zNvuE+$7`BeL~T*;91Si=JE;J-#klxz4{Ul&xT(sCGlD-Vlv# z2o!6h(#L%16Ta-RSoH*`raE}_AWAS>h6zI_kNsOMqFv{{-}es^y9=GoQH{1#BP7DUHm&)IQMprvFRpuW`^b_CN?;n5PFX%$&nHQ5IkAHhkAPZ}og433%v9sz`DS9_nWn*iY zur7Zf!|VP7JoHjy@0Uumtkf*a0bmpNlr!l#7 zc2I6fb6M@7vYG=GHJ!DM2b$ZD)-?8&*Y>vdobEXI^Ui~xqxS0i1t>FEXqvC?uB`2? zZtTx5>nx}|kW+OaCc7ynt2Mjw#GlTq`SKO%*aMaN(HID<$&GO`F#hcthoyd^RtpD^hK3x85i7?dwFUroUo{Te>mJEPTvmeo8oP%UL~H`PCzUV35=j?v)|z+` zw!kAJnk!cXgM*gXWBvSa_T*29Ca!m9G^{*tp z36w#kvPWTzY6BW4$6!>m7aHYBTNsjz0vHMTg+Edjd?Llr1EQXOu9kQ1qs?nK(;HtFjt)HXEw(HOhQQi7=9dND>{%&q?(%qb{7>oFKXQ zvZ)uc>DTDv&~6EJ&rkjCdTjQ=gxubQyza#O1EsBBq?h-D0?Q~pm{ru3UD#e&(F43` z>H4C(@3(NLw&i$9&B2E5(;fZ4sqXwNtLku3h4FHqKm*5w2RJ;gVRmB+2yxks;2bT`@CMYnzVq;jkeoeA=RrvTS|G~A3b6bkpO~o8aF4SAI z&!rT)rZ?ff9RBLYmXgp*v-rd0^52nLWa;ky4eN)CIGveHx%?jBr@r|L{U8h9tL9<- zpuF)~apQaW{P*wB{9TygEZ;bP&nUKAsHaPi2M97inkC)V#MRaaaK&)2@^H3gx;lEh zJNdXf`no&u=uUwQd#;-u$Hm^?)saKDWxH5;Ia|Z>cD7~HoY-z2945op!^YXh$ja2% z!E%p<;m`K%d{2KTdNj}#Vs7AQV?=WVfayEgo4VRsc{tenxw?n3y`#B4u>o#8rkR@^ zJZGo5*-!WB>)Tl|L;Yg&l2b~vQYvy_rB-I`FHa9n4sh|MS-D%9IhmL_nCe>??$F=& z-cG&ucj@Wr8@{in_tTv_-`%__KnUUO`MI$J34F;>4smv1pD=XLgW(Y-(yma*|(EC_5^YmlPM07SBtG zUJP?rM?*UceQT3_7Dh&P7G};47BrXBzx?C$!iIJRuI7*4 z*3&TGZ~O@YY#ac#Dg`;Ocqml@x*h_((SoL63wlX4K z9unLG5q5p_{&z!5-wofwgf#KpD?{^N4~(5S{pY6k<0U`tqD-P_UiQAaO)m=EWdHkVX)_V$0-+yBe% zLth*@@w=`=pC35#i;lievx^$?OWVq74uWVaZ|uvjIgnD+9+TY~m)&{z^k1(E?}~BS zGN9tI7M&F6-Jt%Lw+o3{&|Ygeffue*D8kkzG{CT@aLv+8J%cqdjYnUEVQBFHlIe)> znxy)jN%|CV+v5}7=2787tp*BgOE>pSJ+(eDy)iNUbYgPj%^@6UU+f7RSFel_UyMrq zU~J+6z)Y#Sr&KNjy{O!JP+$`2l2~$AD1l4Ht>Mwh>(^z21K*(j0`+ws`qX#FB~T<fl?#A}16{ae5l9IZkDj7(SA_Kc9`WinJ{U7;Z{`4f0C#R=WWFcR8JB7Lbs zB8JqEpN%x|lJhX4z_8|tlQ}WY5tefDxorBCc=Cl9LTLX zkzUeYRChWmr6DG{GPks+r0#fTQ8&o7md?{zxebXaMR~=I<#oLU)+r1|7zUTJ$}18n#@NGRg*vf6`58C6A=i{48SwKCd6|uv}_DZ9`Pj)MKT~C6?%eZ8QIul^upkHQj(%F zh0y^jk~)v34D`^iN+;H&+6}R0U7}f&fe6D%a!=Lx!F)Bj@Otvr4>+3n21aj8+<1YG zM9Kg%yCt1^N)byw3ov_*v!R`1h95yz73qb|A zLaY6aSMnM74%qPbwqsp1^O8UP{2lKwCOgv0JBsZd;^`DXxA${#@T2)e`FaF0jT|iZ zTALa=SQ zGozUr+8OP&G~8*T|DN8ScX#N$_u=mMcj*0WC$4ut+zsoeJ9htM*Pagy49#4fXgqKC z5Pw#5ATyjx4`j0=0(h~Zyx3q~49_o|4FK~FWO?}0-F@BY{_YG8gBKZ;oS7MymJt{i z8^3#rWvsu?!rausAuK-i z^7XNq`KP++Ro&DRwf6BiiC0&V7+)D>B2qm;fRQm*a!m52Du=L^-k3Nr3sC_8Q{3lE zF+^&JzYOc@@VzV7?_3#L0D!@|e0}~u{~|tm>W?j*CnKU$-B?^EC)|@0#_$XE^bccl zqcAecKgJaR7MQ@`#kl)~(mXgxnWcGUZH1NHb%WT_F)(vgMk7YpIXO%?#;ihns|ssOD(+kj{vdC2@F>x?)fF966dW* z<)FisFh^C23=>Kg`J&sS{F&?56~GgTcOoIAaAEhB3Y*lH;!z&wh<=4!D1A;Onn0{rSJIe0ybL1nGqULkT4yxf?=2 z7b-BaPB=x%4OVarHy~$hO`u!@dR>=1 z92P$mNFM>%$jJrS8Q{D3;>TDhGA=wq(G?IZRX3-h4g1__vw6 zf1g?UA7Gf8OafC;Loyd$$!4Czl1y)jr(TGrUPxy!@lrPPQab%oH2Iv`pa^u&M|7J* zlUwh21v9+EJXv8Z*Fc7ypNkFK*}>PzKgN$0=&5gQ1lM9cb7MUV}>rU0cJFwyNjQjjk~RpwV}QhFvHr0>FmI! zy9N1pMsQq$yzKnlt=KdRZzpr6qq&EJ71PPm)4_`6l~S{d#%HGJPd|ED_Nt#9=VQe7P|}0Wm?rae@93 zKFk0QZ(vvulNI2>4)J0Jcm_rXXB8Br=N87MrN?BZ#bl?1?T_Ilg$1NU1f|7ruw5+F zJDg|BaN2EZ@PYom5BBc)aL-;nBU4*fkCUJMesbo?)XchWYTJK^LuoZnloY_!Ai_X? zp#qbtFd<1c{y>HRTc$$j3C8daXfpZOeX)FH6hovQY@dD&FI~BQ8vq9KYgD`h3hWPm z8a{aRHw~@*0U>d8b^wbP2@1@U8-b%z{302g7&lI=3pbV#6i??wxw3*h*rD0Q4Y_5l z6%B`5x;}32{VjKr&m#Un z?1~%L;%v`x?CaVdy#Wj(HVslm{O3cjx#v{bkb2}FYV7SX*bICjh&k*Q-*!L3x1D&R z(x9UR7J2a>lH*2D9?NpYU1~6xL;~o#g91gm2n-WTmqtXlhxv2YuBisjUODsC`3nOe zx4r>@ojY^(;sB^FEG)ihYCd;i;OxNpug_s7J=*92VCTO2%LO3SxwDtfpFRH-j#;7F zoX?*-cjoJh|Mx#G3|^F9ADS8+BC}K|fKi#eB3L(scnX4n8WJ&bi=b;;T~U98g?vI~ zc0(pzkc!_>?M*HsnG=S1BSSA|q(Cp=oOnSjzAeIRjwOj=1-4Cai{n7w^oQZu)zr2t z8qcWS*fj9;i)8px0HfNU)zf(WLbEkFI6=QC77t&jCts*_*bRn#dv78+aMFtQO*xTV zH-ScQC8fAU857`^^!20BpDY|UM2?3sa)mlV!$Kk=c0+;g{0+h|xQ4z|OudrmUJ5l^ z-^tb+5B)hbxh*O0KxA5TddZ>un&S~^wbALdi8&29c>Z(%R99ug(XyKUhISk>nUqkPMEiShutLjKvSzmV5@yL|A(3ql>%=)~t-mHSojDpsR`u?o^#`L^e?Bc3F zP~Y0$dGOP!`ooQ_r|OzNYHa_cpu8(3r#7dwt)j8NuH{5wMR#^lD=4t^+=i<9{`!{V z6?F$|TaGleA1|ry3604Ojn0Zn$Sta9%gnEjh%apK{rvKf78Dp?Fhgaj2qic9lG|d% z@|!{A4`pLdq{>x^VhvBW0AS)LR|OBRjILZBUixliQ6zr|0E24_$gfc)rb!KnAMz!S zgalyl~eOiYkl?Nc1iqTaTr!VX)eUXy}W)2@xdCu`xAZs9e~tDM_X z&Th)5pURPH&*Zvi3e7Y5#50+0Q;Jhvo@3A=4iy82J;f6A%=0n$sNO*aJPenTTtLGx!bG~K^-dwmN&}|KC zH;1OS-l21x89Z0dV1|1j-NDzz*4x>VVa*O_dIx(N+8XUKH`;4qy4T87&)V3~+0x$6 zl^MqLj`nuu((L^modaBGL9W0smmoJsuCuMLy{)f{x&Q_(0!yo=+UtGt8Uj z=Wa%`+F_!<$I{f^ljiL07L*v3Qd*E!Q<70%9AB8~=I>_gXsT~xyxYw11AYB>_4fYs z!(B+Moq9m5_jd1nch?@c^OK#sf1 zalzh^JkMZ1I+tbRYPZvH-+R0DKHQ`K!R~!}h89tY*;j^Dvp2TsrA6_vO3AOc3XGCp z+W=Dmz*YcY+XrCVI_#c&0PH{$zx*!N6~~B)R=P4QTDdlI59AjB?DDm3fQ^XmetUi9 z7r#B*)OoV9t}8G!jzIwC2@DI2p>v~L$p{rXFU}nk=wle%2%0C?KR7-$r!qCCy0+zL zPu~}<9Vbhwy2@&Lo4Y>2oVWIm+PXeDdg6E8y`S|S{=BvOWM)B4PDxW5;e@13=eyblMgX29es)0B8)8Un=0Zz15Ag_&WKEucj!NaZ@8+_;y+6E z4R9UAk)iNnN|pCz%KK9JJ(+wF=p~ij7Kv^Fy5zDY0+tUjxI=Ts(93!Wj&(2L(p;hYbM3 z&YuV5{prsazdp;qHZ;W_!W<2u06m0Mjs~EM)ctME2+x1^|eLqeV-M&lw1JNQ~`g{EW~ep>{XPPN%^9d#H+ z;_qadH3iObMk~j*MIL8YQX^D0@sJ-&Pt;@$62k$HprPaW#PrK?9TpfTK!;(^)-yG{ zi3qkWH3&5wrb$s4dn>$9_EC2T#wbmON{i%NQ?7BU$9em>PPc*6rL`Dhivc1!_`zN% zCSOP~k7J9ke$o5MrI6(Ii2bcexjm%~pA^=dh|6xtEI+irurn#ADKWDaBv(blk*bE{ zHO;5#a?r>IhUt!a!oSI`fH7CPU>cZnn@=JTFnos4F9>^+eudMH{ ztUZ)p(wd%In^o9YR?}18dbG6qKuzP(s)nPr%_qyTVHpHgJM0VV+ds-H>&(b+Dy=z~ zl382dbgaClx1_2wBd?~krYE49&@u8&L;1$Gkw zCSDMTZ&3gvsuZsnLfK=nVvV>8*RBg!zvDmr7N@b#kBaY;*APwfqp~L>k|)FBN7n@R zL4HYpeDEZNkibQ%DLBTqr4P2Ku_dg<@F#UG{@f0(-S!^FZ% zP-D`mP2iSf>bX$2g~Xb8D$u~%0EP*)&k$7FEfIMF08F6S6ivQ>2eD8*{ZcUfa&+e9 zJ1#yBG#>|d4$YP0>gYqWV%Qnj8Mym9`-Zd3oJ{vx81Az&Gqkrhq1l?c*_hI8XdE}M z2ru_QH`hQ{2Y&}!A3IBywVkh{y&q7@$B;a+_+h>dS(X!|grnjUCMPSQ>%u`5Dp+FHQPBnM!)$TScIj=cH@ zzyMvaWMhj;HC$pI;@D-$@{kbeg{NObOW$6-efj#$5%Jw&;o`;bbp0oP4_m_8#_rI_ zL>Es#7B7Ms5aSsbM-PZ~;Y7J}qZvrASOzcB-8b0TlM|Vky+5}yJ-?>D?bzYtzwYk+ zw4wD#4W{E9={)pBd(Wq>U7sB6|K;IhzdCf}7mXc9vy1Cl)pLhR_QG7qmFKcLF#28%QYvs()?;qw0;grnZ}!;h#`rufk($kyE!-HBn$n%_G$W z3NM*zMXIi#wOOM%IRn0vp6%1>SD&0=j}n4&&Q@P2@H*hI!vZV_q_HQsXD@& zwS^85bRJTtV(5h%BlMC@pczuRohh|#mw$T)mm2>_DK=DKlW?Kjz$V=70QD{9Gyv=c zY|tq0U=1fCQ>&9R>dUGR<`y(3rSXzCsy74$X5)ogJm|h9o!s%bR^$F>f>3NNr`AyYzeR+jV zIfV_WS>=V5Sj@;NYK}?B4UbAs%c+9z5FT4tSbp#?XC=ds=`L*5y|5b;lokV6DGo; zva2ZI6gV3l_W={QT_92nl`i4dx(rqLI{NR)4nsyZ6PlT=$@y2>`B%iAiVf`W%OIUV zRtYCIUw>nwW*qW z4)hXFft-3K)NTU6_!_*0KY|W}C7XEx(2`(#-xe8~^Fo5<#MiRf*YdgVB{MI@bFT$6 zFWzzWb%e#>xw!f}JNY`AyIJh9+HKFWMzjqFW#n;X8ncMr;}4|jdASN}r;gLn1x ze!6=%+`{_VUcH^B2FA|T)?PGwUpJRP7BiIX7v;x^=J-bk_(pMkqdDH;z8(Rdu)IRN zeM0=$!G3;WoPd~sfEfRvL~ckzP;g9eKx8mGnBx`b>&m7(vs@V*ma~t$ji(dMk3r{o zIWlRMG&@s!+nq)x@9)vyvCkwldjFLX_0+<;b_TBICDZE7y&_euh~zkub%?+G-4Fuo$~6Gkoo}x#42u>=#EXNM z#}6Ii)u>h zI;)!wH@2N@Y&%iY)K^s532F=jd=Gum+Q0tGgJK~q#*JrLJ_>OeDS|mfn%dm&Nk?V&wxF#vWxZkFRVS4QQViD-C5drx}fe0a*CP? z%iH1-^Fkw13QC)@i<&}W@-y;U|NT!xLnD)X!IW5Xb5t-t0!kV-E3zdCyhk`r=!r;) zy+z$OE5wS~|kpc3h@ zen=G(sWESgx!1(C@j@~83YL8Cm2~E%V*a)K#%txm_oA5>via}dar1KmfO!PC(gR!p zVAc#vV<&xccLQ#mH!GN7WN*CB+7tk0;)K1-rs%n_GIh140mFisE`Br{rma2O*}=yd zZXJD{?bwbquB%6ghXdQr%G1)`$H9^3?h@+Fj0t4LhO!bPSxHfz@!=kEq0EF(c5);q zEiNcCE-)h|EH^H;Fe$DeDXB0eu`oF@J1QVG$UDZz$=_Mu&Sa0dsgbRjQ&5`>f2( zTpS$O?ld0LJ=mKS;pY?O=NIY2i3W!8e4{wN(SBZ`-as#Ake6?mpHC=7ui${#fS@>T za1t*dh7%CY3y2Q%4)&$_xLecht=;WtY&SbkCnp~_uW$}Koa5@tu%$cOy3@=Z?ROa) zThrW6eD?c^nFo_M*0eKg6O))ctHLehc3Gb?Us1m|=q6Gkz@&;5+1Lu1X?Y(221|(m zqt<-@m{4|~FS$R&zkhWY74uJDO z7)ia;5>lGpd%elWBO~o~8w6eO^I)M8=%pGb&O;RlM2!5G!cO)r zz?3Tr#j+GG;?mooy=1bxlp5m;7e+>JjtK5t2ZoK#UHwjVVc_DfF@JOZYkxr;K|#L4>5iR zDN3$cpdhNcH~tO*MtzUEFt0)4;fBe#6~f=#=)f!OO&K| zNE`=Pu2f*$6gm~D4~PogiR21PGHc$dF)T6@CkSS%WAoW20&Mc7T=zmktQB%VTPtNU*Om4_7?=PIhr$B*mA_@~S8}h0SrI&W^&uuO$?l?{h;OWKO7da4`x^NU+Sy=CRrX608W zrj^8}mgE%FXJ%K%#%Jdj)fbeuB&1cQW;Xu%cLSHdQ;&>J3PcM-BQppv^djOuLyB#q zPlWO(SRO=Y{OWfjk1k(Z8N77s%C)<)G31a$f%M{w9|OQfgbxH#GCWU-UQ?+8hv%Wg zlaP!u%a9`NhHQKdo5S816yY*OWk>05yu@Uj>F09I2E1{i@h1Y+V?IE10#p}{69a%r zbwqf90t1X`*5sPCvDqyRwr%~OyZu^w^R;SvQ$D>Z)T|@wCP8)K$rjK{pxxwao{>{7 zP+h>SEwX@NTL`ULD)aZHYW{mU+%9}Cn|q%2o|U_ek)4H}sp&3b(;dbpJ57yEoNXPv+-*Hwj2tX?nduwYS=f2HyKuc- z0=yZaKHd@j>_`qf!iOEk@{RQIkL3DCb2%{_?@%AFU^bx3C)D3N*vBW#KQJyNATBUC zAuuF4m>a`khqB>0KsYB~cXPVEnX9#}r;{U##t!uhPKxl44g`QXc+wm_U11s8ShEAe zFI`tm-vs%EU9@T)cGysHI10-AL$;RW!?{N6uvkJwbCC)Fb{_zS9(tUjg7OO%(P03v zsq`*k*dn?c zM7Q9|E)d@23vQ3_?+gnThezkHUR4hcTt9ne@X`gW9|FM6ok1HtX=)z8IbkTpuuF_k z?Hnqtvu6g*ojrf<%(-)4W1I898Eh2812EKN7zaG?&42wFzK@WKLllf+@etdSu@ZPg zB7rq8C51%n_Yy4#1c0su%$P?w-9l(dZ!6_X3h7-1au?*|9l89DTzX3;CP!uDYtfx3 zo)c3zq~bjI)BMqCK7WpqaZ((5c}FD14q4P;^80A^Q&H0|*OVF(&w=sHtk4lHRP(_(@Y+~SP(fVv)FzL*!pf*bm63vzb0fsI_q3)$<^3}(` z8wpG4NXYArN^glwYYvL9j@sXlS=^Ufdbqgqa9&AgJ^@li{o(wwjM-ds)pPKv7XeXHi9aRYPBPK}%U} zZ%Iu@X>DhF@291;@GrBq9PE4U$G3dNF4^-v^V5i1^v z*BS`Lk})|yLkS>JYNhnTE{2Io(Lq5M@5f-UZbKFk$ihbjiXr7L$H%7)c09^tg+2kf4Bpqt6 zr@$~OI;OQOe z6_^+ho*wEG#Wwdaw`MyqB7K;#JZ4-F0L&*Lf}IrQn}X;H%S#H$iRY$9`6Wj9#)orL zBBKjZqwl8^>F7g z83Alyp^Z1)-p8F0#P*Hig{MSCq{jHi1Tg}=862jYFWtqP?&jmu(RXTM`T@y~TGdRh zj$w2!$u0xUK1Lc`wq^U9CNSk=${i$CJph1F@$*WIrC*kgFG|PnBEVED0x7mIUmIB_ z+Uxc=-`&1=`PMhzfdacbDp~xu|F~RM-Bnf70RStmZ1Lho*t&T$gW|nIQau9_Jp$r9 z0^>b+aUTAWu3mwj{$cs$O$8OLwJrU9Cw|>^@U!~Xqp*LhZ#!0AhZ*xdeP8q)|5a!2 z>DKO(RSmsWjR*6~Tk^_UvP)W1^BS|tIumo7qW3pcwtRMJ=!Qh~5Tue$w>~+!F*CC{ zg;I$C3@4{--(p_W^wY`NP3;U;6oEr{OH3SZ0jAk*lA{tCu)Z;_S)U;4YFl7wF-=kP zPsF0=^>$byj=92X-xCyY>c``1RA6HuYE`fv$=(1=20K~l5-hkQkuFIj2mp!X4xo!K zoaYN}@C6G1Fuvf<2>qaeSCw~+V@(JQ#!xdkE# zkG?3?1ppHv+it!UUNZ3=sqi*1478(6wkU%KrMDG0ob!%caT^wBGN}wn24ap%4HjZr z$E*llj+pTw!Pe$GVnDcTSwa8?FOgh@Q&;ep77&7$Gn|A10MlS7A{ywoCx=maiPO(j zurHq79CY6f#&(MuA?U$Ja!wmxmvm zySQ-Sn|Y~fO{xU#^%zJ7=o%KRAiZSxB~-J-KZdYi`zfO9Ex;%jj{+Ne9>W$8HjBEs z=Stn0V(KXwv$Bp8TU2YnEm$DGgyU-fFwyv$1Qu){@St=EW94S z1q_2*68n4e^*bLL@3L^U@(f|RhcKK2U7Z6QU4rbHQEtI$e#xaVVQE43EPKGlUK^9W zwif#Kwk9-33%ZLLoo4FhXyRgPMz^=~audA7>?H(&LJu?e^ z8(^xfCEd}+gGT4E*%5)>kpW&20j!81c5Jv;Oc>nvi{QEYd)jz9xN_-!ksSYsK;Q5H zb|{A(?C<#)SaGxY2?B5gc|9Cq5(l!Zq2<&1bsl^~ovF6J)XxbsSB4MRDV! z+8&fnnmHaz6v!}%;vw;attixFeu{cojsUv@0F$e+ga`nG-^lBu05CkQy>#XF#cvT{ zBci**qMKj-_Unq8&WftGvZ~gClKSL~B1iiFwd0pOBed+cF?+=x^=* ztnbt}UHxCR_kPjR{aJn68EM0Stoc;;;MZr)|1*HCqyNjA#{TMtzM^vU9~M=%v-4_M z`E{8&HLU!WFMo4+O1A<3z*#)9+MQqALj|_@dJg-sUe4cpgZsGsh(bzx0l;SNkknl( zzyMJ%X5gB!plqUIN=Q=4w(ekbFYt+s_nn@3F*CDA){AK}9f}0#QGP0_?zD?$>*+1y z^p@W6RIA5Qp#grXSTKY~oTa4J0=+iW>Q&TU+BKDCRjERHVStnTE;#_h5~5sz02`lp zczf*r*!bej8`?`3u3o-)5dn7DD#b`sGah^mT?S%ZBnM$vE?)QnKm}jB_`?OF(x}`i zQeC`kjkG6yW5dHk*T4JEYa_#2#pFG;;vr=$RKiljML#Oj%s%Go!7F6I0?`RrRtfn6M55mo;@t9URouZ>;Um0gNyQiV0AOY6?&6}R*9 zYYQcvdBv@ukQ!P)C7DmX#Z`Ufb%T{nXIch-UDNri!kW(n6{j+C>H%Ql>ON6L7lT(` zBJ8WjiheTHt@^3&Mzt(Pm9Xha|;`!wF6bP0}^RZp12{m zur60r$1kYGkdErE5=ldGX}t_Cs}fmGUL^pmu%xb{sx>#ihM844aQ5FWjp!$)?u<<= z!cu}tCNs5RrwA{KO1CA)Q7JgG@|I#}uTs!D7zc0I?_{5SF|+uFM3TNWpht#`5Hsy6j61M^SPHV_1Yo)u>>|e( z6o;_9G~az?dH8Pb(Vs2%K%l?T&A-qhy>=CZRj{UrCteQ13(JOxF8Gj)!g&YiA^>|& zl@0d?hT(^RG<=Rt(C_v4aNOm=Fw77B=iwh8bGP-1^!82k3(O3S;zcFoMr9Pln&?}xt&lK~y6+*h0ODkZe z3mKV(tQ0|dI4ugu`mw{|kB%St(DvxZ4o8oA*f|8cx`cbVMSHtNc?6~eg{FnOgnEDM zVh{9k_V@P)5A=x)@QLsbhzpITCr2|=BQin8B}XzdBbXUM$+3P>LH6G853zL%^Nz|$ zNMNML)6*hS;=RKHj(cGEucI#rxZr?I?%b56Qy(yEdQUjh&>9(_1^3 z1?8ikfBU=I`kuP_ZfQ+NVR3CmRSTV+@97`qAD$YK!iz}dhooi)r)2rZX1IsOrE&5L z%IeE%y1GyN5?BTDtG)Njj{dKkJI}ZEe$m$VdFQ~F&7J4kx;|}eJtdWOm6SJ?R(0V1 zv9_0)Uz3_$na-=OYWnixwRz1nCXCE2yg*=s#zNNy>Mamnun<&)Uf7$pxHq$Cg&B!@ zLU-cK;wwCHK|bMlPxN$Hc~6eGS;;n>8T8#hfCFWk8B15oP zfF}z0Um|hx>c<**M0(vP;x3z zTEu)$S?$qJk3q4N2ve$nl#fA#DYa`;>QyT07v7E7Fhl^h?|LVvVr2FT5QvOfLGz(; zzth>8GDYGOElcmF7vJGrHw^T`Ov~vdY!@?P+yjOM<7h-onZi3D(#wKC7)_6%;m>RrTZ)$coB4`J(#bs_w$duG)^T8vFl=UpA0e zbvl__!^o`#dI?I~xgwdMsHw2Dm6cVNnwcjot}n0YtE?HQZ93K3^HpQV7fqdCmDdc4 zE4p(<4MnAG*@BuJVV$V7MNlYX^UG41B4Aiqbw^cAM_zt~q@r0|R$o{m;|a>dl7@MN~|-(Ylp>Q{rsL8F6#SlW*@yX-+P00kJ*>n zdCWA+&ORtF_!)3hp z7l0gm9=K(G_*c|q_y7FC|2XucLr#ZXLtMOKJpklN3&3UU_yzdXTN}G4~kn z2zF8|pO(O9qzbvoLQZ@hBT>kL6~m{6GUI|XV?yci;f$n6c1m&{GhM_<7qaMuT!uI+ zfuH6R7jWFy;UoJ`KD0gbk=@ZlPIh*_Zf+4iZc*MIk>2hxz%Y-1q<{!ogimy!ji-y9 zueWDJSWse2cq*(YzqsI#)R-tna(q@sBt1P8WmB40bog;EmyeDg`q1u>Q=msgT0&H6 zVn}?nS7_i-7yF+a`}iLYe{{sf&N3Qs&#al2{ z(8p`?nE`ug@it8`K3Lemt}F! zJikjRFv^8!S%BLa0?cYh#Bbi4K^KNqOa1rFi;8x|pPp5dk*Fc~DmJuI6(n4w*hr2K zvCxE_o_L{d=uOY`#%Fr{GlTvaNHI`aYBeYfOfvy=sdcb3-OvHa)GI30GA!(3RxhjI zPN8`O8{R3Pm-4Y(0rYx2F?s*?_|nZ=Gs8pU7k)tM5T%9HLM$30#;^&D$gZuQLHNt!}ovphb!NIH$E}GsK6QwuO-> zDH55uh#f)zvuP4HWk#62v@GHs4M!uAu^lFyr2JY3x4e1SyIAXorkOpA>m{-D=u4c# zQZRMZQ1Xik?i(-!q+*niyBQD%ew@Uwk%OSBS*>-NLe{mAi^+&x44NbtSKzVag>EseU1IU;g|Ji75AkB z{Br7qrR@Sqr?_rFBxwg-B`#^s6;?|d`e9d>FK();8<5Iyw3w(AAk-)*YUT^;N-Mkb zi<`w2o#Kj)d~sv8P|D1faB?d6`BjzGZG!xAaj7i7P|C>`X9-FLh1CV64cYnCDVas( zHT{=HwG)$zH*U_3O#;B4D0OS(0BlpCeJWQ!RqA%++TAe~y63;U0P<^HXL<t583e>z>OE+j7`&!*;>EtH;T0yP(UBbFVCS-_1V2%Axu0Ys1n@jb&G5dZC<$yfD=lgMiH?DG(Gt1{wHx% z*um`P4?b}??CkICALkp8>=VfeWtG$NYO@OK^NO29Ma=~{HQ6C)p-w@bw%%?wn7Zp} z>*Hnb@8cBY=Njq<^zw-GxAAiP(b1oN;&3Q|9>*0ieWN{X{cMBNBjSNxxtU3M%w!=u zPC$>&$%ui)PY-3r`y_{XB?S8xL9s#x{yuFX9XmNe&TlQBZtEu zI~>E;OtDBG@&+*8w!8lf#F{j{NBG zCqF*=3H*&tK_0=$5pnF4SWb$2binZdFZ-YX=de)E*jWGMX3Z>l%AIy3ciVovp5EXK)JK{URK^g583m{A1I+!xO_3=^{zJtnGAr-&d_& zXX_ezyZg@f_J7{oda`@q=Y6NX={@yLOV9ZVS$A{i+2)Qjpuoy&I!mj%;A#|>whBsH zY1vgmN$;PA%)064)7Y@QJ3Ie;c7A(iexFGMT2v+>MK5a~*y8KiMLZ$H4{^RF+7aQm zP*auh^a~Tm@wR21udz=r15vTm4GV?Z{){LisWlx`^T{HqlIG_cJnMQ23-lIBF7>)f zjd8u8#+2Gsxn@O;pG1J|bK(Cl7_JR6BBo*;IC6XlB*s~DxXZL zmJwrGD!2%4cFzdIUKlJmR*JH6PyjO0B$i%lcm}s)6kfz+Z^Tp5m!`Sb z+S%9YdF+RkV=oc*y`r&g?*0G%VKKY(Tq?gUBfp&^?h;m=$Svtk=Sk^;8d`Qm7Ko&p zo=Ol;eP2st163_&O7Ntr2gI4Ux?kG*WnRUJ{K`}GoW@j!B)71MDX8TYHRTpHRn_(j z3uSykrKGx}xU!=}(q3B8UR2szRMsJ`>MoFUmcciawA41ADX;AZ#g-$g&k@NAN}Ds; z#c6b5TBaadPzLBKEpO!INmvN5GG?|oUtE_Zs7OrDlSq3mk7&jx=da(ej8ENx_bSR_ z-KJ9aOs)ZdJp+JEskg_J+gGkFU$}fnt=rKWccxTNClt@d<(t5>3FU@T`xIV5HQZIm zxhCc$V2&a=4>OXxjrJ+%Gb}2Ssyj{{!dn(BtQf0JJAkW6{q6+(U7BsReiu8>OuH)c z3;oP%?5rSl`|0_Y7$$Je}7-ioeItKeVhxodN`#J=>+4wkq?0obmwm-A; za!6sPBr*~l{T+i6L(>F|Np@l~ee`Isy!(&H(bokIu;JMuq*C5Z3qzKS(?vcKZq23O` zeomo5Ua>JjY49gxMzWZ(S)52VGc|Q|h%z6(&d#|KYct}lvJ=$qd+&{Og7OE2+kjA}q8P0eKCbmqScOSHytkYg=G8pVFnLTBxt!!+1^ zK}LDi3?zRDZ+1phVCd9<%N2`2TB!=Pn2U3^|2hDt5+17Rbtx4%{v)pPg;dYSc8E& z_8^veRA$yw5Cj-{H}0T`1D@;VUtwtd!W*0r1{gzw;%nW)OZ_Y`Y)?;;s7&M}jCe<3 z*N;tLS`(gq;v$YX>VyV;PX_A&7>Y|ejo0A%6ZId0DviGU#n){ zT^?UAYx*@^*ppL!Dz{>=r2f;w>QjuohC=B;N!_5JtgW!Rr=X&?)I;tZM$OsP>FVdWI|PNTHWy=hw6H>exaVJ-ec)tgWoFBTpn1OByR`x{FGh zip!cx%38AXYeAn0N}BnF^-O+6UP)7KVLiV<20Onzaig%LiIY<lIZsf*%`Qz$%`L0!ymD1DfdHGHfH#JAMS-$-6X-Rm-cV?;PB^Xr zv9)z+_`$WCPjto?O3mhkVuQ>G1O)~+iZ!K{aEq94|9|wdJ|Q8_Sj@m+Qp_eb!6WV= zUTE;pYkSJLGp^l+r8Mm7O?x`?p3d@8Z+WRRW9NfzdV6|)4}%5nzsC&U1?;Z?dZ}k$ zp}pR)H)Y%ddQF;kq9QU(7=4E5!b^NG@t#5Q-GAGS9GQi%;IgQ82 zE)!-JaA*ZgW@!#nlAA8hiWg-1rAPQAg#xak*eRhIagp@IcurbE7A>BO=t|FL#c@)? zfo*ZYj=ruo?oP+tT#vcA9d&a)>gi+`=;jjb>yZ%PpB@sN85WWm7R`x|<0kpU2RnuO z2B)S(vzQUg%qUJ~42K@hNCV*&nvo2TkxY6RJtKsc9?8gzVy64WgxGpG9CJN>*y-4h zj(qeFhd=ts(L+Bza^y#_4j=mP_z|~YugKIGzi8m1mve-lb6Ai|cxXU!au}TvPR|UZ zXNEJG5iAzKEQ-ZU%3(8#c&U6wJR>!cNf%YrUb?27xw8cTGn1Ma08DQ^)WJ#@b=x%% zp25Yr|4xF18D;R6!kdL@Lus)zdOLbjGL+e0cMFWj3^a+Z~ zdI<58{5zIo%-d8)8Cvu|Le|f5Mf`O|NNi3`oHYx{|bkvbe*ei>;-_qs+4tC z$ok>J27qO9%ei^7-~LIdH9vcA+>%p{ICP_tRu#(*X|1ycyi)fyY;!SS z!GucL(CeS-44AlqwM5vt>o!1psWouZUsGbd_p(yGtU(JRRX;>mx>gG?dxB1Sazn?N z?rMM;0&IL@@zzbl@X+;(7eGyoUb!+lGBQG)gyFACLsy1?UROszhFu!IeCg5;KVWm% zeti89{2n|ZBSpnw;^V*w;Gv5k-@gC;#^qt{b^*f`L!?LHFM!`pX ziMx99?m-7GF1^(uiS5rQ!S~FhOIkMvA`I?^Z#4@TORrvhr<{L(OZTGnAUj9G<&~z< z@@iy#!y~GRiG{ISvvTEqxprBugUf$oO1(L$-kicx;qwXQ^Q*U3FAm>TXtuP*?MWrl z>(<2D*c8rEfz2)f7;fe$fKkEC`=^^^Q-Y&(@rQ|HBH+>^+eE!(S8d!;8eUB4w7J`hyT)nqT2QSHnAt(>9!sz1?qRYNyy)uLJ@w3j~!@ubOWB31}2Y^L0V{=)RSL5z@>7J&a8`m>Vvu)YuzzY~1T!&)of^wdP2y#y@ac&>T85C5S-=H=MKR*S z(;_`Wz3n_*D1aUFaJBJvwefRti12oc5AaS64$J_0#l>c&gk{Egf>eqM38ke)F&V(B zC^j97dmy$l(&3XK>1h#+%xD%XlF5i-(jysZfr;U+K^}Hq_O>4OHtzQJ-YyP49!H#< zKC-p_@exp9M?FG)!jq%CBm7)Ky`95-T_b|M<6=V6(t=Xc0#i~#Xc-Yqb|i}(PG>~3 zS;@IvR&j2oFe{0hnV!Wv`T4i{ndO}#3nG{7AUZh8;`EtTD~>80stGjW&#vd*0j|( z_ZJq|6_+(OG!J(7eJ*S4tCY6GqpanGq^_s5x{EKA3X5gJ5}Bm7vqak2*zsvs|5sQN{DyGjqEbA4SqPW{7Ei?(ce#{~lm73$N$UtcW8h3D1yll>DLq zM*YN)_xoW1l=j*`4x#qoc*$YD4*Juq^(CDNaJ32Of=8|4sn)cqH*V?m#M6O?VH+Cl zxk)WT09#=fKg6AjRv3KBEn0F6~%{g<%20;*wme|+vbrgw{U_A0oau*BPhN0 z#TEsyt3y{ohg|_NMt%)r$nQlGVvkMG*awXk#G$L!fzB7MU@7wYe||T1>((47u&If~ ziSfnp@jF1TsmVn-#<>y&hK7#&)BzX;Fr5wqh$+t@NHH8LZCKUnS8*teVO3{BH3mzo zTL9}o6u-uKF*tP^o-Bh7!^SeSCR&fy$e)IPphEEwl_&rVJHsBUwW}&Ej=ogtHYhtH z+_|)9m_}S#`;jql;f0puYoM`k7I=m|L#Ve73>`Rjk~r^4;*@#eop~NR$*9sIj=P-4 zLlit+ny`GG9CK`)$h7lJcI~p@};M9%KOCip9(4lg_S3YYfk2h+jzo8cuR{UT?M6`HBINF z^`~-0^}^CtVOe`=?TJ#^+049FerX>ow}F#i2LNLW>v$rWNYW}Os>vy+D6fVqZctp- zlwVjaD6C=UlnYCmg(WTdB`xfnsw{z21RoaF=ZhQi3hOKCdIZJw1(K$`;@VtMRd#M! zfw;D!wnJD{&B`j|0>-k$S^QE4yRfctVCahS=8fsww`QgQV44*&$rdjEr{l^EVA!N` zYfS#^(#V71t9MB*;?oJmIvEhSa%=MOgmOix-yptuYc)b=+M;G~6M!MoD6nC_#59Iz zPtReaDpoS4cU6X+DecaFcLH|4tK`=BJ9Oqz%@IESz^GXJq@9k z%Ca|U+?g-}`F19CyTCAoc~3R-QU&jmxtChp_mJ0p@hyBmnF{unn$8T03K@nDj<;ll z7y#_u!9kb#A*{clj{`*5qyIA8|Em!gb{~700btSLjL=}Ff2<&qTf^kla^kbX0~6c= z6FuERo$S4B?Y$gqJnZd#Ts=d4!(zi|jHE2>Y6K$#a1xi570qUbWzr()^ze*~P#P^HBON}BX0u~BoG1n(GBZ6e zG0ZjC!!^(?I6gFmSAFxdp*;*pZ z$f@-+t$tId+t3?G_@NfoQ>A7NtqST@6lBCkuSETUcu*=)h^UZZkI=%7kHD~pQ&V>* z$7imNPNETU7_=8w5r^>zY?QQxp#^bdcywg=+Lhs}ls+TTQkSn_wGh@Y+75@|o3Ft0 ztN59rt9UXt{KJKj-~aL2=!kK0d|_&00U0*2B*(TeOhHtt?jdzhFj4VQc=V;!f~Z>4 z>Q}IY2m*`D;ne6>)H*x{)1nA_syA(*0>g^pn!&toG_M&=$S}R(F`kW)1W3xwXg&X< z7GPU(U#WVi)GjM@t8!wjrwnY?4Y&mZpN!N9#MwO!{QPHjF=B({Yv?d^$|}5w9g&h* zsK88ka`qbJnl%o19%o~ckVbB|)?uD|W1f43whjyGY|5%b(TfxkaYKXe z1O%A1hYSQ9U=jy|EszYoh4)HKF8|BF{L!4mZB61fu?xG?a_X}{eo6W>1Wml+PM)Ya zjU~w|fooxaFKj3%ZLMiMlP7L2tL-nW>di0jku-gtFFnIA9f-*&;R@?>%UXpM-Gb6q z09am8U9PCQsIbooJy0Qf2G*&^rxSE$!!eHewIR)vATp(FuDwmyA^t<0*yMEnt^R{J5 z`9Q8-m21}I`v99zJ)2NIyE(Q#H1hD)#4_AK$dvZA+mq|JCZF7%dJNyZZ#%?-1Og0X zwBC9chMo@H2E{Pg?ZF<=j1#fR3^p^+3u8Xy05J8|q!uk3O4GK+4Eh^qujt79LfB!< zEa9l5nLBSaWJnljEX~X-h3UnVVOMV48PjY{>R+g)Uu)-Hsps}|Ai3aXxcEjlkGU;4 z4D-%A?ZRuKx88upq8|14(O8HB!0u!5kQ{XV1->->h^ido85n>KhlJhxv-a+LP-7p& zabrUn!GUz2q=K02IyS91IVjcBC)&X?(!n*c^$#AHUof+xw@$*Fmn zk(|VEW>OTF7MIP8C6G>_P#vASYeGrU_Un*|a!zLPSQiXPA$@my5lpyPdbE zt%utYSI1-C&Y*|95<>!M5mD@fXm)aVW^!0&dU$3=6oZa%Vlv{gI0@NVk*KsXBQi6C zQd5G`Qp4$)vD_?JAko5Usey?R?qNP5iDC3S22;q2Vedn><&>iqa7(FsszgWZ1}HEkcGr_LFyz^~+PJF4=H+FLaapB* z0t+;m%CIr1T)REBb_37AmWQuCzHVRc*2`I3sJYFU@GrlVHY)7pMo z*3jSBe4?rKR7=-+P+Cs{^Z2`&FjkV|8(inrQwm0t0N<$q%??nYna3vUb;AhpTBaI5N-qw z^_PbLVArmUTp@(JGBk?kXO}^{!GGX8uArH5X!Lh~7`=2!4I*rEVgXY*FMOJt{Dr79MdLrw%}6B2`4=I;LltpTdWF6FgrxO>e+5nt)*| z2IGnmN1&~tZZoV?Rz?lxK(3G~A{rgBx~N$OovI`;i5jwJ$FnFbF<}V-eTh`tI2L$e zA;7GO^`x{&JyK_2)TSE)JN9F!FcFcY>KPD(Aj0+sH{%E~>@%ANfB} zK^m2)aw7F|lTJabG*Si=<1A*PYvHyZcD^iISSKp$VDhDc(zc@NL1ER2vZk-{rRTFtPo{8cn1VV< z!>P)qbApmK*w2YdoAX4qBC)K#^>k%jZ*IP{sH6d?CaLL_wVjjH_LtZ66%;o~s=Lc; zI^nUftgW`?1YcCm$tf$Tgln)VGpi_tA;>GLW@i;<&~xFD$rUlO3X^HsNol-){rB&$ zUN_zzpTjqy8r`9wz@}7KLIi+Ks9Fwo!o?N{#wlX%g0`vlODJk1M zffmK81AtKsqu6G$V3C9Br^L=b-E&Ow&}>htp3Ak{Q<^O~?5&N~)CJO+w{4i&wJg4# z1$x~DC5qA%vL} z2Y3AZoOnJrF^A1A$rsgBc)hwCvnr{mr( zu3;ecLj01$A{dELOh6YcoDS$>#j?3^98L^}9na$?=knn(mcxo+u_73BkV^5q?BrZQ zQciASHZPi)5t0%coD!Luoxv7y;utYr5#Ek|F1GG=hn;K=I~_mlY=79r{;;#%aZeZL zAV1geU{7?98Uh?6@o*5|PD>W-}6Vxhc84G=9#%|MueC(za#p8CKE= zz^K^})+>!9ND=Gmf75H9VNWp|1{p@Zx1Iv7NczSy9)p=ymHPn0AVkH+^|95_n=8ZD z9$&ip@XC$HHz#46ymH}+y1JoHQr*_obH1joyRN>kz4Kga$60A@cXdO5d+(PG?Vp0) z$`v<=E4#$y9ioyZX>D&?*B7nbUv~6-Ro8H`rSr3{fnT)reBN{7>z1xFjqN94lPIa_ zgbgDAtfZ=kU(k@5Rhi0?{OVsX>#$Sp*$fJ=mkUdK^Gka(3oocPuNhK8oL_o9xA=07 zI5@26y88+zbzv$8o{AAWA}(UA$I{~8(~HDJS#65b3%lm|9bg#Rd`UYq?#d|(A{I38 zUWV00(r-(Q0OY`If8T%@gUzFH(||?Ar&`cq)^nXrwRTMfOSh)c!F_yPqrq-4>rs#b ztB7h)A*jGG+gA<@n_8S0GhDg|umMnABiajll&^pgLt8v1a*)!Zbq(X^hlv6sp2MLL z6lcTN;Cp}g{s;K}5!7(-9i!iU2W$MsjTxX920kh8X)yorfev#tupgN)j0)g*qSqn7 z^afC1Ag5LUMMfjQX#*Zjrl+vr$@KIyv+1eXylI5bV6^5_%j_mRn#~)qrl+4;X0|M| z&&|`%Or~c>bjM|3=hCFUYhMOwiEd zDbzlkNSTpJjiSO54}6K}>A)EruW&O?j{S{Tek4|o`8_-m!&zc)uqtS=zR&xTG54LR$5P0 z)%TZHb(d7O0n7?Y>ctf;lG@IKvWCL)#-g%%qQLUu5dg;I6{Te6L?tlKef}?_SM`(f zr77hj1;{TAo`Des_7nl8+#0>{_~P!x@%H(0whzH;OCqY1^bQo9Zt4wW7-h9Zmt z*ugpg7}bqnoWc8&ab}m)7~zk_;-s9kLd(_9NP7=PjpHQ-KW+0Co>ecso?d!oT70FR z-cgxhf$G`;dV%U1N5zJ*^f*z)0{a!5AqQ`U{b2?8A+?Et_W(K%@8Ic|dV#{&TdMtw zfa)!k;R~OEMV*kT7v4}2_1dLBYw!G71xoJz`w!eAU0tG_T;m-B=sr=ruuxV|Bs(~s z8y?9F384l1CkFb)2fK!P9dqD-v%Qz|aSvBpFLzr{H(L)ETMuU^e-E$7z|i!# zcpfv3!-}AzjUqmam(0&i&dEv6%}EjDq~ztM3Ubo~f;2&1dVT?|K*TIAfki7QNEZq+ zg#1h)mtDlqDKB7&xM9hm?jfEo0dCHIu8#26xjP?mcKDgS%|~`N=zer}J?8D@91`Lh z7U3EZyT3K&v``Nb6^VJOlRrP)KEoY_mCyFaO^Tdt$ z#m%6=WX&hrdOz>%|3%lpF9*;3M%H+$wd;%e)^qKBU$%6g2Y^ZIy9-L{s_J_y(Yw(p zEN$cD*0TB4Wz~b1ubP2g=9w+3atI3;MjDy-3x-yQAvzKXqR`VZi;Z!_nt10WVho-1 zud&5->CFMa;E__UGfS_`AnE3I3BX>!b65mvuc#4xW<2@WBh!0vwjn7w{&x@T!Lr6q znxE?pn(yJFcOrA3Jedu08>{+M-X7}CD3D+NFNy0G&X$k z!nNQ3{>HWI(~2oFe^Pl@qj{jm94YMmqO9;*-6I3w%81?#y&kx=syD5Yanb}}unY(= zM3))UvnZ08OdIBDVA#{?na$~$XMnD0%hQ?J=Q!VIZre1yWx$3xaykfN3;(fCG&<0u zD@qOS<$+!*kZ2eqYrP?234z4IQgK97T^HZy7L0VFbQnc1{UQR4l3!HK5YNL%@3R)O z5MN@`7&WB@A72>eF`U;rcm)j|2h+l=ofX!5 zwOdb=+Ghx`iM1QIAK#vMJf*^JbZXC`)UT=Zm<{&#^s-v&NuLR2D1)CWl{E!QZe0FU zPGYF!IQ`PvE=Fw-sdBvqhi(IeUT9`^<;HD=@x`QeXI%Rn)|hq+NG3P#D5keH7!`)) zTzF;8fFi1EaDQ`yho%cSpc$|Pa65F)KbJ zKRq*_mQ})G76N9Hf@#rysZl}nq=+m=1e+DZ<pH(>Jg!%<8JYLxQgFzMH%A zmI4@+F%M5DfLRt$e$8UoJa$7PHYj}zx z%tUM*a7p74H|11Z!^0#>)4+2bR@F!*Jti}$F%O z6e_SgD)l`8m`?LhhYt8hdczY~fHA%PiNT1^fnK;wfG$|;21FOq3(y6NVi-J>J%FbQ7N$l_PmcYYylJuj^47t797 zw0&Dp_XWSKKapOVU)CwDKLv6vPuwi6A8cqoDXrQ{9ozQ*u6RgQ4pq9RI`bA~@x^vY<8#WOKc#wx@rkO92?gwx zH}NGlJ=d6?sj(4o2X^FedCuMeCc#UL^g?cFw#U^-uL=E@%Cf6~ZTc+8uU*Xyz69DC z+?p5{P-Gckloi{!AA1$mhwQ4*gU%}I%5Cxy}DykY|EeO>H*oB&{U z9*#$ykK@sooAYr`S4STYXFo3|Ur(Qi&{#%B6g@LIEj>Jw9s^2<#ffIJBN)s`COewV ziDq#l8JzG;RvZ^}SW!knadxG&w57GYt-Z3XwXVCRxv#CfNy;zJrx)_#S*gMC;of0> zuKpfQ9?tdvFh{!|A3gd%4;}i4BS(K~WBcP{$9{bD*iUS2e{{?S)=zD1KX!8Th>VD2 zFacfBEPfI%H&Y-ik+ogEu9>^DJ-e_qGrI}W1g|<&279EIWzEeb{kBLi(y(l$mqNQi z7GjKi@+om4q5{)RuV_sxS|fIY=}c=H0Z^= z@%u?>ZEsa|XI*1|ON+cA-3@JLORKshvVLKC2Q#}W zudwxZ-^qbqv-3MBycTyC$vS}9?BdQ0+7Z!}JF~P${D+uKPX?*%%`9SbH-#|^mKO1P zpy);A%~K1XFJcCjWpS5sCDWt*;#nic5!bphfP0ZumK3uGFol z4D09((XMDo36Y92(%_aH^UrX06d4AD?nKj$hAeCpqwqsXDs!)$mmPzL&GfQ(_`Bil8KOvI`ksPoG1Z}XynmjloXkHZ`v`;keegM`@YoA zzf;Yk3sJT3m)~8vC#v~|BR>nW~1U+MBG@i~b?J27NEKhotFBwc?Rq%xMxrOyv`PCdjm9V&>q^t!ti?AxH z+N-L&WQ~IoX*aN~QZ^_mYvbffdHJ<0enpO`PAG1oF$>c&1*KIjqO!&;L1jTnQ<0=4 zB~!r3DHDjKOs?a7r-stdz5(j+a>r2`M;ZZsJLWi*|IK)|l z0DA*6ixOc}6;XEwD~i+^nBo3kHFwbKp}YTHfA78N;om;+i}!H~vG-2$h|CI46ecAX zCMOAEQ*+`Ic?ogcMBmt8J3lvvKzGMLH>V&s2Y;92zV`OMb`JgyHa^Ecvis_7N{QvsVmK-85q>rvj>o;6?7f`rJz=@pd15!1t%r+)m%F`}8|X0i;DE4{ zqzGDi0Oe?>Gs81jq3O)93|17A7sbp9%V3A3v%@ktv0On~o|suA5y={*on5ltzUG0x zrvCQEo)&R!DMOT%E@a1X(xYh!!Es^kL0&GP@Vwk@TwMTH|Krf1A00XNA&zLV{pk4d zkB-~@%-;U6v-1%b=TDsMZM{8x5qY z7{0M~`P%A*(d7$Q&`dv}TK?@HZ&ygW;aZYZwuws{$|~9_q+Nxjc#)J;wl#K~ZSVWC z0#B7&@`@UAMb*;!UTH&rP2-9FQ~%gA_)TZu*IoU;=<5GvThAAbZKq^SeUhry@|x~4 zX_u(1HLtXlFK*`K*0lG2JE44FvY`B0SbDLrw7amlvw)G02LRihU3_7g+p&htFTAqc zd1YC`?0G;Jd})q4EyJc}tQk@xQ7DET&})tq6|s|SVV5xM1&T21!5A9usRi7!EW9R9 z@T-Jj)SO?dZ_PNfLrt)xhFHRV8Knln7(6kUo&mtD^g@hLWi%8YS_3-zw0d0dgcQ~= zWlE_4hAoWU)?K(+0j^b)PPm2AHG5} zB(~GORL#Co%)OhMc{{4yYCiFY-11M^g}nf>sSlM0Q{Y72pSH<<8it0WEfN_g^Q@GXaJefe;$j+(E;1uVIY6^<#a`P$$ zLTPzbdvR%FiKHd3upZNX%RBN*S}1@qv&(^ba1rtIYx%;ed~t1der0Z9ZC+8GKwO)~ z5@zL<7ne7Y=_)03wvajuOQL0`F$(_mKQ2!w@7)|ffUN<*#ud*l4&5CZ zy$5P(Oulk+;_;2Khm(pYDjXyJ81`Q9zEx{+Y4oc~?1ey#X;5JU#@0b?Q{r2rf2P!K z$<kyzvm&5KMDCZ&i{<9Ts` ziT<8Z-d@rEj)5L_{+~rTe0=E0&k_=%=&a zo)*QS#qcuYvNAj)gO0d5*t$F0x;q_pu|MwN?BMNg@9hp4LzsEE*m*iT`+ItY1^dOs zgr=oOvw%`r5p+%{jRo8a&jjJc4W+R|XjxIL+<2ZaJztzvCM{`fk@fVp44!Bm=&x;W zlyx+e$tt+TIW!SBDTkiGN{vpB3y2B!3G#FI^Kta?I^^W^qoYTEeAMP=b`BpMxBtZ6 z@raW%eDa8^3*haj3p{tQ_4W3Sjfqn0ZeUNR_a&fI1g;&_SDmno2!?u zu3Q*-a`DQep{w`D6i@!}r^$+{E|IvdsI);?SXWfiP$FrSRCS6a&1KT|hW694wo`>= z&3U5w0&!zyT~AZzS!v@yZPUrdwsZ9@XF7U+-qQK`>GS{EJ@9pB&-t4A?#k+p`leIm zHGQbqMD^*MGA3X8Pyg4jetHx3?-a4-7PjYsUW?mAknI7|<`7^QC1qKBX_?1SlV@q>i6H@nwnTp%I~R_NG}~Kut$2`W33ivSL#rWEmI=QpvT~OVAy^M zah+lqRW_s!!1gh=X_?uaodZqw)QIK8E%P*%xUoar3NTC;B|!RPRjBE}Xwx8tN^2tu$y*`x!bBU40LD1PT_B|z_WLvF5Wzf%d5nvd z&%BlAJ zHSOoy2fu0R`=+q=bV1EoLDkvZiqnkz78*~=&Xt0?V&_!ki|a}%ngqfsz?iIYpt0p- zRZUk#O}DtBqpIPAq^>W&q^VLiAg<`Dl=as%pAr-`6qGf}T2B_1Gzg38;I~U^Iy15g zlQVNMO$rnkPt0T&&@yru?EJW7Ha)BO-~RWAT>bdg#QjO-@`QY4La{ckSRGevj7@D` z8oqb!*5e5!`sT0Rx_4{hp+fycrFo>$J)w48NG~0RJ5%(c6c|V}cqxq*G#?tw+e+;- zg~mF21=FV#SUgl}HdWeZDlG<0!QIJd*`3BNJ$wa~rkyF>j+`1rrrsJ;Z_05n^fr7R zB`+1uLQFbXLL_HhYPbNgGK|?^*etOC`s@{EIul=yj&v~}IOz@d-cvb;r1$Hu(~tgU ze)u>QZAsK#Xe)L}-xQ5u+`W^C!b&buANy<-(<;4dj2V?(ofU84* zn?r!Ry`RT%KNlNcN4o$grw~`ya1WrDYp6S@u@7xNIbw4pG%PSCDKv>4%PyqT^JqzE z+(<8w=Hympr{&WUax-FcGAMLK^BJ+ZtY|(ngh>l#q=z!nqq+1LJQB-@W@UJU2Y+m5 zbI9TNF;_=ehn?(f+?*V|-JE>f9Xws^+#L~Me(p|wKs67ykdT1HB>&`;faKJ`l#H-U z5MIoXbOtaioSq%a%}dQIq8FCtRMeEWv^Nh9wx2v%-`Q0rtFCOUmeg0~Rur&Gb25e8 zR9;2`GbJ=JG9WzACpf?(*x%066C~G9j@g0&1Au*O=Wy7`<*2LMF*i3GcUK!%S76v- zSC`{HKK`+ZiOlSd6JIEe_h;{JTNa;#0<+BH7<=P>q&l?;g*}KG&kU&SXX`N-g)Wd^ z6uqqWdc&p$>s<&if-a1bQW_ou!&HV#vG}%)N=QSTV5Aiwa45QT80l)~ND7yz57pLc~3XJ;Ffo>zN zMJy*$za4(d^djuyUs739W>8~zWJ2Ae@uUYg78p&r1)7YU{a|pH&b*-ojG1s8wic8J ziVr;UA*Wwxug8I5xR%w*$BM~26Js-1N2e}h9xrtUb|AuVQML}MemIKlUI;KMumg*Y zl;@t59jUNH;$kE<$l+_>|LMlo1L%c=LkZw1or-|R7;#p68PVXqmzDE$W0!Pwrg%{jDS& z*a|tUK;lUl4z}N=Ebdck&7gnF-AY$WFlX3fZ8n z^UKo4PxGaxK!N2{oJ!8B%gCw9QO1 z-Hl$|gE8&SC1S%_b~u-b1f=y-!%p2JRM0oS$t?a#1v+cVtP zH##UFE;JyP9uNa7*u!s^A3Y%P$o?a#2eO6fY!#EO%I6ky)phcc1|_eEogqp;CCEI= zPD@~<9LUc&$jmsvOpnh^iOWuj%{-Zyn|_p?o0y#zpOJDnJ2N~XZo7}Cho6@x9neJp zM#n-TEh;1^DwGx(Lf;!6v^OL$D#(i-u-(tkBZ%%36CH6ZIVL4DA%mHinGYf?A%l^a z#X82~XNVL`l}1)oTYmn0{aaV+uUxKZY*JTM$V&@FMH0SN!YLBwirKK;IhCKDcoF~> z6B)NRJTAsBG;EuX-K5)fVMLz*fS(_cSxuwCW9cuyhB6%A|itJ?mKiU z``>=|_2})VLz7R2fL`NI21YSGo!m56NY^KhizNySPr@kU)+WHHwFxkM9;2il!Tl5u zy$0^u`mvm7>6-%uMgh!dzu#rNd#h{l>(+%YZq9viWA<9}OuKFt09IakNn6^a($=a= zPgmAmDK0%Hlh=r(6-A|IR3(jkSw$YFhymLQL9wR%eEr!E;A>WQ_D7{Pmuec{z3|r0 znsDydrRv7F8!mjHtvX*;ceSwethDenPg2E~*5$CYSKt44r|Z6JY-N}L4D=R}Um&Vh zN#hp)>=~6tPj!6_jl(m~M<-s4jIE81A=q&K@;HL$uLPRhSZ@A-34&m7kyi>hH51pv2Hg6JWKUrfB967BR9Z6?Lq%qBGT=hj^@s$E~6GL2`&e8H!4M49bC300wX~lVX zDk;C!_BIrdM4pl+~1!n8LYjNB$Sr>tYa7{ksupS%jXvgm|KIk&s z2Y~e$m*GD_(DlG#$A(n+Hz%!r*a+kpcxG}u)jOVbTOV~=A9UFs^*A1zfod)kSbf;i zjzvN%?9qo-Y{M&#u~o;|8V380uk}sh2m%+zO}&6cVXS|0ePH?(riNPecMO6n5l>s7jQKw?B1?=6uGsI(8gcogpm`^lq2i**63=o?1!3J!qf9qjKHP78>jgASwb4F!M&?+p(|Lr0W< zbad#UBm2^_;?r_tQ*z?c83(fRkK}QZn7nkMB2Q6Npe-$GY^u3@x#rTP%CqMx8XMJB zWs+hAPbDZ&@OWxbzO*2dmz%=LJd}Dg=3rcSY!odrY`d@D-)!IU!yTSK-0kzakN=JU z`VLyi&ft(;A@B%NEZ8R`6jT~*&z^|5gR0U7lXH4x>WOO{6xhn(*pvQ|$9=<(NfQ#D z4FJ6?*edZ1zt6Y;V3^@cMfZNsh=|u%qUnF=KC>BkVC!E-_k91Nxpy90!TPa&ZLr?$ zG~T(@4FH?_^xE|2*QUR`G1ab{`_Dh=%c?GymY>mmt(lvG_%mozEhi=?cPsA?6O z(*i{`M_j=XmGdQ)bxj|<{r*3nzx2^X432v5{FT2ud-21w7yquR{%YOX_v+98sI>kv zFbou%SXnPp)H66GEPmx5J~IxCJQ*4RfUORrX&xol2!d)Ao*5lq9UFT#GWra2q^Ruq z@ilA(8-I@W`U!WQ@21ki2t;&h5D)$LM3@VUhi;f5y{5k*HW2c&z%5|g)H;=7K1hm+ zh&0kY<}MzNqVEG3Mhf@vg7r$~F z6>LTrHu!|17ezJFB!pHMavt^=|0!@#I1WW5lUD9qNG!S;Zg!4`Cffs(9gv34;4u5% z$5zl}15e@6feMUd*n{>$XW$w-&u})l+Y=jzFw$Fz5l58f!-^#U47C>yRT;)!X2a0BY2@2B+iFqWzh?<9W(XRO z=M}T%XHxP@QrKEDSiHPIUMEyHvE?-d(keidRNYus`*wBXhb0X^R@QtVtNK7x_I|FU z@ld)nhg+P-)w1{{`MhHIPOf zPyX>^vtzlfb6RIyGTH!O%N_dVF2jSCjys=zG1{fO)1_Z%@4DTspEcVSkyQ?o#^PAC zIhL^EZ6mRtwnbdlMbKonuTy`g$9Pw7UIx`=vfeXV?pi^6 z;Sny_8Yk-% zJ0t$o!H~qrz}R4)Jv86FAiP2X_l42o!e}v}-XXr;K|WCldzcc2tV${>69`NBf)XCT zm|JwZP+g}0hMnZ69nL>-An#;+=JEZRC*m`bVMV7Mjmu6xn3o!#aV#eFa6<;q(+nu`rU~hPOy|LT(O`m`r{=vHdU_l`$!b0J% z9O@q);TswX7~7L@^cTPWYGnF}YvQqMd}V0-+2Hs}|0r(oG1G_`f^ja6>lwb&q$iBB zZ&7O#U7Lao_O=dc+v|Gl97KSj@ETY!_by;7m~+`|zhkg2cbb-O^(=nfHuu@J>Ce8N z`uy5tYZm~_s4Y9MDX!NR*B5E)t7@-+%#y0>VKXRF)vyI6OnwPVTqaP|lvZ86{O&(p zdFTILyzoT2Rz}BG&`D3tT*fGRci#LEDJqf{W=!=(>P!&L zL|BE#T^k)>n}xw4axg{!_G*|2GF%fc1}BkWz$!cp!{|J$K*De0a$7tw!+y|RZ-n2$ z-A;s0j-rndmJ0z!+MHdaCD!%O;lj!x&cGZ%vm)`@kOVm=PIIR6e{nhPm~C?=^Q=xk z-rX~B^CpHO0=oc3ST-b*i!4MJW^IsbY*j|v;k6q#uA$Vzj%WP88~DFlz&x}mlApo` z=j-jCf7aI3z54~(=kg9yV#x&Pe_0KNABT_Cgi$?@L6 z5I*(+%v{K@0qh35-#_rkHS}x%r>P;j`pJyC&C@SRf%TD7F|sg`%Z|;4gnIv$B(ggCq(+uUPxsAk(|7;QDYY8`oD9$vSMe1iZReq|m7 zhP~|RfBDu=+j2#3?icj_kC&^fFUa%~PTRD^S#MWi@P^Y|bsrR;`I)Th!#vr!!y{foTrEL<( ztKnOw)}EEA8`Y&3#f5c+XNfkWxYbvL|G8|WD;0UnsX`{DdqZsD=;<(G1?`)IbycVCb&v&oyo4(t(`g!gQ3h)dH+~M!-9pZN+ zJt>QybBdLAIP3WSlmj6NVLnj-K2d?bduaaA!L+!D;J8TN&;Z{Mzr^H3ZefA6Tr4UR z3QGkXHH)j^Xd2b!=ZY28QkI&PBV%NW7)i{ueWwoZO-_tFzJJfDgYj9(2Xa&5(v$Wk zAJ}{JKu~n#c3&@0V4ifJfT$pPOej4%#6Obe7XdH}@{6STM+F5%heqt*6MZBu{^Zfb z^t9NN%snTwVpDVC)AIIbG56&#xL8wPTXyEOx~fuKq>*S~ z6>(J(mXynp@z^qMzN{ci#7JhQB_tmVkK5xHPWSNlCUaH1ws`w&^$FPKOY`uhdj^Ji z(P4$4_6iI42?^WbAF$QSH;X0iHcyOBKXZ*e85~_9&9l)5~c=pOqYMb7^^v+MK8!n%}{9%35+YM*lX*~O*rgIbH1>=QB+hT zR8$ESwK;4}9;f(MziaIuel#%p1kbU^lrPE|Pm(y+QF4tv8^%m}0xhDqP<4^Yp_^wL zi>e}`7h~!rimeUrhFfw`QNJ6{43Yfb%@X3k1fFsMz=)OOCCWAO0vGXdAfLzu>>pit zVD<|0^srqF{gA||_zdT^j6EYIM7(7?T@R3914MrHKX76p5!=ZzEDlc*?%^Jm)Mwx> zfbqVTd4plHM?Z9{S^wFmUlZpc9)n%SQOl$c>}E6Y==#?;h?D--O*|)S#jxLN*QxN| zYuBz}@eqkc+Mm%(4*LdW0q#jM?NvtT6j%FN#Vwpvl*_Z70JD4EV zdzXqvq(m6T9%2qX36vTDfPv~lj3Lk9!UYoT5zvbQm~#-1zBbY$@%sBeq@sqfzU{G^YsI1r4yeBHTqN@9lU;J(c|8zo%^jL;iq^egIpF5ty2aQ!y zabB*hmaD3?rHw*qd1=M@GZ#KmY0s8cT&}En2Pmj4J;zs6Nj0?!O}$cGFO^p*iy9=# z8lk*GEU#d4pwye76bY z*S&7zqc5&aeRX}Zy=$($Yr4DVHsA_GIrXWq*7F*4dRY^+EY71+n(h8BJTrALv(+jdg_D6;! zMnxWrk4-g#nZ=^9{IC>{=@ifG>*GQpStk$ zWex2EBV_V2R^%xQ;!`^o?%nj-Ko_-VbWi2kT(-W4==&O?JYWxrx!|r1(Y53TSTAOJ z>MVEL^vgFo76D)%e=+s(XA_@apZNGQTUphW!lIhWil*X{`pTM1H4SgqpM76jah@-& zfG;GVi@_~s@`|-p7wgWwS5kGZ`t;Sx`pcKz`bl~9`Kr20b&YQom!Hv=H&r%V)>fZW zm7OUpJI9w*XS3Am(o1dKBZH$<$jHhNnUjKxBp*^0Lbv&F3<p5==nHnWRvT!qC6k$oMx4do zQvFD4o8`+dZeAzZQIrJ{55v%mffyqe#I}}OEiKLM&9~Zb-fY1pX3{i<{m;#aHuz;A z$F6<#CDCYDMQm+p|Kf8{ZziLWqSut!f?>X}D0%_LK!*`i;zlAOX3}Hxbsy;mAPK*B z?bynSaoV$e=E-D1pO2!_-eGMMXwg*<{ zV;ram0JBq}d~i)QGNK5hPQ`4!gj)bGC&Fyg(*fK9fLYKhi0(u*Rg)7hcQ@G33o<5g zy?5jV8h7EvBWtFiXQlz-DUxxHyD^fK9`w!;CJt zIeufphOlq1=^jYS{^1m>{#1Tt4!>Taxtd*2!Im^|WDQ3%#TlF;fvR4jZd8_>SK#c{ z8g=<4W!Y6}>08R04-_>Y3AFE@WR@jmi1S5d0%cu3uT-q81!yr?iUMAtP+Y?1s^zL0 zRZ&A8M=ep*DT~gCl=U)glSEywDs3z(KLbB2l~;;o6l$?DiDdX6Qybpi!TZ3((wQITy=w)2!0e~6rcj_Pf>B}(ySZl{jXZN&D zKW(+lIUG1E$KhDQ4~z{yE%O@#QRgrS4R$|p1L?nO>BYupGE;D=t!o|t)@`_>H{UT? z?;0)3u&mC9RI9kX@2R7I6;nAZkJ0F1hV=m6af_3(>fpaQ>BD>CBokOb4Z|-S93?j3 z$rrAfSN+rLgSTJyjX&?5cwrw~!!?2uoETG|*+y0^t`+O>3nSTW7>1skM%E3kHHusC z3}6hdfmMLF&7Lse79b56W*K|gJM*1m`kU9%I49zd$L{jqwPokFt-H3r;j!h-om=U9 zLvr}JJS8`qpOut%Y=8Qpkb`^t_67w;1qJR24cr?Mv@be%UzAU%Uub-2t}t6wB`>T| z2#ba6!UC>_$1CRXN_Zt_iptKYl~rOXC^L;HN5B9S(PBcPk0SAw!mZfwgs9-yP#WmJeUZTj z62p(5+LMyEH;oaS&e)g1+?$@WCoL;3J?~gPKT9HKC=>$GJ`4)p4bTb+gGadb zq6GuJwtM@%vBR4qQrY_F#;0G5Okjr-EVwTZj8kJ6-DyVHoboS5Q_i?VE4$QPSH ziyCmS$uLqvq>>R(6Wq7=-z8B}3q*$zC%qldayux1-JJRO%gNt;I{KfVxjy;A*>vHj z%A#7erdp-0E~`9O*Z6kRg&&nvT@c7Bayi(HlEo@y3(C}`XRA)XRek!cvYN})r{8Wo z_a30D_Vm@Vs&n$fszPmpvbdqB@|?QjoU-IBlUtm@Q2yP|{?O~XH#quenDi;Tuo)SB zhQn0YygNP$afNxllu>@-`6wC0Oc+KWL>!63lhmxs7sP*v;q(-osI{rRHc7ULF}DI6 zTN|9fFNwPY3=K~13gOWB%h9QChBvb9F?0$f*YNYfp|yVOes$N!`dv@lr;G?NwAfJq z+erMwgGOwcp#X+Gp;(#0sm<0UAQ>5M33rXLj!wt*YpvJ5Mk$51Ltq%HF~k&l5z*Fx zxB`H+GU2#>QAOQS7B-eS~I!U z%&t`nAPT-st|xF|Csjmhv?l4W#6bjPU?UEE0mWu{ot!k z=Zo?)zez~Z9M04n&nwNBHl?x4ghdxs)$cGQl}u@cK-D1AoC9uYDlP-V6xws8bw5^B z{;06_$Fiyq<&{6l6*s2lYhZiMQ`F{hwTyxy4zGw^P{=M&@dTQ@d^v|(D3X@3_!^H-mFNAV=3d||OzUde|GX&K0`qT+g)5@1%8%MkMfMMAMQzd)A76eXvz zl1}8NX7PUd^N;n`+2+oPPW@buX|c<=)M2>Ss=N2c&qu$!HqqKX-Ptu|G?Q(ObJp2A zZ?(>v%+&bP+vMDo#8O*kEg;UYyY#-f_dWm&Zy-I3u&{V&Sk#-BjMh7t;$gqvhlv}Q z8fEW$M%n9ojA&5Qo3Y24(%BA7Kfwun)HpHfj<@k0#Olbv#EZVk7ro=p9RR8!(A+Ob z+V2W_M2A*Mw*&m}2>{GIxCZ+S;1bJ9c~c(S7$P#b6tT5Ua1YcfD#g-Lk)&9f z&C3gl4)Ko)iaZpPn4Xl7ek?d4dS^($>$`XUVB40z*|Oztw`|?w=@AeW6p^qeA~ACR z@%W_lBT4B=htrP69N8DVFO0UAPLD=SMvIFIN=yhjd_3|*CWx?@w7j^?{FwCIeOZhH zIjm&1AX}!$k;|A00F|b&va-0YMp>>Es%2cIh@})3s6;F&FN4cWNIB*o6Sh6bX9o>c zmj{iuJ&?A87PNy-_XrAtA9@4^?g*moLhF36cW~IQ06Ng?&0SvpA$xxQ?;qb@crl6% zxz8zvq5qBa+fuzp_>CIGjUfQcfqr_d=~MCx=tZ`xXg(xv4oZHZvC7epcU9aA<4Mc| zP+j)^C5v<3(z{@CESMcjCi}A9a<|iP_eT5tXE$a(`D*I-e;WPWpIm?Vtnb|Ae^zNv zYf9=B>Pq--fUasd^NyzMj7U+*6_*MXHMyK3zO<^M?yd6LE3j&t-fuYjW6)j|H5Vv= zsf+8i7)V;hg3XAkjzbD5Tyf=>H*78pDaJm55e(WORYar+PEw{Kcd5tG2~6yz07hzz z1YpEBPk=;9i-mSanp|>wt?c_Y%j`Si7 zShCm_K!oWG;~gEnH*a*_xQ^4AfnLNt-%K*1aAu0zRS%%L*$N5_=!MrX@v!-!#QCj4j5nE`PCPl?QH`fDVgyT^W zc^ErHE~wZc{8%dBY(p=cL$CTqzjY41v>}8*MZK_&Jh!;e z-sT*}W-lu#AyPv&D*2kXVR2JoNs~a1J!B>2P4I1nU86`^!WU`bH_7KJxMFP{TYf5o zb0UphbNc;G!&Gy}cw5(Wk8z>Pu+*+wZUcaQG5YD3!?#+-I=d#!mKhs<80KK3L$*6J zD8g{00H$|Rk)GHP)_c$Fyl1rE?FI_!7Jy#e`X#;bj={VH=mLPj=Ya$?!#?{7hTA!w z=u8iYy&fbQ4$=fYZR>N-e84k)5|-UNv_kc_P+#GGZ1#E%w>#?)QVO;#81;+E^cX#5 z8C*4hA|F~a_ro)ffoh<@Y@;tNBkNYI8Lr#LUs^}kofF^K#$EuuY~$;|EqKHeuiM{& z@amuWFO*`p{{s5VF@?uquQ6oV2T~Gfk%581{=PKtpm5rOq=XDsdagL1DPsY{a;5pv zC*nQmUR!)UF_yx|Z=0Y07Jr{N{dW4t&;W0WN|~}+Dk|mk)djM0siacQQgb+Jwx&^C zb+Jrcr<9k;#9C!F+yBq&Tekap zhV73`IC=13>cL}KM^EJ?A2^v1b0j9}(B80wsL=Qb)Ns*sdO~bq!hzsJ$D)p>#iVA( zWisNjSc$pp19|L2dF*tdl%Y~(NyNDl5wB3Gsjez*sFoJ1_$o11A;=LjQ}Q#8W+ulR zP4JEg+8W@wEzoN_&3}7f;I@DOc=P~(1tiSGZ=zS z!_d_1`q=dP=xvPnbD=K5zz9lyZEOj#waejp>KI;yw~aXIaZwB-j~m4n5M(6dfY`m@ zeu}M{n4Q{x*D!=DB( zzx^vs+1bjf^QGmdOGzkV1zI`IN>tTCc}0P!lr1cisvEHJtN!h>+N;`%i`5Np*EC$I zZ+aV+ru4L;uu7$^6Dq25xf&+FG@YSLP8T$s|H$l^bB#P29eXl5jsP1Tg+-tRf8_}h zLQfri0k_7`c}NK|N{vyT4l0#{FbVjCu0o7JBj%M;pNOKsq^&6ggNiC{4>8=^#tSJmefvqJO%krdu1Tcn=4y@Axz_5RrNZtq7 z!rZ$|!T^`C0}L0AC$?D8;sw_@pxFu{Y^tNZ|K^P@P++%iw%xkbMkR4H-y$JW#Kb}M zgu%}?g9vMBB?H5#~FMu`X~5--DsR9gH&dzEJRWu zS|szp<{b{WE0O*%>zvJjhaB*yqL$gRWVS40$B%u=Ou!uCTH_aL>u-9G$fFZ-| zIG)3dgFR6{QEG|A!Qcr$JW16C;KBmeu%V5d`vTC8MMy#x53AAjX20+CXX#`>|lOt!aWqB-VdNwyBhnK?;^Dryf{yCil0$ez}PJBFAQHRwUf2m5dOBJqm0l%zeNrY<9Mh{0>v! z6Y8fhyK)TZg(--A6W>^eaMFsM>IfWPcTT?Ch^WW7ddgVuoI)FYFOF0B4lbyXE3Yx7 z>4(!3X?uf!E`d=&F^6I^*%>T3o2lY46&#MLfT3jXIkpe(@>{&UwtD++@%DYw*Kccp zpI3Nbe9D1*IYV5^7Zn%qit<5B!$zN{<*^Idf)ajFLm>dHxKS;ykPC|>$FolEru)3H zd)tnH-QJ-AyMq0<1o&*D2kr_B_6(xEzQgnN9lPjz!{d%89L_j;Joh+YHQ~g*xMR_= z$6{iVCJ@ITY&`83cc+t${w`9s6Bn?}KhQHUtme$S{Ui4#W?oLrz8br|M(DKy_h=Um0HZQ*o>F}? zc+7yuV3?MT-!yU(MiGlTFrdEJ)QlDK`5+EvayVf}Q`~(3n5}Qg>|8KA7tKz%xC8f# zZqwc7u7$5!W)WZ?kNwB*27mv_zT4Kh%^E=yda^yqf;+N-E%CTZ}u;{ z!GkUgGOU@%CKB86VtB)Ih}!FW2VfNWhQ~2?gVJh)BtD9|9>)A%0>~*6%rQY0QEAk7 zjQX;Xuh|Ml{}Fn@BQboEj$I6zawl<+n8an0LyyNu*cM94iNubK{w|`xCcC?b+uO}w zesSZ*baqT8bwp%#*k_1f=qem-oe_)y- z!C33(pWV84&15i4kwY(BI8Mbmhm|5cs7I2KkzHhJmz5eD4S%Y6qX`x;47SXRI@7$) zFh`!5HyG!1`Wd|exC>Hj(F8iovIzTb6ONc70Y%njV2_bZO*B~VPz$bc1{DtSxQ~$y zwnql@Jp`ELz7dHZ$`NWoO7RzxlQ&2Jox!X2(8!#u<4a_ zzpi3+aqy{m=ov7~I~gsZGiW2ug22pBw# zKvpS`mCIC>B55gKs43LeX-m&a-hvvUMTl5#|{s^-q2md^3^F3j`o)XlebFWu}|{KKb1 zfBIsmqx-hegeP8R>kR01K$l)WVK7cwEI8HH02q7{s8BAFi%9yK^@y%6{W8ig%RS0pYA4ygM0?>GKbf|HT@A48{ZA-0 zZ0mi3W1)HPDL(arEq!FxpFo_jI*aLqCL!aDpM=;gp<7BV@EiuwhBk)Zm% z+kXa(!G(3~Iq3;|PN;?$!_myM-wiGN*K54O++#UMd?N#WBk7?Ddk&@@&E#gVm7D^N zh*KnF7jhYL*50J}t$w?=czbX4_IusS_jMm%4_W{{E;2d)6iboEFJcLcSOv;Ft{V0n zEOuc&&`Vk+Q`E?-FP2wcC{Afd$U*d^_`%fesNJ~7Nao|vT!lAVNiK+V!<|G};I~kc2=O0ZEj0p+a9~+)@ zJo;ot%&DxHwA|>l+_(&8d^#f`Bku%9kRwyF3RO8`VLFeU!{_jo5}=p3NK&BS7HT1yF){~LW8|S>0Tii zCFL6ucKB5GZ~xdlJoR+!_VbC^mjJM#iKm0()WE8>zTq{JKuT2+fk_W-h$8pKkn(YC zR1q=2mlP8JI-MC09!S$D4i6ioVx@4RiUsL~W_pVg_s%`0Yk^*{;6kCPtFFI%`pi4DXlzPRd=bVxIrXG^C4SUej-C8Qr5TWhJjw_Fq~YYilswim>q?U zU!-z4IQk597!@8*kqqV6#M;ml5QQ9rxi6^GEIhlSYJ+Z{A%PH9A*NoA-u{LP=J+1B zHmn?zuUz8IN6=71LOR^}zhwM8CMTg^6rJ=>NxwPC{X;7U zb~|mH_uRuVD>&lvt_=^ymhsWC43NN9Ff1Vg7-n?3v92!XXP;gB>dPA~%^kq78#kJ- zUjrf5cJsPBN~*1`gD?ySFO#%H3`8Wu!*H5qYjYcw`AbGQ<7gF<2;BPhSFN9a(cPmP zCFVl}7`B30K#IY)dt;iUL*nuM#U&*7D%q}O~~+wC6R9H0vp9=#8sgVbz) z1aD}!;X$YFE_O*{@z5RYYeyN49Sa0n)ZC&~BE?p48UU3mMs9A{MA7@$1gf3tZ6JnH zYL9}(oWYeo;1+CLM%OHu*74E~(Ao(0#Q?CO)&9v>_VMTTi5Gw?oEbKOgPZ$izU`m= zW^n$`u7&SLmi~Kq>Ay#p|9fchyVv+?X8eixUBQ07QK9tssJ%z_9Zo-z$;-+TF|)*s zd?lM9%a1w~yUovQi;vHn-ac;-1-2_BC^{+r6zfF3EK2~yD$Hlga#->lh9ZZlV2H}Z z@+zggT2XPKtm;y^tWuF9&JRo2v(?9AtIrMq*j69U?SXzkuWj_8?KB$v&e(djV^?_W zp7>)40I>Zh;}4}DIG%g-6!X~eyyHpvsr$2zg&m5f?F;qW8ypZ99dT^kxv)hj7Tr%43bQ>4i4U0EB7e2jy z``4cg|Kituzx++#KmBW0P5nnj+D27TO=tArVf-= zZNrs{>I-=E)%aH9x%Yrun$pu0z@+kW_}U2MH7r3{(kWi!g`YZI%LBuYT%#+1E}W1u zv4(+%V^7_|QDdu^6h+eINl6gx^*BET7)H=V&bn|>#3E}Wf&P1D4fJJ9y&Rk*ws?X% zhKf|v+5CPWccj&QYR?a_-4u% z4glPY?zZ)U_*k}L1qO3^@JP(QgnoMSyiPya-7|3GM#tA*-MV?B4HnU1?O0JH<-yiw z&{!l^3P9D2vaID6iSZ?~!P;)YQ?0EnAi~ry&USL|#n-+Bj zWEg=NY{t=(X0(F#x@)$BghGf0S+Ozs9GZF{l~i=-0wCe zS1&rD;Qb=$i|#;xr#8YlN~poI4Zts1qpqz3YnJ{so9m?oMc6l{VNhV-8eQLB`N@q# z8D*!~wW;i?%qTvZA(t0lkgA)~nMzP#98ocYqZCTY@|d!mTrrnlBvVzh1jR+=7saajto%Zz zpd^c}Vv9=Un$w`bGMO@#NXwI!f^L%+)<_lQjC^T6N6rHsz>#wb6bz;~mmxZl!pddI z|L}QtyKd@c%UEa6Y)G&r3z$_DBRG}g$fJ5&rKeL-fe_uhWlKl3e>g*dYG|G@&iHM;yiu7$r0 z%zf*c{|rYJjPU#Lf*Z$NZZKy3h zd2jXee$(F|#z^tsL*GU7+7YmOyPxNFe=pE*+v!1DX|y-IynnEL$7^qHdwtvX;Ha>~6Ne6_ z96pqO=t%bAV|f6u6R8Cm$MVx6kL{<&g@?q2`|b^<#m7b*O9p^NpUh0a@I!3>I?CiG z7x0d=Im!9#^n!vcE-SNukt<~9OL-hQpCjXC^BC}#+H*M8FOs&4?z@w4YdhUyyoP(D=h3BW<9l!*hYh<#W> zB*$P6dtI193jnk9Vk?-fcM&nxyJ)m6=`2g_#)X@m^Pk_E`OT-pzxYkxzy7-KpZ-mM z`s~k3$}T9BRjR^jRZ$&Jq)}+=)n#YZWf%CeDuKMRxbj^6nRn_N-!3j|Dy}$F)o|tX zx%bfPQQ80^3>H^VBvsb%rPUcsRUSw4%irD_8hxB}8h{3J$xx)&AM%tu3w24eLBAFq<7iM zl)*4f9%poVShIv%sKDR?L>NUcxPU(`#W}#3#j;>PdVxfn!!jcP%!sSUxX=T#3|WQC z02s4?(z-(}d;$E<8vp|ZMlBt>0PY$PX1L5YB$$M^^MAt{8C^ z;)=C*#p--&abVq#7zkIesE8M+$*68wI|)~$+9)?}npp3`5(QT2v0d6axJp5bm<5QF z9#33Az`d}$)@?&Cts}2YAjMqk=HYLD_1VzToa&=F<(Y!|lgyIixkZxF%gV~DMKx~~ z$ZE2=B|wY3GeLuCJD(*Nhk6*^*mmEID$t@@> z5ErvRKM6DvX@yW&%Hb7Blr>DgR#9?Rra8;xm4Hg)E2>3>wVH|xJXr<&6iY||CMgri z%HT_yl_z9zl`O6zD^HM@FUic|r(|%B9M3(L!urLpzwR;3T)#Ea(R~{L)}dRv-a7Li zzq5V$^{~+}V>M0LOw%TySNE9SFkwXl{j3Emiul<8usN_Z?2f^-)U5}4%{I4AHn)s- zbWV45-|jKYo2(ebYQ<1Q0NA4*;{y<09X*R6)Vhs#Om;GRrS~DA%h`v~^;9o8^)A5e z3wAlgfQG~}*dG|24^91#O?_BGBtjY6hx&$}Q-xyOkznt{bJ(LeCf1EZD0L}W~=f=Wm0j4QdV84s4Xn4FXU?YG0E|} zLw$Y1{Jlc_JcE5agMD{`^a=2PW4AYGumAJrwjXZY@%lE8ps4nN3{GAqCz~PUu%vvpjF%_OKZ(Orjzt`d^$rX43JKU98niPwXh&$^&aeQ_@W9;> zG|ve7&TzVCWUv=mzI($0q9goc_6EjA)AvP39Y{>cW&gu3K5>md8l7D2AAZ_D@{H=& z>z!Qd9bX*)fQ_!<{1hUqsR@=2G$JFMU@v1wMHTrxZ6Ef^>@Z|hq$*%$QI#dgC&bI-z8tuz1m#po~pbLd}w)&CFwtUr74 zm&GL)i;El8>T1wZa#a-otf=BF>1D3qO3I7Mnm}^Z)L$;EI;X8XTUv9ztQyUVRdp9Z zfhpA0T!B`qgcnz*hvhQbvlMUmq!Dox6r(=qiD{*n78FpXva<9eS{V52~_ z2{8BL_pFHkFe;UUq)AcTU>oBuah|Wc|B14XyGPdcfe1!)+50d}3f+3nWmxFzvx5A> zNFK9gkzzn+=WuJY<;yQ_-MH3rivSF3i8rYvU&@`))_kL*6`seQu-4`qSSB#aAs~ecfZRqUy2`9cH#n!8MQ!?n{CMn&wF@anS(J0O62rGoZtIObgwH1^9(PK7(<= zV!dOw-ql->Sb78-W;Ezbce?e91YLKq-U!bdFjy9_MB-)L;*8W4QHOoc*>QgpV3g?t zkA7i?-uuMld}8T+W_GTadY_sw*#TWnR4tK$mJ_?d@G#8MziR16s!@>~Xo;e7+F%O^ zH&#-Rx436~tiUVb!aB5O9()GN>Vn@B0cOEqj#q}k7p9?aKJQrK75zAa*Thj?;waB$ zaVwb88h%kDR}TBO8jiSHT-c;4xumJMs?eNMXwJYYkk$)}F7b-4WCunw}v`& zGoAXm4&B1dw%K3(&UWqEh}m%4Y@D#@$MxOAhMqBt8Fo2y7Auu@HmNsGkfvskU&}_* zQfKFUYwL7t>r`9YRA<-i&Ys&w%N?@A2Kn{KXnCYFJ?iMW)6s*KMtIXfd6UQ$s;M6t z2HZlMG}(4jkHkTZXI=1!#oJyyiL{bhqJ!)?@HXgrX6}DN1)9U71p}yHpYho2dTNB< za1=#Y@8mZF)87uy{$*tGe})(SYh)3#=K)&%Gk>w;&`Uh@qJlX{fBFl(>v>QAs=j}< z+xg@*4?oX0Jw1Fvf`enCL*k-Cz#}+# zJDu)93-SrWM&p;HLU+Yh(wcypUaNLc96lSh-YP9|j@ zKc17E!b(Xm$UK&JD&j~?Kx{~8LR4UMh<{8(*x>`wCsX1xa`$I2;fphBM~*5>>pG326LT*BV1rmNTz{tnHwnrYyB|P-^$z1vvje_7?z~ZAj;G%BL2UX%ixf!>_&&g3r$c5{zy)bS>br|G zKKdW@4#HOyZl-8NwDc~Z6mu?^9g6_4F4JOb&%(9#*^j>(`_;$8|MDOG|MwS$^Ot|E zDL!9VRIAn27OHCml48EROkIAasQkQGRm&0-t4kXjn%)LhRn}cPec}DuGjA7_G*s7L zu4}wnS$nalc!n^vxzd47N2S*-_Y>fLF9bXwE*JLmXCUJ}a#zyf- z488TlxPcvDsdb-Cu+nry1+5GjFH(H2So{I6M2B}cmtNA9-tGyYX z$M{3^Yv763^;^w10AQ^cqDU}?oncfAFnSqVzxul6<3HZ&(TxDH3?|I;MRZ}T)TCZN z0e2{a;kL#Vc{*vw)pqI(Kgr{hrz$|xwYC7F~kA4x+rC-zou?%;542!77OiMk+C4=Rz+5SLp zUPfFIHAaarxB&TuKLnl;zvBN(GHf8bZ?waf4iwlk3Sh{d0obSG@G!XFVdapBDx6Q> z2b=db3)~<3se@!J#bZo1JObxPIT3e+*k?vW82pmKXQqJ_(;$Y^TZf+8hu3W*FKwf* zOsdi#xuTkUNsUB%zO?#1fvlb@sg)Ks2^Dn;?OCPv zoJd)p$14Yb@ry3=G&l(p0CtK|n3Atd&XMHuwD~+udN%(^QdVZRFpr^Nuv9F5aW+TI z6qID-D^F&M82sW>Ig+$onOt*PD6ip(E7I~rc?Al-q(rK!=88-B(lSPYN+_?$WJ*t@ zbJB8znG6YwUvw&)cjofncjza+x-r<^jqS|s-E%iuXMX*A`}OO?CjFF2KW6M1F?0jW zChXQ3yKUBDo`O$Ek8aFhoUq#$ES4n$>_zm8?d>zo&690y(=DwN?Ojtk(}I!|2!5go?v2%xaFrBt1GB74G zEFn64e^f|pNaBfvY<><`#n0yF1@8;r8SL*79JDnkXbUl#@1zCzh6e;j(RYXV`b7pF z&pr-9jical3kw8Vp{PtQER%D!VzydPpb;eJrv}D^_(aiYF(E-Q!GY0qzbKktBpp=U z54L-N{QBEBw!i+S$Lnuw+vT-8HX-I{YSM|^lr&D(NoLx?w4<@d6G9UA(BdNK@sWZ1 zB7*iuhaOIhO-{rJ5Rzr$;ms$ z%sR%%Jer#xpOO?57wHud@E?i{ir*7@ATHw2 zf$)O|!V(jsj~D6olVKsDk>#6IQW33vuj_iPB*Hn~awhI)t~2N^~!%FW># z#YjWSTMxJzK-okcTwy0NiJV`@`r*_%{3G`v%;1^7n(3(?G74a>@%28cp>o7sIi!xh zu-}KM7fr^6 zo}P)0w%%)Bw|)NEm+mBv=4LV>g&Jtt-h89|#`P9V*l2-8D1|LwRL(EuS|oA6m`{%p z^<=6F(5&^w^|ntwx!Kt{2=vk!@N5IoWthN(4#T8QH`&uOMOia?bTg=;5MT?)FylfO zNHGvq1kUvs<|u#x$$)mC*r?hg;?TI*V+KSm0Jpk(7P`7Y z!z~yLcML`Zn9j7=gX<2U%V@@vFx;&Z?FBjv9!W(Jd4^|TI34jGu`F)b5uJ}rfHAy& zWOY8p`eE;i8CKs4rJvx|hRx1{D5Okyr-LW4uL}TXrdpWE-H`IllTthmSi$cRWj@50 z0l<)9Mo@3V&z+;|4xrc2x^?&))8I?PzSZq#HLvze!z)>%?c95(>HbYo~=kBI(!tDsh@NRO8x}o8L2yDbjUj)2~ z#9{xGx~^W8MMvU`){^+{4D_N^=&_VecRu>vGoUUzV1W!KbVq~!pb_Hml|_CA^c!;XyTr* zgV7O3_J<$YA95%z_|U%4BM0_pWhP|h0Kg9C6`bOVPZaP|_<|g%oS{(Wh{fq#b}A<; zjhlU{Am>B@<49gcV#cZXlw(mx6a8bty`q9WBSK(#MTYt9jr5C+q5%{S0JLN1aeD*z zMN#yMICLoL$kCXjRkNQ>$=a9-5%KAD&@$)aV+Zix`5R;ROI} zxv zOI!q!^_XG1z0lG%k5$Ca$N%Y9y+8dI!+RfnT~Ymkw)l*;xL&3xmnh0*ni_$kOr)yf zNy=eW)?U7N^`kQvepFaoFE6So$I9WE`lh$EWsN+MHlM5JiYozNX}R*+rXL&avx7jd zF`(DeQOr4XPf-~sQWPjhhGc(2|yq=I}_su z!5B7>^<#HATBrI*BnQ=LPPICGKRF4}MHof_j4U*97_Fq)({hI-BHl4$6%oZsPtRmm z=V0?q{g+={CqusgQrNHzd?Ke`z_5_z*s3G9(|FsdJtjL=-M#O>2x!2p)=xXmL35H zKw3Z`=|MaJqOd9mi&PE)LWn2;7!a=8fF)3)8D2bxB}7t1BzQw-8cgkUi~W(={@B_J z_*u1re(HB0VwwA4w~mesYu}26GTaYhBMLR>*-m94Qnt4NG)7VWH)_`pBEUKF91{@- zF$2OevThmt|H*m_=s2@0ZMW0ycAI3$AZBJ}hAK%VD3zeZOqL~?+SCl0bS9Z(l9?nk zp~7?sVp*g6f;OWfy3y_N?NT)V!La>h9d4R%Tgeam9tws*6SC zoj|g>*0<}=yBhmn?DnFH4hE|+msyvaU(K#QUr~RNURXoR2btAS#BRwiZb+q- zfa;rV6a@aRCJ0e)3+R~~1KUnZVcp z=Sc97l4@ZljznyMHQ-ck1AN%0zBSGNAKm1>c4AjMu@5>-ZQhrSZi#hUBHcDu4cyuQ zV&Rgl3`w63$({~KpK#zy`3k^nM7ahcjHlV;X}0*fomXBvas0UL8GA2x*B~F4U?0yg zfA0wYfH?oKl+aXWW`1pH5+l<&z{AGP{-m4JQAdY2Fe=ah!azO01ps!!&eq1!`MABq zVcSzj9Z%W$xCX_CCuFBYWyS|2gt>Dk^q2pGEkAe0&arp3w*N<7fZe!)>@0FBKPbVjgQAEUr$?-iRBRy>KPg485Qmk5#|~a>=qW{6CLFj7aN$E z7@C?MlaUpl$;dCO|Knfpq5PWMHcjmqDS(-Xos_yWns%vK=4iP=JIyvMYM)58UXp_0 z0mIOub6_SWB@xnKaX>N+mb2t)j6ftaEg#8sIIW5bMZ`j)W>Kb@m#XH4GPwIJ4DlB3 zV-<1y4}Tf`)$bKQ{YC$!H-A;%bgjANQd7&Ns=AJn>Q+`|3vBn;HQnVk-L2?kukCE@XH7+i$&xDR12|SXxbh_nRO5-MZrvuoz7@S9Hs1qlavMVKzFsjuosPmR22~~`i2JjF%)qC5C+ofE=lg_C&fl|A);B4lplvk8X}Ir9O%PA zV&B~AyY)3kAhPg<<9vaMplci+@$5=4L)qfVVs1iQqd+uz+)TAzlQU;vbPx4QC|6gJ2A9pk#I;{3F8)fqo4!Hoa~D zzyM}|G=*kUsokPX^LT%Jaf>`SZXuN`yobW08eT+r@_}VhjipFrp@P&8WrjV8eqUlh zdI=4?lCkHX-Z9ZjZzSZL%V2gh$}VP?oG+@rNMkjXG+wD{zR6^F6_&!s;bK+8^*mO4 z4y&!MEh^q8GR3vpbV$xaq(stYw=9I&Rva%&Slmn zWfZ1om%wk!&aX(#Wabw)e)>6gkTY?=&&c6T4{~Pi_f7rwkAiQ$QSmur0^S%fOd=YW zNhZa@aiPG%=NSbfMls)_kWNXZv#`Y(;Y<$?P4@H}@7^;E44Nd;MHTEGlq+J{6TajT zY=Vb*3n08WJlM4?VP7EWaHp#1Se794qUP>`VuNdm^2ztx1r;o~AcZCT4&3YTNQ_#P zvjgsm`ibYGQ{RnFey1DX*G=pzOo}wuH(Jp>mxoUj=p5CtmKCJROv- z3`gp0IN~a@O|s zv!{MqgIEHo;Y^qjH8cFR91Rcd2wuBmPeFRWS~z>uwPuLUqZM~Vzhf~ zxN~HnTWokhT5>2YD>9prSXi7;z>LYsjik}z^7G>h3Zj5*8How`>9Garf$7nKDN#Ww z(caM^&OyE|!TyeczQ8Tp0PnK_UiN|BZefAGu~ELU@PsJ$@G$qVFt4a6@0jR-g!stx z%vf4(QZ^&2p!Cwse-f#t%+p)O2?UsF3hj0zNC}7kVmW?v8wW!ecU5CsB>oF6p~_JV zAtn6~8>HC_-wgy*2kn+G9EoJZqFs}#vEqyye^N7)Yaht5ib&1FeXLMFkgFDD%6YkJ zPOhF8E6^A=$erySp8M+F)Sv!l{LLR#KmQf??f3s&-*~gB>0(0t7^NuE>VqNAK?qUJ^U+BJl3{WWljbRN{5VeeB3)lpKQcCTp*0a0Hom6=J`t)>B5Z8@nZb;xqjC|o z<2zbHFRf{dsIfheUpNf_Cv;%WveK}o&?C2qneAYV2vt8sVv()}au|sUj0&W{rCOFM z9w`)$Bcj(J62RK{jXb`I$1?-Ni1QG+ zg&kdD>_~$pl*|jn^Fj%xDj=LB4}>D@36n^cP=UdB0BHnZV#$(3_7MO6*a`;dLLUc^ zMj*vBJONyb=HWZ2R340t95wk;9= zJvlG~l96*Y__d_Hj0_@|J;rEYGP)k#_A+#KtcpRC!R_y$oCqJBP59sdx-_7@5M4M? zk8JJn(1CtO#6T~@Gq~T0^t(d+-hIJxUDq!` zffd!iTi*IXS>xLcUH`y$Ytj-t4V0yIbwsl3?dBBRY?l+In3t5vi97< zT3Q}EpIKX2RL?4FO{2533u_syI%ZiDbpTda+*nl7oLg9#nO#y`+L+I*&dR}B;!pnd zvti!E{oYXycY2634FLPYe~G{PS~W7H7mVmd{85R}A`wl1@Zyb(@;IXcu2CwQ0DwtE zQzIkeLxbb}edcez(cZqT;R|L|IA#Ic%y}XZ&<{a%4f7W{0*v_(!>fbM?9>ov+{RIF zek>uPV#6~t{Ff(ftOxBtnhlX`RgTA*R5-&^5vV%(BN@mw-5O5WhL7jecZSJtwU&LE zVGEW-4||k#k!FJ{U*Rj)N93zRk|zV=$Agk(1eoaY5Rgo=G9up?kpb1#h1xB?W>cu& z5gFjM+ZK)Ph(@<1Mly26wExN*wkJ=z+uH_uID~jRh4^?x`uj!)dPn;E#RrFH#QMbs zxrh6@h59)Lcv!pIAF{DNYG-%a&CS};(b~}o))8CVBX;&j?Hx`yJ6pTCp7I2M`8pDY z!5{RD3h;^w@Qx1jiwp9P5A;t6@{02IjrR77^>+>OvUYSheDc(he-mbQOh*gkozOli*r%1 z=;s<14D|Akj|II2+;R^O2Znh@MtaA^gr}v&XVDY0bCYum3M(4^^3{-Odebzu4GPRW zy-WP!}tcAAzD#m5i={ zcZ_aXu6-!gJdmm%5>p}$2*&B)=;WT40l+j1Qq>Y{w}<(21Kfo>eRH3EHU6J}(*FEc z{15)=^QM;fnwlC?3yjI(Cfn9s9wB|gsthJ=PtqV(tKYjPZpIx}}?ge1j<@eh= zt~EAaY;J$Evc9{xvXjZ~$jYm4?fgI>oirGi$4pNt=OG$5$gvi&&TpCkX6DT?;|3uX zPP3#;iA0$ZfLX9N?qzx}0t_|TOLg_aI*5XaBtKCKYy$g~;evV`n|y{dz=(TZi^21| zI?JxsysM*db#T_TM;1a$Ke4MB-v)-kwa&av(1itFGS40x%r#@!DyoN~QJVj|K?O@EEM;UVrcX0SuNJ`1O7&-8NM5NdfpOg?$ zw^7iNpcfw*CVfC@GW-rUmSJxh@#sH%G0mCO72yI=1nO#Cq*^`59K!5|O^qG_hLuBj z0&xw%E>Naj5h|aE)GH!thRgpI^KphaMRw(Cv(}3ON2%Mm`Gsej~KZ_20nh%Boj}_M3WNXB!6VG zzt434-ss)ix`BR^RJNqRAz_aN;zz^Wg@NJO0nXgu$Q*|c8=xi1wIo+R!cz?Lai9Xm zlmuW(ylcJ8aeWaXq+XXO*2HqOGLm-nCla`Q>DTbHJoyZuMb0316{FiYR7|-hP_4rw zpo=4YLKapCdxQ^rg%5#Va6O`0ACx~Gmahsl+kDNI1hZ7&h1(a6?g|Y%QWK^#D<__d z$F^nWJ(=bCD>hz0FIR_fFUN3i=LjE;wEAg9Ds`{G5Y*fLj5HVF3xD zesRG82_d1WVIir(fr2?2reK^`GKr<|P+owRx5gw4sbcHUtjNx3;W<>hHbO!u&m zqYk#m9qmrKI)F5E5BKwr2e?Im7W9q@b&n46hzs{iNk}YWW>?pwmz4s&g3?m_k`e+^ zl7cf+LbH-0vJw*W(-ZR2;(_=zYBjJy`hxyw3dO8IJItBzf1i)W1$UQX7D>A|} zGRz|a(FFkWh=}xviu6m0kIYI>q|=gfa?|pdUDrO4YGxmH#L zQ!m4!8j_@{2T~QFYfc6LQ^Or$VMH)L#9z2SIQO?(pTcDs+I`oA|?*hRhC^e$QZW}U?wCJac7Y8 ztuY+QK>-XjoSFbkYerUS@xXswX21vlB@SQ45~2!Em=9*Q;d*JK{~lo9UlV{m#gsoB z_xBj~r;-;w4qyN=(@%4Gqka9NufDuP5)LuCmjW2E)PwfIM?e>dEm#A@vVo>VGX1g_ zsQ1FPNV=Xe6mby#a4!aS-1?^P>u-3R5d+9CM3)d4W)=zoU=t|3D1b@k`NR#+BcXQG z5)oWssxQzBB$pJg$yy@xg73ih51xELFRT_Ge22I~kYQXbCoU3(VP-jKJE|=VT^?fm zBQF7ll}4h#ctR{2Qj+ea0;7H#kJv*5vIpc>;li|z7t@?czwH5sy9t9VXE433Sgs`ek1X zy*)O&HL0MTUUn&)eL1tFJH4>I>B5gHTiz?Gc#~awt)liStGp|R+0@wie)FYI+OGd= zY0LXr>`U3@Z-yk6My0TFi&|20s?u^QODfxQ@+z3Dy4w2g+WL#F9q-h&-YBiUP*IN^ zDXH{ICi`3_qavrUw!FHVRnl5rccHkvmC0_*E3U6=zEWP>$trIzC~aXB))z4A3JNO= z3)pOSLwbpJq{sGI-Fe?csw9{)Gv5ABz`<1Ul|lHk0@4wUL3^=SB<4Y zzHSR%FDVWt-I1I2NxbhqEQxU!gcp_YtDN{&VtV$9N3@rFoR3F>uX}>8XM&%5te-Ow zE7;>~fa_TRTcC?`po>e8yK|tMqo2!J5680}&h}nzwjM6lPIgxIXOG&RJ?`jm(%I#d zo4bR*KY+o`-^Ve)2dLu{6X+ig#0m>ejSNkX49|!RO$&}l4~a+%4T=kP@$)?C;BffV z=_9AkIC=Rb&}dmDtd#sb-{`1gcDAot9e>T{;E9izo6BF<9LJjQ`cWLUFQ{Rjk^RJ927l4%#@TG-Dn zjtJ)lxwC!4b6?(_{O`Yy{o8M)AOG{6&P)GV*LbbH^JaVJjrvxUR+LY-tfKA0Mj9R#!U8pBAi@ZbwW!7~3$UN!5dKTz zm()-df-W5?15&aKprs=UOhXM%F=N9xIV#&TnD=x>96XPmUgm8LaBE^mW!}=lHM$Kq z)ni+7oGH4cHj)-STv8aoeCtd4-i*9Ru>0Biru?& zpURZNwy-{QCnEiN?*P{B-W&M*i-DnG9cVAI%s?`scpUq{Br}+3B|+N-pYR5cov_y! zu!;pkYEkt=3R+^EXNJNH&;<((ikL)>2~q+n)y_o@_JCnhR7>BVjlnZz2l-M5?hFcA z60{(q_Iimf%JlI)zyNHdZw%9_uw5Geb8`4^EspT#cv!Q-v<_exWla>5SYax8?*Lk) za|pM3ut6l7Ji@R8TLIJzlU{5HTKd`i!6cBm6+d|z-Wugs7qwD7S`Q}r`H$NTyE?JhJ8@g{0^)9A~39=w7tCHO?K1u zs*d+-yMM%}xemH5ETtkKh7q4pnn){8$SBJxtSKm}EiG$mXu4F}aIv=Wa!cpCl?_*F zny*#VU&$+OEvW(!)@J3^lvP~-eb(4|y|k*mqP{b~q@kwqQf>3)lFE+Uq6T_iO+Gx0 z#$qzdSw&U3x#byIEIOm=Z(j*+_vr5Rj1F_A@Appr=0CZA_#?cRbONqHEEp5<$3}+r zJk(x5FSArQ0RWQ-Kz@yL2F>?wYkTi$#iBWd5;k5BNZ8oIFn5l_og3!N4D)7q0@wzj zGE<=i8rOjlm+anf!;QtERf!zr@|r}kDweN^NOuIh`sC$9p1}=JZ{9adJcmb}X%Ai+ zcp*fpHNIk1pj_w6*G42y2L(?C_)q$Ij|PRygF<+G#E~p>R$rIu%}1gC`rXM_f3h6bbu2c?I2#`xL? zxZ3->p7D3F4|aD6b#n=Hat(6x4EOYk@b(DycJKv^IoNqPo_4XtSPh3WC!OqUoE=K@s0|%aeO+2`_pc2ww_-0UOx6dK4-l>?0ntr{5`PuD=g3}CL$y=DK0lF zEssHG6*H^qfA^Q$mf39ruw5gbe(jQ+S+pT&jXUbmZJddYjVb6W6=R$wFl<|)-@$H5 z!@kV7#H-a0sz@P!=?0VYu`f$);3<|L|F znPyQWo97B=hj=r6!?U+~r~dM#={J8={^S>fSKsof|t+2F#URaxyQ35(_SZLItgX1Xx zY}9z5z{boN<2(92z1B@;_!0oNHo3SrwX~-#^fsqQ~0Ebi&hX)5agF_rj zfssbB106QlKLqp|7#O;J_ij%Q{281QMkb04fAJ+eS0NNxKzIp7<6;T+gkjj7Y#!(( z5KW;+gUBXKfg+wmQYxehh6g1?sSNL2V(GkGv48{SNh=%H5|PK~mL_GrMQiJ}qy3&_zHkEdMZ0F2OU@ns7$u^j^5P|2ZU!vTqpMflBwun)@00SXL%A!&#f zWBSxGZsgHnzltNCW!NYNn(UyiuG+e8{>1^BGPJ;@>*$S zU1`kb!piRQrgw76F6EYW6_j=`%er!l+6v0fF{`=?tGlZ^KPYOv$*8;@l3WoGo1cASDHb4wu- zOe&R&5)5HMdW~>r28JhxMK2)plxSfn9Cxe_w1_vo2;zUy!HammZ zS`kX0@Wqe0LX2aD_kc{jqBCw$TS)_81}!$5GR=krUIi6$izi=0cZT3m@5thS;1R4L z;bV?ynJZo4%U1>RRUzmzxMz-T3*l}#wkt92NRdy^rKVlQIA+oV!*rA1DvdjEAtOsY zg;Or^rxVX!3C#!#r$>ckhDXq&!_vb&BE6h~+#Q45oI*TZBfPz%y*)x*-9lW05&~kf zV!~4*T>{6oYJ#qZ- zDVw9Vrw^Y!^}79OD|hEJeqK(Y{_f#{4&H7yPPSG~wx@mE?1TNBBZ53)Bm9%&1Jja2 zveJVy(?Ed*rX>VtBt+$=M9>o=GUHXyxc+qwNxko_OeJZ`DR59H%FZH z^%Tp7|Ih|VoWYbM5u@l8T9ES(i3ocvRpK~isrmshOst%jD(7VAaVvfzx!1C^I!39zW-@M+XwY6*PGj~w{~2wYrza(c2!4p-G!?+e{$`epLSe$x4z}_ zxvraCm)-?>)irjPlsD(Y$FHI_yRi1%AO2dQUjzmAf?n(7#0yOvID4A3EgMZ6@S{=F z8hklEvop20H!;6629z?cj{&+c*mrYk;rYyi@2DS{XLm*?i763u6(%2U>rtrfP@WE| z7)a?bpb?UZw1S}_p4j5?6GCuB89)fw(Bc7K>hSq!`at7`nbZ$0gY}sg0Y+Fn}3lK_tO$co_DATMA_j-M)MGdN8*aTfT6*3YogxPh5zDefI|j`#D2{93ax&yQsP-9X8lM3=7+z`v=i} zh<_AQ6VYMcJ9PW@;O#pCF4rg&PKw1;wl7Wv!}MMWIsFo0q#<8|nc_mM4pFCE)IzU6 zsT@9($R0@LOHu`<(Mzx$8+i6WO28~u;-C#g6gn(0kY0=dz7L44;~=YENPbGVh3KLH zCXirmFKPM0hA==EdBh4KF>xU6kXY1V7!ZpXK&WaY{5JfB7!wQxT@lKk2;?}fUI_G3 zt;j%BVWSuTX^paJV1*Fm6uRWGcqm7MC~g69+xKF&1By27x>SvQvLZb$+1QrCur0$x z#2u*tT>zp{EFtpsyBzg;^P9g4OstB_Ziz^%iOj4mX?~kkcR4MurnvTMUgewg(k@oj zr9yUhL3vkUc}GR_RYv(aX5FQty6bsW*CR9P0%8hi%!ZOWcuq$aql#Hf&cN8s?8?^C z%GT!28)Y?}jpuG+S7ml(LV9spc113;F)Oc*o?l;Hd$F?qQuFz@b1?d$zM!O$QB=n$ zs*g>{r{$HW(^y%w;v!}ZBafYw%79(}dmsO%f5`OZ*Q$Ym@t&UXKmM2Kx4#|i@6+-) zIzC4`GNb_o#^;X05($h_kwqe$5R8}y`}B9eQH%_Ysg(;d>8waR&*#qry#|LT28YKv zBNi^-ERjqr6$@(R1C{EL5@Tqn4#+1`QgR@>7EHw?>6eeWf`=U5gTWCTCkEOJHWPXy zx~Skzh|X%Xs$o2sP!1cBr#$IXo@9k6t;d6chj1|@emo?3%vY=ll!!N>YF(_shClh( zj>3eoQVNi6rajf&J z=|GyWfW$!ea8H*|59crsw+K(qC@*2m7+SlQd2a&|cF>Ui4C>4fd6SC1V&?P}{E7Z{Nl6;6u@r^QESC&gqZ zh9rhNcsW^}wK-~S{f5n{!#37#0lr1G70C?RNoV_4PaZpT(&~ua=@TyYN1VR|SWzjL2Pm**g??F~!=`4L z3hthOu7^?$h_DAT<)TzMPnqcj;%Saxc3@=o&cO5+cgO$mSHmy|tyu07v<^@m+oKE812!;a3kFwFPL`xRB~#q1_}K_z^U@>uo7 zmFGVDilZ?;))0UZ_xz^G1jJg${6ovSfs$W9GUK|=xNe--m|oa3Pa(!IU&^>)GT~G( z^W@IN{O;J~nt5h(?(uh%i~AFc&n6b0lhP}wrA@tMgX+_w+LkG22Z`L>)*Ep!4I*lM z7abX-51O<*6L;c45(n0AQN3Bv(?MA_s6H^fz_9?>cS4C{qQ-Ekak%IPhEkF+J#q|2 zCBqUmMvz8MC|9T|B6(b)j$-zi2EYLGogM@G`fG|lP@FF zn=^|qrWBq_Ejmvt=`O0hm|j?4)_j9mdxKGah23zoy5;S%`m1TV)kT$E`4#8OnyzIO zwx=`CM`hIcM-{LtyXrgM$!0aDX0uuBrsA^35_VH{eP?N9TSMEm+U7Sa>n~O}zL`p| zh)HH<7c}G+G!&J#7nHWq^6QxFa~1WMSnRe!RznH9rLeT2xS};SB`=L$oK7#!&MC{u zDP`oBCnRNuMW%IL{J3Xe^ouW*_wSqT-7|gqXUTv5M}L10pi9dg#)Tx~8Mu6dNMsZX zEdritXh47amINMEiUp-?UV>R-3q0=ZFlTaL&^$D3;_@wG$)rL)t5Pj0R1XwtoJbD0 z8a!(!?+4tt$WdogYI_9J6z4eHC3vx5Z$T;_`?|3`72N&Q*pn_%tVtB>Vlu~>BYZS0 zcsK;K<39j;f#e#IKIJM_MwF`pcqz%uFtpf=qQOcI{Il$<082O??3rW~9fmUVj(BVn z6c|8@2rsN00^5}1&q1H5CclFV?acqa5}XkloDmk765^j2^0BRhIKTRWVxaj-q*WOvg3?6I?_ zkK3L;cG~)|)zKrTk9mf9M`y+ZMWS<(;~1$CnepzSz9;O?965RF$Voi?0^oUt`DQRP zeWU%}IDP!+nNzTioV7XTc=m|>=_B?w$DM6$Je{0^J^f;XeIf!Jyj^WQUC;WspAGbO zhzN3y4fjfj_D@anNlo-kNes?R38SY*toqv=V}S&0#8F`&SL5+mIL{g0is zd-a$VEYM-E5{4Z*32N_*cVs|pc4AakoL_vXPi#2QD=Z}?mPSiwveNQcmu~(@u3NIq z?3<={M<=nmMGcpNO}!fLiQ53MmrJhQl4&-junkdwIwpyw`YkE0O?<($*#0FY$%hCq ztb`nhFsj*w9D|YLQ*8gj4ltSK5f%?|YCDGVNy+4RnQB3*MmwTNHp3Im4)JF14^4mF zGx6uo$A0@?>VNsg;Ki%IscC$xx#R8H#tXHL-4(T+`6Z3a^0xDrKe~MVlgn3sc&Ef*v5AdQ3%U)B zmQ6iQhFqPP+nas(+%mg4KDP~+A4Sw{jhZ&$X~xM7z2&KKY8^zFetdO&es6N|`4|St zZ<}UzOw%C0$V@NHj@luv&TXA(ONagrED7p?N#-p|QBkO)R2P~O4~mBvu87`x3kWP! zS2}`2SZGt6*w>Elt4+YLZ4GgXV*VjH2E#GgBu@JUzz$R|H7NbyeC0*@0M);VFN%g7 zL&8g-d?>>z;sXpt6wh<{lRUm@Xh?yfi1)B78HM0N!eQ2s>N_S%I z6z8Elfgj0~Pmo~>Jn(>9n?Oc2FOx?pD6mHqx~K~jTktoL1CTJ89$kxMQ4AyD_XqAo zQmBN#_5r{ICm-pj9gi&lCmfzcA1UC*q&oXn^$ zsJd8KbB$hnzOd>-epv@SzloM#&tSC_m3NiYy_w6tNGrXVl;0K@$I5246xUu#&##L~ z0X|h{=T}tMb=EguF0bvZYq?tAdab47ou+f|WmB~0IaOMIfKrMh{-Ikx%lO6`4?ZSzy4a+ z+c)v&KTCe`%e%L~kq-^1L4I+D)FT`%08Ah>O2if+-^3Xj?YXCfB@s<3WOGvSjDSDG z=g*OzuyGE@!sP+IF!@)Zm{+Nmlxo-|KY%4yK9CX>j(ZMxbe5n}l1dBH2^1e2pi1!r?z29CNQ_%RFTj`zXIkQ173IV;<-~K%=Dz{t$77`uDaTDdw{xj9(5+SzzG*!jEp#RY|?L8^WyR;@Br@_6X?ev}ZA1LA@!5Uj)UIKIZ1BnW zQG5+XG1~_FE)CmqJzn5;PrWWtVO*&|waQmMB_VG+h%xw#p~ZuYl*OXQOA$t@9|r+q zM1EIFSsC^{U0>H@b7^_$h#L6XjlgaQfY(b`)7t3e(;#m%Vwr_aqcK_sGZyEpi zXYJ2^Ieh)?KQ*;|+}QS3b$u7Jq&}~xhF(xrTGQEi`J+qMest-|M;9)=k3&^1yw%Wh zkzLcCU)qq(tcgm>xp3`A3f-dK_+-qqM#2xN$tlP%0N5x|U)_s@G%EJjsOuq>b9L&|3yEmuAPl3TwMit$ff^$vM2c81lT!`^88Ok|YXLh6 z9BM3E0zo5`F7SjiT!CeHM05N0!0p>Ty*=nc?86ygm<)x-T_o-Beh(-ybZ`u!m17vV zg@GM-64nQ65Rb}`aAcwFo&c=Bw`cg9Tks1NJl;4-;+PZ)Fpq=yI0(RG4?ta^Igl#! z5x)k|LMneOlRuV9AIN2oRH~;6mY}I452! zNMj;yws8OSSgIg0RQ?!?jik8r|6e#f48exwK_P`MQa_}sh`1yeqDWZ)4i3_Yl@TKo zF&JVQU`7E99wiDCRn#zh%Bd?=f*>QGFA}0C)U1irYeMCUScy(pPW=*Q%Ru)U>@*(eidp=MPJp-^(cJ4oxXb%dcnEUSic;Ow1}vp_Szp zRI@8FjlQhO}xBk%iA0>$|V|=4&wsccpwzCYce7X8F9?5zZ8c zGXeDCaV;XjgiK22eJdB0h^{56a!IDdfk9GyBP~%*(GkJI5O=PhGYwm0u4n%~OB58`;hlAV)BcjJ6;>UdHvPh0KVX2l(VMclZqBfMq zEydV|8Uu`Wlw{b70&r;F2Pvk6YtxR@xDCW2^S}^YatqG41d^$zo@=LZsVAQ);Z`vF z-77J<@!|AX1Xx0#cf7w>qMu(%U}#2IG(9RVH#UwDn_Q4UD@tLqvvXNAP+%#Hq<}b| zu++dUPX127LRcQ5F7{pyF23$AK^~4lo@f0$&IWin z1bI1zczHzm#nNIEauUMRBK>27UHsk8csbezc-n`90*mxaii72sk{Fno8j_tJRgfLS z%#F&U1*a#4WF&;s;v;GCpun61J&&F~{o1jUhmM|n{kS!d3^4Y(l?_~+b#;x)NTRW_ z<7kQg@uBWvLEh2P5g8d#S((u^T6`9r&aC+B*CXa>3|7asBn;Nu)?#zRmVA`V;(M7Q zN-T&J!(dkk023&mQky|)SP>8ozLX>9)~P8fFEK{8nN&U|xr1mP$utk;T3D0GC}*)-J1%~3@!AhBUj7giSbN9y_KvH~?UySXy2@(13d`D(GE08`tH0_^%Z9P# zG3xY-*yYXmxISiC8?&sN(1^H=pfayb%i zEbPucd^S3{t{-2w%vFDlOXz)27MXXxqW ze|_s7dgyx)VE6B1G!&H-MQVcsK(B#8EFl8DdT=N+72VO_cOOfN#AJ_?!+J3V7!?@t zG!EapJ97V?l*=<=tdw9}gneKWxP<3h*s_LoKe_y|Qn9R1;sSc9 z)K69FWtDPSt$qshg6{y~;G0TdAbkt~BV+8*>V~aFil>xQ zk<=<++qZ@W0ri?lyN-+4=n+>6c#T$NBrbsL4$%T9&}{HE8vre!mka=A*p-g%i9mj# zlYVbNw%&2`zvHscXS1&|DzBy$o-3-m%B;SW&+e{ld7Dv&gTX4AZ!}-{xT^InxTtD- zxB2ov*L8hT+VXyW?e&QC%E;8xlKLy<&DXOE8nPHQ*?CpPWi9NgbA=@!xmwGrx{Axs zH@ClCT+vy;Zp$vLV-(i2D?2KyJIia%Ggx)Bf|{bz)`Fr2W^qGaK@}svl2y`>Nh>Za zs$~>Z#wTXea!a%G$`e!a(lbjx`lsLDzOVo6bNTIiV}JWX{ga=3{po*m`tK`81~q~a zy?|>F@J^y}{2lJ?$HO2w0M>5LS|#m#a7LPHh~*UaUb1boa+RLW;@(zg;v z+(}@`0bL~b8ArsyTY8=+Sl|j4L3?oo*ntk*(v5DZ(Kx+9J{qt9MC+4+|tc_2agKNu9|%3l_+{lcvf^!dW2tcuy0a;Us_0DW_Va; zL=-(LHYYl%Fd?ltIj@{nSe~6zLZ_EzM`c8~hdTSkdPHZ2`$qedQ7mW9c-T7oyV`m> z9zK2Sb(tb`-<&2}R zV_0%PXmXHSh#UL{_h3)AP;cijU#GADY*>y8^okD&PLGVHMJHs(#?fQL(;|GM{B6CR z&iJ`Ggax|AN4Y1)f(Q#i3u0PmZboDQEvhgZ02W413(HE1peIGp;scYz?Y&)Jx3YTm z$cfk9z|1P37pk!n)`zWZyaEI11vxpzjF`+s_b`7uUr(o?px~60sH`jiSV}Gkv3rIMV3<_DC8knD(d@e^Q{%*REW~TzGqxtytYSo|7JqsW zu&pKWZusOEyIuTb9`puMw+}Ll`LRrmwnKoKMEQvH(muqvdJKAj8wUZHY+4|}@w-C^ zu&FQZng9HS?mzxZ_Q}us&RzILb<;bY7eDU4^kGeXXJK(&b#3?MYd^Yj;}eiySKj=v z`{FyT?QhmMcURVTv1>0dSj~*0rmyY^N6pLn(Pbk53_HL`7qfZOXxhL?UklE$G@7^c z#&yf|_RP{V1GV4Gb6wZ=&Dzb9f+`( z1w(iVs0*yHk;AZ;_iJhazyM>^8InM{$Q4bG@Qs5*(tGy?@fZvPHD3OYoN4u8@sPyN z4`3@8=(M}{`Z2l#7aoZ9^%D^`4Dt)>kO(mREx@&V_c&jF#T^*bVmBDy3;+`nhT*B0 zbWtXID3fE%l}zyvYs>OQ(lUrq`oJxf>Ip!L(p#{!+BJ=Ol?XCG8ZhjsR=2Lvt|7Nx z3@xN8s0aTJN|Me0(vYTjEGK$MBFBG?7jNBz^FpfS0Q{N*9TupP>Lb91nO1LUIOd`TNmT^snar8 zD>CY%1fL`PJg$rMn|%26>9=IaussY#)a?;~?Q+#SpZuDWQGC6y_Pt#8n~B+t#Wh#+ z*j;p1dqc;E?E34h>MK>P@6>g?S625XqvRa3`ZBZTT6xR+#f|UeR9^9mW%@?s6xCd0 zR(5AF>gWXxbVgNfUR7Q}O0U{$hSnb4&Z%4Q+3g)^=wysxq@HX!HtpMSDqQ zYen68cusC%U2$1kWld*AWqV#;MJ}T}JC|L+tYH?_<}ldF>3Ny-;*2a-Vsc(}?UlcO zCH?eok}q#*zxYb~vw!*Kzkc7}cTd6TSM!Fn0*+q98x!$HIRo0ex5VFkDHQT83fZ(o zI3eUulgi=5;E;*K9fNm~STwGZ&#DwauQ|D5ULso%%dr?P63vOJ8FB&y7|!JpU?vN^ z8HgonQ>|Oa1WVN_zD%;GAi0LPOFaWK{ljxZ+ZjH0`Kib4dYgd{0H7g~Xcv_TXaXTe!x`;pDdfGg$DY+_Iyc z{`QraoP_9X6k$Q>;eN?M0qLPZSrOq`QSpqpRAy2}aY{x>8jYPnFN2kp!O9Fu4s{H2 z@Q(F}%!=@e@;&8d>)`9+80790>~81nc*5?a6)Xpv!>6s@u(m#Cb?S{1)~B5ud?N$B z!hIZlot^z$y@NdB5<}vXLcN3B-2z?0Q-VSg{oO;{+<;yoK2D)NE)jn2F`x*;{ZnFt zGvdOsVq5_00g)50RNqJxve?EPI%`8Yd81bU|?c%>%=WoLwBXNJ-eN1=E;} zELKifN|e2y+Zj(c=is1#_{8w^^r(!C6h>}(96tr3w) zZ~DIyq$m;g(mHQ6uNjPMdd%S1Fi&lcO@Q!PMW5mHmfno?GEVN8CV?v}YV#BG-1hh^ zkY;0aVrz0?ABbg{+nRpx90Zxxyf!wqIlKJr^zw7_!oF@C53V%E)rp0@#g+fpH2VzX zmu_NPJ-&fq|0c8-;)P`oB^PNRqY8zMjE>fnmZk0=M4Yd$;f4AsC>o?>?G61_lQ)7jbZ)e`o+P zHi+m$83xbdd~=I)uSdzn7+*}ICj%{Ul%))}%!^{#0|`0e!ecGvBPEes)R!vN0lie( z6}5g%t6$UT)=+h+pK5gLTK$Gbx1rRmW7LI~8m|DIKw`f|9IyuipVqM=r@$m3>fb@G zdLqYqBMC4hZ&ysjAb2|fo{De|^&d(=sbZPZg1|6p2SA$14yLPc$*?i_lC)8i5Og#y z;uBtKGGb*!xkmIE{`*U@O$DH}2C7S_Toq#pYgLXX$Lpm0j)|k#{slKpJQc%_n^3nQ zGHi)Qx1?mcidefPHth1XI|BXl-+pdLXWqyvxt?Csotoc9FFwz#zLHaVzP9b%md+m* zS6n)G=|?r~Z&x(k%qi)}DrzsNc(btjT7K2l%;K)l#B#r|+|0bjTvl6RW_eOZ8J$s0 z&n<_YVt#Q$PC*T`xVgBrt)~78D6mv|WnwzqOj%4;V>#x!wHB5(T|K&^Fl>>U;WpMqfLrjv9LEJ;YA{=8 zUuW6ZO%Y;E>{DWkdPD^_4I=EBVeWqwX800d_CLz;XS%8H3{&5|g6{Ubq}crA2zpFV zdRS;yWJFF(TpkFow5-yM%u*1q8Oh9~wBpo^vaHl%T2xMAaC%q_BPNxV>KhYa<8Eu~ z=j;^b;T-De=;vzdW@qbaciP47*jbyyC#{a0076(FIu1y)J8pOOb?cLFoU(RsvuCns zmDPm-VZL@A_5m?|LGeKzVP0O5fu4~eu8~2WabbaJaY5+`!Rd)1nF$f}#JJp~#Jr>^ zT69EaR3t6dD<;4$$iq1*#49zyHzO%HJ0mnZD~ygSGA|n#1^|mKpaH!?vl7Fz61*aV zPS~CO!5aXuV?Q`_><6zOef7xk*Nz@PbQCq2y@zL79-UUq2uqH#^Km`x1po{7iH-w+ z#nEVKj652%h|Vm1>!V*vHA_0T>rJHN8g$ko+*uMim^47d1HL(`NY!yA@m}ZK=ZDK zSTlfC2(VFP81PJkwi=-3uFANrhW(voTZzTPO$|Chc3>&THWgzVsxfTZBZtpeGo*&M zsYbCsjIwoLQ4vqV&|Xi$jFKZ1y-4K*q{9+b(WUI>e90nTI>!}GjPP_lJtKGT;#3s? z*!}y2Rd~d8FeD7m!}^Iw5!0k_{tA&_I6{R4_<{%{ePg7|cp&4jV{G`|UGCj`3NCL- zD8xx%6un>(N8$ni3^q4t(@;E8C?ClIVE8^+R_mUswNJD-QtpXT`}Aez2o+uoyJ-y> zsYSi}l`8On z4bRM^4=Dr4OB4aa{-K136UId-)X)+lB!Y~h7Zw};L79;;A9$vMBY@@D{wzg7fL31o z><}}(3_o|Iz(`^hWx6ei9vLPX+ZK;)i}YI(;-}|pcZG&;Kf6DjSNX$)oUW+M#t>9K%e= zC});;0mEXFi*lLu3}y{rDZ8MOQB+&ZZf$J7THAQFw6Z(5sF}fPVppFpt!S@pzEoP( z#wu?H9mXteEGuuXsySbTGo4$Q#SM&tn!Li=-2BSyJa%MUI-OAh082_Mh>6erSNT&GwDc%Um zui;?}Z^VLgFU6BeBERHvq?b&FZy}MG3^>Ix-}5{kp_ga@_8+jJ(ZVf`6b#A1+2A@G>7&WXzY22g~7%)sT zzNZ_1rUzn8?yD#FRTH~_E~MA^o_^*#0GM)OAHM^i)E&4fs+io^C`NpIQF>xAJ*gx+wTzx$UjW;S zkfhKvKK5q=otz^*oxpFMTN+WK`+IRJ^HCtd@RS)V#&b?UWa z)<>+Zk~8AVstSFAe9pKy*n2s<1bDcG`g%o&dBsF}#zy+5Bm|}>2c{*0go?;cj?7Jp z%1%toPf5s2jHE?H=K|6ayyGLBB161V5<+s)Lg?wCv`knLIrIodb_gv!wtya+4**LD zrzHg>#oBs%9JaQ3^^M~{c>VYf4jq5x5RmNX4-Owabo9g_E9)a?&iY1%CS<3FCPq8@ zd7kn1bPWj&N=!}2VZ_tvN%ZVgMowBDBPE;pqo4mtWmq9+nA^CgfIS@v+5mb9vF5%C z0Hdb!;UhWv0{JC}y9VAD*2#x$O-2&lC_fdoFtMKKFx=H5!`AUI4ChUd2t-t14^V`u z9?P+N>y^8G5_gv?Z5v)^udq6Za(+pvZ~8% z=ihE>y;8<*udM32aQWlT3n0Qi=XGTps}{6miA3En}D&I2j3bj z>l))4;0nG!x%6y&eqUo+QNw>?W^4ZG{~D&(3^Qxyg+0aCs?xM>nBJOt^qp>EQ!}vz zOKD!4dHn6f!ZWRTQ)Agun|AakyO%)Pr1MMEf zMz0|w0x)oyfC`&eFc)zheTmexHk@*aY$E_ebWz%ZDwI*Dm1w0Q@{72@sqUyHfoy>% zn&t_N0|O!wOWz9`jC3RqQIWpXfEBbIlEGm8JyTa-EtiVeO=Z>X%(AAUk|x+(78cclbfYoK3s|*z%<8P%(o}j; zSZsP$E+`3ROhR^4YXJ;MXG)(!|qA)9@At0C5{0b z+mylQYYaB`D`d_Tda0&vZFJj>;jw|!`;1;0z6`U-2OkZ-a9PL z>)Iany|&fqy$p32dhZ=(hBB04=pBR_sM?Y(xu!XZ6W=7)ac+8Z@};;*Nc29v&oGUS zD3VnW9aL=R-haNm&l#}q{rNf1`@9SY$uQ?#XYIZAT9Ll~;eLS;f$n}Dha7AV96NT% z#{S?jJHW8#kJucraX4V(gv-|6h2g=B4EFH%b@22AZshFG@QVrei;rfeCNa~JgEP}Z zvopilnem0WNhP^)g&A@A>1icd=_P6L+_?DsAccZZb34eo5;?I&&>`?&PdCv{QcjDff*Hyp1bh&OIO}``N})zF1%jf++NezE~q<|SJIH4TTxYi zp>Mz|@v?&BXc?Al#j& zy6Fwg)P{O;eH`)Z+4$tT9^Ih;fq+W^{vm+D(wb47s4)Xp0ei!476mYsWed2M%8clx zF>R7EBPtl-E3x2?{Y#>%Hvz(-f`cZ6QClf`qtriwMQx?%gPT^sz6oITMuIkpu!rvo zr3+)C$$@^^Cm(lq-n`k3lgPlL25xospwm3I!TLyiB?{$B)!DmTK zGg2{>*Y^zcV?N8+r=N*?2PcH+Lq87~CKg)(3`iJ?A?0^u3gBM%m1+R6dj!B%5Vz=z zwjTe#sP(?p0n$1NM~AqrbaRhL!T&FO*%qP{jt0YLN!|h>ro@ff3Shv`Nbvyw)ZQY6 z1h{I}Co_%9D6SgDK?TEC9VFSwT@BVzS zsP3)2s>_+Y6Qy;RO6y*(KK>fq013-0KULOvsigKoW$V@YQ?K)?I*My971zGRZ@ALb z{!U5lD>0c>5h=yovJ<%_tqECzbZ%u%VNH>swYIUNSkPEp)&wLh2eV;M@&qUHOIvaa z>)1tg#g)e!j$Z%(t7$%4T79yt=45eMbAC|`09ZDsEWfB45G#{gnwrB)&gNz07IE?` z(lU!;6S)<&7k=}Hfj|6t?C<|neeglo8?XK955Kw9`H8r{OF7u1?(b3Fye_`gDMR`t zo)L{rj15l?4Ne1q4fL9Z2Ta0Ivz!ioDHk;ACAl0BYaY-`Dp{0^7i5w}I3A6R&Pk+q zRG1~SD#s(piWsdo9t@4$C9G?yZ*XB?XmN04X;gy20&WJzJn*#J^auV#JJQsrj)`zx^D$u8;$(n#-wX7uM$&)NpbHnOTBNb}curK0l>6H6VfM z8SN9477@?OOcfO5)|8aCR#vuERkf8TapF8ez1*W1!CB#<+2KJ+fkAQp{t%dX_7Y+j(b2xAe(97vXdxsb8><>G;*t&Zhadrg);C_1KGt z?O(KYJmBE?BCtgCpv=H5H8PI;?(B>o)nGMJcI-;DJ|x^}|PU-3sY} zts;ieNGjBB+$HApuu#^4npY*h1<{m2ep@J?9Yzv1*E2Z#Y4^lmKO6teU)2Bc!SK~L z{@m31?dIcG&zya|viej}aZ|_nH(t5+{mWOr2LN`u{gvhumr5#6U$;n}!pecZfhK+)j3-tuU2e#bJqrL(Ne+<7wn`ICu-ZS(B< z#O#J~YC~^90Gs{%+4$^+#u8>Oo^(uvd7aQ+%^mHzt}DOXtXf-}@pg2^^wTcX1wBi2$b3-dAZb%Ui9- zrq^CT4B9-x|Dzu5jMwpM!1^I9wRLbFkIw|E5xQu$6HeCak-s8UB_ju9RE{+Bt%-%y zUDsN5gc~x*h`^>$uaYZv#Rz&mu(k||r^$Xdk^BtN8j;o*v0J9NFD4>;F^*=TzeWQyoxoC#^Ej?3G^HNRQ>%}z}8cx5(ufJGYe-Rj1?TOb~&wZ=3 z?mSSj()yPLO|RCS`nI6)wdkzs#GK076R+|cFUDk)CbI<`UIVYVxu)(+Sw#x~SY6A7 zvYPh%(pFfUqNdEe>g>Xrn&$KHdS%_|iuyCvE$0Bw*o9TOdF7corM!~5yuzxKjQq4* zUUF7`d>S_`w-`Q^mRXXR$}1>6@v{$Z{O0$=fBaDVv;Vwt`t&>R|Lh;1{&T3OQ_|n9 z=)NVp{;9aXS0j_m0EUT1&0`~z!vj-8{gXp|#<3wYKpFs;LU~)QTGXhP6tV@WXdbwi zNH{MQ&Iw0nfZ~f~%h)m1JS3{ohoho>XMd8eJL{-I2)gcR^Lt zWY=I$n`VC*w*kPEC_CIxO>E(T(v0O3YROG;3sH(dnhLg?ag&620M2czj9XGTAsM&O zg&CG{b4>S$mVd|KbYj@TCoMd9EsurcTf>_5=eYGnd9?+c%ADMajJ*2X-178{l9UX7 zI;Vz{SIZ5}2y+Yba*OZ@$%@UWD9o)b%WtYEZmy_kt*kj!nVc8z671?0=@pb29LWib z$O?`~V+BMrTmxMlyj*NuT#nj1A2{at{9*g&4?Dhi)agYVmltfD05e{+b2{wga?I7; z#?9lHyN5d~h?x}6N>2*PN(txWgt4=u^0`T6yu?y&LQ!@?QC56mMp{W`PFZGZVPbNA zQb=Niy|>#zXGf1vR%m8&BqtlYUj=#bMFp`uc0w`W87DF)!#_U8IndA6)BT8>3xJcI zyXRp?=jRU}d*L7kK)-nK7)t7G9S%A;9&~huf6_rm7kh@kGc(jBD8ef|E+{rNIyE;c zGl!X&5)hXXl$;%%TN0Tq;8vgg-9JX<`V}eKV6Wlfj$oBoOBMw%j3UJ@4@ ztWj@t%i)QY+Sq?x!GDBWZG7S&fT4l_4z`;6fLK!1GVn0eC93a`SVhbRlPeb_3IMRV z5y{-Z7y#JJ&EAQBcAEb9cim5aIr8lves=ooPpj)*X>GewU3-dG(r~8Zwac%3_v&ju z=s0(+<;10y(=S&xbQG1H$}evD;J2TsjrS3~Ca_>=G_M)WtF-Aw(=3sASvJfVp0Z&f z4jpw>|y&7n;dOx|vOK|7`x@SLTJsNWUyt07RvRV zNZ1O72IwEbub8F>Pz<0X1CuR@q#g&Ml%%zT0Jem&$ik_C0rkz!q0SqmG}zS-B&@G@ zsHdyH2a_$)9kCl0>4H%JBRYG8F0}3-Bg_N+)-mUsH@kXqCK`vG2YN?t+!+4kx@2&4 zN-UX^$`&M&+rYix2A3STxK-w(P~X>R@8gUYv9lw#Ko2li*BTr{aLcekZ*=h7LE?oh zbFce_6iS87FmyEol%i^vJ%tM%xA!Ix5W^INgrPq*zEM|Iq%nrQF%4FqNVpw&OYfO- z5yb~_rbfvkPB)^m1I3E>XsZkW3{eUZi|XISb(ix5 z=i;(!VlvC~Do$0kT}feACuWuvmbL0jm71yg4%X|b$bE-cwR|!UP)6~ z&6&EEi#1K>`Bf)Ns!vojc2qZC5Y%8AOHoN(dDZcv;yO-#Sy@$cK?!hrj*9$gh7l`rc1F>l$AB-uHg{(ck+!KOOGw6m@ka;}F9qU-jwjaMCwY4y_aZcXlSq7bRO7@k z01iC}UDiG{Rae2@)RLYw+{?$;sEi)Y917~)3=c!YCK4|rhKB)MDFE95z+l^2cBRHm zxp`YT@fcM(rX8SV&*jw@a;n&zimcp<^z5qi3_)sAVSH*y3a6TrU7j77$aD>5ctiyN z{VHfGFKDXBtK%2e^93#C6(`G+c!}O|-u|intgOK3yx?dyD&Q2WIBrZ0qi6=i%kx&F~BjVJ5`|r6;lwz_OzY zxQYDY)bf(JVxUtw@r7AH!qQ8#5x{tfsf8&KDNzpIt}i;++IYD$V!}eu3o!=(ERL5S zU!0f1FO12{@r#dk4Px4OdD?ip+cG@u7!22d00%Gc1Ge@r90CkG_WZ%4SlG04IOynf z(8=Y1gUcZ&cY6lYIVjSZ837pP5gg43jblV4Frt!tqEnekoUn}IsGRcPq(XM-@n8R` zPpWw!!2{+R9PY&0P1qF@%_gdQRY0@Wthj~dQfOwgH{(SFhgP{fP1ae(mY9AWON%5C zjy56opP1N?K+}ZJk#(u>5>3bFI1q+wSweIiV#V!I>D=JhT<;J9*vFmb4?oiX{I{a- zy?5)vm0#C3ywTEnrM>|-)#)>@U%2?KSFXNus{NIwwhJ}Q9VJz5JpM^R&DoD{jOk1d zb*4v{>}6Rum{tM9sO%7^7(yC4`C{T1>5##vHVl)S6LY&Wi@TP&_31@`mUR=!s@9qw zj8CrI{`}d*98k2ism0yddruAX8@kyw+XMG(DVH zd|0NDmyIK>L3b9xL zTg6ORmt`AIf{0$5sMi4KGHy`;Jysn_XGlY;h;$%KHI80i$i4Kd3KEP&s@^Zr#LUV>g8{ ztL<;^P%jD*twYVkbgCD>Nw4s;o06`6qfj^}!5~ez&PxEyq_@dLGaO~$U4eRfrRp9$ zgK!3UFISqTV-TY(QZ^)F4FXzN6{Kw~@(LmrX>s9)BqxEGr6Gn{+l2H}^oy;w6^x($e1(hR84!k-8+~BIImEs>W0>se+_jDzREftBbTkhU*-x<NN-NrGTP`-Ad<8(O zwCZ$zX>&F(FtQ3tno258@M}&MRUYS8pRTMs1JG4ncZy%pTv2tLlP?fdH5Zmt=M_{W zB(bw|`BoiA67NjM_y6nH13&-e;7{J~Zfbe!$}8{v@psoh{(H}@&qi-um-KYYMPp{M za6&jbIXY|^>NgGc8vD9+Lj&U?(S$@gBM0=7&%u&Q=Ok$CG7Y#jGBPzfI;~LNQRyC_ z&!_yJP<&@(Y1WERr^?AE&*fILvn#T*%F@$IQ&LKjVI>tNBp0RRR&i2_(iu_y zjsfnT5&o$qIYmun+2!1<(wu_o0)7j>^ms{Xd2(EFR9aEP{o)Xnvh_oAJ{5m$FRPj7oKZ|A@O-`Ger zg~~{c=CI=n@>5GoQu#%RMX1F{b5iZ##^p_|8DT0Kgm=O!uHDx1cCjW~4_*tXFt~S44t$WQuQeMo@BoSXyyJW@%_@ zp>J4Pd^-O}@Bdv2+(?6Y*kU--X*MZ|Z^#$L^4k*C!iZ#MU~IZ~Xc_?Qvs;$G zePa0ShpHdF-+%2}|6g<4_nKR;H8!3r_@|!pMx^DIm;?+kZF1tx^B^e0oBlv{l%cw@&Lof0l z));$x0K69^PoM2a&q7D@hO3BPJDG8GKWm2U;fdGz9ub?Dk zzidbliz*UnU6Fpz{X!!C5t0jm(GVyJCQ~cM9VzyzHvzh2=1HC-0eT;LYB=E3u>rlYLkLPg6btG4QKh)CoAeYsv0_~>d%zdoZwfq0DzU1H&@lQl?dvya(P*~#eiP05>mJk z(K#h$9q<35>-}H!{p4piTaSP1?Ah=B=9eG+<1e>9`)J_CCu0M>YKhPy9*XZQXka=*>0t`%rX1@2~41^&D;=5y`OBl8? zdK=b|aA8=q0B>>`K9}e&ux1ZpUzT3UFs7hK8ku`$qeQr-rfx z*?Bb_PJIriIjiIhxBPq||13w)!Oj%K1jKr<;{D^Z2;CRs8*^w0(o|BbQ0)Uqnl9}Wg6>8(_eay?t z-p9v*;RER9%=Clh=*O`2a61TpyA3M)pFd^?|As@3&W9YG5M`ZVc{un40f6CR#dwA# zdWI)?M<)5kqy@y~1g8~7WR-+x7BdraeZ!NTyh9kQ#EVzo>mQq!pqUMZwn;Tae@Nz( z71|Aj1|6Dd$9o^Q=%NhFy5C4Y3EumAb9(Mc(Srj8-1ziV-W4mB z5x|tU$7BFtlf6Te-2+pdJ(K_V)byvnY5(&#!nc3$Nn87SO~>DCIexjS>Qrm%<(DtN zbM>_!p6v4dZ6qLAPEMd(}%c|D=U}|C8JhP!Ttmw^~>Wh(g4KgO39DbK2a3fz< zklaEtr~FV23=C(PSG2lS4e})9TWV4*+~Zr;3L&WiZE420)ui1;QH}mcDN0ee0!Z7# zwrx^i#4q^9XYr5!Z&rs_1#wCvP=|O%i|2F;&`&9_0Eq00f)K4keEp$vMGnA5q#RTO zve#Fm`C+(O=x!niXKjf|Rgc7~6(RgV_~v9XOomgKTVgFPp>BIr3u|Xozw_R2$Jm0a znZ@VxNZ>NN;zF*VJ%!s)RN29+XfLb1P+Zkf)%0>n^;r)8xS;70zwSb5?S<65W={F} z=!}~9+}f;?*23zJm<)b&N>l{b}Ewll-YfkR|z)$t-hQ(8_5JHLXikoDstHTjO?PA zgzVt3w5;6vAN=^EpT2+VM?e0wsp+i~ZEyYTCm;UhPd9*sb$%us8qf(xO`;LYaKCY| zcO1~Gzgq`4W5Z^tWJW2UM`I50taxlnI5IidZ|v_k4G&Mt<#&}zjA=y^GVz@;;9giW zoE;dMgSWIuen$eIC531zYobf}jTSQ5JL~i)i9xbtG*df)LMk+lB6e8o{6uNSZ z!*db?OED|afR#hIp~@_DlR_&gG(sd^h(NiXXr`X%rk`qNFkMAE_f$EBpVH2K33&Eg zYH?~*b_A?=ZfsOeM09R+3_CtNGdd_a+%w$Y(cj(G-@}9D?GfzhBUjme84bP62~_p$=Nsbh_kP)TTo@=%O7>=;rAg@ zW8aGi=mE60mAWm38lcOn*OF`Ygz|gzYyU$lnC)$5f^Z03^h((nsk+=JhMTCNmniR` zdywkBSPlTT001VE&yI*D`$r~vho-vvrar!I{_r2V|N5iso&UIb{_?My+rHg$;%Zf0 zdsFMBE3dxu`kU{bJ^y;k$xAJ5FP910*oD=<`0d9!(|xV+fx*0Do>&L;GMY9GmTjXM z4dy8!BSBv%qX%F!ST>B4o5raPt?A)7K-AK%acY$?uuZ*bLuXtyPOjbl{F!n3k!fai zX8Exm`PSOR+>Ul~O>J2*O>fM9{^i{Lug2##RTelMKhgoWo7yr=Z)?pPK)EanPv#%| zYkU?Eh>VO7rG^Uk#+KfQ7^Wqo%2-#NL_ZE{E~Q8GA{s>t4oIrdiE0;-i~tzW3v4x$ z2d>(%jY3evx*9cyt8^dIlF0s#eeZ{FMK2=Xz&;bPbw}JH3(Y3RWOs(e7%n<6JlWe1 z05l%9270;xsrox_+$1?+R>KWcXY>yc9S17rDG8$~V%-|QOu^oF#z-#9WhBSQ2-027PnsF+?XUKB~@VJQ@M;CZA_e6Cd9*8=x~H-w0AQ46kynuH(lY^tF8B#k^diS?Ysa2av~M68GFn=n z1W=?It=~MxeX|tt&GVds5VF0}B35Wo@nLO)SsP$NId;gXpb>u;`=-ML$;dvU7={1| z8%m^J5o5R|hKiww06LVei!>WT%?6;CNdFiBOla8s-N%N^;>!u_QwiMTDTOD2gmEg* zWfmPTtT@A~?5JqHQdD~mSM^zTXuo6q+vMbZ`8{*Q- z;?s+=3n~&b3pl*0w473Y^{JBTQ~4zgY1yURqIx#3kz3rHTi5`rkl$8W)!x{6QBZM` zCurgF>+|^y@JF(^jfe-+lL^ zrlz-BTHbi)JHPtvum1VhKi>cV8yhxBgp&YZBLhZ?UIf7OqEU-nI;)V)0eVS=v!aoy z;eK;}k8xyVS|+`th99H+bE*7up>TP0bZKPx_Tb>`;1Hsh1iNBzV#A0_^v2w)Pf!6A zZ5Y$?JG?94ErcH2q)dWwMW%K2#Flh?mAXA>EW0Yx4jdyCB$orDISAz<5%ycu)fasc zExSsTFFsaH?o!HyLtwMd^fOPiu!uK@X7-tC`WcyM{_?pvPHa$8U_iWYct&U(I|f5l z(jo#9LVRObj3}m0M4)e&zi+socZjD)kgI!;b6`S1GA}8uG$~aO!);D1X-_S`m{IpC z`{Y}^y7Rda8NPl|3`VGrCyU|1Vz>tTxP}IKMnrgoMLPHe9&&a);NW=7$qg_J*w6D0 zjt5;_9K0DWfdRhJk&!uBiG>BJr6qt~aRuy%oV2j4F(-kS6JMB-RGgk( zniZB7?d0cS>*;3e>1pfjW#a>X5i2A)!6!P@HQ3j|&(q%5%fTOReBD?~_mDuhU_X~2 zhFcKB-rMZ}(LlaH7v4JnfE{&nJ>=r@f}^XASAZ)s!W{uD!8;t#D}xc0;TxUpAIAyH zD2>eGhi8_sk~u!%i7vh&j^05wZvHl|L8&DL|xpKD(^~=co7|k3Lc=#B4A*Od5LURD4re~ zo9Y>yywx{#{nq5)J~sUJL*;uP47~c*Us})nu=(U0H4W#EpS*nS^&eh){RbB=z1eo^ zN`2FLUTI@P>xEkbO0DU>#(3XgdT5?lGg~&zmTiLt=e>-$tg;Rh@uxRVZ0d~b=4lN1 z)R^uAM9l(3O#rWgGdpIi0Fqf|x90DBp|?DoTG*OddaO1)Fivew%x$a9E4rz*ndK*Q z_nw)Tb~F=fC=|rMYX|Uh+zb%4u7$sDes|{cuf}J#RpxayC0mmi&5S#QV3ifpP=AO` zFHBTHf2m!aWtX7Wj&gEaIkl~vAg*h;owihaDk&`n2%s($n>U6vCE0}ZbO}Fd6nBu> zCL%h(SN0Ti_|PP*izp9?n-scopgq#^C`K4B4vxI_NokQQmJcGVP|NZ>Weye13H{>@lv+($1B(cI%(mvUT!UR32lUPnm?#V~qAqi=HS zn;%P>w4;XZp!;uo_)(Leyj5_Bjj_GpX7Zs`{W9-!aX`2(GOW?atWv&ZmRRjc0$JQ+k(15GpFer>YE3Pc5V`yYUU>_r zur@9wFSoENja^z$R>v)_;FZ?qa>{^&C8gzugr!Bq7Ia*A@4cUV_WkewUZB$Y}e5AH^yw>2@2Zukb z8REhl6`9sRi4ZjkFm0fO3Kdn`3ZP#`pjqhED}(P~-llLx7oZpM^W8#_Lw(j`t>v+P z`l)H*U#h9c+W9ZlaGlyUEPRD{HuDUfIHsRzX1@XqdybXD@`~_q3vu*|@s3Cf4NVDQ zCWZRMvb>{%0u#fS384W|L4lFXpr`<5v|nJXUsO(H20tyUJcV7K%xOs|YEKkgNUgn+ zb^2Sp6W4gTCXld% z{Pe(ta0h=cJ8usMZ!ZU5KYKr4TOV(Ge{ZKCANNo{C#JVUfVUIV*DcuJJv6{A#1EE7 zsJ}bQ*Ur=BkRyPW-HX7(>>XdQb3EYSbcm2JAae(tU5|PBIR}KfGNV0$lRQFGy&}_n zW3zo?vI7&ifL_tr0st^(BAXGG003t18EEej;NZmq7-L0ezWsyWi2zh|Sg%m%;o-P< zy40=8$YdxXVTfD*F9V|(MymNZh=b8$+D+Wds{7PX)!K9c3{%~eplte%2%t-`JSJZd z$>zn98JT=eB%2))PW25u1xE4jumHmbh9`RaRJU#ob>HeCMMJC&TICqsn6a`CT39pe z8=ytPeqxY_1P!N|2M79wVF8CDpS{`Ba|``E`i8FG7`fi593Gh#3NciLs5urz;>FRi zIqZ#zFtku4S&&H<*WcVynuKgF0dtnqW`Y#}c?Nt;hN)aT(n1y`|qu53dYDB?^ zbww)!+i(36gV6o9*4Hw6K(9TWJt-$rghOT~19qiY6Uorfh_aUd=7*;8dI~tb3`f^(w#V zGN4yQ)8*2tb9wyZ6%Ch~Pk+1i#9M_mmoiJwu#)((xz*Vvt+_?bncUiRc12QVQF=~E zVtN6WS5sEeT3Fh^D{INlt1YWOms{AxD?iC8YN~I!T-$u9^~5XX6{kunju-G7%4<%R zSGJK7Vl}V4QBd2)DXPjZuFhr`XJ+$au^MfOELnH7;n#XZN`Jzm9hXm?AQ0s{nn)EUN$x!#SS8~Ub zAE9Evz)-eB3e@mkLPAgCBj6aMG2_q+*8eSlUaN3pn0cbcGw3d+9ZfvO>*QnI^i%4y zqs5G*XNKu#@QQB!OAJz(+}6)MQ%wPSeL>@x6&QE<6jg|fNH32tSFbQv-w03tC~sy= zKu}_cPke}1Y)DXYWO!OkNJ3a>d{{^#|8 zx$X6WjK zviI#0D9B3YWhEA*$Ffs= zqFDe@whS+OhL5d}kDZ^djW6Svucuvrms_xpXShEwKNlo_er{p@fM-CzJVOJ#fSh?b z9dmIw?CfyB5tidYM^y6zrXF>2Ip*Sa5Ez(;pNoH(8#B_473Uh9=oy*rAD`H0FA#6yR96`1;G4D}Xb1vuPP)Ow!w&fBGuUs zcLTTUln)J0qiKQ=)f__MZE~F-8^h^lU}GY*RG>yv3v%U>Ob+X=T#e{Og_L^$wm|?4 zJMeo|K^)>HS4zJSz(_)k6~O2{K-xYnvsM?0MY5XkTEs1aUMjRL-=Y&^gpeT*qqs$} z%}T=PVUCF1i9iF7`LuoxhdHX|P>>E8q8AwgBd)%W2>n8PDy!5UXQJV=t70YEBvMT; zMM3iYu`?~hDdsI4)*MGW#8IGMx+lZxU7_xaZ@<@<#OX+2x8?B9=2yMMtG&W+x|UIV zGFQ-E*7B;Lc|mb)Hdnwcs^bb2jswf&%} z>Fu)@-hKCnfB5i^oxNSsu@S@Aka47c9Eexf4Ml&qT7<*Pvp~NH_nI6Xv<&td2m1`d zu_={mSuR_YN|rI`a%^E>aAtU9eoTmGAemxGp{D2s4D6ngG`&b`^O0(AzL2)WaLec) zsqkq)`_Nct#&MU)UBlF46bVo6qKE?ysWVUUe7QHsOjy@0n$TMaj7>fP=+e(V1N4Fq z0lk2k>8GD5CwHZmZSC9_DokhI)y{t<#SYnyYWgY3Soz{P?{IgYNKgN0MnE(J04yXv zFft>WnHt523k^()jLlApN{wN~2K$8g1V#sD6lEmyk^>X{qH;qr%j454V^Rbm8I=+F z&55-aIVZ0Qs!tUFfU)9Oei2O1ut4vqP@l+1kC0G1hR+dKpj^%XU;rD3904|5Vd3ns zpO0r~ut#`kKyqSeX1ZTO94tm$q<3@}5ICRc(4gd)=-kwVy!1qFYFtiIXj+VGu%E53 zmo3BF&d1l5;bY_LYZu_>!1Q$p0^DMFhWUC$`nyN?yM+0=g!s6yd|ZQly+Z>$gBbRn zPPQISN8KC`x;PyG^m2AR>f(CX(fOF8%P}YSLk_Mst_)|NV7I_XkKlNZuvG7;Oy8Jn z|2TF?8ZRQ7k1MMrIGN)S9OK}{qyT2)6=>%b2p`(H1vq(y9B+T4e{_--4ADVKPpqME zp}z(esYsoQG@q^LMT>`fbw#YAz+NQ>)eWVF7^lJ_&Fa0p!3R=I0I)z$znr^EVtbZ^ z7^^WSmP}*ZhG=qdc%pA``eu*i;~Vl3-+TGZKR2Cv2M&O(r?0;9+7HiN ze4`O_V^8FlG*s4ieB3G4njUD44>X4R2Ga^zILmCn*yS}7E&woioNemR!*@$>TGtv@ z#{qlDEHkh%y$RJE(+iI$=XdbBu&bY3RU1}J)7!JlPj%)My#@1DrWVmD~$p+h7dgAj-0gR6K6B`bAWgDZ~ z$OM>*j7ic)F}`S6mmAg;gt`H4DFDM@H`oH^+iR;>0gOTyyh1!9Fd}43d#H2i@n}d`v=TpqJ@#sIRGt@=r*}hdyaWHR*MOMp+He~TP|Oa z%9b!c8pB_xeiA8;O1G*2uCF2aDO&7-?H!ycj9CFpZ{7j8!r8lTz!=3Xied1&j;K5| zv{gpDl4+54e-WmI=Yu`DqqTnc&F&Z39rVqQ07iZnUmxr!4}&kJhgSnPFQzzJon5&EsL!?x^PJA3vVp2wqFz zoXjmdpIzDxFjiD|C7pM&tod3++iSqU%9}42SD(u+Xv!^UDy{4&ue(s$d?mlUJ*)Ig zLS73aiW{6%m{Z&ooy-dk&rZoI%`2`2^vcL7si-+o$cJ5aCcm^fEw?Hyr!tFE3+PqA zZ{rrXWaibNJqNd{Owf{7Sd~{)!!D@IE37W7XyKPP@dbctW!wTmUSWA+GAA@FIe?Xt zm{Il8l@Bgo`BiJn_gk92({c9Qx8C^R55N7izehGYs2}Oq_uW$6_(Y8Kt4}8rqj!`< zMA6GU&@EAJ1A@G#>#IeL=vvV~+Gk%B+m>!%*8iEl6L;&DkHBQPZ0 zH<}ZgTpGu&OD}B5=2T=P=Ej7?hq7Wr{h~vBqQik**?BXLxVmG((Afn&FKq1(*gK*E z(8<}x)5AH)-zOUWx@hm1D8Kkv|F~FYN+L5g!7nk!KOvHt7{f}Ajm}9*%FjsTq{d|@ z`b079eBEt*JZ-$a9en+r0)m{FL5@L8mk_3VM37skpHDQCnH1(1$MTL2bPx0M3=04# z^#U{v4RG@HIOgGe#NFkHr~6@7R~r|1TNn4E_RhABZuTx-$DBQmxq91s2e}4@d$3}? zXb&thGa#N5npzZ;EizqNKZ*41}U94=E(B;5k~2!>TgnU5R{IEL#%E zmPf_6$HX&2@st(7MuCA%-Rw1g(y9OZC%WJLRr%9j4!{1rkJ`?@SKsp5xtG58?RS2D z;nJJ6^``}utr@xe>ZbF3!{d7M3J!r8AB-Cx!lFZ9deiE-Y29GjFq$^-I`eD0}?iMd&hv&T`Nk%J(b};K-$dGv#A9Dr2Fvt{GBfVy$sW9YSVr5+@^UB0BIdV zCaq224Lxuy+S39&BXEUR5=4pR2!uEIa7zt~^uUmiS$1%8dEcl8Me{o<^Nt!v`Z2zN zMi9|mTa}NmVfUDTnGz2f_;|&GKh7_&(jwb8ONbKUKo3_Tdd_0>Fj^lG?>@L0 zl`ieq4@UvdFr#^*zhCj`C%3xM$rq)E=zU0R>1qCQHx4P2d1m~NfD_gT5$&j$zOM(P zoPoCyy<%T?7nvUG8R+R9?Yt%K>Kz{z-X0askB-g?g|nlhIL$*7FEJ!cjAPDHDWaDQ zC!Lq1vO7}w=W;AZ!<*-!bp%ys%}+r~U*asKGj8I--w3Jv;Bfagg|2-7L&VxcFM8Ed z&@y4b5Y@<7{r)M8S@j)Tn1zj!nXk2o6u{_`Q4ec;hhv21{uiL#+TOm{2Q&Hq798Rt zDh~>5fMH@%LZn}Ye-I9 zm$sEwohh&HC@5*Dsy$QJaJHttqqL%hlUGqyb3!0!s;W9(SX9N%FU!d-&SjUz#AOAA zq(mnawVnRnOILo`()vzQ)3-X#{^<2>$A$pC^xd7R!G426c3VlDIAlQC#mL}A!a2B+%NJGZWfcK1vOZTK3A?Y>uPDht zB%JWn&`bbTRM;njO$~c%nC~hiB69UPl^nV3-gO)xdjE-cz7Ie1grwe<#Na#W%SS6*d26m0`xlK z=IZG0;~f*hOicvP3QkXrV&}wj*ioFUa85>WR!VS2QbcBIL}qeiR$>A-HJz88R*)GO z8*IyPJ%;1S3`f5J7iO?)P_Q#A$Ri@eCpOF{h82_)9+4Fnm=NL@8{``u6c8H{5F5&f zWcfyhx-fkYd$=9+a69VbZO`zr_waUb^>B3Ybae4@aAnxJc^`H4vh@md@(XceMtOuJ zdWEO^L}dlUu|v~JVmVbYoXYU5;=uSEH)f=hPq33W%hro|%q!rSX8-`0jjOMXi!Y#> zdq8ACSX-v7lDVQ1tMS= z;1(>rF|Fy$tCrbq;9gquiq?XWmBwi_;Fwu_0sy8p-=Cb{Ub^?+rkO3>#F~C`%`(4j zoY~Z15t7QXQO-_NE0Io-cx6Jru%bvca@Maz=t8-zibS&E*gx+Y3Yapg)z8CPdpsC!(S*`GZ3qCH5qxN%L-unb?IIpi=5~Q)p{IwP!Vsv z{VzktM!Z5Ob))DdCpvpfAR~pu{XsFDkJiA~tYWgFZd0Nq^P1`niGE8&f-Z&H?NRm4 zsCH*qvoomPz50Xe37qzn{8LHX)|C9V;`&!h8?P1By;5`L+wAg=(#Drc>dt2uwWM)s z)4A2`qNWs1Z9(OE0I-ar(+TXR@YK@i^y19?>P&7`LOQAta|YrhGo|_Uc2zpe>S#!yP@%|GpB#}_M5-_>xVri;-CDVHUaeeK$boy!OUX3hm~d7inpesfm8wOx zb{U{cqx&3|T6<5ey-#e(uv$p!hpQUns@6o_Ehaqf=t=E_EZm+{{sT8yqc)(vLuc7l z7~mWU2hGPSBgP@A%$q>VbO3c0G@)0KG)vvYV?Zys!OSqrj(QSfEwwXGHB*mO(~q@t zUyd*Qk74n@P0RlSpk-M43a-ZGf5FXjaXBF&DFNZxAyIh|QTgE^IZXd#fA45tw@`0q zmbX`AaA16-TM)y^-!(ETDl9A9E6U3=($go}H!z+VmKq$D9TA!l>7N+xpAg1M4hv6( z6&{)#7Mc|9AHj0+@ghwolt>(LbUtkBaF8&tBQ7q-+}&-wyxc-rL1{^$nHeD&nZfDl z@%edKW&F52PFPNQBsVK6FFQOZm6aMBloS!38l8kCL_kx2TW@DuhKHk{zmq@9$)D-q z>*wU}?-9!Ki4OIP4e^f)_KRgPq5^zln7%PV{&9e;QSg7^9~I`t3OwrJan#$}#^2u| zAOHm*9$roa#_U}^(QC@t-Nwz&!6(=StB46c;c326sG$$fc4byE4svz@&Yw+CwQqu;}0y zsh^-Ij&#ecc&0=CH|yXHZNz?2@knL7uQfk{H8Hnqo`G}vnhL%@0NC7?*0h3Ql0*N*(c8M%@i+ZDhp6i=I4a6_LUi?K_GvdF!5(bi$C05m^Ufm~;`SF!sR2 zOF~4l*}-95SJz1A4Iov0)=W$EO*he?miNyg5(UnIe?!5Tf5nHtFsG}!+Hk$ zdxr>yVLaIN&u;Y*V_%XDHqhHA?(9-_4@`|h$Ty6 z2>~zwT`5)(X*9Fa>Mupg0~(B3fya~1>Sc^W1sWoWCS5UXZsAdYK-BnNEs>Hi`a=eY z(qIf5iO0iaSDZ+tCk!CLcxc$`dtv?1fB=Tm3)uQXf`#Lcq(p(%dHM~iV4_~YWG0#_ zc)#8(w4F=|*&g4be!i4`S?9y(nFA}pWIBaJh@r^7M^q%<4D z>g`e87eD@uB!~ZM1`hzKJ&xU!R&t~)R$tI;`f7r7)^@Y{`EyCc*i<3Jjm!EkyzVI(VECXC|ePvqumvQkcWTtntHzS%6nidw56%(Bq9-S2tl@%9~8XKA(AC?{yo)#9B5fPS(0LF|Cb@TVL zadSWB>UPw{?T~}hVS6VkL9}uAu=Dml=H=-T780D65}J_-02Yvx9G01p%HzdxIT1No zQJfrD%+v%%Oc*l}{-XHktT<+Tut$)$qo12ofRB@ZfITD7-Y3AxKhPtXV)?Q}j@7DR;rx&gr{#x)bAw}(L!%ahmUwZ(yK5kgoyr z&;r%?D#F+vfT_$V(@^7k9s9sYxhAmtWqoLxQA&Vz5|}@P3SHPgKnV7x{Sm-$@XN5S zFm1@j*U7xO^++K>k>`CWwix$Ei}2+SSVANrDZPDUE(|cK>sAkeE-WE--$MG;Pcf`-sJCaZ>lT6> z1u#OwMsM7b+~_t8j@=$c@gatU;Y71+ULu>9$`=5xMB+uUWLYX*lFOF~bloFSVK~~1 z_8fZ-K;*geNNZRnCcgx?;4ncV*2qzsT5};j=`D|GGwho^Fp6zT!zSKa9+R36{;1WY zatQbbpTge)>3E@D-$0a~IMS#mD5=oBn&)g{64HwCS4@M>P>mbF*3yi#-Ob$-)Jyqb=}s`ecI$=vd@?1~Fn z{En!M>X4)oc3E3t)v4&z!mNU-v|N6EaeYBieMwnMX~prpk|s82(TfWoM@?8M1H{STg$NwB2?$`8pt9k*zI_3R68sVr#Dw>uGr-Z}i z-fnI8E%oTIMI~F1OBVpQWYT$>>n)Yc0r%1=g# zZp5`0%|q?Mu&9Uw@mi+)9o+Ki~lk&{*P(te+&!%t(|?QpMPdp_)B)3HO2c88s9o%~z_5?Nkx{+^NE zJ~52A+}LzpT3k+aTy{)cc0y!EQdn9-SbA(^Mq~mzCN?`dDlIxVF47|~(81Ho&fUY- z-Q%c>D_Zxu!Ht`}m$$8#mkqjt6yWOT3m8`B7n9-S7w+gAV&@%f>q)AJ9{xw& z{p=Y5F8)kcW{^9JA}A&vj9tL8q0>tysk1o)K9LOXMt?3YD~z$D1gz0Yz*s%_^{JsfHJy0 zVIgAArAAHqVYKOjy{?IE%@hgBGwr}bR!fC&R1rsrmx|g$;KOuFf$pMMiw7WtdzoQf zMxB%?fTc2k)ldb8jK~q;6$)U)M`~|)774L+EKj1q)NBMc`CZg;h)~BdIH0_7 zy$?tjH8v#U$pihp{rx@tK)(RG2Ko@s;6)TDcJ=n69I>mnyBn}-XkY{Z3`HEU2Jt(P zJWRksK((7)qJMrW>lv6H9GM>(LqAs1@md7>C6?eom`t_|>>REN1xW}~--oM0d0(v| zj)=tOk^bUH(B-3j07K;=iVXKKi~yK^ukS@!m+l(?rZR4$w$p$iHA*6rQ4xC%J%Ku8 zV^p9>=Ad=(P(uq!92g=>4Fpj-PfcxTs9uqtJ7k>62lh!s^`d z2(|Sjc!kbEQyzxfO1DnBVo1WITH;o!-Vp1y#N*olVB+zeG0hhGap<00AGz0b?gMuD zE8OZUh4rrjdKEXmTH5@&p!tom`m5a1c7EN<)hAvrs=vq)oGPk5&la3XF98fY&#Al^ zo7>2YFOJWxEUY}05B#jWjl-+u7S-`e8cX;s1*MIcw^wnhNN}1{)KXgAkq-5j)1LNcixcL>iY<_ljQDj82pKo+# zR#kJ`+jWg^H8y|ybo;yC`qr;M{di<#Kr_&-?7AWCz9}E*(~3qdlCeqAuw}4Uf2&i` z->a8Ptrm!4vc`l{uw=41m2y#~TvDszT%>%XOzE1y0OCsgehE>%# zst_qwQLPP5eGzL-_)tRvchDyq0E~!$be3(y0zEk9zxe;Tdhhr)uPa;Blj0(YA}Ll7 zh3LH!Bv=6g1POw@Q>3Wgt83yU6DQ+0xsyASDaqV7lOCrywrmylB3ATX*d$7nD5@8B zH9N_@?~k|l`9LM_>34oG2!fP_4>;?rz1LnVGXG_8?uGC2^T5?Vd1jtFrk;8(f8m~a z;fOwY#oAF^z28`S*jRtm)Ns^Xz0XwKkf$t?vK7oEcJkKr#5a-?UQgb>4K{WKhpy%V zfN`=i)dq#CNG{RwB|4!zPo*x(ljmp4@_~~nb!JVrNu6cT2sBa}pPRyB?x53?n9Kww zBZ0-l&JzR36DNtwP7`r6l~O^rMpjs$(Cg)70f1>sN;D-VRf$oeFOnFFbX8>qRV508 zR#vFyXvB%^jCgulQYJGVekY4y7@HeMqklai;qSJ_ePvtRSGL6c-RoQb?v1TqPuRYL zMN8wcQuyq20Xvs;e(@t}Fb> zxxhz%@cgfzj{noYcE9tjpC3E@&u_l_PkRoWX=y)P*S!B5-}#>tj@fYZZZLW`G z2cfBlzR>+(^fB?&SV7VJa+6T#n`0Me)Nqi}Z zr2EmP5h4LFFVbQ_FE`+p=eC2)m18OxVPF%E#nG`Vqa)#-t`Q(%2tNQ+y}+~vNJb^l zERuIgQkbbaAy6xvcP25-z1_Wo!~z&yfcwzccW@AcF_E{S#$%|r-~Q=2`}yw3r7=t( z1j^-bF1T=_nV8c%Z+g9VJnmaw9}a!31yupW5KO=xI=`u2~$P!-Am>t*p zCO%OB1}KFaP!QW2q%aHImkiROe1kL>edJ4cozWR6yh3R=C|x3&ED-N5QbNQ=8d2M? zk=ZY-M^ONVO83wzuPbH(K#hn|MMa8- zkzcapKy&R?2jG@})g9V!Vj*$O5#De_pW0B$u|Z8V_c|V*`R*U`t*0v5zFpq>p1JXK zaqY2+w)d)bzE{)wZe_#in$|ZP_kN?I?Mzw2v9gBaWsN5-EvGBm-mx^lm2KWD(bcQ< zjl~ta%c}PQ*~%-fF_t$02}3V=OH+BxZgWLPfuT`vZq@1Q4P|Y4#f?Sg)?!P0aak*- zrR0_tn`)|SJIrN`0AL0BazjZ~iKWg|R%0+#S**?G()xk|3orvNSFKQ%w6?$3zWaln z9p5`}=pR4)@E2X3_K_j~K#!~EqNA_JGdetpi7KNr!+>5FJe?PuVMpGW3U;t{}suovFxr=IDjZa4|SQEBFxXZD3Hx&g~M^~?q*kKmd;{N$CYUB%5u zaoBw4X=~f5>YBY~tx1+Cr_)8~AeF?niDpYY4f;28OiIuT~NlVFOCUG+pfHnc|5eZ?7r%411 znUJHFWGaPRja;0k0~k{p3`&y`pi5yiNDW1DW3jfPEU(s*TT!eryV-$Iw`k~zr00KMYr!tEKntu*%53}zCK2cQN2E}B$GmjQ*9GsIG+NX!&T zm_kKE+wo5?xTwjMmjH$?u|b^iiuo16nK5?2N5PmEc2AO@1kg2Rhi&mTDJMP}w-ZPs z05&qQaB2KH1z`O{bG?JJJ%dw&BU6LJk*@yW#h%b-ouS`;;`^^(+WzGy1K;}a_ov_c z@tJpj*xGSy=k8;zdrti5Cm(qOH-h2Y0h|N78=SmH)Esase;9~7BG3hgtR*<4N21Hf zz{s=^?srq@c^HiKPOuanoO*QS=Cj!wPXp5r15?ZX$tAcyyRbevzdSV$XR+nL>CCE z>+Z$~mEPE&9?Y*CM2Fwrp241;{_d_G#5RPnK13`8xIxt6zQAIBWqJU&L z*+bg$cvhe^gfS9^Ey*WnB84+y8zhMhox1S#Co~I976`xqp5ZG`wauvdVPwrsjPEE| z3Xt(HUxW-hQ}<)Yv`pqEmw_i?Cz9$1Z+3$5qELE8f?kwsC}5Z)v}6xrc@UrJUvd+* zApjVVFdOmzC4vsL=fG~6AHKC>!?o%PqQAB)gcvsAU!Cx;j|ZNP1vf^5>%)Py5C3DJ zr2ZQ<9Us(ne8*ICyr|+(N&QJ{%UiXr@76TGS=aV<{qA>L4}QD6@kCY2n|1B)S{hGQ zG@mK0KV@z>tte=g=hjp;9nUj1=M>fG%NmU3ji&M@YxU0JvIeuYwX$|!k)aXzR-UoR zWbLS^KLqs4QnSxsZmOx@V=ikbD{pFSIoR5Eq_%#y(F|bLT3HR>YAY*mv{tslclCu< zZC0^Ftdocft80(#-1YZ6cYSZqfe-%y0PLc@zuVE*Y43qMJszN6w(;5F!ANhX_u@I% zV1Ll%y5{oC!>!Xc=dxd=oTa?(>wfQo*R$va1_rnVZ;}8}9^#<~`Fb$86bdiFr`{jS zA|Qs>n`X`U0r5Y=Neogpq)wUG3!px>a8iL!+{_xqx5C$+O&H#x~gWlkn^_S9H}gweFvN=7?f=8?5o*x-0U;5n3M)u8v36VU6M(`no;( z)E;{JO5HwF`^n0VGu7>_AP2n$OAR zD~shSy_Bb7()h`uEI~o3K1-jcD$ptOG|GHnV@i!)CCinwB-}I(P#oqCRtDfix|o+D z;wA~XDPlfNE}}{K3>lxL6lTb{8B#7kM2KWUTwXu|id^#tA z#!X@fQ`nL;p@JpP;bs|lIcC5xX+gO(zmzW3CS;1Yv#vtLiv=!Ql;|I}|FI$q@#7U1&D{5Y3)_4?O;d z=pE&Ih;nD3Yw#iT+=JzC-L*S!!*ZbS%U#02?o2rDjG+Y|GO+$(3hQ3}5UGe0nkP(H~tu`_06U|NGJhKRo;1cYk{3?H_dPJ=wnJ_})YB{QOrJ{Gmkv zuu$|~D0&|jcEU&~b7&G^>=E!V05EKmOfE-IH;?EQ!m$rHvs1lX#1$WGf|HLfFFXy+ zuKFjJ15=MfmzS>IdEuYEKQ+HPc@1#oVPtNJ$mmyt(@XH>^wqWNnAf}pZ+A+-P4wKW zL@Q5;2x7f4+)1gum3Sc%>zdf`EEyN0c3uQ3Y`4J9IEjvGSX-Y$et_sTg%Sh&GA69Y zJW|Sy%@f8xoDcsA#1#TPa+Yyck5qc8a;}Gn63IfV6z8T{AI1WB@47J-07w|N!BAN5 zxIJoHAOIHb@3o)(A_nXvB8-tMm6 z!9fgOA!hcd>>!C^*d*)kpSaLv|KsPb{^9Fm6F2P+AYL~duA3h3Z7&cn&!We@02mew z-VXv|;g*Dw=s@^BoJ;%>64p%3@nEbaUR-QvlE#Lgvfm{Kg%Dw20ra)Q6dPb*2w&dd1fVYLvRiBH2BbFAOOYrgKrX5_O#^rIiq!}=irSYUlT^z2ukPc>Z2t!Cp3mOMSfkeb+fl(TD_~HZclCF{<@~UrR9y~m94c6`zx#WmXx>S1X%UHmfdez>kd>ABgE>aeHFF4 zDrxvCk8jT)Uf>=^$Dc`rV{qLJv zzq@Duhd=t)-}iLcx-X7*ogW7Vc4;6uF*-GRDKgX_?CbLOb^9j9X3z-Hc@38x0Bp_; z=;gT)@GQXcdT;vSa1*$X#vH)f1MsfC?Ewr6pnM|?1f8TbP&A7q#iPLj7AgV{k;fB$ z=EpcwY>ZK#TwK#@f!X!o+!KVZ8(+*U{uuxaKnvh1bOrT?aKuFGjky=@=(-C|GgD8U z!4((q_^Bt3$c8Pr=8QZgW!|R~!Szw!vJD=LJh4Te*e9O?dO0SaJEAXMsoiVdd8`Ij zbw>%Hmo%5pS95qOK363aWXl9OGMa=H$4uJB+>s_>0f4czTycR&sTYfLxO8zUPtDCS z;jptjS0&F;N;Og`4uJ{TaxPuSN#}7>xojGrLl*-(!SWhiDyA#MG?{>*5b{WODnka_ zy^x=+mKEz&W^gn30)w`l z4m=D1j1B+>3wc<4;wxL@z5+1z>ejDqNqBX8;_J!D@pSrj24e@8Nt1F2dhuXoss(ff zmoDKjB^gKS*=bx3jmuAA@G9y%KJRqG zDcBnT3l>)e@=E+6mXa;R!4u z4o@r$j@|$aBkStm@Rh+!(|tqHu7S|m&cN?Jb^qd{iJ$y*=m-CH@jD;>^3<7s*mvN} zuDvJsAA0xWKMn?feuZvDCLaKFg(3vNrXEFS9tS7ksQxfG`IwT+|II=PVhA`^6#FOy zW>)>t$D!$^%Qv3-CLd#{@$@56}d2M-a;c0jV_N;reH`XsNz}EH%5Y9jO@XC#+ za|kT#o}sJbj#~~yuj}IuxV;4c z=6B!nxE5h23I+&(p&0tEKLh{<==B)e7>Hr9SyT$ZV%`cdWydlK5pkqpa~PTGcu|&7 zcp!+W>aiY(N*T&!d=0CMQNo^Q;N@L*0Rx+S;>Od%Qvk4Fz(8GkALged6pBU4R04&4{43< z)Z&-?HC{r9|Z0P>QwcmHi)=$=3PAOH)ih>sA!DAq;V^W^-!InGRy zPW`0&h2~N?(76%0{3LqyS#BZumV^`g$CFoPeubPW5G2*tZ~A+JhNXqr=A0n0f4z@zI>&u)l|Q) z%F=AiF{>rH0!fZQs*?z_#6qn^lBZDUvoclu1a?XyH;u03va~{Ou25DiQyQi6A`x4~ zU@KYDJQ-KTr}61b9y61d$rf@L{LC~g*|BLt0bL|wiN$oWh#{9UWHN?K%2dc03UQ`d zBFs^8H8Q3gevg!&t&tQJ$W11>(WERXL08|ZO10U<)8+7T)Y3wYBwxuCa}pUD+iA3f zjP!IKwyzTCX$ka~fvvT81~ZP%cs(Wc>+y+S*&6p%%uLz-dQuVqSR5lGk;|aT zfPwLtDjrkKWvSr~e97TuiTOEFalVEnlf=`Q@u`{dsqF13ncI?>i5WbGK*doO^0Ljs zT#GQTjF)Agi8P5Z1}2D4=Oxg1+i2``iIkh0ovBr3s-?VaB~PoQONBHspD7b$h&jm| zfY=NghnvRank$;mcf0&ys)G`nnWa=4v6><9EzI)<;&tDRK!xb_62L}nH^=R_&<_GF z`0kF79vA@FzzC8s3dRP8fr`xzjLi02n)qGPp>-|f#dnDPy|(o!SFp;6o!SP4@nzrIWmpD8&n}NfcxDFHpZrwgHtj0 zJS-YaEzexrn7y&#n|$bvJc?dLv-zoOtCMp$3I^Ebk3P6|>qYbmY+GnN-Y!$-)lmydcfRs~U|O-L0ovoM_cBeC%#U;|h`1g3@g%S3`W ziOVw?v%yAHJ=sG^+vX)`!PbUI28*)Urs^c{;iMdxC@{kUum7PPwTSmIR@4vI@g8L* zg#d<uP7e+Edb)Oa&u7Ep+=uqhHLo9O&*Q360QOSH5v$fB{X#+( z#QNbHdZv&pC>#sJS9lE>SajV(y#IqJ4MpmPqIsMQL%NJTt*6KiX4xcKw!lQd!m2I z7R17%Gqgg)^y?F$wb8&D302u30JhfeT7K^z&uUBui>gneG{e|d+H}&|dZx1V&B~TD zl`W_1cfVWFa=gCdOx>P$syp7TX?v@*{)naigsJ*?j(HzHt2*1*QoZwp5qn^bc}4Yw zMRk^n_A+ZrRn4x3ru{VydjPR&T8iVfvzb-qH8D-dL7X)9}LPuJDIvv>dZ|KrDh?Co}R zU!1sjZiE0>cx*Tdw}buufgYc2WX6N7FPvz0I&h)6lzZOox$gHY_`SPbNa^o|!KklTU5Y4b*E)K6Okz zb) zWyo3lTro!{6k7qst@=g@~yXvXp>oLZCieja-nECCty278?M-fQRWStlF|N zQC^-PFIQTWD=pBl)xd`ky5ecH9ZUc)293{5ET8GnvOt7YY*? z%-44$e|1~@-))I|Eiq{ujTXmD-@#|3%Q-X!hpFLnbz)AAkeS8hXa&+jg``j^*Jlf~ zs<`xwt%R|>C9Y>o+d;9i=*@7GI$9Y{CEVg%v6z> zovmT#sQEc6p-v?&)CzP;wnoO&%2`Tbnt+|iqydJdq3)54qabY>t$< z6j5yefT2l?b>cj+4N zu)g8TJwwwMd!wJ6^Zov;=f8eA@*@DS|MA8_STKAy z6d?eHBzbsODGucsDHE2G;X?y3|@ zA)T69M`;HB;HHleVPKSkgW_1Y2!K&787C=2Q9(o`3Ks5QZf|Uuf$~TM^omgCQlw8A z)7l_i1^`3)<$pk~8qPLQv&@t^z1@xGUgM6Nlb>vc{w|o5J1vHhX0xJgxE+K`& zIwAT+0p-G~Ako=VrijGw7oF_;0K-V)C60gfW4LoKCS(qw?-bzQ!07pI+l9XHuM!w``pQhhQ?_u&bzIzr1% zJVs!Co;R@K3&tj$Vu!BnTWQsv zLQ|uqdY2}*tk76%u4pN-HWwHx^NK9x6-@<&z3EM=)nmvqT?X#(qI_RJ){$cA|MZ((ui+0HOzzUdD0I?9T1kC@m&<0 zS_O!jy83)({!6%>1g3>>weFo>MbmAp5iwh;PBB@L-krs%t6J}7s zla+;8Y`GvAxY3T3xa~W(ZA*N8Yr-3GiCYqrQdpThg`BTcW-1gcwJKAoV9Dee5;0B6 z#jF;kkSP}c72{~+oGcYDH(OAUC(-MbC8jJ(nap6|>vZCRe0g!cIA2YdvXe5?wx_4Y zrKTn_(o(s!3;`oUz@iIy>0E9)mzT!n(gXsUPzV5)E)eWsv0mGe^p&{yuWgUtl9rOh zXQoLq8ESrpl9!n+6ck8>g;IWjSX!i%8`LUOwya3YkO;S?q`j7q@_ItrtMO^C#nawM z%-EjBPUb2ZiXx7-gqK~CsmxF3sdi*a0Km4Sv9SS`&WX!l$Fn)9A^}q&^IVzGl z6UH1LsD;vbI{+9Q@NbS_Ug~uKt-+Cn{!7;fMivH!Z}biU{aWn5v;Y{^H*}?UaJI82 zdiH$a4`;nU`_1@|e%$>JKltdx>F@2?cWTf6)BpD4k37D4e`qlfq5$lEFmlgN?nD5+ zLiYnhE)2yfDzyjRRkU{lK~?sbL?r za2kvnb0+I$r2c@1E6^0h|INLyJXQ=iVxl(w&$deJkB3S!H1qYcP@kZ8M*pCDN z!}xN4c+I~#4M{=Q%SB-r!j%^R4A2WO%a6v9 zs+;zjtj&hfdShvwwX(Im0vOn?>blm7ss?LCjowhI(-li(c@mkvq-;-R&B;A`zy0f9 zbpmw3x_EB1x61(lHZl|h^nx`uJn5k7ht8|xw#%avvo>I0p6gEU4Z9D|J%CZ8HVlydI1%4dvC#_G)hi4rVmmDbBcj^;k1O>)V1);YV`7x$;(dxw=l{wmc@g` zWDih$c!wa9h7Z>&Icneu(ivHCM%I8&yF+Uz$qBExrZ*h0159n$gDYc!Wxyvlko8&g zJoH?C=DqUVLy|3n*S-kOV;)#={>$)(}@Dr7M=LbguA(MctGjohe_8kDkP zmC~3cH)y0qYK}&dn8|oODdpApq}LJw#%NnpSaGSD+tN7OnWAK2HcMU1&N605bCcQf zc)B2t&f5aemCoHx<7}t16WHt|E{7?X3A5GGyew@=o~9&EoUak&D!ADahLTT{vFUO) zTg?;YNX5A-o!NNm{lD+Mi>yVfA0S0njxte6wfZdRPC|o`K6febb#ilV>j=fc^4!_8 z?AU$0wf)Hd`tLsmLN|S(MQ>oy7q}H5of25s35z^H1{Qe`LN_S@72+R*HAAd)qn!*u z5%HxC0-eHx7@*ptz|@1l^kd*#1hiIwsKEhkcK%rebDURzqWPkaaVwl!xxDaf>e_k$ zL6HKg7=TgbhzK0yNvSZF|4U#Dab*(T;wTnC$pEFKpF~XZO>dAo)w(ZAi8v6w&~EA_ z3D~qlga@hn6pX#Rb(8oTAKDkKV!#TSl){QXDIrq(zBjZ?2}CIvqhbME7}9okbD@-D z!g&)&7>PBxK0G`#FyQIz9PH}sA&FmzJgD6uBh19T7s(k3U&bM2AYNE(M2SaVR~OMM z_F&2-Rv8h%5ZHP#V+9cHQg@&IT(_@p-{ohMh;7cAakOc(bAXOheo0y4F+GO~(MhthM`$oWFnvj_+0syCs46ROEU&09EvvLxstkrwrAj9fYq)%^F2BxPez0Tr`@i{h zch?1=U&H4=8}99L4i5$f`+WU9UIehwDF86HgHdj7ytL<;$P52>L#GqU#4&UKd+QFwVE4SjrC22ntDT3O@XyM&tfjH zmS&mq)uvpTUM z{mt(?1EEDvV9^~~^aBYC-37!7z~P;cFiO)Ai9QTPV6!8N zwaH~Pq7G4YCzM}fX)(NllZ+t%o|Vuvw#VT8?F&By^zuiRqO%+GxBoJ84Y=3ixrGy3tGq-JKvk}gqLEL?OZr7s6p@F$oM$W zyy^$gBV zIIcUL*ImwA2xHijU+}|mgmO{{Jn;MP13B{q?zjT?$QWX5W&_(8SWAr2GP3Ec1mR_j ztoU(26~`5*qcjtB6?LT}J?mY*2p-H3SwzEMgG)l7SIY1*eB)z%3hzO|g3)yMil@(CS2B zX~IhY40ZE3&*$_#b^yY;AHmbceUB&nkL`Y>YZKmO^0TYsa3`?lh&&k&VHI)MzYZj9 z(7W-iA6_hL{#IrCw=J!27uO#%)*m%DovhjOjk=C+l+_=%G#xH$JXGC&(%f*kYS)>< z$~`5GN2=Q1EU7u1U$#q@S6OIjE3G|PSiZ|twI|M$!y3meds)D)a_RK|!&( z#A-5GWKy+Mtl|q)njEXCbkDAi_dfbfPxr;q3+FC<{^?Lpr{hw;f1umj-|Zdj3yh6T z+Xw@5+O8lEbD)6U?Y{2vTn8BQ_-=T-H(l-pz%aLO5p8_^*aidW!Wg-i0OrM1FcJ** z5Cz6FE0ctBMP@gmBnXLoSRT7WcszB+tN`%ZO}OumyWmj1Jnp!U*TVC7!u!aEh1@02 z)P|cJH>vVT@XB)^dYNtnFQaEiXzqDPA>d;`W9f-UuGBo_sqgyrf>hB z=;EKzXo{>?I*z}&^T?^X{YNT0_L`fUP4#ugx|-s;>g;lprc5s{&Q=zyG{wr?(yYR& zyu#W-tyPy(nVVCdqtdHQ)#lo^dTl{AgU87fi^M9GMwhG9=JKS96c(GZk4oe4*$Q=r zOqn59FjY#H5|)Cg5ND!Sy(mL2pv%R8UKvs`Cp!!Me9a~RFty2~F`2STN^(ohg=OYE zlOf+^&=lrzH-`J7#dP4GRiK(wA(OyfU zzmdw?M&kg0rSeqCd`+4-7ceY|m$jWCiOUcmfTeJ@q;lh#f+VgeMIdCzr97=lsMDe; zNr75YpyuZ(xH(dRPA)D~OZ8bQqfS$jt1iir7Rkhg617R2XVo8j`|XSUPNKuUOMLeJ zCVk3{a(NQo0nkMxh)zVW(TSU|hRFoj@c7OCOY?n~0Kjha49s`;&v(P^;PuY_`A;rP zeR^T?^UkS{KlA_Y6YsBo=lEYg>wo*3KRR^u8-QMW4xB!FemD?bbo*}tfCa*LLIl8q z;d=qN6TS!N6{BLL6BdkOAu$lfxkbP;q6k2NVlcMr5qbU83QjjisV9&w3tBfVg=bKv z7`ePUxA;YLZZ$f$a^>bTZ}b7?kz*H)m_JcKMG2XRq?-U38TpE3)vuv?o_rZVAKsXr z&^^88nnryG<*$LDh2q25D42&zP$b#Gxc~ltIT;0G*bhSn7AqlQ)GVQc9ujVet*^~E zH?osbVIPQO2xh1tePH+8u>+G1+_U>g&?t!;op7Ng$H>@ZchA^`^HfXdkOYI*zvwP8eaI&t>?mmoeMxn<*|5#^_qq9FeZlAZI zhwpW}{hG&nova(M6Zw31iT~j}FGf1S*%_w{-NY@B8mOld2uU{{+hcg`h$@=0#y~L( znuL*8{-tVzAQmy1B-z$cyfjpQ3+syH#NbA0#xlIBcsM|s75ni3deX^0WzuV_16+5TqDuD5DC&J=&U3@(KOg+;(}C`bb^x%UUOxcXrGbzQYlc_sK)hZ87~GPXFpqbEz}SM*eZ%EJ z4D)G?ogFOH|pu{Q9aZEwh5NQv&)eyYthR%>Q3G~%cP>XMBY?r!a&4Ll%?|H zao0V#SWe6og`-34v;h|a8(?8fgdlEk8{XLs@8u`HD^L7$PyLslMCMSLJb-OyM z+uBPS8uhg`MKx7Ai&0mWuQTTsRTLE084GKQHI`guNtQydlImoo)uqk5T5<|1a&ac6VrI)sIoy0Xk}!i-Q~*p! zB{yh+C-JkD2~5UoJ9fO5nDlCV@~eqyuwF~f*pi+J0G7ZMrEnE#LT#ETCqZBnXDDurIX^Wc%Q7wvYmmcK>pdJ(!N2y{(2ZXOb@|iI*^kdn{`wEz-+b)*_3zz3`DyQ& z_kMWd^mh*&dh5iQZ+G`Qfq}XG3!d;Tf9Q4q&R^)=i9$pQ!|;}<6GM@Qp(w&tAd2QO zI5tEW7*-A=%ZOf4oZ>=(VidtFj6+>3gsTyWEn3k$4oogh&TUN3Zvc8-z4Zm00`6Z-Lr9$uQIslj(m>;HDuDYVQA$q3PslFG= zwTZI>mIn!dQDwuI9WqMIL5VoLD3HVd6N-^wwT*~4aNdY=lEpm~`(fC#q}+aS-N*W& z|1Myd&3lLVBHpq&Z%){5jN7h{j?MNDxH>QP1JeS;BBBf`+>46->K%Xw0JOUCTkuFH z(5@ab?Hv2lOXQ32BSQq<@TnB`4fGFp_t`${bdHSA;V78%hSNFka?c}xc@V(7-a9V$ z9jE)I(|gMa$7=M_#0ia<(Je8=B5V>DQ9=_rBKemR2|LOWaheBt7zwTfLOh8;g|Bd| zZ1~qJ5+Dp1h3{eXnRJG+)#1k9gTRK56a67lE;u*9i%d|Ex?Y4s$2>czmzp|HP_!5s zFd<|V6-H~Q0K(Nc0Fw8S6ShUzA}O6A*(Hhf1~m5m)6u| zsM>33I$G9zymIFmbKPNmtM4t9G0 z!^VcAcH3pU{W8Fqed02pm&Y}a3JxMc#7>y&niKsqHv!Cl%SRX(I<{=Ox48XCzp&hb znSZO{+10=_)<|$Tk_7Wor#cdlje;i9!pAfybgBd%Ohy-6aOR6*tl*Z<~D* zfCZpMR^Zyx;I-$G8(+YRF8m2VYnotKbpDIb)#ssk#4X?Dr~cVz{>v}iGtY>i30vFdS~y^Q|$+jHSIfG)xM`Zmlv}&5+o)0Dwal)E)!%(0mH;BrA%6wr>iQ@tE$MeTJo*sMOJHp#bU5piYv+t z*0R!?YFPOuV@_dyc7DE0spg193?YXu2gu3T$m;!OD@b%i8EAyj?#3Ql%ZDg z^0d4>wb-B$7O6OSGJd|2tCR9_q{4iqs6b7V2;N9Z``Y%zuf`?D&?_-*OLE4xbk=q{ z5G-LLTM7&;MUVybD~>69BORX4iA&>dP03843lcJg$y^~*F5~B@#f4hAF;`*A6YH}C zc?v;}Oq3&+x|oWmKz8z|44@Y&J_StZ&=*hfn(L&Z~CU zbuto5LZ=)gZpv{BLpq$d(S6(=Da@ z0f6C_hn@ew>giwdumC13+z+C5BW5KP!s%C(br2Au>^au)4$j5`fWg_Ek`dyBEY;jX z!P4ZCZ}L%KdO3RKNpNb_hXef7@f@LxDjQPJMSbZ*n>AzrBv6XP`%+OIRBdXC%2fdX z1FS+ZAvzpJV#>o9wW6ws$R`l4fSF+rIkfx|$*6u9wP_MEf&bf^Xo~+Qq!UKW`eL@| zRPiu&(!}v|{{vUxz8&kpq!Z@4?Qq?)1Ay5V5W~i=4i5S{FAQ9~fUtzpLlW^tkO>ut z$jV5LB?VRpVDP64P>gy$rMX812f2^fg1P7Dms*c>-)_G?bZ98Rjb zue)5=Nn+um+jHCHS#;vM4PfRa0EQ!mv8jOApqn>{qswp49hCl))M+sXz%%v4gQ^W=T^>>|LiR?I0r2WX(YhY*M~SgA zDO6Hb$Cv0u0BjW}y-2~0P*`q=}9l~uHqSQ@NVJ4?$O3X4j0xy5Qto>Gw|5UW^xjZ#~-dmk{ci|5V`eD-PI zXP@@>bd3-8xd*yk!~KDYkx7Sb)@i%!uw8cAuQ-W%o>~-ok#gd^+q*!B*Dch|`|tYD z6!9*q=l%EGs7t&@%*K|()7bp>Pd@gB@x)_u-En&F+g$LTe`s@4Ee|{x!5x<$&O;&nx>W%M$IJE3dO!`HuvEC6nOiOy0tzMQ=NWpw__=(R7y zb1(dutMWzY`d?f#PhFU_@`Y>anSFA@G4;e2U3=xdZ+-C2w?8=k_BRiodH3MSw+@_s zxBb}Z#)HS|_8zWo-&ehJcV$aUNljHzMOjg0d0u&$)?~~w6sQWcMWy=oJwV$IweHz% zuvUt+IXt<7CloT-3?`S!lM00@1z(}el*?F3DN`e6WsA6ZGM-K<(8+{4nLww=R7>e{ z5e_md#6ZhLd0MFR9$YUwicF~3oT|tWodC$sj1ppT3cOGT~$A#Rk;V9WT;gkSNnhWZ^tHH@*Avl`erqx*5H0ANm7)ScU7wp$~%#St{%xOHj#=HS>u|H$>;OV_&kuXOg#b@gAn&^vp+d-k)= zsgEyA{{H;ruRja@=Ckn6fA9UTU)g^2pPh$}f3SPc>4OK~`k(*&Q@eB4>sxU97F@xb z2w<4OiXx66(Q#0V;$avL6vA1en9@4B91atQj+itNQD?LCM`RsDI!FP4pR($J{Slrp zh4WQ@wCKg5eF7D$1f}3dsD}L_o)9%PMMh#tIg1=Ni3;(rk2wD(@!dPw%-s;kr5IyfHd9H+(77(>>C8k+3gBAS@D+QXx@w5aLDsAp{GpqdOe2=ZOH#CMx@bexQx{RZtlp6>cX^gos9v(3a z?WXQ{$#lFM%W<)PLac;HMefFs4CT?pDjr9&&_$jY>=M03tj&&XEmAY|pr8X?qX=Uo z7>3=)r=BPv*%Mrm4R3^uG{BpI_@_`YD{gX`$Y}#7#_%M8YWpX?@H$1muswztMu;z* zIRXy>uI#|KD1&)4LwSfZURWxmN{h(7uv$fmRO5lA2_#{wDB|#7GtB8FH;9je^P)Waf$+yupet-ALf7h1m&$a9)X7eWsDh}tC?aQ_7DXKZ3uRZ|8 z3jnOP<1I_$adX4b(uQM}##6eI4t?c5rM_O6U0!JI(3Q1n47K^DhC*YVxnd{!`C416 z>N*-*57jmwF00xL0H!xJn=5zc8X9tn>aq)Kv+}F5@+*r>^_Ge@05F}t0sxGPbuP6w zSJkweN~#Ns%Cd6`fq4lf8XD}a>XQA3-u>-wFP#5;@Y9cbKKU5lIb)XwTtj{COMSkv z;jn#t$}uqw80Lb*jq9%fhKm502RmUmF@G7?9j_M+$>0rxt}cPQJ~+-s9!Bs(y%Yf4 z3gGa*i&&|_3ESbii$*Z6JL8VqHrIWUkxeOekt<`^9x3bxrZFTP$X4{q^XQdl5$tb0 z2iyu@eIA*A4p=n>YvIp;UXknA^om@C?>>uO`%`G{3(wR>;K~dCl|R`hpV}s$IHm!* zp4cKA6XErV2&}bNzVrQmeE)+FPrvt_6K}tN{H^y7zxmC5C*SEf{?^VTXPWk(XxMwS ze$Ror_C3~y7HeZ$XvI`sx2>TsJ1rM6<1h`EM;(8T3e&Hn0ayuL&T$t1?f_8s#KD$kTF$C zx=Nm*lmL1$)iRnw%F0rUiweYr1#GQ~oh_4_vX$msLB4{glW?`-OtpX|7i>>Y`#Q=d z(_Y<{{MBtcUX4!$0Nb9%*iK`|We^6IDcQl6?O-bsGv$C@uO_o!OX0kpl({8^wJn34 z#1$~)a(;FeFGs`DO3-a9ODxQh3AFH6hQ<~#c?U+lenz6F@6D!&>4?00SO|-c!8;{g(&>LsxqW1F@*scWI!1w5!+ld6#c!Y<|Li z6-$V&1()lFi=769p(s|3{zUTJdCuL6xeQ}`29RxylpA3qTzGfI_)qUvM z3NHfHjKmiwHuy#4ok)C9d9i|q$QZ6hh}Y(P7;!;_*LlN-BMm4@^lVlgk(}Y|d2F5m z+IV1N#J4tr^lM|tx7z1=_|1R% zw4n01uIzxZ{*p7YdZAnX>wXML?T3o(UN8}rpnq5XqV_9{3UE_iJrbDIH4olU(lFHrXbq5S( zZAx9aB+Hy<1SG4?E~qqFS_+LdfLn%=+I+oLt1}fDtIU>$imKLPV?|MsSv(Z9kN^OH z07*naRHe?5$+ax5f-A^w-F5soADut@N#7rT-}T2oba!`-4feZ$gbnq3$A&`=0$>jN ztlKf?0lwt|?uE(G5}G1je(srm=A3+jxHb7?EVAyH zdO8+b9Sxv2hb_GE%6C5eAMbtl`)|Gf!JFUwF2LCFcfWo3t#2QA^Zk9Nzq#x9JMG8c zZas3k_2BWw-3RM-?XPazQ{LE8T32tWuW8%Ye){bbZ@qKo;IYFkyLT0o8f99AOe0lj z6`K4kWu97~k)`vwseC?NDq*SQoNNU*SI*VRSsLJ5Qelpoo26!}RZN9EO(aSc3Ta{i zTP4mbD=e+6D6Xg|uv+uWE#|r!YjXpv>YXhWjSZ%%%8I7On%3rGYZ+H2rVBW9iGZ$< zrOBk}0G3KchEhsXN*F3RT`8rjBs^es#YMvW+{`QhFojs3m2K54OgfadE4XPQMjD@$ zkdg6PV)Co;$zR*H1F2X-($=K3?dkORj7-3%?R3)R%9JKDrQ0)vTT(b*Phx#NiTTwR=O%ZO_|Kpq_dPXrI!DbAAUG-@!X~J7e;zV?PF1!CZ1to;@G^^upxF=O=&ndF0oh27mRj|0h2m|MAa8 ze(0Oko`l=%N|`Gfa@WG=)XdEk#c^pap! zYFa3^$oU;n2c2PKWBCM%sNt>U&55qq6A+>>(vUI+eF=3);Hc%rwi%iF+H~Te3VQGe zhIqjfrm(+-6JXRh(IomsVXtdbltBR%!KxMa^r{Dy9h;3V@;D{0M^5lJHAb90qC97j zPr#}FF*T}5G<)cO=*F7WLnr$A#<&-m)w@R=ejkt_Fsl1?o8n=Vc?zsixKhK@1O3jk zXD@VicJ}q5PZR)HXJDLB5 z%m4msySd@LLhJFo@`JhM2l6cY^R4@eYY*jD?ABNBt86(@UVF$|f25>pUrp<&>gJQy zh9d@RN4{kz02o(Onp?iBuzI)JP?KkF&>5W%;JYd}E_dUt_Yi8q1mtrdp%9-cVev%PZC9m|&SL_4;CKZk|D{ z(a9BBu0X|PE9;u}{qk3z0eb!6qt1_i*VA=zbfC{M*ykGR_l#Wf+r}bJ`;5ae>vCLi zJ8>G!?VbndvODJ-DCM}}@t}6Y1twAXCz~{f~@!Wwok_+I< ze#`EFb=&U1gBsS)G3kjUHUNE(#Yzz3g)y_Z{1_Rr9+)LxA~>O|@65uVW^evwa{ddT zUlHPt7`gfa{Y&AQSD$+?KZQ3;VD_1Ndcy_pkg2D3Vu1+gH4)hWFmp{mCw7R>UU~n+ ze}whk_x|DS?|lEw_dhuO{s$-C`}efQWmJ~;5!`+LuPWA~}|fRMExKG}TWNOi}a zvWBLT+Pdo2=DkM_y!qalx86N{^u*!zy}OL%CY@fFtIx?Q$dctKgjyL(CQRY6(*#1A zM8Z-kd0GumtKw!Wx!FLxq?t-N5U&iW6wqr2mz&JxrVF@ywY11;Dy^wBR#oMdTQ#N< zV|8t1Yimt=dws|5`i_p8ww-nDZB@+;`6YTbK$n0^NB4Jes#ugJkz~lEG?|E@lCiTj z44`+}a#0b+iScyV+?*_)P9w2VqKZ5gaM8Yht@+L0+vVv6F^vG4V2660%0^w*LZThm!dTw$h4EzoKN z8kHnBi>($V@iP+GsR^u<_{@~;?6d@4db*UAsp4|f9G;rZRx(*?CRfK{WwU7-Mw&8> zp-iVqlJzB7AOGUVwhNz(p8MnQ`Ew)PL*t|Var+fgOGE|71W_T5+it*yh!R9Zuld2@ zYXd`beS=qe2QGK^&2;w7!n)8i_1VSfr{|&{e;)bm+3;^a4Sn=S|IdEo`p;jC{o_CX zv32L^j`kCW4!!*^|N3*Ad)DCx`gOw{xJk}mSO+J#g*=R6VE)hpKimTSiazoZeL1El zz@eB750RZNhGA6iiozj6xo`)h+G8;`h5$y!xi&rbd_*x%2@YQt8~wz+mu#9bGhd33 zA@W6*-IJ@XX%uKs6+bLq5r5v7bc(_xTvKb#$rXEanKG239>)!oa`?p~Kg9rIruCGK zERqZ~=7^c)06x9!6~!1Banqp`5R^GRg=A#p98lqo6TN*&t_m(9w;8!IIOxA{q5tB= zPB(#QF6q`a6w@k@&_IV z$YrHoc05mz9vHsV)TF^?tO*wSBBc6Cz1o`|^FeBC86BinNtXOm?+k24NR;s^z-vo{ z8-y8CM~yYAis;4z%Cal6y9vaP8bQ9jZ7<}KQC#6;qVDIT`)xj z(aZmM?EfX}t;3@_x3=-V=lxENLfpn(Gnr&ElZ;C;6L%#9cPSK?Kn!h9_0*^Vh2ju5 zqT?3DozfOsBtR*Y_MHEI&wBPG?f3n-)-~6jojoDR?0N3zUh7`>a{q!bYykR=9%A)8 zz?Bm>K0|wntgdx|iu+xzQh-wmb}Z`Nci*XJ(N=PXXL z%#xYP)S0tX8I>|)u_<@1DRWL{{vvJ4bd|9Lp^iCwwjq6n%3LPX7jhEP6HKN0{5c7x ze0_F#N>({yV1~3(U21W5;k=Zzvdr8{T}rM}pC#AiB||r1+FYthDOTvR zjTxnCL!MHdVKn7iEJY?$uHKMsGUumQ@(iX-tu7@#K_L>yvp5nGMVOg8{j<*w9y-u) zXkX)juNzOF?4AHHPj{zxpm*42J!`XHa5ydkfH^@%?{r?ZIj>koFps|NaQ+10{>cqnpUL3{M6m!q>_d3w zvKM{U&OaW!_^bEaqv6ZH0r9%>+bDKNgtXx`2y`CNvXM*A(Zy(+;=lOJHi|yB;|RUr zo%@Ue`(Y99aXz_qsMjCb!)3?)>YN5 zsj7QxOWj*l^>1#jS-NrSl66&!R&ScWV$IAY%kyW>PR-A^6ciN9sF=NI{+p|pFI}-{ z&it8Wm8GefsT!jyS(hYH#Bt*IR1pW!uP6>XhA*UuB^0rkogfn?C-ajN*>VX<$c^N% z!q}`3CNqdm52DjxJQ$nFjuWd>O_to;w4!2tetvRBh9M_6r)+xB>^UWK=a$Z!S2}lY zK}AJcK~92NLFF-{7}RhkJ&eH!XR#3w#s~#uv5=M^W+x}Gl?gmeqClhIsge)_<7-r0 zrCgv*mZoTgYFPxA@;WK_4N7PTH8O}CIgJ`Kl}dUoGV=BC@M)xoU`lidog74?U~@%U zFw9xzM=?bqB-U%86a--Z3@5)FMw&)pMT?~TWL2C-ElyJK)=jgyJbZHX1&=naHG5bI@oaZT4}9nQ>VW&R{kCm8_GjhHD=t}3QMqLM&X26Fb9UcV z1Yk}-bb$n9NEo_&FxL*~mv88cOraXTxBk7Pa+ezJoK5ew*D-&^s3=00rw)zx#kvva7mWuU344U&ggxY6DU z#!-l6H8laA;SyLI*hjT?qJzdpIN5}V7{acO4vcXD00U>m<_1XAKTOP9`O%32`mgVRs@{Lki9s5i%)GxK02>^_ zdz1hJ3ji1aT8_a7E-b4dq!z?q8@_}^hVYJH3X3-ZIF~pE49ty1p=jSg^ymGh{0tN$ z@lFE2iNus%*BD}853rQe6dwd!6TRaD{zvHh_v1=fum4v>!k`l7dUUk)`#CE< zG330JRk0zh^es*H0%O7AX(xz8r7B9{%S)N_EL}Mx|oW8Pj=94CZ2ak|o7lXfov)j5)@Xyo~HplQ~DDGi$Uaxk4q4m-Be>B&ski zv*e?X_U-%X#JAs^{`Q;3Q^&hH+wC2#4q#wi-u|9pYyVlR?K}c7r~R_SfsGKI==Qj3 zci(XM&e>$kuGDTfMbr^_J>4*H`g$g(itU1<%O0kv&N{F$YL2>W(zWJOvY3ic^ZuzjDk#xX0Q`AI#YIzxv0okSfJ0(OD`$OudGDq zRk3K%^o0x2K~7njDJNZ$B&G0}Q=`KFck0ytof`CVXm}`t87&aR2w_kQT_)hFq&&4; zfS^UM6YI2mRWeVd5F0ciZ8AyBnL-YGC2HyvQgAROEEvHIIeaQPVk&ssM@^$f2hn1J z8RT#lEsRbJq0>W|tS}}goW`FP#dtNC^a{eTaPq4Wlt_-4nW*BcG*YcLHYr&sm$Tx< zQ9{-*foK8L8)Loc3Rz_K1Bh=5^jtzb?+kxuMr12KJnW0SM5 z2W@n24A`$*aUcffaNPirwBx$n4giMZdyg0F#vPuYoZh=`+|dlve85ZkXM|ou!{dm7 zficH~ffn9^5M=ie(XB-Qm>aTu4;{k+9_GRlc~I62!6Q)!AqW;A6QM#R`oM^H7HjTt zsSGrY1da2ZLkroj0bImYF({9X;PM!D>j2t>t6Z+3M^3PHc!0>39XC7U9PfPs!_Y~a z`<}xO&0$ve&(I2n;LUXx9owOe40Ioeli6_}7$RzjhxK?L_ISs8eJ~ZQ-7(%eFurj| zgF0aFNDR zt}qshle0z1S!qR!b4r(_X3b2^sx+ooWEU>XE|`~skSV>ym|kj3E7!v`OGMy`m4>2t zWu{b_Dc9uc%%yq@WEwN`W~W$6EEy%4In#})`G%BSlO+!Um{xC5sf7DLdZs?1@OH_y_N(02c zlkg&dtO>Ys1A09{7>1y0?CS4h*M3I;hLG&s)!#=iJ;PHAeuW|F;PD2xiWkbOzXEPu z{LOI|#6eI%MVlUwEI#*Md^ULXh4&KJN)2B5(|_p?gk+v8FJ9Wbea99=!glQ1T(@IO z{mxC>>eg?nUbm%sEsCu*YqwUf-n4b~hOKX|-Lz!YnzFg`vx`cy3JZ!V%H}MXyJ-2s z*@%A4EG?d1oSu`e)Tsnw0fWaObJ!HVfGQA?x%_ArC!E2G;&RDCAx$DCiMYf_

sy zEaZ{J{AeL3g3An|#Jn0B@>*ypLnulp04`tEb zphQlggoUu9qb1C!SVokPN*2?}LQ;$G6H<^?|1pAQl#Jpz!z*o*^&;Jo66XdA>uSRRaveY>aR; zCwi8NTp~fah?vYJ636%{%-@s4W=U596E)vg@~?& z^Ih#7eT_}_&dv*{EHK1QGpc$kbAT4vExDCR37@2Vg2B-!av7r|`VDgJF z8y^q=2Jaduad2zb5QqkyfgWeD=nb@RVeBD*tg5feY_d5HO$Up+r zO{zl)Tm;QMmfeqnmxmiC4AHR*h#GW8qj%g1ZsiZH*muJT+IoaQ53T+Ob{K>C2z)ny zT>)reKVRT#(Dmhoj%J%DFzC|ey=RA>bT9(8c^*It(SLuyKR)1l)aQTH!ngN)_T}eCzxo_u*vaFE+uNE4+MDg|Eza&v1Yl#c2EiXBK8 z0wputW6U21&gak@cH)$90?Y2D*`OTt_&EH$b<&LjYi~4xsyF zQoS_>o$uI~8^)%4&psTw^l0Sj(~&Dr$F4meyZY?xm1iTDpAKLAb@(F8E*QD=3}&N` zK~37`zfasDPP8DL!-`_ieLQuKJRZFCZ0Pbc|HWrRS6}$9JVyZLyYjmSf!XC3gIE8w zpMUz&#+tfKwe?%-0lhZV)UU6q*;rMBU~F^k_6^l_2*Xxy+J*pZ_4=w6Yc?!gv1-&}Vh{IrU+C_>)P?C+ zNB5P^GnYC#FSd1DYQX@8;&|iOv4*k3r$!DQA3AWvfAGlg{zHR%zj1v0mFPe%X@Q3*LJBz0dn>W7w3Mi0f|eO}7VO7yuZ-!%*Pv5cKpmMzXuOC;w+qUE>BR zJOl_vNx>sdxJ_UfP9aWg;U_VSNXHSV@V5aKuix}< z?;0G80D3tl6`R0=v48+u?J9tROT9;)ITdSo9$Pw%-9n0X&qJ@Wl$lYq z7`CZy`^Nemn`^djuCA}%v8Q_1+nefktgoqGyR~k8Ro(i{wQsGjTDfM!l9j9GE?zKa z(VRI8=gwO^Z^oP%#nVf(@^jTDgF>fO=(RG9N~%&ylq#V-31JvR6w8d2&?ORPd>lGE9)pNia4;=AoJ$JhL`U%{Bq5cZz~ahz ztOO22!e+(sm=YdEz>Vf`f=IL};iOlBqyHHkH6@zClWXNE*)utLN&O zGgrFKTxsvT+}w7tvF$=b>p28q$Dsf=eCXI9;$8<14}QJh_t}?@kG`<}>$BeNAGK`T z{q2l-8|E)w|IYhg^jb#|Zed$b2nY0pK?;$mgyAeAO!D>l@A!N_`tVd=m^D5Kl500! z#~}lbdtkAE5YleIR02w*M_u-z0wEKq*5bOMIqHZm9t)_x`+ ztncV)YO$R>?dt5g+=l_oW`oifA*Lq|z$Vd+=+}?LY$rPgFc?9Hb__5OoN(617>0{{ z6K@u*O}HpHQCx%+;@`;ADE5m2LW~=>0=?cq!r+a-ml4CSL0AYc8He{`GSds(7aP)J zdvcsJ3}g<8aAH@g0M3oTW%l7mh;=y-2}9@wYz!e5(E$d83?%Ycpn;$ZDq)z44Y+}R zK`#WfFaxj=N)Sj$4nFGlKj=qP3{>?`dOVN1z2hCO2knmWQ$0V-ee2&z=0*D46)DB5 zQj6bIrq9-8%`@fBODkNY&zyxG3ump(DqC*Mo|{>`e8z(6?DDsgER_mNg;GnMLd zW+&+KlMO{VMT?D=^0e%VtiqYbjN-KX%JjV1ru2%Gj0(NESZ&Np*5|2BMbe~9g*HDm zeR{GcTW=~*X|v=>sq*Ah#JzMWSz1H3R+niqW@t4=d7>&#lE~#sSR6@$%&=ko_Rl^! zj4@vxO^ZZ`t3 z8&2;{oBIYLWLDc%yZsuZ7Qv+BHZdp6KXljQ{mJA189G5B_412%1Try1io!Sc$c5`+ z2&+af|2ldVaWBNWejB~~Z0J0o*V!x2p|ANIdRrgZ2Z41F?=KO6UZbGNLVRoFDj`k$ z4Lw3z*u`hV*Zvr~_J;=%vg>~$5{4d!Zv5AK?a!A0x@zm!ZLQf{w`1GRJ=?bL+ETN< zcE_Gt0I=;FYPYZ3R=)v7RUIN>%imhJX!-IPb1N(7&z!w*?yLoK$||Q9mX?^avlM#0 zOshkYpw&nSNFMG9dc*x*ER1Q9%8gg`__zzNoiB2p|jTFi&92{lv-6hBIe6Jl$tT5QcS}xzyHqsU0MU7aD@>*Fu&Z-3HL|9(r&+ea6y*gkvyx;^iF-qSzq^jvdzuG!oWcF#>G z)*=#>Fo?k){DUxD3eT59NCwQzgJ{{%T{rFt!?Xj4h$o=Jz%?+8J&+A2+K_=r0bCKN z_IDVY1T!K6==HbC)1-NMpzZ?6Jp^O0IQlTQ5$h%-6>)PIfm=8|h`Yq#^ME^$a1#=- zp2Bu-GLbbBn6Wa+z=-tYWR;E}h{VqZ00y>Ff%TmLFhbmc&6!|EGp1s92?-)P9&r5# z4Scp+-Tha)x`&&ZdQP2eMzo61Xb{d)9f(o2wzmVLBEYE)OEv(#i2g7PX&pe_TDx%F z3y;UdWiV_`-_hEBrls9_s?pWbd9G*Rs?~N4I>4Z?0j*$mZ1wAQe-HC^y?5Lgt_Z|+ z`C$J+Ko?ODBLEDj^Thjw{{jJD4C3$R!iWWdwjF`qJGcOpoWpP#k?$q^x6s9QgpngS zA2qSc0iH>JkuQeBDKGD~f#%qUr&QM#;b!KRWq8*<8) zr{v5nnZ2%L&PH>=Qf<~eg&6=$sLWBMmMTm|N@KCXT%MSeW-=FQQ*$-ud`n)XC3hxb zUAWLyYDz2CnTwP3`2b)D`I0l^=eqi;X`_o1)$9zXnTOY`ae_GWuqv#q_=-q$^7?He8F9Yq*s zwO;}r2I~3iAmTts=72HH9>+D0^ST#tFLd5@&FaGSFesfuUJ(J9+lzLzKOq1c0<-Tx z)5Q=1Ek8KZ8gH!v~4-UZIVd)~8Xr+6Q2 zCGl83=%XNR05?qDdE!6!+u#L6+FlS=9V1tNAHIScpRYg)ap?LB|JC2Um!1vZ_>cF> zbMLhmFKwu)Ter1(?WV08x71WsZ{Jq8bIZ2+%~iEqYjH`ee#g44b!#`*tlv_*dSlhn zm1|}%UOW@guch;5FJ3Tf(fraG6*+|kso6Qnrc}AXlxQ#}>J147qf~E{Xbo{Xy;!Yg zCnnOQQkpc5A{Iw-xIt7}Fr7{o@r6oRtVS+QmNH|Rk*x4gMraTtjFrGoPBUq8vbA~n zrjnAZii)(7lJxSD?8;*Fjw`d01UeQ=P2n5ZG#Pa&HR#pwDK7_4c_k!x8kriw;Yahu zWRWyV5Em_qBMBuWp@<~pNAZ|30%nYm8_wYbGnmt8^l5Z@Fq0m}W=8Pnk$fskA>ykf z^mryq#$v}anNlWC&KD@e3^9wsXOemBNIElwga9n+<>1JF21mXe68%ajipZD4B3=oP zd_9~rHInvvSoG`R5g{}x0F95KH-HTVb#}S!&PxvWRYbq6fMGWr-dk=TG1_iYQvZX`ciWE`m~XPd z`A5&-9nUZT7%|$8P$K;WU=tz^So^S1ec(9I(Hy9I{T;xFa?9kw-+(P(h36RxWcGk> zVNMoEDo*mVNld~F%!y~Cz&3vjs=*k&aHlgdDh$ILLD2|f2!dV^Fy_WcIYIsc$+W=F zGXeuv2pbDRyFuvb!Bz^KrN9f$j(sc%`UUQAPH2JXvt94)JKND=Z)|Kw0EUr?NFX8x z)`=@&#IR-}0&c&5!!VpY1OPkJhI_&q8(LeMLDtdH-rdpB-PU0}-Rx@VINNKxZgql; z9tChONFH8C%aF%&+v^2~R;L$w*YId*aB&3QjJ*&i_98GPXY#F?*hyedLi{&z4hHti z1cnhC0@zXzH86tLPOvdRJO2atBaeZnR6s6|MTwJJ3j6T56@51>NAy7E7W}B7KN{%T z6POc*UVHYbbL^>O6rtS{K(C>3+c3~C8;n{e7S5Dq9Bq4n?+_8brUJmcN&afDvo-bWp-@n+|Ei}PXi`z@-B6-jA}GAh=aOI8~5 z7Mrpc8?)x;PJatwSbEXYqFL**iWVEvW@HyH%_v@ylD9CuXqho*zRXaXlv;rREGe~E zom#9*DN0hNX$?7QBU&f3b!kP}Mf3EQQlq)llv*5@V3aB>GEJ5^(IQST%aqxsjOie| zHy0$T%t`7roheIa%2ewzboz9IAx)XAmn9@i;}Ru^gz=?3fxMt-&U^2E{V~R{{a>GM zJlWgYXl-d4=xDR`oB<`hwQppge{{fhe!vdnRG|9hfG%bLFvnGo>pFz{hRb~e-6{bO z7xZYq28LvwTVBY%0R2J$<{P~0hB>Z3d3|>wa5cd{`f%v%_yjVIb=H zC+PD6LgEQ`8}wm8i+}@XGTfu%2*&&u(3|_%eg3ig0@~J~mq3sF(4~M?$LQ4;!&jdV zUwbil<(dD|vyp3m3|{_i5S_g79~6U#lim39rL|kPuHLkH-R7#z+v>K}?AWxmZevx= zmaTO)+jm!OM*y~c-PYPo+qQ4oR=;Lr)ylOS7cO7EVEN)jE0@n%x@7viIR)iqIYq_k z`GuyOJY80?V-qNnM3OimoWl!au|k>DPZmma}q1hc49 z>CqrZWJN^_$&^?UE0MucFd1?NN6zBPSebd|WBd2o8;;`S;Zb|TF-OBO1X`}9la8k2 zD9{-j0^1tX6J#@7gWn0J7IJ$NEw|t zF-VkWfL-ARGHh8y7^@=~!!Ad7A)a)ff~SO30Wj=uU<}ouaahgqTxRUh@e4NS10f<7 zb3;!O_*g;;@duk59tWJa(WBLQYruB7yVuv;+|}5Ki&_C1h3Ho&%xJ~~m+@dQVtggS zxu$09MUVCMZBX%QCny&&ml?M40*j^PVaL6=y3mp#go4{{g5^!I(VF*hb9Om_LRcE_2ec4Jhc1|0vT7(Wds!P zlYcJaOQ7JuxkN8ka}Wo_uEGtn3!qalY-Q{*?6`+O!3bLooH>MA7nX@)uPFpqk8LC9 z8e(uZu8oa7ag08Osq>(I#5@eA5TVD*5BWdSErJMEnD+}gr%dA+d%UYne z%+Xk8SqheDGiRsdFUp^}CcAW*(p0QVtH>-_l3uvjkU2-2Rw>h$#;Wp!Ng0WT{6uwz z&XlLpWGJ;6>XcliF~^Wzl94wnHDkKbTw+QsN|2`}B&TaFrAlLALULA;zCdFtPR*=H z%_>)D)8#6Q#+acoT8!rG6mwR3T8=@VrjV-<1B;7O@C6ABR-8tgRbT(_k3T&4(R=&8 z{QT(Y6FqHBw$^5ATZ`>Xr@Oyr2uN7}XutJ*Acg3-LKMT$Q*`Y1p#2Tv=+_A1-h-?LY=a)apty5^5|{7%uilGK{a2q4UVes-(V1avH#Kqvj5$WGzZks!987+% zy?|!t%g@i<{O`#1KgVwUcL*mH5d+)0Zp+3^Teoc9wrz8D&F1R5s@nRx9ee6`y|Z=4 zt}XRDHr4FdT)V5PcGvnX)vMQUUa@M`@>Q!=tXVT}`LgNr=atT!Ra`MMfBKBPKnE*DG9DtVelq>V>ZjGH88$pm~wEGI!g6LH9V7?l}JX1pFr zc_ldd<)Em4PK$UoB>MF*(rclJlZ7D+do3dR^%&9=5;>Sb4`(yOShR2sEu2daVv?uP zBB#;9rqaVhI1%Jn5;uv>NoLdK2*y~#B>tRv6|Kj#(k%P^LziT+w*mSzLxzW+m>}zTsIn_9N^7Qbrlm5deeULD7J? zs{@`-zjl7G*ZS_K{X5=kuX+3Qwmrw!ZQZ;4&F#B)ecpA(V|QM#JFnQ?S8eWV1I}w! zurt5u!WicA-F6TC=p8};_Pu}ThrtmrMf45bL3c1>U?Alf25$~Ry*3$>PYF5V*Z^sWd>{n&&;$2tP{9!mwm{KF9)h#f zIYh|7l+Qo*)Ia*vhka&= zSlodVhCs#6U>lAzSk2LfXE*n@bvhebd|iE4pr6`t%Z`>I*9|vVNqs-r0fqp~37JC# zS9dXo#x@-As|EHV=qZsGo#e!5F~Vj}1PMGG1)Mf#)4RAO_RG^>~K? zBy13}h{XONfO$A=I8jtX#0%>+fHD8-!P*Tx@DPhT(EEsW9QNS{n5&HkFbry7k5K>) z4Fb&y%&v4JFdIk5{f?je9C!OMfLXC0hYjTNz{UbYoDrn?z^cRQN0jb~b?Db#-(y6- zdi*ff5|OYr$Ad#HSIZZ@qsdy9TKraO@k&+tJhgdFTEUX+@|F7B1-hJtmXf6zC5u&Q z({-7%OgVGY@)s%%C5f5>i84okpeVI8C3jX_vN=hcVK5hHQ*-6Iv;?Kam{yXJQ<;-L z8=;p*my?v7uGHox>kE{I!uaIOM0BjpH>Xce&zhk#N)T&I0 zG9?1A*w{oqKc2&r#m4F0e6!}m_YQpY{{AmNKX&rinU;or6s=7*Tm~B)=pXCvKiiMT zF(cvy(1q(^1I{Zp=M`c$m=gh*6Nnc9T_8$C+{*`wL@#7tej>u{1r;j-FsBbK0>2E6 zJsi346c4w2I&$IB(7A_*jp2{~IFV_0;wA?Vo{0lHPCtC2L9ciY>^U%oJsZ6A%zp)D zSNbphHgxeR_$6Yj8bhFkpzGFu#%}&+^xBKDt1r%7`}5q5KhNL#@92#`5UP#c`p-*S z>$YvJ-C9#$y{)Ed%eJb`)z#J8chv8GyKdJz+ji{Ux_vh!5^HyCtf^mDRlRQW*467b zu3WQj#hSH?SFD=1c-ia)i!0|Ys93P5eE!0M%GtRUGjpfU%qpLmURJ5eEsQf+0KgPU zq&P{WP=GKjMkr(^#PJkTu{u$1G04pZdOU9$JtT}19wm+;%V?66cw=E&T6s~{?3r1W zm3cEN3oB=p&zf62b4EsarXougo66(pXlyN6praOUKSVZBAA{gA)!^y9PkihLXJm!r^K(ALrB3}*- zdo?@~Ll*;~7m5%jJ%m9;%nxB$D3dak95y972$8T*RwN~s%u{mM$sA@fho=&bh_bK zXVYnWW25g>ga7y`|KSsUgkFb^_z!&N{qh^v=U>@B_^j`pk9u|@0NZu4YR8e)n?IYk zsA|*Zw_Dq+4%c~`F= zUO>f&r-V{*GA4Fp0X+CSdQIXJ@m2g3c5PnB90q!#iECge7m>LlhMVA~W9M+-BqCne z0OFAo+#DX-hVH=`f|-Gg{vnvwk6|q$)L@{y7O-#v!-!%S9N_gAN1y-(|8;<)1hSe0 zlEHW?4`C}6803bnq<(}UrMRg%z{Xs+p_8wFtfRw@XM-W0g}7HoYcrI%Iy;F17})g& zfC?McPhuGH534^~C%t_;I=b*zq1PEfT08pAjn3B2a|1A1<$G9w+&2(z5kf=|bU4v5 zjMjB}?>c-xIsJDWJ`kP|$wT5z#0g;lun8E0rZ?RA5J=Jn=FUvyX$e4rKN_Y}2*3zz zBPwA~q{M$N2S5m57!K&@a)L+Hc;Fb?ynu%xvIS3pr9A;z$MCoVeM5jQn9O1yd2Ge` zLtFzJ2b>%I)jIUB*Y^w9ZFs@Ga?@tTY?UkA4n~PWH&fI7&T#=f;Oq({_T(C%; zF-K;ck(5$lDO#kolqVXClTD@OoOwEPg+9GflU^xMWbhL$$*IKxg^4Xu#wO||i3Y7H zS7XS|Et;KMFsrC!o+-5;J!87wP^5)!u);)jwm300UYV^n7N=UKr=%8Zby>;kG-GPM zQj?ab&}-BwI&G>tNrxC%LVS`!p%#h~c>F}3Kv6z@>7LzR{QLd=dp|u4Q^6Yhni_h+ z7qQI+0M<9!-*;nIqX+J$Kk%__{-k+DgYQ5X@b8LjJSm4AzA@`1Z_PO z6n^yhV7LLaVfgO22JiaEVAS8p`NxQLAq*q-tVBV9aCpHRyI+V83z0AJz&(TB)B7IW z0fu1Ad*O-y@^7AtPyAQF-go#iK-cJ%-_L>@Jz`+LkKFuY>?ZgnqB9q6{`cI?KM^e( zgPE3pdM-VCseX4|{mz;lJ8SB9)Ku4Qt*Y6!rKWaEZT+_TT`0EJ@7#g|dYre_ZrWP2 z5y9A|s?{4duimg_#j3T7mabg1V%36`Z_Qe=tZeT5;#qTwD(4hd&dn;Hg#b*XGcgmB zsR=Slyp$xBkR-8;cqvbjAX3Q^Ov$P&Q@lwTBc_FLLZf3zA9Je zrIyl?oaq&%v*(sp&dDh$(q`%6&C*!2fUTvmG$gT+rp^`_3#Iz}1at#va+QpD&Ku;Y zDJ0TV#J!@(uSY~riHabzDKsWIC@d%_Dk><7JdH#Pr4w2YE{V?%W78u99KKd1HS1+* z+T?U~vNt2i(a}?> z%v4JBR7zwJCF%_XpHV?msbL|E2n1kU6-F;zY^*LpZ`OYL z;X8ee#|ZR-R**oyzC%x}{(6#elt+l&OZK^r=)}~JpfGuCK z^>kye({6)<2bR(M z;6F5i;OZgP6OKDa(0cd4F^U$x`;hX&iz(!|piunSefE(D@h}`NZ2jN`pL;Ap#NCu$ z&>uqa1~CsWF5pf6R4$m^2k1qNr*c8^>YmN}b6}1(So&g(9|ADfZ5z?ii~-Dw2$}6h zZ~uj^Gu~6D+D@NtYzNcT^_fzq9j zW;n18-Zl0sf+ZJ}D#5wKG4j}f-4O}EbD?*07`?fOv^^O>lxql#Ijm5M12sp#|3M!Z z&p)*K9`<{ntY!B>M>8=j8v(P`{d1r5E;i$M_?MY4MzsML>pTtvn!V%wo=1Id2rT7z z+~t1MX20KRySKLXNNVA_oEe)k%T{NUtkP!8Pcl}>jb+O8S@F6ueb&6pvgO*$nYyf5 zrmQ(xg-g=&7g!1wnDXW&=nG?&nQ~)(qA5?N&q`3I#V4g0Qu9;NicFTm^sMrNqB-fA zWd>uRMq7}i&XsF&;+5%wc)cVkJxN!fGZw2|)7RHC+}}H9 z9XL-+wzLmia@wF9%eJf}pp}=PlfUVzBy=Kd{wVSuSwSM!PZ>Xm#vtwXmQz`dF6unDp-NN8OwxE=T%1xGpCl7XCEVCp zxk47g!nuqP7HtZhJcUMrpu_-_#)Q5W8T3YU z@HA3L2s0{_9Yv8)cp5%OD;B57;|=nXvi#%wzZgIq3jtW;kx2j}Sl7hCF+$maQ;3K9 zjvwkddZ6pOZ#oZt(|+Kq*8N{KeErq2FTOtb`Mz)W9{lc`lSdAA96H#0;9&p0gZ*E8 z)A#A#?oamiyz_o*{q9qByH0GZ-oIwk$BUQOEL*%84IX0@F|B+TjreXRq~ z3oJR%qa8FI&;jQ1|K#@H@xrt)7<36pHUvFkfMLS{6+l2rJJHcR0co)M6C4XKh2sP* z1MY>>Q$(7PIOD-(-oHJCR^>^A!zV$7NFa>|TT=NzaESef2$?m$zy+3xz_*;B><0!A zDAEi*ggJ_1Z1)a;puYf=i z7I!;_E^zquqXS|j5_k+kD;N&D8-|1pxNi5uc(7aj1J|&azW3zGMns%&cUOB$OB(@C z*xMIjSUaLtgzrXY#~E~HLYIgR+8Y|0;277yTAJZYfNjuN*3sV9-PU1mZu55cUa?wl zS*;);11*{JCK%j77V!rJU>@&X7onub(1m_xtLvu#fWi33fHDQkX96$=WylEzMo4BL zornR9pkH1*V1-b55CRz@ql;BAu+ISBLPVW$M%Ocp%{6eU5M+cPO?(UmEwpI>QWY@W zAgCAs8)PgWBa&qGJ%Rzs!%yr(Pay9HCK`|Wybt;ZVIpOp3!xXl5VciGqsrhTQ z>5KK53)N|J6)Ce4b<<_~5_91aQ|>}d+RXeJt5Wjj=`v>OGiMpIW~Ub}G#4#Q$)7J# z=d$BWaq4V^u|RIf)uk0F4O!-l5=6q3y3CB+%Jl3RDs8qRIa964Q)=@Rx}0P~wnSkP z$&3=YMTr(cRfa~Lrq!hB^qDGMn%wh3BhA?d8%Jb1{zmHyriwQ`B;bGTapzvINI&}TT=&k>}w54`Sb^W%w?KSl~ckbBp zubprItA6*}^}FBMTDJpX)#j}=RW<0#JL-1ssouV;s-|w!wwiTYtJiF+L9uG%7R19= zt>3hG_1d{_uAH-c`K%>NXD?kgXZf4u3m0p$bNNa&Tb9I@C(#oU5PH$#6ZlG{M5Bq- zB+FAZa>T;8nnHoc0Mowi#dU3JJoFYz^3N=E3 zfydRcm};uPMAsBb(q}00W@`%PnsR0s)oF5?kQL41kl4H^Iy;ia38gX+fKfRtCW{pu z68?HnWJnAf=0NsnNl2!~}(NNmMC|qu`44v9YQ0IEzwdP^@}$N!Q840|=iQj#wLy*f3%d zLDao$r;plC9Yv3X!X5+Ix7|nfcOLq>_J4_@W#1Q#-+Xoai!Z+Y_|vc6|M>HF zKi&J`fiFKl{^b{^zxbl*)4k0fe%iGA-Q%^p57+EGxMABDt2chIc-ht^OE;c8(dlua z8+y#)IB#=aLKx<7U3I!3_YF5L3jb|47f0K;>_hzVifk%*n8CIM`cgb@G+ zCnub6C-I3;YD|>JCNKUA!~VYgC)AAiOYj}s0EB_oE-0U1)do>znJi*K(F;1mV33Mq z_<;j{fCq#nM^sW zX#tH!SJ#>Lb|6w%s{#5$0=)>=D0sxudGsZmGX#wyI@8|TjX(<;ov{{#xU8+y+Su&u z=(=D9+stb=8(O4pfJ3Y2HiR4hNUoopFp7}CFj(U46AaAhftb*w0M~TmED-+VV|B&^ z1M^PU_=1fxZY}p=+;TyM18h@>9XA1K2tQmH!|?lsz7N#Bh(2W~eT_bGjy$m;TIL@o z3^;(FAut<#Vjp>I1KG%9#FK13bp8nlE(W1wh0zPn3}7e2eh-00pw#Gd-|utX?XyEe z9{RloVAdk=Fg(N(+&2)C^}2uQasLeMU!I2u`2hdi<2}xYeRwcfw`aW5^{~}`zs>RR z%cG;2#hW#0i}V=_)ammQO*7T0bCXkNq~tG1$yA4jtnPp0St|`6DoHfH}DNC`GnNo_B+8l`@ zRcXwTYty+Bl}KhvRAw1di_Dgy)YJl#DPOJ6(i?M(hAh1{O|3C2m8r3D%ETl+nq*02F4JELH^Ko+2w@f z;lKod5hE_4XBndx5gzv~kNc(rU+I9}XUHfH4*xWA?%v3`2fpE7z);F}$K$!<@%>_T z{$R(OA-s*l_R45WGzgnN^g$as`($YB@$lG_vGc#3z4H9r)!#=h{dV^9v-8(p07PAT zeioq@MzY}>FT9tZjv&5u{SV*8r(@S&3|)FUeCZjW*M;9kuKeM@4DHN=SAR!5%y;>> zmo{$QQdLu3yJN?Wx8L6V{(C##dvE7^@9%o|U38|px_WDM_2z2y<+>facGsXYJ9boU z-?^@)ZcRNMP7nDH!d+-sK^qkbK;US;)QA< zPa#P#>f$siQBtBJbJDf_5q>v(G$U&iz!4cFbx{$;bp`h?3F?iml&jyRC%mkk)%uBRl8-N@o3-4 z@A?sdAwD&))vmzjvYd@ZIenyu0(?Z}0ka`}U7(>OQWn`)o_~$1B(DUa+`o z?) zb;L1#2~H=rceS?lo^BXu>lhiZ-oU-g*PYnX*FlW6B%C-rcqACM4#7Vs_z{Bw1t;8a zNi0CZ2qYsafA|bKcnInhJSdsaW#Ez_5Gfc+i8c^e{|SF&{EiS+Nr(S|)pyU1=kwxs z=OMwaoLIQw9DL;Rqj%(qXAt2Zj6QJ=V?hVj+~acEBbyfh%<6*?UkK_52VR1KSqC41 z2;OxcU1Rq>vZ7=6gFe?U{jR$(Rc4hf%Pd=JDO{9Y zwmdz5VQS7?OTmJa{CS9eDb3~asywDpCze?>DW&nsOi7YipHXbiosn0zK%Z8eQ83Gt zIbCNiL13mg7bWYm)uud|+9FQW3lp^BM592eOH^lQO!-NwOtmgssYzFBGqu_bg~F)P znB~bver%#ZETc08EVhKfma>G&1!aq?>ppn@{jUxlXgGDE3q?~yZ)@{FXPdpd!_#|a z(AGa@9XM+nxCkT+%jj(m{9*UpaCmOm-Pav13}7CRt-72yhyXiH_f5O!ro(&7H~8c5 z=+DDvfANjnb^CvE!Td0oKi&_6pME4}gApGAxNUrB3kzOT(7cSDP65VX%nJC^51)Sm zePb8E@OSXaZ$sGa*MIq`=i;wJ*M1+m@_hKpi(%*ng9*+EzFzb z0!B288A(T=#g1gKseB<#AfoViBo>p*V8+mxkyJLBBPMf2R9-AiAf<}p$U-q$Bw{7X z5P-!RReW_EOU4u^*a*P*iCnsfMdh;@LIKQ`7KurGK3T};$R+V9NzxPh{zK#Q5w(Z;7 z`pstzUwv|7@4pX!^1;D>fB5yg@9*97&c{1pz z6*%{`+<>x=+wv=K+Wnhd_}5 zo*Gya==6;v^zsbB8;Y~I(Cmeiicr)<-yfjMN7%+cunob%06ODG7d#&DA=>ny54aZ$ zPw9h+D(K9;Uf10LH;_vkaDux~s>Bxc1B9Bxb053W1DL@}JP!xl56?L7o$);E_JWwc z%Z=huhjYB4_wLH71KOxFHS8&D_T-&qADGQ)|96)Wa|vs8f}JJlcv;|6{-||tSmM@i7!lG zbEGs@9F-+AXI89Pzw_<4_a4}H3Zd8OlV=cswY69~+Z<;)Jbh<|tbGW*&LaSWGMMAC z(*+G-PJ~<7hzHyZoFGDm-Y@isxI6A;hHVf|#-NSt@W{`Dpy5Ib!cR^w+MoSob=-D1 zf3(_f+g*21xL~dgBqUrzu!%%>j&F3_fA-PP`NyLdoAb_qp-mOHch5p7<|49lHF?bN2DjrRU!BFg^ltFYnpMSg-ipfByN<#oy0f|L@>M z1Z>a7uK)Q`?c4vV-}A3s@4UDB-4AN_yi>hq`UZnCjZO+>QbL)`AQr?_I%OIyI+Pw6N{bAoM~8FBBr$^$%b-h{e5HUX=aB@o zw2YKf2frFPeYpSBVFJMHxbB5Dhe!VoVAfNI2Lkeq1AWK8?KyG)q1Tyjzv?OtblIYL^DD~A z^K$Yn=B%>P%DM9v@7?>Y!+FkTJ7>3Fu-OrM{eNV=XLOt8)~?IXJF$yxxu!x0JqaN|da)fRF6!OAq>(}ziA$VDTdZ%NKWEP8l?{6z z#xulgNtRWkONvzLn13u)XOSVHxDg?=$t zR=|25#btb3}?f zg}Nv^XO2!kSEVih`6ZNU8C)@&CuOpPG#a*oMa42zntAVSHr3V~0)U-9dHKw#%jeFt zT|D2}+U#m;4Olz+?baci?WzL+=0t!|6Pev;<@nJ-793zj(R17DzU6WK=!OLczPlJ` z(EF&T|8Y;>;~@Ou`|o)J4_v;7Hurry&IWr3?Lr6-Gk4_oG_et**E;Ehm5%N4?p)0Jqrv>1cwn+{{Cm(!vt5?Nb~YFEFlj3 z_=g_^+_m5Pum8LI>aV@mfA78iN7wK#uhkzsRD0m#nmzl=O*K0z4doU~bzN;uU427i zLv2Hy!CY-LSDP%RYO~R3sxlZWD~wgU%F9bD4HX7cxxrLvv{V?(X!oe9E`x)r>Mf-e zo44)Syk+MH+eQBKxX9OIBmNZ|^IvgsuO}qH!M|c7--r!+ z1L+kTk&zfhPmIWjiO!78kY=Qc(qm|GYu;YgcJ@2#sc-FPa2w1KN{}L!4N^tK?&kkf zezkvpr0x5|ZO0D-zzDhyHGO&L!siFie(~|CPxljmH5}es`$e7QV6}07Rn_i_3d_!I z6`SARv1!w$w^uJ;v~W&waZa|rsBreGRqGEQIpy*U+w6lj`w+-4r|UX)=KIJ=876z) z3#D=dfMMAFMRhUzu{rFa9}}t3j6`+95@?Z6I=n<&O{p>JOR_75cR(Of@{43AQdd%& zVU%2>&ibkT$0^S`wWzHWOeg9_Q$UD4#G^sDh%HB0T<@^lIi$M!0DIrchX;>;8K(&N1hLi&yOMfJ7ncT-sgqj!kv@lJP z@=_CLJQ@hG2P+7#CRPr~%Y>&I9%7Vt0LiHA#QtZ_-U(+nT!Zwo1H-TfnW(YAQ>-HP zJnJA)8nys{XS~ff(h(f(0D1*R(IO9$%lm}TYlO=41s&$_j>2-2ZZM2YBmxdZ7yyhs z$V9vUX&VSKEF_M#xW~RY(K~yUNjv8QRpC3@S(_x;%lWDWSVEk?K~ub1re7+`UZ|b9 zR*}0xKXZL{{@cpzrBd}`P2p;ua!x!wiy_JvtLJ6#bQxSNov+d4&J!zgwfecb{DlhL z90tfNX)dseDbCK|XQi^`43RdSr=oG?qO1b8IEN?Gixjy^b+KAgERp5Nm3jtC#1|=P znH)NUmk#IRN$JVSthfZ0L{_wZV?|BPXJ3DH>ePu#Cx2);d-}?SbJmuNuJ+cTwPT>O zbFkAoOcD|CY;|&xzGfV8iS)v(de=>y9OlHE;G@7Fyx-OHsHgWanN4xu7r2jeRlGmJ z$(e%;_;%xn19&@O$P0#abUz6WjCT!9bPYW19-M#`?8B%IKvypaticKF02_XZ6E8!v zR6v2f^bcV&eb?|yxa__93RYn7MehwPA);}E6cYPx{N8ut-`&H%^j`lBR@d;W?rXoi zR(tRh%l?ndd-jzW%>b|}OPvV>S>xV@ef#V7?6uT2m@PHcX0zF1HrD{k0ArPIeLN#hGL_<}SpkH+Dp zarr4Mc0vX{Ih~%tU}Z3v3@$I7!%G6Nr7?hEQ7IW=i7BAKQY8E&2{TEOnIg+f5~n8% z05J3{R^QrRYIIJA5Pjgjse5k(82Nb zXh8ClP(*6F9qO&gTH=%fo2(@g$YB8PC}z6%O)0IAT*Hq`+zg}cNvW>Ti8*ySX#{xc z!XX;iGwYi)C59GG{TOrijG+;+hiryT0_^V;#9G$m=V=IFQ+-ZY5POQ&e7ppafm_f% z49&uSJSr_5n}VN%puncMHAOFKCyZG8Kz==Nl6Gb)S&B;a?XchO=p4Lq$#weF#q;N> ztX^~;0+xtJ1NXo#Txg=Yl`l1e#%gOu`Gt8Ll<5PYMeaaZJT8-|E4U4YX^PFD;5y)> z``md?hwYlfc@upcR3QladvKhJ+x;M9;_&_K_5AE1b9Ft~92N@4f?tG!*3_>&2{t$$ z_jOHB0u(pHh(!={o+#@eCKHk#XVUnLFHQG2C@Rdt=pMs9urAyHb9a#|yD|L90ncF& zoB1ejDz!aEg+<{I%YKsC(Tm;6sI8v~PfxvIa46 z!$!th;&f&&+77MmM*uKSU<@v8?ni`VPaL4kf}_MkKMs#T0QS@ht81d&2MX*d0?add zvh~i=je7*yYXsRV_?l&^f^~wd#md}Ox>@h&W^c$UT*Xz*Ru!xgX%`nQ+6)ZKpR+N$ zXl+*hDvn}y99=2Z%+nUEp!4+-%{+-_t~z(VE^oeERiMxm%d(5+;-nxN%4rb{6{XdqI%-9>BQ~NtVK~ zxBkA-t^qhD9qsBLBM$qg_&45R?7xXDkdh4A-_ zvrh+K2Cly9y8cVo)mNlB?B89(SUC)BfZh7N@7C|HnLqxx=D^3*4SRM}8n;&&%FQ*^ z^}Fl#e$=r4V8gzT&2^1Nlcm~HW3B~RR%@}CYHEze>WY$*ojZ1J+gY-GCy)%c&@5Hv zno5(UyxLr0uEV~ul8WsmWm`*1x9%+6vc2@fZ6%wxmAtpD?ER9e_ezW#w;MKWuUfOE zWcj-vEL#80>{ZJOm(I(XTd2s>O0qQqwUVclv6UjGnwKg|O%%i>aAQ*VN$HaGG)V?c znl8`_vu3JAS`kagOlPL0GH9864qYTrgMldCd#!l)$Lj5r)r7Q@I)#o5iABv4%ncDj<2Aj*vArQzZO&p2@cRy3Cp#ihsa zGGlpZ(cI*Ceo7oSDTbRAFQmosQfH(_|1);RKVrlF6&LZZn26WoVqcHNF__a5;=_{T zqBByW7_^AYX za@-}khz|mo~P0jWl#eq_P6JBJ&!C#cZMSWm(jZ%$P zD8JCKL`qemeHQ{7s{fHVCjJs+lv*OWeCQD%u7Qw;um{e{N8Q-JjDt>adNOehPz<9U zZHkCE*bwdA@PRuO97RMJ?z@nnBMh)d1%|c`{5KP2);;3x8gY>jsj150RQ(Xsr0|%8 zBKpwB;Y5;!Myohn_pOdQ9i3ONw0fH_fPQMm5i01{AffVvWVn@ug+$75NOdb;CaJ&8 zR4yVF;7cX>5`dv5BmNG|R=k9pXSQ<}T$ekCZI0{kkp%Q|Id6L>N2|Ep_sBLFoH9T3 zdLMa7=QYVi3<20pna3z};cNPrAj84OWIiOm5Y&DZJm(~q9z|96Gk5oM7Y<%|M(vGx z`(HRo;}_t{7R2r=ryoZrlTjv5?Y=R40EZ)cafBEy>gB*)E?D@;a6E>n>mIjtPjm*K zcH;Q132X4FHHi67NG$JYyJrO00|28KW(Qpr81F!7HQMQWfOH~9mRMaJ zaS(T+!#D0E4D0ZXbp(Kg6CEJGeB(f`OYX52=g9evhikWf3ILPn-xg)BP~@(bXqPGT zR%wdXgr#S&5)(5yT=n9mn@h^}96EUXhvUt#PGQ7> zz4?OYa*MySy|=To&q~^v@h{zZ9k}Ij-b6!(>n1Xcw1~Ow*F3;5GAGRAyMs9fz4*uO z^};*n4$#Z)zUy$^bvSWYj}Nz0QIjFY0&qI}q^o~C&&5e*CM)q3y(w)jfc| z-{9c00QP}lFD8aa;T{-J>SfQ3U%Rip!v3)9zXXO~z?Gn`0z)r@*IxOCL5g8(GpH_% znELVeL2PaQ_t5?ST)qE4uhoG9+jpSUV8(4NliASFShM#40IYt`NA-IT)Hdz~`Bh~y zS{mwW>uPK3K#u{yD$B}tY~S%AtQ{rWOUp|u42DWT7bq|)(%w=>5`n9*-+5cfww>kM zcU5fJQTG0}k`H%PZ7DNtEwjA8)A-)5s`cAS)_k~q*{1gZV@uYrpSyBZ;i9Gbx$|_h zXJ-}X@p6QTqPQ4NSS%+zmK~WQOkt@w+-#vFUn$E|36vs{TuA3;rZLmm5+PSEXUpX* ziIgpp@x*ejNX8aOSbPafAZ77HbT%L47gH?Z%CmTK1?aGd#N=pNYMO}6%n~s)5|&QR z)XLJ8f+PtmQItuOFj#590RQzBSNaop5+3S<1FSaxD~TFmQl z)Blwa_Igs}oAD8E#KL9FjO2vyl*EYC_{fap7-o7jJtaIXKAMq?303@zWLC<&xdlz% zevQ4!r@o~U=qZK~bdi=X;1;z`9z%eg!uBsT&>#D%<>(hpM?SlF_|S!~J~{Wr!PB1} zIC*g2_aE>1c7MY+`|1wwsr{nXbjVb-&rse_UTWIDt#sqYt*ckBU$S`V+*yTrdWB3i zt8m`hHSc}?>9-!&fWtoEat_&HVN@EXav;Ot35HVMaZO!Gs78ewg>V(1G7rg|4&s|2XYrHH zY+Qt4lm!DZDDtm+M!IH-bWXaOL* zD31dw6%vQ|SMktI+2~0(7#FfgjO@RCYfC0wrJh%;k<#d4)0@7U=I92 zV|EX4>t~R{4rCaX5S{)pn}?9h1B63AeY}6{r^riH}1&bB%V^YH7|Qmb6Hjw_KUGLYA{EZ{CKYMepinug{&gNuIlm z#?4M;W+}55D03EZl*QVjl~U~-WzJllRIkjQt<=qzY3A@{d3>opIaA12xK$asAWh)eUVrdRXpki?)Y>tS}mw*Dxq;peJ87Zms#6&uaowabu z`#W|u?B93z_^}Hozi&Bx5&&kqc+PRD$=h+I+uGi1>+HALhCqS29M?d8c?rF6IVUfP zLD7ZQ`8xp|0r(JmnSHpE>T=z2y6*tM>;N#=Z89P3A?~XB;Xat~i?IesEI5TFiS_*` zth>pec39oG@AbS30S5G%KyD2osQlMn0<>VkG3Y9|lKK*m)_e1};I)^*8^3_k8o2$( z@K1jZ{q%3Ry!POKh9CUz;rst{{lWkHTJ64%D$RAh~O|2Y{JOMw6kYp{{Yy?)t`>np$I(p{%rQ2T1_lUQtz2QB_(|QC3z_UQ$`Pv$Cqx zY$>zUmKiMNRW;j7s(@SD3BY!gRBqi>w!O4!Td8q-scB0|^_H^g4@wP)v8}sSy}M=Q zrVm$b+OlHf=0$6eWVs7x3GxI9;;1O@ThY8}alG&ZUMy1yQd=g;QHnK6p;9VRh#5S3 z28Yg-0hts6wVJC?^5rUlOeK&ggi@tIn#C5W*dhf-tYAwNOo^N+QSxL;mOzr2K~H7V zg<6@UK*P^d^9!@tIqD3hEJG#ARB#yzmb^g8*NVc@V*fia=8crZ>FFuq4BCv$#0Yvq z96Koq3yBHQ%$P7*W5sS1MM_ zDq6U1&E`)&KH_!_*loQ|$AFUn49?CR?(3M2=)DE#3U-sBDr7g)(k@xxk$Y?H~-iAyE_0+ z|1qox>r^r1rqP0Hjd|K@qls5!1ZY^8~e*f6V0@v3t-@j}6Y4$1&=FTV0Pk zU60`#J3XVI7BTO_Kk_lo|nyzr%m?p`p=w_yiF|S5~==eNzMwHeucVl z4No;!l)X@0yi%l{C(|#^E?zCuE>ah*)XaWcKlh!iLSWlc8dn!jldwg3S@}ycM0#=d zeEqCdIYrCV`uPgYEN$K*eerUUJfAPlPRS5ua8)$65&)LUgab8KTELd%bESHoNGp-$ z>T>3)v@=j_(WzpT|R66JDazd8|%L~`t_ObzrAqk z#HDj*I?kW9wOn*{Tn=@-&I zs;n-rvXoYu0gb!L3?=2p3WEhM;dm!7Y$w31^23tK4}frG<_}9v>pm!1^Zu^Y@9kXo z;f_`BZ(j7y>YPRS>8jKy-dmBpx1xp9V+G+!!Z?2* zEMBgPo2M4!>)F|ArdpCIXLD3+RiQ$l6-1=R{VO)&^~8kf8EN6nv>EiI@XW**Mq(mA zC4rY1#fS<^3x^fOij86?L}$iC(xM{MVj?r*qe#s#ik=Y5N(SvUgBCl378lP>OJ=9> zMC^T*N}yEd$)onO-@j4>U&p@&fHfce9Oy+v*r#W|_~`U! z2Tpvl7w84+=$_gy>#IMr7!DdM_f?cNmX}s<-@bF>hRv&1f&yDKduBmyt_1#Ka_6jF z{r;C#b?G6!u~VOnQ(z-^!_$0n{ba%LZ`)UYM_TBB9Q(Q_ z$j%D(8{w`-keDncF*a2~^pR2|E&vsJ7jfhjAOO$z3kWWj@6) zG!0LVvZM}3BPTA(c8JA8;^1(Q!C(L|yXUUmjXsVmm%ZoDUTV6CO^&#;@Eb~x}=L)Q!d3^SidG?c?X;Rrl;24C8#z^5+U zIm6zsAc@`pdilrNgX8TWwmhT2A-i{!XsHQMW;Q=|g8{u<{xK)IHmJ(sBUH?OQc?87 zor$sD|FqLRg56_CKKR}Vn}4F+`>fUV?2-!<*wc%)@vlx_%UMuLmn=}_ug#jdPNZ8R z%~?j{=ZdoD%W@Y>auy2Jb5(guRr)2mS*!DxY|1JEt+s%lH9rLu7z@s!^Q77Hk~wOw za+WG@i85z?cK%|KqCl-*Al1y0sES0=JO)pl&Q_*#vQn6ebdH9`%wh_1IFj59j*`w% zilutFvOt?Rk0;6I3RLNIE{iMT@TE+yl$OSePozi3(PNW%IYrAiZ>g>^A3XfkspH?A zKS8#^&YiI~UvRcv^4Z$EZ5_Q1>yXnn?6hB{7KN@*I--kI6ur0L58u~y4+phlOXzK< z^M=zEdJ8$>4Gi=msT{uG120av4W-KYx<8;S!~fLMdi`*9HS^PXW?I2)|{ z>Wkp;a{yTPjaSrV_sw6ruD|NP{X5VInRE+Bs0`fsWB48r>(8Nk{~5afKUW|84`?r7 zSl?~DY3R^;fU$sI1yjQnt0UVpmo5j!MJ!^2$n6jlo=3+qkb`-$#~) z-NxD)qs3&dt*vdWuW7WHECz$A67W=MEC<+GYK`U^Orol+0$H}Rth{7bMfvv1itS~U zB^AbUG$>Y=RhkUdb(Y%3$|_T7g|V!%8j!iQq~gO;pqF7sg=u@G<^2-F#_eV6x9nQK zZO4*LYqA&Ra`c=yQDh{4dbBt^RvH;AilE7pI2x8%FO}!0MN%F|$QH;Y zVvS0y$$}-(s3mHxM4c_sU3Jg&;@9&p~@( zhDs4IAEDxpM82#f#_6 zF3Qi7OBIFrv)8VD|M20H05G5zwd{_5hieGEh-Bm)*}}a=)ekWMjf_;liOhkYgF!Oo zlI(=R`Ii?92>AA$8tfZDbW!=Rf01d5UQ^X7FB-LRV{Gb7e~JwhY$)nbuJ9D!)+T3@n_Umo!-51VZ zA<KjgcpXP6`9&>ikQ-@8ph~;= z(vWK~C^A6K04NVukOO2}3j+)hUAQGi%8WRU z666=0i(qdu+6Wy11eha;BFqWU!V)4@5p7))#6~{>^6QF!w9Plx;eKNGjDY;YP$(}k zbRfWRoC=C9->40PeIH{U2iXI&1J43vz)Ao+pzW9=1@h}@XW$tgc*i?@&o6rbV9zev zMlRY$e`tNM?A`rr#bTL$rEcCvY2FG|!D^0THcvfUk+(#yU#!YsRUp3KhX1)7CWsT+IFF*VKo5N?mKYsD_$t&m2pb@c+^fGt0_gbxkHro&>B*Ghm^n{T` z)e?Q~TLC<6{R#g3vCSDLUf#e2MK>`7VS6Yx#ZxnH(V5)yq^oxUwf9+-ebk6 zWBqyc-hZw=_}|wa{Lj_<|AuP@f5Mb0>i#`9f9<^m=t3i6|II(Tum1Adma< zjO-|_GL)Br6ss&Nsj4ii!i}+VqZu$(ZLY7a+ha1ROV~NYNbdn zk}8A}IUfX#P$pDpvy?imLZ?&ea^%@LQmsy|&r{?U$a4$jx%sl3TxoU=SFVD^Q>ge# zHCLfxXUVwP5_%Rtj-LjLtCJ{-vc+H7<0N9yO&+-(&kYOi| zb$oxc?fBu##|~dQ`eiGuBcHbby}mkh;fq7(zxd?rXCI&X^Z*60Z}!%I-C!XbV1|zk zRlCbdEjxCUfB3=njT_!uwQ9|h#Y^VS1Ar-ID!s07;ldS%K0Ru8^x8Z7Y&h?(&t~tl zI|p!smnbljfp#a*g#((&`xQsB`T`H(6OasO{<)jv(}Mz|xD}!osU;%7{t7t9I@_eK z`b&pVW&uB$ib-*fLLId+M#bq)c{eC|H1#F56GnI`6t%7zXe1{saK)blz#VUTbUXX>PWjJbAjQsi~DD&;y*#p1sgQ z!VdvO=g(hg3B^ZUg8u==NN`k36A@t-n=UmG`PFpsG6_XQF?Xr?a%;;KVqEMzeXjG& z1^<;!0N6E~{TlI}{YWzD?|Zzs2j->>CcZ}=^rZZZy=x#)(9`kQ<%L^AL3Z#|Pznq$ z(PA@>Bb_F@?ofx35PEzviR7hrA~D++!@baO2#~-XvfdX+DH5fD^G!&nmmOZIz$gmn zZe$x7dNP3?LQv9pE`lkAAhzr%#Yi_;&qPNTmJKQ6I~=s3b)KXkl3XZk=OO@WcRy(d zUFAi3;bxl;&Ge2?ArZH}Y(5O*u=?P4e+uNYk&Uzt63Nl&dD`xsX!ilY##@|Y&5rT2 z?LY4@eJ|Folt?Nm zOKI+sbdf%uu1IEN$#o0Gnt3c?zD&1Ji9U`c0!5xmKPPwga#j8!t~8%7E96V_1&RWW zTuDld&COJT^_0-aEtBbMkSGCfzArO3(?isWL6LM&171WH;uKRJyTnZO8- zW$>l?^&56rOb0$cbo|Iyr;dGd?&J@x=T3LFTy(d!c69*2I(n?OAzaoWJ18)O(+wxQ zH|*DtUY=WB{<~dW_k6zFB)9%1y1D>z#8OXe9d|t#RF4^nBv$HiPv1y)|48=$+71C- z0IGrOFM(KtKmIxZWV-e1(2u_k-THO-_U~8k{CVxppI34Hd*IgZS5aX7XZY@ahJX5V z@b>Q@woq@~|Fi$j@4%{|yMGSc{vD_Wc?Ms)lcUo7C~^A$x3 z-5h=P>^xPWMv)^|XDd`18DGd2iMdL(N|mk8%GSfBA}3Fx)k$;olAIjAM$63tJ*8wy zglSw(GK&#MrzNxKscd$Jkjv4C7-~T*j~2;FN)~1Eb0mTs2~W)tWpibEkwh;^6EdbH z#lMx363L{;@mXUU`1udg{Ox{WK5rt_TORYZ_P*x3rmj*qeVufM@J&P zVrMX-!{8t-c3Mhw7%es`Ga;2lGnQ|)U--WB^ik`XV^#t%P+)dyCv3_zf9hNNiDR%j zemL6p9RTd>%SXSueB{d(V&6Fb<-v2G9|U@x`Sc^oe|Y@Bo}+u}zpS%-YBGINjUM{C zlAXq_TT0)1f9v{n@2_69cFEGkv*r}%6=)RNoa};m^Ox@1|GC}PgVPo|0bm2fPY(dQ z>h@rlKIVIpLIK$TL*s@&_<&l$u7};-54&*E%0oXU$~>Oxf%WykryezH3grvvFtSS& zx|RAUq!i55rq@(?>Thkc6WB02DZePRPynNl<0goj0vJU&N|O;Acc@fIzH1B{b#aYO zUFiyrq3L7t(WvVuNlCQ?kUp?UPdP=e$w#1K&dG1oJ?08xo-Yb7FNWx#f!>30aAYDg z$ghVk_dU1!p3`*~4xBh1jBJBhuUk6?E?;t;J#&#N8&b7FKon73mqNjat!+5068oCb z-*NF`^Hd@6!o?O4UQ?4TF$EYnh_?GHB&)vd{DqE_XY7~TuUqX`vGoT(s(y6ioG>`K zzV8M_=ppUZ-bc>B&yE0=J&7IhF`Ri&g+$~r{>oT33`yk4XN>9`$7i0XUKCQ8)r(Th z*@Kkwk~kX=e_`rwaLkR52InG(JqmRs8e^35!DGO@7mI)|k!2Dw!{_#%q!4nTXs4?$or)Sja8N+yAA8~LH?d3p*J)vrepu;c` z(fb4dMh#fOEj09iz-Ct_1*=aPDn`#=716Om&SVmDM{djk^ry9Tmo% zXF_T@y;1+T?1w3J~NLH-KnU}AdTcDn;Qx<^^ zlc{o*x?HtPE)1#Xl4HV>BB#aA z2up~HN=`^*r7^PvoE%9;7C%u;PmwWFWf>{r6uKf^mM756RAtRnr%70ECSh;#^z`%? zE-OjQNfNUXg_*JJg5O{rq>Grw-fBAj1$~XVA)l`(9^o>&to)i-)$8lK?{n z_I2x#FIvC;tOYl~0AR#_h#g=boj&x@iBCQ{zJK30dvRbIDimtpIc zUGHw%vTp6WZ?6OewsO#hp_<#InT=uq z#VYE0B6R+0JOaVw#uM|^V4i3Bfp51ZR?)cGez1n^`&~)(%vBIM-gL2H1SPkq@21>PF3Rw*=p{}}2 z%>$!!7WUbbB9B0g2eu9K=|9!fAMtDNhaYcaYjF^@LCI{;8`A12rhSJa*Cb zy>N9COa0_@6{@cbry&z}BHois?E+T8E9fJxQFtw=M{pB=hocJ?iG4!v#xonXc2V73 zn7d&MPIP*o;$~HF0_jDZhsZ6UmycX}Mu_~v>j6<6-bXecG7Kk#Q43p}F;fa{ki_mm zJqQXo-$Xn5564L#*m$#Z^rGWQlXK+Iai1c8LxymsG-s(;w~Qg5Bhf8o%ZeqMd79!? zs-hK2{nDa&8&&#c3f&TU&QiI48DBL|kTs7jpOwbVk!cqQ6vb(*ECHT_&n=j{N}V@9 zD|fDb*3yExE0x)EMe+hzOo1kYt4L)?lQSggEICW0VF|Ovioz_-OoggIr7n^w^8{iw zhcD&w#cZ~SPUmOR1&Jx_u;}!+B2qKi@|A10l$15@+k52jmuJ5H=E4u(HJ>@z)_lR$ z+U&j3;_GbdvUc>svRel*R?30fU=BOZ38OOUJzk8^bh~chyTyGAmW#~g@%!%u(Ukb8 zyKkgtaI9~5Z0P#Lz_q9SS7A-`Uwb-mZ2}lJa1BJ*i#~wOwHJdoU-e#nP6mcO8w3bl zf7NsCRqwT5`mg_jWV-rd=++;DH+~z$So&Z4FmE3C^rHXfuU$jWy9b{ST>q_S_+{S> zxbyD;w0^wox$#?I=q2V$UHcU?;RbQi`fFvz(u(RzLv2mDrDkWjp~7IU0Abb8SiNuG z4im=pmQRS&&H+XC@12(agj+W@;icH99ROEHOMRA^fel>2Jn_y%`l5n?Yk~ zq=Nh`UY?4RE2nGu3F72raUw^PsVEd`W@U->vdGND*Arr2Pe`1eo)*c<1O=8PVWrC1 z@q*L@K}sw)ZU%i?SjHREY5x_G_GV;y*!0xt;pq`knXwTdyy!7+rbfJ#7(NYtXkvII zEjm0QT$imnb>vg~nQv^TkJ!%~wVnCab_V4a+7VBE>o|4Xeu`{>5r$bOd%v!L0z2|K z0IWIW-8gvW^N-Jb_7NzsQ-=8+=BynK{v1K@{&8sr>5fR{MO3vB}E@^F)kKI$>4hb0|m z#B+#05rZ};)=uHmM@Ae?Ed&_V_(g)?5MZ81plqn#FIXPWBkZ`sc9{pDdyro4`wr(l zhx3lD^IFH1o|Yynu&3!FN+{yOAaT9enB3Y5B)ilOH<4KSE5NYx=c!IGlwudKry0G7 zaDDT|5X>%KY=Qfr9Uoi7+RmJ9ztH6Gu)-X+pm*+y`wJqfWTbq!h-ca+0ip*%?^Hec=i|cLs4{GL{qj zpS${BIB**BQ{sll5~3Gw!fGLPwt?;&XAjOz#$b*vIA0lu(+d=KfIZUtg&SsFBV=O? z+rP;0l&77ZrycI`4i8oI8zmiJ*di8kCzAd!g0v?9unx~78;SIFc%C@DBMvW|(@-;3 zMlp{ABYiOu5mlXcyxj-%8o%ToZS_38%wJ;?X0K4>y{#x%EzvK} zp0!SqyHu>1uPj)m$X_Z_&ei6vl&BX-b&KV>OEtx7B)Vn%tc4u;>_mn-lb^#9=d#3l zP+c zx12rQ-h9D%smXo0#S7<5o$Wzbw$5$`X8R7gP@Th{{4Jyx?tJ4+OEQJ$wg)5^8K^`J zQR?Y?(l;>HH#puqG}b#b0t^HB)iW^K+dtYnFw)ciq`U7iTn4azd9K9F34&%FZ^!K=Uc25|a8*VSKf0Kx#yq4oAoyjE^3uc)pv)YX<*EITX6Y-V$< zp|P=Y_wJo0)AlL@h$&-Dz0p!@!Y;5{b4{(mQeA2&+gSm;L-ZI;l{Iy6qp7aJvTuLm z{*U(T-QQ5(Skur@Q)4lkEFjbna@7Dnb9KY+-F1zP)ux&%qZt6Wv)s57Fj!MpUEjFF zV0in3jq~1FR=j#{-twZX`8s}~h^fzH;i>M#gnK-B8{AvrC`Ve@!aG%K^$F6gO$inNaUm>Fp?sYBW5Iqg(XJ3i2w@^ zPe@E;G3Z%RUapp3m?JFGadKs8Dtekcm7C4Tnkm!H)-V;^*W<(g85{Mlg!ng8l4hhQ zhBFeQc`1pK^aN2#EH5sa9Tmoy5tjK@c*dI%nQukW!^6_TA~HaJ#fPQGOs7RnONo3d zG2)Hb8E?dehbKoz#D%Y1KEL(EQTwT*HeBCO&0a{a)5o2sjyq0%Yd;yHS13E`yTdsB z>sz#Mw0wQ2iMZzhU8g?TPx_bl{eVqi`;P71eWbDe>xSCTEY%09Dt4Eanzn8=Y~EZ3 z3T(~l^-CA8m_K)ZQ4s)4tx$`F3auu$ti*KrlFQl=uyzCiU^X;y47fbQ9^ZAJ|8}tJ zo}V;L3|YGdE^T|@&u@8b3A#+oiM7A`ES|2l-HbUoT3(`w5Gm9 zh6P4^AbR{yyu{r{u17v0!+Nlni&6tqBnts-(&$0K3~4th)I21$2z!-C%@A*$tV2@j zjY1;@Jn8}Qxc4a~zo>^I#*HVFKNSC|Q~h2Pz?^}{b{|TvDS$cLzyq`$x)5EDoG8Lj zfjK?*5EbqR4%a=G>nFSQMn`*pbCZpPA_AX~Ues_eLZ;T{rc0Mv+Adx|S0UU1)ELz$ zM$NRmNZP_krqq?tHrU0#z~7eR#C+d779=;HwSb<>919yi^t+r$!m&p~Be zx$nD3tQ5v1V#068i-;8F1emMq zxw9LCpd8rR{LY()ojfj9QjQ2%TV)r;PdLYAa zc2^KncRNF4QvhJNfMJ~;pchsPagjnI)fv_X0E6WQhK*SLBRF8i{RnM`o)L$8#Oa0C z5l>=VXcZZ=1;?>VdNNQ7RNZCoXsD6dHP+-9ybSyezBD(3qfX3_CZvlJ(?v|anj_Y7 zL>jR|FOuu|5)Dr%S1Pm$nVLc8r=~KJQ<-r|jEK0*xD?*Zd27ll_5#2Tef<65FVB8= ztoihbww8;az&w{*d~K~kYdeUrAS{wh5ATQT?$AUPjH1UBZ*;4MV(GnI_W@mf10#L? zBYgy5z5OGh90MHFOg=5k}@uBwuq z6+6lerBwhwv%yqbZLTv}>P?ON>JJ>;v-dzlU8A|S-VDft16Vb+@EFw=OLa}1rP^$) zG8hfjmP%s{$gfg!O+`&@xw)pqXxdg)xwX1tbH%O=JGZRZ{LakP^JTMToLshSrnqo< z-jempm#$s6;O$j2SI!p|h?Au8DbggamZ>Tbi`6WNRv^_Y7;-@pKO;eykgiGO&7_HD z)9I?TWNu1SYRt6Q=`#|-!jmJWBgUemQ&SSynHdTRGh0p9X)?45x>iKjav5rdD3_<7 zr^%h4%aC#Z5k38Xh0pkZqN4sQHgZ~W)QpUnNKSk-FFu?V70w8cW=BUcBO~ZDA~L5% z(x*o;!lLMrQS?{<7~J`dr0}Am`0v=TH{&BBl4BBS38g#U?*z$p@*C@kqv+Hi zePCEZJoRk|U?={f7XjE2GQRSwOGiI%`T7$880lZe-7jF+iG%yT`)KdC2lgD;Nk-tKYfP{NaZHu#!z1KV1Fxx+RO26Mz-v=4Juyxgs`8@WJ~#+b(RiajJ;W9myU z)~_%z9na+hR7o)Ojep6wNmWN}s!@ybE>d))jEJO$iN6B1%SEZl5Won-P=xs?9rg%& zzerII9USflZrlc=Z0q-NR}5))&+fS0*?FV0bMW#N&*@VafH$OV8N=zDnp*yrhzK;g zf|Dw7;Bv^3c#)XyaXYNJi6}B)6>2z4O+?#$Gl`+X!sErJ_V0eU+4&i2dEugQL_OsG?9^IfLUQ zlH&z14E^q)t59`4clEpko?$t$?gJ7HBUVLD}gg2ExhJ;lauwpsUR{+V01LPWO{e_hY+v#OelyJqCbb9~fn< z#~Ca56FUQ=ZGo{iKLx+>PXE&@zA?;^a*Z`RMw{$o7i=TvI`40(I!c!=6zEop^ee>x zs{G|5-9l;JVtL^*S?&^%3Yaugo4-n^oU6)RuAj9|SF}c;n42NYqj9qn7%G;mFioI~ zXUOOholG}}BhFz7w2JJxQuS=UM9&p!`QmJmELSMa6Df*VB7F)|L1U{jx$0!PIF%`3 zin3A}BBoF!R_Yb19H~SF3XIJbrP3JK%S_|MC$r;I`9-r=mR0PotNrZI$HxzUdFI%W z^QYiFaKQ--yVT^paw$NDTXxy3JpeGfZ5ZAa#4CT-7a9O|&j*+O`&}fjAc(>XGkkk` zeuit{dcXgkAHe3l4cNo5L_8qVoXLYA{D6X7KA%w#s0&Bg|hVvUyCx@uq;=&&kdSw&T8WmUPsXas2n zkB_a>H8p5-ga@x{tUYj`yrynzg>g$|#im_b7Oq>GJzu9^tXsNa={sBATmQiaD>rVK z_x2)rk%}fwPnIMxl{CI4gP)Zq&EYBX6}&8Ax`fHd&XmkclPye@&dX$Er9`JkO^*wk z7B?d@JuWgMc6wrXcw$rnJuO`<$W+Q0T2*FtR)$uduHw_xd?84>dD>aai)JpJl_}=@ zBXaux3Y+m?QBiLu#6)H$#IWOIxN#Be=xLc@GcsmG(jy`n5mC&r2*$K1=8RZYWE>+p zDkCN=J^Ib$i2shC@kT<}8}UG|u<(@VnABLFfO&AY*>?806##bfnEf<1GNb44%rWaJ zkY&egr;b=p9){KN!{PSt4+Fj0jvu*v?CUGXzG^*!>Ae)a&VG)?!;=U1o%nd~_XqZV zyKnc=J@sGJTRy8Xd{SM30NcK!dh-WW@4vt8opqa5u2{Ep$+9_f<`)(f>U0W`l*<;- z>4JAR?PzOt**d%6qzo09y$?i~(=+7p-|z-*`)~&99cmKmqzmz(AJ4oV1~FC&wU@8^ zNmuui0HK!;H~A+;7y=B5^>^V#sWbwusqHSbOAwI2CnIVvBIBrR3p7n2NV!#SBbU8T+5vk$lx`iKu@2t^Dlqq0vvcfR$IlmRKi3?%B+g#L zl+R}<<}j4AMLA0)`o#=IF;}}lp1(?(vy3O7om233{;c)dg15CrYjTR#3gojH;=)X6 zVG38vQ5G}g1qm!wrbw^MU8Kohz?bK0^A_t1S17e}W$IbdteGNN9$%Ep738q_ImsE= z(VWIs#?kl~C?(Nm@Z>a>B!ea93*|hnSS(fwg$g=~FeZ)0N=RbH#M86&^S13URaYN4 zuFHRmia_+?UmzvJo3BcT~OcUhai$oVcMC&StJYH6fYw2?#Zuo; z+t_HXuc@rAEU7FhsVJ|iHr3RF$f~QXHkxY8je8nk)i>7G)>~?7Ej4gWW9{C3^?UZ# zQ5Zu*qM@v^vaF)AywYeWGgOsT8!AoJmWKMeJ$q{!_towDsMJ)mrL1yWrD1DD)yC}~ zF59$X`Mb;C*}C!F?H_E|x*0IGe8Z~zMa7Z=B~QR3M?sUb zX;P-7NL{>Q_JTDFW-gtb%E|a=WY|9=Bj1RNjYvyK;HJd$6C&Bs;q0hzR%8UAiyjfh z3XkPRMzLl@vm)ZzF^TNBsEp`oDUoj`MF72CkDvbEanq+Igh!;tM5n}Pv*jlae}d-2 zlizfnM58=e$E>Hnv7H8_9krfBfOVeuy6wBKt{nfe?fBPMzQy|C$&H^$(DER z&O1)$UAz6qPHTTlQ|IYZ=ZSb~xpWDQh{QmT1;p0YR>CUGj+(S=AiyqMxIpzjpFMj4 zo4^oSXj%j%ckv=6<}SnS7n?4hJJ)gIr1eVMAc!!#6d0Be zfxBchM#w`Ssu@xNk7(Z@PYAVti4L7$AAZWX;CkG$8np*U2*4)bOBbMvL~r1@FPxw9 z)YbLOjSB3!Er>}`SRTZN2iyj81&Np@=OW0$;J6*2g2HH&vJqm0)SEv|UA>xo^4%+Z@ny#kg`0f#Nc}X&pW$t91`xAcaH^!Uib!a>P^O0Z+TD%SHKvlX zUE51`>;iz9Yik?#g2*zMP1QA)#yz`t@7-P7SXb9rSKnCIxVvHB{>B3z?b&}|_rCoN zfU&v;OSP%8%1~Bbv8%LvXK6)Ag|V#4RB5TNso%HHRJYq;X(%z6wpSXqRa9*&HvmvS zEZh0+t`FYZ`N6x}H?QBaVdJ)q8@Fv*`R>|TD;H)j(CZg!v**aPv&6acl-YB#R7EPy z+^pC->L^L}*h8Y>liiu;!M5RZ)l@$JFV)&Z?u(%m-fcQ%Yk4lS8p(ih2Hm~XXuRBj2 z?L7HS$H{LyaedQy;)wP1QQPSw))PlNPoVsQ%l7Y)VFY7j1MJwBtw%q<_|+%pzr@jB z0I(lGdwsMA=yiNw!?8VeM;mLts4*TimhG?D)wpY0{f=#Q+qad!yXk|~s{vqdFIcd6 zX3^|iU5-*A7l_3|nNX6obotuTr~bdL-a9U;b9?*F^DF0^m>3(q&cHB~0cKz-!xVZ~ zL9u|{sENiH6$`!hBB*J$#GDuv>7D6)hHCGF#=16DOSFzW2N8yEf(XMJ z_cP5&$9$R8Pvk9E2S zO6ApoK7CgwsxH#bj6KWb{|GbEzub?FU%dmp*xfbU)jfb?oJsMpv$GTXpnK57NCkOx zcLD$Uus^J404t5%a9!QnrWzQWR4Q+()z?Y5)NRVgp))+dy9v=@cz-eDj1?mp>_lyV zQMVPO6gjb%5K(GR;}~un8t@8hiKw^m*M!mXV&IUjHzmib!r)|ZNU7(ZH=4rbx!;H>&G#OfP+!v;erHKsN9Fa^Q3cp1BIlcIVwyPA0Ck)i(M{_P7_7M3IuWs7 z{L*8`=R@>sSV4Y085wzO17ex6Du@fa!EOVMG};?LJ9r7H)K_&R$U~#Mp`II$sk?;$ zBl(Ib&U84}La)E0)7{e>A7OT)>50zxRBw5vH9et}Jvv}b&#C5$P7;um5hj{4+2NPHdNd9>4Ji=&+F+zmMMe*E@xk)s;t@%a0r>tgS^Od}9+V zU|3;QZ9#ceVR=O<4pXVBtg1mA{0J2M-_0D=96isH&=OtgNjqE-!;s zQBz%2S6g0PRe{MH)pZTEO-JgE9&0-OdGpa@jUdHpQHPb6R}_QN1BMk9=N6ad7MFnN zDyeO3sH|%&DX-2itH>>{$g3zXsHiBaD9bG?*ju>&v-~}~4sBn*YvsmWt9Iw^{476b z^QSvf*QZIB2_sX&;#Y`LR?DK7iDfB~ahX#2>R?{F3om^Mcew+9xrZn{Fk)%AAW;+= z$)kt*c?Qwl186`L*8q2Swzt0|m?h=V!kDfsx}S(6PD>D{#IYj6+?e!*?k)f@2Va^O z+m9g%@D(vUxt?y}t}fxOZd?y9t~ZVA<;iyT;5d78J!s*cUZEZ|wx?&XhZED?(a(J$ z&1oSGFy`jW@bG0b`4QnI`8mU_->EKsi?fw4!pV1b)pdyyUS|+y7r#-SKQl`&1Q^h( z?^}$Y?qQ3%q^@Rq97Z4VHtZt(aSW3Ekw9$0{83jm!?s8fj+YWw`?yAHe}da&+=GT zq}++57>J8v7^Rsg;e}8k6A`gz%L?4G&n_{KW9t=(NGBUtc56O(L#dyTVE zyp(TVivkPtHLwGW^eLlJo}h~|%@f}|YT~(quny#r6@v}cB%)Ah!i;$eU|2asi#*m3 zHJF`&phB|-sS~2@5I4q1B9s;0HKj1WP@1OHrfH37Mr)kWTTt5p7d6HyWEjRIBET@7 zgN$h=+4kr~R9oH*8(s|=Uk@7J^y_E(bg#O#ui8~p=La5b{Pb9ue0``Sg9G=S*tO!s z4bqekWhvXD({`k;-XoA_h6+-mlGgF1D+IEYF)5ql(TkYHk*0eEh)EOJ3brtXBT9{o z&Ww!7;7d}Ylh-9J-wF&%T((&py@D-B;0O|eInhBOGQR+k50mE~9LWq4v%=&YL5xt6 zB#BBBiW9`3#`yA(P$8c$WphLgN~g<;@BRirBpMKw0UrhT2odn*Vv`WZSv# zdO9vCdb$u`$glz1(C~x`RFnbp3XEicEruUj9$0PI1ZFXUK)jq0je zbxoHxd^{|2E%=W5eFa`Nz44(i%MUEmlM3e z<~rq(TWIJ&f5+I>Uq-L}Vg_8@{@3_TZ2ZFG)t@JB;RtgJ@e$ zR!+&j^*h%^Ef`Gqb3U6NeVt)DpUit@|WeeC#UBlD- zcuAowNeG?eLkpq1GCf_H?#_OWZb2@-TpwRH-JR*->gVMZ?C%>E;>lvpcXxi*amfd+ zt_!?8-B^BpqF_dZzqf$q$@g;Sd$@$VJ9FG!IUaOAox%6^;d}Z;(0v6odbqbA-`9uh z;~MDYje}7@Y<+{;>*6=63#W0lV%boA;f$vB ztoq`Anhy!RsNksXQ(vG35dqe60sV(f=Z@B&ZK^&~UwN{o?DMkxll37QifkC)V6me1B7yWTtfK(1DbQG^(dP!+Q;=L%ysg8Fl4R@Q zjKpcI6_D;C>h($ifbhm%o|nNorF`}J!wgBTwMXXWEG*@Uy#66fyk zaHKBco5X;QkC~RII-nPh`-6YLK(B{d^JBFUPjrTRI{iZ(0_?`{fUT!%7}uWpc8psRpPJ^h#3Iy!-12s<=u*n64(%AFeM#e69oA=ZbE#G$sf!H#zQ z@bFcY3Z9@rhuzU=Zfmr+W&sBG7o-%zs6ED8)M>S|TpmT>UhV51&6{@BbgSaop5lM9rR!Ou zWucL&5eZpQ%eKo>whCj{@uIS#Q$Lg>tP2yRMMSNX#;%dauZvCDoSd;k6t^ZgB8|?H zVxUxPW~eYZfE~k&TpE+ImMcx>L?p`-)=F{AO13yEofnzH=EVZNVEMBoJ^>M|&?tt# z&@Vt35G>*fqq%};u_Q?#ijzu{_x&w}JaX;VaS|eh z!!7UpYy8f?tv7#vr>wfJtgfM;x&{DNTw7mIQME6>a9?3jUS(BrOQFgE}_rT$U2T^|&?90#JUyzquTyVJf@PUGZhYE8+gcX&66e}&QC@rrpuc@nQ zXsT;!ZfH4nW`v#QQdm-ABA zO5)OavFXB?<$UQff5CDm&a!!-DSrz~gX2GflI8~`I)%j210#GHVctGrzRv#cOMI6s zrhVY%@8rdD^9*oxXSg|dIlu3`c%I|pe>g1qo5PZKopJYTi7%bb4-SeBV~G7}A|D^I z4=sZ3C8T@t=^k8fK$;J5OF(0YXf(c;kHE*5??Vfty9Lsmm|l)dPp1Gc7Z#l<2o4po zmoHEJ{=`xBCENhhfc(NjB9IKxrM!qs-TJM%)lM%=-}vV#Sp8>D;PkIEUjx9pPGSZJ z0_>}n3mAcT1ZRL9sr#m`>SPrFtl)TAK}&H#V_|Of;hd8Fdk*i~xo6wf?d!5PtV~Z{Z)hMw$XxdhyXAYVOU-u zrdWb5YBrc<zGs$X6tW% zkqrIoX3}7U>pvV8NRipWV-yIfx*~$ah$)2_&}ZKOqYN6OIJlW|AyS1zRA6(K5UR9D z{XwMmNYXb*2iPnW?Vmz0Nl}?p4((M?+yKLB8!5OE$wJUYGU+iy3b^&yWO_(BvPg{g zBfaU7$x2L#hDTboa@2XHaVqXvW;BIu)W5|6s`WTh%ZR)=MaZr08!U)4~X>@l~+PiAqLn>|F4 zCK<$JJ<}TD{24L~zA5TAMTUowpTo4>JxFzSgNnt}sMm8%%~(Ptd?FRaH=5Dc8tNiy zHQXE1j-We{T)_&Yff#l`t5KyqswIm!IH;~K9Gik>4cz*|qzp9BV+nD3$cR15N?0bK z*Nnmda_tQ&HVXi3N^O0q#+@)E7LK*V1Sv9+8R>=25hIncVtuVAVI1&v5S+k8!_1KB z&4B(*zxH*nW~N&^)25tiS3WPO>j;fp8z@NQMXf*umast>o6V0}BZ|$6UA8@T*@t}j z3a)f{WI`6eOqR4UGGSe~Y(*eHF(^F2KQu-Zl_8X;`-RBG(JP~q)^a7u5z))y(z0Sx zvg1?N$>UdvqcXyHv7uqn!E705FS@_bmnmQ{`3!$vXt<2Sle4*U4lWr_AQuQ^TyA7& zn1~h34-5=r`iIfrNu0?~S-$DOp_+=a<3}1!ojh^j`|rBiT8Fy3G(BDV-X3#b|LCyd zvQ~c^Q#K5^UquZA(_=FI4ZY^NR&hw`nzody|%$Vn{sGOqq+>5 z%%~;dzs5TzYL2rRjrfQw>tn1&TCi+0f+65fZ6hyiwwG4y0ey`_m+aF4an`};Bs?9y z8k_tPPDZc(G>RiGv4z=oP2&e{1%-kyC$hYl6zA1*2=t|%$3Dk}qkHP$s8sc&eiZ)$FA zIo{lStg)e`z8)D?3v2^`)z(&2R+f~N7L=A2lvNeymmfS>mXlMqcgNvPnY$7aHipU8 zFy*Tl@=UsPwTCRrEpCH*;%1+e4WTJnLP=VvATdy!#*SYdnZ6}DZM_ICMyH2}m($s4 z9|R@;cR<2_vEu)So$?P>^1Q(Kg+Vc{{t@m>o~KV3&B4#{1JCysdcNn%ba3`{aPW3m z?CCh)Wy#-{Ecgci%+Yb4i|c$3kA*avTX3MCgu{$xGo(Qb3DZZy@QLvC7We?b+{3** zc;22oFAq2g_i*L7d4zd+g?T##xGiD01HD`VF_WGa2C|+L!VBL1VRrYqQ=~{pjPe)1 zQT{&wyL47@p0aP)wHMF}2Uni{>TQ6ve|_{KnV)j*Xyf-s(SmrUw)#|6`IqG-_7udz zyz1P8rTh2h?%MU~)-Btzvo^0;vm$L-pntG~gPW79_Yzn75;q?jJ@C-MLglboIc!lW zY)aLbN;9F>U(p$F8ZDTGXtCb6TH)pR2mofX*aOfgHwWB>4QQ)-Y8V0e^&CltepxIS z>StA#eUWJ$=%FY3Lnc!Go|{2MJ>wIX1W{4A#Wh38qqj}-b27*@8|N_xJaS_1HKrsN zQpfhnf_KJOB%^2+U8H766({EaMyW3BrKR3AGN<-lQd?_edM}d6NHisdWFjJ|LgKW+ zjNXm6^$HnZiIg&#A7hY#`GLWNvy?DU%Jc{a+Un560U`|akXm_VXu#Os*3*Rpng>WO z6uJtjrYq3lSwXm=;9Ip*xXCG0J&Q3W;wp ziG#Fc>CwdTT7!9pSSKXW^0Oqfpe$1uUl9ZS8-U!f;pLFwrP4S9OJ#bkG@^+EYm8V< zoE|nmSD2qE%~KkTrGE(kQ<>m{r?7-b0upr?gg9-W)EIEiMA=5ryRR_4#wERKn!MtdpELoPgB6a1Kq~)6; zqcfzjD$fTv=#XB$p=*3ls4L z(xAZbpdd~_Ae-*XcBivrQ?han*Or$aYiT-t>coY!-*mKH9O~)R0=;^AEC4VSP)dJG z1LBG7U1?O;w5scH)F>z6aca09*I?ghf48--)7;Z(?CmlS4`76cR)hH+)C3k-R?|JJ z1^x7RudqGBXI#vue_|!`E2uin=-BkwIN%B+>PIHAHaIf*2L2t5UH)-`w0yzE@oT?; zO)Ugwms$t!jZJtCRTPh>!oZ#sEo}2r8Pfurme;+}(zntqA&0)$zeWU?2F~b|=mw@ic0|iC{l6eZ~ zZd`A7o|jv=ha20|i{pbW%uEj_e^2*dA5XR)gC7zs3g<^~iw^G{M4JW$u(Lof#f4L9 zQbL3i1!C-*VdCBRt9W>pOkO^X3jo%6;`7cE$6LQ_zVLa|_bv5jn`-}wdtkL^YO79G zmVaGY@kLo_b8%r^{^80)2TJ$v%iXne_ohuBu3fun_1YMDtg|a^iKCa3o9_}=Uk6vZ ziz{u*<{bll8pV*A0BlsHozxnx>P@#$ftl}{E%(gk2WB(oPZJSF6xirszR<#hh<)BPF7oI=94@2)Xuyi@8}kSbPnjzc;(iz@ z72-LDNRecEY61%x#i9ppjbnjPGctp2`;j*|OU3#|N5%4*5GQ1o3W_i|0>B2fFBL`- zT8ITk1Ir`JzNtsMIe=Kx2k8SbRnh--j%}5K*WgniVb& z;YNoGV#E2-LUF7(GEOXx4h`Y6L%FPwa34Rmn|H84nzCnKc~O2-UFBC_9Y25eOy{MG z!`&U)9spRcb!d1}t-YZ}7Y6QNX_QwLgOkd^NyXsA@W4nv5U$J8*KGl4b+sG1JB$Op zKrb7xjl_1`FzRoa47bP;-SZYR(CaQGxy-hwMk^ZSt=K1u;T>ZWQ={WA$Hr&I$Ye_p zUgKAhSRlDTc#$LyzK;0hRqHN$_6-^5X2KsDgj_^sbZ@dO*f zMy~(0Kgg$V@I3nn(AxoYXM;O4Rye<+PaFmhU${q!ku|rSACM2zAG+yn=oce zuyi9+oaHHAV@^3!zH`X^{-?2w$3r zK^HN7BLaLQ{k?>~Zd{rh*UN+J?Zx%=4)b$k(VYXlfL`7hTF(p;g@;A*;$o#=HdZN# zYeRVf&~-+6@eHc23#XLl?ev0+!{<*A{cvjV`%`_Oy}tdb_uDT4V7Q0^>%;`9qZdDK zI)AL;Tua?Qn`*yntU23I{cU~q>DuZORTaldiW&=XiVEo1{5_u@*!j_?TefV^Ubk-L z>d=s|1xq{}Tp5d?0b>qM?#q{D_Vy?gLq;?r0>CsATEjH}nBI7ME*p*XpqF{%xn<;q#roWgGh~@J-5bPRs&0;ex8; z0ZZo2=dMskN(m9pQH%P@b{RBBGF-FiP}CPtAs5O=%2#P^aGVE9hqC-V#xn92GKy&D)mc;BFZU@|=d zDFy&D=pGuh5A>S*8XV9(+SNJK)`m;P8`&#WotsZsQoTZ*hsi#1N zI?VhW0Hy(k!7tIybwoS$h(Mi?Cy&<{1Vx3)kN*e?jB<03@nGoJprQ`3(n!TekrZI< z*lVQ?{qflHjK$kGdfOW%+BdKX3@4kIUtx1Gru2fqnjX}>9MVsNrM6CM$wQLb@LXejNlj-4O{T`WBEF)~iwM`T()dbAT=<~s zK-&!(kY0nP*L{ZR9_@>6?aMaBlhc$&puh=jG`Botwh$?GMl8zU0dz>_sCNEXabh)h@=B1#GoBu0YRk}nMx#YaXj z=Sh;K2`hQh6s|ZaC{)S_5P17>Kz{iI@faZ@T%jU5Ff3HW3YT$(2^?N@7(Xf^GC?d! zkVzB5!^NC1AuE*2@aMRA1cY$qJ9ifq<~7xneR=H2cW2LZwYMtKh}dK99~e_>u4%N_ zH5zy&peX~zR_`dRfgW3Lx2dyD2i)pv*LQd5db^AReO40oJz>yZLx&-5blo-^@0d;S zXnPN>UKpDC(2B8hPw<|AlP#yMW5BH`0NBX*v$2WkkYtFDNQ3EGWp&&nqY@C@L;4uc|6_%@yK9g)jl*Pw2l{g11<4c2L~Apbx1Rb!ec`P7 zhi?#ISU)_Yx(M>?g!24J1&XjU1YoBCU<3a=-Fx;#4{+<-ui=P`01TVJzG}IM9{PqK zj@F|Ev9SggjsdH$JW*46thAu9xS%#ax9s4-+)puwe%GdrJJxSlow3r#*MEVN=VCX% z#jd`K-FzM0=+3U*85yg)ItEkOE$VFXUd!{cR1q^`rUqrJDYvzrWM9;S#z zWpK22^Z?S}3JFLY=;>8-cB?zPjl+s-q>6Y$qrGL&6EC9f9$?jAz?6FJY!%U-Qg8n@ zr?lqhh_2CT9Jzx0p6l(TvjiM&g62m>t z;v#L%@Byk3IHGQYPgh%U8w@*?Ew9x8D&i_sm|l{2!GSa zPGfeI^|i|OhKk`(TG9HU#L6MY&ufg&HKrGo>e85Sq6&$qClOO7EGlZP=*6e{zz~+8 z!_05656n2ztAE+4egO*X!ob~K2fvUdf5?qm%ZptvNZ1&WyhRYdjvbj1CRr(pTN|0U zRvfoFB7Svz##TvkwkUoL&`T7*Dnyj(6B6ysk_M3hD?TA|p&UDx!}xJRNlJKRVq|;< zN1VXs$9VgN(*yYKzHEAsz$aKp3j%bB{MeCR{$ai>9*Yyn6UIhJlewa}a8X>CFh&4J zt}G~&AHWJ@Fxeg?P-^Rr!zG1BtIEGM=b?V9YIt0)xnj@(y>6O}cZ??cz8Bma;I?1^ z;bnbD#igP@8HZa=*~VXz${~s{GP&Y~ZEOnDy~kdTOycer9LFxdp1Atc#MPexS7TRx zp1AhwBuPXhZ~ioK^ViWEzaYKt{5gK(x5-<7pzyl+J8+BI0Gqu1=ftf)C-3}w^!A_c z6qHxt1k1|Wf|81Zc?G#Zt>XNAfL2LyNiqCaQe0M2SW=Q-T2@$HnNya(HE;K_z3U=( zq=asc_TMD)S}XL)5;3#IepzC_HIV_UB11DHV>YCGnDcQ_O?gFALuFHARRfAJAXd|n zhQ_A)Bh5`mk2g0Ssjoq9)ipHL;yargYL3<9RP9@TV0G4k)Xe=cD|4cj@0Fx{B1-*4 z6u%=}wmB#wo59cUj9lpzok@#d6}V!HeDh~3Gk3*Btnv$8x|p4|P@G9m-6V`z#qp1F zphdp#A^JN_JkKZcJ-Xyw|Je6gN%I2}=lMmy%ZhOf61&nuo!kSQoqU`ZdpdpK;`|Q> zhrc;G{KLiReGiXCbef|d(+~1xgkG&*BET*jZ-fOH`{$9GZyT#m z*H@mbEBms#_-J`yLs5R!;X`Ey4(9IOy?6VEA8lCw(T0u7lTtn0X^Wga7r4-{eux00 zIlFpAM<;xD_JT@jQYtI}Fs*jdXu7G>-+))D+46uSq~Tl@>In;8j%YxDg*8KT*_q5Z z9?VLHgjuXF&~`|Ap)i!)-pTu%(92$o!s`e&djKv$*X-~P(w0wRIQ~O{Q5>QGHdk82 zAUper6LeCbHIek#5@kjaj%ppVJM!(=Lr>#u1{L|rl-1r|NTh0s_D`f3`B#aOP;G9< zj@s+O?**%cc*M=2IlN(k)cV|HLF(&qvL!kX4HUqz8w{JkfL^48cn<-lob2t6w-k5{>YU zW9~l4loyl^Hq1U};A}0^6y-?NTQSTK5jBdcONWKQSGe^>LVk5t%zXzwQL+rVWtrBG zV3PyzOg-{Oi_#5wrk$Oc0szxvocwF5159C=QIa07Vbe58+;}~pof*>44C`MG>!*ed zQ*a&7r5eF8D+;6+Xt-&dwv5h0bj;g_fe~)$tki_C7jU96zram0>;`*mm-7WNTwyGWBMAN!M>4! zURz%m(976%QQO&O=<3jSw&}Y&jYEBA)sRi2oX~5o8Fe>cf$%aKaVnUJ2rr}QKBiA$ zoAV3UhJ` zFdeF*IJY?OaKYhXI4>*7F9xa==9ZS{RaNfH-?wJp7TM0FVIRl&Z#rM~ufT~Sl`FjO~GK&s@{;q>_{nmFG`mX%RQT zPE#9TXx}({qWjyg0bQM^KJPer9PEy<627tNK7dyJ~`Qj5N zzf-BK3IZ^V1_xRiO_)Yc9#;)5H z2|l^M-R*&!UbDsxEX2(=PLcJJO53x)X|pHPp_RjqC_A+5f@~ISCdvelBT=SJ>>6NrF~Zn@Jub@-9R;IB1pDU3XGZq zi3L(BoC$w8mZt{OQ?;G|%y1ufzf9=k(CSe17)m@9M=jJGuwjyZSqOl;_|v zyn95cx}hLC3`wTGtw;F<3;uc!ji$$11MY+ofIZb<=PB6$BWjE)8_oqq&Q%VnsxOhe z#GpY+h?ENvj%rdHL@%NZ31^^6g$S+JIvXGm+ngz*hkkY*mEKXz%T_2>(VBd;jHwN-j z!ekk|*fqk$^)VSANz=A)q$`3W(!wH_Mnq?Zi&G=wS4A(~C{J7;owN?Fa3hxm@sb$q zSb5q;Nn#c=B#IZ28k3Y2Azz-bG@B<$4im?7rHMRQl5dF6j~&6}NPGfCbbq0LXjHf; zC6pHv#F4NC(L8ayNSYKGvowSs#R`|P!lZ(TII%2|!PeMyQZYC_+&?zZZ6h09`i@K5&Q@J_n*ml|r+KJ< z1n8wxjq22w4ceP#!yPj=gxxkHw~$`50nZY@CBK19XjCeg%s%(2rbxGkW>W#EoCaul+*kg$cg^u(7MC$wsld_}gVb z*VSJpZ~tL~k0IJ?qWkV~%f4ZB0{SH8AW*eM3t<;Z}Wf3-*T{`@H$+=gker8fuT# zRUWI}U$rkgCu2=c!is%S8T+Er_sY}vf)tBR*&|Qh6`8b6n6QbHw4Rl|Nw9I(vdz0Q zQ#OhPD|~`e7yHG&7Z|^oo9Z3CIygQ%Oth2{8t?8ecc4q>2gEH3Tjms=?!e7(5Ug?* zWxC5&`Ae2^d^z+5Zj0wRE_~N<@jPdzd9E(+d$@!Aa$xv6GW=Wv0z8A6UL3kxn1?6d zGf)!9iRG|kLIWiMbe^XV&y^u?^9ggKv%NioX|DdB3)~mab6WU;>yr0f7r*DSXujLx zg`N%$bQeduYe-n&@wyV+^!nkH;)j!jUZ;l7pBzSXk=VoY2)5yKxcl{%gX3h+w=2jj!P@c0t zfA=S#z;7_%GD5n%98PbnKa z$>8>fPxJ(PX54O}ZgXTz3Emv%p7$ww&B5#~yiuezcnKm*sk)|C-_+`G z&?QJQT!u$l{UbuJ$EdwXQ#1Ttbl4L%W%t^Up9vWVi6oPb_xVSH|%>?gF;+urDGKaxf+jH1_%{D>JGw%0@G zDxAg=p9y!WX4k7>J#JYk3~-!+4;Zpy1_!JGBM@jxX?_XvOQCzNGQWT|Y<#AK-wrj< z(t^`f2+UAc0>ISP=^@L@Aq#d!4;!ZkOw;|wm;Ht*SUtMuotmjO#q-v|2c-?|QE4BA zMP`J{GC0x+p1fWlP3Q2F9xrNQ6II92h16fQ1U9IKpTlAWaebRuoYSZ;~7zcZ-Lw#1&@Q7A9 zYS2uYbk|M#TSzZF-Z5ec@va5;y|6Z8wmvces79Zdi2}0%%&eG4h1ac*F=UR&F96uc z#B<9iw%OPw(MeCm(qk?0@|&@%Z$_`anY{7y=oNVL_4rj}*fXfR!7+ zkKg*;dL2uPqqlw^x%u13oj+i`bEu@~P;p^yN%5il{5*8P1HeiS733Z+$}6lWEGRE1 zsVpw9DKD-p-(Q@!t8o9y1Div)#rl3M@!Kt<@08L$j`G?fTe322eoDam@yz+r0ShEd zPZ>KgJ9F2819_ztr6iB4sJx=0zOJ^VskXTeZS+mmO)a&}#~O}*(E_XGSX0x{hMJbz znq#$PEu|k9e3-R=+3J09s}IDj&WT;MKX&P-Q87D2aodE6TLsCR`AawPvv(wXxO?^X z&$2T=j*DF5$4Xx4ANPJx!lLk0H|a{hlnvo=t3w6JzAO;y^2Ppfi$j+>@>aTt*LX;> zy`tAM(fgVq@Coq&-n{3u@I7aT`EIW7yQA|E0OrJG!gBPZd$WE0B7A%zJs1*qmeiLY z$Kl9X0RlRM=gO9Qg~&X7I8L-ccNdz=e5Zx)IxLv)?DW2q<9kjHi#%N&=xz(VoE_+{ zPBfR;Sn2r_%_`v5xf8=boS^76^ux)aa{#b!$;@RccjF`gY>5)b{!o-zZ=`&Y*<9zW80vL{}sUWZV(4ne> z2g>*FFWU2I&W`P$Y(Rj;#l|_i(Hxw7oSb~IZ0byRaHTo9)7-s$a&q!j3X78LfuT)S zf76~KL(-;=#(MxT(u(`ofaWwDyif*i5ItpWE@D0=-r z03*-xc6t$^hI=Yx=LsM7;fmJ$oSL&m%pjB!va@RLsozA7)WuoCy++_+FYSzeY{ss% z-mV|c$};nR26mW7?U@}^Es=s8Rfn{UQPo8(A=;1FLydnieA9}(U!?C1H~dh2TBfvy z=UT&4ieYF*N7Fp;;E~q+SZ{o!Gu+nz!*usRf$7N7YVT__w^gd^3gzVRu(`dxx3hx^ z^2Iu>l2xBo`)hmUW*(vH4Sji8hPhM^QAdJ97lu~`^DhH}zV)9*u+n299fA% z%?QxKEDWq1s>%3DOp~&`0XX5tnE4ekAHGIn>88+gVVEA&PXWXFbfUZH)OE|%0 zcs6W$p)^b>4CqB1)IV03o`DY27!k=tR@z5g5}wU~TpKjM7%~wDhweqM{&^29-Lr1Z z(@xcscIDHHgOASj-md#%C?;(eSF$oJVi^l`m?T}2uuht=E<9p+m?(`SN(~B&W3c3G zahg1NT}0GMpqC_WjX;(W6dv!(k_Cpx$r7^xa^VqayvSv2L9$4mE{RJI6~*wRi9&fQ ziyPw;B=TZ%eS<^*FnVAle3pM`G~HiJ3lIX>yqR3TU?Gzw3<;NphD(JJ@xsWs5Uz+X zj%J68ef&6%u1t1#>}UJROUsVd)qM5k@pI?C>jz4aDEcwQplz_v(%WU}xTJ4u)wNyJ zbhc@FJN1LTW(679jFVP06Gq)tvjODSO{4B6$}iAfWa|q6X2mAu2PW$SqwS$_?s0lWXdoyWtn_!oxtHK<5zwj$HA3vMy~ugdgVu8 z8`?K6Bfvn4*(TmhT>oth#1_ztXflvumMcG6Z~iiJ>o?1dU&ikKX}j~g_4aS?3kvcp%F63&YLC=m zJV#?qeN#Z@5SKz1WG-9r4GK5#VnZzH#Q&`ULjofK#`ju$u~USo5oqerqe(jTf66emGM9eN*k(hMH5= zRVS+|zpgC%ysY?0VL{DdRA6QM_7#1$CujS%-5WM+U$-tTCDohe=ip3pc4fHWSWgT6vqXao;zduqB`*@e6-1cwP7-v5K zqjhn<3F;(b$0tcqRKXBVbl56O8l~q>s0D$?a#)Ls|YU4ujv6`Cq3HA!?BXfF_9=;L?+;;di)q*wE}SNW(%@vv?9 zew*TcyW&CX;Jx#`cYf%(_kH)BvmMu(PiYg<_k>GUvW01E(K2DoYI*VoSzI<302Z+{ zM34+v3gIP&OP9-%v*n<};@3pPu3`&Q0AK-}m|$+4IBEq~oEpf9fe#DjCh#Jc@@1(k zfjnHCz!4>~xN(3kx<8j5!1rKqz5D@Y5sY9tFf4?hz!t@Y@}oI|m=KO6gdG{k5^^}w zP)=k>SVS0K8qSyb1#sQGf*AgSk9Or1l{QybfBDtt=f6AK*Vd}->#+{@kMwpMySt2? z9mY!+wE!?sVBHT$hx!l1iqFkCn2uN(Db4@`gCXt;|VH6Xl9cdeEO zHh>mYNVFo%C{G6h%z`tRsD{zmv0+a~$rvWfD0WYdUj502mBTru_0!0epKX(G#xDN^ zstd1Qc>@dHg?-J}e;vK~oAt_1V>f=YUj1?G)~}QI{v5si+vNxUo_z4{@q7P*^-f_` zWkF>H97}6zipxqXt1*zb5=VbkmXwzimK6cRipoKv749o3+*xucHD_bMN3pIugzg^; zygrHmfVpjmoR{qXzfoTANqvA`3&nnnI8o}RbzAm)wkJ0)CqFl*Fc&acT2le&s;;Z5 z#eDj@=A(^AS{mz{YU@bcRDDBr4XmcRy5sf0uzeM~HXcgL%86UGH!6KsWZF(?+9y%V zc1hDeid^xbbmJ$>K0UDgvqRgr>|UC=Ly)k=N4m~Mn(ZZ7;})}#mb^7EZ8KYv?jIUY z4~X()$eb9_OM+8e!k2jpm(wIGn6eDEAU=%2VY>NwE%9_&?C!MK!+nvv#}aQJM+OrB z=D_fE@T0o~c?O8-p)no|*#|W7Jb#HpsLVT9OlR`kf+g-^%%<}3;W;~af3U=P;bK?E zg&s}|+?+pfafYK4-NTLP?GoVS?B`Aoq!s7w!=B|IPOHv;OL^r<{qTeW78VteUc={3 z4}Wiuj~YCCqW{cSy(hoyIq?OojxWjduP<9Jq6M+}!chd+`KG!b8f(8pfK{KUsQRkB z^mu7eV?jYpUS0(OSn=M^4u15}p3R%LughM!Y`MQbYl(}OlN;T|h3@J~cXso3aPxHY zpzqqbcW6Kl0Mn?(ND1+}!H7|2f5jnM9vDsc4VFg+>th49dLX@kQY18wR10B|8XAJl ziVNOLs)%^MgAYdSg>$f;8c3Sd9K8s@5OSohIAgHP=q*^LGmpR0;l>iB(x_V`S_Y^( zA>1fQ1}h?p3R5s*RyskWh4{RQrvo>=Mv+)>)5LxwOzOic)Yg|>X<_6CN~^iyWEk^- z$&L2tVPd45-PN-qy>vFJPhi&5LBc7hZx96;{B22TkvJfps0n~(N38~cjrR7cIy-Pk3PP;E59Jp{FO2sc>>E&Y^z_+Fj6MC- zPFNR^siPOe54-y>wsxX%qZjwU0AQUx{oNqNI{G>hY@on~+gcTEZTf-136=U908F8} z0^(1lxGgnHEFn^UdbI&}!gQAB8Z-L$;AgIeZ_50X`n^zB{tB}oP33R}qb@LL z11Y7)LCmi-I49+`9w$}eF9HX8*>ERJLuiMPBT*e7+JIsBUNZs=5T%0m0>G?4sn8<- zqssc@km=Qs;gyOc_L8^_+z$hQ0o#U+FM(bvgxM5shMA@ZL4oO?Dac6}7|a0+ zpDp0GgReSb=s=I9+VE7NeLSFg(51NBHgv14_j+shwevkU&UfE9*LmZ+&g*9{T{+Ww z?VC&2PPI-P|HiO27Q@PS~apIb&lx$&4 zCNL~iknGQi_GL*JEO7`wHiQ?;43Wc%h{}}3uNB8&36Ud;3*pDHg-JmixgSdi3v`%g zfRGU)3+5%TxbY!^cur(IJ0d1Rp2`!&vBNQyBQQX~3W*2|5rl?Evcn^pfjpWon-Rd@ zu=%sRg8ItJ&yOAX{`AT23+L6B+Vrh$x(gRo7h5$KE^1mYYC4GFu&+ly)NfP|TS?_` z!l=KZ*Im=;ZvexLhN~t6l{bGEZURUwv?bf2HTRfHo^&7JJTUVESQNl;oY>e?+sIR! z?U`-lg>`gl^z!S`t3SfR*u(KRBjaSC<>fcjX4u%(A4e|Fj9z;^ejQ*2vg{|@^cxExKD00000NkvXXu0mjf Dr+dXU literal 0 HcmV?d00001 From ab2390d0da8e9f9e32c5befa2c9346bc5ee5147f Mon Sep 17 00:00:00 2001 From: liferoad Date: Mon, 2 Jun 2025 16:00:28 -0400 Subject: [PATCH 13/49] added the rate test for GenerateSequence (#35108) * added the rate test for GenerateSequence * keep the master yaml --- .../external_transform_provider_it_test.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/sdks/python/apache_beam/transforms/external_transform_provider_it_test.py b/sdks/python/apache_beam/transforms/external_transform_provider_it_test.py index d1c5cbfa8e9a..5f115e6d44e9 100644 --- a/sdks/python/apache_beam/transforms/external_transform_provider_it_test.py +++ b/sdks/python/apache_beam/transforms/external_transform_provider_it_test.py @@ -122,6 +122,18 @@ def test_run_generate_sequence(self): assert_that(numbers, equal_to([i for i in range(10)])) + def test_run_generate_sequence_with_rate(self): + provider = ExternalTransformProvider( + BeamJarExpansionService(":sdks:java:io:expansion-service:shadowJar")) + + with beam.Pipeline() as p: + numbers = p | provider.GenerateSequence( + start=0, end=3, rate={ + 'elements': 1, 'seconds': 1 + }) | beam.Map(lambda row: row.value) + + assert_that(numbers, equal_to([0, 1, 2])) + @pytest.mark.xlang_wrapper_generation @unittest.skipUnless( From 8449ca51e36a778e805d42a333a4cd2802abab97 Mon Sep 17 00:00:00 2001 From: claudevdm <33973061+claudevdm@users.noreply.github.com> Date: Mon, 2 Jun 2025 16:03:00 -0400 Subject: [PATCH 14/49] Re-enable logging after importing vllm. (#35103) Co-authored-by: Claude --- sdks/python/apache_beam/ml/inference/vllm_inference.py | 1 + 1 file changed, 1 insertion(+) diff --git a/sdks/python/apache_beam/ml/inference/vllm_inference.py b/sdks/python/apache_beam/ml/inference/vllm_inference.py index c1f94259aca6..c4d5aa2dcd37 100644 --- a/sdks/python/apache_beam/ml/inference/vllm_inference.py +++ b/sdks/python/apache_beam/ml/inference/vllm_inference.py @@ -43,6 +43,7 @@ os.environ["VLLM_CONFIGURE_LOGGING"] = "0" import vllm # pylint: disable=unused-import logging.info('vllm module successfully imported.') + os.environ["VLLM_CONFIGURE_LOGGING"] = "1" except ModuleNotFoundError: msg = 'vllm module was not found. This is ok as long as the specified ' \ 'runner has vllm dependencies installed.' From 42a52a953849a0e56e1201ce4743e536e095f143 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Mon, 2 Jun 2025 16:10:39 -0400 Subject: [PATCH 15/49] Deprecate Java 8 (#35064) * Deprecate Java 8 * Java 8 client now using Java 11 SDK container * adjust non LTS fallback to use the next LTS instead of the nearest LTS. Previously Java18 falls back to Java17, which won't work * Emit warning when Java 8 is used. Java8 is still supported until Beam 3.0 * Clean up subproject build file requiring Java9+ * Require java 11 to build SDK container * fix workflow * Simplify XVR test workflow * Fix Samza PVR --- ...ostCommit_Java_Examples_Dataflow_Java.json | 1 + ..._PostCommit_Java_Examples_Dataflow_V2.json | 2 +- ...eam_PostCommit_Java_Jpms_Flink_Java11.json | 2 +- .../beam_PostCommit_XVR_Direct.json | 2 +- .../beam_PostCommit_XVR_Samza.json | 2 +- .../workflows/beam_PostCommit_XVR_Direct.yml | 17 ++-------- .../workflows/beam_PostCommit_XVR_Flink.yml | 17 ++-------- .../workflows/beam_PostCommit_XVR_Samza.yml | 19 +++-------- .../workflows/beam_PostCommit_XVR_Spark3.yml | 17 ++-------- CHANGES.md | 3 ++ .../beam/gradle/BeamModulePlugin.groovy | 19 +++++------ .../beam/runners/dataflow/DataflowRunner.java | 14 ++------ .../beam/runners/dataflow/dataflow.properties | 3 ++ .../worker/status/WorkerStatusPages.java | 5 +-- .../dataflow/worker/util/MemoryMonitor.java | 13 +------- .../worker/util/MemoryMonitorTest.java | 6 ---- .../sdk/util/construction/Environments.java | 25 +++++++++----- .../util/construction/EnvironmentsTest.java | 15 ++++----- sdks/java/testing/jpms-tests/build.gradle | 33 ++----------------- 19 files changed, 60 insertions(+), 155 deletions(-) diff --git a/.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_Java.json b/.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_Java.json index e69de29bb2d1..77f68d215005 100644 --- a/.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_Java.json +++ b/.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_Java.json @@ -0,0 +1 @@ +{"revision": 1} \ No newline at end of file diff --git a/.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_V2.json b/.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_V2.json index e3d6056a5de9..b26833333238 100644 --- a/.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_V2.json +++ b/.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_V2.json @@ -1,4 +1,4 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 1 + "modification": 2 } diff --git a/.github/trigger_files/beam_PostCommit_Java_Jpms_Flink_Java11.json b/.github/trigger_files/beam_PostCommit_Java_Jpms_Flink_Java11.json index dd9afb90e638..9c289f9b3c75 100644 --- a/.github/trigger_files/beam_PostCommit_Java_Jpms_Flink_Java11.json +++ b/.github/trigger_files/beam_PostCommit_Java_Jpms_Flink_Java11.json @@ -1,3 +1,3 @@ { - "https://github.com/apache/beam/pull/32648": "testing flink 1.19 support" + "revision": 1 } diff --git a/.github/trigger_files/beam_PostCommit_XVR_Direct.json b/.github/trigger_files/beam_PostCommit_XVR_Direct.json index 262d546418db..bcb86e6ab5e7 100644 --- a/.github/trigger_files/beam_PostCommit_XVR_Direct.json +++ b/.github/trigger_files/beam_PostCommit_XVR_Direct.json @@ -1,4 +1,4 @@ { "https://github.com/apache/beam/pull/32648": "testing Flink 1.19 support", - "modification": 1 + "modification": 2 } diff --git a/.github/trigger_files/beam_PostCommit_XVR_Samza.json b/.github/trigger_files/beam_PostCommit_XVR_Samza.json index 9e26dfeeb6e6..a9ac2f4cc406 100644 --- a/.github/trigger_files/beam_PostCommit_XVR_Samza.json +++ b/.github/trigger_files/beam_PostCommit_XVR_Samza.json @@ -1 +1 @@ -{} \ No newline at end of file +{"modification": 1} \ No newline at end of file diff --git a/.github/workflows/beam_PostCommit_XVR_Direct.yml b/.github/workflows/beam_PostCommit_XVR_Direct.yml index b2bd02dccbc9..a2c3ef3a67e0 100644 --- a/.github/workflows/beam_PostCommit_XVR_Direct.yml +++ b/.github/workflows/beam_PostCommit_XVR_Direct.yml @@ -74,29 +74,16 @@ jobs: - name: Setup environment uses: ./.github/actions/setup-environment-action with: - python-version: | - 3.9 - ${{ matrix.python_version }} + python-version: ${{ matrix.python_version }} - name: run PostCommit XVR Direct script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - if: ${{ matrix.python_version != '3.9' }} uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :sdks:python:test-suites:direct:xlang:validatesCrossLanguageRunner arguments: | -PpythonVersion=${{ matrix.python_version }} \ - -PskipNonPythonTask=true \ - - name: run PostCommit XVR Direct script - env: - CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - if: ${{ matrix.python_version == '3.9' }} - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :sdks:python:test-suites:direct:xlang:validatesCrossLanguageRunner - arguments: | - -PpythonVersion=${{ matrix.python_version }} \ - -PskipNonPythonTask=false \ + -PskipNonPythonTask=${{ (matrix.python_version == '3.9' && true) || false }} \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v4 if: ${{ !success() }} diff --git a/.github/workflows/beam_PostCommit_XVR_Flink.yml b/.github/workflows/beam_PostCommit_XVR_Flink.yml index 50e737d4671e..0f177633f771 100644 --- a/.github/workflows/beam_PostCommit_XVR_Flink.yml +++ b/.github/workflows/beam_PostCommit_XVR_Flink.yml @@ -75,29 +75,16 @@ jobs: - name: Setup environment uses: ./.github/actions/setup-environment-action with: - python-version: | - 3.9 - ${{ matrix.python_version }} + python-version: ${{ matrix.python_version }} - name: run PostCommit XVR Flink script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - if: ${{ matrix.python_version != '3.9' }} uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :runners:flink:${{ env.FlinkVersion }}:job-server:validatesCrossLanguageRunner arguments: | -PpythonVersion=${{ matrix.python_version }} \ - -PskipNonPythonTask=true \ - - name: run PostCommit XVR Flink script - env: - CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - if: ${{ matrix.python_version == '3.9' }} - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :runners:flink:${{ env.FlinkVersion }}:job-server:validatesCrossLanguageRunner - arguments: | - -PpythonVersion=${{ matrix.python_version }} \ - -PskipNonPythonTask=false \ + -PskipNonPythonTask=${{ (matrix.python_version == '3.9' && true) || false }} \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v4 if: ${{ !success() }} diff --git a/.github/workflows/beam_PostCommit_XVR_Samza.yml b/.github/workflows/beam_PostCommit_XVR_Samza.yml index e75eb1e3c2d9..ebd95bfd2103 100644 --- a/.github/workflows/beam_PostCommit_XVR_Samza.yml +++ b/.github/workflows/beam_PostCommit_XVR_Samza.yml @@ -75,30 +75,21 @@ jobs: uses: ./.github/actions/setup-environment-action # TODO(https://github.com/apache/beam/issues/32208) move to Java11 after bump to Samza 1.8 with: - java-version: 8 + java-version: | + 11 + 8 python-version: | - 3.9 ${{ matrix.python_version }} - name: run PostCommit XVR Samza script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - if: ${{ matrix.python_version != '3.9' }} uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :runners:samza:job-server:validatesCrossLanguageRunner arguments: | -PpythonVersion=${{ matrix.python_version }} \ - -PskipNonPythonTask=true \ - - name: run PostCommit XVR Samza script - env: - CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - if: ${{ matrix.python_version == '3.9' }} - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :runners:samza:job-server:validatesCrossLanguageRunner - arguments: | - -PpythonVersion=${{ matrix.python_version }} \ - -PskipNonPythonTask=false \ + -Pjava11Home=$JAVA_HOME_11_X64 \ + -PskipNonPythonTask=${{ (matrix.python_version == '3.9' && true) || false }} \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v4 if: ${{ !success() }} diff --git a/.github/workflows/beam_PostCommit_XVR_Spark3.yml b/.github/workflows/beam_PostCommit_XVR_Spark3.yml index 14a275550f13..5b4c3634a037 100644 --- a/.github/workflows/beam_PostCommit_XVR_Spark3.yml +++ b/.github/workflows/beam_PostCommit_XVR_Spark3.yml @@ -74,29 +74,16 @@ jobs: - name: Setup environment uses: ./.github/actions/setup-environment-action with: - python-version: | - 3.9 - ${{ matrix.python_version }} + python-version: ${{ matrix.python_version }} - name: run PostCommit XVR Spark3 script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - if: ${{ matrix.python_version != '3.9' }} uses: ./.github/actions/gradle-command-self-hosted-action with: gradle-command: :runners:spark:3:job-server:validatesCrossLanguageRunner arguments: | -PpythonVersion=${{ matrix.python_version }} \ - -PskipNonPythonTask=true \ - - name: run PostCommit XVR Spark3 script - env: - CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} - if: ${{ matrix.python_version == '3.9' }} - uses: ./.github/actions/gradle-command-self-hosted-action - with: - gradle-command: :runners:spark:3:job-server:validatesCrossLanguageRunner - arguments: | - -PpythonVersion=${{ matrix.python_version }} \ - -PskipNonPythonTask=false \ + -PskipNonPythonTask=${{ (matrix.python_version == '3.9' && true) || false }} \ - name: Archive JUnit Test Results uses: actions/upload-artifact@v4 if: ${{ !success() }} diff --git a/CHANGES.md b/CHANGES.md index cff581230b7d..4d4aee7ed25d 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -63,6 +63,9 @@ ## Beam 3.0.0 Development Highlights * New highly anticipated feature ([X](https://github.com/apache/beam/issues/X)) to address Milestone Y ([#Y](https://github.com/apache/beam/issues/Y)). +* [Java] Java 8 support is now deprecated. It is still supported until Beam 3. + From now, pipeline submitted by Java 8 client uses Java 11 SDK container for + remote pipeline execution ([35064](https://github.com/apache/beam/pull/35064)). ## Highlights 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 c0d0b887a3e5..c6ee2ab76d70 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -454,18 +454,15 @@ class BeamModulePlugin implements Plugin { return 'beam' + p.path.replace(':', '-') } - static def getSupportedJavaVersion() { - if (JavaVersion.current() == JavaVersion.VERSION_1_8) { - return 'java8' - } else if (JavaVersion.current() == JavaVersion.VERSION_11) { + /** Get version for Java SDK container */ + static def getSupportedJavaVersion(String assignedVersion = null) { + JavaVersion ver = assignedVersion ? JavaVersion.toVersion(assignedVersion) : JavaVersion.current() + if (ver <= JavaVersion.VERSION_11) { return 'java11' - } else if (JavaVersion.current() == JavaVersion.VERSION_17) { + } else if (ver <= JavaVersion.VERSION_17) { return 'java17' - } else if (JavaVersion.current() == JavaVersion.VERSION_21) { - return 'java21' } else { - String exceptionMessage = "Your Java version is unsupported. You need Java version of 8, 11, 17 or 21 to get started, but your Java version is: " + JavaVersion.current(); - throw new GradleException(exceptionMessage) + return 'java21' } } @@ -2653,8 +2650,8 @@ class BeamModulePlugin implements Plugin { // see https://issues.apache.org/jira/browse/BEAM-6698 maxHeapSize = '4g' if (config.environment == PortableValidatesRunnerConfiguration.Environment.DOCKER) { - def ver = project.findProperty('testJavaVersion') - def javaContainerSuffix = ver ? "java$ver" : getSupportedJavaVersion() + String ver = project.findProperty('testJavaVersion') + def javaContainerSuffix = getSupportedJavaVersion(ver) dependsOn ":sdks:java:container:${javaContainerSuffix}:docker" } } 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 66cabecb695b..00a91070eb04 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 @@ -492,23 +492,13 @@ && isServiceEndpoint(dataflowOptions.getDataflowEndpoint())) { + "' invalid. Please make sure the value is non-negative."); } - // Verify that if recordJfrOnGcThrashing is set, the pipeline is at least on java 11 - if (dataflowOptions.getRecordJfrOnGcThrashing() - && Environments.getJavaVersion() == Environments.JavaVersion.java8) { - throw new IllegalArgumentException( - "recordJfrOnGcThrashing is only supported on java 9 and up."); - } - if (dataflowOptions.isStreaming() && dataflowOptions.getGcsUploadBufferSizeBytes() == null) { dataflowOptions.setGcsUploadBufferSizeBytes(GCS_UPLOAD_BUFFER_SIZE_BYTES_DEFAULT); } // Adding the Java version to the SDK name for user's and support convenience. - String agentJavaVer = "(JRE 8 environment)"; - if (Environments.getJavaVersion() != Environments.JavaVersion.java8) { - agentJavaVer = - String.format("(JRE %s environment)", Environments.getJavaVersion().specification()); - } + String agentJavaVer = + String.format("(JRE %s environment)", Environments.getJavaVersion().specification()); DataflowRunnerInfo dataflowRunnerInfo = DataflowRunnerInfo.getDataflowRunnerInfo(); String userAgentName = dataflowRunnerInfo.getName(); diff --git a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties index 956c447e5faa..03080508e217 100644 --- a/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties +++ b/runners/google-cloud-dataflow-java/src/main/resources/org/apache/beam/runners/dataflow/dataflow.properties @@ -21,3 +21,6 @@ fnapi.environment.major.version=@dataflow.fnapi_environment_major_version@ legacy.container.version=@dataflow.legacy_container_version@ fnapi.container.version=@dataflow.fnapi_container_version@ container.base_repository=@dataflow.container_base_repository@ + + + diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPages.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPages.java index ef783a00f8e4..0ba18584dc70 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPages.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/status/WorkerStatusPages.java @@ -27,7 +27,6 @@ import javax.servlet.http.HttpServletResponse; import org.apache.beam.runners.dataflow.worker.status.DebugCapture.Capturable; import org.apache.beam.runners.dataflow.worker.util.MemoryMonitor; -import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.eclipse.jetty.server.Server; import org.eclipse.jetty.servlet.ServletHandler; @@ -59,9 +58,7 @@ public class WorkerStatusPages { addServlet(threadzServlet); addServlet(new HealthzServlet(healthyIndicator)); addServlet(new HeapzServlet(memoryMonitor)); - if (Environments.getJavaVersion() != Environments.JavaVersion.java8) { - addServlet(new JfrzServlet(memoryMonitor)); - } + addServlet(new JfrzServlet(memoryMonitor)); addServlet(statuszServlet); // Add default capture pages (threadz, statusz) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitor.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitor.java index 83c49c197509..80d32fc6a983 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitor.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitor.java @@ -56,7 +56,6 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.SdkHarnessOptions; -import org.apache.beam.sdk.util.construction.Environments; 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.ImmutableSet; @@ -246,10 +245,6 @@ public static MemoryMonitor fromOptions(PipelineOptions options) { Duration jfrProfileDuration; if (uploadToGCSPath != null && debugOptions.getRecordJfrOnGcThrashing()) { - if (Environments.getJavaVersion() == Environments.JavaVersion.java8) { - throw new IllegalArgumentException( - "recordJfrOnGcThrashing is only supported on java 9 and up."); - } jfrProfileDuration = Duration.ofSeconds(debugOptions.getJfrRecordingDurationSec()); } else { jfrProfileDuration = null; @@ -314,13 +309,7 @@ private MemoryMonitor( this.localDumpFolder = localDumpFolder; this.workerId = workerId; this.clock = clock; - - if (Environments.getJavaVersion() != Environments.JavaVersion.java8) { - LOG.info("Uploading JFR profiles when GC thrashing is detected"); - this.jfrInterop = new JfrInterop(); - } else { - this.jfrInterop = null; - } + this.jfrInterop = new JfrInterop(); } /** For testing only: Wait for the monitor to be running. */ diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitorTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitorTest.java index d24a57dc4eee..94800cd72f54 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitorTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/MemoryMonitorTest.java @@ -18,12 +18,9 @@ package org.apache.beam.runners.dataflow.worker.util; import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.Matchers.is; -import static org.hamcrest.Matchers.not; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertTrue; -import static org.junit.Assume.assumeThat; import java.io.File; import java.io.IOException; @@ -34,7 +31,6 @@ import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.beam.sdk.util.construction.Environments; import org.hamcrest.Matchers; import org.junit.Before; import org.junit.Rule; @@ -168,8 +164,6 @@ public void uploadToGcs() throws Exception { @Test public void uploadJfrProfilesOnThrashing() throws Exception { - assumeThat(Environments.getJavaVersion(), is(not(Environments.JavaVersion.java8))); - File remoteFolder = tempFolder.newFolder(); monitor = MemoryMonitor.forTest( diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Environments.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Environments.java index 9c47e1a59f77..05ecb21fd956 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Environments.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/Environments.java @@ -95,7 +95,6 @@ public class Environments { .build(); public enum JavaVersion { - java8("java", "1.8", 8), java11("java11", "11", 11), java17("java17", "17", 17), java21("java21", "21", 21); @@ -131,20 +130,30 @@ public static JavaVersion forSpecification(String specification) { } } - JavaVersion fallback = null; + if (specification.startsWith("1.")) { + // for Java 8 and below + specification = specification.substring(2); + } int specificationInt = Integer.parseInt(specification); + JavaVersion fallback = java21; int minDistance = Integer.MAX_VALUE; for (JavaVersion candidate : JavaVersion.values()) { - int distance = Math.abs(candidate.specificationInt - specificationInt); - if (distance <= minDistance) { + int distance = candidate.specificationInt - specificationInt; + if (distance >= 0 && distance <= minDistance) { fallback = candidate; minDistance = distance; } } - LOG.warn( - "Unsupported Java version: {}, falling back to: {}", - specification, - fallback.specification); + if (specification.equals("8")) { + LOG.warn( + "Java8 support is now deprecated and targeted for removal for Beam 3. Falling back to: {}", + fallback.specification); + } else { + LOG.warn( + "Unsupported Java version: {}, falling back to: {}", + specification, + fallback.specification); + } return fallback; } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/EnvironmentsTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/EnvironmentsTest.java index 3c6862bd3b3b..410b52cba23b 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/EnvironmentsTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/EnvironmentsTest.java @@ -291,8 +291,7 @@ public void process(ProcessContext ctxt) {} @Test public void testLtsJavaVersion() { - assertEquals(JavaVersion.java8, JavaVersion.forSpecification("1.8")); - assertEquals("java", JavaVersion.java8.legacyName()); + assertEquals(JavaVersion.java11, JavaVersion.forSpecification("1.8")); assertEquals(JavaVersion.java11, JavaVersion.forSpecification("11")); assertEquals("java11", JavaVersion.java11.legacyName()); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("17")); @@ -303,22 +302,22 @@ public void testLtsJavaVersion() { @Test public void testNonLtsJavaVersion() { - assertEquals(JavaVersion.java8, JavaVersion.forSpecification("9")); + assertEquals(JavaVersion.java11, JavaVersion.forSpecification("9")); assertEquals(JavaVersion.java11, JavaVersion.forSpecification("10")); - assertEquals(JavaVersion.java11, JavaVersion.forSpecification("12")); - assertEquals(JavaVersion.java11, JavaVersion.forSpecification("13")); + assertEquals(JavaVersion.java17, JavaVersion.forSpecification("12")); + assertEquals(JavaVersion.java17, JavaVersion.forSpecification("13")); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("14")); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("15")); assertEquals(JavaVersion.java17, JavaVersion.forSpecification("16")); - assertEquals(JavaVersion.java17, JavaVersion.forSpecification("18")); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("18")); assertEquals(JavaVersion.java21, JavaVersion.forSpecification("19")); assertEquals(JavaVersion.java21, JavaVersion.forSpecification("20")); - assertEquals(JavaVersion.java21, JavaVersion.forSpecification("21")); + assertEquals(JavaVersion.java21, JavaVersion.forSpecification("22")); } @Test(expected = UnsupportedOperationException.class) public void testJavaVersionStrictInvalid() { - assertEquals(JavaVersion.java8, JavaVersion.forSpecificationStrict("invalid")); + JavaVersion ignored = JavaVersion.forSpecificationStrict("invalid"); } @Test diff --git a/sdks/java/testing/jpms-tests/build.gradle b/sdks/java/testing/jpms-tests/build.gradle index ac2050df7d92..bdcb7894eb2e 100644 --- a/sdks/java/testing/jpms-tests/build.gradle +++ b/sdks/java/testing/jpms-tests/build.gradle @@ -22,17 +22,11 @@ plugins { id 'org.apache.beam.module' } -// overwrite javaVersion before applyJavaNature -if (project.hasProperty("testJavaVersion")) { - javaVersion = "1.${project.getProperty('testJavaVersion')}" as String -} else { - javaVersion = "1.11" -} - applyJavaNature( exportJavadoc: false, publish: false, - disableLintWarnings: ['requires-transitive-automatic', 'requires-automatic'] + disableLintWarnings: ['requires-transitive-automatic', 'requires-automatic'], + requireJavaVersion: JavaVersion.VERSION_11 ) provideIntegrationTestingDependencies() enableJavaPerformanceTesting() @@ -40,19 +34,6 @@ enableJavaPerformanceTesting() description = "Apache Beam :: SDKs :: Java :: Testing :: JPMS Tests" ext.summary = "E2E test for Java 9 modules" -// direct compileJava to use specified java version. -project.tasks.compileJava { - if (project.hasProperty('testJavaVersion')) { - options.fork = true - options.forkOptions.javaHome = project.findProperty("java${project.getProperty('testJavaVersion')}Home") as File - if (project.getProperty('testJavaVersion') == '17') { - setJavaVerOptions(options, '17') - } else if (project.getProperty('testJavaVersion') == '21') { - setJavaVerOptions(options, '21') - } - } -} - /* * List of runners to run integration tests on. */ @@ -118,13 +99,3 @@ plugins.withType(JavaPlugin).configureEach{ modularity.inferModulePath = true } } - -// JPMS requires JDK > 8. Test tasks enabled when either -// (i) testJavaVersion property specified (assumed to be >8) or; -// (ii) current Java version is greater than 8 -project.tasks.each { - it.onlyIf { - project.hasProperty('testJavaVersion') - || JavaVersion.VERSION_1_8.compareTo(JavaVersion.current()) < 0 - } -} From 50608c2f897db2822d37bfdfa72e8cff368bf44e Mon Sep 17 00:00:00 2001 From: Danny McCormick Date: Mon, 2 Jun 2025 16:50:32 -0400 Subject: [PATCH 16/49] Remove beam college banners (#35123) --- website/www/site/layouts/partials/header.html | 6 ------ .../beam-college/beam-college-25-desktop.png | Bin 230979 -> 0 bytes .../beam-college/beam-college-25-mobile.png | Bin 113671 -> 0 bytes 3 files changed, 6 deletions(-) delete mode 100644 website/www/site/static/images/banners/beam-college/beam-college-25-desktop.png delete mode 100644 website/www/site/static/images/banners/beam-college/beam-college-25-mobile.png diff --git a/website/www/site/layouts/partials/header.html b/website/www/site/layouts/partials/header.html index 4f3b1124391e..58672df4eb54 100644 --- a/website/www/site/layouts/partials/header.html +++ b/website/www/site/layouts/partials/header.html @@ -213,12 +213,6 @@ -

diff --git a/website/www/site/static/images/banners/beam-college/beam-college-25-desktop.png b/website/www/site/static/images/banners/beam-college/beam-college-25-desktop.png deleted file mode 100644 index 530129731beaff0fb24d67c9ab09f4645eb83932..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 230979 zcmV)HK)t_-P)ooc$8t98gNfhUK(onUkB$3RCtlH<`00%hNeZ5Q421F(^ zBU5YdwbA`5ZD#*(=AKlog9ZYL$OsR25040Ux6glb|6%0!0k)ECcluktm8~!Pm0SH= zZT2hez8=4`?RHRUYe)@tr~Bo${odNCTRk;mkswU9)yF>K*K6&-!ZIj;M)M&C0d`Rp?}3E3J{8yZ04X=^ZKtYUeP%Ox`!K;xGqj70xR z%(s2qwH~xwNf`NCZ4M-!nk6bnMsG9*J&VlWGS6~w^a?3jChNM7LPG({FRn%DNw&4z zT5%&RpIY6*Sl(E+x7b)#C`?4C1=ZF|WW)Ywy|CNI#z$5{3H!s<2lR)Z%AQoMF+d^5 z)5ne#tY!∈w>zo<4#<_?W6*0>}>vA5TQ1il^XZ5@&IVzAbH!p{e&Pw>Eag=OvNL zK8k4cTJKcC*KV|6krtx0*mNJSiu{g!EXIE;_~c4+^L};vVRh$0ephz5 zvu2y_Z0Yanmi@I;{M}~wmF%yaLa?FZJ09B(BV5U1j)YR+NYMjmLc3WQI~&bXe<#5dtUG*Esg|f50`LKBOSZT4KP^2h)`F*BE+QYs+lp8+2ian8< z6*d3XS=xAiYW5+>%c_0asM(mhzDto|eEycbnx#^)R}`VVCmHlG&Vlef~x25C3(m{Xupucd^pF8U>9raiDij}=$>A1hTQ>^S1 z%XW5UR}NaR)4|eVfBtl^aN1wqmmN9MUp(&5oex%b3yEs}Y_NFLpSu{$T@GemOR#~) zu&+ntlF-T4>2eYsBu9x(#HA|9z4Syo>@OF?Q zibXSdHdwHGlo%vYxhXA>-9i$QzfzcTL_&JT1{RfEQcUlEnLPSDv-@dw=hM{w=jnsb z)AGCbNsdhK>G7Ez-Ip*E$DgMTKFiM3;b#dvy)OYi$^PWgmx+_lL#0!Ym$lmJU0!mwLGblhkEL5|Sgc7n&2j@Iv?Hh#X%y(ZHI+^GC_vqGaz< zGT)iE{45`6Y?{4uq^Mbv<8qLxXgrz$*(6S6VDYH4y!Wsq*B)s3$mO_G$&6ewkGs3j zQd&6H0-3>GpIe@fH3p4J?;1s?w~wOGwB-1uUOV$v56+!SG4|&CjwGa{#d|;L;NB&G zCh@jA@wc9ywepmx^iIz-2Km(^7kVQSKmu!+w_QA%D098dbA8+rQZ~72(GM;hb@Z`b zXfV0!g;UM{{84&tKWTwJ&=7 zl?1rw{w5DfnCEf)m2K8;Y5+OAaW6;K^jhp|Fg=Tq!^z2o&o_8n0!z$#d{ZNj>xXPotU$<M1|MBI$w3}>D zZmeh0@V;V;{W|nxt;1}si8N=@7}bmD*r4^%VAjo+Fwfp=fUyq+`Z*F_Ix}&t^lGe4 zwC%>4ygHj9{rkQrN3^$mt35-x5G|=3ocOgjdQ-4vY_O5VRoV)-yv4LzK-*XGZnHIi z((xPAyot0du|Fmu$3AL2(#}dkngMTl5g~f!)uuPTb>5eDK3fCa3!}a2qXkFDQk(c2 zw&q9+JK7dEJn1X#d$TjCeFiOgDMQHydf=K>E5OkkhS6tqJ9NKQw&U#i)UwV-Dx#*g z@7Tv9K*q|JnWY156QwoOrq-HuY38J5M4P*0ZS;~g%FBCNQAmv^)z0$4gZ#?3JY(Oo z?4we$&tIyw2dl(%$G&v=x;t}cT3OkbE?<2|s?b?hTs$~;E|sdrFCo{q3HztsstcND*Cn+z%0(41ta*8TYU7SpouQZ?P{hEv()-p97^oA(Gd z6v<|Gl_eKZA(&jt<69}f&_N44{=R43)Z*F4-ss0}r6nE90`I(hZqkXhuBL7Jb@${i zI;;F%^wYL5_9aSpU9J*iO%7HYMfpx+mVNovnxS5pf!W8Anw!aPQqf-1YwP;4%iC^t zmeoI&4Xeir`xp!4MD530*j(Bxt~LwiO3vPBPGG2JmY^5~nz7RFZU&kW>ejj)zfHA} z96@qG7PJ={eRkv9{U|HRp$7Kbu~U|xCX<%r%&;{J6tLnJHC0h-7Mku|#tm-!IHI?p zhI}6vx`F9=BO*XHq|ka{AizlX$|nr%bc0d9X~ur%{g&={Q(zY;KpaP|tN=ZSR$Qe! zUKN?)sXH3-Ec`aZ^LEU-!QCOk$XJwWx$W^-{K(M_A5OuYQW9Z~q;OLs6@ zhf0Q7@|RP;6wj@GFE<50A2fOPxuKoX6dOe}bEEbh@s{mzXb$b2ktP*yxZO@_a}h>r z0Q4N}>nmtP54GC)tKuJBytmEBHuEM2e<*Kn3H)c@GIy3|J*gRLb7yRP#l0Ol<*>4Um z2k7UqgzNg!qk9^-(TGs+SVMstp#)Rv=Tks;oo3|LwGTup&Z{$XfH0$ zGnO;QwLG?Mk6|afoh{#ii5p&ccJcKkZ1T<5s@A`PTGK5hw%qze zv_o^vsPOt}rH0;rgP)1K1{j>EHI!VB!aBNf!bntCjDIM`u1Ve{OT@?7x2EJ#X{NqN zun^>uXVM>PX6UfNdP8fZTYg0P;q&{|S5%>7oi%`8=%3gf-{H6yyl?V_PQO!>CU(Ny z z6sTPcX5M|1*fc^TcZ){fumfVaaLTi*yZyCoWo@fQ+>o?o#$e^Xo|P13TR79)&YykL zYmYUnlK;8$Z!2coXvPjSm_#l|jP5bm!kO;qS-T4$McfoJDSsu;5}QOme{S(38>e#o z+uWssRN0ptu2|%&4ooFcMrC)TVAR4doJv-U*&W4mvSCLN-tm{Iqc1Z`;=W8Beo>rv z_+{ec%k<&r={-F=e)eVZ=*!fx9MO;p?2as?5;{Aw|9R%XE?0E;MJ_>{li6RnN|NA( z7A?Y0T7(lP|0YKyutsQ`X40}Wy{{oBPqN65G)q&KNzEq^=fRhW(=U^D?ZnxaXXhHB zWO(|Luy6jXJ9kdxv~X$&>{Ka|qNQ`iKoUSUIkMCUMHO%W~C`MitpuI#ehI z_&Dc$aQaR>I4v0^gHm*wD}}AEZ8P2{c|#^~%SPY2QhNILHE((DjO8FVi!zj$ z_41#4`hYCw6TcQy?|Kpy4k}xG*PHyc0IJjSu^q{VB%#@O+cPw&rGx@WWp3%}UE9Nw zsO-6Pedp2M2fIy4>$Me^~NBER2% zjp19g#iNEBYMuv1Wq~W!{*EH%4;H>on8;zsCM_KKEA@;b!qx!A3!XS?!Br4lqAtH8xh3WB^yFp+d)&SV}#h)0v-=m zrTvw&Thi7)E*)4ceQ6crsX_k>{gCxjwQnITL4P}1z1xQ``^)DqT@5T@fbRtWx)Lj`#MwToWwOQSRXn|*@~wVhFK*o!Kk zr&*>gnAO&yHM4j0I#mLisDJn{E$zcp`wM0oSh<>4O`2@_QP^iyl0N+1x3ZgoAQ3Pk zw7+fEw{B)W!wMGjtwgyJ4RBc2COrBU&;UiDHCx+hfR5-hpEPfAp?1i>F?&rKm?flm z%V4>|4lBN_7OtWXxA!?B;{59yRXzp^G~0LI(Cx3d$*lReTLz=3wo=7nb+6t#${}oO z#3skp0O?17@F@je7SklZ2(<4g9o+!E&C zeK7t&{Nm~jXAur?z;K)x^0ylN>UMzV5=1M&2(!m*+o#bWlZlrTMO{Iz@P^pSld~Yt z-cXiuQ5`v5NN`|9QWeLy9W4uN*cx?m>AgjBTm$8hnxRgd!q+l*)NO{OTKp<)I5Tu+ zFlN71P5AOdDJYK%3H9U5WPlh?EBnoWpp|VDR0B{A=L$hPg5(iN*$A#5?3Y}v#vQ5! zEq;f=m5`eOinW-B3IVMiVM2|*YXkkBZG5ugZ$rcrNM7H2JBTYz5Ia?97Z z+Vne3*J@TB5~reIQQvY*I{>qfAq;$EAYh`RK-{LIXubv`mR!ljp{%o*C4X2}EY%p5 zz3!J=JObW|g2IiL_!QD3i02-c>o{*qC3u;ZdeE#GS za=ewDMAKGs4d71F6(+GCJrN=ETkGG*dWm>rE_WqT+~E1t2H zos_GUoC6Lk9Cd$Kd-y~=m}hc%x3jkSApKE{pGbL17X(8Vm1t9`>C^H5(T_Gfh+3x2 zx4q);6%aXILRd}ZR(df;OruK0AMK4IB&bozXrIF#Jv0=m(SyXG-j4twh2Xo|wyzZ; zQ~L~cDD~GK?3rrd$TnXjUTc~N?d9C~yVr1SW1HFsR6hDf+3S@0AGfT*1)_=#_qbweM-vQ6W(G)6;pQo$oxR|W9GOuH zsgWQRm0HNPdilUK7k0U!RZ~`en?ExNq4B|~{m&CeUuJecn^aIWcN0e@$AjE&&#>Sj z#e=dl0WG1O9Y1Cm`PIKCh7KVQv>5cNL+(!;bAQ?dgOipJ;#81#@CBm6CmS!Zs3iVS z9opr}7AKCsOdwI{7WX`6(;Ld!XD45totwx zgj7YFLA0r#Rn9g8w0GW}G8CnM&mmG%IHg)PIcwKWyMz#tKtl(~+pLMvW?nxk{DP41 zOpnXXqN@Bfn4FsWxu@XosIz#`QQ;m$i_kf0(ulb`ZCXNT`XFqC98az2GfP)DD&VHU z%Pvf6ccKzMz3Pm*#d#|SF_@PQI?x!-yzaV4P-8P;V|UuMdniNPo!kPGFhSq66%I-d zB{2223mxC=IcVVH{BcJzF!^&&GNX+0Py=f)ixYHjh8(uLaHNF;b)l4u{LWh8V`s)w zk#r>zi4(|j?wp8I#e+wk`I96LMOwOJwFIx zacJQjJwiQ)Hg0W`Et)C#T;;dP@gxo`#gQxwb$Jj9nYEmUL=vt>Bx}Q2sMpswy&=m| z3_yvdDinWEDQ5aaZtKo8*KL1oGo?F}TjvPt0sbW08yOVvFzZ>IYg)cIQKk7~oAW&+qU%4vnnhlmVH&^|!4}CN`P~_RQD*QovM>DmlEt(nssr{Y!wWk`x zYYOKPIh<82WW>epdxi9>M;q1 zy3wG&YJL|XvbH)QNv97@2Yu=v^`XYRzTs%kx!yX|wV|>7etW?ky!n|L*RRU1{V_0{ zZWca^O24Vt$5)07Y$$(x8g2Y_3Glj~^aojwmqCY3I39NW_L5%H^ne zqOP#yq8_;S81A#mfjVL+m)E@BSj%dUs9RX)^7UommPI8g(`H=!3V27Wci%=H33k< zKvtvJ1w#NyRvp1=i!TV@W?%uBu2uKK(VScD0lT2YUV4GjbJ&X0SG077NDa}2s6hH6qvgKmNpJPi|`Kq8fP z=;IP>;q80&NLpI`AUr_^@YiXbmrs*~0XU*el^ktw+;!POO?o$64?MLx_%{m zPff4qxC1$-a?rC)a|~uC86W*h%|){95W{g6o;XZeQ&0}5p=+0&>SA9N!;zAi8Q49^7_jQ1%>V%cG0l%PveiP*~i())*JM_O9Y z5nF|=a-Ve#K86PE_G+D#T2Lo|OY~+c-W$MMRd(Shk)l6NnJidSaLLif17CA5tepd&Sc3+G% zF<_p0@8>b(<5tJPt#3&aM-_?XNU8@0%4#V zenZ@FXUOY$Wjn>Gg=Qrg&fWCVy-Ci+qvqEG5d^9yWpE~ku7=Zne2k9ry#^n|gT{v} zefO!KyYy*l`d8c9*c$c%5nJ9I&+wuGL^XjdyyCw-i)6=A2RA&V zhq8p%>|N-MESM^A$z7}DSF$~GNg613G_Uuo8J27>9Vn?>+bTdskulwN)W3M%N^chW zK`Y*+L-jA7IQ%TzltaMTZS(ov`7~*4Ze~|;+vI_P!d-)dN5pm$hmH#m;a~hk`QQQ3 zAwsH?@=n5q)4N1_lZP69de_bx#hXN639$c3iQ$pObKri>heUn~87&5YLLNCpoCJtq zk>ke&r+#*a9SNzfuLibefVa*RM}ie$qmFn`xZnE8p4Zv>g6}NzHIpAA&*Wk4zho_p=MA@( zGuu0?_qFfXoSE78$$%#sNL)i*5s{_Ha1IeXt>6Xg8!40F>74wRH!@7H!Q(&vbT4HN zW+yF_d~5Ko)e*0c?C)6@YY>B5u->&vGNf3oJ8ZgQE2B{v$wFU6dV^-$OUOah=lW0< zX~2#xS~&CGu>wGMFROKaG%ur8&~F7iR07TJHhveCgS zq6cdkc^0`+FxO(27@c``6p?d_iYW7=??+30#qgsa^YEH~3>C*Xtx$Lrhb)7k`oMtXg>TE#LD;?ib>ciaz>yy!0LgTjyEVDgmFD)U0;|^wfaCh;m>~S!S(^X z8%<_cfzQ8oQ&b48Qr}hLfw_l|5yS8>52566>d_B7dc<^UwWx43Q9Y;sX&&pqwF#V2ISFm~D%@{+Q}@pN6Ect>V8;t)e4zjL70NKaEEZ#l-fONudf?p7iJk}#6#tAL-XE+&iQf?W{O zwFap$U1H)jdElc#Dy!gn{^*mG%1aDIS{|0KRXm20>$mu_H4=C+M{<|O2dZm1)3j$1 zV6p62#$FFXu#kCy4wvHniQy|B1?06p2L&>0j&~U=3j)*Ti%kikPq9^{rk*JRca&S? zVATw!M>9$)5>uBXO&AN!0X-l|!j$qB{h)rj@VYFIC*ezH z-VU3cTH=krV;Lq9GQJ#}n-2o=%J5`u-l6gSpTCIZ9D(~I(D9-h-Yn&{w|uw(!bkjx zmxuT-A6&J8Nd(dd$rm(Oemg~lv1OeO<9!ih9Z#MS#mp{pM6CyEj1FsjsHtGvbe{=@ zDLylfY+t)hzTqOWcK(v<(_G8YTf$_Sq%L z|AxrjcW6@cxifsN&o;Phn~D~e|xe}+d!`N|cGyRp$%IxbdJ0;N%u5l{P=9(W?abU~KRR)EwGj8lZQ@fz7;niE+P1s*$ID0c$Jf z&s2gpckvA}z_qPnZLCp1LPcfqe2=Yir<70r=-ZTi>u?q$dl+ZzfX8`Wg?AE7glkZL2aSe0C@Dvq7Hh37* z|9zR+(U9Y3=J0#`g&M$+BHG_)=l`ZG&sB<05kjK)?Cjrk{y}pVo-~1>DHhE=+2#mD#tV~iSU-9OC67yuBjBiUK<{GA;08BJ9{n{YJIX`9=aN`IFHIZ)51XZg8A}hY`#3Pgvsp81=uJXg?Qj=#=Mvl5HU`tXlC=wB?W*I zl?oW&_Mo0q+PL2d969G+x}2@gock0DzU@Md2ug@@P-%~OE2GST7nk=Q)P_HyXc0nA zn}o58sr_u4zYdCe!3ZgGWs`*|xSaD;9N00TS#vNn%_Gyol!$Zw_!cq4 z5F|e;3uFp_%yApxAYv^J2qj z@j0>~284ATvcZb!=^#|BH0e8>1VK&86(OSo863@|I+5X+5g6h`8upE77U|Lc@0H^igOvrxi6H)I`Ce&mey>ie_+dfR+vu z4zlsxGd2CuVE(L!dfU0Ye&axd%X^)bU5<-b80w&*5X79J3z#X@s;?QFLx6LSo^`cG zve|5u7}|w6wS3T_r|=7dXfJXvV5fmE5}r${!0!cyfMO9t7Cs3N{|TELk_T?2B-URN z+5b<#Wuk{jV&neDfP+pm(--Z`F3&bSjyV$Gy24Npvv$fj<3?72M5XZ;`ovtlcpqmY zaL5TcwlT~K*Ro2C8tPXM@+#7hx#`)we&`JAL^>F9@CnZNiP zw>t4fVDK9}G5>rx&(BTttNWZn)F&Y=!?E<%A(#0!RvVPs0Cj;QRo zH=u+;fl5nv=A3(%m_VqVkOV4ls>L0W0+>`i=T+BNhV@a&#AV?HMdd`>S1Aa+UU&iC zCWukV>Q%94NBrqjT^d;qmYVq~q9saz0>ZjmUvGeO0w(5DY55FY^sQDj0srxcHt<5( zkzCnzrv45jnN-Q$H1l@}Q7KloDc#CGk3=OPW|I>{mUDL0PMSw*D%*~RZaBZ0`*He_ zn6Ax%MkSlpHu+H6l(=}MS#EGpV{7OnDr+giCe67PZZM|t4a+epRXuRrsY@DDn`<*@ zIbN^#Lk>?k?%Ls+Pw3&oOc!Z+TN!O}E@O1toNog4gHYQMf%^bO*_j|f8z6Kr?9yHS zY!ddyOB2KwZ%^qi1&ea5ggVS;6%FcK^`W_1J@s|)U_$tsJXk4Mbywe99qONJx$R$6 zi1qxIhn$o>814l#AOyG0(KR|@s6MJ#J)`M^2BENXu_5Z0Q0=9J`0-GISaPXqyqB28 z*}rWEqrVTy`dF7~{C(bjlTeaH7m%*$t`M%+3T?30Q|S?m_)L@)_n3AmFK@8wcSkLl ze~Kiz^(@0>O?g;H;(ou|uy zdn8}CBtgCJ0OgYm1YOc+I@ZV^EZQH{A2y|(pXF)~vPc1?6BnVfmkhsR_$RmmMtl(%rKuEFLkY(hVi`>F4(vBI^jo(cEV>QE2jWB3#ot8nZ+CY77D z^+}AHsj^QUnhZpmX2EeD$l5X1_voAN@PM%I0FiuC1gY1WVY5TWWc5Mb?ycwyN{RxP z;rV-ze2Q&yCCw6G<$_UB3JK4V!(wHJ>cyGYMmbgFI9R4ra{&P(oUbvb`|=*qXNVJ# z&v}z%iW#=L)0;Z@H)sP*xCdJIW%A&Q`WkQR;v}=%pQb#3dGe~#kj}jOpP!wn->$N` zBUKIp6zZBJ`%w9P89)9K4#LwoXuiM_ejF^%x#X0kE2%R!69z(oX#U8W^tgK{!*`gu zqSv!tJ8>AAK}?is&|{Ypd;$i;iJ?5v!+a;GVvVqUUe`Pph{Pc9-W|pF!qdmbwj}DD!(ptb7WE~ zY_w_hfY{CybgVWIk}tJ+Udkj(v?66=`yr z7dRIKXlk*;%Ud;ld^CW4CYkI4(ghHN+posxN`fw&RG2{94)>4G~g zA~v~X(WvXpCD%fMTyYlbu3S(>sC@9%kU#HQrk-0fU14%SwqVH_+4X>Isxkw3HCOz( zGo3a`DXshPuuHndoIm=t8db*T;9Z!bz^BR}i#(GS+HqfK#V@&sI5EVGPcooK94;iM z!VO;a<4^bUN0asmq#HJH52xw%@pkAew z8uu7!ixxIi43T+?6zE2sWKGGRz|u0!5qlJ{j>79D+U{g>H^}}b)HT?RrK`;U*Cq;- zILB4F*&BD4YOJOL4aKvM$r;7Eon(}zzNBa|x(YRSE;eLCukdASv!rR8e)DYsBY!T> z$F{zSQMAY9UF}9ny{ABM=fUch4oIIIk+VxSOM}iZc&dm{>%-&6ye`7Vs{VwdFQAR; zp)4Q!c~`n{3s%!giZTD-U-G*&Z{Q26^IU}MylhyYmjk=MB!E9=g2DACka76$x`V8!%Ezpt|C4x%#2Mjh5-`UU@$tEB2i_p#``bJ<;`>9ZDGarsABk>LfHJNfcU{OBOE z-VcYNVD}X|3DGAkFh{JO8saaj45_E7g-O^LL{;|8)?!kFx;135N6iyjJlLB_(v|G} zjLn5*?Q%KTFO}ef#0c&w2zzA6L(x45laS3NK~47nZHM=6C=L6x7 z7Bs-oO8+Szyk&kU{b_OVsoe4@lEUrMXK$-DQe|6m-|Em3m0VZk5!wPTEf)Z~mvx&pvg4GpyF!NGBP_m> zPoAbw+vM<7chIbbD6h$eKIy%j|At<~-}VCfL;zX58)e5zCC77!nR3YRUiG#{pJ0n_ z`1pRSZj7^8Y++E1Lx$Dt9fR*3A_?QV;>yi(xGJ}Y)(4i(#t&HSW#kVD#AQ>|ZMLNg z3CUl&(CQ*Jc$v7M3|7W*9Zl2c?0$`n}FrgX>5GmI8Z!Hf;B>QZT7@?&O9bJDaMY5BgE zDN6>I`=BV!+5${^EpOkwfO$&ub6br3uHog@D!v$#<*-1;5QC-59Z4}SNnFcPl?dft zln>&GX#D!Hngy4?@nx9g2o|ogJTOIj6^e2Im$oimlS-TTvLB_vn`&{LDaD8ii?Mo< zJ2$|z^knL&p!0Vu6oD-1E(_Q%Ye&k z8Ux=t_N1|D?KM~LvPpUP9?qXX=AyYFM5r6qEW;{e!`{K!P6{wQS)oO8|CGUPKMk{L zOPHn0xBRdC+M;PsrbPT+U6= zvhh0oZZE36%vin+XId$cA=$?5%Z6WVFz}((i^W5wev8M&{HfBn*-NE_i$}%kPH%a?w{)n-RnT`-tnBtA zr2Q2`!md)Wo?NT^YyJe9!D4NzySAkl>XOI%@Kn;BH6@HaiDX%6ou)W@(VsmZNNjWG zwpOyP1#Ic==~c7#S2j5-soT1Wy@lgK9f~~zlEZ~lF=_NJ=FU|5XJD}~7HP@tNK_h3 z(?z0}Iya&n)J=9)_Ik3}LFf(b;BF5&2b9z`i%zA44hn^CvoPt$vUPVap($Kg+t2cM>QKh5m^IkWfY?9QJv zJAY2A!|x|^`~7576C(Vd5=NI2R(0Md3-fvWL}QSB6&PBaa!TWrB=-N*%N6i_f?7}l zPn_ztQ+s+!H;qja`cqCJD!b4YQ{LB86DNAP{vxXykHr5cv!_Jz)0FQ2DSst^Y|l6>N4i^+QO`;i=zH{1v(oHry>uR;rj*09%W~Gp})aKf*TQ#ZE-28WGcxZ>)0+Ibi+JRw z%wJE}`kc3WK`L@1Gqy;vx$l{CF3RehS4Bcoatm)I-Hv+!bERX-^o~1btDDoVmV=SF zH>USdp^$nJnl2JzKz~eL0&aHsS<9zH z$PC!+=)IgOEP!XUKYDZGKC(hP*OFO{#Wy#(NN8GHGmhZbnkVCELo0G(5zke>1c5o| z<;#h#ZTnfoxdpr0}c~BSL-bHI9`n#aVsPG9$}K8pW|u?>yXshqtN2LvB+<q@T-Rp`o|cC_TvhEh1HK6#yVY2%{@ ze6Wj$4@>(G7|~(Hb*5BtQtJWKy3kyAu#EeH&4_9%pLA^c!iQx|=;6l~U*%mL+BUvB z636rHuL2lK%K~#B`0xK;D&Ggd&Wn_Eq_Y0f)=C~u7jXYc66;u4`IU!xb^j!-^^G`X zF>HKYOG_kkspVL9Idec}y}@gr)IS^9@2jkG%d535Hz>A->eW=yZQrAwl%|Si%lk)p z4-IW#ebpjNwp7D3w8D{1d)FWA^He2oSLgI>(LmF8^{O}VYY!jAs2Wooq_0q((jU3& zhv_6W5<0-xuZZ=~97=qf^eNJh% z!zPtfP{-z`uAI`DfbBFG<_0V4RGXoG7Q#2=Qg#cT{D zfJ4pcM~JcFFPj^COqQJUrM5KzS+j+D9*mvC`&}F}Ho!0=WPg_X(l_;^@Gj_2?mlpH*<{S()G96XmBq%1&_l_T&;NYMVkkyF0I*#Cnv**vOiJ zGoxwF+Je&=P17ygXW$3ytmLdPVYsLJd7J?Uo#=U(t#rgLcWAhka3Lc}5#qu;+Yvqs zu)}?uyYPNx$8!=Pfe{&4$3MmFM8`t=6K)Tn&XrQtgVtoq!xG!T27(B4@x22GN+HYR zd#2NVi(MqY-SPKbGz=O>td93;SHv5OggDsHDxxJ|XWhH?lA^@Pet+HUlPi7JqaS;i z0}*dN0)&kuXVr{pX!{*?4&Qv3c2~R>vEefxyK>v3;RXr0Cvz(~QI^`!37!?k8G~Cm zc0Q;EgHEm|#>3incm6~v*y6DUn?EyxcA$VyG2edA*rFn{1v55Qx7Dav*(=tzdpfWA z0VtmS;_Si+G2Fs&0mcV5r(V71P~j=X1G2PWq<-+&RqVK{@X@@V3~m}Xlzf`pPnjU- z9&)lAfKoHuGdW~ml3Uf;6%8+XHgFt7(lSYyXqRYi5ddrv^wXAA>03NTJS|spR`1<% zrRZ=M0>&P2774ya==|>g8IAruTuwq zO8BXLB0_{8Kj9??4Rs^+J2K|D{|U@ajsX85JSd*0`>#`be}Yl!W^8f70HT~lY$_@| zM*KTnGyU4glme;F z?3tRi^T(PNq-~m(0m13lM3eT5=?)(a?I{YJJ9VeIlgO%O!=m-+3oksMmO8R+0JI}1 z!V+FMdN7GiXYN98T$1zUK9r-uV790*|6m8a59NTXOmk=D^G<_-CmM<4SV(_rXMb9U#uhcUYaE1KS=*R1&qb1ebVriarex%-e&T<-~%I zpaY1(CQQ_QTs3pycWUJ$r>1^>bcf{!MXq?nCkgw?ks)jL)i9^a%w1r$_02@eXcceV zuVwIV?G;VywP9PDe>a9KXN~X`Tfo4D<{AuxAAi68r4c^F1%UQ8hhok%fTYbw*wJBR zOgD=vv-uMvc*Vb-oa>lgXe2rwW>~Oh)BKM*+ZxS46|8qTsEy!nstl^u9Z?y*8ogo% zxNIggN5bkPjJ2C&vBOv;fV*KA4(7TU6&B!%ZSgaMdgyn7pt=1?J1VuRo>gqpR;st} zui9#pkr~#MXwh2^tJOkU`|+m8^8B>Nz$YZqb9 z(jBl}lrnUH9gq|ZpEwQ7L@GvP#OPN0(G7f)!f7yqiw2c96@!c(wEW3caraFg+PV@mDIjjz?2*}@?8)eyVf#E4fG%sF7~+JRNmB{EG?b?|!T zu$e?}!7hK`0Ar(wH;nH+b(o~=s!&5&^JUUYDye7SD%K`-UZ_5_4yqUKBixVCJd`_b zhF_p|r?lgeNM+NJR+D+-Q@@4wI8z!ni84VIwXE%RFNZRYH{vqGTYk(H7vJf#ydjiC zQTqDI@?@Dq6PZi@MJ@AV{xb z22lx?x#$GT7Vl4`?oX|qe?=(#G%k%)O6@+oi49Ql@)5+hvPcZ^pDx3GdTnX<57*Xe` z@kI0Z;=)IqoePYn;_gGzF3#1iF?j!4bE6zR2$ezYM?(+?sZn>d*d7ZwFk zqkT0TIq{c`@`pN6BfdU#;%zZvq+*7U#jLsdm@E)T#KHRrF0YEH7Hp1QuaxWy5<9_Q zC@nfdL)b;Uk<_2Sy+d0u{#$SI=bn7dlfURXB=ct~37l6!-(dR9VD79x_3qo$yTRh2 z61U~uVqV345}*%m2;MhmQ~pVS#Ut78+hQGt+YYHoT-oi-+X^1GF0c{0-QL0>WDA8Z zV{)J|YgA{d&QnTge-}-fr(Vn!GXfymWt$fX8C%*f=8YaIecSD=Y%3F#v#VP@ZaY1d zMeOU@B~uhG@2Rp+^RsU^b*A@d{89fxd#HJ{Y%H795rG#>&uCK<_UF`jxrdU`+m|~! zuwp#1^@}ZUaUA!hf-i0^001BWNkly_oxf07q})+&dRNtgQ+t%}P41hT@8Hw; z(dUW%&(rE<{HLDX|1`O0+kpi>_&mA)Il@ox{yBd1Y5e%}#K9-IL?NB2`3xGGoA9BZ z-POo-JrfHfQR%j?sd&tJHV}^d^Uh_e7@MH`~Le4>IXUtBQyol8;P9JTNjYr}wK)O@duDUpe zD&Q1E+DvvpQEzSOj1yJW&DuI4nhk@MwwMMFVux7JXdpJ5<x~e= zF{>TmPBqjI9@LN9M2ILl zuRQ;j?e$k|RW;^~`=A46_InI3f*0Jczsz=CnuY)Bt8E_1D<--|AI7iB`_L>xCXYU_ zwO?b_cGSEDta+=%hNhkkiu+`FC5*V&Y5B94Uw0G;s|Krfa8(d*c42aZuX+4xqJLz_ z=)+fC1Vm|%`3{f0cMGIJ!$WWCy|uOs+MVR28ezVnvZ!97$ zH1x5-H>>ry5zQew5|mnauaAA#hQ>gr%P@f`d$b|LYfqZ~Nk(1aPwnM{9ta)~%P@K} zI;L0{ZAs$9N}m5Z)P?%Er{C&)AlmBfYYV$#%6)0+blzcW&ZK!=JkU=bOOYtMxAlVN zt=d#y;G@pc-ovcbHquF3JivM*4@<`QFa%=w&q5wNXw_<}VXe3hJ5+Wb@ml$)b$;qS zt%7FGA5}v5ld-eEq;)_2PZ8b=`y$l)_^){%zrOE*;7_`JpXhp?g#8VuxBi!G@nwAA z{A|q;*kk@FkNJK*XOu5f_SOjs?@1e7mZ;myq0c|Y0HPW}t-@-=<3WuS^;oLYHiv3O zB(#P+c=%Nx6iJm;NRofLm*=H#c)eTp4_WJtk40V&@U6BFT)q()qfeDH6{ydfN_8SQ>)J*1f_!lN@*Ua1{ z0SX$a8jOJu?36qkaKNV9<}I4T)jCabJ`vj8 zq)R|lF5n0qk}tT?Ed1dTFUDh~HmDwTxWN-gkhJdIlnxg_gAWQ=Ro`=)_y%nJ5f`HJ zeQ;1fZ%)HFX4I4aAoD;s0V`1E>6Srwfd$fXG<`gvC{@Q11t4*3BzQ!$%*`HTcwO*Q5uvdf`l|2vAdUCX4r$m?AV-&BV%?|5nn;1+>MBS85NjDY_NAv;~kS zuo(LZPS|5dOU$1m0gfNxi|XYAFX5^Sap-dDST+j$2j*T5wX)YPaCOs(Pb(y?G z{K1T53f>r?@S@~%`xHIhvTIA_pbfkiQL0g2rv@ej;jJQA zwqx1!8ArQ?hE{?}UrL(f6`zuCC3JaP_bXcotNC#k{cREva$OOUK|nXfcDO`0DEn+% z-Ad~ngVX8G=FpBa5LkEej#O#nO|7k zrgUz^uw-*v&lK17Xjl3WP2dH?BO7%kg>*aE1#hs=O{1tz!+n*y* z8Lu4ez=n_ZG>h4bLY-a>V=Wlonmg@_sW5w~l+dV_A~_9UByRRnf5nQB@H3Y>PoD&% z-tfpYedeF5q|aPy%}W>P4bHYI#8U~LNf8%Zw`l&kJ%&*yVq=BJ`WIkv&UA}#Jyrs^ zO=>xBKE=xKjOeM8?}6pPmJnIqrH8N4Op?bHT)vkwsf!WYSzCD-N<;IEHpgLG%GnGj zyK$R)(MrJ5UDcTuHP7oWyOR3a7A8GZ90_Hj-o-JsmTHOZ^#}*|mFrD%A!DO&M~d+D ztf4ns3;2@?35^U+IhX@<<0T+>=BKPk&X+Kr+QToxYn+p=5R^oxECce3c;t6N>7Jzs z$)QCKq-YZt8udHSzgRjM7;$?0lgkNhI>G~mo4Cp{V=}{o&+|w3x>&H;kaP|;{0F0v ziZBzVVR_;iTViatmry8*ZC&Myb-C zgU;gq1Kj>-Q=dk%V?tlF0tCYFa|Mc>EPbD1; zkD4j?&>C{Q%J36p_0o1s{HiS`r8ExaynCERb}R04gn3ucz`3TcKN>A&o%U^5-6WD9 zb*tW_rj3K~&D4)VABozZI(vGfA2~jUMmEV6uh>_|*>HqJQ#g{DBJ2ZLD>?o*Ymr^d z2yzOc0J?eB_ha`P{X-pqH0Nd9&e#X`J!;op&aT0X&ZOqqo<*&m5Nh>?;t@3BK?4DE zv5)X$23(>8kegocABrJN5)(z7#muFu{0YOybI%;zKh>n~tPGSdbVd4iLuwnIxag+rUZev(NE|F+4q@i`!p$%{B zKm7Pt5HR2!m3PDY>%sC~4s*R1!)Jr>y`a`F#WNfxA}5tgJNMs{$CQni0r!q)#2cni(-OR6w0SW35*S-UMth+W+UFS8){ZQa1!~fe3 z7GK^ksf%&Hdi7Yl0*mj_ECRo?SJXi{$IBJsf9R>LeZ#3jYY+_HBo+{(0_B(lpw3P* z4Nup#RCDKCM;JAq!2m=IWG4w(EIcw?;r!kF^dSAKg|fKLZwD11{1knLxIO_w!(9@A z*eqkA^+RULTo~PPQj0PaqLU_OY3P%nQMbk6|0Mk*d1RPmWdlCw?*KM(z_OrhBZ|p9 zQMdiE1M?rXN7*?>C8Rb(GNXbdV=7K%C85R&j^3Xd{wj zDp!MrR;krT14lOH#X>|D?)GGY{b;-MG6{}}t^&%T-1fH8$HtJ27|lw>*h=4W;9v(z zNeo-2ez3_eJdtCEFX8DKa=g%jRiQOYfZ2(7GYuX_8|*6s_|x^-6HM;g&)eOyJ3F)G zPL&96#s(Isn@f-g*|U-=wJEqw0w40=1ZWc-Pg4x=8{-G*^c{p44z0;AgLtdp5Cm~} z>MO*nP!6Z@y32;^g)_}QCaUyzBJp*8d(FRiD3ut=X%pMw%@}csY}AYijJ8AL$qrC@ zU$fyF_|JoC+svc=7%&Mn zJ4OXmbya5Vngts}%GEmkR#ZObJUU=8Y1B0uBod++|B2yWNn-kKSK92AOwb3cw&ENx zSKh&ucqLXW*-B@%54V$88{yzi`c3ip{)#2H=Ehj=#cW_P$p8udz!|p_g4EgK0U|wz z&^8V)P7>VE;>mP{0XONJ>>0qUQqn<@Fw)D!r%~4VjC(-j(YHpI1s{71QNj*^ps(h` z8ko5Q-(Bq*aL~yeeIIPTAv=W`0Qq&dLYYWW1_XmJRPcyvSPRlj0@kp6`qTwz&4?6m?OZLPdY0@OHVRvB@e}QeQeyk_BBJe1zvQ(cZaDGI*%8 zi-6DcfD4KR^*Ehsca_|!`tMY)THa9+pE~418+h6Wp_?(;4V6cw5e)WRWsiWiRhuKObpTrv)5vOdoJw|D^OV%`*VeDGyf#l-mlibdp~ zn#4Uw7wy_ZvO+nlxjj-l3ZM{?09Mk+c}K0O=NmnU2PJzmukEg_^gtd>irCYYL)2wh zRfHzKQ#Qwm{Ps*dXlzbZdq++N9TAQne||>v1|(;UZD_l#T0~0eQN)cU#O95VBeoKIpKuK12Oa9^e)LJh^I!o%bWvNj;mMl=Y>NZhnO?L z9rzb`pV^A!P>|zM$ER|+t_kiG-a(8ehUlzcrC!PAB#Zg%7Z1pu3@UZZKlc%7Ijc%y zLs+WWJbjO_U=7JVV5v=oRV9uB<{El%`UFVf1Cv@jxXd!`I^{Jgassd40TP7Pt*KL)QJd#L4J zqBC3zuOO!H8q+S@u%%7J!ND+Tq}Dc7 zbT5tUtTnqUHj6I2OaScE?bnwR!NT$8cWjzL1A?jw-`9?%%Ae2p7aK-yBr1$dF!0_3 zyos@pv23U-ke2*a>D*64s4i!G>;Ie;{xe-+=>FD&`aKDdA9O`w99+-l7F^Hde!0Hy z`*c%UFB7bfsFE-6Q!jSt%s~M2 z(UBxJ%n{+1BSMQrzt-zVF+zikNG$>L2NTSVx?hfLlfQJwf9+P?<7zbfUUY5&w;6%v z%am@#EKK@VCo}n$QyGN8XvYUZG{!zUy1Geq#j{^~5YU1CXm6RWKLrNS(Mh&hN_$aL z;akqv4}iTq+;8Q03Gq1ply=!bKdQ44wv2Xu=`w-6qd0xGev1fgtD(OH`l#Rf19 zXVr=Wrkbm2oDUjtbEp)^53(e!qikxXk!;#Ns3r2@8MpM47_FMkI6-?H@+@Z{rA*F6 zwPOZzg2@Cj3zZcSihC=vEx04cAS7`LRQ1N9+$!9akTq?1Mh1-u$z(Y>P&-F+LQRxm zPZ=Xa72X%bN%Und`-FV~E9f2@(-*p3S&hXN5}0F+9Wu{AwsF7&!#snIn{-abdDY}TroGizbZqcxo+C=0* zHaV!~19HY{9P#s2?8j_c!6JDVKBFo_oIij}siYUW7iw8Y16UB8pT$XK*~bmaF^{8ZN-I_(cRNr5v42@7Jj{moW$#kTZUS&O5i;xIRx}q$Ft;90L0TNhm$WNZ>QP; zqb>yVtxP7_;(#!ysE#w+cU2pnh^Nv_@v|z7Nfune8%?H9p6#Q`?p9n2MZGmqwQKjb z&c+AIR!LDFm`UwO_d=cb9Z(}r3{T|bXnY@RF@T-zC%592q&oDaLOcUWz36z6-o#rE zK!MrnB}r+xvYqn0>U1*m6n@Av7Cn;8QJQlXuI$3WnP}4el)sMuts4U)nKzSJJKhmj zY2L3rxPgzklyCF{$NWd#;K4(w_=pE$gLDGJm{m9&nOHB}piuAIcqzqtF~uR+qDl;` zJ5uwJf15I?SMzy`=&1EuFDm`S7F{wBR!~qvOczDnhdXLvRXB8s#&T1~CLa_701OMf zEc|UvKR5s*GcQ_$eMY#{BUmTb(}RlJ^smOCVnAEs!(f?WK9%elO@mjmL^x-WfZWMJ z_1HCHICtKkxlmfCT++pcz=-pvXPmJK)E5uJk(qoDw#~|3vHq$j=FKwPe)s5)ylg7 zEmyOr50LWF{nk+3yv;&rQbBsrhQk^(CS+UZ%9RY6VFgUiY&G@P=D9m`o!TkMU^wK$ zR~H_^)33X_?#CIq*qq7nFx{0d$Dz~wJK_QO_q{MEcgB(!;NLQ~?3W2NnM% zvp>lXb2sGtLXXOc-q4m^l^BHI6d}UeuxoRcpW)arDdCH+^EOv6y@}1Gj*@d%B2;`P zW!?VPh2m^Nci0Z5!B;oz(PyUN!AUTDfYbKbVA)6&+GP%6c_ijJ$le!^yUvadgWsIx9MD*Wb!I4>@M#R;mM{olFcm$K8OkgqCfCKF08a;fgL95 z_{|Z8-cof;{?av2$T+ z4RVqbMokzMMT}c)aLtpo@3z(NCRoVpLyWsfBU`!AzPG-OH|qP(F1Pi=u@7%8%xM78 zam}at8UL*pQ|p8Zr{-!BP_{6RK>4eb0Mp=&kpNho)PjU3R$XNxaAcKM*(#2qIzFx)mM`+^~5R}{JminUYBTojxU_BHZQx^*Mhr?CrMIGPQy zjQU&(iOKjReYw~(5Kq8g!oGHLD@+)HQ+s)H(4r|+Y0>`=Jh9rMKxf4BA0n`FuDIeV z04#V zwXQgLX5c#4z1EQRG?5%px3RX7^dh1JZ6W6@OxfjujMk1+-EJ?p6TDoO3tNd8NUnwIdsFj}0b;!r|M{q*}F-Iej38n4ZtQS#4{izma() z8sI-M%4`Z!)_oa|$Nh$on0OIR(_Xv)yCmh!q4MWL@v^P7;@$zvByulCl$e&}0Hx~n z9m|j&#Puk4>yuqa{|TuK(0Vc5W>yGzyRcAwz0a!}{$kWwP`ZA6gR;u>5K##>{Do-` zG$A^Muv2Snh!0oFN0PvMf*eNDwVMQgkyVKnLb+vE?eTZ(VL3Sn-@BlXr&hxM7v8>s z{EYg=FVP#W-Mo*OZM$C~D>R&^rdK)}f0uV2va1|NGSU0hSGs}y#oUFOP!>bv5ZP9G zxaoNdIpL;d1_^l#1dgxn#bB;9(Bto`OyGTw-!ff|`qE1fD?+<3iU^AkL|8vwgH8P0 zt6s-r!U(5MGBWysV#A8fAEcLj@TrtPhNaO*67tE@S}+!=8L%Ci0UxmsJt=Xaxl{Gc z&xmQ&eA*_Zejhc?N}X?=W>xz%{EkqqBN91?Sz5+^>rMP(lO@^PMI!k@kIT-~Ta~;4 zZ!McU?SewIfkHi-B~UB}%&fPvt-s55T()JKl25teZUN2Ul9?9s$9=d7FDaqp+(xtK zs-y#~b%WZ}Sp_vV zckiXMvw0cYRDk^@w|{VzPHqT&;*m}%2v4gSb(HR-Oq6vV7+y5DYsvPKOD&M+1-sM5 zeFcbfDo(Toop%chrr31RbdFqHWzS}8Jm|s0S2cQ|dp6IYdhfzBS7#yoG~qJ_K0E$A zX~Cv;K0UL)HX9(leN|Mr`)OkT^Z0=VQ>1wGnX?w|Ma(ucps5tcO*6>Uwwkw<3#&$$KZBJdIc%DRyNXQDuXyXNyhO>54vsMl$&smpYV%{!u z5bnSYncVQT2G)3tLQcL@lsRWI%o#)i;hyIKZo+Vsrq{L<%RbR9{kt#ZX1mB6vH?fw zX^#XLkuycU;Z3PaweN>E6M!I6oO(ld>1m(wQ27F1A>IOVzO?rcj4rmL9N@J7t{(sy2jE3qAU;b(1U z4Pv4fm`rWx)c4}YI4A*@cROek(XIhV*%(!7#elj-O(o}Z+PVXTA^>fS^zZv#)!V5` zJsG{)NO_mWTI9ZRfnT-tK|9OF=p$5k-gsf?^oG6AnyzYMz4z>dJmkL)SdpQjvQN<_v|)kqEJMB=-H!zHw-!e zv=Y&u!3|ceyCPBPjZl{8X?eF_;?beC?E*9Qu5ak*nl3UXP1}w(js}5N$yM7AsDz*Yq|@8!yPa)M{-!P7fBzz$dtiwqzmUg=!#~~6zp0lgLcKiy&|W-^ zIURpF9N@1$;rM?4V*9JAAk8h1^u@!QN6WfsW2hb&s%M^Xz~RXgqNd{X3mfTpC<<2{ z9j?FnD!dp~DF6T<07*naR8O;V-K|(3+xoG_HLlrt*6ye+TU-S|zs6h5Z>si@leMAR zkq)p~_XCP*Vfujsexxl%b#R^Lt)meAzRFG$@R3%tVSq7or0Z^?hB>T@tFZP6D?4m+ zo2R}=_HvTi&}O7*Y+ofTcUZ5Zs3(2d16YMa)}oUy*+zW+L71+Az+j)M)<#sX`8agG$U2=~Su;11fDB@z>dlahF)zQWO1`e^l=NG4{S|U07CRX%>n6b*!gy%TX-aBo299^K*@gz8Z|u$7b1!z2@MO zad%duRo9%B+bL=9^Ko`duM>TW?o6|(^RgRdOZ07<{4S;uT;0wnn_;_H`Xf`@IR_*Y zAnOg zT*=o(*-ZDox|#D(yT_dez%}>N=~2AkuEg5;{^$+`c<9$&so8@Vvn9-^6yT)Am^7Vp z58=H<7iaVXXDf$HfI>f)3R*Ws<$j-p1y0B7O(Q(FqzB!Sc>9QAF&zCM=nF#Dv3cxK%J8G!)`r(X|dE}+gMF;s=% zmaQ>j7h?WFIQe3}LQq1R@6YI=i5yK@sArY^osmxZ5+xKrN>M=ThkPz#72^YV5Mw11 zrJuUQ^LfIWPe3FlE7Wbv(BUC{rQyVFQ@?j-H$P47{>ga&m8|JRemY~Iu{kVAGJeD< z0wL3rzz9EK^AbY12begCIR}mUP2k9$!bFMFu-&I;CtolZA*dhcJ+#F}#^bopHt2wl z$Wn!hCyEFu2HdARvnFlAL!JH5@}^?OUB+y)86qm(nLT~Lly=i^oH|{=tGB~-eqlZO zVwdwLSeC?8ekZOdJn3M9s=#OFr`!qZw0QWyg*%`eJf#=gw5@*v|5kNt#flj!283rW zyji(!hEAfdG8sTXiuK?qx8M%Zu>uJNfaktS$s3zQkCX1v^yR`G7t9Q9ZQ|#yD*HTs zP{lc$3UJOHY2gm6Yk(egs6YfBwG=0R?n1YyaP1gM#tvfOswT&b5Xe=6P5jd3Ood8I z9c#X!gb<6R%$)#hQtF7+LZ*EIV&i>kw)aF&?)O?NDLz@Dyb^2fARJcfy06HIwMWoQ#59Fy zmDYxx?ACLRmM>S5X=hfRtWT2rnH))?gaWT$(jr|g!0k`mn)rG_Q!S8j2uZB57 z%vZQMgaZFELOCVPC>k1hF~&^ICp)mlYqaov_9Bx-jQL;{C?SU1VXcXY0*fd_YUp!v zeAl|i=qzbl`JqQ#)nd09*lx_%&T^lRK8-qdyz?pfjHX>trc}Ff506F>)=0(m5jxoR ztIscnHM`z>Bl{8Be_KnWo&-c(!3;Q(PQhHG4}Aj@RuD(V2;!*EHw`n)%E~SLrcbX% zl_l3XwEja69gk6Krrkg#{=kqP3MVF2)Xs*H6g&&`NpKg^snCarvp$V|HC&R>pKsIR zlI-cwnh&R~zwBbvO&e(K18Cc~aID|v!l60;?mPtbGj+4#JWM`FdEhaobEQrY%0UPZ zr+tMb$ck;9;)g|>9b(4kFx8b$4lt@=Qxm-5B2LnQ{P8DZujj+`?J4Zb&L;f#B2mKU z-}Ac{LqP2*9Mq+UhoX8Kc^rH~B)`LyYa8hZ{tkiroNwA<<-w=ld?1SQuPBY~41;++_TpXW_>ts2be@Nb-!h2?)u!i&j`>^+6u$ws1 zKVASPM{o4dbb(bEvnpn6a4!DpO}KgWt3E&y^XfLT4Y#_v9CX^kOUrb#1#JaXooDRx+lC2}L#hpc3CpJq=UgqXxhs)NXQF3nY!!AI-6cr`S-M z5A9_sAIvZf)*(e<@u}QUq*n3={DcYzIUa$~H~3(DKYn?+nU0r+fpOs2U^+p>f-X7h z>0=lc^%|swcrW9Njc1U^ny$6l_sYl@70N#NfL0af@qy|iKslgqo$pi4Sz#puz3e5p zj)Cj6os-lCGDe-EeNFV? zuKijspY)1ah9`gNtwO1{l|WwFe3g%tSn1f+cG0LSTXZ&;=-##~5VKhbhj^mL;L{O0 zOx#Lot;EcRTUg2D9syQO=$F)v+i43A=#`914xQcdby0G?;cz!47cEMEMa)$6-O(uz zmaQFe8*gEEzrkTTwFm0_!1clph08f0^3de z+;eIPBSg(A_WL>YNuORt)O29Fz&?|^Vdo$QSV+cG^GtIE>Vulg7OM3m$EdiR*<=yL z?1l0^2m`TJubKqrx;t|@@LB(e_U2B^1=yw~(h2^cF+3nT1aX71Pa-tspegK*C8}>g zV{%Y}f$NzgaIqxziOqSmwBG|pgtszgFq}IZOuhMLE1%f9zz6*xaiDlm{~4s?x*{rI z+$!ev@&S+>1&Jn9T-{N4hXsh|Jx+-GmUnuNK@DFXlVGZn@xb}xQGdy}r(_IMD{599 zTrNNhX$vJUTI8y~uuPhv6)MHv^v>td)+dd?O z)aXfg*~YOB$)Sk~Rc2@l9P1^vZ(hvG_>RB8H<`sb^;oNC=`m#$Hw`&?bgI_b3 zI(r>vteZc=qP<<6>h2WU6N<=SJj3uh>F_!+Wxi<%O}=OD z#;{QhLTL`wa6Rk<6pJ-CV6OruPQCFJNG|C)t4ctlV?OzyI_+lSL3m87;BM-*t(0=c zwLzxd^yG+9PV?7uY6)#l7}awox`FTWjA6c6Y6-amx`!oJP*xB^VmafrNJl92Gwx{)3P`YS}NCTQ@^kv4XDMn`bD2*iQzD9dWK_ae2u9&4G$O*g9Br529 zE4R~)RarQ};yLlL?qmIvOO_bsc8n!iB!^-}Kb63IM+~7Qj4&(TEnt!)ewjMPx}Glj zixD-Gw7WJ(_8)R!7x4s}dx8a@W^#k8pd><(Q_Fycn@=xU`zu(mo*(y}Tr8PHi*`gB zsftaeY@T{(X!;=or0y@109c2X+Pz?2Xw=XoC7pYnT_%|E(XEQ!GW{Xkgrik|lX}H4 zt*J-2DWYA&EJNBHTko{mNIl`&H5WbxRHap7t=j|F@zES0`tAhxZ%by>>G|Sk7O+ed@r8{`%8IXQV|X- z1fL!^X>jy-pM2)uJ{Z&mrA5dh2}_s4%!cm7*-@4uJt{yo-vctVYnPHT83#N-g^ zGYnRm>7k(R4b#3HtiW9rbHN?VG1RU>KdADHH>aJsA!bH>P=%y&yK{J&(G{v7UEYY1 zEjCP7Ma+R4w;7QqxO81Og>V9!F`U5DiSC7VFKGk$=>aHn+|K;fsv&;5@hvcPKrL++ zwY4gTL@~Ho-Dk`FP-OW`AT?ffw7io)cUN|-zA}6N2Z~YM*v5z-Ck;Vv4EdLG;O$c| zmaD(H1w5}w=60gJe$2nOHm_#V6*y=%o4Zxd$D=iRh<2ePR&vp2>H zl&<(?F0fsMPBp&LiqDJ?a<-Urp;Gsih;1;((V@?%sA+M&$*k*^uK>eUnHa!V-U!v6 z)dHL&uk7n#*KUTr=o>{sw7FV1yr+ZHXNtM9Pwt?_((0A|>;F6UzyF_eOG}kE|M%$a zzmL^>qs?NhUX1zgXsZ~#Eo#5_Yw{~!X}cKpt=8_^nek5hZZ}8t85SB(=s$;>=5&mrN9oNZBH`uIdK0ky>oa zmL)HeZOK|W_nhB3x2i!olW+bTeD&e0uDV;@s(bD|zxzAqh|6aQghx(i(4XQLCL^-R z+92Dv6V#ZtI?brL$0PtIBDfKZ%Y-lMe-v|Jzy^xSs$T{UxXctkAq+Nm&9%@m-@@+y zn!5SuD#7eDj_{m6@~@A6h}a{-Ni-*o;v#UakX=0#uym;JpO>wV2|GhzM-lQ=b~&(! zL7XPUIuw;4U63ejU?4JMf^48TF})#V_9ib+sA zgxZ;SI7B!j+#0g&mCNSXj6YKJ5^O__aCCiw*9ib5LL#Bu;*-BLB2 z3`%Kq>x?oAnXWj%$KYdP^{fGuJuB%xN`S-?(xgY*Oep6!lY=lxW7E`uc}4dYYM6wQ z91>ceIJw(LVJi{qBcZ+mLF+T=YX<>Z2^3g|*dv^hjgA+<)T=@CVYN<0Bx1mb2nP)2 z9W&;{*P$A>iv28Ff?N!1&f6gf7@ zHNPEPfG0jQQUf1LOkvxN;30yQ6Cu}%9A<=+a19U2WJ%cTad(N4Z|qKC1;!4;{8{=O zY4GgyAne{T$p+NaS~UrjY<=6{qO7sd`Le{$>(olVIc!LdFtWwgl^-c^zr#LP;#E>i}n zyL<&634sjijDs>&>qfCiGYf{GNtt3lfX#eZhk&fEOGWD0P@BZjNa$|sY|76MB2Yl9 zfS0jWFP9W(wS5mL^fj-oqye@f{i(ILlV>zDC6Pj#7FM0iDQTVZjx+Nv?V4oDnisWS zrY+_paKH`A-NW?)f{@D0$QQw3w~I`uI@s)|sCttcfKIh3wN%o*STTuZCc-4o%f}s5 z4O)sIah7YZfqIp)N9*CB4Hc4MCSzJmvB(yZA0O=$Ut0d{|H*tfg)T|^U zji<*+Q)>}t3`i)n6}y%e#53)APxG2;X;Tw!P;p7lUi7bLgm#ctvgd?%E2tSkffRc* z8}`&QoTQ9AH-Y;velvImbm2U=e!!(=SU1@eH}=p-LGWTDb&VJSXehx+IIc)7421Vt z?*TN@i6_W0rp&N~3{P^1AyXQ$hKX`0!AVTXrW}!7ECw3@M|S&+b6bBo0=VaW;KC9_ z@6zl`;KEjq%v)mcdx?p$y^aBRLn#69hF0Ydw%S4M+a!QbfJ>mj=2ky;KWo!8x2sE2 zZ3m6p^U0&P8Ba(ImSUD*ZZaWOh1Fb;`vQ{>xoi zlbhOO2j|ILhE0jh@-=kSG13D=gIC?IXi`%$3WU?jTCGXX*72?lBMG{$S$i8L2s`ok zdNDl9r9%vO&G<;8Y{Vy*GDu7X4Flg!cUc=Rv_+#(yDynRfB3$4S`8nN__FF-a08_s=^0U zsh2wC*kBV zIxG6VIjPFI!F!*qxaW~bV{_;W-IXUV1adCG7M~?G{3=?o)fbB*SAq`){bsGgg(O|f zqQ;!EfQ=S4WEM-Z3Z*wmNYboWqjT5Es|?y`tLl}n1Pjg;mjrTQ=mU^0qgay)w&bMc z{N?jb)i~p-;)K@+_T@@3rJUDPs}$Q0qe8>x$V3b}6r2gF32CJ;U~D91#o`qc!w8k2 zDDtTnW#~;5wAl=6PWX_toP|vbz(6E{`X8*cfFrlr)M6$etf>Sd=$N~#Y?@=B5NZSe zrbd)Fg3e6KMfFlw;`^{1v24)G(_Wkq??8lUO8nILQ^sc zHAyhxV!&Yh*r=xFF5Af$%DBNutme|517jpsh{_)nwNRlU&m;`Zw47KEtZJdm!ibh- zuEnHmb&5#(o+vAEzLBv)TW+4B0x{PqP!WVLw;YYnX+;?+P<)GWH&%!%R#!0ADLs78 zV*0skTH@fylS&p>ve1BZf~TGieR1T^>VK{A6Mqmm^-T3-zIw`yWD4P2A!Ovpi9%?a zj;d3|Y9mcf6sq}W$ef%gR82dT4?G+B{GQLmel_}MgMs_c2dA8X5%*KlROuFeO!N|- zo#ma&-Gx_4ooQ3@vW)#wk7F{Uxv+ZF@jk5_Mu^0_Q16FCw&&HJ>;F)JHu7f!AB7IO zo%gyaF@HIf5wEo>?bP>kh(@z|>zqyOoeAA>Hgd<=@Sd}^ZL<3t^yj!Nu)|vg-lK~ANlHZ{UO3##H(m%Bn@&B&)wuT?Lpk$%vY%0SE%oI8wrm?yb-s#-D&7^*`c0`NF%8Z$LJEZ!vxjwMFR4G;+ed3Ye+D) zhJHfmOcqlJdpjzaTc%_JS&j&LLp(~|uZp}4V!Dm9Hs94JzIdK?1mh6@bI zIp*N>bW%?dnJIBSDLGWHNu25=Op_w0jF2ZmQ&Cfw)6l~WW4zDQMdfr+N^{akaaZ`3 z#7;47PM=hxt`od2hhDb3d(^%zy$m;Xf;n^F$U+Ehe-OQ zNtQTBCyZy!GseewBG(e8c*2SFf|hGtVR0?N(1~i&$$4}QAOr|as<8<>D=TZ6bu#ZCy>;n zXV`8b$z>9ILp*D)=9M91BW+vN1CCC)LYUptX^^9f1x30@5RM z-|m-W^dkD^6=bBIiIFj&(b8F;AEC}S2N#>iqfuvaC$JUpJohkVK}*UCdUzi#h^Gj0_?(87Y=v$Xqx0Ixu)lnXIEZ08!YfWZkV;kv-4>2l18+?6*IA7&8D zdYsrnVwx_~GHGHRh#8ZoAwuy82*X+wArZ$MxhwXAnq~sm7D=Gc@0ZZn<^EoT=xF8Y%eoctoF#_ZCi0=o@y*Mj*dGAIAH$?YgNn$w@W&u-eV-sFE$ZB-jm>zg-v9*^cCEGxQ3FUFfqMNv|i3{zQ?K`jdsQ`Ds-9lSNYPrwZZ8Lc}C9*&n=+_}!8Kv;Y7g z07*naRNX)SMEvsbUmvSF@k~|LF&H=JqeB=QORw#*wac2$gJT6cdW3r;(0Bt~$zX72p05NMmIsS}!9zmzsPujjL zd_NvmgZW}iWPYV@Dzw>yz7sqtYO#;Qd@=!Q@LH>n>BJ<2Cwepsk6@zly+px1XgDY= zX0Al;$%{go^G)2oNF&G_92Xb_)j=7@XG0U{r~|r;Z_vqD2p=)3*&IZW@PrtDxJ;V1Ia`M`>WB(O;`Wn+UPZR{d@SU&xKB%3r{bGr;A~LgGPdw|EG$fQ;Xpf z#mKZdDTYtd-SEAOADWoI?dDCRYksZlFAsFTbGD^>?$hH>9{Sf8@0)q$_J^O_amY=y&W7(en{1nDVrpSZ;lD;^|21_B z-LJ9BB6LdGTk=!cf+e?nP6HVkv_<)Nf*rFKb5yKQc@`bBa?R4Tt?fmK90t zPy(Y()43$2*DC8x*{o;V(3J#S-K&SaU4NGJJmm#%jp-+yGfIJrLl0~#29s9b)>U{~=Q89S~x(Ca5T|2BQ zSr>vE#Gv>db!BeQ2veeZr6ue#^pNm92)qjz&aGvN9D!?s$`yhA!8c!Hw*$xz^u2lx z%gacd3uYfjDRG)Ro#b!Z=pW`*p9|VxavvqG838^>kxVEY zN*6#zkIZOJQe6aGGEgW%iiy<-XGsGfKe}iS&WR^DMAS%xD`A6Zx@bX3bE}pcK=|FD zOVq`ILUUWbiFuLcdCe{MS4cp<0za&lZ>>i0XY|Mj6~IOjIzN zfF{Gr`J8fUIDzFSc6b>vky#VGlse(^<8gLI10^S=B`Aqc4<#OI%Z1&OvH_7>t>%wf zd`MrLKmg4iSRk~ZRQpn@5vq_D?wvJ#%bQ!SEa)c+-pr-*1(1b0@iZu`QBoP(oXgP?Ni$(6bDqeQ^(&bACd!B=!fcB^t%1uQi}J< zEsNs$kv7%* zko#OAg93p<$a#I79@$-BLma4J zInD(hueu0fj8O#~-JB z`Qu7C&si`*ninj^ZER#=K{FMuVP2aIWxa($!HDLLV_L0H&j2Z;cEH`yY?CBZ=y;^2uw<+NM%(1lRe5kb_#d598cRQ+1c)K*R@*Y=h$B69XV&EXl z^l`PJuznBVh(W;Of|Ys|fQiN2{&>N+wFEbK-}v3iL%F9fgo9A4&1|5x7|VjG-^W)ScJqfAh@leN5Daj zZ6ch(MoS~;vm^6 zCx^*7BX!O}vKl7q;~@tA2H{T8r9$>I8B-xnraDrGpc!PI+~<((@E{xH_MO2I>pD#+ zq1p?_x&(=f^$>CgOi=K_0Vet}Xc!X=juZ5r2@eX!7g1~x=h2?i3@rMtbe}d=5WLtg zMTHpGv+ap6LolpH$ELM}X%0QcjU-wkU_?SQyUy@IRhR@8g`@<4DF+`*FrO|tL~Jti z-{_6txYZ5`Z;p9qV(lw-!5oBFfG6U(#IhW7_ z0S=<^u(zeh+%HuSST;$=N;SJlIKb#)x4ioU+rw zB_Tn)T_ye5)ZxW47Nu9Or%DS+`D1$8c*KD4T2<3-_a>tyiDfwpn9pJX?E)60d@b}& z*b>mEz|Z-?f>dcsQj9GMD5)mN{=={HD?s4kgXX1Rjf!ygn54kY!`2sD-MSofn78!P zl(PDm1j+R#oiS14fr|m9qJ42zvqP+(%lw#@>9Rr>>Omx%u)vv?ql`kv5yU{x&S?0F zsZ*S_k5a>65KZB~jxaSTOr3q^TkSWKr~fOmGk5VO4HH2FY7naZ(LkhhSQZBD@utzE*n zr+|qLHcr8gt&0!#IVdNoPNK%M_Y4}~9GqmX25uh)WKf%tLbzf7Job+f!$f4l07-b9 z)H6g_%Uc4pq0+~>Oo3f<4ZUPsY>FkR)jJPii=l1OAW*WY*CeZDFp&n)o~3}`fA&~{ zB<@kP?tK%qUXuROD|;^~K})vjoBj0Oy@K!KuFcpCe*$fK-sa`fv8)0C$UX#x1R?Q_ zZvSS3CxXN&fozcIQ+QBdyI-LBu5^?NF~5hwc28jX+z#p{idliTpN)_U!h&)G>3fSX zix7)yUaY!7p{mWR9WdCupo(~^&s?&w4 zT+yVtHtk+;iKo^^7$r1O}?2 zJE-8bCBk2didf`FW?mo4gf|P(4J5aU=QtoOGB0G5J(`hE!eWbdnh@|I=e@x)%ki4@ zkSJ?=DsXeIpce5wS~E&|%gR&;Ow-PiGVgPUEf2r)!E4^9WjQetR2rzgFstmV!Q^F| zWTq-aqVq){DAD>avzSzcNBZ+23#E3G4d@d)neNLUw2#TpOmBc zy?rbGHxy8IcZC9r{=sjBPK6p8`$vd;uM^%!=0S?N(i4kmDmVzyT&9dg`_@ntnH3w6 zks+wAYN`;*6+)AR>S;H8|M}4EV~K{wnq=+J#_}J0{HoemDBjo{U%4T&<-_6M_+0pN zUk-igP^kA*_^yABW@l@r&L!?YpZJ>>5)VBazyImP^kU+H3)lZq|NnikLNJRa*@s5m%SZFqjyX-)#n1oq^AA7sO}MwUA5B$ESd~Bye`}h2FGI8k7Axs1YxH8 zfdELp1vzjqN(RiwaYd+{=wWIDEPwq#p~(X3A_0zci?B;JohHnY=#bP@g7CPm0sy|` zu-FB|?${I}7RT%=(dbJFHVTg4xhQv;9!T&aSHUUeiKuqM9HrTf-GV%#e1Mo#^`Ogf zgwFufBg!7*cM8}M*e5i-{>Ow}*oS0H+yi(wsGbLH_HIJrA}~*;EP^wV@v;^oNFv{g zfa@nvAcKGde5;3>(MPnw>V1q1GO;nDeg?zrk&vK3rWS_#J>)UUXeIznG(J2)t%~m_ zTu%@>M&Ae?+9#&0(K`{V7xX}(>;cu{h#d36{@c%anLbvhOcOF$WppX2U?TE@m1YVz zBN_ou1S#VJRTpKjj~)5N7*7;CWsnEi*P)cQA1pjdW{PDK+B|r0f@ux|2b^aE=LP7- z$sytNnG-xd6kUqu%p^)E!-RPqA_sp7^#Djc!#1{T^UE&$I3qwqU}qrH5!G&l5~Fhx zU<)1){-bmnP*Dcy#8KlRj`u<>1cIrNk@pL86Ci92yc~kpc~I2;~n*(c@k3Oz8+mS2Lkwd$=>_D#y7iu#k3`MtApz12tMYxA0o0;Ta* zlUI*~DbNYI97PYoMVj*^<72PKR6m8Hcs2yNLr+t@vCRcLY(HsRrP-i*UjWHnnK zSF;1YD#QSp5Gy22Op$ZNap5u7k8J@mvpgUUBVaiT-e#DE4jFcowG&*JVGa%IbY_(w zo!T(#3Cb4C3Ima6eAj!ZF=>Xtd1Io|K&uJDMa*#_J}5O$U}*red~jNN;NA;K6;_GGny7g3&*4&joOk)S}TK(YG1>|7sEXCs*R z0fRC%vS9ZJ&42LX8@pW0YwUspUO||W-y$q?Utytubj~QJe6?t9V?2GRtGOBGUhA2BUt!It2r;kvb?wS9zyKvO8eoMuMX4J(B z*^E05y3OCbQkBi%jT|9J(QZgU(Z=GoY)bHP3J;6^ZY1%)R!gbJaZy zM#RSgFUj8rta3PUfY`Fi4-}$%aPd)wPAHPY;-H+aU z_n%&P?N1*0cIw`9*Jj;c9nW8r;jNVEVugZ$P}R$(O27?)K~#D1z#WAGga~O>SC(TQ zVB~{`z>+Cpw+MUM6U9rQu5#tR@c$yrW{qlJ1;2q)6e_@|-bS^r{?dVDU2QyBo8Ef; z^S^mjA{L3KT4Jj=M7I8Vi5ANX2-n{Amx&K4VEvn^*?Wb3 z5creLcqoL2ElzqLv&THkc`%oUeJKEEx7;jPB-4|%hRk}14L%;3^`Lb!qfYc|o!kMK zScoZ*E2+XyJ_}5EpR0g_`kKsjBTsn!Gux!Wkpv&`iJi;*kByFik-Xwuat>VIoK!J0 zxooNgjkv`I5=)m0xrdH_r;R%B5EC~zr;sNYr~ zD{auv(G93m2C`A-a?0%&kX0x%!&PUoAwy}rLC!#LxkW1d@NKrF4Hf!9cwaC*iwlEj z+fn5LiMqfy%O*>Bc6hq7Qlu3E+J4Z13D+Xf;AYcLrg;Z*g=({i$-0#%NP>`K!D%=A zz=i6+x+j{f+p>P`>diZDUsHbZ)64$r4b>~wtlzwL&Ds^qS2Z^+Z)!+2)Yqq)Yi?Q- zUB5lD`xD_$e=+>WebK!Wv4j6ud;IbE$){^S@r4&Z@+$*dL)Y~FZS6zP#-d) zKV1x+T&y}-tYX)v!D-sx2(qgMZ&Szu^I7lFux$n3G5nZrxq1*lXOicawCtIam&^(M zbhF+lwI3n2>!n+OH)SXmv-zT~JDy*5bm96@XZ?Lo{^_IN`NtPtd*f}IRKES%Tc^(d z_zQn?VO{@1qID*G`{T9kvke`ybpDmcjplR9;o`tQef!M$*Is{%_NHF^$(!S+U;6yy zQ)~Md$W7rLXY1H>w*@r4j#&cxI(^`pLVKxpaWcr>Ml2>so<|xxal-x$yUb6|BZ>MM zRKTvwx(*a)g}xIhmTRGMn$*z=8s<4dhASvG(P~0P)=e_pHu=1+P-9rjs_&bl)jTl^ z9MuFIb)w{`M-ovINcrIfL_ob01VQaocMni7fDbZ6NTp%VzfwI&0`0|nB%hV!L(r^p z98pVn$iUx6+kU}^-b4XZh=(GYr^_<}{~V z-(|c^ESCYCiScqjaTUQaGJT3AyW+@(B>^qFNdlMg+CN6lhu%KV(v(VMJy` z;mIK-0Cqb`t{t35E|&Zc)(UpXiVz`&UgH6P))A%A#R=h_{RBrgFl<`KFk-!VeD;t? zZ8ncKAXOv|m}F#(oFE9&O%OwnT8js4VDcc9!w+usu9|%=VP^#Y3Gf&1JRR*J=Y3H= zLj7l|haL+JoQd?Fjxdl%j{HoT?lX}dg8afv0OVZeM7VqAJjB+%&bpNFX3r zM@G$kl--mQ=8Gmfu2&CAQee3#(tHNL$qlY6xyE6)8k}Y0q zn4$=zOhBsx0u+*OJGwY8&?H-{nRCVj7vK_$U_qwN#SaM7Ro@5WUorRSQCy5 zV+RGHOR&$unqhg6IA648%L(+=&|F-d3C0fylM$A}iII7*0zuL=iHSUTr=YRo4{@KQ zH~-D87-MrLefXYy(sC^a=HvQ|wPFT74_}bd$=m>V76Vub&JplOG={!`-Z{LNnqlA( zbTDu*&oCvmjlLNI4TV6OYV&3RJ0mo8%FiZvv@lnYpv`l()zs;oH1y2*=aH0k-q%`k zq_hyI=w24R)!gozl1SB`v1C1OhHCRwq(4*bOKljNHH++oDbV26;lto;GksO$U!>ID>tbB0YFEbnO@k|}7nXnS~CXA1^iZ{V$s^5QdF(sc+ELB*mU zOwSx8N*C>QF^RsGx$T(W8@n9xNZ6ZiV3`BSNuVB)0q4xQS+GHbLMV|Ha3`4oz$5CUGJSs4%*F<1r%%Yn)D=E``I>`_Hsh_evcE1cj^cMvHeUAt8$zbsp_#Xg{mr@J8fjn~wj5s6X+COsb&sLwS0Sg-1 z;cS!borOU2uD6v+D&Kn8-fd|u5m45T;ga+z>xi&jynZ)zl;nbpZEi0SVx9-pwzKB# zV;r%LOd61xQ!E^h4^=*0;uR83;G{JlO~xNdbdH{@v4M znZgG~$tsx?aD)&+ld`Ii=EUl`%*Pa~@Uad9>VOF1g3%dQRB}d%k5I zQB?#4s89&Ws<*(xswT+bm*R*{A}~NPY!vYO&sY6Pe=HW=zG>6)^;`FCxc=fNm)+J9 zUA=b0=C!NW-gv{x=0;KNo0}6W*G4zo9Qp94!k_t4_)i9-t&{PCUr%Ia!zZ6ue%nXi z_~>Q-V_W^T2me>{!Shj5@-({|J5e+hMc9vF6+3e@M>Tu@mxr3L>X^sbvivYB$pfmJ zLuo2gB(6+*xSYX0?;F4s>pMch$AAxADN{(q!g<0CQe)eIx)jsi#%7PqRse#4;Xv23 z#+~vyGOlf530^d*6jy&^6wR=d7tD|36;UB?OC~E=#CWl&2*C6}#)IbA zPFc8RnDIS|7?349OZqIP%`D8cLDBkC3J5f@2eXvVf5K%5$Wk_0EYH!wv>+#ls3?;~ zz$@Gl#fpqf!{)z9S&=Q4O-UL+s(eD%D-h`hfseWH3M9)&M$3e(-~|D;^(urc%IPT? zgPr4)g+r~2#W-#SvR-Fd(3J8{Nr)MNQR%fwJ7wdAzs?G(^nmoFEu2s-@49IlEGE!YDuF7cN5*2&K9q#rL4+xs=$@_r zk&!r<cy7KxJ zYj57NWy|I*+qP`ozIFTdZQHkR-MVA*mhBriY+AEw_41Y#MzW-uYFk=rR<5nw@$v1e zTi*PyAGl*<=!0L$*5u}6IX8CS`NVw}V$2TgcFIY!8$m^F2wr|lP1OSmqPO*H#{ottcfpPakxx)5`pYHkQ z_rCGs&oA=d;*WoExBJR({ll|Wi3#)GT+Rxj2oF`*U~lL*fCc_pyAn? z*4YjFoV|bd{NLt(`05+9CH3Ri-??w*mB0R*XV(qp6MHF5Z5ttdsUE@O%{dOC3i^*r zZUN?#=*E^m*sf_fF;GIvsCS!uq`#*25>BGE@B&!bs3$gKlaH(Du?X9x3QDl#ynW6r zi%G+wb&1H+z&N0D>ULf$yH&@;HYY%caFN$Bj(>0k_=Uo{K^G86T)+u!p753tz5TSu zRuI_nxM(I~D&{uxL!IKlH-g0N&_6^Fe$^7g{+m^oK(Bh)af=- zz>@+>sNuLG@q>$r5tt+s1u{tyiU~KY8Cf)qC*+aaj4X`G5zI%;7%0>Z6hZo<=qoT) zX7H|d0FDLCEh?djAB53Hs=aoo2(}=_Am(%88x9w12MS;ma`i}lPR zq(0;>Z6KG6#pIwH?)zql8Fpc&5XL&sM0y{K_B*tR0P|64ha{*mc|6=NMb+ae_QNs1#SJ;y zF9B@}VtC6%#Tjjk+(k@AMiostbO@P|TW<134m)(qJlZf@K=%sq$XrZU<2AQBUFNYBU)7Ws)1=vXen^b#{A7d+lyo6ZZC zDFY%*o6ZrwsU+3rUB?NoG__lro|gu9LC{O7RWH-}B)!lK+kEx5BFilDws2x(Z?Xg& z4O*(_oQClQOGNT2waI)OKub0x6yKMEKef z*4xKq{vs&w7)>{oqHGgUf+tg~c3fFT3NN4to_pvBEASHq3R7^L_TFTx8!OukEv~FO z8dSd=GDT*9Ch%I6SE^(cdjwFIZtNWsBrbZ`ZUKrO#X5We1_VJ~j8S5a`Xz>1TymxQ z3IyC~H{nEO(l%hur3J7+c(Nx~o-JTS!Md5+2MC{l+lCkme6B`VQaCzXNIhu*0dZ00 zdd%`^S>XrKd(d&R;x{cQ2&wBS@K~8!qT+;_FsFU{pr$29dCKl9U~aU+&G5AUl4?)T9)3Cw|w1SSLIKEn0vm4@zl^K}u-*8`-FHd-1u0y6KEh|>m& zNzye}*H2;`juD7a$f1fcBN&;0T#bwv(&oY7eA5yW<2|Lv-csmr&n3w&MGW_pLVQA6Ud(L-qsTfh7vL_8@Q+nN# zX;!jXmUQVwZ^`ZRz_26@gr~QpH|MQ0jh(aA$7JowHdqC*e^2wpjBsmcIHw7fBay=2 zIe+{-RCxq3DGA6O^9Mc8IYQ|ruGSIPOazr_`Km>?A|+H5XBp-!XTV0}#kNSW;1~^F zM(0V~nPyaAIHx@X0NON0mhfP><<&u?=v>Q2b?)U!ng)!&l z7k~CP8GF6^-v4~=)z-(pcjJRkU7IOfam@Lkd6uz4_&5b&uN)=cuc1-WJRiYrQEcCYO#uu2gWf3nlSpcf(!-G zUZ(`{5Q#cjv?{q3+E8Y3MNFyEimNCxP>~U(SgNzoVNS#5a$=*;R%IpPa5!5CPZsGo zu^7GYeCYq^jmKkKHf{RIt-DjJ)_1KfzxX@L{`<8pzxIcl>aXj(Karb{P8A{#UWh#S zY~@I@y-c~U*hvta&eCfw+AS1K`Fzp)gmg}*ClUQV zye_u3w1VBp6@eYKf3dl0ku`ywCw!c~aYk@0^;FS!MoEj7om4J655)?f4ZJkN@-Q<6 zyIWYUPL9z$025ebSjY)rmPko&(CRA}({Vwj%S(0@IGpwrUY?69{(S?B4}t(i+YULj z(+%jVNO*=mY0NUpf#^ z#5b*9x9gT&JGX6HzGlN0R#sj7?6Rq?*CkWSH?Lp6e(l-~Yu9aDzhTqHOq#mhOITX{#N{x z|2_JJj>zcg=)F(Ij7;H>CSgJ+i`9%&R!y?6gmK+H$mkej_*E8UypEK2nh`>o9b*BX zO^xj?nOZnN4;Z7g@nx=JLvOsK|g5NcEdjGAJv4_E3f+RK~{z<2v zba~gC{O}@H7@P*%p&n4^Tb=l;6MfHz6*gM8hD*>!N<+&pqh~6q(}mX(*;FTGfHo}| z;G;3?TYQkdL0pOB@}k+Lkpn(*K=&IfRLQBx@P_}G&@KYHH3~ri+C_*xGzT8@dM)E zj~wcWeJM-GE74gq>_%zfmN}j@X?{vdL}pozGILdPpulVNZJ*?yS<3UjCsG(fC_y|^DtMFeKo*yRvhxQ{4=CL_mG zahp1bgs63~D?`&rrXs=~xvozTPq6$F`zb&(vnO11h(vDb7Z?%TK+qo1bxd+}n8R2E z;gNAeX(9TD68Dh{#oD1F*%x=63HK6IM-01xGoTH|JBjoMnx4=AnW@KYLE;!jdenxu zkFrO*m|EDy+`S%RAPUj1{|uOh1o`!ImM&!la#6SvGX?gZCPpE_d0ba7GX(XVjfI&E z$i+r`&Zsps0g+^M%!!2v*?U@CC~7kB9AUp4Jpi&^UTA=9QwC>wwQP0kN8i9cL-Z1` z^T7Bc6pR^g;=hmo3mgW*9b5iVgp)%62j}BHq8uFg0e2%_-hT0;2OC=^`Z1YK`G8{W z%ltS(uZ9V|LWB*f<$%Cs2saG)515)s92XaRV$m$H+P-4$_E;c5^P=IF&? zFHF%wxU~m7VA0%)-X%TV6a`U5BX9c-RKEBBYG{ zgieCgSv%zUWE6w$1Hy|$r&*>Xy2x85w$1vLTf2^geXih!O3YC!VD;SJ%GrVUwJTI7 z7x~a=1I&>@;he_O06B|bf|AVVXRy_(&>lt6&*JeCYrIimATA*=YP1Bj&}UtE374?d zhbh>?>HmyvgDKo}DJCpzeM!7gQ1j@&Lrl7~S~iTyv{lZ&|3E{dX>`=B$cV;LVYj6@ zFN1Ny3<9NHHpX<<3Rqr1=yk5LeykxdOeX)wj1!{mEut1+)Lqa#J{YL$+~F($pkNi2_zaWGHO-S&D#<>4WxHWE2n8bqk@rNBPx^ z65k)*eu=Z+?J(`#Nwwvh&Cxp7&@;;nK|%>xP%MZ`R=&cLoCY;f%$PH1hGcyojR;=& z%`y#eo{d5TE39P^VXp9X!Vv5G#g8G;35VDNLodw>x^5)X79;u10f|>a*}D`U-1~k& zp~v8xu|(QS0K}P6{0={7yq}VIrlobJrS+1EN+^Ug9&K>XOi9Vg3vN*gp`UUHVdBvY zb)5R;03?h(_iT+tUX=tqY4*6ubDqdje+3pOZ1%QEX1y&I6{pQ@bN<9uX4<{^l=^DG z(b#_0lpnh@4|Q9eCb8YjxbTrHpps7;9aFr1BLb4znF(YcW%nTXTtNgT)pFe1O_vRC zcn`FOoU9$&wQ=RBl?m$fQ>}cecwr1J7}_*2FQ{_miq|OX#WNAJl8SUXqrkGnA+@C)3?XNxe>MP92c>SICPCfnO&;Q#CE2hs~b1c8?*ur(hDK1naM)NV4 z7D*^rC?lkgpC|?xzPT#nUVbcp{kXI7t55&M>F@sAcYg}4@R?WN%sutvkAD5x&}8w- zjJs^i`4G7pBy;9~s`iYQKnHildQV7UBo-*V31Nk<;C+(30UkPk^>1CJvx*vL^5uaJO&nGQ^h!egWZV)frGc~yycc{+fpmne`QU; zsP=>FuWz{FCQ_}q%BaPv)#8@Jco_R09C{xtH(J<*Ao=&2_n6xC$WsP<7pC&^`5*i3L~(wzf7 zj*f*;%56jkp=8R$6x1yl#yve-UG7YlyE0YFY}_53;>np-hXOGpg=bDx$bA6Xado+4(4?M2V9q%s{J@CyU4Q@FfBV zJ=&nut}}j>^8MA0O{caAA;d5SC0&8$Sqw_pY*~4BQ?D7 z-8L*QZ4zE^P$2-Sg*!1|1G;6Ukcx~f5jeYiM!=Zt2TGL^`Po5Ydb%vF9s+8+ft^_H zbi7(%a2~~!2f6NW;?V@QlP5x6>$VEckf=aWyhD-R$lzf<$tuQYW1e=x9!-mbIYbxjmhZ8e?OL+jh}iVcA^+PS&U5= zW0NGpqM1Tuf?8FTbE{Z_ljTd=Df#xqaR9xtG8MRXV1Tl2F7mEBEg}U~9Cz^eumPdj zWx8;!k^aXQt{BZXOfP=o>(6GN_~A=Gd&@}wpS|_&BQO5!uO9vGy6JOON1ZDUFI+R4 zrw=JttOTQ*XUaNq+f0VHb?q}8uz0qSEvgzj814Efc!&O~p2Jbe4(H}%f{_OWwg z-}v6sFTL^Zdl%n%_dWMJKOg$XZ-4YivAJtD^ySCnd(Jk9mBW$k*~k5 zC5PAo!x*_shH;~wTN`1U7t29gZxfpsD{i>73K$VLcFfuY)~c(ijSj00HFnIg8o-vt zX>Y|JI+_=X9~)SSGBE0j*)JQ0Lp7qc3+WY(Vu%hlCqitngW=5Uc+!+iO1d(^jB7Rl z=2Zv>Ogu_gDkgL%AIcJ+VEkI;fd#0H7qZ(rLUjqoH%6@r9m%U!APX;VDx$_XsGX&a z)T<8W5&^hN!RNjNC*%*qm$GQQKhg!u3=l<+5x0zd*MD) zavoKiyksmqP(V{_VZUtBz=&7_n6QzuR0rc|vVA7IV=BrNOW2>Oy3;X5HFY@VXzX&x zYM8C7%zrMt!1y4gq$e6=E9`!B4ug#R;E54xIp12UAqQ2*&jj~vmi(~)j-k>R<%(~PVUerIl(pV96|$%vTpO0F=UP)aI_fJ@s3DX@#KR5J=G1#$v5 zg_eVo^>Ecal+rXWzHd%^;SCas$54^c`UujJn6~EV<3qlE>dDPgN3DY95C51 zfIrepYq(W$&X`1E_4gOkT;>+aNsNfSaFl)a#rKP`uxYJnOkzK@T)Glr5YS}d66ilS zp55(H8|E2mq%5}0F^2ZyBMY$uYQ{S_4^v#mQrYl%SgeMb+$U|W<;RTPxq$!Fh9N>c zqlds)l-iT*6%lkxd7OO!H07z|1nSWnsp0F?PC{b|3KToRA@1`b=?wvY2L+!MTW0cC z$nY+wq;45OR2Gfj!r(6;2_{I+#rd@oq{vty`(B*4FRP);!{ZdE^ZB6gTOSi5MjC#} z#rD!k1giC3DTj)(C0kd9Z9WvQ6xvIvRkE5G&osBqC?YA%TVk`!Sb&EKdUkw%#87_% zePDLd%o3|-r7UTJ))_G+ZuPYs#)%zT+MXHnEd^;V%fn$s`0hMry$YV=FsTLANd>Jy z7{H>nXc`!z;90Wv=E;#(O4yn@iYd~;j7BrpQPGD*dC>49K`fKTeT!i+Hl&G6Chjqs zOlg81r;6V z!srAv7%Mv8Be4`+JZh=bRRmQxCJPw4 z#cxNDP|Yr$jxw@YhyBPxh8=brw!e+=D>h87b$_g$Sgv_3UShDF2drN$}|bE>r>lLX%hP(D(7Ld zOeJ*j610nQnG%f9Hs9?HcgzUx_R9_o0m12f$0cdJi^3N4-L+}P|!db zdVBba0)IDn70vn^)~wYDoVSuFf-n5^t;usg{LQaFTYF;h zs?q$i(R}%_JbU6}`7^FVcw7L}YmIn8c%lg}d?mbih1i?W2Q$tGM)TLDoz#8je*cjd zzV_U!KVli3u=kk)Rm?0e~a{ z^@scyk>25yX9bxRC51XuQG!ViS%Y;mdB>V8RA(tjNcB{~==HHoAv)-P+2m{Fox3xhmjIq^S>McnAWu^>k8}qcT0u;6 zS?SWMV~)a$0#bo$gD=XA`lb$%IZf@V<9B%Cs6_q%pa4Foz-qxFve*DDQ|E5FOW*-; zWn#tYF?|H+0i}-KL;=i6LBxP-TIZcPG5al3P8=H{O3(TegLB7RO~ASZ@OZJgDhMtw zR?5|#0pLO&Z_*ff4NPY(o0N0m74lbw$Hz{r4^b9#-Es|%BO*BGL9;X&j4ar`b(;|-TQ_cOS-JKzE2=Mk>WZiT%hk>G$rU%QSh?cHl`B@P zzGn*q3ddn_Agde$e_p0@qM>dpQ{LHf7zo~ld)}6m`+ijcI zuBrQjw&cA}Chvc;=43H`YB6^2xv2R+RZL8{MwEo6iXpHFa|LppFb*UCqQ`vnQ8F#w zZ;5I2qQ`{u6~5b1@SZM8=QFfjG47fv;l@)>{Lv%dKKaCte!-3fUw-4Aub%(Om%j0x znyE#C4#ymm#-yx9DD83i0%eOn=f)4@&48(Go7uR}`Sp7iYuS}jV~2M%spdlitcxOq z)VI&ZTF*vqKikkYyZdnA?*Hq>Z(exqEsmV^?90D6^e->|UiPUQy65Xy#cE)OfQ{`X z=m61R1Leywf=9KyFl}ooe!VT&O%&xIRO~&+;*+Y|Je) z^Qcis?#Y|~@T(hMKTpy=g$QZnb z4l2j2Lhg2wJ#KWU5FIYW2i(|TA<^R|`rLV+%pw2)AOJ~3K~$Pv`k&}^&Bf@xLQRib z)9uFl>0)iCLkYUwWS3*E$A$~x{RKXElHHUbHbB{vJ(MgyK)2}VakzZCRnzV8b+=(} z-jri%my#)^o9sqL90h7pKj-RoOe0N`Od3F{Rnbi? zG_5jirlt;3_Sz1ou03yZnTwR5+pXuHCZkDdZZ))W$z6Q6m-5zikt)6ve;k#0E$mKWwo#{}?oDE!B$qOkV5fz2^Vkg+_Qm=1#WP@^8)jOqax z4)aMr07||-D2bz8T1B@wF0@T$183Y~HX_pr89J2qm}ZCiAB*j$z_lROfhY&ejwxf{ z2F{U{>xhulY5^m%^(zI&<-`Ql4N2al<&+(O5v~}?#-wmAc0de!#rfdyeC)u2B?BTs za$upZR|t$K#3ZmT_QZRj;AI3(CDmOZd})b)K*Z2sObsTniG4zQlq~(y;OHTCpGUUO z)a(em`SZIeWWWm+%JM9P7_yCRf?>h9vMGgj$InRh63$5W&P2c=w_nt^3ANabsaYd}u>3g2~Et8-PhtgVFd#UfR61dQhHsWa4h^2dd zI{aQ=%Ds}fMJ%pC|FaXcm<*!t=duyPcHI0qI+LC>649P_avAsg6WFN(W8**4?Y1xFoG zNECom$9(F%PuE~r2BXb5#4BG~2uDCJ=wXiuw^86r>px^dOb&yJgi6ma$EQ-cfj0ga zKt=$L;$$r?q)V|TW-CZCpX8+=z)-9`0^>p>0DMUtkWuY(0W0}o6$cci0zWkfEMl`f z?j1qfFjZUZ6&NU(TGV2|sq1y?dx+sKenR>jlF3Z#CbM70^~`WKi?oJrCIZtQ!vcYZ zy+jPGBlaKJ8Z$CT2iYC-imtu~Mjo;_^!yce^WE-QHdvk$QKtAH6A7sttU^s=$eUr` zd_&h9@WEz2{E*+$D|gH^-+_y>&Av1xS@z;C;5nfR3Vyd|Mn)DVP(W~2ZtXz=4~6t{ zamG>{PzR}$IcwV1VvE9czi^O&EJnn3qoO2@&*)gleJYN;J&abL{Lz6g^yjuYzvO{< zvAYpX)>~_P$vV8rr=pka<%7k;*Bd&{Hnq(dnJfe9o;*mq!EtZNZOOCB+f}Dk)+mx( z6R@N3;~0bD^LVo1ySwZ`)qw`0vy-aXn>hN3-qrkB~Gs&54a^Lw)|zYhD~g_qwrUU+5K|9QbAxN0w^vTbw;UQc_UQ&o0$DWs{OVtTXt^0`PQ9QwcquT-M8Jidfka_ z*IxY8vQMpuY}~%2qIx5BXLWD zQ-w+nu2rsxv&X8_w~u+sid>m%)W#f&e+7I3S{OuOsDL%$^5fL%1%l>+BjSqJ2d6OE zbKJGevvM^G76W|#08pLvOC~Um^3R0DWCTHHIGIt{oX5YD9Z@A4GDfRRA0_+fN8<*3G(GDjR5^W&C{M{A^slcqbLUp}PFEoJ9R5 zcr`P@$o&+J&v4FVn_?QeiJTiZBl*<1&}Y6_6N?zxv3u8UBS*Gv*}8e-rcLYCHLqCx ziRGcUKXK&`K7Q4jhFGe(WqC{U@*7%k+<3zc=2)?O`As*jShH%?&D(eICd00e+`4<^ zx((Sa*IoSdvd`TV-njji&Fj|I{9$|j1Ly1SeWLDUvG(L*^3-DD-o?22KfPF!E5@hT zz}Yl!iU`Rd5>~Tawy=MB>p)=WoHdQ%-RZOpcACoH@rrTh@-(ADSt4l%?UuEyIBDQ18;-X9TRLW=cRc>p z`B#4O`a7%o=j&Q$n!0?XJbQxC1@)$xBP;t!^tj^A;%nK^w@LMiaqXbFZ(Xnh?ecfBzTh|q@{`u7LyZa~u|u%=hFkIC3* z6L)%h3-&tx%Yvzud}Cvsjj`qt{ditfE&x*!K-0^(ma~BEvg)Jx@bP?QrWiQpUU#>1 z`Q7>QyUxasolQbKO6)7+OH%CLJ*fdUAOd504PcGJ;prf7(Z!{ICm~=e6r%sbF256 zlSOmB^5CL5=;GnUHN!=dch&yIn-0?XO@|g&+-Z(;EAON<}I1 z;@Y92`MGAOXlkTF_7_cJb7dvfUPQ8+4w}NLJtl!k$@Nl`%q?o+{>8O}g>?f|j%k}o zW9nUg58X9cs3nKFuHoX^0Xj?x<~okm`xaLZ7g4OqYicp)R69S~x`6_o%iLWvNOw&d zdYr@bI4h;_bLb^=Yt??VZP7dhb(d)&_m4?Nh3}_YsP=vCjr;S<`$!9_-ZNj(<;Ivu z7$nSW1Wx2Ivx%8(uv}3gzkY1C@GbZCr2+)2rcmt_QjwRn#9!uU>zdYqtUKM zBi$YikV2h+z&OC59$N5f{_5e7E zc!bmOeRJT=S=O7L%!v`}pukn21QHsxPdIBJ0}?NfQ8voLrmQt1^9nF=@CtiAOawP` zFlEmR{Xyilg96wQ3OO(@rgMFBP<1_6CK+ zm}J5o(Etw!Oa^)*P*I$~gc*sWhZn+klZh_V^!RZQcw~g3OlCDk@1!(R2-6;u>~pZz zv+`rZ-a+~gvSF=~A_+~iCnF4K?q~iWH=st$kU6f%2%&i9*--*`g~F5^K@5R8@AWZU z%b=lh3`gX_P2o|{ArA`ph^}DWehxj%+{1aGs{l`lqtrWx=S?tcl6a16=FBuDe%Jw_ zloF$`Z_ZQ{JG8(QRKgmX+I;gxDeh)xnGzbHryI`@wx~vYRv-4Cq}pc8(bQI2tnZLz ztBpjI!vr9FCv!(jg@X^0Aky`g0bTI$eV-@%Xv@`8v&#o z78vq**`szEeemiDXV>39lYd2gKz-xIpMB+9FK+wWr?1brzc!Y?dfd6516@|KrY2|+ z43%KKoC7nd5H$af=5IRn#4p}??^xmGEAO7KhWz9hq0CT2r-=z87=_iQsL8^ICJR@L zIm?V_$`yX&pDuLIe!uwRH^|Z9JMaD7g`Xs+7q8Efsjsz%73r4|R|DV=a!bVW4>ejy z2!hvI8Tzm@C1CljFmwF8Q^Xmpo+gG(Bv*);W4aK(_lcVEeE6fEuS-OC-b_xwShe4> zdBaAd+BYm;_4_wfUHr@y&PT3ZURTr7VpMw^M!~;fIjiTK#fr77SMS_$Gw{I= z@4oG()oZ?f%Lgxha@i+tN^IS+Ys1>L@joA~fAB*6sdM$L93@XK)|^~SoLY>ZT#TMv zj7=A#Qw3J-iy>}m`@pzYsU^#c!@p;06t#`{60ra|z0UdqExTs#n zJlkFY)XH+o8ui1g83hp4h8+8i6%$I$wXtL5yL2tCk75NC<-&#iJErCHm*J07)^4i*i{Zxmo^CvKuJJ^%;nbozYVSK&b1$KU$rFn;Q^lG|u{Dlm zS&kSjOY_{pUStMn)++b^$BikjbTCFm!L1Oiv9aQpkRKrQz=cBX*be(_Txtv5vTZ`e|s_f zmB*XA=Nc~|*)5^niE(ZRSs$BWS=%~Oef#6lJ&$h~%Kz!T=S@+g|Ngzozy8(@9Hh8W zdAubBwwJO6{pI^rwM)o`jitK*ekic91EGh#-Ak)|`S&p2=kS4sp6u{#gqmf99h+FG z^P&V)a1oj`x{rHH87$cYAdnt&uR7wE-#OEGWbU&kzBT;FlV5Y5efs>fFTecqkAC{| zSKof`)pstw`nDc#UG#pwqyPO^{E6SWE}j7hije4tvY{X-?jhpZ?=j zJ#4O$1f1okxa%wbwYTY(KXFOMH!rE{&60+__O@32_WQT$wO`iSx3zYgptP>{ZN{r_ zUy}08k|ve3?b335?RlRrdh6m(e)@}7UjE?=&%N-i;`6yjUTC}Dz2&Y#ZTs2l_C8wG zb0$t?KnCv^B?LSNLLZ&9vO`1II192#t_-Q_#Lz4g>lSJm z(t?34jELD|9b;9)g2W~Es|D*mLXnt6CTeO@!6`9S9wxJ2zDUX>(*xO}R^TwPX$4M4 zxKcl3i$h`vT;C%$&Pj$WVf+hj8aIsESw|}IEZNzS~+ObiRwNesmOuBLZzi|HA?rorZRj;V`gBCVNIX0a&H=WUDyKfl3&Q zJRtndT1F5c>M9Gg}EMREQnk3s>sl8s+VoF?+#WOu__L9x`PjIyCnK8Wtq^U|t zYL&!cu~d2o)_=ZPlI;PHIie-%@|8g}!5OiKEgmhbVQji|A%qZ|X;}6-*tr*a(S&Eo zWsPjaE_fVa0UXTZSP{V*m$vuVAQ!iUbC-7Q8XEv57H0Cupg0}zSp=sYkG02$XO+)n z7%(E+Pvyb6fEhCB)k;?kdXOAwh7SG%jxi`16p-U?fVGe zupAAhaH=Kv7*a~fo=+t4oG9-?p6*D#uAd!obUCSZ^10E_OCCZvoEMqx!$26wU7Q5w zEb|0ODzeG2z;_=?aYn%e`{6QGAi@S-&C+DS zL5=G<&UnimK3s24DV%37mY)0JJpWyPGNdVC8kQn}epFG~VDH)nEJ5vC{W#zy$fI}o z2J+t7rP&o9^b`H`t}oXTw*%geg_iH%uJF4=u+&1|u7+P)+Lb)sZ_2kCZ&vZBM1->q zo$UB~Xx@B>7_C8tzb$Q0o#zPL52ytwwrY+THF;Kl)Ik%muJQtxu+3)pZo>3~z{gop zf;FMIvGPF@0uJaX5Vr{ea;X*3xZk~i3wS7?glI9_XCB|LjxOB(%@>WL^yP29c;kal zUYBwIx3T;OdHhyVOk1JaC|l6+%qk~5eiCt}a)o8b7ADXA&=mF3>u;}q;K^&vv*bM3 zj#WQVRfL}ft4ID>IZ>!GKgZoGC!AkN=L5OInung8dh$o6v<(kGb&U~*J{!{VtS3Oq zEfvX{wJPb%!UE-!9sJ9pf?ed{)(+HY8ARQojzH>~{Zja3&vedYO&UcIh9l2p~cKACJV zhpP6CwRLqhH5=BgGbI?+{G zX{y*by;#Q_3!sGYQ^ojnF_9zHo?r*E5oGqyBr%o(o*OzqAL_MfLb2M6>oM}|gX`fZ zoGCR=HgfP`YY)2~%S|RV@l}Oe)(;>+WSAHW2Bl~n-bFy$R<1lSDvG%If@=Ydsqm0Q z^`&TSlYv(#P6gJHwfYw%)JrKg$QYrOyz%Bocx+#x@qnQN;}+(UnNd)takcU1uB%MD zW`So{S^o7^i4kF&Z`W40V4HMTDd38v^AcX6E$3>XK6nE#ZF@L&hEn2Ui4sm}Zlrt?+D^6bk1kLqX}Tr>}9%t3|F zB%S9MPv=~VE2R-IZw5c!ha=7rD6dn-x2WVvYGbjUqR5+ zGUZlII-#5sBC}$OL>`+e#7-0vrxqiV=hpuDA8%L{-nMlog%;U)3keY>53XOkZtbd7 zEz4KjoQnP6V^_Xo#6hYu9ItJzt4q~4v@|x|(3D!9YPLsnePc^1b&HWiBt&+vUcLI? zZlz6zZOiN8UpdnJ&@(M3i%s_xo9-<(GCtUFs#tgLV)DMlLQxJ@|9zG-7a$#<2N%?zAEi} zXe^()@0=0hcbxg|Wp~e-&Z(Sqt4H%Z0`vSCo9~=DmdY_NJ?EhRt43+NjFEmif@Tu! zo_S>PC#TQ9+V_ty{PY*^Y}xN5+s>vsW|}(8(w`1G=>m{9Bi;b{VpF@MY2u@~Yqo`e z!n2`0k3aGK*S|XVO6c~-nJhp$|H|% z#2Calq7F~PQJ_On-m`^eM;BuEoc)8Z&iup7xtCvl<=u;=fA73`@r|EeeEqfmpB-=T z;Xhw;yk2_#%aXnEez%rX@yjw^n&tmnuK!H>|G1I=$y)w{ir%-hx8AsT@jd^?@{2d# z{MN$L9sdt|?;RdTk?xIh&tX}da8TCFNSYaiQFOw_Heh4pL|nqINxN)L=bT*@FxXbk zw&a`?XGU_4gk4zJU04=yz|zE?86_u-vyF4MWJzDWp}Jd!^Zj-2cbaEZJ!&g}-ryJCJ#|(}<^l!IkceE83x6 zft=;L#9veK4pY&NxT2k=BKQe1l7pr9GqGzgcd3^NuiW_&6PqkBy7l=xByNmi7N;rik$E(63{?L6%~P} z!o|dHLzjgM06t3qTAa^J1PLQ{co2Nx+r+%jMbnwBF5x6mtYho2Vnfw;@Kf_x4_A=Cp@ zU;_LKrHOu{2SG5@i^wB2Kz(q`zlgYXK654Kq@qNbqI~dmxDYSWsZ4SBL|7m^0^GQO zZsaGW9)mu?0xeiFJPX!>{qI3uC%iI|;+8-m^nyhD#WD6Xz878jPz8<(Q6Mz~UeH4R z37RC^DS7}gP-wY$anQGmxc5ZuxNt<4U!LyMR^Zky<^lDF4|3u2!K}bM{FKNW)9BkF zZ_)DS$PbGAra%cfuJjymw$-O(xB?fUkQ}WRwVbpZc|d>hI+o*t2IPKAQCiA%>rW9E zaHYvw#BnM;np#)3D?QtVKjASJm<|Vh8iQ+EB?eb4wBi*XL14_`l~};S0%Dy*pk7Rk zUjlrPS>!<_2NpKC^d-g+lqNKJaUe1nRx1dH!PgeHJZre*2v#{1V(6ppffa!QuW-q@ z=D`}_Z#c2-a21Mc%OI^tAzQVm$PerGM%HAgK*)@1D)HSnuJY4@DyB}M218S;Y2Br( zRICJWT@4wq4b&kb!Lk9?bz%+1PbZ2ZeF4f5?Zg_RQ45H`^#&APr|_cP!4<_;`{ANp zvKOf|dKOk9Se1bHQo}1lZk~f0rqC*ds{niiSDttazAuGd1hm#pZ-<(}SfS_;TcJNS zU$GS`fDaZaNo4`?SDYew5GJ*BIOs_!a)anDqshl$8cZqyj1LYGA5>_ZO zFnS08BL5`jEQ(erL`-DD7J=?v3TwIKQf;kBf`yPuk#OGvR-_s8Tp8Ie_)7`Lxj0D2 zgLJ$|LtV==53Z|MIH#I$F2iKW>2sZGNtgMxdvmy=bX`$Ysair>cHh$+o{4IHp556z z$J6^9xM)(#$*#YpuS}5BUp+|8m4)Q0Ke=ufdbxe?l0uB6hbrs(WXtO#UFm?z#PvL1 zNy%%r;YIPTVffW3GQy8ysPcnCgo-wfVVrSwLRk)wek4>X&}yAL6Jx9Cub&i8xQ-({ zlE=J4vq>aSlTp@-7ZYLg@scMWhT_r1Nh zv%BY&&yR%8TwlBI++B74s5PwklbRX9(4JJ}^=N~g^>I-(G+=G;j#}R%f8F2H)BD(m z2l_dbXuZbDDk{ST(C4i01MI*^0{AWr5*iS}z(Rl&CXYdS$|IUYIc2G-;C1#{t?Cs{JHYwI4M)HFLz{>BgQ}{l0#Bz~toE zcynS}a#DJV?KahZczTA=_lIXf@AjGi03ZNKL_t(#0)KGowCU4l-ZyIa-p2-9dA8s9 zks0Rv#`Jd%rfzAmZEa24)S9}1*gq{P>syl7wIr@@v2JKFuWw0m1T76glThl{hm7k0 zg@^%AA{gxiRXR+*Xc#uw6)KJaR~(GnU@!3-p|2G|MtbB&qC@-vjLOT%JI!#3t z^ny#jJR@;;2eDxhQq{450R)dAy$Xg&f1G?fl4j*22UG;TuN5slI3u1B(pf19 zL_|D82$@l!60j*o?r|`7otT*|Vt#@ISp(c-^QQLr=YIFI7Y05$^MUEp9+@$1+O&tK zPI+kZ#0MU{fBb~|#*77gFe5X=8n^$6fxXZ7e<>?E-fGLT*;FB-PvVL#DQV(^56+x6 zea7?|o}AJh643 zDCeh*9SPa4GgrFS>^>FqMpJr`SI_B_5ME!qV>Muijsuup;1T6~)OMly%gdH*m#vVm zr}VX1R?DRA#M_8_s*!Js>OmmSt+D(HB!TfZgXIEUvH9QNDiXC&w^yl^qqAs+)4V?I zgkuP(=ZTSP3s@@<5kOKKc%NUJ{y~u^#u4mi4@K2`W`E#s|MGBe5Bg{n{~bSY`9tT0 zs#h*lynH_Y$+NFLc;+`*U;iroyYyYUEuHzpxbxYMoG*Xz{PN#l+_LaSyQ}wBC*g&s zj~!d~*6!gY9}O(r8CSA9zG#=RXeZeMgOxE^{+8~FK_o6#oE9U~uo$qsnEAVruLt>o ztXHzrRI-yuf|N|6oC2^ZCKF~O7Db^#)I3n^g54^wkjke9s8L6Z#k=D3p*`wPqGXkv zh_E1(%=|=ZKsm-sDc?pVLl_P1er8DvJ*k%@SH@cnT zlOSjXR6EE}p}!8TB8DKEIm1(snJwAd0u>G)3gI>~2L_M}fj=)3B7bkIE#jq8TtXZY@0shdrUE0~8f4qnEHvG(efy z;mFj}2$EJLDgu*nF-B0Ck4A_KxGq8u0luWhFTpS}^V!xLvw5kvi33QljEtQT6_pKM zteGt*5ErtHQBgu*v28x;;p7spwzr&5roO5zE$@wKycivOjy_Caz?+5M$T)8jN9RYzk8V(qE9oL@T1aZ8NAt;;F(d|v)?PO>ac%ad%xY=ZoBWFi$H zK>8sX7t0Jf@{t4`+|o#yu|@T|aiyO;=g?wC`#9DzvyTXOv@)Sp4Y3%2hK33fpkp;m zFeF+8Gx(6{V_ZS7nk*A*k)#P+{enf3!qh@Mjb>MaWF3rPWu0_+?qE9%^4+c)CKD-- zT@8{$C5LQtJuBx_0({(S5%sHq_QS%!!B4F!5$tuOKA>6~ptUdH9e{OGdur$kQv#)z zLm3>5mWXw%S#ZbN;aDS&@b#df3Q$yLXi9Z}I#z}SAPZ3tc6 zX*~Vin!&F+i^(lO4toILU$gs8c6Cy-he33GkmJ#=P%YYIM`1$5VZGeYGnb{n)Efr% zi3Xr|N|@Vc-L7AygQk%0vdPVRG4V+VAoYwHuQ0!mX(hxCjm!x2k1vKXX+R>sVHL0f zNgG8c;0lNA!ySSMN%})y{H(S|G_sYb>U;?>3&~~X@sIY-42?CYaf;Ywlc=5MMTBY% z%`u1=z9g$eq0GiKhW@?QU(cAK- zx)GU!&LGAq4#qcz5*oo{m}O(T+3C0b;Q9afdB15>A0m@JO|>64{=P9|#*7}Fnw~j2 z(Rl5L{Y82rJ=T(#nq^Di^(w4HRHF;BO;wI4R6W6ySH-@aDdF#MtF>X!BgkH`J4H&Vc z0a%}^86}gPh{U%hsLEj#gyCHl98QrA8Dfhf#M%QYZw}CpdWz`6@>~YC)j$KSpBwmH zgfns?5iNU=uq^s6(5EB`%R-9Z3Wn8e`NqSL{oSr8cxrbH6?HmotEH@TgG`5$!EX()waNT}xo>je5-thM1R#cCW2kEK zZ7I8yn-r#dFWkwPhf5&G1n3hKnS! zz-d#bsX}Djn6X(I=~lDZ6c_i;>0l}R`;k#bOUi$T4`!vNr6whg9X%r@a1XC(_lN*yw}FGJIzoxqre;Udeeq4P9Hvd!v@Yo zc*epOE-eJ}IPql8(-6IHn*Hi)ybUz`7As@4?b6-iRj-p5l~lcS$kPdqTzR|dawQjA ztCKHcktej-2+05^$LchN!-(=NXLN*<0u~1cuJAv)!R6ln6(NJSE_{9I?d9k4pFI6j z>#1jko_cEVsb_|rdN$(Jtf>^4y#V=(F6Z6G6GJdG{IgQ8urWe_|BvSCiJ z{4L#0dH8P$2R1QEvtMx_N~Xe%Xjv?d)%>OcX(L?72gYK^Y}8$Z%*DGgpicQ30krOH{ zL*Ipng@MUxrC&4&NG?=<1!Oia;L1=T^d1pxWw@OBtGjZ2H8qrxw6LY7Bg%$&j{&{pHbdl=PFx1%lxrf_>HS z6RnXV!9fr#(%4_23^E1{t0rU+fxjv|#??M8KatzT?M$e_tuT#>q`wq%Y7#!zw@nQ}ZD{iMcXlSrCLkB}KuMA`k^MF~vGc2ZUh0~6Q?N!kPUsgBJfuNXg+usEk-aAqaMSOvVf35aPN?So(e=c#2`*7 z#3?`D#~+9?rWpdHh+ho|j7u&@#fomO<`ytrH8(~Vfjrd7@>lok?V0tK$ZDRs(hPZS zpL_Mb2%(m5x+%OZn>xkx*>I{-2`m<5Q`1zjkgc24j3@SyHu69f1#T&1r1a2$LcMs= z-{sbd(z_aznKgwx$D_%{98DOiA~aRqN0E-h!2clW{5c+ojZPj0YIx$1uF7V-)hr?+ z92`D4j5J*GJV;{|opIrWLO=xfEh5Ri7*3*?YDWKA+p1&yQQbTMiX6c!G7>s#0R-qv zQNWU9n}?GtL#fUSefIA9z@HCY@SnMMkCQbB6lkoigF?ZAF~AdFdIJEw zo@Q%9sDFK6(z`(Kc;Vwich&l%?ZMV_*SGDza95oYC1C_AO_jF<8jxl^jMzm8K~UF; z5(YbhDVy4Eb@#l!``Dc|{%BM@qieY5!zDP&icf%7U`yQV;bLzC2Gww}0_e}rhsozif z)1py-IXq%>%kV92nOoX3H??JKY|UKXDpdQFEp5r0ASG>TO_U)5^2HjML?Zdq}x*XpNp8`3#14R}KECCDIW zlM_a)+?tZlMeIVvJ;HT}76F1ZcOp123*(sKlHA6?h=cpexIz&k9k*tAWu?Lenl7KO z?apckeOG0~{t?3M4wwd#@d#GK_5fMhGSG`gwPJpTH4^?EP)$CWMh8jSCQD=62(3Ws zccj7*_1CWbMPVUCiGnXOAYU7lskQ-6C4*N5+MV2HXjMHIC?QC0Pn6sMf$3MRz_es; zC$}>UMH|({wKxeP#CKW81=2+lWRy^}A@xk|qejR=BAjw4)bVvTUza6>vz0hMEa8a} zvQMa0%#3T4{p{4$E9t2~w-l){5VIy23n9jX;C66BOZ>W^c~iS(jbi%2%ZYJ`FTP;= z*=j2tuZ;Ri;fwLCVk@&0=(`Hf@VKRj}4>xixG!?(6)ZEMTg+?utq zHGLBjl7;x#420xl3|f-7z9n&eC=s_BEY46uW6;zHz7>QJiD?MLkmKiC?tJ(vkZWxR zzY7|BH0mN$ex-wllc6>+V3qIXe;n%R>7D%6-g~P3(e?Za)Zt1}IJyRQm0@1-&I1hq zNRexT{cHVC|9QXoy8L*}UdudJdZ8z?$b01c&5cc8$G*{&E@`@m>$z=skctQ5R5ghr z8)Ta#>t+;sqF&##`jcZ9u6BtMY&ouU$>yaS)tol<-h4jQ=)SpBFJIK>v61!GHyFb7 zE737*dABF^USedVN|OiK5)}!>its0`E<6yDD7nZJUBd$ymdiNTXg<=b5gL(L>lM;zKmCH?^h@!lf1Ch+Uy6hDqUpQz zUHbL}BSefA95CW^R+Jb;r(d>Qs-1oF*nZ#=c6WZ}-#@l;$2|o*k0y6J}9!ochoPfv40?o)Jj9)~67ZUoA+h_q`&Wo@bTHsbP z_(i6~bTI*7O02X3&3-G`mN^00h447yV40r_!N+Gr50E= zv<>-&q8!8u>5aokz?6Xf2{BB|c^J?L0Sp4ifOt*tboed+*cD|i@tc?V%}f2(MSd%m z48TFQW>}}1oM@K6(u_|3g-S#FU+n{wt9>kN->_bR{MiHr@@GX4y0Iy-Wa{x3O4a;x zZqxtW89DB(98V_x&Cr8)eb7T4t4PKBa@~5|FmE7Qr7&R2 zSH$1sl0afvAf-TPZZIWn1#Uq7^4!V!?&Ja)b)?7x(|M6EvCxxL=oZN;Bx<225p#;w zS0a*{N$Mt&X9kqF1S2M+$coQ(Sqj}LaO|aJTT%&IBBvCv)I!+~@>#SYsgwns$Yzz4 z!`qQN9n6=eAMDx2#8k>4_kX&3IcUjk?gH^R@r`D27ZCAe zy?v%|j)wZFm8NI+5gO^)ecGc62mQV3Bu&#TdT*P*+Oa$!;ghbL?2<6p$3X$RT+qw6 zY~A9V#AB|uK35OYa{H|FX>CtDAfe$W zNHvcXabcvq+gY;VG2A%Fq3k+&<)b5)`q|lsSQr`Ga*d)rC0Ky*{G~J){?kIa; zs3Txl7rNHfUD$l$&ec8>QlvFqzEKmoq@uE{cIc1|gi1)}W3k*tRB2=#?SG&n1<|WQ zBVHQ>y}mIRFVfmjqCJqfz9Z4@iktbv#CY3pe{K2AFDwsDd<0c{zy~Sr==l4_rKY5q zc3zi1j2d)pV+QT25bBKzE#$tNS%qF3NZSyDHmU2vney%{0H=0>7aB`>qIA%l_8gX(Ai zXiPqSBnAeGt%pFpu>=lcK$5uiVp@hQ8{4f;pYgGmvQm?T?0ESl+kgBf{-H@T86QLv z@&gm@wDJ^lqcv4Xmb9eA ziKz+CXBi5|hSlDG&yoH=1bh>>8ie#D*E!O-n8>u82qJ{d8Eq3- zF7nWmw>@-Mt@8WN4)^r-j9b?>RD25^0spu23~^{RpXx3{-)zs z-aB;h&cbHn>rKYFu7Q8p{k!%1J8$(&TM{zQbumOJ;}^3JLFr>y0%}Coqc6P&@WI(l zi$6Mcv#V!hkuND%-d?~|!IT1x;-@DGYCORL_6A6W>$rZ>#8|E{0(t%*# zZ3Ox?72AUw4sJ>1aI2PL979dj&~A%)lqgFb6y)5S{1a^ zx;_r>@9lw@FV{ZZa`wN*o_c!7=~>Z?3I2~mPT!^f_okOjFp6FgXksk`H0Pxvb`|!1b0GI69N#6(PX$( z#tgxj#YEpTko_<6{$k3R14!IJ4q8HaN{EffwdR`m@HvRqz6+45R>7n*3QK~VHGs634@j)B1sni`ZH>`P)q-R+_H(z0Sa<++5w8tQ zQ!#i5+2>*;6E=M%J6Q;jf?lo&UgX%m<%mUsJuK5ue?AS&sH+~&s@O|4$oETGnRQDYiOVI zkHg7}{Klm+H1Hygc$i$|75X;1_UC3Gv`20nf%->Lp^KSV`R+(`7MWBA z#3KGou>G&*najq3N+4CY*fjXb(0#BpuZ5}}-Ul)K@7x0B_F2xVzY_N_ z9rf^pH=1K!Z8H3+DfYD{^Bc_+(o{7ZChRXmlKb@7gftCC>*l$xdkcL}uWT7pVre7lwoy&wDql7>s!V8`JR>jD7g@2KZA9F!K?gH zRZ5B@l(eQL+zu<8m^!)g!PRWEBRHs58DbBlZfwom&?d?sP^(07paX27J!r58Be{{WE!!a1FmWSFvMt0KTNFD$=tyKGZRc1Y;kAL08`>V+)&aIxl4k+B zP!$7zhSh+|#X5qo>^S=9*1ch?lqjdRa~~y@Lq<96n#ne6P|{zJIE}3f45?A>Ug--b z|L*ndWJ-lE2Uq+1t?-F1GDt{;1Ek=xlTer_9XJ>%of1Y#1|qF?NU0H>U|kTj6)5TR z{I&wX2a`&JiDf|wC`>VhQ;WPA#lF-MAPlCULqM3n%aqhge`<->R_+ri4G+?acqayk z-WVClQ0$4tOfJX(03ZNKL_t(VC$r>aAO($)lS{onsA_$%LP;#(l^aqEfq_Vj#U!W# zqy(@L3n4NxTPv5ci&)q;DEEr_l_4pFx0CpE)ITw^vU1!bay^-d36hC%8qPTD?B=95 zU1N)U_m}vG=XnxlH;Flw*>}c?Jc+ZLZ$sO~y-%-bnN+UW=D761i#2|(N*m0Tv-mdX zKUNf-W&Q1N4*N{FvjF|InS91Fnr58$l5k~3KOXoR3&6(W?Ow_N{ne3zDm3m(a zxM&Cdpp=s-1zy!h0Sxcek~3IOnl(Dpm)^Gs5GQbuxi1u|8##kf`r%#AlY*cSQnA^> zk>YyS(D-M+TjU6fNN|#P#g?|wYQS_mYp-Dx%u^DnRGmGF0Kz~$zu_1%4Gauv2(+HR z{`SEO->>sWN^r2R6{i}=G@u$*lgtFIzcJXaPMPuE{#!l0Gu}IJ#|qDYRsOqH`|nuo z+jH{jU%$F|SCucu4sk!oG@EE^Do_x?3?U5u?g)@?IEoQ5%n2-(OE+&V@}BzlDqpO0 zH5R4A4Y(YK`p9UDO7RBlDsBMevam)Cg@|yl*ym{V;*B74(6APKaG4!Ja|47ENZh#B zTC>|Y`N@&#Ne@2o=mT0=UtV-?hvlrj}m(UfID^~R~;I%?^Wqqj@gU)LY&NJjH;8QLJ2CXe7$hB1ok1K8@2UXu>?D+ zs3VqY;h{;>X_En+m_BT?r5Ft{ z=JfE$v7IPbqp0xkU)v0iZ64LSoc74lz?1~5%)?mVb@5r!PF)sHfEc1si zRmQL0({k_##2xACy1c#wjEH{}iOAr0Ji+hM|9y#2!Y2owFP+tO?vxmB*Up{(>8ei# z=j@1=P#qz60Ou9%Vt<0gY(Y$DU0eZxh5+*+9%lqdk?o7)h!t2OeGwLGV4t`GUIp`! z^jD(6f6|~>G1taM z(Zrars>E_;v2om9R%PKXfZotm7Qta;CQN`J+1nD$5GjES`4PfJ-u*b$E9>3K2{FZ&OqTkSmV*ACJcVLeE_*WIWgiMbji$J#X>WaPQ6bDLA=xqrK6 zpMA%PZLY6tKRxj?M@PyWSMnRKjGWs!O6e+ur!zt%d&3NXGn#*9?-1So8^_*+*PDjt zdJ^Akp1Ck6uH?US$naWIW}Zhw2Inza!3?P!t(J_r&=0Z5ohLokh&HoCuOn+X<4VQ2 z(vNgPM#m@+umQ8hqM0p0io{jX?4(f9ifRZBS?Qm-eV_a5YuCHFuXS}Fx_I-K|2Q-~d+SNmh^yg4214Bk_%On+;i=hRgy z=Sxw}M_&!DR-y@obOyz`D8KpmsVkj_FW)?VrStgZ&cl~(Ug_)>+H(KZek1evAhH}= z{1N);2o9}R#A@)xzkM0!3`MV23@eq`Rf@4DIB>adW9!*tS2{%t!|MT{BvS*5nktTs zRm#v6{)gTMJHhu4To}6AAFH#eBt^>r{lln-wy@{rJueQ$UBg!Tr)=9hd&g0s#D-xU zRdt6LfQ%K;hp&Eq)Y1@&TCFfdt9uFpNr|WxEdw+Y#e&b-HP@e( z<4?^~Yz0b6p#p2OVvi}u<=l1Z!nIppoxL&lokLN1uGCU*dZ8yO*Y(=_hmK$DJbM1- ziHn^lFLWNg*m>#ttxbWmaV6{@5MadlUNJqNwYMe1{K9?ZLXWk;lfJ+&)|OSD9*=ml zDJx&ww1xoXg)GL&pRBt1AtZxnQW9Djlb8ZGhy@VESWT*NafZ%P*!HIDec#zLm%H|# zy8c_|KJ#nMS=kV?J5wwLXE#5$x~=)(CDC$`zP{Ay{OshYJa6ioE;V>@+FbWb)$Jl% z>Rgu^yx2C!C9-w*^j3dzJob-!Mj!<^W1c(X8+^tLJ!m+u4@%g_TKSs?*}A=fzFb1} zxX+$}9!b=Lj5&Sox@}{lFRNO0r79KjZ`%dY?cB4W4?6cJW{2kpf4b@H3sTywW6t4%%=&cwsh$7FGymFbaUyNfEaMB@P?@> zxl_}NX?H{#ATo4FZ6*{3lUwDmd64_~huNOE+)L-E}PYi{tsbju-4X{`$^i?;N}!B*~M1IdIQPus9}ym~5(a zr+~ypRo_=3q$oS!gV7Dav)6B}_Mg6UwJ#c@#8RgL^dtS15q63G^%z?Sli;S_Fg8M( zg{tP#geAo|fNd8Gv`660#5V>L*0xw0g4Rv@EY&-V_fH#@k@m>UnGa2TrAczzndiiajohP( z>|<%G0z2BT5q2KTOdXRXJ}oSSda*%8dsO>z4ZTrWK};=239lLjtJsmW2-ffi)%Tr> ziD_&{yN32iq)5WqYLb*d9Q>H4F{lFfDzL`1!8%D@uaVQ7Mv*(;~ z4ATfT?uQOxR0DeO`>;#9X35VaELH`Z^)WywFfnb6sEjiS*zrZVN1;)}7{6CH4`nmd zp%Wox9KTK#w5E+pr|yy28;cB}EW^4*sQgM9#F^w5A*8x0bP-(|SZb?0RT{p;$+|2R5o<3mFZJTc&wkQy)a>wU3b?+XKZ zpC8cs+<>0v;BVKg0ZS(gTQWZE<>66d9{oY;lJ_UP_vQGv_m11%K5k3syi(l8;It7gNoA! z(S?m-um|s|^1ZR^m{??_uWKDz$1(EwT}vJ!1`y<^GQ38tWpf)tUEuC&{~vZ8J$$)y z--R1Tu5=!~aP!!CNQcke5bL@3m2>2OP+fj$5rK0#&MyFe6!2FK7?kJEDDcF*wx{yL zBcg3%OMS`nTF1}Rga{d|pKp${QKYX#NZK($BE=5wxu{BmWPw?$850|$>h>Z?3u0ZUhJ7lDWU5laR|V3moAw_%3O2=;&R*bt2P zArA05gaiSYuZRMeU?>uz^%bL$FUAZpmh7T(xDbddvNItM@j=4)Fjk4k0NDTyl=X_7 zMY||C36zGlLjy#yYE%;}fl){-7Fvs}Mw1T5Oe%5{Zk1fZF^dVwB6?j?3HYbS3~)-8 zXh;BL9FQXmqq#BQQwx2P5XV$O%K{&IulEvXkV-Z$V5p0ul*|WTR}gu#Or@}q@+tFU zMFb#11Ljlo51Cz;c${_Q8z{eR0K}S&bd@KRIe!u9y~qgTF)sOykQu$pEc- zM21+=p84LS`AoC~2VwI$k_nSE7ou(QVqd%{P*j98K#Z*nKgCu0;xWoF5&Fm^(6Y!E zH{WY4^O%K_zsN5d3`{`e60+rW1wSuPAVn{gdkp9>q&66Jm`W)nYY}xL_InCC&E*k=K`YmYslw{y>;@oDDFptMJ z&y_OIr7FX4OSW5FNyo1bq+m+_t87}Be*a#&uy90 zJacg{cog`;htJ&be|6>L#hc=I&;CnO<||2WG-s)S3-BN_$0MFAah@yjO_vQ%iD?Lb zb3Nuant#y%_~36F86Qk~v-y!lfDab^`77gVd$J@a2=eE6Qs=po=D5Toq|Jq*RNfQ# zYs+;5;DzL}Vb#j8RsQHzN{rARg%(=R@jVSyig6`^h6o@=t@IDAQo^bb7L?ir22#^@ z#A!(r?FbC6RmN;=yLjVP%bDvx{l}qM?;m^*b>L?{JUE~Rg0lheSL2`b*4~>43(o%h z=%02TRnA=R>gk#K*4{zY{ul>v>xQjX#%yT2c=MLX^RvGndhWf0?;U`0e)!&j0o71W ztUd6(Dqnr*%#B+;rF%}6H=itboh)rSxzuxN)?W_}hFHT4D}d)gT2SGl%V-bYxxyz4Xs0$`2Y{C6$`E4pk=;oZRfte)@7;SEN}wQB1x(Ih!S3VF zzI$K@25gN*W8|1xHct*;;g76QV)VAsbxh^s@Ft>%9l^VndMlbwiZ+xrpBS{*W2}+1 zDI{gYuTol1UlYHM-`YMD8$c0J#EXlq353-sgKHG=*MLrGB3HBf6ha9j#G6`00b%jk zF)~|3jgp)TuwYUtkOxzW0=B|Hab_nNcS@pxr3*!?K*JLkmpL4fhny8 zo~$yTxVLu42~mzM&uz{DTjSIs_H&J~N>BphK7j@omO;Qh3Q>Y08n}(*C6LoRw;nFc zkj@9Qw|@Dz0PM2N*>W)M)u!Ruo~&$l((L9Z7Kg6h>~1@L?X?XDer4ag>~BYVdU`)l z&Lz(V3L%*ui)<~&uI}D@(efHl3jtW1>rS2H8lC4A1=j!TxbcsBFq_NvCft=FF*`;K z=kdB!y`l#SiNlzo`bk7Vp^s;xmX@yD8_>rWy6sn0?4;?O9_lwUH16|lw8UJNX?m6- zt5>nbDM4X9MM=jo%j|oeqF1ACX!Q9&5YwH#0p`*VHL$X4-g-j z@HN5*BUK)*)HJ~}fL+WWMb%G9{}NXsWLOfFy$I-7BS-Qibu3bnd|4eP1KtLnQ>1-I zNatYTGH?Wj3=W22v`?XSHFli)_j)C|fw|J;7%oai8H$o9A?d|bGBqheJhC(R;a3+A zUc7nu;>{zMI}gK_MyDvV;FW>ytr-%F1`oX0sdTx!7j6cR`AmT8&$)mUeENmki zkU!WE6tc4_@b$k}`C^DU=-?>E%$TlM;fo-Jeh!zO+0^*RY6uHGmW1J?fIzV0- z8=3bbRD1K*^hG4@@7OnLc9%1R9+>W#$u)jd}f*ut^V2nK_+R`>9hN&df+R z84VVx+9z8RlPu;{6NcuEiHI{>A5M*%eQ)Hp$%DUoazOVi)a!MsefM*4aN*ej%O?z5 zJZ{*tnX#jvem-;gKPJ6>=>Dx8_r29|-?sL#TiQo&>lm@UW8~KM;ak9ySjJ}bB(|X? zqp>A*ZHsMfOUfE%5n8bQao|HN1U|&p1Y_}Y6QTMIYtS5~sWePNb+}g@L490!8lr@q z(ZFC*s)OTSu}eB#IfMd7+=gpOGM4<9GUTmiwle$Psp4JyEynUGw5;m%0}(^Cktg!0 zgw&uGB9-$*sMAIW*FspCQ^hD$@{?x8%Rr%YVyJ>qw0;mWoR&kZ=KpuRX5DFePM=d+ zzoqM(y{MbD;^G|)qEV=gHJUkXqzO~4@yN_a9-1`c=RYw$@#yer z4}(|%d~oWti4Qy|#79D090`%ML~B$+(&MQy#bd+E#)X}JN(c)fIQp~Hm^I+mEQSRc zAAEkmJLTR22Zoo< z1UVob*m{Vsm-sN;S?8a<8}Px@O@I$Z>9F7kj+PamHxlItr~&Da43Az0$kB!~AS9Z} z0Rfn7mA>Hx-pnFzW}$avu{VtveEDodAcF;~a2`%kJui>!f^*%Og`Sw%O%;DT(%szy z_+Yjxjo7^Tj0vWxD4q%qYK6d>K*i>0m5@g64*O=-bBUSpk;hGy4fE)$Q(2JO?R}-m zZBA0?BAHR^H$R1hieKu(>ZXu8A`Z#S6nehC@x`~4$Y0_K>-CX z-bwL-WyoN}=_DqYkiRp&kby-&4@&^Ti^pV;R}krgTL3r2LS(dJ#t^AkidIbJi(@Dx zUqv1<_;R6a00NZMR7*L)f|g1OUn|`LqERs7S%^ASQutjj2B{Q!CZU87@RyJ~afR?T z49E-NZix^_fTIcv#-rJ+iv1;^H$!5sSr%{%UvNZN<{^lXpg`maa$~^VRk;V804`+2 zknG(6fP+ZEULivuLNRg)(U_NkLb?&W5Mp#-GF3K}d78hv>^XkLxPUBY(d*wbe|&{E zveccjLYd}hn^f1Du*4s+z=P1CFMg?XcUbOCnD1o+Vf3yjrFVLD%e1=I zq)MNm&}~_W?)l1@b0di$tne7c+n3zNIDFWwl>y5NC9=#tc6D&n%0N8ZC!(XnN@;?<#Aht? zOsNWq36lNc;jp~sc;&y zySfjZx$!`$KV>dpk))Pq6js%pnd>pnZvKS>`Gafr zCd_Wi%JYbJ51fbGNj5MqtfDZS~2@1 zasmNVi>Kerl(o zE;w+7@8^H}Qv7tit7pQNjv?rrk+6pF8fEZmf7+VX$y+H)3wn!jFz#>FkcnbA`{|6F#RY>a{)5bDE5~uEYgi@LE^`k|b+^ul?k; zJzri*Ti_eHSb-I3f!kW>&X^DQ;ObA0-~GlH53CG6S>1Z?V#S#6N)|WD?HLMZ51~s6 zBpQg;&AbCt3P2%%nE9TJc{*h9w&QGfR<{E{=V3)2d zO1u7z0Od?4)JdT}1ZUlKK2=xa-&j#Ibg17ik?w%Bt^IFt>MG=_OTAaTCY2dOd}Ma4 zX_r(%yY!n@3t%6_F_)f9Nz(*StcJ;knzD}`(+HbjVvQu_M9auX3`aU3{IPL`pSVGh z(zAbrj0sMDoWeQU2`XsVNYw*?B#V(`1JxTnM+J^v>Rhw;+?{nwbfc`dPQ@ZLA`vI2 zq*4ahDpAkyErrO{7Hr4**Qv*uJhUJY3Kk@Vc_tfwG=)UBX*Oor; z?t%N?-aCF<`?zf#z$JXEW7M|x5h87F8@>tLhzZp`V{J?7hL#kt5o}3X4-laRd=Uab zA)dw5h!)dMU=c<(vKg-$syLj&@5=bsO#hNFk&5|H^b0XQW5^#aVFHq2re=$1Kw2mT zCYGBR{l9ZPX1sdq58t#z_Aj41-F~7^10h%lzV_KvW)8!T$ygQ2wRmnJSC&`U)sjUghvI zN!^0IBeGtJaEMPYq_75Z8!6@(Ux~EyUw%D}kR)nGCJzC)3*1a9C?En6S{M+5T@<>4 znkn9V001BWNkld(`r9;1*IrD##%&6h}|j>0-i>LK8?5IgfxXy8@+6)|X*o+!{Y6o^ZK1c*dZh@zU5*mPBXJ`l*8-u1b zfdnEC5chjhj4v%+v=aKQ?{j6OUS^OrAM&#*FFcr$D6P zS($MrW1~2S zUl@4(hj%X=Gcfu2-{1ev-Ur_582>gwghV848@;7<_|~?pt*x0GThceSq;3*&B$VD5 zN^K0KG=!2GgO;^H3;J0wtql@|FuEZSC6ZlHJusphS{RntKl}>FRjnK|V#OS|Yqj@J zpMgzQ)~42BRlaCDzjPE63XM{6013w!y1Vwnd!ZLZ*t9sUSHeX_wLmaQ$#Es;VgiQ0 zD^>gjSPc#!R7e3i5{8T5KR8z?IgpKVrpnkO>0(j%`ksYRO-F12t-`MP7wgF2p8mnaQyaoglWabt2I zjGDfwa7(r;x!4_rQ&`L+!%9Dme4$zlUgBQ&c>v6cZ(cuN_%sBO_=)*{9z6Ok{r@5T zIN{WfBESBHjp7>bJA5py=wC5;JL8IW7>gkpN_WPV?=+O{go84S4N|((P_n}$G8FAF z6^cCMf^dha052B9#khhU0M!-lh|m2T1B~o|ge8c(;_`MFilH2lIWGTmks0zAQB;#1 zkVlkRu!AC&$Xa$_yb-8EB)T34HNq_rlcab@eBS4H1v6lSMT)Nk;LNA;VG-mfEfM*HfWE~qh6HZxp-e)QJQ(-_iUN2J zU@|X;3PF@m952MX7C}4>bA_nX4Vf1ZEfEh8lVW5F$b%G%!$ZM3MC}-;1WKlIEDQM9 zjI=>fFNOqh6Lk(6un2B6lzWy}Ed3Cmfw^Z3P^-PgOiJ8$(IzIgL@ z?|m6n>4{tB6E`MR`oglCDnC7Tt*bj_x!<(FW1H`dD02PjuZPZFy*2xUk%4cH|1h zRO(IycZXiXB5(3C|HbRKmhL>h>WdR#82izYp+(J!0O@kii?+c_sq`7j+*7InAI$ye zNO*}WWeEVRvGYBm-c8M?4X^J>FAxhy_;y(5H5L5%(ASqcuZq6C*?s0pr#Q9FZH6xe zru1Bw`OT)EINSG~zSh&*d+lcTxvO3OwPD|?e;yNeWfXdB*{-Zyx9PP#bG98kd!;M; zt%I${uU_jEo!xur%#9aoTCH=Mw!6QE@;ke4-0E?Dej-!o;qzSSIq-=|!@sFHe2|{) z%E)!aztQx8a`r}N_v8x2_}U(#^N9S>uk8M>b^9(}yH)t^q4?Q*fa;hH<%lX?+qnPO z`I};F-014QeErt)kB+9zbEVE}&VT~l3A3Ai;b0zt=rYgl)s|>C7?>u&;>EzWRHGbnNfYrnz>g#J=Yue6yf2lXV zE|5?g_}&UnUFhpeH*XngVUZ(hAG+GN%6GahaC+D(f5J+?v04dR;oEuivg^dv$YsDw zO<3js{!&jv%b71P-W;&n7uyg>UDsl*4~8xGMb-GD>%cw{7s;9`AyRBzp#N&$5+68u zxYg5h_QtJSJ-u)4IQqRBpRpl0aG7^o`?+J6J7@p%@Ch-f#H{G)Z8~vf!lrgnta$2J zu`sLhiw0lr>aK8|xO<`72$4M%@o3|=w0&{%>h)Vaq8mkbA7FFYj6EiABiLd-|H_@bGFsXsx2aRsKXr@c8wv>d#Ka zM;f+&1mldaqfQV8)uA+@Z9uqK>P;U87* zv*a|3Wv7_e5TYdCmA25g@AP$XqnP!VuHQO&x%2)-inYLPEpew;06w^;>6G(}lcK?5 zM%?V`so!-X6$73mL!c5j#UZiK*8%9{u!<8+N)`fnA=odc7Q%5#i6>*OD;dmyD_^YNed_eZPV4L@k#l6O$MmN?zeYChGs|17 zZ#0RCHag$4|CH_#E_Oat=1-Z^Y@6FWD%T^fuitg5amPt9r^F+4cJ(-f5SZsm1HYfT zCS{{Q$mRpOqW_7Gc_z-20!nw|x1=2Yl@_#k;7*g$hU(JF%;|GuT8{2Oe~N#3kqieI za7oRR+jkKzm*Sv4bn|Ncr8h8YCUQ)Gc6+&BdVUbAZVoU(4sAi>Aw1gBW!tsGiMe!3LCIf~j zLAXHVk0Du6uMFczrxGh@2n-WSu3b5J;pUco=kBWYMG}pMlx5P&NkMk9DlL>^(zxNF z?`6cXkDB`ypP!cd_b)}RdIK&c8TgA|V?Tp|Mm8XyV*umk!%6u+V#tEP}FiVAuV&Jd0`3E>07uyATE$V?}=K`_)QpxUnq z#x(>j_Mo{jB!o$FV@smlZGC-(?Wb>KKK}gZ2PRJ#ci)upW2Zhi<*{j5Pd#p(GWk)x zYM-8#7H={oSuBYbYjnKz@wC{Fr{A-1eAqvy4eouZ|E*cT`1$Wu`~DqI49p)BcJRr8 zy+XBrap09_zn?vNu=VG2ANb3G``_u9u)TfU_I9D#6Z>b>*0vE_TC=vaW^8Ir2OnZB zDVti7H?}0N4<)V-S=NQjYeNZZLU9lyA!rcVy(0kBpT+>gJ5Cr66nF#)nD9=KK&Qya zw~Tw}(I<>f*rhq#9 zXmar^Qsb^fB63At zB$oXwTX1cr$`h&Pqma>o0IFmotRPh{Kit##$PAFFY;0`HyVPhmgftAsuL;IG1M&3%urhXnIHnjqvU{a)uUFVPbUn>9hNPpN!Mm${ zzy9K@a+CE71c_WW!d}{$_dOKM>MLM+i7@;4o_{l+6LGW=+1H6x-E6TaVEm0(`r2AUpWJpkmv--Jp)_skU@ zE1sIsXCFX;MdSq^t`L%lj6%0K1&ELglf|!<6bc#1dca9!;$Ffc1n(ej3D#JSHC9ka z6NdH@DmiBY;&kQ;h8?K^sGUMnlMC?}o0o{qut(ER2Hz?eIKt!vq5$DeF$``OzE+D>2V?&*2d(Hc|g zwl4IVmimko0U4VF@aSS+bea3v4WRx0`rU)W=DQOseWHr!Lf2nf&z-o`2@kc$=ttwHPg{r^a)rIt`D4vDtB42x6Kd_1OgvZ%wqF=Z^lCZueTrQ>FNEj{rrm? z_fD#B0f&Ig{897WPpoMZnt$8LtH1m6!Dlvh6n=c<^3Cqn)7R3M0o-X`>@_a*W-RrK zi#NKuy~nOB{QUT|hE`*l+p>tw$wlL#^UDC@jGgb9Xm1hi`O|w}-tp${up-x>yrw4` z0U#^*;IQ#^K-NTxpS@?{-(cfy{THX6t7(0)wr%4V@I;F~Jd%KU((~N_4%Wksdrw|_ zb<+WnN8EDk+>H|#Zl1f^HL}oSo7bF`2l!yl+rZd5f3<7n$47s(s^!1d?K^z-#zbYLj#hqr%sjg(13!_h;Ui7~AFNzg@|(>Q%fW#_-M@~aE;+~(o=o>BRpbOZ=8bKIh-h3|rM;SYjmf9dRaxvsr&C*Y^sT&EM~G>HPm zgJ}35*ca~^o`d+{B8d5Hr!cV@OL;k7-BKfn&zed_tD)|aZ=Htxm_`QV872`EuH zuB6wSrq;Gz?CL&xx$~7z4nMJN-<$tBvhU)JV^=zF-0B&31x?5ATsVl9R4b#`wr*@YE2{g$r-xts`$4f999H8u)CUHx@U9ENRNa5^#veXA zJngN$fBO8$rLOL_v)4tLk-)`MqF4A596(ZA;Xd{Kh3@!j_LUYmeND9Cg%1x-eS2?1 z3l#F7AAdDysdzvN360%V%c?|tm2&q&_fpr%>s{U3I?nAqbKSVi2RMa2Frdo!gZK7} zZr{>zuB*Fe!j|@-z?X!cjIUK(C$8S=>7Czv;)y@+`{~~fxlUd^eCeiG$^6I1UkzC1 zNvI9UI#KOfqE`udfdfF{4|J2Q5^&nlLD zg>Hy_T;h3YZO4&wH(O3#{pqHCzgoX9vs8#vcVdBi!~)n75M}EA@!g8f*@gfb50HBt@|Dv=z85#qOvgS4^HO?#-sSIZcLKS4_Suc}_D! zgc2wer|K0k{w=eco?G3j;e%pQV;>l}!QZ%oN;0t%tjHAIhz4+lyEm1Gm37~&Xg1eKRn3Za3hqAJL@AW8%- z(NWDInhvvG7&i&TRQqEginkwWQ!tS^S~u#*$Pkhw8wcxRmUMC%2Wqw5P zwhH5P;SyS+M^GL*rvfCN<$ zJP9JPDL`-)K>-Ae2-P0+dXX9dM@tZTy)$TS2qrZKQyM}k>sxI?wcoVY*66jZ{AX&u zGws(oSx>$=V!{KH9yCpUaQw7MlV(WO{(%YiTNC2q63o_w`11S0E=zM{S5&T8q=QEXoG#FN^W0mJ*TgNpR0+o+a;dNO;b-@Tnhwg!4Sr9m{rc)@{X=loy+EtZ2$#pDRYH3Z|<2niA}uLY1^ECEy2 zhEmpr($@sj*99{+w`OhKJL2sxN4@ja$Sp0IjlPjTemQ-{<0F3Z%DvMb8~@;>i4!JF zoix#!VjF3W{p-}By)X6ad7&SACmjgR2xk3n!9h;w&Gh*}y)OjI~T5OkUrTv@Vpm zHYnss!kSQmGZ5zt2svVKDzSEmSjUpOk&)j>o{#}K^8@Ori*^S3Rr{X*=u5GNoBrp0 z15oVJkdAclb_9K{ND{$k5AtAKJRHE$xGgUhyJFRDknnDD37X;-xYhl=R7t4-sIGub zeF;P4O+Sh5(RFd0neUF8y{C4^i8Gfwg%Xm2Yzu8Mk;8ysT3(;|EJKfTwp5AqtB#7l zNghno)B2b^NRTD-6Vp`~QO8MC*@Bw5(H*2`>&n+}HR!~XSV9TyJ*tkk&Z3;k(7IsO z>gHodkHhBt+sjWqHRSY@b zR3vj_USEhKB(4 zTi~I9!j=kYP)uw-B(h28fD}kqD+h2Tn|;5k6rFeSPb}}Xm;#EukXZF_mRur&)zdEzj;Vu z^TN-Mt@EB{;v~X$af=ZmTn6|f*dv#DOcfrXE7$Bi@x6Ip0Pu=MiHCI^zjEZl&7{S? zVZ|=d2?O(*o?Q38*n98jD2}Xsn0@U9Oa^6Xlye|QgKZ4PIAG(rwsG9GH`(vn#smrF z98ds7l(P_s9E<~CY(OAQ=$@XW83`c}l2ArD2fn&h)jcEZ-S>CC^L;=6^K$y=G~HFz z)zw{h>OTG4`v9Jme6x8`q|7-^ZWF_J#dGME>$-+r66EuP#G+^j`Ud7EMsf5u+xtfg*f#WVJunRCyic+zRxvU8Fy~tH-2W~Y?XIxi zRWbLA@`4*J;}et5L~$;@5>(91PpWF{zu!HwGMKU4UFoz>V*K^ZchUhwXzLwa!Q+FX z-M}A22M_$vu<*+puD%kpofWSp1B}(uJ+dO05$&mP+$Xl!S#ep@)iXH06#b0)O6Z!^ zmtucFV#0VI$rbo{agFUB;QARN2izluEVucl z!g^-~+9|k>2L$t>76-^2_lPg59*j;-t}3pYetw(5wsZ**dGVsiQ6s)ZSx zRi1KY3LA(s^gVq{H|?NwP73?jeyL>&*FHFQ{#MugBo-}5EO_1x_$_JRy%WctE@sjd zqAcaiB=(D+;9lN6e|ugcE6P?ZJjAUj!6dJ%4NoL8mhjnC`9uo46JuB#01gAX99oB? z%^YAs&gPI}W-|N5Wk6BiKUE9uWJ2RJQrSZaz?^k-GBOB4fqHA5rJQy^w)+|Y;_ij&>4}W#Ar1(7d5T$SY!Qa`UO01WS~6plshE~5 z`y1~6hv(|%Cb71uoM{?on*-A@N!~jBAY+4;7O9*gJ`yKsnHEtw`ygYIDnE0p%kN6V z!UL?8u-@ibglLT?%2v*Z2YhgR^5K@_)ngNrpI*2#2i;eW@`+UW<@%mW^}YYX1F{Yc z$RW-;2Vg$Ga(E&i|5!IINoJJF%{j;{O6SV&_ra_38 zvmvr&ku3gfFlZAlLncTRA~o?7|6JRH7Mz|TGM7N9Ybcmp98)ZF!4z`WaK=7R`dpg& z$Lg+&)!mo0U6-o5E>?B@a;N9d`8AH;h?hXBK?6{p2MPfn+%0}L6{fcLk=r&qfj@|j zFF{lCo63F1V0LSo27b|YcMpu!G!JeFle_GdpjuwI9h+||RSo^+)jbwpRXFScQXx9- z<|A1d#GqTKzCVZx+ad$9M{M`d(2(aSFbAhPy`Fwks~EOm zaCS-yLL>MTS!R3wCbnp}VD_J~c|PKM2TXD1AN{Z2loo`mkW0Jyi`@ex0`ds_y_FzR;ll=PE=v#TIhyZHv{81uDm4l}n-84Q<70cjVodsJ)JA7oWJb^vvDm zXYZ{jRWHevE!pvr=eD<&@AzWXmY1G<`k9UE*FU%M85>8Z4Nexnytn{d`}cU){=NSb zzn*vPcg)3k332rg=Jvih9$b zFnOMcMujo-F984E5^fxQ$EILq*|ZGd7}EJdCTx~aQGr%~#4@2FSa001BWNkl%HK46ny0@J z(3B_HlF3HUH2m3t6Yxh;p1^Hp_yJ(}pQDKbGT`f_5X(*R-8=yd(g!aH7D}Kul1t#P zAA%7yfGII>Yo4hvpcxwA%KCSR-mm^KXY+(L0Z+`QbxnC(Pe4Ki3`=0Tp%>1~02wsV zXSL)xgIJ;1fFH*PBWF_(+5|(W<~TP2p&uh7VppJYKo+D>?NkiNpj(mJqd*Oy+hIua zx9qTH#bM2=64jbxx|K(3{^a}i#{sXdD82P$Va1x*vbB2;cx?aZRZolh*XNA9_J{H< zPyFzL(d8|(wVS7nz47?N9drJ}M+UM@yo=1utnQbmRc?Chhph`Qy|6&}++z>loql7> zjK%N$bMu+HO(%5EoY0~5?)nqe>&tX&Nucpj?XqLq#m7MTXi1UAvqXdJh-(RCD?oR_ zvyoLc1rR-Lg|iTv3kCwcZ+ZIJ!bh(wXgUBqGm({_o6f!PZ7o`z?D+A{6X~GPXE>zP z=u_gZ(lni$pDtgRBA=BmpOwj>L0*)rK)+9OvJ`Uwc|eB0v$%QWRm@e|#>t(0#7;OO z*gZ(<>@T(Vm$(v2U7*y(Uux$kaiMVme3UL99UyF;A&jNB_yp5&t91}n(enZM9m-F{ ztYDsXNJ5nQUl-t2Amf(8$k8WD3i{>l?l$Z+lBXj`a*#%s;4FqTplLv8`Pzv_hQuvP|eH1q& zRK!T&50bOCm~UtuM)5}oOvd;iY^ViXfP@%R(qaf@7mC8KTjL;Tye}Y_JdPQ1-8l@t zPk``9C;)}Pu#1opx*uWZK{mix7W9ZQfM`d8STP;XDT3mOh8qbYCK1GtvJf-=C}t-F z7h|l@F$&m%c!v@a?QtyB58HPxg#7vbi^f`VuBxfb5x^h3-#dZ^#0K3?ynInC6LTHlhQo^X zS4(ux3G(@&63b}D4ss?^q9uX5IU%Njy zQtFxjmqRyh9wDo2?62+^vVm}KxqSrV5f8zJxzbit%c)7?qiH@zblf6_ z*_@*ppPX#y8Id+XKF8{|!NIYK12^uY!9iAiagfY|EYY)OBvc>+WOVU-r%;e<5!xgQf zT@_1%Wp;a~@&^L}AAB)^#|OWuSRN#^-C604oANzg_gbR*{j{q04{5h2s=_ci@ynuH zcDpLkMOOqd<71Phy3QF%j7^qOl*ZX*C}tjD@^G@hrNx>>=^!+`Firl`-JYht(M>0- zUp#wj>)Bh|PS-qLs{5(Fw`*w38m&b$1wWII#leee!uogYILa$mXYW)8eE-*8$IZ@(7R|E%En$>Ie}J z88`Su661dpUj9{wtBmlH4E*qHPsGdY4k?T? zINKZ;`6RYw`T?0;n!-AnGfw9&-s_#1oQx7TuP@b^rgG?%c?!9-KCu*yWXx1p<|`jh zX5(Z)M>RRbeOK2lX&G3M#u+Ec*B^lg{rb7ue_RG!a{XZ~8ZoPE1zHpSbg%bb?}!On z1m=R0vTdevRwDD+C3wDnyLfkYGK)r1l%bfD$eya{>KmPK%2LixWS=R+Z_dTLronR0 zFojzX=N`xbN*K(#Vgewtdw|LXOE>HX-fR4oCL=$wTQut$#aQ}EzBy4>*FN-iM%AL- z6%M|VgC-w|NEy&R9R<{h^tn0N81VS|VKJ6SRU5xV&=9M=a2T0ua z%RM7mGw;fnOYopq$8qp!1Ot?yp$zTd{RqGZx1^}Q(>(ZI*Y!R6t9kHkb=S|j?vD#$ zLM`FJGOwxp!6m*JAH@7YG!ZC&&{yi}BiRtfUe$Itb&VXCwB=uKxvJ}aFfbl@w%&O! zj^D@axcf+|ng%LrdtAKfCD181AIb6{h%YX<-Xi+Ca(RGk8Cp+%T@iNyZsEyr7B%i= zi2hHFDK^9nJ^4=vUcOWKfV6-C^hS1Q+_67@)|dZ?9$bAEh{?QM#q1o#oJ{VC z6m}8hQcy0;;O3{v=Vo*B4#^i`bA@7rjLARp>QDqLomW1^qIh*1alzI%N$4D zB8OL01Sz&4impWMRG@M$QahKZ-HX)jht(dX8p!c?6oPq|AJwcZRj)l(y`r@0y?w8| z8}jkWqqR@wm9L39verAnW$WATc^Y@TIlKRrX;-(*LazOl=Vsk{Zrb=;fBZjj?I*#t zf1>@xX*W0h5xMr4x6apY`LBmNW}JI=rsrq-HeINH=Ctl<0ui37UVBov=D2q45$&oY z!2Vf=d+1q;Tzj=!iN>uEs7oZ!*a0ggfWSbZ$_gh*Gtc1xlN^EcLt-?klWrn#dJF&) zBdR#8paLTT9WO^83@RXNrhHmFA>`CJPai@`X>dGFWz3U3O{oS5A2^?bF&qKm(YH?@ zA3-Vw_MZSg5_r)(i^4lJSQsPJ1oblI>49YK&`XdA=o~$=L`tIHdh`nd2F)N*n1`XY znTiZjE2rlb=@W=~>K%NUo)YB$=te%DOyjw1G!diWOHZX<42DlogfQU~D4OKzK{{NR zua!Z+c9PU2k0?D`q|q-DctYaO4w)2&Ct9==G7~^Il6h3PZnmH!!_)D2Aeq=8i*%Y& zfu0pmp-ehUsftsM{$ZQw!Df;|Qa=u4CJUE_oh+O?K+iWMc}W$P%jD3+j^v;)+O+*VrDe84l&wHU#2TmSbIgU@1kQ!Xf@oZ@xVT4&#`AF1;=@(T zN;Jz$K|6SDsb=lzhBe_CcjVKwlE?3h@%Hy; zPs&)9l}Rt;r9&Y9z z6KMa<$>84j_V$Ik?(;QW-{0-|+fVh6r^{dY?)LdR-Dm5%FVy!Ozt!cKub!W-@XS)5 zVLQ;G&B0G<=Og**NbN<X*Gg>RA>iiL+%8}h^>-9QlNj{mgvjfMZKN%Ftj@3wtl z)B28i$A7OH>jBBd-G$fY`Cqe*D7T0Lh!B9hAOHbL;IMUYISC2|AP-o66tknng7B$e z4BPQx$RzTYgcytXU}3@&6!1Q4K3|?-ZHhuE(~uGjF?o>2 z7YEZch|Zx>@Hs z3>ZoC5<(I~$Sp{rI?NEnqQg;?jba}IiTUjJQ~3)*JS3nVrXS*7@nuK~KN?&diP?qt z(oPW=os5;+6U>p|fH62`8F$+umKDXa<~ZY_eZ1T%ijg+-$(skPVi@2jCUB-vvhQoU zn|nrq7m1ZyoT4Fqc5@yE9WFI23X-@da0>&($=86??3p09#e_TK5UI}v00_3^R#}BX z!aT9Kn;dzup{aXhbYil(cO>~{vum*=>bD2lM{%o@s!@4dbwmadMML*(r5r*U-3AyxeX-YaPqF9t89u z?MCyw5UJxq4pT}QLcj#*Hio8vbVNfMF58p|94Mdf>*oZD?P5Sj<;|mjO2qu!DDn|6 z4PmTyS4Mt&k3Mzs7Etc&R_Z2eCd0;DeRRLs+Yw6;bEve``R- z#wK7uP(_b@(sf~2gF@Ew$3(_14S0Xn8H(u#m7a*ek-_`Y!fF_qmMwzG9`%pwy2P>U?3aVqDL2fYoyd4J{s#u_b*(>R+v82Lnb z>*I-xb(YdPg)_w}E_s>`G(^MWliJRqoj=_JT4E-pE|R4_p@BfKbDrw4WHuh-gQl5^ zZKvSf-gd5b+5uU9RYybb$n<1(HwHS_mFgCya?qc2g{oudW>ddWI)`N$Akbv)Ar94o z7voYr>bw=ci@7QC*UrGfFMhf=ZND^=ZR;2sx5-et1jyY3Ip;{FTM)(vF)9agc34x+ zd7otceiC2YXzVL?kAV4z$~d7Fl%LcwNa_~By6l%bg)>W{*oTu3qc1j^`c!&_K?Xzn zV41pk0Iei#{Ut8Ir)JS#`v}G>3hNhNy}t+pj_whRnNMZp_ZSF1ptKExC8$dn*mmbo zpl_oe4kt|9whw140wm@^5{p2IrN0E(G1CyKO$4YCdG4dIpqD|QF`(@vz8Fuvy+<*B z5C`4ce^Yr%+Kwj0vyq(DR~1%YSBSo*Gv(tVqg{(62SA3Bc88mHdUK6ryWxWPy?ZB zfG=FA`a_01QQ6irGLELL5hfmyZV*2X2T*g=rz=p#ajoCp?Y?lQySS>;>!@~qjv|h2 z{jRR(OkMYt`rdsvnv60PbJFCWT)vyBX(y`j76q!(ng>CW<^`zd`O5j}+@7lqsJwHL zdQq0bjK>)aqRns*^OW<_x#v#9Z~cy+?mm$&H>YuaJY2vh%IBj{_1K7B*8=|)~tHmIQoWe)$LF+LvFyR+t!F^s;^Hi_^A0w_#l=_uRSKXHH|+{zUcq zeGk2EyL1|%stJ7(uz!+0k&22CO%lq_ zVhB9v@|o=TeF3R7$>ERZ3#4Cu8s{10rxZw}S^QcpQ=fv6Tm@_^ zZ}Nz!4SD2`z^R6Bm|j{J4>FO3nJL2d%ue3e748giGy7X}SlD!G*q za8R?jxN2!>)r!MaD{E(lhjWh{3;XMzYG0Z4@9YSCUU+xrnQfxzsPK~GCyNgsR%>8M zhyGP+YO-HnoUnTKvasTp&fnX7LbvHuHK2qisyARevgVj>%~9Q|Bibd$z|wi*PYv!E z+(dTJ3EL5e0=0cU+@lBtk1&Oh4{sN~MBxZlI_rdSBSi6q85|l4)R{L??pyAz^a^2qJ%Qb^S`fkDkbihY z+-AAA(i5)~g?MDceFGk%9`!R^L%8wU#QLikz=*V>0YLd{#Bd5|XdtqtZ)EBf1+0)y ziK1`QEl^( zk^gwz4{c#X#on`Od(RU7qJ47kf4t&@PoB80Dpqksu7kDzaW7*Ze)&^n&|2RI-) z3&{$@FsTry3xp~`v>g#XF2ax^p&atWK=?0emSB-2aF~LUq;TkULNmm=B{UHsZ5F3T zBqd2L+aQb&f&irqG7mw%HA&nMBwwYbw&|V_gMM-5DDQAdM0r8;WD=v);Lt^gZYD7^0cB*sT0O1 zZ6leg*1=zH^;m{NmOz4R+GB|lfHKkF{j6n}?DZUgNIpMPha7A-)UjB(X{7AOn(n5a zQP3lf1|1mN19FQ<8S>kmW98e5bSafB=tuXYrqg^sxc4>(iRv*6^2bC;1`T+a%r$|V zn0%NmZZ+~3yX*(WWXo9gNvt&-ckTY7K#3=488a?1tXZT?+R(3UAF{#YW*g0T#9}ei z%9c4{QfK_2DQ6IFfv8hcFI;+F`=G9K*fju>4~q7R z_hA0uKZ2*Ua!b5&V~pbYIK`?E2t#)Ekvtm%20!(3lf^fcOG6nCoK?fd zTYR+|^9O?gA6y*7h<1que9+Nb3?x1*pV!(mdWP+=1}zb0F@_N-J{YTT@&W#!-gU=r zJb2psK=7ocXXJWKxApEyPfQ8)!pMS)k7RWy>wn_T2bo%DD8^Xts@zisq`wa{beKQL zkf24>|A#oQqSda-Wq~sL-Ibr^RP+2npcz8`!2}`yVCC{a#^#&Ky+>g{{+VC%Or-q9 zc;(hO1saoQqqyf|l`BIb2*}M>vOGvOIx=xo_h4osYn!RGPUUQ~l(XVw`8Y-9@_b1D zXPKj1kS4!e-+Q}fWO1=(U9om;N!7Z;+7)Oku3A}CWtl;J6}sl+L-WXD#yVXwI|*oO ztpj7X9}F+e*BGa9HXtyHkusctt7UMkMB6zdiM7drW!3Z~HlJ&MFfwkQqcX)G}#1_Sl+ z_jhKc0X}GEnC>0B10?F%ACqU-6jhm}$gw)GLX-~7yRzyBZG&T$Xt|z_*^cR)C&mZE zDge#2&H@EtYkVY;Yke$*6=l=dpeIt<8L6y8k>-u>>Wog({NEuRu zuYhVjp2Wm4fDbz6s^%rK%8tQPwO#1!;P_3 z7}F==!DT3Cq{=_TVb_1XTt7P*@IerV%>a^Pb?4A8jeYYEajk=6sa)HPWY#rW=@zVT z34;)0w;;$EMHWjgIRBudzjQq&+a_FYTo?rUB(5QllX`oq=FM!aV~ETxoNMzW|K$3U6p{y_Qd9=i~kE2z&yUHfomaRe6sEpMJ1CUuQrJtA0BUvb2b zuy$RSqO=WxDMpMoh!lrlxG+S9Gt9E+axPJf8|qsqNR+sS1LlaVoS`O(VHoe?NAN++ zAN0T_JNm|oZ?sr^Re`3-QhacUpAC6k z8H*az9icUfGY0TpdVn7Fdjv7`^fz$naS!#2ZDE~{3lMxzfDX~)gYWTiEC@3mqT)QJ zf(&sk1@&G9(0UyMvOMz*d!E1k~(hP>g!Vp)Bc;)Cg_Sr!=h=6qFm~2H7 zAe!NBP~awqS|<{`OgTe5Lke_V4{?Glr*S^SAR7~QVS(zgL-HUg&i;|7UXZ0AUb&f| z086zc@xeT0b=UBPyFH&?sb|{dF2$P8p|P>a$yEg!p?)fY=YsR0y;<9Dj)eT62_^*tNIK!HNO} zD*Ga}Q=!@gXWs&F5PA1Uz*$~WR<*25yP{0H61mdHtJa=xSQVXroBcyu&9qk%4?J_K z{>4+e^=0ZeU)ogu^t6=N$e{`F%6ofzFI>1#c(^qC@X_dxx7~m3iGP=c5M29rW|nPt zj7x|wE<16!w6vzCnz;5qRmzHAaSU9wa7E&`FJ7qMbXvFZbPWUu9tRfTx)Zv!$8;;v zc0{`zdG|-EmKJHeP`8UU9tCQbB1jqJY@HA6A4qmV>0CUdBZ%Y(S~mg; z#OE2Lk*OJah>fcH=1{bb;4qZkg@qiZaL96~J`8C`cqjEBBaiUHEI1@28Ke-6iGftH z3D1+$Gv{)6;4)W`vBtl+ftm2{6PAeq zAX(4*6WZjJZOL8mxh?e^JrlwF@mxQutt`B+26v~&CA0Nc!I@T-dIDy){`O7yH%34k z1^AhMOUclGco9w+^S4PsTVe-u0c*FwlgheKWm^E6#cssETL1tc07*naR1QTT>*0ct zHFrWDEUj8{xN3Q+c6ph0Rhf2eneNG>RU1y%Z%8@&w|_aFDt`R+lmkznYk2WO?YbiA z`%if!y}hQP6NKAXhS{{~*+q*MELbqt$=+tuwpZWTu=wuF(~%vScr>gSQW6|_dd{<3 zU)b=}Q)VWp&OA#?a~~h?fqux$a{0nJpT#rR|0`(gca2+4={B9IdG<`r#*@`J7h(1K zqq?<6b*qsPIi_7&ru8ZV3Zds=up{U$oDTzq&>jn-+Z3v7^58oI7!eHr8}Yvr1l=e6 zP*Xt|8b51PY*D7dIY%`*F_|oHn~}nr=MnR*e`;vXTc&ZMOy%@s)~BLL);joyR2JF# z?%|0Ym+t)KO1}d1|D7iDR&`5k5yCgJPY; z8E32PG9*n+@J+re=A+-4Jotev-fX69==i(lEw9aMecJ@i{&q#1->CTOc7C(s`!?vV zTi;xSZoK_3F72PXp}(lL);AZzXb^PrKlJwBHf!Db#)8(jjM_f3Z~rrH_7kVJ9irCP z=C{6W{Huz;?y#-F_Q;Lc{C{xozgY_%f6o%SjYi{x-wz-Bz^3(Gi}t@Sp6Ef&`NaFl zKQHpVZimq~o51UYnuk$5svQidTxhvXAg?!qS$jemN(l_(|4Vs-ux~{h;RBWdVI=H7 zqFHTgb&lSfS4gjcCac4Yk={wPGJlQ?6t$JLyX)pM9G(AGjyQL zL57HvIAJ|vUg{SJiB1tL2|z|&CGIGoU{SOk04}3&N<@NA5_TTO3Q62CiMn;!FJBNW zNv&uepPcl-96-xZX>I55wYnY=WI|+};#i|d*{3IQ)aQ5gvjfGJQ9$Ke6e+t>+cPjS zu_9S%9mBfB%jbqlDLxo4w~1tsFTE&K>htYg!XLDVlsWB}qdwS1FjZ|B9}JN?VgjZk zMn4Hn6857Y8{o1uK#q!-c63Ei6tgf?>Tv*s0&aB0S{G!o=u@z{OW8zt`W|GqNg_MZbnlzEb0_D&9Fn^A9c$VO@QtO9C-I zcnU`Q)g-m!Uhw(dunP_NAk$&9tI{igSwi@On0d50PU*N$0#5ioeiiFWNkCqKjO{LQ zAVwP_&edD(thC)NcJ!85?-twbm7HM#hWo0d)^?8=BZksvq7`G~ljpfk%bk_>d&G`= z#iswNh{pkppXF3Tu^`57xA;?x5B>w=gG&M!CvOSKKNxcQuIOvPI9$9X8>8`nTy0+X z&l|S8D{XgGntpjR>Bq+MYBV6M8%~etut)s8qI3M=!__6)#}gT&RBpz8*^}9t{$a>J zxGG;`oGiD>Rn16~@5A{AHy){;7Au>dBL8EY%r--Lt)VZp;y&_4MTZokB#w9yMDUR! z0@=zZQsi%cSJyi_@xz@S)c$`RU}h)FXQaqYF^&cb!bxngwsT-~!X;1r_(A3|Q=!g(e&ybSp|Qi-&X$2O z*Fp_yd2czz2NxWIe1$Vp*u9uh_}TY&{wq#qlq{c{z@U31`3L`cu^tT<;Grb)I++b9 zZycGJ46nG4#(902Zb2I79HVp%24v7J2swBV8s-`#1N~zBy~HMhSsx2CH1_AFMFC+rGpxn$KspGuoh+1x!9yD={RdY;xQ81fH7D%em4twqc0dKc}hvKMCAyo*xX%V)qEv#7`W7HHDr$sI-m%?9VwIG7pkZ(XM!92-yb2V{lsi;*4?d{HEq*tJ<{z}(S-BJw13i2swXK6cS9Mwb ztHRA&f+|{nUAYB+Hi*4&&+2PH{eVB{EkU<%<5rLPmlY1XD?RpsAhIDoXz^vmQY?sU z#q$Tz%ysq#Z4-ANDFGtU&absc9@Qc7ox*$s?^oa(`U*TmuqXT{2ZH_4Q||r-+`uWy zBzUdst0xZbit61WF+OHPI)*upoW^Ez%Uk?InCG;9pbaQb!c!KGm z_OZSP#b^lhrtBR0>RQ9J$VxPSn+8Uar_6N>23FpmmMEK>&MiVlCW%GnQPDB@b5ma` z*M?@gQ;9~^In>xc>W_KE&*lRthQ*;UllYuB!=UcHLA_O4D2o3?HL>w4#o?bCk^5n}P;Co(t8-?;U~wd>Xx zLzQ_pHloPL@bR%x)YOI2{9W#UTl#gb#FSJkVI z>Qm1VBdJYQ+SlSv5?kB5>mbi-pf z#(DZT1aK7sC1p_vg|H3@h(!5%nR*f*1!wScq2kCqyhDCKX<%y>JtZIve4BjKFpt}q z@*t1E5H#bFD-u*nZUp|49YKvmsF8uaxXRq$AGdYjJqK&F0kb2tzx8) z<_pLs*s7AqC*dej3d9mCM!3f&xjc%OZGf!t{UKA)5UQi7bB-RBBQKerFPcl^h>de8 zt(sRZreaL=zf)t&={qD`g_>ac{7lJb%ygVO5O>ui#{e+qS%*SMtT0@{TTZMk0l9dM zC;c56=h5Dhn?j2-3os%ot3t?7U{eS|xQ>NtmqH8<7HT|-)m|lp>1)~+nm ztvaqlGj{zk?bGM(u8KM0`SNSq|FwP75%&3WwJ(2n_nFL}U-d9Mo)U$oeScr?TW`HF zZ{8d`I~%lFSy`ByEPD2(SE8PCo_u>cW(yivkR7wDUz+j3w#_S6uQ6IQ-`3U|RS}6S z7cQJ%P*8wc>*(s;1>@BVq~#ilK1Yc`*)*>t7`J~f=Ie)@Rz`eW5k9@VXePYqz{ zmK+0!(EUi2dkI7tJC|0u7HFIbaENiC+NJ?m1?xALu=@7ZZr;$PoBNgeYGrj8b;#-nCeZN$9 z=U#8I*oE=IfJa$@FrNctj=s`Oaj;H``tF{^ZYXI;@e|erZolFl(gqFCM*VTTQK(Kp z>kMqcBXFUc;kravi2OSG*q^RQJPa;mNLqfhWxj8ueYXhai8SS3k4ezYj&(M-{n;6) zbMOCO&pZerrDmNwpBmLxjMhlUZgXRG3Sk?qk&Q4v4xCPU;P+DNTgH#p{m`!U^##4z z-eV2g(Q3)qZN;P8Eq1*5PG6pX>+AC$g+4#9ZT*vJ+YYPl*iVKo6^+%(C%RfE2YS)p zv04@#>5l&klmT4Q#;~_-?^(8g>NcXjKB^OUez~so9rH&D;3L*ONuLj0JvMevJK57Q zIoLbi-Z-i*A2^-(!1o1!2;aj=9zV22H`NvP*4P~tx+v-czp?0c;EtvqM8!~H7zdy? z+O@v1uqWX&ba1p<`rG&3RNH^@VC$a@pw%6swvX(GZ=M;wEgwY<8VofmKce_$@Q2*) z{U5izX90xA-zyCz8kRQ~_5X4V0G&(RqP;)a5zb&RM)X3;EhBD-Fg+0PJB$WeQQ{sA z2euBr&O?C|E+p_D3Hv2jAr&$Xq+!LpATo(H#sDI(HzEq5m>{~YRrn3KI&P4X4TC8@ zNRh!XKDao9;EVtfIz~t_=mYqmeGCg)#?b%{64IZ249z5nbudVPFyMx0qhvjx7Rsqk zaSVn0oFd7_I7Cqx5oaWn6Y3l(jDbI#@i2-+E3-npFbP=3eV}AUOcIn+0%Hto8z(0y z6|e#^d}bF*VM?^)V9*GFba8SUEbohaW0NS^J2)HH1uQsfDjaTvB5N0;v54_ zImWOyVY0Jo5KB!hZ~jwp&0mh){;949{dd<7^~O;Uit8F9U+6DRA^Ic-IO}Kz`O?P0 z5+AHzyfwSZGF;|_da_?`6U}Hbf6yXK<`72*$}s|l#UTn0FL3%e(433rG?X@JZBN26>X(Cxd zA+(E>xy8#TCm&{4wk`~mxWq!Rq0@f3S%mD3qU!O9NoDik-%sEEQ=u;Ea^ukGgtBD- z;Ld0UKuI1SOuN~Ab8RzYvQ zFVtImi?Mz&W3yKrgXMaQD_Y+Qg{P8b5ZqRXG!fR*}A=DbW{N^hi?FEbfmHkdIB1(jNNX@XE?t2p9Lhom~r%$lKA4#v-KhzO1Z>OhEB!c zXyuX{E$^mQy^}@-m-DW*IP8I@ynuKpc`07iJ2>9dKf3ehd)rRl^1ghprgP|S-{|nz z#Hzw7qf|Lsil9Y{Wv)unJkT>V{>@MKHkVcJ__mH~2hPj3GqrOvAZH)HbirOa4dPQS zNay@8u-G~<7Es=_=W1i%^(MdaChzh_=X|v>Dw(NVm@I$#hz`9WwOzyL)V`Zd+U}u| z@yU%RY8Ipb@@SC-Sy@OsEdyizH=6eT(un^0-bC#*ITdP5(iNf%<-!#ChNHT^(TSR_ z;m@z!d*}PQL=Im26h+&dG&u$bF=OZujr=mlX>%;Ilyj5hA6x*CD_7O=$#-`)lxU3( zVg2HR>@n?w_MtJ<0dxse5*GNfC9xt*c(lt=-R>Si zYrMbyaQC^xx&Ta0jFz>`O2PUh8N>ykq)W35rDdK9)lI zC-w*f)}CFcZ1q7!W7kM?->C0}`j1O%;(ltJ7@z!B{lF$f<`e~7Z?{lJ6d*m%bs`I} z?|l8=4&Pc50qCAx1ha(TgB8t2K9!4u7>`iKbe}j9^9R=-P+EsGgc9fqQYWAwk5+50 zK{7%)Ck7b|I4G!Kv3sm^>=G*TLi55v$XJgL?h?O;@j;)Xbwr;8Pm0PY%=u0DzF}P` zyE2G*H9@6r=Xun(ht-NDmax&n=oqy)ntO#U4zu23{2Y;T*AN-c* z4|=2OGFNZOX<3K7zHd3Q8$7r1*ILFNzX=-p=_8BX4MT$|L^#ExHt?UZTW%P9C)EAz zO)78(4V`d(`@as#F$_2M*Dvzj3_i@e1W4TYbmW2@B07!*yRliod_W6I$KeAiv#3)~ z6PfX0g4Fvaj3+|kA+9=O$jd|hv@6kY9fK)c+r@_78R_zi z_j-@iJgDy(Sy!xGkg1@V0kOkqluzk_fa2w={+K3@k^=GH`LJdI&Xy=dE)uAec%vjB z+K?|rb)PdSjmbFApcX))Kpq5qFkc0c#W>5BbFs#y1mfyEi!~6d3*&)%jUR=6-xry;Yp^Ax#(GNfTfLg1+ckhcQ z?brX^=jF@o&!5t5IR&o$v!`k{9-UZ}`LWDfNWWpyO(?fWaWl#A)L5IvR{z#c^>3oJnA(}XwCzECC=NCT>6^48@GedY$ zueq#$RY>2^d_hZ1cmGGpKvVEOLLr)(i);#oP>8qSzcdI?pgsYD-&FK4z+jbyFEI_L zz@t1kBRf5Kf+H2L!^?77-Xi*f^?pue3eZ;Z$yf* zED2~9Ydi`yi_umBk;NqKAhIB9PUzMj(>--ehgN~>(|=m!m;Q9Vc~Y(fp4-`rzTi$%%&#{Xc$v&C9R9{9WS<=W3omcMI9& z&8KTNo~n7~MD_ZU)f>uyij1=rXqRGq(DO(Y2$euKQw`vQ_&b9`p4u*-rYf+)-xSh zyN5ryeD~AK_4fzHUOIj2rE|BQJXQV2WHwXP-ZeV0>+(HK!I(-=$yX>$ z``=(A`cWj&%^(U`4=);$2ZhfFhU8>}7>7q;!QZkkhKn1zl+nj|=?#!j5xI7=6J(hc zkOi{ur&7$J`#l0nF$VWw-`0l{sr)+KaCPxPQOCwufo7eV{0X_)8oKffQXGgV_gqe@7GSSgA)=)j( z-uMuIhy&R4uKI!Bi}<4a$e^pgYtjBUugPKLlT3E*+4Shycg#EfxpcJZCTU@^_rcg* z&Db3kYW(5E1i6u+%f$dzLKL$d8usq}A5V@9O%C)-4))S*sGn9J>enAc1xAKYVZwHV zdw*?iU!Fe{8y-OIKYH)&4U>a?gWu;EP%mxTc38E4;yBR)ZvRA2`*>TueyBT}C&xy} zb%!g?wSDaHyD8P6VR>zSciPveC--jGIR;*_2>|5IGNPOyZk7=gn6nAwlMIH_h+v{C z0;pdwjr=93pDjKD(Y^tEL~szM{)J(Hk8c*kc~HzQfD#fZ&WI5}i>MoxkvE7w35E_S zV-ZAA!pNlw8<9ej=%Q9w0>wUBCWQ1txNa0n14uDi2XGs~o^a4E;rBtfYaGSq9C_v- z)z*xmnj|C@paV%1hzWf-Dj8G!3`9j}AagX0Stj&4CoFPeb5Ks2wcAf&N4r=Ws_Ya& zJ51zI2zqD>F|a^UgeYbJu;he42;gI!oH7>USj#BJ_s4rZ!{eh9lOq$88Io4GBO-5t z#h2|c2$)gPGB`FdG&V7Y9(32pKTh7Zh?D_F7R|cGunU4Du~(bWS@(kq+en$qe%9Py zvisB>RAozs#yp0xPk{V`)-kNKq5p!e%RB-oenM}7&j?8a0YkKpms`iOZgKKrdHdM- zyC(xbt435Ws*Wm0U@$jWdwLAl4_TG|Dva9Rb zhtU*6#kvN@z9_CuyK?_Q|Jafs#?>F-Ho+6ii5+d&m-li-7>{o8FrN5yT{d&D4Y z5_$LpxJix=`08?fIoCQ`ABvjxvgs+7#^D#7@k0X6%GAg6y9>$Dw1Qfms{66Qr|mrt8;i&sdgb~k0`A&6bn-1+fUvSH}?;W zO`sQsX&HFq^sV`+oG6Xr0v0LU*_tlY4oBMQ9l6sD?bLJ)uRNk#goA$}>NcI*biDdn zLtig?jYlV12gf3%%_f-&A(}@|Q{oexksVPgm!oL!9);hjeb*b28*iPfnv=*DXgY3p z4_lx$a3&~_8mDq@`D)Zd|Lcu24?_MyvsBJ5Lx~10R@#gfW0pB8^K9jUL)^-es$UvV z7bbg0Cx*u-3smiBm1vwrG8d6$IbNk70-}>8TI?mWk>cjT@d?{}H9AfRo4q6BD~qa3 z(zuz2HFK-XP;1o(sK5@<6@=UfUjr)s;%gTq(p*nXyVV07ZsOZ872l3QlcZ`C#_ z-0Va)TMe2sJI>yoAJ5n&DO^JoZowSdy!OdmLs*jO&pC`C@x}JR(si-i@aRO;4-F>! z#Ga5tmT?VZ?ENI052`L|x_XDk`_W7s9Lu}W;vU7?f#5sv`&`3V2Vd#SDQdP6jml(u z|Jaj>O6zFIKe!}}MQ{Fr>-UX)#fyU(uVBU~ND}ycJ$f_Oq$ur!q|gGk?L?UhD2>4~ zAf89Vhb1@|M&#z@$T(2kFp`;)d~hG86JmVOX}9?8WXnS7a25YgdJ*S$55)WeMw1Gl%!ePIpmbDV%r9bbe&Hg*S&JC z*0xBE{KuAovD0RyUDpTGA6%v|3 zYQk)qE{wSLL}`FL9q?Px2xLR%LE_pYhZkgvi;!!tal%RY-Eq<|uhOa|N2^wpR;@l< zg&gN~Wx6MkD}Ac^nKQNP(tcPOn7Of#-FoiM*0Z{8mzp-ll)ma}BCouG2Jq6wi&j<^ z#7Rf4y|MA4x88gm+2Gv7{nc9^d-%SAR>_XJW3T-o^@EKM21bsT9WgaECbjJCZRgFK zvuoEkKqML-{pyts>vsJ8l}l|eoYrkUbL+X&HP0QdLB9ykAiw?umNPz~TX$Hyx}<9P zk*dXIJcY0nyY_`Rdsv~`t^foEZ1a?&JctgoB&r>TA{JSCkcR+MCcNWs`babiV zr&)S{#Vm^l90j~DOAqe|So0J@n*oK8q`5Q`;BxZTO~oY%lA!Oo(6JnYe)2)klscBn zS2qAXO@ymw(KLmkBn}df`b`Pfn?muM${hOjH-;(23_npA@OvKN54stoG7wAHUBbNIg`2SFxYA@R!YrdbO5P z_zS~v?b?&N^(R4Q=-Ct1o6D-Uo~?WF%kj1=#H^h4@O>;*Ov!_DT3>nmlg%r2b$8O!QjobpMu(Ul zBcp|{zrKBVcnCG<_vg1)Z+m0gkBu*%t$Fcu&GYAKo;y>s>2%GrC##=10g&aB$3XLF z^-=ALqg6|fLeD)*G#;S8tU`C>3_`{r&1|2ivdL50AnS=<8#E5W3q~lP86>j?c@yaH z!dfs#`Nvc?foVa`lXVd!>c%+;sG1)B6h=~3LoweQjq;AcS?O}LlD^$L5+Q5;^QF7B zJ;MtQ$>*oY=Vx-L57k}6SL%CfSPnEJll z>&@o7D?A~~jMR+?3J2)_qT{07m3xjs?Ayx+m5x4Q4|Ix`JagmKh}`i(_(;f0p}7fX z{i#rqn;>e&AN1f$y8DywoP5iEdW9olqY}w}j~*ngk?q%7Y(kgxeKRPdFN|-&%}?eW zRZ&w1H`|@``EM}n{%_)gK3g76PEHQ=w*P%Gz?7dj0zUPzL;EL=9sgL`R}w~0!tRK7 zr{I`?56G$UFzKZ>?8PK>pSMdlJ8N)QJuIeXgfqJqpN>p z2hD!9t~Y(>c-y_P8ivmdXw&w-sQp8`_D@~u_L*Dz#}1vlo|zmSnH(MN{N^e2UpOcf zfMiB0TeyE+= zKXdO+_;kD-xD0(qqFUcr@H@Q{0JFA#vFhRQ!1&nM3(=R&e6QICU9%3phJ$|rm=i(7 z@ihSNg09g(uoKCIlK=!6~R{6^?6#;^U-|5PF0vlTd9F{#1x?iEEIcF!VZZ zgv1wLw~T;d=pbz{CXqmo$UBvOV^Fm#Y1ZIZ&a@d6bQ;?Yp!q?S_@ zgO6kEEDFkFERU2yUa<^VZ{b*YBnI~d?W6$zZlMVmOp8~d63L*R|TyC?O=$|M`9c#S(MJLFRFU}h|VQnJuOZej`c}A^3}#Eaw~A86(;!# zqYQ50LDs!Ujr=a;3L0l{bJOLPpoWRFeWi1zbS=$Uq{#`d2kl#CD{QhA);UV+Ooep@ z2pL=DKr%t15K7mppsgV{NtHX~DDBa5C>1SlSPRGi4zj0cLo01dw8$((J{x2VMAfa) zS*$5cme#Op&4e}Byma~e47q)g%B@&~E`-)jrl^H1dM&bWC8!+!S`W)DNK$r)o1eiU zx6rvzZH)uOEKujuINN-cW1*U?z0ncW4Yb^K%~jc@E6`aSw*>joTrl5+F8i@2>4J0) zEz*`1Yo0q+?U|=Gif4?m#3Z_+1s)c%;eFgUboESVnZ`e6frOePa@bI2!Xx9GM8}1E=Xd# zPrNLWU6a7M#mYtg631XEW^>EP2N>5-nQf2+)mnOxTNW<|A$V;1Pz$SK9g8eN=0)j|9ycPx2RU zUU+|rpY+L4W*M4&2B$oHB=);1pNW+JtFXrB=UYGBtzGA^A^Kw6(*xEo^-xb%>thF z9(dw=@QYhM${1!kou_19I4f+l3@z`;Pil%S8eU8Ii^}**geA1$0$ac2#+FcU6CPQ9 zPZ2>5(67l|{lxD6620joo`tUzJwfW#l2}DD4KwsEFmWWQ^`H}k_}2tXCoU}*8PbAf znjE>tWFEo9Oyw+7(OJ0#u1nli;{H=dJBLO$6Ej7om{l^^C~)CHv{Ua&LzxWHh|)3@ z?2n1TS!62~WpT(%4UbR8vMrCLvlam2D2xc~!28$WCgXk0gP}dw8m7gHKl-6=WPEaI zS(Utf5Lwr02V^FhikV5wu3s8DhsH~^o%!1K+dU)4Yr7WZaP325Po2<($smRxnQ49T zWX8 zuzLK2c>Ts$8QBbSXDO_Z$t_UYx>gz-I$7imd{_w-udSD$ZgYmW#I6ebr(ttKwD)_u_^# zDmoUb>8i2Ec3{TOi6LhsZA@ok0}WGJFpO04JO0s%Pqv9|7SZ^GoO?IZH~K z^8Kerut+yr;OUJ?fba_L18VP(O03q_BxF9}IxSDFczknPck%!y#CuEDEJj7?Hwf9L-(I zBW`AV83E%1I|B4Qv5)V@fbp{nEKx z+s@X!_T#-RxmUM%h|cGwp=n=LrSb4^v$Ql9?1;I!sjIVtnVrL5mzfHA3UKDZS?X8j zIonvey180gTMDQmG9n8X&i~?zf5JzG!J(bopI-a!r`s<#zjCJfP;u%JHyi_bQ{Wa>yKzxmuXiV)h;isT3Vt({r7~gC2Ch>7W2^Esn7z{zCdM{r?k!m z9x~)9$ktCkY9KPHX+|%M1*mb)wR%A30e#=L@L zu6_~$YlBq+zX)*Su|=USDdx%>0@x4IFDcv%Ru!n0YaovvK7#uB>3u49Ljchqd@uo6 zG&C!u4{iYl2E|}AgfFK6i`d+_?31n!6AuiJJ{%ot|9nN;yXL;pgz z^#lYotgyI;RolCkEw9WPx(d;z17}iNw$E)7FfYO%#Q5OA+0@oo=Kuuup4Fq)E8<;106(zS-}w8YmKPuI&DjSdqb~oSrCt7i-QKkr zQUeV$Fz=`N-d7z1uGs`!wFq&M%=JaNfg}(-_SOm_RSuj zBDF|Gy98Dg%UDIoto8%$27p=&hD5_bnb!t`j1F;7#%#Y#gzhMYv5zOA%7BHDTeC$a zqh!!N>Nf7V4W#yy5y*^`fgRMzFqytCK4pCn_Cb9m49X*?8kAwsD1}HZBW2c6GKW~k zCWf)b1z>mqyM!E$&SU#DZ=+EKS&gajyUcUqhfBf zip14NVwnHS-B;q`D{&V*Z9iDNy6ls!Ho-_T3vd{GNkxR0WTGi^*gp*xrzErUa+5t9{IE3`bI=xHoIRTHFw~*(HfVof zTy4XwakSwbRij@=%pm|WK(;j;WSUty_B5}LSIWL?l1vUs7!VC^3v9JX zWA!mPWQti~%&Ec(u}w1BEH~LEbBapl01dzK+Tq5sg@)7iDJI(#lU;@|=hy{`*&X^s1>=$G9pJNV)G_l?Z@EjW&#Q=kY{&2C;e~!*OO6wg96EFbg<^s`!DH+TN?-Ql< ziO@ojfml7oE@UOZ1Ptqr=IebHK*!E{FxEk_evgd_X(q-yTIW3%kROyYmmQ_^jn=V7 z?S+~_ICU)40b2JwJ!QRT-2(ASwzEHGETQg#l|WVXjWww8ADEhg6$Z`L27Ff(G`mRI zO7t*H>p!c=^PA)DUmy3Hc_J`Ww2N@weqkTnGhkK`TZqe3$?V@iQ6RB+)~xtGp>!-? zBbn0W|(>szYd+FEnxQA^KN z70;)d>~a(#6z6(~CRmkx3(A^XEU_of|0UJn08Wi~>#`M405?<2k;u!Wl{RXv75$s5 znD(yuC2lKAy_n^@veXCppnpMG;Hol6Do|M7eQkNq!iruSD*LUiV21OHo2!^YW5+x_-N`kOOm#;``#?YM zF0LLk``B5gwQrs*or~KY9MYNX859)Y;^N4@(Eg+!jE#)}Z{^0uk6#OU>9er0|G7MT zTh*}bwS%|UyoG!aq(57#`fsXwX)^>y?7I%4hxJ%j!DRmSDbxtc5kyP!2Vh(o;{7OIpf1ad ztS!ULs4|+e8oekzhA`53FCqG_zzFq7Oc}b!2@IusiL~lr)IFFymUVZ)2PM0eoQ^B$ zauq&^J%yDdP?N17SdEbNDFQvI3KR*~z}G-^hCU_`aJQelX6fjA(e=f+-jPx##v zkE}@9tUR^6e-S+?Ts12^w;0kJ<-sI_dw!`$0dykXt3hYyx2B8~$ieH%yBAixu(rHU zVMX5!l`m}s@!)G4s@_;%_2$Ov!JDcFZ>wSLdH9ankvnR~?KwGkgL!zr;KVPcH-lGV z(}y2^@ci@7d3d-<6`_lhgOiu{H$(e8nAo0X7%)*uJKY$~%dfoTX@!Y5#HN$t+4&7PH&i*F#BU`HbZ>f4^Q`IYlmHpOL_Fh-{LSaQu z6h8*7g(!-FD@z0OV3OIJU1n|x8&$ir-V^7>9kYx1G_|0;H1ExR8qNptt~zEFKbveC zvIX+cf3p9#c1cD@nt4tW{8x1AHW}vN{4(~=7`L~6%=TLEywVP7rhql&>=sSjUN?D1 z{fObXiuEd!#gIAD<28YBx0w>psYBObH$A0!+GKhw+SF1_dd=L8>Y%Lo4vM2@#gf{qV- zE{^WRA>ghwE1hF2J~+A~;yh>T#Ts&vOYeed=s*16>de<0#&x;$u9t%UAc3z3Nb1<0 zlkHkF(?8;a5YX_-x35{69<@BY*Z66-hOy7`I=oF;lEEn$l0;9o|D%*ClN~Pqs}JXc zpTBTv^6zllR(x=RjRIBGi`7xqd~m^Z2xZ8sByaV5-VNhzuE)Ov_om(qYo@vUIZp*T zfZsZ_9%4W3D(UduAy04w=K4X`=p*itM@YfPzOcmwDIsw>(D#woFv2^hc^oOa5H2L8 zA@Dws2hw`b`b8aa#=@QF@-;5R8AWLbB2iQrMa1w3c^}jMu8^GMsAHUB}v}CoEXkmOg<8zkwlJ&o(D2M$TR}Mr+6M@*dd-!oq-@4te+2i`l9{m z6^9Ti!RYqA$$hDj(@8vwhX_;Xy3EAN!7zn-SJbs3?}LmbUJDF7Pcf2Ba$&{*hz1h7 zaU!{Z<1G>`BB2wgi&TCIMwb;Pk0l0i6$VhrEi{mZ&SNPF5;^aK`XI7o|9Z#ism^#t zE~DcN6xDoQ0-wR@;v{!7x_)nROE3YU!!vUBx`^nifpUz%Stvsb!bjL=0p%WnLYVRI zwTLHU@r3i5Seg7*FXDXAeTh*BFhOu6=^j)P*{cWsKt6GlpV}a`qI5Z4f)dh!UdSk^ ze1yC_ISkTRz5`t*-4J3BaP9K}J+vB_!doDUOWmcV+Dg!;2-{UhDWgweLbn=^wzaoa zf_JU*3JbFm?*yvMaw}0ht`81xGsu>Hh2a1)f+Z|a5QElwjnElMzRCR)wLzf*B8aQz z^o00exYl##iD_v-j+mMH;<`$=^kVPi;&~Y9FY@PeomLoKQb`X;K*~h(^f07J<7N+8 zKHPsW6`&Pa>}1>1f%a)!=|iT7Jy|uG9IoMr)J&2+MODP7*f}YK$YqIW|Fdc&npva9 zH74gI6PB+SX3CU*mSPuhu?Ef5qIghRM~ZYJQ{}ItnyDp2j&w$k zlguP4fg6>|`K)Vd@iWT|OLgEe(7m9%3xr~XLVV}K;ullwA8quF)%nlS2LOQ6`C#QpXDF(@vJ;R-~YEHMyC z#90$Ps=0Z0N?D0$ZUb>4&Tt@?WKEuZVFkj(&msd!8olw&!XE69R(wz{vsJpnFhyDz zf=sY0vUs^QNY&P?ikhwPL1pJKzLMiPh4g6^ISiI2CK$A`jms`VRiMhJI7}Q8)&};e z{EmaetXE1!A)Pu@utHTwq?p+%@s3(Es4Uj=^veEfEEr+c@dkt!1w?4&0A3mVhwIoQ zszEBpo=oIWQz%vqDoPsc2{r&En^|y z$+b1+0UN8w?y4KVw|>Bu>du)kWAVzC>QQ^^$L_74wD;FnnaR1e+OMDt<|NPyZSeM5 zzt!afH&=VEEY+l&-`H4fms`wi>Y>|e-7+xSd!`UMsgQ_6PDzIh^Uxg-`}o8Cr=CeO z+Jk2(zjeezsf(jSDBF9NJVGC4Vu%DWq!7w32kPUP52n2;?tI?9gz>R&u8|GcInlxpQD4F zo658&P*_w1v62kG9DOIa%YZp)@yX` zs}GxT0t*=u!qcf5*_g0hg@(x9AD|GYBa zyi%WRP+YUC2A`KKxDWPxDomM|h7nsDjW78F$p+RoX=blHNMW2{y!iAAgNx|&ZLp7~ zi86X|KA&aolw#_bYR1(NZI_@I4MSd^M2FX6Aex?j|W4@;m6@QYe>gN zglt@>ND*7{K?(0#Df_T_%auW@`luVf71xuqm)2Y5Jcg7R%F&EsgW(b&$_a_mInUCE zXC8%Mb9H8+lKa#7;7tHSV%$ry+u>&FSHH^#ccwLrw`qLexA9#s ztK}U}tJA&f*)Y1(ovjeE@V@at!z6pF=z){$8%941S%zsD+Zl9u|3yAH9q-}09*<5! zYkgokcx}$u%O3|fvNKL`zydJ~GcwBGzZ?H8KKRS_OH-U0-}7#L?eAJ&dn-N|Gw$Mu zXITB^t%g+2qdMH(lLh|T2Uh)QAu0LK)bYmFBycVHvAWZY1D>%U0CbByLJq|ka}c$A zb4Vn3#5L-O^b3ad(F*J%k_eP=4ljWYVc$6-jDhuRt5YnOA3CWD=b}O?nG%xBA%P)Q zBL>}#W8e^nIIdfiH6cXku}E>?fu%hYRL{~mj+tJg@XOz9fs>#OzJ$N)FA?OMXzmwjK1L0 z2iN6E$Hw!_0Cc6EBGWV}8NEb8XMSl*`Huosg8N`_{oX{}le13o({QyQl0m2d{zh~s z0U(0vQxIfUz5o+k<2j$>P9IPefqqaeR99XZYKo_fDOFaw$`YoP3N92jd|-t26gSu3 zS&w}Y2Pk60FmWPbt1GX$HZ$$~yjI{qD@9CP2BiWDLgl1W??6(fAhi%FO#BLFUI7)R{q2N{qBN`M zq{>J>gWnc1Lpb0|Q{6gJ9byHjR>UTp@JeA5;YGaR+%py{M zlg)0KC713#+FEn1<8p%|j_O(EK|IKoXtUoO?0&Hsl7tX@0E5))iB_hST{k!y&jH~c zdyM^Jx#W>!71~o55)`T`P*D!b=Bcw@l*6s) zo)pN0syDTwhrhchvlOZW)y_JtEN#VHZ@WZTbyHTbQZ@5I3IqQL)DNmuLfpkCOvk!( z-l&mWJ0+Q*34{?tqe2x??MAojG& zD3(MLI-;y%o4gVU4TNGB8xb}f!AgWyQ3UDY(+~M2e@!*UY9Rtcw>1@AQb1wNv7>O& zh1#ZrEU{CjxnruSUAn0g1d@VH&t{n0Wt!Tjn>wVL*!Sns%r=NY$!2VyY3`C~W;@$q z&+uHT2@ppXL~`J*n@3F*evoLeJ7UnPFONW;oIV??;P5F7nl!zf$l58XU;#nG&sG4nf3u_+IL-5-feApk98F<6jq>VU-im{s#iBw z4ct`q#^&ll00(P^GW&X09W%5??W!HWr=FSHV}=b%c-_TR11X`79zE*f;;hlw$S8!J zpY1SW($w_74`~_q7sx0;d~otphsS$-Ib(*M#)kX?fh0ORKKpEk$Vdo8ey_!n7ya*P zft}yXJTT$MGoyFZj@ni`dSlJ7t+hk8)V#H^`t>c<12d z_w^O5{@n^71+)JeKo-6QWj^R)%&wQ2_JB1YS*Y1gfDwDaf~3!{MlIXQ5UNb!aA_xK zk4ExAib^M3TI~gu;xIQ?hzC`SOG?Bv0`)44FH_JY={YLIwoTO^bwZtdpn^E)nr-y47MtACpDaE24^ca(G(@`#nQ#tMGQOS181uij!NfZbx|0> zL_RJWLAB+@a(Cklag(!EC^NftZEeL}g$F3N*dY&s54z-)xaXC4tSt3f3C_hn=!y%S z$QqbofXpPU2Pv%RyT0^{@0RECYq{$y?VKE=guAd`@iDs;^gS$U~g~Nw|CF9{MD=9?QapC2*n?)9rD-g z#j#;gQO-^d4vr3vjt(B49-n;jaZOEiOhsob`Ur+F_f~%Xn+`>#P&m2mJfj+%d(p@AQqTYxmgb%`wTOX5X+UpMro1 zD2RHAEZ9y+O4%Arw2cc6m&rAroyB*afhP=txWYP~P6YIH6bICr_eJRG6%LZR?~2B! zKw`eH*48=CJYmfTG1CYHS;v1>jaG0&%2O6RyQg zxi)|DwRsb-Eqv!@_UwnXMj&qHLp-gAmkr#t)bB>EwvRNytR7n z)#%aJ;-=!Qo^*Y|v>VA^+&{6CB&EU$`cLD77e{rxwh}xhlJ{43oO!@I>Zn`9VYkR5 z?h#!52S88;2Sop00DmzY)j0$5JL-%S51Alwy(8RBn8@86j`Q52jq_g5CT|aMDZqE8u(3r5BC-eSKy{!j6O)^@M86;QvU@+TD=9-gbv;mQTjmj z%f(tsqaFv#h+u(i{2FSl%?abAFEC7UI&GkQ`^d>Hk`#rjB890-3)4xIKnx;C1Wgl; zhbr8l#*xLaW&V?O5`&waDr(~l&TJjv; zC7GR9m`Ioa%ueOvN2aFqW_3y~CPSTXqCspr=X8Yp5V1k(5(%?F1h?D-xDBmzK?Y5b zVL-$Pvl9VW90U4J0w>#g*wo$(T-VkAhCp))WOz zIeI_4Nt4^d+DIy_-N(#4Yx)tnt0agk{0^Z(YNlxuVf@AmZZ$`Qah_R1-Jd)Kes^UH0Lku9d0kq;JwcyJE;f>Q!>VKRW4 z05Lm>>UbzXbRzbPWkW$7|2GPHb=(26mw2OtFhM}ra}I0;@@tVgjhoD{MlFQUmF?26~ZR%kDh3C`wxFj)Ov^(kAB z;wCJPReuk*Uavy%v{j(GP!;|Q*D1|YDPO0+LHVyJEijxKqgCJn-kF4I165*Vr4=i? zsn2V*r`42$0ENn2h)U_q2i{`NpjGep7%yY%0k$0K`$vF&P#K3uD*Y#Ws|t5u_+*ct z>oNdd0Kam$V(f`bY;zOIqzYx_aH|=;T?>-$*zNX z9*qF{CP-6RY$t?4b}Ae}86hxHD#lfiF~Kr9$fLk>8WB8=laRcUXHt#pYp$NU z^#D{RQr)JKD-pVqhClbSmNO`VuGoo~Y0Q{qCN22|x~(jAhoqz6TpW-GNd*Gdf_Si7 zK^ce#*H!c>tn9O~>ZOg_Sj7zLA>dYAI~)z|vCOpJT|4>Eg?^ug7W8^z z*|#5;)}6e1<#J(RVV^#|ot?R9KWN~qS!*{Ijt{m>`WvMd;K_^MdU|DM>X%=C<>YA3 z?0UAdw~yD18Q+}z_1B9H4TT#wCBM{Z!qC@7Y^j>OtA5;$+A-Ve#%!(~zO8oHmYN}( zs>!rxb?d(#_~46cEBi9Lem!JX2>}Tqq)G6Hyv$|3`IucG4^x!nm$+m@iV_D@`SDqt zbUtS$@gE0vYR00@Z;_=iD|{x+GPs;XRZ^0ruqhcIP|RAL~1-DU1mqy6mC2XrcZe^i|t9lsgS<<$HuAp$` zORMo)APs4->w7}pXpXKjtq>dCWvR%s6&(tf>7hHlor=xb2w|hE=&GfIG_}Tb;qr4o|d2Mo(irGqEW`TZX&!rrg!+%j*@l;t| zm-+$CQHZi!>~W$MpA56oOUlA?t%Q@d)?m?=;5YEW5~r0A&eJ`w#52FtC%4oOJ^5K< z3tR(kxm`(tTv*X(T}8hQRWELW$b$p6@EHd1EZ$l>Vq5LVopq!3)Q{a;H(__}+drOu zGyCYL0|HMBdNShcf90%NQ?RPw`=v{!OdR)G|CdLO9KI|mIU#=GvB4cJ)7v*sf6g-f zSytzMXR5 z?1bHQ<95}JVJF=I62jqIpeBR1gM{$4E!F)uRlU5is^9vmKImfHy%2QB!Qf(C9++R| zzmjz#Wj;C3i9o8O9EfA;oQ1(|RPhDWn3blH=_hbWFMcx75NCksvt9Gbo=-JX*TaA1 zG){qOn5b6H(le~dQhrB1sHH=)ikUG+)(%_zqd2E)yZFOa5lJu#f)6@=dpz~f`A1DH zz1Yw&T*d4-Qsj8is-0WwHKb(~AkN!r2W2WCTs5$W5CTI{W`K$zDrZsmDe5&?H@|2A zlw`TJt(#TGk_cT$q|WhM?V>eB5RCG`+J;JXgz?v(?R)PTw!u-I@m=jO5(p*l~NB@ zc9^~2Gv=6U)DhQ6!h-1J8wCW=1-Tx16LYyAA-s89;^#ur!a1A|Uu1Em=1?*I-WF9YG&LJ{3>A%{xtOMrrT3$}=i&j_c8;|!66W(4wxM!BI&EFBK% zFJ!pDcp_ZfXO>VMQ8vPYUh_${$KwD3O`C^WJT65<<|yzY9R>2e1WpM-Ehr)tl6=p5 zKChuS1s_}}yrj994r)$^K$YT+5~y*?NwiU1DV&Aw)e^1^^pNrU7ILo2F2OBJ(AnJ*OjNQv)hdBkm6VtO zSh-0rys+^;;3X?Clzuh1Iq@O@;sWl4P3Lu4ZgO2}V!N>lP_%4i#!Kj8lox^Z*fc=C zfTGmz2))(IMZAWTB|@*L;N@;28kf}g2(w&E`An=3VYp(72PptiWuiw|8~L0j2yUnr zA6b1@u>et&GJ7wHhCI6j8KPKr6(ONImv}?R6~+$n`e#-cIzvb!%>J1qP#v1P973Ul zjr|0YsgctwIQOCX?U+oGH?Def6le?;KxTqdYzS7C=hM~&iNw)i6ZLp37o z2EY+_@R>`QQuJPNMpU3f4O#!=i$2Ak7`PTEa^P6DGt6_h;e%G+XTyTutq`8F&+JxE zPbJHJ+`_@1Rclqj$JvtK>8yaD8aS$np{iX63a%F@7B%~`>NLWXc!hGSx~mC{P&x#4 z=N9zXp|$AEsB=@R5nEgDYCB(e4*5nD1w?9F#~>4*q(tWP0ZCGfta$HH`=WUWYz}5= z=5x_p8t()s7$~%+bZU<#`!tcU0T<4>gso(gE3DTH{;yr8Vl~?dK^B}&$d>JR0#ZPV z8kHj;61@^zTyy@;qfIqeJ0uyM(z(5ClUsrdY<0xA3Y^iT@wtaqwwp%8K(JAXawMnY z$bu(*;vIq-OG%TwOLnniUPd>b)9F`v5TQ%YxTHX#SpSDqdJy(YLU&-$sb;*Pog8o2mzG#K?opwBJ_C z{Q{Y}y$4MD>4(nsPu}xYZ~J3II!1+lm$PbhK|#Tyg>e(cjT-REOXJ3kPRYuNiw-Lu z{EX%8j?HM-x3JH5I^P}hmxSqqR&Lt%`RD(9{k2yI4;nb*tAFR^<*Y0yNJ!0086C2^ z?_Xc~YW~!N=O*l`9lNJ)%q}qPhwZE#x*a5hgP3){q3VsossS57`t#zt%0BBVdaei4 z9=rorLv{gnUB0V9`s1~-)B~~$fQzv+n)XiVO1dXW=s55dki;fek!MLrLN5`CXlbfR z{_bS*%YPLup-lFWLr;zs2b3=zK5Zu64r;<9wSf+#50#8tK4z*Gp@bR~{2fH8DQ4ff zkV`!Hbx{-6_Jr`?Luz$S$teVcT++-1+Habcs>NZ65iPfzhhgK zuuft=-Uu;}Fmwdjk|E;rAMEI( ztrStJu8a3g>HMM`Qdvg~$=x*BtZ*~#Z`Nd(ZG_)3bfau|lx~(!l@MbJr}RvWP=NYD z){t0p@`Ut8rM@dmsT1MO#f9ZP)>Zc2QrmBP{mVP*`){v(bzAkoEtP{dRu0}&HFR_J z@U1l?x7Uu|SvO`+?fAX*llIh4*;Dt

sVyoVm0A-)crZyK#u)fx(?m412C*tj0Ln zRy)>V?`YROgKh3iXn%cNyNhoNhr5vbw#x zwW|M?>i(N+UfNpMcS~K*jn!QXD?%XPLRoNLX+VCdAI4%}ou+4Yi91fSII&xi#Y57- zo#eOOEYUWMk8wEYm{Ht5-86g`MvK^YvR#tFF`MfR(PJHd3;dUUDNb)G#C3Kt%w}M$ zLskx?-pj$bhZrKkb%iM?N*5HZ5026WMDu(gKm!#!hx-Mqjh{eB`*`A)x*O%^?|6r6 z`99W%aImQU6p2dwWe`KvjG}*(l@d_4`G7XX%6Uk+50Fi{GF+4&ZdF2Hm{uMo_QaXk z2;O}JgezSbFBykWT_bf4Gj*H~9{j(954NH+16b|Ou5|D(KDHI4eDAnR@OSy&{jSHq zdws#Q>+>ewNclHU)4|UBU+06a42R6HootB`Lh?UMUK3&{FB@37Co#pk<{_Z zi~&}z&Hr6KDE*5YrZ_^(#6P}vYd&~)Z_dq(nYZ#H;cgei+}G`6Kf%4j>p@Bhp2dGg z=3*;8*!jwe&sjaw4pct-&5s^2#{dP+IpP|57=(lohh0D|c*G_8F#7}o7=6TP?h!}e zbod>>99w_b6{8cf1L+eEq3_^>j0NJc?9(k$>=pw>vF}c?Sl}G|Hx|~@0s=!?iT^qR za}--+VLdEj4OC;X9>`QY&)(~asy!r4dNS}l7jR3DylcH6F)mOu_az7c7ozLy0!(Lt z#0`u?gk|pWoMZ`1X26(qdM}YPgwDn6Sns*)5JHoB*QG|bxD&zyqy;6jtv>VNUvS8c zBOHc;#5le8JRSQa*&5MIGeKyI=e3AWDw5yJ$!+ST5LcO94~}p z$3;*A(7<@Erz5gPb;EkXt$=b8h?&BLxMTgvLgp1nKy1e&GxGb)_QdnDs&YigDnNCh z-N;HOK1(wK|aQ1l4M(Ga2t%x_k&Un+@P~eHYkJ+W{ji4p)XEp!S zhWgQ)LU}iZ@%IIPE43M+%C}4#b6T=Lt+ENMYY~f3%J<+DB;3>SQ_G#{MHVTJk`8)E z_X35JGMqwah3H)tD20TaASj;2f!4_uxL`0$(g#tj5FrN7*0Lv@4K1A0%}yAD&sI3t za&=w`CwJs+hiuD)AkRz|HTb%wn4FSKFtkQuMgA^e%;S1YEMQ_|P9=684t0nZ(2xoK zrR;wh!i>q}43WN59h?${dJfZ}z_DWN1T1m|#319r%Y^tsqa@`sbI)7=moNZ!kf$aG z9TE&Cn>fwHo!J5&k#x!F`~vOM%+gfjga1@MW|Os|dMW~tbS`+*2wzJ@Y?#h{gkgsi zGgm&+rAcq=G+r7E5%Gq&a1j=)aU>CC2K^Rh%H$vqaT0+bsRC>VpSkE_9A)&223a3# zM{LYSeI_+(-#I*KiEoUKdI{*XV>#6G!H$3|rbp?VC(i$QAm7Rvt zE$E^Jr*QxPAOJ~3K~x^Iv>J5_gIpPMTQfoRguUz@u^O#r>v5&EN=Qz(ARj#8&pKgs zFDgd(tg0ZkCF?katB*Cp7mYVPY{Jefj0Ew)2G8W?ssx5CQrjnl7aFG*T zouYiA)3HK2rJ4F~20i1ykDPuo&FF}%$0m;*#XAri?)+y~^9resf@{y*S)%}b^tT&H&?#7 zvGT3WRYSH_58YWia!1|hU3Fvjg1J3uZ~c_Lb<=-3+iUWNhhF_(C8IiT9^!a#@N-4O zJC}^LGmW+_8tbrQq$@M_A53b0bxgYpZwk|X@Dq0@v@0FnVf|3&pT{|qY0vgQJh;=I zVVZ42?9Pt(>%@V*hOH@@zW+Bc?f2G?1&`*s5j*RcX+MOS_M5BT+)z2FuwuaGs#mwv zyt1|S#qD*yx7PJsSJiC|WHAn0Qyzfy(|Z+UG4{$W^~{B=^sekC3t+uEbWHic@uRRg~`|dW@+p z;Cn=pLE4kQVa4-omXmN2FS4N;+0?QjEB19;L@1^lta=#4dH#_~7oi@HQP5-p1%blS zm!$<-^-L;D10pDI2m@0Vm`ggk9k0sdOuQ^|CdcP(}_G$|qdizSrh0EIw$ zzrU6`v>)y)|CK5~SrZM9xUAqIWokk;ayb`~t@ME(^hf*4?P-iKB&Ez^Y>R#nOawx)cL%#9*1C|}`mZ|GgQ2NS-y4QD}8M5c}h&{iJ+F3tlTkV)#wc~cz zP1sjIdGD`N_tw96@btJ%=GR|(;oPXdS*EvdncCjMmMQIUd8TE`Gi>F>QBQ`BerZ#F z=1=Q#e_EfnF)w}2=zfi(pR!DA*E0DTzB4;&TKi`9<=yrveV%^d^SOWj>B7|Ab(4Ph zb<&>t@w;lrVC2*hyV&Q+;X8jDwEN7f+kfk~wyI~gIVACf@8X~Q5)Smp`UNC}>~dB@ z+NJZg`QV}9kL}2V2=};Yz>wI+{` znW>26vlCLILH@6<*Z9X`~k*6Y=4yl0~CObel#c|JH9Nn>zT@L0=zP)xn`}b1o_<;Xk@{`!&>HhHA zTg4xY8g+5VUm@lo#YP<0`OfYPkW7{w=CKt2tVd!xmvdgrk7o5r`LXiZZ%{uN4e$^jxk}jn;L9x4h=?7X<{bm^V6;*_uemxBxIy5m`9kj&Z{T*_Jgyfb zm<4QZ5l=~hLNknng(jlfYuG&=lKFxo>LRW#B==jNxmdr zS;SRol#E#I`YI~Pjb!AN=y)4vAOn@tw|Gv|D7lJ6a_*ep@eC?F(J&G^x=fM;a;Ij@ z%uA9w?)aR*86~R*0Z@`Ra_3_1ovlYnqf#Yrkp)<)6giGu!g(JA0mK|5Ao^Qs@>F?b z(|v4pFvp9(>imFA)Q6a8~GiBY8ZJkXsjm(wNyR{ z5Ai00m=Ukt4Q~`BE&w&tA__@qau-1lr7TgIG*ZnfJ$4e?%G^~O&tcYY88NvXpx#Z6 zdR6YaLUAMBch;j=rC6Bg)`vu5s^w6tS6Gnsi&ByHv~n1X5Od~IS4jWP78=7)OiR*Q z&;%6=Rv#5=T{14gv$X7|)Tmr;xojsLJ0 z`J=clk~G>2v7scMG!k|Y{Ei%!DHjUgK{y)A0Why^2LCQSuO-?ep#V4zCz++&vO)>z za?&9$GtWw#BIb{iO#B9>66xatM!I7JF70LbLj18nZkdSrqlF7Lc}A1pB}q7CW*Cxi z`bV1sqRswujI2cv^7F$S#OPdtUq}@gtrwa;3Kz)Z4?@D?xm;ll{=)oN9jf_Y!oUaB zf~XYMQ%)-I5ysxw4`!@6;A8zKq(+G(uVgr!pA^-C$P?*Yddf*cDr0h92A@`RK=#GV zC-A7xR@#EnC}#8Z@(cfN{iW@~H-)t>)2bP^Rr|JD)Y89D>)l$lWt9MsH}+O5BOcRu zt#pENw=ku9X(dutbb{fnR|cxiPC6a+AU`m|%E!=Z#)XFK;YI$Dk0A(D4c2MP2EyAB zS(hp*LGd)nbS2+j#XUBi$I$2MIT8X3F&%rn(bh)%k;T=u_FNqdofttn0BV>IUBVIG z1*LPiLuM}t3?*b&!vUgx_PHfbrW!v!_!}Ff4c}V(Y@*RQRk-M?0>*)WCI|Q(>pg5T zi=R(7cT6)qlW2T;xq%(tX_=uzvJnCirkUEOnL4DHm?hXg(ZGy5wzNw%b<8q9&+I-3 ziB`hQKzmhATTQm&DCw95!3PtJ7augWco&pCk8#fg72$5$9Gm1QQc7;`rwiXo8J~@G z(Ai+xGn3XOuf#2{#3LVq4|*dX1i!%5Wg)A|yR9ngv9_}3Dr2{02mKcA4_JP*Tc*Cx zsj&*PGi>+ik-JZg++9CvXYJS>wc~cxP1;>Qc^{hgKb;-5_QdEuf!D@8WtraY zv8Mf*5l_yZ@W%Glxd#h#53SE%m7ch0c*vdcPg$lu&a{80U0k0|FV0%=;h_uDcGpeW zQ$JyE{rKH=qj%PzX)guAxNSdR)ulwJ{2Q!!qS~nUk7|7u?Zh zY=u|69c4jDQ+g9oPqJZ{LxQIp4f_xbCgbNj~TytLwn*Yiyyww)Zm|Lo+0XD9!7a>|~%>3i$n z{-OS(V-0UE+x%Wn=Nsdmu}tgOG#MfhG-Ig27M|6pUDY_7?Q_22 zhOK0q#6BIGCwE{6THfxstN)XIMooVE$CK~>c#4%jd2j6$)?w}cZQ}m3qxYPCb3=9i z^q>30Z+#NR-}b#)o3&fDBc!f!st>Bm}oFWQxhQ(I+vOHFv?F-ZBY2ppUekuHGULKc!-h+ zH%zd-^38yj+t=CF>kFo#&QYanQ}e+cDInQ{Y$R@vTT}pUs&nH9eod!grt#iSYauVk zL|Y!#khtEvo(+@jFMrYZzrzQoIkT+*87FE)nUuL0O1?1s>AOGV0w3J8l<4RG89w;) zwuXsRjX%8hR(x>bbf_d|YKEFjb!I1eSfhvPtzXhP%VXgYWF!TR$08}m zGYT@u7vfoR4F&bX%OpxjLP(>8r0nX*vsl7D#s7DKLGiZ*0i#jg77|n{aSRt5<@M4g zR3q|B_7NFBh@z6yG^ITuS|`&4lRT5Imn4ByG4D8@e;M4g7x93S-fq04eJIo^`R;*?tK#9@}0l~3cNaTq{?ldlq$jC^dR-;=!v z#B3)c1E%p*#o(fwDncKoi16we+|!v96KKMbmmhZuqd_kvI>7pHS7HoGh-SQy4sf8B zWO7IrED|9C?dg!JcvbWIi!d)RTINw=Q1O(^y(uM_Bc%o^K5!hEM%G<1c~&~$Q4Yd2 zn8#`qPSAEDJfcGi564Ca($3^xEIrO7rUcm_J8ud~2$MKeWfv6?W%iFQ_MT_-kA@+c zFXk9w&4E1<;8u(=TY%qT`Vr!HzB~}2fh{t^pl=K(iL56lCuP>RxGy$g@ImnW^_!!k zV5}6Eka{hOIS4Egoto)%a`7h@<9SMmL+Xpzcj_5n-U!cyVJl#W>xYoBt2Gd81<|VU z2MVpDeaTUJfEexbaXya!{l#}bia)6COJ$b}QUg78ChWO&UJhyrE?!G3R@6*_m{4O; zFjl5ubFD1*bY+taEI?$C*OU+6 zS@+f9GmCT=lS?jTmp0~>UCynzoL70dpz6xXipx1AjVUFUzBe~SpE%E~)E75axfhhW zW|VZ!G(VLL`j6)_0T()Eme^saM42J?*)&th<-3Q@-s+HUQWEB3FN8Dzyq?uAfV4&t zre&!>2gOm%0Y2!0bj&@k)FZFdtDw{e*%?ZI%7Y*xe_5{$bpi7?j~L%QEv@gug#MxP zhyEjC+UW28Juqy3zq$D@CmtBMrg+%yQ_QrVvj5DqJ!snRtABf6{YQr{4xE?!MNhkX z6Q8k6eVl1O`I)jwPCJ)`>JIJu<%jJ%BR@Pn{yEE(_G;6_ObMnU@>_7ef z-jmFW6sz6&{0W*}{=v^iQaoCAagYs$TOIoqwWt`3V zqr^v42e#hjdlMg%NF!RU0ctg4`dF#5+F)-p{h;m^6k(8h2a$9{N>gxQHp#8c^u+K7PHHN# zopUjB7K&QojPONW6T3_$&xxINj6B-P!)J>52b}6L+p2ga_L6M0VRB-h*^uD~_+V~{ zXC5Rc@hK?vTZIt{fTopoTU*|tLDMVdKkBzdCaC)Gg+}L$UQA{B-W^ zqYdMC|N3Um@d5Gc-wIzmJ>tvA_^GQ_jr{r0kh7I@B5g6nP1xQs#pIA;B4-nD zF-|p&*aOIAtG!iuSTe_dj1*jazweCh7#TMzC$ zvW}bfhks4YEseJx)i&HaR(oUhiA!OdPmW3~>lmR8Lj)ElR9c-N21iG}NKxb)%WE>S_@s!ixhAYcEowR4YS#hHsXX%+8db$7rWfyt#pdRPea z33bSdLzV~+|F}*^OT7sm#pACB3SA);MYRf!)Vj~oy|?r*+@gyoFTLx9QTF~X;Ddh; zu<|2Dx!Q1ozoxh!~5UV4{o_Qu5-gw$H!c|8g91YgK<->G8D60J>KSO zCvbD_a6T1QDXy~ILJaKy`qS7#-vEPVt7W%gsJZ zjPB_7>cf4XWvXT2ON(`Ok2i|E!+MSdc|J`&Pq4dK20%ow*ri5}=kQuwmy%O3I5qPq zgmIYdM0gg{l}bl!avz3!I8RTzanWL&6gi@2u_&KA_Qr{9EyDSlM;M%^bBfY>p&uWx zIlyg3lp^XWVGF-q3q(?sdECF7M|Ff~jQqD_U%>3AO$+q$Cesf=-r{U=S*W**)4L%y zbyDMs1uPW2nk82iTRH4b}bRPR)n?`ezK)T;Yp1#ACW40#c@99PcAUZ z%ZuJe!YL+9r+CMNYyHEtezdUf;L@uNk&{g7*NXZ*r5MuI4liNon;fC6r+{Ld--T*} zBOn3DyP1`FM=o?@@1WVjc`!nS60Eb1C?jKKy+RqpA{>@gQOxi_#hcQ9mbP1@-hZ|> zm})XxEA&u`V=`PJKRguDCM;b@t=Q)QZE zlt4&`48tn!>;f!|d;lpSX`)G>J+qv(g^@O%ox<}uq2q9;WMh{Uz!r9-#YicFP7kL$QA38pB)ef*-i_7M0W_Lgi7+mpHl zwi|WnG$Da1KOKpn=|B)8NApx9|6qjk7gg*qcGejxNlw5>G_^B>S$ao6c*- z3BQ>|kIyU8%At6+^}qNT94ef7Rmn+s4srAa=HFFr$ewUUY*wMxxk{8qt)lrH->dh* zhOF{97A>9cz{>ZTT@(~9H>o_1L&ZdeIvk8p)&q0y`MVtr-w~7F+jyS|B#-3gZmxS%YB*Ncw2YfQF*}qQc_|ID_k#tE3 zvlL6*Ax-heVg1{H5Lab1P*ja z5wSvFux^;d__}c7_s7gVR=DZX-R*VPm*_4|*!GKd-Q!Q5i!%wpH9Vvo#Xj~pfMsQ=2+ zfK{a!e6YOxs`6fID|!@G^`9B`%K>KEk8HU&w)w`yYo{lbX(#R2Iwc`-`X@2}92F5g za9;Mnl>H+&RZKf{{)598KG^@;`}^uX{?FOhW-a})xBcZYPlIVcnVa?sBAoQ}wV_W` z4gPEGTYtGToSF9RXp`EsZ<^db_r+&k9RJSyJF5Qv{F8!ecvf1(?V}LGLe^Y%?hy;^e_eYAxYh zZ=Ye34G+XiW3H6^@x++6hj1+oYIdkCttQjO@r)Y9Qe~L{H-lQVK80(gU zWOS4^^Q_PhL@NHm63gh}|V;T^tjLnD!23XG`(7;9={zaK><;7~1$qekZ zo&eOPXm`Q)+cwKgAWtJ4d#Q!ns=5yvPtYdEM7T9X$0<@F-^6hkNiO+MLO&0C2NPoB zn=&|Y!)lm$l}N{zWfs~xf^mFjO(rjy|JX|PIH|PF;*k;Pa-3E`r^(^2Je*qhS-2dES`P5?h!18Ev z^PLfvmbV&z>;CSS1N;~5>$9b<=jyT^1!Y|eN<&tb2CgjiFMu$MK6xcxImNC8r7rm; z&UwWS`Jf+Uy*4}mc;D!!mtW)!w@)|Ovh^9}-)=tGf9l55Ngy@vm}(4MRrc=g`lY%H z#f^7wJZNhBIyrOe!O?TKbQf;3?+te!C7CZR(OuYC59H<4jR!f!jq59~>?^-ke(uhl z2d!b^{YTB`uRqvgyd1jq*O5u3evw+2?~Xftdwl4M61H{jj#KtCehG%*oz_26>jyr_ zTC%U$Ti+j7(k6e3^b{$U2=OnpYOQ~S79$3V_2GOy{}RsE2Z}HJ*a+*=9=8NWsLB+d zG>8cZ7eg6w7SYS7$`^%rhihM&d;ISGhn9Oc8$a&aI`zl@G#`8m_~59ruO_HrE$9H>~LZ z03ZNKL_t(cYk6>&mwCRP6~)TDdvMMDU$!@$t>q)0ovD{5{fc!VbG5uyaBr2cOZ0U7`;ogFEbqaR34BMI3gGJnTH@Fzx9CkpcN8 z7YIann0-3IR@evNAp184zd#T}DhIw$2}BYd=0Q0S$1n=k<5moMi2FFqJ4#h@#?p8= z!g&~c5Q}xhns~%%edg)B=P}a`tWEcL1N-7Vp8`nfkdf)#7eS~gk44D-78v}Ont}kv z!Ago%=eoq;8V_3mEV4CAjP?t30}Cpenp@skRb@M0=aE3!GxXkbbzTef6g|)#gVsR+ zJBVbb_lVaMUL^Jhr(0rhjMELytI(gjefvSv*!)VnIGs0^gGQ;IQ?Pi9_qPP%0J<$U zxFY$Ynp1ET@qlWldy2!pJ)>{zG__|@v+ z&aiTDA1cHH55b#;IC3H9aqjUvIw5ykUJP|}#~r*d4x`%=9()kVrrtBbXg62;L1E3- z;>%rEm|PaY74Z`-A%9-Pd~@MPfsz-&ttKAlj<*i6BSmt=7?D^*4AH2UXQa!gO>WDK zcJp-;*Hmw+xZFFf#D(4S?@f-ey3x56SMN8){&3nhR_h576=8o*TrkKW+%BpNs{_#C zg`AU8$Ve*Rbt(UG*?0da{#FW9zg)t9|5XyH?H;4=IoHsWea7g!Md`g}7dg%N#Xqbl zNWNAg_#1>?cI++VuYUDJ@YrF>OD#g+%Bazy4+J_EYR}?G>%}n+<0nM(Hmj zj?t$GNye9O@deihh`a|FvXF-}Q@`ot{%n0nxX$gH%>Nb7(lB^%i%VnO^DzdIt8vr7m28NMf(G|V!|;6eSsGRJ{v za`B~^ht52gWN=7_;j}$)F^JV=kKzAd2%2spJC=R1Pvvm~iM3HRfFmGH19X_i2T{bu zDCnUBA}~InO*7Nb3j>HFPn3?&X(p#cBSb>P3xunnc!+Fkve7BiTv&be!ykTapJ;H* zEVfBEx#yIWU%9uj?&|YcF}h^u2C{`cFsxHzi`P6j)(ayH0$7AaOMRLjYnV}5$X|?y!2U7(aELsJ zkq3OSXJd_>c7V?qh(8FWVd@gLlC^ONK{y8vf&Jly!uw~o6Ls9|NdV5!N5?fSb>37@J=hKnSZFZYougJ~bKro8(aFzx$ntm?6< zqSr?=c5Ucou}o~a^N!`g#}>=iVA?lDJ-i-yx+e6{ff)&@|9U^-t3k8kUSG0t!sgO{ z9J}=C&lf*GdVa{Fl_UEFuYd8$#&I2(Y2U0i?Wg{cX@7T8hZAo+_0Jx513sPo(Vn`0 z9lrSSk%noze|;)RIrPZ27pV__JFLpIILM%VPP&((=CL(WK@(!!4G# zPL_p?{&ZNtier5@G1I=ZYhgL@L3VY%1!X=3rCuvbJabFDGJ%k~WEVSSnVneInu`1m zP_l^-4cyE${$2Wsa(h2Z!8kVAA^>GNU&9~iboo>hp*C?mg-phlXpfL*NmNDb$Obpr zar`F;V;XVHsExpb$R==R7XUjI7fR3Tti4x3%BpaEM1t4p&A>vLoU< zs18?eYKsD3@D`!;RO;QtjIXN8lH)Eh%zMIKh~_xg6DdSU#K%c(ril8NB^*y~61ZQU%nwkY@j6P52qJ_~}Hk;yDmMp*Xx)_HptV z;=_nElMPS3BGOajh$}^~D0PKOCX+1T7A<10VpCVBB0PDSH1#TGY=pFp;x(vQr}9io z@S3UA94AlAym!umSbI)6A~^{J<16vaDe=KchM@eCz%}K(6Dh3dwZ5X?=E}ZX>VrO; zo4mZQ<-t_TqmP?zecpU$M$6rB%l$Y@%QA~4&0@)UbUUl2EFmv%=BKki8#*I;?r@Y_Ub?ORloZ2cO$xcEbP@`=Nr$R82Vh*$WCX*bi6#Si(TR@7np5*Ipa-{e~~JS3f#*X8fw32gjt1|MKgYh5tOTZ}!#m zi!6`6w^$ZiTIO0FhPK@OviZ(O&9|mE-JWQ%j6PZV!njY~3Qjoq()M3_7nb*2SJ7it zdAF6NAqDK}N+C``0md=ShwM&nIlu=gMTvcuX!#Th6ahaE!7I@|8y{!F3nuF5S=RLr z+fwtvk0&?QT)lYbQQNn|mbz?CJTk4!<+~GGOqVws8$VrL6SwELx=VMdD1{Y1Y(7zcGx5OL{)>~2*u510Kryb~D3}uKxsa5SfQXzm!*AU6+VPFLBTmr(? z;|wLo2#ap~p^EBPv?*VZ*pT$)EIX z_l8M!SHF9s<<5=fE9V>k?%z5O3F-2_?hmj(8&0(#N?oVh8x~todk7hL zo13kchcLXjvvtK|g8cy^?9%v2w_B^{KCC=$A*d(*TJGI`SZlbuFXw9X7_0gaAN(k| z`P#*n+t)D7h|0ehsn+E$`rcj}_pt2emOD4C%Di*qzW#^nOFn?iB;KsHZl2};pWz_fZeMR|YI-yNfNkUv*ExqNB?zE5q0s{#hq68b zf~W^POi4(H56Yw>E>WYd8}mMIosa5f=(IVk}n_ z%8Vr>PIQHn!dACMU~Li{qzLz1HzzVzNCpQ?YO%q2zHV3L)gP*^xz5)C{90nPi`S1Y ztp54b&0)C}uFQ;te8+m1I9>1L;+yxIMrV}U$LZWL;Rb}mXQx=qjxzYgaUmNnz@EsY zB+y1^4}4%38_FB*Ub_A8s~x{~PcXU1=_ziYSAx+CV$?CmQww#W_Y&jtbBeUk#OFg|QmOyySW3|~E<7L3*%skwgn z)}ss8AF}1_)%%9uZY|jTTlYCSuh6342%cj^+P!RHD}zvOA1w9hinvA_7zJS~g4k}6 zumdcSI{5}<-(9{r{_VQD^Vc49W3MxIib$Q$?4rc|XSWy{Ln3t{D9q<@K@PQ5tdMQ( ziu(kI!_Kf!gzr$cMvI5zNyC*)2y6$}8OOg~0~0f|*4I0KE%+er5E2H3P*ph%nXP3; zf;|c4GRzP=pBd`uW+$#FBsyk-IqYOWQKY0}=B0D8$^Lu8lH(WF)Leo1`WfI^;<**3_Ccdn4Hppn%N7)$;QC8DMn@lvKzI` zc=2!F>pYT7wy7qsoRW*TA0GVehD{2I1fjw%Nq81^)%=Vm#|(mZlpTe$QIc#xDC#UT z+rcG;qX&mHz6rmWxUW*W**VcjimKd_#zbR-9lI4GX$ggNbOBFofR8SyRfPMWXtYb` zCuR4I-B%uK@Omsgq%0b@PePc8`S!AsIiNbNu%&JtFlXq0nkV1pW(F!@=3m1Ym2Jt{Zsi zf&j2vxKQ@VTq39zq}{q=lTkn+GnHo*^@`+YbokZti7Z8A%FQx-f;2Q zh1>U=6f29qow@m7vF^g(51iasdv$%yRhta6YkukBbGO*0-6yX9m>=h3>Fs~`S&vis`tYxkRS z%P&8}>Y2y$uWD499*vSl!%XQlxDzDa4H5T@tBy37h{Fj@dlz&K^vEw^roDGTsUNxq z`mZeuT2mIXw!HgV&=2-0tay22Ro``WAs>aV&V9+!^mfaGkDG7*tNG5iE%zcV56HAn zXQusw8<`d6C7IbX{yy`w!87NNO4<4DuF5ZU*FV@<`pUbXkL~7~@Ir?@Z+0#k_FT0v z?Qc$0O#92@I54Mj9c>0 zqd$e+xEOC~V&{#ww8UB-&2G8-@8;Y8XudVA`Sy5=Wn_h}=dh2*bx-`|<;`_{)|B-w zC}*ZUyCT#Nmin)PFpKPZJ@ZLFSnQT*cFs0C>%Ml{`_8Owl8e6h}#B%s^Yr zMpiWBXk{r&^dA(ee3#>>KrzZv&rAfUiJ@E{(RwhZ6dpzlLDi*%w;8sDs|H1?iCeim zAye+!2;X6iaM6{HvHWpQHw#9qf_Dl=b>%XrTom zRYEyPu4^+b%bTkD?>H6oUB-9OgByOEVzGSPe3x}15tjS&E%%qS+*{Fd zFWvH}z|ykWV%gjDV0%?@dQNWC7tymPL@%C_zVoyFzrMdoH|Xow7lM37`)j6#*niaB z?wekk*}XO4eQcxp*hciR{kE6wKe}lq1>3*jqv<_p==f#pKl`EX!%c><3$v$3M8__k z`Qz^SS1;sREbJ78mZl8L!xfg6#g>+NmPes2cfMeqNXx_bERQDa-}TbKkKXT|e0;$6 zlP|6*@4Kerg*D|pRzpnIAoQO0=Q?EIgY4QpvY|6|hUCH~`z)oQ(%VkN#y0U7I`}GL zgSn20hOygV1b_QMQxnoVdI8*d)V%lC8{Zu{W4?H2P1TilDaOC=KlyBmY1ZL02Y$QR zG1<8Pw;S}5$SS$yoMgIwzlr|+c=t)~?2=U_jhhXPh1yFVp+#SBs2i77J}#wff5o*A z^Q)I1I9qb&_QNLST-WZvu?H)!d4`_gnIEM>kV7{e|ALzWBKeSlGen|tJY}RDE{HKf zqyp~(M4RV@hpTM4@HQQU7&g(orBWGnrz86jsdE&g7FvlFv+k8}JsTLge)IFHAIjN< z-`bboT4DD;yu9Py_)&1UV$F{a>=`KDOxm#<}yS6%b-<7%%*(Rp-}Eoj)u^$2-4s`;+Z^_s*{! zAIQ|fL4+R*K(!DR^25>|uk}3vZ}#|X?Ar1yZi#LCCL@q0d=SX{+y;vMe#yT|K#a?@ zM74K$tnKl89TnTP^#$~?<0C%w^S@=}eNKAg&&oI4P_w^s{5SXhjq`&S;;)us!1RG` z0f7Yra{7z-&{u_QqjOk^XGL;nnv3{hO1>q9p)7nPR?s2lC%C4tVRF=^U$=i!c;D<_#gDJBh%FIt}6G1y*oXch> z8x>2^@7j^1l0Luj7v{49U8Y)>BPVqw|jyS0?z@s|4UmQs} zf|-mMZi7}uGW-a7VT8miRI(skBG}1T#xe_nDH}G-8T@v;NX95gY`%lB2&WeXiOs?# zM&Xnoma)ajeAz6+F#su|{^W&9%mOLXAc`MM%?qI-i(olwO}Yx*lw&mXJYifC<|D}; z3N$?o&+$Vjqc91&%|)ma!e|HXXVS%1&kvJWMACvNdSM8ZZx$f73a6b&sC5iLv?d{v zt(WRijT>`H__09zM3tg*_;FH80uYcq$k9@CM}i1i5J@ixr%WRxi$bB>QE%8su{=S1 zM}SvaM8P>G;izLJCegJ04k^1>HGAnuzY4{@abTv4@j>LfHua1t8V7$qP`S0BI`Cv& zp`-)76Kgw0H^wO)d?~kJS~u%OF0_lEMBpj3+$yr#Dzx+vx`HcC>)%rW5(jUw#TKE9 zA7$kwvh);Mp));&uKw)I82RlEUg8f^OZMGqb0>^Ke~Hnrch4!h8oS5Le;4w87djK_ zA`!vx*XK3`@Gd~&>_b^>7P@#-c3VZL0k)nZD-V&I54F@!;`^p$B_MZocsC5+4cbVep4D6bm!t4<||=PnJHMB%Plk<6{;c ziOD5U9ZdhQs-R_(+$vfAPz-glqNlEZ{I7eM|Bk1PGr_|&#eVp`{_&GlJ&){W=ImzX zCQ2XM!x$yYEO3CKAer@Q_?dEx3^|(KJe zgK}Gf4@%qq63^HrL8@Y7)DLO$MaeRLD!%bFIEWfH4;7Ure=MGvM{X)fhSdp4Q`zWd zi%f+v>ceE|qbbtIl9<4!|DjmPqq`V> zikxK|q$?gq{gxy(PL@5si!n~eIiPXmqXlLo&eBylQC{!L=K^}g&!r@CKPy6$heB<|=QQ1`Vr0&K5@#&1;EZ!06LujH ztibg`h_NG@>Ftogl=w!o@y~bBh_Lk#S#1`ghkuD5i}Woq$YSi^DYha5=Xas&cFJYD z!~*Sj!^v(0aRcxjWJ3j8;52lyH5zofLi{A?s#OFZybk!_3e+%fivK&Dyu^SrxCF-_ zTcJWcMFKR6w^PpE5;W@3$*e?xkA(MIv8HM0PDP*5&v#KX&=E9vU3{pOK`cJF=$E_b zavMDSQInPg7|emW!ETSNkcSYRW8o>Z@e;Wkl6tT$LZjYho5;nNg3~-jmRp5(+r{oW zpWhEwL9Dll&MLYFN2its(Jua!n*kuGpGfN#-E|A_7)f*q#QecP*7mw`=}mO=d~^hj zSsu>7!=+pxFbYd|B|a&H2IDAxmhOi}N){`e$E9SIbCGad8u2VPexEq1#LeddKM6;N z&<6D=0U`Xd#H$QBiTVe)CyS*UvkF}->x$#+5D<-|T27JAPnQ)`_MqRWe|*YtMZB?p zJeg_RO|>=+O}uopeC2-4hksNfcTe0l^vQ+VwMWZY0C7gU>~N{CsF{3h7#Qd0D8IZ^ zd%U9OR88*>HyTil=v(7#MTVwhhot4JtM$kqY#p93q>r>?Z1Jlrb&saX%yUU{6&7#O z0R^1%u8&P*gDJV&(Rr)@AY>^7nR2Uag>{b7CJ&Mq+aW(UU+uDAy?DRc4Y|OF08K;g z@S1}q$hCjEu=M$3uA%@BgH! z=M$#iEI0dL3I7F0{!@Q@?}Lo9zuc(*>L|4-D&s4^!027uZ(fNW=-h{0`@`Bv&b6No z2Yhh&pVLF%YA3&(9@})P@agBj`QgcQX8p11=MHF|It;G;6Z`5K3O zjr~5AeU8dDS80t?Ih(UOgL=X^iGL@Lv<}oW%TgMkD})TFWV3=~D+!)RctYgqBCxeS z#5)d4~AiVng^(XDbW{&aCgx(Ub2(_XsEPPw>G9xzYyAXm?#0grDRgf}pW? zh3Szbmk2L|-7z7VNwV3&j9#dw+1x|V(Fsj;*?6e_s(diQGrF;nLnmSk4%r{p}9nNdqhQvXJyGhJDPr)ncS?^ z{@UBTY3J_sYdpiBJyO2@O!adGCC?rzd-_o6n*7pLkh7^|=>dQUogqbu+9g}XE;8&@ zS?8(*xhm8ryd3sNArB!A!@>QDe>L4GMQ5VP_XlUHIYU1~qKStlr$0Vh^JK2ZeV^v{ z>y0}lt;R`m^cwi>dc%u_5A+mS-Rh?7QGy8>YE5oFcx`F5m4FM->OIwDW{UtB%RBqg*d@D)a)(`I( zbQ-}WPe?m$w~Loy$j=pfDXxKBpfTDf2Vvu;1q1}tVS)x2965RMY*5eD|9(=jezrlZ z((JVb{Z|hfCPNyt&W;2;aAbBMvtbxAnq66X(OXg=o$h379iT=3iq!UhU_(`ZYyr`% z{}M8@0UqB+=Ch)G*=sHFwf}ni^npwC#agqd=*JfAAM)EjFoO-%0#{kVSN{s3V@v*1 z*wKb7Q_!I+^FtoXL;T;Qx|AQNd4;m5i!My9r zh1;*$2C-qh)&YRcu^3#?ZOb5*CkPS5f!r3%?y$%4SJpu!Dp`L%+DD~X2a@Pu4DbbV z%tQh{1`{qJyH6^$=FZV?_$v~{EZ~k<26HXMY($$_Ac=51KH!?!aH@HnupmPScSOLB z7t4t64a%B^tiu3l5`@w394x<4*N;r&Np*LQr2X5Y)#eejJ&vWdiUP4hYfPQ9iIZAH zGhZL6Jfi44SK71ddeiDer63&61lhY75>pJ70@$=eYO#Y^m#H}{??hdFyKW%;ZtLGN zOKbxrBw9A|QsXxTRV$N}`=lMeovt;Bq=8NtEwv4&Z6cXpkJlWNb)GNnNxIYWLW)`t zAaNwZd_=r2R%#kazn)o=A#S@++H*+J`E@~+c{F2#WeQ0H2)U00(e!dq@5SadQys8)}^b> zaWm{ArHjL*#zE9J%pcsCU1AwZ1J5yvS(%`SD{gr!r_=zTKwrNi956>))RNuu_`5Cd z<(03`(B#WIE>!d;3tQJ^s4YSzj*x+xAtBGGcGQMf_LgQ)Z5K*lvQRDQeBW*}5TC;pj_oQn_p>F?j605+`p8 zwJqgZ^M)A34|^;Akae9`bcda)Ulqt$dx$nfD{_lkuBv@-*^(!o*-#+$yAwSz*mBF zWgOl1`iV+3|IAl+eSErlUXsi_Q;C{nnjjNoDm||^9xLlUUeV)G)VL^B{_T}I-=cf` z6kG-B+ho~0XR2eVR_jcq8CGmW=b+ha(v8*^cPRd`ui})d>zt-1@_ggUNSPg)@`cil zB%yf_{o|49@QaPehja*M&`7inlDv~%l6bxOiCFpXCu+~By3eS(z5i%%jQ~+mvh;#R zt7{0vCNGX){7%)KmUo>|cKe*Kw+W?xFQ^VXQ|}ZEGJ-g<2|$4$DTwR-sa-5J+g zUfH2QgJP*pa}aOn$QS9Q`Ni!QmEGAlTb_@QZH!T*qahZpAo7KPC;CZjw~AhgRi<5U zzNG3tF6rEQsM^h2;(`YYj29g~c(YlsS?KIdy&S8^xz&mW*J4iJ5Dj66ZQ{>KT`ION zbQ&sieT)JiLSM=90D8~mdtF15$RK3hY{|IMvL;08>LYRWp;qGZ**g+eZPE+8%X^97S zwQ<08zh*;WxoeIZSu?aBC~WQ@pIVqEzueHDThe8ds~Vr4Iabvp?;LrmueT2<7n+D%MUxVyhB-Q#*BJ;-SD!Uw2w4@cBm+2W@ zOxWA!s^?|M&2v4JWX`QP?j;$?r z?HvzjocDukzcf#cT;OGgfOGfcq0%P{OP@YkzP6xj-HEdGe>AL!I{umGCsfgU+8OVe z$&i`h2<`YT?O3vQJWD%qU}n5vdhEo^)CH{;IbV{|{u>I(!OWbP9|D6uPdM^X#<`74 zZT(go|Gbj-)iU0Dde@$hTzll{|6@6C`;)xy+!wz3tKV1quYKkp5xpnu#?`FuwsYE< zJ6i2c?abNf(L>m^&+2FitmHpD)`jd8gi&pzfOU>JXQ5&DwylX>V`G(__&mSy# z_DI>21*K0MDP3`>bm_qow?idB{W$>HM;-TR?Dnhe_Nr~Ol~%c+^Jtz;QVXy_fvg}e zNic;qD^0Rlc#8G+vsl=K?ffj3v&rYs6&A4K81$xD+%JLwJ~jk4lM{Q6r(mep>eYcrYzay-3V7+H8A>$q(@7Axamrwe{q=6LVYuF`fC(Fn_4V6s{L z&ISDH7=JjH8+RocIea|?rVpnzCCYKSy7U3$O7ctS)c3d_UO^OFNZtt@V5blJ%{BlE zF>1*v59(A8^eD1EHaScGNSSfmR1<@A%sMed6BdeMF-v5@d5Rc{p#g;>gD;G6^Bb# zA1+-}Q1G7M|X-ccrXtm{oy|k1%{lzaoeY@23cV3i79Cn|d~n^ub*tQWrR}b-x-m0Vs@0Zhwe-xy&FQfV+R4*1V+Goo{7dJz zMh1T$=Kph`_qV%Ze#%OF`|zPxKHmD(>gba%ovz(*s&d`2vbD#`pF0fr;8O=+kzv_E z&9Z$OWFnEFblt0V%~m_^RoUkfd=Su5{$3UzB-k!~Y`Ddo3=bKQB=#Z6A&jwOC~-PgnImzMFaaV9DZxB~KhGdFfCY zn)5jy(4g7prZZJmIZD*wpI@k5AFtegv}RR=l&&9K9x4q#*Py&NWaBONK3;p2?)Wmd ztb1sZtY5*V)&6>@%4WOR+FRrjh{HwlWZkPwMBDr>V`{Fzn$cUl`^;jSnA zga65V6#qAD0N!!i|IKCoe|rNU*rM}0x2f(nWDh=0zG38h+YY1yxqKiP0kp*6AcO!1 z-?9ug$Qi$B=lM%FIeJ)v-j$fS2&;xqv)NtVK~d1hptxjjZe)q^o>28p#~rf zll9m{C>RlQjFIvq=}mcMW$j>l@AZu-n}s` zVYHtw?XU)=MI0d;!{%eLi6aVnEU0H5N$)9wTj(5`xK-U(-#6CSH=Zwp7r_TdDovwk z$2gf)u;kS=4RRhzT85kY$3^u6!g`41Z|WU?K0|F0#bC-LbgXq0gDS;$H-59Wzjbgz z(K38Q1xgfaGSwE)EdUF$Vap`47*qcoz+-2QE8xtk&QWx2Csw@(y4+wMCUM01Pi({K zzpE*FQEn-9I+9X>XZT0Obhb)lxHmYWsO`@?Xs;u?&$o;otX# zTuA#Ea`oQA5Xvb|X0c1^lB_^Qsal8YyGNz>hHllMVxXvZ4wMOkfi4QqOth32!FFt8 z|M>ZeUbbW9olfkuQg7`I>E!I3l@r8q!EdydzVY+QZaXjW(jdk?kX{}{n|lbq&Ows| z?e?R!f^DLuI9wO`;cvyO@AZtK$B(WZVCoId8U zCT^DYYMMYD_-0w}(ja)X-4}yH*hxnv9itP|Xj3*0DH?{}j#oPpE|1|nn8*lj6aAQ1 zF+DS*XdGUH(Ar%GhWd;g7*7{C#R-oe%V)Pw-q%E!VGOZMc?lQ z-TGhmRoZ%pmif^nz7m~`4&JHgqiW#ExmMCcrrPKd>p<$A^pb(msqVoE)Wfy?W9Wq6 zuQkj|mYI`fbu0*oW&UuY4~!4ymvy00fHqnCFw-`?uCUw)YvGw^C~JDhDtbq&`o$?!ooT~= zLO-0JEc3q$eAu?ZiJLY3)xFT#gXP_*kvs^n0&u}JSytRU*fu;-(LGw$J$kvezXLza zUN`S8NRSF}wiDAdxpg-12a_aikH$!>kXw}oJj(XrNi^M>nDkA-my!yeGUB};<1ZvU6=$nVxFR~OjRt5m%e_iqH}a|aD3`gb)T>ao*BAr7CC2boPHV?YxsZG*A)zKkKH9b9ZyLqqAJPbg(VX$>z;(BGDx&_AIjq+YJ*sVFG1)EU%$sKa$9+X+qK61CJe|U82dReci zZlJzv)H#%K-DKbH*NvIZi!{FV@zVXSK{*lQKk~J72qzQS@aho{l3h4864NRa> zfQB;~;77#m=*X8DW%eF$=2Ab}&QtVrUIiMVU4s)>H9e|(c+i#fF88MaWH-PEvx{0x zf4Tc~h_tL_czRmL{0!Eq`MPm-G}IpLpvepAupCvaox)oJ%L5)7?^#8VPBU4dl!=09mrxm_HJsOwtOmm_W8t zcQVeDq9BgE0LHI;VWRZ01m@dIby_YMi{OKsZ{OqPD*Go0^+Nl>`I_gCmbZ;eqQ*o} zEn99jK9t0~b+XbVS@!N9)lPXJUXMIShkVWZ7iu0&m3@7&cIEz(EjJtN^HgC}ORBPC zZi-CNG4jt_ji^;eE4sHA-P?mlG^&nNA#5G8+p9uWdO@0uk8`J(*FK5hgWB;z?c^zN?I*5IPZViq zSl7OSdDyFBqSGX%Y4Mg37n%H)7Xb@#g&NuNW0>Saf8)aptoN zUwn3TZhrc`+9K^VdS#btXCyNd*RgAVN;`36W_<7Q!r!7oe+u^dA;|Z;J+VLLWV~_c z(5ss~eq0lG;g!>MFCQ&`p|Et_;j-rrm8~f#TUAiH0y83)Oxam#dkL$hqIMm=1}KKbbJb`GIgKiXg!Rp;RATx{+u1MM$e zSn;1>h>P!Y%zf`q6`=Fe7aPbo-$z1RZ0Pm>sa*1G_0@-X8K8CyV;JCY)E#mn>kG%T z1EN{s@PZ{{COzx^owp(;v1a;w!hod>5!*w^#-V58|5MP|7 z5bOmJQ=43+U7pf@zsli&+A$w=3f%G`kH@luniWS%o;*~t=19pig{5l?%a9S-aJ+ou z(XuyARK9tx>6zV^Uism>qes3R8;H_sGiD~TXU6tvCs=$?J8@Mzb!%o)G(AphXG*o& zT20m6Z@&5<{>gbaUR|XA+k(*#I2}`V198*g59UbUT#){}vHxnL14nYS+McNi)%1i^ zJ1Npm-PBH81S2wDpq<+H-9I+Qg-7~(Z3*_-7Vhs?kdv|DjlVyYa`){E%`crOdl93V z&mAp$rm*a(Bc)H~m#jL7jmSaGk~}aX$Xq(-shswz9CDNpUkv!5lD`)xsNtk12}7QT zpC~>HA}i*n%H}0XS05_LQFh#H8rUOiBlsYB#~@E>*?!H}S3n)_yKD96jq>WLN{78F z#~fvnto=^YV3e>WPt{RS(JgElyxTk|ZXFV}4AHH_bo=o6y1v5|-LZ7*L8haoZRG7F z^~-SzCx2?^rN&`y(8-Tt#RUEJ zqJE&<6vRankg#7r!WPug2H8vQBW_?okkSqBS(HFuY$@O)r860>vzdcBiBrQR4mUq^ zK6U?Te%yb**=EHT<~ro~Vx1<`R^h(0Qgr(xvKx?N_dhV!f42YG?F+lMzb_a#7>S-q zO=*?2-xUD>a2N*+x@8$yWF2tJDiG}#L8g&VykYe~5D*N3)2suEEQ5=z0%2n>41sGc zLSbXe;f`COa&RH$;cy(4EFk3s-4c+LBRI#1c{pxE5EKZ-q6-6{d~~Ko7}tg%TuTUC zNG=y}O|p%KY#R;{w0$J-L?B2l6x8b-!)TIx!!{PD;E1G2LK5>Bnrax3H4RxrLoyS` z9a7^!@wZ1HzWDvTGNWM12CMklMKf3zpSHz$HY{Tq_c$Pr9F(*#^c9(gNnXzc5GqO7 zx*%NQ03pnbEzWQP=}k~|5sX6&V;&)SDiLsjz)Owud_*`CF=G=0YFQ=N?~S^4&pB2~ zMDCoB!|W%{62iLu=Y^GKp_E0K#A%20T2r_?93i^X9q49JduS40jJ$CRU-1vefJ>fFwY!AURv{98EbV)> z0hwszI{y7oIWG!T$Jhi?>oe5o9nmu~`Rd*hvk;2M)p%UfWfma;#lT%M(=aONGI)$T zikhq6w|&4po?jP>W*d=d(5aV^gB~0qby(qIZgw zngofrp9lBpWuia0Q;IrdUA7t>@w-^RC{*mQTWS?eyYH6aqO_kBR4j^+peoj7sZmkF z#sLBTeI1} ziSSjF8y9!nJ3M(-(Ph0&gdAp<0Fdgh=@_Z+8e1QyKu)-=m-yA)%I4njs`im(K@fMm zGzb_s+mC`uy||_y`SZ43B6NA^DQKAgiCRk!5oi(mq1PXN1LF4}a_KERg%{P`4PB!S z+r^Gv;>Erc0WrC0F@-ktaS3JyeU%NrZ%zBi`e>QWR*{XT$p2^!@;-;gree=Gpd+?h zMNfq=6|KXK-DAs8IXLv#cAMxMeBlp!%WXY`%l##`p2Dv);fUvdyJ^3I{rsimO|-%o5GL5u4K?hQ?tqggbW%s?^_8Jjl%!*=HBiboQe zGc^Dv8YRm}KrR4t8FGtkMM*b66+d3Be>_DxFInoEr>^WBtLYiDOqTPKK{#)IBA{vL z95fd)O_m{-dtrj~w=00R`rNuVcMl^-$8%S1BQtYBhJ0=U^YK|2FX^fdUW&|Qw{&5O zY)cV5^D*L?SCcoP!STh3 zblW34BvxnwjO%LbA3t8vGbf1w8Nm$2oJ47^8b(37vcouDY7{R;g&(izMn~2alr2b+ zqNef^WsVuj=7Dip`>;#8V&NX?!rjcvh2_JO(`TxB7bHoslg7;kQ`uA};JKvB=cP!M z9V0gz2Odjej8kPiELLciBy&zzbmI`he_pRgt+h@9427SmSdc6|jk!yoovlXwW|Sm# z$WUIyH%m1Sp?fz=ms_PN9^OUoq`_+0XDdxpWR^g>l)iTgWNoTCMxNWFFb|@T@%iZ> z*x1CPX1*S`bQ|Er+1X?+#X2gi<#C)XFgCdI>aDT0ek{n*$IA8RuYl9XLinh7YnE!H&Vq|=}sH_)VXy++HBWUvh=%U=B7PP+%;)}&A188_2t~K#~ zy(_NnM<(o(6pi5bVvs&0b?w7-6{ET+$891HBQP~Zm1>%X?pF3$c?g}o z#Srq3>Bg&r8PtAM0vgEH9wIbIU)`xhM~bWZowkc{8X^fA{@?9|c-4J(TV4GqWKSHt z#R2$dO1N?lCpKa6!Rw{)qacfcY!obw1WKsde)KwcHbO>HB$G@^hsEv4Jffmpi2XG5 zPcqD3;EAW=RDIKF4iqwj;;cCo{g*1JN4jbXLIg9Va#OU}=|K(hBBkxaDNI|kynUa#^FmFZq;;scd5~%!7Pbt@I!5j^51y*& z%~p1Vi<(tkqx&?SPI;2o@OH3A|p3KI>$VV&1<1^|5^0FAd4o%7wcVnm0h07F;|6J;<8WWmZx$% zpk9)%UU5LP@(_sfKXteiImpi+E`9N6+4>XZYL~q#mmHN-HsmL<$;GZc@CTu1xU3*@7GotiQYL#G<3()hQ4=-^3CT&t z0Z{l`@{~deX>4#I;8Yx5#KAl44+NK|$=QcFhpd*duMdLKpp!Xl2iM3VV;n}LOXHIT z&>5Oz!-U6|($A5=mF9-IpMf9!;B4x_MS|JBypF*$%UlG^Do1xMp5)+qeGLw>lOLVu zPI1ZEUWhW^;9CRm3yzcCB)^17jvguF;hbMA7Do27 z*=$F8$PaZr$*pDp{F!hCnqxz+26(hn`c`~+QXV8?iNh4S%YPwPe8*ZHA_Jg zFByjUR`#5 z{1P+>pD8F^TUh#HVcCXb?E;PJ;s_L15%x}HFy?p-n(f(wuwqRzm zaAy2C#s{y>OcYH|iIKldE6;BC{C3+@76oe`+xe_X^Cn_4=03RL6zOw*US)P?<72<9 zb-FFNsMXd_jWILGh)fk>eDL(l*b(jIzMz0lzWega0I$uN2|GS~_w6H@2_B)5Yxh!b zo~(N5MEQo}5DmTdC@>$NI$ZK3#7cwy;Ice5GLg>t7$4lLa@?!5%LOB1iHyizr8yeF zIye<~FZppf0}Zl+FBFvRRdx1oO18P0&dQ!Kg4vN540?aePm?`)q;yHXM%FRhJ3f7* zvA?FzQ2O@Y{E-v;N2aRVNA6VhUDEWtpIY*ClxQImt$re;(!D4fO)>#J>2?&aZw!k8*fFg2S z_g@f-&kSO};-J7H!1zMgg9HQ=5F9YD2vuz!0a=VKLb+P}G58pO&ksg5!a1k}%Rr0? z;*FFbV4GcJ8zr%YOt=6D0!kQ#-8r1#(sl>HWOmVv9i+3MaZ*~DWtar;LFC_JxXL-2 z;RlF+ECg@%!-M6fQ4$AKS|nhOP8eY%*)y$U7^gk5{;{c>RecUIjATlp$XevQkxjY7SabZ zwcVqr8_gml)=@YdJA&qgP{o*RxF$`7y4N8Z09FUwnV3I_j|5AWMoW>?ct+W^z+dDP z2|s~#hyT3OyU^Z1Ub6R7cwzEc%^ZEB;^d zRCHTy6G19r90iOHI{Q-UdqdKOK?j^d#L83TSpW#sH@W3J4o|u9pST&OVfj57jd?c}CW0?TM+QnDmyj@(^G1}NY>atzz z>P?}8yx)oga5LW8qq6rBFZYw!c?kc3@xdQCe9+!Q^hpZF%`e|`+$I)m5jt)YeU?%B zLX6xQ72+qc@DlB(V72m{6t%!#Y_zTT=}6h zF8WAG7y4NjCP*LI%`A$S&fmj4w1c|V&|lU$V!lVp$ABZr{7Yv3Je($YL9+B}Lx0QA zglU@0C{2cy_!R%0#GI|_9h{i9%~F~r%Pf;*#);B9O#`jyv=kXSVw0wLERO#4Ea17H zo~mB3i?K!pCdvqZFp+6}G+JVn28=>fSM%V+sq&tAdl<-?lA(AkS$e1p2BU3`(kNMG zovxUZ#H>96n6=lf#zztus~qK{iOfH*!2|!?!P1AKsD(*Vqa@iwQKH~tfVx)XX^fL_ z2ILghyF)i+kt{REIhu`AWy;QxTa5z?QlzM3$z7vnJESVmJnc+%?_YM)yc9SY^*L(5 znu8#qoGfemuQ-Wi3glgxpCEO~RE|zgqjDgDNs8P$Re`SBDTSeAm#s9;kO?vs^Y=*K zKLJpe-|5;#+lrTf>Yv0SkW#h`Gx*_m38vjbU;*`%tg~}y!a1C_i=f>?Y5sPR_eprh zzS>vL-zr=hOgs5g4xy5s;mMP#E;D2RBVhsK#S{qE&b-}%o*!hLP*K+2;wK`ds9~Dc zA?Hwu6SDsivf|sA!RLD_`uwejAgZoD($0h*>8Jd{x!|-Uzjb^@=XfcKn zokIvdC~-z5go0+4Lm=gjEz)Uam)Ta~GG7UL;Ca8^&Atg4nN|leuG_^+eJQ7{qPrD+ zodXkSd^vfEm!XmFOPT-k?kNUn+8?I?KDZ))w%9C8yaGSj`be1_4esU>#XL`$)M zj)rqd^RTe0-(joB9j^_z`-18y(H}f3>oVUgg!CqXwCO*Kvy0%4o(N`;y+XDH4fdk) zzPA2x7hlSelP2VE5nfdF42?~#45D3qKwH3ZyI54+-`+otu0m%bGi9+w_(L9;vhVg* zSZ{^=%gDa*Hy7s%+wOIbx%!A%&h>p$cYleFI$&sNT_Gq(T*-Mn+7Loq>crES`?eC%_*|!eHH7*3&1hx>*+MJ`=AW zdBnv3CC~{tuxTvzXU36@SPPTRl7w_vB#QxIryG^yn4|pWS{>CkG>PdIjRWJ^c-dfv zV|etBPd#rnUT{mCc)trOQYIKi|J-)v=z_6O7}4ybWzfzmYvrOy_Y zJzrS*!qKwz$I4$mUjFKdiq}t-zjL|qm7}Fkdnf+v|Bs?;9uvdqT5aLX1aj?95PWcE zvS&^HwsvZq z*5QMgKUgr>o9?;!{hz-1I>>uVdVI{^Uf+1JAouI-fou0MZ=9@r={P_T$hCj=SlKgx zZMoHdVnz=fZs81`r0{rz9f{+6f>Aj5e7B^U~T6oXHfHS$>~RJ&?PcBOWTrbHCt*fNGk>@u)ewK;UG!M@ZzPFu*qzWO+w-Yto?Q_*1Ty1)7ziho<(l0?jU;1PF*uZ|R_TtR+#i`ND+L^mU!^OK3{eN8Rcw^mP z+BePdTVs~^+#>DAa|jgp;5PNqTyxx)RiVF3vd0nh#x{}(J300rZIa-S|X!T-+ zndoV(tZ8)eyDA7|$%6p21!7(v#QCy(J!^l=CB!G79m4*@M8DgB;sxAdw?NDU#AHA# zC;?xHchI-i0jTiXXvdNq;i!Lv4~SWcw>hpNjzPv{S_X2J25^<)a|k~XUvgVOk|~GW zw#Do-hbZ8QVE!P`_nbmlOv-wf6jKPLKm*(xtUnkkfsDF4q-H_Xk4FKf z`7poS3@0Z6y{br(^%x2PW@0qMkET7&)=f;!^o&lPP0IjlwZz6_Z}^^h@fA@)JVT8 zjYh!`#_JF#T^K^;NZQ6GXPkD*tYaCgC}v5#0-ck3w{?-f*l{P+;u0-g5KQg91Agq9 z6qN-|#B7UY8A)e4h0@LujCmyeS_YIKc)s2^2(BgRR!~=*EA8nVoOF($?IIbgSjGht zHd92cbHgd89WtjK(nUeyFsyI*WRlV{l6H!N_Iq8#@x_@MKI+(b884Q3F&*QBR~wB` z_wEGlr^{|!l(=@1{Y)R~P-Ji{E<%(Z1t%N_g6?lTW*N2M_RoONos_A#|79&@j=ON6xg~0_} zxI51L>*Py483r&!W>E`r!jS`S=^^qbz+8gbe!JKmhtD|sO3GS>ZtaTNq|zb3$J`X@x!T^k zzA;{koJZ7_(L@pNApiDMMbF$MDVqFPWGdz)FyCB+=lPA}l}5WE6}5T1Ow>Gt#>>T; z-YfO}S8DsN)%0Dc>1`Q;@#2)GJm7OC~C#u-OfkTw3cytfr zTLh2Ia|LA!lOf%($sQ@1UrF1BO_Lyf0mumMmcDiv(<-mj&xvOM4NQ^QX0ZIhhj-9c z8A>x8#oRnNak2{VL8~OWAWiXDB2!S_-8nJ|DS%Vuf)sGMUH4-HB5M9?0%M({dSo}9 zAcgL|QrCC6rtdPkqlW(T)xBkyNBPc)N*vTICxDx15qaA|j(zcPJiax6VYH>J3CI4`!5*J=+j&%mm zV}mA;w*Hj3exRyj#4&_+3WmqY1&axSz63TtXhUWS75Gwy=Kbu_my=Y9H=59F^vi=4 zZ=`89<&{3STWN>07z2FiO9B0mq$>%6tWEa8lBMB*rlsF*G4=+1faQJ?!SBMDvp^|b z7a?;&BN0;wyN4#v%e!qoL|Bv%n6_4%g?@$5nIET>+IxzY`%BPhx}fatz|h`ejMl+m z^p)6Q5+CS$`hXDO>R{>6=v418K$*Ypt0dqcInx!D96;l`x?tGi*Dg`H_Ez{K!iK0o04ep~TNe%bmQTZA)-1=M-Hw z7~^x}Xn^6T!SKGhO9f)7{#z_;M+L&{fvhGIdAELq;JW%ZzD2t*hsYpcbv|1%sjqh)v^mTA31=jZMvT zk51O~jmg_bikpXS+#9%B-+!&4|MtB>s&xptb;$G$j!!?x1wrq=iw*ssUa0w_rcY}i zSYJ@lEyGe9Iypk*{e67CW)9$hSXT{=`V7Ux$AhW&Kws^7$LKDhQ%7r?dG3VZu*M?`!6`m`0b;V+GU z2jDMvozefowVy`+Kb?cz|3e#$6&wHh{%g;P<>$b)pB7J#Uq`O}^!Q1w_TM@p%5}Lwh5GeG{Vsmj4?5_SN=R3rk-u`D;;lMH0F3KB8F#qu}LXVs`@4*GL59}YUye``STSa9Pj-Z@+O;ibBduQvVdAhR(f^?UE1PaX3e=qk`^i`zOb|MdO4yH?K~ zdGFDQPv`#4)%53;X4=ml|5p}7%dInXzc=^0<)*Fg&#!sok#C>2i;VMaY(XYTG&6Hs ztG!TFnYzvY{g*dBFT8TLQhKLOCE61k?)S?#OBXx8m~`&-v-PhOlx{dy{ydH^2L9lo z(lrObh^&CT%;@eQhT8c6h!i;Gs_b%AHn}R)4^~+U^BjmTX2aFfaq96jxgcFZW`w|* zPm^!@ql&=>ogj4RO%Nw-Ba2J-y%c8f>jX`*kCyjDh?+MSHN1PS`sstE&N(W}T$On? zj;T+9SRQx>q{^)`0IOYgK=a!1ir#-^ud?k{;7SAT?RClSZ&t7UQ~ z=T7T0(J~i53NH!5dwppTZL+y|=cRl3qBi6$xChZJK4?ftet=8p$_;g0NP?U1EVH7U zTeupWd>@$T%Go0~KMqOs5Q+Tz$R@fe)=F7wVp?;_hk!14$+&kFDBfx;jXXw1b62V&@u z!|W`%Z*>r%0K${mZ^;Sdqn`8^WPtbK@(EB#&etJ^1QxW^ZCva#s@)>=wjfv!FS7fg zw``&$B)%Bv9$4qz26E;y1Run~tw13}2W?{^k)?SQiw|0b0fva&OI{H5{Za5{Kgut+ z2&6z83&iPZ+Zf2O3319%G`i4g2V>$ZT9=^Od80|vFo1fpx@&Yps#*{ZR6&Snj(}if z%wfbK#+b7ATq;mqLN7O>Yh3{*qa84frKNxTjHa6xC?4jJX_jh8V(wNK#+`)E{u?D#UZrs?=3SA1^qr$%L`cm zN%&uCSQI37ih`H6F>w$Z5>``t3@7{)S&S-J~Yjp=t=jRM3DyP!1J2fDp-I#gHZhD9 z5H_KT6k&^zugEQwfo#RTqVOw#4?eS7VTpe69nyv25>|gOO=F2kk-Qj=KZx2ND0Yr! ztfCoLoLC^GxOE;zBpr6j7KTs+YJ56D0Z&3KWUQte2Kq;)4$C@@qFz-(-=dvNNn7-# zM)ZbqlFiHASrRoGxO1hSoMrz|~$AElJ+z1>>b3X_n_ z>h6{Pm}KNbxdzd^0I^dzz2!vBc~y7M(B$OQjNjQhD}Ty07+(6$K@z7Bz;OhLGBQ;* z&rr-smi|!-{6RDc;U#l{z!34r;)C;dF_uX%3z`$p{Np_I)f*=&jP^(^lL7w{wG5&0 zazN8{q@?>W`mgCK&~&FW?P0f?Y*H0wX$p=u$WFyUUnyB8NK{0(GA_j1S(%_#hwi z3emR^`V}XO#{Uz zKdB7py;+E8I65FPSkP>staS0DJ6EEIB|+M;DaSiLx2x@ ziCKwK%pY9AE{^53CtFQ?OYWVSo?1 z8{mVEn41Im;5PAl$t*r7B=Xbl001BWNkl&T=~I-ojPjizzs%IKBU?!(xaY zlm)hO;P&G3(T;tdI*M-X$J7ntG*NBChbsuf<9<*NR3Ij%rR~GX%69)dP2XLvd+liX z%Drm49Hk%|Qnv$fG6|Md(5+b~$!#(eE;*`a^GiQCQ|)o1Awt}8qOuoR(ec^-n+}NV z7@3NpTgg6gC(F7=zqnX~(~5!-LtTyhL}Lsy-EJOSTUfRr74n#~I*hs-=LFDzT7;z` zEOV4rIZE3crQKeYL!QbxSG71><+>k8k!}Yy%MWUhJFH7B@O(k(3&%izaO2VPSC3b` zakBF56P53rseJczS zT_4PS-jV0O#zgx$`uM+g?dMd#J@20@O~*f;r+oF1Z`Qi(P7UqozOK~@XJ&3`XU=QX z2|xVy#yfAn$`oC!Q536%*AsR|ZTrW^?(XjEGjG0rruJnFL98XxVFhe{l9l--%km-b zsQZ3^FCdRXo|=RPTIV8P8~s>Hi!3>x4cCd`vAKW?BxpfAUl_0#p`7%F}F89;%)-1$1%pfBsfv%bHf^G-O1 zU{tAxOaaSdS+Z=uaN)WHJ|k9um!`+F$hRaIfpo5ECKr;N$-;;PQY60t*a%_;IP4bx zmiVhw_M=WTf&AVk`e%i~VB2ZfLQ^hA zm~c&5;Q;O{A~!%9NeUzos!mVAOczg`rN0m&h_m!$Ni$tX>4M%f*{jHLQw~&x8${+i zTW+2U_@G^;!ahgon5%R|c4VK*ZLi9GAK++c$gMa8BBrYeK3G_`?r7Ny$ICVxFWY#k z{M9oRZ=S7u`&`wg^Hm?5um12t_2*X`zP{P|(W$amLbE^H>XCOO>g!K79$E7Mj6ie0 z7wzZBE<7Hw*5c*&-#fCw80}C0YkZJhXZUPR?mDB`rx$5IpELCSW0`9n`(~Z1-F-TP)_a=N~Mtgy1@mzxd9>^kMDjQ22) z?WP}1V&*1GjgqBk)@YIeuplx?Xcov%1?3L3W11%8#mh_*rSp@)ueRJ$Y~=+n{+Hqu z-{+NQ+-z3e8|oXGoZ?us>o$l4S`R1QWmr5nA(F1Q-bBy`kb1M@U%i2#@8I{Tx(Ods7rirxHX z85%N8aS51Z(R2+kbQg6;kS}MomD-#x)=rP2E*MDnZ2$YhjxYZcW&EG*|KRq8UHki{ z-P>Lyk;TVuX%=~1wGO&v<&PtI{fjIE*nL3|{uqR#eIcM2jzAzV4m7q1VPy)@IX1pU zRO4h|1<0xZ%0$hu$#%229s)Mn_VIXb;K@ZWH zK?IA$h2VzSMnfpEeFVr3I)zB=qiEY0z~G1kA)2_LuN*@qwpfVJA&Q3h;zqy+EyE=a z5fI#J8Y2D?OB9T{OT|?t5gLvP)dU>G=U7%Mgfh>l>Xq zsqW$h0mesYgAl(K4)7nwycml}3BdWU9axz8lJR|WvqfJN6hb=A1FGm zgwUbC?o~)Efnh9d|HJvC>6;q8zGibaKW4@Il&o2b5pkF`{Z2 zG7qKfAgKsr6GfW`h=sK{zBpaQ+sWAMl9J$VI~??l{-gf4il*J-WTTVQCp6vjLPV%x z&Jm1dDCH2#^o&eis_NxMLWnmy&1Dw`^@OeSg2kxYog*0|e^CU^AM#Y9LJ%%-iDrz0 z#9kM`O?@RpV;M|2MT6$!dcq&P+<fcnY0y#ICF1k`gz0 zb;kb*e~`o%gUjkmA;&lo>z#d+R$}8NUKT`K>f(#D%gnbxeDRV1iP`4jU-kq1@%zC_ zbOJg};2|XZ!M`N{KIo1^5jlLY*B0{!vBVFhqYwfsa4WfU>_m3V!HOxXO_1ll@F@mM_b%T;*bSLbUUiWSe@4b!%Vc2TSIOE#UU zg1IziOQES6Y$-AVeF8TsNoJBPQ+1BqtnWuXW6nww%FGfue2{AWf7p8u=qirn|JSzzZC z0Wr|yv99&En)b%Fo9>CwKL^HWSrgl4x_Ghk4n+5h0b=M7qzv783i)7_YlJ#53cK_u z4SUs(G+bpbOhByGGgS2)26Zh|oEsRS@J-MLMr&L{RAW-~kZPm#I{Og#Mu9i6AB7Q& z1G%EF;AqU=E8lpnf1=JlOc{yidSz?VAC8y_f*$zLrzAnky4JuH=tlC6 zwGIzfu#V+1*$^VT#FG4v1O@dtB%J6nYQ#orXyxy-E~Ng(9nSRg$YcsGv>>vb*< zAbPN~Z;+~{<@(Y3t85GS7h_&d-w@Rk=)YW0*5dGE#be>x(UBUbxeB@> zMUV|X)s1Bvbcjwti@)-iBFfV8Iz| z3i#Q<5-%{>=YRz_Ty7kiYZ#QJe>7d!FI{VsruQv2KfkT+$0N;K8rqvXyD(HQHx>7s zy1EBLj9Dif?@waz3yuYjA!#xe3{?`2Yk@N3ttDX^)_hraW zt2QQo{^sO8&pg!g>7ed+`-MO6HS_H^_Rbj6^U;9+VA{VA62e6jM}X4svjJD$?3eM> zqko$=Iy&g1thDdv{PVSm0dB9$9{%C&7u4my)@y&?w`Fx?cu3gG;}-b+_3=;V&Dwox z#%9a3jWy40u6<^6?bDmUHE`^D%ad!WA6pHf!$y{Z^v9gxyTy=Z~PIEe43Q z7vgwwH~a$5aQGG!-McK;gGC(L+gKKXT_3mKKP+V9Ta9SZ9;d$%5E>;wbWfH7#9k~< zQbKP;C`3~cpn({0j4CV)iUJT9aH=5g6UJ%413Zpf9($V&)WXpZmrxhQ)Pt7K8n#Ul zz3T=Zr!SB1Y%e$9EjHNWPd0_r9)yqt`RqGw5CIuJ6@&QXU^fcbhkH^5=)CM%ie!!Y z7wTeYPiXVSa4uV_Je&d!r#C=skOUB6qujs_cfdXAbT;%7$muL_IIk!FC!7VZ1-8@K zZ6#EZPRi?Fz)KL`Eq*Jo!h|w;WEoqmq`hz=oW@~$8ieN9OH3{$CXXVM zXR(QOB0k6m16EXyT45fw8uF7sC$hGB{07U@8*3(Qu9>o_X6m+vX}g=J?mRhTSK|vi zPQ19S;pMIMuWYNIy}kbR?R9T%vAnar_M^Q`Z?4n%Oq{mmnTLBm8PxOfpaYZt`uz`I zWM;4ttY4LJPGj!H`Y7@`bP`9iYM1% zkb+eZf6#BW*_$n;X0Mf%ZYwK23Qg{Eqf?2=p_p|dl=8(mOm296X9F9#PTf!&Qhu6! zA!OL39SUHC$3~4`9ymokJNukamXhOFAi(mm)=rl~NMrJFu6~f*Fbuy{u8md|3VL`pBHUkfnd-o7 zm4FDfYY>KH!VJY;VX7&q#>@j}o{HC2<7Nl?f$>@s37X!;vsod~D-B zFb}$K-?|d_R@)mxE_~+o|2{VQbNTeqDig^dr-XnS%V-By6x+u+y3ARu6$By_s=;=n7>H4@;<`xVnFJP{ zdny9K1da$jQrJH=5CeDBoV(WbbA7vAyxJ=j)0m_RN3oM8 zYCIO}oDe1a`MzV%N5c($T?Bdj4Yyk|q#&E$qN$ZrTc1+TQ?`fj^ByX*$wM$a_g62{4Ea?Y- zK4lxO^h?sRWki^wSb6UHoo>j)lA!Tgs<(?)Mec@=%km>F?r|EYM2%~rhO&-y-0q&7 zWpZW@0xpf69gKX4inv7Qx)iV}t4W3eF7wA7Cma&gE-`Rjn@niauB)?6P_rB5nxqX( z)%D!($*O1>6sz=Fs$&~RL?{#Xg6%p!%j6oT_GcA~RxL#R-Voh(5;1>+}1 z>}eX3sAg4lOVao))k9H=*1^$AKX5A7vBwd2fEOiW_j$3-DMs~rG2Gadx;FPjjZ=cg zE>S&yH{94|2hO-GP>hPw_=Kt^rx?Pwo}7@R_X$%;KTPxiVv8UC^KAR=9mb1x^D4Z< z)V|@WaS6JdLjVU$%3Iy=3m6co_K8qE5vy&wc&qWk&F52%9t)LhnVoKKyVh0PdSeuZ zGJGsjBb%oP-vWZ5*=c6aVC6^{f2(~%Rn;xm%WAH;2Pp$1`hKmL4evT%u|s#U^A-dM z^IE9nA20kYictthA|G^Hp!kAhfW@`$KUIvv*)^YqN^N6@y78J9dQAc!{7Lb5bVL3` zUgN$%@fcpZ1wPQ}Yh;r&nC)x_4H2vd6mDSxYj zRLGN-nViE_gF_YB_`;7(Gdd?j%s4MZ^O#}?UK?OK3EB|}>P?nQ*ShX`$W6@pWy2Du zT!T}(o{dAAI=VI=Z+|pXC(AL&n6aI%{cJz+!MZ_NI+r}-urwWc`xcuoFxz-Yu6{^{ zZqBcO*pd}31JkrJh}MT=W*n5_3_TkN83tzPKH3AfJ;%^ymt`21sUMzg2q^=m9ItM9 zG)M1W`i6P@lU#JXi0KZ*a*nj999D0nhl&{*wcjX`R_nJWKDM54}ga_S{3u zG;YkY!cgGN*SfYGzYPAxnQ-8MH0}D@OXoW8I57i(?T~L^Wd;<(4U9W>_L1cpXEs{S zHMkdlC5- z)zV(ycGELX!+L49(J5HDRe#}p2jnO5NYDhtYwg06q1)h|&&n~oM5_W~fb@AqsjFMB zAFjLN7NYb`(E7z|WKpV}l`z-!^~U4QQ7X4E)#LHH1GNx`!Em~hy)!;B7&I_a?HjH3 zj)S*Iy0Ev4y;SV=`=q3Hx8?HVOZ2Wu8lJ->20QRFR{JWTEOQ^h|po5u!a+o%vVYkF*n-FE8U{rjw&{V2B@ z@{@!^`#`cF}6%E(A@7@0} z*W&nN#i%fBx$_i%Un!g`tk1dbeUxWm#g4-GUp@;}tU1@8yXhYSkqUW!5`hoqA3p2w z?MSm4*7j^$pJRA-@yrseD3IuVtA3x%zluc(38KIFb+Ov*2#j&H6kt3L5qp%{k1dhbF+BUrzOZP4!pjV{Fum1TXu=@3a>86CkP0aqP0<1l3Ly$1 zx0<=F?SwoLEb#)cg>8z^g3bqXctkNbbsaFB2pvV%G1&M8f*mawwnPu8t6?id|7Y)R zcpyWYrfq$8Q|-V!y#vaI91D%ZAd`~r$e9kZo_3zNa_BUyH(aHh?@fcVbdTog`{(M1 zvHIkKmRXi-WX7Olo?v~Wy$|^e&x(|LtH%Mq>ja=hjMk_A8CH==Z25=L9Z*Xy6~4A9i%rX zG}3(rP#_`#A_=T+>>d^yWyL1v5|ir+lUE^__Fk*Nv}a~-z?!O&YpX`Dts1kYYV3N; zxD7Rv)>|fT0@HrRmWJs&n`dlod|^A-^)GEb&P@B+TkBujTKC45+BddZ-q}(2(Z1%{ zB}aWnKk?fXX4(&ChWV~Z4@brYr6ep+Ons0U`~TUr|8U^hHwS$*%I?JLgUGbM^2Q_O z&w^?H^W;Zb-x_po_Ft2}dP@m>uzXEH=KMMTjCF$tfFa zCT*&DdQ;7Kb_r_)=})OSa1~sm&nn=9o+~QdnRQ=caw#@Bu?NBK8zr9rS(N~G=eI@aD_;Um`H{!aJf}j}Q>bvCro`#|oXSAjlxNmReCG z+l`GgDVDL-RR5QY(2&VA_Ky&3Y&oDb4WMS=K?4dTvJACB|# z_+>&*!3*bSVS75}z}SLzAc$#GARtkE|7--1N_PNwANHG62i_+NFQv%)IAqM__Yl@| z=ryHxofl%Q62|Fdy(DWku7yTVxzVH8=f;#4Y&#+a;6iBA*=UKVkgK5m7IMMo)+- ze0FQy6u0|EL0e+qX7+_16|M0`gNF?fV9Zh0 z@@;8d!tPU7Zt$}_6t;Ple{Fznu`zvwXc*w>|XPp9a; z$@&tQoxiXy?N+1)5A`E7O7Iu^2w_j&)CE51FX;~WPXr1j-4O4Xq>!;s2NWjh=ED0{ z$%+f%O0^?Hj_OPx6}fY%wLNSWs0jbeRr2TZzkB)AqwQV$i=TVnHXdg0-t{Z(ucVg^ z4=;0zrqmzSzeEO|qswTW%LZd{i36f1!;MG=%S4ce2#@^)ybqurP`@%d1U5=|ADk$L z<4E?COgIrOd_58hVTThjq!YQ|si|ZHjsj8?IHPBr$_EWj&m;}OL6rUh8T3YlAkZP+w_bCU@42uBB z1+L0TP};@ocYd~9wnztmhdw?OoU?bVmrN!_pxu8W~QfLBmZNx;+_G99azBEq|L z{Z3iKm8SOF&F!~WYa!av2W#paVpTqibh3E$)O>VlJkV?treH?9Z;EzUv~q<4oQ!;z z>)qqkUdSBX;?=8laO~NuU58IxWmPq`Uf+gx`!m@l=LEG+B4Sxqt3^83CBRQTk~Qv& zwJzX#tlNblaoNo*YrNWySwX}1HQ8g~>=5S^jo)$<{Z+IM3sbO)v1$wtS1cx{;7p@S zoZ4rxZbYOq65|iPP+V=Fr1o5@cSu%0U%>0ns_nH*Cri}^E`p-6m90Y~D*WTMth8Yf zibaPp{$P$tmc(8ItxuxXAx0I1nV`>HxxKrty%ODHw_7f=mxuM_qoXzU^D36?g9=3L zIO#OMf|=8Pk^Ix(FBrFe;p)**n&wNlZgzF+PIVZYJ8N5SblmE`b?4r){bzhaRTypz z@mz$)JxKXRX4UztUCi=l*0lk%yPUds>xJc&p25n%NN^#P&8?tp#jj?VnaMp8rrgzj zVX717Zyv3`>PdW1oW#PR?dWi@>gQsZwziHtRcEg=o17Im3dXytKrrA{^x+#7u4a3F zgW&I)<5|!^lrK!2oz^r{%jZjFl;QEu|2b}r)ys*}7f#UHwzhM|J| z_f}bq$`i7FL-^4_mjD1D07*naR6B*{77vS8yoQY{{9`pvA<7>xgV4E~ca0ZrFhg-@ zjvjKD zw&^@P)~$8zFm}yH7M9M2J#74=cgoV!n3j!!oih!?mulmWL6dLkymRo>wUcO>msVZ0 z&C%Q8c$@7%B2%w9cb(O01luSFtTtkacGf1iE#Dt#>YomL&^Al&E{6%i3?1-67jPdo zxMUhwC$IzJ4?^XENG{iIs=Ivkb~nuOX6t3ydYcU0NKAaZSlRO6a*Y!k{^c8nrt4l@ zU&}81dROV~$Pzc*ZU4O2qy z5!S0js>dub=$fIi)U@6xKXJ9Q>)yVa_M?s0PF%R@69wc5Q;Wb15HP`LwPzyg0TZD= z^u`4Dki#lU*8>UhwljMJbhx+^i6@!2m4aQr~?;)H*VhVEknZ;-b?hX zZ*=uuByKUFNlE{^yhBr*zzX1f$G^W4=gItyFhsA}^I<Mn0k2tN;GlcbKeNn={o}tIZFHGT&Aqf4&i5ZE3 zzz6?GAG{K@WV1Nr2tXtsiFqEk-$D0~39W;q54Pwm#Tt$*LvP2ee$m2(=0JRqKnz*X z;uboJIt^0cu)ogfI<}Flemhb3BR7!UL@FHUeDaMsrnbOURdWtD<>=2pl&%@Z%rP*- z4Kl1)*=f!PFWkKQ)1l_~e>st;ZoPZ&{+W)haqBFD^3Z}KARt_(J=TOSfoy7Y^_C$G%fdHhrd$U$aqua~W;eEXM%;lMErBAl`lU?r!nMMnE#qjQmwnf7ial^!J~ z?~+QN6=uH`l>sX&1J*!_lE>CmG1Gp`ddoO4kS&upR6oD<_{8k}&xItu6cP7gQ0N<> zLGMK_{3Lqe-(nVg85{I%T+rX+$g~fi@cNs@Q%3ZBGN7B8_V4%q{%KdW#-g@nGz{zZ;JIW>FqaOO5~W zt54IP9vU)c#HeXgUyhEM5gIiyd%3|W7av?c=OOe45|FVIFdrfpffW{ToFXzPn?lZr_!GBMEQ9ds z8brS2C_oruTmsUCGLd3i2^i5yTj9-dPSdkaRkXVUM2U>l_?7ZLk@TVUO`@ED&|U_< zAhl8TIyQ{s0*)6&pV`OK(FTwwztNu5=&P-y@VRkSDQ2SpAd zQNtOmP?mD<3hF$reJc=jg-G1cAzYLe8pQ3=$0@W`q1hDjNz1-`P(-%*!Va7zTh)2I z5V^HVf(Fni$`29wL6=0+9rR|P;@2QPY4}@Yom+q=or_C8CZ-W3egZudCRGHjB9=@u z66{!qJV|8=tepZ*;8+!c{FSvz4y`QGTA4zef`C5D=$>!%$~AfvnLJijdY9rP18Zih z%p=!Sjb;sRZS~l-)#Hc{Zn8YLvtfMh;TPY2_Rx+g*Uo)$x%HnH&&+9W3F&Byy?!C} z#^v-I?RnjutInQT`H$I8o_Xcro{t8x< z$M4M?oe57|rZLIRV)<kDrjz{;*ja+fvC_0k^lT#`q%+4`#S#o%S%_! zz8Dwm_1HJTQ?{L$w4s{SeEf#$vFoazTwVP{DR9p~fe(6>nmtfI=(@t>ihK}!uJPtW z&t72Ym#vFbp8eqGhHdqiSJhr(TPCiz3}auc0%QLy-Pbr)rbxH9yYHDV+*F?JOjflP zTR^N4ar8|846S`3iHhOrkVueZgCZGpz&FsI%faoZjO%8F!RYbLj8E0_QkZyX-iGbmv4t{X>7lK_s-q> zuV+^!?`e+Q*61Fh3_#{cg8=_OEWS~E%0e7W_zQyruH%q&Nz#3!lNHoYS?3nSKBy1u z6T7oMa}?p6japAF(LDN#!p3m*D9~4GCPkH>Y=n{A-Ht0)mwtKv&7l`QasM-2$)C&r zuH`e2wzq~~{%KOz2{8YG}92y=GO6J&+ZFh<0 zgpqatQi(OBVr7`}*eh1WOgk@hB4(x?cdZ2#5=Ud~C-+ozI!1cvnyd-_`BdoMCbr^{ z0#ZIqB%*mH&zy4b^cus3uQwleNYp^WiY1iD7!+{KfOK5~?Cireb*uFk_Sao0(43!I zT;;$imiLyuNEbWa0+%{ zs@5$*;});}VcUsq=1cqQ+U3VuXUMDU64c&FK-qX8zePHS7}d-2>LT^|eaEkCGGF?3 zTZ1e`^Pox{!mRCz_K!C{}T&mW4u`c5n6y+4HVo!nHOjgvJt83Y1j$Wj9 zN#*Z}ce2(#PCcuzx=?-o*V^_?ri2NoNKNd{Q$^*iB}dPd96OhLxMjhn6BCp5&I=V_PK#0(_GO7wy9X(sPSh{n z+q~C&`4`irxLv27jMaJq2gG4r+EeB3-TOzjh-oVnPPcerKNQj>Qu zl+ViX3sy#MX^hy?=obtF<&j}3?;zz%OHFx)&dyk5WQUB3(0B*Kf%%78URq-ES*RR| zaD*MmiVfOuVyFJ%uT__q?>+r=td8x^N?^nBQDJJANY&(J#?0T&?611K-FWez8;*O# zs6AMVL&Yo684hJcUX!Txh*p2Hx|Y4DCFQM?vQ2i&bzn6W7@{lAezpJP@GLzW6gkKZ zL-GvFDqF5Q_r&Vz;W-dx4~EbM#=&X2>8mYy#YADn={l!8@QHTLGcMDfdv|BU&@`PhD3TzFDy#h0druC>YOwkb z&(QsC-^uu6EilT;F}S4boziq{=$o!SH+x6@AXdH{hGEW``r%8pql(SR%9fu`Tww-$ z$dS_?c_udMbIcRuQJH#KCLjh!b`<)2u!_M2X6irR)0nG2pRGCPTWGR}9K;5v0;tGORyM?P>&#%fUZ<(;z;LQeC=oIXWulm9*jb0Hd5JV>eBD9NAvL+C<^VDwh zrC+KpN9;c39IiZ2+unHjwr3olvG$2k;~WX<%TQtlAqDw#5&H?~p=S#Ajo2qcGNnYA zRsrR898M|0&M?{pNjSTTL~V9?%O~sVS&zgzMwlj9sQ56aI{#qHxM;0^FmD2ZhzLKI z*KAO>{Z@6k_(;p-1l`P3W5MB;X~_oO7O+jZ&#PcWj-;*-zu*bbf40n0j=iCihUwmqoSi3zeh7fm*WCKFG7= z9zHuZR?GGY6b8l~3zYBYRIfUA?trCzgR1STl3K4ICFCF$CMSHtRpDDtMr}LkhkTIr zjozWEXHyM%hgx3EB;hk`WGnwzU7vOE%;;#1pJd!_9d$=Y#`DrqJqQMsei(w#AD=IK zsYKN_@j>W*QL?}uu{|nR$Ue14#>W~gV3#6_7^vrDMCbuHxkHN^(H_{Lj)K<|TB-08 zO(BpPhFpoM_ntL$)mIvXUpv8G8Q~NQ(!JUm)Sl3j3ro(jMMi=HJp)@Ix$Id*MrE@Aa4Urqh zY_vG9FfGxYJ9y?=tQsaA*VkWua=nEap---}43Qgz;AvJe1*Uy5auQ~`mY8J4Cf6d9 zTd~P~1&Cn0S62G2tn@Dh zdGv)q=d!X_+D=Rh`lrXJucM#a(KwOSe1m2Dy6Q3O0aJ`xTNMboRn6=geO8-&iz`t- zXm%?EKFIE>QvvD84E)g)8l@abRCkHmFSax7W-l;nJ26m3u2r68FXx85D2e677T$cl zl*53!Pyjmw*5N0KhbXaTHeMwOp1*|1ix#m}0^HkD?q!MhK8glT);-02A_agbvFHN@ z-H+AcmM9JVB>f5UK9y8CXeEF&{=uF~K$F7G5rQC5FyS9?z$bv0vDZg1xcMqbFJ!MJN*3fhHVuI8i9ifU^Pv1(mhIsnFn31pSD6vB_hl$xGma{;SP_YpYn3dwgy6 zQ){cot+!0rXkkOcNt-RKADLKiZ2H{!d4=zsY51n+UUW~-(w=+iJ-4%a?iAg-vzm2) zzZ^(So-l&-0?^<;8C*N#uc6<*dF5tp(EHDyf91dGM1T)|I%xBRzpg4x+rDSz7h@di zUVNnI;{o?ShOUIIEPrZ1PEMxz=&!%;-4+o%FZ#PrUUM4|9X3Jv`|N*2yg4o+f6CV5 z&u#|o^n^_{H-K` z)UA(Z>umDDn_G0TameE+kc(mE2C<8QDY6^`c(o%GW`-OzvlxwQkYZwz-h2iU1%6Ug z;}NEMA+2)RucvETZ`>vMVoy)~xf`Qnz;W0kSowOEIeynE&B>0=TiuKHG;2?FY|~w+ zXt=sb)y76DQmw(zyL|RpZ+^txgqX_H0BAtySdrabAZ0$#cxeKe8v2^-E?4G34 zAmY5oA9_G*Un!}~C^j$(QBRBe-BJrqZQXb8cCJlmdwY1>dk!EZ{F4&?x%@x4us7-> zm$ugiT}gWXZac^pZd|+kx5D2Cg&lSy|6ibcT07GW7NM>>&byc$>-X^s-mw2M^(F-+2cYcK*k*QLOx=vQXc*+#2uBb6ShTv!L_ zYKhJzQ8PSJIXFz=7^U=1BA})v!U!soB;Xfpu}f4BjaAyksD?#D6hd!SLNaH95v2*5H%?pl(eYXxoi?tyl~VxV$?#QU>>PDn>f{oXq986$}U9V z7O7$zg%lrxBXf$@IYu=+iWQa;*%Klhm3++((LkJd zp(>|&u#d|;h0}b6FId+a`rZ*w^bA$n&#Pb-xYQ|6hKEqx4C@3t|44Csz&@s|8V+-g z`G(Lex{6P;^C@=}5{9dNf>jQ4E7;$_2vam+Uk&C)-zTr)63+b>P?Xr>lC2ebK@$4+jY+eXJgt7+|+XBd*9 z>$glZI8ECxT{|d4E6d_SA@WRQXY8M$9k@(GUcCesZL{?K7O974>Kt&~OZz{vL_IiD z#|i{gkp+0-;Y_FxqDDk=Ifh5Gw2!dL=W`Dm7#L^jA6}vvnxS)ILo&pdY`b%o!7fAJ zZ@G5haxFW1|1_ zDebV_iV>j|?$K(W1bB`x*BhmFS*UQFU%}P^y2U_)bqZ73gelOG9DIR&<20 z6-AF8J&_~T4j~H1FeQ6);HVgAux}JvUQDYTwxEJFYNsfbQ@GMKR9W3}L(|;h5w7xM z=f!ZezR?&4fE0e_ED zC~Kaz6DI=-jU(iS{ik>;;d}Rcj$ggK+;DF4=32IW;s(q48+XoKzcYDL%_BKFR*uyB z5GjEH7mDtM^iyt>#xb+y30FZm$ZYNY8Jh17fpm9&^EKOIlUqsUTe}*z9KYOl^RDQE z7JBT=gV{RQ0^^`u{TIKT3ONdr(p@L7WLLHsF5D=ny3la-_G`P3_h%h4Do99VE$T5v z=4FR67iP>lWArfh6_xI#m1NrcmsT>7TlqNOOMX{lEMerS_4$!Ix>g|#&E%N~p+X?bFYxF?D1DDI_ zYy@{;Yq&t+kx&c*(#p{|1IGbY0w}r?1Le)5upoL0Dk%*tNO2H|&hV@q9Qs=2ojfX{ z#P67@N^>LjB0MfM+DR&JD;K?_t$hR~39&6S3XuY=K>%%e77-X`iTsgxCB+jYVr_kK zi6`b-D?masteEMKw2=c--edAkg#UzrS{)0FvI3(^fzh=X_@HO8$-Bhty8=THmYPSd zGLKqY^%(0!)>S{X!7_d$G)UGgpW9qBW5LhKvUA3~khrBEtA78kt0SqWN8WRP zMfc5>Jw4l(E&F!+GyQr##lGO1T`!bE>0jZ`0McwjCdkrF%3-|I#k-FgkUuYnlGs*!7l2 zb9DW4bi?H^&>WhnlbZn%uB^Jq9>EZdr$dTSdI#XqTO&n+E4^*LG;}7s6>O{=7^!hu zsF>3W!EBy|3f8c{TUEc$46bCx=1yO1if+M*yt0;k7?!>a#U4u1Rr!mP64LmC(kTdOECb13 z8maW{&ccKs2!HIG!i8B2R3?jRYC8MKNVR*YdTjXb<*Is^5$x%?S+?==598Y29CqOY zN2G**rZ)a_`CnK*5y;@1!!G{KyK`OA-CN*EclLD4tIG}!3@vj7poWgUQDv^-+!q*7 z9e3IlgoKf0vM3>#AWtEJ;R4CwH=Iid$?cbLp)}KoEVkS+p&XE(s3MHvu?Wd)SVm|v z7Cny1(^+s;Ch*60aEvP_w&;w`%d|5I6=|ar9>vb$6-Sf};a7?dutF?YdTz+kDDs^v zAS$gFszTsz5(N7rH%8Ay)HNWmn|7Oz)iv8CO_+d@=l@`(mAQnjV=T>_*xv4<1F4lQP#OkgFGy zCTqQx=v|kA&e10o5-UT^iTx$PPws3<;nWhSE4%i^D5yl_O6-p{$%Yz_^Jf07h=7fQvNv}k-f^BiiX1L~F)mB;wG1eZPR4Mt-79UU%gLGyStMh~Q! z1L_uNH>o!tU&p|v(50~PcC@C~B)ZUJjEw9(oDPSM1ydGCCg~vsyhf@p5R(NG_7 z0DO=c6jnANWrBS?jo9f}w-1dM8igJkg^}7Fc^TV^Vg2*0O~V%b=g8yOUc&NN_kt2a z?zD?Oyka0WE-)0MirMG#6`_If@_F#V9Nur(6&gpbtsW)^*48gu_vxO-^EZIR{d($Z z5wplprNoT2t(dm>K-1N)e>-JgZ0eV#8L#wYobKql*46#yo`#2Wbr4Zb zF3<^%2+<)=h&d<*CF2N!l$FJ%p#_Ekx%wp30PbkKVp9O|=tmS7AI;YF&(bq%G+xtM za~V9!=NxK!I72%)M?Ymt?XHtoEx1vA?)uug%dI!=+_`su!SAP?3x%FRfB?KN5<)uK z2%*Ddh2UcB!c6;;N^lJG+|yIgb8kiWt>x6_xmd5n{lCe6N`*9b~in@8FC>^+*mVygN2#)Ppr3) zX)p3YpOt3s60--y4Xbo1F*!5qz5q2kd4eBjzNiP}G1){#7a8eRfmgn$B9wZ9ay6L5 zp;#cr(uBGkD{5P5hF6>`Hl-RsI}BAo z`awInmDAZ#;+_I4HM7IyM^dJ1Jqsa744x{6q7)uE-;G8Cc0#IO`y3%_1O`#I6(4XO zZpk%>T?2pWa()&&;ktNGxjY_hkyO+6`MgTV^^kJr@e8DC&_W4M2vmy~%Ws#Eyx1W} zSkDtuISc!U(JncTlnY5ta;;Sr>(9@NlMGIUCg(!Z4_0CdW3x}G*>9zJWJ%>n_80hI z)f4Ng$F8@G-%vAgea&RZ98~lC=9=kSYhU`MY3BA5&t(2KHEi*Bk>9LY{Y`Ui{JpMK zJw5xAqdq)58FCu|=KFBa-M1h4;W?jU=7SNhj_G>y;s2%+K|VO-w`U$$oG`Dar@7~z z`NE~cC9C5WeEH)1nf{;n4N44~cl6hts)IjoES1MB{9(>dKRzCmF?~(di@Td%+|RIY8@-AuV~(B; zIeNxtg?VU(-hZW8c@84Q7%$%Zy`_`w?_XkmY^`N}*=fI0^CRinp?P|{B8n`F^If@y zf!VrP6?_$xt({MlRy~}pv&F1{FcSf_G&Jt(0PPLUNDhqJL}m@;J3iLUN@ z&6jV#lx}toRt83Ec;m#>B`ymUu0hJbmDJ8zcYIue&NEmUvZb;4((Ox~cW&J7Ha2(O z=z_FdEmv+A9clScUgIli|1^_;HVi9@n^^uL0Tl7SbfYjE&`YHpY3&`vz6Ms%Sn4$j zk&I*fBYKTg@K{Q|-g{uwhgzbHSv({_QbT6``Gje_7aq;nuJ5`9jstf)uUy}eb?F~t z+TIx2_Kxj^4`rYw{M17b75=F!{B!v?%O|erpYD!ti~EI-TrRw4fBucZ?4KR-g=7j(T6OGupMANiVw)PJ5!cnM1i0X3`Y^400cQP zi~@8h6CxGLA_@GJ6UAd?IzncWa>RM%<{X~ILK%$i=xw&@xM_L-{ct14P|&D zt}djgh%x1qF_|`c#jCxNv=lnZjgkl_Xx&o~U4cc1{>9+sius7aKE&90=!WbQr{Tgh zWJUzPm?=o{@({)W*Ym`*3GRr`AP;g1;Xkf%BcMhY(Szh^=zi;oXY@pw386P{bXX&q z9-Iz-(uw5G6YJ@=gqQDzp27l*u15aaKV(~bCL46_F7A@l%8h4N!X z9>Ih42%bK02?Fn+UwcCkPE1HPMQjN0N;HIl!*sD4YMZ{O?Sic0m}=G!ktjkOcpS8m zUPz)g5S)(oj3haso|2Z@iw=Fc00{U(V`&<|;0pNOW*dY-wUJam93o_csb(i+HlR@u zjm9bT7=>G*#h$|OqiZ4IHiv9p2v3iY$E`IQUBjM;^Xzl2QbdS`mctQ-Kq~R<_!Et| z(DRo=Oz|%zUZ877_JQNH90OGz1ye@KwF>`a`(&boDgz)OQVb(W3kjf*rhy8_EkY(RoJ!mxMWq zXgJ(QV0g7V>u=COymR*j*a6}LGYr+@XnClJ$CKRjQ_GU-pWFSRxB*k1_MI*{IuBEMX8`aHCQ1?KkpO7Zb8`M zA13rFVy7V;IfwNfB=;E)O9O?|;e7}~AABHnxbY9~h50xucOAAGxYZaUP7mL}km6Z2jOoFv_@u zkZXXfwmjHz0;IFWe98`x^;qw|vXaCl^2!TC9$%Abxj zPu^7Py`qwtpZl7x9lv_}?!EiRuiUP0zeN!e%g%I!96K{=Z8ZcR6v7?a=5s9(QVvJ} zP2_}`vCer$=OUBK3NY-0_4MqCn)8Zj+Cw~rz(<2RUw`=9 zSx@afwkv+7Z_oc7KDhp=2NIG)dwR~?zgKyt_1C<@@cExlUohS0Bj3TPvEfJe?^GZ9 zW!u`~gy1=!e*4|{sN$(>t6%)NY3BCh)5x^nSVMUQNc!{mD#$A^a+Nt?C7AZ!rBz-l zkPntH(;g&HPK73zJ<79E9gdO^_}G$HrAVTr&`H>Ck4H(l4ZNR}Kte8j0{D!Mm9bc1 zQ33DLNn=Y>UnF{CFFbU2Fk`2&jyHrAKS?M6>u@FF4;_NI{)c&mfbd=(6{1aT9r<84 z!f8OC;D(`+K1!&JI5G){=zXHlz9smR!QNXiC^6)rkOw&LlUu2lRWm4k9;AaqdSWXj zaRt~6@}!~q1gMwC8KOuJt&bU7DSgGIuE!h(=Zk%r)o0mWZWQMp?S&VJw%7?Y(5n6u zrG@ztDkrzrMC6b38VR~Zt30(2^puMJc%6j|-2B!F`#?`rXg#9{8AZBkl}QR0Vy&Lg zyTKDF#(4(JbL3V8Z3*HOtT6koWSvN5U@3+mL>=mN;Bqno|^)Ql}P(>K>n-3ap`lQse$9Jj&3>i-1j2eA_=t>Vc^O3hv?D&0ye z-Aa%TmQ)fSw8iAwBeD&$0#mHIrLO%}-Q`>B>n^XvsDfdNv-%4+6z4j>Kh*Tru7(Ge zYiz(%+hpJSvEt{!&_7f6;l9T9+ua>q_kK9iJXo%0UmZue$uYOr`$Ge3`XV}Iz!Wrl zSpBke0V}I^Sa5>N(mp0p=f=h^(R{p)4Hbrcfsr7ia1Bzrhp3`=o{HR#sW$hWt~z_Y zq3xy-Bb2lMjvl}IR&KRNu#y}E{P1H>R+c}HEkb$Cds|15V)vx+>tmSp=>??fzdv{i z_Zp^1#vekbCZ5+?2j1O8{9)=+!YH{lCFy$5flK2QtbApKN?(5pl>6*|SK2zZWn77S z>*7}dZSUH(y*{|@tzm6%+qAuBf8hhCKbJq3KQ3+WJG8xP+xFIQd|QTJ_{_6CWahP! zh^|I6x$|E*cYfiT3cIl1?4rtDVvi7v<6^xia`i3eDTZUpUBf7aNI9p6n5x({6yP9$ zLLt)#XM!<2<1j4(ZKRUn5GW@`$Xpy-?idHBkck4 z^-Dn&-$nZ21xYxNTx+F5|FSxUj2`VPSl5#o_NlCDvztjot53}e}GeSr2F((bfEuC@b+6Ffl8JX7-kbIe;bKEhD}>SzhzP<% zA=Eb(3;X*cAp&X>gTMu`3yC+88EGg69N{4vOAy&18c2+#l6LLBU^GybgE?0Q)j{LI%1gc7oe7hBXvwi!sW%`y|28Ot^2D+r(g- zudvHicw)v6;%hWlLC;7eeF5QCAtBobhH#kh2lkKS?*RJw;>!@Pr8mI`m8UL|s%Mv* z)~U~*Zohr``rYQsxAV(eMyBfBqakV`I|~wLExkpemJsk_90jY@ikzB$f{n2_!GI!k zFPz%pK`4X0G9{o z5PzlSphsno$(r{=@XsS51viazdE+JPiG+Lhxm?aXFjxQVW)R{vb#y6PIx~%J#tS!_ zuXP2NK@6g3<=KVhr=5#T{nNFMtoJA~+2tCn@*_D|f`jrQCh4jg$O(4%Ovi-vHT^Pm z!wX<%5-?g-M1mntx+0-uQXljnsW#$`SZ!E*SaA8B(oorwddSdng`1v-m6uaB&6b{3=Luh z959$ZH|-%sNu_5gsOtQ{v^O&oIB;#%kBmX7^P>awVwo-=oDs(bEhnQ4Fb?!k`hTfX_`h2-!c&B0yOii5k?mpt+8v@!2| z{o=-&mv^0*z7i3+af0EfF7yKiNp&l7Nf^oK-05XC;J6__W!h z%OD#pflfcH?`S8$8h^02Q7les_8#*{!JmZ9SwUVOnNDBT4)sNSOnB1wNc)@!iRCWT z!N1RdM+z>*U;B_m@ucDE2Pk7puZ0|G3ozUsuF!THR3HnUuwtSaaM-lZ)zg8LLdc$L z`gmwUY^y>!a_fmm%xNu93>J{9ho7=BMKa%l2ht9CQmL~LS~c5RAMuRV6Ny`_r!a^Y zP8GB7eILk&lw!Du*JUkJxC2xlE`r7Fv~DAwmwuwc3;8XTs)n03{UgQgmsrUTg3 zt;Qat)aMmHXOkY;A!mO_}DJVU=6-IOi0`*@TJ-~!uETq(fVg1@eA`#*Tlf@FLVcUJe)s71k=?BjIY`WBO ztNZfxJB#)-U%SQg8lS#=JAT)x$D=iFA zA6N-fLD=I6+%1f@ zm|{E$3JY%IsRXBEIv4n!3}2*k$W#Sw*Vmmo-E*HFT~GI=mh0tPudPYEnmhMO@`vrg z(=UGa_{DEVUi@a%pUa<%bh$ila(n#S*E0TbZDn-l-jc4Snx5{v^cG#a)=_+*>WR1m z140jBmX9OO5tuR;2`EH29+NcCiG8UJ~=GM@a=dY9Kz2loSIgEVDj>zWX}W@ zIhK-E4&28GNTq;VE_v?)Zlixx?utkI2IkEF%`1L zb&1w>5k~Rj_0;luQtp&QIM#WY&J|Be;r2wuADtJ)apD=_Dxl0I{4%*CFxCp|!*3T{ zCMx^j6pOVkK~gq@`;^yvu@YZ8HsrtfOZJuC`9}q_TnSK z|4IfGs`8Fiqc}_L9fxkhk!r4Wjn`lj32Sqv&~<=vFdeOL6cWHBas~&VP`DNJYm{32 zcKf2UDtr2lnS{6WC>4ai_Co4X%2rGxZGdPnfJZcjsycC@#GoLHOXu4i@hEZpOo z5;C)1!Z(VuOLr8B`^4~5lb^C5uF%WL-eIeF@Q+L|1c!_NYWu%i#KtAXh}3eeI|z%l z>4geX!BAneDO4#yBIy;^{l%6ZDwZI&ccGAQ;iqt6>9Z~jXsx}C*k94uSm>rCE;ZJD z{Dq>#{!=KHccB52{*x`f;R@(Bsf}ZnYCo%IqO>38BYfxOI(kdFmnFCxBSX&?V_^x_ zdM@|e7yW{}!Ti6JiX>jqa_ZV`wz9pUJ=fGWC`~&eS3itId2(J!I5n%A+-RF?7?7>| z=HRJIx9;A*-;=0nb-^^m_L-1hh+I$!L4e&?gp$Avh)Mfb)(i+vHf>9Lhk8}b(dFFU8Kyk^~2dA zMQAe$fWlVJgWoZ$pKM8@6OIs;!5}LFKImEkX?#6bn0!`L`ju1$7Mn&DgCYFbD)SR- zs-9Y1%}o388*84~XqmjZh8fP&w$x7FR{P@ix|g=qzO<|E^x;z3;x;zvqMhY}#Y|!9n?BAId4t0Mq`C?&j44jrD7O z`*lfnVc5I#ethbK@4lHcH!3#j*P1{~OZ8Mto>uV;hvrO1v zd1}37?3(H)S5`f~(mZmt*?*PUzr^HU1R^KT5{Q}RR%CK1G(z&?JRbGWHeX0#EH{dM zGAATRPKXzAiXtSQrCZGtLWl&WP~bBHdlA_bJ{O8|tG9(h;G-bsEwQD>l4D?9ANCc* zq#7-sFPko^fmCK?)v zwG}%ev5-FFqA+WJC9c!BE>JMrAF$p;fuM;H%?5dv4bd*)C&F963Um0A;3_|Q067M+ zgW!Zw&W9Q{QvOKg(5E+#Lo{kaoRo2WpHL=_DmXrqyc6{7>02&VOx#C27yBnw#zxLZ zAq4Dt?UQeklWVKSt*w4~jb+kW3+wEj+f*}kW9`h1 zwJ&bQeq?+7D?967+gbO=pcApMPUPt|kY5E9$ZM)buQHD;h1ATxrJx`5EH$(1^(Zv4YljG? zFfRlVC8$-ifgrnA&PArr_BIBWpN^|IyXAQM+0HvFt1s?6apitb&+WgL{*(X!AOJ~3 zK~(O0hnhQTF5P@#Yu)o(YF!IWL$e{k4ZH0lvLW!cJ8x)hvn%1aj5Csbx1e7 zzV6b~n`$1-)(y^JLp}HiIp!H1a}6W1^p9rhZ1N0W?`^EUaFgx1@6^?aYb~+`3V)j4 z6GXyZp+Ncqqtvp6idoA|lalqhhg(iwxMeKiZIX-v7=C+h!jJl+pS$HloJA(_-yk{k20%1L-plG&4!A)i)|N(?^yrqzT0!F zv*%_9EVnxN%D-D~3CsVy+yCNn>;K5|fAU=aA6@R=?(?$T?z*EtelBTO)%fJYgM$y+ zMI3R7;V_Ovy;$Uck!5V76K-^j#U0?EM~DN0lrIXEfDy2#b2!n!BLw^4@CZCS8jmH4 zhkFvpgF+%WEQh2jcQMSd1q_~BHtKr;zq($66|0v-%0BL>OBj)vyPTk5^$5VZb4aS8bFuQ- z;jTmp@y5DQki=wu&)Lq*nj%=~x`<;$cg*EN(3VVb;(!n@l3a|*%UF;%5~OqIQ62FX zk&`kKQp_zNaMC0prePAraD>GRo`r^;-5q_X=??G!h$#e8@wwwa0ey;>oWySw0FFpC z)xpXHMRQl6Y<`0IDvWN`G(kl+svnQ9-y8f1A1q;z-Pdfkj#B{|s;yr%kpPAY957TE zd?O3+59i}%aYZ!Lt!8}KA3kKF%dp5ALe(NZuo46jD1`Nf7on;?%py?2L;^(VlduD1 z5W~w%mrojd5ET+6MF1k5m+26S zSpl4V4o_GNV^IML*ydZIfFt+4C29a0Y4mH4>!3&?;FziBq%Rwz2WD_(A8mI)At7ZA zrn5Vxaq7nh=afN`Q#n2=#=^N>z$rrxUm$JBavhyTlpt_E2o->1U6$#n1nIRnrNaVl zl2lc$gyNUrC~BtixEN3n9~p#*hXRhw=6llpM=NQ<_OfJb}!(ux!2(17G6Is;cbtL7X za?pQYPe8|xT5dLv3?OnwYW`&DkH|NS+hCbrembc9^m5%f{rT%v7jI_h&mX@6qdUvR zo2IrKt1XvC7gj#E5%Z9s4LAZ*qEnW0J9H-NpQQ_2Q?;fRe7R0{b}c+|+F5QKkggq? zt0!k(M-+|?%F*@9(T-YOm1AnV)qSt?PIsi@EZbpF4#+{cfT_qhOb*f$rvlTkJjm?C z?C8n=7i;ewXH}8yefvK5-cbpHof|r55CsttvmyeL1Qf%}I5XFI=gvGg&7B#cb8eCf zbWU_b6PwUYlT>sB8As?oea=3ib0B9Bl`KKd)KaAJ5|;C z)^Dv^-yuq}|J0QW*Fn|s+0l!oE$5D3zOm!rh1|xIqVT`_^l)^w$49BA$|dwDlX1^9 zcmj~C%o$wn6onL~eF#lqTn*Bn*cG1mYR^NJo`-5$L^*{~{HQRH*R?%fi!R0++MloO zctKdub)7G5>6%pEHMyZ@%C??q+rYGcW7nZ~{`T#-_mg)ekrad|o_FwY&fTZlXAMIS8yEN zo~hez>4y)hACqJ(`~ z{S%+0Q$j1YOW|%$e6lubl4N$w6dyE4L$#7q=J66on%yL=wB9`>EY&@%fGg4Xpj^?h zH*JJ(N8Seg+Re@?$QxiUDQu zl+Mvac(O>)MZh6MKMm@sOVg_`{=!SG5KzKLHmPn9FudlpsifvO$x-O_1ilEqh zTT0A#JwVQV!TcLf7g!F%EJG_@q03$2;)|+wM^$^GYiKHT2xa;Zj5D;3S`Fh2V^_C5 zwW{r@_3h7W=zw};bLUH&J14+6L-&-1?rB@QXKd?%dgS9nFP44&pHD^oIOo1Q$Ooy) zxIU-5o{>kClpB|mL^YE00o{0@z6YIMsh^lRK=X0AnpWX!HA>-D!k6i;J43A*&K~X2- z%*CEqoX#n#24*e}hv}P9fee8mOh6ojQE5a|5m(dlhuufE9{ld$h3j{I{PE(AA8!0` zn{(gF2FNhQWRNLEn@(K+)UH)Da^3K&8x1C2X zSMELeMp;{eWO;1)BwDro%-Su{?6JjToEP1!innGW78uV55vc3l0_TGZ8sqY9zgXS4@Qbc( zt^>UX4}Sgix$iDry?Eo!#alm$<5?1P^0u3C(3c)XiChq|Zm&&_{;sv30kAa=s$Z|p=agDV$YM2A?!Cs|js9 z8*BOy7?Fv{%C-5D{bkyK!65>Rl-vynTDFatl&}LMQJ6*qP({!~y@|-vMuIq!{t*$Q z(-CYUG)7y(0RrNB9!t^yAZFPJDMn>6hzxn(PEd~^8o0F2!)qP_#9`Ge!5p!FN z4#~GmoIN^=bcoDGfe$Wbv8Qo1B7`Y7fX)%|9q=I}EMMm&4M?eun4HhNQ6v@)2Kv5*5 z6clSrt4Nt18+SN_HO#B+(}u!A-Ug5g^xEnaj6KQ+=g!*k5Q z2c5}pg?JyL5g@W~4_|{=#B(Bn_}+;Tla37L63JvzEVjUxi!W6)o*GX|@w1Q~<}ftG z%*|JYHX<%$6bM;KSvs{K6tscr)o2Q*h%fN)x%_VHWKexg|7fxD!MpNAuaqW#Pqc0k zIn3qH?I*uq^_?cmlu6cIiQWv+tRK{Xx7LZZ{7kgeNU}1tSuAsf3pKl)6Ue=0^=K-9 zU>EX1t;3(p`aybpu~JT`s%Y_O_+>6-sVtzD3N}0yi}91bck~|^Vj+DO@PU&t3vhoK z5pY9&Q~ss+4Wi{v01-8YOcGd%h@UNf!iEK$m9Q1;Q{fbb>ihQ7`%YdKw%e`ScP?Bf z>+Q$mmu~!G`vH*noxD=zI`^AT4*fs*%|ol*zy0jc;9}#!GH_A`R*3GpgNluQMUF|^ zy1zULA)8KIxiP=-=J)YPm zb>x@3PyF)JV{h&}@R~5~cO8E1lcTSGbZE-f?#Xps6E}Cgw7KiWjh!#lf?wd%>)OYy zX@6o>+n5?C|HG?WN1$mRkKiI&TxNwksvM+0A!OPk=%4@;3JxOvbNNZUV`|Q51{j5v z6uKz`GN^!+^8Qr3NG;-R^e-`ZmxMyB-@>mz^oXeW_Uet8ADtQ-=+iM!pF9s*D9cx~ zn&VZ^#H)_#Gk6)Vs*WV<8I#N&OM+fYSZk+*&Tm5_=NAO|Y_a5TWbga;%c6Z~iuvhmT4ID2J3}g5;T5ikeex9+QzMJdvaa-)3qJX)p9*Dp}uoseb+1XUDNBjW^CztbzArBoxRhy zcE31n#-UgK`NyAgsKGnP`?4PQS@qn5KfZat_<865gb44vasS-M1E!RH^82sO|N8Tz z?|gRn?Og}n*x55@XU}UpduQ$Fd39UQ^ak*5p1h^|<%VuioxQZF^Z893&us2^YGeDj z4egKDwu`!N^y=18HLVZhwIrqs_*0gZtR3Q{9M9}qB$*Y5 zxoBMtWhueonspSPWIZ;~`ZJlUGfM}^+h)_cGAa{UNyTf`C9+Fc;PAQDtPn&Bn;$ya z#h~R3E^_%6yZRM5|7n?H@Y4MuB~3BK&2h!Xup(D{nLDl=mgs6v%yLg`r6;a}w}_u& zqxd;oBrA2tRe0jc++t-|i94ax9b4&+5r`Vd^cVGNW2fC3}Pd1!RsB7-$I%2 zy;=;1K%NW4iQv3=6X25hVn2ZNkk2AKWDy+uFkUs>G$@;gO8&@V*T@|Eh^(gQ^hUo_ zyHAP{o@&IT8DXh=5zp-nMx00g6$%7Yhm?((VCtTs=pD@bE@cmJJ*0tz?ZjUC87y-k z@CSS_V=pBbn#_BF@tGKL&yW;278A%NWe;Y8Krj)u5HzG+%px^#IYcTGRT5h1&ho!!D2Z2G9%>_ zfeLGTVCn#h(pApIu| zBcxx@Oe;FwgGGMipK@7THDM?GIkFn`1b6GG&q+W>u#q}~KP6MD5iO4UOZ2mOH$G~gw z{wQDs(#}kHpqx^A9Y&H{E9(%6AJP4RjpF4WveY4Z!q`J*a0UzFJT%(LkJwN?yx>`k zszab}OMt17Z4VZ2A#*ocSKJVEf~eX9x!*EJAgPTCm^%a%F-fFTLwZOf2mTRW1fm0D zP(F+MXhfeg!AO4r#T0;u&r3=`@jjCiu^Rpd1c`Ve0AUBDEMTo4ZS3+YmeV`xAw$TUykvP;c+o%$wp$mFmjO5ZJ zo&yD+cvA2>!9&M^VVMkeLFJ6dFE&L!;#7Pq()b+!?O{43V{nW)$E2}5kSc+gbkqsr z%R|%>2qV5`(bD&s&Zz2eZ6pI$B6Dpj6R`y(E$?ZFP+Jn_Y7kI^ect*(U2-dTCG^c? zu6a1G^!*;^{luqgj$WGy;*+%WeYJ!d+B1jPf2w<&MD>_uKq5X#t5~duBqUn8#O8`# zood4^af$olk{eY8sO)*Fmg8g?B1H+suHdB()XE^Bhd10{i?xK+Qi15q2vA{$MBk# zKkqqeJ9GKKg&QsBul@Vyhv$BA?A`4LR=FWs)ZvRaI=}h8vh`g2>Xz9b^)53`2b4MQ zD|AGz^zpv`S zJvVwy+kM53!G#7X*9MDAE_8%ex$ntuezLZ`#B+Aknzr%vU0)Y>znYrV@cW-R(9a`o3=VJR{pVxLCn6jyL%1h5*n)xp#3Is#G-=9YN?HE6pP5Xaf)Z_mf zuif|S$3kb;I{xjebHCbs^p~F=`o)gkx3~7ZvAyT@?Y*Du-u(Jyf9iq_GqKmrMI!)jW@6~!UAh^+BMEq6zjgXU@Ia)`qS z-qwobl$sPsskKDIVe#)&DzsuLP%AZ=jK8KU_bWHqks>g}b9^c!#Wu;KQ@p^j7*N$> z=k|4L=wo{8T@Jm*NRTo;bzp+E#!wm^#T=FX#;Xp|hZC6Y>3W?+?{b=_(3C)AmG`aX z)ET*#B|9_m;9o390O-x87TO0!WX-c&+z!rlwiF!UI=IcV=Gru&YlH8Ta@|cWIGQ@K4 zYZq@Vm6m~^ycWd=X~dE%h+^}+Tx+Pewv;)eUzz1ChkB&k8C)W2G30|~uE+||<;E6RXG82Cr8$p3D5`@W=JfmVjehk1y~q4=#`+zcKJe0OqFU+qlW_2je%I&pKQOi5TVrFV zul({ipB?@AN4;--eBh0b56s!w``QkGgEP1GyxPz+wV``*L-)%KT@$u+iIPC%!CIKx z2qO=6j9J${dTrY%R3MMQYl&M0m58V!#WhCYxC%HISAcjhh~}p+HQy-iM~Y^t2%=N> z`w%GM?>|4B?>zIqn*j9A%!>ay4?s`wA=r)Tc${jWVo zMy+f6r+oYU1$JmBDx5yWM!#kDq2;cB`|IIdM=t(=!SPOCxmoHu`_jg)2MQbmmfC|v z-?6}WV436I51Rcxu)kZ?`QD}j@78qf>%H)0&xO)GCvX06=g9f*tG_%Yj=g&0cE_o! zwnLY)wjLJCioc&)vg_EJrES4~{3>c*V*<|a5yvQydxUW*T8G10CFwPbz8AU$NjAmo z;*zWfCTew=7vQtjadd3FwNu51Bx<>-pSo~kp1CC<(M)On)ap&q!3~ryP9!n#7jL`h zqZhk;OI*I9Nys-IEHQkkP{>LNg8VKQi42&2frj zBU2$w1TATNU}UO8{0z-@hGxM!A!wYnXbBR)kb#WHBcFGw~Di%s6iVA1Pt5!EB3=g5ZBu*{X;nH&PCZsv| zVsqeDz-`QO4xR6!6N$u>gGkS3$#6wxIAc?tu{p5Br5cf0MsWJxz>F_~()SW#V?LOo z`30vV7ED6$hY5l;Vc?Jy$pmpD_!pVEjD;x(;z3|`;Sj6dMz{)=99&4Mh8nk!#Q@g`lL2p9z(Tyxg&+we z;Tuyt^mrv367+Wq*~b<^7z^0|gyl6;;wY54g<%Q7jgCZli#QDvNs(YaDgm>h*ht&OeD^T_$e0IA}pv=6E}*~sg%;BjcF_^6OWs0c=7=ZL`fZ40Ym}(X@wT_9Uk%_ zn*D_2iU1cBA3U0+jJ10Rkbg$OJ)EpFw`iJvx-`VyA-fPBXjwe7;tJ9vR38}2)C`OkZfMpt|8Ewn#SV)#{p97Gs;|FX1sSXIj( zKRY7q)|CAxdcM0};5xhO@I`URs?PJDAA^;H7p}LSzqYpH{Qv&zk%hKXMb5MT{l(F! zYR|vq+kGk^=9L0}zU3w}RnUqrR?&kB_Kyy%#OPtmfs{p-yP_(=I}l9!DtE$4&+wJt zll#c(R$(BIUf1^c`nD(5w^J0tXE%4exT$l(CNS+M)pt#4=$^W@d&bu8nOnNv`0T`^ z|DIPg=Dr`_9&ihd`rC8*-I&w=cO(4^p7K31W8l@-`u!u*{>^^h&l%7<`TogIj(okL z{hdz_{c?NnFSqr+we7%L4ZW{#?|rSIduBt=bZ{%~o>JdExvqQS=B^1FI>)capkcKT zdGLvKARc_8s`b%TC>|6SvZ5scrIFDqJW=9*xjV8P#e-5y{YRDh_2HCwtX zs(&Bkr%((vYIL{`tdw4r#fBObq7#Y!68A2WtzWCx=bY51*P-J->58Z72d*wuRq#`g zQ8)JWx<$9It-}ds{&z)|NoL(;squ8dqgJPph)0c^s@~B({+gCeF*sf979i;xNw@s< zcU|!;`S9SYkT}l|f6`~e54fD`BYK>wK2mg#8Dg14d*!k`3K(0Z(dHYYUWmT#%u1qI zZ0_5W*}aD`A{r$~q z>zuH;bJC`+SCkaFp?lVj-dDGEJv!@`86)ofa?&9C)WOcFgLjU9V03uE#905&CJgF* zb z7%JipF5=I5?&`SO_T?=w7^{X9f0w1&%=RfQpO(1&-JXx2P7g_MiCp;CEjg zxah%|h>x8AzVp;om_GH=&3Be}WHubWjXa0{3bPJ5%AaPPqP!SXWu%XO%)3k7q#INwcGP_(w2H>7la$cM z%%fyv{s}|WUq%-aPcWd6W0{j55NyF3Ty*7!e&F!Ofk)`FkS7!wArx*rpVuK6&{9Xp z`*tuPq?thhi2|kYfkF`l6(|G7AnPbl+ej^XV4?I~!d*i@Fru@ZQ8~`o6eBXn6`AXb zNjKt>9dYt!bfz;Z3zmpnS45r*)-#=vxz33BuCPV$GdjZ=n`Ve)Vg-`r!67jjPD(7& zgy%UUb0B?Osu7bRvV~+K=dguhafi=$MP)+)C@XO+ZGlo{xiCYgNGZ-9lZi#ma)vK( zhAwi2Er4X=)bYu%U1TfDEY2&=B38uZh^vUsG$OO9v{B z8AvQL{R=&3TH$L<{UR*aMu3lq9>FP?@gtKxObONDbRUoss1%(H60TFayL~7QRtpfKBt|DM*^P$U&hDnR$^05HB94*o)Jrhaz#7&Q%iiwMIurbj(iY6BwiyCv~U^R5VVJx zjM9`RHT&~snXcJl3Bmb`!x6KFEG9b1d=NEyju1qUq${MSL*NhZcC#XqQOG(;K1LV9 zBoRtTD;TU04Sq#q!xoWL6U=z>9i}Lgd#J43xpdP;)otrb_jUtpjHI$a#X+@%8FhL*jYX zGwLcrD@=^nH);){g&xke9BAPS8kMksMX4Xe=yIUFqNsx6KF6jtA$H-Af&^;rO43PwZPqCDPi8mouCZ;V!5X2&OiH4Pwbb%&=GXfP~qiB80 zSXBp&DK?SR-6gG%iVvtbBgymvBm!V(rS4?TEUd| zkP##uD}_sRMp+m`=>w=spaP|_t`N_Ke9(zk>KOW=gRV;}t*Nnsl!e^=;Lby_A;l0j zztXS#N>M)u0%|)EH$r>F=I9e92VMnsIaq22U^vp{5%!J4$xJ8{1&B!*P24{kwSGhl z!Q(g;-jjIw5fPMxV==%&49#026{$*g%KguzrWkyeB5}MS8s|0<1&)?_)mTJM>b;_% z4Ww-pTfDiR7vt;8`_y-lc((*zWYGFqnR$Atbah6rf5mgP1WJ9AWX)NV7@KCZP6;(* zTgiDWooegQMO>en>Mmp4&v&5q=(%vc{XFOa zTfV;b`SDAi9=@>1c4}fncg(7m0mY67ON^j0=YSH&zvSD+%Cv3W&u;1*obR}Inf-xs zBdFXppvWQ4J916iTOS>)YCE_4=tW@$ouDJBO5E^sow7M41ar`xW5uBuwDA zRi5}&U=)vB(E{FqYubddJa%3C6SeJ6ZUPeb{6;YCU#jbTxgI5iTQGoFL-)*`2d34x zKKk<1MI-P1YVu&ml)=tvgKM81I4m@1cD&!;CO&Xr#-Q`F`rrBq)BeJ1{ZGsuls~5b z@G;}w+|l*(U59?Yz4zx^d*9sFJ7-((%$+^6w{*{}@19=YJ+-d;l}%lfHg&$dv2*-J zp0@azb?r}za;^oxz%eUZA6?NhqNZi|N-*sc5Lt+;i>mTOmBaOhmcz9NSKwp-AObT~ zee0v?r7Yy9lzl^CI$T}#RV<}x67n#ZvJw@gEAm$$2=R-k)kVdqj}+2}X)#^3o{vRs z=fx0pt+Cc^&<*x@tD}R;)2nl0))IGB^JvwGUhUkyVCq6N2f<a^Eh;zFRidj?q?KPpO%tow zwy*M>FEcNbuNn(4J|3t?DqTaDyF)7>9v|?*YAGSC_B^x#rO2yVN3Lysbe*V2fZva+ zZGU=02S|}OcZgbb!e)$STHiIbp?f;UBAog0!D$;@qhEV>QbN$X@%?`nd*4fs4tr$Q zTaS);=(+fz?~l5-bNZmOvj$#xZNT@h_Y<|rtvUT~%;|sS_5NZ@@AQG`kMw&iZpdRV zJ~5}Z>FrOByzx=*8#_=rxV?8)L(hzc9&w5(4c(LLyC>CkO{nX7v99xljX1)v0m3Le zu@*)c9$VA)Xie*=>XwJAJP%d346BBDnPXOZqN^ZWYuE~RNEL_&L&{u1m7*?YAu%^g z7!Wg1^%1Gnfoc~3-#-`H!>io?`Nh$Fr!KdCeXZDiX2#Z@x}I;pJaOsJ1&Don?9z?7 ze>?W|HE0POXRrMAC=3%_zIod?b9K%~2ksMXOc_j(99-raTx<-WiAt~~7+7qCR=7)? zXHQ&)=H%S9TO!-zf2p1wbshMDijFwCCeil-fs=}OArz>9AXyGd zut1!V2KIx{35a~1ASGHgWYYLa)iB8cOz$To&TN)Q69vg2xK@R{52Osh98fcccQI*z z*x+Y2V8KX)SrN$2qBD(%JXb^>;I_zIAY8CF!-!4?Oh#M`NMeptEKzt==pw*pkaIS$ zIf7^+hw%9>f^p&VoniA`;duZ7#jeoBF0l*#F91Lpk&6H>PaF$pfud$n4v?5A9uzFg z3A`^8{>P>nF=?1G!ytY~IFFDVrU9@hQW7X6HYYN{0=s~groqP8GzNme@3L{pg6j=i z0B{q_#@7{i1 z01+kv?*m0)GP6Hqc3%Xg?G4P>OFzkJm)RWd9g-~JU9wd65l>8G&t*sk{>7Z06@MwF zQ!{O$U25IbB+89YTJ~<1YC%a112vMUc2N++Ao#gAC}VFZU@ug+-vg@boQ6*-2&=Ay_7^2(UZ)d8%^az zIoC=k=4yoEJ(1EDBk^2Jx)cLq2sw@xN0={-SsraCc1*-aJd{{GimnsIWWiMJrPAqZ~8RRj=d|5Dx zs0gjO)Q=F;ER?(Cq$NZW#Rbxkq^_D&@KKpFrjPS7nWg|PTuKg^5BeAHGZmePXt@T+ zL`7#h`xHlDofDnG5Fd8oQ)4|~ip9Y^3J)OxDQQSaZ5@$AuE)(BuTZK2fife`gDO3p zDIt+WkyTVCPdNhb7=FSrtSu$?_9m)om@Q_)SW&ceItI##uOE%I;7bcgdN@f4-}{G% z(5mbk=Cw;t@KgFhuMhC2()6YLUc&&Ig2w!W6U{zqUw4_i420_)U9SXQqYFRLA-5=% zBr%ND121qrX^TnPul$SSr7}Q;YcMSW(?)>LKTraFGIf2VK!*%`AM}J3pNT(7t4ew3 z-SmW$G+$%_IY?Qqa@$wEWfizk>{13AAVr#qNID5AbDNN20FL)dwXYFW?DQ>m{%lF} zpaRE}Yuc(k=UPu+Z8~xJx1S#R?>$HNpS=9l$;(%6-WEnynfq)tP9l8m*6l4l-yX#o zkk4GZwYuYce1&_^GLQ%iDRl*uy9N|G9w>GMm%>z&zNO9q%N&y$y1zPki6$HQ^2BAa zfScFN+o^v)-h2imipMYC*n8sAM~5%W-q}09`IP7DtG3fulD;|~Q|sxzgZ|$A6wP$91&x|@a*DG~hlQ(uws)ZnC&y#5n(-wpO;+S>q zqt>^LT+=oZ(F8;uT-g#+1Ac*E+T)dmR=PvWVCLeW3NW;T$eF@91#6FdPc6eN$ zj!NIeagr72#<*#)CwmveGBl#L%(I*#F4@{+ky`9~bOy(Ze#e!RPv>hTO!pXUm1=rp zDI4@r-k=g4lI+u8?&Xd6w0z4rLpA20h8GAI>Uu*R;rSP{!LKN%zl>BEpt}iZUd;Yf z%_(tdsbx{`9F`u(J55yb?52(vHg>+W zvGe6kU6VF*Ju(dg7R=gy@bw*sUR=^J_CIr<_}`h6E53Yd#p>e17k2IbWYUzWuf`6p z9($j2a=%kE2cCaz;J2?0JUeS(+bjKR#@+Wy^q|*&@!Op{-k-PV#W%LLy}9Gy>pOd1 z+tE97TaQ>sIXGoY_ms_W#tHRZFKzB5&*JCSwm-YN?Wxr`x6$ghQES^onMW@7jKF{k z3DutXN(l5ODiLvwp<+SdOF z!X;XQ0n6+@OC9$wwGS$EhBlt-2nVHtWK*AC`S~Jf-g7cGr0| zr|Qig?wq-Jb5G}YOLiT5X|XHx&-)=>cv>^H`W7LcGMvEc%9bEScFGnqeGn8nOcdP+ z>_qDGmLz$P8^l3iW1FRyGtXMB^cn7m(?yHgz^SD#rSD2&>&UB8qe6Ojuq-+bThKBy zc8}_R`?DedTZ=+1BxosyNnT=5V0E%3pl6R}0F9Ieno*4mwr8;Ik%7r&ZY5k51gg%x zoFtu<5rrrdK!NJe_W@Rhe1LkrEY<)7LCp-uF6Hi3$7JJgRJu4z=#~A zhdE65P=)7;PKTU<$E6u0W<=a(L}d~yMB^OoJ*7>+eL~(QO>ALSdRc% zpAnVgWO$km|3x85?iVhVgFq+KD1kw!F&bea$S||<67al47)4DmwkLiOoU#|7VG3-K%nrPggHz0-Wct12 z_8XK2E9@_vDD{IWdleZ(XoyE)Zj_HW3*{ruD2@tA)QTYrhM5nc1at~2yeM88mMsa2 zM8y-kz{NP7N)B@whvu<~$b3yQtO#_!4Mz@0@Qd_uV637CE778e96LragB2iIh#F1V z9qRH3CsAUKsfVDAY))WUnQ$WDN~8(|RL8UWM&yzXu{lD&&d3q>?Bp*DXURf{reI~3jKD}ZY(8cV>KMeG3%G2M9!tUz z7#G;34=<210BQ@^Et*b8H-y%CBIAXo(=odq=Q5`w2ph^C#CEdD5$2*NHQnA^yII<7 z#;JIgW?8~!N2>O-6GqdgSazV}H6&ps?kjYfD@*tuY{55yJU5Ba zF!d%sVzB}`vQ$DdNPLocJxWp@Rt~1)3xKdD9%i&)kiyn_Ej!B2NPPBB?L=utA&uS903F+Trd?*A)QAf|zVK#====m;v{X;oo1m8Aw95+J=0R2hg|P`0oU%ES3X z89wCjYJN&ESR^|kMQlnaY>MfDV2axnB%Kccpp-QFmF)K`-5*f2KL!ySg_Q)fC!K}C zXB@k2^nhYy;J+w|fZ53xpAo8bOSmG{S z^ki9`D*Hd#d(vsD$*J?V>i(~=i&Id#tSm6qk zKG(=GT)J||w@hl#-!5BLT`_6fu1{Zl@r5_z2CW%)U&|}~PtO|o4Vw06W({;t z>Q^@AzVXoy{^k#V+FCd7kBg?ivGdrRk9uF*+4~yA9PFLh&@*#$_p6&gIXHPs_r&_H ziJLknYyhXg=Qd%`unjnE@fr{hj#}IL@S4_#R<#aCzrc8ms|e1;XxfKYy2bSlt#Azy z4;o2pfRq8M_eb3*20I9y{7TIzaHdXB`Kn6}<6G{c=c2%iY7vyEQ#=mw`iYiNhxi=b zD@hsJ7BktK*6HqaeVy&}u8S^|(esgYh^5D(_onpjN1~;>!hV7C%=(1al{>T6eX{gt zv`&tfc}|C0hCCCS?(y;BgB2#>QqHq#{K0$-TF^jtBEYc6ej2P*$xNx8BIvR{G85}F zpOjfq9mXtBIyb!wK#WvYNC>2 zj%GqX75DU$>wcvs=u~{u)EG*(NQ0s#3M?}rqj{HP`~rBJF_dXAhVu+t?h3DRM^?F` zR)B;MCPc6C3}4YQVnxfSs+LD5~8@2WIqDDP;_K!86kAH3Z(K*|D$fp=4B-sY=QRFYq zG6`$0@uCXb(E0p^j%RB-01mE!5r(lfZ4h~Ib?YPOSv+i2OMDGP9*n~oC!#B0ZX;2N zgq33@0y9jR6V%YH^U(%vRO94Z&h4Dq*uT)>Q)G-<)%uH%4t`)ewWaslPY+#K?mGK^ z^J$7pi0;NeB10g-b?6Hv)RP1C*gG+Ky4tQBh!k5S*hhVR z*tpan;UVe*9N3}ZIOSwlknFum|;ZZFoG1xLf?0V&nJS$@;$MU z=olRynM*1>7g0a-EGFrqGi0$_Y>Z7|nihdM#H6B-5FsEzNfPTtXQHAI9g5?U9pvK- z8>M~_m}V|{9~&T$%ypVRzgWDeY%UBjN7_!(MUoSS^SyEsounHDKQraEAX^Mw=%PD} z43Ti8^eo1+#PY2(s8n#ebO-Rh6sf`^td?Rr0;4K$&yeIjzzS2oFhdZgT2hJ?m_I~@ z7v#egr%Txrl(u(BvMCkhiA3;lIxr~BTTvK@!(MAgPOoF$&?e4Q!_gh^M4dP z;&#eSqz}(&B3YS=F&DuT0dvegVrt~3N?Z*7&sVc{unLdFZb*xWOf?f13tJ%JT8_$J zPIN0w#0>If2Jci-&#@$JzA5fAHETEnCP6fo(UHF`$4TUgK1kw3G5A>8mc!JH z@De56UH8%uT{6YNUz~m!c6Le0G3)&_`IjJaIZsN@GGqcZqlq03({MIN7G? zQY>nV=s?0l%$Puv*ktMoQ}{U&VT{9RbH!^wO)4`%Rsyn9t&|foDPxDhVvfhnaG}`H z5F3xxnQB4h%S<^)?Z1DiIX)L)nNH9|4*_yKI7kZjV9HdSPDDu3yE7=E2>ArxXeH7G>hi{Xcn9m~K zrI?b0fTPo8kik?2fAox46Pn6S#KePW6eJF(rlql&X!+WRSCm08y+zp^4ua?;Ajni3 zO2V6?r9Uz$PsuAeHnoXkAfomVj>;fPC~!T5~Ng(PNx8xQ2&b+>e1ajfS?!V zOY~M6(t3PipTqki!k>bGDu;R2lj^?1iQ0#(r%bdY^BNdPNc8Ug_O-q5qATo66HJ6> zY5?O^Org7Vp}Cqxe__xM0w1(lj!pJTY}37q;&B>4Kp`}LH;G|O%%eBKZ|T zeCj7FNf&(!zH%-B($xmZmmd^}!~qzl8%pQpD}Cw6LJyQ08h<%I0Rc+?Qgbg9V2NW$ zzTvmj7?5utQt2AMx$D389IbBsdgsCKb{zU{LF38Vj`JiW6o0#qUHtPG$2z|OgRbM7 zYw7!r|M%xdlD|A&;5?JM@5FQKJ73+>v!wadhs~${vgfF<;DlLs6($||QDpU}za9M+ zg@hy=$u~}yxXylZe{8 z%0VeBlLDw-rMckC!L%Prd~ms2nD)_??%3t-xaIByv8-r$sG7Y4A6?ZtdQIEt+P24Q z+n>Y$VmOzvY1&Ul_0KB}5JODZ;ID4)dF|6_I zEMHkOXU=Q0AB+9{LxWP$wEwpe56&Dr?3cg#)z*)9iFe^5xRSLxL336xyCQu!fTb8#WBD%COP@ZE zJ_71piRpuyQhR6_E>&F}>wxGze1A(!T%|9n+4d`Aw{V)52fulyZ|WH+F|AS&ni@k> z`e)!?5|J8yIyBTUnV-~xX{^vN#f&BD6~Qtvsy8LjQV$4XJ*?LxkJC<}O9&O_)8%_U z720|ERWr=Fv{g!x04~|^)3CQcdOcu123;(PS|!IcOsc45=u^(F@)oUfnHhyJw89xy zQ=d6Z~>Dvzd;j=G$PsDnT{M+@LP}SbJal_Txv9_)u zzocwwaq+4Ro0{D%-(QDaUtj7zd}7%4j(?rH_4w@VU7#GS?|!u&ra+le56Z!b_25=K zzOM6y%^fe)%GdQCba7-EC^seYm%tZ)XF7!MXY?#;J9SmF?6_}wRmRcj?-6AJ1AO+o1x{dd-Iz^i=45gu4!AkcOAL#(V_2(n$J93=8h_NiMs5b z`HdN09;X)N{56PRC~`Y;{`+AFYuzMO7#bq9noM`?M&PI3sMlo)iQWcSLZU7VY)qKfm|(4gyapEHbPElZMzvQK zyU}twv(jvg@8fQ4RzrAdJfuwfId~R zi`;?POE*t~Kd~{#P~N-53dv=d03QHEa32PA2>-L3qya?WC4L$#2FzjXD2|E-p`XZ; z1$0r_1}g7F4hAteR2m|iLx<%^$P6(RWhh~f6gg>hk!bO$DiM`#axhlvlS@EpY8q)eDk+4L16@&ICd`7u3=tED%(qFM z8hG`ZvH7wY=TSsI;zvAKI^K$u4DzD@yRNPR+E`9`9=5U@fgk$`%i*bhb5OaCj8J?3QmnWYIC-EQ(ElIEm zlP_n1EQpvag&Pw&QyTRUBW1C{1)GJ6c;Q(XzL0_sl2`Qtjw}h!B?pj!CEiw&**Z{F zuQ|I}2%t#`6%&;I(THu-TZf-yf?E4%YTM0zb7P#;2Fh-FBR5R2t5NmNbsJu9`1HQG z$r0w6EW@OSuo2r$v4GI2QKJ=&ymv08@!}Nn5Zr?Br70#&F6pcz!>@p&4aVVY#WY1p zG)*o6uE$Npn4;5hj))YPECL;kVL}olM@`~s!gjc*MoAu<#A8Kn2&Y4Cg(pqLEvZl* zQi{c5$TQfR+9ckSp%BTJG(ONW1}}p+8`NSVCB#-$3;ltPx7_{)s7Z?15f~fsGM>hS zkD|6d-;5?q8X}rT8M&UJ{PGmk!0|tB;<4jr_$zHl5e?x(Y1L>SGEHKC3okowciJm@_Um}Szs2ur>D{pib8>Vx_Kz0GnV0b60Vj6D24B3M?hI5ub)0$X- z47G<%xjBQl6?a9+?a`U6V5P(9ydslGIV=&$vy?RU)d^a^g+2*1K6n?I;ary7nqQga z6Kk}5ebB3Hw4!;{)x^E}7R!P3nK!B^rMerk*~RtZfOt93o7HXdzJbFMu~b@bIZmt1 z;;riG_(XHBS7=w_gYj0J5u0Mo3VoAfQ%oO8@gb9$ETiZc?7@c9>5}yXV5`hf#;mrm z(hZJ>QA9A5`pUMUfyxg#Y?lF5Kw& z=KHpKfkF1RD^Y%FK_OeRM$0W3nId;UDLL8Pv6!xqoH&5miB24NB-xF zkB^=j>N=AA!)<_}-(SDp?DnkMShuveL@aAI*Sp(0Z{C7?c>1Do>QvZDSMua7r)Tfz zd==mz0LN+dVA@aGf=+>K+5;SXZXLkNC)dLa%a5&Z8@&eRN`+}jgk8TH)Jw57o~Rm6 zOto9w|L9Uz1j5iE%Y|w03?x1%L(NK-0(UZiUlJK3jxZ}$`1l~007!%FYIRdl948mqf|fSr|+(|;LEFCRKF zC48gkk^@wNGMTvCfbg++d)2dmMcnFgBdEd%UhW*a+!?yu6;|zzsCGqHxkWt^v(i1R z+Wk-!Ai_shct+K9uX*>wS82`*WK*#&77Du(9LiO`Q|#I$^Y7 zOV=x+9@*0M>Xz=A4PDc>^}g|ur|sC2fA4y({nT32#NGMP`wzC<`r&Hh!Q?~7BPxx| zNe#zmZtI-Bwfoi0U7`jNc}^CUR9)wUI;gW=+|(h?`^+Y;L7rIKHgKtj;UJ7K zc;Z(=Z5UhSj$ZDLs&qw_xxz(_S>YT4R21Zs0i}FnY1(+WSF{CN5!FPAl?N-i3h*!H zC+kNk%V7TU!G-pLOYDP|+Cz$+v$ytS??3U`kqh>-S6jZm`Uyb5(@(5y^)GM?fexQz z(30k~eaFSM^k3SXx$nfpjh+2Iv<)eOiBN(H9q;ZuxW4QB{u7r)qq6_>m5s)^xwX9$ zm%1b7HHN*nKXmT?n8e0mDYkg*-l=#Z)Og3Hu$|)t{^GQ*1O3#Sf40H@*78Od1J zZs?lCcnJ#Q`f3&RYQ^erWeTm6xW3TqL9<(<{parNPhAlYTwsOBU9Lu9fg`FzN?72) zehl}&)o3U6$muC#bK;=Kt=<9t$tDMP{b?jKu!ws$TtStQxeGjnU2UTI0_PE zJ3&d8>WIoDEC|WM&=of(%|WpUNoEK+=P=hptY*Zdiv$KoB+P?3KtvjXabXLbk(fZ_ z7A{Vj;Xo-Lh5Qqj;Goh{fx}?(5F-|`9c_$aNg)4(#Mva4G-&P=@l&b}DNlz4mwYVg z{@_|*QWSC@HsVqM4vNI0V2tx*K6E+^stDHt!3y#IU`$3ld!8XShAf6ShAO=X0>UFpfxQW8j6*Fp5A`E{gSX5k(PfTF8(KW<_`(g^C=5kAozl zube0+8H4^r?rO^XY7asCH z63JWzOc{nEu7%sEZf^uK)(CX>6Xy*DQDn0Mvh+-Ol54!@A}i1&5=3SZCEp*hD@@L_ zj28%4Dn8L-g{pQ$AH?!TfqjO!yavN!^)P~cv(-AgdJ7V(H;8%zDt-NEAf1jv7W0nXR>DuD>U1w>b|Q^HYx#`4tV(^(~8!N64Bt^_FSSHjez_zn6|J}Fu^-m zFlZnJ8BDRE>=$2kU_C4tj7e(}Z!FDUfuZ6|kQAL^rlEp@!-aAJD)X_i3r^R|zNhG;I@>L=_wIM9@p;ZUg48s6?RbKfoEG zhk$&rfJXnir+0$*<%qMuf%tqOuv}p+B#UX8eF>iaa+` zhVVndCQM<>lol`|qFV-q@Nh}B2Tu^6P6Unvv6L?r-W(M7@1 z1urHFln2UV`Gdi@ig*Jyr;bY3!*^$!a=FSmSstx1?* zJ>UGto+Cjx9fGhK?=QALwzh3S)5!-*9b?zE=QW-56*(6;XdcVqt6CP?PwhDPo%`HX z=h-W}k6v8ZeCmyDJrPyz0R@f+K4>0PV2AP(>ri~4l!gSPrH@90ucRLefQXG$rI77l z8vF8w`$?)0DDF?GA&lH1mCle#XIPam?Ooy3uE-T&4iBqv4X<=3tnxgx((?$+K?

qYTDnpd13#7KOH*}x^Q3Vq%Fs1ZU@tTT0_^A22d0U)BfdpH0?Kc zzOV^YN6&6-e`YPV1w)O#f@%p>SbP~-EKThxbsBuSaJvP}p0U|uE*5dTOO>&=a+^ZJShe+r3 zG4|(KwsR$%#FZzX%i6z!l(PEusv9kQkluA)Tv4FVvZCl=83QtZjhC5^M5}kyGv>3D zFq@dRmge-5TPrw z${k$|-=6pa6IOU0s`iXn;la6#TSl*L9lZ__;kve`)&U}Xwzge-&*G~dzo}yagf;D) zSl9JR9SRBSx@OeFfWy>nz55PK+TSyAyXT$b-|i9h#qB${Z{Pmm&YjzLXa#GQwv)v> zJhP6R9G3f4>4f^@)3G}IvL5DuTfu@*y#}x8p8J$* zkE2LEQ|!!v_R|2smk;C538k)hnC}Htih<%`EwlGqWP5OlT{LpR1&#qrny1!x7{Hl2*zX!T7o_G0tR`a3lc00fFcL=CuoqL8Vohy4=qXwlR=Sq z$cJ_s2@o4;sFRHZ?te&)0GbMFp`iul^<^3u@~Vmpk@TuM7EeAMhS$yp$lCRIdVRb8~}gepiG2Q zMC9;UB2j*b0sc4)qWBq;YQ!ZmwiAh?K-OoFJ~1{Ka3tyk*?$?a9Tzex%Mp?7jDopM z@IaXc4{^ZdV4kp70zt6_WQJ)D6i9N;bfd&=A+m`f4iPCL#e?*zqLBtxs0v9pgu)ln zb%Ap+!9)atVGE>2k~E2FqNvPXIdgH-at!dpm8>lgx!X|k6egt}0=^+c43n9%q23UG zrZ7naFqee?Q~1DOM1es>1|=38nzWm!U&y@O00)yu9w?Kg?g>O;AmvQw1=r(LPDAp; zp^3W*ETWPTi<-n|M_h<2sgfS1b5_Bao7f(j*F?4%0i!US*&`G{Ay{{cC56)*wa9vy zIxPe~D2#jsm_l#{UP??EA|3*JH44=zWK05aTL{PuEYfTio&diiFV}2aSRSKEpmhqC z&2J8!4+SGJ9ZL52SS&NnpPL5>c>0kn8=r!S8~0LyZ0WLJw7 zn8Ia&Rv2V(Q4*d;R4Iol)J@2)rV32NWfY zMgdWnRA|C^QQ!l2dogP;c{1qvOh^e(5}V^<&u7ROx`3~gcx4DZIJ)Nx_|niT*=9Z$ zC9VYc$f7d_lyEK^$uPx5g#eLcKV>Lb4i}6bs5naXalJdQNqriiKBUjm5F~l`O7-5V z8m}@3t`hq+WY%@Ft4%G=JhQWF?K-2d{&Xj0f`eKk85IGVY-cIKD}KSc&;fby@?&_!Ozr2 zV~vCkdBjl;KoVYzNkwQ#Q(mO=IHHoLMW=ygjj$kMBhr*L#iZ~GY2WDXVT4fI(8YFc z8AV%Mz=(}{#mZnNBo`n7oVnyX6bpcczJMEa66psLGSKiGHr9sbt;`aEz@K*d~EoLbx3N zqDH^seYE%&?o%`kZ?BDzE5S9Oc!>Xi=w;aGnbJ@MQC&j4h>-{F``r^iTYu{dVaqm(A%C2%;qIXqk z0TUV{h?%!94iv#GI8demy(&6N@FEMMCO55D>;Nl6n7nXk=-sc>P@gk&s+V$a9Eliz zGDO>HG9ps{kU1>OEhi&?E?zz z|FX3CkAFS<^X#W7R>03ZNKL_t)A=aIFoqt>=QzPfen>b7xf+s4(lKe@5v89;>X zY}z9toLJvAxuI*y7T|i*wsuZy=$W~#qvhxmdpf6o-1hF7OAavYZ(B@z49IxHd35nd zo*73^4*S#RTPD>Xozc)aZA;fH4P6trfaqr;Iv|eU3>e|LO&!nHwm-A3?Wwf@8OE${ zdu%P}enzfp9kIIQp_NePc$g!>9V;$pg*$4wJ90V0!J(DTAuwIF5rh@2(r1tcOT0ho zy;3&5THz6(0)=A91{Ck4!(ZM1GLrz%s1iK5LSdCOQ#U{fbaYcxeR|$KmPE@ZY5kBH z@ag>)ulm93!{^MYYSKF%i4x%@TFRq2H(&?my+*SRNmM#$%B@k2sd$4z&F{e+?38Fy z8%I&Rl4*cA>K?_UbPmX$aVfvh#MGD%mhw-Zaw+Sh??)XjC4LIb@z*6;7$v5bIH1t1 z8X$48mcGm^iGPt+qami@@37e1PA`y8IV*|uuThudc|lJuLm6L=ztk3)6_zZ#Vt;<} zj4BfvVDTq8k}LyVtgp=3gsS*92Nd-wlJzpV{8BX;SZV}`N~8?FE|3sbxgx4ukywv_ zmvOaw_;UA%D)&P*EsxZ+jH-ce^|7@Wh;VJ|lk3pYxVG(Cr2e8F8DEFK#+y(`SO;`( z>Xyz)^@p~%zq7x0;%3)xKW_ck!{`5c`_65Bc;Wk77d*#Hcec#g(eh5`(NTZ?eA7#H zN2b)W4O*Z$Oo_9tOhBT=KZj#<+>dR6Pl)c^-Y zeK&lCCt;;0p~{_5;fkwpp=U7wyRZsZSQ%VtFihW!a8Qy{R+1JnT=pxrzOmZFrMu@N z-#7S)iK){m9=h*RyH9~*usB52EY$fcWMyJtv1nbIXKn2nyS{Bei6f}YIk3?F`;QJj zT;?7u$Ay9m9YgY=lj2Q5_w?@EyUHmds)^x6dTF$XtvdyK3juL37o zvNf!_(`M#fQnf&n-BCE(OM^px={~|L14ubu-WcZcE zfK05%kT4J-8i)k(SU4&Z*%oTwKx&tb6MCql(j2iVkZpJ_60uy+DiSUPZ5@J0kpM)3 zn3!NC=@#+S4nmXB;&j;#Dj00fKrEQd3ia3z27f%XN3q=VdvKbN5v?0+TV{}AiOMr+OhUdW|ZY-QPi`gq|K|YAL z!XAyKB~_+kvUv9l+8#}XK{y+c4gST!DZ3dFrb^xiIB$1QD)GVH0M~%o?G_t@CCdxu zU9eHCP%^}Dy8+=P!~bARK&Xxg9n!tsv=V~rB0&%<6L)g~s64DO#GF}yC{}iJkyL@9 zY`ceIKEbNM5@H5s>3*Qh@P;qo6r@S)sDnHBeJEBHs(fTQ@ZqG0EIw$ziD%9D0^&T7@Egs zzt}D=7A^tH69E!JHowD=!6BWrk-U-dZWdn=5kq=*61L)<%xNZ&$M-zXM!L~Z(mCex zf8I48VJ;v=o4QMQIO!#^*qL|<*#K}u7V)bRiWHJ)BfV65BNC_P&e9CgSnCq6rBuzF z*(7PLhS@$U`__vL>cSjrKU*99lCFjGH0CJVBWiKXZ0E_n3Ju6nt5{{#zo(%MvvHJs zP<6G<^GdoQS9DIHZ^tfvBf&cWg6YiZV$jKyhzxB6%CG!7F%7*@k(mJOBm;t{qfkzq z=!B$6$e7YZ4#hEu0;Bm9kQH%ibSs9$gb0Z$lG}1L`YFRH7*`3Ksu4AAO+yzqlRdBO zx4=RZ+dw6TgGTWvP;iX1FfDx@ujqL7UFupOs-%(lCL{|8Ez z=U;5GFar?n1uunhL}k}NCQHpS&^RLYHzcx@awl}3#0$xn`4{mOVFASD+kp(0@Zw*z z4_tZ?#sL|Weigvx@X^56onzw>1+3qw0*peDprU(X7s)y4Sy06Ym1kO$av`Ub=njn9 zL;1*})A_m63jlO4g&(vceUap-o@N4KDr&TRH>mu?5NV;yp|en%LYNzbjk)k14*Sbb z@If#9*M|dYFkQof^BTQ`gb7|-=9w$%yED-te|q>un`W~TBw9b}MD=xRU1hDWt9QTk zjb4kHMmoxze*)+uUB+K1uG`f&$e%!w!(U0Xir8#t#Sdt~q}j`?Kz`Txtc+1SK}(|< z`;eRbt_x{ybcdoyq#Sy%)Oesom~W23Vja)rU+xr(I2RakMUK!i=N~>k7>wx$6*-2l zY4K~gbDf51BsCMp5bUKqa-m9D7euBd8vOf~3c;#atZnLKPI zs()6sj1+eLDlnx%{R4=wO_Eq`|pl_`<3P$2rE}^oOK*tyv_6aCv9)H9(nvV3U9B!*IPS0mvqjyb$ z9T>CMF<0xiB+B}2o|&7;92E^Tv7kBpX%3JiY9lUYZ>Q-V3BMeEWfs~(<}orae635^ zjhD9g70d5jecmO~U$_JlP@bg@svvl!l+Pt{^JO9iS&=B$Pr*m{xst@3c^7- zQ)S^(&Ls?BFvn^jiTOxj1S zW>mT9MAcUbFePABTTC8WB5zRek2Cg8KJKh(*};Of7^XWI^)gjtQGOCW>Wq=PR2!r)_E&{buLm?CUmPl;HO(aE1q9n?aEW6!O@0K05 z+mhX5S+cE`tQLVx?0an_s|r9S)&goTR*@7*i85tbtRig|D>IQBfLgS<+uhTSn2Gsi zVqzj@ewd%;p7TBD-1{C-cEk(B!^`{byKlSip3nWBb6m}w@BD!~7Qw;mG`-;3@a*Ng zPhYzG)CE=!UcUQ?>+d#qkDlA@-`XV=euN^| zc~#V!i8X&K?UTb-xGJ9Md~*|ddnWa%whtTT!+E1UOY}HtXT-p$+O~vQV)xao$TOLz z5irllk=V_RmI&mJQp}@1x9|#m(XkR+50(M^r4E7@EklEHY(KaQN{;9*x^X~4FGYtt zTxQ_Vg=QXXj8zyIF7#)E4#K*L5kHV(CyoR5l2%10XorgpyfS(&D6Hr1Ds+YreRmh# z*o77#yapa3kUem^;zst(SQd%#(FzY)m=BT?s`Vv|V1N} z!nz&)i%|%wb$el7>0bEgm~riyHUS;4vo|XSZ#)B;7paocf6jSKw(Jv z+;QE}Dg<{qUuNE3L*>2NFmOq?SI2Q6!YZ&PP7qtV z=IhV%kr==2ivK@RDT|_W$N01fY$DJns6?CF%m;BRB{Wx`T_; zNo=&l!aZ)%0!#C_&9LzKIy(cbgf{cJ&_c|Sn?d3l_6MZa&QM7@=xm7rpHNP>FCA!_ zex=qP7tLeGalIP2t-zyvbK6gq-37slyICE_q_EumdAaqSS{ryGod};P9g$tRyZO%Q zals|^2F>zQF-0fq3dpy{K1}*`i+`Jfuh>BFb;C!^zH#x7ntl;tU#tF+;1gzrM}ykJ zkKIV5WVo?E6RkG79{VZs`2z|ZE?^w*w|n|fpzkJ)*9$sumv87`Mt>3>QBbXuTPc76 zxhw3hB)}h_Il((ZOGWp$dq&@y8HmD)h&;&1hISOa;aJHeV@!gCZj_lh8Gtb^tf0=Ov;4W|+TdQ$L^FJt)uRt|gBlc#%N10W&?DL* z5?r$z17LR0Oh-&fVoTT_Iy#o6V6Z`x!8Gurzj<|$&IzDQQt-hKWm)|&x%?qr@KJhH zdidh^@k857_oe9iUf07=PeO{nl0~BQ40lrgx}Wg6@ii~vA#oi=n$aFmBZEcyo1TyZ z5pA5_p}Xel8KaxdIFEj!kHs`g!uhYO5Dy{xf2stKk=rTd`TN3^$gKx?bY)UA?4itM z7-G`L#Wi?vsDSrEmZ5@d89K07flR@_P5grRpL|vP61f!{ud38{AZpo~u5cj60UtIl zjj_T*m+V7#54)*&?-7iRZLA*armSbP(LP50)@@=XJ@R4SXd&!2(GN>+%!@o}{43o= ze(T+>!5gHas;U@McyrHI0K4S{%)zLs@KWV5GrhTH>ELy6OM}|8BJAQhcv#j+5de9M zTL#M2aB;W>Y*8#IcD;fHG1!k-m@+m($#Q{oDKBT|?R^6g%dl?vqAjsax?XEp zt+sFgvGxt|4(#3rBHZoY-UT8g2f+*bk6*by=U;l`XJ5GSKi}B@lMjFOo7c#+|D(6||G4}=*8lha``-Wa|NWo;*Z=#E zzx+S`>Zd<{;jO*jdt>+Oe|`UVZvX1H{=-|ffARXSe(Q~&|Hd1?_>I>nC>e>0ZaiQz z__I&H`ruRUpjYpG;ws?RGS8W;K+T%K6qNc|H(KFsV;N z0?c?xZ>_o>qf@|hpe1X7iQz%f>0r1ZT>}Jwd7GK6vLHxt-l|B1aHiY=`Hn`d0dA>nH@+wjHf#P!_!|D{*&KY#5XPu+R3^!K0NF28Vn_s?(Czj3|#N3ZXG{q6hT z`@!A+{965MukQZwYkS{#rTXVT-2cN5e)(HpdgJbIy!MOVcwJS;um9}xH-7fn8{|_= zaF9|O5gh#ZmHSU$yZ_YXyH8!DFsqMUynFQG{v+G_uC)W_cKgolrnhLqBtTRnOx3zJ#mh{h671Idv*Z>^sO;io6^eJ?Amc=mav7kcxDr&S0#0C@k z{`Zae7e0h>-slQ@gkiCZLbE4QEbDu#a3wyV33{~HN_r&z={S%qp+pyU*v%cNVO#!&u~Nr8+g+&fmaLttMDLCB}fG!S0W zaum8Cvo$Cpff`+f9Vb{;IZ5k;4*9G*j@LVm)l*7Qdl;m8jp0nKYqrWtJ!W+il>@fuahsKUa+&wKrwYuQ$Y|_D+&!UV zVt=aWjF=HA0V&Cb2-zf8#)6a-8VGS5^ElX|T|)Xwz(#I&mZXtc2(MM*Uz1b-v<{f8 zTXivn%#Imc6OKj=n*&2s_%$z(&U*we2sqS7Fv#-ovtC6Yez zn4mf!Hc`^emK1B26jt*3oRAt&OV-eNMhU-NoE=nkq63;MlkX?p0#e?&>(j>Aj*uUh z0b#^ZI@vK9Cn&dxp&_xnV|?z=56(j;h>KI^w(S5}f{@}~yQ6f^d0k{Tnm1cA*(MIu zrOHPDUvNoxP+0ZpDSCKXUo7MHnXZ%BaS6K&@Q17~FlD_RdvGS~z!$A=cjrgxmpY0p{a>=(Q_C2x!yqIk-yGK8817WqyDFzfzgkQA4!_j~2NDpOm zX~O|w3_gp2h>x%i=x`4^5i^-39w!(`nP3k;PHgs<_{8C%%q_~pKmlh zEdYq3OP%05&ievQrpM)BfCGD)skA*F{BR0@=5wJf3@GJMp=BJvkv|g*9FM+GU}4~4 zKrmp1BJ|UZ1H5Pf=nA#iP{0Q>%m;(r2Zj{8%NrUQVal<>@n>ul8htUnWuA=|i!s62 z9QlDUZNvhPb!D*;cx>W7oIww)fZoi9ec6J?8Bwrk9dhsJf%R;$?WKx0Wb|oIvO6&Q z1?w1?UkVJELFYYWhi(g;k>4+vV?b#D0&e_lW;?uw!2ucqmq8^1Y>ug_gA{qN-1B0o znP(yWWG;+(2`D6*o_wt#a9czh$OYZ!#?)7@lM!jc~ zBJ%fc?UIU+C4^)+Um}Iglb7y2byX4JHB$dPcY{I!I@9}e*B^ZTl?R`HjiLlP!`zwY zzxC$*PhJ1t)^^59zxu+~&d`nepWoR1#`W4Cy}tL2x9`31*4=NtTL0>+^*?@n_fM}@ zzx9&`zy0=uKlt_6e?_97*CFd9lKH>+y!*$ z2_?lwjlir>k9%BQe?|7Gj#9jS#a_c;Rx&QQ$wXe{_;4kJ`&MVz!(`*zV}l|bVldk6 z3~1LE>2@-a=EZuOa8k2B+W$nFG%Th%|5}RdI~*-PEW;t!jE_V$ZUVNGkVoaJb_?;n z1Yrr4bf9ux8@!CytOg@IAqhsLu-AD7;RZbKgZS2aiB_#Wi6HzO-eKd%sxg!{Csw8J z5HI7L_yQz?HynMCY&Sfo7*v3xj#nBG-ZC$$V~#t(F%$K0yokD3`o}e|MivcYAQB8G z95@k=ciiR^*P&8gAN0Ij2a+PM)tWXbR6*Njz5U#7*XC|&v!2@8>)qV#-`pMC*rk94 z7xxJfUfgF#PJ3t?fxfSS8|=g5q{)+b0eD|&WufDbahhMny z%P+DH@e5qfXIVD*{44hRc}k;=n?U(0&sApsfhZ9p7r2e6Lw+Cc-#!lVyoFMbkGb;zWoRi4T$! zv2eZS@jwhX8n8bU7^lixRk26`2ghai)-n32X@VOtp^C*3(hpL+zu6L#ol@5ViQULc zbU{%agE=rhurLBlOp!qD5KNFTkZ8?Rmv>Bn8ixo}r5l$aZ5=SvtROaV)K%Bu)U1#= zEW!up1uT+Kk&#w<0@2e-5WBhd1Kx?=joK#2Qs9@OPdEkR1Z_KnisY0>ioYF3b#g=) z%%+SZK1gjY$yKtZ9-}j2-{Bok>$s{;g^ZwY0G;ZG3t=E*yW`@_OIPwl89v3(Bnlrq zF3kq|)F6x{P(Sj*J}II_iV4Uxu!nH~E`Iw7_5VFC%n`CdC?(~tu_O=*N&~XpGXje{ zkHe!GdPC4CK004U-cj5J!5?u@_wYzJ9{akDM80<>d+B?~uYQakQx0Zz-(c9?7kuXW zgYk5@m;bKCzCIbL;c&grfxHFzgcvAxZhv z85|EUI6feaA4tOmO_^XO)(=hzmjShb3M@Sy>^@$?ufp|4AuJM^m?*y}EFH_o zW<p&h2#%n~X>u&j>EFkFiQ2bB+|nd=cg6CGd4?u8J&ana?8G>;{FBISceFl$J0?{GUr zUdolE$3-C4J3+Stch-O%hpNO;M0E%;VD}ttw+Ae3qQx<3gF7f1DW30q>oI}_YP3dj zc|p?f-?b@5001BWNklwLF7yZW^arC{5Bsd=8Hsk4he3z_sE*0Xr?bKb zX~`VuZ`Jg`MtS|mL@?F0P}Yo&ew-{XqdnxyV((+{4A@7ZMcK**mj@ycA*~wVBCK1% zuv}dOHJYKwgVd=OhM4#Zhpuv~V*V#JW+<$6aVuQ&8XfcoiIq^bqdtEY^-Il0b8}8A zxr04ueT8eAiP!j8Uia6SlXXNh6UEP}uhzT$;li%SjEGr_V2jvG1y$P4oWK@bkmg*K z5FzCe-T7MQYPD^(*0M&aW}4JX574H;VzMcOF^Q`{ zVR#eUb}{e;1>h(6n3|&8jd1z{mxaEO$P4A?ILCrv#z=g4J0Bb{w7sz(TGj`Q>&R@xi1b(ZDS{fU*ulw+V2;E(G|>+V^-~IiFQuyR z{5vcVcSR=UjD;(7m-iu|&^@8|9(pbh)but75eBiZF~mr`(uRPb8OZaH(f1BAdXUb@ zNa8XCM82arvL<{mWJA*T>3NBWQd{D^fVB+`-q%$)sfa5Ns1bRp#G@{ZZ=|(3ksKbe zX^NFdDX?rp#S{B@de`0R}bpS$tk*RS9I^eg}E>+k&Vz58EYE=;WN3~%lXZI!>b zRrxR5VxO6-T%xL z4q5sN$p+mKDW%bsdrw^;I7l%EFVghF0QVqnaXm=S=3Z)(?A zTvLB@CCdp_XeL$bnp~D=RdngrYwP_2qQwPI8fION06~GvGS1_;b-AieQ1B@8jP(1) zm=d9WepNNnSs4dJie(Qw)&X_lAxpGc?p(sL zj$1GETOSlCEc$iu%vEYY9BVtNP4;!P6p*T@`sDc}1%4mC;(cWn<48-*ZgiWd< zSiK%9c)J_*)`*!Qk&i0AE5lFxYf{Y^Q8BA_y^&)b^7Lug4^%sjRpB$6o~-tdRwOmB zh~@~Wbsi%=$ZU%MRtIO&AgV>kPZ*Xf);%x!KKGL=X_U zPf$Is!6}wZ0uH9f1=_l!aBM{OFDA4{lE9{MS|A~WxUY5OW;@}$TdmofZNoPq3^Xs` zwk&H2M?F$(%ZilG7Q%Vvk__i!llZCFP$Lqm_D~^UGjqMB$y;E2g!vQ^dM>Fr5<)_^ z_xe`aIqm}lPJ8tPXCJ@cpgv5PjO9l;D97H4t%D5Cmx^8wo zW;{K zGb4Hg^0)zt&-hHWydg_8iCXR&NuG}gr6+Ap$mQ+P2U6#~TNhE&UPHwQY4bINIOQ}Pz6>cAh zE$A)ZYeAoF3=_MjOTsbDn|NMsmY_y%Kx#^EI&Pe644R=iRp>zOf^_NCF!`aohCR#& zL+TeZbO&?@-WNK@g_v(7`VrKXzUT)i{&mQPUzy0M1FWDfllIpQ*$Ldtg2Wa12~-DRc>A(@ATj+Zk^6Bx zJ7#m!5O4|85h2(lZFePstwD}N@HHu3%zXg^a^(d?jsYVU%^hBSic2YVhX+LNRgGH; z^0#S1RE8mNZOLI~gIN{jn&5fc~`D^MNA*+rczQQv1kK+z|%x{w3jNhPQskB3Lnycc@5 zZtiVd`N0?XX&+L$CrZG842G|U$L@uPE@Hd-I(-cp@7f<<$ppI}oa&Bl>5q<7z4e~q z!7y{cfoyc&%ps#$9ZP%27wVSAB#ricaWyq}6IGN8ayPuTdbvjk+_9)0?}! z=a6H7QnDPqK#1@W4&wjO%Xgh|{mkWiAHQ<%;|L>0(j1cbJrAhIDM>&7{rk_|`0wBP z$&C+w`Ngw^sg0c>Gwos5pD&APzfv6i>wCX_>h`fuUHk7}ym9Yy&aS^sru}EgBTz&? zA7?oDj5F;CERX_;Vh&!q`zSj>JJWvP+-^U|Pwd{_>)PJy+}iD0t#{>XZR_=B4jR^E zm}R-Du7To3Dr_XLj;HFz%FkY);K_;bJVt>a7j=bQN95l9zLCQyd@yeD-VagXOWeMG z*sx?I_;o~V;qdhzc*tNz`<#L=9Cj2V2TEo!-KN&xqgHkcrWZf#_|51xO5lc2%o)t?-LjAed5YJ(h^=LP4cI%KY0Gy zgU`P5;B!~+efpJO{@XV`eCO_OFW#P9Doo@GWBKC9N-4Wq%&rwO?(a%5yLu;k=7Z@U z?*HEO`)8iL^1puJ+C8`L^KQ9HWbo6M?|qWHs%NjAozo2?R|8Le0fTSY}_KNf=>beykE>e|MVir`OVa9eigG2^Q z*h|&azR%4_V=!Y0cO+qa7$aD8LsuuL^`$<(xvK3Bfu zB?`(2ZO3n`WGCo|`8J|Hmp>QP9H^ctUbKUo*QZ1GKY&93pO zF-yheod)QX_>bxFsu5BmuypmRl)i(!h85e5ivO><2^;<+&}Jz|OfaaPzu_*;>P_6N z>&@z1Y(Brhk;DgwZ?+Bd?jynDuyngl**`?ib*hY0+an>hcnodvN@ZVy(P#-+B- zxgvarodMs@V2C)8cmXp5niY`O1t(K>LG9tlQ{5amPXuw`L5z)fz0@;7NKtv8fvQsH zJR%s1bDZw$*%qpT2XTk!4J)lLmD^73i0W`oQ-#dHN!jc8QcTT&x~M&~Le@IxcZ@c6 z2Fkd$Qx)DKO=7OMxuGO0wT^iRfNHOK~x~6mc@YW1nuO zinz<&3xab*z)G`A zMcfyauEaib@N4Tk@KJl{IQ!tVQ*^Z5p9#Ji2P&~IniSB_kbGnheBBJ^g={f!Gb}{D zbisDU4RC2J=!ehZ4@OJ+MY%@CydVn$FBIJHfcQ8&7mG~K%f4vv0B?PQbSf6J0^QRk z_@A0qD6P@uAUG9I;c<0wMU1&IMmeMY2Em;fNJvy;%qU?piIXWDRHr2n9{lKg)#Eii z7V&MQh3{Qzie9Bg*7#%O#%jTePT*CNK8~||&4K*j!)trVM&8a20Tsg#*wVD_>oi~Be1|GNUkD>-~l|QEtBe`DIUjX%Mm`v`a#j? zA>iYfMjo7$jRsqV15VH#>{am>Lp{A^EGg}o^Z@~h=#7>HI7&pwVv2ip+A<~&19dp_ z^icKl;d+z^n`SW0!o$>*AnBe-sVxniS!x5flGPvf=G`#pGnn~u2GX%t7 zQLQkxyf9)y_oj-V)o3PslVF)%J+V{^@%J0@cHZap1(m!P01bb z`h!niCo=eP&L!~7B|74h?$8(ZAHR4Px}T%xDPb|0_5?I{*(q?Z^BiSd>7X;Jx2)Hj zHpsM3ip!n%;7+Jf-*i@pMqjj|R7?A9>{b{-fvbK7N4$ z|9#ZG)fewRbLs9gluejkb$A@pgu^QjK6~Z*$@$c{c$8WxK{n;!3?fGl>K6CZ{XD;7=j?|Qkz6Fzgj*j@aZv?J!FyKg$uxov6b#^ZlI~I1_wRP7p0|XN1x=Dd{6q;iXFJsS08U;?-_CBZD3cQ$h>GbwREY>WIuHVb|s+28&I6bo}H- zLhUEPo;V9Iu-82O+d*tgbao`QBeflXAYrc*XASP0C*d0<-X)qw+q(c3S%OF4Z%(ja zH^aLgmi~b%QhY-owwY@8v?}tB)gU=caU4L)DU9-69Ip_~VEV>2PqWmHgY>z|`!f|H zhSSUlJ{MTUrJ*#E2uPuV2jhD z#Dt2IcATnqED#?w#@EbCOZHYPXkC^TDEl!>&>ws+3U0-{8r&87qnqI?H(6HklC zj$Kwse+QH%?#45+bZT;8G>odHP=LJ7n2g}m~tbIGB0f>E25!ebzVo{ z7EMQ=7S@I^c?=CD_FtD)LYaKINnY>>*J<^+c144{jsA`i4;o;~h zxJXYk2A!4bMwRow?t-_SqJD$g!>5c(CoP|$v&Mbtltjkt*E}WZR&e749r8~-95sy3 zr6gxGCl{HIOR>zl<^lM2AUf+*pJE93{PX1J!(ur%9PX_L zk=@3X7L9@06x|&GG3=QTnnTD2SZ8WSL}8;vtIY!`zSKaCkbsH0aFnWYi=jhQ&NUNn z>K#Fogf>E<#z!Q#Vj@r(X(64(La zx-4n1`9%!?%|6B0h5Q3zyR%JB#x^4lM4*_4Z22E#W&e#q4plTF$8gd3f{!t$8|52n+D|xo zkF)l0YDexeZc~;BT{kmj)Ql?+=VIJYyw2F(!;F&emj<{T6X#_H)AEzBzcJ|&-Fj+D z>O|+`16ht1(T}-qB(3)yZDTFrj(Y|O5e^-`;HN=di8I6@Wq%rf(7!~nWjOq|=y&u8 z()i$k_@RC&9NKU6KV(RJa5Nk5onkh5v~vsR{s%I;9;mA0#S{ff{K3FRh?oytVXw{2 zXnzYn=-TPJdK1k7#K?R(k9!k2|AGkELWqDfT>fmO@vOZ>G}qpAT87zPDq%)(PT%V@ zs`Xp;?~&No^ty=!2Tc-SLL#T+XjvNIj*Fj;Fha7P!R>Oy0s;af615cS#5lrx=2r+w36;$;)|}EUCdnS0PBr%ZXE% zg_T;%O08{$)OH=~yFi3J>tx#ZZ4e^t+vZq==UM%;MZU&QTw>GyGAAveM4+74`0Bk+ zk_Lzpgns7Q{ZC*0#c#e=ePi$UU%Wkg>h|2rw`Wd&IDLlAd0B2xp8jC!)Zfm%UjM_d zzWw$muKfLHuiks^I%Oz+jzPh**X}>#7H8dGzK2AkA3cAU;NW>mNJ7d%XWetm!QM@n z_IsVIULvQ!b%uj$0tZDosMfbBz(7lm=yg^FPk>PDIxj1X<_nG|&^^_zQyPdy!;ig& z_OcvG24hU$uLQ$}{^0|^#sJ)~77GRgv1a%&h7}mr!A8g;VLtobDcW}hy;j(~zUWyG z1p6LRJ|n%Qx5_h-LwtmvPGRuLoY&S$N(aF8WuYW^QH9587_SJX$!R3GT%dj98)=_# z>VWdH1q?`MA=+M2PmZ2`yTQypZzDt!VRngYDsyb&oYo zBW7L_i`oO4U+}5`L3wr+{2d7hm-S5&_t5vM9*v`z>lRI3OQ)dkgEbq~0ueTI9*_2w zTE}`FDnd~sZ&JFFeu`PJKX{(NoO^*O9O3!>C(rLcby3`nNtVoA2~onU_dj#--lr)c z(Jxku|K+9IQ`7IgwD7mbkNxej!dg}%9F0|xWsb{9I|xq{N4aZQlNw++oBxoIww7d>kX+B zL5#%}1+UzdH|Q;phiA#(HS1ISO}LE+A}xs)$zN&<-eSAPEOlA%F}K!okGI;S(Kx+A!;_tQdLZeureq5udL} zk0zQ3aR;XD;z`jToBBZgBT}c%s1ytf!YZJS##GSR1CynJJrzWaE$oh%a^49535MyDYT_^)aJAv&mFgaKa(GqY(;$gn}kkMeAg zYL9(vr)!<;BkX>1^>obw-qkwisXP_n@|5;>TKscqUzYFDA?PpBM!+@Nm$ZIW<#g;O zJgLi+{6!;sN6D8t%_>4(7b?e+L#mcr#cIKcs$xMRjM$6he$!RpzmDVLP)wlkxNt+9 zI|EShM@D-@Ryj$)QE{Oh3kpH{rx9RHgYJO?BPbSS(JsBVl(3W*z$U0ls=!@?Pv0KV76L{j#pL*Hx~H9q(rFH8V4 zayT4INacv9haiCNh)qC>86lOz3P*!QYjf+}lLba;@n3aL5V-}Vy~%17)L zc%CxVN+C|4E5C%cb`joL3E^C&sV zv`@^^eq0!wTfs%ZE)4IKTFx^Zq_ZePQrE2~(st5aKt4^-x~Uz26)?3<$$}z~)E(E_ zCe5DpCoXW7BP6;j)-@x1R?sImjCd|LOn(yI2mN45p(F~U{asoET4Fg{z-pZh55kBt z;C(Db)WkLOEaaRLhDbO7p%uCE4!%|~qqN_qyRyEY?~A22KF9|TmC0led?_*o`&dLO z@gI?bwImSr@R#dQ`#k(0jzk9H{lUz+*AN9CFPY@m@0|w|=4fo=Z}0ht+WayY(h5~P zNyZ7N@J2-)MkYOjBw860^S*~kdJn10a7~Ioc=}Lcj5P6q zNZ4{lJ)MmR>j={tzAn;QrQMHkc;k}?KY81%4ORVQqiy#(%FxHEP|HkEnsvmUs z2;vxu$s?4lf{`;27{4+V#Oi?=3&Cuoif3Vyl3kWp6KOH9#qvTAlR7Myck>K9murc@ z^-x|(lMpI`NU+DARE0Dtr@b*K#}Y?iv9cwfrv?da(5U?;2@+#mu^{FT8dgH0CsHIB zdY<>?)ZWi~*G4fpbAkVvLJz1Wt*f!a85ulcHLL`54EAGAPNA7|`#M9|nZt|->)mU+ z-5a}zMQ9F!2uFDI{QhHGdyjAJldtimyH9dN!l$otB(bYr0QB4|_dk7=nxa}-001BW zNklYYzq`T28K*~^fe zmz`Jz1?HajGAflV$VEg87`x@a+cSg-C zWZfG_GT%E@|J6LeQDI<1KEel8yB7GEYXLU~rssTE1h$x<>N{xgW}}0FgM*M5_kW1> z9k0jU_pF~Q6y3nw`|TZW;O$(#l33MH zfRbaR>Kj9@=K7YJGrv4BDz8&l-$wQ_uyfJbgGv%vM-DYozs3%jw7oHpuB~X7n}pzExZE zwYJq-`#K@Qu8n%vX1$wp0jC+{lcVwW-qEeSqvv;zUf6qld+&+!oV4V^{?ixsKX&OZ zQNpVtOa8=_yPvwU_xR<1IPt;ssShSI?=DZg_wvO1FHODw;?(=6rr+b`)~U%`Zp*3l z(trN7>(ytkKKS@GifQ=FD->$*naiX)eu|Q!xTEYpM!A;uAGx^q$a$La7}(nD+ulu| z-|Ia`5R-clN=VYdDbL+mG_O+@r^Z!^G2xLP-#)kA`eIo%n?8tx#1k}(tD!L6#aDlA zeS0Pfjx+QbZyv?ur~yD}{=RRWD6{o3+UnfTxW(X{@N za?ex+aRv!X&4^gUUfzg|jAERBH&G-Erevx_fOHyyp2n&s1+ z^j8K~sh1Nva~wk=mE?d-fe5QCsH`vq1fQ)SW}*o#aSesZQl^w@7snrL9=+LYUc)0d z-O`%1JjD`ru!a8REg4!7cil1X#tR=_S|rHeO@uY1sxnfixC|4syQ!Qp%JRmr=&=Mr zN~=i5IC9h6li_+SzQdz8X|ibJtGZZ~IKLA*bB z>87SM;l_-HTvQKVTPHBOfU8lmz=hfXb4D(~Be-M1`v|LRGlwr^e?70hT#%}C7fCcB zkr~ye7(zvJl%`YTu#eIl@-DI+nnB!`_7gjr8QGs5e3pBr z#C==t7srmoW7B?or$et262-{PJxR{mln(V_(QLy&89x3XeZ?$$rhJ2ffE5Q>Va!D? zBAg%5?@Q#1g8_8(W9Nt61=g9EykC_OHkuO7`Qk*3({1^B4S0`=b{uS^sX zH;@VhLGB>mDS@6T)L}9)ZKX7h`y_6`RkLiDE3t2OTr7I$tYGQsad+YJ9-PC7t2p zT*Efom(*y6GEP{Oqc<*BG)b`@Vi7b1zw+RJxq>#xY@)av$tjR~=1Z=?op}7g+=pqA z3Ko&-7k8&W8z%R_HP;miELgI;50NH18F=qnC22`$Q-nBF?$o$&TT}%fVH_HM$LwxQ z@Xp<-6vri`HHLo1V$c`rbU^gXR7GAe51g-idjq)_l>u(&Q8q##>I+{DwdaN`-T;Z+9`46 zj;vM9I!-Q$hS%ajNw}~z)hgxP7~UNPRyJuns2f?5+yZzIH2UiC6V?(f`sg#<`^K411S5E`ipiQZo^D{(m6+ij@OVc;WsGnlH-u<1p_43^N$Chsof9BdQB|N!8 zfykb@=Ir{r&s@0s^hM&2Pa>tk_WonrWY@c+JJX(c<+)u-NV2t;Uf)fx)>9kx&P|G< z*s)e?TdTFYXM=JUQ`!I#Y#AoXD^yuA`Ilhg8K!h8P zH~dSNdH5Dy>9bF~k3}eNaRfMK0xLdKu-?Pl%RH{=^M1i<_{Vvrbuv8U@Iw@?b~vof zlsDpt&_5jMrUtZtx(RzNj&EV$IBbKN$PDOkv^&idKGJ{;yen0OZWXyh`X(4IBnX06 z$O_~u8XBMqM1nUb!H5l1OS%tzAmsgA(?lg)CsGC{R)`YHtFi>4uSbgZT3UNBN|1Mz z7rbdn^VRNLwQD^j{+v87O{>8_y>_5vHl%BqN_sOewKYHcvlNatjxlNTn z#xom_ZS5Z2-X-PWx!nPxmb*RM)HcewOfd(k2Wi`=wXas&DDr~qK`O~rc`$U|#-1p- z;|&35_76keUuTRmZAUz%4fw7s=@3&NGyfsU>+Q2!sRigBEme{>isgsa{n~T}A5M8( zarE|+JA*@a`bPP)&3^^?aWB&wZ1FZvMz@UU_7U`SzD!@R(l-*_J@}XTqZYFrGm#qN z73#hg@3~*?oUm;+ateuP{6?H&DS0YAbO-THFfAqZlEJ z^wb@p7M3fF=U*h3JI&tv6yZ~up@idf)7^QPn>~-h@RWa zZ3pI~8YtwSs2yEBJ=BZ}GdNQ45nr1Zct#m6p<*2O0X}7Wuz_cn$U?%($!9LTt!ZNT)|mT;}`f+ zkSBG8S=&wM_*%0!o5yZ4yd#vx(2)Oie`yQhHlRbrf>@yfz=FgCvs@mGFDtdR;NMZH zpZEilI$4F6p2>4=k$!`N^PfhH2VtUIlX!vlCxNnx(Ekk zxan->cFkU5^bNTJq^S!EWI35hmxMS(j73lVfIf)=P?Wl&?k+SCdx_ak4JZN!u=P%s zJO@K8b!hSyw&8KJn80#tIkp{@r^YWp(ir4FVHDdT;If2oS?{QH`#M(E^({D&(4ac9MjC}a8rT?%mT`)Bj|A6|hm>*1IYCR}35{#8I0nrPV zgEbFiB6}Hh2)|#!NL^MAq+ei4BJm1o9he^A%-unN7)uj@>>tQ10r*QvNG2t|q+})L z>nt@)AsQfG-q8X$4xSe9L06KsU2te+QQF{5Yi*NP$8U2AWiM zDfFR$_TU9)m1pCv^F;Cp;$ed@*8_&wv+ov=+EgyOrpJV(i z?u!pf2neW9L(-vOL_u{fiFw(iBL_DW=a6bLDrn3FX2LQmmqua>h9wLf%ZdE$ z*pghnjp%+x-z@!zz}!)~M?H-BdM1STNjec+_DQ+sK)nKa)knB@jG!=Xkc9|DRq_Dv zO~frgRD`hm<-|DBW8k%wTsK@frJ|ZDa?e0oMQYVp5q&k~u|dFD%rH$A;q%@(uYoM7 zS_Z*$UP3q!q%HoZp}D<8hY-VHbb2^X3~P(MkIM87{P_a~ueKlVW+PznkoWyS&x6r^ zWMDrMl(eo-?~d<6wMz^K^L{)(;+J&xb}uvpBZb}&4#GVMvofK@F)CT7twVO7GkdC8 z`cMJs<{W^C#zWR90)xW*aiiiUlw9^`o?bLJkY=*4EUW^vi-HNPRq=wDz0WUN8OjV= z30`VmSNWl-bM#j-sn*2o0z-sB=Nd`oZ~z3@M(_{-^k0nYWqO^1>=xdXM$LX*g-Q*p zQ5(~nnQS?(9I)%Xv56ptq9tN3Syk+)=4#Zmx5^PRHNl5ycFS6|Wv%8+`}Xx(*J`bM zz1F=^>)EKMH|x%{?>kp_ragrtyuhZtTUe3v_{DwXE%_)>!n;p%g3xCu-@yJ8=l_2A zr^~bNAN$LntU1$u=-s94PZymIeVKXIo0!t}expStjik6+z?<_g6k``8sm1Q!WD zK7RS`;}rLhOndez-WwpNz}321 zgr!CCSp-1t{m7sU! z#K(n?kYLBXv^m7`uRsQvR3w*FJssT`MDbjzNE%{g`AKt6&PU!Q{F^k$l0%f!O4D-1 za3l&hL+`@~fLlT(>EES_L{!bw(Gy{-uo1{IK%AN zZFn7|&3H7a&Q<9n@KZ9Nkcm~~>Xs&R-IG zIeF{F(OWBn=l|=+FYi5b@$OU9AMJmXCKiZ1KJMyw%l6)5o4b##?;bt3JGiksxL)sN zIGEntb$wiFo%*}3wOZ#2@j=&vw76%0ZkFCzdJ@Vq|D(NzV4b|&ug1JSAv)T_7HG}< zvysP*kZTT3WRD^rf*#1E66fYcD=ZfuwKXnDg-LyoTF%JhYU5V|Tc^vKI0?goqOC`) z9gzysU>dKqkwNRH`VZ?t=mi!OC_zd6>unZJVE=47z!)cFc_28^+7aARgHDrTohvT)i@(|x$Mz`V?*EKaEIx;w*`{_*C{>7-BF8fPm zH)=rveu?O=QFf6t99;Pf3X>jC?j{}p)(JKecJs%oavwZKGdx+1zcrf)!9eWQ@nRd8IXc8ZPy z;T~WmWe%AnUM9jiIz$DOt%sprYE}>%x6SP?FpiR$OaaCTokUB39fjOY3z6h{7y{17 zzYr4gSyZ9;bM*sy66q7P$LGsu-ARbE3WnH=Q|5D_l*gkL;gWP{3VO(yRm$8dZqht5 zI13pZ%#`_jk>8n;FGJhd1w=SgX&HU5Wt5T>w`Fd&jkp!Rv}JB05iqIt#yC^(sB8yu zYa`+&h4GWDOCk~Z7plOvtgnsJ3QJY(LsSnfnVZNoLVJx0PjuT|fmRU4(VLpT_@)s+ zx-zvyvR%p+a;yYJTNgBX$2Hk5b5E>LVoj+{M1{B}qH|o}^epp|ommu- z^JotxJLasfC7h02I*1BS@CC$Gnkzwi1CEJ+{gPW5sLqZB4`^065}P|F>50>m-n35H z%4|uqjtCybc^O+ogh8AOZn0&Q#?8fLXYbS^_F}Ne37Ywo1;5&ToEL2ont?i{wrR~q zJW zKUE&be+Oyc!?s60tPjH(868On8wZ2G;e`C)P-HRuMQ7NX4+gOoVv6GTnI^#fT}SsIqczwfkzZFT0p`Im8#KWFFwc` zsHE4_jv_-V@QIW-HwBbU1vta9k7`kH)l{ihe}a6{J#}snC8J-R>a%Y;OaRVsD4%=S z2pILry6%XDTpxGwk$9SJZXcj3lRI2S*E#e|m5cyF?}>>4zH#yZg!hYh@wlYG22zxc zbY3zh6UbxnDYjFY37^H7m?mdd!TbjiWa_4rQ%gQ;PFcghVmlp?9~=4rF&J$>x#PAU z<>&n93x%V>WAFYAU5~>BZ^~fs28F3h28aAl1Nv{|MZ@|~d~6v>auODM$Q#`szV*?e zb!5b=gY9sXF5~vthcl!=^H=Z`u0!Ozx|W7@BC!#yMOJh`_=F8`k()J zbL`ej-+AZ4V_U!a_@%v%UD|j1K6Qx@fj8~9N%ZsRIf4mCwOuT1#W z_Gm0HVB>hSX4>6pv z3~Iy7-jD1!96g-QRvxwH1w$Hiupg0WH2q#C-~bD@1)VQSD(qriqo&hN{vqX{oh^n#9$rkN=pLP6`t> zwluO7xrzD_o@!XDb!_naxmoW%S5Gm~PjBz`ZWG`ibZ_+*xf^p=!m{K``))yR@&wNk zJbvNcuU{?A-dg|K4=(-b+vorEN9VusqpffLXzR~@xb;o9yuJ0G-o7w$bM^P${^`++ zKYQvj&njFZZS!NK8QgpH!rsyIylgQU+@x8Ke(ug_Zh@y4x;B^$u90@SZLQYAj?#_p zER>CM*HFckH*KbM=)$ATLp+#Lzy-%6+L-fCMS?#b4Yo^OKR?UnLayr3WgH<3;sc}D#658_K@ycw&Q#!f3?XEC zTwJZwT#S9I>2%VH1+gPtA||Bfr%d^sXDb5on4`63gt8Hs%iL@mx~Zv47`=@s@?+xG z%%Vbw1jTz-)qgD`QV&zPo0?HXe1uVuuD^vzy+8+ z$i(j$nlh3jsdFpjZ15U}w=WSi_FuEKC!y~xNAxRpYr~*wAr0UR{D{TbS@_*T$wX3u z>Iia4p5mS%9Y{2Z3-S}C8-_nJGfLJEsv|Sk z-);&=JH#>Q`!uOByVnt;oDxwUNTNGj+erzbD7k2-y@bajzZsGBK{*I4*rPEA;dU)# za7K>Xso<}J>12=cKHi-joRR)mU5rr}WYdl-_4Me|Fk>P$QRF6%1Wg-LRh`ujM2Yd}f{-YN8 zV1*no6M#Qs5Sc@MJ=_Iyr~sq_1z$hAl(Mi(q*(5?*bL0lB~*CdC`^^_z|h^mpJG8G_u#J+?e zjaBK~r&Kr!_Ef+l1j!+ErkWX}*mkOnFE5SIOsTi5M!~YVB+IwkkFf(j=-qlGz>(kv zkPE5std}bqCB%UsKcq0kXGOYbF-b7kkS%9DTjBUw#boY-7$0n0zRmvNCGquUZ*E{g zp?L1zO=(_WCcn6_=uec^v1)8WY2c)Id`sGEo)1V*ii(qby7daPm=qGx0wPYbvv5i1 zA$*F-^}7fiaavOwBmFTqpwy!RMt+KloYh2xsL+ri#)3O8^p6%*z}cxZ9nAS7b36l!M)=6@1I?U_{Fy_CF4n=?|fo?i_Jzv$MAk|YoeZ* zqB$%X4)`-FGt8LzfcS%Vd?#%E=OK%!3;oMTM1H&o!;_}KU*xC#bQ$i(ZoTzXnP0;) zp7YihcSt9+piHxB`QTmmn78*Hb%!{&`^bife)>1a-#NXt z+q1FTvsO!ya#$in7D81Rk9x2yeh>uUxsE&yjtCp z>H3`bpdzNW3uHH9*oG$UK|?znS{Pi*Mqll0{H2Z!Z{W8V?h^JAVf|rPlF*!w&r(Fk zC!v*naIgMgbTKd#>5(GUW}+o+CN*fbneZPmXeD=O+0WVmt{Jw&GcCw0ukwBaSb&MS z$SV}*SxfT5s}CvR_D0iZoK_3D*fj9r^V2cEALD)H)`p(NK9ec;^3P-5Do&OGn4(%u|m}7ikLcK$$?CC zJsg(=o!TL@rU%A1k;a4@xiH!snUT<`l?e44jvs3_$lBR(yu|BbX!ZAY_AhNjpwd`bNu$ z21q$E7{iJ54~%R^s9$OZ2#+J^9jigU2j5=MLm(|U(sr}IaOXUS^Bb>-CohHYgOjr8 z^$?X0T#Ta%(vIYXoD)SO%WkvcWnqnII1eV zEK@u#2B%<>`(;Hl2uWazdL@%)kmZ5x!@~X`0Q9=QQG^t{)i$gR&@hEkZAK)7!UsnM z7ZUp$lVTQ^5bYF#nGK|fS(7o+K|5!Q4E4mh6@oQJ3FHw5O`JIhL$ap<=1LGALQ80} zIFTaG!c~?{NAb7aVYk_mvPDtt0Z5^M`MsiM_EZ&>93uT0#$s7tu}l4nL8buLx&zsx zgw<0ITnO3NJT7vbJD^%A@eyV=H$##`xEq^huw#KVfH*;xI+EwLggaDyAH0UiDI38M zkBMioT`>_$a)W921#!(5_vC3&{F$$@=<<$7!3hCpsQ?F_%Dc5IS_UZxOnJxxLDOs>ZyOo^lhLW6O5GiIS zg4bl@aWMLQpp0VVo0SO*MC5^yVY+?Y!Vzus!xVn(Z1nFTU?8!NO>u?O>;@=fpRo9s zEB4(y@WqUO?}iRw!B9M?+f7Y?aDE5|hiGw}Bt$?Pvv~DL9g{o*Eb5!{Ejo$ zp({KFXLADj;_AzR0tq2y3Kq#uJQJDOrN`|+3j4;3l1zgkp!u+47tLfs>oh9R_{VTK9vK-waFQ09;%12QMHvAGa;bBz!F6N`UAqB27-5>fmMJ@9W3q)0T3!?z zM)@iD4(fzpPADjdBaiS303bxIRr zr6MHp63cn}!VJ%t2XsQc`o!H(ebk^nA)$#^BUY0pw^~y7Z374^UOU-*R^lclm=83H z`iHc59jn#ORWj|n*K4WGde5dH!v2lAGqn3j%}1uUGr-ABaC?^)_7p^J(#JRUKX3;Inj-Z!*`62_p2J_kg-Hy!{SZs(|B1S?vD&SqV{h@-7*}E2M(Q~41rxRr`WIa zj=qwZu^d~559?~8(NyF6TH#o{g`w!L!;xP1P-{Ib@R5Yvr>eb3rf6$Y*x#suBq65~ zC;ZTjJXjAEdYOKP3!P*8?curZ|g+7d-J6@!3b#$i8qk*`P$K~Lz_Szt-M zQ1jNK`H+7y5n+y~>dw$|T6zcUqUl+x{>1g2H|mGBnI8l+Ylun^UJVP#XrA^7b#@MC z8$`CPoiBpu`Opc*WZh}zs;$eFmVC9%t+3C3`+B`&vz}V5x%almEgLl9kY>`~N1%{X zAaVGBM={q1+2x=JaDsrO-@<-rSkz=B6Pa%$fNI!f<3i8k27*+{xL8%Za}9BkzU(+jQc zYBQ2+T=xBDB6^+ecH0Ahgj4zvs+gack2ft<8UnEE!$1gb4xWGIfZ|}1 z_j*QS#5Qf^vtIDn)O}Xnc*Wv0sgHn<;2yeG!4Ef100Xnf|1{#I%l6gB?4sK~7!tZ{ z#5|L6Ne~<%?9{Zsqfn-ULH8G-gtAM~cWno|v*80~*rB5+oAlu8Fc88)Ez@>t$NK@n zL*cA2K6)|F2=!|*khhE%w%oi}Ze(MCLJFQ$C!n+BlH9^R6sQCq#BDEPqtxGsaD->X zdjgog{Uk>KTwn#P`H1Neb6Hu0a52tMHKRo)d3k_f!8! zIX%e{3Fo{gGCY5KSV#z7MiC4bY7h^KTye%4R_iv^9?DNt(wh7v#hj6To%OKNLJi`e zK!nu1@rvs5#L?Iy38pl%A-E)tVqq2mB3*?B@~Gx{Icw~T_8`0<$SDetuk;k5SU4cl zNzK(nEy%GQ*`>OImZ0XY1@b%Q9LGdOISn-N!B+Je7DC6|txd3+g!416VS;^Gl^Kq? zeZAvwThM_lP zPMmPv;W$j2i0sVAR@q$ljCY|GDJ94CbC*S2m^ImmCh?H#V|BVSg|LFMtT3{a5Q>`E zgn|;DEK7L7IoU|!nmKW)M)prQQ^ScE_w7H&WO^-Qk{qg@x0WG zsE!>Pv{CNYF^Ms!{@Mx_nf!u>;&OFK2ILY%Nl*^5JugJS+|L;wm&Kc#J*Iodi(=eL z-YvF2N4+f?V}BYFs~CRif$T%dtT1MO<2cwk;$asOirxH{Sfnoxu7c+G3R!?x1H$h**bo zw3-S7;RvyGvAh&?31kO}N+e?j-^8>=73?*r#PvW%irPXKS0aE}kNLPBa#K~&nP5bB zE?p3v8S@%Lb_kvbMBKWmz*U(exVc7loG42eFQkb(8#w=lUc>5iEMI(cRo3i+Hh_0u z`4*cg46vXL9&9QRq9OC9gpY#!dsfFJf%|hTb1^M>&R@(NjU4mMt5;F1gpeEa*HdzO z*cNpg?+G%`2yt^4mQumco6b)T<6zvciIO>_V{*EOT+3OjI4Lm)q&Cfa(?6oO2G5Tx zKV}ZjbjR^&5KyQO33Ac&Ov}@Ppb%&bYNOs`LFPpU12X>4fWn7(97o)Z4^TqywKsgA zws2ZH{C6n+o*oYV0vr4AiG+i;eY9)xgXbu-IAp+&(RC(J5yE#kYZ(%Xy_pXW^1(2R zf(a2MF_iYMq9tS(W7nT2)Wz7$8F{wxLM=*wZ<;tK4;yy1IZMN7n6AxA?U8@O)Iw$s z!_03`MUE?9vF!9MGU7tZy;s*1TPu$O};lO>Ctc zRcmKx-TQe3<9%kTo4!Y*8AcdFUIVXb>@r+o5k!XwV!0YUWi>ySJZ+Ha5EWN8tyG<9 z-@01uV2@pA+IKRHBX{F177{Y5>xEsvMF#lbIT8RlyZ+I0EGIm__xQQp$Ik72?Bf0( zymjj@Dis{89qdou99|LSkQe*I5=^pmGA?H^?YqDT=r3koglfOvGBJOT+y zuGI$ewI0qg+D%E8>)orh&Q;DvvPM3|t;-d0E6xQ%G5yyBTa6ZLE+?0z(e9e{6~ z&!Ovr@F@?CXnEJvwLctA{F86oVdiJXZy(UHO33nJ`^K!@P}36|ko2pOeu#n0Lxq9E zMlXP#*2@hVYQ(=oL6`V2Vi@#S{km7=*M|>ahz~-EgWMnxsmVK#kLc;p3&av}B-LRa zFQmx^8PGs4P@N3s54G$5zT1~@`k|*8<*&# zC0@xnAxDY4Jgx=@<)aw5IfHBC(KVuX_*6#L)5AWEA=n?rE(yAT{?VYXi3T2V{pF*c7Hb^elmFx(~VANg3$aB^0wqCIv2erd;z zRCpXuLl(9K%HjpA$j>GcD4OJ=071-%a$G_#c^dmld}WF`c$zAtxl?}MlR069Eu6;0 zJxe4^J6>x)Sxe0jpW<8@Ky{VAu`&}niX{?!?HsdHW6?ehuEM~-EnPDrTK(w4M1p+KqdRgs=np4x| zR#y2LB1G)Jw(PsDBk#6m-<1e}nRk&7q?Kb1wm}IvrZEBC-*NgYd{DiWIo==DKot(Z z+bV9$RBiLP2>tj#CL8q1L|%E9o1_J}9?n55j?BUV)vvjETyvF-`-Y+Ms03R&XN#a+ zZKo)X9$~LzLhjN?HgZh;Mungi!Ops7)$e$==$6p)6mSqs3brC|FEWgy6(NjJCX!-~ z(-X6J5H_S}+2&2%CVEEEuMxEJGD8#q-s+H0gNF9h^>xm?l8YueBvQsXh$V90g6E0d zrij!9l^`~sxDpFuV|A4!09g?1$f-gs2ESZna|-U6mf(Gy!IKPQj*TcG7r{WWLL7Ep z@}*b+L0k=Tmp)!<*JHWkf_~y^BT)&`jBtJuGRxWUmKtyikaUG^tEPN`8?^{LXFY1{ zWEW}o6Xc5QolUoZowZpaG^N=~7NpIpRirxm#)`2oh>X!kJnXYiSbr=|fA*hSk50wg z7bl*dB#H&Yea$!7e{sTTwot+o`}%|eq63^U_Hi5-jz`isbQB&FD1~4`2J{*F%=M9= zPn=2nB$l6BDM)m`aZ#AM2TY>BO^cT?YxfF3ha6#$J%1;QJrcK$6>2O<#KY5PNNBQv z734YX9g{$25DbRNcG0So`X+-V+5OcRXA+Cha`NIph&l>PsUFY*2UXEV_2AVD(P28M z2hj-;#ieoC3kK<9y;PP^XgN)lpg9xFaky#8JQ;OtNmh&u1{C~Jz%`8DP+`-ux8$45 zBZP;S&AP7P)p)?LymKfYY$QHd;NX3hki@4g4#P@n;ulxAfOYmXF9HREnJR`W;dxL< zhg2|v6um-oROF}-go^VrLp_x2fv@p|N5G~KbW1;y+e~O=%t_fjjYP&%;th>9Adn?u zCK{Ng15MDQ1;^@vOcdL_pg{(s`avIjFa`<_HC(e1wV(|kBOA?C`7kmVk3V?Wn|_F~ zojp)86X{;8FY+CiVT&-^KrsD(NQasAm4u-Nd$X(`3__V(So8jD_*RC-h3}|I6WAl3zgMaY+gEFX(_143fPpFW_Ru4R4yco!%s8(+{3umQck|LsEl8g!i%in+VU%y7C{l9mAU%NA~eHS6Z;IMr39A`n^R*PUq1L%kYu})7Va~LtTmq}rmRz|h&ruW?&85$t7IELiW8=}SI;2YP ztrD?>-aa<~bIIQfBc0T?|kvvJ70X|uYU9L zkG^>AuiVBb&+qh~+Z(vRN!HJCfKu)?SZrMH+2X&O6seTp;5z#|uOr$ZWm|5`v)a-1 z!a326E2;~EHXe8sC;)nzRVr>uXap#w0FD1V(B~Fv+z3p zW+WG3pXE67a~<-l?90^-WMoB>7fbzjG}6HxYl$2MdkyO{6s+qqXxoQ$CD;q?q%HiH z$`8?^#n9IrQmr)pAqRE)(j2B6nsC|>gg)VrDCM@3WT;aV!m|=K#ckoF!>(5;C)F=W zf-b*QZacLj!ox-XJ~m5UV;XQY;)z-;i<5=AS3po;kpz1tD;>uO4t6s{>m4iivvV$c z8Pi6$1-iN3@iO@W6Y44>O>o~x8UDgZF-+P$Mp~Uv!%a}?VWjBo8)XhSgMaC#PN)UP z%Y9sb%9hCrh4!112K0_e7Gqbw{X~r#2@l*UNmVQ`k=HwqAw(bpM@eMN`bP*GXXMxP z1T~v*-c%KOLz4APi>q&^#u=nCuGtf1_#kVzL%ukCrhLV4^`rq3OL2y-RcgwbhYL%U zlV)?Sf;K=N$U03|Vc7x~(h7u-T!30WU1m4lciV<(K@>my+nSZX?qB%!wq)OJ$x2Dh z2+Xk0(h!e|psx)TGMGcPt`ghVVhidn@A`k;zC-VFVX9n08FCw}e$JJQ#KiA%&SgfB z8QF_+A8RgS?1+Q9YFoIqsfjxvI79~L5nNB@b#oN0tSg8s*U9{dSdyZ*F2-0y^awB& zQ3?44KnAuV!xya0>{zuNZW znoW3C!eG&{rivY!Od0lP_jD08GomU|VOPlv!YR_PaPmbXA$z3LpXI1nQ$D7;p|IpZ1Pd-L>`YeMd zD#164SNipP6kE7}|D4G;E{pOQ!qP`X*&n_{`n}Vy(m*Ei*@`dW2hM%JUeJ$Y_CT3T z^xxs&lQ$H6Pltlf<*?Q{BqaxV&-)IBO`!h%AGh`eMAc#yF4j!*%*HGCt| z3}4dl2p%c)j3c&S5i9hNaP+z!PI1dY=)|dteM<>N(ft}N+Py%I5%*Nkuaq5CH&TLW z3_s{>)4?NWXm@eK={Nx8so%Zd>Z7?JmqmXEkg zL}|nkcUVF^pA~B*X)ys8HSh3JFv3?D;WsX+*D#wQ%pl6P(ZUfDwpc8S@t>2Djd}>s ziWVLrB$tbcg2xx|~H;Kr{TE5fc+G)#-7065PZ8|JQYHOKluM7=8zHlFdN~ams~rlPiZ@#pM?Y z5&|T^7&~ziVxHYD===94~^9GUea*Cz7>U+eWfo^5ioPV&61FT3+~PlIAyAp_QE_BE_2Z){tSulsUtT6}PwJJI`4N zU3hZho#Ds26u3Lyn2P5K-+7a$$>;V22+(EvrC_IT-yq;7V|z0XopGeV`NW5$FArJ_ zW4}x_IN-f;u74MmkZJqfRBTet_uC=| zsv@^l752N#6jI?O9B4+ZYCuqn0m1;hXltX}Db=k_UroUog;hmU(F&1;&bPjYaJ-v) z9falu5q&6&4IEqsbw7U4o!Gw^moJVd4^`R+Jw9_NQ~zUq;Y$fByu_!$#}{*N&w`2% z+uirj^>^QYxcBbEy|*8JeD~Mw7u~7j|40BR?eXREkt{#L6-bW0l#f6pO9vOFJ#1t9 zMh3$cq`fS@JY+ya!&G;3o9lEMSO#*sLLZcH4bqrF`%XNV7wsnOGS+w0F^Ju*rIbxN z>6B_}J(Y^z=y;@#LE>HtByxIp30};9X-7l)s3x|$F7}C%&f!@8k_64hE$g4vwm1}I_Zl*&5Xr>N6a-d z-QUtYDcZ?R*sruZZAI**y~6QOLS&RHH{)^Tj|osywD*+by542r+T6C2huVn4d}PmMAqzQ;BWrQAfN=#&Sv4E1cR; zVu`C^wl8r$u1YL-N>a^Jjn;6w<~C7{#Z+oH6a5p*<9skLqV$fQLn@ryFeT!O6f%Ov zDhyN5s9<)&_&8JY&EL<5AM4kDB>06V=fHfwK|HYU3MU%RZvA>r4hR|WEc337 zaD&Z^-U(YAv4kNXNhVV@V}Imlem1MYcSK!ad~3?xN`m3=(d&JIfQ_sK)Rlpk)dk@_ z=*lu6=M(jCmB!_T;~}LiW1^IB?2&-81{MsNuHkYG6>Q*MMeJmH_ceowWMr#2UKB6~ z0JqUCN`#<*$ZjNPvK|O?;6frmeswTSnNlz#qePlDS=*!`PqCbl{o!%Ke!at(LE=s; zfL6%(nqtjy0iEIUvRp0;n@A+1lpOJZQZPlO^-R`N&WJLFV4{{rF%M&h=y0N;_XT@h z5<1Db3s;26HsY;ID;X9Wsho_2ZT>`%9P)ue`2YX}x=BPqRJXI?g%hE2r-I>d3h{$H zNLnWWhJxpirVz265XQ<0w(9Avst`s+btjPt3jQlFBv5jYBbMi)B0mLxJS%13CH&V5 zMy!ebMMH-*y@VNtTQk9Uqa`rtlOnlw?hseWCOFI%?!XbmrjaX28ekwvIl6&;q6$HW zU`BvS$b`{3DN~=OyUYl2;deJCki1J)g5-Dn1P%qjA zF?z{9Nc9Xf$_wn&jL#5MO9wR!UQ<0|Q!UBRMm=|tb+}B%!I}KonRF*D zUM-i#@0j!BlW{Nw3c1m(lJl@rU!MbSXvz@t_C1$l1(?g0AKIfV6nyA!VUo_6OcxQHxsj`iKDBd`A)+7BIAo=y&p`4z|d3GpDOa7soO9>o+w8I z2xfEz8Fk3$1iB3c@h-52Nwako4k{n43pImvqJotwl2*p!TeXL0ECKeZ)@7q`v${n) zmgHv=UOQ3OCg97!hGx5jw+rrJX0P>zpG;}sy9=DYb2@+UIL;w$bGt(%IW`z6wdrF#o z`a+UB6dQ_6AH;))kc0ipE-z#=b0*O5)<8--WaU>qxSI52Ssou~w3D+v=Z*oPcM zyu_duo46pVS7QRn^NQi}+c_#R8i4QN^cz80Y+$cDEQ@>=_c8HbklW)&2Z z=<)LN%kr@V{fSzi3ER<^+b`O8-ki<7JcFXgcJlojk@hb>P6k~c(jUv5h6fi26_jYf zvaC%EX@7{j6c2I7(fy0D-3u&5vezBi!54vry-s0Um%#2p8G|PF2Xu=zdPJD@33sv#4yGLtJ)gl?p`2#7u%SEy1Ev6TezFEt4>YOK# zsNGrrp)qwNlJqGAoXz_NzLP_Q)x2ry`N1;%HMA$Uulxk!Im}k_51v}i`PTofvwIfg StAQT?0000haIihOvAfgFKBM6~O?)Sd$H|AQq63)HPz32WX&wTbX z*O+6DIhr|s-#gwhE9}a5wu|*qu`w=H$C-^Y=~HKNo$buVcA+)4#6ojiuuHXZu{q9l zxAX1oQVp52y=}WRw=ycPjV!Uy+|IYgR#c&7rQxa_`Q|vixt(3NyT+EsinUv;sB(2= ze^GJ^#pXL>mSBI!cCRI>0#=|EZqKq4J+#0R=Eu2J%!*=}BfF(kAE|n{+CWiOAFH#S z*yHjoJR;LW4e{va*eaG?M^UJlKMN&f*3a0pa;#FWgPfM0>yFhB%bY%a#@b~kNNJ%2 zG`5AGR=)M2?y_5~G*za661KBlyNg|6`T0XrV8T8xwHY(5F&a$?sJbSOS zHnLl?c*J&g15HC6x-4M>nRC3|4qW13*;NcdS8Y*&UV@r{JUt&HtAMp#SM7H6a8pe{ zZFPH&kIV5V>fiPA3`g9K**mQebV7RbOb$;!W8IZHea5q`qp^JYnG{=%zZ++CduDUn za-w@rone(wF}*@+5k{fs-G%gCsM0$7WSzI$y{tU)oW`?e?!Mnj&ago1PrFe+)x+qh z&g2p3z|a;OEOiAN;@xE>;NDKC28VCeXw+goRo z$c_pRozE@vND`O*sH{$1T|PT=l7IS^x#f6nDc-vnd!8h-{awP}`6WAIJCgKy=C^eF zi(?_4UmW4E{K(0ZW&K2+dCPMW*^qMynR!dRHypSn6m`~NEyDb*k9gp90*GoO&FEs7 z;J`wY=OjRxG{B6g;v78jOJkcOAk3N<2*TL(uOkJs(x&<9NIu@X({f^!66R#{@ zebm5?iqjv@vZ{noffkZGXEcQNtfD>9DCVt2gY6(Xx4ok|z-pT!fvO7hFrtXa==4U2 zB`u$&fWF=mMi=Wiqium9is%0Jh<&UNAI_MO{3$)gGWDTTdd?E_j6_%1>o+Or}Kxc#^D@5>}Jq@Vw{EYBwK_b@)73+FO51S4 zhVD9}eo!?C{WExnH`r0Q(Wrg@o;Tia5es!YO(HWE&I0A4Hf~oxSc><5$4kgEm|4RO zd4jKy3Yx6=-!hSH1WdSG3Gl5DI*#mcS7mu8!Ivf{35Cg1Pfva(T%IKAuIbE+o<3C$ zNIk#c$6lQ*Dt$CR`DvgZ?VZZAH_5yZ0|K0+<*H9)LSUr6^2uwHC}=ojsE(4EE|@@r z*2dGYE3c|cV!ig{fXPx#zF_mU)Aj(LuD#zmWbeD z%^$64E5xCMs@DfO`Lo1&YG$F;EDaH;7VfHgu|+fr>TH$VLc;2+0;ohe~u88B_%(`-^kP&#|&unW~>n zPEnXlL86sO9FHWvfX)^f+8#?FYY5Tus@+cJLVRH=7KNVnSod;lM#^9X1`wc-Vsj*> za2dgNwR~v3rF>RNwUsJo;WjO`AaJgA{H@CM7!+BWrK@Ms%p+~i$bp`;JeFwXq&5{e zahF7{{-OtnORKO-P&NPp@EV6IjlwEtlU7y2G^}-(RVk65JRgS} zG3e?mq<4H7k!3w~yOg-)(d#G(BC{&6;m=BMIJVGTJuHSKY^|PXb%zb4I#y}MMB{IU zbx+KVvZUmy{SA7R*jlE!OCPn^;aUVUhTO~ z*UEssXSH%1XV<;c{Gpo)R34dT@7mUjoz>!ikszm6qCHLozU8!VYr3_v5{nvkGQW(U z<<)p@C5D1t)>ZypK_X8^m6edfJab+M&p~j?VW!V{wvp#ti4(2uDNNk)Q#b1GqNjk% zOH83=_hT`d6UP^$y$g}lx1H+}t}Z2HmoSf%?im&$;0Ydp!R5AD_|CE$ zB`Zs{7$B=te5S!r@o?YCQp}}W(?|75Y?@b72{0xZ{YY4WS z|HLclLt&WcTq$41s1T4rCE~AG3mApjf)Q0m2q=k07K!rAV?;)mRb`Sm)&Zv8ShPZf zV41qtgGXbekgI4RDJotIIuq!(JVk}e9g=p-lXQIB6rwooIfvbaz-vn7YY;z;6WJ8_ z8Cb(6klGvD>hnyHb-?t7LrQ(CK2ZsZ$8tr~YNYTMMiIADC?4@sZ?GDo`C~6emIdY* zipW`~kH$`Q^RU!)wyH|1hRS!8ulv2J(+*#mb-`XBAJ<*I&fa|{$(G)5d7C44)lu&D z5dE2;yA9E5t&On6{gCPsj4!tMay`Q=Gsl3{|{DHd0fa;9#PJfC~g zCEY%`B>^`G8wk!Y;3)*LS5gFbtP9Q4=YiA*65m*4R3pIBk z8a{bV==h4TO$n$=yb`#0S`;iBA2LxIxN zYw-l!jYQhxpraLw$O@lgx0nY0R(Cig`de8IbzVs_$6hzKv?L>ip*;x+mSIe}C8;Ff~YwMwxn~rrahs5L(1EUCr zhGLU}L6k#hxqjgjN0KvMEb{}4T?dL*3|iINNo6gxqB&Nv$=FkCyB1bhMw~#k>6kb1 zbd_A70MEDaQ;3mjw*FPkGn~o586%>vzNrVZ4tE!`?uKi95K0roZNjGrn$(?6f%LjN z>74(n^GFnSuF-ABf=QiVDIE#kF%P`!s)J;*2*t$#Q!pIFbjL%BJPNoFn`cxjsd!NJ zluPE+#F-fat}ss1epz?J#00N<<=P-=jeM=J0m~&#GFoMjnPCiCyEoyT zRIsaBwMfy3eU+s{lIi*Ar`Of7XHuJ9hsjb#;-g6wV&m0w0iI4&Kkcb$?VM|o>0LgH z%37uqTHVFdQyP)ZIO%-h8{^yar;z3-0VTI^0@-K^I)GcUy22wsoOpFOP9$)`ZqK|l zMgOI#Lue*>lVT^A$msdNYcX2s^FdsD<^Xh|c~XvR?P!j44jF{(Xg=gmz&%Hf?>!mq z4JrAi??Sod9h^+)*;SIK?g~sNnhyDJKsYRH$xCW`$lo%Il za97ui&fb;(%`WSeXG(gaHl@fy4d9~UQkC^PBNC%3o6y?p@)g}nzY`hTm>YH6hu@- z*T#r*fG}!!dqVzLcv6|U7FcgN$?lONN9eFI&gJ;3X$(9n`-o)cT0~J51=S5QPf$w!%`r zGYYYgd$y-Sx1iN{$}HCqs(?yX+u>7%uGX@E&%*3b_zyB!lYfe{PibpRfm`UWtUT4F z0R8FGDl0*BPoca*3gbBYLD7^F2YFdwv?Dw$?YJ65dZYrK+KK31+#$gI=gA1uyfz9m zaa39b`-*ZWRz_k4WLeW`u0?af66mv}eyGE=!k!^I0 ziNKe1XU)y%0&Aj@%kFo_a=nFkD_L4w4w&-bd?gleiq%*idI%dMY0WajvukQDl@fhi z3bMVl=ABTcQD#jmupbl^cIgey(!)j8I$l$I^RyW#mt3JB&t9-sKLdZtJ0eSLX+y72 z_x_S}#l3YG+KK_ftE!$1jEJxKX6HF#PR0mF%UxINQ$K0Bz|l7zx}+JH?YL-S_;Ki4 z*vRn1n7r@AjFJy*TfhBf@KR3u~O4_ls zB(?2!J~7?u%-<2R&xA+`^pm{k)m%$`E`81JYCDk?6*2*ovXawMh07~J-l2iw-~{5*ZaZZ7 zPQEYg$T=VEpX6NFaVgp1Jo~2*AAA*^ z8@BTLN<^$&75K&lLJ)jd*}*=_(zvu5yN;AVuKJ3cOK?bI=)(GFqO8{x23(sbYFCzs z9#kY!iP3R6$Zb(k_v+xrV;stiJ!jSVURvb>qUtm+c|_sQ<(R@bX$5zomrw%ClVvXo z8CE@V`1zW1Qsoz`lOVJUfY7%V4QpGhbI}S~n4EPVpb1`+3rQGpNlxd%C3aD+beM_- z?i0GZ!ffhm>26VvQWqtQwt|k!7V-!k=TbD?vTVxv5RH+m&}TlIGQ_Ez?zu870_t=cFfJ)diP6sK{zl zjxcf!NJiuv3T7i!Kt8@ihdaDocRbSTUUG;c1aZ%yhLQvFd@C|I9uuG42|E_oi%JDcDz!0byVhGamF5=D_BRCUa=C$&Ip57I$jxWp}zm`b2Th$xd06c$a4qh0Igxds>6r zrG=Plalp$xTS`>0B?xMHt)RBEj8c^ILaQ0;C`{ylml9B>-8in%Iq}Jy@eX!XU52}_ zBJsW!C*rb5E?cd%+!wWzP~j|I4br$4c01PsqgNG#VmT*@LvxHSR9&ujOG%7L_BLmtRmVL(-j&e;xtb$n4SYJC62aLT28lmtx%5@3AAJ&A4fDX z=&BPONk^#T@-!ucHlZ`$&{ZzvdoDi(eT)Iibfy!y+0J+ZVyiZ5E}<%t${#?1osqc1 zF|by|8Tyj>wwo&1j@LZbbdz05Q=K4f2zCG^c0ST0WR9ySPic&jI=dSXT!kX*EIO#b z^d@C5d4vejGHT^b#A$4i){D`Kpixzl-#&rTSS zK)nr@*hvt=rH$>x%@0n_SW6K9ZjVoA0HBLPVyYn_u8fnorQoMLKsGHGneW&O-54vt zNNZ0bTr0X-U^+^mPs&FXsdA-HXdX>rJxVYg)VEiFnmT|x z$+J9l`+~E3ZsLQxc`}!nEekMGa;*eAez;r!e?bh(Q-OvZX~T@Ljs<90E3U@E-k4|( zi>4|i8asj=aUK;ECun`(5!8!rfD|du?dgW*LJfeHl}bq-vt)T18OlugHC$RcWJNc0 zr8^SdHRaVvCbcjsSt@zUw;Vje(Pl5hKo}=v$I+5HcHT)@BMj7IeQlzUfPo=!eYEQG z6U6k9hCCOlEtw3yKrxN9P(ZOYR)f|Rea-A2WSXx7kA!KIrd?CU5>Wl)lVGO2|?ItqMNv%=}sCoV|QA|(RiN>ph+DL|Vzr>W{6O*l85Q4P;u zT8)Z9gN7hslOlHYz2@Wv${UkSGTbW!1;P*=5$6=3WL;``>=JX1BrDz0VVTt3%7dyp z!2xlQQzKaBVL%r^lxI$P6d}AE++L{pN`;m>-j)(~HFu$>Ii zn5C#L+rV{=U}yb8-BYhfAxSMij0lMCk=7^*QCdj=noObUL~PI@ zD9g1-QJ-2-*bLktd{Jk?cdQy}6=DyvJgT1EWj`g>l%CoM!N9txGos1k@d@g<8`d8T zMQIMVM-evD)wFJxFSPClA?{0}N&X%?q3e@^{Se+s_8uM)LXK75B)L7;`a$o8J$i<9 zwxY^AP4L7AuOz~Jr^hXWCAm~i(w8PCb@GjhGL&~(E^|99@@R4%K2WIy^<7!s;g;Yy z3FcNN)(Ve>X*98=RqhE{bgO4=9;+3Af{~&jKpHSPzoxfVnS?*dS6@kzaV#Y%S9Nk;*KF z0#!1RHB#NIG<>+S8oSx;FKddCjEm%1y{aY$jd>u+1mHL|69+iE#_867lMjWKOn zqr59_O7ArfgV~n5&qa!8)|rr!fz$AAn(FRRn)5K%2#jm9+F`%|%9i8OqrIzguL-7K z*JZ34m20WC-Xawh*h+0>EBzLBU=$&=$ifn*jlNYi?B0}wt$LnKQ;f{H8#Ad=nu1aB zBVYHB8Aktf-mOAp%Jdv;srBsJz{-r>kTAI~itBdX`<}MqG;ltmFO3A!8y%w9d5z zVb60Ao2to}?~jKK){_B-O9Z_ns=zGQ3aY2Q)s>~4q_&?+{Mo&B zft5I4qrtuG+(cm3w$~8b_%}=k2z8xf73ZdDq4O{Y6J*CJe*;8{(ue#Vl9+7VJ zuOSK7t?*H4nfYtZUsQJ$k2C5y@g0lEP+C=!JX(kjs+C|W{-P#zF@0>MLE5aliw)IZ z$DrmKo~OWQrU$7IyChO;QsKK@?V8rZLpwo{(dtPCLr+vuVvUwE!m;c9hQriFWfz!8 z$=z;yA0{Lf*@cH=c{ZlE?|@*bR_fH8{P^ccr55xWm0UNF&biX-C+%; zv)CjMDj1|hmW^$F-I11^q9(LK95rF)l%tR-Bd={+0Rz1ennN?X8(6Fws<)|I)1ht| z3WyYURtfQK#wTUN`)TPj0fGtuj^HqA*j=QuR%P-@=tBK5%K9mQVoQL_+ralpA&StM zw!c59SX&)6&Ne0LeGv6%EZrE2)aOYsPZvK;We7%;sUqJ`<}g=mtPz#m^3@?f=<-BKXY3#wHJ<;7(Pd#1K&IXOF zaFy;lrUgN$t)91$ppSi3LG=m*H{{Y#r!!zUdHLL={t8Im110vt;LpeolTaMvMnMSqkSMbrb>y9vA2= za65%7;Ye^iIbZbika;+Na+nlC;twscG7Wmh0KMvkswSpHajap;p|NG9zo>NSlbEmG^AWr4p?s?foRUx zLV+u8Uh^EaRUTKdRoBWYM2*6Df#Z=*r15=-A-W@_Btc!9&%4k&O>hCN?Gh!{Ff`BL z8yblkM{Jv&M}lm@dXVS#@})cIL8!}?5Q%$rqn%jo0sT52gpOl`Abl7-8+=?tnL$Q- zmX%yuj*t#VCCaiIs*e?}S)HbP;De_oG#Oo_xDjU1p~-nR`~<;QQS+SEO{^FPK33s+ z1G0$fku@zPYeI3NhC(uJ2A*o1Z6*xmy`f50xN1cNjAV?E@_ezHqAdy4Li4ksaODS9 zj5`c1RxDRYR)KJpmC3aEDz9lB!Ll!3o+db#EA zRVqB!PIC3wtI-?jB}XA0me7n9l~uk%nhmbPcBT_vS4ILb-LqIMzbl~ z6!@Z`pd(G~LXAY?PZ<&UFppG@9Zvi;Tw-7YTZ%&YTI6@t{U=i*64o&dBCUl^#{Qo& zL@E|HE~DnoNtDddVNSoqAYpF^9#Qpb6jve*$3$7x$|Hr zq^d|AtK$lMP=WoDZF(R94Kc^K=^6y7jgXb*+;4^{6oy|kSGyQ0q~asZH{2C^Dq!8| z9YB;+to0!^=KqPHZjTxEF)VA~rL=)7RTuz`F=+^UIRpRW1 zgIiZ|Bk#><0oMu>uLo9Y2oxG!DK}UGK{hK<-wP$E>lA5JnXa1{-dF7%)k`9o0(pvK zD!q)qtg580lbtxQY%}-}h(yL3BKH0aQPWa_wixBYa8Q@TYFF}i)<@8LDXTRN`#;^C zs$P4f7#7SN>X>%C=gzSjLYr$OMQ=ly0NQaJi0oWm2|tsWygUW;=_8~IxTUAelf;7% z8&I0Fepp@{Cbu&H3|+qmP3?&%hyM4ZsBP&yLLzwWbQZXWbDlDsqy#-~`JIV@0D`Z` zs}-;a$YnZZH8Mvz%|R{SDF-ow7{3KKktfe-f=8^nrYx*Eo%3Xo^#HCC+0v2rXi3%5 ziVuN~BI#KqRTGp0+)K_}bi1o*5+=~KyzEa`wuI(R`ISiYDDp;dn2Q51UnO(sRY(Ef zq8O@JMXpMs#H>izhme8{v6la?xq@uh=OPeR6i}IVEI@0G?+j&ZRB6IuK8RmYJMX@Y zXGo%Gz7a~}Urqk(6d%#Q=w%|TMXr&30zHhBo~jq9D*eT)kgin^-KbS3P9&8c?a1bA zlqAmtDJ>Z*mopL0g18gS>l5fe0HJkO)jIdM)7wlpc26A%)^$=@GRO`~XJyz)XSpTGU4h*WXfk@ZL(xxZ{{ZfxtM$d; zwD7Lvs>6N36RqSqv3t>(US5o*sqN5D2+kBr^`-YY|WB#;Y}- z`;u1qhHE~1qqH+AOp7xvmE^swM)HSc5;&mhK@n3#7r9(c0)YMoi5WYc$(cOU0r~{u zl-MG2vUDJ_Dd80-f{^#fvFftj7YfW!S@eLRiXo7M1(VWud4Ea#)+2l4N$tqAL%QQV z<=VOF_6S$Rk-(bpCEdfhY!)lWrnUQ&xtVAs7PDO@bWn~}ecEI$p#Ib5=>P~isdO?A-( z2v0i!aAP5c77)rzGlMTIx^V(TK=afX$a8GPNqbdWJ%wF?LfjX!^j@+oWxOG{XV;Nm zeVD8ptZr%O{a6igXNIqb^$b}Y3$HV$5ChT#bQV)9_FAE5BE45G_JicN670yb&WZ%2 zk$#fmz?%2WzXD%6u}CUg<1 zx-k;>6{y7MY%}&~$KLjC?|I1K`~;WJK9-Yn!|U0XD$1}NvMmbG!xE3=r;f^W`O3Du zqd*tY;Ydns+D}D?Mw?v4f^Za*McRGI=P`T@-~VZHw@Ahg$O5&r$$xzUor| z1+EN<{-FwI3UX3t#(JESn&)~b3SV=$+vAEs@{<*@*OTEVKz!&+0Yv5DF8?-8Oq9}W zbBEWWhxkYf#^zX-88QjP8M1QqgqUtFBfS>OTLca$jcHl9V?aaPHGZ}kerlm%k#G&S z7buW(3G{2)agHfngO%KLFA`OyD{V(6EUSGz+YU763JaB?&z`HfGlL!H^D0 zjZ6vF3cEMmhJpq%Xk^i{V1Kqo$$BN*^vIN^cVW=c(#rJX+)C#5Ajmcn!(@>KeQYi&6pldG0;GIBcw$5}t0;xL+* zD8q~THbeSUN_H()59et(;KMU*X|6lNnYrqg8Fo2Q15B&T1J& z*Cj{2PbVSvZLX#dp&Wl~54M6qi4~SsNl_-Oh`kz3*Q%7;nA@86u90b)OM<1fKyJAU zoMC3%*zj`PYOreia!-BYL&YwNMVwj6I&oRP9#k_qo(Wq$IJD5CKpWD*q;Oq0LT_(=>f1W%&EA~P&RuIRyWv7TJs zVW&(HIk8Pn?$qN`ynYIfpBPHs3SknexiG;49Qa*IHFo|={MAJE$nD>HoCt6sNCX#Q zfm)GWo;U+-j!Mx6LLP`+%2FT-yM>^XHEqSlu&+3D!mfHXl88csg&%Qn1>obw{P-?~ z1LtP_s&7{ogGEj@I=&UTi@~X<+e{b57-zteCuk@34jIigKf}ixP^p?fTF7gM-2{AL z#XV>hqV)y3+sbK1!YF%3_B;}iUXMf^xhq;th|5sNbvZ{)QeRfAxt(H%*8aj=9+|a| z;qC-)k;f8cx^dY0AJJZj{Y@z+|{ z4t7u@N}L=y5?SAmrQ^|#^{I&}qen~9`f9+6h23n72AxV)2y9L3`u$rjM#C{3EOkEVK8#7^OWR(zS!n)MG zcaH1s;Nb|1WFn7EMmxTW>Y_}NGx$HLq#C>{F}@vzVZ|d8!`^4zt30mO*S*5rkat_g zOYTkBn(mGgvp#t!f4Wt|Y_Y6=Y!a+b^geeuBSB|2+>#8<4lk=USpeA@Ia!9w;_#ED zvrUX08J2@*GG%d;mUp@+%g)(}g(4O&PFX9-&uP0Oln^j|mVYvJ{^YBY=ZS@~xP#+o z`6u$t=k6HFxDXcTPddR9OV*9qnt1Ksq7;c^(I&&Y)J49}N49wWI;D5bf0C_3>E%Lo z=*|hPj2umHxeM}1-0GC8Q+Lo8SLBe`-yyfAA}#K7^$8W>aV-pGf6FVkUKKx+nz*l9 z^}0!<%O7(`XbE*-b9k~A?V`o$6HJbhKXpx|qc)j#L^crNn zI;v6LC2%ahU?l1`f&G#ON#*FO)rr=#_uy1ia9(T5x%b+ji?qh5fX{STljg*`ROrt7 z35sO}Vss$Ni0griMtMiqYLpu@LSMaH{hK%P^8< zY5H?BBV^ui4elaOI@G--P<3de635|gH^L_8Lvya>(3Xk#lw@PmW05^qoVDrayl$%S@E{Hkdo9vx3l*+9Q^I~!vOSBZSR%S+)oZ1z z67u?n(q0I~a5mYp4PD!I6#R!WUebZxoPj~``u6P5inThcGxPm&XIhai3Gqqj5^%MRnc5glwnLk~3PEQ!Bh?=b(ITVc9i`wA z;%?}Wu%j57Nn)ymb$$7#-B*YVl_`fO*_E|yGaHWM;QTPCu+OzIjdGEq)qS?;#=VquwD%!nmX5mDhlCq8)1XEfu{(# zh%3wX++n7r2+ZMG9jYVptOl)Z%x7240S9zoSht#94rz9^F}xhL z&Guq0obe#lIjIu*p+m0m(SQp(4HKTM(WgH62^zYUA*OH*9%q9S1T;Ir_0!H5Cb6MB9UB6&6Q=9l zr#=F&D<{Iq$*GfRcAu3PjwJEytAXI9MNVqhu$tZKUJ5BrkcqiUj+YFXNqTdICyOp*}w3Rc+2@osJ5bJnZu~@E3rTUyq zED@x`D)qNqwrREze5G2HNOfH;&~h>UO4M6-Cz#0Bo#S?BZ7U~-;n#J&){fOiQK;*1UT* z2HEZzH87Vf`f9H3WshLp(15d4U=#8+Uv8(}0T^YPLzlFjIBdtoi8GT9U%&aXJ#bSe zVf{4}XLVqWS(}^-pb6swWMkO^?z*(1uvG~DN`|Wf9$k3iQHT5u>yMG%5K%ZEOoN>F zK%CSW>Yj8?^(Kf^Wp*g&&5`nF9K<7P_K`w@8ZOUaHY|PcmNKVGP$V;MCa+~KL}JAu zuM*cx*jil2cBrhCBy{o9Mt*U-@b48tHuZFj7b%<|KGC@h<9nwENsJaa=G}x+~o3^1Zs``3w z=y{L<(<9#q%Dv$oL1h{{ZLLWw2%d_c`Uc*$e;eMk7{Q^6^r5i=iC3c*guf!Q$>F-M zLGqkA{Nh47m#Vo&!Ry9LBqO@OFC>N3Ij)tv8xc(Nb*A8&WONP-5ijSO(h1G{<*8=cSpSzbCWt zP`XVWZKz6=DPxHTgND@l#7b6s6(HU2(B95kH2r0_pqZv8FuU%MQX;xt(h5KP*R z^Uy0Ti2~a$|5X`rHXXtPq6o(nYZO`S?eK*-$2S!gaQaq6{z+og(1V7)7P19;6&^m` zaD9*pD7VU_I&orZammG;eDo}LN`>YOKUvXUv}!&>Hzp%)wtelAY3l9Oy(6_$g90DM zN^7~{lW9)YE~#!%=o}8Q+YN3KnI9Bc)7Z(_1RCuL-jO;`!^$O>uM(^qjdzY+{w*b+ z7%xt3PVPKP&d~3Ebs-#!XSvver@ZA!TqbSdcL&IxCA6We{VrPGL2gfpZb$Zs$-Q$^ znw{s`;e-sg@*K&dt; z*Ms&uuE@gGw=9md+bJHjW{JOa+n4Pcg_(7SCZNMYGNdz1b9FM;9BIno zOklSFhl;ro?@KN&wua2F7iO3V55i$u(jHVBUfPLZ0qYxit;X7sDkl7;v;wFP`N6f* zHW1w7R;uGx7?I}#jV@}%LU1T8QfV3bJd<@;6D}4SQqv{<`%lqRHHp=Sk{9fMXEFRD zRkG;=p=PGF5a}?GFtB-$Xgic;7HJR>@+~s;=z}3j#d>u$pf1u|s;4L7qlG{OOx-DJ z3+PYD+^4>}5M*7zw-P-VOr3bO_Z1a)|@Fs(vf14yQpW;1=C+b(t(004Yg1H5b%2#|h~0 zbTR=}$%3cnf|^zhbF^;Y{LnVY2Sz3Ts7h7Rjq-Vte{$tRyb_+MjoH;HrYAMwfdN|Z zKw^J)5zRyQIvaT5QqVizPD)@u2g>PzSY{JSHi2_=LJG7RSTp3$Hiu%gq%kH5-(%d1 zN5UH_Ee;Z_+w4UJ?5G9`1bbR`lB1mUR1_7^HKVZ1SsrjECkk3jAG-I#F@pxIxt|V^ zNG4{%YXud=6;)Mp2}=r#PmL%g2)bt-7fcA)K(6)k?FY**c-31k+f}*sC)1nvm+SpX zeONYjccMxws~-e;tL`JHo(GsW7V(7>*v zzWfo4(%i00zz1YE6A@gZ-Q1BX^&))@OG`U4l$ItL{OZz9`+H&Pm@>+reASSeWZQ||e)!uD2%b9E2^4qVjtQNoPUi0D z?@a0Iz%6(5gOBVBaZd07kD)Z5qo(R;RvSS#K%cB<3DmI09J}O)K?7oRC0NhA5rkRA zU;m(k=?VJ9uKX$!m%KmadP_%I#d16WxK>b(*Js7+P(*qtDvT8R77GX;qk=1LHDsWN z7N)9gH7rey=`P7GXsDJ{*g){guMQ|la24cb$62DiWL7z&0DcOTYXe6^lQitiar74Z%s2WB)#A;sj&j$D;bY!35vd|rZ2J4{(yyPAV?dsL|oF1`8Z zKX}uB_|mk;+F|Al8rrHiZY3*s?x{D&^YtO|$i;php z49{QdUs@l|pBn$#pMLW9AN-fkf9>1f_||v+{NrC-`yc;s#T(AdR0kKV^e<`;XRO;d zxR+QP>c!KzXM?=O(1e;>)2!wKpS?nsKNT3wY7fEJkDuw2bwJe0qVN2J=`A{u%_#a)e> zC0f%gC*DIg#0Q!a&?a_5VNKVB8pWj@P%Z`9o<;3x#Or5qX)1le|4Iw%_DMMlXJ?xn zCnvX1Rs-4Pa031=OkrI_4(D?xrc(&+B;`qm5xl(E7n*I#OT*D|2E~Ayx`P`jBD+ZE zly_v?To6@@#Gk-LnBu2WTtwD~iWwrF&|w+M3;klqEC2$yQliASrCSYE4Z?FMf;%>= zp+1NitZERl$*hZiWPUL?~JRu*iZ+ zD=OQl+e6#K;^zIO)xXYM{rvMTzVyAfp7Y1YpK=W*WKRNisw7=d^?(N^!J_m*rh*y^1omEzuj>(d&z75WNzdB zqA7PPyc_3xQQm6Ri}Tq09J>6U>ch0lx*=BgSWFDX2uONbcKm&(_ z5jC~(#kxWmYI~juuUjroIXc5+v7cTar8h>it>~iq;M~=Ix;DJ#^qD*U>{IXmyDxv` z8{dB7i6_4BwQv2_;L(l0{_H2e^bPy@jc-35|HD_i_x|HeZ+;-(j?P={U$Wd!+4x$8 zJ5JRMo1lhJ=|{pD{D%&mpJr$#-!$T$43&4(Wix%0CxSZ7RUMa86CC3txFT8;XyYVX zpY(ver4Ce0M=;DTm|rC28yWN~m5pp(@Sv+k+MEk$IuHT(`fDZcqJM-oX!(Hmn0iF)E_if`WEdr+vx49k zQ-ez#!?X<~r5Qx6*NOBKQj3}kx37YX{zb=-MJ9krc`|HrPTHJN1cpX0F0_WlW>jiK<%U6aSnl0lUVHz{ zmCxQ^&K33_{=H+*`1B8dUupK@N-kZ>XR?JtX8ut6$no^C7iE6fvpvu^Y({U-t+0vXTSRR;|A&{KKA*qzrFv+^M3oIg^kgLYlCxZ zeG7g!{Q3jEd;f9x4_`HoTb@sS;jz;n`0NY+%SS8S(S=J}7gYOcee=_GYe?$b@#@t` zT=+q%J3&)v8uw}XvA)iDJ*d3h^=NcZ~xF{UD zZ_}8nSleBGvD-_dg`x!k9}4X-X%|I*0p-3TM0sJ#NRdM_QPx3yPh&kCiUERt`Q($N_s*2L*OTC!u*MuALNN86 zt(CysRhO9l^+DJ|1F0olX@KOa69rvZb+JtSI+5f#K0?O2Ry-aX=u8OiMfNsg0J)(a z6G-(e^^bLw@GUNr-D*zcDt-yQ0a8zNH?91|NuAy82^naS`z`i&SBiG99AQ*(WvI!V z)t+9}2`Ioxr)O&_5gkXtF75|?C&PN##c#GYQ8)gGHJ81@gzlBWpGfJPhH5$32th-+ zglRbUQ;HBMjYU;mxl4X(n@bZTNOc!YS*^d;!c?46cB!_1M4avNs}g(Y<|w~8Dt3l* zy$32M|Hq!okIxsg&%Wi>(xEHgeC;_;yzqOUy+3{BHOFqa?&x(_UwzGy%db3q;IjR5 z2j|Q4bLC2wY-%++WPSaIx@`yJuhdJ2zi>`C~u4>!*$$O4qgy-Tm;~ zoe!2z-(TJsmwO`_=zw@Twx{E37Jg)HL6KfTETtxLNj|5kpd~hQVId`OXk6UWZxdtz z`ZLnDZI5s>!kHX{#1y7gl9%()j4r74pH&@L|Gem3AA8e>{`upNe*Fpl@4ip{Yv->% zbL(3knrTPp*9RBXhPylX=aewM8^hgeg9}e??XC^4xckhB_k8+~Kla71ee*kZ*P~zk z<~xUvzT^)duJlG1R|n@+2fOKD93iTd*Qor@uj|ZWWkT@hR5|F68Z@suh(uDy)5P#RkQUSvvUX zwwo&Hu#}PX*?pS(oLxK!r7;3wqAs{|Qe4Dl<}|^Ji&I|8Qys405_~`9r781dqS;SQ z0PV0|e3Cp*g!$rk(dj34CQ6LoF(9>hav~&#ld@vK9aG}{4*xJ*t9))4%0!h<>yx)Y zwaYcST-+!zxQj~@wO(9w{Glh|emI4s-^uQGE(9kZWIF||`SOo#=y@oDZ^94&jYqG${^~2PzWmVP1N#o| z+k0_o|F0Z+))O!Mp64ITUHF_|F8y4u@T%V_)$W~ZZ_Rh&x$bDL9TnH({03Bc(d1Zd zXh#A6TvKuql$q(d_RMe@0q8(?Ta(dg=^#|4Sb?*qhc2v*c9)IuOed%?7^2uXwd*IT>@L3C6msSU}ZQ@?y zz3oGQ1za5uy0keo<$gh9aQR)^2JUx7kACs%v|PUatv~zN7hm(IpFDWy*z%mW+TXp3 zqO$zQGR8z}R0u!rbZLBD+qM*qSbwcH+8(ZSxB~FtL_4T4O!~*tXF8y#dcYpBhZeTh zALzY~KhCu?)!%F<5WHv9j?}BOw9&IwcM&rT7(%mN{tzvERVoanyjID}97!ZOsRM0G zQ|h?5H~}<7Vo3K(Q?AS8_QXg@c)e4wm#2U}ZKRZT;(JGilT*iuDR4WHCr83-p;^eD z&Qrc)>gpYyM?ZxKu~-CvQg&o1#krPSbAziO%q;|SW7&-kaU4^AZJ>Qcse5X0sog;? zd^Zu?81cXS_M+#x;;=^MSERT_ zc4St)g$zzV{LK;HnYt61io>_*bT2J(N46?M#4tQG!}R0+8?A7Zsr=c%Bj>)YOuIpq z*9J0+5joZdn$ieyOM{$Mb!DK+=eYG(wEV3hoD<8lF)FvBx#n=bG1$BQKxz3eW)5Dv zANaoIhUx=Jp*p zRI2Pv-}K^2w?Dr=n(suF^*DF;D1VwCv^21TT5JyTE$pdgS`z?iZ6S#3F9YNt-3+j9 zAw{^iamm&OqG8hKnYxF`Sy9YbLk!I<_s?Ix@6y`f>buXp^p8IAryu#E4DQeW+qeGo zp)b7jzdn4~sqrN=!7t%9au#3C3|-7N(f|5~se&w?SRH!d2He{4{A&M#H8l3e-}SL~ z3?KRA7al`Y_}1ee{`6N){`seFx#xk|*6;`G{b#iXhSv=L0cxtnE$BAaBHFYh5K}Ej zc81TL?S{1!w`~Q@RA97qh?S7359}n}2o`on`ZwaGq3@;=wO;v}xX+ben|;+wL_JDi zGl$OlTG%RQNSKbbdWyUhf2Et@<>wx<4=d(~&F6kG9CnVGzc4u|POc98ug(0BE=+K$ z@5qcqrj%6_k780d)63y0CYh6z@`C#wlXUwz@kVf7*2x__cKRff4B$w53663=)<9%H3B;OXl$T0NeSuYE(^Vq1 zUz}P16g zw$_ljxYV~4Afti50&ae#Z|5kXKCqH!Sw6!?Uy1x{!_Zf)a8ey;&Y&7#Zr`tlk~J!J zGTRZwaF=F_73iD-szSCl=WP>@Mhy|D2I)4wImx#M`A$@B3=8X~g7GEETx+mz^TEoA zKi;$ds)O_8TW-GP#_NwAz4qu0*It|3f7u;<8+vW~m@viI7dHlz}_jw01$8Ns; zxzB#?&Bw0af934rTvl(Al>tC?4wHpwOGKiG;yhJcXl}y7zzz#22*}Aya&+?^0pRt`GlH^x3a|o1Xduk9@8E!O#5AJ3nf7oxj?@ppLh3 zrWXLprlnYHs^z_*YS?RFhjDWjIfuJ9B0F?$Z&V7J8m~8Bc=XsTVgRZ=)1}5|S8#1Q z#HBWJW|`l9+AT93#Q^jhp4j!D*}6{zy<@0A2MxCjX;xAV$EP?A53YLoG5cCR3K~93 zx4D*E^;t?%B%IwUE^bZXo5v1JZ%wD`uA~H{=s3x{PWrtLdZtVgc|JKgxg*bxtNMEa ze-k1r!}L#_+<|QPNk*4=QsN0Ng{ywc2@eKY;JXhf+Z0~I^tUHcei`rj#=bnLXD7f2 zD6!nv+|QF3>Iae?^TPN5Uat~0UIGr16nM#U2BF6+9bEQ2mTo6jl>N=EI(ewy`zR+A zLUnAF-=e$%-DXjP+6s23y>PznU?hmpzsnvNB=pi0$dpJnVL)UW zFUWwFEOTpED;h>5Ee@2<{=%0qmJ{bI_{o_zWXwSMm>Y@rQb)Tm zhIN3!u`|SuWizTYqcYr#c&-!eJM~cMHSgZD_sHRWb0(z?$JbwX)X;qNs;hGQ4!`oU z3yk3Z{ibK+^MwQZ_Z`@`@8E&`mmNBE`Qgj1xXd7a#Wh!7ee~MvZo0uCZUle!t+(wz zblH24|G;A}c-nLJXO11e?f7-q>^=J2y(j3wj;}(as^{R60xrRAlw}Arfu=5#`UF!_KpTw^0JHF5A3j2vV)cs8m zjrbmmJu5TeoQ(JARV7wYxn?Z=I_{fzf`0w+i*6lB%RK{L29>~VO$cF!9 z1^@9xeNZsXVH3@+T0F3Lb9a2vTgDgs=Gc}U6;ol$gYsg(vWQRX{A9_`akAu>DpPrq zOAv8O;S%q{r+&)35dQKgFFYv^)8|fxlj*-Fd9sMVew<+9^l@V9@50pI6FZNSPmu^} zzdmr>^!V!`Jc&!6?64pf1`>Ii;Q19__vd+J-5?!^gF;G<-U|rMoFG+`-$7_6li3U` zjXLrxbzqh3TSFrVxy<%O;aOdGAJA6N!7J2L5V^_R6s2=K2@Mo z*xgjhut%tga93Y_WHOwt)*$p#p<4wno zngGA<>MO6h=E#xU{Gpc}de##^`Lw^e^#>~X%v@z|u3RdYilt(qRPdv0N3k$hsT@0c zz2W=Tn{U4D*4qvoJQ&~peP903r#d& zoO#(FJ^Uy4f8k4C`<9{nXMX>0&#P@^+Z5DolwgwF%j^skMd#I`bL-Kw zR)*(QadQ4zbZ#xWU=8WdS{|OiHhg9?vP&1NOb*=Ah@a7opMKZ&_q^%M_x!s3#ql5B zwEcq{qf1}kzvT4;9{o#SYv=t-Ue~|$^@B_8FON&~7b(g%xpeknmtOl+>93vos}ml( zU(?^Mlj$Q#e9exOvrodYM<=iL^h>9%?s+xR_q;lsyb32XukFviwx4*M%+?Hd zC**9fUs7BlN~3v5pia~!Er=$lt#UQAOi50Q!^+H1HfCU(z!T7NMl(u7Wr?zYvc$b| zDjM;|VdC3hNkOuUdXi2C#C(tc4!#!|EB7~|1GWCXjgRbm907s%8t z;bf5NMdvpLFM7vE4U|9s2Y-9+>edA-TNf)3%1Tb|Qh_bsR8(njX&OHChPHE;w%*?V z=g)rW8=v{&V{iG;KcBz6m21f+4j83tA!lq>=Y+DXj1<4U$u1^7qPt-H7{<)KUj~>UHx#T{h__x;mxP-`?)uM;3aQ<|I6QY?8}~l1`Ju~~ z`}n*%Nlv_k(g`pcntIXICy6f#vxJ4dAGb2FudrR zKT%d6ldKJW0lVfUU;)d84=y!vS2n~tz=Ew}^Ym!M07Zy&jStFFG{@*|fYI#k}f|CaKlkH7F~pLzcGU76pV%N6&QK<&Nd%Ji|X zJXa|c_RY`T3>U@zck9&$=05);Pk;Ei-@EU|=O0?{Uv>9`m!FQ0oE~3x=lIZFIUiwri!CYRwY{-a7p)>*1T^gLTFM~{mIp4hsMb<=2tvN$W35Q3Hz6J9CWxnKCS2SYTz%*E zH@^M&%m4R>U;c*=KmNoMSH9uQCDne0$dAybl?!jB{{o4mV^6(e`_20hn_!IyBzyG!Wvo8JeSHAwyhaUdz_k6ti zj{bA2@1ObA56vv%Uzt$Sxy6xR0vYnOq!Yz$49Tuy;DAEyF|Ig;^W5qLAC?1uDQuNf zIy$jL3dA??wK}=#9A9xLsTy7m&!sr?EU-Yx=9fa5i`;uyw5T&AwfhPo$!-K&#E7yC zWJ$IsD**b;tW|J?uo_tC?*h^41CcaEK4$4xAcb|c6w9DI_HsjQg`em$I`LU9UI<*R zL2y^b(7?{ht{^~Rp3AFQ^8)Q>s=@MWw4bUtw*v9pWt`8pB&e;Bex}w3i&uS{LQKyg za`rRYhPAmTNzTZe90~($1?qz?Sm|pGRjA_YjzWk3RL7XO8SibuJUO`eQ0^Ch^YAsf zldnGKx@&K_{Z?q_<2N3^0ehI&UVY_NM)1Q24&(}jO8U|-{pi!b`eRSOv9vpvFB!ol zz0ZQUrIZT!n~vRZ`z^QJc>LDmhe~!!|CXoC|6=#5xBcx^r{gQ`9$#^KeC7J+veTo( zca9IA9v?7{-#I$4F*?|b4|d}H-O=2sQDq}4Z}8viTWtYxemsMHT8>VAsP;m4b11=U zqYpcL)|E2<){zd`%OGkgrg@*Sy0!G4Puu*t@}BKWYW*zZHv<6fC3c4{nWae^ikcWi zJjnRiMmguK-1n~d(NBNrF|5z3gHQg;W4{x9?wsZO@Liz6)T)ZnAM$)CQo~C@RlUM| z1naC#OQU>a_=$gc?7uzmFXyjrrGt_3=kP_07K< zKQ{WSuYd3lzjp8MeeK?NKlSmxr%HKp;#21NUwiydiv9oTiSPS=(wyJ>`Un2&WAR^p z^Y2EFKk~^Z9{Wnt$zS>0qrdy-_dmP#Hy57#@C?5DCNY-9J0lv1p|RtT`IIlv4{_kG zI9O7&76@el+6`-CLSX?Jj6!)+?n@{(PiT_ev;0Tj^S#lL!5R z30XH)+ec!H4q&wpV{r}5Nb_fYc`Jo%A}Nmi(pF({%Zjp-ERfpAS0bzZoX%_b%O2#h zhMzfGS+DXE@ZHyulhrPeeL0)jgQQ|>@64W+54Y<@`7LcmDDv?-FEw}_>AQE zO(gg=*O~@DeDGi{o5`kAv$L}wy!8ja@#9Z_cIDD+?kNbquUy%0N2OBAw^_jBMbn6Rg!U?If`e zbQ%_IWNoaQ1~ME;y2<%-mOtG5tAF^;6HnNqFRgCns_1iMu)>?x`QC@t|0ui`3S>5W zA`0Ay&TsXvfAg7df9LU6{L#bDTDk9{h5KIn$AACLZ$Ezh8@Dg6ZKYAvKsB`deiu)& zRe$BEe+Ss!sCl(5eRp;1%s+kgFF*dpi+z6 zZ9DKLsC@Q@H4UxfOUQSUg#h&So(!I~ymheoH^1?oGhh3XZ?1mxkxzZ~z3=?esh52I zrH4NEpYo6XRQl2Xc*!F_cEKYrIPc$&|M$m_U0~h#=uf3S_p|xW|I+?1ZT!Ml{^A|q z{Ol*x=Z}5)OYeHmnIrWNUa)*$!R9i#}f>FRddA+nH z4jN}H-E!hs+7dTZrv1b&oxzRw+EH8pa0?5bpJcZ(aU;qh{ctq^Ekib{dY02J*-<>P zm0$ES?J6p^fS);KU}7OW$W!=qOF_-UPaF$dyQnpAm>5YlC~57@u1?5{41AUC0Z3Uz z5uj=lSU230;Ev7=d(93()3kY3lwU;)8Lzf5|211k7CUm>WXJ}>CVwrnJrUGy_NlP? znrT9In5bTYJu6cazR?#MvR<<{DHEVKc2{xaTEkLfP{x0zYFOz;bKT+o*6`rwgQcIp zGgmGi+;`2ZUpe#BFV5d^{j+bo?Jdq?v>|#_W9rc zqjQ&JriIK&=l}p907*naR0wX!Hg?aKE5%|Zov&o_<#e{NKYhvddvmk2UZnm27<`-bhS-?)9%8@I2%YjowEm2rvyH*--f;JNWPkT` z`B7bW_ye{3dLQ`Ax4-@PzRh^IbDL zocY|tpZ~fIahk8k?pps7UwG_~9{iVcmOh+qvN8=t4#bAprZZjDqZmi|_7}h<+3mQn zzV+cx{p;U+^2--ix6&wt!Ku!7wdRBC!`<|_vLn(o(cW1FkG25DjCKhl+g5Y zEIm&JaNhEVU-p~*NB@bs+ZOnr`pB2>dHLsluJFi@p8Lp;ob%|5E`IdImp=O9-H-mn zo=1Oj_TP{HuZMNxPwYmQzUX4>-ba4qnUDT#_RFVV@{NbLwdVQ!BagiLw+0v8@u4*S zsRz-qXx!Ygm&9;J)EEjl{f+O$t0ZO~_})rVr1nyb0V1;D2y1<&=f!%S3;lvA;6}ol z9L|zCxh3bgWdoW?;ryZ=*B9}CW`BC$i}DPC;8fyTPoNwYg@D0FGYs@guScSV5am}P zcD{?G8_Y=iG=SQK-IxHHCu9IrQoi~qktd*u393)8cq_0F6OF_B$+jTIAi;2 zp%OKLDce0}0y>mbSw=Q3#t|^9zAV6B4Cj_YVkRiiG-Uf0bzI2S92mZf2K`2@ZcKX&x$>#lq5um19$pZSTL2o4|o#+z2xuh*^|!w^xg~afA$al^xlgyx#EE`_AT>;%AQ;$lPm1aXKty?{KCPD>sMazuIrz< z{hS~8*mIx$tA{WAnS*~qd%fA@Fqb5b-it7vFm5sL9l+u%&wzzpl) zWXqsrf;fy4(yYBwt-VsrFk%Q!L;oiT#5i1tiAD)7Of~^nnN2P&$;Pgf1u2bV;8&ky z6R;bCcN+EFYE#E>ZFb==4NdlLSynGVCnA;bsxtaI3yr*_ zD>nEt-+Otl)Ll4+9B-)Icc?@jQ7FNOGllwQR#G=Cgt==tE_VrG+U5&@yaXXnQI8X6 zSd25`xHIsN6FD=u{n>9}9LF{TipdT63XKK;{H1dY00GglW$FH%#Yk0=fut zswy)*AjUZwh!CL#} zapIm~jW61WTom6VF^1(k8Ip4^!nY-UjTh>k#>--4EsGZeC9F0{Id)NmBBP_AsWvg1 z#;+<IP9b=&}P(BN{J)YscfCXoQa17$Lq+<#br&sXEyz0|i;S6Q&HJUCD;pVHkYaEkC0TWsV+Q|Bn%`?l+FiygaGeY~RP#5D}%6*cA8YGCJf{Z+Rpojh6C z+%AB+xXFa&t{U^FQjp7_!D9{IoxW3YrK+pKfaAqfvqb3}im7N7DUUgvnP4RoO|C&0 zDgeq*N>?%qbIhC!8MFcr9?KvmsKWbiy^id>p@Nzg=IVIbS~PJ z1E+#gg#8jBRI;m*5L(a#CR>3#{G=%_0wSqtnkVGzK0!c1nqYF8&Lnn(tYZt`C_1HR zYAM)`B@>2nqqIhzGUB-coPt2-5$-o4g-T?CJJGWzZ-k5Eoji%b;~koGg;E4u=qo3o z9#u?9(HNPCufQjj0+gm8r70MsYciTAJ(Dbb)aJg)MS;r`0{rFU#*UpjWg78%%A^TH z!`5^dJ7&z3Nt2o2lg3Zz+o!j$cUPIj4+tJ0lLh_ZSxFqZ;CFnA+yS z0_3#_p{*Q?TiGczEp7#0&(Cg?IK(Y#kVoc_whITyyFaBKx2bHxRuJRG6SIoP=9Y}j zDIT2zV!U{GE<58zgR_bTWZ4EF#BKdDt$ow1y)&!;@m@(-|L#eafD8)^Dg0A#Wu%M@ zF>DhxPSZ(M!c?I~rk_+!uNhAqd*Xn8Q-??$5FEsEwL?;*zCGjk)kU!*u zo7sf%BY?8B_iJ{{am{;YB6#uDYA$ew_V(8=&z+L63L1w{?AEExuAs zS9CTwM&m=U7R?s$o!JwdBwxBi-u+v z4a&9+%0!6w&9e0ch^Jb6W?FlpeUJY^A~4+|&qRU;;DTjgh%*`S1hH|Ez>9y(u5&F> za)*sE0~Dr?;kvxSD|L-_(Dgp_k^@F0G`mj{o5)5!^}bFYnNPjwfQh-DafWvB`VKLA zw+LOwNDT0`eXQOsQs)(o7BM`L#ocv`euR?e`&P| z9fEUp6Ehln6s2%!@Df#s{(#BO%ENV?(>2_j1A9b%i5Bl@?dO_uoUy!nt1@wE#hkX} z()#}QQ|$lu;0t=ZVs^XA_<8lWuE7a9tUnPP?{w$%@V9O1;f>;#FaMw)V98fMiik+BL_UWD$!`r;BfBn zoYLu*Cz{$v>#~mi(%9tao&f~M0V!S*Jguk3e8^Cy5f4oydZ(KBV(?WvS9-vygaA8} zP5$Ixqcxw5!vcROMNQQ}L{e^az2UV-bIwpzyt)!GXIGv5XblV?zEhn3$k{uVU+#B} zqX`w)udeMxOeHEwm$9n|bnu-qrPMNbdBpQ4PjwposBwIuO_I!RQM#PZZ4O6MQ)AlgEN_BJ{#A!3{{F_m6gF$ixrV<4@Ej2!A|69|R#BgDXP_EhKa|^5u)S ztAZ|cpdLsP*5K445`{q;Z}6qt1BGoIA3xzTUbLDd!8sAkaEI*}+`uM-_KGqvOffza zne{R?9G4sN{lo&j7@D)nL^O0Z(j5h=) z7`rKqJrYbk)ux_Frd}!L{z`NIM05WX%Ye+{!I@=)v&#l&6b(`z=@)&V?@JlI7pxv0 z(qr(DK_iC^9XE2=q_JbCOq@7wOo&t}?c^g_)~oZWS#1Epj?h-M3qer-TXsJbT2?P; z^FB6h{9<#HE}@HhzRTV3~KaNf3=r zu{hBR(N*I)O)|BM)}{S$xxwz}oni5gGw{g9PbuR2(E?Mt08HG?yLQSH&c0tI?itLJ zh)IQlrpCLAzha?yA&E@*k`N$P{OStC?kCB}t7tkY4f;#>3NPI46tCw+Bed4VfZSI) zFI~z(MtzbW?&NNODxju=}? zlU)}i30eqI29}{1B59i#%{jouh6IlSU94vo!zX;jLLt)A+PcQ0gvTD5+T0c2b)86J4BoMNh~^`v zosB*U1AOh8VCWjB_leO%G@m5Ai+p40b|NRoOKtRW4)w5BN~4#;;Op#zp_0C8F=(LV zSYal$PprP|TJ@n*m7SvXQnfe`NrLD|FmtuZPn1FjnAF@|xei(4EC?4$k#$e~15hVYy$$-1hemWWmY0{PUIm@kf2|Swe#}&dDVL z1@IbFGm2c&xkOk!fr3D<5MmT5L9~Hr08NOGIvOZUkVV}MdkC_$hjvC1>!Mf@i~Lc5 zM+;svpb6ZVHXup>8uAYc!w?W5a4ZlXr5Bo*Oa*^d@eyM5fQ3fUAv{d~NYLPn&UnF4 zlrRa@een&DfcG$aCU_?#nwZ@vXW}s?WPuIg6l)xr+m6sc@O53iRQkk=Dr!$ep%Lmnbsj>Rt(Rwjn1%zq+3FA%SXn)yLppWSnAZ#c_%0DIXxlQ zI3eX**`(#0hP8AoeB}Gd9ri3AvFn9t`&N!TG`-W6$t{56sI31zyU7Y~b#MAVj!tg- z-m*cvR!!aW^tjKac~#GOQBevdLcu4I_(=8w1|~Z-AojIt zWBVv=)K?eqnasSRE@(B-sgK9j_N6|6`fQXrelU(nHV@6RjmRz5~0HZrqlSf))vsbu(otJTk07CJxaRv%)hccRfh+0;MBCQmW@3e}92gn%beAudP> zU4(FEFWjy@PTZr}-^ zz7YvdV;LDaEqo;0ZAz(|!8zDC9x>n{NYT2YvhZXWojnB3zXiq{gX4_d63{>C1q4@_ z`>4$QROW$7bN^Ip{|wupH0zL5>##K2u&kotImM&1i$iiOqc)$PvM##*dJCKO@o_P` zrfoSnV{6&eU1vs2nsatyiyfguzWV0imD~95YA)Zn{o|1%JEC9FPHpE{^w__HIPHJp z?DWTWuAcMt7oVTGeD#6dajUvkXR+n4URXB1#iEgYLpE4vZY`g>sdVzD(n%XiCTu7{ zh=Ul@t=nn1)?vL?@y*3s4;l;??v-4w`r+)|?itp$k-FDEJy+e(G=5`g zmuNkJ`|A_8uivj9lv(7PVC)d1k2!d;(QaR~=eTE-9^Z(ah}A>dDowt0LZMKU&+)jl z5sKSsu4o&r!~Z^^#t422laHF>Cy0v^2gM59BNCD}XQ!Tz!cSppo?oGhtxJa4BhTu& z!P;(vEqIguW(7XEb0=qY#hmtkuRQsm9+pcg=61M~`V^e9n%lR=B^>n&GoqaxLp4TU&|TqxEQ%DU5vYwe3VR^V$eLPC;+g}0C8|2*Gb`u_*lA=yqpuRi zgca;&KSb<_ptM|{SYrTrBLGr4A)f2NLL0u~Js4wTSfd>jE!`@EQ3$^A2y2$!PUfPx zgAL3-UI=qxo{$L+J7v^xD4l3P;O7+Dek80gGVAm>QM!bCS_Lk4?iMEmcu$oH^kY9I z3h@5P=Kg7DD}w+Z3<@{HHY~SjR8G;D+~SZt+xU&ACPr;L^ywp|nt#lQ-!NtCnK`@4 zho}DV!Vu}cX}(8oW%!U@W8;qPTj$T6J$LTxSJu4!=F6)JXL_JR@^94l#o4VQSI&8T zeZ-0@0@Ut)#J;msHy!FLVz>zN<4h3623+r zywE0E7pyXsT&aSuEB2jqi_zh0r-`CAK+&Wq(SCMC; z;>jB!wg|zp_m62T{#E zIBviPcK$qHmeC_WzX{9!s8;|v`TLj{`QLaroHL~2gnt-K{Pag-i00&ee~|nStNn2J z-&T9M-+%4s{t}M+USHI?rsyX)arvLvJm1y?Hlvy75kq>8Fe>;Eqes6R6lApE5BnaB zCOL_8mVjuCL=jqKVK`tw_s93g3q9 z9keu}gyQy(z~oe6L*RPh?fBDAfN$6X8**F^9<7%_h!JT3!G3V(DB*Z;Cwv7=`@z^* zfv|?rE)G|DAxfNH60etHz(ES+0|9ozrzC;)Ct#ycJL<`4!Lcv|Ga4o8yAL3ViNqP zCVe|&d-=S*r-w%Ef2C*Zcao#{T@6uDk*!)i?(W{v!^5pZyH@hxZex1;pPce2z`E%- z_zrDdGxyOYeZ1to`+2yv>)5fqr>A?%mXFP!KlkokeA~v4Uw!&Q@Z&>XOrE#@(u~c; z)3%mQDFBFlr`y$(KepJJwUDdyfO<{qi$ZYido zN@F*rF(A=MPxs&?JGOirwHq2-bbeLnIN=FZ8HCNEHhRVBhh$jo4oAf2=Q^&}1Yj=? z{Uu%ubPDu7WL)?s7&?S$_F1o+f4)C9uPF8KrBgR*x~dH6KVB~VrE0;pW6$n8F*eKQ z7O9j$f;WF+tZD{g07( zw0Pad;o-whuT)c~ER)+tv)d+%d%neegT*7y?2&JV4Yv(e&s<&kd3>=~W$M#^fdB_= zE%dCI)#lpF*3QuC+KiT%GjtCRgB7+Hcd3}&_S&x^x4t%`6-NF&e3y9Rh73v)TaNe!Cx%&*9`9A>lPhKHu%(2#_rlE$RTj&Tjjwqkug56#@`K zy|CI3*O>xN|1R}zE^$$(irH--tB2d++U$1tLV`bUQh^Mg*YUPu5uCV7m#zfI9F?FM zK;W_<>y8a-Wd>%Aqcl2GA1Xib442(t{ z10x$6OGZDzHk8Gs&|&Bp-^J!5eI)wK!-Ot&d@Ev z&?CXvQw_n?M~(RIlWy*xYU!708I)!joM|1FWgVGm8=Gzm$+eBmFP@NFGccgW8T`hG^nliheL417>x!WpRV5CUJ{9~pPx_Hu3d%>>L0C6 z$(TFPG5=qH?1$D7wswR*wtI5d2xZdzIkUQS=>$o9eZ6HeNt-sUcJAB`xn2DE=cfnx z4GVo^?)#T#Y%QL#rF6>HvPs)YCjrFsi^pv!4#_VX4MPOnGDubEtO4vKd@2cc1a53JUxk6swUoSUxzc#IwsEWW zird?TxauyR`c&>*<3vn(g3&EZv%_%her@B&#WyZh)*d~3w|#^r`TL)18)%Vag#Z8` z07*naRGPj&T?t%%`?ItETyrE>bLG(S+g&2H5a*4L&a_=~wD@w>Zfk|<=ld|i*ki8P zrn?GhUj%|jY9wkPxRD7?6abSzY`->mCt|f_3PZ%BJ%#cGVlL`*>#fn% zpKq)_xUJ^fT_Sw5^TGG;-rp75j7!3I#RB*1PvUz*AVRJ2>ssSiHD7PLlkps`&2Yim znP<&c1-I9Y|5aHFJ6&HPyO;OMgYVv{H65(ge^LF}hDzn4>x;eUSyRa0^KQ;d4}N;D z=HT|4Z+F&wv!nX6{OYf^)EwG{$8uR0_q4piXbYS(nkMA3_Wf~&8UGKAVXI>d{uCN%eosc^ubqI{aJ{U8~&uMq2eOkL@#a9cz{-oBSb_A zLCG`#ssK#z0%mm>g)l+Q8s)+fNx(?t4o(9~Cpk!JjA^<`97;lFG)PWYMAy{)B9Xu!tWiBDVm&ETR4feuE8^ z)hKV9QOkOq4nXIZpyyi!wjsbz)D}7!F`Y8OQn5uSA>4=f6@iQUWz-fNfwSdS6XnJ- zwIPCz8y)x1K|*rkvQ8$jj(cD~r{X>h6Gl-q${#-!dbvU$7_aXZtM8^TbXO6+m8O0W z5>5S5%>z@+1Jf;oGps{1tRph5qq1ycGi~Fti^gs&p17fSYJSPY#KV)4znfKXeAc$I zSzC+ezjtBWn_07i+)kFGg|dD7HaE8pz;56;e8ToqE1!h@xmoFk(JdUy+cvob_|mq& zOl}eL;&S+P+m-@XDyh`3ZQE9_z4i)vuT?eAg!CFZf7QZ$mu7D*o?TEjV@uhTt)-I- zN+xe89>1Y@Tz+v#9-2JE@{5M$+J4Hcj6o38#y}PH zUPy4CM1yyd5eG)pq(Yn$>6qg_Y8qxG7`)^3rPr!{I9chnUL&CsBqu!r=UqZ4%U7WX zvbPT_++TF#%d%UKtvTWyum9z4-J(6^s_!m+d$RKJh{8^BI*%AFq&;!1`bJg#(KD5e zP4>^rZnjx>^w_oPjH8#6fBgCI$=e{|r*9}J`=#o+4^DN6(L!GFkZDlh$J=q`*{gEXZ!UwBUHVq?tT&%pEtHJ8TAUBeD_V z`5?C~9Z_^sK$Kg&^30E<>N9^r2dV1FN2p2vQjnR|A7$b*LC3W9rAO<_3L!L{ zEp{|EHeNe_>&;ODtSBKn+(}u9i{dVy!kEYP_2s(yW15CDwwkZDHWQqCh1l7RgsR4C z7wU?CtSkAc@#=Z}di~O^H%5Zm6?x8uJn>SzoBfg)PXPTFb-MZbu!c(~9QLOA(^epO zUGWc)5NB?Se&9!zD(1W2-=}Q&eug(4K5%?~XvqkkKKf!U;mmRG?WwF;i zXu4ln88ZtZzWmR%HZDr(aR1vqxHkOTa`(57!Z4ADsUt?~kQ-6N;z;C!f4KNb5GI=V z6Nh9ZI3g~Yp3)crXw;O^0+N^~gE$&BIFp(a3ZT=5EZPZuqCcabWi`W-OaE1&kPxAd zzXpUg7wzxtDnze9zw!p)v4xzd$Grp&P7_GR#tS75_cq)`eD@-zV{j*z4LGLpHSUBq z@*@gk&EoKOLM}6=W5oSNO@^gNoiZQLz#0(IfCLe}6Dw}N1i|-*I}#b<-oWjMFk&iE zhrxw%WQ++r0bxo*P&^25eGky!Dq|mov5(5wPi-0qA=x}I)iNmEIwaFFG}|^Jrzj-T z7Lrp0LT|!`;z=8er*0~lv8inK*7Df}rE|BG&MzogwEx2Rw>JR6ORZ+OTz2l<;pWz! zSqy@@YuC=*x&@`BB#xXkHFdxfj^(WyX>nl_d@gPE!}urq_v@3HnGq5)x>F|)h|dHE z4gSU(uj5PCZr@oxJb2jR*A~5hdCs6NM(1*P$a zF4i?Pjma-g`SE8!_Pp(7EBBq4xwUN7yC*B|)jzxcWa#elh_B9rU>}ifvtGUrx_fZC z1&;Tdcdm*!lU@F@?z-+S}b!NM~EqG6s(kl-25@#{pG z&Y;;Lhud$D#k33Du}X`*?xe1){y3Wo@ILOiEbe@5UWc2jdSkEluKMbfM=F+d z#r<3*s1Fc)%^3Lg-sUw|r#@B@`cSo@b8=Z6v)kS)SPRi^tsRF)hvi~B+hEHfGnv)7SwDN5W9p4;ZKoSVv@nq zOjuUX%?1EDFqm^VGL(tzqB(aP5YXrd6Un&)5GW7RqKc0*umZ-N4*^BQ#>G7L4Xec4W5Pzz=vj9hvwQqfQMw;0Fje&ia>x**-$bA zLP6>5O{H_Tl+E8-wqR@N(s$2=#O@CbYJc#9ci?g{84VJNkGHoMOfFt5%*} zqQ=M^iyv>IMJbIqhxx>ZlRUflN!qn()w#2$Gk#Zz#J63$HoJE1gxtzYib8vJ9{ECI z=tr05>@1zT?bzIc(pd$i)3%gO*^FKa^ebaGl%NdH!&hGo$u9ze56ZBh8=i{GB7xw2 z)TZ7_w8HVNvua}ibbE;giJEw=VAAQkCK{oAeEH0B%j1dDOB0Q4!!;X@Ua4)g%TrBV z6P&E%2y_kLr`p&tP8W0VLfMt7!gF^FKi?1g>U{eM&6@|#lwPeqdj78I=lh=@ySdqL zb&KX|+XxNl>#p(oLnkVq-FLD;cTIixQk%7f9~RwsVgJd_QQ9w#-(I-=m`Ai$>Vz@` z>Q@Ap;j>e`-uiRZ4`=Uq#ONdn0e5d2J@_&1V-0YX!-i)T64^bU=uDp+-19`>Z6-V5 zo89mZIcC3nW96MYj;8vXuMI^={mG#{6a3?B?3iEZgNE;NKlY%n&1wTm926^|&{q&V zX&Dk+x%k?&$B}^Z-8rC{i>X14m(lTCu)A%ftO zAE{W>*;#Ej^kiH$0fKK?2hnavPRIVnIRSfZ`s20QPn&LE6N>_u{)q-B)Bnbc{TvMs zs_sANouKoJ)Un(YlYG(SUxx|Zjb9^lBCw#0VqxxtNfcQh5khDS!OC!FGnw5Cvc)O6 zFihlxH7qyM$>lJCa(@?~BZMpBa&qJp)8A>8VB|2$jXasab(DFysBOS{F+_l`WNGMI z_{B3uu^xE45@G4yAZW&s3X9bXhQ{heWIQkf3p9eVaGyc2Lk z6cFGWQ3B5`D4DmlZ2q=m3%8dod*{^Tte+P5@+w%fvW{jlSFL)!b?cTgnFR37UW&I@ zr`|({XDu1+nDp3=FKJLqo4H*A1}4+$A_=MJvW_+(q|LjTqi z;y+lr|J;IYWguqfY%iO&wQSbLlIfdhN^?^&Pic~s=N@FZ2H)sQctwXD$^!V}BRur- z2GGuNdh8)+{XQX>E-eL}cRZ(hAEmI}yCmp)rdjHn?1|r9Y7?QsX-v}EVu}T?X)TBv z=$1N?tOfy29=Dmk;wO0~X3s2>N45#5?U`*xcLhLCF(H2YEOXCIrkW;TRps@kg8qVK z;13i0xgOV<#lW@2T`E;5jjP^E5$4821h3kkR2et-PU7-A>ZKSGm)(4^kEjeVZiTez z&h`2-&BM0CjTJHsE9Ezy3b^4KH~b;NsSn^P)fOJOm$UkI%&eO)_QS})+~0Rv{}qD2 zjRddxdRt}8tUF0mZPk)H$t!NYGFZ$~K1}eiNqETg;%^AI!zVlHYae{S4{G(NADi;U z$>AsE_M4aC9L`MFc>yjXbqpCI9a$4ZNm&%Lk3zKgN>o9Vqj3>I_#)Np+E2EL@HzzI z3Ailhd4pUR#s({(^kQ{FM5CHUmr7tdUN7$C_-16=J^Zz-9(wQM1T9mhg<99z1(eBqws6UI;6G}`T(?|-m2HJv?kX7Z$oZQHc+>f+he zt8*t$clY+KBh<;87YuOB|0ip3{9N?t;n_Y}dHE7QZ?}%^AZAyuPVL*Z>Di;(;lqa< zjSa=8&gRS?FmGVcoC4d@cTOzabqq?Jzx~+Ut);UIN@r4iCT~J_e(Xk`(zK1pu?@{Z zH+*20rC*k%Z>FVhs;N)1saLY8N0L~P5~MN!#N|r8oSMr=DXa-Jj-9P4Jnk4=!{Z&R z>!~(2+U*-OSK5Xb%29pendM1Eo+R{2K|=;p(V4>Afagwt%SzGL`cd#z&@d}Oc-CFj z25&X1WKX=z5?z4>ZA{l4>+|mhDD|dfDFN9b>gHuHw-s&+r7U(?a}C zStjyK%A}$tE9SZJTl#K19oTTj+E{U+@#hmwx31pb5ruEy za`g(oA^0-F+zKgl9!&&G<2wYtx4JlFlNSdrpOOL6?_&K5&KuWU2AgLI81SjFhr z31Jq?ou-68@m)e9mEemp*|~PH(iPw*bg09Cq}l+df@X1zL}IFjIaeuj(uU7k4Y)N+MUl}9wAACrVpe?ZAw(Da~A@{o%yQ0}}T z5!384Q|`jJKuQWJhsPhwAi$$Vx0FWfOmKOGmi!PsL!V6>?){qa5lh|(NqfGlg%uMe zT9%Z)$Xvn^kOH_Gy=i#Im|>PI0+~#e5pT&8&Ar`GC|bf642mvrG@Btuq(1cdM|hNy-jc4%YV&Y_n`W-4Nxd@_74fp zZo^Vg4XV_xUbXTGU7-r z3k$*G2m)fbmUty53<*bwT|sJesF9?+R^nlp`V%hM_2zAq$x%9&70$XI3YGCJWi(+r zQY&}SKTPYKrPKxzh51TGaC5(cahygul}OzHmJTt4*5W7U+({xDVrfJ}RpP7|6RgCgRCy){q91Zr>im_40JR}7$=F?O?4>aD zRT%oIjr~A_rzl}`L;+>*)aw_Z$pd0FnO%VHLde|7S}m=!Z~;=>c> zji{OXh$Hj~`@%;ZiyyrkZ-Bnz|>Mpx*+5!ze_q!V!s|5>JPZvKh79Vxq65u2i0& z?+{kFS#zbTzNvePscXE>4@oLG>r0(9uSa6p4kty(`B=Tv!scO|Gv@?R9BwGN(-mHs zgjTlDf)jFPc%oIw3j=iAVZa^e6{*#oyKB90PqL;^T(xlNuhYmPwO!JTz-4zLI6^$v z1W-m9?hO1ZD0CF&PSAVOSC4EHqugVIxmBiV%C@7pB*S{B;x`MZ{|v#&vH(I|o%-0d z*=?^){%2((xg~p*&Fjq{BKRHk(h6Q}O+Ewt|F(w~qRi{~;K%pSmU(j&`rtH5KnHwI zTfl4M)pONf7F?V0#D7L`#5dXNi#y{Q?FAkn=dVuu$L**YD4Yw{{%Q*MzwU!?Sz9sv z33TL}tL=Oq>LC~9)|(@*PWtDyIqj}a`PZ#=6JW?ub8rWZF#ZaH(>lwV!jJJ0jdyJw z)@x)+J(CTI4D1%41So)yz$^I}5rW#XNU@VYF)jNf1XFnw;W%0=yqCyk&N9tVG9$jU zau=*o>^oUZ6y|wIxQxkN5eO!Niwj#sTkqsMUFreOjllJqz_s{mpfe!Bp=P*}gonDg z>;@~u_nyqDc|n&!;9F6Qb7KWYQ5whUnnxm0MB!*H#FwB7(>ZTf9s1g|lXAc~To$bz z)N#kin5IR6I{pbiBeg8Rxu4_?07^l8Q$kvW0=@` zlC5P+b{&JU{L0Z6OppNZT%;<1{!kDZ$GNX4{&+@98=YStswa~{1l^N|x1 z|9NZnV>jovIyA@6yd*$7&qqJsPZt`fo8@_P_G8B0zQW0 zw5OZbqRebn!An7S)d$edE1o_~@b{Ac4Z)pjQKmmuvE-p@yOx&- z4gNMBa@2JEX@%_ilX6^QxC5VwgRbk^>^6VF+Y7L^ogbLLg*?OdSlR>A(6&YF!Ue+{ z%xIxX3F5dxQ-Y|z*#?PXe}ixj7h#wNx5F^-JtEwaXcu-h@RQt`gmFv>#RlvL^1?P3 z!DhGUQJiF#MTyAVkua}?)^-YTF3vx!xibh{FUa1)C5R+0Ar?jYD8VVaobp*%Ipiow zSp3F1MKJ}q1AS=eAq)h3(3Nf`JIgc~+3Pd_*Et<+~8 zRqvi$vC}qo?1a+E|8lHo(-hjqv7`;YZjgcKJ(F}rQ;8L^S$-tc|-Z>{7swo zOz}k}zVzY9ZS0VCd7Fy4kIop&4Z%8?f1s+0jo0xJsUmr*>xjUzSi%JEkkpP2V|KCC!70am_dX0Nip?NnR+Ig zda8`w6oz1hL7qrfcnpnuSSHiSmBz8fAsP)@PYU8UTH9M~tZTGy*In(n{;1UHf$+ru zz?H@8{VA(l)~ zxhwQt<8+XuU3ejULh{ZCZklUFHw-URm%a!N*L8mV$m{z~!fiP}x7d44VK83GI=%nfIY$SJloUON8Z(C%Na2Y0_6 zF}3066TAw$uB@=G=!g0}KEtldQpA>%ntqJ-BmqcAfPw8t&Ei>d%$na`Mun zO*by1)$?iI@1Nd8g1T`i9>SoY#X_UTZjMbXMEic%iJ;@THhez!!M~tRsow zGf@7!dWc1xp#JvSYS89?dhw4huY;^RjW0!bWxLT05SgGCDmzlF-V^HxoswBf?t*JS zPi!)>I6jMLRN|Z}>Q^hi1%F-I*||(Gz(roC%i$j;9Br5MG(?O4~)iCH=xYRU??pWq2O$;j7qtFRMlUJLN+0R{>giR8%et1$#xTOlQ z5JgP;f*vPlQ`jTXS~j@xFru5g0Up%Tf)f(*VOp*j8(7%l#Av5`)$ED0Z!YW3(NQM`*qu z^ZvUNQqv~CvVQ*DS5{AWRXuk7-l-`+&f8M<_7L^`=_4SSN_~P$6h{gZb8>!wAm?2E*I{?xwmr}v+HdjHww`_InXaeQjJW^6!uWAFd~AOJ~3K~%*1 z<6ljgwfdD+Yv-zy#_ZiS^vE~;%FW$xUzA3z>)h@64I}f*2E(W#-O@M9(l^cAClx0% zdnOvu3WuOH1Sa6J9Eno2uz;`3v-^cU`yU8_EG^TE%Rb=U6I-?(32QB{BC zUftz8b!TtZ9J^X=xOnf2W4CsiuBpDg`0TqUCu}b1mu8VfXgyv(;`Q1Q*_y&21h+Pj zlt@sRw(FY0pH5buxn3iqu_T;gP|&o1L>MOSdFX3k%zaRJ;DP3o2Z#4my`!v5 zn16kxTtwz)GT$6o^GPl({boL*nGQnrcI4Emy~>)yduz3yKKSYVst?j`hfi^?CiMLk z7DK&mJQG~~Nls;O95dTnU+S6eFM=hH#`883yp#+Cm3bW&)Z-V;C(r*|n%RM})3I872ZPn^q92srGk zJYxB;n*O2uU9&+gB+;OMwSI9pGL^&#NiU7jk~5B9CBVuUcFB>X!t0??2ALix zpCO?i$TtOf=rCpnOvjQ#66LZ=$Rr{^1ahWSS~hK@u~?Rj;;do7dq6XyeuNFI7-y*Ug<&}426|9!@K@`*$Lo72^dJNKs0@A8hQ7(hz8U6$Y34y0 z=D|6Zk?H1;4ds1a*_@Rz#L+Mf&c4GA%4Le z4;O4SJoDb!mp{I^`h!!iA2>HZ^^1ukhI~ExQG9YIw6$YVOUL4tt{0)$anGX-Gg~yx z`3HL9%`YsZXrV}X^Mpqi3<{bYwd1u<&cFD<$(KI4@XUwj7jCvpQGPmkP4?n9-(DNP za{KlL+8^d$zc9b4YN5kE&*7MEw}bW>{_gIezRxI!Z$2?1!!$U{0y4aRhN*A5saJ-n zXA1OKhVF@m?h1X75_B=BW}Q^QHWxOLuL1+K9t1dR-fB2&I`XNp^K%P3`qVC5{;_iJ zardY~km0Dc!;X4IYoXhF?){UA-(39m_^pcj^)(Gmjz9XdgQB!MK*%>X+5a#C|2=51 z-@IS{efh1FFE6~j`^4x(ll1Mv&aeOEyXI(Mm=>r#NMVE=Hhy>6^YtUX$p(AK|CwhTVn?s;v-6V1XC_AKr!-WKw3nEph?>~_D;_WBZ^ig_LJ z^&@lIRm^RVZ&R7o1_}FcL7eA|mcO_40hgF$b{kBKFJozaec``X`jg}@5n>wkmfyS0tA z4fZQ{>rxM2TKDnU4~p=A)oU-_ow%`hP`YKO{wjPubh>i>u4AkAo_OQkQ%PT5xOA(w zwxQ|9z53ta{Z>`|w`Dg|zrHv%!v^Z3)r&u*AH*lfM`haDrReb$R5=J=K(HJ8Cpm(} zu;H0$^2ju~XJf$aY+C*OJElV);70M`6Vi4{N6V#Sp4ls-@YLCJIM`P&6%O^EBseMV zKmPwd%V_^^BrO`{zYG7a8m@N=y51xBdkAu#f4JZE;=|>Bf1LK=@SCdszKH*AwZ-xN zyBhxKnM@k-+=`tIr|;7FOTpnk`O@RxfG^a@>oE{I5GlykFtO|p2}TI^BYLo-jo=1R zOzwv?5*ft=zkwDm>^u+!Z(g?UFTt7wTPQ={8Fp|s%iL&PiK%i_KV?|P%8Ek zm@tkaM`@AZari5CEKxLX5Q+Oy+S4&ZT8@APMQS-utUxq2E0->#27-b_H%On{89?xj z$IQ7tIC_ZU;t1dL=|`gkIg9{QhCn8`!q6+h&@0i{JIOdeZ5#+8)eLX}>J87ajn1%+ z%d|||bYj4(yI*=^{LRaA9FCWos$aI(yltGG00c7>DMnWwcao{qsX zt?kjtZNC`be&^_p@k2b8_3jc9GH!0v);B*o_wxIv=BPiKx;pNKwXbj6zP`koRC6ca zVc+C%^0GiRcq|kFFR_UZK_)8urI8=H(rs@zu&5~kPXL1Z?KKbvW`f%4gq{; z;1bEc>E_;PCOA?ccyO{QD8&c_4^W_UEyITr}ZE|hkHZt1ikwtPFgQ-)R(IZoA zBjU4}^|%Kjcg%GqH+oXEPC4es6E#~k=r7;*0M@PesbS3r&_I%e6K6q;uh&ST1kIU@L>`oha^8+~ehm|7mWK;ENqO1HS`C7D zorWUdCnF2*DI~0pmAE{NBr^r3ygzZ75^5k@$^?HE8L^gay3KKCgcq?^2@fZ{Kzy<~!jO$=O;TzhSSy>_>w;eChW3y0&oriMeOj_ux35c6vI znx*Sgp3OP9_S3UVl0WDs_ni>XX+luvX~CUmbc4_d!Ym3if;&wN>O3*1%K(YT#FrFn zK0W{ZrXQC?BN9ke5l7? zi7vK`j526dI0ao*bh`PBz#cwF@%R-^iY!*wG3;o{_vk7%+3iknB%`bvvj|<@m@{CkT}DvtfEhC zH-OZt&(8b3b@a#Mw_o0M{HZO+w*K@>P?&aVrY-B+OU0M&H#9m+B&Kh)U41-F=Sjfr zm$xMfs954sU%00`_CCl(=is7aGIw-F0 z=ToRm|B_YVe~15Dp(33FU}&>I?|nIu_|nTh0@?94k5l#x(XKQCcw$h{n>n zd%$uD(5})Y>~~YGaDaoUngi~K8iD|)wJ83H`XIHTyUNfFxRz+>lWgphZt9n68k}ex znqtPKgc%n6Z)!QVaT|&z=2>SIl!v4mMm!yso-yOrFE2S92~7`F_6Mns+B`?y7DwG~ zN8Nk&`j4CHzOXlZ=WrMtj*`ZDV_DIMx%tW$B2_b1$34}j)AmsvQ-`_74R((i82hj#_j-da!~?+;JAi_hEvoZ<7wjU7y)tahW+R=kSpLe zI2yY97=ycs0&M?qrm|CvCj85DiQirPy8PDLpPwC^iSMXdvb+4Ly~j0Y?{tbR{Iujo z%z?9MUtfG~TY2{|ZO)-fQ`4;P>aPK^BR@RzneE08$8WEB_te~+;-e=k$0u73mfkX+ zyW8cBBViw$9-m?!sWLIR0pq-}qIm;!4!4FR_6n%Jd$qpCz1H4nvD;zv zefjR4(&=+&g!gRu@dWoz$F(n>*5<~%R&{8Iw`y3_^6var=VrD(GO68xaUHh~Z#!{N zkK&TU4#y?C{fwi@+EjPcUjLn={wsU^0Y~F|l{a$3*Dc6NR=x1_%2%F$Mjg8@O}+HV z=f@0xC2#V^vYA_tP0F)P&MF$0g9HyrwTw)&3`sW+N;MBmHVsf4d!?Cr0>qPz!StHh z0F@yiK`)Qj$#Hs_)@wtXD`+F0)Hj_tJYNa8zQ`j=wVK&EBsp7Ns0_eq8en0|Te zR>x=!a2&R4>YKJ`u5<`Ln*80RK1#!j`%msJydr(`=BB z)UN*K?9kmMkEiRONY}HY-7yDMwi^j*(AOPFcel+nbYOi=0^Ie<8@NqhJu;eiVm?_S zAJ1IVV~LtwNAYE<)yA)B?(T1b|GyCUeJ@8ogcJA0k7eGk{D8BCkqjt}gxR`I!$6Zn z;2zjd;1Zxij)ha?NPOMI=ma4vfyKUJ z$RG~SS4cJ_JN)FW5K#7uX8ERL?xT()Qsd|Yi9lXT97hv*0;wSg#K%(pku-jfb;|Qd z=P-xI1d)Qy*lkAZXlO&86z?JWKr|_h7;qy}^h0rUafxO)yc1}nRDFs77XR> z_8s=R-S*o3j)qT&;D;TJN9+wcdsDH)aawQv;E8;d987mdxejmfo+%EtG)49T<%A`RX@+0;AD z)F%El?!D6$cj~TI)*kz%>X+NK zx9&GQsBfxkus1Y=FTXM6z4F!>`X^F!9!@g^@l6Eplx`fntr&DVKwR)T z=OKK%n4XCNt(a`*My`nFnQd&HVjQu@2M$$ktPz5zh4?3a=B~$5bRzxmG+NkkPjPI#CCf|6GdyTCol6#yGHZ zJ!o(g$be3bPdp$Mekg=u{9-iBW!UMhVE>w;MPZOe6tbxe$HoRXsBFJzt`lzf#Iv%) zR6cPU_N-wWvqgMsd=e1Z5SfdH{hY0o@`10iutLJIC=ud7fZ`;aDo#taCEq-Peu~6Y zum@|a(#jIFa-}X%p$}H-x-0eFlMTI9`hF@yza&Hd6ehT7ScYX}re#bP5Zp2$+d6TB zZOR5D_>3*ZGq)8l+*TI4@7zT7p$SjFos|CU>0=5<;~t0OV|)F__PQ?|^@r^ZM;s38 z#mnEv#=N|0px1XJ|Kqc;~IAX8g}PE`V;-{&MklEkm65yl1NzkZ2sAWz9N#se!EVrvG%k z;D7ShWVhEhHWgjGuloGli#yAE#p=dyE_%?|0dCU;gS0Y`ABmf8v&n0k+{+h}JR3;GRi)vcygd*bYv)Wu zD^=l#I+WoxB}e~)cg+4B{!a%K;BGaBgK)~eEj;G)^7lb+7y7R)ltmN*6J+6qQeiq{ z0I~rhku&HLBU{i}CZx#2Fs5$|ZgXTbyHV|85;Vcr0gg13C!5iQkQ;sy)&W*jR-p`b zuEVTk&YdEMNamaPgtY9oW8`p36-NF$Wece&Emb&-J|hbK*Wq#TXTaNqtl8l!{Mtgu z>KF}s%@A7_A=nBxHs0;1rh&Z9wxCK7F}2KVRMG?3l18Iuxndx6LQMfe3xw+T zDR*SqkllUH|9{@!@4RPy?wNDv&U4F*W1%6duC1B1xo+;3ea~*JUGnCk=ifX$ zDf6?L%ae1oZ`4$1?%n>Rp`q;l-Li%T%ZIy4UmDkI=io;g7CyFN_%n;Tc^#kGn&kFF z{!zX7R}as%+%n?nmBZbx&wXs^fR2T$vabJTZ)m8#f8W?}@9X>bKW}LG)MieN4tw#S zT{>Xb+seLNm7kjuy*yz0YxCbaG6M--J9%x*#8uVfORGkgR1Gh!99mL2L{~8g@SR`L zSA*X>?WqBRTf3@lU2?5mvMeE)mf%cFKn6}i;0HmsFNM6#9a8be_06()WXLnN?ECTR zE!VJ#2YzkPKivHav;)N;F0yJsPhnqj@th1XPwd1AU} zkqs=p3ap+*g81zUtxxG}{a2f=T*BovS5~d~Y37p;Uk>=+^uHl3!6hh{b+5pOb@TGC z(=zlv;Zo-)2?0lJVPYZ>X;zcqYa$)VD<+ucgjC=R2gKt|zElWsRJc!|Ob{*s3}BZg z9*MTp1x2@!yuon_ni;-`cw{aElc0~i)2o2OF}A0%1rm{iV{wZ}j2F}nj2DM+n}`JG zVer6OuwDkp(xCC##tsq@ps2f%#TH)s!E>^HDI$u3fKKQ5i7Xa}q?(;!7RH3PLSzO9 z6P$#$(BL$&Aael3CynFg^tzEOE(D)5pQrd>=n%gL5x8X@4?5Un@&};JL3L(XLNl$Q zDr;voXmEQEwY^V{t*^?4s|mFggLICe1&$Fq2M~O8Y1PEystK!WCatfXwyt*O+S=J0 z>*j5&TezX_xlQ|?+qUne_m3=j^ZV(VFK32l#izfa-<^B^_U@JIlb81OIyLRD4NF?? z9Q(9SXy^CFJlXKV6a0U99-8>On!n6x^|ziLr{}i*ect1`flvP9xk+2!(|&*K!^>Ab zvsbN(OkNxk+UbS)v#L$s9da1I`{*5wDr4ES;p6<>hsST7x%K-gYpW-%sTsefW*iW_ zxN<~k<&aWb+YSWptE=b(Nn`H;=c~eR^mI|#I%ivhaxkV80KP3#S;IqfDc*_lzZgj`iq(`PW)|S{UmMW zfHZ4JjHyGI!E>qJYnk3_x!!99BtzTf2Fx$ldoI&^EYr6SGr(QmFWovpTlwO~`rKW| zzpA@%^wQ0n$~S|Y`2OwAz0!(vzbWM~lJ^f>=(V}}u{^U^F#^}K#2&Q1(yiF`OtH;& z?%LvAhac0Ly_n!76dXr0+=ABTSc*H9P#9gId@*tyQ-Q5*zO8++{ZXwY<%>#sDZdV{ z=eRa}$$w)h|N9dDdYAj9_z8Ee;;c;JdyaNX4P9d7kZ4m#l!;lvK?%m3FA<#K4hZ77 zv7me^BS@%PE({Q&m?JQVS_(kZvXEG6JYIJC2|mOEnWEbi$w(G*o1*XtXkxLfL}_Ft z6fRx`C@Ml|CY8+*63oQ8Bsvrvtz_thqXpBJfNN$t^U(d0nadVuIEA6fGm{j}F!NGB zK3fiL(>g+#IgY%h01Iq+D$=?5fg!>H5GF7C0Kz)YOrjUzljO#+N67F9Q$*z5CtbwN zIdo0~quPc$> z-}~llte?O6!2In87j6H3$+kl;>^S=3dq@7hAhO^esYwxYA=nW^Uv2|D5x@v488e6am2#!BK<&WRrA*V~00nuF;4vl& zzLPP>%H!o0Jp0FJCWUL}bs#>3R7`1RF)f)Y5D(Dg62BzZMJ6bpJLHJvE4;#iCoMTV z0+7Yovs71+PRLdkocIv#>AKK#D1LQQI0pPAQbaEldl72gCqpb4C&wU`xcFt@dhmd3 z(BPKNxE|cr4S(6n);%8y-WMRAPXy0*j3k1ODy;%N3rw82u4b&xHZr|@Y{s6^s_(|B zzMh!<#iX3iC#!Z%%l>pm=EpO#b}rO?IW_&=1-<;P&V3C31<#@<@&-Ti?&sSRmn}Lm z=}!&MxA~(GN}~*4*y{FzmNPqhem$-Y(EQ$_)(wkW-kJaS)me}IKEGwdORcLX{5d!3 zwMx_O{nou}i#01=eMvj;>Eymog-(2Fm|7mCwN6}9J+ZWUd@*S7s^KM7!wM@0>l_1g z75yj<-Y3`A3)~fG@LXG{+7_B^4au+s!^O$L^_{%*(>q0u`Rr&=wsroddd=P+o=h_N zW$~9C^VS*}x9VrM7DCg-)nu?>hb6Qvex1UhWw2XuonfN!Vb=g&b z^CCAzVv8ymAyDE0D3g$EnTCXkU`OG|fn}h$MT#IdDZZCZE=5ugp(! z{6v`x#Y1A&F|Bdt(ElsFMo;&4m5>@AK_n>_sxPde| zDAOF0Y3Z77>7HR>f&-Mni+oRrjCvM}G_~HFR>!v*3@I3w&%GDWB4R+hLEQolQ9HMuZN*=~r$ep281a}+l~!loa1h`nl>>_${R=Ak=2!H| zv-eTkdcpbTSb^Z3;YB1HXEMQ``DNiZp5S5zq{v~8^4&?) zHaXuYl@cjXx$^=^N5JRaKG%wJxCkR20L{nM=2QcIfxi6Y^#t8b9L;V<8FnhvN3E@WE1g|LYvUw zsJN{tz>6C-ICC7s;srJ@ortmEx)hzQ*Gl8bGq@Dz*7=k8%c%TC&Hv7A{tqVpw{5eZ zx)e9=)@k(I=g*v*l(N@*g{gClV7GvHB;J7p4S@(;*hVvlz9}*bM}I|b<78uk5EdjQ z1p}1{NwIE(v5qm}WF^dvvy>*J7EMBms7tI8u!f^?I7^8EZhkJ}#UcMhN)#ezHSHQ# z;kv@jps2^);Alk^#qoOp&7;QRfLq{HAxN+klXp4{4x5H8>_Q=2f=-#Zm;`Z5VR4a3 z6NIIViQtPBeN!31P!U^zL+l=~meM9Ro`mJzw73Q5OJSR46026y6q=`SXho(LX#OFE z#s>LRY=_5#yJ^%8Z>hTgIEzdHM3Wj^e=BB+d zrN3j$pBk26?BzcHKemOBuN&#PY5nRtdAI(H57JW;(-w@233_bFvpug~n7jM)L395; zV|dbs6V}#^DRGP`a11YU3@&yID0cMExA)Q6du!24^;BECxBlXYZ0qEFhfk!TbAriZX?f0P$B5t}_c;Ut!R^R-~DeoWrrDeXkZISJnd`s^QRU^08>a1tZUc0sY zTO8y^-(Og~>(Ez6FLqw%Xj5qMDzbV|axaFh#hs|_;VguFp_M3ab;CSPX3|ur6^BGN zZpBuwVr$C+TmKTnp_6CPk^FY`O6gMk@4)nHg8$|C|L2K*%jMa&fbYw?SMFTJRi|f8 zo}H4mw?mlJIa&&il0qm!COA?GiIgxfl7gu5Cp6(g5n2}EN681*Wgv@yIXLB(0H~D$ z5+v9%px};W3P^5! z1k-NEEJU+E?fNQ!_N9bC=elWlZjco6l5r}vmPSBN7S(A0vQG-NklJ_%2IGy+1ICSw z;Aj)s!D8c`GGwk!7lXq(%AZFto?xFI{RSI;M$6j zx{A?-4$!j`iyTu|)=ete*YDNU^Zx$q#~&>C^~b1&22I2L;`=vO-o3Hz?u~8mil3JH z!tN1&ZCKp;&Vp7AFSId@{8LUwSnUx*@@W5tXCI>_#dWaOqiM zeWt+DQfubm$(EWQpDZx9&NsX3EQ7YxU1Vh^;Z{N@N1s77FGfIPrU(<{J#`B1 zVMqlQPaS#}_d;vCLQ5;HwR5q-v>*J$J@92Wj2~T$9D!ndL3{C&QvdU4`~Q35U*KBU z9xnUE@S!)#-@A+JP48FjJ24`0PscDL;ycoaDT-30(YcE_PNEw=1Ba9L0?RVJ#+OloE&f-+gK z6;7Bvn16zJ3qUC2slr8(`NYs55fRL%3R@s7)fG(lB`1MykVx&5Vsd&8U-Fq;m+i5O z6q#Lu4>29=eTd+Ay~tre`+kX{BDw(>5`183a!{%qnkt86n!9A0d!$=>WLbb~J@LEs zHXwNaJp16his5Sea9ss(a7=OK#KOvnC6$xbR?pgUXndi4;Ih>Vmj83@hL;Z3#N7F< z2!Gk?{*F7hcSihe+|fzOjOFufj*kCR;)=y(_Aj$XbO8B$Z{dGJ@KzwYw+#Gi={gO( z2%b2#Cn53g;UoOF4{cpJ?(uJi{3+q}CB|=eRT{p2Z_C=e%#>*}ri@!Ob8L9wh%Cd* zO^2qeuAW%z7*|?3s-$v6apf@3-~|;F1h@Cm+Is5j;HbK*t>Ce`W?MqDEy2L?6gecx z6cQ(eMjL%r=x6CFpIcSy9bo`ccMQ|NEuW)!_bm|PuW#MoZdo~C8pwX>`fb&&wiG0JGT;2DMh;$6@%qyDhlO{qU!nK1-#_@KCN!+f zH%tn3MK-4hV+dtJB8*5?7T56#G!hh5j{G$-b6JcML#Z%cWLOLP5@eht^(R;(I;mNI zVja^QH469)W)b=WYJnPItYeB5W&w&qcY;1JMyMPHJ-kc=MNyCOg8SkHH`>KFP|Gb) z&!)-(yF}GaX|gxRJrYgctiCDv)3_O_$bF0=neYsgBn8<5&4AI48KkBO(8$&#OkB$^ z0rNh|yd^WL<%8G4#KvHQz0*j9DrdMWP69uhIPBLu1r0jE%R!n^}c%1OmllUG(xSyeS_UG2Or zho+WPj*8teb$Mz``fEEs2s?dPd+Yc2Vwa3KH5my`O9NZqp8ZJj%)aYB*sL4p+pyrV z|D53420XfIeO^PuiMzKe@7=RIYTjJ4K1P+iX!?k5Th=iL^KnU0w|bSJ{zm*e)?_qiC-_?xKsPnb;A#rInn|skNWO-mz5Q6`4%8G5ZptD z6vhVgEzlYe9Flv1mGhojQrQJ0%nK}TMWUP`?x7>`UVvkOJjPgbw#PK4X87>--Fj$k{Z!U|c{ZRPelxSkF;Z7CM&}q;SUCyO%Ic}B zs;8}~p1G!a&W5^$TfU#S>A+N-X=42LS<4caC%iUiRFBisA8&Y$!`Ao>olnL-HEr=C z_2iI-Xa9QtKPLE^{*SNUR@~5V>h9gjh6cy|hKh#!-(9(|?bUxyesR&lPv2Z?{$^)! zc5=w1B}2nXXKk#Vv9@a3s%mgB6INA^U0F4@xN=k>hERqSI0oh8ca-{S?Y*?NJ~@`& znU>yZ=H5x>UI}uKI1>?E>I_~z#uyZZDKOI5ZfQ9nxc&@I1O9&J-VYaU-nw%?cE_Q^ z=YRY8>aD`BPcGYj!0olYUE)nbP($5lV-V$oa1Ild_}~ny^gTu-Kdt;}aHbW+ctDgf zM5qZggDP-qw$)hNC`RE$YWJ^sa=TJnuK6@BB)I(hgm?Bms*(VdUQAo0C-Jucao?fX z_7{~j;q84|^J)3?)!*;lf9dnX=dRuQ@%rss^zF^#mu~+3v%^oRWjD1Mqu4}VW;c>r z$$MaI1bL0+B(VwKjA?PmgX1SfiYhM@hoHn0-Wc*M?i%Ai-?P>nI&trwvU>gI^_x|D zuk0+i{Eq7K*0f(Y#QnNH_J267qvXsxOV%|t_@neM4E}B1|MRpV4ld*6w^di(&%fdL z?*7ed;tL4(?j1OAGG?7QWX0YN%M77WQiv<5BZTO7%5Y}}NzE_o2`s}UZ=w?hC1-!7__Dc|XcjdC#@~sZlm$ZQZkrOgA4d3qt5{%cT$uuY8Io;eP)6yf&(oJRQkqeBo_RX>N1?1_7;5x_f z0>>C_#n{5i@x@h>N~$KUuAaQUX4<-%*=uX&uB(~1sdfSW>)|7dw;q~RWa&R~LH*>% zIEupG(78STiRoS2zRM^c6{Alm8wy zHDTlEsP)4aMNE8Q#o*_nrxqFKzj+MY$@DcfQ`gi?DXE4Rg)u9u#uQYJETABGfn%Vy z0>5aQZ5xno>7Qchmt^jnDECe@_e_wx$IGC&L4Xs%Nk>Pcpav-&8fzM!Wy}2N=>9X; zf4_~3uy@*i+*S26g-5QMk6ix8#(Ix8%0l6egvTSs1Qq=wjX{bS4~aGTgd2S$jcFer zy?OiI)f;!TU!3R?XKK5w9125dWea@bh|3~t+d`Y0&hl8U=}E2ml`oE*yn^4kjJBI^@5ra@b3%73_GIEio2mNQq2Y-fMF zeHX@i^wP~d`SkEDwcf>ck9@0_R_J%+Zvh5UoU+fTLoLnPAg?RmclgAq+xH&+FN*&A_kZQ@&fR-I96R;amvt{{jlk}9 zZ|H*~q)h?L*NIfqT;L0?JSe zMX^xa0dYth1}58Zxj$tXL7;1$YFz|RVo7et3p1oP32`el>6J-#}qo8UdO{Q4tis62ETvVY#Uj1zS3$ z3MYodPeQPw56G0+`5?GcusgI&$Bul&0I=LURrKgf4g>7bD<+Cw&>yg(hfK2Yt4LX7;+;IUDQdZ>(LgsczB6>c78#a_DR7og-T|EN;!;z2q;- ze=_E&xm`T({qeUkn-l!`Co%@PFI>7jAtz@+MB0?d!pUjx%`TOfynW=Q9mijK_t>Is zhh}f6UAXzs{0;T9S5?gbe6IxoUNvzQE?pl30=x)Up$ymBhpFsCGp$3?EQ3=m15?cX zlg&Wz-br$=M7d{zse7EMYmC%61|#4cLg7eAK!mYhvL)=zgQeg8@Tudc181*axOV%S z+6!mtS5R);y7y7VkC(3BR_{8#eEY#c>DIPO%Yn}QQ!Uq=mrWb$ZMU|E6+tsMV>Dd(%&eB;Ri`DBUtR4F72 zY=5Bbsg?2%kV@qrN-RH=SWlJ8r%EiRFau^#&c?zL%f%rJnw6RrGq$9rR!KiN4TOya zn_av2#Jb%l*6pFR`(f$3#zX7B6I)CVHQwE%T$t^_?fRw_T}v8`d6&}qf04d*$$ZOI zj<5da_{uMj6~14cwb3w7W#|^MxBY8-+b=Z+MHofcB)mDMUA6|KwrgRykx+w#ae^D6 znLzLBaUv=L+(5h}2l3|9$Y+K`U+VWc-8Onym{e~J_UFF{#y za0U=u?v`%ukuLYhw)9Y2d*)etYi<42)&W}EkUaY^t$n1fTfNxB^UUnl4U7K0 z497^zqQ~DD&~a4!_9dGuUw&)?GX5l96asKM+IcutC zuB(~8rh3ZS>WOQr$E~g!TjCg}s~D@P7^AX{&a#cjv<^+T0Ko^QS^B4%`=!Wzl4K-! zf~iNm)ICnZ5K4>`LSc|Ve)FPPd;bW7&kFrGb;a7fr}m$1{J#pORMz<@!Ol%ZrojC zJ3H~ceL%g|T6xHp>h$uHKU_xNQ2W#Mmp?!Bl+OG_p6pR%@hrBw6_}sQm%R!xp;;v$*K^VcT7yYJ|w8$Vu0h~K9K1m9Zo1C4 z=nM)io}lYBX5z94kGK_)lgKBO7b!6lz_Aw0%x;B@c##nOItxcp+zS=TLv^hgAP$Xt z7FybAEnc~nHX6$#YRg}8On=ET{zWA{k}W-|q`%~1dNkYkXqNHOOz9Cgs-k?3^k|m! z*G$Yml0#-#Vtg#!h=q{nLKW4`#x~*5Bf`e;c&Y))!Qc_@4jV_ar=rR|Uf=7nl+qhY zX6at{rHy$}{)R}NZ#3qGrHg^*>y+M5((8@(jU}(UWd3z}t$Ff-^iNKEFuk^iO7?jE zbFas%dp%w&_gre|w89V=VF--^b{m2tL<)jDC){O@8=9wZN)d*TaD#um5&4rUA|Zb9 zMuja##-)EO0N2RUm{Ega*mx?WV@Q9=!@k{8~T(9N_?b1?~~_0nlKYDJWeIPM1S6XcN?YuQLv)9y2Usp4AeKk0hiKSJO^Bog4jtRN; zaXI$U+4hmywqcpJAsLpzK=4#c{}gk-WOHAq0FO6yjWczLl|o~TA%yS1=*CcZ5Ga~x z$vaH%MbniD?;kQ8y!4^%$5%H(-~x6403ZNKL_t*74@k49cO5JJ?uTEl-6n|tdhYVA zx8-v)3afTl&Z#~-)-epxKOo)WxwOnb$_O0q94B>JQQj^>KWR(t_WGZ%-^TDl^^e#0 zocNWo`|Z(73qCpcRKEE!waE+D>RH|MEsttU9Sg0)w$`*%%P)NV{f7sC8UJ=&wsh+B z)mx@hSFYc}pGaBbIF}=x8nm@~%v*KaYJU3e*sroB>oqR*q< z0>ZRT9O4o;vWMP1HHc#_;mC;5c->n0HNN5b9DAN+~4v0W%kSG>aCN@TwQe zyUxUgTpoj~M=(W8VoRxWl(93~P=@lji=piY(EI?SMgGAmX9&p=3dDEKfG^ zAe86_MN!aF>XjmI9dI8Ro8Oa`$v|SCs{1c<(GrpKMD%m9@XxHXz?VL~S3cvkxz@k1BAC zEyRU-<3RJ4R83o1HGOsUj19H30j40sH`FfJQ1jy3M@B|(`e*lN&dh54>%2Bs=e4>r zul3;>ZN>z9l@4?Ne#VnO&2DvX!NY%G77Fh?+w$VPHhadl8Zu<~%UkPS-3Iuse{uW% zC0pwkZ`t?!=DI~2>z>_QH+OU0oQ-wx1~6+?^~{p08HJTI@++t2Ii{#9CaNpOh`;`khQKJptfHzl zyMHL#e-U)|{xjEpzkTmI{lwL$l|TJ`BmU+p?0!@C%eh}~?K^#~X!`WHc4J6$%i0P>Qgbh&E;yC8Dj5s4jU)HSH@D5j5~vPG zxml-+a-+e3L12t<(##Iws-qv0f|Q#9Qh>8Os>xBjJtCESvlRKTe5A6Cpl!26hj@-r zDRfoPqL!kvVYex!!yiZwLfkclHL+x+%wZE><<*~|{$fTE ziEm^&fkqB!MrtR%IV>J6crA9x5}gF`n%w~F&PI0A=ZHjA91Qjt)7XOR=Lnx#3lzNg z{hh{W37@JGU*?R&=S;~YC``)TS&QO0bei1k_iD$1gDic0>RDl_B zM^ShH=v0yw2at?0BlI^S5HGQx^i5C{dJ-;x4+15ghF|Ro&NO#Tk-MjuyJwoa=itKd zzFC%jS=Iq6>%d&wU?8~GK1^3JD!*c^&M~&Qa(rnO{vRCB;H#=Z0?z^&zP5JW27q|o z!cBFHw|_rqLFA&&&#W2TddJA8-W$;-thZaIz>opHp6dp++&|;#bF{|rtd(*!8o9gDRtDU#H zW?pgi+=8lEy2_c_%4yolsT#*5bp;FXG1>M}nYQ5>)}d)u!1o{^c(S>FlDS`^+&e+; zl_2+wLjm43M(PqRg+#eLIS+ln@EMajE!TI6k^Zr<{>z$Q&RqQc68(y-<;dl{FOI*q zdH<&JGq>oM0?mgnAO9H_N1yolM*Zn)Km2^-o0?xTzdSzf&AK+UD6n0D^%;!}8u=MD z?*2t3wa$}4leR0ew#<|JZ>oy??%0lfKOebx1NeUMm)~yRyAPXhj$GnE&Zh@1y!g@g z@w<f%2g6z?L^Pw;OQQf+ z5!V<+(YjP&`07LgGj3cgN6A-KvnX|$%U&8W26vafoNHi2T}AjqA+A!-69$lgxzGZ@ z8ZulWPDiVF5jdnaixo8(y}>H&a%48Zpu8Gi0(cBi2+w?wOR&`i%2H#7^G@bQDmIE! z{}(CPX|GrUc>^D~oj4NBM}w4Tbq+kj}wyI`T4@<*usb66+THK810+b7}pk|l@v;dX(Jy} zu*+me@lL8)h;eo%L~S9ZeUpqN+l6-K05`uuD8U)$q^Qz*lO$RgE{A5wUDM6ofZ*BY9yyjiK=2GpzZ{g|gEY1w8r$$Z z`v{DNSBwD}UQ#(>W!0pWRZ~}0%~)AIV=cu}*4E5lS37@W?ZVCb7HzB@^;+K0VWTJX z3Le`lbkuV(<6nH``R7N>nLhQ!fJeU?-RkU|Coj)yb8UX>8}nO1x<0?vwfSvM&T6x= z&!dxj1kIm6V#Z6;UVQWW7q{0hezX3$?e(C(pW9ToXnpO%HMI*%s}~ei&o8W+TUa@} zpmJuuW4hJ>0(>&#JJ&uo+crAGN&?(6B-K19#XK;@JRnI10p2Ikgc0yWsYjgD4e%X< zFQg$+h9F207(K*{cnC#EF^~#j8yt;=?U$B$y|H(!%C^OD#(el!Q01D>j~_b!+l^Zw z$&ui7r>?Eoe&Fa&zd_#k{iU=oj$OGeevP)ka{BQc>8Tv)NsY-(YxdHaJ#}XHeDjle zavQDbFWJU_d~xXayW*$Di_E9jRp6J2fYIS!9r=iU2<6t@`#|)wS8f%`r!x#E%THbg z)=zq;u2rt-uiz4NW-pY<%y=C5pI?f#^ROM+^GsT;)&yz5#JiQeYV^o(?r1> zYz@ji^JHw3Mhi#Ll1D`iw#}9K{eaJ?t(t042|O3@5i8HO!6hlX&CBry2x?l=SRUDY zmy(gIh+QQx-7K#Jsg>P_hBy>~97O34)%3$s2&I2f?rIQ1dfatJkr5F<0TU-W0$35p zi$DiSZv*Pm|h>e0aQh^=%v%yM#cl$Hn-h$Tbu?2@iy?BV?oBZx zV5=~ousFSv7&*Q{R}hEASQqhk3cRp^(@FTG2+;Eunia!HVyA*_^0WYjtBmZ5e3F!4 zKV*`T;Gs#T;0(EQhTJ8~+%3!8Gt1mN69XxIf#52Ph7Z)*hHC6X3+%&-Dn=Jnj45`E zU0FG)xN>qy)%4QJ87r%1t*V~0x_aK~>Sx!~JiESj!RCF>zjY0Z_6aKc z*c{vFEbGWj>+lTA&@@5t0V#6-WVvsmsdoYqJVELn3k2uzB7s|Z%v_#Eq;}Ki6L%T+ zAu-0$Ikxca2R^d@2<*Ol4?`$`*X_o$&z4m0u$=pa7KdEEaVL5A@y`xjp#O%9|AFE4 z&kv8@QqymJRoAr@t<|Q-a*QFX?0q&=j@VZ7><0%xaWm`9r>?|&d#wIvT$*y__d8!6 zzPRv%16gG!0pHhd;bOsi_wIjp^it$kM~82z?tpP>^aoBZBj7Z$E4~4Io?6+1SxMzg zSSMb=0BZ3r0}<`>8|lx5P9nq6wki{~gB95~kQNpmf+#HWketpyz*?&e!08XP3d+T3 zsRfPPPUQ+{fp3O2*Z{BFWt#}%m^=ybP{>IPhP_dq+#$;p=z?7!jtjVrjfbL@LyAEV z<()TxxG^G|-x$sB9|gXR{{-J0Bmpjmxa@LjQpoJ)!Y~}a zp8{4tB$(ispluAISx`HMcp7csSaXna#h@f8iadeh2o;wg@&R-Wg+GAZD1%@wU11SU ziB{qy^y(wrWTa?{DvHk{07qir@%YCoSdVlb5?w>Sg>O8N)aac=%uU8ToEVDSWWss^ zmhjm!#~GU5DMp_}x)xNOSq#+ytQgl%Fp4eC(u%tNaR#46>Vg{gqoa`brNsTROTcES zq+{JpF(~oMDJv@}AYL_db@i;Z)pOQW z&s|?LZ)5GVn`#$rs$0Bu|B`L{=5IWf^XZyn2cJBD;VVGW52sHTuix~-i%VvX8M0t# zkNLxT&YUpn)n&^*`uxjVx9;7z{mYN1LN|XKHgEN*m)2D-Syl7Gs+z^6)z6hwFDfB? z7gWwGsGO~x@#sJ99l^NDpMJvj&1vVplPi3gw0_^(3GY=nSl)e$rxTTUv%0#;$-sPhY zjBLC~0w26F9w-cQO%aMg#KTfJZiB-yH7gf|;f;BxO>p?bQkYAoS^A@LPAi~I1A_~Y z=Z|XFfQU;0j)Ni=`WMwVlR6R$!3GkRL}3Q5UQm>tu#US7Uj}y|88C493PTsL%M=cX zrNT(%5VgsoTNp$uCCW#JgCr4*%tYuZZK5C52e^KSQABWWUMrH!ikNF6TYZ?A__~2T z`7(b=|8^q9jVBXl=mf$!K`3r?Ys6pQ1a@%-#5(gg!6*VLM0`Ycyyyi|CoER~?WKYKNW1G~JKNzP-!HH5(x(OkkZ4OnLL)GT4x#k|Z zmR>+`P~vJ!Kdp5@KE_gpX>B9&?IR0udHA?u$Al6GD)G|FsjDhymR8N6wSBYK)y&yg zJ0Fxdq>Z)DZLFEQ@lfK2AN_LP)3)!m-){awD)au`d-u-&{B!-G!v~KXz4R-81suV5 z@c5da&$zGoQa7*g*h{M`7q6^-eq{}?dm*UrB2?dV3E#7{j_H7J5a8;HNx7)L$K}|@ zWLrmNSw>`7hJpG9e5c8zzD@m-0N)_MrCzuMMFQq_jWKivf=3%dT+mF<9*ZpH`HXOR zEF+p&LJ9%>9&PZ2=PJt3H(4I5vW35OVBMZmpVa?iJ$rTkFTWk8pD00m|9&T5KGj++ zwaS*-WJ@hF^-DfJSZX_yZTum*?8Kk53@vk|RvKJU_xhKI&tJX8OBYXF`Mv(<->T1F z+jZ!|I>*_tZ;p=KToY1iYnN|+Qey&f+CC2yv0%3+Wf;3o6f$8+3k!&QxKf)4>rR$V z1IRcFnfTneQ{FnUC^$=S$6C6^Hl0*uf54n&$cl=DZ&2}`E=+fhmgK)Osyu?S(h*l@ zODN~J2QKu=HF@PK#8;V|_Dp=pz&($GP5|1V=x)8(6&!aVc(eFPq@gL(4S|slXL>VU z2fCs;vJu=tkxe+>Bp>dw`#=hI*#^1LIVk)OkRASSARe+pPwE(T9%2n>mY|@63I!`; zj0|7#a!+`LlMNtxviQY{cnC4YvYkaYdWCo+qn2$DK_ujb^N7Jiie`5@38B-mVzg0o zM_U%AoKSklD4>rLE`fJjbV@UJNS8XL7(1rHR$`3|sS{xuTJcRVbV?RP_DMAOLP@-V zl`m#ciNhO;!JD8-mjm;$D?p4#Tbwx1*J%V8K#g-IW$x5njv@1mV0EId*-4N?*klH zTl;IQgYv9H)z+a}+puE$D4l(DAu90+MUIIpD<`h50*FstQ#AvW_?qh3>rjl(TURr0 zL(PIsHFGxY|Hn4{`Llj|9aHL$uLV-y!+!(W+W0Ngw{ zRR#gxKgHA!1bC9EHwo|_2~xLsscWne1$c}Q;7n?v&S)DDiK*E$8Q7HdY$GIjq~15Y z+-G^&(@Xbs$TmH@bN_2!98M`ap^{D|>^=Us&%W=mwxX3vLKx-CPisv5H&$xpQ$Xz& z8Tz>0$L7CZ-zrCHiz~9tP#92bd+wtHvAd7u7*1xCpIG|E;TJzX7+7R&k*;r}lAczZ zo?%1+>C_4nSqnRfoTo@>re~huveU-Xg_!PI!Ca>>7sgb}tw6|a4-F=6K~P&!;;wq! z)7hGfd+?%hy6F^=p649DNoZ=d3u-l@#Hq(U=@fV*Y~Va=50`W2HXB>WRbCMk57#kW z;^#krr8j?ws3Iw^Yy+Yi?*Ys(HK;tmX%N};PM|6b>p)jzyWTd=o5z`NhWPM@;oD`) z2o7?laAh0uK)GV+is_)RMzbavG7g9|FsV4PzT`Lo=414PIMk^GXyhJ>U%Y^N{}@3e zgdRwBx@g#1oFFNb`0>hd$Pmf_V%QoBS*1E5Uhl*-H^P3xpGGAyMkHGZJ{ z*(u5$qbRIx0d@dTR<+DijFS+EeHBXj5QqVz-jv57Dt?AQkNTHK?5$+@wS{?&+XVI# znDLDhHol3C5a;Lz6r$vh7wrg+lTOEX0_vCFWqcH0M-GtuN<3RC15T6tl8pgL#z260 zniQO2>YQQfnl5+El6z#xJyhmiK=2%M-&{+-T(Cs_5G@A8M{4b(3MxhyRe%y7 zx3Y5l>Z-{rE2jX*S65A6TRm%a)$Fy^v)9(lUtKeAJrI1~qD_^Dj`uQD&-mQ>?BO#z znA-R8k3+;r^9#RL=_?kO*=C+R5xQ*CH_sOBTU1g7(i`wSuL!009K!c3ZN-c{!nei_ z>U(0YeVob)_#TsG9hqUF@V9vg;5$_w!0{6l;1UY(c&P^naKLx8p>s3{aD8xO6LHjt z@@CX_eS>C8*D6t>1^A@e(iC6oTvN*&<6pCkk7O7g%{DxiZF~wOGe>E)=D%hca`Y#s zyi@mt%J@v4sq-p(srAe=YLh1hh-KJ5qmduSuc#Ov$uvBgVR$0P2=w>NmD^GD8IgpP zG`l-Fu0#o#Wdd6i^~U%zP5O~N#f25k?xonX))p;jm54USG@yk%vuI6;V0dcGUbs+1 z@RX-3*}j=OA!$qt!*i96lxAVuF2^MNL9VH7t_g>&76z!zU1A{OG&SNqfX_(OCC$#b z7ZP60iQ3dCj*1#-mu*s_DEfznN?hN)gk?mNJE7TK;f$^{Pk&S%A+QT97H@bDDC-C1 zc5qYO-8?+<|0-be(4<6Q5E?NeaXbVdCiEf^ObIgxkD2*ZY)J17{iga!rrOCLoBsZ}{A|skRo8Bub4E^X-M)7G%&y(m#a~E^ z3>CAh_YZq%m1%xa&4QvzT60o0hnAhp%6H7jcT5L-t1G7Dir~rk9NU;2+h`Er85I6b zHxErwW-n>|Nna5^k$S`#F#;ZI0D_0&oFu(9M>GL9-6961T*SjbCGEQrLN@be;&c5Jc`_rg@%^h{-IXmjx0bz0G4NDMk z`R4W-GZaFCcG}XEqm#%aZwbnhAwALEVrQfKh~oU~V*q z6Pk$Ycri2Q7bj>7#>h01Ik|!SqWuOJQ@r-fLVtzM?7kU{o&>CudhHL_6&_mFv@b`PY79y=1}93vK=4$lTM9s2?v`chkuCSkG51oL`(&H@=a~D0 z5?5OX=OM&Fi4QBTz;H?tu1*YJ7iCd&hoD1O5Dj+c7IN!{a&-C~Sg zqY&RXhv}-g`AG|Z;8A=qHGP=j)`#m1ByuC_DLie2kyA2@SAJZtEw@%lZX`u95TTJ< z=NO?J@r3p!*T{eqh#nlZ5W#LiHvvlO%bmF91?rWt<&?eZ#vlXZQnZ68q+!!i4G-ZV zn1)J6%1Hq|-Kh(A#fBp)40Q4h9FQ?{7aoeBb`BoSt8r9xF7D2=a}HRz2e~cC^V~+a z zY2NhXet_B!-WAYhV7PMsxCByM4T8d5^3CwwGyu{VA$e%}7l&3T1Ybe(le|@So#oD> zK`WXZ3Jf=})bkU7Mo_TeWtzkFVIA$S=xe`N7s_*>gF+cxi{l;)R@!pJiK<5${G&uT z8_6Cc`0I@1I6nygC`G473HN|?mG+r|0ZQXSq{j%C0QnZ)i3mQQL{QLBs`ri4!?sg0 z;1bYm=$K;YkOqh}_{8hM26my)Cqdsa6=;t3elhw^T!V;4%x3DknX9Y)$};uJk^893z49&nbIpTP=D`~4V6AnS z&N?FBHZtEfCf`1`pkjQ1eL`Wyq$0;;K>13?v=YaRBFF4P$NWOuB-N44-^AEzMsNQn zVrSW_AC|wmv+SjvWs5&5f8nFD=RYia;bZ+Pklx+B)aaP}f&SHLn*DPN?Xz``+1d&Y zn@mS`+b7eqlZm;u@fiEIj?J`=&a{loaxOlR2d9d~C;gHD-%_7=BjMZFgYg}0=p3aF zjns$ma~NLUTwh1fgZ%)|n?0Op19co1ephu{Sc!ga_G^>`NSx++HA;H#DEslyo zYF!4YAqc9)DR|Bn4jW2xCDC0T_XT*xJeYZ@#3^%k9!jU&3sq4D1NKm;T$|rbbNMK6i&^8oV+*ayZmhCGmji6sWpJjEXD)-YXwksf&V}f1Ib0(}Tcm*HD3?{5XEaJi+eSAq145#evh_|q1eg}TrwwiBgq!RYiM z-Rhk{G><14oYa6cPpT0sdJ`OovF*(0t zDq(k~reaoJ#hiTG#N31G4|M>6Kz_e+)%cBH#=f(A**o7Y-LdD5_x8T=-k#Uq-;3$J zJ+JTB{ra}ABlPwut9GSK&aI!Vv(IF9YfyJj(O~3cvI^M^(t8{QzDH-^%9G)szEfF! z%L7y70m-KRiKc#uralQ$?>J+xI3ud>SYx+nLl;utA(1Yksi}H`XD^_cm}>Hfk^w!y zaIdt>l2}#~!bo4|iUhFm$Wu^7l9QApnz?IafjNyy=ukCbkvQn?oSn-xIn6vYqFfZJ z**D1C7YljL@Gt~V)p(SQWzi111hqopJcXfZ%rH86%NZUm_5w#$iq{c<#zyE&sh2#_ zj&LW?BN%Wy%878fcmq(J6L=0$$YW40AS&w|ak3r)(8?Ln*idC>OP(yiIjOM<>h5g3 z{zQLMyDVjdRI6RKgmj1yQ07n5nQ`6&jW z74T7HGD_?yx;{it0;nF30(_$h)X53bPvpd$jmuA;V;E(J|TBDN#Xi0--t4h=oZ8WmtMT-b z-q4Zw?nAY_X&DK5Cj+0e4~I~w3$*9f=qm6U66#U!&2W!n3R63AB;J#7V2#Ggb%zUw zEhk}Lop?D)g78V~SG?&ZBLLi8q9Him5S(ZXO_Dk%lMGKYbxSw(Oht(I%qEDdEd8_0 z1G3G7vMocitwTVJ>ukfdwow|}7`1&uu5DtDZE}u%O0IpX+CEKVpPmOG1`S>pzWE)a zW8#`!iJQNU-12q!wy(pse*5!xu4Swy7!ateL3VH&bhc>Y%%!oyn-XDRM&W z#(B#O%cu-9t~^OY^*tEyos9VI4+1<9gC~6n-#ufc9Fh|@q0(tJ5Cd~C9;t1F|CAPwA!nD}+ z$`xGWj`$FumbuBvU87(zw~>S7LPXw;BhH{()g})y23~MNqteL2Pq_`m7pljh9n9ZzU)y*P0LJhQA>jc%wm2&@SqVgmlcjnUO5sw zEgluP*tS`c(<^W}JQVopyW%^t@qP%1PzVy<#2}yCyWI2@%9q8xeJK8=a5XQ2DS<%rQ^&;w{5t7lBJ zeJA2+M~c#*V8?g};hx+8YVZW?n#NrYFC9EE8YI2`KzDSZed0y4KI~4A-D23j%BaXK ziGIB)UhkKn4Y zWLm~%S^%%(bF9GGi7MNqTpN&g3UFKv`RXMlUwtdjD*ZHh&93;hyW-Y;5wqcoXh`dw z>GPP?pT&P=n)}B3_a~@orfRS{hP-pJwFw+H$+AK(fbWrj?{xF9bn}pOc}R*(-`p_` zNCfpQVemvz-#zKeJ6&UN_Of#n;JZ9nNF*hNA(1)nnhA)YV0CS`9m4tTAb0*mdeJ8}lA&tVvT*f7~enM~gAA4Gx38urzLqnva4K(g@w#SP2_Q4lI?@fgDA#h`g!4sy6*h@!N?n&ec zsCR56QF#eS!0OECPV{oXW5dXJbUFDhy81DqadZ;YW``6b53*yTG6rrZQ4k+|Kwzvs zAjuGzVhl_*c1|#KNj7##mAa)$T{EN}8K&;p0CBl@mfSzpG%!^joFWfRk%yE==CW?<{sEaG@JaNLRW?|vC6LKmiF}QPV;1ZxUXcpK#nxr@4JIy>SRpwQLgHlWb7~csJ z$G-ajyW@;KVvXHn3=}-kbND+rf?i<5nm=RW*`wWnuttJ6lAVmb6ZIlEK>R|Oss)jq z$q5S##Uf3BaRhgb#NcL(QaQnng=*949!{^oB2*>HxlP8dlfS~9P=g|q=Q1jdn+*f5yZm{qN4O~(qk;?*CP;xmGD8k%jK_cH+$j0L6 za!M;6mL7V@wQzaUw4w|l1_Ci!&kXTVL>{U};s#N!oWbSiLQ)_Pm6S7i5VGMYdYIP? zYtDd;KgYqN1kIsG%#i;;UIWdj7o{TYQi)%bK0ug>2K}inA->$l={E2!K@gny8;9)U za0={pVrnM{(cOs%4&}o6&-{zLlkE5l&CIB zY$5Q`bFka!sEB$bID5Ys;pngt4)V}o7~lL7Ks_S3xjQ_aL=pVL*@i@;f3nd()fk*? z3{5t627ISTUDKoY7UD^1>ilUE_bKg_t?ab8f%*@}Ju7mU;Cdg|($SV3cY4ev$r{u~La_kcb z+~Y8zz=s5lv-F0(hLZn)-UlTMeD?$OodEbI^&N}qoAK?Oy~Neu;eycu)=fpX^D%RM zT+wC)-XHDLjqVDM5P7gA2D!P&Qbl3|8Oco1;~u$K!t{4jl&mmTI4%Mpx#AG-x(T7p zmeeW71UUMkWSowKWv_dlQ|v2h-6ij)xDZmxqD?Oj7OQxilIW3IcFS|V2Q;GD*&_!o z+%=7<$tftRoQ5qaBWL%6HYlqzB~e9$dSFP(1r(RjMfIyl)Muj}hpn?Or{o8q6Rb^M zLc$AstddDyL?yxV5)j@*ccT|~=3N+#0CKena{+fV-yinJ%KDELw^K(8Uax2bx7ck& zb`yhUYMKuTLA~5XTrYQ#+YhSmho#`LNY|W&DT{Re6d3pDS}1@+-o+rgpcD`( zUIG++W8D_0Qsc@yCb8P(&_sEsMBuVsY!QM$LlC*>oekMGQJLn^6jvchZW*6r8JSh7`!GqnGbQz{{H%9#vv%a>?8t$n zqHXq$oQ!ugg&!qqK1dytWrr##y0Zw}pu01%&Ed#yd1x9+@4+c1U^nzWAc-O;Nya`2 zMi^V~I0N7t)px7`fBXd1cX?=3c`(IK0$tecObWrv0>jIisW6a50pVqEGofH)9V(DV za#ba;eLg;bl+kUf;K1hEh(--a?2ARkn?iEZo3Qv zD|C?IkUWGBkl>+%PgXqk5kmUG2Mzate?*geqdDbWZWsmPWnvpv)_6y;25%Z;5%49; zCc-?dDY#uw9^i_8RNT$QyEA|;2)?38zFCQDaz%ENk`?8_%iubIk_Awbu#G{5>r~<_ zvGGEYY9^p7m(fKOvtGt%OoUL}$h}yhqkSpkG!t5$OO(irBDX~*Ae>b;j>kJr$b2A} zQ0~q#LkJkssApW{V>bpaV*7t}p^o#*M8i3<{f#V6r z{*Vym(!gYCa5Cs|(~uO?uvB??syrec)5vsrR0gPW^O#I|czQ*`_L7|Uvf{QBCvDG9 zdNV%-lQ!ioZ8D@cweeev@^>c1Z7ms^S~(_5MrtFtv7KQV^3ZgQ*g*S(1iPgHN%%I< zFVWaH0rA~C-qA8xHSGhq`U`sUB#LIuOw7=-6C1_UP-3+9w2R( zv`u!i*|c2(1{d{SuX>klz!cMhsR5Q=jcw`nqAPc~U>kRgd$+*_0%SLv|J!EHIahKv z`};r7JeN6h-l^xz=l#xmWZ=?9*hQN3F_Nh8KC$PN4EfpW*8M}embQdT8hq{4g+vJS~vOD=@n zK((^32K$IGU3$S1xa+27;TKB43~%K%6*^7`>@!=;mfj-^RGglG|EQk7H;BXx_2(dmT%^-?`+rwv{uVc&F#p z2P&RC|vFvu%0auU0&CU$9c^pfi6#rWw`;ahEqJo84X zG&r7X+8D4JxZx2%Lw1UpV1_Qagc-`TwUbWd62_>uM*~&vm?cSD+U6*N&z4`Rh@f#$ zi9>bmpvb!nrf*1r&uC4m&o;&e?Et)RY`t-Ytk};o4aPJ!&VVt4mHEpf9qg@pAz>0(y z43ZFPC@K}nt`mFVGC6}Rii2Yd=JspFfWj|$3lmZvGq8X%o=WPJ#+)PwCW$IinxYeO zNfD21(&k{v%w>;(e{Iq-nBF?b=pbV{K!=CpP`Cta zjO->4!}QmY6B6=r0>~y;$%h6WKLcGnheuG3&GKqiA-MT#Mjn08xQGaTDL}j`dIdn7 zl(>fYbsFN0@f#ZvsVK&qhHhyQj^D~CZ>3M2olC#ny8JhDgUv6^FL?6lvibAN=08MdR$qu zFqd!vw>-xpUD*>jOx zCT*_mouq{|{q+ji?o;+u=p;~*oTn~k(Fe@gGz*mEqSGl+grpm%%$|V%h6HF!fe|_o z%n}1VW_3xik?uJAT8aPki8p1R$5I{TIT4)oJ8TWDdG+xtH$i=m%#0-ah@?MA=8h1> zxr*;*-EGm-+5wpOKDy8zRbQWt;HD3iQcD?3(HU#XD<_^RuyQbf=}Gc5#E?H$mN^c{AHwdpK7~NptTe%9r?qCLg zEES+j;#iWQ+#4jc#FN1;T4_j1*ON$>W{pAe_fif;RYm#zm{LB9E`UP7A86Q%t3ioJ zFRzI~x}pYkE<(J1@Y;Ih_;nr;<2N*!IZod>HOkwT-qN!4pJv6&=D$Ae(dTD8{`|}* zp38sYSxV2q@60Ejo&MOfwJ*H>gYKobwDMO>j*=gC{kyOKNb;hWZ|)3(g}mY$`bmTb*C4`{57>jOJoUFcK)(E0n z;W7ssI8S<e!`-g+3hBLW8RE-Y9Cvm=n4nHRP2S7dnAcs8iVPeuNk}b zc!_u=_-3)lZIq}5f|5=d4BU)HG(ffzMl+NE&tS3Kpu!q7ZsPc$T&fTa2ahnmK{1(Z z{>a&=yhS~8nIiQ{$aB;st3H}-ZmLPOhT#{SSi%#{)$d1{<+TuqD=~gKas0}f*i||# z4jgY71d?749QVX;ganctWO5^N93^wh&@IhF-)>%VOY@TNv@g4Bp7-|;y?p1x|9RIV zFWoiorMn+_Y3jo-O?~90-#_xw-Sb|Y{)G3|wq>`pqL9CZ6gLLD;kS`vCh;4{TY%5N zFP0u0fR^4@)S`o7=}mt%NM9yLoMA_eLguJ5FFYet2tp#ln&akx2GRjT1aUJH4YNKY z!eH++NNSrckq{xK8GMSW=639}48^Lu(1!q{Q!_?QbtPKTXC`N8WBjv~l55=xNa>HT%I7aJX zaQvY%qegZ;2f<>4u%;VC8I zys~g!c^KceGR)^zLLN)7P#!jwlw!$6<>8Af!oYEks4&MdGET{)$0;=q-XNhBOgBMN z!u+PDCI8SF{^fmd{_6hMe|`V!zj*-CYmk0@Kcv@x``{Zto)x*FammdT=B7wDek{F# z5=Lx1aqi-p2Iw><(z{mJ4Wqh@!X_|I!1t62noEM-F~W;DJ4J9@0a%~ZjzQaOq=xaW ztY?ssr+7yJ2m0p5ckO-jS+7?my32*~A+eY8v*b$}Hx352*>VZVHb*kVR#mR<+WQ8f zBV?B8Bc2$TEcv4)Gz+acd7QeO$J{r_nbfYq)J|$p%>{mGrh`ydH?AAxJP*Lx3osp& z(dsn;CU(&vdoAn3$=N*!clFA!+q3scARE^Zzlf?XXbdzS3Vn2Av%DhL$C5hroa4N--!Nz6Eu5`N3olF zBzEI4E~|-NS`)pbI&yJUgL1rU8iPX_XfviE;~od!E3?+zOk4pqN0}Y9 zz)p(j8~qhEylWEJrV95DsoEU!Su)~rFO(pJ7drqj;bpLm!Os)LtS8r zL$v_g3f=+OL5K`a{Hj!%N~q9CmB#C2xunE4^0JxUy8riN-l}@HmW2~{YgyQ`uSaZi z%l0oFt?(>$UDaX0o&44iF{_cwt}0wNNK#Jx0n}s>bD#uP23#ZBRFfGYnC~Kvp}}fm zaitjWQo9(tcvl+7Hc8iFo2rCchGAs$kgD(&R<9->Kr8Q3xKGMtI4WWiNs*!HfL$6* z7TskEQ%0d&L79PzjR3DblnCP76xO1O2w$qg4hMY(*mwx~O%Z7v|5A|Jte5d**GWZa zlY7bX5V)(tqd%3=A#RkLULbjCRC#Dh89+P)5Wg58UKzfuDsnlb>c|x}(JPVT(W`5t z*VIOVuva(4=K1|xR*uQoV`cc@C$~n0 zRkO*u001BWNkl7`4>EXdmkNrcukt z9vQYxVcIrpM5nrXpOBjGPHvAm3D($Yx6s*Gf8sKa#7QNNZmg2h5z}+QS@fu+-3G6J zXN--poi3~h5O>lOA@Qyr@vOnLRv>stGuo{P&h#+~M@4ysB7V8B+dz5wS=e2n>=To* z4RaNdsfCaxc&_O;Z)Bkv#AZ^DEdSeSgk>&59AU!1byW#}-3riHNE$i2hNZNilkXzd zD-p89cR*h>8~p(pR|Wg z9IsVoY|tR`9BokGOCEzZH4&aFuPnbJxcVi=GvH^OtMHv$W)$M3r7%1qnQwt*n8bMK zVn~&dOE9hyxvYx5b5=(QzO zl0zZJ)@LA|$0KZFt=gvwd*fV1jXBQXvvX_*29NOyie2=InHcm6%Jp0%(>rKtkohYk zcbKM3UX5DrjpO*4>}q9f%GFUG4aBmKZKMhaVIa7{-)^d?bhiw(OJ2 zuJIn?hRVSt`zudxF{QE%-90juCSE3h*u3Qp8#C)mOvZl$=p|T-^&*|-8REdPC<2`! zTN7afxvNCZBIRp{^3{TMq%GsV8Yv738q|(~eWANTL_HT~R}%(nRGX?Ia=%8nPh*Hk ztx8;5@d2XWY0Q=3B=Q>(4P_!YtL7+@7bx6d665s(ZjQZJ%Z0*_hejY7AofHB&06nM z`znBgArV!Vs(4fj^&;4j+9mvGjYAf`2c;vPw?=d6h>|jSbxa0fF(_3*73MhT@d^&A z@JCL7IHej6twgV6l-Eh1Tw*L7W&xnrA)v4Ggl}jFVe*8o_k>wxUsF#K-URNe&?Q8# zAkhu%#x5w{4W0w~C3c5lBvZ8fQTWa|TdI&fmfjST&EF+#vJq7Y3UQ52qMnA4NJ5Q2 zTc4o>8Bh9jZx=Wk1h*+x8st@EN5phdE#js_v6U$=Zo|;>NN^>JsK^!0}9-Y5Q>%Bqv@(r3&hQ8%)IyW5(r-U_Ys^R4nSnxiS$G$ z3&927BpeZRB(7bR$l0t~;S$@pYJ}-5mf0RakF>24<(^|AxONOwk)%3mF#5jK8M1UF`$AAX?-61JuAaAXF@MHyjm5Iw&E1iB-y^*=HUQ;_JpD6Jw zB}GfBv2jX@-6Xw1cWdcw)j(BZo-nfv0A=P8RLLcqLO!3Rm z;G`&7@i7W*;t7-%R_xMkA3F5 zR&8#So3^TvG2`9>?!<#~p5E`J##ZG$Zq7?dsX{@Cs#`i*;_?z49;_0P8lV}GXf+9K z1UCt74CmIcSf)btU*Gd?KV=SbyCBQA>nshh9 znj7TroFelhT#nyrYkB@ENKzxAD0I|FZR?|z0~BfJ(vVp}(<3n*tn@LeOxYXnhpA0c z0R-`oWD*Z_CXQFgAFJTst5l*M2hJKa2VqVz7GgU8GkuP?L{*9r>!XrtI8K6!o4_4m zY8&9LLeG&`As^*)$_(wJ-Bqs1A(?N%;)?+67TRhk2qXI9Hbp%}-niDJT0|S+6n!~T zxAv(ZgF5mvup|-IFZ8wcX6WkE;Mm}zylo8qJlW>YOsOc-{-9T@ zcy>Qh-OOc)pD2xQSJ5XKRkwCs+R4RbWqK9O)r|P$p<_7uRKyw?f4dpqF$W!M@KYq% z=vwL*1K*vZzIke?9cZuPT|jLFcZc!^)J|$!Y$T~iW=0REoaKaP5L}%F&3ecxk|>r^ zZr#YE2Q$sv(|k$}SuRb@W8(-fYRg#Y?!}+YwYL_&)h-;=+c8&Z21?y!dIOtl@ z-Bq}Z*%X-8@m_`=qnZ6B+i?EU`&9hbq zXMpi>{(BV)uv`MEGR%SUOLdr~I&xVJ{Z`{AP)s%QIg|^v8B)6v|LEmnO!AeIKU7|X ze_5st|s(M9Ic+SWM1NKm+BwCD{$8>v9D|JcDB)+HmB$ z3k__&cnQBYs^w&xUNGZ4qXaJeXhBJH+YYU1w)h{H|(*c@2|-l+qr>+4agJL-kE1PKu=<>pSFe zBVyn^sgRVTUKq@mt5jjR)RyjYlTEo%*K#@HE!9d0^QE%6%sb8!BpDG1%^7uTn|E}H zK?wmuiD8phVlo)*a%6%Y8L!zyaE28{OTv^10qgh{qfb<64HD8-8Bz!iA%(UN|hsdaaCP-izr-)OHD0cJSUq!qM`wHU5S>2Z?+F%JRN(a0aGRkDU6yw%yJmH zOf1e&e;}CFghbA$!`9Q9IC2us@F+76-bLwM7fAzgjn#1pxE&rhTLQb8?-}G6GBH#X zmZxQXJfrZpOg)UVqwUDtVo9Mt#*v#WKfa2E&*_D3?Dyth0SFfDy04x=byv7% z*92GK!)f*!Ok_qU09ss;nwR0FdShz*vKkt%kVypyc17*ovMktgH#;g8-D(PI`ozkZ zttXZ|OZ*cqdzZ`+!P*|xYeKaKl{;jD(;z#gcjm97lQ}NFP*<0hsdvuf->%R@l!`PL zT$INoVG}!1NRdUxM&QrxuR<(dWNg!JvPH!NsT}PQZ z5zfqLBs<-bh&DkP$ez|nGQ3V8Zjic8+!rDngGA&5>cbYrt(GtGL0Sd4zME5`o_&cy za^7I!sRHBlBZBUg+F_ii#ETAfXrkNH8ei6Ron)1(~z{i!hKay*ezcRyR{b?G)Yv|qmudB#|Z9~ zV-VUx>WYb`?lA_L=*DO|wjL$+38q3xo=&Gaf8h8Bllw5)O~YdyZ8X~{3(KvEaMlD@ zCkgIXoSZr&aSJJc-7X8Vb>e4A<(NU=KsNZ?#UPf_7ENu7rnD=8k10C2*&w(%F`7pf zn!zoHJS*ZYV#2iJ+*(hJ;B~^L6T~ZNm3S**&nBd^mH+=s*-YCSQq$#O3L%7D6`02m zMGdRt5b57=7VF;%62cWi-fT`3W?(==f=0604OB`a95BfuVa}j;Qw!KNM^q5vMCVE= zK}(X}P?#-lMx=T+e#xu}rF$^4LAn_5Ed*z#lU`xMd&!_X%DlD_)U;fM z25A78F+krfh_q98wq6Z86VMg??OjouIx83sKsBY>$E5UZOtYIt6}5M(qXTGIFz$(J ziOsz$mt&!}O(9i6Kwa8TL^%+FYfHV`2;#gHXuyQ+jOv*(M)iupE%rKEJIj)GM$O&n zy5w2;sYLt8ybBm?il1i<4H2}$I zC&b-}L!$AJfndD^GdiDFB4!E6P08GyyUfz57-;)4GJx{3(B!5FlBJtqf!meP0GqZw zDnRU_O9DFM#?e7$w6_`GP~Fx=d_-npvavGdUv5O_yQ2&sHsD&eZtf_`FtT?E&C$&C z%Y&ivsAinSOe5QzxG9<-B9lyBKge#O=;bL5H&!BJJ6 z*E1^2?UFIul_L>!GOu|&om?0de~D`aBgbJlI;GH|S6*hMrcZ;!0aI88rk2{OTr#>? zYMEttr+UhO>{g^UKQ$-O93Nu4td%_Lu4on<`AdVYa^g6=LWE#9s|WS^>DNqNun})* z;;UO?$|P(S)An>*ZE{OS-}vT6v`xZY;Y0CRlD<`Rb{n&S)hw?lm{lDDnq_gYg1K29 zf~Cu@6sP4XCx8-6Ii`W|EvIs3Yh@_2n#fxY8aIs0r7GwHs5MtvNOPR4xER?~B^#hj zHr$|8fiYCWFv8qE8p8uB4`#D7C<`TNd}k;+W2#lZB&Y%3X`s{9T||71*oX$N#k<@T zc2s=5UwHcg5S#lB8sduzmVbwwE=vY@RyrY&tu zUNO+_L6!6<9nCPp?i2yBwWuDL@*k*??Ojo`d(cioKu=><#3zz=iQu($3(9R|=Im66 zl#Wc0RU?r}b&K0TB`)}uTY4_aRmh~0dkowg-mzOUR>z}Cx&^zA4q1F!qG+Z-FeHNAKt8=*G8q&q$>h8Bp>&VjO>4yRjCw53WV0G9LElvY>lMf?3gi|CAVY~4 zJ&;q(NiM;_WRsATJ+drd)P+o`vMEih^*0-|rV(}jj>PXD=G|oh8I>^@HAg5Z!(Vp^ z+2~~Mx72Dt4N76D$%cAyn_9rjGO>q;1vmf-I65NEjtCL~(G$VcNty15vpwSIj3_Ea zhz=2@`#`xzNZ$bWNyC>WJx&_Pn<8AR-b1vNVXJ<<_`UnGyJ&Hd#Zx;QXw)oSTJ|(kkuK z^o`+h`*0T$In3h*6AZR|bjT>AWE^v&ke~#IO9W4Akis+&;AELzCkFV)O9FX?xJDM>mYCMdT5f`J zaFi#i?-pB=`EIE-Uu>FC25!*{UBU3F(S>#x zG)Q?}B!h3N*`@2j;{de%=+PdoD~C#TDFY~_@fVx!a?Gbh#YRR;Ii=Q;(h*&h7@?K) z`mkW7(-a9YahBu4a;sBWwyUT`rx++W;w?wJVjNXx#B0uKOGcqxHOqkDOD4Fqpe~ti z+_k~(E~8B|&2_UP{8c+DhM+p?uwHgb5DA?S6WJ8`wVQ^g8+VvGvJ8N4$@C*eEuM?= z2U6Z)tgVq0I%!oEdE2ofD`i~j4iVtI&TQ|7nG8zudXTc%e%vC#b9^5!yPfF?;~3V~X8z1ll^7Z;nhe{s;X zk&SXz(+PAI92Y6A&AcwM7%-zP_WCZTLWBb;R|;nX!7D@A8m3f;{H+LOL1DR2HLFq- zH+w_^x2jtyDqF!<2VR1i)#4;DM583Rt4vCC+(uNt=~n_W<3f- zXuex)oCDk?*1=FFw-i4@(x`9>+?0AXrO@yS);Cy9U)MFfi>j^`r$k*+3wii8U$=wy zE>q;urFwy;!0V>8L^Ol6&NA~+_8K2MBrKbmExU%sXMQ$#aPt4{M2xJ{+*jUbHqOkkI!o6m^v0-^oox@UlcYyUdqmrbO6v8Q+`&W_F8s*L#Eva(NqS zsigO~R%|AG;V!n4x`>%5&%koaWH8#v$n1U%m?q%WZDO%* z&`rZIx6GhS?4n3FVp7%y)@1;kch2xmF7V|l9c{9dSiIZf6@n(XC%lEGzb#ST!d?IH+PHdz|N0`i6Z0wbS11T08U_hirAbYv0cd|r zYe-LFY)i;SG~@wT;sEU;Jo&EHn_6iyyv#yX1(9g_THtq5D8C~zra3eYy6zUfCsT+Y ze#f+h#M`G8E@N9mV_e!*D%(r;J1au>0)Txgq zk(B1JYXr4joXS)NQy`>8?5! zo3`Fr)E-y22(qBND3Dp;%PR2c@?0`bo9RuT;dNX2n`fnb#!RT_&(+(%$eN1(<|RR# zKnVzza0h0pL~BDiB-oFj!LvBxQKlA<0F5DzGNsL_1vlNzE;CpRx6mI{g*4N%S$CI9 zUPnk+{%2N^V3!gc0JjVS$fW8?=`|7uнjcP&#F(3#dCJt2-+)^!goUc`TPVw+ zRw{^9%y&&@^5hbJbw#b>$=)pl54wx2W3#$tJvLXN-&N#K>{{?Dagxjef6>$H|Mk9= zlV|#JO=2OJ!yv_q_4*~#h5U{d(+C;XMFB41FW@wipdOyQtbp&r!1eX9``_PjN7u6S ze19%CqE88&FT$4toTzcE)}-hV*d=#FlWBWTY7L*$7?{+KJhitAz8&on8~|k475#1c z&+oWgu^xl4&}@hD0ThRIwnXHd-i~yjUm~({005xzjiC$b0+UJGp3{J1uy-r}q55KZ z%czAhPwt8RA)EM|(7*H+JPiORs%++0~CMzxL7Pu32$gn}8CZ2#*1lr9C{MF*Laak(%5V z&gqTc@zk(Goi=zb&>&#_w=b;w)r;%KwP5=g@gP96c$bNH$rI#8e6Lk?ops=0+jC^0N+Ub+0d)~zT+(F2I40o0K-Nk-)2`nfrYq2CL zFIIO;Bu`5(mN|v~993vexrIhkSW;+dS({;A7n<^J>O$@oS(=i|BJ0@bR&I%Z@=WjC z1v`r7uXEhnPtpJd0=V~-83QPOOX$@BvwZIds}6R)xn zKlGTM@5O;ZgH-`Xdw61d8210fR#1GfX~XsoOi&I-@ADe{rSEMX`sCC1HXL5O>F9mY zPtv=iliEQ@q5(j2d~4`}Wt;pMNdJrvvd zX>8|-dGXzodt>8TKv9NmFk1Y885z?O9McrEw}Dm#Zv?tEbjjSI1E;>qn-d?`8qDmC zZ8`e+_nuk_W{ys>=?Gug5`=`K0@$g3AwRw)cz$!h25Zt0o@v8_RvEDp6R z!qb6UHkopGYz$b`1g<#U%n`S89|+m3zl$KeAL z+e0aBSREGat@Ve7_njQv`DtXwiM&~Z&bIJ54gT5%TaSGHH7xszJ)dnnd}hX*8_#X> zJ9_}NVOins$T=SW?_U0}?CnkGRC-g}!WT9Ne)!z#jfc)m?uwq*6o5kjofX1Pc!&-D zCztMdG`{=X20wpuo&>7D-anx^nA(EFz&7c*J6mya#x(}IeLET#Z9A_y0P}(&fxUZn z{FOBaZhK_;c#!#UTxi1;yAYmZYY^TUn5nH5Rz8wa9e_91Rpxh<;C9X|_GK3NjF-05 zpHbwy$rD|_>(rsozPhp&G`l}}+T!L{Hyu0u^|EazhClvv^S;xi&wl8bHbCDh{f{jA z0HZ)$zxe9fNa~D%ywZSkTEBPgk-m4fU2x~Zw0tkT%sI5Mj9K}<8$B@? z!{N`qx~eMdD)43I`)+F*T(Rv$eABV`=Htt^p18Fs28FP#a*J^c*HnimPal|45=glh z$1v;dtz-UuA*gkzOJrAsDHQT(1W8Fkg1ui6$fyQ%1|VgZ`LoMGnESKJu`sg=_>O;2 z0EL*!;Mbo`W@udQ2gwX(SS}RYfdFul;AF3W>#W5Xi$a^1hXg_yV;$5RX*i+O71a-l(9Gq??27maw3R*0LQ zt}bDYyNj*h)|YB*z(MEq{=oX9_r1S;!rhCeVA#%|GHu`{PwdWl!?!gL0xn%hTYq*D zzBE_Wgdvkv?7OBeaz%9re(#>Q`j*BRyr4P7)-@?XzJtMa^%1DTecjeF$W8)EG77z* z7k>BPDp;tDnO>thy`$hle$un_#F?*u_t2``QhcwoU*G2y0k>P)6A95B9q|^st zy8=xvnG?_GiC%vH&@~S&g{{b&6ZZ4u&PY;?_xV-(`!^rG?vbTGe__qK1E(Jw+70Xg zxZ7|82X36V>=!St&6^zuUOS0{u;-!N-52}C3u~`~23tc{J+K4{?Q~|peD095D{{yD zRX=)qcv4&Fyn6q&4=nw+7uM$X4UTQX@z~o0Qb<*RW>fInk1pSN;!ks9yKZ}Y#gCsE z4(~Xz`N-$)S%VWhkdFXp@E>!SPP0NJ}=+i+oH;9DMl)4;ajy(ghAIELVk>K9E%Hh3c>l zle;3x1X+i6Z{6XsEx}h-9}I0fe(n8Ba=K#|cmiQ#){-VK6^^W;LwpgqGmt_YGxsMZs5H)(he~iqr-i z6#K4ih+ON5m>~}3aQU6K!6PTXy0$LjD)z(1d1%p&SBLgp2wa}&g_0@dxR1c>huuq# zp85*p(T$!cyugm>1HN@f+FsiX`)KN;t1qt#fqhPig&Vw=0ryi>!Yxx|>}%#pR)O!k z%{YeZ>LQ@8lcz3zeQ5uCD-NVhA4r?weShV_*M{~b-@O=a{iqAZ@O9Bs>u#zC!R}AF zm&Wkw=Bum2Q|GO|ydne~&sDPT?@fg`7(gUCxhf)N1)vrc73Ql5*~Dv1l|*X31h7{4 znb%T+u*SQ@Tt+ zwNP?Y8&eAc?$OF~C@&dv`{p+FNOZn1u{(H^T*ZHD7^uOE10cNz)*kMAd)tJ&7F|>p zOr1W^{pOaf2hJ=SKD1^3nR$zMrqA@|0qCa0f<#!`{;` ztk@4wNNtaNqt^H6;O?~tPOaT{s^b04=hXY5(B2vbK&Ny?03tx2d9htdGZrKjENJQ9 z=G}ViJP%4aP}a|^*tg-}nfEsw*>?2vX>V=>DV^F80XF^i-Xld~*<(b?N;? z@ZLD+eI)pi&%gJFl|a35ZK3NQUbbo7JxIA51p2GxF|+!>jk5TD0ltrXy#5_QIMA(PN<-Lk&)j2Sn)fH#T1V z;L`D}LAVR;7rgxu;5fY_0u_Jw^zfLb0C>cQ{{AllK z*j!7toY;Bz^MC4HI(ddShZt>6lkQ$T{`ZR}{C?56I~ReXGPVZq++q|x*VRNogTrrj zfq&9ni(ilLf6BM(oL?`1G~c)D)%ZRr1d08i9>W`3K)izze{Z>rVd-WZ!%usbrcL)^ zM8&GIa|$SF9GU3krd!#if$UO3DlilX%M!RekPZw5fR+Z*s{@MBq=*@;RGz6J%LZkX zh{&+WV1v{yrIsm^O)?G=OKfhR6F2TEp%%(j3uRJ;N0^l0pqPm=9|mQSN%rdHHdiS! zG1`iLV`;1h8EvD_HI{yhEct{s%^{QoOQT;3jX9?E``}dd&bCQY7Xv7N`{2q0C%(L+ zb4kjy{%<$Nb|3ws^cgr3_QT5uuDkc`ZMJ)n;N5R+J@u#mf`c3Q`Lbs|+sr_V)|t`**#$WIu=$*bg_@-)R}lDDY(zc)!~gKm6%eAjv0|`?9-Yu!q~d+t2s-zg6%1 z!_qy6PJVU8eM1~ffTJ`EVch(+-ucV-ZajFVX2F)nhW4!4cM328h=lWa<@;Nf@A+&@ zV<4$)@wXpa0XVqg{-LprLC|-5Kl>8k1!w@NzG%~t!>7Ma>x=?b;eZeMhhw`y#bUHG;fhf@Mngsy??0Gi}h1l(mz@Y!*oc4jvboT4b>TY{k-C%XJQlFIs%%KIN4 z+zkW-wt`YFdv`O;;xeex7`XMZj19SDpwjcZA#z&XI3V=NC@a=eS!(mv! z6dM2d#vuF4jR#Nv{+0Exp5LnXclvjH`RBhpyJ8=lJa&EhWz7d$&u>JpiM5KPdV)z+=!TzLz7J3i zYWj^ehrmC9OTOa|t6m!3k4wtu9*Ux*wx9-oYwe-e)*SS2Jr0BaR<*b6txXFy9Eof{ z@!J>IUFZqG3lQ0H;-yswV0IIlg6+N?fcqajv2tu(07kOu(3u%eu08+Gg?BzOy!Mlm zcx2=lsX8FS3wb@f%uMiNKWv=Y?`{K1z7*N}_~MaSu&l z3p#bVDaw&kd{*hL(Qn3jj*i2ac%72~aede+vY=cOHix3paC&mmTLl{2eosHBiRVMR z-&nHm!v9!!S$XJ!fB)c#{+*y2KoNih#t`0HTfq#DY!58h4q7-FoP3f0(u&}oV}Amg zr{_~JggRw{gD7}|30{l@KlJIB_+sT_P-VfYL!b+w-B$eq?O>ew+v<#`e2iQ zg7ANI^!y4hpa*tx)q9)a06wuPgerUWf#+82ORDS#WHTUG>%s9E3BGLa=EI*i_HTc3 z=^i-p|MG>kARa-6zrOY$03CQVZ)o?N@XkZ0zq;#{^{B#LT@Plk$$>cMHTZw>^y;0* z|CHVx1?Fb-M7JIN;`>jo1dKwz&#v5u{Z@P782*FDR&G3WCcQIiZw}$VD50bM#Fo&M z*@MvX6%PzeYz{HL;TNDhsXaWYEtE40`F`00Lm)OMwT7UV-Jkvm;B{VO;4+G$A{#dkk6xa;^Af4=Rp72}$M95`Yn4vUsXVRUC_wIs1cxb^#M=xv$z`18kbMPOZ zT2;PaGw?mBeDSYdT>IgH)6nKb@)HhvG^rHex-a_4+v^T}s}>0khn1f`H4Kgl)p{$Q zUElx&hpG!20^h9iy}V-o(<}D@$|p7ifA!+JZO1+bz5lzH)=g>**;}MdcHXJ25vbC? z`Do>Po1b5{uX@22s0Gvi?Q?6R+m8S4r4P@k^*1nWYE^}cP#?ryXLKi185FW+e&bM{U0|z@ZR>`x3)g? z!Hzp08YY5U!mvQ_>zUxS5m%uvb=pArv+I{_J<{yaM-8-c?;GEhY04es?)(Lkm0)x9BS-s|Cq&djy+(P;q_kT$YQ;KpBvXx|3fovtc zO9jDfhQSuL9Rb?nw|Us>+XQ4c#Nxv14v2? zSy;iE7}L;Uib5lk$Y^soMc*notG(Pxh=h!c!xUOd?^w(5}Ibf2R@4fqx z;Xj7=z=7^p_pgALDYrPF-3-`VS*-}3j|6XdYfE7JanN^@+rz-w`VY1SwjLW#=l7R~ z4?F_`t!e;}ZD#>aDi|?Q>HW`D2Of>@PAXUkiuK=KSUbG$6wm}VDrmIm$0rKk`lzjc zd(DEabqlv%``}V22QIAJe+pEvy)6X0{hyy(z3arE(z~K?e$VVeEer&o&>Dh%0pB1q zAp^HS!GY4x=!n{zLMZ_2X8KMDm~IKJ-g~O(?T=v0DQ$QOZ2ILN*IxV3QV`BS@ck#h zx@ztcpdK`m)g9Y^@=G}2pYI7={(wYLfU`pIKdd^hK9JJ~F5&B0ft__9Y;9b$t#;wo z{5L;>!vPTd?e&Kr!f;JLj23?1{_qfnHEO)jDbNt^mb|m6rGH1=g00o>Z-Ju*91$E8 zLFTZJqay<9JG(cwe9y_pDg2G*~=j6|yU2|T&AKb$eOZMQbtGub*kqNCq zkhCwZIshvPX0T{Z@>^HVU7FDy0k;Hh@5%?3Y(4r#W?u{>yn`N%p~0h{fSdV7m3J}) zuYU6MFq~=Va1b8b6!dRB1_VF9!T(Kg(yQnZR}N&fM<+G}E}au^`e578$De`*hwf5f zwOi<+x8Xm?yr3>Hu<2;^dt3hTv6YKA9sSqm*95m7PxiorIQg4Dt~C$1=QR&3 z8QTy@_5?H9BJdP`{N$?f^#O3S*WJJ5<3peSxM%6wohNT?CS6iu5=@}M^OgRsi{PL1 z75Z*%id|9`g!ezQzl zRk_8WNXDa$-P9B)g{tS*XXkr2?>k-f+ly|+IVgDr>qkl>3a-Mwh;g6+V4`#p=n|6f@Z0uBD1rWjPuEt2l?O8ky{78m|u zEgULuZ;EBi9C&!)jz9m`U#_YQrA+IGcRI4+*mbqxbASDQWaF{N7Joec&V^H8pQsVS z%XC#$X#a^Xztb2?FZ9~#{a4OKqD_5e{e-3f?7i(rz5tPRVZHyGRo<6Z?SGaucp649 zLii^oZJ3btRRb@r+7J7DOuav?BQh(vbNQ~%94(=9>wGic*a#Xt9VlPk|MRC;FWPh@ zy(2QIF__UE-Er)Trv7coO~KsWSa`>YL#Mt1aXPUnnAH{CcI=DqKY`!AfMw6E-2WUA zd=f+pYGcqz4ghd=e!c(qFMkMt_{DQ;U_9yVkvT!s`&XkX503Tt7i>K8 z&WDF^!0n+wuG|kO9Nz>S30`_%9Lz528-z>j_vq)~B*rxb#x@1s-+1^DAa&KiH)_4} z;=4NrwqNM+pWo=e_qFv0PJa3AdCP&MKb=4Pm;d>n#}LHc^D*?XJ-p-ixRyXtRsYP_ zHomgvU^4&d3yNY~D>xa9ZwcJ~_=@dEzj$)Vo|N|RMY9K?UuSFR+#2r_@jZhdf11`2 z{?6meL9YJt`85}U25$-F_Qim+(H$r9`eN?hSZ+@&wJi*8uj>6R8xNehY)%{&?dfHE z7jHfafX2_5ZTNJcVroYi7O>a-`Hi9)SeD76$xabk|@1`qzJ$w_=>f zKe+=OfbTz-?mKYe%Vz?+9Mcx(;-69B)^-)sPA~QWF|X%1cx@Qof7{eWO)r16Y47P9 z>m!*n2VMy8fkSQTbPzYbOUna5&;NLERaU+?56-*yEapv=eAlAM)BC5+8%F(Ggt3l< z%?{FPN|8U066D=h0)ScO4DSsT2fw~1oHfInF@0e0qht5KyUj6mF>K_A7i=HebSz`W zfGgj7X(^3i?yBsWUT8CADvn_m5d8Ox((diQ=Mfx3UZFo@+Q9dk|HPar$N8R?r`=RAD0Ea%>ti7>6mQDk_8RP*d~=C%)_w z{@Uqh4}lMEH*KS$oP$)3OYad+A3(p93GD)lFsA&8hR^5ztA2J`MhFGHN5>CGwd zVj*Llf1#v+%gr#QiWb^}FZB|0`4(1@*3gFT7I_8Ah0?Ejs5m6-u7J%9?_%^Qc{q{z zCf~cbZ2pJvXt(S?vvAeHi(mx{eR;(`c=PK2xMA!5(@QoVhu_v$HsK#375P%`UHsmP zgYBdIXAAzOwuio1GcY&u$>Gyq z_is9~^pnq)?E38cPvAyP0bn%-W(PiorXt&qZ#n$=)K}M!hYbw-ztR8Wr-wHmKC^Q7 z$;kE-PYmr|y6dyd?#QI3Kt@*tB>ZK)+mC$#smH%#9DX4PpI_^3 z8`%EEpa0_Dc09EG#PHrzAkS^oT5?)F&5O1s6)r4#eIs}UoMUS+w&TRV zKDXw=M!&5oa9QtQWZUue2Tu=u^63-t-K%z;{Na-;$AOz~4P998`^Dp{{`x9EH7*K63y>Oxlcr z``_EP@5Gn)zqcJMLA$)S`~YksfPUKa{yQI9HMr>*l7x4?MV2WPuw*Re64~8E@Bv`= z#|J;Zt~%t(A4r?l|1W*ZHt#*%KYXZv^`XuCPW?;o(zJUQPbv1LPwNM=??3+K1Mh9m znlUi>p2awZS2x-3S_rl7d~j9#qhl8p`zHR!2eaPVvUJPw4486(ca)SrB10*?%AY|g zt3m*Z$S(I~m-;d*d>Do<6NAi3Ag_-Y&E-BegADva0y^Vgs#I{fv_=%ByTycW`o+3X zi`&6Zm(mVcW?~D7Z^}@`QeTqW3?>s3&AM3VX)ZCzbL-<#+Pu)xj&`#ae_N2KuSZWs zqp9zaRf>!@K?Ph}9fso-a5jqpTHwtsL|9*05&TKl64cB0F3wRcUtSTsxHN!1tk8E! zS>UpYAZvLrxUxD#5pavofLg8sFZA;5#^`rj1~c-#msbXJihbMyi284PqCf6ll8YbK z3H_$(^w#>Pv_(Ktfv`d#wudhC_%hnVw?Df4qFI9%H2Cc-x0{1mosrD0hz;}?ir)|= z!bB_X+lTY3he4$R-GF5`KD-oejc*P(sV?-D(-Tc;A%Wb6pA$hB+hL9^z`h{h z@8$=W-u%c?a2}ALN^(mO1ba?j49*bWd3;4u`C<^i&n?>vqC2%KGN}#!fcnZgL*IRD z1x#(MCjgU0PtzKt_KB_S;f$_mR(BL;ol7aBGwOgP1-Q2a$C0yvVSt*ZBTm@a6a>AT z)e!*!4#rb@W7$2l0{CYwp)n2qi+W;L%^3o*KLNa&I@*pUj{pE507*naR2#w5bT)-- zB>cgFP40|be*aKfd)S6wdLyphsDpGmtUz{mG`$_0f=8U$6@_`(T5vGq8vO7GzVq0M z%kCe7LyX;Y>Z0DE@o;8O3{F%|I1e?2Kvvu76@mu@bBESo4JS4Rl0AW2?_YY|oS_MI zephEC9b;di$t|JtYP{24UcX}7iH!UKMz;P%d=w+f;cbUAqR6;nd7rPT3E$$0!B$G2 zDN@E&$hePOU4v>J66~6b%L2@0ICO&gaa)hqqrhP#Itp2*`(TIWO5P7RzN{<&C+4XS z4?`w>+5q2j=X(kFk!!2M?tI+imzM`FDG5M1bbV2=A2vI-pYMaht6)>-8;Aa!1SLLN zIST4Zk-EfgKT{hPN?)?L&ZtyGMkVf(1ne#&1VeeLC~H_%C>M3l0$e~Zf}N|lN=;&1 zWeDFsb|ftG>2Fa~ghVIt$@ElQg_exllwrp4FPxR+vXCV?T^3j^@s_E)fNJ6TdR2)w z5D9PTk;J}Snp?KKB3^C;9x$B0(q{~CJE{dL&j<1jWaoQx>4?X-vI@M}1u_iCWT_$X zYpQjG%hUxE832vF6!}Pf%R<2`4m$*T;k-WYY zaFCzWf`FTd4Z^J;RGHL_8HUBAHs}`%`7XvKXgq*62bw&Xgy@ly3>aDI^(u zPkTt)2#VOy~%&d1P)Yo<5lp2d)| zXX2%s$|Wp0V*nD#XCK@JOIFkFncl3KUZgfm2gWu-S~KLfITeg@i4(*q3i`Ipa;CE% zGH{s%G917SS{UiH-U*%UCzCW)O*muvtfnE>u0HX!{Du;%CrWn5g2 zZDKD_f}p2zl;ot`p_e3sg9RpDo0(qVGajdf%LZ`iiKWCT>XF(!y)xk;$0(VlF|OHl0VdaxEcgt5Dd+O|_DZiS>+BX^m?!j~IUj zn26Vn1aZZEYL;%98nm?-L{owddrT zeJ3m4*#r{RsTzUm=v|tHgLVaOn{tFsnvtv|6|yUmi8slwESm5XZDg`)eH{GTgbm8$ z;Pp`p;%HGztwzFixhWf!7>}DWx+%<)lOXABl-Si$h<8x~wy6>)vD?{5kEx9^G{V&S zfYTFTznjvG=b>xn#Qj^3T~Ut_5_ho`+-DJRX7!e+;Y=kanPujK*Rn%tr<)X*7DHZ< zaT@Wmc?}d$jKyc>tOi9w{%<5U`#%}5nr1X(p4crIEfw-I$0-lu8d;AtM?e#HOH^G- z=pf6b!Hfz9I#nsdDsWfqiR@@uLID;Ay}^2M25^~VJc%OX;kkS38=N&TSQrn9wXlv$u+p)zVM1PjIY6C>t|6b-%(F zeLN>jAVv13{ZU98gm&N@cf*)CK*hhw4!04^2k9#;a-APUWJY z`i1qrl=jeNvj@{U!WY*0v{zt{CBG&YrB{Hb;A~cf_&?GL$p{&sbeBT>a_rEvmWpjg z_z2gE+?dm&?L*QdA5D1Onw9&ZF9+Op3O-z16DJoo%i^Q1CvZDWi!TmOdtzQdJqG3S zd6oFJ(f*b5fc2CHY1u)3E3Gw@+8#=+_Pa-@Z!O)<4uQY<@{g`*MsP#REWPFR{vDCj zRyFb?L)AI@JKU&u6ZG}j3eTp^`2bNY^h61)XwDHQ;X5HXTZgb&M{`UZBP9G46fJp4 zk;_yiqtZ{%*Xmmkk7s(3^AzkZG8_gS1*p?<=_`q`Yq-Zkt22bc0u@FOeQNl+(IvA6 zsNq-jCAFsO7AP@pa<=l|jbLZE`ejWB0r7HC5GJiUU`S?@?qe0^;N!VcWGtpI-)=~9LwU6b)*V{A>BxpE z5kxk#Kt@q4#?6M(Oc2-Rv9X9|J?Lxn61+m>Mw8!hpp-{xcHw!V3iS25}jHR@5FD zBr&#|!42aWo$s~mP}=Xgqn^b_NaMuIges%rcqRr_+$xXBoJq)Y=!OzxP!uatXK^s3 z2otPEcCk0BRLVt76IC-yeH>KDWZL3gFY9s0WR#=&&ZN6gH?7LgT04V3ekvuNsj6p^ zCtw>2yuir-`*jt0VK7PZY-knuA-M{y)XO(AMt-IkSXS9o%ecjVK*Dt?q?Jot8#QL) zQi$h{;2C7k9GH0L!pZk8&M}RX?LCWK4+QYSIhcmWX=*rwUqtY*|i{k8_jd;I5BdzG&;5w=lk>08=! zI2w6zXqDva0)FXrsOdvyoX0*A@Y1Dt0x3-a>nnt2>`l>< z#sFr*caKZ~P5@);d>2*@z~D1WSQ2@&3)Qa))Ap&0Qm6I13rE>3=KbQf?g%q;U8Ws6 zDn>?bSl0~LN{d~3|IW1RWzH~XN589Ukxkb0f%JR&4da=K1_{YM19Sz5Q9! z`g5TvVlE=M2=fGV%-|*E;TA%c%+f1?ym$$D1bQhFAd_W#A!N`7IbMW-hFckAnMEyR zV(`Svzt|=jP!e|ItzvQ;RE}mv-lS2uL-}kIOx^iLxrOXz;+xC>yV?2p2b-A3zw+sa zTX{u3`(2Alo?18e{q2sai*i-eBBxNfce9A5OxBEnOUnX%Z*TqOoaF#%7J7Gm0A>WlM>(_QKsBJ&&8>zir-^BfyV}S{e1rLh4mQL7}MaN*c`C8 z%1CVZ1rL*%m?DRntsK=+;hnJ_|E-TMD|~aKO+nT;YA77rTGdLb7L;ID4vd^7jK|(6 zV}>E!_UN*!?u$-))5tL}OE!wH^1dX--j++CzLGQH~QGY5*ETKkLH%TlK;7KrmU zq1}{SXfie*YV*}uc2pfrCD~?0Wwmk5^j=i#|F=2Ir;xnM84dIjGZ~Hk=39^feBV$T z{%O||m(tyd3Y{}%e=yUV3jRZt0C|4YF?4HvBy&3c?M+s(k8w%#^ktAJ&L%Er=@?6? z*5DwGg~%+N;Z{17PzzW{jLs?+p)HPqcn!-d9-_!AzJa_2`vCZbuE|OGj^uD*ZfkVrl=}o?KVbe<=Js;dP z;XgjeBZ8+-?*}vdlw4perE1ETeAmL)2KTPo@#%M)V;LX{3cS#uYbO4QC^!lIz|y8I zhDI`}5iO8`t+4t~o=$%Rs&#?jb*7RDh1e#K0lZGejDBtwhzt!vFHZdX*2TI7?$K2+ z0Mm5dyZGV~|Cary|8@5AlzSHApBS>u%mH`+skMICR1-Y@3+jBZbsa5AuQdiH)%)$9 z0PLtSwO&U<0OS&ETFC$8$(8so_u9i_>iyS0xMXaH7W#Qf^#>e@u;+ z|1LrVU=!8_7LTd+;$KHlFLu%O0h>jGqvg0dU;E;1(e20IU3c(}HUD34?*S!OajgrF zO;ARf4wHLENMwU?yf%*47aRNeI$6o&oNA>wjz2I;&2dUAuPes{QS+st0jwUTk;X?Cx>& z->?WsBAKQ*MoFc={_YUiF|7kru(2b;s6fm%U0e!5+u!?--SJxg2!9M&XST8HTl1G) z>L-@RdkLfl)?n7n?Rbvz$HunA`44nsn&Z%xI-`SMl3ePKmA$*UW9KQf*yL}0qXV(g zNDmB#6oG{wpeL1G3)s}jgRxQcg+Cwd+&gJ*?`VJAIji&XKrFO%@Lpper9J|mr_x{P zb=uX@X9oQ}&xoctoWSyyV70=2$-e1)Fs*$Qs5Hc831EdKSQktReMbg_Jr4F~*SEby zOPMvZ>xw4&d%o2N&i?eJ)uRJ(ED$s}n%c(&sEfV1!#T}7Kwy+B*eI!icpCHzGTz~2 ze`*5ruryzPZ0Y^Ru5r!r@oXEU(odB1XW-)kaho~~Oz*hD8yoA1!Y5dl)aG^^l`i!d zbHW1))*Xcz8G&}}ZR~dJd1p~L17Iy&ABDo^&hBAvI6SX+yfD4A6 zUKl|O;u^egBLA}!?Nc86ixiylZBs+;S2ix&embXwT8K@TMme@j!dN9nT0uCsGz$5g z5`%uOWF4Ol?*3{+XL>&Ci3Z?i0YIu^nUzsnX(XeDeg!2aj%Ri3mCpTl&R=aWi{t@(s|l&I8mQuuygs{5 z&}xETmML?q#gj;V9qFNZZJavRQiY!bQu3^)8w)yDnqHn*VF5mc?>e}1*mV?yJ2nhH z{K3xg_k`eU|DQJRKk@Nz=B@ZzV;3C#!x?@5e*f~!VgruG>kpLCOEDzLO)ST zg;z2HeoIR)`ubkqBpc)Z($s@bKb^hg`r7t9wuI>TqnUkEo>+aerz;IgD#Q(svoP`v zU)Pi;S7V4YmR%F~7Mp^zDDvHDy%4+6-HHABtw1*&Aj;_vOKBEg3viCk1hS(YwO-#b7alIbCd7yWjTias19)Xo%zdu`^vhB_xA!Nos!Yt@K7I6 z;FhOX{OZ+p>9e{B?`Cw3Z)!)@x&FbvF-`3^KDuP!mFE~IkOA)L;o0G97OVm&#e6AD{Cn~u@nO8>YCGi!^3^4jOWtdHBaxj?Zq`fLjWbu zf=~sj{@tfn!1(OhT|hL4#+Qk4w{+a{6#dgtg@4_6?ZbWeIwBAcZyRiC+W}sD=j;8h zIX!e$CAs`afBf2ci=pkCk1ZSJkK4qBmCS-XH()9K@tdEZrMzxl-$>+L&gG|fUiZ*q z!1UK2U-rA#`*ZK@8H;rRf`je8xMs_7Lhw{}mwofYOEch;X=3#=S~{_mzj}==C19PA zaSR7za8|hewf-CD^#R8n?52f&iU6T*BCJOSs$Z~m@2QV(ePPvA_xIWW%`>}hc%%=T z;d@W6$YKB06(AdK`O*9p|NinCK>5V`dnPkUv`?PXbK{)FHW(S`xTdcC8~*OAYT|c2 zu^N0c100J51-Q#Mo4RlGbh;`CyKnM#eZ}2zb7NQjQ)|B6)Pqn`%n5azKJR&A_4lVO z1_Xb%x#voa%&AyITmT|}Fl{j~5dnpcldSh7XaTBz+uw8K^e10+cm8JH@?Xze0c_1> zunV221B-wCz>42JxB{^7b(mD&0iBcBCl@}=Tk(fSS6y92kOCHOn?eY_90*=)WEUI% zHfzbv?k;EpzfLQNe6P6|aB)+8=T(*QZ~MCQ%3{#)v)M~ee)Ny37uJ8vN3#He$NNvr zetX-E^__+D*IZxULH{LcO+1HPU)ahbUz^!`*RyNCGh^`tfH=F^$gGZiEzo^!L&rCp zd!|0K<|coavow+imR%hK0bjfSZ2hYn$CX7~9{Q(!QRx1Jp?MDf%adwxGh@PcHF?j) z?!B6xehs2EidRiIR|@CKf;x79BR>>fcl?3(c8tBN z?V6f6(ERMh|6IKFWXUsYoduw}2G}dIcUob1#@k!A95@HzeJ3v7{rD=h5WQu}iOUy; zXa8mEKmPN-HXk@w{`|Vssi7O{+Sl(n+q3!PhJ9zh?eD^>=ad?6bRXDu=%etOW4n)C zsC{|8qcAiX@LdpYd1K4&;}^qgk8MA6{&`~$UBbv2acl9KRuq10(ZIeFAIJOYKlkd~ zIQZVuLy$xgs(7J)*Rcz+{^MB4qo+T4`28Iao>UdX*B?ylg#jyfehB$@mmJ2pzkO&q zh|HDHTp3OC5g`I{jcbZO)w6H&(etgFPV5}K*xb5fG{t0xqq#l4ZE)G%Goei9 zq;n5QG3Yk5t=fNZ=2e;pG0DS&dk|ea9!`fsg?WBbnQHqGn@Vjdu((3 zkoQqhJb2<0K^C5^v?&b*mq_|cqgDAEPla;W1EkhU$p7?;ytJL zocj2O&#V9sO`C}n8}q!s6^W;H)A6mx&SSmk@_J?m&pph3I)z-}i#?5{JjRyt#Kor8 z?O4k6>2#ieyVf5**S6`z;$5eApCrxD01p217gj@I){M?8=-;sI-f;M%AJ1O_FToqJ zZ6}xSITPHZd9pd4hVzX#`fz;r_7fKtZ9Kl?1m(ifz9`5tHVjr1XEnm>8J*~U=HvhD z9yoQ+pVwdRF>>d0BL~Bi{Rhw8FmEwNPoLSjX#H_mYiu3-d;ihp@Iz?lrH#g=ZexzI z>%Fx{N4bsh-q_C{UA`APe8V6DaR1=PI2rS^rlTO#w(97dzik7P#Uc3Kzir!n>>}c; zbK{9)XFm13z7eE*aupa-xagU+=!hslNWQo9aQ~h&*VVM=l!1wx&@r(xijcYVcOTr? z(1Ad97KBFKxrjKBhB=5t;OsZNUBFNngGk(U^g?j;u{IKO^?T0(Og>nC zWZmvF)i3q~XI!Pm&F;?bO(*fSbHm`agCG5T&Qi$R3BflYLPy`(`uM_uokuQwy|F8| z!~mPdtBCit$GbKTE?9CHIO8fdo?!n3TV+>mnXyMXdX7Y^79LWJNIVo@Oj#6;hFo?MOXg2t<=Vkgf2w|x%e_a0;_bo*)SAC3vZCX2k_0q;@^*ft)4qmusUN1oK z$Iq-h{?R{v_Tp+BoDqi`51;#B!|}2H*l2&OYv9xys}5h{F#r?4czzY*;Dq4UKe!lO zuA193z6s~$c7*x|8;_4|ijDEbx_45S%NmW#8;!>Iw*mpjcw;H$ZQq^09PorNrdyHe zgq|G436F?W(iwk*HyYk@Vs>QU%kD__%r3nC;ZOa50Fby8k1;R48|VFuSp;o&{(0Sr z^Z)qjBYku{_s4-j3z-3cA+mY(;V-$3G;uHAVW(d^J60V$d(Z3rW9kjM-)|QA1Det0 z$Iq_(umAq%U4QQXk|*-LCzt*6pZ|mg!10*o_~P9kKD~I~mm7@{zF6hEo0shV5T|}P z?N_g^-T&do(0rN408sw)g;kgPqIviCfM5UQ`IRYUiyrFS?OnL-l177HIB*1?&=Loc z2bg~MspU|fPE$6*TLm9GH+i~Io4i+huZIkcs zK^LSwbaBe%bI#}(?TP;8ys<~`@jBK{M_2{Q-eRAxfp@)+lN-r z-C$uD#AV)s9jQ};2$}oe+y2of{~P>z{FGqbA2%F2^)YB!W^v>ucPBu~{ptqDBTl3J z#|O{-17V&+H!V?I!xl|Q#GtssnKvta2VVRuzGDMrE{n^U0DI*Nra;=Q7@% zT_>nCfRtUTL9q{F<@GnaW+4QpPxWye?on|uG2l4y_gX@5Bd@{`&k9-BUsVfI z4J+nXRhKPT410c%cEEev$KTVIQ5?SOv6VpKNieZ2GEs?@N7D;K552!*=aKWzNA~<; z?vnJP&_sQ>3M047m{b`>-@6}Q>C6x1mK)d9#P%G&@QeGGVhHS&)TwRnFFU+p|G5X> z-~Li;@BHAd6K6jKQX}Ed{_EDmr$2dY(ax7+djYb0j$eEwzIV(Y7EP{-=9Q81$<6*Y#Fu(I9JLI{3ILj-vDaQkjh-R#+&DRtrZ)*XGWZ+}W% zC~aE%s{LmQ-`+5y(SVrugRSxHCw_rbT>tto1 z&PoeQEz>$idW`Uv!P()RUuplRjsE7FW13r__hjX9Dbu z&Yh?Hi?)5aA>wQyn~UDqaJe^nX?@uDJ{d5|7jvSq19e}v2 zd*F1}!0F~i+pe6`J+_&!4AAq^(gOg&hdOqD>+z+d>7GZ-(Ke$UcBTg6uGwAS``?+r z3|@84#IeJOZX28(*_qN19_5eT{`$H#2hOI?=s+&h;HBa>C{JACj`-fE)kiM*QhoTDo_%{xe|%qb;Mv9d z=EZj%{O}VXFA{A!{TjiRuZ0l&*DtTxaQIwipgqkSxBKHcGdp)7Jv_YvxSZbH4lga< z^I`3~o4;Ha&S~zrv?koSy*BpS>7_ z@>*{P>KTBA&^sQOjI4&6CilooF%}` zt%p9E`Sw%b2DZ-~b5DZc#eN`eN5V*n8~Ur<3RQjBARgPHV?m7+psCqX1SA&)Bv>;NYbm z<8Nz^;QU_ow@ou6JH3mx1zNY?^ax$Bfa}2|@JhPPP=-zGxZFz!K8t}4a4My&^|8*~ zps1I5iI?^4I$ihot(0bvguF)YuG8-Kw*tZK%z&adH;nW~FK;lo%|Js(i@v*O>1r+*AzoOc)EArw&NEH-r9g;gp+QHShfIy z`(r@xfs+@%_2`mO9Kn&aHlIMAK=KC3|$Y$+e_+?!I3rR;!nu#`{s?gfHFGPLCcoC$Cw1b=1SYVh$H z9V2KdS8v69r_-$t5PS!L>~dNWcycf0!cmO|90gCtwhoT>#gG`D>fXEQ$Vc$N%QnNvuDIw82P>-d>ZA%?)e;%*{m&xLjacCW6AZ9VuA zh#*FRFc2JO*T1|T2+lwK{^8}RQ-VN@?BdAvwe0{6+mv9{^Zj7o0D<4lTW-HQcx_c2 zswm!<9~t?(h3I(sZx=orWbr;Fm|J38UlZSR?81+y^=1@=CznT^Q-XAn@b2~rceQ2Y zhYFrty>9oJiFNTy^IQMexo^R$BPj)~lk4Lt_bht3i(bbThk)RJU3%zGy$3GKZv$$7 zZ*Je7!Hd^4wr5rv>D9)=Z3DN>Tk0$c=T#6Qi@T9ZUJMt5X}TDZc%5C#+AED5iR0A5 zx^Oh77P?f4rYK0w)v%aDcAi#5OPRH(F7hifAtCsBI)%aGNYTG~U>S(>HyS%9s_u+m z159y*nfm0al-u5$-?|GM2rxZS8_`t~0SDiFfN}7G5V~Aj6UUPLuenRp7=ouxX{&pA z{f2#Kuc?ZSxP9RrkE{T9xT-n^eE)uP&+cOvAvxmr3%};+Xy0(+r}y?@?b%hh$TT=f z{rZ7rr_cY#osX`-(dpl2^`RbGa}4ds;htAFuz$=ff^oqzFv>lTuT0AiLAkN+I1Xqa z4rqTSLjMnxA3eQdj4uj`WT!teA*zeuEPivtfe%0V)?-Vq@W<|WeI3~G_vSAj5UpC>$oHe5}r6%~xmsSIvGuYkhmG^ev{>QZ$Egh*0#QF6OAb=zI{n1@O znY=k&m4DlG_{=A_KDQG20MI-$v}5zp^H<&9b463EcEOfIXFj?5{$6MU?vMd*Zyf21 zB5n8=k^#iQTLim@!1$&(P`>`}TQQe3e#U3qpK!;+@Z#77Lh$DJgy#5?JsZ`%f$bF53yg+c7l6F7p_eIA2O6k2_wYrMzYS^07@Zc!M;DN%NS- z=$)^wTZ`NjRhMzq}zbITgw+G0c>W{l-bb*8a zmKMG-xAM%i|!nzweG-H!GZ7L=&sg{$4A0iU+lJ*R)gCj zUU0+&x#`(-vgM7 zhK|Ks=zmEF{tE=3Xax$y8XCV7=>AL3L4+{AUTGB|1TO*y?*s?Gxq%RT{9SDyEI<5w zcsEjob87HU?fU?r*Ve=!43D9^4#FoHh6*|i@ij4s(l;Ws+>zq9xtAlN=7=zDD= zAUOZY)gPY!k6+)v6g)b!AS~kjkri3Rky}~3V`^zrLwO|zNIDSw?nhVV6h}CM$NP^z z_}&g6czQvo@X6KvyU$LnirPxUzj}E2@pJ$9)e|ezOCz_E)4JBGFVhSObf*#iceOGKX?} z&8ihZubx@;67(xHxECCp{0OLuuzEx=Zt*HHn#qW-s1vkBR7VBqSPURIi0$mZZW(`f zD_)Tj+g2TgcPgG=XP?@3Wd(IXdwN08|N6#VM=mVcdJ@U`509+`bY95~DB}no#@esg zar(}Om!}a0h~SL2e(#yv?(Kt5`AH3+j2(uJ*tg~6z>y0$IssuPRT|FxVA*qPw^RG6 zfg|VVw+&)RaH>kMbIqc_MWSwtTSR*jRsbj5qqzXI8A*cP49E2l^sJ2Toj^ zG^ZO0z}6H;mwW!)KhkGh?v6CPx8;NN$M}c{{)ghb0ggQbrxCY>Z>=Bai-OLBOYZ;h zldJFR9uG#zOd_kMwL`;Os}Ao!{YiA&Nw70e>7PHpY9u$lISv$lVex)g3mDya_@iIG zyc$NL`-B#Y&Gqk*&7*yW(v0pS8;yqdw*cv0?mJ*>rkfud|H=kMlEBn5W^_zwj-eRa z2Jd;3w1F75v~K_Fnj>(|h{j0C+ZzF+$RC(8{P5noqw5c!gNNHTpK#6YL?aFgbB%!= zCoaHvQ0ZHq2H-}4Wi(I=G7Whi+4OsQuv z51fJ+jE7-U-|XLd>^%40qwTxlF6eOt$avdywwlvO;oPWP%A@Czn`tR$w8P*9>yCmf zU&?M&e)Gzjo`F;33a@eN^DB|3amoSc=S=ThcktW~=P$#O!7c4OPUXM04jO1F-({YJ z|EIU(>HGWEZAUM3?>xP8@M7SDt>e5=bO^bjF*cq_ALj`eYd$gyEu$M z`(JaH>^O3M?d~&uTTi{(z84VzSkEbmObqiW1)y78N)B1YXzL(nfjJ&%wvm$cK%pROku5rgR%Ojc9#@an+ zn%~+ywxBJyE|y(mWYqz$jT}(yY9p(L>Y&?MwFW-9Dp-4!c=AuwSjAqMFe-Gr%NkR1 z*?k4!HRGGy(Xvs0)f;(V9L`(t3Y6ztfr&+shrj0O{Q13o$fEH3M43_0jx*hD zbC!I=+le~U5D}J_nWH%JgE!W<|8({e&xR0o8!5&Iy1l-o9G@h zb6R`u%uZ?##M7JGC(h_}`biAOp&OO9qcGgqs;C3;r!*< zGdo8zz_l~tHfc^5x5TCyhVD5tI7%%2 zb*954c^ju$VBqIX=>3TytN~kDgi% zGse(O4w(dbz%Cy*m4H5q>^14WZX0tE^k;&nvmvlpmG^esbM>HC6*Y}@X23k!^$(K72haX@Dmx1wlSjuKv%Cr_X3Y&;C5Xao&>B*bk zVfV!#Gm-vd`gVMBddYYoIJV`D^LoGg=+g1NXnIR~)=Zi^CB^9-7|!8~IlVEy*?tcspj_inb|Aa!ToZ3|fu$pUhm0b{t_rm?l?Hyqkfn z6%qI)R-6&>E&u^&K(G$l#Gv__%4lwJWMXNAfp72!&D}q0=>gPUQx(0cLVXJ1(+``w zZ=JE&RUFPL451O9t|~XKua2RfQ4suELkH^kI zP8?DjQ{~bc=~XScRKgfR5a|K++|n@L7NGgqXJ86AF-_!0iZW@TR!EEy3aG_E+2~$I zl^m2}b9-T5x|mwHERAkL;^Vx=cu=I~_yiye{ihi*dz0nqXo{r)p?tA%jYg_Jmc}BM zeWfloR)LHy5Szfd%X-w|7u_fLqa@Gktc!!8SSr0nZHoVbI|pK8yaw7G)8ck67GT|* zW40E~g@qnGQ~gl{HjJO(i>3*N9#h5H62HP99SgSDgnA6dIHb+c&$KZ+^~ts{Y;7Sl zNrhUVs&a_v0wju!XL=xvSJFnd#lV`OHkHZaHMJ!+p0~F!;c4-4egh7KLWi=D%@-#C znlIDLglX2?biII+z*b8S%F>F^miE2}{s8JwBm1I=0aOS;larB%lnC6Ktk| zK+hp&^G2O)P(JYqY}L51O{@$16o}H2HOINzQ+?4?xfZnC!p<1bmiC5E_==|1OiPLE zl@?&zqa~)C=_KThisgVMKB{BNFbPkfJP8n{$DCLoFY_A%^wa>Y2-{#$JM3p-umP3; zP-w#=h}izmEvF5SdJeB6LQBI4PZc`|KpqsCMIjt~|I%{+5p+#;47~X5?gO@aTCZfW zmRD*P7OLSq1_(L=D168h0tn=g>V>Fu;T52PxGp8I&MXX}Oe{ltIG4Z#mZ(p`%z|JZ ztMe|o#bIdReF4p+s#LrZ=obWY>1)JQ#71P=@|a{!QHWa$e0MPoTpQIfE*u3?TxDcd zgq%7oFG<>8#F2*=#CS|h6peIwLDg}R_Zl7f0elD#sD zjmDe`p^8gp)iE^3K$T=eHc|2jmQNzMT)af=HqSl@Nj0pnQdG)Jc?tT$FY1%2#44#2 z=4m8dYZt0CWS5>TC484BH6foTR+ovDg-tU|wVLrI+E``sYe_g)lc$|cV&Tj>f)WB> z8B{#rMI&!-Vz>~9De&dEN#G(I1++Dr6<&7$`+O9QDrg;CXF&GVQiY9yk&Pjj(64M@ zSf*N}ayF2Zv}wkxjy9>>p%0p1b3hm@rfi*s(q>bB;DD%^%`G=4F6~KW2ySb(6mGe2 zhPtU5j&pHL5*e!li;vJBv@d0{y0#V$A|}b)8oIs?m}#noRK0hj4HX zs6%SP=zP$C`BkD!Tup(s$JKnT@ETWmq@$!lcFQLd>Ec{Q4X37~`r2gOrny@7jcYI@ z8trl*8)L+3Ere#(o#zRjKO~f^vFyt80dwM1npuUJZTFiis~pZRXE#}-Yn3Z z25pg}$jt!oO*B$YnSbJa9X@SY84#nzrp-}V#neWzQ5`if7UJv^ zT@YXlbm9s<`BksN>r|o8ities7W!Mn(5je-Am@@Gz+g)uXyCL3C_kS_qk18xumk| zbyNu4C4vKm`D${jZ0xH@46>sb8e(3JRrt+(0~8nAl}BsZ1}@fE7%9W>+D)eOM}h%@TV0lHj3c2FT_Mz}#E@AV&8iU{1$+|?H&TX$Md?1ytX-uRmzfZ+iMh01 z7Rlly&Mu!yxxA@J8YOwqgjaoNRVg06Z|P9lhVlTdR#_yiA)1I90gX5LMVL{P`Yk+8 z@E{&<9CMpY4Gu-tBx)0Z^2vbonYn=@i#c4AjOu`d5q5fPExKEoHje}n`vuOxDvO+P#WzB%$Tn64D0n-H19F{^U{A+?&I1NR%*V+@d9-6avi| zh({O)FAck7;RhV?S|VRri@e*?D*PInKT9pbmhPH7ubW>>!)kO@&t<!UpX=fhHS>nz<_B{1LDyH+%V`sT>FSub=@w1#@(PW$crUd4Sm_u4siN ze*Q_`3~(pNwMdnT)27{`2y9{=a9BJXO%(!Bb`h*_lP#Q2Izdax-11bJHQB03ZA>|h z;*HAeXb!|uNL0I`iJFcQ-XH-d>+Wdedq%*qww)OAdc;K3PJsQ=B-0gQT(9t+@)h;s zo6yH>_D1D7k4@PhO>;+cWoWpP$`J8YVu6b^ec`b#fJQp z(2~?EViCq6Lm)W2NGWCa9*kwPN|?W9vl|jT`73Y9E+-_(QCB5pVNSaFD}j4?IIEI= zI)+Pg%EBZ-y&_5v|%vW>eSvev)xL!B^X)TB|#H~ zT>|JtUS75cOxLevL>Hgdq9LrwQZ?WfG}$i@-0s)u8^2zN+T`EE=)_zfRHB;&g#~*0 zOnJ^vm6*UjGapFu>$g~F5`UG$)IEtww1unf34}MGO-ju@5ZBo&pBmjbvx{j{t@XfU z`d>fO@Iz7{CUOkttzKbdsyc=6ZJ_S~VRMvEN>0$6KBv{{hR4i;(&0iwK;%DeF#kJC zAtn!V>Y%39jyH;h_6h1uH!#vy;N3_7blKuHTf`ZNH?kJGV=Wal&=L^|!J-1cBoQSN z;?W}ixn;N`d%`Q{$jOXey-F&b%&e13G65foaSJq8*sVTs0L`j2ftdJsRYcsVXb8@) zuoasIKoe54YjMWkW%n2Q2qORh8!|~mK~yS3puFPU?1yDua-<%H?ED6XqkkmD!mo9< ztMO=^O1ly`xT3}?Z|l9(Cr!q}^b?4)CZn{u^iebduAdSTbS9991j;Zp-C&-@f@N2R zogO2#NwOQsrW1QR%m(0R*z z{gUBoTv(o6wA&b;a*o_oUZINpvMRgyqYCRqm!_DSIPb+jv02Bs622ekzy|@3<|wZ$ zKFPrwg#)=Tw-#VDT=-amr6lQWU78rPUzDWlq&tD97i$6u_2Qwr55uNEqkJ|8*U9_P z0!1Lpk<4l;j9-TdyE!YbVuk9gc2oLJd~2+ zL}ScK@eoGpgC^UMJhg=G8PyS+FPa(<+ug>Bou6|X4TsmTv*Ktp1k!OR@rwc(d=+;B zkYr+ZzbPOPO{=h=GVBbyQ4xBiay6EnnL%P&-N9P|(h};(43KJV=G}{&kusGjG41gQ z&8BSui?BTDYZLKVB?B_|_++D1K&1z}ylCQsie6qrPC*tv5oTnn^p&0aN^?G@to&y4 zTLQfm)FNV3qjeJE6J`1iVRZbji07ZCP%aLpaxkac(GpoW` zl>%_l4j<-}g$TGyg$#*Ru~NwjaJ))jH2=z=Ih;`wQF=sCN-Ee8Ofr-HnlRb9Mbhsj z*2d%(y;n(+o?;73CkW&fQArdSQpx8PS5hxmu_Y%Znv{@KO2`sdStz~UFe$dY;uP@h z6%at1-X~ub5mqf;apy>dg$FNobqm>OI208UEJ+F^7iES7adn%i=$HUjKZhHGYZJ>Sbf1YV8h9B>TY{02zOi}vbyO0E34QvyfO>U$3=`QkFjB1Ct}=~o zp`zOXZOZzrxr=o3IGGg!l0~XCBfX)6sX1%SW|ibDImu^1SUvkH{_fCnlIm!)_9gUe z6qeJj@N)RJtdYnH0gGC5u^Jgbdu5Inxm?m}wjz8V3Pkow|7eFQYVnzCr?wos`;HP= zOOv<{oifdE#OH?+-UwV!p^?G?Ir&{WwgnjashVBsLrHx1H z;al<(iHH??vxP?wZsl>Y7T97=JdVRiV|*&INf3! z*nCNA#7BWWKDkU@^U1eXnmdAxWO~(WCI)>MDSK>mogW^l>025MIu z)gkq;9m|TBb)k+1?}+2S2$qr|&aW;V0fxBW2qs3Nh}@tlpjtkw9%T3!)IPCiVKA?d zeU%kn&t)a=|5+(wMSEosOrQ+^h+A)ls**kfj)B*q>@wnrnbi_@OS!!HL?eMd`$WCy z$>}+3XG5Sdt1=|&yf4cKDQ1RDzGjkOH#8_G+$AM}=@7{vX@TjVI7mSDVfrQ{19P7x z>?f%w;j50_iUQ!gm5yf~6vY+j+?<`s@rt^=;wnK77%zBj~* zxaGQd+4g8ma<$v^F!zJ7ifME5H4>);sn#7)Tha8nwpgms>9K4@d!ziynK;|AjD}lO z@qHq!Qd4jXNpx|Q$as-4XpPyB8M(zWT8oA%)ZWI*Fal<2%PkyAtgua$yh5(3Ef0kd zyiJu{t^P&Ylf}NEq$mZfXsrfsGM9q7%9N&s9~AJ_7UHNvylxHexhz=4v7 z?Fu^a%J;e_>}18kiV_wbm6Ap?0?n@y;~aV{DZuhF!9a?I)`= z1GKzCnr&VUMll^NZRiSlR4XhtVb`ZHG3;=M^&DXHYH;jJaH9Y|9o)I3oSn6oPxnk% z;ZXwgYjebu!@Py2?+uH!^M*~VY_yK$4T}}ACKZ)3Rd#V`qlk7k5wSp|nQ{iVLHOSz zQiwjH#s*||@tRO;*Zjs|nr|>x0iK4XL&A1b8{Z_BRigyv7MYtTMfBxaExnHKE8abv zR!@cQ2F1?dvyk>MZ8_z1DPhKx7WFEkO&s6|FeE1%61y$1t^s%9P%RuHIdDq^&()Ck z3xe`z1N8s+YeC{yJb}~#rWrajcnxNi(~}`G6#?JuQ#OAsp^EH9N;uG?F{^@vvw0Dh zFw~L2lPv^EGVCT*KyU$hMdDIP)QhwVU!Fp_gxXhEQk^T&D=uqCi%e3ZOV_4t3$cL; zO-mAtwA9J7NuP>E$MhPB(VS;VN+M#Zky=c4qtP6leTz`k=#Sam%i6Uroam`RRZOZD zN5Uv>OC_MRSE2y>Dyq1B8*fIoE>|1RWyT`2X zUJm6q?!yN1p3Q5{PR&bbYlz@P2pN6vIqM?{Ekb{zIC-E~T3wjd&TXn9q^=E{2S-V? zolKY%s?A@|#Om4VlT0|HI>dS9u%%E$lO>Q@xD6B0ZRXb@>TRN45X{B6qG$rDz0efk zL|A{#)v0lTS#){5(elbh$)l1e`58*|0$?SCC3cKwFsn?`=MoOb4BvUZERhTMO_Nn#BUmW@DR0Mx;~O zE^{o$Um;VVEwDgj(C{{;N)2Hmd!dQot|HS;6UwESe*RL)iDU}eFk9?ezSl+6$=z?h zYSE$sQhjBSV|FJ2E0pV>ltNphrI<+M7NBEx;h@kERt8X@f{`&*LXsQVkl4%SNyKzW zI(dXA4PgO64S*75cm@&>1~bBDFEP`w@`IY4uuac!2_AM^@W>p_&H5^wmF+fl>USr4 zSoy@pdnIILx9Q}BRkKO+_&}BsHPp7tY|2^)G}(nim@kt6L0(asz1B6A#_)D^yCQrA zn#`7%Rix{9wrH>v-EwDL*eQB(W5Q}n;1f#*gyRI+*Z|B(yrOP*(<|>wX4eV0XRAi? zCaUUTXhvllKQYV3Hc^V`DT0o*OXRhpw%({$l1n}-1xa0!N?sC_4=H0oWTrV!4w1*| zc?H^QK~hDR%$M5oMY{lKQ13DU%7niqLBeYSZtw%+iF`&?h~iv=h>jXM8%hM71PVr5pGh__xKZrYlNhn~WW3Pe+mv(*4GL3iI(CRb zBMQM2WE>tX8+k0a18&8X3uJ*3(Or(1h_DLL8ixQNg_3rU1qwNY3C3-r(on2iI@vx< zg0xGFHd%AByAl$o+q#<4JaKx^^K_$Td@?In&*TvkMt5r$+-{j9pq9a+PVnss2M$T< z9d3&&Yz#X2d4(ah+*%q@aPYf+t33q4Y;QQUyy#?&1l;vzg4&vkJ!TW?l3bf~O~V8W=od%^R-h+Iz}5NZkNoRxF7&ShAy5Gz5&({rT46rDjQVy-F$ z8iFU61i&q{!0{0B$)G>fIpA}?4k?_3;BYj2M}IG<7#Tk~HkhxxIIQG40akcCsMr%4 zR7ks%;=F-jsKP@_~$Mi;RkVVQ_kyG_|^f`reU95 zN5kQYlCdyx6D5@^OyyoPr3rX)TL+a{kZkstN(5uA7t_%>xaG8AT|41+-B%t<7jlay z(WNH%GNh?{Mw=0r7sordoVc2l%Et}bS)@VRGx}C+ z;CqiT@p^d`Bhn1j#YGCszI<1w)iA2glmIVs3x+}f{UENtO2Bm`Bl2go=s24UwL>;0 z;^r_woIi{}ev&}GwXtCMc2>#DPZZ0Om7F4WC0)etQbHzc=4cMyT`Dd;fXf+`>@Edx z%o_0ua+zJi5&+Y{3xn@aCci@ApI8@+LJBKF;>M;lltSoS(56ZbN@1JUBEROdqDkgj zUtM}HC0uBg(FZCT_LHV0m|)(}!*QEcq1PbSkXaTId%d2{kM;&C4!&z~egrzyiAHL# z4~tiK*lgzrQcvb+X^BIl)b1 ziQ-XikBEh6Pzr^6^(C;OFJ%|5P>jykm+u6&c4VsQQjg5O%xpEHv`ws_gX!TOAXRld zyNFHYXpj}Q{jh1dpQSg*pG<9pFN(8HUY*tP9Yq3ZH6dr6*!JR+tg}0oNP2ZBt(qyX z3ujb>663l+3w*tz7YPWtT4Qd0l32dH9+6-5*W9VYzfQG2r%G*8H33zUyCf(~()K?q zK<%W)oB{^2#YDPMP|0T341i|9H|kj>p-dv${7C|+0H8D3jW!`T*mqGIdj>PmSP~MH zyhs#E1IQ4v!iPX51=yF9gslXKa$(fhByeXfLnE&StjUOdp6~Udg2(}QVmO4#sLs*9%0jxSwbLGrDu2AI_Wd=td6VW8b3ESM}C_>(Cc2N~G z1?oeqPj1UhZCc%8gQh`&K4nMQ-KOUrcjD?;@>!C+qh6~tSp1;8r>3k=Fku^*>*Chr zmH+vxG>Nx36jC=>Ae)x5Hl$WfpNmZjpq}ax?&8Xw>e^cExB9TtWEE!4NZ6%1)v&X~ zK-;NS3QHNmyG1?ZWbjQ_;WZ&k=;^3kkOqYFpR>S1clbLp#!uk z*@dRTE{j+zT%d;&cCfu(D|(z(QeHI0u<^sxJ-areNchj zEJSNV2q6~pT5#%Etb>RG0xnidnr#hQ*js06(_z|ef%Qb}cAMbHE2$oH%C-jCE@4Pb z)+SS+5K|f?cBkqH=&@kB4wS+?tCtcpf!k_r!kX4=PEJaeX2#MzAP*3!${})P75*k< z0)0e>%%B2@wKdWQ(k-#6ou&;ck|jS(Fd2p5`ng1(ppNotqnf9~dTl@OI>U6Q#?AEr z0a92vR{ju`VbOe;l0eUgEd>(sCHm`-?Mi_Dsg}AbE>=u*)(QJmsWV?|{9O3{48E@h z%_m^xsGTWGMmZG@$9NKUm(nwC~W1v!hFXr`PN)^5O964ZlAp|VY~a>PU+JIfbyOvjtSczD1V?SrR}CR{opP`6d4skn zY{hAswWceO$nol|5w;26)>21@b))jE?-1QJ!?a!Mtku?~KIE(x?Q95&6slC4VUk`E zbWIhJYes9n2tB<;|3vLmTX{!OnQ9fAs#TIDlfP;$xl`p+Ysr;jdfn2JZ)wlfC8xCJ zPDy-q4W*MRQ_KqQrDXJWj;Fxf%u<2QnI#;5c_YJRATo#VB5LPdGRlLz&V?aHP+lxL zW-}%(v=xg&9Sd7|CB?)+4RiTxb^$92#PhAVsh8mz^OdSL{gW#}MnZ6QL7NWN3D`X( zYI7rbBPPiGEflk!6CRo05Ok$>L>z8I>g@Ak= z|dO>U!a7XLT?!epS#- zwZyN&<%GaTV-70{Jr%y2Ux&i$oP?GyR&pi&eomRf%5blGdr~EL%4aFz2IS;h5f+GC zBuI2dDZx77c!>$b)B@lxXv-{?jjRjFl*;RrQu+$boR5q4LILhvVG6%?upqd?ae z76NWv!W>OrUrsYQ7l@Th&faFDj>}2tuhV%;p3tGZur? zX{Ze%bZr{lGwEu5NaPVjYCJ5*w*3z|06Ss+jNL4MVq zL5|VN^Nu>*BysZ4k=?H8*XXTvA}nDD{gV8e6K^n$wCPjRX?}w?VqMVW2X&!@Z6Lf_ z7j&wOE~zvjxLjGrrh^W~5*&3_|ES!oANaHtOxUQvWoM1tX;P11+0`^CePx>juK;Mp z`ZZ){l)qLipNeH2qg@Qa6Jd9%HPVw=_K@6My(Yt`6(4oZE5Ldv1lOV5lu_K7^W`ZT z;A%sQ1fV^0r Date: Tue, 3 Jun 2025 14:34:13 +0200 Subject: [PATCH 17/49] feat: change text (#35130) Co-authored-by: Bulat Safiullin --- .../site/content/en/case-studies/akvelon.md | 19 +++++-------------- 1 file changed, 5 insertions(+), 14 deletions(-) diff --git a/website/www/site/content/en/case-studies/akvelon.md b/website/www/site/content/en/case-studies/akvelon.md index 7ba691869664..946603d78f50 100644 --- a/website/www/site/content/en/case-studies/akvelon.md +++ b/website/www/site/content/en/case-studies/akvelon.md @@ -77,7 +77,7 @@ Akvelon designed and implemented a **Dataflow Flex Template** using Apache Beam
-### Key features +### Key Features - **Seamless integration with Protegrity UDFs**, enabling native tokenization directly within Beam transforms without requiring external service orchestration - **Support for multiple data formats** such as CSV, JSON, Parquet, allowing flexible deployment across diverse data pipelines - **Stateful processing with `DoFn` and timers**, which improves streaming reliability and reduces overall pipeline latency @@ -146,16 +146,7 @@ The project included prototyping, test infrastructure, and Salesforce plugin pip - Delivered **rapid development cycles** with standardized plugin configurations - Successfully processed **5 million records** in end-to-end tests for **batch and streaming** -## Technology Stack - -- Apache Beam -- Google Cloud Dataflow -- Protegrity Data Protection Platform -- CDAP (Cloud Data Fusion) -- BigQuery -- Salesforce, Zendesk, HubSpot, ServiceNow plugins - -## Final words +## Final Words Akvelon’s contributions to Apache Beam-based solutions - from advanced tokenization for Protegrity and its enterprise customers to enabling plugin interoperability through the CDAP Connector - demonstrate the value of open-source, cloud-native data engineering. By delivering reusable and secure components, Akvelon supports enterprises in modernizing and unifying their data infrastructure. @@ -167,11 +158,11 @@ Akvelon’s contributions to Apache Beam-based solutions - from advanced tokeniz Akvelon guides enterprises through digital transformation on Google Cloud - applying deep expertise in data engineering, AI/ML, cloud infrastructure, and custom application development to design, deploy, and scale modern workloads. -At Akvelon, we’ve built a long-standing partnership with Google Cloud—helping software-driven organizations implement, migrate, modernize, automate, and optimize their systems while making the most of cloud technologies. +At Akvelon, we’ve built a long-standing partnership with Google Cloud - helping software-driven organizations implement, migrate, modernize, automate, and optimize their systems while making the most of cloud technologies. As a **Google Cloud Service** and **Build Partner**, we contribute actively to the ecosystem: -- Contributing code and guidance to **Apache Beam**—including Playground, Tour of Beam, and the Duet AI training set -- Improving project infrastructure and supporting the Apache Beam community—now with an official Apache Beam Committer on our team +- Contributing code and guidance to **Apache Beam** - including [Playground](https://github.com/apache/beam/blob/master/playground/README.md), [Tour of Beam](https://github.com/apache/beam/blob/master/learning/tour-of-beam/README.md), and the [Duet AI training set](https://github.com/apache/beam/blob/master/learning/prompts/README.md) +- Improving project infrastructure and supporting the Apache Beam community - now with an official Apache Beam Committer on our team Backed by deep expertise in data engineering, AI/ML, cloud architecture, and application development, our engineers deliver reusable, secure, and production-ready solutions on Google Cloud for enterprises worldwide. From a1df136a5b2ca572eacb956d92568b2167770acc Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Tue, 3 Jun 2025 10:56:01 -0400 Subject: [PATCH 18/49] Update Custom Remote Inference example to use RemoteModelHandler (#35066) * Update Custom Remote Inference example to use RemoteModelHandler * restore old kernel metadata * Update examples/notebooks/beam-ml/custom_remote_inference.ipynb Co-authored-by: Danny McCormick * Add DLQ --------- Co-authored-by: Danny McCormick --- .../beam-ml/custom_remote_inference.ipynb | 360 +++++++++--------- 1 file changed, 173 insertions(+), 187 deletions(-) diff --git a/examples/notebooks/beam-ml/custom_remote_inference.ipynb b/examples/notebooks/beam-ml/custom_remote_inference.ipynb index a29fb572adad..e510cd38e216 100644 --- a/examples/notebooks/beam-ml/custom_remote_inference.ipynb +++ b/examples/notebooks/beam-ml/custom_remote_inference.ipynb @@ -2,7 +2,7 @@ "cells": [ { "cell_type": "code", - "execution_count": null, + "execution_count": 1, "metadata": { "cellView": "form", "id": "paYiulysGrwR" @@ -137,7 +137,7 @@ "source": [ "!pip install --upgrade pip\n", "!pip install protobuf==3.19.4\n", - "!pip install apache-beam[interactive,gcp]>=2.40.0\n", + "!pip install apache-beam[interactive,gcp]>=2.65.0\n", "!pip install google-cloud-vision==3.1.1\n", "!pip install requests\n", "\n", @@ -192,21 +192,19 @@ }, { "cell_type": "code", - "execution_count": null, + "execution_count": 5, "metadata": { "id": "gE0go8CpnTy3" }, "outputs": [], "source": [ - "from typing import List\n", "import io\n", - "import os\n", "import requests\n", "\n", "from google.cloud import vision\n", "from google.cloud.vision_v1.types import Feature\n", "import apache_beam as beam\n", - "from apache_beam.ml.inference.base import ModelHandler\n", + "from apache_beam.ml.inference.base import RemoteModelHandler\n", "from apache_beam.ml.inference.base import RunInference\n" ] }, @@ -222,7 +220,7 @@ }, { "cell_type": "code", - "execution_count": null, + "execution_count": 6, "metadata": { "id": "_89eN_1QeYEd" }, @@ -257,29 +255,34 @@ "\n", "When you run remote inference, prepare to encounter, identify, and handle failure as gracefully as possible. We recommend using the following techniques:\n", "\n", - "* **Exponential backoff:** Retry failed remote calls with exponentially growing pauses between retries. Using exponential backoff ensures that failures don't lead to an overwhelming number of retries in quick succession.\n", + "* **Exponential backoff:** Retry failed remote calls with exponentially growing pauses between retries. Using exponential backoff ensures that failures don't lead to an overwhelming number of retries in quick succession. The `RemoteModelHandler` base class handles this logic, with the `retry_fn` argument determining which errors are retryable. For this example we will always retry. \n", "\n", - "* **Dead-letter queues:** Route failed inferences to a separate `PCollection` without failing the whole transform. Continue execution without failing the job (batch jobs' default behavior) or retrying indefinitely (streaming jobs' default behavior).\n", - "You can then run custom pipeline logic on the dead-letter (unprocessed messages) queue to log the failure, send an alert, and push the failed message to temporary storage so that it can eventually be reprocessed." + "* **Dead-letter queues:** Route failed inferences to a separate `PCollection` without failing the whole transform. Continue execution without failing the job (batch jobs' default behavior) or retrying indefinitely (streaming jobs' default behavior). This is provided through the `with_exception_handling()` [option](https://beam.apache.org/releases/pydoc/current/apache_beam.ml.inference.base.html#apache_beam.ml.inference.base.RunInference.with_exception_handling) for RunInference. This produces tagged outputs for the failed inferences which can be handled separately from successful ones. You can then run custom pipeline logic on the dead-letter (unprocessed messages) queue to log the failure, send an alert, and push the failed message to temporary storage so that it can eventually be reprocessed." ] }, { "cell_type": "code", - "execution_count": null, + "execution_count": 7, "metadata": { "id": "LnaisJ_JiY_Q" }, "outputs": [], "source": [ - "class CloudVisionModelHandler(ModelHandler):\n", - " \"\"\"DoFn that accepts a batch of images as bytearray\n", - " and sends that batch to the Cloud Vision API for remote inference\"\"\"\n", - " def load_model(self):\n", + "def _always_retry(exception: Exception) -> bool:\n", + " return True\n", + "\n", + "class CloudVisionModelHandler(RemoteModelHandler):\n", + " def __init__(self):\n", + " \"\"\"DoFn that accepts a batch of images as bytearray\n", + " and sends that batch to the Cloud Vision API for remote inference\n", + " \"\"\"\n", + " super().__init__(namespace=\"CloudVisionModelHandler\", retry_filter=_always_retry)\n", + " def create_client(self):\n", " \"\"\"Initiate the Google Vision API client.\"\"\"\n", " client = vision.ImageAnnotatorClient()\n", " return client\n", "\n", - " def run_inference(self, batch, model, inference):\n", + " def request(self, batch, model, inference_args):\n", " feature = Feature()\n", " feature.type_ = Feature.Type.LABEL_DETECTION\n", "\n", @@ -333,7 +336,7 @@ }, { "cell_type": "code", - "execution_count": null, + "execution_count": 9, "metadata": { "colab": { "base_uri": "https://localhost:8080/" @@ -343,129 +346,123 @@ }, "outputs": [ { - "output_type": "stream", "name": "stdout", + "output_type": "stream", "text": [ "('http://farm3.staticflickr.com/2824/10213933686_6936eb402b_z.jpg', label_annotations {\n", - " mid: \"/m/083wq\"\n", - " description: \"Wheel\"\n", - " score: 0.977976143\n", - " topicality: 0.977976143\n", + " mid: \"/m/04_sv\"\n", + " description: \"Motorcycle\"\n", + " score: 0.9922548\n", + " topicality: 0.14033242\n", "}\n", "label_annotations {\n", - " mid: \"/m/0h9mv\"\n", - " description: \"Tire\"\n", - " score: 0.977934957\n", - " topicality: 0.977934957\n", + " mid: \"/m/01prls\"\n", + " description: \"Land vehicle\"\n", + " score: 0.99086833\n", + " topicality: 0.0029524593\n", "}\n", "label_annotations {\n", - " mid: \"/m/043g5f\"\n", - " description: \"Fuel tank\"\n", - " score: 0.958490431\n", - " topicality: 0.958490431\n", + " mid: \"/m/0768fx\"\n", + " description: \"Automotive lighting\"\n", + " score: 0.9853215\n", + " topicality: 0.002913047\n", "}\n", "label_annotations {\n", - " mid: \"/m/05s2s\"\n", - " description: \"Plant\"\n", - " score: 0.95674181\n", - " topicality: 0.95674181\n", + " mid: \"/m/07yv9\"\n", + " description: \"Vehicle\"\n", + " score: 0.98517245\n", + " topicality: 0.010408105\n", "}\n", "label_annotations {\n", - " mid: \"/m/0h8lk_j\"\n", - " description: \"Automotive fuel system\"\n", - " score: 0.941456497\n", - " topicality: 0.941456497\n", + " mid: \"/m/043g5f\"\n", + " description: \"Fuel tank\"\n", + " score: 0.9823826\n", + " topicality: 0.01933147\n", "}\n", "label_annotations {\n", - " mid: \"/m/07yv9\"\n", - " description: \"Vehicle\"\n", - " score: 0.936428607\n", - " topicality: 0.936428607\n", + " mid: \"/m/012f08\"\n", + " description: \"Motor vehicle\"\n", + " score: 0.97732854\n", + " topicality: 0.0009314301\n", "}\n", "label_annotations {\n", - " mid: \"/m/02qwkrn\"\n", - " description: \"Vehicle brake\"\n", - " score: 0.905624092\n", - " topicality: 0.905624092\n", + " mid: \"/m/0h9mv\"\n", + " description: \"Tire\"\n", + " score: 0.9735299\n", + " topicality: 0.0020883244\n", "}\n", "label_annotations {\n", - " mid: \"/m/0h8pb3l\"\n", - " description: \"Automotive tire\"\n", - " score: 0.897686064\n", - " topicality: 0.897686064\n", + " mid: \"/m/083wq\"\n", + " description: \"Wheel\"\n", + " score: 0.9715105\n", + " topicality: 0.0028435893\n", "}\n", "label_annotations {\n", - " mid: \"/m/0768fx\"\n", - " description: \"Automotive lighting\"\n", - " score: 0.897505879\n", - " topicality: 0.897505879\n", + " mid: \"/m/0h8pb3l\"\n", + " description: \"Automotive Tire\"\n", + " score: 0.96993804\n", + " topicality: 5.827098e-05\n", "}\n", "label_annotations {\n", - " mid: \"/m/0h8p7_l\"\n", - " description: \"Automotive exhaust\"\n", - " score: 0.877965152\n", - " topicality: 0.877965152\n", + " mid: \"/m/0h8ls87\"\n", + " description: \"Automotive Exterior\"\n", + " score: 0.9641536\n", + " topicality: 0.00045098987\n", "}\n", ")\n", "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", - " mid: \"/m/054_l\"\n", - " description: \"Mirror\"\n", - " score: 0.969698846\n", - " topicality: 0.969698846\n", + " mid: \"/m/02w3_ws\"\n", + " description: \"Personal care\"\n", + " score: 0.853392\n", + " topicality: 0.00013828959\n", "}\n", "label_annotations {\n", - " mid: \"/m/02jz0l\"\n", - " description: \"Tap\"\n", - " score: 0.962297797\n", - " topicality: 0.962297797\n", + " mid: \"/m/02pkr5\"\n", + " description: \"Plumbing fixture\"\n", + " score: 0.8383083\n", + " topicality: 0.012253191\n", "}\n", "label_annotations {\n", - " mid: \"/m/0h8lr5r\"\n", - " description: \"Bathroom sink\"\n", - " score: 0.933002412\n", - " topicality: 0.933002412\n", + " mid: \"/m/0b_zf\"\n", + " description: \"Plumbing\"\n", + " score: 0.726803\n", + " topicality: 0.016276756\n", "}\n", "label_annotations {\n", - " mid: \"/m/0130jx\"\n", - " description: \"Sink\"\n", - " score: 0.930314779\n", - " topicality: 0.930314779\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02pkr5\"\n", - " description: \"Plumbing fixture\"\n", - " score: 0.920037031\n", - " topicality: 0.920037031\n", + " mid: \"/m/01j2bj\"\n", + " description: \"Bathroom\"\n", + " score: 0.72486097\n", + " topicality: 0.35419264\n", "}\n", "label_annotations {\n", - " mid: \"/m/02dgv\"\n", - " description: \"Door\"\n", - " score: 0.890176594\n", - " topicality: 0.890176594\n", + " mid: \"/m/02jz0l\"\n", + " description: \"Tap\"\n", + " score: 0.6317307\n", + " topicality: 0.00705197\n", "}\n", "label_annotations {\n", - " mid: \"/m/09ggk\"\n", - " description: \"Purple\"\n", - " score: 0.878831089\n", - " topicality: 0.878831089\n", + " mid: \"/m/0130jx\"\n", + " description: \"Sink\"\n", + " score: 0.5732167\n", + " topicality: 0.07520393\n", "}\n", "label_annotations {\n", - " mid: \"/m/01j2bj\"\n", - " description: \"Bathroom\"\n", - " score: 0.866840482\n", - " topicality: 0.866840482\n", + " mid: \"/m/054_l\"\n", + " description: \"Mirror\"\n", + " score: 0.5680867\n", + " topicality: 0.08497098\n", "}\n", "label_annotations {\n", - " mid: \"/m/04wnmd\"\n", - " description: \"Fixture\"\n", - " score: 0.862223864\n", - " topicality: 0.862223864\n", + " mid: \"/m/0h8lr5r\"\n", + " description: \"Bathroom Sink\"\n", + " score: 0.557554\n", + " topicality: 0.007725588\n", "}\n", "label_annotations {\n", - " mid: \"/m/09qqq\"\n", - " description: \"Wall\"\n", - " score: 0.809348285\n", - " topicality: 0.809348285\n", + " mid: \"/m/03jvk\"\n", + " description: \"Household hardware\"\n", + " score: 0.5140049\n", + " topicality: 0.00064662547\n", "}\n", ")\n", "('http://farm8.staticflickr.com/7003/6528937031_10e1ce0960_z.jpg', error {\n", @@ -484,125 +481,113 @@ "}\n", ")\n", "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", - " mid: \"/m/054_l\"\n", - " description: \"Mirror\"\n", - " score: 0.969698846\n", - " topicality: 0.969698846\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02jz0l\"\n", - " description: \"Tap\"\n", - " score: 0.962297797\n", - " topicality: 0.962297797\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/0h8lr5r\"\n", - " description: \"Bathroom sink\"\n", - " score: 0.933002412\n", - " topicality: 0.933002412\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/0130jx\"\n", - " description: \"Sink\"\n", - " score: 0.930314779\n", - " topicality: 0.930314779\n", + " mid: \"/m/02w3_ws\"\n", + " description: \"Personal care\"\n", + " score: 0.853392\n", + " topicality: 0.00013828959\n", "}\n", "label_annotations {\n", " mid: \"/m/02pkr5\"\n", " description: \"Plumbing fixture\"\n", - " score: 0.920037031\n", - " topicality: 0.920037031\n", + " score: 0.8383083\n", + " topicality: 0.012253191\n", "}\n", "label_annotations {\n", - " mid: \"/m/02dgv\"\n", - " description: \"Door\"\n", - " score: 0.890176594\n", - " topicality: 0.890176594\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/09ggk\"\n", - " description: \"Purple\"\n", - " score: 0.878831089\n", - " topicality: 0.878831089\n", + " mid: \"/m/0b_zf\"\n", + " description: \"Plumbing\"\n", + " score: 0.726803\n", + " topicality: 0.016276756\n", "}\n", "label_annotations {\n", " mid: \"/m/01j2bj\"\n", " description: \"Bathroom\"\n", - " score: 0.866840482\n", - " topicality: 0.866840482\n", + " score: 0.72486097\n", + " topicality: 0.35419264\n", "}\n", "label_annotations {\n", - " mid: \"/m/04wnmd\"\n", - " description: \"Fixture\"\n", - " score: 0.862223864\n", - " topicality: 0.862223864\n", + " mid: \"/m/02jz0l\"\n", + " description: \"Tap\"\n", + " score: 0.6317307\n", + " topicality: 0.00705197\n", "}\n", "label_annotations {\n", - " mid: \"/m/09qqq\"\n", - " description: \"Wall\"\n", - " score: 0.809348285\n", - " topicality: 0.809348285\n", + " mid: \"/m/0130jx\"\n", + " description: \"Sink\"\n", + " score: 0.5732167\n", + " topicality: 0.07520393\n", "}\n", - ")\n", - "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", + "label_annotations {\n", " mid: \"/m/054_l\"\n", " description: \"Mirror\"\n", - " score: 0.969698846\n", - " topicality: 0.969698846\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/02jz0l\"\n", - " description: \"Tap\"\n", - " score: 0.962297797\n", - " topicality: 0.962297797\n", + " score: 0.5680867\n", + " topicality: 0.08497098\n", "}\n", "label_annotations {\n", " mid: \"/m/0h8lr5r\"\n", - " description: \"Bathroom sink\"\n", - " score: 0.933002412\n", - " topicality: 0.933002412\n", + " description: \"Bathroom Sink\"\n", + " score: 0.557554\n", + " topicality: 0.007725588\n", "}\n", "label_annotations {\n", - " mid: \"/m/0130jx\"\n", - " description: \"Sink\"\n", - " score: 0.930314779\n", - " topicality: 0.930314779\n", + " mid: \"/m/03jvk\"\n", + " description: \"Household hardware\"\n", + " score: 0.5140049\n", + " topicality: 0.00064662547\n", + "}\n", + ")\n", + "('http://farm8.staticflickr.com/7026/6388965173_92664a0d78_z.jpg', label_annotations {\n", + " mid: \"/m/02w3_ws\"\n", + " description: \"Personal care\"\n", + " score: 0.853392\n", + " topicality: 0.00013828959\n", "}\n", "label_annotations {\n", " mid: \"/m/02pkr5\"\n", " description: \"Plumbing fixture\"\n", - " score: 0.920037031\n", - " topicality: 0.920037031\n", + " score: 0.8383083\n", + " topicality: 0.012253191\n", "}\n", "label_annotations {\n", - " mid: \"/m/02dgv\"\n", - " description: \"Door\"\n", - " score: 0.890176594\n", - " topicality: 0.890176594\n", - "}\n", - "label_annotations {\n", - " mid: \"/m/09ggk\"\n", - " description: \"Purple\"\n", - " score: 0.878831089\n", - " topicality: 0.878831089\n", + " mid: \"/m/0b_zf\"\n", + " description: \"Plumbing\"\n", + " score: 0.726803\n", + " topicality: 0.016276756\n", "}\n", "label_annotations {\n", " mid: \"/m/01j2bj\"\n", " description: \"Bathroom\"\n", - " score: 0.866840482\n", - " topicality: 0.866840482\n", + " score: 0.72486097\n", + " topicality: 0.35419264\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/02jz0l\"\n", + " description: \"Tap\"\n", + " score: 0.6317307\n", + " topicality: 0.00705197\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/0130jx\"\n", + " description: \"Sink\"\n", + " score: 0.5732167\n", + " topicality: 0.07520393\n", + "}\n", + "label_annotations {\n", + " mid: \"/m/054_l\"\n", + " description: \"Mirror\"\n", + " score: 0.5680867\n", + " topicality: 0.08497098\n", "}\n", "label_annotations {\n", - " mid: \"/m/04wnmd\"\n", - " description: \"Fixture\"\n", - " score: 0.862223864\n", - " topicality: 0.862223864\n", + " mid: \"/m/0h8lr5r\"\n", + " description: \"Bathroom Sink\"\n", + " score: 0.557554\n", + " topicality: 0.007725588\n", "}\n", "label_annotations {\n", - " mid: \"/m/09qqq\"\n", - " description: \"Wall\"\n", - " score: 0.809348285\n", - " topicality: 0.809348285\n", + " mid: \"/m/03jvk\"\n", + " description: \"Household hardware\"\n", + " score: 0.5140049\n", + " topicality: 0.00064662547\n", "}\n", ")\n" ] @@ -610,11 +595,12 @@ ], "source": [ "with beam.Pipeline() as pipeline:\n", - " _ = (pipeline | \"Create inputs\" >> beam.Create(image_urls)\n", + " main, failed = (pipeline | \"Create inputs\" >> beam.Create(image_urls)\n", " | \"Read images\" >> beam.Map(read_image)\n", - " | \"Inference\" >> RunInference(model_handler=CloudVisionModelHandler())\n", - " | \"Print image_url and annotation\" >> beam.Map(print)\n", - " )" + " | \"Inference\" >> RunInference(model_handler=CloudVisionModelHandler()).with_exception_handling()\n", + " )\n", + " _ = main | \"Print image_url and annotation\" >> beam.Map(print)\n", + " _ = failed.failed_inferences | \"Print failed inferences\" >> beam.Map(print)" ] }, { From 2bac79706c18666b6a2b19e16b73423bb773e73a Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 3 Jun 2025 11:46:41 -0400 Subject: [PATCH 19/49] Remove Java 8 container (#35125) --- .../beam_Publish_Beam_SDK_Snapshots.yml | 1 - .../xlang_KafkaIO_Python.txt | 2 +- build.gradle.kts | 2 +- .../beam/gradle/BeamModulePlugin.groovy | 2 +- .../arm/build.gradle | 1 - .../google-cloud-dataflow-java/build.gradle | 1 - sdks/java/container/build.gradle | 1 - sdks/java/container/java8/build.gradle | 24 ---------- .../container/java8/java8-security.properties | 47 ------------------- .../java8/option-java8-security.json | 9 ---- .../sdk/options/PortablePipelineOptions.java | 2 +- .../apache_beam/examples/kafkataxi/README.md | 6 +-- .../Run Beam SQL with beam_sql magic.ipynb | 2 +- .../python/test-suites/dataflow/common.gradle | 2 +- settings.gradle.kts | 1 - .../en/documentation/runtime/environments.md | 7 ++- 16 files changed, 12 insertions(+), 98 deletions(-) delete mode 100644 sdks/java/container/java8/build.gradle delete mode 100644 sdks/java/container/java8/java8-security.properties delete mode 100644 sdks/java/container/java8/option-java8-security.json diff --git a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml index 8aead7937cf5..49fcff4e91f0 100644 --- a/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml +++ b/.github/workflows/beam_Publish_Beam_SDK_Snapshots.yml @@ -62,7 +62,6 @@ jobs: job_phrase: ["N/A"] container_task: - "go:container" - - "java:container:java8" - "java:container:java11" - "java:container:java17" - "java:container:java21" diff --git a/.github/workflows/performance-tests-pipeline-options/xlang_KafkaIO_Python.txt b/.github/workflows/performance-tests-pipeline-options/xlang_KafkaIO_Python.txt index d0100ffbfc34..603f2fcc79c5 100644 --- a/.github/workflows/performance-tests-pipeline-options/xlang_KafkaIO_Python.txt +++ b/.github/workflows/performance-tests-pipeline-options/xlang_KafkaIO_Python.txt @@ -17,7 +17,7 @@ --runner=DataflowRunner --region=us-central1 --temp_location=gs://temp-storage-for-perf-tests/ ---sdk_harness_container_image_overrides=.*java.*,gcr.io/apache-beam-testing/beam-sdk/beam_java8_sdk:latest +--sdk_harness_container_image_overrides=.*java.*,gcr.io/apache-beam-testing/beam-sdk/beam_java11_sdk:latest --publish_to_big_query=true --metrics_dataset=beam_performance --metrics_table=python_kafkaio_results diff --git a/build.gradle.kts b/build.gradle.kts index a0844249bf08..af2c143f8c44 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -282,7 +282,7 @@ tasks.register("javaPreCommit") { dependsOn(":runners:spark:3:job-server:build") dependsOn(":runners:twister2:build") dependsOn(":sdks:java:build-tools:build") - dependsOn(":sdks:java:container:java8:docker") + dependsOn(":sdks:java:container:java11:docker") dependsOn(":sdks:java:core:build") dependsOn(":sdks:java:core:jmh:build") dependsOn(":sdks:java:expansion-service:build") diff --git a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy index c6ee2ab76d70..88023feb7eac 100644 --- a/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy +++ b/buildSrc/src/main/groovy/org/apache/beam/gradle/BeamModulePlugin.groovy @@ -2371,7 +2371,7 @@ class BeamModulePlugin implements Plugin { // development image at docker.io (see sdks/CONTAINERS.md): // // format: apache/beam_$NAME_sdk:latest - // ie: apache/beam_python3.7_sdk:latest apache/beam_java8_sdk:latest apache/beam_go_sdk:latest + // ie: apache/beam_python3.12_sdk:latest apache/beam_java21_sdk:latest apache/beam_go_sdk:latest // // Both the root and tag can be defined using properties or explicitly provided. project.ext.containerImageName = { diff --git a/runners/google-cloud-dataflow-java/arm/build.gradle b/runners/google-cloud-dataflow-java/arm/build.gradle index ae63cdf8bdb7..c5c907ae797e 100644 --- a/runners/google-cloud-dataflow-java/arm/build.gradle +++ b/runners/google-cloud-dataflow-java/arm/build.gradle @@ -47,7 +47,6 @@ evaluationDependsOn(":sdks:java:io:google-cloud-platform") evaluationDependsOn(":sdks:java:core") evaluationDependsOn(":examples:java") evaluationDependsOn(":runners:google-cloud-dataflow-java:worker") -evaluationDependsOn(":sdks:java:container:java8") evaluationDependsOn(":sdks:java:container:java11") processResources { diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 9e24c0db09cf..63125a68efb2 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -48,7 +48,6 @@ evaluationDependsOn(":sdks:java:io:google-cloud-platform") evaluationDependsOn(":sdks:java:core") evaluationDependsOn(":examples:java") evaluationDependsOn(":runners:google-cloud-dataflow-java:worker") -evaluationDependsOn(":sdks:java:container:java8") evaluationDependsOn(":sdks:java:container:java11") ext.dataflowLegacyEnvironmentMajorVersion = '8' diff --git a/sdks/java/container/build.gradle b/sdks/java/container/build.gradle index ac4104638b65..bc9bc45ec371 100644 --- a/sdks/java/container/build.gradle +++ b/sdks/java/container/build.gradle @@ -80,7 +80,6 @@ artifacts { } task pushAll { - dependsOn ":sdks:java:container:java8:docker" dependsOn ":sdks:java:container:java11:docker" dependsOn ":sdks:java:container:java17:docker" dependsOn ":sdks:java:container:java21:docker" diff --git a/sdks/java/container/java8/build.gradle b/sdks/java/container/java8/build.gradle deleted file mode 100644 index 9d105648240c..000000000000 --- a/sdks/java/container/java8/build.gradle +++ /dev/null @@ -1,24 +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. - */ - -project.ext { - imageJavaVersion = '8' -} - -// Load the main build script which contains all build logic. -apply from: "../common.gradle" \ No newline at end of file diff --git a/sdks/java/container/java8/java8-security.properties b/sdks/java/container/java8/java8-security.properties deleted file mode 100644 index f637d3ef7567..000000000000 --- a/sdks/java/container/java8/java8-security.properties +++ /dev/null @@ -1,47 +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. - -# Java 8 java.security properties file override for JVM -# base properties derived from: -# openjdk version "1.8.0_342" -# OpenJDK Runtime Environment (build 1.8.0_342-b07) -# OpenJDK 64-Bit Server VM (build 25.342-b07, mixed mode) - -# Java has now disabled TLSv1 and TLSv1.1. We specifically put it in the -# legacy algorithms list to allow it to be used if something better is not -# available (e.g. TLSv1.2). This will prevent breakages for existing users -# (for example JDBC with MySQL). See -# https://bugs.java.com/bugdatabase/view_bug.do?bug_id=JDK-8202343 -# for additional details. -jdk.tls.disabledAlgorithms=SSLv3, RC4, DES, MD5withRSA, \ - DH keySize < 1024, EC keySize < 224, 3DES_EDE_CBC, anon, NULL, \ - include jdk.disabled.namedCurves - -jdk.tls.legacyAlgorithms= \ - K_NULL, C_NULL, M_NULL, \ - DH_anon, ECDH_anon, \ - RC4_128, RC4_40, DES_CBC, DES40_CBC, \ - 3DES_EDE_CBC, TLSv1, TLSv1.1 - -# /dev/random blocks in virtualized environments due to lack of -# good entropy sources, which makes SecureRandom use impractical. -# In particular, that affects the performance of HTTPS that relies -# on SecureRandom. -# -# Due to that, /dev/urandom is used as the default. -# -# See http://www.2uo.de/myths-about-urandom/ for some background -# on security of /dev/urandom on Linux. -securerandom.source=file:/dev/./urandom \ No newline at end of file diff --git a/sdks/java/container/java8/option-java8-security.json b/sdks/java/container/java8/option-java8-security.json deleted file mode 100644 index 47f2938bf7cd..000000000000 --- a/sdks/java/container/java8/option-java8-security.json +++ /dev/null @@ -1,9 +0,0 @@ -{ - "name": "java-security", - "enabled": true, - "options": { - "properties": { - "java.security.properties": "/opt/apache/beam/options/java8-security.properties" - } - } -} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PortablePipelineOptions.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PortablePipelineOptions.java index ec82aebbef66..bfeddf5c697f 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PortablePipelineOptions.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/options/PortablePipelineOptions.java @@ -98,7 +98,7 @@ public interface PortablePipelineOptions extends PipelineOptions, FileStagingOpt @Description( "Options for configuring the default environment of portable workers. This environment will be used for all executable stages except for external transforms. Recognized options depend on the value of defaultEnvironmentType:\n" - + "DOCKER: docker_container_image (optional), e.g. 'apache/beam_java8_sdk:latest'. If unset, will default to the latest official release of the Beam Java SDK corresponding to your Java runtime version (8 or 11).\n" + + "DOCKER: docker_container_image (optional), e.g. 'apache/beam_java21_sdk:latest'. If unset, will default to the latest official release of the Beam Java SDK corresponding to your Java runtime version (11, 17 or 21).\n" + "EXTERNAL: external_service_address (required), e.g. 'localhost:50000'\n" + "PROCESS: process_command (required), process_variables (optional). process_command must be the location of an executable file that starts a Beam SDK worker. process_variables is a comma-separated list of environment variable assignments which will be set before running the process, e.g. 'FOO=a,BAR=b'\n\n" + "environmentOptions and defaultEnvironmentConfig are mutually exclusive. Prefer environmentOptions.") diff --git a/sdks/python/apache_beam/examples/kafkataxi/README.md b/sdks/python/apache_beam/examples/kafkataxi/README.md index 72a8d8f85c03..86924200b1f2 100644 --- a/sdks/python/apache_beam/examples/kafkataxi/README.md +++ b/sdks/python/apache_beam/examples/kafkataxi/README.md @@ -134,8 +134,8 @@ prerequisites and additional information. ```sh export DOCKER_ROOT="Your Docker Repository Root" -./gradlew :sdks:java:container:java8:docker -Pdocker-repository-root=$DOCKER_ROOT -Pdocker-tag=latest -docker push $DOCKER_ROOT/beam_java8_sdk:latest +./gradlew :sdks:java:container:java11:docker -Pdocker-repository-root=$DOCKER_ROOT -Pdocker-tag=latest +docker push $DOCKER_ROOT/beam_java11_sdk:latest ``` For portable Flink/Spark in local mode, instead of above command just build the @@ -184,5 +184,5 @@ python -m apache_beam.examples.kafkataxi.kafka_taxi \ --num_workers $NUM_WORKERS \ --job_name $JOB_NAME \ --bootstrap_servers $BOOTSTRAP_SERVER \ - --sdk_harness_container_image_overrides ".*java.*,${DOCKER_ROOT}/beam_java8_sdk:latest" + --sdk_harness_container_image_overrides ".*java.*,${DOCKER_ROOT}/beam_java11_sdk:latest" ``` diff --git a/sdks/python/apache_beam/runners/interactive/examples/Run Beam SQL with beam_sql magic.ipynb b/sdks/python/apache_beam/runners/interactive/examples/Run Beam SQL with beam_sql magic.ipynb index d68fa8cfb38d..0a98f687947f 100644 --- a/sdks/python/apache_beam/runners/interactive/examples/Run Beam SQL with beam_sql magic.ipynb +++ b/sdks/python/apache_beam/runners/interactive/examples/Run Beam SQL with beam_sql magic.ipynb @@ -94,7 +94,7 @@ "\n", "- Have the Java expansion service shadowjar built. Go to the root directory of your local beam repo and then execute:\n", " `./gradlew :sdks:java:extensions:sql:expansion-service:shadowJar`.\n", - "- Based on your jdk version, pull the docker image `docker pull apache/beam_java11_sdk` or `docker pull apache/beam_java8_sdk`.\n", + "- Based on your jdk version, pull the docker image `docker pull apache/beam_java11_sdk` or java17, java21.\n", "- Then tag the image with your current Beam dev version. You can check the dev version under `apache_beam.version.__version__`. For example, if you're using jdk11 and dev version is `x.x.x.dev`, execute `docker image tag apache/beam_java11_sdk:latest apache/beam_java11_sdk:x.x.x.dev`." ] }, diff --git a/sdks/python/test-suites/dataflow/common.gradle b/sdks/python/test-suites/dataflow/common.gradle index c18b12654f78..7fc180fe1359 100644 --- a/sdks/python/test-suites/dataflow/common.gradle +++ b/sdks/python/test-suites/dataflow/common.gradle @@ -629,7 +629,7 @@ project(":sdks:python:test-suites:xlang").ext.xlangTasks.each { taskMetadata -> "--project=${gcpProject}", "--region=${gcpRegion}", "--sdk_container_image=gcr.io/apache-beam-testing/beam-sdk/beam_python${project.ext.pythonVersion}_sdk:latest", - "--sdk_harness_container_image_overrides=.*java.*,gcr.io/apache-beam-testing/beam-sdk/beam_java8_sdk:latest" + "--sdk_harness_container_image_overrides=.*java.*,gcr.io/apache-beam-testing/beam-sdk/beam_java11_sdk:latest" ], pytestOptions: basicPytestOpts, additionalDeps: taskMetadata.additionalDeps, diff --git a/settings.gradle.kts b/settings.gradle.kts index 4156d6d46194..8a9be3358a65 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -167,7 +167,6 @@ include(":sdks:java:bom:gcp") include(":sdks:java:build-tools") include(":sdks:java:container") 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") diff --git a/website/www/site/content/en/documentation/runtime/environments.md b/website/www/site/content/en/documentation/runtime/environments.md index 96eaa0e76b0c..262be67d1cdf 100644 --- a/website/www/site/content/en/documentation/runtime/environments.md +++ b/website/www/site/content/en/documentation/runtime/environments.md @@ -131,14 +131,13 @@ This method requires building image artifacts from Beam source. For additional i ``` $> docker images --digests REPOSITORY TAG DIGEST IMAGE ID CREATED SIZE - apache/beam_java8_sdk latest sha256:... ... 1 min ago ... apache/beam_java11_sdk latest sha256:... ... 1 min ago ... apache/beam_java17_sdk latest sha256:... ... 1 min ago ... - apache/beam_python3.6_sdk latest sha256:... ... 1 min ago ... - apache/beam_python3.7_sdk latest sha256:... ... 1 min ago ... - apache/beam_python3.8_sdk latest sha256:... ... 1 min ago ... + apache/beam_java21_sdk latest sha256:... ... 1 min ago ... apache/beam_python3.9_sdk latest sha256:... ... 1 min ago ... apache/beam_python3.10_sdk latest sha256:... ... 1 min ago ... + apache/beam_python3.11_sdk latest sha256:... ... 1 min ago ... + apache/beam_python3.12_sdk latest sha256:... ... 1 min ago ... apache/beam_go_sdk latest sha256:... ... 1 min ago ... ``` From 7fbc8b4ece81a7f474b986a728bfff21fba7b049 Mon Sep 17 00:00:00 2001 From: Derrick Williams Date: Tue, 3 Jun 2025 11:54:03 -0400 Subject: [PATCH 20/49] add extra_transforms block documentation to chain transform documentation (#35101) --- .../content/en/documentation/sdks/yaml.md | 46 +++++++++++++++++++ 1 file changed, 46 insertions(+) diff --git a/website/www/site/content/en/documentation/sdks/yaml.md b/website/www/site/content/en/documentation/sdks/yaml.md index 3d266ade6160..73d1eebaae95 100644 --- a/website/www/site/content/en/documentation/sdks/yaml.md +++ b/website/www/site/content/en/documentation/sdks/yaml.md @@ -273,6 +273,52 @@ pipeline: path: /path/to/output.json ``` +If a `chain` pipeline has required error consumption or needs additional +transforms not supported in a typical `chain` context, use an +`extra_transforms` block. + +``` +pipeline: + type: chain + transforms: + - type: ReadFromCsv + config: + path: /path/to/input*.csv + + - type: MapToFields + name: SomeStep + config: + language: python + fields: + col1: col1 + # This could raise a divide-by-zero error. + ratio: col2 / col3 + error_handling: + output: errors + + - type: MapToFields + name: AnotherStep + config: + language: python + fields: + col1: col1 + # This could raise a divide-by-zero error. + inverse_ratio: 1 / ratio + error_handling: + output: errors + + - type: WriteToJson + config: + path: /path/to/output.json + + extra_transforms: + - type: WriteToJson + name: WriteErrors + input: [SomeStep.errors, AnotherStep.errors] + config: + path: /path/to/errors.json +``` + ### Source and sink transforms As syntactic sugar, you can name the first and last transforms in your pipeline From a247978ffc9c7aa430db6ca3ec58d3820fbfb57d Mon Sep 17 00:00:00 2001 From: Derrick Williams Date: Tue, 3 Jun 2025 13:15:02 -0400 Subject: [PATCH 21/49] add note about testing (#35075) --- sdks/python/apache_beam/yaml/generate_yaml_docs.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sdks/python/apache_beam/yaml/generate_yaml_docs.py b/sdks/python/apache_beam/yaml/generate_yaml_docs.py index ae1a04661e62..8e7471c90de9 100644 --- a/sdks/python/apache_beam/yaml/generate_yaml_docs.py +++ b/sdks/python/apache_beam/yaml/generate_yaml_docs.py @@ -604,6 +604,9 @@ def main(): github.

+

Note: These examples below are automatically tested for + correctness and may be used as a starting point for your own + pipelines.

''')) From 2116fb1d3ce38ec4e9968db97a4746e3fb54a73f Mon Sep 17 00:00:00 2001 From: bullet03 Date: Tue, 3 Jun 2025 19:32:01 +0200 Subject: [PATCH 22/49] [Website] update akvelon case study: update text and fix landing page (#35133) * feat: change text * fix: add learn more to quotes Akvelon --------- Co-authored-by: Bulat Safiullin --- website/www/site/data/en/quotes.yaml | 1 + 1 file changed, 1 insertion(+) diff --git a/website/www/site/data/en/quotes.yaml b/website/www/site/data/en/quotes.yaml index 936b7472d333..4ae6cca442af 100644 --- a/website/www/site/data/en/quotes.yaml +++ b/website/www/site/data/en/quotes.yaml @@ -85,6 +85,7 @@ icon: icons/quote-icon.svg logoUrl: /images/logos/powered-by/akvelon.png linkUrl: case-studies/akvelon/index.html + linkText: Learn more - text: With Apache Beam and Dataflow, Credit Karma achieved a 99% uptime for critical data pipelines, a significant jump from 80%. This reliability, coupled with faster development (1 engineer vs. 3 estimated), has been crucial for enabling real-time financial insights for our more than 140 million members. icon: icons/quote-icon.svg logoUrl: images/logos/powered-by/credit-karma.png From f237833fa35233a71307e21703616e62ca29fbee Mon Sep 17 00:00:00 2001 From: Vitaly Terentyev Date: Tue, 3 Jun 2025 22:27:51 +0400 Subject: [PATCH 23/49] Fix PostCommit Python Xlang IO Dataflow job (#35131) * Add support for iterable type * Fix formatting --- sdks/python/apache_beam/coders/row_coder.py | 3 +++ sdks/python/apache_beam/typehints/schemas.py | 4 ++++ 2 files changed, 7 insertions(+) diff --git a/sdks/python/apache_beam/coders/row_coder.py b/sdks/python/apache_beam/coders/row_coder.py index dc473b1d6d7d..1becf408cfbf 100644 --- a/sdks/python/apache_beam/coders/row_coder.py +++ b/sdks/python/apache_beam/coders/row_coder.py @@ -161,6 +161,9 @@ def _nonnull_coder_from_type(field_type): return BytesCoder() elif type_info == "array_type": return IterableCoder(_coder_from_type(field_type.array_type.element_type)) + elif type_info == "iterable_type": + return IterableCoder( + _coder_from_type(field_type.iterable_type.element_type)) elif type_info == "map_type": return MapCoder( _coder_from_type(field_type.map_type.key_type), diff --git a/sdks/python/apache_beam/typehints/schemas.py b/sdks/python/apache_beam/typehints/schemas.py index 23cad5813458..de4cdb9fdf75 100644 --- a/sdks/python/apache_beam/typehints/schemas.py +++ b/sdks/python/apache_beam/typehints/schemas.py @@ -538,6 +538,10 @@ def typing_from_runner_api( return LogicalType.from_runner_api( fieldtype_proto.logical_type).language_type() + elif type_info == "iterable_type": + return Sequence[self.typing_from_runner_api( + fieldtype_proto.iterable_type.element_type)] + else: raise ValueError(f"Unrecognized type_info: {type_info!r}") From d176eb8ce348aa6a35c23ce09637a145e7572916 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 3 Jun 2025 14:28:46 -0400 Subject: [PATCH 24/49] Bump google.golang.org/grpc from 1.72.0 to 1.72.2 in /sdks (#35113) Bumps [google.golang.org/grpc](https://github.com/grpc/grpc-go) from 1.72.0 to 1.72.2. - [Release notes](https://github.com/grpc/grpc-go/releases) - [Commits](https://github.com/grpc/grpc-go/compare/v1.72.0...v1.72.2) --- updated-dependencies: - dependency-name: google.golang.org/grpc dependency-version: 1.72.2 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 58647415ce82..f8ca61d56c3f 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -62,7 +62,7 @@ require ( golang.org/x/text v0.25.0 google.golang.org/api v0.232.0 google.golang.org/genproto v0.0.0-20250407143221-ac9807e6c755 - google.golang.org/grpc v1.72.0 + google.golang.org/grpc v1.72.2 google.golang.org/protobuf v1.36.6 gopkg.in/yaml.v2 v2.4.0 gopkg.in/yaml.v3 v3.0.1 diff --git a/sdks/go.sum b/sdks/go.sum index b4ba58fa157c..f0aa7ad5f960 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -2249,8 +2249,8 @@ google.golang.org/grpc v1.52.3/go.mod h1:pu6fVzoFb+NBYNAvQL08ic+lvB2IojljRYuun5v google.golang.org/grpc v1.53.0/go.mod h1:OnIrk0ipVdj4N5d9IUoFUx72/VlD7+jUsHwZgwSMQpw= google.golang.org/grpc v1.54.0/go.mod h1:PUSEXI6iWghWaB6lXM4knEgpJNu2qUcKfDtNci3EC2g= google.golang.org/grpc v1.56.3/go.mod h1:I9bI3vqKfayGqPUAwGdOSu7kt6oIJLixfffKrpXqQ9s= -google.golang.org/grpc v1.72.0 h1:S7UkcVa60b5AAQTaO6ZKamFp1zMZSU0fGDK2WZLbBnM= -google.golang.org/grpc v1.72.0/go.mod h1:wH5Aktxcg25y1I3w7H69nHfXdOG3UiadoBtjh3izSDM= +google.golang.org/grpc v1.72.2 h1:TdbGzwb82ty4OusHWepvFWGLgIbNo1/SUynEN0ssqv8= +google.golang.org/grpc v1.72.2/go.mod h1:wH5Aktxcg25y1I3w7H69nHfXdOG3UiadoBtjh3izSDM= google.golang.org/grpc/cmd/protoc-gen-go-grpc v1.1.0/go.mod h1:6Kw0yEErY5E/yWrBtf03jp27GLLJujG4z/JK95pnjjw= 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= From 74b9ccbb80af990ed9f806d4637ce8b795ea818c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 3 Jun 2025 14:29:13 -0400 Subject: [PATCH 25/49] Bump cloud.google.com/go/bigquery from 1.67.0 to 1.69.0 in /sdks (#35061) Bumps [cloud.google.com/go/bigquery](https://github.com/googleapis/google-cloud-go) from 1.67.0 to 1.69.0. - [Release notes](https://github.com/googleapis/google-cloud-go/releases) - [Changelog](https://github.com/googleapis/google-cloud-go/blob/main/CHANGES.md) - [Commits](https://github.com/googleapis/google-cloud-go/compare/spanner/v1.67.0...spanner/v1.69.0) --- updated-dependencies: - dependency-name: cloud.google.com/go/bigquery dependency-version: 1.69.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 12 ++++++------ 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index f8ca61d56c3f..823a662a6c26 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -25,13 +25,13 @@ go 1.23.0 toolchain go1.24.1 require ( - cloud.google.com/go/bigquery v1.67.0 + cloud.google.com/go/bigquery v1.69.0 cloud.google.com/go/bigtable v1.37.0 cloud.google.com/go/datastore v1.20.0 cloud.google.com/go/profiler v0.4.2 cloud.google.com/go/pubsub v1.49.0 cloud.google.com/go/spanner v1.82.0 - cloud.google.com/go/storage v1.52.0 + cloud.google.com/go/storage v1.53.0 github.com/aws/aws-sdk-go-v2 v1.36.3 github.com/aws/aws-sdk-go-v2/config v1.29.14 github.com/aws/aws-sdk-go-v2/credentials v1.17.67 diff --git a/sdks/go.sum b/sdks/go.sum index f0aa7ad5f960..bdf56826061a 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -135,8 +135,8 @@ cloud.google.com/go/bigquery v1.47.0/go.mod h1:sA9XOgy0A8vQK9+MWhEQTY6Tix87M/Zur cloud.google.com/go/bigquery v1.48.0/go.mod h1:QAwSz+ipNgfL5jxiaK7weyOhzdoAy1zFm0Nf1fysJac= cloud.google.com/go/bigquery v1.49.0/go.mod h1:Sv8hMmTFFYBlt/ftw2uN6dFdQPzBlREY9yBh7Oy7/4Q= cloud.google.com/go/bigquery v1.50.0/go.mod h1:YrleYEh2pSEbgTBZYMJ5SuSr0ML3ypjRB1zgf7pvQLU= -cloud.google.com/go/bigquery v1.67.0 h1:GXleMyn/cu5+DPLy9Rz5f5IULWTLrepwbQnP/5qrVbY= -cloud.google.com/go/bigquery v1.67.0/go.mod h1:HQeP1AHFuAz0Y55heDSb0cjZIhnEkuwFRBGo6EEKHug= +cloud.google.com/go/bigquery v1.69.0 h1:rZvHnjSUs5sHK3F9awiuFk2PeOaB8suqNuim21GbaTc= +cloud.google.com/go/bigquery v1.69.0/go.mod h1:TdGLquA3h/mGg+McX+GsqG9afAzTAcldMjqhdjHTLew= cloud.google.com/go/bigtable v1.37.0 h1:Q+x7y04lQ0B+WXp03wc1/FLhFt4CwcQdkwWT0M4Jp3w= cloud.google.com/go/bigtable v1.37.0/go.mod h1:HXqddP6hduwzrtiTCqZPpj9ij4hGZb4Zy1WF/dT+yaU= cloud.google.com/go/billing v1.4.0/go.mod h1:g9IdKBEFlItS8bTtlrZdVLWSSdSyFUZKXNS02zKMOZY= @@ -213,8 +213,8 @@ cloud.google.com/go/datacatalog v1.8.0/go.mod h1:KYuoVOv9BM8EYz/4eMFxrr4DUKhGIOX cloud.google.com/go/datacatalog v1.8.1/go.mod h1:RJ58z4rMp3gvETA465Vg+ag8BGgBdnRPEMMSTr5Uv+M= cloud.google.com/go/datacatalog v1.12.0/go.mod h1:CWae8rFkfp6LzLumKOnmVh4+Zle4A3NXLzVJ1d1mRm0= cloud.google.com/go/datacatalog v1.13.0/go.mod h1:E4Rj9a5ZtAxcQJlEBTLgMTphfP11/lNaAshpoBgemX8= -cloud.google.com/go/datacatalog v1.25.0 h1:jIin9caDEyByOLKDCJGBwDHj1/yLqMvZdutvby/WYN8= -cloud.google.com/go/datacatalog v1.25.0/go.mod h1:Bodb/U9ZV549+0sQPoX6WtYnbFwqayuYldw5p6PmbH4= +cloud.google.com/go/datacatalog v1.26.0 h1:eFgygb3DTufTWWUB8ARk+dSuXz+aefNJXTlkWlQcWwE= +cloud.google.com/go/datacatalog v1.26.0/go.mod h1:bLN2HLBAwB3kLTFT5ZKLHVPj/weNz6bR0c7nYp0LE14= cloud.google.com/go/dataflow v0.6.0/go.mod h1:9QwV89cGoxjjSR9/r7eFDqqjtvbKxAK2BaYU6PVk9UM= cloud.google.com/go/dataflow v0.7.0/go.mod h1:PX526vb4ijFMesO1o202EaUmouZKBpjHsTlCtB4parQ= cloud.google.com/go/dataflow v0.8.0/go.mod h1:Rcf5YgTKPtQyYz8bLYhFoIV/vP39eL7fWNcSOyFfLJE= @@ -573,8 +573,8 @@ cloud.google.com/go/storage v1.23.0/go.mod h1:vOEEDNFnciUMhBeT6hsJIn3ieU5cFRmzeL cloud.google.com/go/storage v1.27.0/go.mod h1:x9DOL8TK/ygDUMieqwfhdpQryTeEkhGKMi80i/iqR2s= cloud.google.com/go/storage v1.28.1/go.mod h1:Qnisd4CqDdo6BGs2AD5LLnEsmSQ80wQ5ogcBBKhU86Y= cloud.google.com/go/storage v1.29.0/go.mod h1:4puEjyTKnku6gfKoTfNOU/W+a9JyuVNxjpS5GBrB8h4= -cloud.google.com/go/storage v1.52.0 h1:ROpzMW/IwipKtatA69ikxibdzQSiXJrY9f6IgBa9AlA= -cloud.google.com/go/storage v1.52.0/go.mod h1:4wrBAbAYUvYkbrf19ahGm4I5kDQhESSqN3CGEkMGvOY= +cloud.google.com/go/storage v1.53.0 h1:gg0ERZwL17pJ+Cz3cD2qS60w1WMDnwcm5YPAIQBHUAw= +cloud.google.com/go/storage v1.53.0/go.mod h1:7/eO2a/srr9ImZW9k5uufcNahT2+fPb8w5it1i5boaA= cloud.google.com/go/storagetransfer v1.5.0/go.mod h1:dxNzUopWy7RQevYFHewchb29POFv3/AaBgnhqzqiK0w= cloud.google.com/go/storagetransfer v1.6.0/go.mod h1:y77xm4CQV/ZhFZH75PLEXY0ROiS7Gh6pSKrM8dJyg6I= cloud.google.com/go/storagetransfer v1.7.0/go.mod h1:8Giuj1QNb1kfLAiWM1bN6dHzfdlDAVC9rv9abHot2W4= From 5f3b85514971b9850639b86d81f3b9f1d790320e Mon Sep 17 00:00:00 2001 From: claudevdm <33973061+claudevdm@users.noreply.github.com> Date: Tue, 3 Jun 2025 15:01:55 -0400 Subject: [PATCH 26/49] Add known issues. (#35138) * Add known issues. * Add fixes notes. --------- Co-authored-by: Claude --- CHANGES.md | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/CHANGES.md b/CHANGES.md index 4d4aee7ed25d..dd5bcce0c64b 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -99,6 +99,8 @@ * Fixed X (Java/Python) ([#X](https://github.com/apache/beam/issues/X)). * (Java) Fixed cassandraIO ReadAll does not let a pipeline handle or retry exceptions ([#34191](https://github.com/apache/beam/pull/34191)). +* [Python] Fixed vLLM breaks dataflow logging. ([35053](https://github.com/apache/beam/pull/35053)) +* [Python] Fixed vLLM leaks connections causing a throughput bottleneck and underutilization of GPU ([35053](https://github.com/apache/beam/pull/35053)) ## 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)). @@ -159,7 +161,9 @@ ## Known Issues -N/A +* [Python] GroupIntoBatches may fail in streaming pipelines. This is caused by cloudpickle. To mitigate this issue specify `pickle_library=dill` in pipeline options ([#35062](https://github.com/apache/beam/issues/35062)) +* [Python] vLLM breaks dataflow logging. To mitigate this issue, set the `VLLM_CONFIGURE_LOGGING=0` environment variable in your custom container. +* [Python] vLLM leaks connections causing a throughput bottleneck and underutilization of GPU. To mitigate this issue increase the number of `number_of_worker_harness_threads`. # [2.64.0] - 2025-03-31 From d1119ed16321ca16d8dc8dee88adacc9a26f000d Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 3 Jun 2025 15:16:04 -0400 Subject: [PATCH 27/49] Bump @octokit/plugin-paginate-rest and @octokit/rest (#34167) Bumps [@octokit/plugin-paginate-rest](https://github.com/octokit/plugin-paginate-rest.js) to 11.4.3 and updates ancestor dependency [@octokit/rest](https://github.com/octokit/rest.js). These dependencies need to be updated together. Updates `@octokit/plugin-paginate-rest` from 2.17.0 to 11.4.3 - [Release notes](https://github.com/octokit/plugin-paginate-rest.js/releases) - [Commits](https://github.com/octokit/plugin-paginate-rest.js/compare/v2.17.0...v11.4.3) Updates `@octokit/rest` from 18.12.0 to 21.1.1 - [Release notes](https://github.com/octokit/rest.js/releases) - [Commits](https://github.com/octokit/rest.js/compare/v18.12.0...v21.1.1) --- updated-dependencies: - dependency-name: "@octokit/plugin-paginate-rest" dependency-type: indirect - dependency-name: "@octokit/rest" dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- scripts/ci/issue-report/package-lock.json | 465 ++++++++++------------ scripts/ci/issue-report/package.json | 2 +- 2 files changed, 201 insertions(+), 266 deletions(-) diff --git a/scripts/ci/issue-report/package-lock.json b/scripts/ci/issue-report/package-lock.json index 8d1723290aff..a4499bd7e2da 100644 --- a/scripts/ci/issue-report/package-lock.json +++ b/scripts/ci/issue-report/package-lock.json @@ -5,166 +5,185 @@ "packages": { "": { "dependencies": { - "@octokit/rest": "^18.12.0", + "@octokit/rest": "^21.1.1", "nodemailer": "^6.7.5" } }, "node_modules/@octokit/auth-token": { - "version": "2.5.0", - "resolved": "https://registry.npmjs.org/@octokit/auth-token/-/auth-token-2.5.0.tgz", - "integrity": "sha512-r5FVUJCOLl19AxiuZD2VRZ/ORjp/4IN98Of6YJoJOkY75CIBuYfmiNHGrDwXr+aLGG55igl9QrxX3hbiXlLb+g==", - "dependencies": { - "@octokit/types": "^6.0.3" + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/@octokit/auth-token/-/auth-token-5.1.2.tgz", + "integrity": "sha512-JcQDsBdg49Yky2w2ld20IHAlwr8d/d8N6NiOXbtuoPCqzbsiJgF633mVUw3x4mo0H5ypataQIX7SFu3yy44Mpw==", + "license": "MIT", + "engines": { + "node": ">= 18" } }, "node_modules/@octokit/core": { - "version": "3.6.0", - "resolved": "https://registry.npmjs.org/@octokit/core/-/core-3.6.0.tgz", - "integrity": "sha512-7RKRKuA4xTjMhY+eG3jthb3hlZCsOwg3rztWh75Xc+ShDWOfDDATWbeZpAHBNRpm4Tv9WgBMOy1zEJYXG6NJ7Q==", + "version": "6.1.4", + "resolved": "https://registry.npmjs.org/@octokit/core/-/core-6.1.4.tgz", + "integrity": "sha512-lAS9k7d6I0MPN+gb9bKDt7X8SdxknYqAMh44S5L+lNqIN2NuV8nvv3g8rPp7MuRxcOpxpUIATWprO0C34a8Qmg==", + "license": "MIT", "dependencies": { - "@octokit/auth-token": "^2.4.4", - "@octokit/graphql": "^4.5.8", - "@octokit/request": "^5.6.3", - "@octokit/request-error": "^2.0.5", - "@octokit/types": "^6.0.3", - "before-after-hook": "^2.2.0", - "universal-user-agent": "^6.0.0" + "@octokit/auth-token": "^5.0.0", + "@octokit/graphql": "^8.1.2", + "@octokit/request": "^9.2.1", + "@octokit/request-error": "^6.1.7", + "@octokit/types": "^13.6.2", + "before-after-hook": "^3.0.2", + "universal-user-agent": "^7.0.0" + }, + "engines": { + "node": ">= 18" } }, "node_modules/@octokit/endpoint": { - "version": "6.0.12", - "resolved": "https://registry.npmjs.org/@octokit/endpoint/-/endpoint-6.0.12.tgz", - "integrity": "sha512-lF3puPwkQWGfkMClXb4k/eUT/nZKQfxinRWJrdZaJO85Dqwo/G0yOC434Jr2ojwafWJMYqFGFa5ms4jJUgujdA==", + "version": "10.1.3", + "resolved": "https://registry.npmjs.org/@octokit/endpoint/-/endpoint-10.1.3.tgz", + "integrity": "sha512-nBRBMpKPhQUxCsQQeW+rCJ/OPSMcj3g0nfHn01zGYZXuNDvvXudF/TYY6APj5THlurerpFN4a/dQAIAaM6BYhA==", + "license": "MIT", "dependencies": { - "@octokit/types": "^6.0.3", - "is-plain-object": "^5.0.0", - "universal-user-agent": "^6.0.0" + "@octokit/types": "^13.6.2", + "universal-user-agent": "^7.0.2" + }, + "engines": { + "node": ">= 18" } }, "node_modules/@octokit/graphql": { - "version": "4.8.0", - "resolved": "https://registry.npmjs.org/@octokit/graphql/-/graphql-4.8.0.tgz", - "integrity": "sha512-0gv+qLSBLKF0z8TKaSKTsS39scVKF9dbMxJpj3U0vC7wjNWFuIpL/z76Qe2fiuCbDRcJSavkXsVtMS6/dtQQsg==", + "version": "8.2.1", + "resolved": "https://registry.npmjs.org/@octokit/graphql/-/graphql-8.2.1.tgz", + "integrity": "sha512-n57hXtOoHrhwTWdvhVkdJHdhTv0JstjDbDRhJfwIRNfFqmSo1DaK/mD2syoNUoLCyqSjBpGAKOG0BuwF392slw==", + "license": "MIT", "dependencies": { - "@octokit/request": "^5.6.0", - "@octokit/types": "^6.0.3", - "universal-user-agent": "^6.0.0" + "@octokit/request": "^9.2.2", + "@octokit/types": "^13.8.0", + "universal-user-agent": "^7.0.0" + }, + "engines": { + "node": ">= 18" } }, "node_modules/@octokit/openapi-types": { - "version": "11.2.0", - "resolved": "https://registry.npmjs.org/@octokit/openapi-types/-/openapi-types-11.2.0.tgz", - "integrity": "sha512-PBsVO+15KSlGmiI8QAzaqvsNlZlrDlyAJYcrXBCvVUxCp7VnXjkwPoFHgjEJXx3WF9BAwkA6nfCUA7i9sODzKA==" + "version": "23.0.1", + "resolved": "https://registry.npmjs.org/@octokit/openapi-types/-/openapi-types-23.0.1.tgz", + "integrity": "sha512-izFjMJ1sir0jn0ldEKhZ7xegCTj/ObmEDlEfpFrx4k/JyZSMRHbO3/rBwgE7f3m2DHt+RrNGIVw4wSmwnm3t/g==", + "license": "MIT" }, "node_modules/@octokit/plugin-paginate-rest": { - "version": "2.17.0", - "resolved": "https://registry.npmjs.org/@octokit/plugin-paginate-rest/-/plugin-paginate-rest-2.17.0.tgz", - "integrity": "sha512-tzMbrbnam2Mt4AhuyCHvpRkS0oZ5MvwwcQPYGtMv4tUa5kkzG58SVB0fcsLulOZQeRnOgdkZWkRUiyBlh0Bkyw==", + "version": "11.4.3", + "resolved": "https://registry.npmjs.org/@octokit/plugin-paginate-rest/-/plugin-paginate-rest-11.4.3.tgz", + "integrity": "sha512-tBXaAbXkqVJlRoA/zQVe9mUdb8rScmivqtpv3ovsC5xhje/a+NOCivs7eUhWBwCApJVsR4G5HMeaLbq7PxqZGA==", + "license": "MIT", "dependencies": { - "@octokit/types": "^6.34.0" + "@octokit/types": "^13.7.0" + }, + "engines": { + "node": ">= 18" }, "peerDependencies": { - "@octokit/core": ">=2" + "@octokit/core": ">=6" } }, "node_modules/@octokit/plugin-request-log": { - "version": "1.0.4", - "resolved": "https://registry.npmjs.org/@octokit/plugin-request-log/-/plugin-request-log-1.0.4.tgz", - "integrity": "sha512-mLUsMkgP7K/cnFEw07kWqXGF5LKrOkD+lhCrKvPHXWDywAwuDUeDwWBpc69XK3pNX0uKiVt8g5z96PJ6z9xCFA==", + "version": "5.3.1", + "resolved": "https://registry.npmjs.org/@octokit/plugin-request-log/-/plugin-request-log-5.3.1.tgz", + "integrity": "sha512-n/lNeCtq+9ofhC15xzmJCNKP2BWTv8Ih2TTy+jatNCCq/gQP/V7rK3fjIfuz0pDWDALO/o/4QY4hyOF6TQQFUw==", + "license": "MIT", + "engines": { + "node": ">= 18" + }, "peerDependencies": { - "@octokit/core": ">=3" + "@octokit/core": ">=6" } }, "node_modules/@octokit/plugin-rest-endpoint-methods": { - "version": "5.13.0", - "resolved": "https://registry.npmjs.org/@octokit/plugin-rest-endpoint-methods/-/plugin-rest-endpoint-methods-5.13.0.tgz", - "integrity": "sha512-uJjMTkN1KaOIgNtUPMtIXDOjx6dGYysdIFhgA52x4xSadQCz3b/zJexvITDVpANnfKPW/+E0xkOvLntqMYpviA==", + "version": "13.3.1", + "resolved": "https://registry.npmjs.org/@octokit/plugin-rest-endpoint-methods/-/plugin-rest-endpoint-methods-13.3.1.tgz", + "integrity": "sha512-o8uOBdsyR+WR8MK9Cco8dCgvG13H1RlM1nWnK/W7TEACQBFux/vPREgKucxUfuDQ5yi1T3hGf4C5ZmZXAERgwQ==", + "license": "MIT", "dependencies": { - "@octokit/types": "^6.34.0", - "deprecation": "^2.3.1" + "@octokit/types": "^13.8.0" + }, + "engines": { + "node": ">= 18" }, "peerDependencies": { - "@octokit/core": ">=3" + "@octokit/core": ">=6" } }, "node_modules/@octokit/request": { - "version": "5.6.3", - "resolved": "https://registry.npmjs.org/@octokit/request/-/request-5.6.3.tgz", - "integrity": "sha512-bFJl0I1KVc9jYTe9tdGGpAMPy32dLBXXo1dS/YwSCTL/2nd9XeHsY616RE3HPXDVk+a+dBuzyz5YdlXwcDTr2A==", + "version": "9.2.2", + "resolved": "https://registry.npmjs.org/@octokit/request/-/request-9.2.2.tgz", + "integrity": "sha512-dZl0ZHx6gOQGcffgm1/Sf6JfEpmh34v3Af2Uci02vzUYz6qEN6zepoRtmybWXIGXFIK8K9ylE3b+duCWqhArtg==", + "license": "MIT", "dependencies": { - "@octokit/endpoint": "^6.0.1", - "@octokit/request-error": "^2.1.0", - "@octokit/types": "^6.16.1", - "is-plain-object": "^5.0.0", - "node-fetch": "^2.6.7", - "universal-user-agent": "^6.0.0" + "@octokit/endpoint": "^10.1.3", + "@octokit/request-error": "^6.1.7", + "@octokit/types": "^13.6.2", + "fast-content-type-parse": "^2.0.0", + "universal-user-agent": "^7.0.2" + }, + "engines": { + "node": ">= 18" } }, "node_modules/@octokit/request-error": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/@octokit/request-error/-/request-error-2.1.0.tgz", - "integrity": "sha512-1VIvgXxs9WHSjicsRwq8PlR2LR2x6DwsJAaFgzdi0JfJoGSO8mYI/cHJQ+9FbN21aa+DrgNLnwObmyeSC8Rmpg==", + "version": "6.1.7", + "resolved": "https://registry.npmjs.org/@octokit/request-error/-/request-error-6.1.7.tgz", + "integrity": "sha512-69NIppAwaauwZv6aOzb+VVLwt+0havz9GT5YplkeJv7fG7a40qpLt/yZKyiDxAhgz0EtgNdNcb96Z0u+Zyuy2g==", + "license": "MIT", "dependencies": { - "@octokit/types": "^6.0.3", - "deprecation": "^2.0.0", - "once": "^1.4.0" + "@octokit/types": "^13.6.2" + }, + "engines": { + "node": ">= 18" } }, "node_modules/@octokit/rest": { - "version": "18.12.0", - "resolved": "https://registry.npmjs.org/@octokit/rest/-/rest-18.12.0.tgz", - "integrity": "sha512-gDPiOHlyGavxr72y0guQEhLsemgVjwRePayJ+FcKc2SJqKUbxbkvf5kAZEWA/MKvsfYlQAMVzNJE3ezQcxMJ2Q==", + "version": "21.1.1", + "resolved": "https://registry.npmjs.org/@octokit/rest/-/rest-21.1.1.tgz", + "integrity": "sha512-sTQV7va0IUVZcntzy1q3QqPm/r8rWtDCqpRAmb8eXXnKkjoQEtFe3Nt5GTVsHft+R6jJoHeSiVLcgcvhtue/rg==", + "license": "MIT", "dependencies": { - "@octokit/core": "^3.5.1", - "@octokit/plugin-paginate-rest": "^2.16.8", - "@octokit/plugin-request-log": "^1.0.4", - "@octokit/plugin-rest-endpoint-methods": "^5.12.0" + "@octokit/core": "^6.1.4", + "@octokit/plugin-paginate-rest": "^11.4.2", + "@octokit/plugin-request-log": "^5.3.1", + "@octokit/plugin-rest-endpoint-methods": "^13.3.0" + }, + "engines": { + "node": ">= 18" } }, "node_modules/@octokit/types": { - "version": "6.34.0", - "resolved": "https://registry.npmjs.org/@octokit/types/-/types-6.34.0.tgz", - "integrity": "sha512-s1zLBjWhdEI2zwaoSgyOFoKSl109CUcVBCc7biPJ3aAf6LGLU6szDvi31JPU7bxfla2lqfhjbbg/5DdFNxOwHw==", + "version": "13.8.0", + "resolved": "https://registry.npmjs.org/@octokit/types/-/types-13.8.0.tgz", + "integrity": "sha512-x7DjTIbEpEWXK99DMd01QfWy0hd5h4EN+Q7shkdKds3otGQP+oWE/y0A76i1OvH9fygo4ddvNf7ZvF0t78P98A==", + "license": "MIT", "dependencies": { - "@octokit/openapi-types": "^11.2.0" + "@octokit/openapi-types": "^23.0.1" } }, "node_modules/before-after-hook": { - "version": "2.2.2", - "resolved": "https://registry.npmjs.org/before-after-hook/-/before-after-hook-2.2.2.tgz", - "integrity": "sha512-3pZEU3NT5BFUo/AD5ERPWOgQOCZITni6iavr5AUw5AUwQjMlI0kzu5btnyD39AF0gUEsDPwJT+oY1ORBJijPjQ==" - }, - "node_modules/deprecation": { - "version": "2.3.1", - "resolved": "https://registry.npmjs.org/deprecation/-/deprecation-2.3.1.tgz", - "integrity": "sha512-xmHIy4F3scKVwMsQ4WnVaS8bHOx0DmVwRywosKhaILI0ywMDWPtBSku2HNxRvF7jtwDRsoEwYQSfbxj8b7RlJQ==" - }, - "node_modules/is-plain-object": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/is-plain-object/-/is-plain-object-5.0.0.tgz", - "integrity": "sha512-VRSzKkbMm5jMDoKLbltAkFQ5Qr7VDiTFGXxYFXXowVj387GeGNOCsOH6Msy00SGZ3Fp84b1Naa1psqgcCIEP5Q==", - "engines": { - "node": ">=0.10.0" - } - }, - "node_modules/node-fetch": { - "version": "2.6.7", - "resolved": "https://registry.npmjs.org/node-fetch/-/node-fetch-2.6.7.tgz", - "integrity": "sha512-ZjMPFEfVx5j+y2yF35Kzx5sF7kDzxuDj6ziH4FFbOp87zKDZNx8yExJIb05OGF4Nlt9IHFIMBkRl41VdvcNdbQ==", - "dependencies": { - "whatwg-url": "^5.0.0" - }, - "engines": { - "node": "4.x || >=6.0.0" - }, - "peerDependencies": { - "encoding": "^0.1.0" - }, - "peerDependenciesMeta": { - "encoding": { - "optional": true + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/before-after-hook/-/before-after-hook-3.0.2.tgz", + "integrity": "sha512-Nik3Sc0ncrMK4UUdXQmAnRtzmNQTAAXmXIopizwZ1W1t8QmfJj+zL4OA2I7XPTPW5z5TDqv4hRo/JzouDJnX3A==", + "license": "Apache-2.0" + }, + "node_modules/fast-content-type-parse": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/fast-content-type-parse/-/fast-content-type-parse-2.0.1.tgz", + "integrity": "sha512-nGqtvLrj5w0naR6tDPfB4cUmYCqouzyQiz6C5y/LtcDllJdrcc6WaWW6iXyIIOErTa/XRybj28aasdn4LkVk6Q==", + "funding": [ + { + "type": "github", + "url": "https://github.com/sponsors/fastify" + }, + { + "type": "opencollective", + "url": "https://opencollective.com/fastify" } - } + ], + "license": "MIT" }, "node_modules/nodemailer": { "version": "6.7.5", @@ -174,221 +193,137 @@ "node": ">=6.0.0" } }, - "node_modules/once": { - "version": "1.4.0", - "resolved": "https://registry.npmjs.org/once/-/once-1.4.0.tgz", - "integrity": "sha512-lNaJgI+2Q5URQBkccEKHTQOPaXdUxnZZElQTZY0MFUAuaEqe1E+Nyvgdz/aIyNi6Z9MzO5dv1H8n58/GELp3+w==", - "dependencies": { - "wrappy": "1" - } - }, - "node_modules/tr46": { - "version": "0.0.3", - "resolved": "https://registry.npmjs.org/tr46/-/tr46-0.0.3.tgz", - "integrity": "sha1-gYT9NH2snNwYWZLzpmIuFLnZq2o=" - }, "node_modules/universal-user-agent": { - "version": "6.0.0", - "resolved": "https://registry.npmjs.org/universal-user-agent/-/universal-user-agent-6.0.0.tgz", - "integrity": "sha512-isyNax3wXoKaulPDZWHQqbmIx1k2tb9fb3GGDBRxCscfYV2Ch7WxPArBsFEG8s/safwXTT7H4QGhaIkTp9447w==" - }, - "node_modules/webidl-conversions": { - "version": "3.0.1", - "resolved": "https://registry.npmjs.org/webidl-conversions/-/webidl-conversions-3.0.1.tgz", - "integrity": "sha1-JFNCdeKnvGvnvIZhHMFq4KVlSHE=" - }, - "node_modules/whatwg-url": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/whatwg-url/-/whatwg-url-5.0.0.tgz", - "integrity": "sha1-lmRU6HZUYuN2RNNib2dCzotwll0=", - "dependencies": { - "tr46": "~0.0.3", - "webidl-conversions": "^3.0.0" - } - }, - "node_modules/wrappy": { - "version": "1.0.2", - "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz", - "integrity": "sha1-tSQ9jz7BqjXxNkYFvA0QNuMKtp8=" + "version": "7.0.2", + "resolved": "https://registry.npmjs.org/universal-user-agent/-/universal-user-agent-7.0.2.tgz", + "integrity": "sha512-0JCqzSKnStlRRQfCdowvqy3cy0Dvtlb8xecj/H8JFZuCze4rwjPZQOgvFvn0Ws/usCHQFGpyr+pB9adaGwXn4Q==", + "license": "ISC" } }, "dependencies": { "@octokit/auth-token": { - "version": "2.5.0", - "resolved": "https://registry.npmjs.org/@octokit/auth-token/-/auth-token-2.5.0.tgz", - "integrity": "sha512-r5FVUJCOLl19AxiuZD2VRZ/ORjp/4IN98Of6YJoJOkY75CIBuYfmiNHGrDwXr+aLGG55igl9QrxX3hbiXlLb+g==", - "requires": { - "@octokit/types": "^6.0.3" - } + "version": "5.1.2", + "resolved": "https://registry.npmjs.org/@octokit/auth-token/-/auth-token-5.1.2.tgz", + "integrity": "sha512-JcQDsBdg49Yky2w2ld20IHAlwr8d/d8N6NiOXbtuoPCqzbsiJgF633mVUw3x4mo0H5ypataQIX7SFu3yy44Mpw==" }, "@octokit/core": { - "version": "3.6.0", - "resolved": "https://registry.npmjs.org/@octokit/core/-/core-3.6.0.tgz", - "integrity": "sha512-7RKRKuA4xTjMhY+eG3jthb3hlZCsOwg3rztWh75Xc+ShDWOfDDATWbeZpAHBNRpm4Tv9WgBMOy1zEJYXG6NJ7Q==", + "version": "6.1.4", + "resolved": "https://registry.npmjs.org/@octokit/core/-/core-6.1.4.tgz", + "integrity": "sha512-lAS9k7d6I0MPN+gb9bKDt7X8SdxknYqAMh44S5L+lNqIN2NuV8nvv3g8rPp7MuRxcOpxpUIATWprO0C34a8Qmg==", "requires": { - "@octokit/auth-token": "^2.4.4", - "@octokit/graphql": "^4.5.8", - "@octokit/request": "^5.6.3", - "@octokit/request-error": "^2.0.5", - "@octokit/types": "^6.0.3", - "before-after-hook": "^2.2.0", - "universal-user-agent": "^6.0.0" + "@octokit/auth-token": "^5.0.0", + "@octokit/graphql": "^8.1.2", + "@octokit/request": "^9.2.1", + "@octokit/request-error": "^6.1.7", + "@octokit/types": "^13.6.2", + "before-after-hook": "^3.0.2", + "universal-user-agent": "^7.0.0" } }, "@octokit/endpoint": { - "version": "6.0.12", - "resolved": "https://registry.npmjs.org/@octokit/endpoint/-/endpoint-6.0.12.tgz", - "integrity": "sha512-lF3puPwkQWGfkMClXb4k/eUT/nZKQfxinRWJrdZaJO85Dqwo/G0yOC434Jr2ojwafWJMYqFGFa5ms4jJUgujdA==", + "version": "10.1.3", + "resolved": "https://registry.npmjs.org/@octokit/endpoint/-/endpoint-10.1.3.tgz", + "integrity": "sha512-nBRBMpKPhQUxCsQQeW+rCJ/OPSMcj3g0nfHn01zGYZXuNDvvXudF/TYY6APj5THlurerpFN4a/dQAIAaM6BYhA==", "requires": { - "@octokit/types": "^6.0.3", - "is-plain-object": "^5.0.0", - "universal-user-agent": "^6.0.0" + "@octokit/types": "^13.6.2", + "universal-user-agent": "^7.0.2" } }, "@octokit/graphql": { - "version": "4.8.0", - "resolved": "https://registry.npmjs.org/@octokit/graphql/-/graphql-4.8.0.tgz", - "integrity": "sha512-0gv+qLSBLKF0z8TKaSKTsS39scVKF9dbMxJpj3U0vC7wjNWFuIpL/z76Qe2fiuCbDRcJSavkXsVtMS6/dtQQsg==", + "version": "8.2.1", + "resolved": "https://registry.npmjs.org/@octokit/graphql/-/graphql-8.2.1.tgz", + "integrity": "sha512-n57hXtOoHrhwTWdvhVkdJHdhTv0JstjDbDRhJfwIRNfFqmSo1DaK/mD2syoNUoLCyqSjBpGAKOG0BuwF392slw==", "requires": { - "@octokit/request": "^5.6.0", - "@octokit/types": "^6.0.3", - "universal-user-agent": "^6.0.0" + "@octokit/request": "^9.2.2", + "@octokit/types": "^13.8.0", + "universal-user-agent": "^7.0.0" } }, "@octokit/openapi-types": { - "version": "11.2.0", - "resolved": "https://registry.npmjs.org/@octokit/openapi-types/-/openapi-types-11.2.0.tgz", - "integrity": "sha512-PBsVO+15KSlGmiI8QAzaqvsNlZlrDlyAJYcrXBCvVUxCp7VnXjkwPoFHgjEJXx3WF9BAwkA6nfCUA7i9sODzKA==" + "version": "23.0.1", + "resolved": "https://registry.npmjs.org/@octokit/openapi-types/-/openapi-types-23.0.1.tgz", + "integrity": "sha512-izFjMJ1sir0jn0ldEKhZ7xegCTj/ObmEDlEfpFrx4k/JyZSMRHbO3/rBwgE7f3m2DHt+RrNGIVw4wSmwnm3t/g==" }, "@octokit/plugin-paginate-rest": { - "version": "2.17.0", - "resolved": "https://registry.npmjs.org/@octokit/plugin-paginate-rest/-/plugin-paginate-rest-2.17.0.tgz", - "integrity": "sha512-tzMbrbnam2Mt4AhuyCHvpRkS0oZ5MvwwcQPYGtMv4tUa5kkzG58SVB0fcsLulOZQeRnOgdkZWkRUiyBlh0Bkyw==", + "version": "11.4.3", + "resolved": "https://registry.npmjs.org/@octokit/plugin-paginate-rest/-/plugin-paginate-rest-11.4.3.tgz", + "integrity": "sha512-tBXaAbXkqVJlRoA/zQVe9mUdb8rScmivqtpv3ovsC5xhje/a+NOCivs7eUhWBwCApJVsR4G5HMeaLbq7PxqZGA==", "requires": { - "@octokit/types": "^6.34.0" + "@octokit/types": "^13.7.0" } }, "@octokit/plugin-request-log": { - "version": "1.0.4", - "resolved": "https://registry.npmjs.org/@octokit/plugin-request-log/-/plugin-request-log-1.0.4.tgz", - "integrity": "sha512-mLUsMkgP7K/cnFEw07kWqXGF5LKrOkD+lhCrKvPHXWDywAwuDUeDwWBpc69XK3pNX0uKiVt8g5z96PJ6z9xCFA==", + "version": "5.3.1", + "resolved": "https://registry.npmjs.org/@octokit/plugin-request-log/-/plugin-request-log-5.3.1.tgz", + "integrity": "sha512-n/lNeCtq+9ofhC15xzmJCNKP2BWTv8Ih2TTy+jatNCCq/gQP/V7rK3fjIfuz0pDWDALO/o/4QY4hyOF6TQQFUw==", "requires": {} }, "@octokit/plugin-rest-endpoint-methods": { - "version": "5.13.0", - "resolved": "https://registry.npmjs.org/@octokit/plugin-rest-endpoint-methods/-/plugin-rest-endpoint-methods-5.13.0.tgz", - "integrity": "sha512-uJjMTkN1KaOIgNtUPMtIXDOjx6dGYysdIFhgA52x4xSadQCz3b/zJexvITDVpANnfKPW/+E0xkOvLntqMYpviA==", + "version": "13.3.1", + "resolved": "https://registry.npmjs.org/@octokit/plugin-rest-endpoint-methods/-/plugin-rest-endpoint-methods-13.3.1.tgz", + "integrity": "sha512-o8uOBdsyR+WR8MK9Cco8dCgvG13H1RlM1nWnK/W7TEACQBFux/vPREgKucxUfuDQ5yi1T3hGf4C5ZmZXAERgwQ==", "requires": { - "@octokit/types": "^6.34.0", - "deprecation": "^2.3.1" + "@octokit/types": "^13.8.0" } }, "@octokit/request": { - "version": "5.6.3", - "resolved": "https://registry.npmjs.org/@octokit/request/-/request-5.6.3.tgz", - "integrity": "sha512-bFJl0I1KVc9jYTe9tdGGpAMPy32dLBXXo1dS/YwSCTL/2nd9XeHsY616RE3HPXDVk+a+dBuzyz5YdlXwcDTr2A==", + "version": "9.2.2", + "resolved": "https://registry.npmjs.org/@octokit/request/-/request-9.2.2.tgz", + "integrity": "sha512-dZl0ZHx6gOQGcffgm1/Sf6JfEpmh34v3Af2Uci02vzUYz6qEN6zepoRtmybWXIGXFIK8K9ylE3b+duCWqhArtg==", "requires": { - "@octokit/endpoint": "^6.0.1", - "@octokit/request-error": "^2.1.0", - "@octokit/types": "^6.16.1", - "is-plain-object": "^5.0.0", - "node-fetch": "^2.6.7", - "universal-user-agent": "^6.0.0" + "@octokit/endpoint": "^10.1.3", + "@octokit/request-error": "^6.1.7", + "@octokit/types": "^13.6.2", + "fast-content-type-parse": "^2.0.0", + "universal-user-agent": "^7.0.2" } }, "@octokit/request-error": { - "version": "2.1.0", - "resolved": "https://registry.npmjs.org/@octokit/request-error/-/request-error-2.1.0.tgz", - "integrity": "sha512-1VIvgXxs9WHSjicsRwq8PlR2LR2x6DwsJAaFgzdi0JfJoGSO8mYI/cHJQ+9FbN21aa+DrgNLnwObmyeSC8Rmpg==", + "version": "6.1.7", + "resolved": "https://registry.npmjs.org/@octokit/request-error/-/request-error-6.1.7.tgz", + "integrity": "sha512-69NIppAwaauwZv6aOzb+VVLwt+0havz9GT5YplkeJv7fG7a40qpLt/yZKyiDxAhgz0EtgNdNcb96Z0u+Zyuy2g==", "requires": { - "@octokit/types": "^6.0.3", - "deprecation": "^2.0.0", - "once": "^1.4.0" + "@octokit/types": "^13.6.2" } }, "@octokit/rest": { - "version": "18.12.0", - "resolved": "https://registry.npmjs.org/@octokit/rest/-/rest-18.12.0.tgz", - "integrity": "sha512-gDPiOHlyGavxr72y0guQEhLsemgVjwRePayJ+FcKc2SJqKUbxbkvf5kAZEWA/MKvsfYlQAMVzNJE3ezQcxMJ2Q==", + "version": "21.1.1", + "resolved": "https://registry.npmjs.org/@octokit/rest/-/rest-21.1.1.tgz", + "integrity": "sha512-sTQV7va0IUVZcntzy1q3QqPm/r8rWtDCqpRAmb8eXXnKkjoQEtFe3Nt5GTVsHft+R6jJoHeSiVLcgcvhtue/rg==", "requires": { - "@octokit/core": "^3.5.1", - "@octokit/plugin-paginate-rest": "^2.16.8", - "@octokit/plugin-request-log": "^1.0.4", - "@octokit/plugin-rest-endpoint-methods": "^5.12.0" + "@octokit/core": "^6.1.4", + "@octokit/plugin-paginate-rest": "^11.4.2", + "@octokit/plugin-request-log": "^5.3.1", + "@octokit/plugin-rest-endpoint-methods": "^13.3.0" } }, "@octokit/types": { - "version": "6.34.0", - "resolved": "https://registry.npmjs.org/@octokit/types/-/types-6.34.0.tgz", - "integrity": "sha512-s1zLBjWhdEI2zwaoSgyOFoKSl109CUcVBCc7biPJ3aAf6LGLU6szDvi31JPU7bxfla2lqfhjbbg/5DdFNxOwHw==", + "version": "13.8.0", + "resolved": "https://registry.npmjs.org/@octokit/types/-/types-13.8.0.tgz", + "integrity": "sha512-x7DjTIbEpEWXK99DMd01QfWy0hd5h4EN+Q7shkdKds3otGQP+oWE/y0A76i1OvH9fygo4ddvNf7ZvF0t78P98A==", "requires": { - "@octokit/openapi-types": "^11.2.0" + "@octokit/openapi-types": "^23.0.1" } }, "before-after-hook": { - "version": "2.2.2", - "resolved": "https://registry.npmjs.org/before-after-hook/-/before-after-hook-2.2.2.tgz", - "integrity": "sha512-3pZEU3NT5BFUo/AD5ERPWOgQOCZITni6iavr5AUw5AUwQjMlI0kzu5btnyD39AF0gUEsDPwJT+oY1ORBJijPjQ==" - }, - "deprecation": { - "version": "2.3.1", - "resolved": "https://registry.npmjs.org/deprecation/-/deprecation-2.3.1.tgz", - "integrity": "sha512-xmHIy4F3scKVwMsQ4WnVaS8bHOx0DmVwRywosKhaILI0ywMDWPtBSku2HNxRvF7jtwDRsoEwYQSfbxj8b7RlJQ==" + "version": "3.0.2", + "resolved": "https://registry.npmjs.org/before-after-hook/-/before-after-hook-3.0.2.tgz", + "integrity": "sha512-Nik3Sc0ncrMK4UUdXQmAnRtzmNQTAAXmXIopizwZ1W1t8QmfJj+zL4OA2I7XPTPW5z5TDqv4hRo/JzouDJnX3A==" }, - "is-plain-object": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/is-plain-object/-/is-plain-object-5.0.0.tgz", - "integrity": "sha512-VRSzKkbMm5jMDoKLbltAkFQ5Qr7VDiTFGXxYFXXowVj387GeGNOCsOH6Msy00SGZ3Fp84b1Naa1psqgcCIEP5Q==" - }, - "node-fetch": { - "version": "2.6.7", - "resolved": "https://registry.npmjs.org/node-fetch/-/node-fetch-2.6.7.tgz", - "integrity": "sha512-ZjMPFEfVx5j+y2yF35Kzx5sF7kDzxuDj6ziH4FFbOp87zKDZNx8yExJIb05OGF4Nlt9IHFIMBkRl41VdvcNdbQ==", - "requires": { - "whatwg-url": "^5.0.0" - } + "fast-content-type-parse": { + "version": "2.0.1", + "resolved": "https://registry.npmjs.org/fast-content-type-parse/-/fast-content-type-parse-2.0.1.tgz", + "integrity": "sha512-nGqtvLrj5w0naR6tDPfB4cUmYCqouzyQiz6C5y/LtcDllJdrcc6WaWW6iXyIIOErTa/XRybj28aasdn4LkVk6Q==" }, "nodemailer": { "version": "6.7.5", "resolved": "https://registry.npmjs.org/nodemailer/-/nodemailer-6.7.5.tgz", "integrity": "sha512-6VtMpwhsrixq1HDYSBBHvW0GwiWawE75dS3oal48VqRhUvKJNnKnJo2RI/bCVQubj1vgrgscMNW4DHaD6xtMCg==" }, - "once": { - "version": "1.4.0", - "resolved": "https://registry.npmjs.org/once/-/once-1.4.0.tgz", - "integrity": "sha512-lNaJgI+2Q5URQBkccEKHTQOPaXdUxnZZElQTZY0MFUAuaEqe1E+Nyvgdz/aIyNi6Z9MzO5dv1H8n58/GELp3+w==", - "requires": { - "wrappy": "1" - } - }, - "tr46": { - "version": "0.0.3", - "resolved": "https://registry.npmjs.org/tr46/-/tr46-0.0.3.tgz", - "integrity": "sha1-gYT9NH2snNwYWZLzpmIuFLnZq2o=" - }, "universal-user-agent": { - "version": "6.0.0", - "resolved": "https://registry.npmjs.org/universal-user-agent/-/universal-user-agent-6.0.0.tgz", - "integrity": "sha512-isyNax3wXoKaulPDZWHQqbmIx1k2tb9fb3GGDBRxCscfYV2Ch7WxPArBsFEG8s/safwXTT7H4QGhaIkTp9447w==" - }, - "webidl-conversions": { - "version": "3.0.1", - "resolved": "https://registry.npmjs.org/webidl-conversions/-/webidl-conversions-3.0.1.tgz", - "integrity": "sha1-JFNCdeKnvGvnvIZhHMFq4KVlSHE=" - }, - "whatwg-url": { - "version": "5.0.0", - "resolved": "https://registry.npmjs.org/whatwg-url/-/whatwg-url-5.0.0.tgz", - "integrity": "sha1-lmRU6HZUYuN2RNNib2dCzotwll0=", - "requires": { - "tr46": "~0.0.3", - "webidl-conversions": "^3.0.0" - } - }, - "wrappy": { - "version": "1.0.2", - "resolved": "https://registry.npmjs.org/wrappy/-/wrappy-1.0.2.tgz", - "integrity": "sha1-tSQ9jz7BqjXxNkYFvA0QNuMKtp8=" + "version": "7.0.2", + "resolved": "https://registry.npmjs.org/universal-user-agent/-/universal-user-agent-7.0.2.tgz", + "integrity": "sha512-0JCqzSKnStlRRQfCdowvqy3cy0Dvtlb8xecj/H8JFZuCze4rwjPZQOgvFvn0Ws/usCHQFGpyr+pB9adaGwXn4Q==" } } } diff --git a/scripts/ci/issue-report/package.json b/scripts/ci/issue-report/package.json index 035c089ebc7f..22fd31eca7ab 100644 --- a/scripts/ci/issue-report/package.json +++ b/scripts/ci/issue-report/package.json @@ -1,6 +1,6 @@ { "dependencies": { - "@octokit/rest": "^18.12.0", + "@octokit/rest": "^21.1.1", "nodemailer": "^6.7.5" } } From f69a4a7bc463af2ece64c91ed83f1ea87a42170c Mon Sep 17 00:00:00 2001 From: scwhittle Date: Tue, 3 Jun 2025 19:33:55 +0000 Subject: [PATCH 28/49] Explicitly handle singleton iterators instead of using helper and catching exceptions which may be from generating iterable (#35124) --- .../beam/sdk/values/PCollectionViews.java | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java index 6df68ea964d6..05612cb9fc77 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/PCollectionViews.java @@ -540,14 +540,16 @@ public Materialization> getMaterialization() { @Override public T apply(IterableView primitiveViewT) { - try { - return Iterables.getOnlyElement(primitiveViewT.get()); - } catch (NoSuchElementException exc) { + Iterator iterator = primitiveViewT.get().iterator(); + if (!iterator.hasNext()) { return getDefaultValue(); - } catch (IllegalArgumentException exc) { + } + T result = iterator.next(); + if (iterator.hasNext()) { throw new IllegalArgumentException( "PCollection with more than one element accessed as a singleton view."); } + return result; } @Override @@ -638,14 +640,16 @@ public Materialization> getMaterialization() { @Override public T apply(MultimapView primitiveViewT) { - try { - return Iterables.getOnlyElement(primitiveViewT.get(null)); - } catch (NoSuchElementException exc) { + Iterator iterator = primitiveViewT.get(null).iterator(); + if (!iterator.hasNext()) { return getDefaultValue(); - } catch (IllegalArgumentException exc) { + } + T result = iterator.next(); + if (iterator.hasNext()) { throw new IllegalArgumentException( "PCollection with more than one element accessed as a singleton view."); } + return result; } @Override From 5e79d4b39cf9c0a8d3737952cb864db1d4005265 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 3 Jun 2025 16:20:02 -0400 Subject: [PATCH 29/49] Build last snapshot against RC00 tag instead of release branch (#35142) The last snapshot Beam Java SDK (aka "RC00" release) build is triggered manually, to verify a RC1 build will be successful. It has been built against release-2.xx branch, where the Dataflow container tag replaced from beam-master to the 2.xx.0 However, the versioned containers are not yet released, causing a timing gap that Beam 2.xx.0-SNAPSHOT won't work on Dataflow between release branch cut and RC1 rolled out to Dataflow Since we now have a v2.xx.0-RC00 tag, build RC00 against this tag resolves the issue. --- contributor-docs/release-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/contributor-docs/release-guide.md b/contributor-docs/release-guide.md index 53105184b7d8..555db8ffdcbf 100644 --- a/contributor-docs/release-guide.md +++ b/contributor-docs/release-guide.md @@ -232,7 +232,7 @@ The following must be manually done or confirmed: - [ ] 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 current limitation in the workflow, you must navigate to https://github.com/apache/beam/actions/workflows/beam_Release_NightlySnapshot.yml and click "Run workflow" and select the branch just created (release-2.xx) to build a snapshot. +- [ ] Due to current limitation in the workflow, you must navigate to https://github.com/apache/beam/actions/workflows/beam_Release_NightlySnapshot.yml and click "Run workflow" and select the **tag** just created (v2.xx.0-RC00) to build a snapshot. - [ ] Manually update `CHANGES.md` on `master` by adding a new section for the next release ([example](https://github.com/apache/beam/commit/96ab1fb3fe07acf7f7dc9d8c829ae36890d1535c)). From e0f7a072c7f1b0e7e96ffc6dc0b136595d7cf495 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 3 Jun 2025 16:46:55 -0400 Subject: [PATCH 30/49] Bump nodemailer from 6.7.5 to 6.9.9 in /scripts/ci/issue-report (#35143) Bumps [nodemailer](https://github.com/nodemailer/nodemailer) from 6.7.5 to 6.9.9. - [Release notes](https://github.com/nodemailer/nodemailer/releases) - [Changelog](https://github.com/nodemailer/nodemailer/blob/master/CHANGELOG.md) - [Commits](https://github.com/nodemailer/nodemailer/compare/v6.7.5...v6.9.9) --- updated-dependencies: - dependency-name: nodemailer dependency-version: 6.9.9 dependency-type: direct:production ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- scripts/ci/issue-report/package-lock.json | 15 ++++++++------- scripts/ci/issue-report/package.json | 2 +- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/scripts/ci/issue-report/package-lock.json b/scripts/ci/issue-report/package-lock.json index a4499bd7e2da..ae2ee083eec8 100644 --- a/scripts/ci/issue-report/package-lock.json +++ b/scripts/ci/issue-report/package-lock.json @@ -6,7 +6,7 @@ "": { "dependencies": { "@octokit/rest": "^21.1.1", - "nodemailer": "^6.7.5" + "nodemailer": "^6.9.9" } }, "node_modules/@octokit/auth-token": { @@ -186,9 +186,10 @@ "license": "MIT" }, "node_modules/nodemailer": { - "version": "6.7.5", - "resolved": "https://registry.npmjs.org/nodemailer/-/nodemailer-6.7.5.tgz", - "integrity": "sha512-6VtMpwhsrixq1HDYSBBHvW0GwiWawE75dS3oal48VqRhUvKJNnKnJo2RI/bCVQubj1vgrgscMNW4DHaD6xtMCg==", + "version": "6.9.9", + "resolved": "https://registry.npmjs.org/nodemailer/-/nodemailer-6.9.9.tgz", + "integrity": "sha512-dexTll8zqQoVJEZPwQAKzxxtFn0qTnjdQTchoU6Re9BUUGBJiOy3YMn/0ShTW6J5M0dfQ1NeDeRTTl4oIWgQMA==", + "license": "MIT-0", "engines": { "node": ">=6.0.0" } @@ -316,9 +317,9 @@ "integrity": "sha512-nGqtvLrj5w0naR6tDPfB4cUmYCqouzyQiz6C5y/LtcDllJdrcc6WaWW6iXyIIOErTa/XRybj28aasdn4LkVk6Q==" }, "nodemailer": { - "version": "6.7.5", - "resolved": "https://registry.npmjs.org/nodemailer/-/nodemailer-6.7.5.tgz", - "integrity": "sha512-6VtMpwhsrixq1HDYSBBHvW0GwiWawE75dS3oal48VqRhUvKJNnKnJo2RI/bCVQubj1vgrgscMNW4DHaD6xtMCg==" + "version": "6.9.9", + "resolved": "https://registry.npmjs.org/nodemailer/-/nodemailer-6.9.9.tgz", + "integrity": "sha512-dexTll8zqQoVJEZPwQAKzxxtFn0qTnjdQTchoU6Re9BUUGBJiOy3YMn/0ShTW6J5M0dfQ1NeDeRTTl4oIWgQMA==" }, "universal-user-agent": { "version": "7.0.2", diff --git a/scripts/ci/issue-report/package.json b/scripts/ci/issue-report/package.json index 22fd31eca7ab..55342e739685 100644 --- a/scripts/ci/issue-report/package.json +++ b/scripts/ci/issue-report/package.json @@ -1,6 +1,6 @@ { "dependencies": { "@octokit/rest": "^21.1.1", - "nodemailer": "^6.7.5" + "nodemailer": "^6.9.9" } } From 780a0fa350dd4e46728dcaab7b3ed302596d2027 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Tue, 3 Jun 2025 19:03:51 -0400 Subject: [PATCH 31/49] Fix tests affected by Java 8 container turn down (#35145) * Fix tests affected by Java 8 container turn down * still use Java 8 for Samza runner --- .github/workflows/beam_PostCommit_Go_VR_Samza.yml | 6 ++++-- runners/google-cloud-dataflow-java/build.gradle | 6 ++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml index 29a4f941adb4..a056d13e3e8c 100644 --- a/.github/workflows/beam_PostCommit_Go_VR_Samza.yml +++ b/.github/workflows/beam_PostCommit_Go_VR_Samza.yml @@ -74,10 +74,12 @@ jobs: uses: ./.github/actions/setup-environment-action # TODO(https://github.com/apache/beam/issues/32208) move to Java11 after bump to Samza 1.8 with: - java-version: 8 + java-version: | + 11 + 8 - name: run Go Samza ValidatesRunner script env: CLOUDSDK_CONFIG: ${{ env.KUBELET_GCLOUD_CONFIG_PATH}} uses: ./.github/actions/gradle-command-self-hosted-action with: - gradle-command: :sdks:go:test:samzaValidatesRunner \ No newline at end of file + gradle-command: :sdks:go:test:samzaValidatesRunner -Pjava11Home=$JAVA_HOME_11_X64 diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 63125a68efb2..bc69bae4a942 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -346,10 +346,8 @@ tasks.register('buildAndPushDistrolessContainerImage', Task.class) { // task directly ('dependsOn buildAndPushDockerJavaContainer'). This ensures the correct // task ordering such that the registry doesn't get cleaned up prior to task completion. def buildAndPushDockerJavaContainer = tasks.register("buildAndPushDockerJavaContainer") { - def javaVer = getSupportedJavaVersion() - if(project.hasProperty('testJavaVersion')) { - javaVer = "java${project.getProperty('testJavaVersion')}" - } + def javaVer = getSupportedJavaVersion(project.findProperty('testJavaVersion') as String) + dependsOn ":sdks:java:container:${javaVer}:docker" def defaultDockerImageName = containerImageName( name: "${project.docker_image_default_repo_prefix}${javaVer}_sdk", From 731001d5bb2fb6bed36e8c7a714ce80b98f054ca Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Wed, 4 Jun 2025 09:52:09 -0400 Subject: [PATCH 32/49] Bump github.com/aws/aws-sdk-go-v2/service/s3 in /sdks (#35146) Bumps [github.com/aws/aws-sdk-go-v2/service/s3](https://github.com/aws/aws-sdk-go-v2) from 1.79.3 to 1.80.0. - [Release notes](https://github.com/aws/aws-sdk-go-v2/releases) - [Changelog](https://github.com/aws/aws-sdk-go-v2/blob/main/changelog-template.json) - [Commits](https://github.com/aws/aws-sdk-go-v2/compare/service/s3/v1.79.3...service/s3/v1.80.0) --- updated-dependencies: - dependency-name: github.com/aws/aws-sdk-go-v2/service/s3 dependency-version: 1.80.0 dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] Co-authored-by: dependabot[bot] <49699333+dependabot[bot]@users.noreply.github.com> --- sdks/go.mod | 4 ++-- sdks/go.sum | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sdks/go.mod b/sdks/go.mod index 823a662a6c26..98b5272f3455 100644 --- a/sdks/go.mod +++ b/sdks/go.mod @@ -36,7 +36,7 @@ require ( github.com/aws/aws-sdk-go-v2/config v1.29.14 github.com/aws/aws-sdk-go-v2/credentials v1.17.67 github.com/aws/aws-sdk-go-v2/feature/s3/manager v1.17.74 - github.com/aws/aws-sdk-go-v2/service/s3 v1.79.3 + github.com/aws/aws-sdk-go-v2/service/s3 v1.80.0 github.com/aws/smithy-go v1.22.3 github.com/docker/go-connections v0.5.0 github.com/dustin/go-humanize v1.0.1 @@ -149,7 +149,7 @@ require ( github.com/aws/aws-sdk-go-v2/internal/ini v1.8.3 // indirect github.com/aws/aws-sdk-go-v2/internal/v4a v1.3.34 // indirect github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.3 // indirect - github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.7.1 // indirect + github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.7.2 // indirect github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.15 // indirect github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.15 // indirect github.com/aws/aws-sdk-go-v2/service/sso v1.25.3 // indirect diff --git a/sdks/go.sum b/sdks/go.sum index bdf56826061a..550a2959aca0 100644 --- a/sdks/go.sum +++ b/sdks/go.sum @@ -790,8 +790,8 @@ github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.3 h1:eAh2A4b github.com/aws/aws-sdk-go-v2/service/internal/accept-encoding v1.12.3/go.mod h1:0yKJC/kb8sAnmlYa6Zs3QVYqaC8ug2AbnNChv5Ox3uA= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.1.3/go.mod h1:Seb8KNmD6kVTjwRjVEgOT5hPin6sq+v4C2ycJQDwuH8= github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.2.3/go.mod h1:R+/S1O4TYpcktbVwddeOYg+uwUfLhADP2S/x4QwsCTM= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.7.1 h1:4nm2G6A4pV9rdlWzGMPv4BNtQp22v1hg3yrtkYpeLl8= -github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.7.1/go.mod h1:iu6FSzgt+M2/x3Dk8zhycdIcHjEFb36IS8HVUVFoMg0= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.7.2 h1:BCG7DCXEXpNCcpwCxg1oi9pkJWH2+eZzTn9MY56MbVw= +github.com/aws/aws-sdk-go-v2/service/internal/checksum v1.7.2/go.mod h1:iu6FSzgt+M2/x3Dk8zhycdIcHjEFb36IS8HVUVFoMg0= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.9.3/go.mod h1:wlY6SVjuwvh3TVRpTqdy4I1JpBFLX4UGeKZdWntaocw= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.10.3/go.mod h1:Owv1I59vaghv1Ax8zz8ELY8DN7/Y0rGS+WWAmjgi950= github.com/aws/aws-sdk-go-v2/service/internal/presigned-url v1.12.15 h1:dM9/92u2F1JbDaGooxTq18wmmFzbJRfXfVfy96/1CXM= @@ -803,8 +803,8 @@ github.com/aws/aws-sdk-go-v2/service/internal/s3shared v1.18.15/go.mod h1:ZH34PJ github.com/aws/aws-sdk-go-v2/service/kms v1.16.3/go.mod h1:QuiHPBqlOFCi4LqdSskYYAWpQlx3PKmohy+rE2F+o5g= github.com/aws/aws-sdk-go-v2/service/s3 v1.26.3/go.mod h1:g1qvDuRsJY+XghsV6zg00Z4KJ7DtFFCx8fJD2a491Ak= github.com/aws/aws-sdk-go-v2/service/s3 v1.43.0/go.mod h1:NXRKkiRF+erX2hnybnVU660cYT5/KChRD4iUgJ97cI8= -github.com/aws/aws-sdk-go-v2/service/s3 v1.79.3 h1:BRXS0U76Z8wfF+bnkilA2QwpIch6URlm++yPUt9QPmQ= -github.com/aws/aws-sdk-go-v2/service/s3 v1.79.3/go.mod h1:bNXKFFyaiVvWuR6O16h/I1724+aXe/tAkA9/QS01t5k= +github.com/aws/aws-sdk-go-v2/service/s3 v1.80.0 h1:fV4XIU5sn/x8gjRouoJpDVHj+ExJaUk4prYF+eb6qTs= +github.com/aws/aws-sdk-go-v2/service/s3 v1.80.0/go.mod h1:qbn305Je/IofWBJ4bJz/Q7pDEtnnoInw/dGt71v6rHE= github.com/aws/aws-sdk-go-v2/service/secretsmanager v1.15.4/go.mod h1:PJc8s+lxyU8rrre0/4a0pn2wgwiDvOEzoOjcJUBr67o= github.com/aws/aws-sdk-go-v2/service/sns v1.17.4/go.mod h1:kElt+uCcXxcqFyc+bQqZPFD9DME/eC6oHBXvFzQ9Bcw= github.com/aws/aws-sdk-go-v2/service/sqs v1.18.3/go.mod h1:skmQo0UPvsjsuYYSYMVmrPc1HWCbHUJyrCEp+ZaLzqM= From 3e8fcc3379f6884ba38c70e3c0f0fb274f3fe62f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Rados=C5=82aw=20Stankiewicz?= Date: Wed, 4 Jun 2025 15:56:20 +0200 Subject: [PATCH 33/49] fix jdbc transform validation (#35141) * fix jdbc transform validation * add test * annotations * spotless --- .../jdbc/JdbcReadSchemaTransformProvider.java | 2 +- .../JdbcReadSchemaTransformProviderTest.java | 46 +++++++++++++++++++ 2 files changed, 47 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java index b136b3e3e057..6777be50ab50 100644 --- a/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java +++ b/sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProvider.java @@ -214,7 +214,7 @@ protected JdbcIO.DataSourceConfiguration dataSourceConfiguration() { @Override public PCollectionRowTuple expand(PCollectionRowTuple input) { - config.validate(); + config.validate(jdbcType); // If we define a partition column, we follow a different route. @Nullable String partitionColumn = config.getPartitionColumn(); @Nullable String location = config.getLocation(); diff --git a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProviderTest.java b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProviderTest.java index 0837ea686ddf..23decba1a633 100644 --- a/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProviderTest.java +++ b/sdks/java/io/jdbc/src/test/java/org/apache/beam/sdk/io/jdbc/JdbcReadSchemaTransformProviderTest.java @@ -22,6 +22,7 @@ import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertThrows; +import com.google.auto.service.AutoService; import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.SQLException; @@ -47,6 +48,25 @@ @RunWith(JUnit4.class) public class JdbcReadSchemaTransformProviderTest { + @AutoService(SchemaTransformProvider.class) + public static class ReadFromDerbySchemaTransformProvider extends JdbcReadSchemaTransformProvider { + + @Override + public String identifier() { + return "beam:schematransform:org.apache.beam:derby_read:v1"; + } + + @Override + public String description() { + return inheritedDescription("Derby", "ReadFromDerby", "derby", 5432); + } + + @Override + protected String jdbcType() { + return "derby"; + } + } + private static final JdbcIO.DataSourceConfiguration DATA_SOURCE_CONFIGURATION = JdbcIO.DataSourceConfiguration.create( "org.apache.derby.jdbc.EmbeddedDriver", "jdbc:derby:memory:testDB;create=true"); @@ -199,6 +219,32 @@ public void testReadWithJdbcTypeSpecified() { pipeline.run(); } + @Test + public void testReadWithJdbcDerbyTransformTypeSpecified() { + JdbcReadSchemaTransformProvider provider = null; + for (SchemaTransformProvider p : ServiceLoader.load(SchemaTransformProvider.class)) { + if (p instanceof ReadFromDerbySchemaTransformProvider) { + provider = (JdbcReadSchemaTransformProvider) p; + break; + } + } + assertNotNull(provider); + + PCollection output = + PCollectionRowTuple.empty(pipeline) + .apply( + provider.from( + ReadFromDerbySchemaTransformProvider.JdbcReadSchemaTransformConfiguration + .builder() + .setJdbcUrl(DATA_SOURCE_CONFIGURATION.getUrl().get()) + .setLocation(READ_TABLE_NAME) + .build())) + .get("output"); + Long expected = Long.valueOf(EXPECTED_ROW_COUNT); + PAssert.that(output.apply(Count.globally())).containsInAnyOrder(expected); + pipeline.run(); + } + @Test public void testReadWithPartitions() { JdbcReadSchemaTransformProvider provider = null; From 2d6e1354248e1e570411e00b8f6ecf93efc775ab Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Wed, 4 Jun 2025 16:05:42 -0400 Subject: [PATCH 34/49] Fix Java Example ARM PostCommit --- .../beam_PostCommit_Java_Examples_Dataflow_ARM.json | 1 + runners/google-cloud-dataflow-java/arm/build.gradle | 7 +++---- 2 files changed, 4 insertions(+), 4 deletions(-) create mode 100644 .github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_ARM.json diff --git a/.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_ARM.json b/.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_ARM.json new file mode 100644 index 000000000000..0967ef424bce --- /dev/null +++ b/.github/trigger_files/beam_PostCommit_Java_Examples_Dataflow_ARM.json @@ -0,0 +1 @@ +{} diff --git a/runners/google-cloud-dataflow-java/arm/build.gradle b/runners/google-cloud-dataflow-java/arm/build.gradle index c5c907ae797e..2e74d7727f21 100644 --- a/runners/google-cloud-dataflow-java/arm/build.gradle +++ b/runners/google-cloud-dataflow-java/arm/build.gradle @@ -18,6 +18,8 @@ import groovy.json.JsonOutput +import static org.apache.beam.gradle.BeamModulePlugin.getSupportedJavaVersion + plugins { id 'org.apache.beam.module' } applyJavaNature( automaticModuleName: 'org.apache.beam.runners.dataflow', @@ -79,10 +81,7 @@ dependencies { examplesJavaIntegrationTest project(path: ":examples:java", configuration: "testRuntimeMigration") } -def javaVer = "java8" -if (project.hasProperty('testJavaVersion')) { - javaVer = "java${project.getProperty('testJavaVersion')}" -} +def javaVer = getSupportedJavaVersion(project.findProperty('testJavaVersion') as String) def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' def gcpRegion = project.findProperty('gcpRegion') ?: 'us-central1' def dataflowValidatesTempRoot = project.findProperty('gcpTempRoot') ?: 'gs://temp-storage-for-validates-runner-tests' From 1076b905ceac2ecb584e8880423dc408f35d715d Mon Sep 17 00:00:00 2001 From: bullet03 Date: Thu, 5 Jun 2025 15:09:56 +0200 Subject: [PATCH 35/49] fix: add missed word (#35163) Co-authored-by: Bulat Safiullin --- website/www/site/content/en/case-studies/akvelon.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/website/www/site/content/en/case-studies/akvelon.md b/website/www/site/content/en/case-studies/akvelon.md index 946603d78f50..08255a7a1b05 100644 --- a/website/www/site/content/en/case-studies/akvelon.md +++ b/website/www/site/content/en/case-studies/akvelon.md @@ -86,7 +86,7 @@ Akvelon designed and implemented a **Dataflow Flex Template** using Apache Beam This design provided both Protegrity and its enterprise clients with a reusable, open-source architecture for scalable data privacy and processing. ### The Results -- **Enabled data tokenization at** for regulated industries +- **Enabled data tokenization at scale** for regulated industries - **Accelerated adoption of Dataflow templates** across Protegrity’s customer base - **Delivered an [open-source Flex Template](https://github.com/apache/beam/blob/master/examples/java/src/main/java/org/apache/beam/examples/complete/datatokenization/README.md)** that benefits the entire Apache Beam community From ef06ee969de303d80e46d7e54a046d0ee3df3329 Mon Sep 17 00:00:00 2001 From: Derrick Williams Date: Thu, 5 Jun 2025 10:54:39 -0400 Subject: [PATCH 36/49] Add postcommit yaml xlang workflow and split tests accordingly (#35119) * move precommit yaml xlang to postcommit * add json file to trigger path * update pull request targets * add readme workflow changes * add cloud sdk setup * switch out to ubuntu-latest * add back precommit workflow * switch names and add postcommit * switch out to postCommitYamlIT * add test_files_dir flag * add conftest.py file for capturing directory flag * shift yaml tests around to appropriate locations * add back precommit to readme * add license for conftest.py * revert precommit to previous name * remove github.event.comment.body trigger --- .../beam_PostCommit_Yaml_Xlang_Direct.json | 4 + .github/workflows/README.md | 1 + .../beam_PostCommit_Yaml_Xlang_Direct.yml | 96 +++++++++++++++++++ .../beam_PreCommit_Yaml_Xlang_Direct.yml | 2 +- sdks/python/apache_beam/yaml/conftest.py | 34 +++++++ .../{tests => extended_tests}/bigquery.yaml | 0 .../{tests => extended_tests}/enrichment.yaml | 0 .../{tests => extended_tests}/spanner.yaml | 0 .../apache_beam/yaml/integration_tests.py | 10 +- .../assign_timestamps.yaml | 0 .../{extended_tests => tests}/create.yaml | 0 .../extract_windowing_Info.yaml | 0 .../{extended_tests => tests}/filter.yaml | 0 .../{extended_tests => tests}/flatten.yaml | 0 .../ml_transform.yaml | 0 .../{extended_tests => tests}/partition.yaml | 0 .../pytransform.yaml | 0 .../runinference.yaml | 0 .../validate_with_schema.yaml | 0 .../{extended_tests => tests}/windowinto.yaml | 0 sdks/python/build.gradle | 21 +++- 21 files changed, 165 insertions(+), 3 deletions(-) create mode 100644 .github/trigger_files/beam_PostCommit_Yaml_Xlang_Direct.json create mode 100644 .github/workflows/beam_PostCommit_Yaml_Xlang_Direct.yml create mode 100644 sdks/python/apache_beam/yaml/conftest.py rename sdks/python/apache_beam/yaml/{tests => extended_tests}/bigquery.yaml (100%) rename sdks/python/apache_beam/yaml/{tests => extended_tests}/enrichment.yaml (100%) rename sdks/python/apache_beam/yaml/{tests => extended_tests}/spanner.yaml (100%) rename sdks/python/apache_beam/yaml/{extended_tests => tests}/assign_timestamps.yaml (100%) rename sdks/python/apache_beam/yaml/{extended_tests => tests}/create.yaml (100%) rename sdks/python/apache_beam/yaml/{extended_tests => tests}/extract_windowing_Info.yaml (100%) rename sdks/python/apache_beam/yaml/{extended_tests => tests}/filter.yaml (100%) rename sdks/python/apache_beam/yaml/{extended_tests => tests}/flatten.yaml (100%) rename sdks/python/apache_beam/yaml/{extended_tests => tests}/ml_transform.yaml (100%) rename sdks/python/apache_beam/yaml/{extended_tests => tests}/partition.yaml (100%) rename sdks/python/apache_beam/yaml/{extended_tests => tests}/pytransform.yaml (100%) rename sdks/python/apache_beam/yaml/{extended_tests => tests}/runinference.yaml (100%) rename sdks/python/apache_beam/yaml/{extended_tests => tests}/validate_with_schema.yaml (100%) rename sdks/python/apache_beam/yaml/{extended_tests => tests}/windowinto.yaml (100%) diff --git a/.github/trigger_files/beam_PostCommit_Yaml_Xlang_Direct.json b/.github/trigger_files/beam_PostCommit_Yaml_Xlang_Direct.json new file mode 100644 index 000000000000..a975cd1cd104 --- /dev/null +++ b/.github/trigger_files/beam_PostCommit_Yaml_Xlang_Direct.json @@ -0,0 +1,4 @@ +{ + "comment": "Modify this file in a trivial way to cause this test suite to run", + "revision": 1 +} diff --git a/.github/workflows/README.md b/.github/workflows/README.md index b9069c530e53..a7cdf99d53cb 100644 --- a/.github/workflows/README.md +++ b/.github/workflows/README.md @@ -401,6 +401,7 @@ PostCommit Jobs run in a schedule against master branch and generally do not get | [ PostCommit XVR PythonUsingJavaSQL Dataflow ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml) | N/A |`beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.json`| [![.github/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_PythonUsingJavaSQL_Dataflow.yml?query=event%3Aschedule) | | [ PostCommit XVR Samza ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml) | N/A |`beam_PostCommit_XVR_Samza.json`| [![.github/workflows/beam_PostCommit_XVR_Samza.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Samza.yml?query=event%3Aschedule) | | [ PostCommit XVR Spark3 ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml) | N/A |`beam_PostCommit_XVR_Spark3.json`| [![.github/workflows/beam_PostCommit_XVR_Spark3.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_XVR_Spark3.yml?query=event%3Aschedule) | +| [ PostCommit YAML Xlang Direct ](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Yaml_Xlang_Direct.yml) | N/A |`beam_PostCommit_Yaml_Xlang_Direct.json`| [![.github/workflows/beam_PostCommit_Yaml_Xlang_Direct.yml](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Yaml_Xlang_Direct.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_PostCommit_Yaml_Xlang_Direct.yml?query=event%3Aschedule) | | [ Python Validates Container Dataflow ARM ](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml) | ['3.8','3.9','3.10','3.11'] |`beam_Python_ValidatesContainer_Dataflow_ARM.json`|[![.github/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml/badge.svg?event=schedule)](https://github.com/apache/beam/actions/workflows/beam_Python_ValidatesContainer_Dataflow_ARM.yml?query=event%3Aschedule) | ### PerformanceTests and Benchmark Jobs diff --git a/.github/workflows/beam_PostCommit_Yaml_Xlang_Direct.yml b/.github/workflows/beam_PostCommit_Yaml_Xlang_Direct.yml new file mode 100644 index 000000000000..93efb4f4f0c2 --- /dev/null +++ b/.github/workflows/beam_PostCommit_Yaml_Xlang_Direct.yml @@ -0,0 +1,96 @@ +# 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: PostCommit YAML Xlang Direct + +on: + schedule: + - cron: '30 5/6 * * *' + pull_request_target: + paths: ['release/trigger_all_tests.json', '.github/trigger_files/beam_PostCommit_Yaml_Xlang_Direct.json'] + 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: write + checks: write + contents: read + deployments: read + id-token: none + issues: write + 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.id || github.event.sender.login }}' + cancel-in-progress: true + +env: + DEVELOCITY_ACCESS_KEY: ${{ secrets.DEVELOCITY_ACCESS_KEY }} + GRADLE_ENTERPRISE_CACHE_USERNAME: ${{ secrets.GE_CACHE_USERNAME }} + GRADLE_ENTERPRISE_CACHE_PASSWORD: ${{ secrets.GE_CACHE_PASSWORD }} + +jobs: + beam_PostCommit_Yaml_Xlang_Direct: + if: | + github.event_name == 'workflow_dispatch' || + github.event_name == 'pull_request_target' || + (github.event_name == 'schedule' && github.repository == 'apache/beam') + runs-on: [self-hosted, ubuntu-latest, main] + timeout-minutes: 100 + name: ${{ matrix.job_name }} (${{ matrix.job_phrase }}) + strategy: + matrix: + job_name: ["beam_PostCommit_Yaml_Xlang_Direct"] + job_phrase: ["Run Yaml_Xlang_Direct PostCommit"] + steps: + - uses: actions/checkout@v4 + - 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: default + java-version: '11' + - name: Set up Cloud SDK + uses: google-github-actions/setup-gcloud@v2 + - name: run PostCommit Yaml Xlang Direct script + uses: ./.github/actions/gradle-command-self-hosted-action + with: + gradle-command: :sdks:python:postCommitYamlIntegrationTests -PbeamPythonExtra=ml_test + - name: Archive Python Test Results + uses: actions/upload-artifact@v4 + if: failure() + with: + name: Python Test Results + path: '**/pytest*.xml' + - name: Publish Python Test Results + uses: EnricoMi/publish-unit-test-result-action@v2 + if: always() + with: + commit: '${{ env.prsha || env.GITHUB_SHA }}' + comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} + files: '**/pytest*.xml' + large_files: true \ No newline at end of file diff --git a/.github/workflows/beam_PreCommit_Yaml_Xlang_Direct.yml b/.github/workflows/beam_PreCommit_Yaml_Xlang_Direct.yml index 92a20e5003e7..2e6a33f66882 100644 --- a/.github/workflows/beam_PreCommit_Yaml_Xlang_Direct.yml +++ b/.github/workflows/beam_PreCommit_Yaml_Xlang_Direct.yml @@ -105,4 +105,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/pytest*.xml' - large_files: true + large_files: true \ No newline at end of file diff --git a/sdks/python/apache_beam/yaml/conftest.py b/sdks/python/apache_beam/yaml/conftest.py new file mode 100644 index 000000000000..e8fb2c61581d --- /dev/null +++ b/sdks/python/apache_beam/yaml/conftest.py @@ -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. +# + +# This global will be set by pytest_configure +# and can be imported by other modules. +# Initialize with a default that matches the default. +yaml_test_files_dir = "tests" + + +def pytest_addoption(parser): + parser.addoption( + "--test_files_dir", + action="store", + default="tests", + help="Directory with YAML test files, relative to integration_tests.py") + + +def pytest_configure(config): + global yaml_test_files_dir + yaml_test_files_dir = config.getoption("test_files_dir") diff --git a/sdks/python/apache_beam/yaml/tests/bigquery.yaml b/sdks/python/apache_beam/yaml/extended_tests/bigquery.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/tests/bigquery.yaml rename to sdks/python/apache_beam/yaml/extended_tests/bigquery.yaml diff --git a/sdks/python/apache_beam/yaml/tests/enrichment.yaml b/sdks/python/apache_beam/yaml/extended_tests/enrichment.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/tests/enrichment.yaml rename to sdks/python/apache_beam/yaml/extended_tests/enrichment.yaml diff --git a/sdks/python/apache_beam/yaml/tests/spanner.yaml b/sdks/python/apache_beam/yaml/extended_tests/spanner.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/tests/spanner.yaml rename to sdks/python/apache_beam/yaml/extended_tests/spanner.yaml diff --git a/sdks/python/apache_beam/yaml/integration_tests.py b/sdks/python/apache_beam/yaml/integration_tests.py index d8c04d9718f8..f6c60ae5f121 100644 --- a/sdks/python/apache_beam/yaml/integration_tests.py +++ b/sdks/python/apache_beam/yaml/integration_tests.py @@ -53,6 +53,7 @@ from apache_beam.utils import python_callable from apache_beam.yaml import yaml_provider from apache_beam.yaml import yaml_transform +from apache_beam.yaml.conftest import yaml_test_files_dir @contextlib.contextmanager @@ -603,8 +604,15 @@ def parse_test_files(filepattern): globals()[suite_name] = type(suite_name, (unittest.TestCase, ), methods) +# Logging setup logging.getLogger().setLevel(logging.INFO) -parse_test_files(os.path.join(os.path.dirname(__file__), 'tests', '*.yaml')) + +# Dynamically create test methods from the tests directory. +# yaml_test_files_dir comes from conftest.py and set by pytest_configure. +_test_files_dir = yaml_test_files_dir +_file_pattern = os.path.join( + os.path.dirname(__file__), _test_files_dir, '*.yaml') +parse_test_files(_file_pattern) if __name__ == '__main__': logging.getLogger().setLevel(logging.INFO) diff --git a/sdks/python/apache_beam/yaml/extended_tests/assign_timestamps.yaml b/sdks/python/apache_beam/yaml/tests/assign_timestamps.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/extended_tests/assign_timestamps.yaml rename to sdks/python/apache_beam/yaml/tests/assign_timestamps.yaml diff --git a/sdks/python/apache_beam/yaml/extended_tests/create.yaml b/sdks/python/apache_beam/yaml/tests/create.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/extended_tests/create.yaml rename to sdks/python/apache_beam/yaml/tests/create.yaml diff --git a/sdks/python/apache_beam/yaml/extended_tests/extract_windowing_Info.yaml b/sdks/python/apache_beam/yaml/tests/extract_windowing_Info.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/extended_tests/extract_windowing_Info.yaml rename to sdks/python/apache_beam/yaml/tests/extract_windowing_Info.yaml diff --git a/sdks/python/apache_beam/yaml/extended_tests/filter.yaml b/sdks/python/apache_beam/yaml/tests/filter.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/extended_tests/filter.yaml rename to sdks/python/apache_beam/yaml/tests/filter.yaml diff --git a/sdks/python/apache_beam/yaml/extended_tests/flatten.yaml b/sdks/python/apache_beam/yaml/tests/flatten.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/extended_tests/flatten.yaml rename to sdks/python/apache_beam/yaml/tests/flatten.yaml diff --git a/sdks/python/apache_beam/yaml/extended_tests/ml_transform.yaml b/sdks/python/apache_beam/yaml/tests/ml_transform.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/extended_tests/ml_transform.yaml rename to sdks/python/apache_beam/yaml/tests/ml_transform.yaml diff --git a/sdks/python/apache_beam/yaml/extended_tests/partition.yaml b/sdks/python/apache_beam/yaml/tests/partition.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/extended_tests/partition.yaml rename to sdks/python/apache_beam/yaml/tests/partition.yaml diff --git a/sdks/python/apache_beam/yaml/extended_tests/pytransform.yaml b/sdks/python/apache_beam/yaml/tests/pytransform.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/extended_tests/pytransform.yaml rename to sdks/python/apache_beam/yaml/tests/pytransform.yaml diff --git a/sdks/python/apache_beam/yaml/extended_tests/runinference.yaml b/sdks/python/apache_beam/yaml/tests/runinference.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/extended_tests/runinference.yaml rename to sdks/python/apache_beam/yaml/tests/runinference.yaml diff --git a/sdks/python/apache_beam/yaml/extended_tests/validate_with_schema.yaml b/sdks/python/apache_beam/yaml/tests/validate_with_schema.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/extended_tests/validate_with_schema.yaml rename to sdks/python/apache_beam/yaml/tests/validate_with_schema.yaml diff --git a/sdks/python/apache_beam/yaml/extended_tests/windowinto.yaml b/sdks/python/apache_beam/yaml/tests/windowinto.yaml similarity index 100% rename from sdks/python/apache_beam/yaml/extended_tests/windowinto.yaml rename to sdks/python/apache_beam/yaml/tests/windowinto.yaml diff --git a/sdks/python/build.gradle b/sdks/python/build.gradle index d4708803b0f6..accacb1953bb 100644 --- a/sdks/python/build.gradle +++ b/sdks/python/build.gradle @@ -125,7 +125,7 @@ tasks.register("generateYamlDocs") { } tasks.register("yamlIntegrationTests") { - description "Runs integration tests for yaml pipelines." + description "Runs precommit integration tests for yaml pipelines." dependsOn installGcpTest // Need to build all expansion services referenced in apache_beam/yaml/*.* @@ -143,6 +143,25 @@ tasks.register("yamlIntegrationTests") { } } +tasks.register("postCommitYamlIntegrationTests") { + description "Runs postcommit integration tests for yaml pipelines." + + dependsOn installGcpTest + // Need to build all expansion services referenced in apache_beam/yaml/*.* + // grep -oh 'sdk.*Jar' sdks/python/apache_beam/yaml/*.yaml | sort | uniq + dependsOn ":sdks:java:extensions:schemaio-expansion-service:shadowJar" + dependsOn ":sdks:java:extensions:sql:expansion-service:shadowJar" + dependsOn ":sdks:java:io:expansion-service:build" + dependsOn ":sdks:java:io:google-cloud-platform:expansion-service:build" + + doLast { + exec { + executable 'sh' + args '-c', "${envdir}/bin/pytest -v apache_beam/yaml/integration_tests.py --deselect apache_beam/yaml/integration_tests.py::BigqueryTest::test_ReadFromBigQuery_ExternalJavaProvider_0 --test_files_dir='extended_tests'" + } + } +} + // Create Python wheels for given platform and Python version // build identifiers for cibuildwheel def platform_identifiers_map = [ From a27d022875b616c1ea59e90d72939a8eaa9f7b6e Mon Sep 17 00:00:00 2001 From: Jack McCluskey <34928439+jrmccluskey@users.noreply.github.com> Date: Thu, 5 Jun 2025 12:23:17 -0400 Subject: [PATCH 37/49] Replace usages of deprecated pkg_resources package (#35153) * Remove usages of deprecated pkg_resources package * use stdlib importlib.resources * remove extra comma * linting * import order --- sdks/python/gen_protos.py | 5 ++--- sdks/python/setup.py | 18 ++++++++++++------ 2 files changed, 14 insertions(+), 9 deletions(-) diff --git a/sdks/python/gen_protos.py b/sdks/python/gen_protos.py index a2cd1bd4cef3..d07f046fd229 100644 --- a/sdks/python/gen_protos.py +++ b/sdks/python/gen_protos.py @@ -21,6 +21,7 @@ import argparse import contextlib import glob +import importlib.resources import inspect import logging import os @@ -31,8 +32,6 @@ from collections import defaultdict from importlib import import_module -import pkg_resources - LOG = logging.getLogger() LOG.setLevel(logging.INFO) @@ -474,7 +473,7 @@ def generate_proto_files(force=False): protoc_gen_mypy = _find_protoc_gen_mypy() from grpc_tools import protoc - builtin_protos = pkg_resources.resource_filename('grpc_tools', '_proto') + builtin_protos = importlib.resources.files('grpc_tools') / '_proto' args = ( [sys.executable] + # expecting to be called from command line ['--proto_path=%s' % builtin_protos] + diff --git a/sdks/python/setup.py b/sdks/python/setup.py index d6c4088c1be4..a98eaab33361 100644 --- a/sdks/python/setup.py +++ b/sdks/python/setup.py @@ -31,9 +31,7 @@ # pylint: disable=ungrouped-imports import setuptools -from pkg_resources import normalize_path -from pkg_resources import parse_version -from pkg_resources import to_filename +from packaging.version import parse from setuptools import Command # pylint: disable=wrong-import-order @@ -43,6 +41,14 @@ from distutils.errors import DistutilsError # isort:skip +def to_filename(name: str) -> str: + return name.replace('-', '_') + + +def normalize_path(filename): + return os.path.normcase(os.path.realpath(os.path.normpath(filename))) + + class mypy(Command): user_options = [] @@ -101,7 +107,7 @@ def get_version(): RECOMMENDED_MIN_PIP_VERSION = '19.3.0' try: _PIP_VERSION = distribution('pip').version - if parse_version(_PIP_VERSION) < parse_version(RECOMMENDED_MIN_PIP_VERSION): + if parse(_PIP_VERSION) < parse(RECOMMENDED_MIN_PIP_VERSION): warnings.warn( "You are using version {0} of pip. " \ "However, the recommended min version is {1}.".format( @@ -116,7 +122,7 @@ def get_version(): REQUIRED_CYTHON_VERSION = '3.0.0' try: _CYTHON_VERSION = distribution('cython').version - if parse_version(_CYTHON_VERSION) < parse_version(REQUIRED_CYTHON_VERSION): + if parse(_CYTHON_VERSION) < parse(REQUIRED_CYTHON_VERSION): warnings.warn( "You are using version {0} of cython. " \ "However, version {1} is recommended.".format( @@ -355,7 +361,7 @@ def get_portability_package_data(): 'fasteners>=0.3,<1.0', # TODO(https://github.com/grpc/grpc/issues/37710): Unpin grpc 'grpcio>=1.33.1,<2,!=1.48.0,!=1.59.*,!=1.60.*,!=1.61.*,!=1.62.0,!=1.62.1,<1.66.0; python_version <= "3.12"', # pylint: disable=line-too-long - 'grpcio>=1.67.0; python_version >= "3.13"', + 'grpcio>=1.67.0; python_version >= "3.13"', 'hdfs>=2.1.0,<3.0.0', 'httplib2>=0.8,<0.23.0', 'jsonschema>=4.0.0,<5.0.0', From a9e04989c1678ace6215d286c75756a09dbc3d8f Mon Sep 17 00:00:00 2001 From: Hai Joey Tran Date: Thu, 5 Jun 2025 12:43:26 -0400 Subject: [PATCH 38/49] Improve error message when accidentally using PBegin/Pipeline (#35156) * Create test * Implement new error message * Add beam.Create into unit test pipeline --- sdks/python/apache_beam/io/gcp/bigquery_test.py | 2 +- sdks/python/apache_beam/pipeline.py | 5 +++++ sdks/python/apache_beam/pipeline_test.py | 17 +++++++++++++++++ 3 files changed, 23 insertions(+), 1 deletion(-) diff --git a/sdks/python/apache_beam/io/gcp/bigquery_test.py b/sdks/python/apache_beam/io/gcp/bigquery_test.py index 1f5e90dbfb2e..3931b0822595 100644 --- a/sdks/python/apache_beam/io/gcp/bigquery_test.py +++ b/sdks/python/apache_beam/io/gcp/bigquery_test.py @@ -968,7 +968,7 @@ def test_to_from_runner_api(self): schema=schema) # pylint: disable=expression-not-assigned - p | 'MyWriteToBigQuery' >> original + p | beam.Create([]) | 'MyWriteToBigQuery' >> original # Run the pipeline through to generate a pipeline proto from an empty # context. This ensures that the serialization code ran. diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 4d85e273f0b6..7aaebc0f196c 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -796,6 +796,11 @@ def apply( type_options = self._options.view_as(TypeOptions) if type_options.pipeline_type_check: transform.type_check_inputs(pvalueish) + if isinstance(pvalueish, pvalue.PBegin) and isinstance(transform, ParDo): + full_label = self._current_transform().full_label + raise TypeCheckError( + f"Transform '{full_label}' expects a PCollection as input. " + "Got a PBegin/Pipeline instead.") pvalueish_result = self.runner.apply(transform, pvalueish, self._options) diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py index b18bc0d668e2..20aebdd7790f 100644 --- a/sdks/python/apache_beam/pipeline_test.py +++ b/sdks/python/apache_beam/pipeline_test.py @@ -702,6 +702,23 @@ def test_track_pcoll_unbounded_flatten(self): self.assertIs(pcoll2_unbounded.is_bounded, False) self.assertIs(merged.is_bounded, False) + def test_incompatible_pcollection_errmsg(self): + with pytest.raises(Exception, + match=r".*Map\(print\).*Got a PBegin/Pipeline instead."): + with beam.Pipeline() as pipeline: + _ = (pipeline | beam.Map(print)) + + class ParentTransform(PTransform): + def expand(self, pcoll): + return pcoll | beam.Map(print) + + with pytest.raises( + Exception, + match=r".*ParentTransform/Map\(print\).*Got a PBegin/Pipeline instead." + ): + with beam.Pipeline() as pipeline: + _ = (pipeline | ParentTransform()) + def test_incompatible_submission_and_runtime_envs_fail_pipeline(self): with mock.patch( 'apache_beam.transforms.environments.sdk_base_version_capability' From 055472f5fc39aaffa09c365711974d5e616844e3 Mon Sep 17 00:00:00 2001 From: Hai Joey Tran Date: Thu, 5 Jun 2025 12:45:12 -0400 Subject: [PATCH 39/49] add friendly error message for when transform is applied to no output (#35160) * add friendly error message for when transform is applied to no output * update test name * Fix pubsub unit tests that depend on old behavior --- sdks/python/apache_beam/io/gcp/pubsub_test.py | 3 +++ sdks/python/apache_beam/pipeline.py | 19 +++++++++++++++++ sdks/python/apache_beam/pipeline_test.py | 21 +++++++++++++++++++ 3 files changed, 43 insertions(+) diff --git a/sdks/python/apache_beam/io/gcp/pubsub_test.py b/sdks/python/apache_beam/io/gcp/pubsub_test.py index feee9dc0082b..fadc49461a3c 100644 --- a/sdks/python/apache_beam/io/gcp/pubsub_test.py +++ b/sdks/python/apache_beam/io/gcp/pubsub_test.py @@ -27,6 +27,7 @@ import mock import apache_beam as beam +from apache_beam import Pipeline from apache_beam.io import Read from apache_beam.io import Write from apache_beam.io.gcp.pubsub import MultipleReadFromPubSub @@ -364,6 +365,7 @@ def test_expand_with_wrong_source(self): @unittest.skipIf(pubsub is None, 'GCP dependencies are not installed') class TestWriteStringsToPubSubOverride(unittest.TestCase): + @mock.patch.object(Pipeline, '_assert_not_applying_PDone', mock.Mock()) def test_expand_deprecated(self): options = PipelineOptions([]) options.view_as(StandardOptions).streaming = True @@ -385,6 +387,7 @@ def test_expand_deprecated(self): # Ensure that the properties passed through correctly self.assertEqual('a_topic', write_transform.dofn.short_topic_name) + @mock.patch.object(Pipeline, '_assert_not_applying_PDone', mock.Mock()) def test_expand(self): options = PipelineOptions([]) options.view_as(StandardOptions).streaming = True diff --git a/sdks/python/apache_beam/pipeline.py b/sdks/python/apache_beam/pipeline.py index 7aaebc0f196c..c4ac12a84cb6 100644 --- a/sdks/python/apache_beam/pipeline.py +++ b/sdks/python/apache_beam/pipeline.py @@ -802,7 +802,12 @@ def apply( f"Transform '{full_label}' expects a PCollection as input. " "Got a PBegin/Pipeline instead.") + self._assert_not_applying_PDone(pvalueish, transform) + pvalueish_result = self.runner.apply(transform, pvalueish, self._options) + if pvalueish_result is None: + pvalueish_result = pvalue.PDone(self) + pvalueish_result.producer = current if type_options is not None and type_options.pipeline_type_check: transform.type_check_outputs(pvalueish_result) @@ -849,6 +854,20 @@ def apply( self.transforms_stack.pop() return pvalueish_result + def _assert_not_applying_PDone( + self, + pvalueish, # type: Optional[pvalue.PValue] + transform # type: ptransform.PTransform + ): + if isinstance(pvalueish, pvalue.PDone) and isinstance(transform, ParDo): + # If the input is a PDone, we cannot apply a ParDo transform. + full_label = self._current_transform().full_label + producer_label = pvalueish.producer.full_label + raise TypeCheckError( + f'Transform "{full_label}" was applied to the output of ' + f'"{producer_label}" but "{producer_label.split("/")[-1]}" ' + 'produces no PCollections.') + def _generate_unique_label( self, transform # type: str diff --git a/sdks/python/apache_beam/pipeline_test.py b/sdks/python/apache_beam/pipeline_test.py index 20aebdd7790f..0bbd14b6afc7 100644 --- a/sdks/python/apache_beam/pipeline_test.py +++ b/sdks/python/apache_beam/pipeline_test.py @@ -61,6 +61,7 @@ from apache_beam.transforms.window import IntervalWindow from apache_beam.transforms.window import SlidingWindows from apache_beam.transforms.window import TimestampedValue +from apache_beam.typehints import TypeCheckError from apache_beam.utils import windowed_value from apache_beam.utils.timestamp import MIN_TIMESTAMP @@ -157,6 +158,26 @@ def test_create(self): pcoll3 = pcoll2 | 'do' >> FlatMap(lambda x: [x + 10]) assert_that(pcoll3, equal_to([14, 15, 16]), label='pcoll3') + def test_unexpected_PDone_errmsg(self): + """ + Test that a nice error message is raised if a transform that + returns None (i.e. produces no PCollection) is used as input + to a PTransform. + """ + class DoNothingTransform(PTransform): + def expand(self, pcoll): + return None + + class ParentTransform(PTransform): + def expand(self, pcoll): + return pcoll | DoNothingTransform() + + with pytest.raises( + TypeCheckError, + match=r".*applied to the output.*ParentTransform/DoNothingTransform"): + with TestPipeline() as pipeline: + _ = pipeline | ParentTransform() | beam.Map(lambda x: x + 1) + @mock.patch('logging.info') def test_runner_overrides_default_pickler(self, mock_info): with mock.patch.object(PipelineRunner, From afd1adb3d532ee37ca65decfe1e85b48e3d15452 Mon Sep 17 00:00:00 2001 From: Tanu Sharma <53229637+TanuSharma2511@users.noreply.github.com> Date: Thu, 5 Jun 2025 23:27:53 +0530 Subject: [PATCH 40/49] =?UTF-8?q?Add=20warning=20if=20temp=20location=20bu?= =?UTF-8?q?cket=20has=20soft=20delete=20enabled=20for=20Go=20SD=E2=80=A6?= =?UTF-8?q?=20(#34996)?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Add warning if temp location bucket has soft delete enabled for Go SDK (resolves #31606) * Corrected Formatting * Applied suggested changes --- sdks/go/pkg/beam/runners/dataflow/dataflow.go | 30 ++++++++++++ sdks/go/pkg/beam/util/gcsx/gcs.go | 13 +++++ sdks/go/pkg/beam/util/gcsx/gcs_test.go | 47 +++++++++++++++++++ 3 files changed, 90 insertions(+) diff --git a/sdks/go/pkg/beam/runners/dataflow/dataflow.go b/sdks/go/pkg/beam/runners/dataflow/dataflow.go index 73667fb8ee6e..25dbe365dccc 100644 --- a/sdks/go/pkg/beam/runners/dataflow/dataflow.go +++ b/sdks/go/pkg/beam/runners/dataflow/dataflow.go @@ -268,6 +268,9 @@ func getJobOptions(ctx context.Context, streaming bool) (*dataflowlib.JobOptions if *stagingLocation == "" { return nil, errors.New("no GCS staging location specified. Use --staging_location=gs:///") } + + checkSoftDeletePolicyEnabled(ctx, *stagingLocation, "staging_location") + var jobLabels map[string]string if *labels != "" { if err := json.Unmarshal([]byte(*labels), &jobLabels); err != nil { @@ -412,6 +415,8 @@ func getJobOptions(ctx context.Context, streaming bool) (*dataflowlib.JobOptions opts.TempLocation = gcsx.Join(*stagingLocation, "tmp") } + checkSoftDeletePolicyEnabled(ctx, opts.TempLocation, "temp_location") + return opts, nil } @@ -456,3 +461,28 @@ func getContainerImage(ctx context.Context) string { } panic(fmt.Sprintf("Unsupported environment %v", urn)) } + +func checkSoftDeletePolicyEnabled(ctx context.Context, bucketName string, locationName string) { + bucket, _, err := gcsx.ParseObject(bucketName) + if err != nil { + log.Warnf(ctx, "Error parsing bucket name: %v", err) + return + } + client, err := storage.NewClient(ctx) + if err != nil { + log.Warnf(ctx, "Error creating GCS client: %v", err) + return + } + defer client.Close() + + if enabled, err_msg := gcsx.SoftDeletePolicyEnabled(ctx, client, bucket); err_msg != nil { + log.Warnf(ctx, "Error checking SoftDeletePolicy: %v", err_msg) + } else if enabled { + log.Warnf(ctx, "Bucket %s specified in %s has soft-delete policy enabled. "+ + "Dataflow jobs use Cloud Storage to store temporary files during pipeline execution. "+ + "To avoid being billed for unnecessary storage costs, turn off the soft delete feature "+ + "on buckets that your Dataflow jobs use for temporary storage. "+ + "For more information, see https://cloud.google.com/storage/docs/use-soft-delete#remove-soft-delete-policy.", + bucketName, locationName) + } +} diff --git a/sdks/go/pkg/beam/util/gcsx/gcs.go b/sdks/go/pkg/beam/util/gcsx/gcs.go index 1dd85924447f..0986851c16d0 100644 --- a/sdks/go/pkg/beam/util/gcsx/gcs.go +++ b/sdks/go/pkg/beam/util/gcsx/gcs.go @@ -63,6 +63,19 @@ func Upload(ctx context.Context, client *storage.Client, project, bucket, object } +var getBucketAttrs = func(ctx context.Context, client *storage.Client, bucketName string) (*storage.BucketAttrs, error) { + return client.Bucket(bucketName).Attrs(ctx) +} + +// SoftDeletePolicyEnabled returns true if SoftDeletePolicy is enabled on bucket +func SoftDeletePolicyEnabled(ctx context.Context, client *storage.Client, bucketName string) (bool, error) { + attrs, err := getBucketAttrs(ctx, client, bucketName) + if err != nil { + return false, err + } + return attrs.SoftDeletePolicy != nil && attrs.SoftDeletePolicy.RetentionDuration > 0, nil +} + // Get BucketAttrs with RetentionDuration of SoftDeletePolicy set to zero for disabling SoftDeletePolicy. func getDisableSoftDeletePolicyBucketAttrs() *storage.BucketAttrs { attrs := &storage.BucketAttrs{ diff --git a/sdks/go/pkg/beam/util/gcsx/gcs_test.go b/sdks/go/pkg/beam/util/gcsx/gcs_test.go index 463ba3ea1833..cad3ad3754f0 100644 --- a/sdks/go/pkg/beam/util/gcsx/gcs_test.go +++ b/sdks/go/pkg/beam/util/gcsx/gcs_test.go @@ -16,9 +16,11 @@ package gcsx import ( + "context" "strings" "testing" + "cloud.google.com/go/storage" "github.com/apache/beam/sdks/v2/go/pkg/beam/internal/errors" ) @@ -106,3 +108,48 @@ func TestGetDisableSoftDeletePolicyBucketAttrs(t *testing.T) { t.Errorf("attrs has RetentionDuration %v which is not correct", attrs.SoftDeletePolicy.RetentionDuration) } } + +func TestSoftDeletePolicyWhenEnabled(t *testing.T) { + // Save original and defer restore + original := getBucketAttrs + defer func() { getBucketAttrs = original }() + + // Inject mock behavior + getBucketAttrs = func(ctx context.Context, client *storage.Client, bucketName string) (*storage.BucketAttrs, error) { + return &storage.BucketAttrs{ + SoftDeletePolicy: &storage.SoftDeletePolicy{ + RetentionDuration: 1029, + }, + }, nil + } + + // You can pass nil for client because the mock ignores it + enabled, err := SoftDeletePolicyEnabled(context.Background(), nil, "mock-bucket") + if err != nil { + t.Fatalf("Unexpected error: %v", err) + } + if !enabled { + t.Errorf("Expected soft delete to be enabled, got false") + } +} + +func TestSoftDeletePolicyWhenDisabled(t *testing.T) { + original := getBucketAttrs + defer func() { getBucketAttrs = original }() + + getBucketAttrs = func(ctx context.Context, client *storage.Client, bucketName string) (*storage.BucketAttrs, error) { + return &storage.BucketAttrs{ + SoftDeletePolicy: &storage.SoftDeletePolicy{ + RetentionDuration: 0, + }, + }, nil + } + + enabled, err := SoftDeletePolicyEnabled(context.Background(), nil, "mock-bucket") + if err != nil { + t.Fatalf("Unexpected error: %v", err) + } + if enabled { + t.Errorf("Expected soft delete to be disabled, got true") + } +} From 462379dd77c0c80dbefb4b705b09e6b5b1837ae3 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 29 May 2025 10:51:11 -0400 Subject: [PATCH 41/49] Constrain DequeCoder type correctly, as it does not support nulls The DequeCoder uses ArrayDeque internally, which disallows null elements. We could switch Deque implementations, but this change is better. To quote the JDK docs: "While Deque implementations are not strictly required to prohibit the insertion of null elements, they are strongly encouraged to do so. Users of any Deque implementations that do allow null elements are strongly encouraged not to take advantage of the ability to insert nulls. This is so because null is used as a special return value by various methods to indicated that the deque is empty." --- .../src/main/java/org/apache/beam/sdk/coders/DequeCoder.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DequeCoder.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DequeCoder.java index 81f706881416..21312a68530d 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DequeCoder.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/DequeCoder.java @@ -22,15 +22,16 @@ import java.util.List; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeParameter; +import org.checkerframework.checker.nullness.qual.NonNull; /** * A {@link Coder} for {@link Deque}, using the format of {@link IterableLikeCoder}. * * @param the type of the elements of the Deques being transcoded */ -public class DequeCoder extends IterableLikeCoder> { +public class DequeCoder extends IterableLikeCoder> { - public static DequeCoder of(Coder elemCoder) { + public static DequeCoder of(Coder elemCoder) { return new DequeCoder<>(elemCoder); } From bcff0321a0a24d42986adf5dd9fbf25f25508738 Mon Sep 17 00:00:00 2001 From: Minbo Bae <49642083+baeminbo@users.noreply.github.com> Date: Fri, 6 Jun 2025 04:48:45 +0900 Subject: [PATCH 42/49] Do not overwrite class states if a cached dynamic class is returned in cloudpickle.load (#35063) * Fix class states overwritten after cloudpickle.load * Fix further * Fix lint --- CHANGES.md | 1 + .../internal/cloudpickle/cloudpickle.py | 74 ++++++++++--------- .../apache_beam/internal/pickler_test.py | 40 ++++++++++ 3 files changed, 82 insertions(+), 33 deletions(-) diff --git a/CHANGES.md b/CHANGES.md index dd5bcce0c64b..bdd1959a1067 100644 --- a/CHANGES.md +++ b/CHANGES.md @@ -101,6 +101,7 @@ * (Java) Fixed cassandraIO ReadAll does not let a pipeline handle or retry exceptions ([#34191](https://github.com/apache/beam/pull/34191)). * [Python] Fixed vLLM breaks dataflow logging. ([35053](https://github.com/apache/beam/pull/35053)) * [Python] Fixed vLLM leaks connections causing a throughput bottleneck and underutilization of GPU ([35053](https://github.com/apache/beam/pull/35053)) +* (Python) Fixed cloudpickle overwriting class states every time loading a same object of dynamic class ([#35062](https://github.com/apache/beam/issues/35062)). ## 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)). diff --git a/sdks/python/apache_beam/internal/cloudpickle/cloudpickle.py b/sdks/python/apache_beam/internal/cloudpickle/cloudpickle.py index ce4240d96591..30868cdf1b49 100644 --- a/sdks/python/apache_beam/internal/cloudpickle/cloudpickle.py +++ b/sdks/python/apache_beam/internal/cloudpickle/cloudpickle.py @@ -95,6 +95,7 @@ # appropriate and preserve the usual "isinstance" semantics of Python objects. _DYNAMIC_CLASS_TRACKER_BY_CLASS = weakref.WeakKeyDictionary() _DYNAMIC_CLASS_TRACKER_BY_ID = weakref.WeakValueDictionary() +_DYNAMIC_CLASS_STATE_TRACKER_BY_CLASS = weakref.WeakKeyDictionary() _DYNAMIC_CLASS_TRACKER_LOCK = threading.Lock() PYPY = platform.python_implementation() == "PyPy" @@ -1150,39 +1151,46 @@ def _function_setstate(obj, state): def _class_setstate(obj, state): - state, slotstate = state - registry = None - for attrname, attr in state.items(): - if attrname == "_abc_impl": - registry = attr - else: - # Note: setting attribute names on a class automatically triggers their - # interning in CPython: - # https://github.com/python/cpython/blob/v3.12.0/Objects/object.c#L957 - # - # This means that to get deterministic pickling for a dynamic class that - # was initially defined in a different Python process, the pickler - # needs to ensure that dynamic class and function attribute names are - # systematically copied into a non-interned version to avoid - # unpredictable pickle payloads. - # - # Indeed the Pickler's memoizer relies on physical object identity to break - # cycles in the reference graph of the object being serialized. - setattr(obj, attrname, attr) - - if sys.version_info >= (3, 13) and "__firstlineno__" in state: - # Set the Python 3.13+ only __firstlineno__ attribute one more time, as it - # will be automatically deleted by the `setattr(obj, attrname, attr)` call - # above when `attrname` is "__firstlineno__". We assume that preserving this - # information might be important for some users and that it not stale in the - # context of cloudpickle usage, hence legitimate to propagate. Furthermore it - # is necessary to do so to keep deterministic chained pickling as tested in - # test_deterministic_str_interning_for_chained_dynamic_class_pickling. - obj.__firstlineno__ = state["__firstlineno__"] - - if registry is not None: - for subclass in registry: - obj.register(subclass) + # This breaks the ability to modify the state of a dynamic type in the main + # process wth the assumption that the type is updatable in the child process. + with _DYNAMIC_CLASS_TRACKER_LOCK: + if obj in _DYNAMIC_CLASS_STATE_TRACKER_BY_CLASS: + return obj + _DYNAMIC_CLASS_STATE_TRACKER_BY_CLASS[obj] = True + + state, slotstate = state + registry = None + for attrname, attr in state.items(): + if attrname == "_abc_impl": + registry = attr + else: + # Note: setting attribute names on a class automatically triggers their + # interning in CPython: + # https://github.com/python/cpython/blob/v3.12.0/Objects/object.c#L957 + # + # This means that to get deterministic pickling for a dynamic class that + # was initially defined in a different Python process, the pickler + # needs to ensure that dynamic class and function attribute names are + # systematically copied into a non-interned version to avoid + # unpredictable pickle payloads. + # + # Indeed the Pickler's memoizer relies on physical object identity to break + # cycles in the reference graph of the object being serialized. + setattr(obj, attrname, attr) + + if sys.version_info >= (3, 13) and "__firstlineno__" in state: + # Set the Python 3.13+ only __firstlineno__ attribute one more time, as it + # will be automatically deleted by the `setattr(obj, attrname, attr)` call + # above when `attrname` is "__firstlineno__". We assume that preserving this + # information might be important for some users and that it not stale in the + # context of cloudpickle usage, hence legitimate to propagate. Furthermore it + # is necessary to do so to keep deterministic chained pickling as tested in + # test_deterministic_str_interning_for_chained_dynamic_class_pickling. + obj.__firstlineno__ = state["__firstlineno__"] + + if registry is not None: + for subclass in registry: + obj.register(subclass) return obj diff --git a/sdks/python/apache_beam/internal/pickler_test.py b/sdks/python/apache_beam/internal/pickler_test.py index c8c1dd5e5f7a..7048f680de87 100644 --- a/sdks/python/apache_beam/internal/pickler_test.py +++ b/sdks/python/apache_beam/internal/pickler_test.py @@ -190,6 +190,46 @@ def test_dataclass(self, pickle_lib): self.assertEqual(DataClass(datum='abc'), loads(dumps(DataClass(datum='abc')))) ''') + @parameterized.expand([ + param(pickle_lib='dill'), + param(pickle_lib='cloudpickle'), + ]) + def test_class_states_not_changed_at_subsequent_loading(self, pickle_lib): + pickler.set_library(pickle_lib) + + class Local: + def fun(self): + pass + + obj = Local() + + serialized = dumps(obj) + + obj2 = loads(serialized) + obj2_type_id = id(type(obj2)) + obj2_func_id = id(obj2.fun.__func__) + + # The obj and obj2 may have different classes (dill) or the same class ( + # cloudpickle). The cloudpickle tracks the weak references of + # dumped/loaded dynamic classes with class_tracker_id, reusing the same + # class as possible if the original class is the same. Besides, + # dill creates a new class for each loaded object. + + loads(serialized) # obj3 + + # The obj2 and obj3 may have different classes (dill) or the same class ( + # cloudpickle) However, the obj2's class states must not be changed after + # obj3 loading. + # https://github.com/apache/beam/issues/35062 + self.assertEqual( + obj2_type_id, + id(type(obj2)), + 'Class must not changed after subsequent loading.') + self.assertEqual( + obj2_func_id, + id(obj2.fun.__func__), + 'Class states must not changed after subsequent loading.') + def maybe_get_sets_with_different_iteration_orders(self): # Use a mix of types in an attempt to create sets with the same elements # whose iteration order is different. From 019c45ed617e2dd96f2c436e2adaaee11b782b66 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 5 Jun 2025 15:57:28 -0400 Subject: [PATCH 43/49] Make SDK harness change effective on Iceberg Dataflow test (#35173) --- sdks/java/io/iceberg/build.gradle | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sdks/java/io/iceberg/build.gradle b/sdks/java/io/iceberg/build.gradle index 8d5cef919949..4d79fb061d05 100644 --- a/sdks/java/io/iceberg/build.gradle +++ b/sdks/java/io/iceberg/build.gradle @@ -95,6 +95,7 @@ dependencies { testRuntimeOnly library.java.slf4j_jdk14 testImplementation project(path: ":runners:direct-java", configuration: "shadow") testRuntimeOnly project(path: ":runners:google-cloud-dataflow-java") + testRuntimeOnly project(path: ":sdks:java:harness") hadoopVersions.each {kv -> "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-client:$kv.value" "hadoopVersion$kv.key" "org.apache.hadoop:hadoop-minicluster:$kv.value" @@ -165,7 +166,7 @@ task dataflowIntegrationTest(type: Test) { "--project=${gcpProject}", "--tempLocation=${gcpTempLocation}", "--tempRoot=${gcpTempLocation}", - "--experiments=use_runner_v2" + "--experiments=use_runner_v2,use_staged_dataflow_worker_jar" ] if (project.hasProperty('enableManagedTransforms')) { args.add("--experiments=enable_managed_transforms") From 96ef1c954e817583c2681529e612fed52b9f08b0 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 5 Jun 2025 16:34:27 -0400 Subject: [PATCH 44/49] Fix beam_PostCommit_Java_Examples_Dataflow_V2 (#35172) --- .../beam_PostCommit_Java_Examples_Dataflow_V2.yml | 8 ++++---- runners/google-cloud-dataflow-java/build.gradle | 9 +++------ 2 files changed, 7 insertions(+), 10 deletions(-) diff --git a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml index 03f34e96c58e..ab7ea4063cf8 100644 --- a/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml +++ b/.github/workflows/beam_PostCommit_Java_Examples_Dataflow_V2.yml @@ -77,8 +77,8 @@ jobs: - name: run PostCommit Java Examples Dataflow V2 script uses: ./.github/actions/gradle-command-self-hosted-action with: - gradle-command: :runners:google-cloud-dataflow-java:examplesJavaRunnerV2IntegrationTest - max-workers: 12 + gradle-command: :runners:google-cloud-dataflow-java:examplesJavaRunnerV2IntegrationTest + max-workers: 12 - name: Setup Java 17 environment uses: ./.github/actions/setup-environment-action with: @@ -99,7 +99,7 @@ jobs: -PdisableSpotlessCheck=true \ -PdisableCheckStyle=true \ -PskipCheckerFramework \ - -PtestJavaVersion=java17 \ + -PtestJavaVersion=17 \ -Pjava17Home=$JAVA_HOME_17_X64 \ -PdockerTag=${{ env.DOCKER_TAG }} \ - name: Archive JUnit Test Results @@ -115,4 +115,4 @@ jobs: commit: '${{ env.prsha || env.GITHUB_SHA }}' comment_mode: ${{ github.event_name == 'issue_comment' && 'always' || 'off' }} files: '**/build/test-results/**/*.xml' - large_files: true \ No newline at end of file + large_files: true diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index bc69bae4a942..912bf6c29439 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -281,7 +281,7 @@ def createRunnerV2ValidatesRunnerTest = { Map args -> tasks.register('examplesJavaRunnerV2IntegrationTestDistroless', Test.class) { group = "verification" dependsOn 'buildAndPushDistrolessContainerImage' - def javaVer = project.findProperty('testJavaVersion') + def javaVer = getSupportedJavaVersion(project.findProperty('testJavaVersion') as String) def repository = "us.gcr.io/apache-beam-testing/${System.getenv('USER')}" def tag = project.findProperty('dockerTag') def imageURL = "${repository}/beam_${javaVer}_sdk_distroless:${tag}" @@ -309,7 +309,7 @@ tasks.register('buildAndPushDistrolessContainerImage', Task.class) { // See https://github.com/GoogleContainerTools/distroless/tree/main/java#image-contents. def allowed = ["java17", "java21"] doLast { - def javaVer = project.findProperty('testJavaVersion') + def javaVer = getSupportedJavaVersion(project.findProperty('testJavaVersion') as String) if (!allowed.contains(javaVer)) { throw new GradleException("testJavaVersion must be one of ${allowed}, got: ${javaVer}") } @@ -837,10 +837,7 @@ task GCSUpload(type: JavaExec) { } def buildAndPushDistrolessDockerJavaContainer = tasks.register("buildAndPushDistrolessDockerJavaContainer") { - def javaVer = getSupportedJavaVersion() - if(project.hasProperty('testJavaVersion')) { - javaVer = "java${project.getProperty('testJavaVersion')}" - } + def javaVer = getSupportedJavaVersion(project.findProperty('testJavaVersion') as String) dependsOn ":sdks:java:container:distroless:${javaVer}:docker" def defaultDockerImageName = containerImageName( name: "${project.docker_image_default_repo_prefix}${javaVer}_sdk_distroless", From 4eda603c6316a77105e2047f723771ca6e93700b Mon Sep 17 00:00:00 2001 From: Derrick Williams Date: Thu, 5 Jun 2025 17:05:21 -0400 Subject: [PATCH 45/49] [YAML]: Update postgres IT test and readme (#35169) * update postgres test without driver_class_name * update readme on how to run integration tests * fix misspelling * fix trailing whitespace --- sdks/python/apache_beam/yaml/README.md | 25 +++++++++++++++++++ .../yaml/extended_tests/postgres.yaml | 20 +++++++++++++-- 2 files changed, 43 insertions(+), 2 deletions(-) diff --git a/sdks/python/apache_beam/yaml/README.md b/sdks/python/apache_beam/yaml/README.md index b90809eaf03e..56b5e7bedff7 100644 --- a/sdks/python/apache_beam/yaml/README.md +++ b/sdks/python/apache_beam/yaml/README.md @@ -43,3 +43,28 @@ https://beam.apache.org/documentation/sdks/yaml/ For information about contributing to Beam YAML see https://docs.google.com/document/d/19zswPXxxBxlAUmswYPUtSc-IVAu1qWvpjo1ZSDMRbu0 + +## Integration Tests + +The integration_tests.py dynamically creates test methods based on the yaml +files provided in the `tests` and `extended_tests` directories and runs the +pipeline. It also contains context managers for setting up test environments for +both precommit tests (e.g. `tests` folder) and postcommit tests +(e.g. `extended_tests` folder). + +To run the precommit tests: + +```bash +pytest -v integration_tests.py + +or + +pytest -v integration_tests.py::Test +``` + +To run the postcommit tests: + +```bash +pytest -v integration_tests.py --test_files_dir="extended_tests" +``` + diff --git a/sdks/python/apache_beam/yaml/extended_tests/postgres.yaml b/sdks/python/apache_beam/yaml/extended_tests/postgres.yaml index 9ecc2f481677..8957c782753e 100644 --- a/sdks/python/apache_beam/yaml/extended_tests/postgres.yaml +++ b/sdks/python/apache_beam/yaml/extended_tests/postgres.yaml @@ -35,7 +35,7 @@ pipelines: url: "{TEMP_DB}" query: "INSERT INTO tmp_table (value, rank) VALUES(?,?)" - # Postgres read pipeline + # Postgres read pipeline with driver_class_name - pipeline: type: chain transforms: @@ -50,4 +50,20 @@ pipelines: - {value: 123, rank: 0} - {value: 456, rank: 1} - {value: 789, rank: 2} - \ No newline at end of file + + # Postgres read pipeline without driver_class_name + # This is to prevent https://github.com/apache/beam/issues/35122 from + # happening again. + - pipeline: + type: chain + transforms: + - type: ReadFromPostgres + config: + url: "{TEMP_DB}" + query: "SELECT * FROM tmp_table" + - type: AssertEqual + config: + elements: + - {value: 123, rank: 0} + - {value: 456, rank: 1} + - {value: 789, rank: 2} \ No newline at end of file From ba8c1545c51eb9df5e60b7748746925e63148517 Mon Sep 17 00:00:00 2001 From: Yi Hu Date: Thu, 5 Jun 2025 18:43:24 -0400 Subject: [PATCH 46/49] Bump Java beam-master container (#35170) --- runners/google-cloud-dataflow-java/build.gradle | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/runners/google-cloud-dataflow-java/build.gradle b/runners/google-cloud-dataflow-java/build.gradle index 912bf6c29439..90d388b8bb68 100644 --- a/runners/google-cloud-dataflow-java/build.gradle +++ b/runners/google-cloud-dataflow-java/build.gradle @@ -52,8 +52,8 @@ evaluationDependsOn(":sdks:java:container:java11") ext.dataflowLegacyEnvironmentMajorVersion = '8' ext.dataflowFnapiEnvironmentMajorVersion = '8' -ext.dataflowLegacyContainerVersion = 'beam-master-20250602' -ext.dataflowFnapiContainerVersion = 'beam-master-20250602' +ext.dataflowLegacyContainerVersion = 'beam-master-20250605' +ext.dataflowFnapiContainerVersion = 'beam-master-20250605' ext.dataflowContainerBaseRepository = 'gcr.io/cloud-dataflow/v1beta3' processResources { From 30df249c60ce9c1ac86ea189c6cbb5fb7137fdc2 Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Tue, 3 Jun 2025 13:31:14 -0400 Subject: [PATCH 47/49] Make WindowedValue a public interface The following mostly-automated changes: - Moved WindowedValue from util to values package - Make WindowedValue an interface with companion class WindowedValues --- .../apache/beam/runners/core/DoFnRunner.java | 2 +- .../apache/beam/runners/core/DoFnRunners.java | 2 +- .../GroupAlsoByWindowViaWindowSetNewDoFn.java | 6 +- .../core/GroupByKeyViaGroupByKeyOnly.java | 4 +- .../beam/runners/core/KeyedWorkItem.java | 2 +- .../beam/runners/core/KeyedWorkItemCoder.java | 4 +- .../beam/runners/core/KeyedWorkItems.java | 2 +- .../core/LateDataDroppingDoFnRunner.java | 5 +- .../beam/runners/core/LateDataUtils.java | 2 +- ...oundedSplittableProcessElementInvoker.java | 2 +- .../beam/runners/core/ProcessFnRunner.java | 2 +- .../core/PushbackSideInputDoFnRunner.java | 2 +- .../beam/runners/core/ReduceFnRunner.java | 2 +- .../beam/runners/core/SideInputHandler.java | 2 +- .../beam/runners/core/SimpleDoFnRunner.java | 11 +- .../SimplePushbackSideInputDoFnRunner.java | 2 +- .../SplittableParDoViaKeyedWorkItems.java | 5 +- .../core/SplittableProcessElementInvoker.java | 2 +- .../beam/runners/core/StatefulDoFnRunner.java | 5 +- .../runners/core/KeyedWorkItemCoderTest.java | 9 +- .../core/LateDataDroppingDoFnRunnerTest.java | 5 +- ...edSplittableProcessElementInvokerTest.java | 4 +- .../beam/runners/core/ReduceFnRunnerTest.java | 2 +- .../beam/runners/core/ReduceFnTester.java | 7 +- .../runners/core/SideInputHandlerTest.java | 5 +- .../runners/core/SimpleDoFnRunnerTest.java | 36 +- ...SimplePushbackSideInputDoFnRunnerTest.java | 28 +- .../core/SplittableParDoProcessFnTest.java | 7 +- .../runners/core/StatefulDoFnRunnerTest.java | 45 +-- .../beam/runners/core/WindowMatchers.java | 2 +- .../beam/runners/core/WindowMatchersTest.java | 6 +- .../triggers/TriggerStateMachineTester.java | 6 +- .../direct/AbstractModelEnforcement.java | 2 +- .../direct/BoundedReadEvaluatorFactory.java | 7 +- .../runners/direct/CloningBundleFactory.java | 2 +- .../beam/runners/direct/CommittedBundle.java | 2 +- .../CreateViewNoopEvaluatorFactory.java | 2 +- .../direct/DirectTransformExecutor.java | 2 +- ...ycleManagerRemovingTransformEvaluator.java | 2 +- .../direct/FlattenEvaluatorFactory.java | 2 +- .../GroupAlsoByWindowEvaluatorFactory.java | 5 +- .../GroupByKeyOnlyEvaluatorFactory.java | 5 +- .../ImmutabilityCheckingBundleFactory.java | 2 +- .../ImmutabilityEnforcementFactory.java | 2 +- .../direct/ImmutableListBundleFactory.java | 2 +- .../direct/ImpulseEvaluatorFactory.java | 8 +- .../beam/runners/direct/ModelEnforcement.java | 2 +- .../beam/runners/direct/MultiStepCombine.java | 2 +- .../runners/direct/PCollectionViewWriter.java | 2 +- .../beam/runners/direct/ParDoEvaluator.java | 2 +- .../direct/ParDoMultiOverrideFactory.java | 7 +- .../direct/PassthroughTransformEvaluator.java | 2 +- .../beam/runners/direct/QuiescenceDriver.java | 4 +- .../runners/direct/SideInputContainer.java | 2 +- ...ttableProcessElementsEvaluatorFactory.java | 6 +- .../direct/StatefulParDoEvaluatorFactory.java | 2 +- .../runners/direct/StepTransformResult.java | 2 +- .../direct/TestStreamEvaluatorFactory.java | 9 +- .../runners/direct/TransformEvaluator.java | 2 +- .../beam/runners/direct/TransformResult.java | 2 +- .../direct/UnboundedReadEvaluatorFactory.java | 11 +- .../runners/direct/UncommittedBundle.java | 2 +- .../runners/direct/ViewEvaluatorFactory.java | 2 +- .../direct/WindowEvaluatorFactory.java | 5 +- .../BoundedReadEvaluatorFactoryTest.java | 11 +- .../direct/CloningBundleFactoryTest.java | 21 +- .../runners/direct/CommittedResultTest.java | 4 +- .../direct/DirectTransformExecutorTest.java | 19 +- ...ManagerRemovingTransformEvaluatorTest.java | 9 +- .../runners/direct/EvaluationContextTest.java | 11 +- .../direct/FlattenEvaluatorFactoryTest.java | 26 +- .../GroupByKeyEvaluatorFactoryTest.java | 27 +- .../GroupByKeyOnlyEvaluatorFactoryTest.java | 27 +- ...ImmutabilityCheckingBundleFactoryTest.java | 19 +- .../ImmutabilityEnforcementFactoryTest.java | 9 +- .../ImmutableListBundleFactoryTest.java | 31 +- .../direct/ImpulseEvaluatorFactoryTest.java | 5 +- .../KeyedPValueTrackingVisitorTest.java | 9 +- .../runners/direct/ParDoEvaluatorTest.java | 13 +- .../direct/SideInputContainerTest.java | 31 +- .../StatefulParDoEvaluatorFactoryTest.java | 5 +- .../TestStreamEvaluatorFactoryTest.java | 15 +- .../UnboundedReadEvaluatorFactoryTest.java | 13 +- .../direct/ViewEvaluatorFactoryTest.java | 7 +- .../runners/direct/WatermarkManagerTest.java | 32 +- .../direct/WindowEvaluatorFactoryTest.java | 9 +- .../FlinkBatchPortablePipelineTranslator.java | 13 +- .../flink/FlinkBatchTransformTranslators.java | 16 +- .../flink/FlinkBatchTranslationContext.java | 7 +- ...FlinkStreamingAggregationsTranslators.java | 15 +- ...nkStreamingPortablePipelineTranslator.java | 39 +- .../FlinkStreamingTransformTranslators.java | 25 +- .../FlinkStreamingTranslationContext.java | 5 +- .../adapter/BeamFlinkDataSetAdapter.java | 7 +- .../adapter/BeamFlinkDataStreamAdapter.java | 7 +- .../metrics/DoFnRunnerWithMetricsUpdate.java | 2 +- .../functions/AbstractFlinkCombineRunner.java | 2 +- .../functions/FlinkAssignContext.java | 2 +- .../functions/FlinkAssignWindows.java | 5 +- .../functions/FlinkDoFnFunction.java | 7 +- .../FlinkExecutableStageFunction.java | 2 +- .../FlinkExecutableStagePruningFunction.java | 2 +- .../FlinkExplodeWindowsFunction.java | 2 +- .../FlinkMergingNonShuffleReduceFunction.java | 2 +- .../FlinkMultiOutputPruningFunction.java | 2 +- .../FlinkNonMergingReduceFunction.java | 5 +- .../functions/FlinkPartialReduceFunction.java | 2 +- .../functions/FlinkReduceFunction.java | 2 +- .../functions/FlinkStatefulDoFnFunction.java | 2 +- .../functions/HashingFlinkCombineRunner.java | 5 +- .../functions/ImpulseSourceFunction.java | 5 +- .../functions/SideInputInitializer.java | 2 +- .../SingleWindowFlinkCombineRunner.java | 5 +- .../functions/SortingFlinkCombineRunner.java | 11 +- .../translation/types/KvKeySelector.java | 2 +- .../types/WindowedKvKeySelector.java | 2 +- .../wrappers/ImpulseInputFormat.java | 5 +- .../wrappers/SourceInputFormat.java | 5 +- .../wrappers/streaming/DoFnOperator.java | 2 +- .../ExecutableStageDoFnOperator.java | 11 +- .../streaming/KvToFlinkKeyKeySelector.java | 2 +- .../PartialReduceBundleOperator.java | 5 +- .../streaming/SdfFlinkKeyKeySelector.java | 2 +- .../streaming/SingletonKeyedWorkItem.java | 2 +- .../SingletonKeyedWorkItemCoder.java | 7 +- .../streaming/SplittableDoFnOperator.java | 9 +- .../streaming/WindowDoFnOperator.java | 5 +- .../streaming/WorkItemKeySelector.java | 2 +- .../streaming/io/DedupingOperator.java | 2 +- .../streaming/io/StreamingImpulseSource.java | 5 +- .../streaming/io/TestStreamSource.java | 5 +- .../streaming/io/UnboundedSourceWrapper.java | 5 +- .../io/source/bounded/FlinkBoundedSource.java | 2 +- .../bounded/FlinkBoundedSourceReader.java | 5 +- .../unbounded/FlinkUnboundedSource.java | 2 +- .../unbounded/FlinkUnboundedSourceReader.java | 5 +- .../stableinput/BufferedElements.java | 2 +- .../stableinput/BufferingDoFnRunner.java | 2 +- .../flink/FlinkPipelineOptionsTest.java | 9 +- .../streaming/BoundedSourceRestoreTest.java | 2 +- .../FlinkExecutableStageFunctionTest.java | 15 +- .../functions/ImpulseSourceFunctionTest.java | 2 +- .../streaming/DedupingOperatorTest.java | 18 +- .../wrappers/streaming/DoFnOperatorTest.java | 342 +++++++++--------- .../ExecutableStageDoFnOperatorTest.java | 45 +-- .../streaming/StreamRecordStripper.java | 2 +- .../streaming/WindowDoFnOperatorTest.java | 19 +- .../io/UnboundedSourceWrapperTest.java | 2 +- .../bounded/FlinkBoundedSourceReaderTest.java | 2 +- .../FlinkUnboundedSourceReaderTest.java | 2 +- .../stableinput/BufferedElementsTest.java | 8 +- .../stableinput/BufferingDoFnRunnerTest.java | 4 +- .../dataflow/BatchStatefulParDoOverrides.java | 7 +- .../runners/dataflow/BatchViewOverrides.java | 18 +- .../dataflow/DataflowPipelineTranslator.java | 6 +- .../beam/runners/dataflow/DataflowRunner.java | 12 +- .../dataflow/util/CloudObjectTranslators.java | 2 +- .../runners/dataflow/util/CloudObjects.java | 2 +- .../dataflow/BatchViewOverridesTest.java | 84 ++--- .../dataflow/util/CloudObjectsTest.java | 4 +- .../worker/AssignWindowsParDoFnFactory.java | 5 +- .../worker/BatchModeUngroupingParDoFn.java | 2 +- ...reateIsmShardKeyAndSortKeyDoFnFactory.java | 2 +- .../worker/DataflowOutputCounter.java | 2 +- .../DataflowPortabilityPCollectionView.java | 2 +- .../worker/DataflowProcessFnRunner.java | 5 +- .../worker/GroupAlsoByWindowFnRunner.java | 5 +- .../GroupAlsoByWindowParDoFnFactory.java | 2 +- .../worker/GroupAlsoByWindowsParDoFn.java | 2 +- .../worker/GroupingShuffleReader.java | 4 +- .../worker/GroupingShuffleReaderFactory.java | 2 +- ...uffleReaderWithFaultyBytesReadCounter.java | 2 +- .../IntrinsicMapTaskExecutorFactory.java | 2 +- .../runners/dataflow/worker/IsmReader.java | 2 +- .../dataflow/worker/IsmReaderFactory.java | 4 +- .../dataflow/worker/IsmReaderImpl.java | 2 +- .../dataflow/worker/IsmSideInputReader.java | 2 +- .../beam/runners/dataflow/worker/IsmSink.java | 2 +- .../dataflow/worker/IsmSinkFactory.java | 4 +- .../PairWithConstantKeyDoFnFactory.java | 2 +- .../worker/PartialGroupByKeyParDoFns.java | 13 +- .../worker/PartitioningShuffleReader.java | 4 +- .../PartitioningShuffleReaderFactory.java | 2 +- .../dataflow/worker/PubsubDynamicSink.java | 2 +- .../runners/dataflow/worker/PubsubReader.java | 7 +- .../runners/dataflow/worker/PubsubSink.java | 4 +- ...eifyTimestampAndWindowsParDoFnFactory.java | 7 +- .../runners/dataflow/worker/ShuffleSink.java | 4 +- .../dataflow/worker/ShuffleSinkFactory.java | 2 +- .../dataflow/worker/SimpleParDoFn.java | 2 +- .../worker/SplittableProcessFnFactory.java | 6 +- ...StreamingGroupAlsoByWindowReshuffleFn.java | 2 +- ...amingKeyedWorkItemSideInputDoFnRunner.java | 2 +- ...amingPCollectionViewWriterDoFnFactory.java | 2 +- ...StreamingPCollectionViewWriterParDoFn.java | 2 +- .../worker/StreamingSideInputDoFnRunner.java | 2 +- .../worker/StreamingSideInputFetcher.java | 5 +- .../ToIsmRecordForMultimapDoFnFactory.java | 2 +- .../worker/UngroupedWindmillReader.java | 9 +- .../dataflow/worker/ValuesDoFnFactory.java | 2 +- .../worker/WindmillKeyedWorkItem.java | 5 +- .../worker/WindmillReaderIteratorBase.java | 2 +- .../runners/dataflow/worker/WindmillSink.java | 4 +- .../worker/WindowingWindmillReader.java | 4 +- .../dataflow/worker/WorkerCustomSources.java | 7 +- .../BatchGroupAlsoByWindowAndCombineFn.java | 2 +- .../worker/util/BatchGroupAlsoByWindowFn.java | 2 +- .../BatchGroupAlsoByWindowReshuffleFn.java | 2 +- .../BatchGroupAlsoByWindowViaIteratorsFn.java | 2 +- ...tchGroupAlsoByWindowViaOutputBufferFn.java | 2 +- .../worker/util/ValueInEmptyWindows.java | 9 +- .../ComputationWorkExecutorFactory.java | 8 +- .../worker/AvroByteReaderFactoryTest.java | 6 +- .../worker/AvroByteSinkFactoryTest.java | 4 +- .../worker/CombineValuesFnFactoryTest.java | 42 +-- ...eIsmShardKeyAndSortKeyDoFnFactoryTest.java | 2 +- ...ataflowPortabilityPCollectionViewTest.java | 2 +- .../worker/DefaultParDoFnFactoryTest.java | 4 +- .../worker/GroupingShuffleReaderTest.java | 21 +- .../IntrinsicMapTaskExecutorFactoryTest.java | 6 +- .../dataflow/worker/IsmReaderFactoryTest.java | 8 +- .../dataflow/worker/IsmReaderTest.java | 2 +- .../worker/IsmSideInputReaderTest.java | 53 +-- .../runners/dataflow/worker/IsmSinkTest.java | 2 +- .../PairWithConstantKeyDoFnFactoryTest.java | 2 +- .../worker/PartialGroupByKeyParDoFnsTest.java | 75 ++-- .../worker/PartitioningShuffleReaderTest.java | 25 +- .../worker/PubsubDynamicSinkTest.java | 11 +- .../dataflow/worker/PubsubReaderTest.java | 11 +- .../dataflow/worker/PubsubSinkTest.java | 20 +- .../dataflow/worker/ReaderFactoryTest.java | 5 +- .../dataflow/worker/ReaderTestUtils.java | 2 +- ...TimestampAndWindowsParDoFnFactoryTest.java | 7 +- .../worker/ShuffleReaderFactoryTest.java | 4 +- .../worker/ShuffleSinkFactoryTest.java | 14 +- .../dataflow/worker/ShuffleSinkTest.java | 15 +- .../dataflow/worker/SimpleParDoFnTest.java | 48 +-- .../worker/StreamingDataflowWorkerTest.java | 19 +- .../StreamingGroupAlsoByWindowFnsTest.java | 2 +- ...ngGroupAlsoByWindowsReshuffleDoFnTest.java | 2 +- ...gKeyedWorkItemSideInputDoFnRunnerTest.java | 5 +- ...gPCollectionViewWriterDoFnFactoryTest.java | 4 +- .../StreamingSideInputDoFnRunnerTest.java | 7 +- .../worker/StreamingSideInputFetcherTest.java | 5 +- ...ToIsmRecordForMultimapDoFnFactoryTest.java | 2 +- .../worker/UngroupedShuffleReaderTest.java | 7 +- .../worker/UserParDoFnFactoryTest.java | 18 +- .../worker/ValuesDoFnFactoryTest.java | 2 +- .../worker/WindmillKeyedWorkItemTest.java | 8 +- .../WindmillReaderIteratorBaseTest.java | 5 +- .../worker/WorkerCustomSourcesTest.java | 4 +- .../graph/LengthPrefixUnknownCodersTest.java | 15 +- ...tchGroupAlsoByWindowReshuffleDoFnTest.java | 9 +- ...pAlsoByWindowsViaOutputBufferDoFnTest.java | 2 +- .../util/GroupAlsoByWindowProperties.java | 72 ++-- .../worker/util/ListOutputManager.java | 2 +- .../control/BundleCheckpointHandlers.java | 5 +- .../control/ProcessBundleDescriptors.java | 4 +- .../BatchSideInputHandlerFactory.java | 2 +- .../translation/PipelineTranslatorUtils.java | 9 +- .../runners/fnexecution/wire/WireCoders.java | 7 +- .../control/RemoteExecutionTest.java | 11 +- .../control/SdkHarnessClientTest.java | 19 +- .../fnexecution/data/GrpcDataServiceTest.java | 23 +- .../BatchSideInputHandlerFactoryTest.java | 30 +- .../fnexecution/wire/CommonCoderTest.java | 12 +- .../wire/LengthPrefixUnknownCodersTest.java | 14 +- .../apache/beam/runners/jet/DAGBuilder.java | 7 +- .../runners/jet/JetTransformTranslators.java | 5 +- .../runners/jet/JetTranslationContext.java | 8 +- .../org/apache/beam/runners/jet/Utils.java | 18 +- .../jet/processors/AbstractParDoP.java | 5 +- .../runners/jet/processors/AssignWindowP.java | 5 +- .../jet/processors/BoundedSourceP.java | 5 +- .../beam/runners/jet/processors/FlattenP.java | 2 +- .../beam/runners/jet/processors/ImpulseP.java | 4 +- .../jet/processors/StatefulParDoP.java | 2 +- .../jet/processors/UnboundedSourceP.java | 4 +- .../beam/runners/jet/processors/ViewP.java | 7 +- .../runners/jet/processors/WindowGroupP.java | 11 +- .../apache/beam/runners/jet/TestStreamP.java | 4 +- .../org/apache/beam/runners/local/Bundle.java | 2 +- .../samza/adapter/BoundedSourceSystem.java | 5 +- .../samza/adapter/UnboundedSourceSystem.java | 5 +- .../samza/metrics/DoFnRunnerWithMetrics.java | 2 +- .../samza/metrics/SamzaGBKMetricOp.java | 2 +- .../runners/samza/metrics/SamzaMetricOp.java | 2 +- .../samza/runtime/AsyncDoFnRunner.java | 2 +- .../samza/runtime/ClassicBundleManager.java | 2 +- .../beam/runners/samza/runtime/DoFnOp.java | 5 +- .../runtime/DoFnRunnerWithKeyedInternals.java | 2 +- .../samza/runtime/FutureCollector.java | 2 +- .../samza/runtime/FutureCollectorImpl.java | 2 +- .../runners/samza/runtime/GroupByKeyOp.java | 5 +- .../samza/runtime/KvToKeyedWorkItemOp.java | 2 +- .../apache/beam/runners/samza/runtime/Op.java | 2 +- .../beam/runners/samza/runtime/OpAdapter.java | 2 +- .../beam/runners/samza/runtime/OpEmitter.java | 2 +- .../beam/runners/samza/runtime/OpMessage.java | 2 +- .../runners/samza/runtime/PortableDoFnOp.java | 2 +- .../samza/runtime/SamzaAssignContext.java | 2 +- .../samza/runtime/SamzaDoFnRunners.java | 2 +- .../samza/runtime/SingletonKeyedWorkItem.java | 2 +- ...SplittableParDoProcessKeyedElementsOp.java | 7 +- .../runners/samza/runtime/WindowAssignOp.java | 5 +- .../translation/GroupByKeyTranslator.java | 9 +- .../ParDoBoundMultiTranslator.java | 17 +- .../samza/translation/ReadTranslator.java | 2 +- .../translation/ReshuffleTranslator.java | 5 +- .../SamzaImpulseSystemFactory.java | 4 +- .../SamzaPublishViewTranslator.java | 2 +- .../SamzaTestStreamSystemFactory.java | 5 +- .../SplittableParDoTranslators.java | 2 +- .../samza/translation/TranslationContext.java | 5 +- .../beam/runners/samza/util/SamzaCoders.java | 5 +- .../util/SamzaPipelineTranslatorUtils.java | 6 +- .../beam/runners/samza/util/WindowUtils.java | 8 +- .../samza/adapter/TestSourceHelpers.java | 4 +- .../TestSamzaRunnerWithTransformMetrics.java | 11 +- .../samza/runtime/AsyncDoFnRunnerTest.java | 7 +- .../runtime/ClassicBundleManagerTest.java | 2 +- .../runtime/FutureCollectorImplTest.java | 2 +- .../io/BoundedDatasetFactory.java | 4 +- .../translation/EvaluationContext.java | 2 +- .../translation/PipelineTranslator.java | 2 +- .../translation/SparkSessionFactory.java | 8 +- .../translation/TransformTranslator.java | 2 +- .../translation/batch/Aggregators.java | 5 +- .../batch/CombineGloballyTranslatorBatch.java | 5 +- .../CombineGroupedValuesTranslatorBatch.java | 2 +- .../batch/CombinePerKeyTranslatorBatch.java | 5 +- .../batch/DoFnPartitionIteratorFactory.java | 2 +- .../translation/batch/DoFnRunnerFactory.java | 2 +- .../batch/DoFnRunnerWithMetrics.java | 2 +- .../batch/FlattenTranslatorBatch.java | 2 +- .../translation/batch/GroupByKeyHelpers.java | 5 +- .../batch/GroupByKeyTranslatorBatch.java | 5 +- .../batch/ImpulseTranslatorBatch.java | 5 +- .../batch/ParDoTranslatorBatch.java | 2 +- .../batch/ReadSourceTranslatorBatch.java | 2 +- .../batch/ReshuffleTranslatorBatch.java | 2 +- .../batch/WindowAssignTranslatorBatch.java | 5 +- .../GroupAlsoByWindowViaOutputBufferFn.java | 5 +- .../batch/functions/SideInputValues.java | 4 +- .../translation/helpers/EncoderHelpers.java | 5 +- .../translation/batch/AggregatorsTest.java | 7 +- .../batch/functions/SideInputValuesTest.java | 9 +- .../helpers/EncoderHelpersTest.java | 5 +- .../coders/SparkRunnerKryoRegistrator.java | 3 +- .../beam/runners/spark/io/SourceRDD.java | 5 +- .../spark/io/SparkUnboundedSource.java | 7 +- .../SparkGroupAlsoByWindowViaWindowSet.java | 7 +- .../spark/stateful/StateSpecFunctions.java | 9 +- .../spark/translation/BoundedDataset.java | 17 +- .../translation/DoFnRunnerWithMetrics.java | 2 +- .../spark/translation/EvaluationContext.java | 5 +- .../translation/GroupCombineFunctions.java | 4 +- .../GroupNonMergingWindowsFunctions.java | 15 +- .../spark/translation/MultiDoFnFunction.java | 2 +- .../ReifyTimestampsAndWindowsFunction.java | 5 +- .../spark/translation/SideInputMetadata.java | 2 +- .../translation/SparkAssignWindowFn.java | 5 +- .../SparkBatchPortablePipelineTranslator.java | 15 +- .../spark/translation/SparkCombineFn.java | 18 +- ...parkExecutableStageExtractionFunction.java | 2 +- .../SparkExecutableStageFunction.java | 4 +- ...arkGroupAlsoByWindowViaOutputBufferFn.java | 5 +- .../translation/SparkInputDataProcessor.java | 2 +- .../translation/SparkPCollectionView.java | 2 +- ...rkStreamingPortablePipelineTranslator.java | 19 +- .../translation/TransformTranslator.java | 27 +- .../spark/translation/TranslationUtils.java | 7 +- .../streaming/ParDoStateUpdateFn.java | 11 +- .../StatefulStreamingParDoEvaluator.java | 11 +- .../StreamingTransformTranslator.java | 21 +- .../translation/streaming/TestDStream.java | 7 +- .../streaming/UnboundedDataset.java | 2 +- .../streaming/WatermarkSyncedDStream.java | 2 +- .../spark/util/SparkSideInputReader.java | 2 +- .../GroupNonMergingWindowsFunctionsTest.java | 13 +- .../spark/translation/SparkCombineFnTest.java | 7 +- .../SparkExecutableStageFunctionTest.java | 21 +- .../SparkInputDataProcessorTest.java | 13 +- .../translation/TransformTranslatorTest.java | 25 +- .../ValueAndCoderLazySerializableTest.java | 13 +- .../Twister2BatchTranslationContext.java | 2 +- .../twister2/Twister2TranslationContext.java | 2 +- .../wrappers/Twister2BoundedSource.java | 5 +- .../wrappers/Twister2EmptySource.java | 2 +- .../batch/AssignWindowTranslatorBatch.java | 2 +- .../batch/FlattenTranslatorBatch.java | 2 +- .../batch/GroupByKeyTranslatorBatch.java | 7 +- .../batch/ImpulseTranslatorBatch.java | 2 +- .../batch/PCollectionViewTranslatorBatch.java | 11 +- .../ParDoMultiOutputTranslatorBatch.java | 2 +- .../batch/ReadSourceTranslatorBatch.java | 2 +- .../functions/AssignWindowsFunction.java | 5 +- .../functions/ByteToElemFunction.java | 4 +- .../functions/ByteToWindowFunction.java | 4 +- .../ByteToWindowFunctionPrimitive.java | 9 +- .../translators/functions/DoFnFunction.java | 2 +- .../functions/ElemToBytesFunction.java | 9 +- .../functions/GroupByWindowFunction.java | 5 +- .../translators/functions/ImpulseSource.java | 5 +- .../functions/MapToTupleFunction.java | 13 +- .../functions/OutputTagFilter.java | 2 +- .../twister2/utils/TranslationUtils.java | 5 +- .../twister2/utils/Twister2AssignContext.java | 2 +- .../utils/Twister2SideInputReader.java | 2 +- .../apache/beam/sdk/transforms/Create.java | 7 +- .../util/construction/CoderTranslators.java | 21 +- .../construction/ModelCoderRegistrar.java | 12 +- .../construction/RunnerPCollectionView.java | 2 +- .../apache/beam/sdk/values/WindowedValue.java | 60 +++ .../WindowedValues.java} | 168 ++++----- .../data/BeamFnDataInboundObserverTest.java | 17 +- .../beam/sdk/transforms/CreateTest.java | 9 +- .../beam/sdk/transforms/RedistributeTest.java | 10 +- .../beam/sdk/transforms/ReshuffleTest.java | 10 +- .../beam/sdk/util/WindowedValueTest.java | 43 ++- .../construction/CoderTranslationTest.java | 6 +- .../util/construction/ModelCodersTest.java | 2 +- .../euphoria/core/testkit/WindowingTest.java | 1 + .../jmh/PrecombineGroupingTableBenchmark.java | 11 +- .../harness/jmh/ProcessBundleBenchmark.java | 4 +- .../beam/fn/harness/AssignWindowsRunner.java | 5 +- .../beam/fn/harness/BeamFnDataReadRunner.java | 2 +- .../fn/harness/BeamFnDataWriteRunner.java | 2 +- .../beam/fn/harness/CombineRunners.java | 4 +- .../apache/beam/fn/harness/FlattenRunner.java | 2 +- .../beam/fn/harness/FnApiDoFnRunner.java | 83 ++--- .../apache/beam/fn/harness/MapFnRunners.java | 2 +- .../fn/harness/PTransformRunnerFactory.java | 2 +- .../fn/harness/PrecombineGroupingTable.java | 7 +- ...littablePairWithRestrictionDoFnRunner.java | 5 +- ...bleSplitAndSizeRestrictionsDoFnRunner.java | 5 +- .../beam/fn/harness/WindowedSplitResult.java | 2 +- .../harness/control/ProcessBundleHandler.java | 2 +- .../data/PCollectionConsumerRegistry.java | 7 +- .../beam/fn/harness/debug/ElementSample.java | 2 +- .../beam/fn/harness/debug/OutputSampler.java | 5 +- .../fn/harness/state/FnApiStateAccessor.java | 6 +- .../fn/harness/AssignWindowsRunnerTest.java | 35 +- .../fn/harness/BeamFnDataReadRunnerTest.java | 7 +- .../fn/harness/BeamFnDataWriteRunnerTest.java | 7 +- .../beam/fn/harness/CombineRunnersTest.java | 4 +- .../beam/fn/harness/FlattenRunnerTest.java | 9 +- .../beam/fn/harness/FnApiDoFnRunnerTest.java | 95 ++--- .../beam/fn/harness/MapFnRunnersTest.java | 13 +- .../PTransformRunnerFactoryTestContext.java | 2 +- .../harness/PrecombineGroupingTableTest.java | 6 +- ...ablePairWithRestrictionDoFnRunnerTest.java | 19 +- ...plitAndSizeRestrictionsDoFnRunnerTest.java | 33 +- .../data/BeamFnDataGrpcClientTest.java | 7 +- .../data/PCollectionConsumerRegistryTest.java | 4 +- .../fn/harness/debug/DataSamplerTest.java | 5 +- .../fn/harness/debug/OutputSamplerTest.java | 41 ++- .../components/deadletterqueue/DLQRouter.java | 1 + .../deadletterqueue/sinks/ThrowingSink.java | 2 +- .../deadletterqueue/DLQRouterTest.java | 1 + .../io/gcp/bigquery/BigQueryHelpersTest.java | 4 +- .../sdk/io/iceberg/RecordWriterManager.java | 2 +- .../io/iceberg/WriteGroupedRowsToFiles.java | 5 +- .../io/iceberg/WriteUngroupedRowsToFiles.java | 5 +- .../io/iceberg/RecordWriterManagerTest.java | 9 +- 465 files changed, 2211 insertions(+), 1925 deletions(-) create mode 100644 sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java rename sdks/java/core/src/main/java/org/apache/beam/sdk/{util/WindowedValue.java => values/WindowedValues.java} (88%) diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java index 7c5c80fa4340..dce4fcaa68bd 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunner.java @@ -20,7 +20,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java index a02b11d5c630..abe8b99b1a26 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/DoFnRunners.java @@ -29,10 +29,10 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java index 3e42bb54494e..9e2f4f4efce1 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupAlsoByWindowViaWindowSetNewDoFn.java @@ -25,10 +25,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SystemDoFnInternal; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; @@ -99,7 +99,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputManager.output(mainTag, WindowedValue.of(output, timestamp, windows, pane)); + outputManager.output(mainTag, WindowedValues.of(output, timestamp, windows, pane)); } @Override @@ -109,7 +109,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane)); + outputManager.output(tag, WindowedValues.of(output, timestamp, windows, pane)); } }; } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java index 803128e24d73..ab4dbabd5d4e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/GroupByKeyViaGroupByKeyOnly.java @@ -29,10 +29,10 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItem.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItem.java index e035998c38dc..4901c5cbed5b 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItem.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItem.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.core; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * Interface that contains all the timers and elements associated with a specific work item. diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java index 756a89c88aca..657e8d85638a 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItemCoder.java @@ -28,8 +28,8 @@ import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** A {@link Coder} for {@link KeyedWorkItem KeyedWorkItems}. */ diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItems.java index 4c6df0c21051..db2ef3595505 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItems.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/KeyedWorkItems.java @@ -20,7 +20,7 @@ import java.util.Collections; import java.util.Objects; import org.apache.beam.runners.core.TimerInternals.TimerData; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java index ca1a0393b95d..0fb8ff52d1e1 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunner.java @@ -25,8 +25,9 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.joda.time.Instant; @@ -140,7 +141,7 @@ public Iterable> filter( timerInternals.currentOutputWatermarkTime()); } else { nonLateElements.add( - WindowedValue.of( + WindowedValues.of( element.getValue(), element.getTimestamp(), window, element.getPane())); } } diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java index d6f237bf571c..fbb7b315c3b1 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/LateDataUtils.java @@ -21,7 +21,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java index 51cd8c690aee..b7ba99e32f55 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvoker.java @@ -45,11 +45,11 @@ import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimator; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; 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.util.concurrent.Futures; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java index 28bdf45abd92..b8a4dcf7cb7f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ProcessFnRunner.java @@ -27,9 +27,9 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java index 1b2d90eb6181..61feaffad919 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/PushbackSideInputDoFnRunner.java @@ -20,7 +20,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java index b9eef50ed59f..ead53515558f 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/ReduceFnRunner.java @@ -47,9 +47,9 @@ import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java index 8b2cb5c3db2d..31afffda44ea 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SideInputHandler.java @@ -37,8 +37,8 @@ import org.apache.beam.sdk.transforms.Materializations.MultimapView; import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java index b375d38c5a98..1b9c4640b85e 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimpleDoFnRunner.java @@ -54,10 +54,11 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SystemDoFnInternal; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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.FluentIterable; @@ -331,7 +332,7 @@ public void output(OutputT output, Instant timestamp, BoundedWindow window) { @Override public void output(TupleTag tag, T output, Instant timestamp, BoundedWindow window) { - outputWindowedValue(tag, WindowedValue.of(output, timestamp, window, PaneInfo.NO_FIRING)); + outputWindowedValue(tag, WindowedValues.of(output, timestamp, window, PaneInfo.NO_FIRING)); } } @@ -447,7 +448,7 @@ public void outputWindowedValue( Collection windows, PaneInfo paneInfo) { SimpleDoFnRunner.this.outputWindowedValue( - tag, WindowedValue.of(output, timestamp, windows, paneInfo)); + tag, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -888,7 +889,7 @@ public void outputWindowedValue( PaneInfo paneInfo) { checkTimestamp(timestamp(), timestamp); SimpleDoFnRunner.this.outputWindowedValue( - tag, WindowedValue.of(output, timestamp, windows, paneInfo)); + tag, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -1117,7 +1118,7 @@ public void outputWindowedValue( PaneInfo paneInfo) { checkTimestamp(this.timestamp, timestamp); SimpleDoFnRunner.this.outputWindowedValue( - tag, WindowedValue.of(output, timestamp, windows, paneInfo)); + tag, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java index fb04b536b4c0..2be8071f9835 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunner.java @@ -24,8 +24,8 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; 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.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java index caa5565541a4..772428f84b65 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableParDoViaKeyedWorkItems.java @@ -45,7 +45,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformReplacements; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.PTransformTranslation.RawPTransform; @@ -59,6 +58,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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.ImmutableMap; @@ -295,7 +296,7 @@ public ProcessFn( this.elementTag = StateTags.value( "element", - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( elementCoder, inputWindowingStrategy.getWindowFn().windowCoder())); this.restrictionTag = StateTags.value("restriction", restrictionCoder); this.watermarkEstimatorStateTag = diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableProcessElementInvoker.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableProcessElementInvoker.java index 00a72b9197bb..1ff66d6e517c 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableProcessElementInvoker.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/SplittableProcessElementInvoker.java @@ -25,8 +25,8 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.splittabledofn.RestrictionTracker; import org.apache.beam.sdk.transforms.splittabledofn.WatermarkEstimator; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java index 0b4e980b7f5f..52bbb3a306a0 100644 --- a/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java +++ b/runners/core-java/src/main/java/org/apache/beam/runners/core/StatefulDoFnRunner.java @@ -38,7 +38,8 @@ import org.apache.beam.sdk.transforms.windowing.NonMergingWindowFn; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.joda.time.Duration; @@ -98,7 +99,7 @@ public StatefulDoFnRunner( this.sortBufferTag = StateTags.makeSystemTagInternal( - StateTags.bag(SORT_BUFFER_STATE, WindowedValue.getFullCoder(inputCoder, windowCoder))); + StateTags.bag(SORT_BUFFER_STATE, WindowedValues.getFullCoder(inputCoder, windowCoder))); rejectMergingWindowFn(windowFn); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java index 65c9d099d62b..6099f1fdc8e1 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/KeyedWorkItemCoderTest.java @@ -23,7 +23,8 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.testing.CoderProperties; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; import org.junit.Test; @@ -50,9 +51,9 @@ public void testEncodeDecodeEqual() throws Exception { TimeDomain.EVENT_TIME)); Iterable> elements = ImmutableList.of( - WindowedValue.valueInGlobalWindow(1), - WindowedValue.valueInGlobalWindow(4), - WindowedValue.valueInGlobalWindow(8)); + WindowedValues.valueInGlobalWindow(1), + WindowedValues.valueInGlobalWindow(4), + WindowedValues.valueInGlobalWindow(8)); KeyedWorkItemCoder coder = KeyedWorkItemCoder.of(StringUtf8Coder.of(), VarIntCoder.of(), GlobalWindow.Coder.INSTANCE); diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java index 89cfb4df2486..5a6257ba0a1b 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/LateDataDroppingDoFnRunnerTest.java @@ -29,7 +29,8 @@ import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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; @@ -97,7 +98,7 @@ public void testLateDataFilter() throws Exception { private WindowedValue createDatum(T element, long timestampMillis) { Instant timestamp = new Instant(timestampMillis); - return WindowedValue.of( + return WindowedValues.of( element, timestamp, Arrays.asList(WINDOW_FN.assignWindow(timestamp)), PaneInfo.NO_FIRING); } } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java index 14a52128f1a9..13a6709a43fd 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/OutputAndTimeBoundedSplittableProcessElementInvokerTest.java @@ -41,8 +41,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.joda.time.Duration; import org.joda.time.Instant; @@ -139,7 +139,7 @@ public void outputWindowedValue( SplittableProcessElementInvoker.Result rval = invoker.invokeProcessElement( DoFnInvokers.invokerFor(fn), - WindowedValue.of(null, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + WindowedValues.of(null, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), new OffsetRangeTracker(initialRestriction), new WatermarkEstimator() { @Override diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java index ea8bb406b60d..048bfb069451 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnRunnerTest.java @@ -78,9 +78,9 @@ import org.apache.beam.sdk.transforms.windowing.Window.ClosingBehavior; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java index d047a9133930..193ff51f2051 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/ReduceFnTester.java @@ -57,11 +57,12 @@ import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Equivalence; @@ -552,7 +553,7 @@ public final void injectElements(List> values) throws E windowFn.assignWindows( new TestAssignContext( windowFn, value, timestamp, GlobalWindow.INSTANCE)); - return WindowedValue.of(value, timestamp, windows, PaneInfo.NO_FIRING); + return WindowedValues.of(value, timestamp, windows, PaneInfo.NO_FIRING); } catch (Exception e) { throw new RuntimeException(e); } @@ -610,7 +611,7 @@ public void outputWindowedValue( KV copy = SerializableUtils.ensureSerializableByCoder( KvCoder.of(StringUtf8Coder.of(), outputCoder), output, "outputForWindow"); - WindowedValue> value = WindowedValue.of(copy, timestamp, windows, pane); + WindowedValue> value = WindowedValues.of(copy, timestamp, windows, pane); outputs.add(value); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java index b76d8fe6b823..293a7fe72c7e 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SideInputHandlerTest.java @@ -32,9 +32,10 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.joda.time.Instant; @@ -249,6 +250,6 @@ public void testMultipleSideInputs() { @SuppressWarnings({"unchecked", "rawtypes"}) private WindowedValue> valuesInWindow( List values, Instant timestamp, BoundedWindow window) { - return (WindowedValue) WindowedValue.of(values, timestamp, window, PaneInfo.NO_FIRING); + return (WindowedValue) WindowedValues.of(values, timestamp, window, PaneInfo.NO_FIRING); } } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java index a50eb7d9647e..a0435fe3026d 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimpleDoFnRunnerTest.java @@ -46,9 +46,10 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ListMultimap; @@ -108,7 +109,7 @@ public void testProcessElementExceptionsWrappedAsUserCodeException() { thrown.expect(UserCodeException.class); thrown.expectCause(is(fn.exceptionToThrow)); - runner.processElement(WindowedValue.valueInGlobalWindow("anyValue")); + runner.processElement(WindowedValues.valueInGlobalWindow("anyValue")); } @Test @@ -170,7 +171,7 @@ public void testTimerSet() { Instant currentTime = new Instant(42); when(mockTimerInternals.currentInputWatermarkTime()).thenReturn(currentTime); - runner.processElement(WindowedValue.valueInGlobalWindow("anyValue")); + runner.processElement(WindowedValues.valueInGlobalWindow("anyValue")); verify(mockTimerInternals) .setTimer( @@ -303,14 +304,14 @@ public void testBackwardsInTimeNoSkew() { runner.startBundle(); // An element output at the current timestamp is fine. runner.processElement( - WindowedValue.timestampedValueInGlobalWindow(Duration.ZERO, new Instant(0))); + WindowedValues.timestampedValueInGlobalWindow(Duration.ZERO, new Instant(0))); Exception exception = assertThrows( UserCodeException.class, () -> { // An element output before (current time - skew) is forbidden runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( Duration.millis(1L), new Instant(0))); }); @@ -354,7 +355,8 @@ public void testSkew() { runner.startBundle(); // Outputting between "now" and "now - allowed skew" succeeds. runner.processElement( - WindowedValue.timestampedValueInGlobalWindow(Duration.standardMinutes(5L), new Instant(0))); + WindowedValues.timestampedValueInGlobalWindow( + Duration.standardMinutes(5L), new Instant(0))); Exception exception = assertThrows( @@ -362,7 +364,7 @@ public void testSkew() { () -> { // Outputting before "now - allowed skew" fails. runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( Duration.standardHours(1L), new Instant(0))); }); @@ -405,12 +407,12 @@ public void testInfiniteSkew() { runner.startBundle(); runner.processElement( - WindowedValue.timestampedValueInGlobalWindow(Duration.millis(1L), new Instant(0))); + WindowedValues.timestampedValueInGlobalWindow(Duration.millis(1L), new Instant(0))); runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( Duration.millis(1L), BoundedWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)))); runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( // This is the maximum amount a timestamp in beam can move (from the maximum timestamp // to the minimum timestamp). Duration.millis(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()) @@ -444,7 +446,7 @@ public void testTimerBackwardsInTimeNoSkew() { runner.startBundle(); // A timer with output timestamp at the current timestamp is fine. runner.processElement( - WindowedValue.timestampedValueInGlobalWindow(KV.of("1", Duration.ZERO), new Instant(0))); + WindowedValues.timestampedValueInGlobalWindow(KV.of("1", Duration.ZERO), new Instant(0))); Exception exception = assertThrows( @@ -452,7 +454,7 @@ public void testTimerBackwardsInTimeNoSkew() { () -> { // A timer with output timestamp before (current time - skew) is forbidden runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("2", Duration.millis(1L)), new Instant(0))); }); @@ -495,7 +497,7 @@ public void testTimerSkew() { runner.startBundle(); // Timer with output timestamp between "now" and "now - allowed skew" succeeds. runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("1", Duration.standardMinutes(5L)), new Instant(0))); Exception exception = @@ -504,7 +506,7 @@ public void testTimerSkew() { () -> { // A timer with output timestamp before (current time - skew) is forbidden runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("2", Duration.standardHours(1L)), new Instant(0))); }); @@ -547,14 +549,14 @@ public void testTimerInfiniteSkew() { runner.startBundle(); runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("1", Duration.millis(1L)), new Instant(0))); runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("2", Duration.millis(1L)), BoundedWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)))); runner.processElement( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of( "3", // This is the maximum amount a timestamp in beam can move (from the maximum diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java index 7887faccbb06..b7dda599b8d2 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SimplePushbackSideInputDoFnRunnerTest.java @@ -56,10 +56,11 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.IdentitySideInputWindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -181,7 +182,7 @@ public void processElementSideInputNotReady() { createRunner(ImmutableList.of(singletonView)); WindowedValue oneWindow = - WindowedValue.of( + WindowedValues.of( 2, new Instant(-2), new IntervalWindow(new Instant(-500L), new Instant(0L)), @@ -201,7 +202,7 @@ public void processElementSideInputNotReadyMultipleWindows() { createRunner(ImmutableList.of(singletonView)); WindowedValue multiWindow = - WindowedValue.of( + WindowedValues.of( 2, new Instant(-2), ImmutableList.of( @@ -231,7 +232,7 @@ public void processElementSideInputNotReadySomeWindows() { IntervalWindow bigWindow = new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(250L)); WindowedValue multiWindow = - WindowedValue.of( + WindowedValues.of( 2, new Instant(-2), ImmutableList.of(littleWindow, bigWindow, GlobalWindow.INSTANCE), @@ -240,13 +241,14 @@ public void processElementSideInputNotReadySomeWindows() { runner.processElementInReadyWindows(multiWindow); assertThat( multiWindowPushback, - containsInAnyOrder(WindowedValue.timestampedValueInGlobalWindow(2, new Instant(-2L)))); + containsInAnyOrder(WindowedValues.timestampedValueInGlobalWindow(2, new Instant(-2L)))); assertThat( underlying.inputElems, containsInAnyOrder( - WindowedValue.of( + WindowedValues.of( 2, new Instant(-2), ImmutableList.of(littleWindow), PaneInfo.NO_FIRING), - WindowedValue.of(2, new Instant(-2), ImmutableList.of(bigWindow), PaneInfo.NO_FIRING))); + WindowedValues.of( + 2, new Instant(-2), ImmutableList.of(bigWindow), PaneInfo.NO_FIRING))); } @Test @@ -258,7 +260,7 @@ public void processElementSideInputReadyAllWindows() { SimplePushbackSideInputDoFnRunner runner = createRunner(views); WindowedValue multiWindow = - WindowedValue.of( + WindowedValues.of( 2, new Instant(-2), ImmutableList.of( @@ -279,7 +281,7 @@ public void processElementNoSideInputs() { SimplePushbackSideInputDoFnRunner runner = createRunner(ImmutableList.of()); WindowedValue multiWindow = - WindowedValue.of( + WindowedValues.of( 2, new Instant(-2), ImmutableList.of( @@ -396,7 +398,7 @@ public void testLateDroppingForStatefulDoFnRunner() throws Exception { .thenReturn(true); WindowedValue multiWindow = - WindowedValue.of( + WindowedValues.of( 1, new Instant(0), ImmutableList.of(new IntervalWindow(new Instant(0), new Instant(0L + WINDOW_SIZE))), @@ -433,20 +435,20 @@ public void testGarbageCollectForStatefulDoFnRunner() throws Exception { // first element, key is hello, WINDOW_1 runner.processElementInReadyWindows( - WindowedValue.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); assertEquals(1, (int) stateInternals.state(windowNamespace(WINDOW_1), stateTag).read()); // second element, key is hello, WINDOW_2 runner.processElementInReadyWindows( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 1), elementTime.plus(Duration.millis(WINDOW_SIZE)), WINDOW_2, PaneInfo.NO_FIRING)); runner.processElementInReadyWindows( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 1), elementTime.plus(Duration.millis(WINDOW_SIZE)), WINDOW_2, diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java index c6fe4e9bc671..7820fd36a835 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/SplittableParDoProcessFnTest.java @@ -61,12 +61,13 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; @@ -208,7 +209,7 @@ public void close() throws Exception { /** Performs a seed {@link DoFn.ProcessElement} call feeding the element and restriction. */ void startElement(InputT element, RestrictionT restriction) throws Exception { startElement( - WindowedValue.of( + WindowedValues.of( KV.of(element, restriction), currentProcessingTime, GlobalWindow.INSTANCE, @@ -324,7 +325,7 @@ public void testTrivialProcessFnPropagatesOutputWindowAndTimestamp() throws Exce MAX_OUTPUTS_PER_BUNDLE, MAX_BUNDLE_DURATION); tester.startElement( - WindowedValue.of( + WindowedValues.of( KV.of(42, new SomeRestriction()), base, Collections.singletonList(w), diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java index b65e6af84d96..11f3d244b37f 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/StatefulDoFnRunnerTest.java @@ -44,9 +44,10 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.checkerframework.checker.nullness.qual.Nullable; @@ -152,7 +153,7 @@ public void testDataDroppedBasedOnInputWatermarkWhenOrdered() throws Exception { new IntervalWindow(timestamp, timestamp.plus(Duration.millis(WINDOW_SIZE))); runner.processElement( - WindowedValue.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); long droppedValues = container @@ -165,7 +166,7 @@ public void testDataDroppedBasedOnInputWatermarkWhenOrdered() throws Exception { timerInternals.advanceInputWatermark(timestamp.plus(Duration.millis(ALLOWED_LATENESS + 1))); runner.processElement( - WindowedValue.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); droppedValues = container @@ -195,7 +196,7 @@ private void testLateDropping(boolean ordered) throws Exception { Instant timestamp = new Instant(0); runner.processElement( - WindowedValue.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), timestamp, window, PaneInfo.NO_FIRING)); long droppedValues = container @@ -220,7 +221,7 @@ private void testGarbageCollect(boolean ordered) throws Exception { // first element, key is hello, WINDOW_1 runner.processElement( - WindowedValue.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); if (ordered) { // move forward in time so that the input might get flushed @@ -232,14 +233,14 @@ private void testGarbageCollect(boolean ordered) throws Exception { // second element, key is hello, WINDOW_2 runner.processElement( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 1), elementTime.plus(Duration.millis(WINDOW_SIZE)), WINDOW_2, PaneInfo.NO_FIRING)); runner.processElement( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 1), elementTime.plus(Duration.millis(WINDOW_SIZE)), WINDOW_2, @@ -306,9 +307,9 @@ private void testOutput( // write two elements, with descending timestamps runner.processElement( - WindowedValue.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), elementTime, WINDOW_1, PaneInfo.NO_FIRING)); runner.processElement( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 2), elementTime.minus(Duration.millis(1)), WINDOW_1, @@ -327,17 +328,17 @@ private void testOutput( Arrays.asList( KV.of( outputTag, - WindowedValue.of( + WindowedValues.of( 2, elementTime.minus(Duration.millis(1)), WINDOW_1, PaneInfo.NO_FIRING)), - KV.of(outputTag, WindowedValue.of(3, elementTime, WINDOW_1, PaneInfo.NO_FIRING))), + KV.of(outputTag, WindowedValues.of(3, elementTime, WINDOW_1, PaneInfo.NO_FIRING))), outputs); } else { assertEquals( Arrays.asList( - KV.of(outputTag, WindowedValue.of(1, elementTime, WINDOW_1, PaneInfo.NO_FIRING)), + KV.of(outputTag, WindowedValues.of(1, elementTime, WINDOW_1, PaneInfo.NO_FIRING)), KV.of( outputTag, - WindowedValue.of( + WindowedValues.of( 3, elementTime.minus(Duration.millis(1)), WINDOW_1, PaneInfo.NO_FIRING))), outputs); } @@ -346,17 +347,17 @@ private void testOutput( // another window elementTime = elementTime.plus(Duration.millis(WINDOW_SIZE)); runner.processElement( - WindowedValue.of(KV.of("hello", 1), elementTime, WINDOW_2, PaneInfo.NO_FIRING)); + WindowedValues.of(KV.of("hello", 1), elementTime, WINDOW_2, PaneInfo.NO_FIRING)); runner.processElement( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 2), elementTime.minus(Duration.millis(1)), WINDOW_2, PaneInfo.NO_FIRING)); runner.processElement( - WindowedValue.of( + WindowedValues.of( KV.of("hello", 3), elementTime.minus(Duration.millis(2)), WINDOW_2, @@ -375,25 +376,25 @@ private void testOutput( Arrays.asList( KV.of( outputTag, - WindowedValue.of( + WindowedValues.of( 3, elementTime.minus(Duration.millis(2)), WINDOW_2, PaneInfo.NO_FIRING)), KV.of( outputTag, - WindowedValue.of( + WindowedValues.of( 5, elementTime.minus(Duration.millis(1)), WINDOW_2, PaneInfo.NO_FIRING)), - KV.of(outputTag, WindowedValue.of(6, elementTime, WINDOW_2, PaneInfo.NO_FIRING))), + KV.of(outputTag, WindowedValues.of(6, elementTime, WINDOW_2, PaneInfo.NO_FIRING))), outputs); } else { assertEquals( Arrays.asList( - KV.of(outputTag, WindowedValue.of(1, elementTime, WINDOW_2, PaneInfo.NO_FIRING)), + KV.of(outputTag, WindowedValues.of(1, elementTime, WINDOW_2, PaneInfo.NO_FIRING)), KV.of( outputTag, - WindowedValue.of( + WindowedValues.of( 3, elementTime.minus(Duration.millis(1)), WINDOW_2, PaneInfo.NO_FIRING)), KV.of( outputTag, - WindowedValue.of( + WindowedValues.of( 6, elementTime.minus(Duration.millis(2)), WINDOW_2, PaneInfo.NO_FIRING))), outputs); } diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java index ee33eb28f69d..0cdb4042471b 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchers.java @@ -22,7 +22,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.hamcrest.Description; import org.hamcrest.Matcher; diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java index a5f36be1134b..9dd8ac502fde 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/WindowMatchersTest.java @@ -21,7 +21,7 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; import org.junit.Test; @@ -39,7 +39,7 @@ public void testIsWindowedValueExact() { long windowEnd = 200; assertThat( - WindowedValue.of( + WindowedValues.of( "hello", new Instant(timestamp), new IntervalWindow(new Instant(windowStart), new Instant(windowEnd)), @@ -60,7 +60,7 @@ public void testIsWindowedValueReorderedWindows() { long windowEnd2 = 150; assertThat( - WindowedValue.of( + WindowedValues.of( "hello", new Instant(timestamp), ImmutableList.of( diff --git a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java index 096f3ac608b6..21dfddce7a6e 100644 --- a/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java +++ b/runners/core-java/src/test/java/org/apache/beam/runners/core/triggers/TriggerStateMachineTester.java @@ -48,8 +48,9 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.checkerframework.checker.nullness.qual.Nullable; @@ -248,7 +249,8 @@ public final void injectElements(Collection> values) th activeWindows.addActiveForTesting(window); } - windowedValues.add(WindowedValue.of(value, timestamp, assignedWindows, PaneInfo.NO_FIRING)); + windowedValues.add( + WindowedValues.of(value, timestamp, assignedWindows, PaneInfo.NO_FIRING)); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java index 40faf5a0fe2a..bb7c299abf85 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/AbstractModelEnforcement.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * An abstract {@link ModelEnforcement} that provides default empty implementations for each method. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java index bbb1ae842444..b31798e7e46b 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactory.java @@ -33,11 +33,12 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ReadTranslation; import org.apache.beam.sdk.util.construction.SplittableParDo.PrimitiveBoundedRead; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.Iterables; @@ -152,7 +153,7 @@ public void processElement(WindowedValue> element) UncommittedBundle output = evaluationContext.createBundle(outputPCollection); while (contentsRemaining) { output.add( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp())); contentsRemaining = reader.advance(); } @@ -221,7 +222,7 @@ public Collection>> getInitialInputs( CommittedBundle> inputShard = evaluationContext .>createRootBundle() - .add(WindowedValue.valueInGlobalWindow(BoundedSourceShard.of(bundle))) + .add(WindowedValues.valueInGlobalWindow(BoundedSourceShard.of(bundle))) .commit(BoundedWindow.TIMESTAMP_MAX_VALUE); shards.add(inputShard); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java index 7295b9227d83..c3d30bfbf744 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CloningBundleFactory.java @@ -22,8 +22,8 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedBundle.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedBundle.java index 72546fc83f99..936429442638 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedBundle.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CommittedBundle.java @@ -20,8 +20,8 @@ import org.apache.beam.runners.local.Bundle; import org.apache.beam.runners.local.StructuralKey; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CreateViewNoopEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CreateViewNoopEvaluatorFactory.java index fe1389cb89cd..d85890d217f7 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CreateViewNoopEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/CreateViewNoopEvaluatorFactory.java @@ -20,7 +20,7 @@ import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTransformExecutor.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTransformExecutor.java index ee71e33ff93f..fb07a3fa919f 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTransformExecutor.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DirectTransformExecutor.java @@ -27,8 +27,8 @@ import org.apache.beam.runners.core.metrics.MetricsContainerImpl; import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.values.WindowedValue; 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.MoreObjects; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java index dbc52d82727e..6508cb2a78b2 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluator.java @@ -19,7 +19,7 @@ import org.apache.beam.runners.core.TimerInternals.TimerData; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java index 92a65fe289e3..e1167d47a035 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/FlattenEvaluatorFactory.java @@ -21,9 +21,9 @@ import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.Flatten.PCollections; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java index c12a2ff44f8e..45cb58b4a492 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupAlsoByWindowEvaluatorFactory.java @@ -44,11 +44,12 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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; @@ -254,7 +255,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - bundle.add(WindowedValue.of(output, timestamp, windows, pane)); + bundle.add(WindowedValues.of(output, timestamp, windows, pane)); } @Override diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java index 3371d477fe2e..121c6f8bd5d9 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactory.java @@ -33,9 +33,10 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** @@ -136,7 +137,7 @@ public TransformResult> finishBundle() { StructuralKey.of(key, keyCoder), (PCollection>) Iterables.getOnlyElement(application.getOutputs().values())); - bundle.add(WindowedValue.valueInGlobalWindow(groupedKv)); + bundle.add(WindowedValues.valueInGlobalWindow(groupedKv)); resultBuilder.addOutput(bundle); } return resultBuilder.build(); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java index 3faaa986bee3..370829451d36 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactory.java @@ -27,8 +27,8 @@ import org.apache.beam.sdk.util.IllegalMutationException; import org.apache.beam.sdk.util.MutationDetector; import org.apache.beam.sdk.util.MutationDetectors; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.SetMultimap; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java index bc7fc2867e71..84509bded752 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactory.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.util.MutationDetector; import org.apache.beam.sdk.util.MutationDetectors; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * {@link ModelEnforcement} that enforces elements are not modified over the course of processing an diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java index b601f41a6171..5cfb6fa59374 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImmutableListBundleFactory.java @@ -25,8 +25,8 @@ import javax.annotation.Nonnull; import org.apache.beam.runners.local.StructuralKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactory.java index 09337f3728b4..09eb1cfd8747 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactory.java @@ -23,9 +23,10 @@ import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; @@ -67,7 +68,8 @@ public void processElement(WindowedValue element) throws Exception PCollection outputPCollection = (PCollection) Iterables.getOnlyElement(transform.getOutputs().values()); result.addOutput( - ctxt.createBundle(outputPCollection).add(WindowedValue.valueInGlobalWindow(new byte[0]))); + ctxt.createBundle(outputPCollection) + .add(WindowedValues.valueInGlobalWindow(new byte[0]))); } @Override @@ -94,7 +96,7 @@ public Collection> getInitialInputs( int targetParallelism) { return Collections.singleton( ctxt.createRootBundle() - .add(WindowedValue.valueInGlobalWindow(new ImpulseShard())) + .add(WindowedValues.valueInGlobalWindow(new ImpulseShard())) .commit(BoundedWindow.TIMESTAMP_MIN_VALUE)); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java index 335727e6c8d0..de8ffa3b9b4d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ModelEnforcement.java @@ -19,8 +19,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; /** * Enforcement tools that verify that executing code conforms to the model. diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java index 9a7b7f7a9b5d..e1573abf7864 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/MultiStepCombine.java @@ -46,13 +46,13 @@ import org.apache.beam.sdk.transforms.windowing.DefaultTrigger; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.PTransformTranslation.RawPTransform; import org.apache.beam.sdk.util.construction.SingleInputOutputOverrideFactory; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.EnsuresNonNull; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PCollectionViewWriter.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PCollectionViewWriter.java index f3828d8a49dd..c37fcec735de 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PCollectionViewWriter.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PCollectionViewWriter.java @@ -17,9 +17,9 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; /** * A {@link PCollectionViewWriter} is responsible for writing contents of a {@link PCollection} to a diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java index 8370f95a419e..6be73b645c68 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoEvaluator.java @@ -42,10 +42,10 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java index dd9ef1cc3984..3a5237fb19e4 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ParDoMultiOverrideFactory.java @@ -43,7 +43,6 @@ import org.apache.beam.sdk.transforms.windowing.Repeatedly; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformReplacements; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.ReplacementOutputs; @@ -56,6 +55,8 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; @@ -190,7 +191,7 @@ PCollection>> groupToKeyedWorkItem( return input // Stash the original timestamps, etc, for when it is fed to the user's DoFn .apply("Reify timestamps", ParDo.of(new ReifyWindowedValueFn<>())) - .setCoder(KvCoder.of(keyCoder, WindowedValue.getFullCoder(kvCoder, windowCoder))) + .setCoder(KvCoder.of(keyCoder, WindowedValues.getFullCoder(kvCoder, windowCoder))) // We are going to GBK to gather keys and windows but otherwise do not want // to alter the flow of data. This entails: @@ -317,7 +318,7 @@ public void processElement(final ProcessContext c, final BoundedWindow window) { c.output( KV.of( c.element().getKey(), - WindowedValue.of(c.element(), c.timestamp(), window, c.pane()))); + WindowedValues.of(c.element(), c.timestamp(), window, c.pane()))); } } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java index bad15989a57b..530d5021ddb1 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/PassthroughTransformEvaluator.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.direct; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; class PassthroughTransformEvaluator implements TransformEvaluator { public static PassthroughTransformEvaluator create( diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java index 13cf2ebfe844..8915c25bef87 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/QuiescenceDriver.java @@ -34,9 +34,9 @@ import org.apache.beam.runners.local.ExecutionDriver; import org.apache.beam.runners.local.PipelineMessageReceiver; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValues; 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.slf4j.Logger; @@ -196,7 +196,7 @@ private void fireTimers() { (PCollection) Iterables.getOnlyElement( transformTimers.getExecutable().getMainInputs().values())) - .add(WindowedValue.valueInGlobalWindow(work)) + .add(WindowedValues.valueInGlobalWindow(work)) .commit(evaluationContext.now()); processBundle( bundle, diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java index 62b0a06ddff1..3fe6cf4a11b8 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SideInputContainer.java @@ -38,9 +38,9 @@ import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java index a5bf31e406d2..373490f69c44 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/SplittableProcessElementsEvaluatorFactory.java @@ -36,11 +36,11 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.MoreExecutors; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; @@ -155,7 +155,7 @@ public void outputWindowedValue( Collection windows, PaneInfo pane) { outputManager.output( - transform.getMainOutputTag(), WindowedValue.of(output, timestamp, windows, pane)); + transform.getMainOutputTag(), WindowedValues.of(output, timestamp, windows, pane)); } @Override @@ -165,7 +165,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane)); + outputManager.output(tag, WindowedValues.of(output, timestamp, windows, pane)); } }; SideInputReader sideInputReader = diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java index 9d8a815ede6b..8edbd45f2b5d 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactory.java @@ -37,10 +37,10 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java index ae4da1ec0717..8cfe07bdbb71 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/StepTransformResult.java @@ -31,7 +31,7 @@ import org.apache.beam.runners.direct.WatermarkManager.TimerUpdate; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java index 85d0371a451e..8ca2a909e367 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactory.java @@ -33,7 +33,6 @@ import org.apache.beam.sdk.testing.TestStream.WatermarkEvent; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ReplacementOutputs; import org.apache.beam.sdk.util.construction.TestStreamTranslation; import org.apache.beam.sdk.values.PBegin; @@ -41,6 +40,8 @@ import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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; @@ -110,7 +111,7 @@ public void processElement(WindowedValue> element) throws Exc (PCollection) Iterables.getOnlyElement(application.getOutputs().values())); for (TimestampedValue elem : ((ElementEvent) event).getElements()) { bundle.add( - WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp())); + WindowedValues.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp())); } resultBuilder.addOutput(bundle); } @@ -127,7 +128,7 @@ public void processElement(WindowedValue> element) throws Exc TestStreamIndex next = streamIndex.next(); if (next.getIndex() < events.size()) { resultBuilder.addUnprocessedElements( - Collections.singleton(WindowedValue.timestampedValueInGlobalWindow(next, watermark))); + Collections.singleton(WindowedValues.timestampedValueInGlobalWindow(next, watermark))); } } @@ -231,7 +232,7 @@ public Collection>> getInitialInputs( CommittedBundle> initialBundle = evaluationContext .>createRootBundle() - .add(WindowedValue.valueInGlobalWindow(TestStreamIndex.of(testStream.original))) + .add(WindowedValues.valueInGlobalWindow(TestStreamIndex.of(testStream.original))) .commit(BoundedWindow.TIMESTAMP_MAX_VALUE); return Collections.singleton(initialBundle); } diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java index 5d3a6a784f00..3eb10e0dce34 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformEvaluator.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.direct; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * An evaluator of a specific application of a transform. Will be used for at least one {@link diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java index 7e1a8f61c4b3..2817ddb97510 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/TransformResult.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java index bfcb0a591824..e45990a760ea 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactory.java @@ -35,11 +35,12 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ReadTranslation; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.checkerframework.checker.nullness.qual.Nullable; @@ -143,7 +144,7 @@ public void processElement( do { if (deduplicator.shouldOutput(reader.getCurrentRecordId())) { output.add( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp())); } numElements++; @@ -168,7 +169,7 @@ public void processElement( .addOutput(output) .addUnprocessedElements( Collections.singleton( - WindowedValue.timestampedValueInGlobalWindow(residual, watermark))); + WindowedValues.timestampedValueInGlobalWindow(residual, watermark))); } else { Instant watermark = reader.getWatermark(); if (watermark.isBefore(BoundedWindow.TIMESTAMP_MAX_VALUE)) { @@ -176,7 +177,7 @@ public void processElement( // Might be better to finalize old checkpoint. resultBuilder.addUnprocessedElements( Collections.>singleton( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( UnboundedSourceShard.of( shard.getSource(), shard.getDeduplicator(), @@ -344,7 +345,7 @@ static class InputProvider initialShards.add( evaluationContext .>createRootBundle() - .add(WindowedValue.valueInGlobalWindow(shard)) + .add(WindowedValues.valueInGlobalWindow(shard)) .commit(BoundedWindow.TIMESTAMP_MAX_VALUE)); } return initialShards.build(); diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundle.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundle.java index 1885703d7121..83c64dc3bbd2 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundle.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/UncommittedBundle.java @@ -18,8 +18,8 @@ package org.apache.beam.runners.direct; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java index d5ea6f06b95b..823462494bd7 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/ViewEvaluatorFactory.java @@ -22,8 +22,8 @@ import org.apache.beam.runners.direct.CommittedResult.OutputType; import org.apache.beam.runners.direct.DirectWriteViewVisitor.WriteView; import org.apache.beam.sdk.runners.AppliedPTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** diff --git a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java index 1f5292b4e7c9..17139a7495c3 100644 --- a/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java +++ b/runners/direct-java/src/main/java/org/apache/beam/runners/direct/WindowEvaluatorFactory.java @@ -23,9 +23,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.WindowIntoTranslation; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; @@ -90,7 +91,7 @@ public void processElement(WindowedValue compressedElement) throws Excep for (WindowedValue element : compressedElement.explodeWindows()) { Collection windows = assignWindows(windowFn, element); outputBundle.add( - WindowedValue.of( + WindowedValues.of( element.getValue(), element.getTimestamp(), windows, element.getPane())); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java index 121c4325427a..12f8eae152fc 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/BoundedReadEvaluatorFactoryTest.java @@ -53,9 +53,10 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.hamcrest.Matchers; @@ -262,7 +263,7 @@ public void boundedSourceInMemoryTransformEvaluatorShardsOfSource() throws Excep UncommittedBundle> rootBundle = bundleFactory.createRootBundle(); for (BoundedSource split : splits) { BoundedSourceShard shard = BoundedSourceShard.of(split); - rootBundle.add(WindowedValue.valueInGlobalWindow(shard)); + rootBundle.add(WindowedValues.valueInGlobalWindow(shard)); } CommittedBundle> shards = rootBundle.commit(Instant.now()); @@ -301,7 +302,7 @@ public void boundedSourceEvaluatorClosesReader() throws Exception { TransformEvaluator> evaluator = factory.forApplication( sourceTransform, bundleFactory.createRootBundle().commit(Instant.now())); - evaluator.processElement(WindowedValue.valueInGlobalWindow(BoundedSourceShard.of(source))); + evaluator.processElement(WindowedValues.valueInGlobalWindow(BoundedSourceShard.of(source))); evaluator.finishBundle(); CommittedBundle committed = output.commit(Instant.now()); assertThat(committed.getElements(), containsInAnyOrder(gw(2L), gw(3L), gw(1L))); @@ -321,7 +322,7 @@ public void boundedSourceEvaluatorNoElementsClosesReader() throws Exception { TransformEvaluator> evaluator = factory.forApplication( sourceTransform, bundleFactory.createRootBundle().commit(Instant.now())); - evaluator.processElement(WindowedValue.valueInGlobalWindow(BoundedSourceShard.of(source))); + evaluator.processElement(WindowedValues.valueInGlobalWindow(BoundedSourceShard.of(source))); evaluator.finishBundle(); CommittedBundle committed = output.commit(Instant.now()); assertThat(committed.getElements(), emptyIterable()); @@ -453,6 +454,6 @@ public void close() throws IOException { } private static WindowedValue gw(Long elem) { - return WindowedValue.valueInGlobalWindow(elem); + return WindowedValues.valueInGlobalWindow(elem); } } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java index e03f453e21ad..6d98e3c32acc 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CloningBundleFactoryTest.java @@ -39,9 +39,10 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.WithKeys; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.joda.time.Instant; @@ -61,8 +62,8 @@ public class CloningBundleFactoryTest { @Test public void rootBundleSucceedsIgnoresCoder() { - WindowedValue one = WindowedValue.valueInGlobalWindow(new Record()); - WindowedValue two = WindowedValue.valueInGlobalWindow(new Record()); + WindowedValue one = WindowedValues.valueInGlobalWindow(new Record()); + WindowedValue two = WindowedValues.valueInGlobalWindow(new Record()); CommittedBundle root = factory.createRootBundle().add(one).add(two).commit(Instant.now()); @@ -76,9 +77,9 @@ public void bundleWorkingCoderSucceedsClonesOutput() { created .apply(WithKeys.of("foo")) .setCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())); - WindowedValue> fooOne = WindowedValue.valueInGlobalWindow(KV.of("foo", 1)); + WindowedValue> fooOne = WindowedValues.valueInGlobalWindow(KV.of("foo", 1)); WindowedValue> fooThree = - WindowedValue.valueInGlobalWindow(KV.of("foo", 3)); + WindowedValues.valueInGlobalWindow(KV.of("foo", 3)); CommittedBundle> bundle = factory.createBundle(kvs).add(fooOne).add(fooThree).commit(Instant.now()); @@ -103,7 +104,7 @@ public void keyedBundleWorkingCoderSucceedsClonesOutput() { .setCoder(KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())) .apply(GroupByKey.create()); WindowedValue>> foos = - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( KV.>of("foo", ImmutableList.of(1, 3))); CommittedBundle>> keyedBundle = factory @@ -127,7 +128,7 @@ public void bundleEncodeFailsAddFails() { thrown.expect(UserCodeException.class); thrown.expectCause(isA(CoderException.class)); thrown.expectMessage("Encode not allowed"); - bundle.add(WindowedValue.valueInGlobalWindow(new Record())); + bundle.add(WindowedValues.valueInGlobalWindow(new Record())); } @Test @@ -138,7 +139,7 @@ public void bundleDecodeFailsAddFails() { thrown.expect(UserCodeException.class); thrown.expectCause(isA(CoderException.class)); thrown.expectMessage("Decode not allowed"); - bundle.add(WindowedValue.valueInGlobalWindow(new Record())); + bundle.add(WindowedValues.valueInGlobalWindow(new Record())); } @Test @@ -150,7 +151,7 @@ public void keyedBundleEncodeFailsAddFails() { thrown.expect(UserCodeException.class); thrown.expectCause(isA(CoderException.class)); thrown.expectMessage("Encode not allowed"); - bundle.add(WindowedValue.valueInGlobalWindow(new Record())); + bundle.add(WindowedValues.valueInGlobalWindow(new Record())); } @Test @@ -162,7 +163,7 @@ public void keyedBundleDecodeFailsAddFails() { thrown.expect(UserCodeException.class); thrown.expectCause(isA(CoderException.class)); thrown.expectMessage("Decode not allowed"); - bundle.add(WindowedValue.valueInGlobalWindow(new Record())); + bundle.add(WindowedValues.valueInGlobalWindow(new Record())); } static class Record {} diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java index 87f6d773c9dd..a24d1c98c46f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/CommittedResultTest.java @@ -31,11 +31,11 @@ import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.resourcehints.ResourceHints; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PValues; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.Matchers; @@ -85,7 +85,7 @@ public void getUncommittedElementsEqualInput() { CommittedBundle bundle = bundleFactory .createBundle(created) - .add(WindowedValue.valueInGlobalWindow(2)) + .add(WindowedValues.valueInGlobalWindow(2)) .commit(Instant.now()); CommittedResult> result = CommittedResult.create( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTransformExecutorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTransformExecutorTest.java index 196840c9d860..7884064bdd9b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTransformExecutorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DirectTransformExecutorTest.java @@ -40,9 +40,10 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.WithKeys; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.util.concurrent.MoreExecutors; import org.hamcrest.Matchers; @@ -176,9 +177,9 @@ public TransformResult finishBundle() throws Exception { } }; - WindowedValue foo = WindowedValue.valueInGlobalWindow("foo"); - WindowedValue spam = WindowedValue.valueInGlobalWindow("spam"); - WindowedValue third = WindowedValue.valueInGlobalWindow("third"); + WindowedValue foo = WindowedValues.valueInGlobalWindow("foo"); + WindowedValue spam = WindowedValues.valueInGlobalWindow("spam"); + WindowedValue third = WindowedValues.valueInGlobalWindow("third"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(foo).add(spam).add(third).commit(Instant.now()); when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); @@ -222,7 +223,7 @@ public TransformResult finishBundle() throws Exception { } }; - WindowedValue foo = WindowedValue.valueInGlobalWindow("foo"); + WindowedValue foo = WindowedValues.valueInGlobalWindow("foo"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(foo).commit(Instant.now()); when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); @@ -295,8 +296,8 @@ public TransformResult finishBundle() throws Exception { } }; - WindowedValue fooElem = WindowedValue.valueInGlobalWindow("foo"); - WindowedValue barElem = WindowedValue.valueInGlobalWindow("bar"); + WindowedValue fooElem = WindowedValues.valueInGlobalWindow("foo"); + WindowedValue barElem = WindowedValues.valueInGlobalWindow("bar"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(fooElem).add(barElem).commit(Instant.now()); when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); @@ -334,7 +335,7 @@ public TransformResult finishBundle() throws Exception { } }; - WindowedValue fooBytes = WindowedValue.valueInGlobalWindow("foo"); + WindowedValue fooBytes = WindowedValues.valueInGlobalWindow("foo"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(fooBytes).commit(Instant.now()); when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); @@ -372,7 +373,7 @@ public TransformResult finishBundle() throws Exception { } }; - WindowedValue fooBytes = WindowedValue.valueInGlobalWindow("foo"); + WindowedValue fooBytes = WindowedValues.valueInGlobalWindow("foo"); CommittedBundle inputBundle = bundleFactory.createBundle(created).add(fooBytes).commit(Instant.now()); when(registry.forApplication(downstreamProducer, inputBundle)).thenReturn(evaluator); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java index 342b489ab12a..1a9cdc187bf9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/DoFnLifecycleManagerRemovingTransformEvaluatorTest.java @@ -33,7 +33,8 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.hamcrest.Matchers; import org.joda.time.Instant; import org.junit.Before; @@ -56,8 +57,8 @@ public void delegatesToUnderlying() throws Exception { ParDoEvaluator underlying = mock(ParDoEvaluator.class); TransformEvaluator evaluator = DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager); - WindowedValue first = WindowedValue.valueInGlobalWindow(new Object()); - WindowedValue second = WindowedValue.valueInGlobalWindow(new Object()); + WindowedValue first = WindowedValues.valueInGlobalWindow(new Object()); + WindowedValue second = WindowedValues.valueInGlobalWindow(new Object()); evaluator.processElement(first); verify(underlying).processElement(first); @@ -82,7 +83,7 @@ public void removesOnExceptionInProcessElement() throws Exception { DoFnLifecycleManagerRemovingTransformEvaluator.wrapping(underlying, lifecycleManager); try { - evaluator.processElement(WindowedValue.valueInGlobalWindow(new Object())); + evaluator.processElement(WindowedValues.valueInGlobalWindow(new Object())); } catch (Exception e) { assertThat(lifecycleManager.get(), not(Matchers.theInstance(original))); return; diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java index b3f3ab529bdf..3e470c63c5dd 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/EvaluationContextTest.java @@ -61,11 +61,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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; @@ -153,13 +154,13 @@ public void writeToViewWriterThenReadReads() { for (Object materializedValue : materializeValuesFor(view.getPipeline().getOptions(), View.asIterable(), 1)) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(1222), window, PaneInfo.ON_TIME_AND_ONLY_FIRING)); } for (Object materializedValue : materializeValuesFor(view.getPipeline().getOptions(), View.asIterable(), 2)) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(8766L), second, @@ -175,7 +176,7 @@ public void writeToViewWriterThenReadReads() { for (Object materializedValue : materializeValuesFor(view.getPipeline().getOptions(), View.asIterable(), 4444)) { overwrittenValuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(8677L), second, @@ -389,7 +390,7 @@ public void isDoneWithPartiallyDone() { // Impulse produces one element UncommittedBundle rootBundle = context.createBundle(impulse); - rootBundle.add(WindowedValue.valueInGlobalWindow(new byte[0])); + rootBundle.add(WindowedValues.valueInGlobalWindow(new byte[0])); CommittedResult handleResult = context.handleResult( null, diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java index 1aea14dac614..a8097d4898f1 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/FlattenEvaluatorFactoryTest.java @@ -30,9 +30,9 @@ import org.apache.beam.sdk.transforms.Flatten; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.hamcrest.Matchers; import org.joda.time.Instant; @@ -74,15 +74,15 @@ public void testFlattenInMemoryEvaluator() throws Exception { TransformEvaluator rightSideEvaluator = factory.forApplication(flattenedProducer, rightBundle); - leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(1)); - rightSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(-1)); + leftSideEvaluator.processElement(WindowedValues.valueInGlobalWindow(1)); + rightSideEvaluator.processElement(WindowedValues.valueInGlobalWindow(-1)); leftSideEvaluator.processElement( - WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024))); - leftSideEvaluator.processElement(WindowedValue.valueInGlobalWindow(4, PaneInfo.NO_FIRING)); + WindowedValues.timestampedValueInGlobalWindow(2, new Instant(1024))); + leftSideEvaluator.processElement(WindowedValues.valueInGlobalWindow(4, PaneInfo.NO_FIRING)); rightSideEvaluator.processElement( - WindowedValue.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING)); + WindowedValues.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING)); rightSideEvaluator.processElement( - WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096))); + WindowedValues.timestampedValueInGlobalWindow(-4, new Instant(-4096))); TransformResult rightSideResult = rightSideEvaluator.finishBundle(); TransformResult leftSideResult = leftSideEvaluator.finishBundle(); @@ -99,15 +99,15 @@ public void testFlattenInMemoryEvaluator() throws Exception { assertThat( flattenedLeftBundle.commit(Instant.now()).getElements(), containsInAnyOrder( - WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1024)), - WindowedValue.valueInGlobalWindow(4, PaneInfo.NO_FIRING), - WindowedValue.valueInGlobalWindow(1))); + WindowedValues.timestampedValueInGlobalWindow(2, new Instant(1024)), + WindowedValues.valueInGlobalWindow(4, PaneInfo.NO_FIRING), + WindowedValues.valueInGlobalWindow(1))); assertThat( flattenedRightBundle.commit(Instant.now()).getElements(), containsInAnyOrder( - WindowedValue.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.timestampedValueInGlobalWindow(-4, new Instant(-4096)), - WindowedValue.valueInGlobalWindow(-1))); + WindowedValues.valueInGlobalWindow(2, PaneInfo.ON_TIME_AND_ONLY_FIRING), + WindowedValues.timestampedValueInGlobalWindow(-4, new Instant(-4096)), + WindowedValues.valueInGlobalWindow(-1))); } @Test diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java index 63b6fb5c29e9..e6057f3b4d87 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyEvaluatorFactoryTest.java @@ -31,9 +31,10 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashMultiset; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multiset; @@ -91,12 +92,12 @@ public void testInMemoryEvaluator() throws Exception { new GroupByKeyOnlyEvaluatorFactory(evaluationContext) .forApplication(DirectGraphs.getProducer(groupedKvs), inputBundle); - evaluator.processElement(WindowedValue.valueInGlobalWindow(firstFoo)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(secondFoo)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(thirdFoo)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(firstBar)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(secondBar)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(firstBaz)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(firstFoo)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(secondFoo)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(thirdFoo)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(firstBar)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(secondBar)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(firstBaz)); evaluator.finishBundle(); @@ -107,9 +108,9 @@ public void testInMemoryEvaluator() throws Exception { KeyedWorkItems.elementsWorkItem( "foo", ImmutableSet.of( - WindowedValue.valueInGlobalWindow(-1), - WindowedValue.valueInGlobalWindow(1), - WindowedValue.valueInGlobalWindow(3))), + WindowedValues.valueInGlobalWindow(-1), + WindowedValues.valueInGlobalWindow(1), + WindowedValues.valueInGlobalWindow(3))), keyCoder))); assertThat( barBundle.commit(Instant.now()).getElements(), @@ -118,15 +119,15 @@ public void testInMemoryEvaluator() throws Exception { KeyedWorkItems.elementsWorkItem( "bar", ImmutableSet.of( - WindowedValue.valueInGlobalWindow(12), - WindowedValue.valueInGlobalWindow(22))), + WindowedValues.valueInGlobalWindow(12), + WindowedValues.valueInGlobalWindow(22))), keyCoder))); assertThat( bazBundle.commit(Instant.now()).getElements(), contains( new KeyedWorkItemMatcher<>( KeyedWorkItems.elementsWorkItem( - "baz", ImmutableSet.of(WindowedValue.valueInGlobalWindow(Integer.MAX_VALUE))), + "baz", ImmutableSet.of(WindowedValues.valueInGlobalWindow(Integer.MAX_VALUE))), keyCoder))); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java index 135f42b01323..9fd07742627d 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/GroupByKeyOnlyEvaluatorFactoryTest.java @@ -31,9 +31,10 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashMultiset; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multiset; @@ -90,12 +91,12 @@ public void testInMemoryEvaluator() throws Exception { new GroupByKeyOnlyEvaluatorFactory(evaluationContext) .forApplication(DirectGraphs.getProducer(groupedKvs), inputBundle); - evaluator.processElement(WindowedValue.valueInGlobalWindow(firstFoo)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(secondFoo)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(thirdFoo)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(firstBar)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(secondBar)); - evaluator.processElement(WindowedValue.valueInGlobalWindow(firstBaz)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(firstFoo)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(secondFoo)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(thirdFoo)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(firstBar)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(secondBar)); + evaluator.processElement(WindowedValues.valueInGlobalWindow(firstBaz)); evaluator.finishBundle(); @@ -106,9 +107,9 @@ public void testInMemoryEvaluator() throws Exception { KeyedWorkItems.elementsWorkItem( "foo", ImmutableSet.of( - WindowedValue.valueInGlobalWindow(-1), - WindowedValue.valueInGlobalWindow(1), - WindowedValue.valueInGlobalWindow(3))), + WindowedValues.valueInGlobalWindow(-1), + WindowedValues.valueInGlobalWindow(1), + WindowedValues.valueInGlobalWindow(3))), keyCoder))); assertThat( barBundle.commit(Instant.now()).getElements(), @@ -117,15 +118,15 @@ public void testInMemoryEvaluator() throws Exception { KeyedWorkItems.elementsWorkItem( "bar", ImmutableSet.of( - WindowedValue.valueInGlobalWindow(12), - WindowedValue.valueInGlobalWindow(22))), + WindowedValues.valueInGlobalWindow(12), + WindowedValues.valueInGlobalWindow(22))), keyCoder))); assertThat( bazBundle.commit(Instant.now()).getElements(), contains( new KeyedWorkItemMatcher<>( KeyedWorkItems.elementsWorkItem( - "baz", ImmutableSet.of(WindowedValue.valueInGlobalWindow(Integer.MAX_VALUE))), + "baz", ImmutableSet.of(WindowedValues.valueInGlobalWindow(Integer.MAX_VALUE))), keyCoder))); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java index 5dcfa90cb4c7..bdfbb04d78c8 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityCheckingBundleFactoryTest.java @@ -30,8 +30,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.IllegalMutationException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; @@ -65,11 +66,11 @@ public void setup() { public void rootBundleSucceeds() { UncommittedBundle root = factory.createRootBundle(); byte[] array = new byte[] {0, 1, 2}; - root.add(WindowedValue.valueInGlobalWindow(array)); + root.add(WindowedValues.valueInGlobalWindow(array)); CommittedBundle committed = root.commit(Instant.now()); assertThat( - committed.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(array))); + committed.getElements(), containsInAnyOrder(WindowedValues.valueInGlobalWindow(array))); } @Test @@ -78,7 +79,7 @@ public void noMutationKeyedBundleSucceeds() { factory.createKeyedBundle(StructuralKey.of("mykey", StringUtf8Coder.of()), transformed); WindowedValue windowedArray = - WindowedValue.of( + WindowedValues.of( new byte[] {4, 8, 12}, new Instant(891L), new IntervalWindow(new Instant(0), new Instant(1000)), @@ -94,7 +95,7 @@ public void noMutationCreateBundleSucceeds() { UncommittedBundle intermediate = factory.createBundle(transformed); WindowedValue windowedArray = - WindowedValue.of( + WindowedValues.of( new byte[] {4, 8, 12}, new Instant(891L), new IntervalWindow(new Instant(0), new Instant(1000)), @@ -113,7 +114,7 @@ public void mutationBeforeAddKeyedBundleSucceeds() { byte[] array = new byte[] {4, 8, 12}; array[0] = Byte.MAX_VALUE; WindowedValue windowedArray = - WindowedValue.of( + WindowedValues.of( array, new Instant(891L), new IntervalWindow(new Instant(0), new Instant(1000)), @@ -130,7 +131,7 @@ public void mutationBeforeAddCreateBundleSucceeds() { byte[] array = new byte[] {4, 8, 12}; WindowedValue windowedArray = - WindowedValue.of( + WindowedValues.of( array, new Instant(891L), new IntervalWindow(new Instant(0), new Instant(1000)), @@ -149,7 +150,7 @@ public void mutationAfterAddKeyedBundleThrows() { byte[] array = new byte[] {4, 8, 12}; WindowedValue windowedArray = - WindowedValue.of( + WindowedValues.of( array, new Instant(891L), new IntervalWindow(new Instant(0), new Instant(1000)), @@ -168,7 +169,7 @@ public void mutationAfterAddCreateBundleThrows() { byte[] array = new byte[] {4, 8, 12}; WindowedValue windowedArray = - WindowedValue.of( + WindowedValues.of( array, new Instant(891L), new IntervalWindow(new Instant(0), new Instant(1000)), diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java index 3a574102f664..60db1eb85e02 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutabilityEnforcementFactoryTest.java @@ -28,8 +28,9 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.util.IllegalMutationException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.joda.time.Instant; import org.junit.Before; import org.junit.Rule; @@ -71,7 +72,7 @@ public void processElement(ProcessContext c) throws Exception { @Test public void unchangedSucceeds() { - WindowedValue element = WindowedValue.valueInGlobalWindow("bar".getBytes(UTF_8)); + WindowedValue element = WindowedValues.valueInGlobalWindow("bar".getBytes(UTF_8)); CommittedBundle elements = bundleFactory.createBundle(pcollection).add(element).commit(Instant.now()); @@ -86,7 +87,7 @@ public void unchangedSucceeds() { @Test public void mutatedDuringProcessElementThrows() { - WindowedValue element = WindowedValue.valueInGlobalWindow("bar".getBytes(UTF_8)); + WindowedValue element = WindowedValues.valueInGlobalWindow("bar".getBytes(UTF_8)); CommittedBundle elements = bundleFactory.createBundle(pcollection).add(element).commit(Instant.now()); @@ -107,7 +108,7 @@ public void mutatedDuringProcessElementThrows() { @Test public void mutatedAfterProcessElementFails() { - WindowedValue element = WindowedValue.valueInGlobalWindow("bar".getBytes(UTF_8)); + WindowedValue element = WindowedValues.valueInGlobalWindow("bar".getBytes(UTF_8)); CommittedBundle elements = bundleFactory.createBundle(pcollection).add(element).commit(Instant.now()); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java index 5b008c3235ef..6b84908d626e 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImmutableListBundleFactoryTest.java @@ -37,9 +37,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.Matcher; import org.joda.time.Duration; @@ -131,9 +132,9 @@ public void getElementsBeforeAddShouldReturnEmptyIterable() { @Test public void getElementsAfterAddShouldReturnAddedElements() { - WindowedValue firstValue = WindowedValue.valueInGlobalWindow(1); + WindowedValue firstValue = WindowedValues.valueInGlobalWindow(1); WindowedValue secondValue = - WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1000L)); + WindowedValues.timestampedValueInGlobalWindow(2, new Instant(1000L)); afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue)); } @@ -141,7 +142,7 @@ public void getElementsAfterAddShouldReturnAddedElements() { @Test public void addElementsAtEndOfTimeThrows() { Instant timestamp = BoundedWindow.TIMESTAMP_MAX_VALUE; - WindowedValue value = WindowedValue.timestampedValueInGlobalWindow(1, timestamp); + WindowedValue value = WindowedValues.timestampedValueInGlobalWindow(1, timestamp); UncommittedBundle bundle = bundleFactory.createRootBundle(); thrown.expect(IllegalArgumentException.class); @@ -152,7 +153,7 @@ public void addElementsAtEndOfTimeThrows() { @Test public void addElementsPastEndOfTimeThrows() { Instant timestamp = BoundedWindow.TIMESTAMP_MAX_VALUE.plus(Duration.standardMinutes(2)); - WindowedValue value = WindowedValue.timestampedValueInGlobalWindow(1, timestamp); + WindowedValue value = WindowedValues.timestampedValueInGlobalWindow(1, timestamp); UncommittedBundle bundle = bundleFactory.createRootBundle(); thrown.expect(IllegalArgumentException.class); @@ -163,21 +164,21 @@ public void addElementsPastEndOfTimeThrows() { @SuppressWarnings("unchecked") @Test public void withElementsShouldReturnIndependentBundle() { - WindowedValue firstValue = WindowedValue.valueInGlobalWindow(1); + WindowedValue firstValue = WindowedValues.valueInGlobalWindow(1); WindowedValue secondValue = - WindowedValue.timestampedValueInGlobalWindow(2, new Instant(1000L)); + WindowedValues.timestampedValueInGlobalWindow(2, new Instant(1000L)); CommittedBundle committed = afterCommitGetElementsShouldHaveAddedElements(ImmutableList.of(firstValue, secondValue)); WindowedValue firstReplacement = - WindowedValue.of( + WindowedValues.of( 9, new Instant(2048L), new IntervalWindow(new Instant(2044L), Instant.now()), PaneInfo.NO_FIRING); WindowedValue secondReplacement = - WindowedValue.timestampedValueInGlobalWindow(-1, Instant.now()); + WindowedValues.timestampedValueInGlobalWindow(-1, Instant.now()); CommittedBundle withed = committed.withElements(ImmutableList.of(firstReplacement, secondReplacement)); @@ -194,23 +195,25 @@ public void withElementsShouldReturnIndependentBundle() { @Test public void addAfterCommitShouldThrowException() { UncommittedBundle bundle = bundleFactory.createRootBundle(); - bundle.add(WindowedValue.valueInGlobalWindow(1)); + bundle.add(WindowedValues.valueInGlobalWindow(1)); CommittedBundle firstCommit = bundle.commit(Instant.now()); - assertThat(firstCommit.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(1))); + assertThat( + firstCommit.getElements(), containsInAnyOrder(WindowedValues.valueInGlobalWindow(1))); thrown.expect(IllegalStateException.class); thrown.expectMessage("3"); thrown.expectMessage("committed"); - bundle.add(WindowedValue.valueInGlobalWindow(3)); + bundle.add(WindowedValues.valueInGlobalWindow(3)); } @Test public void commitAfterCommitShouldThrowException() { UncommittedBundle bundle = bundleFactory.createRootBundle(); - bundle.add(WindowedValue.valueInGlobalWindow(1)); + bundle.add(WindowedValues.valueInGlobalWindow(1)); CommittedBundle firstCommit = bundle.commit(Instant.now()); - assertThat(firstCommit.getElements(), containsInAnyOrder(WindowedValue.valueInGlobalWindow(1))); + assertThat( + firstCommit.getElements(), containsInAnyOrder(WindowedValues.valueInGlobalWindow(1))); thrown.expect(IllegalStateException.class); thrown.expectMessage("committed"); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactoryTest.java index 758b6811c496..5c3718b14456 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ImpulseEvaluatorFactoryTest.java @@ -32,8 +32,9 @@ import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; import org.junit.Before; @@ -64,7 +65,7 @@ public void testImpulse() throws Exception { ImpulseEvaluatorFactory factory = new ImpulseEvaluatorFactory(context); - WindowedValue inputShard = WindowedValue.valueInGlobalWindow(new ImpulseShard()); + WindowedValue inputShard = WindowedValues.valueInGlobalWindow(new ImpulseShard()); CommittedBundle inputShardBundle = bundleFactory.createRootBundle().add(inputShard).commit(Instant.now()); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java index a228da6eff54..95338e4bf330 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/KeyedPValueTrackingVisitorTest.java @@ -40,11 +40,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; import org.junit.Before; @@ -117,7 +118,7 @@ public void unkeyedInputWithKeyPreserving() { KvCoder.of( StringUtf8Coder.of(), IterableCoder.of( - WindowedValue.getValueOnlyCoder( + WindowedValues.getValueOnlyCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of())))))); PCollection>> unkeyed = @@ -141,7 +142,7 @@ public void keyedInputWithKeyPreserving() { Create.of( KV.of( "hello", - WindowedValue.of( + WindowedValues.of( KV.of("hello", 3), new Instant(0), new IntervalWindow(new Instant(0), new Instant(9)), @@ -149,7 +150,7 @@ public void keyedInputWithKeyPreserving() { .withCoder( KvCoder.of( StringUtf8Coder.of(), - WindowedValue.getValueOnlyCoder( + WindowedValues.getValueOnlyCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()))))); TupleTag>> keyedTag = new TupleTag<>(); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java index d7f82c4e045b..4429385d7975 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ParDoEvaluatorTest.java @@ -45,11 +45,12 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.IdentitySideInputWindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -102,11 +103,11 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { ParDoEvaluator evaluator = createEvaluator(singletonView, fn, inputPc, output); IntervalWindow nonGlobalWindow = new IntervalWindow(new Instant(0), new Instant(10_000L)); - WindowedValue first = WindowedValue.valueInGlobalWindow(3); + WindowedValue first = WindowedValues.valueInGlobalWindow(3); WindowedValue second = - WindowedValue.of(2, new Instant(1234L), nonGlobalWindow, PaneInfo.NO_FIRING); + WindowedValues.of(2, new Instant(1234L), nonGlobalWindow, PaneInfo.NO_FIRING); WindowedValue third = - WindowedValue.of( + WindowedValues.of( 1, new Instant(2468L), ImmutableList.of(nonGlobalWindow, GlobalWindow.INSTANCE), @@ -120,14 +121,14 @@ public void sideInputsNotReadyResultHasUnprocessedElements() { assertThat( result.getUnprocessedElements(), Matchers.>containsInAnyOrder( - second, WindowedValue.of(1, new Instant(2468L), nonGlobalWindow, PaneInfo.NO_FIRING))); + second, WindowedValues.of(1, new Instant(2468L), nonGlobalWindow, PaneInfo.NO_FIRING))); assertThat(result.getOutputBundles(), Matchers.contains(outputBundle)); assertThat(fn.processed, containsInAnyOrder(1, 3)); assertThat( Iterables.getOnlyElement(result.getOutputBundles()).commit(Instant.now()).getElements(), containsInAnyOrder( first.withValue(8), - WindowedValue.timestampedValueInGlobalWindow(6, new Instant(2468L)))); + WindowedValues.timestampedValueInGlobalWindow(6, new Instant(2468L)))); } private ParDoEvaluator createEvaluator( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java index cb8d655aa602..2f5b48192716 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/SideInputContainerTest.java @@ -43,11 +43,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.util.concurrent.MoreExecutors; import org.joda.time.Duration; @@ -126,13 +127,13 @@ public void getAfterWriteReturnsPaneInWindow() throws Exception { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("one", 1))) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(1L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING)); } for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("two", 2))) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(20L), FIRST_WINDOW, PaneInfo.ON_TIME_AND_ONLY_FIRING)); } container.write(mapView, valuesBuilder.build()); @@ -150,7 +151,7 @@ public void getReturnsLatestPaneInWindow() throws Exception { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("one", 1))) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(1L), SECOND_WINDOW, @@ -159,7 +160,7 @@ public void getReturnsLatestPaneInWindow() throws Exception { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("two", 2))) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(20L), SECOND_WINDOW, @@ -177,7 +178,7 @@ public void getReturnsLatestPaneInWindow() throws Exception { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("three", 3))) { overwriteValuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(300L), SECOND_WINDOW, @@ -232,7 +233,7 @@ public void writeForMultipleElementsInDifferentWindowsSucceeds() throws Exceptio for (Object materializedValue : materializeValuesFor(singletonView.getPipeline().getOptions(), View.asSingleton(), 2.875)) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, FIRST_WINDOW.maxTimestamp().minus(Duration.millis(200L)), FIRST_WINDOW, @@ -241,7 +242,7 @@ public void writeForMultipleElementsInDifferentWindowsSucceeds() throws Exceptio for (Object materializedValue : materializeValuesFor(singletonView.getPipeline().getOptions(), View.asSingleton(), 4.125)) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, SECOND_WINDOW.maxTimestamp().minus(Duration.millis(2_000_000L)), SECOND_WINDOW, @@ -266,7 +267,7 @@ public void writeForMultipleIdenticalElementsInSameWindowSucceeds() throws Excep for (Object materializedValue : materializeValuesFor(iterableView.getPipeline().getOptions(), View.asIterable(), 44, 44)) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, FIRST_WINDOW.maxTimestamp().minus(Duration.millis(200L)), FIRST_WINDOW, @@ -287,7 +288,7 @@ public void writeForElementInMultipleWindowsSucceeds() throws Exception { for (Object materializedValue : materializeValuesFor(singletonView.getPipeline().getOptions(), View.asSingleton(), 2.875)) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, FIRST_WINDOW.maxTimestamp().minus(Duration.millis(200L)), ImmutableList.of(FIRST_WINDOW, SECOND_WINDOW), @@ -312,7 +313,7 @@ public void finishDoesNotOverwriteWrittenElements() throws Exception { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("one", 1))) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(1L), SECOND_WINDOW, @@ -321,7 +322,7 @@ public void finishDoesNotOverwriteWrittenElements() throws Exception { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("two", 2))) { valuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, new Instant(20L), SECOND_WINDOW, @@ -372,7 +373,7 @@ public void isReadyForSomeNotReadyViewsFalseUntilElements() { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("one", 1))) { mapValuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, SECOND_WINDOW.maxTimestamp().minus(Duration.millis(100L)), SECOND_WINDOW, @@ -391,7 +392,7 @@ public void isReadyForSomeNotReadyViewsFalseUntilElements() { for (Object materializedValue : materializeValuesFor(mapView.getPipeline().getOptions(), View.asMap(), KV.of("too", 2))) { newMapValuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, FIRST_WINDOW.maxTimestamp().minus(Duration.millis(100L)), FIRST_WINDOW, @@ -405,7 +406,7 @@ public void isReadyForSomeNotReadyViewsFalseUntilElements() { for (Object materializedValue : materializeValuesFor(singletonView.getPipeline().getOptions(), View.asSingleton(), 1.25)) { singletonValuesBuilder.add( - WindowedValue.of( + WindowedValues.of( materializedValue, SECOND_WINDOW.maxTimestamp().minus(Duration.millis(100L)), SECOND_WINDOW, diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java index 7746e48317d5..cd3f0ef925e4 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/StatefulParDoEvaluatorFactoryTest.java @@ -60,7 +60,6 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; @@ -69,6 +68,8 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; 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; @@ -203,7 +204,7 @@ public void process(ProcessContext c) {} // depend on the window. String key = "hello"; WindowedValue> firstKv = - WindowedValue.of(KV.of(key, 1), new Instant(3), firstWindow, PaneInfo.NO_FIRING); + WindowedValues.of(KV.of(key, 1), new Instant(3), firstWindow, PaneInfo.NO_FIRING); WindowedValue>> gbkOutputElement = firstKv.withValue( diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java index 188c5e66b1c4..164aad503ff9 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/TestStreamEvaluatorFactoryTest.java @@ -32,9 +32,10 @@ import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.hamcrest.Matchers; import org.joda.time.Duration; @@ -159,18 +160,18 @@ public void producesElementsInSequence() throws Exception { .commit(Instant.now()) .getElements(), Matchers.containsInAnyOrder( - WindowedValue.valueInGlobalWindow(1), - WindowedValue.valueInGlobalWindow(2), - WindowedValue.valueInGlobalWindow(3))); + WindowedValues.valueInGlobalWindow(1), + WindowedValues.valueInGlobalWindow(2), + WindowedValues.valueInGlobalWindow(3))); assertThat( Iterables.getOnlyElement(thirdResult.getOutputBundles()) .commit(Instant.now()) .getElements(), Matchers.containsInAnyOrder( - WindowedValue.valueInGlobalWindow(4), - WindowedValue.valueInGlobalWindow(5), - WindowedValue.valueInGlobalWindow(6))); + WindowedValues.valueInGlobalWindow(4), + WindowedValues.valueInGlobalWindow(5), + WindowedValues.valueInGlobalWindow(6))); assertThat(fifthResult.getOutputBundles(), Matchers.emptyIterable()); assertThat(fifthResult.getWatermarkHold(), equalTo(BoundedWindow.TIMESTAMP_MAX_VALUE)); diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java index d31ce1f78ba5..5413a694e92b 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/UnboundedReadEvaluatorFactoryTest.java @@ -72,11 +72,12 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ContiguousSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.DiscreteDomain; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -311,7 +312,7 @@ public void evaluatorReusesReaderAndClosesAtTheEnd() throws Exception { when(context.createBundle(pcollection)).thenReturn(output); WindowedValue> shard = - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( UnboundedSourceShard.unstarted(source, NeverDeduplicator.create())); CommittedBundle> inputBundle = bundleFactory @@ -356,7 +357,7 @@ public void evaluatorClosesReaderAndResumesFromCheckpoint() throws Exception { when(context.createBundle(pcollection)).thenReturn(output); WindowedValue> shard = - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( UnboundedSourceShard.unstarted(source, NeverDeduplicator.create())); CommittedBundle> inputBundle = bundleFactory @@ -402,7 +403,7 @@ public void evaluatorThrowsInCloseRethrows() throws Exception { when(context.createBundle(pcollection)).thenReturn(output); WindowedValue> shard = - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( UnboundedSourceShard.unstarted(source, NeverDeduplicator.create())); CommittedBundle> inputBundle = bundleFactory @@ -469,7 +470,7 @@ private void processElement(final TestUnboundedSource source) throws Exc final UnboundedSourceShard shard = UnboundedSourceShard.of(source, new NeverDeduplicator(), reader, null); final WindowedValue> value = - WindowedValue.of( + WindowedValues.of( shard, BoundedWindow.TIMESTAMP_MAX_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); TestUnboundedSource.readerClosedCount = 0; evaluator.processElement(value); @@ -480,7 +481,7 @@ private void processElement(final TestUnboundedSource source) throws Exc * is the epoch offset by the value of the element. */ private static WindowedValue tgw(Long elem) { - return WindowedValue.timestampedValueInGlobalWindow(elem, new Instant(elem)); + return WindowedValues.timestampedValueInGlobalWindow(elem, new Instant(elem)); } private static class LongToInstantFn implements SerializableFunction { diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java index 91e094927e70..07f0136fff39 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/ViewEvaluatorFactoryTest.java @@ -33,9 +33,10 @@ import org.apache.beam.sdk.transforms.Values; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.WithKeys; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Instant; import org.junit.Rule; @@ -72,14 +73,14 @@ public void testInMemoryEvaluator() throws Exception { TransformEvaluator> evaluator = new ViewEvaluatorFactory(context).forApplication(producer, inputBundle); - evaluator.processElement(WindowedValue.valueInGlobalWindow(ImmutableList.of("foo", "bar"))); + evaluator.processElement(WindowedValues.valueInGlobalWindow(ImmutableList.of("foo", "bar"))); assertThat(viewWriter.latest, nullValue()); evaluator.finishBundle(); assertThat( viewWriter.latest, containsInAnyOrder( - WindowedValue.valueInGlobalWindow("foo"), WindowedValue.valueInGlobalWindow("bar"))); + WindowedValues.valueInGlobalWindow("foo"), WindowedValues.valueInGlobalWindow("bar"))); } private static class TestViewWriter implements PCollectionViewWriter { diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java index 847acd493553..785c59bae84f 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WatermarkManagerTest.java @@ -61,11 +61,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -320,12 +321,12 @@ public void getWatermarkMultiIdenticalInput() { CommittedBundle root = bundleFactory .createRootBundle() - .add(WindowedValue.valueInGlobalWindow(null)) + .add(WindowedValues.valueInGlobalWindow(null)) .commit(clock.now()); CommittedBundle createBundle = bundleFactory .createBundle(impulse) - .add(WindowedValue.timestampedValueInGlobalWindow(new byte[1], new Instant(33536))) + .add(WindowedValues.timestampedValueInGlobalWindow(new byte[1], new Instant(33536))) .commit(clock.now()); Map, Collection>> initialInputs = @@ -485,14 +486,15 @@ public void updateWatermarkWithKeyedWatermarkHolds() { CommittedBundle firstKeyBundle = bundleFactory .createKeyedBundle(StructuralKey.of("Odd", StringUtf8Coder.of()), impulse) - .add(WindowedValue.timestampedValueInGlobalWindow(new byte[1], new Instant(1_000_000L))) - .add(WindowedValue.timestampedValueInGlobalWindow(new byte[3], new Instant(-1000L))) + .add( + WindowedValues.timestampedValueInGlobalWindow(new byte[1], new Instant(1_000_000L))) + .add(WindowedValues.timestampedValueInGlobalWindow(new byte[3], new Instant(-1000L))) .commit(clock.now()); CommittedBundle secondKeyBundle = bundleFactory .createKeyedBundle(StructuralKey.of("Even", StringUtf8Coder.of()), impulse) - .add(WindowedValue.timestampedValueInGlobalWindow(new byte[2], new Instant(1234L))) + .add(WindowedValues.timestampedValueInGlobalWindow(new byte[2], new Instant(1234L))) .commit(clock.now()); manager.updateWatermarks( @@ -648,11 +650,11 @@ public void updateWatermarkWithHoldsShouldBeMonotonic() { @Test public void updateWatermarkWithUnprocessedElements() { - WindowedValue first = WindowedValue.valueInGlobalWindow(new byte[1]); + WindowedValue first = WindowedValues.valueInGlobalWindow(new byte[1]); WindowedValue second = - WindowedValue.timestampedValueInGlobalWindow(new byte[2], new Instant(-1000L)); + WindowedValues.timestampedValueInGlobalWindow(new byte[2], new Instant(-1000L)); WindowedValue third = - WindowedValue.timestampedValueInGlobalWindow(new byte[3], new Instant(1234L)); + WindowedValues.timestampedValueInGlobalWindow(new byte[3], new Instant(1234L)); CommittedBundle impulseBundle = bundleFactory.createBundle(impulse).add(first).add(second).add(third).commit(clock.now()); @@ -683,12 +685,12 @@ public void updateWatermarkWithUnprocessedElements() { @Test public void updateWatermarkWithCompletedElementsNotPending() { WindowedValue first = - WindowedValue.timestampedValueInGlobalWindow(new byte[1], new Instant(22)); + WindowedValues.timestampedValueInGlobalWindow(new byte[1], new Instant(22)); CommittedBundle impulseBundle = bundleFactory.createBundle(impulse).add(first).commit(clock.now()); WindowedValue second = - WindowedValue.timestampedValueInGlobalWindow(new byte[2], new Instant(22)); + WindowedValues.timestampedValueInGlobalWindow(new byte[2], new Instant(22)); CommittedBundle neverImpulseBundle = bundleFactory.createBundle(impulse).add(second).commit(clock.now()); @@ -795,14 +797,14 @@ public void updateWatermarkWithDifferentWindowedValueInstances() { Collections.>singleton( bundleFactory .createBundle(impulse) - .add(WindowedValue.valueInGlobalWindow(new byte[1])) + .add(WindowedValues.valueInGlobalWindow(new byte[1])) .commit(Instant.now())), BoundedWindow.TIMESTAMP_MAX_VALUE); CommittedBundle impulseBundle = bundleFactory .createBundle(impulse) - .add(WindowedValue.valueInGlobalWindow(new byte[1])) + .add(WindowedValues.valueInGlobalWindow(new byte[1])) .commit(Instant.now()); manager.updateWatermarks( impulseBundle, @@ -1792,7 +1794,7 @@ private final CommittedBundle timestampedBundle( UncommittedBundle bundle = bundleFactory.createBundle(pc); for (TimestampedValue value : values) { bundle.add( - WindowedValue.timestampedValueInGlobalWindow(value.getValue(), value.getTimestamp())); + WindowedValues.timestampedValueInGlobalWindow(value.getValue(), value.getTimestamp())); } return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE); } @@ -1806,7 +1808,7 @@ private final CommittedBundle multiWindowedBundle(PCollection pc, T... new IntervalWindow(BoundedWindow.TIMESTAMP_MIN_VALUE, new Instant(0))); for (T value : values) { bundle.add( - WindowedValue.of(value, BoundedWindow.TIMESTAMP_MIN_VALUE, windows, PaneInfo.NO_FIRING)); + WindowedValues.of(value, BoundedWindow.TIMESTAMP_MIN_VALUE, windows, PaneInfo.NO_FIRING)); } return bundle.commit(BoundedWindow.TIMESTAMP_MAX_VALUE); } diff --git a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java index b334c8431683..ccf6b53fc8fb 100644 --- a/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java +++ b/runners/direct-java/src/test/java/org/apache/beam/runners/direct/WindowEvaluatorFactoryTest.java @@ -41,8 +41,9 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -70,11 +71,11 @@ public class WindowEvaluatorFactoryTest { private BundleFactory bundleFactory; private WindowedValue valueInGlobalWindow = - WindowedValue.timestampedValueInGlobalWindow(3L, new Instant(2L)); + WindowedValues.timestampedValueInGlobalWindow(3L, new Instant(2L)); private final PaneInfo intervalWindowPane = PaneInfo.createPane(false, false, Timing.LATE, 3, 2); private WindowedValue valueInIntervalWindow = - WindowedValue.of( + WindowedValues.of( 2L, new Instant(-10L), new IntervalWindow(new Instant(-100), EPOCH), intervalWindowPane); private IntervalWindow intervalWindow1 = @@ -86,7 +87,7 @@ public class WindowEvaluatorFactoryTest { private final PaneInfo multiWindowPane = PaneInfo.createPane(false, true, Timing.ON_TIME, 3, 0); private WindowedValue valueInGlobalAndTwoIntervalWindows = - WindowedValue.of( + WindowedValues.of( 1L, EPOCH.plus(Duration.standardDays(3)), ImmutableList.of(GlobalWindow.INSTANCE, intervalWindow1, intervalWindow2), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java index 090923a1cb92..dd0ced6ba11c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchPortablePipelineTranslator.java @@ -58,8 +58,6 @@ import org.apache.beam.sdk.transforms.join.UnionCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.NativeTransforms; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; @@ -70,6 +68,9 @@ import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; @@ -361,7 +362,7 @@ private static void translateExecutableStage( if (stagePayload.getUserStatesCount() > 0 || stagePayload.getTimersCount() > 0) { Coder valueCoder = - ((WindowedValue.FullWindowedValueCoder) windowedInputCoder).getValueCoder(); + ((WindowedValues.FullWindowedValueCoder) windowedInputCoder).getValueCoder(); // Stateful stages are only allowed of KV input to be able to group on the key if (!(valueCoder instanceof KvCoder)) { throw new IllegalStateException( @@ -437,7 +438,7 @@ private static void translateFlatten( }) .returns( new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( (Coder) VoidCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions())); } else { @@ -508,7 +509,7 @@ private static void translateGroupByKey( CoderRegistry.createDefault(), inputElementCoder.getValueCoder()); Coder>>> outputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(inputElementCoder.getKeyCoder(), accumulatorCoder), windowingStrategy.getWindowFn().windowCoder()); @@ -554,7 +555,7 @@ private static void translateImpulse( PTransformNode transform, RunnerApi.Pipeline pipeline, BatchTranslationContext context) { TypeInformation> typeInformation = new CoderTypeInformation<>( - WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions()); DataSource> dataSource = new DataSource<>( diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java index 383730daeef8..a56dc9feee8f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTransformTranslators.java @@ -63,7 +63,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CreatePCollectionViewTranslation; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; @@ -76,6 +75,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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; @@ -294,7 +295,7 @@ public void translateNode( inputCoder.getKeyCoder(), windowingStrategy.getWindowFn().windowCoder())); final TypeInformation>>> outputTypeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( inputCoder.getKeyCoder(), IterableCoder.of(inputCoder.getValueCoder())), windowingStrategy.getWindowFn().windowCoder()), @@ -345,7 +346,7 @@ public void translateNode( TypeInformation>>> partialReduceTypeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(inputCoder.getKeyCoder(), accumulatorCoder), windowingStrategy.getWindowFn().windowCoder()), context.getPipelineOptions()); @@ -411,8 +412,8 @@ public void translateNode( outputType, FlinkIdentityFunction.of(), getCurrentTransformName(context)); - WindowedValue.WindowedValueCoder> kvWvCoder = - (WindowedValue.WindowedValueCoder>) outputType.getCoder(); + WindowedValues.WindowedValueCoder> kvWvCoder = + (WindowedValues.WindowedValueCoder>) outputType.getCoder(); KvCoder kvCoder = (KvCoder) kvWvCoder.getValueCoder(); DataSet>> reshuffle = retypedDataSet @@ -648,7 +649,8 @@ public void translateNode( TypeInformation> typeInformation = new CoderTypeInformation<>( - WindowedValue.getFullCoder(unionCoder, windowingStrategy.getWindowFn().windowCoder()), + WindowedValues.getFullCoder( + unionCoder, windowingStrategy.getWindowFn().windowCoder()), context.getPipelineOptions()); List> sideInputs; @@ -789,7 +791,7 @@ public void translateNode( }) .returns( new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( (Coder) VoidCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions())); } else { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java index 952546251c7a..2ccbf24879ae 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkBatchTranslationContext.java @@ -27,11 +27,12 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.DataSet; @@ -142,8 +143,8 @@ TypeInformation> getTypeInfo(PCollection collection) { TypeInformation> getTypeInfo( Coder coder, WindowingStrategy windowingStrategy) { - WindowedValue.FullWindowedValueCoder windowedValueCoder = - WindowedValue.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.FullWindowedValueCoder windowedValueCoder = + WindowedValues.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); return new CoderTypeInformation<>(windowedValueCoder, options); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingAggregationsTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingAggregationsTranslators.java index 1683ced890c7..60448db365df 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingAggregationsTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingAggregationsTranslators.java @@ -45,11 +45,12 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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; @@ -210,8 +211,8 @@ WindowDoFnOperator getWindowedAggregateDoFnOperato SingletonKeyedWorkItemCoder.of( keyCoder, inputKvCoder.getValueCoder(), windowingStrategy.getWindowFn().windowCoder()); - WindowedValue.FullWindowedValueCoder> windowedWorkItemCoder = - WindowedValue.getFullCoder(workItemCoder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.FullWindowedValueCoder> windowedWorkItemCoder = + WindowedValues.getFullCoder(workItemCoder, windowingStrategy.getWindowFn().windowCoder()); // Key selector WorkItemKeySelector workItemKeySelector = new WorkItemKeySelector<>(keyCoder); @@ -381,17 +382,17 @@ SingleOutputStreamOperator>>> batchGroupByK KvCoder.of(inputKvCoder.getKeyCoder(), accumulatorCoder); Coder>>> windowedAccumCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( accumKvCoder, input.getWindowingStrategy().getWindowFn().windowCoder()); Coder>>>> outputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(inputKvCoder.getKeyCoder(), IterableCoder.of(accumulatorCoder)), input.getWindowingStrategy().getWindowFn().windowCoder()); TypeInformation>>>> accumulatedTypeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( inputKvCoder.getKeyCoder(), IterableCoder.of(IterableCoder.of(inputKvCoder.getValueCoder()))), @@ -468,7 +469,7 @@ SingleOutputStreamOperator>> batchCombinePerKey( accumKvCoder = KvCoder.of(inputKvCoder.getKeyCoder(), accumulatorCoder); windowedAccumCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( accumKvCoder, input.getWindowingStrategy().getWindowFn().windowCoder()); } catch (CannotProvideCoderException e) { throw new RuntimeException(e); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java index 7d83c2e0019f..cad90de8ceed 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingPortablePipelineTranslator.java @@ -77,8 +77,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.ModelCoders; import org.apache.beam.sdk.util.construction.NativeTransforms; import org.apache.beam.sdk.util.construction.PTransformTranslation; @@ -97,6 +95,9 @@ import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; @@ -326,7 +327,7 @@ private void translateFlatten( }) .returns( new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( (Coder) VoidCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions())); context.addDataStream(Iterables.getOnlyElement(transform.getOutputsMap().values()), result); @@ -427,8 +428,8 @@ private SingleOutputStreamOperator>>> add inputElementCoder.getValueCoder(), windowingStrategy.getWindowFn().windowCoder()); - WindowedValue.FullWindowedValueCoder> windowedWorkItemCoder = - WindowedValue.getFullCoder(workItemCoder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.FullWindowedValueCoder> windowedWorkItemCoder = + WindowedValues.getFullCoder(workItemCoder, windowingStrategy.getWindowFn().windowCoder()); WorkItemKeySelector keySelector = new WorkItemKeySelector<>(inputElementCoder.getKeyCoder()); @@ -442,7 +443,7 @@ private SingleOutputStreamOperator>>> add Coder> accumulatorCoder = IterableCoder.of(inputElementCoder.getValueCoder()); Coder>>> outputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(inputElementCoder.getKeyCoder(), accumulatorCoder), windowingStrategy.getWindowFn().windowCoder()); @@ -520,11 +521,11 @@ private DataStream> translateBoundedSource( BoundedSource boundedSource = (BoundedSource) ReadTranslation.boundedSourceFromProto(payload); @SuppressWarnings("unchecked") - WindowedValue.FullWindowedValueCoder wireCoder = - (WindowedValue.FullWindowedValueCoder) + WindowedValues.FullWindowedValueCoder wireCoder = + (WindowedValues.FullWindowedValueCoder) instantiateCoder(outputCollectionId, pipeline.getComponents()); - WindowedValue.FullWindowedValueCoder sdkCoder = + WindowedValues.FullWindowedValueCoder sdkCoder = getSdkCoder(outputCollectionId, pipeline.getComponents()); CoderTypeInformation> outputTypeInfo = @@ -566,11 +567,11 @@ private static DataStream> translateUnboundedSource( try { @SuppressWarnings("unchecked") - WindowedValue.FullWindowedValueCoder wireCoder = - (WindowedValue.FullWindowedValueCoder) + WindowedValues.FullWindowedValueCoder wireCoder = + (WindowedValues.FullWindowedValueCoder) instantiateCoder(outputCollectionId, pipeline.getComponents()); - WindowedValue.FullWindowedValueCoder sdkCoder = + WindowedValues.FullWindowedValueCoder sdkCoder = getSdkCoder(outputCollectionId, pipeline.getComponents()); CoderTypeInformation> outputTypeInfo = @@ -581,7 +582,7 @@ private static DataStream> translateUnboundedSource( TypeInformation>> withIdTypeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( ValueWithRecordId.ValueWithRecordIdCoder.of(sdkCoder.getValueCoder()), windowStrategy.getWindowFn().windowCoder()), pipelineOptions); @@ -625,7 +626,7 @@ private static DataStream> translateUnboundedSource( * @param components the Pipeline components (proto) * @return SDK-side coder for the PCollection */ - private static WindowedValue.FullWindowedValueCoder getSdkCoder( + private static WindowedValues.FullWindowedValueCoder getSdkCoder( String pCollectionId, RunnerApi.Components components) { PipelineNode.PCollectionNode pCollectionNode = @@ -640,8 +641,8 @@ private static WindowedValue.FullWindowedValueCoder getSdkCoder( RehydratedComponents.forComponents(componentsBuilder.build()); try { @SuppressWarnings("unchecked") - WindowedValue.FullWindowedValueCoder res = - (WindowedValue.FullWindowedValueCoder) rehydratedComponents.getCoder(coderId); + WindowedValues.FullWindowedValueCoder res = + (WindowedValues.FullWindowedValueCoder) rehydratedComponents.getCoder(coderId); return res; } catch (IOException ex) { throw new IllegalStateException("Could not get SDK coder.", ex); @@ -677,7 +678,7 @@ private void translateImpulse( TypeInformation> typeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions()); long shutdownAfterIdleSourcesMs = context.getPipelineOptions().getShutdownSourcesAfterIdleMs(); @@ -706,7 +707,7 @@ private void translateStreamingImpulse( TypeInformation> typeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions()); ObjectMapper objectMapper = new ObjectMapper(); @@ -817,7 +818,7 @@ private void translateExecutableStage( if (stateful || hasSdfProcessFn) { // Stateful/SDF stages are only allowed of KV input. Coder valueCoder = - ((WindowedValue.FullWindowedValueCoder) windowedInputCoder).getValueCoder(); + ((WindowedValues.FullWindowedValueCoder) windowedInputCoder).getValueCoder(); if (!(valueCoder instanceof KvCoder)) { throw new IllegalStateException( String.format( diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java index 36cf035a33be..8a7ec8b59f60 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTransformTranslators.java @@ -78,7 +78,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.ReadTranslation; @@ -93,6 +92,8 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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.ImmutableMap; @@ -204,7 +205,7 @@ public void translateNode( TypeInformation>> withIdTypeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( ValueWithRecordId.ValueWithRecordIdCoder.of(coder), output.getWindowingStrategy().getWindowFn().windowCoder()), context.getPipelineOptions()); @@ -308,7 +309,7 @@ void translateNode(Impulse transform, FlinkStreamingTranslationContext context) TypeInformation> typeInfo = new CoderTypeInformation<>( - WindowedValue.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions()); SingleOutputStreamOperator> impulseOperator; @@ -966,7 +967,7 @@ public void translateNode( new KvToFlinkKeyKeySelector<>(inputKvCoder.getKeyCoder()); Coder>>> outputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( inputKvCoder.getKeyCoder(), IterableCoder.of(inputKvCoder.getValueCoder())), windowingStrategy.getWindowFn().windowCoder()); @@ -1125,8 +1126,8 @@ public void translateNode( inputKvCoder.getValueCoder(), input.getWindowingStrategy().getWindowFn().windowCoder()); - WindowedValue.ValueOnlyWindowedValueCoder> windowedWorkItemCoder = - WindowedValue.getValueOnlyCoder(workItemCoder); + WindowedValues.ValueOnlyWindowedValueCoder> windowedWorkItemCoder = + WindowedValues.getValueOnlyCoder(workItemCoder); CoderTypeInformation>> workItemTypeInfo = new CoderTypeInformation<>(windowedWorkItemCoder, context.getPipelineOptions()); @@ -1179,7 +1180,7 @@ public void flatMap( new SingletonKeyedWorkItem<>( in.getValue().getKey(), in.withValue(in.getValue().getValue())); - out.collect(WindowedValue.valueInGlobalWindow(workItem)); + out.collect(WindowedValues.valueInGlobalWindow(workItem)); } } } @@ -1210,7 +1211,7 @@ public void translateNode( }) .returns( new CoderTypeInformation<>( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( (Coder) VoidCoder.of(), GlobalWindow.Coder.INSTANCE), context.getPipelineOptions())); context.setOutputDataStream(context.getOutput(transform), result); @@ -1311,8 +1312,8 @@ void translateNode(TestStream testStream, FlinkStreamingTranslationContext co } }; - WindowedValue.FullWindowedValueCoder elementCoder = - WindowedValue.getFullCoder(valueCoder, GlobalWindow.Coder.INSTANCE); + WindowedValues.FullWindowedValueCoder elementCoder = + WindowedValues.getFullCoder(valueCoder, GlobalWindow.Coder.INSTANCE); DataStreamSource> source = context @@ -1403,7 +1404,7 @@ private SourceContextWrapper(SourceContext> ctx) { public void collect(WindowedValue> element) { OutputT originalValue = element.getValue().getValue(); WindowedValue output = - WindowedValue.of( + WindowedValues.of( originalValue, element.getTimestamp(), element.getWindows(), element.getPane()); ctx.collect(output); } @@ -1413,7 +1414,7 @@ public void collectWithTimestamp( WindowedValue> element, long timestamp) { OutputT originalValue = element.getValue().getValue(); WindowedValue output = - WindowedValue.of( + WindowedValues.of( originalValue, element.getTimestamp(), element.getWindows(), element.getPane()); ctx.collectWithTimestamp(output, timestamp); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java index 10ba64a77148..24c7b4336e2a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/FlinkStreamingTranslationContext.java @@ -27,13 +27,14 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.flink.api.common.typeinfo.TypeInformation; @@ -112,7 +113,7 @@ public void setCurrentTransform(AppliedPTransform currentTransform) { public Coder> getWindowedInputCoder(PCollection collection) { final Coder valueCoder = collection.getCoder(); - return WindowedValue.getFullCoder( + return WindowedValues.getFullCoder( valueCoder, collection.getWindowingStrategy().getWindowFn().windowCoder()); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataSetAdapter.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataSetAdapter.java index 4865a25f70eb..cefc8dd126fc 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataSetAdapter.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataSetAdapter.java @@ -32,7 +32,6 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.util.Preconditions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.values.PBegin; @@ -41,6 +40,8 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.java.DataSet; @@ -223,11 +224,11 @@ private FlinkBatchPortablePipelineTranslator.PTransformTranslator flink new MapOperator>( flinkInput, BeamAdapterCoderUtils.coderToTypeInformation( - WindowedValue.getValueOnlyCoder( + WindowedValues.getValueOnlyCoder( BeamAdapterCoderUtils.typeInformationToCoder( flinkInput.getType(), coderRegistry)), pipelineOptions), - x -> WindowedValue.valueInGlobalWindow(x), + x -> WindowedValues.valueInGlobalWindow(x), "AddGlobalWindows")); }; } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataStreamAdapter.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataStreamAdapter.java index 0a7a1fec803b..435182909fb7 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataStreamAdapter.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/adapter/BeamFlinkDataStreamAdapter.java @@ -34,7 +34,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.Preconditions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PipelineOptionsTranslation; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -42,6 +41,8 @@ import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.common.eventtime.WatermarkStrategy; @@ -239,7 +240,7 @@ public void processElement( Collector> out) throws Exception { out.collect( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( value, ctx.timestamp() == null ? BoundedWindow.TIMESTAMP_MIN_VALUE @@ -247,7 +248,7 @@ public void processElement( } }, BeamAdapterCoderUtils.coderToTypeInformation( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( BeamAdapterCoderUtils.typeInformationToCoder( flinkInput.getType(), coderRegistry), GlobalWindow.Coder.INSTANCE), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java index 0e437926bd7c..f1ec36564a4b 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/metrics/DoFnRunnerWithMetricsUpdate.java @@ -25,7 +25,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java index c4be4736104f..45ba611f6469 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/AbstractFlinkCombineRunner.java @@ -24,8 +24,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.flink.util.Collector; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java index eb31fcac6eb1..f153d9144be0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignContext.java @@ -19,7 +19,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java index d8c7219bdb63..07a47707827d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkAssignWindows.java @@ -20,7 +20,8 @@ import java.util.Collection; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; @@ -43,7 +44,7 @@ public void flatMap(WindowedValue input, Collector> collecto Collection windows = windowFn.assignWindows(new FlinkAssignContext<>(windowFn, input)); for (W window : windows) { collector.collect( - WindowedValue.of(input.getValue(), input.getTimestamp(), window, input.getPane())); + WindowedValues.of(input.getValue(), input.getTimestamp(), window, input.getPane())); } } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java index f437daf86e71..e12610dfe51c 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkDoFnFunction.java @@ -36,9 +36,10 @@ import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.flink.api.common.functions.AbstractRichFunction; @@ -219,7 +220,7 @@ public void setCollector(Collector> collector) { public void output(TupleTag tag, WindowedValue output) { Objects.requireNonNull(collector) .collect( - WindowedValue.of( + WindowedValues.of( new RawUnionValue(0 /* single output */, output.getValue()), output.getTimestamp(), output.getWindows(), @@ -252,7 +253,7 @@ public void setCollector(Collector> collector) { public void output(TupleTag tag, WindowedValue output) { Objects.requireNonNull(collector) .collect( - WindowedValue.of( + WindowedValues.of( new RawUnionValue(outputMap.get(tag), output.getValue()), output.getTimestamp(), output.getWindows(), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java index e9d2c3c4e004..6c13f8872e9d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunction.java @@ -58,10 +58,10 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.common.functions.AbstractRichFunction; import org.apache.flink.api.common.functions.GroupReduceFunction; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java index 467358c154db..6dd4a4135543 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStagePruningFunction.java @@ -21,7 +21,7 @@ import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.configuration.Configuration; import org.apache.flink.util.Collector; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExplodeWindowsFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExplodeWindowsFunction.java index b307878fb025..3066b17fcad0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExplodeWindowsFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkExplodeWindowsFunction.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.util.Collector; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java index b1b95c6b58e4..c5d6fc3774f7 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMergingNonShuffleReduceFunction.java @@ -24,9 +24,9 @@ import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java index 787b1729cb72..a660f5e9d65d 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkMultiOutputPruningFunction.java @@ -21,7 +21,7 @@ import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.functions.FlatMapFunction; import org.apache.flink.api.common.functions.RichFlatMapFunction; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java index fff44e327d02..bcc5a244d3b1 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkNonMergingReduceFunction.java @@ -24,8 +24,9 @@ import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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; @@ -101,7 +102,7 @@ public void reduce( Objects.requireNonNull(wv).getValue().getValue())); } coll.collect( - WindowedValue.of( + WindowedValues.of( KV.of(first.getValue().getKey(), values), combinedTimestamp, first.getWindows(), diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java index 0839812c27b1..31da50372588 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkPartialReduceFunction.java @@ -24,9 +24,9 @@ import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.RichGroupCombineFunction; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java index 13998693de7f..bc1cf36639a0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkReduceFunction.java @@ -24,9 +24,9 @@ import org.apache.beam.sdk.transforms.CombineFnBase; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.functions.RichGroupReduceFunction; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java index 16eac410f278..95fc4e8ab8b6 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/FlinkStatefulDoFnFunction.java @@ -49,10 +49,10 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.flink.api.common.functions.RichGroupReduceFunction; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java index 455a8540e505..19c6a6324bf0 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/HashingFlinkCombineRunner.java @@ -32,8 +32,9 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.java.tuple.Tuple2; @@ -115,7 +116,7 @@ public void combine( AccumT accumulator = entry.getValue().f0; Instant windowTimestamp = entry.getValue().f1; out.collect( - WindowedValue.of( + WindowedValues.of( KV.of( key, flinkCombiner.extractOutput( diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java index 577949bcbc8d..779c47dd0e6f 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunction.java @@ -17,7 +17,8 @@ */ package org.apache.beam.runners.flink.translation.functions; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; @@ -58,7 +59,7 @@ public void run(SourceContext> sourceContext) throws Excep if (Iterables.isEmpty(impulseEmitted.get())) { synchronized (sourceContext.getCheckpointLock()) { // emit single impulse element - sourceContext.collect(WindowedValue.valueInGlobalWindow(new byte[0])); + sourceContext.collect(WindowedValues.valueInGlobalWindow(new byte[0])); impulseEmitted.add(true); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java index d44428a84f51..200403a81e95 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SideInputInitializer.java @@ -33,8 +33,8 @@ import org.apache.beam.sdk.transforms.Materializations.MultimapView; import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.flink.api.common.functions.BroadcastVariableInitializer; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SingleWindowFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SingleWindowFlinkCombineRunner.java index 58ecfdf96e80..f33ecda636be 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SingleWindowFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SingleWindowFlinkCombineRunner.java @@ -24,8 +24,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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; @@ -97,7 +98,7 @@ public void combine( final AccumT accumulator = combinedState.f0; final Instant windowTimestamp = combinedState.f1; out.collect( - WindowedValue.of( + WindowedValues.of( KV.of( key, flinkCombiner.extractOutput( diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java index 149c3e284032..976b3dfe0868 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/functions/SortingFlinkCombineRunner.java @@ -26,8 +26,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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.Lists; @@ -109,7 +110,7 @@ public void combine( } else { // emit the value that we currently have out.collect( - WindowedValue.of( + WindowedValues.of( KV.of( key, flinkCombiner.extractOutput( @@ -130,7 +131,7 @@ public void combine( // emit the final accumulator out.collect( - WindowedValue.of( + WindowedValues.of( KV.of( key, flinkCombiner.extractOutput( @@ -161,7 +162,7 @@ private void mergeWindow(List>> elements) { WindowedValue> value = elements.get(j); elements.set( j, - WindowedValue.of( + WindowedValues.of( value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); } currentStart = i; @@ -174,7 +175,7 @@ private void mergeWindow(List>> elements) { WindowedValue> value = elements.get(j); elements.set( j, - WindowedValue.of( + WindowedValues.of( value.getValue(), value.getTimestamp(), currentWindow, value.getPane())); } } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java index 7e879fb27ddf..30bb94f1d1d5 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/KvKeySelector.java @@ -19,8 +19,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/WindowedKvKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/WindowedKvKeySelector.java index 13f55a4c1456..22766ea0c3e2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/WindowedKvKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/types/WindowedKvKeySelector.java @@ -20,8 +20,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; 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.primitives.Bytes; import org.apache.flink.api.common.typeinfo.TypeInformation; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/ImpulseInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/ImpulseInputFormat.java index 153c875ca143..2a0cf06cc3f2 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/ImpulseInputFormat.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/ImpulseInputFormat.java @@ -19,7 +19,8 @@ import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.RichInputFormat; import org.apache.flink.api.common.io.statistics.BaseStatistics; @@ -88,7 +89,7 @@ public WindowedValue nextRecord(WindowedValue windowedValue) { if (windowedValue != null) { return windowedValue; } - return WindowedValue.valueInGlobalWindow(new byte[0]); + return WindowedValues.valueInGlobalWindow(new byte[0]); } @Override diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java index a1b8bced7a1d..8a8d4cf3e9a5 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/SourceInputFormat.java @@ -29,7 +29,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.flink.api.common.io.DefaultInputSplitAssigner; import org.apache.flink.api.common.io.InputFormat; import org.apache.flink.api.common.io.RichInputFormat; @@ -162,7 +163,7 @@ public WindowedValue nextRecord(WindowedValue t) throws IOException { final Instant timestamp = reader.getCurrentTimestamp(); // advance reader to have a record ready next time inputAvailable = readerInvoker.invokeAdvance(reader); - return WindowedValue.of(current, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); + return WindowedValues.of(current, timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); } return null; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java index 01b12cfa717a..701df1c14267 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperator.java @@ -84,10 +84,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.NoopLock; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; 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.Joiner; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java index b48d6f2e4ec4..d5a7ff035efc 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperator.java @@ -97,7 +97,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; @@ -105,6 +104,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.StatusRuntimeException; @@ -530,7 +531,7 @@ void setTimer(Timer timerElement, TimerInternals.TimerData timerData) { FlinkKey encodedKey = (FlinkKey) keySelector.getKey( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( (InputT) KV.of(timerElement.getUserKey(), null))); // We have to synchronize to ensure the state backend is not concurrently accessed by the // state requests @@ -562,7 +563,8 @@ class SdfFlinkTimerInternalsFactory implements TimerInternalsFactory { @Override public TimerInternals timerInternalsForKey(InputT key) { try { - FlinkKey encodedKey = (FlinkKey) keySelector.getKey(WindowedValue.valueInGlobalWindow(key)); + FlinkKey encodedKey = + (FlinkKey) keySelector.getKey(WindowedValues.valueInGlobalWindow(key)); return new SdfFlinkTimerInternals(encodedKey); } catch (Exception e) { throw new RuntimeException("Couldn't get a timer internals", e); @@ -658,7 +660,8 @@ class SdfFlinkStateInternalsFactory implements StateInternalsFactory { @Override public StateInternals stateInternalsForKey(InputT key) { try { - FlinkKey encodedKey = (FlinkKey) keySelector.getKey(WindowedValue.valueInGlobalWindow(key)); + FlinkKey encodedKey = + (FlinkKey) keySelector.getKey(WindowedValues.valueInGlobalWindow(key)); return new SdfFlinkStateInternals(encodedKey); } catch (Exception e) { throw new RuntimeException("Couldn't get a state internals", e); diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToFlinkKeyKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToFlinkKeyKeySelector.java index a852a724c040..76c5e7ece6ed 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToFlinkKeyKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/KvToFlinkKeyKeySelector.java @@ -19,8 +19,8 @@ import org.apache.beam.runners.flink.adapter.FlinkKey; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/PartialReduceBundleOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/PartialReduceBundleOperator.java index 03570143231b..c8f36e234415 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/PartialReduceBundleOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/PartialReduceBundleOperator.java @@ -34,10 +34,11 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Sessions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Multimap; @@ -173,7 +174,7 @@ protected DoFn, KV> getDoFn() { @ProcessElement public void processElement(ProcessContext c, BoundedWindow window) throws Exception { WindowedValue> windowedValue = - WindowedValue.of(c.element(), c.timestamp(), window, c.pane()); + WindowedValues.of(c.element(), c.timestamp(), window, c.pane()); state.put(Objects.requireNonNull(c.element()).getKey(), windowedValue); } }; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SdfFlinkKeyKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SdfFlinkKeyKeySelector.java index b316726e74f8..1e9d27db21e9 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SdfFlinkKeyKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SdfFlinkKeyKeySelector.java @@ -19,8 +19,8 @@ import org.apache.beam.runners.flink.adapter.FlinkKey; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java index 6f2f473feddc..9284a7ae99a3 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItem.java @@ -20,7 +20,7 @@ import java.util.Collections; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** Singleton keyed word item. */ public class SingletonKeyedWorkItem implements KeyedWorkItem { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java index 747bd9f71894..424782650f3a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SingletonKeyedWorkItemCoder.java @@ -27,7 +27,8 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** Singleton keyed work item coder. */ @@ -46,14 +47,14 @@ public static SingletonKeyedWorkItemCoder of( private final Coder keyCoder; private final Coder elemCoder; private final Coder windowCoder; - private final WindowedValue.FullWindowedValueCoder valueCoder; + private final WindowedValues.FullWindowedValueCoder valueCoder; private SingletonKeyedWorkItemCoder( Coder keyCoder, Coder elemCoder, Coder windowCoder) { this.keyCoder = keyCoder; this.elemCoder = elemCoder; this.windowCoder = windowCoder; - valueCoder = WindowedValue.FullWindowedValueCoder.of(elemCoder, windowCoder); + valueCoder = WindowedValues.FullWindowedValueCoder.of(elemCoder, windowCoder); } public Coder getKeyCoder() { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java index d80dd60a5925..3f8758f610b1 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/SplittableDoFnOperator.java @@ -44,10 +44,11 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.flink.api.java.functions.KeySelector; @@ -152,7 +153,7 @@ public void outputWindowedValue( Collection windows, PaneInfo pane) { outputManager.output( - mainOutputTag, WindowedValue.of(output, timestamp, windows, pane)); + mainOutputTag, WindowedValues.of(output, timestamp, windows, pane)); } @Override @@ -162,7 +163,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane)); + outputManager.output(tag, WindowedValues.of(output, timestamp, windows, pane)); } }, sideInputReader, @@ -181,7 +182,7 @@ protected void fireTimer(TimerInternals.TimerData timer) { return; } doFnRunner.processElement( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( KeyedWorkItems.timersWorkItem( (byte[]) keyedStateInternals.getKey(), Collections.singletonList(timer)))); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java index 60b20f375f22..fad97e2d7dfd 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperator.java @@ -39,10 +39,11 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.java.functions.KeySelector; @@ -149,7 +150,7 @@ protected DoFn, KV> getDoFn() { protected void fireTimer(TimerData timer) { timerInternals.onFiredOrDeletedTimer(timer); doFnRunner.processElement( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( KeyedWorkItems.timersWorkItem( (K) keyedStateInternals.getKey(), Collections.singletonList(timer)))); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java index d809f4287983..36bdfd596eed 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WorkItemKeySelector.java @@ -21,7 +21,7 @@ import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.flink.adapter.FlinkKey; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.api.java.typeutils.ResultTypeQueryable; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java index 9d238aa36110..f45632a566cb 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/DedupingOperator.java @@ -21,8 +21,8 @@ import org.apache.beam.runners.flink.adapter.FlinkKey; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.state.ValueState; import org.apache.flink.api.common.state.ValueStateDescriptor; import org.apache.flink.api.common.typeutils.base.LongSerializer; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java index 871d7a5a3989..e8b6ee786152 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/StreamingImpulseSource.java @@ -18,7 +18,8 @@ package org.apache.beam.runners.flink.translation.wrappers.streaming.io; import java.nio.charset.StandardCharsets; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.flink.streaming.api.functions.source.RichParallelSourceFunction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -60,7 +61,7 @@ public void run(SourceContext> ctx) { while (running && (messageCount == 0 || count < subtaskCount)) { synchronized (ctx.getCheckpointLock()) { ctx.collect( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( String.valueOf(count).getBytes(StandardCharsets.UTF_8))); count++; } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestStreamSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestStreamSource.java index bacba9402459..52b81b8f9a43 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestStreamSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/TestStreamSource.java @@ -23,8 +23,9 @@ import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.flink.streaming.api.functions.source.RichSourceFunction; import org.apache.flink.streaming.api.watermark.Watermark; import org.joda.time.Instant; @@ -56,7 +57,7 @@ public void run(SourceContext> ctx) throws CoderException { for (TimestampedValue element : ((TestStream.ElementEvent) event).getElements()) { Instant timestamp = element.getTimestamp(); WindowedValue value = - WindowedValue.of( + WindowedValues.of( element.getValue(), timestamp, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); ctx.collectWithTimestamp(value, timestamp.getMillis()); } diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java index d6beafad8d93..7811f1f85a67 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapper.java @@ -36,11 +36,12 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.operators.ProcessingTimeService.ProcessingTimeCallback; @@ -341,7 +342,7 @@ private void emitElement( Instant timestamp = reader.getCurrentTimestamp(); WindowedValue> windowedValue = - WindowedValue.of( + WindowedValues.of( new ValueWithRecordId<>(item, recordId), timestamp, GlobalWindow.INSTANCE, diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java index 685bb870c5fd..0667a37789fc 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSource.java @@ -22,7 +22,7 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSource; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceSplit; import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java index 6b23dd13c9b8..05872accc36a 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReader.java @@ -32,7 +32,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.connector.source.ReaderOutput; @@ -125,7 +126,7 @@ public InputStatus pollNext(ReaderOutput> output) throws Except final @Nonnull Source.Reader splitReader = currentReader; T record = splitReader.getCurrent(); WindowedValue windowedValue = - WindowedValue.of( + WindowedValues.of( record, splitReader.getCurrentTimestamp(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); if (timestampExtractor == null) { diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java index 8ef2edfa606e..5d545fd53a1e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSource.java @@ -22,8 +22,8 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSource; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceSplit; import org.apache.beam.sdk.io.UnboundedSource; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.connector.source.Boundedness; import org.apache.flink.api.connector.source.SourceReader; import org.apache.flink.api.connector.source.SourceReaderContext; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java index 54c23bea3fb1..8837f3b9ec4e 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReader.java @@ -40,8 +40,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.connector.source.ReaderOutput; @@ -254,7 +255,7 @@ private void emitRecord( Instant timestamp = reader.getCurrentTimestamp(); WindowedValue> windowedValue = - WindowedValue.of( + WindowedValues.of( new ValueWithRecordId<>(item, recordId), timestamp, GlobalWindow.INSTANCE, diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElements.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElements.java index 91887dd04ed7..0c2ba87c4baa 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElements.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElements.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java index 312f8812ff1c..10a3182f90f4 100644 --- a/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java +++ b/runners/flink/src/main/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunner.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; 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.flink.api.common.state.ListState; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java index 5d08beb938fd..dc07caf5bfd7 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/FlinkPipelineOptionsTest.java @@ -36,8 +36,9 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.api.common.ExecutionMode; @@ -112,7 +113,7 @@ public void testDefaults() { @Test(expected = Exception.class) public void parDoBaseClassPipelineOptionsNullTest() { TupleTag mainTag = new TupleTag<>("main-output"); - Coder> coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + Coder> coder = WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); new DoFnOperator<>( new TestDoFn(), "stepName", @@ -138,7 +139,7 @@ public void parDoBaseClassPipelineOptionsSerializationTest() throws Exception { TupleTag mainTag = new TupleTag<>("main-output"); - Coder> coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + Coder> coder = WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); DoFnOperator doFnOperator = new DoFnOperator<>( new TestDoFn(), @@ -174,7 +175,7 @@ mainTag, coder, new SerializablePipelineOptions(FlinkPipelineOptions.defaults()) // execute once to access options testHarness.processElement( new StreamRecord<>( - WindowedValue.of( + WindowedValues.of( new Object(), Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); testHarness.close(); diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/BoundedSourceRestoreTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/BoundedSourceRestoreTest.java index a7183df67b44..c785d02fad1f 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/BoundedSourceRestoreTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/streaming/BoundedSourceRestoreTest.java @@ -30,10 +30,10 @@ import org.apache.beam.sdk.io.CountingSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter; import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource.BoundedToUnboundedSourceAdapter.Checkpoint; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.streaming.api.TimeCharacteristic; import org.apache.flink.streaming.api.operators.StreamSource; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java index 0435c70c7672..f4e7901f2f39 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/FlinkExecutableStageFunctionTest.java @@ -48,9 +48,10 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.Struct; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.flink.api.common.cache.DistributedCache; @@ -182,9 +183,9 @@ public void expectedInputsAreSent() throws Exception { FnDataReceiver> receiver = Mockito.mock(FnDataReceiver.class); when(bundle.getInputReceivers()).thenReturn(ImmutableMap.of("input", receiver)); - WindowedValue one = WindowedValue.valueInGlobalWindow(1); - WindowedValue two = WindowedValue.valueInGlobalWindow(2); - WindowedValue three = WindowedValue.valueInGlobalWindow(3); + WindowedValue one = WindowedValues.valueInGlobalWindow(1); + WindowedValue two = WindowedValues.valueInGlobalWindow(2); + WindowedValue three = WindowedValues.valueInGlobalWindow(3); function.mapPartition(Arrays.asList(one, two, three), collector); verify(receiver).accept(one); @@ -195,9 +196,9 @@ public void expectedInputsAreSent() throws Exception { @Test public void outputsAreTaggedCorrectly() throws Exception { - WindowedValue three = WindowedValue.valueInGlobalWindow(3); - WindowedValue four = WindowedValue.valueInGlobalWindow(4); - WindowedValue five = WindowedValue.valueInGlobalWindow(5); + WindowedValue three = WindowedValues.valueInGlobalWindow(3); + WindowedValue four = WindowedValues.valueInGlobalWindow(4); + WindowedValue five = WindowedValues.valueInGlobalWindow(5); Map outputTagMap = ImmutableMap.of( "one", 1, diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunctionTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunctionTest.java index e29cd8d6ad5c..554c4207b4a5 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunctionTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/functions/ImpulseSourceFunctionTest.java @@ -28,7 +28,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.state.ListState; import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.OperatorStateStore; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DedupingOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DedupingOperatorTest.java index 9576baa4c66b..4b0a38a60615 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DedupingOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DedupingOperatorTest.java @@ -25,8 +25,9 @@ import java.nio.charset.StandardCharsets; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.DedupingOperator; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.runtime.checkpoint.OperatorSubtaskState; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; @@ -53,22 +54,23 @@ public void testDeduping() throws Exception { harness.processElement( new StreamRecord<>( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( new ValueWithRecordId<>(key1, key1.getBytes(StandardCharsets.UTF_8))))); harness.processElement( new StreamRecord<>( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( new ValueWithRecordId<>(key2, key2.getBytes(StandardCharsets.UTF_8))))); harness.processElement( new StreamRecord<>( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( new ValueWithRecordId<>(key1, key1.getBytes(StandardCharsets.UTF_8))))); assertThat( stripStreamRecordFromWindowedValue(harness.getOutput()), - contains(WindowedValue.valueInGlobalWindow(key1), WindowedValue.valueInGlobalWindow(key2))); + contains( + WindowedValues.valueInGlobalWindow(key1), WindowedValues.valueInGlobalWindow(key2))); OperatorSubtaskState snapshot = harness.snapshot(0L, 0L); @@ -83,17 +85,17 @@ public void testDeduping() throws Exception { harness.processElement( new StreamRecord<>( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( new ValueWithRecordId<>(key2, key2.getBytes(StandardCharsets.UTF_8))))); harness.processElement( new StreamRecord<>( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( new ValueWithRecordId<>(key3, key3.getBytes(StandardCharsets.UTF_8))))); assertThat( stripStreamRecordFromWindowedValue(harness.getOutput()), - contains(WindowedValue.valueInGlobalWindow(key3))); + contains(WindowedValues.valueInGlobalWindow(key3))); harness.close(); } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java index f0d8816bdeab..c2556d7229ba 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/DoFnOperatorTest.java @@ -81,12 +81,13 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; 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.collect.FluentIterable; @@ -145,7 +146,7 @@ public void setUp() { @Test public void testSingleOutput() throws Exception { - Coder> coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + Coder> coder = WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); TupleTag outputTag = new TupleTag<>("main-output"); @@ -173,11 +174,11 @@ outputTag, coder, new SerializablePipelineOptions(FlinkPipelineOptions.defaults( testHarness.open(); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("Hello"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("Hello"))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), - contains(WindowedValue.valueInGlobalWindow("Hello"))); + contains(WindowedValues.valueInGlobalWindow("Hello"))); testHarness.close(); } @@ -186,8 +187,8 @@ outputTag, coder, new SerializablePipelineOptions(FlinkPipelineOptions.defaults( @SuppressWarnings("unchecked") public void testMultiOutputOutput() throws Exception { - WindowedValue.ValueOnlyWindowedValueCoder coder = - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + WindowedValues.ValueOnlyWindowedValueCoder coder = + WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); TupleTag mainOutput = new TupleTag<>("main-output"); TupleTag additionalOutput1 = new TupleTag<>("output-1"); @@ -239,25 +240,25 @@ public void testMultiOutputOutput() throws Exception { testHarness.open(); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("one"))); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("two"))); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("hello"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("one"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("two"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("hello"))); assertThat( this.stripStreamRecord(testHarness.getOutput()), - contains(WindowedValue.valueInGlobalWindow("got: hello"))); + contains(WindowedValues.valueInGlobalWindow("got: hello"))); assertThat( this.stripStreamRecord(testHarness.getSideOutput(tagsToOutputTags.get(additionalOutput1))), contains( - WindowedValue.valueInGlobalWindow("extra: one"), - WindowedValue.valueInGlobalWindow("got: hello"))); + WindowedValues.valueInGlobalWindow("extra: one"), + WindowedValues.valueInGlobalWindow("got: hello"))); assertThat( this.stripStreamRecord(testHarness.getSideOutput(tagsToOutputTags.get(additionalOutput2))), contains( - WindowedValue.valueInGlobalWindow("extra: two"), - WindowedValue.valueInGlobalWindow("got: hello"))); + WindowedValues.valueInGlobalWindow("extra: two"), + WindowedValues.valueInGlobalWindow("got: hello"))); testHarness.close(); } @@ -341,11 +342,11 @@ public void onProcessingTime(OnTimerContext context) { }; VarIntCoder keyCoder = VarIntCoder.of(); - WindowedValue.FullWindowedValueCoder inputCoder = - WindowedValue.getFullCoder(keyCoder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.FullWindowedValueCoder inputCoder = + WindowedValues.getFullCoder(keyCoder, windowingStrategy.getWindowFn().windowCoder()); - WindowedValue.FullWindowedValueCoder outputCoder = - WindowedValue.getFullCoder( + WindowedValues.FullWindowedValueCoder outputCoder = + WindowedValues.getFullCoder( StringUtf8Coder.of(), windowingStrategy.getWindowFn().windowCoder()); KeySelector, FlinkKey> keySelector = @@ -391,7 +392,7 @@ public void onProcessingTime(OnTimerContext context) { // this should register the two timers above testHarness.processElement( - new StreamRecord<>(WindowedValue.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>(WindowedValues.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); assertThat(stripStreamRecordFromWindowedValue(testHarness.getOutput()), emptyIterable()); @@ -415,14 +416,15 @@ public void onProcessingTime(OnTimerContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), containsInAnyOrder( - WindowedValue.of( + WindowedValues.of( eventTimeMessage + eventTimerId, timerTimestamp, window1, PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( eventTimeMessage + eventTimerId2, timerTimestamp.minus(Duration.millis(1)), window1, PaneInfo.NO_FIRING), - WindowedValue.of(processingTimeMessage, new Instant(10), window1, PaneInfo.NO_FIRING))); + WindowedValues.of( + processingTimeMessage, new Instant(10), window1, PaneInfo.NO_FIRING))); testHarness.close(); } @@ -431,7 +433,7 @@ public void onProcessingTime(OnTimerContext context) { public void testWatermarkUpdateAfterWatermarkHoldRelease() throws Exception { Coder>> coder = - WindowedValue.getValueOnlyCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())); + WindowedValues.getValueOnlyCoder(KvCoder.of(StringUtf8Coder.of(), StringUtf8Coder.of())); TupleTag> outputTag = new TupleTag<>("main-output"); List emittedWatermarkHolds = new ArrayList<>(); @@ -553,7 +555,7 @@ void emitWatermarkIfHoldChanged(long currentWatermarkHold) { // process first element, set hold to `now', setup timer for `now + 1' testHarness.processElement( new StreamRecord<>( - WindowedValue.timestampedValueInGlobalWindow(KV.of("Key", "Hello"), now))); + WindowedValues.timestampedValueInGlobalWindow(KV.of("Key", "Hello"), now))); assertThat(emittedWatermarkHolds, is(equalTo(Collections.singletonList(now.getMillis())))); @@ -566,7 +568,7 @@ void emitWatermarkIfHoldChanged(long currentWatermarkHold) { // process second element, verify we emitted changed hold testHarness.processElement( new StreamRecord<>( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("Key", "Hello"), now.plus(Duration.millis(2))))); assertThat( @@ -601,9 +603,9 @@ public void processElement(ProcessContext context) { VarIntCoder keyCoder = VarIntCoder.of(); Coder> inputCoder = - WindowedValue.getFullCoder(keyCoder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.getFullCoder(keyCoder, windowingStrategy.getWindowFn().windowCoder()); Coder> outputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( StringUtf8Coder.of(), windowingStrategy.getWindowFn().windowCoder()); KeySelector, FlinkKey> keySelector = @@ -644,11 +646,11 @@ public void processElement(ProcessContext context) { // this should not be late testHarness.processElement( - new StreamRecord<>(WindowedValue.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>(WindowedValues.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), - contains(WindowedValue.of("13", new Instant(0), window1, PaneInfo.NO_FIRING))); + contains(WindowedValues.of("13", new Instant(0), window1, PaneInfo.NO_FIRING))); testHarness.getOutput().clear(); @@ -656,11 +658,11 @@ public void processElement(ProcessContext context) { // this should still not be considered late testHarness.processElement( - new StreamRecord<>(WindowedValue.of(17, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>(WindowedValues.of(17, new Instant(0), window1, PaneInfo.NO_FIRING))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), - contains(WindowedValue.of("17", new Instant(0), window1, PaneInfo.NO_FIRING))); + contains(WindowedValues.of("17", new Instant(0), window1, PaneInfo.NO_FIRING))); testHarness.getOutput().clear(); @@ -668,7 +670,7 @@ public void processElement(ProcessContext context) { // this should now be considered late testHarness.processElement( - new StreamRecord<>(WindowedValue.of(17, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>(WindowedValues.of(17, new Instant(0), window1, PaneInfo.NO_FIRING))); assertThat(stripStreamRecordFromWindowedValue(testHarness.getOutput()), emptyIterable()); @@ -703,18 +705,18 @@ public void testStateGCForStatefulFn() throws Exception { testHarness.processElement( new StreamRecord<>( - WindowedValue.of(KV.of("key1", 5), new Instant(1), window1, PaneInfo.NO_FIRING))); + WindowedValues.of(KV.of("key1", 5), new Instant(1), window1, PaneInfo.NO_FIRING))); testHarness.processElement( new StreamRecord<>( - WindowedValue.of(KV.of("key2", 7), new Instant(3), window1, PaneInfo.NO_FIRING))); + WindowedValues.of(KV.of("key2", 7), new Instant(3), window1, PaneInfo.NO_FIRING))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.of( + WindowedValues.of( KV.of("key1", 5 + offset), new Instant(1), window1, PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( KV.of("key2", 7 + offset), new Instant(3), window1, PaneInfo.NO_FIRING))); // 2 entries for the elements and 2 for the pending timers @@ -737,9 +739,9 @@ public void testStateGCForStatefulFn() throws Exception { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.of( + WindowedValues.of( KV.of("key1", timerOutput), new Instant(9), window1, PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( KV.of("key2", timerOutput), new Instant(9), window1, PaneInfo.NO_FIRING))); testHarness.close(); @@ -763,11 +765,11 @@ public void testGCForGlobalWindow() throws Exception { // Check global window cleanup via final watermark, _without_ cleanup timers testHarness.processElement( new StreamRecord<>( - WindowedValue.of( + WindowedValues.of( KV.of("key1", 5), new Instant(23), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); testHarness.processElement( new StreamRecord<>( - WindowedValue.of( + WindowedValues.of( KV.of("key2", 6), new Instant(42), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); // timers set by the transform @@ -800,7 +802,7 @@ public void testGCForGlobalWindow() throws Exception { // Any new state will also be cleaned up on close testHarness.processElement( new StreamRecord<>( - WindowedValue.of( + WindowedValues.of( KV.of("key2", 6), new Instant(42), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING))); // Close sends Flink's max watermark and will cleanup again @@ -848,8 +850,8 @@ public void onTimer(OnTimerContext context, @StateId(stateId) ValueState } }; - WindowedValue.FullWindowedValueCoder> coder = - WindowedValue.getFullCoder( + WindowedValues.FullWindowedValueCoder> coder = + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()), windowingStrategy.getWindowFn().windowCoder()); @@ -896,7 +898,7 @@ public void testKeyedParDoSideInputs() throws Exception { void testSideInputs(boolean keyed) throws Exception { - Coder> coder = WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + Coder> coder = WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); TupleTag outputTag = new TupleTag<>("main-output"); @@ -1029,7 +1031,7 @@ public void processElement( KvCoder coder = KvCoder.of(keyCoder, VarLongCoder.of()); FullWindowedValueCoder> kvCoder = - WindowedValue.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); TypeInformation keyCoderInfo = ValueTypeInfo.of(FlinkKey.class); @@ -1048,9 +1050,9 @@ public void processElement( testHarness.open(); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("a", 100L)))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("a", 100L)))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("a", 100L)))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("a", 100L)))); OperatorSubtaskState snapshot = testHarness.snapshot(0, 0); testHarness.close(); @@ -1070,19 +1072,19 @@ public void processElement( // after restore: counter = 2 testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("a", 100L)))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("a", 100L)))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("a", 4L)))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("a", 4L)))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("a", 5L)))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("a", 5L)))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("a", 100L)))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("a", 100L)))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow(KV.of("a", 4L)), - WindowedValue.valueInGlobalWindow(KV.of("a", 5L)))); + WindowedValues.valueInGlobalWindow(KV.of("a", 4L)), + WindowedValues.valueInGlobalWindow(KV.of("a", 5L)))); testHarness.close(); } @@ -1092,7 +1094,7 @@ public void nonKeyedParDoSideInputCheckpointing() throws Exception { sideInputCheckpointing( () -> { Coder> coder = - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); TupleTag outputTag = new TupleTag<>("main-output"); ImmutableMap> sideInputMapping = @@ -1132,7 +1134,7 @@ public void keyedParDoSideInputCheckpointing() throws Exception { () -> { StringUtf8Coder keyCoder = StringUtf8Coder.of(); Coder> coder = - WindowedValue.getFullCoder(keyCoder, IntervalWindow.getCoder()); + WindowedValues.getFullCoder(keyCoder, IntervalWindow.getCoder()); TupleTag outputTag = new TupleTag<>("main-output"); KeySelector, FlinkKey> keySelector = @@ -1236,7 +1238,7 @@ public void nonKeyedParDoPushbackDataCheckpointing() throws Exception { pushbackDataCheckpointing( () -> { Coder> coder = - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); TupleTag outputTag = new TupleTag<>("main-output"); @@ -1277,7 +1279,7 @@ public void keyedParDoPushbackDataCheckpointing() throws Exception { () -> { StringUtf8Coder keyCoder = StringUtf8Coder.of(); Coder> coder = - WindowedValue.getFullCoder(keyCoder, IntervalWindow.getCoder()); + WindowedValues.getFullCoder(keyCoder, IntervalWindow.getCoder()); TupleTag outputTag = new TupleTag<>("main-output"); @@ -1406,11 +1408,11 @@ public void onEventTime(OnTimerContext context) { }; VarIntCoder keyCoder = VarIntCoder.of(); - WindowedValue.FullWindowedValueCoder inputCoder = - WindowedValue.getFullCoder(keyCoder, windowingStrategy.getWindowFn().windowCoder()); + WindowedValues.FullWindowedValueCoder inputCoder = + WindowedValues.getFullCoder(keyCoder, windowingStrategy.getWindowFn().windowCoder()); - WindowedValue.FullWindowedValueCoder outputCoder = - WindowedValue.getFullCoder( + WindowedValues.FullWindowedValueCoder outputCoder = + WindowedValues.getFullCoder( StringUtf8Coder.of(), windowingStrategy.getWindowFn().windowCoder()); TupleTag outputTag = new TupleTag<>("main-output"); @@ -1444,7 +1446,7 @@ public void onEventTime(OnTimerContext context) { // this should register a timer testHarness.processElement( - new StreamRecord<>(WindowedValue.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); + new StreamRecord<>(WindowedValues.of(13, new Instant(0), window1, PaneInfo.NO_FIRING))); assertThat(stripStreamRecordFromWindowedValue(testHarness.getOutput()), emptyIterable()); @@ -1471,7 +1473,7 @@ public void onEventTime(OnTimerContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), - contains(WindowedValue.of(outputMessage, timerTimestamp, window1, PaneInfo.NO_FIRING))); + contains(WindowedValues.of(outputMessage, timerTimestamp, window1, PaneInfo.NO_FIRING))); testHarness.close(); } @@ -1514,8 +1516,8 @@ OneInputStreamOperatorTestHarness, WindowedValue> creat @Test public void testBundle() throws Exception { - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + WindowedValues.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); TupleTag outputTag = new TupleTag<>("main-output"); FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); @@ -1535,7 +1537,7 @@ public void finishBundle(FinishBundleContext context) { DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); DoFnOperator doFnOperator = @@ -1561,17 +1563,17 @@ public void finishBundle(FinishBundleContext context) { testHarness.open(); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("a"))); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("b"))); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("c"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("a"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("b"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("c"))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("a"), - WindowedValue.valueInGlobalWindow("b"), - WindowedValue.valueInGlobalWindow("finishBundle"), - WindowedValue.valueInGlobalWindow("c"))); + WindowedValues.valueInGlobalWindow("a"), + WindowedValues.valueInGlobalWindow("b"), + WindowedValues.valueInGlobalWindow("finishBundle"), + WindowedValues.valueInGlobalWindow("c"))); // draw a snapshot OperatorSubtaskState snapshot = testHarness.snapshot(0, 0); @@ -1583,7 +1585,7 @@ public void finishBundle(FinishBundleContext context) { List>> bufferedElements = pushedBackElementsHandler.getElements().collect(Collectors.toList()); assertThat( - bufferedElements, contains(KV.of(0, WindowedValue.valueInGlobalWindow("finishBundle")))); + bufferedElements, contains(KV.of(0, WindowedValues.valueInGlobalWindow("finishBundle")))); testHarness.close(); @@ -1614,7 +1616,7 @@ public void finishBundle(FinishBundleContext context) { newHarness.open(); // startBundle will output the buffered elements. - newHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("d"))); + newHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("d"))); // check finishBundle by timeout newHarness.setProcessingTime(10); @@ -1622,9 +1624,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(newHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("finishBundle"), - WindowedValue.valueInGlobalWindow("d"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("finishBundle"), + WindowedValues.valueInGlobalWindow("d"), + WindowedValues.valueInGlobalWindow("finishBundle"))); // No bundle will be created when sending the MAX watermark // (unless pushed back items are emitted) @@ -1633,9 +1635,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(newHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("finishBundle"), - WindowedValue.valueInGlobalWindow("d"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("finishBundle"), + WindowedValues.valueInGlobalWindow("d"), + WindowedValues.valueInGlobalWindow("finishBundle"))); // close() will also call dispose(), but call again to verify no new bundle // is created afterwards @@ -1644,9 +1646,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(newHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("finishBundle"), - WindowedValue.valueInGlobalWindow("d"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("finishBundle"), + WindowedValues.valueInGlobalWindow("d"), + WindowedValues.valueInGlobalWindow("finishBundle"))); } @Test @@ -1655,8 +1657,8 @@ public void testBundleKeyed() throws Exception { StringUtf8Coder keyCoder = StringUtf8Coder.of(); KvToFlinkKeyKeySelector keySelector = new KvToFlinkKeyKeySelector<>(keyCoder); KvCoder kvCoder = KvCoder.of(keyCoder, StringUtf8Coder.of()); - WindowedValue.ValueOnlyWindowedValueCoder> windowedValueCoder = - WindowedValue.getValueOnlyCoder(kvCoder); + WindowedValues.ValueOnlyWindowedValueCoder> windowedValueCoder = + WindowedValues.getValueOnlyCoder(kvCoder); TupleTag outputTag = new TupleTag<>("main-output"); FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); @@ -1682,7 +1684,7 @@ public void finishBundle(FinishBundleContext context) { DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(kvCoder.getValueCoder(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(kvCoder.getValueCoder(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); DoFnOperator, KV, String> doFnOperator = @@ -1711,19 +1713,19 @@ public void finishBundle(FinishBundleContext context) { testHarness.open(); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key", "a")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key", "a")))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key", "b")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key", "b")))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key", "c")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key", "c")))); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("a"), - WindowedValue.valueInGlobalWindow("b"), - WindowedValue.valueInGlobalWindow("finishBundle"), - WindowedValue.valueInGlobalWindow("c"))); + WindowedValues.valueInGlobalWindow("a"), + WindowedValues.valueInGlobalWindow("b"), + WindowedValues.valueInGlobalWindow("finishBundle"), + WindowedValues.valueInGlobalWindow("c"))); // Take a snapshot OperatorSubtaskState snapshot = testHarness.snapshot(0, 0); @@ -1735,7 +1737,7 @@ public void finishBundle(FinishBundleContext context) { List>> bufferedElements = pushedBackElementsHandler.getElements().collect(Collectors.toList()); assertThat( - bufferedElements, contains(KV.of(0, WindowedValue.valueInGlobalWindow("finishBundle")))); + bufferedElements, contains(KV.of(0, WindowedValues.valueInGlobalWindow("finishBundle")))); testHarness.close(); @@ -1768,7 +1770,7 @@ public void finishBundle(FinishBundleContext context) { // startBundle will output the buffered elements. testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key", "d")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key", "d")))); // check finishBundle by timeout testHarness.setProcessingTime(10); @@ -1777,9 +1779,9 @@ public void finishBundle(FinishBundleContext context) { stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( // The first finishBundle is restored from the checkpoint - WindowedValue.valueInGlobalWindow("finishBundle"), - WindowedValue.valueInGlobalWindow("d"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("finishBundle"), + WindowedValues.valueInGlobalWindow("d"), + WindowedValues.valueInGlobalWindow("finishBundle"))); testHarness.close(); } @@ -1794,13 +1796,13 @@ public void testCheckpointBufferingWithMultipleBundles() throws Exception { TupleTag outputTag = new TupleTag<>("main-output"); StringUtf8Coder coder = StringUtf8Coder.of(); - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(coder); + WindowedValues.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValues.getValueOnlyCoder(coder); DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); Supplier> doFnOperatorSupplier = @@ -1830,7 +1832,7 @@ public void testCheckpointBufferingWithMultipleBundles() throws Exception { // start a bundle testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow("regular element"))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow("regular element"))); // This callback will be executed in the snapshotState function in the course of // finishing the currently active bundle. Everything emitted in the callback should @@ -1842,11 +1844,11 @@ public void testCheckpointBufferingWithMultipleBundles() throws Exception { // the callback which would otherwise cause an infinitive recursion doFnOperator.setBundleFinishedCallback(null); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow("trigger another bundle"))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow("trigger another bundle"))); doFnOperator.invokeFinishBundle(); testHarness.processElement( new StreamRecord<>( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( "check that the previous element is not flushed"))); } catch (Exception e) { throw new RuntimeException(e); @@ -1858,16 +1860,16 @@ public void testCheckpointBufferingWithMultipleBundles() throws Exception { // Check that we have only the element which was emitted before the snapshot assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), - contains(WindowedValue.valueInGlobalWindow("regular element"))); + contains(WindowedValues.valueInGlobalWindow("regular element"))); // Check that we would flush the buffered elements when continuing to run testHarness.processWatermark(Long.MAX_VALUE); assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("regular element"), - WindowedValue.valueInGlobalWindow("trigger another bundle"), - WindowedValue.valueInGlobalWindow("check that the previous element is not flushed"))); + WindowedValues.valueInGlobalWindow("regular element"), + WindowedValues.valueInGlobalWindow("trigger another bundle"), + WindowedValues.valueInGlobalWindow("check that the previous element is not flushed"))); testHarness.close(); @@ -1879,14 +1881,14 @@ public void testCheckpointBufferingWithMultipleBundles() throws Exception { testHarness2.open(); testHarness2.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow("after restore"))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow("after restore"))); assertThat( stripStreamRecordFromWindowedValue(testHarness2.getOutput()), contains( - WindowedValue.valueInGlobalWindow("trigger another bundle"), - WindowedValue.valueInGlobalWindow("check that the previous element is not flushed"), - WindowedValue.valueInGlobalWindow("after restore"))); + WindowedValues.valueInGlobalWindow("trigger another bundle"), + WindowedValues.valueInGlobalWindow("check that the previous element is not flushed"), + WindowedValues.valueInGlobalWindow("after restore"))); } @Test @@ -1896,8 +1898,8 @@ public void testExactlyOnceBuffering() throws Exception { options.setCheckpointingInterval(1L); TupleTag outputTag = new TupleTag<>("main-output"); - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + WindowedValues.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); numStartBundleCalled = 0; DoFn doFn = @@ -1924,7 +1926,7 @@ public void finishBundle(FinishBundleContext context) { DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); Supplier> doFnOperatorSupplier = @@ -1952,8 +1954,8 @@ public void finishBundle(FinishBundleContext context) { testHarness.open(); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("a"))); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("b"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("a"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("b"))); assertThat(Iterables.size(testHarness.getOutput()), is(0)); assertThat(numStartBundleCalled, is(0)); @@ -1966,9 +1968,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("a"), - WindowedValue.valueInGlobalWindow("b"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("a"), + WindowedValues.valueInGlobalWindow("b"), + WindowedValues.valueInGlobalWindow("finishBundle"))); doFnOperator = doFnOperatorSupplier.get(); testHarness = new OneInputStreamOperatorTestHarness<>(doFnOperator); @@ -1983,9 +1985,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("a"), - WindowedValue.valueInGlobalWindow("b"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("a"), + WindowedValues.valueInGlobalWindow("b"), + WindowedValues.valueInGlobalWindow("finishBundle"))); // repeat to see if elements are evicted doFnOperator.notifyCheckpointComplete(1L); @@ -1994,9 +1996,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("a"), - WindowedValue.valueInGlobalWindow("b"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("a"), + WindowedValues.valueInGlobalWindow("b"), + WindowedValues.valueInGlobalWindow("finishBundle"))); } @Test @@ -2007,8 +2009,8 @@ public void testExactlyOnceBufferingFlushDuringDrain() throws Exception { options.setEnableStableInputDrain(true); TupleTag outputTag = new TupleTag<>("main-output"); - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + WindowedValues.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); numStartBundleCalled = 0; DoFn doFn = @@ -2035,7 +2037,7 @@ public void finishBundle(FinishBundleContext context) { DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); Supplier> doFnOperatorSupplier = @@ -2063,8 +2065,8 @@ public void finishBundle(FinishBundleContext context) { testHarness.open(); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("a"))); - testHarness.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow("b"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("a"))); + testHarness.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow("b"))); assertThat(Iterables.size(testHarness.getOutput()), is(0)); assertThat(numStartBundleCalled, is(0)); @@ -2077,9 +2079,9 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), contains( - WindowedValue.valueInGlobalWindow("a"), - WindowedValue.valueInGlobalWindow("b"), - WindowedValue.valueInGlobalWindow("finishBundle"))); + WindowedValues.valueInGlobalWindow("a"), + WindowedValues.valueInGlobalWindow("b"), + WindowedValues.valueInGlobalWindow("finishBundle"))); doFnOperator = doFnOperatorSupplier.get(); testHarness = new OneInputStreamOperatorTestHarness<>(doFnOperator); @@ -2102,8 +2104,8 @@ public void testExactlyOnceBufferingKeyed() throws Exception { StringUtf8Coder keyCoder = StringUtf8Coder.of(); KvToFlinkKeyKeySelector keySelector = new KvToFlinkKeyKeySelector<>(keyCoder); KvCoder kvCoder = KvCoder.of(keyCoder, StringUtf8Coder.of()); - WindowedValue.ValueOnlyWindowedValueCoder> windowedValueCoder = - WindowedValue.getValueOnlyCoder(kvCoder); + WindowedValues.ValueOnlyWindowedValueCoder> windowedValueCoder = + WindowedValues.getValueOnlyCoder(kvCoder); DoFn, KV> doFn = new DoFn, KV>() { @@ -2131,7 +2133,7 @@ public void finishBundle(FinishBundleContext context) { DoFnOperator.MultiOutputOutputManagerFactory> outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(kvCoder, GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(kvCoder, GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); Supplier, KV, KV>> @@ -2165,13 +2167,13 @@ public void finishBundle(FinishBundleContext context) { testHarness.open(); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key", "a")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key", "a")))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key", "b")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key", "b")))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key2", "c")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key2", "c")))); testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow(KV.of("key2", "d")))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow(KV.of("key2", "d")))); assertThat( testHarness.getOutput() + " should be empty", @@ -2185,11 +2187,11 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), containsInAnyOrder( - WindowedValue.valueInGlobalWindow(KV.of("key", "a")), - WindowedValue.valueInGlobalWindow(KV.of("key", "b")), - WindowedValue.valueInGlobalWindow(KV.of("key2", "c")), - WindowedValue.valueInGlobalWindow(KV.of("key2", "d")), - WindowedValue.valueInGlobalWindow(KV.of("key3", "finishBundle")))); + WindowedValues.valueInGlobalWindow(KV.of("key", "a")), + WindowedValues.valueInGlobalWindow(KV.of("key", "b")), + WindowedValues.valueInGlobalWindow(KV.of("key2", "c")), + WindowedValues.valueInGlobalWindow(KV.of("key2", "d")), + WindowedValues.valueInGlobalWindow(KV.of("key3", "finishBundle")))); doFnOperator = doFnOperatorSupplier.get(); testHarness = @@ -2206,11 +2208,11 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), containsInAnyOrder( - WindowedValue.valueInGlobalWindow(KV.of("key", "a")), - WindowedValue.valueInGlobalWindow(KV.of("key", "b")), - WindowedValue.valueInGlobalWindow(KV.of("key2", "c")), - WindowedValue.valueInGlobalWindow(KV.of("key2", "d")), - WindowedValue.valueInGlobalWindow(KV.of("key3", "finishBundle")))); + WindowedValues.valueInGlobalWindow(KV.of("key", "a")), + WindowedValues.valueInGlobalWindow(KV.of("key", "b")), + WindowedValues.valueInGlobalWindow(KV.of("key2", "c")), + WindowedValues.valueInGlobalWindow(KV.of("key2", "d")), + WindowedValues.valueInGlobalWindow(KV.of("key3", "finishBundle")))); // repeat to see if elements are evicted doFnOperator.notifyCheckpointComplete(1L); @@ -2219,11 +2221,11 @@ public void finishBundle(FinishBundleContext context) { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), containsInAnyOrder( - WindowedValue.valueInGlobalWindow(KV.of("key", "a")), - WindowedValue.valueInGlobalWindow(KV.of("key", "b")), - WindowedValue.valueInGlobalWindow(KV.of("key2", "c")), - WindowedValue.valueInGlobalWindow(KV.of("key2", "d")), - WindowedValue.valueInGlobalWindow(KV.of("key3", "finishBundle")))); + WindowedValues.valueInGlobalWindow(KV.of("key", "a")), + WindowedValues.valueInGlobalWindow(KV.of("key", "b")), + WindowedValues.valueInGlobalWindow(KV.of("key2", "c")), + WindowedValues.valueInGlobalWindow(KV.of("key2", "d")), + WindowedValues.valueInGlobalWindow(KV.of("key3", "finishBundle")))); } @Test(expected = IllegalStateException.class) @@ -2233,8 +2235,8 @@ public void testFailOnRequiresStableInputAndDisabledCheckpointing() { StringUtf8Coder keyCoder = StringUtf8Coder.of(); KvToFlinkKeyKeySelector keySelector = new KvToFlinkKeyKeySelector<>(keyCoder); KvCoder kvCoder = KvCoder.of(keyCoder, StringUtf8Coder.of()); - WindowedValue.ValueOnlyWindowedValueCoder> windowedValueCoder = - WindowedValue.getValueOnlyCoder(kvCoder); + WindowedValues.ValueOnlyWindowedValueCoder> windowedValueCoder = + WindowedValues.getValueOnlyCoder(kvCoder); DoFn, KV> doFn = new DoFn, KV>() { @@ -2250,7 +2252,7 @@ public void processElement(ProcessContext context) { DoFnOperator.MultiOutputOutputManagerFactory> outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(kvCoder, GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(kvCoder, GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); // should make the DoFnOperator creation fail @@ -2282,13 +2284,13 @@ public void testBundleProcessingExceptionIsFatalDuringCheckpointing() throws Exc TupleTag outputTag = new TupleTag<>("main-output"); StringUtf8Coder coder = StringUtf8Coder.of(); - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(coder); + WindowedValues.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValues.getValueOnlyCoder(coder); DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); DoFnOperator doFnOperator = @@ -2321,7 +2323,7 @@ public void finishBundle() { // start a bundle testHarness.processElement( - new StreamRecord<>(WindowedValue.valueInGlobalWindow("regular element"))); + new StreamRecord<>(WindowedValues.valueInGlobalWindow("regular element"))); // Make sure we throw Error, not a regular Exception. // A regular exception would just cause the checkpoint to fail. @@ -2371,8 +2373,8 @@ private static DoFnOperator getOperatorForCleanupInspect options.setParallelism(4); TupleTag outputTag = new TupleTag<>("main-output"); - WindowedValue.ValueOnlyWindowedValueCoder windowedValueCoder = - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()); + WindowedValues.ValueOnlyWindowedValueCoder windowedValueCoder = + WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()); IdentityDoFn doFn = new IdentityDoFn() { @FinishBundle @@ -2385,7 +2387,7 @@ public void finishBundle(FinishBundleContext context) { DoFnOperator.MultiOutputOutputManagerFactory outputManagerFactory = new DoFnOperator.MultiOutputOutputManagerFactory<>( outputTag, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), new SerializablePipelineOptions(options)); return new DoFnOperator<>( @@ -2455,11 +2457,11 @@ public void processElement(ProcessContext c) { private WindowedValue> valuesInWindow( Iterable values, Instant timestamp, BoundedWindow window) { - return WindowedValue.of(values, timestamp, window, PaneInfo.NO_FIRING); + return WindowedValues.of(values, timestamp, window, PaneInfo.NO_FIRING); } private WindowedValue valueInWindow(T value, Instant timestamp, BoundedWindow window) { - return WindowedValue.of(value, timestamp, window, PaneInfo.NO_FIRING); + return WindowedValues.of(value, timestamp, window, PaneInfo.NO_FIRING); } private interface TestHarnessFactory { diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java index f9a2f094f602..982871e59f14 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/ExecutableStageDoFnOperatorTest.java @@ -97,10 +97,11 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.NoopLock; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.Struct; @@ -249,7 +250,7 @@ public void sdkErrorsSurfaceOnClose() throws Exception { doThrow(expected).when(bundle).close(); thrown.expectCause(is(expected)); - operator.processElement(new StreamRecord<>(WindowedValue.valueInGlobalWindow(0))); + operator.processElement(new StreamRecord<>(WindowedValues.valueInGlobalWindow(0))); testHarness.close(); } @@ -272,9 +273,9 @@ public void expectedInputsAreSent() throws Exception { FnDataReceiver> receiver = Mockito.mock(FnDataReceiver.class); when(bundle.getInputReceivers()).thenReturn(ImmutableMap.of("input", receiver)); - WindowedValue one = WindowedValue.valueInGlobalWindow(1); - WindowedValue two = WindowedValue.valueInGlobalWindow(2); - WindowedValue three = WindowedValue.valueInGlobalWindow(3); + WindowedValue one = WindowedValues.valueInGlobalWindow(1); + WindowedValue two = WindowedValues.valueInGlobalWindow(2); + WindowedValue three = WindowedValues.valueInGlobalWindow(3); OneInputStreamOperatorTestHarness, WindowedValue> testHarness = new OneInputStreamOperatorTestHarness<>(operator); @@ -296,8 +297,8 @@ public void expectedInputsAreSent() throws Exception { @Test public void outputsAreTaggedCorrectly() throws Exception { - WindowedValue.ValueOnlyWindowedValueCoder coder = - WindowedValue.getValueOnlyCoder(VarIntCoder.of()); + WindowedValues.ValueOnlyWindowedValueCoder coder = + WindowedValues.getValueOnlyCoder(VarIntCoder.of()); TupleTag mainOutput = new TupleTag<>("main-output"); TupleTag additionalOutput1 = new TupleTag<>("output-1"); @@ -332,10 +333,10 @@ public void outputsAreTaggedCorrectly() throws Exception { tagsToIds, new SerializablePipelineOptions(FlinkPipelineOptions.defaults())); - WindowedValue zero = WindowedValue.valueInGlobalWindow(0); - WindowedValue three = WindowedValue.valueInGlobalWindow(3); - WindowedValue four = WindowedValue.valueInGlobalWindow(4); - WindowedValue five = WindowedValue.valueInGlobalWindow(5); + WindowedValue zero = WindowedValues.valueInGlobalWindow(0); + WindowedValue three = WindowedValues.valueInGlobalWindow(3); + WindowedValue four = WindowedValues.valueInGlobalWindow(4); + WindowedValue five = WindowedValues.valueInGlobalWindow(5); // We use a real StageBundleFactory here in order to exercise the output receiver factory. StageBundleFactory stageBundleFactory = @@ -462,7 +463,7 @@ public void testWatermarkHandling() throws Exception { outputManagerFactory, WindowingStrategy.of(FixedWindows.of(Duration.millis(10))), StringUtf8Coder.of(), - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()), IntervalWindow.getCoder())); KeyedOneInputStreamOperatorTestHarness< @@ -498,7 +499,7 @@ public void testWatermarkHandling() throws Exception { // Trigger a new bundle IntervalWindow intervalWindow = new IntervalWindow(new Instant(0), new Instant(9)); WindowedValue> windowedValue = - WindowedValue.of(KV.of("one", 1), Instant.now(), intervalWindow, PaneInfo.NO_FIRING); + WindowedValues.of(KV.of("one", 1), Instant.now(), intervalWindow, PaneInfo.NO_FIRING); testHarness.processElement(new StreamRecord<>(windowedValue)); // The output watermark should be held back during the bundle @@ -636,7 +637,7 @@ public void testEnsureStateCleanupWithKeyedInput() throws Exception { outputManagerFactory, WindowingStrategy.globalDefault(), keyCoder, - WindowedValue.getFullCoder(keyCoder, GlobalWindow.Coder.INSTANCE)); + WindowedValues.getFullCoder(keyCoder, GlobalWindow.Coder.INSTANCE)); KeyedOneInputStreamOperatorTestHarness, WindowedValue> testHarness = @@ -760,7 +761,7 @@ private void testEnsureDeferredStateCleanupTimerFiring(boolean withCheckpointing outputManagerFactory, windowingStrategy, keyCoder, - WindowedValue.getFullCoder(kvCoder, windowingStrategy.getWindowFn().windowCoder())); + WindowedValues.getFullCoder(kvCoder, windowingStrategy.getWindowFn().windowCoder())); @SuppressWarnings("unchecked") RemoteBundle bundle = Mockito.mock(RemoteBundle.class); @@ -771,7 +772,7 @@ private void testEnsureDeferredStateCleanupTimerFiring(boolean withCheckpointing IntervalWindow window = new IntervalWindow(new Instant(0), new Instant(1000)); IntervalWindow.IntervalWindowCoder windowCoder = IntervalWindow.IntervalWindowCoder.of(); WindowedValue> windowedValue = - WindowedValue.of( + WindowedValues.of( KV.of("one", 1), window.maxTimestamp(), ImmutableList.of(window), PaneInfo.NO_FIRING); FnDataReceiver receiver = Mockito.mock(FnDataReceiver.class); @@ -933,7 +934,7 @@ public void testEnsureStateCleanupOnFinalWatermark() throws Exception { outputManagerFactory, windowingStrategy, keyCoder, - WindowedValue.getFullCoder(kvCoder, windowCoder)); + WindowedValues.getFullCoder(kvCoder, windowCoder)); KeyedOneInputStreamOperatorTestHarness< FlinkKey, WindowedValue>, WindowedValue> @@ -1094,8 +1095,8 @@ private static ProcessBundleDescriptors.BagUserStateSpec mockBagUserState(String @Test public void testSerialization() { - WindowedValue.ValueOnlyWindowedValueCoder coder = - WindowedValue.getValueOnlyCoder(VarIntCoder.of()); + WindowedValues.ValueOnlyWindowedValueCoder coder = + WindowedValues.getValueOnlyCoder(VarIntCoder.of()); TupleTag mainOutput = new TupleTag<>("main-output"); TupleTag additionalOutput = new TupleTag<>("additional-output"); @@ -1129,7 +1130,7 @@ public void testSerialization() { ExecutableStageDoFnOperator operator = new ExecutableStageDoFnOperator<>( "transform", - WindowedValue.getValueOnlyCoder(VarIntCoder.of()), + WindowedValues.getValueOnlyCoder(VarIntCoder.of()), Collections.emptyMap(), mainOutput, ImmutableList.of(additionalOutput), @@ -1166,7 +1167,7 @@ public void testStableInputApplied() { outputManagerFactory, WindowingStrategy.globalDefault(), null, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE), stagePayloadWithStableInput, options); @@ -1189,7 +1190,7 @@ private ExecutableStageDoFnOperator getOperator( outputManagerFactory, WindowingStrategy.globalDefault(), null, - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE)); + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE)); } private ExecutableStageDoFnOperator getOperator( diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/StreamRecordStripper.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/StreamRecordStripper.java index 004dc83164c2..264761f260ee 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/StreamRecordStripper.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/StreamRecordStripper.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.flink.translation.wrappers.streaming; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; 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.collect.FluentIterable; import org.apache.flink.streaming.runtime.streamrecord.StreamRecord; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java index 6380108ddb94..3bee828f23dd 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/WindowDoFnOperatorTest.java @@ -46,10 +46,11 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.flink.api.java.functions.KeySelector; @@ -107,12 +108,12 @@ public void testRestore() throws Exception { assertThat( output, containsInAnyOrder( - WindowedValue.of( + WindowedValues.of( KV.of(1L, 120L), new Instant(9_999), window, PaneInfo.createPane(true, true, ON_TIME)), - WindowedValue.of( + WindowedValues.of( KV.of(2L, 77L), new Instant(9_999), window, @@ -184,9 +185,9 @@ public void testTimerCleanupOfPendingTimerList() throws Exception { assertThat( stripStreamRecordFromWindowedValue(testHarness.getOutput()), containsInAnyOrder( - WindowedValue.of( + WindowedValues.of( KV.of(1L, 100L), new Instant(99), window, PaneInfo.createPane(true, true, ON_TIME)), - WindowedValue.of( + WindowedValues.of( KV.of(2L, 150L), new Instant(199), window2, @@ -214,9 +215,9 @@ private WindowDoFnOperator getWindowDoFnOperator(boolean strea SingletonKeyedWorkItemCoder workItemCoder = SingletonKeyedWorkItemCoder.of(VarLongCoder.of(), VarLongCoder.of(), windowCoder); FullWindowedValueCoder> inputCoder = - WindowedValue.getFullCoder(workItemCoder, windowCoder); + WindowedValues.getFullCoder(workItemCoder, windowCoder); FullWindowedValueCoder> outputCoder = - WindowedValue.getFullCoder(KvCoder.of(VarLongCoder.of(), VarLongCoder.of()), windowCoder); + WindowedValues.getFullCoder(KvCoder.of(VarLongCoder.of(), VarLongCoder.of()), windowCoder); FlinkPipelineOptions options = FlinkPipelineOptions.defaults(); options.setStreaming(streaming); @@ -265,7 +266,7 @@ static ItemBuilder builder() { StreamRecord>> toStreamRecord() { WindowedValue> keyedItem = - WindowedValue.of(KV.of(key, value), new Instant(timestamp), window, NO_FIRING); + WindowedValues.of(KV.of(key, value), new Instant(timestamp), window, NO_FIRING); return new StreamRecord<>(keyedItem); } diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java index 8a588894acd3..f57198e08e3e 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/UnboundedSourceWrapperTest.java @@ -49,10 +49,10 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.UnboundedReadFromBoundedSource; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; import org.apache.flink.api.common.ExecutionConfig; import org.apache.flink.configuration.Configuration; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java index bf9ea348230a..88c972a70f65 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/bounded/FlinkBoundedSourceReaderTest.java @@ -33,8 +33,8 @@ import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.FlinkSourceSplit; import org.apache.beam.runners.flink.translation.wrappers.streaming.io.source.SourceTestMetrics.TestMetricGroup; import org.apache.beam.sdk.io.Source; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceReader; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java index 26dd753df0e1..5bb31a910f82 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/io/source/unbounded/FlinkUnboundedSourceReaderTest.java @@ -41,9 +41,9 @@ import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.flink.api.common.eventtime.Watermark; import org.apache.flink.api.connector.source.ReaderOutput; import org.apache.flink.api.connector.source.SourceOutput; diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java index 5652d23662fd..8e8db7496b80 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferedElementsTest.java @@ -27,8 +27,8 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.hamcrest.Matchers; import org.joda.time.Instant; @@ -46,14 +46,14 @@ public void testCoder() throws IOException { StringUtf8Coder elementCoder = StringUtf8Coder.of(); // Generics fail to see here that this is Coder org.apache.beam.sdk.coders.Coder windowCoder = GlobalWindow.Coder.INSTANCE; - WindowedValue.WindowedValueCoder windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of(elementCoder, windowCoder); + WindowedValues.WindowedValueCoder windowedValueCoder = + WindowedValues.FullWindowedValueCoder.of(elementCoder, windowCoder); KV key = KV.of("one", 1); BufferedElements.Coder coder = new BufferedElements.Coder(windowedValueCoder, windowCoder, key); BufferedElement element = new BufferedElements.Element( - WindowedValue.of("test", new Instant(2), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); + WindowedValues.of("test", new Instant(2), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); BufferedElement timerElement = new BufferedElements.Timer( "timerId", diff --git a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunnerTest.java b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunnerTest.java index 58f6fd8484bf..c399185bc5cd 100644 --- a/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunnerTest.java +++ b/runners/flink/src/test/java/org/apache/beam/runners/flink/translation/wrappers/streaming/stableinput/BufferingDoFnRunnerTest.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.flink.api.common.state.ListState; import org.apache.flink.runtime.state.OperatorStateBackend; @@ -168,7 +168,7 @@ private static BufferingDoFnRunner createBufferingDoFnRunner( doFnRunner, "stable-input", StringUtf8Coder.of(), - WindowedValue.getFullCoder(VarIntCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(VarIntCoder.of(), GlobalWindow.Coder.INSTANCE), operatorStateBackend, null, concurrentCheckpoints, diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java index 0478a2c5f0a1..e701ab0dbe67 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchStatefulParDoOverrides.java @@ -38,7 +38,6 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformReplacements; import org.apache.beam.sdk.util.construction.ReplacementOutputs; import org.apache.beam.sdk.values.KV; @@ -46,6 +45,8 @@ import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; @@ -239,7 +240,7 @@ public PCollection>>>>> expand .setCoder( KvCoder.of( keyCoder, - KvCoder.of(InstantCoder.of(), WindowedValue.getFullCoder(kvCoder, windowCoder)))) + KvCoder.of(InstantCoder.of(), WindowedValues.getFullCoder(kvCoder, windowCoder)))) // Group by key and sort by timestamp, dropping windows as they are reified .apply("PartitionKeys", new GroupByKeyAndSortValuesOnly<>()) @@ -258,7 +259,7 @@ public void processElement(final ProcessContext c, final BoundedWindow window) { KV.of( c.element().getKey(), KV.of( - c.timestamp(), WindowedValue.of(c.element(), c.timestamp(), window, c.pane())))); + c.timestamp(), WindowedValues.of(c.element(), c.timestamp(), window, c.pane())))); } } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java index 6af6c499cfad..8cb82c2ca42d 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/BatchViewOverrides.java @@ -64,8 +64,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.SystemDoFnInternal; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollection.IsBounded; @@ -74,6 +72,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; 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.MoreObjects; @@ -327,7 +328,7 @@ public void processElement(ProcessContext c, BoundedWindow untypedWindow) throws coder.hash(ImmutableList.of(c.element().getKey())), KV.of( KV.of(c.element().getKey(), window), - WindowedValue.of( + WindowedValues.of( c.element().getValue(), c.timestamp(), untypedWindow, c.pane())))); } } @@ -1024,7 +1025,7 @@ public void processElement(ProcessContext c) throws Exception { c.output( IsmRecord.of( ImmutableList.of(GlobalWindow.INSTANCE, indexInBundle), - WindowedValue.of(c.element(), c.timestamp(), GlobalWindow.INSTANCE, c.pane()))); + WindowedValues.of(c.element(), c.timestamp(), GlobalWindow.INSTANCE, c.pane()))); indexInBundle += 1; } } @@ -1237,7 +1238,7 @@ public void processElement(ProcessContext c, BoundedWindow untypedWindow) throws c.output( KV.of( ismCoderForHash.hash(ImmutableList.of(window)), - KV.of(window, WindowedValue.of(c.element(), c.timestamp(), window, c.pane())))); + KV.of(window, WindowedValues.of(c.element(), c.timestamp(), window, c.pane())))); } } @@ -1361,7 +1362,7 @@ private static WindowedValue valueInEmptyWindows(T value) { return new ValueInEmptyWindows<>(value); } - private static class ValueInEmptyWindows extends WindowedValue { + private static class ValueInEmptyWindows implements WindowedValue { private final T value; @@ -1394,6 +1395,11 @@ public PaneInfo getPane() { return PaneInfo.NO_FIRING; } + @Override + public Iterable> explodeWindows() { + return Collections.emptyList(); + } + @Override public String toString() { return MoreObjects.toStringHelper(getClass()).add("value", getValue()).toString(); diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java index e0e752d0a2e9..08d84705c5c7 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/DataflowPipelineTranslator.java @@ -93,7 +93,6 @@ import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.DoFnInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.SdkComponents; @@ -108,6 +107,7 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; @@ -675,7 +675,7 @@ public void addOutput(String name, PCollection value) { translator.producers.put(value, translator.currentTransform); // Wrap the PCollection element Coder inside a WindowedValueCoder. Coder coder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( value.getCoder(), value.getWindowingStrategy().getWindowFn().windowCoder()); addOutput(name, value, coder); } @@ -688,7 +688,7 @@ public void addCollectionToSingletonOutput( // The inputValueCoder for the input PCollection should be some // WindowedValueCoder of the input PCollection's element // coder. - checkState(inputValueCoder instanceof WindowedValue.WindowedValueCoder); + checkState(inputValueCoder instanceof WindowedValues.WindowedValueCoder); // The outputValueCoder for the output should be an // IterableCoder of the inputValueCoder. This is a property // of the backend "CollectionToSingleton" step. 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 00a91070eb04..87b9dc6e6ae1 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 @@ -20,7 +20,6 @@ import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.beam.sdk.io.gcp.pubsub.PubsubIO.ENABLE_CUSTOM_PUBSUB_SINK; import static org.apache.beam.sdk.io.gcp.pubsub.PubsubIO.ENABLE_CUSTOM_PUBSUB_SOURCE; -import static org.apache.beam.sdk.options.ExperimentalOptions.hasExperiment; import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; import static org.apache.beam.sdk.util.SerializableUtils.serializeToByteArray; import static org.apache.beam.sdk.util.StringUtils.byteArrayToJsonString; @@ -135,7 +134,6 @@ import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.NameUtils; import org.apache.beam.sdk.util.ReleaseInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ReflectHelpers; import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.sdk.util.construction.DeduplicatedFlattenFactory; @@ -168,6 +166,7 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.TextFormat; @@ -2177,7 +2176,7 @@ private static void translate( // Using a GlobalWindowCoder as a place holder because GlobalWindowCoder is known coder. stepContext.addEncodingInput( - WindowedValue.getFullCoder(VoidCoder.of(), GlobalWindow.Coder.INSTANCE)); + WindowedValues.getFullCoder(VoidCoder.of(), GlobalWindow.Coder.INSTANCE)); stepContext.addInput(PropertyNames.PARALLEL_INPUT, input); } } @@ -2234,12 +2233,13 @@ public void translate(Impulse transform, TransformTranslator.TranslationContext } else { StepTranslationContext stepContext = context.addStep(transform, "ParallelRead"); stepContext.addInput(PropertyNames.FORMAT, "impulse"); - WindowedValue.FullWindowedValueCoder coder = - WindowedValue.getFullCoder( + WindowedValues.FullWindowedValueCoder coder = + WindowedValues.getFullCoder( context.getOutput(transform).getCoder(), GlobalWindow.Coder.INSTANCE); byte[] encodedImpulse; try { - encodedImpulse = encodeToByteArray(coder, WindowedValue.valueInGlobalWindow(new byte[0])); + encodedImpulse = + encodeToByteArray(coder, WindowedValues.valueInGlobalWindow(new byte[0])); } catch (Exception e) { throw new RuntimeException(e); } diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java index 62351739228f..c0a83c5a8226 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjectTranslators.java @@ -43,10 +43,10 @@ import org.apache.beam.sdk.util.InstanceBuilder; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; /** Utilities for creating {@link CloudObjectTranslator} instances for {@link Coder Coders}. */ diff --git a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java index 208b46141f03..f5de65a363f9 100644 --- a/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java +++ b/runners/google-cloud-dataflow-java/src/main/java/org/apache/beam/runners/dataflow/util/CloudObjects.java @@ -33,9 +33,9 @@ import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java index 72d6786c02d0..1f160ffdf3af 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/BatchViewOverridesTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -38,10 +38,11 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; 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.ImmutableMap; import org.joda.time.Instant; @@ -133,26 +134,27 @@ public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception ImmutableList.of( KV.of( windowA, - WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), KV.of( windowA, - WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)), KV.of( windowA, - WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)))), + WindowedValues.of( + 121L, new Instant(14), windowB, PaneInfo.NO_FIRING)))), KV.of( 2, (Iterable>>) ImmutableList.of( KV.of( windowC, - WindowedValue.of( + WindowedValues.of( 210L, new Instant(25), windowC, PaneInfo.NO_FIRING))))); // The order of the output elements is important relative to processing order @@ -161,22 +163,22 @@ public void testBatchViewAsListToIsmRecordForNonGlobalWindow() throws Exception contains( IsmRecord.of( ImmutableList.of(windowA, 0L), - WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowA, 1L), - WindowedValue.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(111L, new Instant(3), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowA, 2L), - WindowedValue.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(112L, new Instant(4), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowB, 0L), - WindowedValue.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(120L, new Instant(12), windowB, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowB, 1L), - WindowedValue.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(121L, new Instant(14), windowB, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(windowC, 0L), - WindowedValue.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING)))); + WindowedValues.of(210L, new Instant(25), windowC, PaneInfo.NO_FIRING)))); } @Test @@ -221,27 +223,27 @@ public void testToIsmRecordForMapLikeDoFn() throws Exception { ImmutableList.of( KV.of( KV.of(1L, windowA), - WindowedValue.of( + WindowedValues.of( 110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), // same window same key as to previous KV.of( KV.of(1L, windowA), - WindowedValue.of( + WindowedValues.of( 111L, new Instant(2), windowA, PaneInfo.NO_FIRING)), // same window different key as to previous KV.of( KV.of(2L, windowA), - WindowedValue.of( + WindowedValues.of( 120L, new Instant(3), windowA, PaneInfo.NO_FIRING)), // different window same key as to previous KV.of( KV.of(2L, windowB), - WindowedValue.of( + WindowedValues.of( 210L, new Instant(11), windowB, PaneInfo.NO_FIRING)), // different window and different key as to previous KV.of( KV.of(3L, windowB), - WindowedValue.of( + WindowedValues.of( 220L, new Instant(12), windowB, PaneInfo.NO_FIRING)))), KV.of( 2, @@ -250,7 +252,7 @@ public void testToIsmRecordForMapLikeDoFn() throws Exception { // different shard KV.of( KV.of(4L, windowC), - WindowedValue.of( + WindowedValues.of( 330L, new Instant(21), windowC, PaneInfo.NO_FIRING))))); // The order of the output elements is important relative to processing order @@ -259,22 +261,22 @@ public void testToIsmRecordForMapLikeDoFn() throws Exception { contains( IsmRecord.of( ImmutableList.of(1L, windowA, 0L), - WindowedValue.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(1L, windowA, 1L), - WindowedValue.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(111L, new Instant(2), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(2L, windowA, 0L), - WindowedValue.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)), + WindowedValues.of(120L, new Instant(3), windowA, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(2L, windowB, 0L), - WindowedValue.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(210L, new Instant(11), windowB, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(3L, windowB, 0L), - WindowedValue.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)), + WindowedValues.of(220L, new Instant(12), windowB, PaneInfo.NO_FIRING)), IsmRecord.of( ImmutableList.of(4L, windowC, 0L), - WindowedValue.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING)))); + WindowedValues.of(330L, new Instant(21), windowC, PaneInfo.NO_FIRING)))); // Verify the number of unique keys per window. assertThat( @@ -351,12 +353,12 @@ public void testToIsmRecordForMapLikeDoFnWithoutUniqueKeysThrowsException() thro ImmutableList.of( KV.of( KV.of(1L, windowA), - WindowedValue.of( + WindowedValues.of( 110L, new Instant(1), windowA, PaneInfo.NO_FIRING)), // same window same key as to previous KV.of( KV.of(1L, windowA), - WindowedValue.of( + WindowedValues.of( 111L, new Instant(2), windowA, PaneInfo.NO_FIRING))))); thrown.expect(IllegalStateException.class); @@ -495,19 +497,19 @@ public void testToMapDoFn() throws Exception { ImmutableList.of( KV.of( windowA, - WindowedValue.of( + WindowedValues.of( KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)), KV.of( windowA, - WindowedValue.of( + WindowedValues.of( KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValue.of( + WindowedValues.of( KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValue.of( + WindowedValues.of( KV.of(3L, 31L), new Instant(15), windowB, @@ -518,7 +520,7 @@ public void testToMapDoFn() throws Exception { ImmutableList.of( KV.of( windowC, - WindowedValue.of( + WindowedValues.of( KV.of(4L, 41L), new Instant(25), windowC, @@ -569,29 +571,29 @@ public void testToMultimapDoFn() throws Exception { ImmutableList.of( KV.of( windowA, - WindowedValue.of( + WindowedValues.of( KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)), // [BEAM-5184] Specifically test with a duplicate value to ensure that // duplicate key/values are not lost. KV.of( windowA, - WindowedValue.of( + WindowedValues.of( KV.of(1L, 11L), new Instant(3), windowA, PaneInfo.NO_FIRING)), KV.of( windowA, - WindowedValue.of( + WindowedValues.of( KV.of(1L, 12L), new Instant(5), windowA, PaneInfo.NO_FIRING)), KV.of( windowA, - WindowedValue.of( + WindowedValues.of( KV.of(2L, 21L), new Instant(7), windowA, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValue.of( + WindowedValues.of( KV.of(2L, 21L), new Instant(13), windowB, PaneInfo.NO_FIRING)), KV.of( windowB, - WindowedValue.of( + WindowedValues.of( KV.of(3L, 31L), new Instant(15), windowB, @@ -602,7 +604,7 @@ public void testToMultimapDoFn() throws Exception { ImmutableList.of( KV.of( windowC, - WindowedValue.of( + WindowedValues.of( KV.of(4L, 41L), new Instant(25), windowC, diff --git a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java index 68c0260a9bdd..66e957fbd746 100644 --- a/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java +++ b/runners/google-cloud-dataflow-java/src/test/java/org/apache/beam/runners/dataflow/util/CloudObjectsTest.java @@ -64,12 +64,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.InstanceBuilder; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; @@ -154,7 +154,7 @@ public static Iterable> data() { .add(IterableCoder.of(VarLongCoder.of())) .add(KvCoder.of(VarLongCoder.of(), ByteArrayCoder.of())) .add( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(VarLongCoder.of(), ByteArrayCoder.of()), IntervalWindow.getCoder())) .add(ByteArrayCoder.of()) diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java index 7697271792a8..4a752047e256 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/AssignWindowsParDoFnFactory.java @@ -31,8 +31,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; @@ -111,7 +112,7 @@ public BoundedWindow window() { }); WindowedValue res = - WindowedValue.of(elem.getValue(), elem.getTimestamp(), windows, elem.getPane()); + WindowedValues.of(elem.getValue(), elem.getTimestamp(), windows, elem.getPane()); receiver.process(res); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java index 3667893393fd..8e2b325b580a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/BatchModeUngroupingParDoFn.java @@ -19,8 +19,8 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn; import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java index e917a150e0a2..bd991560c186 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactory.java @@ -32,9 +32,9 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; 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/DataflowOutputCounter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java index e4292f2252a2..7c5859a9d324 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowOutputCounter.java @@ -24,7 +24,7 @@ import org.apache.beam.runners.dataflow.worker.counters.NameContext; import org.apache.beam.runners.dataflow.worker.util.common.worker.ElementCounter; import org.apache.beam.runners.dataflow.worker.util.common.worker.OutputObjectAndByteCounter; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java index 44907f3631d8..ab3464c4ce53 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionView.java @@ -30,7 +30,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; @@ -38,6 +37,7 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java index 22718b745105..741716b82848 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/DataflowProcessFnRunner.java @@ -30,8 +30,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; @@ -67,7 +68,7 @@ private static WindowedValue> placeIntoElementWindo WindowedValue> compressedElem) { checkTrivialOuterWindows(compressedElem); WindowedValue> res = - WindowedValue.of( + WindowedValues.of( compressedElem.getValue(), BoundedWindow.TIMESTAMP_MIN_VALUE, getUnderlyingWindow(compressedElem.getValue()), diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java index 95bbcddc4cce..d21ff3023914 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowFnRunner.java @@ -28,8 +28,9 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.joda.time.Instant; /** @@ -104,7 +105,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - WindowedValue windowed = WindowedValue.of(output, timestamp, windows, pane); + WindowedValue windowed = WindowedValues.of(output, timestamp, windows, pane); outputManager.output(mainOutputTag, windowed); } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java index a7610acba280..3424d3fe2380 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowParDoFnFactory.java @@ -50,10 +50,10 @@ import org.apache.beam.sdk.transforms.CombineWithContext.Context; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java index a611769eb5ee..203883f4a34e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupAlsoByWindowsParDoFn.java @@ -35,10 +35,10 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.StreamingOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReader.java index 8af6068f2a3a..f991c7a588a0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReader.java @@ -49,13 +49,13 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterator; import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.util.common.Reiterator; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderFactory.java index daf1e87f4296..e69f41ade145 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderFactory.java @@ -33,8 +33,8 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleReadCounterFactory; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderWithFaultyBytesReadCounter.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderWithFaultyBytesReadCounter.java index 1540431bb178..d852d18410f2 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderWithFaultyBytesReadCounter.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderWithFaultyBytesReadCounter.java @@ -21,8 +21,8 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ShuffleReadCounterFactory; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactory.java index 499dcaa4e77c..91fb640a1757 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactory.java @@ -63,9 +63,9 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.fn.IdGenerator; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.graph.MutableNetwork; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReader.java index b19e77cde871..13defcc84212 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReader.java @@ -25,7 +25,7 @@ import org.apache.beam.runners.dataflow.util.RandomAccessData; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; import org.apache.beam.sdk.io.fs.ResourceId; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactory.java index d0bf0c96f9a0..d37f2a0f74e5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactory.java @@ -36,8 +36,8 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.WeightedValue; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderImpl.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderImpl.java index 86aab6cd1f97..d9430b401416 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderImpl.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmReaderImpl.java @@ -61,7 +61,7 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.util.VarInt; import org.apache.beam.sdk.util.WeightedValue; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; 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.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java index c2e8c334a098..687cbe1d7830 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReader.java @@ -66,7 +66,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PCollectionViews.HasDefaultValue; @@ -82,6 +81,7 @@ import org.apache.beam.sdk.values.PCollectionViews.SingletonViewFn; import org.apache.beam.sdk.values.PCollectionViews.SingletonViewFn2; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; 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.MoreObjects; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSink.java index 748e3a1542af..43a3a71c6097 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSink.java @@ -45,7 +45,7 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.util.MimeTypes; import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; 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.io.CountingOutputStream; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSinkFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSinkFactory.java index 99d38246b42f..ec650481a333 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSinkFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/IsmSinkFactory.java @@ -33,8 +33,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.math.DoubleMath; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java index 2ca04d9168af..6951e3a95b20 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactory.java @@ -32,9 +32,9 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; /** * A {@link ParDoFnFactory} which returns a {@link ParDoFn} that transforms all {@code diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java index ca709de7effb..a7dca89d31e0 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFns.java @@ -44,9 +44,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.io.ByteStreams; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.io.CountingOutputStream; @@ -113,7 +114,7 @@ static ParDoFn create( GroupingTables.bufferingAndSampling( new WindowingCoderGroupingKeyCreator<>(keyCoder), PairInfo.create(), - new CoderSizeEstimator<>(WindowedValue.getValueOnlyCoder(keyCoder)), + new CoderSizeEstimator<>(WindowedValues.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator<>(inputCoder), 0.001, /*sizeEstimatorSampleRate*/ maxSizeBytes /*maxSizeBytes*/); @@ -128,7 +129,7 @@ static ParDoFn create( new WindowingCoderGroupingKeyCreator<>(keyCoder), PairInfo.create(), valueCombiner, - new CoderSizeEstimator<>(WindowedValue.getValueOnlyCoder(keyCoder)), + new CoderSizeEstimator<>(WindowedValues.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator<>(combineFn.getAccumulatorCoder()), 0.001, /*sizeEstimatorSampleRate*/ maxSizeBytes /*maxSizeBytes*/); @@ -242,7 +243,7 @@ public WindowingCoderGroupingKeyCreator(Coder coder) { public Object createGroupingKey(WindowedValue key) throws Exception { // Ignore timestamp for grouping purposes. // The PGBK output will inherit the timestamp of one of its inputs. - return WindowedValue.of( + return WindowedValues.of( coder.structuralValue(key.getValue()), ignored, key.getWindows(), key.getPane()); } } @@ -304,7 +305,7 @@ public void processElement(Object elem) throws Exception { WindowedValue> input = (WindowedValue>) elem; for (BoundedWindow w : input.getWindows()) { WindowedValue> windowsExpandedInput = - WindowedValue.of(input.getValue(), input.getTimestamp(), w, input.getPane()); + WindowedValues.of(input.getValue(), input.getTimestamp(), w, input.getPane()); groupingTable.put(windowsExpandedInput, receiver); } } @@ -361,7 +362,7 @@ public void processElement(Object elem) throws Exception { WindowedValue> input = (WindowedValue>) elem; for (BoundedWindow w : input.getWindows()) { WindowedValue> windowsExpandedInput = - WindowedValue.of(input.getValue(), input.getTimestamp(), w, input.getPane()); + WindowedValues.of(input.getValue(), input.getTimestamp(), w, input.getPane()); if (!sideInputFetcher.storeIfBlocked(windowsExpandedInput)) { groupingTable.put(windowsExpandedInput, receiver); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReader.java index 81dfd59ff2b4..149915533790 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReader.java @@ -28,9 +28,9 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderFactory.java index 1a8c7e7d49ff..5e672c08dddb 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderFactory.java @@ -28,8 +28,8 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java index b27d26ed21ac..9098cdc6717e 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSink.java @@ -32,7 +32,7 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.grpc.v1p69p0.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.ImmutableMap; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java index 5d2c491ff72b..024b790e8ca9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubReader.java @@ -34,8 +34,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; @@ -136,7 +137,7 @@ protected WindowedValue decodeMessage(Windmill.Message message) throws IOExce } else { value = coder.decode(data, Coder.Context.OUTER); } - return WindowedValue.timestampedValueInGlobalWindow( + return WindowedValues.timestampedValueInGlobalWindow( value, WindmillTimeUtils.windmillToHarnessTimestamp(message.getTimestamp())); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java index b4962422b37e..0616f08ae25f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/PubsubSink.java @@ -35,8 +35,8 @@ import org.apache.beam.sdk.transforms.SimpleFunction; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java index a5971bb6bdaf..39a3bf24fc24 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactory.java @@ -26,9 +26,10 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn; import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.checkerframework.checker.nullness.qual.Nullable; /** A {@link ParDoFnFactory} to create instances of {@link ReifyTimestampAndWindowsParDoFn}. */ @@ -70,10 +71,10 @@ public void processElement(Object untypedElem) throws Exception { WindowedValue> typedElem = (WindowedValue>) untypedElem; receiver.process( - WindowedValue.of( + WindowedValues.of( KV.of( typedElem.getValue().getKey(), - WindowedValue.of( + WindowedValues.of( typedElem.getValue().getValue(), typedElem.getTimestamp(), typedElem.getWindows(), diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSink.java index 02e2d48c369a..49ab341eee1c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSink.java @@ -34,9 +34,9 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; 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.primitives.Ints; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactory.java index 53175eaf91e5..a03f43defecc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactory.java @@ -28,7 +28,7 @@ import org.apache.beam.runners.dataflow.worker.util.WorkerPropertyNames; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java index 558848f488a7..16b198704449 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFn.java @@ -53,9 +53,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.DoFnInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; 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; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SplittableProcessFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SplittableProcessFnFactory.java index 4bd130769150..d0461b4165af 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SplittableProcessFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/SplittableProcessFnFactory.java @@ -53,10 +53,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.DoFnInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.joda.time.Duration; @@ -161,7 +161,7 @@ public void outputWindowedValue( Collection windows, PaneInfo pane) { outputManager.output( - mainOutputTag, WindowedValue.of(output, timestamp, windows, pane)); + mainOutputTag, WindowedValues.of(output, timestamp, windows, pane)); } @Override @@ -171,7 +171,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane)); + outputManager.output(tag, WindowedValues.of(output, timestamp, windows, pane)); } }, sideInputReader, diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowReshuffleFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowReshuffleFn.java index be14e7cc3bcf..cf334d451ebc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowReshuffleFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowReshuffleFn.java @@ -25,8 +25,8 @@ import org.apache.beam.runners.dataflow.worker.util.StreamingGroupAlsoByWindowFn; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.ReshuffleTrigger; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java index 36a93f6282bd..e5d8a18be76f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunner.java @@ -34,7 +34,7 @@ import org.apache.beam.sdk.state.ValueState; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; 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.Lists; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactory.java index b9bf203e529e..b4c8af80d8cd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactory.java @@ -32,8 +32,8 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; /** * A {@link ParDoFnFactory} which constructs {@link StreamingPCollectionViewWriterParDoFn}s used to diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java index c04ccc383c73..61730b0c8d88 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterParDoFn.java @@ -24,8 +24,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java index e01be8035dec..b5b723adb2b5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunner.java @@ -23,7 +23,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java index c1a18cbb5dc4..76913baa6aa7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcher.java @@ -48,8 +48,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.Parser; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; @@ -91,7 +92,7 @@ public StreamingSideInputFetcher( this.blockedMapAddr = blockedMapAddr(mainWindowCoder); this.elementsAddr = StateTags.makeSystemTagInternal( - StateTags.bag("elem", WindowedValue.getFullCoder(inputCoder, mainWindowCoder))); + StateTags.bag("elem", WindowedValues.getFullCoder(inputCoder, mainWindowCoder))); this.oldTimersAddr = StateTags.makeSystemTagInternal(StateTags.bag("timer", TimerDataCoder.of(mainWindowCoder))); this.timersAddr = diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java index 2627441ce39a..f9e2d6de2461 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactory.java @@ -35,9 +35,9 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; 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/UngroupedWindmillReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java index cdce6b88ba66..a56afd636137 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/UngroupedWindmillReader.java @@ -32,9 +32,10 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; @@ -124,10 +125,10 @@ protected WindowedValue decodeMessage(Windmill.Message message) throws IOExce @SuppressWarnings("unchecked") T result = (T) KV.of(decode(kvCoder.getKeyCoder(), key), decode(kvCoder.getValueCoder(), data)); - return WindowedValue.of(result, timestampMillis, windows, pane); + return WindowedValues.of(result, timestampMillis, windows, pane); } else { notifyElementRead(data.available() + metadata.available()); - return WindowedValue.of(decode(valueCoder, data), timestampMillis, windows, pane); + return WindowedValues.of(decode(valueCoder, data), timestampMillis, windows, pane); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java index 3fe48870d246..3ddb3c2003db 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactory.java @@ -27,9 +27,9 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.Values; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; /** * A {@link ParDoFnFactory} which returns a {@link ParDoFn} with similar behavior to {@link diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java index 40ee075f5fb4..29457678576b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItem.java @@ -34,8 +34,9 @@ import org.apache.beam.sdk.coders.StructuredCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicate; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicates; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; @@ -110,7 +111,7 @@ public Iterable> elementsIterable() { InputStream inputStream = message.getData().newInput(); ElemT value = valueCoder.decode(inputStream, Coder.Context.OUTER); - return WindowedValue.of(value, timestamp, windows, pane); + return WindowedValues.of(value, timestamp, windows, pane); } 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/WindmillReaderIteratorBase.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java index 405440ba1193..5e68641bf661 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBase.java @@ -21,7 +21,7 @@ import java.util.NoSuchElementException; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Optional; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java index abdbeb9a469f..a75374ba64af 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindmillSink.java @@ -36,11 +36,11 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; import org.apache.beam.sdk.values.ValueWithRecordId.ValueWithRecordIdCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java index e7c1cdfe0e9e..d91a5412b917 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/WindowingWindmillReader.java @@ -32,8 +32,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; 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 b9aecfda611f..6323e757561e 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 @@ -59,8 +59,9 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.BackOff; import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.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.collect.ImmutableList; @@ -647,7 +648,7 @@ public boolean advance() throws IOException { @Override public WindowedValue getCurrent() throws NoSuchElementException { - return WindowedValue.timestampedValueInGlobalWindow( + return WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp()); } @@ -858,7 +859,7 @@ public boolean advance() throws IOException { @Override public WindowedValue> getCurrent() throws NoSuchElementException { WindowedValue result = - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp()); return result.withValue( new ValueWithRecordId<>(result.getValue(), reader.getCurrentRecordId())); diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowAndCombineFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowAndCombineFn.java index c3f1af8d6637..77f45eed23e3 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowAndCombineFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowAndCombineFn.java @@ -36,9 +36,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; 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; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowFn.java index 516c2b6428a4..5256142f6ad1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowFn.java @@ -18,8 +18,8 @@ package org.apache.beam.runners.dataflow.worker.util; import org.apache.beam.runners.dataflow.worker.GroupAlsoByWindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; /** {@link GroupAlsoByWindowFn} whose input is a {@link KV}, for use in batch. */ public abstract class BatchGroupAlsoByWindowFn diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleFn.java index 815ac1f822e7..8a1602a50eea 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleFn.java @@ -24,8 +24,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.ReshuffleTrigger; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaIteratorsFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaIteratorsFn.java index 5e9e79a40bc3..aabbf08be726 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaIteratorsFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaIteratorsFn.java @@ -32,12 +32,12 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterator; import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.util.common.Reiterator; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaOutputBufferFn.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaOutputBufferFn.java index 67995cdb0ded..767063661149 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaOutputBufferFn.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowViaOutputBufferFn.java @@ -33,9 +33,9 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java index 5084af0d187e..ca0e2279fb03 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/util/ValueInEmptyWindows.java @@ -22,7 +22,7 @@ import java.util.Objects; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; @@ -37,7 +37,7 @@ * the Dataflow worker uses this in places to pass a {@link WindowedValue} to a method that requires * one without having to provide the global window, which could alter size estimations, etc. */ -public class ValueInEmptyWindows extends WindowedValue { +public class ValueInEmptyWindows implements WindowedValue { private final T value; public ValueInEmptyWindows(T value) { @@ -49,6 +49,11 @@ public PaneInfo getPane() { return PaneInfo.NO_FIRING; } + @Override + public Iterable> explodeWindows() { + return Collections.emptyList(); + } + @Override public T getValue() { return value; diff --git a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java index d5e0b3a24e2a..269799903300 100644 --- a/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java +++ b/runners/google-cloud-dataflow-java/worker/src/main/java/org/apache/beam/runners/dataflow/worker/windmill/work/processing/ComputationWorkExecutorFactory.java @@ -55,7 +55,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.fn.IdGenerator; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.graph.MutableNetwork; @@ -231,16 +231,16 @@ ComputationWorkExecutor createComputationWorkExecutor( * stage. This encodes many assumptions about how the streaming execution context works. */ private @Nullable Coder extractKeyCoder(Coder readCoder) { - if (!(readCoder instanceof WindowedValue.WindowedValueCoder)) { + if (!(readCoder instanceof WindowedValues.WindowedValueCoder)) { throw new RuntimeException( String.format( "Expected coder for streaming read to be %s, but received %s", - WindowedValue.WindowedValueCoder.class.getSimpleName(), readCoder)); + WindowedValues.WindowedValueCoder.class.getSimpleName(), readCoder)); } // Note that TimerOrElementCoder is a backwards-compatibility class // that is really a FakeKeyedWorkItemCoder - Coder valueCoder = ((WindowedValue.WindowedValueCoder) readCoder).getValueCoder(); + Coder valueCoder = ((WindowedValues.WindowedValueCoder) readCoder).getValueCoder(); if (valueCoder instanceof KvCoder) { return ((KvCoder) valueCoder).getKeyCoder(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactoryTest.java index 58730408b3a6..0c8a3fe36ae4 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteReaderFactoryTest.java @@ -29,7 +29,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.checkerframework.checker.nullness.qual.Nullable; import org.hamcrest.core.IsInstanceOf; import org.junit.Assert; @@ -74,7 +74,7 @@ NativeReader runTestCreateAvroReader( @Test public void testCreatePlainAvroByteReader() throws Exception { Coder coder = - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); NativeReader reader = runTestCreateAvroReader( pathToAvroFile, null, null, CloudObjects.asCloudObject(coder, /*sdkComponents=*/ null)); @@ -90,7 +90,7 @@ public void testCreatePlainAvroByteReader() throws Exception { @Test public void testCreateRichAvroByteReader() throws Exception { Coder coder = - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); NativeReader reader = runTestCreateAvroReader( pathToAvroFile, 200L, 500L, CloudObjects.asCloudObject(coder, /*sdkComponents=*/ null)); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteSinkFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteSinkFactoryTest.java index 34f6f70e8201..59c467663171 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteSinkFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/AvroByteSinkFactoryTest.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.hamcrest.core.IsInstanceOf; import org.junit.Assert; import org.junit.Test; @@ -58,7 +58,7 @@ private Sink runTestCreateAvroSink(String filename, Coder coder) throws Ex @Test public void testCreateAvroByteSink() throws Exception { Coder coder = - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); Sink sink = runTestCreateAvroSink(pathToAvroFile, coder); assertThat(sink, new IsInstanceOf(AvroByteSink.class)); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactoryTest.java index 786f3ae76b83..4ca632acc66d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CombineValuesFnFactoryTest.java @@ -50,11 +50,11 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -235,17 +235,17 @@ public void testCombineValuesFnAll() throws Exception { combineParDoFn.startBundle(receiver); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("a", Arrays.asList(5, 6, 7)))); + WindowedValues.valueInGlobalWindow(KV.of("a", Arrays.asList(5, 6, 7)))); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("b", Arrays.asList(1, 3, 7)))); + WindowedValues.valueInGlobalWindow(KV.of("b", Arrays.asList(1, 3, 7)))); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("c", Arrays.asList(3, 6, 8, 9)))); + WindowedValues.valueInGlobalWindow(KV.of("c", Arrays.asList(3, 6, 8, 9)))); combineParDoFn.finishBundle(); Object[] expectedReceivedElems = { - WindowedValue.valueInGlobalWindow(KV.of("a", String.format("%.1f", 6.0))), - WindowedValue.valueInGlobalWindow(KV.of("b", String.format("%.1f", 3.7))), - WindowedValue.valueInGlobalWindow(KV.of("c", String.format("%.1f", 6.5))), + WindowedValues.valueInGlobalWindow(KV.of("a", String.format("%.1f", 6.0))), + WindowedValues.valueInGlobalWindow(KV.of("b", String.format("%.1f", 3.7))), + WindowedValues.valueInGlobalWindow(KV.of("c", String.format("%.1f", 6.5))), }; assertArrayEquals(expectedReceivedElems, receiver.receivedElems.toArray()); } @@ -268,17 +268,17 @@ public void testCombineValuesFnAdd() throws Exception { combineParDoFn.startBundle(receiver); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("a", Arrays.asList(5, 6, 7)))); + WindowedValues.valueInGlobalWindow(KV.of("a", Arrays.asList(5, 6, 7)))); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("b", Arrays.asList(1, 3, 7)))); + WindowedValues.valueInGlobalWindow(KV.of("b", Arrays.asList(1, 3, 7)))); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("c", Arrays.asList(3, 6, 8, 9)))); + WindowedValues.valueInGlobalWindow(KV.of("c", Arrays.asList(3, 6, 8, 9)))); combineParDoFn.finishBundle(); Object[] expectedReceivedElems = { - WindowedValue.valueInGlobalWindow(KV.of("a", new CountSum(3, 18))), - WindowedValue.valueInGlobalWindow(KV.of("b", new CountSum(3, 11))), - WindowedValue.valueInGlobalWindow(KV.of("c", new CountSum(4, 26))) + WindowedValues.valueInGlobalWindow(KV.of("a", new CountSum(3, 18))), + WindowedValues.valueInGlobalWindow(KV.of("b", new CountSum(3, 11))), + WindowedValues.valueInGlobalWindow(KV.of("c", new CountSum(4, 26))) }; assertArrayEquals(expectedReceivedElems, receiver.receivedElems.toArray()); } @@ -301,17 +301,17 @@ public void testCombineValuesFnMerge() throws Exception { combineParDoFn.startBundle(receiver); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( KV.of( "a", Arrays.asList(new CountSum(3, 6), new CountSum(2, 9), new CountSum(1, 12))))); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( KV.of("b", Arrays.asList(new CountSum(2, 20), new CountSum(1, 1))))); combineParDoFn.finishBundle(); Object[] expectedReceivedElems = { - WindowedValue.valueInGlobalWindow(KV.of("a", new CountSum(6, 27))), - WindowedValue.valueInGlobalWindow(KV.of("b", new CountSum(3, 21))), + WindowedValues.valueInGlobalWindow(KV.of("a", new CountSum(6, 27))), + WindowedValues.valueInGlobalWindow(KV.of("b", new CountSum(3, 21))), }; assertArrayEquals(expectedReceivedElems, receiver.receivedElems.toArray()); } @@ -334,15 +334,15 @@ public void testCombineValuesFnExtract() throws Exception { combineParDoFn.startBundle(receiver); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("a", new CountSum(6, 27)))); + WindowedValues.valueInGlobalWindow(KV.of("a", new CountSum(6, 27)))); combineParDoFn.processElement( - WindowedValue.valueInGlobalWindow(KV.of("b", new CountSum(3, 21)))); + WindowedValues.valueInGlobalWindow(KV.of("b", new CountSum(3, 21)))); combineParDoFn.finishBundle(); assertArrayEquals( new Object[] { - WindowedValue.valueInGlobalWindow(KV.of("a", String.format("%.1f", 4.5))), - WindowedValue.valueInGlobalWindow(KV.of("b", String.format("%.1f", 7.0))) + WindowedValues.valueInGlobalWindow(KV.of("a", String.format("%.1f", 4.5))), + WindowedValues.valueInGlobalWindow(KV.of("b", String.format("%.1f", 7.0))) }, receiver.receivedElems.toArray()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java index e95cfc0a0ff5..a43932d243be 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/CreateIsmShardKeyAndSortKeyDoFnFactoryTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow.worker; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionViewTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionViewTest.java index 413e03f11ff8..f5e6dca64fdd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionViewTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DataflowPortabilityPCollectionViewTest.java @@ -23,9 +23,9 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.Materializations; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactoryTest.java index 371fce54ac65..06138c02653b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/DefaultParDoFnFactoryTest.java @@ -37,8 +37,8 @@ import org.apache.beam.sdk.util.DoFnInfo; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.hamcrest.Matchers; @@ -121,7 +121,7 @@ public void testCreateSimpleParDoFn() throws Exception { SimpleParDoFn simpleParDoFn = (SimpleParDoFn) parDoFn; parDoFn.startBundle(new OutputReceiver()); // DoFnInfo may not yet be initialized until an element is processed - parDoFn.processElement(WindowedValue.valueInGlobalWindow("foo")); + parDoFn.processElement(WindowedValues.valueInGlobalWindow("foo")); @SuppressWarnings("rawtypes") DoFnInfo doFnInfo = simpleParDoFn.getDoFnInfo(); DoFn innerDoFn = (TestDoFn) doFnInfo.getDoFn(); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java index fa04028b5dd0..cc006d5b1651 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/GroupingShuffleReaderTest.java @@ -76,10 +76,11 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.Reiterable; import org.apache.beam.sdk.util.common.Reiterator; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; @@ -171,7 +172,7 @@ public void tearDown() throws IOException { private List writeShuffleEntries( List>>> input, boolean sortValues) throws Exception { Coder>>> sinkElemCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), KvCoder.of(BigEndianIntegerCoder.of(), BigEndianIntegerCoder.of())), @@ -200,7 +201,7 @@ private List writeShuffleEntries( ++kvCount; actualSizes.add( shuffleSinkWriter.add( - WindowedValue.of( + WindowedValues.of( KV.of(key, value), timestamp, Lists.newArrayList(window), @@ -336,7 +337,7 @@ private void runTestReadFromShuffle( ValuesToRead valuesToRead) throws Exception { Coder>>>> sourceElemCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of( @@ -478,7 +479,7 @@ private void runTestBytesReadCounterForOptions( TestShuffleReadCounterFactory shuffleReadCounterFactory = new TestShuffleReadCounterFactory(); Coder>>>> sourceElemCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of( @@ -637,7 +638,7 @@ public void testReadFromShuffleDataAndFailToSplit() throws Exception { null, null, stop, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), IntervalWindow.getCoder()), @@ -701,7 +702,7 @@ public void testConsumedParallelism() throws Exception { null, null, null, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), IntervalWindow.getCoder()), @@ -773,7 +774,7 @@ public void testReadFromShuffleAndDynamicSplit() throws Exception { null, null, null, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), IntervalWindow.getCoder()), @@ -879,7 +880,7 @@ public void testGetApproximateProgress() throws Exception { null, null, null, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), IntervalWindow.getCoder()), @@ -958,7 +959,7 @@ public void testShuffleReadCounterMultipleExecutingSteps() throws Exception { null, null, stop, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), IterableCoder.of(BigEndianIntegerCoder.of())), IntervalWindow.getCoder()), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactoryTest.java index 1d83594bb3f6..e77ae309d359 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IntrinsicMapTaskExecutorFactoryTest.java @@ -94,9 +94,9 @@ import org.apache.beam.sdk.util.DoFnInfo; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; 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.ImmutableSet; @@ -129,7 +129,7 @@ public class IntrinsicMapTaskExecutorFactoryTest { private static final CloudObject windowedStringCoder = CloudObjects.asCloudObject( - WindowedValue.getValueOnlyCoder(StringUtf8Coder.of()), /*sdkComponents=*/ null); + WindowedValues.getValueOnlyCoder(StringUtf8Coder.of()), /*sdkComponents=*/ null); private IntrinsicMapTaskExecutorFactory mapTaskExecutorFactory; private PipelineOptions options; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactoryTest.java index 09dba0fe5fd5..98e51963b10f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderFactoryTest.java @@ -37,8 +37,8 @@ import org.apache.beam.sdk.io.fs.ResourceId; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; 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.ImmutableMap; import org.junit.After; @@ -90,7 +90,7 @@ public void tearDown() throws IOException { @Test public void testFactory() throws Exception { WindowedValueCoder coder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( IsmRecordCoder.of( 1, 0, ImmutableList.>of(StringUtf8Coder.of()), VarLongCoder.of()), GlobalWindow.Coder.INSTANCE); @@ -114,7 +114,7 @@ public void testFactory() throws Exception { @Test public void testFactoryReturnsCachedInstance() throws Exception { Coder coder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( IsmRecordCoder.of( 1, 0, ImmutableList.>of(StringUtf8Coder.of()), VarLongCoder.of()), GlobalWindow.Coder.INSTANCE); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderTest.java index cabf7388875a..65061983c1bf 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmReaderTest.java @@ -70,7 +70,7 @@ import org.apache.beam.sdk.testing.CoderPropertiesTest.NonDeterministicCoder; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.WeightedValue; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicate; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicates; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java index 0ce92f9d932c..8420977dc47d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSideInputReaderTest.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.dataflow.worker; import static org.apache.beam.runners.dataflow.util.Structs.getString; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables.concat; import static org.hamcrest.MatcherAssert.assertThat; @@ -101,11 +101,12 @@ import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; 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.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableListMultimap; @@ -179,7 +180,7 @@ public void tearDown() throws IOException { @Test public void testSingleton() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); final WindowedValue element = valueInGlobalWindow(42L); final PCollectionView view = Pipeline.create().apply(Create.empty(VarLongCoder.of())).apply(View.asSingleton()); @@ -216,7 +217,7 @@ public void testSingleton() throws Exception { @Test public void testSingletonInWindow() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of(1, 0, ImmutableList.>of(INTERVAL_WINDOW_CODER), valueCoder); @@ -287,7 +288,7 @@ public void testSingletonMap() throws Exception { 1, 0, ImmutableList.>of(GLOBAL_WINDOW_CODER), - WindowedValue.getFullCoder(mapCoder, GLOBAL_WINDOW_CODER)); + WindowedValues.getFullCoder(mapCoder, GLOBAL_WINDOW_CODER)); final Source source = initInputFile(fromValues(Arrays.asList(element)), recordCoder); final IsmSideInputReader reader = sideInputReader(view.getTagInternal().getId(), source); @@ -323,14 +324,14 @@ public void testSingletonMapInWindow() throws Exception { ImmutableMap.>>builder() .put( firstWindow, - WindowedValue.of( + WindowedValues.of( ImmutableMap.builder().put("foo", 0L).put("bar", -1L).build(), new Instant(7), firstWindow, PaneInfo.NO_FIRING)) .put( secondWindow, - WindowedValue.of( + WindowedValues.of( ImmutableMap.builder().put("bar", -1L).put("baz", 1L).build(), new Instant(53L), secondWindow, @@ -349,7 +350,7 @@ public void testSingletonMapInWindow() throws Exception { 1, 0, ImmutableList.>of(INTERVAL_WINDOW_CODER), - WindowedValue.getFullCoder(mapCoder, INTERVAL_WINDOW_CODER)); + WindowedValues.getFullCoder(mapCoder, INTERVAL_WINDOW_CODER)); final Source source = initInputFile(fromValues(elements.values()), recordCoder); final IsmSideInputReader reader = sideInputReader(view.getTagInternal().getId(), source); @@ -402,7 +403,7 @@ public void testSingletonMultimapInWindow() throws Exception { ImmutableMap.>>>builder() .put( firstWindow, - WindowedValue.of( + WindowedValues.of( (Map) ImmutableListMultimap.builder() .put("foo", 0L) @@ -415,7 +416,7 @@ public void testSingletonMultimapInWindow() throws Exception { PaneInfo.NO_FIRING)) .put( secondWindow, - WindowedValue.of( + WindowedValues.of( (Map) ImmutableListMultimap.builder() .put("bar", -1L) @@ -441,7 +442,7 @@ public void testSingletonMultimapInWindow() throws Exception { 1, 0, ImmutableList.>of(INTERVAL_WINDOW_CODER), - WindowedValue.getFullCoder(mapCoder, INTERVAL_WINDOW_CODER)); + WindowedValues.getFullCoder(mapCoder, INTERVAL_WINDOW_CODER)); final Source source = initInputFile(fromValues(elements.values()), recordCoder); final IsmSideInputReader reader = sideInputReader(view.getTagInternal().getId(), source); @@ -486,7 +487,7 @@ public void testSingletonMultimapInWindow() throws Exception { @Test public void testIterable() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of( 1, 0, ImmutableList.of(GLOBAL_WINDOW_CODER, BigEndianLongCoder.of()), valueCoder); @@ -536,7 +537,7 @@ public void testIterable() throws Exception { @Test public void testIterableAtN() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of( 1, 0, ImmutableList.of(GLOBAL_WINDOW_CODER, BigEndianLongCoder.of()), valueCoder); @@ -588,7 +589,7 @@ public void testIterableAtN() throws Exception { @Test public void testIterableInWindow() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of( 1, 0, ImmutableList.of(INTERVAL_WINDOW_CODER, BigEndianLongCoder.of()), valueCoder); @@ -661,7 +662,7 @@ public void testIterableInWindow() throws Exception { @Test public void testList() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of( 1, 0, ImmutableList.of(GLOBAL_WINDOW_CODER, BigEndianLongCoder.of()), valueCoder); @@ -713,7 +714,7 @@ public void testList() throws Exception { @Test public void testListInWindow() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of( 1, 0, ImmutableList.of(INTERVAL_WINDOW_CODER, BigEndianLongCoder.of()), valueCoder); @@ -792,7 +793,7 @@ public void testMap() throws Exception { // Note that we purposely use byte[]s as keys to force structural equality testing // versus using java equality testing. Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); final ListMultimap> elements = ImmutableListMultimap.>builder() .put(new byte[] {0x00}, valueInGlobalWindow(12L)) @@ -874,7 +875,7 @@ public void testMapInWindow() throws Exception { // Note that we purposely use byte[]s as keys to force structural equality testing // versus using java equality testing. Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); final ListMultimap> firstWindow = ImmutableListMultimap.>builder() .put(new byte[] {0x00}, valueInIntervalWindow(12L, 10)) @@ -1000,7 +1001,7 @@ public void testMultimap() throws Exception { // Note that we purposely use byte[]s as keys to force structural equality testing // versus using java equality testing. Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); final ListMultimap> elements = ImmutableListMultimap.>builder() .put(new byte[] {0x00}, valueInGlobalWindow(12L)) @@ -1083,7 +1084,7 @@ public void testMultimapInWindow() throws Exception { // Note that we purposely use byte[]s as keys to force structural equality testing // versus using java equality testing. Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); final ListMultimap> firstWindow = ImmutableListMultimap.>builder() .put(new byte[] {0x00}, valueInIntervalWindow(12L, 10)) @@ -1211,7 +1212,7 @@ public void testMultimapViewInWindow() throws Exception { // the multimap, we specifically use the same instance of the byte[]. byte[] duplicateKey = new byte[] {0x01}; Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), INTERVAL_WINDOW_CODER); final ListMultimap> firstWindow = ImmutableListMultimap.>builder() .put(new byte[] {0x00}, valueInIntervalWindow(12L, 10)) @@ -1349,7 +1350,7 @@ public void testIterableSideInputReadCounter() throws Exception { // Test startup: Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); IsmRecordCoder> ismCoder = IsmRecordCoder.of( 1, 0, ImmutableList.of(GLOBAL_WINDOW_CODER, BigEndianLongCoder.of()), valueCoder); @@ -1398,7 +1399,7 @@ public void testIterableSideInputReadCounter() throws Exception { @Test public void testIsmReaderReferenceCaching() throws Exception { Coder> valueCoder = - WindowedValue.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); + WindowedValues.getFullCoder(VarLongCoder.of(), GLOBAL_WINDOW_CODER); final WindowedValue element = valueInGlobalWindow(42L); final PCollectionView view = Pipeline.create().apply(Create.empty(VarLongCoder.of())).apply(View.asSingleton()); @@ -1626,7 +1627,7 @@ private static void verifyMap( } WindowedValue valueInIntervalWindow(long value, long startOfWindow) { - return WindowedValue.of( + return WindowedValues.of( value, new Instant(startOfWindow), intervalWindow(startOfWindow), PaneInfo.NO_FIRING); } @@ -1772,7 +1773,7 @@ private Source newIsmSource(IsmRecordCoder> coder, Strin Source source = new Source(); source.setCodec( CloudObjects.asCloudObject( - WindowedValue.getFullCoder(coder, GLOBAL_WINDOW_CODER), /*sdkComponents=*/ null)); + WindowedValues.getFullCoder(coder, GLOBAL_WINDOW_CODER), /*sdkComponents=*/ null)); source.setSpec(new HashMap()); source.getSpec().put(PropertyNames.OBJECT_TYPE_NAME, "IsmSource"); source.getSpec().put(WorkerPropertyNames.FILENAME, tmpFilePath); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSinkTest.java index d395ec8acf6d..78c393111c34 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/IsmSinkTest.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.testing.CoderPropertiesTest.NonDeterministicCoder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Rule; import org.junit.Test; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactoryTest.java index be1ed3c270c4..fa8ca0e3dd91 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PairWithConstantKeyDoFnFactoryTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow.worker; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java index 19827432e134..19af6effb44d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartialGroupByKeyParDoFnsTest.java @@ -66,10 +66,11 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObserver; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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.ImmutableSet; @@ -114,7 +115,7 @@ public void testPartialGroupByKey() throws Exception { TestOutputReceiver receiver = new TestOutputReceiver( new ElementByteSizeObservableCoder( - WindowedValue.getValueOnlyCoder( + WindowedValues.getValueOnlyCoder( KvCoder.of(keyCoder, IterableCoder.of(valueCoder)))), counterSet, NameContextsForTests.nameContextForTest()); @@ -124,27 +125,27 @@ public void testPartialGroupByKey() throws Exception { GroupingTables.buffering( new WindowingCoderGroupingKeyCreator(keyCoder), PairInfo.create(), - new CoderSizeEstimator(WindowedValue.getValueOnlyCoder(keyCoder)), + new CoderSizeEstimator(WindowedValues.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator(valueCoder)), receiver); pgbkParDoFn.startBundle(receiver); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 4))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("there", 5))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 6))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("joe", 7))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("there", 8))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 9))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 4))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("there", 5))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 6))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("joe", 7))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("there", 8))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 9))); pgbkParDoFn.finishBundle(); assertThat( receiver.outputElems, IsIterableContainingInAnyOrder.containsInAnyOrder( - WindowedValue.valueInGlobalWindow(KV.of("hi", Arrays.asList(4, 6, 9))), - WindowedValue.valueInGlobalWindow(KV.of("there", Arrays.asList(5, 8))), - WindowedValue.valueInGlobalWindow(KV.of("joe", Arrays.asList(7))))); + WindowedValues.valueInGlobalWindow(KV.of("hi", Arrays.asList(4, 6, 9))), + WindowedValues.valueInGlobalWindow(KV.of("there", Arrays.asList(5, 8))), + WindowedValues.valueInGlobalWindow(KV.of("joe", Arrays.asList(7))))); // Exact counter values depend on size of encoded data. If encoding // changes, then these expected counters should change to match. @@ -167,7 +168,7 @@ public void testPartialGroupByKeyWithCombiner() throws Exception { TestOutputReceiver receiver = new TestOutputReceiver( new ElementByteSizeObservableCoder( - WindowedValue.getValueOnlyCoder(KvCoder.of(keyCoder, valueCoder))), + WindowedValues.getValueOnlyCoder(KvCoder.of(keyCoder, valueCoder))), counterSet, NameContextsForTests.nameContextForTest()); @@ -179,27 +180,27 @@ public void testPartialGroupByKeyWithCombiner() throws Exception { new WindowingCoderGroupingKeyCreator(keyCoder), PairInfo.create(), combineFn, - new CoderSizeEstimator(WindowedValue.getValueOnlyCoder(keyCoder)), + new CoderSizeEstimator(WindowedValues.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator(valueCoder)), receiver); pgbkParDoFn.startBundle(receiver); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 4))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("there", 5))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 6))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("joe", 7))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("there", 8))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 9))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 4))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("there", 5))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 6))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("joe", 7))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("there", 8))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 9))); pgbkParDoFn.finishBundle(); assertThat( receiver.outputElems, IsIterableContainingInAnyOrder.containsInAnyOrder( - WindowedValue.valueInGlobalWindow(KV.of("hi", 19)), - WindowedValue.valueInGlobalWindow(KV.of("there", 13)), - WindowedValue.valueInGlobalWindow(KV.of("joe", 7)))); + WindowedValues.valueInGlobalWindow(KV.of("hi", 19)), + WindowedValues.valueInGlobalWindow(KV.of("there", 13)), + WindowedValues.valueInGlobalWindow(KV.of("joe", 7)))); // Exact counter values depend on size of encoded data. If encoding // changes, then these expected counters should change to match. @@ -222,7 +223,7 @@ public void testPartialGroupByKeyWithCombinerAndSideInputs() throws Exception { TestOutputReceiver receiver = new TestOutputReceiver( new ElementByteSizeObservableCoder( - WindowedValue.getValueOnlyCoder(KvCoder.of(keyCoder, valueCoder))), + WindowedValues.getValueOnlyCoder(KvCoder.of(keyCoder, valueCoder))), counterSet, NameContextsForTests.nameContextForTest()); @@ -234,7 +235,7 @@ public void testPartialGroupByKeyWithCombinerAndSideInputs() throws Exception { new WindowingCoderGroupingKeyCreator(keyCoder), PairInfo.create(), combineFn, - new CoderSizeEstimator(WindowedValue.getValueOnlyCoder(keyCoder)), + new CoderSizeEstimator(WindowedValues.getValueOnlyCoder(keyCoder)), new CoderSizeEstimator(valueCoder)), receiver, mockSideInputFetcher); @@ -246,27 +247,27 @@ public void testPartialGroupByKeyWithCombinerAndSideInputs() throws Exception { when(elemsBag.read()) .thenReturn( ImmutableList.of( - WindowedValue.valueInGlobalWindow(KV.of("hi", 4)), - WindowedValue.valueInGlobalWindow(KV.of("there", 5)))); + WindowedValues.valueInGlobalWindow(KV.of("hi", 4)), + WindowedValues.valueInGlobalWindow(KV.of("there", 5)))); when(mockSideInputFetcher.storeIfBlocked( ArgumentMatchers.>>any())) .thenReturn(false, false, false, true); pgbkParDoFn.startBundle(receiver); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 6))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("joe", 7))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("there", 8))); - pgbkParDoFn.processElement(WindowedValue.valueInGlobalWindow(KV.of("hi", 9))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 6))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("joe", 7))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("there", 8))); + pgbkParDoFn.processElement(WindowedValues.valueInGlobalWindow(KV.of("hi", 9))); pgbkParDoFn.finishBundle(); assertThat( receiver.outputElems, IsIterableContainingInAnyOrder.containsInAnyOrder( - WindowedValue.valueInGlobalWindow(KV.of("hi", 10)), - WindowedValue.valueInGlobalWindow(KV.of("there", 13)), - WindowedValue.valueInGlobalWindow(KV.of("joe", 7)))); + WindowedValues.valueInGlobalWindow(KV.of("hi", 10)), + WindowedValues.valueInGlobalWindow(KV.of("there", 13)), + WindowedValues.valueInGlobalWindow(KV.of("joe", 7)))); // Exact counter values depend on size of encoded data. If encoding // changes, then these expected counters should change to match. @@ -291,7 +292,7 @@ public void testCreateWithCombinerAndBatchSideInputs() throws Exception { TestOutputReceiver receiver = new TestOutputReceiver( - new ElementByteSizeObservableCoder(WindowedValue.getValueOnlyCoder(kvCoder)), + new ElementByteSizeObservableCoder(WindowedValues.getValueOnlyCoder(kvCoder)), counterSet, NameContextsForTests.nameContextForTest()); @@ -328,7 +329,7 @@ public void testCreateWithCombinerAndStreaming() throws Exception { TestOutputReceiver receiver = new TestOutputReceiver( - new ElementByteSizeObservableCoder(WindowedValue.getValueOnlyCoder(kvCoder)), + new ElementByteSizeObservableCoder(WindowedValues.getValueOnlyCoder(kvCoder)), counterSet, NameContextsForTests.nameContextForTest()); @@ -355,7 +356,7 @@ public void testCreateWithCombinerAndStreamingSideInputs() throws Exception { TestOutputReceiver receiver = new TestOutputReceiver( - new ElementByteSizeObservableCoder(WindowedValue.getValueOnlyCoder(kvCoder)), + new ElementByteSizeObservableCoder(WindowedValues.getValueOnlyCoder(kvCoder)), counterSet, NameContextsForTests.nameContextForTest()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java index 864346278da6..9fe44e5789eb 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PartitioningShuffleReaderTest.java @@ -34,8 +34,9 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Duration; import org.joda.time.Instant; @@ -55,21 +56,21 @@ public class PartitioningShuffleReaderTest { private static final List>> KVS = Arrays.asList( - WindowedValue.of(KV.of(1, "in 1a"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(1, "in 1b"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(2, "in 2a"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(2, "in 2b"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(3, "in 3"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(4, "in 4a"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(4, "in 4b"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(4, "in 4c"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(4, "in 4d"), timestamp, Lists.newArrayList(window), NO_FIRING), - WindowedValue.of(KV.of(5, "in 5"), timestamp, Lists.newArrayList(window), NO_FIRING)); + WindowedValues.of(KV.of(1, "in 1a"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(1, "in 1b"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(2, "in 2a"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(2, "in 2b"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(3, "in 3"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(4, "in 4a"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(4, "in 4b"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(4, "in 4c"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(4, "in 4d"), timestamp, Lists.newArrayList(window), NO_FIRING), + WindowedValues.of(KV.of(5, "in 5"), timestamp, Lists.newArrayList(window), NO_FIRING)); private void runTestReadFromShuffle(List>> expected) throws Exception { Coder>> elemCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(BigEndianIntegerCoder.of(), StringUtf8Coder.of()), IntervalWindow.getCoder()); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java index e258dbe1d85c..d8822ce4937b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubDynamicSinkTest.java @@ -32,7 +32,8 @@ import org.apache.beam.sdk.coders.VoidCoder; import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Instant; @@ -73,7 +74,7 @@ public void testWriteDynamicDestinations() throws Exception { SinkRegistry.defaultRegistry() .create( cloudSinkSpec, - WindowedValue.getFullCoder(VoidCoder.of(), IntervalWindow.getCoder()), + WindowedValues.getFullCoder(VoidCoder.of(), IntervalWindow.getCoder()), null, mockContext, null) @@ -119,14 +120,14 @@ public void testWriteDynamicDestinations() throws Exception { .toByteString()) .build()); writer.add( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( new PubsubMessage(payload1, null).withTopic("topic1"), new Instant(baseTimestamp))); writer.add( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( new PubsubMessage(payload2, null).withTopic("topic2"), new Instant(baseTimestamp + 1))); writer.add( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( new PubsubMessage(payload3, null).withTopic("topic3"), new Instant(baseTimestamp + 2))); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java index 4af9555b3bb0..c28fdac04888 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubReaderTest.java @@ -30,7 +30,8 @@ import org.apache.beam.runners.dataflow.worker.windmill.Windmill; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.joda.time.Instant; import org.junit.Before; @@ -84,7 +85,7 @@ private void testReadWith(String parseFn) throws Exception { (PubsubReader) factory.create( cloudSourceSpec, - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()), + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()), null, mockContext, null); @@ -92,13 +93,13 @@ private void testReadWith(String parseFn) throws Exception { NativeReader.NativeReaderIterator> iter = reader.iterator(); assertTrue(iter.start()); assertEquals( - iter.getCurrent(), WindowedValue.timestampedValueInGlobalWindow("e0", new Instant(0))); + iter.getCurrent(), WindowedValues.timestampedValueInGlobalWindow("e0", new Instant(0))); assertTrue(iter.advance()); assertEquals( - iter.getCurrent(), WindowedValue.timestampedValueInGlobalWindow("e1", new Instant(1))); + iter.getCurrent(), WindowedValues.timestampedValueInGlobalWindow("e1", new Instant(1))); assertTrue(iter.advance()); assertEquals( - iter.getCurrent(), WindowedValue.timestampedValueInGlobalWindow("e2", new Instant(2))); + iter.getCurrent(), WindowedValues.timestampedValueInGlobalWindow("e2", new Instant(2))); assertFalse(iter.advance()); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java index a03f7bc24a6d..5327cd172410 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/PubsubSinkTest.java @@ -35,7 +35,8 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.joda.time.Instant; import org.junit.Before; @@ -77,16 +78,19 @@ private void testWriteWith(String formatFn) throws Exception { (PubsubSink) factory.create( cloudSinkSpec, - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()), + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()), null, mockContext, null); Sink.SinkWriter> writer = sink.writer(); - assertEquals(2, writer.add(WindowedValue.timestampedValueInGlobalWindow("e0", new Instant(0)))); - assertEquals(2, writer.add(WindowedValue.timestampedValueInGlobalWindow("e1", new Instant(1)))); - assertEquals(2, writer.add(WindowedValue.timestampedValueInGlobalWindow("e2", new Instant(2)))); + assertEquals( + 2, writer.add(WindowedValues.timestampedValueInGlobalWindow("e0", new Instant(0)))); + assertEquals( + 2, writer.add(WindowedValues.timestampedValueInGlobalWindow("e1", new Instant(1)))); + assertEquals( + 2, writer.add(WindowedValues.timestampedValueInGlobalWindow("e2", new Instant(2)))); writer.close(); assertEquals( @@ -161,7 +165,7 @@ public void testExceptionAfterEncoding() throws Exception { (PubsubSink) factory.create( cloudSinkSpec, - WindowedValue.getFullCoder(new ErrorCoder(), IntervalWindow.getCoder()), + WindowedValues.getFullCoder(new ErrorCoder(), IntervalWindow.getCoder()), null, mockContext, null); @@ -170,10 +174,10 @@ public void testExceptionAfterEncoding() throws Exception { assertThrows( "encode error", CoderException.class, - () -> writer.add(WindowedValue.timestampedValueInGlobalWindow("e0", new Instant(0)))); + () -> writer.add(WindowedValues.timestampedValueInGlobalWindow("e0", new Instant(0)))); assertThrows( "encode error", CoderException.class, - () -> writer.add(WindowedValue.timestampedValueInGlobalWindow("e0", new Instant(0)))); + () -> writer.add(WindowedValues.timestampedValueInGlobalWindow("e0", new Instant(0)))); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderFactoryTest.java index ea3d02a099e1..3af59d67e3dd 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderFactoryTest.java @@ -30,7 +30,8 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.checkerframework.checker.nullness.qual.Nullable; import org.hamcrest.CoreMatchers; import org.hamcrest.core.IsInstanceOf; @@ -115,7 +116,7 @@ public boolean advance() throws IOException { @Override public WindowedValue getCurrent() { - return WindowedValue.valueInGlobalWindow("something"); + return WindowedValues.valueInGlobalWindow("something"); } } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderTestUtils.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderTestUtils.java index 343661ae9a6b..473663901c29 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderTestUtils.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReaderTestUtils.java @@ -31,7 +31,7 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader.NativeReaderIterator; import org.apache.beam.runners.dataflow.worker.util.common.worker.NativeReader.Progress; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; /** diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java index 55b8d62261c7..eb8f91af18b9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ReifyTimestampAndWindowsParDoFnFactoryTest.java @@ -25,8 +25,9 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.Receiver; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; @@ -61,7 +62,7 @@ private void verifyReifiedIsInTheSameWindows(WindowedValue> elem @Test public void testSingleWindow() throws Exception { verifyReifiedIsInTheSameWindows( - WindowedValue.of( + WindowedValues.of( KV.of(42, "bizzle"), new Instant(73), new IntervalWindow(new Instant(5), new Instant(15)), @@ -71,7 +72,7 @@ public void testSingleWindow() throws Exception { @Test public void testMultiWindowStaysCompressed() throws Exception { verifyReifiedIsInTheSameWindows( - WindowedValue.of( + WindowedValues.of( KV.of(42, "bizzle"), new Instant(73), ImmutableList.of( diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleReaderFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleReaderFactoryTest.java index febf8b4380f9..c1d87fb6f78c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleReaderFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleReaderFactoryTest.java @@ -34,8 +34,8 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.checkerframework.checker.nullness.qual.Nullable; import org.hamcrest.core.IsInstanceOf; import org.junit.Assert; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactoryTest.java index 0edbf80b4ee1..03a2b28090dc 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkFactoryTest.java @@ -31,8 +31,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.hamcrest.core.IsInstanceOf; import org.junit.Assert; import org.junit.Test; @@ -91,7 +91,7 @@ void runTestCreateUngroupingShuffleSink( void runTestCreatePartitioningShuffleSink( byte[] shuffleWriterConfig, Coder keyCoder, Coder valueCoder) throws Exception { FullWindowedValueCoder coder = - WindowedValue.getFullCoder(KvCoder.of(keyCoder, valueCoder), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(KvCoder.of(keyCoder, valueCoder), IntervalWindow.getCoder()); ShuffleSink shuffleSink = runTestCreateShuffleSinkHelper(shuffleWriterConfig, "partition_keys", coder, coder); Assert.assertEquals(ShuffleSink.ShuffleKind.PARTITION_KEYS, shuffleSink.shuffleKind); @@ -110,7 +110,7 @@ void runTestCreatePartitioningShuffleSink( void runTestCreateGroupingShuffleSink( byte[] shuffleWriterConfig, Coder keyCoder, Coder valueCoder) throws Exception { FullWindowedValueCoder coder = - WindowedValue.getFullCoder(KvCoder.of(keyCoder, valueCoder), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(KvCoder.of(keyCoder, valueCoder), IntervalWindow.getCoder()); ShuffleSink shuffleSink = runTestCreateShuffleSinkHelper(shuffleWriterConfig, "group_keys", coder, coder); Assert.assertEquals(ShuffleSink.ShuffleKind.GROUP_KEYS, shuffleSink.shuffleKind); @@ -128,7 +128,7 @@ void runTestCreateGroupingSortingShuffleSink( byte[] shuffleWriterConfig, Coder keyCoder, Coder sortKeyCoder, Coder sortValueCoder) throws Exception { FullWindowedValueCoder coder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(keyCoder, KvCoder.of(sortKeyCoder, sortValueCoder)), IntervalWindow.getCoder()); ShuffleSink shuffleSink = @@ -149,7 +149,7 @@ void runTestCreateGroupingSortingShuffleSink( @Test public void testCreateUngroupingShuffleSink() throws Exception { FullWindowedValueCoder coder = - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); runTestCreateUngroupingShuffleSink(new byte[] {(byte) 0xE1}, coder, coder); } @@ -164,7 +164,7 @@ public void testCreateGroupingShuffleSink() throws Exception { runTestCreateGroupingShuffleSink( new byte[] {(byte) 0xE2}, BigEndianIntegerCoder.of(), - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder())); + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder())); } @Test diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java index 6db4562847c1..025eb7c2fbb1 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ShuffleSinkTest.java @@ -36,8 +36,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Duration; @@ -87,7 +88,7 @@ public class ShuffleSinkTest { private void runTestWriteUngroupingShuffleSink(List expected) throws Exception { Coder> windowedValueCoder = - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), new GlobalWindows().windowCoder()); + WindowedValues.getFullCoder(BigEndianIntegerCoder.of(), new GlobalWindows().windowCoder()); BatchModeExecutionContext executionContext = BatchModeExecutionContext.forTesting(PipelineOptionsFactory.create(), "STAGE"); ShuffleSink shuffleSink = @@ -104,7 +105,7 @@ private void runTestWriteUngroupingShuffleSink(List expected) throws Ex try (Sink.SinkWriter> shuffleSinkWriter = shuffleSink.writer(shuffleWriter, "dataset")) { for (Integer value : expected) { - actualSizes.add(shuffleSinkWriter.add(WindowedValue.valueInGlobalWindow(value))); + actualSizes.add(shuffleSinkWriter.add(WindowedValues.valueInGlobalWindow(value))); } } @@ -132,7 +133,7 @@ void runTestWriteGroupingShuffleSink(List> expected) throws PipelineOptionsFactory.create(), null, ShuffleSink.ShuffleKind.GROUP_KEYS, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(BigEndianIntegerCoder.of(), StringUtf8Coder.of()), IntervalWindow.getCoder()), executionContext, @@ -145,7 +146,7 @@ void runTestWriteGroupingShuffleSink(List> expected) throws for (KV kv : expected) { actualSizes.add( shuffleSinkWriter.add( - WindowedValue.of( + WindowedValues.of( KV.of(kv.getKey(), kv.getValue()), timestamp, Lists.newArrayList(window), @@ -181,7 +182,7 @@ void runTestWriteGroupingSortingShuffleSink(List PipelineOptionsFactory.create(), null, ShuffleSink.ShuffleKind.GROUP_KEYS_AND_SORT_VALUES, - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( BigEndianIntegerCoder.of(), KvCoder.of(StringUtf8Coder.of(), BigEndianIntegerCoder.of())), @@ -194,7 +195,7 @@ void runTestWriteGroupingSortingShuffleSink(List try (Sink.SinkWriter>>> shuffleSinkWriter = shuffleSink.writer(shuffleWriter, "dataset")) { for (KV> kv : expected) { - actualSizes.add(shuffleSinkWriter.add(WindowedValue.valueInGlobalWindow(kv))); + actualSizes.add(shuffleSinkWriter.add(WindowedValues.valueInGlobalWindow(kv))); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnTest.java index 76cc74673fb0..bb92fca3d8be 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/SimpleParDoFnTest.java @@ -55,9 +55,9 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.DoFnInfo; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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.ImmutableMap; @@ -236,41 +236,41 @@ public void testOutputReceivers() throws Exception { userParDoFn.startBundle(receiver, receiver1, receiver2, receiver3); - userParDoFn.processElement(WindowedValue.valueInGlobalWindow(3)); - userParDoFn.processElement(WindowedValue.valueInGlobalWindow(42)); - userParDoFn.processElement(WindowedValue.valueInGlobalWindow(666)); + userParDoFn.processElement(WindowedValues.valueInGlobalWindow(3)); + userParDoFn.processElement(WindowedValues.valueInGlobalWindow(42)); + userParDoFn.processElement(WindowedValues.valueInGlobalWindow(666)); userParDoFn.finishBundle(); Object[] expectedReceivedElems = { - WindowedValue.valueInGlobalWindow("processing: 3"), - WindowedValue.valueInGlobalWindow("processing: 42"), - WindowedValue.valueInGlobalWindow("processing: 666"), - WindowedValue.valueInGlobalWindow("finished"), + WindowedValues.valueInGlobalWindow("processing: 3"), + WindowedValues.valueInGlobalWindow("processing: 42"), + WindowedValues.valueInGlobalWindow("processing: 666"), + WindowedValues.valueInGlobalWindow("finished"), }; assertArrayEquals(expectedReceivedElems, receiver.receivedElems.toArray()); Object[] expectedReceivedElems1 = { - WindowedValue.valueInGlobalWindow("tag1: processing: 3"), - WindowedValue.valueInGlobalWindow("tag1: processing: 42"), - WindowedValue.valueInGlobalWindow("tag1: processing: 666"), - WindowedValue.valueInGlobalWindow("tag1: finished"), + WindowedValues.valueInGlobalWindow("tag1: processing: 3"), + WindowedValues.valueInGlobalWindow("tag1: processing: 42"), + WindowedValues.valueInGlobalWindow("tag1: processing: 666"), + WindowedValues.valueInGlobalWindow("tag1: finished"), }; assertArrayEquals(expectedReceivedElems1, receiver1.receivedElems.toArray()); Object[] expectedReceivedElems2 = { - WindowedValue.valueInGlobalWindow("tag2: processing: 3"), - WindowedValue.valueInGlobalWindow("tag2: processing: 42"), - WindowedValue.valueInGlobalWindow("tag2: processing: 666"), - WindowedValue.valueInGlobalWindow("tag2: finished"), + WindowedValues.valueInGlobalWindow("tag2: processing: 3"), + WindowedValues.valueInGlobalWindow("tag2: processing: 42"), + WindowedValues.valueInGlobalWindow("tag2: processing: 666"), + WindowedValues.valueInGlobalWindow("tag2: finished"), }; assertArrayEquals(expectedReceivedElems2, receiver2.receivedElems.toArray()); Object[] expectedReceivedElems3 = { - WindowedValue.valueInGlobalWindow("tag3: processing: 3"), - WindowedValue.valueInGlobalWindow("tag3: processing: 42"), - WindowedValue.valueInGlobalWindow("tag3: processing: 666"), - WindowedValue.valueInGlobalWindow("tag3: finished"), + WindowedValues.valueInGlobalWindow("tag3: processing: 3"), + WindowedValues.valueInGlobalWindow("tag3: processing: 42"), + WindowedValues.valueInGlobalWindow("tag3: processing: 666"), + WindowedValues.valueInGlobalWindow("tag3: finished"), }; assertArrayEquals(expectedReceivedElems3, receiver3.receivedElems.toArray()); } @@ -381,7 +381,7 @@ public void testErrorPropagation() throws Exception { } try { - userParDoFn.processElement(WindowedValue.valueInGlobalWindow(3)); + userParDoFn.processElement(WindowedValues.valueInGlobalWindow(3)); fail("should have failed"); } catch (Exception exn) { // Exception should be a UserCodeException since we're calling @@ -453,7 +453,7 @@ public void testUndeclaredSideOutputs() throws Exception { thrown.expect(UserCodeException.class); thrown.expectCause(instanceOf(IllegalArgumentException.class)); thrown.expectMessage("Unknown output tag"); - userParDoFn.processElement(WindowedValue.valueInGlobalWindow(5)); + userParDoFn.processElement(WindowedValues.valueInGlobalWindow(5)); } @Test @@ -513,7 +513,7 @@ public void processElement(ProcessContext c) throws Exception { // This test ensures proper behavior of the state sampling even with lazy initialization. try (Closeable trackerCloser = tracker.activate()) { try (Closeable processCloser = operationContext.enterProcess()) { - userParDoFn.processElement(WindowedValue.valueInGlobalWindow(5)); + userParDoFn.processElement(WindowedValues.valueInGlobalWindow(5)); } } } @@ -603,7 +603,7 @@ public void processElement(ProcessContext c) { parDoFn.startBundle(new TestReceiver()); for (int input : inputData) { - parDoFn.processElement(WindowedValue.valueInGlobalWindow(input)); + parDoFn.processElement(WindowedValues.valueInGlobalWindow(input)); } return operationContext diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java index 0cf1bb330dad..11dd51f7fb3a 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingDataflowWorkerTest.java @@ -169,14 +169,15 @@ import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; import org.apache.beam.sdk.util.VarInt; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.sdk.values.WindowingStrategy.AccumulationMode; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; @@ -434,7 +435,7 @@ private ParallelInstruction makeSourceInstruction(Coder coder) { .setSpec(CloudObject.forClass(UngroupedWindmillReader.class)) .setCodec( CloudObjects.asCloudObject( - WindowedValue.getFullCoder(coder, IntervalWindow.getCoder()), + WindowedValues.getFullCoder(coder, IntervalWindow.getCoder()), /* sdkComponents= */ null)))) .setOutputs( Collections.singletonList( @@ -444,7 +445,7 @@ private ParallelInstruction makeSourceInstruction(Coder coder) { .setSystemName(DEFAULT_OUTPUT_SYSTEM_NAME) .setCodec( CloudObjects.asCloudObject( - WindowedValue.getFullCoder(coder, IntervalWindow.getCoder()), + WindowedValues.getFullCoder(coder, IntervalWindow.getCoder()), /* sdkComponents= */ null)))); } @@ -489,7 +490,7 @@ private ParallelInstruction makeDoFnInstruction( .setSystemName(DEFAULT_OUTPUT_SYSTEM_NAME) .setCodec( CloudObjects.asCloudObject( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( outputCoder, windowingStrategy.getWindowFn().windowCoder()), /* sdkComponents= */ null)))); } @@ -526,7 +527,7 @@ private ParallelInstruction makeSinkInstruction( .setSpec(spec) .setCodec( CloudObjects.asCloudObject( - WindowedValue.getFullCoder(coder, windowCoder), + WindowedValues.getFullCoder(coder, windowCoder), /* sdkComponents= */ null)))); } @@ -1562,7 +1563,7 @@ public void testAssignWindows() throws Exception { .setName("output") .setCodec( CloudObjects.asCloudObject( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( StringUtf8Coder.of(), IntervalWindow.getCoder()), /* sdkComponents= */ null)))); @@ -2401,7 +2402,7 @@ private List makeUnboundedSourcePipeline( options.setNumWorkers(1); CloudObject codec = CloudObjects.asCloudObject( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( ValueWithRecordId.ValueWithRecordIdCoder.of( KvCoder.of(VarIntCoder.of(), VarIntCoder.of())), GlobalWindow.Coder.INSTANCE), @@ -3241,7 +3242,7 @@ public void testExceptionInvalidatesCache() throws Exception { CloudObject codec = CloudObjects.asCloudObject( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( ValueWithRecordId.ValueWithRecordIdCoder.of( KvCoder.of(VarIntCoder.of(), VarIntCoder.of())), GlobalWindow.Coder.INSTANCE), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java index 5ae4ab020f20..c27b386a76c8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowFnsTest.java @@ -72,9 +72,9 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.util.AppliedCombineFn; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.joda.time.Duration; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java index 2069899aa699..3574aa22b70d 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingGroupAlsoByWindowsReshuffleDoFnTest.java @@ -44,9 +44,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.hamcrest.Matchers; import org.joda.time.Instant; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java index 9d850a8bd935..c400c686f764 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingKeyedWorkItemSideInputDoFnRunnerTest.java @@ -50,9 +50,10 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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.ImmutableSet; @@ -156,7 +157,7 @@ public void testStartBundle() throws Exception { private WindowedValue createDatum(T element, long timestampMillis) { Instant timestamp = new Instant(timestampMillis); - return WindowedValue.of( + return WindowedValues.of( element, timestamp, Arrays.asList(WINDOW_FN.assignWindow(timestamp)), PaneInfo.NO_FIRING); } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactoryTest.java index 374d4ac6308d..fcf7e4d7deb9 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingPCollectionViewWriterDoFnFactoryTest.java @@ -29,7 +29,7 @@ import org.apache.beam.runners.dataflow.worker.util.common.worker.ParDoFn; import org.apache.beam.sdk.coders.BigEndianIntegerCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Test; import org.junit.runner.RunWith; @@ -52,7 +52,7 @@ public void testConstruction() throws Exception { CloudObject coder = CloudObjects.asCloudObject( - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE), /*sdkComponents=*/ null); ParDoFn parDoFn = new StreamingPCollectionViewWriterDoFnFactory() diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java index 2ea8e0bcd9de..9a6544122c69 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputDoFnRunnerTest.java @@ -59,9 +59,10 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -207,7 +208,7 @@ public void testMultipleWindowsNotReady() throws Exception { long timestamp = 1L; WindowedValue elem = - WindowedValue.of( + WindowedValues.of( "e", new Instant(timestamp), Arrays.asList(window1, window2), PaneInfo.NO_FIRING); runner.startBundle(); @@ -449,7 +450,7 @@ private PCollectionView createView() { } private WindowedValue createDatum(String element, long timestamp) { - return WindowedValue.of( + return WindowedValues.of( element, new Instant(timestamp), Arrays.asList(createWindow(timestamp)), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java index 0aea96f2f906..0f5cd1a0d233 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/StreamingSideInputFetcherTest.java @@ -45,9 +45,10 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -197,7 +198,7 @@ private PCollectionView createView() { } private WindowedValue createDatum(String element, long timestamp) { - return WindowedValue.of( + return WindowedValues.of( element, new Instant(timestamp), Arrays.asList(createWindow(timestamp)), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactoryTest.java index 02ca8e923a23..006d61493c08 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ToIsmRecordForMultimapDoFnFactoryTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow.worker; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java index 1e504d97b81d..5e8fe648e7d7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UngroupedShuffleReaderTest.java @@ -30,7 +30,8 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.joda.time.Duration; import org.joda.time.Instant; @@ -48,7 +49,7 @@ public class UngroupedShuffleReaderTest { void runTestReadFromShuffle(List expected) throws Exception { Coder> elemCoder = - WindowedValue.getFullCoder(BigEndianIntegerCoder.of(), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(BigEndianIntegerCoder.of(), IntervalWindow.getCoder()); BatchModeExecutionContext executionContext = BatchModeExecutionContext.forTesting(PipelineOptionsFactory.create(), "STAGE"); @@ -70,7 +71,7 @@ void runTestReadFromShuffle(List expected) throws Exception { for (Integer value : expected) { actualSizes.add( shuffleSinkWriter.add( - WindowedValue.of( + WindowedValues.of( value, timestamp, Lists.newArrayList(window), PaneInfo.NO_FIRING))); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java index 17f48e71581d..771ffe192baf 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/UserParDoFnFactoryTest.java @@ -65,9 +65,9 @@ import org.apache.beam.sdk.util.DoFnInfo; import org.apache.beam.sdk.util.SerializableUtils; import org.apache.beam.sdk.util.StringUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.joda.time.Duration; @@ -193,7 +193,7 @@ public void testFactoryReuseInStep() throws Exception { Receiver rcvr = new OutputReceiver(); parDoFn.startBundle(rcvr); - parDoFn.processElement(WindowedValue.valueInGlobalWindow("foo")); + parDoFn.processElement(WindowedValues.valueInGlobalWindow("foo")); TestDoFn fn = (TestDoFn) ((SimpleParDoFn) parDoFn).getDoFnInfo().getDoFn(); assertThat(fn, not(theInstance(initialFn))); @@ -215,7 +215,7 @@ public void testFactoryReuseInStep() throws Exception { assertThat(fn.state, equalTo(TestDoFn.State.FINISHED)); secondParDoFn.startBundle(rcvr); - secondParDoFn.processElement(WindowedValue.valueInGlobalWindow("spam")); + secondParDoFn.processElement(WindowedValues.valueInGlobalWindow("spam")); TestDoFn reobtainedFn = (TestDoFn) ((SimpleParDoFn) secondParDoFn).getDoFnInfo().getDoFn(); secondParDoFn.finishBundle(); assertThat(reobtainedFn.state, equalTo(TestDoFn.State.FINISHED)); @@ -252,13 +252,13 @@ public void testFactorySimultaneousUse() throws Exception { Receiver rcvr = new OutputReceiver(); parDoFn.startBundle(rcvr); - parDoFn.processElement(WindowedValue.valueInGlobalWindow("foo")); + parDoFn.processElement(WindowedValues.valueInGlobalWindow("foo")); // Must be after the first call to process element for reallyStartBundle to have been called TestDoFn firstDoFn = (TestDoFn) ((SimpleParDoFn) parDoFn).getDoFnInfo().getDoFn(); secondParDoFn.startBundle(rcvr); - secondParDoFn.processElement(WindowedValue.valueInGlobalWindow("spam")); + secondParDoFn.processElement(WindowedValues.valueInGlobalWindow("spam")); // Must be after the first call to process element for reallyStartBundle to have been called TestDoFn secondDoFn = (TestDoFn) ((SimpleParDoFn) secondParDoFn).getDoFnInfo().getDoFn(); @@ -290,7 +290,7 @@ public void testFactoryDoesNotReuseAfterAborted() throws Exception { Receiver rcvr = new OutputReceiver(); parDoFn.startBundle(rcvr); - parDoFn.processElement(WindowedValue.valueInGlobalWindow("foo")); + parDoFn.processElement(WindowedValues.valueInGlobalWindow("foo")); TestDoFn fn = (TestDoFn) ((SimpleParDoFn) parDoFn).getDoFnInfo().getDoFn(); parDoFn.abort(); @@ -308,7 +308,7 @@ public void testFactoryDoesNotReuseAfterAborted() throws Exception { TestOperationContext.create(counters)); secondParDoFn.startBundle(rcvr); - secondParDoFn.processElement(WindowedValue.valueInGlobalWindow("foo")); + secondParDoFn.processElement(WindowedValues.valueInGlobalWindow("foo")); TestDoFn secondFn = (TestDoFn) ((SimpleParDoFn) secondParDoFn).getDoFnInfo().getDoFn(); assertThat(secondFn, not(theInstance(fn))); @@ -378,7 +378,7 @@ public void testCleanupRegistered() throws Exception { IntervalWindow firstWindow = new IntervalWindow(new Instant(0), new Instant(10)); parDoFn.processElement( - WindowedValue.of("foo", new Instant(1), firstWindow, PaneInfo.NO_FIRING)); + WindowedValues.of("foo", new Instant(1), firstWindow, PaneInfo.NO_FIRING)); verify(stepContext) .setStateCleanupTimer( @@ -435,7 +435,7 @@ public void testCleanupTimerForGlobalWindowWithAllowedLateness() throws Exceptio GlobalWindow globalWindow = GlobalWindow.INSTANCE; parDoFn.processElement( - WindowedValue.of("foo", new Instant(1), globalWindow, PaneInfo.NO_FIRING)); + WindowedValues.of("foo", new Instant(1), globalWindow, PaneInfo.NO_FIRING)); assertThat( globalWindow.maxTimestamp().plus(allowedLateness), diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactoryTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactoryTest.java index c0e5a301ea2a..5e5f4e61a466 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactoryTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/ValuesDoFnFactoryTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.dataflow.worker; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java index 1dc4ba111110..546b33e42f1f 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillKeyedWorkItemTest.java @@ -39,7 +39,7 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.hamcrest.Matchers; import org.joda.time.Instant; @@ -97,9 +97,9 @@ public void testElementIteration() throws Exception { assertThat( keyedWorkItem.elementsIterable(), Matchers.contains( - WindowedValue.of("hello", new Instant(5), WINDOW_1, paneInfo(0)), - WindowedValue.of("world", new Instant(7), WINDOW_2, paneInfo(2)), - WindowedValue.of("earth", new Instant(6), WINDOW_1, paneInfo(1)))); + WindowedValues.of("hello", new Instant(5), WINDOW_1, paneInfo(0)), + WindowedValues.of("world", new Instant(7), WINDOW_2, paneInfo(2)), + WindowedValues.of("earth", new Instant(6), WINDOW_1, paneInfo(1)))); } private void addElement( diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java index ebd51a0fba63..b6a4cb86c686 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WindmillReaderIteratorBaseTest.java @@ -25,7 +25,8 @@ import java.util.Arrays; import java.util.List; import org.apache.beam.runners.dataflow.worker.windmill.Windmill; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.junit.Test; import org.junit.runner.RunWith; @@ -41,7 +42,7 @@ protected TestWindmillReaderIterator(Windmill.WorkItem work) { @Override protected WindowedValue decodeMessage(Windmill.Message message) { - return WindowedValue.valueInGlobalWindow(message.getTimestamp()); + return WindowedValues.valueInGlobalWindow(message.getTimestamp()); } } diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java index 2f63393218c5..df3b959c82c5 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/WorkerCustomSourcesTest.java @@ -30,7 +30,7 @@ import static org.apache.beam.sdk.testing.SourceTestUtils.readFromSource; import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; import static org.apache.beam.sdk.util.SerializableUtils.deserializeFromByteArray; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables.getStackTraceAsString; import static org.hamcrest.MatcherAssert.assertThat; @@ -119,13 +119,13 @@ import org.apache.beam.sdk.testing.ExpectedLogs; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.ValueWithRecordId; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java index 55cb0636e35d..c306ccaa7b9c 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/graph/LengthPrefixUnknownCodersTest.java @@ -53,8 +53,9 @@ import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.graph.MutableNetwork; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.graph.NetworkBuilder; @@ -68,17 +69,17 @@ @RunWith(JUnit4.class) public class LengthPrefixUnknownCodersTest { private static final Coder>> windowedValueCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()), GlobalWindow.Coder.INSTANCE); private static final Coder>> prefixedWindowedValueCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), LengthPrefixCoder.of(VarIntCoder.of())), GlobalWindow.Coder.INSTANCE); private static final Coder>> prefixedAndReplacedWindowedValueCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), LENGTH_PREFIXED_BYTE_ARRAY_CODER), GlobalWindow.Coder.INSTANCE); @@ -106,7 +107,7 @@ public void testLengthPrefixUnknownCoders() throws Exception { @Test public void testLengthPrefixForLengthPrefixCoder() throws Exception { Coder>> windowedValueCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), LengthPrefixCoder.of(VarIntCoder.of())), GlobalWindow.Coder.INSTANCE); @@ -114,7 +115,7 @@ public void testLengthPrefixForLengthPrefixCoder() throws Exception { forCodec(CloudObjects.asCloudObject(windowedValueCoder, /*sdkComponents=*/ null), false); Coder>> expectedCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), LengthPrefixCoder.of(VarIntCoder.of())), GlobalWindow.Coder.INSTANCE); @@ -127,7 +128,7 @@ public void testLengthPrefixForLengthPrefixCoder() throws Exception { @Test public void testLengthPrefixAndReplaceUnknownCoder() throws Exception { Coder>> windowedValueCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()), GlobalWindow.Coder.INSTANCE); Map lengthPrefixedCoderCloudObject = diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleDoFnTest.java index 3d2052ca84fc..613fa619452b 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleDoFnTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/BatchGroupAlsoByWindowReshuffleDoFnTest.java @@ -38,9 +38,10 @@ import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Duration; import org.joda.time.Instant; @@ -158,11 +159,11 @@ public void testReshuffleFiresEveryElement() throws Exception { gabwFactory, windowingStrategy, "key", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result.size(), equalTo(3)); diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/CombiningGroupAlsoByWindowsViaOutputBufferDoFnTest.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/CombiningGroupAlsoByWindowsViaOutputBufferDoFnTest.java index eeb9f83b244d..cc69a91828c7 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/CombiningGroupAlsoByWindowsViaOutputBufferDoFnTest.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/CombiningGroupAlsoByWindowsViaOutputBufferDoFnTest.java @@ -41,7 +41,7 @@ @RunWith(JUnit4.class) public class CombiningGroupAlsoByWindowsViaOutputBufferDoFnTest { - private class CombiningGABWViaOutputBufferDoFnFactory + private static class CombiningGABWViaOutputBufferDoFnFactory implements GroupAlsoByWindowDoFnFactory { private final Coder keyCoder; diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java index 8a42107e09f4..2ade098c9ef8 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/GroupAlsoByWindowProperties.java @@ -43,10 +43,11 @@ import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheBuilder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.CacheLoader; @@ -116,11 +117,11 @@ public static void groupsElementsIntoFixedWindows( gabwFactory, windowingStrategy, "key", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -154,12 +155,12 @@ public static void groupsElementsIntoSlidingWindowsWithMinTimestamp( gabwFactory, windowingStrategy, "key", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(5), Arrays.asList(window(-10, 10), window(0, 20)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(15), Arrays.asList(window(0, 20), window(10, 30)), @@ -203,17 +204,17 @@ public static void combinesElementsInSlidingWindows( gabwFactory, windowingStrategy, "k", - WindowedValue.of( + WindowedValues.of( 1L, new Instant(5), Arrays.asList(window(-10, 10), window(0, 20)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( 2L, new Instant(15), Arrays.asList(window(0, 20), window(10, 30)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( 4L, new Instant(18), Arrays.asList(window(0, 20), window(10, 30)), @@ -252,9 +253,11 @@ public static void groupsIntoOverlappingNonmergingWindows( gabwFactory, windowingStrategy, "key", - WindowedValue.of("v1", new Instant(1), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING), - WindowedValue.of("v2", new Instant(4), Arrays.asList(window(1, 5)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( + "v1", new Instant(1), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING), + WindowedValues.of( + "v2", new Instant(4), Arrays.asList(window(1, 5)), PaneInfo.NO_FIRING), + WindowedValues.of( "v3", new Instant(4), Arrays.asList(window(0, 5)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -282,11 +285,11 @@ public static void groupsElementsInMergedSessions( gabwFactory, windowingStrategy, "key", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -319,9 +322,9 @@ public static void combinesElementsPerSession( gabwFactory, windowingStrategy, "k", - WindowedValue.of(1L, new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of(2L, new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of(1L, new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + WindowedValues.of(2L, new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), + WindowedValues.of( 4L, new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -354,11 +357,11 @@ public static void groupsElementsIntoFixedWindowsWithEndOfWindowTimestamp( gabwFactory, windowingStrategy, "key", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -391,11 +394,11 @@ public static void groupsElementsIntoFixedWindowsWithLatestTimestamp( gabwFactory, windowingStrategy, "k", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(1), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(2), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(13), Arrays.asList(window(10, 20)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -427,11 +430,11 @@ public static void groupsElementsInMergedSessionsWithEndOfWindowTimestamp( gabwFactory, windowingStrategy, "k", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(15), Arrays.asList(window(15, 25)), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -464,11 +467,11 @@ public static void groupsElementsInMergedSessionsWithLatestTimestamp( gabwFactory, windowingStrategy, "k", - WindowedValue.of( + WindowedValues.of( "v1", new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v2", new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "v3", new Instant(15), Arrays.asList(unmergedWindow), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -504,9 +507,10 @@ public static void combinesElementsPerSessionWithEndOfWindowTimestamp( gabwFactory, windowingStrategy, "k", - WindowedValue.of(1L, new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), - WindowedValue.of(2L, new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), - WindowedValue.of(4L, new Instant(15), Arrays.asList(secondWindow), PaneInfo.NO_FIRING)); + WindowedValues.of(1L, new Instant(0), Arrays.asList(window(0, 10)), PaneInfo.NO_FIRING), + WindowedValues.of(2L, new Instant(5), Arrays.asList(window(5, 15)), PaneInfo.NO_FIRING), + WindowedValues.of( + 4L, new Instant(15), Arrays.asList(secondWindow), PaneInfo.NO_FIRING)); assertThat(result, hasSize(2)); @@ -613,7 +617,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - this.output.add(WindowedValue.of(output, timestamp, windows, pane)); + this.output.add(WindowedValues.of(output, timestamp, windows, pane)); } public List>> getOutput() { diff --git a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/ListOutputManager.java b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/ListOutputManager.java index 742ba68cf634..eccd6ec41666 100644 --- a/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/ListOutputManager.java +++ b/runners/google-cloud-dataflow-java/worker/src/test/java/org/apache/beam/runners/dataflow/worker/util/ListOutputManager.java @@ -21,8 +21,8 @@ import java.util.List; import java.util.Map; import org.apache.beam.runners.core.DoFnRunners.OutputManager; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; 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; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java index c5ca5ae54361..14d00934a242 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/BundleCheckpointHandlers.java @@ -32,7 +32,8 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.joda.time.Duration; import org.joda.time.Instant; @@ -127,7 +128,7 @@ public void onCheckpoint(ProcessBundleResponse response) { stateInternals .state(stateNamespace, StateTags.value(tag, residualCoder)) .write( - WindowedValue.of( + WindowedValues.of( stateValue.getValue(), stateValue.getTimestamp(), ImmutableList.of(window), diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java index 9d5589a54ede..aeb8aae81022 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/control/ProcessBundleDescriptors.java @@ -47,8 +47,6 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.state.TimerSpecs; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; @@ -58,6 +56,8 @@ import org.apache.beam.sdk.util.construction.graph.TimerReference; import org.apache.beam.sdk.util.construction.graph.UserStateReference; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; 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.ImmutableMap; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java index b26d3a643ef1..68f274f3de9c 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactory.java @@ -32,11 +32,11 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; import org.apache.beam.sdk.util.construction.graph.SideInputReference; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.FluentIterable; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMultimap; diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java index 762a31749457..5b7880ad56ee 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/translation/PipelineTranslatorUtils.java @@ -36,12 +36,13 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; @@ -169,15 +170,15 @@ private static void fireTimer( } } - public static WindowedValue.WindowedValueCoder getWindowedValueCoder( + public static WindowedValues.WindowedValueCoder getWindowedValueCoder( String pCollectionId, RunnerApi.Components components) { RunnerApi.PCollection pCollection = components.getPcollectionsOrThrow(pCollectionId); PipelineNode.PCollectionNode pCollectionNode = PipelineNode.pCollection(pCollectionId, pCollection); - WindowedValue.WindowedValueCoder coder; + WindowedValues.WindowedValueCoder coder; try { coder = - (WindowedValue.WindowedValueCoder) + (WindowedValues.WindowedValueCoder) WireCoders.instantiateRunnerWireCoder(pCollectionNode, components); } catch (IOException e) { throw new RuntimeException(e); diff --git a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java index f0dc2f65a0ea..0377d59a31fe 100644 --- a/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java +++ b/runners/java-fn-execution/src/main/java/org/apache/beam/runners/fnexecution/wire/WireCoders.java @@ -24,12 +24,13 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.model.pipeline.v1.RunnerApi.ExecutableStagePayload.WireCoderSetting; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.ModelCoders; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.SyntheticComponents; import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; /** Helpers to construct coders for gRPC port reads and writes. */ public class WireCoders { @@ -93,7 +94,7 @@ public static Coder> instantiateRunnerWireCoder( String protoCoderId = addRunnerWireCoder(pCollectionNode, builder, wireCoderSetting); Coder javaCoder = RehydratedComponents.forComponents(builder.build()).getCoder(protoCoderId); checkArgument( - javaCoder instanceof WindowedValue.FullWindowedValueCoder, + javaCoder instanceof WindowedValues.FullWindowedValueCoder, "Unexpected Deserialized %s type, expected %s, got %s", RunnerApi.Coder.class.getSimpleName(), FullWindowedValueCoder.class.getSimpleName(), diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java index b6b1b7be3355..556cc7993ae1 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/RemoteExecutionTest.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.fnexecution.control; import static org.apache.beam.sdk.options.ExperimentalOptions.addExperiment; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.allOf; @@ -129,7 +129,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; @@ -142,6 +141,8 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.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.collect.ImmutableMap; @@ -1914,13 +1915,13 @@ public void onWindowExpiration( outputValues.get(mainOutputTransform), containsInAnyOrder( valueInGlobalWindow(KV.of("mainX", "")), - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("event", "Y"), BoundedWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(100L))), - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("processing", "Z"), BoundedWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(200L))), - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of("onWindowExpiration", "key"), BoundedWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(5000L))))); assertThat( diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java index 7ce6b53c5eb6..7b19376426cc 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/control/SdkHarnessClientTest.java @@ -82,12 +82,13 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow.Coder; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -143,7 +144,7 @@ public void setup() throws Exception { .putAllWindowingStrategies(userProto.getComponents().getWindowingStrategiesMap()) .putAllCoders(userProto.getComponents().getCodersMap()); RunnerApi.Coder fullValueCoder = - CoderTranslation.toProto(WindowedValue.getFullCoder(StringUtf8Coder.of(), Coder.INSTANCE)) + CoderTranslation.toProto(WindowedValues.getFullCoder(StringUtf8Coder.of(), Coder.INSTANCE)) .getCoder(); pbdBuilder.putCoders("wire_coder", fullValueCoder); @@ -515,18 +516,18 @@ public void testNewBundleAndProcessElements() throws Exception { BundleProgressHandler.ignored())) { FnDataReceiver> bundleInputReceiver = Iterables.getOnlyElement(activeBundle.getInputReceivers().values()); - bundleInputReceiver.accept(WindowedValue.valueInGlobalWindow("foo")); - bundleInputReceiver.accept(WindowedValue.valueInGlobalWindow("bar")); - bundleInputReceiver.accept(WindowedValue.valueInGlobalWindow("baz")); + bundleInputReceiver.accept(WindowedValues.valueInGlobalWindow("foo")); + bundleInputReceiver.accept(WindowedValues.valueInGlobalWindow("bar")); + bundleInputReceiver.accept(WindowedValues.valueInGlobalWindow("baz")); } // The bundle can be a simple function of some sort, but needs to be complete. assertThat( outputs, containsInAnyOrder( - WindowedValue.valueInGlobalWindow("spam"), - WindowedValue.valueInGlobalWindow("ham"), - WindowedValue.valueInGlobalWindow("eggs"))); + WindowedValues.valueInGlobalWindow("spam"), + WindowedValues.valueInGlobalWindow("ham"), + WindowedValues.valueInGlobalWindow("eggs"))); } @Test diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java index 9f49309c3040..f84467077501 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/data/GrpcDataServiceTest.java @@ -49,7 +49,8 @@ import org.apache.beam.sdk.fn.stream.OutboundObserverFactory; import org.apache.beam.sdk.fn.test.TestStreams; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.inprocess.InProcessChannelBuilder; @@ -66,7 +67,7 @@ public class GrpcDataServiceTest { @Rule public transient Timeout globalTimeout = Timeout.seconds(600); private static final String TRANSFORM_ID = "888"; private static final Coder> CODER = - LengthPrefixCoder.of(WindowedValue.getValueOnlyCoder(StringUtf8Coder.of())); + LengthPrefixCoder.of(WindowedValues.getValueOnlyCoder(StringUtf8Coder.of())); @Test public void testMessageReceivedBySingleClientWhenThereAreMultipleClients() throws Exception { @@ -105,9 +106,9 @@ public void testMessageReceivedBySingleClientWhenThereAreMultipleClients() throw aggregator.start(); FnDataReceiver> consumer = aggregator.registerOutputDataLocation(TRANSFORM_ID, CODER); - consumer.accept(WindowedValue.valueInGlobalWindow("A" + i)); - consumer.accept(WindowedValue.valueInGlobalWindow("B" + i)); - consumer.accept(WindowedValue.valueInGlobalWindow("C" + i)); + consumer.accept(WindowedValues.valueInGlobalWindow("A" + i)); + consumer.accept(WindowedValues.valueInGlobalWindow("B" + i)); + consumer.accept(WindowedValues.valueInGlobalWindow("C" + i)); aggregator.sendOrCollectBufferedDataAndFinishOutboundStreams(); } waitForInboundElements.countDown(); @@ -175,9 +176,9 @@ public void testMultipleClientsSendMessagesAreDirectedToProperConsumers() throws assertThat( serverInboundValues.get(i), contains( - WindowedValue.valueInGlobalWindow("A" + i), - WindowedValue.valueInGlobalWindow("B" + i), - WindowedValue.valueInGlobalWindow("C" + i))); + WindowedValues.valueInGlobalWindow("A" + i), + WindowedValues.valueInGlobalWindow("B" + i), + WindowedValues.valueInGlobalWindow("C" + i))); } assertThat(clientInboundElements, empty()); } @@ -191,15 +192,15 @@ private BeamFnApi.Elements elementsWithData(String id) throws CoderException { .setTransformId(TRANSFORM_ID) .setData( ByteString.copyFrom( - encodeToByteArray(CODER, WindowedValue.valueInGlobalWindow("A" + id))) + encodeToByteArray(CODER, WindowedValues.valueInGlobalWindow("A" + id))) .concat( ByteString.copyFrom( encodeToByteArray( - CODER, WindowedValue.valueInGlobalWindow("B" + id)))) + CODER, WindowedValues.valueInGlobalWindow("B" + id)))) .concat( ByteString.copyFrom( encodeToByteArray( - CODER, WindowedValue.valueInGlobalWindow("C" + id)))))) + CODER, WindowedValues.valueInGlobalWindow("C" + id)))))) .addData( BeamFnApi.Elements.Data.newBuilder() .setInstructionId(id) diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java index f9a19dacc3df..70f9ef0dd4e1 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/translation/BatchSideInputHandlerFactoryTest.java @@ -41,13 +41,13 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.util.construction.graph.ImmutableExecutableStage; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.util.construction.graph.PipelineNode.PCollectionNode; import org.apache.beam.sdk.util.construction.graph.SideInputReference; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValues; import org.joda.time.DateTime; import org.joda.time.DateTimeZone; import org.joda.time.Instant; @@ -121,7 +121,7 @@ public void emptyResultForEmptyCollection() { public void singleElementForCollection() { when(context.getSideInput(COLLECTION_ID)) .thenReturn( - Arrays.asList(WindowedValue.valueInGlobalWindow(KV.of(null, 3)))); + Arrays.asList(WindowedValues.valueInGlobalWindow(KV.of(null, 3)))); BatchSideInputHandlerFactory factory = BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context); @@ -142,9 +142,9 @@ public void groupsValuesByKey() { when(context.getSideInput(COLLECTION_ID)) .thenReturn( Arrays.asList( - WindowedValue.valueInGlobalWindow(KV.of("foo", 2)), - WindowedValue.valueInGlobalWindow(KV.of("bar", 3)), - WindowedValue.valueInGlobalWindow(KV.of("foo", 5)))); + WindowedValues.valueInGlobalWindow(KV.of("foo", 2)), + WindowedValues.valueInGlobalWindow(KV.of("bar", 3)), + WindowedValues.valueInGlobalWindow(KV.of("foo", 5)))); BatchSideInputHandlerFactory factory = BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context); @@ -170,12 +170,12 @@ public void groupsValuesByWindowAndKey() { when(context.getSideInput(COLLECTION_ID)) .thenReturn( Arrays.asList( - WindowedValue.of(KV.of("foo", 1), instantA, windowA, PaneInfo.NO_FIRING), - WindowedValue.of(KV.of("baz", 2), instantA, windowA, PaneInfo.NO_FIRING), - WindowedValue.of(KV.of("foo", 3), instantA, windowA, PaneInfo.NO_FIRING), - WindowedValue.of(KV.of("foo", 4), instantB, windowB, PaneInfo.NO_FIRING), - WindowedValue.of(KV.of("bar", 5), instantB, windowB, PaneInfo.NO_FIRING), - WindowedValue.of(KV.of("foo", 6), instantB, windowB, PaneInfo.NO_FIRING))); + WindowedValues.of(KV.of("foo", 1), instantA, windowA, PaneInfo.NO_FIRING), + WindowedValues.of(KV.of("baz", 2), instantA, windowA, PaneInfo.NO_FIRING), + WindowedValues.of(KV.of("foo", 3), instantA, windowA, PaneInfo.NO_FIRING), + WindowedValues.of(KV.of("foo", 4), instantB, windowB, PaneInfo.NO_FIRING), + WindowedValues.of(KV.of("bar", 5), instantB, windowB, PaneInfo.NO_FIRING), + WindowedValues.of(KV.of("foo", 6), instantB, windowB, PaneInfo.NO_FIRING))); BatchSideInputHandlerFactory factory = BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context); @@ -206,10 +206,10 @@ public void iterableAccessPattern() { when(context.getSideInput(COLLECTION_ID)) .thenReturn( Arrays.asList( - WindowedValue.of(1, instantA, windowA, PaneInfo.NO_FIRING), - WindowedValue.of(2, instantA, windowA, PaneInfo.NO_FIRING), - WindowedValue.of(3, instantB, windowB, PaneInfo.NO_FIRING), - WindowedValue.of(4, instantB, windowB, PaneInfo.NO_FIRING))); + WindowedValues.of(1, instantA, windowA, PaneInfo.NO_FIRING), + WindowedValues.of(2, instantA, windowA, PaneInfo.NO_FIRING), + WindowedValues.of(3, instantB, windowB, PaneInfo.NO_FIRING), + WindowedValues.of(4, instantB, windowB, PaneInfo.NO_FIRING))); BatchSideInputHandlerFactory factory = BatchSideInputHandlerFactory.forStage(EXECUTABLE_STAGE, context); diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java index 2e24713eccbf..4f0e67286d62 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/CommonCoderTest.java @@ -80,13 +80,13 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.ShardedKey; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.util.construction.CoderTranslator; import org.apache.beam.sdk.util.construction.ModelCoderRegistrar; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.MoreObjects; @@ -131,10 +131,10 @@ public class CommonCoderTest { .put(getUrn(StandardCoders.Enum.DOUBLE), DoubleCoder.class) .put( getUrn(StandardCoders.Enum.WINDOWED_VALUE), - WindowedValue.FullWindowedValueCoder.class) + WindowedValues.FullWindowedValueCoder.class) .put( getUrn(StandardCoders.Enum.PARAM_WINDOWED_VALUE), - WindowedValue.ParamWindowedValueCoder.class) + WindowedValues.ParamWindowedValueCoder.class) .put(getUrn(StandardCoders.Enum.ROW), RowCoder.class) .put(getUrn(StandardCoders.Enum.SHARDED_KEY), ShardedKey.Coder.class) .put(getUrn(StandardCoders.Enum.CUSTOM_WINDOW), TimestampPrefixingWindowCoder.class) @@ -347,8 +347,8 @@ private static Object convertValue(Object value, CommonCoder coderSpec, Coder co } else if (s.equals(getUrn(StandardCoders.Enum.WINDOWED_VALUE)) || s.equals(getUrn(StandardCoders.Enum.PARAM_WINDOWED_VALUE))) { Map kvMap = (Map) value; - Coder valueCoder = ((WindowedValue.FullWindowedValueCoder) coder).getValueCoder(); - Coder windowCoder = ((WindowedValue.FullWindowedValueCoder) coder).getWindowCoder(); + Coder valueCoder = ((WindowedValues.FullWindowedValueCoder) coder).getValueCoder(); + Coder windowCoder = ((WindowedValues.FullWindowedValueCoder) coder).getWindowCoder(); Object windowValue = convertValue(kvMap.get("value"), coderSpec.getComponents().get(0), valueCoder); Instant timestamp = new Instant(((Number) kvMap.get("timestamp")).longValue()); @@ -365,7 +365,7 @@ private static Object convertValue(Object value, CommonCoder coderSpec, Coder co PaneInfo.Timing.valueOf((String) paneInfoMap.get("timing")), (int) paneInfoMap.get("index"), (int) paneInfoMap.get("on_time_index")); - return WindowedValue.of(windowValue, timestamp, windows, paneInfo); + return WindowedValues.of(windowValue, timestamp, windows, paneInfo); } else if (s.equals(getUrn(StandardCoders.Enum.DOUBLE))) { return Double.parseDouble((String) value); } else if (s.equals(getUrn(StandardCoders.Enum.ROW))) { diff --git a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java index 061c75284f85..ce44ad69dece 100644 --- a/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java +++ b/runners/java-fn-execution/src/test/java/org/apache/beam/runners/fnexecution/wire/LengthPrefixUnknownCodersTest.java @@ -31,10 +31,10 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.coders.LengthPrefixCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.checkerframework.checker.nullness.qual.Nullable; import org.junit.Rule; @@ -77,10 +77,10 @@ public static Collection data() { return ImmutableList.of( /** Test wrapping unknown coders with {@code LengthPrefixCoder}. */ new Object[] { - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(UnknownCoder.INSTANCE, UnknownCoder.INSTANCE), GlobalWindow.Coder.INSTANCE), - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( LengthPrefixCoder.of(UnknownCoder.INSTANCE), LengthPrefixCoder.of(UnknownCoder.INSTANCE)), @@ -91,10 +91,10 @@ public static Collection data() { * Test bypassing unknown coders that are already wrapped with {@code LengthPrefixCoder}. */ new Object[] { - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(UnknownCoder.INSTANCE, LengthPrefixCoder.of(UnknownCoder.INSTANCE)), GlobalWindow.Coder.INSTANCE), - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( LengthPrefixCoder.of(UnknownCoder.INSTANCE), LengthPrefixCoder.of(UnknownCoder.INSTANCE)), @@ -103,10 +103,10 @@ public static Collection data() { }, /** Test replacing unknown coders with {@code LengthPrefixCoder}. */ new Object[] { - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of(LengthPrefixCoder.of(UnknownCoder.INSTANCE), UnknownCoder.INSTANCE), GlobalWindow.Coder.INSTANCE), - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( LengthPrefixCoder.of(ByteArrayCoder.of()), LengthPrefixCoder.of(ByteArrayCoder.of())), diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/DAGBuilder.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/DAGBuilder.java index f7792e5e31ca..dd7585d6e9fa 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/DAGBuilder.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/DAGBuilder.java @@ -36,9 +36,10 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** Utility class for wiring up Jet DAGs based on Beam pipelines. */ @SuppressWarnings({ @@ -205,12 +206,12 @@ private int getNextFreeOrdinal(Vertex vertex, boolean inbound) { } private static class PartitionedKeyExtractor implements FunctionEx { - private final WindowedValue.WindowedValueCoder> coder; + private final WindowedValues.WindowedValueCoder> coder; PartitionedKeyExtractor(Coder coder) { this.coder = Utils.isKeyedValueCoder(coder) - ? (WindowedValue.WindowedValueCoder>) coder + ? (WindowedValues.WindowedValueCoder>) coder : null; } diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java index f25b95fdb4f6..5ae3e1234b26 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTransformTranslators.java @@ -46,7 +46,6 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CreatePCollectionViewTranslation; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; @@ -59,6 +58,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; @SuppressWarnings({ @@ -269,7 +270,7 @@ public Vertex translate( PCollection> input = (PCollection>) Utils.getInput(appliedTransform); - WindowedValue.WindowedValueCoder> inputCoder = + WindowedValues.WindowedValueCoder> inputCoder = Utils.getWindowedValueCoder(input); Map.Entry, PCollection> output = Utils.getOutput(appliedTransform); Coder outputCoder = Utils.getCoder((PCollection) output.getValue()); diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTranslationContext.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTranslationContext.java index 08ac6dc91ee5..cd73fcb65f56 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTranslationContext.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/JetTranslationContext.java @@ -19,8 +19,8 @@ import org.apache.beam.runners.core.construction.SerializablePipelineOptions; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; class JetTranslationContext { @@ -41,12 +41,12 @@ DAGBuilder getDagBuilder() { return dagBuilder; } - WindowedValue.FullWindowedValueCoder getTypeInfo(PCollection collection) { + WindowedValues.FullWindowedValueCoder getTypeInfo(PCollection collection) { return getTypeInfo(collection.getCoder(), collection.getWindowingStrategy()); } - WindowedValue.FullWindowedValueCoder getTypeInfo( + WindowedValues.FullWindowedValueCoder getTypeInfo( Coder coder, WindowingStrategy windowingStrategy) { - return WindowedValue.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); + return WindowedValues.getFullCoder(coder, windowingStrategy.getWindowFn().windowCoder()); } } diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java index 38cb96b13a07..cb1cd69d33c6 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/Utils.java @@ -45,13 +45,14 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; @@ -118,8 +119,8 @@ static boolean isBounded(AppliedPTransform appliedTransform) { static boolean isKeyedValueCoder(Coder coder) { if (coder instanceof KvCoder) { return true; - } else if (coder instanceof WindowedValue.WindowedValueCoder) { - return ((WindowedValue.WindowedValueCoder) coder).getValueCoder() instanceof KvCoder; + } else if (coder instanceof WindowedValues.WindowedValueCoder) { + return ((WindowedValues.WindowedValueCoder) coder).getValueCoder() instanceof KvCoder; } return false; } @@ -132,8 +133,9 @@ static Coder getCoder(PCollection pCollection) { } } - static WindowedValue.WindowedValueCoder getWindowedValueCoder(PCollection pCollection) { - return WindowedValue.FullWindowedValueCoder.of( + static WindowedValues.WindowedValueCoder getWindowedValueCoder( + PCollection pCollection) { + return WindowedValues.FullWindowedValueCoder.of( pCollection.getCoder(), pCollection.getWindowingStrategy().getWindowFn().windowCoder()); } @@ -260,9 +262,9 @@ public static WindowedValue decodeWindowedValue(byte[] item, Coder coder) } } - public static WindowedValue.FullWindowedValueCoder deriveIterableValueCoder( - WindowedValue.FullWindowedValueCoder elementCoder) { - return WindowedValue.FullWindowedValueCoder.of( + public static WindowedValues.FullWindowedValueCoder deriveIterableValueCoder( + WindowedValues.FullWindowedValueCoder elementCoder) { + return WindowedValues.FullWindowedValueCoder.of( ListCoder.of(elementCoder.getValueCoder()), elementCoder.getWindowCoder()); } diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java index c5bd4ecaf2ef..dcc4c39cd954 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AbstractParDoP.java @@ -59,9 +59,10 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.reflect.DoFnInvoker; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -132,7 +133,7 @@ abstract class AbstractParDoP implements Processor { Map.Entry::getKey, e -> Utils.deriveIterableValueCoder( - (WindowedValue.FullWindowedValueCoder) e.getValue()))); + (WindowedValues.FullWindowedValueCoder) e.getValue()))); this.outputCoders = outputCoders; this.inputValueCoder = inputValueCoder; this.outputValueCoders = outputValueCoders; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java index 82151a812d63..da940a2b8cc7 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/AssignWindowP.java @@ -27,7 +27,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; @@ -73,7 +74,7 @@ private AssignWindowP( throw new RuntimeException(e); } WindowedValue outputValue = - WindowedValue.of( + WindowedValues.of( inputValue.getValue(), inputValue.getTimestamp(), windows, diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java index 4da880e618b1..5784ef308546 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/BoundedSourceP.java @@ -36,7 +36,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** * Jet {@link com.hazelcast.jet.core.Processor} implementation for reading from a bounded Beam @@ -79,7 +80,7 @@ public Object next() { try { Object item = currentReader.getCurrent(); WindowedValue res = - WindowedValue.timestampedValueInGlobalWindow(item, currentReader.getCurrentTimestamp()); + WindowedValues.timestampedValueInGlobalWindow(item, currentReader.getCurrentTimestamp()); if (!currentReader.advance()) { nextShard(); } diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/FlattenP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/FlattenP.java index 62648d2b4c64..f981eb40dfbd 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/FlattenP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/FlattenP.java @@ -27,7 +27,7 @@ import org.apache.beam.runners.jet.DAGBuilder; import org.apache.beam.runners.jet.Utils; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** Jet {@link com.hazelcast.jet.core.Processor} implementation for Beam's Flatten primitive. */ @SuppressWarnings({ diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ImpulseP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ImpulseP.java index 671dbd582baf..6775029d14fd 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ImpulseP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ImpulseP.java @@ -29,7 +29,7 @@ import javax.annotation.Nonnull; import org.apache.beam.runners.jet.Utils; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** * /** * Jet {@link com.hazelcast.jet.core.Processor} implementation for Beam's Impulse primitive. @@ -55,7 +55,7 @@ private ImpulseP(boolean active, Coder outputCoder, String ownerId) { @Override public boolean complete() { if (active) { - return tryEmit(Utils.encode(WindowedValue.valueInGlobalWindow(new byte[0]), outputCoder)); + return tryEmit(Utils.encode(WindowedValues.valueInGlobalWindow(new byte[0]), outputCoder)); } else { return true; } diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java index 3b3d79193622..f696873ec5c0 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/StatefulParDoP.java @@ -41,10 +41,10 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java index 38a75d47e703..4d426693f9b1 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/UnboundedSourceP.java @@ -36,7 +36,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** * Jet {@link com.hazelcast.jet.core.Processor} implementation for reading from an unbounded Beam @@ -81,7 +81,7 @@ protected void init(@Nonnull Processor.Context context) throws IOException { Function, byte[]> mapFn = (reader) -> Utils.encode( - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp()), outputCoder); diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java index 1106585501ff..2ea7a998a896 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/ViewP.java @@ -33,7 +33,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.joda.time.Instant; @@ -63,7 +64,7 @@ private ViewP( this.timestampCombiner = windowingStrategy.getTimestampCombiner(); this.inputCoder = inputCoder; this.outputCoder = - Utils.deriveIterableValueCoder((WindowedValue.FullWindowedValueCoder) outputCoder); + Utils.deriveIterableValueCoder((WindowedValues.FullWindowedValueCoder) outputCoder); this.ownerId = ownerId; } @@ -90,7 +91,7 @@ public boolean complete() { .map( e -> { WindowedValue outputValue = - WindowedValue.of( + WindowedValues.of( e.getValue().values, e.getValue().timestamp, Collections.singleton(e.getKey()), diff --git a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java index 81a9e8562fe0..4aad7091f2da 100644 --- a/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java +++ b/runners/jet/src/main/java/org/apache/beam/runners/jet/processors/WindowGroupP.java @@ -52,10 +52,11 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.WindowTracing; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; @@ -95,7 +96,7 @@ public class WindowGroupP extends AbstractProcessor { private WindowGroupP( SerializablePipelineOptions pipelineOptions, - WindowedValue.WindowedValueCoder> inputCoder, + WindowedValues.WindowedValueCoder> inputCoder, Coder outputCoder, WindowingStrategy windowingStrategy, String ownerId) { @@ -130,7 +131,7 @@ private WindowGroupP( Utils.ByteArrayKey keyBytes = new Utils.ByteArrayKey(Utils.encode(key, inputValueCoder.getKeyCoder())); WindowedValue updatedWindowedValue = - WindowedValue.of( + WindowedValues.of( value, windowedValue.getTimestamp(), windowedValue.getWindows(), @@ -146,7 +147,7 @@ private WindowGroupP( @SuppressWarnings("unchecked") public static SupplierEx supplier( SerializablePipelineOptions pipelineOptions, - WindowedValue.WindowedValueCoder> inputCoder, + WindowedValues.WindowedValueCoder> inputCoder, Coder outputCoder, WindowingStrategy windowingStrategy, String ownerId) { @@ -234,7 +235,7 @@ public void outputWindowedValue( Collection windows, PaneInfo pane) { WindowedValue>> windowedValue = - WindowedValue.of(output, timestamp, windows, pane); + WindowedValues.of(output, timestamp, windows, pane); byte[] encodedValue = Utils.encode(windowedValue, outputCoder); //noinspection ResultOfMethodCallIgnored appendableTraverser.append(encodedValue); diff --git a/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java b/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java index 96b2614b1a71..291ca91b5c18 100644 --- a/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java +++ b/runners/jet/src/test/java/org/apache/beam/runners/jet/TestStreamP.java @@ -33,7 +33,7 @@ import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.joda.time.Instant; /** @@ -69,7 +69,7 @@ private TestStreamP(byte[] payload, TestStream.TestStreamCoder payloadCoder, Cod false) .map( tv -> - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( tv.getValue(), tv.getTimestamp())) .map(wV -> Utils.encode(wV, outputCoder)); } else { diff --git a/runners/local-java/src/main/java/org/apache/beam/runners/local/Bundle.java b/runners/local-java/src/main/java/org/apache/beam/runners/local/Bundle.java index c556db45c75b..7e0b42e561c3 100644 --- a/runners/local-java/src/main/java/org/apache/beam/runners/local/Bundle.java +++ b/runners/local-java/src/main/java/org/apache/beam/runners/local/Bundle.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.local; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java index eb2b423a1171..260957e05ecc 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/BoundedSourceSystem.java @@ -42,7 +42,8 @@ import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.samza.Partition; import org.apache.samza.SamzaException; @@ -314,7 +315,7 @@ private X invoke(FnWithMetricsWrapper.SupplierWithException fn) throws Ex private void enqueueMessage(BoundedReader reader) throws InterruptedException { final T value = reader.getCurrent(); final WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow(value, reader.getCurrentTimestamp()); + WindowedValues.timestampedValueInGlobalWindow(value, reader.getCurrentTimestamp()); final SystemStreamPartition ssp = readerToSsp.get(reader); final IncomingMessageEnvelope envelope = new IncomingMessageEnvelope( diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java index 3b53894954d7..41214d7800e2 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/adapter/UnboundedSourceSystem.java @@ -46,7 +46,8 @@ import org.apache.beam.sdk.io.UnboundedSource.CheckpointMark; import org.apache.beam.sdk.io.UnboundedSource.UnboundedReader; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.ImmutableMap; import org.apache.samza.Partition; @@ -400,7 +401,7 @@ private void enqueueMessage(UnboundedReader reader) throws InterruptedException final Instant time = reader.getCurrentTimestamp(); final SystemStreamPartition ssp = readerToSsp.get(reader); final WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow(value, time); + WindowedValues.timestampedValueInGlobalWindow(value, time); final OpMessage opMessage = OpMessage.ofElement(windowedValue); final IncomingMessageEnvelope envelope = diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java index 7df33b531586..b815649a7659 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/DoFnRunnerWithMetrics.java @@ -21,7 +21,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaGBKMetricOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaGBKMetricOp.java index 897c4a05b157..9b6375171c23 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaGBKMetricOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaGBKMetricOp.java @@ -26,7 +26,7 @@ import org.apache.beam.runners.samza.runtime.Op; import org.apache.beam.runners.samza.runtime.OpEmitter; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.samza.config.Config; import org.apache.samza.context.Context; import org.apache.samza.operators.Scheduler; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOp.java index 8738d90c66c5..0f5334546c7c 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/metrics/SamzaMetricOp.java @@ -27,7 +27,7 @@ import org.apache.beam.runners.samza.runtime.Op; import org.apache.beam.runners.samza.runtime.OpEmitter; import org.apache.beam.runners.samza.util.PipelineJsonRenderer; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.samza.config.Config; import org.apache.samza.context.Context; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java index 76dfe5b720d8..d07a9bda78c6 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunner.java @@ -29,8 +29,8 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; import org.slf4j.Logger; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/ClassicBundleManager.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/ClassicBundleManager.java index 1d75afbb4c57..cf68cdb5b65c 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/ClassicBundleManager.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/ClassicBundleManager.java @@ -29,7 +29,7 @@ import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; 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.samza.operators.Scheduler; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java index bb396ae0456b..11c1e8e3955a 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnOp.java @@ -52,11 +52,12 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.samza.config.Config; @@ -481,7 +482,7 @@ static CompletionStage> createOutputFuture( Function valueMapper) { return valueFuture.thenApply( res -> - WindowedValue.of( + WindowedValues.of( valueMapper.apply(res), windowedValue.getTimestamp(), windowedValue.getWindows(), diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java index a07cf5fe2c8d..34e3405660c2 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/DoFnRunnerWithKeyedInternals.java @@ -22,8 +22,8 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** This class wraps a DoFnRunner with keyed StateInternals and TimerInternals access. */ diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollector.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollector.java index c606b7569357..750e42d96e26 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollector.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollector.java @@ -19,7 +19,7 @@ import java.util.Collection; import java.util.concurrent.CompletionStage; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * A future collector that buffers the output from the users {@link diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollectorImpl.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollectorImpl.java index f54f5e441b84..e364eb7c4078 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollectorImpl.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/FutureCollectorImpl.java @@ -25,7 +25,7 @@ import java.util.concurrent.CompletionStage; import java.util.concurrent.atomic.AtomicBoolean; import org.apache.beam.runners.samza.util.FutureUtils; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; class FutureCollectorImpl implements FutureCollector { private final AtomicBoolean collectorSealed; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java index 1b19275dd967..b77a9de56a00 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/GroupByKeyOp.java @@ -40,10 +40,11 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.samza.config.Config; import org.apache.samza.context.Context; @@ -235,7 +236,7 @@ public void processTimer(KeyedTimerData keyedTimerData, OpEmitter implements Op, KeyedWorkItem, K> { diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/Op.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/Op.java index 93e6a9c2f613..14e0151bb708 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/Op.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/Op.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.samza.runtime; import java.io.Serializable; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.samza.config.Config; import org.apache.samza.context.Context; import org.apache.samza.operators.Scheduler; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java index cbe81a1359e2..f2eecbbbc9c7 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpAdapter.java @@ -34,7 +34,7 @@ import org.apache.beam.runners.samza.util.FutureUtils; import org.apache.beam.runners.samza.util.SamzaPipelineExceptionListener; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.samza.config.Config; import org.apache.samza.context.Context; import org.apache.samza.operators.Scheduler; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpEmitter.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpEmitter.java index cefbf0f8a2bf..c74d1cf1e11e 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpEmitter.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpEmitter.java @@ -19,7 +19,7 @@ import java.util.Collection; import java.util.concurrent.CompletionStage; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** Output emitter for Samza {@link Op}. */ diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java index dc288e6f901d..9ee7ffd48f2f 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/OpMessage.java @@ -17,7 +17,7 @@ */ package org.apache.beam.runners.samza.runtime; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.checkerframework.checker.nullness.qual.Nullable; import org.joda.time.Instant; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableDoFnOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableDoFnOp.java index f4e3641aeb9e..743a42d14791 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableDoFnOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/PortableDoFnOp.java @@ -48,11 +48,11 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.samza.config.Config; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaAssignContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaAssignContext.java index 1a0caaa99c6d..0f8933e66cbf 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaAssignContext.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaAssignContext.java @@ -19,7 +19,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java index 7625704de8ee..f1fd1b303044 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SamzaDoFnRunners.java @@ -57,7 +57,6 @@ import org.apache.beam.sdk.transforms.DoFnSchemaInformation; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.util.construction.graph.PipelineNode; @@ -65,6 +64,7 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.samza.context.Context; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SingletonKeyedWorkItem.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SingletonKeyedWorkItem.java index 7ec9906c5032..5a59e8616cc5 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SingletonKeyedWorkItem.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SingletonKeyedWorkItem.java @@ -20,7 +20,7 @@ import java.util.Collections; import org.apache.beam.runners.core.KeyedWorkItem; import org.apache.beam.runners.core.TimerInternals; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** Implementation of {@link KeyedWorkItem} which contains only a single value. */ class SingletonKeyedWorkItem implements KeyedWorkItem { diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java index 2a6ba3e62a78..e8d8b64c381a 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/SplittableParDoProcessKeyedElementsOp.java @@ -44,11 +44,12 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection.IsBounded; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.samza.config.Config; @@ -178,7 +179,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputManager.output(tag, WindowedValue.of(output, timestamp, windows, pane)); + outputManager.output(tag, WindowedValues.of(output, timestamp, windows, pane)); } }, NullSideInputReader.empty(), @@ -260,7 +261,7 @@ public void processTimer( private void fireTimer(byte[] key, TimerData timer) { LOG.debug("Firing timer {} for key {}", timer, key); fnRunner.processElement( - WindowedValue.valueInGlobalWindow( + WindowedValues.valueInGlobalWindow( KeyedWorkItems.timersWorkItem(key, Collections.singletonList(timer)))); } } diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/WindowAssignOp.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/WindowAssignOp.java index 48fb96917cb3..ed82589d2125 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/WindowAssignOp.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/runtime/WindowAssignOp.java @@ -20,7 +20,8 @@ import java.util.Collection; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** Samza operator for {@link org.apache.beam.sdk.transforms.windowing.Window.Assign}. */ public class WindowAssignOp implements Op { @@ -42,7 +43,7 @@ public void processElement(WindowedValue inputElement, OpEmitter emitter) windows.stream() .map( window -> - WindowedValue.of( + WindowedValues.of( inputElement.getValue(), inputElement.getTimestamp(), window, diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java index 527522e66728..18d105e35e9f 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/GroupByKeyTranslator.java @@ -44,12 +44,13 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.AppliedCombineFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.samza.operators.MessageStream; @@ -117,7 +118,7 @@ public void translatePortable( final MessageStream>> inputStream = ctx.getMessageStreamById(inputId); final WindowingStrategy windowingStrategy = WindowUtils.getWindowStrategy(inputId, pipeline.getComponents()); - final WindowedValue.WindowedValueCoder> windowedInputCoder = + final WindowedValues.WindowedValueCoder> windowedInputCoder = WindowUtils.instantiateWindowedCoder(inputId, pipeline.getComponents()); final TupleTag> outputTag = new TupleTag<>(Iterables.getOnlyElement(transform.getTransform().getOutputsMap().keySet())); @@ -151,14 +152,14 @@ static MessageStream>> doTranslate RunnerApi.PCollection input, MessageStream>> inputStream, WindowingStrategy windowingStrategy, - WindowedValue.WindowedValueCoder> windowedInputCoder, + WindowedValues.WindowedValueCoder> windowedInputCoder, TupleTag> outputTag, PortableTranslationContext ctx) { final boolean needRepartition = ctx.getPipelineOptions().getMaxSourceParallelism() > 1; final Coder windowCoder = windowingStrategy.getWindowFn().windowCoder(); final KvCoder kvInputCoder = (KvCoder) windowedInputCoder.getValueCoder(); final Coder>> elementCoder = - WindowedValue.FullWindowedValueCoder.of(kvInputCoder, windowCoder); + WindowedValues.FullWindowedValueCoder.of(kvInputCoder, windowCoder); @SuppressWarnings("unchecked") final SystemReduceFn reduceFn = diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java index e8668f5244b3..6b4d464b8e7c 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ParDoBoundMultiTranslator.java @@ -56,7 +56,6 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.RunnerPCollectionView; import org.apache.beam.sdk.util.construction.graph.PipelineNode; @@ -68,6 +67,8 @@ import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.samza.operators.MessageStream; @@ -262,8 +263,8 @@ private static void doTranslatePortable( .getInputsOrThrow(sideInputId.getLocalName()); final WindowingStrategy windowingStrategy = WindowUtils.getWindowStrategy(sideInputCollectionId, components); - final WindowedValue.WindowedValueCoder coder = - (WindowedValue.WindowedValueCoder) instantiateCoder(sideInputCollectionId, components); + final WindowedValues.WindowedValueCoder coder = + (WindowedValues.WindowedValueCoder) instantiateCoder(sideInputCollectionId, components); // Create a runner-side view final PCollectionView view = createPCollectionView(sideInputId, coder, windowingStrategy); @@ -305,7 +306,7 @@ private static void doTranslatePortable( index.incrementAndGet(); }); - WindowedValue.WindowedValueCoder windowedInputCoder = + WindowedValues.WindowedValueCoder windowedInputCoder = WindowUtils.instantiateWindowedCoder(inputId, pipeline.getComponents()); // TODO: support schema and side inputs for portable runner @@ -321,7 +322,7 @@ private static void doTranslatePortable( final Coder timerKeyCoder = stagePayload.getTimersCount() > 0 ? ((KvCoder) - ((WindowedValue.FullWindowedValueCoder) windowedInputCoder).getValueCoder()) + ((WindowedValues.FullWindowedValueCoder) windowedInputCoder).getValueCoder()) .getKeyCoder() : null; @@ -463,7 +464,7 @@ public Map createPortableConfig( // PCollectionView to represent a portable side input. private static PCollectionView createPCollectionView( SideInputId sideInputId, - WindowedValue.WindowedValueCoder coder, + WindowedValues.WindowedValueCoder coder, WindowingStrategy windowingStrategy) { return new RunnerPCollectionView<>( @@ -484,7 +485,7 @@ MessageStream>> groupAndBroadcastSideInput( String sideInputCollectionId, RunnerApi.PCollection sideInputPCollection, WindowingStrategy windowingStrategy, - WindowedValue.WindowedValueCoder coder, + WindowedValues.WindowedValueCoder coder, PortableTranslationContext ctx) { final MessageStream> sideInput = ctx.getMessageStreamById(sideInputCollectionId); @@ -494,7 +495,7 @@ MessageStream>> groupAndBroadcastSideInput( WindowedValue wv = opMessage.getElement(); return OpMessage.ofElement(wv.withValue(KV.of(null, wv.getValue()))); }); - final WindowedValue.WindowedValueCoder> kvCoder = + final WindowedValues.WindowedValueCoder> kvCoder = coder.withValueCoder(KvCoder.of(VoidCoder.of(), coder.getValueCoder())); final MessageStream>>> groupedSideInput = GroupByKeyTranslator.doTranslatePortable( diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReadTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReadTranslator.java index 59caecdc41c2..a666e36ad02f 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReadTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReadTranslator.java @@ -28,10 +28,10 @@ import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.samza.operators.KV; import org.apache.samza.serializers.KVSerde; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReshuffleTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReshuffleTranslator.java index bfcbd3edae40..c318505d9849 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReshuffleTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/ReshuffleTranslator.java @@ -26,12 +26,13 @@ import org.apache.beam.sdk.coders.KvCoder; import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.NativeTransforms; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.samza.operators.MessageStream; import org.apache.samza.serializers.KVSerde; @@ -84,7 +85,7 @@ public void translatePortable( final String inputId = ctx.getInputId(transform); final MessageStream>> inputStream = ctx.getMessageStreamById(inputId); - final WindowedValue.WindowedValueCoder> windowedInputCoder = + final WindowedValues.WindowedValueCoder> windowedInputCoder = WindowUtils.instantiateWindowedCoder(inputId, pipeline.getComponents()); final String outputId = ctx.getOutputId(transform); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemFactory.java index 7352e9a434a0..4035c1610e9f 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemFactory.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaImpulseSystemFactory.java @@ -27,7 +27,7 @@ import java.util.stream.Collectors; import org.apache.beam.runners.samza.runtime.OpMessage; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.samza.Partition; import org.apache.samza.config.Config; import org.apache.samza.metrics.MetricsRegistry; @@ -118,7 +118,7 @@ private static List constructMessages(SystemStreamParti ssp, DUMMY_OFFSET, /* key */ null, - OpMessage.ofElement(WindowedValue.valueInGlobalWindow(new byte[0]))); + OpMessage.ofElement(WindowedValues.valueInGlobalWindow(new byte[0]))); final IncomingMessageEnvelope watermarkMessage = IncomingMessageEnvelope.buildWatermarkEnvelope( diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishViewTranslator.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishViewTranslator.java index 08b6196b6c43..08cf1057aabf 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishViewTranslator.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaPublishViewTranslator.java @@ -23,8 +23,8 @@ import org.apache.beam.runners.samza.util.SamzaCoders; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.TransformHierarchy; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.samza.operators.MessageStream; /** Translates {@link SamzaPublishView} to a view {@link MessageStream} as side input. */ diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamSystemFactory.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamSystemFactory.java index 7287994e9e8c..72cd711a6acc 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamSystemFactory.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SamzaTestStreamSystemFactory.java @@ -29,8 +29,9 @@ import org.apache.beam.sdk.testing.TestStream; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.samza.Partition; import org.apache.samza.SamzaException; @@ -144,7 +145,7 @@ public Map> poll( // timestamp. for (TimestampedValue element : ((TestStream.ElementEvent) event).getElements()) { WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( element.getValue(), element.getTimestamp()); final OpMessage opMessage = OpMessage.ofElement(windowedValue); final IncomingMessageEnvelope envelope = diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java index 9a0e9a78da79..215c7e1fbcb7 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/SplittableParDoTranslators.java @@ -34,11 +34,11 @@ import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.samza.operators.MessageStream; import org.apache.samza.serializers.KVSerde; diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java index ae6015c5f00d..82725d1ce2e5 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/translation/TranslationContext.java @@ -38,13 +38,14 @@ import org.apache.beam.sdk.runners.TransformHierarchy; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.samza.application.descriptors.StreamApplicationDescriptor; @@ -368,7 +369,7 @@ public StoreIdGenerator getStoreIdGenerator() { producer.send(id, new OutgoingMessageEnvelope(sysStream, 0, null, msg)); }; final WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow("dummy", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("dummy", new Instant()); sendFn.accept(OpMessage.ofElement(windowedValue)); sendFn.accept(new WatermarkMessage(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis())); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaCoders.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaCoders.java index a118b5a704b5..f546303b9b9b 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaCoders.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaCoders.java @@ -22,8 +22,9 @@ import java.io.IOException; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.samza.serializers.Serde; /** Utils for Coders creation/conversion in Samza. */ @@ -38,7 +39,7 @@ public static Coder> of(PCollection pCollection) { final Coder coder = pCollection.getCoder(); final Coder windowCoder = pCollection.getWindowingStrategy().getWindowFn().windowCoder(); - return WindowedValue.FullWindowedValueCoder.of(coder, windowCoder); + return WindowedValues.FullWindowedValueCoder.of(coder, windowCoder); } public static Serde toSerde(final Coder coder) { diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java index 4ab1bcc8b0ee..999506e1e021 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/SamzaPipelineTranslatorUtils.java @@ -20,9 +20,9 @@ import java.io.IOException; import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.fnexecution.wire.WireCoders; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValues; /** Utilities for pipeline translation. */ @SuppressWarnings({ @@ -31,12 +31,12 @@ public final class SamzaPipelineTranslatorUtils { private SamzaPipelineTranslatorUtils() {} - public static WindowedValue.WindowedValueCoder instantiateCoder( + public static WindowedValues.WindowedValueCoder instantiateCoder( String collectionId, RunnerApi.Components components) { PipelineNode.PCollectionNode collectionNode = PipelineNode.pCollection(collectionId, components.getPcollectionsOrThrow(collectionId)); try { - return (WindowedValue.WindowedValueCoder) + return (WindowedValues.WindowedValueCoder) WireCoders.instantiateRunnerWireCoder(collectionNode, components); } catch (IOException e) { throw new RuntimeException(e); diff --git a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/WindowUtils.java b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/WindowUtils.java index b4531d078933..7f39a9739c59 100644 --- a/runners/samza/src/main/java/org/apache/beam/runners/samza/util/WindowUtils.java +++ b/runners/samza/src/main/java/org/apache/beam/runners/samza/util/WindowUtils.java @@ -21,10 +21,10 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.runners.fnexecution.wire.WireCoders; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.util.construction.graph.PipelineNode; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; /** Utils for window operations. */ @@ -57,15 +57,15 @@ public static WindowingStrategy getWindowStrategy( } /** - * Instantiate {@link WindowedValue.WindowedValueCoder} for given collection id from {@link + * Instantiate {@link WindowedValues.WindowedValueCoder} for given collection id from {@link * RunnerApi.Components}. */ - public static WindowedValue.WindowedValueCoder instantiateWindowedCoder( + public static WindowedValues.WindowedValueCoder instantiateWindowedCoder( String collectionId, RunnerApi.Components components) { PipelineNode.PCollectionNode collectionNode = PipelineNode.pCollection(collectionId, components.getPcollectionsOrThrow(collectionId)); try { - return (WindowedValue.WindowedValueCoder) + return (WindowedValues.WindowedValueCoder) WireCoders.instantiateRunnerWireCoder(collectionNode, components); } catch (IOException e) { throw new RuntimeException(e); diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestSourceHelpers.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestSourceHelpers.java index b020a9acd6ce..60c0c27b8d1f 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestSourceHelpers.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/adapter/TestSourceHelpers.java @@ -29,7 +29,7 @@ import org.apache.beam.runners.samza.runtime.OpMessage; import org.apache.beam.sdk.io.Source; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.samza.system.IncomingMessageEnvelope; import org.apache.samza.system.SystemStreamPartition; import org.joda.time.Instant; @@ -131,7 +131,7 @@ static IncomingMessageEnvelope createElementMessage( ssp, offset, null, - OpMessage.ofElement(WindowedValue.timestampedValueInGlobalWindow(element, timestamp))); + OpMessage.ofElement(WindowedValues.timestampedValueInGlobalWindow(element, timestamp))); } static IncomingMessageEnvelope createWatermarkMessage( diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaRunnerWithTransformMetrics.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaRunnerWithTransformMetrics.java index 68674d202cdb..dc660ce85af5 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaRunnerWithTransformMetrics.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/metrics/TestSamzaRunnerWithTransformMetrics.java @@ -43,9 +43,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.FixedWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.samza.context.Context; import org.apache.samza.metrics.Counter; @@ -175,9 +176,9 @@ public void testSamzaRunnerWithDefaultMetrics() { @Test public void testSamzaInputAndOutputMetricOp() { final WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow("value-1", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("value-1", new Instant()); final WindowedValue windowedValue2 = - WindowedValue.timestampedValueInGlobalWindow("value-2", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("value-2", new Instant()); final WatermarkMessage watermarkMessage = new WatermarkMessage(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()); @@ -250,9 +251,9 @@ public void testSamzaInputAndOutputMetricOp() { @Test public void testSamzaInputAndOutputGBKMetricOp() { final WindowedValue windowedValue = - WindowedValue.timestampedValueInGlobalWindow("value-1", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("value-1", new Instant()); final WindowedValue windowedValue2 = - WindowedValue.timestampedValueInGlobalWindow("value-2", new Instant()); + WindowedValues.timestampedValueInGlobalWindow("value-2", new Instant()); final WatermarkMessage watermarkMessage = new WatermarkMessage(BoundedWindow.TIMESTAMP_MAX_VALUE.getMillis()); diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunnerTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunnerTest.java index f6f88eb8b280..4040040d3e04 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunnerTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/AsyncDoFnRunnerTest.java @@ -46,10 +46,11 @@ import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Ignore; import org.junit.Rule; @@ -214,10 +215,10 @@ public void testKeyedOutputFutures() { final String appleKey = "apple"; final WindowedValue> input1 = - WindowedValue.valueInGlobalWindow(KV.of(appleKey, 1)); + WindowedValues.valueInGlobalWindow(KV.of(appleKey, 1)); final WindowedValue> input2 = - WindowedValue.valueInGlobalWindow(KV.of(appleKey, 2)); + WindowedValues.valueInGlobalWindow(KV.of(appleKey, 2)); asyncDoFnRunner.processElement(input1); asyncDoFnRunner.processElement(input2); diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/ClassicBundleManagerTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/ClassicBundleManagerTest.java index 1072e628a70c..8ff29ecf3838 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/ClassicBundleManagerTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/ClassicBundleManagerTest.java @@ -35,7 +35,7 @@ import java.util.concurrent.CountDownLatch; import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.samza.operators.Scheduler; import org.joda.time.Instant; import org.junit.Before; diff --git a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/FutureCollectorImplTest.java b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/FutureCollectorImplTest.java index a808fd223960..b0b9b5450f23 100644 --- a/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/FutureCollectorImplTest.java +++ b/runners/samza/src/test/java/org/apache/beam/runners/samza/runtime/FutureCollectorImplTest.java @@ -25,7 +25,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionStage; import java.util.stream.Collectors; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Assert; import org.junit.Before; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/io/BoundedDatasetFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/io/BoundedDatasetFactory.java index c290a40287a5..c0d46e77c1d6 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/io/BoundedDatasetFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/io/BoundedDatasetFactory.java @@ -19,7 +19,7 @@ import static java.util.stream.Collectors.toList; import static org.apache.beam.runners.spark.structuredstreaming.translation.utils.ScalaInterop.emptyList; -import static org.apache.beam.sdk.util.WindowedValue.timestampedValueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.timestampedValueInGlobalWindow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static scala.collection.JavaConverters.asScalaIterator; @@ -36,7 +36,7 @@ import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.io.BoundedSource.BoundedReader; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; 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.ImmutableSet; import org.apache.spark.InterruptibleIterator; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java index da89ece0a2f7..32cbe5b0acab 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/EvaluationContext.java @@ -20,7 +20,7 @@ import java.util.Collection; import javax.annotation.Nullable; import org.apache.beam.sdk.annotations.Internal; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Throwables; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.SparkSession; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java index 71325208c5ad..a681dea2fde5 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/PipelineTranslator.java @@ -46,13 +46,13 @@ import org.apache.beam.sdk.runners.TransformHierarchy.Node; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.View; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java index 13cc22940263..5b9e5b6fae86 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/SparkSessionFactory.java @@ -74,11 +74,11 @@ import org.apache.beam.sdk.transforms.join.UnionCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.resources.PipelineResources; import org.apache.beam.sdk.values.PCollectionViews; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Collections2; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.spark.SparkConf; @@ -269,9 +269,9 @@ public void registerClasses(Kryo kryo) { // bounded windows and windowed value coders kryo.register(GlobalWindow.Coder.class); kryo.register(IntervalWindow.IntervalWindowCoder.class); - kryo.register(WindowedValue.FullWindowedValueCoder.class); - kryo.register(WindowedValue.ParamWindowedValueCoder.class); - kryo.register(WindowedValue.ValueOnlyWindowedValueCoder.class); + kryo.register(WindowedValues.FullWindowedValueCoder.class); + kryo.register(WindowedValues.ParamWindowedValueCoder.class); + kryo.register(WindowedValues.ValueOnlyWindowedValueCoder.class); // various others kryo.register(OffsetRange.Coder.class); diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java index 2a01af5fb76e..24783040704e 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/TransformTranslator.java @@ -34,12 +34,12 @@ import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PInput; import org.apache.beam.sdk.values.POutput; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/Aggregators.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/Aggregators.java index 24d0f2eced69..183445642a0b 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/Aggregators.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/Aggregators.java @@ -43,7 +43,8 @@ import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Collections2; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -534,7 +535,7 @@ public final Collection> finish(MapT buffer) { } private WindowedValue windowedValue(Entry> e) { - return WindowedValue.of(extract(e.getValue()._2), e.getValue()._1, e.getKey(), NO_FIRING); + return WindowedValues.of(extract(e.getValue()._2), e.getValue()._1, e.getKey(), NO_FIRING); } } diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGloballyTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGloballyTranslatorBatch.java index c2bc05bed413..c77637ab5b91 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGloballyTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGloballyTranslatorBatch.java @@ -30,8 +30,9 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; @@ -120,6 +121,6 @@ private Coder accumulatorCoder( } private static Fun1>> windowedValue() { - return v -> single(WindowedValue.valueInGlobalWindow(v)); + return v -> single(WindowedValues.valueInGlobalWindow(v)); } } diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGroupedValuesTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGroupedValuesTranslatorBatch.java index cf9a2faac9cb..fa59cdf2452c 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGroupedValuesTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombineGroupedValuesTranslatorBatch.java @@ -23,9 +23,9 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.CombineWithContext; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; import org.apache.spark.sql.expressions.Aggregator; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTranslatorBatch.java index 63f3da0bd3dc..02c56a8081cf 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/CombinePerKeyTranslatorBatch.java @@ -37,9 +37,10 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; @@ -146,7 +147,7 @@ public void translate(Combine.PerKey transform, Context cxt) { } private static Fun1, WindowedValue>> globalKV() { - return t -> WindowedValue.valueInGlobalWindow(KV.of(t._1, t._2)); + return t -> WindowedValues.valueInGlobalWindow(KV.of(t._1, t._2)); } private Encoder accumEncoder( diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnPartitionIteratorFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnPartitionIteratorFactory.java index 907b847c7e8a..8d289f6b870c 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnPartitionIteratorFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnPartitionIteratorFactory.java @@ -30,8 +30,8 @@ import org.apache.beam.runners.spark.structuredstreaming.metrics.MetricsAccumulator; import org.apache.beam.runners.spark.structuredstreaming.translation.batch.DoFnRunnerFactory.DoFnRunnerWithTeardown; import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.AbstractIterator; import org.checkerframework.checker.nullness.qual.NonNull; import scala.Function1; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java index 6b548eff88c2..a2f81019910f 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerFactory.java @@ -37,11 +37,11 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; 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.Lists; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java index 6b7e353b164a..db4ee5be5786 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/DoFnRunnerWithMetrics.java @@ -28,7 +28,7 @@ import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** DoFnRunner decorator which registers {@link MetricsContainer}. */ diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java index fd02a3ef5242..63786829bd53 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/FlattenTranslatorBatch.java @@ -24,9 +24,9 @@ import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionList; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Encoder; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyHelpers.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyHelpers.java index 28ab07114c6a..b7c139068d1b 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyHelpers.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyHelpers.java @@ -27,8 +27,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import scala.Tuple2; import scala.collection.TraversableOnce; @@ -89,7 +90,7 @@ static Fun1, V>, WindowedValue>> } static WindowedValue> windowedKV(Tuple2 key, V value) { - return WindowedValue.of(KV.of(key._2, value), key._1.maxTimestamp(), key._1, NO_FIRING); + return WindowedValues.of(KV.of(key._2, value), key._1.maxTimestamp(), key._1, NO_FIRING); } static Fun1, V> value() { diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java index bbda0f0cfb65..7ab70f3652c8 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/GroupByKeyTranslatorBatch.java @@ -58,9 +58,10 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.sql.Column; import org.apache.spark.sql.Dataset; @@ -155,7 +156,7 @@ public void translate(GroupByKey transform, Context cxt) { .groupByKey(valueKey(), keyEnc) .mapValues(valueValue(), cxt.valueEncoderOf(inputCoder)) .mapGroups(fun2((k, it) -> KV.of(k, iterableOnce(it))), cxt.kvEncoderOf(outputCoder)) - .map(fun1(WindowedValue::valueInGlobalWindow), cxt.windowedEncoder(outputCoder)); + .map(fun1(WindowedValues::valueInGlobalWindow), cxt.windowedEncoder(outputCoder)); } else if (useCollectList && eligibleForGroupByWindow(windowing, false) diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ImpulseTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ImpulseTranslatorBatch.java index 28f929679dd6..78afdfa5451e 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ImpulseTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ImpulseTranslatorBatch.java @@ -23,9 +23,10 @@ import org.apache.beam.sdk.coders.ByteArrayCoder; import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.spark.sql.Dataset; @@ -39,7 +40,7 @@ class ImpulseTranslatorBatch extends TransformTranslator> dataset = cxt.createDataset( - ImmutableList.of(WindowedValue.valueInGlobalWindow(EMPTY_BYTE_ARRAY)), + ImmutableList.of(WindowedValues.valueInGlobalWindow(EMPTY_BYTE_ARRAY)), cxt.windowedEncoder(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE)); cxt.putDataset(cxt.getOutput(), dataset); } diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java index c212f377eef5..0f43f329b0df 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ParDoTranslatorBatch.java @@ -45,12 +45,12 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; import org.apache.spark.broadcast.Broadcast; import org.apache.spark.sql.Dataset; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java index e395c3f35908..e83ada473d0c 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReadSourceTranslatorBatch.java @@ -24,10 +24,10 @@ import org.apache.beam.sdk.io.BoundedSource; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.sql.Encoder; import org.apache.spark.sql.SparkSession; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReshuffleTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReshuffleTranslatorBatch.java index e2bb27a750f1..2c541ba4ae43 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReshuffleTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/ReshuffleTranslatorBatch.java @@ -22,9 +22,9 @@ import java.io.IOException; import org.apache.beam.runners.spark.structuredstreaming.translation.TransformTranslator; import org.apache.beam.sdk.transforms.Reshuffle; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.internal.SQLConf; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java index 140f2f88d8cd..adac966f457d 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/WindowAssignTranslatorBatch.java @@ -23,8 +23,9 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.Dataset; @@ -93,7 +94,7 @@ public T element() { return window; } }); - return WindowedValue.of(element, timestamp, windows, value.getPane()); + return WindowedValues.of(element, timestamp, windows, value.getPane()); }; } diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java index b348eca0c436..61c568dc2585 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/GroupAlsoByWindowViaOutputBufferFn.java @@ -35,10 +35,11 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.api.java.function.FlatMapGroupsFunction; import org.joda.time.Instant; @@ -154,7 +155,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputs.add(WindowedValue.of(output, timestamp, windows, pane)); + outputs.add(WindowedValues.of(output, timestamp, windows, pane)); } @Override diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValues.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValues.java index 57d97cff1625..23c8d49c3091 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValues.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValues.java @@ -19,7 +19,7 @@ import static org.apache.beam.runners.spark.structuredstreaming.translation.utils.ScalaInterop.fun1; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; -import static org.apache.beam.sdk.util.WindowedValue.getFullCoder; +import static org.apache.beam.sdk.values.WindowedValues.getFullCoder; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.apache.spark.sql.Encoders.BINARY; @@ -42,8 +42,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.spark.sql.Dataset; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; diff --git a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java index 7a19c1c0e027..55f22f92a132 100644 --- a/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java +++ b/runners/spark/3/src/main/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpers.java @@ -45,8 +45,9 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.PaneInfoCoder; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -354,7 +355,7 @@ private static Expression deserializeWindowedValue( ifNotNull(timestamp, invoke(Utils.class, "maxTimestamp", timestamp.dataType(), windows)); Expression[] fields = new Expression[] {value, timestamp, windows, pane}; - return nullSafe(pane, invoke(WindowedValue.class, "of", WINDOWED_VALUE, fields)); + return nullSafe(pane, invoke(WindowedValues.class, "of", WINDOWED_VALUE, fields)); } private static Expression serializeMutablePair( diff --git a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/AggregatorsTest.java b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/AggregatorsTest.java index d642a6e9866c..fa5312684fc1 100644 --- a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/AggregatorsTest.java +++ b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/AggregatorsTest.java @@ -39,8 +39,9 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Streams; import org.apache.spark.sql.Encoder; @@ -124,7 +125,7 @@ public void testMerge() { } private WindowedValue windowedValue(Integer value, Instant ts) { - return WindowedValue.of(value, ts, sliding.assignWindows(ts), PaneInfo.NO_FIRING); + return WindowedValues.of(value, ts, sliding.assignWindows(ts), PaneInfo.NO_FIRING); } } @@ -235,7 +236,7 @@ public void testMerge() { } private WindowedValue sessionValue(Integer value, Instant ts) { - return WindowedValue.of(value, ts, new IntervalWindow(ts, SESSIONS_GAP), PaneInfo.NO_FIRING); + return WindowedValues.of(value, ts, new IntervalWindow(ts, SESSIONS_GAP), PaneInfo.NO_FIRING); } private IntervalWindow sessionWindow(int fromMinutes) { diff --git a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValuesTest.java b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValuesTest.java index 3b828ee7d181..74d5a0292edb 100644 --- a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValuesTest.java +++ b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/batch/functions/SideInputValuesTest.java @@ -21,8 +21,8 @@ import static org.apache.beam.runners.spark.structuredstreaming.translation.helpers.EncoderHelpers.windowedValueEncoder; import static org.apache.beam.runners.spark.structuredstreaming.translation.utils.ScalaInterop.seqOf; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; -import static org.apache.beam.sdk.util.WindowedValue.getFullCoder; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.getFullCoder; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.assertj.core.api.Assertions.assertThat; import com.esotericsoftware.kryo.Kryo; @@ -36,7 +36,8 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.spark.serializer.KryoSerializer; import org.apache.spark.sql.Dataset; @@ -102,7 +103,7 @@ private static IntervalWindow intervalWindow(int start, int end) { } private static WindowedValue valueInWindows(T value, BoundedWindow... windows) { - return WindowedValue.of(value, Instant.EPOCH, Lists.list(windows), PaneInfo.NO_FIRING); + return WindowedValues.of(value, Instant.EPOCH, Lists.list(windows), PaneInfo.NO_FIRING); } public static class SparkKryo extends ExternalResource { diff --git a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java index ecb79d97b3e2..11c47ad03cb3 100644 --- a/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java +++ b/runners/spark/3/src/test/java/org/apache/beam/runners/spark/structuredstreaming/translation/helpers/EncoderHelpersTest.java @@ -64,9 +64,10 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -139,7 +140,7 @@ public void testBeamWindowedValueEncoderMappings() { BASIC_CASES.forEach( (coder, data) -> { List> windowed = - Lists.transform(data, WindowedValue::valueInGlobalWindow); + Lists.transform(data, WindowedValues::valueInGlobalWindow); Encoder encoder = windowedValueEncoder(encoderFor(coder), windowEnc); serializeAndDeserialize(windowed.get(0), (Encoder) encoder); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/SparkRunnerKryoRegistrator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/SparkRunnerKryoRegistrator.java index 44f8d6df683b..68c602ff7f59 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/SparkRunnerKryoRegistrator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/coders/SparkRunnerKryoRegistrator.java @@ -65,7 +65,8 @@ public void registerClasses(Kryo kryo) { try { kryo.register( - Class.forName("org.apache.beam.sdk.util.WindowedValue$TimestampedValueInGlobalWindow")); + Class.forName( + "org.apache.beam.sdk.values.WindowedValues$TimestampedValueInGlobalWindow")); kryo.register( Class.forName( "org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.HashBasedTable$Factory")); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java index c846ac20a626..e65dccd23f24 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SourceRDD.java @@ -35,7 +35,8 @@ import org.apache.beam.sdk.io.UnboundedSource; import org.apache.beam.sdk.metrics.MetricsContainer; import org.apache.beam.sdk.metrics.MetricsEnvironment; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.apache.spark.Dependency; import org.apache.spark.HashPartitioner; @@ -189,7 +190,7 @@ private boolean tryProduceNext() { checkState(next == null, "unexpected non-null value for next"); if (seekNext()) { next = - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp()); return SUCCESSFULLY_OBTAINED_NEXT; } else { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java index 34ef3331ae49..bea1557a7103 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/io/SparkUnboundedSource.java @@ -41,7 +41,8 @@ import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Splitter; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext$; @@ -120,8 +121,8 @@ public static UnboundedDataset re .register(); // output the actual (deserialized) stream. - WindowedValue.FullWindowedValueCoder coder = - WindowedValue.FullWindowedValueCoder.of( + WindowedValues.FullWindowedValueCoder coder = + WindowedValues.FullWindowedValueCoder.of( source.getOutputCoder(), GlobalWindow.Coder.INSTANCE); JavaDStream> readUnboundedStream = mapWithStateDStream diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java index b18b31a67463..1b9290ca7175 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/SparkGroupAlsoByWindowViaWindowSet.java @@ -51,11 +51,12 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Joiner; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Predicate; @@ -110,7 +111,7 @@ public void outputWindowedValue( final Instant timestamp, final Collection windows, final PaneInfo pane) { - windowedValues.add(WindowedValue.of(output, timestamp, windows, pane)); + windowedValues.add(WindowedValues.of(output, timestamp, windows, pane)); } private List>>> getWindowedValues() { diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java index b72481128f53..f995b47876fd 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/stateful/StateSpecFunctions.java @@ -37,7 +37,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.Stopwatch; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; @@ -164,15 +165,15 @@ public Tuple2, Metadata> apply( // read microbatch as a serialized collection. final List readValues = new ArrayList<>(); - WindowedValue.FullWindowedValueCoder coder = - WindowedValue.FullWindowedValueCoder.of( + WindowedValues.FullWindowedValueCoder coder = + WindowedValues.FullWindowedValueCoder.of( source.getOutputCoder(), GlobalWindow.Coder.INSTANCE); try { // measure how long a read takes per-partition. boolean finished = !microbatchReader.start(); while (!finished) { final WindowedValue wv = - WindowedValue.of( + WindowedValues.of( (T) microbatchReader.getCurrent(), microbatchReader.getCurrentTimestamp(), GlobalWindow.INSTANCE, diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java index 5cebc513e8b9..00e408cd8477 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/BoundedDataset.java @@ -24,8 +24,9 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaRDDLike; @@ -56,7 +57,7 @@ public class BoundedDataset implements Dataset { } BoundedDataset(Iterable values, JavaSparkContext jsc, Coder coder) { - this.windowedValues = Iterables.transform(values, WindowedValue::valueInGlobalWindow); + this.windowedValues = Iterables.transform(values, WindowedValues::valueInGlobalWindow); this.jsc = jsc; this.coder = coder; } @@ -64,8 +65,8 @@ public class BoundedDataset implements Dataset { @SuppressWarnings("ConstantConditions") public JavaRDD> getRDD() { if (rdd == null) { - WindowedValue.ValueOnlyWindowedValueCoder windowCoder = - WindowedValue.getValueOnlyCoder(coder); + WindowedValues.ValueOnlyWindowedValueCoder windowCoder = + WindowedValues.getValueOnlyCoder(coder); rdd = jsc.parallelize(CoderHelpers.toByteArrays(windowedValues, windowCoder)) .map(CoderHelpers.fromByteFunction(windowCoder)); @@ -73,7 +74,7 @@ public JavaRDD> getRDD() { return rdd; } - List getBytes(WindowedValue.WindowedValueCoder wvCoder) { + List getBytes(WindowedValues.WindowedValueCoder wvCoder) { if (clientBytes == null) { JavaRDDLike bytesRDD = rdd.map(CoderHelpers.toByteFunction(wvCoder)); clientBytes = bytesRDD.collect(); @@ -85,12 +86,12 @@ Iterable> getValues(PCollection pcollection) { if (windowedValues == null) { WindowFn windowFn = pcollection.getWindowingStrategy().getWindowFn(); Coder windowCoder = windowFn.windowCoder(); - final WindowedValue.WindowedValueCoder windowedValueCoder; + final WindowedValues.WindowedValueCoder windowedValueCoder; if (windowFn instanceof GlobalWindows) { - windowedValueCoder = WindowedValue.ValueOnlyWindowedValueCoder.of(pcollection.getCoder()); + windowedValueCoder = WindowedValues.ValueOnlyWindowedValueCoder.of(pcollection.getCoder()); } else { windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of(pcollection.getCoder(), windowCoder); + WindowedValues.FullWindowedValueCoder.of(pcollection.getCoder(), windowCoder); } JavaRDDLike bytesRDD = rdd.map(CoderHelpers.toByteFunction(windowedValueCoder)); List clientBytes = bytesRDD.collect(); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java index 34836cd6e7ae..a6b1f65571dc 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/DoFnRunnerWithMetrics.java @@ -27,7 +27,7 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** DoFnRunner decorator which registers {@link MetricsContainerImpl}. */ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java index 99f2a3e4c360..8adc78af59ef 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/EvaluationContext.java @@ -35,12 +35,13 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -208,7 +209,7 @@ private void putDataset( Coder coder = ((PCollection) pvalue).getCoder(); Coder wCoder = ((PCollection) pvalue).getWindowingStrategy().getWindowFn().windowCoder(); - dataset.cache(storageLevel(), WindowedValue.getFullCoder(coder, wCoder)); + dataset.cache(storageLevel(), WindowedValues.getFullCoder(coder, wCoder)); } datasets.put(pvalue, dataset); leaves.add(dataset); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java index 1d8901ed5ffc..03f7885f7e9f 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupCombineFunctions.java @@ -25,9 +25,9 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.Reshuffle; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; 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; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java index 14630fbb0a1f..3944cde5bee3 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctions.java @@ -30,9 +30,10 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; 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.Iterables; @@ -87,7 +88,7 @@ JavaRDD>>> groupByKeyAndWindow( Partitioner partitioner) { final Coder windowCoder = windowingStrategy.getWindowFn().windowCoder(); FullWindowedValueCoder> windowedKvCoder = - WindowedValue.FullWindowedValueCoder.of(KvCoder.of(keyCoder, valueCoder), windowCoder); + WindowedValues.FullWindowedValueCoder.of(KvCoder.of(keyCoder, valueCoder), windowCoder); JavaPairRDD windowInKey = bringWindowToKey( rdd, keyCoder, windowCoder, wv -> CoderHelpers.toByteArray(wv, windowedKvCoder)); @@ -131,7 +132,7 @@ static JavaPairRDD final W window = (W) Iterables.getOnlyElement(item.getWindows()); final byte[] windowBytes = CoderHelpers.toByteArray(window, windowCoder); WindowedValue> valueOut = - WindowedValue.of(item.getValue(), item.getTimestamp(), window, item.getPane()); + WindowedValues.of(item.getValue(), item.getTimestamp(), window, item.getPane()); final ByteArray windowedKey = new ByteArray(Bytes.concat(keyBytes, windowBytes)); return new Tuple2<>(windowedKey, mappingFn.apply(valueOut)); }); @@ -179,7 +180,7 @@ static class GroupByKeyIterator Iterator> inner, Coder keyCoder, WindowingStrategy windowingStrategy, - WindowedValue.FullWindowedValueCoder> windowedValueCoder) + WindowedValues.FullWindowedValueCoder> windowedValueCoder) throws Coder.NonDeterministicException { this.inner = Iterators.peekingIterator(inner); @@ -258,7 +259,7 @@ private WindowedValue> decodeItem(Tuple2 item) { final Instant timestamp = windowingStrategy.getTimestampCombiner().assign(window, windowedValue.getTimestamp()); // BEAM-7341: Elements produced by GbK are always ON_TIME and ONLY_FIRING - return WindowedValue.of( + return WindowedValues.of( KV.of(key, value), timestamp, window, PaneInfo.ON_TIME_AND_ONLY_FIRING); } } @@ -297,7 +298,7 @@ JavaRDD>>> groupByKeyInGlobalWindow( Iterators.transform( iter, kvs -> - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( KV.of( CoderHelpers.fromByteArray(kvs._1.getValue(), keyCoder), Iterables.transform( diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java index 353cf9d0ab90..01656810a33b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/MultiDoFnFunction.java @@ -39,10 +39,10 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; import org.apache.spark.api.java.function.PairFlatMapFunction; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ReifyTimestampsAndWindowsFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ReifyTimestampsAndWindowsFunction.java index dfb58c5c4c53..37f9d1b4e86b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ReifyTimestampsAndWindowsFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/ReifyTimestampsAndWindowsFunction.java @@ -17,9 +17,10 @@ */ package org.apache.beam.runners.spark.translation; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.spark.api.java.function.Function; /** @@ -32,7 +33,7 @@ public class ReifyTimestampsAndWindowsFunction public KV> call(WindowedValue> elem) throws Exception { return KV.of( elem.getValue().getKey(), - WindowedValue.of( + WindowedValues.of( elem.getValue().getValue(), elem.getTimestamp(), elem.getWindows(), elem.getPane())); } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SideInputMetadata.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SideInputMetadata.java index 6c407806812c..3e7cd6e700f1 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SideInputMetadata.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SideInputMetadata.java @@ -20,7 +20,7 @@ import java.io.Serializable; import org.apache.beam.runners.spark.util.SideInputBroadcast; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * Metadata class for side inputs in Spark runner. Contains serialized data, type information and diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAssignWindowFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAssignWindowFn.java index fd00cadf3d40..8c0e9a341266 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAssignWindowFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkAssignWindowFn.java @@ -22,7 +22,8 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Window.Assign; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.spark.api.java.function.Function; import org.joda.time.Instant; @@ -62,6 +63,6 @@ public BoundedWindow window() { return boundedWindow; } }); - return WindowedValue.of(element, timestamp, windows, PaneInfo.NO_FIRING); + return WindowedValues.of(element, timestamp, windows, PaneInfo.NO_FIRING); } } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java index e37941a35557..ba3aa0e4d24a 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkBatchPortablePipelineTranslator.java @@ -49,8 +49,6 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.NativeTransforms; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; @@ -58,6 +56,9 @@ import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -174,8 +175,8 @@ private static void translateGroupByKey( Coder inputValueCoder = inputKvCoder.getValueCoder(); WindowingStrategy windowingStrategy = getWindowingStrategy(inputId, components); WindowFn windowFn = windowingStrategy.getWindowFn(); - WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(inputValueCoder, windowFn.windowCoder()); + WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(inputValueCoder, windowFn.windowCoder()); JavaRDD>>> groupedByKeyAndWindow; Partitioner partitioner = getPartitioner(context); @@ -233,7 +234,7 @@ private static void translateExecutableStage( Coder> windowedInputCoder = instantiateCoder(inputPCollectionId, components); Coder valueCoder = - ((WindowedValue.FullWindowedValueCoder) windowedInputCoder).getValueCoder(); + ((WindowedValues.FullWindowedValueCoder) windowedInputCoder).getValueCoder(); // Stateful stages are only allowed of KV input to be able to group on the key if (!(valueCoder instanceof KvCoder)) { throw new IllegalStateException( @@ -247,8 +248,8 @@ private static void translateExecutableStage( Coder innerValueCoder = ((KvCoder) valueCoder).getValueCoder(); WindowingStrategy windowingStrategy = getWindowingStrategy(inputPCollectionId, components); WindowFn windowFn = windowingStrategy.getWindowFn(); - WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(innerValueCoder, windowFn.windowCoder()); + WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(innerValueCoder, windowFn.windowCoder()); JavaPairRDD>> groupedByKey = groupByKeyPair(inputDataset, keyCoder, wvCoder); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java index 1075ae0d2a7d..667fa7181bcb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkCombineFn.java @@ -49,11 +49,12 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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; @@ -244,7 +245,7 @@ public void merge( public Collection> extractOutput() { if (windowAccumulator != null) { return Collections.singletonList( - WindowedValue.of( + WindowedValues.of( windowAccumulator, accTimestamp, accWindow, PaneInfo.ON_TIME_AND_ONLY_FIRING)); } return Collections.emptyList(); @@ -309,7 +310,7 @@ public void add(WindowedValue value, SparkCombineFn context) throws E // merge this.map.put( window, - WindowedValue.of( + WindowedValues.of( context.combineFn.mergeAccumulators( Lists.newArrayList(thisAcc.getValue(), acc.getValue()), context.ctxtForValue(acc)), @@ -442,7 +443,7 @@ void mergeWindows(SparkCombineFn fn) throws Exception { toBeMerged.forEach(this.map::remove); this.map.put( mergeResult.getKey(), - WindowedValue.of( + WindowedValues.of( mergeResult.getValue(), mergedInstant, mergeResult.getKey(), @@ -504,7 +505,7 @@ static class WindowedAccumulatorCoder WindowedAccumulator.Type type) { this.toValue = toValue; - this.accumCoder = WindowedValue.FullWindowedValueCoder.of(accumCoder, windowCoder); + this.accumCoder = WindowedValues.FullWindowedValueCoder.of(accumCoder, windowCoder); this.windowComparator = windowComparator; this.wrap = IterableCoder.of(this.accumCoder); this.type = type; @@ -524,7 +525,7 @@ public void encode(WindowedAccumulator value, OutputS } else { outStream.write(1); accumCoder.encode( - WindowedValue.of( + WindowedValues.of( swwa.windowAccumulator, swwa.accTimestamp, swwa.accWindow, PaneInfo.NO_FIRING), outStream); } @@ -809,9 +810,6 @@ private WindowedAccumulator.Type getType(WindowingStrategy windowingStrate } private static BoundedWindow getWindow(WindowedValue value) { - if (value.isSingleWindowedValue()) { - return ((WindowedValue.SingleWindowedValue) value).getWindow(); - } return Iterables.getOnlyElement(value.getWindows()); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageExtractionFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageExtractionFunction.java index be76e5c0ae08..ae38d097e69d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageExtractionFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageExtractionFunction.java @@ -20,7 +20,7 @@ import java.util.Collections; import java.util.Iterator; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.api.java.function.FlatMapFunction; @SuppressWarnings({ diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java index 4863f7f1f163..757740e2df5a 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunction.java @@ -59,10 +59,10 @@ import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.apache.spark.api.java.function.FlatMapFunction; import org.apache.spark.broadcast.Broadcast; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java index cff2cf6bc684..ea620bfe6605 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkGroupAlsoByWindowViaOutputBufferFn.java @@ -35,10 +35,11 @@ import org.apache.beam.runners.core.triggers.TriggerStateMachines; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TriggerTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.spark.api.java.function.FlatMapFunction; import org.joda.time.Instant; @@ -146,7 +147,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputs.add(WindowedValue.of(output, timestamp, windows, pane)); + outputs.add(WindowedValues.of(output, timestamp, windows, pane)); } @Override diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.java index 4b4d23b0c47c..d202ac04cac0 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessor.java @@ -34,8 +34,8 @@ import org.apache.beam.runners.core.TimerInternals; import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; 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.util.concurrent.ThreadFactoryBuilder; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java index 4cc25bd6ffa2..6802cb5aa8e7 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkPCollectionView.java @@ -23,8 +23,8 @@ import org.apache.beam.runners.spark.coders.CoderHelpers; import org.apache.beam.runners.spark.util.SideInputBroadcast; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.api.java.JavaSparkContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java index a252cc044fcb..4850f886241b 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/SparkStreamingPortablePipelineTranslator.java @@ -47,13 +47,14 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.util.construction.graph.PipelineNode; import org.apache.beam.sdk.util.construction.graph.PipelineNode.PTransformNode; import org.apache.beam.sdk.util.construction.graph.QueryablePipeline; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -142,14 +143,14 @@ private static void translateImpulse( Iterable> windowedValues = Collections.singletonList( - WindowedValue.of( + WindowedValues.of( new byte[0], BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING)); - WindowedValue.FullWindowedValueCoder windowCoder = - WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.FullWindowedValueCoder windowCoder = + WindowedValues.FullWindowedValueCoder.of(ByteArrayCoder.of(), GlobalWindow.Coder.INSTANCE); JavaRDD> emptyByteArrayRDD = context .getSparkContext() @@ -184,13 +185,13 @@ private static void translateGroupByKey( UnboundedDataset> inputDataset = (UnboundedDataset>) context.popDataset(inputId); List streamSources = inputDataset.getStreamSources(); - WindowedValue.WindowedValueCoder> inputCoder = + WindowedValues.WindowedValueCoder> inputCoder = getWindowedValueCoder(inputId, components); KvCoder inputKvCoder = (KvCoder) inputCoder.getValueCoder(); WindowingStrategy windowingStrategy = getWindowingStrategy(inputId, components); WindowFn windowFn = windowingStrategy.getWindowFn(); - WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of( + WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of( inputKvCoder.getValueCoder(), windowFn.windowCoder()); JavaDStream>>> outStream = @@ -238,7 +239,7 @@ private static void translateExecutableStage( "Side inputs to executable stage are currently unsupported."); } ImmutableMap< - String, Tuple2>, WindowedValue.WindowedValueCoder>> + String, Tuple2>, WindowedValues.WindowedValueCoder>> broadcastVariables = ImmutableMap.copyOf(new HashMap<>()); SparkExecutableStageFunction function = @@ -344,7 +345,7 @@ private static void translateReshuffle( UnboundedDataset inputDataset = (UnboundedDataset) context.popDataset(inputId); List streamSources = inputDataset.getStreamSources(); JavaDStream> dStream = inputDataset.getDStream(); - WindowedValue.WindowedValueCoder coder = + WindowedValues.WindowedValueCoder coder = getWindowedValueCoder(inputId, pipeline.getComponents()); JavaDStream> reshuffledStream = diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java index 1fea8b9329c6..12448b19e1bb 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java @@ -61,7 +61,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformMatchers; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; @@ -72,6 +71,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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.AbstractIterator; @@ -144,8 +145,8 @@ public void evaluate(GroupByKey transform, EvaluationContext context) { // --- coders. final Coder keyCoder = coder.getKeyCoder(); - final WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); + final WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); JavaRDD>>> groupedByKey; Partitioner partitioner = getPartitioner(context); @@ -224,7 +225,7 @@ public void evaluate( JavaRDD>> outRDD = inRDD.map( in -> - WindowedValue.of( + WindowedValues.of( KV.of( in.getValue().getKey(), combineFn.apply( @@ -256,8 +257,8 @@ public void evaluate(Combine.Globally transform, EvaluationCont final CombineWithContext.CombineFnWithContext combineFn = (CombineWithContext.CombineFnWithContext) CombineFnUtil.toFnWithContext(transform.getFn()); - final WindowedValue.FullWindowedValueCoder wvoCoder = - WindowedValue.FullWindowedValueCoder.of( + final WindowedValues.FullWindowedValueCoder wvoCoder = + WindowedValues.FullWindowedValueCoder.of( oCoder, windowingStrategy.getWindowFn().windowCoder()); final boolean hasDefault = transform.isInsertDefault(); @@ -299,7 +300,7 @@ public void evaluate(Combine.Globally transform, EvaluationCont outRdd = jsc.parallelize(Lists.newArrayList(CoderHelpers.toByteArray(defaultValue, oCoder))) .map(CoderHelpers.fromByteFunction(oCoder)) - .map(WindowedValue::valueInGlobalWindow); + .map(WindowedValues::valueInGlobalWindow); } else { outRdd = jsc.emptyRDD(); } @@ -557,8 +558,8 @@ private static JavaPairRDD, WindowedValue> statef boolean requiresSortedInput) { Coder keyCoder = kvCoder.getKeyCoder(); - final WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(kvCoder.getValueCoder(), windowCoder); + final WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(kvCoder.getValueCoder(), windowCoder); if (!requiresSortedInput) { return GroupCombineFunctions.groupByKeyOnly(kvInRDD, keyCoder, wvCoder, partitioner) @@ -697,7 +698,7 @@ protected WindowedValue> computeNext() { if (Arrays.equals(prefix, keyPart)) { WindowedValue wv = CoderHelpers.fromByteArray(read._2(), wvCoder); consumed(); - return WindowedValue.of( + return WindowedValues.of( KV.of(key, wv.getValue()), wv.getTimestamp(), wv.getWindows(), wv.getPane()); } } @@ -786,7 +787,7 @@ public void evaluate( Coder>> coderInternal = (Coder) IterableCoder.of( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( output.getCoderInternal(), output.getWindowingStrategyInternal().getWindowFn().windowCoder())); @@ -817,8 +818,8 @@ public void evaluate(Reshuffle transform, EvaluationContext context) { @SuppressWarnings("unchecked") final WindowFn windowFn = (WindowFn) windowingStrategy.getWindowFn(); - final WindowedValue.WindowedValueCoder> wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder, windowFn.windowCoder()); + final WindowedValues.WindowedValueCoder> wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder, windowFn.windowCoder()); JavaRDD>> reshuffled = GroupCombineFunctions.reshuffle(inRDD, wvCoder); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java index f2455e64b956..8d66125efbc6 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TranslationUtils.java @@ -40,11 +40,12 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterators; @@ -102,7 +103,7 @@ public CombineGroupedValues(SparkCombineFn, InputT, ?, OutputT> fn @Override public WindowedValue> call(WindowedValue>> windowedKv) throws Exception { - return WindowedValue.of( + return WindowedValues.of( KV.of( windowedKv.getValue().getKey(), fn.getCombineFn() @@ -414,7 +415,7 @@ public static Map, Coder>> getTupleTagCoders( pCollection.getWindowingStrategy().getWindowFn().windowCoder(); @SuppressWarnings("unchecked") Coder> windowedValueCoder = - (Coder>) (Coder) WindowedValue.getFullCoder(coder, wCoder); + (Coder>) (Coder) WindowedValues.getFullCoder(coder, wCoder); coderMap.put(output.getKey(), windowedValueCoder); } return coderMap; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/ParDoStateUpdateFn.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/ParDoStateUpdateFn.java index b9c7e7d6d63d..1be4042a151d 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/ParDoStateUpdateFn.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/ParDoStateUpdateFn.java @@ -47,10 +47,11 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.apache.spark.streaming.State; @@ -93,7 +94,7 @@ public class ParDoStateUpdateFn, O private final String stepName; private final DoFn doFn; private final Coder keyCoder; - private final WindowedValue.FullWindowedValueCoder wvCoder; + private final WindowedValues.FullWindowedValueCoder wvCoder; private transient boolean wasSetupCalled; private final SerializablePipelineOptions options; private final TupleTag mainOutputTag; @@ -116,7 +117,7 @@ public ParDoStateUpdateFn( String stepName, DoFn doFn, Coder keyCoder, - WindowedValue.FullWindowedValueCoder wvCoder, + WindowedValues.FullWindowedValueCoder wvCoder, SerializablePipelineOptions options, TupleTag mainOutputTag, List> additionalOutputTags, @@ -257,8 +258,8 @@ public TimerInternals timerInternals() { (Coder) outputCoders.get(tupleTag); @SuppressWarnings("nullness") - final WindowedValue.FullWindowedValueCoder outputWindowCoder = - WindowedValue.FullWindowedValueCoder.of(outputCoder, windowCoder); + final WindowedValues.FullWindowedValueCoder outputWindowCoder = + WindowedValues.FullWindowedValueCoder.of(outputCoder, windowCoder); return Tuple2.apply( tupleTag, diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StatefulStreamingParDoEvaluator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StatefulStreamingParDoEvaluator.java index c135d6edbf8a..fdd5c180c21e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StatefulStreamingParDoEvaluator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StatefulStreamingParDoEvaluator.java @@ -45,12 +45,13 @@ import org.apache.beam.sdk.transforms.reflect.DoFnSignature; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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; @@ -126,8 +127,8 @@ public void evaluate( final Coder keyCoder = inputCoder.getKeyCoder(); final Coder valueCoder = inputCoder.getValueCoder(); - final WindowedValue.FullWindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(valueCoder, windowFn.windowCoder()); + final WindowedValues.FullWindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(valueCoder, windowFn.windowCoder()); final MetricsContainerStepMapAccumulator metricsAccum = MetricsAccumulator.getInstance(); final Map, KV, SideInputBroadcast>> sideInputs = @@ -173,7 +174,7 @@ public void evaluate( stepName, doFn, keyCoder, - (WindowedValue.FullWindowedValueCoder) wvCoder, + (WindowedValues.FullWindowedValueCoder) wvCoder, options, transform.getMainOutputTag(), transform.getAdditionalOutputTags().getAll(), @@ -198,7 +199,7 @@ public void evaluate( final WindowedValue windowedValue = CoderHelpers.fromByteArray( tuple._2(), - WindowedValue.FullWindowedValueCoder.of( + WindowedValues.FullWindowedValueCoder.of( outputCoder, windowFn.windowCoder())); return Tuple2.apply(tuple._1(), windowedValue); })); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java index b505d9bb3d41..0963a3c7a750 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/StreamingTransformTranslator.java @@ -76,7 +76,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.SplittableParDo; @@ -87,6 +86,8 @@ import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -140,7 +141,7 @@ public void evaluate(Impulse transform, EvaluationContext context) { context .getSparkContext() .parallelize( - Collections.singletonList(WindowedValue.valueInGlobalWindow(new byte[0]))); + Collections.singletonList(WindowedValues.valueInGlobalWindow(new byte[0]))); ConstantInputDStream> inputStream = new ConstantInputDStream<>(context.getStreamingContext().ssc(), rdd.rdd(), classTag); JavaDStream> stream = new JavaDStream<>(inputStream, classTag); @@ -227,8 +228,8 @@ public void evaluate(CreateStream transform, EvaluationContext context) { private Queue>> buildRdds( Queue>> batches, JavaStreamingContext jssc, Coder coder) { - final WindowedValue.FullWindowedValueCoder windowCoder = - WindowedValue.FullWindowedValueCoder.of(coder, GlobalWindow.Coder.INSTANCE); + final WindowedValues.FullWindowedValueCoder windowCoder = + WindowedValues.FullWindowedValueCoder.of(coder, GlobalWindow.Coder.INSTANCE); final Queue>> rddQueue = new LinkedBlockingQueue<>(); @@ -237,7 +238,7 @@ private Queue>> buildRdds( StreamSupport.stream(timestampedValues.spliterator(), false) .map( timestampedValue -> - WindowedValue.of( + WindowedValues.of( timestampedValue.getValue(), timestampedValue.getTimestamp(), GlobalWindow.INSTANCE, @@ -368,8 +369,8 @@ public void evaluate(GroupByKey transform, EvaluationContext context) { final WindowFn windowFn = (WindowFn) windowingStrategy.getWindowFn(); // --- coders. - final WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); + final WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); JavaDStream>>> outStream = SparkGroupAlsoByWindowViaWindowSet.groupByKeyAndWindow( @@ -410,7 +411,7 @@ public void evaluate( Coder> coderInternal = (Coder) - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( ListCoder.of(output.getCoderInternal()), output.getWindowingStrategyInternal().getWindowFn().windowCoder()); @@ -634,8 +635,8 @@ public void evaluate(Reshuffle transform, EvaluationContext context) { @SuppressWarnings("unchecked") final WindowFn windowFn = (WindowFn) windowingStrategy.getWindowFn(); - final WindowedValue.WindowedValueCoder> wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder, windowFn.windowCoder()); + final WindowedValues.WindowedValueCoder> wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder, windowFn.windowCoder()); JavaDStream>> reshuffledStream = dStream.transform(rdd -> GroupCombineFunctions.reshuffle(rdd, wvCoder)); diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java index e0bda8ba9491..abf2e7c3242e 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/TestDStream.java @@ -32,8 +32,9 @@ import org.apache.beam.sdk.testing.TestStream.WatermarkEvent; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.Preconditions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TimestampedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.JavaSparkContext$; @@ -61,7 +62,7 @@ public class TestDStream extends InputDStream> { public TestDStream(TestStream test, StreamingContext ssc) { super(ssc, classTag()); - this.coder = WindowedValue.getFullCoder(test.getValueCoder(), GlobalWindow.Coder.INSTANCE); + this.coder = WindowedValues.getFullCoder(test.getValueCoder(), GlobalWindow.Coder.INSTANCE); this.events = test.getEvents(); } @@ -126,7 +127,7 @@ private RDD> buildRdd(ElementEvent event) { List binaryData = new ArrayList<>(); for (TimestampedValue elem : event.getElements()) { WindowedValue wv = - WindowedValue.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp()); + WindowedValues.timestampedValueInGlobalWindow(elem.getValue(), elem.getTimestamp()); binaryData.add(CoderHelpers.toByteArray(wv, coder)); } diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java index ec7e00d24fcf..3c4674cd2260 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/UnboundedDataset.java @@ -23,7 +23,7 @@ import org.apache.beam.runners.spark.translation.Dataset; import org.apache.beam.runners.spark.translation.TranslationUtils; import org.apache.beam.sdk.coders.Coder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.JavaDStream; import org.slf4j.Logger; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java index fef2d2ec675e..d9c40be501c0 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/streaming/WatermarkSyncedDStream.java @@ -22,7 +22,7 @@ import java.util.Queue; import java.util.concurrent.TimeUnit; import org.apache.beam.runners.spark.util.GlobalWatermarkHolder; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Stopwatch; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.util.concurrent.Uninterruptibles; import org.apache.spark.api.java.JavaRDD; diff --git a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java index 4e24d7e50e6a..414f2abc01a9 100644 --- a/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java +++ b/runners/spark/src/main/java/org/apache/beam/runners/spark/util/SparkSideInputReader.java @@ -34,10 +34,10 @@ import org.apache.beam.sdk.transforms.Materializations.MultimapView; import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctionsTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctionsTest.java index fd299924af91..f4d154fa772f 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctionsTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/GroupNonMergingWindowsFunctionsTest.java @@ -34,9 +34,10 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.primitives.Bytes; import org.joda.time.Duration; @@ -123,8 +124,8 @@ private GroupByKeyIterator createG throws Coder.NonDeterministicException { StringUtf8Coder keyCoder = StringUtf8Coder.of(); - final WindowedValue.FullWindowedValueCoder> winValCoder = - WindowedValue.getFullCoder( + final WindowedValues.FullWindowedValueCoder> winValCoder = + WindowedValues.getFullCoder( KvCoder.of(StringUtf8Coder.of(), VarIntCoder.of()), winStrategy.getWindowFn().windowCoder()); @@ -157,7 +158,7 @@ static ItemFactory forWindow( } private final Coder keyCoder; - private final WindowedValue.FullWindowedValueCoder> winValCoder; + private final WindowedValues.FullWindowedValueCoder> winValCoder; private final byte[] windowBytes; private final W window; @@ -178,7 +179,7 @@ private Tuple2 create(K key, V value) { byte[] windowedValue = CoderHelpers.toByteArray( - WindowedValue.of( + WindowedValues.of( KV.of(key, value), Instant.now(), window, PaneInfo.ON_TIME_AND_ONLY_FIRING), winValCoder); return new Tuple2<>(kaw, windowedValue); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java index 9cb4b44c897c..f56a20b49151 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkCombineFnTest.java @@ -39,8 +39,9 @@ import org.apache.beam.sdk.transforms.windowing.TimestampCombiner; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; 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.Lists; @@ -233,7 +234,7 @@ public void testGlobalWindowMergeAccumulatorsWithEarliestCombiner() throws Excep WindowedValue> first = input("key", 1, ts); WindowedValue> second = input("key", 2, ts); WindowedValue> third = input("key", 3, ts); - WindowedValue accumulator = WindowedValue.valueInGlobalWindow(0L); + WindowedValue accumulator = WindowedValues.valueInGlobalWindow(0L); SparkCombineFn.SingleWindowWindowedAccumulator, Integer, Long> acc1 = SparkCombineFn.SingleWindowWindowedAccumulator.create(KV::getValue, accumulator); SparkCombineFn.SingleWindowWindowedAccumulator, Integer, Long> acc2 = @@ -293,7 +294,7 @@ private WindowedValue inputValue(V value, Instant timestamp, WindowFn cast = (WindowFn) windowFn; - return WindowedValue.of( + return WindowedValues.of( value, timestamp, cast.assignWindows(assignContext(cast, value, timestamp)), diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java index 7276820e99e8..98601389f5c9 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkExecutableStageFunctionTest.java @@ -55,9 +55,10 @@ import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Timer; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.junit.Before; import org.junit.Test; @@ -116,7 +117,7 @@ public void sdkErrorsSurfaceOnClose() throws Exception { SparkExecutableStageFunction function = getFunction(Collections.emptyMap()); doThrow(new Exception()).when(remoteBundle).close(); List> inputs = new ArrayList<>(); - inputs.add(WindowedValue.valueInGlobalWindow(0)); + inputs.add(WindowedValues.valueInGlobalWindow(0)); function.call(inputs.iterator()); } @@ -132,9 +133,9 @@ public void expectedInputsAreSent() throws Exception { FnDataReceiver> receiver = Mockito.mock(FnDataReceiver.class); when(bundle.getInputReceivers()).thenReturn(ImmutableMap.of(inputId, receiver)); - WindowedValue one = WindowedValue.valueInGlobalWindow(1); - WindowedValue two = WindowedValue.valueInGlobalWindow(2); - WindowedValue three = WindowedValue.valueInGlobalWindow(3); + WindowedValue one = WindowedValues.valueInGlobalWindow(1); + WindowedValue two = WindowedValues.valueInGlobalWindow(2); + WindowedValue three = WindowedValues.valueInGlobalWindow(3); function.call(Arrays.asList(one, two, three).iterator()); verify(receiver).accept(one); @@ -145,9 +146,9 @@ public void expectedInputsAreSent() throws Exception { @Test public void outputsAreTaggedCorrectly() throws Exception { - WindowedValue three = WindowedValue.valueInGlobalWindow(3); - WindowedValue four = WindowedValue.valueInGlobalWindow(4); - WindowedValue five = WindowedValue.valueInGlobalWindow(5); + WindowedValue three = WindowedValues.valueInGlobalWindow(3); + WindowedValue four = WindowedValues.valueInGlobalWindow(4); + WindowedValue five = WindowedValues.valueInGlobalWindow(5); Map outputTagMap = ImmutableMap.of( "one", 1, @@ -230,7 +231,7 @@ public void close() {} SparkExecutableStageFunction function = getFunction(outputTagMap); List> inputs = new ArrayList<>(); - inputs.add(WindowedValue.valueInGlobalWindow(0)); + inputs.add(WindowedValues.valueInGlobalWindow(0)); Iterator iterator = function.call(inputs.iterator()); Iterable iterable = () -> iterator; @@ -244,7 +245,7 @@ public void close() {} public void testStageBundleClosed() throws Exception { SparkExecutableStageFunction function = getFunction(Collections.emptyMap()); List> inputs = new ArrayList<>(); - inputs.add(WindowedValue.valueInGlobalWindow(0)); + inputs.add(WindowedValues.valueInGlobalWindow(0)); function.call(inputs.iterator()); verify(stageBundleFactory).getBundle(any(), any(), any(), any(BundleProgressHandler.class)); verify(stageBundleFactory).getProcessBundleDescriptor(); diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java index be2b788f12f5..c6ccec7bf451 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/SparkInputDataProcessorTest.java @@ -35,8 +35,9 @@ import org.apache.beam.sdk.state.TimeDomain; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.joda.time.Instant; @@ -101,7 +102,7 @@ public void testBoundedProcessBlocksOnMaxInputsUntilTheyAreConsumed() throws Exc setUpCtx(processor.getOutputManager(), desiredWriteCount, writeCount); Iterator> input = - Lists.newArrayList(WindowedValue.valueInGlobalWindow("tick")).iterator(); + Lists.newArrayList(WindowedValues.valueInGlobalWindow("tick")).iterator(); Iterator result = processor.createOutputIterator(input, ctx); CountDownLatch maxReached = new CountDownLatch(1); @@ -146,7 +147,7 @@ public void testUnboundedProcessWritesAllInputsAndNotBlock() { setUpCtx(processor.getOutputManager(), desiredWriteCount, writeCount); Iterator> input = - Lists.newArrayList(WindowedValue.valueInGlobalWindow("tick")).iterator(); + Lists.newArrayList(WindowedValues.valueInGlobalWindow("tick")).iterator(); Iterator result = processor.createOutputIterator(input, ctx); // this will trigger input processing via doFn @@ -168,7 +169,7 @@ public void testBoundedProcessLifecycle() { SparkProcessContext ctx = setUpCtx(processor.getOutputManager(), desiredWriteCount, writeCount); - WindowedValue value = WindowedValue.valueInGlobalWindow("tick"); + WindowedValue value = WindowedValues.valueInGlobalWindow("tick"); Iterator> input = Lists.newArrayList(value).iterator(); Iterator result = processor.createOutputIterator(input, ctx); @@ -193,7 +194,7 @@ public void testUnboundedProcessLifecycle() { SparkProcessContext ctx = setUpCtx(processor.getOutputManager(), desiredWriteCount, writeCount); - WindowedValue value = WindowedValue.valueInGlobalWindow("tick"); + WindowedValue value = WindowedValues.valueInGlobalWindow("tick"); Iterator> input = Lists.newArrayList(value).iterator(); Iterator result = processor.createOutputIterator(input, ctx); @@ -269,7 +270,7 @@ class TestDoFn extends DoFn { @DoFn.ProcessElement public void processElement(@Element String value) { for (int i = 0; i < desiredCount; i++) { - output.output(new TupleTag<>("key"), WindowedValue.valueInGlobalWindow(value + "_" + i)); + output.output(new TupleTag<>("key"), WindowedValues.valueInGlobalWindow(value + "_" + i)); producedCount.incrementAndGet(); } } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java index 2faf38a106df..f5ad228152fc 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/TransformTranslatorTest.java @@ -43,12 +43,13 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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; @@ -81,8 +82,8 @@ public void testIteratorFlatten() { @Test public void testSplitBySameKey() { VarIntCoder coder = VarIntCoder.of(); - WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder, GlobalWindow.Coder.INSTANCE); + WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder, GlobalWindow.Coder.INSTANCE); Instant now = Instant.now(); List window = Collections.singletonList(GlobalWindow.INSTANCE); PaneInfo paneInfo = PaneInfo.NO_FIRING; @@ -90,23 +91,25 @@ public void testSplitBySameKey() { Arrays.asList( new Tuple2( new ByteArray(CoderHelpers.toByteArrayWithTs(1, coder, now)), - CoderHelpers.toByteArray(WindowedValue.of(1, now, window, paneInfo), wvCoder)), + CoderHelpers.toByteArray(WindowedValues.of(1, now, window, paneInfo), wvCoder)), new Tuple2( new ByteArray( CoderHelpers.toByteArrayWithTs(1, coder, now.plus(Duration.millis(1)))), CoderHelpers.toByteArray( - WindowedValue.of(2, now.plus(Duration.millis(1)), window, paneInfo), wvCoder))); + WindowedValues.of(2, now.plus(Duration.millis(1)), window, paneInfo), + wvCoder))); List> secondKey = Arrays.asList( new Tuple2( new ByteArray(CoderHelpers.toByteArrayWithTs(2, coder, now)), - CoderHelpers.toByteArray(WindowedValue.of(3, now, window, paneInfo), wvCoder)), + CoderHelpers.toByteArray(WindowedValues.of(3, now, window, paneInfo), wvCoder)), new Tuple2( new ByteArray( CoderHelpers.toByteArrayWithTs(2, coder, now.plus(Duration.millis(2)))), CoderHelpers.toByteArray( - WindowedValue.of(4, now.plus(Duration.millis(2)), window, paneInfo), wvCoder))); + WindowedValues.of(4, now.plus(Duration.millis(2)), window, paneInfo), + wvCoder))); Iterable> concat = Iterables.concat(firstKey, secondKey); Iterator>>> keySplit; @@ -120,15 +123,15 @@ public void testSplitBySameKey() { // first key assertEquals( Arrays.asList( - WindowedValue.of(KV.of(1, 1), now, window, paneInfo), - WindowedValue.of(KV.of(1, 2), now.plus(Duration.millis(1)), window, paneInfo)), + WindowedValues.of(KV.of(1, 1), now, window, paneInfo), + WindowedValues.of(KV.of(1, 2), now.plus(Duration.millis(1)), window, paneInfo)), list); } else { // second key assertEquals( Arrays.asList( - WindowedValue.of(KV.of(2, 3), now, window, paneInfo), - WindowedValue.of(KV.of(2, 4), now.plus(Duration.millis(2)), window, paneInfo)), + WindowedValues.of(KV.of(2, 3), now, window, paneInfo), + WindowedValues.of(KV.of(2, 4), now.plus(Duration.millis(2)), window, paneInfo)), list); } } diff --git a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/ValueAndCoderLazySerializableTest.java b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/ValueAndCoderLazySerializableTest.java index 450d3e0bb013..0f2ef2ce5f45 100644 --- a/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/ValueAndCoderLazySerializableTest.java +++ b/runners/spark/src/test/java/org/apache/beam/runners/spark/translation/ValueAndCoderLazySerializableTest.java @@ -32,7 +32,8 @@ import org.apache.beam.sdk.coders.IterableCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.joda.time.Instant; import org.junit.Test; @@ -48,8 +49,8 @@ public void serializableAccumulatorSerializationTest() Iterable> accumulatedValue = Arrays.asList(winVal(0), winVal(1), winVal(3), winVal(4)); - final WindowedValue.FullWindowedValueCoder wvaCoder = - WindowedValue.FullWindowedValueCoder.of( + final WindowedValues.FullWindowedValueCoder wvaCoder = + WindowedValues.FullWindowedValueCoder.of( BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); final IterableCoder> iterAccumCoder = IterableCoder.of(wvaCoder); @@ -74,8 +75,8 @@ public void serializableAccumulatorKryoTest() { Iterable> accumulatedValue = Arrays.asList(winVal(0), winVal(1), winVal(3), winVal(4)); - final WindowedValue.FullWindowedValueCoder wvaCoder = - WindowedValue.FullWindowedValueCoder.of( + final WindowedValues.FullWindowedValueCoder wvaCoder = + WindowedValues.FullWindowedValueCoder.of( BigEndianIntegerCoder.of(), GlobalWindow.Coder.INSTANCE); final IterableCoder> iterAccumCoder = IterableCoder.of(wvaCoder); @@ -104,6 +105,6 @@ public void serializableAccumulatorKryoTest() { } private WindowedValue winVal(T val) { - return WindowedValue.of(val, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); + return WindowedValues.of(val, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); } } diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2BatchTranslationContext.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2BatchTranslationContext.java index 0acfe6af5f79..628425a52e45 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2BatchTranslationContext.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2BatchTranslationContext.java @@ -20,8 +20,8 @@ import edu.iu.dsc.tws.tset.env.BatchTSetEnvironment; import edu.iu.dsc.tws.tset.sets.batch.BatchTSetImpl; import edu.iu.dsc.tws.tset.sets.batch.SinkTSet; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.WindowedValue; /** Twister2BatchTranslationContext. */ public class Twister2BatchTranslationContext extends Twister2TranslationContext { diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java index 7102e8f8ef0e..a194d86f7460 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/Twister2TranslationContext.java @@ -29,11 +29,11 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.runners.AppliedPTransform; import org.apache.beam.sdk.transforms.PTransform; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.TransformInputs; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** Twister2TranslationContext. */ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java index 4525a67b28ea..1bd866d07130 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2BoundedSource.java @@ -34,10 +34,11 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.ReadTranslation; import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** Twister2 wrapper for Bounded Source. */ @SuppressWarnings({ @@ -177,7 +178,7 @@ private boolean tryProduceNext() { try { if (seekNext()) { next = - WindowedValue.timestampedValueInGlobalWindow( + WindowedValues.timestampedValueInGlobalWindow( reader.getCurrent(), reader.getCurrentTimestamp()); return SUCCESSFULLY_OBTAINED_NEXT; } else { diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2EmptySource.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2EmptySource.java index c15065024fc7..8e875e132a1b 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2EmptySource.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translation/wrappers/Twister2EmptySource.java @@ -18,7 +18,7 @@ package org.apache.beam.runners.twister2.translation.wrappers; import edu.iu.dsc.tws.api.tset.fn.BaseSourceFunc; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * Empty Source wrapper. diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/AssignWindowTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/AssignWindowTranslatorBatch.java index dbb5f47db4b2..39cba2ed2ddb 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/AssignWindowTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/AssignWindowTranslatorBatch.java @@ -26,7 +26,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; /** Assign Window translator. */ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/FlattenTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/FlattenTranslatorBatch.java index 00729e8dbc15..4cf5d347220b 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/FlattenTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/FlattenTranslatorBatch.java @@ -28,9 +28,9 @@ import org.apache.beam.runners.twister2.translation.wrappers.Twister2EmptySource; import org.apache.beam.runners.twister2.translators.BatchTransformTranslator; import org.apache.beam.sdk.transforms.Flatten; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PValue; +import org.apache.beam.sdk.values.WindowedValue; /** Flatten translator. */ @SuppressWarnings({ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/GroupByKeyTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/GroupByKeyTranslatorBatch.java index a1ec33785a69..78c90b966ac7 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/GroupByKeyTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/GroupByKeyTranslatorBatch.java @@ -33,9 +33,10 @@ import org.apache.beam.sdk.transforms.GroupByKey; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; /** GroupByKey translator. */ @@ -53,8 +54,8 @@ public void translateNode(GroupByKey transform, Twister2BatchTranslationCo WindowingStrategy windowingStrategy = input.getWindowingStrategy(); WindowFn, BoundedWindow> windowFn = (WindowFn, BoundedWindow>) windowingStrategy.getWindowFn(); - final WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); + final WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder.getValueCoder(), windowFn.windowCoder()); KeyedTSet keyedTSet = inputTTSet.mapToTuple(new MapToTupleFunction(inputKeyCoder, wvCoder)); diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ImpulseTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ImpulseTranslatorBatch.java index fa876c745027..afc325e06fc4 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ImpulseTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ImpulseTranslatorBatch.java @@ -24,8 +24,8 @@ import org.apache.beam.runners.twister2.translators.BatchTransformTranslator; import org.apache.beam.runners.twister2.translators.functions.ImpulseSource; import org.apache.beam.sdk.transforms.Impulse; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; /** Impulse translator. */ public class ImpulseTranslatorBatch implements BatchTransformTranslator { diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/PCollectionViewTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/PCollectionViewTranslatorBatch.java index 3b17bdac9e91..d6c69c8a6324 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/PCollectionViewTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/PCollectionViewTranslatorBatch.java @@ -32,9 +32,10 @@ import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.View; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CreatePCollectionViewTranslation; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; /** PCollectionView translator. */ @@ -74,8 +75,8 @@ public void translateNode( case Materializations.MULTIMAP_MATERIALIZATION_URN: KvCoder kvCoder = (KvCoder) coder; final Coder keyCoder = kvCoder.getKeyCoder(); - final WindowedValue.WindowedValueCoder kvwvCoder = - WindowedValue.FullWindowedValueCoder.of( + final WindowedValues.WindowedValueCoder kvwvCoder = + WindowedValues.FullWindowedValueCoder.of( kvCoder.getValueCoder(), windowFn.windowCoder()); BatchTSet> multimapMaterialization = inputDataSet @@ -86,8 +87,8 @@ public void translateNode( context.setSideInputDataSet(input.getTagInternal().getId(), multimapMaterialization); break; case Materializations.ITERABLE_MATERIALIZATION_URN: - final WindowedValue.WindowedValueCoder wvCoder = - WindowedValue.FullWindowedValueCoder.of(coder, windowFn.windowCoder()); + final WindowedValues.WindowedValueCoder wvCoder = + WindowedValues.FullWindowedValueCoder.of(coder, windowFn.windowCoder()); BatchTSet> iterableMaterialization = inputDataSet .direct() diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java index 97aeff608d7a..9230c6efd868 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ParDoMultiOutputTranslatorBatch.java @@ -35,11 +35,11 @@ import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.join.RawUnionValue; import org.apache.beam.sdk.transforms.reflect.DoFnSignatures; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Maps; diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ReadSourceTranslatorBatch.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ReadSourceTranslatorBatch.java index 14e858fe5108..ac071db64815 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ReadSourceTranslatorBatch.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/batch/ReadSourceTranslatorBatch.java @@ -24,9 +24,9 @@ import org.apache.beam.runners.twister2.translation.wrappers.Twister2BoundedSource; import org.apache.beam.runners.twister2.translators.BatchTransformTranslator; import org.apache.beam.sdk.io.BoundedSource; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.SplittableParDo; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; /** Source translator. */ public class ReadSourceTranslatorBatch diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java index cc7b9d9a5373..ad235a72d2e5 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/AssignWindowsFunction.java @@ -30,10 +30,11 @@ import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; /** Assign Windows function. */ @@ -69,7 +70,7 @@ public void compute(Iterator> input, RecordCollector> map(Tuple input) { WindowedValue> element; if (value == null) { - value = WindowedValue.valueInGlobalWindow(null); + value = WindowedValues.valueInGlobalWindow(null); } element = - WindowedValue.of( + WindowedValues.of( KV.of(key, value.getValue()), value.getTimestamp(), value.getWindows(), diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java index d8202882b4b6..8cef3618023e 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/DoFnFunction.java @@ -50,7 +50,6 @@ import org.apache.beam.sdk.transforms.reflect.DoFnInvokers; import org.apache.beam.sdk.util.DoFnWithExecutionInformation; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; @@ -58,6 +57,7 @@ import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ElemToBytesFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ElemToBytesFunction.java index e2029910cb8d..0121eb2cfcc0 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ElemToBytesFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ElemToBytesFunction.java @@ -24,7 +24,8 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.checkerframework.checker.nullness.qual.Nullable; /** Map to tuple function. */ @@ -33,7 +34,7 @@ }) public class ElemToBytesFunction implements MapFunc> { - private transient WindowedValue.WindowedValueCoder wvCoder; + private transient WindowedValues.WindowedValueCoder wvCoder; private static final Logger LOG = Logger.getLogger(ElemToBytesFunction.class.getName()); private transient boolean isInitialized = false; @@ -44,7 +45,7 @@ public ElemToBytesFunction() { this.isInitialized = false; } - public ElemToBytesFunction(WindowedValue.WindowedValueCoder wvCoder) { + public ElemToBytesFunction(WindowedValues.WindowedValueCoder wvCoder) { this.wvCoder = wvCoder; wvCoderBytes = SerializableUtils.serializeToByteArray(wvCoder); } @@ -73,7 +74,7 @@ private void initTransient() { return; } wvCoder = - (WindowedValue.WindowedValueCoder) + (WindowedValues.WindowedValueCoder) SerializableUtils.deserializeFromByteArray(wvCoderBytes, "Coder"); this.isInitialized = true; } diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java index be3d25a8740b..3880b8e907cd 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/GroupByWindowFunction.java @@ -42,7 +42,6 @@ import org.apache.beam.sdk.options.PortablePipelineOptions; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.util.construction.SdkComponents; @@ -50,6 +49,8 @@ import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.joda.time.Instant; @@ -182,7 +183,7 @@ public void outputWindowedValue( Instant timestamp, Collection windows, PaneInfo pane) { - outputs.add(WindowedValue.of(output, timestamp, windows, pane)); + outputs.add(WindowedValues.of(output, timestamp, windows, pane)); } @Override diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ImpulseSource.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ImpulseSource.java index 6fbceaa148e4..3d53fe9166c1 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ImpulseSource.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/ImpulseSource.java @@ -18,12 +18,13 @@ package org.apache.beam.runners.twister2.translators.functions; import edu.iu.dsc.tws.api.tset.fn.SourceFunc; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** A {@link SourceFunc} which executes the impulse transform contract. */ public class ImpulseSource implements SourceFunc> { private static final WindowedValue IMPULSE_VALUE = - WindowedValue.valueInGlobalWindow(new byte[0]); + WindowedValues.valueInGlobalWindow(new byte[0]); private boolean impulseEmitted = false; diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java index d96fa224ddac..da1080b697ac 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/MapToTupleFunction.java @@ -26,8 +26,9 @@ import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** Map to tuple function. */ @SuppressWarnings({ @@ -37,7 +38,7 @@ public class MapToTupleFunction implements MapFunc, WindowedValue>> { private transient Coder keyCoder; - private transient WindowedValue.WindowedValueCoder wvCoder; + private transient WindowedValues.WindowedValueCoder wvCoder; private static final Logger LOG = Logger.getLogger(MapToTupleFunction.class.getName()); private transient boolean isInitialized = false; @@ -49,7 +50,7 @@ public MapToTupleFunction() { this.isInitialized = false; } - public MapToTupleFunction(Coder inputKeyCoder, WindowedValue.WindowedValueCoder wvCoder) { + public MapToTupleFunction(Coder inputKeyCoder, WindowedValues.WindowedValueCoder wvCoder) { this.keyCoder = inputKeyCoder; this.wvCoder = wvCoder; keyCoderBytes = SerializableUtils.serializeToByteArray(keyCoder); @@ -61,10 +62,10 @@ public Tuple map(WindowedValue> input) { Tuple element = null; WindowedValue>> temp = - WindowedValue.of( + WindowedValues.of( KV.of( input.getValue().getKey(), - WindowedValue.of( + WindowedValues.of( input.getValue().getValue(), input.getTimestamp(), input.getWindows(), @@ -98,7 +99,7 @@ private void initTransient() { } keyCoder = (Coder) SerializableUtils.deserializeFromByteArray(keyCoderBytes, "Coder"); wvCoder = - (WindowedValue.WindowedValueCoder) + (WindowedValues.WindowedValueCoder) SerializableUtils.deserializeFromByteArray(wvCoderBytes, "Coder"); this.isInitialized = true; } diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/OutputTagFilter.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/OutputTagFilter.java index 7b553a24e36c..b8d00c3925ed 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/OutputTagFilter.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/translators/functions/OutputTagFilter.java @@ -22,7 +22,7 @@ import edu.iu.dsc.tws.api.tset.fn.RecordCollector; import java.util.Iterator; import org.apache.beam.sdk.transforms.join.RawUnionValue; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** Output tag filter. */ @SuppressWarnings({ diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/TranslationUtils.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/TranslationUtils.java index 26e4c6b0db44..faf1abc197de 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/TranslationUtils.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/TranslationUtils.java @@ -22,7 +22,8 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.coders.CoderException; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** doc. */ @SuppressWarnings({ @@ -59,7 +60,7 @@ public static T fromByteArray(byte[] serialized, Coder coder) { * @return Deserialized object. */ public static WindowedValue fromByteArray( - byte[] serialized, WindowedValue.WindowedValueCoder coder) { + byte[] serialized, WindowedValues.WindowedValueCoder coder) { try { return CoderUtils.decodeFromByteArray(coder, serialized); } catch (CoderException e) { diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2AssignContext.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2AssignContext.java index 491917a8839c..1155dded04d2 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2AssignContext.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2AssignContext.java @@ -19,7 +19,7 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.joda.time.Instant; diff --git a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2SideInputReader.java b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2SideInputReader.java index a4735080f8f5..e2e2a281a9fc 100644 --- a/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2SideInputReader.java +++ b/runners/twister2/src/main/java/org/apache/beam/runners/twister2/utils/Twister2SideInputReader.java @@ -36,9 +36,9 @@ import org.apache.beam.sdk.transforms.Materializations.MultimapView; import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.BoundedWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java index dc81e23b10ee..2dce2bed7d38 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/transforms/Create.java @@ -58,7 +58,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; @@ -66,6 +65,8 @@ import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TimestampedValue.TimestampedValueCoder; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.collect.ImmutableList; @@ -872,7 +873,9 @@ public PCollection expand(PBegin input) { Pipeline.applyTransform( input, Create.of(windowedValues) - .withCoder(WindowedValue.getFullCoder(coder, windowCoder))); + .withCoder( + org.apache.beam.sdk.values.WindowedValues.getFullCoder( + coder, windowCoder))); PCollection output = intermediate.apply(ParDo.of(new ConvertWindowedValues<>())); output.setCoder(coder); diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java index f82a9759547a..84a90721a983 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/CoderTranslators.java @@ -34,8 +34,8 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.util.InstanceBuilder; import org.apache.beam.sdk.util.ShardedKey; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.InvalidProtocolBufferException; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -122,28 +122,29 @@ public List> getComponents(FullWindowedValueCoder from) { @Override public FullWindowedValueCoder fromComponents(List> components) { - return WindowedValue.getFullCoder( + return WindowedValues.getFullCoder( components.get(0), (Coder) components.get(1)); } }; } - static CoderTranslator> paramWindowedValue() { - return new CoderTranslator>() { + static CoderTranslator> paramWindowedValue() { + return new CoderTranslator>() { @Override - public List> getComponents(WindowedValue.ParamWindowedValueCoder from) { + public List> getComponents( + WindowedValues.ParamWindowedValueCoder from) { return ImmutableList.of(from.getValueCoder(), from.getWindowCoder()); } @Override - public byte[] getPayload(WindowedValue.ParamWindowedValueCoder from) { - return WindowedValue.ParamWindowedValueCoder.getPayload(from); + public byte[] getPayload(WindowedValues.ParamWindowedValueCoder from) { + return WindowedValues.ParamWindowedValueCoder.getPayload(from); } @Override - public WindowedValue.ParamWindowedValueCoder fromComponents( + public WindowedValues.ParamWindowedValueCoder fromComponents( List> components, byte[] payload, CoderTranslation.TranslationContext context) { - return WindowedValue.ParamWindowedValueCoder.fromComponents(components, payload); + return WindowedValues.ParamWindowedValueCoder.fromComponents(components, payload); } }; } diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoderRegistrar.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoderRegistrar.java index 7fde7bd17484..5b0d5aedd619 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoderRegistrar.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/ModelCoderRegistrar.java @@ -37,8 +37,7 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; import org.apache.beam.sdk.util.ShardedKey; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.BiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableBiMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; @@ -66,9 +65,9 @@ public class ModelCoderRegistrar implements CoderTranslatorRegistrar { .put(Timer.Coder.class, ModelCoders.TIMER_CODER_URN) .put(LengthPrefixCoder.class, ModelCoders.LENGTH_PREFIX_CODER_URN) .put(GlobalWindow.Coder.class, ModelCoders.GLOBAL_WINDOW_CODER_URN) - .put(FullWindowedValueCoder.class, ModelCoders.WINDOWED_VALUE_CODER_URN) + .put(WindowedValues.FullWindowedValueCoder.class, ModelCoders.WINDOWED_VALUE_CODER_URN) .put( - WindowedValue.ParamWindowedValueCoder.class, + WindowedValues.ParamWindowedValueCoder.class, ModelCoders.PARAM_WINDOWED_VALUE_CODER_URN) .put(DoubleCoder.class, ModelCoders.DOUBLE_CODER_URN) .put(RowCoder.class, ModelCoders.ROW_CODER_URN) @@ -90,9 +89,10 @@ public class ModelCoderRegistrar implements CoderTranslatorRegistrar { .put(IterableCoder.class, CoderTranslators.iterable()) .put(Timer.Coder.class, CoderTranslators.timer()) .put(LengthPrefixCoder.class, CoderTranslators.lengthPrefix()) - .put(FullWindowedValueCoder.class, CoderTranslators.fullWindowedValue()) .put( - WindowedValue.ParamWindowedValueCoder.class, + WindowedValues.FullWindowedValueCoder.class, CoderTranslators.fullWindowedValue()) + .put( + WindowedValues.ParamWindowedValueCoder.class, CoderTranslators.paramWindowedValue()) .put(DoubleCoder.class, CoderTranslators.atomic(DoubleCoder.class)) .put(RowCoder.class, CoderTranslators.row()) diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RunnerPCollectionView.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RunnerPCollectionView.java index 3af8836c4622..8d1d02f4b555 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RunnerPCollectionView.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/util/construction/RunnerPCollectionView.java @@ -23,12 +23,12 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.transforms.ViewFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.PValue; import org.apache.beam.sdk.values.PValueBase; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.sdk.values.WindowingStrategy; import org.checkerframework.checker.nullness.qual.Nullable; diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java new file mode 100644 index 000000000000..2a5236f0147f --- /dev/null +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValue.java @@ -0,0 +1,60 @@ +/* + * 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.values; + +import java.util.Collection; +import org.apache.beam.sdk.transforms.windowing.BoundedWindow; +import org.apache.beam.sdk.transforms.windowing.PaneInfo; +import org.joda.time.Instant; + +/** + * A value along with Beam's windowing information and all other metadata. + * + * @param the type of the primary data for the value. + */ +public interface WindowedValue { + /** The primary data for this value. */ + T getValue(); + + /** The timestamp of this value in event time. */ + Instant getTimestamp(); + + /** Returns the windows of this {@code WindowedValue}. */ + Collection getWindows(); + + /** @deprecated Do not use; only here for ease of migration/container update. */ + @Deprecated + PaneInfo getPane(); + + /** The {@link PaneInfo} associated with this WindowedValue. */ + default PaneInfo getPaneInfo() { + return getPane(); + } + + /** + * A representation of each of the actual values represented by this compressed {@link + * WindowedValue}, one per window. + */ + Iterable> explodeWindows(); + + /** + * A {@link WindowedValue} with identical metadata to the current one, but with the provided + * value. + */ + WindowedValue withValue(OtherT value); +} diff --git a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java similarity index 88% rename from sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java rename to sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java index d11166001f05..3c044990de37 100644 --- a/sdks/java/core/src/main/java/org/apache/beam/sdk/util/WindowedValue.java +++ b/sdks/java/core/src/main/java/org/apache/beam/sdk/values/WindowedValues.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.sdk.util; +package org.apache.beam.sdk.values; 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; @@ -52,12 +52,14 @@ import org.joda.time.Instant; /** - * An immutable triple of value, timestamp, and windows. + * Implementations of {@link WindowedValue} and static utility methods. * - * @param the type of the value + *

These are primarily intended for internal use by Beam SDK developers and runner developers. + * Backwards incompatible changes will likely occur. */ @Internal -public abstract class WindowedValue { +public class WindowedValues { + private WindowedValues() {} // non-instantiable utility class /** Returns a {@code WindowedValue} with the given value, timestamp, and windows. */ public static WindowedValue of( @@ -143,70 +145,47 @@ public static WindowedValue timestampedValueInGlobalWindow( * Returns a new {@code WindowedValue} that is a copy of this one, but with a different value, * which may have a new type {@code NewT}. */ - public abstract WindowedValue withValue(NewT value); - - /** Returns the value of this {@code WindowedValue}. */ - public abstract T getValue(); - - /** Returns the timestamp of this {@code WindowedValue}. */ - public abstract Instant getTimestamp(); - - /** Returns the windows of this {@code WindowedValue}. */ - public abstract Collection getWindows(); - - /** Returns the pane of this {@code WindowedValue} in its window. */ - public abstract PaneInfo getPane(); - - /** Returns {@code true} if this WindowedValue has exactly one window. */ - public boolean isSingleWindowedValue() { - return false; + public static WindowedValue withValue( + WindowedValue windowedValue, NewT newValue) { + return WindowedValues.of( + newValue, + windowedValue.getTimestamp(), + windowedValue.getWindows(), + windowedValue.getPaneInfo()); } - /** - * Returns a collection of {@link WindowedValue WindowedValues} identical to this one, except each - * is in exactly one of the windows that this {@link WindowedValue} is in. - */ - public Iterable> explodeWindows() { - if (isSingleWindowedValue()) { - return ImmutableList.of(this); + public static boolean equals( + @Nullable WindowedValue left, @Nullable WindowedValue right) { + if (left == null) { + return right == null; } - ImmutableList.Builder> windowedValues = ImmutableList.builder(); - for (BoundedWindow w : getWindows()) { - windowedValues.add(of(getValue(), getTimestamp(), w, getPane())); - } - return windowedValues.build(); - } - @Override - public boolean equals(@Nullable Object other) { - if (!(other instanceof WindowedValue)) { + if (right == null) { return false; - } else { - WindowedValue that = (WindowedValue) other; - - // Compare timestamps first as they are most likely to differ. - // Also compare timestamps according to millis-since-epoch because otherwise expensive - // comparisons are made on their Chronology objects. - return this.getTimestamp().isEqual(that.getTimestamp()) - && Objects.equals(this.getValue(), that.getValue()) - && Objects.equals(this.getWindows(), that.getWindows()) - && Objects.equals(this.getPane(), that.getPane()); } + + // Compare timestamps first as they are most likely to differ. + // Also compare timestamps according to millis-since-epoch because otherwise expensive + // comparisons are made on their Chronology objects. + return left.getTimestamp().isEqual(right.getTimestamp()) + && Objects.equals(left.getValue(), right.getValue()) + && Objects.equals(left.getWindows(), right.getWindows()) + && Objects.equals(left.getPaneInfo(), right.getPaneInfo()); } - @Override - public int hashCode() { + public static int hashCode(WindowedValue windowedValue) { // Hash only the millis of the timestamp to be consistent with equals - return Objects.hash(getValue(), getTimestamp().getMillis(), getWindows(), getPane()); + return Objects.hash( + windowedValue.getValue(), + windowedValue.getTimestamp().getMillis(), + windowedValue.getWindows(), + windowedValue.getPane()); } - @Override - public abstract String toString(); - private static final Collection GLOBAL_WINDOWS = Collections.singletonList(GlobalWindow.INSTANCE); - /** A {@link WindowedValue} which holds exactly single window per value. */ + /** A {@link WindowedValues} which holds exactly single window per value. */ public interface SingleWindowedValue { /** @return the single window associated with this value. */ @@ -214,10 +193,10 @@ public interface SingleWindowedValue { } /** - * An abstract superclass for implementations of {@link WindowedValue} that stores the value and + * An abstract superclass for implementations of {@link WindowedValues} that stores the value and * pane info. */ - private abstract static class SimpleWindowedValue extends WindowedValue { + private abstract static class SimpleWindowedValue implements WindowedValue { private final T value; private final PaneInfo pane; @@ -236,6 +215,19 @@ public PaneInfo getPane() { public T getValue() { return value; } + + @Override + public Iterable> explodeWindows() { + if (this.getWindows().size() == 1) { + return ImmutableList.of(this); + } + ImmutableList.Builder> windowedValues = ImmutableList.builder(); + for (BoundedWindow w : this.getWindows()) { + windowedValues.add( + WindowedValues.of(this.getValue(), this.getTimestamp(), w, this.getPaneInfo())); + } + return windowedValues.build(); + } } /** The abstract superclass of WindowedValue representations where timestamp == MIN. */ @@ -258,24 +250,19 @@ public ValueInGlobalWindow(T value, PaneInfo pane) { super(value, pane); } - @Override - public WindowedValue withValue(NewT newValue) { - return new ValueInGlobalWindow<>(newValue, getPane()); - } - @Override public Collection getWindows() { return GLOBAL_WINDOWS; } @Override - public boolean isSingleWindowedValue() { - return true; + public BoundedWindow getWindow() { + return GlobalWindow.INSTANCE; } @Override - public BoundedWindow getWindow() { - return GlobalWindow.INSTANCE; + public WindowedValue withValue(NewT newValue) { + return new ValueInGlobalWindow<>(newValue, getPane()); } @Override @@ -329,24 +316,19 @@ public TimestampedValueInGlobalWindow(T value, Instant timestamp, PaneInfo pane) super(value, timestamp, pane); } - @Override - public WindowedValue withValue(NewT newValue) { - return new TimestampedValueInGlobalWindow<>(newValue, getTimestamp(), getPane()); - } - @Override public Collection getWindows() { return GLOBAL_WINDOWS; } @Override - public boolean isSingleWindowedValue() { - return true; + public BoundedWindow getWindow() { + return GlobalWindow.INSTANCE; } @Override - public BoundedWindow getWindow() { - return GlobalWindow.INSTANCE; + public WindowedValue withValue(NewT newValue) { + return new TimestampedValueInGlobalWindow<>(newValue, getTimestamp(), getPane()); } @Override @@ -405,11 +387,6 @@ public Collection getWindows() { return Collections.singletonList(window); } - @Override - public boolean isSingleWindowedValue() { - return true; - } - @Override public BoundedWindow getWindow() { return window; @@ -459,13 +436,14 @@ public TimestampedValueInMultipleWindows( } @Override - public WindowedValue withValue(NewT newValue) { - return new TimestampedValueInMultipleWindows<>(newValue, getTimestamp(), windows, getPane()); + public Collection getWindows() { + return windows; } @Override - public Collection getWindows() { - return windows; + public WindowedValue withValue(NewT newValue) { + return new TimestampedValueInMultipleWindows<>( + newValue, getTimestamp(), getWindows(), getPane()); } @Override @@ -622,7 +600,7 @@ public WindowedValue decode(InputStream inStream, Context context) // Because there are some remaining (incorrect) uses of WindowedValue with no windows, // we call this deprecated no-validation path when decoding - return WindowedValue.createWithoutValidation(value, timestamp, windows, pane); + return WindowedValues.createWithoutValidation(value, timestamp, windows, pane); } @Override @@ -706,7 +684,7 @@ public WindowedValue decode(InputStream inStream) throws CoderException, IOEx public WindowedValue decode(InputStream inStream, Context context) throws CoderException, IOException { T value = valueCoder.decode(inStream, context); - return WindowedValue.valueInGlobalWindow(value); + return WindowedValues.valueInGlobalWindow(value); } @Override @@ -744,7 +722,7 @@ public static class ParamWindowedValueCoder extends FullWindowedValueCoder /** * Returns the {@link ParamWindowedValueCoder} for the given valueCoder and windowCoder using - * the supplied parameterized timestamp, windows and pane info for {@link WindowedValue}. + * the supplied parameterized timestamp, windows and pane info for {@link WindowedValues}. */ public static ParamWindowedValueCoder of( Coder valueCoder, @@ -787,7 +765,7 @@ public static ParamWindowedValueCoder of(Coder valueCoder) { Collection windows, PaneInfo pane) { super(valueCoder, windowCoder); - this.windowedValuePrototype = WindowedValue.of(EMPTY_BYTES, timestamp, windows, pane); + this.windowedValuePrototype = WindowedValues.of(EMPTY_BYTES, timestamp, windows, pane); } @Override @@ -816,7 +794,7 @@ public WindowedValue decode(InputStream inStream) throws CoderException, IOEx @Override public WindowedValue decode(InputStream inStream, Context context) throws CoderException, IOException { - return windowedValuePrototype.withValue(valueCoder.decode(inStream, context)); + return WindowedValues.withValue(windowedValuePrototype, valueCoder.decode(inStream, context)); } @Override @@ -849,9 +827,9 @@ public static byte[] getPayload(ParamWindowedValueCoder from) { // ParamWindowedValueCoder ByteArrayOutputStream baos = new ByteArrayOutputStream(); WindowedValue windowedValue = - WindowedValue.of(EMPTY_BYTES, from.getTimestamp(), from.getWindows(), from.getPane()); - WindowedValue.FullWindowedValueCoder windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), from.getWindowCoder()); + WindowedValues.of(EMPTY_BYTES, from.getTimestamp(), from.getWindows(), from.getPane()); + WindowedValues.FullWindowedValueCoder windowedValueCoder = + WindowedValues.FullWindowedValueCoder.of(ByteArrayCoder.of(), from.getWindowCoder()); try { windowedValueCoder.encode(windowedValue, baos); } catch (IOException e) { @@ -862,17 +840,17 @@ public static byte[] getPayload(ParamWindowedValueCoder from) { } /** Create a {@link Coder} from its component {@link Coder coders}. */ - public static WindowedValue.ParamWindowedValueCoder fromComponents( + public static WindowedValues.ParamWindowedValueCoder fromComponents( List> components, byte[] payload) { Coder windowCoder = (Coder) components.get(1); - WindowedValue.FullWindowedValueCoder windowedValueCoder = - WindowedValue.FullWindowedValueCoder.of(ByteArrayCoder.of(), windowCoder); + WindowedValues.FullWindowedValueCoder windowedValueCoder = + WindowedValues.FullWindowedValueCoder.of(ByteArrayCoder.of(), windowCoder); try { ByteArrayInputStream bais = new ByteArrayInputStream(payload); WindowedValue windowedValue = windowedValueCoder.decode(bais); - return WindowedValue.ParamWindowedValueCoder.of( + return WindowedValues.ParamWindowedValueCoder.of( components.get(0), windowCoder, windowedValue.getTimestamp(), diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java index 91556cf2ac19..f915f9285367 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/fn/data/BeamFnDataInboundObserverTest.java @@ -36,7 +36,8 @@ import org.apache.beam.sdk.fn.test.TestExecutors.TestExecutorService; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.junit.Rule; import org.junit.Test; import org.junit.runner.RunWith; @@ -46,7 +47,7 @@ @RunWith(JUnit4.class) public class BeamFnDataInboundObserverTest { private static final Coder> CODER = - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE); private static final String TRANSFORM_ID = "transformId"; private static final String TIMER_FAMILY_ID = "timerFamilyId"; @@ -94,13 +95,13 @@ public void testConsumptionOfValuesHappensOnAwaitCompletionCallersThread() throw assertThat( values, contains( - WindowedValue.valueInGlobalWindow("ABC"), - WindowedValue.valueInGlobalWindow("DEF"), - WindowedValue.valueInGlobalWindow("GHI"))); + WindowedValues.valueInGlobalWindow("ABC"), + WindowedValues.valueInGlobalWindow("DEF"), + WindowedValues.valueInGlobalWindow("GHI"))); assertThat( timers, contains( - WindowedValue.valueInGlobalWindow("UVW"), WindowedValue.valueInGlobalWindow("XYZ"))); + WindowedValues.valueInGlobalWindow("UVW"), WindowedValues.valueInGlobalWindow("XYZ"))); future.get(); } @@ -220,7 +221,7 @@ public void testBadProducerDataFailureVisibleToAwaitCompletionCallerAndProducer( private BeamFnApi.Elements dataWith(String... values) throws Exception { ByteStringOutputStream output = new ByteStringOutputStream(); for (String value : values) { - CODER.encode(WindowedValue.valueInGlobalWindow(value), output); + CODER.encode(WindowedValues.valueInGlobalWindow(value), output); } return BeamFnApi.Elements.newBuilder() .addData( @@ -239,7 +240,7 @@ private BeamFnApi.Elements lastData() throws Exception { private BeamFnApi.Elements timerWith(String... values) throws Exception { ByteStringOutputStream output = new ByteStringOutputStream(); for (String value : values) { - CODER.encode(WindowedValue.valueInGlobalWindow(value), output); + CODER.encode(WindowedValues.valueInGlobalWindow(value), output); } return BeamFnApi.Elements.newBuilder() .addTimers( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java index 85c8d0d04ede..e9cb2664341f 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/CreateTest.java @@ -62,13 +62,14 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptor; import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; import org.checkerframework.checker.nullness.qual.Nullable; @@ -347,9 +348,9 @@ public void testCreateTimestampedDefaultOutputCoderUsingTypeDescriptor() throws public void testCreateWindowedValues() { List> data = Arrays.asList( - WindowedValue.of("a", new Instant(1L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), - WindowedValue.of("b", new Instant(2L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of("a", new Instant(1L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + WindowedValues.of("b", new Instant(2L), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), + WindowedValues.of( "c", new Instant(3L), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING)); // The easiest way to directly check the created PCollection with PAssert and without relying on diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java index 397c71f6beb9..d21eb77819a2 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/RedistributeTest.java @@ -45,13 +45,13 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -180,22 +180,22 @@ public void testRedistributePreservesMetadata() { pipeline .apply( Create.windowedValues( - WindowedValue.of( + WindowedValues.of( "foo", BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "foo", new Instant(0), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.of( + WindowedValues.of( "bar", new Instant(33), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1)), - WindowedValue.of( + WindowedValues.of( "bar", GlobalWindow.INSTANCE.maxTimestamp(), GlobalWindow.INSTANCE, diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java index 4df02ea0cf59..099dcb78a6e0 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/transforms/ReshuffleTest.java @@ -48,12 +48,12 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.Sessions; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TypeDescriptors; import org.apache.beam.sdk.values.ValueInSingleWindow; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Lists; @@ -171,22 +171,22 @@ public void testReshufflePreservesMetadata() { pipeline .apply( Create.windowedValues( - WindowedValue.of( + WindowedValues.of( "foo", BoundedWindow.TIMESTAMP_MIN_VALUE, GlobalWindow.INSTANCE, PaneInfo.NO_FIRING), - WindowedValue.of( + WindowedValues.of( "foo", new Instant(0), GlobalWindow.INSTANCE, PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.of( + WindowedValues.of( "bar", new Instant(33), GlobalWindow.INSTANCE, PaneInfo.createPane(false, false, PaneInfo.Timing.LATE, 1, 1)), - WindowedValue.of( + WindowedValues.of( "bar", GlobalWindow.INSTANCE.maxTimestamp(), GlobalWindow.INSTANCE, diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java index 0a18f076762a..db1579333e57 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/WindowedValueTest.java @@ -20,6 +20,8 @@ import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.hamcrest.Matchers.equalTo; +import static org.hamcrest.Matchers.instanceOf; +import static org.hamcrest.Matchers.not; import java.util.Arrays; import org.apache.beam.sdk.coders.Coder; @@ -31,6 +33,8 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.PaneInfo.Timing; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.joda.time.Duration; @@ -52,7 +56,7 @@ public class WindowedValueTest { public void testWindowedValueCoder() throws CoderException { Instant timestamp = new Instant(1234); WindowedValue value = - WindowedValue.of( + WindowedValues.of( "abc", new Instant(1234), Arrays.asList( @@ -62,7 +66,7 @@ public void testWindowedValueCoder() throws CoderException { PaneInfo.NO_FIRING); Coder> windowedValueCoder = - WindowedValue.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); + WindowedValues.getFullCoder(StringUtf8Coder.of(), IntervalWindow.getCoder()); byte[] encodedValue = CoderUtils.encodeToByteArray(windowedValueCoder, value); WindowedValue decodedValue = @@ -76,24 +80,25 @@ public void testWindowedValueCoder() throws CoderException { @Test public void testFullWindowedValueCoderIsSerializableWithWellKnownCoderType() { CoderProperties.coderSerializable( - WindowedValue.getFullCoder(GlobalWindow.Coder.INSTANCE, GlobalWindow.Coder.INSTANCE)); + WindowedValues.getFullCoder(GlobalWindow.Coder.INSTANCE, GlobalWindow.Coder.INSTANCE)); } @Test public void testParamWindowedValueCoderIsSerializableWithWellKnownCoderType() { CoderProperties.coderSerializable( - WindowedValue.getParamWindowedValueCoder(GlobalWindow.Coder.INSTANCE)); + WindowedValues.getParamWindowedValueCoder(GlobalWindow.Coder.INSTANCE)); } @Test public void testValueOnlyWindowedValueCoderIsSerializableWithWellKnownCoderType() { - CoderProperties.coderSerializable(WindowedValue.getValueOnlyCoder(GlobalWindow.Coder.INSTANCE)); + CoderProperties.coderSerializable( + WindowedValues.getValueOnlyCoder(GlobalWindow.Coder.INSTANCE)); } @Test public void testExplodeWindowsInNoWindowsCrash() { thrown.expect(IllegalArgumentException.class); - WindowedValue.of("foo", Instant.now(), ImmutableList.of(), PaneInfo.NO_FIRING); + WindowedValues.of("foo", Instant.now(), ImmutableList.of(), PaneInfo.NO_FIRING); } @Test @@ -102,7 +107,7 @@ public void testExplodeWindowsInOneWindowEquals() { BoundedWindow window = new IntervalWindow(now.minus(Duration.millis(1000L)), now.plus(Duration.millis(1000L))); WindowedValue value = - WindowedValue.of("foo", now, window, PaneInfo.ON_TIME_AND_ONLY_FIRING); + WindowedValues.of("foo", now, window, PaneInfo.ON_TIME_AND_ONLY_FIRING); assertThat(Iterables.getOnlyElement(value.explodeWindows()), equalTo(value)); } @@ -119,7 +124,7 @@ public void testExplodeWindowsManyWindowsMultipleWindowedValues() { BoundedWindow futureFutureWindow = new IntervalWindow(now, now.plus(Duration.millis(2000L))); PaneInfo pane = PaneInfo.createPane(false, false, Timing.ON_TIME, 3L, 0L); WindowedValue value = - WindowedValue.of( + WindowedValues.of( "foo", now, ImmutableList.of(pastWindow, centerWindow, futureWindow, futureFutureWindow), @@ -128,29 +133,29 @@ public void testExplodeWindowsManyWindowsMultipleWindowedValues() { assertThat( value.explodeWindows(), containsInAnyOrder( - WindowedValue.of("foo", now, futureFutureWindow, pane), - WindowedValue.of("foo", now, futureWindow, pane), - WindowedValue.of("foo", now, centerWindow, pane), - WindowedValue.of("foo", now, pastWindow, pane))); + WindowedValues.of("foo", now, futureFutureWindow, pane), + WindowedValues.of("foo", now, futureWindow, pane), + WindowedValues.of("foo", now, centerWindow, pane), + WindowedValues.of("foo", now, pastWindow, pane))); - assertThat(value.isSingleWindowedValue(), equalTo(false)); + assertThat(value, not(instanceOf(WindowedValues.SingleWindowedValue.class))); } @Test public void testSingleWindowedValueInGlobalWindow() { WindowedValue value = - WindowedValue.of(1, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); - assertThat(value.isSingleWindowedValue(), equalTo(true)); + WindowedValues.of(1, Instant.now(), GlobalWindow.INSTANCE, PaneInfo.NO_FIRING); + assertThat(value, instanceOf(WindowedValues.SingleWindowedValue.class)); assertThat( - ((WindowedValue.SingleWindowedValue) value).getWindow(), equalTo(GlobalWindow.INSTANCE)); + ((WindowedValues.SingleWindowedValue) value).getWindow(), equalTo(GlobalWindow.INSTANCE)); } @Test public void testSingleWindowedValueInFixedWindow() { Instant now = Instant.now(); BoundedWindow w = new IntervalWindow(now, now.plus(Duration.millis(1))); - WindowedValue value = WindowedValue.of(1, now, w, PaneInfo.NO_FIRING); - assertThat(value.isSingleWindowedValue(), equalTo(true)); - assertThat(((WindowedValue.SingleWindowedValue) value).getWindow(), equalTo(w)); + WindowedValue value = WindowedValues.of(1, now, w, PaneInfo.NO_FIRING); + assertThat(value, instanceOf(WindowedValues.SingleWindowedValue.class)); + assertThat(((WindowedValues.SingleWindowedValue) value).getWindow(), equalTo(w)); } } diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CoderTranslationTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CoderTranslationTest.java index 7dbe544d5948..b8f92ff0053e 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CoderTranslationTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/CoderTranslationTest.java @@ -52,9 +52,9 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; import org.apache.beam.sdk.util.ShardedKey; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; 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.ImmutableSet; import org.hamcrest.Matchers; @@ -85,7 +85,7 @@ public class CoderTranslationTest { .add( FullWindowedValueCoder.of( IterableCoder.of(VarLongCoder.of()), IntervalWindowCoder.of())) - .add(WindowedValue.ParamWindowedValueCoder.of(IterableCoder.of(VarLongCoder.of()))) + .add(WindowedValues.ParamWindowedValueCoder.of(IterableCoder.of(VarLongCoder.of()))) .add(DoubleCoder.of()) .add( RowCoder.of( diff --git a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ModelCodersTest.java b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ModelCodersTest.java index f0a277760ac4..b21368f9fc30 100644 --- a/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ModelCodersTest.java +++ b/sdks/java/core/src/test/java/org/apache/beam/sdk/util/construction/ModelCodersTest.java @@ -32,10 +32,10 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.coders.VarLongCoder; import org.apache.beam.sdk.transforms.windowing.IntervalWindow.IntervalWindowCoder; -import org.apache.beam.sdk.util.WindowedValue.FullWindowedValueCoder; import org.apache.beam.sdk.util.construction.ModelCoders.KvCoderComponents; import org.apache.beam.sdk.util.construction.ModelCoders.WindowedValueCoderComponents; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValues.FullWindowedValueCoder; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; diff --git a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/WindowingTest.java b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/WindowingTest.java index 80eb878de9c8..1b4179ff9e9c 100644 --- a/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/WindowingTest.java +++ b/sdks/java/extensions/euphoria/src/test/java/org/apache/beam/sdk/extensions/euphoria/core/testkit/WindowingTest.java @@ -16,6 +16,7 @@ * limitations under the License. */ package org.apache.beam.sdk.extensions.euphoria.core.testkit; // + // import static org.junit.Assert.assertEquals; // // import java.time.Instant; diff --git a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/PrecombineGroupingTableBenchmark.java b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/PrecombineGroupingTableBenchmark.java index ded8358a10d4..54f773d402cc 100644 --- a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/PrecombineGroupingTableBenchmark.java +++ b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/PrecombineGroupingTableBenchmark.java @@ -30,8 +30,9 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Sum; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.openjdk.jmh.annotations.Benchmark; import org.openjdk.jmh.annotations.Level; import org.openjdk.jmh.annotations.Param; @@ -74,13 +75,13 @@ private static List>> generateTestData(String case "uniform": for (int i = 0; i < TOTAL_VALUES; ++i) { int key = random.nextInt(KEY_SPACE); - elements.add(WindowedValue.valueInGlobalWindow(KV.of(Integer.toString(key), key))); + elements.add(WindowedValues.valueInGlobalWindow(KV.of(Integer.toString(key), key))); } break; case "normal": for (int i = 0; i < TOTAL_VALUES; ++i) { int key = (int) (random.nextGaussian() * KEY_SPACE); - elements.add(WindowedValue.valueInGlobalWindow(KV.of(Integer.toString(key), key))); + elements.add(WindowedValues.valueInGlobalWindow(KV.of(Integer.toString(key), key))); } break; case "hotKey": @@ -91,12 +92,12 @@ private static List>> generateTestData(String } else { key = random.nextInt(KEY_SPACE); } - elements.add(WindowedValue.valueInGlobalWindow(KV.of(Integer.toString(key), key))); + elements.add(WindowedValues.valueInGlobalWindow(KV.of(Integer.toString(key), key))); } break; case "uniqueKeys": for (int i = 0; i < TOTAL_VALUES; ++i) { - elements.add(WindowedValue.valueInGlobalWindow(KV.of(Integer.toString(i), i))); + elements.add(WindowedValues.valueInGlobalWindow(KV.of(Integer.toString(i), i))); } Collections.shuffle(elements, random); break; diff --git a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java index 42c288673734..3b8fbeaf3dd0 100644 --- a/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java +++ b/sdks/java/harness/jmh/src/main/java/org/apache/beam/fn/harness/jmh/ProcessBundleBenchmark.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness.jmh; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkState; import static org.junit.Assert.assertEquals; @@ -83,13 +83,13 @@ import org.apache.beam.sdk.transforms.Impulse; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.transforms.WithKeys; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.util.construction.graph.ExecutableStage; import org.apache.beam.sdk.util.construction.graph.FusedPipeline; import org.apache.beam.sdk.util.construction.graph.GreedyPipelineFuser; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Strings; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java index 8e4f6a2b2036..957fa057054f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/AssignWindowsRunner.java @@ -30,9 +30,10 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.transforms.windowing.WindowFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; @@ -109,6 +110,6 @@ public BoundedWindow window() { } }; Collection windows = windowFn.assignWindows(ctxt); - return WindowedValue.of(input.getValue(), input.getTimestamp(), windows, input.getPane()); + return WindowedValues.of(input.getValue(), input.getTimestamp(), windows, input.getPane()); } } diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java index 834d78e28a57..8e1231347a1e 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataReadRunner.java @@ -48,9 +48,9 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.fn.data.RemoteGrpcPortRead; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.grpc.v1p69p0.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.collect.ImmutableMap; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java index 7995961243ea..78d13d14bf68 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/BeamFnDataWriteRunner.java @@ -31,9 +31,9 @@ import org.apache.beam.model.pipeline.v1.RunnerApi; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.fn.data.RemoteGrpcPortWrite; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java index 9ac7f6de5476..718fe7ee8b67 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/CombineRunners.java @@ -35,11 +35,11 @@ import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.windowing.GlobalWindows; import org.apache.beam.sdk.util.SerializableUtils; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; 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.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FlattenRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FlattenRunner.java index c0986010c635..d79a79ada540 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FlattenRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FlattenRunner.java @@ -23,8 +23,8 @@ import java.io.IOException; import java.util.Map; import org.apache.beam.sdk.fn.data.FnDataReceiver; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; /** Executes flatten PTransforms. */ diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java index e264fa14788a..460c4cc6cab7 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/FnApiDoFnRunner.java @@ -99,8 +99,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; -import org.apache.beam.sdk.util.WindowedValue.WindowedValueCoder; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; @@ -109,6 +107,9 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; +import org.apache.beam.sdk.values.WindowedValues.WindowedValueCoder; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.util.Durations; @@ -376,7 +377,7 @@ public final void addRunnerForPTransform(Context context) throws IOException { components.getPcollectionsMap().get(pTransform.getInputsOrThrow(mainInputTag)); Coder maybeWindowedValueInputCoder = rehydratedComponents.getCoder(mainInput.getCoderId()); // TODO: Stop passing windowed value coders within PCollections. - if (maybeWindowedValueInputCoder instanceof WindowedValue.WindowedValueCoder) { + if (maybeWindowedValueInputCoder instanceof WindowedValues.WindowedValueCoder) { inputCoder = ((WindowedValueCoder) maybeWindowedValueInputCoder).getValueCoder(); } else { inputCoder = maybeWindowedValueInputCoder; @@ -1011,28 +1012,28 @@ public Object restriction() { return WindowedSplitResult.forRoots( splitResult.getPrimaryInFullyProcessedWindowsRoot() == null ? null - : WindowedValue.of( + : WindowedValues.of( KV.of(splitResult.getPrimaryInFullyProcessedWindowsRoot().getValue(), fullSize), splitResult.getPrimaryInFullyProcessedWindowsRoot().getTimestamp(), splitResult.getPrimaryInFullyProcessedWindowsRoot().getWindows(), splitResult.getPrimaryInFullyProcessedWindowsRoot().getPane()), splitResult.getPrimarySplitRoot() == null ? null - : WindowedValue.of( + : WindowedValues.of( KV.of(splitResult.getPrimarySplitRoot().getValue(), primarySize), splitResult.getPrimarySplitRoot().getTimestamp(), splitResult.getPrimarySplitRoot().getWindows(), splitResult.getPrimarySplitRoot().getPane()), splitResult.getResidualSplitRoot() == null ? null - : WindowedValue.of( + : WindowedValues.of( KV.of(splitResult.getResidualSplitRoot().getValue(), residualSize), splitResult.getResidualSplitRoot().getTimestamp(), splitResult.getResidualSplitRoot().getWindows(), splitResult.getResidualSplitRoot().getPane()), splitResult.getResidualInUnprocessedWindowsRoot() == null ? null - : WindowedValue.of( + : WindowedValues.of( KV.of(splitResult.getResidualInUnprocessedWindowsRoot().getValue(), fullSize), splitResult.getResidualInUnprocessedWindowsRoot().getTimestamp(), splitResult.getResidualInUnprocessedWindowsRoot().getWindows(), @@ -1081,7 +1082,7 @@ private HandlesSplits.SplitResult trySplitForWindowObservingTruncateRestriction( } // Note that the assumption here is the fullInputCoder of the Truncate transform should be the // the same as the SDF/Process transform. - Coder fullInputCoder = WindowedValue.getFullCoder(inputCoder, windowCoder); + Coder fullInputCoder = WindowedValues.getFullCoder(inputCoder, windowCoder); return constructSplitResult( windowedSplitResult, downstreamSplitResult, @@ -1113,7 +1114,7 @@ private static WindowedSplitResult computeWindowSplit WindowedSplitResult.forRoots( primaryFullyProcessedWindows.isEmpty() ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( currentElement.getValue(), KV.of(currentRestriction, currentWatermarkEstimatorState)), @@ -1122,7 +1123,7 @@ private static WindowedSplitResult computeWindowSplit currentElement.getPane()), splitResult == null ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( currentElement.getValue(), KV.of(splitResult.getPrimary(), currentWatermarkEstimatorState)), @@ -1131,7 +1132,7 @@ private static WindowedSplitResult computeWindowSplit currentElement.getPane()), splitResult == null ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( currentElement.getValue(), KV.of(splitResult.getResidual(), watermarkAndState.getValue())), @@ -1140,7 +1141,7 @@ private static WindowedSplitResult computeWindowSplit currentElement.getPane()), residualUnprocessedWindows.isEmpty() ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( currentElement.getValue(), KV.of(currentRestriction, currentWatermarkEstimatorState)), @@ -1439,7 +1440,7 @@ private HandlesSplits.SplitResult trySplitForElementAndRestriction( PTransformTranslation.SPLITTABLE_PROCESS_SIZED_ELEMENTS_AND_RESTRICTIONS_URN + "/GetSize"); } - Coder fullInputCoder = WindowedValue.getFullCoder(inputCoder, windowCoder); + Coder fullInputCoder = WindowedValues.getFullCoder(inputCoder, windowCoder); return constructSplitResult( windowedSplitResult, null, @@ -1913,7 +1914,7 @@ public PipelineOptions getPipelineOptions() { @Override public void output(OutputT output, Instant timestamp, BoundedWindow window) { outputTo( - mainOutputConsumer, WindowedValue.of(output, timestamp, window, PaneInfo.NO_FIRING)); + mainOutputConsumer, WindowedValues.of(output, timestamp, window, PaneInfo.NO_FIRING)); } @Override @@ -1923,7 +1924,7 @@ public void output(TupleTag tag, T output, Instant timestamp, BoundedWind if (consumer == null) { throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } - outputTo(consumer, WindowedValue.of(output, timestamp, window, PaneInfo.NO_FIRING)); + outputTo(consumer, WindowedValues.of(output, timestamp, window, PaneInfo.NO_FIRING)); } } @@ -1978,7 +1979,7 @@ public void output(OutputT output) { // Don't need to check timestamp since we can always output using the input timestamp. outputTo( mainOutputConsumer, - WindowedValue.of( + WindowedValues.of( output, currentElement.getTimestamp(), currentWindow, currentElement.getPane())); } @@ -1992,7 +1993,7 @@ public void output(TupleTag tag, T output) { // Don't need to check timestamp since we can always output using the input timestamp. outputTo( consumer, - WindowedValue.of( + WindowedValues.of( output, currentElement.getTimestamp(), currentWindow, currentElement.getPane())); } @@ -2002,7 +2003,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { // runners can provide proper timestamps. outputTo( mainOutputConsumer, - WindowedValue.of(output, timestamp, currentWindow, currentElement.getPane())); + WindowedValues.of(output, timestamp, currentWindow, currentElement.getPane())); } @Override @@ -2013,7 +2014,7 @@ public void outputWindowedValue( PaneInfo paneInfo) { // TODO(https://github.com/apache/beam/issues/29637): Check that timestamp is valid once all // runners can provide proper timestamps. - outputTo(mainOutputConsumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(mainOutputConsumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -2026,7 +2027,7 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } outputTo( - consumer, WindowedValue.of(output, timestamp, currentWindow, currentElement.getPane())); + consumer, WindowedValues.of(output, timestamp, currentWindow, currentElement.getPane())); } @Override @@ -2043,7 +2044,7 @@ public void outputWindowedValue( if (consumer == null) { throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } - outputTo(consumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(consumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -2135,7 +2136,7 @@ public Instant timestamp(DoFn doFn) { outputTo( mainOutputConsumer, (WindowedValue) - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), KV.of(output, currentWatermarkEstimatorState)), @@ -2180,7 +2181,7 @@ public Instant timestamp(DoFn doFn) { outputTo( mainOutputConsumer, (WindowedValue) - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), KV.of(output, currentWatermarkEstimatorState)), @@ -2220,7 +2221,7 @@ public Instant timestamp(DoFn doFn) { outputTo( mainOutputConsumer, (WindowedValue) - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), KV.of(output, currentWatermarkEstimatorState)), @@ -2348,7 +2349,7 @@ public Instant timestamp(DoFn doFn) { outputTo( mainOutputConsumer, (WindowedValue) - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), KV.of(output, currentWatermarkEstimatorState)), @@ -2388,7 +2389,7 @@ public Instant timestamp(DoFn doFn) { outputTo( mainOutputConsumer, (WindowedValue) - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), KV.of(output, currentWatermarkEstimatorState)), @@ -2450,7 +2451,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { checkTimestamp(timestamp); outputTo( mainOutputConsumer, - WindowedValue.of( + WindowedValues.of( output, timestamp, currentElement.getWindows(), currentElement.getPane())); } @@ -2461,7 +2462,7 @@ public void outputWindowedValue( Collection windows, PaneInfo paneInfo) { checkTimestamp(timestamp); - outputTo(mainOutputConsumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(mainOutputConsumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -2474,7 +2475,7 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp } outputTo( consumer, - WindowedValue.of( + WindowedValues.of( output, timestamp, currentElement.getWindows(), currentElement.getPane())); } @@ -2491,7 +2492,7 @@ public void outputWindowedValue( if (consumer == null) { throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } - outputTo(consumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(consumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } } @@ -2824,7 +2825,7 @@ public BoundedWindow window() { public void output(OutputT output) { outputTo( mainOutputConsumer, - WindowedValue.of( + WindowedValues.of( output, currentTimer.getHoldTimestamp(), currentWindow, currentTimer.getPane())); } @@ -2833,7 +2834,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { checkOnWindowExpirationTimestamp(timestamp); outputTo( mainOutputConsumer, - WindowedValue.of(output, timestamp, currentWindow, currentTimer.getPane())); + WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPane())); } @Override @@ -2843,7 +2844,7 @@ public void outputWindowedValue( Collection windows, PaneInfo paneInfo) { checkOnWindowExpirationTimestamp(timestamp); - outputTo(mainOutputConsumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(mainOutputConsumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -2855,7 +2856,7 @@ public void output(TupleTag tag, T output) { } outputTo( consumer, - WindowedValue.of( + WindowedValues.of( output, currentTimer.getHoldTimestamp(), currentWindow, currentTimer.getPane())); } @@ -2868,7 +2869,7 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } outputTo( - consumer, WindowedValue.of(output, timestamp, currentWindow, currentTimer.getPane())); + consumer, WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPane())); } @Override @@ -2881,7 +2882,7 @@ public void outputWindowedValue( checkOnWindowExpirationTimestamp(timestamp); FnDataReceiver> consumer = (FnDataReceiver) localNameToConsumer.get(tag.getId()); - outputTo(consumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(consumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } @SuppressWarnings( @@ -3120,7 +3121,7 @@ public void output(OutputT output) { checkTimerTimestamp(currentTimer.getHoldTimestamp()); outputTo( mainOutputConsumer, - WindowedValue.of( + WindowedValues.of( output, currentTimer.getHoldTimestamp(), currentWindow, currentTimer.getPane())); } @@ -3129,7 +3130,7 @@ public void outputWithTimestamp(OutputT output, Instant timestamp) { checkTimerTimestamp(timestamp); outputTo( mainOutputConsumer, - WindowedValue.of(output, timestamp, currentWindow, currentTimer.getPane())); + WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPane())); } @Override @@ -3139,7 +3140,7 @@ public void outputWindowedValue( Collection windows, PaneInfo paneInfo) { checkTimerTimestamp(timestamp); - outputTo(mainOutputConsumer, WindowedValue.of(output, timestamp, windows, paneInfo)); + outputTo(mainOutputConsumer, WindowedValues.of(output, timestamp, windows, paneInfo)); } @Override @@ -3152,7 +3153,7 @@ public void output(TupleTag tag, T output) { } outputTo( consumer, - WindowedValue.of( + WindowedValues.of( output, currentTimer.getHoldTimestamp(), currentWindow, currentTimer.getPane())); } @@ -3165,7 +3166,7 @@ public void outputWithTimestamp(TupleTag tag, T output, Instant timestamp throw new IllegalArgumentException(String.format("Unknown output tag %s", tag)); } outputTo( - consumer, WindowedValue.of(output, timestamp, currentWindow, currentTimer.getPane())); + consumer, WindowedValues.of(output, timestamp, currentWindow, currentTimer.getPane())); } @Override diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/MapFnRunners.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/MapFnRunners.java index ddd6faa46d16..dbc9fc05020a 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/MapFnRunners.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/MapFnRunners.java @@ -23,7 +23,7 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PTransform; import org.apache.beam.sdk.fn.data.FnDataReceiver; import org.apache.beam.sdk.function.ThrowingFunction; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; /** diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java index 7b53f6511045..8e7c670d5c26 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PTransformRunnerFactory.java @@ -36,8 +36,8 @@ import org.apache.beam.sdk.function.ThrowingRunnable; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.transforms.DoFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.values.WindowedValue; /** A factory able to instantiate an appropriate handler for a given PTransform. */ public interface PTransformRunnerFactory { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java index 06cc17062bdb..d6827e821782 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/PrecombineGroupingTable.java @@ -37,8 +37,9 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.Weighted; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.annotations.VisibleForTesting; import org.joda.time.Instant; @@ -468,8 +469,8 @@ private void output( entry.compact(); receiver.accept( isGloballyWindowed - ? WindowedValue.valueInGlobalWindow(KV.of(entry.getKey(), entry.getAccumulator())) - : WindowedValue.of( + ? WindowedValues.valueInGlobalWindow(KV.of(entry.getKey(), entry.getAccumulator())) + : WindowedValues.of( KV.of(entry.getKey(), entry.getAccumulator()), entry.getOutputTimestamp(), entry.getGroupingKey().getWindows(), diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java index f19e98ee9e73..72565f608ce5 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunner.java @@ -39,12 +39,13 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.checkerframework.checker.nullness.qual.Nullable; @@ -203,7 +204,7 @@ private void processElementForWindowObservingPairWithRestriction(WindowedValue> getMultiplexingConsumer(String pCollecti } // TODO: Stop passing windowed value coders within PCollections. - if (maybeWindowedValueInputCoder instanceof WindowedValue.WindowedValueCoder) { + if (maybeWindowedValueInputCoder instanceof WindowedValues.WindowedValueCoder) { coder = ((WindowedValueCoder) maybeWindowedValueInputCoder).getValueCoder(); } else { coder = maybeWindowedValueInputCoder; diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/ElementSample.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/ElementSample.java index 4ef1674c9ec6..180e9634500f 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/ElementSample.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/ElementSample.java @@ -18,7 +18,7 @@ package org.apache.beam.fn.harness.debug; import javax.annotation.Nullable; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; /** * A record class that wraps an element sample with additional metadata. This ensures the ability to diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java index f7fabae0cc21..f526f1d01110 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/debug/OutputSampler.java @@ -29,7 +29,8 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; /** * This class holds samples for a single PCollection until queried by the parent DataSampler. This @@ -79,7 +80,7 @@ public OutputSampler( // element itself is sampled. Or, it's non a WindowedValueCoder and the value inside the // windowed value must be sampled. This is because WindowedValue is the element type used in // all receivers, which doesn't necessarily match the PBD encoding. - if (coder instanceof WindowedValue.WindowedValueCoder) { + if (coder instanceof WindowedValues.WindowedValueCoder) { this.valueCoder = null; this.windowedValueCoder = (Coder>) coder; } else { diff --git a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java index 2b9d8b79f113..164589bc40aa 100644 --- a/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java +++ b/sdks/java/harness/src/main/java/org/apache/beam/fn/harness/state/FnApiStateAccessor.java @@ -65,13 +65,13 @@ import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.CombineFnUtil; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.BeamUrns; import org.apache.beam.sdk.util.construction.PCollectionViewTranslation; import org.apache.beam.sdk.util.construction.RehydratedComponents; import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TimestampedValue; import org.apache.beam.sdk.values.TupleTag; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.sdk.values.WindowingStrategy; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -189,9 +189,9 @@ public static Factory factoryForPTransformContext( .get(context.getPTransform().getInputsOrThrow(mainInputTag)); Coder maybeWindowedValueInputCoder = rehydratedComponents.getCoder(mainInput.getCoderId()); Coder inputCoder; - if (maybeWindowedValueInputCoder instanceof WindowedValue.WindowedValueCoder) { + if (maybeWindowedValueInputCoder instanceof WindowedValues.WindowedValueCoder) { inputCoder = - ((WindowedValue.WindowedValueCoder) maybeWindowedValueInputCoder).getValueCoder(); + ((WindowedValues.WindowedValueCoder) maybeWindowedValueInputCoder).getValueCoder(); } else { inputCoder = maybeWindowedValueInputCoder; } diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java index 838715d1beda..9fbf21683b63 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/AssignWindowsRunnerTest.java @@ -44,12 +44,13 @@ import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.WindowFn; import org.apache.beam.sdk.transforms.windowing.WindowMappingFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.Environments; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.util.construction.WindowingStrategyTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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.ImmutableSet; import org.joda.time.Duration; @@ -74,22 +75,22 @@ public void singleInputSingleOutputSucceeds() throws Exception { AssignWindowsRunner runner = AssignWindowsRunner.create(windowFn); assertThat( - runner.assignWindows(WindowedValue.valueInGlobalWindow(1)), + runner.assignWindows(WindowedValues.valueInGlobalWindow(1)), equalTo( - WindowedValue.of( + WindowedValues.of( 1, BoundedWindow.TIMESTAMP_MIN_VALUE, windowFn.assignWindow(BoundedWindow.TIMESTAMP_MIN_VALUE), PaneInfo.NO_FIRING))); assertThat( runner.assignWindows( - WindowedValue.of( + WindowedValues.of( 2, new Instant(-10L), new IntervalWindow(new Instant(-120000L), Duration.standardMinutes(3L)), PaneInfo.ON_TIME_AND_ONLY_FIRING)), equalTo( - WindowedValue.of( + WindowedValues.of( 2, new Instant(-10L), windowFn.assignWindow(new Instant(-10L)), @@ -112,17 +113,17 @@ public void singleInputMultipleOutputSucceeds() throws Exception { IntervalWindow thirdWindow = new IntervalWindow(new Instant(0), Duration.standardMinutes(4L)); WindowedValue firstValue = - WindowedValue.timestampedValueInGlobalWindow(-3, new Instant(-12)); + WindowedValues.timestampedValueInGlobalWindow(-3, new Instant(-12)); assertThat( runner.assignWindows(firstValue), equalTo( - WindowedValue.of( + WindowedValues.of( -3, new Instant(-12), ImmutableSet.of(firstWindow, secondWindow), firstValue.getPane()))); WindowedValue secondValue = - WindowedValue.of( + WindowedValues.of( 3, new Instant(12), new IntervalWindow(new Instant(-12), Duration.standardMinutes(24)), @@ -131,7 +132,7 @@ public void singleInputMultipleOutputSucceeds() throws Exception { assertThat( runner.assignWindows(secondValue), equalTo( - WindowedValue.of( + WindowedValues.of( 3, new Instant(12), ImmutableSet.of(secondWindow, thirdWindow), @@ -205,7 +206,7 @@ public Coder windowCoder() { .addRunnerForPTransform(context); WindowedValue value = - WindowedValue.of( + WindowedValues.of( 2, new Instant(-10L), ImmutableList.of( @@ -216,14 +217,14 @@ public Coder windowCoder() { assertThat( outputs, containsInAnyOrder( - WindowedValue.of( + WindowedValues.of( 2, new Instant(-10L), ImmutableSet.of( GlobalWindow.INSTANCE, new IntervalWindow(new Instant(-500), Duration.standardMinutes(3))), PaneInfo.ON_TIME_AND_ONLY_FIRING), - WindowedValue.of( + WindowedValues.of( 2, new Instant(-10L), ImmutableSet.of( @@ -265,7 +266,7 @@ public Coder windowCoder() { thrown.expect(IllegalArgumentException.class); runner.assignWindows( - WindowedValue.of( + WindowedValues.of( 2, new Instant(-10L), ImmutableList.of( @@ -300,13 +301,13 @@ public void factoryCreatesFromJavaWindowFn() throws Exception { assertThat( fn.apply( - WindowedValue.of( + WindowedValues.of( 22L, new Instant(5), new IntervalWindow(new Instant(0L), new Instant(20027L)), PaneInfo.ON_TIME_AND_ONLY_FIRING)), equalTo( - WindowedValue.of( + WindowedValues.of( 22L, new Instant(5), new TestWindowFn().assignWindow(new Instant(5)), @@ -338,7 +339,7 @@ public void factoryCreatesFromKnownWindowFn() throws Exception { (ThrowingFunction) factory.forPTransform("transform", windowPTransform); WindowedValue output = fn.apply( - WindowedValue.of( + WindowedValues.of( 22L, new Instant(5), new IntervalWindow(new Instant(0L), new Instant(20027L)), @@ -347,7 +348,7 @@ public void factoryCreatesFromKnownWindowFn() throws Exception { assertThat( output, equalTo( - WindowedValue.of( + WindowedValues.of( 22L, new Instant(5), new IntervalWindow(new Instant(5L), Duration.standardMinutes(12L)), diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java index 6ec40c10bfb2..40ebebacd285 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataReadRunnerTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -65,8 +65,9 @@ import org.apache.beam.sdk.fn.test.TestExecutors; import org.apache.beam.sdk.fn.test.TestExecutors.TestExecutorService; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.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.ImmutableMap; @@ -88,7 +89,7 @@ public class BeamFnDataReadRunnerTest { private static final Coder ELEMENT_CODER = StringUtf8Coder.of(); private static final String ELEMENT_CODER_SPEC_ID = "string-coder-id"; private static final Coder> CODER = - WindowedValue.getFullCoder(ELEMENT_CODER, GlobalWindow.Coder.INSTANCE); + WindowedValues.getFullCoder(ELEMENT_CODER, GlobalWindow.Coder.INSTANCE); private static final String CODER_SPEC_ID = "windowed-string-coder-id"; private static final RunnerApi.Coder CODER_SPEC; private static final RunnerApi.Components COMPONENTS; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java index f9b03ba4b0a9..70a894e7b375 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/BeamFnDataWriteRunnerTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -51,8 +51,9 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.stub.StreamObserver; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.hamcrest.collection.IsMapContaining; @@ -71,7 +72,7 @@ public class BeamFnDataWriteRunnerTest { private static final Coder ELEM_CODER = StringUtf8Coder.of(); private static final String WIRE_CODER_ID = "windowed-string-coder-id"; private static final Coder> WIRE_CODER = - WindowedValue.getFullCoder(ELEM_CODER, GlobalWindow.Coder.INSTANCE); + WindowedValues.getFullCoder(ELEM_CODER, GlobalWindow.Coder.INSTANCE); private static final RunnerApi.Coder WIRE_CODER_SPEC; private static final RunnerApi.Components COMPONENTS; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java index 6c2b7cb20ace..fc3cad970f73 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/CombineRunnersTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -38,12 +38,12 @@ import org.apache.beam.sdk.transforms.Combine; import org.apache.beam.sdk.transforms.Combine.CombineFn; import org.apache.beam.sdk.transforms.Create; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.ModelCoders; import org.apache.beam.sdk.util.construction.PipelineTranslation; import org.apache.beam.sdk.util.construction.SdkComponents; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Iterables; import org.junit.Before; import org.junit.Test; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java index 98bd85b4005f..0a839588e537 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FlattenRunnerTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -32,9 +32,10 @@ import org.apache.beam.model.pipeline.v1.RunnerApi.PCollection; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.fn.data.FnDataReceiver; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.PTransformTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -175,8 +176,8 @@ public void testFlattenWithDuplicateInputCollectionProducesMultipleOutputs() thr FnDataReceiver> input = context.getPCollectionConsumer("inputATarget"); - input.accept(WindowedValue.valueInGlobalWindow("A1")); - input.accept(WindowedValue.valueInGlobalWindow("A2")); + input.accept(WindowedValues.valueInGlobalWindow("A1")); + input.accept(WindowedValues.valueInGlobalWindow("A2")); assertThat( mainOutputValues, diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java index 6ca085495a3d..b9c879ad827a 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/FnApiDoFnRunnerTest.java @@ -19,8 +19,8 @@ import static java.util.Arrays.asList; import static org.apache.beam.sdk.options.ExperimentalOptions.addExperiment; -import static org.apache.beam.sdk.util.WindowedValue.timestampedValueInGlobalWindow; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.timestampedValueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions.checkArgument; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.allOf; @@ -116,7 +116,6 @@ import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.UserCodeException; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; import org.apache.beam.sdk.util.construction.CoderTranslation.TranslationContext; import org.apache.beam.sdk.util.construction.PTransformTranslation; @@ -132,6 +131,8 @@ import org.apache.beam.sdk.values.PCollectionView; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.util.Durations; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; @@ -1166,7 +1167,7 @@ private org.apache.beam.sdk.util.construction.Timer dynamicTimerInGlobalW private WindowedValue valueInWindows( T value, BoundedWindow window, BoundedWindow... windows) { - return WindowedValue.of( + return WindowedValues.of( value, window.maxTimestamp(), ImmutableList.builder().add(window).add(windows).build(), @@ -1413,7 +1414,7 @@ public void testProcessElementForSizedElementAndRestriction() throws Exception { RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(pProto.getComponents()); Coder inputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( CoderTranslation.fromProto( pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), rehydratedComponents, @@ -1707,7 +1708,7 @@ public void testProcessElementForSizedElementAndRestrictionSplitBeforeTryClaim() RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(pProto.getComponents()); Coder inputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( CoderTranslation.fromProto( pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), rehydratedComponents, @@ -1921,7 +1922,7 @@ public void testProcessElementForSizedElementAndRestrictionNoTryClaim() throws E RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(pProto.getComponents()); Coder inputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( CoderTranslation.fromProto( pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), rehydratedComponents, @@ -2120,7 +2121,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(pProto.getComponents()); Coder inputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( CoderTranslation.fromProto( pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), rehydratedComponents, @@ -2256,7 +2257,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc assertEquals( decode(inputCoder, primaryRoot.getElement()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( "5", @@ -2269,7 +2270,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc firstValue.getPane())); assertEquals( decode(inputCoder, residualRoot.getApplication().getElement()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( "5", @@ -2282,7 +2283,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc firstValue.getPane())); assertEquals( decode(inputCoder, residualRootForUnprocessedWindows.getApplication().getElement()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( "5", @@ -2315,37 +2316,37 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( "5:5", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(5)), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( "5:6", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(6)), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( "5:7", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(7)), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( "2:0", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(0)), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( "2:1", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( "2:0", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(0)), window2, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( "2:1", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)), window2, @@ -2407,22 +2408,22 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( "7:0", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(0)), window1, splitValue.getPane()), - WindowedValue.of( + WindowedValues.of( "7:1", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(1)), window1, splitValue.getPane()), - WindowedValue.of( + WindowedValues.of( "7:2", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(2)), window1, splitValue.getPane()), - WindowedValue.of( + WindowedValues.of( "7:3", GlobalWindow.TIMESTAMP_MIN_VALUE.plus(Duration.millis(3)), window1, @@ -2490,7 +2491,7 @@ public void testProcessElementForWindowedSizedElementAndRestriction() throws Exc assertEquals( expectedOutputWatermarkMap, residualRoot.getApplication().getOutputWatermarksMap()); assertEquals( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( "7", @@ -2625,7 +2626,7 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser RehydratedComponents rehydratedComponents = RehydratedComponents.forComponents(pProto.getComponents()); Coder inputCoder = - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( CoderTranslation.fromProto( pProto.getComponents().getCodersOrThrow(inputPCollection.getCoderId()), rehydratedComponents, @@ -2710,14 +2711,14 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("7", KV.of(new OffsetRange(0, 3), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), splitValue.getTimestamp(), window1, splitValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("7", KV.of(new OffsetRange(0, 3), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), @@ -2730,7 +2731,7 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser Iterables.getOnlyElement(expectedElementSplit.getPrimaryRoots()); ByteStringOutputStream primaryBytes = new ByteStringOutputStream(); inputCoder.encode( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("7", KV.of(new OffsetRange(0, 6), GlobalWindow.TIMESTAMP_MIN_VALUE)), 6.0), splitValue.getTimestamp(), @@ -2747,7 +2748,7 @@ public void testProcessElementForTruncateAndSizeRestrictionForwardSplitWhenObser Iterables.getOnlyElement(expectedElementSplit.getResidualRoots()); ByteStringOutputStream residualBytes = new ByteStringOutputStream(); inputCoder.encode( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("7", KV.of(new OffsetRange(0, 6), GlobalWindow.TIMESTAMP_MIN_VALUE)), 6.0), splitValue.getTimestamp(), @@ -3020,28 +3021,28 @@ public void testProcessElementForWindowedTruncateAndSizeRestriction() throws Exc assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), window2, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), firstValue.getTimestamp(), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), @@ -3143,14 +3144,14 @@ public void testProcessElementForWindowedTruncateAndSizeRestriction() throws Exc assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), @@ -3333,14 +3334,14 @@ public static class SplitTest { private KV createSplitInWindow( OffsetRange primaryRestriction, OffsetRange residualRestriction, BoundedWindow window) { return KV.of( - WindowedValue.of( + WindowedValues.of( KV.of( currentElement.getValue(), KV.of(primaryRestriction, currentWatermarkEstimatorState)), currentElement.getTimestamp(), window, currentElement.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( currentElement.getValue(), KV.of(residualRestriction, watermarkAndState.getValue())), @@ -3354,7 +3355,7 @@ private KV createSplitAcrossWindows( return KV.of( primaryWindows.isEmpty() ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( currentElement.getValue(), KV.of(currentRestriction, currentWatermarkEstimatorState)), @@ -3363,7 +3364,7 @@ private KV createSplitAcrossWindows( currentElement.getPane()), residualWindows.isEmpty() ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( currentElement.getValue(), KV.of(currentRestriction, currentWatermarkEstimatorState)), @@ -3375,7 +3376,7 @@ private KV createSplitAcrossWindows( private KV createSplitWithSizeInWindow( OffsetRange primaryRestriction, OffsetRange residualRestriction, BoundedWindow window) { return KV.of( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), @@ -3384,7 +3385,7 @@ private KV createSplitWithSizeInWindow( currentElement.getTimestamp(), window, currentElement.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of( currentElement.getValue(), @@ -3400,7 +3401,7 @@ private KV createSplitWithSizeAcrossWindows( return KV.of( primaryWindows.isEmpty() ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( KV.of( currentElement.getValue(), @@ -3411,7 +3412,7 @@ private KV createSplitWithSizeAcrossWindows( currentElement.getPane()), residualWindows.isEmpty() ? null - : WindowedValue.of( + : WindowedValues.of( KV.of( KV.of( currentElement.getValue(), @@ -3428,7 +3429,7 @@ public void setUp() { window2 = new IntervalWindow(Instant.ofEpochMilli(10), Instant.ofEpochMilli(20)); window3 = new IntervalWindow(Instant.ofEpochMilli(20), Instant.ofEpochMilli(30)); currentElement = - WindowedValue.of( + WindowedValues.of( "a", Instant.ofEpochMilli(57), ImmutableList.of(window1, window2, window3), @@ -4188,13 +4189,13 @@ public void testConstructSplitResultWithInvalidElementSplits() throws Exception FnApiDoFnRunner.constructSplitResult( WindowedSplitResult.forRoots( null, - WindowedValue.valueInGlobalWindow("elementPrimary"), - WindowedValue.valueInGlobalWindow("elementResidual"), + WindowedValues.valueInGlobalWindow("elementPrimary"), + WindowedValues.valueInGlobalWindow("elementResidual"), null), HandlesSplits.SplitResult.of( ImmutableList.of(BundleApplication.getDefaultInstance()), ImmutableList.of(DelayedBundleApplication.getDefaultInstance())), - WindowedValue.getFullCoder(VoidCoder.of(), GlobalWindow.Coder.INSTANCE), + WindowedValues.getFullCoder(VoidCoder.of(), GlobalWindow.Coder.INSTANCE), Instant.now(), null, "ptransformId", @@ -4209,7 +4210,7 @@ private Coder getFullInputCoder( KvCoder.of( KvCoder.of(elementCoder, KvCoder.of(restrictionCoder, watermarkStateCoder)), DoubleCoder.of()); - return WindowedValue.getFullCoder(inputCoder, windowCoder); + return WindowedValues.getFullCoder(inputCoder, windowCoder); } private HandlesSplits.SplitResult getProcessElementSplit(String transformId, String inputId) { diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java index ea388b386db3..e881c35ce710 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/MapFnRunnersTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -36,8 +36,9 @@ import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.CoderTranslation; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.joda.time.Duration; import org.joda.time.Instant; @@ -153,7 +154,7 @@ public void testFullWindowedValueMappingWithCompressedWindow() throws Exception context .getPCollectionConsumer("inputPC") .accept( - WindowedValue.of( + WindowedValues.of( "abc", new Instant(12), ImmutableSet.of(firstWindow, GlobalWindow.INSTANCE, secondWindow), @@ -162,9 +163,9 @@ public void testFullWindowedValueMappingWithCompressedWindow() throws Exception assertThat( outputConsumer, containsInAnyOrder( - WindowedValue.timestampedValueInGlobalWindow("ABC", new Instant(12)), - WindowedValue.of("ABC", new Instant(12), secondWindow, PaneInfo.NO_FIRING), - WindowedValue.of("ABC", new Instant(12), firstWindow, PaneInfo.NO_FIRING))); + WindowedValues.timestampedValueInGlobalWindow("ABC", new Instant(12)), + WindowedValues.of("ABC", new Instant(12), secondWindow, PaneInfo.NO_FIRING), + WindowedValues.of("ABC", new Instant(12), firstWindow, PaneInfo.NO_FIRING))); } public ThrowingFunction, WindowedValue> diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java index d05e7c72eaf7..7b4387738a4c 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PTransformRunnerFactoryTestContext.java @@ -52,8 +52,8 @@ import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.DoFn.BundleFinalizer; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.Timer; +import org.apache.beam.sdk.values.WindowedValue; import org.joda.time.Instant; /** diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java index 64f160d183d2..384f46bb4527 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/PrecombineGroupingTableTest.java @@ -17,8 +17,8 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.timestampedValueInGlobalWindow; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.timestampedValueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.anyOf; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -48,8 +48,8 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Combine.CombineFn; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ArrayListMultimap; import org.hamcrest.Description; import org.hamcrest.TypeSafeDiagnosingMatcher; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java index 7c52da950882..370fd6e6de2c 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittablePairWithRestrictionDoFnRunnerTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -58,7 +58,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.util.ByteStringOutputStream; import org.apache.beam.sdk.util.CoderUtils; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.PipelineTranslation; @@ -68,6 +67,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.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.ImmutableMap; @@ -127,7 +128,7 @@ private org.apache.beam.sdk.util.construction.Timer dynamicTimerInGlobalW private WindowedValue valueInWindows( T value, BoundedWindow window, BoundedWindow... windows) { - return WindowedValue.of( + return WindowedValues.of( value, window.maxTimestamp(), ImmutableList.builder().add(window).add(windows).build(), @@ -353,7 +354,7 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( "5", KV.of( @@ -362,7 +363,7 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception firstValue.getTimestamp(), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( "5", KV.of( @@ -371,7 +372,7 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception firstValue.getTimestamp(), window2, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( "2", KV.of( @@ -380,7 +381,7 @@ public void testProcessElementForWindowedPairWithRestriction() throws Exception secondValue.getTimestamp(), window1, secondValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( "2", KV.of( @@ -468,7 +469,7 @@ public void testProcessElementForWindowedPairWithRestrictionWithNonWindowObservi assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( "5", KV.of( @@ -477,7 +478,7 @@ public void testProcessElementForWindowedPairWithRestrictionWithNonWindowObservi firstValue.getTimestamp(), ImmutableList.of(window1, window2), firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( "2", KV.of( diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java index 95b9129eb763..a19f65bd5716 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/SplittableSplitAndSizeRestrictionsDoFnRunnerTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.containsInAnyOrder; @@ -55,7 +55,6 @@ import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.transforms.windowing.SlidingWindows; import org.apache.beam.sdk.transforms.windowing.Window; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.construction.PTransformTranslation; import org.apache.beam.sdk.util.construction.ParDoTranslation; import org.apache.beam.sdk.util.construction.PipelineTranslation; @@ -65,6 +64,8 @@ import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionView; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.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.ImmutableMap; @@ -89,7 +90,7 @@ public class SplittableSplitAndSizeRestrictionsDoFnRunnerTest implements Seriali private WindowedValue valueInWindows( T value, BoundedWindow window, BoundedWindow... windows) { - return WindowedValue.of( + return WindowedValues.of( value, window.maxTimestamp(), ImmutableList.builder().add(window).add(windows).build(), @@ -313,7 +314,7 @@ public void testProcessElementForWindowedSplitAndSizeRestriction() throws Except window1, window2); WindowedValue secondValue = - WindowedValue.of( + WindowedValues.of( KV.of("2", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), firstValue.getTimestamp().plus(Duration.standardSeconds(1)), ImmutableList.of(window1, window2), @@ -327,56 +328,56 @@ public void testProcessElementForWindowedSplitAndSizeRestriction() throws Except assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(2, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), firstValue.getTimestamp(), window1, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), window2, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(2, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), firstValue.getTimestamp(), window2, firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), secondValue.getTimestamp(), window1, secondValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(1, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), secondValue.getTimestamp(), window1, secondValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), secondValue.getTimestamp(), window2, secondValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(1, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), @@ -470,28 +471,28 @@ public void testProcessElementForWindowedSplitAndSizeRestriction() throws Except assertThat( mainOutputValues, contains( - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(0, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 2.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("5", KV.of(new OffsetRange(2, 5), GlobalWindow.TIMESTAMP_MIN_VALUE)), 3.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(0, 1), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), firstValue.getTimestamp(), ImmutableList.of(window1, window2), firstValue.getPane()), - WindowedValue.of( + WindowedValues.of( KV.of( KV.of("2", KV.of(new OffsetRange(1, 2), GlobalWindow.TIMESTAMP_MIN_VALUE)), 1.0), diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java index 9776828b4ae2..15f83f2582c7 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/BeamFnDataGrpcClientTest.java @@ -18,7 +18,7 @@ package org.apache.beam.fn.harness.data; import static org.apache.beam.sdk.util.CoderUtils.encodeToByteArray; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.contains; import static org.hamcrest.Matchers.empty; @@ -51,7 +51,8 @@ import org.apache.beam.sdk.fn.test.TestStreams; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Server; @@ -68,7 +69,7 @@ public class BeamFnDataGrpcClientTest { private static final Coder> CODER = LengthPrefixCoder.of( - WindowedValue.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE)); + WindowedValues.getFullCoder(StringUtf8Coder.of(), GlobalWindow.Coder.INSTANCE)); private static final String INSTRUCTION_ID_A = "12L"; private static final String INSTRUCTION_ID_B = "56L"; private static final String TRANSFORM_ID_A = "34L"; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java index 4f0e1092d90a..f4207d472f84 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/data/PCollectionConsumerRegistryTest.java @@ -17,7 +17,7 @@ */ package org.apache.beam.fn.harness.data; -import static org.apache.beam.sdk.util.WindowedValue.valueInGlobalWindow; +import static org.apache.beam.sdk.values.WindowedValues.valueInGlobalWindow; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.containsInAnyOrder; import static org.junit.Assert.assertEquals; @@ -71,10 +71,10 @@ import org.apache.beam.sdk.metrics.MetricsEnvironment; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.util.ByteStringOutputStream; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterable; import org.apache.beam.sdk.util.common.ElementByteSizeObservableIterator; import org.apache.beam.sdk.util.construction.SdkComponents; +import org.apache.beam.sdk.values.WindowedValue; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.ManagedChannel; import org.apache.beam.vendor.grpc.v1p69p0.io.grpc.Server; diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java index fc1d089b11ce..5f9c1ecc5a96 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/DataSamplerTest.java @@ -40,7 +40,8 @@ import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.options.ExperimentalOptions; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; @@ -71,7 +72,7 @@ byte[] encodeByteArray(byte[] b) throws IOException { } WindowedValue globalWindowedValue(T el) { - return WindowedValue.valueInGlobalWindow(el); + return WindowedValues.valueInGlobalWindow(el); } BeamFnApi.InstructionResponse getAllSamples(DataSampler dataSampler) { diff --git a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java index 01ddd22b5603..11e44493f377 100644 --- a/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java +++ b/sdks/java/harness/src/test/java/org/apache/beam/fn/harness/debug/OutputSamplerTest.java @@ -31,7 +31,8 @@ import org.apache.beam.model.fnexecution.v1.BeamFnApi; import org.apache.beam.sdk.coders.VarIntCoder; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.grpc.v1p69p0.com.google.protobuf.ByteString; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableList; import org.junit.Test; @@ -50,11 +51,11 @@ public BeamFnApi.SampledElement encodeInt(Integer i) throws IOException { } public BeamFnApi.SampledElement encodeGlobalWindowedInt(Integer i) throws IOException { - WindowedValue.WindowedValueCoder coder = - WindowedValue.FullWindowedValueCoder.of(VarIntCoder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.WindowedValueCoder coder = + WindowedValues.FullWindowedValueCoder.of(VarIntCoder.of(), GlobalWindow.Coder.INSTANCE); ByteArrayOutputStream stream = new ByteArrayOutputStream(); - coder.encode(WindowedValue.valueInGlobalWindow(i), stream); + coder.encode(WindowedValues.valueInGlobalWindow(i), stream); return BeamFnApi.SampledElement.newBuilder() .setElement(ByteString.copyFrom(stream.toByteArray())) .build(); @@ -94,7 +95,7 @@ public void testSamplesFirstN() throws IOException { // Purposely go over maxSamples and sampleEveryN. This helps to increase confidence. for (int i = 0; i < 15; ++i) { - outputSampler.sample(WindowedValue.valueInGlobalWindow(i)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(i)); } // The expected list is only 0..9 inclusive. @@ -109,11 +110,11 @@ public void testSamplesFirstN() throws IOException { @Test public void testWindowedValueSample() throws IOException { - WindowedValue.WindowedValueCoder coder = - WindowedValue.FullWindowedValueCoder.of(VarIntCoder.of(), GlobalWindow.Coder.INSTANCE); + WindowedValues.WindowedValueCoder coder = + WindowedValues.FullWindowedValueCoder.of(VarIntCoder.of(), GlobalWindow.Coder.INSTANCE); OutputSampler outputSampler = new OutputSampler<>(coder, 10, 10, false); - outputSampler.sample(WindowedValue.valueInGlobalWindow(0)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(0)); // The expected list is only 0..9 inclusive. List expected = ImmutableList.of(encodeGlobalWindowedInt(0)); @@ -126,7 +127,7 @@ public void testNonWindowedValueSample() throws IOException { VarIntCoder coder = VarIntCoder.of(); OutputSampler outputSampler = new OutputSampler<>(coder, 10, 10, false); - outputSampler.sample(WindowedValue.valueInGlobalWindow(0)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(0)); // The expected list is only 0..9 inclusive. List expected = ImmutableList.of(encodeInt(0)); @@ -145,7 +146,7 @@ public void testActsLikeCircularBuffer() throws IOException { OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20, false); for (int i = 0; i < 100; ++i) { - outputSampler.sample(WindowedValue.valueInGlobalWindow(i)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(i)); } // The first 10 are always sampled, but with maxSamples = 5, the first ten are downsampled to @@ -173,7 +174,7 @@ public void testCanSampleExceptions() throws IOException { VarIntCoder coder = VarIntCoder.of(); OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20, false); - WindowedValue windowedValue = WindowedValue.valueInGlobalWindow(1); + WindowedValue windowedValue = WindowedValues.valueInGlobalWindow(1); ElementSample elementSample = outputSampler.sample(windowedValue); Exception exception = new RuntimeException("Test exception"); @@ -200,9 +201,9 @@ public void testNoDuplicateExceptions() throws IOException { OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20, false); ElementSample elementSampleA = - outputSampler.sample(WindowedValue.valueInGlobalWindow(1)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(1)); ElementSample elementSampleB = - outputSampler.sample(WindowedValue.valueInGlobalWindow(2)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(2)); Exception exception = new RuntimeException("Test exception"); String ptransformIdA = "ptransformA"; @@ -229,7 +230,7 @@ public void testExceptionOnlySampledIfNonNullProcessBundle() throws IOException VarIntCoder coder = VarIntCoder.of(); OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20, false); - WindowedValue windowedValue = WindowedValue.valueInGlobalWindow(1); + WindowedValue windowedValue = WindowedValues.valueInGlobalWindow(1); ElementSample elementSample = outputSampler.sample(windowedValue); Exception exception = new RuntimeException("Test exception"); @@ -253,11 +254,11 @@ public void testExceptionSamplesAreNotRemoved() throws IOException { VarIntCoder coder = VarIntCoder.of(); OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20, false); - WindowedValue windowedValue = WindowedValue.valueInGlobalWindow(0); + WindowedValue windowedValue = WindowedValues.valueInGlobalWindow(0); ElementSample elementSample = outputSampler.sample(windowedValue); for (int i = 1; i < 100; ++i) { - outputSampler.sample(WindowedValue.valueInGlobalWindow(i)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(i)); } Exception exception = new RuntimeException("Test exception"); @@ -290,8 +291,8 @@ public void testOnlySampleExceptions() throws IOException { VarIntCoder coder = VarIntCoder.of(); OutputSampler outputSampler = new OutputSampler<>(coder, 5, 20, true); - WindowedValue windowedValue = WindowedValue.valueInGlobalWindow(1); - outputSampler.sample(WindowedValue.valueInGlobalWindow(2)); + WindowedValue windowedValue = WindowedValues.valueInGlobalWindow(1); + outputSampler.sample(WindowedValues.valueInGlobalWindow(2)); ElementSample elementSample = outputSampler.sample(windowedValue); Exception exception = new RuntimeException("Test exception"); @@ -333,7 +334,7 @@ public void testConcurrentSamples() throws IOException, InterruptedException { for (int i = 0; i < 1000000; i++) { ElementSample sample = - outputSampler.sample(WindowedValue.valueInGlobalWindow(i)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(i)); outputSampler.exception(sample, new RuntimeException(""), "ptransformId", "pbId"); } @@ -352,7 +353,7 @@ public void testConcurrentSamples() throws IOException, InterruptedException { for (int i = -1000000; i < 0; i++) { ElementSample sample = - outputSampler.sample(WindowedValue.valueInGlobalWindow(i)); + outputSampler.sample(WindowedValues.valueInGlobalWindow(i)); outputSampler.exception(sample, new RuntimeException(""), "ptransformId", "pbId"); } diff --git a/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouter.java b/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouter.java index 7cff6cb3e791..6b4485db2167 100644 --- a/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouter.java +++ b/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouter.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.components.deadletterqueue; + import java.util.HashMap; import java.util.Map; import org.apache.beam.sdk.transforms.PTransform; diff --git a/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/sinks/ThrowingSink.java b/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/sinks/ThrowingSink.java index 3e5994e41e3b..9869cd09da3b 100644 --- a/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/sinks/ThrowingSink.java +++ b/sdks/java/io/components/src/main/java/org/apache/beam/sdk/io/components/deadletterqueue/sinks/ThrowingSink.java @@ -17,13 +17,13 @@ */ package org.apache.beam.sdk.io.components.deadletterqueue.sinks; + import org.apache.beam.repackaged.core.org.apache.commons.lang3.ObjectUtils.Null; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.ParDo; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; -import org.apache.beam.sdk.values.Row; import org.checkerframework.checker.nullness.qual.NonNull; public class ThrowingSink extends PTransform<@NonNull PCollection, @NonNull PDone> { diff --git a/sdks/java/io/components/src/test/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouterTest.java b/sdks/java/io/components/src/test/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouterTest.java index 14e076df1dcc..8d2e5d296f4f 100644 --- a/sdks/java/io/components/src/test/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouterTest.java +++ b/sdks/java/io/components/src/test/java/org/apache/beam/sdk/io/components/deadletterqueue/DLQRouterTest.java @@ -17,6 +17,7 @@ */ package org.apache.beam.sdk.io.components.deadletterqueue; + import org.apache.beam.sdk.io.components.deadletterqueue.sinks.ThrowingSink; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; diff --git a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpersTest.java b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpersTest.java index 472870805be5..0d36d7bb46d0 100644 --- a/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpersTest.java +++ b/sdks/java/io/google-cloud-platform/src/test/java/org/apache/beam/sdk/io/gcp/bigquery/BigQueryHelpersTest.java @@ -43,7 +43,7 @@ import org.apache.beam.sdk.transforms.windowing.IntervalWindow; import org.apache.beam.sdk.util.CoderUtils; import org.apache.beam.sdk.util.FluentBackoff; -import org.apache.beam.sdk.util.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableSet; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.Sets; import org.joda.time.Duration; @@ -175,7 +175,7 @@ public void testTableRowInfoCoderSerializable() { @Test public void testComplexCoderSerializable() { CoderProperties.coderSerializable( - WindowedValue.getFullCoder( + WindowedValues.getFullCoder( KvCoder.of( ShardedKeyCoder.of(StringUtf8Coder.of()), TableRowInfoCoder.of(TableRowJsonCoder.of())), diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java index bbde6c9e864e..142170acc67b 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/RecordWriterManager.java @@ -33,8 +33,8 @@ import java.util.concurrent.TimeUnit; import org.apache.beam.sdk.schemas.Schema; import org.apache.beam.sdk.util.Preconditions; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.WindowedValue; 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.Splitter; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.cache.Cache; diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java index 6a61aafbe8b9..42de901587b6 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteGroupedRowsToFiles.java @@ -24,10 +24,11 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.ShardedKey; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.base.Preconditions; import org.apache.iceberg.catalog.Catalog; import org.checkerframework.checker.nullness.qual.MonotonicNonNull; @@ -98,7 +99,7 @@ public void processElement( String tableIdentifier = element.getKey().getKey(); IcebergDestination destination = dynamicDestinations.instantiateDestination(tableIdentifier); WindowedValue windowedDestination = - WindowedValue.of(destination, window.maxTimestamp(), window, pane); + WindowedValues.of(destination, window.maxTimestamp(), window, pane); RecordWriterManager writer; try (RecordWriterManager openWriter = new RecordWriterManager(getCatalog(), filePrefix, maxFileSize, Integer.MAX_VALUE)) { diff --git a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java index 80bf962283f0..e76d18ae7106 100644 --- a/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java +++ b/sdks/java/io/iceberg/src/main/java/org/apache/beam/sdk/io/iceberg/WriteUngroupedRowsToFiles.java @@ -31,7 +31,6 @@ import org.apache.beam.sdk.transforms.windowing.BoundedWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; import org.apache.beam.sdk.util.ShardedKey; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.KV; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PCollectionTuple; @@ -41,6 +40,8 @@ import org.apache.beam.sdk.values.Row; import org.apache.beam.sdk.values.TupleTag; import org.apache.beam.sdk.values.TupleTagList; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; 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; @@ -233,7 +234,7 @@ public void processElement( Row data = element.getValue(); IcebergDestination destination = dynamicDestinations.instantiateDestination(dest); WindowedValue windowedDestination = - WindowedValue.of(destination, window.maxTimestamp(), window, pane); + WindowedValues.of(destination, window.maxTimestamp(), window, pane); // Attempt to write record. If the writer is saturated and cannot accept // the record, spill it over to WriteGroupedRowsToFiles diff --git a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/RecordWriterManagerTest.java b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/RecordWriterManagerTest.java index 5d93f10a43d1..1fef79e81240 100644 --- a/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/RecordWriterManagerTest.java +++ b/sdks/java/io/iceberg/src/test/java/org/apache/beam/sdk/io/iceberg/RecordWriterManagerTest.java @@ -43,8 +43,9 @@ import org.apache.beam.sdk.schemas.logicaltypes.SqlTypes; import org.apache.beam.sdk.transforms.windowing.GlobalWindow; import org.apache.beam.sdk.transforms.windowing.PaneInfo; -import org.apache.beam.sdk.util.WindowedValue; import org.apache.beam.sdk.values.Row; +import org.apache.beam.sdk.values.WindowedValue; +import org.apache.beam.sdk.values.WindowedValues; import org.apache.beam.vendor.guava.v32_1_2_jre.com.google.common.collect.ImmutableMap; import org.apache.commons.lang3.RandomStringUtils; import org.apache.hadoop.conf.Configuration; @@ -117,7 +118,7 @@ private WindowedValue getWindowedDestination( .setFileFormat(FileFormat.PARQUET) .setTableIdentifier(tableIdentifier) .build(); - return WindowedValue.of( + return WindowedValues.of( icebergDestination, GlobalWindow.TIMESTAMP_MAX_VALUE, GlobalWindow.INSTANCE, @@ -745,7 +746,7 @@ public void testColumnSpecificMetricsCollection() throws IOException { .setFileFormat(FileFormat.PARQUET) .build(); WindowedValue singleDestination = - WindowedValue.valueInGlobalWindow(destination); + WindowedValues.valueInGlobalWindow(destination); RecordWriterManager writerManager = new RecordWriterManager(catalog, "test_file_name", 1000, 3); Row row1 = Row.withSchema(BEAM_SCHEMA).addValues(1, "aaa", true).build(); @@ -807,7 +808,7 @@ public void testDefaultMetrics() throws IOException { .setFileFormat(FileFormat.PARQUET) .build(); WindowedValue singleDestination = - WindowedValue.valueInGlobalWindow(destination); + WindowedValues.valueInGlobalWindow(destination); RecordWriterManager writerManager = new RecordWriterManager(catalog, "test_file_name", 1000, 3); Row row1 = Row.withSchema(BEAM_SCHEMA).addValues(1, "aaa", true).build(); From 5e016355dd12156efd42bd23742e842865e3091d Mon Sep 17 00:00:00 2001 From: Kenneth Knowles Date: Thu, 5 Jun 2025 10:24:32 -0400 Subject: [PATCH 48/49] Run integration tests for moving WindowedValue and making public --- .github/trigger_files/IO_Iceberg_Integration_Tests.json | 3 ++- .../trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json | 3 ++- .github/trigger_files/beam_PostCommit_Java_DataflowV1.json | 3 ++- .github/trigger_files/beam_PostCommit_Java_DataflowV2.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Dataflow.json | 3 ++- ...eam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Direct.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Flink.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Samza.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_Spark.json | 3 ++- ...stCommit_Java_ValidatesRunner_SparkStructuredStreaming.json | 3 ++- .../beam_PostCommit_Java_ValidatesRunner_ULR.json | 3 ++- 13 files changed, 26 insertions(+), 13 deletions(-) diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests.json b/.github/trigger_files/IO_Iceberg_Integration_Tests.json index 37dd25bf9029..98be2d60cbf9 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 3 + "modification": 3, + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json b/.github/trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json index 3a009261f4f9..316d6e41388d 100644 --- a/.github/trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json +++ b/.github/trigger_files/IO_Iceberg_Integration_Tests_Dataflow.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run.", - "modification": 2 + "modification": 2, + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_DataflowV1.json b/.github/trigger_files/beam_PostCommit_Java_DataflowV1.json index 1efc8e9e4405..156f95fbeb1c 100644 --- a/.github/trigger_files/beam_PostCommit_Java_DataflowV1.json +++ b/.github/trigger_files/beam_PostCommit_Java_DataflowV1.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 1 + "modification": 1, + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json b/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json index bbdc3a3910ef..085929db9af8 100644 --- a/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json +++ b/.github/trigger_files/beam_PostCommit_Java_DataflowV2.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "modification": 3 + "modification": 3, + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json index 8aad4bae0703..85482285d1ae 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow.json @@ -1,5 +1,6 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", "modification": 2, - "https://github.com/apache/beam/pull/34294": "noting that PR #34294 should run this test" + "https://github.com/apache/beam/pull/34294": "noting that PR #34294 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json index 96e098eb7f97..c695f7cb67b7 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_Streaming.json @@ -2,5 +2,6 @@ "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", "https://github.com/apache/beam/pull/31268": "noting that PR #31268 should run this test", - "https://github.com/apache/beam/pull/31490": "noting that PR #31490 should run this test" + "https://github.com/apache/beam/pull/31490": "noting that PR #31490 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.json index 96e098eb7f97..c695f7cb67b7 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Dataflow_V2.json @@ -2,5 +2,6 @@ "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", "https://github.com/apache/beam/pull/31268": "noting that PR #31268 should run this test", - "https://github.com/apache/beam/pull/31490": "noting that PR #31490 should run this test" + "https://github.com/apache/beam/pull/31490": "noting that PR #31490 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json index 38ae94aee2fa..27b4484dbd23 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Direct.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "https://github.com/apache/beam/pull/31761": "noting that PR #31761 should run this test" + "https://github.com/apache/beam/pull/31761": "noting that PR #31761 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json index e375c3e6c505..6572defd2538 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Flink.json @@ -1,5 +1,6 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/32440": "testing datastream optimizations", - "runFor": "#33606" + "runFor": "#33606", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Samza.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Samza.json index 1d4a0589a276..f838c61661aa 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Samza.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Samza.json @@ -1,5 +1,6 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", - "https://github.com/apache/beam/pull/31270": "re-add specialized Samza translation of Redistribute" + "https://github.com/apache/beam/pull/31270": "re-add specialized Samza translation of Redistribute", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json index 64f9b2e34efa..9649514a5df5 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_Spark.json @@ -8,5 +8,6 @@ "https://github.com/apache/beam/pull/34123": "noting that PR #34123 should run this test", "https://github.com/apache/beam/pull/34080": "noting that PR #34080 should run this test", "https://github.com/apache/beam/pull/34155": "noting that PR #34155 should run this test", - "https://github.com/apache/beam/pull/34560": "noting that PR #34560 should run this test" + "https://github.com/apache/beam/pull/34560": "noting that PR #34560 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json index fb9656ec5f68..77f63217b86d 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_SparkStructuredStreaming.json @@ -6,5 +6,6 @@ "https://github.com/apache/beam/pull/33267": "noting that PR #33267 should run this test", "https://github.com/apache/beam/pull/34123": "noting that PR #34123 should run this test", "https://github.com/apache/beam/pull/34080": "noting that PR #34080 should run this test", - "https://github.com/apache/beam/pull/34155": "noting that PR #34155 should run this test" + "https://github.com/apache/beam/pull/34155": "noting that PR #34155 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } diff --git a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_ULR.json b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_ULR.json index b970762c8397..26d472693709 100644 --- a/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_ULR.json +++ b/.github/trigger_files/beam_PostCommit_Java_ValidatesRunner_ULR.json @@ -1,4 +1,5 @@ { "comment": "Modify this file in a trivial way to cause this test suite to run", - "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test" + "https://github.com/apache/beam/pull/31156": "noting that PR #31156 should run this test", + "https://github.com/apache/beam/pull/35159": "moving WindowedValue and making an interface" } From 17b38c11b0a27244b4880569def2d30a969b2849 Mon Sep 17 00:00:00 2001 From: Shunping Huang Date: Fri, 6 Jun 2025 07:55:23 -0400 Subject: [PATCH 49/49] Add timer tests to make sure event-time timer firing at the right time. (#35109) * Add timer tests to make sure event-time timer firing at the right time. * Add more tests. * Disable the failed event-time timer tests for FnApiRunner. * Fix lints and reformat. * Disable another new test in FnApiRunnerTest and PortableRunnerTest due to flakiness. * Disable a new test in FlinkRunnerTest * Take out the early firing test case because it depends on bundle size. --- .../fn_api_runner/fn_runner_test.py | 92 +++++++++++++++++++ .../portability/portable_runner_test.py | 20 ++++ 2 files changed, 112 insertions(+) diff --git a/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner_test.py b/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner_test.py index 114fe0783434..aafa088ceb10 100644 --- a/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner_test.py +++ b/sdks/python/apache_beam/runners/portability/fn_api_runner/fn_runner_test.py @@ -73,6 +73,7 @@ from apache_beam.transforms import environments from apache_beam.transforms import userstate from apache_beam.transforms import window +from apache_beam.transforms.periodicsequence import PeriodicImpulse from apache_beam.utils import timestamp from apache_beam.utils import windowed_value @@ -768,6 +769,97 @@ def process_clear_timer(self): expected = [('fired', ts) for ts in (20, 200)] assert_that(actual, equal_to(expected)) + def _run_pardo_et_timer_test( + self, n, timer_delay, reset_count=True, clear_timer=True, expected=None): + class EventTimeTimerDoFn(beam.DoFn): + COUNT = userstate.ReadModifyWriteStateSpec( + 'count', coders.VarInt32Coder()) + # event-time timer + TIMER = userstate.TimerSpec('timer', userstate.TimeDomain.WATERMARK) + + def __init__(self): + self._n = n + self._timer_delay = timer_delay + self._reset_count = reset_count + self._clear_timer = clear_timer + + def process( + self, + element_pair, + t=beam.DoFn.TimestampParam, + count=beam.DoFn.StateParam(COUNT), + timer=beam.DoFn.TimerParam(TIMER)): + local_count = count.read() or 0 + local_count += 1 + + _LOGGER.info( + "get element %s, count=%d", str(element_pair[1]), local_count) + if local_count == 1: + _LOGGER.info("set timer to %s", str(t + self._timer_delay)) + timer.set(t + self._timer_delay) + + if local_count == self._n: + if self._reset_count: + _LOGGER.info("reset count") + local_count = 0 + + # don't need the timer now + if self._clear_timer: + _LOGGER.info("clear timer") + timer.clear() + + count.write(local_count) + + @userstate.on_timer(TIMER) + def timer_callback(self, t=beam.DoFn.TimestampParam): + _LOGGER.error("Timer should not fire here") + _LOGGER.info("timer callback start (timestamp=%s)", str(t)) + yield "fired" + + with self.create_pipeline() as p: + actual = ( + p | PeriodicImpulse( + start_timestamp=timestamp.Timestamp.now(), + stop_timestamp=timestamp.Timestamp.now() + 14, + fire_interval=1) + | beam.WithKeys(0) + | beam.ParDo(EventTimeTimerDoFn())) + assert_that(actual, equal_to(expected)) + + def test_pardo_et_timer_with_no_firing(self): + if type(self) in [FnApiRunnerTest, + FnApiRunnerTestWithGrpc, + FnApiRunnerTestWithGrpcAndMultiWorkers, + FnApiRunnerTestWithDisabledCaching, + FnApiRunnerTestWithMultiWorkers, + FnApiRunnerTestWithBundleRepeat, + FnApiRunnerTestWithBundleRepeatAndMultiWorkers]: + raise unittest.SkipTest("https://github.com/apache/beam/issues/35168") + + # The timer will not fire. It is initially set to T + 10, but then it is + # cleared at T + 4 (count == 5), and reset to T + 5 + 10 + # (count is reset every 5 seconds). + self._run_pardo_et_timer_test(5, 10, True, True, []) + + def test_pardo_et_timer_with_no_reset(self): + if type(self) in [FnApiRunnerTest, + FnApiRunnerTestWithGrpc, + FnApiRunnerTestWithGrpcAndMultiWorkers, + FnApiRunnerTestWithDisabledCaching, + FnApiRunnerTestWithMultiWorkers, + FnApiRunnerTestWithBundleRepeat, + FnApiRunnerTestWithBundleRepeatAndMultiWorkers]: + raise unittest.SkipTest("https://github.com/apache/beam/issues/35168") + + # The timer will not fire. It is initially set to T + 10, and then it is + # cleared at T + 4 and never set again (count is not reset). + self._run_pardo_et_timer_test(5, 10, False, True, []) + + def test_pardo_et_timer_with_no_reset_and_no_clear(self): + # The timer will fire at T + 10. After the timer is set, it is never + # cleared or set again. + self._run_pardo_et_timer_test(5, 10, False, False, ["fired"]) + def test_pardo_state_timers(self): self._run_pardo_state_timers(windowed=False) diff --git a/sdks/python/apache_beam/runners/portability/portable_runner_test.py b/sdks/python/apache_beam/runners/portability/portable_runner_test.py index 85d1607e9fa1..e128b6a73e4c 100644 --- a/sdks/python/apache_beam/runners/portability/portable_runner_test.py +++ b/sdks/python/apache_beam/runners/portability/portable_runner_test.py @@ -222,6 +222,26 @@ def test_sdf_with_truncate(self): def test_draining_sdf_with_sdf_initiated_checkpointing(self): raise unittest.SkipTest("Portable runners don't support drain yet.") + def test_pardo_et_timer_with_no_firing(self): + if type(self) in [PortableRunnerTest, + PortableRunnerTestWithSubprocesses, + PortableRunnerTestWithSubprocessesAndMultiWorkers, + PortableRunnerTestWithExternalEnv, + PortableRunnerTestWithLocalDocker, + PortableRunnerOptimizedWithoutFusion]: + raise unittest.SkipTest("https://github.com/apache/beam/issues/35168") + super().test_pardo_et_timer_with_no_firing() + + def test_pardo_et_timer_with_no_reset(self): + if type(self) in [PortableRunnerTest, + PortableRunnerTestWithSubprocesses, + PortableRunnerTestWithSubprocessesAndMultiWorkers, + PortableRunnerTestWithExternalEnv, + PortableRunnerTestWithLocalDocker, + PortableRunnerOptimizedWithoutFusion]: + raise unittest.SkipTest("https://github.com/apache/beam/issues/35168") + super().test_pardo_et_timer_with_no_reset() + @unittest.skip("https://github.com/apache/beam/issues/19422") class PortableRunnerOptimized(PortableRunnerTest):

*o;%ZoQ-U* zrQ?wqJgTpywyLYS9^Tr}-BR1tRMp*732(0KX|6UR`A$V$didcQN&ch6ZBfWc8%LQ9-Pu8eU5aL@ z3IynNbU9T>`2B9P$1-8ljT?3228~oF6RLzXwTP@6rR&8^)i8B@n5G(~GYIh71Ap22 z=YM+R5C8bufBeI%zx#*RUiybEzyHUbul}}n?_Y#;M98fxL|ysFWHx9^`^;&dY0+mX z1RcqFv&}ey#d{`-&6}LuW#GOV)u`Y;B`s{fY#pKFV)q)Nd-tFwJS#ne!DneAMsg* z0APNbXj;pd(a@)r)EV`VTS1r{!_25?K?8SMNtsbmtiuSW1n-d#xx}Pt1=BYnj2a|? zi4iLwt0K0@&{a}Yr2^Y9++8^euVyusiP|e=eT@cUubW4h5mEe7Mnnb050e(j;-o6) zRA(oZi<9GpDGij1=r$~SEGu3sXdgMSqRUit83CvkZR$K^#f&M7CT*R7N^(pZfc=8x zG%QX6j6urSfg3YE4ic8q=pS+GZ)x<q+v|}n6^N$&w7w<| zrjt+TVWWW$>!$QKBigGm@FwmMW@4Dl!4IHebxc$bsjs!Ob}tUz#K!d#dRvjOdSqKY z0p3FCZ>6JPVg^zrAW2wgi;6qzRD|a(8>^}J-ad2n%9(f09r@tWv6~-X`t<6_D;GDu z`truzdpCc)`^^tOd~^5i?S~I;efRB$pL}%j*0E|H*gX0eyP&=BHnN z`Tkd5y?gW9E8l&0<>t+|zWw&?o8Mi%b?e3tw?Dpp`_mtO`0TS!uYdN*`*(f>1`aCE zTS`O(dbIQft$smsaVZnwS#mr#9pC-m2iUJX@JTA*(}SP67r*rI=b!() z9s%k3>%XWMKlb4hfc^O3&KKW(ap&P3@Mpo3U*=sX^Yg(YXZD-lZrSqN*PxLuEr_MRQX{FT9>khYLpgxS}o;uC@by0Nz*K4X=|( z1*ueQrI1-EB$xAvTzVlM3Ax>#Au7GHs-miLe`{k^FRTIC)rjnD!1lE2q{DH)H~R=+ zao^&?+(OXf@i=5MeoxPyHrVfaI{yc<=U=eB|BC4Pzxz7=5!v$xql%aaIRU^5N&o6% zbU7JH20hWaN#B$?IAhPH!%K^aNsArMvU@TqeS z({gGd6`HZ>l}hQz*oa`5$6+ysl=9Jl*A4~xLI6m!xrO#zl%A{9l)N=iVTqhrq@o7>%RW6{9i`G|8VI#rCgav@tbXw8Rg zOX0~v%#~hnsx-pp-lnqFs%;JXx7U{)Xx`V4Y2yhAayiRt9L+4a-?^|37y4Bf901~j616<14# zOE5KZzAsYe)Fv_k;PK^KAdGGwf}v{{!nX-lm>shfyvl_|X{ZB!;o z<$YsXX2aFtyo{KaI zuudu6GBTtert3ruheBi;WdrYJ9cAfwWIdlOry_-z&T%?g#llI42FDqgQ9>Ud4Hsa0 z=zXo|#tH(gp4`=hZ?6UP;-I^6?RA*;I!aHAkTAeO!^!Yw9<~=aFgb_JBS01Fsr}Gu z8Q@%RZ5gViymVI=02rUxN9u3sZ>vK0v{12q3L!&4K_g*xsP=j~3Xbn;?yfmN>S+T! zL$%fQH&&qAYJg)K>TepR^iK%LLUR8Y8$In%gy+oJ?F1o#4NCJ_RM^ z5LE(-T1ZxlD7rDaQN}SVL?N$jHM5Wk&sud#nOHa?AKZ919DJRbZTz|tXiDm0aa&Hw`#b1#NHixAS>>ChA)Q`k49RgX2-9em@@ z|Gw?j-&O7Ti;Pc+&rbmd2JF{z++T>zr$f^|w`GLKs5-cN*Y;QUY=3R<_P-q1{>p)E zFW2mTrLpXlhTVVI|JrYfy$zu$LoqhLvJhBHgf=tbBf032LhMi>zL}4M_HZ$=kzdFs ze33ba%ceHzDMp<7SoEq8zrX#giI|ODA`mR9@SJ+p08A4C@p;vr06n7kex+ zze^SNOf1akGJbO*>{wlJA4<(0PR*7M;__~$Jex_+;neiWV(|RYbg>Ym3Ml*A8+Laz zY;UdJ)_7n?-R`ow-Sur1eS(y7nptn#z<4=Tg-LbMsey_}xs9tHGk_M9wJ~W} znKDE4azQ);Dme5&i<Oj{uf;njR_VgaEcU`G{Un`bJ^O0VT~(TK`^&qvXAEc2f>` zFS90Xp2$Gbq)nQXQB$hKF%j3P62ONURB@dm4j87BE$E~%y)-m27STwepjC^ZB#!an zkX#f2Qa~M_jPF+n{4(L(C}&p0oaWKIT&kN*nq;6IETmJ2^~$I=fU#kMgNHIR`>d=1 zD-)rk_81s_W){N07*tbxwTwaFz|1_XiHCKL(JUimjeu+zrdy>v^C(NpA2JU!^nA($ z8?S_j)gvJffDXWzYzQGH^s)NeScC2C!S?9-9+wb27U_=3*Xs5>xZ!r zolHb0W3U5YY>bMQvhYF*Vwj3yp?Z+bRp{30zPf$z>fN~Z8i-+-E+J)rf^6@H)u7g=M*3hkx~(2|VAoJzJ8ht|zoERp;UEzX0gQ$2Pz#1esR$af z)ilnH`K_7Q^vT2dt8X2@1ORsB#FrnO`{3=P@4R*F#~;7B{{R5&!NUjlZ{Pasjq69x zp7dWj6Z!nc@h?6(^WIy9Gbe(V&c#2uarEO4j-NfbkWXn-QDG{^PDUBUtY~dXwzQ}y zWVD5hZEbPp*lP5xvrAuma{kV(j~?Fp_NNE8Acyeq@3ZX_mEwCpN5T_?JcP0kpD{px zzwv24mF@yRKFV5r^6B3PJkc-U>IKjIo1R@!dlA5%x#ef$|92hOb9($;;NQ$oKfU|W zRj!PMVf7HW0~WX8i<@8FFMR}t-M)AKu><@4j-4;<-1>Uuo>%v6d+E)WU)#I2tnR?} zie0qB%2MBFL2Be@V<%Egv* z$;BivU;tn)B9T~AUD@1F4ew|mp}M(*0U-q`<5TCIro{z+Ar;OiL-|yokP0TF^IorA zC?IsS{u$o+Z#4YND&a1NqIO2x;xl*qZ1|L4osD?`zzXreayn8>1ac*S#r^KUv~6a} z8uU-k&br6tVlI;+6Eejtl3)luOvCaiNC6cs;ZlGULS}e+a&{>l&OT~$F-V}F ziB6}(Gog9t1*$9dz4FK3{r-3V^7Ys$8^SMO=9+}^P74Ivd1owhH>18ec%dNO<{6FriT9W5># zDFS*W4lO}i%SQ_-e=0m3p0fvMZQYTUb#oH?ACJC^eu%gr5MoCP|On>m@EJzeylE(XpnN6xN&LvJ6; ze)7)Y8&{9IeWtqp=Dpobd*HCGu)3{H2f&N&Z>enQY3@U{PiRIrmf~OBfHv7&I+YDf zNeA0^R`2^A5mik>)e2cXOiFuu%WfvU4>i~@*jr6RH4ZcS{q`}ho&x|DvWP+!;kto_aFx`nGXP+Q2|%y7u7ojzI%QNR^y6`z zEb-{P6@xsclSI_ej^?m(1lrfE5`#`~Tof1=`9Z4~o*U!Mjj*Q$bRaK}I>o{}+2|=D z-YccL#1!`kQ8(0Sq;;C<-8M#_n%J%)wj1gFHZEGv7}U@PH1t6e7Yk$)ktWAz1|i8P z8ZwKSrV+Y^PcjIoIvz#GBLRkqiG2WI3K|N~YXlF$OoZ(L00Rz;)Y*V+tUxxF6Cj2) z)B0evzBW>K(@;+f1>OuWMjvbk7!wbnWK67#MUXNG0s;cnT8nC_9&9|=Q?t9bb`QC` zj){P=vE9Uhmcfo%L{}pT(Zwbq_*5*u4~A&1?X4@vwm0HBntEyv0*r}BC_-0rXJr|p zxso>6#zJ>WnOLKk!6m{+nW$N(Dj9YiT8_PSZsW@N!yjEe^W}%3 z??Ky^Z+-XKwJRGNMaNP`dG?s+`a6YpFD-umjw{U1KQckipa4{qIgc<+Uf z`1>^d)Tej;mSOSq_`iSp)b*z`DV}-YGZ&xv^!Wi$Qpr<0%%0S_c>cy0-u}G)KLYIW z7nS-yKDaj%o9>~s^)VXyhU!#S=~v%>bML2T{oil?xM$aE>Q$^NJE22To)YC+R@w6+1n`)@jV`AJQgTq zBTI|%Od=KuhaGkY4vT53uj%bCALn#HwV13DelF2-g6zyP|6@!(=?9yqXEY(52H!!_+P zPuh(}lWt^0AeW3djOtMy4OqQ_&K7uELsv^3q8ouXsoBO;LnV5q? z!ZFBRA`Zcz;CM9h2%9S7(=}p_c8p_`3+C*a#gKP35m-wGH&S76FK}Q0U`Lk{nC-dHu`RU_8#rfk!-|;2?(IwxprMctz*>fAwkKSJW^v01? zakh)xy{EITthaqnoKyLQw%XFov6XO4%l7_p@V{NCji7iJHH|t3=*nG%X6ve^Cq6(zzrC=E;+#> z8k~~iTvD7@K?V$)QBi^Sa!K$rNKlyR1NfVWoY|mMnB; z3go?@-O5m^p)PIF0s{x_fVQYXn=u1QDN{yO$~ciSOe76}SQ81ICa#@G>VYkrfY@eG zCiDu3Wcu-#RtlYXsTm6cj7^9^8&Zt~RU-lAun)o*bdIx6%$*f7foAx$DJH=>gm4Rp zvohwioar4SYsu|8a+_(W!%By1$(<^Eo0i;VW)2z{16o=?V3>}HG_cS%ArSz~C?Fbu z2@_IuB8rAj*7K=aE_s4YFbF783PMU5lv9yoq<#P}z%wyvkb~@^_q7o^>QK$)gZ2Ay zZ8cPQ3%w6U?QJD?HlUlUNS%$;-nOCMR?rIZ{Zcwc!6b}P@xx>crKcU)RE20P@2fBC zs@#rhK0xbhW+2XF$|0rGJ8Rzt<^YpC!w4P22eE!oXA3gnDKl|t@XgxpX z>Gyo$?Jv+%@BhE6$6xdivR~Ifxxtdqx+x8CYSjRv-kTMrHp)^`e zBobK6#PZo_A+wNOR3-rFog?q>F{)P&KY!Dn%5HM^@L?5ZYjpCO9^K&K9!5R7vs)C!oHHU zuVo!aRy>C{=bUruHXLksM@?C8%f5bCS$FI9minE|RSmtZor9g-y)83t($i8bl@0UITt>fu+Urzt-4mQS9d~Ae?l-V} z8k%zqZRB;E_&rWBaz;h*X_-?pqJ0$OmJ+8G6pw=BmXoHHL(?kSyp}zyX80ypJ~c~2 z?V2Dp=_$=BbQS*KOM=ebHhPx|)u0%xHxN4Q^!{lfJ|w5bHJpT=p9BufG?ul>^A0tz zUwMbN$m}vKPnlML19KXoyhN8F=g>i^i6s@F z^O_-g0RabAs=i^*JFHo&5h!CdrmTj9MVGW_A@5~TXRYc*8vvLpZGxOu+N4RFv?(JL zOHAsufMK95MHB(OlyTkTjKuh(h@w^!)r-GJ`w7XLg+97LzsP4kfxlPlB0-x{Q==6{b~%hiS>(+99}x+@+`YYiR=p z7RJISI7TQI(U6`;(D8^99Kr;fq-GHSz{crV4Tq#+;(`4F00S5kpgV_gaOn_gn9$Ee zv=h7Pv9M}%a|NPq9|2ZP>uqHYbdbB6u&p(?wpt)!dp)hUjf3ct4k6_Xyp%>5C1csh zUQ|;JqN%(e0IX_Tcjb0$TLpEXnSt)Y_qCzn;18twAxtkB*-h^6K({vzw$uW2bye-_ zuG!yNvk%kRC?uol16}RqyL#%%D7}pwOoxPl(vMK(9IRQ%nRSk5<5TBO6yCXb^z$2+ zzxwFXyBF5Z97|n3vvlpsv1?b3UObsOR&X6zG@e{CUOeo%aL9kGFtwi4t)-NQa++gH zmZL@ck-Yh6(RgghaI|PRxoSOo$a&$2=dF{oZ=DWYIT^crJaO@8?9#FDmD7=T&O|St zh+Mz4{PpLTZ{Pm-9+bFv|H;vmk5B4)dZzw4s{p!w_*=f;^Ll=||8+h8gkO3HWa)l+ z%nNUS;mE%~@CD0S|5X8cf%my}=WZ;UK+y*}iM3tCO6oA;;?*;NYERj(QliICKmBHJ zb47J$T|coGM@0~6DBKXT8`;^}*G8uk#iM*4kAcS_2M|5*zOK&h_U@ieGG9j4Yx`>v`;itZc<@l^V)25wvY&{ z=VEJ%3rnfcqbMQ-uvw?otW;~{(s3o5KWwwx7NT=*hek2RQ%ZyisYoRsHEQHWjlyP9 zFNC~#2qE6kj6LAB`8*b%*BqR&LG~*VSWX84lU7o`Ld=`+O$Mfn0hb{@W6y=AR-mJh z=T;!E18pl_Px&@7{*A1EEi<>C@vUb3%Sm58=F5a;lR*!FUn=BY@H@gY)=YSI1=z1* z{74~jw2;`y$5yiuaAh?cUC%9Soy7d(^;94Um3Ek%wmFC1Tv1J`w| z7(cU?J-?Ydx3PG3J$q&?cXlIxW;1{KP~pUCY9s9{L>!rrB^k7&Lbg=Eobqckeobm# z1(cpwCTA4!Y1zVzG(IO?n3Ke2#EDr+a&{~QX*4-A4BEt;G&ZY<%xOdO=2?%?;ZRwu zikWGBEM%UaRj4OO98q5vwtf$6?+#e`j*hxAIBY+>V;8J>dsAgKya|qhH^ZuDJ*H2u zox6Ohu$K2`Vn&?;uO36rOb&bOqdewdFT4(mX{V4o`?||TEQD@s$R?xD>G=Tzlp-H8 zvfWa=O*CL&cTUjT0K)*l0(!PZgtU*Ky>c=D*p!3-Ix|YzjEe5paXfNLP|Kc{k>$8X zExE-&X_55r!yR~Oq-URbs6&Ho)?i!g^u8GZW?oDRjnfyj+_*uIG!ADilAJ?PaHxt- zE#$hU%qwmS(DIad=~37abfzqYNkB4fvE;y3r=X<7f*Y7EV`0(&%vot+%(!TWXqR_D zt6cd>3+RBh;Iu#=L4He@wE??kNL%$8n=WISNEyd778Sr4a9|m;CSwLvGi1$>QU)DB z7iiN)Jp?l-sHiG2Ou3+yC-m}!LH-C}j~E86ia{HkM?wk_uwTJaq59C6z$fI)@)%w& z)x{#tjM8VMta%mBHB6cycE|?nCvdG6YLA)Lr^2>r2pvj%n*!T1LGIGhdJQb3k%M*& zlWiiBO+?XiaT+#uf`!v?2r344f=yI1@#72})L~#Hx!+<)QfCH)wyEB695AK1}W zv73Q}%jiTByaisnr@wh01Jx#BqD>N}oQu;*=`PDy*k?R(ICcH)6JLIO<+BgYT{*vU z;Y8-`)A_3x)~;P#dG~bs(&5mhL$hxmnSJMY@bZ!Hh0WlF&H3|(W-cC?zjP#c`Dpa= zvB>3Pp)1G3SB^)n9FM(yBKGd7KodF?TYEl8_)Nyc<81ZkQ1L6Y_$56J9!RB&&&M7g>HkN7 z{d$(i|5D)I`#=5k*;ikgCT$4HAPiOChNu#2xHmq3zq9~W+LHkNS#am>!@IZc-2P2N zPjffAlSoHX85kxT4_XAai$ETjP>$NoYK>YVcYv@_^H+8$<^eEl_cQS#l!ha zNAedB7cL%Nx_ory;?b3}M+?VS7giI~3o{ebc9GL0m^6x9M!}SU?a?tkdWKt1oi@>? zjnrua=uoFj6ss1mSD~#Mg4;ASYoiAynIR`L;^KtdBEMHQJ*~02v`Uj=R5d1^kQtpu z_qSGU|j}9*NDRD3rp%f%>MpHg%lei@1@nu=~y!DA{;8KJ@)0AQ1*W#G~rrbVk^ z(PCV*nDRDr-fjX{3J!>H1t@V*vuM)-bfs+&z)B%Oc~Y;)nABM)qNp$3Ys#99S+gl^ zGyrq~hLz4(dAurCVi=TCAJ;+j0w(N<4=bg0jD^QX!T?}0Vc>6vt@uZ|vjT>PGvwh> zXGd9HG1IRWI7LJQwGX&1Nnf3T)M;h(Yl&T2V#heHb-1reNodnh;RY7?{iEz7B#VGx z;^7P&teSyVF);vOIzD-viB++Q<4nAqjs*Y%CQM2m5Mer5z0KT#Ht7&z3_8=YWuR^! zKo=6U)q5~ao{cVWuMr>~zrmGp*T8C<_BX+j}fVG$J>8L1cE#FnQ z=Z)Tmaz1{Lf#~X~+uK#Qo7CSRb@f#G+UeBQGYNpG>*sRU&lRqo%U?gA z``~=>gY!$*&J?emDZGDn`NoBn8y8nTytr}W!shj}tM8pzdgoLg=-tz~8<*F={q)k0 z-+gfB$G`q~_q$TA9AMakyAK}Rfp#_hZ0f&Pg2CTs@E1`5it_ye5}u~T3(4^edmvu^ z!am~XDfPUbeEPo}`73(HoW0PiffxJui@$E1I;^p)ru>exS5AKZ^0rXm5PA})8g={#Vj;0flNMO!l(lZ!TG6~1y(nUh<*a&}I zBGM|wE}J$Hnl7e7%jxh+I=q?=FQvozR5%@)&V(Js1xs7|Ts1+ZZ;cBq&^pdXekCTb7Fbn#L~j4l?2pZi5*{w9a)JiXWVuhpG0Y? zsojFZ!4gsTk+sx`jYSAlC0v~Z?RxIaq2jqC%V!Qj3_HD!qNQ2qs2?dRxTf3yLfE*{IUGW^@Wv`H$1Jj>Nt{7 z;@Ierd~8TMf{}_4icyqOf>cWpIt9u&jy9?B3JHQnZNU+nhUi^fZm)O*p;eOX20A1& z*XbO$xeZ#ENik_0cUZL3F5{d_W>)r5`*$|)dae46ztn7dqhasXmV<9L9@y4cxwEBi zXM4jASk3OX%2s3>M@)`H9r1`mt778O_1&K~T5+48 z(umOVSSidn32+5*D`z$4tmYEKj5(VY+VyM)0Mlk|+O$=hG*6^0nv@C3l1~}N(?(U= zG?6v|2c|F4%a}140li89gVw`RMs*Te7=!vyeJMTUz#h@-kpqirB!FH}jl}WMpkg>U z4y7Z`OL=}Nbl%FWi0R?crbP_z2-7QO%`5o;U?%#22H(zy?UxQz+gXEpG6XOswsoYx zT83^^lRCAu9y1SR6=2N*oSus@aBx~CR?8+#a0nVcNy)+~Sp)?WH%3800l*=IoPro5 z^azkJ-eB7ZzDG($u#p|u)~ephU5MH;WNjIux{TObMeA)QcQw*_TUmqcI9L_B0U`uKtF-cz7eStG>UZjyTvc*wN6_ zRE21%MKsr7U=6^80e}Gq1^`xH_GZu~%U3TQe(&7M^>f8*XA4(P=dPd0UO%0^ek$|+=}f6z{NN09 z@!HwNYv=MGoCly<2HXPr=+fGai)-(nTYmrS%8hgDH_oqpcyaxsi-$hEaQOYR>+hae zdh2-pt>d|OP8C0V>&Q1BUB3Oz2X}t>{Pqvuln#Bq{SYeB0er(fC_((EpJC&fJ;E>C zuKPThp4W>3_Tb@ds3ha#QX=k;F%*bABTU;XrN9bj95KK=T$@9*CF3G!a| zOEJTH_aEN<;_FXue01&h-CK8my8jyj6T{#Vgd(y?KoIds(owof&e5txc7x2Sk*H*R zshBk)q6q~wk&p#6EMx(6NyT)Pj5F=fVq&EbTgrtCi;+b@qj)e9oSm6=&CR+4 zes4597Z1;;B6Hc;JhYc9;e*hXjjU#%ZMhJK^6}*~aAW?}jBh;`+{{M~<-)+z94ZDk zmt&jDiOrS7q271;EmDXTOYcCP0lhnE))FGV+U5%4Cd=xjCxm2g~N${k)U9Nowr zSxJF6N&0PJk7i*;mknB1Q_}!lCzc~8SE8p@q9<2lCsw1!S0cxjfj~cTaxHmcZ2{10 ztvK(U8tv%XzN_qa-S8@-P7Jtva$^xd>okBCpwwpW16u7q(JUnwae|jT%q!`F2T~qck zkq|e;Krwg}G82oywIeXi7*Z>q+(M)@(&#NBL9bj+kd9M1BV^hzjXF#x@yR4EjwvJz zONO{29EFYKOZob#NxN^}?)TY3e!b5lwrdbv;(?xqt!?{XYuL4=VfPN$!CkQO9W4iT zwv_K`tJ%?ByQ8CePiIZTKr3BDmMA!U0f|iRB@#MdO#pDO$w#pQKBl>?9DyGYNkl_* zJPrdB3osf9*)FH~^t^z9H?L!O2d1Z1l^~lPf%fUOs$>WttB_`J9p3I_KPbpXd9`XJj^|35EH|XUuoJqhxUk z5m?ty8MIAve{rt;##q5nS?*w24%A<&1iB6OWePDreNP^g?%cHQytJ;|)LmIgAP~c# zazt}uLS$>IxjEI`niRN_OH;IhG!S)&Ma4^0 z@+^9xB~%<95)92B5&|?7q=pnI$HR>w)bJ2W7>^V!V}|p|N_voni2)3YQ*)BTgeeAb zMx-)cBal##avEApMhfr&e4M{1I8Y~~$+%=Gml7)D%6L>B87-j^WOSm4jL~waK*Bf} zIF}H}!oi6F9(1(7CWJ0$k=STo4%RzH%Pq||m*&Lgq#25{W16ZmdUlo{KREp8_Sq+Q zFWkC#?8do+x6jSrJ~wyk?A)!>Q@2k|+&VRJx z3*asg)?cBo_S-M^`)a>@`1$?gXZK&eerj*o1>VTde@RY{vu4I;+hX&xVvBPVigIEr zi&LtL)5~&G3bGQj(#=`cIOv!bV@-=qPmKk~+|2j_TU=3AVm;OykU^mzaDI2a7gPY>^(9@;-SaA2~3|9I~{D1BgjV5$#r2pH8NU_a9% z2d9P(f$_}H!t5}h$igfb4=&7(9-bdNyl?c#{;{L`#}{*Gc6e^AXL7K!ue+hPGP@uz zHMhu?oSmAKnb@|gdgsoPjEt~IlQ6<42oL8Q!ug>(j$X%3P15!5tUfr|e{_EMz+}&8 ze`{lHL2g!Zabb3AQ(f<#&XFO&;Gwyx;i-|m!#%BiyQ&6uRZR5M?;B}5G}XB<+kI%J z>(Er!q3Nzev)%KP9TUTi!@YImgDvwDUHhiGMu(bP+w#;pia%nLHjLBMTs%6s3%t$6 zX~W~QBPW20&G#Rf>pcv)8US8D5Wa<}opW{;|%5$-PIW`wvg` z9|GT{N$|Y_>e+X28hpF@_Dz7lIC5j29+ng(c^i(#z6v8cMV3OD*NO(b*}c%p_x~S#DAV%h@cQJf*y> zYR{hX-JRLZ4d&upRf?HmkmKYmq?8sQB>J<1{1^y7MxZYp=|@NTF)@H%{(KTbKw)YG z%4n@JLZQ$KWD2%k&zB2HI(cx0MJtmA2jP6x2DLguD^_yD!*~VhVda?-4f*Dd;`p7# zaV`0#daJH9PE=|ZQXd-UO>^BtU5rojy&Vef~fX9 zQzuZj?C?wtsmQ>njT2hsXj8CzatN|m&!~$PR-5>>(c)H%w$p0Zn;SJyk~mPBJY11F zQk6bh4H%X+RbL2A1~yhq0va_}F9K|?xn{l@I)X7k)MRZjRK(E;b^D!bu9yN3Yl>&= z0Gn;BoM`~`s+n%Ao~o}Jt0^C=F0;=c0*sYT*ozne#_Uswg(G$kgI0h6_bSU7DbEEM z8?{T=a9J?~u<}Lyf&gZ>E{I|F(nP3~W3V*0-`--Xza*=t*uJ_sKfNo@Uc>?ES87*I zN@sTRPFrF}W_(9RY->t%Lvmz0_+7|MY_Y~QrAO5zh1VzQ%OZqU2|1by%ajIJN6P`g z3iO<872T$w=IYpmhLA#&FgHSwYZR3y=&O<=@}kw55we^pRi;UqW7g)z=xruVRw4k3^pqm#K9T~cQ zboj>ck?Y4tuLF{un!Iyn`Yy;hAYoIt&rIDuJ$?Jk>>Vicw@=UCI5BhU)cnnpbJvbe zUOqB*@!0sKQ&V@Y9C`KV`lnY9KfZeO`R%jM@1KAE@Dkb~^Zlz&pWoWoIR6aot^zRt zvav-Ne97-@_%}TKcS!h`IY7I;K>}xI6;$%@+iwq^J-Kw_#>JaAuHL)%@WqRdUwB}D z<}bfMyHNiEVwanL-Lb{~l@|leeE#WG{fkHb^WR#0M}Iu+&(Hbq@-N>9`-=p5_2I*h zFP=Sr`|{1FcW*wvdHL?e>vu0d{rnl~ECs#nU;dJrl~hoWT2g4KtFX4z=GK*^*OjK# zms-nmEhRb0g*nLuAhv|;baQ%=DK*h#O^GSYN-W8ZFU?DCsK_1c*$EJJaBld}+|U9D zkfizX!~1~Hj2@U7J2*FSaDHN8e&Wc!iG`UFs0w@vkO^|Bg}IT1`QgL!qete(7jtxe z{OG>%BlBZNz(pU*=&^l3StgGim^glL^2DL3lM7SF4om_Sn;F?N(%;e6T9KL&ql=X5 z%sP3rmLDpJN)0t8%2{$8O@e1Bs9YtJuVV9+Y@w2soD$L7)v#}}=fKR~(ZS}HhWymz z2%SnOm4v7@GE-D|hBc+CsJ=zM8IGMJ-L)Ee+W_ zTMKq~7B@CoGi-W&I5R#`SzDjg(pF$e)A2+>et{ctB#+#@*pdFWxv{-={PaV;GbVZt zjP9HtYMkt;8sAerwx@cuyLzOndT?hs$iU9B-j34U&4s&~3cA|Mdv@3LbvO0yY1!M= zvZt%Hzo%okZ|C^Hu89Gt&>wsVSeWgd8g6QBu%@SI#1bM`N>xYebn%gLvz}*^5~LIu z1LjNdg44YyVz@XmSQX0^MlkpWjv`*Iu^6OrI!S!E5qv<+N)tqf6IEiQC`1rzPOPj< ztSr-IqzcTDoJc*(sG@~QsNrIQniIf5yJ9^Ykvp~_oVLNYJ7c|l7(o~bovs%t6Ab#~ zNSRSCRCDEOu1G``^6=&ekufxcK=i|s;9{MW%n4@lNC~m>%ABa$Y*TY#Y(rjDV~(*g zE37g}Rv5+2jbP@4(@SGRO3b_#o3=7ZQW`I*Pt#Q;Dr!?yfL;yh8X#fqxecPXR z8Q@+G)e}&gsLGM5($T8YDQG&eY`nU3ytZV#su&7X#8C_Y2KAt-C>#OMDucF787>7F z%eM;_I0?1n7=o5F0||r5H|*7kP$m5z0JGQAL+LBFLHEFlGWQlhdtG)yYnhXGXC>PK z1{qjeT1>MgvNb)n-Imyt7TcU|Zm<|@6Z8!U`qFTLO%@zO3rJw2O2YZY;US9v%ajM_ zXqh>Bc1{>KD~y+KlGyKo=?cx-j4*MINok8zWSdlZQQGWCb#|22YE)YE;uM|GETP9M z*l{vOyo3pLL{o5*H2hE|jvwU3fx9X20U9z=OAQJSA*or|unvxxyR7E#M*DA~c_s1Y)OegS|1O0lp3tRR0m z6)z^EgW+CmJc5aVQ3Abb;3Ny`^Gii~aEX4&CTVqXa${w>HBnbxY-y^pj`cPGF#-U) zeencf7|5MV$L?P~cJJcCtuwPXj*nkEI&^iR|I(rU%ODGVR}K$eJvw~t_~^}(6Sq$R zbWHz<|!}$0oOQ)vqTs`{w@r{qKfBg97 z@uznHV9!3ifBxwM0N9In@1MW_@am^uKK=CDKRWyU+tTJgp#sYP4Zxt-_a}S({^Om8 zPi77sE~%?FC7TryYIT&}m>iu^Xlv|jK6&xXtB>!0`jeUcKF0_V%Pv5Rt()wM_JwW0 z!+!Z}{}(^~(ID|pFX_L_|7B_7A8&k7AFtlJSyf**b71z-^T)5=zxnj*=bs=O`vb1O z{v|s%siY{ix-_l5KDVo-q`fw`qc*#>Dyz0Qqqf9WSCLaum{F2r$xVw(i;qZ8hy;U@ z?1b{X1i-NRitLg8-2hQX4@@3EFmY^t3?S#29XUt$gG?ShFmrhS^pOKI#}CaN**~=~ zH?}Ybq-u20rGQr***|d%JPn|0e(d;u0IKO@2d0kf8wbx>fFL)z02$c$5h#FyilJ>*?)j%`da5qt#RygCwP6gk(4e6~OTGXL$NBy19$GUn~ic%SAf9Dk{d9o}N?=T%{(fw#s6) zYPC8to$VV$*p9?*r_jB5VkC|4i>B`I4%+C6+C&oug{VkyqO-H_YIpw)6k0%<#n|0i zJ=wQwZgkHqG+Ej`H@I_RZ*5OwW_^BSNt!k{L6H+Lx5kJwqQ&VkQcJYdVpdyIjF}lx z8P>RzjCf0Wd}^j8BRey%xS+bOvaPjl@26# zLS0;#E-6x#5E^2Vpv6STpdD*{HaZ};dQ+VP`QAvrk1x&DgY4o;bao}UxDeccQ3+$i zA`5er8L4z*c#uqrSE~haW^sa<6B!n)QxKFwvPQ@-$hamYR!ZNFcH7{!VXfP$RW9GJ z_gv=y-@YB~;ZG0Zgo-0=3Fh1swOPkiuz6xyAku?PM@JiEVLB0sf+W(h!2$+`iWAFO zd6|(FHez1X)9xt~Nzo znJBGKQ@3R3TCMtaTUeVdtTWfxVGFN{73ZiZl_q|PftkpI=`c<@C4sc3$2D7`8j_41nF+1d_=c3|#?+|lIDKuLwkAeZ8YU=E zvy*t}SOy|nNiPiJ=IPiu8m3iF$&gbr)$~jq3j$cA80st)t0^{XtYKo19HTNbRFWU5 z$%#dB`E1ASujFQ4Gaq*#;HThN^X=qBtpy)Q2@ONVk$O-jN}p#LJC^RBB|NI zat8SIrD%C{VKA0~2>`xDM|g9C0szB+gi+w03=E8l@(o7%ugjt|PTjk58Vnv@J$3Km;hU%Dt{oe{dU){C z!JZ2TdM+O5Ilq7Jg@b*U4i8;FF?Q?JxJo@nF@yEAMKE7SV*o%+vpTB$i^zGZH@7}%m z{L_bDpe@M$$#}kS>;F8V_s8}9-~9%h`S};{Wv^Vjm6cyg=Zg?zER2kR(-24|3e658 zh2WVIsxDezTw4mRp@&W#JALW=)mt}yeEQ_wXTVel7>il7v_(XH`0cm%ze2}F0)3&K z9chb;pdpikEbR}}{k-_lJ{S6bFtEi6{y(4p4D~Vr|I}~2BmJ5%&FS+efBFSl1orc9 zP*>mgpMUz}X@4m$N~-3xEJ!c1CFP~Z z+mfTQQld+2iPiZjB)|lGzoY8@90I*{Rry=(2n>YrM24HD?Vc+PX`LRPly!K5U znxBM@a})cg#}3Rw=OL2Kj~$#J-9J0LZ?b=Wq8BPCp9UG4pBkE*9GadOnj9aT7#je` z+37KmnW@p4$>F)F(YeXt@sYhXjist+4fyHCNT?_w1%!lpo!wK+a2f_H{ zKoK1!r6Xl@q?jBM#*I#ur)100a%IuU+~`Dpc0p8Oi6u8bF*_$A-5Q&aWQs9IgoUdG z5-y8F7K(#qN-Bd5^MI{h@AS7-4u4(k@VB*&UvJp{jg!wx*TA)I=nd}J%{!1AcLZ*9 z4&3DEw{C~;Mj!ZgEY4RhWLD*8j&`?C4|dHAch5jIlRHPcYFo-u(@X*l7b&H~WDJ;; z86f2#<$SzcOjW4`QE?G*spiOpNKI6z#uTcHjxfZh z0yj9Kw`@kNU+c4SozJSR2uG}ls!5GAWLZV=W~yF8)u~xw8g`h99j0LDBxF?xPD&5f zaLG!h2gzrp%MZ&OmaN#eXEW3w6BwK7T)mLQOla~(A1sDbe7?>?-S7u^cT5LmNWRoST z!!7)N=f}E|RV?-DqhW`$LF%i~RPC$$cmV|?1RhXnTwM)`6Hfnq9#f$^5I(AGqKZAofnK@#|=U00gAyQT2t!NEtj0liM#y?hb? z4CEnz*`>qRPflGqJbdX;-$lT!1A8wW?7euf=OXa1BM`uDo|?RV0-_h-)0N|s*G^7f zKQ#l6AXiU-@$7ZLzZ26ppz+Mjv-|E`IsEkg#gDHae0m4a^#q^`3?P8Ld-3t@^Y^cx zzI*-T-J7TH-aP;K@$Ju$&-?;4T(_fW5nzA5l>Zm`-DbWNVnBgz7Xh8J*E{@n_4eJ; zh6W6a4E!{JhJaCFuwcJHItM2$>;fgsAw62&E~>l#*>JsxGQ-scz|NZS8Jr z?e6I4?dlpH=ougDo0=Ngw}0x$vHhnm9KUq^)}v?7-+zJ{KK_#_{$ug=Kb={8FCQ1d z_NUU4e|s-~X7(q0E2*0g?<&HTx=78lS3d$*v@b{i7WmV%moFC^AwmXLS6$HBP}EVE z+ftFyT9MgNoxQs*x4R)9jBAUml?B!kTT($r0uZm9^w_-g*z#;>$NKyK+~J8tU6UHnewoydNNH|K#BQ zsiE1a!HJPQ{oRebTFQ2}mUnkl?{2H^Xsu~!s%dDbuCA@Ds41^$tZm=bzPo#OXIE!W z@7{rd-jSi+zW&b4!gQLF7bIn(EFkC=_@o|137=M0{zYyyuAo&R>a4{7nXCP!W z1dstaSs02Jg#}0=0;Cavib$L(nIE67jI$`yZN~f(OI}fGUQtF)L0VL-j>bkI&`w_d z8@B9Ny4vY)tG9o(%IWJ>PT#C^Ub?|``DU+`PVlu($n{Qu3W$wc{MK&rUhU|+!QIc% z*UuH^>qaAC%5rTZdv?tXLdBLKlY_g5_q4QD=OsnTMRbH9IDk(L;4=dGEVM{S70LOs zP;Gc(yeTauEICGD)=P{^QK&rJY$~ZN-?giKsIPlsxO-x>v%jygxw$AkGcq(vC^v8= zp&XS-8j%vNPtr>x#RLJ?Az;f=r|-Ynxa{kVE5CDG^MlvA4ZiEwd#?rA0^hXGZ`Jqi z%h!0XbwW8Km>7;$CW$hLq74kKoT?JhR06z^ju%i#5(XxO;7><*QG9oxo!7drUbbz? z(k)*v+x#t*ElZX;EcE1f+wdtxvX>da0m}qM;P@HrtnZx zm@-1mR|)A{dVE}1NtUTL%h*vA+nQ_ISzvClg%z5485&ZCidY2bWnvYZxs4f$(il#@ zfsv!9S0%`H<{4{Klobh*rZml>gaM3o=SS_dhGnZrFS~xXdM_-F>j|Av;xfDrMC=V zY_a$JL_IXMII-A=zM*;y>M~V1UROO-d-qAqs;s93SBzbLaG04z5R0PM>wVrF7zX5wyZQb$^RV{%kWnz=nQzQt;8 zOpB_wL^dQvG{k8ujk01bzbs5pq~~Nv2&qD>O+hV*6y}F<)8!NZurwJtT|rG%(()nz zz$AG_QGrQTXi`|UA(^29Tc{{2R005I)rm91q)-uuUYM*E#LGERLYjd`G;j&gBC1IU z#TY_Xkl{R}J1f8$962a=U|l*6PR&GX*;p_J0OO*)#KZs<6K4?8BZMrYh@<9E1!N3h zm4t#7kdu)q7lfjz*(E*=*9;Mo0eT;tsW0`t`%lP!?-677O-Xet3K8(z#=2 zPu_m?0Kntit*bnh$PW+qBg6eD{yrpcZ=#nU*%uz{kEFp-G(->$fu;xI*+E1uhQz~> zL+}(n0SpKn94Um%Qt+fkjXE|mAvdS6sjXsXU;UoZT_f|u2aX-ReDms~=eM7~e(>_$ z-YDczq#}D)!k>W@4a~Y$Vy(wtn(K?Y+N(Nh^J@!}n@TKg zkjoYa`S^yr+_sH{|TTT|7m}Fq^aqagj}p)x(25lcW7pqrJ1^ zeG{WSJzXu8C7CI4+E}9~E>fNpXG~2=Fhv_xVS1599io&8R7zQ>J~TSoY_VkIc zRn=8hHCD!?$I~<-jFN|va}Z*hAGqN1@ctnLm=GTzCIyHn{=#5{lo2Qcp#cLUO0jeW zoTK*T>b$rbABxBg$8pAU-LXs$8b461W2z!Vrlhd6+~laFa55X^9k^|W&)Q8c%hzmQ zvc~zlADq5j>9}OgjumU1SFCqi=>S{j0AKHb*tpq$-A11c-dIm00p{c9>gwX?=Hf&q z;mQj0#(KN$0E5;Hj16@4cD1)OR%NCqNO-|4B7#nU(SuPe4v{Tlb7W$nE=(B{8qQBay{$&56{DRt4Zuy}Q7yhb0V)yHdP zCNWb*^~5{>;QalPEla=Jxa`}l>y~fd{Db?Z)$Z%od#u~yyJ?fp`X5|YtaMqm$=AUh zr$o78H%DvW_~z5LqV*Hm$YSvHKeI(ljVS00AS7OT3}#(#j!2vniL7DGKyOn$w?I; zjAXZPysJ%yZ%&p1{c4I6wI;}06O^6l5Wt2@;z!F8#wsimRT-1DS(CN7(+vdxV3YMw z89i_>`zDqUz-H|R2Hgg0s zL+yA&&1fyu6mc=bHRVIq<-;`v1OB8rbk@d+pn1}+W8sae4QW;|RVFfe{F2FMoHYX`~CjT+#|!NM8X04&Un z5a@;Tcf~u#l-umgyce-xwbUDATzqAB&DUsHriW%_1yg9J7=HVx%lw< zx%*d71AyH*f9TqY$x90Z7Z2$lk8fVyfBNLs{Rh{dy?FG~FQ0z?-M#*2h5-No9z1^W>fo_cgA+4j zvj?x=y?^`RkCO-X9eEMyQXFpw4if@HuEba(&_9>@$rhmhEza&2Nla&dWa+uoWz{zWMm^ z&8LsR4*$5{_T>%!4ztBxR=-b3LdN#tmtUW~`#8RTzqK@5ZW4>b*ZJ8LqaEt+a=-Sv4*Wf>(o$$6Ra zfLl;3AkkSV(K%_cxoLoBG5MKEyE~dDhWn?+2F6Bu$Lx#1I$G*elcU)z93Jb7NBiKA z-q=77bf6pD--SUXmK5g?^zRuT?wc6ynHuRC8SLJ@tEs#sGdfhP2*E4GG*e`FQd)+_ z6it=LQEVQH$%a$u@L(E(%0SVW7zX%p7KNx((r^t=E2F4H7!@C*1y4)ATcJaV&a12>e4|)!EtH&|F{J+>}>b zBvVL8L^PRmQMtHGo29M2v8SiAZ(z@^u3crd zrK!1z5s7+3f<_mk(8efDDTe6uNKK5Ar)IO&^k6B`3+J}hZO!+NE0;O0S>@)m(br?6 zugiMxt$;~R@NFAB)~$5S0fyrcxN3nULM1jRd3sq$n2ao;I>8-QI4xV@ zxNP~hrGQ(@w=Dqx``+n0=;5}d%N+o|KtaEOdwsKH^EY2@{PsWBef9UXe_IX!=DZPx z@L+K9>ozZQ+P+35=R`;86O+ubNwJ1FlSnV83TR;ld7&+~EW=on8QuUGmL1V-3oVG^ zrD-VuVAxtJ`h*4qNEXYy;3Qz_9K-(@=S0 zokf}~MwS^_0AR^Hco^O}jtR?C2RA3l8so%`vEue5RY$U>%NjmVXdWzz8z}_C=^+DKoI4f)oCiu9}0)1FGIJiEe{agUO z2>xy~q;D|Xi-Ql~kq|<9P>NYoTaun-F%(!$t+iSGyQ@wg9(i*2?9)3Jp4`6h=;pac zH&5TaxNrs7)uFx%2fEKg2G)CiKSZyK2m3EC41z%8g`rCe!h zOUIy9Ur+$mj!giPT|GH{`~1TFYo}j4y8iyv{r7Jket7ffGc<(=0=d_RS5JX{y$1k$ z`S|_IA74Gc_vptP*RGwteC6DakMF(v_|{(U_dnIp|8d{z{ZBvLdhp|cV<%g?d#nXT z;W4p7wURF8&?OAIlui+lf*8m^Djdl~1yBQh@P3|HFL$hmC*IQ?>+X&B^dWeGqc6cL zfZ~gw_`^xQz65XkN$5Oa8x*py56Q=i?Bhue@TCTkWddzda(YEwLEG-q-Gdc-#%l*> zTgDD{j33!Od9-u-Sl9INzI|tU=T7aJInjUMRQJr`%AGwql~uVFMJ09R-6K6m&K&d`AS7FtTh$QW2TGe|}Y6qCVos5$`_{L-*t7_JXVjG!qnR0W184hrN1xMDrsurL^% z9K`2f#6q-4jOL4wTpohO3}DcFX(VS%z$QPpHJ(me1H5*i0=A<990PqE5FVQYc5Lw8 z>WlS@h&Ps2mPSPx17LpJ9UUFFI#8+9nyT{Af$q`4ox68eme<&9g(rxyftOD(AIpJ+^N2b>AA`gL}Z7a-Swd?AY+rQuH>*$H`3#6iGBAQSyk{eZ$aHS+d$qyAF8Hg=DfPYI?IDWTc z8^o~h9ha@#zI=t_(&Y|IR{(%*Te@`f*Go2j_4WF1zFPnFk_}%k+whH(>qaUai^ln_ zSoL=TA)ul-V`pUt$C4E$?|LiGg*#H z*H8)~LozhfA``bZNm3cdFN~n2$%6_)gR9Mgh9qfonyS^RYqx~~fPw5Sj2^B`YDiUN zY6!&!Mum}`B0w7m9vLDc)IKU&(3~J|ikI(9)pcfs?#YfAD2y2_GLM!ej{gC$`kd)T zXbN$vv1GEL6snSMt(pS>YlHwcRR`5*KrN;~{!0U^Ew=ahU96{vHocsH+9Z}uG*ph) zL#^ncX1!>n7HSw)G0`H#18yZti8dWGO-703q2mQOi=2`yCno{0 z>DjqPQM#6E4dW-unTbl4MZ-&yu`_hS%m{g`k^>U2=9%Tp2mv_~h!+PB00uB-;1aa- zAT158B%^4)JGe*>F%bsnMTa|!NdbUfDh4_PxEC2EA|V6>m^3&@$02Ju6d9c$XOaa( zG#%;72!ip6;4>$d6$A^xBLT_S7&sS? zC&Jvxa8DY_mx1=<65vuME>gv-E=VcKj>}6kwpM5M?F&+_II5J41)};|Kh>HONWN893H;BFa#c6I52SG03g}W#e>7({H4Rd z#)dB}jGSK>xo~9k!eJ=q4h>&CGJfOC{(DzXJ-Tz@>B9@pA76U=;?CzcPd~qT_6gAI z_0xARAHRF`0Urm!AN({Q@p)%^}A{LoXBJ&9Z9u~_D3S`3JbU2LS??d$QzAjnUN{d= zth-x~3y2TS!=K>og>`dByMmM8=z(!te1a#=1L&GJ+1rii>5B6Nml0PCEQm{!$3+$F z>}(v{-!yT!aeSeE_+aDEf#%VJZQ~1VBL`as_qC55Y8^RH+A(A)spW@9e18QUaaH)49$f?OXA-B{L7WQ_s?IudG_+vyFWhs6{`M!8D&ZGC;0}@ z+=*OxdspqlA8$Q*{NU-6hp(Q4Z{zc~ub#dJdHwV+h1rJ6qUhe8xidpmll>)oTQk~g zEG?C(b;T*gS#f#k(Yfi-dFcRMG5P5+h1S>-TYPb5T(%{uC_8oU?vC-Hf$^dKiQ&Gn z;l90lI$K&=k}S!1JlfyS4d(0O>$x57vm?NFI~a)hAq{nPLjygF02>?X>+S8{+1XxJ zQEZ9`(`gi8MuRCiH8LYtotVLkObFISV5O=+E+0l`!I>O?8pE3$>=zsy$fV*!NMr$? zB*tMyD6AL}B=8Rm@e35dP(lPsj3%h5L@f&?rv^%hfg(b%E<|sQ)1*WPtGHMpF(?F! z72-&8Qb>d##3)7cS%I7ovLcif5yy#2BI!(Mi4G-HB0{7{feay#!g&H;CdZr32n-S6 zrAo9&isbVHL%5#71a}NV84+4lU(?W3YcS~j{k*qt-?nYr79O8lU02@K-Beqj6P+mG z%7VBGq9RhPG3!GUOfrLpBB0~Aq(Jb?M-K{Mq63&%@EHfm!uwMKDPl%mRpH*jJ$w4Q zi>k}A%1X*xJ8ZR$=HkNG;_R5b)TnH;F~eX=Hzel98dD9yQnDBLNJa98QITFa*baod z6WnzR%w?0G`;H(OmJi*~awqt#a$obk^AGDh*1I4*0?2^`E`g(DOAQivgj`|ND5BI9 z3B@U33p81}<2%T}9F{HHx|pR)w=A^}mM`19bm@j~zFGU#-&cS0_cdRCx8d6*Yrb;z z+D0U!e0*IuZ&@W3Giu9h?e#^ag=usq*42C4R@V(!VnA9_cuBgkF(;-WE3z@m*peMl z7#)(X1(!=)wt<~v;99k`lBkfH1aWx`FGo+xRAS1bp!)dw1aZAZ-jt?pO4Br^LbZr{ zi(>l9;!4cCbS1Xb$SMk@CvuUA98{)+oG!#w7y5HRhEua0Trtlsw^8QFYYfd94IRqChdObLRhEO;0amyOFgwG*AP*{?2Od^v?a52)%S#6V`UL_p%-%j~cV^PgjCdek zEvYfB8L_RIam{H_&8d;K@%oxreSK_bb)>q&pePNKl!S|m!uSASc}i-Ah>*xb$MJ#^ zL^!LOo+_i5xj`{pRJ;hEAR#8pNU2JyO&^k>n~ z!gy`Rx;x=Kwv+tb*%(+b+!N*L2^Cr9sJoPBa*;l;g^uO6IzcIVW!lapum?LD`D&xM1%myZlyIWltf$k>&`qgNp1 z8Uge=zaKDc==_1f^ZN%c92~lMXz=2~@P$JoXATU40CWM8ojo{o{_xnf)BEieA}21N zox5;m=KifSub8i3_KW+!GwhH2qxU>i`vddiHcpds9TBi6LfTnfO2|5>7=R zXb1>k1aCku4~(Y=8ZgYw0|Nl&3I=W{7Z4ZZ4i}U&px0tN0M9UPu4orGjH?UA)fwly z1MTdDbaoDM10RLGh%iN5OhwmV?Z8ag-ieC6Q+54wEu#l_jvwltIy`*v?AW2Rl^y*C zOI}<~nL0k6rI*k(Jg%NE3YTOSXPvxs_WjR4{{iVw_FDO~SFe@S)V1&F9i5uJe*5n0 zx9|5KJ}y)$DSR?11kIIDdIxvDc=_-nlckv{WA%B@Yc#Mn=huvBJnWPFM_G7e&)Xl9YOkNCqSf&fvnROkYYcj7kk+ zQz=4XumFYU`XcDgKEV#2#7&;0jXu;ZezdIt95yRgn}SSaC2AGL}vth|o}#@?cdc*N`Yl$dV@Ia-vem+94 zVoi`(g_CJXiZFsgkCSVIB#HoTNDzl-h)*c0EVWqU`68CDzlXEiHWznC29Fw_7M)~` z6ly4e!A^dJjfh}J6x|I)^Fapt!-)RgI8X2c~s8=n$~o72_F} zVAwU#Jv=+PYjCfnAUn0Bq_V5KxU zWCVxeN5%S6&?q($M#sA05aj z^}|-Q6~C+_G}B(^@JO znU!KFO$%?ZMb@N;Hf0%VQ`I>UtYif~SwYMS=cK8E^CCF4NwV5RNq#sjTSLgv5-MXj zwQ<5asC+}&nxSn-RyJ6aE$P~>d{b9ZRBi-4MS&@eWMwG`MzU8d9d6-aEL>DY7`rV= z-ISo{Nc$aNd6B~-3+zsUEIEMx&cDhmo^+w z%4Y248-QVR_K8G5uj#rHkjXk|Wf*t}?Y&Z0FvGh!N2BO9T+U;648ZG}-)ZctQ&D~k2vBAu{67gC_*WXUNR zVq#JVCYpta;ULrG6swAE=Afcjfo47?K}<-Nk&|V?X-a0Aieu9Y3XF1#f&i%c9k3YL}^3M7B3x@!udM+IrxN>yl>e2D5N5&VA zR}PO|UKqV}XynqN(MtsNzc32f*wKlrXAa!C zeDeDFBl~9thX;2aJUDUb;;A1W+&69=?AG&~@+a(|d28-hcV{{)4+W&YwMX z^zea$2WRK^PoKVU=E=*K(5;|BKKj_%i?Y+ppK}-y5Ep$|%U?t0hP( z#s}~3OZ4@^dby$8+=Dzl(O}@}3b=)Fb3=oHvkTJ239)?#YKJTI&}B#9_U(aA&Zr&E zDCceP?OR}uTi}iki0#`DJDiX^9Ff}{V2&<9e*6fdt+B1Fd$4hEu50>e|NfIRCoZ13 ze(&bP7x$jMIe6-V&Ya8-HK`MlG|BPG_(*M{F{?cH?9~guEEX&Nz5oEA7rg$7*?k9( zoqYe(FW_B#{Q1{wH*Xy|acpFAd{<9rdw0vgQ1^w4Cttky@%8H`Z(cun`})bd*FeI+ zr-6rWUf%ypS$;%CfuX7>tg+J6Tpd-OuPe*Zchp<8WxRO`!^wYKSroL8d1u3`Dts zs5B51x*(uI9082Mfzen2bS8$&po&QpJ{(Ei?uA?Lgj(r{8)b8bRR#aKb#xHH;3EWS`({lgt4(ytpX=yq6I{XjLFtWpovR!yf!5}x}Yk) zyvdkfDFgmzjAiJI0G|Y9C{}KOq6o)GLvhkDtUMI(%%3Cnp+U2n!6GqNAt11V5O{At zl!qs5yPKb50N$G>A`m!mAKW&#pf#SjwcvGn<2HI@wzwiTJNj+h_W6rWB|7zrD}WBDZW{TTu4eAWYieed#v zBYcMs!7q@8q)DifaG^R{rH>2M#T$4!0r+|SZp+u-Zu-w}H$gLme_!+6Usr)F`Hz*~ zfc$mUw|`r`^lz({{`LEB|F+`WzyGlGt2N91bLCfS9oIX!ZF6vTaNO6E?A`~;*{P_rmIx9qE!?Vy?l=c@{Vfh}ecV3W0wiY@NCG69_|pRTK(s;?cd zty-iPv>kH|j@T1f%xDj6;>8L23PI@8;?4I8N_7=)HE zLmi|R8zBOK4HVjjifluLwt@VNKRm1#0LlK3Y6B6K3@M)7tlG2D` z8eTv~hv1P=uU?ctGtiG3;7twi0ulxcjE3@~V*|(tZ=APtkc$Js(;0;5<3bJep};)@ z-5juXjFAyuY#c&JM$?0Qj@iN=0r$V`=L6p2`ac_gp{Ld+XHDt<$4-FU&r; zeCXEcnM;64hX$`49=U#O{KoOg8^@<^o|wLI9AxU+k;yAZCaxWwx^fJv)VOqP{PNL> zYvBBenaf8e&dd*;o*z1UaQMu@kqgJBubn%5<=oLj`=(l2>x(Oj+xP68+&2ZVbp6Jq z2luYLd3NXB(>rgU-UWI0^zN%CcOKopdE?53GbfK8Uf6f=(EODf*IvH+050zK%lgmn zK7agdAO8tm{}N~^)$avnO-=cUmng#7Z-$^TcDeBptB3g z#U1V8f%S67c(?|6Y)824KsY-iowviCoB|vj0kL3?+h9&x{2e#?ZQJO-ZIj6!XuOKwg1wfncetcL#MW!oo0U%mTq=HitPKSRx;p!fdkZ?E3Ge)ROg^A}GZKl|~? zvmc*5fB53XgIBK}yasvs;ML1}ub$s|_4LN8r`Mi8zVesuj_loSnLArjcC}dc?zHY{ z&)D0ZF|s#zqPL)@ExolWp}s7(rqo%ir6>-_JKDCbnnKp7G)S@sa+%zV5bN?dkai zGGi1?uEhzJD1j6$l48XQtV~S+XCyj|NQ;$);uWEIr3NVy`?J|_7CV5!M6#(wJ|4^T z^}=l0=D%#S-;&L~OSZt4Z1(@#7XPm{!@gbX{nc8ZC2M`YTjRBS8*-Be$-|FH)MsX< zH#KY1tt6Ek#it>;II@HaF5I%HFr_(KVU9PX=foCPB$QN|vhoDRNSaa=tWwkU270KO ztTB<)kx*2T!TK0!qAk=~%n6G|aY9gRh93pFBhYaNe3Ns)W+&hETfA30_^)%rI(QJa zxdg4>9=H-@2V#{ga4!&zQc>S=}W5sHhmZKy3CWG}3QNLFz$ z(h!VHj1mb0I7}p$isJ)@v4~QdFCD(of5Qsr<=<^vx?=mPZ7>%vJPb}n(IpH~s8AK9 z(#GofdJ%#axYlL$Hyi)<)%w4Fz5efiTl39dS1$RF72o~G_e=iQif{kF<=_0ThRC%r>;!!rL?Tjj8Gs8Q!WQlo%MvApsKK^#+W4 z3JaAjCX|PA+L9rFb);yzZIFQt6h@Df#zO$BOr5CBgi@C^S(h_aZ{O|<=$C!Aa0)O9 zvMY#KGxbp0Ly)PO;>oJQ$;yI>ihPjq^1QK%ys@f$04;#7>6+q+n!@oaC=*qHf8}Ep zWkcm9&<(Ka|DUh70FLw67BFwsZq@F4H((Mo<7k+fnHe;i5i>J0Gs~8lQHptgnKOouM$^iha$;BAxnHpFqQtoP* z?P;3w*UjHEnHulx3)TSb$J?~*I+@yP<~=}T69N}ts@}{jNR!mP4W6VyR616 zD$%gZ&4S7(NuG`c#0z-X;woZ>nwG6&6-5e4ZIVK>urx|u6{{(;tLsxN#a2aGw6-`# zTN-PqOtREwM3=-F$`egsUTD*#ss%|(Zi<=@49q4Z+e8$*mc-^jy4(fimz277^9JBU?nw1f48KBQeXkMiI**ff%M_lC&JEhD{NZ&{{4{&88@r zBo&9MU=ZbW0=US>!r6#W;9)WfM$e;(sTd9x#=?Y2={OCK8m$p)1ysF+T3wVl+*8xv zR=%#kZg%I;*>84S`gYe(-;Z4VVeI6>Tv-Z%*~YZ}n&+7EL#Pwl<6K(E_p4&6Ep zHQKm#V&An>`)`~*bp6c1pHA<;aANMrq6x5_w4bxQ{T+~@Xhqeqq8TD&3}J< z|M6qMyvDl+`tr(3iyInyH*DHDJ3lhNZ}-^f(Ib0rUp@Ww?!{M6Z@+(b|HI1%Z=c_VdSAAK#t%>Ga;Cvlp*j`1RHE-#&imkUZ|XTVQulxKZkd$!3%LnenbQ(0?vRVXfPNhJe&-RprBApU}FRn2o?ds zK*Jfx2zmsJ4i6>6g9)%;A}oXw0np_~4E7@gfe0W51BO8WqoZ(QUUlE#-V^7}-@NlgX-$Di-szkm7q#miT}K7aA_<;$l4TCbnqdG+hf7f-J}e|+uP1Q{Qeob$t8TAGZH^a{Gmo+b*2kdg931!?S&RCp%}yJElfj zMz__DZmSyKT(x7kcvFAzjiZn5u zADzO9N?=FDaqaQ^n0TH&ifPi(G;)H3kL6RbTs&JrV=D0eSO-tkSKinqUf3^v2ww)# z9fPP2Ug$5~5MR0?zI2KB$^-2P=OegQjXpcOsBbW@vsaax1OUc0iIp+Rs8nlGeri%d zW>iLcRAyFmZeDJELtcHY!fwP1=_EN{5f>AcS7gl0SEpucGjcTPS^CU^vcA>B<9plI zZ;2_$lf{^*T4pFS$QSSCgK_Z%fpzu6y9H7`!&qKHG-nXORM#MiO9;(1gz6Sd_6#5Z zdbxOGT)a?#VQ!uXPY;Bj4?Z-UgC?t)R1KG^KhE~Rx=4;cU zWC?1njZf86$!ao51<9o(POC_c6vjnqb8-y%1+tW6L5!7S(XfmXikgMwV*)8bz`Yzo zogBhk{Rp9O4wfXPN+QL^M7=HBrixM16^sy4uv@sRbBLpZ|FY$N%a(dAU+U?+#NFj9 zH|H9wNvy8MCq+`;P9!J6b%b!h|Tv9-zavKW3rB)ibcuS$@wtcYD* z8QYnotxuHH#Y@U9%(fK8K(2AH&^%mj+fbXZwkmE-W&Gxbv@H$k8)}nQmPXgcN?McU z9jS_31=fo9PUE18<q4|}AGlTt;eSluw6aC!)VAFrqG6O!f0ZPp*oMt)M zw*au7_Ob5v38<*Kb+T)*Et^nxQ z`kK6Tm07FH(uYgZRu!iXU*lRP=NeBU8&Nj1(1Egn34Od5p5urmRUcRK{xaOyY_-LxD|Q5T&k8i7JlO7scp+ zhgBq+GYygi1v_5BN>Xv`(5Way3y%;bBpX;*9TR5}&~+@Lo=ws+34ml~A=M(HDd{K` z17i|U%tCskm>I1U*c5!Dm}L;LOcJi1&rq_+IxY<`OhiPkg{%8&rgjdTJhaZ)fKw zHmzOR+TUG2ysC5a=2d(4Y#W`}Gch|pIz7H~Vs!P^Emdvp>80hFm9^EqE4w!A>{`F0 zzJE!a;e(%V-1_O%g>UyApWQLN zbJf=2=7Fa8+~~+8eQJI}Q%A$hzS$?we_dQ{`_JvMf6?m$#D>3D9sgKpCz{ zXdo^ugp3FxMfl-Ed~twbp}@uhDWRbZIGjtc=Vx!7n?H5!=JVe^E;=Hxz)zq51OZ+h z0BpnVv4q0v+WvLzt2Z4#cM)1PglgUXN1u`3K7D@k+lRLw7Taq48_&Oe`uO(!+c$4t zynOxa`LjpQp4@u=%Hk>@X=DYpt zj?Qm6Jhx~6-1xrv$$fK^lMujm4-T)3OG;&Nd0Z}^%?2LEW--|e8ciq^)z&qRkB$KS znt}kfxxJ?|yS&t#oG7;$<$8)L`8iSL7133-rlJycPNpC}ieZuCl`OQF1mmLosUd!pAWxjHH{L&l5rGpE`9_{0 zSs$4hNmnw0h{3)XA1|b*H^S2w?&TKZ>fr0_=;OKE%X68B`&aI+|8#Z!r<=<^-2lB@ zzHoH~ldn8nmwCBG5HTLXKL73T0^?%bk}Xk-_PSmhQT| zo{H3gs-(f%l%d*`_B=~fg0$GiD~RM2MF|?xv@0uP`->vmGF43}imGTqRTQryP1Tj9 z?aMa|msy8PqgIvLR+UC?X-wbNoB>p?)k;E1m7Hm>H;=9jcjIRX@MFc^*0v3|d75=-RugeebGH z2w;n+Fb{R^Ur?@n{cV6!V7{-vb-uS{x~FOJM9TRE8{0R~y0^b|Zyy-j=6YLaAs}_l zEN*cP^Z)~!8SGow4C`Bzu*FqGXb-HnbGoM;0@xzM`a7q3+sAua$Dytq(AL*vck4ta zRPQy>)i4QRtYM1tYde%c59pP*vAzJ%Yi(KPnv#s+qV$2hqv+VA1=8RxB8HAblu-~$I@Tnhfw0NBc7*_ll$K9h z=(s_X(FrOx1u#rT#YxDRMVQGbSP=m!BV*+h91jg!1lZzc7$ix``S00u1^9sn4- zbLP;`3jn)*`oPsw2QHl4|J|{GR(YZ0d^2Fba%phLTK^k(hD!<+l=Uq5*5 z!od?q=ML_j+&?$9e|{POY{!mWy@RW5u?ch*m&+6I_(C>^$L4TZOr}(-Xm0JC7@wG* zo&XG+o*3P+YunJ;RTa%u*=33Og`NLo9JceuWbI%Tw60WT7Bhq)j(TW8}dM{~($V zj2BANGvh1sqWCrT@dQtA>7u{kj*#TiK@`Ef=0Q90R>>1o!qWV(#! zj}G!h`TL;*zyv|ZQlu=JoI{k*VGN`%HYk{apsV==F*Sq|5=ip(CwTf|U3}0kJ}4Jo zl(R3|#T(`9voQLioP5xZ-WX>etdlo(1t6Ix#u-8#+SwE7;tqFnU98zchWg_X{)F%_ zI+~#sh@wDz1qoJWl!l~}pyjLx5eX$BNAR)1%!p7X94)|ejXY(nLTnWy z*+_4cuMft@7vmd<^~bSM1TMiN#K+Os+rh_onUBvBFHgX*FWp_gfJo-@g`4Y_9&TT_ zySRl0h7&PgyF2~UX_=Fc2cAY^a_KZCkxIwYm?Q)i?&a$$;!`W~ll$s&dn=L$tCI(+ z6ZQgmk(V~0{r^GI*N>DUq=sWXGZQ0tU6nRaYxHOVe8!c?lP=gpMj$Bi2AIvwb zEQ(xR8nd+_V_Q?&n##E5RBc71pe;$+7%NI-hg#4c>1-R(2r zN1aU*9re(rS7+lSRQ^@JxbFqVMS5+kDF*ZcBZy7a1)FONH`L@V8T28 zvYe8lpaO;gfaMx_`DQ_pRa}#xtBBR)7{z5kyzJWCNJUkOwJ1)XZkA?QmAN)eicXSh z5HDo@Qq3Hkk%ciZu@)X#!^CMhL^Ts{-~o(L^lU01jfqdU$+!v{MnXbpSOg=V3Op=E zB`^t?S^z2`Q^lbQiD(6#0HCE@L)a~re+Wg ze7cm5qa%ZP_y{$ZqTx{@m8_z)$ieoCu7;A9s{B3c`+nFza`v0CACExwUYC!J{d9ct z%6C%>IfrxCPwc&ZVjsZR%@g}>odhI1c;)1Q^WQ^VUia^x*s^<5b$erMPP#rR-jI@{ zNlH*BCB&DM6|{Ebx3pz9G-cK|rc~FZ*SF-g_vUx>@ zX0$YyY+TbeJ+k`5(edjSk3IPL(vyd`Z```FYV)ANE~ct*6gh@2$5O-?mYkTFY1zDE z`1HABx9|M)_{p8uZ(e--4CxMZB=G_&7Bdq6#jt-f;PdB?e||oE>WDejAha=sF?3E8 zg&9d;S%^#{RTHP)IXCj=^T)-$Q-4CGSARTy@%)<~erW3H(MH=NXoL_V0$>a;pwSc} zicCV1h)7Z)5qvR~u<#PLF)eX;=gu1sA49LZcr?tv2?XtOz4`Qc!^lj-;D#*|dv85_ zy6BF7-LUv>|5q2s{~3R??enKUKLL|``{wD(XFtFC_2%1WAa1>Ve*4{vpWnW``{w1{ z*Dvq=WB-vI6Z2d49i6>!?c(i+cYl6#@5a4bx9(oObN9lfONY*$p8n~=zH1lvT{%B< z?b7tm*JmI9eBk;0BQGC(cmL|KQ^#fx@0~a}KfQl;YHD(9$Ijh7{lk&=H~=uANFo%9 zIXoVh$7iuvDwVdkZ)j>_YJO&N@9f0B*{SKNksZ6Y3~%UPxwdx4uBx%o>a}a~TRW1g z8{_jz?Dj;hNXx-;F#cFS515;C&CUxdFe+&dT-7>p0{L-}~ZTwOw)z0p45 zG&uO&cZIpQ!5!TY4ju?c4}`NP!o?He6hL%`G5mdTZr*@lNN~}?3+d>M2EOIsiFEWp zI)R(rBOKjf&d#AO&S9ReumEo~!ViZGrl5#YCf6jQsaV0J2+xQBPk4Ywguf3uIE)Gp zBnEq+yu5I}LDX=Jm>{r8xluBbmd`LN5>#LHl6*J)o|J-kxxe5PYE2Bs9hu5oC;OpubCS z@QT2I75;w9eSDXCdoS_!{?fzaOW&uPS(n^f8dsGl&$n=MO`N)9ZGEb)CCk{JYiLSW z)Wiu(tn5M)vn~z-SVxw2xXiYyD6%_4HJEQ&Qxd(iIcsNY=GvS~?rY@O}Z(ms0ZVCd);UoM6ds`^|hd1TdE2-R~BrlD%@04u&N||RdMPt zM6dYHWP4jeR9ie0eQ9z1>G3VGmP)g#GE!b@;1}yTg(_yQl#(ST7N{8&QQ`~*C6k1C>GfwWCEm#32-?T z4FI-K*UXNBI!G}!e7crT7Z6cW3RcY`19VA9Xf6h(WDpl03JFMnG+<^dWGEFDz(Rm8 zW;&X4XG*|uuSnwH^#;`Wx5iZXqA ziYy^U5M^eYwC2pr@}9xkp|u6gT^TjaNfq_!b?tdAeHpdw>D8@8t-X~!D=WJP3LBa; ziVAWIGE0gxS{qA-2b%V5TY3EO)b-0}?%la@@yfaK=2Du9Oi^Np5;$FoHpMI32kMU; zpSySO+Oy~PU%h(t_VtrjFP{R2efa&uCup$_TCe;3H^Bb-=!fj*kKYz@`EEYCom-PD zj}=PdnEV(j*9HKFWmyUANV+IW9G#mqvVZR7@4qi*d@SykE&e%u^}0SG0mET(bqcyt zMpY=Nay3b+Aj@PFg^a8aGc@w->dF(BetP#A!qvym5D7m*^S{^wUj5;VSL^po4Qv@X z{nO3=%sKq)p8wyplz-I;KzBka_WQec&tJWG@aE-P`dynFKY?fXyfpTBx^?Z&y=w|~5K^UT>(dryBq ze)-(&txNmvUp@Hn+Ba9u?ECiM#5ePk2j?Jw0R`K+YoxPh&=eKJK*@Qd$cm z3%e>CR=3BM#IX!yj5Hi6hLH^{L7bT#9SP{gvYVMUGsC3iMJl)^FKNwjjtup|B7CtCa3-FnU{Mv+FnYLCnETQI=Vks*4na;X;Vyub?%)}~vjK== z{aldlo;Yt0yni4YjnfDyCM`EMMxK$b%F0w{Wh>G%rKu_W#Aso1j5IY#oD|QpnQ0as z)hJ_Gc`FoZx{9uVLZ91?~nGsIGv!GI>?kwlc*sNzfbz`!I@ zc6PF*r8H%rHe;|Vd7vVGuqv@R*IXJa&M~p_%$(X}byb4AJ;&IQXKKmPl-fD@MtY%% z-k2n5OILPgYuA+72lGrFY09n)^{Qgqj^^y`Et$g=aZM?j+F0pezO^Y%7EK9?B?jj3 zG39bF6 zYgy1k6*ZoRN)TWHz>*}SR3!wkSRpP(fK8UuQk9GXi@3ldEw(Bu;&cU8MWJ1tZI>ReQI=^@rWxh&N`9P@Z-*d%>ag7`EL5$vGL;vMh?u6jErtwyLElnnw3pM{ly(^ z>2)=c`8lFk8^fe#niO<{Toi3it*a^T8z^k+O|1j~Yszfs%&coqu4u|`>S|cIu5EZ- zO-ENjRdrr*epy9feO+05OU+1>z8H@O<7YcC@uoaN6JjR^7`DF z10y%@Tz>N8-t%7`zJBre&8sJGq4e=zUcZEr8s2|+10c1)j6W8R#acZ21j<)j%C4Yc_L2`L*+$L*j5rdiomuKI8jXS-B%KAvgKrN7#lx%`Rds#*DqYVdExra z6BjNWJNx78@gM9tdG@Tlxo=M<7nfrNB8Ey!ktirqHBG6dDdbGGBEPow;ft40%KQII zq&_S(;l1(j(W-5GM)!U5{I}mf{sCQi{PM-+TQ`3D^bty!{7+f@e*x@&1pwsRw=ZA4 zeEjy+BZy(I9{%?7(Qhvw{sv7TjQ!*1=Pw?-eSiP$`+M(xyY=GL)u+#IJb(S@-G{e- z{PF(JPrtr-4*uC*ynTJ=!JVIOT)cVv@`DFgetCTL$%9Kjoj-o;@XW#ai39VKz^Eq1 z$9L`=Y3u0KnXF7MurHZJs(=_K5^+V`$XILt%7L+ok(t@?>6ty-wyo~&sjF=)$S+AK ztBxO^XuolF^p{76etq)IvnNNMJUDvq)}bpGW)B_Qo{?n_L;L$7ynNv9-Y|FfP){FB zIE)GY^NR3N0g_K8$tg5FNf<}dWl5|BrtFsNoVHv|s+?}XU}fP5Q3TDv6~$W>Y02`m zWJyvyFE)y6GmC6`mH?(eS-(z1lrZFWlP1wlQiyzTNQWRlhd{sOL4gjzL9XFp?yz9s zKz``JK%BoX%F_dYBxD6ZsY{5XbBL33sIyb313=gEAcv&^4l9D){0OiJHWAGy;Dkgt z6A^&(_eOeoqrClbLB6;kZ9Poya50e@siIlU zv?v=dF;SY9s?5rgWo9X}GUXXb(zJM4YLYTNS(+R#Pl;EgL~GLQ*1QC5S|m-wLvpAv zDj^V$@InMR1UfJCTj~<*=#TIY#)f!>2e90)cDUR+b zu~(<53Zi(~dPbp@Ul}j2PE@pJ>D#jP^(pcqD;of;#KLM!lGeowyK{7FN^RYl8W2qh zlEFgj&eohQjp?h(V(SxS4GEHgJWI7zU?up)5d*W>$O;9u-6md@Y1~{Izq2Z3q%LEk zId{6Fc(%73I#3M0ai6aBb2jV zKiOV4-BAzblM6W%W34sgZMA@5QwtXX;ifvk_1dY98VGEVgh5&FQ$0;nJrKa4gDWAc z>zM0bDEaCI(K*}OIoH!U)7duN(KNllswp5_?cg~ZC!u-MWLv{%WA$i5)$Y0qfGz+q zK(9Rw3jnJt1q=fK`l$grL#L{7LjEDjiy-3H(Qc}`n#279T&?`=eixc6KZ4>VUzZ5uv)XzbLXJrKYa+8zQGyLfc$;;{*+(Z=za zb4SPLCN`{F*SmVdsx{j-4{qJuwQ)n!>fyrn*5s=4$lNShTntmMqN+r6jf7{`=u#3R zvvSNCxz_ArYi_BnxWZCcs?E%^6qMvPwN!OwaWGq7{} z#`(RIdk;*m*)$lP8buS*Bqmv6ZsPi_E6@CR?Eb?Wzx?{}*)I=Y0)RbVEPnMAQD|a6(?8*J{?&Ih0 z&s{iw?b>3NKeWvF>F<=2zj7BB{yO-L7Gw^Je?2Dv*t^%SUq1sBe*+i>VeIbP7eBv! ze(Uuw*Iqrn@{bF5e!lzS`NMZ_?!JC=;lac2uiiR)=i$%K-@N|x=bKM|{POPo<2Ub~ zzWwd?)92@J-Mw(@&b2%D?>v0?@X`HiS1%qra^S%H)WP|w`Ps>_(Xs7YcQ-Y4YV;Ns zS0t1wB?`4jDi=tl5{=XtYpiOn+^~K1$n=h#W1CvLs#3EoF^Q_!BvoZ?+{AeO=^y%j zx-fk6+J@UVw%okF{7P`SiRH{iz zB~344m^gwMjxt54Ns}6~mHI5XAeu_jz)|8bgaCn+F{nC~ASPClkifUw$p$q=ui{&^ z46P6=q%t%TyoBe8j#wV#w>-did0@b@z>sCYIf8?n!vZ~#!Tz|A0DMp&J}8_T9!v`K z2zPZ3f&k_e;EmL@mfR9t2*E4P-H zT1s+sc`2sCoXFyQeO{KiFjt?OYRpZFDo&9k=x`DyoXH3yQT#Dz*HHf@z7Ah`f3@6i zsdJ!{Q;_=#KM$us05F&*4CMtwct^m!;BapQ!XpAOGaP7H7?J3SK=|PBaE=f!lymha zh0R75@rZne#Gn@IRZ5elxHzl6G;OdxyT2-FWp&D`s-*5BTScNY-@?h%(#!0U@@NT= zu!dB1W2&kwS^xl6XknGxcnyh?`UKHHp?OttWP7TjFGmj~Y<+e7?)KbG^{GQ8_6i#Z zK&~&xP-7F?NI{8|&};xO8M!N3x+XhvQ%U@;nzWJn%<<;D>8{eb{_6RGy8S~<3(D2D zf2e)m02KQN+7AqL?C)!z?`;Lpn(Tlw{Q#d9$8m^NHIr?%ldTK$)|v_MBk&LatgU{c z6-o#M564@oC);YK+G_#EAf9zLjCa&Q#amsCV*r8B!IiDha$;}$G*r3VJ>Am@ZGd&P z&UChbm~5*bYp$7S0k2Xw)6od=tgU{$rFOKTa(8Xn?%MLv#;TD=ucT$#V6I)8gz@rLrup}e>?Maf;s_PR)IomE$D(bSqXRYqmCNm*@C=F2&0d_tar zUS*e5$4N>|oD2yeT|`LaATz~;93>?~PRi9XQl!K<0WLvAuyIlGVnU*nY-OWuTnu1X zqLh-RWM!ziIR=Pf1y*@UtQG*wCZc4RBtXJ4EV3NC3Oepf$pZk3k@L)4s)m8rK*U1n zI0Q8lCm|sKy~G4)(@V!D%g7i2Fn}=-0&E08nh|Qc!8QmOIssk7qskdXfG#LVXfWsD z5#;ayfH58p&PIm|NN8w{k%|{nFcf$o1s2FehcN-e2oVMW$t0uzYFb3}RHHI4##EFL zxw5fx>tM&`!H$V7!{6^8IrGitUb5DnpI%SEwdGt z8*&TfX<6Ku1cucniH%Lj%dMy`YiOwMXldx{Y#Zw9TD@{$)5dk1w{BdyZXmNTnJFQW zdH9;P`a|D;d-&TUKm2gy=FJOFp4@%*>*HUa+<)=(-s|U&-oJkO>e>Cr_pd#9eB=4^ zd%rxp|L()9Pk(-X@!PA*KVLm|_S><2=?ONMa-`|i~m1#)QMrP=vvh~6!7B`Z{ zvXB@iA{~U8%8caGObUw8KsVc{P;sr9tB>LvtZcQOuGBC!Mz+buFhtSRCYe1^VNYB; zGX3WBA9sI!mXu$F;t0u5TUw>qXfCd6xO(UALURA&@sfYh3&e}xKJ1#FIeGaql$`M)yzdnID}yw(rQ96K8JRxccbnofmJO zy#Mg}(;q;>etG})-m_nCJ$`!q;nOqM?|ggV+KCG{FI>BO?e@KM=g%KJcwlyBa(-@V zdS-gh*u>VYd#Y=jWokWJAQ8w^Vuea9R|u3+kzT?x@Z>gGT(-5YtFmLbrMNCTImeuk zrpw4Nx3{P7*pfRvnt5<8>*&GUL;G|0?JYUDziDc`rM^CeBg6$GeFD*bVfauCgUD2> zg;DW*dm_zjCur&FHlMCQJF@G>kja z-yzh?85Zb+MF&$z{&<8hDj*Q!8;J7`BKd~U14Ec0A#^~mpkT6J0Ny)*=9P=~U^bRzGe}dC^m+OA@~Y(e zri|8(td6ei&i2IG3S(Z9GSz0tO^dE7iL0%MtF5$^7uqZG%=w9Qy$B^>AX%JnGTk4A zaSZhN!gKjQJ-+${&sO$6$O>C@&<^AB#mW z=-y~#5S0?a=HW$huEDH|iDoFo1Rh;tRPa_%*ScMjLftgco7gk5}8xq9;U{I&1bQJ(tSC(dT zL-MxfjJ1_X{rOP^dRk+mv_IF_6t75PBkZ_9e3%o=aW8*eL| z>Mon+DOA86e-)Ukh|DtU3)aK5J%#7tKcfEL89_J3#k0rdef)>Jjp0ND?q z$ar)07>K5-vBs)B_2m;S)#J^Tqm5-_O=T0!6=O~10G3ms_LKGoGS1-Vz#bbI4?Qw>1WM04$Qd&7Kp(@aOhL`%(hQ}t+L<*vH2 zoi!zZVY}%#UAP6u&AzzBSI=WHVNq zRpokFsYX()7L}-l*5Fbqr*CDa5lHATtFHHnIB@{$;JX}m5@&a?_hsYY>{ zNt|w$q?sgfO0G@90RT(VNz8njjzv__fqtRIM7WrUkdZLJx5Nav4k}V60ce2$6L3+0 zi_o&kMiEmlWEjOPHJ2h|5CFP>W&saVFi8?B4p54Y2;t!1OiVZn3losh0AnHwRQ^Q_ z^`k}vF;F2)REV68wn`Z~4&Ef7Ch8;&xk*h0DTDQ;JBNFAuI(S&y87ts&eIUUCIG{J z1Q(;S}9didZ&mLSmxp&p*bl>(p?dvwy53a1} z=_~8#ENyQ~178!6JBk4ei+tZ84>ly6ik@QW|doU<|!c9vPXIpV8P_-__mL z-`BNj_27n0>xS2@s%ok<#p$sO1V=%u@26;^aFJC_S;Qpt z`?W}#_Y3)l-<~_&)YE?E%0&P+z@#_7eZ2elmwV5i{dn_I*Xpj;{?>C>&cFZk@$X~6 z7IxA8{PW@S7q=fjfzB0MP_e)C>+kpo9d-5@7~1cj-v9Rd>v!*8ynXY_tLOKBd356+ zN6(+$G`?fU)UF>coPO}~#fv}wc=q|z^G_e&{rTtHKR-Tr{mbb8h^Vh%o@uwfoUpoBV_oGu&TX*f)IXngcU=NA~9&*`GBxow9p( z%7#s8g(XUcFf1JJ9Y*jCCk4YPa2%V#RO!IKIKv!IF-Fr&CXPkJjFj-AWuh1<&(39< zDRcvluE*h3@GyQz2nP)l(2z1dQYJ&n6-b2)rI5mKre@|;1xlE zD*}U-2L!DM40a3+b_xr24hwJz_3=gq1>j&|6buhoMYNeLWqKjK9D^JjgH|{ML4DaB zf>t<(uJFft2I733Lfw6d2)ta%Hs~pGaS#dR81B6y#LEE!VT2zR7f3*b5W`@kz!0KO z7}X~n%9HdBqWJg|JblnEz8H6Jw7Wy7!;*ldU;8in%5T}1J}bWRS+PJYx3B$tzxMI{ z($nL=T^+ylTILk)j1=JnF zf}YOul|v={-D!;#nlwAmuW$xAg35r?m;qSqQXkL%+vV&3a{J;7&m~`b1AqYo^KtMC za19Fg3_}2bdBISAD6Bsk=a0ttqR}u09m8P;k#ONGI#wWJs`V0!)smbfv*@TohEOXO zYsJwC)~d41zJ|Qss^s3%*x~Bbfs*)|L}i|tm7*jSMha@;Rkexg`ebcQg0j>m%F#2k zb&OmCt0G!l70v6&)C>XrN>c)cwI$1%k|Z1I5;oPRtf@%sNH+mvsEZZ%=bCD5;w&*S zmK2i5!!#SYJ+aDFS=P-}$zv_KlO2FwrPF7MG6ot1^VstZQz zi$)uZMw^O98Vh&V7wl;)+S63Lr=@hXrE;vL1_b1DZFOVqHIrQpQ_#K|@Vv$e;8;N5 zfW|e|j5kz`H&y|R0STLGs{`}`fE-<x;8CRpe}}%wAibK9nB^0M?moZnPU}ExIzTG+)Nglk*GY{5&Z;k%38|A`|F{BsS7c z4>yy8l1118J-b-XDK&8OH4Hm599)UzBJ3n%!Xre>87T%K2*9u;oiJ7*un3rT8P_UiDybMPiy)(* zc$iRtC=nhhC7~q*xQv7b`lV(NMR_deIQQ-khRFIBGj8?J|bwZ6qi2szoINsj_Q>e@Zze8$Gk)O+)D-IEH_-fB*8cUvAyG zb7_A6NK&RvprW#sB$@(GQsIbdEJcfD=m~5+5sY*Lkzpn>j5wZ&sEX!DBUv;V2F-;b zx#4&zimW2jbzGWWiB+huDkD)JN!DB0COgk)WvbLvxq__JP?UOx3hJLukSYmM6;>#h z+hgt7SzLpbtdfzH61rN>)To3OQ+{3TnX6YmFKE+;PYY{_fBtdm^0|`6ij0!1uHo)I zvlGD5-u?dh{pUYM=Evi5qE&WjWU9HWxngwh)Z>>g7Sa(R@&SO2A3WI5-~YqqpWgko zGWkCMwvdzv1$5vi#Kk`r6BK{{`0gL~UcNqc^~#xRmtKDQ@E!p3&p)32{{HsMC%0eT zJ$v)i%(2O%XOCTbaQDuyFK<45b@A?#lRw@5=Jb#A-+i}ZW~L2%EH>5`H8vEtw$%3Z zb_}m7s;!dfG;FCvs8)*Aa2?8sGVTBQJK0J^G3*nL@L;{RbhE=OUpw%+8QjAdvU;=_a zCD4c9?@NgAz#v@UFc(;ab6A99NSIS-m`ix52Rz6f<`;+y6BuP;vzn=r2#q?jgzt?A zasVwo%u^@Z=UFT5T8 z>FM}|yVDnLF8}S}@rAel7hZl}xO;u+=DOU^#Tn-6NA^buF-Q>sC8QwuB$SXu)9@Ku zDOSV`Bu4n4{KKdbBsoW(V9RW%tsWXI?rOIeWN}T(a0bB}9pQsU1mLlr@bIrZ9se)q zFaOK!Yv5sD`?@Ui_gdoXv&=WZIRxeogS&;pT|(h*VJMFXv=;&d0>Nfsc&sogHk?kx z3wRujhO1O+tQLhqP2?@BSu1e|3vsK3{@-3VUEv+I} zRuLnsOH?n?E7!!yF|Yu@vUSXoC}DN1s5j5pldEe!Nur$#LX>teG3C0|1*H zY@HkIm;u}xXq)SAndxnw?OQ1AS}3#{>#Q4TtJ&2A9Z<5Xv0_(!*{*sZJq4@tQ-^a? zHkapZuP)qCTeP#fXlG@?&WgP4rP&(`Q?`_(@2<|-U6Zp5L~ZV_x`Lf`1-lwbb~l#q zYAoB+QaRdMHPQ@Bu6(?$dXbC3w}4&ksVf_+s~D{TW9eiw&r8TOOsa@C-h}nOLf9ZqrBFnD%VN@ zx^g7!bUrnaj*TJ05@|>~J|vnH8c7T?kpiOVp>bSfu9{hH5do4}X`wbIES`_H(!(wE z2w-5jdTzRsk*c62$*7SmR3sB==U}W%jE#-6Fi>_LE=57hkCYZitJ3rUU^q1m>NFoM zXU40!b{Q*L&NpyrfJqi1Q^O)?*+ek1;lXTph!7Vc#KQp31lVvMI!s9;DCu}19KXZyC@n>TM?Ke)1|p|!rTy?M=+4PC4I3+pQ4^Ro0w3EGrIU3#i2DP9q6 z*F@Vhi;KDk`#btOOKVG`lOrj79E=ofiq~w|x$5}ogL6lx8+z;Y33`SCe6UFe*S4HJ z|LwW+r`o&QvP*Kx>xx#bZ5tU|d;Gh-7p|PDZ7ZisaWom4tU!`AD1r)!S0V^XI7Ne^ zYSCm3hOEVaiNH+K#Bh{0E=Ns5vm=5De!(RF5K177i6qJ;Sh*UjGU3!_qBfFkvhmGU zx=KZsNC*-sQL3P;bab@_%@<(=Qk+-;CIl&;W0ni;8n#)=G>BP7snDj0%!=<=GjR8p zCs3CBLL0{8XHO*OrNw0@FKWOYR{{$wWK9Uta_2vph~pI6lSDV6NMJr3rDFriS+7Z998fJJVtl$vlZGn*3jZ=>O9j{(l4!mmzrp zcnO>$Cz51joS1?Yb8sRlUZTb;bR><1syDLCGQ0whQ=%z4Dp^G($nbCu9K%E5g!pha zHh_)|s z#TVz{g>v>qI>9Oa!36Im{$Kvn`^zu90l=KT_I6+5<@vRz=NI0-U-$%m?HjxzAjBgg z+&#h{0L%&I=z?%_3U_e~bzUCq*cBhyCE$} z5fepMi6acXTLj7- zj`TpFT*8qqVJJ5k+6#&D!(q`p7M4#Br=o)Juy86C#U%N|!zdhvOd~@Q;C^8~G!`i? zTGvvM*;k#_l5Yb5TUDLhl4&f9mgJb&`4(fnS9${3U}ow&`43`DH!J7XhRUnOFW(#og_O*v~#d((C*Y15}2#sH}T zf)z0mJ`#zVHFgv-Ncm=k$gC7P)G`nn;@VV@m&Aa0*>N~%Ai%&$z!rMe(n6b_b}SgENFWCQNg!L1O<)+unHlH2~3ErC<-)0!dpW zZSNZ^9X@%mJX+McRHPRnJP_uQqHd|uB|$wBwNIw@!I}U<_!VRjbEWj5Hj_VPu!r@e zlYq2ho{GapSS*AOnZYyx7+4Gg5*A31;iM^^GRNauuLrSO5xbM{`gDFDVs*&P4$SGs zy>?T|Wp8s@b9PJCYD}4pDT_H{a}_+Xa&q0y4fC%oEndHH@bt@tp015Mw?269(Y|8` z#x_l+ifyMaocr>dZ{GRjy?jq5*PGmZ^!dAw@BHjHKmTiONe}?(>pw5o@IU#%54xr% zvLi!X(^G5r?s@$A=e4!OzwgfeFU>n@%ZbY$k3Ido4dh$D|N86S{`Jd$`_1ou`Nu!~ z?#sXa_?JJu{NVD2!z1OHXy4ZC;e}lb*Ul`wdUoN)l`V%4X8U_R`LrVwj#k?8BUN8V z!klzF+Jdgmkh?43DggP{6Fyw+Rbg%vbs=&aEVaRMH>!(K*1Xl8Ge^pSjIO&@jtS#m5_yvdLYz`A#RP_p;Nl7O1B5~1k}E;HW<@{ zqWXB)kgzG8Qi+)-H3RU+OBA%B%nP%jm> z$OT-CC(#NaQY2Kf7;x>JttPlxq!SA@Jg%}y?@(y%KzE*fmi3)ySgW36H?L%KK%Ql? zfP1YHLVOJk8*RATA)&DbS!08&ZndPYR=Y24WMf>J5mH!DAqh3hxvdHwtb>($2(_ry z4n5&9tDQ!lr9Ho#6@i+<*Pbuaqgw0~sDsIaPPRmL@ARm8?RJ2+oTrCpSi=ie+ z$xtFp6l0-ys}g4@FcyM}bcDujCY*X1B^Tmi5h3Hq`Tw-$8G&4E_t{k>#^5oudUba( zwSQ*#z`E*=!OVfR9WSiy*iwy*=WNxmZY*QnT8(V#3ao3Z)p3j_%~ijy>?V2ww= zp0;c)xp(&kw{|&pb-TA0oof^1!NKU<=B`7dsTYRQqY=7IFWA~)-_hkAiyEeq)~Tpz zW8AVU?>$saoL*Nsw{_sk?lHiy8waLuzOerK3mdK<+I-{iHo&m!2e-d^VB595o38HN ze0ATJOS?B++Ouidy{6A?Teq}%VqxRxskMElCVEbeSB?)APmNb*M@oANk!@+;p-N_U zu1a>eiT-ww)5Be7MoTA$I!}$1PLFjjtnFV~KXP^>@UXG7n@1Nn4WHXOeqq}r zFtGDmr_OGfs?}d?oLB?|n&>|>*1Is?3rz0Z`oW7cqn9>MoZB$GxNc}^dIaR``jLg{ z!82?77p4XlCI{yyhUO=S=BI||*N)7u8wPBfTibVHqI70`&&hS=z2*2+$UNz{O!{rT zCM~e89*d?+kET&k4CW-{oOVK%MtCu}HLn$=34Wa5r%6df#S17{K6!KAC>x0y`~13; zPV9zToU*2bPTb?Caz=U1r0jC*vL;oV1#dH}!fIh!FAHMaFfQz{Ysb?5sX}Bp>uYn7 zA-z21&}E$ZHjg>uFy`GBaP?}GK??Je2w<3BhuKjXE#sTyLMti;pfbq$Itj;uN*%Zy zkjzQc`pWw#m6uY1LDWJ5q*<3sHUPKWTGXXMyp)<&h;a#r0w7ew22@Hy0s|re7}H95 zMhv1587Agv#2gypndF>^5$Q|22eQ72Vr)}?Vb8k0+5KCu%pbk;%EJAti*MaN|IWP& z-@SM4z55p*-@9=4#^UVkft`Cc9e(-1-2AZ{*U#O*`^w#07a!ce`qrB_-+pxC-M4Rk z@Yc;IZ{J?dtVzIBkk&ZON=55>+`}u*r&8 z?HaE~<#5T(X2faMhKzKQrc=7*XbnlNF+~}Zlp#s#V>&uQTjN%9)M^UZQ(bLuefa(# z|Ms_k{_zk0Xe)8YH_~{Q%Up=4c ztJqU9Ya(n<2Yj8;^iX^IWI50swd8Duv{4(^A^`>LlFFSD#0@E&kjyF-o5fNa1i4Wp zpwTBymaN5*HD~%`#qoTg-DwPItTC-Qgmb{;UMsXj%u-U!fa;ivHBy>`dE}T^jkp1X zj7qmt;r2mJAL0uT;jk_m!9#vs)UAseAe)%0WiXTt4br;R!qqEzYj`-K^??~g2z7hZ z!JsM}Mniri;KYI^)T@zMU^bwankUkuA{t$uG z0NSk0-*C=m@|N-Z4`OOl0ljY3tg&voAze zyqdCG3nu;}N%K_Jy0bU1z2e@~Vcu2oY|NR~rS-?w=H|Cn_7BDP4J60orVgWcTZesX zyJNtwsk+tc;+Bmm`|gh5k%8>|hMvniMghHU9$0(ph3Omnr?2myzHw+1$hG|&ukD|? zx@W^HyVhOVHT}xY_2-|TKDT2%pcg>b;-=C04TC2pDhI0R1Kr6Nz_pw_P)ZyuCtvEx z?8^DK#_fA@!52%Jm#bNjqdnP|D(Pdjaprh`VRjH;ta4_eZ+>cEas9yj`u_Qi{Y#sM zmo|=^+cbWD^Tg6j&7bC{hUUimP7YPh40kV$_ne*Vogc3*O!S^zJFv8NU=e)p)bQfe z(85&z;<|zPslK_1p1HB!xv`#8BRwaFdrpp2=O+6WrUy<=luu2S7dH03IF#R(%R5 z<`)**eDGkR4rk}1+L#0EBB&*af8F2&+eY@U|M zLYtWKCNUwD({ia!CezAb3c{?Y%0Z}HI?Sm-tV)$#uJIshk5c79P^SWM$dy(IHi;mk z0I`TEm%Fvb(Q zU|8+(>ik~JVTDX2Rx9Bq^$DFZL(xecogfTJLLb-C36hEv+9*LrNkg1A#?AJmr!><0 z_@j@XmKT0gE2jUrJknntsTTU%$G42V{qf`4CsY(pLgYR&d!3n z*dFcc?p(L7ck|ZqU3*Snz5dz%^V8q`?W?EfHUH->w`Bx=^T)5g{@q{x`iC#S`qt0? z<3IlGH^2VrZ~osmpL}rc&h^{heSG2my%VoqT72#L2S57pzyINHzxdOimTq4UmGaJ9 z#GVN`Grm}NJU`l&8_j#WocaWz@Q8($R*8)*vkPFSROwYJeJZ7oKms~-#Gp%9$OI5i zn=NB^=j@rjcyX*P)aldv3EHQ&$0)*qX?+wGGP5z>8c|)dtVyDmVlFM}G|25%xx)p- z3-bk0p9k>(4>QP}l+*%Hq*B;Xo|fC9Xjmgzy+%^ElE3mfcD(`!ZvRNSe)S2O|4?7|f32=-kigbtI@DdZWn&7HiiL}_m{4$43MQ&# zq6kAF2LOA@u;r_)6;bjv8oma+WUY**V51Iq8+2YHW>vv7tZ}Fj11dm8d{}6(7(?Np zQYo{VG$WPFzUk_=zV!CqGTq_bQ1nk^>`znmYrB2gibo4({ev-j zUBR|@AUxA)+gLE}s(8mEn)P{e?M%y=;{M^}p8n)Wgf5ySTMM?CjJ4{($~I&oOm8pv z_Ld^YM%x!>`mXF6yK!LcjeS$s_fB5hJ#lT%)V00quI!n7W%uNjT~n8LOkCVCerfyo zg)QS}0aP|kEX<6|P4}Ois?JVy9q!NUEl0pl+nn=k%DA^?Jv;Kg{hg701^;$1YmQiU zWxR(0)r!#r9pQtW(HA-*hr8l0cc%f*j`kH!4%bxd^mygWy52L>y^9+Lm(~xRU0#h` zSU0e+c3^(IZ*HV#Zn%4Xq_Q;LePM0inbFGWq4FZIvB|!r$-%{m;WMLsvqNQo!;|2y z(e9JO&EQ*5xc%ZV}M>&BT~?c zQz$1SX$ec&8I?4F@)H=hKmma9lDH%VoJri z8ey%Z-i{|pD2z)TQl?MIE?ChHJC-vmBZN3fDbh67?j*8(T-)_!R!GbWF^C@GYeXzk$^{H_;0k~)fT*Vo3mP<0 z3+312HWdsQR-+ezIMj0R)(xOl6ECIYDZqe!ftz|rpcb-lser;@mtI9HgnEc=$AkgB zycDxdl;T^5I}dCeI=*LS@z}ms&!4<|?d*eFm+ri}aAEQA=@UB+9-7&*V`|^w9j6zL zUAwaQ`mKxa-hbuE!z)kTxD2izzxC>8k87t^ehwJ+_~vJC-}rmpdi6tKZ?~7;zIFCJ z;Bc=Ty?OcA=O5kwfKvb3!%EvuDawqVUoa08g6axuOT7uR~j*V@C2q!gf(M&koB zNK$1(RZc{1g(S2@pcU|NUTp~qW3n-Z0B0#Ef!?Jwcu<2|P1-TF8Bx+QAYo!H>?>q~ z#cF;_~!MW|M17P^}R2@{Oea={q_%keCgC|zPo*NX5#*L zAO7W=FPGbPEH4RuU2E9!i{JfzXwxQJCS%R#On_lso#6egZG5t{aZAVa#-7dFHyu8{ zbo1V8kKg~``#=2GU;g&zzxxvqtzZ4=uRs0u?>_vOAHMs=zr6pwAAD=y?BSzJvrE^{ zP3_v~X^V|)-*)oqjaT1!|MgG5|Nakt@}pn<#}9w?tJ60wg)13XA?(ZteR+SfCz0%n z2TC3~fg>KF)W#B-8B!Yua!Qmw1dEW`l+}>;JIZNiC2Q|WSlhz(yx&>ygiE20(e|F{ z-j0zDPu5NQwdSzimb4k-HmMm~C2o3-v${#fmC_p2X$K00d4rfIgn9g!+o5z&LR!uu zcr6&K0d8r6TUZFI8EUK(tp?~?$zQR8yYd-kox*A!*s^PS|M8KX2iwx6ZyMNEN~lv}(~t6ysKNz5Lx(JeK*lnyWA@W`z;*q~QijFEi0d$iA$ zj+6f7MPZvorq^+CghyZk4bIo#6561!+7vds(&@&*v$fl}xQeAxu&`Q$B+ex;CJHk# znS@p;EgG1{WICk?hs0_bUn$}!q>x@GB#@P?x>Zg8&1zL-GB#8vT*Y3wf>mEHWYr5e z|J2;@4{KKa-*t7XczoRLaO9Kbgiq_CG!B*01S_m4?l)2qr!nTy`7MZ@l+r3DB5Q)B zEwF^8l(8|GjY`>wgs+AqTBVFcAX2H&BZxtzwrEvmjgnT$^(q;q<`b%xWqQ?#MIdXW z65yK6u&gGX3IJx9jUi%+kehX=ozw@wyNdy{Vj35rvS0|U#Epc-VRF0eKA$}vc5WE# zJiM`g=TL6XaBfF`YP!=mlr;B7^pkC_t=-X$#lTp~3Njcqbo(?t0bR+3R|DGNq+wIZ zwW~KUoi}djv~BHl^m&o3CHKo~^DnK-@9K|jDFwS-c(+}#wavOdX-ud&B^xpt&;#VW z(33oW2WaOuCyT&i=7(KsrXgR}+ zGyO{&`WL2p=O%jRruygB_P;dTexN4>(6yt(zrEc%6*CNaG$TH8Dq`H2u+1c`+cJ*r zY5TUMb$v*`BkkIg^X*8v_O|)=?hoyP`>Cq}z~d%ZkVcxfmP0QTZQepAvt<|0R|+G(F<#70()s)_+A z>*Z~PD2cEm5Hqger7%$lW~J1EHWJEdq&Y1VggI#q zbj*N+DI{alfy7LB-lb33v@Sv#(jy@5TIWODhC&8NXoN+zC`4vNVp2*BaxfMHT-6M0 zdBG4cEI<=ZLTN!|HVks&O7M*UU?8N7uZM*e6%2%oR)}>nzF~P45daL6@(n7P36tXd z7NeZ!*1&Na-kosG^cVI`_n+9eY5wKCmzR#+ys~ip>f+ME%M;_JWYX5&6&joFIez-! z)K~;zgjzm`JJ0zymRaG{}q6=58rs@z5ACw zdi~0053jxZ+PRw-kAC+4-5-7LyPv-Q#*_EneCO>44<6jVdi~a+W3#;zlX|CJpp>dj zq%Rju^%V?p3uKiG^n8wv$<{Ggx|UYFg`s9NshAClrZuuP_2M;LR6scNuwE@tLwuEx zj|l{78EjOm9a>fGR5YNULLM&Qss&7h!$P?{oX=5nxoVzNCqj*o$}C3>a+Fr021G&0 z1(=Yl5aM=PAM&aFUUeu;#%hI#csQkvfF$u?jEKgJxui88ry>r*PwB#1T}(|zL27*z z^)U?|K=1&n_NfuK5_UjxyF%s0wO-Ph^1Zxp`uBgUwMP7N?NF7kfrfqmM?c=UearmC zbHDw|pZ~rsY`N+1pKIs6eDnCpC&_A$zpHF-E9g@xTYJIZ)s-0?FKynQTf4q2h{$sHXKFWP63H9RlG(;^BTBewXD8Q zT;Hl}Wxy@N|{;&2nk9+`D%7k&TB=^ljVeDHm-WNqy1@SrtqS z&}-GRoYgA?Y!0r2%xIKvuH{r7C{dQYCU<~`g-y1TL z01T~iQN4&&Cum*8YpD}(1R6r^_UI#E{#o#MR(u^rb36_VT?iD7(n$?^Ney_wVB|6b zDmQ}v+c1bkf${JWQLBvC3QIAwL1EOaV%I;{@ZU&MoJ^Q0vyvm{Fy(w90r4@YQ7m}2 zwGl|zv+Pz0t)^oRe|tFE6^V3(JZ(N#TgaXZ8WLVI=u*3_h+Qu+;7qlmNe%&XVq++u zPz$tbfd&<8Q3Z`FYPdob1~qEN35O21Yil2p*|bU%!NKIKg#7Pt*w1n~tAv6E;EqZu z(8^Xs)&io0m>4Y3p)g<`n3uU}YusT98!aIt?beYF(%>?>{Z6x4pUp-Gd)s%d>wjTw z&z_OCy<=@N<>+Lary3%A!qkSYz~*vzGUptQn?@4m{;(bZtl}kV7z^o!6Z);)-raqH z^*Ph_qH{ym)MbZuRQ*TSwY@l*-B$KZW^5U~WWcN1nm0`+4IbH=93`Fz8#ZU%K%P#n z?ONQ_dvV9`)jeZZ_l&%VXG3ml*LiEn8 z^ZB%Wch0x35PG3Aa-@nj0^l8Y`U~0TAj090CCxn(gm6 z+23`n+IFm(Khaw_HCUV_5W^*H1SE4qjWL4P8f8>bgJ?O zMO4jq!OehPNxjr1X9QHdoJo~4;YkAqh}G`VcLhwP0B%5$zkqol?JWi^VH!(@RUoPR$)2pR9ItMT_0B;fc!1JNXFvYzd!N2@`_`4c2ln@lO{6-C?r1`1wLt_SQKPiq5v$~#Z2>A` zL|hn;;xdUAkXALL1#4|W7!7jpM%P>~ZUR%@c%{RZ42m>xZRG|NvTG|hd@djvArb2& z5G|J(l~MyL)uS>!3ekv22Me__nH~acSDV#@Rf}7+n2CUNN-$|=2$@a%mKG^bq9--M zkUpNF5?O65O(jxvDorNhnrMiO`;93d9k&ocoCsl>%aeE z_lrlPrS4#*Cs66J=GvScoxV~z)<0O9*;?JUvwQ2Ff!&A4_8;4DYqPIDT)Ow} z+1I~&{`GgSzV+nx`(NDq=$~Kv@cZ8i6hq#E#}uW=kj|KLR%fOLx6ZiR(zaAAQfd!% zWlc$!%BNO36&jyP>p^uMMCVfxK1>@@8B&xfWiX|T#+1d{=5?3izTSLbpfk|lQj#9_?+WhFatJb&-*cHk^SgQqf&#+dk;4_*O3Ly=qKOTwA54$6-ahs&<)x4%!|VJT!H zT;3#NG;&tyC|NvavN#C^DTgT(vuNcsF43x6q>^V^TApDwvsDtpM+G{3+5SX!AQkUT zCkBhD;Yzf(7^)OK?QOPf68F2Lx`NZ~-jSpQ z&h=zIKq~J?%%kYxFwN+dkV@I6YiEGh9ABSe_f~IW<^439dsu zr~12(R@z@G<-j9>*qs_Ko*plq80&m-Fb4p3dR^tnU~W2Mp7hbfR${`bA9v9GMxsO_ zof>(YTGF9~^Mo|55~Wp=Fw6?cS#4wuU@4WzEouxaxj{L{Eo!lG*LWn&UTHI+T0qVU zDp?_f6;QPJmy0|40wkDW42I~cR~OX?gE%j5K_eQG58;K>k_4&9TZo)Z6VyR%F0vRj z75s+&tiLDe%{Xd{hhZ865(cg*yAE8<3cg#T@Mw^yjSx+8kpU7=G9Dph0*2`%TwK7c zd6-f{!@^p#C`t|DB~?}o0*VG0=Be!l!=5fG0^redkqO8du6X)Mm=TrYvRaLzMlPTf z;HE?e@ql&MaX4ntb%g9A#n`5?(&613mrfnNxOD8?(y_(GmrtD7H?wIdopty2r}yoi zI6JrR(XE9K?w$Sc-qMr%AZMT4KmYOTmp*y$%BK&ne){G$kWU_7`}oakPaa);^7geS zk6-Zh)HBKuz`HMmEBCWP;IhPHD#Z zDhVGIa})xmg3pq3nQ|@*;js`t6BV#jVxC$i*2?*~l#fZch=`{Y3Q(~~C6f^f5ztaZ zBmiH51za(=RnWSI*UCbbnm|AwPZ*NGx3Z>8+>(tL(g97xj(RE7qrrT&6+%3O5b7MWyHiyU>RNT1%{9^0^Q*Z=sRzxsQ#sO65sUoDry1K;|u z`?_lDhyV7g-)!8yKVIpNR0m`IBjMh`aNlsSXD~i6+O=WZ=)M=H4$V#+oZWbAanIT7 z$FDy+efQlncfY%M|Gje$-hcI-PXWE|KlzutAAIpGZ%5FPbr|Cm86oL}gABQ-kX_}b zm3EyoAGBp$VxyE#vH2uZLb0W~Rw3TRRW&iOMxLId4iKi4-J0_{IwJmRTd=-i0YOB_S`DL zidEbiz@BSe@$J@?YecOg4WhD{=&(=gw@Zy!i-NtHzh(t@&9j__21vrw(=}GvykeVA z>I}l}u*?yV+B{Ob2X=Z;k5?1&Q4y~x8FUqrp3aOp8IT#&Or@Z;*360BAYwEKTI#q> z&oSzsZK+?yY35)`#Adgr+TzvWSnsgAqpXi5A&W^ss%0jP%Ar&0Q85OwV0JK_9B9joR+A&Wv4Lu8WH>c4>aX@tsU+X5Z$#xSDkY0h3G_O--H5u(N*k@T z>f|Q1VmWdHrm_(lC#iK&wAY|C;&lS%3NCw%RKh?MOcm0K!AupzQOUWuT%ZF}JeAU> zLmedMpj1vh?xgfS+UTQbk4|Gn^)6Ct(a{#Y)n>@1Lw&`>wvo=QJ*gdo*-hQisk~=2 zW$%yBQ+d~>QfNG5A6(Y2p_s8JpsUo-r2zoz32LXZmR-GpZ58+0485((H5gF!x{>W= z?+YV|{loFy{n1{Z#>!tY6{q)VBUEy7w;Vo(R#;9$5$UN!Qjk#6*cDRG)CsZvd z4JSj1Qd(hJD=ZkGUN>Gdun094()QS(vO(Bml=hjSwLZ-{zjne+%!Ezb6V6>(?}7Hv z;bQzqSK`G|5|9k!ST#G_*LJ)wH#<M>}60%pK}W9UJSIo9^CT@J$Bj zaThsa(~Q~40VCc`VnvOjpoZFUxSfCt8YoA|@)|ga38PAG8>uLn)d^G(M|dfYMopyD>s!G#}1PS{Q zfk)1Z>EL#!Hfh3qxVYU%xB2L-o2n+TxipYfFDZGa>=*1sJ9kv2|jONg)Pi1}Np$qO~@@np*8fZ52@k2Mrn!ySlc_2#{9W z)GQYP12ZDEazrC0BNalTcBq4ZkaB3HsJ1^$DFzclAEnCsEd7Px%xLl8&gr>h`xj3i zzIgWdrSm5)oIQ5xoqzw%nGauE`0(!1M|aOZ zxp(Qw>z6@3dT{x}2Nyql;g#Dr7azZQ{TDy` z=b!%c`=`zvwR!E1aHy?+usXTEI=-$vG?FcKwpH8BA)81mtG!^H%TzI$Dh3E!&1^*& zjk3mOxEZ8YW-M)Jk~Imm61-M3MTj*@o>Ib=^MQeJcN#xRN+!$K7cN)#x0)`al=X{EVs*G8w>+Y|Ky<)n`h5%pS!g8-1Wm(?;pGQ*6i(f&fIUfCdfj)OHwwJjW^evSCq zRg!1dC>vyUluo)W`LL}s>8jL-miDLq@MPjeXme}r*I{gZ72=PTwe^?y|YeEriD5ME`wIP=-hO>sbve8Flr%mL>_GA(z8OnLcn*Y9XO#x7m00 z1~!+R8#}Ccdta7GMiilU4J&dKT}c>|Qw%d(`Tt0rNpKDVmZiBzo8BF!)9g;kSm#D$GH zu@O7A&PQzwnKnf%0Au^xLQetqawQF}FO^fry0gc6GROP!M|<)wRx$@WV&DOXI%6-i zM{4(0v&UEI~ z)~eUgW(>j60WR^jGM49C65~M zq*b4=>f;tYkg!@SdJ6>xMuq4pkN~kM#80)Vp1tm%ecTpYTXfKT)kWf*ap4@04(D+_7_5%#=DN}**Jgvz~!^Y zuV0?Kd1e0gwfVa@7Vq6S^ZJds*RRdqJb&o^D|{a zzI|`$joWAL-8gmU)zeF74sPE%IyBT#EGN_Lp<+*_cerD8qWk$>)7y8?$i+=etu}(+I+QreDi3kk`ie}45YP5(bS}DVxTQdRVxE!Hp^R@0KMcbjZjkq z)YvRpz!HBD0{v&HAN$6}UL(q70o+f(*@z>qZS!a7Zm!2N2}qf)w1 z#Eqg}6!R$6UIcY36%M)FE(azfGDwA#SVT#{q(eeO3OlNFs&Efwj5saHuq~Skca?{? zY`=W(^`HFux66Bt|9)!a{|4A{uimG}Z+`E8{N&utyDu)B-+A=p)b1C?cJ3e9zISxj z!Rf;%wjN*DHGgUE*{g>x-+cM{{bM)YIDO~w(*5^ddGpD&w?Df6_`}!Ud-CazfAVj? z{r$HB<$xpWbYxuatlO3K#j0(|{*G`t?QM^x2HO3_kkSU`;2f@s0a1L!BoGr!hPqV4?r_-_tH3ye-a0!Rt zw}43s+}H@MsTZw!`T^K;+-INVJolZ}=f2at;-8wqVAZp&tLxYeEfQ98aRI&R_>6i1vq8vi6te3$&8u72G;kVO5T6N)Tjj8fHW|VxN2bG(?J&lY3bRo_ zpb}guKqd78#ws40Ay+^o0Vb{8o)?m9g)ERPRgP7gOR6_Ma0Si&9T_IDVU`)56K|fnZ zaCErPq!-yuunW8vCmwXFd?vX~Bi18Q8k6ZUg{dZago9K&bV99SrGN$8l&ir721;Tz zicFNmOv=opSWieOwZcf?b_#@W=qZnh_L@vVlR0QK2Mx5BChVkAuQIqy?vOo`^$%B5 zTZVE6#tYl4vB|7+IAQGz(*Rme=>;;JFav-A346MD_!MBnF~dx|b60P0UBNt4bgVB} zhr+}}g5Fm0>>G~l9gT163AWo59%=nljM`HUD1Bliy)fN5 zH&r+>3f}AEhpM4{Mel(w-`)=Qu6Fm4aG=w`de~()>=6Uol9A9LTtb7`CCqR&2th zo$`<~A=Cc0Ab`_7Z_4sCDD(HD1?f~K?;kLb~`p|)}uF=_$2yjsMCE21VH zpb@Yuz$XJNB>1hAgsTTCCS(DanGq?tt9ES4vWh(|Qv_7=k{Hl0aM#oAUw~fDT9+vi zE@o4;0x7Oq$~IwAI}QUnQjowyVo?j3^_cn#;SIyZecPrMW)I%DJa_Br!kw!N_ivni z7zkroQkW8zu?i3rljRC)#N&;(NcP(BuF%caUa<|bah5lI@@w>hc??LvR!GHKHQ~0~Vlj6r<4I?o@suqR_viAl zj?QSl5CSvZOe$LGa&_b^ZBb*|MJJ3@Os5UwxE~_|gvv`OoS4Fi!ZwB63d_xq)Fcxd zqykdJ)d;!7a?}F8S|}idG80Vrb?Xmqo7}zCoQRs!X-`L2u+)-SIHdUNs4TNm%YbMu`K-u&c?AN}H&|MBIQ;D>+9-|ly20J{9% zLL^*D6-Fz$!D6T*;YtM(l}xHTX^&e-uTJA4bS|8*AqWj2Cb`*KBk4NVm85KB8@EhnpUWR3ANbc zuI|Y`qu10VXl~>;HV7Km2$}_I7Q{^F z>qPZ+g4L^dE1zk7_B)If|JbtP+l-aZFjqgvs$0ok(kvBPFs~0mC4-Pv-VPZm-ho!@OR^y_9HWj23504zi%i@ML9oF{`9tP#lnw*{y;um;C}uZ{ zSm0Mb%cz|L`3#d?&lk#7SR@+o`W+Ca;3!~$S|z8!T#44(sX)XUPr2MaqsgJijd+uY z`3$qZS;hs=0RfOO6R1(8wc~n^PVdp9Mgln8w;R^{V}1R1T9_@cN@@4$V@V5@QHdn%K3LCX99bh45*IUExK*E>HhsOGH zyQjMLkGHRFbC0GhLvd4Yh+5y_0|6N8j~V)-hViuhDZsitc-2D;1gX)OaZ{)3`KoU+ zLvQMItWBFnqtsN&0LFX!!~2F}o65eNNfv;cCd1^8PG4FpQmy_!vN(4`%(%bgKhzVN z9m|}U$R6&C9PA11FZ=eFd@mGz`#U^4bM~2(aWbkK59=l)`mu<9z^^Sk(25i5cB*r< zBth`vYEBqohEZk!X$>N*h?<+Bq*)qDYJ_35H6U+}W34$IuR!q&IzgvSP}U23E%1;X z8#Ez9W^~MsPy6)SYJ0_e`wKykL&Ye_3!RZ#iAO08sCKBc2DU@R*#7qL{x*;Z5VBg` zM_+EXuMIHl<9L5N(6yu8IUr|1(M}I{%nlX~RT3bxBZbrB?Rz@{>jL^IFSXWZ zobVWj9aNQ8RVcKZ#<~q?g+{7Ib&mzW2`?M4qFz~~ksh07&~50olS3{8VAX(w9`PE6 zebk^wH|)`m`z@0}`*^@Q>Nkyt&BFnz&!HZ25@n;josyKy@}fmvFw4_=sLQUYIH?q+ zXfq>acWw7x$*n6o$qu`&-KJ}IkR2{P09bF#SqhkQ4s9m@Uc}ZLcNPPdoXe1~=|eOT zF%bbhW9S#67$Qf$R!K)hTU1OV(Q(!5%g4TWh~ zYy}b~hS$OT*;u|;5KfHb6(d`Rw-#q*F?WG4dPMx3IetvG-%}WOtPi}wd;JQ6KhG#Z( zPfT`pRe}y5h7fE8!8dt~&Pc%Q_Zt0voyUpVXvm05^{C3C1#>%B(wnVzq`OPe&aPy6 zpnG!T=$7rHGaJ*ToX(+!b$p?U#X*@YjK!#xY_u?x3^u}Tg}DyqRW9PHm2#7T3`F+BUDJ8_r~mxgcRxCN>;4O;m$tn$yKipk8RWU>cGeD3mI3r7#kbOuV3G{d(Y5;-CdhUgS|0hS}(VWcqF40 zZ)is9nTj=R1z=dS0OM+%WVDzBZ-p#bYaM&_O4h16exrbZ5t{+A+YO0~r>oE3J?QT2 zG$%u_32%}y>m-e)8K0$A0?&Zl9=SINc>|Egr|fJW#>1L#I0=bXb} zW-vMD$efczQlcnQl0^k7P=QpkB~h~EB*~W7EAOt$-d%g`_3nC|vx48njMw%4$2!&3 z(*!Vs)L(!1o_h}SPB!CqX4f4o#%>;GCzp8#vulr#qqc*qM_gpkbD?&_5upwi!y2|+ z!jK3$`K(SJgCSyqo!c0_pJlK<&0v1Ehq1G_TZ^de4wKfP;wywawFJUUc)&$PT*<7j zPzop0UT=_qEf7~F+)77;Ah#I+3{XVpKoFM^1Z3BXU_}R?cSkQ{CyTX%&D$lC3e8q6 zO{+Z=&tl=(Y%({c^!T;@AQBE6BSAFkw?w_hkXz@)l$b$gh19q~Pol8PBr|J!)e^p0 z2l+i}9|ZuW3)1=!ZH)RXF`qf;LIVU51J^N5w4L{1#{z^WMuv-lKrv|aSYgtN#RGIY zlC5Uyjm+9?^ZefNBNK(0JT;YZj;9>6d1|viwp8&?WnAra#5gvSb&n@-urn65O(z}8 zMS6a#h!nupQqi$eCTEhi@rZe;-C)IiY$SSoJh9#k6bWU@z*|h(_m=4rrm`_V-*l)B z7U{D?v3n=e0AOcF0J;Fol3;th;XhWUj~1x|IoCoQosOC&!=|x-aUzILhONVX<46GD z)6(?ltAx5>m8FeBU|>muAdAWhRz=CK%9tf_9Y2K#5_)b-+na!yMT-~!tRIsPI+VjM z?Ko+erOh)Q6Ogd=D6tyG52QUu^TFe#(5Y(lR4vwCF4iVld(046*S&*<^Mkp&nrZOk z-Ocnpt^CE|!u?~VD-*RVsd~d^AUVLgiQynO{b>csqS;B z8zfYB>#BsVNuqs}xrl31s3dKbXRN9Ua4*U-6n2lr>4A{58Fsc})JW3Xij#dI2e7e$ zsB0ibmHm#A-&qY)r63>~mT;QFc0-25@;)c5@Z`H5T4KXl{qyN>J~T$sun z+8pew`rV`!wM%toA%yT0Fhi*A6hI7}Qx_~`BISlJSEe%sEEYxl9>h;sqCO%Sp|kOD zIa?eVERT)lTH~d$xz^I&vDLkeiLpR7Y$gqWUJ`^SH1PNcKok#z&?pq?1v*^Yoa#S* z;q2_bWi((BnAsvD!%nM9{n6TBrrMwFAMJ0=Ooj3V#6!z%7+WQ0$aox$TyC|PynbJ? zk{g+*&n*qFY|QLEym0X3+Ua{{kDQISCLNiQD_hErPBj;nL1aezsY=+AB9N%X7&aI~ zdP6|3^XXu(UhRhD4yD|oQUK+`m69!JiOpi*i&Be3W|6AwN-YK{Eh;Uh#)GIQjV0>- zf%*Qvxv@yI=I^V8TCMc-WO;GAZ*^w+(CUfH=RWwC@3#}^Kl*rUN^HA&{a;rlZ>?zl zxILV>U1ekI_*PlPzx?{ckAD2iU;X31{rGDi{KKp7zI*e_U%CG38{hcuKmGDgA8uEc z`cu0Zbi3=yr|faRHRi*TQAZ{YuG1mfsdAaMZj-@-N-eTpRVPEbld10IBVwh)VDQ=y z+G6%&?tIAK$WgVttCY2+L;4`0@nP;-zP7nOa`N=jy%*>1I-Z&=8e@=zUPf_Ud?%-O7ni$-&)dc4?BKGtmKd}5h`1b$N>92q7{-TTwo270b|ZmGbxPoryScpprCnlCmsreDD0l|F05uA2 zDBEJ?6Jp!SkRj+ zrH4mqN7l#Bt_^Rtk~0NrJcUoDoq%4e^$@@q*dLDC#^TsyiWmr(27~7QtwtU*DPlT_ z?e7n+RmqjIYc_+81dXF%(^ApBQg)vhO8|ha)aV>0OCsFG6t-UUlpGqTV0X(6o#+c+ z7)xImP2M#WJ=q^T*$kcR4;`v_4wk5cd2%CepY`d+Xzh?k+vieMY|^4xoHq$Gh@c3d zWfDbo+_)Y%7B{G3`lUTFh?6mk+c0Y%O09I&eXz*OITJ(qkHT>Of4o4U;4SegrW>A;Y9`h%%4T_Q`+~xYt60Ud{yO;-2}0 zXF20r&xiMwVuz}U!cHw~_O%B-}G0`&7s_>_uw0qTj7)l2DD% zmN9LW(AQmtmLDxU)H$0nVU)&=@}O3nFsVvTQ`7GN46FO`M!?aElB01?E9&l#IBR~a z=&=^O_MFFB^x=&tS@JvLgfZ?g#|bpyw8jYwz*5p_^_d_SBzJ1t6*}5ufB;31d>ddc zof6kb0ocOr{+1fY*8JJ_a3VMovK!mE_>i2?$S{={+|j95x)8O`rjIyH0AP(I-OTyN z8kyCZq0@&~AG>nyrRxvB@$%Dey!7DFL#exZM1 zxqoqeWbM%S-Xp`S8`=H_8S@wjNMhiL^jyA<%Z1r&EsF+0o5o!Txcw((fu%=}I*|*i5$S?wsEg zwU}e5sXgOm(EDJBg48aJ(xC<~x6H1RSQS$63R)B*vrO7JtSX^zl01<$T zAA=9G#V9=Pu}4X3)NPKqj1jjn;xR`_bJ$^tld;i(u@fg3FJ9WXd~tC9tR-QVIC_P) zPM&cWN4JZi-PJAMAwl>GRLaq^#YVZ#*-oty>t!-TA=XJ)DlQKe%55+>8)il)8cUm{ znI$q;kXj5KV#W^kt~*$}?&Nhc)MAlI$Jc8a3Tc;A*dY>hhydG!og!h6M97fvnNt29 zF5~u|-FGlMfOU0n8M`@M9bAC1o-RNyUf1o+Jv&8QxzlNir!AR+IhEChLP{^C1!wJu zAJEHHh`DoNPd;iO?cFlLogC&4KCeR}>XwQba&fne&ye$2DiI%6%gjLMA%=gkpqC}%fWM(jCKsc25Nue-f;0@JjIEUDYzR)a%eSMX)#k$; z0V0&7bA?bU?Q>C91uR#ZG>F>*JBoAC4KLF~Bf$G=#=Nwj>~#U*~nG-DZ{3 z2(C-SsG0>ygjRz!9itj$SFLExB`m23kxLL+!1AEG5cZV9WIjNoNg__z0#+k}F<-NpKfXD6VxhS)5T7Y}Mv~Zc)_r(13jo$0KTHw8x@Pkv09ez9 z^!ttdJ^(7~M9jWaBsUvAfUAud;nLsTsmMp6BIH@8)to%#ejW1;@lS}_a{6@a)A?-$jNH-WG&oY_*{*gZ6vm4 z$kG>DxeLSj3&Q|ig?mTx_Y7w)jTi5qDBnL(y)xZ*|8)Q5>7o0lhAvMJJ~%gU??mPN zXyM#&?%s*Qy%WVVgX#Tw-$Dpm2)6ZW&Sz~Ci!x4;$TZl}rYWywi~;~Njd`u(K5QmT zO^3+okb5N)T1o|0vca`{aI+LVP>CL{#*WltM;oa_wbbE8_E@uUsFprhPwuZo_Z1^+ z*}z=PJsHNwL-wW%D%<3JE~w#xYXsbI8EQ^_AEj@43{?WoS(I6u7SPMD5qi~vm{FO- z4J8sSl4#wFHE3)&LXM~X!%1&n*jWz|IS-m}=(Ci$8g!KXc*%zY8!P&pMW3r2Aj<)F zf|7%W1yHX-%YG*TQq?lR^PQNy# z2-~VS+B;lL&5hLeFHJmf{`m9P9sv6F)+^V)_N8auz4h$7uRr~@ zR~~)&`o+f{JaOsV-hCTmt&v=LAQ{SfEffMfR4TgyavNRwxVK#OmHWbtfpEPUt~El{ zdZ^wA*6QJUBi$M)P0Rq%8CuyGTHOceHFxyP?4cvM!G<&BY$vNg9KM##Rx>#oHc!hI z>9~+Zo+yM*-gWBbS8v_=%GdUvItW{3WHRUwN=FFk+*Xh9BwCZXiN*BDWUSfeEhdPR2MM5hzaI8MK)GNq0(%T9LMyiedTA9HEVXIGs8VcE z3QRJdUMxhUYKI1PLrP4h!sM_+t+6R2I-wkqQ8A}K;|ydSu^JsHQ)l*lloVcQx6Uv+h3>h!7UQ~SdMQ8fk5iCsL?9;WtArg~?$VrRE}7awA=l)XFnyP0Z%H9*4z z#*p!O8nM)1_MPm`9n8+pGj@HhcPH>CfnLLd zG(9p&w^Rb?)wU3cc#lZXB^C6@gbcZ`L&*7j_nyym?%K^|^az;%UAs7)0Aro}UI3My zoX(vBw%mbRlR0a;U{0iwcuXG+8lpj4Cgv=}9N7Sw4|xkwm_V5-NxO!xOx&%Mv)U!2 z)FPv*ow2GBaMco)LdsMsm`Wu>CSyn>-9r8zK6f{VwTIoii`BV@*Ts@?8B#t6R*Nkt z--xndJx2#~4KN#433LiIszsd^%l|t^kk1Y6HDVIHDv-wR#)aFi-M%oIwEm%B2B}3tKcyO?EcxCj&eBXLs zbgDqM;x-^&`&-GCS`d(|6|syb9CJBpG=>cZ(V?(yFkl{!*e7DP)rx1W>RGK&D`j#b zZXNI%0M!qlwRIBlAsjnh8!oZq_W zx2=VUeK871*#0zqv=}^I37@Ejj#olpd!`<}t1ofSVD`ST(#47LrOE2W@%Hx8Waa*e z%9ZKHgLC~4&bJ#GsEd)eTmJScPUED`Rxlq z$E?pjNt>sh86?v3AuSS@(L0JOH6UK@gt+*`%LFZV~JCgJc z#>qy+Sq@?)@R&53r>y{B1uvfUU|9+)c!{E)$kTYvLuA}oY-=rZ!es+w+Fl0+7zShu z?0ZpN+-(nF#&)uXNlPK^dWqW!_oPvaK~l3qVAse9ow8l61BGd;F73q!^5JGK)RzrS zHcK-D<>TvfPdsq`rKcbM^2<+s{k3P`yY<}rZ#?(yH=cR-&Fin-eCWc-<*DJ^*l=!S zv>4C&F~7x38*L$nCmo4YYq3TvF)*4LnMk(AlC6pC*lc!uHZe3F9~do;&sC=vTPqu* z8~ghgR$D837mnXGd-zDCoH4tN8jFOl>SZe!Ohpe{*(=bpwPsmsymHT_vsWLx_QET# zU%vW~BVd#37&Ei!Z@+i_cR&B`Z-4&1U;OmDKmWH6{`<%O_+S72z3+eT+c$3Bn4Dj7 z1p{Ox;LoHgBcscQjvP3B_N(9i&hNKR_gke1zxMujmXDsAK6rd`@8QA4_43$6xLh#N zn2t7SsCH_G+yN;ZdZj}rwW=jnjnt|YniL#Z%+v}QT0u{HzJ|-x@TI6ig{!0%p~5EC zIn)NHR$-QjATjR8sF;IJ;Gu#uTy+s?D_L|q^A0la2-m3OpuaF4ug&ELH~J?IPPbMj z(?iWrUoF_I2m0#Kp@Gc!NNsj{@$lg%UwUP`Ztpg|{@N$ye=8tX|r_gB1n$FvEB`jZYl&W~CYEGvW9tYQ=Vd!{05CbqwXq9S7qc!Tn6Ev3aU-MBYLrJ*iE(UaGx8ja&=^fp&9a{sE zU3>Vu?_llN$?X;+Iv%9$6!ZCdjmlw$NmNI+3+V<+#m4M>uu>PB(GIC-mw>Z_)w7e; zb0@3w4nPHdFGnTeYLtK+Ju)d^ShqyNkW0H|;vTuMS1Dqu!~kP=a(X`7y?ZCSiy>ro z3z)muT{~G_d$>JdzY7HTAQ42nsdlzx%jB%Nj4>HSVnI9~a}^^*+UF}qoe7`J40E+| z9t^kz-d&vt)yXk~!lvcJ0Dx?^oX=KBIBEr3tzs$^Ou2$3m38w4d)S;kZ02rucL%SV zE$6XSVm2h>=rv5OhN%Ie?u8U=or>LF7ADu)O=LXm&9uwj+h`22!X^~bzzTs}B!U%6 z3uL9OuBg-Mv*-y#il~5Gezs%hXFB(Mj=}gGi`}b%Aj*q`V{kAGwX;#8fLOXn6p1I0 zcoGUm;Bd?kjhkaBVK&+( zwVbmUcVt7ZLYT<<+oL>j+!n{tAgXO&h{-KpqbuQt-H5=XQsO3rv}dZRv9a2*&B-&% z18YqHFwa0Dw)#@v*3NvET;2wODk|=ZKLo z+N5;g;A9$~$q?%e|3+V6HtX^!J4<#E0N6qjCq*3rMeke$TaIA+(yrwoy5L6^gT|$> zc_n69ilK`!bRlePzFEqtN^3@mi6KY7n^_P)`=<*DlZ)3q-w3_P+t^zc&a3k&Un z9gi%HJia#m*v9mun{$tV*qFPzHg#ow;Nn!};$-Q zu+R8yvjJ?zZ=d#Br-6-mEfYT5lpmk;;gbPkE=nz?0?X;paw@o%jjUut8~Mooa%_J- zbf^$HoR1#NM-Eluha1U*wb;Hg*vTIotQ_hq9Bkw_E9tppU^3zv4G<$f`v7Uon#3i$ zqV9sLE(qvXAEm1h>VjQaCZH;5PNM3#Nd@#PsFV0%u}?3F04>AdoWqc^KoPw%VTB5$ zxj*V^#mT`q)r`3T!vM+30W9G(ZdK?&f>tPGgWD~{oM;p`hA|*s?N!X%4n<-T5WNEac(m+xmcQ7YRs=S7uLr2 z9-7*JbYy-1_~zk-qo=DA6MBbPZ`WF!Fk(}iT}G6&Dvc@yEGm?v&)=(VTM z-g~joSG2g`VwHaK+NmFY@5?{`_rLq)Pe1tik3aa;PyX>&Kl!I${^TEi_QUV|;2++9 z?%5ladd(a5jn7WL`L(bA^7nuE(SQE;Z-Jm}m2qkhvVHu?*T405H(r19t#5zltAF{_8&BLJ7_Yf_R)vG-%2~&iuU9g*!kVZpM1)e%vn-7Tdsr`DnP;jz-;-PAr?@2Y+O{v zHSt9@iJH`*VFI8F=$9kqCDL@PKipi7^(_;HSvWNzj|^#p1J1;FW$NUS+RC&o>4ZEg zl@kbDcNcIkuy_Xpl5{ah2Vm zwp+9gLWWv7S}h<`r&zFy$J)VRbntsS1Y>c50u{9F5 zMhv!qVVzRmP96iOTbHo6N66m8VeDjfb#QyOMijd{#XJR$qwzEVm_3)b9#1RjRSUaSqOE=<%AR(KdNBYPN2g(F6>7}v%O>bl45b_r zoq{LhsUW!uQu3r?j$9%|G*%CW1#pE`skEy#xQ?S1?dCIf3)s5^ye@@OZiQeGs4%bkz}SMkgu4$SLSGJ3%U10qFfi`HoX zUjPB3GPteDQX&sdj%%Z*m-;tbiG@aZwJ*BS7u`FUSnrEWq=>N?KAZE*WXQpwc_3iw z^BRW2mUcEr#5$eA7V@r{G(H-(40@4%T0b7MkH>8@S=UOHUartnNe2K}lT@tc9Mcht zTdC6wxb_x;D=E)H%(ap8?<+?3mty-Xv9&^AIY+No0;@&uTEV+n_Jdf?QVR)UF$Iu` zEhOv$aXe{^N|+Q!5an^V^|C!g3m^UVH*>-!goQocv}vW=b6|c9Nz!;BN)A;L6V1X@vphFcKe#yYh4UwFK7RR)7ao86i_bs!@Rc*IRwbDUEv?R+Jag=>3wPgr z@3}Mg+;#Bm$@#-a#`hg*tsg1QZYIa)%CpP$xz)b;)u{u=mQLI?cjWZKv9ogrk4MX8 zrBOqG7jw9@v48R0rSo@RzHjxw-pSeV(`OET{TsL5{K~C+AGne%=Zpk0y*O~sefz$A z>+v6a_sc)|=XZYgqxXOD?;rf?KmHNKZ~o&SfA?Si@Y|n#|A*iIU~_YRaNp}jUx#aSn3k{Rsw{Gy zO@Wb+FJX<8h;Yr}NE^|Z&K@_AX`4TX`?GkU9P5K*O0DEzTU>h!5t~PGD3rp$YgwXOFgr1#>tshX=FS8s=_M$EUk*|8(cB&og@N zWcTdkv3PnNLVL^spPq8bZAOs+_>ER%K;)=JYBF*)Y5=D_V*V~6yIag-O85*Br<2#) z!DDxdMBOqOOQq_S%Ng=Ew|H=SNd!kLVX4Kv8gZ9`-y!96Nx3}|ZWkXgthbBL+QaU? zgR$oh7K4lEjggow3+yvv&Bm~FG*Hh)>X~3M;x9z(A(zCg0hc5^ol0baB^Di^7cg#- z378*m53h44b594imo1kFby@+eWyxhMnS`rSa8ydBOx7zAF+^O3h{aLxc^Xl#lHV!k zbt@&k8hMWb{B2SoWNa8>LQ0txwueZa!_p(--pT0P$>?J6IUI?oS0H3cq_Eveggh26 z#?uKzCb7z)ZZCc@D?~=M#9~sp-EFUmrc8;PIhjYIX*d`K0MiB{AoP(05=)yC8EYzQ z&tz<=G@3|S;t5+K4UVAcG@6QI>6jxGbY=r|CFw3@4AB4-^a6$sC`ZaX%*oH zu35=|g&ivP9iq<9vvz&H_s-iHcXaZ*q=-UghgoWo#01P)~eoxoNG8}?xzhS0dzV^ zOvUkqf@i7hSuA-*w#slcT#ESwIvufkC0$_!W5lDI3tDEp$PA?$vdJgi+9@|QMZuFE z-E6=-?MG(=)>)rrK4{xWQ^)JE!@!CL(`SZqXNQZYhYEL%md}mX&Wu#fj8@J})J}|) zPK{Mgj24d%<&F<#j`XJv)&bkXdrSU}f_F7buBF|p8FD%0nvdc0F>Dq*oe(-5vdo68 z;1?}~>?>h>Jx1(JIuE2>hjZ@ZMe1}daDE_qX(WARJpa&4>EVUOW2-}tua7^qZ|0dJ zOV1o#dFII4vxnEOA6kF<@WxYH0T_05WBSVC;KkYcy_1!5!-dlW>66XG(Q4#iKCmz2 zT}!wDy=KDrOoW&T5ddA|G(H`0&jbLX=((6@Ax_V?)8gs*xNjy*jr)iJ!c;@GRgEHzW;ObO99hwAY8hfJ~TL(oUDxNVjLj zz!sJ90mHVZ$G{D?mqKs5S_m`xF$3V1!=SXmG6$jxIL)+OAEj)m08vSKC;ChCBmFCr z!v`0qE}uF4+@qInz3}K)Ub*(QFFo`RZm_xcm>z46R9 zUVrw5>kmJC`L1WKU3vB9Q*XTV?CUQ*`=ytkyZQ377r$_2XnCbNyIh!D$V|)@W)`b+ zt797nmrtHsJ9E!sd+_ko*xKIm(16qD^!Ob!bAxBkop|{1t5+Vqde^0U&fkCjS+vO*Zx&ntk}{nOnD>{MNfK{qP50|BoMi`$s=`?|a{U`=7u6 z^`HLRH-G-q@BHuw-?;zc!Rt?6`t~=!^6vZZ{o>cZ{>5*9`-iQHO@BM}Z?_%UPVoEV z#~=Uk&mVsL@yG9e_q&N&mCB^-AumkW#YTjsRWmgTmIeS!#nvgeLItSS3rU!;xJ$|H zQn6%KxyU5yQFODF3=zbYKwN=_qclt92BF?6qobBcfr!`L-V%nV%tX@cjG4$dO2#do zw2dx0$Rh48INU|jU8Eg3k3CCbdD4-mTzNV^m^uIGrGNUj@Bino|NGZ}{$o3z{-clE zf9+PwmA^3zMEf`Z*!IZb$L)TY|N8I$vG>@iPx%}Dbl;%2+4A+b=>9>f*>pF`&T7h8 zijw7Ms2LA7!c@`k%K2S|kR#*sl>E`YP;Mk#oe8Ig9sY*iTZ6n+6IsAR^-y{^YmH$# zuioI*D;x@zj@hGTauKl{Q%O-tud=sO0<5d^j`nyUb0@F2TOpETsL@YpoDQMU0A5PA zMk6o)%Rxm(RAMrV4F;A*)vaomqUe>2IPCyuc3!Vo$dF2zN+naJ>{UpCatU;DkwGCb zs>CLhz#!+q5|&QFMHF1Uf~DTdYv-{#IK8`B-M4q|`Cq$t?i2|iFKtbyoW%@PPI`-p zSUu;<1icB`9rI{#gpVktHob~KAeOuR_JDEM*A|X$u z6ljz}tx^Q31P~;IVZKJilnFQr0aqntsRTXpb`Q+1Ho6pSt(>Wqv)WbOA(l?fH!2l2 z9ZxI0y=Two_UyjBtNV`bE)aM20Ne_+7K@!GRW?0K!xNzrIj&SX)G7kjIL*e8*OE+_ zQfUOpRw{$0v!-Oq5KHJHacwZ94TJ&AkZ979&SKdtkUTMm^xYE`dvyfQ)h~X;K#nXv01k?c(EB7OH2OAf{2%t52t~(hLmei=?!!i@?um@Z;_@v|`duRIh4;5zf{?)$tW`AP28s2EefPVD{OznkV zG3!vo3g}fOp@D#TTgArW_NB6Swc?x4x`+J6x*HnunE<^eBYpOV)BqB{1WN}0hfJC&K=TdSgSVZe3KAj|JR79;B5D`=x1b_j7Xd)KKZ&HDX zV7ipcm~cW_5yBy;7TIOxd+8#(WiaCI4fp#sA_FA#+ z8vwj{Oxl>+8o-QzUPiUZ0?7ctq8@vU#)@$-@TlolWvW$QpBX>8zIgv#N1uEA%B>f! zef5jizwzqzcVBzzTdzO${;j9Jee3$S-?;uh;K!|}-hcDiZ@qEj-CHldbL-|iw{Cp< z%P)WM*2{nQ)+_IQ`K34CxUuiZy4^>5(&<>EKRrBNnpqlH**A6II1sPpQ|IT8ooTIX zjIA!OA3IPTC=#TZ^4Uqh!yRxtd`^_K`Vv$)P1RfJnf0-Krw%;v;XS$#Zvq z^_%bgz74QX+8(x5ig3I0)`#t!y-(T;us;0gFQ0t!_3!+>D;m?eTyhi@>-7S?u2-$< zlFNFO@;1gGIY+PLAsW6BoSgu}#0-s~Q`y_D27-!u)!lrEtu%>65Qn4e)!3AodZN-2q>KIN@8ynO?~)<~!| z>>F%)2W#{|jvh<~29uGYbZR`4o6JQA5@f+o7DKMQ-(L*`DpasUB>O$d0bg{$Pgk6- zg4LO^QyGUXY=$T;;z!Iu6pv$STqV$pAQytiod}I6tl(rP>6WrP#q4gWfTfX3Y(|J8 zRD_*}=$TqA2L_5`liQsN%%!jsD$J?EajC^DMD#pJ&DO{{8W~qDVJk!|pg3X)02o`X zVryg~M5(gDS{#9#Mis7?SpdeA5{p)1gM?8%z$9g~lK!3R8A-*ky@Pwxq|N^dLTrtC#NN_3Ypb$MH>$dLhU98?7A(y9;0)`18NUYa!HHr=)a70#*jMH9EE#r2{g`IM7w@R_q z6%)9fh6B{iq!C-RBBP3_;O$^`-L_}XZM!>f@9O+q$4>CzyVxwIRLoaN*-8=5AXO6W z5@Z@T3{gg7&}B=;tl6A3SG48|XeMh)XUwULC6l#g(?~pOh(=BEB$~`P@&&S7^w!J4 zfx5q0^VcikzFK6k8R)MCn-#i{1Yd4bm{7PZER%t2vngl_3I#Qxpu*>qc|2mbOX7A) zTn@R*Ap>s{iu7oeJ4L+DGrK?2y<-P^j~JHP{Dj$y!ww`Arz^$i-l^vP!NO`iv3D@D z(M)VKQ!BN|_*VYmP}tInphHn>oz^!zNWTwl(Cu7A0I=Dddn!$gL@YH@+aTf5khMKG z7O{?ot&=Hyw&1Q&dcUG)#H-s^p(Z01kF+DC?5>yvOAQcreqpq9-*o+f#leS`hc3+n0&4=tuVcS~Nwl9ev&AU#N=yT1`rP0KLleufleNU{l9$z21eqi>-vGo^E z9enZ3;TO&vy?OfRjg!ZoJAU}t;|HEOw*QHPtB-8XJg_`^|7`2tiN3ps%V!4i$Lom$ z`QYBPZ$05zPLQ)vVmd@j`-!Q53(#xI=bjIF7o-09uy-!xnew|Pea>OhRz-DntD)sU zN8I*t+BqHcOozRrL2A(Fu8>#;H~S4LmqzA-m6%p;ZD&vmEn<#C!6!9hS}XNxqybnK zgk>>Ana6Ymhdyi9RGj*T+fcMCigr18rv|;|x>FAz7lOngox}%=C`jOeMIhWN9;Foo zj0zeOQ)+S43iy?#KJ)0SFFpO`FFyCx*Pj3SYtO#>`ZM3Ub^Tj! zJi8TO=lc6^KL7q(FTMBWm*08wg?HQgFMRO!i+}(2D_?o-xx4N*Hh5 zYArWB(b@vo%;D3^r_WCvJenUF>7NTQHM~&)L}9i z!hvvUq#Rf|x^njeXQwwtQIA1_h{ZaQ+9W3eSU49l5=K3)r($HZ7;cP~MrIn(j6a%5 zU>=XwX0_3txy|+O{!6>)dpps*jj=Y&wiaErt>up&fArxN6)t??VL56M8O=NxVJo#v zrMgF<=vK&k7S86u-J^RzE_-u~0d*dSvyj_uhW{n_qqP?Kgh*n_vAco8$k9Etq}s%Rl}8 z(&LYO$`lHlV+l{A85|s-`%Cmdnra5AK99fO7afkoTEXmSJUyJE%K@qw@K(a{fq1Y? zdUMujjSAO&-df0A_PPpAJY_Zcbt+seLis9((h@|&<$x=O>0LTYz(!^$y5J2KBWB9R z*NAvpu?&S(xD_G^mD3@$S%pBk0J>~0t;?@<`C!torvp0Dr}ubtlp7)(3e+S*V6jmv z)G0U$Q5TQX!Q(MxQXZrdA!;pVFp?G{WkWnR*kw{Da;H@N_zJz)xkPWYU9DCfE*X%o@GTV02k+A&N)^-T5$8YG;t!VlJsg zvyAht z->Ve%s-;~D5VBs4ngwgwIxR=97MV3l+#p4@JyLGBfFo4MJDJSS?cVYEuAMvCJzSNP zB@^uE-NTS_Bqo*0(eBfyBy^^bgUH43d=@WPiE15e9i<8gd!dAvid0{PsN}JH+FdEp zjk>R|5oz_O##`B`k<`R+e7HX`)E6IWga#^}a>C%nct&}rhQCuO=zugL!m0B4H2$DA z7=gkuC`Xh_sU!BU&FVcjA1rk)e{`!R`KR&z)K;M5ldf2W;}7(UfnvKYw&# zWojVG|Nm~Av_ zYkCcJ3K|NUM?#i~m>tA)(lL{DO=gIKL+Mv^4SAsHh`Hg?1k{X_o;yrw=VR9S2)Yut zA1HW^*CQvJ@eAXn%X5t@^UW)B&8y49kFAY8I6nYLc4eXe@qLqz?wx#ObMgyolaK73 zeeA&e7dED^Zq7ZtIe%qs=JMLy{p)iVmnZL;A3i@laAvG_a;S8AIDfo9b+8)VUkvWc z`8RXE^{i(l?Osl~+BTMS%|-FK5HaJ&XZ+Y)5T6g>0Aq_G>w4UNFylOur_MG4m&a2N zP3NCn?Z3WvgXC=3GZqCP;c z*`Q}OKuvm`BQ9GF)%Dqs2@e5|fa5b^?~vD3aoS@hy;rC48g&7y(XN(hxJ(6?1_|y9pOAChYM~dFk!rCAl0_}LsKpQlWHPA(sFu_#0DdT)(r45L zQGM8oM65{2toNDV?WB1iSiq}50Ca8V(^H5B90dQ{UcwB?+bx?7YD%vLcL`byPDtTK zRAC1iA}kI>4Ghe#SNI82I_R#X0)wT*>`?94#{6A}HZL4M^ymW@UwHnhmtMU7+_Tqi zJoDHqH?O_*+OuE3_3S&ho&&P=-W$(<^Ua&@z4^jBZ@lpO%h$hf<*ojzdJe)6$}y?dL}^Row!FCV{q<n(nVGfvWJiU{F08iz2O!eUipn4WOjX?kjT>H14A{nNkx@Q2w!L7Y7ve~&s0K8wU(t(13lnsWgJMtg~e=0$kj{uh+JS& zvSEn;k&`JJi#m;dn?2;P1qe?v=uJmM`ONj3FTMM%zkl}T&1!#98mCE(BP!e=x9a$?oFV0P37I>4yLa?584?Lw(=Hum^Er%eof;Ku zt!l(!G7+eWu(_haSfi9|RZ>GmPd=cuYC43B+ZnrWXZGC5V(nzHce6Nm_Od?D=)Jv% zc}Gw0ZC%~KoOZK&yF~(qOx&XY#w}&2r9E;nOD$umr5uf%saCdg@gbn+8n#}|HK>GU ztrAD1W@s0W*(>5XX_o=j@9NpLlhwfo-wn!Yz!0{S%TWo1dbtQy3(QKMNdb9mcs5Ry zi>`VDtJd&p%~@?YN>zKYNK{K?qwJ~{s9G`7>Pw8aVk52m%tUo@wz4pjpB_(*wNk^) z_)wj$Bvo!xw^newxcd$%yIZg3<5q>2R{4WaFro{`4bh|_meR)(hG<`EqoAym4ZA{P;}s*kt{{P;q}Nw|^)%nI>Do zcH_#iI5v^QTOo973A1S^WCI2^7(#)5O}9ID;Da>M@*77&DA-QMF%WZESG#jqipb&e zkcKhhgNOaPlu1a6cc=Bd5ek|On$}W|{dw|G*?)QpFaNl>7&mcKltdz;zP^RS68P2!!AwtUl^;LYUK{s;|I%;{rTW}*0-1h z2IiU$xo1M2S%9v%Zz&O2i1}v%)U=|J*KjmugnfDq z_(tYvs-8?_!+|jALSZdtHo9?J$VcUqQ5v@!b_-onCnRw}GE%3a5ly>Ur5Oe&a%ttbNa~w|epJlUEgQBM&_P%%eB1Kk~}+PrUKU^KZTS;u~MQ zdGpyP&YwTs->MgjvGwJ#n@>G-{fP$$2I`2#WOSm=0Dkz?=BsbM_{U3lohdmg!Z z?B09l5AU5nwO(Hy)KMn6Nv1YSbT$oa(}^GUD7KHaeW&)XsbVvv9z4ssi5FiN9drwx8EMF|sRqeZ{yQin^O!v&RogTBd zw|l#1e@H8ZRyzH-lTHf7g+iLpuCzOA^r32FBi_hLfFy#<0>0;czI?y;L%>C`DRiPv zW0z?tiPbymD{I3on9Z68mF_G8i=@THDhNbk)u~AX;lqU}13q*%x82M#MqV(n9Kvxn zfd}D^$J)!wFTefnum9vvzxo${^vM@ref*0rKK%4&um13z-o^$Z@N$bqjS(^!n=o4^ zAgdh4)$a5IP7&eEA>RbpIT_d=Q810e{h;lbOXtmNU-kA z&V-w5U?|Usd8fPN@z>m97N^1n*r|n_5aP!z;4|ShX<4}x9U3uTl%W#X3E88Jui%VT zJ?^ZXkCWcKBUtodqIFC=E-@-}q!ki5i^py8`Vn8i;sd@FwFTq0P{QhuBi;xWh+_T_ z>I+ysUdZ8uMA6K%5KqHAcuSII+M>tIupQSCu!b-r4jbkoNFPH7Sklh|bb&O%9&w;v z1_n0fB!K0B&=AnLNj;>UxF{RDESJg5W;KSwEGdLV(q)BcEyi1Lk+9k^h(d*kztn4$ z=9-~O0^-pt>d_Y_hR#XH&&j0crP50Z#gI~UK`MKGeB#{L#0#V2=f=mMA00g}oftD2 zz@%XU*2^qL83Iipune=vZ5AbIG4eLEh=asRZG(R>HuU_^MZwJ{ zvoQoWDa{JhfqRl3Pt=9lF+Bk(F@p@(t7x-=GQ%F0PKN1hk}Z|&l`2=C66#H^JTNfl z)TA3QtX>JX8`*_^VR@mpx!O6}Zy)S7cebXsH!I+omHFIUn~Zoao0aGE(o2wf)COss zyg3xG#ADWY@@e5>4g{4cP`Lt;$&$G&ozF4(JXg#Mg)Ey*k+A?B^?33T+(U9PK3ECG zs3XamuUP|sin%&eDSGqgHp8J`VUXMAi z_~gdcCnuX99${7O*19+P_5FHgx02i`B~~+m*`U1}pc{g<#y~9r?})ZR z*}b1!iP$%Qd&Pydn6MdhZbV$`ApkJ-T5NH(QzOlgX26?64Ht%6iw6AOf4o$W}7Cor-P5Bdg)Sa?rczclDespx0{9 zwGr{IhdqmaXH%r|6p|$1j6gLK;f+rFV0meNx+O4_z>u^JwNo}5ia>e|Y|uh_wOOaq z$s|UlTqm87k6e*ld;yV-vj!D5F~qARDZ-p*;kpZ*4v# z2{S8#h&qDm0+7n3m${7!r%~y#=p2wnG^^aG&Vi^nlT3hA{L?=4PDBsr6&lPd4pz<5 zIyr#Wvt2Hq8Q8Nrj%THYJ^}$A1{n6NK_YL~IAMbeHh8S2!80W6qpShe7T`fbOZr$M zAW+$`tC9#*;{N%1VW&TRu-HAjw)^VCTaWKwKRw>r*jPH)+vs(s=6jvhwZ)Ci)vL!x zPhNif#t+|k^wOitTFUI8-ndxL^PNkFd9VF zwA9E-g|sTxRVU`O-lm>OA+QtHQ4qwFO58Gm=uIqa4^y){c!Er8)vC_d0jpe7(E7o9fKu`u$j|!9=5o!)fLD z{AA-NpMUj#{O)&;-+1g!uz{i@Iu-Vo{d`*F5)LxT!+y-{LriW+?=We3gNo5;IIT_u zN446nRnuA(sWdQZ%mLZ`gflL9Q_fT|xU}B>7ys&)|L_7dN|NWJ>-Yzy9-c;l}a$E&Gz*#EimRHit-Q;39y_Cu=$I7dj{9H283S>Kx z!b~zy@v&)M$PA`9@sQE%km*^4p3|9JW`h$9tP^tE=!Eski21S-7^=rg1_@7=kJa44 zis&owe4N(U)EDKK#tm`}j=%zk`TS@ggoR?ZaMJ3Jf&eE(B!!0(D4`{=59L&@)76ob;*2~VzMlUO5Y71iJ?R?Ogti;?g*23s4EM%~mCV-_WCS;I-4SSi8 z6R6K+<>>i|;X$`5m3mmG9U0W_>qhn3D=O7Rnf$U$aYe4YAd{Vw$wrI@1&%6gsKja- zHfx3qgPFtefrqI^4C)E9QD%YV7^0@EkdtI%PBLJJ1)I!b`oY-9A6zxLlpmVrCy|>re-JRavoLXNBRP!>za^9#KvYKReLho_F;UJz)QMtUm zSmw)Bp;8wsbw_Oygix*6i+Qe;chpL*TG>@8h{deClFjrcqs_dx6p2nH!?jqVnrKd! z4%X++4pwfhwa>PCXWO$^`{n&+cC!*)EBF^u;xm9X{aC|`xBNsWWCw#FFthc-z`$mL zc-xP*{dmJ|0RdnJ=+%$Vb8)sErV=)J$|mcFu=yC3A=IM$1pu(MD6^Af4|9&|Rsa1? z>R~tg;#}#?&FMEc+VAYlet2#9gQKONoUVOzvhv>H{ErUjKe)R1!NI~uhf5zGfiEq; zcewQC#_Xeosnb?|ubMoV${sdy`;(dNN^&(D-7F-xO6iqMXeAX`jC*qbnn$cH#>GwBlpB>J8 zdAj_od;7nn>npk)rM zD%x^1oF+`b!=k7zf@y(@dGsG*q2EGcO|Hs-5i$R2R4p{%p+Xj=0B=E4o z&UYb^j(db$#9N4Y>mW0)W)~-mi?#f6vvO;1?cG=IfAphQUwh^3<;Q2QzjXh`(f0P{ zN`GN4U#sP(8r}8v>E-2gy&f(UL;0dNl@`NcS2P^UX9LMZFrD$GlT1h?0#=J#g?m)) z1XgT^W;e>&mQRc$e9Fvc5Lb?#+vz@h>(Siif(<0c9D+G86|R#&QUxr<9T4g?8w8_X z(5k6%%&qiRH1T;^WPaQ?t#FjARFq^uE*|nBf&g=NIv|ifQcvh$9wYq@*um>LyOyR* zcE-ZntpZDUoO~$Yjz*{E<_^#9chfB9Gc@DKm+*|*;a z7%VZH#vypj3{Id4vw1|TxvY?1m>3&TOUHDwVfDBgH85Z_?X+{VeP6k5i^L2LQBK&D z1g<45S_~?8n!oz9KYsGg6L-SyN;BSqi%SV&-sP)ABh5UM@tHli&S6n;kdiek7^BK= z8Z=7bAT4Lra=1~nMeFIqyT?!7e&zKK-+AS|x8D27`~Tt3|K0EZVQ|y#*<9$Lo7CU_ z?f>_WeH8!xZw5UT2eXKO@t^i+I`#tb9Fh6z0APxj_>E6ee@MtZSS+p11&)$;S% z;H1Z26#aRJujuiWgPu~5j!}?X3-k+i7<7zA%c_;sgv5Gf47xaMydXiwEpC*FvGxS( z$gts>D^TXSm`#bx&&e(h>ts3#g9V-lgzT|26U*SCI39@O{ut_wSp5+!6eS`_Dw?Dt zaV8dHq7i#6N`*r|vLX@OrYrKsFr8N5!aT0#aX4j*dZU1BcrNTU9xQgvXT* z0EUT4FBuZ>5Kn|f)X$n7gjpaA3@WqehP8@ejY5YZY`_^WC#DxB%grR`#f-RaSTTBT z^vb!h;q#KQA+_wXN^(&#c2Oa@q?BJ#1A7COZIGCaa*J7oAriA;RI8Eb^%A{dRIeY0 zASG$j(3BiUCs1e%(T^hrDQc2fj3Y+Xs8J;`>nC6{kTabfr6YnT=jUT0>Y@O^E+`Zd zz1G556Zv4Zk@CfOAxOotzG68>b12Z(F_S_~BY04x5Gwp=8zZACD3k% zyWRBsTyc3Z*=_slWg(Mf5|O}UCD-qk78mO4>#gmbnZ4c4?pC~2)3LTIX6=~Gs^IN< zuM>%c@MMC@XZdQyIXUTRw0x~;U#lJHbbQk-XRXSW^Fk%>YgPlTns>4oZr4g{z1(6e z-YKQJl~kt?sU;HCWT!uQe6VtJd+z#b>;6vf_C^~3Y-cjLQI2esf(vPJF(b^zn3@Z2 z`f-4+c95R-+m=$oTF%vr&{J-x>4jTfbkYd}jLn8A5WO(9ly*#qs1P!iA>{pttsfy` zRw*ZaE{`kLA{3z4&3f>DJMm~Xch*k5v{-#>tNqqi=Y0UGYs()WuY7W{3asnLM+@)n z_ut!_`|bEKZ-zRF3PZ!*XJ`6g!yAUTc?c&NW}$oO`^vaC4!vH(lGCuJ1If zo3;GjROw))S8-Pb;AZ(}`F6nfJFQe}1+1)%6tsuwOm6{;S9LzI}4<+n3M21^|0_^XrE`RrCY zwGIFlk1U4+OF`d~-@O_2@1?>^UZF)}4H_-kkdjEv6_N*Yozu;wn>$jpMUb}&p-M0=O2FYvyVRb=!2cB*TRKzs5%uW)!0Z(jHP_p0w0NUk!Y}33>FH` zcsy9hd9yLpg=txtnjO-x7j^b45I2gu<)}+W2h=vdf{B=BHYZ>E{!*YT{jQVZM@Tm7J+C8xC;c z03Y>HJ|1=2oyn-Lkmr+eG87;JKGN^8c|^i1G6A;`^}Aw$SUIZ(=~z1jW8k3RYJZ@&3g|K`g-`8S{a;h%i-r~mf1 z|NcMx>;L{Q{`G(NKmFOC|BwH}|N6iEUw`xa|NHm9|Es|o?LiE9_V<7HU;pZNzx%tt z{`8k$erHTKGNc`wu$t*a+TWT9&9t*i)9r&^bElo|ro}8vhfpp;iV5CV35OeiIvnDL z$PmheDTp`8Fv%!%dBk{rNPqsa1^`UTrCgCl(q9XDD^72j)}F6!drgJsdSfIx%=tIrNOC?hwx z+oVEGQb;QWpIy}GP5P(iCKq>RyUSCtoDZ@YUKl?A-3u=~KPW7e46EfMdc{-r=*Eot z5xss$uOBh!rLa+DgAH~}W3wm_v&sUiP>b4zY3+4)3 z+F7sgmAs>x54I|?naMTJJ_WrAWIJZ5Pwsa<*Hp^y(|y^T$VP*VboltWV$H znZ2yb3ua@V1qtVOv{HZB-rJ&2w2thAe|uE^jLv@ zbwlK|Z&0f}?L~W0dn-tJO~U|SZ6DGOSpB9EX8aFocHL&2*-qF`%3c7lC;h@p^QG6< zrrz1@zPmU3-of1ahYKGZEe<9R_xnHEpL=6{`jy40hn@OqIdhng?-ybR<>ak)_4e%4 zL94V{PA%l(?U=6_@z%oLj7LnkxR}7CooqeqX+?dDx%gTky^@ZsCIhQ+?^4*g5OnlB z+?-EX4!PE2zRjd>CllPwM0V0)aNHlX%?+H^A`fTMuPx+1*lzsdxc|lJ@;7((e*N&~ zuV1|P>nCU5KGm;5DaXB&kB_!Kxw`ws-IJf+zV_*@s~?^0zq_~k`fB$d0oKgloXXy8 z>1X>M)l)%Ep&G!juguLb@r!((wpO`S^Bw z@@R4P=Jx8{gWcPEdv}knE=;#EgTbs)ayEj(EQnF3*Q)e7t=V8U=}jghgrTHf2ThDh zV4Xhc^=AFva@b$-yF(bv$VZ9M^IrW#6jo+!hB9ZVyRe!Q1m5&86#-9^CO@R`n516w zpt+RWtl)GLcDT!D<+^pJ!0#pc9AD$Dl!x008qa)}TOSgN|~{R|>g1 z`NYBU?2EUKK6vZ#mp^?MaO>-zzW2okZ~o$g*FSpiwYT1Sb$w@7h{TCNj0z+>x71(c&fd87_UG?x-QE>b0>tYKluB(?$RL>#QJZZh&O`A58Wmx)U2oxauqdaJ z5^qc4YpK!;eE+1^yS5&0C(E!No2_HuX z1t#dA0xa&bX>dI(;!H^JW+Ii@>D?RGlJ%0thN^K?VMP>HREFSE7*(Q}3dJ?JO-Iuj zno!zM6>e4In31LpgJ6t|UB^%sw_8YMeC3+ARAmzh-0w4Rv<%ZL2*U`ZIH$N|AXSbS z=7^)N6pi{pa-p&MDQ`8O>G$F@Grp;2urm{A&jeeY}_-aMK*IPKrK zw|VFB?%kKS@4S5U;@c;$zIXKGt*bBn@W$)!-FWTY7vKB%{V#v{`5*o9fBD_-o>kTV z#sBi3Uw-fH<)h8H{ng3EneR*(#>aINqgsWEq^WelJ=sk3JN50k((v(rYNfkbk!J(ZthLO{J8WPr!~ z9OPsSJf)#9B@P1vlRVvWTZ`)430H0^2_jXCIinoPBBRRje|G-)?_Rw0{Mh)o(V(zG zQVSwOEmA8yVS%MqiyX76a707EM#icq5j6(s2;9I@Mi*~%i+YjO*eQ)2yc;Mbi7P0p zg0jd7SZ;%Whm9F^6OdU$+wdT7Px6k8!;#<-4!NityP%N4yggbgBPp4~_UOA+O2r0%7nvG!9nDQF?*3cwJVHQ_oRy8$4k($}zLT z25Id$mxv^$ry3i}%O?kui?d?Fr(-M<)MRjrR4zjo@;2rh$`HL(}!-Y_qh|pWa_9t#w1w1>a;Y)T%_Li^*QCxii0Xb#3EdWq)_^ zc&B%3XZHT~?DeI|Xk zjZOe>xDfy_fUc(`dwTA(Rh^cAX(Uf8+dg;zF!Km6e}38pZKs&6Bz3juygeBK2KMUG zp#Jck-Ps=<%)PVQ1L%5hum8jK_8Tj$C;i%uT4FyJ+)jrM3W-~->RG>aXKwmxv$T|n z6da7(Vsuzclu2VVYV2T0u~>N0#u8T2tfS335ivy>Ti(If++y1&&IQHAu=803;Bwf# z=y$G1ecS24UOse?j~(O_*UFibYVx!myVXiOoXx$mQ2b!0^^4PmFK(`WH2~PH-#kA1 z&C6%sym;rU2RFaIf8(>0-H)zreR1#f%d;ClzkBl2Th~519RLg%*vpH}$Gz&=bm4Bh zc&A;u*2rCLlnz;MiJwIDW(uwfLA(-s6*D|H%~ z+Qo47bb2xw?_`tH@lc#ZMU8}&UJ4kb2~?LO%sIkXpp9h?E{m2ar>*IsCS6pPfd`$X z5cL3Hu!=V+c(al-DeXELqni*QMVLX7ye-Dy5gKz_^iEjkfCe{89FQJREkIg5sCm$q z-f9r7dM9r3(iR71<#?;C>e*vZUXcpQ!|jel0Ck)59A3B@K5$!=BfJt!+ofsb#bVT4SO#7RKxhIg-T`$6 zNo7+>Em8?IK4KUdHI3@+Mu2x6Wme&Oxm9O%u$13pVp$`{D{VFfj%o?gz%VA>ZsbWl zj~jSGZzqi`Yjt{2R{#>+239cgqSfOCQW2oqcj3^f1=lcBmRno&KeaKte- z1oKgh9Nf=SDQTU>$H!aK`QBoDrWc->^-WEO8lCL?N_u`ZGrv_>K1eU@#QK{F5VITc znYCnhqp^LXcl>br@J?(0cKh)5;_3av$FHBf{KkWKfBe(m{`TMh&;RjX{(g++1dUCeX+;X&)7L^Jzcor*%ARa@x#p zaQSU|9@bDgJ)>9Ssv+&=A#x>4rJ}D|4%BPTVi}MO@QjTogjCXz$?z#~9&u*k zk!m(EnTr$?W(s+J^!)dR&i`QS$_uiIaRgS=G!P&o&q7W+?6O-utj)*MK?mUzEFJ(b z4tz;V6LKp&YE)lQO^j$H8jHrl!XS||kpr(+Tjd6uPKO#^7`yTZLzgcq6a%TEa0PCW zp|Fau=op)kWkBkorZ5EtX-TV&v;qm!ks$pOk$}$}^bR&bb_gbi0FFkHH#u38lhKQm zma_nVK!Cq#7)(uAlsIDG2t3G7cmLG=Umq*wghK zgc#$rfGz+q@X*zm3eL;~JDotg z z+MBzx)x9=fJzcEtHWTaB;Cdz4Pw>4MKNGgkgxUF|v*o8M4x|@lmY)I)(5vaOOu66+ zZv@bqat+$W)r(%tFW?PlRtGktqHb-$Z^ZL$3Ee)o&(D_`E;_~!26 zZyw$G?UVZeU|&DD_0`$UPp=<*a=iQb-4g(_FYev?`Q4kJ-9G)v^}`<@ZoIWI_sY`r zOLO%HozlHd`F5v#+$>ycR zIVa0HO$J0ZCbQ^Gm3VT#R-4JDyUEy;&y^%B5m=iO-j}%V~tWDrVrT60UH#snSunAi2oIRRy=ktL|2^bGZ!O2k276=f2pF0zEq`VdYSXQm)CN%ad z0Ao5~$l#Em9t{(K8;hmgTifx;1mPo4rv+inlna|%>%RNpJ3s&Wi<#wlwNXRRm@6Q< z!-Car)!`b{ZHP2|)zx}=aY{`4lr{}O(S&hKhic5E(P-1FEowbsvIsU5q!kgLD;=jJ zJ}}m4N#i&qksuQiL^2MKgBXLxM$IE5rjc>;xB^wmP?-#sCmNZV&BaI+$U{PgqL;Or zQG)?w)iSF|MZ$Uks4K3aa5X`jIk$;(8Cg+h=XG}0z;b3mM4T>&=VfNos8%@vX$*GY zYM8{R)L=?JjMILE@>|IOt|m1~Qg7gJi-Qy5QGcZ<<}*w@4&)@&>E##JiwnEi{&t|b z;F+BFH3kuAEyjBrmCfVnqr26ull0vwLh-aPGH z-D&JDHuvXh8?AJ|^c|T&F40IX4qB&6)daz%GLh+4tlRQVmAF!xFJ`!0(h_vRJ^=|h zEkU;}}l(n@Rc$ z1=Q`(+i9~Pkil@Sx3F^K?()sslbajycFS8XdP;d$K5I{=_)JdB795#^D_0QHIWCz2 z7zOVodfiGUOIk3t&*as*3RneAc<(jXG$9;F&Qt&CC`msz9Ju!8o`Jq8PN@o8zoY+?2_Vw%q+8Tm<-ZfRx6Y^hB!sD z$eBdD$zeA;S;)abqF~?{4UGe&X$j28kZQszM*+?$quXQj`HVg{!+s*- zM+0t)&xr-yn18Ufv7SW@0%q}(SeRg9G#jIAZb(OJWQYdhZDPV7F2=(7fIGrbENn2z z!SG_iO_Y;%CPS`VipwOaL=+6MNYDcq1_j(mz+(-1&48DJ9ayH0#Wf6~qT#{vZPKK~ z2N$0S+Qo?ANw}P0-X5^yUWSTz9OZ&*vgz$~UCoAbs^**;FwEHi^lJL1JK@>c$ZR*( zp8=6wo=-3KQgbuY2iw#8Td7_<(w+q9^0%syZZp&GRM+NvS2q{0Zp?4bZS2hNZ_VA@ zUwm-5c)U1yyinb5r*@n1wX(mL5N4uWJ7n)hxOR}4bmQ$HHJ{*DG9m!jv=0SQcfeJ_ z1fT^N1^@D-d9>+<`(fKmg!C9kIr+IM(YTqWua#Xlrvd4VRw!rg9HNdeN-toDddRB zh*~YdfIkw6Bx1>#cDGO{maCO`JQj~fdCs0oCAZgBH~QVVYGEqkEqm;F7N2x5eZOlh z9y}~0PA4<>yOjss>g{IntOEd+ztzm#?c^Tyif=5}Ki-@D;$-FP+grc9ckMSX-u>-M zXWzbf=WF0wH?IEV+TNG^Zf_Qi?8i&zICv3r{7%&`3i`kj>CPC={cC)bmX{} zUrvU$O39-}>1?@owzc?hZ}XLtt4~f(?jIjt-`lSj3swZSzz~L75DS7>z$46tGc>`P zwI)<=$_7I7CyUw`rbN1uK) zzrE|p*PP{sv(ofbn_$>=WV69)C0wuh%N1{-;>;A;Sc(h=Yyl7I;b0Mr6=tI#)3U=l zei(KuNiXD2@mohr2PZq}T3QGSHj%JUh~VY#Jh=78fBG-~=+FMNKGPnOj3~7V3l1Zs zSq*C>dZ`>%h;eUzW~$h2*}UACY2u1@NCC?by9uWC8bl`1jmS(AfI6jBV`6b;(8I{C zr4c2r8!-Z6jRJsy7=y<~p-}*^ar1}_8CPO5DLQK4)UifvdVSVkNJ6}Q0*1$oMv2KJ zgCVKKB*#q}JEG&zrxS^^h7dF)XJk1Y!|LrUEQ*-NWApk9Bsr#%UmPE~qL|PV7z$p; zsF0h-?EqaK7~+iJ|Hk5>_+-FeNN8}$?@Ci?(m@wv^y+CgS{CqB2@JUlsm^X-#2-`;-l$?WlQb#pDdIG5^oW4%^< zp;g%E=9U|=Uga%cP?fCRVQ_ z^eU@SWr0Qw>Jg*nvR)&@&~UxpKRjsfY$ZA^AsxrTK<^P@7jN-6L7L;p3N=`MtQ`iv_^n>#=$P6nWSoT3rGRP(c$Wh6Zl}6@iYJG(!fRlBV=dhr#JEI$dUu z2l4wX0k16-00CZsiU#e801z(&4;w@T@!PCHoQML3+3X=J8?sS;it=+@#1$^Z5|wDW z6!wKUwRU1cK02-#l^PUA8YcoiE}3NFQClPkpauC{2A5#+h>+W1cJd~XF$$!P$J8vM zVgSI*YQkus&6tQ&F2WJ!{3%Z$<#Pk2j=CI~sJm8n)*G%?$35Nhv>M*&hOgZW%uI)S zy=cFmnD6IS7fTx}#r5UF%3@}r*Vx;fJw7Zf&&N8=Xr}>EoMf+^ou8dtUzyq8nci9L z@2xKGEwAq`ZEen-?k_*QwszRB9rerm(*uC5mi#MuZ$B;q{Q^;SSX+J)IN4mBTg|%W z<1BD5;9G;{up%_=!vMg_oT0*-dQtmaLZ~_r0I-%D?gp^wfX$&9VrA!AE_g4;1AyJA z2ky6%Pv%N5%~xI-Fl_3LwZC;dqPu+v6vBO9J_GARo6Sd0M* zcM`ZkA=j!khy}A^1Vu9z3#?MeWs-5TLFaPv6oq3bBA1TaNql{A;b?PXZ*gI*-DqcG z6_1#uY-tp!(Ab<`+{;I9PZu7~O#y~IoUNbD0M*Ms=$D?%SKeBl`ed*7<@NP%@9h2h z!O5>5-TwCB&2P@G|NQjulf#|QZ=QVf;NCY6&%Sza@5}pVzqoVfXSZ&Bdi@w^*^dTw z9KDzNjfXR}hrPywUh{Uhacg$^=IqR!x!$AI<(D?s-#plPZFl3KUhTM@b;{OvaBD%& zqF3Bbgtk+WrKoQs6WK2&Z_iBLTk70ho_(~n`s8r;@wKA|SFfIIZq?E`fuLZG0n!*v zYQ0XbF)DPh8YHMlF%ViPSLf2X*;s5jn_f=D7sA2$uz$wqs0&oZj+JR^)@sODbOoEK zOv44BVwAbaA~^=h&=80MXH8S)0HO?{8V{lX7_)0e3FRoGlhRr#Z&bJtjUP8gXiLnF zd8{U{)f}bq2!#b~mJpy6frM!kM2H@&Qw~vRj3uHh;U^Ff5jzf$HmFggtbRKN=oRy@ zZpJFw-~dknfH^7H#aI(QJ{xl9BfiOOyx%C#PghrFC(n*I-+%SqXKz3G)u-?L;TIqN z;g|1!`^7t-e){hHSD#ERt%j$%!DcTs-4C~BJ;e&2O8ZL1Os5;2Y}w-()Eh<|J_E;_ z1P~k~Fg72Fdd&twf(F(85+ADs0x5T`k?;0e#cIwM^tik(JI|3cl_{i8Zk!&Tp74H; z8AntG<%DKpOf!B_K6FVjE`xNytNi(lzmP{=;)p>xqMy)UIum8kSe4_tkzw@}IV?4j zdIh3XqDG^gR1sE%4c6N&GOKFXFgju!9W{+TjZukdR1QnjR;j`|p`jIY%*Lk$I>Z?m z>$u4vw^(!}2|`N{YLZYA~6h+9*Fc>!1!-=+UyChGZ!NRq6|?DYet4NqnFfULmJtTTB*P= zSGH7H*qlE;+dO-9_3o>!gWKixtNG=>xZwsdictdmfJi1|j#@P^ zJ>zf9xF=ig$p&A_W6^*);2dms8Q>8gqozz3)Wgq7FPxK|dtv;!^Apccn8$RCUdX!B zvjsBBC~UxE6naW)Ahc5Bgc4Fi3}$3)G7QqvI2G~k7>BFz(4og6pn z>^7^%%|w&-c$!P*oVk*(R7tj)#o4we6&cgXo*TXNgR!A=a`^?dZrBXTFig+dVJBdW z2#a=zBh?rpG3tg?ic5oKTjSuHpOXy95rfiVdVYNPyO+*=_u}*48$LGzDYU!=D2ovU zWzgX|gtgj(0v++vu`rv6(UA}t^ix4E>38C8o^XqHzmsx{sK^r@H|}-m?Zl`_b6F#k z;jqR|8#vMI@*(~J67a)*HxUjp(I_2_fLG(wAv!8reK;Dx!3We9rSLFLhOJ`K7A!hE zX$NFCLw2hp?(nCbsX`!@^q3IM<*|#GCN5u;4xN{cD=^d^iMg^_K9wXQA=vBGi>y|# zYgkGzP+AVxa8?a(Q8JK>G|F(jn$Y78ob%avFYS#Ap^P`24+nE$Z#v}6#+;R+qdw(m zw!EEAUgwb~kcM{mfi9-JdBe z%~w~JCf8O#7BaK9Gqbz7e067GcV%^Vd2OSAxV?DyaCN_5J?fR7-UExQmV8^)&~nDz ziwRR6TZxBfqwGwC?uO~bq}Y$z8*XcbhpGb9^4b8vD!d^>C;`uAV|Ku~qF@d|V{M;x zE=*23tzN^p-#FeBq1_BOFtEwc!%pTgK*e0;mHFzc3$>T$Dre2~MvwyWu$g_*E#I3i z9+qOg00)NM1Zoahp*U}!%B0h=kVd1_Xbl#WM6Cph;X1wc@}&zeeE)k2$%viC-EM(n zNux$dBJk2o_hftfaAoCSX>qgLUaFL){edY_oaC4~gUx#Fho$(d%k6iz=il1wzPi$Q zd8zhfvHH?t_3hQhPo4toS9kY+d;j#;4{m&Q_v+71c0N1W`SRxRFVAlO^1=OY?mzhQ z-ax;8ap(5WZr=Rt`teVW_kVn_@%Gl@%l+n~naPK}=A*gxS#SDIuYJ2edv9Ux@%q}E z2YYWH?LFC8zBb+1s}%PNxr0LPS}DJmNo*z~>+#Tf%)eKNAJ=ntJGFb=$(x<}t$ypy z((KvB%Dv6a`@4JBSJ#$mlWCXNgHdjbbb(BXX7k=aKVRHwwss~PTjknbwSG7`xtYx@ zg+u+Ie>UK4dBiGDR#>b+Kv}Ckjc7{*Tn7(HD2*F3q&dgHDbfHUXNLjF5`-aY(|Hkv z(=6rmQbHxc`=44Y|5nwDE_RUV_ zA6(!2$-6K8{DW7&{OMbtfBNo+KYsG~rIVd&8?}YmWP2vvTS?8XBxV*u^){c((b2fS zTu!ySLMpFicrDEtd54i3WXxdXfgBtY7VJ?Q9@H@jlQ++KQV!m0cLyAiSRkDa#bf?N zJRA;(TrL+YFix+7;26@yyJ9{gZW_~0jA#{CbP6eAf_ToINkwY)WUCXYO`3QXj4dO& ziE+cksBZjOKc9mD8oMoZBES|A|7fjZqV z$LlFdYqKd)R0=_IlW9mkac=C&#fi~N(yoAh5tCXJ6K2VT zX=2PI9Wls;G}6mz`G`qx79B#e5TEY1_D<$*Je)l_>mQ$O+<$HP_LKV7acOP0w7yf? z*s5%9Gb28~= zolZ#-R==M}rMObn+w26}yW~blM>wZ}D+j1{pGpzaW3%Iq3`Mq!-Rh zfPP(3T4h@Bx5pTFIRM07&%;Imwm2-9h(NT#!dhH0FBkTi1lsIiU?*c12#b@1MO?!` zO46uhU?r}XS+%1k^#p82JwAIRZI7nKbRkr&C!5V+DX$|eFN|M&ZtT(z#z)U7)K?6M z6t$@+(kSq#*NgewW&lXD=J}DK?_av~y^9zB`Ni|!yL9dc!xt{gN93?p2J0>TqCO$$WZW#}pzR(G z=LidBgDAper?m_rM}Zofjdo0Cg=8qE7oCtlXbFYTP!QM>pGt_Cv?HBzX5!9VkWGke zT41vxli`@OolDa`kgwH!LWDLDI#fjY2a_~Bi(esqaK^sI|%b|c_a@s7h=_fFqk+tvFFh)zy=ey}Ngg&TjAAIM_PeoL}s$E_F_K z7mt>vj{22@c5b^KT`T%G%V8j4GZD7PL3I~48|7L7d@jK*B*d8@^Q=ECP_CIE(ezl$ zc3qy*Ogb$yVX7UXi<~Km$$LS3E=1G>$f3R*Hc4i^=ysaDT5{cPL>_gsk7f&x2YVcs zK|E|_w!`d#!+N_MezBFiRgG=MJsl@mAk1+@pRgf$7dJOp3(%=o@_M~aXE5`khax$G zpslD?rIab<6K0c+qA^jlb9Ndr8U=z}>vnJL?H#YKUR_++o$YQmntP4bo&Nl0CS9kn zN!rr)(6^iU4~|woJYIfxulMFw`?a;^>nn}-Hm5&3od4={<5#!$e{=8JH+Qc6>~#Cn zqm3_5kA8J_^V_pK-<+L&b9VphduLys-TnOTouA*j^|PC&KRMZd|6t>t?d8{&yH9$p zN3*TR^PNWvGxz4ZcjkL%OY<*nuDyM@|K{HIi)%{{*Op#fUjfg&ytV%F+RFXj%wb}mF+4=p>>|UpP ztv7#tZt=9+yV>hqZ8q0(`Gfk@PPx34j4#Jy^O4Y8B-D%er@cbMNmn_%NFsn`i~8^e(hIaui&BGDja9ngz_qb%-uy1F^S6A3pP_pmV! zpYpq6F5Ux_&Te%x7RO*y6(Z$zW__+gI8cm-+ttk3jlB=ved(>Y9zJ>b_PvM4TSs%1 zPCAeeMH=dFCRU{<)F@|l`uJ$nQ_h8}`Di&4OGW~IpVQ}aiB18wp_pK&+`JYutE^@X zNgyt#Jr;H6bCG(zG`HB zjC#<+&#q+zon6o~A~+gYK}`Xw(R!9O38Ka2Ktw@D5*iy0eoA6CjvI7iTGa)~Fksjf z`G^uxnCu4Jje4>kPd13UNSL=;MaXQ|YHUiWRjDFP8rq^JzzB&-ELJ1$v?p_(Y9l@~ zUs&9nTtApSyuN(>-onY9_TKT-_CaIkptW~cU0d~(!Icl%6HY4ScGu#unM%An8ETh2 zlNr7cw#7w*8~x6Csq~ywa!xieY&2NBK6^aHq_SeU4(Jt{>8BQ!vr9{XMhy+Q3?hyC zT}0T8xao03ab9u$oc#H7vKP)tE?kmbQNS`Csa6sSC9P7k1~ka>*?^GoIkRq8QosSm z1gj(Lq&)&60(x<wM?LwNz9lrXfwxLh7c&#%yiW5jXGRGi#tk(s^igyns{?VsJ_~hir9m+)oms1 zM$!Rd!gQDtM??xKO9peX;-wOnfQxWhKz&iJkjWfQT{votC(MbsJr$=jDOWb*$pL_6 z1BH~gF5)eOJoQ0W!RK%EM_ZY6M>JX=a7L_-klmZ|dy-yP)EP{ug1U`zyIyM5Nv*}%I%F0V8MK9}!?Buh zvNo13q*C>%M13+^7iDscm7(>1m$SAe(ACZK^@jWU5`%-8k&&8-$)@@F_T{C%t(E(eCBNvAym2lcR~9?OazUQ&*kuY0r1J*AMje&(AF$9IPK-TijV1 z9Pg{CNp-Z>P7bv$4mEF%wXF{{EVQL2YM9xE#6(@FD`IW-8v7G;f85>|rzUGWBWZVM zz}(+N|r22gh6M7l!+m zCq{>QJI!XJOs=##-IUeoaJXoiwwO&eD@oHfCvCIattO*MuhJ%&(BjzG;qvmq?9A@O z*yhO4&dA8!<<$qvYcuK0SSYZPi;o2DQ%Uc`g@HFalP@<%pRNu*n(uu)+x2Rx@B7UO z0I**U7Js?6`u)zzcYDh}U)%WgXz$mP!@ryy19}y6diL|_$;abQ06W-uzq|Hoeg4_< z^uw9ayQ6*A2fA*K^xc{oxtJIM`gJlsdShnl%fkv3@WaTWxFE>gil*Z(eV2S#4=rXe>=xldW z?M|}QVFYQh8(JNPR@zW+RppHGtO?1RRn;VjI%C9g7ER8skD1f~En zW75(p1z?y>2|Exejmh0=g{L^h2I|+NK>`n%v;bg9AYqmw!+?{;tcDbA&Uvj_w>{^f z>ZBTQp51^`w_$NIY(4)#}0&-YjN)(2)r zS_WFG+tb0E+vqjW>2UqXQ1ireysgKRulLq8h8o+Foju{kCQCGmJ89fOe^L_<843`7 zhu&i`1#IS!nG9!JBKIx>)oAA(FkLw?HaRz zplzg=rh;B~IuWR?VHz4@tu4v+)=Xz-+w{!H`quc?PRqn3;ipSsPML&VF5y(j`D{ca zP)o#G1w!gHc8ktoGthR@Nog*w`0)u~PQ>O0PN{bJizK6q!wxZ;Mjged-EN)7gWGM0g#_OA>6#y< z5LfUO5KGE~G%~eaTbv>!>8|dnZyfCCnH;n+4jV&~f#M}@0s?|9sM2B5`n-A{qjCjs zcR=N5^x+5@k5lo4E1UJ?bKZQh%J@OE|-YPWn$E6i??^z4o%byj@OON)J{z2$0qV)qv`%GXFjR% zIq7sF*4*gI#xbX!jk8N7SIgnja&ZaJFTPTMTM)E(MV^>X@WiB)N_m*3bfhH{YfC1Y zqOO?L!C0I=N<(T8qZ+r98kbG)a~S+~!e`ZbDcosP+YKs8C($bSD6nRwK&>VG;p*Pu z&Y6W&Yn#FC6k!sUl*&`Xa&To5DBUvh^GW^ zv&rOkSp#0J!z@tA0l?T844F`w4M%9TE@(1D>}sDGcM_m#uy{>$i1NlAu9(vqbvwdt zD?pdWX0e;iRs%`tY&2aYK=Y^vLut{ zSkxMi*b`A_Jnqe;efhMnI^nB}1sW5+hDe|>9BK;3TOygZSfnoCO}T?v#+wS#aj!e> z3FktgOvoMfs!TerRLp{)a)GEqDCWx)d^yaP2r9&!3JD+5V>nHrPO}0OICes2*UPLr zg;g)7G#Z!5o%BX)LdiljRUc2)CsGZ`R6_~?%oX=22^`jG=<0l|e;_$LS~EG_G{4xo zxY9DW)H*lcwY)UAxiPr0);c{CZfYU|enZe7X>M$vnHt?#uj%Ool>{jFl%y8Yp(=;P z!uXkdCRxa*>r&yk&uS;?>yty>4Xfi_yHh=D0}T^}@N|7_xh+3l8|n;EJrV0rlI{+X zBWdS&&OMx>+q^`o9dY+Nh2_u9e$U?)A<^WK{4eRaUSPTI3;Iqyu^vRxP0Z;DI=$u<&cHsf^!)@(Hl z#skxhH4D8RtK*}q(^In(Bh~pVhO04^*5&s5{UMjzXQ8ZS((Lni1HORG<}iRa-(=}& zZrNK|I$2r1Ha)f0)3w;vyxG%rYi9a%XlOc>SjZ;UYf}>u&ulhud$JwS>&?#ii?zY0 z3q8-~JKwMN|FAv&)AscDn^WIz%>J;q{_E-9@23ZUJv;gB?CdWmr+)$M`T5UhXFmdf zog9CAc<}YX?)&}C*PBZ(SLUA1Pdu0$x;5N=W4QOu^w{m0$@9r^keky}cjjjSy>2hg z-&~lxwXk?+aq-T=+?~0Zdvh~)rzdWYjoca;xHZy$Yozbya4%>tMtaVMx-Le0?@kS0 zAMQKp@4grtx-&by)6;V_ItqkrrLE;;eC&8^e7ApKySI05uy1#uXK%3g+Q`7+*x=#l z0NB{+>so7XT57DB$|pvW!GVaUFX-&{TDv?{m)p|mCOh1w78g-R;dv8UV^&w2aFCn{ zOBv9dOU6zE;mWz3st7C1opo9zNg5K6_ zs=YSXR>%zvv>zO79bKF*?C-WuPo~=&BXv=4%1!{2aGO$1)%orYe_g?r%(!wjp~mK5 zeLWS68iE0wcA_>L?f?>IHw0aVfKBHy;10FYs=#TP)~nR}wZ@>u;Gv8z%Hg%Ox7M6r zTm1Uf`M>`D^5>7w7ZxU}YqNdBeW^m8$){2+h1%Zs>b{O-M{BgXDc0N+ZYa338PJ>i zs;jAJ*x+-E3A_ZLOUeV{CDzKN1R^&eG6F$OYL%J5NrT>MF}qxp*JJg$EnYk6rqq-U zQbA%B1gWJ0RH`tODkm)^DadG+n9Ne61*EuM6=juMD4B&)Se=*y(8~v+w7C#Ft)jhZ zw^v1biW@Eua7~TNiBLv`SuZncAPpkIr9xaHHKJMwk7 z>T)KxQSZ@6NhwFgFH>?V5ePCGG;VTUnAk` zq!Kf(p-IwjxA<*JlM*sOVlyP8B*5oTi&EuKgAa9!A79AIZxhn7i$s`n}pmBv0)A;o! za1aD2)IlIt;JiB0W3UHE%1uyigTq7FX$$SPdAtsX(_EAvzrz=x%@!kqA!4bNBNAR^ zvA!s)Ea&kng~C5qRQ$2L;u2pdCJg$3-xOx-@o+GoN;K4Dn+x$mmZ?cFg=DA^a6KJv zj=QsNTiEPLIy_0IEo`+%X-C{kCj;JeD4dT6QXyZ0vHD%8Nh{N;VU-HPafwO=<1(&{ zUnZ<9gIJJWiPI$NB<0rP$}B~U*$5O=VGaYyID+|5tS%fcgp&=?M13^X5Kk83zLeiU zo48UTODb_@GWpSw*2Tr1_4SU$rToB9puW+a%lm6=5ULI7Y!pS(PVu_xl@wSsG>h1qY99(SkDdh_Sq)u)}gqqfva*1ei_ z9k<12V^pU_(PmOK8IWe9y4Pcys*W$V)~^oqY|TvWEY5E&FV0O)rs65BO7Ea(k1Jqz zcnk(m0vH{3cfcF6ne8U6Ase)Z}J!@wt3-r7p4E zoW0gvI3I1hJJa=Kb>P+J$jjA%r}Le!mb$-Q8~AQx_}kTy_bX!`w-~K zPL6&%zy8blIiS}s7Z*RDo&Ruh`tj`KyVK)ujt{;*+EhI*nb9vN25yh{ z-BESH#mges)FMqkZd4F}|%cZ5e3-e#j&3!pDeRp!=%Zc%O z6Qg%0M($0G-UW}H8UeWr+R34d!LGBxj+lHq?7(Zt}s(!rjIB zTl2HmrzdaD&74jEq>UT{^BW&J8XMRf?B3{VU2UyjYOJ0sq^I-ou@o~D^8u0#gk3#; zdxyu|>N0{f)5b=-zLr#_wDOcrkv5`9Ba*SGvsNNwC9)1<#%@YdhP0i`x~Pa*=hLIW zy#fRtG^pHarALhbwlN0Hr<01RKvBgsVY4=1#DgRrwrB!IG+@L6K%0oLCKOOcv7?&+TQ+~VNPRR92u0qJi6l&V7^9B`Gx6JHRP_&YADJ}G; ztGnE&)2XoAlr|c*dw_MRXm@eYywi=?XvF3Mj{*t-%#85+4FNA^H=|a)(yUQ}j#?{+ zbTSQXFom3SCR{hz);`(aJlOm6!-sd@eN5ys1}CAmVFI;?uM(F@*_Q-WmqkJzu9lJ( z*koV>m%s!SD5q3jQ7$j9kXKa5Dl26bm5PcoS$UO$B_m{>m^)P$Q&~_h#$l_u9E`1Z zXqI*t@~t_HQW_b(#*ZoOl0Q@_MO8|GM~#ZMV|GevAQUE2OVhSkh>ZAD9?&_eRZfG- zM!;%Bh{~mUL}k}NMyNtwRtlF?z~!(;qOs}JPPN=D6&nD+_*|U`aS@iJ$5j&wH7AlS ziCBS27DDlq&tfG6Qa&5v0vd^xumqEe5HVZAWx*W2LI}EMj!Y<4$ssK!*5CparZ5}S zHZ!0XMY*Y~HQb82}Secm3l?!D$M2Nr& zjly8j8;qDvr(_scV_kA)xNUvBbA7mZz9X~PmYuFoOx8t)GTz>py+39J+?uNPPu2K3 zgQi+5((c#yMQwv|Pm9NpHc0AhNV8Yd&JbM@b6?VtG{ALM$Cl(NT$575Mrn;Hu^XH(&| z<^q7w;q>JB>eBK0^6}Qj>hw%)GGo)5>?X=?0lk7nuQrf`#b&hFb%vBbusAqyvN(Tj zdVFuF_n^1)sJHuCSI16E%T{y!K~K}kVC(Ivp2y2$&o-tXZ%lr*JotE}|M}{`GoW8{ z9dA~;KdknBy*Bu6Y4E$v>0b{we?PtU^YPvfhkHLBAOCuO@$1F)U$0;P1h9*r&dz^4 zJNtNk`t8ZJ4@Z0N_P1VbEInVDeX=m|V0z@<aD~}hJ9?s1_oSl0xGjo4>>cQ;PSM$^N=cew@P8Qp_ z$$Qfycc+K$ObuR)cAt;+oQ?IIkM`Uc>%TG4e`|W^#`N&b>9LzL;}=t7r{g1MlOrb+ zL&xI-#o_o9Lx*Dn2SYu(eH~jpEnD5qtF3hl4Y}#s#6)#$I2G)VxVwY)F2AkaV{Y@9 z+T5mU5{qkPQLQwhk%m-oM5hSrl|Gfkua+@7G;Yz=`0Vuzo%c{76F`?70L-gF?Mlg~ zFUhcgK^-z_0|r&m$w(ETmybXg68jXD-vDmnj767lnd1%$h zme~>^jn`>lJmzRX=cf(e@Qejb5uY*aBV&FdXjj-ZVgn=~L;|B&Y=&i4g~p?IXZ-f4 z&)~6Id}Pl+-O2In-tOSYSalc}9<8Bcj#P7sV(bbtuCS;;-K?}|BzlyG$yjo@Of0;@ z=Uf%?${=C!#2757l8J;Wg_KY!O9kHl>(HVUhT2r9U8C`uoGGs}?gjrz-`L2@ zw;v|vW=I#QqA-?}EyRG0z~vD4&m2(PbFPTEECk3FpM|k2(8>y|l8v(j8X=G(ol9>F z0B)i&ON2j3VFbP8zIqo%@<1Tx~=c)|_8>7Jj6Rr(}(mp+@EoWC;Dl5HIR#skB z0ZBw6SWwBWC}USt^4Y))KrJuA&wVU`hF0IUk@)fC$B? zSc1ZkLJXl0P&!C06XK8<6AKk07R=)-#F!q}8B`((hbQBr7PZ<{oMA@oHexOfLZi4B zGe-$qoYZ;sDwh#=k_t+#u;5Cw8dghSOsqGljHK2;>YQ#H?Xp^JCdz6wnY6G(c)8^2 zXP2*h2DtS_$>*2LKD%7{SxFgS*ym+cUsSO_W3jJ7VyRJ2`e}FEAIb#cIe#cld%|`) zYS#NjL+84Es=Y8Wj??m3$?A8_Rcu&~U8@Bg`>1s2UCZr8c zyxFI1WAvR7(?FVzYDLv%*-+TjAJFxOO&J4}BSe!4^G1VvuZ=nCjve(Rj(U=3eYx|# z+);btuqC=v9~=u1y$-a`fiI^#*SqSDnzDO^%wC~-zqRpNfA7h}=*8UB>B{WI*7A+* zjh)$<&g$xv*Bi9i{Z?DR4i3pcA(0$wY1tedJ(!v}o*qA+89yHBz1H1&-q&^B-+kWS zcWZR`(ZbZz)tTp8bFcQ6UhFJB*_?j7KK6WL_{G}5i^ZNdD?RU5d%s@o`(}0Uo3*hY zcjkXP+W!6I;HPW5KO7$XaC`#X>(}cS0AN7Bem=ea^ZAV*FD`z#I0FECe+>X^`_0zs ztM!Fv%hQkM$G@Bzx<5C1e}44-!ub8g$p?$m_ZI=fW`Hf-UtRfXZR_Fs_QTb!hszs} zS2mxoZa!OCf3m#%cxmy`(!#@~`9~{@kCzr6EX>}Wow_?W`PI_QS4&g(mnQGdjozCd zyEi|2cXs67?8vRjzUu&HW4$2PCkL-j4qcxfzL*}mm>Ie;H*#}s?AF5Mjk$^Ivt#Ge zqi0h?H)coAr-qJ3d-n!9wt8DuI|>U;*~yydNGi}9c6Bkf-iWi+V+Q&K=oJGPQ%fUS zSsdIn6dC4_39VATTO|YNO4x`DZH`)WVNx446TrQ^I@F^@eLAeTW{N=_Ag};|hRkZf zt$-1USTxDv7)1j^>LZjP<+5bl)|A^?9MBPP#{Kq8$W@;XHDsg7AYGdZ4fnQobrgmM zT5nvxcK_j>!y898?p@zI+PVMe&OiP0&;Rh-#}DtG-M)7*wm9j{#WWsD>$6gEzbzRA zEr>Y^=;by>JeGLCmMl&eu5#-kQpO|1T)lv&6ASgQ!h)EC;0X0l0aG;N>>J89wgrtg z7&VEM7OBJtiF87tR?Jfs!$F0WGHwOLF5y>Q=2Ttbamqx33RqkzfyyCKg+#c?sKOZPb%n61iXo$C+c&B-K5K+G^zzS!ox7B z(Tq6&mOKDmGOJ5z_aJsRP6H+d)ILAKgbaZo$pp=jkk0SNXz*Jar+ql>M;%3^6(`Aa zIf#Hm@3rdPq>cu5h0Bdnh=5^}Ok$GCNx9ml(ba7XooUY}o71`0Ttj!Ev9~VW z822V=cZzmpXiv@?$OWg@XR}S!0vrn$b3UkomNe8Hi6*h<~q-Et$IUf}$ji}aXvW8v7Uxlq2pI#d< z8KQP`(ql{etZ6S@9Sk-^!gY*0M#D6Ld>cXvk^UR z^4M)rhKhoZkrz}tmT)j!oe9^ZT;TW%(m=wj0jGhsI2eyB67VH~SEZ88Id_)9Xaj7< zVG9nEDqvtbza2#7DNa>oOZc2gf1oCmXpANrBY<8KjkH2sS;8wX6R@jbQBkU3iIgz% zar`O-T@j0~ibMclWx`^vaCu);vHnA8>18&XBaw2Yuu!QmI-OdIsuGJTguDt7mkkR9 za*0%ps7MWJz(LWcw-S5_PXP0hsc>6UZnD2=ZL)KHtZlWgX1P5*Q;3Y`{gbu9zT)r= z`)JNPoN0{oyg55|V{YtxVe;1M!mahCo2yILmlk)&Mwa?|r#m{92M712 zXUg1!1$)_7*FE)o?ZS=oc>;16S|6!%~o0Z=0*GGQXn)!Ke<+sC~-w*eGIXw8~^z<(m z7eKrKy?(uZqMJ>OghH=nGoJOCJ5o`1Bq@OW+E;o95-@XV{TkJslPug^W%n0vM{{}kL@o4vm{ zaWUR^YkK(J!uZW1x<>9SPTX6bzP~#AU~TT+$_&T@aJSXjFIQ&nE=}E8n!3F(aXK}4 zFxs^<(7IM6S#F{_I+9@q68>f{l_OPIBbp|ZF`YbaP{d5ixCv!+N)IM;A`&Mib*bh4 z;>bfaFs)AuT@qolE<|d5I^3&8ftCek=b_9V;tS8ymA5TeDkR3kL_QZ{9xo z{jVQiy?V5}w=p<6kZY>*=MvU<(3)VJnTS0VvLzWR9wZ}PB1juUE=$yBP5M7g6z;%W z1}TLL^fDGEtdO&`Zhie=V{LD(jWMf9sXMG;GL+IJgtT0tmM17qjUnKvglt4mF5#4l zi&rsS;Z~GzSy#F2QUSjlf-5DGaxqj~cMyVwIEEC@LXmox#Q?bFwGtkS!Dk_Zb~EF& zhP_q*u!KKY6A#rT{kcde9}6UdPR3>RISjN_X3+5v1k!3%4yWE1BK%REFQj(+)h>{L z${obOwh#DaQ11_rp)eVaXnlTg2d$4$dHjgWgF0Pkaq20X#%;l90=H=plTt=Vr3T3- zfJr1qSZb1D7F=W1E41j;?AWir{aDvrtF;j#6?~ajQZ4|RKxMzK5+Gs`23NrBE5hNlIcMuUscIJ63jQcfylMnENz z5EqJ7kWek*tEGG`Bql_EK&@KLZh#DGHddUq7Sihwi&g6?UbL(CyUB>po&`ZNpPPlQ0ii2`W;nFg38BZxd&1x#-_GO}xLONIzwWWMi+Gol7t+{|b7qn&lwv69f z9dO2Mh*4R=FTYZ8<%{yGmn+LEIF%fJRaxcLOC^`f*enqu<13|X1;katVq7Lx$wjzA zs8WhGD6B#xYE+^{K;$|U(IZMAjw-1PlS&a-u98V`;7k%2fyD~3NG|3{ge;iPf_PF? zZlDMqrBUeRkVcGAa>A|BIF)j2~5F8{IQ@}EktUgfgngw7IT9Lc0L7As!$ z91DkPvffP0?04D%E^mx+NBpjkH(i&lX(`mS73#YihlEY2*ocTt7)%ytf&ug* zf=+Y9?M(T>XWEwy#~R|P=0q^>Lrh4yh*c)2tQ2vo#KKBZarH7lDNsRN;_)vF1Xso4 zQV1#&Lseo)r3fw)h%T`>pI1~>h(tVvOrpk&PMg|H3KR+eFkoN+V4&K7F@;i#0S1dv z3GgtU96U2iAQPmr@%HAL;m*Qbf79wn>&8&SN@s4iAwE?XnyC+u<$Zsr*F;TVB<;$R ziWWD~7jg84o$cVWLnv~j0vK3}N8RL6bwrE+U`eB-$%zjG8w=`NU8*q3N^1o(Y1?MK z_qZ#5+?zb=O`i_bocGt9^;DmCS6^$2?bd~M3!$Udc;%cjhQK6 zIk#sf?#ztdn;ZRdcI3_^VA#OjiLv`LQ;!xGo~|rB-&lCMKKIqi#N&;L=UZd1Hiq7; z_Pt;0{d&Fso3;M$*9SjtPW-q%_shXL0N5|rc78j$_WQ-fZx=UyJqPXe->%;%>euPb zA5Skno}7I=JN^FblM$=2qx?cL{_dryn>+JCvV|7v@$n4O)M+uJXT4Xwz>IYd-LTE z$i|E9^`{#vPc~MbZLU7uTz(36WOMPw*2=4$wdY$akJlHTZ!bUJUU|H=^k{4G;l|>_ z_2sYD058GKmB-twPj@$-Y_C1uUVF5?`qjn~$o=*CyQ?#|mnP1p2M@=3w)cS z&_!S&6CMHXWyHdyI&L#$TvXm;2W{F#Wxe(cD45;Wq|+RyE!mK(Arl7ZN)#o`os0Ub zReKe{lEU)zgdTk58XGJbwJu>B|>i-oAYb0JgEUwtcw2w7=6h z)EB7B)7hjw6SrkzmSor*XN(cADdIIpy;Ra?NqS6ChuUYxXcBXh3X8@Ru(VHfjBSiX z>ceUqsv@QKfYz6^>S>i&2cX5_Vmv@EmQrvmO`*6Y zlCZ{7WH_p40_uQQ>-TAWZZ+^Zt6FN3NsCvN$s~jn(o4j8nNSBXCPA#2%BsdpDk&~w zLf(%b-wup*YOHG9tf_!Gm-iY>tA(!vzWpjymNN_ALhTMU0GQ9C^ScdrfjG+8)^q_I@FvF6cV0%*dBGt4N|s*&4alR0?9NAr5;lN z8mScum0XT1AxsXbl|r?=xGE?r7NQV@%cNQ*q(#L#4AP)dEiTvNN}~!lsk9`fCgiwQ zil}9Bm81wT1d?JB2`UjtMSNJuEiPFpsFDczatUHkYbhNuD*p7vi-Ef;NE)jtf-UfCvc`=$FE7P&$mb%j`%6{nZgyGQdmpPK1I|KELEst_Xy{y#T#R#o}@?%!XuDBDfp?tcrb^&EbHT)~KWW zE;VHkD5X4^lp}#zu&@deaAgojB9P%qg%%SiVW}2jN%(B3K&i(giEu+9+u4-ssZTET zH*F8MthDFn>f@{Jnd$n_bbT10Ydq(jsbjhsvfifZW+-4_{ZV&=%M>@rGiGIzTip@R zxB0Z45mQf`iWA}n2inKz`vUq-uP%&oQX2kj#<5xNJMKxG_GgZIbDscKchO%s@2)v% z%UtiSelXhnY^Lk+Wc%~Ep0~>*Z&$`&FOI)hn0U5032f%s%Iw4W31Bl{EsWoo=)W^J zdLIbH?AXCT*ZIWot-0yzGn3EuHec^=fu8-*^4z_d(K}N^cjiWJ&kWs~8n`nxd~a&v z{@l!?<;7=fD=#*do~_S4-I#f{Iq`CH^zFvrJ3z11o^Lk>zuy}9xHbB5YwE|H#h>@q z0lj{`w)fl7(eLNyzXNog0f1fnc7FXYr`LZvzVXA+`NyNv?~abYKRN#H_|udeJA3c9 zcHXRSyaaz`ZTZFO^2^nw=PUC7W?&bdug^VOUwFE)_-uRW`OfmQ?X~AcGVQ+H-hZ*R zUu<^|UV`lHzxtHd`^Efy`|ZKO+k^eT=k?z1tNmhL?rpu;*?hIT`D$A`Waj^69aPQ^e{)=mS&ky&W9qvB6w)13v zOblX_${KAbjZy|$Vl zoua9*tvCx}O)QXM+{u6|8}?;F-ug^*w5N5ryQQn8W@Ucl@t4Oxef{7ce|++gd^{>+jw^e)sOhhYxT6`j7wcw}1P$+m9b5n;YqD+Etx(RA;Dk+>(l#lZ-j$ zH^+R|lpj>*WP(S!s2pRa8<;+BII2a@k_;Aj#P*mwN$|A2)m14 zD1>NR#6_q5u6)pu4FU_aCV&h2?3sYS5N8@vzUrty8}X(i_OMTFw4eB#?)3v8|tI=jgDki>j|qo48g?7aLU9a$WW4sW*mvUJ(0G=V%Aj5k_ell0V3eU z9eRZsk&}R1Fr=4?wNf6~m*Va~ut7<9%^0PYY85DfTkNKt{gt-ff{Mblb^`S096%sc z$Wz0JLuU**i~)rHfJ>k$Qh<$! zsuaRXxu{$Lv6T=P5%Iy3pgcClgGnVBrEM`U6>|LnaFwruD`w(rmk~mRBDqkC$>kbUsYgMweYb8Rp zxHvhX8iKWQg%O7dT&%~$TJ+NbC>oVTr?u)VE`!r+vN;LDtilOIhD%`t5-LPI8885D z1((GVaM-YbErob;nUc^D4ijM~v^Jdd0eWd|VV#zi$w;2cDFxd)J5nYpyTmCi6|e+K z39t>0R4i4?9YF_^3I|hhUn=2>L|mbOKje$VgV98oiG__03MdFqAvV(1P$pj8RPe`x zpax;Vyw593F7v8%E-K!bZ5(XsoM>(ys;=!!*LCHZf~q<$j7o#SJ4g|n07!eNwHwQ3*56w*Q zL?^S}W~a8pXX*}8LrHH}$XRRGM+sTRqAWOZK(AJxrXy?w0E-dACbz1e(f9haZEj5z z=j9FXRKm917&z*R9rvY92Xohl>TV1bZVlJp9IU@H()eVy>&?pG`}NURi~Vm`2fyAJ z|89Hc$Nj~R2g~2=F1_1c{ICNyR=(a_{^r{9tF4(gyYuf4mR@hqKU$f5xwG^P*v8u2 zldYwPE7SMp#_xQh*X`-yd-G#oE{xw>7{9+T^>BIq@!ArwmKR&AueMfRZqC2hoPN1E z_I7jl>y3eLH~PQd==*MS=;O}V_gmvX?#%wQz5LVe#xDmuzaJg^_2lIDle2#~zX1UD z%gO1lr{}+%T>ts_#!tuBe>gh({_yzY@$vUZN8cV4XXW^?z58wxNZ7`!^>vWf>uayq zR$r_vK3`dQwYl_ibL9p2AGViX?kvCDUVFK_{&H{o1pw0CF2K^u{k>QF2d@CM_7C1% zJA8Y1^!niF^|hn7hbQk3Pd^-_Rzcp#X1Gqe>Mt!)#rIdOw zxlb);2xZiwija7S#NrlJ(yC2R>X;b=ZPKQXTlE=0EVnK1wHJb}ngE>xeYnR8RII>w z>LR|nc(9O+)Wm{ysW3oSVF^jfB!uvD3{3?%C%IB68zxU#^q)?SaR2*Fe3(6#-tB|l15|)X%l@ME^ky8PO zH^qPedRZ6;6?MAufk1uSoe$Gl#`!7U+Gw~r6KT%*Y7*{j+!|xlPMd->NVFQcUZZzd z91*`i6N}g9v#pK!j`ny{vptd31tNMT3dGA8j1qycF&Hs1VPhyrFdk#rZ3sGaJ~N;f zLg6x_0@A@eOvFLNRSI#rR8%Grmce|!N}_R@P)aM&DP(#S#-(Z?JOsi;MFN$Sr4&^v z_$*Y&#>ITS3?dPTkcqTHk)9`^#A+YjKGFH)n>TNN`gvw!iwXvLn1Y8ZSO{D$6;#NC z6>_Lj4pqs;Y$f=+LJk6+io;b4aHr0l3hrM&`Gbr^i?~z391P)U)dE5#(4%Y|E?4j? zQK3L1PmY0aDsw7;%Fz}?99mgD)f|AKC64WlYCn;~9 z*3oK(UJmp~fyCej6_iltlG_-aHJ}%w{41Q&tK6y*@R8!N zSYjUKvLtemXgU(k#)9dDCm#34!l6Vcl3>DdCYp{0;^2{>_CV>dJJ(cQ-_@9D&N9`E z#c$=v1edvGN+ZT(Lxt{|mZ7?qp_;m`4A8GsW5gOD1)#DJ@k@BzE1crlgDY$_!!-RuBky(JH09QkYjJhZR;*AD~sm z3GYp)+hk67LXDYdb2d;D^`?W~w3l!Ye7Ud+yz*iZUj{*_OsJ58ZGb4AOu>^Yxk`kG zpnMD$Aeacjgeb~aDg{afR|+e1DsP0b&^Du@ZsS_peUY8h8kJXjwK)F+whXrMY0XwD}ZbMZnV+*!zV6f*gE zpe7p`>}?tyY&qLs{O2EE{O8}^{L2r|fBo>)KmYRPufM)~|MlayA0GeXfBf}t|Mow> z{{BtR)PO4+a^ym`e8^E7an{B>^(kLt2DEf-jLHNpDIj5P9Ye{ixYU4X?K*fFr3D(5CS;i6bkc`zFvC|npQ=g19Wy6gbKx}^@7HUez z+N(oNc~4Eso{p;BcBPd>ETq;+S%YpS7fm(f@~!oCT`hGzo%xOqZ?0M&3~K{H0~0bb zLEWe6%QapH<|dU+9pcnrP7OjK#mVe-GJq}?fPz#|4i`_@l)-SRSX=@$5n_w9a=f_S zJ;qXq%Ow0V5%&s@b*VUC66>XNv!M9MCeu}frCr3!Y1g3ZG?5Ft|1O5CSXIMjsS zX7al@N_mA8Vk2@crl?fH<#I6#k#I377X=-Tgpa|%_PDs1jqgFnMDGWP^f0{&u4IRR}&^o6G}BAtPj{MWM9FK`<*;T*Jns7HScZ4iRaTQUb-z8r%#RroxRX zi4v~jRhFEn-l^H~<)K@!H%mw;6p+-sJVo^D{R(P9#4efI7ML;W%RfbjrrQn`g}`uU0Y3| zC6mbbtqv`wSID#=O14~B2Jy=vQMptiwU{jNus-ZT{gjSzQc0hqI^?PbP-PrR58*Wd zk*$I_l_Cxc7Vs5fF(wtEGL8iLG&DobS0F+FEd&#zz@k(VwN|RtNz`ho29s$K!fv#B z?F2<=%vw2k7chwslL?Swgs2>l4A>ZMCNPpvS&VkZr?rycfdF7?quv{EcXc(kH)q@G zBIDh)Gd*>at?8+jZ8Iy=(o*LV4JEwn0xvg+;1skD8wDRA7MJ{!*87_YlK)o_2Z@gbns zT-V!;k?*g~{%}11@o?^&-Kh_oMTWgu8hE=h^k#MV^~x}y+N+h}SF6KsH^yGC4ZmC) zdb-&Ac)sV+T-R6g-S?(DzMSp5H#>N5cIf`nB;eM=)tRT83&3(-Y^}W5UVFB){(Nua z|L5vGg5$dKzCFD&5(9GQoa61BbIv&@pc|P4GbxFbm;*!-1klLnMkEOa&5TA86Ea8w zBsH3uckz|WRbF`EDp%PXFYK*1E-(DQ&6)45a@DU64=>P-j^{bgIp?1HQx)>VU#oY1 z9>4SJ*sT|1H(rch*%*~mh>it)f4_;4?y#Q^R8hbHaeK9@p zvO%L)GgGf-XJ5?ByqunWIX(Ysrv7Sr{`Cx`{%Up}e)JN4G&2X4MxZ|Ra%TF~?DVU- z>DTj6PN58&n}Yg!ZT8K4BT%W=-YhJ>Szdk#Sy_Iyy8LE!>D6-W#nRlbb&#{_Pt&7+ znHc%g*gdeZ)!X0KuAI9&)b({PbT;KX9=7-SOhv0YrIp2$h+o2kgyj4X3NT^9AOUEl z&2p$jbfTyciJ9a{t2*a0W*vsGULMp*V-{`JW9^D~PZpA=imCogq%-O(hTWxjpeLIc zEM-8ydNQ%&o!L?XzSib^CfwbV4@aE&e6TiiXZz)&jbBz?{bh0e#pBi0iL1BHUHJaQ zrJLVeyaWH72IwURyTW}1V3jc)uohzO?v%eb6YR|e`U7JHw7b$Pl&jOb9p% zViuXBCSSoD>I}rYqv`&1=6E49)Ri180c<2Tbq5PiV3Ua)3Z6~EbD-&CF(e$Sm_-E1 zXjlsw;o$^a9FGZIMmaQ2Q8(J1%|-%+SO~~mj5?AYYuxEbdqR~2pfP)*oIKtc8_0+I zQsMq=bg&q#WSzOV!fWH3G%~x%6!Q49iA*n`tH1Z`>0@7??>~1Y+u!3zL;+e-mqX?T zZdnkAUT9MRx;Q2=+bHB31za7Mqv0_X9Izj-Mhu&ZVbd`HR~D;-&Bn0V?MzTH8s8wb zCOi_W3dg6nvM8-gax`5ZP`!X`=Ccrg`p@afBE|LuSJAO6?>{eSyH zpqBu^S{amf4*Zx;SMf+9YAX}d#>7#j0=8Dk)~R)&fUlI7IgJD%70V@da7b7#oh;^1 zr5w6~!+|HMqcb&RhMY)2a4Zd3X6JBJqz-!X5nR)kEl0mLKaQJ6X|3!qmmCcCSv0m zlx7^}tG1Re+gc7|Iu2vokK$2(f_9lH7I4rBGoe5w5vUMQ2VkBhV0A>SavQ=_(ijRF zTf-EZd1{~38d19vx=7LKid*So2Jo+mfIW)u;K@blLbB3RELO6qayC-RMoXncr7Ky^ zrMl9Yawb#Bn3ueYR2<%4PbKv2rMo@tA!^hM3t*Y87cju~InG8&6h3 ziSA&sCz95*~BZt#spPodRBSl~t!OsO)ZQp_D2XqlK(@s3&>xc>czz z&RZvnKb-6wJy*Fsn7vkxeUtWlop64eat{T}pkKW{Q_dofsMvXnu-h#MfOXoDZnx}o z!kjbn%XVbYqd4JJ^*iN35iYN1U(Q$`^aW?nW@gW27cX``zS_HXrT0(QkN@?~=~pA) z0=;%;u0w!fo7Ky&?tSy?@cCZ>W4F%!dh5(fVA$>Rzur9e)Af@--8}W|%CV&j=w|k- z7yE%;YgbP`xqj~Ht&2Yn1GldJ`TmWctHVE6fB0!^_^)Gk8*(#-&IV0@=z#9r`DN_Z zuVdF=j$M9Jy}UDdd3WN{+lg=Ap}F*a^76atmHqLn`(rov$A|Z;_diUHf1H{8G&}RR zx!T|77k-r>Vszw z?tqHbubuzlMEPIII#)8M1b(!-XODW(i1e{r)J?FLMyw+mKU5@)pQExdF>dPl9nP@Q093 zIYa?VfG+NJcoG3`Iv7ZY{n>~go=j)llMlOcK~Ev%E`|b?c(gwkA1ow}7vQhN1~c%7 z630rwFnz=;u^a4(V63~?b?W%QH|I`%f9c$ft7or$-!;_dO!$ooB76oB|cr}cLPT;d~To!@HAqWHnp#ab4V>m2$sA7xC5OE?F zHATY1aOnWB&kRGuBsVchtsHU(huqF4Vz^`smxAFzNunwFa*s)DH}XverbbPXOYw*R zE8t>=l86XGE=j~Ci2+?4vXo7fvhWfn0iglF?$3?>{lES5|M8?8uiD#>;MxgX3Qf$Q zBTR;rO_y=lDuKkRRJm0uw?gmN*`j89%&hln`D#9TYL~1rvtrT7O&Y3z1%Uay0^+N7 z%wZh9iAZcF;X7yqnvg5i>y>5$xEEiEqKhr%iL?@}TWblLKL~J`sXm26o6edlfobT-~7s|OO1&W1mXDL=LqgkoyY~- zqkle^4qiY4ht^IZ;TTLZk5A?c2|PYYD5Oaw45^GIS8!wsmIMKLRhU#}x5XWBx%>{h z*QPS7x#%iT5mzCBz=H&O@f2dUO)t@d>>_%XQEE^K)nc_>t1zogE>jG?av?`FVjt{I zp6}1zKGS{gOlACh-`KgH;bXZg<>)s#?}eoMblBEuQ5*|cd%dQ#K@!*U%Pv`uN73)o zblM~pm-KAPTCfNJU;}R1kXPC3kfxNhl8JXUXCEC5Pc;l|<@?H$8w1a89sA|(>6Z__ zUax+?GkXKFJ9A@a>e^QI$_9AY{cm61z3^iA+>1NsUk;yt2`YB;%%86u`|A>ee%Y})RoPt@AhWCe>eTz{?w)Y$xHiFSKd!ve_y@!e(c)8_^pEp zbP3|enJNJ6U%78?Mof0(VkpQ)kiI8cBspreQrGh5ZE&Bh!UFznUn*c-61 zF|e`5aN@oD05Cw>tML)gua}b}@L*m|j;&8kY)(}-p`4!Dn4Vmpp4^z7-kh8HoQ>I; z^||RcK(g7{^|{){{Nnl?0BjyD=j*_*2E9I$Z0^ncJTRj@ux%uev&f416qlfFOORtyeFBj%s)&XPFzsybk zSiSdj?9S@ltMk{+-x=z$Ft#{M6eQz_9YX8 z+00O>c($kWLSODocluN%eWDy6EJ4o_$E9EyMEK$}C2YQ01PZ~DaiPwZaUpbsgX1w; z8Ds*FuC?nU=}0IO0eV3h%tZlU-cr%LP+ zq+qKNGQ!94*zM>TBdvvwE~IQ>P+FMOHgsZ-*2W^WvPkW0fEfisl5hnU9eN3Si_`m<(|L2Qv zSAwrYnrPS#E*ap4;p19a9XJ6E_{7$$_!hm;Y|;dLrg#Lgn|OK!x~f8surx}hT1J)& zXlfqM$Y-i)Bnh4(rBdY#j-D&Ciy0D93%TVmwi(d%RZIJqtsRFj*dsXn5j-AvLlH4K zY93d^=colhFP=^!b7<5atrGyl(d$q<45$Q{)o3&7vv`vpPtu{ZB2*!vgVxec z>%g*cG#QPnWwVu3sg>hQ8ss(!hJkG&v>nB_w~}xgo2k^()!E&dD5eAXlqZ*VWwOq6 z%9W0Xiivbjwm6utob2p8QOpeHgXjQs)KdrsN`XW<80vJnvkr|{!%}keK0~rAn(T^1 zO98-Gb|BezzH65=A_&USWuOJf*m^5K(wdEQJWqqvP0AB8N%h zu_yvI6%~dG+LdJ74YU}gpdp)`n5l6|U%9`YPtE|tj z0&bxO)&SV~jJ?Y#DLarpC(`ef_ShvEH3MLFHSZYd4^5s*&R@tsyxjfl*0I0bJ@sFZV9|dhhFB?_Kz1_}njd z&i;Jo>@Rn}{`uawzl?nU>w_z=syE+E4Zohc`}5fCU&e>u%shBG^WeqI*ss%LFJ{IY zslJ-6zMdU_GdsFIdw*kQcw_q3)-?RB>pRom?@j~2zJEXU-TR5}-%nnBH*w>A^~S;Y z&4cmV`_;Su1+W<~u=?NU7k{fQ{8n51Ft@Njx41vI^lokmW!PkGXR^VtiK&h8$v0yY z>(xdkz{$n|V87nI3jkXm8-G(BeKR%odV1{D^w|2$ByekMdTM)SdV6|mb7pF*u`AW~ z{48XrHn&}$-v$)T&8(xNe{<_|fUdblYU}e0>$QIcU( zFSV&3ryu-j>cR8s-G}#Y%v?Qxd#LkDPx{WW66BkF_;}dS<1ysT@|;PQHA)LsWhWrb zD2r;5IFu%N#3+xO)j+Stc8F*tWK=p;hzH$DN)s|`!dAWC)Yzcgps=f@4voyNlDf63 zkk#bSE5m+Ap_qW!l`vSzbQ+>d&P&wns zMBIg#Gat6+L*QPva?D%`>2rR4HlWM;^jVLk;0yF6(x(c+p17+J^cEvvTKUu6o#zI6 zE}j^?a=!2K+0Ji6Nz#v-kXW^WP&}ZU~k%APC(1!DhA!fu%{RacEtnWjy?@=Px=JSpPh9@~>)XR$-~Qpu^{YMSPlU?}bHu82DP?wv)PW!lMCy9N)Vb9K^Z3D(^O)TjEfc1u_8K2!eYpIohN$#$N%Mj{vzI+6`N%k zE)g%Jw(+n>D9wlPEm$_4E|;=(DxOIzw3wtWo6KuRoF=|S!_rFVN-13_r^pbzm`71? zI0imlNh3?iWGREL?91w6q>-Zu_#e9bI&X!yd+i22ofl0lZj3 zqll*!anvHNP9kw=G(LkNXwV0=dcOu3=1aRwpU2|xt?gt2ODrL?xnH%m{jmjtIf5a! zlAt*T&%@xERGD69^?Ph#k1gV{gwTC>ZDE%)>h`4q!Avk-4EqxfzKn^bv|_1P3>`}p z5I7pPNY7KbCANeSF^aHEd>aYVjKehJ!NBy%LN-yzgz~9iA?wWnz0$Ts+>wlhi>Y*X zwlr8M9nXb|L37k@jJutMFnn-5d3U7b33oYyCATXH995b9#z;O8%z7e4f381SJf7`2 z+f^CLgMQgUMr*(Xq9!vbpxx)lgiHxvppwu<{0=G!&maqBe4SD5^_!znXEf&W2JC!< z-%cPj;|NCygjNa#$D-r745Em|R0=t>w%1_M>=08Ht;<$}h-4X#`y)}kvsl~xU+M>GyC)TKtH%C480wGx9G?j(>5CQ_ln z{=)g=op--Ie)nAO$eHdRhDta3Qdhbn=TferknLE|aysGc_ZtfqMaix%ITXjj+FqZ$ z$E_$kWM!x1OwuwCRF@s%Zku2Lz~+=Ei2IZF0W5s-I%(z zK6!0@>iWja&5fBKHfKlHXGdPo+MImoV=rgNUeArK*Ty#H zM>gh$x8`ndHST3eRpym00!9@pIV1hr#=I0 zvszsT8+$OiF*d$|0<8LGa^lU*#QN;S=G-(83$i;uxBG8sq1^iKl}5SM&?$5n5Gu9V z_4&Dt`Pv4ctG=*NUxa*S7?e=oTv$Q}3G3iqpj^xU$^v9{d-37!^5dPQM^M>Xd9?j- zZTs=#t+glHYtObHKizrqZ0pH0lw=RrHXc5Fy}a;paqj2()SqU@f1DZn)6DqO$$Kj! z*JdxDy??6b)4bM6VCi;i%MMkiT~&5y3l?Qui>^hCqY|b}+B9*C z2I!S`7$YXNS0nN2x*a4mNKV{!Aingin;RPKvyz*qEb3LP&_->bz!J@u0L_C z6z<7*3sJ=rwu*&jG2ei2G$N*wPnEI>0tNs>RBm{u~$KwIvm1ctzv04Otp%xkTGNu zDk36E1QeB+s)Py$CuZWs46=mHkP8CY#21n7G-8$Gg~SdH{s_7G57@(hY;S5J5<%bi zjg|BQvlej}#CAPOFQbC4646u=icHcNrWDc?e3ni?meNQFjUwl=wNjx`AvQy|LV#xz zny{@$0borH32Q^Q3j7j-X{VC~dMRR&3yla*kAQyh406P-ReANgfDQm=2xwg?Te#>q z`3xj3`ODTLhuYejam24$JO0qz0xUd&!2!LnOeUU12diL;g?gvM67cA~c2mf03XaoWY*N(>^ z8kIK@^CaT_bSjYI20a7-0Du5VL_t)~d9xWuGU-aCeA#TQE1&JlmrryiD^aaigV<<-vu7uqgwTCj^P{tE42J(H0{6MRl>{Rw#id zt`P8*=voP_!yq@QL>d{D$7;r7h%CC^VG1V$@pK@W31ssTf5^q)F^{yjA3{$Z!+^_T znKYV&$5D%UT14wL%k5emmwXuC+(PPTqvA+hDh(B{f6cLiLofYao%D9jp>RxZ>i zAp)g>r;q}15uHM2R3dt*!lKeTOmdS}>#*7ae!a`7ciSy~dob?L7n8kRsaxL;jb1)= z_v^mllcn2(nakzy#bWq;#(z9&I}vmA`^~)`W3O935YP`rO%;!-%cz)uGtT+4SPKrDrz>f4Y0>*9RBYCogW#T;83(u~)nO zt_~S~zi{te{fD=;JBh!<+LXTk~URYSpdziH$kP zG-PvrY6H#0Myc9UBV9TFsD<9@p-UGd6(7c^q z+L@?rjRU}DH>)!!z$PX)CK{)OjgP$5bWc(Q9*VrU9z? zt@+u{`D|d@4S;RWqvci|4G7s5WMO`%Uf*6^2BQKvZ7-mRYH(}mGp@Fm;5aDP;`Y+= z&N5in%Ff~{Sl8~#E>wd$|FyM~`+NKiOM*w)^<$?%MMmAlV}T7kZzT9_RiEj$w3%S78S^w#{ z^LWrY6tWKntrgHRrzY#vtKO!P zSe1y^r12ZI4wc-e*91&@i%Oo3hq`)8wwOomF(^DHs)FCbCXl2|UBv1x2Ex7Z_&_pu zsw;oC7m(vE#t@4V8Y@H4VNVCWU2#t(;Y1Th=e>GU{=O8rSN2?~`^rG(Oeb(9)?bYF z78CuQiN2z*5Vs|Lx`0h=Rq>57Xy`avF@{ZSBB56(Byi}^5frKrz7oU>UFAz))7q&d z=vE%a;=k&^;^=gf-4aQ}BB@w78xG|H(auPsH_#qHxKSZiUjLlsRR5lYp(`a`k++ zTEI~7NMaV22bdu>Q;8jP8kWv%BLT)}WPy+&mqBY!MuZ?n6p553kbAc%zo zL_kwY*jgp3WgIR+z=z*wtCVz^lqMBZ5I$bO#`2gQ92&avo{g@L@8D3{*_2imshNQX z731oaVw-`XR?whfF&cyxGv8$38gwj;ilLN|Qv1szJWhCnADZf#-;dD*VQz^Q>!Pd=IodIEN- zN{l1pj<&ZQYVY{6t^KQx4(JCkBs{4_1z#`Rf%1!>=cPowU>9-LdZ~@ozh$XEMGEh0r$z5HjCT&~`Fn z9g5hxz4}gvvfHES@u*J4Ehi#|iVZ1Sg#Aw0fLqaJ7M4x?D>>Wv@#y@y%*v&%A8#N3 z<^I{%V;8q)FYnG>+nc}nu6F1B!tlY;y@REZ_lx)5FZ{5-cz1v4-u@Dl_untxe_Q`y zuRaVVn#H?2i+8u{_qS`K+qLmVs@t{6t=jZ<4VwO0sBF$pZq82t*f#6qn+s!`3->n{ z?rtp%@6_+?)`o$R?-~F*n7h3{1L6hxb?spC#`}pID8R;U?2p|!7`^vlZ1mIAL<3-R zf3Ge6V{zs8#ns;zAO2Q<^l{yO^guIx=M?o8CT#%FgX=e8#1HmcL>)yd7t z$<3+BR}t+4%QF7i6anOsah*)>gy1cHszA8bgSm@%FFZfh^Q^ z7Z!FF09`BFXo-^SGr#~{JIkxvORJ6PuBDyj2EdkAcY$83YXGpfD{F68pS*qe^xdOp zyN}j(9zEJyLjkt8_I&TjbHLcn<7be~wI>^oS6?sJUoWET68~D8{7ZfEFF>-X(X|J| zwHudaFQ1?O?lds$`%3C-=+jhkMIgW}vqFrvC_07TsPh`MP)2NKmr<8Ygo}M8M<%EY*bGsx zz^rLz;oEs6i9_kmx#NBD{K;JL96DtMe&Z=b(cW;>=gq}j*{H88>F-Xt$_ZO3;_Qlf zdy>AMRD393zSLX!uCI8ZJA1lPJUdW0IpE30~}V0d@$3Yp0M^23r`LYnZC-I{+@FKl{0`RR5v+g*03*r747HqR(Bg%H4i*!~V^O6dx&om} z5SCKP)1ebjL=j5Ib~d?6s7LHJ zAQ@Y$qN`;Txrm|=(KS*iiBdiu;o-%6ypZ>WqY#$3bV92dvFn*?36{q`0xu<@jVNS@ zOlrB^D6?tNO#+QF@G!nr%QGq2dKCbSBt@u78BGPHfF$LTB?6jUE;gH0E}O!kS2=VN zy%NV@9BuD7)Z7N%aHtJ)s2zem3UUnX9(oBbmCPoGsN6b*Q=@Pw4Sua9sJDa+)}XtTFTI~hk~(`XEG z2M&WFlN2VSKb?#evf)DBlg>C(DNi;N?kZ#kI{PmS_Q6NIH|EJW9VrKhSFjujc15Dy zkw`fh=?=#GBH^;%n{{|nc7MhlNJCt~oF~#1jCV(EA+u1$=SaA0373hm*-|b?#s`KO z+-9@K!s0WV+m7-PL9x5Dcc|YV3#jx;j*yKbW1HHWTd*BXINVo&v5s~ei%v%`vLIl| zxHt|O!z2*66tV!_a}16Mv|@!-B{iueIwVjTu?eZlR;#&xH4dvim#ON zm6Fc@L$nf^PDy67aZEN(tr3|_3YSwC@Vb)WbWgF^Tj=dCTsYJ7_3`}Wkq}=C1C|Uq=B}M*#-JdcSlZh;^_$@?qrx zlm}3UBa8R;7ooiWegQ4t0nZjj_7+BW>!0a0zFn_27&f(ApWUeg#-_IB!O1=!9N&T8 zTpZb1ytlV-cMq(tjsooc{O$euTL<%Z_Cdm?uYH)lc`$i%zk2ha3i@?>e{}d@Y~Px#TNpB?`9qi1U?(e>MvhVVV{MkzObh&u8uk*}6AfMu?r5y|+d?o1;Hjzs{N^Chy zXl`fV&@(t$#3O|EBX|tp>M$026p#N3gZZ)@x}+H3RugnCiR30Cseuq8PbLi{lHEf? z1LsclpF7!e`b6c_Q0aJow3PG2Lq?BXpq5evY&?rb;&SL>kU2GCurOt+78>mcg+N5u z9IZ%*qDv|?Abh=urxozDLat8CPzY&KJ{1vAgj@oLfnzZ`XsB$pkSR?BViTU&gd;cO zsqGX7hR!5#xak?#kmAGwoKQeQkk30{2uubLGLS8Th)qOz6qyJe zqLYa@8kxYLWTTU09EyaA?h7TNVYui_7W%nlS&Zxx^HG@+|O8#)sXYgt{G7eqHqVO3Mls8Qu`s{kEUMxX4JU)jnWOBK98l{a)Xd@F_$t36-0>hYM5mzo1X%rH@LTZ$Y z)MBAh!Vn5cd;!plZ!|&&O5t&7{BBd!Z;g6_=|H|SUMd7n_N1biWQc3jmw1?$n?H44c}iPwdvJd$ox@C>N@Gi=%HF z*n78d?>)E|(5wE#L4Ek3e&<7N7$oe&+^tV@cRozt-ml*HP`&k``oo9ukx$jJPvaAR zo0$21w*L3}@;{ay1HJxp<@rCCpZ{&~>4)0m5B0T=^~VSE58utM>`b8m+pNxPOw6uN zOuw#9yqQ4J^{;Y#{LOg7z^13cz}|p({mZNV-zr#?CnF;x`19g4S>CW^z7Z^XK&Y@ zzkT-n-H*?9pFY`ITibs6bobfw-RIADpFRVCZLF z>?8o}`PA6MkssX$Tut5tBNgmwVI_w?^(Wsy%vTz@%yPhh=Vq%Bqq$w2Fw+8g<#SVP7E~ zEBBSMgGI2b#EFW#ls2a#+K`K_L-0%MG*CHT$($&qPIM(tbjAh?iN0cSsJC#u zCsNM33o&0;BGj9S4wTX-0bS=$-MV`E*46&+&Q-oSQTckPbb26EN(;3zER)ni>Oe2w zA>j}>K&<8_Qd=7xPvoJeDmPP!%@j%#k$jXuIt;!A)&U1PyjCV1 zFJSTwGLc0tFo^j&5l1a#$he?@L;)SwxXcQM#cHRcSITW8Q`;!?Rx-VX$Y>$ZnhE44 zETt7sYbDa!NYqv$xs^W)m0^5DO#V4wE?cAG z=(J#9aEB0?%Pz8@>+JbvJxvAZ64KQYsuBSJQRPyCNce>*8#iSW#xx|ZON69;JJ+aU zs1Tk}DRCMVZnNBBl-jjQuMreo>M?Ll3W`EVmWgOeIYXnQY2;LugrSi$RB}MH&}>w> z?4Zw70TUhXI@_fYIdD z@uh4sn*=Sg+@$8nh3#~F8x7Y+CALw?7#a=BpcA+Zs))_kA_AR)DMQ#QrO0GKYz~Fn zX$<rEZYrjBK@CyMFg zh2-&kdZ?5;S%yk%FdgiP1S-K$PbgIO`Eqt|#_mZwyjfSMD-i7o20Pt`fR?A?(Zw9H zfJG6pX$YSw6^V3miAJu{DO_%=(`Ay$gmfB_NWxT&-c%~u70yI{T%@8ZSB_E+U@rXcTw3w z#cS-->g30j$qy@4r~|-0tc)D2jGzOF3nTjrBL@Jm#j*XxiTwr0)O*O{)Vn&$DuC9` zLUngx0;mRgho<^|aqK;MlacoeqX+eoj|=w>7VhoW0AP0x7KT66@4lbCeK33L!|a{? z$y@KnuYauG{%vC9V|DDq_{7KY>Ay|P{(WxYAN7@gEI$6{($l{$qWOLC*{6jkAM2=x z9n>DZpI_aZS=^bfZBEW^f`Ls=zp75W93OiDf zd!SZmWgZaqZ-zDW>%Rb22Zrq|)Iq#9!Os>KcYt1t%X`*@>jc4 zKrhHQ`S4K0S#}#!CRIcyOIS5sA=f}ElyRFwMrBYh_o^jeU@quj>y#e7!f#Lm%)AB_ zsF+zHHY<>j-JJ5fOYv}bDc99oO7!N^C(ES^$4cjq1Bd)w8BZw|8_4y2f8xSjaKLlr zZw3qJ`?6=cvnRVtrv~!Jy5p5xppbB+L$*}FTZ-pT4qW)*&Twsh?BVL&h5EPmhlj3x z+jVBpn~fnxHIYYeqv6{qI6RBYmhvbLAl>9b{NHWy~1l!xJ@#LQRy}7!wz%8V@kNuh1xnPSs|h-rF4~oqLh;rGKLxi zQ^nD0L>80GZWiek1Sa`#Q}b6xTK=P{1)aNUZ9fbi*3$k5z*u`*2bDk)}@u( z)gG z!sE~cJcgLhY3xI&GHZ+uqt>cXn8Ae5)4hQxj)>D0bJ>HgNH!kJ$N6$WI|aGEAstNZum?W3a~=(V%mK+?&z}1Gdg~sTh;H@oQu28$6Y0( zxW^(t0RXefQ%Yt*&-p%!&Vkj>WuIT~e>r;o&E!P@*zVkQ02nCI{^I=)OCujpT#cgW zTCN@}H#}_h!N-RWKCIqH=ff87gM@up8vC$3aj-P;0R`CP!Q%A6(#-zi6a+~2c5(9E z0-E=YBPizMO+;~^LaWFCb zY4X9RiHU>i^vCMVZo38z1e&s*wkN;VJ{Cj=vZ?(1G>rei+@btIO3H*TmXOpwB9~^ z^!DMyf9uyH6kv}5U{BvYdioA*?BSz#kDt7I`s4c_|MKq7e}4O?=X;PJpYJ?-3fX=3 zbQ|dPWNmBh;pWQn`a=ER0Gs-Ge(JBYlRpB$sv~O;?ylawRR;;Xd~W>0vHPcchL52a z6}wu=d|e113pvYN?FnU8^SQxpZzk@|L|y5yCGOSw?6FGm(!INPrf2TVOy8QG z7`psTxGSyqTRAEbkx$36$LD!YPzUh{hSJndZpDz=u*5bzxfPGjI8ekQvDqvz85t^NW3juG zB$Y8V8kSDS)ayhJn=;@6G|*Kdl9We~@F@z(=knEheEs$5KhT8Yx#RWvj(} zy$l{3O9_bOk`WqFL?wz@7%sDe%f@iI9UKmp&m~KRY^@x)B`_Q1E~nDx7Tb-WSS-DQ zty8en(l6wZpga;#CnLIa%$Q9W(oqw7;Q*gG>2;?4T0a<=-Wc^bvJrba;44Ht*^u0A z;24#Bi=JoH1C3ca4d55tOJcW*%|@X?%~v4=I_Ypr%O8)ne9_eWA1y6kc678;X&n^m z5iG6=4-FrBE;1=r}ALgQsDrY^u&>4-^ajQo)tYIDlkb zrOdJ3{Hea&sowO7?&z_uKz||7m-ALquCAE79QTwH-by0UpN$_cp==vSnCOrOqW&g3&^Gs%;QaBoEI*U}Xnx?I3FYNS@ZSfh}r6e_(+t`c)NAh*n7!q;6& z_x%hUYQ_;-2sp4PGG8Fj>kzY1>9Csp9tUV^)aQx$ z)MhnL!exqBbRnC_WHw<5U$tV;S*uD+L}q zEIcztt;Zy4ApDsZ)XAL{o%G#K^) zoUbv8h)#yh-rAqO@ow_^{^ae0$$KBC#(tZc{xmiBadP&z$@$->>;ITt{^$Iof6PDp zeeTik^J~A?p8Q^a`a8<7rym!dd{|i9uRnTQU)`Bs+MZk3nyatR%)A`0zIgE9_2}68 z==jFy`1%-1vdzh<&DohZ^Rw&qxh-^Hu(q>^PJIEW_Q0R&3ysRcZoR%!1BTV#E}=&n zBwN^CSlnG&+FM@PTYmU<<h99Y?()jk(jwg9-s!2b01$LDXKKi_@wWM^${@5$QE+T+c~==K~N z%S&(SwHI^1t%<+RO+X0-_UGA&r<0>k#_l~DxwClt>ipGjrZ1kZUN}B>uK)h2?*EUe z^YCt?N*ni{Z-FE(>SZ*NMk7sc8fnyfuayPXb6SUQ0~LP(E$?1&TO<2@l+z-QrHX7RgQQYBEDkS(-8OPg05^p%=+EMu)jT@8R%#p8t(2t(cjSD z+H|yg;KR4N-#*s;&WYhq-aYx{r|(_AaQf2Ow=SG{=kodYuYK{+$c6W>oj>)(**DLg z={$BQ-`SRGX$%(9Y{+Ax9k@pb7IOY`eyStvdIGUF0Ta+-OGTT&$UG2bi1g$4n4N9mjSgjV`U$3iI8I&5MQfmYN ztI=!KHiY#1>2Mg(OWrmriYCDc@lHhY*~3vfljgDsi<`0gc(aSP0!jrWU^s~6Vqy_7 zlXmAa;J>v?E<|KNfZUx5x~ZUx4v4_X5El-d4Cdt49tx|q#st?u@BtH4Yk{?d-R5zU zkpLMEVF7RrvdM_NewmFqQ<1-k<(yE=Itv9)r5tH*je-C2-a$CzlCiG*b=2Yk5KP53+xNd)GE(3rt$s?%xYnZtT{ zeX<@ln?brk5E|gfR-?+IRhZOzJ4E?tI><30o{Knf4~4l%05F}+2$B~YN7`Kg(ZD(d zo5(oiy+@pbXKbw8DT?;-w2xy0&O~FOb*QI!usK{xgbT4?E-WQ{o@7u;1YA+C7;*~% zC$JMw%p;||?!4bs@bhVxkaC90zHD10+aB|0Jr>%kFsf7r1&nAZ7Ipi@RHc~fYR`0a zq&nI%2Rj>w2f^pK_3%K;@!^&?j}#9NmX8e=j|{|nS|bN5iQZi0C*q+{#P1GzoFO0WbK6}U!ckV1v@nF3#;q&?5T0!=6q^eE zh~)MOF2Cpud3~u!yp$?-lyl8xe=JOJv<5P$3_7JjUvJRW8?@kBr`M=pBTN$DjpL&s zUpASn6f;fbY;!SQO!8hHrBJg2wvi5$Ce^Tcze)>|j{{nLtxldl1OlgmO(2_7m<%e5 z5y-2`Y^XOHlrRi@j!VQGL7!Yl6pgxo)iqY44aHbFlWi?j+AGCMKAVfjqMn9wva7v# ze7Nm{6WyPl=sI_#<7|J^$6bY!mFS_YR|Z&wA4-bH3%<9TqOC65aFjohk%psOr`O&a z#E+-BYe!0p?;pJL`Qd+E`NzZ24}Tp$^LXm>o%xH;7Oy;Cy!LYW#*393FIGohtc|=_ zAKiW$}1*PqH}Rt4DN=(DAf=gT9zizBj-El=z$jqj|CKV6-8x;iP7 zY;kHAU~Fk@7yO%r(Vh7lJM&j|XJ!4W#>Jhv3p;aP0)TB#pWB%_w>|m!&cuahQ`eqP zj>?4_Q*$pTW?xOr{s90swe-i#+V3-~0APOrbX5WN$CA7lCQBFq*uut>`L##0E5A-J zJ)BwiX=+BEMcivEb>-TFYd3xd0J|{?04A@2%})QkIQwvM?$_n{Uzg^7+pEg}e3E0~ z@xtP-bFz5t8Q5P)TU`2WX&J;Gz^V+}c)YyvXlYgca%H7jbGYy-M#bm&K&@=+aMm_yeV%BTVMZWtxB(z)t?p?ewdy9*Yw1< z)8qH2NAFFJemyaAcXH&`*tMG@mp89pT>kRx+}RIi&b%}A(W%J~-W>VI;Y)Azf7su0 zv>YGE1P-QrIf*O=x&BJ}Xh&mPCfb$?^_Ei2ak-ej9QHOPL#^pZOFU4CdK+RsKreYu zm`}=w{LQ)4(80FD$NC4~8tOdWS2^4@_|eHXzxepbxet$>|L~n_U!0vDJu`OW{i~PX zyL#o_YnMJAzxLVW=$Vo0r@uJYee%shSC^Db;4T3o>{`1?kHI>-S!IFhVXeZf(b|m$ z++@H&SoA2YM@)LN-Uu7a2#k{QOo;)v>j=z15oV6I@HEU35D*lWfLY2&VKU4}%t+#9 zfrfYz{0N8z7~c)FLr2L?5j7YpcWfeXBQ5X2(%S7Ri^b?bEhLW6q@5)YS}p{`-Hgpi zn?a7tI?VEpCX$K@;Q6~V0m3iV!SMf7H{;eqkQ$Xc8EP>?PtZn+HBgM6VU3(%WSwSC zw6dZO!S*ZlwJM!Lrk6a0SPO;@vq@fSh{yobQKZEs+I@br=%PTvno85@G?R{-MB3t} z5f6)cIh%(wi!=d-qf*{qF8WF(Iudhuy|BPq1lH~rF^}6Jxk#Tx2PGol0)e`Ct;11g zveZJB8l&X^1n-BSeI|pNusi%-G8#g>66_KjK`)z%dds;)d(+?eQry{)lN!sx)}~}v zYr3Z`-rkUEF9B9XN^#OlBb_OsxRNmkNspeqL_dS0l|~-1d5?hb1cvVo_RpZNuiQA*%%BJJfsKLS@miINJLDc zi}3k5(Ifa%u}EV>sL~K`Ybo^gv>YC8KRnbp)K?nr&kps*dpk43{h7g@V0)9dtrF^L zigs1fJx%%k77*a}=z4o#@S`Jr?+C_gvdA|{b zjSib#3Hq_50#&0I=tax1KKD+FICnJhuko*V&byXBQt#&&soiSFe3{`O3d8 zT>$`-4GaKmuZ{y4*y7C3OS8W$&;7Q%@Y~YDW4Tge2>|T3D#ofRw)jMzSDb&eSI4om z1T^dM3c%G0P_8|IRb%~$3@~}c>&e<){GWk6S=#^r`!B{oY=LVPV7s^OKD+(()7xL~ z+yr0Vdc3~*%i7w*)wN&O)_+-9eYmvpV1D6W(^CLo|D2h)KQnr7YV@m#k*_9h+#0{S zIeK~X#>M4}XBW^YN*%caL2;HT3ad+wq3vKo$VZ+Yn@1Q{KbvrQ-)% zdditLS;9gffPpoHJx%dIOETCJ^Ebw1fEB`0E+pmrQYjj2DWne#biH}%@ZiazzSD=g zP7M!!c=FBjAN7B9s`bQR|9dCjyL$2CksIe{X3o#feKb1q;mEZQu3!D&>eV;Te%3fN z_0}dN;`GF1vw*aJtO%M!Ot!BFeMeL{nwQB8% zp0ES=GBKpZ$s%sv>|`t=7{@taT}BEAKhjee&=)cy5n&IT2!g;v5*`Gjgjg+Jpsa45 zjs^K-6b$-ihOp2MFw!CnW@8*?(k5^JV(dl=0b$`WRH7UKhKxIfv?QjyVln_T~U2fDEZEV8~O~LwcC7 z%e`?0+)N=R+^n@3>mcnujq){B{cGy_*Oavf^h%x0WG3wnfwaK;cd8^WCAP~=GX{F252EB@)5Dx77Wv&Mm3^0;0SOpGT#)Nw)8~f!9`VHUU}~A+_WroAdF2 zi-GhCjiyec1v!u!w!;o5g-HVDaS?tO8E|1<(dv}Tdu%+*`rHnlR2fwJYu~7;tEp3I zl?F%;K5Y!?AZeAxu&<_CO>?0Bf9e!(sMIw&ZMA2=NpFJ`M!nu{3uaO+eFqC|P3}kl zsxczcbC9-0IR?^CuNe!1_okUj-_#+XghObPv=IT z&RyS`y+;8*qj{v>q79P(p zJXx53vIt_~uUf?AWnfxQR#vuFSGQKzwpKT`*EZ!?UEf+;-`)VMTHjg+*NrN-Za!Jx z!_}?rjayGZ0Qb6eXZQA3JGbsuLw*1-CeJ8t-hQ&N`P=%&FRN>SUcW4@{4~G#W?n)r5V?Aw{K`!nNrCP%&+yMBA@8UWbFjf*Q6&nEh=f&Yyj6?o;4lCr95s z{>fnbTTSUBrD%UH*cj%z3ZawzZKnsidKMMnPO_^9nWB%CTzEkg>IP~_R!G9b%arWI~pS;_B>S&^)>`com{fV}QrbGS1 zAH8?v(~rCVak}%=TP??qR)&Xj2fKxM91#SV#KG7FQ#Ob+L!=3ohZmcPup1p#z1^%v zj0#w%FzFQ#WUyH64hKmPBt@b?n;eM20k{I_;xJHlh^2sn0ES_Hx5MK$^Q=j3#7@Hk zO+-ANhLo$E67oqQ7w6JpHXUG6ek|g)`2<@)WHVv0kdzNe@i`Fz2gI_0yOqUY(gw*# zK3Ywz%`Dgv5de%JLY$CtJ2Nh4+U3aw*{}!Px%)NxdI*;LWl*Gwz_k>ur71NLI<|4v{!Wc7ZmTcoyiFTzTm65TOtmPhhb)lgSFX z4B~N`c-+F{4iDq-v51Fu1YLU#bs?5jAXX(}(V`ZZ#y~nwc-^SeiHaf@4vWbIz!*f+ zwMJ7tWL27M^(NasgQ*6BYavs;)dGGg83{3o7#WF>ktiLFx%0Vvf6w1SO?eNHux!v> zjChJscP=C*yp+V4P*bh8?hQrF8(`F{R~*o)YmMp}qjJB#9>f8iVxPKpzqbB>US1AS zm^2V(u}~<=VT_NrJMq0qLM5zKTfv)60{67LU5LOSJSo?CdnjkvDaM>aloR6e7I7i& z+>j||48-#Ba3&&# zB*M*LB8hSiE5Ib@z+IF(EJm}zWGNCWg}f;j?#1j9>dm;D`^$~pd1sK(KuV2Hh2bbC z5Cmm0nl$?rb!rsl6X{5$z0}j!I5f~O*qb}p9&T;$G?hFpl|V;Jq`NcGdoa@39`0xj zbu`8fR&oO^O-B!QogV6a_sHQ-P7Qx@qW3*OvO|6E4|Tsg(0;0?IM|ZzX-M=GL+v>> z?Slkdo+Thrh(PsLSZz@24a!Ks*-^;~6slF%Db@9Aja-Fjr%)l_B7u*3Ts9X6e3bHg zaj)AhIT4=4T>>F;Mg%^fs5wwuBjc)8v9G@VfKpjor>?0{?LVM+ZD0N0YwG`AuXtUh z+Najk8nha#!GM@FCX)&_IYWU=Q-d?;(;?;>y-H~|=xvA+f@)MMy~$w35F6ouG1Ta= z>t$luwRQ)1eE_}8G;Nd5USR?A40f{yfj|bTv)e2JLx%j|L+MT?U5Q8_6U{Vc^R0zI zCTM2N7S@Ui80MxaKTmil#!IHMzP^s^2Z!4~J9_ZF&f>9hv^VbRi95T)Y+uAVlyJXY zk^y!kA)GGzk7k^M5$Z^iAC9qK^cObH9=(6*A3t9I@Zsp0N0Xmz&0To9bOl%xu&&?N z$6r!yL8hySz{#2eA9DBAh0r<4N0O&RSe0ApO@)UUR zY<2qS%5-%x3?NN5u$_gG-MO*d+0otE>$@{oc7cHbfX!UknK{2b`}vdUPajWyx;=ei zclPS@+0o~76E7ELe_vSqV|MY+>4kq!FM{}UYFRe0sg;*gt1qY5exJYjYT@Rq#hWjd zZarVU{dDd2_S&t-i))YOSASbr`+08Z$Jv=5C&z()-TCbEug-q{-ItdhT)y_x6}h3* z!-?^qrY3%xk?S~q*=x5^1=z20RlQuh@px|j(cBz}CwtW#3v&QtTg!`EtILm9m!7OH z%dr9&whFG>YwJ7f8{0CvWQOf*-jqu=)^5q=^6R%{*{ZI1ZQc6n$*r%pZ{OX%BR}4` z^Y!+fyIZ%bi(xmv+LCuMZvuM#w6yf&!otrBOV#ik_dQGn5 z7`eQ1`TXMfGxMLn4*&*Y=FB@|@16W)pyO0a?nEVVs1#|7i@oL8o4qZk20DAonXYWC zH4$u2hPyH`5C^m18UmUOh^d%JP5~HXbb=YB;RJ*YGy2e zWJU}%IP6+07;R1UMt!Ybt27xQyMq#3UZ3CR_j`i@F&F@MhkODNjhk4xrys*v~QJxQjoxFy` z6?Qv7mzu;?xI;_eK&@b&*HMH5LF!Cqr5W7Oh{kF)*d5@;SHdO@Vlk2g%(JN13}NvEo67@$$yI_*(jwqCk+MmQ#ZBW8FC7b8L?@tE9jsHD^*RKCaEshzoIp(! z35F2^NmzK^K$7)9zl?A_WKo!qTFA1`XsR<=R4`I+wkS{=KqeK9P|+9@OR}+)Go8)# zb^k5UoMRFmn}@J_NL23Oh-y(oovyA{35Gs-5Zqs*lv`;i>-KBw4(L>`tLyh^l{H39 ztt?n_v5m&6H#%S-XCSkty^ISukdFu7QChT0i$>!x!3^$@oP=MrdwHOiq@Q(0oxZf> z&v-m3QA!D7Qs83(7j@!3#z0vVh)!WO)S2~a)NJNxRB~ZH9}x(GE9v+1Kw5(m<7dT) z;EKB3QI9L=7QG@RFfeLSLOPvXR$vD-1puSGtUDql^O0ORnka?cQ4uC#HKJvFbg?~? zEQV3is?w|KwQ9A|0Bi$1uGK1ESJc--CMl6FcJ-Bedh;Fa+0K?&wM5h1Qu1{)2fNxr z-R<$dgR$POXm@+0t0~jp(r~1s>3C<)KZXvSIeF~!cZSZqdFYc9C(fT9{NQlg@vc;R zE>MVivLQaS8C1Q+xK(#dcx9V;c% zP3cT?I+zQRURsAi;4L<=;NwU-6Fwo~WkZ}V?rf+;P7SxcKiGDrY6133p%A z*&A^UM5WWk=wO&ToZ#PS3LeY5dc(xgtT-s|WlmKM>}>$BU&lXvJoWk3+?P+6<>JE^ zt0OY5HphOi(rZt`WQJ{wfyXab$6tV08GF7o@pN(g+43a7)$>&Vt=Xqb(;%L&%z*2& z<>_ZKx@3C&#jwT6-G%X;`H`LZ(e1gho!ODynaevf7Xia|X1)aU+MfM%d+O7jsn2Dq z&0l{e(`)9B<%NGQt^7F;tZN?7>wl-0|Cn5QIkEg=d1`10=EojZ57Z{Oa!b^A%R z4aYC58$U0vJXl=#xf(yt&wdXWHa+?6^!PW^B)~{b$zH)x? z!rA$AAJ54K_U_D?_eTG5>XZJ?x7+e>H6@QWMEml=;g-zXL!HNaTH7;GnOj+zTiv;M zPd?dS&h!9QW#kj_Isn7cp{8V@H6Lp!CVM(7Cr=$d2K4LXK<|lxhQ3yJDx|T=Q|P#h zqy0`q#5Dw@!67YeH91K{auLB08;x?&Fc%IGZr)0xuzV1Q6{5@r(ySv8Fgn%4#yW!@ zq^)QE#`Y<>O0$ zfstCAG|`Ne7k~z70GKgMMhk{P1a2lttrgj?k#}C#!!X3qz`#(y*X(jZf(T1q2Y8`! zc~_x;1-vGnF!PvABpp5u54q`B2s~eT-3~G9aNu5ugU4x+!Z{2Kdi7@0euLo+gZ=g)Yho#_Nx{9lpvQ^yrHN+psB0T)l~;LRjpC0cUXxU~^(h#83-sYhisYY)~Sw4g>a$AucB>Np_Ei3Wl7~I3M=XKGqcxrI_T4`3R0w z>9qA~b&XQ>dYz))U@#MQlp|0lPWm`8?22Xlxk@bE5O;+{v%{=})OJA9xP;xn1}`%i^2Aazq85P)+n`BKmd|~@OLx= z=H!Rk8jp519Byqr-rfJ+vEef(2R=M@=*+Q!_l7G&&4EG)bCaaRI74nGAU{p3;IKLg zi+nr{4v`KEX}6+gnshXk(}h$B1ticbbw)J=sU?Z(>1poo?+hm*jK_^jF1zT&JW{o4 zMxufqF6wngBtGCoXjBPl0gSaK$Y??IumuoEsngYKvJJY5ICaQo`3NuJMjB2ZVV3^5n$7q5U7|z90Zr%X|nqy#w*<+XqCn>k& ziA4#KH{mvfaS$TS`UDW}s5g|2Ci3ZYDVuFfNAe+eT%`OoB47|>gLntx1=%jgMLk?h zA_JT&Dx~s(zJsO19i=xrN=GZH{&b)}) z;~a=l?^OJ2pB(+?)%PEaeEjR!*~e4ob`~zlwS_C!U#^V;T>XA?;*Xn?e{N3hNm%td z`C@hA`SRoo5G#`}mnNUdvxR#Ao7v;m?&6e;vE}JK1KV>i05Ca#fh~-0&y8%&-q@NO z-JKiVnZ7LR*X+fenG4%9K)*iOn)+mC`rOmG%TE_ZUMx)gu`>U^Yb*b|vbqN_dH!&6 z;rEG!mlKQ6$L5|*&cB#heKx!HbbkH$(#>Zpw*bI)*6uu6-UI*x@o;YW$CLqoZEvgrfB|$>Ycy^@ z-MstsraV3P?9RQXx4+)K{q+uTuWCHGEdy-pj$D7Z{ndW~CbvVB8zIUPw)kLS{-=fc zALnLe37ek!r(B3Qet$}?LcBYEnOZE+;zCKu_GG;#5$OZ1MBK7CcE>A-g2tDkO1H5E~O5Z zQXRQidoI>lPIpwYLw)UUo*Eu{t8egRf6wvWcw?IKIn`FUMysnc7!q)By48B%aYex{8q9wOC9*gF(=~6je zEEg-y<)+qrLsP2M5J+bnPA5!~Hl9bF@@-25{Zu$e$u)HzHtus}BWyf~1tlv;?!!(h zC1D8{p7he0Fq@B%X)hjk6LB{h7OepuTx}sI5@0PJ3h_{+U%(~K%26P*wD6cC1RRdB z2M97u;WEQ8I?R$Gk_wYtQgEguRHW;ShI%8UK~XJ1sBjEOn2sjZ@?!un%0Q47jyEwh z2-M}Wu)Idzc4k(Ckp$pGBX<(7H5h>~+2ozeUW>;IyS+#tg2fVCHt#7Gsj%N%1(;o; z%}xUG3cxQ_cAFBh=rNm_K~WwfMT&DV7~@dGy8U|X0fTWrWPHP5kXO76mVJ8ofWErR zTb?;Ylomt*S?i5vpk?4YI)Y$hz`$Z;Bkh)H< z+OMhKuLZbL?$;@Sc-0vcN|Q!m(yJ_Hn8Z<@vO6hMz)`tY!eOQm;9kLOFj|UADL3l2 zn^+UXm@EK}KyklClnwdWpy)~X{BfV#FPf~T*ALXb zw!iN68s+OXiv3EB-Uh6KaJVVV$M7L8lJRC6Vu?bO7HOSHuhgpa76U?9byj1&K`XbV zkVkfXonEhkVU@}JhD!7II;Fy3GNCvypnX~m%wpbh%HLM-wKo8Oxmy~ZgxeG*4G?jZa{&2v20LmTX@xIFRuVB&5N(XCUf6n)D9G_~9gXx)OZ5 z5*mmyLkVs;$sSJ&v;XM(_R9M|-Z=Ac?Chhd^E-=|pRQbcv34Dp)*rVfK>&vR2^e;B z@;?BZr~*u8n2a$QU=sjf&sJvut6r?l{x{^w!|Ft${G)2f09%}Rx-bR+_GIQdh@IIR z+f$czXD&XSzPK~>#rE`PfL>3gKHZ-A;@QGApm#5qXZ~EB|M&Xx|E?_mdtvF-)ZFv& znHOU-FDGW7kIn3kPHj)jJe^s5KEJ**xAt^#^Z6Puu&;L4zk0N|0o?1+()z=B8DKxm zRHqQHUb*+h1;DWT=P%ql_rb>>Qa?nfLp6OHvziVx8=thGQDoz+TFYZ z==HR^zV-CZy=Qmt@7%e!b^9)W*5liEAMXL|t9yF|^jo)YJ-K!3(dMn+WO{8rTv>gv z1l()x!TcNm7>Ms@le>0pFgwk=?C+l ze7Fa&_YRgn>S;LDk{T{Y-#l0b1~%AQ>MA4$s>@&iT7BhoS1tywJ%t32uwDSRVydf{ z>L{iTR`R{=4a37-M^6q8p6WSxtfT!5qGtnTQ2=d8*myZ@6tRetN=Q`N z>nz8(Qj{%5xMG;j_=tpLkGKH3K%ikK65?P9uz@E1PC^n8ma;K8DiU~@!=tRthueJ) z05IH75kBTSBcoqbkEx1b!QXx+SJ%C;&7KJ$n z#Nm)YYe`2v0&DD6n6hI|k`PH+AbA(XIY|gL)fzRxz-mqKej~ikU_M~9>^Fdb_Zy4{ zV7MLu;sw_k;5xm01^=whY(YH|pG?y6q_@&o7#jYYkn*|nL5Cka0Z5GkK53OO02K(W z(yXtC!PTI(!PQfJAro%LT`n%*qr4*F;n{#l`*<5K?=~kSDprcc$}!SMKn!ei+lT;8 z`f)122yt()lnhsrkxDYzl!;YR;WEHjEZv%GIM~=e)X_TFmTXE9ZW{bjd9h8elh+T8 zkiujF{MVtV5e&yHXLEXRPnhzD>0p2hdpyZNFdcIG1dUPm_x&}m9jFKNdZSLYPodVx zEm%lQq8;*L8=owM3e8CXFpPF+3|d*&G)lEetFyu?7&x`ON?C2?VpN*UK$gMB`VFP} ze`@MptEvCHqV9i``?a{qRS3Bn6Jk?FYAH*t4W72fKv!$DuPZsw9X;5b>aG+9nv4CF z%5Zb*k@of@9oe=F7iMid0uhkG0hutEaZ$-kBpmaa?cmwTy+4e&9ngyeK;l{0YK9D2 zvq`7dD)a`m$!;I@o%L92x!l)xyKBqBVG7UpNmSD&u}fQ|ls6ByX|tEzJ%$11fq%bSWf+jKZNRXpOHZf21OSs;Z%lo* zGxZ6;*zVkyPZzJhSe^P~eg5~g`9Ied|Gl#G=lsIUso58kv(Lw;Urf$Co0xt!3FtMy zJG1y~VdL4-%@-@TUv1odv2pj==G{FB0|5JFVfDw^`R}Kuz8kyo?d41Nzr6I#mlwag zc=`Lw*S^1eARbPQ|2jSKXlC;9^b|nXqp4|d z1sDS=wk=O8&VkrkSlC)#++JSVUR&9bt!r%$VE>&$+%qtF3h~z6Z4h_9-nn~!`|JDL z_wH}q|KFRar zJ78c_V|#ID^2V)+>l>q2REhWB7tek?`p&6O2Rc9KuDsWo zKiQlbE=Aw!uDpA=d#J4p5Y=DG0D$#3WCj~^;M&(9W30Q7>@KDc7BjsKh3>{;Z(C)c zr)~I1ckj{8zLN(#j`$acUx*T1HkvgPq{bp2 zjIv*&-LH`=vGsO4Oi`Fyq5}ac=wo96F6QSz3Z9FCV3IxnFngHi3n6byiYtYfe1Odd z*(`_vne+mNp<%%eA|lv9I+&20sSxRtY@7@*%teu59uG6954U-2fLm;o2LQtYjy-_U zVFvdxMjEX(={0s6K$n`p^$deZ9+Tj#GK^G$k(Q<{qF|Bdczl@4ueR6~IwL@r4uTCZ zY_yn-HjB}2K{$$yhP|bVtI*`kRXpV;e?zmBFJJ))h?h(+#%|(p4S_09v)XPpk~TAC zb8r~#BpDv(op>PV^n_eC#-X&paxW6tx*yQXXnDhE-fuM509GMLtpy;>SYtHR8ca1t zxE6-@n@kYLvGF9I&PO{AmJc8QTdWd~R$_#oRH2%Bi&|k(DJ`0MSgkbcbT*^G0Yf;7 zup|%~D~pkm%N>ilqJSc9OyYrt;~v(4!z!!JLfZnFP_!6Doi-y4ks*qSGK3eiaW=b1 z0Ru}kXHzY?OiMo1lufr4a-HRJcT?*?N8324jGYsk!QKy7Sel`~2<3Tau4P?Xahy-5HHw7K;`!>kvd?gbt`R`_=0G zTIB(w!a!Ttv`=VA@QrD)r2qitZEpeq3v{-G+M7ddWsvC=`kS&nrA${Q(HxVKPP+io zE(qmq2#1VKI}Bgrky!wyI%Oja~tMml^mtOCux(nT?PEuvn>9)@ZZ}*kN{X z4$ebGlb(D&oKE>!L2d!e_yjQ?+*BJ~dlL^K!%Iy?9F1I9j+>B4606l#HHkJ0J^8R2}bjLW(OYkn- zEwLc{0U?p{<%-dGJ`~IZLWPJB_v2o-LjwLMp-vukvY3m--7E|Ku(0Hgc-%p^6cpV) zM)K4Cs1S_1qzKQ27&^!^Q4bUKk`akX1V9L>NU)p+ABJ#4G*}MA8iGl{uu`zOA$G8m z>M6#$;=Y3s*HF?s5a9>I%+a*)PGjKxrtq<}cqq;dgvkB?alR-2&nxf$Jbw0<$oC>`sC>zccaa_QaX3=}&j&zkI%O z*f00 zXKQzM*Y9kt+pxr_`{CO7_t(aL zxH0y_*!U0fVJmW@`X9$e9!!k>EFaB0^2_wdZ&RbcO^!dBntC)P3mK3wc{&l0Y;GQS z*w%8DVXG@U>(!aV{{U?F=3i@IyH$Y6op1lS_02c(<2%5~zJBu6 z*T3Dl{oC!$UvA0=TK>9m>lXm9m6Zp}GQhr<3+SihD#Yo@@21A(S;Q*9zMdWh09zlq zyn5~8@}+aDUw*bB$Jv#0XBN+XbmR1iPx|CDR!+C1->k%sRN|+48&3_k5404!@`=7u zrl*(!;spS9w7p@lk~^4B^pw)Tz`9G>{>Jh^Yx7`F$Ix(3&u~{$Z$sliL)VGUQgF!o_6?=>b%plzdiCoH6~I`H!JxK+ zjFvRB4C>`yxem70!0^5*!1zSk z-_TkdI@WaT)ZcO~$!IZvIxQMhskLjBh^7`&fKb3Hz1;*6h=s-NB8y2J=He)i3wMbq z&sZ74L^(_}W~9(MtFF$ZG1y_&&pRWcg_Vt)3kjr;w6JEaO>e|uJ}SkVvzd+(;8vll zp`ov(aj>mvpuKsZE!UElVjdW`)EeY-Quab&G6D$I>h&P%Az1-Uq`f+i?8ZHQA(0T0 zF*f4yq(V~MX90hMQCnlsA5iOFt5g2HPO(p|(V++_@vf*Rlm+^gu5{*7l{lb?0#d0U zy$UkQE^IMts_QY4R4jDzFBTh1~(fslxe zv-4rMCld{oQiS;IoD|L2X&RwOE5+J4kQl_cXvrII2*!(nSlW|J`(g=CG$8>qYwXB^7ckeF z%C~1d*)ZvM?^TCVet82s?iGNDJ40@7)EkWZLovTU;PU!;kDv8~cu!OiLNwziC@;;1 zTuji#M7(r7$RtBtD(EZ5LZz5D>-A>cu}TmyESmRabHS!!yr+`w&O|$+uD*oS7ZuuE zHo&b9nj`Nu1>elMk0!<82saQQ-falp`s~ez<7a=HJojko{F9jry9<|g7cM_ry88S2 zm|U_^t>gG(bMn>3#P6GvFIOi3y;+~D| zefd-_;+WW790vxrGkare`ug_twQT^fi7%c`ez`k-ZhQQbC*vPIo;Fz!mGKtm(w#Zr)PnFJ)M}@otWL3n143Ax-+-7HM9D3Ve{F_ z?VaUY+pD)h$oe(E@?du1UsE&x92@`U+SU7)E`D?I(sx&`{dj%!hpVGMUK@XKee(M% z#(ump{$K=zd|;Ryqt_pf-}q$$#I;{0um3hV`etPt;@V1e^-Hcl+`V~ouYzM|Nid)xQ!?cTe; zefJ*FvL|1E{phPZzuuOcAO3Rl=EIGRpI2A+0Jdjf-vfG8H!^=aJO0h|*u9C7I}wZlB-k{b6Fvo5E5}QnIL^WFx&%lfv7b=vNUK7`_#I9O2a;d z{*8LgK0q?1=5?idpBfBSMupW3K!N}b03MbMU}3=|VfL7lEd;2%hb_3tw3AGTL|l~T z6_XO3l!&N^$DCB!O=jGDDdZ|8xMUP|aw><#;>4+#h(`oRh_3T&#hm0kmMJ+agNxYL?4sWEmZSt1*Y%QJyBu9A{;DJ&Gxerh2VG zWiUWi7JenW z3@~fG6;U7v__EGwHL+}<+?4Ab?l}2=|NEc)t=OJ+$9W^J(K$34yGCi#)|u4@40XV! zfl9*+W^=MOH;2h(`y#?qO4wL0?}UXk2&{8h^@L4rx9A-fz+{J$pnNpybf5x8dBO9y zsS&N(ssnyz6Y)SP7HiG`fHe)Y0f2!Aa_U`-k#5?6n(7Vm%werw-ZUpSbTAoI)$YA@ z@-{kcy-A}+j1WNpfDzsR7me|80I+~36?Vtu#=$DUd7z?t0X-%*%ghm)1! z{%9eZZ%S1 z!U+<^2pi2{6pJGiYGrJUj}I1Nnf6ktrzzV}N;ju-&B?}&e6cB0YR(rL^T|@mmkffx z-(W}ds7-H^pH%gEIUowR@f7K1aUQqYO**4e-mPfWIGi?j(itxM&LnxZ{M$=L( z?e#?kcbEz1ys>i7n|28?a7(jRC+(12RKQR8ybigzp927Mg(ZK)6HWSK34bu;4g^H6 z-{}d7?yw*RS;k8OoXc~J9u{OCbS%ImgM2zDZ`(+E+-bKzCk67JVBQA+mdXVSxkzI^ z+MWw{r2}0_Pfy%cX3hN}`u)bpJLSO1g6DYJGaMC$!t~JuKXbYV0POM9`A3uIo=ku7 zbm8*zrK``DuE>@28*FHYJK|E+SIGH>6a_uJF~xUF1*?RF)vRQ zR_o@U{%0j@&%plw*khQy3^qTqGYjZ-b$j~C_SD7gi7$4?zt|c%yEXdp8HnXxjv${320x-6_ zu(7qc@nq%Z*6PjQ7FQq6Ej^f-|JUTyKSxKuA07Q+bmU(*u77|124K}sHzvQkc>TLe zH-LvdxIXdV#>9gWa2@;p`nB(`U;pv?)t^VNJsiI-18m~DEMZgQkEf@e%*gZt0SxTF z(}_C^i`z>}Pga(;0KnE)Wq`@^hc|aNH@DVqZmr$gW7yN`MB>xC_jkU!w*`oG|NhQ5 z_jm5!+qwTWz}WV^Z?^9J#jsy)-Td|TO}Pu=+8)3b9?Z#WVBgQm^M^pc{sP$K=&jLf zvVL8^xN_}_)yr}P$I3atuyZ#7%+8&;a`MQhgI(`+l;3JhzS*2U+LC#Dpyg0Yez-Z` zUrzTGQ-cjzpkKo+rQ_XggDvI3rs7a@xxW!yD~CJVhr2sZym|QKdv6^)(p&Co$h4%> zEh&FG;EVgEpvM*P(Jq%BwwkPV8&2W8lakzGB;Zbj#dwg72Ixpgj%b)oCxm=nC={Lf zlC#ht7AsOilc%xCRVs;@j5C#VCgOY~f^fVNw#cU#t97+HW1Sv8pn>+OjQf?w11iIQ z6>u{B>uT+OonBt}vA|l|VF&qKB7lcn7KwyCIFXXLVt~xL0KnM11OSFdoc0js2=i!= zwSv(##M+}AmUQyPppXmG@qm`XU(>5p4wF5=5kS#G3?iY3%S!kOhX=PfQE;WhJnp9* zUXl#5MA*s1{dT__;yH*HAYSeWW%Ecj$)zU=l^xYHl$OMeG-aV#oz<=|KuSHNfgsp! zMQ9XZZ3t(#$*r?2m?T6im8Rjt2j4o~IDEL!*PH5SbEaZoOfU<;=O7gZL6jA9kzByZ zds(N8W;wecI?`!>r72NqN|!46N=I9!wcX;BYOSckj@2NxS_IKz4z0tku-O2#l!&bw zhzdmk#=svLX=-Uc@>b``_xj%X=x?!dz%Bv;sWc9a(ymcq2BpJLYgQ?(TJXDVP8yZu zV_-19mkNV)+o?k>N?5P981#0tg|S;0xjP@iV0Hn=C7SkgT##e@JO~4Bt}!Y#HiMOS zkU>Gp1>KoIxRNaOHYb|1eAI0h34_B_XH?6x&+?&P@;NlMGQf-q2+~?DI>f56SO6?l zR)YqC^mZGNbcg6>n8|1tFz z-fd&u;`aM{XG%dcvt^+rOR^=Inc^@s<#wpkX*-1!(xhQ#W-K!|X-a83eE+TQD4l!P zx7J=-isLv+oai~vIs5Dl(IHPD9mrHCD(bWTq)!JMWk#_Mg;Xz29sRFe5(AX z?=r{glJ#Yo1EqnS-x(krwB3T3AlycBB+S5O!VJ+i%7?Q7C+jC@AC9;kI>aEisN^P9 zu|Zj&lH?Zyfs}#)8Hqq7Y{Tt(*s8T!^ftTM2AOQI$pVH57BKcPTAX&$M{@~(s5G9e z$wtM3htZOnRL!CC(wb}{7fV*8L2xm@#fj(;NQWR=u@|^Q>2U1y1s5w!g3*)JY1cz~ zsa7h{Nwu&RTsEKKJSkQLNs4s^anxgX&=w16FcMlLsWXr|Euq!mV&82o2^&bl%Fs3t zECqR-gxBQ~XAbE=gp0*}fuP6dcYFPAx0hx)%E{su)TFkm!2m)8SdhgcJR0SkVpm+w z8K&JSPRR0NH@Gw(OmOj(Kc4i(VxEwfD2s8mNue&n*6`?IAAX@UaISW}5gKUOCHT%QGw1pxNfjk&L@U;n3E;sV&(?7z_339&Td{J6LSOx(ZRw>{naQOw=w z`ZUqLJ=*quu<704<+no(@5UQH&v*Z}H2ml4IQT|?txW&5H2vq?)E`q5pU1~OPfUCn zpZYvD`C(*YdwlA*$+<6+^V{R|+mlNlrWQZUEWMvwc{jhhHMhJuJ^ylI>gm|{qv4^) zBSWuuR<>SEjK7|k*qoT!8lTx5o_gLt0^-FWFtG6#gQL#}hMx5GJq6L*{d}PN)o|bI zk-m-5K7g^UiJ`5D;dhhc?%}_}*oLeNW8bxOe~K{d*rD-2Zg%-cH;HNCWZV{=IkqTgmtBjT;-wE3Xz7U(PSQ znqPdiu<&Ab=GpXw*imI>?BV3d!%1-~*o~3?m7(sXzK%rzt*(~E_U5%NaV2c7M^}!*JnIm?HubARdOiC>)?%DIb##Lq1N0V@k}aAt=3zwsD-^ zNy@~9F+_pDDj3$-kwS$oU!nj8rZ$-@xI@gsU@ZcuB|I6t0>7;Ri~==LvEkc zCT8;&Sj-YI&c;v$WS3aXavLC-T>=1R5Cx;aXaEnQe!suGk}D~vqRIcDf>i^{qSC_6jSn5yt^@vGk0@%>Xj5@hVudy0+c7xigS6Xy( zvrcI>X&|cs1u%vk3_=Jb<{=;oHlsGBS-VeJv|CxAfUQI{60JW_d*VO+{c286gyv%m8>T2PPKf}6O@sx|R0|oDR;@%U zEs_-&3~CIvLKcI=q6hBhFst=ixy7WjSq(9(k>H4>Pe@Tz%?kMMqo9d8U`6~tJYzV znG^~zmhzz9l0c{;NuqX?M zNGwdTDG$J>FYO6u_(a+tPxvD-f%6gsVe-0>l9;C~#MJt!qXGJSY3O{;dp7GnllJ_Q zWWNvNC!=KN;mqUii*H7+1BPu)w!NR}+@9|N@wHMEpw}O3V}Gtq?o8;-f}=>XwW&W> zCd62q{sY+9DgfKe-|MsDjN;nN&W5m^3=T1CW9qlXNf2KaCd8N>`#de~U;Z-F_i3j4 z(^S{z$xZ;U4L$S0PKk}1ced}`+rD>C1lYa1pYGnF*yk1=T z3b2=pi!T=!Ue3=w+v%n9Xd39(zgdX?JB8S})Z4b$)4J5%y4=;e+}$?Se7W(($=MQ*;;b1rxi6;~BbS9N6&6HL|v$;Sz z=}pGyaDWJSonZj5FqKS-nw2YM$|~Ju)qr6DU7pfv&(68K0-TgP<1r-UhXl^y@hEN3 zK5^zy2>`aMNCg7GOk4&llI@bozf~!V?N$Tpa)i8iB80|+aLfmVIa|i9SFL80md63LlHWY4OG`8tEy7DvUDn!jlxB-lR7)dnfjI*AGZ;?; zE&EndP^?qALYyxbb%xw#+F|veb`H~%RshIigS=29CNktHfqqGgG~!xUK46#}c$l(S ztJ|4B1P{ukCUvn9NSIb`(kTFSAe*?5?UT6`TdfR^3>o6MNSC>F;usBH#~-DJWX79c@1 z<`i5^$nEmF5I1IXLRy6h%cM-drY7mJF3f{aG)c^tdB#%1^;= z)a;!MnyM{n`3lx;Dkf z-GqRe3GlWW)u>K|=#_|0iK1Av(+%Ib1CIK~-p11Tguq?@CH zK~E$i1VWsfr(o2oHL8kLMf;@t@)Y?1mPX9%U{SmEViTN7yJKWld&kGa3&5nPb9{xNn7U0;P1p3weX`wVuh__k-@egDw*@aXpN z*!B=0+2r=v^!xGY_mi_9<`&+~FK$dt15mvf8+|g=|8%(j#n|BUv0&1j-S+m|EiHFjuRrX({;aqCMSth(q3+kiBEz=E z2H%a30)V|6pLh#mV&eTyD#!NRyvVTmg^vJW%S)eESHy|Lwe>GIZhpCS`^&96Vk*bY zTi|^A=Ixzr%XdHAx%1)PT>!A{yLZF_zPImuxOEq>?fo6Gm&(?yn{U?Ffqrc+Ep3SC zT70pz@M3BH<-+W9u@jhB%=g~d5QtkN{i}mL%L84@1Dy*!t@GV2%YEWp;&M;hMDwNA zU%qcRy1((e{ikY@-<3xie>(J2ZRT)E_((QZ6Bf$7OkLD_C>uRgng9lNtfuS)h}!Za z`^%3Wt~q*m|Mx!}J#q4Qq5^nTB$_X;}WUYp=DGL(iOm9SHVxzsof zJWNNCW`@?eoB$zGyKSFF``<$8H~CVT!A$tPY&=Rw!c08MWz$?P8?3HM)YnIA>ij@M zQ%Nl7v$n#x2Y z=_JIsg;IlfKeCWE z$YqBpo7S$C8kI)eig2XG2^VPO|0&qBS6-xb7>!Pw8Zjyyda)fR1Ir*yu~{KADJ6Qb z+*Bd(51p()Bh6RK^3=*am6*$}Fj)Y=3Pq|ZigfBCgGOS~NsU^C87PYh+}91b1NM+q zz|DqvT>Or918R{PRJ)6d#F7&Q(r*hSdu1}6!^8yGWO*V|62>_KIHkg@)7Z^A$O@RG zvFl7IP-CkR1!`k5AXXcRIA|OP_u`;Oh2ByK{ZxGu~u26Rqd4*e^V%ShuJNc z6`Rx$i-@ycIYCHrt^f_(OXmPrs4-#|#Dg&*CXn{Va)Ed*5QuxcF}Eko5IhD0fRQ#6 zYB4)Z8jDt@l9)_tfp-D1)N7T6`FRp)F@P2-woDO=&SRJjCmaX{K^WxpxcI2gp9wJ$ z&YKLRsx#@@l2lE$M5I?XR+)}eriDxd;z$Dln@GaOvR0O{u#C9(nPKcSEt-=Hx6qiC z!K^flcp|G72>00y}A&#jri@63QB zpqChHlmD|e_0Q_W->c()uTK241|T>2=lbODD-%0QWulq=u{`}nOxzg%JU{k%ZuHB{ z(3h$H-=_Kix;{*FZI5?;9Bq3ye0{6GX|wzCX7|O9W7mG0Zv8Oc@_xMi({%Tjncm-~ z`@T#Jd>ZZlFxa=%+qK!|yM?)lQ(^TpX0^E1z8CU+`v zhznr%$473D4BQy*TOI6P>hDu$vQ{E*2qNW*Ajjt;GgIp+T>6f}6e< z4F|DA0?(9CWffd)y?6g1p{8D_tOdcARL~_AOnJ4tx|*#h!&5OhA*SK_L`NQb>Zfg~5Y_IC;p!Mh|55!g$<;#(}YsaD*jN0cR{|aFGQ@F>}&D+HGzW z5=(VJCdLMNQ9MGDQHDwKblT@kdf@CTnVsW=+&T3MHY z!c43a4)Lx`G*DFr&N0VV=&r*40YjV8HtNg$O8l;lF$Tqv6G3z2{pMx+)<2?6cH zj8582xecUSN3a@{RKu7KBXlUPv_lFAl0$Y4g6MHf13T0X8vxUPYL8bP_@S=qXho4B{~hzuYUn1ZlKo4%?7#IB)6Is4vPZt#%Z%rFuYS@*npV;!vL-d z)bd@$#k&e6-xd__kx3OMEkYroL?Dn1LNp2hrnc$zc8dwJ8G+pZLqH5R(rN(#oXLeC zEP=Uk)=PUs9$#GWru=-=OK_yhsuM3pt)Bwq>Xs1O}43xvviY=nVa zu-dB0R~Ab2GAn5ZhCapFag20Q>cCUaX{luZH(6B#wKIgpv46KCvIYFO_yM782od;6S zdM^QniRp+_?eAtf0l+@Z_kCF!_c(f|N}07*na zRC_;9^!_%{^JTnqd!qBxB%oLO+ks{=`meq0xUe7p=^fnc8QK^a*%%soH8TEm zaOhEA?}Og%dp%tb`?{YD_C6izeKs=ia$@xL7NS z?j8WJTi34LYi@en(( zvw3;#=H`uCZ*Jb$T3g#(UfNn&db7IldU^f@02t7(xtXsC^k2(xJeU~0H97!dbwDK6 zVsHCmZ`;a1=PCf$K<9YN<-Uef!&lGrUHIwTp^6JfYR(<1IlsTE<@k}lQzzTLJ9erf zcRU+Emb+&}cY>rBYZXN0yYcRdrlly--^p zs67BesH_ub6oH3TR5O+3Xewq8_$+QlgF1?hhCM3P9<_R}T9c>M7MV>7*r6qzRxbxf zgJ?Q}rh*QjTwxv<7?z4+i3k+)*<+kN;O9|Hc%$cT%Gut>ur!rXr82AS1Y~rW#4_g=johRHiUv+pc7p;0 zWH68r%sb7DLka8UHkHDrRNB;1i&84WOf4}g0O#_x(tTQ~Ox!fBm*`bRS_NRI#3*Ju z6dS~*%rb*cV$#cOM!DS}w*n{IX$xZ4X&rhq4j=?6%-TINAYnz{iplg+rAceYY;GUR z1v#4&0)#WbW+P-aLl&#UVzBFMPBZF;P>%!SP~f?AkYR$fH|hz-`Cx(%qy!-*00Wci zRr?gOJcV+fOtHIIbUQU8)hf4i03rqEiy^_3d3-?HLa_~!I!yEvZFXann zeW9$^m-KjIj4ME4Zq(tlL8#ee(<}8#iLzL(DB?X>F74yps7|XWDlRN6E+{JA2eeaQ z(5R2~xG=L(tcrK+6C^0c&ND z3WNgI$YHGl)hkiG+^$i9wwYI7ZG&@_HMf4{Y=-# zS#cTc&(+a?ZcP5Q2GnZm9{{i$6Mx^F{5Nh*f#aX+<2xxHBE$Yz9sROA^83p8A1jkz z7RNu$kA9h(__8qeZ_eR(-^Y>e?cvT(!yO+++CjV@Y6b2EVx#-=>+XinQ*FO5bbOp_ z-Ws~H*?0NfP}AF?=BU|=nmhp+xR-uP?Jg`XNv z9%(*#yz$uK&L6&;zI5?kU*D^#>D7+T>)#(gl8MzOLkG)KhiY;M_Lr5E#)C;;raF_U z$p%UyLN>&uyq**ontQx4FV5gPn?a`67AjTyq{@B&=D3ToSE|`7RqrZPd{eCWuT)-S zwpw@rOJ(qEj?7ii6*Y8aji<8KTUF<)sSng0Icm4CggX`F za}hY`GPofV2f;xc54*4sZ5K!_g{Vm!goY$^6se;q15H~vx7Ei(AwQ9bxiclMxLC;8 zolfDwAmRq>{^4pn;g4bQ9rJp!R5>?nL6?50Sbn& zoShZS6aW~bCn!Z}rc;!w@O zx{8CGZ~7t>S7&unig=& zh>8<{K$Z3?igpzh?kX(aCzB~ndNX9kDU4#B4ku(M><~DjRv5KA5Ua_qgDI=ai?V*g z9ioId7fAU6Nj{SHCQJPBj4vz#j0Y-{uaE$4?JAOdlV1d0S!FSk;7*+l_{xHjstA{G zt8JP+l7chFsa++pDWrC#)S;5XDme^fS_2-}5@yJa zU;^s&p-wMC2>@VDMj#0qGgx&+isC(zeR;|vg;}MxtJRQF{vjSX$7B(0#GYAV|x6^=>Fu_5L#}VY<6Q#k%`t+|E?$;9T zY{v6z#&a&?zEbL4yYj=vFaTK7+o`s<(;ecD=K0>=mIeX9M7!Eq1KUYN6w&n$K-aCQ zf9_2Gb9?6Ro6~>a1fQA`A-6XE+w#aCz`(v%+?X5vZEoyq+mz3f1KXp0pGJB=4Rn6& zZ+kz`3Sz6b`Au)*X3y2T)86=~xAjq9>yv?wXT#ku$NM*CM&Ha%Y|T!-Uzh{&7C4j0FyLY1j|cj0 zG&QX>G~8*t`k<-tVbkRY%~u|`H$CaP{=BF4Re#6pfzHXd4?RJBK@%u}dA?30Vh?%zrj-^gXVR7x#P;h7AU%{fcTnaUavY())#mQc0dS9`!; zdmyy`u&=s~EzOb17#a!Mcn^>pjT5m6ED;GfV8#~3{6b(JRR(hL#{n*@)=>xO)8BI+gMf>prvEM($g6zEr!L4%A2hZGK*!A0v_ zl#X_p-5#-_moGpjlCF|0naQ|vrEInw55|c|#FI@sBM~$ZMj~-%s>EGd?MjxS{+NmJ zs4$$2XZ*ECeKp4;^~d7}k9o^0UC9KUOipu$Qthh@P-09a@D$sYW#lSSbese4A2kRVpkhnOP|@t5kL|e?$ivGu zPh%H%7VA*h&ahM{5-crG?>~^LtpgSs$>xITj4zR7f}@C>VD+Fxr7L4krzh6l{k~I-?2!W-qQTdi)VKl=Q|+!ij7sp7LjMp-gEA zyiMLHZ$)i+QpvaZVn>p1^NRKqNzEAIj(EMrCM50T(GaWcz2PwWqY4gZblJrA{Z%+rE9>bGm|Py z1QS7DkmLPspyv*TF_XZ$7zfAUejgPMVPLey(pC~PI~`UEv5~-UoWX3E341g!&}EI> zp_M=y5DJlD1`%VhMrxPK97+kSl)_4dLoH4%Vmbq9u({2sV50&k8+3YtE|=E{lXksD zt1>F3db!M~P(xa^L#1}eK^SngRor5UPze{EqCF`ll;bj$;bbPndTBr}Fo=->Be!eS zh{52rnw&N}<*>6b#6cMEz!>nSYp#m2m+LbP6@k+k7D(8I9Dk{lzntU7eyVykcy)99 z`kTqtEwQGQs9(P?4}th&bwt#xwaLHNrv67<2@~lh0u0dW)(kK+QMpzpK>P_}W$d>_ zajWv@nPCv0riZqt2E?{0JL_1VhI_Yr+uwIzf7jdmuBU0Uv+-re<>ze|o_AdMeXjGb zEE%6o9KzW#c7d1G;LV{zg2(&DRyc`lc?Dd+%jucE)2A;UJ$(B3;p0d5*Bq&iRHcA_fM60nIN(A;EE(gy zJGJZhl#h+N$$-noU~1T2XfWn!40&3;SY%HuCa2sZmy1<7Wb$tmvORjOfn$l199ddU zl~w}%a#!yM*2PuSdTS2&>kbC@ABr3}8m_JPRaCHS@ zLFbZz`pWRpI{%?+p}xXXTjs7QV=7Ak!^lhoj&c?sVHQv$Zx{8;i`YY$ML@)Q^E_te zF(^opDFKbTZGNZKhe06%jyd5NVG9y)n6>*DqlY$eZi|n%_ys8Bv-;eQpa%&tW)3xS zsMY7hV{SCW7+px7R=rQ9Hd8JD1{=>~;ixBD#->WCM3zWo;BXuXC-6ub62gc-3{;E= zN3FaUiN=UzmPnSMp}3jl)EF>Jg3Xo*)rW+dBZ1n(@xv#ghmHyrl|pGLlT0F^021_* z@sPJX;Vy}CB{6?49xY1>alsXGx}y{yVR#?y@er(s5P}@xap@e;4v=WbLpeAa7FgWR z6G1;@Zf&Ixl=&W!qFy{GP6#rhje&ZvxolA>Be(SeI4Cn!ws#?-~D>(`01a& zJM+uQ^XJZ8ZoJfV{c3CT&lgTdOXEPNOcZAKc*GsV!65AOp#eV`4e_}|uq+j?%p@u^ ztlzCSYE^2L%Ahl%5Xw;)Poi$8k$`}K85zXFVRnEt7SUoxKraoZRiSDHqL4!hxlJjv zh&7>gs1@TVHpDr_uO)CdMS49@btOU*SoF{PP7hW*u}vNkGKWg<$|qN zz22ZkFe}ZXP71=EHgAxPCIkL}n{+uCH?U_Hcpu$9#sXXOiY5QulecSM!JdNRydtT> zpf{mnvoWXNMMoI}ZV(szii`8r>SDcKtOzXD&y^LcBnV+EtBj>ff>y+^S5~mAxNvWA zQGrxa0G?b}P*7YXQAyQ0h0&;BTy{WV1lX!pZn5Y*3?|Spq*qw9a@=8dQ3&{yS|gLn ziuM&2>}(Y%`tRPt|F^fGNTtC%lsm!!2t`Vw(M;4I^K*U=>vK~+0rl_*$HP3oQ#->) z1pF44Q{%8{A&UkQoAQApupH6{er80E7?r@i99o$}r$7xF+@y0_Gz2)B)VNNCYt^Jd zP3qOSUP~BtE|Zxt!Csp)gt((lF6I&<0GSldAP(H_ARsGlH4zq*)1*T+T2!S$6egz* z;T@zj$zH+gH7zw5rX*?#3k`=v)sXEq0}{qZ*_QWXS2p`DyzKn- zWykp!9p|36{QB(bnFkk6-8}pA&2y)2UifA0+^JP?(r^x7?9Sy2cdsjW+<;%CO zHr~8;^=8wxn@vq?jg4#9uC88dSZO@JdhPs;rVBTkFWzctc+lDSsPFotfwre3-7m&_ zUyb!|j1Rt<8htyr!<*TuR}&)-ySmmdU%A=PaPR8nyVou~Y!w0asJ-b?`?aT?*PnN{ zzU*#$-PiTHzh?sgY*=jT4C2k`*w)1OTe0HQq&TTKH4Uup?d;4ufX1c8cT0=!#ZJv) zYv!%R<<0p;z%76fa1I_3A6#150xAXowz~FeY4P>KJOJ3s#rYRIm3#rip3Y1?m>$17 zH3As+UMX?57|f32x;?)Z6B$-h@2{-$msk5M$^(_<=|hKN2kM0K5}`aNRAhZsIleqg zXOm263IHrnU&Ys!aMOnR^$?GwC-A2I;hXpvuhz%cnBp@FM z3s2-3)c>!j_`e0kGLsqe`<<~U8BbD)3=v7ez6j)xK)$fW?Sr_WLj+iij+Zd0tljIi z2Los<=?KNGf?r3H3K-EjDIjWG)nUFGNZ8>}{gKGQW8wNkzN%_Eox);aEE1wq5xz9a zWP)5S>Q0B;F`q9TprbBl5TkugH&3yw(?#R7hr(Q#Os|ug^lHScMa@RiZl)24b0R** z#EPBL!2Qa`x*TFj6xQUXj5MjIaFJoWOXYM$%#DNX*Y*ewUp?O|(@RjbUayJR(9Y(%@02uIe+m?OXszL@s7#Gp~ah%YxibuJXpN_Wcl8c zr8|$8?>t(*`*8X0{kip`HzFbkD)ya%zjR62et=J9RYfr)60?EE2 z>7D|~o`S-?MfrL10w5U}N5CvrsFCfJNyL)&D&J^p%|wzDf7h| z|FT{AMZ5DQyTm4w`FrvU3S|m5NR`rjiR9aTh2QKg+^Jt&m?x1KU>g`HxcsaT^$H>0CAh>QAdI+S4i&^LmVjHB z&r1Ni=U5%?P(vm)WH4YB6K*vFIVK^i3o_sqEn-x`TA*A?%%mo*TFR;i;j-yztDdzO z+!l*qHwiW~2U$4C#zTl7VUjK`?V>{n?y;dR$d1`8c9Y3!G}!byyH*Qp^q9s-=nS|P zFwE*QI0Go+gYhuVXW3X)AXgU)$2}-%6L%EAMzweW1`TEwlc`wP!Gmz1UO+E|@F9@X z;DA(u(|kC|UptV#T))R2`!w12d7}5zSkLxw&->o?t&XO*9gUkU4KEtczPNVo`StS;n@)e4Z2Nnq_qX}> zPm@5uE8EeAL_Wbg1J6z}RT-tMS2&>9Nh}$&IOr*W;tldVAL{G|c_{^ZNN;?=&_% zxPJ9Ju5ZJkUAU(=y)L zFxqr(x}{PJuF#chA=EL&*#=^qtz+m&QXHR{1?8vc$<%esc z`*T8NJW!kRR>YAIEdne^5fR3n5~whP`<<>R<4OC174dLo8seB@vl&E@NiVgUBo>p* zVktHn3r!{^PFMp$B9mn*tAKj}fO)Fxxym|VU_xb`P*ulQ)(YiSLTQ^etzH5=fl7N(Fi`eDHvi84urOiw;YLJ>CjTna%djKHHwx}=iM)w*&up7J_h z&B4gQqv84kOi9L>iUB&hQW1A9OeO?RHsXv3HWqis1SZa+0o3UyxUk0;5Gj=&z?u6tjZAWVxcSpfE2lzi3~9B(DgBbZ?;qfM-vB-mbjedkXiOFdM=;r3Upr zg=C*xx>qjQrzp->Nq4dqrAo7s_K`3NNi||KnqsXcU!mM9k?kpz0Fr%MP_VnWphziG zY7`os67bS&))J^S6X6qnT5vf824!3*1R1nOg~}jP8f8+QtWYW4U05W}XYMNiWZhFx zB++Sr@**rj3N%8aM#QAH>r{55nRFt8fO^FsLqRGW5Uacpunx5uahr*-nE|?J#7u!B z49wWzw5k!E8WVR@=)qr)2Qe@X9c|Y!Ruh1h&tdgJMi0Q40}f!|Z;=RN?i9smC|t0r z%o2&JNTQS$$s|Q`X_2_SuTW-|=unl#WiXR^J+3j62AH!!9xKG#_%xfW4u&!u;fCx^ zi`8j2IqiU3;{T6RsQSM zfu>4tb9G?u!eMa%Y^ohF?9F8RzY~c|;#RQVR)&9D8Tn&%XCe{c3W(R*#2;(_J8if& zDJIabje+NUSsMC0C(`TltY}~#C;LB)_kI}ZemB_puD|1LSIcJG)s5DMXID=@x_I*O z{%oR~lL{lb}(^Jh;k{QC3K*`HT`J+*S?)atKi)-Ii2y?k-)O2hiqOE<1vTE2RI zzTx!Txs$W!PtIRFB{J;t+2yOhu3Y_f{o46EZCCDhUVGfz@}$4*`B2a6@u7{$F%Yju zhhO&f-@J5r?&Qg@0K40C`F_jQ2Q610ipNGEWKVW-I-Yj4KI`sy*4Oo7py&1Q;OpU` zubCZyVE|woV`Hz!$2O-XK)jxud^0os7O-k=ZUY$A%*@8z-0N9zGP@~0GPgN5zcDWo z<;~(E;MS|f1z=!b>GcZ4PA`=w(-ZfnM(<6J-k%u*B)dO7dUs;@768~_&)50G9e}lq z(ciJq+cw#DWxD<9a`*MMo@i6u%1CW> z`pCh|@k5D2b;*PKlLz)kYpa7*WqdBhW)oB@Vqhs5VpkEUmL{~U)4(%GJmf5i6X_5d zd7a?I=69k|7I>J=hg*Fp5^*{LPO}HqQ79ma zo^}}+T1%5!mJ$KxbrS%Y0%`UVNQ|=vXcJFqU8uxn&ey1ROC`Hyay1NFIk$nPjEu|5 zb2dS6_=3)88urI5?tq;S!M-ROh*I%{JCi~~VXN0?_xq7x&`7&v4!azN;DFzmO0#8E zT;+ZsVM6smU){mz;ltjVD!Mf7OvK1|h{*=AkQ)uK1`1NaMu!I@!<5)#(@)Ytmf|Ri zB`Gk-#?fMh^1u0o-xd|`lS_*9N`+OUai|T1&f&46J_7Z#h~TlaZVSztXcppmEChZ) zMoBw}1X^Jhsb-~d#7kiT7wX6DZW|b$t6_x-RvB?4&J(e+MD?Mn%Keq4HPux|jvqdC z?w4!rR|m%1CKr3>*ZUT446ogt2K2iBZ27^98;@Sye)9Ux(~Y~&Ht#*(xcB1q-DfW! zJb(S<<>uFT`f~Hx%Qw$oZQOr&r>)~sb#1t`#8XxlJoU@>le4q8A3S;Rd;>iH(bHEC zAHTeF|H<;&?b*fEiRtN{-p-z$md2~6tIAW96Ezu)a)nB&6kD+rNTm6NMTN!11%-tL zl46zFXh7^@#dozlUnR>^OY=3-0UR}tFmaNdWFoOk(=~Nt6pgq(@*jh ziak=X!}VTyQIS$2SIe||mBFCqXuL8NN_c@`n<2Z_U{tELQuY7G)LU@3k*({xy*lZT zNsJ|`5<`iZ(Xu34Xc=RO!|5>CG0Mys>NKRg!;YC5!mE4ld(OT8=+07l?Q_O>#~h`y zWIK*sPd!s}ejh|+A~YJ0Oy*I*=L9b~nx6~Mbv(BKfbcjDk7LjwKsu!a5^@M^5)LBb zAPP$&;cKMQq=g>cAxIf3`DY3+2Z?{*s+4%;xat(Zi>s@1KHkbL2dv~&N zWAx@)|J8-g3vce%|8cYG+4bs|H|t+qum9s}{V$j69$%=PI#)e)zGm`V?Ln|H`0f4={ zfBSXwoj1*QFp{<2ecN^~o2GY}mUmsPb3K{G{_dshMzNKlft6vb%k#?U$inFG%EVZ< zL*^9lu#JZgS079*O-+C-Kb%^5^k4;y52uzMOf5fru=Mc3^5aKX^W~qOtYqK%`UbFt zY$e~h$%%Ip<8LNL0mHuK?SnA@us?<|fIaQS(DgGWVI99@4eUvG+f;kwa;%<1^n<){>rqsBI!;8ZiSuTdQqAx zsjCjvRG5o{uu%i9_;?tmh{X(rjEais8ky8rdH5(O30w% zS#?sE3A`2of4oi&*P`Z{G+d*aCKnZO8OMkPM*+hqqyiqB0+_9ku{27=q!(GOBAWv- z*%6~HD`YmY%^@(GrFN^tZi7t*#BAo9%q*pnz(+`8fxu!!eO`S@6#QhI<#o2QT1#oA zy`s`tU12VbD+4~%>(B&j5{rtf7Ey!{5utIEh`=C{J5Zh9tn-?s1{9X_1#$sL$UlW6 z9?iiW%O{+`6OR$_#|Z^_bR3CG?C0_oF>EEa8{&srRC#R{Ea53CU6k_mZK z5`oR&D1<_zTy9gUT`I9bz*Rtq8b&P&ja}_5a>dGHiSk6WygXHZuJ+o!s~rQ)Bab^D z{MP&AkAYuaOgx``^m_j3n}ufpUGsojYap)|)?Y8IygLkQ%F9>3wYFVNr+t-`(eB=^KVH58em1kbHnY4sw}hc?W`6k{fZ5#g+u8Yd zGjG>c=eE{oUi|)~v#qJNrpo1V3dI5@n@OcpsdO5JN+V+4#H4aLR33-GqT?Z2KAnPN z(y+?HEDD}OC2;6Ch=%Pqz#6L2$y`7$23^47O8G1?_AV3^2C$*$0OpYj@Du{Xrt@Hy zUZZdrRR*O{Dr9pxbOsXuj7WpXYyrfPu<2NQzKt~FU zRrw-uD0qlSVAEI@qm8u3^3L)&wwcGV!cnH_eu>uxz4SCizZ?* z0StgNj!EG^(UO=Vc>9l+nCD$zDU5L8dYr?H%?v{kDHE9P7 z0|%WIzS-UzyN|mt37hQLn96KE=-heSv;P!Z&2jj2@cmCi2Tz6pt^mFEo(%0h9y)kD zwEt-E;PJr01V0?#=$r^`5r1j;7U)drKWR z7uv1?hE3l)|L4uxrLJoq9=2@`-CXIryx4i+-Tk^3H>;jrOik9u9-U49asABi*UtWY z<;;T%HDe8xqYYJ~^;ILam8135lMS^~XKFwmoUMCwvEkvRhKE5a=zu3yMh z#?Dw(VIjlEC7CI?irnuNc}I1m(q z?U?Iqe|hiD^IJDw-pis3gVsF|0Ilivrnha4fMHpTHBPtRpXtOlJ1_Kh0ER90_bg#m zfCtyc#?~jsSI0*|)+fiYiNr}PfM<*2qsvp{fLm*iAA)iD!6bJ4k>{9*ce1j{F`H(qUVk!yi z!(m;x$WRd8iOzhO4J~QeweOtSM1uMFNirq?~L(Vr#$zlkPFsM=iTaEGzT9!`D)Ty{; zEo{}ocAdmylzQ|+w}xv&nR+>-LpcT&kT9~Cm&c+VBjSz_3XT$Sxhw`*j4+iFo=zz= z8~7%pz+w^GY(k4!Y_%#}E-^T;m;k_pRx{V2V{3p*p@7HWi%K09T`*vd#dMLVDN$}p zRv8i%*0L(+>C^7&N=rPZ4!FS8Lu^LbD32~-Fy$N0{$5LYIk zBk-{T-1jH*kL8h0;HW?3laJ)!kLBlMxAk;9xH4e1GPEerA)eNN=#5-0N|NyKA~r$H z#fiD!tsm?z?`$n?Z!PccF758mu5Z0rTEVig`g(5p_1s+da9^q}9V`v{6NP%WRbn&g z1Ab?5u`ga6sfb2TC*&qo9tmGSrsQHRa0s~+DkPO*%RQV9@Q*Zw3Z2oQ#$_O|$fw8! z`7}}igMug}N{3!yRT}*Ur4tqD1u~P=;4xSN*si8{c`R8~8n3QOoxMlxx2Qu{QC8;?U_6GTkih)#~-sxtBY$JOKaKZuWl@7QHPOlWqomZZE0z7 zeSPupVC`^sZDsM@s~5kIjSihV*WmNJQI!k<|BEofMucp>h$|8?1qhi75!nm^L?=K@ ztbQYhPU12M92y>?1Ec|FgAh41D%MJejoo@EMF@(8E8|mPD340WA?B076FCe8Di-ir zFo({E*#ZGiAcSF<&EZ0PF)UV!cygGE&@^beIWASYiWTX%|UMJieC2l2IW^b|07){>CtH3~pT5GKeXHXb6MBhZsT@L(JyN zVF93*K`c}w0u?M&aCit6=296<0st5=VThKeld?|-Ep4@t{)R|rrMIiXn@PJeX?Ld7 z^StH!&ZExFsgA6HWwsu6Z9nSX|Ecc~Fzly+_fLlYQ7)_>*vP??k;BJBhmQsh9`%Fl zKLFh7*_rIzp6uS9?Ae(B=;~P?>RcP>TJ7yv?QUDnG_AHbu4NjRGPma2uFc*5ai;Oo zo5piXT~~KT@2&S=UFwe&CNV;y00wG65dLrp=)PnYW-U8)Z1osQ{&4ICIG#FlVN%MU)pHq`#s9#9Nm3mX&J!9*U3;KEnwbl_nMl4O*Irp9j(N zSWrP0U^tjdmLZ5uE%BPfe!Do}mIuA6kXIA&YC|rW&n)qoMJ@wyGT=`Vr&j4VW#9VJ zh@5IbBY_K*`IN>Io7$@<3g{;p_#74qhd?xin5|R6R)f%G64*5Yrw)jn+-Km~6m-3a zp_ejra)w3<07jAWiF{5jjdYxZ`+-<+f=0+;(};XFP0VA+g$xC7L>_m$ zw(`p$wpj&MGgGaE)Ji6{bxejB)UX~^2b|8*xGGqvjl}iQv@Tk1OjVl8%57z7cUhS^ z8dL?%V!Hy?fLl^7VnD=JvBWA-c{QeDyE0(b1Rcggk4UQ|aah>O0IU%)<%c}-kyC^t zr|?H|u=Z@n3AmG_`~oI{E$RXR;Pr6`Y%&!gfAEDDLubcI5f z@7^j(m*~AFnGNM>cmh4&a`Mz(q+gl%k62(Zq@8m!rhKqV2Vq!cdW1U~0v!Kngb9+OR_(g*}X9+^y})9G9` zi;n<|@x(HLR4EZE1Q4G^ z)S#CejWVNNX4DBaD2z%48o5M|a-=W`+iy-Lut|`Zt#zuMA&WO+a}}AaKCQ*4MXd_1 zT1Z#G3>CuE!W=z<82Li8Kx`38EMm0>H5F+L0lCH{0RXcFbw;-u5kpuv0!qQKipx`Qxl%4u0DapyK}VQO5erfBMLH2n z3WOEUfC)T4MJi#7gb;^A;lXqsn`~AJl1@$Y>CoiW^1iyzP(!4*+SgU#X)ksCbfan; z16T$CEURDNs`~bQ0Q7p?clfyP;K{(=lYWqcp9c3I18#wn{aJ2xZBKM=Ph>#0#xv_9 z?Hfa#n}eO}109<~K)eSw&lv3#xX~@9$X!=o%Yc9vNO8 z9a$O~1^`c!>BpuJ2fG2ZejV%r^!lYAq$|5FvGZwf`yazSZzqP|O^wb23Qi8c9qN8P zlzBJSJvTKp`_rSTp`p5S=e_ZSzpSFDrp}T`=!+wUaIrB`Y%C6&V-Z_v)Ru~9iku3! zfp3s7q!17Sri@1sFv)yuM=6CbAR$Nrn|+c_KS8CQq|!)2q15Y97MCcBV`xcC6HV$9 zX;UI?FD!f{0Zcu`z@4Dy=dsCD1s}0$wIR1X5we$t z4N;%asbw0ZOuYosgHcLXOPDGN0~JvvJTk(>bEr5jjesz5JURiUllTlGpF!fY$RaLL z$RmmP6c7PI7YiV{n5)BfI#cB`l32))VQtd58YxF3LS1@CBBToy>LW2jw9F8%GNr1F z$*iO%686#(dn#a0xHTT7(k0jU)P^FHIbyTMJQk1=x24!?D)NX7Y9fc7N28y_Q;ru< zkK_`+0qpxz`QPW{9WNl9z!QEb$U9CV0{lZNlyA~=bqbb70;$CmIUg_Pe7E=c>+Yw| zJD)%8e)+Qb@#8PAU;o(DSasq2)%%UVynglb%RlR{oC&6jyrn+9*9c*?eFfH_FH#my z)|Qsmm$@PVI-h%zh(Adr9L+DliUbk}$M7T^lfzXgxJnrS9!0<=@qmI*$Q&vSW>9(D z^7^wc=H|Obhdt3Eu}RL;@dQS`-mCLO0U8TSDx$?@v69NN)S2^jH=A$wPIN!|bMTip zq%WtZmJ#Zq2T3FKp~CZtN{=?5%9=uWatFZtVRh;COcv z&}(addwYL#dw+FnXK7;-Xc@+~b%3;;mH(mL#_sa^4#?W(_Qux6*4FyL{?^A2yZifF zJ6r2po9ml^W2>uw{`sP-yEk24sWDkaY7Jkh0t{mc1Qc+e#lrf(5LgU6o01O^a~VV& zi$>x?6qrp#STqCz*bu5E2r2}T>J=bZZ%G{rD?~t^j^&>M49g?r;)wYL`22i)K>;2Q z?guC|A{`<#*i?wiWOC^=HozLh6Cwf$!WZxmK8y$vkxayfIe=$0I)%k%LTo0J!-PeM z5S8;JVxB|L?x0Df7r|19C5EU1 zCP@gaE;iSL)0NxxfDv5LKje5!djhAs&a}9Ub!=5aE2`ky^uoB$B|C4M0BAJ z5g2)hfyL7>*m5dcLgPs2EHRBOqjTg8jtt_;IgkW`By5(1!vORWumQbjJUR$d#O5hr zrkGCV;TQ-3kuX$hzCzAJ5ISGP5{X$3qbz6=-$;8O-K`xuS2A*;Wb$&Xuh!q5c1&JO zY)-WU@md|nwlnWM>H+}Uf7E^OsOR8e&;G;i-Knna2VFaldv+go?>*?=$4>Tu18^R= z*UscOfOTL18}8Z~>e?8{toLPByW3Ygfq~s$&D>k=zyP+;aecP^+Dz+}>E?59?>8*8 zUzl%gcy+Vl+4(G8!p?m7pv zPlL2KW+xJ}0BgPb^8W4YBKzjqOxsLX$4qx-rl)E|g(p+XKTXX|49!ms ztvnuEc{(}!^ON4r&TuMi3Kki|5oyMT zo<_h`Aq**p4u0$8d`KoCaCx~91n7lTexlI;!-ztW+~-%7#N@>>IRIFq)ReBUmsk2u z*Z59XyQ?egWl2*a>PRQurKxb$X4-~SL=qU#66b(RyehOmbvza(9i^S)V1u#Xx zrwN2)KA)vg$P4YVB9}bmRTLL$6UF*e6vS8>vz8^CRcUK^+*}rQRK%PWQCCHYyRz6{ zlknFjeKkpMZPHm)?5~Z*FIPqylDd#xY**7|yj&LH1T8O@OQp$Rwci@4uZUl$aiqh1 zr;cqwStdElDCb&~90SVM%DFl@TO&er5{Xp>W5p6hY^8vWA{-^3g~E^mfs`UhDPbxA z>4g*_k1XU81sswXhExiQL_*+$XUYY7rO2q1*;ERL))>SJnwyJb#*(Binl`7atrgYg z(h5sy%2SmJ*2bd^32(}63>wTKv#rEwje4yypSi@N_gm#w9iovl#R5FU${|xv5oo9I z^kcxla)?KB2uE}9M^5D*$;m&OhdY`_IFgS)iYMZEFs#!FEP9q!0qNwBK|$9^zT5x$ z@BOb|*7kSbt}J$q4TsVRiC)7~NEH^{{oam|pC-!AmIqRWmXH;7r~z!$E`u)~j#rhY zYDzuDKE6_#H4m(JS1y(E10HvrfO**Q0(=gMgtflp&?(^B&1W}>USm@O8ayLx+l z{{t|*+RK;ZHWlB%7Z?$xQ(+IAeQ{4DT^uegj+CWRXU^5%Yz79_^Z3Qcvze)v^S@y9 zT7S8``4(_%eQRN3dkH{mbAM&)V0r6cX%q12U~ThYW9$9K_WSj%!?n%*wavW^K(C#{ z?cMiVyN8?G;G_HNTYKRAw|o<|HE;&Q8+K-Wdv9xJZ)bP=@NnncH1nUi;~%U$0!h!~o=xiDlcp*ZUY7w(rY4_=EE}c-!rn4z@9)rebG5|JA zGND~Aa>x{JRO?miyedF1V^CuaY0O1Rb0KOEX=@1bePdZMA@j@L3)-qXY z3R_9$C@2CoORVETA_|pHgd_k_3@RV+iAF)d0YpV02A{=5ATo=L!y2y;vZbd$$aDb- zmN8U10jh+B0tU>dN@Q%eP3h6W4H3iGjk2*HDcLM{<_V?=C7;9S}Y26s@ zSR3j92DUZSz24WkjM1xgrL$!vb8o5R_CouO#mtR)z_7L-r<>2eZmNINR5#sJ_xwi1 zllsU|+}ZBcUDJx{L{JsXtm1P(Qe4JqInrL@AFC-F2KrSI?Wri~t}N*&i?pZ1nes?y zMR8Ace6YSeQxU&e?SeWt7BU1#&R zDa1D|cb?z7@v7zjQtHvbq1+}P~Y=;FhP zrHA8yVGC2ki;pJWJb8Hc?j2jONaOcwz!(neq9xW;(pj3e#p9M(i8fTIixlZ&MdoCY zJsEUFeM+Ym(Po>Bpojq6JDT+zg(8Q^0Q5RZqXU3}1A<6|21A-yQW1?S<4Ikr%u-(I ztvge6?wq}{%$SNAV#VfU%$`c9eO`f1&y>oDFfRw7i%QC)Q*!Av@Cgez5T3^-z#Ib1 zrSSQ5v6v>42puL}BB)D+)rqh!6*X6sn#xk9vZS#rVJ=TvD^sS@xCw(<+))vCR>r;H z0(-hNdz`eF$E@kF_jIhVF5#;RyDN%p3BSs3;pwDgF)N2f%4HDfBDTV1utfd3h)3ix zA}&4b(D9vmk=r107=(76)TI~LR1%j~Q)H95^)m36Zb1=)l&2HJIx$}_>Jk z^r9ZiB>{jP$pI3E0|6fPubhG(ux@J9WBH_GL`ngd$I+{qT9m4mGW0UYtoUyC>(`x6 zA79K(pS^im=d}v73b9cwGOF09ukW-~+xQ^bcv2#ksaDydMVx^}N^ zbN|cM;g^nqA#zzuF8cvip_zPFIo|cltzoo)4;&a+-?B|*7M}m*dOzcUN8N= zu`lXjo<>!Vu-m+36~*e z!%Bg~AV-t}t^yJ3q^L!uvTBtUjl!&w=;S~|P7-pD<>j2h=bywEoCNfu;c-kNiA^R$ zBm$L?52yxi4sr`lPwNOd5khW3wSJ z(&%I$b4(UZAVQRC)L=50Y-XLspfRfqc7-dT^A?(&E|XLyz~0K@v)K|ZPb-vrw90@^ z70~K}dSjs;>zf?XnL`>&k;Wd;IwCrE)aXlC3rn5euvrJ}jZb0=@UV);*D-inDo;fb z=^%|=EH?_60umV}Fhq1n!lLpSSkXl;6)Tv@rLhGpSO8J!1v$7MKnToyhKS0LP(b)9 zwp@oOwE_hS^93}ijODZ{f;QRNVq^d1*!a!T_FCWLdzFhLx2CS9M$Q(`_TSuo)b(u{ zJqEC;%+6%T-c;w#M90=x+xB=n(5!!y3&3o$D?53JjXRUw0B@_q_g4n*0|VO}&TI^3 zHu^Jby}-aama+g_?!33yaSH%!q5b+CFtD~C-?m(M)p+Llozt&wpZ@)7`e99IIN|OJ zn(tYZS5$&?3goOp*dQ0y%J}6XR-H=N7_beNmkd?K`YIw_)~lxGx2>(yZ7nk$?eo|=!>+}F&iVfKnQmZXO*5JM zz`b5K-v)WpdG;_FzLjzYX=g8Xs7EGO_$*V(HQN{N(83gYlI|Q?sK(AahgW&nCw% z{dm~`=vC-fhk}|&SX&%1#7Zpjgf$*_mL?skxHVB?j0X)-pCcIz0;h=uR4%>9Bo`QE zQkxoenj~fe5$hX5JxQS)r%+DN>13%4Ed&5dsS+toGHpne*($4j^$otdI%6`Xi$$!Z z2|!1&)r#lwj*}@s+D=g^IaI7wMIM8k%cSP97&rhhE*mh6guo;iriz3#xtL>6DT>^h zct{azm!kEH7IiRBI?M%}N-MNhbp;#+36Jaz0Va#_=JFOh}Q4 zsUTQ+Uzmun30w#d(E+e?=oGR5L0zsub#>_6S${));7qN*whDZ=`iRpKa~0LZoXJ9s z--3EfDxXzk(cswRJQ|L~p;2I9K8!pH?T1{#54ohHIoSN+zfR$gi93>mJDN-Q zJ{NZspae%gfg>Kp;lC%~bGdA?R7_XPnR=9=lLLVLwe|61@8pQJ$fI-_WDdR9s+T)V zQmfXRDk>@qVQVXVR#(_<3))pKt;(efBtigSDvL6YmX}8_AaclLK7%2HIZ81@%Ez&o zCy7L0V8;sZd2~8WD1c;Aj$FZ0qI?a?SIZP8Q)~Co&WFDazWmp3uig}wCq&o^R}KJ} zEo|~7JjG=tk(!li8+&sbyGvVp%h+t; z?l+NI$GmDEldbK;?Va~KJMXu40K*Qq0J?S#cL2Jw0Ncq16S8ao$+ohD1KHYpzqJo? zxV8TQWPAS*Q0vpD_h0||`03N(`}ZIp4-Y+xF1gC9XXzR^mtA_fk@{t>0CO4#{ez` zv1km4%H}fJTo#wd`UW#@_B@?IVKV77@T*265-4~En_;wA!qHf1MO9^OeX6|NANIKm zO`(`067v>E3j>92RHYCqL_)1d?ojH&7Jac<7dDuS&6cpq5;9sM22)sT32B|hMo-)l zNZJca9pQ3!QHkB@K=nFUETwSi1PT_7M<8(sRG0z@v2D&|HVK5p zA~3jQp$L+SnJ^1SCmpAdj?oyW*g^_N&OkIAu?~@_VYNYQvZ6{2Um|BK6daF5da2Ak zaJ_W0@$~%~e`BTR*Vc2-+b=%7Q90dnbq(7G)&>B!g_Wi6+{rR*cOr{1aDXMVH`xU+ z_HEh_TevZi*&J)%0PZ!?vO08sd7x>fuX(+%eZ9ARDbuph-n`I$e<^c+spIZK+s%If z)^=^S{pxh1jx`(f8jj|EVw(QCWsu{8Q^_XCdR zpr<9|yX$w}^w_W2O&1O7vl`i1t^ATsbycsuVKUxxIy=Ilk&5!i=gvR7cH{Zo#_6{9 z*-U0O(=peXnd`)8v^dngFxWZU+dkXV`mU?_ZAa5|M-xETo9w${*ux9}STp!2$XsvR zVt;0FunW8dqt{^fJUAHcnjh_*AML{oY;1I4Y~=p{Y;tTqJB66Fu90_R!yw>b8e4EU z^lAz#z=7599UXi*)Ca~FVne-e#|M`mj;%f%U4JsQ z{BUw*YGU*8!==gbh4E1Uu*XAvRrR%MuSZ?z*F=f{y)==K7ARM7u{l=aPNxEu<^IaF zry}Mliv+8p$+P7ZmuiYnCrm{Sg-fUOnN%JN>M*n9vOESO2V22GJ;|Wc6sRf`*2Mw6 zN`Zb^%d4HI>ueQOwzBf<6lJkK9OUVArx?tC6%>4r$DN{5aS%k}vIrbzE{mGaW#VC0 zKAT>^rV}s;;{rgE1po$Oj#eqO8pR$58V;ySLMo7GNF56sl2K~~P%(^QSW07#s#4#X zN>@#}sVr{50G4u90`ZDR&X)PAqmI%*pf*x`KJKpcS`&6x+GmaXVV#T&(?IYX3Ld6$ z4RV>!D)Cx*RxQt}f-Nd8xJCBbWC14{bZ80^}B5y^pKI8=SQhIC@G4g%EsS|{J0*e8Ve;kkhA)k<)GbCaG z29`_2f=wafzRv|PBOJq#juXf~;E6}^1S}+CE{jQ(iV>3@HtWCJ`S)MjpFZ_Xje3(I zW5lHkxlpeS^;oq&o7!y2r zq~P&1Vninc>PF!)^C-kqBoc|s<$&8}4KQ~-qSL}!6=0a!ZtEF+xcl+H4!{2Q^7h`j z>p#lPsKhALdX(m%-c{m=q$8133hxhNlyg&RW-xvmLY`QAzCF<*S#k-@bOcvF_}-K(W6t z>P%G@Rh>?zE8-=Ius7f_IZUVpRe3eW600FkPcSP;B60<3+vqW@`Vv`qh zFGo?CySUO9jyb(9wcCjrEdrUAAy89AI;O@hFgT?ml+ET-AcO?*C`=B8!6va`Di@(K zIV2{BD3P%ZdXdW_^%zB1fj1>bqhw1JkN{;#3<8-+DAnT z#F+UpIex$X_US;%D!dlAKkvBmDs$;g*N@91&8y>WtK(RM=FQ0t0I=(~O+hRxaVn@?r zN8@70orU(Biyaui7TT}Pbzluw!dAg$R&JtIX-*U@oX!P5g z1J1i1%e{cTEfVNTM7omEmXgrjpzns;e#L4yuU9o_6&H2t>t<84-`}4|J*;ndcJA(tYEcNz*aiOPcX`p*~xMvA4tiN-vuVbdC1!GtSSQo}H0I)Y5jo{(C z&ik`HEwjC?@4yqfTju*a7KgeP!3#%vmq&UQMtZUU>&M;(1LztZnHe6;&LWP^VHv~b z4=2ZF#z$wyfq?=28htyC6`p_nVEFaq;H$BL=fnNahx%Smj65I17zP0LOMe$;V1t-} z{W{eBbg=U`K(c|Z*F!yvqXR4BgX@pR*B(x+PL6FnoLZZjTpS;JJ2BGN(Ht#H8Hxh> zNXQf~v83YWMAQOckczp>Q^DG*a9vddq`oXvn=GzNr7u;VZ@f}=@wBzXqYhX!g;uTK zruEpwCLI~(=F{n?sI*fI1|3CJ;gBwo)TPpvifVW58E18!3EK*mG8LCd9S(|Ebeu{( zg2x}h;r^AM_X7?OUQ6P!@i3IjrRBp+q5!xT1RRCPV-aCCna^iPB_g96oh~?Q5uVo=$@gnM&ifvV^-TUQ}Nizf@IN zm$0XU&T@aCw#Z*o=&kg6%Ke^-ur}y~SNg50fKA~uE4)TPGQcoMC1Po$ERBLHlTc*} zCaQtJ-MLcD0S7V#O)Q}Zg+#0~GB=;a2FF5^Q5OstLq(c^M<4QO3Z0su)g1KzfayYR zrOzR>7;-tRVnnVgUc{q$q$7D4#(;YP1N%M)e>9&&=BBNZOl`w_e0-#?o zUjXQ((eTu2o>l|vwOlodT5W?5p6-79@BPpJz4P(kBaa{XVud=l+7>c7A~wvx%8TRW zNdU0YGw06SZn@LYPHox_9b!U2ge}3y=arK4?gT1eAqtxxO4b%7v$gr z7!N*t{QJND`tSex_m|I~K7RQ6@x#ACKD__>;r$no!^6*jj?Z7eyL9umKN>gt0(ys6 zX|@Vf3a*ULQNXa0&zAFOQivvH(WGpqoXbS9hg1oZB4&_9bc%?HH7*e`X=0{WFVwnJ zd^MjT#Fn;$5V>SBheGAjaTIbco_qrQr~Z(0^h9nhjzD41uzgn$4T9Nhn2k*&3Sfam zuGX8J9&*>oz4&XNdIy9Y(KF>(?40W___0#1gl;QZ8r0VT+nAC3~g3}g2q&lDdE=@do+HBN~hz?wXn)6QJ6(?gFt1H0EVedQq&+u#0(}q$Am)Q z&-qS62^hU)V$VrZ?&&+y}xCxuXQbJV5{A&OPR*S%-zMzoyF`5j-|}a zrOq2mnd=Lk*XKH~&bD2dZn-qmeDU4ghSyh5|Jsn4NC*1D)>gOrra^p3!8x(m>Z@U*{Y)N!T_6JS=m6x~=h5%bi!PcmDxcSM$5B zrs=N6>F)b8y{&Wo?F)mM`N2$&BV~_DUiX}lLN2E`d7B?7lzU)4Vv+wLCerJTbUDI;6QsGZk}IBwZD8 zcU9tBd^O3!`m*r3s&GR^VO0X?mm?W;mxZHe%PX&)3D+epaj&u1r4KpuL5B`pxZPGB zD#=cWQFEAdrV>>HpNq#0=`sK?TSc`sU12Oqs{KWX&X@;rzQ^IeFUT%c!{PsxoA=$R z+~Wj70f&i$p#qqR7eFK-mjrVNEQrWvk$BizM_8kl+ieo7S!mF6R44ARcfintS<>g>T5g| zX*A%I`8-m;TjqDFiahdwUFI<>{APvEsPvn~b~WFm-~kWQt00w<2HZ=oW};fQTE|fv zSV{v!uA)ihbg7gk5d(B#ttJtKCKhv)Dn3>u-XOABWDct)U~?u53#;Ph;()|%W-1j0 zJWc_dgM-*O29!r~_ayQOI^(;I&!0EHeqQ}}@aXlkimPWlWnu93A~mT{RZQ*BL1Gs6Mu-&Q z%lKR|hc2L!1SGPMOchcf86+}@3_hJDXjD5;BH%-EIa4M90Gs^z&%ICo&)%p1+W+#`^y{7y?e4(C{-38GzFB*=xch2tZ)W3Qe(U|>_TlRG;U)mm0pQK2y$@gZKYZPL z|7G{^^WOiJ-Gfg+r@(Wz4?kte^a&sh5NrS7BM3Oaa_|qUcK#D^y!ZYSmJgo~K79K7 z@4x@||N8HLe*p;l`uWq}pFjTn>EmCYJ_3>fjD0;g_`I>T_vn{rr_W#V6qi^FLpqm7 zVKO30F`ySu$z#d^x)?Mm02PxYW>O_=K(Z{-Agm;=oJ*H+s8SY$a%2{XEo?UUv{Hja zW|WC_QYu2nGYJJW!il`RlX*-T?0e+Gi}vpt3!!k z-MQL3_wEEE0hWNumCzJMk=m)!1&sQzMHjK^A{KMpYEL+TgxO+dN5bq)*?dWxrAVi7 z0x37>y;`|eD3x+05-zIdD-A%KxI#5kfYM+oRjgp?j8dmZ=kuBa9;?@(_t@25r%o>8 zK^y`XrbwkMy&5*DI7T_kBCR*zOevE)^Tkqb8X@N#o1f+Z?0B9KUexwWq35|>hx;w8YP!e?iCGk+aYf)(s#vd zJtZ<0ALG4U2}t-GXo&q?*@BjhWh4!dkqiHU;rCcs#@sZz#dVqleh5?d28|eiA!=@02 zx_<3xe%y5VS@Y$$J$L5@+h_aRXM5V``+Ao~N2bRHAM|%NT&VLTi``{$XL*UWw8&Z- zw3mgP6%l`3(qCWds3^9jL#CKdTV&UVoxaMF;)Zm%E)}XxdCSA5VizzlLy^NAa_S51 zs7(urxC9P0k41sh3RNhek4KHArMg&BUlLaag0S915lT-|7~kjO{*{~meO>{$=AI;y zekcHzkar|MKbJwJh+wK1p@%L63-_iCIRYF>H{Wi zp-Y0!x*M$~0?CyENo+iq5VgUR6_vbHvgAo7>VVBe?Ok z>Wf^Kib8uyfz@nMiluBOi$tNID5PvME{%*yrD4+{LN-Lo0Sx2w$Pk33F|ZUSfeh}Y zm?R2|j3Oe_bCPqAaPb$7k|pHf#T-9L(z-c(cYCA@UPVkmYddoH zO}qy{3!v*Qcrn(s8EW4Qwr_&8MZZacVnP! zy}voZEf`%JL(S`8gc;Vn>4h=2HPF26ZP^)W+U~ux-gafB@xt<*b1!dH&73QmsIbS1 zi~|nYK%OE{tdCY&V^!8ztsNv%Z3|Ue{N<*>V#8300q`tPZVOa6BUSEbjVDrFG;*$D z;&ScejZ2etm&dPP9KCjV_{P=PjjJPdHzw}fnQm%*(%w1S+4Jmf&(n_Xxz3)Kef^6A zgNuVh%R|0(e{drhTJr}s0>SN2_;ob;CLVnq3jvI61cwrVYdQeK0I?F*)d}bYoNNhZ zSl6R%2G0)Q?2VpsqGY&jZQiUDp#mWIOsU`y}>F5iIZ z6$`zJhv%aH*?0gjY;GhpHyoS=0E>r?00zd#Vc)~xV4@0f=t&F)*i@kZacJOSUu*bk z<;0E3xwfl|eT@r!O)vYJUwM1xL%v5rZ&!O$UU{LWFi%sQuPe=06giZ|c1>BHsk+E= zu3S@5Ao1A6`8J8$B5@hCCH8_#XYAD;O`+Xd>2cN+ste3Qt5Tk;Rl5xuw^5U;7aL>@ z2^+&Ek(FYR%OM9oR9+$}$Y&W0NEYiP5_K#i^H?gpVRJnFDCws%vyQcZkO-+fmro?!z48(f=jQzqH)7e7X`C|9AGxl>umg;K=itErv+!-;t8$iJ$~H;?3#lq0KrE8SL9jU(zL+Rh(G)tmLQj!v2x0|6 zqyR4hGb4z_6q%GNlhPG3x>#M8?gdu z4vUioG16(&G%`7pLP;eMetZ3WZ{fr1{)uSSoy*0SDvK_bxxfchY+=b5IpnMya&{I7 zAv2qpg`uNpLJCPpLb1?zK9Q;7@Qq@Yh666UQkz8Xkcll4kwwbY2&f_`i-^a#Q5PGoFfDK z{O#cD-r@HjKmPl_{{H*#?}vwn`v*T@fF1sIcmM|gY=8g9?zh8FpZA`>TDjTOZY!+N zxr!7vCsQe*ifI%Po+QBGIA}Brg@%=j2>TX?h~kp4JR*Tl2Es~{v8i$v02ouvY{I{`qijxAsN#!%Zv!o5~|8;-96wt=ezE5M z^^23YE|1^3G|8Kzg~Y7o$?2llW8!&ZX*2?HBkig@8A#8_Z87Fvx(mSYKY!TJ?RpeqEj7>CDK zvCxZX@Od04R}jz(Oy;7&xp-&}K2#+(8xI~8&4U0HdmQmTjCv;n{SQNf$vYRjIMERClIC z>TwBO4!|KMuIx90J2a%kOIG&dA zM_Otc9-Bj@oXkP|k&^NI>9jvn(gDDduqcw0Pf>{QA|6q~$BVgmAsf$U;du}eqGZqr zSrC~d7YOsriW0Z9#HBh@sI4heRsxb0n$DM+&Xt(YmRir37^>ZXQtoRNjteEu^Cgb+ zMYgjZYpuszoo}iwFafGvDCg%H&>}icNTtglo`wq_GpmNwPH`@{AXS>Q`6|9y2I-~J ze7ml+Q0*yD6&A=n?q3YEn+0|=&#Y(ZRZN{6(#jYbIZ-Oc2!$Ad5GRsQ6dFiv1Yzkc zJd=~DF#!)Fise+flC9UVv|6q~FS1&BCKFwuB!~nQnS>}15cxcgS_#~Xz~fTHBBFpt z;B(L{D1%H&BM?B+$;5014aH)jSllc+E0s)7CDKwbq?1`_Kr%E-D6r=8Z7#0K32Dt_ zg$BhFEJ#E z+^zhXl6ER16HTWuWPC^=RNAev$;ThQ{Tp6IJox9Q!@uWNS8HxuQra}SJVRks(dAoL zFV)>VS66@aZcj^KBse`kHoq~w{QCL!hn06RG{$q$4 zEExv^H->a#3L-ltD=Q^ED=i~42a%0JXJc_lk%(7aUDek&5Dkxng0b$Nft~?h=b*2( zXCM$CX=-axsbxGqLoDH%EQVY4_1;j_A07tWij7P}hbJQOiTLQ$@Yqy*?0$4)A~Z4{ z8lC73dN0;rc2(wC3r)FYR$GNt;a1CBYE`~Y?=jd*tp!!NC1>-iE*D?FeYUf^p}FO{ z-JxZ1C^RNnAm*AaMn_(*!_IIK7cZu;sxcy_GWoMvntLOSw*R}P#S2nt?ZuVST>9{oCSpD>N#p7G0Gq=jX z`1p3ogPVopm-0t1xF;?bPhP7y%KhtA_pVeXQUlZL>iMY~m&dMM9JzUMvi|CWmRk?n z?@V?yO?S4<^mae(>wD7E`=qC5e#pD%53T@Qg=6cX=td~E5r}LCqdU>qRwTL^3FJNFxL%_y5mi>1V0TR{&By24_0J01K78+WK zcvqruE#hh{xB}>v02rXx3S2rLei;dXBp4Q6h=%5)fw_2KE&;H)NZ@%i_+l8?7+gj_ zI~@35dL;n%C;}I8OozQgjW?_9s!O?=j>`Oj3uTcTwWAGJV~us*wuZV}*T6T8uBgaR zTBa^2&=z^jmF4zxmHG;}N>Et@ge)I0Oy+UPJuY>T+fh@If40(Enr|*~ftZRN*77`K zk;ULKm6+lYzmc zAQ7o(Bp8!&vXXOhlCpD-Wn>)7%sifz^=E3vpUIiWk`lU>nVwA}(UoF~LI}`>0XJC! zCQiU23fKf83(2PBK; &Wu4Nmxv?Qe0ryR1|8fO4L;zZH>o#p~P6@(O2c`DqZSw zySdiwxL9PVDKJ$Rm}}u!&K4Rf-SQ%V71(*}1Qi?)F z)97ewJ)||U^fso}Ds>cSJ>@c|M`XnFwiEA`5)N1g4vBM!STZv*RhR-F6gd~xe zCXomYdWKwv<8qFUQ6`J{SPm1(pdo2A1Oz``hG0P`4jU&BA=vy(fH4j~lflWLFcIK( z*kBPj^EqZGLua9Dj5L)VFOs6TJPe^nV*-^6|=QsnZ?i(h`#yq^W*-%KOOx0^Y?#meE8JT z*Ogo1K7aGV-Tt;dUtd#KXMOiTTWDnH-jlKCYtw7*o^5|zdi!_**z8@U^JpBGQ2*B7r$dQNb{rK|j zaDH*^R&$rD;+)Y_CbQW%Y6(Zlk(xx#6014i$daSaE3R}Yx8n_%a#MLl?{cu!VngoKROsas9g-51lrKhH)Adp!QM1deOn@uY&arg9e zjSi1ZPE1B4F`q9qcK@L_I@aY2jX#8%n`i4< zt~Ivb=;*2+@^-~yfnaFx_MI!vJOf_{$rWO|)9Nm86&4p2l@_@Q9a1%q&cPFzNGv@Y zLC!>yvrwcQ1TH%RnVyZx;PaVzdDfz0cX3I9r_kke*$ft~OevBo1WL6?r4~pe42hU7 z7Ska%4o^X#$VfC9iJ{~mi5Uo78t^$X8O>ypIBW`=j&^IfFC%yNo`gS7_P-fv-x_Y+ zh_&s+fnaq4bZtjEz?hh{??gKQ&*1&Ac+XCx6X+NG>=zI+05D%GFt818>-u2x5x0H; ztoIH8*w#?XHb7Lc^>wK2O{o2C=(AHsJ(M%q7z+usM8-g@f+#)EIEmCu@L;51e}sXz0muKZD5#iQHRkMGnzx^wo?opaN5=caC+e{l1{ zgPZ54uAiH}e(}ML%j4HB+`Dz@LF0`_ZS@bk+8*_E0eU^_@0;!KpM}T1#Xw*+99{{9 z0K--S;Z?oYdO$WpeH-BQ$ly8v82D;4tU>d@KJc^VZ=Kf9-0mf-VY5mT|H-zb1Z5> ziA`HmV7^i1zEfLrxxA>VqO_pA)Kpw(DlfB~sW6qnHFbuvVrzAUw%DU^=c$326&0w8 z3KVei^<{;{SI>LSS2`*@&Z=T_sY_d6*5sR1E{(yX%dN1xDqRwz5Tn3Sd zf}eJx=FrI`A&V~Ik_8NkfX)yz>0%a%2N76w42_JU5_3qHbZmACDl-|84V(KwojH;7SH4+>jO2%ZK$V@(-aXOupLsN4gqnKmW%JKlc3KSlMEy?w$^UDmymFBVgoxEe?@2mn_wg#Y(zDO;;!wN~OT0r%J>GK5#ESh7I7&C5r`QA*^CJ z9tZBw|LZ(6X*QRa2C*}F0=!%W>8)IwOW-QxJM&p48&zYZXp9Vd#pYtC(xzre=|mn5k~4)Cg)~<_<4Bv{TIN#|M=$-!xCF! zKlk^4?(YBCdH2~H87seV!%m>yhE?22Tpy5%36oP<)7}X zNFb5~0yc&d(a2I7SH}?v~N`=sF)qqr2 z7uVgodgu0ymga^V*RNfL0eEn89K0Z3}aAf@9=*0Bs#DkHEsnL56#wH(*-g`7WF+B{vH1TKzOeXJ-+yn0% z@kaY=Z(+a3uJ^%N?Y+6&eRaP5!t=Isv&}WLO*M1PwR6oi&s)wuZ9Mn1@%*#Ki_aP_ zKfZn8;jK%LZeN9YQYg3_3IUA0^7$9NffYXtuvLE$KAkcQ8yLt|C<1H@?gbSeels%iCLVvC$W9DU zEwCN&@5FpZ0E3G={C!|d%m+3@;6)fX*jZ&cn<7kJTyNH2SDq20$_9T(9!rb3U}f-dI$^v>_NnPKRh@d?7x1oN-BmV z;7?z{RjD{y4NoQKaM)Cyh^MleO(jLnW-&;V8t}L{b_Ke;SQG1$X5* z>ZXAQ{)A;wkTh%-35llR2y7~u$E5IBR6dKsgYZlWhDJb;vFTVuDhipDgE*F+@n;%* zEXeV+jN=*UX+$hp24EnAbaJAcj}^1w21r6SnoGwB=y(y6AYx*;G&GxpfN%&FK8Hob zikJ+&T$pRpRTNlii!5i0Y!^yRwH`y2%XYpn_i~x*N=5F)GGnD%mT#o%B}Aows21Xt z0*s80Q-~-!B@ILiFXEj-A^u29{UbFcm565Pr2?l`m}dcoYA7qwmKI6!b46~aAn)jb zFdfsNrKsdMu^^L4&0#ZmRx{u35ZH4?4wuLQM6E>bDV5|G=}Icqo^pA9iLtCk;VNe7 ztaO!up)$}_T1cbgo9%3!g&mL!+lERd|sQ z1Gg0x;DnOj&emOVSC{GBR*6~3R|}akh_B_ztO~WqWT>#%E3Nr8x$bHw02ozD#RWfgM=n$k;GKGjLuXrun-R1n!0_wzU;q-pY+;&p8(PkfsV>Fz9i~3{wDxPU%wrGO-vGSI{do-~<8 z>+t(8fc>wH9sKom_d6i)l{;+()t79gXB2ijUCd)jcuJ#EZ`bnGTmUd4Y+zKZkcO8q zh%zQY3=t(vq7r=q$ZfTvBK$ zDliolm+*qZ#A|? zM<&PaJszKYG&VUsIypTy^=S0L%;?n2$bFC}qf<{O?t?s@oO<|R`hIxWTYb%AEfHBt zgn4IFjd!n(j0LBrMjt<%n3S6scO8U+Ai;tgJp$1z=ZEQ3fQ8CuV2j z;70rDm`pICaL7un)MD2f&2p87FP1?ZAsONmSt2sTBhZ;xA_+yMVYz&&L_{^q=<&A8 zUmu6R-XHur-u-T*V=LCN8R^&zwSsJg+Y*yDfU*Cm7)U1o*jD83M)dA_s0}a-7})wi z^Ll?1Y+c=Vmbz{yJnYVDf5ZAv)26?9CxI@Ix8aU=p^kS!7+pvC9O?WRZvWtKeKXkj zW~gy{pnhZE&c06hkZo=n;!2nAf-#pcPYo@1l&e!+CKlrq_8)V))^wQ^@_YQ&y z$SZ$fDHvLh#Nf5WU}P%T1XTn0*t}?VWW|S5qM(^@azSwT)`J3k-1oCHU^UnZsr>e%!Iu&;h~wR_fZ01 zlcB*;fB(hWQmLFP(+UMDK2t)cu<+T~j8hq>k`d`x7L{w%$Xrf|!y>SmM0s|3L7u*} zNLO5_EA;4!3blm=`eKivxWHCX?5-)Z6y~yI0t)zLjA}!^L!E1u*wg}(Ok|Qd%ks?m z4g!mjNg%=iLji!Hk}=p+0wD#DOTwVj31~pCR7_?Hd?0OhHW7=Y;80XNnnu9Ti8zQ% zU{L|dP*g$|9+Qm7IhB>2=xLsQEGr$wrtvLWj#*9DO7T)I2tm%niP?A&n6?~>v z!cYk*QZ`Y@Ac-KVluea#@FEUI0MXP!d4UOlD(~`{+>2-IXN&W%R2JQ+b(||!pE6z2kiru>M z5_@%}rTPqTQ&GN?lWSsGb!@YarjnyLtP}$7G!9LWh?Jf}MPaGTT@0cwu26VNg}DWy zT#u>{rkBu>uP7)L+w<8v8>BIDj5dKSmuImvG{(fnn3AT@kt9ljP{P&gm2M|ZDWk}w zB(WH<3d>`om~@he11ZH+u>eRIUL+uhgy{?@n^Ong^?q@Y}oSH^RJlyGcr05BkA z;7fF^fvGb>S{+d=!3jlpp_Cv}{#IC1rOmbT)N-y$EYZs)2C32}H+uBuGHY&aZvNTa z!de%I${~jvIrGQ_0i7&m5v6R3j7yVosZuUODS$LWhMG^3GEf{Gj)x@)2`C!+bWTbN zDjh|~LQ*zgClQ$y0<)ZNk#o&*x=O;;X)3Q>^No%_d9}K@^?vQ`=k>RrHs62Tc=LXB z=k4;&o7K1PSKofz{P6AV?%})LgSTJz-hBM}{_|d9e=D)w1u*tM0Q-9Q8Mdng!2Uma z9Rcjy!4HrxFu>p{#6+3muXn;fJu!hzF7Y~e34p;2`|;!FKmQE?_V?kB zpZni`9&rmE5C8iARSbC8^4fM=udn3Xbyw8|qo;(Y)-psQfl{irXapJoT|y^|!3Rhu z$sqtRyaXahm?Sxys^YSBe34ne(J<+9xNR<5$7X4mJRL`35=x9hl}QG%DH)k5WFpFL zQJg8URFzrEi_G~>jZrUB$k_@dPiv6r%?eME)8RB449cP+M@fmpYExI$mY%&-W6gEA zii&y$d}9;Sz`7G-4B@keuG)3Xx~pWlD@eCpw|hal6BhsUDV8mbGb z0JuO$zclVjSy7Fqt@HZCz4)UClQR$RKY4h6=E3Bn`{R%9jZO>)Mxq0~U5!l*H`RVfz8=LFOs|pobAxlW3bIDXTiNYiZ72Lwo{Bsv-&YwSb;oSKP z=P#T+d$zQ!m@na`pwm+Tz_8gM7&=~{5bDfoqeY?B3AH+bT*($n=^P=6#V0Y@Bq|L@ zp<&s4icAKX6i~}K&-TR7mwQ9I zj<^2S*WQ+`!KN*5)0Pijja(mSTpwuI7_46(05H3~)OGDu$E8>8mtVGBdC_uZwh_?l z;*)v+u&Yn&uY)|fbNvy37C_gH%MWf{n`yp1+ur!}Zqw{Q`<$nLlXe(>T}%)2w}+aB@ljKbsQux~x?U5O1YM~9Xo zgA0*?g~-rC*uNADuZ%?DI*vp^2kc=ZkymhF6$3wX&mO@+L}-rlORe2JVd(TF+V0?nZy2*^|Er+!cV^Ut&sCv#HMNoXXClEWb8 zFiBK}Na?nl%L?^H`G%r=>zPt8x0V+=&lDBZmh1A&C>s8BHoVY>qL64pK1IaCau_HU z70seDqymb7i=Eh?q<~It7`Ngn*Bz&A=g2 z(V5Ap^b|~X8V;FBz@Xqh-FN`DY$84bgG)nWQc#G~=xmCFr!H_R3!FTwo@r15j8W7g zvO+*ti>V3$O(|gOWpt$oTuX^!7DdLP%efQ@3(I4m*klBoKvi(m9*gr_dEvEl<#m@U z?_9OllyHo4oQRGVP_a^os1o3m0-#@Hy_{y&P>d?72}DEHt1$xpu`I;z$r*``c!s$1|9>nN$$#57ggwsE42mX zrt)e{QMsnD+**0gTu}?=vb;iBZlTEL=9!#ywUMc}it{`?b1qq?!3pIQnTBhyaSS%1 zM8;4nWqEc;tD>tERGEw@7Ez=k9G8U`u;>agT_z@p1QfXxE8ru!JetCCOf;W!imA0kOOMEDLuNREyURN?6b*;QM`QOV??0V={9@_V>dxy=U*CV-efRm>+fUyjnND#ZBobq|Kt zkx0SW|NIp$9Ry!JDwKzx^a24ww*MX8;@X8bz~I`#-Gk4Gf_r%P3y=)V!Q|`Vx5TDb zLgU~nd_c0j{U7_Vhy6JC{`1GrzkmGv>)^-FJs@F+fMGum4!$2ImJ@%KbpRDR`0L$= zFM;?(<&`@HHCJ*g&T3r+Vx2*xR>=%XfmRGDz)cp7rT}+XfLjm=IIEmRQSq31AV~=5a zJ$MTEHg)gO6b!KYW08qb|8Qg=8t9J(27^QWLw!91Jw1bceM1BN0|VXl^*4%2TsEgs zqgQbF5X2^H4ARS2Ynz+u?%ck0^XAR#*RNl_dbRd^t;VFu!smc*8*rIO$K<4=b5LX~ zTf|bR1u6{>08Fjn$rLQ1gw7LEd3-vL2T>scjZKg%SvEbt+$MY2Q~&v%_v=XKr+C|& zNXt$HP^xVs*s>OA24jL-t?R)SFm8m}*8Qy;!FITwJ`wOk0Go||*udZuzmAp=`;G$e z)3D=jc^hnf2N)J?e;a7~5bF38>jD4+9`-H>lk8pO?uTgiyKv{Ww`FsvdE3{z6KIFa z?qPr>iWmnQSNrZPc3xlTxCXM=dE;fr^||INPa7@)dOd5rmH^m|r}Z~y?%aIR0CH>k z_H_WTxsIl}?v~kswweCchdnJbJ?+52ju`gJH}rgP;FaIE7K;IZtwv)j;mC3@43~98 z;~TNz&FJt(GzQ#jb0oeN4#Sm-;m~$Gaky6`v>NiRh6dpx!$9wnzvrlQ9t2F_O8P+8 zdboEhJ_zXbdd$B)>fapkZ;tv`;@;)BcM0fMWN0xA8`yFzygCwHNn{D`*c*Bo3%(i- z!x@RZ91a0=y&R9ukHI6zixC)LAhYq{Q@D+<_i@xW9r4`@4aK}Yjlz0BD@l3g@*aOU! zW74BI>?AbuR8CqFA}tA(kwL~GIV`+b%G4MnmRxgbmEKdPD<~~Ecg217vf5oNvALD5 zVwJl@;wa!-^Y}Iw0GP<`rYnskiJB_YFw`ct-b$3pA+<{Cv_cvcU8w*D211caND=`- z%%jLfERB+>QiAu;lqwc*FT0DE>*3_(^W25P{0bhBFna;pXyaJye0v_y^cno3KStLbXPZ)re#Gqb>516M9H>p1shhvCFw~7Lg5n z8E~9R;4+C~_z7*AO3KtK*g82&CubWJ9Fv@HQz+e9O@T&klX4X7Y$76ujK;IcOc_UN zQHm^bpzU0voM+aEokn?{S!hu~N6TpwX{!8>Fh zzrZE+iE{aakBL?tpT6zE>IUewxBuAj#S111}0Z<%w#}B z27{oG^9w!J{Cu-kCsC+GYMnxF(o~;2I~WK@M`O4Gj%EnA7e}FFp>r@qB!i6y&{Avo z8l6z16{xgaxr!xIaFi-BPY5wMI6yV4L82Ek>dSL?hWp-z+IRfTuOqEHfLMW+4S(~x zuW8NKxDBi;*b3&WzQ#3w6Ue&1c_Y-i7HnDdH?Iygt--JXX@W}-fr|AutOIoQH*5^l zZw=MI4m7dyK1=r%k>%${Q z3_DV>jo}fH_1N%6EWR0!!`%x*p%uR$Fl;jt*^EZk!@<>%Z#guy5*}QO!0mchL;b74 z-qj%Ru%4Ab*Gi(8ek0nyH9WLE;@ukY0)TCd`eAy-h8AFY`9PMUFu)S}b!1?ng;?-K z*gp@SpAv-WH4>a33%|M-ogatuYCJYS7ELT6!cU0-fIUfc@|_9~jRgim1D)<%3k4!& z6Ob9GthAhr7;F}i2Ml z67G;h`2{cxI-5+yLlA;Q%)p@2FsKv^A{C2F!=qENs1zhB8Hq?jWCA%$%E<;7(xhwv zIcyphgJUsdCWF#uky$P3yj)dYuH0$o84V1zoGh1MMI5R|%r>j2Y7r1JhE4`Xk__(N zM3S({GA2vIkvWw@lMG(pQi)kwB~>A&$ayp+k0^z35*ks#B5C+!t(dHrQjBt@P0O-r zId(1Esb`qAIFT?3g*~2zIG&Po3;--G^H^pk_@7ebq*71j zAWmjyBmt+yW}HH2oJ6IcKxLkU8}*{oa0E1yg@)Kj8bo08@N8ZtiIGO6quBzs&I}l4 z0C+2@R27uVa*Nmo8%3^xR}m#Dic&|E!+Z6zJO@*&rYYorUO>X&r4qFgZq=bticChn zUN5rQL^eBM633d$b>_2k-H_eIb`)>`9qlf@$;LHWfrK&iR)*G0QD{jDEz@9T8jMt> zhNd$7cI?#Y-;SR;k)FwtN?-$X=!;677cP}nR=aF@8ih&3Q7|zKEP{f>G2ug-aBMnR zE@bGGOr4rz(DKY$fmJKC>BSDc)TK9;+nv?9wla%8PXkGrcrJw^hJ-qaDpxDEt5_NV zRnDcUL|l_bXw`9z3a(zt)kq;3pU9!&nM4AcNahhK0wN@4kXY1|oNTdDd;NC9ou;Ns zH?P&THMS4+Hg>oBMgUBS39rNy2?B{vq7McvxH&>mrrN1C;(j$lf;0? zAZ!jzsTAn-a=Ba}7V{KJsaCJC=jL6!dh2eV&lkTp{NO2Yuc>E?AP;7j9zI)|nO&Tj zd-ZJY)r%L4^Ye=@=9gZ~zkKrS(Y?t?Ak;fP5qtLR>D;rYv(FxZ%sri+eSGiPqwy#A z<4-1IV0=0~`sCr*%)_yVQ)Bnx8G6jz-B#Xh{bUB*`;q*MR|9a)(Q1g7x?X_Ur+tHq#*xfDQTM34N zaVyle83f|hvL0v#0k5nEnpQ(C>#&fuZ1`G$Fs=2&0NWaDPBc8M-x_QH^xE;&zY8`2 zdc6-fe~7exjJ5;)`aIkR@)5>Z_s3|@he+@HaL>DN*V|y%n^5<=X!rYA@0(EPPVnw_ zpkq7G4uovm-v;Dty}uFsCsz9!m%8g0?%saUbbYqz>TL71xz<}R+8XBD8(*}7G(2x> zm}_sG1+R3sEe&k_N@+k7o&qmyI%`o-$KL(3=BRSY&^a)98FZ!hZf_( z=ON!**gGHhzZwm`8VkJ~56@48=M(ysP_Za{D09^RECw$jK8gAsMZNdK-iffczoS_p z6Q;o@GaNsimXw;2nw*k+{Pd|mlTQ7Sdg^$_>62MWNjWJg$kb$HN;)nR$0T5x1RM)b z;*+o}9GZc~FtG?K`grE4e)ufEh;1qR?|lG&GftXR?Vr zK9<9V`$SME7&Z&Vgc98%Fv(bSCK*p;vv3Rsl1#}a5;F+6bRs?#k4wd3GjOtDm5R_1Y{e@{yLM+)3|1mqt{DZeMD z{hplpkEG1sk~2=^U{FlHz+ktOR9i}Gbe;;a)x}U5AdL}_jH$OUw0e=lCd+ej3_7Mp z&4Qz0Xn=1i=qd%@2=vY>G#CYDv((`fSna^Hc+Nbw6VS^AS#uc{yC|nRbM9dwNm z(%T@wFrA5{)Y5bYp4|rN3?!-Yx8F}B{dO|xWM&pmDc3no2D`qX$WdKe=q@y?Oj53r zO%ad@Y!ZP@AoIvr2#@E{!AD8csh9>W+o)q1H3^34q`4+#fkj(vwViP~&p6B;qs*#c zD)|g0m!}g8!JmXl3d^sU2`@jYL@d36Wl*qmGL}+AlW<8qI+05zafviB36fEH3LtLe zl*|k|mtRsFJA$|_Tgp2J$R`Q2Gt(C zUy*zrKC{whwQfecS(e6!>ikh9&6r_aVrSe}4b@FL)d55UxuE*+2Yid2^?I zFnH$b?b3^Np4w~fs!Q%O7o4S47Eh5r*P*iML?)o;Jn(4%bWs#+Krey>Vruvt9gi-j zGgJ(^3WC&3o}Mc<@}&m8#w;>fBo$?j%NNTopDVe1zNq15Rm<(#tLKXxb_I|Rfr!E6 zP?;<$#H6q}bg_i5(`n2WgUjV`p8Ud+sjlT7cb_X0cg!Weerz$#q+uO z7f<2L&AfQ_aCT;L_R;tpoC%m`Gxwj&Oin)-zdt!Pb$@*F-sruFk@2zk$Z&XcBodDY zy@S2y&Q;dbmiv4?!*PEsJQN8I1_A^Az(7ym-K*CxpS^J5=AHV+)((*Cb@k;{wNj-5 zPr+db2rL1~;?UGuvBj#iSQG|>ObY-e=WzsN7LUl{kRS-hq~us6w6+>|y#D-?-nuvA zy*neF>yfr~fG9tVt6$s-wj41Geg)`PsAVTQQR7WQ{84E4Y*;luuAU_2p^z+w>2N;tFnMO^Fw9^N z0~5fo2mshCuXh2amv0_kbsPe}&SKEN6!Ncx{VVW6%-+>N--;g?ST6wB8t^bcuTb}T zq;EYour)lm5g!5oTO08$55oqw81XJf{D5JLks#cbemuT95=%S~7G53+&4+z2BL4Z| z;QVlCVJ!M;;upGpEg_DE0lJ<>d`}~OxR^fbn~wS>!~T2WU|anyE{BnncJf5(@zbfN zQqohB0KiV2{3Ge)pJ@PKNhdOsQjlqB=*$#U8UPqb8a5-7l!b;cI5r+I49mo#=osK< zza<_2Gc7p{J|CHcpwrPT7Lv)#hM;T)Glxb;QRze$n<@~H`CJSW0`7&N(~uB^f}kua zHH}C}#bdK5L>!AnV6oA306cOwiJV2IWROVdBv|IsvAE;uaNi@iNeA4jBOP2zfsLKc zPCt=;`p=|er_z&>v(ipwrJu-7PbHv95-!)M=38}iy@Y92u&o*fpqEa;G^*J~HKda> zbrP{dZ7j1HN-PqGie~`ViUhh8e3F!n5m2!rI$g`9Xjl{-o0qGQ6ljGmHP@lw=V}Cb z8kNVca@p~0Rtf@rDie7k9RWP-7~Ehm^N-~8q#T6QpmtXkaW&#(EaG?$+~xg52EZ*M z8I_ZYL8PEisVEF!FnIGHr!tP620tt9_oP%nvOiA4U3ZTqCr(UGblUwrCHr&^4$I)t zMM_Ajea6@;o+koQoG! z1~80e(h6)wX|7dMn5!*zs0u8mGN+?5*HPgxdMskIimR7!bYhM{!qAII3NBd&m*3D- zLb_JUG$e9Ens= zR#sY9ce}Z{xwE^g_3qtU^$j=gG&Hq!Hnw&(wsyC54R#HM0%MaSk7n=BEOKXdZ>q|=;VDj?S>H>IWVe!?gmmn`+E-Zk5Up-%VHMcN7_wxCZ z`DYJbJbm!u*@Nd#ra@+Brk~6_c=Yi8)Ps8u9!xx#f&)mF;Mv&Zy^(kycJq zT5AD#*>J_%0J54`LR|AUg2}SKaoGpv%^P4Ggp1|@z*c*1Zw@qU!}Mx=6MzFR9s+tr zn%@G9#X3HX_I@27_zchm05%FHeIMg}0A24#`rZ!rzljaJiTAx7?T0%BN4mDccQ*p< z8-cd%P&?f4KG3lRFy`x68|+x=ZCmPWUcB45a2KGfaj~mqsi%FhyL|zqH*v&?w`}oX@U5Q86Mn={XYlgp84if-d1q>S=S&PSjr6o%3 zeJ_1pkmo~#FT4XU14A!^zE?qi!siATeFID0o+V$;a$++Kj_)o280=or{;l}X=CE&b z#J@i3Umby05#jp7XaF#5DH>dkM^;B;@QErTFu;}oz#{$^F+i{I^H^|UEVeKndkN=1 z02@gFEb2c3*rSMVD&n7v1n)&cx2|3!k#R}sr;{?1Q!-N1Gt<*D(odx%A4@rXEba8M z^wWT0N!e+s$V|Yn419J5E;|LCo{q~vP~n9laJNlllYo{1y#1bX@>EU+3Zi1UY)s<8 zBQ%GD%aKJ%&huPUoilB5Gy364Q3Nm8lPb*c=t)FD@7 zX$*x4#-b!$UZOrPDX!R7cDuUt%0-4q1f&pf2<}A!7=|H*_uL|!L1AfphCMIUU{NAS zF;PTp1Y8!298DktkHj&VSOybr&;{Hp26ZL|6N$uwL?H1e`5DrHRcJZmdz9kR-EM)%J9?Jq@&4N2b=E zA&iR$cL``V3(x1X)M|NBk|8rwpOvk0WUA~L3P*_6)f-O`KwvTGM3K zbQv6P%uvJih;6_8ia=9%0&`|Y&iS(Jb0z7;xwhQ2)NGqI*Pd3CnVf3_{-m<#jdoK) zrb!kjq071ac%9Umta3Os>2_^~LznI_R&gEP3(v&H2 za%|s>C{fQds+l@DQ>$calq{84XppLtHL3)qKq3#u&icfE|GxkK{+}QJ`f>c}_`}DKpFe&6^5x67&tJd8 zeEa_KJLr7-`t{qVFW(@#zWzA=`Uf0o438y-0rn$2uK45Aw;!Ls|M>EEVLfCRK-$jx zD6|l z+NM_|DOJe|tyOKX>ogW6T}&l#@O&AwxHPk|@tJF&b9~r2H9j~!KKOQ|f6U#})bhNv zJU2E@Wr$T;l1z3-^4YVc^>6A2oI|dmQP=R8(>>Zf={?)M)xW&qU-bqyyg{!w%0DH&>*{*q4Kr-B?IE-=&~JO_g=~Yfn0+k|Jq`C@)kb{{(63-~ZK$<& zyS;X&qjoP$ul=5<2Yt^D2VNYxUVj*F{5aC`-qm;r&;@?Aob?BN^#{(z{lUgPSL?p3 zeb?2rKh(54(6H^S-ydk&cQ$VKyx!?~z1Lg6)AMG#3#1{~R_||m>2G+k`R19o9tc^T zuj!S)<;`Z(Yj5-G%~rU#q!(CGXOq9DGt}D`?1SYPTdsi}w`*$<*j4{Y{DaP*%M~0P z3JeUwfkXf;$Snx4O{de-+YMZ8wY_brt!1&bdAYq6aBH!>dA_xAu^lpOxvhD%wQ0HO z%~Iow<)#;_%`Y~>%OP&GKHuzo1q{sB(-7!w4)nG8`{59x7p4s|tgmgOuie|;e{HaDb)a{7pl5locf|!Uw(ROz9O#e2t8wlG3FTz%4*8rdkaMN0ErBQn>ys2wSb1w;^}y*fJzZDnHrJYW|XDHi|uAv zda@!tRgrGhW;^sbX|Xw0U;@_itb|-^V!kag->yqf)TEo#X>p3w*w{=34u^i22+ND?VWZ%|on=7J(qPQKESCQ40~TT@LrISKg%nzRg|B?Y3( znkh+!Fq7EqicH5ZND9@El9FF_A^+mJyo(jN7s}Ji3LM4R>7_YY=L#I9ISHB8gez}<~eEwiPl(;RAtP2;fXG8_r{S-`+jN^$_kl8SO|rP(}F|7f+!I#6DdlBRiwhO6f^ZQmR`j(=mZ9}JW&UDDT`B5`3wXB6NN*A zTdhbuA`%-D1%i)`B19snF$gL$ij0V&pb&H{mQBR62m~IPqvA@8Qi(|_Ns!6oPSAZ*k zvEM&^1vUor>D%ugzyE~!_T%H%KR%r-90db9Kfp_H5ai3hs}Mm8khb^X)4=50%QcV7 zuH7xU@>|iRTX`360)SYIbsXdU9xdV({+0>v{R<)zuY` z9^QNN=6Od)>)7b%^xN6#nT5%j`O)b)_tfmr)ZEC-%E;^*h`K^f5%aF4NlxUX0}+~0YqWq-K!-AKo~;nw|;)_r%&9ymDI`fi|Qx4&u6*?KV8 zelXCn*W0$$*&J$b47D``nqLEk`5QrA23qR_ZH>Ox#!ZlRxRbKCqtVyhw$vyf{r=pGCXxq?GOAOV-lKQI^=gssmx=pPvH_VsRbcdm7` zFSWOx1aND*@%3c=i`kZj`L@P|)`sP_hPgL&3k}bfnw~E=)vdO?SZ#T}-cq;TR_Ez_ zwb@uwD6w)py5z5Oj4e+L2#GOWLSy}uJ=4Uo*)z3%K>?eAFXYlCIMuZTF_)i&DQ+1K8ZkzqsQP*KR3C^Rw>1PK_1!&Asq z4wJ~HqbP)E0v22xKw^kk@Df47M-wp7L^KjsJ`>428kx_)Gf4;%I+B2z)iN8!4NT+YJoJ>AkRpYWh5)JZTf47lQ~I=zi^bQjPla*8&~qH zFJxC$q@OLZ73XJE6lY&3%{*UXEzZ-XB*i(Bv(M(`oy#|8n0Z=Q#G%f#>oOf`yH#mT z)n_`)#X07pY;!@DwX`6)C@;AvC%vLDwJ2MYYN9K6Xa2;$^CgH&YFO5E3LNtJVHaxO!`6B*^IWW6#~ueHYNQ)0C#`dFLJXwxO6$ED^Z8q9Jg7e}T< zQ>f7-Vsui1qWR6;JGZND+`hi%-Fy#63;zgD7lSp2-@bx;|MKnk&wu>+@y~yJ`uUGf zzyJ9uOsszc4CXTg7(Dp*B(U&7@OgHQj)$gau0MKKdi`$6)jNfkZ|7Fs%CEehd*Mn} zd38ofWk%_FOHPi|tmB&`VvAaks1hgZ8u)qU8_%3OJeyHAqhjr@RgjR zioAO@H|rao_w{#9jJxNi#{tGbJM8YStE+we>Y01U33O+68qi{P34W}stZpnUuT9J? zj?XVnF0M>10b=>5m;G;-eIT=|!TI&A`SsBJT5xGSxaJM5dxNVRe$aY-Tbtqd{M&xt zwr_K5Gt4m1`Zj|>e<&2#4)_8A|EABg=2=Y?BGmsf(D%aE^VHk**xUICV9XzW26i&2=x=)xXnP7D zfVDmjwmu0oKk_%$`WqgG=W;#VZhyGb`EbAY>4Ee4!QhKS*XzTfH}BmIfL`z2O@~7b z2d?_Vp~iPZP5a>GaO=@n`{7vI!C1$^Xve$Jj{T7~C}e}}?_3}qhwkp9;l4vx&%r?V zeox14ciVPnOR%***!*UzrGC4$A=K6w>TC;iwg*AF+qd9EFe!T?`&UbZ(Z(aTWo7t zXm6fvc{B0m`E=8p`HqIU);F_F&*vMTEi~0FHPx*&JzH+9TWSD#ywvb$wY6@e4BKAFFpskXKr6ibJY0c(t*>Lv*|F;ES_zlVuetg*Tu$I&>tUF! z48jVI#lda}u>N+?&h)mw?P(qBY#HzE>TYc^#T%kxB4Us+(I`}8I8kUc5|5>Em^`U~ zBjy8=p((^j4Du8*`ZOx~3>p!ELBat>OblH8hm1#3i2z-{qM{;6ScXE(1qQE{GBh%V zR>3xa`(`~)r{QTee6^abl(EzjwqC{t=P2cH%O@3dED)wrMGvD3me(s3bd`b){**CR zlo&5eFmd!cfHwqwHWIK^0MPp-&Gt1L6vk_j9phCv4a!!dzQk>DsI zmJ)-bMWQKDXjpR|gF~ux> z=u?1CC}ITMrVFwVmga*FiWG$)Ma1CY5MneIxZ3F$^e-_e1cj+hNzbXeUU2n(-sJ~b zRku<~DwB#T;&X~)0liW)=t@Hj4OVzU@e;`iLK&VX#q&fsz7Q{#;zSafQlYik4CxLH zP^02JTSckuY?&b=oo9;U#hK(*yW9!@X5}SXSf;pNbhhODi|0$OUdlOt&R$xQQcz$k zF3P!3kzQV$QjlxT&K4VD`D(e*mT1d!7_BC8oJy0H3b*dg%`|3Z8nZJK3Ue(bdB)uI zxSX_vf~>>>@DtBYEy-0|)UH(@QGT%POrU zB^sNVYZMFO6@mn%%&J#dwXziOUru9m=ya(nv4O`{(CK0_k%J>~utF6_V^SoiC6=8p zep2_)HPrX^?bzF?ao1q~z(DWx+sP%si3RYnxV*f&wz{#tzUlD>*VZ>@m)2%h*5}u~ z^K1U4jja{$&XQ+)ZY{Xr*;@AQEPHmAH@4Qi+Z(=Zk8j)S-|_`^{DGZ7aMyp*0rBtn z0n9c-AOZh&D73S+1&)K4zD>`DXKi(TWqEyNWqoCBV`Y79d2JPBX>ECN34A>eV+)|O z1Ya&ekS#9GPEL&uy1>^z4xDUeYGQg~d}@4bY-DtJ7-Zc2cP0klYK^1LfsxLx{>#^{ zE3_)MfF+RfHF{-cR@&K$(#p!SRaIwmbJLVc5t~bA@fb7?mCm6F#2mFw6c;OsmC-8G z<5qj>0$~H&>VF;RtvdmjuM^gB{9TUN1`Fu@;Z0I}O|_d1_dTzF^S!wrX#CF*;=|pp zNALQcozUyZ4RH15(A{t}(sbx+?VZ8Hbyz6kR^Ki7|Xtd*S zq~l6&(A0Ge@Lq$P`AyIfVnMnaRF&rTm=s1xsuY7NBT=4}sK`#z z3{Rk< zQ|#)}^A4$jA5FwWpb3D%a19ZNVagdK@z-d~sc7`+D9mX%mWVooz=3n3FaUm3$U^XW zkR)JWVSqtP0}=*Li^2RFjiw7_>7|ty?mnry|MKkZ$2k{o*h(+NW#ozD%vg@_6o&K* z0u@OhV%Th)Km-6r5-LbSIgT&F38iR(2+il?1OkRy$u?-UX}08|0&7`GYH5)!!!AiN zD;!pJMmpbYVZ|94M*T1L(%h13RfU%;Gb_riMMd_K(#(nq;AC-WHmNC2VlY?!I!Z1L08Arf0Yf*b#Acn^YKluUo9qekY4PTa zxa6$3)ZF+utD38z6WLfS1&tx15O^e#fI{LB2s{cw#OIe+c!RqiKEXnH$f0jApTGa{ z@sA(JKmU0D^XG@3e}4Gm=P@|=_3vk0|5-r_pbG*FJ{9|qmVH~^Si4qpJFn_|(Un_8 zSD=9vT)Lir@nS*c#k_OX8Ksquk_*<7v++5ZQj3lcCxELIHUKbx_1tBu>bb zQ|S^iO++H|aCkQO0pchE5>-g$Dfx-^#Cwl^bC0^Fr{7MDPt4BFFE6dY`as}4OKX58 z>uZ}HAWXik^$q{hns;e^bJ-JE-Q3#n?RW$G9{>K@=8h+@?+YFJLI<0{eP3|jAKD9q z_JaTa1ouM0y%-yt1NWOf*X#YE`h(#{kazBe z!{Nrm;pU@}mJj3YA1AtwCpwSDAh!U$K8$pJ818sK)c$^`<9N91y{q%6zvHl{?R{^@ zac|d$p3dXGo)3N9N4*_~eH{nA?MMARN6x+jXFuT9k*oLE-T%?m4^a1Eu=fbiuCME` zuk)b4`^YtL;2Jy_8af;qh8P13n;hGl9NQin-W?y^85;pG+Z`VV*%=?-9vKM^4F(3B zp6*UqzR}zS7&hDZ2B2&5)zgU=k4Nij#-2Zzd|5m3{K3qN+L`Ayv(F#QzpS0DdjL9f zFKQNFJzTASveEL=-`xZN<{t#a>ezI3cwjjqq?gat?HlU#4g%BaUmxsU3GW03Ivei( zjnRSCFtUm-C_Xc8KLEe3;%L?VH!MIbRzC~PDOdm4fK6%m7= z;CN~g4=$c76U7_E@g||s$kpk9Q~`kTG#ZXZ%h9OW8YM$11FgWIP~5(R8L zpGK9kSV}fu!xrm#LM>aW=Sy{bfr`yjb0lV!Iuim+o^4iUTC};T`rK6Tnd6Hc8P!F3 z*UkcH#pl@o!;*@#OgRpQQUZ5%!2`w+V~9i~1&`%WC`zFy$*9S;#ulU*@&M`z%WhmY zq^3sU;f9dWcuFLebOr+eMhREaQ(_3TNcgym8jYtR@Hi%qlu?l2C@>Q^#0V@Fpe_;z zpOXQ{I*lZpipEC7;39x~MWIedAx}jkPa)7}P}mqe1w~{car7u8&@aO27#u*|3GL2A zVMFh3P-BIVq(YZe#yI7 zR&?cj>Gi7o>hq5B(wqwyvMbJ7vK%6v633#WY19ZD?pIU{f`rA|Tp zG6@noQ9>un*rYJSND2-B7&8noSPhY=l_lwQsU}@=tS(7svgs{pv0AfSpk`48L;?%k zzhO`W@b#dg@aQNU27FV|M4U`#7@C+q`uO|t=RZDv`Th9IcldDY+mDZb{QUUiXBc2V zKLD70gU`KS>3kSq-@|qD-@gC(0Sp!#}#_4T}q zm3ftAMHkN(oU6<#uS^35R(w9Wuo&FIfiGK}s8-nxs#L8iMWsvC%1siwltL8|0Kh0h z0^k-+M4?Nl6cLpQzIP#wE@fFVQ(m_;OoA)_)ZFac((0-QWM$O@lxuy%x9;(8`nQ2g zd3;+No1wMM;QHp4H?Zpu0Z{D+w%);Z`(S(T!_MA^-TmX;ebBz&+dJCXJ=ofQ2NK%a zhY3M?ZH4x?LVJK){|*4#b|4h;`~5zjAC4vZf<7;>u}#qWya3cdE5Ck#u@jO3i~*Rf zuX@+lHo|p`pxszmSzcV6pP6|(H#f7mFt;!_voJd|H#sJvBW&^%f}F?8L+j z%s6OgN5|f}hDP4hzsb(ch)*<>mK9W8tSBkXcci7N)G{WE!eElQJf={@5r{Zag~(u3 zCz$m~DLQMi(ryyA*Id{hZVL3j3J$yu^#47Icml9s=c7Q!qfjSYV?NmWXr~?GD%gB~ zs|C=jcDL=}PWz+Xj>o&5kN0{Wzw3wk1#kt>wKMPvWN+xryOG9&ktTQ$sMtu$`?0p; zNdT~}_oM9}hTA?4w|^Y&{OIm@KhW~7yJ5HURj8%T|K`bNUG3(Rn&9&%dkwGOw>BMj zw0`XAIPUBE(AV{GpznCl`QF(FNcP^@ec03fuDxZar6KU$taz*P{vuKvy80q@u#;Ff2|=>-4_ zlWaLWga`n(FwiyI2LRSF)6+iH-2wnM+S`5g;(0O!hd@UH5+ShBF_>rs7J+|t z0JdmUWE3hAfsMfs(ST&2#gnllDiKR0#GnDaFfrgdi^4=A5T|0IkYub#E0q`(Vxv+L zuNTH@xf&HurvcUl=p{B7_yAxEC0C)~0DviFEQN$3of_E};9?jQvP>*Zj*rhz&#gXJcJ;DE9}55m3pY*z z(M1I$0|%pt8lVn!2B0on&=G+qA_(+I z6y zC|7OPQTVhd0_qeR@hhDD9s?^@se~vr0w7IcPK?Vz0x@Q1Y8)wY zYrN8CR@sxeIxUh)JA=a@$z+Kk)|{JdDacC9b6E0iDFu$S;*7MS3{`>#mcY_*I3^{A zgo(h#03nkZm28E8t&;HcN|ss-=tY&YiDIZEN*(h@vkAH&vJpKVoIX<71&Y#Qx`wAKM4VE3kd0;0w zak8y<+c#>jD&plvyTx8ymUX@=r?M>nVo}+piehkoaP~YLLM%R?T2yYy%~4wuR92(L z9;Zsxsgl)llZ>Nak@$G3h{TjpsX`(d08C7!Nf}frlPYCXWGoPlN?38Z$~`(UIXyo; zyR^IlEXcp+39NgA9$&}{=oJLG+5_z-#MqA4zZcj#2yGo~?;LFH9E1R+b`N*dYs)4*a_^`Ksw7Yi%&j~YZD+qA)4&(%2pxxTq3x#%oet~}ifj}s@wH1Uzid(^8 zz~>M5ceaAt0BQcsKp18|?}p#wfklkrIz~^p7Jms=dMwN>EiJ4pE-WvA{_On1>>SAa zZ1~@WnQ4&4>9>n-r{*UoW+x|RMn^}V)jd9UzVzDl>Pwd|SdvTUKwZ7 zX|!^Yl&924%$C?xYrNGSo0_DsXjzXgW(8dJ{{B}!=c~ZLi$EWgFbJ@YhamorhragO zK-(jKb4{@ML3q9woU^>$`T(|VHGp9|fMMN__xqpk54_wPd<`c@JK&U zVe4*uH{5bG*7jkd^LVoJc&z={-S%my_1M*N*x$I_TDSSS#`EId>XX~cHP@GZySjMe z((>(VYxi#XYVU15eYp4f<-5j)gO=v^9c_p0t$Qua+l}?U=TA2t*DlxGTex>;?#|7* zTQ}$L++KQcZ{fkcxtja)j~*^OeY*VWyM3~&d8D&txV!UQc@cv}rZd6+mo$!m z$rE#gGJ#wz*XWc=wNxq>3dKAcn+iNE1_OivjU}Q9WDK22q0lKP94;yb5gmg-Adpcp z(WjzLqexhZPO33$RTixzP9@YUP99irRC1n5DbT2S3OQRUVTwd_0iVETA>kex;06~F zML=L#1csU~w5Zh?3EG@wV?Ny6IljaJ>-~ys@c>|FGZRYfreb?SNmgQUwlpaoD`4Y< zbhcI~G)jbVB1wWoW7TME8l_dKwyIRgO1VYO)`F`RTazAdDst%ZQ({XT$rticD+`j( z=Qu7EWnVjMJD=;Q%ulZ>OexLL*pjhq4%}u8O@v#t060O65zsUWl1zd%gd`H0&Ey$0 zN%=V$rDY6>6r6w{Qez0fw8&BTFu-ucXc9RFPOqk*NsMUl7C3>3$I!8cWOaH`Do@3Y zCSs%Tgfl2Se3}&nVFLna1lm(EsM9ELamGaxN#L>>g(e{})MzB>bQBymgeSoA4T!Nb zF+i}8;QU`f;AenZ2FD@c&?4px3Kxw7#!SbMX($q$D~(~Z0l?@oElsMX%9KQjk|jv1cfeHsZNw~RSdF#Ko*b*Y!Z@;j=`cKy>Re$21O>~ zsU##GUv=f?&i<$4kAHlG687iEFF!v4IQ{XD-~alruYdmQ(~rN7|M>Ix`(K~G|Mlhj zKfZnY>-)DqK>)_S0;)kL`?LO%#YN^^hcUyfx5ulK!R?`ht>!RgbU-Y&oXJyjX;KGyg!K>*3t*Voq9S5{Y-mzS4TmRHtSR@Rml zSC{6Om**Fk7MGTnz#EH;iwg@&^K;8{v&(Zc%d;~}voi}o(MCoF+dCTD+8bZIe3F@E zm&-+ZovN@f_uk#x4{PsTyINgPn3I;4Vs}_FvXTq)ty%V1vxa@GC}q9xm9PJm7d{Mo z5p)6xd+h6a1i0k~=z>$hYJJT${^kdP<_E#n2U~3qg3b3rE%!ht)b;>A0PB3R1Je6! z#|gJ$+39}?_sQ&gzBBl0&;4e9q~Tz+>0qS!XteG9cn37F$*vEh?MDMmM|}uUwtJdSm*=t?660r|;aExqok_re?afcINTp*}5k) z&mYgds#~gmv)0n$?e6sV^#);GqB}G^yfr)m5*!-x55Vcn{=Ob>Z})m{*J^kBT+5rW z=e5oU*Sl|D>b+InfBSOp&FY@(mA%(5IB!)A-@P!iuE-7y zDX}yyxhx~GEIs@(9RSQ+k!h}gNQ*B@x0GZVbF6%m65LfYw0vQlSZmt$qB z6_G?TpjRY1Tn&#Q0UE`COFxZ{p)nBQ$zD-J5`sdbOJ(x7crsrIObbD#!ceF&6j~&a z5=A7VXfza^hN9BYR63f%#L`$8aD+-=im54C@g}>0Dg?m8o*Okr=3G;GEM4EMQdx5*>+w1t8$7iXjjXBqAWw8SLLIJOk(zi8&RG z`xSwWz~bTfCz*^SQ(~y(C<+nFWmDx!svOdbB#{xtQi@CoLRYGJ1|waiCd=gvwd$lI zlrEQ(B@&WULRU)pMx`tzR+Oy!CDx`jr5kiMtujfaPBkbkI*yvhQL))FCYp+i#Gqgq zD>?>A#2`snG=+cxzZM*tEMAjbm~Jb}u~p=l3)76*R(*PkG|@m7vk-JFf{KkHp%7#Y zng+?llyC$ZnIu*hYfp|#x9Dwg+7!JmNvBTG&;;NY2m*}6rE?TQmW9t@Zw~ABM^3_ z4BVg_1$-@wA*Yaqc#?pKV-cc>s7PpF5Mbag22dGC1mz5#&2?Ns2p8=t@b`8C}C3WpPa ze*W^u`%hn{7A6ZWWXt14qF8~!W=Jo!6;)@I0ESf+mt8!URbG}-P-rhMvKHsY=cKAr zjp`Hw;4V|iqX55@Gnh&yN5kdE3RoHzUCspbVkkHaKx;Lhp%u|J0-Bt|2NHJWvU7ZD zYH?%5v+dt{7u-G!Y#jx+kHWYDd4IyKEx2Oj@y`B-6KH`JFbuj^*t*_9hP@Bd3m!iK z7@$`$1Y~RP#K1syfV71H2C}oW2eZAqv$YFA1zJF_y`A0nhld|Oe0+cSVHdoD2f@35 zaG`)d;Pd&sUM~Qc2YzgLHa0c@$X3@pYir)smGEfe%G%oM+W-AnUtQT)S=v}#_N*>% zEHAG@r~%uYnV6m)A96dNJgo)9dRTL}yQ5`ncxZfd*fr4C+0pXq<D!vvsL=f}zJ;gQYwd9R zm8I6^#n#68=GPN19`@b6+Ipp;wYnUnxw54BLQ%_w!p=)2L$@zX)Lwr3_}c8V8_RF* zZnQk|cGLxW>w^O=!J&?S1FX{r+M#Zr3rLvL=XQo!<@C7vA;SQ`U?9mn!-E^cgX?bR z>JVTU96to~f#fpxYGceM7mH9mWE@811ekDokv{^Ieg`nu-kS8c5gJ)La> zeO<1;?*8tM&eoa{1EL%hi`IUI51n z3bOL@Gjj?uGIDHL`3^^BYGO*f)~FWAg)9M!%%l<+6eIy3jloAEfh8bLMI%llP|;XS z3?3arMpL9LeU=6A$yDq}EYGx*XP8UUlFBk*r!3QQHY@pDcIt&3(3(rq&Bf{Q`8G|u zNt7t(o5V`1+L&dG&oRa4CdB8&Clw|o7Fgm7%;wURq!Ozt)gVex#TVL=&gZ0DEJ&=# z0x_S@NxNF&s4hw-5*LXgoIzkA%upCWcPxX7XVI}NI>1CIS zl`M^hsn+leu^gR&s#MXGDz0A7)9cx4HFy`0jHOo5qTTP6-&nB$OJ42hoRuHbRw1kcOV5}%6W-7R!4bG`nmkn;tW$} zvMgRrldv#MJcfb8&_VE6CIQbT(}YZ5R06e_DdkBFa-~aQkfBnyoKmY5??|*&y40~SzW@Y&onKqHeD8ueORu&FjW&g&I5Fp3a!y4`QDxSJYvuVD z3X}5E;V5>^ z6l|*nz7>KDdk|>38)^j-c0bU3FW7cJ)K(Mhco+odbisuaJspqW#OJOj!Tu8otKT1K z{4mk}aSA^DIv#D`@2~eaKVGl9yY%4t{Ov2V*Dp<7shqrA`S#l7scV-eu3VhFd~xd1 z#i^>w$qVPl%g>IKl)4LxhYE|_Ma4rUrGq7<1Epny;f$77j8#;QS6&{gx;k?4>TuOn zclGtrtG6fb+@HEzGkLFO{N97HyZ1+b`)%mf4cE=<&TChl*RH#6-g4c#?Y?t&^xlK1 zM^ENnyqJIWYO($e$o#99voBw~eO@#Vs!HFS&3jRr{i-bYRdHs0 zSytDj64&kX0ASNkuFpTewOoI9z4ei|^O?W*O>nRo0_;EZf&lCB5B2%oPT!DoGhC^$ z;pztgegCivFl@u^+!z^LA01p98CY{Wm%~rL7MxuR1Kl${9e`d_U2T&cZ6j?h10Bsh z9Zek_P2JtC1I})jt9N9mZ_GU~G3=Te8JZq-Pq|%VLj%KrW^K*&ubR`#nwPI0KYLR1u=eil+gC4Lx=?Yp z1o(PJR$6jOiZL!xt20Vv3V~3{<%wBbK84M|bEyn9Kd!)dZY>qTZuX31D&gWPz`@Es{(F?u8&z0K-sZZ~_@%0}F6PXP{uIIuprYBWO(6p))a1Gy(I- z|G{1~ffz-^p=nr6oWzF#*YF;7f`C z=TJy!77fRv5d<`fghP?=NI+_NOcIYt;4+9@aE0fPc^n*rj-ryIi8v&gKxENL96Gq# z6F5{Hi;7~BQA{eH&!vb(Or?UW)$k2^eyl+hZxWeIY>gIp7)zt&>-Ai%maEeNwvnY$ z9FK?R@yJ3URVty$WWQ*Wk3iy+i9#AdNM|bfvUsID zK@CR*H3E%UtuQG`JPMKmH(MdFDHJ|~!e>&2EVfd>RttqDl_Ws}^oy+(!isq*kS#h% z#vsaJhpJ%F)I5MLnu^cTiCG3Q#|Zb+;+i16KzMO-fk`U2NJVkr5>El>A_;MTVI(0Q z&n1xgbUd4aAmc&MR6L4;2AUX&!`N~Pmo|6bef~3?9|xa)|M2z4x1ay~`CtF*#K6A% z{^#d#;8d`0aDed>A-@>II-+%w`?eomq+w(VztOdr@{MfXTgv@eFdRcs0 zX?#XmQc+b#c12oZwpr(hlc(sU7M&s`R-GCvO;iJBvo%6NtXQ5XlbXb0lTeZ%6~=?_ zNCtkJIeH0GE23!yRGomX!Zd2cO`xt|QPnJ~%i&Jo*HD>EPWl$h)wO0f6o7zuy7?+dT;G9)xxeK?_J0 z1U3e;y&nhxD(!5Ac6YY-cXtm!yA2dAJX9D?2omf627Z7xtY6-B&+6>bY@fT|=^mI{ zTv%OQ-&pr-u6qMsPsr!n3Iw-;;m2tHO`qQbvbhP_1^#5n7w~UxdcA92&#HH0#k;l) zXHTy#tu4*3FE6Yu&8;oWt}V{4&d;pOzFnS~nwuD%7;=sc_Dl`+jyv0*Ke(1|l0UA@ z4-U8b`{1I8zMiM+?GHBEYc@L`Zno9L1HLET6*IFKU+8=^XwdH&cD-wqr4o6##hFcB>8$<1Nt1s@()?Aso zSv7I_+{9&ouBx%>^CQ*ghc8#TFI5g#SB_k)96o=}U2%4#qQVV8Rai7oQ0UCdAILB4 zFDUFUEb0X*f*C9+A1XaRR8i$Rf5~~aYM`QO;KId$s!Kyxt`6V0>Arqr=sIX`4PC!6 zaOFz>rRv`5i#-=FbyZ#LxKP=CuCnvO#lFi|`mS8-2Pgk__dglB^V`6U8=aRf)|VGO z%}alfk@6rdsV+aQsiL5{qM+q`q4UPMvDzz>kFU+u-ClZiceUZcMk`?U^I%_nXrLwJ zYWKT3e6WvDR5kPQ#|0Ki6E>qE}fVVD(n|C+mh)!hg5 zYu?#4H_$cH`!~QQI$DR@n}>TlhFv}Gq5iRv!HH4V}@y=D1W;_*x)>p0-TQE-_=JSh?=6oG^w<0w30T(a7nW@Jg3C@L<7L;!yaEJ`Gh zfL#$ebRq}%27|<<6WBoPXefX#J`*oski<-)h(VIDs4^Z^0#L-J!LTSoCPT`jOZgNL z2ZStQvSd7tTtJ6=r!gsfI)O{Yu_+if1$-W=R6v)B=n5%Qt>DIjOJai1WMpeK45bpF zi?7phG#ZvlMGy)j$mCNP+!-7`l0XFCHJ-)$MQRkW)l7s6^5eVRsT z5wew(6aAt}h(rO7Dx!%rGLBM+W08O?qp9Gg0EMA}FMyI!RKB#ixBv0agD-y_eEIX> z^N*vie|-M=k1zlD=a;|!2|AyleuWtpX4S{l1t`_RlZh|M3}W*^lF|zrX+T zb@%wd|IXh!_$H$~(VVTxEKka>bmW}3WuH&ZypWPsZpl8IoORBcTwsn(k5i=Tm8r2n z!c@tz3X2|KTM#Ri#LE;Gxy&q;CQ4OStu#d=N>qpvl;9@=_p;Lo83rJod{LZ;Z;&by zO^+H{=X|^CyGQsZ{UiOYSI_2ME3j`2G=^ODf!;cQU!Aw>;U-8&?OMw{ zZ~OgF&tq6m-~KSzS_3c^Xs-=+)^2yz0)}n3*KD=d_}cD!+iHAWaLL4_rn_@*Zq2^F zx!CyIdRtAP=jqPC%bmg3JI>dk|BtKperhvW_kCxtwPqYJSwa%ZIp>^n21HKA#yJ}d z28?ZNOwP&185?7wP(TtHW0H;YcxLUrPu01%&iUcqI)BLR7PHs6b!NWR)q3?xB)LER z?SA@sx*CG_s@~R~dvW>r<1Bxz)hd{?mhl&aY z^Ycau3kLIY2Xb=yv$A{Ko?cH@k2k9)E2}FjyDKZFC)?MPo!9Rx94IX7FFxK^P}ZAY z(vx4@onPErT-sM&-cwrETXwt;P^zL5bh=AQdrHcBOOAIHm9`fYw-pq(7L>FVl|Cpa zZq6@iEiP>_-GhvWGFN^5FQ>} zcsLY3*ajON1^@%D3!rOu^shvSaQ$#NZodY*UiWv79RLjOfwesw?tT7b`1!L(uUoqUH}2l{buYL(8SoY*DoNe-n;~zr>~yF4!CXdB@le}&Dhh| zW3cin&?|tn7f;4tJp=tWFP_4ez;djk0DmaD+3&9^}1*)+LU05DyF&06lU9rxN#daV@>@44*UD|xOnZe5{H zmL*f?s+xH!Pjz-9o30e6*3!XxPvG@FWI5l}2VnoGuTC_p$ajliQ4c~l&mh+*J> zW5M%)wW1;+-taUjiwGVvh*&^48Wu@~rLNORs1!09K?g>Kgl5xlLJmzKBT2;=9y~Ul z$z>CGT%v%V$zdnralauEe@jmOdt&n6jwSuuu|&}MH8JfMHkjQ@$Qe8=f{R14@JJTn z;FJ}Tg++7l>Ew*V$YY1njvYY&FPRRe14)_a6dX1civi}0$|6aWLbiY|5K&bcrp~~$ z+Sxg|;_L#Y(#ogF;VE5!7NJckb7{pE8CS=ZTBN{-9YG}g8>~%t^f!RUv||aFH1I+8GUHz2@(zQ5c5w*Bq<_K)AUe*C`u{rBy!-*;g#upiKheFkO=w7myo4d@GbIRd4VYU>htoaISp~jT2 zQ+YLFn^I!eh%8F3P6EW%h&Xx?ThC`3czlz9XXFViLXKI)(DN8NKHxN7!oW(X1Ua23 zrO@P5nv}s(h%YzZ8DChQUD;aL+*{oIu)MLewz>0u=i|=ar=7k1?cIG?Y&u>Y4YYt; zptG~RzrFo&bL#{2WWc4ZzF&y0PR@lMzZh@p8*FH9xpAlIYF)#{nwytu>Mnt{y5>sF z&AK~x?zOabboUPojXrwv^2OMj@$t#GZ)X6F=KO(R2=FJe05}x;FU2tyUI@pR7EibIcf@`tB#!_f47Fvyj)*```a9}w!zZjT{%uI$R-vlPc{cm2+ zj=!9K^>p&pX#MrGIY!yDmfFS9)=1y2=m1bdr0-^=_hzuOI@onR+H)h;QwspL2VCyyigDW!+u1?;&G=BTsi~3V9>Q9f|Ix}(i!W=+g&&_c6%|LV2 zbi>70*G@h;S2A)UZ@40Fs4Q=|H2+aq$-}bZ!Qz76qP&5U!ojklf#QOJ!u-BmU#~Bx zKQFg8JF7d(+v)LiI9;7CcbD7K;dXa;Jb+;xo~#Z}c6UxecWzNvUU8SNsLhkx?DXDu zcv?K!54_pUo~)Lv+%{ifM_yq|cJ6&oPIH#;L3SSKG`T@P?|xQpQM{PV#x@6|x4mt?PKvql>4WTs=2>_1u%|7oOK#eARIE&7JF$ zcVYbDd|Q2}voX@sv^dbZJlqu>>Wuz%_N(u&mBWWnq2YnRU>|714+o(egA5x141-%= zj|csa2j}7iE#vm9dt$JAe4zVvU&mNa`^)b3XWi|Od)r?;>K}Xh@bybTqo;sIQ{!V( zZ=hx-UeCN8pMLWyzFhngXc}Ax&yK%5&?Ic(ozrh#gL~pnjlnx#zdRrx==>j7!26HC zeD>xx@?51psC$br^uzDzgG~nU0(6l@4o#P3co9bJS{|4oKzG$g-5OY!%DI zOJksp;ZjpbsB{LFX%HJqJ?^tbo^vHR7s~7vK1G&^B;zC!G7|B~Bq+?G48$P>A{h%e zv69i4#5k*f<%$Cg4Rkm;<8TrJxUocZW)c<$l$?o!TTsXhfH8o&1T^L!NaWws(*KbL zOjX8jseoQE4?S>c$ymTeYzhvQ1_pZ`L2Z*M%o044a151}kby|Tpi{8uG&}~N3ji#Q z0FZ-002jrkpt)4MkVX&Ib^bw&(*0k9yd>~Crc$HnS?5rG8J;NSd>N~ z|2;kJ?dNb)}tlm8Bn`lbBBQL-p9DuRPaV`Zi>u?Q9(6}LDrJY7h|afxX} zB-|^?NCiJn0t%UcLZuUkcqSv0LcozRJPAjpm*{PBdzRAUQ+jh{`6a6Sa)ZIE;2DH$ znBq?%_vqzrjnb`Bxs?*L7|$UjVp4$p0>WprA&MCU5sf5c0zXIBis%{{OQ&Y))fBaqs+F-#YPLzqv#14Djl`im)d|Cjog7aS2alIi|qVq|Nc`9qM$&hDOWtn6yJ;AQ!;Kp8bb8mTbZ|(hucke#{T)~plyRcR#&~ChiwW_!GceXxm$8Fil##U@)eJT*@ zAANfB?)}P3S8~fMT=|989G}tcHMl%_r(5fAsw_68$*i`T^>(|(;kJ3Q-8uPr#bxD{ zr!HKEneJ=r8k+7mclY* z>;0EwFWomKA+Wj>*<4$Ezp=daZe{!3>h8PckDEY? zA2(y4x0k>x;;?b(|v4+!A_b$J^b^hhG6Qie#hmPm>m-xE#y*+tA z*?oDw-h5wAL2g%WPG?SbcV14nFB=%FzU=HCucyoH?#a#q=xTSn+FdRHuy%*D&1S!E zw%oT`+FY(yr@Pe&p3P}><+M1nftqce+a_zH(G1j}H{LRsZX4~523v#PT&FkHYE3uw zmU^SDR&S}*S#BEbb!JD6$y#Nw09EVF*VOup62)n*_#{VkL8@x7*_+(n_JX3p)2Bu+ zpC7BeHdc4-dCldQbywfqu6o;aW9Grl*#|f0TN^?hw?luGW^P{`>4N!uhr7cgJ>iGF zp@;nkxC)O91p&Q=`oa$(z(NOymjT8e4*4I!WkWwOVgKd8;wQkmUiWnYdX4pVyzK3K z)Y+v_huifo#wwHV5S!#8@UQ+;<4Rr-} zW03>UOPy=b7g{W3ZfjYVxx@oHrV_h4->4}t*iX7$r(N1YwWY$EeKFTsZr9}*j0Glh ziOpDIQRJw^ZW&QQOC+TylQPnP1z}*(LW;z#%Q{n(d!an<;&EG9HrpUgqhXT>aCt3> zfJ`RBnq2LMY&98N(VO+z0_MFEV#RX98WFbo2;;^PV2%oO6$G*S|hl8&OFP-Gx9u(Svw-dB1O zIIy+7AL}1}yxWgEu#aE2zy0{|^N$a||FIJX*zV`=ar+eq7~J=YM+uq;U0#57m_U>nhuD#C( z+5;_M7yuY>U^^ez-tB~!)?Q3a-)?U|e&J$vX_+H8-;$MUaAoNnZnf2}Fj`~=lUSn{ zs5AnViX{=l+7oOZnZ+ToI7B9s#AZ=gObUxZW-?fOo=l@MJ8gyKr5CEM-MriQpu4mG z(eR_^qqke{s_i-)mqeD(=u(DIBP=>z*wxedVr=a7>o;#ErY0t4Ca3%}GxKwE{)5Pk z#b|haDY~`3vh!~3!~6BmyYIg4ZT#Hd{Eu(j|Ly1QfB*C2fB*B-AK-_3x4ga_U5SPl z7lP5CKN^^Y%Zano3$xRJVS&ks>4}%M)#v#PY;|Sc^0V&6!6rbjg}$57f%?UP2B2s! zuw2!PJ+&)+^(#GffMN0W#!`26tg9*t0M=F=dT@2-&V`A((=V=7j9xAqzEIe8BCD<3 z-FDp5Tj_gzspQeQqWjCREm%>dw<*+nZ=t^>Hr%;5(j6J@1ON+-^n@Pv1crNpBYh!QMP(p3&>J4^ zUw}Lt2*z_q{dN2+zV8(;V%Z0D5P}%bAQewS-_`b{yZupj`>WBR$yd+*MVgt313@6o z0$f32!OmP<@r_S-#XmIxbZ`+;ZE757Zt5?p{U7i<=>O{_vyBfA7d&Os~1z*-#pSjp-DYn|m9l)Gv3k*_^g0ADj^Q?4SIm`oVuk&yvUfT7Sw(b&UC%;9wO0l?rYVk$Bji%%nwp<}~l zrpLo8P!tS9L{nL%a2kVQbQ^-j;%u#GcGAT2aLP#YOQ*hX$$c#fUj2v+! z4f%_}B%x_I7!i!E*r-n zr{R$ZGLkN(^L1Rlfy2?UxCWlUECTPz(h33k-BD{oylW-VH5l1a%t9fh{o2g_`WlV~K zPgY1MY9#|8+G=6}fLV-UmqqF_DYMM#EQ8!3Q`p5aGmpr_ARxU`uq=$#XEkRVF^sfC z+~HJW5|W$_Oc;U)10Yjzn1T~${L#$=fW=o4e{AkU%k}Y(KR^HZAA8?_?tJ~R_XVyW z9z=8OfBv!m<&U`j{P_9Lz0W_^c6a)p4){)HSqe?oB7?P1YtB>1t!%(BTY)bBbe6l! zWXw@m^0oFNgQLV^E--7d3=*4)XH>8>VzEh~bm*lP6;CIjDE1XdB4BD_38c2 z*X_M;yL+%uC)D2Oj~~A70i*$XZGC*Vxf@$~_k3dN#+|0Z%F|gTWsbZ;dv2l4S7`F) z8k|{Ln_FqJOLZo(4g!oPS29E*DxXW@a0smU*-r))O(mw`P)X?YBk4(pQWFlRCLGI1 z!ZPqYrBH5C>0Cx_hpFscP zKfZn2+kC&W2~sJ1JU4IBrv<+p9#;-gr)(*-U5JKzjlT~NXxS+gAZGygH7Sy z`h~u_NPk_VuMPlgsjoiLRTJy3UGA-mb=53&)-HF|0)Q=cS4TRpFLYdAXuUpv@A72* znU~cSPcD^?oGKk=BfTCMURlQu%AeY}(DDNm$x7F%KwfZ)olv-0KSJa3kRb2j6HunmfdzsC- z%w}H!EsK4P%e%oB*9zrzB4vX_bxWZF?*s0yldBt4`bI6lo2}X6Zn0-Suw^~4``TOu z9iE~#Pfz11+;*zmToe)3NJ zO!KX&=K9Izn{V&cP2I1bYj5;--k$He9qc@)2i6?wYYq*xMn*apA9jU@Is$`jA<%gQ zQy@aYB9;(cae55~NBV=0VD6~Ut&w{w&Kn{a@92k!u05(IwTKOLH%nVo@U6y8owK7R4!^tH3vEEAY^ z>k3T9Vw<_t8IRF$no1m|5-X6o)UGSA0IF&8L2K0H>GVZL*J+Qd(hK^E9EG+(>pbOg zpYqzu9L55(q0po&1Ryj5BJ!*f6rY^VBxbOw7y&bbO-9f%d0LV8c&_(E9$h6!rC?IX zm}FucVEFW8fDi&EnUHxb6MZN>9gN+8Oi3toLK^b7W68fAP5Sjn@~?;E0-d8thm+Ed zq#%-jwaUy)!DJ+%z)R8+GT;&Bc&I(FFKMvG3Icg3BO{4OfOR|Z_+$b$orp`rXJ(M` z@qlou(jrk=6%-yV9S_IJOt^7{gIOlzT%J};mawsW;Js*g*fQxdAy*@3DJ3)+pDq{D z6+)g~!853NMm9{sC-)0ip zEieaTmQ|Z;ReQBEn?&W1YF%=+oR-PRK+{otmFQw^)tT$(DFS>V?obN;7=o0JreV|Y zsDzA^#0+Ff<@vzk)*9Rc`xzHBVOzjv{r<<-KmOy>@Bf4W9p8S!V0x$nfc-pR7!2b0 z{2h#-R~xQ43oXtPo1@rbFEUySG#anam@W04bmp9JSqte}X{KiF`;weV(3^Z_!Nogcs>PuD8YOUv{)_$Ov(sC z21Uw(W?D*DSyX4QUTE*>8+`Qa(Tj=a<1=q2{j+m$)iDx@t}HLVTVLIIx3Rzd{_Dq` zpI`U?|+3i?j@T8r!rPW2UysDek%BO!B%h*Ac&l1l)* zKtsR1AmLT1WOaH~y-roHQQlBUt3~`P9Oe}c;1IV8poqt>=JIN|{F{8?O`+(fNPJTu ztPu#WbGX-7oN5+$N_c}OxWN}xbNMwq0eH4vB)%h2G|M$@YGaqt+G}z2Se?C2Z?8MI zKf7o!uXMEZ)bkUUo}IY-;?%Y0r>?y?b$#qi^|Q*W&p``xw)*9zy62Z}K0bT>>G`Us zm#)36u9>`ZXRh@@u)8(X-8SETf2Q^BY)8|4=RJQ%(;U=Ye`nKtcT=$UUa05JLhoGw zu<$@@c(4t4F90ym&JVN&hdaWLxh=K-a`TC(z68mZ$CapLafZ(bf9AyZvED%hQ3L=~qu@$6w5jkNJT~dV2tyzgPu> zPU8vyO}!0HPX=cuLo-uB2qAzXNTUOW`DdWwcTNVTr-D<{K!M3A@WLRx@-J@9L;`cb zwSyJ_%s)3h4Kq(qKYBV^cD6$8HmY(=dN6R6IL&2VW3gM~gW(%sF0L;$Y4eThTw^?% z!y4ZbGnz_GS?4^~<5sm#1po$c<*amD%k7p@o2A4C@CF3jnGtBrNb@M}eJ6Nl#46IF^b$l9+l3 z;3^^I@X?e*M^X+QP5lkF0BI?IKazAfB@Im>NHtOum=6<&1B#**+jC4 zK@l^7Wz6JKu{;`C!llUsbeV{)l*`N}jngT!n59;W$ZQZ8G+bb$U_7COst~iZQi)y5 zP)l$k4qnW`iP$&+BZG-gr=T;Lc#4QY7BjFs5{ga0@W?m;6-fu*5-NjD0)CVt=h8I- zhEB-P3X`dre@i)b2mmY<01WCEu3gD6i7D}-9!#x>ZIH1Ha-Kycc4$;yh#av=h^A*C z$w>HP&>^4kEDC`^Mw789A{x&kFy#!XO{~t=XmU-;9FyFuQ)X$EUX9eLlDTyNVEO{! zqiw)`8GL4*UP=^mxn`|4*Jbe8_*wx2Ov%*(rdmjqb15rFIftnc@C`zN zfydXeNqk%iE+w6e!t&_^seq`GG0l3R(<1TMFrb{PPe#-%W2YZWF@ zCJmKAM%#0}15clKKN`_GHOcs+$+(1cVmg|d2?2&oNkXA2FJ2EX?*M?Ue}H>no1gyJ z{Pgqv{`U{x|NQ*(pZoEH%+Pp!`F;QE@B80=!Vc6=sLwxkK74*NGf{r7*j;S%mN|i3 z#TILxM(Yt7vZRheqd7;fbE`BCwc4gt+w@9{PG-^xwOYPf!&9k+YL(e(b^Ba8izV~Bo@B8}hw~g&Do4cR4|7E{Ed;#d%hXDJw3$AZ}2uGJY21d?Ytt~x!t>EnppP>mRl@zU*!O{Id7&KlXoq z+1=k--&~2VEiSA?7nWeOJpfoR5}aQM%rDH(FUaJbtt&hP2&M=N6(pd%L4_mIzHeH&$ePQh8 z=_l8ZKe|%-2w?0?en*k(uG@G+FTEsVpA*whaY>~lbO9#46q|9LNvRUDZ-}_nLSDUC za9bw6uTeE?)Xgf@eTAY$scMm{nq{gMm9|x*Yt!gEbOz9A*BQEumTse^LucwTnA>&6 zcF-|edQ8>_FHc`-n5$}OSq>iJkS#!?g;zB>zp%?pFA;ej?N;JpSr0t2ufSO}){ z1>EY7=ZNYD(1J;%M*8Re0@z>(o(Y3b#nVUi!O;1EuJQhkr|tKiv^72JY<}L=_N=Sz zVQ0(R=cAF?w+k~9VW8=^A;6m%Ac!T19B?I0Ebup7n4J#)t1~+toSOm){Uz`)uGyL3 z%uHZz794D=YBCRG*t@5c2 z#U{r|VA>$t>=iEi38x|6*W?+vW+{qCMsk4tVxt9Y1e=Ovkx(o=hKWNlaH+t1kunmA z8A%jOIt`D2DUCAIDcCd$G)c!W0AQ&{6Vr|*BaS7e9Zg6*oRE6tX!4Ob$&MzbgZ4LI z#S&5y5Qt0~UTIdGI9q!Da^=Z0<@v=vgH_KH^9VFDnuH-Ri9|jlgHB9^(ePLV9iL9d zr4uq!aF`SV8b!xqnHZXoAU6qBW*I}wz%cL_7LhCk&WQqv%b}8C`9?Wa4Cp{dl89t3 zN9Sfb$=w#QT~F6YXeu#XBPB?CWsPDmq27)+grr4xaXpRE=1Ofs%n4#c-9#BP<;t0Bo*hfrz1rN^^=qyvm4{X*A? zs9FJCFJc(PY$Lc;D|8y=9xF_Q?6c~9MxK_P$pjV>g{Ed8$jB62S}HaTflVjS@Js=d zDQ1hcLTiq}S_nLlLE+H=l9}@L&N7=R-z;^hr5=sOXR;L8!3`3tk}T!ZR8qcOXDZIh zIaTB;_G&zOu}#fT3dtfiuw1}_k>y;Xltoo?xPV?Jfyl(?DCuY#G8vbKVv>jwK2<5F zYE^8jQRKEsvu#qZMU`za=G)Y6wcI9AJLOup+MZ{!cr_FbAqkx#(P>6rj?YA*rRPdh z2?>dq!zs8V1PLC^OU0s5WXi=Gw?eDC%e&uJ<4cIZgaL+aeg3)q_2=&Q-*>tL6fvDFW0I6iiiSbgF}VgF z*C=Edc{ClHp)BiDOWJ9fu3B#EcXiA_eaEB2utO zBng9OlW|-cn4nQ5T!xfS5pzgF7WmhK2NyY191mDAB3VqNf_b5grEwU~Uc25g@Mz@a z(&rz}SuDNN6b< zUXDhVB9LLQ;ASuY0p|BF024M1MzN`h*U#FT>MC+<_b!z!jJAe{@5Y9J32RvBy9F4w z2#5u9($_7;Ne1Y(+TQ^BOTB<+Fn~VTdTp-x^3>gn6OHFz*PjI#dsTaStg7)4Z&)qWtmF;_xsuR>T--a?N40J=is5+{m|xk;`EL>z_Y%Wc?By-wximJAdQuT=l(hU0bA~BUINqU(@2Rc`#RVA86*rz3CfGbM-CL z_05y@%`>-K=kB!zTG~S$ozcF&*ziH@`w+y>a95zeeWAZ2G61O70r@o08X9U34}s2u zP~TmsfqS9Bmf&FPe1D6-za=ox9vEm14z`Dedcx3w0cib|0ucx}u;5T%U>NofG_Vp4EJuPsOAEnh2uynXp@84-pPiYTc{@GxV*FLlqk#+cRkqS>U4cVg zV3+0S6l7JQ#VXD>9hMtm-_i(x*0; z+C8VUY!wb;F#w*KZ;_#S6cnGHDdu8CoD4n_#baQ&G~j6f71HU1R0<9#g^W$3;xd>- zU{jC`a1ozQf=hhI5Xd7*sYi~b9!o+bBqBiPXhQO_W2uSB=?Te*-x5;)aX9(6W2q_F z%uE&q&!cgaLXBPT%J&qWC@McwUUBw#Nkx$_-=14&&nwF|c}#qbh%V#MrNALj@f=bn zi-2YkhG6ZiR0Wr*;R(%RsZ}gC@NjH&5-t(=O&p&AJSkJF zW|{O{n@Qlbh+HPI)1b;S8FS4hV064{Lzd3sGkQzy*(El?(25YHo z1Do~v=ZCLFZa29B_c7asS zljtO3wM3$lNE9-uN(}%eP^*Pnok*+WE7dZs#+~iTFU)p0O+qo3&Sx^Dz?gGLGCD~? zrmJZz6_cfA)3scxhQm_xd=0J{Ft&i9So@0&Z{ zwqSn!F9+#~cLB*hegzl0sxCM z%pabc15@wd^o)P%?aZ4O&j#Bbw%nW>X;~P#9~-_6OxQwiLlihLpg!Qk8lZZB{i<8; zzqJe))>jwLGIb-|UKMPsntO0%Zgqz1+r#y3p}Lkp?fv=Md;Yrn!P{-~P3`{s z9rLZ-!H({5Pfv89e{p1Jadag1c1E4 zYYGnB4-G+L%?|*Ewa@o=f_vus+X4e!f#IIOP~QQ)paUE14GzPszMusUVM(x|UeK8y z=y}`M^`@_Lth?noG+xaw``Q4&9(UY-*nV$(w0|`?{Vq2Delf7M6neiD+*}N-NBtWy z|9WIDF4$R%%&$l0*B1P%VgE{a4r;+47wm&);|i<;g#)W0|LOvG7zEmg2H(ZN+lDun zLK}-gcm`uJyb=vThK1(itB5ey%iO$wW^Qg`dUAZ`?a<4or*B+QXIm6LqdZTq$}?&U zESf^2CfA_NGl3S+NMB$DSaO|qJ5E}ZIT9dso1R;ka;d4|HsYz{g=?jbPXV2uEK9_&ud|^SQ&zxg4diCaPlh&yb8N@uTh_4m! z)jY0>CD!x21vY1%g(;&`MRcK7qH~+AIc|wgpN=E`Eg|I>n2J6jAtjN?rV%Kle2g3Y%|`8FwNC2pn2C5QGaODWG$$+Og%B1#$qLzS_(I+4I2{)H%E0OJNX zz4Q_QFbT|5Z{0+ zdBu75To>?IzEiH^^WMBlNB$|N?}St9QcBGtg+(DZD|uQmQOu<(r9zuQpYO7l=U9uq zt}>rC+fG%9DKa5N!o%=@Gi9?iVxC?qw#elUxzNC4$*Gx4WC|e_!yyvIz@>@V2JHdB z_ztt!Wd!bCoozH^>x|i2bB@85Z+8_rb5D4RPGzes5(FXRaC%~LCJKzXM^ML(ppGQr zk`ZJyib}|2(1{!_Q>?hv)*spYwEW@w^2Z;mAAeul|GBaM$NK)ycVGV4`u_X&x1T%T z{@nfY=iaw}|MczO0k=N?_!C+&(1{B$?9*d7jsoCIsXmw$Eck$i+()&-VJ6|_|_u7G4-~Rq?2LSB* z`<<`x0x_^g3{0{A;mgk6XF#&e?T=G|g~ral(>L#)yxClK{chRS#?s66MHg-qp0CQT zyyyZZthCZvaNL|*qI2hJY;It}Bszmgt>G!;bP<=xreF_3VCgsn1(QZVrs5E(IAjI| z2SaR`Bn*p;VN>B@5f&NE1k*b5!F4d(%486*Ogw={q=-l~2^sh>u8N<3vb=R*`04n} z%v?0Ou)GYgwX(Xh0uXkP_G4{rW%=FO%KP=T_v%+&1U%bD?!*AH7K2JZ)kn<9ga(f-Cr-|grCXn_v{ZDSM&APrct zzFUjE4Uw+eNLNjyyC&389cZuiw^hxyT$^paHhcft)Sb)Iw=YfK2JL14gQ{SAZLFg{ zdjCeC`rORfFA}?QF!sx!sYYf%a;Nd zFGR0ji(bAMJbOBN;e7b)*@g2L=1-oUDX)y2yAnKoF>>)*?AndR8}-ZecVcx7(Yjkp z_4ig9T9$5i#P0PjH4nv_2EvWq!G_LAV|TQ%6DZQy9;$yZU)vO@yBE6M61m?IZtV)S z_k_Fq7W(_6Lxa(g!N|j**yEAKzX0|y`gkPzXec()9~tUe7y_mWxU04x>~zF^SW9T| zK@hYsomov(UYUv@os+0*i(r{!tagGU|rM!T9rZ=bI(%)DQm-&qOnt%g6WEr8C}Qeb;I zu(LS7y%gA54!mCq0IF>+&Tq!%-$ng#r1>|Z0BLie-CPWA#scqSpbfrX1g`~cU^C{2 z_b);&zK>sjzZBYB3cgzouP=pHVx*=@3cIP)0tQ@7 zz77x$pvzw2a#e!cTwJ39D_|29BBES^5%AC=UZ$82ct()$i4q}REX1%G=`>CkEZ^5 zBShLCLbn6isMbKy=(r-gOyZUc9ROn@D2G(+Q9yuYt0W$!1n|tMkvh~exB3_G z?NX!+vYbU#@pxv1++$XF%rdt@V3l(WLbjeSv`IBSt)t9VcrmBoyw`Wq;VLpSB-A8y za$+>)~eMvo3d!^IPMp;B<^ zFv~C|IhmM&Vh~Xb3QHl&DlPMt9DgwIB>ZlFarfKO2k5}oKK@+Y|GEAN0PN4rZ$G!c z|FQk`&z;YI?0x;`=kEYu|NQh{Yl)w~|M~gbpMYNbUw&WT-fno%V9z%?fpsZ0W|vv4 z**c9yrPQhAYPCYGQfjp_jZUJ~inV&7S}Rm2^=4yHao+LcMNXGhqL8pfJcdw67mAB2 zE1SC8E3ci^ICVm;kgMP@Bn*jBR(bi<==0I(+1X$y7LIL1)^?Y-K0*7n_kDBk=eynC zH+Ftsk7I0e=g0fquiGE~#jvm2d!K>c?|xd@+G}sJ>srch+%La=xBS}e z;!AY}7ixT`uX-vjILpr3ODb)J6^5(=ohwIWb;*n{9Feb5(7`N3$N|6v;X=!O~8PF^K-^wz~8oV6OcE7s3yt))yS&YIq7F&t}hQ*d+ z(WPiS={*40LO2?Mg`(p{TmlP$`H+9wKmGFQcP$%FZyoH4>v6g!~r&NCj#33 z#)ZCH3otYh?utd?S$e_4aL>(PS9P$nI?!=rz5}-YIK}|6=I&pcZ>|cq0&{jV+5>3U zxY~PXwf)x8ovP@~OOfm6qgCf)K-Cu(t1rf`pO0NR8@+S}D0cZw?Ap1=)pLtA*OuyP zmmBKW8yi<|*2S)0U#`BfRCQyex)u;CdZl`$sv&j_T(4iMZCq(+TEE-2a;JIuPV>^O zmZiGp#fH{cWBbybp2gdMTU`qc9g$n@;kuS!{exhAbEx5du;Jdq-3QU;_GoKow6ii2k1nG zf!gEVs}*i~4FK;2{n>nfhaYqR&tRt;01P1N09^sliEDlkII~{=5JcBpfA@4>*F-O@ zT{+(0F$Ngc+xoow!Dz?*k@lw7k9t>vlkXz4ThY1QrNGD4h5fb2$JOxOYG`jIw7(h# z{oU0N)N*KNIS633y$taN`2;l&`n&L+@XksI=%5eo--6_uKe+jQEU>i*Oc>PWa(HbK z&?^W4wghv#1R|kea6Y~dHa9ynGd(@^c5-@p@bM#8Q5M(8mu1U!MZj_yl|GF!N2AS$ zeLyD13AgWDPWEZ1K2NUCms^T8_Hv{1gw=c6YbmqIJPK{D-c)Ec7g=2u9?waiCdW)w z2=HPqMI{Cd!-@r12_GlpljK5@lur-?35Y^2mPbP}VAUivn}A{wGFT)ei%eHaPFCG) zA9>aGXyR`F^XpB$$1l}evdctj6M?}`#}JODAPyxYAC#2*dqU!21R|Y6L9y9rE*C2l z0E}T_o?JeW&vE5CPn|2ZxO7}Oiz1|u1!S^_0zg2MFtA)=8U>w7LZuQi(uqhk1;v%H z+&LDTTW58uODlX8r%F^h^^s%A|MqJFtd?@**x_SGDHJ@HLr+dm`8%**si}w3QVypl zCuAZ?Qi0HFR(jn)T3=4~$&;sU+_--4Ud@9CRd?>*Y<_U7t*!LTSv-^VTWVr50f}N` zi4rnR!xUIWs%(uiTg|r$1QwCdt`Gq)e2jyVPJ4vJW)iU zDcEeCKxmfBTspBs#WhRC4z)VRY$&ohDqIB@^G@6-si-Q*JLPbfn8C#PFe2e-dJ2w0 zGuo`4Jhv&^Xe+h=2Uc<+tMpQK?rBH%35zjHs?f8 z+@^e|rN}FB=m}yjLn&h^ZWHgULlnPh| zEzf2aIIU8zUFEZ?d`78Lscgs&SGShl?XK>8THpJ&@!`kjho699 zn|pt5?EHiP+x>lO@B8-0Z#y5q?d|^n7z6a$g_-g{MbBP+R1XSA5jI?7IL^1)BO2Wc(`*aG5O2RNmI5r9RFd!12 z#*{N5z@#LmjHa?{F4xyQ9e?YehlQjg(dF18VAsm(>e}kc2GGjNdOW>7#M;X8;?iOa zrU^a}_&*YjMIx~%Oq&Q4iRT4|6*dF&^K)+=KW@2ty`Z>Ab>_IKwf^Mn(4Fw$?Ql=S zLjUbhFU%~p&|e?!t%Cp?02pfs_0++fzrA&#p4xCPOx^1T0PCs&81uum#Owb1*CD++ zZbrKsmilh5^xawSZCdNRz51YT<3Zh8Q|)r&^`-i&OZ8WxwU=WzE-Y1Dj9obwyL@)> z+PTGR=a;TuT&caWT3^3*>(**R!^W*!%QZENRn?2vt5)vJet-ABxD^=a_V;(r4|G8d zcFhfT&kS_W^mR@Ffc15b_jSA(=zP`N2I%#)^F9FBqps$_+vmW0Z7l+`6*vIczqG%) z@OcgJY~jQD!pF7n$K~*c7058isyLRQOIry7Xzi^neB6iv_qMwh{;(ba4}suqaq>a1 zEdjhmwwD*Smcs9s7XZLkW8r0Z`XU?)g(9JFa6SlZ*qncUc6J6NDl{E9Dr1S+RARALIDKccvnw6eVzs$QX$1f)HP}l{uHz0{sa>0+Qv-kF zR;#^QO_p9{QP34Uf|w1n2q^?arH~*O;t$dZ$$3OEhalqMgj|A%jTg|dV620Q^hg*E zg&^Vy%$EG~SDJ^%23~~*#}@|QMEhQayPwTH7<_%>UQg-S>qb`|OQb{*sK?SVM^cc# zCZ+x@G3h7@jiAxcT;Ria(CzX0Xubf)<1?iKzFJ5TF)~?1Gy{)f;L$8%JV=<1ViVKo zxO6H$8HY;4AyGsmj*OwRNKzTsY*sj3YMoI^V^NPLCBrh!hZ6qn(BXun1Qv@zpMBI!hdMX$>7Sc(e^&Y!QidhNv3YZq?TU8$?Tdb9pYZJn>= zIFZiz^;mKu3V~+eGI<1|gv8RZ1QwypBNsWuToaFP6!T1SzD+K6%cWkqDo?G>(+aF& zpgiecQops=hb)(E5PH66?{~kO|EGHq_W}`tFhT^9iAA#sAT%YH z6({ECBx1crl@qVY*XfGmGiy=`8q@P@)3VBw5^^;FU_S)@^!9RQR#$6PXS{kP>)n-&CWR%CH<*SuCKAA@(3K&d{T$~cG&P`G0CW}(k;?x*vY8+J| zjG|GnY<47t9E3qe(kK*(m?4)k6(Y8p&(&~+u>x6woTcQ1Q;}gbR5XW1luIZYB{NRV zP1JCcWK1oOu3|71j7W0SeRu$zfFN*43>llF;$v9^@FsgVBH*rnSKux9gIf^+_o5&_ zfDzDPctjKhO_%b!$0i+zmyW}0$MKcx^vZJvF>DViNBnkEk$C*^>&f+(v+K_n9|5tx zUH<|FV%RSb!>+)C*Wb=RT)$m;n^~F~pRLR&)}6u6u*qzOzf_$`;|ZHOdRNTOEthY@qWM$3sA`_g@a*e?7eT1^{++@%8B9%klZ=lZ#IPT^E-> zLkCP>emr@94RUbye#^DfGxof)!&vgfSkN|@-!hQb*q2+|l~wgPqq04%yd}A~A>m=Q zHm6jRR-jDImd2+GViLIu4O=Fs@F0MNpkcx2uz={`+pq^e1l|4)&@14_zdZQqhoHMZ zg*jE<3*6J>SkAKp0fv0rd;^K&==(0A2rx%jtjs zW{29}o43~I=4a}gvsGGxT#3}h5^|E+&w45yFZ!&IZ24yn$DcUH+iqCpo@~E4ulWyv zHG@yV1w=1n1EgX@_13|9+d#e3(Bw9?ct%=%V~=;n9`B7k*)g=cy6U!j8Ue!&`dR_T zb~+k%p47Wrs$31_z{9-NfMLZRfU&AF*Q0WGb&apS-cwiSt*-|L=Bj}L7zQNFSKsWe zZT8f+Z8vrJ>K}V*Tivy7p1LQlhE8{5m%HwXv+l98vBTNi>1ytBHg`LkdK^vNc97;S zM|-!kqtDql=olKZ4;icjLx5hE!9nYg!9G0f93OK|jJhVrTsK`aCdb^*roAtog7|=t zJso#XkGO9-Mfm|{hh`D|1&97@j`nQ~_iv6EwnmLxqo&Of)4Iv9Y8u=ag(IMA02#a>U&}Aes1WV8SH#BW%OE>0l=WyLdc)2 z06Z5UJ5b*502KqDIxqHI(75fm*ai%PNOtbELQ{+W*+mdQsmlWw$PK-~2wm(rZ-nf0 z+i~i1oNYTHhHbl!ww=2kI{=v1VRPH8&}ns6+s*ktTbrBf8-QMmz`_=m7Au>oSt=G= z$Koaeof1jYp?*=~bQzE^Nrp_FACp)dAD^#+I-O<;)ENRr8egX8N%bO$Uc!sxVR^U+ zIy{sd9z=jWz=Q>0BO>USXbuU%r9=USvB=?cTqp$rBO@>z3SPj9W>Uf3d;|p>MZscO zRE|oaN=?bGtZL{p8eS|+E!dtext=e3p0D{pUaW7w+T4A)zCAf-?w@>9)!M5~%_FfT zFck3~Eb9A!;J@4t{3$Ff1dj~|21X?Zkca^QW<)ZKM2?`4qUaALcWK<9i5rRYBgNNS^3AhIfyc_)B$2&j#boeEUuyA{<*Jg3w2*;4O3JKC%&*gzG^ZEUrll0b zF=UKrCXpDR#ROr{!8kO6&7{brOf~p7D<#P) zS+YW%su9Erqv@DnB0PeL#fiC8HK124FDaIvtl%by*jg?(mO~L!ZuYA|tt4>~fvB+i z$dEfxfp@|mfZPs$aO(zOkq_=f2HuGbxf2;2h=##Qkw_{!AvfpQ;>PCo>E_Ob?cmaO z_`!AZ(R=puHozFb!v|ns-;S@poL+x9zy1Pfzx-axas7*X*WJ?FnD#Is zrz|b4AT=&MQI!-g*TqS+F(QqMuaff=AWEK8CReF4veHV+3bM14rE(#S%_6eccorK= zWAG))e$#}-x$W@oxwiKm-hI1w-|5?TdUrkB2mZ6Wj`okvj?S-+-v4~`{+Hv+&&MBr zJ-YmM^#0qiKd0}%om_lAz4+zq;^&L^AN}1Lf4O1U?(xOKrlZ|3^{9KatZTIBi4jOx zesh0LT~~JX6Tq;v@+Kf*2@k8{a!WPo1*()>MN*bXtLFg;lPPIJ0h&R8;oxC7SO_}w zepJvM*nNMeF{p@Xf zI^Jm=d*T54HP-Im7S;{`W*u(0q1QhH>}IqMH$jR9u1!r26V#I)(91X4wmsUuGyY_E zqGNCT$UDb__$_96JtE;BXQ~TIk``BIc*iqAN zuW7T_wu4w}+ALsf?69?Vxw`sYy#uzsesli-BwmAqwn2zwHUKk|(K=$XkC_}3Bd+NQ z`_#C7YRow^0T||fHsyIf<#{&lnHjwS7}P8Zs+EUk4*doO%^U)NZH@qd4Q&h$t(ykd zO<**vf$I?iU>JbbKLs$%xG_GoHafUAVptg-S}+Ylf!ZGSzXswp*!gs*dvc&-`OWmU zb#>3axeqO9Z63I{ZdS04!RYa8lNYeZdA8$%@B<*^bDRSY1Ebe|;^fcp4;LkbZ4L!~&0;gWtOYjs6Sw*{to{{^JZ0LHAK1VPr%!n}FZ~>mo+bvBAX1AS?{aCh7|^^Q$XT ziV73*^5S!{QVI$Rs%u&X3>`L;$c*8abFk3I!5IrNSr_IE@m; zphh!j2nH1(GZ-HoOhkr}F=1qEI0+jV9eF!6@J?93j}Pws+YjIW>tFu-{SV*Wx(n>? z_Wj`dcY_1&1_$1{8~EexyRh&eG$#69@Lli@a4Yly$WOudehh-%6K;pz3qplQ5K#fq zu-jn)KLp+bcei)K9^4Ik5P}G!bICX&;=9}5{mZTILNIUy6C2G$AUH^pj7(M2aAE>h zh{O8<2IxhQF)3;$BbLce;>j{(f+R6ZE1<^+{uB@u8itJwCnCb|Q9;AO9B8> z=O)AyB&zdu38hIHHEFqZX$2tFsfD$f8O6y27Un+eJ`4dTG3b1`P?0Fnw*L5sl!mx$$vAeF855xL3S5DUKjy--`_UDJ&{jiS?VfXr$oFJ24qCedru8FAVy9YsWcfD8#Dz@xcDlAOC3tG+-EbF$Ld?4hq49 zpcqjA#zX^l8{sxuN zL-}_3v%fA8+7|Zd_~Pu<+{^O1lDzWV^oO|#Sy|fDG<8z4B0fPHs}-tL0uZH2q)_Nn zlPfAq%E}88lYwJ$sB9jEDI4WHc6F>dpt;=@+ouRyYyt8+*yMMNKaCUHbb_lq2 zd~tYmeth=f4C){Z(Dmi`^2^E9ZxFyBhJ8K0{CfHx8DDV=;f)^zmh7S)$g+g<)@W6W!51`#(A@^| zW%ZB0yO*i)W{%MXoh8=l9By%$TD-%pJEQHp6Hj)hI`^i!4ko+yMmlzl?c0X7?V;A4 zq1N5OmYsn{Z*RT(NsX)Jk-MSNSyf`ID72RqIm*ji6&23%N>@d-=TV)ny53t==XzA* zuB!D1z^vX|*XXNn0z7k7H#n>6JvA-vs#f=-7Efi1v%J|+*4S&ZgNhJx~9fl(_@}z6Q1Xj z?q?tq&S#^}8Ix<;-|BE;$Tl&!HP*K^)(_C-uifYe09!G3tr~mQjQwlI0YAq62fh9Y zW5XNc#`Uqm)scbK;h_~^V19)7Kw_BxNe;Bgno>5d2FWZQk>b3(r5 zpDjGwcAaj!PyNsWIp6b~?|Q-0j&?nJ+u$_=UMJ90GsG`Hxi&U7HaFJR*H=JR*VgCP zSK9_V#R*bYJV%fs0s!Odg`!lyex=J*c`>ntIz^^jl`hg{OOgs?@eso#noIyNF;~Y$ zanK?7&>&oJFfJqn9~OuSd4LQGBSuCt&>-P7By>bAAp*A70(?gRpLz6EI z(=UzBUXQ(5oSIvIx@ehMv_4&Ny;$?T-Q1hq+PkpJRPaq5hM#`>5|y| z7zDiGzAUALt6qLphNE= zgYO`NZbb)z+=&Xh7aa_&>o!PK;C)o^J#=UQEv+X`zO{d8Kfdyu{k-en+I)Qd>wm9={h?nUe);wC=WiGOM)m$9!LB_!+lDby zaZP1radA>!L40PmHZ?s)pQ=hslI!9{ni!!<9TTrDF3GQNsLaex5=nVf4inIe!V^+> zJUpElr%#_bw00)~qk7*X2LbWR)n>3lu52GJUovK8>T%Q6(}ogC35H_Mepy5rL0@ zVZy-+Gzbv_ZK{h542TR4LV)|Bh%g*H7!whUhC^Lp5HRTIpy==b1S|*{3d4pGI2g8! z#*&b^GAd6&W6LPQ7(sUF!`{*H7YpkPTdsA7&+ggv?CkIC9vOK8+uI?IFN3FZP#@Se5ttzrq z6xu3^?d4_mvNC%)z}O>qMKy@MtkO|g?X0Q+@zm7$>KXyP+yG#84UXDIS8a={s@VaE zRa$Q?sk4J=3FZU|mng+|R~bPe+{7M#rSVHa=(_8vrqnL-e`< z7`R>^?pYq{1OQtf?prtYubT$eOatpc$o$*_;|mlcZJw&{Vg<+Foq`W$AT)3W2XK}Ca3 z>z>O7a^QCExg5K0D0?0!$brv&=yid|_uY;I@Tuqb(f{f1IRc#9^&M|}4|Y5|USRFO zz@U;JpkV(1*v9(W`o`)8w5fUH&C)_yOAR+p#7!27(iOa93Ao_tW&9MWI9;aBi&5vr z%F<+tG@&Y$8=oryQKt(ODLio^pQfb2Y4A`&SQrroJS?0X5sVKB!Ul&B!eA751T`vx z8XZB2LbGu3nMvJ40~1eQPQG|M^=fWrcIEl}#?yt(nT4%qOV$@F&KGN*SDU-9xAuT< z&6@Y$Sq|rIM+^4j1;@#v>vYi#a|O&0nwq~2j2VF zJGcMkr`zA%yMHSP60%!?5AKEp-U|+T5CRJb3k{D73l0yu6MFA%1Rz#u6bTVYjP^H( ziU`BPLa|}N=#U^3)ObD+5e&v46by!o3PgY#-+L@EMVTmp6W~9F-hq*#2x1C~k3|Wv zk!(aHD++QqK(fspaK8PhY>-@`C%CW$*ds{*~kS%6ASO zvjQ;;S_%7ldiC}FM<8C`!1(_9+r`xv05BjoN2edoz<74~a&Er7p&`4xGPSrgDK9?( zFf1)2CN))+48D4C3avILKf9^9s=OjEHcraqvFXsZFd>Bx0LI5N*=hNOD`w~J(Z&AJ z#i9S;E)Zw~^2PPp#m9?_pU*DpbG%(_`|O!SHB(uS-kvm{sB5{ z^U4n}FkS;ey!v!{`HOS+Z1mM~pC+ZwWpLc`PaZ| zV{rr&Aj0wRNMckp85u=Ff$s}=u}0t{BT3OnDjLJW zqd9~~CKg6T!6>Lu5-OOC45Q#88AJ?+gyZ5#e1b5BqtFQ?DwaaSQ^yIEaRN<}vfxo! z=kVyn?9%LpbJf0Waql|4d)qsQP~|+p*ul}^;qk%#(Y_yGkecm57XV-&+dj_?y+F3T z-W!;Cy$-8&xx2R}C5@kv!EbMVxcGWzb@ol;qe8t#Fws%HG5W-6YPF5D*`RsCM%!=$ z;Fit!d+a9v;mdx2Z5bLijdeiLZkEdIqfmLh)70P@ZQhxDd@$2__^jt}rq>U!uKkIQ zy|Kq$L$hb7!P!@3?JBo-KXP`}x;yJUPikz9rRJI+ zDY2K7T1v`o6^|U%HLe|MDXFuRH*7s> zw$!&f`~`>B?w*aVCu^Opn_aD2J?)$QPd0|Sw~YN30=}_6yVV07g=+Ue zJ1G3kFda6Z(+=WwIlV52*JXD*tZqB>e4hjQjXtM!-{(5o^&aj(NAv7>pbZ#qhuz_@ z+H5xSA0L}r8yn_z0JHV=t?_3wy4+NbP9jKC3ewe_Bsn`t#!Hq9(-g97W$Z&8)D$sA zB25v<_3XGTAuzDmOp#L0mn8B9IzF0%4kN%KDDWtHbR;bbMhb&M2@9u&N6;dp=tvxo z7@L{UKG5^@?Yoz=%g^7fyjt9NvtoX;W_!5`;&{8^p55@i-P(R@-Ji7`&RU_o1LEa4 z0aMZr_-VsOc+#@ zNRFUVBItl;lyDLeM#M*uanVd7LCR!m#NeG41~^2F3dKb|K*D|s3-~c4;0`S4c4*+O z;GmxZgMPRl0CGDx-~sHxy|DZD;DPrMArGR0!ts$2P~Zp(BAgH&j17jcje~^};bEj` z7&#h@!T8AAQNjQ6;Lo>UKalx2hJ+G|4TMv|DKauaLPTlw?$tWLK6HRF!6x zQD zNYNxIjwqp!<(%Y_lKvNSFE)4HIgVF%E-Z&v?$e)l&VM<${Bm^t4dCJQ!`F+e-!4A< zdj8?t`PJ96>(6H&KY0(1EKcv?@%#Pbi+3Hc zt^xBtTYV>nXN&a%(^dUb)%{cD9V6wBO=V9^51ae5YdbR@wWpRgCl%Ev7S_e)RLWBe zCCRyhgfy-;5q$3u3>ut*hm+7SLUaTn8coOHStKM48%05(n0WBDB?_qosEIQL!6zaF zq-Y*7ibIU%QqTehR>&Yp88jJ{C8tWYyx0VZRws^6kffxlQ!};t?BwFw>c_@m(~Ea6 zm#s?{?}l^7=GpUXLlU;TdjRz7@bKv1;CO%Uzz?u}pkfDmdwaV(kd67*#BSpA0dzs} zc^w|t*3?vATDmMhPc<{vxwSaEKKJHHQ-w-MY$?^Rj-~RvU7jz$rS_%@TYb5;w#-&tVlIEU zRr=6eR%|XSwN_R*YHD2db8MFg#0sR61 z8yT>V47kS)-l<{Fq{$7n;TUpFncOoYP$j*eVIVfBXdaq5^cz?ov;a2R>jxObu(e?T zv%XbRzdr+O!$Ye#F%5$271Q97X<*UV2Sz}z1!Mo5v47S80qpfq@5{m7>As%1XH(Yo zMZ0;;wzX#W2LhNC>Z}3ibptT)h`;BP8F0ws2hvSkHZTH!*}P8YO$Qbbx83Hp-vA7H z5CF{OghsC)V8GrEw%td&zWr@z{&2_Z@Ih^$9WJ}gZnIdye^4xD^VX&rDy9c9PL62O zb;2a2C{ruUh~=khxO$~9RVB;P#5|0PE!2tAOm&ZNT`bl_f*bUK2?0wcJFK~JE7YZ#drM#4uk z$v8d($tFcI@o)wf04&_!fIEVW48n%r)YjjLfR2Q`6B_nYVDJwC0Y3)b`zZw4RvU(e z4z~r31fq~3C?uLm5hN%Q9wy~fW#!gpW>%$SlqY4DC8ZZ7vZT}?RA2-a zhNGew0(y*Imyo4T&erFYWfatAJ*>+ptj)+SO-?OL$a|EjO;?3uLhc0Jz7upO5FUzS zGngtROCx8i#W)rz3>gs=9u^c1N8m|fjW)Nks;PglZ)#@z-MgntD{nR}^A^YamhIj8 z)~m&p(dRGwMkgK*nwq-%iW_Sa^HNeC>hmh|ifhX&nrez`t8yyJGRsOb%8Il($wHl! z6C-4(cw{AqqT*3iLW)vEl<_GFK1I%#y%WK*qM= zI_#U8s~?zo)HhYpGhWs?QucVbw9W9asXwQ-E3@iJdU@N8fyHE($x{o2K)-Z)raBhO z=S5P+1a4yB9s}6&eXHJC6CLzk1D3ziFs94>y@jjTRHcEdVg!VV2=0f0`|(=0BFffPU56 zM;q*>2Ip|o@15)?AMea`?9OxmfE`SCA53v(Csdcn9I9eO*t&NWM zX3vv$XM3Bywb|OxU}>l~H#TlGHLkTZthP2Rx701Q)Xg_nFFdYa@i&rR>1kc=ZC@F9 zyf)OiY3ef%_iqh@u@~r4>-pRo(TXjgKK=~=A0DQG~{7_RB-(F z8rbGo-}*=|l;NJ=Gupdq>RmBGBLEoS){1cenk@uQ=D%1p0qg3YH}uUJdgqLNv!=ec z{>j7FgS|679nXda)@ENiwpJYG^&5JDi+^;40@pX6f@=UUx6SOfn%xkz%zl6YT-`7X zYT|sefgOxD4O#r#+o4_UK+aq)@EU;HwQM_WdmhKpp7&@E0L=XdzySaJ-CJxH|2(1< z;+MY^(f_eA@q9A5AYG(a3DRRkS#gqVohUO_niZ?c*Cm$gV+ypQRH-moEKU(gQ@Ami z0$q+smnGI^NFOy9&&*ENJgz13Nk|p~#YP}lK)K*(Hi|4HDUuY$b!CqYy`wK?UN0}b zU0t7DwY*!izgu_AZaCj)sZ_L{?RGR&`}gO?632b6IOkMtLbiDUG6oKbjoHAcfON;S_QtgMs97qS-7!FAR@| z;qszcEEtsk=wa8e|+1U@BB zK#dd86Qq<_Ay&pji)qn3d;~Kp8qkZ4AdAq_WDZwHXR2x3SPoOg!tn_xHU`0h4v->< z@pKhcnj({?DLFbJCr->yRC2V6KjAnG6nJC8<6&5A2pSWHK?R{B@e;ZwD^_2elvr1M1y3hq$AKsrF><;_%8ZpUG-42DjF_VpNs`6N6sX@W zUrUwixger=CRamaD%lBz`Q6iRp00Uj?MJISSGJ>1_M@wLi+ytL&B&{#!_Q`BW@p#z zPWS$id;eha&9k)P40)1Ro+#}Z8C*DK29lV1_0A! zmP++G9Bnd7qa%tX2qqIwAxDsjfMGBqHiCphGDsK}1;+(q&7~^%1SuOQW)dV!tcZq} zGI3HCT1ZC==oqLeC!iOPsbF!G3`s0k8N-W@6Xs`XD;}m)mt~d~r)T9RmsC}B7)OlH z-#uO2db8nLv~F*@_N<-*ptZi8lf8q}{e$C!1Bhe*=KK4{2m2=gV|##Me*nxMsC}Z( z1647C5!_s_F3rApI=b-gCDgopb!lVnMR!}NR*sL8APo)KYa`7ohPriQ{g&}&{;p*P74fc^HCor(7Ht*Eq?U^TlVSCdZ`%|4qQ{6|?T?Z5G z`{QlKV+EumnxMHojc(v{!$YO25LT&zPL*Yh4$yQ^Txv^}s zvDDmBvDI9$-dMKP_6SOA)n;q;W=qXRWA#c+XfFG2$$(AOnyrR#}h`Uz!s8FkY6S5++MT zdXZSqRiyG{2~4S$o|2^=pB`LZpEFIGlnH7K8-?W}F`Q_+gc75VEv>B>7&kqid;4a2 z1)%Hg>c-rsS~b*25Gvurqu4|&kBnjw;8cItYz97xK@6u7g9-RBGC7jT004_(FaX1l zY&OI#CLILHV3YAVM^Zh=h}o(R3`D2YwemgoF&jM+V{GA$WKoCj0?1 z{C-r#-SDuVLIQ8Y0)o&mI2jeqz(&&1QFQQ+B@u-rI5iw2#Bh@N{B*e}TP?^@(^DkW zWDpr6Ma|T!8OcgUqMW7`QDS&B4V$Co=<;I<8nV>c3T6z06~m&+=m;h@oPxM%|BV%3 z=qd_7p2tfNv9toFT7VZa;Z(w(IBE%=%LvD!Ly*zIXcP>K3c zeP@5m?OnA3V*|Q2)6m(TT~?^g(u)&gnK4SbM$U>?v2`kzR?du(uwuo6c(FQNu1*z+ zbWD+sAx&h(qzko~Vs$ECn#hmIPHHrmCKj!)&HF39_m-odEr%CfW5eq7Sg}sTi{Xpo zW%`1&(x%GF*6O${9a&7oix3PYDWf#K!`S<%wK=`CEUDwB2}VD7HU!y z)vdK_Hv8V`)zSGUfU%PgUr(>T9bbX**P{<#4z4~OU4K5h0#y5YcJbxx{TINnv&+xt zA8rne{&E2xzx)jF|8~ROG5V%y=tXtk^rP;HiYH?LV1+FJU;_YPS=Als6>TY{0AMvh zzj$%UWSI)X7e)hmQK>K@0Y)H%69{1hd<2;gO(&z6RHBGOQ-~-EAwkN;idjIm@De6Y z0!9v2#KcQjWI2bS;?cMcyw%rb=3%n~?H*J>9t<{yS#W?`5u9i}*lBg0y7wd(yL-ot1`c*?M@Gt-{ z>p$paYye2x8gAMgZiX%lb!H<(FSD`6Y^=48G&&%FwR@)@`=)_}J=vXlvOD>Bf8xph zMEmY|^Ui3~_GpvGRPQ#{xrb`3J(cFpN?UierQ^|RbIDR&{!&feyXu^`)!DO;vfq|x zfy`E9gS@HCeqE9MvMl38dB&SZxwBRI?;bszEiZUpkU5!|GM=HI$W9&4(U0XO59cI| z=IbX*vSum^o;@mhSyl47wqmxidZE33sk3RNr){mbeZA+gxv#@&=&~8RZ6-jkUW=*U z2HXn(Y;?#uYH)&a+~}MzI46f3lLqIM0ixHG>85CYYoZ^-JUIvewl>nUZ0uYG02}Gq z80`ZibUh3KY;70-Yyjxjje$Xh^VzW^E7z?=zc9QAsr`;|X020CYY+FuOy%uSDN zEzR1^EB`%t_&@VcLbmlg!0f-{%HuzL;2#YzZ}Ja@{jnX;?t}ua-9DFV*W*0e^`0E= z0EQg^fcaeh`9tu3v)=TicRQf{?{-MVY&IK2GK+b0etDs+p^6tT2i)RiXt-$_cB)#K z8LKMPYYLN9c{)|DR+6p|rijETe0eHQp2!iaX@y0p^Ybs(*OzDK-WFGukhlaQ7YpwG z%j!ypMu%U{&dn{ay<6FsTQ$F1GtY0@0lJp0zE%6qvU7I{#J;=i*j;h%fpHPUdAQ^{ zS#z)RP%l9kLvxiDR$%!$pc&V1OEADbUbRnswi91t%+ z` z26FxF?;uxS|918D?;pPX9fH}ff4%w@fbBmn0m(l9?S^NUU;g9b)4!j8`gZmU)GGe? z`sbq$ACE4t56>_6j?R4hhc=II*=&6}`>uCvq@uM+k(Nx;NLe~1FA?1NYdFBn5@b@n zOq;2UNt4MEc}hKBsTXL{LoC15R!xhzPL0co~Vpupd2a6MvDJmK_jxA2)NI?>W z)orzAw|DOpko)uD`!6R~zn)$H<}Wz>Y}?s?wzTxtyt%Qxz3JVy`F4GWX9s83Czqej zuf73>oxJ~Y`u^+5#V7C4ho_6ScGJtI!ROW8(-j@#xBsZ_ZV&ErMT>0xA22oWEO$A#dqAsB2377HWbqiJL;he?$R=n4^8CLqfA zAOtCoB+tQm-Cn!Rwzj%F`}XDdxT(Fhal$mPKKG)dsaPdJ%LQ zWp$)+WvFIjsP2Yae(`FApk=JzG&QZ8nl^x+L7h12p)$phIt!4nu|~&avuC=^JKgS^ zZrh$|-=1vU9&gzjZ`+$_^``}3%wwu`3_WrURM~rgepTAKs;o~cR~ic!Yw{MVa%QWt zUYBRSC`_Hoi66_*Ldn#b(zU}GamKV*Lu!mMRXdWNFp{D(CB_<}?G6$04hEn1N zQ(^~FHHHkGDLZ*IKV!Nm_j&2VH&vyxb(QnYwaZVMR=QhPx?5H{nzy>!E&UyKL$`ex zI{nK!YIIDP9OGa#I>ro8#tklYNtmI35!l!;w0U`9c<8OM@6|x(&G{)$dmm4C zx4jweUwAgT2>@nUb(sGLy>8aM{t^HDAym`hwpl#TqL<~L_3QuZVGzK;r!EITmmgq& zXHK8n@jJi{wvYCqDa0M9TGi?H1I)joW_8+a4gfF+UskKlY`1S&EE}sEt1Z19VtouR zP0dbMF;i9S^cZPwys9uomYX2S(kgPb(oCfg+y|!#B*|=9JX@ck9vpmZ-dNsRUt3#S zHH;e4a#GUs)4B$`-ptP}t!*x?ZY^(E7B_5*TlPh>ec9?>wr<}541$((Z^eGF;@Dqx z9xOZd7oGcyE&#COW%uc_=X}|FvFL{M3kca;%T8VIAXy>+-UFvaM^K{?Ogxf_kEUb% zJ9^PbCJx2Kqd8O*pN-;i0Jm-o48eii3&~+cG3jtRHIzgMB47Zz?xEp#P!V_h=R}8* zQ7{S;M!~=tcm$UU62&Hk)3MN=YZ^L?f`ySVfMK`71HuSV0A0}x9Fj}I2q;()1;Zmm zQX^POvL;)h%8wCetA#nSqP#d^Uc4Yz$IFfr+YOtCOo5JCw1bMTLU4!(OUBJ78-kUvRM zVz^ovj?aLTA>~B-zXX7I2nIS79U6!Tz74zoATm&>7Kthsnq6a z^+kGZdMuibLXe|aA}W)O#}OhT5CI|Z`*3t9k`Rd}AqaE~f{4J|}?fTy#l3jvleg(h!_sh?KWPkkt`Pu&l zDt7+q>-jHV&VT-V{_)c}wAJb7(~FNsXCDquE_{b4R^QIs^|id(N6c6`Gfu%vis2=w z1xX+ZX|gmXU7^-XRC@>-YAQ)dCd!BaW)wMvp{6obB%%PxRxKtsVAfV$!G?J3R&~bQ4 z0#Op58^aLA342EjzN3r%v(HeiJmA*Vucse=J^S$6(fgl=pFNW#X=Q0~iN!g|g}Ld4 zc@>TIeWRoA);2r`r-$d)M;D)t-hVzg|Jkv3F)_QhTw3)SO73gFbW-p!yy?ol90!c ziy2BWkT9x06s3r&5>u5zDzy3t^($kj`J7lGFNPr+u7fNS^z^G6FnZE!{L~ld)wGplb)6oqm@Y{?5eVYx#trtHMv4|n2;MN z;Ra{NQJ!}duME|$4c0-WhKBl^#W1rGV%UbUVcpcYVQPT(hYbUO)mcUXz3QD4P41}{ z?@XI-rp-Im>YHlyPBi<*ns>)qcE_6c##;6#+jhnpJ*FDh&?9I6BYSs+y|dB*4D3ne za>K)gn%ud{thZ(9uZmKiW+#lLYX%c!1M#x{ctu~Fav)ySr&ILBE4yN(T`}@*m9$$S z?NcgyRmv`qSanalrY}h|kgOg67)#L&XC_Y3wV+>H-F48y&Qdo19Z7XrXJ${tl3t z?shQFjt|X0o!DBQb6Ed-t?QrTus{oA;PIRCjhkh#-&F~!)PQgW%@#Vqh3#h1%j~}2#aPkM*An0;zyY2fv*Wr%$aM$Z!{c>&lw+#AS&F1pE7u0*qA3wv)4u{!h-Q2Wn zj=y}GRFun3QgG5W?6equmQM9BMFAWlBNk|xI8!Z7m-CVZyd;hwo{^BL=ox5PS)I3; zH!S|5{N=Ufmv5fEdiQ!|V`JTHTivj%Y*<&f9IIyMs@=2Z^sPI0*X%ohUTc2qT5;~L zx{g+y2WzgQHTUtV`)Jt%a=h$0UG<)?`ra>l&zD@M^Y+8RXS2%G6eN>?z>8ur0llK3X+uT?oeHN@U=(670TYDB1mMs? zL`)D7eIFZj4+DRIjSM74hmcX>bR2N62nIkG0bE3~31Kv3C>0e-27eqOjD!wA!{9U& znny-(2q-=o&4-SXM|1HQ7AiS6=3!lyI!7Z&R|>OYgxPW8+;~Y|A}1q;n-R-OjbZ9z zB-x4T!ZcYAA_-(%fr!E8;4#FAFjN2x6AZ_PA&HSV z8bPF1wD$H|efy_B|8GE}t6%^A@wfkia0Te~+yDLe>;H58^}jxT{jcj^{rLusSN?0i ziv9h=?*RM1K-VA}^Uo^+54-sE+xaiQp8fpg3;^sJD$j$q_g;QHxwtw!yWTy0zht$h zSClYfWUP2K2N+l)1TcPrT$mtHq)1gMVjyAh*&1Dz3NS1&N24!@%_vDoFG;ItZyI>< zYG&0s=Q!Nhy>cCY+Wqj^d2-S_F(^-#Ff?>VES;nxVZ}I%07w{xsba7+G`uhx%ffI~ zDp|6grs3n|6r7xlm64#+yo5Le51PpZ4oBu=@r)?C0K=AI=^_M6h-30eWDb?cVX~AG za7)2Umar1J>_k3C$EC_CahbYTON)D_*M}Eh0lFaZ`ta-d^}nB9{dRcq@yXC2PC$#| zU?LbOp!4CxC^#931HUgJw)RQen^o}BvA=VAxqJSzdHc-tYPEjgX+`%`X~$H_ zC?J5*m0~}=#AJXmrHHN;(KQ0Pnom`5m}(whClu)fs$@x=UXh%t&d*7BRGd*;k=s;V z*j85z0QRs@mzEQwN>IjUq_y-LC*H2UTyZaK?QPf(t?pxo_td*{vAYK;*CCKg04=|P z9qpeT?m-fEuzLfr!`+?z?d@G3)b)4UZnuq349DqIJOQ1Ho~qSdBHdky_hmjbpsQ zInn5vZ1POE_@1_Trdm9}#wMD!$C`IWns-c%dt*&|W6fSut;bLe;_9!mbyvE2t6jad z=8i`z&BaTAVb!^dmD%r#)1Kw%rZQtq$?`s(q+ciNk5dfl6uoiMZk@CzPS&lF^e82r zQc*E*Uykj6EyHfrWt{z^&C`!?M5W#!Wfml5uc; zsQ>LyF9fiGE{I+|ZBw1CZ;ZVQ(<2Mdr_9R>4hvLp_>ZYV|8f@uFsKj_+7NbAe|W>N z-$%E<+VIA={xNgtwcEiBi|-GHd0Y^~JpT1Cugks-Er59dzz%l25WRfvZ7!p2qH3Gh$h@1cp)|I zH-O>!q;xKkAY!t0YMe+APp2g?=&@AF77}p_iLix?jicgY$ar*2BE(RMXnSZnkPII~ z#l_OF0I(!BDU}cOqM>H%0vc9KPZLs;c*F!20RsSI665LV7%mYfVUU$vuqB*`h68p; z7$hDkztnW}%UL!m0x>&wl`T(!t7W@%X@C08*D zAgA1_G|MC^5riw@P}yWE1J4&T#A+^{m6}M5O(G>E6H|CH>ABi^dvJE)*}J7zKQ4iL z0lnUSc=Y-wM6Y+hK6&#Cf)?V|2RMEdkuBt5uRg54L{1ht_^|ro=am;fMI~(I#rvh_ z-$yX^-2!6RtA!_MmGS)Q%Z1e!;3Xg&frZu18#iRxW~NTY&oYW~tl}KAFjFhaR4el| zYS6C&wYgZARc_APnOU$ar*L=P_HWAeRqQ!(q2hc?OZ$y`14H&(w%|x$1pww>dNIBF z%oYq>Xs*^5>xC8`T|>i5i0MKCK}=?9SOO!Lt7p;`OrBP)$j*ijagvfrP*Cs+3IL3x zVp7#Cii$y$lCeA-o`<7|h*SxIEKH?~2@D~X!lTiIEV)%{D9MoIDug)_w4s23FU?S# ztT{Io2+b@#pIwf013|6v-^@P-1A8^U`mAlBpR1Lm2}x-JQaYcK!oel8u!)ScSaLGR zv^>+W^T^jteZ3>_#&FKm^xQdA)3vXvZBJ$M_LKD`N2~MqpU&KMSXHo{Z_cKwbVP{) zFOuLzVl1DJVKM<)iRhvi6&MDIB@(vaa9eQL7(6b8L8HmUEVYEAl`%CkhFVJ3NNFlD zQzM1*A)Z0XGe|f(A>SyJSS6|~nJG`5k*h8!H164vcW7_P@q^n>9o~KN>mA>GmA`ve z#5Sm#8fJG(`BMHt!g8p#G zKRX?m3;EG>qR$ud!g;nY;BeU6+MBYnOTO@)_8hMgV zjMF8+l(b0xO6U_jyW489=QGnUO!`e>( zx?Ei+U7aVrou_;qr;&lRoD6}4wE@76d76%Rn~u2akGN_MxoVF%YmZG;9~rLPH+XW- zgJU}%9NB*Fo3fkxiUxLOca>SXN-W*Q865>yNK=lXAy?m&YpkUGsRO|8k$nw{BInBP-U)W5C#+Mb;^4}5j&n*(=`d~?6z$itIIhtF1wR-x5kvxZ49p4Eg|?YrA?SV|Tf@z#=TP$jYh1fB+Pmu~ zY=d^^qz%nR+5nF5@1)H!Wpmja-bsgd%;p}Oa*x_PBX;kI!#Cy%jXI~J02^_JhMl1y z*Yu!k`XL;+!^58WVb8*dcX7mv#^~{=ZxOx)kQ=x^ZZ0WLq!B>8Qkdx=TVyefDrJzx z^ym~Kj07P)ol8O6fXSpO999g41fbePz;482HxrR^0kL9fB)ESonHWbSC9LNeK*G5(`g|GD%7fNx>z^*f=qhB;``YJRC@n zgtmQ$<a^u%S)N*wq2QW?Oe0fWs#KS#z!nl2gd{dIRlrN;u>PPdFo-f$ z0;`Iv7jv|Ho|enk3HW+3Nk~s5rfx}&kHaJhlv2J*vN<*Wi`Wf+T=(ZxT6#`-_Ra&P zg=HB!vqEpuD77*olbAxlV6aJ8N~!`VDraIS$;spt3Xgl>_~~)?4A5)&%mG@0VZxxb))tg=cRUp1y%k_PNLK$^Lrw(aYIKul$RTYP&kQ1_fO!;brKBIT@lH zizr(!$r(l3uNk{-_8e*#e0Du5VL_t(AoR~zF(>PidPtW1# zcwi)AtCeTekdpizdM*|Cw(d6NBWFgw)w+oNI1O1v?dA0cD)t#|XMV2v{ zO-SOv&lSWUkKqv#+1OYbCZ3LoqopL!QfYF*{?lhVZw;MmyLO!Zu`(}Ny0E_`7(}8#ay<|MS1j5&g#1@e+R1k*_D%1mVKb&SX1Bi-rJKm27N=5v*YeLn}5+2Sn>v!0@F*=(~Ex#u<77j zC@}jQz-9vgFz-iz!8!K8Ku=zQRihJ|EOMP*n4O`jIJ|RrxkfJ9s+4WfDB@M3I5j{1 z)E@J_mJ<(~D@GeZyiQHEow2o@nQA*d(RymM1)S{Yc*~JVq+iD%wzlIyFMI28d)skm z`w3^;30HfCw*vrnD$sr^(0Ve|hDKl0abNRsZ__bP!!dW=5%=XIuFJ<9AYtbYj-38# z@Wif%$96*Q9oc?;U*W*6oa?*uuk9+hvLk<>Jh!JLv#&g-yC}0Q$J}huHySi`T2-B1 zTW8QUT8wSk89jx0{iQ|MwwK-5we#lwJ$DZsynpo2!-`{rr%;9~D@HGz9SJ~tqJk~#`dOrEe*Gu>%VQPy;gte>SfTc zs-Egf0NBlz%Xd4Q?sPZbi2@9`1!Rg;Qb2rQOHLb53PJ0)6<|?@Mb|H*NB-NZU;oCy z(B^Qz6(WLRxtx(%M0>#N4EfyCes9o+`j15}k@`KJ=!QIz2AVD>`f@}8=CA{Qra+u* zQ#L!u)1BeruTGy3<`^Z#R(W}rtSm!O3bDwGP3j_3_Rjp17b|Yxz20F@&+6p|CG}^;RNyWj5tw+?U-wN%sDgW3XeFaM_n_J5l?v3HwW%D zhlj3tBb}xijwGbNnPh$WG^za6BS}BBim_9B>bO?2JsywW7#EY48k3)` zs64Z~s<9$-msMS=)|Kfpc4ubq&n-N*E&tFqO<9Jtz#uI$0>cD3dU3u!bAN$(SB|#W zss@G`rJ4f0vdF;D^C=28Q_mM=O9fd%hMtqo2i)RQc$_o=7qs#Z%6yG5Q^~g~*;)}@ z&Sc6NEG37l6_G{ESZwmzn9ZBxV`)M*fk#`Hy!FfYO@G<^MNC?pIoEXH(4Mcp+Lm9C zsWYl*EOG)iF^Q0zN=i+}C-I~#jZr~iQ;2MO*;o5-58G!Szgq%ldiB%No1d57d{}w= z%gVc-AHM_r`sL|&zk-B4efu$=0QQkv5r%!d^ac9$JgQ%)repc}PfO2!Tt+JP!{YPr z7GJzueExR+*}J*NZ{bsZ_VJs*(&L(rHm+X5(#r(dCP8+FFv~2-HOlk#szQxCTc*iX zW^c>dUvc1ULsfNKV?$q0$IWkhA3V4^Jbq`&eIL}`6P^eyIcFbx7hZ&bA5Y%SJ$V~` z{CwQ!-hbw6x{`qx6F|bqatd8V=jzye1DC6VNCgHx�xT1!NVSs$o*pETWQ$mw{`u zz{6-7HciE1s2ChAgQcR=WekddMinq*CWYKAed#WFc9Ar*fTgn#q{?)mIF-jsK_yA3Fe!=2NMa_e;4OhXEIpnC7^4vcJgPzj?!`7pIYt@ZBo~_HB9lyNmMSx4>RgpRUt`ME zWaa6KOEdQF%s=#1$^PAiyUH^5?Z`c_Cx8D}`Fr;lmF>yP+n)3F>ErdiSG#VFT^sa0 z00Z;P+WZTyz_K^A5|~~A51k2r^sw;s0;=f<%mKY-0`o9}f%(1DK2Ol&_QMBQ|COHH z0;@vB*XhM(lQh#J%gdBoj0}}5L92?>Ym!WAj6sem&{I0k?t0jAa-{LZMAJzd0@x|Y zWZS9n)(TX(5uq387mW7SV~)1tj@IMO-{NXJ;c2h%wV(90R`{DwK!9G}#$(aZ-Eh=Z z4<2^ZbNQ&N=I{hs4ffU0$=wf6?7Dw!$GxN5Z+~5WYk$eNdkU}bDZIL;5Eyo4d)~md z-0q@`u7Zrt9CLf7p*7Rgnq_Xywsz)a_7r9Jm*rpEUUYNM_PhJ{+&=i#{li~BIDTZX z;@HsX<0F+7W9LpyT&$d`Id7}0ayDIZw$wUWYwc~9?H%<1FtkZZPqVYP#nFpQ%huCu z>xHq^*3;(b?X>qm+9x{!U`?a#jT2oh6J2Oyl(CMcu?`qNHav_r-FVP=`C$|4!2tn) z4YfB9wl_a)Yktt$1p0NWx$b&H&6Uel0I>er3+?ABuGU|?+tGNxtNBiM%k92S09d5U z#>W-Pz$%|}D&PVXO-3aQ?(s(M|Fn?#|3|M+0E^BadLu;~s5--mDB_Mx6T01Yzt0(f z*96*DF1pLy-vZ2q{)yraF$``wBMgg7FGjj_j7&{bw>PPavlS&7x*fU7Qmeecs46sP z?1VGGL+t~dBNM}bF0{pu+l|_DqWE1NztbDAdjd|6-|Y*+vCZupb9#pDXpuh9Yupu% zati`4@(GA!Oh*RHBFk zp~`3sC6gc|Vp!N@1}>3COrVfsBLLe%By1*Nw-B*FGElAr2K~2=h>T>^SqcFRFXm!J z>@LOvQY@ zHRez2H~x9e7pW<+IT^COd$RW(E6v}VnY%Z0`_Yp9=k^vKEy+2MZ{3wG%hjk0P3mHc zG*2ha(J6~erk&Ybi&SD&X|r{5vs{^@k>~087CBYPrKmV`9S0{RC34bZs41~D0*1@R za9Hps2Y{)>IV!GM4gh1Td2|Vb$OG{tV`#VyNwJ@8TL0yijd8eSv?`m9jUy(nPu{XI zX^T>?s5o`3^7M)F@?xn%m`+Sh#3v>Yk`nPr05AeMU8#~PwQ5sNZs*k-(%9~$T-~Rd-O(dc`e)IE_H@|#j*qdKJPA9(pC3?~8lZt&JSwz300E4_= zdiMR|^Y0g5AcieG|8D*%e7L`vd-87P(esY$*Tfb**Q6BZWxyG@G&fV8Z&el=l?7^9 zrbLqi0Lwm9d91dprE#FI{l?8J!=pEC?t8Aluzz+kv}6mfxaJ;v7oP{0UxyyQ2|s-| z`|RE9lh>h@#{>6o>5EM1{B*nsPnMB5I;POflUPJzvruT1^7LwsUPIRiND3-ZP9cDD zNvUaK5>7(HOX);8lcHkL)l8m&BQ^=RDlSDpXNWmUi&~eXw-j5fWmaRcS)HZNDlP1} ze%m+yXnN&U_z_wXH~;9>>?$0;3avgJ@w!{T?JqpABV7Pek4@$Qy3!Ll*aS8jF#>!t z7ng{7x1_~UQe!E2szPPmu~$=A%24abA}L-VP8SFh3U#(hm#fm{Y0);>1%{j=Q%PCoo?Qip50oAJs$^$*=AIqddv@mT+>u?nJuA1& znq8iC@bvMz-huX;qXQ2;_om?FG(6>7a0OTV(~qXZPr|dSvvbRka2ToBbOlc;JL7eS+;0EG#6(r~IlV~*0MlqiRw!)`%81Kl;&P7%O5Dq?=LHe=IZv68@spN+PmY}6~tDQ}?&eqEgpjUgHqYKgNV-ZJx zyQ8lSV()E*IC>GiT>U-H-tGv-+NZkP#yeXkyIZGv+F%5djd!*}#yVPtBdrcmkNM_$ z$Z%U@wBP(-1Yi$Z8}ByN-EO>mv!NE%Zq#1rzjVI)VkIyP=~qYNgPzvA-7RzwpCCOu$aAFXTLXP=0)j~b73lbQrpeJbMkcnvIa-RpONMfN2C z1VsVnblL%6KCcU{5%i)xQBn72PoxVp0vP&2%`+ok9~p*Lk0B3peuSHI%H0WM|<3nCq`V1B;fUiyuOgj7jk$3T|oe^Kv2K8 zH{|k6JH0a#?$DUiKjsWhxWnV_@R(BqrndcpMLp7Z6x#CS6I#@(CCwK7~of zfN9a`@u&!k5=SP-!3ijd5Q_|q5d|2G$&r>ENo)pA#3Rauc)1WK6%iCtrcp;y$`Vio zG7X*D@>zH(8!rVR+e)sKq+u-88=zX5># z6&Qvl65oD6#Tsvafvmp!`Qyz<9mk{BzaZ4T`mpi}RUxjtLbV&KFMnKq`6Jw1e)(bf zIm*)W_e(E+fGod6SH%{deFrd`fBGHDlh?xzcmAI3e2YqwZxQ5}MY$G9j!}`Pm*>dU zxr*E!*}ISIIa`0RX`rv;)}6k=(OWjpLtl6#v@kKV?3jJ*S$OVSdKp@MJ^d(3ukXVE zFaYG!*UoVGMC}=lj!TnM7)qMR%$H}QCZ5V1h1eqJ=)px~Y?FwpWfGMXoRkazgC7H4 z$|TBJa8AM0^64rjOUn|PB@(kjtdmM~GMPbHwr|J9=GxlM#_G1Zi!F^UeOE>uf#Bk^ zFluu6CXDJ2(Z(*Gg~jgsw@MD}VH+eE9ubr)iHpbZaS5!{2*A(}JBbTO!>4cw$sBwf z1AgkbBsQI)QPEV2bOA4g!@+Pk$!ty{>Q2o{VzLtHjCd+7o<>Q6mo=3dsn;OHMYg$# zC89BrLP(+#Bli$v@OS{&<`m54l$6+XY%HFT%3?6o60S+kwIfxUSaDvVFP+~9d~wCOtl=fwHyWUa!<7@_c)wwzw+&vNTLKnc= zoIS11{3o09g`TR)-b?3iwbtKh zZ@Aal^q{-tUU%D_zOIP}_a;XM$Au?~pp-W$Our9=&$X66#UR1c@j`nInn-`$3=ryni zz&`Tp6Tp7|Lf>|~oNm|f)Wn&~)rO+XtX+jiE}iVUHE@4oXv#eWPXkW{(DM1vesyTu zRc{Ez2LPLP`=)_mz%7?|#)$wHw)y5JJ%BM#E;O|;;hjU)^Ip$0hmn& z7AAv>lklzJ5=vlUA}|ja8*znh3=Z!-ahR(WGF42fj6x8Qi9#|{Md#`{R0S2sC#SNh zDGXX74V7;Is4y(l^AOH8fcJE;aI4Iwn)YNM&PEnHT~Gm(EDVGE>r-DR?fHtKm>& z42F!wSM$gMB1^@FlVG7mNmcUV>1kW3m{@u`e2B%-@$n4!#G|M1SQrlT52|9Lrqrw{ zF>8tp@@y4D#!e(+*2Qo7Z1dVLwys+fv;NDN4eJv&#o&`;i7A^239;lvs+hX#;P&>O z*4l>4g{377J`3C{4wtYoaqAbG)_<|?%h>oBEFN#l%DZ}ZWd6y!WpJ;^t7s}5- zuccQ%N3VyiqWrRoE{uK5YNP~l6+Qmv<<~zi!y|!MuOjGri7tA53@YdVhM`%+XlY_( zE)i_(hlOX~qda{ZT6%n;rAeM^;Abk?76sd^y`r(=-AIQzzn?q--sp zspfEWLb+9~%Qk8<^iqRr$HA}ayE3${+U4Jq|6shyrW|JZ$x)e_^Hd z`W4Iee4>nzDkP?f$Y~-fMo0pN#k0~ASXc;P40Jn*o1V-ir0_`*$dOX{46Kli7xUo< zkNQ%hRgK&vChAC?K%-%pOhDJx$n`B$a7ZL>#p5INicA>daq%R43JX+>97Diu#o=PG z*p11F8o?lnF9(Gs?FY zZQoJ2dsoq}orT-Va|(+sIr;jmB4bvGxp;5!soLt+tGD~^PJBD!x<3_+0Bp$_Tyck1 z0yB@N=OC-ovk`#JF3q5_j>YLnX<{fahXCdU2?GOjcXqZIO)9a3sW(XS^Ymr;rt$*Q zwvx=eT$5ZS;Bbg^Y7*+u&5UER;>q}pMHa!0hU24cr>ELZ0KlA`C&0kmU6l|xa6q~% z9Nj12ep}m7d;1CCmb3kYqwR#Vqr%;B5j_u=QD4Jx zZyiY3VORBGPxTS^#UoC@*o8y3bBAqb57{GeT{!BxaBS-IH^WEwJ~*^%=-9rIlLtpD zk4&Dgn7nXms_L}u;+cu_C&$m8965b_gWM_wK;p+9gw~bu&b%w*2&%$80~!>&i-zMuD&kkKo7*$ z*EQMOInmQO(c3ZJ(=pcFHrmxX6w$9Jz#=W@n@2iYhB}&~0J|UAGvy}GtM=lR>hrx9 z&vsup({=vz)%xn&Z4D25+JIgWfVJKm=&}z#9C>i}*7YlccW(j7P)Ee^kx0>?%VoFQ z(I%7Q_Q_E@(91Svw~aY~WcG=vDabh5dfPtbL@I{jao9d}-*7n{QF@_kU{1Ry0x++~ z4cg^(qaEbnzaUAGNoLP)kNo}XH=H4!p}x>A1h7e`qyN_R;}=d}?rgj__He@Ka7IMV zk9J)R`Te0l07lf;4DG)f2^g3M0OtKY4)3(hJ2T~(ne>DwAYRDaq;K9HTyjpYI6})& zfKB-3$9)my@_~>oI)acTdtk{1iWadn0NAJ(0c>!}QCU|bHp{@e*cuic&WxpWij+bW z69HfVgLEz>g-J_>_cdzTipy?Di34aS7yB3qUg^(Ic!zM5ZX##qrL4km-fwz{Nu9dgO@UgV?tyCf~#%xPKvZQZj#Pf# z(cgZ)wk9J#C!L0iNso)iC&i>Btl6~jFJG=*vu-_(NIh`$j4imd@cjMqD>QA0IP{5O zuYX>A`7>hJ>tB{$M@jYzplcZc?3ZYMdvN42uYO(x1Of;Vxk_fGGdA3rUui1Zcf6G8y_l>JP zw{G`87``##zUK@N`PQ{Dq6a|;A5k(gH z@_A;tz^veC#Sp1Qta;?e?y|uV7Z`Uqsc}dRh?ME)0zjA-b9bOGAKA(v!{|c|Z zntk+QDj3>%DS{_wuilgGX)Ey>p!)I1>vPe@6^Y>rP`pMc#Mk6BA6$DP?<{GhdR zy#183^OU>$Bq})UIOXX&8*74opJP@a`m2c^;9^!Pr{waRKKzjVx91q=fx zJLbN0A{uY)Nl(owXH|v$!tsf-N2bmnvtK&txO~Q4f5B6K(bHJ%Zn)^Ud;tUtWXn-g z<*2>rynNA7S7p0gWvjnvYr14_taj8_+kj+{#!L3*8b`||7f4upqqC#M(b?wc>Tq;- z+Izc@WA%4W^>en{nh8XE}ZT-cXFWi0_Yb2?0#4CU~l_SU;BgJ zw)}_4%uxJPgYmId;Q(&@?#L1$;pYavEi|?LA0J~=-$Za z!^x?UiK)?v$uYZq!r_>-qjtR?Nkq5&M#-_&XKrbi!LbP7q??uE4_|aZJ!C(|)=%(L4 z;|t9A{4*Y3*aM?KYGX5Y?=%2x$~!X&LE?qz1?U2RLF}QW3Ggi+q8gCQ7NM6bwB!sd zIsA}Ch|M=Y;h!IN&kQ^L-#!@HdFUWV!>21~WC@v}U_rJ8e}xBn5+^~lwyKZkiw%UGKg_xY#cc)k)EDN zNsT2Y#*vc}$;k=Slo$$_943L5f`OB4DkhPV%$2kF3T6s9K9LZcK!}S;-x5!ZOJ!kk z0!k_$ZC4!`0bn=~uNW!;0>4EnpN?#dP5OhbJR@^oLFw_G<;S*5tYTnbA`x^dF&=Q0 z8oM!h>qbn>2F%vYX$hOs65^>Cj+SF7Hl4U|yz@%;i3{gdRwF!kQ_5BVSON~Se#_?1 zzFhm+m+Pr4eoN1{(~sURy!-&H`h=@UJ~HXkJ!m@d&9BREA?SD+{OrvykdKdi{qw@B z4`>Dwf@%)mM=*va5?5aS1O~SB?1xVkiD<6!4()a}5Gn zmRyyi%qq{k*j#<}?#*lW?_M7qyfxvt>kJNpvxgqpW?#4#U;0;G2Oqr+Jw^bVefmE9 z^uz4)4|C6coO|{|=+U#g_R*u&$1K~7>U_B>SE0;T$n#}#pjV+%lrQCE2pC2VRmY?o zSey(V%fz85L49Zh2{^KlA`>vQVy;QXHb_y`gGMT~>J7y?`oe5&UWOve$TKKuY9U3% zrz=GTd-vWLp7hQ=4lX=LJASRc3O|13o?AR#b&jRt5oKhol!TMfFk(ugfS4kpri!S^ zTp}n}x(J+)ij|OYQaV9~y6XZy`6ihpQ$?1t(s>M=kdNi_()qkJE*H+xFf1mH&&LY| zfUy)7GnL5%n58gTiIm8k1DO<0Cd82l@f5USYYK;v!lI?X84$dN@JVs_3B*ynL*N9bjMF0 zuWM;&?dfdo>uMY5?YnvR)}Z~~sAtd?7d;#9%?&(!;_^~e>0T@WwEJ)Z&I69FS zVHikQFgO?Vhy7>@F>vF?jq+_pN|n%RQ557EaxzrAOS0SQ&NtOv*tV_MU^R*qQUGdF zYD`?}ra0Wzc`;^Z(pUa|E#aKGSGWA7)^v^;ofsI-Djt}Drb5iaBz0IqcYfbI?#O5S66{j zd%|0D9Bp?A=&GskUOwfkJL7Gr^fsLHH(m%fRRx-={E%jpKx<8)^>U!S!QWc%X{z-# zUks=ji=Vf<$gS)-i-O=vp z>U8&XIeWUIn&r6KYrEQOzt-!x*5|m=`@4AcbU=V!kg*BzMtA;z8fm_kV zUP#5@W=HeGwx;{d^|u>puh(7#di7PG19Y{YJuy&w{#MK7Tdj5XI-9`20AR!Y9S;XO zhi_e73%{)d15$A5nF=DV4h+2PTV?(WXI`kMNN+LqRa z^B2yXyKtuF@}<_c=BwAP+`4`9;n0IY(73S?K%L!&`fhx@b`>F*%k6Y~>=2LF8J#xt zdICOFiWo$b%x+Z2fl4}1Hc4-3I^*Px-?3z??IL?8<>A7?zQNg_b9SAYl=D z1r{BFCD-&a1ilM#1y`KGC1BpTH$3DDT)Fpff5ma3K|xorXmTn^Ok$}ye1m|k1$6PO zI}G|#EmlZQVo}m~=vooFV#McRxEx>@>c=4#(zR-i$-p%m1sN8u$-vg@Xc`p*?lcnnJQZkuGX3JPp`|DA$bZP(Ta>-O7!0iNV6iMi zl1PeMpR{gs%9gFE@$kVEN5E`MjoSbuOH0_07N5jOWhiNuB2#r^ZAX9qp;Kq%CVc`i zB^H+uk19A)HpOmTyKckhYu0N`StIt~!iyglU;n!F`qzaQfGZ#qc-3E*U;p|y0E6Sd zuDtn&j|@Zk>rynYe_a8#{Vl*=qe>8T^(&HiMH-_0wxhlXztkKP@(J#YpedKX56kEg;f zoQrRKEAOK8ntlTG`hNE5kF(DJU_Z`1|9)jXn_OH%MeW@AI&Rabb~KldcG_##ru@ix5r;@gK0bp;l@h=`X` zuu@uzketjXrimy-Ig6xVM@OofOHr}ON)AQErKGUOfU9nbQ0)IJXeXfqDIn**TUBtEMpDR8nZEJa*vL zKv#1^P5JgxW2RN6R#9jqOj=wrZfgP|Hl7rpjE}{oZ{3lvz14hbtmTvg?U_<(Z$0g3 zJ>}{GR-N|@oby~c?;AMp?LX)3k1(vi(%T;aSl`)D-??D_xj_H9D8r_E&LVDge+1Y} z_c_F{&NIQbGyayd{^m2m=ChHU^EaIhG@cDLpAWTEg<7jZZ5LiU5&xs)<92-ucHy-?`#TmwgkGG{ar1A?p9w{v$w0+*WCi6r>n^ekLhai z^)!2Xnql|E(}@6ftenkHr&|?iPmcj|2}_+YBxZ?Zr0aayL4fo`h4ew(=BI?^;TDY+juELug1aN z)+oS+209-MbdKG-@#@*5*RNkMuPiSt&abR2K7PFX_{s9imrtKPeL6ioGd?!)?YH0d z_H_64_1w90>*lTNR;wu`4MU`m*j%nct+Hli7MGXp-nZ}QiIdfrYCAf*u3fthCocEz z-ya(vpR`R)fs;8W5Wvv(I?9ydR zUUY^&!U+I21qSvTz&@H8y!b=QaQ5a3t)NFvFWG!^qt3v$!=qmvKg!dH87dB2%VVmU zGzE>N;qdeVwvH<>bMy9Dvi6#pdS1Gag%t`R=>lOYUy#Z}oj3qsAYR~GQHF^!t&*%v zIM)+cOrp#TVNQl1$09Gz)RkrFi!%fkIZeZ78boXpDlq1m6~YXyDAOd$G7GXy{0uGM ztm2tfbgh)E6cJ=XR7}rh#89!DiI`XtCW)R7Nu;JH(y^&*5?RC~3g~1JgTSXHQPUDA zsi0^nbZi;y=9yyX zX>kBB)6Puej?9d`xz=6Txo?!$*A1{I!7N@wCRl(b|*YHVWs`t|ES|6={N-QW1;pDn!jY4O!B zi?6`IK16*BvFi0-KS>w>48Zo0UI=E0XDEns2z4ufy;r|1y!>(Q)%%6lKP|reY3X;L z`v6%&1sxF&dx7+8{^bu~V2dw)SVDD)kppC7??Nk2`tIH?`+BD?Pn%tyUDa^$!Pw~F zxb5Dk?V-&(?3*18u1wB6b#W<28QZOyJ2JBNWa-Kcyi6h8#ARCf9IF5TMpDx#8X8T{pzE1r1q~~pg1CUMXt`*>dTCVQlqX&5AJ0yvdT>gEDg6MG46{^oBz0W z?H|AR^3QA5e!gbypFjUHCMo5sLr3opj*i*f;||}1(?97B*gVrykhCFO~ArzVm1ZQ0?|KP)+Q>QCR zO7cz||N3&x*}nG1dpE8$)K!;nD>dh2OEp>&ot&H=pM;Bz!^I>J6ELLs6zmoeC;j4~ zl6&>XCRo9;b_7}g!Z7##GT3&C;kg_+)}Q1`{@&Wqu$OEVo8!=2S(Gy<{EW|-3*)oARx zINMVb?yi}EJG*Lw-SvJ*Pg8K9Ei}*(=xs%C>I8JP`n%ggeI3ERjzC{~ps&r}*XHZ} zEj_K?-WE@9ix=H&_jPxA+dCk>&MsGHrwtf(rN;@}y54WQ(gSf`?Q>u2cLBz(^*aW7 z?fpIWz8*A-2rAYC0P7s>27t9k=@nh_6~!1@`xT+rjk-(MQQ^jgo~kp=r;mbuU2nK} zqw&)1mbwSs%@KgL4)t{mU+sQ)t;aF;;Gh5Dmw)+>|MCz2_|HH8`pbu3-oN|q)%1*i zX!!mJyf4Q`?GAeoewOodi%W~Ek5+*jTAlV!fBw^FYrg#R7hnEm&Dt;4ZCJB@(}pdv z2}!BxcoL1tk;s&KqdC8*=)j?GS~}Z%ulDv{>ArRQ`n`L%0cpt4oNmOVD6{-tx7X$N zIK5t{*XKedgh<7JTLBN6X$*Kl)IKH5aqlUQpwWk_!f&WBSVsvCC<)~7UW3^ zvlXRT*1bhV-|R4L&lP21rIN0VadG(6L=rBMN=WC@NCG;XO&_c{+IQo2V^`0%1N#JO zDTzxZa>?lod>WOIgiT$$e(n0rac3{J&OZ5m0eJHo!3F>pnJtXU7V@pX1z2<xvl5O^hYvHATig~XI9?6TuqsdZQ)UL zECN8wz-Ab^NW!!zE#OIim6&2O#_xAIdG6^a0cjei7qLhL5jg-*R z#dNHcfd}OR-%|6LdNB zRV}rulsOtxnFXRRGMb95x*Q`%#)}7i-@4_qb({Wf&ALCXS@)MU>%LgC{GmFo|bI-zvVNYh~ zAI*kW=jQ-m;9*Nq37eilP3waZhQa5~&6_tH8!q3xab`mq18JCMKj2(7qEUDY^P!?oi9gNdQ>)IakLSlx_gnIsZVF9~jnu zK5zwcE^wt17#08t>pSiHsA82L8P#BnttuXG}#DK=$IMk4E47M zdRhTsUI186N9bzr^tE1pe<#X7w|}535IOerc6fT*-97EDo(@k>r?02W)6wB-YxlHu zxH>uS&1)>r?JE*WawWbmMY0D%`kqq3isa)-%UD&sSV;sJ_uu-CK3~+xkm`J&0alV8i{L zBiDK!UhDNtjsBPa^e_MWfB(P#^I!j;|NcM!$N%x~|8Zh!WY_NUulDV9x~AT~d-cQn zAKw4;;}1W+|L*+{@7}%h`Ta7v{4bxwt7hG2U!s@L=U;C4?8^;*S+oAnU#$E4&)5F_ z7wi7``C7R7<%UfucwEu8(w+Nv7H=y$a{Qa)6-UulsaLNxHn&`=sqMcqFln3c2R(t1 z*XEcQpFlH^;A?if3$^KR2a%k)1CYo;(CzZO-Dqi-#~bhf&wMZjeEwh{3NUc5&@7Cf z;t$S1J{||qOA#1`02Y`9dPNcjfH{H-z^%W@3RqV(k*i|pE#MBXc*2jIGswLhfu#vg zcyQ8ruC7*UQPLD_mWIdC@E9r<7?{Aw7n%ehVH`DEl_kpi%2f1CcE+A;S)LjA#Lyb4 z8a-8`BP%sDjh>-1QdL^2N=H#@X(}C4Yh)YD9HT{O&6ea82(okd*;Yw`)wm;1Taqcu zGpdR#;%qI)BxCC(EWM0vP;gCZ2vZC6l2BFfX(lAf1b7LT0G=dbVz|^e8WCMQBPOM> zi3~Z9s}hS13b{!oGbvS8tpYyllp>atOBFD2EFytJ#&QTLOdKSgOTvp8X+nA;mxAHb z(nV~Hke$F|V#GY0QiKDBX+$iOjAN0(S^E}x+8+cKp(IDHEi)Lmo2V z42d1`Fm%r%np8wM9kQAi!@KpG6TTaL@k4)VN$hhxXCcGsd_5M%;xA=Bq15kp%Qr<0*^}&a^XCM zW6-Gc^OSkHLW_ZCREsPczCi(0;F}dZvs{>oiqV-$arwcoAKKi$`KO^}1hARMZ^Dn? z&OLc!n-1?jafBddrV7bvQd+u{PEfFLXeUVyO(z6~fq0P=9I}E#lCha;fyAOvW~&W_ z`s^Lq`a%m$CdBc1SiT^QD@bAUFl;`CEllA+_?QU5()fIWL`+vHSXwnptAWr|DvClu zQOHR$30tp}W$BgKdUd{0oo5i46as@xV3bR&3PZ8UQkG#Z%QO~Qr6wg+z(~L*t=X{Q z&uiBI-REEa5k7s^Zh+J0&(^FbGkFc2{f^+gC$tz0uLNgTVGM;)xqJwWbM8rS9=sEw zYkL0K%>1+2#TT;+&%wZE7oN^8JPC(aW@lF-0Q)$X7)BSv7DLmh=da%%nwqi=!cQCZ zW`$vN@c!MJOBYMG@3a;asVr8Oke^CQO&}!2VBm`0`r+-mi9x4668T)iE>f!^S?0pCC$9Qd!G(RZcW+u!Bx?eq@x zy8F6aechhE9&cZdr>D!+)#2{yaCLV&`+FQ$`|OBbkiMyb9{ZJE=aoKZ=VQ8w=_V2URSHnU$3dUcB!i8!kL!S$GWRd1HJCF)!%8azg~Co zPD}kzAG-84dZlaZYR~BP{)gB4?4v{f@*n=)zy7a({m=jPzy9@~|2jH8WXUwdC2ZcY zqx6UOZ~xc-{XhQOfB$d)*MIrX|M5Tlpa1SZ{QH0Ur+)%%;_$eC`s^>CtwopE|MJDU z&(>`C7{J(_#M&Z)|M5*cDxCO^EKr%=uI2#P213*_GIEUh&_aT7IdHl%00AS9@1u%Db72-mm zTKxpD$jl)q*D7)^lvQ^`)NHgVaRDTC6%V2GgKUghQ-#g1tx*OAYdq& zLOoBNB`Q6bx$Q_n-u`^c&SFt!HcM}zsSRYgiYQf*HCmch5A>p|jSQ8MsWr0=R*o@) zYswU4=85wPxmlUKEQ_i%*HE4>wCLD+g*3;gF3yzX=;3pTZIA-PI0hxhsAlSvbd8** zkx^6-DMc+s9tOza(i0fu7!sNuqzD-jy+Uc$sjYf#mR_5!H{@E3c^OKJPHIqzbqcnE z&r=H-Qg%8EpF&T^(6Ja6DUDB070^@p6bzS=D&nMwcxf_Wx%c9^~K`7}P z$Vq>Y*tq>Lm|N)izRNg?5|YzC3fq3}674}Lvx z>u&#z+h=Po=aiMOWc+kGK8=P?qmwWcVgf$>&+FH-rK%Cv%;F0m&5se-eE1ty{VrjV z)nG`rA|8ey_Z!SOllJDjozYHxs4lOweOr4}>_RKyT&g`mN z_nbe^o;`5CPfGWhd+T=nYSohLB(C&|p7p%z{oaoz6H;XYj!GsrYq>f(O~z%ZcwCKu zDrO%!e|~V>Y6YL>Yp<5T54rPhdH3D&&YOkp=S>5*XmVjRhY&BK0D8p=NHIc6oS2p% zp~nd*5lq;)pCADDXf&prE4L`*Nis#UOqZ#ZT9olDAYOz>*to%pq_M&&%rFWol+1(- zqfo+W)Hp7WAQqG5GKxY*l1ae{NhTwRB|yS(LOw$+6PVOoos6p&OOw@Ht(YohQ)FzG znlDdLYclof41>rZqY9aLW@2DW$bO_3aIgR7=JDU$J-_wz0(UtcXbeZB?7cf^UD=*p z-d=);az5XJe1dzPE!&?jJ5K<&;K}N9+vg=f%-&zI+E=yOTVw02Ty8H1rUd}D&{nRtYWbZ_*TV2 z`;GaQvbo06xu)`&rplSt>bZ`(*|xgb_WHSwhS^S##+fdFuJ)eTbrJ?H~$K-Uso%G0D29SmjHV8mR@f!ywZI2BA{1)b?NQ; z3IMSCt#v=P*8kYrIM~(vq_6E+eW`i}N@fr-en^Zj zA~!GdkN^Dh-~ayCzy1y6Z-4&FfBo-2|L6by&;Q)s-C=Rq-$DcPI^Y4hweKK^*8w-g zK{vz!H{^ad)P6T4czM9x>wqU0?q=^fx5hmR#t@i-EW%%IWPtm9|Tv~~A9cr>_d zTV3DWzW-n_FaMZGrckIfdZRfbJtsHs$jQ^^uND@UmRHr*HMg{NclO-u8@Ttw{lTY0 z6T@TEQ`7Uab6>4%9&S*#0?`HO1xYqP2luOUkqlM>7VPs2_PKfc%$#ioRu@*L=a#0y zmAPP_T3nlU0?frQz%Afdu%}~wZDIjF6`NdufltU_EzvPNz0`I4jxj5lA|g>FIJ_Vp z&&Lr(M5=PRXbS)-#F^t=?4A}2+Vqz5PNC8WtXAS59J&mm)(geWQ&ZstOY3uoX65(bt}jbc-x zIJ9U1J66oYt0Y9V7%Sz(OBj)yM07&PZ^Y>`MV{IK=yg6R|7vE|`J}W{MtzPlk&E|@ z^z-yb9>jRKVUQjH{^;ostimEHHYy$>sUA8flH?Rx-SKLh##{aStbVQKGq&*OV3XEOMyLQ#fDm@1T|$+X#8 z!y$tzN5fARQnd_%ikhIH#mnee1p}{S;T24>nnl&Im?oAaU7$!8N-Z3gf|9_Dk0!-N zQxb?Gp1^2OWu(iKlLR^y9b6@Rc2Z7mQElzj%8Kk0N5TJQLxxeLQ54*`@oaj2dSz$c z@p^In{nGZwrJeUnJ8zcv-aN6+rye~*Q3wcf7FI}s#X}hr2hJrl;9dZ1u{y@ zxtt_}9ZE!*gdhTXx%q&9-iHt`CE=rj}R1=Ll>D zv%&>BaMkMA#E%aim6TUx<>%?sQ-oS2UBJLm+8w0KJMA2TGUvOBee}0kjsK zY@6-9KHpQc+@h7DP8TovD#bV=&fAsu3QBM)^=mDt#qlq6u1|Vt);fgh1ROM zwyL@I>bcg+`Q{4PdDvVv+fqH-UOU%O55x;(zN=}zr+FTz7_1!j&h~a&d%6L-W(Inv z`@5|JJ+rs^LEy>F-i6x(fMI}Mlii(Dy`Au+2exSddckL113fN!0Tpu=5+TC?y@tBm zpLMpn0QRh{`B_U7I6nT`xB-_c|4?5I+vTe(TOoum!;Z$mj;2TLP`{q`wGRz+JsaqJ+~4tVpyy71_wnP0$h1VCK#T|4-3#O4?TzyF;;Ec(vvfI9+t&=Yyk1L-%94;=cu4`=X?!EQ> z{lO>C#>OXSXXck@<`-udVCELV`O@6N@;t!U;>!FIpqFiS!9KeH(FL<$w=S%J%q-fb z7gvE-O)qb`7zO|aLFMYySXhI_!zHL+PJnFyu#L{H++42^-TP!r?|qDsj!>y^2w;&cODBPA=Oc!@AkDkdvsY@jPu zMX73@Ma?m)xh5^oq~(~jbc2SZmJ=0HqEt)}3u6WB5E>pG8yt*_;41n0R6|-$dd`u& zoRfz$j_0Hu%{Jwvt5YmugNm<_u+$>1R?3hIC=y6Aij*Be!v>OK!suAMh|W-fdu4!0 z@MDW`auN7b2ntcOfEgoXMsi4)_>kWS)8*1^h32p(<+Lg5ToMQXm^xhwK5Bi#{XD!q zzwN6}(zW(BXB>TEtZ2jfW8?Sx`18a|8)?R#Cd;Ze~?68OXJ39czU;xA3uWrA!Z@;nc zzFpmW=h%Jk*!^JN13&Nw+l!Aj*bMUXsG2VXF+>4Fs8@@PdahDN7jX$pDuKy3S6K4Hv$4UksXGq`yKeQ>wKWx#m$u)$H8Qj0 z1lXIE%}*;kpBA^@FYdfu-hDIU*t$|tm3s21;YcP)!67O+L^YSF<|WG61SyLEx8q{7 zH9}xQ>ST>2$E;0N3AJ3VhC`RJqiFa*LP98+9L-e_1nuw^^=(rW;)RmTe_z_vl`w#}7g0I&_4eRFxm zHE0TUB!bM&F2cD%6V@Le4qh)UwPdCWwNi?JPGTqGs4=m`u*ihKhy?$*#Gpi41eqC4 zW=4?cp=4?ZpAmjCU3{zJi8WO1NuX#fBW zSXb#1U|aXKxt=1|W3Z);>&qQQw$2iJSGldT+}3r&-d$nqtbjE`S6Xi@wSt#bi*3~l z?bWblqorc8wQ8ZcdbX*`+Eg{uRs*aH&}*T)X`u%;YXE?)40J8_bHI!WRbRi$MFMFn*|j5Z^0>m;U)* z{_tP__2<9+?Qeiz|Mi#u`upGh{^tE#3WL7i!_5{Aym_uYMP^M_|` zz4t5Xdrn;})>tx_0vUA;Vzv;rJXdGsm;3xw3{@zLpQ@#Z^<7@X6j>XKj80A^%`lDG~vz)4tP}Km25@1&#B$bSzQwmeGk_??J z!=%YeS7fCEjM24nvRXz|$r2S3l3bD~li;PocrgpyL*uz*O{zBcR9^n+V<#@2J$CtQ z=BdM3r;nu^Ic&(vR;Q&&%zBAMCrQ$CwFxDPQdTU77|uwDU?fBV2@_DkJuywe zCrG$BDIYHv#frF*d`7&4OOWw>aZ$gKW@+R(YW)#?$|-a531jLhQ~GI(*u)RVh51DU zp#0GXk)GdqdF?}@zC)t@BBPI-y8r-I)7qYU@)T1h3nu}9#rs8syJIllc_O~`@bV3g zu59jG-u=n8_u26rV8Gd$@wKba*_Q#&A-evd^Uy^yXkZ&}{80_xrHQ$P5v%p_`1n&`4+}OcY~pxl-~O_) z^Kp6S18nZ!d$+Lls_o7X)m@$K_wNAHKrFwSj^$BQ0;-lz)d|Q79#PIC$$1QwNT^q+ zl66Tr$(cve%~^VhK_E7Y_&PC`MTw$OfPTeuz^xfSp2q{+isK04c%pco5HAt~_ae(> z&fX0XjxQt#MFgNSBxJWZ#|wW!kde2s`D zWwRt4GH@9()*lxYkPs7qjSa#^2gZa2$Aw`T#M74wre+*)w`j-f)%ACd&9@+{8?Ry3 zUpY1FrDOe-WBoOtm+PeqU^dse1E|;@q?a=*D_bion;^dc437Gp2Oc)NXtmBxPR{;t z|KXMEg-VNt$V-f&M8#2}u%w7M;D?k@A|sqDN{}fjLJ5J%ivJHQ48mG z`SFp)%Yb21?M2gVMbqttfLnlG3%zA?y`}SgHx~QLfq{W6gY-G8hyY++Mf2^~Tv_TU zUhXJc=`06X?z*wmS+Usu6=2IPHx^rNEVP1ms$5xUs{}%}*itzMJPc}CwY9Z+3h=DG zezCg=Hq&=D%y$Elwaj$3Si9P0dpl=)d#3xkfLcvCkK^5NRWfig7yz>#r+#&JOm=-; ztvuS*HrfjxgSp&mq_=Yz1fZ+C9T?b9R~wM9r@)-rn*On7c((?is|=*S99UN&Ko=0N zwyPJLuABqz)mvHIUkw5FeN*j^?G2D#FfEUJT3z}zbgTQ>&90{dJpf?+U9B>iz&8+s z2}C0UP+opWq#yWvK_Pv-5{THkrkcIyJOAr{{ojB7^Pm6o&wmCC+t}HlvFQiB+!Cp@ zqbJVdiL`I_x$k#FIst~*?}gash1`cg?nij;M;_dd+IIl+t%vu11jY@C_CRA$M~)nO z@#;-^O|_>l1{3VBG@0)V4uhM*Ve9&%(WUW7ydf=DtTFOr zYKA}(g-!Gei~;~7QdlH9o52=Pm|QH8f+tbA0!dOz`sKo+j-G+RXCo8Sb5n2$^YT3G zOSI1|*deYKS6n2sf|qmP9K5+|op-oOh!9=C!&aPG+L&H+@@#2qdT|TD3^sQF{hG7i zeDo;gPzF^Dl#9exk~k_dTTSI^7=UL~5sssz2{aszicJ>N5=BI+91xC1mr$4r4wg;w zjgJaS#7EGnaa?vhpO2TxX?h)9Z=mZfbc30$*YlE71gYu5v^0Kd5uS zbghi6k`v_;qC!H}C}=trQ?KEt=-4I&$EcEJB&%}MMQLWXNln)&sXACiq-m5Kqn-(S z)1+LJRBBNgvrNY>oj7s%?D5OzPZV6tyKpAw%$fA#$J34+P0l~0&(6^0q$$9ktJg5) zVgN9zoJ)`}v3yDlJ28rd2QZ6c;)8JEF?77h41~c5=Y#MVQG6y=#PP$${zjgqR^@6e z$Bbzw&B@1&$;XW5d_70S48w&5#f12T`MG<0eTVRb07IZcu?ZK8OFM7hEpKc}J$#rX z5QHV-{h}k?e0+ZE?(rSGhR|_|jK2Frt1rH|o+JT)0Uo%PcKyPzf9%@$)x&=6^7u7> za02Wr!<-HW)Cv|G9nQ{1_$2K0XJ;q~T8Tuai5zHY>gaLiQPg>MOTCuEQz7 zAJ?4i9&on(tIw-2Ushq?{}({7<-Hg8$A^y>ExpQd56V27uGXAuZf_@Z=)*T^2H?5nAOca&e1ZC@p;?C!t&(8(&Xautk{&C`k#*fcDgiR1CGJh)-2bE{TbG>elUkl;l!id^ZeA1YuAlSo7m3W;Jd zO)2LZwbE3xC|S?dD_ANqMZ%*<_$-x#t&*};VnwP>onZt=p{uc8#-)o`cs3<8F)jcf z6@ZTk#K#2TBEs=eG2}R^h;^;JdTwdk>0WPEH{Pyoy#sM!%(3wX_!fv`{mt6OTW|uE z3m$=l!E;znTzg?#-(7KRudF)bY^?tZFz2L1mw~zVSecrfyZ6J7r_Z0|DtK|Us0dnUW1*!S01R;JYXHnZ+A8N-Zp^h-&b3!hw^V@u6`Sv@2LPMz zZnC!5TRWO&yIN84127DFSZ~*; z^SRehx07C7ZBIH{TjF^_3NNUcb_Q?Q&b;rRK}$TCQB^ zD7xZ87ku>93@2{<*wNr>+<4s6=G3o_;hWvVw|fA?9uIUqy48ELr(G%)paHZ3F{mI6 zG7yalM49gXdk+R5Jsy5qT3HqqAC2%v=HwsV+%Sy!#AZ)O~C$Cq}-7dJ-dR)?)CL({g$6N~qrPITRQSkuybsjxz&O^uGF zB;XlX0yQ`yHY74SA~r5KGQux7(8t%u*UvX5E>@}0oH>7?uA%kr_YX(Mr|0HZ<{`SE zcsb*kSy+RaU!7f8on3TT7rq+UjBVR$+x`kL=wTbMv!&S#%FlXdb$+L1%4hjUH&8A9-D6+)&`#M4Q+aC1rlS;fp}&!*D|9&;%ORa9N2AMB$}D2G zl9|B3hT|i`a1p_Aq3B?r0~m09c>UJH6HjNB)i!k9x_hmv+LWD*W3htb<2?QSzx6@@ zym|Nqc=-him4-*-3oCn{Y%jhz3vs`L5OS?ehKzv#bIO);K$LTR81ScGyXWEBuV0ph zLFU0<^*i8C&L>-6*1kSegIW7ObAG{BzcLK4#|}B<>Q00JgX@~%%g<}h8TMbtb~s7q zoZs;YJ}rY!&VB}5w7>otE?k3+j&H{n=L>5}6zRa4g@R;(I7y<<)>`rm`V5sSS*lEt zijsuWtRxA;B%tXybS;Oe;|P<4!W1#b#N%3c{1l!jjju=-fxq6s z*kEKZ1{3Pz8}1(x9|n&5k)C)ax3Tx$v$?ggukEwo25rr{vOa6yoLk*pTzj>$`QEwdK0yR-R~R&@>rjR%%cM+F)5V+^N@6656iuUrQAqwctPeIWfItYRGvauHM5&yi(X#ah z5Qav>RI5SQI*q`rS7oPa^HPOo6jZiD*D)}@K zGoFeM#zp&MBm4TWeboVC&oOT`$+S-fe8Y z1L$%A3>;lq1+KRG+OhV=0V|EqA;8vlKx~ek75g>>7>u)_W7*+a*E~N5ABIg$&ELLz z|IqOx6aguQ5*AGkj-`Yp&?1TKXod*O7Gni6yi7xpgQzJI1rcCODkrfxu^~b3iBaAc zv$a3eTpVt=I^I+`)m&hOz4h0oyNYLe%I10i!%E@9u-@XO-Vy*XS1%#}*i6gSiH3_) zjaTMcix%2T=Q~OlK-x>^T1)2vz}g|emRic@n@i`K%NKx(wN=iy+?Z`F1(|OypJ^(c zX)d2_t(OlRv{ZzllQbXNy7tv(nR#$4whfLIfNRh=y( zZB3)?Er4NT&iT9}&RM-$w1e`fv%g~Z9*Xj>4U%od8329F`>Sg5DYpPfe1K=3Gu*$cp(Bky+gc1Vnf5R z(Gl3Fz{ntU00!xcO3TXHdGTi8_5;7b$ZrpL9P~u(_e6f{>Gh4L*FL0&yPpRl#2p#F z9~rd|9eKbjc)y3w0feuY2g?0Bx9|2L_Ivtxc=~#J`k{GBb$!>J$5W1_9q@7S`&IB+ z^y0G}d@Q{BdG+Pz<-L!~JMZV$UQI4+b@x5u@-@+ML@I;p9~$oE>x~ZdMF;u-U-b4x zdHW#H-k#BM5j+7`qt)f+9xf~_?jN`_GCFCUTbf(4&n~XcEd0L!Hn+4kySQOph6_bz z?K?BS0L;3w32s%V=hv;yOwF%NI!ABZ9~sL&ollpLxmqgMKxOMGObv;trT~D^6;y_T z%#>3YG8$XSqRHuaA%Q3%CJ1oxe0&_A7|SC^F)86xxIGFBWT0!+ zG`*T_F#_gsEn0?7%Fs(#CYivZ)Mi>TPv)l_%~qxwQVwSvzjF5Ig|kO4o)L_l&BICmBIu*-7h9I zj+j8AlVfS*5F*YmA=W)K@L+)NZ)C|5z%5I*R;uSw_=%B(h``vO;P~Lsgs^aYWEd{e zCnCTr0ORh9Ie#u%VfBA9a)vr?! zor4fx1Mt8m!>^r(F2cQnt%J^fdgs){R}f&3XK%p!z_)&a414hzT&%G51O6&FCK&v} zb-3E>cdNkmAS^#GZtUHB^y8_*^OpQ1Ri;{(W6V8!xTvP2xVq@l^>atg<{Glp@-(?L zQz6V$@KO~lixhYm$07tyM%QsTW++~=OqnoM%+#?NdKNp0FH99lQp6MyEhIiNI3|q5 zB(nH)92Msu5x5_Pq>1D$Hy=Hn-xyuqnsVNk%{!jM^-t^1;RLCTxAv_M_N|{BJD;5S z46?fO(FWgqYukRay!qn($VlpmJc?^GNjZn55d)&=QVoZW=Upkdc=0u|e1b-uWwvPAQA50 z8b*$dq{c>3Vz3M>_|CSzTPuzizoN?-Xkd_Gn{PKZ->+}J`$vF5AzOR3w)S#;{q_0= zkh2%7YrA#_jBV9fNVIJ&!#I2ESJqwoUM?;`2?GG@yLmhFP*x%bA4?0zb0SH6EKQh5 z6XEFM1iC1mBT0~HsS+hgrX*=}RGpTlR#8~2n1BFxKlFZTeDIZg^ZlC3LyZOF&4n|q zMbj3nCo>HtGxf!@4W-i!CDTo1*5(^d1FN2Du9|MGhLbfq8)iBh zrrR5*VHR|ETnSYNs2oC^iJ4ZDDKwM_Q3jJ7upwKR;jH;+1}%L9hNVe^1tUF{=X zZO^*k>?k;HqorZ6rGBur0gjfhtNNj)@@{4M?aH!&8%3ShFTS-M(ixyB-a6 z-R2sr5B zK+>(pPOHF+Sn?r94^3aj@c#0901i4xINvNv28lJ1v9&{ zx#ZYc09v-NVV&QYhEt|&BQs0wcW$fFv@AJUWMuG7bO12AGLfkwGvp+?ERiNB(PV&O zbf%I;kkf~1x+^cqn{iYz-rkdnkS>gWb7 z)1;&6l~j$Gt`%}Ea!ICMnQ1iSrlp=ZtUr{g&q>cXnV$iBH;!hWJehOmOwPHphb~;m zzkDg@{JHG2r?O5TH|6K@jDTATX_}62R&b0mrd~qVh`{g3G$?2~8CfHtY9s^!BZim| z91-dj;Dd<@^^S`?5bS?2$nQ7G6qzAYZO%~hWy~l-q<>@p5CY#w{}60gNJ2PBV0@@w zbPzJg7ZVhel9}7pd#j(J@59JxFkhj;V#0q*7I>+2I48H!B^i^mo|rc>SVTuJ(5I9`9Tzx-%>{$|#(IXbsI zI5G2Z)cSaQVaU2NHtzuUFUy-R9eeLx(-EOyecfrrHLD&z{aSm?oBHNyQlwo>{;$aAz14dp5r{vb;0Bx;N)|zPR?%HUDsB z>y2&ey?y%wK-b#t&mg}7%)b5Jw)w`k@p@(B)xySRd21b8E1}CdY>j|tkSLNgsd;I~ zP93>?`P`+;=Z>8|s!uf%n50l_Oh`NgSa2NHH#!C#8R-)n8$d`5r?apE2~jGiD3n0L zSUNq&0K+pHBq?T9cCtD*MV)2l8>CD*mnG+Ml?4u*I#UGyn5j<4o32*u_Y(cf2h9nwEpsB)3vGQ;)%ABiH?%#&SGnKk+ZF%c%i3w(Frryuh@Bg zw)NU3!qo&Ok1(FrFf>fc)q0+F8pdKU1%wTRm7GXAW*yj!0L*v z^`%pFr4#iaWmC<-z-njP>u1{GfT-E7#+go#W@}fgwWoc$rvugr;X*FhUg)CNcsJY< zWxN}@*H~xMSZBjo2+9b^U64ApqE=rc37)^;C5y4ow|}c4W)M)%I-A*1FL@6*)Z7C z__(+EY5!OKash1gb}yjU$nDiuCjM4)jO+`*`{x5P>LU z5C;6@2yZVhj2FTi;UDZzBoPI`!q|K^i_2lN1$>Uhs4Fchd@}fOc6N4dVQF^BZiVH< zRVTpKW>(f_Z0pvQb!c77n={Lsw)MUB?U%Oo=W|P&Gtj^s0APb-Q|F2cc{)B{N9XIQ zEDf2dp)ge>mXgR+Br@a#05H0o!cx!~3I;_=C5wqb%ZM@(Sw*Akc|;W_R>X+nv9Qph zWJHCAqSAufQ>NBHQ|jqzEmNx@%Vl)6if1uu59cTkWigFfx>m)|tGO1VBrA=RWMrB3 zEQ^6=Q1epE((H6WS`s(O#4#Jek!>+hb#k(b&(sNpDJof(L71%NnzX_cGx!Y^X(_2k z^HPr;N;;gMdh}@eiR0-fPUM|E3jmgO=2Y706Io|Zr<^#U&Q6zQ7*)9zNxF_}R0S{3AmS`XJqaP=)%tg#~y-1pmgEt2Lynlm;SAGolG-dT}zRRZ7s-npI1XONAzUpC;NUYCBkx)A@c{`!wD z19O25d;^@m@#@3u#?JS{qZKVJCof+~J(RCY%haW2>r-=5a*v+4P*_&m(sA?qyN}17 zO)pz*J4;)y?7Q#a_F&IHz=dBgesay`Sl)Z_!^r61q;(O%VEdhI_a``C127L3Cn3P# zb2VsS5VWw@5opviT)q@k6b(?vIw{) z0mmQ|Cn=O^N@HN;P?WNW6a&i4Nq}Mv!f7!JK^Xkq|Yr9|8_kLd6``NMg z1qLu|_oIFH1Hjud__pm=568xm4rh|Y45k`f@e)I(Dd$+u(G&S+&z%H0ax!0GRL4<> z;W%tK9uE?V!v`i{{jk_TLSh7s8N=cw2xTO>f&`a;Dd`#wTc_a~bRvsMlbNc@ut<_M z%1nbiStZa2xJn*J!3WTi7&JV!j4c;)RbqiwCN?PK7PT^2D>JL8VrDc0=Nfhu!NNx| z@lgyMmQ7&FnCXXe?hOvvH(mqr+T8sJ;$oPKXD%Ca%Guj>kWH9hRO~f?*%~x3kXNu@ z5w_aHDxz(5X9Z^4zPja}I)-Pf+RUcS(B<$TMP3jkd;XHK+S zJl9uTc=JXP(64)q<=;0~+-t77-v&DmA9glB?ri}8b1hpQ8t53l)j4{*2LNp3R?qOw z-ed;96#{eXx7p^E<>rcfSLe2oH1=z}Nv~n43qSmzxh@ zn5P?{7w|A251@1hecbk;54d@Ip#lSvvk%SMxAs2%areVNzxe52UVi%TFF*h5%b)+p z^Pm4?@Av<)^XcFBKKx5l*B!sGP%m!;CIEv5KQIIs$_oI@2Z{1V0F0siFibA*^w|q} z`NyCq2p7R#!dx^wH$xn00Q!y6Bi)-moqKdO2q8j@gwp>yiUxc5xkK z#j#`G*j==5TbI^n7S^nDtK+jPJ@@XLvXl85mQc@TtEmhn5HBK2L1f7j=~5g`ilfVj zJT*&`rV*MXR5=yGOigEK=u|bGW#F-mqBt%soIyiS)rmRl{B@6uGP~Zy|lo< zxF!R~pp&E~8;<5nGEx{iH2@e@rC?~(;*?}ziiMwK0x)ChHM}H)Al1auYuS1oKiLdO z#!fbpR1%_$LsRhp!zdadQ6(lQr9_Q_VbI9Z(=A62r=B>Hd^F#3IM0%QDEaVVz_9c) zr;JDPb@@5w*E#@;1=TV9v?E zL5x9#_(tLrPF=WMTvmDb#0iC79gd6j48R=lMtTMM`Nc#A;o~BRi4?x5uKV`V&byU8 zK#1QtAcwvh7yy`aH_OjoTi%`hhrha)OT7M}wD|MttB;d*NBg}yhcBMhq?&mO36;;m z(dqFtI-bs?a`<$vkii#GxdIATAki9gj-M?kuW9VLb?4E@=!|V)?YVvDo&Cj!<>#*+ zj6ci2biA;xa%f?B+OY>GZtQ-v!9~!Y;iL~pFXy8;S8wDixCb=cqUGgJuF~b|?mPPq zK*-Cbwe5KuxO=iq&o7S6OxAYPYSL9q1)VOT3N?I9l3bS}lbVG*1D9*&0eS(3@hyD5 zNsuU}MpCdLgt#yQHZ&nV1REb3A0LIshQ)^ahx!LagoP&%k`A5fc{n*ZyESHeIlcCJ ze*N|G#%o|#_U(7hymvSO=Ggu0qSu-eU_i@OcRtv+Kdx-NUD$V?&Y{fHXO5pacQPw4i^!w~#6*TBBt+m7qY_Er7(&1Y69}OsYBZCVz?a}e z;3`+bt6Zr862>*?g=Ui~JynyHA~Y#^dZ{Eu4aY|rqymkIuMvxMDv3eIS1TA20b3yy zXr&UPLTr>vjB)@lj!H-n(Bk0iGD0Mv7abSN#3%Ae969Ik=@XAftgEnK_!bb$#js!F zd<;Ic#1*H6E!yYi zS4PICs~cO)nHdtjTxyg`OcJ3^z*94w7$Z`}2>@V1H5nL~OhwihXnHeUXQ0YeWC}eh zJU%c28yX!KjEe~r(_@N`CH+u!{%PICp@z%Djn0+MZE#bS*)D)C*knHq=mn0gS0|e; zO*CAZtiNKdFPLe#K3QKlR(EB*0Z{GwGyqs#Awbt`Ly@)q+Ei`fWbL(y+G~?_MH98x zN2?1*stYITivhjH>q|!KKuU*ei-+q<#~Lffo8can_nPNZ8&Cm8+%BNw3;xEwun(;M)Mb0I>kNfNTvm);?;e0Y{JrbyfH4D(}@)+^Q(; zFE1V_D;X#)?kc?6e&u4zrE^V}&eoqlRd?=G%hd~gCD(4>D85};bhoYq(yOWRht`@0 z?F|o|(+(fQ%3-V1y}E#Yjoj*n`sEBTu&1|r2m5;f!18l4y!}xKKO_Q>DHw$gLit2^ z1AzHO`D4N`fL_qLLIJwGy#i5Qfk@9l@SUj8_{i+zr`iS{J)Ky)QP=MulJM;TIE>@l z10LVFd3@t}U_WL*I?Tf-+QT~v1c?emd-y|!dH8w(dGho}diWvSe1VDqCv!iD@QTFZ z@=u>_9k|_f_rc8{hn`L?f!o~e4}aMG{BICoU;ecB4hQYwr(#f8%+8fq#U8*2M*-X5Ep zpIh3P0mNEf1N54K^je!*TC*;$0f2!o%>lN-CXO}hg2Os*ADvn3ynXlRh10nwv$Kw; zTXN0XRJAfmt}shv27y4s;cGb(qa-OO>FC81rb8)I1%oOl(^V9jhDuY@#mUl))439} zE`~|Riv(o3lqi*gdsC`XO;IUn8Z}F+;h6M-)Fi&eB+p1wWTlH!leuOiQwRR-t0BOW zEt0epQ7XV#5>2P3Xq6nZfoC>y40=J588{}8GpbrbRq_%g44i~URPzZMDPAK_)Tx+e zy)q|De<&~M_z~0LLx#K@bN(THZVos%A3LJS%U0!ONwZRQhx1Iw4;hbTt8y&DWHn1K zVd$kagOqBJvMmahSw=TX7)B{mCrc1;yh4NBf`dH5f;>X~JtG1xT{u4#VO|EWX;5cD|sj{pR8O0(QBQ;%{lV{PTC`Ed{z$}zyC?x4pu8~U=QKG2{p?GX44j1P1o-iCP9FL16 zB!tFC`G)wT0(?dxnt z^>?<-cdJ_;!O;#xZS%#%!eYm*f#TZg3q{wDo;#6|o0grQdFt%(vjs?>C~mZddvjYfglEKNz#=VYie%mRZ< zpqGkFanWRcF zNDOj;h6hxPA)}B52}E8jM@A4TNm4agV`P|;xkd|DYh(*#3ofp30zSl~ zpm&y>d{lGsan1S1b?2WoUIYxAYy%Qj3^LtueX9N1R9oS6OTl>E#nGDcW3?A2>#j`H z7L3+h9;pHETpR~4>k3=|o2kD(SyM1oS2$639iVHXu4t_0+Hh6DP!(X<_0ihmk=ml6 z>T5$a*N5v$p4AjTsVNz*D<5sF7-_fxGTKlvTwgiVP&3q2|Evj^R+DS@WDs!vv)%DeCf5Gl52oc-Pf+P zUOLx&;Y`DYQ#EIf*PT1re(iEk*|l3YitkpHd|z97zX|A9)%Pt`_uJuaDPP+T``VuN zw*qtlfVtW?0J_HR^p4%_8@&VctM^e~_oG__r;Z%*@k1ee5r9vKU^FTijS0nI!hL)r z{e2?*0J?ygp+kHjzyiQsGUOR1%sT=XEjO9Y7T&l&G=KZi427-w?S9xv|DBuHckZ6w zcpUr|xz8)e-8%~58-wzX^$Caz0Kdw^AK?xF=8pscCtsZm?dgd@dZ5t=Kc66QSBH&{ zA`oTzq~_k+KaN^Q7uPpF{&DB$KW%;b)6nA1;Y-)GX_<79)GsQ^-PhOC&l~NJ0RTq( zA(6fagg?e7BsAyvnejP?ee=C_VfWGFnU>bR(`U|G%odqU#uEsn3UyIw<+CxXb-_Lh zpar?*u&%7n*fwXDH|9V9!xq+Om*7(|fHCXB>iFF9;MnxtM}v1B{4j9uPDfvFQ)g>+ zQ(a|!Ratde!HvSp<%NY6CFM0$Rn7G$3eNHLV%S5kp|Xv1o`I8jGNZh);#^UIK(CAE zvYeBjgaomeqL9Z_tlq`#!X_m2!iUc7S5gG0t8tfS!>=hA=hz#@s0OQD*cqSn@E*uja;1d+!8ybuW z3q%BXqx{i9u_1t7k;GWv$N)@;uhOV5EG|2J{#^tO2tFc`8W+bS z#4SrvoxcZ}O!LpNKzjqcA;j~}ZkYC5EPxD*P z?>u{Yz5d3TlJmz4PaL^)IP-X>!mJ{384-;BBK>?kebJu2C}e;)GQh{f z4|4zwu3w}Z+Sex}B8Eg(rzKmCrLy%jwt*-~6`m@)d~^J8VOtn@{C)R>TTiAZW>!|`SAg}~r)C$QjE>%ae7~x; zBt27i=1guu!KtJ)Jy*u!Y50m%6;CfD3MkPOY&Zc6Tqg{R563}(h2s-qDTF9eLQrgY zU{t7Yct}8WeCDYu15f7%=XQseUrah)%>V=2e&^Wx$+7F~JbeBM&X(N$yaHrv^CKYH zI$+q&7YMKo0I&~^?T?P__x7zf_N^C7>pN46_Q8>f?wdDDD$6buTsU$n-;`#G#>ae% zaQm&7hr6F&C>9r;NR1`aqsg=w8k-=J5hV(;OhZ>27+O8YU=mo81s1a;HCd67qRC8F zryEr%TDe6fF)KM*F;gjGtK?jz9H5J`#K%7_1jht-IZU}uQBqzxySTBj{nO^oXL$StV6M1Ih!9{q?>Bco{0gvtVAxj) z0|0{zTYmvC2Ky4%c2@wp99s^DYdpus(z0WI(f)X7ytt}1^XRdZ{KKZ~Y;9Vq+^pwm z1Pm#SEKDTw6G;4cj*KKyQDr*tcfit_xCS#{t>r)mo(s;`b!ULLK!GFn|Q0&@-E>RDCc(@Jo>K3r4u z6r5KV4%HP6)n0#EeeFrrb&#Rj(&4(Yk-D?z<= zebwWJs=*Z5*XOGvMIa+uASi{8=?S&V5O0M>oU%y@H(y!9{%@y}sYwow#KImw8*!dM;&-&Yk z24L?70N5|?)i-*(fAmiO$esSDxB7vBJ-L1B%GuLF!TtyzFQ7yo08s((!4f*e8xsZ@ z20wE`{d~d$y~6;3`~k@@p}y!)AHRrz2z)$SuBvLk`DkJ_`*^{Agzs+;0(yCU>+bc9 zhx<1GXZ{{OQJ8=@zrgr_;DnHXm;iu7=kJdOAcO|y<9X2A;{e+IAllsngLK0n4`969 zeNYF{$e_qbeNvJ_XE=W0>iF{d%FEB=tFP~lto%6bm~=e9J7g`WZk3xeL*s}@zd$b^ zZ!Zke)7uLXfOhji(M7`6{#y@6tW%5ID;pmc?5`gWFO^m_Nfdg&pm0x=Ph?EO(GzFy z4-Nx3!S2sdVE_Pt07*naR661E#>~pb-0Jq+%I4hi=ECyU{Nnc9^7f2vduDla7A^~O zOwQRSW|t;r7RDy$hQ_T=$0i>Sk3M?(?7`#5cOU+E`{9GzKRoEab+`A{t@_TEyfcTz z1`%6L=Nnm?blKIiOD)~4wQVhV=gyI(BC=9UQb-b|B8Eo6)GA1FAqYhyV;a@^BYDcK z3?O7&lL6pLn3T*m7@2x4%cv2hB?*$vER&w5SJMq@wo#iX2LDuOT)m2=R>gDJbcIA< zkkb`xmR`h9(^AaZM1zKrYT;+4GA$;S$t1~4SLNks^K*gWNz+sHhx2qt4#Dzcs*#_S z%t|pZlZ~RR6j@%XG}oj#nkLIlVkK!P1_jNi02m``g%rJ*VU)%47+#U#9^s(?W5}p* zL}Un%u-}9wgam?X#UG9F!vu!pBn#B*?1E&9ev1r-fkFp zogG9W51^3l-WV@`pf7&t5I=CagKHfT=;!I{iwO$BgoGf2{V~x2QLJc^ES{+&2rTT> zQyJBLy+2GYjo4mTH{LDnez3oAE)j$E!_VN42FkSwn?YP{`k!F^@Wsoag^8>6S5!H2 zNvia8(Z#{>@i|yXT$^8Vz(pPl%MTuYfAP}MlF|#6Ro62zlUY1EjZbIEc_cnPj*5#W z0#p#e&0PczABIhcz~Lf^2?@-^SfFJI;Jz<1C@v~Ik(7P5u;=mY!?~Ry=bmJ+&bj&4 z*)$L71v@8Sez8CQvJ9(*A6B$lJr80R-UReWSNcG7Ja%=Y5^Cngr=0x6>^GDNMN&xTrNu{1u)~Q6#@-l zn2fCuQH7iYIwd3_78w?Jzz=o67vbiI1j?SkzzftOjY(Hi-@Lf83()0c*e3w3e<&Wp z^Zz4Z{|{ep!5zo3t$*jnC$YuSFi-a|Gl`i;%*@OzSqvr_WLp-qY)i6ap)HGUH7?NR+IJOfcJMk})As90h z=`hypxZFFq*xNtdKRDgnN4CL`PAl67>zjMer{^jgo1#+FB9hZX5>ou5%@N9395Dp*=&+E%l5ij%oy{VaXGSgd zwaq*zf7+Ha)|~siwQ%}Y>C4+Cvv*48@07p1T{?TGY_7e0_IBCqjiTx1{ORUGK(FbR z;+f{6nI^!n;@Osxx#p62gJv6xUo;c~O*fakXepg)E`8BbHq})2qNQTG6`|L3Ta|%c zl@m>6lg$;A&6O_z!&hVp^t>fzS9q1O80+f7L1@cvCCc=(N8_mNXyfL?taa|2KQ?Y#y8 zy?SQ{`hcbfdMA6jr~3PAN{cB}f+fZhu}1_ez!fCBj{*RO!83^jCIttR@eCS)K{p7j z^C&0^9*vNUK;`jcG7AQ$S8qHXC9@^pT{FJ|=w)X0y}9`n%WD=yYb+Dci%b_mG$Bmk z(THp^0!1bP1K|K80me-6R@X3QKcY+w(M7a54u_$VF=Q+XXGNmm$yB0HCQHpPdN}-i zXm;(z+WGp$ADi#~SikuF_3@|Cx!uzG`}Q7zc$jN}C0OFH76cUVb~a=zOCpxq+J_}1 zca2QE*?aqH>+R@^^^)pa3Z0voHSVgZC6+*mic1+DpIKf<0JZ|qwSBz4eZ01Pw7z}1 zx_Pp?g&Yn8TH8Kb-aL7;akRX)_v+2o;_K~&<;~ewZ(c4gzg&1dJ-0ada{l@B?9++q z@v-M4qhoy|0}ndy7grYt#Rs?s+lMFV+aI)bcRzX5*TB)a_+TX)5JkUQi-XSzd?(3xq^!JF3c8iM91_Y?|zM3Fk`-mW2h~73d5R7WQ zm&(VLp%Ty}Ob0(Vmq5MD*@dsuvJ@hbz05Aa*)hULruPy0`l)+H+3HYmU`G}taAQ10)k4Dr@P`K!bA+K6CXM6hjmplyV&XL7htT8v9KP zAknB)WV3`x=Ftdjs?y26sH*br!$-Bv4RPsl_TIL99baHClDf(zZc2%#QsJkOd1_;G za%NWc{yGF^C|3xWgtV#r*WZ>4k>MpmGUQEww6Dh>kkY+|7~^jnXFvB%&fMzmY`EW+ zQIVyO^_SR5un^u9Yx$$Kg^3Ny9E&wYV@%L!Q!LgDhqWRTFmw`$O$ApfmQF&ENq9P) z#DXaTh$5xKIy&E(DRt-A1*(El1Iur<^h_^Jt)IO*L;$w)79i>~l8b1l?y-j)9W&(R zUjke9W%up-7jG5|8uM*~SK9ejE z2c_gbo>+diax%U3Zej1|<-?!X0l+T4>>&m9Kkpk-{}I1+B*1o z`|tzez)k?ke%U?yfE3C){%PmL1o%*-Id? z`3#8)5-XVst;*9+?c=BL^497Dwf=hBfIx@fV8;+}wfVY)`uN8NMQ6lCrNsb-xrh1c z{D56^6*)LCWpW$?M$y1G&t-^3Y^j*508o^`Vm^t(z(6!>3fYv1zeYd#nDH|9^a7EQGky=X0*YAFEY%*~SNo5eFX0Jn->G!;zL=e=ky z1e$69JS%z$+*nibOk?2;Kv#XiL|y(wef~sa!E^9A&4te!3nrRMCR@rTTgm~nCYsA$ zw3a__EPh&FFbUoVYAk!vTshSQR6W^L{j9O-X>;APmZqo8jZa$u$y%PaG>e>Iyncw!m0`N53f^@nP0Pj12{gg$M*D zjmUzDOqjrci3|pr#iVkWBqof9Xn2SMV1{SHLY+fV{molXhNU`}?@Y|ESXf;#2LQA9 z!P?Y@iXn0+R6Y$7G3i1Ei9;cC!6iy3^Qa^)(s3TZ%!+Jff;GE_F$d1c41=}8lL!z@ zg=s{r4UK?;sdx&72n+ZrMWustt5X|iFSp;%?)?06=flhGU#3>y4^HpY-RkuVPvOdK z@h}@jp;!{IrdTUe43Lc(-bU(ZTU6h4zwdc_=kt=FMZ)3q(&#!laCpRVkH$=S;0$;!s@%KG8*>K?$@^7`(=>gN3N+QRbc z!t0f}C1A)FU%Z@uG5vC4a_Z^Cv!~C-`iFYk?zLtXrDhkUcJ)3S9T^-P90C@tuI+|j zbcE8=S?TSl@pYEF*~&cZ6+W(NKX;Fquz<8gr-%?$fS)?R&m}y}H8Kh~FioId7ZTtQ z8KTttDE++EdT+Upo7hE55kpuuj-_DheB7M%0b-pkh0Eq^BsyP5hY&AKh+Y{IWE&l+ z4i8oY>eWGkYT(|2{QwA!A0_ejkb1jGya?RY-#KPRv z+S0_@@~XAvj}{i!%)$F$kzxVXM;eL?KlTxK_2;oMIsV@2@I z*<1w=j07f(gJ@J9gQes$b!>($#BqfA&J2y8)HB8{r6TXvz{u$9z1iK%Hzz+E81@mV zYHv9EjI@wOvg(c7V4v3xkFU2k*#~NM!D5F9sZ+FCqnCMvySH_;FRpDaZ|yCtt_@5M zHs7xaOYqIiPp)sOii?kwE5s0!PGit%Oge+hgu(4chiNd2&gCJ=>wFeZEtEM+Ia)SZ zOv7_2I1Y`-7X&8dbd0|O0Gru*H^2MytAn3bkAB)d`-Cv;0ywbWkr3kLZwHrO58wWA zcmc+*hZlbyoc+Fa_{;Vo@Lr$ykv#if_fCJ=K@u0=t?eH^otv(`)$A7Fg@$m~aOU84 zHNjYe8|*5uusAFlW%@KpN&dIf0x{2c-VJt9NBql5fng2Gdy z;{+?0~H>OI3f}v|z^i{OYH4OOGV8Jrh91nyw!=p^`XmcXLfKobRJ4qKRX}?cePR@fMEc+Kt{hDg#vC9G!??o8ALjZ zMrYHA3?hL}B2%%hwxYqC6>9@GSG${D-m4mE%o(i77^}}6ug~qTNa`w%8>z{7)|fxh zTsYZMGy&+C|PVZ5>7X=CH_rsikO4dYFZ$ zNAI+b-n%i@apUQuThBZ1OmqT(bxij>dfC@G)7Sa3uWN3gcYd&Uaj183uy0{#U~zbG zVR&$EsLu$n>4E--?YER_sWF6ThPSjNplnE344p_|Ktv{k#A1_}YyyKxVzDS37G1yr zLXLlM7!(dn<}isYmdMsVAUZ*+bG&MU2IxZUmpQn?Ex=-z%%#9$CM;&r1i*xWhXV|Y z7!Z({35gkWAw*z-=NfpJpiva6SnD8E+ml#4CWkL(vo#`~l+Pj(u+~_d!pSwYyr$yL ziS*3QFUs&nSZltdN-MI~C3v6f^TOA4AM z6nF$h1;-TxCluKGM3Ff%D?ANPhKUpyX7S^bGe#%p-fSY=+5moQ>jc;@0I;>4^VJ=s z`8XnA$Tp7OtRKBuKUhH$J@%J3c3-V-Exp-z^=5tP&D#9ym4(ISx%pSKa|<)GvoEHn zCtplHfAPF;r1x>p!{M?1XU|8+$Hzv;o;~jAPsz{Kdb_H8oz&hADo;nPudCM2-66=w zD>lqKF48q3%q}=kqxW|R3384Iw+jw204xxAFNMFi)YDDwd>plbK6c?jn$SR1aG*L6C_o*oSBLn?{M}`GPr2S(?CqxT z@lg4>YXiNUqXO;212my};J37)df>+FVne{F2@iCRi*ikhbxw@dMI#u~h5E|8Tr8;6 z?<_5^SX%#JjrswF{sD#h&YEof5Az?cT)TSZn&}l&bK`aJ4-@l$Ts6I7a_x%cRe%jh z$PG`3@96G+)bpgWq0&FfSMDMeIZ4FMGO4>*!}vap^nC<7pprryZV!g3UxlNR561DR#IXX zBw`aebQ?O+f=sX=;4KLRYZ4hvp@B6Tg~O)tI7}qqDds9gOqqbfW03_g&tAlL5%FBO zLN~6!l`Z!cIRxuMG9rs_wmg2ZIJtSUc=&$p^q1|6Pdk^NkY;0sj?epVKLH13sP6G) z^W@@o_kFJzcl$7feWb)WR_+#~whvH*CWdy649>5vKAW2=tj+Qa*V=lEW75LQub0Nf zM~S5ZGKEN?l4&#wolYeXaA*t$I9fWFMdLGJ2}__8@a=dsIStPzqM0~r2v3(u;|r@E zKYKm)`fzIfVs7W>*M}e0PJZ4x`?!l_ZTx=l_Uqo;ue)!5-@QbZ1qbi`Jbd@Z(d9o6 zFaO*-`?7cX<>36cgR{>DeF6Y`vvc^Mzbh^$LFO#SvI%BntE&XdAF);@7_>PKYl20a z5b!7llO|CxT4{27DEmU;@DmkH3ZmAB(+)#r%M_HYH%pDFjO@*@{ZDqS37AbQ>5# zGhr+nrilfRNa*0{(>FS`hir6xLgI(#znxuvJ%9Hbg0A!5PcMMJ7@=h#8It8;oE)C~ z3ea_A7>!57{>th2E4}{s{l0-=2*CD$4?Ex8IayoZ?H_rXUr`;Ol^2_l9hI6LmXH|~ zlcJA^^9&Ak_V;!4adz={@(Ofz^>b7^s`w%n43_(J;4Q7l6e|+hhD^heAtI56C(?){ z5=OzN->=Ty7;0bdYgv9$_wr8Z(DjVoiln}()Q1Jptx5h5aw7XHQip3YpEl$H0d9@f zW)D|qAgak6xt=u&RFnPmI^b6JPSWWgwP4;kQ&PY|>`1RuP+M?0x z1wa5}V|8H6AHQBWe!Xb4x^S$fWURV$?0U&?P4Qq&$v{o{U~TnCecdQfZSAxA`tkbO zF+i`{^6sjlzS^?krs{$E^8VV2!TOrP`r7{5>Ykd4$K@q}UJpx)9u^ngE6i=nN^Q+Z zYEF-D%TByslG$~=xVyfjr?I@Rsj|PhX0WMtsHI`#M$^cx=F#@H(fc>YAKrTQ_}0Xe zJ5$~FUl;&3)B9wmw|jn|Z?>;@whuV4{>8!mg~5Tvp~1!Bp@rdrnZX_eV1ola4 zqwp1I8jC<<(imKbCFBZ~>3QXovnxpVyX`XsU0bK?+vjV*cx|5=0cQLWg0aox)y<<7 z1I7?5wz`9KW?9*My|VFoc@1dk)yl%s^4#KU0JFLI#p&6XQ`3`^lg}n5pFMv*@$}im z(AcvZ_aEqEBIF+SYHuf7qzt^P#@oq0z}+#(N2m98M3N8n8h?LnfWLiMh%N-FETsw3 zYXbe${yuU~H@=fCNy;;&Vok|5Caf8Rlt(k+GZD*bPo~91Ah6>L z97G~#iNaf{3$k^IcJxYe4bBL=)8F&@_+sPjCjhYR_rLA_1u%o<0-!=t5Wm%tGS-*+ z@UO!U$SJX-5C47m^M4=w^mXU**QZPC4ISO#>FE+j4MhZFSVR&Zf{`po9$Co1a>yu{ zi1>Ue5jZVyant!cNWf=Ggj}VVqY`tqa-LQJEBIV{k=R2f_g2V#r4lcJ*q!eXtPM;F zOsOoqF*rK;%2pLw4-gPR2w9+ z3z9ensv}Y(Z$IuBpMLRp;BjoG-qwes_TWXPh80y6#l%O+l@bUzE(XMaVG4zW!q`}$ zEO8V(jmuz2*nnOl2N6rdB8e$DE)mBe;kXoaDHQQ?R0kg{i}lyt0%u~ zo_*TB_-*Inx4pN>2AHAECNQaA4c%V8A{e{;6R_>zTU_xA0NCNhZ~NyT-|XyXmgjSI zJXlW0aIj`nYcn$X8WC#(91;O*O z65$#iV@@Vm()a{C-OLCK9|Dfdj*A$O)Ub;cy#{d^x`+5vr8c0 zz`)C|r{}+)oEx(_{@xWsJQ*@SMbLF}_UXv58TR*nm_f*v7yy`oVaS%)&i=*z!Q0)P z)0NfTp26|_%DUwIl7!s+nDo4er0lS`jDW}_?~q9MpkQY|FIT<0cZjEpud_rgWpd~& zu=GcQpq6x+C7EJLp`plhES>_MX;=cmM#`l%R-`yN&UG1t{epEekyJ+ZoT1R2@ z-Q36bQLG|SEM1ts>vFtOhb@XnLYwkoiS3CIZ}}^T%JBykugx7F;JE@ zP?|nik~&zPF;hBwRe2g>sYD+-4yiUunS z2PzBtDhqn6ihHX{dn?QPt15}PUOS>wHo>Ua~)s_Q{b(Vv9DNuK1 zS!YGb{o?!wMFn^Bvu|XjwPq&Y%t^gll=+}E`*C%CXH8*uZE0^~MPEyGe@pdXYwb{5 z!!Saxma+D>F~imu;=P{Sp6qHz^5(lA&GbE)>+hW(=%4Q&n0nF$G}qs^fV}KSOxVcK z+|aJGMh)`@@YH)pcjF`AhVfZ z4Q&J%jmM>OfjCqSi^^frkt6vm@F=*3wzy(>?TW>ZrZ{sFhs2UGIC3UO&fqFoJO!Jl z-~xaFba51bUL2~BMF2LA%ahr=Mx}r?O|i(qhoy8BD(vM7oknY~QrQVaN}gE7S7?b` zjuj2!sqI|!VP0YJo*@bDL2>pzAsC4Jy#k;7K7isGqD%80Xm_j_nWr>s;M=uZm2%WQ^y?a38&4)`X`$m9)abxEKnHyTU z0)TPwf>l-_U*3VmjXN$}0ORrX6y;@yZ zd_DVeerD$7%=FCk^vuM>^!T%>fw8BB)m2CdQ+Ior8Z9@W)S{ng=N*_wV+LDH`fpKUU4*-UNNOZn{uT-;S zV!o|R;c2h&aZ-3Ysr`}dH1OBK2Wox1l}JNDSA~ze(%(~|_f+XUbwNH#KUYnFhc?LD zE<*1R9c&*NWE&N19}}*N2+@WItAY`R>7qjIT9-cOI`VZDtV7>Aq z%F>F2{Z8YmB|xNWmKIknEUwvDnd8tFIJ_kWZ)t4ON5J8oQrq7b694?qp`7-llPl%zib)W8Sfz>M56;UX2~1V zG>k`>!TW~Iun+$_`sH6oAO7Fr&;Re>{U7TmKMl{VT)*4t8ym~jNbnr8HH^iu$z&0O zsp3O&HdV?1R3q^q0*g*$LEwU>@p(Xyh|iP>VY!H{QHmVxgbr$fqfF+mkbBCdUQ&gx zTYUS>U?C0dGWx0P0cU9n49Y#>CG;UbMzOe+<9Tiq4||X5phu>xsZ$mnn+{{ z1xq4WV608i=4dLGE`T{IzRX1~bQDu0bS#%dwuuHlij9bMzgdoQ;y zUmbo}IsOId5PkY-`}_;yy#T-rC6agF{GW)0CTC}e9o9R)!s z2)3Yr{{-HWO0t1ySSAa{W)nCZ5???Tipc^YMIfN?c~l-hI67f^as3eR>Ebis*6G{- zEr2eg45z{VI_V5X0JBp>MtT`1|C!T)9B>8zJARL3CmvoN?7!RHK6|~q`M7tupsFFY zs4_9XI6kW|CN)1YIVU(a%`ZIGD>%%_*T>n{-36>k9o0Mum&KtYUyO(<6yUiW6wI)q z!!}eHO`uq#ahBlz;4|_0@%q<&H#P@a*1Kz$A670sD4V}mHrAZ?xFr5odhm^u!1j#L zyV>C#xzSIG5(cW$2P)J1%Tfm`(gw;?2Fue10lLc52Y@Ql`^z(WOEP*&()vqN`%1x> zIZ&G2UzXEbn$=U1(OU|?8zTm3dut#odiazD9pj%|P4rp=yvo`2Cy4q8k%W z+6~8+?*sca+uyx7)DO%SVAynb&&%Fk;~Z#VcxZlP@a0e+0NC_E@6*1Xr$ha5v5{z; zjTO<_oM4F}V}a+z(qRk@2Hc|Y_;fxWm@oh^B9lSpFc@OK!GR%XD9LOV9;Oo*R004B+zg~8jeuI7J`YECs6TO5;hV=6fwa(1&K zQLk87TM{TLNB5}YoaEf{;DihZeVBuPSa3pST5(-cL0wpCNnk>Ob5H_L?aGwe$?QCx ze8OVWi%aV6rsUN)cm_u$W%rG|SlKw-*g4(Wz1ZA7-#~hipRey-0EVsao^S44Y<{B} z^8SXQVrI)h{FK6aoyqJ4F zIX(JpvZJRrDm_W*;V5^vmAYxQ{?3kp?n+Nv05heRv(DekHo(^|D8M;9*eN{79(bSt zf7ggG=ine%Cb6UvEU1861QdivGl>`$3CyuD8T=yXJm7L90$Y{N&)v??4Lo~!c3J__ zy4{YcP#`7iINP6~f_b%2L`sGnV!9^g$A3?@PL z5h3c(Kt+gN6&~o45bc%}uqYSk4TE-YXuZBjR~#{SE){K>l*3mo9OPB z;1-k|*7a;+?eqgu2=VeOvJHl0(*F%GV@KxyafJEM7)V6)-@t?cbN1%&vVZ1Hb$h2* zbTm^b1O^4eAz(N}JdX+~*i03drRFmge42zq;xY&?m~5iJeqhUkA0Ht=lgar^S4 zA?*;U`FmgpAs$@&;!H7|i;{tChvu9rf

HbW;&cV$kh>flx(52f9sZVJUS8}glU~N1*CxOimYy)W}08u}W!{NZa z=RGW8*s_@5DdP0G+e*Ze430y*9@AogU9+tKzuP3x<0jQru(K_d$2e7vO z7~KZCHc+SlO0N-B@@C+5K5Y<{RZdex*Qn5df=m?6^&O-bfh`yprcYVOXd`!UZfZoQ zK+7>bXf!eL3gUFmcwgCCnceb?vyIKbg0O8yPi_&m7Ut#<&Vd0Mh<#fGcTP|}cVG%B zuH%eNY~GBT_8_hUi7cLr=sq@Yb`0u^xB7eL^M&*BPX}W86uHp(Tv=hkK6ua{J2O_t zAU1;np=HSl-O;qVFUV^a?*JSQo%r01O&kB?8d@LZ{RN4B0&~8km9?X)W#( z!4jcd0RWxB8w|ym$(2?f9v(r%ccJP1-a<})+ zzb`GT=tDZHb@8P}B-CH?QxMO6y02Gh$8V4xq$YW!2eI*41V>m~6GdKl)S5um9j&Ab zkh|I28G_h;|BBB<@iM#BbzL^`{V_(R1Yg|u8}11m(6z|_y0FXXWaF~WoMK-by}%a$ zH`>&TKpL$Ez1xvn%3DR3H)-koVO{~{#lYAYS}j2~M@pKIAUHHxwmDkCVh?9wpP)Uf z2mxt}x)W=5uskN)CkHt@JF_N2jQ$JeTT&Lhrp0Ls6KDqFE$$frDFwKR_p?_ovEAJO z_hMzekY1?)3xF-ta&5*FR+ZMWfYy!NPZx68Udv_qVEOln%*#1W>R?podV8!`Ga1f@ zZQiZMio?u#fSri(7J=iy{8o*LJGbGV$&dfZ@7Z1a;rHd2me2eeU;KF>HWxzg-?0*s zFX|r~**|Wr{p{WSGr7LjVy_EuUS~w=5dE0kTVU0){w;HhlUCp~ZcM(4fK(QUToy0NUPo02aWJLVsJz}C zDzUASUc`8|+rgunBfqmCJzhW`7*ZdX+&OY#$;o8|g3?zc!RwD}*D?YCio-ap7gs3( zib*H1bdO#0!LC=oLl?b8U(r;;3$kj=2feP{{!t#5aZ$r>_woRXWFSdk%vCS;Yyp-F zxJ1JWGnftW2rM2huA%mv)#hB*4y4+c!RQ)kA5m7XPOVNSSQh$a>O;mJ%Bwohn8m zJ*(o*(TQn4pDc6ZTux3;Z7!Okh3P?Sf!@{uqeW!h8_=bHn7TqAyU+6qIQMbUaG<4& z_%y$RNiRt77LHV9fyuC1=g|WB$cc!1q|c!#22-~yARN)_96!55T?P2h1+ho z2mWx+`?-RR^_W8N^s_BK&tDRAxmcv7|4a|4JZeh;C;$VziJMmBE+vn>!o5qjB*3VkC2)?Ez@P?I#F+AJFx+emry~cJKGS z2GDBjxk;7*inmCYM-9O1S5!5)+W`?@L>8&wy(o=8zwJZ=DqQMCrzJ8B4 zrJ^x z;ic^A)|3gRvR&+h5(qY8(ps+JCSK?@;I+|<D;dUD-ZY{i8#z)LF~fX&Z0D^}VA#PTtL zPwO1aJU2TQ=C*#{4qBKbV_U=%Y3-481&kZdqXs~zkxsE9F{@IXk_I8p1*_nN+9DjX zD67|<6&RF}QeF5l<%H$jC_mD-awUlMPFPz9FH#WgrR(bV;Bil_&pr6D4&G=0jaR#z z;>E?X@JMAh@vCG(nT;#cCwBI0>Cz`byjz z&*%5ncqBKmCMULJp+ZI4mjz{PIXRFDA_Pf+7vle40!#e0MjAzD(2w8KZCuA7mh+Y) z6b=W{|HNqxz|4xWKphx{4JZUPbvwIpU~$U8j%wKXxB-V4Kyi?7eza{~?g*H?n)1>R z)h7qs!0Kj$Mhf0(POH7GvB5Z?QO5FKouiu(g_zkj$!IRT0V#rltZ2RNVmi1spNZTkG zTsWQ&tGNU4!+9b{@BdN&j9rcu+!hfdEQ(Cl8i6)p)`6(^OJ+W9N$ak;AJ;f1M5m+! zv21dXMSN`yVmrWOc2Tk@lQLKnTG8(Jt|oK0*fMQ5`{vDC`Q(#NfT9#V zs8(ec7BiwiPXU5v-3}84@p~snp3D`m+ekMg4a58G1np&ip8|8xhb{?L-wRYlp1kHM z^n&`%>QsT+&)NJAV-{7{6LqH;4pd!RDb|Mcy2ph@(r{o*$pO^2h9#LGwsab?W3-61 zD0mdi`C9FDVvX8?nuuB<>qz{7y|)nf>i4k7(hf!kFxN%TUFJMWr#NX{jHCyJqHw)p zu1zy!2KP^jx%j44bUpUIW-YoempP~)RuH&SYMz2vb@&6If*KVvTF|B^$DaW`$C7nr zE7;Lm5S5Q2S%J!m&H3t_RTB<)lm#Woei=(`H>)7_T3*Xdd9eW7Q^SF?XtM)-Y{fW* z!UypK?W2C7${`#;aK~h@jk~eP0s~{(5dXFO{*Qj*Aoj1?rTw+_M(Z9{%=)8cz=_*0 zf*SADK~k^jY`J027VmW+UUX7cHO(MEyOA&+H$KBc-iXy~=uTm#W1{1f>d^ zobqt85*4PPuzRu2o#I6ABg=h1X`a7EPhAG)u%;1UQEdkIGW&|=V?J=7mf;4vfz*_G zX-XT=1+TzzZgUX*^;-4)nwC)zL3HBxi07r_;}i?kRaB91?-5R}SP8(TI)4}EXVz^^ zd9gu(7)Ce*nabe(=Y1f(#G{lg)h(Lur+n`KH(#}5PnYwy&S{_?yswEv-wmk_)FCdQ zm=r|H5p6)^jNM*{P-}pkipp?256VWtE|q}2_?(bN9sCAXDxu zt`~sA1}pU$xzRh$DIJLL&1dMNuHcb#vx|@gHYwUBaz%R})^BFKUpyazzLMr^0S|PEZ58550Cm9?uqTcz8)M9xTK-3feDr2257h$W1TQ-v5F)P z>vgl5SG|)FH$plfw+@{3?=1wSA}gmqqO^=OierYLO*Qa8yLrn9DQ>L+Sj*hmo<1d* zq|FNZ?df%L=m~VEsEkLph+NIC>!8yBHdfS3X~TZ(I9Dq6G1pf0U=;Ub3%;1tWHslw zvH_50h4rf{f*+*`8zV$<7Dl|yi61hXlVf8AfhxUu*oT;20H@-%-c$m11g}$IAL>h! zvM7yv8`iT0X{yzjXw!|Wt1J6H-OJ9EPM{!mvw<_#Qwbcz9;TgRAoeL}IA+N(NYJCO z#sDoPRHvHDyiJX=VmbyIG2JbL*W_`*dk^!)@^~pX({p*KFP7I&mLXlnegU)gd)_P` zMnPDN4kff83}DU-n;{GUy}D2eH2vRPwTst;^q=bRnbzt7(o=^i7k!sn>m!GKRbt=DFS}2q zd_^CH;sQ+h3K%j4FslwZ*&v{H;K@n)JL`ST;w(}L*a3uHt~{OX)5He}Y@);90!TX}NWqjnfHC(^b9Uno*oe!wnFjQ+=Rv zDySzJQN_aZ6$uJPffF)de<@ywW>ge!mmBFRF4~@6&_x@W1}>d3q2yORN}qE(5Ews3 zTx!aFE&gu+Bn3m|4sr${1# zQV@A{J=wxz0<@XAv*T72y9vGnPJ&5xp<~bB0{^-}sTOV7RL~ZxwKG;{XHeet#zCrr zR?~&nE_f#jYPT+BHG106+YO)iJRM!DIRv{2in?**NK0YHhurjxHm}nEgG5BOw@dHQiRsi@Gr`%-^iX$is}N=t29en+Dw zW6Nu(ao^p!o>ut;2|(Mr^XxYUeg#2JAF*3qQ#n4S#8@0>7{bvNZhZblq!g4nqqGKK zu~wL&VykgV5qUF>X8Vx?fmv>E@6oC2bBn~@-rgYuVULPaPN|!Uu;yLLD1j@F?Y5Ra+d(lRlM{zG{qp~QuAg$(+SaTO#WxF}IwY8nu zyWIj}9~Y3iYo~HI4{|pZ=fPWhR_Pe6vp=`2A{FYY)r-1XmlN~xt#bgE+F>dMBXkS z_VMxA0$R`I(9rctiSH_EKGE<J~CqyW&@ir$9<7O0C#bl^`6P-ojb@Cb##G2vjHb+uNSQYMFVzLNoy9tx?&XCv;MbWGp|E}=5RnNicF%2VV$<5^T=pV zmPqvxJ4PwH*9wH}*(jlr!{?8}2(sWk*h6BcD;>z<=Y&K7Xw3vrrLE2^n6JBzn;qbE zhAH8qsd2AjdBK@0R2YbmQbV99|F+YGjbu8M4svL4BCqY)_ z`|X~8A9&?0m~sx zfbNX!j_JlWLGURrhFMC2V4nnQr_0xp2qXBk=LKp8&Y>R=+!##)jg%!lo;ODlM4x%i zq#pJiuI5!Xm2<~{s-PJKqxuoq_h@Saz_rNeE;F-y%(FM+pd40Yjl;b_plFvmu>|C7 zukMBHXuZ+$xTH%E_qdW0=N?zD1Z^a;D1FhSP-@U|ofXyERX@6Nfx^*A_wvEkr&497 zQ8ViM1&Tf_$G=|=@vuC3C+%>%?86JzSe~o00ASJJnRnLETRSe`cbK70KzpM9G2$*z z2&(c~`(JrH*Vn7$%Rp>68~HZdcpb!AdPM=upy~u0xw_sifb)es z+@H!$Be8o2p}fJ7&Z5UJ^b!GNB>`-JrpH}+Gjn&0l3YoK{@I$gm-n%YM-cJg1f`_^S%0nBLezVWA=d>Dj+b^=tIWb?%&Y z(38k1^e#fb6~+g|^wg+xz(qr%MKTIvU3O#$S_QeflmiK=kvWZmrFF6K`a*Xg<3)O| z^=S$WJ2mu7nfMeL;E=G`N@)Zn(eryfnmr;qTagxIVIHF`4aX{>!8;>OsFV|p>TM0M zjvfiOVzuJwX7D8`V^lli&4`5I)UHo=+>e&UNt|DP)563=bRsqbSP74W@HrS=Vv26F z=LBDuap0WN`;7PIWb$aZtduslt*aYA>5kU z#R+se-I;8$$sx)dU3Bn#6r9wIHgg=XnGwwP(bZnuzi{60n1!IU1mA2GWrs5!jXY}i zSx%gie=u2moIdsEsBy90*xhkjH&p34w3!wztJf_Jd~`l(NAo)Sh#{|;XF+{X0^ZdR zURXJQ62JU2KH=j%8O6;v^-)-^Gjpr~Y)LDi{wCL^+tJi2))(A<`FI>AI`po2eWj_} z>udS!)6b1k@SsgOOg0cn_<|r2PECK?cC3McNHTDLyl^*8SA7eu867 zy!T}fjk6MaSBR98x|*zNd?mkBG;oG19X?rw2mk$@q-9E zD$bMy>i}%h3P2a=K<$hM7;{8mS!gvQb4%x{zdrw4v4qx}sKl0P7xy66`MtSEG_;Y; zxrKr+wwDz;eG+pmlp_jgJe{*z%E296Z8tk()fLT0OjvV1@fUL}&z?>4;?+Su{IJP~ zuO8&9FCOI8`GdSXyOS5&8+o?5k&V2SlXfNJe7!stKt4UlFg;4$FONwJX_m*V0C~Uo z-Fxw73b2H*jcqRfT7LXH|3LrMgIJZ=`sE&+_8@jWff$*hF%d*zox`BT3g!tfuRQKg zOea<&vA4IQ?DmWIqpj5UkRO5U1HUgK8l`xrtdc#uJw@(PP6V(|CUo)iFDGR~_`HwA zhC#;y@>a1M*GU}^p;T{X-W!0;E?_jk0sz+{`bX&w{n_LEtlOAr1$tU(1Fvbban7Gl zxH$|pWw(qksulcary!y5jYYMK zbepb};JE4Rpc;YAmDy!S^nDvMQeGUi8HY&U6ua(?SG{jfBK0)Z1gibUYeu>&Lh$+R zk&}w?Eoi0KA{Np)6a>U<#?WZ!v1S7Ja$)(ywhZ0q8rN3smS30RYqHw>b5kT>b0E^< z&Y58 zTSlj~A)sMtB7U}&HJ=dN*f&>K^41z0d@i4Br>+G=942E+X7M7Ttqo_53qgSPiOJ~aR&QNa(R_oJh`1)aNpM>d#MYP-ErV}%4AQGa7@@R% zjEW?Z_r(Jo4Kt5H8SaZifZ2kvn`hXO;+a2Jh-g@QHGpyyQO16rb1W4?oQ{_b4z`gw zY|WK`1a@FEa)0;v9@N8%%vhw<=heDM?BJi(J&*JOcEQ@C;G00Tq08vhV-G}K6OVM@ zxX}^dkKYW-5-6tFJz5l2_w3r$v{0tfK0m`T%i9=q0s?Md{+vVp%OmpqT7YytoXKA8 zodPfo24pqj+UjI>$n;vLWo(ygeRej>H@<$5Z-3)KzVi#W@~sbUtzcJd0uJv=?*=bU7xEB&`m zEcOTYNR)zB%4V1})Hu3_K6}R*rYoqk%PVB1VJ}HCqWDUG#vyfe^&EBlR1WL@;Q>)% z?`(p2fJZ92d+oxNkiq!zS#cF0IQBeotP=pwhJWHeYLbT$i)!M`gcyef-2 zb#1ZN`G$JOLntIrMIiXfjJ_IOXq##u#z#l%Wde8t_W)%jkAZV_1&SfoSVmqY!D8II zekQbV$^w8f!{FQZ>b~YA@q!nd705IZVDu4A=c?s~J3~Od#y3Cx=_Q@`6X#rdOx{J? zdLY3+0qPyA`8=p)4LTFtW^}Dz35@u#^#>h6eizqUMv#-=N9R_K{v8Egk{nrqO3IA1 zZtL~}d*_^ySc2Klzjz~`eDbM9Vc)%bCwC8%6|3sCOs#=F;Gou`uuUSh%osUBi^-vH zvc3*kM`9hY9)Zc@xq;{Ut+|h!MJ7Vkfmxln8Y+SJ2c(uRT zfw*08NC0A~^5dT!ZJ(sW>rp{CLV`vUY=J5Xm_zA@-xcT8ytO;_UG9&4arO4}R9H|+ zeQ67DZCr-Z=N2iE#g0WOEO(4Loz|IkC>cMJl1fUdxq6t^?b&-?tF2P-I@yl92gVaX zul}%`W_wj_o{RtQJxK zcLRuZ7NbSSbfZxtd{{v2Zhx@=*o|Di8=Y{09z^trBT>Zxj4VDYquf+bo}yGyb2U4?C#fR z*Il!D!b<9^bd~qm*cpvbMFNe+GD^#hBBe^WS^LJrs!;+ndXMZ5vY)&G#DRKrI%Wg8 z&G6)x_iK{_FT6p7cmc3yCue3HRXdjy0XMeQQx-E)g9VWrR{t*Z#iQvi54MG#u!yTU zM2cw{i<7PQ(Kenhs1v!lW+Z!n*B%=M7Ju*>$?L-mW7Nerd!JvnX*EVN0!h*g3v640 z2c++q{tjT9vfsIJ0f=7Zp{$}JA zAh>(pjAUm&SnOrvF_4}>P$T&F0PR_(^u+;j;d%IZRsw0`yWw?3~Gt&K;i3>c3E(| zzH8z!C?f-qV~~zlaq!bQ1*?d22MAgJzyp%fOCVr-%HY8h-E$i01hWGSD^o!fmYoMb zUBK%nAAc%u-@cRUn;YZ99%fJo;C{Hv@01$bqvF}rY7QVC`&0kpYdqoXdd>GgbXxl- z`lcrOGjw1!eiNulW(8{Ja4>q$@r(DOj)9#J2dK7aV5vrCj)kmhc#;dBdrpaHtOKZu zXl4dU4yMsR#eV2Pm@zVw18YTaUY?-I#yxu!c1MA=7CbE=mcZ9Dq@I~ny3uCbKoWBk zcAems5{bU9XtTN2hbVA1&C&=cN!SR2$-eP|ner)S@M;bR_jY!D*mW_gk8yMfUFM$x z*oTKZ*T6-lmh`~HHtXW!CY{r4zrd2l8snOGF0e!}QL&gliWYC_3x^@-1lD}E9b{Cs zfyH|p=X-hm@?L)NyL1IUdq>BEWq}~_3~Jb_xxVIdT}Qo zp56SjKlr_W`G4A+=hXP<2l7kH$^BZtMtuA0aC?bcDl0q0!LYv@3LT1p#$luFW-sK6 z&t|#1+bke#nRI*KX`Q=qFraualCR|yY%y#%XU5@l60gq!1(ka!1n7VOp*;BQ8Oou=Y*EkOl=fq$3D8qNGn zw|DeK<9s6u2`4a8aEPXoA!th(zHWh6Jc~914mf5;`FAYany$@12(8PXP06 z0LUE+xUi^nu72#0MRQ9GgAAZcJ}rSdC_hFW;$t&Jd*OHtjPRBO)|F{hOiFC-Httn~ ztk9_ar94sf+jtCc3nEhP$W(xU678C8Ty000QJKr|6`mv3L5i?|TW8A>z^nz|ZqC-a z9jI;1J5L$KW+>nTc5_8Cq9mL{+yuBb1@pG~wb#P5WS}d*)(8vcIQEfBDO8hS_M%F! z5>q;?VcEnlk7;x9tc^wc{Cj3JE%4$j2Oiufv_SQzKmDovV`Cy`j2R-8ya{Kgye=ssYedGw;`)3b96EY*Rk zKLfz*M$XPIj7M*Fk2YY~I$bFWQhn}E&F{CZfa-i++LjFp*!ZX|&)@IO5P!Ra>Fcdg zIRFvdDfqC?IgKEKcy?gIUadv!kBS2{NVLpRJ@1Q)XVF?W90u_02{GIWp?$=@(fw!B zWN1(gtg^E(;a zjcnSLoXDk|4L9;)yjfmbKcxY6{n@>IYXQ06d37tlxB%a8E@1bYA1r|P!&~|8 z2iFUry^?Rgx?TY9t$gd%0&rhm$u}%o`|iK{KmND>`^O?Zi2V=c!{ti<1L=Ok@#~D% zk!lcDimx!26mtHL88y49=L-Nkk+*M(+})lmH(|3}{KgEmfqTmrGLbdn+dk zh&7ROTbZ+0@2Mv|b)Bl1J}+)0`pUudW{?J$6HpG=;5LZ5Rv58lqrme)i&-mt0&GcV6Oj=7#}^ee`sC$B}{;~k7}tmXSPeLz_aFA{T!Fn45& zP!>!hQv|W@{go9Dxc~8@Cok$UKJ3OjTM2+6@1Ggq%4j7$c!0K85blj0pkaG}Xt+@l zX5ZZDeX?N6BgQ`N3Jkqd(>CY;1h4|&R2k7Tg#zEef^0vbh3{sA&u=X%OBlyWPX)Fo z<2E{$G4ixZZ!+Rm*ySuE8g9qZvt6b8L|CZpVg*0F(sN-8juD*kj_+Im z-u%8xa>+rN;Dey02M6wf^!I}Im*;G&(?L~tFVZgM-4CS?kyT`LHTn1nI5%zXCPjc- z0@oI)^*-X&NBy~#^&Q0gJv9Jqz84^ZtGPR`ejanq@mkpIqJNHK#B~r^gOYq>Un@1v ztskpwSI#xUXe1`^3ea+M&Pc7<V4x;!;Zj*&@k3?e52nlV#MFV-({q31DNoJ=ue$Q$y*0njOFY z3_llHA9kmr&%y76)8@Y7y2*#`HJ&o>_AvmZp|-Sj1&%7Tn}yu_-k>$N6n-t_!uHK% z055{$UeFk2V~?S_GeSETMcUL>*TODvWfk;2fBD*epJ*qqtv5|!wAVxTuzTV`IM+dc zTGc;6cQt4cP^+*pk4`dhTA-`iXJldlh<6=Cd70bG2GyR~amaq>I{J@~4^Ass_F3KU zlyaEa>MmM+G!dpQ3&f6fB*U<=L#&y&CmxO{w%?BN5bK<^6D{mgFHsvG=$_oBxAOI0 znB|*adyo%a+{w#}{Q@rUm;1K-Zg{FA?7h{ss{U%Y)rK}7#L9AWy;8pCl2sG+f(EqxzgI6_xcR&kV zP2KNy4*<^YMR6h8dn*zS{6y-_^cLBGjCq`&B}Wn2H#wC{IuxGi_W~?wa_?9f&!3=4B!P% z3EVbo9-+FkW71DNYYh(EGG6-kcGGo;swscOZO4sw_k6eMMB1#faG|YUk1?sNh9+oC z@DQ61=mm}I2W|L75}0*ym{@{oV{&}@RI=qB5QU8tABt825RKs+Ei6j!y4@u4qOCkR0>EnO0-I`8;Vk_N8-BIM;hR z1di8gjEn<&+^(F{CCpBIf8hFc$L!VSfXxcp`8z zX|2n5SMt-J{!Bjk^fT9o-R!5d1H)6CJEwZuz8JkNYCBp(D7k1plOEQ7&?T3XQJkDfX%{GD-UPJoy?iJ=L=ddH&*syn6l0 zL99vjQN>^gnq9-%Gg~@CdJR13xlyu|#accsnJ8(nTmD{u|AgNJvqD?Vg<&Ar9dOlQ zs-?85KUOSpaOcz z$eydqzsJLaoE+}tbiR`_xs{7@Cl|wwT+~Z>eR``j0{QUbN3Vgn7fpMznra$)`)`0}jT<_jSS0pt0kF5XCyO!IE;s1ZH`d;@ zDbCSSJ-L(ZdFZy96q6$UgBKh1e3Jlz#qUBsPaaoFs8EF1_>qX(jqniln1_Q4_M7)* zWa@qLqxL(A7I71OW;eqX`g1xGE?KoVp6P75u-O?)C)x;Z&MNW&ohlB&c+UNuAzIf^scZxV`dkvn$##I3L%#Hd(ZKlg3-zsM%R}N*Lb#B-5k=44)NtlgtEDqArvdW5 z*j?PF8?4Y$HZVbO0$DBb4WNOOiXfJ6%D|b8=h)|>aJ@n9NQC2S0DzY(!kH6U@Ik9U zkDuca1wrP46>I=7td`P8x~_Ae0_=#y_%fp8wzmqkT*d@Lg1QT`=U&<@c_^Yk%%O z@BO^pkx4tEKXY5#5)t0Zx;o#|ZKN?L>iRo{AgVeT@_nM^oa1|s$5}dC!gFbycO@DE zumhg7n0ej!(BUiYXrYj3V$+9N%F3tD>(S~q^u5kH<~T9k zoB%d|7ubKZNA^@I(ZqF?V_5g}x`*Q@zT|!UT_tB;U(aH%Rd3@y5Nj4H6c8{vy@c(~ z2U6`iJ0@`-B!G@*uBe~w&YSPyq^CFqXw3!nd%}IsB<^eKHAjar%o$e~!Fkm#s#}uo zWcC7_TH3Vj>6ro77Ym4e`SO*Vou6A16)U1Xb&YqfTf4-*p@R5)aXXVV7r{Kxn3fg< zdZQ+P%_91<0C%_&gFYtcv5%>1v ztgFva;48bHxdXB@f)#!v3!}0&XD}A#Bq~nYNI}h$Hbl{ftw}xa=}}JVwS4EBRlf0! zy}W+8lV>NYB(z6?>D*-!X2*4$qjqXZwHO!RYv8x-O>V2T`vv^oE(3BSWxAGfc(=TE zb@}Sw`Q&f%5?AYE0kMC1`E{ffyIwd?L2Uo42zi(^e1~~Ct?5*D59e}wGn&knw<9aq z1Hm1FYiBLy#uM9oZ4%oB9z5#SN_alTgPKc3dP=zbKKTLg5rri;(u;qTPh$(rq<)Yei!jEL6Q%(ffn6jR5~HRu{}NI^=e1o*IP zWQ4iF=q?YX3vAJ$g%78p_&swFJ+2mhM6*|pL0K6wPWgyh(b)J6=l~H`K&@dIRvU95 zOs*COYK_KPGSXy!H=pB-cnh0?gr*UZVnYNMg;3Fj3+fbI8o!nF&Q-R(q1vRO5+)QF zf?!8OSQ#uv#G=K$Fb)@@jD-?mlF`XP?F2nBpaq(=1i}E4m>V;rw*$+;-b)ePH?W0e^~`8H&*{hU(a_4)69NEnR`0HuyNIEmEf z)Zf}|&}6kfdm}&l_)~fF_MP0`JsKa@l)0!8E9lNus6@~M*>nuTArb-%tk>>dmv}Zl z9Y+%BU7_Q#kLeqauwzgO=s)8AB6*W6M1o^C@O1}*R3jK6Cs<(XqPm`iOM$=bD_o-y& z`2t`so?Xbri{}fFT|n%`MeNCJ84s5*n{q(UWxsQYt(#l@_lW_{(seM=yQ>q^t5GX9crO}Kb%6%-YBzv|3N0U8 zr0Z@Q+0xAxrl#$UGl_*T)Z~XMFjF2Z(+6mWJ0Z%L;BP=zC^;O^8Yiy%)V4+bI~j7=7}co>BsfdbDn^O_g8^ zPOI{iumCIOM$aj$nD9A0J?*7McUrjRAht#UmyN@6U+w(5iq?q6(wKLEy$7*!6opz* z16n?p(X{ol@m0_R5GKsAm4&7A4Z;CBejJ2InyGJsiU!cg z>T3U72{0K&QsAQ!Ji4j?T0o`0P=YQYC?g;*of(PO!X)I=QiM`)W^@5w0$gGd2QRl9 z%*;06jkBl58^KC9$NN^Qq2~!uOW&g5MF*k@Kk~9lcNq?@NCKDvdU;I*P%%WMLmG|6 zJos;JaXcTF9;D*I>*@dL6G*5A;vh+44KcsVTjj?lppvKG@#M#^C&wkL@ADp(qfY?r z$Kdf^Nr0plW0g@gX38p^mqJI<=jzXkO@^=DTI?}AuB{WvXcnL5O>NM=^ficLE`41b1_bA}0oN#;B;zjDX-?x%v)i1}TlGKZnqI#ra~Nw^3W3Q_c1B z^NZ#8`2uF28Ye3tPl>@^x+w(NsbWX$E=PWQ*dKshyomkL^yYW_2g3vtS}3dh=JV&z6NhxclG3d)9m6R)dBFk3 ze&K-tIMh$p9GW>~oxJ^?t7%BNQ){qIL(#YL;`~AWt?zE->mMHE<+Ht<4qD_nrGsTA zsw>1yA6e3FwcMK!Db}*d^ja~GTlrUi=O4=N^3y92`?tSef2B|>wqG4T55#sqCe%*X z8lmb|w7T6v?AteO@oM!eFGPx0&u=Ouv?Tj|8wf1!VH<#*F&Hoa5q%VbTyQD?jC66Z zwOe@I0+j&usLB|rUCLKHs-#v(tjs7p^Dwu*ub0=-1)`dx?8EFJ)~N{6$io>es;CP_ zRTh-vYu!Bddtqu)7p%5I9btgkDo%mn!|oBSU_E+F=?)q;(kdba0dUaYo=6$k-(^+A zQ3tl-(3Cg%x9Bnd;^l|Ra`~n*0(%r8pJYg2W{~cggN^&x3a2V*cry2biqzTh%2}I!q zxKR_=6SZV2BCZiJu~dGM;5(L#5c$7Ld_CvV+;V*MXX2dHjj97GLLO>?hdlzw0`xFq zJhucsVTO8e+_(YG9mp@eyAb!%RV^~bZ~wK9y1?~RT4V-R=)!MK1ifc88pw!7flczy zR79I%4a#~BtpFO)uf2zkD68-#oH-i%2IsvYWnPhw^Y2axx;2R9X4TA=_5Ao17q&{I zf<>TpO8Y?qB&Hwd<6K+L@eVY4yXFGg3hhHWE*Z50;2JuhBnga}?C@`d3(;6unPZ+9 zy~Mg{h#JPc7jlMO3t&R_ssVTyMe5+r{AVA3W|3G0v#Lz+@OW^3WYfY!39ZnnxV%G(6J!=8bs36+T^0?ic z;d2DJr6b)^dpztWzP$+PN2Cy-r3Kf1cZPOWa{?>dS1Vh~3T$M~V+%l_rh%(Q@4wob z3A7OE!a6wJh6C?ysV_`kOO*?B|2B;ZRT(&OO<7enP+2Z*U%g%Sc2zL&ppzGCA3&Yc z8X51%MS7#(D38i|r?-jJ#6vbRIDj57qmc%S#ty(LR#T$yN6=0quO5YcT;}9`6m>D6 zrkytx#4;$8jZM~>payO{lVWa`=O{X#iO5^O?mRNgModb*@eCVv?U!q5PM7qet{ZuE z{wROuyC?GX5BBonVga!gv1+aXtl4AkcFa{g@7s<$d_@pANG&_d#4Nk^zxX@ZT4_{Vt5v5rT)%#W#h zhX>-FEfhowWM*Kj;h}h~p;BG)+)C@SCefVzWsl0>P%i)^+-RR@~=C z)D>%Wp0OLa&UkjOyXsj+ENGkv4j)XmV~_JerQ@81OF=s@pzVcaQ$(o`VL-+Zk{<(f zyNbZxB+xcnr-)FUzzln>K><+jQdJ6ig@9gz8AW&994`JTPCu?K6Wf@_!JEnM0A*%6hw!586&37;@pbcrV1`U?~@l) z6?X>WmN6xmu7tY)CH4v5JX(BSgmZvkpZD^}bmP7ys7%mtY~?f3e++;zKT5?grjxo- zIanPhkM{(~;$9Pn=RE7qAspHkBa&l?*AbbUyK^+(pi)hR2dC4}Ia{2s{bmPXNar%6 ztyvF|5yDhzutUaXoWM90%QB7Y9hVN~%zymxr}FV9pUaysF6Ht{L9Bxgf>v`Eu)snd ztJEN8u%K&z&CTXJ^UA?sy0^PU-M!{cxVqIjJ~=dj`?|Xz8KKVOt|G4kxyh16K?MH| zHjMNSX}w3`dq{P?SHHJtH*eBgkV><44@y%dmy~qCEDqE?PJ>aFku0My1ILS@Znu75 z+#qTydCT{07Rms!$lCUIb_raLrNDJm>c9d_75w2+okoCj8?7e8yJN`N*&L&mBu@y*CouEffTNvHF^0>2kW?)kM(@v z>-e=#e%DQyc{7BLVJH9~#2)GCvX+b@fFux0G30K;!uvFeJVQdvy}E*qS-uddvq*}D zdyf9M-yeeZZ8URGL&RnesspU+%p3?!@0_04zj?0>{OZCyJw+dAfZ~B9dYJ6yk~tEp z3x3Z{Dy@2pTbI}x9JK0ZApd1Ev3h5gO;0PRS*cFobWACjp{GgUm{FCKU7MwK>p-4t z%9zzy)HVMNo_Ff*VlA}%{4vcTY&_PO;crCcSg6-WVa;Gx=?t5L+5j9(?-N)c5^K^@ z1CPOl-o1~Pj3_68MQH>#P6ervPAsNBTd=v5IB!oOaipSn697a2Ld?OOJk8~=2{?Mb zCQ43|G)>&$nYIRkCdWJMoA{VfNI`gSU|}7YwAfgMV36~tGlmFaM+%IaSq~8M19N;X zO!*ODeUip+e&79jvKwB$Z;gVJR?wb+)&LFQ+0)RMcwfg6UjgDC#1e=yqT0O|ziW)u z)9>dod}Alr%wk7BN3^N#z?N@t9T?~woEVSr^PPw5b`+cWv*J>*s5khmru>_6%fM!Z8WAs=H4OR2V5wUhD5?zuzDE zH+etsJ3(*S8y=VK*p26kdl59MmSbqy$J*b*(w#l-N6r1Dq8S0D*-D}PA-}!?Wct0M z>{}`VHK!8PA+jM{n`{$q`|4)QNG!)(?=`v(x(2lGwRB_+#GalS2-TpHMr9`yd8--G zHtEys{7$10XqNTVX$LP}yfhB&#q*bPadsgm%kw6axp9>tYi5PQE$)eQuJv_hAa}bx z_RC2g_dCWiTviM7NKfWqSYl^t1QQeq9v|+E6Kh@M4mi(c-#R!_W&d3h29#7!qF*FZ?^jvS|_6x3Ou|0q9hyO_afFE6h*uSbWjpeVsZ*}~M zAT~C6PbcUdwhA^ae3-Ygdl;?R!Q<}vax>1wgem9hnrZ;6e&!XMlglYN0oY%wsCAnR z2u_iHr=`-U+6W$`dNL-$KRfS~p4^6_#zq}*B>@GHHklDgH6S|ytYD*1V0v=mDQ_(x z(#f)i43}!S?I*OCMLO0&*nqUQPXYtX%lFy9+iM=JRgZm|0H7IV5CL_;L2jFvhUoe* z2ed%MYhYA}8$lYzEbNk_N$>yDZ?>*iqqh!b+t(;&5`YC59*tB-`$nC7J)fXb#`$eL}uVx zg+?%)k~#qV$6zFgjh{wJ0N!-`y4?dfS+QQYnI$HHyMcSa`_%H5(d#V|(TLKWQ^j!; zXh@{uBWe>jo{TSeI|kG3}z3^z%`R=seI4e0yufZE|2K9&!2Y?n-NF-er&r*BWRG^v*)uz z7Y~?GShnir7hyw(6^ztdd!NN_@g9O5el|d0V-(MiNM}?e3w^4j(n3Y{ir>BhK_!cB zj)7@(p{p3Xcs)6tIkpj(6@DY5qXFCqb^+{}8Ss4pbL!|PqpV$2-aYd;m$-gTK(0l2 zGY)D=Gm28X)qK<+N;FOr0Gm16?#Qbi?Av#j@{^x_B0u}tC-U~~wOrpl%3cp^vUG!2 zq}%zqW9Et>{snZN24p5kk0S(hP}KrpCD`Z-@Q&N!~iQS_43t%2>VB=b=87!e)u_)|;RETh&V%E?Oa1Jb2gnQZS;S@r=w>>Ua@(#10 zT|R5V2JJ*?^1h;yO+-leCg1gd$SD^n#r_>QWmL6*3%l8Ps#(v|KfebJ;1DuVpyng; z+=t{TU6i*8r|rI5i=06YW4)|wl9ze$h}OP;HBm)dwSlFlZh4}jZ4qeGyjKbr10 zL2fMGKBB@XM_+y01K1vvWnowePt4A^vXw3*(lD+L^k6UmvPnAkMTLo86CAoR$_;}& zU039sZ%GUtmqny>=}=#xM#_>!>pnmzz5y{)Y`nUDCd*_uw{b` z_T47cqmQNa;}(%KMk=rvcyHHyHiFpo*kk2L@>%~s*4`v$wrxodinVt(fB&EE-g{nC zE;lt`;2DMNfd?KiK|%}=6C_3miRP9iBWxi{@Ty)tGhk#{qAc(=BZLGJ9w33l03=I@ z0TYA}5(Q;r-n-4||Gvv+@5PFU?~Ao|{^#8Hs_Nyr`)2-`J2Q9gy;iI*zW5>n8Z}3G zlS#zuDRD%ACH#(7RsA1Xw;C3& z=Zo6j_J}JO!uz=>^4P_VlN()pZjxYx5?Ob6<=&Lc_qELBu>o@c41FX%PMwV0uuK&U z)@zWEiOYJU;pk=jn&jh&3`+@^ z?A1#8#umgr!P52-p@{?>JZ>wHggqgsZJ!Ge0wB{~dt6%|H6M{CV5bC4ev~rfenN-3 zYZg<%In7c*Z1df?mhH^x@>G{=kPMDE$1Lrgg=I4q#nC~=YQ2@K%WL`Y;apz7dRsv3 zJ7Zz*RvQ3U?=)o<2_!>OQ0s;g=<9R6Fd>z{kp~LmA-D6WJ!WafNOK-{_T4d6@ziwt zh}};eN>rtAet3LYI7fE#EUFVDFBVFuB=4xN3I~504$hE~-wL^CL>5&Da|-`$W;&z7L?sAP=KMosDZ{qtXQYJz z+0>05%@B9u%z};gvg~i=&%8|X7Zk)Eu4Qkr67A^avtwcvMPjry!??wd;`&=jHL=u) zO{vby-d~b`w;lTv5UaNWV*k~0_hNU#d=kW__94Rwop?|C5ddypK*zFdM{;wsZ;Y%a z-4EL`aJJG+>KvhT6MH9mi-L8*nZ3={LE{MavPUS4ZD^HI9k;H7G^5?mP*K3|^^lED zAGVS4=gk%d@uJN>K@t)aoMoJumuA%u24`V4G_XNOFFmfAGO(yaTHXN$4+a$BD`rue zt`?Kr!daUwJXB_c_Mvr&-pNBvlHfiS#WVnRtUxIiT#<=!5wMiwNOsT*Epy=vU25+I zsx$gK6O^*`Z-he=3uC)3mLNGp=cAuR2f*bP)K3HamghBD&ZP(dv5BE0AAcqvfYc#M zyd=NoY=Sxq5DR5>XeF`ObyXS?NoKu=v0C6iy-l40V^})R4$siJ1*0V_6XRd%XSZcq zq*)9qF)1iDV@r49HLdoaz=v#(Yd!H#pERN&*`ryR#kHpgaX19X`f>d}`J=Y%x|jj@4|v!Xs=uH$tq z$Yd|65hHu;*N4{@^Fsg@GeIqlO{2|2`LbBd4_wO z^t$-)>?EOs0?RF&nH*>CKT%JYa9xm$DC8{p!8oQkmSk;XM>Ib1agxyhg$Z#WJv=id z0Ieiy4+l@mfsDsZ4CD4qmX1sLIRUU?J6g8g*T^Vkg#qFA0r?@7*?F0Cp-XD@#6j24Iw_!r<#Vut1(zRGLOs&adyCgFiUCC zVEHkwz4&qHTB?C*o%9@lvI4PB-P`^d3N8YeR2c%-Pl@Y|QU$S7U94|p>r(1MKio1* zWMZweNa|{Z04%OEFx%dN#dX3~MWRE0bA)7dpPp=HF!d=G&^`=wa$qG12|v_R`#7E< zSbSa*dlbu6WA`Q<-a)qIaxw3x@1^{ z034v(J?b%lFwZlHWd}tBmV+BV8FK}DB2`L|Gi-rGsFF+zbzp2&Ize7w>hegz1$0-< zJYg;CnoucLJ)#yc&vmotkmSt8e~*Ke%d?Av$mtt!D*ud@QI&NnNE zlxy!V*AC)%@%vj48ws@{QErSdKdwo0-(HIpz`Z>^Y5Km&6{2>V@1L@wp3v0?D;wt{ zd$kfD<9>Vuyk-Z`WWdT_S`BIkbXK50In!z54eers>F>LG%&C4?SfxyWDUEIHn%|i0 zyPQ=-S;l5x_oc@8tOPk6nMofLRUfOr&Ge>dPT(!y=N9@`rq#moJ(|N{u5*oZ^%`uD zwa}dz@XM&fo%D0G#N*z+JbV7a+S?u+9n0~_X#t%F4tUFJo|mye-CVJ-yh#aGAj1Y{ zW-Za&Djj&u*!4eFL@DjA%K7tYdEPpLeSQT(|P)e{&{3ewpQXc`J+lUQ+V-MaaMERhLiR zQq=E>q|l7Ff78D0V@&K{EbsZN^@AsK=aY}xrzdYndmh` z_Pb3&J*vTzxVF||YhBCMlpi<4qEXqh9H2u@mq`#du2<24W&qe&XP-a|1+l#kfbGVM z8B3zP&?mOGS+&eaEV}xeHu^c@g_>C_S0|FUz1<)Ij*DuTD~W7+Cd`OT3|C#`DqHK5 z8;ey3v0~Rn*`kC6&_4|9V0OvJ0I7yg?5!u(;Po!V_tg^~aE|q+#icbX)>!S{JD53h z>fSE9s4aBDK-)@vBon+y90v1~GqP^I{-GyzA(W^uk9A@Y%-VM>J?XA%&e<-Aok}xu zoKu2Wd}sqy{e1EBxL^_R`ssFnVxc{8(2`U89hgS|cx?87sV&q(R}c$FBM8%M2$0?{ zIR_p`$O!B!*n$-yG%y449KO~T0!P1LGHNK8-GAzNj>Q~cIxwA3pUb22&+Nzy;97R6 zLZQTP7r}}F+OqF9N4;n!ljQ~&)nWZ}vMmumX#Sgq`Osq(jyNe0DL`__?Xky0yUDo#$XF=c? zi`u1#l(8Od*k5xp$M6|#{q4OWLWf9^O9=X?hM?>8W;?_bwzXEP4)m!0gX6KPMuVSZ zFQt_YRVDC(o`usJd)(o?sAg6H?Eb-#RX9+sm2rwa0 zeBIx!z1*k!yne4^P>G;}eN8-5O>6Wm7g_Q#Mt->=JYC|RRL zZml($m=2`VPPTckfSH-x+%1cTJ1-ZhfVe?h(9YmNcw+*sV`6z+7&HkQ8f!DkEV^Nc zaB>SQ2uKEm-N}r4_!!KmG^Mq)HtZHHQ>2K*W)wrx<*N|!P)X!Te6lQv%0nbF#6k+u~r6Gy~gEyff8JO zczY~dt>9iuY%Q&Itb?a!Xj}2r!Elt{Qh{qu^%|D z0Snm>i!(qc7E&hQ8K5J8&6R1h?-^kXQC2JJriK)N@)D0C^nwtopcZg~`JRAEUmy}? zHDhkN{rTK?*1EG$V~_SR4C8x)7ViXSk0DJ7=3km?%PWKT>OTdoehQLDIW|Z*{-k31&?j|7s{0NcS@t4_%P z<$S(62j&Qee6+Htv1gjRMVe2D@1J}&ZIPsij*(e)Cs)EL%%Pcw)JMb67K%pm6SI3r znC*IKA}i1cNfz|BwWOBB^^L3B96S>N-40SBhw}lsmNy_EnC@ad<>8MoNbHW;*83*d zwq02LKpfn8+gdP`I0kkEWPTuXI=n`#cc|3dm6bB178%WGB;dbufmXJvSJgJRs+HAN zw_54oVE?c@cPRV&-Z|{(@T6>qa;()4^!eq1RYlOoNLr_MZrv^S?C<5?p|#0Xf=EHJ z0@%ZHzue!q`nD>e@M;Kyxa1b|E~++wEoRO@&$YMrI`)>lwr6ra1>ef@T5a8ZubKvA_54Xo;@$Z+-OBr2^P47RH*TnpvnDnHc^XG=zL$+Sg&*$-{an ztA~ZG?iXcH0K|3*JBm%Km)X} zY@6!!*HTX)Hb;X-aNlwZiw3t?k0&F%fm@ zVj*WM$q~RovnAJ7CL{;oRZz~wMStHC45tosJ?7inTRVSyR;zC>3|urtFfCJg-@+5S z6v#@J2n*XGlYtU2SpXf8ut3z6JsfL*6ReX>1l3L>q)83#3?*3%SaRq}vl#K*NRG#q zQ4)MV61<>e^GQMm(&xuCE4+(XiM@x6cmQ53J_IRteG!s|wv3G~?X`K=XzIYo3cx~^ z5bkG!fK;z(u-JVEM`jk+twVupXF2J~bM7U7U=DfhkVZvBPP>69$C#Ev96~M+Z;XK%H z9{GK}gV)6C#rKR6y4J*y3IOSxC;Lqy7i8_*A)|)l7$Du(Y4V<)gpuOB;WIOUq9HA^ zjcEeJb_$r>t(05Td&t{wOhAna-diM_7p}JW7jL=I1%I9{^5bKru)`Fq5Qq4$zTU^ zR6y;~v2G_;#!G?ilF4HGhXx|;{$StsQ(1z7$%Q3!6>zM;w*Xm9B2%ELKvaRLf^GdB z25KRHG*;C#ymQn=cc#^u;e~xJS>oAZVaGXNdh(jJPv;rFIb3kxjXf@QqLyx|M^HT#c#_LaPD-}e~AeknI) z_kX{!B$FG*VqMazp^2>M=9^RbSzVr+;n-{nXxkph-CY5)cLl)ay)sy4=$NwgNoAXd zH#9MfTAlD33Fb1`iDppyz^rxXI%;X?jd0*cQjR7}AZAYdzQl!MTXz%m~r| zNKw^GM{R`Rjqo5ADzYOTfMOzW3P^$1$tLSIvS>Czg9O772Fu>uh3=F|YJmA3ZAHS? zL`#U!e^1~DJ3uzo=6Oo7`LqRC$kVyw_$tR(O&m)dVF(=!zfOrZhiqu}x!`~k;6&%j zFvjy~7(=d*QBd68he#6Vs58n*YKFD|lwpht+ky50{mizY9k$)0E0EdCffpbH^kN*4 z{c{q<&k;(J8TRaA)CS<^73XlJ-Qed_2Wlgd*E-kF!Bv?66icijlh#$a45K|h2fKDT znb>t{Y!P4Unlg5Nt6j@%-cTNtT?%5+v@ye$d9v*N!&=VHFA8uylVAM&Yx(B8*K&Jv zU!3)gF?Jf_W_KmBupVK|JS#Xq?F}cKLlnT3GKAe*YJMx7;`*sOdO|o?Er9j2jkQoD z$P`PE)q7h5B}!|VRLsWawvO?X5sefux9Gmu_h2oFFr_;C0k-QJ&{o1o&75q1ZCe#% zvVJG2K$uNv`qmChe7~PZr1K`XB0)L?3oq_4C9e-2vIN-5?ck-WMx0kmzS8OdMjnke z)EM=Rv4jKlaX!W=VWfaffCfY=Gc@xYv%P6r7qZ;nvnmD$`+G+2XnDy41+NA89`5fM zJA14k_sJ7ua}V_SqeFR8{yx>`PPHn6GO`B-WO>c!Y_P;iN^@_W8w3pOeKdw(i`~9P zh^3oVCy+Q!e`qh!=~MHv7u_I-%Uk*U*+zc!MV7<8yYk$+jH#`&<02s^;H7K-*c}=1 zeV@7&Z@FOYzxmVuRH@#2dn_mTd$RuBFVnwX9zJ>e&AT9WdU&DJ%UraQcu=o*k@Tqg zloPqWnHNJ*jKg-nJU{n-!`VF^&v-Wl5)9X9o(~zBR`14%wd=XE;IM}vDu{I>Hdy6? z2Q6QrVfwANB-M+^#sn7b7Kx;gQKpNb{o<~MP-^)a36O1`83qz3SKEw0&zXYMhx-Ss z9a}r>j3#w>ctD#ul)8!pkmkDr&^G!v-o~cp>|`+SE> zc0J5Eu)3oJ&cq`?TKV4=Jm=WA)w$NW zAv>T>Dqu8_RGl^Z^m+;=H##2#f_+!*p=YVLj*2_N1KM%>VY%I6xH)w-=_eA6JGTqL zfTpfS^QiOhcI_R;Lx6&hTz5Yq{yUO~{uA}MeJ3rbZ^?uNM~KZ=D3x-566ahxVWkdl zgwVwh+aSS9ZUPsIrC~sa4Kmu zqHGSwu}TOCUj4ZlUZ1(VBF*A6=xk<$v{=)&gPd-zVA(+IT8{Sa<>iY(UVb*19(E=Q zVmFwJ(%6_sm;a9sVkP}Q{XZ*zXZP7hAXaY$#A@r=-}=}$#es0ONbvv{mB}fM7n(cl zQ2^_>FIQJ0*B2U=*e@4&;TqL#0dm7yB94vb2RSB<#O!Scay6O~U~lv|I9Rzp+Mq_A zz^ykom~_vYEdr3S0|~=i_!-T_sf)UBa+9GLeDLe@b0m}{gy(98yeAQ%46dABJX}4ND*J92h>_YF>cpfeqU@drahoOSl?Pler&4e{t zYb{%=9?+AObz`9*RyDEG>`$Pppw=V?S|e7&7{P!uq`F0^G%&GhRKXx};bI~KDq6Ww z#q+g*Hp0s=!hXFwD;Gkipoc6)amPpkvh`ObvKiwGkLz7&8p9C*LP# zM@O$+xV~WlMWmo(`$z&avb!EuSGg6twR8k1EET5=EY&bbWDL-nsE4mC@2(xp7*l$S zM7~jiF?GFFaz|NNUmgR)>o^|@v<)2U@s1w0wqpm-?MIB8F-J27V^+82Zw0lQR5DEO zW$RA$xJ4AoqA%sS4Ib;gW+|`BK5DPjR_Oj(l1~#$wbf|f}tzk%4TGqHYY00bcs5QOr3f||J2&G&!AuC>=~@*X~JouDRC!skcWX_7>3 zTjI)OgA9O?G0+HRR$r@YU&46QeLbn0<6>Z)ScO2NR{`2=#_Hst7QJ7qzFy1bJjHK> z)Xp@sm`7VHkF~OKW}DT4U0EAhYfryLo)25mva{&SlcN`q4`?mggS`W*u(UYXb9sTv z4$5|L;P={ASGBA?YT25Ob?)HQq6tFW(p87AaWF%F9h&J>!9K@ESCcla9n!dwS-O*> z{Tun=XQO=fbR&DStMb}PG^)-K5IC?IKH){b~!cA(77Jrjn-`auEj+7@26-R607n}AY-c??ZIPh5zKhpbBF79!uiV@G;f^%AC&6ujJcrUm1vf_3E`;T;2naCd+rw zZ&>Ly83F30+Jl(a%@c8~_zZIh@t&%rd_3>#@0La$fm=%|a82BOe|!#YrFdP>KEqiR ze7jfuG16ohtIcddEV^5@vH+f1kA74KW!gA{M>0LWxCXhs9^=HunOuX7i@FZUVl`SA zKTfbxk@bcA&Y4%V6~=h&8ks|a_2-x%qix#k>`B|~ibb5@GB!l)VEbkuQ*2B(Z0mNj z2^oR5iPg1B2^@{6Dv5Jgyd!eDyOg73Oty7sk4JmoSfryKId0zQMuj+UI2ZdTnFR46^kj z@(0vruVh(3?D4_1{P2q`FP^MqE|=vwul8gS-(0VmOdn3h>?el$<6G+f+n;@v|Hs{@ zKMG=h`elA0tWfYPfY`PaLhE-WEP`S53W!~8dT&2_cOVaU1;{?kOlq(h6|guBotlD+ z*%A+&<{29rf7iQ{<}m>kOKg$dF<`XbptP3PlBJiYB4-z|BTsduR?>|9G-^=JpIY&mM? z+I2!_9k!G;k$>s}Y_>OsVEv~u;Q+=}Xj)PH+`{lxliPxBX_$Zxp%_p8BD3M4GqSBY z(Q9Q$RBBz}4gj!^v}1`(?Ksy$rK_|MmF#QXST0LMkm*d#Z)Jtt;V=U|3dZUX)dgBd zgKXMvL?H$6PT`Ta&W}h18tlGg32K@&GIQge0bD29RlHuHNw|d zc6ELs*JtN)aaJCe=da8Dmsght=yYLSZ=InfsIf3Al0r>qJyzz`>|vt*UYs!uCI@pU zKx~}@w7CL%)OF2R<6KN0)#wt{x0o-Z%goeHy3SYMPwyobfw151qqv>kqD{C0(p3Q$ zSLHmte}5)lfBm&JH+cX4LT>N9x4qSZFUO!sU2AsuBBAjK5F4$eIZgdZVdO&KD%#VH zqR5>)j!8IejlJ)hop)K-Zs%|u#QNvEy+j>pL^Tt$#>XKX$PPBqraZ#<6%5aC$()7u z>)LFs6KR)`4Ua1Lj2xDhb=`TK0Yc3-TT%U4*xeRnXqUF4I_C@I@yty=sl;7kYxl&q;#k)e+{e1qv|0wCp!H$3GQsKLjr{lv zk!MFMndM8F^_y5nIkwv|+u}z??%MJDeIT}c(jV+rCeYhQLG17So1&EH@8F%&5BlVn zRH)#KW%cmbPF4lOt92)v)qc5<2jzk;3ZR?2aX`tOmT@Ke&m$9P#_E;H6uh=Tr~P-G1JVdACoH|B}%4zPNwI-&dKoSM;cYOQ+IB9&ni0$7|A z0QdxHO2-NZY1)pn9XrlUqQEvl>}+0>6{wGu#BBc*=OfN}N|DevVK^GS*zQD{I$|=p z_VIkPIvQKroRiF}*Gg~z0&(^Ip1>lmS;Mq4t3C~^su`#rrNN(7!Jv65AElbvz=RIOcl5Spl$7^hos^gqKzXJgHe0(l6wS0Rb3rU-h0X%EOse)4pCw^ip0H_HdD@jvd zZ|&QJd8M3Drx%<6WCHuEYYxGQs8+1TsrOS!+lGe%Z{ zt+KW1P~DclH;@qewy`o^RtX`y%)&a0CGV=Ww>+>?UM49}If0hP(lT99s_Re|^F6Cy z>(!Z-R;J4~bmi)-Rz2q$s>K9|MTG~H*JEvZz4Zm8FJ}*OxVV+4M>m$#b+WH|*sHP( zY$wTF?R4Q>_}(%6Js`H6ga7bv|9yGC`~1g2?BA0!d6)iXc{u+*5S#g>lOwBwa^FAE zd9sPT-4@U~9u;G|l&ecE3$b4YRtNnF45d><&;oVU185B9qi_VZsQk`mBBd+=N_DWU z(6xgrzk@$y>H*X_XsexnOhbe3$ieJ9o>*?6yvjjz@i8aSy|Tdo+Jg;blM$dx8Tkz| zuy%2pp2%%NQVZk>$lA=`jx_;=;S&i1Qxlrn&ql)7r9)JpuGyAac0>);LcK&V;4-kc`NpIc+PZk&LyCerfz@Ax^Z<^J$+!R_=QDPt z;bK{n8FYW!2UQfRwyAN5y2eo2d2)tDvwW@Oh@gy1GwM|vJ29duN;prhTQCw_(Cs`= zofKY^48T5Oi%(3cY7Lvom=Pmm`6T;fc=f?Y^H}N0Rz0A$yEfQ+X&(3q3)v$YE}nw?Hbe3{@(b$8)sXUT^%gsuE4YaQUKJAv8ZM70T6w- z)~x`~0vPX=F)bkTp#ajAg2J1!a9!QX3j5geH}?io6`ZchqIRz!wtUY0ZFx;Yjpch* z%G}<281i0O8+}~9^P%jc$E?8kUhA0N0|;N0_upI_2;QiJT)tbkyK+nle6P+f>UMEy zfcL6^TxEppzAU2K?K-ZTE_8bqqMUgICn5=T960keHj54_*2cU6BL`=sX5A0FKR>0pZ`ucGegRb@CF7Jlu4Sr2MVS8kqRRw_R3(vg-6!o}~4Be$jHmf>#ZC4 z{kfGfwGS#M(0Qy(Y?(9$KU-j{Y&NHk3R)sIiKna@$AGq_4l~J8x7zdlnb#}jOjuyFS~N_77k*qXD$}< zF-5^)`&0w51Wk=MH({yxAlJn?yxbPVZiD?~=D&Mxb9AX<$Bx1EYC{8J$FY7-qFlij z`+mZ-btoyeMo0jnKwZDwe~d{8;C*@Zfo*^q0-P;?U)35_Pk8q+&#g0uE(q^%EYoY& zzK5J$%UbVzhjd{^=|!6u-OiVbT{85$t%M@VRW$>Hzd=J8%D^5S9a?QXEe$faj$O0* z+qW(wuw=Kg$XJ3y=k-0!ph<#WtO4;-+s(Q}07l%-1B`nlNq~ibA&$fWo}=b?$vmf5aUzh{Q9E}J1>0d1RVuqHF-+f2p(9j+JH(nLM!Fjie@8R}MX z0EtijdUg#d5yaL3ZO3h>z?OhiG+F6kV?Z=*Mi2k5J@H8yQ)5dNgciVP;Pn2%z#iGs z0Zi(M{q(uDhV~V37O=P~pi=?r?Zu@tp#=mQ^LchDmuF{k{o!0L%XX#P`MF%5U)rY3 z=#7T#3s};S{*8tHuM2Q2&ndfFKO7mEU!v$JTM(3ZFk#nHKP$Qj}uvs&9e)Bn&Q&X9sl?{Yn1+Xg(ODlC=`aEyA zP>%JHD-9fVOmQz~Mk2yByIW+4tIb`N>wJ6rAaC9&fPEw1e)mSsE|eiFw#O6p0a}x7 zCT&1e!j9t6g3NUfHm|WwUXg=#dg?4s-V6JaQvJ=bhtrl|dMg0_*v)NN^!nGLm_?#voq8-t;xbp zI1xC{Y^>#2(o!S<+iOUAWv5A(0B$2embTJ#ty0M_YNl7Kds~l;IW2SR>Kc)cGH0|w z#?7@mg15%3szl)4A{sikO*U|M7g|``?r-e5)ygXp8H)?H*5=^aNv&{&f_ZDL@o?EbzS>>n9xt951%_Kz(Y%$ph< z*nUUrTkC;?46wKUltI1FP}f(%Kq%O*!-1&*tQ~RJ1_o} zxBspGk$m;B7d{GN_4bo5yWcKf{@b4fu^&6Ko!Qlv6mhGkV)=b5t#xrelgo<&^wtI4 z=N@d4E{|F`-!-tL{gdQ6`O?Qjtz)i}+dW(k=nf0{R1z>ks9*P+MQAp(EH>bW2&YZ7 z9x@G|OymsW7kmxxiVUDnj8%YG1*|N-ZXisJ?AC!TWdVL3KVneY=LxzH(xo+z(mE5v zI0dk#x$34yB6t9v3$W6ju$JPpz0Y=gF2Eh6Dpers>zzzIqw+*EEcfb}aDu9QtxT`BGrAlmEtdAH8Mu5XG*T zt*b3NxxTZ3lJQym9OJktL9Wwx^2R0r?`)-AzYII)`xz_S{IHG)Td)jAw5lmv#cPsM z92>#wBLLf6UrI_$ximv<_Z~JC$z9yfZDzeTwYb1<{|IiiG9InJ5f%fb9cPL3|+<%^Ac_GBaT?zU{( zh?q$AQGYGB@;U$g?|&u#-N#=0B#8a#m-)x#t^eXDK`brOC%(~lLf|lV(*ZXtgJLPy zm-FJW6ri_JCN`PzPym~N4A zI}GDQPT-yh&9#S;0}L=!gJWS$IBg&4LTwg5-iF>KGWH>%aPK+UZ`|U0=oJ%uPV|(M z>yd;t^}4WH_^rCp5oq35*9MrfrUX(FhP_=HSgS4!zAsBXjX4$*j)?Yr(laLzw4Tog z?)_=IJuRsf3szGYeuNfMAFCl-s7Ky-->pY4U=gFl5|X%jo~KLz4;vY52SQb|M)TE- z8S1f|(M@4QnJCz%53_b&kN2eRh)-+CBSZN75Qt)kG%NCJsKELxbriZXmgsQnKKzmsobZF{m34<11sTz*PZdSDrJl zs$leP?Vz-5!_BIKQU$Iz<#7QLx^1uT%IkVxz*hkqYt@uqT+6C#+X6WC^>qQ33RD$n z-k%p>S~h#1CPtlK%3b;U_QSbczdx6oa({#S>-Wx_>UO1%3xLt*?>=0(w)D)kruu$O zrnZCzaH+sOfUPp43S<>5>bD;8>(y} zr3k`a#CoC6_ZM<}dLpOKo|f&IJSm&jr93`4u_Efq%xbxw7HsNc&is&*rBuVXsrI^0 zkZ9GS=~XAbJ1^SxWD0r!ViEn3Xx%{w2B%q{L$GcPJ3d-HaQ*$8^81zHzAR%%ndoo7 zdu1iHKAc_2y@GtzK#$JAj#8UF*RVLq5=M47P3Xu9Hxellpr4)O+L=Bn1fS?L(piUX z$jDESDQImU^Emi9?vz##Tb*a$C(056oU+15#``4fVRbzGvHyaQqoEFa8xhJRC2&x# zt;|7q@E9^J9c?8#b6Fw%S&fEOI|{mlOm9=2fR06gVjw8~%?1QMeuhdkjjmm_h9TMS zZT_v-p?`JF>jUn+s(>bB>3k-es$f;wRhRuBY68uxZ6K~zn$WET%{TY>2^nkS(bVy_td1G%~Gi)Y-I zf#A$#DxO2i+Gt>gi(Cd?f=_H8g@wH-ox)#eMpPt#NE>d~e^J7UTH1IU!|Vt>j1`Dl zX{pBZ_pjn#DH39H6ZRd^qEYvilqoG>a&=Zf%>j*jv>z zp80FK-2sd$NL7G(TR^F@uB-Cd3QFU>%F62J6+o=a>RQ2ZIZl1-`q%rYETDSThqi)% zYE%`pLdUvYZJ>qoY6MowYUA}^*Q1B(ZLC@k^3H(atSl1RHrT2f(@glm%%cnK|MiM&X=z@Ad8?UP}#zm$XW zS*OpQ$deb(%l1^BJ$;5+lx2>VyUTLjKNP3t>#rTeUZ~Te z*KA7}0kN=u$B+{GTnqu~Ci76_@kS;pHR3Fuvv!}IjrOcd)cFR-{83xMTi2Y+XmS8@ zf*SWsdng^iYeLCFGR{-b+X46{jGI=UP@`@2G0#^ERwru&^u8_P_9zy9b2K-ooxcR* zp^b!jw8-~Wzn6aj`azvwyC@p_gU?7jtH^=dEmOG*wwtV`s5_rU}6UYtj>CS6M~F=)!z3=%+mQ}4wsEj zZxij{SL}0|nIhDxC#=g0ww+adtueAQB!SILKf73{3A|4cpi6CL^12MoCHc?)_jo z%)$1(wTu;iQ$6kqnhb=J+{7`8)T9Rj_n64AS}7P5I38Jo#qbF|1o|Byu}F=<>!z6&WKlX17p z?oMuCT$$wz56q1TnXJyK9yb3F$RxalrXNQwqs6)F2j&{5uF}Lt_+~3fW1!VEtYxzd z{&KH+RMVF}7(1$f@lG|Oswq9w5>E~~l^sRw)OXm$P^H7#aHGZm1 z+tV&P2Ya%2bYKAT=*h9{D=0lW@&e}NUPJpwr^oX2X#thbp2=s=U&v>leI_rSJ(uTY zdwTk$fY?2&>M&Xela&wdY!NnFw;iK0Lkul|J{!wrjPL39KRuCWpOw#j`I$T|-=m+e zU4isC4a9z0({U z^L9kmBWjWkxiV8K=PG0L>iXIey55$t_Vq8mvFZfZDi_)K!te82q9!caSj6Mp>683u zk*+Y4PGg(I2vBJ@7cA~a)>VDatO16;8+hG`zgsO_3!3A+B*+qQO@u`|&GB`oo;J#& zO(JMJ+pMRoYLf_h31YcE@k?2j3H7K`H~DQzn#fu68}h`4I+)Wl+(u9*zc?KD8hK+i z6$7sk1%<74!-hKUL)3(K$A@%%wvt&I_2BxvL1h8&zEw4}OoSW3*bcdo(&1h}Oz@}m zsl_9Ra0!}7P_EXq?Kdmt%%Yp35iH1Kn4_`)J8sPuF7@%wTwV)yURGNT>D_7;74KFhNcWni!5`N@MEE>|+| zH=^xk)oIr{>JuOI*Kk|>bMn_d_1dT91b?^z z@kx+1jC)iD!hu{~j&gOGWwp|iykDk=BK=;Nb#9(-@r%2jtyj55=SeT)@IVv00wrnB z;v*(@9OHsJ@ad}+AkyQS4ZUG_RCz?14t#h6iYFBlEEJ!ieHqZi0NNKEgHLdL9a_8b zAa;T4r^3^m+?&8;oFcje^a{H9X|mnjOGpWsP%gS^%S#EfarQpvB`c-QKJ= zd(fNSt1i}>9GEU=7TV(xAel8my}$cto}bDIYH2W;X5>r5TtxnRy(Sr+z=-HKPRFy$JFPnz$&#&xx zWli;OO`bA#RUeo4s5W)2(1U%7TP2Iv>5;6qXHOD<@o6n z*?)Rmo;#I;Cnx3csq7cfy43A(-}I>a$H&IJ9#cbJ7Le1j>EA~O2iBfayN79|0?T|Z z!0h41^S8fla|*|WJosw30(3SEDzwBA(icAwz+R1=tx_&pfhmXuFzIrDXX!`) za6Rz+2G_$@8zcaclBk^*fLL5Z17VC_)rhBHoUtLolu4h0*4@`AVXCcWACn91x$^5x zmRM{q4qTtdF`l>gWD+`$wYY|3m8BmC{Ebqd+1H?+jz?m*NoReze>XYjg4xhkbVx`{ zt5G$d3Q5|k4lGG$)dgm(+nOqfoe{*&W4%!pbv9diSJIhx__7KUkRJ3*kW(T^S=)X# zmCxZj*in>6ulMYxsdmOXhb?C&EmE0*>Ru1$YdJl?D>j^+N={(6O5G@%YT2F2w03*t zxux7(70>uw7tYf%nvM#X>Aa0+^6-FZ^imh4Kj>j*WXr3*#6;QIKiiA>79dP>k=)}+ zV1PBG#6?J7_pmb)e0%`8xO)u%=*6zLWLObO&jS;iJn>>+BLLI7cnCuWno9_3eN|;> ztv)T7TC%V#bLo%ANO15ZDh_@yjw7WI3`w{Qz!y*nCXNPGFtHQf!_d1|e9j(Lwrq7I zVC2f;s_`GjO|3PWP-=_6Gf5N?x4{JF)j{lB=%;oF|LCO90?h|nYO5@e>`-KeXuFNt zof>YLbsp|8IYB=YW}JG(feamO_H!**I6@WxgYD?ghiS@Ixxlhs=bm}(#-on~*Z@C6 zS+?phd4mj-TgP0jmK#hMR|CjthI@2B=_EeC-7gaG66FQpad5r8lp7jdLhX`ZRYOD8 z`*EM-^q2^ z@8<29$pl;maOIj;1-%M9Z*EO~V4BkcR5h839jBCa)TFMB>Sj?@z#0s!U7|Cr2h*f8 zDas`lTN}%RhLy`}D`icQT(!2gqPjR`YfE~s*}|$_*V0Vo`PHbB05pxz1xQQN6eB-Z zW>vF&6?7^%Jvoxo&z{MPAH0<3FTaqdpTCeN1sJ~gjUUR(Kl>ZoAUW@ z{8&zY@VPwsVcGA`{79bt_=obMJU)5$%-C4nJkd1C-f|&_`v+yt6!1FRGsn)xVL5J< zGbot9zC5?AX$9?SpO{c&hb!(F8MsKOH@WEL^r=b+%CVN?{On}`uqr3e?>|0q5UXu0 z%XgQ_Y~|B3=1Q<}Hhw$HR${58uE#u?c~5qqC!IOxBnlG7Y=Bo|HCYoDYevUPGy}j{ zuAxydSDM%}i}FA2s10t^?R9p5((9=mx9-c>xw+L2U6%#CzLwWz41V|OwVV~FX;a>~ z5dhfoer>9)8;U+UOW6rZS~cadvS)c%l3mWsO{(lD7Fd=Av|rLyM}U#<)YuXxrV1Tn zy|;NehC^zMWpIFg5{C|9OYVslloXbVo zN4Gtd?P=44r8gB?%xib9#5(A6p>UdwkPLGfa8{7KXSHV48K2FQ%*%cU^Q|1s@8o## zUQP;#J)B?5UUy%9o&M&vU;9Qg{>Qzw&S10s!{7f({)bP!=X*iyPk$#r5q9pvt+IjR zucUk61og4))d#w6Q5KnAZf->GZuVrmJ~U?5Bn7sH4))o7>Jgpr!EOlxXv@mvD>90n zeK(I5qU<_jUL0%01v6Szt^`PA*BH`v-ciuWgC(|nq6F(Y)~W|%0KJ5PN$Ek;mUe8| zTKz@2dO!9!xj|1zvNBdVR|CKaC@}%33H8Zfpr`RRsRe4o-_%J`E9oraY@#>ic^IKr zb8Yu2>u zt@^X7|MXioNucmUZ0JORC*4V8yt1Bc>Y88>j4zEio_#E1NMx8&=*Ad6jm4a7Y#Ux} zuUFJmoi^{7gwP1qbDTs603vYe82Ya?kzVInbFSDSj&*#=c3TNKWY$_u)|?T7MMo@6 z&{_>vqH26_X@F*>YS#PqZsWC5RX2L=WvkW-GBuG%r2r}w_;4YYuioPCOSvug_wO&{ zPIavXsHzrqrHNRY$fZeM1r+OJt4ILAtj}+6U6*P=b>$4{uzv9FSnI8~VMXUXU62%z zlUcEnI#SPFjs-@6jn78~IAj{XrR=YBGd(GM}Ku^Q*%CIg^ zkBn8_QyIXECvyDasT@61#`Q#&PYSSo`9e-#l51*946Mlrl!@Kn z^Xz_I612-*xh9JaOpMMgINUnERsNui^VNly_tGW@nml63q8K_H#nu`YE3Ph32L)^& zKRuNv&z~B|Rz2)VIVMf;njIck*fNzxPm{9jclS;tguS=VPe`wFSqd3`p+lGw8fKu% z3hY`7WvVoxT}U!mtKCQ&nbtOx>~e!lS(M=*hm3YD%a~xPBCUmcT^yvdGCp6wdS_J$ zv`g2GCc|Zr2x{fjc6*_qV6ZC->-J{~P#gRS3&da3H~W#!2`8I?I0@{8RCbOLBG|q@ z>|5fe;7n2RC>7MEnkQkGfwf|RlihAfM4kM+N;t<35G&a6{rg60guffsd&E(&sm>Iy z(m8+3&Q_Ma+N?IAYqzx4E+gyC{?M?WJD)x@(!VB3YAfZ4iY~e%>k3D`SX>iE7$ils zE&GpwLxJ_8OOe#ucOB$`Si=$Z>%pM*^A1uwNbW+m=3PTIXT+bc5i?y=mwWpTUab_C z6GUpu_3tBXKU>~sde(&l)&(}zx2A@Ht@91ip3dh?YOaOWkJZo8^}M%IW$qAZPn$*e zAp6~=EW0;ldsQ6kOPR~9fmrCvK&Cyq9KX`-W1MK2-hcOFow|7YUJyI!+$GaRHn&a2 zzwe-010)n0&rJvWaK9)X@?J46vtnq9;Y{<$#@6{Tw+rb(3Jw%)L#tuEk%jby1&|Y( z10=Am4S0iIXJmFO?0^HWd4x|0eg`%+7<1?~Yd0}zx^yAQ3Ee$lrR*zhA(h(h(AZoB z-%WpHRDh!1P`umJ;$ z9IHfaH|n&tlh zfV;99rYq}ir5W_zKRcC00gLm)eK{^*(LnC; zVFAGVWhIzd0+#~BemOUm%bG=~4)4Lw`(**A_h=vcV9g8ey*;fu3zwI2@!`FEc=J}? zy?!kp-o3MqUaF0aCN2UHn;eko1kpsYMfr`V<+q)_cviq`0oKo-6~KKW`zI#`%vI9R zm2)8|l_fnB78_=SQ3HKU83Dq;1hEb#b*#>%CPeP83)PS`pDS zi^}x&PR~u)xL&!;1Fh7cQi4}+-^sV%y_R?HKgdNnk5))t0MJt{)b9|w&8-l~;e3xs z))JYpqO?YS59Sjr4neFFCw2ZaoX$kp8A?nUC^ANbpWlE^aj;4Pfs(Nrt|maqs1}i; zHniJtf<-^KHL=QB?gU#FjyoI|y$wvP9y1b(a-5^g`v8@XOKkDJbyR$)c9DQ8jnh9N zF&aXlCRRNLA_S=|faUpi5@f{nP8Ox{&YU}pF)uJko9Q)l{puW{cdOBk3P%K}i>23c zwI%~g-_P2tm-(g5el)?ktqgE*=-sy#0HW5o0AYR0EztyNlM&3q{%f_8lVYb-qNfrj z`~3L~iOVww*E3wdktl;?5b*&TIU#CnYww(EBt$jYUAw-!Ezb1&VjjLLfcCY_(^+}` z*7UHpUpIym{U^C)46}CoRHrW9z8}PT=dORdJUo1qQ1UB)*pF#AZ4&lcR;w?DW>4;J z_vGSSi})_f$((!WI!Sn+kt~qFT#nU?Vlq>BZUk1ycF$nsa_fzOO{MXjp?9zk=BA!( z^PYARNTLy$CVTBBjS)mExK)dApQ_UK2WZ>m@u21Y_J!(*=#I7K%66ch;zP0LbB zJdWQ07B-BZor48{9jg-LV3*mFmIkMv!T3o+n0AOv(HZ(|6Kfjpt|IXQV7084g4jI^ zmv~6j>gAd)(L=j^s(LFj1?~yEp|k|EkrfPTE6i>(#-P;8t!_F;E81W}WAIjX!)<4h z3M^4+p|3RwQ)((8(7<+Gj75f@C>Qt?h?Q82(?>TU1AvKK860lTg6&o;uW0XXbT_gM_Ph0+!VR4wD*?)QqZFOQ2ewLEYgssh-P zr%&Wr0ih}fc=CAxu1^clD`mMbE?vX)nqc$_&1UvUC4ACnS@|(AxBS6t(4Yh1%SW! z;&XZO@-qY2Dup=Eg|L9^c{wKu$3-SJfw2?%%FSXPb&D~kJvxD!l9(H8gZ;6!L1en# zeG0^q2uTenVie1pcR(N8IYYI!PD$F@+vmY$Ifrk{x%%$AH}cIl-^qnGthl{1$rQVU zS!g^<6a262u_ql>If1*6ln_Aw9`g|FXNNouSR1cPUiJ1km2sZ9mi2UkNo}x&7}>K4 zF$9|>T>w~T0qH<{w2`(c#c@`xKCh!QZnZ0xgjSLfJb=7ZHM9h-b-Af|61;A0f^yxj ziHd@Zh=v+CF)euk*PXHW%EBu&(D9MagHc~LKtELr2 zlCtVWH3^dBh%&0$QeD?NlLwU7EUd4R0M(Xi^8rnoQm0uHIS-GchPr-^zDGaX7*R;` z80DLh)kWQJ{SJMvmCiyg%H&$(S_GzPqH#*9k=-Z3)iFvmxsUg#{sbz=N|#F?$xPx zu8hbqF3Q5$7Z16d(Di!xjX+nIU@yA3&ecz~>_&oF0KD;WS#I7Wf}r$K?3WLvI9y5d}FY^qokOqE@P-bxXDR-d_hoVWQIlu!v?h?lN*?x z3{x|yI(XF17E6G4iQfX?I*b7lGAAfx81(F%g+-0o!9j+^8F4mKC$UKYRL27ug*7H% zt=i0t++=`N*;lPUr(Pp#)+t-YntCi6t=aqxrC7)xo}~VBz2Y_!!RV0CD>tEkCHPi zdtL5dYsP=s%Kh1^x5lcTX$62+{`%G1_Xc9`E^bUpFqnSTs}N}UEUW8Uz^gVjs2#Yx z9ju1@^M;}STE5MbI30LtS*LF4%z}c_tiNwLllkFNmPd!Of2^I~j%1+=NLdIDwCq#? zValTFreIB()6*9Pe129oyMIycKa(fAJ%4U2KHt9+!PjOdjx5_6yikz;{?afB)%IlMl?xvCNM4Wp=nPeOa9JxVv(!-J+L4b@6E| z$Guko^8(sJl_tzkPOFaykG9Jl_4*WGrL4C)4)<0L?Aq$oYGs8xbvCpE*oOLW0`hV$6o4z}K6(B`_RIO&E8Efl?7kgqnmcpt`d^n( zP%|FmG@LYNYn^h;;$4(xU|x~T)i{RJ^`NyZST{byUh*Dg~MMwG2%%Q_>P6SuKC zx!RFS8QAZ>eJ!ub{e>n5D#Nc9qpRO&5Qg6+&6*;9Y_??y-Lc!BZn}_SZKSKLAeL5? zb(yOZ%}N1aA-m@FjB<6kMqYR%4q|EB=x~DD>9zcgxqYrjjk&hen!R%*d@=HBT3_0KB>=`DqQRCUF?e+at2f)6 z%MST9`WqcM)2Z;N$KGsPwriQoH|3=N(_*u}mOfvU@x6wW29e}Ex$J+sTbBR1|FQfR z-?z*6f>^!%2%TUoqkh~ptk8yKePTvMfI5oZ5OvcDRYTD} z_%{%8hjF#f>vzsi>kYcs?4gbJe2D`rgnslEdfU|Wl5Vm$Ypd6(ap(6T00m&j*Wtli z2ICyM7k30A3=?1x<+Q}hIC+iH9(qpxeR$@0Qrg1X0YYKmCDkegsoOQ`(i}fGt}|YtU`$PN1m5Sr9X~HIv4-JC z2eBK}J7k9$>nfv(QTHoj^^}QK0DEzECTH*7TRYN=w;ya{sj91Y<@Go3<+6Y!P1w4A z|G|>C?#`~P8`gMVz$jX_YQh&=wwhExyK^yV%)qP5ZW2S0SU_Df)_LthEkjkt%HH9j z^m|@IHDl}c=ALkLtl9fVR+egB#+I^&hg#RP0I~f7WEGGp_&j?4v;f&>Ap=kldRPFc z0=dIyPpneF(X-R?+=>0I_a_Q=HR0-u7oMn9!0KVyNB32*r@B>rZh7qBRkg9b>R7ej z>%w)Un$h1i+I*E2DWFvr*lNU@bf7m+)+<9es8@ZJp1YOGFEnFblikX_Re4Yc;o1}2 zlr_IAfb3Fh;GSQU>$$eMYV~@>L+TbDG+R=RLyxJPCk6V?KL4Wpw$J3b0@woX51*bI zW85Ddn9N~T&hc!22@uP=wK(1YO4M^pO+cdT2YPX1#f5cA8NzOiA=$P^ma)!VdoI;2 zXWBTjOL#}d{A>V}F4hlPdaP{s4{Nz8WA$}$!oL2+xALkuWN*uOS1s&%3pFPaiBi{t zkxF4bQs!~2bq%Le?J+_@SQyzoZkY6a)GdQ^eQOy;fftseEnz25oK0I7DdTkd(H zvV&S@LKnuu?%VVBnt@q;&cpmO=v3`>b;30POUqat9+_;wWgqkA^Y$#c*^hSsX+g3m z+clU2aI#F;U6V z$v=^9_*Ulmrp$*e#;~U%b8P-vZb|;jpM53&==*m0r6BgFU-rc{&;Qz^&og6sXh0qoeOk-iki-C{Qh`|h=x_1QUZdtKL2TDm&7{w1PD)Oo6ow`37!zL{sQ>Nl z*a*B`bJ}|;EtJn-O}%=~YsihjyQgvNswqu41{<_m&A|)}*j$4Dft^gKaq;u!Nn5?O z)ALrrj;8cFC}T;$sQX#iWZyf3XtyDq;P%}b#wpkH_V&hfd4 zrn|g#`M~9sXGQpPZTJsR$5C?X;~>{P&LH;lJt_B33oz4A{;BFwUwBCWL__=K@sWbo=cf)-pE>imkCIoa zQC*bJRu*!vJf|$Lf?@^A#bS%-VpV`@+C!p~N&(8krO&IbRb>kX@KA`HEt&27m-GB^ zexvWl^m!9gGpT3 zsX53ADl^GCY$L}i65)j1?7GT4vKdGqI|8)E*6zj$vwVyoX|0z7=xWNn!;we0T#K(# zl-qJ%RWfjKaUoxS{hj>$=U>a4*YD-Mo@32wN0gq$;;mD%+5HBrbrtjwz%qlnRwHN} z(`K$3B+K=6k_jDotFfu$ji`|Uwy@cFFQf!*SUegT@U+4*)LOk>EyAOR5HxUK{wB#4 zz>)zDFfXR1rcTstY-bTvi(VR;!d!}E>xY-UB7i0Fvy;2QYYZ>EjV99OlMvV~vt=X; zWnzpn=4|zUTL4R00D-L!yPXCKV}^WW0En%QkqB8yoELUwOx}jolc#15Q0Had%xm)i zO<2|>EM;AlfmHyjI#p^}`Zpw7D4a>*d=P;p6< zauKy0k?gMW7$ti)*4lOpl5N|ZcZu`i(ubLShyNBYec7WnEfY3pYUeuzvG>K!yp>_~ zhsAdN6Pe|=GE4X6yo^!Gj14n>ZMQ7{aRIUa`1^MGr6BhA{@Qq6e)!)Ip%cgHl0)pj zdz)gtsXj(0fQERK8Xtz4Y&Hwo6aag7Gb={FyteW-a+auJ45xZo$ug74%n}H7jSkTk zeU{S@nN%s*4KI*|GXpjd^rT6+Av8b!BLZ0XuSf=RwOyovtIcKtV!K%d6+CDDRO-j2@v9?+nBV&!!C-;L*k2 zX~K>UcpXuDu$BByZ!>#nfaS2<9~mRtQzD_QHnqm(!@VUIDNtT81hepd&OIiP9`e{6 zhqmctA#*#UG&X{BjBrM(vQt}iLOw!|lZkWTZIXRORw>K6m49>dgeyZTbZRVfJB8;S z;lP%$PxT^_1Fz%3pV)NV-&*;m1;7WCKfza|(xIX#urvS}^U6TP?RPfd%e8S}c|vVh)W z1+2=X7T~240cBQ|aaEasvWgB|3yA&fnE_aR{out@V^S6DDub%YOZx@1&W{cZ#4ZYW zn;$H#@xV+0TzRZaq)Gxr6Ryhq8t3JDD>JFdPO25vzqOe6%)x6ixr6J&z1hU*$!Dtv zcM^)1K5RX)E}_~$+TwcLp`Pv)90gQzSYO|oEMr?<({5+>n#we8eAD--{6J@uR-;g+ zMxDfcEwiH~u{%ehvT`t*wz^7Kb9?L0mC{PS`?w3@}}2j&59p zNmt$|4$tLUQ&|^$1G0$pX$LN~FWQak&6~IK>h(MM`7gea*Kgj*oocd&I{&zqk;!Fi z6TVKJz5dt3uJo`CJs@w(!g7vxh?J0NW%S0KQ+T16-FCW=EglV8k~5wz535D`Yh@Db z*Folx2ppvWfW5FpR79V;`rB$gNsaPJoM)`xR!vdA)1@NPRc6wYOZ$+QgssaWRlh5f zb*bKR1(z(_MUaU-+D?s;tlL!h(c93FRAd{u9#7WoaKLbm;RxBfX)p1-=L)K0-Rli# zI-}~K)XGUNJfYbu1E3;+u3-vXm+r`Goz@vjPj;w~L)l#Er2|>tSe z?|zy7Ey#$_-{T&oe}+(U59 z5*XC*q-q=lo&dZP9eB~=6#%FOvGi=2AQ?r=Wy(gj$zB9`4zkApup4MY31VFv)58=J z6cZ3jl^!(Nkl}51jp0kxvXVVzvQuOO$E^zGZGW5tmiXZBicbSuNFeJau_}nwaKCzY zTj-C;Vv-@I?v@$(?DSz19ITrra~J}i#HQ33YPw014DzABYafU68_ zQmw9@3#*dQ8$h=xg4(YpIuN>S+DKs8U~IWbiu$nam}VKK)ClG}JD>34QXSO+VUkQ3 zT@U|)8ALLZF;coljLyKC%&CmM+uK{YDS-d|yASemGH{Z(D<*ik>VXjmT zHdb)#7zaWYv>uPGTp9X#&Z(3d8?ZCqGuL#ShmHUP&iW&`7QY#k&S1oS%~ zmQsQaNrVL3S^WIaW_O5`#b-CCdKjC;WvS&M{hZh2HNLSPLW%@#N<2iWydl1!>yUKZ zd*ho1#4{nf{oqPqpe61WWkE-8pi;y9+sO0St_Q1>v9abtkoqapO~<5hG-g3U)v}^( z8O^AqgZFa11CtCwYfHW^#-fHpr0YP#WcJt>&gFHY|$S|H+*MFUcGD( zUD7H7j7@wjlJY0GCHX)4za)R>m+kgTL9E_>@@4nWlw{vH;%)va*>8#O<{~PPcdGF7ku0+xfG2`rr(oX*%2_J{6}&ECPNQ_W-kFfw&n8Nt{!D<6{3eI-S5R=r?o z`znXdNbf1mw0voEmx0> z9aWv{(UT(!^)C_PH@#*VOQY#xi_@!^b8lu60|j7|1Q<|N(5gWC0Lg*AupWVx$19}18?Hz|QOfl$^}8SnuNc&CjF%GLuw zEy^ON(hO^auw1}?IFmX%-nXt`%2uC#@tHjP(GTQV0kMbApV+xoklXLi1Hg8Aj@3Wy z0Ydu*nBs0EvPqSckg&tfyhgr0=7=$*EvvaBvC0_78RMAqg;JWZgQ=!`)%oXSf%UAn zu<7XK;>xwWy1JIjiwpVg^&9!4Kl;b=^PhiH0PTBOt+obW>8l~2PvMb$`zY0>l7Xhv zX?rF96adh6j^lq3Fu@%5b*D0=E$-*q=u1|bye62d{S|;t$rNM;fUWj$!iY+9e(1=zHkF{UVPfO8 zTtU0p%Dfx#t!sJBQ^;|yx!-nd8o?wuB`pbo^V*xAZt3N7_-xW}M zTfT2}eRs1SS!0y^Cj+vRH7?12{j4sh3Z!G`kW zkv%fzZ}fq=ZAxg-sf8`p8Y|+JS$IMA!(Eo?54l03}W! z!&o((O~AKnJRIOnCuG#j+Du)@iJ1;NSqT)>b8$W-U6KBt3v=0Tc%Z zM^^UZu-v=GwVP;@Y#5-Wryh&ziRJ->?VNAX!#aC)20LfLHq-0I@|f}bh-*ol=^e_qSaJO_F7}}F>Kk2{IRMti(8tZ( z%E>J4`&5glI%lmqp-Fq!*Vl5RI$JICrDa+WmQ!YQp-lq{7}a}i&#G*zCU7ZB+U+e& zcdAOTyUj{&|9{rrB-XMd$qqC3d*ASf7;>mqi{MfVEwpGrBLOb85zrd}+6d4>1W2H} zs+%Aw2wHVjWmj{d1pzgPCc$ozfBo{_v4NC`gJ!yW_Ik@?TuV*t}D1N#`V;WSsJCy^>3k`sX9vwZ%iNr zVuqqGm%kiS%RCrSaA!qo?0B}VpHseSxw)13 z=GvmC+B(+&?5kIDb#i9(0BxhI4IJcVE2mmO_tkTY!d_I+dQl&(A;ih4Y>oGNX)Sc8 zE69-T9K5QItT`q1Tvy}Ls+)D4sRN;D-Wk!XvZ~ikBi`D~Kf$%IH#D#%E0q8fSng2HX0fYw3T%`#C*xpu7fjf$H!jiBUOm0Fon1!vKi+9;qDntg4F}h&kJw_5=|O5r-{i#Zzust9q&e`p z9>j{Y4C!4E%SddTC4*hCwU{>X8fgh?k9OU@!~(iKJIu^&?hAN*KHJ>iE9`ERFG_Yz zpf<3`%=8{~PutBFB-lIDlP(i_7;dBGbx796D?UaI0Nd4O>~}*VP7nDz`UIB^b*|1-7b6FuMzwSi^y_ z`)i2C)^GUJKlv3+4tRSb5c~5#kc)bR|DgNw&d9vk0mZ~HyER+1RkPC`RJ~^!KyK%#X8?@mX2g0iGVflm=4$b2*HR z?-Q-6HxJDN))zzit2B;f2wG>PBn<4AP$lp}fgDB@`?-06E8_!sieeQhiiK(29UtE_ z;bsFSR%lef_31ovTsyrPS$EteIvs`&m{y}wo|uljgnAaM$Amh47$|P=z2si#GylsOGLlSigE*W~{%#fxFWLZm~}Z&$CcAkarZAmx4XCV9%O;y{TDJ z8P1ZNpSp0a>T0nL>dfJx7f4(11*$?CLCpyKa1FW2fOBaXsBy0DGK$GowA-OZfFeL< z>_}xWxveCtd|hqOSJj#RNT$sP_3<$xvy)JEuDG#miPFCY5IgVWzx~JmQcix|``#=P z`)6|Uv!A5DQ~!JzPkv7jYXGmnRHzZMX4DnL)`hd(C8H6%ep%$=q7G_0s_!4Y_qqDa z7AVMQh1ODi1e+70p%(S*G+FmJlrRX9p1_QCgPQkFq_vEl8;zjz{!0HkN#v<_SBRbH)^jslR#CPN2m)UBPy{U!NYM=4y*HIDeJ_jwlBcWr8?Lno1XH=Ckh{Ef`66Nebw=i$7fZvf|Md+e5;kjnWF(H#SOfJ~LDT5L zE6rvb%NAuN!R@ZrVG0A^eYTHCz>+fB)q+r5aBV(!%PV-V`k?{E91Hy(%xEnY~vBj z-7Ypm*B`|ETsS>tNX)3TEP-tx)6NtuRL}PO^wjPORIJi>7)_!^=OEif7DPzqE}o`} zCK9$`=atB+)}Z>KmzSo_pn9z5uTKofPPb4EV5cxGYP-6T`SPmXz2HgzRtx1WO#k)Le((0;$|JhUd4(jI>YQ59 zR`v+`3ay++6mv!dwr*?=`h8s)ji7E+`KUS=0I{mtU`1^Y?wP}&;MFJtV-8gTr$&fr z%353{5)E();s*K&<*xLe!6Cil)JbX)z8=lLJ%dz!Ta98@vNINKcdEBP`_g z=%J-<>$N+tW2&9Go;`ae&z?RrMS{m)d?{DgJNE&SgV!lhL7|9{TPrQn?E3;cZs7<@ z2U-)Hi#{8I7nns!%p69BCVLoH_5(qJ_lCm|&n02OmVbs(LxS28z^fINwk#t7Y`;f) zu~XMPhYoTVNexl_MD+l7eiL+NxlUP3G}AwHgxP&>-usRC{cb+R+>kcsS~jt%678?JR8p@uU$#)5hRxz0x80^ij-i=n0L z0@-59S!V8C!eE-)=0#RI;klIUW0|)f%C!AN^6qJ!_m?uDK{MKfc&$LJgS&c9Z+7G5 zt*7$Y`ahPRzsYCb2*m2`r+<)trykVr*bfNoSnojpx*NQ3SIx>6iM47+Yc{a18^$Ov zo=k_$N}bPXVl+n+t8!er zF%Q)9^$4}5aSg-Rcmoi~}BV~S`5Be&E3b_b4fS41KE2E5Wy%T>S=d;NF_)X?Oxh>r*O9Pdl+f;=&BY8IdDC!#?!jYQ~;||SQ`Di zckcnXt@n-ltm{$%gk4L3S*}Mt!im=fKsKm4)~#!2`p*g;jZvwmteZ=HkG5M?`7iBI zrLAtYjcqjurhu?uWMNoz4k1-VN))h}u(!%7;WuVW`+F2sJCbQrfzt{)&tAQ(p!8I3 zudhs{;Tp{au1$vQT&`bMK&slTdVl`PKcjrsr(YX2;G{mDKi9{vu8H~&iEo~x_??U*9Sdt!j-{j)N9DohIJ6F_@)uzfJk8tzS={rTH5XO&9n*-7aV-67 zT&Cj9C!~#2h9XMo4#mgc4VF2>aQa(npFp;R&9c!Z&lS-$R&Y3$Vl=in2RX5TsrzkY z&3w=CePtZj`~o0$@@%9_I_O<1kS zg$1>Ml3yKVgIU)MF0KC5Vxh&Tr`6H^tUCW6NZNf-?bO$j=NkjDO4+OKK%*BW6tn+O z5Zk8WU;g4Rw()cjJlF9=`SdooC=hT~7E{pmAF-&SBlKvb_Ad zE{dDu`ub7b6q5m1?;heQOVvqT+~LTWmZ_6sglTHFoz=@QM5J+uP7=BvO?TG6q(xpS zjy%o5_2Eg<^l&ayK<; zQ)-~d-;%<(Ij~7#xXj?Cj{8@g2@hz_0Wb}~mR3yBd*rqVRA^pF-y_q{V^loqdrFxR zmAjE`z3Vkm4yG=w8B!n`Z8g3!&c{JWvS_M-R&luzJGP0D){Oxf0Z1Wm30**(!&G)% zgw=<1WOp`!fc0W%jSaHHN-uao$|mLT%7Zu`5mpqs$lM|b*p%9LN^BnNpo7b29Nm!C zqVo}kU+z?W4-$&J;dH^vFH93c$I`|Ubdu5xdu+CPP6IpmjNrzujDNKpoiF8-ffXKI zuzO)q*aOo*Tpt{HhZ$|zS{)Xn_9Xw!h|a6w)Q&V2R4osnm$+O;nk%Oq(l5HPX&QL5 z@m}}06=+^nkauxb0Z9d!FDfuRd8Luq6O;eSWw7&EsIk{rfO69nnBKaCm_}Lk-?vxS za&viUnz+{&=hoywi>59W6rX52-B)t*Lhmo-M1g4qo2rLw5R-w|2NY)^XN$m8*j~{tM#)Yd14(@}Lv|Ls5eR(NV9>@sW4mdT=ZU6~vm< z*gJ1o5nNUk0u*w|5_AoX%GR&3^t#UH?BeNrZ8l>=Dq-6pTiR_n03JYBbgDzW09GU? z7B~5)803xMJ2Q1fpru;y=!Jwfj{$fLm+a68M1$>H?@0HmUegy}y^v2n{!~8uC1?y`USJKj(GPBT1vsVnNxus*5(l9AsN_V zrT<()@x&fU&4Vx+_rTK4PTyYrL}%Lb;&U@Ipu^oA86DEM3T?ImMw8d7W08D+ce%;r z09q32CC0F=PAgm(Sapy%+8R#G1`gH3`C4U zk%I;;Ci9z*#GNIKJvyO_PyVii+bdL(EV1ET$5POd-rcN zLnt2hHpvkCz-&cCW8FMA*JItJ#=7ww)D3N=MZPZYWZXbcSFdmpf>@0xp?NEvmfj5M zAWL5kUSn+@Ckm>Rzp=$e;v?O7t`pKETm!y5n)en|g?SDT%O1@=sXlR%lpLwjc<-FT zSi4O^VvABUdmgMuY%RKoF*QIoGKxJ!gs@M0^j`U1Q4yjjZb+H(ftI8G&gA1~8QK{< zQSl-Q1F)bCsJCaFoTzg32(*D1@v+pFr^Y$lZ0y)HvZvQ?>Od^%1dX3%rY5Rh*=8^J zOcmhT!YLA7qem0S<(fjxAR!7a0B|EB&jL?*v@|Lx93u?_ctvWE5aHuQcRQGcGyz-g zTm;G~Lied08;f6LekWya2y_dBAl&G%dW{4To*<+6{X*)p0+^-j^G$vnUk;F4e53SZ zXvKW`fx-C92M(5&YxM43RiNZxRvi`twd%MWXh*FB@2sUQ7Z3FHLwk=!;ZbO$bc1`^ z#Nhrt(}mT$f?zZ3a5gvf`fsdicWaVJ%A>tFJGaX6lUFC!QA}H_K6~=40?ZQwnb+DW ztU5lo*Bg_wnicq7I;T~e253dU7DZ|EhD)U(C`VPJrV2v8ekqsr@$4(@PUfw3Uwwtg zXJ5;iQUP9{SDeUgZ% z(vh;6HWLQSVLNELXlo2G05~}?fO11lNJ%p-Pf*rq1?DKnhT-ci$0i6ToSU|!sADk4 zN4d!;vb(ACLGLv}ymw>z`%R3mZROAw7>&Tiil2{YU|j1ro# zjUU_2BU4%kBjYMO68!i)G?0*DEbcf91hHl>S|OMfQ)YL3!zl+?Wn{cjmSEF!YPzSL zjqtx(;aFTlI2%qQ(Cat=s3Fsa^s|1(_Q{Z@YJ+CxL;9;B2`9fDD9$;==RQD;-S^R_uTpgFw$B)>jf(GPaQYJgE2ay>GZ~^>F^-oA6;pz7L4~=klWG#g=aJ-i^BU)v9+7 zkU+s%1;x*P)}{xG>P_F1i}O*gE_9(R3&V#q8qZUbW8FvYtXLyQT=> zKpWbvLjoA}bh9f9Y>|c*fY=f#5-)6O3%ilm>+4*QT@=@iUTPJexsk0IDby-j<$H1= z28972#UpB|d)-qS_T227g>M3$iWnHZ>$|gL%3Y1Nm-3tx?c_K2QDy-<}9Gd4<4dNaKwuRV2B5h9F$m2cNU%(@nJ54LQ@cVH)BmT3PjuV?_}3) z=^IJ%;vCZ&RmUdboNY5(dL(e+9cZkLD!aQZRu(OA@$Y3gIR>e0@(>SbXk*GHp z*>>JE^_(^Jub@^7!LBvsUhmqTSRJ8F1t}KMS87rH%&1^TL5%WNZ*EPFN*e*pCQEfK zJMDI*jRcg(dU7V)3TSq(&t+2|Z}k4E0vx@coZEX-bG8FzpU@Mmbrpdsh`u|)Pk=;Bd*&zk2LC9ABRayqRw1#Er?X7VD!SaBL)8T zQ8~rh+3XPA%$!%gHbAQp6jyXm8LR=<%v0eO3(x|sdOcK*O!aeBUw8G^eOc7|!8;F3 zL)Sp;!O}Eg1x*@6lP@f{PXrqhYpDlw1KHN4+4{Y?7a0SrjoeWY0;u`g8CWER_ebPV z5SZvvV0%2ukn-|Kk2=7n310ucWdy(8KmXzj`Qo!Ltg*q9r%$R4Qd`jNM;b{)C&+$X z@ih2oPAEZYA(WP|2ekJdIdyQ*)+~VW(3%c-CZ+-0_zl<%Mowcm@i;_!o*k!}qdIhB zIIvry$A|&`0Qf`#XgNTDa$t2O%!%0x?j3yr9Oz&kfV z12Nw6-RWX%Ze)eMBZ`bsL&OfHybil<)2!93sRre?leeg~!uHKg#Z#wJHlJ7f_MwdP z=dwsI>KI(s@45w0g=1E@#u$kDVRQm~=#BZXdV6CKtGAvP+aKXw5c{jz(HamNauBMA z46$li)`@vvUcZ_vkkyI0uAabTmNp?ZVat3%RUf66jGQw^E4o6?j5IUFh$KBvy?EDk zliaYcyh<`5&9FnCVb^}UD>Wir!lEr2@?85idBfnFbBOh8vC^&InwJO zXFOY^5}N{zY(^#G;GIR-%8rsreOI$y>dH5n5sqpK&Z&82R^2bbzodkgM`Bm#A&te^ zk;9$h;ebS|HdSs(L5bcJ-01Im1lGB?UW`T}dFl*a(*@=@8ea>F6$ZdlYr;TYBqQ-G zWTOg1!Fv-Hx_h_I1!iQ#)0n9%>_U{5kttLPx4F;gNxxw;<=I76dai6r0b*x?$axo$ z;ezTTU3+}#{(areHz!13SkVjWgBP3)VR-FSb=?&J4t=Z67o`s;LF&z5aXi>Qt^? zoLKbqTImBP7g7|6-fU&Oon@(@^L8>Io0YGsAPBFgdQ7?gJJtD{RXL!(Z>%7CsP{$v z+)BHcRg+pFy{ziGTI=I5gnXPSden_ayK5aPXk9sN;J$V+t7D;Ka)_3^ON)MifqzhdfYAXZ zixLSzREp?l4p**xV?MSy3otzN;I#o&MhPXQ{SA))=-Yz#-|dDM)zdEQP9=iddL6Fn z^|`2y%6WA#KKS5~{PO>MU!FdBW)6^YW^)J5Eecym3}~^r_L~hrhGbq>FX2_{vln5!-rq}V`DgAn6ma0Qum&jH1Mfo0p0?LF2a0pdKmsbV9QGu&R)`E)9I`&qTOpU8UrQkKK(dMx#M(fS!Hk&Nrotz46K{6Q$zKl7Kr z`13d8!|LsOf!OzcQjsO~Vtf0N_V0IZadJsq6-0^blSK4TLE?}W#)UmQndSOwFtvj1 ztwuUM(x!uwOnV;^OGx=A%7M%?s0Ea*ZYfolop6q$-n6&yja{glAf_|*yVa2T4jscx zOK#p*8IZeXubXJOTlm@eGM4s&0QXJ{Omr4B$c&*mV2m9Vj1L`?r44 z01%fU;gr#5KH@Bhs2it?-=DzoVP&)vUxT}(6W%uz7)0!Vk!I{12)#wYo&T$gI7n=? z1kI$>^52YJQINVXUa$=w?T(#V0u>7Zs2CPe{3jU#^vNT%1?N==d_DhZU8^)wt0`;) z#%eG%A~AS$w&1r$lipf@pXG%FYvaRK0IU*PN;`ng;R>_^O_wV;HWh@$5~(kDk}~G4 zsgdM`6*=i$InDOB9-ji*t#atMCUa$#{^#eiIXgG?fbIE}-0I(_=k@suna-}PjVsj! z6qwGJ8!1<}21IkcEBHLn->pvdSf`)?+WI)^V<{H>T^fi^)_H58uPLv!{`+81YQRzs z>lhr?V_wx`JXF4`>bX|Hs+?e@2WVr0L!%2E%DoEa@7Hm-r~Frq!0Onn-H`|hHg)(D zv|BZhqBInxzNp5pIu+VF_kR8CTMrGyt}P0C&p@m`KO8xQK%=ked|Zxr@dMqfN1* z5NSfEEw*(<-VVUOMQF(bW})mJoeRIkV~OJ-zdm_tHkM@~jVd+JmlISH3dRZpjCD-e zyo_26FS_-jxCC$o0~UJKtgf>xBF9nXHsF=byjf5n^<^ zOeq>%n9Un$r`vpJO^Wtafb>PnR3PNkGIaZD1NPO(8dw|k;;C`P8P-iwZA@mz%Y>|9 zj-paMUXu#c*4hZ}sQ$I>X~}u6CR;czJ(9~X`Y=RrAtz^9YeOe0@KCGQv!fm7*fN2R zXS~K>VdrOA3Gjkv7Mml>?Dy7!T+d4jxVO6oW?jw_H7d7SgkA4idotZrTX$L=@=vSX z`v@TRRUO!?dR#jpJqBCQ9#!Kvk=UxA{O{jHAC|Z81!8~xcja-NrT?8s7akO8EC1Ks z@E%R8lo=_i#6`QP1HF=)8?AIctOjTySJw(kGpGU}VS_#p(j$n*77%3uPxfK(f;IjUBu>r^27*Oyi-Qwcaol9?GU#Ux^+o#PI>8> zrg83%L_1uj@Pr*deZU<~G6G&?X53r}@v0Je4b z&_%=!AHA0zK++b-EsR5PPP0bzsv&hCPs#)gdTx=p8EObU_llnPKJVA7KJ7E6RN=zY z7U|th(s%tb!!o)^zZEepSD-++xIgI>tbX7e5}&31I5gufls|i4 zl@E@N?f&rIeR=EN19_mG-|GEnedww83J4FBzl-^3*Iu0>b(YkaYo=qfvqlDU1vf>b zsrvf%)^uUD6ijIdDqXcuz^cGl=caANa$)pOR!s{z{sL|;+-wisLFus$gF zRVhqc)z{Vtx~2)$iL_R`#}<9P_oKJ18`t{beHkAfOFaq6hb!w;rrpkzcA(dKaxd1>4TilOB zPP9WQc=`S#un8T=G^!hA4!c{6X!q+`K)X*46!dYRjocr9+R-Eo;`;Mr_PPufAX&5) z{Qntyjiuq4LCXF0^w#N~fkUA(uC3a)Rs`LH(bieZGJl;;c|?-CU`Mn?XAKEjwfA#} zL(cC^otlE}HORb{ZV%;WSd1HO;}PK61G1@IO9pYQNx`m*V&l6^$w0F28wOn2)aRaM zt@ym{y#h4{@nFjUieCopz#A1T5xnu@E&_4d|2>*ra&2}}_7(^pX=YKej5d|IxxTT# zZvjkiZaliG_t0I}N2O6(m(g3hI~TUJ8W=cdL*7-Jcw24anH=QLWRX9q|Nhci&6fGr z<~nN|Y_7MJBta$kVIa0FllV$1;7jdi-kf1VpwT#5S#N5O%B?v@w03`AsTHRRrGtr z&%+G~HkXtmm8P%ltQYmxC?YqTTX1|PFAgjMurV56dTGec}PatLK*rXz>8MwTYz z4bg3MYf~D+8gf#HC~i^^xK^O35nN5jYg7{*#3Qe_8k{->`t#)LakwqLKv+muIIJ#qPPQ# zvLX_#pjK5LaBgdGg*~g{oK-gu1t7NKZ?#E!9hG9RuD@A9@ZrM;23l24>-gQb>)-ce zaeQbxvD4a%xsoXmWQ(FZm)24_sO|=*kx{rr+CU#y@oVb64j2h55e1C7X%VI*J*RKa z;RGNSf{Us`w3Gv@#ah$tBv7oW` z?v)1`fF;GCcbNJ98T(xZI8PB>bo)Sx!(e*Ki%8j%%E0}gHKs@b5R>$g*df$LG9vL( zngrvF_)qIV$Y}Q+*(Q@q=@n>709xNq6DSKgpaPY6$V(m45E#PNgtUeXK{()k-I>Ei zF#rXT(OK984>*)?zU_kAj zwg>*eNNhcX|7Z9QhlKA=2Z+TW`myqh>Gi|NAu`b$1(Nc~H6xhw z!m36Nrx>+PUh|xH`O4JfSIz*sQU?@7Y7vf| zryr$Fc>?@Z{%^ea z$akS4(slYMII4_D4%VR323+Xh#r_wd6DauAHn?j|%^!MEl>WBLR4vd&P9vZSVvU-h zQK;;d!Heh=wl-jCj?BW<7aW{6s)IFkKw4kVmB5wRjsXR21Eg%cgW1w46W|YP!P;uw zqOgYtM@FGouN~-m8%a7-BOK4VPU{AupjC@pbd#AgG^zEjAW_qZN+HlF?1Ou9@Kyz} zjKJ3C(&5TDvPnVg!aH&C(XMf!#&KtE3>d&U?0_ci5F@tDAD1qXhcWbO|JM35 zg-zk}{N|Ln7g2f-jw(5}wr)Lx!sxmOWcrGJ76#{_c0?o|AlCQJA+DFFrKhOh;E6IC zI}VK#8&Oz(Kd(hMFJdn60(3!)zMEFU9HJ!FUH&VA(2M~_Ms}Y&+Z-#7A#Jh;xn|2A z8vRUPK#*cC9*FRU56+EU9~{d&@4Rc>DOYuUFRf_V)B6Kx5#ACO!2k_p=0TTA=uhMPIgWYOG=`E~BWK$;I02 zkSahO91it9%h7lx2jdG_$On>lAIn(2t}AhC0RmB#Da^hGVv{$qNZ&UX6V7%WSQ*m81`&^lE(wc5Omh-bLuU}6U#10h@ zX%?`UA=6Eem8~Q7)JPSOIe;M-Jb5qzkZWArEFN~(e(Af70H;A`&HGP8=PE&g6}_V* zdkcXJ(-$+V05c-0862S$i&mYU-45<+ahT!;&ePeRz5_tTc)6hzq`g@P&^%g61Wm5# z=aEtWtnpkbu(U{}0@%sZ28^0#M0ZmXs3%M)>}te&Ist1C)LA6P#Wz{A3Fn<|HqJF> z6jpzD%wGMj8#X~KZwkQ9_r7wIV{l%2AEqqWxm?oRMK$ zKM68J8O|7@?-7>qkqHN?AcDnJ*1&d)c(vR}Y913v>#b{2bM_d0X~iJj9Km>xh#+K0 z$R!jJn$#M9KccQVHkQ8aqTp|*{v^Mi37Wi+!n5%%-sxzen@94^QGu$0S?xGxQB$2; ziSJV&TaCOGVl3@-rb0(3FsQLskZU})aS@U^{{E$Nb`^x8!_5NJ35&Xd4<7Rd6bX0E zmfn#mF&yRrzaX)J4z+~{e31FF^HZ0r42d2NutGD=gmD?7Da z%fY*GRj{geE#^9STO+aeWoZ%E14#;Cm!m~qONLE?6F&?6hb|L>IctHM2r%?cHl3*k z`KWHe8D_Fcqe}wypdx|`+!K<-+#@y10}^m1h0v9D&@%-#-*wZh_Urn3V|Gfrsa>32 z%F7og^2OuF^3jJM%SWGlQumHaQ$A@xGBM$`M`7wHsW(XBRxQ zJ&e8?gEa#{gM;9{c7ot3fr`qAa09G#NIfFkWo{&LXTrH1p9}9$>0+=* zFEclkv{YNF0Xc^p9<^OEYXr@97JM&IcxLV5m)WySuy?EFp;5F}%OuA~ntj^Hkx~iL z(+XhUH)VoRPO6d#Xplr zKl`!#yaCkCqvAZIch2Yg`(C7}4#b=gASg%;FN{WTah|G|trtimunD@KzPUFU3Cfdn zMDMv7Xct39}QBu++`qY(ra<@q9n|aF#W8>$+UfxW3QwD5C#Ej`L z193_uUwYBakeXDK{)U6cTYJ5d~wD3VMxuo-~XMBL$%<5R)9nv}7!x z9GA%Mko8sIco1f0CsNQZGZ z^9vSH1iI^I7^*dFb17|J6&!Cr0KV~XFkp|P-xNWwEdfAr2xpNC_l)-r$T$IVO!3A2RIGJq|zD3yc0xGn7xSkg87qEo0t4Nd3=~ljG^{( zlaV~>*1J9G+@kokq|VBUuW{&X>$+EJkY!sckUcxQke4q{9l9Gd2`()5O?EP* z9G5wEU3P~}WEeyWir(C)0EFT*=8EsxT{aNLFf)+A{nW0X&*cHdx!-zA4pyx=yVHB; z_tKf?^U5jJ?5jJitB7)QZtA&j>wTy;=HXj!%Y*w5l&fGl$Crf%kr6|>5&Y(C-v_O6E+DgSo@nGcCdRa zG(16cDm#Jv#rJSs3M z8X{j-k7y}he^tHh%WB*=vw7F-FfoA7W>xvorZYpbX}?m@^+IWok!II!Qd>{0n`Bek zPm!@_^u)|(YR7Nii$4Q&NHb2Z!;0TSeu|4^np!K|d4fEct;K=}wqKIQvFcke&hdK= zVrPtJBh2&Bj(PSXCrWZ)_1iZ!X&36t?t@s3)D-lA(6MlNAouQ!3j^;}-^&6n0$Jr4 zvpbjq9OFF#zAI z0x1~s?Q+U7Zw(rEd{>T}?c2DXt*x*B?Z#KH35fW@*Ij$pH>a^V8_B<)5pV(5t6(j` z>h621@mRWlwO3O2Pm^BWTXDSdDLzl59I+fmce+JP%0mhYg&<0Uf9v^=IRw8Moo3H& zA5fR^+9^nzT(M%Fdag9qg>jLo`|Ofh1=Y_5-QxSE#K-Lh@3rv+`Z&dv?jE@%3E3x20YcL$dp zTY!>bJoN?m#%S?s(f8~TSOu*c&1B39V(Z@kusfv^DEC~uZPg7%)drRoz^?WF@Yp(Y zX^YuGBe3fVU{`~2W}S;XvR#w`+K6^>nuT-RK!(RZ3!+qaZ>dc*tVzz8Ec>YR0{NN+`_@htctFK({3? z-`ruBNVhvHx+x1tthhuLO+*d-T-X31HLKZ07kfWd@$P`gr5!h8CP<`s1Z)JLzpG-C zEZUXg$(p_Wxn~E?-hf>NH?AUtk8K=eS z(CJK?B{~?v*;l)FC`Sj%&mZeOS0Fsq&z{OKeJ*+Xv83CNtjWQmY!F--VZYF=OV{a) zx{pNpqiP1#HY-_=HpjrB+faknr@|&_m_+ zzT%!fLTWf(Mv)m6%gFwb)9aDfg>f74egkqX3QN@n|I{2~=3HfucuO(yl?)YUnn$c)5tym@`5;K`O2an|l47KL?M z1NpcN{KWhtXw50sSf;Zf(8C?OXlaUH7e*#Hw+J|G=4A;7s}*J7^>7$+s5!XtZ}bqR z(i#mXq2a*3)5HK7BMNDrTNEEBn^K&)eSC_4LGi(&u4PXui9m*ONf;P__SQ{?#ahA? zs0H8#INf|6Ak8kd0~;^Wqg-8q)Y}|wBg@>hn8UeY=eTJOZD{2^s|1X;mo1t}(c&I7 zCb8yKN+}R{j$Y?2r;GT8k z(kQG&VDBB(*Vd8_yp?P{2?w#GXO#*fKpq9++>)5LwzkFXj?GDc$-Nta1As3bcA>rM z>?6S|9YT7NzV4uH@|OOuqj5xjcRHm3;XA zBl-NZ$8vFbX-+I#-`c!Fl&ahTu)Qr$OzYDQWCj#c2f~^g$qIjcj~5rWx`Ga`IXKPY zjRFRu-i+VFXGUK~Y|-0S?#Gvj??$C`jaAVUhja_qfF7|2IIu}+XwF6P4B&R&g_20Z z`e(hA5FEmogmhTucR>1SYN`63^Xeee)8;mx_W1pRYs*OJJ|Eft?cj$%wy@(Owr;O6 z0IgsyZT=L>Z0)Pp`1zpOO`pvw%{GSpCAby|?(7K3K(^_<N^!>-(2zy{-|?`wt$N z3W@UTH3BLCqil*}V+0O;G@9U;W0_>>l^oa>+--?weNOmUXw)OH zhoA@`b7}>C?C7-)I>ET`j+Iv2nirtJX%YO+niKM2IW_xI&RJ&1q6AMY1j ze^6oYx?3}Nj@FG!!HqX_5h;jW$d^wxa(Y@p%0|8MdhutSg$u~*v?QTFpttYb#0AH<5iug z3^qDXj)+u0dR)8(A<@Q*FTp(|FGjg_1KRPXJ)`S4%BAqav}i(;@QCMp@S>QE9-&op zj}EXMXozdDZf~R8lJncLa|aiTXePjMF=K&FT`uF*(o<9XJ;iwMcNc?a4PLMBkCPpP zQ5pbXt>_@y17pNWn_@m46M*yN8w4Uf#veklT0-PXkEe-m(-|rO_AJ7dbEMH+Ya|JU zDEq>e5nXQt8x^%$_pb{9^EcNNou0zJIxG|+TDONO-W=s$*fbcK>S+bB*)Z)#pehOa zvE0n_9*8vq%u1N#bH=Am4tW_y8toqRkd9h2xmKn2>DDM4n;TdZc4~jm&;>Ov`^`qS z8*frVx=evKv_zrk$*9T5GTNn3&6sy0X4ZI?MH10IFl8i>_sti7%d=^7E(2S--8r7dK& zE7$V=u~7)Lz$>GXfmr9R(ZFniqC{Z?mUIGvvr!;fSP(_zJT3cCE}yumo0mvK3jwR! z5Bpv^Fw(xu{_f_ce_d+t;Y{T^#oC_XxEDlXi^*!u6{udB{_BeuFXW3ap2#O3e=3hY zd?a6g_1x45bS_EuigK3`{8igqe@x)9XUaC-DVQuBgBF=CX3hqt{>foY&SmtrvAL7+ zbI|EONCA7;TvEHj9`CPzx(Tzi*0Bz7ntgW8`Vv_J2eGYYV-64;cBcCsScmTVrbeOa z9bJ)#ibphgWZQGEFNh9Yxt&XYo4Yv{{NC>PXuJ8T7X2n|(C3Cz_YllTXRP!kM>X z5Kelw`W)cld5K&BNODt6E+)NAVf2IsN|@$CY6Ii9=y;ASH&cwXgNCf9es<7d^<&rT zqUj74f$8^loBWHfMnO#a56 z0{sAb$r&;*&^S~8Yg&lgjRRlP zGWJb^;}y}->`s)6yo2AmVAU|wu8T;&4p3vDT0*PXP9}Xe>t@z9d$31)4S-&}Kt`|S zgf*@I)`;u2cqCe*uBMJs@5+I_t#k0E-mf(hTj%1f&K%7$9oEg}7&@>IwHucM*tJGs z?;Xle!EIiz9Kc!^u55*0qz>@-2xM&@+8io~ayh{C0;Y`Cs%_5o>#nWC92>CvJq-x$ zD{DxbujB|6WIaHd>))$3t8wAkGMrAM+YhA)ot~as1XlU4A3pj>KL6x%dHUruIoFPA z_$dcMV#S{pZ7eP%-wbNvfx%|MMhvNwWt_2m@V+=^I=8UN^)pM%DKDU*?PRM`$|JGZ z1(~u$=`Hah$SiIAGCl3S*a^>JJSm-*A$RD(oQ2a>($FIBMfU6}gGzi10L$oTiB%c2 zs0JWGd+H*vLkE~iDH_=RyyMc-o^Y>`JAj>h)Y&yOg|bgQpieCR#(o$8xO`Ivv!`I( z1+mQdCeRpi_oz;+ru_XnX1^v2HqCLo zKw1}%b;zzJup)=mx`DMWnEG8>prv}+$4B?PU<0sj$S{dfZXr{sd z%`T$L-|Lp-{o&u2Klz@Y@-h4&AJW@Te_#G)J@3E0hhDz}f$w{jf(2dfJcl#aO;`!S zvX-lhB(Gm><@t+^Y;KbA(JZ`f0_+N8K`Y3v$?DP{lYimR`alO^iQZfiC=ZZa?i7Pb zTC|910|J?Z1E*&yJZ2ns04wl#qQ5gZKqb=S1(Dae+0p_-nbY3yXtcV7=PM|v8A3ZQ zqS0dm3p}jW22nymB88D^<3411)l6}-$iThqIKQ6DWZ%P1W{AiP&i$fUe0B zfw@4XMs9cKiXp7vz+lWY`&qa@Z_aK{q5|PN_#?gRPc=Qf&n=Dt0t~3xVo{?b$ znPa2>c&0!cpX?k3CGB&Jx1%ft)SKQeaqIlX$D?#pLY&y^*Ot17dY{>{$fA{E-GeC z;ULny;OwCY)M)}#^JqnOne+`EpMfb|r29ky)U)U>4K39wxEwZGSR+PNs{b|sYpL7{ zQ0F>V^>Nx7fW4XZUIFavG=keY##?O!Q*E0@T9xyUL2TFA#SaNZzsc1yw5h?(rh?d$GkNmli9G)DiG2L=C-UU+GdVlIcEy*JQH+~x z4OlB`{NCNnR2bd>raAbq9D+M;bO*X5qS4gPw$$wc%>V%IGx*{o_>U|qV=CW)9_(~# zDR_yoNpYlDDb4W{%wc2@G#%^?8>+)`=CrYK7R4i!acL6gb;q&AFYE!xpjWkd(5M8h zfkDRc-))y0&TluC(Og5gZx$!luwbnJ4Zg2aL~uYX;V>3}UtT9tDOjl8A^<-BsMz5& z1@7^b9~?m+D4vbddK-+V1IML(oiz`rcE~AnE`OyF(L_~;&|&wi!aTR&h(#|m=Fpy+ z04M-sql2i_pVA2qDp=J|>psUwt=$Tu@&Px@jTmucs1x$L(K!UuiAgelUpr zqaRoMoaDa_9CqjRZ%v2V3l6I0%+0|+?P(_6^j4UVVqo_9^GkXCLfgPCj2AnLi>~XK zt4u^rF29wuDZ=DlG^I0&xM%9c0iX>OCz#tjLM4lF)JlmCBz%mhW+G6@sIWxJmHwM)c;c)?_8N`=2LkVv z9;|Qhh>X}sHhVdMTg!88{606I6I2VLVi!PP58MC>Dz>$as2kSN8a|jIuyins*|4z` z4jMpk%1HmwAP%n2H@F_18foj)T)gYhgoqYf`WjWk=p%lF>>y-YA`HF){u$1s5%e_` zZD4M+@3-nw&{Yhe%pS=Rb)FFE%kXZ^n+Km*z1T5h157#6YKlZ%>T7O*mt!vErTO2S zXI30D=ce?*bp&ta-b1%G#SVzZBRD<2I*<~<7Vp15WFY>Ns@xA~Na zCM6fsmJ(owg-iZ5K6wXhvUV*ox5(`!7hWR_#)A`XRxM*Oq3fMd4eB+oXwYP}1}GG0 z7cQ$&Mi;yh#+++Mxv>%0O3s)SfEG+>zYfVASC`X+OkZi8?*topj_Z;_K|!Xu(K(VJ z0_%k!MiWq8a6PXZE#%VY>Mv~S;B6PF-s^a18_CsRI zkgvzNVLaklB?@mxjehH!mURid?a61 z5c~LxFXhW;FRBAu>^jcU>^Z@pNBJ7pvfrsZd4TL3>`~@g=mQJA1C4WF7Q}4kqH*2# zz`o9TD0v8+Dcf7hj80jMz=EbT&`Hd&*eUkMlv-AV>O3ZvpCQjAO>!~feaI?0TaS)K z31`dZN4HiZQX54@V>@3%i>(%;;JA5Sv?*N60?94CPr+X0&4HymM`%7l3YkE{Yub;h z8WxySs0C^9p3RQpAXh16`v`;pmI0E4^Pbx|(%H~G&*VFcN!)oM>j;hbWDPTD2W(C% zs5F(HESu_dQ=!E>X{~G@)V)%HuI@)l@o)-7cKYIOLQ2Hfj7CqDhH!Ln1SOM2NTg*r z(#mJm_bk+LN!3|UrtpdfaDI7zB&YdcjdQ~~Ku z_tg6LBHhTaeJZ=ted&O{l-rLIUjC7o`AfoFgG>U;z=-CUoK>GX^>HYw?IEMrQZ>D_rY+To( zw8VP=J_#yHYc9Y$+s8&@g4{gsy$&v=E&t4jU+x+oVDV7$D0TcExf}67%0H_9I}hD8 za=T^<@Me7{aN{>)Y{DcVg57g?R90vSZ8{o|9jemVzd87sY!%F-?Ywt0#<>}7MidFC z6c!qQMNyeGSlDf%C$}3arukWb!^pBbB9qYxkj8?PQyy8}>tjHj;Gk+?!(%Ewn?V~J zr!X;Eo?-)?Arr&sA6Eb`vO88n=;^PifoMuWvX=(KhLU1t0BjU+g*&}LzqM#;&l&6I zBioO*IY>azy^l1P+~!rGwx!S+Y|)+Q*oFB-t{f$3MvIDz8RLw?IxEf^d;_%&ZE{&I zZv*YoSdGG#5CR7uG3KjZFKVM^`bq$&Q<`E8n=?SEx&*f%5%0kf)lG-;UzF>J>*t@H zp_~x3pxGS>1*|$pHTtR`c02pura+)k*s`@YvOAS!GXCqrinO#~OFMBr{H-6!y|>>s z0J~ONK?SfHh1CeGwvu&y**YgNFD)|8Y+s^7OmK$l2%%Uy(69eQu{s$jc0Q*?J zeELi-E^j0yPiODr&cqOA*x|c3teTkPIRJP64i-?g{j5bIlc#E_V1apqxyV6 z#NGfAcvT9SV%+jRHKR50=`#i`gW|Tt3uZIaL8JqP1W2ssAV^Pe%p7n^6g1BOd}haU z>mrr9Yb~jdrb=fgB=%6Oo;H54Bl|QlI2g=y6mmlwt<0MobWTne-A|Z9?tpN#vC2ye zIJK$VY+iF@I4U0VpW!sm93xz--f<1Cu8zatd|c~*S-hr0^<|YBbbGr)s^4iI8e#Q4 zr?tM_O};N!^tJ-j-@d!9 zz8j?es@G4)L;3sf{WH1S%JYqGKN!Tm_fKW|*-zwm>YxAKx4r&dAhv{;goq0qi&pvV zAfu$}!A!?;eYup&%az<*k9C45z*w7*Hw$Gl_&{2h?0~@u$3C?Y&QA#uF>+_-2s+J} z!dr?bK!PEX5~+9gYmW0TB7KyMT6zv@Q_!E-#X~xfk;BIEhv2R^o-37MqEKfpfREBTTj<0pka6<5T>EfV@WMK zoxxwS%?#5_&t;lm%n6nyKvx%k>ldAn7PBYMgB!s)uMf9)Gk~NVkRvCJE@8)5uoW1P z!sJCdEg_{Q(xhWF6kykuTAZXjU<_#sj6<%pfyrqEGx*ECu5gx_(W(?YEK4b7hP*wx z;?xtrM+9tWK-<2@1J&Zpyg7c&vjh^>!Ehxx_?iO>Z5{Li@+*BR{5_Lz^eLy>dcb!szBU$ zxS44_0#r8fXy7XDq)^ZfWnTXV8fK2E$ljdlWwbKq3sVgE{&3`dVFUmn$FDhV1Rgm) zLkHJ+uv);klatq{6|_E;FDiKb<@+DVmrtI_+1aJFp3Rs~CMh`6^YiN%wvhIyx1-$} z$gn>>i2W{yk6_(FEHzn&*5IUc(uY{{0~J0ne+oFazEt`>hQFTn_MaiQV8IBA0!8dw zVeZfzY&(ykMc4^y3)g#UCl0=A21p%vyl@g}pHom%`pC3L)VbTvthlAoA-wG8170&8 zz5u{ZD71!-`aEwtJzDIaT`;Io84~vNN!W#r_OpdEI$m(l;UwPgV82*2SN8+w@8;NR zdHypf3nCp=D|rE&b&^vjyHE|uf7yVr)onP1PJh=U=OyNkWiXectPfVv)?L3p0JEJh zWioy2hLNpmi_JgnE;x*7k3vXFq~Kg_zuZB;&;uSBy%Biz(t$msw+bS+({4f(c7m7N8QIKKW!oepg;spg?n;SyVo}}t zeOEWvbsl&jmlbnOO*ST?umOg7P2iXWePee1tg1$!6RP1{1zIBj*XV-%EvnaegDLYG zdfM8eax;OYL^RoA#-(+RvAH%k51CC6+R8v=zO-v4oLfkXQcb-Fo0iF*SmE2r%wP=I zt%lXGoR^Gjx6LZ=T_gAZ8n3HuY%xvaGhR&O1rf<7Va!+iH}#wm$>=p{6X^sdox}C$ zV&^-8SVU)dmFJFos_F(4`-VrvbVB-Z4gl+P6Ra=ra;eu!N6ICz_)8`JCa|#gDIQ_5 z4xA=?e)RL#*0Bm=O=Td9MPL=c>fdJ879M#WDmY!;JCeh<9vFS#!H?dNhd+8(?!Wz( z9KZ99EGvM`b%XQhs{+`11Pd=JH01{b0X!0$IQasg@)~70OQFz#!DstnI#aY2q#`(_ z!VQoY3zh~9X{U0)->`G61xNr6WojF;04Vm+HLiJmZ9;ZM0qoV~l}TqkefCT~_~1kN z_@htj*YjDmE* zgeGxVpmQwo-A!*gWkwhT=BN8Hk-aQPqbcNpMTrYYH97sN&!X@#BGTe^j8Sgx zCyCtj;O4i;FHtMGu!7^Dl$E3u>m$|)bYG)D4A;68*}F&30`rXuKA{aebKlsa>A_@= z+|2q+A^-NK2%SDWHjKVdFd8gqrZ$S*aLo1JH)xO}prNJA$a4xnIJ3T`bjlM8fLPTh zw`LsL^k*a2b)UMvzLBfzE6Wt>@u<_G4N;5(s}fz>{3QGdr=%@FO(Fdw<|6F(V8No% zWeKMpf+-1|(%_RLh%!xEJGRZ`wRO_GzPhOQE5Bb{$oa*YoL^p4Z1q~sPrs6@v(II7 z@=MuW{!)hBr?N~hWHDUS?*x^^<^>jPmqm|x*U^w-I=;cJj`BbLtwDcpi_YlzQP!rn0ST`&BF_0^&+^)K20IGxVj8xl8WD-@ByFO= zg47F}2KpY5U=rqVDVsyBF3jFR2>4rAla3g%J) z&($_PfK)A{aUpXF;BKQcI0&(|T)d%)>0<(%*?19b2bgw55e1*R0mND%fP$mQk`oQ7ChqAt>1zd;5 zjaAvKr0peD0~RU>GXSg|Yi=O6Z`i^qdKWDEu(AM-6O5oK+c^~ijf6Rr7};v`g3h9U z4`(aLTXTzaM%Osuwd}J}1scJgv7wVUZJn=vSvOym)O!5*iM;>I59G7YAIsCHb+f(N zi1R?*3Do1=5y*r?Oz@d--8$e^_V?I)=imJ;(e5ETvl#iEY0rLJOY`ZQc0O8Gb3DVC zm&VD=4V1CCOr)*7=2Rl{W_ANkSm{foMd(pEKels{<4pjxIH!V0Y&bIjpMBAzN6r)K zM&@?y%u)1fIz^!`n|$+k8;}*w89Kg*R9dmG720m`_s$}tt!o$Q1#KMq=(U11j;s9^ zqLeY`N@h0(oMub4Y7{+(8;jL59h@mwvyI*8qFpug^OTeE~pEii`z(wB@fYj#na$(SQo zKf9^Gb)$g#;!19=uI2LT(m=I>*vkrN?cbL-a&d7g*Oy<(bp5#u+mB_P9?SjJYgvyP zg}s4^C^3H%vQJzeyw1G7La_U0x7%?efA772CAV+-fxEZg48;CZ+5G&+^0$TEyT0Yt z1KGw|$=ya>_6&$GqT;-qysFQvs+Tj=0k6Y(yRH+WUbN(#6Rif+XsmbHSs}gJJgn#o z%}6YGBKmivA}~33Y`IE+WpkGuNCf(>;*0qXfK5KDVbuLU2Xg3A#3(R6$FM&$l8j6; z%IX#9eSEUL*CGa40tklhqpzWmfRZkLzD(($;}z~27M#`ud z0e%5l5VrpF^WNH`W_Om{fn7;y@4E{l(}__HO_MsG$Twb3W$Wi$j_B5moc?VxXaIIr zk*tes|bMaF9w4XP!L* zBLK0fa~8sC`(|Y{3q&4NH&`(K9u;ew93jQfX-N9@E$uqveVo4rroicD_AvR9pds06Ty+9#claj|A}HW>uu6PpufibTmg#Pih8WL2kElst4~# zV|~Y+E;RIf?P$^=2H|w0+hG&bihqyZ=s4`6L)g{|s&>BFB3kJ>%Q+&b1>jVV(*Z9! z`5~v%qG!~Xp6qu`bzp)jhCSz9w{BE@K)JBuY6EFC7p8#14&P#76EW5yqN~fd)q3*oH zwlNx&TT_&~sr3eoTR=MG0u$%1e}8W`2P$=xVuhJE+Q0(yQ^!=>NWXmbS{{A;v3&UG z6ZxyZdS9M?^|hRzU3&2*w0QM1jMy=T&N)iDjl|jUSX0dFn6`U=x4Hxas$Gj2 zK!nqc?vsf!T}|)PMx68_Ua7<^M{3`bMW}*ux7}<1T%uBnIAx?T8`!;v_S)=&;xa}p z5GgB;x+4=I-Sv$2yJk1&Rt~8TyQSr-0ZtHWJVVFU0qVuD3X?3tLU~0OpnN5qb&FiZQG!<~_ zSSgUzOsMi^*GrYwI;du4l$)FSTlIJJ_cY?FU{)i-7iVX3etIUaUc9shAt%)#JUM$M zC#SFE{QR^&cW%u?E-zlo?Zq?6(-S#fJ(Y*+mvXdxT}{uWs})EKr9;4RzwkID?E2uw z8-my(|9bda@~__P&EJ0ejX5Q2|QGgh^~r646ykjOg>;!HeJ>=LxS-B!#h)y(`b-832a- zx^1Uj6oBdHG%d5)-awTAAXd*2d@a*uT)2K>@2y9nKz2&PcI)1~D?5zXFvN5KCyNOA z?>8OZHPSqkVQNuPDmri@6Q0@3c_iKW3mS4jVuzcQwC`Zbx{>FZ)V8aFb<%MNF%McZ zffni7OWnzSfrVYK+zO)nC$GtVu7ZAoI?T(m5deIFkxAc1&*`=(YbuElH z_+eas~i<{ z{@%@pRv3jSAhIa&8>u<}tfXeckIinfwN2+%)3GOY^ZnwBFXWfM{6IeW_%nI*=wmrK zxfGRjw(MB(8UQ*Rd)9<>tB9sC6Vrg2lomXR$8IX~y{#MK*lH1IKM7*Qk3;JPMrdpA zOHv^#%oah2Rs1@!q*Gc?;e}4KOW$zk%*R7hVfAWhT(50)t{l*~ny+4#M>%smb>CU7rtoFXU z5g+%gFfjf53L{4D4bH{8b( zDpDi8@1oEaiwa=^RH8Clsg=x#quGKfuvSv_WLVqwONUKn9KkOgLKGp>K&B=r# z#fTl(YOD_t#>IJF)L5p1htK_v2ivB62M{QGN78U_h1FumE?f0$y zz*h$<3O>qM|6Z&YejawmxEF7BF{RmcSy2GE69=5iiFMG~z^ZOwn`vwFY=`!o7Qvl7 zqDxM#IZtKRq_gz5#0#p}k!u=@7i_Jn19Y^O)%`p}k<98m);am6Z z`9ZsGRe(DjEG=3)fqr4?17k+t_R_ez2!Xi0$zgAzV;miuuB~;`L6ufK8d1s)$Y#%2 zl+^4)%%Rf2UEg&p9AaTahU=rvCxpnbPRE$5bP6Z5GgW(~=~(T^_4U^;IBaaSOoX44&fX)UvtbN;2RaU=nk(QL&l>VBlgz7dB)I+@_MX zFC90tWeTjocv$Q?&~#Fp^b|8wEz9Rl{Qg$l2S^IhFUC}5n=}LFLBPh1_4f{lGZ0+) z@4G^uuU@C%V2>)a8(7tdcFsvU`{00;#aE1CvS_ti<}t@IWEZJ*N+&{>J}G?!u?#Tv z4tqEN-KN!35$wAXGO05t3p40imQ5kOq)Belp$Pk%1sOWA@#P-E%!ce~%rb)r3LE!4 z+Beifj3|)SZ#HMQ+LgNS2}RS0(TjZyOK zIMgYeu4I%7UhkjD-+cE%9voiEQn|2sQ?K77%uLXZCy~9e?623|qTye2D~lq3@4df} zi|_NHySLvQ#QvFF{Pf52pVd?On|EKk3!3bao*02_Xk~Vo-kw6VX7=>V=5{3~XSo8Q zWt|vmG!6~Kj>~MCY^jh7Oh`LTUi25m zQ7t0Ks0kx5ts<7+)5!cY!GFe_0Wa?x)yN?Dn`UCcK4O`K5)qXV`mxj0M6&vn7dOj2 znVfh}+X$P`wti?1qb`iDmYLpNi;`iSm`3x6OWV}D3m(_pqig$yrvaT7S>o%QBX>6w z5zZkYxv~SqcYd44ofy%dqly=}%f|>&n`pzCeQqxSRL7+kb7~iyH2S>J$IK{Yg5GnT zBUZJq|0O67=tiMhtk6`|9c;?8^H?>CDl;A@+eUY3a=@FMcdM`L(7<3qs|F*mCmBAV#nhH+z;j8t$W@!Rw)G9GWOnqQ42Iet6W!6jZ`h(GTAJGSEoBLQWlYB zu2pH*rLibGI}x^sfHFnhC?j+GWD{Ph(m5{OoS}UPdP{FJ^8PVIM2>vO(l)pRP>%Je z>I1u(0QTA%8oYS^ssh)i^66(^%BP=xAunH^)XjV&H0dQKj)Pd9#gJQvtZ=MxP*GqN z`%sCo$vmGpul9aQ^AT_oeA3no&^#W82HJYP*a+NYALk}#*c|TW9Ps$~S_>4!IA^ve zZJf|{Z`m0)eM4m85}X(AbJj$Hkx4qJN!l@l%@U7}&if!X0x{kEFdH1v<)Iy4Y9QRF zhktfLhSaaq6!w7>zrHPNn~VKk3&2?wCi>fwo9;|;g}n5mV2#8 zU*KiG87X$|qk|Vdi-1b!YAPDMQ2=R~#OkD3cSv|rivcwNYHPNxAmB}2woa#*UC-SD z8w&bWPEy;fTLIa%0ot4T?_K@3stqVu_9Ct;qcv!x_I7(?>IA0Mu0?XnJr;a-y|!aH zm!sv0{9A9G%Wu7XDThmKZlF|xTLBkk51h7@{%!Vt?@hn%_J0ijk^B#D^v+*#`;9@Y z-hTEIdAC0Nr+1&fyQler=3uC|LJ*6o1dpACR5n4BK)qO1&tWN7H)FksDz~NY(G>52 zEy|@8*JKC2vvh4txzX=zB289}?&)0PWa*qml#mOufxv{c?-1!*yLG9lAmYVN7MQ`s1hiOJJnA@%v9hv|{V;s@_iiKnb3TB8(qcF!i zaC!QcQCvRG9gyvT6pf#8Ur{h{JGnEFLDleq^r=51%4Wp)uze-QT<6A^JqnXaJ4xQ) zfW6PXwJD6fT)7!M&XbNpU>h7FZ{A;B;(A(1T<*ah`KComZ&12FW30Ok zwareXQ}tLzN-gbkb9v3VsqF(B+IwzNJ|RP$GK?M~8JzeDoKB%eFLhQqNAY!obHgmc zwem(4%1u_j^RUWRwXe_T<=|1-gQc~(HL$BRBTeCJF_bC>XggMurWyehvv>5%jG)f( zw}D^ndNribUey9M1HS;Qc|BUnUmFjo{ME98RlP5c4h*bnwDsPP-mZ^t%h5XzjXq#S zTiSK&-k~g1oLp%GdW<;ELC;ISQ|S~7FCuc;u(H2iVQYp&i7WzC5_<_reGWxboOXem zvW|7ou1frtz=@ze7`orhcbp%=JRYQQmD4lbkomPPbnl1SMMia8A|V~je3F}H$T0$Bd-ugP8!s2yAOwNYTJ2WAhjxPy9w_Q!06JPb>V?2aSknDU~N z^aAfz_b0*Lr$;}}F(ReKhhim4*frPNmToMh-^!v_RbdwGyIM(GWit zHiT{eu(v8}D4nk1CK|jpudU-_-Nj*=mDd+qRan&vo_zm?a{VZw6rX zeSh|gKbQag4d4Bpx4#w;``+)#i#k4kz<=Kf-0a;Nil>yv+(4|pMAE)cSlB(TU0GK^ za_Dkfr#kUf;P=2jYy7H$tv;o$J)IwiMe_iDEKRN#Z*)LH!DIqc`!30lEQ+04!gK73 z6S(P<1J-;07G($!+n$R}I{C!66v#}_2SPDULOP?}MZ!yX<|wW~ z6~9-M&VD`Wd00~e=%fZv&Mtk^^V6sy8!~65ZJ^YA^)x06_JFZE5XY)wD>4&A1`{_$ zZXL^*8&Al(nmKWqCDI<3ct?`dz+`cMf8Dd4ugjB z9pAw4xs0Nb_F=z%jP^y^HP(Z_$qN1Zbobd{YIb%1qJ->I{CZ-A8{k zMF9n`WA-AKg?GtP@T%$mg;JR`>Z*6G7(QIe^1iDLEFWsi*dtl!{kYzb6ufH7*p=~Q z6;K->wnuM=oCt1)6pjYsAyY>CL(~Bf5!V}QgS<;0_vTFVJ;J$w`M_}PUL-Nm z3G??S%;6OFsnKXSEBfdwqj+5ZyQy)Ny)t}f8}O@FujJXYujR`pU&+TGeO~{5AunH^ z)=k}`;jvKg8*vO+qYJblH(S!FfTYX_Dglf;n|6+~#CbHc#Vi1#t>X~mKIGQ8BlQkF zfbS5|QwboOGSxjw~4m%A_EAnflWK>t|^=O>}?c#HleLC zI-n^vDqN#UaDRi=anBHqiPrY5Vqt0y+aW;dIpWP+kd z$@0S9$$%F&V8aXB03lJJyPLFN176u=E7|r!D;t6&0wgWK8yU8?2D~-2Gwg*8G^y^I zbEqLRD=Wu0M%?@R{^xw(Ip;=ZcV&&dg{b#3Uc?>#;hgV$XA)%|Q7dnRs{&xSAbwbOM)LwS$Uy~#jU$vKvV+-D*yF9R zW;~vnU{0uKOyLc1+tzgk|Kf&;uV+{EVBEOF3Nn+qUa7p2M(jCslQ(^x_5CTs+Wt)y|@JAlm=E>fi5WujZfqe5+o*4?uNu(f+pS zJ<9vf-k9K7>!Q`mVI3;FI$yIhvi<$%F3IZI9x`BEwuZwFUN=@M3!xBNQ?`4726WuQ za~8IXEl4cUb;pG{cmk<0clc5U5H6Rnp1|O1%WF4ZT3Tbx+C^ghEE85UkYyZhfK~^t z0MHCkFrx(FQDnzT@mkT7@4azW>gO6W_~8B{4Iy~_^{aCA>f7?#{PXthdvbbmjyB){ z>;rft<8QF%WJs39-ZSjSs$5u+fQB+(1!z0GTjl^s@n@wLHl7uH86kwDeJ!>%=H9e9TKG0Ave&uhaO+*5^tVjy?Y zR79b+<`B{Vb0D@J>ji;Lr+gm)zGs9I8;9BZExpu^_sK& zr;=tpX&Eubg_b}uZ;6)j(wfba5+${4WiuSh^B0cf^Upt&&p&r8&s}^puYm{iXZpFP zDq3fobSvO}y5o2>kFmVP=&_@OXiM;^wCn=KzrJ zb5rgKfE-8aFIln|{m_Nkv$7<>fc-ugh@S%B>Bd0^Fx^ZDXI;VS)FTllE;wn?*XF{L zf9pp@B?D~ha2coh@V156^x2{vQNMA;q_GtL%Q)B_#&?|d6`;lDdiy)XkQu`cg^V%o z@jAOHK%aGSVLWjvA61I?s@C((g1#pKb%7r-W)u<<6&z}C*aa51#w?EjkU}q2xcetm zm9Vjo_0bjL3eRG3OXhY`H*d8B4-MNOYtiDm3_Y=X8`HXCVnfnACBQ;3B-KG{uvjzPAC_!XV$TvFC4;?{rP^dyyeR*cIF};9 zB*7rDcvoA~1)Zv_^=bmK9dvHyp9cpDHVJ0?&kjO6nC&3<;DUi!`?tN{fvdjP|9#;A z46FS-=)m>Res(b2Ki3)7{!{mJv}7L zsd6&KUAYZLM+^Gj_H|;HynK>Rt9`Zrp%saF;JkZ42kL5PcKtC1HW|)^>jo=;GLI(H z1f`kl6*so>#tJBiSM7VR_nz;{`N91Ma_iO|xpwWEy!`ST^Ut^BtsC#o@1H0rqoL%% z4}~szK}7(_@{}n*P0iWFUcqK43?B;jZ%IQgCA=&oIJ6GvYH;RM&jsT-?XvETmSnLDd5?kJ3Vp+C65VRh-yp={*J60^KXklM^Fckm9YFF%F$peEh1f0c}c9 zs&`ehZf<%u#RL}^lX>fr<bX~+O^ z&d3gyFd9iruO%J<~m`I&!Tp60QjcnxvIW;SDHrR6;e7P^k3T3?4Vh+WI>{Gz^& zPv(i6fo;#uBtTO^fEYt7kTZb^G*?J=qYmhApR?&zkPX)PLf!f3)Wwafw@$0EEM4$f z;FAs05C(NC#A4FtEO6<%b&Jgl!$-HZRkm=jM1ke3UB%{9feso3)}=AZHnmnbzCox5 z6YGLaLZD}tk2A45i>oyzwvX@N^Tk$=69K5-Tex2+h&rgVxGuXdhDdoV(-Z}%`EQ?> zChq>~VBPDuI)mydV>0!JD4Kb?C?#!(~ZFNrOggi{-8l58v4Nz>?7Be;49LOdmbfxrZ6_WGyYP)eYsf zxq-DrmNKw|gYyad*E(cPsQH`Rq1_8b&xz4GZm2}rC5KNKV*n3^V@b?mcKXp!*Rhw( zHWgpzOT!ze5te-e<@Pur2)52xS+Z>rvems}M<+dl`k`h}-@bWAS=j3{czxrIYjW%M z9l3Y^cvjR!Ca{fTx__#UbsenM^MHgH9iO6J{v2bVvfI%@3 zi~6l#uJc^+9(9fN`|PEidMP&;iMpngwRfqZ_aYcq%`?D#^Y8TFgfOrWA=e)aiF2@P zADtx1H})?v%!8j782BVG^xc9ADB9{flIZ$9MAEK3?}X%5Pa1KB7sOO^OIbux=z|$z z^jPs3!A>K6v0#73G60Zm&Ew8J$xp13gL!4oH6hErQp%Q2-rZ7HdS8|Z4QQxq+vnPn z#CqO|fz(~EPS6Rai586QvjOr~L++Z%&VCNf;h2gy8_^o)VYb zm&5g=d5}-VY3jCo&c}|A24Z`Z?(hD0e=2uB(zo}HPY%TXGkO0HzbF6B{7hf}sUHko zEVSk&iBmfW6e(bh)#@&>y1qPH@S9l(4sy2J%tC%v8WqLUQBDJ)|E3r)xl$ri*0PyR z@HqZ1 zD+p>{8d&Q?HCmzgN&o~vS_T)0>-B*$09M;?5-;t7q>uow_M&2BU0eRNT6SyixWz<` zttQG%%7jmz^+})<0r##9;rR>(p}F;_fS;vT(2d)eL}LuRL(Xadiym3D#?YBnZt-WU zP)51~D-8W+6(@RSkq*GANW=!AyAHL;w;;?zL9Btd3Bmn({fy)q&ctHu0A{2Ga13{+p;}Mqs@0iLhSD_n-@G8Th$kKRyrW$ddP%i{dBe{F(;P z8D1eF00RL@U5vprB4NZH!=+uuo%~&_IsScr#3tg z08cV-Ov{KdQSl`lM!&pmfXE*{>Ub?b>WD3T3VQsA}H9)G2yrGK&uUVcR*pz^N{ zepCJj`Gk&748->1``?j6`6-PL7RO>BY)6McNY`UD=V)U;vXs zuL&~+?e8M2&&Q*$KPnnkCSU=qlN1y?SeBK7IkT+#xq1w#VFkNCi>a#qjEiati&7(G z&X~NB79_076}x#g@fg?<8{dpOYPsd}*!e^76WG!cx5TPZYj;kx#4%Rh;xs7^tq4$y z?}#WpR1tE3_f;6ibJ(1jf>5>{b7DejWdZnztJQd{W*O5O<^dO(0z>#QwZM9_gb`yC z7{=0KN&#R8w6yKlH9zr`SPUL8rksU^a#1+jAr)~+r09E!l^3c*nb zfGG_dfsmf?(-(w?tt3}!0>OgyLm=*B&hU}svGC`F_haC>1WZUuK=Oj>=%A(Dc?v@l z2>cpRy3J2tBZGAuQXtv)L9YO?UMn-F$yuGP?JQ|$a1X4xz`$w&2QxTD!vRegJ6I{R z+kvT;!vaX{jH@P)tt%4524i2lY|)N!1G;SyOitcd!5S$vFVkgV0!3qF8)Ff>f<`=S zVTq_DV+cUFXZX9trCY+R(*u)}AkGt5B@9&a zs4#5A7`UUOBYE`jk-T^Nw%mI6J-PNqFQs)uZoKuj+vdttZyq+rONurCQq%cR@%MBHN zm}G2qS2c#YySQfnrE3Msl-Jc-typ^;8^$U}DKfj)MVJ7d)yNKGiCgl-McJ$dk^pR| z1vsN+{8m_wF)E#yj(BXIa4!E07*~C_7>I3%gJrqRL7F>}D#N}e-h-bwowq^EqcQ*t zCN-?q+_M%SGmR`eY2O<$znTa(vCL&pW*S196f)mMna?%g z_Oz6pzPp|GL9YdSS6+PXfqeN3r}F$W_hr4gKR@pl*kEm=^3jsE+_BNgAadRTSx>s{;n%S16S&{Y?tPRetEXiFU}VIT5HC3@Y#0r zcXwc~0069JhlWTL=+8|^HQ{{~y*0!TQg9^q0HRu{Z8!b}aV1VmnZu$Fe zaSlVo3Qtz1Ol#SB7zUUCXh4_09thMd#P)&91SY`>%OjpP0UAkwRg%&s^QSc+RgsqX z4(4FQBwA7)GT*1hkJ$GsGl-4FP@q^tsZgLsGK0zahT`cqtCoD$Kda!rpfpO{E*FtIg_2{47bR?U7Wp@=fb5E>9H3k0TGl0Xpx;dLEV1xJv;>xQ6%5YqL?!8jT+CDyR^kFAYU7Lq_Y+t_XzJnoQR^N#8v<9dv4_Kvl+$<+plO zgRNZj?36a}V^<`_!P@n^+nUOQj!J#2vG zy0knfLvtp!wckbzEXHu0Vvg7>$0`TWz&x1&?2*R5e(}X~`N|h2dFIm53|o(T)UGwf z&@t+vgdZQo(#wrwyWYy*zw*!J=p&u@3CAZ3V*gl<{^57!Z_mGeNgo3ub zm$9dEYq}^Jzh2y3U?Phgu1f@XGg)1UabGAyvYZwD+F6BRpsUdqUuJAt8n>iXl5!10fbME`#uMD1Ck&b&fDkpF$Ph)rqMxG z@9;DtHggGJr#vN?ABktoW5`?l!p#@}V`L_VNWHO5~N<)h{MnVt0KNWyh zQcIgWumh5akc=Q3u-2HtwVk{0nvLoS%8JjpWEC>IFmOZbUWO~k%YhNFm*|p&&DL6E zuX77>(`lPm3KxJk#*|VaGtrIP$OHrB36(wja~ENkFHaJKz}4c?p8dPpdp;_l6=KB1 zLJX*62ICJWFcg+7HhKG89ZST6>DcVZ7}_XPY3ke+)TF z-sTzKERz(*QOFlEu8f^73we;`f=bwQzLx&;BN-*W&RQ7JRG8Y`tg$_`wU0hII?+U~ z9zyWeTkmQT*Y)e~%pmrz+0G46CMiYXf@wJp$F1t#FjH!!xt=^f-wp8WzrA#P8 z#5WhnJ5W~cyMWx)cx8-@-cxVD;xWs60M?DY68GtHl7d>_Uwd{3tWwa};AOJ0?A3J+4-WMjyA&E= zx^9GlkI+PvB-2I)V8+Sz>fcaaPG8;gp$TGEuFMS3GcBR@?4?KYv(F#NmtH)R&%dyp z?ePcmb&iGPnGM;Lm(Pbf+~2$R{rR6){^(EThac^%Pd+|z5PRkC${TH^jSn_Y|E&)K zvDh~HYj*>$W`Meu^X*E`PY-4gdtv^=MbYdC6)634lYv<6)D&6lJ)4y^RQo9R(oRf0 z5h?C+nxHXrX`;L#rG>?pOK=J9R+E3y494GEoVNtf1^Q#%zWn||&~wcDzXyaF=)G4HcD;Fn z#VJQ2cP-vOPH>Tf@vL-LO=b*9@v$u}lr$?KFc#aPVG$%7?%f{@1cKlO7goh{_%)K@ z70GaM-Acg3zoi6>C38anOC>CrKf9K@eqB>1B@aas>{Y}_hLXTq;OBV_LAoXlgCaf_ z`Wa?_uM9a0A;)?)Utu6}O;!VMtPxL~KalIDo}#K(ig^)!MND8MWp!I>&h4kB;T;-FtHD<{i0u^@d!VU)Qg{CAV(h zmG|yHl1IHBY~NQ{*AvcN@_iI50ag884IFlP*yZ|W6I~z;WDv+8RH!iQp}E_t5wr|h z&Q~{JBXJO8nKJHOOzedbFvga~e|Rsr&Wq5f3s-;0HR8F@eN@wtOr$z$%wcH)M$kZB z{)01{$xw8!mR^G9xG}LzhBmo0O0-TVQylZ*G5pXC&Gj3f)z;KL3u`q{C%BBz%=OKcSd3ZfX6u#R=Wxf%!5$4l{)F zj50XWWZfgFKNXw6*lAPt46}YNV%zy1SR8HFI~qevPn#!JO4MlRI0Mm3^ItB_Yw-|@&{!H1q@%)t zf&H$vMJ-E!SothB8H!o(%DgqiCxMk)Sh?7+o|ES8wai|kl$3s^`DWHl(qs={km+R{ zAsI69)}{p@wllD+0r5Jk>HI;npoMqLd7TsWOKozv3Z2++Xz^ld*stGv29*lXdYh!VHVvd2@#CY0< zc2;l^(~FJh$tr{W^!Q!WCoPPn#ODok0Mv5M2lEyQSG~Fd2RC>c6m>Qt79JM>KSv)P zV#L359WStvAJ!_mXgRtaUC+(-W)CAcJw25>@86X>ckaqtZ{3vFUb`x9ym4Kw-*`vf zd;ftPoo?lPG$6T4Ah8KE#1U6rS&X0VThX}x?vXk?*vRJa5Z^V1tH+62=#{N;GqOXk zXoUQWXyW7uNVWnL#_yGxq31dD(Xb&4*=3+HN$ha zC>x^5D^%n-+k9UnVO&l&B$kaNBPM+w8y34>2e{Ue(aJY+zxh~TEfdT|A{^JfdDyg% z6%x6fiRHMGy|nTL_&2Hvw8=e!EPcs_JS&htd}%G)iJ=c$tr(yN$TYcFoV*h#N@8Ms z3hrav<)%sEIWh&|g|;5)>5cq?e!gDiqu-NEXHuv8a{0oMeCZ2Y`RYrfg4t&-o=O={ z6~J~GHlf7FlZ@S`KA!O1`QHAQ)!&l;?PHwviN_}nV*Bw2-;sOs0Q~mTf8)s@HiWJz zubx?OG52VtdyH;z({rx4D!`6e zq%hj5cp$yS$4sfbi}>^8@t$C*RRFde%d}?f6>8B5Tp=1K1rSBemLi3MB^3(#8o~pN zo|a7MMXT55%EzQzv1@k`E!RVVFyq*cNkHfLOWs4U1z?PtwV*ZO!K~*19Tf69F8(~l zfmVOs5%Itils}92d!h-Mu{MGAxBFpF^&B6@SY8TUfnssPX-F}h4u%VAZi2-l&?q$& z6h6;p-{5@sohmcV+vAhwxAr`)E`;*KWIxrd&J2}WWJw21VcOS@|_wLEH>u<|D@7$I* zuilWiXE1yJ!I3;V*-1}h=?+v@6EP;13ApeW2xTzK9HsmKtrxsj@AqK21PWCf%LDb(T^LezA&75?a5p_fJDuTN0EJv;jGIxvo6b2 z9EL6-0bnwlRdl9^x804tHd!Nn@XU#J?dl0#mFdD% zdGgbbeLe6&$NcgC@&|t^Z+@sRJ;m{$r!Evg_k z^CTU}_Uur$=kw3qM$XUs=h5O#GwYgV=Z{v{UPB6cX`G(vschUXgxTJvWJeBKr@1S~ zl-vatm_01Yv5+wgxAi2ENo*V4g0YPq7y+auS(^pL-9;_Zq@Z*-qCj*%%$03mAP3Y_ z5Uh~kwf}r{+0dqTMgmo0x906R%`pBz9AV&c01E^P??f#1eVJ&L;rg9pxD9-2Go0XP2k_pY!<&)87oMN0^nM_{xL#^ zs>OTvSJBtXKBs_xrQVuWty~V9G3ZYrW9-`7$IliV0PKQkMkoS6kujjgt_p!Ym_Ca= zr2!?v>~{~-0E5gX6^t)7JKgTd4J-aXg%dQ6TZwQ2T@xj-tEh`vQIyt8=m`M1mqEhn zEKvqxLQ3?Yv+76%ZvvP9?4tp+?3bfxF)Z?eofv6 zh`n{|t{k6@ayHs?@4&uU`MA7BFwA0LP?=cT3%UlxJpeN8LKeDh1eR2404<4qpTllE z@oNx|iM4w$`96^mlVD1fNKX|oDw+C)e1ZIFX*Yvi_4mo-jZsM^auFamnlW%pip{h* z_YXt0+|}L!Bxe@Z;@L40_`AtHG-8;-<*&&`+OQ_}Dc4Sf0Wf|tJd8sPBU(VH@LUE| zqnoAWbs!0;%l6&mq)wI-YLGhHE9(8T8r^vQCdUgV9{Z^(aj<;n+t_QM{Z3W)7D`u*?7=jJ>Ax1a7CPXS_^DX@x!2UBJT^JbU< z?D<+w&n-^27rIx4)q!R;5Zl4rZfBsj0}%r)mZ%dwu$h4m*E0;CsnPWrmYc_q;=YH0 z0Cr`~2@r$n*`~S~hBz--Qv$n2@KMD=52%n#Il*W*Fi<54lY(TaQI&IqJs9KIg+T{* zn+_IN8y{u6CWOi!h@~5IwTdu;E^xNHGtVfcCCPF@0bU`rD}`pT5$;9>TiM-!R;=E} zE^5dnflF69nQLX32LGJNaUzCbGI?mQgEwt}E65cPKSj69009!$uZWEs2>eo+98*Bd z8Wk`+a6bVXMD}fj1F%y}zwhdxgaR6$0kdnuwFXfMHjYVgI&Zav-B6j;jH^0@KrkC| ze-0v>EQ-?agV>?OGjnWnvY4-1#{L$9-r`Je23`eVlFw7U3zZ`ibM?aoE_`30Y>I_> zRCW2T0A%ewt!Fl{SUK9$mUMqZGy>s!;<3l>=J0hJRWD$C4HefMG*riuEY|=oG{o@P zVn5Xnw6ZP-HJzVhQ$6$n@5=zDU9kVDfC*f~V4>CK#!BNfMQDy=RX0eAYaL&AsYILb zTHK5D9yNFbm@|O=#}b4JU?CH>v#dQo-^#ULkC`a{i*fqz-_LhZ%c5J;#ajK z;i|g)Td$+t>`F`fb>MZba>U9UE)$px;E*w<3=6mW5{#jZF7%8>*LQ=!0aP+b>0)jv z{{rlr&2bIgAjQFG&e!1HbD2O*P?1mw*|!DgYTC?Z7dZyK&S^=p-6B3uIkBr$CU^>d zHrIPvZx#tjn*2APpW|osURhF%IVMrci^K?X1{xYM^kYEFalddX)J^@%I?TPaT4hj`S`90ZwvH7L}<7C1l5=m7E>J?#fqRYVz%GUY4KLgf6hK9v5AN z-od0wKh~kdkjOt?eOdm`Pv(rv<5K~#{kZZSxi*i*e<42^%%|fiK&%9)vHEL$G0mWL zI*_y7T8>ZqMp!F5J2hiL3np5N*Q0f;XDiah47V&7BJ1(>jifHF)M_usCH6xL6N6EW za{#EcXrsz!Vb;82oGAng1suQY8;C+uEqxQl1ai^vegvO-W4pJivZG!xu;EozfY)7=w)!2q3_`u z1K6cd1W^OEN(M^yAp_ec?AgMQEhz_g;R z(E`{D^q!@`vUAoQ%8ua`U`K-CroyeRd=&H!RQOYX!UR3DyFZVIGjI7ljQf~CbKZsy zmiQgMm)AVQrb*CvB(wMb2QYlsf#w}Qnjs{r5FqlF1vC{yK0S_I52 zfra@J%%#*w9IGfB7YYv5G5B}N3C=$Zyx!4`wLPs#hJjA+8peCAR)wIfqE&3HO@x#b zR8JsN?TR?&&Y3Jp2F9Ltc_O1tW2WTyX(>*Ry_am1nD_pBmq#twU4FjrRn~&FZc-+L z&bM1RIzCkZd*}W8a^uE3^6Cs=uU>sy-krhgoxAtt?t|l5W;d{g1X00KVi)=o8v_f& zM{G~52w-7|dD4sEGnA7)J|&o{B8w7R22>5WMkrO5J7&;&tfvFj6oTO2*-!AQ7}w0q z2HVW%ju!xInj6aa9qY7?vG|#~0hdGx<9Bn1%nXk8J{qMrpnQx%5Ol^@F2a1+?0y45BOS1PJ-x(m*!rutkZev*g1oD%* zC6GWtMGxUd5Zik`YaCK?Ws!-c$}8xmrSW5AFYy`~Qi~~N@>XUGqIh}4=c>X&y;OJo zID1z7o(mFj!kxmTff0->U0)LlYEQy)BKjHz7h19a9OA5&;B%Tllxje#A%wZ15Roam zq*)Rb1+obo0%i&#CNOo`tg+g3mkIw?0Ma~y)`ZSJP$kon5CKD9m~fVXt|?TqzQOwW zjtpcY;bH^0D)a_SVU*w=%#Cgdk}=2xyY}AX(iyC)BvvxD-{icDzi&({V`qzJ|0DjC z#vb;NS6gsKeFe}ORUdNvuOtOM%v{`ErpA{7rqf5AwLAlegS|b7(``W0bX^Sq(Wv`vP?Jfx1+^e zS`A8z|LzRz(F|bk-G3zSzWbhBz5ezLVBeA(Z{L*n-oGag9v*8;S+2b?iGMB?PiCcD zvy~y-h6WY`s1v#xz%>wPa|tg~|J@CEu(SeshtL5gaSb$50YY0DjXap&SEhHg+Rrp} zSnG9I6|ZDqujE}Q47rJ_D`C(hRE9DXPr~kb{n&M#G|r%;6A4JP%B^e*8BZZgYjSy- zmYyw-m`gci=dyi)r^06@bFmi>fsZ&}fY@xzGZe255z8FU z(84PR-A@3kyK)k5U=ZODwz54mPMJ9I(p=uuM1PKG%!4+yeXY#k^}Kgj+n&xVeg60P zEqV5tBl)?n4f6G`t>vXJ6gk|S%+GW#)&X>KuP=#*rCIsp{ELi#!&EWNHdnjk;hx6oJoF{T47Dr@@vUkPk zWnQ#h8>soP*hH#p96rz0|?oF}+JjMUmylMp<3-#$2%!jrX~R(CgRj zAXa0f5a+2GvIr*M2b5$`fwE+3``ZDzFDxXVRd+NCm z?!#-8V$~vQ#4+laBtynn;1z(gg&q+oGI`6`+*I?mkZl7j@yA{!IdYHaK3}{i!UFF$ z7Ex}F%$lTmR~TF(5zJ*~TFYUNWX3A~?Gs*_$96OM)Tu)exP8fpiMD3$AnWw(RPH@^ zD7S9kk!#o9mLI+RioE>e*W~p#uFIXf4-~-mTCg1qcD9_+3Ta6IG{hJ&Bijh%8x0F$YW$P#D$>0oJdQUB#yC-f4? zlbFLEBF9E01TAUB2-`335(rd-Sz^CL6`ZBK^1Kcw@5315K`0&N{hGEc)x_1&lDhJ_ zb%Y!96+)u>TGDY38A_v`Ap$I|Wz7lfb0mKzGL%SC&IiSE_T~v?z01UL@$8G|-jh2^ zfEWpKCKmB*t@%1CzMe3jMIM{y3}!cEViV90a9*8peG&i+-44(`A>nTf13-UH)^^qm zgdL1Y8{2N|oP&kk&LH;u^npyfyYl5P4f5@8JtJTFYL(ADFLJQz0JbNbF*Z3(UD{Va z>G&W38y}P_KloGmf1mQxANu%ILF^T)8TrX4rN;pkjmPSZSjXPMYj;Ub zP7me0Cwgsr`7J8a8GxOg+vc1a3h_6oEPUNy(IKu*H}Gx*2n#DRgH$GXjSv#Pi2K5w z|80P|mqNpaqa{iAz=Y&5xHn7*ul<^q;9-lI7X1YGeMRgO#htnL>EbY3nFW_eG$gQC zQYx$=ZA!`}Q^;h4eZPX)7}@e}B4D-6?nzaoSCjG~NFY;83!XyIGB06u;^iA1n}e1) z@5z*2tQ8gh&eaeQNfM?&SBOiic%CfKCzODx49#$55}{~VI8QTKu4_NFS`&LiyBDh)yO230Eqlz^Qj_`+)r2n6AIR~ka6|*;3lTQ zgirwsn=Q*uV~kC#;^7TaI1XgEn^*;cDhzYbf*myE24j9pA^S>z%RN&)QH#G%zNgB9 zQTO`Ld?tcKCYuQ-9hXT=Y+mC8W|TVsAsPLAcyd+*D;y&KmITwi|mb$R=pn{w;Ddvf>gLpeF$stneEnriWX z7AAxH!*y4TL&sR~$@}M40}JR6a2JQt=LyItnNtEFSO{|2@i4*&$e^vXO|WH`*C92h zV4^HKS(PHr1T!OjAH;IE()-7xHv)G))dX?VJ%?+3s-Tpbolt=h zDl?3zNaC{OWHe+ul$02&#}?9KG{T~=p1+iPL$-L`T zB}QxSY+;dSvw%L9%a?ZYOW(XCzw*m3$QNI1^3442p`6XnZFwDqr6z@%?@8nKk&ff_ zB)@az&*bcrI^q8D>4I3R8T(E7P&H%Y2*DeItQZ=x1z0U0zJt#;4RZ9b$myevRxH@< zR?3v^+7`Wsn|IF`Gzi-^Pj`P=d}_1F-<@g_la}XNRgaO?g}%?k5|jeGF>}3x76mQ> zAXAE4yd#<{7>E(8N4i4%jtAYha61o?;i+!Dff?LYsBhf+g2B-Z(62r2+~V!w4Potv zwW{%<&d!xM4YPS0K-I)V29s8k!{{(H2!JdRD%>fBmNS)2v+_9ntKDA~3g`nJdCFjH z3S65dffXv^f+ho5ShJoa*=@3VRNVcVP#$`nRNL2$lK~+TgT^{pPj&`0Ovn&5P`?5L zu46PSNI6YV7Dg0I`L$g$;#tc4Ko2258){#jk>Rw<{9b@%9=D+ z06NCQEkCCWB~Dg`h`DC?KxFC@K*irR!9oQs=Ez-+!G;Ea1=yoOL(NoBW=e{Kc>&X; zHBOPW-*C?c5^cXVf5m`!+&Ed-n&LSb_d3Mb`gl^{U@u0(85px{XQFv3M$DJSqdSxGW(EcwJZNQH(-p5cC%5^?Q~LMXM;W~kKw0;S+T z+p8;oMwKB62s(hRF{ds|C}pQ8RcKyB)Y(-*Sh24K?RS~CfmMR>A>gAt=Q=hJ%W`bE zKHTN>Zg&06e31-b?UcpWxf%6rTp}(LTyI3GgK)|j8s|xTm>=(i$-lt>R1aEX?RXJd z0X#OR(c!+~J^r>JwQB_mmvya_ya)Nv3Yta4C6LiK+uMoBn`IvVrZ;PTP2 z+ZyKBX$Zm2Ys=R0R4yOveO>@#?;LY3Jtn3Dm%gcX zDJGWvwe=o6Q`9!(ZbU}sGstAxX0CcPDDW4RDE1Wrg%C&zm}h`$8jTE{ku=Us7<1;j zNHC`S+)$(A7XYB{Y~BO*)YzCEZOkh}2BOVx^_sS}54j)3V_XG_*FHypOV+i(<=w(SIvOj~Orb8>Pjr!z=>AB_xp zH?Egoc~!#+ZqM&Me0V$?ZU#;)dd~%rfygcq`tSW7O)Mg@#uRJ8zNE6Xc-WQ2troJD zf`%@vWDt9jMqB@f?#=$1f7i+bNZj1-wzXO%piB^)>YOUHr$?X=>>995%1an)8fB7b z@I|({cO|UN#dQBUa$1_nGy+F!H)}w{uDe2PH{8@-Yc{L`8k0)=loV%K5Ej*Th_P%g z3)|RSKJs}hOm~9pf|rh)AjX|Q%NCjDLMZcyR|BJORV0eD>N>X2(+;RO&e$Jb!9vA7 z5;>cweC;lG0{w=T*18sIlqysCEQ9L@7a90jcmc5I$|OU_j|@#~J5OqCy^6w!n!UTp z4U0nKL5ziF7#_nHlBq7_8?*~2=4mycp^?^{Z~T}htQ&a+oy_r=@B938t1NoopW}SJ zq5pB#H|u&XpSv{4&wo|qSAY4k{KD7Qa$)m8HgzY1?ym}rkm7>1rtS)eb}zQ$X%5%F zJx4&k^MgN;8z258Pk(&+AoicijqiU~{)73~FMgc!7;q`J**QqCvea6f4(7?bFi+wo zIXj<$Y+K1V1Ke#d@{78xV@izDIHP&7o^{v%G$IbjVqeYpns8omm$qj#iZ~NRMbDTX zk6n^rINWs)z#GNti+6)zk!<^3NlHeXavHHFEcGmNiNG8?~^4BMak=AmT2h?F+gO=F;XG?5K$rce_h?5B6*syHAW2!CI`5i1*CE(BFf; zo=`DegM73}!b#m@}r8@uk5gHve0=^x5vynNwdc`F9H+;CAJ7zTFChWXzSVT9(psm z2a7DOCWv}Tq~hz`rasCh{iOI-jzvlLyZ;JsQam2i;w}L&y?DJi!^wEm5CS+KjH_B; zR44r-0O%#D@QCpY7kBkFZgMfO6PZQE>54PA>{LZC&U5>9VPM!i(y)p(>e9&pt0fdq zQw}8{gFEP4bzQZaSob^a1%jc1#9$bPT;CedHZIqT%_sF$?78C46FLJz97~4hi7M!zN#ON#t9tg!WDmlfBb58dZ#?u^*&*cBx z{D%CuAN~X%?)cO}Y(HiY+xr3josV}Gx@npN89_`=@8qRnx8<^&&H(o4WVX(xm0JDX zecQ%@mUyGC@QGCaJb66;TEXYmGVfO!n+ld`5tA9NhztrT+*rIhYJ>Kp?FqOL20UK? z%i_u;l$kxFy5IGA2_#tV1I-m`2d?vLz0r6a29-0D@o^9viq8aPDhHyM;rT6F*<1G3 zV4+PO20-OA5jbTf&%>;$U0eJ-Hg~D?3GT0rta!jg3V>B*8t4nMJ60hyPN?f^RS8BE_ik`8 zxsoq@e(`nVv9AEOmBTDFvW)p4dtz~6T#6y+<1M!?!2AKkl1)#p)kWos1<0$>R@S?9 zNq|;pwO}9#wknuV0!OfwW$?8p?AVmRB(GT#2#h$>>hsFFuge6n0FzS4$PB{f2mB7( zSoVD-HGue{>setuZAtf6mY?T+G%$f2w^j8rVGh1Io*Zk5>nuY%9yeqc@*{0IdoQs* zX+1rD0Fd_|>K&+Vm}}{y-Q=uc2egllj^$_;w6|`(CvU&|t^(LU|KXp@)oVB8op)}@ z@#&d@jqXL=5wzjOJ;`f&r2#OG6Ze+Q9PSP*YbqvRVT?EIO_`?jd`qz2$8N0*?pi#5 zAjvZaJ*mpfej+dT%o4a(pL!0wZL6lNoySCuEFnmP6+01_gdP7Rkq%~%N617j+r;}A zhP>7a_IXU{LieP(x7wvlt_^V6z=Y;yl8FkKMO z)9-LEE?adkm`q^<3<6lI`GdN^1a-cQEg`F-noVwmumIm)FE{~P%5a=843HQnCLi-} z8cu*JJsd0keMBwXsINXcUdU{@t?OJW4N0qI2(aF)i3bY^kbg1{hiVhisK6l^u z@N>_I{L(KT%CCI;XXI;N-N>`g_E^|snRe&#*~GZ|y!-b2I3TuN`N5ya|MTHb@ZpZn z1jJtX2lDDX2j7zqOz`??N0G#Df^9N4h9^u62eUO&C`ak$DPm8cT3 zok;;@Mm@?!5%`9pu##vJtrE49iUCpkn+fI#*W?|s(ygYs!UrXv(7fT&S0!6bumiUt_s z%7A&k(qgRV8*&O{RhzW1|K=2aOS~ zg%skqLv@`UyvQ74+XKa!Z)-F%ukjgTZRT~pSxs{BqR5xNDDrE+`rHg;FUt#`+sTDZ zZ+dW~p%eD?Vmh@j%!o7C2?OPr%jH}>I_Z?P5NRX{r>q58qRm;%EGQ;bz6(Q8N?nO zY64GR)aSdcp4nqSvjXY&M60W%5-4f0yZZ?iAR7hPR|8;52@5f5#0fT>cxEddeo-EZ z%n;mRE_9b0#r`%psKkJR(G#y>8cHtW!?G}6GI=QeO)4QKDqeex#iBLe+L9~SuGg*+ z2?KPCO#tr{px%pnPq}yu6F#pBL?+MXbfZCJDTEw&OB&O{6YFc|1-MPnY5AFkfc|kz zz*EvLHz23s6RT*MN)~2EKn!bMV6<4dHCqpKoobj#N%zj%lbrCVqbBSOH6-_$D{I*U zL5=rK_SAkK5?EUtjmj|QR%l$NwdD+?WI##amB1$dE(ws~y_OW^WDmGxWCyaO_PNRj z@a0YTQjzckkVbloYn7ms8-NDt5TH@^UVa}`HWf<0CIcyfPKE{r&<^I&uh(y@#{Gw> z9?Ef{MAJIp-}aAM!JDa;pVB$<9Pztk{vu6zTUdrxlNx-GB0 z_L{u@`s;H2%^Py(y}NSS6TNmO!#XC|X+;XTMIRkM*US)(j|7?TzOUNkA5B+f*Ae2d2=|Y$lWkl;HvV>9SFv z3$fQ3`sa-46AYkW$H_!5>)tYns_pIF0i*$wbL}~SEqH&>wf)&T`BPRfaH<&?GR?ZQ zbZ_h^2kTLu`<%$neO2Ud{q@h!Ki6{c!pRJ9kLSOiEAY!ziwI&}&xkDKi_N(_)q(N- z7puP^|MKa+@$rt&1jP2^556P!=7;<3k8_Szn9fo(&eZ~tD>`${o25?LM}0f4}RJ)do4u%>8fr!4CX`qp*hrD4on35Ar-Wg!y{ z+4F<9+ii`4ft$3bh3a6a+0|K(aZ%hJQbS#x4esp}KSYLDI0_!R5K5DPdOe^`Uw@{8 z1S-8w6rX|92}S@SJ%^RYLfUzg5EqdpQ^I5nf-ittF!Ao@p>fgD23}+a!+3x-UuoeB z8Yb6rV+Mu|&fOAajCM&zL2a(qXx|mH>jsfldKJxUd#( zP!ShWMa13E`U7tH5cE`X7%PHUtkG7?{Vdp<6M$&A@r%c}(i^}y(+ObPi@tl@Dpo*Y zoG%Oksm>NsVUdjxj0257-%}JIWQzUz+%p86LjuM>9D99rE6N0Ev%#w`{jzhn9j2AsQ3P!gxcs=bUw9e%I z{rhtJ_Iq;u+FNq<>NWZC%P-4~`S%WB``=9r*nDb{{uThlBnx8vgmJhAsst?Hh&9ir zA-B&F_Lx|Vapf7t(T!@?$3?uqd^eW09cW=~SB4KTv5Ow4A~{}`V@bJ^Wxq)N93%wS z+|(q;r<%;gCL>s%CI_`9hmEFSMFq}T4r5}&fZdL@J(r%N$ed5tF06dgQk*Vy8ODqu zb}9q2P7+JVq6{+jvk8sKkXn>M=m0pP^CjA=IzYwTI-!8`sXqzl3!~Cr8a5zEVU3~p zABc!^Qf7^$DBowrwFt!qz-j+!zZqsgx#;&0yjEs-n zM)8HoLMVJlM9dJE5dQsHIe{W?aMr)Z3G@13#e9s5#bNShyo#g9Zuk!70U6x<| z#Y^)1<@0&poX(?gW<(6szV3OI%$(*o9LpPovh{k_s?b8^{o9v2!IhWq868VzCD`(&8hUR z6%DQX`l8ZuTm3cd^zYh(c-ne^cHjJL1K0(I6Fg8_z2+>zTtBQW9BT0)Q;`Z#W0wGe zcM23D&TRl-wZ#m8+D|)}br|Bn#-g(g>-B*GtDfk^*LqlE^O_PccZ31@^&+UeIuk&p zP-rO#Hq}IT$dV;ksMrLa?Q;Q;3|Jvb4s0zC3*DGJxk8_+3{dvs2}0olT=v3VB)Jfz zO)MZ-Eump#>{rB0QJ@kHvi0Jm@kpaG^Ip?LERK)0d}#|9i?VWnWmOjjR}>-?h-+NN z%82DKkXi+a1fwuMjZ;ier5vvoUpHk%g|E#p0Am?{8wiF%HWbx}c^gpNy+7XyK(k2* zWfl{sf-)weBF?h{+}f4NpslO^U$1k@0eY;n(XqY%J&$JJ0J$wm@x;0xzoNvAmT{rKsjn2=wonSmQ8R4_l zd!P5oqX!S=(W4_ddT=ba-nl8)uicOv*Kf!xFTW~R-@Gah?>&#( z?;mLrJH3RlQfNeoKZWc0X5-==FfnXHK^$NL4klUxU{%uYxWrIG23f_CVsWu3X~>RU zWM-{&m*pAQcuxM)jm|xndk)Q^p`2T*#yTsk$_D^TfJwJAV(5z*L5=IRP2pw1K43x? zliK#jm0*`fC(j=Y1lF1s;r7`E(I)CBiD!b zFqFP7ld;1!q2iw?j`Fww8^0lw5!%D#n0A1=r5b64H#umk+QRj6WEk0VUaJn1&I}_VM+9|Aj5|X(p zd(LM7YX$`8l}fbuI29xtSRPNtbmB|5$$X<5#p6>|1Z&)H<&O8a|0{+v}MQw8Rvc7P)HiUW$$~)>5}m4CHoSOTGE*$^gd)0G%D6m zjDuE?$#ReDLs|;mhy+-xP9hFUvr%ayUi>UFEhr@qgJ{LjC1#SjAwGVZt_dtT*074+EpFWQeBJG|(UFcz<2ww?B)7E*_sPh@Bto``?w%&AlN~FJ2SO>d0)dl{a4V%UD|@a#d%O~{!9wuXqQEs35fs_r_lJf6 zrC4Qwar9DyzEGzULcj376T6J~?4goT$69V=DMJBQWUj6T1CDx5!~-&Z+Q?>jC>p_g z7SsXWY%YRv=lmOBh%fW9uoOYv}g#f-S9#as)e5IcJ3-vLY-tUP|HvvO8Z8 z151F&(1xrevJ9~Q8KdR(#T>9YgX!N7hDZE5q1=~79E~-8e*j=D!Z`8iz6(N6&Zx9oYC-d+# zA^YKJYfE`4SZlUDt6xFcaqhn={`B_Q#3c3~2_V}NstZpPZYJrQ$rI*swE}vx$Q(0y z%R4*@N?0L5m=z3sq6$lt8HkNUGCbo%fM?G_l0-^)FG}8b{>=Dy;PG_L^xjVaSOld8 zVgXwDvr(rM7)JppnK!V%Pra2hRwl%k$4M|4XTn;~l_l6l-5X=b6V^*XJTT&tX>eFs zwkcTj5nc_-4UI3 zhx`*_fDd{6)527o%XI5=L%%qZ8R7Ic#g7MedBk* zad_c^vYrPA2WCl4Y^b1jE0(AbnUB)AIU>8!I;|jfuSHodvaVjAQi_KGFlJ}TGOU$d zwtn9$Sjf?7q`{@mxK9EJGBRbcapT_;+>sXMN6nx&XrQSsziabELBDcqJI5B;zisTt zloV&*pXXv|K1TKe>GX z@i60tb+rnFrq&14wPFHDnPtq0vRo)5R6uuTWy$*~7~kUK1y-(wn`}hwW4iG4^AL4n7AB3bwdWRXpCa2J2lA%LYZpd!-U+P_hE0?d&5#p-)7GfdnLB z8^DCSKxo%Er#)vCSAWHx3H9j&6Zix`n8Y{Nsz7v?<)`!9b58_J4RGoR%t_i5|GIy5 zSb+rw0vVCw|8eeZ{-?wm1f7B-U@v64LG%98=h;uU+3o+t5*I57TrNf~CX=#{ zKr0OMy}jMim+`ztQgE-^bA)7MNwhRi>5N8pNLphh?yW@_6kF#Ku+u}R-Wh~f1u2wu zi+Cd(3qRm}bN`n#Y1lFD-GV1!=;c8tdH$IqU;B#4-~O8~$S;5E8F}W?u^g_?XRy8% zvQP|T63Y54it}hLhxKWW-D;Hot!S}x>zcT;&s~_nE4PvcK zX@Hx8*ba8gBA>K`Q*XUCPHVY&XOahxddHbFF(zoG%)$Bj&R7X#?`m7c(=*wg?~JAB z7ImlJv4Px;Gher}ubYF7v#^<=<+|&OQU!rEkM<2$(Pq!>xK=#Qv?awG| z`SmjZhO5fP0UM#qssT)mXoD-P#fOA1)?&mRJf?WbJ}RiJC9=!`VoiRfINCjM$PK3F zf=~$70_^5AF2H9nwJAhHxJg%FG8v_HY*1a%%+R!BV2 zkl_O!>)T*Vb;UPYIj^xzKG$@`3cklA7Y;DH{E)4Uw##JZ`*RP%daw>706aacfXdoC zq|taP9vklL8t{XTX{~V%-VdIOmtvI+&U0mw!UBXW=bc8uegl(ih-uqjhC!rz&{J1& z#@O%Kt4zcgX5)b_LEMeLH>C6W*_?F?7B`IEbbk`yc)5SEN4XwpBo1Z_di~O7tZ7&0 z9o+W)+(W01j*jK#yEo;nx8IRhUweH9vDf9*S6`DGZ@oPO+k0|!YDq(iZH7S&nX}%z zePZYUz}iqP`3=U2yM7l>n?ODOlEVp@Bd6g6HD!(KqMT?^EZAr+-vNeCXJ0p)F0a=x z@=;?G^PUryGC-sJigP}$)pIN?yP1&y@pFHn8QgIw@9YsPQuzbS`I9!xN}u;B6(OY{Ae65x2B z9GYq$=KK6i{wew1-OR}nSz-;f0PBnS6Rs7(7HPf)t@9RRpqsB*-bK_P&oy=3j0Tay zxy);*I6$|mAGvf!@#nP=0wjV@>C$oMi(hT5bZ3YKQI2xqK;%oGFY=9Vi2VAmJ+Ec8 z4%d(7HFU0etk4K(2puO881-8gMx+ykPkGFr{FkfWl7IfxU-*QM&m6?|+WIGqPy{2d4!S{8(mX512GG+5p_c!i;L2GjE(5vO=uw5$}MoCQ$P%G zTSbUKt_RDp5&iB2cc5nAGmdwAZY)esFk<4thyeY`N}+hLyyV!jDMAKd1l=31a>BJ&%g*`Y(_A)R@y6s7Fl|NWACb!#@8OLM705ohuuv@ z9ar(h00X$e)Wjl{&@?3y{fuc#*R0i9B^peOiw#D}JPmXs^AZO3FxLSKc}XD!M?gpr z8J)h85wwV3<(@;>M~t8Z?Zig}U{f8?_HmEhTZ_sAK0K5MP5hZ6CVY7kGf>0*2M-_0oxAVLn{Qm5UvDY{d;QuqxjF;c_us#( z%?^5t%c)}Z7GwSlTu~nPtm&$Wvl>poj3xpaA@s2EBxnuhHRf%o9-*gGXL|wwq>_av zD<%#aqE&WL7WP0HjnfoAv^E@^p3^OEwBQ8F|=#qk@J+RS53ARbQbA9WZ z8~LSgUYLRG1$p7wt*pwU`4c7c-89uEB(7ms^Dkt|b7?tYu7>y+46PE&=!)#Nzpc!zW8H z8f>h@{pI&L-o91{vv3cEYZlct7F19^GNZ%W*<()kDWeRC38;Ok#(HyxTLJ{dXAD^O zQ?aRt=zGkdw@hdC;LISl1Jg$jkLC95Tk_63?`ZXbH)bIF^2;yF&0BZo_wURy^IVl@ z(ZtKO4C}SdQ3pKQHWYxsBD!o2*+pszAo!DJP>zlk2ltxcpzDl@Y>bO7rjV<}sJA`B zs9^nR$ly@iYj&U@cH_W{@@A1>QwSE=Aja+`gHz9Ocr1x4&c}0wv7EyQmf&pPc%o6u zV}y*MF3FxIw(QP~GIapM8pbiaT_O^D4mYr!c4ipQ`fuy;v5apGS!C5ZCaa*+Q0`2- zTZ)H6u!eR2qOLw?ZKEm!#Ebu=F+>oXcS8ny@oQQ9E#%K~ANiJCeAnKzF^_N_f_46V zakT_YIgSLeLH-26wJ%4re&?M%?m4K7bz#OmbA8(4b5C3>B7l;PbWHSO1C$9o|=+?g7K&PZMv)W`p|`3?DpPyg{x?)Xaq zvHkeN-<3!6Nd1R7iW?go}i(Ec0eA7Sq~+t+EWBKxI2)gixE{;#t%OoGPr& zaHWh>#11h@iP@a`c>Uk=hlU!loc*4(99Ff80(G#KxDB|ZGi<$Nlv>W+irzA!9v}*F zQ+B^rO~HVNJp;Qz_4m5FzMznN54kYbsxvrd?Y9)1Z>$1}`}dlVla>1D&o&{$aS&7! zX!iMPl-dH=3ubO1Xb5!Nniqh(VhtY(659~hNq1f1oY zxtetz3Tp~5HK95Iz>f*-90pQcX;`V0(fVDpEDO+%88Qa63)ZW%D|BPf$HmPAYp_Q%>w2$JM%q+l8x2-)BL5K}hgHo+{-e{f4 zWDO=BQ7H)K3k+8!P$)d5%K#@53k5sDUJEP_W!W?fXhNj}#Kh|J7Bk4O4_NS?U_xs~ z;QqZj#t=%xJ8SKO+RW4m*0U#S0MHVqxkj{Kr$TpLVr%$V3D7oZ-hzjvvj0f{8-EM{ zT{!IqWc1lrD5W%77p1NMcQat=;YX|Wft;LeHR1Kvt@q^Z>o?@JS6-1H{>5Kt6@qtf zzAx|Hxhszzoof~};pecn*z~?;y3Ax#10YY0XYMhwy}3tORUZrfQcxOAzmmj1AEn>1 z`rJx!H)#Ky7MZ8FK+KcV*RMLV!k*Fl=!Sby5WA7};S6Niu?zBBV>ufVOFbE~s(fJ^ zWbbO!1Rk~#@$(21lkyml^|*gqX`Y0Oj5Ifl$@bg4R}Yoa#xImp@POq+AbmUnrU4-< zC=bb+=BAs-8d4na`Bcy$2~SjxHwCjzE3xdr(i@X+YjkYlv7Zk3Ru|P94>13_ediC zum9*z<%duH)~9>?rGeNh|FQh2T`YfZ9@uC7%?|>x{Mf>>Vmbidr(GsG!F>U2M{7Mh z?$+|~QISW-JqfD=!UHV2riHM32*PgKIx8o1HNnaZi`lN1LkDtFQLv3!_=pGlFBsTH z{bRz6S+DNq9#H*Yy|StVHGtiOT7)CwFWZ+@Ev5)@!)on->&}BsjWfJZ8(!KRrfl5pWI7 z;`A8zNjJOx?#zA@k8wx~j_yag?@J20vS%=*M-Zuog_`FL)gw%q>FV4oEM6of-5G^z zqQ4=m1J4B;+SiF0IRMB4c8c>R$iC%wCgV~1I_bBr^$ZipqFdf9dsC(r29VEVVkNw* zS8wuEssmAnIV_h|=(yv&+b62Wx>mj=BEhh;Ab2nusDsP3-KL0Rc29@+^%wI`grw84 zw-!tBpZmWr0qpX4Ja#YU(=JB^#G3-^Q{xWF1n*zNLni9u_x96{$GtwbX1uc>=EGdZ zHb4DJJ*=v;us3hslxtVtlsDdZQ(k%HHF^8(cje)OM{?SeUC)D=V`jRRb_0MjCQ5XF zX`Ey;m#57ggd7Yjf9@o=IZSHINo4&FB?=F+)E4)mCkqMZ7SE0SIIQg1J6m|LS!v=c ztBQClYRG096MD^I#noXTXiVOn9I7PDeqRDiezy=5?u4@2WqGn-u57R$eP&i&g?gfrSE6cj&?BMtlQWmmk&k0`~{J}_RELzn}7WU`SMGn zTsl0SKi0{P(McBrrwd5`6eIdUhkpCz)sg((mH$~DKlx{H{N;hz`GLRxpKE7hTKCQu16|16a7F3P z%;xoTuBy3Ew&OO%AB8n%!lLd0*sNr#0~9JNKGu+2MVi&V*=X(8wP)muf>m`1_SU%P z=NgZtp#u|ZCX(SO9!G{MNXqJ3nVDtmAAvbpvS_{79vgE9jeS#8v+`Jz(v&iprW8W3 zn8R?fS7mvtGYk6(S(976;g*SN1OZr=u428*g%M$QFu4@dm}+5y+Z66M0N5<1n9MX_ zPE>>_;2a%jF}pjmR3{hy6=kxltd0mPKIk)oF>YK#SbxUaPR{0zWErKId)VWo2ZT3N zu!$IJP$+DtSTJ7Mb!x^?gD1ggtj7CtK}sfNN*S#QSQYW?^TKXB5^=NK)LfTL=Hed7 zinS8dLJ6_9DvcEd;?S(D)YR~dsxG(|B zyTsgf9cN;-*52p*_rI5Zaq8+ zGRVZ1O*^9uf&_tE?(V!AnWT{nf}V9FlMFKHHu6HcEs>H)krLmJ`}|+$?7gz9>ieqJ zKKx0MG~2%1#LxRb&e>=0wb!ck)mL9th4WnIVCFN}nAgD32)*sW=cjYyuddzrBuMu$ zNji&PWas_8WWr)x?RiyupT3>dT5jCZPFyd__rL$7JpRNJmdy3Uf>)8yy2g!{ zZfrTNATVIucKoKl%tzp{+ zfYZ2NN6k@%fHIw-tNS`c1aDp?U9ILSKfm=L7W&n>pO+;ZxRmA0aJVN(Om-nt;=E0J z;OA22G|Nk7RKeS*yPYy=iLs9*35Z6Aj=*re-S=*2WY6cX`fpaY`S9mhE9GEa^P?N* z5#|?uAR}By)*ttdn5S$Ek!7&Rz#e+) z4T!7=!GJIJKJ2L=Z@cvpUc*pVhIO{`Crl*kvPfwlEcuFHQN$CnTu)jTqJLl&I+Kihne zt$P`0rupgl4}#Wc^J)(yz2ewRMifQ2XW~F1v|n~V6^ymu`83yqDT_4J&M0OEaUhQw zp~=M|>NU7@f@W405ZxF|;C8=cE&P-Kt0HqZ<^)hH#I9Iyg^%`70vs?`&)-XpYcIH5mLGcNJgdSX9yw6Loqr@YMYw2ysp{8zqH2OQ@ z&kW3|091%a49>I`?8%wcfqnMJ&&iWdJt^P)?h{s4>-pzjlsk9!lKy*)YK&26Fqi*S zUa&nrf*eVr6Or`yTDk-}oQGyLA{_!2wgO~G)?^+^SwHhw*N75yV^xp`pzBxH$=0;6 zL$Xqs;W?#}H6;^kwi|U+vkTXh>@%4;L{L_Cqm+;X3~1m&$7ROctc+IZXUzolEyjt@ zJ3wZaXAvxI4_q5Tbw+TSbn7kV6q9N_p=ob~$b=4sT~rZzpjg`kvbc5v^agoF1Dk7- zxd>)8I5HJZr?`zJ2;__{uuQzh*RAFjYvo|3+uslK>m_W+ETwxV=a+YXt7t=vy;X^n z%^3M7B^qGVtj&snBvphZwwmJ-sV55Kv%+zD7mjtnp}e-)%WpmQ-^%_C-|(;b z_{#;c%Z>lu*X#S2VX@{1WnGE)tsRl^-&T_*7PUUYVRF6af|V~WR8mkEAgf^ztrc5g zbhU--1?p^8V_S7Xm(qLOOpMxxme3D|)kkMMS0D_Ncx7w|VmTGN%~g=pBx82rT0m^C zBlIlMN!VzaT1$@@mS7?(HWUXT)j+LkK7HZgTrldZWT2AbbvgK(eI8f^l_n^w>+M66 z17;Z^dYG+r&!Pjc31Zhj_e?C-y)Ywqj|^H*R+s}+k=ZS^mnBeV>Jp00fV#EVFIWs5 z6UqNpy=2*{&TM=SooAEOVZz%c)8zyiJZEE<2XFhU#u05-8*t5l=hnK6XP{4ES*$tO zQxSnA#LKN!B)T~RJED78Wu^<*4`=6mvy#>#>8ZuvBsb5>EO5>>I#mF+&}|itk3RQW zB4j85_PWwS_{LzQ8Eb4tV0fuFTnC2NMu6u|h6U#=e04!J7&uw{7h6SbT?K+*#4cxA z<1;%z#Mxq?zr-w;pmqeCSeT#K-uxWh2Euvm*W(d}%NgIa9lVZ}T;NTv-Rxki(2k~a zW7od`m5Ym=>HKfs)|Rn1<%Q>-m&d>NJ$d}`C*+AIz9%og^y>1rlLgo~PaZ-Kq_@_*F$&okFXU+ylSTL!{gZnBu41@~W@plXN%Sbonn*HC6L7r5I!DS8qLe=k2F;)dPZ9zR#UIi~$mUrS{<7?Oo_s8xl>mJlbv#7IqEDO67Bs#D@Y| zRUPqJdrz)yEwrC>PEzmAFne;EeK|>j*i0tMT$%J0F9Wd>)lvA&grE~Jmb+|xX@s*@ zYq!4HZX)SXI^sft*h!KsoO)bi)T}N9qkfM8Wa)A7S<>+1HGFMUgpm>*x%`9Z9EkL# zMQx2N(_pjwc<)t_&-~(6{_0mgAfNd7Bp=HN@b9)iTPB>gqTO`Z?`<6UhD;g(P!K%rwm0K6Zmt?|y>(0E#!W5s zt$zMit?*)?v_9-Pz-d_~$kse~8U_y$@4W8nZiUV~YMUi$6f!J36c101Wk9%eYmN@(1%(5KYqT)`O70@~%qcb{vaJC&$l^)Kkt)gR z^;+2(z`cZxCQt)l?d*9J>DF_Wj_<{}`!yXPHS9vm>n$bE-8$Z;H55YktB<51ZoY+V zv*dd&`8j|qor(aVE$++N-I~7_ju~VRy#5A29l)TM*l?`USxmr}>-3F|l~c^mXF`%> zqM81ubIMvM@3f`%g}nO8tCq~A@_~Q(FaMQ%?|VP6u3R^7-dRBI9BXdw&fB{GJTb0V za*;_0mdmwJ^5iX(J@Kd4RP!02{s3!Z$H-yW3p#Y+dFk34Id=G6jby@>&!FY?+B!if zc-#z@)MX4C#x)&u@oNpclMK23&Dw?<$b+0<2ez^UuXuMz9oQnfXPuq(O~zDOEKxrV z2ojkSED0;EVDDIUZ^ESJvkzY@oPNo_Q|>@JGtDu*se|3`r?5*>XseTr$ll8J1dj&70CyMIhl1KDG-og zotY>|y;f@kgV@pwz)h`r+Su%&!EVlKq|BfqJIJL9Ote@nBz#MqS|yVv5;Orfm)gB2 zvvV&oFme9&`d?u?_RT%{pxBZzAAG)XbCaC-cFe1frL**ANAJX4{`bDh$NIYMD(23q zW+^D>DsSGrEjO;;lpjCyto-OlKbFUz_?~>{yN}DW&pjuvy>?U1&i1BhOdBQUtXej_ zxWtOrV0N!?lxEZDUtA#33+sq176GWgsRdwbgfrdzuXyc%_3guqH7+P|hkLG=>tx_{ zxv0~w;MJUMtrH8d67r_jN5yhXd*{OV3}q@W_R*#KxFm#`$xW1jRzIln!f)dEiwm2Can-hL<|u(g9f4 z*w>i9!BNys3#(%|wHT5eGw}(&aE+`^l18S)9i)>?)-&gM2kCL9^?C^G-h)S+d1xK` zN(V@s-5*isoYDav1Ip3$%}s1>N7bQq4R|B0AT47ABA#pMaujfJQEOJXpCtfxqHLe? zMY7U3#nim>qsG3DN0E;_AoB5#7x~IpJ|LfY_*kx8yCZeJZOM@S$WSe&y!#-w`BeVz zhOf&1^6sAb?H_;XAXbmZzR|wFJkD>v8Hnwe`0$?lCv&-`4B%tPrK4rF+Pqyn>p@<9 z^<2)+G+Vb?jl4E{JwQ#E8d2^jLu0;Jm4)3S?BX8p>~j@meRNW**WMcI`Efyb2ER2I zD7^LzYZBb*dkkL8GxS_rXtbP3(HnCwxLagq{3ft}{x<`#N8W~3U_;9#0M^>3E}y62 zB!bumYQUJA9mW0*DoY_i!U8q0f|6a;J=uZfugYK{Jgq}G!vig-Ey$#ue5U?{GHGBF zea$^Yo(#$Q8T+#zP<4QSj39hQRwIoIXZj1K6lUCB|o2Bxf|Z7`>Du}O?$G7xiw;)5~7%cnMh^>)S~SS^ATNex`$(vnpTh->Vm zV3qoWLP}Mb)kP2>Ugn&?;;9U{iyY`-nb5+muC2a&_W5{3!;j7p!X(X)$MvziS!Dte zEe$t(TOSfYhhIK^<#k}SZ>kXMOp>vo!;rwM#%uUDed|y*B2sf6H8H!UiCWiRy& z@~XV>!i(~+zVlsUV4q#U>M!V#X4Gv>Sp=$R$`|NZ~ADn{~I8q8HC_N@ZxsSk6rj5xiV4SO2 z9uL+6&Zz@X)@=3tXYL4#;^iZ?J+?K!Kou0uCC{pJTUPvf2x5n@6&dPh0+**rOE|?V zhfmh)Kb%)ded!vquHC}Rj(AV)O0vc0k|l2)OYU~c=SYI_RfL4&Dp}jsBdl(_)qXaX z5u@CTduPek@$Qu48Fx;tojaZb)>br)pxq(R^_PStY8oZ3a_!12pZfV@dHA70zVL2OWiq{*bz+%PuRKK9 z9$BNvdN6%fCogcYDbOf(?Eq1sAg?pOdS->}r-M0XGnl2%g}|&?_s|jUn)d-fbORYj zmWpYs>y9vRc$DkL45s~>JtNn;eod}<%8fF1P#9sB4@2k=+Dx(#y69=-_3+Hwz8j8o zfI)EI$<$h{#D@0`o9_1sI`j1s&n7!|Vf@470c4hLvDe&mQpe}7A+WY2tNj%3<8zJY z*(J!C2H&0osuncYy7{x+bzQRX{z#OXQ)4Rhc(bID9tuXXko_Rr!Hd=xgvk2LcV;;c z0JMHipIeoz3bnsX=nF&cV;37=unE-R`TbCnR`EXDI4XnL1iU_8!I~M`^o^{>u7f$o ze=C!3d|T*Sbe4FQF!Xaihk*b>UT+^(Np*op+ri&o|1WR8wVX2qdV#)ha8ovy362Bd zv|nSEN9rRZJphU%&DHiV9s4SAu9$!P`|?0k^EyRi16|7(rh&bA^OpSRnP=pwr+z5k ze?nyhe}CBeRUJT22NmSl>s)aPu+vqew^HWIIPR# z%o#M}MIfIk_DXl8_(E*ZOsXOii}CRCT4S^Z*W|7u66>)LoXzVoWtqSUHkQddY)sY>ukUp599QE&Te2E=UL!!QteZ8EG3325nFw!MfmmC=S{D{I zWjT+m)K%9Wx^+pctH2xWYWMHqvrG`1Ym)m>3`|&NpJRvnI{(ix?Y}h9c52c|nJ{)O zRki)U9zlb&g|?r2N#}UjePi6M20NlopP zZUnpl+k*rbzkH|h;SYUKhJX7dE_ZQ}+>g;h4;FE0}G6Q@M(5SmOL0qBti;#JgqUpr+ww0GPpmZQ8 z0SGO;V>_X1PFJ15*y47N%vqG!ngOZ_>Ih<|DT{EENflv>+PwiHD;EZje-BW6vGXxt z9mqyzN761vcbCb-siyzbOkiuwFxU(AH3`CeGvACXV^byDG`7|C_sL$qX_ly+Nv4jQ zeu8F(W>~zx8E7*AielhaH6Yl4=9}T%n_p~jZt37K=`-{WZfr)eGPV_rktVtgWH!Mv z1sf^uy{jYzKil~gK#nfc@B-P`STQ23exU!7=@c+jTV-OmS7dv1#o6m|Kq;^q8Tpuvs`75<3s8 zM|fAUEHf4b=xzPYRzl1hM9lFwuCD?h84V=lvM-5$rF&hzUe>Q|=?=w)V;^$qYwVGgv9EhhR^kyYy?qCy=R`yQUk22!Y94BtM!%y4%b7$*J79etiL}*B21` ztbG6bPs!s?JT5sIwvG{k@*YZb^={WoSexmDeLK+D+A(i z;LOcy>o1!Na}OG^X23%0fLa5Ly-N|IYgX~7A~{m7$`VSS0dus(Je_ChW{Rwe=l3AC z;n6kz5Od7uLhkThE4$z7{pa<%u9-|pZyt;zxG!j zkcWPu%11u5lPgENWed`WFoZRl!zjqR4Ptfhf9sF_p*;U?p7fm_cMXXBqd$@7e(&q@ z=<@I$(w|?CMc*8}#;acUMufx)w1C+BrEHc7Imj!ooyeU#Ip&A!`qOb!4&7bFl;N*E^t!3d4$xf%c;wZHoB`S}H|s{oKPX{~m;PG87Y0l+fx zHv#kggxT*m$Of+kjvOK$5kRRF6$0w;|`YSK|{Xs5 z209($*$qgQbz=jU6gZG2Q&vd1*|nfLfdDWxf-whMY;>^M4Az}Y^v}IYtCdhhUYAEayhB_(51m zIQ-I`bEbT5^0HVnUPyDCi{-cg zzM<`zC96Fh^5gIPz?AqW+b_$1`L3Px&w1Q6AXbmx`V99`e2w|C- z8=)Tb{%gk^&H+pft+oyHY*PoXXXn~!N7>YUw7U&=T8jN{ zk5He5%bGK5Src)#Za!pk^{`SZ0OJ#lXK@Q>NlaQV(y}@SR9MJx;bfa!v#}p*4>p}` z3&RC)*FR6!?n;cY9%Ou!W!Y?pkZ*+JAT9qD0Xqa^PmdU4eP-dR*Bd3Z0E(+;4+UVT zV217b1_EfWM4+X#3ffDGWKxx&dlD0>251?rjN9mm5BeVMVQ-pi>aGDcqknmU9#$kY zuh#g$630-Jmg8d10y4oH+hk^aRf5<8XToL$886mfmW(2V##oYp*SQ|aZa8s^M3Ck> z^476f0ai;eD8BhLYXNE3)oSQ>OC$3*N7ovzyD09}#q*C}nFE|ku%#`YNp9W}Oj^M!R+@4NURH$BA~5l+WWsYC0&N1^9#HDfdn65N zGSGgpU%T`3#YsO|0PIWh^i$8s(@#Gwk3arBdFqExFTX=6t!WO%(D7skU{OP}#hj_( zI12RltUy3Xhv{RdeUyYEvxo=e^CtkTaJ1t8zHIg01P=z=HWW};d|5U7RSgCt;SE%1^BYcg0$&V7_kbD6}<6~reO+F^SS zk{<0Z0I{5RQ6L{*CU2m<3JolN-zQ+=_iQbjQiqyf$T868zMq@z7i{@THnw2$wR}8q zBG$=QtOske-^x|u^Er3Z4VGX}b0&8ykinN+OYsLm$6i*OG27;nxnTo98#cjD3%&w+ z=)vrycTHq+V2fy#-P@34$%G_Z1Dr93yX&@Fg#&WXS&ehSC1qU>3LU#f8|qPw0ZlPV zmxE!sCwh<7^K!gJ_H29S2i~WR4Me{9xg+_~7w(e}f8c!CZ%>vB+F4$XN2^&uWhY*p zN#4yvN8;bVTUb~!*z}AC;c5@@PobCO=d$yO!1L`x-Sm8OJtSk+pMyG_ zo^c6ayLVx^0azH|37NCh)9Q7p(N!Qm9WKH{b_W|XL2WbLj<9B7HReq|Ui zj3<)LwpLE+$ZIqLj0U?YZq!st5S@EybtE6~2v#zs1gQ}uK<&rIpF3EFm@j?kjkt*n zx`$)?ZfjEq(8Aki7nYKMxo^l&tuOls2G%uqESybmnkC4X(vw=<1jJ-U*bRuaD7#0r zVvj{KWX*fE+iTehQma~QST06i-`tZAV2iPLCK#QChP-EvdB|9VI%7&-yhpn=3n|@L zhxV~X!pv%eZLB2;r$91+s}aN&H^dSwuD}Usu_5{+EWr@>tkf#`tef^6&0shQ1o48_ z&*j#?u1ovoxng(kj^0MCKr?n!Y?$#o{A>7=&$F!Fl{#gdvuA?@fQ}%Y)dzOF$(-1e zlM`cLUwQczdHRPxv~FBaJ^7?O_4JR7fxWW;*wgciU@!HG%~kBQ-GxpRkd2qrs$K2m zAskD>a(8guarOn}&K~b?HEN0j8tNVMnF%0$MaQlxQIUYQl7Pg=1@mA+mnsFl#{wzUPM7dlsH z@Z!v1b_}y9S>#4cwjMKL;z?>AJ>;6^wD)L^16jz0@W}~SOMvzDgM-AtD*15CU=t%e zaX8Okw_4(N&Wu9x+3khl_ms|=GlrdD*1KwL`g{nC^?-ZWq9Vmu-QkZWJGFu+sUh8< z6UcmpUP?vhmU|c@D~I!7b-^*-Yx_sE?WG%su6DKCK1*m8c8K&`1(U0lNj~<|RUY~5 zMjrXxvHZf%ZRP6rMB4tAwa_iy$GhBaI(QcE%JEM&cM%J#$6W*9<|zT# zwCl>G_-n}868xoXAc|A` zI^oQ17a^VsX@ZyJQ3j`Le8B5M;@lQ`J2q7aA(pt2QpR~GHP}u~u-RSgz!u~)-%rU5 zS#sgz;7&23eRT^Vn3x%okmZ~+Bp_NA?#hwQU9B5C00??1t?kw;Rw!U)m$GP-F!~(K z_Lu}CvFOY>Vn{i%o{dqMgg~vtIxN99`35~lLW6_0ChU?+U0<90nqY&)z&dSy^Soz+ zZF5RO8g#NPD52&pc4rm}*WZ(M=MFv{fY`hjW}k=tHeOe9o;C?hY|dK=&20kZoCgg~ z41ZaIxx%)HoxHD+Oy-HnAt5uE-F{W-t?4Z297mMq_MJO&^ZIpp;kg%_ef{ne^4&lC zjwN)x@cfH%^VTi7(7Cps0EUa~o689Z6zqCR&p5*2FotLh0L$)%boTU^8k%>wSF+q{ zT_Khp(_2%3gN@du1n3MWUWG#EDj8vR(X*kS2Z_dDwWZ*k%6LhNugj zD6*3=zFBr~V(Gg`{L1Ujn&iINDKBysH34!=4*(m6s(Fq%G|;rSm|p^9XYVf^BTV*q zju5v;lC+IE?C5BTbL)OK!j^5=qI>XW7X#{> zj{vdMmsYBrUk~^3?D?Ne2>INdwWzcd0vXKt$tpHm%xb}&_QfEV% z<^Frx0${Jn7azS>e*WV{KJvl6Y>Kv+-A6gWteByDAsGL655($l-Bm2C9(NsxH8%De z<n8TSI#=d?F2BaNbuT>+i;3z?W(jFIB0{FWZPUhl55A;gxRuep*W~x zF{bBl_ailW%)1Ps1Upd3x`Dwk#x*J!Ff2U#=WwNf{oU z`okvmtRMku#lHq$@p}t@&-@Mv>e$$9P6A9`d<5zAd+I-IV8l{G2@h?6dOh^7W%9r#|6UYltk-BYqsC&=dlCX1cNIs}Q@+||n> zHh?oG7Zr~%sI*C=4yMr`YIFvaU^+P^FRA@$QMM!wFxz1>U!NsUX_VwJ#h0%?A6GhX z_P2qcxWPHima-nHc6-k@vX&I8^8wlGc1>N<<gvhE3tMfWF-3e zCYc%r$Ow>6FnR?TI?$Xy1jPzs2RnZafT-hQ`xzq&8f|6tM|>@cWCa16oA~m-6p2Pg zLOrtYBV}l@o(t}ScFb+%n` z`PyM^vt)VR{8+zcd3ZBJSu;QjCaZd4(_$ocA{Q4Tr)N#hE|v*hkEM>+3>4UAGFE`H z$wm3-b8UYY$qauheY6kO(hW*x?9!p*;$ik8k7;)1X+jAl->CF2&9pP@1x~TI{UreG zdNvX$=waGPU8q99)Pq^qVD2LU!#6PN$77+W4UF)J%^9L|-5|CbFpSX6wTAmD2vzOv zblh+>DhQxFLFJg@U?B~jm6;=B)jIgU}G6-i4JIdCJ}}Y;?4n=0W_Gt8OF+k=pN3? zN5{}OvU1(}I7k@1J>o_IsRdH(G>X6c&-!b#dqBO}V*zz5LS4 z^8EA9$q%0Rp8W91C*{XKdPZJ&{#kkX9a8biry6O$! z4|t0tm&A3j4Uruqgb+z-piDI}Sfdz|IGLfVavT-_`{5$L`0-gD{rr9MnTPI?d+xcp zyyvEk(PZO#Z<>kb~2FOvzSfC%D>d6UXA4x1xmU~k`<7HAd*TXt$IW22lC#EJ+?ROR`Rz{Nxh59^Zp0i|%8Pl#pK_()Jowx6iU0|$wJ!W6; z)$avbKq(+9vVJNPlq{sz+NEDiNwhF5=OTD(e=)RuRCGxdDTh&tin;iJj9b7@{F+Sq^Y+ zMoJonA>=3(NdUN6Oa3ZtM&gT)u{sLWFzXcCsAg=t!TAP(yvo*rwO>T$E<}Z|D-X>JWr$WrB@<_GIPoF67P~O;Eiquf6iBJpbHt^5ds}Bu_s119|$H zA1qohp8mdK$(~9A z%n_0kHkO*7Spuu_G{T6K0hR92Swa|<9rM7InA+bwaGQ0d|5J5y|fYaT# z0%uj2loDhlC+u@s+uoRut#$S@=Lj7_iE1VOe5E59dL3>!NK#`yhn74_ec`j`UQ*?! zWu`zM0i+y+(LyZ^xrdWKO(|o5FYWeYsNtZKWmW)78CPR*Xvyad9F}9!Em&T1E2V#z z#JcqX=;mKB*$TD#mSl^Z1O0w$B^BxnWr!bHNPo90a&p?_ z{A}@t_FH?UvsPXmKog~Ags%1MTDrwp$7!-k2IpGMVBef2C?&Gq2Nc|9xVlfYifmZ0 zNi)RzhmwFpVA>l?-qH%V4r?9fih(`&B8U!9oMvS8S^PX23B50}sF_s5#f$LQglNJN&7lQ*3>tRCMz|J%{1F=K5EG}z$`w=ef z#wa#W{3(;3FhT`7QM_K_v;4EiGN778rym@>d(alFs?H1jURw+6X6q$^Mi}8uB(}|9 z1axlHLI!5Cje5Jq>$l$Cwt#t8qkg2Hq1T|ylp}OAoLX@q`-hJ)mL?@2b5t?^YSyVu zC#tLrJC8mICW?Y1A&j9~YOkr$Yt-b;E&eWGZ5p3v1N%GAsck(l&V3iVgRAoA={r)) zi+CTN<+>gym>8}PnKM$sv$k%I%`?iQWsBDtjvPOwb&M8mQe^^T$5O_6CfmlfjXKqv zjVGR-U!2S7>8VKsUV7<8dFDqyk{>?xq&)GxC*+x@pOGK_@M(GR#TVrE?K{?~%c_*9 z2DVn=R@W>+6Pqj;c2tGyuEPPT>nk_maAg8r$!Jj66oCWede~NZ-}yB_6J8;;z?Lhl zZ%lSiaB53T9NocVSGE9m3P7fOo+kIPwY8BeSGHbJ21%D*X(B7a&|d6Oce{1qkco-7L2Uo!PV100rwK|qHXVTFeI7PS$aT}R#mT_>+$%8$?V0H~j#;tA&yB>tiIF## zlt`D@N9TO3Lp0jqsCu;qM*cX`-GuENlNEsF%YEQh4hsKH=f!@uXV2&s_CCxzi+~Mo z??TUC-EIvvE464zwav)rVT@nU{Mn^1 z#x6X?np%H#WQQ5GVD&wtCqyb4Ca!h;@P|kFl`nlzKL6=!^5Dmc9A7TutK*MH}ndqIt8)zgDK6jRJ4RV9^iN#S4Z!)-`o~2y@U1D9b0|)2n zjVTk;q_{qk9&&tK&8+%WwzPOPpEC9JYX`tkxv6FDF&t`>-K>eKximJWF_mzvNMp4Ia}iv=mH0P zU$eqOMd&JOT}D+3dhE%%mcnqe#Q$kTIcC(Q=-g97eFX5TUoHhfEne*zzX0Sdo{2{e z02Gv^wb#HQ=|aV-M#baA2LS`=iUCo^sTR5e{}(|FPMbjx#ypiA1`3` z`DdS(>#w~krzd9-*=QI$oqtw>OV4GjY;#RGfp&lEg$|>Gnhlzn7%&3ULrD{vcDr}q z%^eWi4f$efrDGeZQEU=x!{r=h)v$vIq5{M+iIQ@w2FC|W*a?Q^HOD&lH(P5rI}EA! zo-rT#1`PV`o>~5w2Q%hh=qZaQ++-F2pAY6PKoRYW0Q~DNJHrOYToKkRo&z|g&F|(e z*3Cq%?v(Xy$;xK9JKt}_!RT@@i?G;9DckD7Z-PvJ<^YbRcM5yC1s$9a26QXrEgq!= z!EAydZGAr)dRVIqvxkBp*s~ z=bDlMo93*d(eJITVvpq7X!g6>sI6aqUnc{ydOnZTdFN0~ks%Z z_PA?7tR9bjz5Tz-Fs|6xcX9-L#m4I!-2h)sawHAdcxdJYDq&urHt6vG-8J=gTP?uX9in}&(Lgr1y1@p2c^5vs4_u_iE8w+`59yy4m}oX zMJeSV(C5uJSu<<>bLX{d20hu_0ZDpZnP4BuWd#fXwV@m@DKzY5Wi%tn0L>dH8DY4W zQjlKu!LCaD9I_yVmjle{Q3Ah2aaiw(ldltTEG*LVoF)lV!Nhc=f8-K~oH$1S3o!Q{>k z^{B@5H$f?)&dI|Ex#1S>vi@%e1bJOKwvZX{eArlW^X9Y5Bn>j7-Qz67jQDh}?J9dMX_q`i@UPGH{Onv#PVUI{8?PE@efGHpwEpmKdcc)MAB9!P2=o1ggv z*<_}#cEXY4^YkAt2Wy-c&e|tPD7f!%e|WteXzgt|0}L+N=3c?dGcDGIMeE?q)2`V2 zasLkVJmel3>m{9`Br93~*^VxC8v`dHs^rY=5)jLk7w?q(EARVYtV)qXdrgc8$Q56) z{_qSk$u4a^pUr@I>d~&6nr&W3ODT@qlYHhocD?Mzl=+E8x;?KFFiYeW6g>Aj^u&>~ zpqff9tV7xjR`{J??H0`)zscTVJm~y?pyi zf4(!38t0G6>RBOs^b11Z{#5@mx8dJESae= znw8Rx(UVRP3S)L>Iv=@l`q;Gy%@=t0)S9``Ef&7>M6|g}ED(hLOP2O3Mc7w-C@WztKOva!a2eoDa=D1l1 zo-r6!R4`y&Ohoxs8zr@~+*})W+9G7G*mgBp92?h}O$l0@kLcJc8~wH7|)|<&<`8GwnLq za00PWYXV<{q%Da`P;$=aOI?7hZMNc7U-aJzWuFM=J!{;>~GD7W%4lZHx z3+G@%T-niN0x<`%tUc>Vo$UZLCzK-tt-n^*+{btylA>f{0fJY1hS!5uTjaFx>`|u) zvki05pIP^8;tEP!&kO>g>(_vB_UnXFd^HR`uR1f+QV7)Y?{t=E;nPaS6tJ_~8ZKlt zUD8up&4MQYj0KV{mz%(W6v_rMHobGvIWMk-p2Jp=TquY&B)?(Y0=mHi9GQBB9DCjNIvuMkv#m1M{?iQ>&t!JSuSDn zy0A7-)tFnqIR~*USLaDx_23D;)uYfh{PFOp{7>)T1b<=2T@7OOchirx2ac>Qt!h?OZ>8E3OXRY; zs$Q|c4a3f}K9|2Su`4nx#f_W-IyNHURR~x*UIW6u zU<|mkA7Yk)rqWFP_o0+V}{;~VGm z)V=zTf8Ogmuz3nXx1#eYCVeM^XnFpq zB^hfsfzL2x=(6yCnA7$z=j`N_Ff6p$CX+B}*{svk6S;Nk*79{jUVH6TdFADo<;4ZC zYU!*OUwC2p`9+fssI$**0I@NTHLks4epyMk zjhA)PP;In=mIMN^>63w`PF?L*@zQz8?y{1DkITrp7pz^%g_I{ItThlEg zSTU#Z85PL6VH`Big3N7#SmAwYnq?Y6U|-mJQ}%3jZu57^OKTq8|a!F898{Rjq^o*IN` zJxCS+^^;RgC^-wAt6%>fVV)A;J`%040cvxB6r_WR+M{^7CNn|M-+z&HV^xD_0m#HH zTsPSnJ_mwWjuR8m`i(#;695Nv820%n=P`wu5vNZtMczKTCv3 zXPmF!guLp-2Bl^OB#+Q&CP3R+vY{pj3^s3%kFOYms-RO~q!Fw~c^FouH~{GbJ9~lQ z8b$!vjWKBkiaBn|^y2qI0FMmxILO8$+uJ3t4&F6;dO|UGujlK}mPt1R8qweY1BN7U ziG~SeN}~XK3q5hrIBg1+H^!~gc1nyTs$ldW80QHvEB-QSrX%!;zsGuWPb9#SMJvo6 zOy0CYOUNnnssQ%RsR7s<*Kf+r>#vzq;Kk>kH}Lw>3kqIek()QKFR!_6&fYwkeD#69_{l%_zEk=daJ)!NcsQsMvP>0bI5DUr@6{{C507wNz^Zn<7P29Wm z0Yp`NMk&d}l_BO>DJd7pO;_M7%<;W6GO-zPi1h)M%UJZ9iC-I50daI=45cZpVMleI z?eGjz?m|GDb{5vq?q?iPhK*ma4FX%n4uw)Exk6PBtoy7q!D@i&_fxmM$Oi`N70lQdJJn0CKHdXyaq z?R?nM{l0et&~&-GhWrJ$RuhF$z6N%auVHkI^U3tHtmTSmj5j~ngxxjA2=9+ZIMt&j zh7BkynV3VT4 zn4g{%xpjM5HWQtw%0L|1=HUA-3ezh*s9v|w_G~yttjXE4uzK`zf>qtWT7{;gXFYrN zvX>7W0$F1W7~1HV)3^qPoM}a{_)NPxO=|`hqlx_AG|Idt{tYW-4b)}>P3Pxl3xGYf z4WXAp*w_qS3MyOwO>potX=`W|=Zd8khy& z%Pb~kr7lLsbzR=yyWDETJF5V^V&gK7(!4L&VEa_bMGlHWrXB|gg7fWev zP^;IpeguQnpO4{vW%zc^>|r1Gkg(p%c9V$+m7y{TJt$cRpU1vvD4U=*mRjJxO|Wku z%x>;WgpJbIq2;shOUCsP1cF#PXpxx>a5FQ^KhMzYaq!)8Bktkrq4}2XK}Uo0^l4(2 zM@?p|jRH`c%x(VGS0$FCWvBvxqp+S0F!aj@epLi!#RT!*4N55 z8(vF%ACZ`F`da|WUV9y2TciWY0VTndg88O{H%mAI0>>efGJQT~IYNByOj!rm_=e;} zvcX^x;a20Z^B%HN2uXro6eH3BR;~$2$-vn~r^td-n21fH!p5@8=urS`WeqnQj42ah z^}C&D(fSX*zsaY6p~{yYxlca#@ICUt`_Gp7eB1ucQ)zTLx$9;CmU`I31ftU2>s$Kr zH{D!r``>%)Pws9OR*&}th*diK_r7Ki=QlY22x7yFmp6JQ`ba&iR0{#GL2MDX!oK)b z5@7WZi|ZyYrYd(%_sfQ&$wX=-uGqP(&S*nHVbjQ>ce73W$tL?)kttp>DS(rkwFr%& z1i-Mmcirn&;d|+g>tPw7G!L!&j3~ENlc`&Q;%?7WX8M*`W)Imf>@s4%Oh!f;4P9MGF;}q3OkT#k$Qlb{<8zLZX%(p zNG4aX$v~x6cr!v1KCb4F0SuhGn`y$bMxFqJEi%{+0u8TR5TJ3+`@_IVLzX{5ZGhIO znm_ek!=NEONFLIR z5obli!G^OZ0CW|tDRds0v*c^6XszjC6c)JmCxB-{X8~(1y1l_cVpW0$0A!epk`wVc zONmz`)O0K?t3o7W%O()wb405R%!~5J_MY&h*TQ<<~oPOl-WjPzJjtbUTzv2N16>eR3f8V?QUI z{(@#n@RX#WF~SC=CiSq+m*zkzf9$1;9RZRX+Z)D#zpPwSdP-(L46@(1-%`KOn^>+U+<6ChTPKm1SSPv>*_JIesy zqd)iT-cLH*3!Ab|V+YbaQOiAMhWm=PhSQu~ExEO8a_9C=&d-+L?6fZG=mr)^Th@Md z+0YxjV6Ad6Tj!54sQey~22jP9o=aJ)Zp#ExHke^fqEB~z%p=Vi7R7d=l7{R#(+{fAT@xN zf$3pPV+i25+wWrxJUQ#~>}T5~C)Y%J>7;!msWi52gt2CQPS%fQ5}AR&hE15x(H?!& znFxn4y4IAnXL99=mE9_vAsDwxaKTCpwM2QIo;4nZULQLc*hx8wU)RW5j-8IyH->D$ z5hZ}4kq74*k~o-<5FmdJ7O5c{e(IVhP&aFuQYK2F%N3oWtOd!v4G=Rn^UKy*4_w7J zL{KvJ`qBdIHh>{AUoLOixmJFsAXUH4EmCoMcCvucGhkD|jv4Gd*cPDoYyk1z>x(k2SyU8@LJ20>Mr2tl$*!|o9#Jq~%&e%$XxCJsN zA&^`C)x<9LcDIgnTL-ZNr39&`4_#z&BA`Yx8`)hXmeztKvVw#dXtAMDH8z%=k2(gN zfGHJ*Eo;bvDWz`O^st)HRH&KF90W3Q2FStOImW@gN4r)rB&H+@bLeohI#QBs&m!6G`^9 z6p5Czee<2_HEf&Yey5TqWm&i5EFXNo0@x$@)vta?KJlQ)`|mlIZ8;MWl&WluA{FsO zQs~}0Y!gXx9f*CS;|>1yRJ|d8`>}r{`%C|NmmTjJ5c}AFCHvq1n%rKV&fnCZ-?Glb z9yn_;wpeV1ZRqSLhT{D-bu`(#j7Q$J>+Ga08^^E!*>Ty7MhABT!~dh{X#ImG%Mf(k z;uzqk2Z24X)OV{_tv_E`K!yID`cdBFWz6e3Jii98!N7K9y}V4+8w?KtYj-eX-auaW z#-6pDQO-s#VTj$s^^lnw8OshXB??M=oykpX`V#MVkytebi1m;rv;QHxn5PKAyUx$Q1OQq=gKJj@xq9tbwu^J&=W;eX z!tA9cpsZ|Ek($^;Cn*c|*Jy7WAQq0sh%hXBl_vh@UZkO|KguHUTWPmAl=!t?LUUbZUg+?EKvH;wKA$ zJz0ROf>`}tf2WCDcb4x@m)|LfRp!+|Zm_SrlqBt@kRh0qVDIxypQcI(y!NTi4>CzU zmz5>6PF?y10P7lAv2s`gYFm37<4uHB%Sz`w*lW$nQqa?oV}N5Ou&E^~9tdi+QhK7N z8DM*(+nK$oM3~wBJoge1aVFN7NvuPFE>G$qd)^9i1x9>nyltk%*~mAZVkd{JElD&cyuAUL^B|lOycV7 zyj@gAgech|3+ZI9UufgFS5H2&Z2EYwpn^&>Aj^+v-J{D7TptDCb`4-@he^nge_&Fn z|C=2R>q@@l|BmYjA;}+UA)m;31!pPN=RP#Ym3a*yX*e#eyO$XSUV|N{T|%^wk31@q z6rWdQg0(}aR`{6Flt8=HEe~}8uzPvmeMLU;agks9>IdYDkK8X0+H(T-E+KWK&&2*{SEoQ+5__MEZ=^F zzx&BxwuKYy9^-lmStzSb5)N9mtUNHW`X+4ycXcrw*W~ueMs^o8(8C&+uhjDhWetcG2Bo#SOdHw;l{ zg@8q|vAMzI>&?)ih~C^)NpNP8%?^AKDfrRNLl54E^}wzpHVWuVwa_j~nb1RJrx^o` zwMP9vD-X4GhDbZPu-ChL)E<5(E30p&e#-B;CgUuU#x|N3l+XqSyCFT(ocF3PC1}$8 zBL}3p;dCzlfy0sa#*j4u4>#ui8E1Xgj%nbzbki-KJY&y`Mc*~iNw6igOjI%kWBJ4J z)hmnRbhH53t!1Q-G`i5ma;&_R7qfFq$C_qGA38y7N374NKrTQj3!Mjs=;D2~tQZhh zGU&C8{k^kSxwMXP!gC=5yN6ym=4QwfrqsWgrdC-?yib$HCS-h1=v$7c!bIka=bD`X z-gkO!mkO}y$%Q3-Dp)nv^};o;1h4k5fz?y{S1WiZGpk?5)-L~Eob&e!0M=Hqb`7uv zrVVfw-?G&AEgh$wgJqKu>?VK}*FWme&Y1MJt{WUhGB`T#@qDzU^wv7aZH?)&2n9f_ zwK5&XaB$f8B#O;r18rEhv5v@8GHH;kY3Ue6ail|CBgH79lqQz!O}(Kz(8z` zPGH7WA(P#s4@agUkIsQ~TolNb1`TKhyYFnaf&QT+)2C6lur16`uNH|jW;B>M*qG+P z6-0EB5W0V0-Froj)V~wNhIFH^qyVRv4Lk~In4Sr3>5L^Pl&G3wgaWnXl3|94m>l^zz zRwz^2gZa&^yAa)tpZ+%0aTL$_!5C8UNV0Hyo=s^p0 zR{FaZuj7LE?j4?#G?e3Ia~S3Jt^Km;D049&dz?XmtaGZ9(>9lh1?^#YD>ao~jz~8N zlQ2xPfdOS=)xfYERc6X2n>1kq#SIHf)#L`!0aS?GhYkSIhzMR+%))kE=GGIp7;Z(UE`EN5ub1naVTiqFyPrE4 z6|`6l#+pnRjg~#y7SCMoqn9g$;fJ*+a4x|t`uLC$j(U{T=pw9U25Kgt&2iy;a;;UB zKI5N2+BT_)*UY-ODfd`8Dw6j~0NbExY+!zjiJKkBHerp>$-5e9B@L@}Tjc7stIM1_ zmd(-D^{K@Hst}lP-VDix)S+_*C<_>vNKxUyjKO9{xiO4mG)@SoPP3_zRgBGw2{;uj zdL6RcS;3qyST~{Pn^USW^&s|unVr}yLrdW8Q@&w|QK~KN^sjxnxz~T~qa>Es-}J